From 7051722023b98f1720142c7b3b41948d275ea455 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 1 May 2016 19:05:20 -0700 Subject: [PATCH 0001/1470] [SPARK-13425][SQL] Documentation for CSV datasource options ## What changes were proposed in this pull request? This PR adds the explanation and documentation for CSV options for reading and writing. ## How was this patch tested? Style tests with `./dev/run_tests` for documentation style. Author: hyukjinkwon Author: Hyukjin Kwon Closes #12817 from HyukjinKwon/SPARK-13425. (cherry picked from commit a832cef11233c6357c7ba7ede387b432e6b0ed71) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 52 +++++++++++++++++++ .../apache/spark/sql/DataFrameReader.scala | 47 +++++++++++++++-- .../apache/spark/sql/DataFrameWriter.scala | 8 +++ 3 files changed, 103 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index ed9e716ab78e3..cc5e93dcadf4d 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -282,6 +282,45 @@ def csv(self, paths): :param paths: string, or list of strings, for input path(s). + You can set the following CSV-specific options to deal with CSV files: + * ``sep`` (default ``,``): sets the single character as a separator \ + for each field and value. + * ``charset`` (default ``UTF-8``): decodes the CSV files by the given \ + encoding type. + * ``quote`` (default ``"``): sets the single character used for escaping \ + quoted values where the separator can be part of the value. + * ``escape`` (default ``\``): sets the single character used for escaping quotes \ + inside an already quoted value. + * ``comment`` (default empty string): sets the single character used for skipping \ + lines beginning with this character. By default, it is disabled. + * ``header`` (default ``false``): uses the first line as names of columns. + * ``ignoreLeadingWhiteSpace`` (default ``false``): defines whether or not leading \ + whitespaces from values being read should be skipped. + * ``ignoreTrailingWhiteSpace`` (default ``false``): defines whether or not trailing \ + whitespaces from values being read should be skipped. + * ``nullValue`` (default empty string): sets the string representation of a null value. + * ``nanValue`` (default ``NaN``): sets the string representation of a non-number \ + value. + * ``positiveInf`` (default ``Inf``): sets the string representation of a positive \ + infinity value. + * ``negativeInf`` (default ``-Inf``): sets the string representation of a negative \ + infinity value. + * ``dateFormat`` (default ``None``): sets the string that indicates a date format. \ + Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This \ + applies to both date type and timestamp type. By default, it is None which means \ + trying to parse times and date by ``java.sql.Timestamp.valueOf()`` and \ + ``java.sql.Date.valueOf()``. + * ``maxColumns`` (default ``20480``): defines a hard limit of how many columns \ + a record can have. + * ``maxCharsPerColumn`` (default ``1000000``): defines the maximum number of \ + characters allowed for any given value being read. + * ``mode`` (default ``PERMISSIVE``): allows a mode for dealing with corrupt records \ + during parsing. + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. \ + When a schema is set by user, it sets ``null`` for extra fields. + * ``DROPMALFORMED`` : ignores the whole corrupted records. + * ``FAILFAST`` : throws an exception when it meets corrupted records. + >>> df = sqlContext.read.csv('python/test_support/sql/ages.csv') >>> df.dtypes [('C0', 'string'), ('C1', 'string')] @@ -663,6 +702,19 @@ def csv(self, path, mode=None, compression=None): known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). + You can set the following CSV-specific options to deal with CSV files: + * ``sep`` (default ``,``): sets the single character as a separator \ + for each field and value. + * ``quote`` (default ``"``): sets the single character used for escaping \ + quoted values where the separator can be part of the value. + * ``escape`` (default ``\``): sets the single character used for escaping quotes \ + inside an already quoted value. + * ``header`` (default ``false``): writes the names of columns as the first line. + * ``nullValue`` (default empty string): sets the string representation of a null value. + * ``compression``: compression codec to use when saving to file. This can be one of \ + the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and \ + deflate). + >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 3d43f2022f669..2d4a68f3c3a94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -290,7 +290,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers * (e.g. 00012)
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
  • + * during parsing.
  • *
      *
    • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the * malformed string into a new field configured by `columnNameOfCorruptRecord`. When @@ -300,7 +300,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    *
  • `columnNameOfCorruptRecord` (default `_corrupt_record`): allows renaming the new field * having malformed string created by `PERMISSIVE` mode. This overrides - * `spark.sql.columnNameOfCorruptRecord`.
  • + * `spark.sql.columnNameOfCorruptRecord`.
  • * * @since 1.4.0 */ @@ -326,7 +326,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
  • + * during parsing.
  • *
      *
    • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the * malformed string into a new field configured by `columnNameOfCorruptRecord`. When @@ -336,7 +336,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    *
  • `columnNameOfCorruptRecord` (default `_corrupt_record`): allows renaming the new field * having malformed string created by `PERMISSIVE` mode. This overrides - * `spark.sql.columnNameOfCorruptRecord`.
  • + * `spark.sql.columnNameOfCorruptRecord`.
  • * * @since 1.6.0 */ @@ -393,6 +393,45 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * This function goes through the input once to determine the input schema. To avoid going * through the entire data once, specify the schema explicitly using [[schema]]. * + * You can set the following CSV-specific options to deal with CSV files: + *
  • `sep` (default `,`): sets the single character as a separator for each + * field and value.
  • + *
  • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding + * type.
  • + *
  • `quote` (default `"`): sets the single character used for escaping quoted values where + * the separator can be part of the value.
  • + *
  • `escape` (default `\`): sets the single character used for escaping quotes inside + * an already quoted value.
  • + *
  • `comment` (default empty string): sets the single character used for skipping lines + * beginning with this character. By default, it is disabled.
  • + *
  • `header` (default `false`): uses the first line as names of columns.
  • + *
  • `ignoreLeadingWhiteSpace` (default `false`): defines whether or not leading whitespaces + * from values being read should be skipped.
  • + *
  • `ignoreTrailingWhiteSpace` (default `fDataFraalse`): defines whether or not trailing + * whitespaces from values being read should be skipped.
  • + *
  • `nullValue` (default empty string): sets the string representation of a null value.
  • + *
  • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
  • + *
  • `positiveInf` (default `Inf`): sets the string representation of a positive infinity + * value.
  • + *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity + * value.
  • + *
  • `dateFormat` (default `null`): sets the string that indicates a date format. Custom date + * formats follow the formats at `java.text.SimpleDateFormat`. This applies to both date type + * and timestamp type. By default, it is `null` which means trying to parse times and date by + * `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()`.
  • + *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns + * a record can have.
  • + *
  • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed + * for any given value being read.
  • + *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records + * during parsing.
  • + *
      + *
    • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + * a schema is set by user, it sets `null` for extra fields.
    • + *
    • `DROPMALFORMED` : ignores the whole corrupted records.
    • + *
    • `FAILFAST` : throws an exception when it meets corrupted records.
    • + *
    + * * @since 2.0.0 */ @scala.annotation.varargs diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 28f5ccd26bc52..a57d47d28ceb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -606,6 +606,14 @@ final class DataFrameWriter private[sql](df: DataFrame) { * }}} * * You can set the following CSV-specific option(s) for writing CSV files: + *
  • `sep` (default `,`): sets the single character as a separator for each + * field and value.
  • + *
  • `quote` (default `"`): sets the single character used for escaping quoted values where + * the separator can be part of the value.
  • + *
  • `escape` (default `\`): sets the single character used for escaping quotes inside + * an already quoted value.
  • + *
  • `header` (default `false`): writes the names of columns as the first line.
  • + *
  • `nullValue` (default empty string): sets the string representation of a null value.
  • *
  • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
  • From 7d63c36e1efe8baec96cdc16a997249728e204fd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 1 May 2016 20:21:02 -0700 Subject: [PATCH 0002/1470] [SPARK-15049] Rename NewAccumulator to AccumulatorV2 ## What changes were proposed in this pull request? NewAccumulator isn't the best name if we ever come up with v3 of the API. ## How was this patch tested? Updated tests to reflect the change. Author: Reynold Xin Closes #12827 from rxin/SPARK-15049. (cherry picked from commit 44da8d8eabeccc12bfed0d43b37d54e0da845c66) Signed-off-by: Reynold Xin --- ...wAccumulator.scala => AccumulatorV2.scala} | 69 ++++++++++--------- .../org/apache/spark/ContextCleaner.scala | 2 +- .../org/apache/spark/HeartbeatReceiver.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../org/apache/spark/TaskContextImpl.scala | 2 +- .../org/apache/spark/TaskEndReason.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 4 +- .../apache/spark/executor/TaskMetrics.scala | 18 ++--- .../apache/spark/scheduler/DAGScheduler.scala | 10 +-- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../apache/spark/scheduler/TaskResult.scala | 8 +-- .../spark/scheduler/TaskScheduler.scala | 4 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 2 +- .../spark/InternalAccumulatorSuite.scala | 2 +- .../spark/executor/TaskMetricsSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 6 +- .../ExternalClusterManagerSuite.scala | 4 +- .../spark/scheduler/TaskSetManagerSuite.scala | 6 +- .../sql/execution/metric/SQLMetrics.scala | 6 +- 23 files changed, 85 insertions(+), 84 deletions(-) rename core/src/main/scala/org/apache/spark/{NewAccumulator.scala => AccumulatorV2.scala} (82%) diff --git a/core/src/main/scala/org/apache/spark/NewAccumulator.scala b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala similarity index 82% rename from core/src/main/scala/org/apache/spark/NewAccumulator.scala rename to core/src/main/scala/org/apache/spark/AccumulatorV2.scala index 1571e15b76ac2..c65108a55eba4 100644 --- a/core/src/main/scala/org/apache/spark/NewAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala @@ -21,9 +21,6 @@ import java.{lang => jl} import java.io.ObjectInputStream import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong -import javax.annotation.concurrent.GuardedBy - -import scala.collection.JavaConverters._ import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.util.Utils @@ -39,7 +36,7 @@ private[spark] case class AccumulatorMetadata( * The base class for accumulators, that can accumulate inputs of type `IN`, and produce output of * type `OUT`. */ -abstract class NewAccumulator[IN, OUT] extends Serializable { +abstract class AccumulatorV2[IN, OUT] extends Serializable { private[spark] var metadata: AccumulatorMetadata = _ private[this] var atDriverSide = true @@ -95,7 +92,7 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { } /** - * Creates an [[AccumulableInfo]] representation of this [[NewAccumulator]] with the provided + * Creates an [[AccumulableInfo]] representation of this [[AccumulatorV2]] with the provided * values. */ private[spark] def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { @@ -106,16 +103,16 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { final private[spark] def isAtDriverSide: Boolean = atDriverSide /** - * Tells if this accumulator is zero value or not. e.g. for a counter accumulator, 0 is zero + * Returns if this accumulator is zero value or not. e.g. for a counter accumulator, 0 is zero * value; for a list accumulator, Nil is zero value. */ - def isZero(): Boolean + def isZero: Boolean /** * Creates a new copy of this accumulator, which is zero value. i.e. call `isZero` on the copy * must return true. */ - def copyAndReset(): NewAccumulator[IN, OUT] + def copyAndReset(): AccumulatorV2[IN, OUT] /** * Takes the inputs and accumulates. e.g. it can be a simple `+=` for counter accumulator. @@ -126,7 +123,7 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { * Merges another same-type accumulator into this one and update its state, i.e. this should be * merge-in-place. */ - def merge(other: NewAccumulator[IN, OUT]): Unit + def merge(other: AccumulatorV2[IN, OUT]): Unit /** * Access this accumulator's current value; only allowed on driver. @@ -155,7 +152,7 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { "Accumulator must be registered before send to executor") } val copy = copyAndReset() - assert(copy.isZero(), "copyAndReset must return a zero value copy") + assert(copy.isZero, "copyAndReset must return a zero value copy") copy.metadata = metadata copy } else { @@ -191,6 +188,9 @@ abstract class NewAccumulator[IN, OUT] extends Serializable { } +/** + * An internal class used to track accumulators by Spark itself. + */ private[spark] object AccumulatorContext { /** @@ -199,20 +199,21 @@ private[spark] object AccumulatorContext { * once the RDDs and user-code that reference them are cleaned up. * TODO: Don't use a global map; these should be tied to a SparkContext (SPARK-13051). */ - private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[NewAccumulator[_, _]]] + private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[AccumulatorV2[_, _]]] private[this] val nextId = new AtomicLong(0L) /** - * Return a globally unique ID for a new [[Accumulator]]. + * Returns a globally unique ID for a new [[Accumulator]]. * Note: Once you copy the [[Accumulator]] the ID is no longer unique. */ def newId(): Long = nextId.getAndIncrement + /** Returns the number of accumulators registered. Used in testing. */ def numAccums: Int = originals.size /** - * Register an [[Accumulator]] created on the driver such that it can be used on the executors. + * Registers an [[Accumulator]] created on the driver such that it can be used on the executors. * * All accumulators registered here can later be used as a container for accumulating partial * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. @@ -222,21 +223,21 @@ private[spark] object AccumulatorContext { * If an [[Accumulator]] with the same ID was already registered, this does nothing instead * of overwriting it. We will never register same accumulator twice, this is just a sanity check. */ - def register(a: NewAccumulator[_, _]): Unit = { - originals.putIfAbsent(a.id, new jl.ref.WeakReference[NewAccumulator[_, _]](a)) + def register(a: AccumulatorV2[_, _]): Unit = { + originals.putIfAbsent(a.id, new jl.ref.WeakReference[AccumulatorV2[_, _]](a)) } /** - * Unregister the [[Accumulator]] with the given ID, if any. + * Unregisters the [[Accumulator]] with the given ID, if any. */ def remove(id: Long): Unit = { originals.remove(id) } /** - * Return the [[Accumulator]] registered with the given ID, if any. + * Returns the [[Accumulator]] registered with the given ID, if any. */ - def get(id: Long): Option[NewAccumulator[_, _]] = { + def get(id: Long): Option[AccumulatorV2[_, _]] = { Option(originals.get(id)).map { ref => // Since we are storing weak references, we must check whether the underlying data is valid. val acc = ref.get @@ -248,7 +249,7 @@ private[spark] object AccumulatorContext { } /** - * Clear all registered [[Accumulator]]s. For testing only. + * Clears all registered [[Accumulator]]s. For testing only. */ def clear(): Unit = { originals.clear() @@ -256,10 +257,10 @@ private[spark] object AccumulatorContext { } -class LongAccumulator extends NewAccumulator[jl.Long, jl.Long] { +class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { private[this] var _sum = 0L - override def isZero(): Boolean = _sum == 0 + override def isZero: Boolean = _sum == 0 override def copyAndReset(): LongAccumulator = new LongAccumulator @@ -269,7 +270,7 @@ class LongAccumulator extends NewAccumulator[jl.Long, jl.Long] { def sum: Long = _sum - override def merge(other: NewAccumulator[jl.Long, jl.Long]): Unit = other match { + override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match { case o: LongAccumulator => _sum += o.sum case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") @@ -281,10 +282,10 @@ class LongAccumulator extends NewAccumulator[jl.Long, jl.Long] { } -class DoubleAccumulator extends NewAccumulator[jl.Double, jl.Double] { +class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[this] var _sum = 0.0 - override def isZero(): Boolean = _sum == 0.0 + override def isZero: Boolean = _sum == 0.0 override def copyAndReset(): DoubleAccumulator = new DoubleAccumulator @@ -294,7 +295,7 @@ class DoubleAccumulator extends NewAccumulator[jl.Double, jl.Double] { def sum: Double = _sum - override def merge(other: NewAccumulator[jl.Double, jl.Double]): Unit = other match { + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { case o: DoubleAccumulator => _sum += o.sum case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") @@ -306,11 +307,11 @@ class DoubleAccumulator extends NewAccumulator[jl.Double, jl.Double] { } -class AverageAccumulator extends NewAccumulator[jl.Double, jl.Double] { +class AverageAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[this] var _sum = 0.0 private[this] var _count = 0L - override def isZero(): Boolean = _sum == 0.0 && _count == 0 + override def isZero: Boolean = _sum == 0.0 && _count == 0 override def copyAndReset(): AverageAccumulator = new AverageAccumulator @@ -324,7 +325,7 @@ class AverageAccumulator extends NewAccumulator[jl.Double, jl.Double] { _count += 1 } - override def merge(other: NewAccumulator[jl.Double, jl.Double]): Unit = other match { + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { case o: AverageAccumulator => _sum += o.sum _count += o.count @@ -344,16 +345,16 @@ class AverageAccumulator extends NewAccumulator[jl.Double, jl.Double] { } -class ListAccumulator[T] extends NewAccumulator[T, java.util.List[T]] { +class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { private[this] val _list: java.util.List[T] = new java.util.ArrayList[T] - override def isZero(): Boolean = _list.isEmpty + override def isZero: Boolean = _list.isEmpty override def copyAndReset(): ListAccumulator[T] = new ListAccumulator override def add(v: T): Unit = _list.add(v) - override def merge(other: NewAccumulator[T, java.util.List[T]]): Unit = other match { + override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match { case o: ListAccumulator[T] => _list.addAll(o.localValue) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") @@ -370,10 +371,10 @@ class ListAccumulator[T] extends NewAccumulator[T, java.util.List[T]] { class LegacyAccumulatorWrapper[R, T]( initialValue: R, - param: org.apache.spark.AccumulableParam[R, T]) extends NewAccumulator[T, R] { + param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] { private[spark] var _value = initialValue // Current value on driver - override def isZero(): Boolean = _value == param.zero(initialValue) + override def isZero: Boolean = _value == param.zero(initialValue) override def copyAndReset(): LegacyAccumulatorWrapper[R, T] = { val acc = new LegacyAccumulatorWrapper(initialValue, param) @@ -383,7 +384,7 @@ class LegacyAccumulatorWrapper[R, T]( override def add(v: T): Unit = _value = param.addAccumulator(_value, v) - override def merge(other: NewAccumulator[T, R]): Unit = other match { + override def merge(other: AccumulatorV2[T, R]): Unit = other match { case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.localValue) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 63a00a84af3cd..a51338c017071 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -144,7 +144,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { registerForCleanup(rdd, CleanRDD(rdd.id)) } - def registerAccumulatorForCleanup(a: NewAccumulator[_, _]): Unit = { + def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { registerForCleanup(a, CleanAccum(a.id)) } diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 9eac05fdf9f3d..29018c75b9224 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} */ private[spark] case class Heartbeat( executorId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], // taskId -> accumulator updates + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates blockManagerId: BlockManagerId) /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2cb3ed0296a47..d0f88d4e4d109 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1282,7 +1282,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register the given accumulator. Note that accumulators must be registered before use, or it * will throw exception. */ - def register(acc: NewAccumulator[_, _]): Unit = { + def register(acc: AccumulatorV2[_, _]): Unit = { acc.register(this) } @@ -1290,7 +1290,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register the given accumulator with given name. Note that accumulators must be registered * before use, or it will throw exception. */ - def register(acc: NewAccumulator[_, _], name: String): Unit = { + def register(acc: AccumulatorV2[_, _], name: String): Unit = { acc.register(this, name = Some(name)) } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 9e53257462867..1a8f8cf11c1be 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -188,6 +188,6 @@ abstract class TaskContext extends Serializable { * Register an accumulator that belongs to this task. Accumulators must call this method when * deserializing in executors. */ - private[spark] def registerAccumulator(a: NewAccumulator[_, _]): Unit + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index bc3807f5db180..c904e083911cd 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -122,7 +122,7 @@ private[spark] class TaskContextImpl( override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) - private[spark] override def registerAccumulator(a: NewAccumulator[_, _]): Unit = { + private[spark] override def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { taskMetrics.registerAccumulator(a) } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 82ba2d0c274be..ef333e397f64e 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -118,7 +118,7 @@ case class ExceptionFailure( fullStackTrace: String, private val exceptionWrapper: Option[ThrowableSerializationWrapper], accumUpdates: Seq[AccumulableInfo] = Seq.empty, - private[spark] var accums: Seq[NewAccumulator[_, _]] = Nil) + private[spark] var accums: Seq[AccumulatorV2[_, _]] = Nil) extends TaskFailedReason { /** @@ -138,7 +138,7 @@ case class ExceptionFailure( this(e, accumUpdates, preserveCause = true) } - private[spark] def withAccums(accums: Seq[NewAccumulator[_, _]]): ExceptionFailure = { + private[spark] def withAccums(accums: Seq[AccumulatorV2[_, _]]): ExceptionFailure = { this.accums = accums this } 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 4d61f7e23248b..4f74dc92d7dfc 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -353,7 +353,7 @@ private[spark] class Executor( logError(s"Exception in $taskName (TID $taskId)", t) // Collect latest accumulator values to report back to the driver - val accums: Seq[NewAccumulator[_, _]] = + val accums: Seq[AccumulatorV2[_, _]] = if (task != null) { task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart) task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) @@ -478,7 +478,7 @@ private[spark] class Executor( /** Reports heartbeat and metrics for active tasks to the driver. */ private def reportHeartBeat(): Unit = { // list of (task id, accumUpdates) to send back to the driver - val accumUpdates = new ArrayBuffer[(Long, Seq[NewAccumulator[_, _]])]() + val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() for (taskRunner <- runningTasks.values().asScala) { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 0b64917219a7e..56d034fd033e6 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -201,7 +201,7 @@ class TaskMetrics private[spark] () extends Serializable { output.RECORDS_WRITTEN -> outputMetrics._recordsWritten ) ++ testAccum.map(TEST_ACCUM -> _) - @transient private[spark] lazy val internalAccums: Seq[NewAccumulator[_, _]] = + @transient private[spark] lazy val internalAccums: Seq[AccumulatorV2[_, _]] = nameToAccums.values.toIndexedSeq /* ========================== * @@ -217,13 +217,13 @@ class TaskMetrics private[spark] () extends Serializable { /** * External accumulators registered with this task. */ - @transient private lazy val externalAccums = new ArrayBuffer[NewAccumulator[_, _]] + @transient private lazy val externalAccums = new ArrayBuffer[AccumulatorV2[_, _]] - private[spark] def registerAccumulator(a: NewAccumulator[_, _]): Unit = { + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { externalAccums += a } - private[spark] def accumulators(): Seq[NewAccumulator[_, _]] = internalAccums ++ externalAccums + private[spark] def accumulators(): Seq[AccumulatorV2[_, _]] = internalAccums ++ externalAccums } @@ -271,15 +271,15 @@ private[spark] object TaskMetrics extends Logging { /** * Construct a [[TaskMetrics]] object from a list of accumulator updates, called on driver only. */ - def fromAccumulators(accums: Seq[NewAccumulator[_, _]]): TaskMetrics = { + def fromAccumulators(accums: Seq[AccumulatorV2[_, _]]): TaskMetrics = { val tm = new TaskMetrics val (internalAccums, externalAccums) = accums.partition(a => a.name.isDefined && tm.nameToAccums.contains(a.name.get)) internalAccums.foreach { acc => - val tmAcc = tm.nameToAccums(acc.name.get).asInstanceOf[NewAccumulator[Any, Any]] + val tmAcc = tm.nameToAccums(acc.name.get).asInstanceOf[AccumulatorV2[Any, Any]] tmAcc.metadata = acc.metadata - tmAcc.merge(acc.asInstanceOf[NewAccumulator[Any, Any]]) + tmAcc.merge(acc.asInstanceOf[AccumulatorV2[Any, Any]]) } tm.externalAccums ++= externalAccums @@ -289,7 +289,7 @@ private[spark] object TaskMetrics extends Logging { private[spark] class BlockStatusesAccumulator - extends NewAccumulator[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { + extends AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { private[this] var _seq = ArrayBuffer.empty[(BlockId, BlockStatus)] override def isZero(): Boolean = _seq.isEmpty @@ -298,7 +298,7 @@ private[spark] class BlockStatusesAccumulator override def add(v: (BlockId, BlockStatus)): Unit = _seq += v - override def merge(other: NewAccumulator[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) + override def merge(other: AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) : Unit = other match { case o: BlockStatusesAccumulator => _seq ++= o.localValue case _ => throw new UnsupportedOperationException( 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 a96d5f6fbf082..4dfd532e93624 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -209,7 +209,7 @@ class DAGScheduler( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo): Unit = { eventProcessLoop.post( CompletionEvent(task, reason, result, accumUpdates, taskInfo)) @@ -1091,14 +1091,14 @@ class DAGScheduler( event.accumUpdates.foreach { updates => val id = updates.id // Find the corresponding accumulator on the driver and update it - val acc: NewAccumulator[Any, Any] = AccumulatorContext.get(id) match { - case Some(accum) => accum.asInstanceOf[NewAccumulator[Any, Any]] + val acc: AccumulatorV2[Any, Any] = AccumulatorContext.get(id) match { + case Some(accum) => accum.asInstanceOf[AccumulatorV2[Any, Any]] case None => throw new SparkException(s"attempted to access non-existent accumulator $id") } - acc.merge(updates.asInstanceOf[NewAccumulator[Any, Any]]) + acc.merge(updates.asInstanceOf[AccumulatorV2[Any, Any]]) // To avoid UI cruft, ignore cases where value wasn't updated - if (acc.name.isDefined && !updates.isZero()) { + if (acc.name.isDefined && !updates.isZero) { stage.latestInfo.accumulables(id) = acc.toInfo(None, Some(acc.value)) event.taskInfo.accumulables += acc.toInfo(Some(updates.value), Some(acc.value)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index e57a2246d8729..0a2c2dc03937b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -71,7 +71,7 @@ private[scheduler] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index e7ca6efd84aee..362f8e51ce056 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -153,7 +153,7 @@ private[spark] abstract class Task[T]( * Collect the latest values of accumulators used in this task. If the task failed, * filter out the accumulators whose values should not be included on failures. */ - def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[NewAccumulator[_, _]] = { + def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[AccumulatorV2[_, _]] = { if (context != null) { context.taskMetrics.accumulators().filter { a => !taskFailed || a.countFailedValues } } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index b472c5511b738..69ce00f30d9a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{NewAccumulator, SparkEnv} +import org.apache.spark.{AccumulatorV2, SparkEnv} import org.apache.spark.storage.BlockId import org.apache.spark.util.Utils @@ -36,7 +36,7 @@ private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] class DirectTaskResult[T]( var valueBytes: ByteBuffer, - var accumUpdates: Seq[NewAccumulator[_, _]]) + var accumUpdates: Seq[AccumulatorV2[_, _]]) extends TaskResult[T] with Externalizable { private var valueObjectDeserialized = false @@ -61,9 +61,9 @@ private[spark] class DirectTaskResult[T]( if (numUpdates == 0) { accumUpdates = null } else { - val _accumUpdates = new ArrayBuffer[NewAccumulator[_, _]] + val _accumUpdates = new ArrayBuffer[AccumulatorV2[_, _]] for (i <- 0 until numUpdates) { - _accumUpdates += in.readObject.asInstanceOf[NewAccumulator[_, _]] + _accumUpdates += in.readObject.asInstanceOf[AccumulatorV2[_, _]] } accumUpdates = _accumUpdates } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 75a0c56311977..9881a1018cf57 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.NewAccumulator +import org.apache.spark.AccumulatorV2 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId @@ -67,7 +67,7 @@ private[spark] trait TaskScheduler { */ def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean /** 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 8fa4aa121c123..666b6365585c7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -384,7 +384,7 @@ private[spark] class TaskSchedulerImpl( */ override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { 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 b79f643a7481f..b724050f5bb89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -647,7 +647,7 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - var accumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty + var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + reason.asInstanceOf[TaskFailedReason].toErrorString val failureException: Option[Throwable] = reason match { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 9c90049715ddc..09eb9c1dbdc62 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -273,7 +273,7 @@ private[spark] object AccumulatorSuite { * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the * info as an accumulator update. */ - def makeInfo(a: NewAccumulator[_, _]): AccumulableInfo = a.toInfo(Some(a.localValue), None) + def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.localValue), None) /** * Run one or more Spark jobs and verify that in at least one job the peak execution memory diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 688eb6bde9041..25977a4660244 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -213,7 +213,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { private class SaveAccumContextCleaner(sc: SparkContext) extends ContextCleaner(sc) { private val accumsRegistered = new ArrayBuffer[Long] - override def registerAccumulatorForCleanup(a: NewAccumulator[_, _]): Unit = { + override def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { accumsRegistered += a.id super.registerAccumulatorForCleanup(a) } diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 94f6e1a3a77c1..27a1e7bb35bad 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -203,7 +203,7 @@ class TaskMetricsSuite extends SparkFunSuite { acc1.add(1) acc2.add(2) val newUpdates = tm.accumulators() - .map(a => (a.id, a.asInstanceOf[NewAccumulator[Any, Any]])).toMap + .map(a => (a.id, a.asInstanceOf[AccumulatorV2[Any, Any]])).toMap assert(newUpdates.contains(acc1.id)) assert(newUpdates.contains(acc2.id)) assert(newUpdates.contains(acc3.id)) @@ -230,8 +230,8 @@ private[spark] object TaskMetricsSuite extends Assertions { * Note: this does NOT check accumulator ID equality. */ def assertUpdatesEquals( - updates1: Seq[NewAccumulator[_, _]], - updates2: Seq[NewAccumulator[_, _]]): Unit = { + updates1: Seq[AccumulatorV2[_, _]], + updates2: Seq[AccumulatorV2[_, _]]): Unit = { assert(updates1.size === updates2.size) updates1.zip(updates2).foreach { case (acc1, acc2) => // do not assert ID equals here 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 9912d1f3bc5a7..5a5c3a0cd1935 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -112,7 +112,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def stop() = {} override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager @@ -483,7 +483,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def defaultParallelism(): Int = 2 override def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def applicationAttemptId(): Option[String] = None @@ -2012,7 +2012,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou task: Task[_], reason: TaskEndReason, result: Any, - extraAccumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty, + extraAccumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty, taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { val accumUpdates = reason match { case Success => task.metrics.accumulators() diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 16027d944fdfd..72ac848f12b46 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.apache.spark.{LocalSparkContext, NewAccumulator, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{AccumulatorV2, LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId @@ -67,6 +67,6 @@ private class DummyTaskScheduler extends TaskScheduler { override def applicationAttemptId(): Option[String] = None def executorHeartbeatReceived( execId: String, - accumUpdates: Array[(Long, Seq[NewAccumulator[_, _]])], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true } 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 339fc4254d53a..122a3ecb4954b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -37,7 +37,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Seq[NewAccumulator[_, _]], + accumUpdates: Seq[AccumulatorV2[_, _]], taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } @@ -184,7 +184,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val accumUpdatesByTask: Array[Seq[NewAccumulator[_, _]]] = taskSet.tasks.map { task => + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } @@ -791,7 +791,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg private def createTaskResult( id: Int, - accumUpdates: Seq[NewAccumulator[_, _]] = Seq.empty): DirectTaskResult[Int] = { + accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 7bf9225272612..40c00ee1881ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution.metric import java.text.NumberFormat -import org.apache.spark.{NewAccumulator, SparkContext} +import org.apache.spark.{AccumulatorV2, SparkContext} import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.util.Utils -class SQLMetric(val metricType: String, initValue: Long = 0L) extends NewAccumulator[Long, Long] { +class SQLMetric(val metricType: String, initValue: Long = 0L) extends AccumulatorV2[Long, Long] { // This is a workaround for SPARK-11013. // We may use -1 as initial value of the accumulator, if the accumulator is valid, we will // update it at the end of task and the value will be at least 0. Then we can filter out the -1 @@ -33,7 +33,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends NewAccumul override def copyAndReset(): SQLMetric = new SQLMetric(metricType, initValue) - override def merge(other: NewAccumulator[Long, Long]): Unit = other match { + override def merge(other: AccumulatorV2[Long, Long]): Unit = other match { case o: SQLMetric => _value += o.localValue case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") From ccb53a20e4c3bff02a17542cad13a1fe36d7a7ea Mon Sep 17 00:00:00 2001 From: Ben McCann Date: Sun, 1 May 2016 22:43:28 -0700 Subject: [PATCH 0003/1470] Fix reference to external metrics documentation Author: Ben McCann Closes #12833 from benmccann/patch-1. (cherry picked from commit 214d1be4fd4a34399b6a2adb2618784de459a48d) Signed-off-by: Reynold Xin --- docs/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index 9912cde743a45..88002ebdc39cb 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -341,7 +341,7 @@ keep the paths consistent in both modes. # Metrics Spark has a configurable metrics system based on the -[Coda Hale Metrics Library](http://metrics.codahale.com/). +[Dropwizard Metrics Library](http://metrics.dropwizard.io/). This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV files. The metrics system is configured via a configuration file that Spark expects to be present at `$SPARK_HOME/conf/metrics.properties`. A custom file location can be specified via the From 1145ea01b994faf458ab00301b8c4ac757d0dd0b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 2 May 2016 10:21:14 -0700 Subject: [PATCH 0004/1470] [SPARK-14637][SQL] object expressions cleanup ## What changes were proposed in this pull request? Simplify and clean up some object expressions: 1. simplify the logic to handle `propagateNull` 2. add `propagateNull` parameter to `Invoke` 3. simplify the unbox logic in `Invoke` 4. other minor cleanup TODO: simplify `MapObjects` ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #12399 from cloud-fan/object. (cherry picked from commit 0513c3ac93e0a25d6eedbafe6c0561e71c92880a) Signed-off-by: Michael Armbrust --- .../sql/catalyst/expressions/objects.scala | 218 ++++++++---------- 1 file changed, 100 insertions(+), 118 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala index 1e418540a2624..523eed825f262 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala @@ -64,33 +64,29 @@ case class StaticInvoke( val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") - if (propagateNull) { - val objNullCheck = if (ctx.defaultValue(dataType) == "null") { - s"${ev.isNull} = ${ev.value} == null;" - } else { - "" - } - - val argsNonNull = s"!(${argGen.map(_.isNull).mkString(" || ")})" - ev.copy(code = s""" - ${argGen.map(_.code).mkString("\n")} - - boolean ${ev.isNull} = !$argsNonNull; - $javaType ${ev.value} = ${ctx.defaultValue(dataType)}; + val callFunc = s"$objectName.$functionName($argString)" - if ($argsNonNull) { - ${ev.value} = $objectName.$functionName($argString); - $objNullCheck - } - """) + val setIsNull = if (propagateNull && arguments.nonEmpty) { + s"boolean ${ev.isNull} = ${argGen.map(_.isNull).mkString(" || ")};" } else { - ev.copy(code = s""" - ${argGen.map(_.code).mkString("\n")} + s"boolean ${ev.isNull} = false;" + } - $javaType ${ev.value} = $objectName.$functionName($argString); - final boolean ${ev.isNull} = ${ev.value} == null; - """) + // If the function can return null, we do an extra check to make sure our null bit is still set + // correctly. + val postNullCheck = if (ctx.defaultValue(dataType) == "null") { + s"${ev.isNull} = ${ev.value} == null;" + } else { + "" } + + val code = s""" + ${argGen.map(_.code).mkString("\n")} + $setIsNull + final $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : $callFunc; + $postNullCheck + """ + ev.copy(code = code) } } @@ -111,7 +107,8 @@ case class Invoke( targetObject: Expression, functionName: String, dataType: DataType, - arguments: Seq[Expression] = Nil) extends Expression with NonSQLExpression { + arguments: Seq[Expression] = Nil, + propagateNull: Boolean = true) extends Expression with NonSQLExpression { override def nullable: Boolean = true override def children: Seq[Expression] = targetObject +: arguments @@ -130,60 +127,53 @@ case class Invoke( case _ => None } - lazy val unboxer = (dataType, method.map(_.getReturnType.getName).getOrElse("")) match { - case (IntegerType, "java.lang.Object") => (s: String) => - s"((java.lang.Integer)$s).intValue()" - case (LongType, "java.lang.Object") => (s: String) => - s"((java.lang.Long)$s).longValue()" - case (FloatType, "java.lang.Object") => (s: String) => - s"((java.lang.Float)$s).floatValue()" - case (ShortType, "java.lang.Object") => (s: String) => - s"((java.lang.Short)$s).shortValue()" - case (ByteType, "java.lang.Object") => (s: String) => - s"((java.lang.Byte)$s).byteValue()" - case (DoubleType, "java.lang.Object") => (s: String) => - s"((java.lang.Double)$s).doubleValue()" - case (BooleanType, "java.lang.Object") => (s: String) => - s"((java.lang.Boolean)$s).booleanValue()" - case _ => identity[String] _ - } - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) val obj = targetObject.genCode(ctx) val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") - // If the function can return null, we do an extra check to make sure our null bit is still set - // correctly. - val objNullCheck = if (ctx.defaultValue(dataType) == "null") { - s"boolean ${ev.isNull} = ${ev.value} == null;" + val callFunc = if (method.isDefined && method.get.getReturnType.isPrimitive) { + s"${obj.value}.$functionName($argString)" } else { - ev.isNull = obj.isNull - "" + s"(${ctx.boxedType(javaType)}) ${obj.value}.$functionName($argString)" } - val value = unboxer(s"${obj.value}.$functionName($argString)") + val setIsNull = if (propagateNull && arguments.nonEmpty) { + s"boolean ${ev.isNull} = ${obj.isNull} || ${argGen.map(_.isNull).mkString(" || ")};" + } else { + s"boolean ${ev.isNull} = ${obj.isNull};" + } val evaluate = if (method.forall(_.getExceptionTypes.isEmpty)) { - s"$javaType ${ev.value} = ${obj.isNull} ? ${ctx.defaultValue(dataType)} : ($javaType) $value;" + s"final $javaType ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(dataType)} : $callFunc;" } else { s""" $javaType ${ev.value} = ${ctx.defaultValue(javaType)}; try { - ${ev.value} = ${obj.isNull} ? ${ctx.defaultValue(javaType)} : ($javaType) $value; + ${ev.value} = ${ev.isNull} ? ${ctx.defaultValue(javaType)} : $callFunc; } catch (Exception e) { org.apache.spark.unsafe.Platform.throwException(e); } """ } - ev.copy(code = s""" + // If the function can return null, we do an extra check to make sure our null bit is still set + // correctly. + val postNullCheck = if (ctx.defaultValue(dataType) == "null") { + s"${ev.isNull} = ${ev.value} == null;" + } else { + "" + } + + val code = s""" ${obj.code} ${argGen.map(_.code).mkString("\n")} + $setIsNull $evaluate - $objNullCheck - """) + $postNullCheck + """ + ev.copy(code = code) } override def toString: String = s"$targetObject.$functionName" @@ -246,11 +236,13 @@ case class NewInstance( val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) - val setup = - s""" - ${argGen.map(_.code).mkString("\n")} - ${outer.map(_.code).getOrElse("")} - """.stripMargin + var isNull = ev.isNull + val setIsNull = if (propagateNull && arguments.nonEmpty) { + s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" + } else { + isNull = "false" + "" + } val constructorCall = outer.map { gen => s"""${gen.value}.new ${cls.getSimpleName}($argString)""" @@ -258,27 +250,13 @@ case class NewInstance( s"new $className($argString)" } - if (propagateNull && argGen.nonEmpty) { - val argsNonNull = s"!(${argGen.map(_.isNull).mkString(" || ")})" - - ev.copy(code = s""" - $setup - - boolean ${ev.isNull} = true; - $javaType ${ev.value} = ${ctx.defaultValue(dataType)}; - if ($argsNonNull) { - ${ev.value} = $constructorCall; - ${ev.isNull} = false; - } - """) - } else { - ev.copy(code = s""" - $setup - - final $javaType ${ev.value} = $constructorCall; - final boolean ${ev.isNull} = false; - """) - } + val code = s""" + ${argGen.map(_.code).mkString("\n")} + ${outer.map(_.code).getOrElse("")} + $setIsNull + final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; + """ + ev.copy(code = code, isNull = isNull) } override def toString: String = s"newInstance($cls)" @@ -306,13 +284,14 @@ case class UnwrapOption( val javaType = ctx.javaType(dataType) val inputObject = child.genCode(ctx) - ev.copy(code = s""" + val code = s""" ${inputObject.code} - boolean ${ev.isNull} = ${inputObject.value} == null || ${inputObject.value}.isEmpty(); + final boolean ${ev.isNull} = ${inputObject.isNull} || ${inputObject.value}.isEmpty(); $javaType ${ev.value} = - ${ev.isNull} ? ${ctx.defaultValue(dataType)} : ($javaType)${inputObject.value}.get(); - """) + ${ev.isNull} ? ${ctx.defaultValue(javaType)} : ($javaType) ${inputObject.value}.get(); + """ + ev.copy(code = code) } } @@ -338,14 +317,14 @@ case class WrapOption(child: Expression, optType: DataType) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val inputObject = child.genCode(ctx) - ev.copy(code = s""" + val code = s""" ${inputObject.code} - boolean ${ev.isNull} = false; scala.Option ${ev.value} = ${inputObject.isNull} ? scala.Option$$.MODULE$$.apply(null) : new scala.Some(${inputObject.value}); - """) + """ + ev.copy(code = code, isNull = "false") } } @@ -474,7 +453,7 @@ case class MapObjects private( s"${loopVar.isNull} = ${genInputData.isNull} || ${loopVar.value} == null;" } - ev.copy(code = s""" + val code = s""" ${genInputData.code} boolean ${ev.isNull} = ${genInputData.value} == null; @@ -504,7 +483,8 @@ case class MapObjects private( ${ev.isNull} = false; ${ev.value} = new ${classOf[GenericArrayData].getName}($convertedArray); } - """) + """ + ev.copy(code = code) } } @@ -539,14 +519,16 @@ case class CreateExternalRow(children: Seq[Expression], schema: StructType) } """ } + val childrenCode = ctx.splitExpressions(ctx.INPUT_ROW, childrenCodes) val schemaField = ctx.addReferenceObj("schema", schema) - ev.copy(code = s""" - boolean ${ev.isNull} = false; + + val code = s""" $values = new Object[${children.size}]; $childrenCode final ${classOf[Row].getName} ${ev.value} = new $rowClass($values, this.$schemaField); - """) + """ + ev.copy(code = code, isNull = "false") } } @@ -579,14 +561,14 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) // Code to serialize. val input = child.genCode(ctx) - ev.copy(code = s""" + val javaType = ctx.javaType(dataType) + val serialize = s"$serializer.serialize(${input.value}, null).array()" + + val code = s""" ${input.code} - final boolean ${ev.isNull} = ${input.isNull}; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - ${ev.value} = $serializer.serialize(${input.value}, null).array(); - } - """) + final $javaType ${ev.value} = ${input.isNull} ? ${ctx.defaultValue(javaType)} : $serialize; + """ + ev.copy(code = code, isNull = input.isNull) } override def dataType: DataType = BinaryType @@ -617,17 +599,17 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B serializer, s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") - // Code to serialize. + // Code to deserialize. val input = child.genCode(ctx) - ev.copy(code = s""" + val javaType = ctx.javaType(dataType) + val deserialize = + s"($javaType) $serializer.deserialize(java.nio.ByteBuffer.wrap(${input.value}), null)" + + val code = s""" ${input.code} - final boolean ${ev.isNull} = ${input.isNull}; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - ${ev.value} = (${ctx.javaType(dataType)}) - $serializer.deserialize(java.nio.ByteBuffer.wrap(${input.value}), null); - } - """) + final $javaType ${ev.value} = ${input.isNull} ? ${ctx.defaultValue(javaType)} : $deserialize; + """ + ev.copy(code = code, isNull = input.isNull) } override def dataType: DataType = ObjectType(tag.runtimeClass) @@ -658,15 +640,13 @@ case class InitializeJavaBean(beanInstance: Expression, setters: Map[String, Exp """ } - ev.isNull = instanceGen.isNull - ev.value = instanceGen.value - - ev.copy(code = s""" + val code = s""" ${instanceGen.code} if (!${instanceGen.isNull}) { ${initialize.mkString("\n")} } - """) + """ + ev.copy(code = code, isNull = instanceGen.isNull, value = instanceGen.value) } } @@ -696,13 +676,15 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) "If the schema is inferred from a Scala tuple/case class, or a Java bean, " + "please try to use scala.Option[_] or other nullable types " + "(e.g. java.lang.Integer instead of int/scala.Int)." - val idx = ctx.references.length - ctx.references += errMsg - ExprCode(code = s""" + val errMsgField = ctx.addReferenceObj("errMsg", errMsg) + + val code = s""" ${childGen.code} if (${childGen.isNull}) { - throw new RuntimeException((String) references[$idx]); - }""", isNull = "false", value = childGen.value) + throw new RuntimeException(this.$errMsgField); + } + """ + ev.copy(code = code, isNull = "false", value = childGen.value) } } From eb7336a754574879fc28c3c10cb68651329a346d Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Mon, 2 May 2016 11:03:37 -0700 Subject: [PATCH 0005/1470] =?UTF-8?q?[SPARK-14845][SPARK=5FSUBMIT][YARN]?= =?UTF-8?q?=20spark.files=20in=20properties=20file=20is=20n=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? initialize SparkSubmitArgument#files first from spark-submit arguments then from properties file, so that sys property spark.yarn.dist.files will be set correctly. ``` OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.files"), ``` ## How was this patch tested? manul test. file defined in properties file is also distributed to driver in yarn-cluster mode. Author: Jeff Zhang Closes #12656 from zjffdu/SPARK-14845. (cherry picked from commit 0a3026990bd0cbad53f0001da793349201104958) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 1 + 1 file changed, 1 insertion(+) 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 78da1b70c54a5..206c130c76373 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -173,6 +173,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orNull name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull + files = Option(files).orElse(sparkProperties.get("spark.files")).orNull ivyRepoPath = sparkProperties.get("spark.jars.ivy").orNull packages = Option(packages).orElse(sparkProperties.get("spark.jars.packages")).orNull packagesExclusions = Option(packagesExclusions) From 08ae32e6104e998b3c9a4822e563e63aeae55578 Mon Sep 17 00:00:00 2001 From: Andrew Ray Date: Mon, 2 May 2016 11:12:55 -0700 Subject: [PATCH 0006/1470] [SPARK-13749][SQL] Faster pivot implementation for many distinct values with two phase aggregation ## What changes were proposed in this pull request? The existing implementation of pivot translates into a single aggregation with one aggregate per distinct pivot value. When the number of distinct pivot values is large (say 1000+) this can get extremely slow since each input value gets evaluated on every aggregate even though it only affects the value of one of them. I'm proposing an alternate strategy for when there are 10+ (somewhat arbitrary threshold) distinct pivot values. We do two phases of aggregation. In the first we group by the grouping columns plus the pivot column and perform the specified aggregations (one or sometimes more). In the second aggregation we group by the grouping columns and use the new (non public) PivotFirst aggregate that rearranges the outputs of the first aggregation into an array indexed by the pivot value. Finally we do a project to extract the array entries into the appropriate output column. ## How was this patch tested? Additional unit tests in DataFramePivotSuite and manual larger scale testing. Author: Andrew Ray Closes #11583 from aray/fast-pivot. (cherry picked from commit 99274418684ebae5b98d15b4686b95c1ac029e94) Signed-off-by: Yin Huai --- .../sql/catalyst/analysis/Analyzer.scala | 85 ++++++---- .../expressions/aggregate/PivotFirst.scala | 152 ++++++++++++++++++ .../spark/sql/DataFramePivotSuite.scala | 92 ++++++++++- 3 files changed, 296 insertions(+), 33 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e98036a970d44..2f8ab3f43586d 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 @@ -363,43 +363,68 @@ class Analyzer( object ResolvePivot extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p: Pivot if !p.childrenResolved | !p.aggregates.forall(_.resolved) => p + case p: Pivot if !p.childrenResolved | !p.aggregates.forall(_.resolved) + | !p.groupByExprs.forall(_.resolved) | !p.pivotColumn.resolved => p case Pivot(groupByExprs, pivotColumn, pivotValues, aggregates, child) => val singleAgg = aggregates.size == 1 - val pivotAggregates: Seq[NamedExpression] = pivotValues.flatMap { value => - def ifExpr(expr: Expression) = { - If(EqualTo(pivotColumn, value), expr, Literal(null)) + def outputName(value: Literal, aggregate: Expression): String = { + if (singleAgg) value.toString else value + "_" + aggregate.sql + } + if (aggregates.forall(a => PivotFirst.supportsDataType(a.dataType))) { + // Since evaluating |pivotValues| if statements for each input row can get slow this is an + // alternate plan that instead uses two steps of aggregation. + val namedAggExps: Seq[NamedExpression] = aggregates.map(a => Alias(a, a.sql)()) + val namedPivotCol = pivotColumn match { + case n: NamedExpression => n + case _ => Alias(pivotColumn, "__pivot_col")() + } + val bigGroup = groupByExprs :+ namedPivotCol + val firstAgg = Aggregate(bigGroup, bigGroup ++ namedAggExps, child) + val castPivotValues = pivotValues.map(Cast(_, pivotColumn.dataType).eval(EmptyRow)) + val pivotAggs = namedAggExps.map { a => + Alias(PivotFirst(namedPivotCol.toAttribute, a.toAttribute, castPivotValues) + .toAggregateExpression() + , "__pivot_" + a.sql)() + } + val secondAgg = Aggregate(groupByExprs, groupByExprs ++ pivotAggs, firstAgg) + val pivotAggAttribute = pivotAggs.map(_.toAttribute) + val pivotOutputs = pivotValues.zipWithIndex.flatMap { case (value, i) => + aggregates.zip(pivotAggAttribute).map { case (aggregate, pivotAtt) => + Alias(ExtractValue(pivotAtt, Literal(i), resolver), outputName(value, aggregate))() + } } - aggregates.map { aggregate => - val filteredAggregate = aggregate.transformDown { - // Assumption is the aggregate function ignores nulls. This is true for all current - // AggregateFunction's with the exception of First and Last in their default mode - // (which we handle) and possibly some Hive UDAF's. - case First(expr, _) => - First(ifExpr(expr), Literal(true)) - case Last(expr, _) => - Last(ifExpr(expr), Literal(true)) - case a: AggregateFunction => - a.withNewChildren(a.children.map(ifExpr)) - }.transform { - // We are duplicating aggregates that are now computing a different value for each - // pivot value. - // TODO: Don't construct the physical container until after analysis. - case ae: AggregateExpression => ae.copy(resultId = NamedExpression.newExprId) + Project(groupByExprs ++ pivotOutputs, secondAgg) + } else { + val pivotAggregates: Seq[NamedExpression] = pivotValues.flatMap { value => + def ifExpr(expr: Expression) = { + If(EqualTo(pivotColumn, value), expr, Literal(null)) } - if (filteredAggregate.fastEquals(aggregate)) { - throw new AnalysisException( - s"Aggregate expression required for pivot, found '$aggregate'") + aggregates.map { aggregate => + val filteredAggregate = aggregate.transformDown { + // Assumption is the aggregate function ignores nulls. This is true for all current + // AggregateFunction's with the exception of First and Last in their default mode + // (which we handle) and possibly some Hive UDAF's. + case First(expr, _) => + First(ifExpr(expr), Literal(true)) + case Last(expr, _) => + Last(ifExpr(expr), Literal(true)) + case a: AggregateFunction => + a.withNewChildren(a.children.map(ifExpr)) + }.transform { + // We are duplicating aggregates that are now computing a different value for each + // pivot value. + // TODO: Don't construct the physical container until after analysis. + case ae: AggregateExpression => ae.copy(resultId = NamedExpression.newExprId) + } + if (filteredAggregate.fastEquals(aggregate)) { + throw new AnalysisException( + s"Aggregate expression required for pivot, found '$aggregate'") + } + Alias(filteredAggregate, outputName(value, aggregate))() } - val name = if (singleAgg) value.toString else value + "_" + aggregate.sql - Alias(filteredAggregate, name)() } + Aggregate(groupByExprs, groupByExprs ++ pivotAggregates, child) } - val newGroupByExprs = groupByExprs.map { - case UnresolvedAlias(e, _) => e - case e => e - } - Aggregate(newGroupByExprs, groupByExprs ++ pivotAggregates, child) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala new file mode 100644 index 0000000000000..9154e96e34e9c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import scala.collection.immutable.HashMap + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.types._ + +object PivotFirst { + + def supportsDataType(dataType: DataType): Boolean = updateFunction.isDefinedAt(dataType) + + // Currently UnsafeRow does not support the generic update method (throws + // UnsupportedOperationException), so we need to explicitly support each DataType. + private val updateFunction: PartialFunction[DataType, (MutableRow, Int, Any) => Unit] = { + case DoubleType => + (row, offset, value) => row.setDouble(offset, value.asInstanceOf[Double]) + case IntegerType => + (row, offset, value) => row.setInt(offset, value.asInstanceOf[Int]) + case LongType => + (row, offset, value) => row.setLong(offset, value.asInstanceOf[Long]) + case FloatType => + (row, offset, value) => row.setFloat(offset, value.asInstanceOf[Float]) + case BooleanType => + (row, offset, value) => row.setBoolean(offset, value.asInstanceOf[Boolean]) + case ShortType => + (row, offset, value) => row.setShort(offset, value.asInstanceOf[Short]) + case ByteType => + (row, offset, value) => row.setByte(offset, value.asInstanceOf[Byte]) + case d: DecimalType => + (row, offset, value) => row.setDecimal(offset, value.asInstanceOf[Decimal], d.precision) + } +} + +/** + * PivotFirst is a aggregate function used in the second phase of a two phase pivot to do the + * required rearrangement of values into pivoted form. + * + * For example on an input of + * A | B + * --+-- + * x | 1 + * y | 2 + * z | 3 + * + * with pivotColumn=A, valueColumn=B, and pivotColumnValues=[z,y] the output is [3,2]. + * + * @param pivotColumn column that determines which output position to put valueColumn in. + * @param valueColumn the column that is being rearranged. + * @param pivotColumnValues the list of pivotColumn values in the order of desired output. Values + * not listed here will be ignored. + */ +case class PivotFirst( + pivotColumn: Expression, + valueColumn: Expression, + pivotColumnValues: Seq[Any], + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends ImperativeAggregate { + + override val children: Seq[Expression] = pivotColumn :: valueColumn :: Nil + + override lazy val inputTypes: Seq[AbstractDataType] = children.map(_.dataType) + + override val nullable: Boolean = false + + val valueDataType = valueColumn.dataType + + override val dataType: DataType = ArrayType(valueDataType) + + val pivotIndex = HashMap(pivotColumnValues.zipWithIndex: _*) + + val indexSize = pivotIndex.size + + private val updateRow: (MutableRow, Int, Any) => Unit = PivotFirst.updateFunction(valueDataType) + + override def update(mutableAggBuffer: MutableRow, inputRow: InternalRow): Unit = { + val pivotColValue = pivotColumn.eval(inputRow) + if (pivotColValue != null) { + // We ignore rows whose pivot column value is not in the list of pivot column values. + val index = pivotIndex.getOrElse(pivotColValue, -1) + if (index >= 0) { + val value = valueColumn.eval(inputRow) + if (value != null) { + updateRow(mutableAggBuffer, mutableAggBufferOffset + index, value) + } + } + } + } + + override def merge(mutableAggBuffer: MutableRow, inputAggBuffer: InternalRow): Unit = { + for (i <- 0 until indexSize) { + if (!inputAggBuffer.isNullAt(inputAggBufferOffset + i)) { + val value = inputAggBuffer.get(inputAggBufferOffset + i, valueDataType) + updateRow(mutableAggBuffer, mutableAggBufferOffset + i, value) + } + } + } + + override def initialize(mutableAggBuffer: MutableRow): Unit = valueDataType match { + case d: DecimalType => + // Per doc of setDecimal we need to do this instead of setNullAt for DecimalType. + for (i <- 0 until indexSize) { + mutableAggBuffer.setDecimal(mutableAggBufferOffset + i, null, d.precision) + } + case _ => + for (i <- 0 until indexSize) { + mutableAggBuffer.setNullAt(mutableAggBufferOffset + i) + } + } + + override def eval(input: InternalRow): Any = { + val result = new Array[Any](indexSize) + for (i <- 0 until indexSize) { + result(i) = input.get(mutableAggBufferOffset + i, valueDataType) + } + new GenericArrayData(result) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + + override lazy val aggBufferAttributes: Seq[AttributeReference] = + pivotIndex.toList.sortBy(_._2).map(kv => AttributeReference(kv._1.toString, valueDataType)()) + + override lazy val aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + + override lazy val inputAggBufferAttributes: Seq[AttributeReference] = + aggBufferAttributes.map(_.newInstance()) +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index 368aa5cd141f0..b17284aa94d2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.expressions.aggregate.PivotFirst import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ import testImplicits._ - test("pivot courses with literals") { + test("pivot courses") { checkAnswer( courseSales.groupBy("year").pivot("course", Seq("dotNET", "Java")) .agg(sum($"earnings")), @@ -32,14 +34,14 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ ) } - test("pivot year with literals") { + test("pivot year") { checkAnswer( courseSales.groupBy("course").pivot("year", Seq(2012, 2013)).agg(sum($"earnings")), Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil ) } - test("pivot courses with literals and multiple aggregations") { + test("pivot courses with multiple aggregations") { checkAnswer( courseSales.groupBy($"year") .pivot("course", Seq("dotNET", "Java")) @@ -94,4 +96,88 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ Row(2012, 15000.0, 20000.0) :: Row(2013, 48000.0, 30000.0) :: Nil ) } + + // Tests for optimized pivot (with PivotFirst) below + + test("optimized pivot planned") { + val df = courseSales.groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + val queryExecution = sqlContext.executePlan(df.queryExecution.logical) + assert(queryExecution.simpleString.contains("pivotfirst")) + } + + + test("optimized pivot courses with literals") { + checkAnswer( + courseSales.groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + .select("year", "dotNET", "Java"), + Row(2012, 15000.0, 20000.0) :: Row(2013, 48000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot year with literals") { + checkAnswer( + courseSales.groupBy($"course") + // pivot with extra columns to trigger optimization + .pivot("year", Seq(2012, 2013) ++ (1 to 10)) + .agg(sum($"earnings")) + .select("course", "2012", "2013"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot year with string values (cast)") { + checkAnswer( + courseSales.groupBy("course") + // pivot with extra columns to trigger optimization + .pivot("year", Seq("2012", "2013") ++ (1 to 10).map(_.toString)) + .sum("earnings") + .select("course", "2012", "2013"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot DecimalType") { + val df = courseSales.select($"course", $"year", $"earnings".cast(DecimalType(10, 2))) + .groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + .select("year", "dotNET", "Java") + + assertResult(IntegerType)(df.schema("year").dataType) + assertResult(DecimalType(20, 2))(df.schema("Java").dataType) + assertResult(DecimalType(20, 2))(df.schema("dotNET").dataType) + + checkAnswer(df, Row(2012, BigDecimal(1500000, 2), BigDecimal(2000000, 2)) :: + Row(2013, BigDecimal(4800000, 2), BigDecimal(3000000, 2)) :: Nil) + } + + test("PivotFirst supported datatypes") { + val supportedDataTypes: Seq[DataType] = DoubleType :: IntegerType :: LongType :: FloatType :: + BooleanType :: ShortType :: ByteType :: Nil + for (datatype <- supportedDataTypes) { + assertResult(true)(PivotFirst.supportsDataType(datatype)) + } + assertResult(true)(PivotFirst.supportsDataType(DecimalType(10, 1))) + assertResult(false)(PivotFirst.supportsDataType(null)) + assertResult(false)(PivotFirst.supportsDataType(ArrayType(IntegerType))) + } + + test("optimized pivot with multiple aggregations") { + checkAnswer( + courseSales.groupBy($"year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings"), avg($"earnings")), + Row(Seq(2012, 15000.0, 7500.0, 20000.0, 20000.0) ++ Seq.fill(20)(null): _*) :: + Row(Seq(2013, 48000.0, 48000.0, 30000.0, 30000.0) ++ Seq.fill(20)(null): _*) :: Nil + ) + } + } From 1c2082b643dc01fdeb2405c97dcf5a9551cc782d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 2 May 2016 11:28:21 -0700 Subject: [PATCH 0007/1470] [SPARK-14579][SQL] Fix the race condition in StreamExecution.processAllAvailable again ## What changes were proposed in this pull request? #12339 didn't fix the race condition. MemorySinkSuite is still flaky: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.2/814/testReport/junit/org.apache.spark.sql.streaming/MemorySinkSuite/registering_as_a_table/ Here is an execution order to reproduce it. | Time |Thread 1 | MicroBatchThread | |:-------------:|:-------------:|:-----:| | 1 | | `MemorySink.getOffset` | | 2 | | availableOffsets ++= newData (availableOffsets is not changed here) | | 3 | addData(newData) | | | 4 | Set `noNewData` to `false` in processAllAvailable | | | 5 | | `dataAvailable` returns `false` | | 6 | | noNewData = true | | 7 | `noNewData` is true so just return | | | 8 | assert results and fail | | | 9 | | `dataAvailable` returns true so process the new batch | This PR expands the scope of `awaitBatchLock.synchronized` to eliminate the above race. ## How was this patch tested? test("stress test"). It always failed before this patch. And it will pass after applying this patch. Ignore this test in the PR as it takes several minutes to finish. Author: Shixiong Zhu Closes #12582 from zsxwing/SPARK-14579-2. (cherry picked from commit a35a67a83dbb450d26ce0d142ab106e952670842) Signed-off-by: Michael Armbrust --- .../sql/execution/streaming/StreamExecution.scala | 10 +++++----- .../apache/spark/sql/streaming/MemorySinkSuite.scala | 9 +++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index fc18e5f065a04..ce68c0968fb60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -242,12 +242,12 @@ class StreamExecution( // method. See SPARK-14131. // // Check to see what new data is available. - val newData = microBatchThread.runUninterruptibly { - uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - } - availableOffsets ++= newData - val hasNewData = awaitBatchLock.synchronized { + val newData = microBatchThread.runUninterruptibly { + uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) + } + availableOffsets ++= newData + if (dataAvailable) { true } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala index 1f2834054519b..74ca3977d6635 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala @@ -26,6 +26,15 @@ class MemorySinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ test("registering as a table") { + testRegisterAsTable() + } + + ignore("stress test") { + // Ignore the stress test as it takes several minutes to run + (0 until 1000).foreach(_ => testRegisterAsTable()) + } + + private def testRegisterAsTable(): Unit = { val input = MemoryStream[Int] val query = input.toDF().write .format("memory") From 972fd22e3933e58e637781a1da0b6a18afaced17 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 2 May 2016 12:40:21 -0700 Subject: [PATCH 0008/1470] [SPARK-14830][SQL] Add RemoveRepetitionFromGroupExpressions optimizer. ## What changes were proposed in this pull request? This PR aims to optimize GroupExpressions by removing repeating expressions. `RemoveRepetitionFromGroupExpressions` is added. **Before** ```scala scala> sql("select a+1 from values 1,2 T(a) group by a+1, 1+a, A+1, 1+A").explain() == Physical Plan == WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1)#6,(1 + a#0)#7,(A#0 + 1)#8,(1 + A#0)#9], functions=[], output=[(a + 1)#5]) : +- INPUT +- Exchange hashpartitioning((a#0 + 1)#6, (1 + a#0)#7, (A#0 + 1)#8, (1 + A#0)#9, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1) AS (a#0 + 1)#6,(1 + a#0) AS (1 + a#0)#7,(A#0 + 1) AS (A#0 + 1)#8,(1 + A#0) AS (1 + A#0)#9], functions=[], output=[(a#0 + 1)#6,(1 + a#0)#7,(A#0 + 1)#8,(1 + A#0)#9]) : +- INPUT +- LocalTableScan [a#0], [[1],[2]] ``` **After** ```scala scala> sql("select a+1 from values 1,2 T(a) group by a+1, 1+a, A+1, 1+A").explain() == Physical Plan == WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1)#6], functions=[], output=[(a + 1)#5]) : +- INPUT +- Exchange hashpartitioning((a#0 + 1)#6, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1) AS (a#0 + 1)#6], functions=[], output=[(a#0 + 1)#6]) : +- INPUT +- LocalTableScan [a#0], [[1],[2]] ``` ## How was this patch tested? Pass the Jenkins tests (with a new testcase) Author: Dongjoon Hyun Closes #12590 from dongjoon-hyun/SPARK-14830. (cherry picked from commit 6e6320122ea84247c67e2d0fb0e6af54e2c5bb31) Signed-off-by: Michael Armbrust --- .../sql/catalyst/optimizer/Optimizer.scala | 15 +++++++++++++- .../optimizer/AggregateOptimizeSuite.scala | 20 ++++++++++++++++++- 2 files changed, 33 insertions(+), 2 deletions(-) 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 0b70edec8e37a..a147fff274139 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 @@ -68,7 +68,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) ReplaceExceptWithAntiJoin, ReplaceDistinctWithAggregate) :: Batch("Aggregate", fixedPoint, - RemoveLiteralFromGroupExpressions) :: + RemoveLiteralFromGroupExpressions, + RemoveRepetitionFromGroupExpressions) :: Batch("Operator Optimizations", fixedPoint, // Operator push down SetOperationPushDown, @@ -1439,6 +1440,18 @@ object RemoveLiteralFromGroupExpressions extends Rule[LogicalPlan] { } } +/** + * Removes repetition from group expressions in [[Aggregate]], as they have no effect to the result + * but only makes the grouping key bigger. + */ +object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a @ Aggregate(grouping, _, _) => + val newGrouping = ExpressionSet(grouping).toSeq + a.copy(groupingExpressions = newGrouping) + } +} + /** * Computes the current date and time to make sure we return the same result in a single query. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala index e458eb8a1d362..c94dcb33546f8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.sql.catalyst.SimpleCatalystConf +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.Literal @@ -25,10 +28,14 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor class AggregateOptimizeSuite extends PlanTest { + val conf = new SimpleCatalystConf(caseSensitiveAnalysis = false) + val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) + val analyzer = new Analyzer(catalog, conf) object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Aggregate", FixedPoint(100), - RemoveLiteralFromGroupExpressions) :: Nil + RemoveLiteralFromGroupExpressions, + RemoveRepetitionFromGroupExpressions) :: Nil } test("remove literals in grouping expression") { @@ -42,4 +49,15 @@ class AggregateOptimizeSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("remove repetition in grouping expression") { + val input = LocalRelation('a.int, 'b.int, 'c.int) + + val query = input.groupBy('a + 1, 'b + 2, Literal(1) + 'A, Literal(2) + 'B)(sum('c)) + val optimized = Optimize.execute(analyzer.execute(query)) + + val correctAnswer = analyzer.execute(input.groupBy('a + 1, 'b + 2)(sum('c))) + + comparePlans(optimized, correctAnswer) + } } From 4a7e75a203b0a8ecabcb241208aaee5201f6c6e6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 2 May 2016 12:58:59 -0700 Subject: [PATCH 0009/1470] [SPARK-14781] [SQL] support nested predicate subquery ## What changes were proposed in this pull request? In order to support nested predicate subquery, this PR introduce an internal join type ExistenceJoin, which will emit all the rows from left, plus an additional column, which presents there are any rows matched from right or not (it's not null-aware right now). This additional column could be used to replace the subquery in Filter. In theory, all the predicate subquery could use this join type, but it's slower than LeftSemi and LeftAnti, so it's only used for nested subquery (subquery inside OR). For example, the following SQL: ```sql SELECT a FROM t WHERE EXISTS (select 0) OR EXISTS (select 1) ``` This PR also fix a bug in predicate subquery push down through join (they should not). Nested null-aware subquery is still not supported. For example, `a > 3 OR b NOT IN (select bb from t)` After this, we could run TPCDS query Q10, Q35, Q45 ## How was this patch tested? Added unit tests. Author: Davies Liu Closes #12820 from davies/or_exists. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 5 +- .../sql/catalyst/expressions/subquery.scala | 15 ++- .../sql/catalyst/optimizer/Optimizer.scala | 41 ++++++-- .../spark/sql/catalyst/plans/joinTypes.scala | 10 ++ .../plans/logical/basicLogicalOperators.scala | 4 + .../analysis/AnalysisErrorSuite.scala | 11 ++- .../spark/sql/execution/SparkStrategies.scala | 1 + .../joins/BroadcastHashJoinExec.scala | 66 ++++++++++++- .../joins/BroadcastNestedLoopJoinExec.scala | 94 ++++++++++++++----- .../spark/sql/execution/joins/HashJoin.scala | 31 +++++- .../joins/ShuffledHashJoinExec.scala | 13 +-- .../execution/joins/SortMergeJoinExec.scala | 40 ++++++++ .../org/apache/spark/sql/SubquerySuite.scala | 25 +++++ .../execution/joins/ExistenceJoinSuite.scala | 50 +++++++++- 14 files changed, 345 insertions(+), 61 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 61a7d9ea24f4f..6e3a14dfb920d 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 @@ -115,8 +115,9 @@ trait CheckAnalysis extends PredicateHelper { case f @ Filter(condition, child) => splitConjunctivePredicates(condition).foreach { case _: PredicateSubquery | Not(_: PredicateSubquery) => - case e if PredicateSubquery.hasPredicateSubquery(e) => - failAnalysis(s"Predicate sub-queries cannot be used in nested conditions: $e") + case e if PredicateSubquery.hasNullAwarePredicateWithinNot(e) => + failAnalysis(s"Null-aware predicate sub-queries cannot be used in nested" + + s" conditions: $e") case e => } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index cd6d3a00b7cfb..eed062f8bc180 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -92,7 +92,7 @@ case class PredicateSubquery( extends SubqueryExpression with Predicate with Unevaluable { override lazy val resolved = childrenResolved && query.resolved override lazy val references: AttributeSet = super.references -- query.outputSet - override def nullable: Boolean = false + override def nullable: Boolean = nullAware override def plan: LogicalPlan = SubqueryAlias(toString, query) override def withNewPlan(plan: LogicalPlan): PredicateSubquery = copy(query = plan) override def toString: String = s"predicate-subquery#${exprId.id} $conditionString" @@ -105,6 +105,19 @@ object PredicateSubquery { case _ => false }.isDefined } + + /** + * Returns whether there are any null-aware predicate subqueries inside Not. If not, we could + * turn the null-aware predicate into not-null-aware predicate. + */ + def hasNullAwarePredicateWithinNot(e: Expression): Boolean = { + e.find{ x => + x.isInstanceOf[Not] && e.find { + case p: PredicateSubquery => p.nullAware + case _ => false + }.isDefined + }.isDefined + } } /** 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 a147fff274139..e1c969f50f2be 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 @@ -100,8 +100,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) EliminateSorts, SimplifyCasts, SimplifyCaseConversionExpressions, - EliminateSerialization, - RewritePredicateSubquery) :: + EliminateSerialization) :: Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: Batch("Typed Filter Optimization", fixedPoint, @@ -109,7 +108,10 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("LocalRelation", fixedPoint, ConvertToLocalRelation) :: Batch("OptimizeCodegen", Once, - OptimizeCodegen(conf)) :: Nil + OptimizeCodegen(conf)) :: + Batch("RewriteSubquery", Once, + RewritePredicateSubquery, + CollapseProject) :: Nil } /** @@ -1078,7 +1080,14 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { def createOrderedJoin(input: Seq[LogicalPlan], conditions: Seq[Expression]): LogicalPlan = { assert(input.size >= 2) if (input.size == 2) { - Join(input(0), input(1), Inner, conditions.reduceLeftOption(And)) + val (joinConditions, others) = conditions.partition( + e => !PredicateSubquery.hasPredicateSubquery(e)) + val join = Join(input(0), input(1), Inner, joinConditions.reduceLeftOption(And)) + if (others.nonEmpty) { + Filter(others.reduceLeft(And), join) + } else { + join + } } else { val left :: rest = input.toList // find out the first join that have at least one join condition @@ -1091,7 +1100,8 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val right = conditionalJoin.getOrElse(rest.head) val joinedRefs = left.outputSet ++ right.outputSet - val (joinConditions, others) = conditions.partition(_.references.subsetOf(joinedRefs)) + val (joinConditions, others) = conditions.partition( + e => e.references.subsetOf(joinedRefs) && !PredicateSubquery.hasPredicateSubquery(e)) val joined = Join(left, right, Inner, joinConditions.reduceLeftOption(And)) // should not have reference to same logical plan @@ -1201,9 +1211,16 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) val newRight = rightFilterConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) - val newJoinCond = (commonFilterCondition ++ joinCondition).reduceLeftOption(And) + val (newJoinConditions, others) = + commonFilterCondition.partition(e => !PredicateSubquery.hasPredicateSubquery(e)) + val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And) - Join(newLeft, newRight, Inner, newJoinCond) + val join = Join(newLeft, newRight, Inner, newJoinCond) + if (others.nonEmpty) { + Filter(others.reduceLeft(And), join) + } else { + join + } case RightOuter => // push down the right side only `where` condition val newLeft = left @@ -1543,6 +1560,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Note that will almost certainly be planned as a Broadcast Nested Loop join. Use EXISTS // if performance matters to you. Join(p, sub, LeftAnti, Option(Or(anyNull, condition))) + case (p, predicate) => + var joined = p + val replaced = predicate transformUp { + case PredicateSubquery(sub, conditions, nullAware, _) => + // TODO: support null-aware join + val exists = AttributeReference("exists", BooleanType, false)() + joined = Join(joined, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) + exists + } + Project(p.output, Filter(replaced, joined)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 13f57c54a5623..80674d9b4bc9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Attribute object JoinType { def apply(typ: String): JoinType = typ.toLowerCase.replace("_", "") match { @@ -69,6 +70,14 @@ case object LeftAnti extends JoinType { override def sql: String = "LEFT ANTI" } +case class ExistenceJoin(exists: Attribute) extends JoinType { + override def sql: String = { + // This join type is only used in the end of optimizer and physical plans, we will not + // generate SQL for this join type + throw new UnsupportedOperationException + } +} + case class NaturalJoin(tpe: JoinType) extends JoinType { require(Seq(Inner, LeftOuter, RightOuter, FullOuter).contains(tpe), "Unsupported natural join type " + tpe) @@ -84,6 +93,7 @@ case class UsingJoin(tpe: JoinType, usingColumns: Seq[UnresolvedAttribute]) exte object LeftExistence { def unapply(joinType: JoinType): Option[JoinType] = joinType match { case LeftSemi | LeftAnti => Some(joinType) + case j: ExistenceJoin => Some(joinType) case _ => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b2297bbcaa9c5..830a7ac77dd6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -273,6 +273,8 @@ case class Join( override def output: Seq[Attribute] = { joinType match { + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case LeftOuter => @@ -295,6 +297,8 @@ case class Join( case LeftSemi if condition.isDefined => left.constraints .union(splitConjunctivePredicates(condition.get).toSet) + case j: ExistenceJoin => + left.constraints case Inner => left.constraints.union(right.constraints) case LeftExistence(_) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 1b08913ddd0e8..10bff3d6d82ed 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -459,11 +459,14 @@ class AnalysisErrorSuite extends AnalysisTest { val a = AttributeReference("a", IntegerType)() val b = AttributeReference("b", IntegerType)() val c = AttributeReference("c", BooleanType)() - val plan1 = Filter(Cast(In(a, Seq(ListQuery(LocalRelation(b)))), BooleanType), LocalRelation(a)) - assertAnalysisError(plan1, "Predicate sub-queries cannot be used in nested conditions" :: Nil) + val plan1 = Filter(Cast(Not(In(a, Seq(ListQuery(LocalRelation(b))))), BooleanType), + LocalRelation(a)) + assertAnalysisError(plan1, + "Null-aware predicate sub-queries cannot be used in nested conditions" :: Nil) - val plan2 = Filter(Or(In(a, Seq(ListQuery(LocalRelation(b)))), c), LocalRelation(a, c)) - assertAnalysisError(plan2, "Predicate sub-queries cannot be used in nested conditions" :: Nil) + val plan2 = Filter(Or(Not(In(a, Seq(ListQuery(LocalRelation(b))))), c), LocalRelation(a, c)) + assertAnalysisError(plan2, + "Null-aware predicate sub-queries cannot be used in nested conditions" :: Nil) } test("PredicateSubQuery correlated predicate is nested in an illegal plan") { 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 238334e26b45c..9747e58f43717 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 @@ -118,6 +118,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { private def canBuildRight(joinType: JoinType): Boolean = joinType match { case Inner | LeftOuter | LeftSemi | LeftAnti => true + case j: ExistenceJoin => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 587c603192cce..7c194ab72643a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -48,8 +48,6 @@ case class BroadcastHashJoinExec( override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) buildSide match { @@ -85,6 +83,7 @@ case class BroadcastHashJoinExec( case LeftOuter | RightOuter => codegenOuter(ctx, input) case LeftSemi => codegenSemi(ctx, input) case LeftAnti => codegenAnti(ctx, input) + case j: ExistenceJoin => codegenExistence(ctx, input) case x => throw new IllegalArgumentException( s"BroadcastHashJoin should not take $x as the JoinType") @@ -407,4 +406,67 @@ case class BroadcastHashJoinExec( """.stripMargin } } + + /** + * Generates the code for existence join. + */ + private def codegenExistence(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val (broadcastRelation, relationTerm) = prepareBroadcast(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + val existsVar = ctx.freshName("exists") + + val matched = ctx.freshName("matched") + val buildVars = genBuildSideVars(ctx, matched) + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + // filter the output via condition + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |$eval + |${ev.code} + |$existsVar = !${ev.isNull} && ${ev.value}; + """.stripMargin + } else { + s"$existsVar = true;" + } + + val resultVar = input ++ Seq(ExprCode("", "false", existsVar)) + if (broadcastRelation.value.keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |boolean $existsVar = false; + |if ($matched != null) { + | $checkCondition + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $existsVar = false; + |if ($matches != null) { + | while (!$existsVar && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition + | } + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index a659bf26e32df..2a250ecce6de4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -50,19 +50,16 @@ case class BroadcastNestedLoopJoinExec( UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil } - private[this] def genResultProjection: InternalRow => InternalRow = { - if (joinType == LeftSemi) { + private[this] def genResultProjection: InternalRow => InternalRow = joinType match { + case LeftExistence(j) => UnsafeProjection.create(output, output) - } else { + case other => // Always put the stream side on left to simplify implementation // both of left and right side could be null UnsafeProjection.create( output, (streamed.output ++ broadcast.output).map(_.withNullability(true))) - } } - override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def output: Seq[Attribute] = { joinType match { case Inner => @@ -73,6 +70,8 @@ case class BroadcastNestedLoopJoinExec( left.output.map(_.withNullability(true)) ++ right.output case FullOuter => left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case x => @@ -197,6 +196,28 @@ case class BroadcastNestedLoopJoinExec( } } + private def existenceJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = { + assert(buildSide == BuildRight) + streamed.execute().mapPartitionsInternal { streamedIter => + val buildRows = relation.value + val joinedRow = new JoinedRow + + if (condition.isDefined) { + val resultRow = new GenericMutableRow(Array[Any](null)) + streamedIter.map { row => + val result = buildRows.exists(r => boundCondition(joinedRow(row, r))) + resultRow.setBoolean(0, result) + joinedRow(row, resultRow) + } + } else { + val resultRow = new GenericMutableRow(Array[Any](buildRows.nonEmpty)) + streamedIter.map { row => + joinedRow(row, resultRow) + } + } + } + } + /** * The implementation for these joins: * @@ -204,7 +225,8 @@ case class BroadcastNestedLoopJoinExec( * RightOuter with BuildRight * FullOuter * LeftSemi with BuildLeft - * Anti with BuildLeft + * LeftAnti with BuildLeft + * ExistenceJoin with BuildLeft */ private def defaultJoin(relation: Broadcast[Array[InternalRow]]): RDD[InternalRow] = { /** All rows that either match both-way, or rows from streamed joined with nulls. */ @@ -231,27 +253,50 @@ case class BroadcastNestedLoopJoinExec( new BitSet(relation.value.length) )(_ | _) - if (joinType == LeftSemi) { - assert(buildSide == BuildLeft) - val buf: CompactBuffer[InternalRow] = new CompactBuffer() - var i = 0 - val rel = relation.value - while (i < rel.length) { - if (matchedBroadcastRows.get(i)) { - buf += rel(i).copy() + joinType match { + case LeftSemi => + assert(buildSide == BuildLeft) + val buf: CompactBuffer[InternalRow] = new CompactBuffer() + var i = 0 + val rel = relation.value + while (i < rel.length) { + if (matchedBroadcastRows.get(i)) { + buf += rel(i).copy() + } + i += 1 } - i += 1 - } - return sparkContext.makeRDD(buf) + return sparkContext.makeRDD(buf) + case j: ExistenceJoin => + val buf: CompactBuffer[InternalRow] = new CompactBuffer() + var i = 0 + val rel = relation.value + while (i < rel.length) { + val result = new GenericInternalRow(Array[Any](matchedBroadcastRows.get(i))) + buf += new JoinedRow(rel(i).copy(), result) + i += 1 + } + return sparkContext.makeRDD(buf) + case LeftAnti => + val notMatched: CompactBuffer[InternalRow] = new CompactBuffer() + var i = 0 + val rel = relation.value + while (i < rel.length) { + if (!matchedBroadcastRows.get(i)) { + notMatched += rel(i).copy() + } + i += 1 + } + return sparkContext.makeRDD(notMatched) + case o => } val notMatchedBroadcastRows: Seq[InternalRow] = { val nulls = new GenericMutableRow(streamed.output.size) val buf: CompactBuffer[InternalRow] = new CompactBuffer() - var i = 0 - val buildRows = relation.value val joinedRow = new JoinedRow joinedRow.withLeft(nulls) + var i = 0 + val buildRows = relation.value while (i < buildRows.length) { if (!matchedBroadcastRows.get(i)) { buf += joinedRow.withRight(buildRows(i)).copy() @@ -261,10 +306,6 @@ case class BroadcastNestedLoopJoinExec( buf } - if (joinType == LeftAnti) { - return sparkContext.makeRDD(notMatchedBroadcastRows) - } - val matchedStreamRows = streamRdd.mapPartitionsInternal { streamedIter => val buildRows = relation.value val joinedRow = new JoinedRow @@ -308,13 +349,16 @@ case class BroadcastNestedLoopJoinExec( leftExistenceJoin(broadcastedRelation, exists = true) case (LeftAnti, BuildRight) => leftExistenceJoin(broadcastedRelation, exists = false) + case (j: ExistenceJoin, BuildRight) => + existenceJoin(broadcastedRelation) case _ => /** * LeftOuter with BuildLeft * RightOuter with BuildRight * FullOuter * LeftSemi with BuildLeft - * Anti with BuildLeft + * LeftAnti with BuildLeft + * ExistenceJoin with BuildLeft */ defaultJoin(broadcastedRelation) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 9c173d7bf1011..d46a80423fa35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{RowIterator, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.{IntegralType, LongType} @@ -43,6 +44,8 @@ trait HashJoin { left.output ++ right.output.map(_.withNullability(true)) case RightOuter => left.output.map(_.withNullability(true)) ++ right.output + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case x => @@ -50,6 +53,8 @@ trait HashJoin { } } + override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning + protected lazy val (buildPlan, streamedPlan) = buildSide match { case BuildLeft => (left, right) case BuildRight => (right, left) @@ -110,15 +115,14 @@ trait HashJoin { (r: InternalRow) => true } - protected def createResultProjection(): (InternalRow) => InternalRow = { - if (joinType == LeftSemi) { + protected def createResultProjection(): (InternalRow) => InternalRow = joinType match { + case LeftExistence(_) => UnsafeProjection.create(output, output) - } else { + case _ => // Always put the stream side on left to simplify implementation // both of left and right side could be null UnsafeProjection.create( output, (streamedPlan.output ++ buildPlan.output).map(_.withNullability(true))) - } } private def innerJoin( @@ -184,6 +188,23 @@ trait HashJoin { } } + private def existenceJoin( + streamIter: Iterator[InternalRow], + hashedRelation: HashedRelation): Iterator[InternalRow] = { + val joinKeys = streamSideKeyGenerator() + val result = new GenericMutableRow(Array[Any](null)) + val joinedRow = new JoinedRow + streamIter.map { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + val exists = !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + result.setBoolean(0, exists) + joinedRow(current, result) + } + } + private def antiJoin( streamIter: Iterator[InternalRow], hashedRelation: HashedRelation): Iterator[InternalRow] = { @@ -212,6 +233,8 @@ trait HashJoin { semiJoin(streamedIter, hashed) case LeftAnti => antiJoin(streamedIter, hashed) + case j: ExistenceJoin => + existenceJoin(streamedIter, hashed) case x => throw new IllegalArgumentException( s"BroadcastHashJoin should not take $x as the JoinType") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 3ef2fec352203..0036f9aadc5d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} @@ -44,17 +44,6 @@ case class ShuffledHashJoinExec( "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) - override def outputPartitioning: Partitioning = joinType match { - case Inner => PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning)) - case LeftAnti => left.outputPartitioning - case LeftSemi => left.outputPartitioning - case LeftOuter => left.outputPartitioning - case RightOuter => right.outputPartitioning - case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) - case x => - throw new IllegalArgumentException(s"ShuffledHashJoin should not take $x as the JoinType") - } - override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 775f8ac50818f..f0efa52c3d862 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -53,6 +53,8 @@ case class SortMergeJoinExec( left.output.map(_.withNullability(true)) ++ right.output case FullOuter => (left.output ++ right.output).map(_.withNullability(true)) + case j: ExistenceJoin => + left.output :+ j.exists case LeftExistence(_) => left.output case x => @@ -269,6 +271,44 @@ case class SortMergeJoinExec( override def getRow: InternalRow = currentLeftRow }.toScala + case j: ExistenceJoin => + new RowIterator { + private[this] var currentLeftRow: InternalRow = _ + private[this] val result: MutableRow = new GenericMutableRow(Array[Any](null)) + private[this] val smjScanner = new SortMergeJoinScanner( + createLeftKeyGenerator(), + createRightKeyGenerator(), + keyOrdering, + RowIterator.fromScala(leftIter), + RowIterator.fromScala(rightIter) + ) + private[this] val joinRow = new JoinedRow + + override def advanceNext(): Boolean = { + while (smjScanner.findNextOuterJoinRows()) { + currentLeftRow = smjScanner.getStreamedRow + val currentRightMatches = smjScanner.getBufferedMatches + var found = false + if (currentRightMatches != null) { + var i = 0 + while (!found && i < currentRightMatches.length) { + joinRow(currentLeftRow, currentRightMatches(i)) + if (boundCondition(joinRow)) { + found = true + } + i += 1 + } + } + result.setBoolean(0, found) + numOutputRows += 1 + return true + } + false + } + + override def getRow: InternalRow = resultProj(joinRow(currentLeftRow, result)) + }.toScala + case x => throw new IllegalArgumentException( s"SortMergeJoin should not take $x as the JoinType") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 0bf4c6f960639..ff3f9bb33f9a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -152,6 +152,19 @@ class SubquerySuite extends QueryTest with SharedSQLContext { Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) } + test("EXISTS predicate subquery within OR") { + checkAnswer( + sql("select * from l where exists (select * from r where l.a = r.c)" + + " or exists (select * from r where l.a = r.c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + checkAnswer( + sql("select * from l where not exists (select * from r where l.a = r.c and l.b < r.d)" + + " or not exists (select * from r where l.a = r.c)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + test("IN predicate subquery") { checkAnswer( sql("select * from l where l.a in (select c from r)"), @@ -187,6 +200,18 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } + test("IN predicate subquery within OR") { + checkAnswer( + sql("select * from l where l.a in (select c from r)" + + " or l.a in (select c from r where l.b < r.d)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + intercept[AnalysisException] { + sql("select * from l where a not in (select c from r)" + + " or a not in (select c from r where c is not null)") + } + } + test("complex IN predicate subquery") { checkAnswer( sql("select * from l where (a, b) not in (select c, d from r)"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index b32b6444b6d9a..8093054b6dd19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -18,15 +18,15 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys -import org.apache.spark.sql.catalyst.plans.{Inner, JoinType, LeftAnti, LeftSemi} +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.Join -import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} +import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StructType} class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { @@ -89,6 +89,18 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { ExtractEquiJoinKeys.unapply(join) } + val existsAttr = AttributeReference("exists", BooleanType, false)() + val leftSemiPlus = ExistenceJoin(existsAttr) + def createLeftSemiPlusJoin(join: SparkPlan): SparkPlan = { + val output = join.output.dropRight(1) + val condition = if (joinType == LeftSemi) { + existsAttr + } else { + Not(existsAttr) + } + ProjectExec(output, FilterExec(condition, join)) + } + test(s"$testName using ShuffledHashJoin") { extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { @@ -98,6 +110,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(ShuffledHashJoinExec( + leftKeys, rightKeys, leftSemiPlus, BuildRight, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) } } } @@ -111,6 +129,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastHashJoinExec( + leftKeys, rightKeys, leftSemiPlus, BuildRight, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) } } } @@ -123,6 +147,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { SortMergeJoinExec(leftKeys, rightKeys, joinType, boundCondition, left, right)), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(SortMergeJoinExec( + leftKeys, rightKeys, leftSemiPlus, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) } } } @@ -134,6 +164,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { BroadcastNestedLoopJoinExec(left, right, BuildLeft, joinType, Some(condition))), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastNestedLoopJoinExec( + left, right, BuildLeft, leftSemiPlus, Some(condition)))), + expectedAnswer, + sortAnswers = true) } } @@ -144,6 +180,12 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { BroadcastNestedLoopJoinExec(left, right, BuildRight, joinType, Some(condition))), expectedAnswer, sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastNestedLoopJoinExec( + left, right, BuildRight, leftSemiPlus, Some(condition)))), + expectedAnswer, + sortAnswers = true) } } } From 56dbf165c0206a59701f61649ec654b9a0b15a3f Mon Sep 17 00:00:00 2001 From: Pete Robbins Date: Mon, 2 May 2016 13:16:46 -0700 Subject: [PATCH 0010/1470] [SPARK-13745] [SQL] Support columnar in memory representation on Big Endian platforms ## What changes were proposed in this pull request? parquet datasource and ColumnarBatch tests fail on big-endian platforms This patch adds support for the little-endian byte arrays being correctly interpreted on a big-endian platform ## How was this patch tested? Spark test builds ran on big endian z/Linux and regression build on little endian amd64 Author: Pete Robbins Closes #12397 from robbinspg/master. (cherry picked from commit 8a1ce4899fb9f751dedaaa34ea654dfbc8330852) Signed-off-by: Davies Liu --- .../parquet/VectorizedPlainValuesReader.java | 28 +++++++++- .../vectorized/OffHeapColumnVector.java | 54 +++++++++++++++---- .../vectorized/OnHeapColumnVector.java | 41 ++++++++++---- .../vectorized/ColumnarBatchSuite.scala | 9 ++++ 4 files changed, 110 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java index 2672e0453b392..9475c853a03ff 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.parquet; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import org.apache.spark.sql.execution.vectorized.ColumnVector; import org.apache.spark.unsafe.Platform; @@ -31,6 +33,9 @@ public class VectorizedPlainValuesReader extends ValuesReader implements Vectori private byte[] buffer; private int offset; private int bitOffset; // Only used for booleans. + private ByteBuffer byteBuffer; // used to wrap the byte array buffer + + private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); public VectorizedPlainValuesReader() { } @@ -39,6 +44,9 @@ public VectorizedPlainValuesReader() { public void initFromPage(int valueCount, byte[] bytes, int offset) throws IOException { this.buffer = bytes; this.offset = offset + Platform.BYTE_ARRAY_OFFSET; + if (bigEndianPlatform) { + byteBuffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + } } @Override @@ -103,6 +111,9 @@ public final boolean readBoolean() { @Override public final int readInteger() { int v = Platform.getInt(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Integer.reverseBytes(v); + } offset += 4; return v; } @@ -110,6 +121,9 @@ public final int readInteger() { @Override public final long readLong() { long v = Platform.getLong(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Long.reverseBytes(v); + } offset += 8; return v; } @@ -121,14 +135,24 @@ public final byte readByte() { @Override public final float readFloat() { - float v = Platform.getFloat(buffer, offset); + float v; + if (!bigEndianPlatform) { + v = Platform.getFloat(buffer, offset); + } else { + v = byteBuffer.getFloat(offset - Platform.BYTE_ARRAY_OFFSET); + } offset += 4; return v; } @Override public final double readDouble() { - double v = Platform.getDouble(buffer, offset); + double v; + if (!bigEndianPlatform) { + v = Platform.getDouble(buffer, offset); + } else { + v = byteBuffer.getDouble(offset - Platform.BYTE_ARRAY_OFFSET); + } offset += 8; return v; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index b1901411351a2..b8dd16227ec17 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.nio.ByteBuffer; import java.nio.ByteOrder; import org.apache.commons.lang.NotImplementedException; @@ -28,6 +29,9 @@ * Column data backed using offheap memory. */ public final class OffHeapColumnVector extends ColumnVector { + + private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these two allocations need to maintain binary compatible. We can // directly pass this buffer to external components. private long nulls; @@ -39,9 +43,7 @@ public final class OffHeapColumnVector extends ColumnVector { protected OffHeapColumnVector(int capacity, DataType type) { super(capacity, type, MemoryMode.OFF_HEAP); - if (!ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { - throw new NotImplementedException("Only little endian is supported."); - } + nulls = 0; data = 0; lengthData = 0; @@ -221,8 +223,16 @@ public void putInts(int rowId, int count, int[] src, int srcIndex) { @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - null, data + 4 * rowId, count * 4); + if (!bigEndianPlatform) { + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 4 * rowId, count * 4); + } else { + int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; + long offset = data + 4 * rowId; + for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { + Platform.putInt(null, offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); + } + } } @Override @@ -259,8 +269,16 @@ public void putLongs(int rowId, int count, long[] src, int srcIndex) { @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, - null, data + 8 * rowId, count * 8); + if (!bigEndianPlatform) { + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 8 * rowId, count * 8); + } else { + int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; + long offset = data + 8 * rowId; + for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { + Platform.putLong(null, offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); + } + } } @Override @@ -297,8 +315,16 @@ public void putFloats(int rowId, int count, float[] src, int srcIndex) { @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - null, data + rowId * 4, count * 4); + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 4, count * 4); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + long offset = data + 4 * rowId; + for (int i = 0; i < count; ++i, offset += 4) { + Platform.putFloat(null, offset, bb.getFloat(srcIndex + (4 * i))); + } + } } @Override @@ -336,8 +362,16 @@ public void putDoubles(int rowId, int count, double[] src, int srcIndex) { @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId * 8, count * 8); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + long offset = data + 8 * rowId; + for (int i = 0; i < count; ++i, offset += 8) { + Platform.putDouble(null, offset, bb.getDouble(srcIndex + (8 * i))); + } + } } @Override diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index e97276800daa8..b1ffe4c21049b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.vectorized; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.Arrays; import org.apache.spark.memory.MemoryMode; @@ -27,6 +29,9 @@ * and a java array for the values. */ public final class OnHeapColumnVector extends ColumnVector { + + private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these arrays need to maintain binary compatible. We can // directly pass this buffer to external components. @@ -211,10 +216,11 @@ public void putInts(int rowId, int count, int[] src, int srcIndex) { @Override public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - for (int i = 0; i < count; ++i) { + for (int i = 0; i < count; ++i, srcOffset += 4) { intData[i + rowId] = Platform.getInt(src, srcOffset); - srcIndex += 4; - srcOffset += 4; + if (bigEndianPlatform) { + intData[i + rowId] = java.lang.Integer.reverseBytes(intData[i + rowId]); + } } } @@ -251,10 +257,11 @@ public void putLongs(int rowId, int count, long[] src, int srcIndex) { @Override public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; - for (int i = 0; i < count; ++i) { + for (int i = 0; i < count; ++i, srcOffset += 8) { longData[i + rowId] = Platform.getLong(src, srcOffset); - srcIndex += 8; - srcOffset += 8; + if (bigEndianPlatform) { + longData[i + rowId] = java.lang.Long.reverseBytes(longData[i + rowId]); + } } } @@ -286,8 +293,15 @@ public void putFloats(int rowId, int count, float[] src, int srcIndex) { @Override public void putFloats(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, - floatData, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, floatData, + Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + for (int i = 0; i < count; ++i) { + floatData[i + rowId] = bb.getFloat(srcIndex + (4 * i)); + } + } } @Override @@ -320,8 +334,15 @@ public void putDoubles(int rowId, int count, double[] src, int srcIndex) { @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { - Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, - Platform.DOUBLE_ARRAY_OFFSET + rowId * 8, count * 8); + if (!bigEndianPlatform) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, + Platform.DOUBLE_ARRAY_OFFSET + rowId * 8, count * 8); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + for (int i = 0; i < count; ++i) { + doubleData[i + rowId] = bb.getDouble(srcIndex + (8 * i)); + } + } } @Override diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index a63007fc3bf25..7e576a8657998 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.execution.vectorized import java.nio.charset.StandardCharsets +import java.nio.ByteBuffer +import java.nio.ByteOrder import scala.collection.JavaConverters._ import scala.collection.mutable @@ -280,6 +282,13 @@ class ColumnarBatchSuite extends SparkFunSuite { Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, 2.234) Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, 1.123) + if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { + // Ensure array contains Liitle Endian doubles + var bb = ByteBuffer.wrap(buffer).order(ByteOrder.LITTLE_ENDIAN) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, bb.getDouble(0)) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, bb.getDouble(8)) + } + column.putDoubles(idx, 1, buffer, 8) column.putDoubles(idx + 1, 1, buffer, 0) reference += 1.123 From 740f96f6362a49fd95a6d56d93b966094166bbf2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 May 2016 14:57:00 -0700 Subject: [PATCH 0011/1470] [SPARK-15054] Deprecate old accumulator API ## What changes were proposed in this pull request? This patch deprecates the old accumulator API. ## How was this patch tested? N/A Author: Reynold Xin Closes #12832 from rxin/SPARK-15054. (cherry picked from commit d5c79f564f3557037c5526e2ee5f963dd100fb34) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/Accumulable.scala | 6 ++++-- .../scala/org/apache/spark/Accumulator.scala | 19 +++++++++++-------- .../scala/org/apache/spark/SparkContext.scala | 5 +++++ 3 files changed, 20 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index c76720c4bb8b2..799c7e4fd5006 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -17,14 +17,13 @@ package org.apache.spark -import java.io.{ObjectInputStream, Serializable} +import java.io.Serializable import scala.collection.generic.Growable import scala.reflect.ClassTag import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.util.Utils /** @@ -49,6 +48,7 @@ import org.apache.spark.util.Utils * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ +@deprecated("use AccumulatorV2", "2.0.0") class Accumulable[R, T] private ( val id: Long, // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile @@ -162,6 +162,7 @@ class Accumulable[R, T] private ( * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ +@deprecated("use AccumulatorV2", "2.0.0") trait AccumulableParam[R, T] extends Serializable { /** * Add additional data to the accumulator value. Is allowed to modify and return `r` @@ -191,6 +192,7 @@ trait AccumulableParam[R, T] extends Serializable { } +@deprecated("use AccumulatorV2", "2.0.0") private[spark] class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] extends AccumulableParam[R, T] { diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index 9b007b9776088..e52d36b7b564b 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -17,13 +17,6 @@ package org.apache.spark -import java.util.concurrent.atomic.AtomicLong -import javax.annotation.concurrent.GuardedBy - -import scala.collection.mutable -import scala.ref.WeakReference - -import org.apache.spark.internal.Logging import org.apache.spark.storage.{BlockId, BlockStatus} @@ -58,7 +51,8 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * @param name human-readable name associated with this accumulator * @param countFailedValues whether to accumulate values from failed tasks * @tparam T result type - */ +*/ +@deprecated("use AccumulatorV2", "2.0.0") class Accumulator[T] private[spark] ( // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile @transient private val initialValue: T, @@ -75,6 +69,7 @@ class Accumulator[T] private[spark] ( * * @tparam T type of value to accumulate */ +@deprecated("use AccumulatorV2", "2.0.0") trait AccumulatorParam[T] extends AccumulableParam[T, T] { def addAccumulator(t1: T, t2: T): T = { addInPlace(t1, t2) @@ -82,6 +77,7 @@ trait AccumulatorParam[T] extends AccumulableParam[T, T] { } +@deprecated("use AccumulatorV2", "2.0.0") object AccumulatorParam { // The following implicit objects were in SparkContext before 1.2 and users had to @@ -89,21 +85,25 @@ object AccumulatorParam { // them automatically. However, as there are duplicate codes in SparkContext for backward // compatibility, please update them accordingly if you modify the following implicit objects. + @deprecated("use AccumulatorV2", "2.0.0") implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double): Double = 0.0 } + @deprecated("use AccumulatorV2", "2.0.0") implicit object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int): Int = 0 } + @deprecated("use AccumulatorV2", "2.0.0") implicit object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long): Long = t1 + t2 def zero(initialValue: Long): Long = 0L } + @deprecated("use AccumulatorV2", "2.0.0") implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float): Float = t1 + t2 def zero(initialValue: Float): Float = 0f @@ -112,6 +112,7 @@ object AccumulatorParam { // Note: when merging values, this param just adopts the newer value. This is used only // internally for things that shouldn't really be accumulated across tasks, like input // read method, which should be the same across all tasks in the same stage. + @deprecated("use AccumulatorV2", "2.0.0") private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { def addInPlace(t1: String, t2: String): String = t2 def zero(initialValue: String): String = "" @@ -119,12 +120,14 @@ object AccumulatorParam { // Note: this is expensive as it makes a copy of the list every time the caller adds an item. // A better way to use this is to first accumulate the values yourself then them all at once. + @deprecated("use AccumulatorV2", "2.0.0") private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] } // For the internal metric that records what blocks are updated in a particular task + @deprecated("use AccumulatorV2", "2.0.0") private[spark] object UpdatedBlockStatusesAccumulatorParam extends ListAccumulatorParam[(BlockId, BlockStatus)] diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d0f88d4e4d109..302dec25c66bd 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1219,6 +1219,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `+=` method. Only the driver can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] = { val acc = new Accumulator(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) @@ -1230,6 +1231,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the * driver can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) : Accumulator[T] = { val acc = new Accumulator(initialValue, param, Some(name)) @@ -1243,6 +1245,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param) @@ -1257,6 +1260,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param, Some(name)) @@ -1270,6 +1274,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by * standard mutable collections. So you can use this with mutable Map, Set, etc. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R, T] From 990611cd879d443298e61f672fee41432aab36ef Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 May 2016 15:27:16 -0700 Subject: [PATCH 0012/1470] [SPARK-15052][SQL] Use builder pattern to create SparkSession ## What changes were proposed in this pull request? This patch creates a builder pattern for creating SparkSession. The new code is unused and mostly deadcode. I'm putting it up here for feedback. There are a few TODOs that can be done as follow-up pull requests: - [ ] Update tests to use this - [ ] Update examples to use this - [ ] Clean up SQLContext code w.r.t. this one (i.e. SparkSession shouldn't call into SQLContext.getOrCreate; it should be the other way around) - [ ] Remove SparkSession.withHiveSupport - [ ] Disable the old constructor (by making it private) so the only way to start a SparkSession is through this builder pattern ## How was this patch tested? Part of the future pull request is to clean this up and switch existing tests to use this. Author: Reynold Xin Closes #12830 from rxin/sparksession-builder. (cherry picked from commit ca1b2198581b8de1651a88fc97540570a2347dc9) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/SparkSession.scala | 131 +++++++++++++++++- 1 file changed, 127 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 4c2a7b8ae9060..3836ce2daa56e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -18,9 +18,7 @@ package org.apache.spark.sql import java.beans.Introspector -import java.util.Properties -import scala.collection.immutable import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag @@ -30,7 +28,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{CATALOG_IMPLEMENTATION, ConfigEntry} +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst._ @@ -49,7 +47,16 @@ import org.apache.spark.util.Utils /** - * The entry point to Spark execution. + * The entry point to programming Spark with the Dataset and DataFrame API. + * + * To create a SparkSession, use the following builder pattern: + * + * {{{ + * SparkSession.builder() + * .master("local") + * .config("spark.some.config.option", "some-value"). + * .getOrCreate() + * }}} */ class SparkSession private( @transient val sparkContext: SparkContext, @@ -635,6 +642,122 @@ class SparkSession private( object SparkSession { + /** + * Builder for [[SparkSession]]. + */ + class Builder { + + private[this] val options = new scala.collection.mutable.HashMap[String, String] + + /** + * Sets a name for the application, which will be shown in the Spark web UI. + * + * @since 2.0.0 + */ + def appName(name: String): Builder = config("spark.app.name", name) + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: String): Builder = synchronized { + options += key -> value + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: Long): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: Double): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both [[SparkConf]] and SparkSession's own configuration. + * + * @since 2.0.0 + */ + def config(key: String, value: Boolean): Builder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a list of config options based on the given [[SparkConf]]. + * + * @since 2.0.0 + */ + def config(conf: SparkConf): Builder = synchronized { + conf.getAll.foreach { case (k, v) => options += k -> v } + this + } + + /** + * Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. + * + * @since 2.0.0 + */ + def master(master: String): Builder = config("spark.master", master) + + /** + * Enables Hive support, including connectivity to a persistent Hive metastore, support for + * Hive serdes, and Hive user-defined functions. + * + * @since 2.0.0 + */ + def enableHiveSupport(): Builder = synchronized { + if (hiveClassesArePresent) { + config(CATALOG_IMPLEMENTATION.key, "hive") + } else { + throw new IllegalArgumentException( + "Unable to instantiate SparkSession with Hive support because " + + "Hive classes are not found.") + } + } + + /** + * Gets an existing [[SparkSession]] or, if there is no existing one, creates a new one + * based on the options set in this builder. + * + * @since 2.0.0 + */ + def getOrCreate(): SparkSession = synchronized { + // Step 1. Create a SparkConf + // Step 2. Get a SparkContext + // Step 3. Get a SparkSession + val sparkConf = new SparkConf() + options.foreach { case (k, v) => sparkConf.set(k, v) } + val sparkContext = SparkContext.getOrCreate(sparkConf) + + SQLContext.getOrCreate(sparkContext).sparkSession + } + } + + /** + * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. + * @since 2.0.0 + */ + def builder: Builder = new Builder + private val HIVE_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" private val HIVE_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" From 05bb5b6f64d8b4114e3434bc467385d8cba86fd0 Mon Sep 17 00:00:00 2001 From: poolis Date: Mon, 2 May 2016 16:15:07 -0700 Subject: [PATCH 0013/1470] [SPARK-12928][SQL] Oracle FLOAT datatype is not properly handled when reading via JDBC The contribution is my original work and that I license the work to the project under the project's open source license. Author: poolis Author: Greg Michalopoulos Closes #10899 from poolis/spark-12928. (cherry picked from commit 917d05f43bddc1728735979fe7e62fe631b35e6f) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/jdbc/OracleDialect.scala | 6 ++++++ .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala index 46b3877a7cab3..b795e8b42df0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -38,6 +38,12 @@ private case object OracleDialect extends JdbcDialect { // This is sub-optimal as we have to pick a precision/scale in advance whereas the data // in Oracle is allowed to have different precision/scale for each value. Option(DecimalType(DecimalType.MAX_PRECISION, 10)) + } else if (sqlType == Types.NUMERIC && md.build().getLong("scale") == -127) { + // Handle FLOAT fields in a special way because JDBC ResultSetMetaData converts + // this to NUMERIC with -127 scale + // Not sure if there is a more robust way to identify the field as a float (or other + // numeric types that do not specify a scale. + Option(DecimalType(DecimalType.MAX_PRECISION, 10)) } else { None } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 783511b781096..47a1017caa533 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 @@ -600,6 +600,15 @@ class JDBCSuite extends SparkFunSuite assert(derbyDialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "BOOLEAN") } + test("OracleDialect jdbc type mapping") { + val oracleDialect = JdbcDialects.get("jdbc:oracle") + val metadata = new MetadataBuilder().putString("name", "test_column").putLong("scale", -127) + assert(oracleDialect.getCatalystType(java.sql.Types.NUMERIC, "float", 1, metadata) == + Some(DecimalType(DecimalType.MAX_PRECISION, 10))) + assert(oracleDialect.getCatalystType(java.sql.Types.NUMERIC, "numeric", 0, null) == + Some(DecimalType(DecimalType.MAX_PRECISION, 10))) + } + test("table exists query by jdbc dialect") { val MySQL = JdbcDialects.get("jdbc:mysql://127.0.0.1/db") val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") From fbc73f73186873cfd60581e58aff4a8d919e39b4 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 2 May 2016 16:32:31 -0700 Subject: [PATCH 0014/1470] [SPARK-14785] [SQL] Support correlated scalar subqueries ## What changes were proposed in this pull request? In this PR we add support for correlated scalar subqueries. An example of such a query is: ```SQL select * from tbl1 a where a.value > (select max(value) from tbl2 b where b.key = a.key) ``` The implementation adds the `RewriteCorrelatedScalarSubquery` rule to the Optimizer. This rule plans these subqueries using `LEFT OUTER` joins. It currently supports rewrites for `Project`, `Aggregate` & `Filter` logical plans. I could not find a well defined semantics for the use of scalar subqueries in an `Aggregate`. The current implementation currently evaluates the scalar subquery *before* aggregation. This means that you either have to make scalar subquery part of the grouping expression, or that you have to aggregate it further on. I am open to suggestions on this. The implementation currently forces the uniqueness of a scalar subquery by enforcing that it is aggregated and that the resulting column is wrapped in an `AggregateExpression`. ## How was this patch tested? Added tests to `SubquerySuite`. Author: Herman van Hovell Closes #12822 from hvanhovell/SPARK-14785. --- .../sql/catalyst/analysis/Analyzer.scala | 11 ++- .../sql/catalyst/analysis/CheckAnalysis.scala | 42 +++++++++- .../sql/catalyst/expressions/subquery.scala | 39 +++++---- .../sql/catalyst/optimizer/Optimizer.scala | 82 +++++++++++++++++-- .../plans/logical/basicLogicalOperators.scala | 2 +- .../analysis/AnalysisErrorSuite.scala | 11 +-- .../org/apache/spark/sql/SubquerySuite.scala | 47 +++++++++++ 7 files changed, 195 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 2f8ab3f43586d..59af5b7095a77 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 @@ -1081,10 +1081,10 @@ class Analyzer( // Step 2: Pull out the predicates if the plan is resolved. if (current.resolved) { // Make sure the resolved query has the required number of output columns. This is only - // needed for IN expressions. + // needed for Scalar and IN subqueries. if (requiredColumns > 0 && requiredColumns != current.output.size) { - failAnalysis(s"The number of fields in the value ($requiredColumns) does not " + - s"match with the number of columns in the subquery (${current.output.size})") + failAnalysis(s"The number of columns in the subquery (${current.output.size}) " + + s"does not match the required number of columns ($requiredColumns)") } // Pullout predicates and construct a new plan. f.tupled(rewriteSubQuery(current, plans)) @@ -1099,8 +1099,11 @@ class Analyzer( */ private def resolveSubQueries(plan: LogicalPlan, plans: Seq[LogicalPlan]): LogicalPlan = { plan transformExpressions { + case s @ ScalarSubquery(sub, conditions, exprId) + if sub.resolved && conditions.isEmpty && sub.output.size != 1 => + failAnalysis(s"Scalar subquery must return only one column, but got ${sub.output.size}") case s @ ScalarSubquery(sub, _, exprId) if !sub.resolved => - resolveSubQuery(s, plans)(ScalarSubquery(_, _, exprId)) + resolveSubQuery(s, plans, 1)(ScalarSubquery(_, _, exprId)) case e @ Exists(sub, exprId) => resolveSubQuery(e, plans)(PredicateSubquery(_, _, nullAware = false, exprId)) case In(e, Seq(l @ ListQuery(_, exprId))) if e.resolved => 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 6e3a14dfb920d..800bf01abd674 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.plans.{Inner, RightOuter, UsingJoin} +import org.apache.spark.sql.catalyst.plans.UsingJoin import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ @@ -60,9 +60,6 @@ trait CheckAnalysis extends PredicateHelper { val from = operator.inputSet.map(_.name).mkString(", ") a.failAnalysis(s"cannot resolve '${a.sql}' given input columns: [$from]") - case ScalarSubquery(_, conditions, _) if conditions.nonEmpty => - failAnalysis("Correlated scalar subqueries are not supported.") - case e: Expression if e.checkInputDataTypes().isFailure => e.checkInputDataTypes() match { case TypeCheckResult.TypeCheckFailure(message) => @@ -104,6 +101,36 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis(s"Window specification $s is not valid because $m") case None => w } + + case s @ ScalarSubquery(query, conditions, _) if conditions.nonEmpty => + // Make sure we are using equi-joins. + conditions.foreach { + case _: EqualTo | _: EqualNullSafe => // ok + case e => failAnalysis( + s"The correlated scalar subquery can only contain equality predicates: $e") + } + + // Make sure correlated scalar subqueries contain one row for every outer row by + // enforcing that they are aggregates which contain exactly one aggregate expressions. + // The analyzer has already checked that subquery contained only one output column, and + // added all the grouping expressions to the aggregate. + def checkAggregate(a: Aggregate): Unit = { + val aggregates = a.expressions.flatMap(_.collect { + case a: AggregateExpression => a + }) + if (aggregates.isEmpty) { + failAnalysis("The output of a correlated scalar subquery must be aggregated") + } + } + + query match { + case a: Aggregate => checkAggregate(a) + case Filter(_, a: Aggregate) => checkAggregate(a) + case Project(_, a: Aggregate) => checkAggregate(a) + case Project(_, Filter(_, a: Aggregate)) => checkAggregate(a) + case fail => failAnalysis(s"Correlated scalar subqueries must be Aggregated: $fail") + } + s } operator match { @@ -220,6 +247,13 @@ trait CheckAnalysis extends PredicateHelper { | but one table has '${firstError.output.length}' columns and another table has | '${s.children.head.output.length}' columns""".stripMargin) + case p if p.expressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) => + p match { + case _: Filter | _: Aggregate | _: Project => // Ok + case other => failAnalysis( + s"Correlated scalar sub-queries can only be used in a Filter/Aggregate/Project: $p") + } + case p if p.expressions.exists(PredicateSubquery.hasPredicateSubquery) => failAnalysis(s"Predicate sub-queries can only be used in a Filter: $p") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index eed062f8bc180..5001f9a41e07e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -44,6 +44,15 @@ abstract class SubqueryExpression extends Expression { protected def conditionString: String = children.mkString("[", " && ", "]") } +object SubqueryExpression { + def hasCorrelatedSubquery(e: Expression): Boolean = { + e.find { + case e: SubqueryExpression if e.children.nonEmpty => true + case _ => false + }.isDefined + } +} + /** * A subquery that will return only one row and one column. This will be converted into a physical * scalar subquery during planning. @@ -55,28 +64,26 @@ case class ScalarSubquery( children: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId) extends SubqueryExpression with Unevaluable { - - override def plan: LogicalPlan = SubqueryAlias(toString, query) - override lazy val resolved: Boolean = childrenResolved && query.resolved - - override def dataType: DataType = query.schema.fields.head.dataType - - override def checkInputDataTypes(): TypeCheckResult = { - if (query.schema.length != 1) { - TypeCheckResult.TypeCheckFailure("Scalar subquery must return only one column, but got " + - query.schema.length.toString) - } else { - TypeCheckResult.TypeCheckSuccess - } + override lazy val references: AttributeSet = { + if (query.resolved) super.references -- query.outputSet + else super.references } - + override def dataType: DataType = query.schema.fields.head.dataType override def foldable: Boolean = false override def nullable: Boolean = true - + override def plan: LogicalPlan = SubqueryAlias(toString, query) override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(query = plan) + override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" +} - override def toString: String = s"subquery#${exprId.id} $conditionString" +object ScalarSubquery { + def hasCorrelatedScalarSubquery(e: Expression): Boolean = { + e.find { + case e: ScalarSubquery if e.children.nonEmpty => true + case _ => false + }.isDefined + } } /** 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 e1c969f50f2be..a3ab89dc71145 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec import scala.collection.immutable.HashSet +import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, DistinctAggregationRewriter, EliminateSubqueryAliases, EmptyFunctionRegistry} @@ -100,6 +101,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) EliminateSorts, SimplifyCasts, SimplifyCaseConversionExpressions, + RewriteCorrelatedScalarSubquery, EliminateSerialization) :: Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: @@ -1081,7 +1083,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { assert(input.size >= 2) if (input.size == 2) { val (joinConditions, others) = conditions.partition( - e => !PredicateSubquery.hasPredicateSubquery(e)) + e => !SubqueryExpression.hasCorrelatedSubquery(e)) val join = Join(input(0), input(1), Inner, joinConditions.reduceLeftOption(And)) if (others.nonEmpty) { Filter(others.reduceLeft(And), join) @@ -1101,7 +1103,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val joinedRefs = left.outputSet ++ right.outputSet val (joinConditions, others) = conditions.partition( - e => e.references.subsetOf(joinedRefs) && !PredicateSubquery.hasPredicateSubquery(e)) + e => e.references.subsetOf(joinedRefs) && !SubqueryExpression.hasCorrelatedSubquery(e)) val joined = Join(left, right, Inner, joinConditions.reduceLeftOption(And)) // should not have reference to same logical plan @@ -1134,7 +1136,7 @@ object OuterJoinElimination extends Rule[LogicalPlan] with PredicateHelper { * Returns whether the expression returns null or false when all inputs are nulls. */ private def canFilterOutNull(e: Expression): Boolean = { - if (!e.deterministic || PredicateSubquery.hasPredicateSubquery(e)) return false + if (!e.deterministic || SubqueryExpression.hasCorrelatedSubquery(e)) return false val attributes = e.references.toSeq val emptyRow = new GenericInternalRow(attributes.length) val v = BindReferences.bindReference(e, attributes).eval(emptyRow) @@ -1203,7 +1205,6 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition)) => val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = split(splitConjunctivePredicates(filterCondition), left, right) - joinType match { case Inner => // push down the single side `where` condition into respective sides @@ -1212,7 +1213,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newRight = rightFilterConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val (newJoinConditions, others) = - commonFilterCondition.partition(e => !PredicateSubquery.hasPredicateSubquery(e)) + commonFilterCondition.partition(e => !SubqueryExpression.hasCorrelatedSubquery(e)) val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And) val join = Join(newLeft, newRight, Inner, newJoinCond) @@ -1573,3 +1574,74 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { } } } + +/** + * This rule rewrites correlated [[ScalarSubquery]] expressions into LEFT OUTER joins. + */ +object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { + /** + * Extract all correlated scalar subqueries from an expression. The subqueries are collected using + * the given collector. The expression is rewritten and returned. + */ + private def extractCorrelatedScalarSubqueries[E <: Expression]( + expression: E, + subqueries: ArrayBuffer[ScalarSubquery]): E = { + val newExpression = expression transform { + case s: ScalarSubquery if s.children.nonEmpty => + subqueries += s + s.query.output.head + } + newExpression.asInstanceOf[E] + } + + /** + * Construct a new child plan by left joining the given subqueries to a base plan. + */ + private def constructLeftJoins( + child: LogicalPlan, + subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = { + subqueries.foldLeft(child) { + case (currentChild, ScalarSubquery(query, conditions, _)) => + Project( + currentChild.output :+ query.output.head, + Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + } + } + + /** + * Rewrite [[Filter]], [[Project]] and [[Aggregate]] plans containing correlated scalar + * subqueries. + */ + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a @ Aggregate(grouping, expressions, child) => + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) + if (subqueries.nonEmpty) { + // We currently only allow correlated subqueries in an aggregate if they are part of the + // grouping expressions. As a result we need to replace all the scalar subqueries in the + // grouping expressions by their result. + val newGrouping = grouping.map { e => + subqueries.find(_.semanticEquals(e)).map(_.query.output.head).getOrElse(e) + } + Aggregate(newGrouping, newExpressions, constructLeftJoins(child, subqueries)) + } else { + a + } + case p @ Project(expressions, child) => + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val newExpressions = expressions.map(extractCorrelatedScalarSubqueries(_, subqueries)) + if (subqueries.nonEmpty) { + Project(newExpressions, constructLeftJoins(child, subqueries)) + } else { + p + } + case f @ Filter(condition, child) => + val subqueries = ArrayBuffer.empty[ScalarSubquery] + val newCondition = extractCorrelatedScalarSubqueries(condition, subqueries) + if (subqueries.nonEmpty) { + Project(f.output, Filter(newCondition, constructLeftJoins(child, subqueries))) + } else { + f + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 830a7ac77dd6c..7b4615db0661d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -109,7 +109,7 @@ case class Filter(condition: Expression, child: LogicalPlan) override protected def validConstraints: Set[Expression] = { val predicates = splitConjunctivePredicates(condition) - .filterNot(PredicateSubquery.hasPredicateSubquery) + .filterNot(SubqueryExpression.hasCorrelatedSubquery) child.constraints.union(predicates.toSet) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 10bff3d6d82ed..2e88f61d491cd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -111,7 +111,8 @@ class AnalysisErrorSuite extends AnalysisTest { "scalar subquery with 2 columns", testRelation.select( (ScalarSubquery(testRelation.select('a, dateLit.as('b))) + Literal(1)).as('a)), - "Scalar subquery must return only one column, but got 2" :: Nil) + "The number of columns in the subquery (2)" :: + "does not match the required number of columns (1)":: Nil) errorTest( "scalar subquery with no column", @@ -499,12 +500,4 @@ class AnalysisErrorSuite extends AnalysisTest { LocalRelation(a)) assertAnalysisError(plan3, "Accessing outer query column is not allowed in" :: Nil) } - - test("Correlated Scalar Subquery") { - val a = AttributeReference("a", IntegerType)() - val b = AttributeReference("b", IntegerType)() - val sub = Project(Seq(b), Filter(EqualTo(UnresolvedAttribute("a"), b), LocalRelation(b))) - val plan = Project(Seq(a, Alias(ScalarSubquery(sub), "b")()), LocalRelation(a)) - assertAnalysisError(plan, "Correlated scalar subqueries are not supported." :: Nil) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index ff3f9bb33f9a6..80bb4e05385f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -234,4 +234,51 @@ class SubquerySuite extends QueryTest with SharedSQLContext { sql("select a from l group by 1 having exists (select 1 from r where d < min(b))"), Row(null) :: Row(1) :: Row(3) :: Nil) } + + test("correlated scalar subquery in where") { + checkAnswer( + sql("select * from l where b < (select max(d) from r where a = c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Nil) + } + + test("correlated scalar subquery in select") { + checkAnswer( + sql("select a, (select sum(b) from l l2 where l2.a = l1.a) sum_b from l l1"), + Row(1, 4.0) :: Row(1, 4.0) :: Row(2, 2.0) :: Row(2, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, null) :: Row(6, null) :: Nil) + } + + test("correlated scalar subquery in select (null safe)") { + checkAnswer( + sql("select a, (select sum(b) from l l2 where l2.a <=> l1.a) sum_b from l l1"), + Row(1, 4.0) :: Row(1, 4.0) :: Row(2, 2.0) :: Row(2, 2.0) :: Row(3, 3.0) :: + Row(null, 5.0) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + + test("correlated scalar subquery in aggregate") { + checkAnswer( + sql("select a, (select sum(d) from r where a = c) sum_d from l l1 group by 1, 2"), + Row(1, null) :: Row(2, 6.0) :: Row(3, 2.0) :: Row(null, null) :: Row(6, null) :: Nil) + } + + test("non-aggregated correlated scalar subquery") { + val msg1 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1") + } + assert(msg1.getMessage.contains("Correlated scalar subqueries must be Aggregated")) + + val msg2 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1") + } + assert(msg2.getMessage.contains( + "The output of a correlated scalar subquery must be aggregated")) + } + + test("non-equal correlated scalar subquery") { + val msg1 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a < l1.a) sum_b from l l1") + } + assert(msg1.getMessage.contains( + "The correlated scalar subquery can only contain equality predicates")) + } } From 65b94f46021577288ef6c88e00b5b4ed28da33b8 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Mon, 2 May 2016 16:48:20 -0700 Subject: [PATCH 0015/1470] [SPARK-14747][SQL] Add assertStreaming/assertNoneStreaming checks in DataFrameWriter ## Problem If an end user happens to write code mixed with continuous-query-oriented methods and non-continuous-query-oriented methods: ```scala ctx.read .format("text") .stream("...") // continuous query .write .text("...") // non-continuous query; should be startStream() here ``` He/she would get this somehow confusing exception: > Exception in thread "main" java.lang.AssertionError: assertion failed: No plan for FileSource[./continuous_query_test_input] at scala.Predef$.assert(Predef.scala:170) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at ... ## What changes were proposed in this pull request? This PR adds checks for continuous-query-oriented methods and non-continuous-query-oriented methods in `DataFrameWriter`:
    can be called on continuous query? can be called on non-continuous query?
    mode yes
    trigger yes
    format yes yes
    option/options yes yes
    partitionBy yes yes
    bucketBy yes
    sortBy yes
    save yes
    queryName yes
    startStream yes
    insertInto yes
    saveAsTable yes
    jdbc yes
    json yes
    parquet yes
    orc yes
    text yes
    csv yes
    After this PR's change, the friendly exception would be: > Exception in thread "main" org.apache.spark.sql.AnalysisException: text() can only be called on non-continuous queries; at org.apache.spark.sql.DataFrameWriter.assertNotStreaming(DataFrameWriter.scala:678) at org.apache.spark.sql.DataFrameWriter.text(DataFrameWriter.scala:629) at ss.SSDemo$.main(SSDemo.scala:47) ## How was this patch tested? dedicated unit tests were added Author: Liwei Lin Closes #12521 from lw-lin/dataframe-writer-check. (cherry picked from commit 35d9c8aa69c650f33037813607dc939922c5fc27) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/DataFrameWriter.scala | 59 ++++++- .../DataFrameReaderWriterSuite.scala | 156 ++++++++++++++++++ 2 files changed, 210 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index a57d47d28ceb5..a8f96a9b45b28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -53,6 +53,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def mode(saveMode: SaveMode): DataFrameWriter = { + // mode() is used for non-continuous queries + // outputMode() is used for continuous queries + assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode this } @@ -67,6 +70,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def mode(saveMode: String): DataFrameWriter = { + // mode() is used for non-continuous queries + // outputMode() is used for continuous queries + assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode.toLowerCase match { case "overwrite" => SaveMode.Overwrite case "append" => SaveMode.Append @@ -103,6 +109,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { */ @Experimental def trigger(trigger: Trigger): DataFrameWriter = { + assertStreaming("trigger() can only be called on continuous queries") this.trigger = trigger this } @@ -236,6 +243,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { */ def save(): Unit = { assertNotBucketed() + assertNotStreaming("save() can only be called on non-continuous queries") val dataSource = DataSource( df.sparkSession, className = source, @@ -253,6 +261,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ def queryName(queryName: String): DataFrameWriter = { + assertStreaming("queryName() can only be called on continuous queries") this.extraOptions += ("queryName" -> queryName) this } @@ -276,6 +285,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ def startStream(): ContinuousQuery = { + assertNotBucketed + assertStreaming("startStream() can only be called on continuous queries") + if (source == "memory") { val queryName = extraOptions.getOrElse( @@ -348,6 +360,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { private def insertInto(tableIdent: TableIdentifier): Unit = { assertNotBucketed() + assertNotStreaming("insertInto() can only be called on non-continuous queries") val partitions = normalizedParCols.map(_.map(col => col -> (None: Option[String])).toMap) val overwrite = mode == SaveMode.Overwrite @@ -446,6 +459,8 @@ final class DataFrameWriter private[sql](df: DataFrame) { } private def saveAsTable(tableIdent: TableIdentifier): Unit = { + assertNotStreaming("saveAsTable() can only be called on non-continuous queries") + val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) (tableExists, mode) match { @@ -486,6 +501,8 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { + assertNotStreaming("jdbc() can only be called on non-continuous queries") + val props = new Properties() extraOptions.foreach { case (key, value) => props.put(key, value) @@ -542,7 +559,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def json(path: String): Unit = format("json").save(path) + def json(path: String): Unit = { + assertNotStreaming("json() can only be called on non-continuous queries") + format("json").save(path) + } /** * Saves the content of the [[DataFrame]] in Parquet format at the specified path. @@ -558,7 +578,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def parquet(path: String): Unit = format("parquet").save(path) + def parquet(path: String): Unit = { + assertNotStreaming("parquet() can only be called on non-continuous queries") + format("parquet").save(path) + } /** * Saves the content of the [[DataFrame]] in ORC format at the specified path. @@ -575,7 +598,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.5.0 * @note Currently, this method can only be used together with `HiveContext`. */ - def orc(path: String): Unit = format("orc").save(path) + def orc(path: String): Unit = { + assertNotStreaming("orc() can only be called on non-continuous queries") + format("orc").save(path) + } /** * Saves the content of the [[DataFrame]] in a text file at the specified path. @@ -596,7 +622,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.6.0 */ - def text(path: String): Unit = format("text").save(path) + def text(path: String): Unit = { + assertNotStreaming("text() can only be called on non-continuous queries") + format("text").save(path) + } /** * Saves the content of the [[DataFrame]] in CSV format at the specified path. @@ -620,7 +649,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 2.0.0 */ - def csv(path: String): Unit = format("csv").save(path) + def csv(path: String): Unit = { + assertNotStreaming("csv() can only be called on non-continuous queries") + format("csv").save(path) + } /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options @@ -641,4 +673,21 @@ final class DataFrameWriter private[sql](df: DataFrame) { private var numBuckets: Option[Int] = None private var sortColumnNames: Option[Seq[String]] = None + + /////////////////////////////////////////////////////////////////////////////////////// + // Helper functions + /////////////////////////////////////////////////////////////////////////////////////// + + private def assertNotStreaming(errMsg: String): Unit = { + if (df.isStreaming) { + throw new AnalysisException(errMsg) + } + } + + private def assertStreaming(errMsg: String): Unit = { + if (!df.isStreaming) { + throw new AnalysisException(errMsg) + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala index 00efe21d39de4..c7b2b99822613 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala @@ -368,4 +368,160 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B "org.apache.spark.sql.streaming.test", Map.empty) } + + private def newTextInput = Utils.createTempDir(namePrefix = "text").getCanonicalPath + + test("check trigger() can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.trigger(ProcessingTime("10 seconds"))) + assert(e.getMessage == "trigger() can only be called on continuous queries;") + } + + test("check queryName() can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.queryName("queryName")) + assert(e.getMessage == "queryName() can only be called on continuous queries;") + } + + test("check startStream() can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.startStream()) + assert(e.getMessage == "startStream() can only be called on continuous queries;") + } + + test("check startStream(path) can only be called on continuous queries") { + val df = sqlContext.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.startStream("non_exist_path")) + assert(e.getMessage == "startStream() can only be called on continuous queries;") + } + + test("check mode(SaveMode) can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.mode(SaveMode.Append)) + assert(e.getMessage == "mode() can only be called on non-continuous queries;") + } + + test("check mode(string) can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.mode("append")) + assert(e.getMessage == "mode() can only be called on non-continuous queries;") + } + + test("check bucketBy() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[IllegalArgumentException](w.bucketBy(1, "text").startStream()) + assert(e.getMessage == "Currently we don't support writing bucketed data to this data source.") + } + + test("check sortBy() can only be called on non-continuous queries;") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[IllegalArgumentException](w.sortBy("text").startStream()) + assert(e.getMessage == "Currently we don't support writing bucketed data to this data source.") + } + + test("check save(path) can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.save("non_exist_path")) + assert(e.getMessage == "save() can only be called on non-continuous queries;") + } + + test("check save() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.save()) + assert(e.getMessage == "save() can only be called on non-continuous queries;") + } + + test("check insertInto() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.insertInto("non_exsit_table")) + assert(e.getMessage == "insertInto() can only be called on non-continuous queries;") + } + + test("check saveAsTable() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.saveAsTable("non_exsit_table")) + assert(e.getMessage == "saveAsTable() can only be called on non-continuous queries;") + } + + test("check jdbc() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.jdbc(null, null, null)) + assert(e.getMessage == "jdbc() can only be called on non-continuous queries;") + } + + test("check json() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.json("non_exist_path")) + assert(e.getMessage == "json() can only be called on non-continuous queries;") + } + + test("check parquet() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.parquet("non_exist_path")) + assert(e.getMessage == "parquet() can only be called on non-continuous queries;") + } + + test("check orc() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.orc("non_exist_path")) + assert(e.getMessage == "orc() can only be called on non-continuous queries;") + } + + test("check text() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.text("non_exist_path")) + assert(e.getMessage == "text() can only be called on non-continuous queries;") + } + + test("check csv() can only be called on non-continuous queries") { + val df = sqlContext.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[AnalysisException](w.csv("non_exist_path")) + assert(e.getMessage == "csv() can only be called on non-continuous queries;") + } } From a79797149423568128301507026d7675a6aa6ecb Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 2 May 2016 17:50:40 -0700 Subject: [PATCH 0016/1470] [SPARK-15050][SQL] Put CSV and JSON options as Python csv and json function parameters ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-15050 This PR adds function parameters for Python API for reading and writing `csv()`. ## How was this patch tested? This was tested by `./dev/run_tests`. Author: hyukjinkwon Author: Hyukjin Kwon Closes #12834 from HyukjinKwon/SPARK-15050. (cherry picked from commit d37c7f7f042f7943b5b684e53cf4284c601fb347) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 232 +++++++++++++++++++++---------- 1 file changed, 155 insertions(+), 77 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index cc5e93dcadf4d..e2ee9db049489 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -166,7 +166,10 @@ def stream(self, path=None, format=None, schema=None, **options): return self._df(self._jreader.stream()) @since(1.4) - def json(self, path, schema=None): + def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, + allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, + allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, + mode=None, columnNameOfCorruptRecord=None): """ Loads a JSON file (one object per line) or an RDD of Strings storing JSON objects (one object per record) and returns the result as a :class`DataFrame`. @@ -177,31 +180,36 @@ def json(self, path, schema=None): :param path: string represents path to the JSON dataset, or RDD of Strings storing JSON objects. :param schema: an optional :class:`StructType` for the input schema. + :param primitivesAsString: infers all primitive values as a string type. If None is set, + it uses the default value, ``false``. + :param prefersDecimal: infers all floating-point values as a decimal type. If the values + do not fit in decimal, then it infers them as doubles. If None is + set, it uses the default value, ``false``. + :param allowComments: ignores Java/C++ style comment in JSON records. If None is set, + it uses the default value, ``false``. + :param allowUnquotedFieldNames: allows unquoted JSON field names. If None is set, + it uses the default value, ``false``. + :param allowSingleQuotes: allows single quotes in addition to double quotes. If None is + set, it uses the default value, ``true``. + :param allowNumericLeadingZero: allows leading zeros in numbers (e.g. 00012). If None is + set, it uses the default value, ``false``. + :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character + using backslash quoting mechanism. If None is + set, it uses the default value, ``false``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. - You can set the following JSON-specific options to deal with non-standard JSON files: - * ``primitivesAsString`` (default ``false``): infers all primitive values as a string \ - type - * `prefersDecimal` (default `false`): infers all floating-point values as a decimal \ - type. If the values do not fit in decimal, then it infers them as doubles. - * ``allowComments`` (default ``false``): ignores Java/C++ style comment in JSON records - * ``allowUnquotedFieldNames`` (default ``false``): allows unquoted JSON field names - * ``allowSingleQuotes`` (default ``true``): allows single quotes in addition to double \ - quotes - * ``allowNumericLeadingZeros`` (default ``false``): allows leading zeros in numbers \ - (e.g. 00012) - * ``allowBackslashEscapingAnyCharacter`` (default ``false``): allows accepting quoting \ - of all character using backslash quoting mechanism - * ``mode`` (default ``PERMISSIVE``): allows a mode for dealing with corrupt records \ - during parsing. * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted \ record and puts the malformed string into a new field configured by \ ``columnNameOfCorruptRecord``. When a schema is set by user, it sets \ ``null`` for extra fields. * ``DROPMALFORMED`` : ignores the whole corrupted records. * ``FAILFAST`` : throws an exception when it meets corrupted records. - * ``columnNameOfCorruptRecord`` (default ``_corrupt_record``): allows renaming the \ - new field having malformed string created by ``PERMISSIVE`` mode. \ - This overrides ``spark.sql.columnNameOfCorruptRecord``. + + :param columnNameOfCorruptRecord: allows renaming the new field having malformed string + created by ``PERMISSIVE`` mode. This overrides + ``spark.sql.columnNameOfCorruptRecord``. If None is set, + it uses the default value ``_corrupt_record``. >>> df1 = sqlContext.read.json('python/test_support/sql/people.json') >>> df1.dtypes @@ -214,6 +222,24 @@ def json(self, path, schema=None): """ if schema is not None: self.schema(schema) + if primitivesAsString is not None: + self.option("primitivesAsString", primitivesAsString) + if prefersDecimal is not None: + self.option("prefersDecimal", prefersDecimal) + if allowComments is not None: + self.option("allowComments", allowComments) + if allowUnquotedFieldNames is not None: + self.option("allowUnquotedFieldNames", allowUnquotedFieldNames) + if allowSingleQuotes is not None: + self.option("allowSingleQuotes", allowSingleQuotes) + if allowNumericLeadingZero is not None: + self.option("allowNumericLeadingZero", allowNumericLeadingZero) + if allowBackslashEscapingAnyCharacter is not None: + self.option("allowBackslashEscapingAnyCharacter", allowBackslashEscapingAnyCharacter) + if mode is not None: + self.option("mode", mode) + if columnNameOfCorruptRecord is not None: + self.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) if isinstance(path, basestring): return self._df(self._jreader.json(path)) elif type(path) == list: @@ -270,53 +296,62 @@ def text(self, paths): [Row(value=u'hello'), Row(value=u'this')] """ if isinstance(paths, basestring): - paths = [paths] - return self._df(self._jreader.text(self._sqlContext._sc._jvm.PythonUtils.toSeq(paths))) + path = [paths] + return self._df(self._jreader.text(self._sqlContext._sc._jvm.PythonUtils.toSeq(path))) @since(2.0) - def csv(self, paths): + def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, + comment=None, header=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, + nullValue=None, nanValue=None, positiveInf=None, negativeInf=None, dateFormat=None, + maxColumns=None, maxCharsPerColumn=None, mode=None): """Loads a CSV file and returns the result as a [[DataFrame]]. This function goes through the input once to determine the input schema. To avoid going through the entire data once, specify the schema explicitly using [[schema]]. - :param paths: string, or list of strings, for input path(s). - - You can set the following CSV-specific options to deal with CSV files: - * ``sep`` (default ``,``): sets the single character as a separator \ - for each field and value. - * ``charset`` (default ``UTF-8``): decodes the CSV files by the given \ - encoding type. - * ``quote`` (default ``"``): sets the single character used for escaping \ - quoted values where the separator can be part of the value. - * ``escape`` (default ``\``): sets the single character used for escaping quotes \ - inside an already quoted value. - * ``comment`` (default empty string): sets the single character used for skipping \ - lines beginning with this character. By default, it is disabled. - * ``header`` (default ``false``): uses the first line as names of columns. - * ``ignoreLeadingWhiteSpace`` (default ``false``): defines whether or not leading \ - whitespaces from values being read should be skipped. - * ``ignoreTrailingWhiteSpace`` (default ``false``): defines whether or not trailing \ - whitespaces from values being read should be skipped. - * ``nullValue`` (default empty string): sets the string representation of a null value. - * ``nanValue`` (default ``NaN``): sets the string representation of a non-number \ - value. - * ``positiveInf`` (default ``Inf``): sets the string representation of a positive \ - infinity value. - * ``negativeInf`` (default ``-Inf``): sets the string representation of a negative \ - infinity value. - * ``dateFormat`` (default ``None``): sets the string that indicates a date format. \ - Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This \ - applies to both date type and timestamp type. By default, it is None which means \ - trying to parse times and date by ``java.sql.Timestamp.valueOf()`` and \ - ``java.sql.Date.valueOf()``. - * ``maxColumns`` (default ``20480``): defines a hard limit of how many columns \ - a record can have. - * ``maxCharsPerColumn`` (default ``1000000``): defines the maximum number of \ - characters allowed for any given value being read. - * ``mode`` (default ``PERMISSIVE``): allows a mode for dealing with corrupt records \ - during parsing. - * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. \ + :param path: string, or list of strings, for input path(s). + :param schema: an optional :class:`StructType` for the input schema. + :param sep: sets the single character as a separator for each field and value. + If None is set, it uses the default value, ``,``. + :param encoding: decodes the CSV files by the given encoding type. If None is set, + it uses the default value, ``UTF-8``. + :param quote: sets the single character used for escaping quoted values where the + separator can be part of the value. If None is set, it uses the default + value, ``"``. + :param escape: sets the single character used for escaping quotes inside an already + quoted value. If None is set, it uses the default value, ``\``. + :param comment: sets the single character used for skipping lines beginning with this + character. By default (None), it is disabled. + :param header: uses the first line as names of columns. If None is set, it uses the + default value, ``false``. + :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param ignoreTrailingWhiteSpace: defines whether or not trailing whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. + :param nanValue: sets the string representation of a non-number value. If None is set, it + uses the default value, ``NaN``. + :param positiveInf: sets the string representation of a positive infinity value. If None + is set, it uses the default value, ``Inf``. + :param negativeInf: sets the string representation of a negative infinity value. If None + is set, it uses the default value, ``Inf``. + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to both date type and timestamp type. By default, it is None + which means trying to parse times and date by + ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. + :param maxColumns: defines a hard limit of how many columns a record can have. If None is + set, it uses the default value, ``20480``. + :param maxCharsPerColumn: defines the maximum number of characters allowed for any given + value being read. If None is set, it uses the default value, + ``1000000``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. + + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. When a schema is set by user, it sets ``null`` for extra fields. * ``DROPMALFORMED`` : ignores the whole corrupted records. * ``FAILFAST`` : throws an exception when it meets corrupted records. @@ -325,9 +360,43 @@ def csv(self, paths): >>> df.dtypes [('C0', 'string'), ('C1', 'string')] """ - if isinstance(paths, basestring): - paths = [paths] - return self._df(self._jreader.csv(self._sqlContext._sc._jvm.PythonUtils.toSeq(paths))) + if schema is not None: + self.schema(schema) + if sep is not None: + self.option("sep", sep) + if encoding is not None: + self.option("encoding", encoding) + if quote is not None: + self.option("quote", quote) + if escape is not None: + self.option("escape", escape) + if comment is not None: + self.option("comment", comment) + if header is not None: + self.option("header", header) + if ignoreLeadingWhiteSpace is not None: + self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) + if ignoreTrailingWhiteSpace is not None: + self.option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) + if nullValue is not None: + self.option("nullValue", nullValue) + if nanValue is not None: + self.option("nanValue", nanValue) + if positiveInf is not None: + self.option("positiveInf", positiveInf) + if negativeInf is not None: + self.option("negativeInf", negativeInf) + if dateFormat is not None: + self.option("dateFormat", dateFormat) + if maxColumns is not None: + self.option("maxColumns", maxColumns) + if maxCharsPerColumn is not None: + self.option("maxCharsPerColumn", maxCharsPerColumn) + if mode is not None: + self.option("mode", mode) + if isinstance(path, basestring): + path = [path] + return self._df(self._jreader.csv(self._sqlContext._sc._jvm.PythonUtils.toSeq(path))) @since(1.5) def orc(self, path): @@ -687,7 +756,8 @@ def text(self, path, compression=None): self._jwrite.text(path) @since(2.0) - def csv(self, path, mode=None, compression=None): + def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, + header=None, nullValue=None): """Saves the content of the [[DataFrame]] in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -701,25 +771,33 @@ def csv(self, path, mode=None, compression=None): :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). - - You can set the following CSV-specific options to deal with CSV files: - * ``sep`` (default ``,``): sets the single character as a separator \ - for each field and value. - * ``quote`` (default ``"``): sets the single character used for escaping \ - quoted values where the separator can be part of the value. - * ``escape`` (default ``\``): sets the single character used for escaping quotes \ - inside an already quoted value. - * ``header`` (default ``false``): writes the names of columns as the first line. - * ``nullValue`` (default empty string): sets the string representation of a null value. - * ``compression``: compression codec to use when saving to file. This can be one of \ - the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and \ - deflate). + :param sep: sets the single character as a separator for each field and value. If None is + set, it uses the default value, ``,``. + :param quote: sets the single character used for escaping quoted values where the + separator can be part of the value. If None is set, it uses the default + value, ``"``. + :param escape: sets the single character used for escaping quotes inside an already + quoted value. If None is set, it uses the default value, ``\`` + :param header: writes the names of columns as the first line. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) if compression is not None: self.option("compression", compression) + if sep is not None: + self.option("sep", sep) + if quote is not None: + self.option("quote", quote) + if escape is not None: + self.option("escape", escape) + if header is not None: + self.option("header", header) + if nullValue is not None: + self.option("nullValue", nullValue) self._jwrite.csv(path) @since(1.5) @@ -797,7 +875,7 @@ def _test(): globs['sqlContext'] = SQLContext(sc) globs['hiveContext'] = HiveContext._createForTesting(sc) globs['df'] = globs['sqlContext'].read.parquet('python/test_support/sql/parquet_partitioned') - globs['sdf'] =\ + globs['sdf'] = \ globs['sqlContext'].read.format('text').stream('python/test_support/sql/streaming') (failure_count, test_count) = doctest.testmod( From 86167968f7dea8a44fae2d7bdb0bfe8d735e5004 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 2 May 2016 18:12:31 -0700 Subject: [PATCH 0017/1470] [SPARK-15047][SQL] Cleanup SQL Parser ## What changes were proposed in this pull request? This PR addresses a few minor issues in SQL parser: - Removes some unused rules and keywords in the grammar. - Removes code path for fallback SQL parsing (was needed for Hive native parsing). - Use `UnresolvedGenerator` instead of hard-coding `Explode` & `JsonTuple`. - Adds a more generic way of creating error messages for unsupported Hive features. - Use `visitFunctionName` as much as possible. - Interpret a `CatalogColumn`'s `DataType` directly instead of parsing it again. ## How was this patch tested? Existing tests. Author: Herman van Hovell Closes #12826 from hvanhovell/SPARK-15047. (cherry picked from commit 1c19c2769edecaefabc2cd67b3b32f901feb3f59) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 35 +++---------------- .../sql/catalyst/parser/AstBuilder.scala | 31 +++------------- .../sql/catalyst/parser/ParseDriver.scala | 10 ++---- .../sql/catalyst/parser/PlanParserSuite.scala | 15 ++++---- .../spark/sql/execution/SparkSqlParser.scala | 12 ++++--- .../spark/sql/hive/HiveDDLCommandSuite.scala | 5 +-- 6 files changed, 31 insertions(+), 77 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4d5d125ecdd7e..cc4e5c853e679 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -121,17 +121,13 @@ statement | UNCACHE TABLE identifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE - tableIdentifier partitionSpec? #loadData + tableIdentifier partitionSpec? #loadData + | TRUNCATE TABLE tableIdentifier partitionSpec? + (COLUMNS identifierList)? #truncateTable | ADD identifier .*? #addResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration - | kws=unsupportedHiveNativeCommands .*? #failNativeCommand - | hiveNativeCommands #executeNativeCommand - ; - -hiveNativeCommands - : TRUNCATE TABLE tableIdentifier partitionSpec? - (COLUMNS identifierList)? + | unsupportedHiveNativeCommands .*? #failNativeCommand ; unsupportedHiveNativeCommands @@ -267,14 +263,6 @@ nestedConstantList : '(' constantList (',' constantList)* ')' ; -skewedLocation - : (constant | constantList) EQ STRING - ; - -skewedLocationList - : '(' skewedLocation (',' skewedLocation)* ')' - ; - createFileFormat : STORED AS fileFormat | STORED BY storageHandler @@ -609,11 +597,6 @@ explainOption : LOGICAL | FORMATTED | EXTENDED | CODEGEN ; -transactionMode - : ISOLATION LEVEL SNAPSHOT #isolationLevel - | READ accessMode=(ONLY | WRITE) #transactionAccessMode - ; - qualifiedName : identifier ('.' identifier)* ; @@ -661,8 +644,7 @@ nonReserved | VIEW | REPLACE | IF | NO | DATA - | START | TRANSACTION | COMMIT | ROLLBACK | WORK | ISOLATION | LEVEL - | SNAPSHOT | READ | WRITE | ONLY + | START | TRANSACTION | COMMIT | ROLLBACK | SORT | CLUSTER | DISTRIBUTE | UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE | FIRST | AFTER | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT @@ -778,13 +760,6 @@ START: 'START'; TRANSACTION: 'TRANSACTION'; COMMIT: 'COMMIT'; ROLLBACK: 'ROLLBACK'; -WORK: 'WORK'; -ISOLATION: 'ISOLATION'; -LEVEL: 'LEVEL'; -SNAPSHOT: 'SNAPSHOT'; -READ: 'READ'; -WRITE: 'WRITE'; -ONLY: 'ONLY'; MACRO: 'MACRO'; IF: 'IF'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1f923f47dd0ba..c3974625aa222 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -81,26 +81,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * ******************************************************************************************** */ protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree) - /** - * Make sure we do not try to create a plan for a native command. - */ - override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null - /** * Create a plan for a SHOW FUNCTIONS command. */ override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { import ctx._ if (qualifiedName != null) { - val names = qualifiedName().identifier().asScala.map(_.getText).toList - names match { - case db :: name :: Nil => - ShowFunctions(Some(db), Some(name)) - case name :: Nil => - ShowFunctions(None, Some(name)) - case _ => - throw new ParseException("SHOW FUNCTIONS unsupported name", ctx) - } + val name = visitFunctionName(qualifiedName) + ShowFunctions(name.database, Some(name.funcName)) } else if (pattern != null) { ShowFunctions(None, Some(string(pattern))) } else { @@ -117,7 +105,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { if (describeFuncName.STRING() != null) { string(describeFuncName.STRING()) } else if (describeFuncName.qualifiedName() != null) { - describeFuncName.qualifiedName().identifier().asScala.map(_.getText).mkString(".") + visitFunctionName(describeFuncName.qualifiedName).unquotedString } else { describeFuncName.getText } @@ -554,19 +542,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { query: LogicalPlan, ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) { val expressions = expressionList(ctx.expression) - - // Create the generator. - val generator = ctx.qualifiedName.getText.toLowerCase match { - case "explode" if expressions.size == 1 => - Explode(expressions.head) - case "json_tuple" => - JsonTuple(expressions) - case name => - UnresolvedGenerator(visitFunctionName(ctx.qualifiedName), expressions) - } - Generate( - generator, + UnresolvedGenerator(visitFunctionName(ctx.qualifiedName), expressions), join = true, outer = ctx.OUTER != null, Some(ctx.tblName.getText.toLowerCase), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index d0132529f18ea..d042e191a9e86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -53,19 +53,15 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser => astBuilder.visitSingleStatement(parser.singleStatement()) match { case plan: LogicalPlan => plan - case _ => nativeCommand(sqlText) + case _ => + val position = Origin(None, None) + throw new ParseException(Option(sqlText), "Unsupported SQL statement", position, position) } } /** Get the builder (visitor) which converts a ParseTree into a AST. */ protected def astBuilder: AstBuilder - /** Create a native command, or fail when this is not supported. */ - protected def nativeCommand(sqlText: String): LogicalPlan = { - val position = Origin(None, None) - throw new ParseException(Option(sqlText), "Unsupported SQL statement", position, position) - } - protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = { logInfo(s"Parsing command: $command") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 5e896a33bd3bf..b7af2ceda6437 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -53,7 +53,7 @@ class PlanParserSuite extends PlanTest { assertEqual("show functions foo", ShowFunctions(None, Some("foo"))) assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar"))) assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*"))) - intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name") + intercept("show functions foo.bar.baz", "Unsupported function name") } test("describe function") { @@ -263,11 +263,14 @@ class PlanParserSuite extends PlanTest { } test("lateral view") { + val explode = UnresolvedGenerator(FunctionIdentifier("explode"), Seq('x)) + val jsonTuple = UnresolvedGenerator(FunctionIdentifier("json_tuple"), Seq('x, 'y)) + // Single lateral view assertEqual( "select * from t lateral view explode(x) expl as x", table("t") - .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x")) + .generate(explode, join = true, outer = false, Some("expl"), Seq("x")) .select(star())) // Multiple lateral views @@ -277,12 +280,12 @@ class PlanParserSuite extends PlanTest { |lateral view explode(x) expl |lateral view outer json_tuple(x, y) jtup q, z""".stripMargin, table("t") - .generate(Explode('x), join = true, outer = false, Some("expl"), Seq.empty) - .generate(JsonTuple(Seq('x, 'y)), join = true, outer = true, Some("jtup"), Seq("q", "z")) + .generate(explode, join = true, outer = false, Some("expl"), Seq.empty) + .generate(jsonTuple, join = true, outer = true, Some("jtup"), Seq("q", "z")) .select(star())) // Multi-Insert lateral views. - val from = table("t1").generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x")) + val from = table("t1").generate(explode, join = true, outer = false, Some("expl"), Seq("x")) assertEqual( """from t1 |lateral view explode(x) expl as x @@ -294,7 +297,7 @@ class PlanParserSuite extends PlanTest { |where s < 10 """.stripMargin, Union(from - .generate(JsonTuple(Seq('x, 'y)), join = true, outer = false, Some("jtup"), Seq("q", "z")) + .generate(jsonTuple, join = true, outer = false, Some("jtup"), Seq("q", "z")) .select(star()) .insertInto("t2"), from.where('s < 10).select(star()).insertInto("t3"))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8128a6efe3ccb..dfc56a7d98ba2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import scala.util.Try import org.antlr.v4.runtime.{ParserRuleContext, Token} +import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier @@ -31,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} - +import org.apache.spark.sql.types.DataType /** * Concrete parser for Spark SQL statements. @@ -780,9 +781,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitFailNativeCommand( ctx: FailNativeCommandContext): LogicalPlan = withOrigin(ctx) { - val keywords = if (ctx.kws != null) { - Seq(ctx.kws.kw1, ctx.kws.kw2, ctx.kws.kw3, ctx.kws.kw4, ctx.kws.kw5, ctx.kws.kw6) - .filter(_ != null).map(_.getText).mkString(" ") + val keywords = if (ctx.unsupportedHiveNativeCommands != null) { + ctx.unsupportedHiveNativeCommands.children.asScala.collect { + case n: TerminalNode => n.getText + }.mkString(" ") } else { // SET ROLE is the exception to the rule, because we handle this before other SET commands. "SET ROLE" @@ -1109,7 +1111,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // just convert the whole type string to lower case, otherwise the struct field names // will no longer be case sensitive. Instead, we rely on our parser to get the proper // case before passing it to Hive. - CatalystSqlParser.parseDataType(col.dataType.getText).catalogString, + typedVisit[DataType](col.dataType).catalogString, nullable = true, Option(col.STRING).map(string)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index c97c28c40c964..8dc3c64353276 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -254,12 +254,13 @@ class HiveDDLCommandSuite extends PlanTest { } test("use native json_tuple instead of hive's UDTF in LATERAL VIEW") { - val plan = parser.parsePlan( + val analyzer = TestHive.sparkSession.sessionState.analyzer + val plan = analyzer.execute(parser.parsePlan( """ |SELECT * |FROM (SELECT '{"f1": "value1", "f2": 12}' json) test |LATERAL VIEW json_tuple(json, 'f1', 'f2') jt AS a, b - """.stripMargin) + """.stripMargin)) assert(plan.children.head.asInstanceOf[Generate].generator.isInstanceOf[JsonTuple]) } From 733cbaa3c0ff617a630a9d6937699db37ad2943b Mon Sep 17 00:00:00 2001 From: bomeng Date: Mon, 2 May 2016 18:20:29 -0700 Subject: [PATCH 0018/1470] [SPARK-15062][SQL] fix list type infer serializer issue ## What changes were proposed in this pull request? Make serializer correctly inferred if the input type is `List[_]`, since `List[_]` is type of `Seq[_]`, before it was matched to different case (`case t if definedByConstructorParams(t)`). ## How was this patch tested? New test case was added. Author: bomeng Closes #12849 from bomeng/SPARK-15062. (cherry picked from commit 0fd95be3cd815154a11ce7d6998311e7c86bc6b9) Signed-off-by: Michael Armbrust --- .../spark/sql/catalyst/ScalaReflection.scala | 11 +++++++---- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 15 ++++++++++++++- 2 files changed, 21 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index be0d75a8304e9..d158a64a85bc0 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 @@ -509,6 +509,13 @@ object ScalaReflection extends ScalaReflection { serializerFor(unwrapped, optType, newPath)) } + // Since List[_] also belongs to localTypeOf[Product], we put this case before + // "case t if definedByConstructorParams(t)" to make sure it will match to the + // case "localTypeOf[Seq[_]]" + case t if t <:< localTypeOf[Seq[_]] => + val TypeRef(_, _, Seq(elementType)) = t + toCatalystArray(inputObject, elementType) + case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => @@ -524,10 +531,6 @@ object ScalaReflection extends ScalaReflection { val TypeRef(_, _, Seq(elementType)) = t toCatalystArray(inputObject, elementType) - case t if t <:< localTypeOf[Seq[_]] => - val TypeRef(_, _, Seq(elementType)) = t - toCatalystArray(inputObject, elementType) - case t if t <:< localTypeOf[Map[_, _]] => val TypeRef(_, _, Seq(keyType, valueType)) = t 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 0672551b2972d..c00e9c7e39561 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 @@ -23,8 +23,9 @@ import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.typeOf import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{BoundReference, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, NewInstance, SpecificMutableRow} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils case class PrimitiveData( @@ -277,6 +278,18 @@ class ScalaReflectionSuite extends SparkFunSuite { assert(anyTypes === Seq(classOf[java.lang.Object], classOf[java.lang.Object])) } + test("SPARK-15062: Get correct serializer for List[_]") { + val list = List(1, 2, 3) + val serializer = serializerFor[List[Int]](BoundReference( + 0, ObjectType(list.getClass), nullable = false)) + assert(serializer.children.size == 2) + assert(serializer.children.head.isInstanceOf[Literal]) + assert(serializer.children.head.asInstanceOf[Literal].value === UTF8String.fromString("value")) + assert(serializer.children.last.isInstanceOf[NewInstance]) + assert(serializer.children.last.asInstanceOf[NewInstance] + .cls.isInstanceOf[Class[org.apache.spark.sql.catalyst.util.GenericArrayData]]) + } + private val dataTypeForComplexData = dataTypeFor[ComplexData] private val typeOfComplexData = typeOf[ComplexData] From dcce0aaafedc496e3e69c02c51ad31f01de05287 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 2 May 2016 18:27:49 -0700 Subject: [PATCH 0019/1470] [SPARK-15077][SQL] Use a fair lock to avoid thread starvation in StreamExecution ## What changes were proposed in this pull request? Right now `StreamExecution.awaitBatchLock` uses an unfair lock. `StreamExecution.awaitOffset` may run too long and fail some test because `StreamExecution.constructNextBatch` keeps getting the lock. See: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/865/testReport/junit/org.apache.spark.sql.streaming/FileStreamSourceStressTestSuite/file_source_stress_test/ This PR uses a fair ReentrantLock to resolve the thread starvation issue. ## How was this patch tested? Modified `FileStreamSourceStressTestSuite.test("file source stress test")` to run the test codes 100 times locally. It always fails because of timeout without this patch. Author: Shixiong Zhu Closes #12852 from zsxwing/SPARK-15077. (cherry picked from commit 4e3685ae5e5826e63bfcd7c3729e3b9cbab484b5) Signed-off-by: Michael Armbrust --- .../execution/streaming/StreamExecution.scala | 79 ++++++++++++------- 1 file changed, 52 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ce68c0968fb60..3108346913800 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal @@ -53,8 +54,12 @@ class StreamExecution( val trigger: Trigger) extends ContinuousQuery with Logging { - /** An monitor used to wait/notify when batches complete. */ - private val awaitBatchLock = new Object + /** + * A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation. + */ + private val awaitBatchLock = new ReentrantLock(true) + private val awaitBatchLockCondition = awaitBatchLock.newCondition() + private val startLatch = new CountDownLatch(1) private val terminationLatch = new CountDownLatch(1) @@ -242,17 +247,22 @@ class StreamExecution( // method. See SPARK-14131. // // Check to see what new data is available. - val hasNewData = awaitBatchLock.synchronized { - val newData = microBatchThread.runUninterruptibly { - uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - } - availableOffsets ++= newData + val hasNewData = { + awaitBatchLock.lock() + try { + val newData = microBatchThread.runUninterruptibly { + uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) + } + availableOffsets ++= newData - if (dataAvailable) { - true - } else { - noNewData = true - false + if (dataAvailable) { + true + } else { + noNewData = true + false + } + } finally { + awaitBatchLock.unlock() } } if (hasNewData) { @@ -269,9 +279,12 @@ class StreamExecution( currentBatchId += 1 logInfo(s"Committed offsets for batch $currentBatchId.") } else { - awaitBatchLock.synchronized { + awaitBatchLock.lock() + try { // Wake up any threads that are waiting for the stream to progress. - awaitBatchLock.notifyAll() + awaitBatchLockCondition.signalAll() + } finally { + awaitBatchLock.unlock() } } } @@ -332,9 +345,12 @@ class StreamExecution( new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) sink.addBatch(currentBatchId - 1, nextBatch) - awaitBatchLock.synchronized { + awaitBatchLock.lock() + try { // Wake up any threads that are waiting for the stream to progress. - awaitBatchLock.notifyAll() + awaitBatchLockCondition.signalAll() + } finally { + awaitBatchLock.unlock() } val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 @@ -374,8 +390,12 @@ class StreamExecution( } while (notDone) { - logInfo(s"Waiting until $newOffset at $source") - awaitBatchLock.synchronized { awaitBatchLock.wait(100) } + awaitBatchLock.lock() + try { + awaitBatchLockCondition.await(100, TimeUnit.MILLISECONDS) + } finally { + awaitBatchLock.unlock() + } } logDebug(s"Unblocked at $newOffset for $source") } @@ -383,16 +403,21 @@ class StreamExecution( /** A flag to indicate that a batch has completed with no new data available. */ @volatile private var noNewData = false - override def processAllAvailable(): Unit = awaitBatchLock.synchronized { - noNewData = false - while (true) { - awaitBatchLock.wait(10000) - if (streamDeathCause != null) { - throw streamDeathCause - } - if (noNewData) { - return + override def processAllAvailable(): Unit = { + awaitBatchLock.lock() + try { + noNewData = false + while (true) { + awaitBatchLockCondition.await(10000, TimeUnit.MILLISECONDS) + if (streamDeathCause != null) { + throw streamDeathCause + } + if (noNewData) { + return + } } + } finally { + awaitBatchLock.unlock() } } From 435d903d3f3d26514d7d9b986ec88a3bd69a4df3 Mon Sep 17 00:00:00 2001 From: Marcin Tustin Date: Mon, 2 May 2016 19:37:57 -0700 Subject: [PATCH 0020/1470] [SPARK-14685][CORE] Document heritability of localProperties ## What changes were proposed in this pull request? This updates the java-/scala- doc for setLocalProperty to document heritability of localProperties. This also adds tests for that behaviour. ## How was this patch tested? Tests pass. New tests were added. Author: Marcin Tustin Closes #12455 from marcintustin/SPARK-14685. (cherry picked from commit 8028f3a0b4003af15ed44d9ef4727b56f4b10534) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/SparkContext.scala | 5 ++++ .../spark/api/java/JavaSparkContext.scala | 9 ++++-- .../org/apache/spark/SparkContextSuite.scala | 28 +++++++++++++++++++ 3 files changed, 40 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 302dec25c66bd..58618b41920af 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -608,6 +608,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * scheduler pool. User-defined properties may also be set here. These properties are propagated * through to worker tasks and can be accessed there via * [[org.apache.spark.TaskContext#getLocalProperty]]. + * + * These properties are inherited by child threads spawned from this thread. This + * may have unexpected consequences when working with thread pools. The standard java + * implementation of thread pools have worker threads spawn other worker threads. + * As a result, local properties may propagate unpredictably. */ def setLocalProperty(key: String, value: String) { if (value == null) { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index dfd91ae338e89..fb6323413e3ea 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -712,8 +712,13 @@ class JavaSparkContext(val sc: SparkContext) } /** - * Set a local property that affects jobs submitted from this thread, such as the - * Spark fair scheduler pool. + * Set a local property that affects jobs submitted from this thread, and all child + * threads, such as the Spark fair scheduler pool. + * + * These properties are inherited by child threads spawned from this thread. This + * may have unexpected consequences when working with thread pools. The standard java + * implementation of thread pools have worker threads spawn other worker threads. + * As a result, local properties may propagate unpredictably. */ def setLocalProperty(key: String, value: String): Unit = sc.setLocalProperty(key, value) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a759f364fe059..63987084ffc0e 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -323,4 +323,32 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { assert(sc.getConf.getInt("spark.executor.instances", 0) === 6) } } + + + test("localProperties are inherited by spawned threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.setLocalProperty("testProperty", "testValue") + var result = "unset"; + val thread = new Thread() { override def run() = {result = sc.getLocalProperty("testProperty")}} + thread.start() + thread.join() + sc.stop() + assert(result == "testValue") + } + + test("localProperties do not cross-talk between threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + var result = "unset"; + val thread1 = new Thread() { + override def run() = {sc.setLocalProperty("testProperty", "testValue")}} + // testProperty should be unset and thus return null + val thread2 = new Thread() { + override def run() = {result = sc.getLocalProperty("testProperty")}} + thread1.start() + thread1.join() + thread2.start() + thread2.join() + sc.stop() + assert(result == null) + } } From a7e8cfa64de26be2e517e2eda237a9e8a58008c5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 May 2016 21:12:48 -0700 Subject: [PATCH 0021/1470] [SPARK-15079] Support average/count/sum in Long/DoubleAccumulator ## What changes were proposed in this pull request? This patch removes AverageAccumulator and adds the ability to compute average to LongAccumulator and DoubleAccumulator. The patch also improves documentation for the two accumulators. ## How was this patch tested? Added unit tests for this. Author: Reynold Xin Closes #12858 from rxin/SPARK-15079. (cherry picked from commit bb9ab56b960153d374d7e8838f62a18e7e45481e) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/Accumulator.scala | 17 --- .../org/apache/spark/AccumulatorV2.scala | 137 ++++++++++++------ .../scala/org/apache/spark/SparkContext.scala | 22 --- .../org/apache/spark/AccumulatorSuite.scala | 17 +-- .../spark/util/AccumulatorV2Suite.scala | 89 ++++++++++++ 5 files changed, 181 insertions(+), 101 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index e52d36b7b564b..23245043e2465 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -17,9 +17,6 @@ package org.apache.spark -import org.apache.spark.storage.{BlockId, BlockStatus} - - /** * A simpler value of [[Accumulable]] where the result type being accumulated is the same * as the types of elements being merged, i.e. variables that are only "added" to through an @@ -117,18 +114,4 @@ object AccumulatorParam { def addInPlace(t1: String, t2: String): String = t2 def zero(initialValue: String): String = "" } - - // Note: this is expensive as it makes a copy of the list every time the caller adds an item. - // A better way to use this is to first accumulate the values yourself then them all at once. - @deprecated("use AccumulatorV2", "2.0.0") - private[spark] class ListAccumulatorParam[T] extends AccumulatorParam[Seq[T]] { - def addInPlace(t1: Seq[T], t2: Seq[T]): Seq[T] = t1 ++ t2 - def zero(initialValue: Seq[T]): Seq[T] = Seq.empty[T] - } - - // For the internal metric that records what blocks are updated in a particular task - @deprecated("use AccumulatorV2", "2.0.0") - private[spark] object UpdatedBlockStatusesAccumulatorParam - extends ListAccumulatorParam[(BlockId, BlockStatus)] - } diff --git a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala index c65108a55eba4..a6c64fd680573 100644 --- a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/AccumulatorV2.scala @@ -257,23 +257,66 @@ private[spark] object AccumulatorContext { } +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for 64-bit integers. + * + * @since 2.0.0 + */ class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { private[this] var _sum = 0L + private[this] var _count = 0L - override def isZero: Boolean = _sum == 0 + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def isZero: Boolean = _count == 0L override def copyAndReset(): LongAccumulator = new LongAccumulator - override def add(v: jl.Long): Unit = _sum += v + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def add(v: jl.Long): Unit = { + _sum += v + _count += 1 + } + + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Long): Unit = { + _sum += v + _count += 1 + } - def add(v: Long): Unit = _sum += v + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ def sum: Long = _sum + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum.toDouble / _count + override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match { - case o: LongAccumulator => _sum += o.sum - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + case o: LongAccumulator => + _sum += o.sum + _count += o.count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } private[spark] def setValue(newValue: Long): Unit = _sum = newValue @@ -282,66 +325,68 @@ class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { } +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for double precision + * floating numbers. + * + * @since 2.0.0 + */ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[this] var _sum = 0.0 - - override def isZero: Boolean = _sum == 0.0 - - override def copyAndReset(): DoubleAccumulator = new DoubleAccumulator - - override def add(v: jl.Double): Unit = _sum += v - - def add(v: Double): Unit = _sum += v - - def sum: Double = _sum - - override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { - case o: DoubleAccumulator => _sum += o.sum - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - - private[spark] def setValue(newValue: Double): Unit = _sum = newValue - - override def localValue: jl.Double = _sum -} - - -class AverageAccumulator extends AccumulatorV2[jl.Double, jl.Double] { - private[this] var _sum = 0.0 private[this] var _count = 0L - override def isZero: Boolean = _sum == 0.0 && _count == 0 + override def isZero: Boolean = _count == 0L - override def copyAndReset(): AverageAccumulator = new AverageAccumulator + override def copyAndReset(): DoubleAccumulator = new DoubleAccumulator + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ override def add(v: jl.Double): Unit = { _sum += v _count += 1 } - def add(d: Double): Unit = { - _sum += d + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Double): Unit = { + _sum += v _count += 1 } + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count + + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ + def sum: Double = _sum + + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum / _count + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { - case o: AverageAccumulator => + case o: DoubleAccumulator => _sum += o.sum _count += o.count - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - - override def localValue: jl.Double = if (_count == 0) { - Double.NaN - } else { - _sum / _count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - def sum: Double = _sum + private[spark] def setValue(newValue: Double): Unit = _sum = newValue - def count: Long = _count + override def localValue: jl.Double = _sum } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 58618b41920af..e391599336074 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1340,28 +1340,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli acc } - /** - * Create and register an average accumulator, which accumulates double inputs by recording the - * total sum and total count, and produce the output by sum / total. Note that Double.NaN will be - * returned if no input is added. - */ - def averageAccumulator: AverageAccumulator = { - val acc = new AverageAccumulator - register(acc) - acc - } - - /** - * Create and register an average accumulator, which accumulates double inputs by recording the - * total sum and total count, and produce the output by sum / total. Note that Double.NaN will be - * returned if no input is added. - */ - def averageAccumulator(name: String): AverageAccumulator = { - val acc = new AverageAccumulator - register(acc, name) - acc - } - /** * Create and register a list accumulator, which starts with empty list and accumulates inputs * by adding them into the inner list. diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 09eb9c1dbdc62..00200962549e4 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException -import org.apache.spark.AccumulatorParam.{ListAccumulatorParam, StringAccumulatorParam} +import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer @@ -234,21 +234,6 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex acc.merge("kindness") assert(acc.value === "kindness") } - - test("list accumulator param") { - val acc = new Accumulator(Seq.empty[Int], new ListAccumulatorParam[Int], Some("numbers")) - assert(acc.value === Seq.empty[Int]) - acc.add(Seq(1, 2)) - assert(acc.value === Seq(1, 2)) - acc += Seq(3, 4) - assert(acc.value === Seq(1, 2, 3, 4)) - acc ++= Seq(5, 6) - assert(acc.value === Seq(1, 2, 3, 4, 5, 6)) - acc.merge(Seq(7, 8)) - assert(acc.value === Seq(1, 2, 3, 4, 5, 6, 7, 8)) - acc.setValue(Seq(9, 10)) - assert(acc.value === Seq(9, 10)) - } } private[spark] object AccumulatorSuite { diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala new file mode 100644 index 0000000000000..41cdd02492261 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import org.apache.spark.{DoubleAccumulator, LongAccumulator, SparkFunSuite} + +class AccumulatorV2Suite extends SparkFunSuite { + + test("LongAccumulator add/avg/sum/count/isZero") { + val acc = new LongAccumulator + assert(acc.isZero) + assert(acc.count == 0) + assert(acc.sum == 0) + assert(acc.avg.isNaN) + + acc.add(0) + assert(!acc.isZero) + assert(acc.count == 1) + assert(acc.sum == 0) + assert(acc.avg == 0.0) + + acc.add(1) + assert(acc.count == 2) + assert(acc.sum == 1) + assert(acc.avg == 0.5) + + // Also test add using non-specialized add function + acc.add(new java.lang.Long(2)) + assert(acc.count == 3) + assert(acc.sum == 3) + assert(acc.avg == 1.0) + + // Test merging + val acc2 = new LongAccumulator + acc2.add(2) + acc.merge(acc2) + assert(acc.count == 4) + assert(acc.sum == 5) + assert(acc.avg == 1.25) + } + + test("DoubleAccumulator add/avg/sum/count/isZero") { + val acc = new DoubleAccumulator + assert(acc.isZero) + assert(acc.count == 0) + assert(acc.sum == 0.0) + assert(acc.avg.isNaN) + + acc.add(0.0) + assert(!acc.isZero) + assert(acc.count == 1) + assert(acc.sum == 0.0) + assert(acc.avg == 0.0) + + acc.add(1.0) + assert(acc.count == 2) + assert(acc.sum == 1.0) + assert(acc.avg == 0.5) + + // Also test add using non-specialized add function + acc.add(new java.lang.Double(2.0)) + assert(acc.count == 3) + assert(acc.sum == 3.0) + assert(acc.avg == 1.0) + + // Test merging + val acc2 = new DoubleAccumulator + acc2.add(2.0) + acc.merge(acc2) + assert(acc.count == 4) + assert(acc.sum == 5.0) + assert(acc.avg == 1.25) + } +} From 52308103ee9bfb12a505505f6d38f1d09a05208f Mon Sep 17 00:00:00 2001 From: Andrew Ray Date: Mon, 2 May 2016 22:47:32 -0700 Subject: [PATCH 0022/1470] [SPARK-13749][SQL][FOLLOW-UP] Faster pivot implementation for many distinct values with two phase aggregation ## What changes were proposed in this pull request? This is a follow up PR for #11583. It makes 3 lazy vals into just vals and adds unit test coverage. ## How was this patch tested? Existing unit tests and additional unit tests. Author: Andrew Ray Closes #12861 from aray/fast-pivot-follow-up. (cherry picked from commit d8f528ceb61e3c2ac7ac97cd8147dafbb625932f) Signed-off-by: Yin Huai --- .../expressions/aggregate/PivotFirst.scala | 6 +++--- .../apache/spark/sql/DataFramePivotSuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala index 9154e96e34e9c..9ead571c5374a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala @@ -141,12 +141,12 @@ case class PivotFirst( copy(mutableAggBufferOffset = newMutableAggBufferOffset) - override lazy val aggBufferAttributes: Seq[AttributeReference] = + override val aggBufferAttributes: Seq[AttributeReference] = pivotIndex.toList.sortBy(_._2).map(kv => AttributeReference(kv._1.toString, valueDataType)()) - override lazy val aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + override val aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) - override lazy val inputAggBufferAttributes: Seq[AttributeReference] = + override val inputAggBufferAttributes: Seq[AttributeReference] = aggBufferAttributes.map(_.newInstance()) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index b17284aa94d2f..c6d67519b0e94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -180,4 +180,21 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ ) } + test("pivot with datatype not supported by PivotFirst") { + checkAnswer( + complexData.groupBy().pivot("b", Seq(true, false)).agg(max("a")), + Row(Seq(1, 1, 1), Seq(2, 2, 2)) :: Nil + ) + } + + test("pivot with datatype not supported by PivotFirst 2") { + checkAnswer( + courseSales.withColumn("e", expr("array(earnings, 7.0d)")) + .groupBy("year") + .pivot("course", Seq("dotNET", "Java")) + .agg(min($"e")), + Row(2012, Seq(5000.0, 7.0), Seq(20000.0, 7.0)) :: + Row(2013, Seq(48000.0, 7.0), Seq(30000.0, 7.0)) :: Nil + ) + } } From 27efd92e3683f88233ebe755855dac337069246f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 3 May 2016 00:18:10 -0700 Subject: [PATCH 0023/1470] [SPARK-6717][ML] Clear shuffle files after checkpointing in ALS ## What changes were proposed in this pull request? When ALS is run with a checkpoint interval, during the checkpoint materialize the current state and cleanup the previous shuffles (non-blocking). ## How was this patch tested? Existing ALS unit tests, new ALS checkpoint cleanup unit tests added & shuffle files checked after ALS w/checkpointing run. Author: Holden Karau Author: Holden Karau Closes #11919 from holdenk/SPARK-6717-clear-shuffle-files-after-checkpointing-in-ALS. --- .../org/apache/spark/ContextCleaner.scala | 2 +- .../apache/spark/ml/recommendation/ALS.scala | 35 ++- .../spark/ml/recommendation/ALSSuite.scala | 203 ++++++++++++++---- 3 files changed, 195 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index a51338c017071..c895fb32069cf 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -212,7 +212,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } - /** Perform shuffle cleanup, asynchronously. */ + /** Perform shuffle cleanup. */ def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning shuffle " + shuffleId) 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 541923048a3fa..509c944fed74c 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 @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.DefaultFormats import org.json4s.JsonDSL._ -import org.apache.spark.Partitioner +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} @@ -706,13 +706,15 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) // TODO: Generalize PeriodicGraphCheckpointer and use it here. + val deps = itemFactors.dependencies if (shouldCheckpoint(iter)) { - itemFactors.checkpoint() // itemFactors gets materialized in computeFactors. + itemFactors.checkpoint() // itemFactors gets materialized in computeFactors } val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) if (shouldCheckpoint(iter)) { + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -723,8 +725,10 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, userLocalIndexEncoder, solver = solver) if (shouldCheckpoint(iter)) { + val deps = itemFactors.dependencies itemFactors.checkpoint() itemFactors.count() // checkpoint item factors and cut lineage + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -1355,4 +1359,31 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { * satisfies this requirement, we simply use a type alias here. */ private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner + + /** + * Private function to clean up all of the shuffles files from the dependencies and their parents. + */ + private[spark] def cleanShuffleDependencies[T]( + sc: SparkContext, + deps: Seq[Dependency[_]], + blocking: Boolean = false): Unit = { + // If there is no reference tracking we skip clean up. + sc.cleaner.foreach { cleaner => + /** + * Clean the shuffles & all of its parents. + */ + def cleanEagerly(dep: Dependency[_]): Unit = { + if (dep.isInstanceOf[ShuffleDependency[_, _, _]]) { + val shuffleId = dep.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId + cleaner.doCleanupShuffle(shuffleId, blocking) + } + val rdd = dep.rdd + val rddDeps = rdd.dependencies + if (rdd.getStorageLevel == StorageLevel.NONE && rddDeps != null) { + rddDeps.foreach(cleanEagerly) + } + } + deps.foreach(cleanEagerly) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 4c4eb72cd16e7..1704037395780 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -17,15 +17,19 @@ package org.apache.spark.ml.recommendation +import java.io.File import java.util.Random import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.ml.recommendation.ALS._ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} @@ -34,8 +38,9 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { @@ -255,37 +260,7 @@ class ALSSuite rank: Int, noiseStd: Double = 0.0, seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { - // The assumption of the implicit feedback model is that unobserved ratings are more likely to - // be negatives. - val positiveFraction = 0.8 - val negativeFraction = 1.0 - positiveFraction - val trainingFraction = 0.6 - val testFraction = 0.3 - val totalFraction = trainingFraction + testFraction - val random = new Random(seed) - val userFactors = genFactors(numUsers, rank, random) - val itemFactors = genFactors(numItems, rank, random) - val training = ArrayBuffer.empty[Rating[Int]] - val test = ArrayBuffer.empty[Rating[Int]] - for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { - val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) - val threshold = if (rating > 0) positiveFraction else negativeFraction - val observed = random.nextDouble() < threshold - if (observed) { - val x = random.nextDouble() - if (x < totalFraction) { - if (x < trainingFraction) { - val noise = noiseStd * random.nextGaussian() - training += Rating(userId, itemId, rating + noise.toFloat) - } else { - test += Rating(userId, itemId, rating) - } - } - } - } - logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + - s"and ${test.size} for test.") - (sc.parallelize(training, 2), sc.parallelize(test, 2)) + ALSSuite.genImplicitTestData(sc, numUsers, numItems, rank, noiseStd, seed) } /** @@ -304,14 +279,7 @@ class ALSSuite random: Random, a: Float = -1.0f, b: Float = 1.0f): Seq[(Int, Array[Float])] = { - require(size > 0 && size < Int.MaxValue / 3) - require(b > a) - val ids = mutable.Set.empty[Int] - while (ids.size < size) { - ids += random.nextInt() - } - val width = b - a - ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + ALSSuite.genFactors(size, rank, random, a, b) } /** @@ -520,6 +488,79 @@ class ALSSuite } } +class ALSCleanerSuite extends SparkFunSuite { + test("ALS shuffle cleanup standalone") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Test checkpoint and clean parents + val input = sc.parallelize(1 to 1000) + val keyed = input.map(x => (x % 20, 1)) + val shuffled = keyed.reduceByKey(_ + _) + val keysOnly = shuffled.keys + val deps = keysOnly.dependencies + keysOnly.count() + ALS.cleanShuffleDependencies(sc, deps, true) + val resultingFiles = getAllFiles + assert(resultingFiles === Set()) + // Ensure running count again works fine even if we kill the shuffle files. + keysOnly.count() + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } + + test("ALS shuffle cleanup in algorithm") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Generate test data + val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0) + // Implicitly test the cleaning of parents during ALS training + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + val als = new ALS() + .setRank(1) + .setRegParam(1e-5) + .setSeed(0) + .setCheckpointInterval(1) + .setMaxIter(7) + val model = als.fit(training.toDF()) + val resultingFiles = getAllFiles + // We expect the last shuffles files, block ratings, user factors, and item factors to be + // around but no more. + val pattern = "shuffle_(\\d+)_.+\\.data".r + val rddIds = resultingFiles.flatMap { f => + pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } + assert(rddIds.toSet.size === 4) + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } +} + class ALSStorageSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { @@ -591,7 +632,7 @@ private class IntermediateRDDStorageListener extends SparkListener { } -object ALSSuite { +object ALSSuite extends Logging { /** * Mapping from all Params to valid settings which differ from the defaults. @@ -620,4 +661,82 @@ object ALSSuite { "intermediateStorageLevel" -> "MEMORY_ONLY", "finalStorageLevel" -> "MEMORY_AND_DISK_SER" ) + + // Helper functions to generate test data we share between ALS test suites + + /** + * Generates random user/item factors, with i.i.d. values drawn from U(a, b). + * @param size number of users/items + * @param rank number of features + * @param random random number generator + * @param a min value of the support (default: -1) + * @param b max value of the support (default: 1) + * @return a sequence of (ID, factors) pairs + */ + private def genFactors( + size: Int, + rank: Int, + random: Random, + a: Float = -1.0f, + b: Float = 1.0f): Seq[(Int, Array[Float])] = { + require(size > 0 && size < Int.MaxValue / 3) + require(b > a) + val ids = mutable.Set.empty[Int] + while (ids.size < size) { + ids += random.nextInt() + } + val width = b - a + ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + } + + /** + * Generates an implicit feedback dataset for testing ALS. + * + * @param sc SparkContext + * @param numUsers number of users + * @param numItems number of items + * @param rank rank + * @param noiseStd the standard deviation of additive Gaussian noise on training data + * @param seed random seed + * @return (training, test) + */ + def genImplicitTestData( + sc: SparkContext, + numUsers: Int, + numItems: Int, + rank: Int, + noiseStd: Double = 0.0, + seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { + // The assumption of the implicit feedback model is that unobserved ratings are more likely to + // be negatives. + val positiveFraction = 0.8 + val negativeFraction = 1.0 - positiveFraction + val trainingFraction = 0.6 + val testFraction = 0.3 + val totalFraction = trainingFraction + testFraction + val random = new Random(seed) + val userFactors = genFactors(numUsers, rank, random) + val itemFactors = genFactors(numItems, rank, random) + val training = ArrayBuffer.empty[Rating[Int]] + val test = ArrayBuffer.empty[Rating[Int]] + for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { + val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) + val threshold = if (rating > 0) positiveFraction else negativeFraction + val observed = random.nextDouble() < threshold + if (observed) { + val x = random.nextDouble() + if (x < totalFraction) { + if (x < trainingFraction) { + val noise = noiseStd * random.nextGaussian() + training += Rating(userId, itemId, rating + noise.toFloat) + } else { + test += Rating(userId, itemId, rating) + } + } + } + } + logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + + s"and ${test.size} for test.") + (sc.parallelize(training, 2), sc.parallelize(test, 2)) + } } From 07a02e8bb6a2a32508627d4a0cb487b38d595184 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 12:38:21 +0100 Subject: [PATCH 0024/1470] [MINOR][DOCS] Fix type Information in Quick Start and Programming Guide Author: Sandeep Singh Closes #12841 from techaddict/improve_docs_1. (cherry picked from commit dfd9723dd3b3ff5d47a7f04a4330bf33ffe353ac) Signed-off-by: Sean Owen --- docs/programming-guide.md | 2 +- docs/quick-start.md | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index cf6f1d89147f0..d375926a910e6 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -328,7 +328,7 @@ Text file RDDs can be created using `SparkContext`'s `textFile` method. This met {% highlight scala %} scala> val distFile = sc.textFile("data.txt") -distFile: RDD[String] = MappedRDD@1d4cee08 +distFile: org.apache.spark.rdd.RDD[String] = data.txt MapPartitionsRDD[10] at textFile at :26 {% endhighlight %} Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(s => s.length).reduce((a, b) => a + b)`. diff --git a/docs/quick-start.md b/docs/quick-start.md index d481fe0ea6d70..72372a6bc8543 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -33,7 +33,7 @@ Spark's primary abstraction is a distributed collection of items called a Resili {% highlight scala %} scala> val textFile = sc.textFile("README.md") -textFile: spark.RDD[String] = spark.MappedRDD@2ee9b6e3 +textFile: org.apache.spark.rdd.RDD[String] = README.md MapPartitionsRDD[1] at textFile at :25 {% endhighlight %} RDDs have _[actions](programming-guide.html#actions)_, which return values, and _[transformations](programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions: @@ -50,7 +50,7 @@ Now let's use a transformation. We will use the [`filter`](programming-guide.htm {% highlight scala %} scala> val linesWithSpark = textFile.filter(line => line.contains("Spark")) -linesWithSpark: spark.RDD[String] = spark.FilteredRDD@7dd4af09 +linesWithSpark: org.apache.spark.rdd.RDD[String] = MapPartitionsRDD[2] at filter at :27 {% endhighlight %} We can chain together transformations and actions: @@ -123,7 +123,7 @@ One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can i {% highlight scala %} scala> val wordCounts = textFile.flatMap(line => line.split(" ")).map(word => (word, 1)).reduceByKey((a, b) => a + b) -wordCounts: spark.RDD[(String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 +wordCounts: org.apache.spark.rdd.RDD[(String, Int)] = ShuffledRDD[8] at reduceByKey at :28 {% endhighlight %} Here, we combined the [`flatMap`](programming-guide.html#transformations), [`map`](programming-guide.html#transformations), and [`reduceByKey`](programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (String, Int) pairs. To collect the word counts in our shell, we can use the [`collect`](programming-guide.html#actions) action: @@ -181,7 +181,7 @@ Spark also supports pulling data sets into a cluster-wide in-memory cache. This {% highlight scala %} scala> linesWithSpark.cache() -res7: spark.RDD[String] = spark.FilteredRDD@17e51082 +res7: linesWithSpark.type = MapPartitionsRDD[2] at filter at :27 scala> linesWithSpark.count() res8: Long = 19 From 38f6e66afdc92865628238e53ccc156fef976770 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 May 2016 12:39:37 +0100 Subject: [PATCH 0025/1470] [SPARK-15053][BUILD] Fix Java Lint errors on Hive-Thriftserver module ## What changes were proposed in this pull request? This issue fixes or hides 181 Java linter errors introduced by SPARK-14987 which copied hive service code from Hive. We had better clean up these errors before releasing Spark 2.0. - Fix UnusedImports (15 lines), RedundantModifier (14 lines), SeparatorWrap (9 lines), MethodParamPad (6 lines), FileTabCharacter (5 lines), ArrayTypeStyle (3 lines), ModifierOrder (3 lines), RedundantImport (1 line), CommentsIndentation (1 line), UpperEll (1 line), FallThrough (1 line), OneStatementPerLine (1 line), NewlineAtEndOfFile (1 line) errors. - Ignore `LineLength` errors under `hive/service/*` (118 lines). - Ignore `MethodName` error in `PasswdAuthenticationProvider.java` (1 line). - Ignore `NoFinalizer` error in `ThreadWithGarbageCleanup.java` (1 line). ## How was this patch tested? After passing Jenkins building, run `dev/lint-java` manually. ```bash $ dev/lint-java Checkstyle checks passed. ``` Author: Dongjoon Hyun Closes #12831 from dongjoon-hyun/SPARK-15053. (cherry picked from commit a7444570764b0a08b7e908dc7931744f9dbdf3c6) Signed-off-by: Sean Owen --- dev/checkstyle-suppressions.xml | 6 ++++++ .../main/java/org/apache/hive/service/Service.java | 2 +- .../java/org/apache/hive/service/ServiceUtils.java | 2 +- .../org/apache/hive/service/auth/HttpAuthUtils.java | 10 +++++----- .../apache/hive/service/auth/KerberosSaslHelper.java | 2 +- .../java/org/apache/hive/service/auth/SaslQOP.java | 10 ++++++---- .../java/org/apache/hive/service/cli/CLIService.java | 7 +++---- .../apache/hive/service/cli/HiveSQLException.java | 9 +++++---- .../org/apache/hive/service/cli/OperationState.java | 1 + .../org/apache/hive/service/cli/RowBasedSet.java | 4 ++-- .../cli/operation/ClassicTableTypeMapping.java | 2 +- .../service/cli/operation/GetColumnsOperation.java | 2 -- .../service/cli/operation/GetFunctionsOperation.java | 3 +-- .../service/cli/operation/GetSchemasOperation.java | 8 -------- .../cli/operation/GetTableTypesOperation.java | 4 ++-- .../service/cli/operation/GetTablesOperation.java | 4 ++-- .../service/cli/operation/GetTypeInfoOperation.java | 2 +- .../service/cli/operation/LogDivertAppender.java | 12 ++++++------ .../service/cli/operation/MetadataOperation.java | 1 - .../hive/service/cli/operation/SQLOperation.java | 2 +- .../hive/service/cli/operation/TableTypeMapping.java | 6 +++--- .../hive/service/cli/session/HiveSessionBase.java | 3 --- .../hive/service/cli/session/HiveSessionHook.java | 2 +- .../service/cli/session/HiveSessionHookContext.java | 6 +++--- .../service/cli/session/HiveSessionImplwithUGI.java | 3 +-- .../hive/service/cli/session/HiveSessionProxy.java | 2 +- .../hive/service/cli/session/SessionManager.java | 2 +- .../hive/service/cli/thrift/ThriftCLIService.java | 4 ++-- .../hive/service/cli/thrift/ThriftHttpServlet.java | 9 ++++----- .../org/apache/hive/service/server/HiveServer2.java | 4 ++-- 30 files changed, 63 insertions(+), 71 deletions(-) diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml index a1a88ac8cdac5..bfc2e734e517d 100644 --- a/dev/checkstyle-suppressions.xml +++ b/dev/checkstyle-suppressions.xml @@ -36,4 +36,10 @@ files="src/test/java/org/apache/spark/sql/hive/test/Complex.java"/> + + + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java index 2111837cac8d1..b95077cd62186 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java @@ -29,7 +29,7 @@ public interface Service { /** * Service states */ - public enum STATE { + enum STATE { /** Constructed but not initialized */ NOTINITED, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java index e712aaf2348f8..edb5eff9615bf 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java @@ -41,4 +41,4 @@ public static int indexOfDomainMatch(String userName) { } return endIdx; } -} \ No newline at end of file +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java index 3ef55779a6bde..5021528299682 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -56,7 +56,7 @@ public final class HttpAuthUtils { private static final String COOKIE_CLIENT_USER_NAME = "cu"; private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; - private final static Set COOKIE_ATTRIBUTES = + private static final Set COOKIE_ATTRIBUTES = new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); /** @@ -93,10 +93,10 @@ public static String getKerberosServiceTicket(String principal, String host, */ public static String createCookieToken(String clientUserName) { StringBuffer sb = new StringBuffer(); - sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName). - append(COOKIE_ATTR_SEPARATOR); - sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR). - append((new Random(System.currentTimeMillis())).nextLong()); + sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName) + .append(COOKIE_ATTR_SEPARATOR); + sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR) + .append((new Random(System.currentTimeMillis())).nextLong()); return sb.toString(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java index 11d26699fe78d..52eb752f1e026 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -96,7 +96,7 @@ private static class CLIServiceProcessorFactory extends TProcessorFactory { private final ThriftCLIService service; private final Server saslServer; - public CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { + CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { super(null); this.service = service; this.saslServer = saslServer; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java index 479ebf32cec3f..ab3ac6285aa02 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java @@ -25,10 +25,12 @@ * Possible values of SASL quality-of-protection value. */ public enum SaslQOP { - AUTH("auth"), // Authentication only. - AUTH_INT("auth-int"), // Authentication and integrity checking by using signatures. - AUTH_CONF("auth-conf"); // Authentication, integrity and confidentiality checking - // by using signatures and encryption. + // Authentication only. + AUTH("auth"), + // Authentication and integrity checking by using signatures. + AUTH_INT("auth-int"), + // Authentication, integrity and confidentiality checking by using signatures and encryption. + AUTH_CONF("auth-conf"); public final String saslQop; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java index a3af7b2d662b3..791ddcbd2c5b6 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java @@ -481,8 +481,8 @@ public synchronized String getDelegationTokenFromMetaStore(String owner) @Override public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String owner, String renewer) throws HiveSQLException { - String delegationToken = sessionManager.getSession(sessionHandle). - getDelegationToken(authFactory, owner, renewer); + String delegationToken = sessionManager.getSession(sessionHandle) + .getDelegationToken(authFactory, owner, renewer); LOG.info(sessionHandle + ": getDelegationToken()"); return delegationToken; } @@ -490,8 +490,7 @@ public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory au @Override public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { - sessionManager.getSession(sessionHandle). - cancelDelegationToken(authFactory, tokenStr); + sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); LOG.info(sessionHandle + ": cancelDelegationToken()"); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java index 1334dde66375b..86e57fbf31fe0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java @@ -111,7 +111,7 @@ public HiveSQLException(TStatus status) { /** * Converts current object to a {@link TStatus} object - * @return a {@link TStatus} object + * @return a {@link TStatus} object */ public TStatus toTStatus() { // TODO: convert sqlState, etc. @@ -125,8 +125,8 @@ public TStatus toTStatus() { /** * Converts the specified {@link Exception} object into a {@link TStatus} object - * @param e a {@link Exception} object - * @return a {@link TStatus} object + * @param e a {@link Exception} object + * @return a {@link TStatus} object */ public static TStatus toTStatus(Exception e) { if (e instanceof HiveSQLException) { @@ -155,7 +155,8 @@ private static List toString(Throwable cause, StackTraceElement[] parent if (parent != null) { int n = parent.length - 1; while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { - m--; n--; + m--; + n--; } } List detail = enroll(cause, trace, m); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java index 51ffb40369b24..1165180118413 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java @@ -85,6 +85,7 @@ public static void validateTransition(OperationState oldState, if (OperationState.CLOSED.equals(newState)) { return; } + break; default: // fall-through } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java index a0ee2109dc540..7452137f077db 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java @@ -130,8 +130,8 @@ public void remove() { } private static class RemovableList extends ArrayList { - public RemovableList() { super(); } - public RemovableList(List rows) { super(rows); } + RemovableList() { super(); } + RemovableList(List rows) { super(rows); } @Override public void removeRange(int fromIndex, int toIndex) { super.removeRange(fromIndex, toIndex); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java index 87ac39b05186a..05a6bf938404b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java @@ -42,7 +42,7 @@ public enum ClassicTableTypes { private final Map hiveToClientMap = new HashMap(); private final Map clientToHiveMap = new HashMap(); - public ClassicTableTypeMapping () { + public ClassicTableTypeMapping() { hiveToClientMap.put(TableType.MANAGED_TABLE.toString(), ClassicTableTypes.TABLE.toString()); hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(), diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java index 309f10f640f96..5efb0759383ac 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -27,10 +27,8 @@ import java.util.Map.Entry; import java.util.regex.Pattern; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java index 6df1e8a227f3e..5273c386b83d4 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -23,7 +23,6 @@ import java.util.Set; import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; @@ -103,7 +102,7 @@ public void runInternal() throws HiveSQLException { .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); for (String functionName : functionNames) { FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); - Object rowData[] = new Object[] { + Object[] rowData = new Object[] { null, // FUNCTION_CAT null, // FUNCTION_SCHEM functionInfo.getDisplayName(), // FUNCTION_NAME diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index e56686abb7c5a..d6f6280f1c398 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -18,16 +18,8 @@ package org.apache.hive.service.cli.operation; -import java.util.ArrayList; -import java.util.List; - import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.service.cli.FetchOrientation; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.OperationState; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java index a09b39a4e0855..3ae012a72764f 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -44,8 +44,8 @@ public class GetTableTypesOperation extends MetadataOperation { protected GetTableTypesOperation(HiveSession parentSession) { super(parentSession, OperationType.GET_TABLE_TYPES); - String tableMappingStr = getParentSession().getHiveConf(). - getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); tableTypeMapping = TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java index 0e2fdc657c4fa..1a7ca79163d71 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -64,8 +64,8 @@ protected GetTablesOperation(HiveSession parentSession, this.catalogName = catalogName; this.schemaName = schemaName; this.tableName = tableName; - String tableMappingStr = getParentSession().getHiveConf(). - getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); tableTypeMapping = TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); if (tableTypes != null) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java index 2a0fec27715da..0f72071d7e7d1 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -35,7 +35,7 @@ */ public class GetTypeInfoOperation extends MetadataOperation { - private final static TableSchema RESULT_SET_SCHEMA = new TableSchema() + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, "Type name") .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java index 70340bd13cbc1..cb804318ace9c 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java @@ -60,15 +60,15 @@ private static class NameFilter extends Filter { /* Patterns that are excluded in verbose logging level. * Filter out messages coming from log processing classes, or we'll run an infinite loop. */ - private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|"). - join(new String[] {LOG.getName(), OperationLog.class.getName(), + private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {LOG.getName(), OperationLog.class.getName(), OperationManager.class.getName()})); /* Patterns that are included in execution logging level. * In execution mode, show only select logger messages. */ - private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|"). - join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", + private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", "org.apache.hadoop.mapreduce.Job", "SessionState", Task.class.getName(), "org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor"})); @@ -88,7 +88,7 @@ private void setCurrentNamePattern(OperationLog.LoggingLevel mode) { } } - public NameFilter( + NameFilter( OperationLog.LoggingLevel loggingMode, OperationManager op) { this.operationManager = op; this.loggingMode = loggingMode; @@ -131,7 +131,7 @@ public int decide(LoggingEvent ev) { /** This is where the log message will go to */ private final CharArrayWriter writer = new CharArrayWriter(); - private void setLayout (boolean isVerbose, Layout lo) { + private void setLayout(boolean isVerbose, Layout lo) { if (isVerbose) { if (lo == null) { lo = CLIServiceUtils.verboseLayout; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java index 4595ef56fcee4..6c819876a556d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -18,7 +18,6 @@ package org.apache.hive.service.cli.operation; -import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hive.conf.HiveConf; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java index 33ee16b80beb2..a35405484a765 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -326,7 +326,7 @@ public TableSchema getResultSetSchema() throws HiveSQLException { return resultSchema; } - private transient final List convey = new ArrayList(); + private final transient List convey = new ArrayList(); @Override public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java index 3a8a07f44f20d..e392c459cf586 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java @@ -27,18 +27,18 @@ public interface TableTypeMapping { * @param clientTypeName * @return */ - public String mapToHiveType (String clientTypeName); + String mapToHiveType(String clientTypeName); /** * Map hive's table type name to client's table type * @param clientTypeName * @return */ - public String mapToClientType (String hiveTypeName); + String mapToClientType(String hiveTypeName); /** * Get all the table types of this mapping * @return */ - public Set getTableTypeNames(); + Set getTableTypeNames(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java index 9b04d679df1c7..b72c18b2b2135 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java @@ -18,8 +18,6 @@ package org.apache.hive.service.cli.session; -import java.util.Map; - import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.service.cli.SessionHandle; @@ -27,7 +25,6 @@ import org.apache.hive.service.cli.thrift.TProtocolVersion; import java.io.File; -import java.util.Map; /** * Methods that don't need to be executed under a doAs diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java index 06388cc795b90..7e4f2b0478a52 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java @@ -33,5 +33,5 @@ public interface HiveSessionHook extends Hook { * @param sessionHookContext context * @throws HiveSQLException */ - public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; + void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java index 156c8147f9d93..0a10dba8b479d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java @@ -30,17 +30,17 @@ public interface HiveSessionHookContext { * Retrieve session conf * @return */ - public HiveConf getSessionConf(); + HiveConf getSessionConf(); /** * The get the username starting the session * @return */ - public String getSessionUser(); + String getSessionUser(); /** * Retrieve handle for the session * @return */ - public String getSessionHandle(); + String getSessionHandle(); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java index a29e5d1d81c17..762dbb2faadec 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.shims.Utils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hive.service.auth.HiveAuthFactory; @@ -83,7 +82,7 @@ public UserGroupInformation getSessionUgi() { return this.sessionUgi; } - public String getDelegationToken () { + public String getDelegationToken() { return this.delegationTokenStr; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java index 5b10521febfc2..8e539512f7410 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java @@ -57,7 +57,7 @@ public Object invoke(Object arg0, final Method method, final Object[] args) return invoke(method, args); } return ugi.doAs( - new PrivilegedExceptionAction () { + new PrivilegedExceptionAction() { @Override public Object run() throws HiveSQLException { return invoke(method, args); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index e31570bdfba57..c6d177798f558 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -151,7 +151,7 @@ public synchronized void start() { } private void startTimeoutChecker() { - final long interval = Math.max(checkInterval, 3000l); // minimum 3 seconds + final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds Runnable timeoutChecker = new Runnable() { @Override public void run() { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 5a0f1c83c70f3..866beb19f5f91 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -722,8 +722,8 @@ private String getProxyUser(String realUser, Map sessionConf, } // If there's no authentication, then directly substitute the user - if (HiveAuthFactory.AuthTypes.NONE.toString(). - equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { + if (HiveAuthFactory.AuthTypes.NONE.toString() + .equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { return proxyUser; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java index 56c8cb6e5459d..e15d2d0566d2b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.security.PrivilegedExceptionAction; -import java.util.Arrays; import java.util.Map; import java.util.Random; import java.util.Set; @@ -241,9 +240,9 @@ private String getClientNameFromCookie(Cookie[] cookies) { * Each cookie is of the format [key]=[value] */ private String toCookieStr(Cookie[] cookies) { - String cookieStr = ""; + String cookieStr = ""; - for (Cookie c : cookies) { + for (Cookie c : cookies) { cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; } return cookieStr; @@ -458,7 +457,7 @@ private String getPrincipalWithoutRealmAndHost(String fullPrincipal) private String getUsername(HttpServletRequest request, String authType) throws HttpAuthenticationException { - String creds[] = getAuthHeaderTokens(request, authType); + String[] creds = getAuthHeaderTokens(request, authType); // Username must be present if (creds[0] == null || creds[0].isEmpty()) { throw new HttpAuthenticationException("Authorization header received " + @@ -469,7 +468,7 @@ private String getUsername(HttpServletRequest request, String authType) private String getPassword(HttpServletRequest request, String authType) throws HttpAuthenticationException { - String creds[] = getAuthHeaderTokens(request, authType); + String[] creds = getAuthHeaderTokens(request, authType); // Password must be present if (creds[1] == null || creds[1].isEmpty()) { throw new HttpAuthenticationException("Authorization header received " + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java index 1500e537cef5b..9bf96cff572e8 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java @@ -236,8 +236,8 @@ ServerOptionsExecutor getServerOptionsExecutor() { /** * The executor interface for running the appropriate HiveServer2 command based on parsed options */ - static interface ServerOptionsExecutor { - public void execute(); + interface ServerOptionsExecutor { + void execute(); } /** From b802979ad42fd58ed1d8c6e23629169bc2891cbe Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 May 2016 19:45:12 +0800 Subject: [PATCH 0026/1470] [SPARK-15081] Move AccumulatorV2 and subclasses into util package ## What changes were proposed in this pull request? This patch moves AccumulatorV2 and subclasses into util package. ## How was this patch tested? Updated relevant tests. Author: Reynold Xin Closes #12863 from rxin/SPARK-15081. (cherry picked from commit d557a5e01e8f819d3bd9e6e43d2df733f390d764) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/Accumulable.scala | 1 + .../org/apache/spark/ContextCleaner.scala | 2 +- .../org/apache/spark/HeartbeatReceiver.scala | 2 +- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../scala/org/apache/spark/TaskEndReason.scala | 2 +- .../apache/spark/executor/InputMetrics.scala | 2 +- .../apache/spark/executor/OutputMetrics.scala | 2 +- .../spark/executor/ShuffleReadMetrics.scala | 2 +- .../spark/executor/ShuffleWriteMetrics.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 1 + .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../apache/spark/scheduler/TaskResult.scala | 4 ++-- .../spark/scheduler/TaskResultGetter.scala | 2 +- .../apache/spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../spark/{ => util}/AccumulatorV2.scala | 18 +++++++++--------- .../org/apache/spark/AccumulatorSuite.scala | 1 + .../spark/InternalAccumulatorSuite.scala | 1 + .../scala/org/apache/spark/SparkFunSuite.scala | 1 + .../spark/executor/TaskMetricsSuite.scala | 1 + .../spark/scheduler/DAGSchedulerSuite.scala | 3 +-- .../ExternalClusterManagerSuite.scala | 3 ++- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../apache/spark/util/AccumulatorV2Suite.scala | 2 +- .../columnar/InMemoryTableScanExec.scala | 4 +++- .../sql/execution/metric/SQLMetrics.scala | 4 ++-- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 ++ 30 files changed, 44 insertions(+), 34 deletions(-) rename core/src/main/scala/org/apache/spark/{ => util}/AccumulatorV2.scala (95%) diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index 799c7e4fd5006..5c6761eb764b5 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, LegacyAccumulatorWrapper} /** diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index c895fb32069cf..5678d790e9e76 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, ThreadUtils, Utils} /** * Classes that represent cleaning tasks. diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 29018c75b9224..73495a8d7d7fd 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} +import org.apache.spark.util._ /** * A heartbeat from executors to the driver. This is a shared message used by several internal diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 1a8f8cf11c1be..27abccf5ac2a9 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.Source -import org.apache.spark.util.{TaskCompletionListener, TaskFailureListener} +import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener} object TaskContext { diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index ef333e397f64e..42690844f9610 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} // ============================================================================================== // NOTE: new task end reasons MUST be accompanied with serialization logic in util.JsonProtocol! diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 6f7160ac0d3a3..3d15f3a0396e1 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index db3924cb6937e..dada9697c1cf9 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index fa962108c3064..f7a991770d402 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 0e70a4f522849..ada2e1bc08593 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import org.apache.spark.LongAccumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator /** diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 56d034fd033e6..085aa7fbd63dd 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 0a2c2dc03937b..8c761124824ae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,7 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.CallSite +import org.apache.spark.util.{AccumulatorV2, CallSite} /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 362f8e51ce056..95bcc7bc9653a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -28,7 +28,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} +import org.apache.spark.util.{AccumulatorV2, ByteBufferInputStream, ByteBufferOutputStream, Utils} /** * A unit of execution. We have two kinds of Task's in Spark: diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 69ce00f30d9a4..80f2bf41224b5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -22,9 +22,9 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{AccumulatorV2, SparkEnv} +import org.apache.spark.SparkEnv import org.apache.spark.storage.BlockId -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} // Task result. Also contains updates to accumulator variables. private[spark] sealed trait TaskResult[T] diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index b438c285fdf1f..685ef55c66876 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -27,7 +27,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.Logging import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{LongAccumulator, ThreadUtils, Utils} /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 9881a1018cf57..cd13eebe74a99 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,9 +17,9 @@ package org.apache.spark.scheduler -import org.apache.spark.AccumulatorV2 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 /** * Low-level task scheduler interface, currently implemented exclusively by 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 666b6365585c7..393680f4c1542 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. 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 b724050f5bb89..cd634bbf6fe74 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -32,7 +32,7 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} /** * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of diff --git a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/AccumulatorV2.scala rename to core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index a6c64fd680573..0e280f6f6a290 100644 --- a/core/src/main/scala/org/apache/spark/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import java.{lang => jl} import java.io.ObjectInputStream import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong +import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext} import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.util.Utils private[spark] case class AccumulatorMetadata( @@ -204,8 +204,8 @@ private[spark] object AccumulatorContext { private[this] val nextId = new AtomicLong(0L) /** - * Returns a globally unique ID for a new [[Accumulator]]. - * Note: Once you copy the [[Accumulator]] the ID is no longer unique. + * Returns a globally unique ID for a new [[AccumulatorV2]]. + * Note: Once you copy the [[AccumulatorV2]] the ID is no longer unique. */ def newId(): Long = nextId.getAndIncrement @@ -213,14 +213,14 @@ private[spark] object AccumulatorContext { def numAccums: Int = originals.size /** - * Registers an [[Accumulator]] created on the driver such that it can be used on the executors. + * Registers an [[AccumulatorV2]] created on the driver such that it can be used on the executors. * * All accumulators registered here can later be used as a container for accumulating partial * values across multiple tasks. This is what [[org.apache.spark.scheduler.DAGScheduler]] does. * Note: if an accumulator is registered here, it should also be registered with the active * context cleaner for cleanup so as to avoid memory leaks. * - * If an [[Accumulator]] with the same ID was already registered, this does nothing instead + * If an [[AccumulatorV2]] with the same ID was already registered, this does nothing instead * of overwriting it. We will never register same accumulator twice, this is just a sanity check. */ def register(a: AccumulatorV2[_, _]): Unit = { @@ -228,14 +228,14 @@ private[spark] object AccumulatorContext { } /** - * Unregisters the [[Accumulator]] with the given ID, if any. + * Unregisters the [[AccumulatorV2]] with the given ID, if any. */ def remove(id: Long): Unit = { originals.remove(id) } /** - * Returns the [[Accumulator]] registered with the given ID, if any. + * Returns the [[AccumulatorV2]] registered with the given ID, if any. */ def get(id: Long): Option[AccumulatorV2[_, _]] = { Option(originals.get(id)).map { ref => @@ -249,7 +249,7 @@ private[spark] object AccumulatorContext { } /** - * Clears all registered [[Accumulator]]s. For testing only. + * Clears all registered [[AccumulatorV2]]s. For testing only. */ def clear(): Unit = { originals.clear() diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 00200962549e4..cade67b1d2bae 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 25977a4660244..840f55ce2f6e5 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 4aae2c9b4a8e4..0081bca63959c 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -21,6 +21,7 @@ package org.apache.spark import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} import org.apache.spark.internal.Logging +import org.apache.spark.util.AccumulatorContext /** * Base abstract class for all unit tests in Spark for handling common functionality. diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 27a1e7bb35bad..eae26fa742a23 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.Assertions import org.apache.spark._ import org.apache.spark.storage.{BlockStatus, StorageLevel, TestBlockId} +import org.apache.spark.util.AccumulatorV2 class TaskMetricsSuite extends SparkFunSuite { 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 5a5c3a0cd1935..844c780a3fdd2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -28,11 +28,10 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.{CallSite, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 72ac848f12b46..59c1b359a780a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.scheduler -import org.apache.spark.{AccumulatorV2, LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { test("launch of backend and scheduler") { 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 122a3ecb4954b..9b7b945bf3677 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{AccumulatorV2, ManualClock} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala index 41cdd02492261..815b134884016 100644 --- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import org.apache.spark.{DoubleAccumulator, LongAccumulator, SparkFunSuite} +import org.apache.spark.SparkFunSuite class AccumulatorV2Suite extends SparkFunSuite { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 94b87a5812e4c..a36071a97bcd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Accumulable, Accumulator, AccumulatorContext} +import org.apache.spark.{Accumulable, Accumulator} import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -33,6 +33,8 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.AccumulatorContext + private[sql] object InMemoryRelation { def apply( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 40c00ee1881ff..0f68aaaee1f21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.metric import java.text.NumberFormat -import org.apache.spark.{AccumulatorV2, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} class SQLMetric(val metricType: String, initValue: Long = 0L) extends AccumulatorV2[Long, Long] { 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 12dbed89d561b..5ef20267f874a 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,13 +22,13 @@ import scala.language.postfixOps import org.scalatest.concurrent.Eventually._ -import org.apache.spark.AccumulatorContext import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.storage.{RDDBlockId, StorageLevel} +import org.apache.spark.util.AccumulatorContext private case class BigData(s: String) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 8572ed16aa261..964787015a877 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -30,6 +30,8 @@ import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.ui.SparkUI +import org.apache.spark.util.{AccumulatorMetadata, LongAccumulator} + class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ From f03bf7eacb834d2eaeba197ccf704bb721f0b4af Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 3 May 2016 13:13:35 +0100 Subject: [PATCH 0027/1470] [SPARK-14897][CORE] Upgrade Jetty to latest version of 8 ## What changes were proposed in this pull request? Update Jetty 8.1 to the latest 2016/02 release, from a 2013/10 release, for security and bug fixes. This does not resolve the JIRA necessarily, as it's still worth considering an update to 9.3. ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #12842 from srowen/SPARK-14897. (cherry picked from commit 57ac7c182465e1653e74a8ad6c826b2cf56a0ad8) Signed-off-by: Sean Owen --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 66f1d8ea902ac..1dce91b7dfe4f 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ 10.10.1.1 1.7.0 1.6.0 - 8.1.14.v20131031 + 8.1.19.v20160209 3.0.0.v201112011016 0.8.0 2.4.0 From 932e1b5b2ad19153b5a5aa2255837569406486dd Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 3 May 2016 16:46:13 +0200 Subject: [PATCH 0028/1470] [SPARK-14971][ML][PYSPARK] PySpark ML Params setter code clean up ## What changes were proposed in this pull request? PySpark ML Params setter code clean up. For examples, ```setInputCol``` can be simplified from ``` self._set(inputCol=value) return self ``` to: ``` return self._set(inputCol=value) ``` This is a pretty big sweeps, and we cleaned wherever possible. ## How was this patch tested? Exist unit tests. Author: Yanbo Liang Closes #12749 from yanboliang/spark-14971. (cherry picked from commit d26f7cb0121767da678bbbbf3a0e31c63d5e3159) Signed-off-by: Nick Pentreath --- python/pyspark/ml/classification.py | 21 ++-- python/pyspark/ml/clustering.py | 18 +-- python/pyspark/ml/evaluation.py | 9 +- python/pyspark/ml/feature.py | 111 ++++++------------ .../ml/param/_shared_params_code_gen.py | 3 +- python/pyspark/ml/param/shared.py | 87 +++++--------- python/pyspark/ml/pipeline.py | 3 +- python/pyspark/ml/recommendation.py | 35 ++---- python/pyspark/ml/regression.py | 36 ++---- python/pyspark/ml/tuning.py | 6 +- 10 files changed, 110 insertions(+), 219 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4331f73b73253..f032963334469 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -464,8 +464,7 @@ def setImpurity(self, value): """ Sets the value of :py:attr:`impurity`. """ - self._set(impurity=value) - return self + return self._set(impurity=value) @since("1.6.0") def getImpurity(self): @@ -826,8 +825,7 @@ def setLossType(self, value): """ Sets the value of :py:attr:`lossType`. """ - self._set(lossType=value) - return self + return self._set(lossType=value) @since("1.4.0") def getLossType(self): @@ -956,8 +954,7 @@ def setSmoothing(self, value): """ Sets the value of :py:attr:`smoothing`. """ - self._set(smoothing=value) - return self + return self._set(smoothing=value) @since("1.5.0") def getSmoothing(self): @@ -971,8 +968,7 @@ def setModelType(self, value): """ Sets the value of :py:attr:`modelType`. """ - self._set(modelType=value) - return self + return self._set(modelType=value) @since("1.5.0") def getModelType(self): @@ -1098,8 +1094,7 @@ def setLayers(self, value): """ Sets the value of :py:attr:`layers`. """ - self._set(layers=value) - return self + return self._set(layers=value) @since("1.6.0") def getLayers(self): @@ -1113,8 +1108,7 @@ def setBlockSize(self, value): """ Sets the value of :py:attr:`blockSize`. """ - self._set(blockSize=value) - return self + return self._set(blockSize=value) @since("1.6.0") def getBlockSize(self): @@ -1162,8 +1156,7 @@ def setClassifier(self, value): .. note:: Only LogisticRegression and NaiveBayes are supported now. """ - self._set(classifier=value) - return self + return self._set(classifier=value) @since("2.0.0") def getClassifier(self): diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 50ebf4fde1cf5..b90daf16e2f71 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -156,8 +156,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("2.0.0") def getK(self): @@ -272,8 +271,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("1.5.0") def getK(self): @@ -287,8 +285,7 @@ def setInitMode(self, value): """ Sets the value of :py:attr:`initMode`. """ - self._set(initMode=value) - return self + return self._set(initMode=value) @since("1.5.0") def getInitMode(self): @@ -302,8 +299,7 @@ def setInitSteps(self, value): """ Sets the value of :py:attr:`initSteps`. """ - self._set(initSteps=value) - return self + return self._set(initSteps=value) @since("1.5.0") def getInitSteps(self): @@ -422,8 +418,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("2.0.0") def getK(self): @@ -437,8 +432,7 @@ def setMinDivisibleClusterSize(self, value): """ Sets the value of :py:attr:`minDivisibleClusterSize`. """ - self._set(minDivisibleClusterSize=value) - return self + return self._set(minDivisibleClusterSize=value) @since("2.0.0") def getMinDivisibleClusterSize(self): diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 455795f9a083a..2a41678741ea1 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -147,8 +147,7 @@ def setMetricName(self, value): """ Sets the value of :py:attr:`metricName`. """ - self._set(metricName=value) - return self + return self._set(metricName=value) @since("1.4.0") def getMetricName(self): @@ -217,8 +216,7 @@ def setMetricName(self, value): """ Sets the value of :py:attr:`metricName`. """ - self._set(metricName=value) - return self + return self._set(metricName=value) @since("1.4.0") def getMetricName(self): @@ -284,8 +282,7 @@ def setMetricName(self, value): """ Sets the value of :py:attr:`metricName`. """ - self._set(metricName=value) - return self + return self._set(metricName=value) @since("1.5.0") def getMetricName(self): diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index b95d288198b53..f21e3062ef4f1 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -114,8 +114,7 @@ def setThreshold(self, value): """ Sets the value of :py:attr:`threshold`. """ - self._set(threshold=value) - return self + return self._set(threshold=value) @since("1.4.0") def getThreshold(self): @@ -190,8 +189,7 @@ def setSplits(self, value): """ Sets the value of :py:attr:`splits`. """ - self._set(splits=value) - return self + return self._set(splits=value) @since("1.4.0") def getSplits(self): @@ -295,8 +293,7 @@ def setMinTF(self, value): """ Sets the value of :py:attr:`minTF`. """ - self._set(minTF=value) - return self + return self._set(minTF=value) @since("1.6.0") def getMinTF(self): @@ -310,8 +307,7 @@ def setMinDF(self, value): """ Sets the value of :py:attr:`minDF`. """ - self._set(minDF=value) - return self + return self._set(minDF=value) @since("1.6.0") def getMinDF(self): @@ -325,8 +321,7 @@ def setVocabSize(self, value): """ Sets the value of :py:attr:`vocabSize`. """ - self._set(vocabSize=value) - return self + return self._set(vocabSize=value) @since("1.6.0") def getVocabSize(self): @@ -340,8 +335,7 @@ def setBinary(self, value): """ Sets the value of :py:attr:`binary`. """ - self._set(binary=value) - return self + return self._set(binary=value) @since("2.0.0") def getBinary(self): @@ -433,8 +427,7 @@ def setInverse(self, value): """ Sets the value of :py:attr:`inverse`. """ - self._set(inverse=value) - return self + return self._set(inverse=value) @since("1.6.0") def getInverse(self): @@ -500,8 +493,7 @@ def setScalingVec(self, value): """ Sets the value of :py:attr:`scalingVec`. """ - self._set(scalingVec=value) - return self + return self._set(scalingVec=value) @since("1.5.0") def getScalingVec(self): @@ -573,8 +565,7 @@ def setBinary(self, value): """ Sets the value of :py:attr:`binary`. """ - self._set(binary=value) - return self + return self._set(binary=value) @since("2.0.0") def getBinary(self): @@ -647,8 +638,7 @@ def setMinDocFreq(self, value): """ Sets the value of :py:attr:`minDocFreq`. """ - self._set(minDocFreq=value) - return self + return self._set(minDocFreq=value) @since("1.4.0") def getMinDocFreq(self): @@ -832,8 +822,7 @@ def setMin(self, value): """ Sets the value of :py:attr:`min`. """ - self._set(min=value) - return self + return self._set(min=value) @since("1.6.0") def getMin(self): @@ -847,8 +836,7 @@ def setMax(self, value): """ Sets the value of :py:attr:`max`. """ - self._set(max=value) - return self + return self._set(max=value) @since("1.6.0") def getMax(self): @@ -956,8 +944,7 @@ def setN(self, value): """ Sets the value of :py:attr:`n`. """ - self._set(n=value) - return self + return self._set(n=value) @since("1.5.0") def getN(self): @@ -1023,8 +1010,7 @@ def setP(self, value): """ Sets the value of :py:attr:`p`. """ - self._set(p=value) - return self + return self._set(p=value) @since("1.4.0") def getP(self): @@ -1106,8 +1092,7 @@ def setDropLast(self, value): """ Sets the value of :py:attr:`dropLast`. """ - self._set(dropLast=value) - return self + return self._set(dropLast=value) @since("1.4.0") def getDropLast(self): @@ -1175,8 +1160,7 @@ def setDegree(self, value): """ Sets the value of :py:attr:`degree`. """ - self._set(degree=value) - return self + return self._set(degree=value) @since("1.4.0") def getDegree(self): @@ -1257,8 +1241,7 @@ def setNumBuckets(self, value): """ Sets the value of :py:attr:`numBuckets`. """ - self._set(numBuckets=value) - return self + return self._set(numBuckets=value) @since("2.0.0") def getNumBuckets(self): @@ -1355,8 +1338,7 @@ def setMinTokenLength(self, value): """ Sets the value of :py:attr:`minTokenLength`. """ - self._set(minTokenLength=value) - return self + return self._set(minTokenLength=value) @since("1.4.0") def getMinTokenLength(self): @@ -1370,8 +1352,7 @@ def setGaps(self, value): """ Sets the value of :py:attr:`gaps`. """ - self._set(gaps=value) - return self + return self._set(gaps=value) @since("1.4.0") def getGaps(self): @@ -1385,8 +1366,7 @@ def setPattern(self, value): """ Sets the value of :py:attr:`pattern`. """ - self._set(pattern=value) - return self + return self._set(pattern=value) @since("1.4.0") def getPattern(self): @@ -1400,8 +1380,7 @@ def setToLowercase(self, value): """ Sets the value of :py:attr:`toLowercase`. """ - self._set(toLowercase=value) - return self + return self._set(toLowercase=value) @since("2.0.0") def getToLowercase(self): @@ -1462,8 +1441,7 @@ def setStatement(self, value): """ Sets the value of :py:attr:`statement`. """ - self._set(statement=value) - return self + return self._set(statement=value) @since("1.6.0") def getStatement(self): @@ -1540,8 +1518,7 @@ def setWithMean(self, value): """ Sets the value of :py:attr:`withMean`. """ - self._set(withMean=value) - return self + return self._set(withMean=value) @since("1.4.0") def getWithMean(self): @@ -1555,8 +1532,7 @@ def setWithStd(self, value): """ Sets the value of :py:attr:`withStd`. """ - self._set(withStd=value) - return self + return self._set(withStd=value) @since("1.4.0") def getWithStd(self): @@ -1724,8 +1700,7 @@ def setLabels(self, value): """ Sets the value of :py:attr:`labels`. """ - self._set(labels=value) - return self + return self._set(labels=value) @since("1.6.0") def getLabels(self): @@ -1795,8 +1770,7 @@ def setStopWords(self, value): """ Specify the stopwords to be filtered. """ - self._set(stopWords=value) - return self + return self._set(stopWords=value) @since("1.6.0") def getStopWords(self): @@ -1810,8 +1784,7 @@ def setCaseSensitive(self, value): """ Set whether to do a case sensitive comparison over the stop words """ - self._set(caseSensitive=value) - return self + return self._set(caseSensitive=value) @since("1.6.0") def getCaseSensitive(self): @@ -2027,8 +2000,7 @@ def setMaxCategories(self, value): """ Sets the value of :py:attr:`maxCategories`. """ - self._set(maxCategories=value) - return self + return self._set(maxCategories=value) @since("1.4.0") def getMaxCategories(self): @@ -2137,8 +2109,7 @@ def setIndices(self, value): """ Sets the value of :py:attr:`indices`. """ - self._set(indices=value) - return self + return self._set(indices=value) @since("1.6.0") def getIndices(self): @@ -2152,8 +2123,7 @@ def setNames(self, value): """ Sets the value of :py:attr:`names`. """ - self._set(names=value) - return self + return self._set(names=value) @since("1.6.0") def getNames(self): @@ -2261,8 +2231,7 @@ def setVectorSize(self, value): """ Sets the value of :py:attr:`vectorSize`. """ - self._set(vectorSize=value) - return self + return self._set(vectorSize=value) @since("1.4.0") def getVectorSize(self): @@ -2276,8 +2245,7 @@ def setNumPartitions(self, value): """ Sets the value of :py:attr:`numPartitions`. """ - self._set(numPartitions=value) - return self + return self._set(numPartitions=value) @since("1.4.0") def getNumPartitions(self): @@ -2291,8 +2259,7 @@ def setMinCount(self, value): """ Sets the value of :py:attr:`minCount`. """ - self._set(minCount=value) - return self + return self._set(minCount=value) @since("1.4.0") def getMinCount(self): @@ -2306,8 +2273,7 @@ def setWindowSize(self, value): """ Sets the value of :py:attr:`windowSize`. """ - self._set(windowSize=value) - return self + return self._set(windowSize=value) @since("2.0.0") def getWindowSize(self): @@ -2412,8 +2378,7 @@ def setK(self, value): """ Sets the value of :py:attr:`k`. """ - self._set(k=value) - return self + return self._set(k=value) @since("1.5.0") def getK(self): @@ -2545,8 +2510,7 @@ def setFormula(self, value): """ Sets the value of :py:attr:`formula`. """ - self._set(formula=value) - return self + return self._set(formula=value) @since("1.5.0") def getFormula(self): @@ -2638,8 +2602,7 @@ def setNumTopFeatures(self, value): """ Sets the value of :py:attr:`numTopFeatures`. """ - self._set(numTopFeatures=value) - return self + return self._set(numTopFeatures=value) @since("2.0.0") def getNumTopFeatures(self): diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index a2acf956bc2ad..12f14534f4b88 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -85,8 +85,7 @@ def set$Name(self, value): """ Sets the value of :py:attr:`$name`. """ - self._set($name=value) - return self + return self._set($name=value) def get$Name(self): """ diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 538c0b718ad9b..9ed63e48ae40d 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -34,8 +34,7 @@ def setMaxIter(self, value): """ Sets the value of :py:attr:`maxIter`. """ - self._set(maxIter=value) - return self + return self._set(maxIter=value) def getMaxIter(self): """ @@ -58,8 +57,7 @@ def setRegParam(self, value): """ Sets the value of :py:attr:`regParam`. """ - self._set(regParam=value) - return self + return self._set(regParam=value) def getRegParam(self): """ @@ -83,8 +81,7 @@ def setFeaturesCol(self, value): """ Sets the value of :py:attr:`featuresCol`. """ - self._set(featuresCol=value) - return self + return self._set(featuresCol=value) def getFeaturesCol(self): """ @@ -108,8 +105,7 @@ def setLabelCol(self, value): """ Sets the value of :py:attr:`labelCol`. """ - self._set(labelCol=value) - return self + return self._set(labelCol=value) def getLabelCol(self): """ @@ -133,8 +129,7 @@ def setPredictionCol(self, value): """ Sets the value of :py:attr:`predictionCol`. """ - self._set(predictionCol=value) - return self + return self._set(predictionCol=value) def getPredictionCol(self): """ @@ -158,8 +153,7 @@ def setProbabilityCol(self, value): """ Sets the value of :py:attr:`probabilityCol`. """ - self._set(probabilityCol=value) - return self + return self._set(probabilityCol=value) def getProbabilityCol(self): """ @@ -183,8 +177,7 @@ def setRawPredictionCol(self, value): """ Sets the value of :py:attr:`rawPredictionCol`. """ - self._set(rawPredictionCol=value) - return self + return self._set(rawPredictionCol=value) def getRawPredictionCol(self): """ @@ -207,8 +200,7 @@ def setInputCol(self, value): """ Sets the value of :py:attr:`inputCol`. """ - self._set(inputCol=value) - return self + return self._set(inputCol=value) def getInputCol(self): """ @@ -231,8 +223,7 @@ def setInputCols(self, value): """ Sets the value of :py:attr:`inputCols`. """ - self._set(inputCols=value) - return self + return self._set(inputCols=value) def getInputCols(self): """ @@ -256,8 +247,7 @@ def setOutputCol(self, value): """ Sets the value of :py:attr:`outputCol`. """ - self._set(outputCol=value) - return self + return self._set(outputCol=value) def getOutputCol(self): """ @@ -280,8 +270,7 @@ def setNumFeatures(self, value): """ Sets the value of :py:attr:`numFeatures`. """ - self._set(numFeatures=value) - return self + return self._set(numFeatures=value) def getNumFeatures(self): """ @@ -304,8 +293,7 @@ def setCheckpointInterval(self, value): """ Sets the value of :py:attr:`checkpointInterval`. """ - self._set(checkpointInterval=value) - return self + return self._set(checkpointInterval=value) def getCheckpointInterval(self): """ @@ -329,8 +317,7 @@ def setSeed(self, value): """ Sets the value of :py:attr:`seed`. """ - self._set(seed=value) - return self + return self._set(seed=value) def getSeed(self): """ @@ -353,8 +340,7 @@ def setTol(self, value): """ Sets the value of :py:attr:`tol`. """ - self._set(tol=value) - return self + return self._set(tol=value) def getTol(self): """ @@ -377,8 +363,7 @@ def setStepSize(self, value): """ Sets the value of :py:attr:`stepSize`. """ - self._set(stepSize=value) - return self + return self._set(stepSize=value) def getStepSize(self): """ @@ -401,8 +386,7 @@ def setHandleInvalid(self, value): """ Sets the value of :py:attr:`handleInvalid`. """ - self._set(handleInvalid=value) - return self + return self._set(handleInvalid=value) def getHandleInvalid(self): """ @@ -426,8 +410,7 @@ def setElasticNetParam(self, value): """ Sets the value of :py:attr:`elasticNetParam`. """ - self._set(elasticNetParam=value) - return self + return self._set(elasticNetParam=value) def getElasticNetParam(self): """ @@ -451,8 +434,7 @@ def setFitIntercept(self, value): """ Sets the value of :py:attr:`fitIntercept`. """ - self._set(fitIntercept=value) - return self + return self._set(fitIntercept=value) def getFitIntercept(self): """ @@ -476,8 +458,7 @@ def setStandardization(self, value): """ Sets the value of :py:attr:`standardization`. """ - self._set(standardization=value) - return self + return self._set(standardization=value) def getStandardization(self): """ @@ -500,8 +481,7 @@ def setThresholds(self, value): """ Sets the value of :py:attr:`thresholds`. """ - self._set(thresholds=value) - return self + return self._set(thresholds=value) def getThresholds(self): """ @@ -524,8 +504,7 @@ def setWeightCol(self, value): """ Sets the value of :py:attr:`weightCol`. """ - self._set(weightCol=value) - return self + return self._set(weightCol=value) def getWeightCol(self): """ @@ -549,8 +528,7 @@ def setSolver(self, value): """ Sets the value of :py:attr:`solver`. """ - self._set(solver=value) - return self + return self._set(solver=value) def getSolver(self): """ @@ -573,8 +551,7 @@ def setVarianceCol(self, value): """ Sets the value of :py:attr:`varianceCol`. """ - self._set(varianceCol=value) - return self + return self._set(varianceCol=value) def getVarianceCol(self): """ @@ -603,8 +580,7 @@ def setMaxDepth(self, value): """ Sets the value of :py:attr:`maxDepth`. """ - self._set(maxDepth=value) - return self + return self._set(maxDepth=value) def getMaxDepth(self): """ @@ -616,8 +592,7 @@ def setMaxBins(self, value): """ Sets the value of :py:attr:`maxBins`. """ - self._set(maxBins=value) - return self + return self._set(maxBins=value) def getMaxBins(self): """ @@ -629,8 +604,7 @@ def setMinInstancesPerNode(self, value): """ Sets the value of :py:attr:`minInstancesPerNode`. """ - self._set(minInstancesPerNode=value) - return self + return self._set(minInstancesPerNode=value) def getMinInstancesPerNode(self): """ @@ -642,8 +616,7 @@ def setMinInfoGain(self, value): """ Sets the value of :py:attr:`minInfoGain`. """ - self._set(minInfoGain=value) - return self + return self._set(minInfoGain=value) def getMinInfoGain(self): """ @@ -655,8 +628,7 @@ def setMaxMemoryInMB(self, value): """ Sets the value of :py:attr:`maxMemoryInMB`. """ - self._set(maxMemoryInMB=value) - return self + return self._set(maxMemoryInMB=value) def getMaxMemoryInMB(self): """ @@ -668,8 +640,7 @@ def setCacheNodeIds(self, value): """ Sets the value of :py:attr:`cacheNodeIds`. """ - self._set(cacheNodeIds=value) - return self + return self._set(cacheNodeIds=value) def getCacheNodeIds(self): """ diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 146e403a8f97b..eb6d3310d693b 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -71,8 +71,7 @@ def setStages(self, value): :param value: a list of transformers or estimators :return: the pipeline instance """ - self._set(stages=value) - return self + return self._set(stages=value) @since("1.3.0") def getStages(self): diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 08770d9981277..db02684262cc0 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -177,8 +177,7 @@ def setRank(self, value): """ Sets the value of :py:attr:`rank`. """ - self._set(rank=value) - return self + return self._set(rank=value) @since("1.4.0") def getRank(self): @@ -192,8 +191,7 @@ def setNumUserBlocks(self, value): """ Sets the value of :py:attr:`numUserBlocks`. """ - self._set(numUserBlocks=value) - return self + return self._set(numUserBlocks=value) @since("1.4.0") def getNumUserBlocks(self): @@ -207,8 +205,7 @@ def setNumItemBlocks(self, value): """ Sets the value of :py:attr:`numItemBlocks`. """ - self._set(numItemBlocks=value) - return self + return self._set(numItemBlocks=value) @since("1.4.0") def getNumItemBlocks(self): @@ -223,15 +220,14 @@ def setNumBlocks(self, value): Sets both :py:attr:`numUserBlocks` and :py:attr:`numItemBlocks` to the specific value. """ self._set(numUserBlocks=value) - self._set(numItemBlocks=value) + return self._set(numItemBlocks=value) @since("1.4.0") def setImplicitPrefs(self, value): """ Sets the value of :py:attr:`implicitPrefs`. """ - self._set(implicitPrefs=value) - return self + return self._set(implicitPrefs=value) @since("1.4.0") def getImplicitPrefs(self): @@ -245,8 +241,7 @@ def setAlpha(self, value): """ Sets the value of :py:attr:`alpha`. """ - self._set(alpha=value) - return self + return self._set(alpha=value) @since("1.4.0") def getAlpha(self): @@ -260,8 +255,7 @@ def setUserCol(self, value): """ Sets the value of :py:attr:`userCol`. """ - self._set(userCol=value) - return self + return self._set(userCol=value) @since("1.4.0") def getUserCol(self): @@ -275,8 +269,7 @@ def setItemCol(self, value): """ Sets the value of :py:attr:`itemCol`. """ - self._set(itemCol=value) - return self + return self._set(itemCol=value) @since("1.4.0") def getItemCol(self): @@ -290,8 +283,7 @@ def setRatingCol(self, value): """ Sets the value of :py:attr:`ratingCol`. """ - self._set(ratingCol=value) - return self + return self._set(ratingCol=value) @since("1.4.0") def getRatingCol(self): @@ -305,8 +297,7 @@ def setNonnegative(self, value): """ Sets the value of :py:attr:`nonnegative`. """ - self._set(nonnegative=value) - return self + return self._set(nonnegative=value) @since("1.4.0") def getNonnegative(self): @@ -320,8 +311,7 @@ def setIntermediateStorageLevel(self, value): """ Sets the value of :py:attr:`intermediateStorageLevel`. """ - self._set(intermediateStorageLevel=value) - return self + return self._set(intermediateStorageLevel=value) @since("2.0.0") def getIntermediateStorageLevel(self): @@ -335,8 +325,7 @@ def setFinalStorageLevel(self, value): """ Sets the value of :py:attr:`finalStorageLevel`. """ - self._set(finalStorageLevel=value) - return self + return self._set(finalStorageLevel=value) @since("2.0.0") def getFinalStorageLevel(self): diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 0f08f9b9737e2..04f566dfecd60 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -469,8 +469,7 @@ def setIsotonic(self, value): """ Sets the value of :py:attr:`isotonic`. """ - self._set(isotonic=value) - return self + return self._set(isotonic=value) def getIsotonic(self): """ @@ -482,8 +481,7 @@ def setFeatureIndex(self, value): """ Sets the value of :py:attr:`featureIndex`. """ - self._set(featureIndex=value) - return self + return self._set(featureIndex=value) def getFeatureIndex(self): """ @@ -532,8 +530,7 @@ def setSubsamplingRate(self, value): """ Sets the value of :py:attr:`subsamplingRate`. """ - self._set(subsamplingRate=value) - return self + return self._set(subsamplingRate=value) @since("1.4.0") def getSubsamplingRate(self): @@ -562,8 +559,7 @@ def setImpurity(self, value): """ Sets the value of :py:attr:`impurity`. """ - self._set(impurity=value) - return self + return self._set(impurity=value) @since("1.4.0") def getImpurity(self): @@ -595,8 +591,7 @@ def setNumTrees(self, value): """ Sets the value of :py:attr:`numTrees`. """ - self._set(numTrees=value) - return self + return self._set(numTrees=value) @since("1.4.0") def getNumTrees(self): @@ -610,8 +605,7 @@ def setFeatureSubsetStrategy(self, value): """ Sets the value of :py:attr:`featureSubsetStrategy`. """ - self._set(featureSubsetStrategy=value) - return self + return self._set(featureSubsetStrategy=value) @since("1.4.0") def getFeatureSubsetStrategy(self): @@ -982,8 +976,7 @@ def setLossType(self, value): """ Sets the value of :py:attr:`lossType`. """ - self._set(lossType=value) - return self + return self._set(lossType=value) @since("1.4.0") def getLossType(self): @@ -1120,8 +1113,7 @@ def setCensorCol(self, value): """ Sets the value of :py:attr:`censorCol`. """ - self._set(censorCol=value) - return self + return self._set(censorCol=value) @since("1.6.0") def getCensorCol(self): @@ -1135,8 +1127,7 @@ def setQuantileProbabilities(self, value): """ Sets the value of :py:attr:`quantileProbabilities`. """ - self._set(quantileProbabilities=value) - return self + return self._set(quantileProbabilities=value) @since("1.6.0") def getQuantileProbabilities(self): @@ -1150,8 +1141,7 @@ def setQuantilesCol(self, value): """ Sets the value of :py:attr:`quantilesCol`. """ - self._set(quantilesCol=value) - return self + return self._set(quantilesCol=value) @since("1.6.0") def getQuantilesCol(self): @@ -1300,8 +1290,7 @@ def setFamily(self, value): """ Sets the value of :py:attr:`family`. """ - self._set(family=value) - return self + return self._set(family=value) @since("2.0.0") def getFamily(self): @@ -1315,8 +1304,7 @@ def setLink(self, value): """ Sets the value of :py:attr:`link`. """ - self._set(link=value) - return self + return self._set(link=value) @since("2.0.0") def getLink(self): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index eb1f029ebb4e3..b21cf92559522 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -198,8 +198,7 @@ def setNumFolds(self, value): """ Sets the value of :py:attr:`numFolds`. """ - self._set(numFolds=value) - return self + return self._set(numFolds=value) @since("1.4.0") def getNumFolds(self): @@ -350,8 +349,7 @@ def setTrainRatio(self, value): """ Sets the value of :py:attr:`trainRatio`. """ - self._set(trainRatio=value) - return self + return self._set(trainRatio=value) @since("2.0.0") def getTrainRatio(self): From a373c39a98a395e78ac4c0116c47a9eec39ac3e6 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Tue, 3 May 2016 09:29:49 -0700 Subject: [PATCH 0029/1470] [SPARK-15091][SPARKR] Fix warnings and a failure in SparkR test cases with testthat version 1.0.1 ## What changes were proposed in this pull request? Fix warnings and a failure in SparkR test cases with testthat version 1.0.1 ## How was this patch tested? SparkR unit test cases. Author: Sun Rui Closes #12867 from sun-rui/SPARK-15091. (cherry picked from commit 8b6491fc0b49b4e363887ae4b452ba69fe0290d5) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_client.R | 2 +- R/pkg/inst/tests/testthat/test_context.R | 2 +- R/pkg/inst/tests/testthat/test_mllib.R | 4 ++-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 11 +++++------ 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_client.R b/R/pkg/inst/tests/testthat/test_client.R index a0664f32f31c1..28276a020df91 100644 --- a/R/pkg/inst/tests/testthat/test_client.R +++ b/R/pkg/inst/tests/testthat/test_client.R @@ -32,7 +32,7 @@ test_that("no package specified doesn't add packages flag", { }) test_that("multiple packages don't produce a warning", { - expect_that(generateSparkSubmitArgs("", "", "", "", c("A", "B")), not(gives_warning())) + expect_warning(generateSparkSubmitArgs("", "", "", "", c("A", "B")), NA) }) test_that("sparkJars sparkPackages as character vectors", { diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index ca04342cd5124..0e5e15c0a96c9 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -138,7 +138,7 @@ test_that("sparkJars sparkPackages as comma-separated strings", { # check normalizePath f <- dir()[[1]] - expect_that(processSparkJars(f), not(gives_warning())) + expect_warning(processSparkJars(f), NA) expect_match(processSparkJars(f), f) }) diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index 37d87aa8a0469..5f8a27d4e094c 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -450,9 +450,9 @@ test_that("spark.survreg", { if (requireNamespace("survival", quietly = TRUE)) { rData <- list(time = c(4, 3, 1, 1, 2, 2, 3), status = c(1, 1, 1, 0, 1, 1, 0), x = c(0, 2, 1, 1, 1, 0, 0), sex = c(0, 0, 0, 0, 1, 1, 1)) - expect_that( + expect_error( model <- survival::survreg(formula = survival::Surv(time, status) ~ x + sex, data = rData), - not(throws_error())) + NA) expect_equal(predict(model, rData)[[1]], 3.724591, tolerance = 1e-4) } }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 5cf9dc405b169..081f7b1663272 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1196,9 +1196,9 @@ test_that("date functions on a DataFrame", { c(as.POSIXlt("2012-12-13 21:34:00 UTC"), as.POSIXlt("2014-12-15 10:24:34 UTC"))) expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], c(as.POSIXlt("2012-12-13 03:34:00 UTC"), as.POSIXlt("2014-12-14 16:24:34 UTC"))) - expect_more_than(collect(select(df2, unix_timestamp()))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) l3 <- list(list(a = 1000), list(a = -1000)) df3 <- createDataFrame(sqlContext, l3) @@ -1502,7 +1502,6 @@ test_that("toJSON() returns an RDD of the correct values", { test_that("showDF()", { df <- read.json(sqlContext, jsonPath) - s <- capture.output(showDF(df)) expected <- paste("+----+-------+\n", "| age| name|\n", "+----+-------+\n", @@ -1510,7 +1509,7 @@ test_that("showDF()", { "| 30| Andy|\n", "| 19| Justin|\n", "+----+-------+\n", sep = "") - expect_output(s, expected) + expect_output(showDF(df), expected) }) test_that("isLocal()", { @@ -1896,7 +1895,7 @@ test_that("Method as.data.frame as a synonym for collect()", { expect_equal(as.data.frame(irisDF2), collect(irisDF2)) # Make sure as.data.frame in the R base package is not covered - expect_that(as.data.frame(c(1, 2)), not(throws_error())) + expect_error(as.data.frame(c(1, 2)), NA) }) test_that("attach() on a DataFrame", { From 17996e7d02b6566d21c352c37ea0ed3e543ded59 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 May 2016 09:43:47 -0700 Subject: [PATCH 0030/1470] [SPARK-15088] [SQL] Remove SparkSqlSerializer ## What changes were proposed in this pull request? This patch removes SparkSqlSerializer. I believe this is now dead code. ## How was this patch tested? Removed a test case related to it. Author: Reynold Xin Closes #12864 from rxin/SPARK-15088. (cherry picked from commit 5503e453ba00676925531f91f66c0108ac6b1fca) Signed-off-by: Davies Liu --- .../sql/execution/SparkSqlSerializer.scala | 108 ------------------ .../scala/org/apache/spark/sql/RowSuite.scala | 10 -- 2 files changed, 118 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala 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 deleted file mode 100644 index c590f7c6c3e8b..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution - -import java.nio.ByteBuffer -import java.util.{HashMap => JavaHashMap} - -import scala.reflect.ClassTag - -import com.esotericsoftware.kryo.{Kryo, Serializer} -import com.esotericsoftware.kryo.io.{Input, Output} -import com.twitter.chill.ResourcePool - -import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.serializer.{KryoSerializer, SerializerInstance} -import org.apache.spark.sql.types.Decimal -import org.apache.spark.util.MutablePair - -private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { - override def newKryo(): Kryo = { - val kryo = super.newKryo() - kryo.setRegistrationRequired(false) - kryo.register(classOf[MutablePair[_, _]]) - kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) - kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericInternalRow]) - kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) - kryo.register(classOf[java.math.BigDecimal], new JavaBigDecimalSerializer) - kryo.register(classOf[BigDecimal], new ScalaBigDecimalSerializer) - - kryo.register(classOf[Decimal]) - kryo.register(classOf[JavaHashMap[_, _]]) - - kryo.setReferences(false) - kryo - } -} - -private[execution] class KryoResourcePool(size: Int) - extends ResourcePool[SerializerInstance](size) { - - val ser: SparkSqlSerializer = { - val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) - new SparkSqlSerializer(sparkConf) - } - - def newInstance(): SerializerInstance = ser.newInstance() -} - -private[sql] object SparkSqlSerializer { - @transient lazy val resourcePool = new KryoResourcePool(30) - - private[this] def acquireRelease[O](fn: SerializerInstance => O): O = { - val kryo = resourcePool.borrow - try { - fn(kryo) - } finally { - resourcePool.release(kryo) - } - } - - def serialize[T: ClassTag](o: T): Array[Byte] = - acquireRelease { k => - JavaUtils.bufferToArray(k.serialize(o)) - } - - def deserialize[T: ClassTag](bytes: Array[Byte]): T = - acquireRelease { k => - k.deserialize[T](ByteBuffer.wrap(bytes)) - } -} - -private[sql] class JavaBigDecimalSerializer extends Serializer[java.math.BigDecimal] { - def write(kryo: Kryo, output: Output, bd: java.math.BigDecimal) { - // TODO: There are probably more efficient representations than strings... - output.writeString(bd.toString) - } - - def read(kryo: Kryo, input: Input, tpe: Class[java.math.BigDecimal]): java.math.BigDecimal = { - new java.math.BigDecimal(input.readString()) - } -} - -private[sql] class ScalaBigDecimalSerializer extends Serializer[BigDecimal] { - def write(kryo: Kryo, output: Output, bd: BigDecimal) { - // TODO: There are probably more efficient representations than strings... - output.writeString(bd.toString) - } - - def read(kryo: Kryo, input: Input, tpe: Class[BigDecimal]): BigDecimal = { - new java.math.BigDecimal(input.readString()) - } -} 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 4552eb6ce00a5..34936b38fb5d4 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} -import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -55,15 +54,6 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { assert(row.isNullAt(0)) } - test("serialize w/ kryo") { - val row = Seq((1, Seq(1), Map(1 -> 1), BigDecimal(1))).toDF().first() - val serializer = new SparkSqlSerializer(sparkContext.getConf) - val instance = serializer.newInstance() - val ser = instance.serialize(row) - val de = instance.deserialize(ser).asInstanceOf[Row] - assert(de === row) - } - test("get values by field name on Row created via .toDF") { val row = Seq((1, Seq(1))).toDF("a", "b").first() assert(row.getAs[Int]("a") === 1) From 45bc65519d702147d9fb4169efd235199407fa14 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Tue, 3 May 2016 10:10:25 -0700 Subject: [PATCH 0031/1470] [SPARK-14884][SQL][STREAMING][WEBUI] Fix call site for continuous queries ## What changes were proposed in this pull request? Since we've been processing continuous queries in separate threads, the call sites are then `run at :0`. It's not wrong but provides very little information; in addition, we can not distinguish two queries only from their call sites. This patch fixes this. ### Before [Jobs Tab] ![s1a](https://cloud.githubusercontent.com/assets/15843379/14766101/a47246b2-0a30-11e6-8d81-06a9a600113b.png) [SQL Tab] ![s1b](https://cloud.githubusercontent.com/assets/15843379/14766102/a4750226-0a30-11e6-9ada-773d977d902b.png) ### After [Jobs Tab] ![s2a](https://cloud.githubusercontent.com/assets/15843379/14766104/a89705b6-0a30-11e6-9830-0d40ec68527b.png) [SQL Tab] ![s2b](https://cloud.githubusercontent.com/assets/15843379/14766103/a8966728-0a30-11e6-8e4d-c2e326400478.png) ## How was this patch tested? Manually checks - see screenshots above. Author: Liwei Lin Closes #12650 from lw-lin/fix-call-site. (cherry picked from commit 5bd9a2f697dac44a4777e24321a2eb4a3d54e24b) Signed-off-by: Shixiong Zhu --- .../apache/spark/sql/execution/SQLExecution.scala | 7 +++++-- .../sql/execution/streaming/StreamExecution.scala | 12 ++++++++++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 397d66b31153f..31c9f1aef2f3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -23,7 +23,6 @@ import org.apache.spark.SparkContext import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} -import org.apache.spark.util.Utils private[sql] object SQLExecution { @@ -46,7 +45,11 @@ private[sql] object SQLExecution { val executionId = SQLExecution.nextExecutionId sc.setLocalProperty(EXECUTION_ID_KEY, executionId.toString) val r = try { - val callSite = Utils.getCallSite() + // sparkContext.getCallSite() would first try to pick up any call site that was previously + // set, then fall back to Utils.getCallSite(); call Utils.getCallSite() directly on + // continuous queries would give us call site like "run at :0" + val callSite = sparkSession.sparkContext.getCallSite() + sparkSession.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart( executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 3108346913800..3c5ced2af73f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.util.ContinuousQueryListener import org.apache.spark.sql.util.ContinuousQueryListener._ -import org.apache.spark.util.UninterruptibleThread +import org.apache.spark.util.{UninterruptibleThread, Utils} /** * Manages the execution of a streaming Spark SQL query that is occurring in a separate thread. @@ -101,10 +101,18 @@ class StreamExecution( @volatile private[sql] var streamDeathCause: ContinuousQueryException = null + /* Get the call site in the caller thread; will pass this into the micro batch thread */ + private val callSite = Utils.getCallSite() + /** The thread that runs the micro-batches of this stream. */ private[sql] val microBatchThread = new UninterruptibleThread(s"stream execution thread for $name") { - override def run(): Unit = { runBatches() } + override def run(): Unit = { + // To fix call site like "run at :0", we bridge the call site from the caller + // thread to this micro batch thread + sparkSession.sparkContext.setCallSite(callSite) + runBatches() + } } /** From c5b7e1f70424b176b655936dbcace93e4d4a7210 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 3 May 2016 10:58:26 -0700 Subject: [PATCH 0032/1470] [SPARK-14716][SQL] Added support for partitioning in FileStreamSink # What changes were proposed in this pull request? Support partitioning in the file stream sink. This is implemented using a new, but simpler code path for writing parquet files - both unpartitioned and partitioned. This new code path does not use Output Committers, as we will eventually write the file names to the metadata log for "committing" them. This patch duplicates < 100 LOC from the WriterContainer. But its far simpler that WriterContainer as it does not involve output committing. In addition, it introduces the new APIs in FileFormat and OutputWriterFactory in an attempt to simplify the APIs (not have Job in the `FileFormat` API, not have bucket and other stuff in the `OutputWriterFactory.newInstance()` ). # Tests - New unit tests to test the FileStreamSinkWriter for partitioned and unpartitioned files - New unit test to partially test the FileStreamSink for partitioned files (does not test recovery of partition column data, as that requires change in the StreamFileCatalog, future PR). - Updated FileStressSuite to test number of records read from partitioned output files. Author: Tathagata Das Closes #12409 from tdas/streaming-partitioned-parquet. (cherry picked from commit 4ad492c40358d0104db508db98ce0971114b6817) Signed-off-by: Tathagata Das --- python/pyspark/sql/tests.py | 4 +- .../execution/datasources/DataSource.scala | 5 +- .../datasources/fileSourceInterfaces.scala | 28 +++ .../datasources/parquet/ParquetRelation.scala | 116 ++++++++- .../execution/streaming/FileStreamSink.scala | 230 ++++++++++++++++-- .../streaming/FileStreamSinkLog.scala | 13 + .../execution/streaming/HDFSMetadataLog.scala | 5 +- .../sql/streaming/FileStreamSinkSuite.scala | 218 +++++++++++++++-- .../spark/sql/streaming/FileStressSuite.scala | 40 ++- 9 files changed, 605 insertions(+), 54 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 4995b263e1930..cd5c4a7b3e9f8 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -940,7 +940,7 @@ def test_stream_save_options(self): cq.processAllAvailable() output_files = [] for _, _, files in os.walk(out): - output_files.extend([f for f in files if 'parquet' in f and not f.startswith('.')]) + output_files.extend([f for f in files if not f.startswith('.')]) self.assertTrue(len(output_files) > 0) self.assertTrue(len(os.listdir(chk)) > 0) finally: @@ -967,7 +967,7 @@ def test_stream_save_options_overwrite(self): cq.processAllAvailable() output_files = [] for _, _, files in os.walk(out): - output_files.extend([f for f in files if 'parquet' in f and not f.startswith('.')]) + output_files.extend([f for f in files if not f.startswith('.')]) self.assertTrue(len(output_files) > 0) self.assertTrue(len(os.listdir(chk)) > 0) self.assertFalse(os.path.isdir(fake1)) # should not have been created diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 63dc1fd71e6d8..6114142cefea2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -203,13 +203,14 @@ case class DataSource( def createSink(): Sink = { providingClass.newInstance() match { case s: StreamSinkProvider => s.createSink(sparkSession.wrapped, options, partitionColumns) - case format: FileFormat => + + case parquet: parquet.DefaultSource => val caseInsensitiveOptions = new CaseInsensitiveMap(options) val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") }) + new FileStreamSink(sparkSession, path, parquet, partitionColumns, options) - new FileStreamSink(sparkSession, path, format) case _ => throw new UnsupportedOperationException( s"Data source $className does not support streamed writing") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 25f88d9c39487..0a3461151c627 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -64,6 +64,20 @@ abstract class OutputWriterFactory extends Serializable { bucketId: Option[Int], // TODO: This doesn't belong here... dataSchema: StructType, context: TaskAttemptContext): OutputWriter + + /** + * Returns a new instance of [[OutputWriter]] that will write data to the given path. + * This method gets called by each task on executor to write [[InternalRow]]s to + * format-specific files. Compared to the other `newInstance()`, this is a newer API that + * passes only the path that the writer must write to. The writer must write to the exact path + * and not modify it (do not add subdirectories, extensions, etc.). All other + * file-format-specific information needed to create the writer must be passed + * through the [[OutputWriterFactory]] implementation. + * @since 2.0.0 + */ + private[sql] def newWriter(path: String): OutputWriter = { + throw new UnsupportedOperationException("newInstance with just path not supported") + } } /** @@ -223,6 +237,20 @@ trait FileFormat { // Until then we guard in [[FileSourceStrategy]] to only call this method on supported formats. throw new UnsupportedOperationException(s"buildReader is not supported for $this") } + + /** + * Returns a [[OutputWriterFactory]] for generating output writers that can write data. + * This method is current used only by FileStreamSinkWriter to generate output writers that + * does not use output committers to write data. The OutputWriter generated by the returned + * [[OutputWriterFactory]] must implement the method `newWriter(path)`.. + */ + def buildWriter( + sqlContext: SQLContext, + dataSchema: StructType, + options: Map[String, String]): OutputWriterFactory = { + // TODO: Remove this default implementation when the other formats have been ported + throw new UnsupportedOperationException(s"buildWriter is not supported for $this") + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index b1513bbe945bb..79185df67325b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -41,13 +41,13 @@ import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{AtomicType, DataType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration private[sql] class DefaultSource @@ -372,8 +372,120 @@ private[sql] class DefaultSource } } } + + override def buildWriter( + sqlContext: SQLContext, + dataSchema: StructType, + options: Map[String, String]): OutputWriterFactory = { + new ParquetOutputWriterFactory( + sqlContext.conf, + dataSchema, + sqlContext.sparkContext.hadoopConfiguration, + options) + } } +/** + * A factory for generating OutputWriters for writing parquet files. This implemented is different + * from the [[ParquetOutputWriter]] as this does not use any [[OutputCommitter]]. It simply + * writes the data to the path used to generate the output writer. Callers of this factory + * has to ensure which files are to be considered as committed. + */ +private[sql] class ParquetOutputWriterFactory( + sqlConf: SQLConf, + dataSchema: StructType, + hadoopConf: Configuration, + options: Map[String, String]) extends OutputWriterFactory { + + private val serializableConf: SerializableConfiguration = { + val job = Job.getInstance(hadoopConf) + val conf = ContextUtil.getConfiguration(job) + val parquetOptions = new ParquetOptions(options, sqlConf) + + // We're not really using `ParquetOutputFormat[Row]` for writing data here, because we override + // it in `ParquetOutputWriter` to support appending and dynamic partitioning. The reason why + // we set it here is to setup the output committer class to `ParquetOutputCommitter`, which is + // bundled with `ParquetOutputFormat[Row]`. + job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) + + ParquetOutputFormat.setWriteSupportClass(job, classOf[CatalystWriteSupport]) + + // We want to clear this temporary metadata from saving into Parquet file. + // This metadata is only useful for detecting optional columns when pushdowning filters. + val dataSchemaToWrite = StructType.removeMetadata( + StructType.metadataKeyForOptionalField, + dataSchema).asInstanceOf[StructType] + CatalystWriteSupport.setSchema(dataSchemaToWrite, conf) + + // Sets flags for `CatalystSchemaConverter` (which converts Catalyst schema to Parquet schema) + // and `CatalystWriteSupport` (writing actual rows to Parquet files). + conf.set( + SQLConf.PARQUET_BINARY_AS_STRING.key, + sqlConf.isParquetBinaryAsString.toString) + + conf.set( + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, + sqlConf.isParquetINT96AsTimestamp.toString) + + conf.set( + SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, + sqlConf.writeLegacyParquetFormat.toString) + + // Sets compression scheme + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + new SerializableConfiguration(conf) + } + + /** + * Returns a [[OutputWriter]] that writes data to the give path without using + * [[OutputCommitter]]. + */ + override private[sql] def newWriter(path: String): OutputWriter = new OutputWriter { + + // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter + private val hadoopTaskAttempId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) + private val hadoopAttemptContext = new TaskAttemptContextImpl( + serializableConf.value, hadoopTaskAttempId) + + // Instance of ParquetRecordWriter that does not use OutputCommitter + private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) + + override def write(row: Row): Unit = { + throw new UnsupportedOperationException("call writeInternal") + } + + protected[sql] override def writeInternal(row: InternalRow): Unit = { + recordWriter.write(null, row) + } + + override def close(): Unit = recordWriter.close(hadoopAttemptContext) + } + + /** Create a [[ParquetRecordWriter]] that writes the given path without using OutputCommitter */ + private def createNoCommitterRecordWriter( + path: String, + hadoopAttemptContext: TaskAttemptContext): RecordWriter[Void, InternalRow] = { + // Custom ParquetOutputFormat that disable use of committer and writes to the given path + val outputFormat = new ParquetOutputFormat[InternalRow]() { + override def getOutputCommitter(c: TaskAttemptContext): OutputCommitter = { null } + override def getDefaultWorkFile(c: TaskAttemptContext, ext: String): Path = { new Path(path) } + } + outputFormat.getRecordWriter(hadoopAttemptContext) + } + + /** Disable the use of the older API. */ + def newInstance( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + throw new UnsupportedOperationException( + "this verison of newInstance not supported for " + + "ParquetOutputWriterFactory") + } +} + + // NOTE: This class is instantiated and used on executor side only, no need to be serializable. private[sql] class ParquetOutputWriter( path: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index 70aea7fa49a1e..e19101032967b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -19,11 +19,20 @@ package org.apache.spark.sql.execution.streaming import java.util.UUID +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.{SparkEnv, SparkException, TaskContext, TaskContextImpl} import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.UnsafeKVExternalSorter +import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, PartitioningUtils} +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.SerializableConfiguration object FileStreamSink { // The name of the subdirectory that is used to store metadata about which files are valid. @@ -40,28 +49,24 @@ object FileStreamSink { class FileStreamSink( sparkSession: SparkSession, path: String, - fileFormat: FileFormat) extends Sink with Logging { + fileFormat: FileFormat, + partitionColumnNames: Seq[String], + options: Map[String, String]) extends Sink with Logging { private val basePath = new Path(path) private val logPath = new Path(basePath, FileStreamSink.metadataDir) private val fileLog = new FileStreamSinkLog(sparkSession, logPath.toUri.toString) - private val fs = basePath.getFileSystem(sparkSession.sessionState.newHadoopConf()) + private val hadoopConf = sparkSession.sessionState.newHadoopConf() + private val fs = basePath.getFileSystem(hadoopConf) override def addBatch(batchId: Long, data: DataFrame): Unit = { if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) { logInfo(s"Skipping already committed batch $batchId") } else { - val files = fs.listStatus(writeFiles(data)).map { f => - SinkFileStatus( - path = f.getPath.toUri.toString, - size = f.getLen, - isDir = f.isDirectory, - modificationTime = f.getModificationTime, - blockReplication = f.getReplication, - blockSize = f.getBlockSize, - action = FileStreamSinkLog.ADD_ACTION) - } - if (fileLog.add(batchId, files)) { + val writer = new FileStreamSinkWriter( + data, fileFormat, path, partitionColumnNames, hadoopConf, options) + val fileStatuses = writer.write() + if (fileLog.add(batchId, fileStatuses)) { logInfo(s"Committed batch $batchId") } else { throw new IllegalStateException(s"Race while writing batch $batchId") @@ -69,17 +74,192 @@ class FileStreamSink( } } - /** Writes the [[DataFrame]] to a UUID-named dir, returning the list of files paths. */ - private def writeFiles(data: DataFrame): Array[Path] = { - val file = new Path(basePath, UUID.randomUUID().toString).toUri.toString - data.write.parquet(file) - sparkSession.read - .schema(data.schema) - .parquet(file) - .inputFiles - .map(new Path(_)) - .filterNot(_.getName.startsWith("_")) + override def toString: String = s"FileSink[$path]" +} + + +/** + * Writes data given to a [[FileStreamSink]] to the given `basePath` in the given `fileFormat`, + * partitioned by the given `partitionColumnNames`. This writer always appends data to the + * directory if it already has data. + */ +class FileStreamSinkWriter( + data: DataFrame, + fileFormat: FileFormat, + basePath: String, + partitionColumnNames: Seq[String], + hadoopConf: Configuration, + options: Map[String, String]) extends Serializable with Logging { + + PartitioningUtils.validatePartitionColumnDataTypes( + data.schema, partitionColumnNames, data.sqlContext.conf.caseSensitiveAnalysis) + + private val serializableConf = new SerializableConfiguration(hadoopConf) + private val dataSchema = data.schema + private val dataColumns = data.logicalPlan.output + + // Get the actual partition columns as attributes after matching them by name with + // the given columns names. + private val partitionColumns = partitionColumnNames.map { col => + val nameEquality = if (data.sparkSession.sessionState.conf.caseSensitiveAnalysis) { + org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + } else { + org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution + } + data.logicalPlan.output.find(f => nameEquality(f.name, col)).getOrElse { + throw new RuntimeException(s"Partition column $col not found in schema $dataSchema") + } + } + + // Columns that are to be written to the files. If there are partitioning columns, then + // those will not be written to the files. + private val writeColumns = { + val partitionSet = AttributeSet(partitionColumns) + dataColumns.filterNot(partitionSet.contains) } - override def toString: String = s"FileSink[$path]" + // An OutputWriterFactory for generating writers in the executors for writing the files. + private val outputWriterFactory = + fileFormat.buildWriter(data.sqlContext, writeColumns.toStructType, options) + + /** Expressions that given a partition key build a string like: col1=val/col2=val/... */ + private def partitionStringExpression: Seq[Expression] = { + partitionColumns.zipWithIndex.flatMap { case (c, i) => + val escaped = + ScalaUDF( + PartitioningUtils.escapePathName _, + StringType, + Seq(Cast(c, StringType)), + Seq(StringType)) + val str = If(IsNull(c), Literal(PartitioningUtils.DEFAULT_PARTITION_NAME), escaped) + val partitionName = Literal(c.name + "=") :: str :: Nil + if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName + } + } + + /** Generate a new output writer from the writer factory */ + private def newOutputWriter(path: Path): OutputWriter = { + val newWriter = outputWriterFactory.newWriter(path.toString) + newWriter.initConverter(dataSchema) + newWriter + } + + /** Write the dataframe to files. This gets called in the driver by the [[FileStreamSink]]. */ + def write(): Array[SinkFileStatus] = { + data.sqlContext.sparkContext.runJob( + data.queryExecution.toRdd, + (taskContext: TaskContext, iterator: Iterator[InternalRow]) => { + if (partitionColumns.isEmpty) { + Seq(writePartitionToSingleFile(iterator)) + } else { + writePartitionToPartitionedFiles(iterator) + } + }).flatten + } + + /** + * Writes a RDD partition to a single file without dynamic partitioning. + * This gets called in the executor, and it uses a [[OutputWriter]] to write the data. + */ + def writePartitionToSingleFile(iterator: Iterator[InternalRow]): SinkFileStatus = { + var writer: OutputWriter = null + try { + val path = new Path(basePath, UUID.randomUUID.toString) + val fs = path.getFileSystem(serializableConf.value) + writer = newOutputWriter(path) + while (iterator.hasNext) { + writer.writeInternal(iterator.next) + } + writer.close() + writer = null + SinkFileStatus(fs.getFileStatus(path)) + } catch { + case cause: Throwable => + logError("Aborting task.", cause) + // call failure callbacks first, so we could have a chance to cleanup the writer. + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(cause) + throw new SparkException("Task failed while writing rows.", cause) + } finally { + if (writer != null) { + writer.close() + } + } + } + + /** + * Writes a RDD partition to multiple dynamically partitioned files. + * This gets called in the executor. It first sorts the data based on the partitioning columns + * and then writes the data of each key to separate files using [[OutputWriter]]s. + */ + def writePartitionToPartitionedFiles(iterator: Iterator[InternalRow]): Seq[SinkFileStatus] = { + + // Returns the partitioning columns for sorting + val getSortingKey = UnsafeProjection.create(partitionColumns, dataColumns) + + // Returns the data columns to be written given an input row + val getOutputRow = UnsafeProjection.create(writeColumns, dataColumns) + + // Returns the partition path given a partition key + val getPartitionString = + UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) + + // Sort the data before write, so that we only need one writer at the same time. + val sorter = new UnsafeKVExternalSorter( + partitionColumns.toStructType, + StructType.fromAttributes(writeColumns), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + logDebug(s"Sorting complete. Writing out partition files one at a time.") + + val sortedIterator = sorter.sortedIterator() + val paths = new ArrayBuffer[Path] + + // Write the sorted data to partitioned files, one for each unique key + var currentWriter: OutputWriter = null + try { + var currentKey: UnsafeRow = null + while (sortedIterator.next()) { + val nextKey = sortedIterator.getKey + + // If key changes, close current writer, and open a new writer to a new partitioned file + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + currentKey = nextKey.copy() + val partitionPath = getPartitionString(currentKey).getString(0) + val path = new Path(new Path(basePath, partitionPath), UUID.randomUUID.toString) + paths += path + currentWriter = newOutputWriter(path) + logInfo(s"Writing partition $currentKey to $path") + } + currentWriter.writeInternal(sortedIterator.getValue) + } + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + if (paths.nonEmpty) { + val fs = paths.head.getFileSystem(serializableConf.value) + paths.map(p => SinkFileStatus(fs.getFileStatus(p))) + } else Seq.empty + } catch { + case cause: Throwable => + logError("Aborting task.", cause) + // call failure callbacks first, so we could have a chance to cleanup the writer. + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(cause) + throw new SparkException("Task failed while writing rows.", cause) + } finally { + if (currentWriter != null) { + currentWriter.close() + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala index b694b6155ad99..4254df44c97a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala @@ -54,6 +54,19 @@ case class SinkFileStatus( } } +object SinkFileStatus { + def apply(f: FileStatus): SinkFileStatus = { + SinkFileStatus( + path = f.getPath.toUri.toString, + size = f.getLen, + isDir = f.isDirectory, + modificationTime = f.getModificationTime, + blockReplication = f.getReplication, + blockSize = f.getBlockSize, + action = FileStreamSinkLog.ADD_ACTION) + } +} + /** * A special log for [[FileStreamSink]]. It will write one log file for each batch. The first line * of the log file is the version number, and there are multiple JSON lines following. Each JSON diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 9fe06a6c36cb9..fca3d51535a29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -216,8 +216,9 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) new FileContextManager(metadataPath, hadoopConf) } catch { case e: UnsupportedFileSystemException => - logWarning("Could not use FileContext API for managing metadata log file. The log may be" + - "inconsistent under failures.", e) + logWarning("Could not use FileContext API for managing metadata log files at path " + + s"$metadataPath. Using FileSystem API instead for managing log files. The log may be " + + s"inconsistent under failures.") new FileSystemManager(metadataPath, hadoopConf) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 8cf5dedabcee1..609ca976a0160 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -17,33 +17,223 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql.StreamTest -import org.apache.spark.sql.execution.streaming.MemoryStream +import java.io.File + +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.{DirectoryFileFilter, RegexFileFilter} + +import org.apache.spark.sql.{ContinuousQuery, Row, StreamTest} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.execution.datasources.parquet +import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils class FileStreamSinkSuite extends StreamTest with SharedSQLContext { import testImplicits._ - test("unpartitioned writing") { + + test("FileStreamSinkWriter - unpartitioned data") { + val path = Utils.createTempDir() + path.delete() + + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val fileFormat = new parquet.DefaultSource() + + def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { + val df = sqlContext + .range(start, end, 1, numPartitions) + .select($"id", lit(100).as("data")) + val writer = new FileStreamSinkWriter( + df, fileFormat, path.toString, partitionColumnNames = Nil, hadoopConf, Map.empty) + writer.write().map(_.path.stripPrefix("file://")) + } + + // Write and check whether new files are written correctly + val files1 = writeRange(0, 10, 2) + assert(files1.size === 2, s"unexpected number of files: $files1") + checkFilesExist(path, files1, "file not written") + checkAnswer(sqlContext.read.load(path.getCanonicalPath), (0 until 10).map(Row(_, 100))) + + // Append and check whether new files are written correctly and old files still exist + val files2 = writeRange(10, 20, 3) + assert(files2.size === 3, s"unexpected number of files: $files2") + assert(files2.intersect(files1).isEmpty, "old files returned") + checkFilesExist(path, files2, s"New file not written") + checkFilesExist(path, files1, s"Old file not found") + checkAnswer(sqlContext.read.load(path.getCanonicalPath), (0 until 20).map(Row(_, 100))) + } + + test("FileStreamSinkWriter - partitioned data") { + implicit val e = ExpressionEncoder[java.lang.Long] + val path = Utils.createTempDir() + path.delete() + + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val fileFormat = new parquet.DefaultSource() + + def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { + val df = sqlContext + .range(start, end, 1, numPartitions) + .flatMap(x => Iterator(x, x, x)).toDF("id") + .select($"id", lit(100).as("data1"), lit(1000).as("data2")) + + require(df.rdd.partitions.size === numPartitions) + val writer = new FileStreamSinkWriter( + df, fileFormat, path.toString, partitionColumnNames = Seq("id"), hadoopConf, Map.empty) + writer.write().map(_.path.stripPrefix("file://")) + } + + def checkOneFileWrittenPerKey(keys: Seq[Int], filesWritten: Seq[String]): Unit = { + keys.foreach { id => + assert( + filesWritten.count(_.contains(s"/id=$id/")) == 1, + s"no file for id=$id. all files: \n\t${filesWritten.mkString("\n\t")}" + ) + } + } + + // Write and check whether new files are written correctly + val files1 = writeRange(0, 10, 2) + assert(files1.size === 10, s"unexpected number of files:\n${files1.mkString("\n")}") + checkFilesExist(path, files1, "file not written") + checkOneFileWrittenPerKey(0 until 10, files1) + + val answer1 = (0 until 10).flatMap(x => Iterator(x, x, x)).map(Row(100, 1000, _)) + checkAnswer(sqlContext.read.load(path.getCanonicalPath), answer1) + + // Append and check whether new files are written correctly and old files still exist + val files2 = writeRange(0, 20, 3) + assert(files2.size === 20, s"unexpected number of files:\n${files2.mkString("\n")}") + assert(files2.intersect(files1).isEmpty, "old files returned") + checkFilesExist(path, files2, s"New file not written") + checkFilesExist(path, files1, s"Old file not found") + checkOneFileWrittenPerKey(0 until 20, files2) + + val answer2 = (0 until 20).flatMap(x => Iterator(x, x, x)).map(Row(100, 1000, _)) + checkAnswer(sqlContext.read.load(path.getCanonicalPath), answer1 ++ answer2) + } + + test("FileStreamSink - unpartitioned writing and batch reading") { val inputData = MemoryStream[Int] val df = inputData.toDF() val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath - val query = - df.write - .format("parquet") - .option("checkpointLocation", checkpointDir) - .startStream(outputDir) + var query: ContinuousQuery = null + + try { + query = + df.write + .format("parquet") + .option("checkpointLocation", checkpointDir) + .startStream(outputDir) + + inputData.addData(1, 2, 3) + + failAfter(streamingTimeout) { + query.processAllAvailable() + } - inputData.addData(1, 2, 3) - failAfter(streamingTimeout) { query.processAllAvailable() } + val outputDf = sqlContext.read.parquet(outputDir).as[Int] + checkDataset(outputDf, 1, 2, 3) - val outputDf = sqlContext.read.parquet(outputDir).as[Int] - checkDataset( - outputDf, - 1, 2, 3) + } finally { + if (query != null) { + query.stop() + } + } } + + test("FileStreamSink - partitioned writing and batch reading [IGNORES PARTITION COLUMN]") { + val inputData = MemoryStream[Int] + val ds = inputData.toDS() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: ContinuousQuery = null + + try { + query = + ds.map(i => (i, i * 1000)) + .toDF("id", "value") + .write + .format("parquet") + .partitionBy("id") + .option("checkpointLocation", checkpointDir) + .startStream(outputDir) + + inputData.addData(1, 2, 3) + failAfter(streamingTimeout) { + query.processAllAvailable() + } + + // TODO (tdas): Test partition column can be read or not + val outputDf = sqlContext.read.parquet(outputDir) + checkDataset( + outputDf.as[Int], + 1000, 2000, 3000) + + } finally { + if (query != null) { + query.stop() + } + } + } + + test("FileStreamSink - supported formats") { + def testFormat(format: Option[String]): Unit = { + val inputData = MemoryStream[Int] + val ds = inputData.toDS() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: ContinuousQuery = null + + try { + val writer = + ds.map(i => (i, i * 1000)) + .toDF("id", "value") + .write + if (format.nonEmpty) { + writer.format(format.get) + } + query = writer + .option("checkpointLocation", checkpointDir) + .startStream(outputDir) + } finally { + if (query != null) { + query.stop() + } + } + } + + testFormat(None) // should not throw error as default format parquet when not specified + testFormat(Some("parquet")) + val e = intercept[UnsupportedOperationException] { + testFormat(Some("text")) + } + Seq("text", "not support", "stream").foreach { s => + assert(e.getMessage.contains(s)) + } + } + + private def checkFilesExist(dir: File, expectedFiles: Seq[String], msg: String): Unit = { + import scala.collection.JavaConverters._ + val files = + FileUtils.listFiles(dir, new RegexFileFilter("[^.]+"), DirectoryFileFilter.DIRECTORY) + .asScala + .map(_.getCanonicalPath) + .toSet + + expectedFiles.foreach { f => + assert(files.contains(f), + s"\n$msg\nexpected file:\n\t$f\nfound files:\n${files.mkString("\n\t")}") + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala index 5b49a0a86a04f..50703e532fa4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala @@ -41,7 +41,15 @@ import org.apache.spark.util.Utils class FileStressSuite extends StreamTest with SharedSQLContext { import testImplicits._ - test("fault tolerance stress test") { + testQuietly("fault tolerance stress test - unpartitioned output") { + stressTest(partitionWrites = false) + } + + testQuietly("fault tolerance stress test - partitioned output") { + stressTest(partitionWrites = true) + } + + def stressTest(partitionWrites: Boolean): Unit = { val numRecords = 10000 val inputDir = Utils.createTempDir(namePrefix = "stream.input").getCanonicalPath val stagingDir = Utils.createTempDir(namePrefix = "stream.staging").getCanonicalPath @@ -93,18 +101,36 @@ class FileStressSuite extends StreamTest with SharedSQLContext { writer.start() val input = sqlContext.read.format("text").stream(inputDir) - def startStream(): ContinuousQuery = input + + def startStream(): ContinuousQuery = { + val output = input .repartition(5) .as[String] .mapPartitions { iter => val rand = Random.nextInt(100) - if (rand < 5) { sys.error("failure") } + if (rand < 10) { + sys.error("failure") + } iter.map(_.toLong) } - .write - .format("parquet") - .option("checkpointLocation", checkpoint) - .startStream(outputDir) + .map(x => (x % 400, x.toString)) + .toDF("id", "data") + + if (partitionWrites) { + output + .write + .partitionBy("id") + .format("parquet") + .option("checkpointLocation", checkpoint) + .startStream(outputDir) + } else { + output + .write + .format("parquet") + .option("checkpointLocation", checkpoint) + .startStream(outputDir) + } + } var failures = 0 val streamThread = new Thread("stream runner") { From 31e5a2a76115e681a2f8ebef06352d417113ae05 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 3 May 2016 11:16:55 -0700 Subject: [PATCH 0033/1470] [SPARK-14860][TESTS] Create a new Waiter in reset to bypass an issue of ScalaTest's Waiter.wait ## What changes were proposed in this pull request? This PR updates `QueryStatusCollector.reset` to create Waiter instead of calling `await(1 milliseconds)` to bypass an ScalaTest's issue that Waiter.await may block forever. ## How was this patch tested? I created a local stress test to call codes in `test("event ordering")` 100 times. It cannot pass without this patch. Author: Shixiong Zhu Closes #12623 from zsxwing/flaky-test. (cherry picked from commit b545d752195f6dcba4c512b8a1d5bf5b74279dc8) Signed-off-by: Tathagata Das --- .../sql/util/ContinuousQueryListenerSuite.scala | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala index 3498fe83d02eb..2596231a12446 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.util import java.util.concurrent.ConcurrentLinkedQueue -import scala.util.control.NonFatal - import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ import org.scalatest.concurrent.AsyncAssertions.Waiter @@ -164,8 +162,8 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with } class QueryStatusCollector extends ContinuousQueryListener { - - private val asyncTestWaiter = new Waiter // to catch errors in the async listener events + // to catch errors in the async listener events + @volatile private var asyncTestWaiter = new Waiter @volatile var startStatus: QueryStatus = null @volatile var terminationStatus: QueryStatus = null @@ -175,11 +173,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with startStatus = null terminationStatus = null progressStatuses.clear() - - // To reset the waiter - try asyncTestWaiter.await(timeout(1 milliseconds)) catch { - case NonFatal(e) => - } + asyncTestWaiter = new Waiter } def checkAsyncErrors(): Unit = { From 2b715251d3bc9fa7de282356e00cd8bf0fcbcdee Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 11:38:43 -0700 Subject: [PATCH 0034/1470] [SPARK-15087][CORE][SQL] Remove AccumulatorV2.localValue and keep only value ## What changes were proposed in this pull request? Remove AccumulatorV2.localValue and keep only value ## How was this patch tested? existing tests Author: Sandeep Singh Closes #12865 from techaddict/SPARK-15087. (cherry picked from commit ca813330c716bed76ac0034c12f56665960a1105) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/Accumulable.scala | 10 ++++-- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 6 ++-- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../org/apache/spark/util/AccumulatorV2.scala | 32 ++++++------------- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../org/apache/spark/AccumulatorSuite.scala | 4 +-- .../sql/execution/metric/SQLMetrics.scala | 4 +-- .../spark/sql/execution/ui/SQLListener.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 2 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- 11 files changed, 30 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index 5c6761eb764b5..812145aaeedb7 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -110,7 +110,13 @@ class Accumulable[R, T] private ( /** * Access the accumulator's current value; only allowed on driver. */ - def value: R = newAcc.value + def value: R = { + if (newAcc.isAtDriverSide) { + newAcc.value + } else { + throw new UnsupportedOperationException("Can't read accumulator value in task") + } + } /** * Get the current value of this accumulator from within a task. @@ -121,7 +127,7 @@ class Accumulable[R, T] private ( * The typical use of this method is to directly mutate the local value, eg., to add * an element to a Set. */ - def localValue: R = newAcc.localValue + def localValue: R = newAcc.value /** * Set the accumulator's value; only allowed on driver. 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 4f74dc92d7dfc..64e87a95d0342 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -362,7 +362,7 @@ private[spark] class Executor( Seq.empty } - val accUpdates = accums.map(acc => acc.toInfo(Some(acc.localValue), None)) + val accUpdates = accums.map(acc => acc.toInfo(Some(acc.value), None)) val serializedTaskEndReason = { try { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 085aa7fbd63dd..7f4652c2dd765 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -99,7 +99,7 @@ class TaskMetrics private[spark] () extends Serializable { /** * Storage statuses of any blocks that have been updated as a result of this task. */ - def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.value // Setters and increment-ers private[spark] def setExecutorDeserializeTime(v: Long): Unit = @@ -301,12 +301,12 @@ private[spark] class BlockStatusesAccumulator override def merge(other: AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) : Unit = other match { - case o: BlockStatusesAccumulator => _seq ++= o.localValue + case o: BlockStatusesAccumulator => _seq ++= o.value case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: Seq[(BlockId, BlockStatus)] = _seq + override def value: Seq[(BlockId, BlockStatus)] = _seq def setValue(newValue: Seq[(BlockId, BlockStatus)]): Unit = { _seq.clear() 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 393680f4c1542..8ce8fb15efbc2 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( // deserialized. This brings trouble to the accumulator framework, which depends on // serialization to set the `atDriverSide` flag. Here we call `acc.localValue` instead to // be more robust about this issue. - val accInfos = updates.map(acc => acc.toInfo(Some(acc.localValue), None)) + val accInfos = updates.map(acc => acc.toInfo(Some(acc.value), None)) taskIdToTaskSetManager.get(id).map { taskSetMgr => (id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, accInfos) } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 0e280f6f6a290..d8f380e1230e5 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -126,23 +126,9 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { def merge(other: AccumulatorV2[IN, OUT]): Unit /** - * Access this accumulator's current value; only allowed on driver. + * Defines the current value of this accumulator */ - final def value: OUT = { - if (atDriverSide) { - localValue - } else { - throw new UnsupportedOperationException("Can't read accumulator value in task") - } - } - - /** - * Defines the current value of this accumulator. - * - * This is NOT the global value of the accumulator. To get the global value after a - * completed operation on the dataset, call `value`. - */ - def localValue: OUT + def value: OUT // Called by Java when serializing an object final protected def writeReplace(): Any = { @@ -182,7 +168,7 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { if (metadata == null) { "Un-registered Accumulator: " + getClass.getSimpleName } else { - getClass.getSimpleName + s"(id: $id, name: $name, value: $localValue)" + getClass.getSimpleName + s"(id: $id, name: $name, value: $value)" } } } @@ -321,7 +307,7 @@ class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { private[spark] def setValue(newValue: Long): Unit = _sum = newValue - override def localValue: jl.Long = _sum + override def value: jl.Long = _sum } @@ -386,7 +372,7 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { private[spark] def setValue(newValue: Double): Unit = _sum = newValue - override def localValue: jl.Double = _sum + override def value: jl.Double = _sum } @@ -400,12 +386,12 @@ class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { override def add(v: T): Unit = _list.add(v) override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match { - case o: ListAccumulator[T] => _list.addAll(o.localValue) + case o: ListAccumulator[T] => _list.addAll(o.value) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: java.util.List[T] = java.util.Collections.unmodifiableList(_list) + override def value: java.util.List[T] = java.util.Collections.unmodifiableList(_list) private[spark] def setValue(newValue: java.util.List[T]): Unit = { _list.clear() @@ -430,10 +416,10 @@ class LegacyAccumulatorWrapper[R, T]( override def add(v: T): Unit = _value = param.addAccumulator(_value, v) override def merge(other: AccumulatorV2[T, R]): Unit = other match { - case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.localValue) + case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.value) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def localValue: R = _value + override def value: R = _value } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index aeab71d9df603..18547d459eb5c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -841,7 +841,7 @@ private[spark] object JsonProtocol { val accumUpdates = Utils.jsonOption(json \ "Accumulator Updates") .map(_.extract[List[JValue]].map(accumulableInfoFromJson)) .getOrElse(taskMetricsFromJson(json \ "Metrics").accumulators().map(acc => { - acc.toInfo(Some(acc.localValue), None) + acc.toInfo(Some(acc.value), None) })) ExceptionFailure(className, description, stackTrace, fullStackTrace, None, accumUpdates) case `taskResultLost` => TaskResultLost diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index cade67b1d2bae..6cbd5ae5d428a 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -70,7 +70,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex // serialize and de-serialize it, to simulate sending accumulator to executor. val acc2 = ser.deserialize[LongAccumulator](ser.serialize(acc)) // value is reset on the executors - assert(acc2.localValue == 0) + assert(acc2.value == 0) assert(!acc2.isAtDriverSide) acc2.add(10) @@ -259,7 +259,7 @@ private[spark] object AccumulatorSuite { * Make an [[AccumulableInfo]] out of an [[Accumulable]] with the intent to use the * info as an accumulator update. */ - def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.localValue), None) + def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None) /** * Run one or more Spark jobs and verify that in at least one job the peak execution memory diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 0f68aaaee1f21..f82e0b8bca77a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -34,7 +34,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato override def copyAndReset(): SQLMetric = new SQLMetric(metricType, initValue) override def merge(other: AccumulatorV2[Long, Long]): Unit = other match { - case o: SQLMetric => _value += o.localValue + case o: SQLMetric => _value += o.value case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } @@ -45,7 +45,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato def +=(v: Long): Unit = _value += v - override def localValue: Long = _value + override def value: Long = _value // Provide special identifier as metadata so we can tell that this is a `SQLMetric` later private[spark] override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 9118593c0e4ce..29c54111ea7bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -164,7 +164,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi taskEnd.taskInfo.taskId, taskEnd.stageId, taskEnd.stageAttemptId, - taskEnd.taskMetrics.accumulators().map(a => a.toInfo(Some(a.localValue), None)), + taskEnd.taskMetrics.accumulators().map(a => a.toInfo(Some(a.value), None)), finishTask = true) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 8de4d8bbd4e07..d41e88a0aa853 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -302,7 +302,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("metrics can be loaded by history server") { val metric = SQLMetrics.createMetric(sparkContext, "zanzibar") metric += 10L - val metricInfo = metric.toInfo(Some(metric.localValue), None) + val metricInfo = metric.toInfo(Some(metric.value), None) metricInfo.update match { case Some(v: Long) => assert(v === 10L) case Some(v) => fail(s"metric value was not a Long: ${v.getClass.getName}") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 964787015a877..5e08658e5efa2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -366,7 +366,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { // The listener should only track the ones that are actually SQL metrics. val sqlMetric = SQLMetrics.createMetric(sparkContext, "beach umbrella") val nonSqlMetric = sparkContext.accumulator[Int](0, "baseball") - val sqlMetricInfo = sqlMetric.toInfo(Some(sqlMetric.localValue), None) + val sqlMetricInfo = sqlMetric.toInfo(Some(sqlMetric.value), None) val nonSqlMetricInfo = nonSqlMetric.toInfo(Some(nonSqlMetric.localValue), None) val taskInfo = createTaskInfo(0, 0) taskInfo.accumulables ++= Seq(sqlMetricInfo, nonSqlMetricInfo) From 69e030226b9546bb411280635087fbafaea3848b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Garillot?= Date: Tue, 3 May 2016 11:42:47 -0700 Subject: [PATCH 0035/1470] [SPARK-9819][STREAMING][DOCUMENTATION] Clarify doc for invReduceFunc in incremental versions of reduceByWindow MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - that reduceFunc and invReduceFunc should be associative - that the intermediate result in iterated applications of inverseReduceFunc is its first argument Author: François Garillot Closes #8103 from huitseeker/issue/invReduceFuncDoc. (cherry picked from commit 439e361010e51d2213c92ccabed5093be92a72ee) Signed-off-by: Shixiong Zhu --- python/pyspark/streaming/dstream.py | 4 +++- .../org/apache/spark/streaming/api/java/JavaDStreamLike.scala | 3 ++- .../org/apache/spark/streaming/api/java/JavaPairDStream.scala | 3 ++- .../scala/org/apache/spark/streaming/dstream/DStream.scala | 3 ++- .../apache/spark/streaming/dstream/PairDStreamFunctions.scala | 3 ++- 5 files changed, 11 insertions(+), 5 deletions(-) diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 2056663872198..67a0819601efa 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -454,7 +454,9 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio This is more efficient than `invReduceFunc` is None. @param reduceFunc: associative and commutative reduce function - @param invReduceFunc: inverse reduce function of `reduceFunc` + @param invReduceFunc: inverse reduce function of `reduceFunc`; such that for all y, + and invertible x: + `invReduceFunc(reduceFunc(x, y), x) = y` @param windowDuration: width of the window; must be a multiple of this DStream's batching interval @param slideDuration: sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 43632f37ccb16..a0a40fcee26d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -240,7 +240,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2a80cf4466588..dec983165fb3b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -336,7 +336,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse function + * @param invReduceFunc inverse function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 583f5a48d1a6c..01dcfcf24b0fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -793,7 +793,8 @@ abstract class DStream[T: ClassTag] ( * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index b6394e36b5152..2f2a6d13dd79b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -290,7 +290,8 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param reduceFunc associative and commutative reduce function - * @param invReduceFunc inverse reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which From a08d2c9541d0962a6344b0f4e7000b86a753de87 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 11:45:51 -0700 Subject: [PATCH 0036/1470] [SPARK-15082][CORE] Improve unit test coverage for AccumulatorV2 ## What changes were proposed in this pull request? Added tests for ListAccumulator and LegacyAccumulatorWrapper, test for ListAccumulator is one similar to old Collection Accumulators ## How was this patch tested? Ran tests locally. cc rxin Author: Sandeep Singh Closes #12862 from techaddict/SPARK-15082. (cherry picked from commit 84b3a4a873de8444eda1bd162ed52f79941ec864) Signed-off-by: Reynold Xin --- .../spark/util/AccumulatorV2Suite.scala | 61 ++++++++++++++++++- 1 file changed, 60 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala index 815b134884016..ecaf4f0c64364 100644 --- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import org.apache.spark.SparkFunSuite +import org.apache.spark._ class AccumulatorV2Suite extends SparkFunSuite { @@ -86,4 +86,63 @@ class AccumulatorV2Suite extends SparkFunSuite { assert(acc.sum == 5.0) assert(acc.avg == 1.25) } + + test("ListAccumulator") { + val acc = new ListAccumulator[Double] + assert(acc.value.isEmpty) + assert(acc.isZero) + + acc.add(0.0) + assert(acc.value.contains(0.0)) + assert(!acc.isZero) + + acc.add(new java.lang.Double(1.0)) + + val acc2 = acc.copyAndReset() + assert(acc2.value.isEmpty) + assert(acc2.isZero) + + assert(acc.value.contains(1.0)) + assert(!acc.isZero) + assert(acc.value.size() === 2) + + acc2.add(2.0) + assert(acc2.value.contains(2.0)) + assert(!acc2.isZero) + assert(acc2.value.size() === 1) + + // Test merging + acc.merge(acc2) + assert(acc.value.contains(2.0)) + assert(!acc.isZero) + assert(acc.value.size() === 3) + } + + test("LegacyAccumulatorWrapper") { + val acc = new LegacyAccumulatorWrapper("default", AccumulatorParam.StringAccumulatorParam) + assert(acc.value === "default") + assert(!acc.isZero) + + acc.add("foo") + assert(acc.value === "foo") + assert(!acc.isZero) + + acc.add(new java.lang.String("bar")) + + val acc2 = acc.copyAndReset() + assert(acc2.value === "") + assert(acc2.isZero) + + assert(acc.value === "bar") + assert(!acc.isZero) + + acc2.add("baz") + assert(acc2.value === "baz") + assert(!acc2.isZero) + + // Test merging + acc.merge(acc2) + assert(acc.value === "baz") + assert(!acc.isZero) + } } From 50adfcdce394f49057f768aa4af5164f96703ed9 Mon Sep 17 00:00:00 2001 From: Zheng Tan Date: Tue, 3 May 2016 12:22:52 -0700 Subject: [PATCH 0037/1470] [SPARK-15059][CORE] Remove fine-grained lock in ChildFirstURLClassLoader to avoid dead lock ## What changes were proposed in this pull request? In some cases, fine-grained lock have race condition with class-loader lock and have caused dead lock issue. It is safe to drop this fine grained lock and load all classes by single class-loader lock. Author: Zheng Tan Closes #12857 from tankkyo/master. (cherry picked from commit f5623b460224ce363316c63f5d28947215078fc5) Signed-off-by: Shixiong Zhu --- .../spark/util/MutableURLClassLoader.scala | 31 +++---------------- 1 file changed, 5 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala index 0a3180da87987..034826c57ef1d 100644 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -19,7 +19,6 @@ package org.apache.spark.util import java.net.{URL, URLClassLoader} import java.util.Enumeration -import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -48,32 +47,12 @@ private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoa private val parentClassLoader = new ParentClassLoader(parent) - /** - * Used to implement fine-grained class loading locks similar to what is done by Java 7. This - * prevents deadlock issues when using non-hierarchical class loaders. - * - * Note that due to some issues with implementing class loaders in - * Scala, Java 7's `ClassLoader.registerAsParallelCapable` method is not called. - */ - private val locks = new ConcurrentHashMap[String, Object]() - override def loadClass(name: String, resolve: Boolean): Class[_] = { - var lock = locks.get(name) - if (lock == null) { - val newLock = new Object() - lock = locks.putIfAbsent(name, newLock) - if (lock == null) { - lock = newLock - } - } - - lock.synchronized { - try { - super.loadClass(name, resolve) - } catch { - case e: ClassNotFoundException => - parentClassLoader.loadClass(name, resolve) - } + try { + super.loadClass(name, resolve) + } catch { + case e: ClassNotFoundException => + parentClassLoader.loadClass(name, resolve) } } From 4bacdebe95c380a281bb2fc02270dc181e26aa14 Mon Sep 17 00:00:00 2001 From: Devaraj K Date: Tue, 3 May 2016 13:25:28 -0700 Subject: [PATCH 0038/1470] [SPARK-14234][CORE] Executor crashes for TaskRunner thread interruption ## What changes were proposed in this pull request? Resetting the task interruption status before updating the task status. ## How was this patch tested? I have verified it manually by running multiple applications, Executor doesn't crash and updates the status to the driver without any exceptions with the patch changes. Author: Devaraj K Closes #12031 from devaraj-kavali/SPARK-14234. (cherry picked from commit 659f635d3bd0c0d025bf514dfb1747ed7386ba45) Signed-off-by: Shixiong Zhu --- .../org/apache/spark/executor/Executor.scala | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 64e87a95d0342..b695aecc13ea1 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,6 +23,7 @@ import java.net.URL import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -194,6 +195,10 @@ private[spark] class Executor( /** Whether this task has been killed. */ @volatile private var killed = false + /** Whether this task has been finished. */ + @GuardedBy("TaskRunner.this") + private var finished = false + /** How much the JVM process has spent in GC when the task starts to run. */ @volatile var startGCTime: Long = _ @@ -207,10 +212,25 @@ private[spark] class Executor( logInfo(s"Executor is trying to kill $taskName (TID $taskId)") killed = true if (task != null) { - task.kill(interruptThread) + synchronized { + if (!finished) { + task.kill(interruptThread) + } + } } } + /** + * Set the finished flag to true and clear the current thread's interrupt status + */ + private def setTaskFinishedAndClearInterruptStatus(): Unit = synchronized { + this.finished = true + // SPARK-14234 - Reset the interrupted status of the thread to avoid the + // ClosedByInterruptException during execBackend.statusUpdate which causes + // Executor to crash + Thread.interrupted() + } + override def run(): Unit = { val taskMemoryManager = new TaskMemoryManager(env.memoryManager, taskId) val deserializeStartTime = System.currentTimeMillis() @@ -336,14 +356,17 @@ private[spark] class Executor( } catch { case ffe: FetchFailedException => val reason = ffe.toTaskEndReason + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) case _: TaskKilledException | _: InterruptedException if task.killed => logInfo(s"Executor killed $taskName (TID $taskId)") + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) case CausedBy(cDE: CommitDeniedException) => val reason = cDE.toTaskEndReason + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) case t: Throwable => @@ -373,6 +396,7 @@ private[spark] class Executor( ser.serialize(new ExceptionFailure(t, accUpdates, false).withAccums(accums)) } } + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) // Don't forcibly exit unless the exception was inherently fatal, to avoid From a2369c63f905e31c78eb64e547b279b33e30a07c Mon Sep 17 00:00:00 2001 From: yzhou2001 Date: Tue, 3 May 2016 13:41:04 -0700 Subject: [PATCH 0039/1470] [SPARK-14521] [SQL] StackOverflowError in Kryo when executing TPC-DS ## What changes were proposed in this pull request? Observed stackOverflowError in Kryo when executing TPC-DS Query27. Spark thrift server disables kryo reference tracking (if not specified in conf). When "spark.kryo.referenceTracking" is set to true explicitly in spark-defaults.conf, query executes successfully. The root cause is that the TaskMemoryManager inside MemoryConsumer and LongToUnsafeRowMap were not transient and thus were serialized and broadcast around from within LongHashedRelation, which could potentially cause circular reference inside Kryo. But the TaskMemoryManager is per task and should not be passed around at the first place. This fix makes it transient. ## How was this patch tested? core/test, hive/test, sql/test, catalyst/test, dev/lint-scala, org.apache.spark.sql.hive.execution.HiveCompatibilitySuite, dev/scalastyle, manual test of TBC-DS Query 27 with 1GB data but without the "limit 100" which would cause a NPE due to SPARK-14752. Author: yzhou2001 Closes #12598 from yzhou2001/master. (cherry picked from commit a4aed71719b4fc728de93afc623aef05d27bc89a) Signed-off-by: Davies Liu --- .../sql/execution/joins/HashedRelation.scala | 136 ++++++++++++------ .../execution/joins/HashedRelationSuite.scala | 35 +++++ 2 files changed, 129 insertions(+), 42 deletions(-) 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 b280c76c70a61..315ef6a8796f0 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,7 +17,10 @@ package org.apache.spark.sql.execution.joins -import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import java.io._ + +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.{SparkConf, SparkEnv, SparkException} import org.apache.spark.memory.{MemoryConsumer, MemoryMode, StaticMemoryManager, TaskMemoryManager} @@ -116,7 +119,7 @@ private[execution] object HashedRelation { private[joins] class UnsafeHashedRelation( private var numFields: Int, private var binaryMap: BytesToBytesMap) - extends HashedRelation with Externalizable { + extends HashedRelation with Externalizable with KryoSerializable { private[joins] def this() = this(0, null) // Needed for serialization @@ -171,10 +174,21 @@ private[joins] class UnsafeHashedRelation( } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - out.writeInt(numFields) + write(out.writeInt, out.writeLong, out.write) + } + + override def write(kryo: Kryo, out: Output): Unit = Utils.tryOrIOException { + write(out.writeInt, out.writeLong, out.write) + } + + private def write( + writeInt: (Int) => Unit, + writeLong: (Long) => Unit, + writeBuffer: (Array[Byte], Int, Int) => Unit) : Unit = { + writeInt(numFields) // TODO: move these into BytesToBytesMap - out.writeLong(binaryMap.numKeys()) - out.writeLong(binaryMap.numValues()) + writeLong(binaryMap.numKeys()) + writeLong(binaryMap.numValues()) var buffer = new Array[Byte](64) def write(base: Object, offset: Long, length: Int): Unit = { @@ -182,25 +196,32 @@ private[joins] class UnsafeHashedRelation( buffer = new Array[Byte](length) } Platform.copyMemory(base, offset, buffer, Platform.BYTE_ARRAY_OFFSET, length) - out.write(buffer, 0, length) + writeBuffer(buffer, 0, length) } val iter = binaryMap.iterator() while (iter.hasNext) { val loc = iter.next() // [key size] [values size] [key bytes] [value bytes] - out.writeInt(loc.getKeyLength) - out.writeInt(loc.getValueLength) + writeInt(loc.getKeyLength) + writeInt(loc.getValueLength) write(loc.getKeyBase, loc.getKeyOffset, loc.getKeyLength) write(loc.getValueBase, loc.getValueOffset, loc.getValueLength) } } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - numFields = in.readInt() + read(in.readInt, in.readLong, in.readFully) + } + + private def read( + readInt: () => Int, + readLong: () => Long, + readBuffer: (Array[Byte], Int, Int) => Unit): Unit = { + numFields = readInt() resultRow = new UnsafeRow(numFields) - val nKeys = in.readLong() - val nValues = in.readLong() + val nKeys = readLong() + val nValues = readLong() // This is used in Broadcast, shared by multiple tasks, so we use on-heap memory // TODO(josh): This needs to be revisited before we merge this patch; making this change now // so that tests compile: @@ -227,16 +248,16 @@ private[joins] class UnsafeHashedRelation( var keyBuffer = new Array[Byte](1024) var valuesBuffer = new Array[Byte](1024) while (i < nValues) { - val keySize = in.readInt() - val valuesSize = in.readInt() + val keySize = readInt() + val valuesSize = readInt() if (keySize > keyBuffer.length) { keyBuffer = new Array[Byte](keySize) } - in.readFully(keyBuffer, 0, keySize) + readBuffer(keyBuffer, 0, keySize) if (valuesSize > valuesBuffer.length) { valuesBuffer = new Array[Byte](valuesSize) } - in.readFully(valuesBuffer, 0, valuesSize) + readBuffer(valuesBuffer, 0, valuesSize) val loc = binaryMap.lookup(keyBuffer, Platform.BYTE_ARRAY_OFFSET, keySize) val putSuceeded = loc.append(keyBuffer, Platform.BYTE_ARRAY_OFFSET, keySize, @@ -248,6 +269,10 @@ private[joins] class UnsafeHashedRelation( i += 1 } } + + override def read(kryo: Kryo, in: Input): Unit = Utils.tryOrIOException { + read(in.readInt, in.readLong, in.readBytes) + } } private[joins] object UnsafeHashedRelation { @@ -324,8 +349,8 @@ private[joins] object UnsafeHashedRelation { * * see http://java-performance.info/implementing-world-fastest-java-int-to-int-hash-map/ */ -private[execution] final class LongToUnsafeRowMap(var mm: TaskMemoryManager, capacity: Int) - extends MemoryConsumer(mm) with Externalizable { +private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, capacity: Int) + extends MemoryConsumer(mm) with Externalizable with KryoSerializable { // Whether the keys are stored in dense mode or not. private var isDense = false @@ -624,58 +649,85 @@ private[execution] final class LongToUnsafeRowMap(var mm: TaskMemoryManager, cap } } - private def writeLongArray(out: ObjectOutput, arr: Array[Long], len: Int): Unit = { + private def writeLongArray( + writeBuffer: (Array[Byte], Int, Int) => Unit, + arr: Array[Long], + len: Int): Unit = { val buffer = new Array[Byte](4 << 10) var offset: Long = Platform.LONG_ARRAY_OFFSET val end = len * 8L + Platform.LONG_ARRAY_OFFSET while (offset < end) { val size = Math.min(buffer.length, (end - offset).toInt) Platform.copyMemory(arr, offset, buffer, Platform.BYTE_ARRAY_OFFSET, size) - out.write(buffer, 0, size) + writeBuffer(buffer, 0, size) offset += size } } - override def writeExternal(out: ObjectOutput): Unit = { - out.writeBoolean(isDense) - out.writeLong(minKey) - out.writeLong(maxKey) - out.writeLong(numKeys) - out.writeLong(numValues) - - out.writeLong(array.length) - writeLongArray(out, array, array.length) + private def write( + writeBoolean: (Boolean) => Unit, + writeLong: (Long) => Unit, + writeBuffer: (Array[Byte], Int, Int) => Unit): Unit = { + writeBoolean(isDense) + writeLong(minKey) + writeLong(maxKey) + writeLong(numKeys) + writeLong(numValues) + + writeLong(array.length) + writeLongArray(writeBuffer, array, array.length) val used = ((cursor - Platform.LONG_ARRAY_OFFSET) / 8).toInt - out.writeLong(used) - writeLongArray(out, page, used) + writeLong(used) + writeLongArray(writeBuffer, page, used) } - private def readLongArray(in: ObjectInput, length: Int): Array[Long] = { + override def writeExternal(output: ObjectOutput): Unit = { + write(output.writeBoolean, output.writeLong, output.write) + } + + override def write(kryo: Kryo, out: Output): Unit = { + write(out.writeBoolean, out.writeLong, out.write) + } + + private def readLongArray( + readBuffer: (Array[Byte], Int, Int) => Unit, + length: Int): Array[Long] = { val array = new Array[Long](length) val buffer = new Array[Byte](4 << 10) var offset: Long = Platform.LONG_ARRAY_OFFSET val end = length * 8L + Platform.LONG_ARRAY_OFFSET while (offset < end) { val size = Math.min(buffer.length, (end - offset).toInt) - in.readFully(buffer, 0, size) + readBuffer(buffer, 0, size) Platform.copyMemory(buffer, Platform.BYTE_ARRAY_OFFSET, array, offset, size) offset += size } array } - override def readExternal(in: ObjectInput): Unit = { - isDense = in.readBoolean() - minKey = in.readLong() - maxKey = in.readLong() - numKeys = in.readLong - numValues = in.readLong() + private def read( + readBoolean: () => Boolean, + readLong: () => Long, + readBuffer: (Array[Byte], Int, Int) => Unit): Unit = { + isDense = readBoolean() + minKey = readLong() + maxKey = readLong() + numKeys = readLong() + numValues = readLong() - val length = in.readLong().toInt + val length = readLong().toInt mask = length - 2 - array = readLongArray(in, length) - val pageLength = in.readLong().toInt - page = readLongArray(in, pageLength) + array = readLongArray(readBuffer, length) + val pageLength = readLong().toInt + page = readLongArray(readBuffer, pageLength) + } + + override def readExternal(in: ObjectInput): Unit = { + read(in.readBoolean, in.readLong, in.readFully) + } + + override def read(kryo: Kryo, in: Input): Unit = { + read(in.readBoolean, in.readLong, in.readBytes) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 3ee25c0996035..9826a64fe280a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -21,6 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.SharedSQLContext @@ -151,6 +152,40 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { } } + test("Spark-14521") { + val ser = new KryoSerializer( + (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() + val key = Seq(BoundReference(0, IntegerType, false)) + + // Testing Kryo serialization of HashedRelation + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, IntegerType, false), BoundReference(1, IntegerType, true))) + val rows = (0 until 100).map(i => unsafeProj(InternalRow(i, i + 1)).copy()) + val longRelation = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) + val longRelation2 = ser.deserialize[LongHashedRelation](ser.serialize(longRelation)) + (0 until 100).foreach { i => + val rows = longRelation2.get(i).toArray + assert(rows.length === 2) + assert(rows(0).getInt(0) === i) + assert(rows(0).getInt(1) === i + 1) + assert(rows(1).getInt(0) === i) + assert(rows(1).getInt(1) === i + 1) + } + + // Testing Kryo serialization of UnsafeHashedRelation + val unsafeHashed = UnsafeHashedRelation(rows.iterator, key, 1, mm) + val os = new ByteArrayOutputStream() + val out = new ObjectOutputStream(os) + unsafeHashed.asInstanceOf[UnsafeHashedRelation].writeExternal(out) + out.flush() + val unsafeHashed2 = ser.deserialize[UnsafeHashedRelation](ser.serialize(unsafeHashed)) + val os2 = new ByteArrayOutputStream() + val out2 = new ObjectOutputStream(os2) + unsafeHashed2.writeExternal(out2) + out2.flush() + assert(java.util.Arrays.equals(os.toByteArray, os2.toByteArray)) + } + // This test require 4G heap to run, should run it manually ignore("build HashedRelation that is larger than 1G") { val unsafeProj = UnsafeProjection.create( From b67668bc4f6d79aeeccc63578e6f0f7a7febd48d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 3 May 2016 13:43:20 -0700 Subject: [PATCH 0040/1470] [SPARK-11316] coalesce doesn't handle UnionRDD with partial locality properly ## What changes were proposed in this pull request? coalesce doesn't handle UnionRDD with partial locality properly. I had a user who had a UnionRDD that was made up of mapPartitionRDD without preferred locations and a checkpointedRDD with preferred locations (getting from hdfs). It took the driver over 20 minutes to setup the groups and put the partitions into those groups before it even started any tasks. Even perhaps worse is it didn't end up with the number of partitions he was asking for because it didn't put a partition in each of the groups properly. The changes in this patch get rid of a n^2 while loop that was causing the 20 minutes, it properly distributes the partitions to have at least one per group, and it changes from using the rotation iterator which got the preferred locations many times to get all the preferred locations once up front. Note that the n^2 while loop that I removed in setupGroups took so long because all of the partitions with preferred locations were already assigned to group, so it basically looped through every single one and wasn't ever able to assign it. At the time I had 960 partitions with preferred locations and 1020 without and did the outer while loop 319 times because that is the # of groups left to create. Note that each of those times through the inner while loop is going off to hdfs to get the block locations, so this is extremely inefficient. ## How was the this patch tested? Added unit tests for this case and ran existing ones that applied to make sure no regressions. Also manually tested on the users production job to make sure it fixed their issue. It created the proper number of partitions and now it takes about 6 seconds rather then 20 minutes. I did also run some basic manual tests with spark-shell doing coalesced to smaller number, same number, and then greater with shuffle. Author: Thomas Graves Closes #11327 from tgravescs/SPARK-11316. (cherry picked from commit 83ee92f60345f016a390d61a82f1d924f64ddf90) Signed-off-by: Davies Liu --- .../org/apache/spark/rdd/CoalescedRDD.scala | 158 +++++++++++------- .../scala/org/apache/spark/rdd/RDDSuite.scala | 69 ++++++++ 2 files changed, 165 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index e75f1dbf8107a..c19ed1529bbf6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -169,42 +169,37 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) var noLocality = true // if true if no preferredLocations exists for parent RDD - // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currPrefLocs(part: Partition, prev: RDD[_]): Seq[String] = { - prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) - } - - // this class just keeps iterating and rotating infinitely over the partitions of the RDD - // next() returns the next preferred machine that a partition is replicated on - // the rotator first goes through the first replica copy of each partition, then second, third - // the iterators return type is a tuple: (replicaString, partition) - class LocationIterator(prev: RDD[_]) extends Iterator[(String, Partition)] { - - var it: Iterator[(String, Partition)] = resetIterator() - - override val isEmpty = !it.hasNext - - // initializes/resets to start iterating from the beginning - def resetIterator(): Iterator[(String, Partition)] = { - val iterators = (0 to 2).map { x => - prev.partitions.iterator.flatMap { p => - if (currPrefLocs(p, prev).size > x) Some((currPrefLocs(p, prev)(x), p)) else None + class PartitionLocations(prev: RDD[_]) { + + // contains all the partitions from the previous RDD that don't have preferred locations + val partsWithoutLocs = ArrayBuffer[Partition]() + // contains all the partitions from the previous RDD that have preferred locations + val partsWithLocs = ArrayBuffer[(String, Partition)]() + + getAllPrefLocs(prev) + + // gets all the preffered locations of the previous RDD and splits them into partitions + // with preferred locations and ones without + def getAllPrefLocs(prev: RDD[_]) { + val tmpPartsWithLocs = mutable.LinkedHashMap[Partition, Seq[String]]() + // first get the locations for each partition, only do this once since it can be expensive + prev.partitions.foreach(p => { + val locs = prev.context.getPreferredLocs(prev, p.index).map(tl => tl.host) + if (locs.size > 0) { + tmpPartsWithLocs.put(p, locs) + } else { + partsWithoutLocs += p + } } - } - iterators.reduceLeft((x, y) => x ++ y) - } - - // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD - override def hasNext: Boolean = { !isEmpty } - - // return the next preferredLocation of some partition of the RDD - override def next(): (String, Partition) = { - if (it.hasNext) { - it.next() - } else { - it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning - it.next() - } + ) + // convert it into an array of host to partition + (0 to 2).map(x => + tmpPartsWithLocs.foreach(parts => { + val p = parts._1 + val locs = parts._2 + if (locs.size > x) partsWithLocs += ((locs(x), p)) + } ) + ) } } @@ -228,33 +223,32 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } /** - * Initializes targetLen partition groups and assigns a preferredLocation - * This uses coupon collector to estimate how many preferredLocations it must rotate through - * until it has seen most of the preferred locations (2 * n log(n)) + * Initializes targetLen partition groups. If there are preferred locations, each group + * is assigned a preferredLocation. This uses coupon collector to estimate how many + * preferredLocations it must rotate through until it has seen most of the preferred + * locations (2 * n log(n)) * @param targetLen */ - def setupGroups(targetLen: Int, prev: RDD[_]) { - val rotIt = new LocationIterator(prev) - + def setupGroups(targetLen: Int, partitionLocs: PartitionLocations) { // deal with empty case, just create targetLen partition groups with no preferred location - if (!rotIt.hasNext) { + if (partitionLocs.partsWithLocs.isEmpty) { (1 to targetLen).foreach(x => groupArr += new PartitionGroup()) return } noLocality = false - // number of iterations needed to be certain that we've seen most preferred locations val expectedCoupons2 = 2 * (math.log(targetLen)*targetLen + targetLen + 0.5).toInt var numCreated = 0 var tries = 0 // rotate through until either targetLen unique/distinct preferred locations have been created - // OR we've rotated expectedCoupons2, in which case we have likely seen all preferred locations, - // i.e. likely targetLen >> number of preferred locations (more buckets than there are machines) - while (numCreated < targetLen && tries < expectedCoupons2) { + // OR (we have went through either all partitions OR we've rotated expectedCoupons2 - in + // which case we have likely seen all preferred locations) + val numPartsToLookAt = math.min(expectedCoupons2, partitionLocs.partsWithLocs.length) + while (numCreated < targetLen && tries < numPartsToLookAt) { + val (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) tries += 1 - val (nxt_replica, nxt_part) = rotIt.next() if (!groupHash.contains(nxt_replica)) { val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup @@ -263,20 +257,18 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) numCreated += 1 } } - - while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates - var (nxt_replica, nxt_part) = rotIt.next() + tries = 0 + // if we don't have enough partition groups, create duplicates + while (numCreated < targetLen) { + var (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) + tries += 1 val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup groupHash.getOrElseUpdate(nxt_replica, ArrayBuffer()) += pgroup - var tries = 0 - while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure at least one part - nxt_part = rotIt.next()._2 - tries += 1 - } + addPartToPGroup(nxt_part, pgroup) numCreated += 1 + if (tries >= partitionLocs.partsWithLocs.length) tries = 0 } - } /** @@ -289,10 +281,15 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) * imbalance in favor of locality * @return partition group (bin to be put in) */ - def pickBin(p: Partition, prev: RDD[_], balanceSlack: Double): PartitionGroup = { + def pickBin( + p: Partition, + prev: RDD[_], + balanceSlack: Double, + partitionLocs: PartitionLocations): PartitionGroup = { val slack = (balanceSlack * prev.partitions.length).toInt + val preflocs = partitionLocs.partsWithLocs.filter(_._2 == p).map(_._1).toSeq // least loaded pref locs - val pref = currPrefLocs(p, prev).map(getLeastGroupHash(_)).sortWith(compare) + val pref = preflocs.map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) @@ -320,7 +317,10 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } } - def throwBalls(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { + def throwBalls( + maxPartitions: Int, + prev: RDD[_], + balanceSlack: Double, partitionLocs: PartitionLocations) { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p, i) <- prev.partitions.zipWithIndex) { @@ -334,8 +334,39 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } } } else { + // It is possible to have unionRDD where one rdd has preferred locations and another rdd + // that doesn't. To make sure we end up with the requested number of partitions, + // make sure to put a partition in every group. + + // if we don't have a partition assigned to every group first try to fill them + // with the partitions with preferred locations + val partIter = partitionLocs.partsWithLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partIter.hasNext && pg.numPartitions == 0) { + var (nxt_replica, nxt_part) = partIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // if we didn't get one partitions per group from partitions with preferred locations + // use partitions without preferred locations + val partNoLocIter = partitionLocs.partsWithoutLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partNoLocIter.hasNext && pg.numPartitions == 0) { + var nxt_part = partNoLocIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // finally pick bin for the rest for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group - pickBin(p, prev, balanceSlack).partitions += p + pickBin(p, prev, balanceSlack, partitionLocs).partitions += p } } } @@ -349,8 +380,11 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) * @return array of partition groups */ def coalesce(maxPartitions: Int, prev: RDD[_]): Array[PartitionGroup] = { - setupGroups(math.min(prev.partitions.length, maxPartitions), prev) // setup the groups (bins) - throwBalls(maxPartitions, prev, balanceSlack) // assign partitions (balls) to each group (bins) + val partitionLocs = new PartitionLocations(prev) + // setup the groups (bins) + setupGroups(math.min(prev.partitions.length, maxPartitions), partitionLocs) + // assign partitions (balls) to each group (bins) + throwBalls(maxPartitions, prev, balanceSlack, partitionLocs) getPartitions } } 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 8dc463d56d182..a663dab772bf9 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -377,6 +377,33 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") } + test("coalesced RDDs with partial locality") { + // Make an RDD that has some locality preferences and some without. This can happen + // with UnionRDD + val data = sc.makeRDD((1 to 9).map(i => { + if (i > 4) { + (i, (i to (i + 2)).map { j => "m" + (j % 6) }) + } else { + (i, Vector()) + } + })) + val coalesced1 = data.coalesce(3) + assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") + + val splits = coalesced1.glom().collect().map(_.toList).toList + assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) + + assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") + + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = data.coalesce(20) + val listOfLists = coalesced4.glom().collect().map(_.toList).toList + val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) } + assert(sortedList === (1 to 9). + map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") + } + test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable @@ -418,6 +445,48 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("coalesced RDDs with partial locality, large scale (10K partitions)") { + // large scale experiment + import collection.mutable + val halfpartitions = 5000 + val partitions = 10000 + val numMachines = 50 + val machines = mutable.ListBuffer[String]() + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) + + val firstBlocks = (1 to halfpartitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } + val blocksNoLocality = (halfpartitions + 1 to partitions).map { i => + (i, List()) + } + val blocks = firstBlocks ++ blocksNoLocality + + val data2 = sc.makeRDD(blocks) + + // first try going to same number of partitions + val coalesced2 = data2.coalesce(partitions) + + // test that we have 10000 partitions + assert(coalesced2.partitions.size == 10000, "Expected 10000 partitions, but got " + + coalesced2.partitions.size) + + // test that we have 100 partitions + val coalesced3 = data2.coalesce(numMachines * 2) + assert(coalesced3.partitions.size == 100, "Expected 100 partitions, but got " + + coalesced3.partitions.size) + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance3 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance3 <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance3) + } + } + // Test for SPARK-2412 -- ensure that the second pass of the algorithm does not throw an exception test("coalesced RDDs with locality, fail first pass") { val initialPartitions = 1000 From ca9917160849597a839a8b6b39321435734d936b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 May 2016 13:47:58 -0700 Subject: [PATCH 0041/1470] [SPARK-15073][SQL] Hide SparkSession constructor from the public ## What changes were proposed in this pull request? Users should use the builder pattern instead. ## How was this patch tested? Jenks. Author: Andrew Or Closes #12873 from andrewor14/spark-session-constructor. (cherry picked from commit 588cac414a9cf1e0f40a82cc6a78f77e26825f29) Signed-off-by: Reynold Xin --- .../apache/spark/examples/sql/RDDRelation.scala | 9 +++------ .../scala/org/apache/spark/repl/SparkILoop.scala | 4 ++-- .../main/scala/org/apache/spark/repl/Main.scala | 4 ++-- .../scala/org/apache/spark/sql/SparkSession.scala | 14 ++++++++++++-- 4 files changed, 19 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 8ce4427c53997..b4118b16e2918 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.sql -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.{SaveMode, SparkSession} // One method for defining the schema of an RDD is to make a case class with the desired column @@ -27,14 +26,12 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("RDDRelation") - val sc = new SparkContext(sparkConf) - val spark = new SparkSession(sc) + val spark = SparkSession.builder.appName("RDDRelation").getOrCreate() // Importing the SparkSession gives access to all the SQL functions and implicit conversions. import spark.implicits._ - val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF() + val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) // Any RDD containing case classes can be registered as a table. The schema of the table is // automatically inferred using scala reflection. df.registerTempTable("records") @@ -70,7 +67,7 @@ object RDDRelation { parquetFile.registerTempTable("parquetFile") spark.sql("SELECT * FROM parquetFile").collect().foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 6a811adcf9b7a..c4f64505a2165 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1030,10 +1030,10 @@ class SparkILoop( def createSparkSession(): SparkSession = { if (SparkSession.hiveClassesArePresent) { logInfo("Creating Spark session with Hive support") - SparkSession.withHiveSupport(sparkContext) + SparkSession.builder.enableHiveSupport().getOrCreate() } else { logInfo("Creating Spark session") - new SparkSession(sparkContext) + SparkSession.builder.getOrCreate() } } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 8e381ff6ae5a1..a171759809256 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -94,10 +94,10 @@ object Main extends Logging { def createSparkSession(): SparkSession = { if (SparkSession.hiveClassesArePresent) { - sparkSession = SparkSession.withHiveSupport(sparkContext) + sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate() logInfo("Created Spark session with Hive support") } else { - sparkSession = new SparkSession(sparkContext) + sparkSession = SparkSession.builder.getOrCreate() logInfo("Created Spark session") } sparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 3836ce2daa56e..aa7c335c53d24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -54,6 +54,7 @@ import org.apache.spark.util.Utils * {{{ * SparkSession.builder() * .master("local") + * .appName("Word Count") * .config("spark.some.config.option", "some-value"). * .getOrCreate() * }}} @@ -63,7 +64,7 @@ class SparkSession private( @transient private val existingSharedState: Option[SharedState]) extends Serializable with Logging { self => - def this(sc: SparkContext) { + private[sql] def this(sc: SparkContext) { this(sc, None) } @@ -573,7 +574,7 @@ class SparkSession private( * common Scala objects into [[DataFrame]]s. * * {{{ - * val sparkSession = new SparkSession(sc) + * val sparkSession = SparkSession.builder.getOrCreate() * import sparkSession.implicits._ * }}} * @@ -586,6 +587,15 @@ class SparkSession private( } // scalastyle:on + /** + * Stop the underlying [[SparkContext]]. + * + * @since 2.0.0 + */ + def stop(): Unit = { + sparkContext.stop() + } + protected[sql] def parseSql(sql: String): LogicalPlan = { sessionState.sqlParser.parsePlan(sql) } From c50746a4a32565f181422aa888eaa4f03f94a25f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 May 2016 14:15:25 -0700 Subject: [PATCH 0042/1470] [SPARK-15095][SQL] drop binary mode in ThriftServer ## What changes were proposed in this pull request? This PR drop the support for binary mode in ThriftServer, only HTTP mode is supported now, to reduce the maintain burden. The code to support binary mode is still kept, just in case if we want it in future. ## How was this patch tested? Updated tests to use HTTP mode. Author: Davies Liu Closes #12876 from davies/hide_binary. (cherry picked from commit d6c7b2a5cc11a82e5137ee86350550e06e81f609) Signed-off-by: Reynold Xin --- .../hive/thriftserver/HiveThriftServer2.scala | 18 ++++---- .../HiveThriftServer2Suites.scala | 41 +++---------------- .../hive/thriftserver/UISeleniumSuite.scala | 2 +- 3 files changed, 14 insertions(+), 47 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 94b1ced9908d8..cef5912c62409 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} +import org.apache.hive.service.cli.thrift.ThriftHttpCLIService import org.apache.hive.service.server.HiveServer2 import org.apache.spark.SparkContext @@ -34,7 +34,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.{HiveSharedState, HiveUtils} +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab import org.apache.spark.sql.internal.SQLConf @@ -271,7 +271,7 @@ object HiveThriftServer2 extends Logging { private[hive] class HiveThriftServer2(sqlContext: SQLContext) extends HiveServer2 - with ReflectedCompositeService { + with ReflectedCompositeService with Logging { // state is tracked internally so that the server only attempts to shut down if it successfully // started, and then once only. private val started = new AtomicBoolean(false) @@ -281,20 +281,18 @@ private[hive] class HiveThriftServer2(sqlContext: SQLContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) - val thriftCliService = if (isHTTPTransportMode(hiveConf)) { - new ThriftHttpCLIService(sparkSqlCliService) - } else { - new ThriftBinaryCLIService(sparkSqlCliService) + if (isBinaryTransportMode(hiveConf)) { + logWarning("Binary mode is not supported, use HTTP mode instead") } - + val thriftCliService = new ThriftHttpCLIService(sparkSqlCliService) setSuperField(this, "thriftCLIService", thriftCliService) addService(thriftCliService) initCompositeService(hiveConf) } - private def isHTTPTransportMode(hiveConf: HiveConf): Boolean = { + private def isBinaryTransportMode(hiveConf: HiveConf): Boolean = { val transportMode = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE) - transportMode.toLowerCase(Locale.ENGLISH).equals("http") + transportMode.toLowerCase(Locale.ENGLISH).equals("binary") } 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 55a93ea06ba57..1a894ae857298 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 @@ -55,8 +55,8 @@ object TestData { val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") } -class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.binary +class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { + override def mode: ServerMode.Value = ServerMode.http private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { // Transport creation logic below mimics HiveConnection.createBinaryTransport @@ -70,7 +70,8 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { try f(client) finally transport.close() } - test("GetInfo Thrift API") { + // TODO: update this test to work in HTTP mode + ignore("GetInfo Thrift API") { withCLIServiceClient { client => val user = System.getProperty("user.name") val sessionHandle = client.openSession(user, "") @@ -566,7 +567,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } class SingleSessionSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.http override protected def extraConf: Seq[String] = "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil @@ -616,38 +617,6 @@ class SingleSessionSuite extends HiveThriftJdbcTest { } } -class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.http - - test("JDBC query execution") { - withJdbcStatement { statement => - val queries = Seq( - "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", - "CACHE TABLE test") - - queries.foreach(statement.execute) - - assertResult(5, "Row count mismatch") { - val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") - resultSet.next() - resultSet.getInt(1) - } - } - } - - test("Checks Hive version") { - withJdbcStatement { statement => - val resultSet = statement.executeQuery("SET spark.sql.hive.version") - resultSet.next() - assert(resultSet.getString(1) === "spark.sql.hive.version") - assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) - } - } -} - object ServerMode extends Enumeration { val binary, http = Value } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index bf431cd6b0260..b6b9de1ba6639 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -36,7 +36,7 @@ class UISeleniumSuite implicit var webDriver: WebDriver = _ var server: HiveThriftServer2 = _ val uiPort = 20000 + Random.nextInt(10000) - override def mode: ServerMode.Value = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.http override def beforeAll(): Unit = { webDriver = new HtmlUnitDriver { From f21eddb2901405cfd9c5fb85a1f73d48e3edb515 Mon Sep 17 00:00:00 2001 From: yinxusen Date: Tue, 3 May 2016 14:19:13 -0700 Subject: [PATCH 0043/1470] [SPARK-14973][ML] The CrossValidator and TrainValidationSplit miss the seed when saving and loading ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14973 Add seed support when saving/loading of CrossValidator and TrainValidationSplit. ## How was this patch tested? Spark unit test. Author: yinxusen Closes #12825 from yinxusen/SPARK-14973. (cherry picked from commit 2e2a6211c4391d67edb2a252f26647fb059bc18b) Signed-off-by: Joseph K. Bradley --- .../apache/spark/ml/tuning/CrossValidator.scala | 17 ++++++++++------- .../spark/ml/tuning/TrainValidationSplit.scala | 17 ++++++++++------- .../spark/ml/tuning/ValidatorParams.scala | 9 +++++---- .../spark/ml/tuning/CrossValidatorSuite.scala | 3 +++ .../ml/tuning/TrainValidationSplitSuite.scala | 2 ++ 5 files changed, 30 insertions(+), 18 deletions(-) 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 a41d02cde755b..7d42da4a2ffae 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 @@ -30,7 +30,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.HasSeed import org.apache.spark.ml.util._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, Dataset} @@ -39,7 +38,7 @@ import org.apache.spark.sql.types.StructType /** * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ -private[ml] trait CrossValidatorParams extends ValidatorParams with HasSeed { +private[ml] trait CrossValidatorParams extends ValidatorParams { /** * Param for number of folds for cross validation. Must be >= 2. * Default: 3 @@ -179,11 +178,13 @@ object CrossValidator extends MLReadable[CrossValidator] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val numFolds = (metadata.params \ "numFolds").extract[Int] + val seed = (metadata.params \ "seed").extract[Long] new CrossValidator(metadata.uid) .setEstimator(estimator) .setEvaluator(evaluator) .setEstimatorParamMaps(estimatorParamMaps) .setNumFolds(numFolds) + .setSeed(seed) } } } @@ -267,14 +268,16 @@ object CrossValidatorModel extends MLReadable[CrossValidatorModel] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val numFolds = (metadata.params \ "numFolds").extract[Int] + val seed = (metadata.params \ "seed").extract[Long] val bestModelPath = new Path(path, "bestModel").toString val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) val avgMetrics = (metadata.metadata \ "avgMetrics").extract[Seq[Double]].toArray - val cv = new CrossValidatorModel(metadata.uid, bestModel, avgMetrics) - cv.set(cv.estimator, estimator) - .set(cv.evaluator, evaluator) - .set(cv.estimatorParamMaps, estimatorParamMaps) - .set(cv.numFolds, numFolds) + val model = new CrossValidatorModel(metadata.uid, bestModel, avgMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + .set(model.numFolds, numFolds) + .set(model.seed, seed) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index f2b7badbe5132..f6f2bad401a17 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -30,7 +30,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{DoubleParam, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.HasSeed import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -38,7 +37,7 @@ import org.apache.spark.sql.types.StructType /** * Params for [[TrainValidationSplit]] and [[TrainValidationSplitModel]]. */ -private[ml] trait TrainValidationSplitParams extends ValidatorParams with HasSeed { +private[ml] trait TrainValidationSplitParams extends ValidatorParams { /** * Param for ratio between train and validation data. Must be between 0 and 1. * Default: 0.75 @@ -177,11 +176,13 @@ object TrainValidationSplit extends MLReadable[TrainValidationSplit] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val trainRatio = (metadata.params \ "trainRatio").extract[Double] + val seed = (metadata.params \ "seed").extract[Long] new TrainValidationSplit(metadata.uid) .setEstimator(estimator) .setEvaluator(evaluator) .setEstimatorParamMaps(estimatorParamMaps) .setTrainRatio(trainRatio) + .setSeed(seed) } } } @@ -265,14 +266,16 @@ object TrainValidationSplitModel extends MLReadable[TrainValidationSplitModel] { val (metadata, estimator, evaluator, estimatorParamMaps) = ValidatorParams.loadImpl(path, sc, className) val trainRatio = (metadata.params \ "trainRatio").extract[Double] + val seed = (metadata.params \ "seed").extract[Long] val bestModelPath = new Path(path, "bestModel").toString val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) val validationMetrics = (metadata.metadata \ "validationMetrics").extract[Seq[Double]].toArray - val tvs = new TrainValidationSplitModel(metadata.uid, bestModel, validationMetrics) - tvs.set(tvs.estimator, estimator) - .set(tvs.evaluator, evaluator) - .set(tvs.estimatorParamMaps, estimatorParamMaps) - .set(tvs.trainRatio, trainRatio) + val model = new TrainValidationSplitModel(metadata.uid, bestModel, validationMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + .set(model.trainRatio, trainRatio) + .set(model.seed, seed) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala index 7a4e106aeb999..26fd73814d70a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala @@ -25,15 +25,15 @@ import org.apache.spark.SparkContext import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} -import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter, MetaAlgorithmReadWrite, - MLWritable} +import org.apache.spark.ml.param.shared.HasSeed +import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter, MetaAlgorithmReadWrite, MLWritable} import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.sql.types.StructType /** * Common params for [[TrainValidationSplitParams]] and [[CrossValidatorParams]]. */ -private[ml] trait ValidatorParams extends Params { +private[ml] trait ValidatorParams extends HasSeed with Params { /** * param for the estimator to be validated @@ -137,7 +137,8 @@ private[ml] object ValidatorParams { } val jsonParams = validatorSpecificParams ++ List( - "estimatorParamMaps" -> parse(estimatorParamMapsJson)) + "estimatorParamMaps" -> parse(estimatorParamMapsJson), + "seed" -> parse(instance.seed.jsonEncode(instance.getSeed))) DefaultParamsWriter.saveMetadata(instance, path, sc, extraMetadata, Some(jsonParams)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index 3e734aabc5544..061d04c93215f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -136,6 +136,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] @@ -186,6 +187,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) assert(cv.getEvaluator.uid === cv2.getEvaluator.uid) @@ -259,6 +261,7 @@ class CrossValidatorSuite assert(cv.uid === cv2.uid) assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index dbee47c8475d7..df9ba418b8978 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -127,6 +127,7 @@ class TrainValidationSplitSuite val tvs2 = testDefaultReadWrite(tvs, testParams = false) assert(tvs.getTrainRatio === tvs2.getTrainRatio) + assert(tvs.getSeed === tvs2.getSeed) } test("read/write: TrainValidationSplitModel") { @@ -149,6 +150,7 @@ class TrainValidationSplitSuite assert(tvs.getTrainRatio === tvs2.getTrainRatio) assert(tvs.validationMetrics === tvs2.validationMetrics) + assert(tvs.getSeed === tvs2.getSeed) } } From 70c5ca220c7548a7ebaba16e5dbf7e9ed47b5338 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 3 May 2016 23:20:18 +0200 Subject: [PATCH 0044/1470] [SPARK-15056][SQL] Parse Unsupported Sampling Syntax and Issue Better Exceptions #### What changes were proposed in this pull request? Compared with the current Spark parser, there are two extra syntax are supported in Hive for sampling - In `On` clauses, `rand()` is used for indicating sampling on the entire row instead of an individual column. For example, ```SQL SELECT * FROM source TABLESAMPLE(BUCKET 3 OUT OF 32 ON rand()) s; ``` - Users can specify the total length to be read. For example, ```SQL SELECT * FROM source TABLESAMPLE(100M) s; ``` Below is the link for references: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Sampling This PR is to parse and capture these two extra syntax, and issue a better error message. #### How was this patch tested? Added test cases to verify the thrown exceptions Author: gatorsmile Closes #12838 from gatorsmile/bucketOnRand. (cherry picked from commit 71296c041e59159bd7c5836cf652c02843974077) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 7 ++++++- .../spark/sql/catalyst/parser/AstBuilder.scala | 12 +++++++++++- .../spark/sql/catalyst/parser/PlanParserSuite.scala | 6 +++++- 3 files changed, 22 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index cc4e5c853e679..3ab448dd9e259 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -389,7 +389,8 @@ sample : TABLESAMPLE '(' ( (percentage=(INTEGER_VALUE | DECIMAL_VALUE) sampleType=PERCENTLIT) | (expression sampleType=ROWS) - | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON identifier)?)) + | sampleType=BYTELENGTH_LITERAL + | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON (identifier | qualifiedName '(' ')'))?)) ')' ; @@ -895,6 +896,10 @@ TINYINT_LITERAL : DIGIT+ 'Y' ; +BYTELENGTH_LITERAL + : DIGIT+ ('B' | 'K' | 'M' | 'G') + ; + INTEGER_VALUE : DIGIT+ ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c3974625aa222..1d4e1ec3b894c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -632,8 +632,18 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val fraction = ctx.percentage.getText.toDouble sample(fraction / 100.0d) + case SqlBaseParser.BYTELENGTH_LITERAL => + throw new ParseException( + "TABLESAMPLE(byteLengthLiteral) is not supported", ctx) + case SqlBaseParser.BUCKET if ctx.ON != null => - throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx) + if (ctx.identifier != null) { + throw new ParseException( + "TABLESAMPLE(BUCKET x OUT OF y ON colname) is not supported", ctx) + } else { + throw new ParseException( + "TABLESAMPLE(BUCKET x OUT OF y ON function) is not supported", ctx) + } case SqlBaseParser.BUCKET => sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index b7af2ceda6437..aaf84268afbe6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -372,9 +372,13 @@ class PlanParserSuite extends PlanTest { assertEqual(s"$sql tablesample(bucket 4 out of 10) as x", Sample(0, .4d, withReplacement = false, 10L, table("t").as("x"))(true).select(star())) intercept(s"$sql tablesample(bucket 4 out of 10 on x) as x", - "TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported") + "TABLESAMPLE(BUCKET x OUT OF y ON colname) is not supported") intercept(s"$sql tablesample(bucket 11 out of 10) as x", s"Sampling fraction (${11.0/10.0}) must be on interval [0, 1]") + intercept("SELECT * FROM parquet_t0 TABLESAMPLE(300M) s", + "TABLESAMPLE(byteLengthLiteral) is not supported") + intercept("SELECT * FROM parquet_t0 TABLESAMPLE(BUCKET 3 OUT OF 32 ON rand()) s", + "TABLESAMPLE(BUCKET x OUT OF y ON function) is not supported") } test("sub-query") { From 51bb0bcc85519be6608cd41a9be4801419314550 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 May 2016 14:40:47 -0700 Subject: [PATCH 0045/1470] [SQL-15102][SQL] remove delegation token support from ThriftServer ## What changes were proposed in this pull request? These API is only useful for Hadoop, may not work for Spark SQL. The APIs is kept for source compatibility. ## How was this patch tested? No unit tests needed. Author: Davies Liu Closes #12878 from davies/remove_delegate. (cherry picked from commit 028c6a5dba01e5d82c34701f40d15916c9d3e9d0) Signed-off-by: Reynold Xin --- .../service/cli/thrift/ThriftCLIService.java | 65 ++----------------- 1 file changed, 7 insertions(+), 58 deletions(-) diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 866beb19f5f91..ad7a9a238f8a9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -18,6 +18,7 @@ package org.apache.hive.service.cli.thrift; +import javax.security.auth.login.LoginException; import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; @@ -25,8 +26,6 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import javax.security.auth.login.LoginException; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; @@ -36,17 +35,7 @@ import org.apache.hive.service.ServiceUtils; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.auth.TSetIpAddressProcessor; -import org.apache.hive.service.cli.CLIService; -import org.apache.hive.service.cli.FetchOrientation; -import org.apache.hive.service.cli.FetchType; -import org.apache.hive.service.cli.GetInfoType; -import org.apache.hive.service.cli.GetInfoValue; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.OperationHandle; -import org.apache.hive.service.cli.OperationStatus; -import org.apache.hive.service.cli.RowSet; -import org.apache.hive.service.cli.SessionHandle; -import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.*; import org.apache.hive.service.cli.session.SessionManager; import org.apache.hive.service.server.HiveServer2; import org.apache.thrift.TException; @@ -223,23 +212,7 @@ public InetAddress getServerIPAddress() { public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws TException { TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); - - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - String token = cliService.getDelegationToken( - new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getOwner(), req.getRenewer()); - resp.setDelegationToken(token); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error obtaining delegation token", e); - TStatus tokenErrorStatus = HiveSQLException.toTStatus(e); - tokenErrorStatus.setSqlState("42000"); - resp.setStatus(tokenErrorStatus); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } @@ -247,19 +220,7 @@ public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws TException { TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); - - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - cliService.cancelDelegationToken(new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getDelegationToken()); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error canceling delegation token", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } @@ -267,25 +228,13 @@ public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenRe public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws TException { TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); - if (hiveAuthFactory == null) { - resp.setStatus(unsecureTokenErrorStatus()); - } else { - try { - cliService.renewDelegationToken(new SessionHandle(req.getSessionHandle()), - hiveAuthFactory, req.getDelegationToken()); - resp.setStatus(OK_STATUS); - } catch (HiveSQLException e) { - LOG.error("Error obtaining renewing token", e); - resp.setStatus(HiveSQLException.toTStatus(e)); - } - } + resp.setStatus(notSupportTokenErrorStatus()); return resp; } - private TStatus unsecureTokenErrorStatus() { + private TStatus notSupportTokenErrorStatus() { TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); - errorStatus.setErrorMessage("Delegation token only supported over remote " + - "client with kerberos authentication"); + errorStatus.setErrorMessage("Delegation token is not supported"); return errorStatus; } From c212307b9a187c626c7162838e34551f6700e611 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Tue, 3 May 2016 14:54:43 -0700 Subject: [PATCH 0046/1470] [SPARK-15104] Fix spacing in log line Otherwise get logs that look like this (note no space before NODE_LOCAL) ``` INFO [2016-05-03 21:18:51,477] org.apache.spark.scheduler.TaskSetManager: Starting task 0.0 in stage 101.0 (TID 7029, localhost, partition 0,NODE_LOCAL, 1894 bytes) ``` Author: Andrew Ash Closes #12880 from ash211/patch-7. (cherry picked from commit dbacd99983a81d35476618be1615807ebba88f05) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 cd634bbf6fe74..bfa1e86749a4b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -479,7 +479,7 @@ private[spark] class TaskSetManager( // val timeTaken = clock.getTime() - startTime val taskName = s"task ${info.id} in stage ${taskSet.id}" logInfo(s"Starting $taskName (TID $taskId, $host, partition ${task.partitionId}," + - s"$taskLocality, ${serializedTask.limit} bytes)") + s" $taskLocality, ${serializedTask.limit} bytes)") sched.dagScheduler.taskStarted(task, info) return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, From 0d16b7f3a5fbb2eb047edca838bcbde9037227a3 Mon Sep 17 00:00:00 2001 From: Shuai Lin Date: Tue, 3 May 2016 18:02:12 -0700 Subject: [PATCH 0047/1470] [MINOR][DOC] Fixed some python snippets in mllib data types documentation. ## What changes were proposed in this pull request? Some python snippets is using scala imports and comments. ## How was this patch tested? Generated the docs locally with `SKIP_API=1 jekyll build` and viewed the changes in the browser. Author: Shuai Lin Closes #12869 from lins05/fix-mllib-python-snippets. (cherry picked from commit c4e0fde876fff259308d1d58ab51ae2697ae31f1) Signed-off-by: Andrew Or --- docs/mllib-data-types.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index 5e3ee472a72c3..2ffe0f1c2b311 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -314,12 +314,12 @@ matrices. Remember, local matrices in MLlib are stored in column-major order. Refer to the [`Matrix` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrix) and [`Matrices` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrices) for more details on the API. {% highlight python %} -import org.apache.spark.mllib.linalg.{Matrix, Matrices} +from pyspark.mllib.linalg import Matrix, Matrices -// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +# Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) dm2 = Matrices.dense(3, 2, [1, 2, 3, 4, 5, 6]) -// Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) +# Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) sm = Matrices.sparse(3, 2, [0, 1, 3], [0, 2, 1], [9, 6, 8]) {% endhighlight %} From 5625b037a0c952b97e1afa6a44443113c0847ade Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 3 May 2016 18:02:57 -0700 Subject: [PATCH 0048/1470] [SPARK-14422][SQL] Improve handling of optional configs in SQLConf ## What changes were proposed in this pull request? Create a new API for handling Optional Configs in SQLConf. Right now `getConf` for `OptionalConfigEntry[T]` returns value of type `T`, if doesn't exist throws an exception. Add new method `getOptionalConf`(suggestions on naming) which will now returns value of type `Option[T]`(so if doesn't exist it returns `None`). ## How was this patch tested? Add test and ran tests locally. Author: Sandeep Singh Closes #12846 from techaddict/SPARK-14422. (cherry picked from commit a8d56f538878443da6eae69449858ad4e2274151) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/sql/DataFrameWriter.scala | 9 +++++---- .../scala/org/apache/spark/sql/RuntimeConfig.scala | 6 +++++- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 9 ++++----- .../apache/spark/sql/internal/SQLConfEntrySuite.scala | 11 +++++++++++ 4 files changed, 25 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index a8f96a9b45b28..0793b62faefbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -296,7 +296,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { new Path(userSpecified).toUri.toString }.orElse { val checkpointConfig: Option[String] = - df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION, None) + df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION) checkpointConfig.map { location => new Path(location, queryName).toUri.toString @@ -334,9 +334,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { partitionColumns = normalizedParCols.getOrElse(Nil)) val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) - val checkpointLocation = extraOptions.getOrElse("checkpointLocation", { - new Path(df.sparkSession.sessionState.conf.checkpointLocation, queryName).toUri.toString - }) + val checkpointLocation = extraOptions.getOrElse("checkpointLocation", + new Path(df.sparkSession.sessionState.conf.checkpointLocation.get, queryName).toUri.toString + ) + df.sparkSession.sessionState.continuousQueryManager.startQuery( queryName, checkpointLocation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 670288b23400e..4fd6e42640d24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.internal.config.ConfigEntry +import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} import org.apache.spark.sql.internal.SQLConf @@ -86,6 +86,10 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { sqlConf.getConf(entry) } + protected[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T] = { + sqlConf.getConf(entry) + } + /** * Returns the value of Spark runtime configuration property for the given key. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0bcf0f817a1d7..5e19984debaa7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -546,7 +546,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def optimizerInSetConversionThreshold: Int = getConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD) - def checkpointLocation: String = getConf(CHECKPOINT_LOCATION) + def checkpointLocation: Option[String] = getConf(CHECKPOINT_LOCATION) def filesMaxPartitionBytes: Long = getConf(FILES_MAX_PARTITION_BYTES) @@ -717,12 +717,11 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { /** * Return the value of an optional Spark SQL configuration property for the given key. If the key - * is not set yet, throw an exception. + * is not set yet, returns None. */ - def getConf[T](entry: OptionalConfigEntry[T]): T = { + def getConf[T](entry: OptionalConfigEntry[T]): Option[T] = { require(sqlConfEntries.get(entry.key) == entry, s"$entry is not registered") - Option(settings.get(entry.key)).map(entry.rawValueConverter). - getOrElse(throw new NoSuchElementException(entry.key)) + Option(settings.get(entry.key)).map(entry.rawValueConverter) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala index cc6919913948d..95bfd05c1f260 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala @@ -153,6 +153,17 @@ class SQLConfEntrySuite extends SparkFunSuite { assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c", "d", "e")) } + test("optionalConf") { + val key = "spark.sql.SQLConfEntrySuite.optional" + val confEntry = SQLConfigBuilder(key) + .stringConf + .createOptional + + assert(conf.getConf(confEntry) === None) + conf.setConfString(key, "a") + assert(conf.getConf(confEntry) === Some("a")) + } + test("duplicate entry") { val key = "spark.sql.SQLConfEntrySuite.duplicate" SQLConfigBuilder(key).stringConf.createOptional From 4c7f5a74d702c3bed0b07cfa498cbe7e653fce03 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 3 May 2016 18:04:04 -0700 Subject: [PATCH 0049/1470] [SPARK-14645][MESOS] Fix python running on cluster mode mesos to have non local uris ## What changes were proposed in this pull request? Fix SparkSubmit to allow non-local python uris ## How was this patch tested? Manually tested with mesos-spark-dispatcher Author: Timothy Chen Closes #12403 from tnachen/enable_remote_python. (cherry picked from commit c1839c9911e37488230a68dec9041eb5958b6f1c) Signed-off-by: Andrew Or --- .../src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 926e1ff7a874d..755c4b6ec11d6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -305,8 +305,9 @@ object SparkSubmit { } // Require all python files to be local, so we can add them to the PYTHONPATH - // In YARN cluster mode, python files are distributed as regular files, which can be non-local - if (args.isPython && !isYarnCluster) { + // In YARN cluster mode, python files are distributed as regular files, which can be non-local. + // In Mesos cluster mode, non-local python files are automatically downloaded by Mesos. + if (args.isPython && !isYarnCluster && !isMesosCluster) { if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { printErrorAndExit(s"Only local python files are supported: $args.primaryResource") } From 5e15615d1258db8c31a8c0c9f9f33965bc14a910 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 May 2016 18:05:40 -0700 Subject: [PATCH 0050/1470] [SPARK-15084][PYTHON][SQL] Use builder pattern to create SparkSession in PySpark. ## What changes were proposed in this pull request? This is a python port of corresponding Scala builder pattern code. `sql.py` is modified as a target example case. ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #12860 from dongjoon-hyun/SPARK-15084. (cherry picked from commit 0903a185c7ebc57c75301a27d215b08efd347f99) Signed-off-by: Andrew Or --- examples/src/main/python/sql.py | 35 ++++++------- python/pyspark/sql/session.py | 91 ++++++++++++++++++++++++++++++++- 2 files changed, 105 insertions(+), 21 deletions(-) diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 2c188759328f2..ea6a22dbfe824 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -20,33 +20,28 @@ import os import sys -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType if __name__ == "__main__": - sc = SparkContext(appName="PythonSQL") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PythonSQL").getOrCreate() - # RDD is created from a list of rows - some_rdd = sc.parallelize([Row(name="John", age=19), - Row(name="Smith", age=23), - Row(name="Sarah", age=18)]) - # Infer schema from the first row, create a DataFrame and print the schema - some_df = sqlContext.createDataFrame(some_rdd) + # A list of Rows. Infer schema from the first row, create a DataFrame and print the schema + rows = [Row(name="John", age=19), Row(name="Smith", age=23), Row(name="Sarah", age=18)] + some_df = spark.createDataFrame(rows) some_df.printSchema() - # Another RDD is created from a list of tuples - another_rdd = sc.parallelize([("John", 19), ("Smith", 23), ("Sarah", 18)]) + # A list of tuples + tuples = [("John", 19), ("Smith", 23), ("Sarah", 18)] # Schema with two fields - person_name and person_age schema = StructType([StructField("person_name", StringType(), False), StructField("person_age", IntegerType(), False)]) # Create a DataFrame by applying the schema to the RDD and print the schema - another_df = sqlContext.createDataFrame(another_rdd, schema) + another_df = spark.createDataFrame(tuples, schema) another_df.printSchema() # root - # |-- age: integer (nullable = true) + # |-- age: long (nullable = true) # |-- name: string (nullable = true) # A JSON dataset is pointed to by path. @@ -57,7 +52,7 @@ else: path = sys.argv[1] # Create a DataFrame from the file(s) pointed to by path - people = sqlContext.jsonFile(path) + people = spark.read.json(path) # root # |-- person_name: string (nullable = false) # |-- person_age: integer (nullable = false) @@ -65,16 +60,16 @@ # The inferred schema can be visualized using the printSchema() method. people.printSchema() # root - # |-- age: IntegerType - # |-- name: StringType + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) # Register this DataFrame as a table. - people.registerAsTable("people") + people.registerTempTable("people") # SQL statements can be run by using the sql methods provided by sqlContext - teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") for each in teenagers.collect(): print(each[0]) - sc.stop() + spark.stop() diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 35c36b4935090..fb3e318163e87 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -19,6 +19,7 @@ import sys import warnings from functools import reduce +from threading import RLock if sys.version >= '3': basestring = unicode = str @@ -58,16 +59,98 @@ def toDF(self, schema=None, sampleRatio=None): class SparkSession(object): - """Main entry point for Spark SQL functionality. + """The entry point to programming Spark with the Dataset and DataFrame API. A SparkSession can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. + To create a SparkSession, use the following builder pattern: + + >>> spark = SparkSession.builder \ + .master("local") \ + .appName("Word Count") \ + .config("spark.some.config.option", "some-value") \ + .getOrCreate() :param sparkContext: The :class:`SparkContext` backing this SparkSession. :param jsparkSession: An optional JVM Scala SparkSession. If set, we do not instantiate a new SparkSession in the JVM, instead we make all calls to this object. """ + class Builder(object): + """Builder for :class:`SparkSession`. + """ + + _lock = RLock() + _options = {} + + @since(2.0) + def config(self, key=None, value=None, conf=None): + """Sets a config option. Options set using this method are automatically propagated to + both :class:`SparkConf` and :class:`SparkSession`'s own configuration. + + For an existing SparkConf, use `conf` parameter. + >>> from pyspark.conf import SparkConf + >>> SparkSession.builder.config(conf=SparkConf()) + >> SparkSession.builder.config("spark.some.config.option", "some-value") + Date: Tue, 3 May 2016 18:06:35 -0700 Subject: [PATCH 0051/1470] [SPARK-15097][SQL] make Dataset.sqlContext a stable identifier for imports ## What changes were proposed in this pull request? Make Dataset.sqlContext a lazy val so that its a stable identifier and can be used for imports. Now this works again: import someDataset.sqlContext.implicits._ ## How was this patch tested? Add unit test to DatasetSuite that uses the import show above. Author: Koert Kuipers Closes #12877 from koertkuipers/feat-sqlcontext-stable-import. (cherry picked from commit 9e4928b7e00788913553c1cb0722048001b91601) Signed-off-by: Andrew Or --- .../main/scala/org/apache/spark/sql/Dataset.scala | 3 ++- .../scala/org/apache/spark/sql/DatasetSuite.scala | 13 +++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 08be94e8d4f12..1bea72c4711f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -211,7 +211,8 @@ class Dataset[T] private[sql]( private implicit def classTag = unresolvedTEncoder.clsTag - def sqlContext: SQLContext = sparkSession.wrapped + // sqlContext must be val because a stable identifier is expected when you import implicits + @transient lazy val sqlContext: SQLContext = sparkSession.wrapped protected[sql] def resolve(colName: String): NamedExpression = { queryExecution.analyzed.resolveQuoted(colName, sparkSession.sessionState.analyzer.resolver) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index eee21acf7510b..68a12b062249e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -653,6 +653,11 @@ class DatasetSuite extends QueryTest with SharedSQLContext { dataset.join(actual, dataset("user") === actual("id")).collect() } + + test("SPARK-15097: implicits on dataset's sqlContext can be imported") { + val dataset = Seq(1, 2, 3).toDS() + checkDataset(DatasetTransform.addOne(dataset), 2, 3, 4) + } } case class OtherTuple(_1: String, _2: Int) @@ -713,3 +718,11 @@ class JavaData(val a: Int) extends Serializable { object JavaData { def apply(a: Int): JavaData = new JavaData(a) } + +/** Used to test importing dataset.sqlContext.implicits._ */ +object DatasetTransform { + def addOne(ds: Dataset[Int]): Dataset[Int] = { + import ds.sqlContext.implicits._ + ds.map(_ + 1) + } +} From 940b8f60b90d0acf6910abfd368af25cefdf4ffa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 May 2016 18:07:53 -0700 Subject: [PATCH 0052/1470] [SPARK-14414][SQL] Make DDL exceptions more consistent ## What changes were proposed in this pull request? Just a bunch of small tweaks on DDL exception messages. ## How was this patch tested? `DDLCommandSuite` et al. Author: Andrew Or Closes #12853 from andrewor14/make-exceptions-consistent. (cherry picked from commit 6ba17cd147277a20a7fbb244c040e694de486c36) Signed-off-by: Andrew Or --- .../spark/sql/catalyst/parser/SqlBase.g4 | 26 +-- .../analysis/NoSuchItemException.scala | 14 +- .../catalyst/catalog/InMemoryCatalog.scala | 10 +- .../sql/catalyst/catalog/SessionCatalog.scala | 18 +- .../sql/catalyst/parser/ParserUtils.scala | 2 +- .../catalog/SessionCatalogSuite.scala | 6 +- .../spark/sql/execution/SparkSqlParser.scala | 152 +++------------ .../sql/execution/command/AnalyzeTable.scala | 6 +- .../spark/sql/execution/command/ddl.scala | 70 +------ .../sql/execution/command/functions.scala | 10 +- .../spark/sql/execution/command/tables.scala | 39 ++-- .../spark/sql/execution/command/views.scala | 3 +- .../execution/command/DDLCommandSuite.scala | 174 +++--------------- .../sources/CreateTableAsSelectSuite.scala | 13 +- .../execution/HiveCompatibilitySuite.scala | 10 +- .../spark/sql/hive/HiveExternalCatalog.scala | 2 +- .../sql/hive/client/HiveClientImpl.scala | 12 +- .../sql/hive/execution/HiveCommandSuite.scala | 4 +- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- .../sql/hive/execution/SQLViewSuite.scala | 3 +- 20 files changed, 141 insertions(+), 435 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 3ab448dd9e259..273ad9289169d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -81,18 +81,8 @@ statement DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions | ALTER VIEW tableIdentifier DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions - | ALTER TABLE tableIdentifier partitionSpec? - SET FILEFORMAT fileFormat #setTableFileFormat | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation - | ALTER TABLE tableIdentifier partitionSpec? - CHANGE COLUMN? oldName=identifier colType - (FIRST | AFTER after=identifier)? (CASCADE | RESTRICT)? #changeColumn - | ALTER TABLE tableIdentifier partitionSpec? - ADD COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)? #addColumns - | ALTER TABLE tableIdentifier partitionSpec? - REPLACE COLUMNS '(' colTypeList ')' (CASCADE | RESTRICT)? #replaceColumns - | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? - (FOR METADATA? REPLICATION '(' STRING ')')? #dropTable + | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? tableIdentifier #dropTable | CREATE (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier identifierCommentList? (COMMENT STRING)? @@ -170,6 +160,10 @@ unsupportedHiveNativeCommands | kw1=ALTER kw2=TABLE tableIdentifier kw3=TOUCH | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=COMPACT | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CONCATENATE + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=SET kw4=FILEFORMAT + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=ADD kw4=COLUMNS + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CHANGE kw4=COLUMNS? + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=REPLACE kw4=COLUMNS | kw1=START kw2=TRANSACTION | kw1=COMMIT | kw1=ROLLBACK @@ -647,9 +641,9 @@ nonReserved | NO | DATA | START | TRANSACTION | COMMIT | ROLLBACK | SORT | CLUSTER | DISTRIBUTE | UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION - | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE | FIRST - | AFTER | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT - | DBPROPERTIES | DFS | TRUNCATE | METADATA | REPLICATION | COMPUTE + | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE + | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT + | DBPROPERTIES | DFS | TRUNCATE | COMPUTE | STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER | REVOKE | GRANT | LOCK | UNLOCK | MSCK | REPAIR | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE | ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEX | INDEXES | LOCKS | OPTION | LOCAL | INPATH @@ -836,8 +830,6 @@ TOUCH: 'TOUCH'; COMPACT: 'COMPACT'; CONCATENATE: 'CONCATENATE'; CHANGE: 'CHANGE'; -FIRST: 'FIRST'; -AFTER: 'AFTER'; CASCADE: 'CASCADE'; RESTRICT: 'RESTRICT'; CLUSTERED: 'CLUSTERED'; @@ -849,8 +841,6 @@ DATABASE: 'DATABASE' | 'SCHEMA'; DATABASES: 'DATABASES' | 'SCHEMAS'; DFS: 'DFS'; TRUNCATE: 'TRUNCATE'; -METADATA: 'METADATA'; -REPLICATION: 'REPLICATION'; ANALYZE: 'ANALYZE'; COMPUTE: 'COMPUTE'; STATISTICS: 'STATISTICS'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index 11ef9e1160a9a..2412ec46c783d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -25,13 +25,17 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ -class NoSuchDatabaseException(db: String) extends AnalysisException(s"Database $db not found") +class NoSuchDatabaseException(db: String) extends AnalysisException(s"Database '$db' not found") class NoSuchTableException(db: String, table: String) - extends AnalysisException(s"Table or View $table not found in database $db") + extends AnalysisException(s"Table or view '$table' not found in database '$db'") -class NoSuchPartitionException(db: String, table: String, spec: TablePartitionSpec) extends - AnalysisException(s"Partition not found in table $table database $db:\n" + spec.mkString("\n")) +class NoSuchPartitionException( + db: String, + table: String, + spec: TablePartitionSpec) + extends AnalysisException( + s"Partition not found in table '$table' database '$db':\n" + spec.mkString("\n")) class NoSuchFunctionException(db: String, func: String) - extends AnalysisException(s"Function $func not found in database $db") + extends AnalysisException(s"Function '$func' not found in database '$db'") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 60eb7329f71dc..1d2ca2863fae9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -164,7 +164,7 @@ class InMemoryCatalog extends ExternalCatalog { catalog(db).tables.remove(table) } else { if (!ignoreIfNotExists) { - throw new AnalysisException(s"Table or View '$table' does not exist in database '$db'") + throw new AnalysisException(s"Table or view '$table' does not exist in database '$db'") } } } @@ -211,7 +211,7 @@ class InMemoryCatalog extends ExternalCatalog { loadPath: String, isOverwrite: Boolean, holdDDLTime: Boolean): Unit = { - throw new AnalysisException("loadTable is not implemented for InMemoryCatalog.") + throw new UnsupportedOperationException("loadTable is not implemented") } override def loadPartition( @@ -223,7 +223,7 @@ class InMemoryCatalog extends ExternalCatalog { holdDDLTime: Boolean, inheritTableSpecs: Boolean, isSkewedStoreAsSubdir: Boolean): Unit = { - throw new AnalysisException("loadPartition is not implemented for InMemoryCatalog.") + throw new UnsupportedOperationException("loadPartition is not implemented.") } // -------------------------------------------------------------------------- @@ -304,8 +304,8 @@ class InMemoryCatalog extends ExternalCatalog { partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = synchronized { requireTableExists(db, table) if (partialSpec.nonEmpty) { - throw new AnalysisException("listPartition does not support partition spec in " + - "InMemoryCatalog.") + throw new UnsupportedOperationException( + "listPartition with partial partition spec is not implemented") } catalog(db).tables(table).partitions.values.toSeq } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index a445a253ff9b2..ff6303471e145 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.catalog -import java.io.File - import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -284,10 +282,12 @@ class SessionCatalog( * This assumes the database specified in `oldName` matches the one specified in `newName`. */ def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = { - if (oldName.database != newName.database) { - throw new AnalysisException("rename does not support moving tables across databases") - } val db = oldName.database.getOrElse(currentDb) + val newDb = newName.database.getOrElse(currentDb) + if (db != newDb) { + throw new AnalysisException( + s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") + } val oldTableName = formatTableName(oldName.table) val newTableName = formatTableName(newName.table) if (oldName.database.isDefined || !tempTables.contains(oldTableName)) { @@ -315,7 +315,7 @@ class SessionCatalog( if (externalCatalog.tableExists(db, table)) { externalCatalog.dropTable(db, table, ignoreIfNotExists = true) } else if (!ignoreIfNotExists) { - logError(s"Table or View '${name.quotedString}' does not exist") + throw new AnalysisException(s"Table or view '${name.quotedString}' does not exist") } } else { tempTables.remove(table) @@ -534,7 +534,7 @@ class SessionCatalog( if (!functionExists(identifier)) { externalCatalog.createFunction(db, newFuncDefinition) } else if (!ignoreIfExists) { - throw new AnalysisException(s"function '$identifier' already exists in database '$db'") + throw new AnalysisException(s"Function '$identifier' already exists in database '$db'") } } @@ -632,9 +632,9 @@ class SessionCatalog( } protected def failFunctionLookup(name: String): Nothing = { - throw new AnalysisException(s"Undefined function: $name. This function is " + + throw new AnalysisException(s"Undefined function: '$name'. This function is " + s"neither a registered temporary function nor " + - s"a permanent function registered in the database $currentDb.") + s"a permanent function registered in the database '$currentDb'.") } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 64713cddf4e0d..58e2bdb6e24fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -39,7 +39,7 @@ object ParserUtils { stream.getText(Interval.of(0, stream.size())) } - def parseException(message: String, ctx: ParserRuleContext): ParseException = { + def operationNotAllowed(message: String, ctx: ParserRuleContext): ParseException = { new ParseException(s"Operation not allowed: $message", ctx) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index ba5d8ce0f48f8..a704ca78f99dc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -233,9 +233,9 @@ class SessionCatalogSuite extends SparkFunSuite { intercept[AnalysisException] { catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = true) } - // If the table does not exist, we do not issue an exception. Instead, we output an error log - // message to console when ignoreIfNotExists is set to false. - catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false) + intercept[AnalysisException] { + catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false) + } catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = true) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index dfc56a7d98ba2..b000cc99533e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -142,7 +142,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * A command for users to list the columm names for a table. + * A command for users to list the column names for a table. * This function creates a [[ShowColumnsCommand]] logical plan. * * The syntax of using this command in SQL is: @@ -155,8 +155,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val lookupTable = Option(ctx.db) match { case None => table - case Some(db) if table.database.isDefined => - throw new ParseException("Duplicates the declaration for database", ctx) + case Some(db) if table.database.exists(_ != db) => + throw operationNotAllowed( + s"SHOW COLUMNS with conflicting databases: '$db' != '${table.database.get}'", + ctx) case Some(db) => TableIdentifier(table.identifier, Some(db.getText)) } ShowColumnsCommand(lookupTable) @@ -214,7 +216,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitExplain(ctx: ExplainContext): LogicalPlan = withOrigin(ctx) { val options = ctx.explainOption.asScala if (options.exists(_.FORMATTED != null)) { - logWarning("Unsupported operation: EXPLAIN FORMATTED option") + throw operationNotAllowed("EXPLAIN FORMATTED", ctx) } // Create the explain comment. @@ -260,9 +262,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: CreateTableHeaderContext): TableHeader = withOrigin(ctx) { val temporary = ctx.TEMPORARY != null val ifNotExists = ctx.EXISTS != null - assert(!temporary || !ifNotExists, - "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.", - ctx) + if (temporary && ifNotExists) { + throw operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) + } (visitTableIdentifier(ctx.tableIdentifier), temporary, ifNotExists, ctx.EXTERNAL != null) } @@ -274,7 +276,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitCreateTableUsing(ctx: CreateTableUsingContext): LogicalPlan = withOrigin(ctx) { val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) if (external) { - throw new ParseException("Unsupported operation: EXTERNAL option", ctx) + throw operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } val options = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) val provider = ctx.tableProvider.qualifiedName.getText @@ -423,7 +425,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case "jar" | "file" | "archive" => resourceType -> string(resource.STRING) case other => - throw new ParseException(s"Resource Type '$resourceType' is not supported.", ctx) + throw operationNotAllowed(s"CREATE FUNCTION with resource type '$resourceType'", ctx) } } @@ -459,10 +461,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { if (ctx.PURGE != null) { - throw new ParseException("Unsupported operation: PURGE option", ctx) - } - if (ctx.REPLICATION != null) { - throw new ParseException("Unsupported operation: REPLICATION clause", ctx) + throw operationNotAllowed("DROP TABLE ... PURGE", ctx) } DropTable( visitTableIdentifier(ctx.tableIdentifier), @@ -554,7 +553,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitAddTablePartition( ctx: AddTablePartitionContext): LogicalPlan = withOrigin(ctx) { if (ctx.VIEW != null) { - throw new ParseException(s"Operation not allowed: partitioned views", ctx) + throw operationNotAllowed("ALTER VIEW ... ADD PARTITION", ctx) } // Create partition spec to location mapping. val specsAndLocs = if (ctx.partitionSpec.isEmpty) { @@ -605,10 +604,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitDropTablePartitions( ctx: DropTablePartitionsContext): LogicalPlan = withOrigin(ctx) { if (ctx.VIEW != null) { - throw new ParseException(s"Operation not allowed: partitioned views", ctx) + throw operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx) } if (ctx.PURGE != null) { - throw new ParseException(s"Operation not allowed: PURGE", ctx) + throw operationNotAllowed("ALTER TABLE ... DROP PARTITION ... PURGE", ctx) } AlterTableDropPartition( visitTableIdentifier(ctx.tableIdentifier), @@ -616,35 +615,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.EXISTS != null) } - /** - * Create an [[AlterTableSetFileFormat]] command - * - * For example: - * {{{ - * ALTER TABLE table [PARTITION spec] SET FILEFORMAT file_format; - * }}} - */ - override def visitSetTableFileFormat( - ctx: SetTableFileFormatContext): LogicalPlan = withOrigin(ctx) { - // AlterTableSetFileFormat currently takes both a GenericFileFormat and a - // TableFileFormatContext. This is a bit weird because it should only take one. It also should - // use a CatalogFileFormat instead of either a String or a Sequence of Strings. We will address - // this in a follow-up PR. - val (fileFormat, genericFormat) = ctx.fileFormat match { - case s: GenericFileFormatContext => - (Seq.empty[String], Option(s.identifier.getText)) - case s: TableFileFormatContext => - val elements = Seq(s.inFmt, s.outFmt) ++ Option(s.serdeCls).toSeq - (elements.map(string), None) - } - AlterTableSetFileFormat( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - fileFormat, - genericFormat)( - parseException("ALTER TABLE SET FILEFORMAT", ctx)) - } - /** * Create an [[AlterTableSetLocation]] command * @@ -660,79 +630,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { visitLocationSpec(ctx.locationSpec)) } - /** - * Create an [[AlterTableChangeCol]] command - * - * For example: - * {{{ - * ALTER TABLE tableIdentifier [PARTITION spec] - * CHANGE [COLUMN] col_old_name col_new_name column_type [COMMENT col_comment] - * [FIRST|AFTER column_name] [CASCADE|RESTRICT]; - * }}} - */ - override def visitChangeColumn(ctx: ChangeColumnContext): LogicalPlan = withOrigin(ctx) { - val col = visitColType(ctx.colType()) - val comment = if (col.metadata.contains("comment")) { - Option(col.metadata.getString("comment")) - } else { - None - } - - AlterTableChangeCol( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - ctx.oldName.getText, - // We could also pass in a struct field - seems easier. - col.name, - col.dataType, - comment, - Option(ctx.after).map(_.getText), - // Note that Restrict and Cascade are mutually exclusive. - ctx.RESTRICT != null, - ctx.CASCADE != null)( - parseException("ALTER TABLE CHANGE COLUMN", ctx)) - } - - /** - * Create an [[AlterTableAddCol]] command - * - * For example: - * {{{ - * ALTER TABLE tableIdentifier [PARTITION spec] - * ADD COLUMNS (name type [COMMENT comment], ...) [CASCADE|RESTRICT] - * }}} - */ - override def visitAddColumns(ctx: AddColumnsContext): LogicalPlan = withOrigin(ctx) { - AlterTableAddCol( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - createStructType(ctx.colTypeList), - // Note that Restrict and Cascade are mutually exclusive. - ctx.RESTRICT != null, - ctx.CASCADE != null)( - parseException("ALTER TABLE ADD COLUMNS", ctx)) - } - - /** - * Create an [[AlterTableReplaceCol]] command - * - * For example: - * {{{ - * ALTER TABLE tableIdentifier [PARTITION spec] - * REPLACE COLUMNS (name type [COMMENT comment], ...) [CASCADE|RESTRICT] - * }}} - */ - override def visitReplaceColumns(ctx: ReplaceColumnsContext): LogicalPlan = withOrigin(ctx) { - AlterTableReplaceCol( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), - createStructType(ctx.colTypeList), - // Note that Restrict and Cascade are mutually exclusive. - ctx.RESTRICT != null, - ctx.CASCADE != null)( - parseException("ALTER TABLE REPLACE COLUMNS", ctx)) - } - /** * Create location string. */ @@ -753,7 +650,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { .map { orderedIdCtx => Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => if (dir.toLowerCase != "asc") { - throw parseException("Only ASC ordering is supported for sorting columns", ctx) + throw operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) } } @@ -789,7 +686,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // SET ROLE is the exception to the rule, because we handle this before other SET commands. "SET ROLE" } - throw parseException(keywords, ctx) + throw operationNotAllowed(keywords, ctx) } /** @@ -799,7 +696,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.identifier.getText.toLowerCase match { case "file" => AddFile(remainder(ctx.identifier).trim) case "jar" => AddJar(remainder(ctx.identifier).trim) - case other => throw new ParseException(s"Unsupported resource type '$other'.", ctx) + case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) } } @@ -836,10 +733,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { "Please use registerTempTable as an alternative.", ctx) } if (ctx.skewSpec != null) { - throw new ParseException("Operation not allowed: CREATE TABLE ... SKEWED BY ...", ctx) + throw operationNotAllowed("CREATE TABLE ... SKEWED BY", ctx) } if (ctx.bucketSpec != null) { - throw new ParseException("Operation not allowed: CREATE TABLE ... CLUSTERED BY ...", ctx) + throw operationNotAllowed("CREATE TABLE ... CLUSTERED BY", ctx) } val tableType = if (external) { CatalogTableType.EXTERNAL @@ -926,9 +823,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case (c: GenericFileFormatContext, null) => visitGenericFileFormat(c) case (null, storageHandler) => - throw new ParseException("Operation not allowed: ... STORED BY storage_handler ...", ctx) + throw operationNotAllowed("STORED BY", ctx) case _ => - throw new ParseException("expected either STORED AS or STORED BY, not both", ctx) + throw new ParseException("Expected either STORED AS or STORED BY, not both", ctx) } } @@ -960,7 +857,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { outputFormat = s.outputFormat, serde = s.serde) case None => - throw new ParseException(s"Unrecognized file format in STORED AS clause: $source", ctx) + throw operationNotAllowed(s"STORED AS with file format '$source'", ctx) } } @@ -1041,7 +938,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) { if (ctx.identifierList != null) { - throw new ParseException(s"Operation not allowed: partitioned views", ctx) + throw operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx) } else { val identifiers = Option(ctx.identifierCommentList).toSeq.flatMap(_.identifierComment.asScala) val schema = identifiers.map { ic => @@ -1128,6 +1025,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { recordReader: Token, schemaLess: Boolean): ScriptInputOutputSchema = { if (recordWriter != null || recordReader != null) { + // TODO: what does this message mean? throw new ParseException( "Unsupported operation: Used defined record reader/writer classes.", ctx) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala index 54ff5ae7d9d0e..de2db44b0e1b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala @@ -21,7 +21,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} @@ -99,8 +99,8 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { } case otherRelation => - throw new UnsupportedOperationException( - s"Analyze only works for Hive tables, but $tableName is a ${otherRelation.nodeName}") + throw new AnalysisException(s"ANALYZE TABLE is only supported for Hive tables, " + + s"but '${tableIdent.unquotedString}' is a ${otherRelation.nodeName}.") } Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 44647116b4889..aa06c014fb0a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -30,30 +30,9 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.types._ - // Note: The definition of these commands are based on the ones described in // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL -/** - * A DDL command that is not supported right now. Since we have already implemented - * the parsing rules for some commands that are not allowed, we use this as the base class - * of those commands. - */ -abstract class UnsupportedCommand(exception: ParseException) extends RunnableCommand { - - // Throws the ParseException when we create this command. - throw exception - - override def run(sparkSession: SparkSession): Seq[Row] = { - Seq.empty[Row] - } - - override val output: Seq[Attribute] = { - Seq(AttributeReference("result", StringType, nullable = false)()) - } - -} - /** * A command for users to create a new database. * @@ -251,8 +230,8 @@ case class AlterTableSetProperties( val table = catalog.getTableMetadata(tableName) val newProperties = table.properties ++ properties if (DDLUtils.isDatasourceTable(newProperties)) { - throw new AnalysisException( - "alter table properties is not supported for tables defined using the datasource API") + throw new AnalysisException("ALTER TABLE SET TBLPROPERTIES is not supported for " + + "tables defined using the datasource API") } val newTable = table.copy(properties = newProperties) catalog.alterTable(newTable) @@ -319,15 +298,14 @@ case class AlterTableSerDeProperties( // should never happen if we parsed things correctly require(serdeClassName.isDefined || serdeProperties.isDefined, - "alter table attempted to set neither serde class name nor serde properties") + "ALTER TABLE attempted to set neither serde class name nor serde properties") override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) // Do not support setting serde for datasource tables if (serdeClassName.isDefined && DDLUtils.isDatasourceTable(table)) { - throw new AnalysisException( - "alter table serde is not supported for datasource tables") + throw new AnalysisException("ALTER TABLE SET SERDE is not supported for datasource tables") } val newTable = table.withNewStorage( serde = serdeClassName.orElse(table.storage.serde), @@ -361,7 +339,7 @@ case class AlterTableAddPartition( val table = catalog.getTableMetadata(tableName) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - "alter table add partition is not allowed for tables defined using the datasource API") + "ALTER TABLE ADD PARTITION is not allowed for tables defined using the datasource API") } val parts = partitionSpecsAndLocs.map { case (spec, location) => // inherit table storage format (possibly except for location) @@ -420,7 +398,7 @@ case class AlterTableDropPartition( val table = catalog.getTableMetadata(tableName) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - "alter table drop partition is not allowed for tables defined using the datasource API") + "ALTER TABLE DROP PARTITIONS is not allowed for tables defined using the datasource API") } catalog.dropPartitions(tableName, specs, ignoreIfNotExists = ifExists) Seq.empty[Row] @@ -428,12 +406,6 @@ case class AlterTableDropPartition( } -case class AlterTableSetFileFormat( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - fileFormat: Seq[String], - genericFormat: Option[String])(exception: ParseException) - extends UnsupportedCommand(exception) with Logging /** * A command that sets the location of a table or a partition. @@ -462,7 +434,7 @@ case class AlterTableSetLocation( val newPart = if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - "alter table set location for partition is not allowed for tables defined " + + "ALTER TABLE SET LOCATION for partition is not allowed for tables defined " + "using the datasource API") } else { part.copy(storage = part.storage.copy(locationUri = Some(location))) @@ -485,34 +457,6 @@ case class AlterTableSetLocation( } -case class AlterTableChangeCol( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - oldColName: String, - newColName: String, - dataType: DataType, - comment: Option[String], - afterColName: Option[String], - restrict: Boolean, - cascade: Boolean)(exception: ParseException) - extends UnsupportedCommand(exception) with Logging - -case class AlterTableAddCol( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - columns: StructType, - restrict: Boolean, - cascade: Boolean)(exception: ParseException) - extends UnsupportedCommand(exception) with Logging - -case class AlterTableReplaceCol( - tableName: TableIdentifier, - partitionSpec: Option[TablePartitionSpec], - columns: StructType, - restrict: Boolean, - cascade: Boolean)(exception: ParseException) - extends UnsupportedCommand(exception) with Logging - private[sql] object DDLUtils { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 5aa779ddeb6c7..73c1ef70a2d8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -51,9 +51,8 @@ case class CreateFunction( val catalog = sparkSession.sessionState.catalog if (isTemp) { if (databaseName.isDefined) { - throw new AnalysisException( - s"It is not allowed to provide database name when defining a temporary function. " + - s"However, database name ${databaseName.get} is provided.") + throw new AnalysisException(s"Specifying a database in CREATE TEMPORARY FUNCTION " + + s"is not allowed: '${databaseName.get}'") } // We first load resources and then put the builder in the function registry. // Please note that it is allowed to overwrite an existing temp function. @@ -153,9 +152,8 @@ case class DropFunction( val catalog = sparkSession.sessionState.catalog if (isTemp) { if (databaseName.isDefined) { - throw new AnalysisException( - s"It is not allowed to provide database name when dropping a temporary function. " + - s"However, database name ${databaseName.get} is provided.") + throw new AnalysisException(s"Specifying a database in DROP TEMPORARY FUNCTION " + + s"is not allowed: '${databaseName.get}'") } catalog.dropTempFunction(functionName, ifExists) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 6078918316d9e..489c980c1654e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -162,37 +162,36 @@ case class LoadData( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog if (!catalog.tableExists(table)) { - throw new AnalysisException( - s"Table in LOAD DATA does not exist: '$table'") + throw new AnalysisException(s"Target table in LOAD DATA does not exist: '$table'") } - val targetTable = catalog.getTableMetadataOption(table).getOrElse { - throw new AnalysisException( - s"Table in LOAD DATA cannot be temporary: '$table'") + throw new AnalysisException(s"Target table in LOAD DATA cannot be temporary: '$table'") } - if (DDLUtils.isDatasourceTable(targetTable)) { - throw new AnalysisException( - "LOAD DATA is not supported for datasource tables") + throw new AnalysisException(s"LOAD DATA is not supported for datasource tables: '$table'") } - if (targetTable.partitionColumnNames.nonEmpty) { - if (partition.isEmpty || targetTable.partitionColumnNames.size != partition.get.size) { - throw new AnalysisException( - "LOAD DATA to partitioned table must specify a specific partition of " + - "the table by specifying values for all of the partitioning columns.") + if (partition.isEmpty) { + throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + s"but no partition spec is provided") + } + if (targetTable.partitionColumnNames.size != partition.get.size) { + throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + s"but number of columns in provided partition spec (${partition.get.size}) " + + s"do not match number of partitioned columns in table " + + s"(s${targetTable.partitionColumnNames.size})") } - partition.get.keys.foreach { colName => if (!targetTable.partitionColumnNames.contains(colName)) { - throw new AnalysisException( - s"LOAD DATA to partitioned table specifies a non-existing partition column: '$colName'") + throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + s"but the specified partition spec refers to a column that is not partitioned: " + + s"'$colName'") } } } else { if (partition.nonEmpty) { - throw new AnalysisException( - "LOAD DATA to non-partitioned table cannot specify partition.") + throw new AnalysisException(s"LOAD DATA target table '$table' is not partitioned, " + + s"but a partition spec was provided.") } } @@ -200,7 +199,7 @@ case class LoadData( if (isLocal) { val uri = Utils.resolveURI(path) if (!new File(uri.getPath()).exists()) { - throw new AnalysisException(s"LOAD DATA with non-existing path: $path") + throw new AnalysisException(s"LOAD DATA input path does not exist: $path") } uri } else { @@ -231,7 +230,7 @@ case class LoadData( if (scheme == null) { throw new AnalysisException( - "LOAD DATA with non-local path must specify URI Scheme.") + s"LOAD DATA: URI scheme is required for non-local input paths: '$path'") } // Follow Hive's behavior: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 1641780db8bc2..0f656ef53e395 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -58,8 +58,7 @@ case class CreateViewCommand( private val tableIdentifier = tableDesc.identifier if (allowExisting && replace) { - throw new AnalysisException( - "It is not allowed to define a view with both IF NOT EXISTS and OR REPLACE.") + throw new AnalysisException("CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed.") } override def run(sparkSession: SparkSession): Seq[Row] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index be0f4d78a5237..bd428a06f5099 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -23,17 +23,18 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ + // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { private val parser = new SparkSqlParser(new SQLConf) - private def assertUnsupported(sql: String): Unit = { + private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { val e = intercept[ParseException] { parser.parsePlan(sql) } assert(e.getMessage.toLowerCase.contains("operation not allowed")) + containsThesePhrases.foreach { p => assert(e.getMessage.toLowerCase.contains(p)) } } test("create database") { @@ -347,27 +348,13 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed2, expected2) } - // ALTER VIEW view_name ADD [IF NOT EXISTS] PARTITION partition_spec PARTITION partition_spec ...; - test("alter view: add partition") { - val sql1 = + test("alter view: add partition (not supported)") { + assertUnsupported( """ |ALTER VIEW view_name ADD IF NOT EXISTS PARTITION |(dt='2008-08-08', country='us') PARTITION |(dt='2009-09-09', country='uk') - """.stripMargin - // different constant types in partitioning spec - val sql2 = - """ - |ALTER VIEW view_name ADD PARTITION - |(col1=NULL, cOL2='f', col3=5, COL4=true) - """.stripMargin - - intercept[ParseException] { - parser.parsePlan(sql1) - } - intercept[ParseException] { - parser.parsePlan(sql2) - } + """.stripMargin) } test("alter table: rename partition") { @@ -392,7 +379,7 @@ class DDLCommandSuite extends PlanTest { """.stripMargin) } - // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE] + // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] // ALTER VIEW table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] test("alter table/view: drop partitions") { val sql1_table = @@ -403,24 +390,17 @@ class DDLCommandSuite extends PlanTest { val sql2_table = """ |ALTER TABLE table_name DROP PARTITION - |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') PURGE + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') """.stripMargin val sql1_view = sql1_table.replace("TABLE", "VIEW") - // Note: ALTER VIEW DROP PARTITION does not support PURGE - val sql2_view = sql2_table.replace("TABLE", "VIEW").replace("PURGE", "") + val sql2_view = sql2_table.replace("TABLE", "VIEW") val parsed1_table = parser.parsePlan(sql1_table) - val e = intercept[ParseException] { - parser.parsePlan(sql2_table) - } - assert(e.getMessage.contains("Operation not allowed")) - - intercept[ParseException] { - parser.parsePlan(sql1_view) - } - intercept[ParseException] { - parser.parsePlan(sql2_view) - } + val parsed2_table = parser.parsePlan(sql2_table) + assertUnsupported(sql1_table + " PURGE") + assertUnsupported(sql2_table + " PURGE") + assertUnsupported(sql1_view) + assertUnsupported(sql2_view) val tableIdent = TableIdentifier("table_name", None) val expected1_table = AlterTableDropPartition( @@ -429,8 +409,10 @@ class DDLCommandSuite extends PlanTest { Map("dt" -> "2008-08-08", "country" -> "us"), Map("dt" -> "2009-09-09", "country" -> "uk")), ifExists = true) + val expected2_table = expected1_table.copy(ifExists = false) comparePlans(parsed1_table, expected1_table) + comparePlans(parsed2_table, expected2_table) } test("alter table: archive partition (not supported)") { @@ -441,29 +423,6 @@ class DDLCommandSuite extends PlanTest { assertUnsupported("ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')") } - /* - test("alter table: set file format") { - val sql1 = "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + - "OUTPUTFORMAT 'test' SERDE 'test'" - val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "SET FILEFORMAT PARQUET" - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableSetFileFormat( - tableIdent, - None, - List("test", "test", "test"), - None)(sql1) - val expected2 = AlterTableSetFileFormat( - tableIdent, - Some(Map("dt" -> "2008-08-08", "country" -> "us")), - Seq(), - Some("PARQUET"))(sql2) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } */ - test("alter table: set file format (not allowed)") { assertUnsupported( "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + @@ -527,58 +486,6 @@ class DDLCommandSuite extends PlanTest { assertUnsupported("ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES") } - /* - test("alter table: change column name/type/position/comment") { - val sql1 = "ALTER TABLE table_name CHANGE col_old_name col_new_name INT" - val sql2 = - """ - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' FIRST CASCADE - """.stripMargin - val sql3 = - """ - |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT - |COMMENT 'col_comment' AFTER column_name RESTRICT - """.stripMargin - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val parsed3 = parser.parsePlan(sql3) - val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableChangeCol( - tableName = tableIdent, - partitionSpec = None, - oldColName = "col_old_name", - newColName = "col_new_name", - dataType = IntegerType, - comment = None, - afterColName = None, - restrict = false, - cascade = false)(sql1) - val expected2 = AlterTableChangeCol( - tableName = tableIdent, - partitionSpec = None, - oldColName = "col_old_name", - newColName = "col_new_name", - dataType = IntegerType, - comment = Some("col_comment"), - afterColName = None, - restrict = false, - cascade = true)(sql2) - val expected3 = AlterTableChangeCol( - tableName = tableIdent, - partitionSpec = None, - oldColName = "col_old_name", - newColName = "col_new_name", - dataType = IntegerType, - comment = Some("col_comment"), - afterColName = Some("column_name"), - restrict = true, - cascade = false)(sql3) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected3) - } */ - test("alter table: change column name/type/position/comment (not allowed)") { assertUnsupported("ALTER TABLE table_name CHANGE col_old_name col_new_name INT") assertUnsupported( @@ -592,44 +499,6 @@ class DDLCommandSuite extends PlanTest { """.stripMargin) } - /* - test("alter table: add/replace columns") { - val sql1 = - """ - |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') - |ADD COLUMNS (new_col1 INT COMMENT 'test_comment', new_col2 LONG - |COMMENT 'test_comment2') CASCADE - """.stripMargin - val sql2 = - """ - |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT - |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT - """.stripMargin - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val meta1 = new MetadataBuilder().putString("comment", "test_comment").build() - val meta2 = new MetadataBuilder().putString("comment", "test_comment2").build() - val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableAddCol( - tableIdent, - Some(Map("dt" -> "2008-08-08", "country" -> "us")), - StructType(Seq( - StructField("new_col1", IntegerType, nullable = true, meta1), - StructField("new_col2", LongType, nullable = true, meta2))), - restrict = false, - cascade = true)(sql1) - val expected2 = AlterTableReplaceCol( - tableIdent, - None, - StructType(Seq( - StructField("new_col1", IntegerType, nullable = true, meta1), - StructField("new_col2", LongType, nullable = true, meta2))), - restrict = true, - cascade = false)(sql2) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } */ - test("alter table: add/replace columns (not allowed)") { assertUnsupported( """ @@ -678,6 +547,7 @@ class DDLCommandSuite extends PlanTest { val parsed2 = parser.parsePlan(s"DROP TABLE IF EXISTS $tableName1") val parsed3 = parser.parsePlan(s"DROP TABLE $tableName2") val parsed4 = parser.parsePlan(s"DROP TABLE IF EXISTS $tableName2") + assertUnsupported(s"DROP TABLE IF EXISTS $tableName2 PURGE") val expected1 = DropTable(TableIdentifier("tab", Option("db")), ifExists = false, isView = false) @@ -722,20 +592,20 @@ class DDLCommandSuite extends PlanTest { val sql1 = "SHOW COLUMNS FROM t1" val sql2 = "SHOW COLUMNS IN db1.t1" val sql3 = "SHOW COLUMNS FROM t1 IN db1" - val sql4 = "SHOW COLUMNS FROM db1.t1 IN db2" + val sql4 = "SHOW COLUMNS FROM db1.t1 IN db1" + val sql5 = "SHOW COLUMNS FROM db1.t1 IN db2" val parsed1 = parser.parsePlan(sql1) val expected1 = ShowColumnsCommand(TableIdentifier("t1", None)) val parsed2 = parser.parsePlan(sql2) val expected2 = ShowColumnsCommand(TableIdentifier("t1", Some("db1"))) val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql3) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) comparePlans(parsed3, expected2) - val message = intercept[ParseException] { - parser.parsePlan(sql4) - }.getMessage - assert(message.contains("Duplicates the declaration for database")) + comparePlans(parsed4, expected2) + assertUnsupported(sql5) } test("show partitions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index cb88a1c83c999..c1dc9b983407f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -22,6 +22,7 @@ import java.io.{File, IOException} import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils @@ -104,7 +105,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with sql("SELECT a, b FROM jsonTable"), sql("SELECT a, b FROM jt").collect()) - val message = intercept[AnalysisException]{ + val message = intercept[ParseException]{ sql( s""" |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable @@ -115,9 +116,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with |SELECT a * 4 FROM jt """.stripMargin) }.getMessage - assert( - message.contains(s"a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause."), - "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") + assert(message.toLowerCase.contains("operation not allowed")) // Overwrite the temporary table. sql( @@ -155,7 +154,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with } test("CREATE TEMPORARY TABLE AS SELECT with IF NOT EXISTS is not allowed") { - val message = intercept[AnalysisException]{ + val message = intercept[ParseException]{ sql( s""" |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable @@ -166,9 +165,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with |SELECT b FROM jt """.stripMargin) }.getMessage - assert( - message.contains("a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause."), - "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") + assert(message.toLowerCase.contains("operation not allowed")) } test("a CTAS statement with column definitions is not allowed") { 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 f082035852cc0..1eebecaf654ad 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 @@ -503,7 +503,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // We have converted the useful parts of these tests to tests // in org.apache.spark.sql.hive.execution.SQLQuerySuite. "drop_database_removes_partition_dirs", - "drop_table_removes_partition_dirs" + "drop_table_removes_partition_dirs", + + // These tests use EXPLAIN FORMATTED, which is not supported + "input4", + "join0", + "plan_json" ) /** @@ -699,7 +704,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input26", "input28", "input2_limit", - "input4", "input40", "input41", "input49", @@ -728,7 +732,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "insert1_overwrite_partitions", "insert2_overwrite_partitions", "insert_compressed", - "join0", "join1", "join10", "join11", @@ -866,7 +869,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "partition_type_check", "partition_varchar1", "partition_wise_fileformat9", - "plan_json", "ppd1", "ppd2", "ppd_clusterby", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index ee048b2588304..5ffd8ef149a1e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -75,7 +75,7 @@ private[spark] class HiveExternalCatalog(client: HiveClient) extends ExternalCat private def requireDbMatches(db: String, table: CatalogTable): Unit = { if (table.identifier.database != Some(db)) { throw new AnalysisException( - s"Provided database $db does not match the one specified in the " + + s"Provided database '$db' does not match the one specified in the " + s"table definition (${table.identifier.database.getOrElse("n/a")})") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index cdfadfaaeac78..47d9546c4f515 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -26,9 +26,12 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{PartitionDropOptions, TableType => HiveTableType} -import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema, Function => HiveFunction, FunctionType, PrincipalType, ResourceType, ResourceUri} +import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} +import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType} +import org.apache.hadoop.hive.metastore.api.{NoSuchObjectException, PrincipalType} +import org.apache.hadoop.hive.metastore.api.{ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState @@ -43,7 +46,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.util.{CircularBuffer, Utils} +import org.apache.spark.util.{CausedBy, CircularBuffer, Utils} /** * A class that wraps the HiveClient and converts its responses to externally visible classes. @@ -616,7 +619,8 @@ private[hive] class HiveClientImpl( try { Option(client.getFunction(db, name)).map(fromHiveFunction) } catch { - case he: HiveException => None + case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + None } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 8b3f2d1a0cd07..b8fef23f5464a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -125,7 +125,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message1 = intercept[AnalysisException] { sql("SHOW TBLPROPERTIES badtable") }.getMessage - assert(message1.contains("Table or View badtable not found in database default")) + assert(message1.contains("'badtable' not found in database 'default'")) // When key is not found, a row containing the error is returned. checkAnswer( @@ -289,7 +289,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message = intercept[NoSuchTableException] { sql("SHOW COLUMNS IN badtable FROM default") }.getMessage - assert(message.contains("badtable not found in database")) + assert(message.contains("'badtable' not found in database")) } test("show partitions - show everything") { 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 f20ab36efbf03..f7da9e73f6cbe 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 @@ -1502,7 +1502,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } - assert(cause.getMessage.contains("Only ASC ordering is supported for sorting columns")) + assert(cause.getMessage.contains("Column ordering must be ASC, was 'DESC'")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 0d88b3b87f501..51848470502af 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -105,7 +105,8 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val e = intercept[AnalysisException] { sql("CREATE OR REPLACE VIEW IF NOT EXISTS testView AS SELECT id FROM jt") } - assert(e.message.contains("not allowed to define a view")) + assert(e.message.contains( + "CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed")) } } } From fd3accca6c2b192fe193cb71c44b4db70ddca1cc Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 May 2016 21:59:03 -0700 Subject: [PATCH 0053/1470] [SPARK-15095][SQL] remove HiveSessionHook from ThriftServer ## What changes were proposed in this pull request? Remove HiveSessionHook ## How was this patch tested? No tests needed. Author: Davies Liu Closes #12881 from davies/remove_hooks. (cherry picked from commit 348c1389842c4d9a8807a41cf22caaa82f81d7ab) Signed-off-by: Reynold Xin --- .../service/cli/session/HiveSessionHook.java | 37 ------------------- .../service/cli/session/SessionManager.java | 20 ---------- 2 files changed, 57 deletions(-) delete mode 100644 sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java deleted file mode 100644 index 7e4f2b0478a52..0000000000000 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHook.java +++ /dev/null @@ -1,37 +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.hive.service.cli.session; - -import org.apache.hadoop.hive.ql.hooks.Hook; -import org.apache.hive.service.cli.HiveSQLException; - -/** - * HiveSessionHook. - * HiveServer2 session level Hook interface. The run method is executed - * when session manager starts a new session - * - */ -public interface HiveSessionHook extends Hook { - - /** - * @param sessionHookContext context - * @throws HiveSQLException - */ - void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException; -} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index c6d177798f558..0457b3781ebf0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -268,17 +268,6 @@ public SessionHandle openSession(TProtocolVersion protocol, String username, Str if (isOperationLogEnabled) { session.setOperationLogSessionDir(operationLogRootDir); } - try { - executeSessionHooks(session); - } catch (Exception e) { - try { - session.close(); - } catch (Throwable t) { - LOG.warn("Error closing session", t); - } - session = null; - throw new HiveSQLException("Failed to execute session hooks", e); - } handleToSession.put(session.getSessionHandle(), session); return session.getSessionHandle(); } @@ -361,15 +350,6 @@ public static void clearProxyUserName() { threadLocalProxyUserName.remove(); } - // execute session hooks - private void executeSessionHooks(HiveSession session) throws Exception { - List sessionHooks = HookUtils.getHooks(hiveConf, - HiveConf.ConfVars.HIVE_SERVER2_SESSION_HOOK, HiveSessionHook.class); - for (HiveSessionHook sessionHook : sessionHooks) { - sessionHook.run(new HiveSessionHookContextImpl(session)); - } - } - public Future submitBackgroundOperation(Runnable r) { return backgroundOperationPool.submit(r); } From 2ff6676062edf6b4d463c30aeba623869887d21a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 May 2016 22:56:40 -0700 Subject: [PATCH 0054/1470] [SPARK-15107][SQL] Allow varying # iterations by test case in Benchmark This patch changes our micro-benchmark util to allow setting different iteration numbers for different test cases. For some of our benchmarks, turning off whole-stage codegen can make the runtime 20X slower, making it very difficult to run a large number of times without substantially shortening the input cardinality. With this change, I set the default num iterations to 2 for whole stage codegen off, and 5 for whole stage codegen on. I also updated some results. N/A - this is a test util. Author: Reynold Xin Closes #12884 from rxin/SPARK-15107. (cherry picked from commit 695f0e9195209c75bfc62fc70bfc6d7d9f1047b3) Signed-off-by: Reynold Xin --- .../org/apache/spark/util/Benchmark.scala | 21 ++- .../unsafe/sort/RadixSortSuite.scala | 2 +- .../BenchmarkWholeStageCodegen.scala | 137 +++++++++++------- 3 files changed, 93 insertions(+), 67 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 1fc0ad7a4d6d3..0c685b1918a59 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -38,7 +38,7 @@ import org.apache.commons.lang3.SystemUtils private[spark] class Benchmark( name: String, valuesPerIteration: Long, - iters: Int = 5, + defaultNumIters: Int = 5, outputPerIteration: Boolean = false) { val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] @@ -46,8 +46,8 @@ private[spark] class Benchmark( * Adds a case to run when run() is called. The given function will be run for several * iterations to collect timing statistics. */ - def addCase(name: String)(f: Int => Unit): Unit = { - addTimerCase(name) { timer => + def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { + addTimerCase(name, numIters) { timer => timer.startTiming() f(timer.iteration) timer.stopTiming() @@ -59,8 +59,8 @@ private[spark] class Benchmark( * until timer.startTiming() is called within the given function. The corresponding * timer.stopTiming() method must be called before the function returns. */ - def addTimerCase(name: String)(f: Benchmark.Timer => Unit): Unit = { - benchmarks += Benchmark.Case(name, f) + def addTimerCase(name: String, numIters: Int = 0)(f: Benchmark.Timer => Unit): Unit = { + benchmarks += Benchmark.Case(name, f, if (numIters == 0) defaultNumIters else numIters) } /** @@ -75,7 +75,7 @@ private[spark] class Benchmark( val results = benchmarks.map { c => println(" Running case: " + c.name) - Benchmark.measure(valuesPerIteration, iters, outputPerIteration)(c.fn) + Benchmark.measure(valuesPerIteration, c.numIters, outputPerIteration)(c.fn) } println @@ -83,12 +83,11 @@ private[spark] class Benchmark( // The results are going to be processor specific so it is useful to include that. println(Benchmark.getJVMOSInfo()) println(Benchmark.getProcessorName()) - printf("%-35s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", "Per Row(ns)", "Relative") - println("-----------------------------------------------------------------------------------" + - "--------") + println("-" * 96) results.zip(benchmarks).foreach { case (result, benchmark) => - printf("%-35s %16s %12s %13s %10s\n", + printf("%-40s %16s %12s %13s %10s\n", benchmark.name, "%5.0f / %4.0f" format (result.bestMs, result.avgMs), "%10.1f" format result.bestRate, @@ -128,7 +127,7 @@ private[spark] object Benchmark { } } - case class Case(name: String, fn: Timer => Unit) + case class Case(name: String, fn: Timer => Unit, numIters: Int) case class Result(avgMs: Double, bestRate: Double, bestMs: Double) /** diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 52428634e5205..b03df1a94d84c 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -244,7 +244,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) timer.stopTiming() } - benchmark.run + benchmark.run() /** Running benchmark: radix sort 25000000 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala index 841263d3dab93..7ca4b75f480ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala @@ -36,6 +36,8 @@ import org.apache.spark.util.Benchmark * Benchmark to measure whole stage codegen performance. * To run this: * build/sbt "sql/test-only *BenchmarkWholeStageCodegen" + * + * Benchmarks in this file are skipped in normal builds. */ class BenchmarkWholeStageCodegen extends SparkFunSuite { lazy val conf = new SparkConf().setMaster("local[1]").setAppName("benchmark") @@ -44,31 +46,50 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { lazy val sc = SparkContext.getOrCreate(conf) lazy val sqlContext = SQLContext.getOrCreate(sc) - def runBenchmark(name: String, values: Long)(f: => Unit): Unit = { - val benchmark = new Benchmark(name, values) + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality) - Seq(false, true).foreach { enabled => - benchmark.addCase(s"$name codegen=$enabled") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", enabled.toString) - f - } + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + f + } + + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + sqlContext.setConf("spark.sql.codegen.wholeStage", "true") + f } benchmark.run() } - // These benchmark are skipped in normal build - ignore("range/filter/sum") { - val N = 500L << 20 - runBenchmark("rang/filter/sum", N) { + ignore("aggregate without grouping") { + val N = 500L << 22 + val benchmark = new Benchmark("agg without grouping", N) + runBenchmark("agg w/o group", N) { + sqlContext.range(N).selectExpr("sum(id)").collect() + } + /* + agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + agg w/o group wholestage off 30136 / 31885 69.6 14.4 1.0X + agg w/o group wholestage on 1851 / 1860 1132.9 0.9 16.3X + */ + } + + ignore("filter & aggregate without group") { + val N = 500L << 22 + runBenchmark("range/filter/sum", N) { sqlContext.range(N).filter("(id & 1) = 1").groupBy().sum().collect() } /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - rang/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - rang/filter/sum codegen=false 14332 / 16646 36.0 27.8 1.0X - rang/filter/sum codegen=true 897 / 1022 584.6 1.7 16.4X + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X + range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X */ } @@ -86,28 +107,32 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { */ } - ignore("range/sample/sum") { - val N = 500 << 20 - runBenchmark("range/sample/sum", N) { - sqlContext.range(N).sample(true, 0.01).groupBy().sum().collect() + ignore("sample") { + val N = 500 << 18 + runBenchmark("sample with replacement", N) { + sqlContext.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() } /* - Westmere E56xx/L56xx/X56xx (Nehalem-C) - range/sample/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - range/sample/sum codegen=false 53888 / 56592 9.7 102.8 1.0X - range/sample/sum codegen=true 41614 / 42607 12.6 79.4 1.3X + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X + sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X */ - runBenchmark("range/sample/sum", N) { - sqlContext.range(N).sample(false, 0.01).groupBy().sum().collect() + runBenchmark("sample without replacement", N) { + sqlContext.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() } /* - Westmere E56xx/L56xx/X56xx (Nehalem-C) - range/sample/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - range/sample/sum codegen=false 12982 / 13384 40.4 24.8 1.0X - range/sample/sum codegen=true 7074 / 7383 74.1 13.5 1.8X + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X + sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X */ } @@ -151,23 +176,23 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { } ignore("aggregate with linear keys") { - val N = 20 << 20 + val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) def f(): Unit = sqlContext.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() - benchmark.addCase(s"codegen = F") { iter => + benchmark.addCase(s"codegen = F", numIters = 2) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "false") f() } - benchmark.addCase(s"codegen = T hashmap = F") { iter => + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") f() } - benchmark.addCase(s"codegen = T hashmap = T") { iter => + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") f() @@ -176,36 +201,37 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { benchmark.run() /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - codegen = F 2067 / 2166 10.1 98.6 1.0X - codegen = T hashmap = F 1149 / 1321 18.3 54.8 1.8X - codegen = T hashmap = T 388 / 475 54.0 18.5 5.3X + + Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + codegen = F 6619 / 6780 12.7 78.9 1.0X + codegen = T hashmap = F 3935 / 4059 21.3 46.9 1.7X + codegen = T hashmap = T 897 / 971 93.5 10.7 7.4X */ } ignore("aggregate with randomized keys") { - val N = 20 << 20 + val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) sqlContext.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") def f(): Unit = sqlContext.sql("select k, k, sum(id) from test group by k, k").collect() - benchmark.addCase(s"codegen = F") { iter => + benchmark.addCase(s"codegen = F", numIters = 2) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "false") f() } - benchmark.addCase(s"codegen = T hashmap = F") { iter => + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") f() } - benchmark.addCase(s"codegen = T hashmap = T") { iter => + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") f() @@ -214,13 +240,14 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { benchmark.run() /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - codegen = F 2517 / 2608 8.3 120.0 1.0X - codegen = T hashmap = F 1484 / 1560 14.1 70.8 1.7X - codegen = T hashmap = T 794 / 908 26.4 37.9 3.2X + + Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + codegen = F 7445 / 7517 11.3 88.7 1.0X + codegen = T hashmap = F 4672 / 4703 18.0 55.7 1.6X + codegen = T hashmap = T 1764 / 1958 47.6 21.0 4.2X */ } @@ -231,18 +258,18 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { def f(): Unit = sqlContext.range(N).selectExpr("id", "cast(id & 1023 as string) as k") .groupBy("k").count().collect() - benchmark.addCase(s"codegen = F") { iter => + benchmark.addCase(s"codegen = F", numIters = 2) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "false") f() } - benchmark.addCase(s"codegen = T hashmap = F") { iter => + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") f() } - benchmark.addCase(s"codegen = T hashmap = T") { iter => + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => sqlContext.setConf("spark.sql.codegen.wholeStage", "true") sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") f() From 64ad9ba27aafa333a6897861fe2aeb4bb83506fc Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 4 May 2016 14:16:57 +0800 Subject: [PATCH 0055/1470] [SPARK-14237][SQL] De-duplicate partition value appending logic in various buildReader() implementations ## What changes were proposed in this pull request? Currently, various `FileFormat` data sources share approximately the same code for partition value appending. This PR tries to eliminate this duplication. A new method `buildReaderWithPartitionValues()` is added to `FileFormat` with a default implementation that appends partition values to `InternalRow`s produced by the reader function returned by `buildReader()`. Special data sources like Parquet, which implements partition value appending inside `buildReader()` because of the vectorized reader, and the Text data source, which doesn't support partitioning, override `buildReaderWithPartitionValues()` and simply delegate to `buildReader()`. This PR brings two benefits: 1. Apparently, it de-duplicates partition value appending logic 2. Now the reader function returned by `buildReader()` is only required to produce `InternalRow`s rather than `UnsafeRow`s if the data source doesn't override `buildReaderWithPartitionValues()`. Because the safe-to-unsafe conversion is also performed while appending partition values. This makes 3rd-party data sources (e.g. spark-avro) easier to implement since they no longer need to access private APIs involving `UnsafeRow`. ## How was this patch tested? Existing tests should do the work. Author: Cheng Lian Closes #12866 from liancheng/spark-14237-simplify-partition-values-appending. (cherry picked from commit bc3760d405cc8c3ffcd957b188afa8b7e3b1f824) Signed-off-by: Cheng Lian --- .../ml/source/libsvm/LibSVMRelation.scala | 17 +------- .../datasources/FileSourceStrategy.scala | 2 +- .../datasources/csv/DefaultSource.scala | 17 ++------ .../datasources/fileSourceInterfaces.scala | 40 +++++++++++++++++++ .../datasources/json/JSONRelation.scala | 10 +---- .../datasources/parquet/ParquetRelation.scala | 14 +++++++ .../datasources/text/DefaultSource.scala | 13 ++++++ .../execution/datasources/csv/CSVSuite.scala | 3 -- .../spark/sql/hive/orc/OrcRelation.scala | 11 +---- 9 files changed, 74 insertions(+), 53 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index ba2e1e2bc269d..5f78fab4ddba0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -204,25 +204,10 @@ class DefaultSource extends FileFormat with DataSourceRegister { val converter = RowEncoder(dataSchema) - val unsafeRowIterator = points.map { pt => + points.map { pt => val features = if (sparse) pt.features.toSparse else pt.features.toDense converter.toRow(Row(pt.label, features)) } - - def toAttribute(f: StructField): AttributeReference = - AttributeReference(f.name, f.dataType, f.nullable, f.metadata)() - - // Appends partition values - val fullOutput = (dataSchema ++ partitionSchema).map(toAttribute) - val requiredOutput = fullOutput.filter { a => - requiredSchema.fieldNames.contains(a.name) || partitionSchema.fieldNames.contains(a.name) - } - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput) - - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) - } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 615906a52e8de..8a93c6ff9a4f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -106,7 +106,7 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}") - val readFile = files.fileFormat.buildReader( + val readFile = files.fileFormat.buildReaderWithPartitionValues( sparkSession = files.sparkSession, dataSchema = files.dataSchema, partitionSchema = files.partitionSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala index 75143e609aaf7..948fac0d58ad7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala @@ -117,20 +117,9 @@ class DefaultSource extends FileFormat with DataSourceRegister { CSVRelation.dropHeaderLine(file, lineIterator, csvOptions) - val unsafeRowIterator = { - val tokenizedIterator = new BulkCsvReader(lineIterator, csvOptions, headers) - val parser = CSVRelation.csvParser(dataSchema, requiredSchema.fieldNames, csvOptions) - tokenizedIterator.flatMap(parser(_).toSeq) - } - - // Appends partition values - val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullOutput, fullOutput) - - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) - } + val tokenizedIterator = new BulkCsvReader(lineIterator, csvOptions, headers) + val parser = CSVRelation.csvParser(dataSchema, requiredSchema.fieldNames, csvOptions) + tokenizedIterator.flatMap(parser(_).toSeq) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 0a3461151c627..24e2bf6d136ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.FileRelation import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.{StringType, StructType} @@ -238,6 +239,45 @@ trait FileFormat { throw new UnsupportedOperationException(s"buildReader is not supported for $this") } + /** + * Exactly the same as [[buildReader]] except that the reader function returned by this method + * appends partition values to [[InternalRow]]s produced by the reader function [[buildReader]] + * returns. + */ + private[sql] def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + val dataReader = buildReader( + sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) + + new (PartitionedFile => Iterator[InternalRow]) with Serializable { + private val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + + private val joinedRow = new JoinedRow() + + // Using lazy val to avoid serialization + private lazy val appendPartitionColumns = + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + override def apply(file: PartitionedFile): Iterator[InternalRow] = { + // Using local val to avoid per-row lazy val check (pre-mature optimization?...) + val converter = appendPartitionColumns + + // Note that we have to apply the converter even though `file.partitionValues` is empty. + // This is because the converter is also responsible for converting safe `InternalRow`s into + // `UnsafeRow`s. + dataReader(file).map { dataRow => + converter(joinedRow(dataRow, file.partitionValues)) + } + } + } + } + /** * Returns a [[OutputWriterFactory]] for generating output writers that can write data. * This method is current used only by FileStreamSinkWriter to generate output writers that diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index 62446583a55b7..4c97abed53ce6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -106,22 +106,14 @@ class DefaultSource extends FileFormat with DataSourceRegister { val columnNameOfCorruptRecord = parsedOptions.columnNameOfCorruptRecord .getOrElse(sparkSession.sessionState.conf.columnNameOfCorruptRecord) - val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - (file: PartitionedFile) => { val lines = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map(_.toString) - val rows = JacksonParser.parseJson( + JacksonParser.parseJson( lines, requiredSchema, columnNameOfCorruptRecord, parsedOptions) - - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - rows.map { row => - appendPartitionColumns(joinedRow(row, file.partitionValues)) - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 79185df67325b..cf5c8e94f468d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -255,6 +255,20 @@ private[sql] class DefaultSource schema.forall(_.dataType.isInstanceOf[AtomicType]) } + override private[sql] def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + // For Parquet data source, `buildReader` already handles partition values appending. Here we + // simply delegate to `buildReader`. + buildReader( + sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) + } + override def buildReader( sparkSession: SparkSession, dataSchema: StructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index 348edfcf7a851..f22c0241d930e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -83,6 +83,19 @@ class DefaultSource extends FileFormat with DataSourceRegister { } } + override private[sql] def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + // Text data source doesn't support partitioning. Here we simply delegate to `buildReader`. + buildReader( + sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) + } + override def buildReader( sparkSession: SparkSession, dataSchema: StructType, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 07f00a086865c..28e59055fa1c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -22,9 +22,6 @@ import java.nio.charset.UnsupportedCharsetException import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import scala.collection.JavaConverters._ - -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index d6a847f3ba3a2..89d258e844280 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -157,20 +157,11 @@ private[sql] class DefaultSource } // Unwraps `OrcStruct`s to `UnsafeRow`s - val unsafeRowIterator = OrcRelation.unwrapOrcStructs( + OrcRelation.unwrapOrcStructs( conf, requiredSchema, Some(orcRecordReader.getObjectInspector.asInstanceOf[StructObjectInspector]), new RecordReaderIterator[OrcStruct](orcRecordReader)) - - // Appends partition values - val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullOutput, fullOutput) - - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) - } } } } From b99f715e86773ed8b3173df0121f6e5e34e18978 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 4 May 2016 00:10:20 -0700 Subject: [PATCH 0056/1470] [SPARK-15029] improve error message for Generate ## What changes were proposed in this pull request? This PR improve the error message for `Generate` in 3 cases: 1. generator is nested in expressions, e.g. `SELECT explode(list) + 1 FROM tbl` 2. generator appears more than one time in SELECT, e.g. `SELECT explode(list), explode(list) FROM tbl` 3. generator appears in other operator which is not project, e.g. `SELECT * FROM tbl SORT BY explode(list)` ## How was this patch tested? new tests in `AnalysisErrorSuite` Author: Wenchen Fan Closes #12810 from cloud-fan/bug. (cherry picked from commit 6c12e801e84565265d2939b920901d1805d5935f) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/Analyzer.scala | 123 ++++++++++++------ .../sql/catalyst/analysis/unresolved.scala | 3 +- .../sql/catalyst/expressions/generators.scala | 17 +-- .../expressions/jsonExpressions.scala | 8 +- .../plans/logical/basicLogicalOperators.scala | 2 +- .../spark/sql/catalyst/util/package.scala | 2 + .../analysis/AnalysisErrorSuite.scala | 20 ++- .../scala/org/apache/spark/sql/Dataset.scala | 13 +- .../spark/sql/execution/GenerateExec.scala | 2 +- .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 7 +- 11 files changed, 126 insertions(+), 73 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 59af5b7095a77..527d5b635a7f9 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef -import org.apache.spark.sql.catalyst.util.usePrettyExpression +import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.types._ /** @@ -91,6 +91,7 @@ class Analyzer( ResolvePivot :: ResolveOrdinalInOrderByAndGroupBy :: ResolveMissingReferences :: + ExtractGenerator :: ResolveGenerate :: ResolveFunctions :: ResolveAliases :: @@ -178,8 +179,8 @@ class Analyzer( case e if !e.resolved => u case g: Generator => MultiAlias(g, Nil) case c @ Cast(ne: NamedExpression, _) => Alias(c, ne.name)() - case e: ExtractValue => Alias(e, usePrettyExpression(e).sql)() - case e => Alias(e, optionalAliasName.getOrElse(usePrettyExpression(e).sql))() + case e: ExtractValue => Alias(e, toPrettySQL(e))() + case e => Alias(e, optionalAliasName.getOrElse(toPrettySQL(e)))() } } }.asInstanceOf[Seq[NamedExpression]] @@ -1278,20 +1279,54 @@ class Analyzer( } /** - * Rewrites table generating expressions that either need one or more of the following in order - * to be resolved: - * - concrete attribute references for their output. - * - to be relocated from a SELECT clause (i.e. from a [[Project]]) into a [[Generate]]). + * Extracts [[Generator]] from the projectList of a [[Project]] operator and create [[Generate]] + * operator under [[Project]]. * - * Names for the output [[Attribute]]s are extracted from [[Alias]] or [[MultiAlias]] expressions - * that wrap the [[Generator]]. If more than one [[Generator]] is found in a Project, an - * [[AnalysisException]] is throw. + * This rule will throw [[AnalysisException]] for following cases: + * 1. [[Generator]] is nested in expressions, e.g. `SELECT explode(list) + 1 FROM tbl` + * 2. more than one [[Generator]] is found in projectList, + * e.g. `SELECT explode(list), explode(list) FROM tbl` + * 3. [[Generator]] is found in other operators that are not [[Project]] or [[Generate]], + * e.g. `SELECT * FROM tbl SORT BY explode(list)` */ - object ResolveGenerate extends Rule[LogicalPlan] { + object ExtractGenerator extends Rule[LogicalPlan] { + private def hasGenerator(expr: Expression): Boolean = { + expr.find(_.isInstanceOf[Generator]).isDefined + } + + private def hasNestedGenerator(expr: NamedExpression): Boolean = expr match { + case UnresolvedAlias(_: Generator, _) => false + case Alias(_: Generator, _) => false + case MultiAlias(_: Generator, _) => false + case other => hasGenerator(other) + } + + private def trimAlias(expr: NamedExpression): Expression = expr match { + case UnresolvedAlias(child, _) => child + case Alias(child, _) => child + case MultiAlias(child, _) => child + case _ => expr + } + + /** Extracts a [[Generator]] expression and any names assigned by aliases to their output. */ + private object AliasedGenerator { + def unapply(e: Expression): Option[(Generator, Seq[String])] = e match { + case Alias(g: Generator, name) if g.resolved => Some((g, name :: Nil)) + case MultiAlias(g: Generator, names) if g.resolved => Some(g, names) + case _ => None + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case p: Generate if !p.child.resolved || !p.generator.resolved => p - case g: Generate if !g.resolved => - g.copy(generatorOutput = makeGeneratorOutput(g.generator, g.generatorOutput.map(_.name))) + case Project(projectList, _) if projectList.exists(hasNestedGenerator) => + val nestedGenerator = projectList.find(hasNestedGenerator).get + throw new AnalysisException("Generators are not supported when it's nested in " + + "expressions, but got: " + toPrettySQL(trimAlias(nestedGenerator))) + + case Project(projectList, _) if projectList.count(hasGenerator) > 1 => + val generators = projectList.filter(hasGenerator).map(trimAlias) + throw new AnalysisException("Only one generator allowed per select clause but found " + + generators.size + ": " + generators.map(toPrettySQL).mkString(", ")) case p @ Project(projectList, child) => // Holds the resolved generator, if one exists in the project list. @@ -1299,11 +1334,9 @@ class Analyzer( val newProjectList = projectList.flatMap { case AliasedGenerator(generator, names) if generator.childrenResolved => - if (resolvedGenerator != null) { - failAnalysis( - s"Only one generator allowed per select but ${resolvedGenerator.nodeName} and " + - s"and ${generator.nodeName} found.") - } + // It's a sanity check, this should not happen as the previous case will throw + // exception earlier. + assert(resolvedGenerator == null, "More than one generator found in SELECT.") resolvedGenerator = Generate( @@ -1311,7 +1344,7 @@ class Analyzer( join = projectList.size > 1, // Only join if there are other expressions in SELECT. outer = false, qualifier = None, - generatorOutput = makeGeneratorOutput(generator, names), + generatorOutput = ResolveGenerate.makeGeneratorOutput(generator, names), child) resolvedGenerator.generatorOutput @@ -1323,44 +1356,50 @@ class Analyzer( } else { p } + + case g: Generate => g + + case p if p.expressions.exists(hasGenerator) => + throw new AnalysisException("Generators are not supported outside the SELECT clause, but " + + "got: " + p.simpleString) } + } - /** Extracts a [[Generator]] expression and any names assigned by aliases to their output. */ - private object AliasedGenerator { - def unapply(e: Expression): Option[(Generator, Seq[String])] = e match { - case Alias(g: Generator, name) if g.resolved && g.elementTypes.size > 1 => - // If not given the default names, and the TGF with multiple output columns - failAnalysis( - s"""Expect multiple names given for ${g.getClass.getName}, - |but only single name '${name}' specified""".stripMargin) - case Alias(g: Generator, name) if g.resolved => Some((g, name :: Nil)) - case MultiAlias(g: Generator, names) if g.resolved => Some(g, names) - case _ => None - } + /** + * Rewrites table generating expressions that either need one or more of the following in order + * to be resolved: + * - concrete attribute references for their output. + * - to be relocated from a SELECT clause (i.e. from a [[Project]]) into a [[Generate]]). + * + * Names for the output [[Attribute]]s are extracted from [[Alias]] or [[MultiAlias]] expressions + * that wrap the [[Generator]]. + */ + object ResolveGenerate extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case g: Generate if !g.child.resolved || !g.generator.resolved => g + case g: Generate if !g.resolved => + g.copy(generatorOutput = makeGeneratorOutput(g.generator, g.generatorOutput.map(_.name))) } /** * Construct the output attributes for a [[Generator]], given a list of names. If the list of * names is empty names are assigned from field names in generator. */ - private def makeGeneratorOutput( + private[sql] def makeGeneratorOutput( generator: Generator, names: Seq[String]): Seq[Attribute] = { - val elementTypes = generator.elementTypes + val elementAttrs = generator.elementSchema.toAttributes - if (names.length == elementTypes.length) { - names.zip(elementTypes).map { - case (name, (t, nullable, _)) => - AttributeReference(name, t, nullable)() + if (names.length == elementAttrs.length) { + names.zip(elementAttrs).map { + case (name, attr) => attr.withName(name) } } else if (names.isEmpty) { - elementTypes.map { - case (t, nullable, name) => AttributeReference(name, t, nullable)() - } + elementAttrs } else { failAnalysis( "The number of aliases supplied in the AS clause does not match the number of columns " + - s"output by the UDTF expected ${elementTypes.size} aliases but got " + + s"output by the UDTF expected ${elementAttrs.size} aliases but got " + s"${names.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 f82b63ad96764..1f1897dc36df2 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 @@ -142,8 +142,7 @@ object UnresolvedAttribute { case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expression]) extends Generator { - override def elementTypes: Seq[(DataType, Boolean, String)] = - throw new UnresolvedException(this, "elementTypes") + override def elementSchema: StructType = throw new UnresolvedException(this, "elementTypes") override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") 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 65d7a1d5a0904..12c35644e564c 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 @@ -41,19 +41,16 @@ import org.apache.spark.sql.types._ */ trait Generator extends Expression { - // TODO ideally we should return the type of ArrayType(StructType), - // however, we don't keep the output field names in the Generator. - override def dataType: DataType = throw new UnsupportedOperationException + override def dataType: DataType = ArrayType(elementSchema) override def foldable: Boolean = false override def nullable: Boolean = false /** - * The output element data types in structure of Seq[(DataType, Nullable)] - * TODO we probably need to add more information like metadata etc. + * The output element schema. */ - def elementTypes: Seq[(DataType, Boolean, String)] + def elementSchema: StructType /** Should be implemented by child classes to perform specific Generators. */ override def eval(input: InternalRow): TraversableOnce[InternalRow] @@ -69,7 +66,7 @@ trait Generator extends Expression { * A generator that produces its output using the provided lambda function. */ case class UserDefinedGenerator( - elementTypes: Seq[(DataType, Boolean, String)], + elementSchema: StructType, function: Row => TraversableOnce[InternalRow], children: Seq[Expression]) extends Generator with CodegenFallback { @@ -117,10 +114,10 @@ case class Explode(child: Expression) extends UnaryExpression with Generator wit } // hive-compatible default alias for explode function ("col" for array, "key", "value" for map) - override def elementTypes: Seq[(DataType, Boolean, String)] = child.dataType match { - case ArrayType(et, containsNull) => (et, containsNull, "col") :: Nil + override def elementSchema: StructType = child.dataType match { + case ArrayType(et, containsNull) => new StructType().add("col", et, containsNull) case MapType(kt, vt, valueContainsNull) => - (kt, false, "key") :: (vt, valueContainsNull, "value") :: Nil + new StructType().add("key", kt, false).add("value", vt, valueContainsNull) } override def eval(input: InternalRow): TraversableOnce[InternalRow] = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index ecd09b7083f2e..c14a2fb122618 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -26,7 +26,7 @@ import com.fasterxml.jackson.core._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -356,9 +356,9 @@ case class JsonTuple(children: Seq[Expression]) // and count the number of foldable fields, we'll use this later to optimize evaluation @transient private lazy val constantFields: Int = foldableFieldNames.count(_ != null) - override def elementTypes: Seq[(DataType, Boolean, String)] = fieldExpressions.zipWithIndex.map { - case (_, idx) => (StringType, true, s"c$idx") - } + override def elementSchema: StructType = StructType(fieldExpressions.zipWithIndex.map { + case (_, idx) => StructField(s"c$idx", StringType, nullable = true) + }) override def prettyName: String = "json_tuple" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 7b4615db0661d..8b438e40e6af9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -85,7 +85,7 @@ case class Generate( override lazy val resolved: Boolean = { generator.resolved && childrenResolved && - generator.elementTypes.length == generatorOutput.length && + generator.elementSchema.length == generatorOutput.length && generatorOutput.forall(_.resolved) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index f879b34358a9b..3d2a624ba3b30 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -153,6 +153,8 @@ package object util { "`" + name.replace("`", "``") + "`" } + def toPrettySQL(e: Expression): String = usePrettyExpression(e).sql + /** * Returns the string representation of this expression that is safe to be put in * code comments of generated code. The length is capped at 128 characters. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 2e88f61d491cd..a41383fbf6562 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count} -import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} @@ -330,6 +329,25 @@ class AnalysisErrorSuite extends AnalysisTest { "The start time" :: "must be greater than or equal to 0." :: Nil ) + errorTest( + "generator nested in expressions", + listRelation.select(Explode('list) + 1), + "Generators are not supported when it's nested in expressions, but got: (explode(list) + 1)" + :: Nil + ) + + errorTest( + "generator appears in operator which is not Project", + listRelation.sortBy(Explode('list).asc), + "Generators are not supported outside the SELECT clause, but got: Sort" :: Nil + ) + + errorTest( + "more than one generators in SELECT", + listRelation.select(Explode('list), Explode('list)), + "Only one generator allowed per select clause but found 2: explode(list), explode(list)" :: Nil + ) + test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) // Since we manually construct the logical plan at here and Sum only accept diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 1bea72c4711f1..31dd64e909bb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1578,16 +1578,13 @@ class Dataset[T] private[sql]( */ @Experimental def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { - val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] + val elementSchema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] - val elementTypes = schema.toAttributes.map { - attr => (attr.dataType, attr.nullable, attr.name) } - val names = schema.toAttributes.map(_.name) - val convert = CatalystTypeConverters.createToCatalystConverter(schema) + val convert = CatalystTypeConverters.createToCatalystConverter(elementSchema) val rowFunction = f.andThen(_.map(convert(_).asInstanceOf[InternalRow])) - val generator = UserDefinedGenerator(elementTypes, rowFunction, input.map(_.expr)) + val generator = UserDefinedGenerator(elementSchema, rowFunction, input.map(_.expr)) withPlan { Generate(generator, join = true, outer = false, @@ -1614,13 +1611,13 @@ class Dataset[T] private[sql]( val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil // TODO handle the metadata? - val elementTypes = attributes.map { attr => (attr.dataType, attr.nullable, attr.name) } + val elementSchema = attributes.toStructType def rowFunction(row: Row): TraversableOnce[InternalRow] = { val convert = CatalystTypeConverters.createToCatalystConverter(dataType) f(row(0).asInstanceOf[A]).map(o => InternalRow(convert(o))) } - val generator = UserDefinedGenerator(elementTypes, rowFunction, apply(inputColumn).expr :: Nil) + val generator = UserDefinedGenerator(elementSchema, rowFunction, apply(inputColumn).expr :: Nil) withPlan { Generate(generator, join = true, outer = false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 934bc38dc47cb..8b62c5507c0c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -66,7 +66,7 @@ case class GenerateExec( // boundGenerator.terminate() should be triggered after all of the rows in the partition val rows = if (join) { child.execute().mapPartitionsInternal { iter => - val generatorNullRow = new GenericInternalRow(generator.elementTypes.size) + val generatorNullRow = new GenericInternalRow(generator.elementSchema.length) val joinedRow = new JoinedRow iter.flatMap { row => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index f023edbd96dbe..3220f143aa23f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -145,7 +145,7 @@ private[sql] class HiveSessionCatalog( udaf } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { val udtf = HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), children) - udtf.elementTypes // Force it to check input data types. + udtf.elementSchema // Force it to check input data types. udtf } else { throw new AnalysisException(s"No handler for Hive UDF '${clazz.getCanonicalName}'") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 5aab4132bc4ce..c53675694f620 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -202,9 +202,10 @@ private[hive] case class HiveGenericUDTF( @transient protected lazy val collector = new UDTFCollector - override lazy val elementTypes = outputInspector.getAllStructFieldRefs.asScala.map { - field => (inspectorToDataType(field.getFieldObjectInspector), true, field.getFieldName) - } + override lazy val elementSchema = StructType(outputInspector.getAllStructFieldRefs.asScala.map { + field => StructField(field.getFieldName, inspectorToDataType(field.getFieldObjectInspector), + nullable = true) + }) @transient private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray From 1fba7595f2e9cac984692238c6698bd5cfb0a453 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 4 May 2016 16:44:09 +0800 Subject: [PATCH 0057/1470] [SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] " DDL command ## What changes were proposed in this pull request? This PR implements native `DESC [EXTENDED | FORMATTED]
    ` DDL command. Sample output: ``` scala> spark.sql("desc extended src").show(100, truncate = false) +----------------------------+---------------------------------+-------+ |col_name |data_type |comment| +----------------------------+---------------------------------+-------+ |key |int | | |value |string | | | | | | |# Detailed Table Information|CatalogTable(`default`.`src`, ...| | +----------------------------+---------------------------------+-------+ scala> spark.sql("desc formatted src").show(100, truncate = false) +----------------------------+----------------------------------------------------------+-------+ |col_name |data_type |comment| +----------------------------+----------------------------------------------------------+-------+ |key |int | | |value |string | | | | | | |# Detailed Table Information| | | |Database: |default | | |Owner: |lian | | |Create Time: |Mon Jan 04 17:06:00 CST 2016 | | |Last Access Time: |Thu Jan 01 08:00:00 CST 1970 | | |Location: |hdfs://localhost:9000/user/hive/warehouse_hive121/src | | |Table Type: |MANAGED | | |Table Parameters: | | | | transient_lastDdlTime |1451898360 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.TextInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat| | |Num Buckets: |-1 | | |Bucket Columns: |[] | | |Sort Columns: |[] | | |Storage Desc Parameters: | | | | serialization.format |1 | | +----------------------------+----------------------------------------------------------+-------+ ``` ## How was this patch tested? A test case is added to `HiveDDLSuite` to check command output. Author: Cheng Lian Closes #12844 from liancheng/spark-14127-desc-table. (cherry picked from commit f152fae306dc75565cb4648ee1211416d7c0bb23) Signed-off-by: Cheng Lian --- .../sql/catalyst/catalog/interface.scala | 5 +- .../catalog/ExternalCatalogSuite.scala | 1 + .../spark/sql/execution/SparkSqlParser.scala | 14 ++- .../command/createDataSourceTables.scala | 2 + .../spark/sql/execution/command/tables.scala | 104 ++++++++++++++---- .../sql/execution/command/DDLSuite.scala | 6 +- .../sql/hive/client/HiveClientImpl.scala | 4 +- .../hive/execution/CreateTableAsSelect.scala | 3 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 1 + .../spark/sql/hive/client/VersionsSuite.scala | 1 + .../sql/hive/execution/HiveDDLSuite.scala | 17 +++ 12 files changed, 131 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 3851e4c706742..2c6e9f53b27b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -48,6 +48,7 @@ case class CatalogStorageFormat( inputFormat: Option[String], outputFormat: Option[String], serde: Option[String], + compressed: Boolean, serdeProperties: Map[String, String]) @@ -89,6 +90,7 @@ case class CatalogTable( sortColumnNames: Seq[String] = Seq.empty, bucketColumnNames: Seq[String] = Seq.empty, numBuckets: Int = -1, + owner: String = "", createTime: Long = System.currentTimeMillis, lastAccessTime: Long = -1, properties: Map[String, String] = Map.empty, @@ -123,10 +125,11 @@ case class CatalogTable( locationUri: Option[String] = storage.locationUri, inputFormat: Option[String] = storage.inputFormat, outputFormat: Option[String] = storage.outputFormat, + compressed: Boolean = false, serde: Option[String] = storage.serde, serdeProperties: Map[String, String] = storage.serdeProperties): CatalogTable = { copy(storage = CatalogStorageFormat( - locationUri, inputFormat, outputFormat, serde, serdeProperties)) + locationUri, inputFormat, outputFormat, serde, compressed, serdeProperties)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index d739b177430c7..ae7c503e65cc0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -507,6 +507,7 @@ abstract class CatalogTestUtils { inputFormat = Some(tableInputFormat), outputFormat = Some(tableOutputFormat), serde = None, + compressed = false, serdeProperties = Map.empty) lazy val part1 = CatalogTablePartition(Map("a" -> "1", "b" -> "2"), storageFormat) lazy val part2 = CatalogTablePartition(Map("a" -> "3", "b" -> "4"), storageFormat) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index b000cc99533e3..60388df596464 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -243,10 +243,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { // FORMATTED and columns are not supported. Return null and let the parser decide what to do // with this (create an exception or pass it on to a different system). - if (ctx.describeColName != null || ctx.FORMATTED != null || ctx.partitionSpec != null) { + if (ctx.describeColName != null || ctx.partitionSpec != null) { null } else { - DescribeTableCommand(visitTableIdentifier(ctx.tableIdentifier), ctx.EXTENDED != null) + DescribeTableCommand( + visitTableIdentifier(ctx.tableIdentifier), + ctx.EXTENDED != null, + ctx.FORMATTED() != null) } } @@ -766,6 +769,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Note: Keep this unspecified because we use the presence of the serde to decide // whether to convert a table created by CTAS to a datasource table. serde = None, + compressed = false, serdeProperties = Map()) } val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) @@ -777,6 +781,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), + compressed = false, serdeProperties = rowStorage.serdeProperties ++ fileStorage.serdeProperties) // TODO support the sql text - have a proper location for this! @@ -830,7 +835,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** Empty storage format for default values and copies. */ - private val EmptyStorageFormat = CatalogStorageFormat(None, None, None, None, Map.empty) + private val EmptyStorageFormat = CatalogStorageFormat(None, None, None, None, false, Map.empty) /** * Create a [[CatalogStorageFormat]]. @@ -911,6 +916,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { entry("field.delim", ctx.fieldsTerminatedBy) ++ entry("serialization.format", ctx.fieldsTerminatedBy) ++ entry("escape.delim", ctx.escapedBy) ++ + // The following typo is inherited from Hive... entry("colelction.delim", ctx.collectionItemsTerminatedBy) ++ entry("mapkey.delim", ctx.keysTerminatedBy) ++ Option(ctx.linesSeparatedBy).toSeq.map { token => @@ -1051,7 +1057,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case c: RowFormatSerdeContext => // Use a serde format. - val CatalogStorageFormat(None, None, None, Some(name), props) = visitRowFormatSerde(c) + val CatalogStorageFormat(None, None, None, Some(name), _, props) = visitRowFormatSerde(c) // SPARK-10310: Special cases LazySimpleSerDe val recordHandler = if (name == "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index f670f63472bf7..e07ab99ef3e82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -349,6 +349,7 @@ object CreateDataSourceTableUtils extends Logging { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = options ), properties = tableProperties.toMap) @@ -368,6 +369,7 @@ object CreateDataSourceTableUtils extends Logging { inputFormat = serde.inputFormat, outputFormat = serde.outputFormat, serde = serde.serde, + compressed = false, serdeProperties = options ), schema = relation.schema.map { f => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 489c980c1654e..31c804f7a3d5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -19,16 +19,17 @@ package org.apache.spark.sql.execution.command import java.io.File import java.net.URI +import java.util.Date import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogRelation, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, UnaryNode} -import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType} +import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType, StructType} import org.apache.spark.util.Utils case class CreateTableAsSelectLogicalPlan( @@ -269,10 +270,10 @@ case class LoadData( /** * Command that looks like * {{{ - * DESCRIBE (EXTENDED) table_name; + * DESCRIBE [EXTENDED|FORMATTED] table_name; * }}} */ -case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean) +case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isFormatted: Boolean) extends RunnableCommand { override val output: Seq[Attribute] = Seq( @@ -289,29 +290,92 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean) val result = new ArrayBuffer[Row] sparkSession.sessionState.catalog.lookupRelation(table) match { case catalogRelation: CatalogRelation => - catalogRelation.catalogTable.schema.foreach { column => - result += Row(column.name, column.dataType, column.comment.orNull) - } - - if (catalogRelation.catalogTable.partitionColumns.nonEmpty) { - result += Row("# Partition Information", "", "") - result += Row(s"# ${output(0).name}", output(1).name, output(2).name) - - catalogRelation.catalogTable.partitionColumns.foreach { col => - result += Row(col.name, col.dataType, col.comment.orNull) - } + if (isExtended) { + describeExtended(catalogRelation, result) + } else if (isFormatted) { + describeFormatted(catalogRelation, result) + } else { + describe(catalogRelation, result) } case relation => - relation.schema.fields.foreach { field => - val comment = - if (field.metadata.contains("comment")) field.metadata.getString("comment") else "" - result += Row(field.name, field.dataType.simpleString, comment) - } + describeSchema(relation.schema, result) } result } + + // Shows data columns and partitioned columns (if any) + private def describe(relation: CatalogRelation, buffer: ArrayBuffer[Row]): Unit = { + describeSchema(relation.catalogTable.schema, buffer) + + if (relation.catalogTable.partitionColumns.nonEmpty) { + append(buffer, "# Partition Information", "", "") + append(buffer, s"# ${output(0).name}", output(1).name, output(2).name) + describeSchema(relation.catalogTable.partitionColumns, buffer) + } + } + + private def describeExtended(relation: CatalogRelation, buffer: ArrayBuffer[Row]): Unit = { + describe(relation, buffer) + + append(buffer, "", "", "") + append(buffer, "# Detailed Table Information", relation.catalogTable.toString, "") + } + + private def describeFormatted(relation: CatalogRelation, buffer: ArrayBuffer[Row]): Unit = { + describe(relation, buffer) + + val table = relation.catalogTable + + append(buffer, "", "", "") + append(buffer, "# Detailed Table Information", "", "") + append(buffer, "Database:", table.database, "") + append(buffer, "Owner:", table.owner, "") + append(buffer, "Create Time:", new Date(table.createTime).toString, "") + append(buffer, "Last Access Time:", new Date(table.lastAccessTime).toString, "") + append(buffer, "Location:", table.storage.locationUri.getOrElse(""), "") + append(buffer, "Table Type:", table.tableType.name, "") + + append(buffer, "Table Parameters:", "", "") + table.properties.foreach { case (key, value) => + append(buffer, s" $key", value, "") + } + + append(buffer, "", "", "") + append(buffer, "# Storage Information", "", "") + table.storage.serde.foreach(serdeLib => append(buffer, "SerDe Library:", serdeLib, "")) + table.storage.inputFormat.foreach(format => append(buffer, "InputFormat:", format, "")) + table.storage.outputFormat.foreach(format => append(buffer, "OutputFormat:", format, "")) + append(buffer, "Compressed:", if (table.storage.compressed) "Yes" else "No", "") + append(buffer, "Num Buckets:", table.numBuckets.toString, "") + append(buffer, "Bucket Columns:", table.bucketColumnNames.mkString("[", ", ", "]"), "") + append(buffer, "Sort Columns:", table.sortColumnNames.mkString("[", ", ", "]"), "") + + append(buffer, "Storage Desc Parameters:", "", "") + table.storage.serdeProperties.foreach { case (key, value) => + append(buffer, s" $key", value, "") + } + } + + private def describeSchema(schema: StructType, buffer: ArrayBuffer[Row]): Unit = { + schema.foreach { column => + val comment = + if (column.metadata.contains("comment")) column.metadata.getString("comment") else "" + append(buffer, column.name, column.dataType.simpleString, comment) + } + } + + private def describeSchema(schema: Seq[CatalogColumn], buffer: ArrayBuffer[Row]): Unit = { + schema.foreach { column => + append(buffer, column.name, column.dataType.toLowerCase, column.comment.orNull) + } + } + + private def append( + buffer: ArrayBuffer[Row], column: String, dataType: String, comment: String): Unit = { + buffer += Row(column, dataType, comment) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 12acb9f2761df..0ae099ecc2bdd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -76,6 +76,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map()) catalog.createTable(CatalogTable( identifier = name, @@ -89,7 +90,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog: SessionCatalog, spec: TablePartitionSpec, tableName: TableIdentifier): Unit = { - val part = CatalogTablePartition(spec, CatalogStorageFormat(None, None, None, None, Map())) + val part = CatalogTablePartition( + spec, CatalogStorageFormat(None, None, None, None, false, Map())) catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) } @@ -264,6 +266,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map()) val expectedTable = CatalogTable( @@ -288,6 +291,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map()) val expectedTable = CatalogTable( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 47d9546c4f515..cddc0b6e34a44 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -351,6 +351,7 @@ private[hive] class HiveClientImpl( sortColumnNames = Seq(), // TODO: populate this bucketColumnNames = h.getBucketCols.asScala, numBuckets = h.getNumBuckets, + owner = h.getOwner, createTime = h.getTTable.getCreateTime.toLong * 1000, lastAccessTime = h.getLastAccessTime.toLong * 1000, storage = CatalogStorageFormat( @@ -358,6 +359,7 @@ private[hive] class HiveClientImpl( inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), serde = Option(h.getSerializationLib), + compressed = h.getTTable.getSd.isCompressed, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap ), properties = h.getParameters.asScala.toMap, @@ -788,7 +790,7 @@ private[hive] class HiveClientImpl( inputFormat = Option(apiPartition.getSd.getInputFormat), outputFormat = Option(apiPartition.getSd.getOutputFormat), serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), + compressed = apiPartition.getSd.isCompressed, serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) } - } 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 08d4b99d30ea8..9dfbafae872f6 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 @@ -56,7 +56,8 @@ case class CreateTableAsSelect( outputFormat = tableDesc.storage.outputFormat .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), - serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName))) + serde = tableDesc.storage.serde.orElse(Some(classOf[LazySimpleSerDe].getName)), + compressed = tableDesc.storage.compressed) val withSchema = if (withFormat.schema.isEmpty) { // Hive doesn't support specifying the column list for target table in CTAS diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 8dc3c64353276..c4ebc604dc13c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -579,7 +579,7 @@ class HiveDDLCommandSuite extends PlanTest { assert(source2.table == "table2") } - test("load data") { + test("load data") { val v1 = "LOAD DATA INPATH 'path' INTO TABLE table1" val (table, path, isLocal, isOverwrite, partition) = parser.parsePlan(v1).collect { case LoadData(t, path, l, o, partition) => (t, path, l, o, partition) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index c3a9f2479ce74..4bdcb96feb0ad 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -732,6 +732,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv inputFormat = None, outputFormat = None, serde = None, + compressed = false, serdeProperties = Map( "path" -> sessionState.catalog.hiveDefaultTableFilePath(TableIdentifier(tableName))) ), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 9341b3816fea7..a6a5ab3988fc9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -157,6 +157,7 @@ class VersionsSuite extends SparkFunSuite with Logging { outputFormat = Some( classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), serde = Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName()), + compressed = false, serdeProperties = Map.empty )) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 687a4a7e512ad..373d1a1e0ebc1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -348,4 +348,21 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } } + + test("desc table") { + withTable("tab1") { + val tabName = "tab1" + sql(s"CREATE TABLE $tabName(c1 int)") + + assert(sql(s"DESC $tabName").collect().length == 1) + + assert( + sql(s"DESC FORMATTED $tabName").collect() + .exists(_.getString(0) == "# Storage Information")) + + assert( + sql(s"DESC EXTENDED $tabName").collect() + .exists(_.getString(0) == "# Detailed Table Information")) + } + } } From dcbe85ff20736915bba6c3269221c3367ee798c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dominik=20Jastrz=C4=99bski?= Date: Wed, 4 May 2016 14:25:51 +0200 Subject: [PATCH 0058/1470] =?UTF-8?q?[SPARK-14844][ML]=20Add=20setFeatures?= =?UTF-8?q?Col=20and=20setPredictionCol=20to=20KMeansM=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Introduction of setFeaturesCol and setPredictionCol methods to KMeansModel in ML library. ## How was this patch tested? By running KMeansSuite. Author: Dominik Jastrzębski Closes #12609 from dominik-jastrzebski/master. (cherry picked from commit abecbcd5e9598471b705a2f701731af1adc9d48b) Signed-off-by: Nick Pentreath --- .../org/apache/spark/ml/clustering/KMeans.scala | 8 ++++++++ .../apache/spark/ml/clustering/KMeansSuite.scala | 15 +++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 7c9ac02521ff6..42a25396adb40 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -105,6 +105,14 @@ class KMeansModel private[ml] ( copyValues(copied, extra) } + /** @group setParam */ + @Since("2.0.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val predictUDF = udf((vector: Vector) => predict(vector)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 2ca386e4229ca..241d21961fae1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -117,6 +117,21 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR assert(clusterSizes.forall(_ >= 0)) } + test("KMeansModel transform with non-default feature and prediction cols") { + val featuresColName = "kmeans_model_features" + val predictionColName = "kmeans_model_prediction" + + val model = new KMeans().setK(k).setSeed(1).fit(dataset) + model.setFeaturesCol(featuresColName).setPredictionCol(predictionColName) + + val transformed = model.transform(dataset.withColumnRenamed("features", featuresColName)) + Seq(featuresColName, predictionColName).foreach { column => + assert(transformed.columns.contains(column)) + } + assert(model.getFeaturesCol == featuresColName) + assert(model.getPredictionCol == predictionColName) + } + test("read/write") { def checkModelData(model: KMeansModel, model2: KMeansModel): Unit = { assert(model.clusterCenters === model2.clusterCenters) From ae79032dcf160796851ca29116cca146c4d86ada Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Wed, 4 May 2016 08:45:43 -0500 Subject: [PATCH 0059/1470] [SPARK-4224][CORE][YARN] Support group acls ## What changes were proposed in this pull request? Currently only a list of users can be specified for view and modify acls. This change enables a group of admins/devs/users to be provisioned for viewing and modifying Spark jobs. **Changes Proposed in the fix** Three new corresponding config entries have been added where the user can specify the groups to be given access. ``` spark.admin.acls.groups spark.modify.acls.groups spark.ui.view.acls.groups ``` New config entries were added because specifying the users and groups explicitly is a better and cleaner way compared to specifying them in the existing config entry using a delimiter. A generic trait has been introduced to provide the user to group mapping which makes it pluggable to support a variety of mapping protocols - similar to the one used in hadoop. A default unix shell based implementation has been provided. Custom user to group mapping protocol can be specified and configured by the entry ```spark.user.groups.mapping``` **How the patch was Tested** We ran different spark jobs setting the config entries in combinations of admin, modify and ui acls. For modify acls we tried killing the job stages from the ui and using yarn commands. For view acls we tried accessing the UI tabs and the logs. Headless accounts were used to launch these jobs and different users tried to modify and view the jobs to ensure that the groups mapping applied correctly. Additional Unit tests have been added without modifying the existing ones. These test for different ways of setting the acls through configuration and/or API and validate the expected behavior. Author: Dhruve Ashar Closes #12760 from dhruve/impr/SPARK-4224. (cherry picked from commit a45647746d1efb90cb8bc142c2ef110a0db9bc9f) Signed-off-by: Tom Graves --- .../org/apache/spark/SecurityManager.scala | 124 ++++++++--- .../deploy/history/FsHistoryProvider.scala | 2 + .../scheduler/ApplicationEventListener.scala | 4 + .../GroupMappingServiceProvider.scala | 38 ++++ .../ShellBasedGroupsMappingProvider.scala | 45 ++++ .../scala/org/apache/spark/util/Utils.scala | 19 ++ .../apache/spark/SecurityManagerSuite.scala | 198 ++++++++++++++++++ docs/configuration.md | 55 ++++- docs/monitoring.md | 4 +- docs/security.md | 6 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 8 +- 11 files changed, 468 insertions(+), 35 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala create mode 100644 core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index e8f68224d5976..f72c7ded5ea52 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -50,17 +50,19 @@ import org.apache.spark.util.Utils * secure the UI if it has data that other users should not be allowed to see. The javax * servlet filter specified by the user can authenticate the user and then once the user * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.acls.enable' and 'spark.ui.view.acls' - * control the behavior of the acls. Note that the person who started the application - * always has view access to the UI. + * authorized to view the UI. The configs 'spark.acls.enable', 'spark.ui.view.acls' and + * 'spark.ui.view.acls.groups' control the behavior of the acls. Note that the person who + * started the application always has view access to the UI. * - * Spark has a set of modify acls (`spark.modify.acls`) that controls which users have permission - * to modify a single application. This would include things like killing the application. By - * default the person who started the application has modify access. For modify access through - * the UI, you must have a filter that does authentication in place for the modify acls to work - * properly. + * Spark has a set of individual and group modify acls (`spark.modify.acls`) and + * (`spark.modify.acls.groups`) that controls which users and groups have permission to + * modify a single application. This would include things like killing the application. + * By default the person who started the application has modify access. For modify access + * through the UI, you must have a filter that does authentication in place for the modify + * acls to work properly. * - * Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators + * Spark also has a set of individual and group admin acls (`spark.admin.acls`) and + * (`spark.admin.acls.groups`) which is a set of users/administrators and admin groups * who always have permission to view or modify the Spark application. * * Starting from version 1.3, Spark has partial support for encrypted connections with SSL. @@ -184,6 +186,9 @@ private[spark] class SecurityManager(sparkConf: SparkConf) import SecurityManager._ + // allow all users/groups to have view/modify permissions + private val WILDCARD_ACL = "*" + private val authOn = sparkConf.getBoolean(SecurityManager.SPARK_AUTH_CONF, false) // keep spark.ui.acls.enable for backwards compatibility with 1.0 private var aclsOn = @@ -193,12 +198,20 @@ private[spark] class SecurityManager(sparkConf: SparkConf) private var adminAcls: Set[String] = stringToSet(sparkConf.get("spark.admin.acls", "")) + // admin group acls should be set before view or modify group acls + private var adminAclsGroups : Set[String] = + stringToSet(sparkConf.get("spark.admin.acls.groups", "")) + private var viewAcls: Set[String] = _ + private var viewAclsGroups: Set[String] = _ + // list of users who have permission to modify the application. This should // apply to both UI and CLI for things like killing the application. private var modifyAcls: Set[String] = _ + private var modifyAclsGroups: Set[String] = _ + // always add the current user and SPARK_USER to the viewAcls private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Utils.getCurrentUserName()) @@ -206,11 +219,16 @@ private[spark] class SecurityManager(sparkConf: SparkConf) setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) + setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", "")); + setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", "")); + private val secretKey = generateSecretKey() logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + "; ui acls " + (if (aclsOn) "enabled" else "disabled") + - "; users with view permissions: " + viewAcls.toString() + - "; users with modify permissions: " + modifyAcls.toString()) + "; users with view permissions: " + viewAcls.toString() + + "; groups with view permissions: " + viewAclsGroups.toString() + + "; users with modify permissions: " + modifyAcls.toString() + + "; groups with modify permissions: " + modifyAclsGroups.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. // This is needed by the HTTP client fetching from the HttpServer. Put here so its @@ -302,17 +320,34 @@ private[spark] class SecurityManager(sparkConf: SparkConf) setViewAcls(Set[String](defaultUser), allowedUsers) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setViewAclsGroups(allowedUserGroups: String) { + viewAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + logInfo("Changing view acls groups to: " + viewAclsGroups.mkString(",")) + } + /** * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" */ def getViewAcls: String = { - if (viewAcls.contains("*")) { - "*" + if (viewAcls.contains(WILDCARD_ACL)) { + WILDCARD_ACL } else { viewAcls.mkString(",") } } + def getViewAclsGroups: String = { + if (viewAclsGroups.contains(WILDCARD_ACL)) { + WILDCARD_ACL + } else { + viewAclsGroups.mkString(",") + } + } + /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. @@ -322,17 +357,34 @@ private[spark] class SecurityManager(sparkConf: SparkConf) logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setModifyAclsGroups(allowedUserGroups: String) { + modifyAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + logInfo("Changing modify acls groups to: " + modifyAclsGroups.mkString(",")) + } + /** * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" */ def getModifyAcls: String = { - if (modifyAcls.contains("*")) { - "*" + if (modifyAcls.contains(WILDCARD_ACL)) { + WILDCARD_ACL } else { modifyAcls.mkString(",") } } + def getModifyAclsGroups: String = { + if (modifyAclsGroups.contains(WILDCARD_ACL)) { + WILDCARD_ACL + } else { + modifyAclsGroups.mkString(",") + } + } + /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. @@ -342,6 +394,15 @@ private[spark] class SecurityManager(sparkConf: SparkConf) logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } + /** + * Admin acls groups should be set before the view or modify acls groups. If you modify the admin + * acls groups you should also set the view and modify acls groups again to pick up the changes. + */ + def setAdminAclsGroups(adminUserGroups: String) { + adminAclsGroups = stringToSet(adminUserGroups) + logInfo("Changing admin acls groups to: " + adminAclsGroups.mkString(",")) + } + def setAcls(aclSetting: Boolean) { aclsOn = aclSetting logInfo("Changing acls enabled to: " + aclsOn) @@ -398,36 +459,49 @@ private[spark] class SecurityManager(sparkConf: SparkConf) def aclsEnabled(): Boolean = aclsOn /** - * Checks the given user against the view acl list to see if they have + * Checks the given user against the view acl and groups list to see if they have * authorization to view the UI. If the UI acls are disabled * via spark.acls.enable, all users have view access. If the user is null - * it is assumed authentication is off and all users have access. + * it is assumed authentication is off and all users have access. Also if any one of the + * UI acls or groups specify the WILDCARD(*) then all users have view access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" + - viewAcls.mkString(",")) - !aclsEnabled || user == null || viewAcls.contains(user) || viewAcls.contains("*") + viewAcls.mkString(",") + " viewAclsGroups=" + viewAclsGroups.mkString(",")) + if (!aclsEnabled || user == null || viewAcls.contains(user) || + viewAcls.contains(WILDCARD_ACL) || viewAclsGroups.contains(WILDCARD_ACL)) { + return true + } + val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) + logDebug("userGroups=" + currentUserGroups.mkString(",")) + viewAclsGroups.exists(currentUserGroups.contains(_)) } /** - * Checks the given user against the modify acl list to see if they have - * authorization to modify the application. If the UI acls are disabled + * Checks the given user against the modify acl and groups list to see if they have + * authorization to modify the application. If the modify acls are disabled * via spark.acls.enable, all users have modify access. If the user is null - * it is assumed authentication isn't turned on and all users have access. + * it is assumed authentication isn't turned on and all users have access. Also if any one + * of the modify acls or groups specify the WILDCARD(*) then all users have modify access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkModifyPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" + - modifyAcls.mkString(",")) - !aclsEnabled || user == null || modifyAcls.contains(user) || modifyAcls.contains("*") + modifyAcls.mkString(",") + " modifyAclsGroups=" + modifyAclsGroups.mkString(",")) + if (!aclsEnabled || user == null || modifyAcls.contains(user) || + modifyAcls.contains(WILDCARD_ACL) || modifyAclsGroups.contains(WILDCARD_ACL)) { + return true + } + val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) + logDebug("userGroups=" + currentUserGroups) + modifyAclsGroups.exists(currentUserGroups.contains(_)) } - /** * Check to see if authentication for the Spark communication protocols is enabled * @return true if authentication is enabled, otherwise false diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 07cbcec8e5f0e..110d882f05598 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -245,6 +245,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) ui.getSecurityManager.setViewAcls(attempt.sparkUser, appListener.viewAcls.getOrElse("")) + ui.getSecurityManager.setAdminAclsGroups(appListener.adminAclsGroups.getOrElse("")) + ui.getSecurityManager.setViewAclsGroups(appListener.viewAclsGroups.getOrElse("")) LoadedAppUI(ui, updateProbe(appId, attemptId, attempt.fileSize)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 9f218c64cac2d..28c45d800ed06 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -32,6 +32,8 @@ private[spark] class ApplicationEventListener extends SparkListener { var endTime: Option[Long] = None var viewAcls: Option[String] = None var adminAcls: Option[String] = None + var viewAclsGroups: Option[String] = None + var adminAclsGroups: Option[String] = None override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = Some(applicationStart.appName) @@ -51,6 +53,8 @@ private[spark] class ApplicationEventListener extends SparkListener { val allProperties = environmentDetails("Spark Properties").toMap viewAcls = allProperties.get("spark.ui.view.acls") adminAcls = allProperties.get("spark.admin.acls") + viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") + adminAclsGroups = allProperties.get("spark.admin.acls.groups") } } } diff --git a/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala new file mode 100644 index 0000000000000..ea047a4f75d55 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala @@ -0,0 +1,38 @@ +/* + * 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.security + +/** + * This Spark trait is used for mapping a given userName to a set of groups which it belongs to. + * This is useful for specifying a common group of admins/developers to provide them admin, modify + * and/or view access rights. Based on whether access control checks are enabled using + * spark.acls.enable, every time a user tries to access or modify the application, the + * SecurityManager gets the corresponding groups a user belongs to from the instance of the groups + * mapping provider specified by the entry spark.user.groups.mapping. + */ + +trait GroupMappingServiceProvider { + + /** + * Get the groups the user belongs to. + * @param userName User's Name + * @return set of groups that the user belongs to. Empty in case of an invalid user. + */ + def getGroups(userName : String) : Set[String] + +} diff --git a/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala new file mode 100644 index 0000000000000..f71dd08246b2f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala @@ -0,0 +1,45 @@ +/* + * 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.security + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * This class is responsible for getting the groups for a particular user in Unix based + * environments. This implementation uses the Unix Shell based id command to fetch the user groups + * for the specified user. It does not cache the user groups as the invocations are expected + * to be infrequent. + */ + +private[spark] class ShellBasedGroupsMappingProvider extends GroupMappingServiceProvider + with Logging { + + override def getGroups(username: String): Set[String] = { + val userGroups = getUnixGroups(username) + logDebug("User: " + username + " Groups: " + userGroups.mkString(",")) + userGroups + } + + // shells out a "bash -c id -Gn username" to get user groups + private def getUnixGroups(username: String): Set[String] = { + val cmdSeq = Seq("bash", "-c", "id -Gn " + username) + // we need to get rid of the trailing "\n" from the result of command execution + Utils.executeAndGetOutput(cmdSeq).stripLineEnd.split(" ").toSet + } +} 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 ea49991493fd7..a8bb0002a7b25 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2181,6 +2181,25 @@ private[spark] object Utils extends Logging { .getOrElse(UserGroupInformation.getCurrentUser().getShortUserName()) } + val EMPTY_USER_GROUPS = Set[String]() + + // Returns the groups to which the current user belongs. + def getCurrentUserGroups(sparkConf: SparkConf, username: String): Set[String] = { + val groupProviderClassName = sparkConf.get("spark.user.groups.mapping", + "org.apache.spark.security.ShellBasedGroupsMappingProvider") + if (groupProviderClassName != "") { + try { + val groupMappingServiceProvider = classForName(groupProviderClassName).newInstance. + asInstanceOf[org.apache.spark.security.GroupMappingServiceProvider] + val currentUserGroups = groupMappingServiceProvider.getGroups(username) + return currentUserGroups + } catch { + case e: Exception => logError(s"Error getting groups for user=$username", e) + } + } + EMPTY_USER_GROUPS + } + /** * Split the comma delimited string of master URLs into a list. * For instance, "spark://abc,def" becomes [spark://abc, spark://def]. diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index 8bdb237c28f66..9801b2638cc15 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -19,8 +19,18 @@ package org.apache.spark import java.io.File +import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} +class DummyGroupMappingServiceProvider extends GroupMappingServiceProvider { + + val userGroups: Set[String] = Set[String]("group1", "group2", "group3") + + override def getGroups(username: String): Set[String] = { + userGroups + } +} + class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf") { @@ -37,6 +47,45 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user3") === false) } + test("set security with conf for groups") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls.groups", "group1,group2") + // default ShellBasedGroupsMappingProvider is used to resolve user groups + val securityManager = new SecurityManager(conf); + // assuming executing user does not belong to group1,group2 + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf2.set("spark.authenticate", "true") + conf2.set("spark.authenticate.secret", "good") + conf2.set("spark.ui.acls.enable", "true") + conf2.set("spark.ui.view.acls.groups", "group1,group2") + // explicitly specify a custom GroupsMappingServiceProvider + conf2.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager2 = new SecurityManager(conf2); + // group4,group5 do not match + assert(securityManager2.checkUIViewPermissions("user1") === true) + assert(securityManager2.checkUIViewPermissions("user2") === true) + + val conf3 = new SparkConf + conf3.set("spark.authenticate", "true") + conf3.set("spark.authenticate.secret", "good") + conf3.set("spark.ui.acls.enable", "true") + conf3.set("spark.ui.view.acls.groups", "group4,group5") + // explicitly specify a bogus GroupsMappingServiceProvider + conf3.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager3 = new SecurityManager(conf3); + // BogusServiceProvider cannot be loaded and an error is logged returning an empty group set + assert(securityManager3.checkUIViewPermissions("user1") === false) + assert(securityManager3.checkUIViewPermissions("user2") === false) + } + test("set security with api") { val conf = new SparkConf conf.set("spark.ui.view.acls", "user1,user2") @@ -60,6 +109,40 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions(null) === true) } + test("set security with api for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setViewAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + + // change groups so they do not match + securityManager.setViewAclsGroups("group4,group5") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager2 = new SecurityManager(conf2) + securityManager2.setAcls(true) + securityManager2.setViewAclsGroups("group1,group2") + + // group1,group2 do not match because of BogusServiceProvider + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + // setting viewAclsGroups to empty should still not match because of BogusServiceProvider + securityManager2.setViewAclsGroups("") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + } + test("set security modify acls") { val conf = new SparkConf conf.set("spark.modify.acls", "user1,user2") @@ -84,6 +167,29 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions(null) === true) } + test("set security modify acls for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setModifyAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // change groups so they do not match + securityManager.setModifyAclsGroups("group4,group5") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // change so they match again + securityManager.setModifyAclsGroups("group2,group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + } + test("set security admin acls") { val conf = new SparkConf conf.set("spark.admin.acls", "user1,user2") @@ -122,7 +228,48 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user3") === false) assert(securityManager.checkUIViewPermissions(null) === true) + } + + test("set security admin acls for groups") { + val conf = new SparkConf + conf.set("spark.admin.acls.groups", "group1") + conf.set("spark.ui.view.acls.groups", "group2") + conf.set("spark.modify.acls.groups", "group3") + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + + // group1,group2,group3 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + // change admin groups so they do not match. view and modify groups are set to admin groups + securityManager.setAdminAclsGroups("group4,group5") + // invoke the set ui and modify to propagate the changes + securityManager.setViewAclsGroups("") + securityManager.setModifyAclsGroups("") + + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change modify groups so they match + securityManager.setModifyAclsGroups("group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change view groups so they match + securityManager.setViewAclsGroups("group2") + securityManager.setModifyAclsGroups("group4") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === true) + + // change modify and view groups so they do not match + securityManager.setViewAclsGroups("group7") + securityManager.setModifyAclsGroups("group8") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) } test("set security with * in acls") { @@ -166,6 +313,57 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user8") === true) } + test("set security with * in acls for groups") { + val conf = new SparkConf + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.admin.acls.groups", "group4,group5") + conf.set("spark.ui.view.acls.groups", "*") + conf.set("spark.modify.acls.groups", "group6") + + val securityManager = new SecurityManager(conf) + assert(securityManager.aclsEnabled() === true) + + // check for viewAclsGroups with * + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // check for modifyAcls with * + securityManager.setModifyAclsGroups("*") + securityManager.setViewAclsGroups("group6") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // check for adminAcls with * + securityManager.setAdminAclsGroups("group9,*") + securityManager.setModifyAclsGroups("group4,group5") + securityManager.setViewAclsGroups("group6,group7") + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === true) + } + + test("security for groups default behavior") { + // no groups or userToGroupsMapper provided + // this will default to the ShellBasedGroupsMappingProvider + val conf = new SparkConf + + val securityManager = new SecurityManager(conf) + securityManager.setAcls(true) + + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + + // set groups only + securityManager.setAdminAclsGroups("group1,group2") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + } + test("ssl on setup") { val conf = SSLSampleConfigs.sparkSSLConfig() val expectedAlgorithms = Set( diff --git a/docs/configuration.md b/docs/configuration.md index 6512e16faf4c1..9191570d07625 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1231,7 +1231,7 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + @@ -1305,6 +1330,18 @@ Apart from these, the following properties are also available, and may be useful the list means any user can have access to modify it. + + + + + @@ -1328,6 +1365,18 @@ Apart from these, the following properties are also available, and may be useful have view access to this Spark job. + + + + +
    spark.acls.enable false - Whether Spark acls should are enabled. If enabled, this checks to see if the user has + Whether Spark acls should be enabled. If enabled, this checks to see if the user has access permissions to view or modify the job. Note this requires the user to be known, so if the user comes across as null no checks are done. Filters can be used with the UI to authenticate and set the user. @@ -1243,8 +1243,33 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of users/administrators that have view and modify access to all Spark jobs. This can be used if you run on a shared cluster and have a set of administrators or devs who - help debug when things work. Putting a "*" in the list means any user can have the privilege - of admin. + help debug when things do not work. Putting a "*" in the list means any user can have the + privilege of admin. +
    spark.admin.acls.groupsEmpty + Comma separated list of groups that have view and modify access to all Spark jobs. + This can be used if you have a set of administrators or developers who help maintain and debug + the underlying infrastructure. Putting a "*" in the list means any user in any group can have + the privilege of admin. The user groups are obtained from the instance of the groups mapping + provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. +
    spark.user.groups.mappingorg.apache.spark.security.ShellBasedGroupsMappingProvider + The list of groups for a user are determined by a group mapping service defined by the trait + org.apache.spark.security.GroupMappingServiceProvider which can configured by this property. + A default unix shell based implementation is provided org.apache.spark.security.ShellBasedGroupsMappingProvider + which can be specified to resolve a list of groups for a user. + Note: This implementation supports only a Unix/Linux based environment. Windows environment is + currently not supported. However, a new platform/protocol can be supported by implementing + the trait org.apache.spark.security.GroupMappingServiceProvider.
    spark.modify.acls.groupsEmpty + Comma separated list of groups that have modify access to the Spark job. This can be used if you + have a set of administrators or developers from the same team to have access to control the job. + Putting a "*" in the list means any user in any group has the access to modify the Spark job. + The user groups are obtained from the instance of the groups mapping provider specified by + spark.user.groups.mapping. Check the entry spark.user.groups.mapping + for more details. +
    spark.ui.filters None
    spark.ui.view.acls.groupsEmpty + Comma separated list of groups that have view access to the Spark web ui to view the Spark Job + details. This can be used if you have a set of administrators or developers or users who can + monitor the Spark job submitted. Putting a "*" in the list means any user in any group can view + the Spark job details on the Spark web ui. The user groups are obtained from the instance of the + groups mapping provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. +
    #### Encryption diff --git a/docs/monitoring.md b/docs/monitoring.md index 88002ebdc39cb..697962ae3a4ce 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -162,8 +162,8 @@ The history server can be configured as follows: If enabled, access control checks are made regardless of what the individual application had set for spark.ui.acls.enable when the application was run. The application owner will always have authorization to view their own application and any users specified via - spark.ui.view.acls when the application was run will also have authorization - to view that application. + spark.ui.view.acls and groups specified via spark.ui.view.acls.groups + when the application was run will also have authorization to view that application. If disabled, no access control checks are made. diff --git a/docs/security.md b/docs/security.md index 32c33d285747a..d2708a80703ec 100644 --- a/docs/security.md +++ b/docs/security.md @@ -16,10 +16,10 @@ and by using [https/SSL](http://en.wikipedia.org/wiki/HTTPS) via the `spark.ui.h ### Authentication -A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. +A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable`, `spark.ui.view.acls` and `spark.ui.view.acls.groups` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. -Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable` and `spark.modify.acls`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. -Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the config `spark.admin.acls`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. +Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable`, `spark.modify.acls` and `spark.modify.acls.groups`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. +Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the configs `spark.admin.acls` and `spark.admin.acls.groups`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. ## Event Logging diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index ee002f6223902..44181610d770d 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -464,11 +464,15 @@ object YarnSparkHadoopUtil { } } + // YARN/Hadoop acls are specified as user1,user2 group1,group2 + // Users and groups are separated by a space and hence we need to pass the acls in same format def getApplicationAclsForYarn(securityMgr: SecurityManager) : Map[ApplicationAccessType, String] = { Map[ApplicationAccessType, String] ( - ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, - ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls + ApplicationAccessType.VIEW_APP -> (securityMgr.getViewAcls + " " + + securityMgr.getViewAclsGroups), + ApplicationAccessType.MODIFY_APP -> (securityMgr.getModifyAcls + " " + + securityMgr.getModifyAclsGroups) ) } From 343c2850440279f8447a866459029b5dfc5429d0 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Wed, 4 May 2016 10:25:14 -0700 Subject: [PATCH 0060/1470] [SPARK-15022][SPARK-15023][SQL][STREAMING] Add support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `ManualClock` ## What changes were proposed in this pull request? Currently in `StreamTest`, we have a `StartStream` which will start a streaming query against trigger `ProcessTime(intervalMS = 0)` and `SystemClock`. We also need to test cases against `ProcessTime(intervalMS > 0)`, which often requires `ManualClock`. This patch: - fixes an issue of `ProcessingTimeExecutor`, where for a batch it should run `batchRunner` only once but might run multiple times under certain conditions; - adds support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `AdvanceManualClock`, by specifying them as fields for `StartStream`, and by adding an `AdvanceClock` action; - adds a test, which takes advantage of the new `StartStream` and `AdvanceManualClock`, to test against [PR#[SPARK-14942] Reduce delay between batch construction and execution ](https://github.com/apache/spark/pull/12725). ## How was this patch tested? N/A Author: Liwei Lin Closes #12797 from lw-lin/add-trigger-test-support. (cherry picked from commit e597ec6f1c8ba1f9c10de06534bda1862b0c59aa) Signed-off-by: Shixiong Zhu --- .../spark/sql/ContinuousQueryManager.scala | 7 ++-- .../execution/streaming/StreamExecution.scala | 9 ++--- .../execution/streaming/TriggerExecutor.scala | 9 +++-- .../org/apache/spark/sql/StreamTest.scala | 33 ++++++++++++++----- .../ProcessingTimeExecutorSuite.scala | 19 +++++++++-- .../sql/streaming/ContinuousQuerySuite.scala | 2 +- .../sql/streaming/FileStreamSourceSuite.scala | 8 ++--- .../spark/sql/streaming/StreamSuite.scala | 24 +++++++++++--- .../streaming/StreamingAggregationSuite.scala | 6 ++-- .../util/ContinuousQueryListenerSuite.scala | 6 ++-- 10 files changed, 89 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala index f82130cfa849d..eab557443d1d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.ContinuousQueryListener +import org.apache.spark.util.{Clock, SystemClock} /** * :: Experimental :: @@ -175,6 +176,7 @@ class ContinuousQueryManager(sparkSession: SparkSession) { df: DataFrame, sink: Sink, trigger: Trigger = ProcessingTime(0), + triggerClock: Clock = new SystemClock(), outputMode: OutputMode = Append): ContinuousQuery = { activeQueriesLock.synchronized { if (activeQueries.contains(name)) { @@ -206,8 +208,9 @@ class ContinuousQueryManager(sparkSession: SparkSession) { checkpointLocation, logicalPlan, sink, - outputMode, - trigger) + trigger, + triggerClock, + outputMode) query.start() activeQueries.put(name, query) query diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 3c5ced2af73f6..ea367b699f568 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.util.ContinuousQueryListener import org.apache.spark.sql.util.ContinuousQueryListener._ -import org.apache.spark.util.{UninterruptibleThread, Utils} +import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} /** * Manages the execution of a streaming Spark SQL query that is occurring in a separate thread. @@ -50,8 +50,9 @@ class StreamExecution( checkpointRoot: String, private[sql] val logicalPlan: LogicalPlan, val sink: Sink, - val outputMode: OutputMode, - val trigger: Trigger) + val trigger: Trigger, + private[sql] val triggerClock: Clock, + val outputMode: OutputMode) extends ContinuousQuery with Logging { /** @@ -88,7 +89,7 @@ class StreamExecution( private val uniqueSources = sources.distinct private val triggerExecutor = trigger match { - case t: ProcessingTime => ProcessingTimeExecutor(t) + case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock) } /** Defines the internal state of execution */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala index a1132d510685c..569907b369a54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala @@ -65,8 +65,13 @@ case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = s"${intervalMs} milliseconds, but spent ${realElapsedTimeMs} milliseconds") } - /** Return the next multiple of intervalMs */ + /** + * Returns the start time in milliseconds for the next batch interval, given the current time. + * Note that a batch interval is inclusive with respect to its start time, and thus calling + * `nextBatchTime` with the result of a previous call should return the next interval. (i.e. given + * an interval of `100 ms`, `nextBatchTime(nextBatchTime(0)) = 200` rather than `0`). + */ def nextBatchTime(now: Long): Long = { - (now - 1) / intervalMs * intervalMs + intervalMs + now / intervalMs * intervalMs + intervalMs } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala index dff6acc94b3f0..6fb1aca769e36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} /** * A framework for implementing tests for streaming queries and sources. @@ -138,11 +138,17 @@ trait StreamTest extends QueryTest with Timeouts { private def operatorName = if (lastOnly) "CheckLastBatch" else "CheckAnswer" } - /** Stops the stream. It must currently be running. */ + /** Stops the stream. It must currently be running. */ case object StopStream extends StreamAction with StreamMustBeRunning - /** Starts the stream, resuming if data has already been processed. It must not be running. */ - case object StartStream extends StreamAction + /** Starts the stream, resuming if data has already been processed. It must not be running. */ + case class StartStream( + trigger: Trigger = ProcessingTime(0), + triggerClock: Clock = new SystemClock) + extends StreamAction + + /** Advance the trigger clock's time manually. */ + case class AdvanceManualClock(timeToAdd: Long) extends StreamAction /** Signals that a failure is expected and should not kill the test. */ case class ExpectFailure[T <: Throwable : ClassTag]() extends StreamAction { @@ -199,8 +205,8 @@ trait StreamTest extends QueryTest with Timeouts { // If the test doesn't manually start the stream, we do it automatically at the beginning. val startedManually = - actions.takeWhile(!_.isInstanceOf[StreamMustBeRunning]).contains(StartStream) - val startedTest = if (startedManually) actions else StartStream +: actions + actions.takeWhile(!_.isInstanceOf[StreamMustBeRunning]).exists(_.isInstanceOf[StartStream]) + val startedTest = if (startedManually) actions else StartStream() +: actions def testActions = actions.zipWithIndex.map { case (a, i) => @@ -280,7 +286,7 @@ trait StreamTest extends QueryTest with Timeouts { try { startedTest.foreach { action => action match { - case StartStream => + case StartStream(trigger, triggerClock) => verify(currentStream == null, "stream already running") lastStream = currentStream currentStream = @@ -291,6 +297,8 @@ trait StreamTest extends QueryTest with Timeouts { metadataRoot, stream, sink, + trigger, + triggerClock, outputMode = outputMode) .asInstanceOf[StreamExecution] currentStream.microBatchThread.setUncaughtExceptionHandler( @@ -301,6 +309,13 @@ trait StreamTest extends QueryTest with Timeouts { } }) + case AdvanceManualClock(timeToAdd) => + verify(currentStream != null, + "can not advance manual clock when a stream is not running") + verify(currentStream.triggerClock.isInstanceOf[ManualClock], + s"can not advance clock of type ${currentStream.triggerClock.getClass}") + currentStream.triggerClock.asInstanceOf[ManualClock].advance(timeToAdd) + case StopStream => verify(currentStream != null, "can not stop a stream that is not running") try failAfter(streamingTimeout) { @@ -470,7 +485,7 @@ trait StreamTest extends QueryTest with Timeouts { addRandomData() case _ => // StartStream - actions += StartStream + actions += StartStream() running = true } } else { @@ -488,7 +503,7 @@ trait StreamTest extends QueryTest with Timeouts { } } } - if(!running) { actions += StartStream } + if(!running) { actions += StartStream() } addCheck() testStream(ds)(actions: _*) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala index dd5f92248bf5c..7f99d303ba08a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -21,19 +21,34 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.ProcessingTime -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{Clock, ManualClock, SystemClock} class ProcessingTimeExecutorSuite extends SparkFunSuite { test("nextBatchTime") { val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(100)) + assert(processingTimeExecutor.nextBatchTime(0) === 100) assert(processingTimeExecutor.nextBatchTime(1) === 100) assert(processingTimeExecutor.nextBatchTime(99) === 100) - assert(processingTimeExecutor.nextBatchTime(100) === 100) + assert(processingTimeExecutor.nextBatchTime(100) === 200) assert(processingTimeExecutor.nextBatchTime(101) === 200) assert(processingTimeExecutor.nextBatchTime(150) === 200) } + test("calling nextBatchTime with the result of a previous call should return the next interval") { + val intervalMS = 100 + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMS)) + + val ITERATION = 10 + var nextBatchTime: Long = 0 + for (it <- 1 to ITERATION) { + nextBatchTime = processingTimeExecutor.nextBatchTime(nextBatchTime) + } + + // nextBatchTime should be 1000 + assert(nextBatchTime === intervalMS * ITERATION) + } + private def testBatchTermination(intervalMs: Long): Unit = { var batchCounts = 0 val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMs)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala index 3be0ea481dc53..f469cde6bef80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala @@ -45,7 +45,7 @@ class ContinuousQuerySuite extends StreamTest with SharedSQLContext { TestAwaitTermination(ExpectNotBlocked), TestAwaitTermination(ExpectNotBlocked, timeoutMs = 2000, expectedReturnValue = true), TestAwaitTermination(ExpectNotBlocked, timeoutMs = 10, expectedReturnValue = true), - StartStream, + StartStream(), AssertOnQuery(_.isActive === true), AddData(inputData, 0), ExpectFailure[SparkException], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 6b1ecd08c13c3..bc5c0c1f69331 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -268,7 +268,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { CheckAnswer("keep2", "keep3"), StopStream, AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") @@ -292,7 +292,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { "{'value': 'drop4'}\n{'value': 'keep5'}\n{'value': 'keep6'}", src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddTextFileData( "{'value': 'drop7'}\n{'value': 'keep8'}\n{'value': 'keep9'}", @@ -385,7 +385,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { CheckAnswer("keep2", "keep3"), StopStream, AddParquetFileData(Seq("drop4", "keep5", "keep6"), src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddParquetFileData(Seq("drop7", "keep8", "keep9"), src, tmp), CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") @@ -449,7 +449,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { CheckAnswer("keep2", "keep3"), StopStream, AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), - StartStream, + StartStream(), CheckAnswer("keep2", "keep3", "keep5", "keep6"), AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 6f3149dbc5033..bcd3cba55a55d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.streaming import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.functions._ import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.util.ManualClock class StreamSuite extends StreamTest with SharedSQLContext { @@ -34,11 +34,11 @@ class StreamSuite extends StreamTest with SharedSQLContext { testStream(mapped)( AddData(inputData, 1, 2, 3), - StartStream, + StartStream(), CheckAnswer(2, 3, 4), StopStream, AddData(inputData, 4, 5, 6), - StartStream, + StartStream(), CheckAnswer(2, 3, 4, 5, 6, 7)) } @@ -70,7 +70,7 @@ class StreamSuite extends StreamTest with SharedSQLContext { CheckAnswer(1, 2, 3, 4, 5, 6), StopStream, AddData(inputData1, 7), - StartStream, + StartStream(), AddData(inputData2, 8), CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8)) } @@ -136,6 +136,22 @@ class StreamSuite extends StreamTest with SharedSQLContext { testStream(ds)() } } + + // This would fail for now -- error is "Timed out waiting for stream" + // Root cause is that data generated in batch 0 may not get processed in batch 1 + // Let's enable this after SPARK-14942: Reduce delay between batch construction and execution + ignore("minimize delay between batch construction and execution") { + val inputData = MemoryStream[Int] + testStream(inputData.toDS())( + StartStream(ProcessingTime("10 seconds"), new ManualClock), + /* -- batch 0 ----------------------- */ + AddData(inputData, 1), + AddData(inputData, 2), + AddData(inputData, 3), + AdvanceManualClock(10 * 1000), // 10 seconds + /* -- batch 1 ----------------------- */ + CheckAnswer(1, 2, 3)) + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index fa3b122f6d2da..bdf40f5cd45d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -50,7 +50,7 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext { AddData(inputData, 3, 2), CheckLastBatch((3, 2), (2, 1)), StopStream, - StartStream, + StartStream(), AddData(inputData, 3, 2, 1), CheckLastBatch((3, 3), (2, 2), (1, 1)), // By default we run in new tuple mode. @@ -113,10 +113,10 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext { .as[(Int, Long)] testStream(aggregated)( - StartStream, + StartStream(), AddData(inputData, 1, 2, 3, 4), ExpectFailure[SparkException](), - StartStream, + StartStream(), CheckLastBatch((1, 1), (2, 1), (3, 1), (4, 1)) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala index 2596231a12446..54acd4db3c71d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala @@ -48,7 +48,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with val input = MemoryStream[Int] withListenerAdded(listener) { testStream(input.toDS)( - StartStream, + StartStream(), Assert("Incorrect query status in onQueryStarted") { val status = listener.startStatus assert(status != null) @@ -102,7 +102,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with def isListenerActive(listener: QueryStatusCollector): Boolean = { listener.reset() testStream(MemoryStream[Int].toDS)( - StartStream, + StartStream(), StopStream ) listener.startStatus != null @@ -133,7 +133,7 @@ class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with listener.reset() require(listener.startStatus === null) testStream(MemoryStream[Int].toDS)( - StartStream, + StartStream(), Assert(listener.startStatus !== null, "onQueryStarted not called before query returned"), StopStream, Assert { listener.checkAsyncErrors() } From b3c4912173a5d03f104c748b2bb7ea6b148b43c9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 May 2016 10:38:27 -0700 Subject: [PATCH 0061/1470] [SPARK-15109][SQL] Accept Dataset[_] in joins ## What changes were proposed in this pull request? This patch changes the join API in Dataset so they can accept any Dataset, rather than just DataFrames. ## How was this patch tested? N/A. Author: Reynold Xin Closes #12886 from rxin/SPARK-15109. (cherry picked from commit d864c55cf8c92466336e796d0c98d83230e330af) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/sql/Dataset.scala | 12 ++++++------ .../main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 31dd64e909bb8..c77b13832c8b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -564,7 +564,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame): DataFrame = withPlan { + def join(right: Dataset[_]): DataFrame = withPlan { Join(logicalPlan, right.logicalPlan, joinType = Inner, None) } @@ -589,7 +589,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, usingColumn: String): DataFrame = { + def join(right: Dataset[_], usingColumn: String): DataFrame = { join(right, Seq(usingColumn)) } @@ -614,7 +614,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, usingColumns: Seq[String]): DataFrame = { + def join(right: Dataset[_], usingColumns: Seq[String]): DataFrame = { join(right, usingColumns, "inner") } @@ -635,7 +635,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, usingColumns: Seq[String], joinType: String): DataFrame = { + def join(right: Dataset[_], usingColumns: Seq[String], joinType: String): DataFrame = { // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right // by creating a new instance for one of the branch. val joined = sparkSession.executePlan( @@ -663,7 +663,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, joinExprs: Column): DataFrame = join(right, joinExprs, "inner") + def join(right: Dataset[_], joinExprs: Column): DataFrame = join(right, joinExprs, "inner") /** * Join with another [[DataFrame]], using the given join expression. The following performs @@ -686,7 +686,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { + def join(right: Dataset[_], joinExprs: Column, joinType: String): DataFrame = { // Note that in this function, we introduce a hack in the case of self-join to automatically // resolve ambiguous join conditions into ones that might make sense [SPARK-6231]. // Consider this case: df.join(df, df("key") === df("key")) 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 fe63c80815a99..3e295c20b6d9f 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 @@ -931,8 +931,8 @@ object functions { * @group normal_funcs * @since 1.5.0 */ - def broadcast(df: DataFrame): DataFrame = { - Dataset.ofRows(df.sparkSession, BroadcastHint(df.logicalPlan)) + def broadcast[T](df: Dataset[T]): Dataset[T] = { + Dataset[T](df.sparkSession, BroadcastHint(df.logicalPlan))(df.unresolvedTEncoder) } /** From 21e368de4d17a52a380959bb709caca091a3fac7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 4 May 2016 10:54:51 -0700 Subject: [PATCH 0062/1470] [SPARK-14951] [SQL] Support subexpression elimination in TungstenAggregate ## What changes were proposed in this pull request? We can support subexpression elimination in TungstenAggregate by using current `EquivalentExpressions` which is already used in subexpression elimination for expression codegen. However, in wholestage codegen, we can't wrap the common expression's codes in functions as before, we simply generate the code snippets for common expressions. These code snippets are inserted before the common expressions are actually used in generated java codes. For multiple `TypedAggregateExpression` used in aggregation operator, since their input type should be the same. So their `inputDeserializer` will be the same too. This patch can also reduce redundant input deserialization. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #12729 from viirya/subexpr-elimination-tungstenaggregate. (cherry picked from commit b85d21fb9dc3d498d9a10e065d254abde797efb6) Signed-off-by: Davies Liu --- .../expressions/EquivalentExpressions.scala | 7 +- .../expressions/codegen/CodeGenerator.scala | 74 ++++++++++++++++++- .../aggregate/TungstenAggregate.scala | 31 ++++++-- .../aggregate/TypedAggregateExpression.scala | 38 ++-------- 4 files changed, 109 insertions(+), 41 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala index d0ad7a05a0c37..b8e2b67b2fe9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala @@ -68,7 +68,10 @@ class EquivalentExpressions { * is found. That is, if `expr` has already been added, its children are not added. * If ignoreLeaf is true, leaf nodes are ignored. */ - def addExprTree(root: Expression, ignoreLeaf: Boolean = true): Unit = { + def addExprTree( + root: Expression, + ignoreLeaf: Boolean = true, + skipReferenceToExpressions: Boolean = true): Unit = { val skip = root.isInstanceOf[LeafExpression] && ignoreLeaf // There are some special expressions that we should not recurse into children. // 1. CodegenFallback: it's children will not be used to generate code (call eval() instead) @@ -77,7 +80,7 @@ class EquivalentExpressions { // TODO: some expressions implements `CodegenFallback` but can still do codegen, // e.g. `CaseWhen`, we should support them. case _: CodegenFallback => false - case _: ReferenceToExpressions => false + case _: ReferenceToExpressions if skipReferenceToExpressions => false case _ => true } if (!skip && !addExpr(root) && shouldRecurse) { 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 e4fa429b37546..67f671926561a 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 @@ -46,6 +46,25 @@ import org.apache.spark.util.Utils */ case class ExprCode(var code: String, var isNull: String, var value: String) +/** + * State used for subexpression elimination. + * + * @param isNull A term that holds a boolean value representing whether the expression evaluated + * to null. + * @param value A term for a value of a common sub-expression. Not valid if `isNull` + * is set to `true`. + */ +case class SubExprEliminationState(isNull: String, value: String) + +/** + * Codes and common subexpressions mapping used for subexpression elimination. + * + * @param codes Strings representing the codes that evaluate common subexpressions. + * @param states Foreach expression that is participating in subexpression elimination, + * the state to use. + */ +case class SubExprCodes(codes: Seq[String], states: Map[Expression, SubExprEliminationState]) + /** * A context for codegen, tracking a list of objects that could be passed into generated Java * function. @@ -148,9 +167,6 @@ class CodegenContext { */ val equivalentExpressions: EquivalentExpressions = new EquivalentExpressions - // State used for subexpression elimination. - case class SubExprEliminationState(isNull: String, value: String) - // Foreach expression that is participating in subexpression elimination, the state to use. val subExprEliminationExprs = mutable.HashMap.empty[Expression, SubExprEliminationState] @@ -571,6 +587,58 @@ class CodegenContext { } } + /** + * Perform a function which generates a sequence of ExprCodes with a given mapping between + * expressions and common expressions, instead of using the mapping in current context. + */ + def withSubExprEliminationExprs( + newSubExprEliminationExprs: Map[Expression, SubExprEliminationState])( + f: => Seq[ExprCode]): Seq[ExprCode] = { + val oldsubExprEliminationExprs = subExprEliminationExprs + subExprEliminationExprs.clear + newSubExprEliminationExprs.foreach(subExprEliminationExprs += _) + + val genCodes = f + + // Restore previous subExprEliminationExprs + subExprEliminationExprs.clear + oldsubExprEliminationExprs.foreach(subExprEliminationExprs += _) + genCodes + } + + /** + * Checks and sets up the state and codegen for subexpression elimination. This finds the + * common subexpressions, generates the code snippets that evaluate those expressions and + * populates the mapping of common subexpressions to the generated code snippets. The generated + * code snippets will be returned and should be inserted into generated codes before these + * common subexpressions actually are used first time. + */ + def subexpressionEliminationForWholeStageCodegen(expressions: Seq[Expression]): SubExprCodes = { + // Create a clear EquivalentExpressions and SubExprEliminationState mapping + val equivalentExpressions: EquivalentExpressions = new EquivalentExpressions + val subExprEliminationExprs = mutable.HashMap.empty[Expression, SubExprEliminationState] + + // Add each expression tree and compute the common subexpressions. + expressions.foreach(equivalentExpressions.addExprTree(_, true, false)) + + // Get all the expressions that appear at least twice and set up the state for subexpression + // elimination. + val commonExprs = equivalentExpressions.getAllEquivalentExprs.filter(_.size > 1) + val codes = commonExprs.map { e => + val expr = e.head + val fnName = freshName("evalExpr") + val isNull = s"${fnName}IsNull" + val value = s"${fnName}Value" + + // Generate the code for this expression tree. + val code = expr.genCode(this) + val state = SubExprEliminationState(code.isNull, code.value) + e.foreach(subExprEliminationExprs.put(_, state)) + code.code.trim + } + SubExprCodes(codes, subExprEliminationExprs.toMap) + } + /** * Checks and sets up the state and codegen for subexpression elimination. This finds the * common subexpressions, generates the functions that evaluate those expressions and populates diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index d0ba37ee1338b..d2dc80a7e42eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -244,8 +244,12 @@ case class TungstenAggregate( } } ctx.currentVars = bufVars ++ input - // TODO: support subexpression elimination - val aggVals = updateExpr.map(BindReferences.bindReference(_, inputAttrs).genCode(ctx)) + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttrs)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val aggVals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } // aggregate buffer should be updated atomic val updates = aggVals.zipWithIndex.map { case (ev, i) => s""" @@ -255,6 +259,9 @@ case class TungstenAggregate( } s""" | // do aggregate + | // common sub-expressions + | $effectiveCodes + | // evaluate aggregate function | ${evaluateVariables(aggVals)} | // update aggregation buffer | ${updates.mkString("\n").trim} @@ -650,8 +657,12 @@ case class TungstenAggregate( val updateRowInVectorizedHashMap: Option[String] = { if (isVectorizedHashMapEnabled) { ctx.INPUT_ROW = vectorizedRowBuffer - val vectorizedRowEvals = - updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val vectorizedRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } val updateVectorizedRow = vectorizedRowEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(vectorizedRowBuffer, dt, i, ev, updateExpr(i).nullable, @@ -659,6 +670,8 @@ case class TungstenAggregate( } Option( s""" + |// common sub-expressions + |$effectiveCodes |// evaluate aggregate function |${evaluateVariables(vectorizedRowEvals)} |// update vectorized row @@ -701,13 +714,19 @@ case class TungstenAggregate( val updateRowInUnsafeRowMap: String = { ctx.INPUT_ROW = unsafeRowBuffer - val unsafeRowBufferEvals = - updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val unsafeRowBufferEvals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } val updateUnsafeRowBuffer = unsafeRowBufferEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(unsafeRowBuffer, dt, i, ev, updateExpr(i).nullable) } s""" + |// common sub-expressions + |$effectiveCodes |// evaluate aggregate function |${evaluateVariables(unsafeRowBufferEvals)} |// update unsafe row buffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 535e64cb34442..edca816cb1d16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -31,31 +31,9 @@ object TypedAggregateExpression { def apply[BUF : Encoder, OUT : Encoder]( aggregator: Aggregator[_, BUF, OUT]): TypedAggregateExpression = { val bufferEncoder = encoderFor[BUF] - // We will insert the deserializer and function call expression at the bottom of each serializer - // expression while executing `TypedAggregateExpression`, which means multiply serializer - // expressions will all evaluate the same sub-expression at bottom. To avoid the re-evaluating, - // here we always use one single serializer expression to serialize the buffer object into a - // single-field row, no matter whether the encoder is flat or not. We also need to update the - // deserializer to read in all fields from that single-field row. - // TODO: remove this trick after we have better integration of subexpression elimination and - // whole stage codegen. - val bufferSerializer = if (bufferEncoder.flat) { - bufferEncoder.namedExpressions.head - } else { - Alias(CreateStruct(bufferEncoder.serializer), "buffer")() - } - - val bufferDeserializer = if (bufferEncoder.flat) { - bufferEncoder.deserializer transformUp { - case b: BoundReference => bufferSerializer.toAttribute - } - } else { - bufferEncoder.deserializer transformUp { - case UnresolvedAttribute(nameParts) => - assert(nameParts.length == 1) - UnresolvedExtractValue(bufferSerializer.toAttribute, Literal(nameParts.head)) - case BoundReference(ordinal, dt, _) => GetStructField(bufferSerializer.toAttribute, ordinal) - } + val bufferSerializer = bufferEncoder.namedExpressions + val bufferDeserializer = bufferEncoder.deserializer.transform { + case b: BoundReference => bufferSerializer(b.ordinal).toAttribute } val outputEncoder = encoderFor[OUT] @@ -82,7 +60,7 @@ object TypedAggregateExpression { case class TypedAggregateExpression( aggregator: Aggregator[Any, Any, Any], inputDeserializer: Option[Expression], - bufferSerializer: NamedExpression, + bufferSerializer: Seq[NamedExpression], bufferDeserializer: Expression, outputSerializer: Seq[Expression], outputExternalType: DataType, @@ -106,11 +84,11 @@ case class TypedAggregateExpression( private def bufferExternalType = bufferDeserializer.dataType override lazy val aggBufferAttributes: Seq[AttributeReference] = - bufferSerializer.toAttribute.asInstanceOf[AttributeReference] :: Nil + bufferSerializer.map(_.toAttribute.asInstanceOf[AttributeReference]) override lazy val initialValues: Seq[Expression] = { val zero = Literal.fromObject(aggregator.zero, bufferExternalType) - ReferenceToExpressions(bufferSerializer, zero :: Nil) :: Nil + bufferSerializer.map(ReferenceToExpressions(_, zero :: Nil)) } override lazy val updateExpressions: Seq[Expression] = { @@ -120,7 +98,7 @@ case class TypedAggregateExpression( bufferExternalType, bufferDeserializer :: inputDeserializer.get :: Nil) - ReferenceToExpressions(bufferSerializer, reduced :: Nil) :: Nil + bufferSerializer.map(ReferenceToExpressions(_, reduced :: Nil)) } override lazy val mergeExpressions: Seq[Expression] = { @@ -136,7 +114,7 @@ case class TypedAggregateExpression( bufferExternalType, leftBuffer :: rightBuffer :: Nil) - ReferenceToExpressions(bufferSerializer, merged :: Nil) :: Nil + bufferSerializer.map(ReferenceToExpressions(_, merged :: Nil)) } override lazy val evaluateExpression: Expression = { From 54d90bd3eb18253b3f3d67ae8a52471314a061ad Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Wed, 4 May 2016 10:59:36 -0700 Subject: [PATCH 0063/1470] [MINOR] Add python3 compatibility in python examples ## What changes were proposed in this pull request? Add python3 compatibility in python examples ## How was this patch tested? manual tests Author: Zheng RuiFeng Closes #12868 from zhengruifeng/fix_gmm_py. (cherry picked from commit 4530250f5a51a77f9d0b91f036f8e44e0f943a32) Signed-off-by: Davies Liu --- examples/src/main/python/ml/als_example.py | 4 ++++ examples/src/main/python/mllib/gaussian_mixture_model.py | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index 922173308c6aa..0c9ac583b2dab 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -17,6 +17,10 @@ from __future__ import print_function +import sys +if sys.version >= '3': + long = int + from pyspark import SparkContext from pyspark.sql import SQLContext diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index 69e836fc1d06a..6b46e27ddaaa8 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -20,6 +20,10 @@ """ from __future__ import print_function +import sys +if sys.version >= '3': + long = int + import random import argparse import numpy as np From c5961543206cbccc7a9b3103110f1a84b4ccddce Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 May 2016 11:00:01 -0700 Subject: [PATCH 0064/1470] [SPARK-15115][SQL] Reorganize whole stage codegen benchmark suites ## What changes were proposed in this pull request? We currently have a single suite that is very large, making it difficult to maintain and play with specific primitives. This patch reorganizes the file by creating multiple benchmark suites in a single package. Most of the changes are straightforward move of code. On top of the code moving, I did: 1. Use SparkSession instead of SQLContext. 2. Turned most benchmark scenarios into a their own test cases, rather than having multiple scenarios in a single test case, which takes forever to run. ## How was this patch tested? This is a test only change. Author: Reynold Xin Closes #12891 from rxin/SPARK-15115. (cherry picked from commit 6274a520fa743b7d079fde4a3033da5c3a2532a1) Signed-off-by: Reynold Xin --- .../sort/RecordPointerAndKeyPrefix.java | 2 +- .../unsafe/sort/UnsafeSortDataFormat.java | 3 +- .../unsafe/sort/RadixSortSuite.scala | 78 ---- .../AggregateBenchmark.scala} | 392 +++--------------- .../execution/benchmark/BenchmarkBase.scala | 54 +++ .../execution/benchmark/JoinBenchmark.scala | 229 ++++++++++ .../execution/benchmark/MiscBenchmark.scala | 135 ++++++ .../execution/benchmark/SortBenchmark.scala | 132 ++++++ 8 files changed, 603 insertions(+), 422 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/execution/{BenchmarkWholeStageCodegen.scala => benchmark/AggregateBenchmark.scala} (52%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java index de92b8db47131..e9571aa8bb052 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java @@ -17,7 +17,7 @@ package org.apache.spark.util.collection.unsafe.sort; -final class RecordPointerAndKeyPrefix { +public final class RecordPointerAndKeyPrefix { /** * A pointer to a record; see {@link org.apache.spark.memory.TaskMemoryManager} for a * description of how these addresses are encoded. diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java index 12fb62fb77f0f..d19b71fbc1bcb 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java @@ -29,7 +29,8 @@ * Within each long[] buffer, position {@code 2 * i} holds a pointer pointer to the record at * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. */ -final class UnsafeSortDataFormat extends SortDataFormat { +public final class UnsafeSortDataFormat + extends SortDataFormat { public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat(); diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index b03df1a94d84c..def0752b46f6a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.unsafe.array.LongArray import org.apache.spark.unsafe.memory.MemoryBlock -import org.apache.spark.util.Benchmark import org.apache.spark.util.collection.Sorter import org.apache.spark.util.random.XORShiftRandom @@ -184,81 +183,4 @@ class RadixSortSuite extends SparkFunSuite with Logging { assert(res1.view == res2.view) } } - - ignore("microbenchmarks") { - val size = 25000000 - val rand = new XORShiftRandom(123) - val benchmark = new Benchmark("radix sort " + size, size) - benchmark.addTimerCase("reference TimSort key prefix array") { timer => - val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) - timer.stopTiming() - } - benchmark.addTimerCase("reference Arrays.sort") { timer => - val ref = Array.tabulate[Long](size) { i => rand.nextLong } - timer.startTiming() - Arrays.sort(ref) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort one byte") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong & 0xff - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort two bytes") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong & 0xffff - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort eight bytes") { timer => - val array = new Array[Long](size * 2) - var i = 0 - while (i < size) { - array(i) = rand.nextLong - i += 1 - } - val buf = new LongArray(MemoryBlock.fromLongArray(array)) - timer.startTiming() - RadixSort.sort(buf, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.addTimerCase("radix sort key prefix array") { timer => - val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) - timer.startTiming() - RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) - timer.stopTiming() - } - benchmark.run() - - /** - Running benchmark: radix sort 25000000 - Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 3.13.0-44-generic - Intel(R) Core(TM) i7-4600U CPU @ 2.10GHz - - radix sort 25000000: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - reference TimSort key prefix array 15546 / 15859 1.6 621.9 1.0X - reference Arrays.sort 2416 / 2446 10.3 96.6 6.4X - radix sort one byte 133 / 137 188.4 5.3 117.2X - radix sort two bytes 255 / 258 98.2 10.2 61.1X - radix sort eight bytes 991 / 997 25.2 39.6 15.7X - radix sort key prefix array 1540 / 1563 16.2 61.6 10.1X - */ - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala similarity index 52% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 7ca4b75f480ba..b31338e82768d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BenchmarkWholeStageCodegen.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -15,59 +15,35 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.execution.benchmark import java.util.HashMap -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.SparkConf import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} -import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{IntegerType, LongType, StructType} +import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 import org.apache.spark.unsafe.map.BytesToBytesMap import org.apache.spark.util.Benchmark /** - * Benchmark to measure whole stage codegen performance. + * Benchmark to measure performance for aggregate primitives. * To run this: - * build/sbt "sql/test-only *BenchmarkWholeStageCodegen" + * build/sbt "sql/test-only *benchmark.AggregateBenchmark" * * Benchmarks in this file are skipped in normal builds. */ -class BenchmarkWholeStageCodegen extends SparkFunSuite { - lazy val conf = new SparkConf().setMaster("local[1]").setAppName("benchmark") - .set("spark.sql.shuffle.partitions", "1") - .set("spark.sql.autoBroadcastJoinThreshold", "1") - lazy val sc = SparkContext.getOrCreate(conf) - lazy val sqlContext = SQLContext.getOrCreate(sc) - - /** Runs function `f` with whole stage codegen on and off. */ - def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { - val benchmark = new Benchmark(name, cardinality) - - benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") - f - } - - benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - f - } - - benchmark.run() - } +class AggregateBenchmark extends BenchmarkBase { ignore("aggregate without grouping") { val N = 500L << 22 val benchmark = new Benchmark("agg without grouping", N) runBenchmark("agg w/o group", N) { - sqlContext.range(N).selectExpr("sum(id)").collect() + sparkSession.range(N).selectExpr("sum(id)").collect() } /* agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative @@ -77,79 +53,19 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { */ } - ignore("filter & aggregate without group") { - val N = 500L << 22 - runBenchmark("range/filter/sum", N) { - sqlContext.range(N).filter("(id & 1) = 1").groupBy().sum().collect() - } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X - range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X - */ - } - - ignore("range/limit/sum") { - val N = 500L << 20 - runBenchmark("range/limit/sum", N) { - sqlContext.range(N).limit(1000000).groupBy().sum().collect() - } - /* - Westmere E56xx/L56xx/X56xx (Nehalem-C) - range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - range/limit/sum codegen=false 609 / 672 861.6 1.2 1.0X - range/limit/sum codegen=true 561 / 621 935.3 1.1 1.1X - */ - } - - ignore("sample") { - val N = 500 << 18 - runBenchmark("sample with replacement", N) { - sqlContext.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() - } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X - sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X - */ - - runBenchmark("sample without replacement", N) { - sqlContext.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() - } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X - sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X - */ - } - ignore("stat functions") { val N = 100L << 20 runBenchmark("stddev", N) { - sqlContext.range(N).groupBy().agg("id" -> "stddev").collect() + sparkSession.range(N).groupBy().agg("id" -> "stddev").collect() } runBenchmark("kurtosis", N) { - sqlContext.range(N).groupBy().agg("id" -> "kurtosis").collect() + sparkSession.range(N).groupBy().agg("id" -> "kurtosis").collect() } - - /** - Using ImperativeAggregate (as implemented in Spark 1.6): + /* + Using ImperativeAggregate (as implemented in Spark 1.6): Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz stddev: Avg Time(ms) Avg Rate(M/s) Relative Rate @@ -172,29 +88,31 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { ------------------------------------------------------------------------------------------- kurtosis codegen=false 14847 / 15084 7.0 142.9 1.0X kurtosis codegen=true 1652 / 2124 63.0 15.9 9.0X - */ + */ } ignore("aggregate with linear keys") { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) - def f(): Unit = sqlContext.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + def f(): Unit = { + sparkSession.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + } benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "3") f() } @@ -216,24 +134,24 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) - sqlContext.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") - def f(): Unit = sqlContext.sql("select k, k, sum(id) from test group by k, k").collect() + def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 0) f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", 3) f() } @@ -255,23 +173,23 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 20 val benchmark = new Benchmark("Aggregate w string key", N) - def f(): Unit = sqlContext.range(N).selectExpr("id", "cast(id & 1023 as string) as k") + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 1023 as string) as k") .groupBy("k").count().collect() benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "3") f() } @@ -292,23 +210,23 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 20 val benchmark = new Benchmark("Aggregate w decimal key", N) - def f(): Unit = sqlContext.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") .groupBy("k").count().collect() benchmark.addCase(s"codegen = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "3") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "3") f() } @@ -329,7 +247,7 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { val N = 20 << 20 val benchmark = new Benchmark("Aggregate w multiple keys", N) - def f(): Unit = sqlContext.range(N) + def f(): Unit = sparkSession.range(N) .selectExpr( "id", "(id & 1023) as k1", @@ -343,19 +261,19 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { .collect() benchmark.addCase(s"codegen = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "0") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0") f() } benchmark.addCase(s"codegen = T hashmap = T") { iter => - sqlContext.setConf("spark.sql.codegen.wholeStage", "true") - sqlContext.setConf("spark.sql.codegen.aggregate.map.columns.max", "10") + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "10") f() } @@ -372,187 +290,22 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { */ } - ignore("broadcast hash join") { - val N = 20 << 20 - val M = 1 << 16 - val dim = broadcast(sqlContext.range(M).selectExpr("id as k", "cast(id as string) as v")) - - runBenchmark("Join w long", N) { - sqlContext.range(N).join(dim, (col("id") % M) === col("k")).count() - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w long codegen=false 3002 / 3262 7.0 143.2 1.0X - Join w long codegen=true 321 / 371 65.3 15.3 9.3X - */ - - runBenchmark("Join w long duplicated", N) { - val dim = broadcast(sqlContext.range(M).selectExpr("cast(id/10 as long) as k")) - sqlContext.range(N).join(dim, (col("id") % M) === col("k")).count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w long duplicated codegen=false 3446 / 3478 6.1 164.3 1.0X - Join w long duplicated codegen=true 322 / 351 65.2 15.3 10.7X - */ - - val dim2 = broadcast(sqlContext.range(M) - .selectExpr("cast(id as int) as k1", "cast(id as int) as k2", "cast(id as string) as v")) - - runBenchmark("Join w 2 ints", N) { - sqlContext.range(N).join(dim2, - (col("id") % M).cast(IntegerType) === col("k1") - && (col("id") % M).cast(IntegerType) === col("k2")).count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w 2 ints codegen=false 4426 / 4501 4.7 211.1 1.0X - Join w 2 ints codegen=true 791 / 818 26.5 37.7 5.6X - */ - - val dim3 = broadcast(sqlContext.range(M) - .selectExpr("id as k1", "id as k2", "cast(id as string) as v")) - - runBenchmark("Join w 2 longs", N) { - sqlContext.range(N).join(dim3, - (col("id") % M) === col("k1") && (col("id") % M) === col("k2")) - .count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w 2 longs codegen=false 5905 / 6123 3.6 281.6 1.0X - Join w 2 longs codegen=true 2230 / 2529 9.4 106.3 2.6X - */ - - val dim4 = broadcast(sqlContext.range(M) - .selectExpr("cast(id/10 as long) as k1", "cast(id/10 as long) as k2")) - - runBenchmark("Join w 2 longs duplicated", N) { - sqlContext.range(N).join(dim4, - (col("id") bitwiseAND M) === col("k1") && (col("id") bitwiseAND M) === col("k2")) - .count() - } - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w 2 longs duplicated codegen=false 6420 / 6587 3.3 306.1 1.0X - Join w 2 longs duplicated codegen=true 2080 / 2139 10.1 99.2 3.1X - */ - - runBenchmark("outer join w long", N) { - sqlContext.range(N).join(dim, (col("id") % M) === col("k"), "left").count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - outer join w long codegen=false 3055 / 3189 6.9 145.7 1.0X - outer join w long codegen=true 261 / 276 80.5 12.4 11.7X - */ - - runBenchmark("semi join w long", N) { - sqlContext.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi").count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - semi join w long codegen=false 1912 / 1990 11.0 91.2 1.0X - semi join w long codegen=true 237 / 244 88.3 11.3 8.1X - */ - } - - ignore("sort merge join") { - val N = 2 << 20 - runBenchmark("merge join", N) { - val df1 = sqlContext.range(N).selectExpr(s"id * 2 as k1") - val df2 = sqlContext.range(N).selectExpr(s"id * 3 as k2") - df1.join(df2, col("k1") === col("k2")).count() - } - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - merge join codegen=false 1588 / 1880 1.3 757.1 1.0X - merge join codegen=true 1477 / 1531 1.4 704.2 1.1X - */ - - runBenchmark("sort merge join", N) { - val df1 = sqlContext.range(N) - .selectExpr(s"(id * 15485863) % ${N*10} as k1") - val df2 = sqlContext.range(N) - .selectExpr(s"(id * 15485867) % ${N*10} as k2") - df1.join(df2, col("k1") === col("k2")).count() - } - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - sort merge join codegen=false 3626 / 3667 0.6 1728.9 1.0X - sort merge join codegen=true 3405 / 3438 0.6 1623.8 1.1X - */ - } - - ignore("shuffle hash join") { - val N = 4 << 20 - sqlContext.setConf("spark.sql.shuffle.partitions", "2") - sqlContext.setConf("spark.sql.autoBroadcastJoinThreshold", "10000000") - sqlContext.setConf("spark.sql.join.preferSortMergeJoin", "false") - runBenchmark("shuffle hash join", N) { - val df1 = sqlContext.range(N).selectExpr(s"id as k1") - val df2 = sqlContext.range(N / 5).selectExpr(s"id * 3 as k2") - df1.join(df2, col("k1") === col("k2")).count() - } - - /** - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - shuffle hash join codegen=false 1101 / 1391 3.8 262.6 1.0X - shuffle hash join codegen=true 528 / 578 7.9 125.8 2.1X - */ - } ignore("cube") { val N = 5 << 20 runBenchmark("cube", N) { - sqlContext.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") + sparkSession.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") .cube("k1", "k2").sum("id").collect() } /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------- cube codegen=false 3188 / 3392 1.6 608.2 1.0X cube codegen=true 1239 / 1394 4.2 236.3 2.6X - */ + */ } ignore("hash and BytesToBytesMap") { @@ -802,7 +555,7 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { } } - /** + /* Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------- @@ -822,49 +575,4 @@ class BenchmarkWholeStageCodegen extends SparkFunSuite { benchmark.run() } - ignore("collect") { - val N = 1 << 20 - - val benchmark = new Benchmark("collect", N) - benchmark.addCase("collect 1 million") { iter => - sqlContext.range(N).collect() - } - benchmark.addCase("collect 2 millions") { iter => - sqlContext.range(N * 2).collect() - } - benchmark.addCase("collect 4 millions") { iter => - sqlContext.range(N * 4).collect() - } - benchmark.run() - - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - collect 1 million 439 / 654 2.4 418.7 1.0X - collect 2 millions 961 / 1907 1.1 916.4 0.5X - collect 4 millions 3193 / 3895 0.3 3044.7 0.1X - */ - } - - ignore("collect limit") { - val N = 1 << 20 - - val benchmark = new Benchmark("collect limit", N) - benchmark.addCase("collect limit 1 million") { iter => - sqlContext.range(N * 4).limit(N).collect() - } - benchmark.addCase("collect limit 2 millions") { iter => - sqlContext.range(N * 4).limit(N * 2).collect() - } - benchmark.run() - - /** - model name : Westmere E56xx/L56xx/X56xx (Nehalem-C) - collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - collect limit 1 million 833 / 1284 1.3 794.4 1.0X - collect limit 2 millions 3348 / 4005 0.3 3193.3 0.2X - */ - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala new file mode 100644 index 0000000000000..c99a5aec1cd6e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.Benchmark + +/** + * Common base trait for micro benchmarks that are supposed to run standalone (i.e. not together + * with other test suites). + */ +private[benchmark] trait BenchmarkBase extends SparkFunSuite { + + lazy val sparkSession = SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.autoBroadcastJoinThreshold", 1) + .getOrCreate() + + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality) + + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f + } + + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + f + } + + benchmark.run() + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala new file mode 100644 index 0000000000000..46db41a8abad9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.IntegerType + +/** + * Benchmark to measure performance for aggregate primitives. + * To run this: + * build/sbt "sql/test-only *benchmark.JoinBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class JoinBenchmark extends BenchmarkBase { + + ignore("broadcast hash join, long key") { + val N = 20 << 20 + val M = 1 << 16 + + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("Join w long", N) { + sparkSession.range(N).join(dim, (col("id") % M) === col("k")).count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + Join w long codegen=false 3002 / 3262 7.0 143.2 1.0X + Join w long codegen=true 321 / 371 65.3 15.3 9.3X + */ + } + + ignore("broadcast hash join, long key with duplicates") { + val N = 20 << 20 + val M = 1 << 16 + + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("Join w long duplicated", N) { + val dim = broadcast(sparkSession.range(M).selectExpr("cast(id/10 as long) as k")) + sparkSession.range(N).join(dim, (col("id") % M) === col("k")).count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w long duplicated codegen=false 3446 / 3478 6.1 164.3 1.0X + *Join w long duplicated codegen=true 322 / 351 65.2 15.3 10.7X + */ + } + + ignore("broadcast hash join, two int key") { + val N = 20 << 20 + val M = 1 << 16 + val dim2 = broadcast(sparkSession.range(M) + .selectExpr("cast(id as int) as k1", "cast(id as int) as k2", "cast(id as string) as v")) + + runBenchmark("Join w 2 ints", N) { + sparkSession.range(N).join(dim2, + (col("id") % M).cast(IntegerType) === col("k1") + && (col("id") % M).cast(IntegerType) === col("k2")).count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 ints codegen=false 4426 / 4501 4.7 211.1 1.0X + *Join w 2 ints codegen=true 791 / 818 26.5 37.7 5.6X + */ + } + + ignore("broadcast hash join, two long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim3 = broadcast(sparkSession.range(M) + .selectExpr("id as k1", "id as k2", "cast(id as string) as v")) + + runBenchmark("Join w 2 longs", N) { + sparkSession.range(N).join(dim3, + (col("id") % M) === col("k1") && (col("id") % M) === col("k2")) + .count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 longs codegen=false 5905 / 6123 3.6 281.6 1.0X + *Join w 2 longs codegen=true 2230 / 2529 9.4 106.3 2.6X + */ + } + + ignore("broadcast hash join, two long key with duplicates") { + val N = 20 << 20 + val M = 1 << 16 + val dim4 = broadcast(sparkSession.range(M) + .selectExpr("cast(id/10 as long) as k1", "cast(id/10 as long) as k2")) + + runBenchmark("Join w 2 longs duplicated", N) { + sparkSession.range(N).join(dim4, + (col("id") bitwiseAND M) === col("k1") && (col("id") bitwiseAND M) === col("k2")) + .count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 longs duplicated codegen=false 6420 / 6587 3.3 306.1 1.0X + *Join w 2 longs duplicated codegen=true 2080 / 2139 10.1 99.2 3.1X + */ + } + + ignore("broadcast hash join, outer join long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("outer join w long", N) { + sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "left").count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *outer join w long codegen=false 3055 / 3189 6.9 145.7 1.0X + *outer join w long codegen=true 261 / 276 80.5 12.4 11.7X + */ + } + + ignore("broadcast hash join, semi join long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("semi join w long", N) { + sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi").count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *semi join w long codegen=false 1912 / 1990 11.0 91.2 1.0X + *semi join w long codegen=true 237 / 244 88.3 11.3 8.1X + */ + } + + ignore("sort merge join") { + val N = 2 << 20 + runBenchmark("merge join", N) { + val df1 = sparkSession.range(N).selectExpr(s"id * 2 as k1") + val df2 = sparkSession.range(N).selectExpr(s"id * 3 as k2") + df1.join(df2, col("k1") === col("k2")).count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *merge join codegen=false 1588 / 1880 1.3 757.1 1.0X + *merge join codegen=true 1477 / 1531 1.4 704.2 1.1X + */ + } + + ignore("sort merge join with duplicates") { + val N = 2 << 20 + runBenchmark("sort merge join", N) { + val df1 = sparkSession.range(N) + .selectExpr(s"(id * 15485863) % ${N*10} as k1") + val df2 = sparkSession.range(N) + .selectExpr(s"(id * 15485867) % ${N*10} as k2") + df1.join(df2, col("k1") === col("k2")).count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *sort merge join codegen=false 3626 / 3667 0.6 1728.9 1.0X + *sort merge join codegen=true 3405 / 3438 0.6 1623.8 1.1X + */ + } + + ignore("shuffle hash join") { + val N = 4 << 20 + sparkSession.conf.set("spark.sql.shuffle.partitions", "2") + sparkSession.conf.set("spark.sql.autoBroadcastJoinThreshold", "10000000") + sparkSession.conf.set("spark.sql.join.preferSortMergeJoin", "false") + runBenchmark("shuffle hash join", N) { + val df1 = sparkSession.range(N).selectExpr(s"id as k1") + val df2 = sparkSession.range(N / 5).selectExpr(s"id * 3 as k2") + df1.join(df2, col("k1") === col("k2")).count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *shuffle hash join codegen=false 1101 / 1391 3.8 262.6 1.0X + *shuffle hash join codegen=true 528 / 578 7.9 125.8 2.1X + */ + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala new file mode 100644 index 0000000000000..470c78120b194 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure whole stage codegen performance. + * To run this: + * build/sbt "sql/test-only *benchmark.MiscBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class MiscBenchmark extends BenchmarkBase { + + ignore("filter & aggregate without group") { + val N = 500L << 22 + runBenchmark("range/filter/sum", N) { + sparkSession.range(N).filter("(id & 1) = 1").groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X + range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X + */ + } + + ignore("range/limit/sum") { + val N = 500L << 20 + runBenchmark("range/limit/sum", N) { + sparkSession.range(N).limit(1000000).groupBy().sum().collect() + } + /* + Westmere E56xx/L56xx/X56xx (Nehalem-C) + range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + range/limit/sum codegen=false 609 / 672 861.6 1.2 1.0X + range/limit/sum codegen=true 561 / 621 935.3 1.1 1.1X + */ + } + + ignore("sample") { + val N = 500 << 18 + runBenchmark("sample with replacement", N) { + sparkSession.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X + sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X + */ + + runBenchmark("sample without replacement", N) { + sparkSession.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X + sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X + */ + } + + ignore("collect") { + val N = 1 << 20 + + val benchmark = new Benchmark("collect", N) + benchmark.addCase("collect 1 million") { iter => + sparkSession.range(N).collect() + } + benchmark.addCase("collect 2 millions") { iter => + sparkSession.range(N * 2).collect() + } + benchmark.addCase("collect 4 millions") { iter => + sparkSession.range(N * 4).collect() + } + benchmark.run() + + /** + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + collect 1 million 439 / 654 2.4 418.7 1.0X + collect 2 millions 961 / 1907 1.1 916.4 0.5X + collect 4 millions 3193 / 3895 0.3 3044.7 0.1X + */ + } + + ignore("collect limit") { + val N = 1 << 20 + + val benchmark = new Benchmark("collect limit", N) + benchmark.addCase("collect limit 1 million") { iter => + sparkSession.range(N * 4).limit(N).collect() + } + benchmark.addCase("collect limit 2 millions") { iter => + sparkSession.range(N * 4).limit(N * 2).collect() + } + benchmark.run() + + /** + model name : Westmere E56xx/L56xx/X56xx (Nehalem-C) + collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + collect limit 1 million 833 / 1284 1.3 794.4 1.0X + collect limit 2 millions 3348 / 4005 0.3 3193.3 0.2X + */ + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala new file mode 100644 index 0000000000000..0e1868dd66565 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -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. + */ + +package org.apache.spark.sql.execution.benchmark + +import java.util.{Arrays, Comparator} + +import org.apache.spark.unsafe.array.LongArray +import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.util.Benchmark +import org.apache.spark.util.collection.Sorter +import org.apache.spark.util.collection.unsafe.sort._ +import org.apache.spark.util.random.XORShiftRandom + +/** + * Benchmark to measure performance for aggregate primitives. + * To run this: + * build/sbt "sql/test-only *benchmark.SortBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class SortBenchmark extends BenchmarkBase { + + private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { + new Sorter(UnsafeSortDataFormat.INSTANCE).sort( + buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { + override def compare( + r1: RecordPointerAndKeyPrefix, + r2: RecordPointerAndKeyPrefix): Int = { + refCmp.compare(r1.keyPrefix, r2.keyPrefix) + } + }) + } + + private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { + val ref = Array.tabulate[Long](size * 2) { i => rand } + val extended = ref ++ Array.fill[Long](size * 2)(0) + (new LongArray(MemoryBlock.fromLongArray(ref)), + new LongArray(MemoryBlock.fromLongArray(extended))) + } + + ignore("sort") { + val size = 25000000 + val rand = new XORShiftRandom(123) + val benchmark = new Benchmark("radix sort " + size, size) + benchmark.addTimerCase("reference TimSort key prefix array") { timer => + val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) + timer.stopTiming() + } + benchmark.addTimerCase("reference Arrays.sort") { timer => + val ref = Array.tabulate[Long](size) { i => rand.nextLong } + timer.startTiming() + Arrays.sort(ref) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort one byte") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong & 0xff + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort two bytes") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong & 0xffff + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort eight bytes") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort key prefix array") { timer => + val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) + timer.startTiming() + RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.run() + + /* + Running benchmark: radix sort 25000000 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 3.13.0-44-generic + Intel(R) Core(TM) i7-4600U CPU @ 2.10GHz + + radix sort 25000000: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + reference TimSort key prefix array 15546 / 15859 1.6 621.9 1.0X + reference Arrays.sort 2416 / 2446 10.3 96.6 6.4X + radix sort one byte 133 / 137 188.4 5.3 117.2X + radix sort two bytes 255 / 258 98.2 10.2 61.1X + radix sort eight bytes 991 / 997 25.2 39.6 15.7X + radix sort key prefix array 1540 / 1563 16.2 61.6 10.1X + */ + } +} From e868a15a7a3ce8895092131f45110c27b734bfb7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 4 May 2016 11:02:48 -0700 Subject: [PATCH 0065/1470] [SPARK-15103][SQL] Refactored FileCatalog class to allow StreamFileCatalog to infer partitioning ## What changes were proposed in this pull request? File Stream Sink writes the list of written files in a metadata log. StreamFileCatalog reads the list of the files for processing. However StreamFileCatalog does not infer partitioning like HDFSFileCatalog. This PR enables that by refactoring HDFSFileCatalog to create an abstract class PartitioningAwareFileCatalog, that has all the functionality to infer partitions from a list of leaf files. - HDFSFileCatalog has been renamed to ListingFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from recursive directory scanning. - StreamFileCatalog has been renamed to MetadataLogFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from the metadata log. - The above two classes has been moved into their own files as they are not interfaces that should be in fileSourceInterfaces.scala. ## How was this patch tested? - FileStreamSinkSuite was update to see if partitioning gets inferred, and on reading whether the partitions get pruned correctly based on the query. - Other unit tests are unchanged and pass as expected. Author: Tathagata Das Closes #12879 from tdas/SPARK-15103. (cherry picked from commit 0fd3a4748416233f034ec137d95f0a4c8712d396) Signed-off-by: Tathagata Das --- .../execution/datasources/DataSource.scala | 8 +- .../datasources/ListingFileCatalog.scala | 127 +++++++++++ .../PartitioningAwareFileCatalog.scala | 155 +++++++++++++ .../datasources/fileSourceInterfaces.scala | 215 +----------------- .../streaming/MetadataLogFileCatalog.scala | 59 +++++ .../streaming/StreamFileCatalog.scala | 58 ----- .../sql/streaming/FileStreamSinkSuite.scala | 64 +++++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 9 +- 8 files changed, 410 insertions(+), 285 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6114142cefea2..618ea3d669bd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -136,7 +136,7 @@ case class DataSource( val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) SparkHadoopUtil.get.globPathIfNecessary(qualified) }.toArray - val fileCatalog: FileCatalog = new HDFSFileCatalog(sparkSession, options, globbedPaths, None) + val fileCatalog = new ListingFileCatalog(sparkSession, globbedPaths, options, None) format.inferSchema( sparkSession, caseInsensitiveOptions, @@ -258,7 +258,7 @@ case class DataSource( case (format: FileFormat, _) if hasMetadata(caseInsensitiveOptions.get("path").toSeq ++ paths) => val basePath = new Path((caseInsensitiveOptions.get("path").toSeq ++ paths).head) - val fileCatalog = new StreamFileCatalog(sparkSession, basePath) + val fileCatalog = new MetadataLogFileCatalog(sparkSession, basePath) val dataSchema = userSpecifiedSchema.orElse { format.inferSchema( sparkSession, @@ -310,8 +310,8 @@ case class DataSource( }) } - val fileCatalog: FileCatalog = - new HDFSFileCatalog(sparkSession, options, globbedPaths, partitionSchema) + val fileCatalog = + new ListingFileCatalog(sparkSession, globbedPaths, options, partitionSchema) val dataSchema = userSpecifiedSchema.map { schema => val equality = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala new file mode 100644 index 0000000000000..bdf43e02f4a07 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.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.execution.datasources + +import scala.collection.mutable +import scala.util.Try + +import org.apache.hadoop.fs.{FileStatus, LocatedFileStatus, Path} +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType + + +/** + * A [[FileCatalog]] that generates the list of files to process by recursively listing all the + * files present in `paths`. + * + * @param parameters as set of options to control discovery + * @param paths a list of paths to scan + * @param partitionSchema an optional partition schema that will be use to provide types for the + * discovered partitions + */ +class ListingFileCatalog( + sparkSession: SparkSession, + override val paths: Seq[Path], + parameters: Map[String, String], + partitionSchema: Option[StructType]) + extends PartitioningAwareFileCatalog(sparkSession, parameters, partitionSchema) { + + @volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _ + @volatile private var cachedLeafDirToChildrenFiles: Map[Path, Array[FileStatus]] = _ + @volatile private var cachedPartitionSpec: PartitionSpec = _ + + refresh() + + override def partitionSpec(): PartitionSpec = { + if (cachedPartitionSpec == null) { + cachedPartitionSpec = inferPartitioning() + } + cachedPartitionSpec + } + + override protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = { + cachedLeafFiles + } + + override protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = { + cachedLeafDirToChildrenFiles + } + + override def refresh(): Unit = { + val files = listLeafFiles(paths) + cachedLeafFiles = + new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f) + cachedLeafDirToChildrenFiles = files.toArray.groupBy(_.getPath.getParent) + cachedPartitionSpec = null + } + + protected def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext) + } else { + val statuses: Seq[FileStatus] = paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + logInfo(s"Listing $path on driver") + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + + val statuses = { + val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) + if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats + } + + statuses.map { + case f: LocatedFileStatus => f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should a + // a big deal since we always use to `listLeafFilesInParallel` when the number of paths + // exceeds threshold. + case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + } + }.filterNot { status => + val name = status.getPath.getName + HadoopFsRelation.shouldFilterOut(name) + } + + val (dirs, files) = statuses.partition(_.isDirectory) + + // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) + if (dirs.isEmpty) { + mutable.LinkedHashSet(files: _*) + } else { + mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) + } + } + } + + override def equals(other: Any): Boolean = other match { + case hdfs: ListingFileCatalog => paths.toSet == hdfs.paths.toSet + case _ => false + } + + override def hashCode(): Int = paths.toSet.hashCode() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala new file mode 100644 index 0000000000000..9d997d628579c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -0,0 +1,155 @@ +/* + * 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.datasources + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{expressions, InternalRow} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.{StringType, StructType} + + +/** + * An abstract class that represents [[FileCatalog]]s that are aware of partitioned tables. + * It provides the necessary methods to parse partition data based on a set of files. + * + * @param parameters as set of options to control partition discovery + * @param partitionSchema an optional partition schema that will be use to provide types for the + * discovered partitions +*/ +abstract class PartitioningAwareFileCatalog( + sparkSession: SparkSession, + parameters: Map[String, String], + partitionSchema: Option[StructType]) + extends FileCatalog with Logging { + + protected val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) + + protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] + + protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] + + override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + if (partitionSpec().partitionColumns.isEmpty) { + Partition(InternalRow.empty, allFiles().filterNot(_.getPath.getName startsWith "_")) :: Nil + } else { + prunePartitions(filters, partitionSpec()).map { + case PartitionDirectory(values, path) => + Partition( + values, + leafDirToChildrenFiles(path).filterNot(_.getPath.getName startsWith "_")) + } + } + } + + override def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq + + protected def inferPartitioning(): PartitionSpec = { + // We use leaf dirs containing data files to discover the schema. + val leafDirs = leafDirToChildrenFiles.keys.toSeq + partitionSchema match { + case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => + val spec = PartitioningUtils.parsePartitions( + leafDirs, + PartitioningUtils.DEFAULT_PARTITION_NAME, + typeInference = false, + basePaths = basePaths) + + // Without auto inference, all of value in the `row` should be null or in StringType, + // we need to cast into the data type that user specified. + def castPartitionValuesToUserSchema(row: InternalRow) = { + InternalRow((0 until row.numFields).map { i => + Cast( + Literal.create(row.getUTF8String(i), StringType), + userProvidedSchema.fields(i).dataType).eval() + }: _*) + } + + PartitionSpec(userProvidedSchema, spec.partitions.map { part => + part.copy(values = castPartitionValuesToUserSchema(part.values)) + }) + case _ => + PartitioningUtils.parsePartitions( + leafDirs, + PartitioningUtils.DEFAULT_PARTITION_NAME, + typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled(), + basePaths = basePaths) + } + } + + private def prunePartitions( + predicates: Seq[Expression], + partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { + val PartitionSpec(partitionColumns, partitions) = partitionSpec + val partitionColumnNames = partitionColumns.map(_.name).toSet + val partitionPruningPredicates = predicates.filter { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + + if (partitionPruningPredicates.nonEmpty) { + val predicate = partitionPruningPredicates.reduce(expressions.And) + + val boundPredicate = InterpretedPredicate.create(predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }) + + val selected = partitions.filter { + case PartitionDirectory(values, _) => boundPredicate(values) + } + logInfo { + val total = partitions.length + val selectedSize = selected.length + val percentPruned = (1 - selectedSize.toDouble / total.toDouble) * 100 + s"Selected $selectedSize partitions out of $total, pruned $percentPruned% partitions." + } + + selected + } else { + partitions + } + } + + /** + * Contains a set of paths that are considered as the base dirs of the input datasets. + * The partitioning discovery logic will make sure it will stop when it reaches any + * base path. By default, the paths of the dataset provided by users will be base paths. + * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path + * will be `/path/something=true/`, and the returned DataFrame will not contain a column of + * `something`. If users want to override the basePath. They can set `basePath` in the options + * to pass the new base path to the data source. + * For the above example, if the user-provided base path is `/path/`, the returned + * DataFrame will have the column of `something`. + */ + private def basePaths: Set[Path] = { + val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) + userDefinedBasePath.getOrElse { + // If the user does not provide basePath, we will just use paths. + paths.toSet + }.map { hdfsPath => + // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). + val fs = hdfsPath.getFileSystem(hadoopConf) + hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 24e2bf6d136ad..c87e672961091 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -304,232 +304,31 @@ case class Partition(values: InternalRow, files: Seq[FileStatus]) * as the partitioning characteristics of those files. */ trait FileCatalog { + + /** Returns the list of input paths from which the catalog will get files. */ def paths: Seq[Path] + /** Returns the specification of the partitions inferred from the data. */ def partitionSpec(): PartitionSpec /** * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with not partition values. + * unpartitioned, this will return a single partition with no partition values. * - * @param filters the filters used to prune which partitions are returned. These filters must + * @param filters The filters used to prune which partitions are returned. These filters must * only refer to partition columns and this method will only return files * where these predicates are guaranteed to evaluate to `true`. Thus, these * filters will not need to be evaluated again on the returned data. */ def listFiles(filters: Seq[Expression]): Seq[Partition] + /** Returns all the valid files. */ def allFiles(): Seq[FileStatus] - def getStatus(path: Path): Array[FileStatus] - + /** Refresh the file listing */ def refresh(): Unit } -/** - * A file catalog that caches metadata gathered by scanning all the files present in `paths` - * recursively. - * - * @param parameters as set of options to control discovery - * @param paths a list of paths to scan - * @param partitionSchema an optional partition schema that will be use to provide types for the - * discovered partitions - */ -class HDFSFileCatalog( - sparkSession: SparkSession, - parameters: Map[String, String], - override val paths: Seq[Path], - partitionSchema: Option[StructType]) - extends FileCatalog with Logging { - - private val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(parameters) - - var leafFiles = mutable.LinkedHashMap.empty[Path, FileStatus] - var leafDirToChildrenFiles = mutable.Map.empty[Path, Array[FileStatus]] - var cachedPartitionSpec: PartitionSpec = _ - - def partitionSpec(): PartitionSpec = { - if (cachedPartitionSpec == null) { - cachedPartitionSpec = inferPartitioning(partitionSchema) - } - - cachedPartitionSpec - } - - refresh() - - override def listFiles(filters: Seq[Expression]): Seq[Partition] = { - if (partitionSpec().partitionColumns.isEmpty) { - Partition(InternalRow.empty, allFiles().filterNot(_.getPath.getName startsWith "_")) :: Nil - } else { - prunePartitions(filters, partitionSpec()).map { - case PartitionDirectory(values, path) => - Partition( - values, - getStatus(path).filterNot(_.getPath.getName startsWith "_")) - } - } - } - - protected def prunePartitions( - predicates: Seq[Expression], - partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { - val PartitionSpec(partitionColumns, partitions) = partitionSpec - val partitionColumnNames = partitionColumns.map(_.name).toSet - val partitionPruningPredicates = predicates.filter { - _.references.map(_.name).toSet.subsetOf(partitionColumnNames) - } - - if (partitionPruningPredicates.nonEmpty) { - val predicate = partitionPruningPredicates.reduce(expressions.And) - - val boundPredicate = InterpretedPredicate.create(predicate.transform { - case a: AttributeReference => - val index = partitionColumns.indexWhere(a.name == _.name) - BoundReference(index, partitionColumns(index).dataType, nullable = true) - }) - - val selected = partitions.filter { - case PartitionDirectory(values, _) => boundPredicate(values) - } - logInfo { - val total = partitions.length - val selectedSize = selected.length - val percentPruned = (1 - selectedSize.toDouble / total.toDouble) * 100 - s"Selected $selectedSize partitions out of $total, pruned $percentPruned% partitions." - } - - selected - } else { - partitions - } - } - - def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq - - def getStatus(path: Path): Array[FileStatus] = leafDirToChildrenFiles(path) - - private def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext) - } else { - val statuses: Seq[FileStatus] = paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - logInfo(s"Listing $path on driver") - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - - val statuses = { - val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) - if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats - } - - statuses.map { - case f: LocatedFileStatus => f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should a - // a big deal since we always use to `listLeafFilesInParallel` when the number of paths - // exceeds threshold. - case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) - } - }.filterNot { status => - val name = status.getPath.getName - HadoopFsRelation.shouldFilterOut(name) - } - - val (dirs, files) = statuses.partition(_.isDirectory) - - // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) - if (dirs.isEmpty) { - mutable.LinkedHashSet(files: _*) - } else { - mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) - } - } - } - - def inferPartitioning(schema: Option[StructType]): PartitionSpec = { - // We use leaf dirs containing data files to discover the schema. - val leafDirs = leafDirToChildrenFiles.keys.toSeq - schema match { - case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => - val spec = PartitioningUtils.parsePartitions( - leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = false, - basePaths = basePaths) - - // Without auto inference, all of value in the `row` should be null or in StringType, - // we need to cast into the data type that user specified. - def castPartitionValuesToUserSchema(row: InternalRow) = { - InternalRow((0 until row.numFields).map { i => - Cast( - Literal.create(row.getUTF8String(i), StringType), - userProvidedSchema.fields(i).dataType).eval() - }: _*) - } - - PartitionSpec(userProvidedSchema, spec.partitions.map { part => - part.copy(values = castPartitionValuesToUserSchema(part.values)) - }) - case _ => - PartitioningUtils.parsePartitions( - leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled(), - basePaths = basePaths) - } - } - - /** - * Contains a set of paths that are considered as the base dirs of the input datasets. - * The partitioning discovery logic will make sure it will stop when it reaches any - * base path. By default, the paths of the dataset provided by users will be base paths. - * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path - * will be `/path/something=true/`, and the returned DataFrame will not contain a column of - * `something`. If users want to override the basePath. They can set `basePath` in the options - * to pass the new base path to the data source. - * For the above example, if the user-provided base path is `/path/`, the returned - * DataFrame will have the column of `something`. - */ - private def basePaths: Set[Path] = { - val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) - userDefinedBasePath.getOrElse { - // If the user does not provide basePath, we will just use paths. - paths.toSet - }.map { hdfsPath => - // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). - val fs = hdfsPath.getFileSystem(hadoopConf) - hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - } - } - - def refresh(): Unit = { - val files = listLeafFiles(paths) - - leafFiles.clear() - leafDirToChildrenFiles.clear() - - leafFiles ++= files.map(f => f.getPath -> f) - leafDirToChildrenFiles ++= files.toArray.groupBy(_.getPath.getParent) - - cachedPartitionSpec = null - } - - override def equals(other: Any): Boolean = other match { - case hdfs: HDFSFileCatalog => paths.toSet == hdfs.paths.toSet - case _ => false - } - - override def hashCode(): Int = paths.toSet.hashCode() -} /** * Helper methods for gathering metadata from HDFS. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala new file mode 100644 index 0000000000000..20ade12e3796a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources._ + + +/** + * A [[FileCatalog]] that generates the list of files to processing by reading them from the + * metadata log files generated by the [[FileStreamSink]]. + */ +class MetadataLogFileCatalog(sparkSession: SparkSession, path: Path) + extends PartitioningAwareFileCatalog(sparkSession, Map.empty, None) { + + private val metadataDirectory = new Path(path, FileStreamSink.metadataDir) + logInfo(s"Reading streaming file log from $metadataDirectory") + private val metadataLog = new FileStreamSinkLog(sparkSession, metadataDirectory.toUri.toString) + private val allFilesFromLog = metadataLog.allFiles().map(_.toFileStatus).filterNot(_.isDirectory) + private var cachedPartitionSpec: PartitionSpec = _ + + override protected val leafFiles: mutable.LinkedHashMap[Path, FileStatus] = { + new mutable.LinkedHashMap ++= allFilesFromLog.map(f => f.getPath -> f) + } + + override protected val leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = { + allFilesFromLog.toArray.groupBy(_.getPath.getParent) + } + + override def paths: Seq[Path] = path :: Nil + + override def refresh(): Unit = { } + + override def partitionSpec(): PartitionSpec = { + if (cachedPartitionSpec == null) { + cachedPartitionSpec = inferPartitioning() + } + cachedPartitionSpec + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala deleted file mode 100644 index 4f699719c2768..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming - -import org.apache.hadoop.fs.{FileStatus, Path} - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.datasources.{FileCatalog, Partition, PartitionSpec} -import org.apache.spark.sql.types.StructType - -class StreamFileCatalog(sparkSession: SparkSession, path: Path) extends FileCatalog with Logging { - val metadataDirectory = new Path(path, FileStreamSink.metadataDir) - logInfo(s"Reading streaming file log from $metadataDirectory") - val metadataLog = new FileStreamSinkLog(sparkSession, metadataDirectory.toUri.toString) - val fs = path.getFileSystem(sparkSession.sessionState.newHadoopConf()) - - override def paths: Seq[Path] = path :: Nil - - override def partitionSpec(): PartitionSpec = PartitionSpec(StructType(Nil), Nil) - - /** - * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with not partition values. - * - * @param filters the filters used to prune which partitions are returned. These filters must - * only refer to partition columns and this method will only return files - * where these predicates are guaranteed to evaluate to `true`. Thus, these - * filters will not need to be evaluated again on the returned data. - */ - override def listFiles(filters: Seq[Expression]): Seq[Partition] = - Partition(InternalRow.empty, allFiles()) :: Nil - - override def getStatus(path: Path): Array[FileStatus] = fs.listStatus(path) - - override def refresh(): Unit = {} - - override def allFiles(): Seq[FileStatus] = { - metadataLog.allFiles().map(_.toFileStatus) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 609ca976a0160..e937fc3e876e2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -22,12 +22,14 @@ import java.io.File import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.{DirectoryFileFilter, RegexFileFilter} -import org.apache.spark.sql.{ContinuousQuery, Row, StreamTest} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream} +import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream, MetadataLogFileCatalog} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils class FileStreamSinkSuite extends StreamTest with SharedSQLContext { @@ -147,7 +149,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { } } - test("FileStreamSink - partitioned writing and batch reading [IGNORES PARTITION COLUMN]") { + test("FileStreamSink - partitioned writing and batch reading") { val inputData = MemoryStream[Int] val ds = inputData.toDS() @@ -157,7 +159,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { var query: ContinuousQuery = null try { - query = + query = ds.map(i => (i, i * 1000)) .toDF("id", "value") .write @@ -171,12 +173,58 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { query.processAllAvailable() } - // TODO (tdas): Test partition column can be read or not val outputDf = sqlContext.read.parquet(outputDir) + val expectedSchema = new StructType() + .add(StructField("value", IntegerType)) + .add(StructField("id", IntegerType)) + assert(outputDf.schema === expectedSchema) + + // Verify that MetadataLogFileCatalog is being used and the correct partitioning schema has + // been inferred + val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { + case LogicalRelation(baseRelation, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => + baseRelation.asInstanceOf[HadoopFsRelation] + } + assert(hadoopdFsRelations.size === 1) + assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileCatalog]) + assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id")) + assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value")) + + // Verify the data is correctly read checkDataset( - outputDf.as[Int], - 1000, 2000, 3000) + outputDf.as[(Int, Int)], + (1000, 1), (2000, 2), (3000, 3)) + + /** Check some condition on the partitions of the FileScanRDD generated by a DF */ + def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = { + val getFileScanRDD = df.queryExecution.executedPlan.collect { + case scan: DataSourceScanExec if scan.rdd.isInstanceOf[FileScanRDD] => + scan.rdd.asInstanceOf[FileScanRDD] + }.headOption.getOrElse { + fail(s"No FileScan in query\n${df.queryExecution}") + } + func(getFileScanRDD.filePartitions) + } + // Read without pruning + checkFileScanPartitions(outputDf) { partitions => + // There should be as many distinct partition values as there are distinct ids + assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3) + } + + // Read with pruning, should read only files in partition dir id=1 + checkFileScanPartitions(outputDf.filter("id = 1")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(filesToBeRead.map(_.filePath).forall(_.contains("/id=1/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 1) + } + + // Read with pruning, should read only files in partition dir id=1 and id=2 + checkFileScanPartitions(outputDf.filter("id in (1,2)")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(!filesToBeRead.map(_.filePath).exists(_.contains("/id=3/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 2) + } } finally { if (query != null) { query.stop() 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 0b24d358548b0..7a799b6c87e9d 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 @@ -536,17 +536,12 @@ private[hive] class MetaStoreFileCatalog( sparkSession: SparkSession, paths: Seq[Path], partitionSpecFromHive: PartitionSpec) - extends HDFSFileCatalog( + extends ListingFileCatalog( sparkSession, - Map.empty, paths, + Map.empty, Some(partitionSpecFromHive.partitionColumns)) { - override def getStatus(path: Path): Array[FileStatus] = { - val fs = path.getFileSystem(sparkSession.sessionState.newHadoopConf()) - fs.listStatus(path) - } - override def partitionSpec(): PartitionSpec = partitionSpecFromHive } From 45862f6c935c36969a62a3fbb863cce55c4a6426 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 May 2016 14:26:05 -0700 Subject: [PATCH 0066/1470] [SPARK-15126][SQL] RuntimeConfig.set should return Unit ## What changes were proposed in this pull request? Currently we return RuntimeConfig itself to facilitate chaining. However, it makes the output in interactive environments (e.g. notebooks, scala repl) weird because it'd show the response of calling set as a RuntimeConfig itself. ## How was this patch tested? Updated unit tests. Author: Reynold Xin Closes #12902 from rxin/SPARK-15126. (cherry picked from commit 6ae9fc00ed6ef530a9c42c8407fc66fd873239cc) Signed-off-by: Andrew Or --- python/pyspark/sql/conf.py | 1 - python/pyspark/sql/session.py | 3 --- .../org/apache/spark/sql/RuntimeConfig.scala | 7 +++---- .../sql/{internal => }/RuntimeConfigSuite.scala | 16 ++++++++-------- 4 files changed, 11 insertions(+), 16 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/{internal => }/RuntimeConfigSuite.scala (85%) diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index 7428c919915f3..609d882a95a32 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -23,7 +23,6 @@ class RuntimeConfig(object): """User-facing configuration API, accessible through `SparkSession.conf`. Options set here are automatically propagated to the Hadoop configuration during I/O. - This a thin wrapper around its Scala implementation org.apache.spark.sql.RuntimeConfig. """ def __init__(self, jconf): diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index fb3e318163e87..04842f6185c71 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -71,9 +71,6 @@ class SparkSession(object): .config("spark.some.config.option", "some-value") \ .getOrCreate() - :param sparkContext: The :class:`SparkContext` backing this SparkSession. - :param jsparkSession: An optional JVM Scala SparkSession. If set, we do not instantiate a new - SparkSession in the JVM, instead we make all calls to this object. """ class Builder(object): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 4fd6e42640d24..7e07e0cb84a87 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -35,9 +35,8 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * * @since 2.0.0 */ - def set(key: String, value: String): RuntimeConfig = { + def set(key: String, value: String): Unit = { sqlConf.setConfString(key, value) - this } /** @@ -45,7 +44,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * * @since 2.0.0 */ - def set(key: String, value: Boolean): RuntimeConfig = { + def set(key: String, value: Boolean): Unit = { set(key, value.toString) } @@ -54,7 +53,7 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { * * @since 2.0.0 */ - def set(key: String, value: Long): RuntimeConfig = { + def set(key: String, value: Long): Unit = { set(key, value.toString) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala similarity index 85% rename from sql/core/src/test/scala/org/apache/spark/sql/internal/RuntimeConfigSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index a629b73ac046d..cfe2e9f2dbc44 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.internal +package org.apache.spark.sql import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.RuntimeConfig class RuntimeConfigSuite extends SparkFunSuite { @@ -26,10 +25,9 @@ class RuntimeConfigSuite extends SparkFunSuite { test("set and get") { val conf = newConf() - conf - .set("k1", "v1") - .set("k2", 2) - .set("k3", value = false) + conf.set("k1", "v1") + conf.set("k2", 2) + conf.set("k3", value = false) assert(conf.get("k1") == "v1") assert(conf.get("k2") == "2") @@ -41,13 +39,15 @@ class RuntimeConfigSuite extends SparkFunSuite { } test("getOption") { - val conf = newConf().set("k1", "v1") + val conf = newConf() + conf.set("k1", "v1") assert(conf.getOption("k1") == Some("v1")) assert(conf.getOption("notset") == None) } test("unset") { - val conf = newConf().set("k1", "v1") + val conf = newConf() + conf.set("k1", "v1") assert(conf.get("k1") == "v1") conf.unset("k1") intercept[NoSuchElementException] { From eeb18f6d70bc75f6d1292938292ad066d85ced8a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 4 May 2016 14:28:26 -0700 Subject: [PATCH 0067/1470] [SPARK-15121] Improve logging of external shuffle handler ## What changes were proposed in this pull request? Add more informative logging in the external shuffle service to aid in debugging who is connecting to the YARN Nodemanager when the external shuffle service runs under it. ## How was this patch tested? Ran and saw logs coming out in log file. Author: Thomas Graves Closes #12900 from tgravescs/SPARK-15121. (cherry picked from commit 0c00391f77359efdbf9dbd26d4c8186be8839922) Signed-off-by: Andrew Or --- .../spark/network/shuffle/ExternalShuffleBlockHandler.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index f8d03b3b9433a..fb1226c09e246 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -35,6 +35,7 @@ import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; import org.apache.spark.network.shuffle.protocol.*; +import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -86,7 +87,8 @@ protected void handleMessage( blocks.add(blockManager.getBlockData(msg.appId, msg.execId, blockId)); } long streamId = streamManager.registerStream(client.getClientId(), blocks.iterator()); - logger.trace("Registered streamId {} with {} buffers", streamId, msg.blockIds.length); + logger.trace("Registered streamId {} with {} buffers for client {} from host {}", streamId, + msg.blockIds.length, client.getClientId(), NettyUtils.getRemoteAddress(client.getChannel())); callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); } else if (msgObj instanceof RegisterExecutor) { From c0715f33b456f8379117a6ecae3ff2cda6f59a7c Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Wed, 4 May 2016 14:29:54 -0700 Subject: [PATCH 0068/1470] [SPARK-12299][CORE] Remove history serving functionality from Master Remove history server functionality from standalone Master. Previously, the Master process rebuilt a SparkUI once the application was completed which sometimes caused problems, such as OOM, when the application event log is large (see SPARK-6270). Keeping this functionality out of the Master will help to simplify the process and increase stability. Testing for this change included running core unit tests and manually running an application on a standalone cluster to verify that it completed successfully and that the Master UI functioned correctly. Also added 2 unit tests to verify killing an application and driver from MasterWebUI makes the correct request to the Master. Author: Bryan Cutler Closes #10991 from BryanCutler/remove-history-master-SPARK-12299. (cherry picked from commit cf2e9da612397233ae7bca0e9ce57309f16226b5) Signed-off-by: Andrew Or --- .../spark/deploy/master/ApplicationInfo.scala | 9 -- .../apache/spark/deploy/master/Master.scala | 109 +--------------- .../spark/deploy/master/MasterMessages.scala | 2 - .../deploy/master/ui/ApplicationPage.scala | 6 +- .../master/ui/HistoryNotFoundPage.scala | 73 ----------- .../spark/deploy/master/ui/MasterPage.scala | 8 +- .../spark/deploy/master/ui/MasterWebUI.scala | 40 +----- .../spark/status/api/v1/ApiRootResource.scala | 2 +- .../api/v1/ApplicationListResource.scala | 30 ----- .../deploy/master/ui/MasterWebUISuite.scala | 118 +++++++++++------- docs/monitoring.md | 5 - 11 files changed, 86 insertions(+), 316 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala 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 4ffb5283e99a4..53564d0e95152 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 @@ -41,7 +41,6 @@ private[spark] class ApplicationInfo( @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ - @transient @volatile var appUIUrlAtHistoryServer: Option[String] = None // A cap on the number of executors this application can have at any given time. // By default, this is infinite. Only after the first allocation request is issued by the @@ -66,7 +65,6 @@ private[spark] class ApplicationInfo( nextExecutorId = 0 removedExecutors = new ArrayBuffer[ExecutorDesc] executorLimit = desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE) - appUIUrlAtHistoryServer = None } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -136,11 +134,4 @@ private[spark] class ApplicationInfo( System.currentTimeMillis() - startTime } } - - /** - * Returns the original application UI url unless there is its address at history server - * is defined - */ - def curAppUIUrl: String = appUIUrlAtHistoryServer.getOrElse(desc.appUiUrl) - } 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 edc9be2a8a8cb..faed4f4dc9573 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 @@ -17,25 +17,17 @@ package org.apache.spark.deploy.master -import java.io.FileNotFoundException -import java.net.URLEncoder import java.text.SimpleDateFormat import java.util.Date -import java.util.concurrent.{ConcurrentHashMap, ScheduledFuture, TimeUnit} +import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration.Duration -import scala.language.postfixOps import scala.util.Random -import org.apache.hadoop.fs.Path - import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ -import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI @@ -43,9 +35,7 @@ import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ -import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.serializer.{JavaSerializer, Serializer} -import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ThreadUtils, Utils} private[deploy] class Master( @@ -59,10 +49,6 @@ private[deploy] class Master( private val forwardMessageThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("master-forward-message-thread") - private val rebuildUIThread = - ThreadUtils.newDaemonSingleThreadExecutor("master-rebuild-ui-thread") - private val rebuildUIContext = ExecutionContext.fromExecutor(rebuildUIThread) - private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs @@ -85,8 +71,6 @@ private[deploy] class Master( private val addressToApp = new HashMap[RpcAddress, ApplicationInfo] private val completedApps = new ArrayBuffer[ApplicationInfo] private var nextAppNumber = 0 - // Using ConcurrentHashMap so that master-rebuild-ui-thread can add a UI after asyncRebuildUI - private val appIdToUI = new ConcurrentHashMap[String, SparkUI] private val drivers = new HashSet[DriverInfo] private val completedDrivers = new ArrayBuffer[DriverInfo] @@ -199,7 +183,6 @@ private[deploy] class Master( checkForWorkerTimeOutTask.cancel(true) } forwardMessageThread.shutdownNow() - rebuildUIThread.shutdownNow() webUi.stop() restServer.foreach(_.stop()) masterMetricsSystem.stop() @@ -391,9 +374,6 @@ private[deploy] class Master( case CheckForWorkerTimeOut => timeOutDeadWorkers() - case AttachCompletedRebuildUI(appId) => - // An asyncRebuildSparkUI has completed, so need to attach to master webUi - Option(appIdToUI.get(appId)).foreach { ui => webUi.attachSparkUI(ui) } } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -844,7 +824,6 @@ private[deploy] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach { a => - Option(appIdToUI.remove(a.id)).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) } completedApps.trimStart(toRemove) @@ -852,9 +831,6 @@ private[deploy] class Master( completedApps += app // Remember it in our history waitingApps -= app - // If application events are logged, use them to rebuild the UI - asyncRebuildSparkUI(app) - for (exec <- app.executors.values) { killExecutor(exec) } @@ -953,89 +929,6 @@ private[deploy] class Master( exec.state = ExecutorState.KILLED } - /** - * Rebuild a new SparkUI from the given application's event logs. - * Return the UI if successful, else None - */ - private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { - val futureUI = asyncRebuildSparkUI(app) - ThreadUtils.awaitResult(futureUI, Duration.Inf) - } - - /** Rebuild a new SparkUI asynchronously to not block RPC event loop */ - private[master] def asyncRebuildSparkUI(app: ApplicationInfo): Future[Option[SparkUI]] = { - val appName = app.desc.name - val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogDir = app.desc.eventLogDir - .getOrElse { - // Event logging is disabled for this application - app.appUIUrlAtHistoryServer = Some(notFoundBasePath) - return Future.successful(None) - } - val futureUI = Future { - val eventLogFilePrefix = EventLoggingListener.getLogPath( - eventLogDir, app.id, appAttemptId = None, compressionCodecName = app.desc.eventLogCodec) - val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) - val inProgressExists = fs.exists(new Path(eventLogFilePrefix + - EventLoggingListener.IN_PROGRESS)) - - val eventLogFile = if (inProgressExists) { - // Event logging is enabled for this application, but the application is still in progress - logWarning(s"Application $appName is still in progress, it may be terminated abnormally.") - eventLogFilePrefix + EventLoggingListener.IN_PROGRESS - } else { - eventLogFilePrefix - } - - val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) - val replayBus = new ReplayListenerBus() - val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime) - try { - replayBus.replay(logInput, eventLogFile, inProgressExists) - } finally { - logInput.close() - } - - Some(ui) - }(rebuildUIContext) - - futureUI.onSuccess { case Some(ui) => - appIdToUI.put(app.id, ui) - // `self` can be null if we are already in the process of shutting down - // This happens frequently in tests where `local-cluster` is used - if (self != null) { - self.send(AttachCompletedRebuildUI(app.id)) - } - // Application UI is successfully rebuilt, so link the Master UI to it - // NOTE - app.appUIUrlAtHistoryServer is volatile - app.appUIUrlAtHistoryServer = Some(ui.basePath) - }(ThreadUtils.sameThread) - - futureUI.onFailure { - case fnf: FileNotFoundException => - // Event logging is enabled for this application, but no event logs are found - val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir.get}." - logWarning(msg) - msg += " Did you specify the correct logging directory?" - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = Some(notFoundBasePath + s"?msg=$msg&title=$title") - - case e: Exception => - // Relay exception message to application UI page - val title = s"Application history load error (${app.id})" - val exception = URLEncoder.encode(Utils.exceptionString(e), "UTF-8") - var msg = s"Exception in replaying log for application $appName!" - logError(msg, e) - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = - Some(notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title") - }(ThreadUtils.sameThread) - - futureUI - } - /** Generate a new app ID given a app's submission date */ private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala index a055d097674ce..a952cee36eb44 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -39,6 +39,4 @@ private[master] object MasterMessages { case object BoundPortsRequest case class BoundPortsResponse(rpcEndpointPort: Int, webUIPort: Int, restPort: Option[Int]) - - case class AttachCompletedRebuildUI(appId: String) } 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 96274958d1422..8875fc223250d 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 @@ -75,7 +75,11 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • -
  • Application Detail UI
  • + { + if (!app.isFinished) { +
  • Application Detail UI
  • + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala deleted file mode 100644 index e021f1eef794f..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.master.ui - -import java.net.URLDecoder -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.spark.ui.{UIUtils, WebUIPage} - -private[ui] class HistoryNotFoundPage(parent: MasterWebUI) - extends WebUIPage("history/not-found") { - - /** - * Render a page that conveys failure in loading application history. - * - * This accepts 3 HTTP parameters: - * msg = message to display to the user - * title = title of the page - * exception = detailed description of the exception in loading application history (if any) - * - * Parameters "msg" and "exception" are assumed to be UTF-8 encoded. - */ - def render(request: HttpServletRequest): Seq[Node] = { - val titleParam = request.getParameter("title") - val msgParam = request.getParameter("msg") - val exceptionParam = request.getParameter("exception") - - // If no parameters are specified, assume the user did not enable event logging - val defaultTitle = "Event logging is not enabled" - val defaultContent = -
    -
    - No event logs were found for this application! To - enable event logging, - set spark.eventLog.enabled to true and - spark.eventLog.dir to the directory to which your - event logs are written. -
    -
    - - val title = Option(titleParam).getOrElse(defaultTitle) - val content = Option(msgParam) - .map { msg => URLDecoder.decode(msg, "UTF-8") } - .map { msg => -
    -
    {msg}
    -
    ++ - Option(exceptionParam) - .map { e => URLDecoder.decode(e, "UTF-8") } - .map { e =>
    {e}
    } - .getOrElse(Seq.empty) - }.getOrElse(defaultContent) - - UIUtils.basicSparkPage(content, title) - } -} 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 363f4b84f885e..75de3ede78b41 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 @@ -206,7 +206,13 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {killLink} - {app.desc.name} + { + if (app.isFinished) { + app.desc.name + } else { + {app.desc.name} + } + } {app.coresGranted} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index ae16ce90c84b7..a0727ad83fb66 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -19,8 +19,6 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging -import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, - UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -30,60 +28,26 @@ import org.apache.spark.ui.JettyUtils._ private[master] class MasterWebUI( val master: Master, - requestedPort: Int, - customMasterPage: Option[MasterPage] = None) + requestedPort: Int) extends WebUI(master.securityMgr, master.securityMgr.getSSLOptions("standalone"), - requestedPort, master.conf, name = "MasterUI") with Logging with UIRoot { + requestedPort, master.conf, name = "MasterUI") with Logging { val masterEndpointRef = master.self val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) - val masterPage = customMasterPage.getOrElse(new MasterPage(this)) - initialize() /** Initialize all components of the server. */ def initialize() { val masterPage = new MasterPage(this) attachPage(new ApplicationPage(this)) - attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(ApiRootResource.getServletHandler(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST"))) } - - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - ui.getHandlers.foreach(attachHandler) - } - - /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - ui.getHandlers.foreach(detachHandler) - } - - def getApplicationInfoList: Iterator[ApplicationInfo] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++ - completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) } - } - - def getSparkUI(appId: String): Option[SparkUI] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - (activeApps ++ completedApps).find { _.id == appId }.flatMap { - master.rebuildSparkUI - } - } } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index ba9cd711f18e2..2cd51a9ed5417 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -205,7 +205,7 @@ private[spark] object ApiRootResource { /** * This trait is shared by the all the root containers for application UI information -- - * the HistoryServer, the Master UI, and the application UI. This provides the common + * the HistoryServer and the application UI. This provides the common * interface needed for them all to expose application info as json. */ private[spark] trait UIRoot { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 0f30183682469..02fd2985fa20d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -21,7 +21,6 @@ import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam} import javax.ws.rs.core.MediaType import org.apache.spark.deploy.history.ApplicationHistoryInfo -import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo} @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class ApplicationListResource(uiRoot: UIRoot) { @@ -84,33 +83,4 @@ private[spark] object ApplicationsListResource { } ) } - - def convertApplicationInfo( - internal: InternalApplicationInfo, - completed: Boolean): ApplicationInfo = { - // standalone application info always has just one attempt - new ApplicationInfo( - id = internal.id, - name = internal.desc.name, - coresGranted = Some(internal.coresGranted), - maxCores = internal.desc.maxCores, - coresPerExecutor = internal.desc.coresPerExecutor, - memoryPerExecutorMB = Some(internal.desc.memoryPerExecutorMB), - attempts = Seq(new ApplicationAttemptInfo( - attemptId = None, - startTime = new Date(internal.startTime), - endTime = new Date(internal.endTime), - duration = - if (internal.endTime > 0) { - internal.endTime - internal.startTime - } else { - 0 - }, - lastUpdated = new Date(internal.endTime), - sparkUser = internal.desc.user, - completed = completed - )) - ) - } - } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index 0c9382a92bcaf..69a460fbc7dba 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -17,74 +17,96 @@ package org.apache.spark.deploy.master.ui +import java.io.DataOutputStream +import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets import java.util.Date -import scala.io.Source -import scala.language.postfixOps +import scala.collection.mutable.HashMap -import org.json4s.jackson.JsonMethods._ -import org.json4s.JsonAST.{JInt, JNothing, JString} -import org.mockito.Mockito.{mock, when} -import org.scalatest.BeforeAndAfter +import org.mockito.Mockito.{mock, times, verify, when} +import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.DeployMessages.MasterStateResponse +import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, RequestKillDriver} import org.apache.spark.deploy.DeployTestUtils._ import org.apache.spark.deploy.master._ -import org.apache.spark.rpc.RpcEnv +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} -class MasterWebUISuite extends SparkFunSuite with BeforeAndAfter { +class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { - val masterPage = mock(classOf[MasterPage]) - val master = { - val conf = new SparkConf - val securityMgr = new SecurityManager(conf) - val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, securityMgr) - val master = new Master(rpcEnv, rpcEnv.address, 0, securityMgr, conf) - master - } - val masterWebUI = new MasterWebUI(master, 0, customMasterPage = Some(masterPage)) + val conf = new SparkConf + val securityMgr = new SecurityManager(conf) + val rpcEnv = mock(classOf[RpcEnv]) + val master = mock(classOf[Master]) + val masterEndpointRef = mock(classOf[RpcEndpointRef]) + when(master.securityMgr).thenReturn(securityMgr) + when(master.conf).thenReturn(conf) + when(master.rpcEnv).thenReturn(rpcEnv) + when(master.self).thenReturn(masterEndpointRef) + val masterWebUI = new MasterWebUI(master, 0) - before { + override def beforeAll() { + super.beforeAll() masterWebUI.bind() } - after { + override def afterAll() { masterWebUI.stop() + super.afterAll() } - test("list applications") { - val worker = createWorkerInfo() + test("kill application") { val appDesc = createAppDesc() // use new start date so it isn't filtered by UI val activeApp = new ApplicationInfo( - new Date().getTime, "id", appDesc, new Date(), null, Int.MaxValue) - activeApp.addExecutor(worker, 2) - - val workers = Array[WorkerInfo](worker) - val activeApps = Array(activeApp) - val completedApps = Array[ApplicationInfo]() - val activeDrivers = Array[DriverInfo]() - val completedDrivers = Array[DriverInfo]() - val stateResponse = new MasterStateResponse( - "host", 8080, None, workers, activeApps, completedApps, - activeDrivers, completedDrivers, RecoveryState.ALIVE) - - when(masterPage.getMasterState).thenReturn(stateResponse) - - val resultJson = Source.fromURL( - s"http://localhost:${masterWebUI.boundPort}/api/v1/applications") - .mkString - val parsedJson = parse(resultJson) - val firstApp = parsedJson(0) - - assert(firstApp \ "id" === JString(activeApp.id)) - assert(firstApp \ "name" === JString(activeApp.desc.name)) - assert(firstApp \ "coresGranted" === JInt(2)) - assert(firstApp \ "maxCores" === JInt(4)) - assert(firstApp \ "memoryPerExecutorMB" === JInt(1234)) - assert(firstApp \ "coresPerExecutor" === JNothing) + new Date().getTime, "app-0", appDesc, new Date(), null, Int.MaxValue) + + when(master.idToApp).thenReturn(HashMap[String, ApplicationInfo]((activeApp.id, activeApp))) + + val url = s"http://localhost:${masterWebUI.boundPort}/app/kill/" + val body = convPostDataToString(Map(("id", activeApp.id), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify the master was called to remove the active app + verify(master, times(1)).removeApplication(activeApp, ApplicationState.KILLED) + } + + test("kill driver") { + val activeDriverId = "driver-0" + val url = s"http://localhost:${masterWebUI.boundPort}/driver/kill/" + val body = convPostDataToString(Map(("id", activeDriverId), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify that master was asked to kill driver with the correct id + verify(masterEndpointRef, times(1)).ask[KillDriverResponse](RequestKillDriver(activeDriverId)) } + private def convPostDataToString(data: Map[String, String]): String = { + (for ((name, value) <- data) yield s"$name=$value").mkString("&") + } + + /** + * Send an HTTP request to the given URL using the method and the body specified. + * Return the connection object. + */ + private def sendHttpRequest( + url: String, + method: String, + body: String = ""): HttpURLConnection = { + val conn = new URL(url).openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod(method) + if (body.nonEmpty) { + conn.setDoOutput(true) + conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded") + conn.setRequestProperty("Content-Length", Integer.toString(body.length)) + val out = new DataOutputStream(conn.getOutputStream) + out.write(body.getBytes(StandardCharsets.UTF_8)) + out.close() + } + conn + } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 697962ae3a4ce..78a3470abf40f 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -27,11 +27,6 @@ in the UI to persisted storage. ## Viewing After the Fact -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over -the course of its lifetime, then the Standalone master's web UI will automatically re-render the -application's UI after the application has finished. - If Spark is run on Mesos or YARN, it is still possible to construct the UI of an application through Spark's history server, provided that the application's event logs exist. You can start the history server by executing: From 23789e358589505121113d504adee97e56c00929 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 4 May 2016 14:31:36 -0700 Subject: [PATCH 0069/1470] [SPARK-15031][EXAMPLE] Use SparkSession in Scala/Python/Java example. ## What changes were proposed in this pull request? This PR aims to update Scala/Python/Java examples by replacing `SQLContext` with newly added `SparkSession`. - Use **SparkSession Builder Pattern** in 154(Scala 55, Java 52, Python 47) files. - Add `getConf` in Python SparkContext class: `python/pyspark/context.py` - Replace **SQLContext Singleton Pattern** with **SparkSession Singleton Pattern**: - `SqlNetworkWordCount.scala` - `JavaSqlNetworkWordCount.java` - `sql_network_wordcount.py` Now, `SQLContexts` are used only in R examples and the following two Python examples. The python examples are untouched in this PR since it already fails some unknown issue. - `simple_params_example.py` - `aft_survival_regression.py` ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #12809 from dongjoon-hyun/SPARK-15031. (cherry picked from commit cdce4e62a5674e2034e5d395578b1a60e3d8c435) Signed-off-by: Andrew Or --- .../ml/JavaAFTSurvivalRegressionExample.java | 12 ++--- .../spark/examples/ml/JavaALSExample.java | 15 +++--- .../examples/ml/JavaBinarizerExample.java | 15 +++--- .../ml/JavaBisectingKMeansExample.java | 18 +++---- .../examples/ml/JavaBucketizerExample.java | 18 +++---- .../examples/ml/JavaChiSqSelectorExample.java | 15 +++--- .../ml/JavaCountVectorizerExample.java | 19 +++---- .../spark/examples/ml/JavaDCTExample.java | 15 +++--- ...JavaDecisionTreeClassificationExample.java | 13 ++--- .../ml/JavaDecisionTreeRegressionExample.java | 13 ++--- .../examples/ml/JavaDeveloperApiExample.java | 15 ++---- .../ml/JavaElementwiseProductExample.java | 15 +++--- .../JavaEstimatorTransformerParamExample.java | 16 +++--- ...aGradientBoostedTreeClassifierExample.java | 11 ++--- ...vaGradientBoostedTreeRegressorExample.java | 14 ++---- .../examples/ml/JavaIndexToStringExample.java | 18 +++---- .../spark/examples/ml/JavaKMeansExample.java | 14 ++---- .../spark/examples/ml/JavaLDAExample.java | 14 ++---- ...LinearRegressionWithElasticNetExample.java | 13 ++--- .../JavaLogisticRegressionSummaryExample.java | 13 ++--- ...gisticRegressionWithElasticNetExample.java | 13 ++--- .../examples/ml/JavaMaxAbsScalerExample.java | 12 ++--- .../examples/ml/JavaMinMaxScalerExample.java | 12 ++--- ...delSelectionViaCrossValidationExample.java | 16 +++--- ...lectionViaTrainValidationSplitExample.java | 14 ++---- ...MultilayerPerceptronClassifierExample.java | 13 ++--- .../spark/examples/ml/JavaNGramExample.java | 18 +++---- .../examples/ml/JavaNaiveBayesExample.java | 14 ++---- .../examples/ml/JavaNormalizerExample.java | 13 ++--- .../examples/ml/JavaOneHotEncoderExample.java | 18 +++---- .../examples/ml/JavaOneVsRestExample.java | 14 ++---- .../spark/examples/ml/JavaPCAExample.java | 18 +++---- .../examples/ml/JavaPipelineExample.java | 16 ++---- .../ml/JavaPolynomialExpansionExample.java | 17 +++---- .../ml/JavaQuantileDiscretizerExample.java | 29 +++++------ .../examples/ml/JavaRFormulaExample.java | 18 +++---- .../ml/JavaRandomForestClassifierExample.java | 14 ++---- .../ml/JavaRandomForestRegressorExample.java | 14 ++---- .../ml/JavaSQLTransformerExample.java | 19 +++---- .../examples/ml/JavaSimpleParamsExample.java | 14 ++---- .../JavaSimpleTextClassificationPipeline.java | 15 +++--- .../ml/JavaStandardScalerExample.java | 13 ++--- .../ml/JavaStopWordsRemoverExample.java | 18 +++---- .../examples/ml/JavaStringIndexerExample.java | 18 +++---- .../spark/examples/ml/JavaTfIdfExample.java | 18 +++---- .../examples/ml/JavaTokenizerExample.java | 18 +++---- .../ml/JavaVectorAssemblerExample.java | 14 ++---- .../examples/ml/JavaVectorIndexerExample.java | 12 ++--- .../examples/ml/JavaVectorSlicerExample.java | 19 +++---- .../examples/ml/JavaWord2VecExample.java | 19 +++---- .../spark/examples/sql/JavaSparkSQL.java | 33 ++++++------- .../streaming/JavaSqlNetworkWordCount.java | 19 ++++--- examples/src/main/python/ml/als_example.py | 14 +++--- .../src/main/python/ml/binarizer_example.py | 10 ++-- .../python/ml/bisecting_k_means_example.py | 16 +++--- .../src/main/python/ml/bucketizer_example.py | 10 ++-- .../main/python/ml/chisq_selector_example.py | 10 ++-- .../python/ml/count_vectorizer_example.py | 10 ++-- .../src/main/python/ml/cross_validator.py | 49 +++++++++---------- .../src/main/python/ml/dataframe_example.py | 14 +++--- examples/src/main/python/ml/dct_example.py | 10 ++-- .../decision_tree_classification_example.py | 9 ++-- .../ml/decision_tree_regression_example.py | 9 ++-- .../python/ml/elementwise_product_example.py | 10 ++-- .../ml/estimator_transformer_param_example.py | 13 +++-- ...radient_boosted_tree_classifier_example.py | 9 ++-- ...gradient_boosted_tree_regressor_example.py | 9 ++-- .../main/python/ml/index_to_string_example.py | 10 ++-- examples/src/main/python/ml/kmeans_example.py | 16 +++--- .../ml/linear_regression_with_elastic_net.py | 10 ++-- .../logistic_regression_with_elastic_net.py | 10 ++-- .../main/python/ml/max_abs_scaler_example.py | 10 ++-- .../main/python/ml/min_max_scaler_example.py | 10 ++-- .../multilayer_perceptron_classification.py | 12 ++--- examples/src/main/python/ml/n_gram_example.py | 10 ++-- .../src/main/python/ml/naive_bayes_example.py | 11 ++--- .../src/main/python/ml/normalizer_example.py | 10 ++-- .../main/python/ml/onehot_encoder_example.py | 10 ++-- examples/src/main/python/ml/pca_example.py | 10 ++-- .../src/main/python/ml/pipeline_example.py | 13 +++-- .../python/ml/polynomial_expansion_example.py | 10 ++-- .../ml/random_forest_classifier_example.py | 9 ++-- .../ml/random_forest_regressor_example.py | 9 ++-- .../src/main/python/ml/rformula_example.py | 10 ++-- .../ml/simple_text_classification_pipeline.py | 32 ++++++------ .../src/main/python/ml/sql_transformer.py | 10 ++-- .../main/python/ml/standard_scaler_example.py | 10 ++-- .../python/ml/stopwords_remover_example.py | 10 ++-- .../main/python/ml/string_indexer_example.py | 10 ++-- examples/src/main/python/ml/tf_idf_example.py | 10 ++-- .../src/main/python/ml/tokenizer_example.py | 10 ++-- .../main/python/ml/train_validation_split.py | 10 ++-- .../python/ml/vector_assembler_example.py | 10 ++-- .../main/python/ml/vector_indexer_example.py | 10 ++-- .../main/python/ml/vector_slicer_example.py | 10 ++-- .../src/main/python/ml/word2vec_example.py | 10 ++-- .../binary_classification_metrics_example.py | 6 ++- examples/src/main/python/sql.py | 2 +- .../python/streaming/sql_network_wordcount.py | 19 +++---- .../ml/AFTSurvivalRegressionExample.scala | 11 ++--- .../apache/spark/examples/ml/ALSExample.scala | 14 +++--- .../spark/examples/ml/BinarizerExample.scala | 12 ++--- .../spark/examples/ml/BucketizerExample.scala | 11 ++--- .../examples/ml/ChiSqSelectorExample.scala | 14 ++---- .../examples/ml/CountVectorizerExample.scala | 11 ++--- .../apache/spark/examples/ml/DCTExample.scala | 12 ++--- .../spark/examples/ml/DataFrameExample.scala | 14 ++---- .../DecisionTreeClassificationExample.scala | 11 ++--- .../examples/ml/DecisionTreeExample.scala | 18 +++---- .../ml/DecisionTreeRegressionExample.scala | 11 ++--- .../examples/ml/DeveloperApiExample.scala | 17 +++---- .../ml/ElementwiseProductExample.scala | 12 ++--- .../ml/EstimatorTransformerParamExample.scala | 13 ++--- ...GradientBoostedTreeClassifierExample.scala | 11 ++--- .../GradientBoostedTreeRegressorExample.scala | 11 ++--- .../examples/ml/IndexToStringExample.scala | 13 ++--- .../spark/examples/ml/KMeansExample.scala | 11 ++--- .../apache/spark/examples/ml/LDAExample.scala | 13 ++--- ...inearRegressionWithElasticNetExample.scala | 11 ++--- .../ml/LogisticRegressionSummaryExample.scala | 13 ++--- ...isticRegressionWithElasticNetExample.scala | 12 ++--- .../examples/ml/MaxAbsScalerExample.scala | 14 ++---- .../examples/ml/MinMaxScalerExample.scala | 12 ++--- ...elSelectionViaCrossValidationExample.scala | 14 +++--- ...ectionViaTrainValidationSplitExample.scala | 12 ++--- ...ultilayerPerceptronClassifierExample.scala | 11 ++--- .../spark/examples/ml/NGramExample.scala | 12 ++--- .../spark/examples/ml/NaiveBayesExample.scala | 13 +++-- .../spark/examples/ml/NormalizerExample.scala | 12 ++--- .../examples/ml/OneHotEncoderExample.scala | 12 ++--- .../spark/examples/ml/OneVsRestExample.scala | 13 ++--- .../apache/spark/examples/ml/PCAExample.scala | 12 ++--- .../spark/examples/ml/PipelineExample.scala | 13 ++--- .../ml/PolynomialExpansionExample.scala | 12 ++--- .../ml/QuantileDiscretizerExample.scala | 16 +++--- .../spark/examples/ml/RFormulaExample.scala | 12 ++--- .../ml/RandomForestClassifierExample.scala | 11 ++--- .../ml/RandomForestRegressorExample.scala | 11 ++--- .../examples/ml/SQLTransformerExample.scala | 11 ++--- .../examples/ml/SimpleParamsExample.scala | 19 +++---- .../ml/SimpleTextClassificationPipeline.scala | 15 +++--- .../examples/ml/StandardScalerExample.scala | 12 ++--- .../examples/ml/StopWordsRemoverExample.scala | 12 ++--- .../examples/ml/StringIndexerExample.scala | 12 ++--- .../spark/examples/ml/TfIdfExample.scala | 11 ++--- .../spark/examples/ml/TokenizerExample.scala | 12 ++--- .../examples/ml/VectorAssemblerExample.scala | 12 ++--- .../examples/ml/VectorIndexerExample.scala | 12 ++--- .../examples/ml/VectorSlicerExample.scala | 17 +++---- .../spark/examples/ml/Word2VecExample.scala | 11 ++--- .../spark/examples/mllib/LDAExample.scala | 6 +-- .../mllib/RankingMetricsExample.scala | 11 ++--- .../mllib/RegressionMetricsExample.scala | 18 ++++--- .../streaming/SqlNetworkWordCount.scala | 21 ++++---- python/pyspark/context.py | 5 ++ 155 files changed, 852 insertions(+), 1232 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java index 22b93a3a85c52..ecb7084e03122 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java @@ -21,23 +21,19 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.regression.AFTSurvivalRegression; import org.apache.spark.ml.regression.AFTSurvivalRegressionModel; import org.apache.spark.mllib.linalg.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaAFTSurvivalRegressionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaAFTSurvivalRegressionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaAFTSurvivalRegressionExample").getOrCreate(); // $example on$ List data = Arrays.asList( @@ -52,7 +48,7 @@ public static void main(String[] args) { new StructField("censor", DataTypes.DoubleType, false, Metadata.empty()), new StructField("features", new VectorUDT(), false, Metadata.empty()) }); - Dataset training = jsql.createDataFrame(data, schema); + Dataset training = spark.createDataFrame(data, schema); double[] quantileProbabilities = new double[]{0.3, 0.6}; AFTSurvivalRegression aft = new AFTSurvivalRegression() .setQuantileProbabilities(quantileProbabilities) @@ -66,6 +62,6 @@ public static void main(String[] args) { model.transform(training).show(false); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 088037d427f5b..9a9a10489b506 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -17,11 +17,9 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.io.Serializable; @@ -83,18 +81,17 @@ public static Rating parseRating(String str) { // $example off$ public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaALSExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaALSExample").getOrCreate(); // $example on$ - JavaRDD ratingsRDD = jsc.textFile("data/mllib/als/sample_movielens_ratings.txt") + JavaRDD ratingsRDD = spark + .read().text("data/mllib/als/sample_movielens_ratings.txt").javaRDD() .map(new Function() { public Rating call(String str) { return Rating.parseRating(str); } }); - Dataset ratings = sqlContext.createDataFrame(ratingsRDD, Rating.class); + Dataset ratings = spark.createDataFrame(ratingsRDD, Rating.class); Dataset[] splits = ratings.randomSplit(new double[]{0.8, 0.2}); Dataset training = splits[0]; Dataset test = splits[1]; @@ -121,6 +118,6 @@ public Rating call(String str) { Double rmse = evaluator.evaluate(predictions); System.out.println("Root-mean-square error = " + rmse); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java index 0a6e9c2a1f93c..88e4298a6191e 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -20,10 +20,11 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Binarizer; @@ -37,21 +38,19 @@ public class JavaBinarizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBinarizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaBinarizerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, 0.1), RowFactory.create(1, 0.8), RowFactory.create(2, 0.2) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset continuousDataFrame = jsql.createDataFrame(jrdd, schema); + Dataset continuousDataFrame = spark.createDataFrame(data, schema); Binarizer binarizer = new Binarizer() .setInputCol("feature") .setOutputCol("binarized_feature") @@ -63,6 +62,6 @@ public static void main(String[] args) { System.out.println(binarized_value); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 1d1a518bbca12..51aa35084e845 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -18,12 +18,10 @@ package org.apache.spark.examples.ml; import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; @@ -44,25 +42,23 @@ public class JavaBisectingKMeansExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBisectingKMeansExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaBisectingKMeansExample").getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(0.1, 0.1, 0.1)), RowFactory.create(Vectors.dense(0.3, 0.3, 0.25)), RowFactory.create(Vectors.dense(0.1, 0.1, -0.1)), RowFactory.create(Vectors.dense(20.3, 20.1, 19.9)), RowFactory.create(Vectors.dense(20.2, 20.1, 19.7)), RowFactory.create(Vectors.dense(18.9, 20.0, 19.7)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); BisectingKMeans bkm = new BisectingKMeans().setK(2); BisectingKMeansModel model = bkm.fit(dataset); @@ -76,6 +72,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java index 68ffa702ea5e2..0c24f52cf58ab 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Bucketizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,23 +35,21 @@ public class JavaBucketizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaBucketizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaBucketizerExample").getOrCreate(); // $example on$ double[] splits = {Double.NEGATIVE_INFINITY, -0.5, 0.0, 0.5, Double.POSITIVE_INFINITY}; - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(-0.5), RowFactory.create(-0.3), RowFactory.create(0.0), RowFactory.create(0.2) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset dataFrame = jsql.createDataFrame(data, schema); + Dataset dataFrame = spark.createDataFrame(data, schema); Bucketizer bucketizer = new Bucketizer() .setInputCol("features") @@ -64,7 +60,7 @@ public static void main(String[] args) { Dataset bucketedData = bucketizer.transform(dataFrame); bucketedData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java index b1bf1cfeb2153..684cf9a7145d2 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java @@ -21,10 +21,11 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.ml.feature.ChiSqSelector; import org.apache.spark.mllib.linalg.VectorUDT; @@ -39,23 +40,21 @@ public class JavaChiSqSelectorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaChiSqSelectorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaChiSqSelectorExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(7, Vectors.dense(0.0, 0.0, 18.0, 1.0), 1.0), RowFactory.create(8, Vectors.dense(0.0, 1.0, 12.0, 0.0), 0.0), RowFactory.create(9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("features", new VectorUDT(), false, Metadata.empty()), new StructField("clicked", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); ChiSqSelector selector = new ChiSqSelector() .setNumTopFeatures(1) @@ -66,6 +65,6 @@ public static void main(String[] args) { Dataset result = selector.fit(df).transform(df); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java index ec3ac202bea4e..0631f9d6d5d74 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java @@ -19,36 +19,31 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.CountVectorizer; import org.apache.spark.ml.feature.CountVectorizerModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaCountVectorizerExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaCountVectorizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaCountVectorizerExample").getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("a", "b", "c")), RowFactory.create(Arrays.asList("a", "b", "b", "c", "a")) - )); + ); StructType schema = new StructType(new StructField [] { new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); // fit a CountVectorizerModel from the corpus CountVectorizerModel cvModel = new CountVectorizer() @@ -66,6 +61,6 @@ public static void main(String[] args) { cvModel.transform(df).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java index 4b15fde9c35fa..ec57a2445138a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java @@ -20,10 +20,11 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.DCT; @@ -38,20 +39,18 @@ public class JavaDCTExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDCTExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaDCTExample").getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(0.0, 1.0, -2.0, 3.0)), RowFactory.create(Vectors.dense(-1.0, 2.0, 4.0, -7.0)), RowFactory.create(Vectors.dense(14.0, -2.0, -5.0, 1.0)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); DCT dct = new DCT() .setInputCol("features") .setOutputCol("featuresDCT") @@ -59,7 +58,7 @@ public static void main(String[] args) { Dataset dctDf = dct.transform(df); dctDf.select("featuresDCT").show(3); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java index 8214952f80695..733bc4181c70b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java @@ -17,8 +17,6 @@ // scalastyle:off println package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -28,18 +26,17 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaDecisionTreeClassificationExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDecisionTreeClassificationExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaDecisionTreeClassificationExample").getOrCreate(); // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - Dataset data = sqlContext + Dataset data = spark .read() .format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); @@ -100,6 +97,6 @@ public static void main(String[] args) { System.out.println("Learned classification tree model:\n" + treeModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java index a4f3e97bf318a..bd6dc3edd3631 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java @@ -17,8 +17,6 @@ // scalastyle:off println package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -29,17 +27,16 @@ import org.apache.spark.ml.regression.DecisionTreeRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaDecisionTreeRegressionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaDecisionTreeRegressionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaDecisionTreeRegressionExample").getOrCreate(); // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - Dataset data = sqlContext.read().format("libsvm") + Dataset data = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. @@ -85,6 +82,6 @@ public static void main(String[] args) { System.out.println("Learned regression tree model:\n" + treeModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } 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 0ba94786d4e5f..90023ac06b79b 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 @@ -21,9 +21,7 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.Classifier; import org.apache.spark.ml.classification.ClassificationModel; import org.apache.spark.ml.param.IntParam; @@ -35,7 +33,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** @@ -51,9 +49,7 @@ public class JavaDeveloperApiExample { public static void main(String[] args) throws Exception { - SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaDeveloperApiExample").getOrCreate(); // Prepare training data. List localTraining = Lists.newArrayList( @@ -61,8 +57,7 @@ public static void main(String[] args) throws Exception { new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - Dataset training = jsql.createDataFrame( - jsc.parallelize(localTraining), LabeledPoint.class); + Dataset training = spark.createDataFrame(localTraining, LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); @@ -80,7 +75,7 @@ public static void main(String[] args) throws Exception { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); + Dataset test = spark.createDataFrame(localTest, LabeledPoint.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). Dataset results = model.transform(test); @@ -93,7 +88,7 @@ public static void main(String[] args) throws Exception { " even though all coefficients are 0!"); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java index 37de9cf3596a9..a062a6fcd0c19 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -20,7 +20,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.ArrayList; @@ -41,16 +41,15 @@ public class JavaElementwiseProductExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaElementwiseProductExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaElementwiseProductExample").getOrCreate(); // $example on$ // Create some vector data; also works for sparse vectors - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create("a", Vectors.dense(1.0, 2.0, 3.0)), RowFactory.create("b", Vectors.dense(4.0, 5.0, 6.0)) - )); + ); List fields = new ArrayList<>(2); fields.add(DataTypes.createStructField("id", DataTypes.StringType, false)); @@ -58,7 +57,7 @@ public static void main(String[] args) { StructType schema = DataTypes.createStructType(fields); - Dataset dataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset dataFrame = spark.createDataFrame(data, schema); Vector transformingVector = Vectors.dense(0.0, 1.0, 2.0); @@ -70,6 +69,6 @@ public static void main(String[] args) { // Batch transform the vectors to create new column: transformer.transform(dataFrame).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java index 604b193dd489b..5ba8e6cf44902 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java @@ -21,8 +21,6 @@ import java.util.Arrays; // $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; // $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.LogisticRegressionModel; @@ -32,23 +30,21 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for Estimator, Transformer, and Param. */ public class JavaEstimatorTransformerParamExample { public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("JavaEstimatorTransformerParamExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession + .builder().appName("JavaEstimatorTransformerParamExample").getOrCreate(); // $example on$ // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans into // DataFrames, where it uses the bean metadata to infer the schema. - Dataset training = sqlContext.createDataFrame( + Dataset training = spark.createDataFrame( Arrays.asList( new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), @@ -89,7 +85,7 @@ public static void main(String[] args) { System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); // Prepare test documents. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)) @@ -107,6 +103,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java index 553070dace882..a7c89b9d19974 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -29,18 +29,17 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaGradientBoostedTreeClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaGradientBoostedTreeClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaGradientBoostedTreeClassifierExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = sqlContext.read().format("libsvm") + Dataset data = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. @@ -99,6 +98,6 @@ public static void main(String[] args) { System.out.println("Learned classification GBT model:\n" + gbtModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java index 83fd89e3bd59b..6d3f21fdafa04 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -30,19 +28,17 @@ import org.apache.spark.ml.regression.GBTRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaGradientBoostedTreeRegressorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaGradientBoostedTreeRegressorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaGradientBoostedTreeRegressorExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -87,6 +83,6 @@ public static void main(String[] args) { System.out.println("Learned regression GBT model:\n" + gbtModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java index 9b8c22f3bdfde..ccd74f2920ce3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.ml.feature.IndexToString; import org.apache.spark.ml.feature.StringIndexer; @@ -39,24 +37,22 @@ public class JavaIndexToStringExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaIndexToStringExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaIndexToStringExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("category", DataTypes.StringType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexerModel indexer = new StringIndexer() .setInputCol("category") @@ -70,6 +66,6 @@ public static void main(String[] args) { Dataset converted = converter.transform(indexed); converted.select("id", "originalCategory").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index c5022f4c0b8fe..e6d82a0513a30 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -19,12 +19,10 @@ import java.util.regex.Pattern; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.expressions.GenericRow; // $example on$ import org.apache.spark.ml.clustering.KMeansModel; @@ -72,16 +70,14 @@ public static void main(String[] args) { int k = Integer.parseInt(args[1]); // Parses the arguments - SparkConf conf = new SparkConf().setAppName("JavaKMeansExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaKMeansExample").getOrCreate(); // $example on$ // Loads data - JavaRDD points = jsc.textFile(inputFile).map(new ParsePoint()); + JavaRDD points = spark.read().text(inputFile).javaRDD().map(new ParsePoint()); StructField[] fields = {new StructField("features", new VectorUDT(), false, Metadata.empty())}; StructType schema = new StructType(fields); - Dataset dataset = sqlContext.createDataFrame(points, schema); + Dataset dataset = spark.createDataFrame(points, schema); // Trains a k-means model KMeans kmeans = new KMeans() @@ -96,6 +92,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java index 351bc401180cc..b8baca5920e8d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java @@ -19,9 +19,7 @@ // $example on$ import java.util.regex.Pattern; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.ml.clustering.LDA; import org.apache.spark.ml.clustering.LDAModel; @@ -30,7 +28,7 @@ import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.expressions.GenericRow; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -67,15 +65,13 @@ public static void main(String[] args) { String inputFile = "data/mllib/sample_lda_data.txt"; // Parses the arguments - SparkConf conf = new SparkConf().setAppName("JavaLDAExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaLDAExample").getOrCreate(); // Loads data - JavaRDD points = jsc.textFile(inputFile).map(new ParseVector()); + JavaRDD points = spark.read().text(inputFile).javaRDD().map(new ParseVector()); StructField[] fields = {new StructField("features", new VectorUDT(), false, Metadata.empty())}; StructType schema = new StructType(fields); - Dataset dataset = sqlContext.createDataFrame(points, schema); + Dataset dataset = spark.createDataFrame(points, schema); // Trains a LDA model LDA lda = new LDA() @@ -91,7 +87,7 @@ public static void main(String[] args) { topics.show(false); model.transform(dataset).show(false); - jsc.stop(); + spark.stop(); } // $example off$ } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java index 08fce89359fc5..b6ea1fed25f87 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.regression.LinearRegression; import org.apache.spark.ml.regression.LinearRegressionModel; @@ -26,18 +24,17 @@ import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaLinearRegressionWithElasticNetExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaLinearRegressionWithElasticNetExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaLinearRegressionWithElasticNetExample").getOrCreate(); // $example on$ // Load training data - Dataset training = sqlContext.read().format("libsvm") + Dataset training = spark.read().format("libsvm") .load("data/mllib/sample_linear_regression_data.txt"); LinearRegression lr = new LinearRegression() @@ -61,6 +58,6 @@ public static void main(String[] args) { System.out.println("r2: " + trainingSummary.r2()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java index 73b028fb44409..fd040aead4101 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary; import org.apache.spark.ml.classification.LogisticRegression; @@ -26,18 +24,17 @@ import org.apache.spark.ml.classification.LogisticRegressionTrainingSummary; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.functions; // $example off$ public class JavaLogisticRegressionSummaryExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaLogisticRegressionSummaryExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaLogisticRegressionSummaryExample").getOrCreate(); // Load training data - Dataset training = sqlContext.read().format("libsvm") + Dataset training = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); LogisticRegression lr = new LogisticRegression() @@ -80,6 +77,6 @@ public static void main(String[] args) { lrModel.setThreshold(bestThreshold); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java index 691166852206c..f00c7a05cd12a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java @@ -17,25 +17,22 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.LogisticRegressionModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaLogisticRegressionWithElasticNetExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaLogisticRegressionWithElasticNetExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaLogisticRegressionWithElasticNetExample").getOrCreate(); // $example on$ // Load training data - Dataset training = sqlContext.read().format("libsvm") + Dataset training = spark.read().format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); LogisticRegression lr = new LogisticRegression() @@ -51,6 +48,6 @@ public static void main(String[] args) { + lrModel.coefficients() + " Intercept: " + lrModel.intercept()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java index a2a072b253f39..80cdd364b9376 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java @@ -17,25 +17,21 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.feature.MaxAbsScaler; import org.apache.spark.ml.feature.MaxAbsScalerModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; public class JavaMaxAbsScalerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaMaxAbsScalerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaMaxAbsScalerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); MaxAbsScaler scaler = new MaxAbsScaler() .setInputCol("features") .setOutputCol("scaledFeatures"); @@ -47,7 +43,7 @@ public static void main(String[] args) { Dataset scaledData = scalerModel.transform(dataFrame); scaledData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java index 4aee18eeabfcf..022940fd1e67c 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.feature.MinMaxScaler; @@ -30,12 +28,10 @@ public class JavaMinMaxScalerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JaveMinMaxScalerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaMinMaxScalerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); MinMaxScaler scaler = new MinMaxScaler() .setInputCol("features") .setOutputCol("scaledFeatures"); @@ -47,6 +43,6 @@ public static void main(String[] args) { Dataset scaledData = scalerModel.transform(dataFrame); scaledData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java index c4122d1247a94..a4ec4f58154f6 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java @@ -21,8 +21,6 @@ import java.util.Arrays; // $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineStage; @@ -37,21 +35,19 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for Model Selection via Cross Validation. */ public class JavaModelSelectionViaCrossValidationExample { public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("JavaModelSelectionViaCrossValidationExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession + .builder().appName("JavaModelSelectionViaCrossValidationExample").getOrCreate(); // $example on$ // Prepare training documents, which are labeled. - Dataset training = sqlContext.createDataFrame(Arrays.asList( + Dataset training = spark.createDataFrame(Arrays.asList( new JavaLabeledDocument(0L, "a b c d e spark", 1.0), new JavaLabeledDocument(1L, "b d", 0.0), new JavaLabeledDocument(2L,"spark f g h", 1.0), @@ -102,7 +98,7 @@ public static void main(String[] args) { CrossValidatorModel cvModel = cv.fit(training); // Prepare test documents, which are unlabeled. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new JavaDocument(4L, "spark i j k"), new JavaDocument(5L, "l m n"), new JavaDocument(6L, "mapreduce spark"), @@ -117,6 +113,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java index 4994f8f9fa857..63a0ad1cb883c 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator; import org.apache.spark.ml.param.ParamMap; @@ -29,7 +27,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example demonstrating model selection using TrainValidationSplit. @@ -44,13 +42,11 @@ */ public class JavaModelSelectionViaTrainValidationSplitExample { public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("JavaModelSelectionViaTrainValidationSplitExample"); - SparkContext sc = new SparkContext(conf); - SQLContext jsql = new SQLContext(sc); + SparkSession spark = SparkSession + .builder().appName("JavaModelSelectionViaTrainValidationSplitExample").getOrCreate(); // $example on$ - Dataset data = jsql.read().format("libsvm") + Dataset data = spark.read().format("libsvm") .load("data/mllib/sample_linear_regression_data.txt"); // Prepare training and test data. @@ -87,6 +83,6 @@ public static void main(String[] args) { .show(); // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java index 0ca528d8cd079..d547a2a64be5b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java @@ -18,11 +18,9 @@ package org.apache.spark.examples.ml; // $example on$ -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel; import org.apache.spark.ml.classification.MultilayerPerceptronClassifier; import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; @@ -34,14 +32,13 @@ public class JavaMultilayerPerceptronClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaMultilayerPerceptronClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaMultilayerPerceptronClassifierExample").getOrCreate(); // $example on$ // Load training data String path = "data/mllib/sample_multiclass_classification_data.txt"; - Dataset dataFrame = jsql.read().format("libsvm").load(path); + Dataset dataFrame = spark.read().format("libsvm").load(path); // Split the data into train and test Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L); Dataset train = splits[0]; @@ -66,6 +63,6 @@ public static void main(String[] args) { System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels)); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java index 608bd80285655..325b7b5874800 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java @@ -17,15 +17,13 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.NGram; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -37,16 +35,14 @@ public class JavaNGramExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaNGramExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaNGramExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0.0, Arrays.asList("Hi", "I", "heard", "about", "Spark")), RowFactory.create(1.0, Arrays.asList("I", "wish", "Java", "could", "use", "case", "classes")), RowFactory.create(2.0, Arrays.asList("Logistic", "regression", "models", "are", "neat")) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), @@ -54,7 +50,7 @@ public static void main(String[] args) { "words", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) }); - Dataset wordDataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset wordDataFrame = spark.createDataFrame(data, schema); NGram ngramTransformer = new NGram().setInputCol("words").setOutputCol("ngrams"); @@ -66,6 +62,6 @@ public static void main(String[] args) { System.out.println(); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java index 41d7ad75b9d45..1f24a23609ebe 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java @@ -17,16 +17,13 @@ package org.apache.spark.examples.ml; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.NaiveBayes; import org.apache.spark.ml.classification.NaiveBayesModel; import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ /** @@ -35,13 +32,12 @@ public class JavaNaiveBayesExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaNaiveBayesExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaNaiveBayesExample").getOrCreate(); // $example on$ // Load training data - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Split the data into train and test Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L); Dataset train = splits[0]; @@ -59,6 +55,6 @@ public static void main(String[] args) { System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels)); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java index 31cd752136689..4b3a718ea92c8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.feature.Normalizer; @@ -29,12 +27,11 @@ public class JavaNormalizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaNormalizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaNormalizerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Normalize each Vector using $L^1$ norm. Normalizer normalizer = new Normalizer() @@ -50,6 +47,6 @@ public static void main(String[] args) { normalizer.transform(dataFrame, normalizer.p().w(Double.POSITIVE_INFINITY)); lInfNormData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java index 882438ca28eb7..d6e4d21ead161 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.OneHotEncoder; import org.apache.spark.ml.feature.StringIndexer; import org.apache.spark.ml.feature.StringIndexerModel; @@ -39,26 +37,24 @@ public class JavaOneHotEncoderExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaOneHotEncoderExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaOneHotEncoderExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("id", DataTypes.DoubleType, false, Metadata.empty()), new StructField("category", DataTypes.StringType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexerModel indexer = new StringIndexer() .setInputCol("category") @@ -72,7 +68,7 @@ public static void main(String[] args) { Dataset encoded = encoder.transform(indexed); encoded.select("id", "categoryVec").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java index 1f13b48bf82ae..9cc983bd115df 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java @@ -19,8 +19,6 @@ import org.apache.commons.cli.*; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.OneVsRest; @@ -31,7 +29,7 @@ import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructField; // $example off$ @@ -60,9 +58,7 @@ private static class Params { public static void main(String[] args) { // parse the arguments Params params = parse(args); - SparkConf conf = new SparkConf().setAppName("JavaOneVsRestExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaOneVsRestExample").getOrCreate(); // $example on$ // configure the base classifier @@ -82,7 +78,7 @@ public static void main(String[] args) { OneVsRest ovr = new OneVsRest().setClassifier(classifier); String input = params.input; - Dataset inputData = jsql.read().format("libsvm").load(input); + Dataset inputData = spark.read().format("libsvm").load(input); Dataset train; Dataset test; @@ -92,7 +88,7 @@ public static void main(String[] args) { train = inputData; // compute the number of features in the training set. int numFeatures = inputData.first().getAs(1).size(); - test = jsql.read().format("libsvm").option("numFeatures", + test = spark.read().format("libsvm").option("numFeatures", String.valueOf(numFeatures)).load(testInput); } else { double f = params.fracTest; @@ -131,7 +127,7 @@ public static void main(String[] args) { System.out.println(results); // $example off$ - jsc.stop(); + spark.stop(); } private static Params parse(String[] args) { diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java index a792fd7d47cc9..6b1dcb68ba304 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.PCA; import org.apache.spark.ml.feature.PCAModel; import org.apache.spark.mllib.linalg.VectorUDT; @@ -39,22 +37,20 @@ public class JavaPCAExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPCAExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaPCAExample").getOrCreate(); // $example on$ - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.sparse(5, new int[]{1, 3}, new double[]{1.0, 7.0})), RowFactory.create(Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0)), RowFactory.create(Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); PCAModel pca = new PCA() .setInputCol("features") @@ -65,7 +61,7 @@ public static void main(String[] args) { Dataset result = pca.transform(df).select("pcaFeatures"); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java index 305420f208b79..556a4573263f1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java @@ -19,11 +19,7 @@ // $example on$ import java.util.Arrays; -// $example off$ -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; -// $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -33,20 +29,18 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * Java example for simple text document 'Pipeline'. */ public class JavaPipelineExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPipelineExample"); - SparkContext sc = new SparkContext(conf); - SQLContext sqlContext = new SQLContext(sc); + SparkSession spark = SparkSession.builder().appName("JavaPipelineExample").getOrCreate(); // $example on$ // Prepare training documents, which are labeled. - Dataset training = sqlContext.createDataFrame(Arrays.asList( + Dataset training = spark.createDataFrame(Arrays.asList( new JavaLabeledDocument(0L, "a b c d e spark", 1.0), new JavaLabeledDocument(1L, "b d", 0.0), new JavaLabeledDocument(2L, "spark f g h", 1.0), @@ -71,7 +65,7 @@ public static void main(String[] args) { PipelineModel model = pipeline.fit(training); // Prepare test documents, which are unlabeled. - Dataset test = sqlContext.createDataFrame(Arrays.asList( + Dataset test = spark.createDataFrame(Arrays.asList( new JavaDocument(4L, "spark i j k"), new JavaDocument(5L, "l m n"), new JavaDocument(6L, "mapreduce spark"), @@ -86,6 +80,6 @@ public static void main(String[] args) { } // $example off$ - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java index 48fc3c8acb0c0..e328454c70591 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -17,15 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.PolynomialExpansion; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -39,9 +36,7 @@ public class JavaPolynomialExpansionExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaPolynomialExpansionExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaPolynomialExpansionExample").getOrCreate(); // $example on$ PolynomialExpansion polyExpansion = new PolynomialExpansion() @@ -49,17 +44,17 @@ public static void main(String[] args) { .setOutputCol("polyFeatures") .setDegree(3); - JavaRDD data = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Vectors.dense(-2.0, 2.3)), RowFactory.create(Vectors.dense(0.0, 0.0)), RowFactory.create(Vectors.dense(0.6, -1.1)) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), }); - Dataset df = jsql.createDataFrame(data, schema); + Dataset df = spark.createDataFrame(data, schema); Dataset polyDF = polyExpansion.transform(df); List rows = polyDF.select("polyFeatures").takeAsList(3); @@ -67,6 +62,6 @@ public static void main(String[] args) { System.out.println(r.get(0)); } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java index 7b226fede9968..94e3fafcab189 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java @@ -17,13 +17,11 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.QuantileDiscretizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -36,19 +34,16 @@ public class JavaQuantileDiscretizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaQuantileDiscretizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaQuantileDiscretizerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize( - Arrays.asList( - RowFactory.create(0, 18.0), - RowFactory.create(1, 19.0), - RowFactory.create(2, 8.0), - RowFactory.create(3, 5.0), - RowFactory.create(4, 2.2) - ) + List data = Arrays.asList( + RowFactory.create(0, 18.0), + RowFactory.create(1, 19.0), + RowFactory.create(2, 8.0), + RowFactory.create(3, 5.0), + RowFactory.create(4, 2.2) ); StructType schema = new StructType(new StructField[]{ @@ -56,7 +51,7 @@ public static void main(String[] args) { new StructField("hour", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); QuantileDiscretizer discretizer = new QuantileDiscretizer() .setInputCol("hour") @@ -66,6 +61,6 @@ public static void main(String[] args) { Dataset result = discretizer.fit(df).transform(df); result.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java index 8c453bf80d645..8282ce01d36f3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.RFormula; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,9 +35,7 @@ public class JavaRFormulaExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRFormulaExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaRFormulaExample").getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ @@ -49,13 +45,13 @@ public static void main(String[] args) { createStructField("clicked", DoubleType, false) }); - JavaRDD rdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(7, "US", 18, 1.0), RowFactory.create(8, "CA", 12, 0.0), RowFactory.create(9, "NZ", 15, 0.0) - )); + ); - Dataset dataset = sqlContext.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(data, schema); RFormula formula = new RFormula() .setFormula("clicked ~ country + hour") .setFeaturesCol("features") @@ -63,7 +59,7 @@ public static void main(String[] args) { Dataset output = formula.fit(dataset).transform(dataset); output.select("features", "label").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java index 05c2bc9622e1b..21e783a96897f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -29,19 +27,17 @@ import org.apache.spark.ml.feature.*; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaRandomForestClassifierExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRandomForestClassifierExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaRandomForestClassifierExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -98,6 +94,6 @@ public static void main(String[] args) { System.out.println("Learned classification forest model:\n" + rfModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java index d366967083a19..ece184a8784fc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -30,19 +28,17 @@ import org.apache.spark.ml.regression.RandomForestRegressor; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example off$ public class JavaRandomForestRegressorExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaRandomForestRegressorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaRandomForestRegressorExample").getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = - sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -87,6 +83,6 @@ public static void main(String[] args) { System.out.println("Learned regression forest model:\n" + rfModel.toDebugString()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java index 7e3ca99d7cb93..492718bbdba81 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java @@ -19,36 +19,31 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.SQLTransformer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaSQLTransformerExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaSQLTransformerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaSQLTransformerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, 1.0, 3.0), RowFactory.create(2, 2.0, 5.0) - )); + ); StructType schema = new StructType(new StructField [] { new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("v1", DataTypes.DoubleType, false, Metadata.empty()), new StructField("v2", DataTypes.DoubleType, false, Metadata.empty()) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); SQLTransformer sqlTrans = new SQLTransformer().setStatement( "SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); @@ -56,6 +51,6 @@ public static void main(String[] args) { sqlTrans.transform(df).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index cb911ef5ef586..f906843640eda 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -21,8 +21,6 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionModel; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.classification.LogisticRegression; @@ -30,7 +28,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -42,9 +40,7 @@ public class JavaSimpleParamsExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaSimpleParamsExample").getOrCreate(); // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans @@ -55,7 +51,7 @@ public static void main(String[] args) { new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); Dataset training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); + spark.createDataFrame(localTraining, LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -96,7 +92,7 @@ public static void main(String[] args) { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); + Dataset test = spark.createDataFrame(localTest, LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegressionModel.transform will only use the 'features' column. @@ -109,6 +105,6 @@ public static void main(String[] args) { + ", prediction=" + r.get(3)); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index a18a60f448166..9516ce1f4fb18 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -21,8 +21,6 @@ import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -31,7 +29,7 @@ import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; /** * A simple text classification pipeline that recognizes "spark" from input text. It uses the Java @@ -44,9 +42,8 @@ public class JavaSimpleTextClassificationPipeline { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder().appName("JavaSimpleTextClassificationPipeline").getOrCreate(); // Prepare training documents, which are labeled. List localTraining = Lists.newArrayList( @@ -55,7 +52,7 @@ public static void main(String[] args) { new LabeledDocument(2L, "spark f g h", 1.0), new LabeledDocument(3L, "hadoop mapreduce", 0.0)); Dataset training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); + spark.createDataFrame(localTraining, LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -80,7 +77,7 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "spark hadoop spark"), new Document(7L, "apache hadoop")); - Dataset test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); + Dataset test = spark.createDataFrame(localTest, Document.class); // Make predictions on test documents. Dataset predictions = model.transform(test); @@ -89,6 +86,6 @@ public static void main(String[] args) { + ", prediction=" + r.get(3)); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java index e2dd759c0a40c..10f82f223386e 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import org.apache.spark.ml.feature.StandardScaler; @@ -30,12 +28,11 @@ public class JavaStandardScalerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStandardScalerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaStandardScalerExample").getOrCreate(); // $example on$ - Dataset dataFrame = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); StandardScaler scaler = new StandardScaler() .setInputCol("features") @@ -50,6 +47,6 @@ public static void main(String[] args) { Dataset scaledData = scalerModel.transform(dataFrame); scaledData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java index 0ff3782cb3e90..23ed071c9f6e5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.StopWordsRemover; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -38,28 +36,26 @@ public class JavaStopWordsRemoverExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStopWordsRemoverExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaStopWordsRemoverExample").getOrCreate(); // $example on$ StopWordsRemover remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered"); - JavaRDD rdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField( "raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) }); - Dataset dataset = jsql.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(data, schema); remover.transform(dataset).show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java index ceacbb4fb3f33..d4c2cf96a7cf3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.StringIndexer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,30 +35,28 @@ public class JavaStringIndexerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaStringIndexerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaStringIndexerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "a"), RowFactory.create(1, "b"), RowFactory.create(2, "c"), RowFactory.create(3, "a"), RowFactory.create(4, "a"), RowFactory.create(5, "c") - )); + ); StructType schema = new StructType(new StructField[]{ createStructField("id", IntegerType, false), createStructField("category", StringType, false) }); - Dataset df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); StringIndexer indexer = new StringIndexer() .setInputCol("category") .setOutputCol("categoryIndex"); Dataset indexed = indexer.fit(df).transform(df); indexed.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index 107c835f2e01e..a8169917772c1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -19,10 +19,8 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.IDF; import org.apache.spark.ml.feature.IDFModel; @@ -31,7 +29,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -40,21 +38,19 @@ public class JavaTfIdfExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTfIdfExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaTfIdfExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "Hi I heard about Spark"), RowFactory.create(0, "I wish Java could use case classes"), RowFactory.create(1, "Logistic regression models are neat") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - Dataset sentenceData = sqlContext.createDataFrame(jrdd, schema); + Dataset sentenceData = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); Dataset wordsData = tokenizer.transform(sentenceData); int numFeatures = 20; @@ -76,6 +72,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java index 9225fe2262f57..a65735a5e5143 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -17,14 +17,12 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.RegexTokenizer; import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.Dataset; @@ -38,23 +36,21 @@ public class JavaTokenizerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTokenizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaTokenizerExample").getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(0, "Hi I heard about Spark"), RowFactory.create(1, "I wish Java could use case classes"), RowFactory.create(2, "Logistic,regression,models,are,neat") - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.IntegerType, false, Metadata.empty()), new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - Dataset sentenceDataFrame = sqlContext.createDataFrame(jrdd, schema); + Dataset sentenceDataFrame = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); @@ -70,6 +66,6 @@ public static void main(String[] args) { .setOutputCol("words") .setPattern("\\W"); // alternatively .setPattern("\\w+").setGaps(false); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java index 953ad455b1dcd..9569bc2412def 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java @@ -17,14 +17,11 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Arrays; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.VectorAssembler; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -38,9 +35,7 @@ public class JavaVectorAssemblerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorAssemblerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaVectorAssemblerExample").getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ @@ -51,8 +46,7 @@ public static void main(String[] args) { createStructField("clicked", DoubleType, false) }); Row row = RowFactory.create(0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0); - JavaRDD rdd = jsc.parallelize(Arrays.asList(row)); - Dataset dataset = sqlContext.createDataFrame(rdd, schema); + Dataset dataset = spark.createDataFrame(Arrays.asList(row), schema); VectorAssembler assembler = new VectorAssembler() .setInputCols(new String[]{"hour", "mobile", "userFeatures"}) @@ -61,7 +55,7 @@ public static void main(String[] args) { Dataset output = assembler.transform(dataset); System.out.println(output.select("features", "clicked").first()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java index b3b5953ee7bbe..217d5a06d10a4 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java @@ -17,9 +17,7 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ import java.util.Map; @@ -32,12 +30,10 @@ public class JavaVectorIndexerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorIndexerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaVectorIndexerExample").getOrCreate(); // $example on$ - Dataset data = jsql.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); VectorIndexer indexer = new VectorIndexer() .setInputCol("features") @@ -57,6 +53,6 @@ public static void main(String[] args) { Dataset indexedData = indexerModel.transform(data); indexedData.show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java index 2ae57c3577eff..4f1ea824a3a9f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java @@ -17,14 +17,13 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; // $example on$ +import java.util.List; + import com.google.common.collect.Lists; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; @@ -38,9 +37,7 @@ public class JavaVectorSlicerExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaVectorSlicerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaVectorSlicerExample").getOrCreate(); // $example on$ Attribute[] attrs = new Attribute[]{ @@ -50,13 +47,13 @@ public static void main(String[] args) { }; AttributeGroup group = new AttributeGroup("userFeatures", attrs); - JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + List data = Lists.newArrayList( RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) - )); + ); Dataset dataset = - jsql.createDataFrame(jrdd, (new StructType()).add(group.toStructField())); + spark.createDataFrame(data, (new StructType()).add(group.toStructField())); VectorSlicer vectorSlicer = new VectorSlicer() .setInputCol("userFeatures").setOutputCol("features"); @@ -68,7 +65,7 @@ public static void main(String[] args) { System.out.println(output.select("userFeatures", "features").first()); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java index c5bb1eaaa3446..d9b1a79b52548 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java @@ -19,37 +19,32 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.Word2Vec; import org.apache.spark.ml.feature.Word2VecModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaWord2VecExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaWord2VecExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession.builder().appName("JavaWord2VecExample").getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("Hi I heard about Spark".split(" "))), RowFactory.create(Arrays.asList("I wish Java could use case classes".split(" "))), RowFactory.create(Arrays.asList("Logistic regression models are neat".split(" "))) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - Dataset documentDF = sqlContext.createDataFrame(jrdd, schema); + Dataset documentDF = spark.createDataFrame(data, schema); // Learn a mapping from words to Vectors. Word2Vec word2Vec = new Word2Vec() @@ -64,6 +59,6 @@ public static void main(String[] args) { } // $example off$ - jsc.stop(); + spark.stop(); } } 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 354a5306ed45f..ec2142e756955 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 @@ -21,14 +21,12 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; public class JavaSparkSQL { public static class Person implements Serializable { @@ -53,13 +51,12 @@ 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 sqlContext = new SQLContext(ctx); + SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate(); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. - JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( + String file = "examples/src/main/resources/people.txt"; + JavaRDD people = spark.read().text(file).javaRDD().map( new Function() { @Override public Person call(String line) { @@ -74,12 +71,11 @@ public Person call(String line) { }); // Apply a schema to an RDD of Java Beans and register it as a table. - Dataset schemaPeople = sqlContext.createDataFrame(people, Person.class); + Dataset schemaPeople = spark.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. - Dataset teenagers = - sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + Dataset teenagers = spark.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. @@ -100,12 +96,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. - Dataset parquetFile = sqlContext.read().parquet("people.parquet"); + Dataset parquetFile = spark.read().parquet("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); Dataset teenagers2 = - sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -121,7 +117,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 - Dataset peopleFromJsonFile = sqlContext.read().json(path); + Dataset peopleFromJsonFile = spark.read().json(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. @@ -135,8 +131,7 @@ public String call(Row row) { peopleFromJsonFile.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlContext. - Dataset teenagers3 = - sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + Dataset teenagers3 = spark.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. @@ -152,8 +147,8 @@ public String call(Row row) { // a RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); - JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - Dataset peopleFromJsonRDD = sqlContext.read().json(anotherPeopleRDD.rdd()); + JavaRDD anotherPeopleRDD = spark.createDataFrame(jsonData, String.class).toJSON().javaRDD(); + Dataset peopleFromJsonRDD = spark.read().json(anotherPeopleRDD); // Take a look at the schema of this new DataFrame. peopleFromJsonRDD.printSchema(); @@ -166,7 +161,7 @@ public String call(Row row) { peopleFromJsonRDD.registerTempTable("people2"); - Dataset peopleWithCity = sqlContext.sql("SELECT name, address.city FROM people2"); + Dataset peopleWithCity = spark.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -177,6 +172,6 @@ public String call(Row row) { System.out.println(name); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 7aa8862761d2b..44f1e800fe448 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -22,14 +22,13 @@ import java.util.regex.Pattern; import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.VoidFunction2; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.api.java.StorageLevels; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.Time; @@ -82,7 +81,7 @@ public Iterator call(String x) { words.foreachRDD(new VoidFunction2, Time>() { @Override public void call(JavaRDD rdd, Time time) { - SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); + SparkSession spark = JavaSparkSessionSingleton.getInstance(rdd.context().getConf()); // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame JavaRDD rowRDD = rdd.map(new Function() { @@ -93,14 +92,14 @@ public JavaRecord call(String word) { return record; } }); - Dataset wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRecord.class); + Dataset wordsDataFrame = spark.createDataFrame(rowRDD, JavaRecord.class); // Register as table wordsDataFrame.registerTempTable("words"); // Do word count on table using SQL and print it Dataset wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word"); + spark.sql("select word, count(*) as total from words group by word"); System.out.println("========= " + time + "========="); wordCountsDataFrame.show(); } @@ -111,12 +110,12 @@ public JavaRecord call(String word) { } } -/** Lazily instantiated singleton instance of SQLContext */ -class JavaSQLContextSingleton { - private static transient SQLContext instance = null; - public static SQLContext getInstance(SparkContext sparkContext) { +/** Lazily instantiated singleton instance of SparkSession */ +class JavaSparkSessionSingleton { + private static transient SparkSession instance = null; + public static SparkSession getInstance(SparkConf sparkConf) { if (instance == null) { - instance = new SQLContext(sparkContext); + instance = SparkSession.builder().config(sparkConf).getOrCreate(); } return instance; } diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index 0c9ac583b2dab..e36444f18506e 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -21,8 +21,7 @@ if sys.version >= '3': long = int -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.evaluation import RegressionEvaluator @@ -31,15 +30,14 @@ # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="ALSExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("ALSExample").getOrCreate() # $example on$ - lines = sc.textFile("data/mllib/als/sample_movielens_ratings.txt") - parts = lines.map(lambda l: l.split("::")) + lines = spark.read.text("data/mllib/als/sample_movielens_ratings.txt").rdd + parts = lines.map(lambda row: row.value.split("::")) ratingsRDD = parts.map(lambda p: Row(userId=int(p[0]), movieId=int(p[1]), rating=float(p[2]), timestamp=long(p[3]))) - ratings = sqlContext.createDataFrame(ratingsRDD) + ratings = spark.createDataFrame(ratingsRDD) (training, test) = ratings.randomSplit([0.8, 0.2]) # Build the recommendation model using ALS on the training data @@ -56,4 +54,4 @@ rmse = evaluator.evaluate(predictions) print("Root-mean-square error = " + str(rmse)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/binarizer_example.py b/examples/src/main/python/ml/binarizer_example.py index 317cfa638a5a9..072187e64564c 100644 --- a/examples/src/main/python/ml/binarizer_example.py +++ b/examples/src/main/python/ml/binarizer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Binarizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BinarizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() # $example on$ - continuousDataFrame = sqlContext.createDataFrame([ + continuousDataFrame = spark.createDataFrame([ (0, 0.1), (1, 0.8), (2, 0.2) @@ -40,4 +38,4 @@ print(binarized_feature) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index e6f6bfd7e84ed..836a89cde0d60 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -17,28 +17,26 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.clustering import BisectingKMeans, BisectingKMeansModel from pyspark.mllib.linalg import VectorUDT, _convert_to_vector, Vectors from pyspark.mllib.linalg import Vectors from pyspark.sql.types import Row # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession """ A simple example demonstrating a bisecting k-means clustering. """ if __name__ == "__main__": - - sc = SparkContext(appName="PythonBisectingKMeansExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PythonBisectingKMeansExample").getOrCreate() # $example on$ - data = sc.textFile("data/mllib/kmeans_data.txt") - parsed = data.map(lambda l: Row(features=Vectors.dense([float(x) for x in l.split(' ')]))) - training = sqlContext.createDataFrame(parsed) + data = spark.read.text("data/mllib/kmeans_data.txt").rdd + parsed = data\ + .map(lambda row: Row(features=Vectors.dense([float(x) for x in row.value.split(' ')]))) + training = spark.createDataFrame(parsed) kmeans = BisectingKMeans().setK(2).setSeed(1).setFeaturesCol("features") @@ -54,4 +52,4 @@ print(center) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/bucketizer_example.py b/examples/src/main/python/ml/bucketizer_example.py index 4304255f350db..288ec62bdfbb5 100644 --- a/examples/src/main/python/ml/bucketizer_example.py +++ b/examples/src/main/python/ml/bucketizer_example.py @@ -17,21 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import Bucketizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BucketizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() # $example on$ splits = [-float("inf"), -0.5, 0.0, 0.5, float("inf")] data = [(-0.5,), (-0.3,), (0.0,), (0.2,)] - dataFrame = sqlContext.createDataFrame(data, ["features"]) + dataFrame = spark.createDataFrame(data, ["features"]) bucketizer = Bucketizer(splits=splits, inputCol="features", outputCol="bucketedFeatures") @@ -40,4 +38,4 @@ bucketedData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py index 997a504735360..8f58fc28deb9b 100644 --- a/examples/src/main/python/ml/chisq_selector_example.py +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import ChiSqSelector from pyspark.mllib.linalg import Vectors # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="ChiSqSelectorExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (7, Vectors.dense([0.0, 0.0, 18.0, 1.0]), 1.0,), (8, Vectors.dense([0.0, 1.0, 12.0, 0.0]), 0.0,), (9, Vectors.dense([1.0, 0.0, 15.0, 0.1]), 0.0,)], ["id", "features", "clicked"]) @@ -41,4 +39,4 @@ result.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/count_vectorizer_example.py b/examples/src/main/python/ml/count_vectorizer_example.py index e839f645f70b5..9dbf9959d17ef 100644 --- a/examples/src/main/python/ml/count_vectorizer_example.py +++ b/examples/src/main/python/ml/count_vectorizer_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import CountVectorizer # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="CountVectorizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("CountVectorizerExample").getOrCreate() # $example on$ # Input data: Each row is a bag of words with a ID. - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, "a b c".split(" ")), (1, "a b b c a".split(" ")) ], ["id", "words"]) @@ -41,4 +39,4 @@ result.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/cross_validator.py b/examples/src/main/python/ml/cross_validator.py index 5f0ef20218c4a..a61d0f63d277c 100644 --- a/examples/src/main/python/ml/cross_validator.py +++ b/examples/src/main/python/ml/cross_validator.py @@ -17,15 +17,14 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.evaluation import BinaryClassificationEvaluator from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.ml.tuning import CrossValidator, ParamGridBuilder -from pyspark.sql import Row, SQLContext # $example off$ +from pyspark.sql import Row, SparkSession """ A simple example demonstrating model selection using CrossValidator. @@ -36,25 +35,23 @@ """ if __name__ == "__main__": - sc = SparkContext(appName="CrossValidatorExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("CrossValidatorExample").getOrCreate() # $example on$ # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - 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), - (4, "b spark who", 1.0), - (5, "g d a y", 0.0), - (6, "spark fly", 1.0), - (7, "was mapreduce", 0.0), - (8, "e spark program", 1.0), - (9, "a e c l", 0.0), - (10, "spark compile", 1.0), - (11, "hadoop software", 0.0) - ]) \ - .map(lambda x: LabeledDocument(*x)).toDF() + training = spark.createDataFrame([ + (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), + (4, "b spark who", 1.0), + (5, "g d a y", 0.0), + (6, "spark fly", 1.0), + (7, "was mapreduce", 0.0), + (8, "e spark program", 1.0), + (9, "a e c l", 0.0), + (10, "spark compile", 1.0), + (11, "hadoop software", 0.0) + ], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -82,12 +79,12 @@ cvModel = crossval.fit(training) # 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")]) \ - .map(lambda x: Document(*x)).toDF() + test = spark.createDataFrame([ + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop") + ], ["id", "text"]) # Make predictions on test documents. cvModel uses the best model found (lrModel). prediction = cvModel.transform(test) @@ -96,4 +93,4 @@ print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py index d2644ca335654..b3e671038eb7f 100644 --- a/examples/src/main/python/ml/dataframe_example.py +++ b/examples/src/main/python/ml/dataframe_example.py @@ -26,16 +26,14 @@ import tempfile import shutil -from pyspark import SparkContext -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.mllib.stat import Statistics if __name__ == "__main__": if len(sys.argv) > 2: print("Usage: dataframe_example.py ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="DataFrameExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("DataFrameExample").getOrCreate() if len(sys.argv) == 2: input = sys.argv[1] else: @@ -43,7 +41,7 @@ # Load input data print("Loading LIBSVM file with UDT from " + input + ".") - df = sqlContext.read.format("libsvm").load(input).cache() + df = spark.read.format("libsvm").load(input).cache() print("Schema from LIBSVM:") df.printSchema() print("Loaded training data as a DataFrame with " + @@ -54,7 +52,7 @@ labelSummary.show() # Convert features column to an RDD of vectors. - features = df.select("features").map(lambda r: r.features) + features = df.select("features").rdd.map(lambda r: r.features) summary = Statistics.colStats(features) print("Selected features column with average values:\n" + str(summary.mean())) @@ -67,9 +65,9 @@ # Load the records back. print("Loading Parquet file with UDT from " + tempdir) - newDF = sqlContext.read.parquet(tempdir) + newDF = spark.read.parquet(tempdir) print("Schema from Parquet:") newDF.printSchema() shutil.rmtree(tempdir) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py index 264d47f404cb1..1bf8fc6d140c9 100644 --- a/examples/src/main/python/ml/dct_example.py +++ b/examples/src/main/python/ml/dct_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import DCT from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="DCTExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("DCTExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (Vectors.dense([0.0, 1.0, -2.0, 3.0]),), (Vectors.dense([-1.0, 2.0, 4.0, -7.0]),), (Vectors.dense([14.0, -2.0, -5.0, 1.0]),)], ["features"]) @@ -42,4 +40,4 @@ print(dcts) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index 86bdc65392bbb..d2318e24369d8 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -21,20 +21,19 @@ from __future__ import print_function # $example on$ -from pyspark import SparkContext, SQLContext from pyspark.ml import Pipeline from pyspark.ml.classification import DecisionTreeClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="decision_tree_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,3 +71,5 @@ # summary only print(treeModel) # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index 8e20d5d8572a5..9e8cb382a9bfe 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import DecisionTreeRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="decision_tree_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # We specify maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,3 +68,5 @@ # summary only print(treeModel) # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index c85cb0d89543c..6fa641b772d95 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -17,23 +17,21 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import ElementwiseProduct from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="ElementwiseProductExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() # $example on$ data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] - df = sqlContext.createDataFrame(data, ["vector"]) + df = spark.createDataFrame(data, ["vector"]) transformer = ElementwiseProduct(scalingVec=Vectors.dense([0.0, 1.0, 2.0]), inputCol="vector", outputCol="transformedVector") transformer.transform(df).show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py index 9a8993dac4f65..4993b5a9846c2 100644 --- a/examples/src/main/python/ml/estimator_transformer_param_example.py +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -18,20 +18,19 @@ """ Estimator Transformer Param Example. """ -from pyspark import SparkContext, SQLContext + # $example on$ from pyspark.mllib.linalg import Vectors from pyspark.ml.classification import LogisticRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="EstimatorTransformerParamExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() # $example on$ # Prepare training data from a list of (label, features) tuples. - training = sqlContext.createDataFrame([ + training = spark.createDataFrame([ (1.0, Vectors.dense([0.0, 1.1, 0.1])), (0.0, Vectors.dense([2.0, 1.0, -1.0])), (0.0, Vectors.dense([2.0, 1.3, 1.0])), @@ -69,7 +68,7 @@ print model2.extractParamMap() # Prepare test data - test = sqlContext.createDataFrame([ + test = spark.createDataFrame([ (1.0, Vectors.dense([-1.0, 1.5, 1.3])), (0.0, Vectors.dense([3.0, 2.0, -0.1])), (1.0, Vectors.dense([0.0, 2.2, -1.5]))], ["label", "features"]) @@ -84,4 +83,4 @@ print row # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index f7e842f4b303a..b09ad41da313c 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import GBTClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="gradient_boosted_tree_classifier_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("gradient_boosted_tree_classifier_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,4 +71,4 @@ print(gbtModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py index f8b4de651c768..caa7cfc4e11f5 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import GBTRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="gradient_boosted_tree_regressor_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("gradient_boosted_tree_regressor_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,4 +68,4 @@ print(gbtModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/index_to_string_example.py b/examples/src/main/python/ml/index_to_string_example.py index fb0ba2950bbd6..dd04b2c4b0bfd 100644 --- a/examples/src/main/python/ml/index_to_string_example.py +++ b/examples/src/main/python/ml/index_to_string_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import IndexToString, StringIndexer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="IndexToStringExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame( + df = spark.createDataFrame( [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], ["id", "category"]) @@ -42,4 +40,4 @@ converted.select("id", "originalCategory").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index fa57a4d3ada1b..7d9d80e6452bd 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -20,10 +20,9 @@ import sys import numpy as np -from pyspark import SparkContext from pyspark.ml.clustering import KMeans, KMeansModel from pyspark.mllib.linalg import VectorUDT, _convert_to_vector -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession from pyspark.sql.types import Row, StructField, StructType """ @@ -35,8 +34,8 @@ """ -def parseVector(line): - array = np.array([float(x) for x in line.split(' ')]) +def parseVector(row): + array = np.array([float(x) for x in row.value.split(' ')]) return _convert_to_vector(array) @@ -50,14 +49,13 @@ def parseVector(line): path = sys.argv[1] k = sys.argv[2] - sc = SparkContext(appName="PythonKMeansExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PythonKMeansExample").getOrCreate() - lines = sc.textFile(path) + lines = spark.read.text(path).rdd data = lines.map(parseVector) row_rdd = data.map(lambda x: Row(x)) schema = StructType([StructField(FEATURES_COL, VectorUDT(), False)]) - df = sqlContext.createDataFrame(row_rdd, schema) + df = spark.createDataFrame(row_rdd, schema) kmeans = KMeans().setK(2).setSeed(1).setFeaturesCol(FEATURES_COL) model = kmeans.fit(df) @@ -67,4 +65,4 @@ def parseVector(line): for center in centers: print(center) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/linear_regression_with_elastic_net.py b/examples/src/main/python/ml/linear_regression_with_elastic_net.py index a4cd40cf26726..99b7f7fe99de6 100644 --- a/examples/src/main/python/ml/linear_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/linear_regression_with_elastic_net.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.regression import LinearRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="LinearRegressionWithElasticNet") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("LinearRegressionWithElasticNet").getOrCreate() # $example on$ # Load training data - training = sqlContext.read.format("libsvm")\ + training = spark.read.format("libsvm")\ .load("data/mllib/sample_linear_regression_data.txt") lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) @@ -42,4 +40,4 @@ print("Intercept: " + str(lrModel.intercept)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py index b0b1d27e13bb0..0d7112e72354f 100644 --- a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import LogisticRegression # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="LogisticRegressionWithElasticNet") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("LogisticRegressionWithElasticNet").getOrCreate() # $example on$ # Load training data - training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") lr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) @@ -41,4 +39,4 @@ print("Intercept: " + str(lrModel.intercept)) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/max_abs_scaler_example.py b/examples/src/main/python/ml/max_abs_scaler_example.py index d9b69eef1cd84..1cb95a98f08dc 100644 --- a/examples/src/main/python/ml/max_abs_scaler_example.py +++ b/examples/src/main/python/ml/max_abs_scaler_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import MaxAbsScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="MaxAbsScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = MaxAbsScaler(inputCol="features", outputCol="scaledFeatures") @@ -40,4 +38,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/min_max_scaler_example.py b/examples/src/main/python/ml/min_max_scaler_example.py index 2f8e4ade468b9..8d91a59e2b0bd 100644 --- a/examples/src/main/python/ml/min_max_scaler_example.py +++ b/examples/src/main/python/ml/min_max_scaler_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import MinMaxScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="MinMaxScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures") @@ -40,4 +38,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/multilayer_perceptron_classification.py b/examples/src/main/python/ml/multilayer_perceptron_classification.py index f84588f547fff..8bededc14db4e 100644 --- a/examples/src/main/python/ml/multilayer_perceptron_classification.py +++ b/examples/src/main/python/ml/multilayer_perceptron_classification.py @@ -17,21 +17,19 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import MultilayerPerceptronClassifier from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="multilayer_perceptron_classification_example") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder.appName("multilayer_perceptron_classification_example").getOrCreate() # $example on$ # Load training data - data = sqlContext.read.format("libsvm")\ + data = spark.read.format("libsvm")\ .load("data/mllib/sample_multiclass_classification_data.txt") # Split the data into train and test splits = data.randomSplit([0.6, 0.4], 1234) @@ -52,4 +50,4 @@ print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/n_gram_example.py b/examples/src/main/python/ml/n_gram_example.py index f2d85f53e7219..b7fecf0d685c0 100644 --- a/examples/src/main/python/ml/n_gram_example.py +++ b/examples/src/main/python/ml/n_gram_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import NGram # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="NGramExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("NGramExample").getOrCreate() # $example on$ - wordDataFrame = sqlContext.createDataFrame([ + wordDataFrame = spark.createDataFrame([ (0, ["Hi", "I", "heard", "about", "Spark"]), (1, ["I", "wish", "Java", "could", "use", "case", "classes"]), (2, ["Logistic", "regression", "models", "are", "neat"]) @@ -39,4 +37,4 @@ print(ngrams_label) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py index db8fbea9bf9b1..e37035542c7e8 100644 --- a/examples/src/main/python/ml/naive_bayes_example.py +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -17,21 +17,18 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.classification import NaiveBayes from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="naive_bayes_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("naive_bayes_example").getOrCreate() # $example on$ # Load training data - data = sqlContext.read.format("libsvm") \ + data = spark.read.format("libsvm") \ .load("data/mllib/sample_libsvm_data.txt") # Split the data into train and test splits = data.randomSplit([0.6, 0.4], 1234) @@ -50,4 +47,4 @@ print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/normalizer_example.py b/examples/src/main/python/ml/normalizer_example.py index d490221474c24..ae2553761928c 100644 --- a/examples/src/main/python/ml/normalizer_example.py +++ b/examples/src/main/python/ml/normalizer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Normalizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="NormalizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Normalize each Vector using $L^1$ norm. normalizer = Normalizer(inputCol="features", outputCol="normFeatures", p=1.0) @@ -40,4 +38,4 @@ lInfNormData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/onehot_encoder_example.py b/examples/src/main/python/ml/onehot_encoder_example.py index 0f94c26638d35..9acc363dc9e25 100644 --- a/examples/src/main/python/ml/onehot_encoder_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import OneHotEncoder, StringIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="OneHotEncoderExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, "a"), (1, "b"), (2, "c"), @@ -45,4 +43,4 @@ encoded.select("id", "categoryVec").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py index a17181f1b8a51..adab151734aa8 100644 --- a/examples/src/main/python/ml/pca_example.py +++ b/examples/src/main/python/ml/pca_example.py @@ -17,26 +17,24 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import PCA from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="PCAExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PCAExample").getOrCreate() # $example on$ data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] - df = sqlContext.createDataFrame(data, ["features"]) + df = spark.createDataFrame(data, ["features"]) pca = PCA(k=3, inputCol="features", outputCol="pcaFeatures") model = pca.fit(df) result = model.transform(df).select("pcaFeatures") result.show(truncate=False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/pipeline_example.py b/examples/src/main/python/ml/pipeline_example.py index 3288568f0c287..ed9765d96165e 100644 --- a/examples/src/main/python/ml/pipeline_example.py +++ b/examples/src/main/python/ml/pipeline_example.py @@ -18,21 +18,20 @@ """ Pipeline Example. """ -from pyspark import SparkContext, SQLContext + # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.feature import HashingTF, Tokenizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - - sc = SparkContext(appName="PipelineExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PipelineExample").getOrCreate() # $example on$ # Prepare training documents from a list of (id, text, label) tuples. - training = sqlContext.createDataFrame([ + training = spark.createDataFrame([ (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -48,7 +47,7 @@ model = pipeline.fit(training) # Prepare test documents, which are unlabeled (id, text) tuples. - test = sqlContext.createDataFrame([ + test = spark.createDataFrame([ (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -61,4 +60,4 @@ print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 89f5cbe8f2f41..328b559320f85 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import PolynomialExpansion from pyspark.mllib.linalg import Vectors # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="PolynomialExpansionExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() # $example on$ - df = sqlContext\ + df = spark\ .createDataFrame([(Vectors.dense([-2.0, 2.3]),), (Vectors.dense([0.0, 0.0]),), (Vectors.dense([0.6, -1.1]),)], @@ -40,4 +38,4 @@ print(expanded) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index c3570438c51d9..b0a93e050c54b 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import RandomForestClassifier from pyspark.ml.feature import StringIndexer, VectorIndexer from pyspark.ml.evaluation import MulticlassClassificationEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="random_forest_classifier_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("random_forest_classifier_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Index labels, adding metadata to the label column. # Fit on whole dataset to include all labels in index. @@ -72,4 +71,4 @@ print(rfModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py index b77014f379237..4bb84f0de8ff3 100644 --- a/examples/src/main/python/ml/random_forest_regressor_example.py +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -20,21 +20,20 @@ """ from __future__ import print_function -from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline from pyspark.ml.regression import RandomForestRegressor from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="random_forest_regressor_example") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("random_forest_regressor_example").getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") # Automatically identify categorical features, and index them. # Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -69,4 +68,4 @@ print(rfModel) # summary only # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/rformula_example.py b/examples/src/main/python/ml/rformula_example.py index b544a14700762..45cc116ac2f61 100644 --- a/examples/src/main/python/ml/rformula_example.py +++ b/examples/src/main/python/ml/rformula_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import RFormula # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="RFormulaExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() # $example on$ - dataset = sqlContext.createDataFrame( + dataset = spark.createDataFrame( [(7, "US", 18, 1.0), (8, "CA", 12, 0.0), (9, "NZ", 15, 0.0)], @@ -41,4 +39,4 @@ output.select("features", "label").show() # $example off$ - sc.stop() + spark.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 b4f06bf888746..3600c12211d05 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -17,11 +17,10 @@ from __future__ import print_function -from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.feature import HashingTF, Tokenizer -from pyspark.sql import Row, SQLContext +from pyspark.sql import Row, SparkSession """ @@ -34,16 +33,15 @@ if __name__ == "__main__": - sc = SparkContext(appName="SimpleTextClassificationPipeline") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - 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() + training = spark.createDataFrame([ + (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) + ], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -55,12 +53,12 @@ model = pipeline.fit(training) # Prepare test documents, which are unlabeled. - Document = Row("id", "text") - test = sc.parallelize([(4, "spark i j k"), - (5, "l m n"), - (6, "spark hadoop spark"), - (7, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() + test = spark.createDataFrame([ + (4, "spark i j k"), + (5, "l m n"), + (6, "spark hadoop spark"), + (7, "apache hadoop") + ], ["id", "text"]) # Make predictions on test documents and print columns of interest. prediction = model.transform(test) @@ -68,4 +66,4 @@ for row in selected.collect(): print(row) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/sql_transformer.py b/examples/src/main/python/ml/sql_transformer.py index 9575d728d8159..26045db4be6bc 100644 --- a/examples/src/main/python/ml/sql_transformer.py +++ b/examples/src/main/python/ml/sql_transformer.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import SQLTransformer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="SQLTransformerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ (0, 1.0, 3.0), (2, 2.0, 5.0) ], ["id", "v1", "v2"]) @@ -37,4 +35,4 @@ sqlTrans.transform(df).show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/standard_scaler_example.py b/examples/src/main/python/ml/standard_scaler_example.py index ae7aa85005bcd..c50804f6bf4e0 100644 --- a/examples/src/main/python/ml/standard_scaler_example.py +++ b/examples/src/main/python/ml/standard_scaler_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StandardScaler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StandardScalerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() # $example on$ - dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") scaler = StandardScaler(inputCol="features", outputCol="scaledFeatures", withStd=True, withMean=False) @@ -40,4 +38,4 @@ scaledData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/stopwords_remover_example.py b/examples/src/main/python/ml/stopwords_remover_example.py index 01f94af8ca752..57362673df786 100644 --- a/examples/src/main/python/ml/stopwords_remover_example.py +++ b/examples/src/main/python/ml/stopwords_remover_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StopWordsRemover # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StopWordsRemoverExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() # $example on$ - sentenceData = sqlContext.createDataFrame([ + sentenceData = spark.createDataFrame([ (0, ["I", "saw", "the", "red", "baloon"]), (1, ["Mary", "had", "a", "little", "lamb"]) ], ["label", "raw"]) @@ -37,4 +35,4 @@ remover.transform(sentenceData).show(truncate=False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/string_indexer_example.py b/examples/src/main/python/ml/string_indexer_example.py index 58a8cb5d56b73..aacd4f999bb1b 100644 --- a/examples/src/main/python/ml/string_indexer_example.py +++ b/examples/src/main/python/ml/string_indexer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import StringIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="StringIndexerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame( + df = spark.createDataFrame( [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], ["id", "category"]) indexer = StringIndexer(inputCol="category", outputCol="categoryIndex") @@ -36,4 +34,4 @@ indexed.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index 141324d458530..25df8166efc2e 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import HashingTF, IDF, Tokenizer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="TfIdfExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() # $example on$ - sentenceData = sqlContext.createDataFrame([ + sentenceData = spark.createDataFrame([ (0, "Hi I heard about Spark"), (0, "I wish Java could use case classes"), (1, "Logistic regression models are neat") @@ -46,4 +44,4 @@ print(features_label) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/tokenizer_example.py b/examples/src/main/python/ml/tokenizer_example.py index ce9b225be5357..5be4b4cfe3a52 100644 --- a/examples/src/main/python/ml/tokenizer_example.py +++ b/examples/src/main/python/ml/tokenizer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Tokenizer, RegexTokenizer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="TokenizerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() # $example on$ - sentenceDataFrame = sqlContext.createDataFrame([ + sentenceDataFrame = spark.createDataFrame([ (0, "Hi I heard about Spark"), (1, "I wish Java could use case classes"), (2, "Logistic,regression,models,are,neat") @@ -41,4 +39,4 @@ # alternatively, pattern="\\w+", gaps(False) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/train_validation_split.py b/examples/src/main/python/ml/train_validation_split.py index 161a200c61b6d..2e43a0f8aec54 100644 --- a/examples/src/main/python/ml/train_validation_split.py +++ b/examples/src/main/python/ml/train_validation_split.py @@ -15,13 +15,12 @@ # limitations under the License. # -from pyspark import SparkContext # $example on$ from pyspark.ml.evaluation import RegressionEvaluator from pyspark.ml.regression import LinearRegression from pyspark.ml.tuning import ParamGridBuilder, TrainValidationSplit -from pyspark.sql import SQLContext # $example off$ +from pyspark.sql import SparkSession """ This example demonstrates applying TrainValidationSplit to split data @@ -32,11 +31,10 @@ """ if __name__ == "__main__": - sc = SparkContext(appName="TrainValidationSplit") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("TrainValidationSplit").getOrCreate() # $example on$ # Prepare training and test data. - data = sqlContext.read.format("libsvm")\ + data = spark.read.format("libsvm")\ .load("data/mllib/sample_linear_regression_data.txt") train, test = data.randomSplit([0.7, 0.3]) lr = LinearRegression(maxIter=10, regParam=0.1) @@ -65,4 +63,4 @@ for row in prediction.take(5): print(row) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py index 04f64839f188d..019a9ea6f7503 100644 --- a/examples/src/main/python/ml/vector_assembler_example.py +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.mllib.linalg import Vectors from pyspark.ml.feature import VectorAssembler # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorAssemblerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() # $example on$ - dataset = sqlContext.createDataFrame( + dataset = spark.createDataFrame( [(0, 18, 1.0, Vectors.dense([0.0, 10.0, 0.5]), 1.0)], ["id", "hour", "mobile", "userFeatures", "clicked"]) assembler = VectorAssembler( @@ -39,4 +37,4 @@ print(output.select("features", "clicked").first()) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_indexer_example.py b/examples/src/main/python/ml/vector_indexer_example.py index 146f41c1dd903..3cf5b8ebf17ee 100644 --- a/examples/src/main/python/ml/vector_indexer_example.py +++ b/examples/src/main/python/ml/vector_indexer_example.py @@ -17,18 +17,16 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import VectorIndexer # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorIndexerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() # $example on$ - data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") indexer = VectorIndexer(inputCol="features", outputCol="indexed", maxCategories=10) indexerModel = indexer.fit(data) @@ -37,4 +35,4 @@ indexedData.show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index 31a753073c13c..0531bcdb06675 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -17,20 +17,18 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import VectorSlicer from pyspark.mllib.linalg import Vectors from pyspark.sql.types import Row # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="VectorSlicerExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() # $example on$ - df = sqlContext.createDataFrame([ + df = spark.createDataFrame([ Row(userFeatures=Vectors.sparse(3, {0: -2.0, 1: 2.3}),), Row(userFeatures=Vectors.dense([-2.0, 2.3, 0.0]),)]) @@ -41,4 +39,4 @@ output.select("userFeatures", "features").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/word2vec_example.py b/examples/src/main/python/ml/word2vec_example.py index 53c77feb10145..6766a7b6aa454 100644 --- a/examples/src/main/python/ml/word2vec_example.py +++ b/examples/src/main/python/ml/word2vec_example.py @@ -17,19 +17,17 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Word2Vec # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="Word2VecExample") - sqlContext = SQLContext(sc) + spark = SparkSession.builder.appName("Word2VecExample").getOrCreate() # $example on$ # Input data: Each row is a bag of words from a sentence or document. - documentDF = sqlContext.createDataFrame([ + documentDF = spark.createDataFrame([ ("Hi I heard about Spark".split(" "), ), ("I wish Java could use case classes".split(" "), ), ("Logistic regression models are neat".split(" "), ) @@ -42,4 +40,4 @@ print(feature) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py index 4e7ea289b2532..8f0fc9d45da2b 100644 --- a/examples/src/main/python/mllib/binary_classification_metrics_example.py +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -18,7 +18,7 @@ Binary Classification Metrics Example. """ from __future__ import print_function -from pyspark import SparkContext, SQLContext +from pyspark import SparkContext # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS from pyspark.mllib.evaluation import BinaryClassificationMetrics @@ -27,7 +27,7 @@ if __name__ == "__main__": sc = SparkContext(appName="BinaryClassificationMetricsExample") - sqlContext = SQLContext(sc) + # $example on$ # Several of the methods available in scala are currently missing from pyspark # Load training data in LIBSVM format @@ -52,3 +52,5 @@ # Area under ROC curve print("Area under ROC = %s" % metrics.areaUnderROC) # $example off$ + + sc.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index ea6a22dbfe824..59a46cb283c49 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -63,7 +63,7 @@ # |-- age: long (nullable = true) # |-- name: string (nullable = true) - # Register this DataFrame as a table. + # Register this DataFrame as a temporary table. people.registerTempTable("people") # SQL statements can be run by using the sql methods provided by sqlContext diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index 1ba5e9fb78993..588cbfee14baf 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -33,13 +33,14 @@ from pyspark import SparkContext from pyspark.streaming import StreamingContext -from pyspark.sql import SQLContext, Row +from pyspark.sql import Row, SparkSession -def getSqlContextInstance(sparkContext): - if ('sqlContextSingletonInstance' not in globals()): - globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) - return globals()['sqlContextSingletonInstance'] +def getSparkSessionInstance(sparkConf): + if ('sparkSessionSingletonInstance' not in globals()): + globals()['sparkSessionSingletonInstance'] =\ + SparkSession.builder.config(conf=sparkConf).getOrCreate() + return globals()['sparkSessionSingletonInstance'] if __name__ == "__main__": @@ -60,19 +61,19 @@ def process(time, rdd): print("========= %s =========" % str(time)) try: - # Get the singleton instance of SQLContext - sqlContext = getSqlContextInstance(rdd.context) + # Get the singleton instance of SparkSession + spark = getSparkSessionInstance(rdd.context.getConf()) # Convert RDD[String] to RDD[Row] to DataFrame rowRdd = rdd.map(lambda w: Row(word=w)) - wordsDataFrame = sqlContext.createDataFrame(rowRdd) + wordsDataFrame = spark.createDataFrame(rowRdd) # Register as table wordsDataFrame.registerTempTable("words") # Do word count on table using SQL and print it wordCountsDataFrame = \ - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() except: pass diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala index 21f58ddf3cfb7..3795af83094a6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.regression.AFTSurvivalRegression import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * An example for AFTSurvivalRegression. @@ -31,12 +30,10 @@ import org.apache.spark.sql.SQLContext object AFTSurvivalRegressionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("AFTSurvivalRegressionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("AFTSurvivalRegressionExample").getOrCreate() // $example on$ - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (1.218, 1.0, Vectors.dense(1.560, -0.605)), (2.949, 0.0, Vectors.dense(0.346, 2.158)), (3.627, 0.0, Vectors.dense(1.380, 0.231)), @@ -56,7 +53,7 @@ object AFTSurvivalRegressionExample { model.transform(training).show(false) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index a79e15c767e1f..41750ca779146 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.recommendation.ALS // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession // $example on$ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType @@ -43,13 +42,11 @@ object ALSExample { // $example off$ def main(args: Array[String]) { - val conf = new SparkConf().setAppName("ALSExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("ALSExample").getOrCreate() + import spark.implicits._ // $example on$ - val ratings = sc.textFile("data/mllib/als/sample_movielens_ratings.txt") + val ratings = spark.read.text("data/mllib/als/sample_movielens_ratings.txt") .map(Rating.parseRating) .toDF() val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2)) @@ -75,7 +72,8 @@ object ALSExample { val rmse = evaluator.evaluate(predictions) println(s"Root-mean-square error = $rmse") // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala index 2ed8101c133cf..93c153f923232 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Binarizer // $example off$ -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} object BinarizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("BinarizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() // $example on$ val data = Array((0, 0.1), (1, 0.8), (2, 0.2)) - val dataFrame: DataFrame = sqlContext.createDataFrame(data).toDF("label", "feature") + val dataFrame: DataFrame = spark.createDataFrame(data).toDF("label", "feature") val binarizer: Binarizer = new Binarizer() .setInputCol("feature") @@ -42,7 +39,8 @@ object BinarizerExample { val binarizedFeatures = binarizedDataFrame.select("binarized_feature") binarizedFeatures.collect().foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala index 6f6236a2b0588..779ad33dbda0a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala @@ -18,23 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Bucketizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object BucketizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("BucketizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() // $example on$ val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) val data = Array(-0.5, -0.3, 0.0, 0.2) - val dataFrame = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val dataFrame = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val bucketizer = new Bucketizer() .setInputCol("features") @@ -45,7 +42,7 @@ object BucketizerExample { val bucketedData = bucketizer.transform(dataFrame) bucketedData.show() // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 2be61537e613a..84ca1f0b564d2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -18,20 +18,16 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.ChiSqSelector import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object ChiSqSelectorExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("ChiSqSelectorExample") - val sc = new SparkContext(conf) - - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() + import spark.implicits._ // $example on$ val data = Seq( @@ -40,7 +36,7 @@ object ChiSqSelectorExample { (9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) ) - val df = sc.parallelize(data).toDF("id", "features", "clicked") + val df = spark.createDataset(data).toDF("id", "features", "clicked") val selector = new ChiSqSelector() .setNumTopFeatures(1) @@ -51,7 +47,7 @@ object ChiSqSelectorExample { val result = selector.fit(df).transform(df) result.show() // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index 7d07fc7dd113a..9ab43a48bff8e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object CountVectorizerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("CounterVectorizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("CounterVectorizerExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, Array("a", "b", "c")), (1, Array("a", "b", "b", "c", "a")) )).toDF("id", "words") @@ -51,6 +48,8 @@ object CountVectorizerExample { cvModel.transform(df).select("features").show() // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala index dc26b55a768a7..b415333c710c7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala @@ -18,18 +18,15 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.DCT import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DCTExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DCTExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("DCTExample").getOrCreate() // $example on$ val data = Seq( @@ -37,7 +34,7 @@ object DCTExample { Vectors.dense(-1.0, 2.0, 4.0, -7.0), Vectors.dense(14.0, -2.0, -5.0, 1.0)) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val dct = new DCT() .setInputCol("features") @@ -47,7 +44,8 @@ object DCTExample { val dctDf = dct.transform(df) dctDf.select("featuresDCT").show(3) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 7e608a281203e..2f892f8d72c1a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -23,11 +23,10 @@ import java.io.File import com.google.common.io.Files import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} /** * An example of how to use [[org.apache.spark.sql.DataFrame]] for ML. Run with @@ -62,14 +61,11 @@ object DataFrameExample { } def run(params: Params) { - - val conf = new SparkConf().setAppName(s"DataFrameExample with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"DataFrameExample with $params").getOrCreate() // Load input data println(s"Loading LIBSVM file with UDT from ${params.input}.") - val df: DataFrame = sqlContext.read.format("libsvm").load(params.input).cache() + val df: DataFrame = spark.read.format("libsvm").load(params.input).cache() println("Schema from LIBSVM:") df.printSchema() println(s"Loaded training data as a DataFrame with ${df.count()} records.") @@ -94,11 +90,11 @@ object DataFrameExample { // Load the records back. println(s"Loading Parquet file with UDT from $outputDir.") - val newDF = sqlContext.read.parquet(outputDir) + val newDF = spark.read.parquet(outputDir) println(s"Schema from Parquet:") newDF.printSchema() - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala index 224d8da5f0ec3..a0a2e1fb33dce 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.DecisionTreeClassificationModel @@ -26,16 +25,14 @@ import org.apache.spark.ml.classification.DecisionTreeClassifier import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DecisionTreeClassificationExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DecisionTreeClassificationExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("DecisionTreeClassificationExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -88,6 +85,8 @@ object DecisionTreeClassificationExample { val treeModel = model.stages(2).asInstanceOf[DecisionTreeClassificationModel] println("Learned classification tree model:\n" + treeModel.toDebugString) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index d2560cc00ba07..cea1d801aa73a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -33,7 +33,7 @@ import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.evaluation.{MulticlassMetrics, RegressionMetrics} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for decision trees. Run with @@ -134,18 +134,18 @@ object DecisionTreeExample { /** Load a dataset from the given path, using the given format */ private[ml] def loadData( - sqlContext: SQLContext, + spark: SparkSession, path: String, format: String, expectedNumFeatures: Option[Int] = None): DataFrame = { - import sqlContext.implicits._ + import spark.implicits._ format match { - case "dense" => MLUtils.loadLabeledPoints(sqlContext.sparkContext, path).toDF() + case "dense" => MLUtils.loadLabeledPoints(spark.sparkContext, path).toDF() case "libsvm" => expectedNumFeatures match { - case Some(numFeatures) => sqlContext.read.option("numFeatures", numFeatures.toString) + case Some(numFeatures) => spark.read.option("numFeatures", numFeatures.toString) .format("libsvm").load(path) - case None => sqlContext.read.format("libsvm").load(path) + case None => spark.read.format("libsvm").load(path) } case _ => throw new IllegalArgumentException(s"Bad data format: $format") } @@ -167,17 +167,17 @@ object DecisionTreeExample { testInput: String, algo: String, fracTest: Double): (DataFrame, DataFrame) = { - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.getOrCreate() // Load training data - val origExamples: DataFrame = loadData(sqlContext, input, dataFormat) + val origExamples: DataFrame = loadData(spark, input, dataFormat) // Load or create test set val dataframes: Array[DataFrame] = if (testInput != "") { // Load testInput. val numFeatures = origExamples.first().getAs[Vector](1).size val origTestExamples: DataFrame = - loadData(sqlContext, testInput, dataFormat, Some(numFeatures)) + loadData(spark, testInput, dataFormat, Some(numFeatures)) Array(origExamples, origTestExamples) } else { // Split input into training, test. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala index ad32e5635a3ea..26b52d0489e1c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator @@ -26,17 +25,15 @@ import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.regression.DecisionTreeRegressor // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object DecisionTreeRegressionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("DecisionTreeRegressionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("DecisionTreeRegressionExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Here, we treat features with > 4 distinct values as continuous. @@ -78,6 +75,8 @@ object DecisionTreeRegressionExample { val treeModel = model.stages(1).asInstanceOf[DecisionTreeRegressionModel] println("Learned regression tree model:\n" + treeModel.toDebugString) // $example off$ + + spark.stop() } } // scalastyle:on println 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 8d127f9b35420..2aa1ab1ec855f 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 @@ -18,13 +18,12 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.{ClassificationModel, Classifier, ClassifierParams} import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util.Identifiable import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql.{Dataset, Row, SparkSession} /** * A simple example demonstrating how to write your own learning algorithm using Estimator, @@ -38,13 +37,11 @@ import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} object DeveloperApiExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("DeveloperApiExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("DeveloperApiExample").getOrCreate() + import spark.implicits._ // Prepare training data. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -62,13 +59,13 @@ object DeveloperApiExample { val model = lr.fit(training.toDF()) // Prepare test data. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) // Make predictions on test data. - val sumPredictions: Double = model.transform(test.toDF()) + val sumPredictions: Double = model.transform(test) .select("features", "label", "prediction") .collect() .map { case Row(features: Vector, label: Double, prediction: Double) => @@ -77,7 +74,7 @@ object DeveloperApiExample { assert(sumPredictions == 0.0, "MyLogisticRegression predicted something other than 0, even though all coefficients are 0!") - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala index 629d322c4357f..f289c28df9b53 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -18,22 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.ElementwiseProduct import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object ElementwiseProductExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ElementwiseProductExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() // $example on$ // Create some vector data; also works for sparse vectors - val dataFrame = sqlContext.createDataFrame(Seq( + val dataFrame = spark.createDataFrame(Seq( ("a", Vectors.dense(1.0, 2.0, 3.0)), ("b", Vectors.dense(4.0, 5.0, 6.0)))).toDF("id", "vector") @@ -46,7 +43,8 @@ object ElementwiseProductExample { // Batch transform the vectors to create new column: transformer.transform(dataFrame).show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala index 65e3c365abb3f..91076ccbc14d4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -18,25 +18,22 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.param.ParamMap import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object EstimatorTransformerParamExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("EstimatorTransformerParamExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() // $example on$ // Prepare training data from a list of (label, features) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (1.0, Vectors.dense(0.0, 1.1, 0.1)), (0.0, Vectors.dense(2.0, 1.0, -1.0)), (0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -76,7 +73,7 @@ object EstimatorTransformerParamExample { println("Model 2 was fit using parameters: " + model2.parent.extractParamMap) // Prepare test data. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (1.0, Vectors.dense(-1.0, 1.5, 1.3)), (0.0, Vectors.dense(3.0, 2.0, -0.1)), (1.0, Vectors.dense(0.0, 2.2, -1.5)) @@ -94,7 +91,7 @@ object EstimatorTransformerParamExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala index cd62a803820cf..412c54db7d5c1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier} import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object GradientBoostedTreeClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("GradientBoostedTreeClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("GradientBoostedTreeClassifierExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -91,7 +88,7 @@ object GradientBoostedTreeClassifierExample { println("Learned classification GBT model:\n" + gbtModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala index b8cf9629bbdab..fd43553cc69df 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object GradientBoostedTreeRegressorExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("GradientBoostedTreeRegressorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("GradientBoostedTreeRegressorExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -79,7 +76,7 @@ object GradientBoostedTreeRegressorExample { println("Learned regression GBT model:\n" + gbtModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala index 4cea09ba12656..d873618726c6e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala @@ -18,21 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{IndexToString, StringIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object IndexToStringExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("IndexToStringExample") - val sc = new SparkContext(conf) - - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a"), (1, "b"), (2, "c"), @@ -54,7 +50,8 @@ object IndexToStringExample { val converted = converter.transform(indexed) converted.select("id", "originalCategory").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala index 7af011571f76e..d2573fad3596f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala @@ -19,11 +19,10 @@ package org.apache.spark.examples.ml // scalastyle:off println -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.clustering.KMeans import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} // $example off$ /** @@ -37,13 +36,11 @@ object KMeansExample { def main(args: Array[String]): Unit = { // Creates a Spark context and a SQL context - val conf = new SparkConf().setAppName(s"${this.getClass.getSimpleName}") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() // $example on$ // Crates a DataFrame - val dataset: DataFrame = sqlContext.createDataFrame(Seq( + val dataset: DataFrame = spark.createDataFrame(Seq( (1, Vectors.dense(0.0, 0.0, 0.0)), (2, Vectors.dense(0.1, 0.1, 0.1)), (3, Vectors.dense(0.2, 0.2, 0.2)), @@ -64,7 +61,7 @@ object KMeansExample { model.clusterCenters.foreach(println) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala index f9ddac77090ec..c23adee1a3ead 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala @@ -18,11 +18,10 @@ package org.apache.spark.examples.ml // scalastyle:off println -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.clustering.LDA import org.apache.spark.mllib.linalg.{Vectors, VectorUDT} -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.types.{StructField, StructType} // $example off$ @@ -41,16 +40,14 @@ object LDAExample { val input = "data/mllib/sample_lda_data.txt" // Creates a Spark context and a SQL context - val conf = new SparkConf().setAppName(s"${this.getClass.getSimpleName}") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() // $example on$ // Loads data - val rowRDD = sc.textFile(input).filter(_.nonEmpty) + val rowRDD = spark.read.text(input).rdd.filter(_.nonEmpty) .map(_.split(" ").map(_.toDouble)).map(Vectors.dense).map(Row(_)) val schema = StructType(Array(StructField(FEATURES_COL, new VectorUDT, false))) - val dataset = sqlContext.createDataFrame(rowRDD, schema) + val dataset = spark.createDataFrame(rowRDD, schema) // Trains a LDA model val lda = new LDA() @@ -71,7 +68,7 @@ object LDAExample { transformed.show(false) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala index f68aef708201c..cb6e2492f513f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala @@ -18,22 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.regression.LinearRegression // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object LinearRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LinearRegressionWithElasticNetExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("LinearRegressionWithElasticNetExample").getOrCreate() // $example on$ // Load training data - val training = sqlContext.read.format("libsvm") + val training = spark.read.format("libsvm") .load("data/mllib/sample_linear_regression_data.txt") val lr = new LinearRegression() @@ -56,7 +53,7 @@ object LinearRegressionWithElasticNetExample { println(s"r2: ${trainingSummary.r2}") // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala index 89c5edf1ace9c..50670d7b38416 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala @@ -18,23 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.{BinaryLogisticRegressionSummary, LogisticRegression} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions.max object LogisticRegressionSummaryExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LogisticRegressionSummaryExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("LogisticRegressionSummaryExample").getOrCreate() + import spark.implicits._ // Load training data - val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val lr = new LogisticRegression() .setMaxIter(10) @@ -71,7 +68,7 @@ object LogisticRegressionSummaryExample { lrModel.setThreshold(bestThreshold) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala index 6e27571f1dc16..fcba813d5be4b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala @@ -18,22 +18,20 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.LogisticRegression // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object LogisticRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LogisticRegressionWithElasticNetExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder.appName("LogisticRegressionWithElasticNetExample").getOrCreate() // $example on$ // Load training data - val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val lr = new LogisticRegression() .setMaxIter(10) @@ -47,7 +45,7 @@ object LogisticRegressionWithElasticNetExample { println(s"Coefficients: ${lrModel.coefficients} Intercept: ${lrModel.intercept}") // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala index aafb5efd698e4..896d8fadbe67c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala @@ -15,23 +15,19 @@ * limitations under the License. */ -// scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.MaxAbsScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object MaxAbsScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MaxAbsScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new MaxAbsScaler() .setInputCol("features") .setOutputCol("scaledFeatures") @@ -43,7 +39,7 @@ object MaxAbsScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala index 9a03f69f5af03..bcdca0fa0440d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.MinMaxScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object MinMaxScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MinMaxScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new MinMaxScaler() .setInputCol("features") @@ -44,7 +41,8 @@ object MinMaxScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala index d1441b5497a86..5fb3536060c91 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression @@ -28,7 +27,7 @@ import org.apache.spark.ml.tuning.{CrossValidator, ParamGridBuilder} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * A simple example demonstrating model selection using CrossValidator. @@ -42,13 +41,12 @@ import org.apache.spark.sql.SQLContext object ModelSelectionViaCrossValidationExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ModelSelectionViaCrossValidationExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder.appName("ModelSelectionViaCrossValidationExample").getOrCreate() // $example on$ // Prepare training data from a list of (id, text, label) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -98,7 +96,7 @@ object ModelSelectionViaCrossValidationExample { val cvModel = cv.fit(training) // Prepare test documents, which are unlabeled (id, text) tuples. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -114,7 +112,7 @@ object ModelSelectionViaCrossValidationExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala index fcad17a817580..6bc082982c894 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala @@ -17,13 +17,12 @@ package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * A simple example demonstrating model selection using TrainValidationSplit. @@ -36,13 +35,12 @@ import org.apache.spark.sql.SQLContext object ModelSelectionViaTrainValidationSplitExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("ModelSelectionViaTrainValidationSplitExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder.appName("ModelSelectionViaTrainValidationSplitExample").getOrCreate() // $example on$ // Prepare training and test data. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) val lr = new LinearRegression() @@ -75,6 +73,6 @@ object ModelSelectionViaTrainValidationSplitExample { .show() // $example off$ - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala index d7d1e82f6f849..a11fe1b4b2627 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.classification.MultilayerPerceptronClassifier import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * An example for Multilayer Perceptron Classification. @@ -31,13 +30,11 @@ import org.apache.spark.sql.SQLContext object MultilayerPerceptronClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("MultilayerPerceptronClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("MultilayerPerceptronClassifierExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm") + val data = spark.read.format("libsvm") .load("data/mllib/sample_multiclass_classification_data.txt") // Split the data into train and test val splits = data.randomSplit(Array(0.6, 0.4), seed = 1234L) @@ -63,7 +60,7 @@ object MultilayerPerceptronClassifierExample { println("Precision:" + evaluator.evaluate(predictionAndLabels)) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala index 77b913aaa3fa0..1b71a398905e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.NGram // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NGramExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NGramExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("NGramExample").getOrCreate() // $example on$ - val wordDataFrame = sqlContext.createDataFrame(Seq( + val wordDataFrame = spark.createDataFrame(Seq( (0, Array("Hi", "I", "heard", "about", "Spark")), (1, Array("I", "wish", "Java", "could", "use", "case", "classes")), (2, Array("Logistic", "regression", "models", "are", "neat")) @@ -41,7 +38,8 @@ object NGramExample { val ngramDataFrame = ngram.transform(wordDataFrame) ngramDataFrame.take(3).map(_.getAs[Stream[String]]("ngrams").toList).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala index 5ea1270c9781c..8d54555cd3c65 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ -import org.apache.spark.ml.classification.{NaiveBayes} +import org.apache.spark.ml.classification.NaiveBayes import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NaiveBayesExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NaiveBayesExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("NaiveBayesExample").getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Split the data into training and test sets (30% held out for testing) val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) @@ -53,6 +50,8 @@ object NaiveBayesExample { val precision = evaluator.evaluate(predictions) println("Precision:" + precision) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala index 6b33c16c74037..4622d69ef98cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Normalizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object NormalizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("NormalizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Normalize each Vector using $L^1$ norm. val normalizer = new Normalizer() @@ -46,7 +43,8 @@ object NormalizerExample { val lInfNormData = normalizer.transform(dataFrame, normalizer.p -> Double.PositiveInfinity) lInfNormData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala index cb9fe65a85e86..338436100c5ce 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object OneHotEncoderExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("OneHotEncoderExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, "a"), (1, "b"), (2, "c"), @@ -52,7 +49,8 @@ object OneHotEncoderExample { val encoded = encoder.transform(indexed) encoded.select("id", "categoryVec").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index 0b5d31c0ff90d..e2351c682d753 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit.{NANOSECONDS => NANO} import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.classification.{LogisticRegression, OneVsRest} @@ -31,7 +30,7 @@ import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.DataFrame // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * An example runner for Multiclass to Binary Reduction with One Vs Rest. @@ -110,18 +109,16 @@ object OneVsRestExample { } private def run(params: Params) { - val conf = new SparkConf().setAppName(s"OneVsRestExample with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName(s"OneVsRestExample with $params").getOrCreate() // $example on$ - val inputData = sqlContext.read.format("libsvm").load(params.input) + val inputData = spark.read.format("libsvm").load(params.input) // compute the train/test split: if testInput is not provided use part of input. val data = params.testInput match { case Some(t) => // compute the number of features in the training set. val numFeatures = inputData.first().getAs[Vector](1).size - val testData = sqlContext.read.option("numFeatures", numFeatures.toString) + val testData = spark.read.option("numFeatures", numFeatures.toString) .format("libsvm").load(t) Array[DataFrame](inputData, testData) case None => @@ -175,7 +172,7 @@ object OneVsRestExample { println(fprs.map {case (label, fpr) => label + "\t" + fpr}.mkString("\n")) // $example off$ - sc.stop() + spark.stop() } private def time[R](block: => R): (Long, R) = { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala index 535652ec6c793..14394d562468b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala @@ -18,18 +18,15 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.PCA import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PCAExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PCAExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("PCAExample").getOrCreate() // $example on$ val data = Array( @@ -37,7 +34,7 @@ object PCAExample { Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) ) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val pca = new PCA() .setInputCol("features") .setOutputCol("pcaFeatures") @@ -47,7 +44,8 @@ object PCAExample { val result = pcaDF.select("pcaFeatures") result.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala index 6c29063626bac..61b34aebd9c32 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -18,7 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.{Pipeline, PipelineModel} import org.apache.spark.ml.classification.LogisticRegression @@ -26,18 +25,16 @@ import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PipelineExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PipelineExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("PipelineExample").getOrCreate() // $example on$ // Prepare training documents from a list of (id, text, label) tuples. - val training = sqlContext.createDataFrame(Seq( + val training = spark.createDataFrame(Seq( (0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), @@ -71,7 +68,7 @@ object PipelineExample { val sameModel = PipelineModel.load("/tmp/spark-logistic-regression-model") // Prepare test documents, which are unlabeled (id, text) tuples. - val test = sqlContext.createDataFrame(Seq( + val test = spark.createDataFrame(Seq( (4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), @@ -87,7 +84,7 @@ object PipelineExample { } // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala index 3014008ea0ce4..4d8c672a55c34 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -18,18 +18,15 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.PolynomialExpansion import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object PolynomialExpansionExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("PolynomialExpansionExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() // $example on$ val data = Array( @@ -37,7 +34,7 @@ object PolynomialExpansionExample { Vectors.dense(0.0, 0.0), Vectors.dense(0.6, -1.1) ) - val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") .setOutputCol("polyFeatures") @@ -45,7 +42,8 @@ object PolynomialExpansionExample { val polyDF = polynomialExpansion.transform(df) polyDF.select("polyFeatures").take(3).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala index e64e673a485ed..0839c609f1d95 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -15,25 +15,21 @@ * limitations under the License. */ -// scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.QuantileDiscretizer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object QuantileDiscretizerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("QuantileDiscretizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("QuantileDiscretizerExample").getOrCreate() + import spark.implicits._ // $example on$ val data = Array((0, 18.0), (1, 19.0), (2, 8.0), (3, 5.0), (4, 2.2)) - val df = sc.parallelize(data).toDF("id", "hour") + val df = spark.createDataFrame(data).toDF("id", "hour") val discretizer = new QuantileDiscretizer() .setInputCol("hour") @@ -43,7 +39,7 @@ object QuantileDiscretizerExample { val result = discretizer.fit(df).transform(df) result.show() // $example off$ - sc.stop() + + spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala index bec831d51c581..699b621db90af 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.RFormula // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RFormulaExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RFormulaExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() // $example on$ - val dataset = sqlContext.createDataFrame(Seq( + val dataset = spark.createDataFrame(Seq( (7, "US", 18, 1.0), (8, "CA", 12, 0.0), (9, "NZ", 15, 0.0) @@ -43,7 +40,8 @@ object RFormulaExample { val output = formula.fit(dataset).transform(dataset) output.select("features", "label").show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala index 6c9b52cf259e6..4192a9c737045 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RandomForestClassifierExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RandomForestClassifierExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RandomForestClassifierExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. @@ -91,7 +88,7 @@ object RandomForestClassifierExample { println("Learned classification forest model:\n" + rfModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala index 4d2db017f346f..5632f0419a80c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala @@ -18,24 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.evaluation.RegressionEvaluator import org.apache.spark.ml.feature.VectorIndexer import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RandomForestRegressorExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RandomForestRegressorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RandomForestRegressorExample").getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Automatically identify categorical features, and index them. // Set maxCategories so features with > 4 distinct values are treated as continuous. @@ -78,7 +75,7 @@ object RandomForestRegressorExample { println("Learned regression forest model:\n" + rfModel.toDebugString) // $example off$ - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala index 202925acadff2..f03b29ba327c6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.SQLTransformer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object SQLTransformerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SQLTransformerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame( + val df = spark.createDataFrame( Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") val sqlTrans = new SQLTransformer().setStatement( @@ -39,6 +36,8 @@ object SQLTransformerExample { sqlTrans.transform(df).show() // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index f4d1fe57856a1..dff771950715a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.param.ParamMap import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -35,15 +34,13 @@ import org.apache.spark.sql.{Row, SQLContext} object SimpleParamsExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleParamsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("SimpleParamsExample").getOrCreate() + import spark.implicits._ // Prepare training data. // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes // into DataFrames, where it uses the case class metadata to infer the schema. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -59,7 +56,7 @@ object SimpleParamsExample { .setRegParam(0.01) // Learn a LogisticRegression model. This uses the parameters stored in lr. - val model1 = lr.fit(training.toDF()) + val model1 = lr.fit(training) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this @@ -82,7 +79,7 @@ object SimpleParamsExample { println("Model 2 was fit using parameters: " + model2.parent.extractParamMap()) // Prepare test data. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) @@ -91,14 +88,14 @@ object SimpleParamsExample { // LogisticRegressionModel.transform will only use the 'features' column. // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. - model2.transform(test.toDF()) + model2.transform(test) .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => println(s"($features, $label) -> prob=$prob, prediction=$prediction") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 960280137cbf9..05199007f015a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -20,12 +20,11 @@ package org.apache.spark.examples.ml import scala.beans.BeanInfo -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} @BeanInfo case class LabeledDocument(id: Long, text: String, label: Double) @@ -43,13 +42,11 @@ case class Document(id: Long, text: String) object SimpleTextClassificationPipeline { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() + import spark.implicits._ // Prepare training documents, which are labeled. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledDocument(0L, "a b c d e spark", 1.0), LabeledDocument(1L, "b d", 0.0), LabeledDocument(2L, "spark f g h", 1.0), @@ -73,7 +70,7 @@ object SimpleTextClassificationPipeline { val model = pipeline.fit(training.toDF()) // Prepare test documents, which are unlabeled. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( Document(4L, "spark i j k"), Document(5L, "l m n"), Document(6L, "spark hadoop spark"), @@ -87,7 +84,7 @@ object SimpleTextClassificationPipeline { println(s"($id, $text) --> prob=$prob, prediction=$prediction") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala index e3439677e78d6..55f777c6e228a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StandardScaler // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StandardScalerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StandardScalerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() // $example on$ - val dataFrame = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val scaler = new StandardScaler() .setInputCol("features") @@ -46,7 +43,8 @@ object StandardScalerExample { val scaledData = scalerModel.transform(dataFrame) scaledData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala index 8199be12c155b..85e79c8cb38ae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala @@ -18,31 +18,29 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StopWordsRemover // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StopWordsRemoverExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StopWordsRemoverExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() // $example on$ val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = spark.createDataFrame(Seq( (0, Seq("I", "saw", "the", "red", "baloon")), (1, Seq("Mary", "had", "a", "little", "lamb")) )).toDF("id", "raw") remover.transform(dataSet).show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala index 3f0e870c8dc6b..e01a768da988d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.StringIndexer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object StringIndexerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("StringIndexerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() // $example on$ - val df = sqlContext.createDataFrame( + val df = spark.createDataFrame( Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) ).toDF("id", "category") @@ -42,7 +39,8 @@ object StringIndexerExample { val indexed = indexer.fit(df).transform(df) indexed.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index 396f073e6b322..910ef62a2670a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{HashingTF, IDF, Tokenizer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object TfIdfExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("TfIdfExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() // $example on$ - val sentenceData = sqlContext.createDataFrame(Seq( + val sentenceData = spark.createDataFrame(Seq( (0, "Hi I heard about Spark"), (0, "I wish Java could use case classes"), (1, "Logistic regression models are neat") @@ -50,6 +47,8 @@ object TfIdfExample { val rescaledData = idfModel.transform(featurizedData) rescaledData.select("features", "label").take(3).foreach(println) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala index c667728d6326d..4f0c47b3c84c7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.{RegexTokenizer, Tokenizer} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object TokenizerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("TokenizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() // $example on$ - val sentenceDataFrame = sqlContext.createDataFrame(Seq( + val sentenceDataFrame = spark.createDataFrame(Seq( (0, "Hi I heard about Spark"), (1, "I wish Java could use case classes"), (2, "Logistic,regression,models,are,neat") @@ -48,7 +45,8 @@ object TokenizerExample { val regexTokenized = regexTokenizer.transform(sentenceDataFrame) regexTokenized.select("words", "label").take(3).foreach(println) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala index 768a8c0690477..56b7263b192e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.VectorAssembler import org.apache.spark.mllib.linalg.Vectors // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorAssemblerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorAssemblerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() // $example on$ - val dataset = sqlContext.createDataFrame( + val dataset = spark.createDataFrame( Seq((0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0)) ).toDF("id", "hour", "mobile", "userFeatures", "clicked") @@ -43,7 +40,8 @@ object VectorAssemblerExample { val output = assembler.transform(dataset) println(output.select("features", "clicked").first()) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala index 3bef37ba360b9..214ad91634ee3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala @@ -18,20 +18,17 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.VectorIndexer // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorIndexerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorIndexerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() // $example on$ - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") val indexer = new VectorIndexer() .setInputCol("features") @@ -48,7 +45,8 @@ object VectorIndexerExample { val indexedData = indexerModel.transform(data) indexedData.show() // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala index 01377d80e7e5c..716bf023a8087 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -18,31 +18,29 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ +import java.util.Arrays + import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} import org.apache.spark.ml.feature.VectorSlicer import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.Row import org.apache.spark.sql.types.StructType // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object VectorSlicerExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("VectorSlicerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() // $example on$ - val data = Array(Row(Vectors.dense(-2.0, 2.3, 0.0))) + val data = Arrays.asList(Row(Vectors.dense(-2.0, 2.3, 0.0))) val defaultAttr = NumericAttribute.defaultAttr val attrs = Array("f1", "f2", "f3").map(defaultAttr.withName) val attrGroup = new AttributeGroup("userFeatures", attrs.asInstanceOf[Array[Attribute]]) - val dataRDD = sc.parallelize(data) - val dataset = sqlContext.createDataFrame(dataRDD, StructType(Array(attrGroup.toStructField()))) + val dataset = spark.createDataFrame(data, StructType(Array(attrGroup.toStructField()))) val slicer = new VectorSlicer().setInputCol("userFeatures").setOutputCol("features") @@ -52,7 +50,8 @@ object VectorSlicerExample { val output = slicer.transform(dataset) println(output.select("userFeatures", "features").first()) // $example off$ - sc.stop() + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala index e77aa59ba32b2..292b6d9f77763 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala @@ -18,21 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.ml.feature.Word2Vec // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object Word2VecExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("Word2Vec example") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("Word2Vec example").getOrCreate() // $example on$ // Input data: Each row is a bag of words from a sentence or document. - val documentDF = sqlContext.createDataFrame(Seq( + val documentDF = spark.createDataFrame(Seq( "Hi I heard about Spark".split(" "), "I wish Java could use case classes".split(" "), "Logistic regression models are neat".split(" ") @@ -48,6 +45,8 @@ object Word2VecExample { val result = model.transform(documentDF) result.select("result").take(3).foreach(println) // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index e89d555884dd0..c2bf1548b5685 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -27,7 +27,7 @@ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel, Regex import org.apache.spark.mllib.clustering.{DistributedLDAModel, EMLDAOptimizer, LDA, OnlineLDAOptimizer} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * An example Latent Dirichlet Allocation (LDA) app. Run with @@ -189,8 +189,8 @@ object LDAExample { vocabSize: Int, stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.getOrCreate() + import spark.implicits._ // Get dataset of document texts // One document per line in each text file. If the input consists of many small files, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala index fdb01b86dd787..cd4f0bb0de601 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -18,22 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.mllib -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.evaluation.{RankingMetrics, RegressionMetrics} import org.apache.spark.mllib.recommendation.{ALS, Rating} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RankingMetricsExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("RankingMetricsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.appName("RankingMetricsExample").getOrCreate() + import spark.implicits._ // $example on$ // Read in the ratings data - val ratings = sc.textFile("data/mllib/sample_movielens_data.txt").map { line => + val ratings = spark.read.text("data/mllib/sample_movielens_data.txt").rdd.map { line => val fields = line.split("::") Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble - 2.5) }.cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala index add634c957b40..22c47a694d346 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -18,22 +18,22 @@ package org.apache.spark.examples.mllib -import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} // $example off$ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession object RegressionMetricsExample { def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("RegressionMetricsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder.appName("RegressionMetricsExample").getOrCreate() // $example on$ // Load the data - val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_linear_regression_data.txt").cache() + val data = spark + .read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + .rdd.map(row => LabeledPoint(row.getDouble(0), row.get(1).asInstanceOf[Vector])) + .cache() // Build the model val numIterations = 100 @@ -61,6 +61,8 @@ object RegressionMetricsExample { // Explained variance println(s"Explained variance = ${metrics.explainedVariance}") // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 918e124065e4c..2f0fe704f709b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -19,9 +19,8 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext, Time} @@ -60,9 +59,9 @@ object SqlNetworkWordCount { // Convert RDDs of the words DStream to DataFrame and run SQL query words.foreachRDD { (rdd: RDD[String], time: Time) => - // Get the singleton instance of SQLContext - val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) - import sqlContext.implicits._ + // Get the singleton instance of SparkSession + val spark = SparkSessionSingleton.getInstance(rdd.sparkContext.getConf) + import spark.implicits._ // Convert RDD[String] to RDD[case class] to DataFrame val wordsDataFrame = rdd.map(w => Record(w)).toDF() @@ -72,7 +71,7 @@ object SqlNetworkWordCount { // Do word count on table using SQL and print it val wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") println(s"========= $time =========") wordCountsDataFrame.show() } @@ -87,14 +86,14 @@ object SqlNetworkWordCount { case class Record(word: String) -/** Lazily instantiated singleton instance of SQLContext */ -object SQLContextSingleton { +/** Lazily instantiated singleton instance of SparkSession */ +object SparkSessionSingleton { - @transient private var instance: SQLContext = _ + @transient private var instance: SparkSession = _ - def getInstance(sparkContext: SparkContext): SQLContext = { + def getInstance(sparkConf: SparkConf): SparkSession = { if (instance == null) { - instance = new SQLContext(sparkContext) + instance = SparkSession.builder.config(sparkConf).getOrCreate() } instance } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index cb15b4b91f913..aec0215b40945 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -952,6 +952,11 @@ def dump_profiles(self, path): """ self.profiler_collector.dump_profiles(path) + def getConf(self): + conf = SparkConf() + conf.setAll(self._conf.getAll()) + return conf + def _test(): import atexit From 1e7d9bfb5a41f5c2479ab3b4d4081f00bf00bd31 Mon Sep 17 00:00:00 2001 From: Sebastien Rainville Date: Wed, 4 May 2016 14:32:36 -0700 Subject: [PATCH 0070/1470] [SPARK-13001][CORE][MESOS] Prevent getting offers when reached max cores Similar to https://github.com/apache/spark/pull/8639 This change rejects offers for 120s when reached `spark.cores.max` in coarse-grained mode to mitigate offer starvation. This prevents Mesos to send us offers again and again, starving other frameworks. This is especially problematic when running many small frameworks on the same Mesos cluster, e.g. many small Sparks streaming jobs, and cause the bigger spark jobs to stop receiving offers. By rejecting the offers for a long period of time, they become available to those other frameworks. Author: Sebastien Rainville Closes #10924 from sebastienrainville/master. (cherry picked from commit eb019af9a9cadb127eab1b6d30312169ed90f808) Signed-off-by: Andrew Or --- .../mesos/CoarseMesosSchedulerBackend.scala | 53 +++++++++++++------ .../cluster/mesos/MesosSchedulerUtils.scala | 4 ++ .../CoarseMesosSchedulerBackendSuite.scala | 13 +++++ 3 files changed, 53 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 50b452c72f8aa..2c5be1f528894 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -109,10 +109,14 @@ private[spark] class CoarseMesosSchedulerBackend( private val slaveOfferConstraints = parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) - // reject offers with mismatched constraints in seconds + // Reject offers with mismatched constraints in seconds private val rejectOfferDurationForUnmetConstraints = getRejectOfferDurationForUnmetConstraints(sc) + // Reject offers when we reached the maximum number of cores for this framework + private val rejectOfferDurationForReachedMaxCores = + getRejectOfferDurationForReachedMaxCores(sc) + // A client for talking to the external shuffle service private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { if (shuffleServiceEnabled) { @@ -279,18 +283,32 @@ private[spark] class CoarseMesosSchedulerBackend( } private def declineUnmatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { - for (offer <- offers) { - val id = offer.getId.getValue - val offerAttributes = toAttributeMap(offer.getAttributesList) - val mem = getResource(offer.getResourcesList, "mem") - val cpus = getResource(offer.getResourcesList, "cpus") - val filters = Filters.newBuilder() - .setRefuseSeconds(rejectOfferDurationForUnmetConstraints).build() - - logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus" - + s" for $rejectOfferDurationForUnmetConstraints seconds") + offers.foreach { offer => + declineOffer(d, offer, Some("unmet constraints"), + Some(rejectOfferDurationForUnmetConstraints)) + } + } - d.declineOffer(offer.getId, filters) + private def declineOffer( + d: SchedulerDriver, + offer: Offer, + reason: Option[String] = None, + refuseSeconds: Option[Long] = None): Unit = { + + val id = offer.getId.getValue + val offerAttributes = toAttributeMap(offer.getAttributesList) + val mem = getResource(offer.getResourcesList, "mem") + val cpus = getResource(offer.getResourcesList, "cpus") + + logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem" + + s" cpu: $cpus for $refuseSeconds seconds" + + reason.map(r => s" (reason: $r)").getOrElse("")) + + refuseSeconds match { + case Some(seconds) => + val filters = Filters.newBuilder().setRefuseSeconds(seconds).build() + d.declineOffer(offer.getId, filters) + case _ => d.declineOffer(offer.getId) } } @@ -326,11 +344,12 @@ private[spark] class CoarseMesosSchedulerBackend( d.launchTasks( Collections.singleton(offer.getId), offerTasks.asJava) - } else { // decline - logDebug(s"Declining offer: $id with attributes: $offerAttributes " + - s"mem: $offerMem cpu: $offerCpus") - - d.declineOffer(offer.getId) + } else if (totalCoresAcquired >= maxCores) { + // Reject an offer for a configurable amount of time to avoid starving other frameworks + declineOffer(d, offer, Some("reached spark.cores.max"), + Some(rejectOfferDurationForReachedMaxCores)) + } else { + declineOffer(d, offer) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 1e322ac679419..7355ba317d9a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -352,4 +352,8 @@ private[mesos] trait MesosSchedulerUtils extends Logging { sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForUnmetConstraints", "120s") } + protected def getRejectOfferDurationForReachedMaxCores(sc: SparkContext): Long = { + sc.conf.getTimeAsSeconds("spark.mesos.rejectOfferDurationForReachedMaxCores", "120s") + } + } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala index b18f0eb162b1d..15d59e7052912 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala @@ -147,6 +147,19 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite verifyDeclinedOffer(driver, createOfferId("o1"), true) } + test("mesos declines offers with a filter when reached spark.cores.max") { + val maxCores = 3 + setBackend(Map("spark.cores.max" -> maxCores.toString)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List( + (executorMemory, maxCores + 1), + (executorMemory, maxCores + 1))) + + verifyTaskLaunched("o1") + verifyDeclinedOffer(driver, createOfferId("o2"), true) + } + test("mesos assigns tasks round-robin on offers") { val executorCores = 4 val maxCores = executorCores * 2 From 701c667296fdc8a374852ec38366c91a02350e17 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 4 May 2016 14:40:54 -0700 Subject: [PATCH 0071/1470] [SPARK-15116] In REPL we should create SparkSession first and get SparkContext from it ## What changes were proposed in this pull request? see https://github.com/apache/spark/pull/12873#discussion_r61993910. The problem is, if we create `SparkContext` first and then call `SparkSession.builder.enableHiveSupport().getOrCreate()`, we will reuse the existing `SparkContext` and the hive flag won't be set. ## How was this patch tested? verified it locally. Author: Wenchen Fan Closes #12890 from cloud-fan/repl. --- .../org/apache/spark/repl/SparkILoop.scala | 20 ++++++-------- .../apache/spark/repl/SparkILoopInit.scala | 11 +++----- .../scala/org/apache/spark/repl/Main.scala | 27 +++++++++---------- .../org/apache/spark/repl/SparkILoop.scala | 11 +++----- 4 files changed, 26 insertions(+), 43 deletions(-) diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index c4f64505a2165..b1e95d8fdb600 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1003,7 +1003,7 @@ class SparkILoop( // NOTE: Must be public for visibility @DeveloperApi - def createSparkContext(): SparkContext = { + def createSparkSession(): SparkSession = { val execUri = System.getenv("SPARK_EXECUTOR_URI") val jars = SparkILoop.getAddedJars val conf = new SparkConf() @@ -1019,22 +1019,18 @@ class SparkILoop( if (execUri != null) { conf.set("spark.executor.uri", execUri) } - sparkContext = new SparkContext(conf) - logInfo("Created spark context..") - Signaling.cancelOnInterrupt(sparkContext) - sparkContext - } - @DeveloperApi - // TODO: don't duplicate this code - def createSparkSession(): SparkSession = { - if (SparkSession.hiveClassesArePresent) { + val builder = SparkSession.builder.config(conf) + val sparkSession = if (SparkSession.hiveClassesArePresent) { logInfo("Creating Spark session with Hive support") - SparkSession.builder.enableHiveSupport().getOrCreate() + builder.enableHiveSupport().getOrCreate() } else { logInfo("Creating Spark session") - SparkSession.builder.getOrCreate() + builder.getOrCreate() } + sparkContext = sparkSession.sparkContext + Signaling.cancelOnInterrupt(sparkContext) + sparkSession } private def getMaster(): String = { diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index f1febb9497c78..29f63de8a0fa1 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -123,19 +123,14 @@ private[repl] trait SparkILoopInit { def initializeSpark() { intp.beQuietDuring { command(""" + @transient val spark = org.apache.spark.repl.Main.interp.createSparkSession() @transient val sc = { - val _sc = org.apache.spark.repl.Main.interp.createSparkContext() + val _sc = spark.sparkContext _sc.uiWebUrl.foreach(webUrl => println(s"Spark context Web UI available at ${webUrl}")) println("Spark context available as 'sc' " + s"(master = ${_sc.master}, app id = ${_sc.applicationId}).") - _sc - } - """) - command(""" - @transient val spark = { - val _session = org.apache.spark.repl.Main.interp.createSparkSession() println("Spark session available as 'spark'.") - _session + _sc } """) command("import org.apache.spark.SparkContext._") diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index a171759809256..005edda2bee76 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -71,35 +71,32 @@ object Main extends Logging { } } - def createSparkContext(): SparkContext = { + def createSparkSession(): SparkSession = { val execUri = System.getenv("SPARK_EXECUTOR_URI") conf.setIfMissing("spark.app.name", "Spark shell") - // SparkContext will detect this configuration and register it with the RpcEnv's - // file server, setting spark.repl.class.uri to the actual URI for executors to - // use. This is sort of ugly but since executors are started as part of SparkContext - // initialization in certain cases, there's an initialization order issue that prevents - // this from being set after SparkContext is instantiated. - .set("spark.repl.class.outputDir", outputDir.getAbsolutePath()) + // SparkContext will detect this configuration and register it with the RpcEnv's + // file server, setting spark.repl.class.uri to the actual URI for executors to + // use. This is sort of ugly but since executors are started as part of SparkContext + // initialization in certain cases, there's an initialization order issue that prevents + // this from being set after SparkContext is instantiated. + conf.set("spark.repl.class.outputDir", outputDir.getAbsolutePath()) if (execUri != null) { conf.set("spark.executor.uri", execUri) } if (System.getenv("SPARK_HOME") != null) { conf.setSparkHome(System.getenv("SPARK_HOME")) } - sparkContext = new SparkContext(conf) - logInfo("Created spark context..") - Signaling.cancelOnInterrupt(sparkContext) - sparkContext - } - def createSparkSession(): SparkSession = { + val builder = SparkSession.builder.config(conf) if (SparkSession.hiveClassesArePresent) { - sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate() + sparkSession = builder.enableHiveSupport().getOrCreate() logInfo("Created Spark session with Hive support") } else { - sparkSession = SparkSession.builder.getOrCreate() + sparkSession = builder.getOrCreate() logInfo("Created Spark session") } + sparkContext = sparkSession.sparkContext + Signaling.cancelOnInterrupt(sparkContext) sparkSession } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala index d74b7965316fc..bbdb992d8a5e2 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -36,19 +36,14 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) def initializeSpark() { intp.beQuietDuring { processLine(""" + @transient val spark = org.apache.spark.repl.Main.createSparkSession() @transient val sc = { - val _sc = org.apache.spark.repl.Main.createSparkContext() + val _sc = spark.sparkContext _sc.uiWebUrl.foreach(webUrl => println(s"Spark context Web UI available at ${webUrl}")) println("Spark context available as 'sc' " + s"(master = ${_sc.master}, app id = ${_sc.applicationId}).") - _sc - } - """) - processLine(""" - @transient val spark = { - val _session = org.apache.spark.repl.Main.createSparkSession() println("Spark session available as 'spark'.") - _session + _sc } """) processLine("import org.apache.spark.SparkContext._") From aca46ecf8ebc1e477cf1ca8aabf45861bf12e225 Mon Sep 17 00:00:00 2001 From: sethah Date: Wed, 4 May 2016 16:46:13 -0700 Subject: [PATCH 0072/1470] [MINOR][SQL] Fix typo in DataFrameReader csv documentation ## What changes were proposed in this pull request? Typo fix ## How was this patch tested? No tests My apologies for the tiny PR, but I stumbled across this today and wanted to get it corrected for 2.0. Author: sethah Closes #12912 from sethah/csv_typo. (cherry picked from commit b28137764716f56fa1a923c4278624a56364a505) Signed-off-by: Andrew Or --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 2d4a68f3c3a94..5bf696c1c3927 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -407,7 +407,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `header` (default `false`): uses the first line as names of columns.
  • *
  • `ignoreLeadingWhiteSpace` (default `false`): defines whether or not leading whitespaces * from values being read should be skipped.
  • - *
  • `ignoreTrailingWhiteSpace` (default `fDataFraalse`): defines whether or not trailing + *
  • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing * whitespaces from values being read should be skipped.
  • *
  • `nullValue` (default empty string): sets the string representation of a null value.
  • *
  • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
  • From fa3c5507fb7b612f78750abfd60dfdde0ad86da3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 4 May 2016 17:39:30 -0700 Subject: [PATCH 0073/1470] [SPARK-14896][SQL] Deprecate HiveContext in python ## What changes were proposed in this pull request? See title. ## How was this patch tested? PySpark tests. Author: Andrew Or Closes #12917 from andrewor14/deprecate-hive-context-python. (cherry picked from commit fa79d346e1a79ceda6ccd20e74eb850e769556ea) Signed-off-by: Andrew Or --- python/pyspark/sql/column.py | 2 -- python/pyspark/sql/context.py | 9 ++++++++- python/pyspark/sql/streaming.py | 2 +- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 43e9baece2de9..90fb76f9b5a5b 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -418,8 +418,6 @@ def over(self, window): >>> window = Window.partitionBy("name").orderBy("age").rowsBetween(-1, 1) >>> from pyspark.sql.functions import rank, min >>> # df.select(rank().over(window), min('age').over(window)) - - .. note:: Window functions is only supported with HiveContext in 1.4 """ from pyspark.sql.window import WindowSpec if not isinstance(window, WindowSpec): diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 2096236d7f36f..78ab2e81bfce2 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -17,6 +17,7 @@ from __future__ import print_function import sys +import warnings if sys.version >= '3': basestring = unicode = str @@ -434,7 +435,6 @@ def streams(self): return ContinuousQueryManager(self._ssql_ctx.streams()) -# TODO(andrew): deprecate this class HiveContext(SQLContext): """A variant of Spark SQL that integrates with data stored in Hive. @@ -444,8 +444,15 @@ class HiveContext(SQLContext): :param sparkContext: The SparkContext to wrap. :param jhiveContext: An optional JVM Scala HiveContext. If set, we do not instantiate a new :class:`HiveContext` in the JVM, instead we make all calls to this object. + + .. note:: Deprecated in 2.0.0. Use SparkSession.builder.enableHiveSupport().getOrCreate(). """ + warnings.warn( + "HiveContext is deprecated in Spark 2.0.0. Please use " + + "SparkSession.builder.enableHiveSupport().getOrCreate() instead.", + DeprecationWarning) + def __init__(self, sparkContext, jhiveContext=None): if jhiveContext is None: sparkSession = SparkSession.withHiveSupport(sparkContext) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index bf03fdca91394..8238b8e7cde6b 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -221,7 +221,7 @@ def _test(): globs['os'] = os globs['sc'] = sc globs['sqlContext'] = SQLContext(sc) - globs['hiveContext'] = HiveContext(sc) + globs['hiveContext'] = HiveContext._createForTesting(sc) globs['df'] = \ globs['sqlContext'].read.format('text').stream('python/test_support/sql/streaming') From d90359d63b49e1fd1079858c6aad9b444ba2c253 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 4 May 2016 18:27:25 -0700 Subject: [PATCH 0074/1470] [SPARK-6339][SQL] Supports CREATE TEMPORARY VIEW tableIdentifier AS query ## What changes were proposed in this pull request? This PR support new SQL syntax CREATE TEMPORARY VIEW. Like: ``` CREATE TEMPORARY VIEW viewName AS SELECT * from xx CREATE OR REPLACE TEMPORARY VIEW viewName AS SELECT * from xx CREATE TEMPORARY VIEW viewName (c1 COMMENT 'blabla', c2 COMMENT 'blabla') AS SELECT * FROM xx ``` ## How was this patch tested? Unit tests. Author: Sean Zhong Closes #12872 from clockfly/spark-6399. (cherry picked from commit 8fb1463d6a832f187f323d97635e5bec1e93c6f3) Signed-off-by: Yin Huai --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 13 ++- .../spark/sql/execution/command/views.scala | 86 ++++++++++---- .../spark/sql/hive/HiveDDLCommandSuite.scala | 2 +- .../sql/hive/execution/SQLViewSuite.scala | 110 ++++++++++++++++-- 5 files changed, 175 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 273ad9289169d..ee27d69ab3f9e 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -84,7 +84,7 @@ statement | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? tableIdentifier #dropTable - | CREATE (OR REPLACE)? VIEW (IF NOT EXISTS)? tableIdentifier + | CREATE (OR REPLACE)? TEMPORARY? VIEW (IF NOT EXISTS)? tableIdentifier identifierCommentList? (COMMENT STRING)? (PARTITIONED ON identifierList)? (TBLPROPERTIES tablePropertyList)? AS query #createView diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 60388df596464..146e036bb4843 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -935,7 +935,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * For example: * {{{ - * CREATE VIEW [IF NOT EXISTS] [db_name.]view_name + * CREATE [TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name * [(column_name [COMMENT column_comment], ...) ] * [COMMENT view_comment] * [TBLPROPERTIES (property_name = property_value, ...)] @@ -958,7 +958,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.query, Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty), ctx.EXISTS != null, - ctx.REPLACE != null + ctx.REPLACE != null, + ctx.TEMPORARY != null ) } } @@ -975,7 +976,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.query, Map.empty, allowExist = false, - replace = true) + replace = true, + isTemporary = false) } /** @@ -989,7 +991,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { query: QueryContext, properties: Map[String, String], allowExist: Boolean, - replace: Boolean): LogicalPlan = { + replace: Boolean, + isTemporary: Boolean): LogicalPlan = { val sql = Option(source(query)) val tableDesc = CatalogTable( identifier = visitTableIdentifier(name), @@ -1000,7 +1003,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { viewOriginalText = sql, viewText = sql, comment = comment) - CreateViewCommand(tableDesc, plan(query), allowExist, replace, command(ctx)) + CreateViewCommand(tableDesc, plan(query), allowExist, replace, isTemporary, command(ctx)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 0f656ef53e395..70ce5c8429010 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, Row, SparkSession} -import org.apache.spark.sql.catalyst.SQLBuilder +import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} @@ -37,6 +37,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} * already exists, throws analysis exception. * @param replace if true, and if the view already exists, updates it; if false, and if the view * already exists, throws analysis exception. + * @param isTemporary if true, the view is created as a temporary view. Temporary views are dropped + * at the end of current Spark session. Existing permanent relations with the same + * name are not visible to the current session while the temporary view exists, + * unless they are specified with full qualified table name with database prefix. * @param sql the original sql */ case class CreateViewCommand( @@ -44,6 +48,7 @@ case class CreateViewCommand( child: LogicalPlan, allowExisting: Boolean, replace: Boolean, + isTemporary: Boolean, sql: String) extends RunnableCommand { @@ -55,12 +60,23 @@ case class CreateViewCommand( require(tableDesc.tableType == CatalogTableType.VIEW) require(tableDesc.viewText.isDefined) - private val tableIdentifier = tableDesc.identifier - if (allowExisting && replace) { throw new AnalysisException("CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed.") } + // Disallows 'CREATE TEMPORARY VIEW IF NOT EXISTS' to be consistent with 'CREATE TEMPORARY TABLE' + if (allowExisting && isTemporary) { + throw new AnalysisException( + "It is not allowed to define a TEMPORARY view with IF NOT EXISTS.") + } + + // Temporary view names should NOT contain database prefix like "database.table" + if (isTemporary && tableDesc.identifier.database.isDefined) { + val database = tableDesc.identifier.database.get + throw new AnalysisException( + s"It is not allowed to add database prefix ${database} for the TEMPORARY view name.") + } + override def run(sparkSession: SparkSession): Seq[Row] = { // If the plan cannot be analyzed, throw an exception and don't proceed. val qe = sparkSession.executePlan(child) @@ -70,29 +86,59 @@ case class CreateViewCommand( require(tableDesc.schema == Nil || tableDesc.schema.length == analyzedPlan.output.length) val sessionState = sparkSession.sessionState - if (sessionState.catalog.tableExists(tableIdentifier)) { - if (allowExisting) { - // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view - // already exists. - } else if (replace) { - // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` - sessionState.catalog.alterTable(prepareTable(sparkSession, analyzedPlan)) + if (isTemporary) { + createTemporaryView(tableDesc.identifier, sparkSession, analyzedPlan) + } else { + // Adds default database for permanent table if it doesn't exist, so that tableExists() + // only check permanent tables. + val database = tableDesc.identifier.database.getOrElse( + sessionState.catalog.getCurrentDatabase) + val tableIdentifier = tableDesc.identifier.copy(database = Option(database)) + + if (sessionState.catalog.tableExists(tableIdentifier)) { + if (allowExisting) { + // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view + // already exists. + } else if (replace) { + // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` + sessionState.catalog.alterTable(prepareTable(sparkSession, analyzedPlan)) + } else { + // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already + // exists. + throw new AnalysisException( + s"View $tableIdentifier already exists. If you want to update the view definition, " + + "please use ALTER VIEW AS or CREATE OR REPLACE VIEW AS") + } } else { - // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already - // exists. - throw new AnalysisException(s"View $tableIdentifier already exists. " + - "If you want to update the view definition, please use ALTER VIEW AS or " + - "CREATE OR REPLACE VIEW AS") + // Create the view if it doesn't exist. + sessionState.catalog.createTable( + prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) } - } else { - // Create the view if it doesn't exist. - sessionState.catalog.createTable( - prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) } - Seq.empty[Row] } + private def createTemporaryView( + table: TableIdentifier, sparkSession: SparkSession, analyzedPlan: LogicalPlan): Unit = { + + val sessionState = sparkSession.sessionState + val catalog = sessionState.catalog + + // Projects column names to alias names + val logicalPlan = { + if (tableDesc.schema.isEmpty) { + analyzedPlan + } else { + val projectList = analyzedPlan.output.zip(tableDesc.schema).map { + case (attr, col) => Alias(attr, col.name)() + } + sparkSession.executePlan(Project(projectList, analyzedPlan)).analyzed + } + } + + catalog.createTempTable(table.table, logicalPlan, replace) + } + /** * Returns a [[CatalogTable]] that can be used to save in the catalog. This comment canonicalize * SQL based on the analyzed plan, and also creates the proper schema for the view. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index c4ebc604dc13c..3d74235dc52bc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -39,7 +39,7 @@ class HiveDDLCommandSuite extends PlanTest { parser.parsePlan(sql).collect { case CreateTable(desc, allowExisting) => (desc, allowExisting) case CreateTableAsSelectLogicalPlan(desc, _, allowExisting) => (desc, allowExisting) - case CreateViewCommand(desc, _, allowExisting, _, _) => (desc, allowExisting) + case CreateViewCommand(desc, _, allowExisting, _, _, _) => (desc, allowExisting) }.head } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 51848470502af..72f9fba13d4bb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -37,11 +37,21 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sqlContext.sql(s"DROP TABLE IF EXISTS jt") } - test("nested views") { - withView("jtv1", "jtv2") { - sql("CREATE VIEW jtv1 AS SELECT * FROM jt WHERE id > 3").collect() - sql("CREATE VIEW jtv2 AS SELECT * FROM jtv1 WHERE id < 6").collect() + test("nested views (interleaved with temporary views)") { + withView("jtv1", "jtv2", "jtv3", "temp_jtv1", "temp_jtv2", "temp_jtv3") { + sql("CREATE VIEW jtv1 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE VIEW jtv2 AS SELECT * FROM jtv1 WHERE id < 6") checkAnswer(sql("select count(*) FROM jtv2"), Row(2)) + + // Checks temporary views + sql("CREATE TEMPORARY VIEW temp_jtv1 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE TEMPORARY VIEW temp_jtv2 AS SELECT * FROM temp_jtv1 WHERE id < 6") + checkAnswer(sql("select count(*) FROM temp_jtv2"), Row(2)) + + // Checks interleaved temporary view and normal view + sql("CREATE TEMPORARY VIEW temp_jtv3 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE VIEW jtv3 AS SELECT * FROM temp_jtv3 WHERE id < 6") + checkAnswer(sql("select count(*) FROM jtv3"), Row(2)) } } @@ -57,6 +67,33 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("error handling: fail if the temp view name contains the database prefix") { + // Fully qualified table name like "database.table" is not allowed for temporary view + val e = intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW default.myabcdview AS SELECT * FROM jt") + } + assert(e.message.contains("It is not allowed to add database prefix")) + } + + test("error handling: disallow IF NOT EXISTS for CREATE TEMPORARY VIEW") { + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW IF NOT EXISTS myabcdview AS SELECT * FROM jt") + } + assert(e.message.contains("It is not allowed to define a TEMPORARY view with IF NOT EXISTS")) + } + + test("error handling: fail if the temp view sql itself is invalid") { + // A table that does not exist for temporary view + intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM table_not_exist1345") + } + + // A column that does not exist, for temporary view + intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT random1234 FROM jt") + } + } + test("correctly parse CREATE VIEW statement") { withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { sql( @@ -69,18 +106,70 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("correctly parse CREATE TEMPORARY VIEW statement") { + withView("testView") { + sql( + """CREATE TEMPORARY VIEW + |testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') + |TBLPROPERTIES ('a' = 'b') + |AS SELECT * FROM jt + |""".stripMargin) + checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) + } + } + + test("should NOT allow CREATE TEMPORARY VIEW when TEMPORARY VIEW with same name exists") { + withView("testView") { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + } + + assert(e.message.contains("Temporary table") && e.message.contains("already exists")) + } + } + + test("should allow CREATE TEMPORARY VIEW when a permanent VIEW with same name exists") { + withView("testView", "default.testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + } + } + + test("should allow CREATE permanent VIEW when a TEMPORARY VIEW with same name exists") { + withView("testView", "default.testView") { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + sql("CREATE VIEW testView AS SELECT id FROM jt") + } + } + test("correctly handle CREATE VIEW IF NOT EXISTS") { withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { withTable("jt2") { - sql("CREATE VIEW testView AS SELECT id FROM jt") + withView("testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") - val df = (1 until 10).map(i => i -> i).toDF("i", "j") - df.write.format("json").saveAsTable("jt2") - sql("CREATE VIEW IF NOT EXISTS testView AS SELECT * FROM jt2") + val df = (1 until 10).map(i => i -> i).toDF("i", "j") + df.write.format("json").saveAsTable("jt2") + sql("CREATE VIEW IF NOT EXISTS testView AS SELECT * FROM jt2") - // make sure our view doesn't change. + // make sure our view doesn't change. + checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) + } + } + } + } + + test(s"correctly handle CREATE OR REPLACE TEMPORARY VIEW") { + withTable("jt2") { + withView("testView") { + sql("CREATE OR REPLACE TEMPORARY VIEW testView AS SELECT id FROM jt") checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) - sql("DROP VIEW testView") + + sql("CREATE OR REPLACE TEMPORARY VIEW testView AS SELECT id AS i, id AS j FROM jt") + // make sure the view has been changed. + checkAnswer(sql("SELECT * FROM testView ORDER BY i"), (1 to 9).map(i => Row(i, i))) } } } @@ -215,5 +304,4 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } } - } From 689b0fc819a16d6013bd4bfec2063354841a916e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 4 May 2016 18:47:27 -0700 Subject: [PATCH 0075/1470] [SPARK-14993][SQL] Fix Partition Discovery Inconsistency when Input is a Path to Parquet File #### What changes were proposed in this pull request? When we load a dataset, if we set the path to ```/path/a=1```, we will not take `a` as the partitioning column. However, if we set the path to ```/path/a=1/file.parquet```, we take `a` as the partitioning column and it shows up in the schema. This PR is to fix the behavior inconsistency issue. The base path contains a set of paths that are considered as the base dirs of the input datasets. The partitioning discovery logic will make sure it will stop when it reaches any base path. By default, the paths of the dataset provided by users will be base paths. Below are three typical cases, **Case 1**```sqlContext.read.parquet("/path/something=true/")```: the base path will be `/path/something=true/`, and the returned DataFrame will not contain a column of `something`. **Case 2**```sqlContext.read.parquet("/path/something=true/a.parquet")```: the base path will be still `/path/something=true/`, and the returned DataFrame will also not contain a column of `something`. **Case 3**```sqlContext.read.parquet("/path/")```: the base path will be `/path/`, and the returned DataFrame will have the column of `something`. Users also can override the basePath by setting `basePath` in the options to pass the new base path to the data source. For example, ```sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")```, and the returned DataFrame will have the column of `something`. The related PRs: - https://github.com/apache/spark/pull/9651 - https://github.com/apache/spark/pull/10211 #### How was this patch tested? Added a couple of test cases Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #12828 from gatorsmile/readPartitionedTable. (cherry picked from commit ef55e46c9225ddceebeaf19398519cbe651c1728) Signed-off-by: Yin Huai --- .../PartitioningAwareFileCatalog.scala | 42 ++++++++----- .../ParquetPartitionDiscoverySuite.scala | 60 +++++++++++++++++++ 2 files changed, 88 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index 9d997d628579c..2c44b399cb95f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -133,23 +133,37 @@ abstract class PartitioningAwareFileCatalog( /** * Contains a set of paths that are considered as the base dirs of the input datasets. * The partitioning discovery logic will make sure it will stop when it reaches any - * base path. By default, the paths of the dataset provided by users will be base paths. - * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path - * will be `/path/something=true/`, and the returned DataFrame will not contain a column of - * `something`. If users want to override the basePath. They can set `basePath` in the options - * to pass the new base path to the data source. - * For the above example, if the user-provided base path is `/path/`, the returned + * base path. + * + * By default, the paths of the dataset provided by users will be base paths. + * Below are three typical examples, + * Case 1) `sqlContext.read.parquet("/path/something=true/")`: the base path will be + * `/path/something=true/`, and the returned DataFrame will not contain a column of `something`. + * Case 2) `sqlContext.read.parquet("/path/something=true/a.parquet")`: the base path will be + * still `/path/something=true/`, and the returned DataFrame will also not contain a column of + * `something`. + * Case 3) `sqlContext.read.parquet("/path/")`: the base path will be `/path/`, and the returned * DataFrame will have the column of `something`. + * + * Users also can override the basePath by setting `basePath` in the options to pass the new base + * path to the data source. + * For example, `sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")`, + * and the returned DataFrame will have the column of `something`. */ private def basePaths: Set[Path] = { - val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) - userDefinedBasePath.getOrElse { - // If the user does not provide basePath, we will just use paths. - paths.toSet - }.map { hdfsPath => - // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). - val fs = hdfsPath.getFileSystem(hadoopConf) - hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + parameters.get("basePath").map(new Path(_)) match { + case Some(userDefinedBasePath) => + val fs = userDefinedBasePath.getFileSystem(hadoopConf) + if (!fs.isDirectory(userDefinedBasePath)) { + throw new IllegalArgumentException("Option 'basePath' must be a directory") + } + Set(fs.makeQualified(userDefinedBasePath)) + + case None => + paths.map { path => + // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). + val qualifiedPath = path.getFileSystem(hadoopConf).makeQualified(path) + if (leafFiles.contains(qualifiedPath)) qualifiedPath.getParent else qualifiedPath }.toSet } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 5bffb307ec80e..cb2c2522b20cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -191,6 +191,29 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha checkThrows[AssertionError]("file://path/a=", "Empty partition column value") } + test("parse partition with base paths") { + // when the basePaths is the same as the path to a leaf directory + val partitionSpec1: Option[PartitionValues] = parsePartition( + path = new Path("file://path/a=10"), + defaultPartitionName = defaultPartitionName, + typeInference = true, + basePaths = Set(new Path("file://path/a=10")))._1 + + assert(partitionSpec1.isEmpty) + + // when the basePaths is the path to a base directory of leaf directories + val partitionSpec2: Option[PartitionValues] = parsePartition( + path = new Path("file://path/a=10"), + defaultPartitionName = defaultPartitionName, + typeInference = true, + basePaths = Set(new Path("file://path")))._1 + + assert(partitionSpec2 == + Option(PartitionValues( + ArrayBuffer("a"), + ArrayBuffer(Literal.create(10, IntegerType))))) + } + test("parse partitions") { def check( paths: Seq[String], @@ -413,6 +436,43 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } + test("read partitioned table using different path options") { + withTempDir { base => + val pi = 1 + val ps = "foo" + val path = makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps) + makeParquetFile( + (1 to 10).map(i => ParquetData(i, i.toString)), path) + + // when the input is the base path containing partitioning directories + val baseDf = sqlContext.read.parquet(base.getCanonicalPath) + assert(baseDf.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) + + // when the input is a path to the leaf directory containing a parquet file + val partDf = sqlContext.read.parquet(path.getCanonicalPath) + assert(partDf.schema.map(_.name) === Seq("intField", "stringField")) + + path.listFiles().foreach { f => + if (f.getName.toLowerCase().endsWith(".parquet")) { + // when the input is a path to a parquet file + val df = sqlContext.read.parquet(f.getCanonicalPath) + assert(df.schema.map(_.name) === Seq("intField", "stringField")) + } + } + + path.listFiles().foreach { f => + if (f.getName.toLowerCase().endsWith(".parquet")) { + // when the input is a path to a parquet file but `basePath` is overridden to + // the base path containing partitioning directories + val df = sqlContext + .read.option("basePath", base.getCanonicalPath) + .parquet(f.getCanonicalPath) + assert(df.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) + } + } + } + } + test("read partitioned table - partition key included in Parquet file") { withTempDir { base => for { From e12ec46c6b9b89dff5b8bfe610227d84b16388aa Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 4 May 2016 21:19:53 -0700 Subject: [PATCH 0076/1470] [SPARK-15131][SQL] Shutdown StateStore management thread when SparkContext has been shutdown ## What changes were proposed in this pull request? Make sure that whenever the StateStoreCoordinator cannot be contacted, assume that the SparkContext and RpcEnv on the driver has been shutdown, and therefore stop the StateStore management thread, and unload all loaded stores. ## How was this patch tested? Updated unit tests. Author: Tathagata Das Closes #12905 from tdas/SPARK-15131. (cherry picked from commit bde27b89a243247bd3069b24cb4bb1eee94edfd7) Signed-off-by: Shixiong Zhu --- .../streaming/state/StateStore.scala | 46 ++++++++++--------- .../streaming/state/StateStoreSuite.scala | 15 +++++- .../streaming/StreamingAggregationSuite.scala | 10 +++- 3 files changed, 48 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 952150632519e..9948292470ff8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -113,7 +113,7 @@ case class KeyRemoved(key: UnsafeRow) extends StoreUpdate * the store is the active instance. Accordingly, it either keeps it loaded and performs * maintenance, or unloads the store. */ -private[state] object StateStore extends Logging { +private[sql] object StateStore extends Logging { val MAINTENANCE_INTERVAL_CONFIG = "spark.streaming.stateStore.maintenanceInterval" val MAINTENANCE_INTERVAL_DEFAULT_SECS = 60 @@ -155,6 +155,10 @@ private[state] object StateStore extends Logging { loadedProviders.contains(storeId) } + def isMaintenanceRunning: Boolean = loadedProviders.synchronized { + maintenanceTask != null + } + /** Unload and stop all state store providers */ def stop(): Unit = loadedProviders.synchronized { loadedProviders.clear() @@ -187,44 +191,44 @@ private[state] object StateStore extends Logging { */ private def doMaintenance(): Unit = { logDebug("Doing maintenance") - loadedProviders.synchronized { loadedProviders.toSeq }.foreach { case (id, provider) => - try { - if (verifyIfStoreInstanceActive(id)) { - provider.doMaintenance() - } else { - unload(id) - logInfo(s"Unloaded $provider") + if (SparkEnv.get == null) { + stop() + } else { + loadedProviders.synchronized { loadedProviders.toSeq }.foreach { case (id, provider) => + try { + if (verifyIfStoreInstanceActive(id)) { + provider.doMaintenance() + } else { + unload(id) + logInfo(s"Unloaded $provider") + } + } catch { + case NonFatal(e) => + logWarning(s"Error managing $provider, stopping management thread") + stop() } - } catch { - case NonFatal(e) => - logWarning(s"Error managing $provider") } } } private def reportActiveStoreInstance(storeId: StateStoreId): Unit = { - try { + if (SparkEnv.get != null) { val host = SparkEnv.get.blockManager.blockManagerId.host val executorId = SparkEnv.get.blockManager.blockManagerId.executorId coordinatorRef.foreach(_.reportActiveInstance(storeId, host, executorId)) logDebug(s"Reported that the loaded instance $storeId is active") - } catch { - case NonFatal(e) => - logWarning(s"Error reporting active instance of $storeId") } } private def verifyIfStoreInstanceActive(storeId: StateStoreId): Boolean = { - try { + if (SparkEnv.get != null) { val executorId = SparkEnv.get.blockManager.blockManagerId.executorId val verified = coordinatorRef.map(_.verifyIfInstanceActive(storeId, executorId)).getOrElse(false) - logDebug(s"Verified whether the loaded instance $storeId is active: $verified" ) + logDebug(s"Verified whether the loaded instance $storeId is active: $verified") verified - } catch { - case NonFatal(e) => - logWarning(s"Error verifying active instance of $storeId") - false + } else { + false } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index dd23925716b06..f8f8bc7d6ff84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -47,8 +47,14 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth private val keySchema = StructType(Seq(StructField("key", StringType, true))) private val valueSchema = StructType(Seq(StructField("value", IntegerType, true))) + before { + StateStore.stop() + require(!StateStore.isMaintenanceRunning) + } + after { StateStore.stop() + require(!StateStore.isMaintenanceRunning) } test("get, put, remove, commit, and all data iterator") { @@ -352,7 +358,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth } } - ignore("maintenance") { + test("maintenance") { val conf = new SparkConf() .setMaster("local") .setAppName("test") @@ -366,20 +372,26 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth val provider = new HDFSBackedStateStoreProvider( storeId, keySchema, valueSchema, storeConf, hadoopConf) + quietly { withSpark(new SparkContext(conf)) { sc => withCoordinatorRef(sc) { coordinatorRef => + require(!StateStore.isMaintenanceRunning, "StateStore is unexpectedly running") + for (i <- 1 to 20) { val store = StateStore.get( storeId, keySchema, valueSchema, i - 1, storeConf, hadoopConf) put(store, "a", i) store.commit() } + eventually(timeout(10 seconds)) { assert(coordinatorRef.getLocation(storeId).nonEmpty, "active instance was not reported") } // Background maintenance should clean up and generate snapshots + assert(StateStore.isMaintenanceRunning, "Maintenance task is not running") + eventually(timeout(10 seconds)) { // Earliest delta file should get cleaned up assert(!fileExists(provider, 1, isSnapshot = false), "earliest file not deleted") @@ -418,6 +430,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth require(SparkEnv.get === null) eventually(timeout(10 seconds)) { assert(!StateStore.isLoaded(storeId)) + assert(!StateStore.isMaintenanceRunning) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index bdf40f5cd45d9..8da7742ffe0b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.streaming +import org.scalatest.BeforeAndAfterAll + import org.apache.spark.SparkException import org.apache.spark.sql.StreamTest import org.apache.spark.sql.catalyst.analysis.Update import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scala.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext @@ -29,7 +32,12 @@ object FailureSinglton { var firstTime = true } -class StreamingAggregationSuite extends StreamTest with SharedSQLContext { +class StreamingAggregationSuite extends StreamTest with SharedSQLContext with BeforeAndAfterAll { + + override def afterAll(): Unit = { + super.afterAll() + StateStore.stop() + } import testImplicits._ From 2023faf6c5433cdca4cd654bf16b165a57e2b5dd Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 4 May 2016 21:30:13 -0700 Subject: [PATCH 0077/1470] [MINOR] remove dead code --- python/pyspark/sql/dataframe.py | 9 --------- 1 file changed, 9 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index bbe15f5f900da..5378c32a1a96e 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -363,15 +363,6 @@ def coalesce(self, numPartitions): """ return DataFrame(self._jdf.coalesce(numPartitions), self.sql_ctx) - @since(1.3) - def repartition(self, numPartitions): - """Returns a new :class:`DataFrame` that has exactly ``numPartitions`` partitions. - - >>> df.repartition(10).rdd.getNumPartitions() - 10 - """ - return DataFrame(self._jdf.repartition(numPartitions), self.sql_ctx) - @since(1.3) def repartition(self, numPartitions, *cols): """ From 0914296cb847c8abaca2fc523b8dc22cdfafb776 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 4 May 2016 22:18:55 -0700 Subject: [PATCH 0078/1470] [SPARK-15132][MINOR][SQL] Debug log for generated code should be printed with proper indentation ## What changes were proposed in this pull request? Similar to #11990, GenerateOrdering and GenerateColumnAccessor should print debug log for generated code with proper indentation. ## How was this patch tested? Manually checked. Author: Kousuke Saruta Closes #12908 from sarutak/SPARK-15132. (cherry picked from commit 1a9b341581478f39421c7313e5e3981e8ab46abb) Signed-off-by: Reynold Xin --- .../sql/catalyst/expressions/codegen/GenerateOrdering.scala | 2 +- .../spark/sql/execution/columnar/GenerateColumnAccessor.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 5635c91830f4a..dc4825cdd8765 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 @@ -136,7 +136,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR } }""" - logDebug(s"Generated Ordering: ${CodeFormatter.format(code)}") + logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index e2e33e32463fc..bd5cb800dde3c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -224,7 +224,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera } }""" - logDebug(s"Generated ColumnarIterator: ${CodeFormatter.format(code)}") + logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] } From e28d21d3f5c71598bc45d920db706772c7deb19e Mon Sep 17 00:00:00 2001 From: Abhinav Gupta Date: Wed, 4 May 2016 22:22:01 -0700 Subject: [PATCH 0079/1470] [SPARK-15045] [CORE] Remove dead code in TaskMemoryManager.cleanUpAllAllocatedMemory for pageTable ## What changes were proposed in this pull request? Removed the DeadCode as suggested. Author: Abhinav Gupta Closes #12829 from abhi951990/master. (cherry picked from commit 1a5c6fcef131c60db4bc8852ca96445b9cb3bfac) Signed-off-by: Davies Liu --- .../org/apache/spark/memory/TaskMemoryManager.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 2796114fc545a..89521c1d6c307 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -379,7 +379,6 @@ public long getOffsetInPage(long pagePlusOffsetAddress) { */ public long cleanUpAllAllocatedMemory() { synchronized (this) { - Arrays.fill(pageTable, null); for (MemoryConsumer c: consumers) { if (c != null && c.getUsed() > 0) { // In case of failed task, it's normal to see leaked memory @@ -387,14 +386,16 @@ public long cleanUpAllAllocatedMemory() { } } consumers.clear(); - } - for (MemoryBlock page : pageTable) { - if (page != null) { - memoryManager.tungstenMemoryAllocator().free(page); + for (MemoryBlock page : pageTable) { + if (page != null) { + logger.warn("leak a page: " + page + " in task " + taskAttemptId); + memoryManager.tungstenMemoryAllocator().free(page); + } } + Arrays.fill(pageTable, null); } - Arrays.fill(pageTable, null); + // release the memory that is not used by any consumer. memoryManager.releaseExecutionMemory(acquiredButNotUsed, taskAttemptId, tungstenMemoryMode); From 433bc34b1b51739f27a04dff6119bce0e15a48de Mon Sep 17 00:00:00 2001 From: Lining Sun Date: Thu, 5 May 2016 10:47:39 +0100 Subject: [PATCH 0080/1470] [SPARK-15123] upgrade org.json4s to 3.2.11 version ## What changes were proposed in this pull request? We had the issue when using snowplow in our Spark applications. Snowplow requires json4s version 3.2.11 while Spark still use a few years old version 3.2.10. The change is to upgrade json4s jar to 3.2.11. ## How was this patch tested? We built Spark jar and successfully ran our applications in local and cluster modes. Author: Lining Sun Closes #12901 from liningalex/master. (cherry picked from commit 592fc455639462fcf00ec02860d7c33470b73273) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 6 +++--- dev/deps/spark-deps-hadoop-2.3 | 6 +++--- dev/deps/spark-deps-hadoop-2.4 | 6 +++--- dev/deps/spark-deps-hadoop-2.6 | 6 +++--- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- pom.xml | 2 +- 6 files changed, 16 insertions(+), 16 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index a4ef9a9af2939..8d3063b8169f1 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -110,9 +110,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 19c8fad984b30..48c8ebddc3218 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -101,9 +101,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index c2365f9cc5625..df55fca1e7400 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -102,9 +102,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 0d8afd19c5616..2125b917e319c 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -108,9 +108,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsr305-1.3.9.jar jta-1.1.jar jtransforms-2.4.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index f4274a9441a04..eece3c7b2d23d 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -108,9 +108,9 @@ joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar json-20090211.jar -json4s-ast_2.11-3.2.10.jar -json4s-core_2.11-3.2.10.jar -json4s-jackson_2.11-3.2.10.jar +json4s-ast_2.11-3.2.11.jar +json4s-core_2.11-3.2.11.jar +json4s-jackson_2.11-3.2.11.jar jsp-api-2.1.jar jsr305-1.3.9.jar jta-1.1.jar diff --git a/pom.xml b/pom.xml index 1dce91b7dfe4f..f33b2abae53f1 100644 --- a/pom.xml +++ b/pom.xml @@ -619,7 +619,7 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.2.10 + 3.2.11 com.sun.jersey From 0c4e42beaa16d99da093302bd768f868e9583399 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 5 May 2016 10:51:03 +0100 Subject: [PATCH 0081/1470] [SPARK-12154] Upgrade to Jersey 2 ## What changes were proposed in this pull request? Replace com.sun.jersey with org.glassfish.jersey. Changes to the Spark Web UI code were required to compile. The changes were relatively standard Jersey migration things. ## How was this patch tested? I did a manual test for the standalone web APIs. Although I didn't test the functionality of the security filter itself, the code that changed non-trivially is how we actually register the filter. I attached a debugger to the Spark master and verified that the SecurityFilter code is indeed invoked upon hitting /api/v1/applications. Author: mcheah Closes #12715 from mccheah/feature/upgrade-jersey. (cherry picked from commit b7fdc23ccc5967de5799d8cf6f14289e71f29a1e) Signed-off-by: Sean Owen --- NOTICE | 11 +- core/pom.xml | 18 ++- .../spark/status/api/v1/ApiRootResource.scala | 11 +- .../spark/status/api/v1/SecurityFilter.scala | 16 +-- dev/deps/spark-deps-hadoop-2.2 | 42 +++--- dev/deps/spark-deps-hadoop-2.3 | 26 ++-- dev/deps/spark-deps-hadoop-2.4 | 27 ++-- dev/deps/spark-deps-hadoop-2.6 | 25 ++-- dev/deps/spark-deps-hadoop-2.7 | 25 ++-- external/kafka-assembly/pom.xml | 10 -- external/kinesis-asl-assembly/pom.xml | 13 +- pom.xml | 124 +++++++++++++++--- sql/hive-thriftserver/pom.xml | 12 -- yarn/pom.xml | 15 +++ 14 files changed, 244 insertions(+), 131 deletions(-) diff --git a/NOTICE b/NOTICE index 2a6fe237dcbea..f4b1260a22170 100644 --- a/NOTICE +++ b/NOTICE @@ -12,7 +12,9 @@ Common Development and Distribution License 1.0 The following components are provided under the Common Development and Distribution License 1.0. See project link for details. (CDDL 1.0) Glassfish Jasper (org.mortbay.jetty:jsp-2.1:6.1.14 - http://jetty.mortbay.org/project/modules/jsp-2.1) + (CDDL 1.0) JAX-RS (https://jax-rs-spec.java.net/) (CDDL 1.0) Servlet Specification 2.5 API (org.mortbay.jetty:servlet-api-2.5:6.1.14 - http://jetty.mortbay.org/project/modules/servlet-api-2.5) + (CDDL 1.0) (GPL2 w/ CPE) javax.annotation API (https://glassfish.java.net/nonav/public/CDDL+GPL.html) (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined) (Common Development and Distribution License (CDDL) v1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) @@ -22,15 +24,10 @@ Common Development and Distribution License 1.1 The following components are provided under the Common Development and Distribution License 1.1. See project link for details. + (CDDL 1.1) (GPL2 w/ CPE) org.glassfish.hk2 (https://hk2.java.net) (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/) (CDDL 1.1) (GPL2 w/ CPE) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.8 - https://jersey.dev.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.8 - https://jersey.dev.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.8 - https://jersey.dev.java.net/jersey-server/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/) + (CDDL 1.1) (GPL2 w/ CPE) Jersey 2 (https://jersey.java.net) ======================================================================== Common Public License 1.0 diff --git a/core/pom.xml b/core/pom.xml index 7349ad35b9595..07b589637602b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -194,12 +194,24 @@ json4s-jackson_${scala.binary.version} - com.sun.jersey + org.glassfish.jersey.core + jersey-client + + + org.glassfish.jersey.core + jersey-common + + + org.glassfish.jersey.core jersey-server - com.sun.jersey - jersey-core + org.glassfish.jersey.containers + jersey-container-servlet + + + org.glassfish.jersey.containers + jersey-container-servlet-core org.apache.mesos diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 2cd51a9ed5417..681f295006e3c 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -21,10 +21,10 @@ import javax.servlet.ServletContext import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} -import com.sun.jersey.api.core.ResourceConfig -import com.sun.jersey.spi.container.servlet.ServletContainer import org.eclipse.jetty.server.handler.ContextHandler import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.glassfish.jersey.server.ServerProperties +import org.glassfish.jersey.servlet.ServletContainer import org.apache.spark.SecurityManager import org.apache.spark.ui.SparkUI @@ -191,12 +191,7 @@ private[spark] object ApiRootResource { val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) jerseyContext.setContextPath("/api") val holder: ServletHolder = new ServletHolder(classOf[ServletContainer]) - holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", - "com.sun.jersey.api.core.PackagesResourceConfig") - holder.setInitParameter("com.sun.jersey.config.property.packages", - "org.apache.spark.status.api.v1") - holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, - classOf[SecurityFilter].getCanonicalName) + holder.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.spark.status.api.v1") UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) jerseyContext.addServlet(holder, "/*") jerseyContext diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala index 95fbd96ade5ab..b4a991eda35f3 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala @@ -16,18 +16,16 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.WebApplicationException +import javax.ws.rs.container.{ContainerRequestContext, ContainerRequestFilter} import javax.ws.rs.core.Response +import javax.ws.rs.ext.Provider -import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter} - +@Provider private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext { - def filter(req: ContainerRequest): ContainerRequest = { - val user = Option(req.getUserPrincipal).map { _.getName }.orNull - if (uiRoot.securityManager.checkUIViewPermissions(user)) { - req - } else { - throw new WebApplicationException( + override def filter(req: ContainerRequestContext): Unit = { + val user = Option(req.getSecurityContext.getUserPrincipal).map { _.getName }.orNull + if (!uiRoot.securityManager.checkUIViewPermissions(user)) { + req.abortWith( Response .status(Response.Status.FORBIDDEN) .entity(raw"""user "$user"is not authorized""") diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 8d3063b8169f1..669c88369356f 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -1,14 +1,13 @@ JavaEWAH-0.3.2.jar RoaringBitmap-0.5.11.jar ST4-4.0.4.jar -activation-1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -49,12 +48,6 @@ datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar derby-10.10.1.1.jar eigenbase-properties-1.1.5.jar -gmbal-api-only-3.0.0-b023.jar -grizzly-framework-2.1.2.jar -grizzly-http-2.1.2.jar -grizzly-http-server-2.1.2.jar -grizzly-http-servlet-2.1.2.jar -grizzly-rcm-2.1.2.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar @@ -73,6 +66,9 @@ hadoop-yarn-client-2.2.0.jar hadoop-yarn-common-2.2.0.jar hadoop-yarn-server-common-2.2.0.jar hadoop-yarn-server-web-proxy-2.2.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar @@ -80,30 +76,26 @@ jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar -javax.servlet-3.1.jar -javax.servlet-api-3.0.1.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-grizzly2-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar -jersey-test-framework-core-1.9.jar -jersey-test-framework-grizzly2-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.7.1.jar -jettison-1.1.jar jetty-util-6.1.26.jar jline-2.12.jar joda-time-2.9.3.jar @@ -123,7 +115,6 @@ libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar lz4-1.3.0.jar -management-api-3.0.0-b012.jar mesos-0.21.1-shaded-protobuf.jar metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar @@ -135,6 +126,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -162,12 +154,12 @@ snappy-0.2.jar snappy-java-1.1.2.4.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar -stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 48c8ebddc3218..c6e835d6a5867 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -6,9 +6,9 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -68,6 +68,9 @@ hadoop-yarn-client-2.3.0.jar hadoop-yarn-common-2.3.0.jar hadoop-yarn-server-common-2.3.0.jar hadoop-yarn-server-web-proxy-2.3.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar @@ -75,25 +78,28 @@ jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -127,6 +133,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -160,6 +167,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index df55fca1e7400..79c09b2d0e86c 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -6,9 +6,9 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -68,6 +68,9 @@ hadoop-yarn-client-2.4.0.jar hadoop-yarn-common-2.4.0.jar hadoop-yarn-server-common-2.4.0.jar hadoop-yarn-server-web-proxy-2.4.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar @@ -75,26 +78,28 @@ jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar jackson-databind-2.5.3.jar -jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar jackson-module-scala_2.11-2.5.3.jar -jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -128,6 +133,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -161,6 +167,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 2125b917e319c..be9b52cbef265 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -6,13 +6,13 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -73,6 +73,9 @@ hadoop-yarn-client-2.6.0.jar hadoop-yarn-common-2.6.0.jar hadoop-yarn-server-common-2.6.0.jar hadoop-yarn-server-web-proxy-2.6.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar httpclient-4.3.2.jar httpcore-4.3.2.jar @@ -87,20 +90,24 @@ jackson-module-scala_2.11-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -134,6 +141,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -167,6 +175,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index eece3c7b2d23d..08baa3a137289 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -6,13 +6,13 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.5.2-1.jar aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -asm-3.1.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -73,6 +73,9 @@ hadoop-yarn-client-2.7.0.jar hadoop-yarn-common-2.7.0.jar hadoop-yarn-server-common-2.7.0.jar hadoop-yarn-server-web-proxy-2.7.0.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar httpclient-4.3.2.jar httpcore-4.3.2.jar @@ -87,20 +90,24 @@ jackson-module-scala_2.11-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar javax.inject-1.jar +javax.inject-2.4.0-b34.jar javax.servlet-3.0.0.v201112011016.jar +javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar -jaxb-impl-2.2.3-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-1.9.jar -jersey-core-1.9.jar -jersey-guice-1.9.jar -jersey-json-1.9.jar -jersey-server-1.9.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar jets3t-0.9.3.jar -jettison-1.1.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.jar @@ -135,6 +142,7 @@ netty-all-4.0.29.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar paranamer-2.6.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar @@ -168,6 +176,7 @@ stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.0.2.jar +validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index 62818f5e8f434..e1b5a7e713d76 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -65,16 +65,6 @@ protobuf-java provided - - com.sun.jersey - jersey-server - provided - - - com.sun.jersey - jersey-core - provided - net.jpountz.lz4 lz4 diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index d1c38c7ca5d69..e057b78abdc8c 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -66,13 +66,18 @@ provided - com.sun.jersey - jersey-server + org.glassfish.jersey.core + jersey-client provided - com.sun.jersey - jersey-core + org.glassfish.jersey.core + jersey-common + provided + + + org.glassfish.jersey.core + jersey-server provided diff --git a/pom.xml b/pom.xml index f33b2abae53f1..852136a998aba 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 3.3.2 3.2.10 2.7.8 - 1.9 + 2.22.2 2.9.3 3.5.2 1.3.9 @@ -360,7 +360,6 @@ provided - org.apache.commons commons-lang3 @@ -588,16 +587,44 @@ - com.sun.jersey + com.fasterxml.jackson.module + jackson-module-jaxb-annotations + ${fasterxml.jackson.version} + + + org.glassfish.jersey.core jersey-server ${jersey.version} - ${hadoop.deps.scope} - com.sun.jersey - jersey-core + org.glassfish.jersey.core + jersey-common + ${jersey.version} + + + org.glassfish.jersey.core + jersey-client + ${jersey.version} + + + org.glassfish.jersey.containers + jersey-container-servlet + ${jersey.version} + + + org.glassfish.jersey.containers + jersey-container-servlet-core ${jersey.version} - ${hadoop.deps.scope} + + + org.glassfish.jersey + jersey-client + ${jersey.version} + + + javax.ws.rs + javax.ws.rs-api + 2.0.1 org.scalanlp @@ -621,17 +648,6 @@ json4s-jackson_${scala.binary.version} 3.2.11 - - com.sun.jersey - jersey-json - ${jersey.version} - - - stax - stax-api - - - org.scala-lang scala-compiler @@ -814,6 +830,18 @@ junit junit + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -926,6 +954,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -954,6 +994,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -983,6 +1035,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1011,6 +1075,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1039,6 +1115,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 987103b1a9536..a8b7aaff62a15 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -64,18 +64,6 @@ ${hive.group} hive-beeline - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-json - - - com.sun.jersey - jersey-server - org.seleniumhq.selenium diff --git a/yarn/pom.xml b/yarn/pom.xml index 328bb6678db99..db7f3e51d3be5 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -30,6 +30,7 @@ Spark Project YARN yarn + 1.9 @@ -147,20 +148,34 @@ test + + com.sun.jersey jersey-core test + ${jersey-1.version} com.sun.jersey jersey-json test + ${jersey-1.version} com.sun.jersey jersey-server test + ${jersey-1.version} + + + com.sun.jersey.contribs + jersey-guice + test + ${jersey-1.version} C:\TEMP\ Win XP --> C:\TEMP Solaris --> /var/tmp/ Linux --> /var/tmp ``` Second, a couple of test cases are added to verify if the commands work properly. #### How was this patch tested? Added a test case for it and correct the previous test cases. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #12081 from gatorsmile/mkdir. (cherry picked from commit 8cba57a75cf9e29b54d97366a039a97a2f305d5d) Signed-off-by: Andrew Or --- .../sql/catalyst/catalog/SessionCatalog.scala | 4 + .../spark/sql/execution/command/ddl.scala | 5 +- .../sql/execution/command/DDLSuite.scala | 249 +++++++++++------- .../sql/hive/execution/HiveDDLSuite.scala | 150 ++++++++++- 4 files changed, 311 insertions(+), 97 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index ff6303471e145..eff420eb4c5ac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -146,6 +146,10 @@ class SessionCatalog( currentDb = db } + /** + * Get the path for creating a non-default database when database location is not provided + * by users. + */ def getDefaultDBPath(db: String): String = { val database = if (conf.caseSensitiveAnalysis) db else db.toLowerCase new Path(new Path(conf.warehousePath), database + ".db").toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index aa06c014fb0a2..085bdaff4e03b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -40,7 +40,10 @@ import org.apache.spark.sql.types._ * unless 'ifNotExists' is true. * The syntax of using this command in SQL is: * {{{ - * CREATE DATABASE|SCHEMA [IF NOT EXISTS] database_name + * CREATE (DATABASE|SCHEMA) [IF NOT EXISTS] database_name + * [COMMENT database_comment] + * [LOCATION database_directory] + * [WITH DBPROPERTIES (property_name=property_value, ...)]; * }}} */ case class CreateDatabase( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 0ae099ecc2bdd..6085098a709e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -95,49 +95,81 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) } + private def appendTrailingSlash(path: String): String = { + if (!path.endsWith(File.separator)) path + File.separator else path + } + test("the qualified path of a database is stored in the catalog") { val catalog = sqlContext.sessionState.catalog - val path = System.getProperty("java.io.tmpdir") - // The generated temp path is not qualified. - assert(!path.startsWith("file:/")) - sql(s"CREATE DATABASE db1 LOCATION '$path'") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri - assert("file" === pathInCatalog.getScheme) - assert(path === pathInCatalog.getPath) - - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir"))) { - sql(s"CREATE DATABASE db2") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + withTempDir { tmpDir => + val path = tmpDir.toString + // The generated temp path is not qualified. + assert(!path.startsWith("file:/")) + sql(s"CREATE DATABASE db1 LOCATION '$path'") + val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri assert("file" === pathInCatalog.getScheme) - assert(s"${sqlContext.conf.warehousePath}/db2.db" === pathInCatalog.getPath) - } + val expectedPath = if (path.endsWith(File.separator)) path.dropRight(1) else path + assert(expectedPath === pathInCatalog.getPath) + + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + sql(s"CREATE DATABASE db2") + val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + assert("file" === pathInCatalog.getScheme) + val expectedPath = appendTrailingSlash(sqlContext.conf.warehousePath) + "db2.db" + assert(expectedPath === pathInCatalog.getPath) + } - sql("DROP DATABASE db1") - sql("DROP DATABASE db2") + sql("DROP DATABASE db1") + sql("DROP DATABASE db2") + } } test("Create/Drop Database") { - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir") + File.separator)) { - val catalog = sqlContext.sessionState.catalog + withTempDir { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val databaseNames = Seq("db1", "`database`") + sql(s"CREATE DATABASE $dbName") + val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + val expectedLocation = + "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + assert(db1 == CatalogDatabase( + dbNameWithoutBackTicks, + "", + expectedLocation, + Map.empty)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.databaseExists(dbNameWithoutBackTicks)) + } finally { + catalog.reset() + } + } + } + } + } + test("Create/Drop Database - location") { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + withTempDir { tmpDir => + val path = tmpDir.toString + val dbPath = "file:" + path databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - - sql(s"CREATE DATABASE $dbName") + sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + System.getProperty("java.io.tmpdir") + - File.separator + s"$dbNameWithoutBackTicks.db" assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", - expectedLocation, + if (dbPath.endsWith(File.separator)) dbPath.dropRight(1) else dbPath, Map.empty)) sql(s"DROP DATABASE $dbName CASCADE") assert(!catalog.databaseExists(dbNameWithoutBackTicks)) @@ -149,77 +181,78 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("Create Database - database already exists") { - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir") + File.separator)) { - val catalog = sqlContext.sessionState.catalog - val databaseNames = Seq("db1", "`database`") - - databaseNames.foreach { dbName => - try { - val dbNameWithoutBackTicks = cleanIdentifier(dbName) - sql(s"CREATE DATABASE $dbName") - val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + System.getProperty("java.io.tmpdir") + - File.separator + s"$dbNameWithoutBackTicks.db" - assert(db1 == CatalogDatabase( - dbNameWithoutBackTicks, - "", - expectedLocation, - Map.empty)) - - val message = intercept[AnalysisException] { + withTempDir { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") - }.getMessage - assert(message.contains(s"Database '$dbNameWithoutBackTicks' already exists.")) - } finally { - catalog.reset() + val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + val expectedLocation = + "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + assert(db1 == CatalogDatabase( + dbNameWithoutBackTicks, + "", + expectedLocation, + Map.empty)) + + val message = intercept[AnalysisException] { + sql(s"CREATE DATABASE $dbName") + }.getMessage + assert(message.contains(s"Database '$dbNameWithoutBackTicks' already exists.")) + } finally { + catalog.reset() + } } } } } test("Alter/Describe Database") { - withSQLConf( - SQLConf.WAREHOUSE_PATH.key -> (System.getProperty("java.io.tmpdir") + File.separator)) { - val catalog = sqlContext.sessionState.catalog - val databaseNames = Seq("db1", "`database`") + withTempDir { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val catalog = sqlContext.sessionState.catalog + val databaseNames = Seq("db1", "`database`") - databaseNames.foreach { dbName => - try { - val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = - "file:" + System.getProperty("java.io.tmpdir") + - File.separator + s"$dbNameWithoutBackTicks.db" - - sql(s"CREATE DATABASE $dbName") - - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), - Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: - Row("Location", location) :: - Row("Properties", "") :: Nil) - - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") - - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), - Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: - Row("Location", location) :: - Row("Properties", "((a,a), (b,b), (c,c))") :: Nil) - - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") - - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), - Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: - Row("Location", location) :: - Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) - } finally { - catalog.reset() + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) + val location = "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + + sql(s"CREATE DATABASE $dbName") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", location) :: + Row("Properties", "") :: Nil) + + sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", location) :: + Row("Properties", "((a,a), (b,b), (c,c))") :: Nil) + + sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", location) :: + Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) + } finally { + catalog.reset() + } } } } @@ -251,7 +284,43 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } - // TODO: test drop database in restrict mode + test("drop non-empty database in restrict mode") { + val catalog = sqlContext.sessionState.catalog + val dbName = "db1" + sql(s"CREATE DATABASE $dbName") + + // create a table in database + val tableIdent1 = TableIdentifier("tab1", Some(dbName)) + createTable(catalog, tableIdent1) + + // drop a non-empty database in Restrict mode + val message = intercept[AnalysisException] { + sql(s"DROP DATABASE $dbName RESTRICT") + }.getMessage + assert(message.contains(s"Database '$dbName' is not empty. One or more tables exist")) + + catalog.dropTable(tableIdent1, ignoreIfNotExists = false) + + assert(catalog.listDatabases().contains(dbName)) + sql(s"DROP DATABASE $dbName RESTRICT") + assert(!catalog.listDatabases().contains(dbName)) + } + + test("drop non-empty database in cascade mode") { + val catalog = sqlContext.sessionState.catalog + val dbName = "db1" + sql(s"CREATE DATABASE $dbName") + + // create a table in database + val tableIdent1 = TableIdentifier("tab1", Some(dbName)) + createTable(catalog, tableIdent1) + + // drop a non-empty database in CASCADE mode + assert(catalog.listTables(dbName).contains(tableIdent1)) + assert(catalog.listDatabases().contains(dbName)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.listDatabases().contains(dbName)) + } test("create table in default db") { val catalog = sqlContext.sessionState.catalog diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 373d1a1e0ebc1..d55ddb251d00d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -20,21 +20,37 @@ package org.apache.spark.sql.hive.execution import java.io.File import org.apache.hadoop.fs.Path +import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} -import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTableType} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils -class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { +class HiveDDLSuite + extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { import hiveContext.implicits._ + override def afterEach(): Unit = { + try { + // drop all databases, tables and functions after each test + sqlContext.sessionState.catalog.reset() + } finally { + super.afterEach() + } + } // check if the directory for recording the data of the table exists. - private def tableDirectoryExists(tableIdentifier: TableIdentifier): Boolean = { + private def tableDirectoryExists( + tableIdentifier: TableIdentifier, + dbPath: Option[String] = None): Boolean = { val expectedTablePath = - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdentifier) + if (dbPath.isEmpty) { + hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdentifier) + } else { + new Path(new Path(dbPath.get), tableIdentifier.table).toString + } val filesystemPath = new Path(expectedTablePath) val fs = filesystemPath.getFileSystem(hiveContext.sessionState.newHadoopConf()) fs.exists(filesystemPath) @@ -56,7 +72,7 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("drop managed tables") { + test("drop managed tables in default database") { withTempDir { tmpDir => val tabName = "tab1" withTable(tabName) { @@ -83,7 +99,7 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("drop external data source table") { + test("drop external data source table in default database") { withTempDir { tmpDir => val tabName = "tab1" withTable(tabName) { @@ -365,4 +381,126 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { .exists(_.getString(0) == "# Detailed Table Information")) } } + + private def createDatabaseWithLocation(tmpDir: File, dirExists: Boolean): Unit = { + val catalog = sqlContext.sessionState.catalog + val dbName = "db1" + val tabName = "tab1" + val fs = new Path(tmpDir.toString).getFileSystem(hiveContext.sessionState.newHadoopConf()) + withTable(tabName) { + if (dirExists) { + assert(tmpDir.listFiles.isEmpty) + } else { + assert(!fs.exists(new Path(tmpDir.toString))) + } + sql(s"CREATE DATABASE $dbName Location '$tmpDir'") + val db1 = catalog.getDatabaseMetadata(dbName) + val dbPath = "file:" + tmpDir + assert(db1 == CatalogDatabase( + dbName, + "", + if (dbPath.endsWith(File.separator)) dbPath.dropRight(1) else dbPath, + Map.empty)) + sql("USE db1") + + sql(s"CREATE TABLE $tabName as SELECT 1") + assert(tableDirectoryExists(TableIdentifier(tabName), Option(tmpDir.toString))) + + assert(tmpDir.listFiles.nonEmpty) + sql(s"DROP TABLE $tabName") + + assert(tmpDir.listFiles.isEmpty) + sql(s"DROP DATABASE $dbName") + assert(!fs.exists(new Path(tmpDir.toString))) + } + } + + test("create/drop database - location without pre-created directory") { + withTempPath { tmpDir => + createDatabaseWithLocation(tmpDir, dirExists = false) + } + } + + test("create/drop database - location with pre-created directory") { + withTempDir { tmpDir => + createDatabaseWithLocation(tmpDir, dirExists = true) + } + } + + private def appendTrailingSlash(path: String): String = { + if (!path.endsWith(File.separator)) path + File.separator else path + } + + private def dropDatabase(cascade: Boolean, tableExists: Boolean): Unit = { + withTempPath { tmpDir => + val path = tmpDir.toString + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { + val dbName = "db1" + val fs = new Path(path).getFileSystem(hiveContext.sessionState.newHadoopConf()) + val dbPath = new Path(path) + // the database directory does not exist + assert(!fs.exists(dbPath)) + + sql(s"CREATE DATABASE $dbName") + val catalog = sqlContext.sessionState.catalog + val expectedDBLocation = "file:" + appendTrailingSlash(dbPath.toString) + s"$dbName.db" + val db1 = catalog.getDatabaseMetadata(dbName) + assert(db1 == CatalogDatabase( + dbName, + "", + expectedDBLocation, + Map.empty)) + // the database directory was created + assert(fs.exists(dbPath) && fs.isDirectory(dbPath)) + sql(s"USE $dbName") + + val tabName = "tab1" + assert(!tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + sql(s"CREATE TABLE $tabName as SELECT 1") + assert(tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + + if (!tableExists) { + sql(s"DROP TABLE $tabName") + assert(!tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + } + + val sqlDropDatabase = s"DROP DATABASE $dbName ${if (cascade) "CASCADE" else "RESTRICT"}" + if (tableExists && !cascade) { + val message = intercept[AnalysisException] { + sql(sqlDropDatabase) + }.getMessage + assert(message.contains(s"Database $dbName is not empty. One or more tables exist.")) + // the database directory was not removed + assert(fs.exists(new Path(expectedDBLocation))) + } else { + sql(sqlDropDatabase) + // the database directory was removed and the inclusive table directories are also removed + assert(!fs.exists(new Path(expectedDBLocation))) + } + } + } + } + + test("drop database containing tables - CASCADE") { + dropDatabase(cascade = true, tableExists = true) + } + + test("drop an empty database - CASCADE") { + dropDatabase(cascade = true, tableExists = false) + } + + test("drop database containing tables - RESTRICT") { + dropDatabase(cascade = false, tableExists = true) + } + + test("drop an empty database - RESTRICT") { + dropDatabase(cascade = false, tableExists = false) + } + + test("drop default database") { + val message = intercept[AnalysisException] { + sql("DROP DATABASE default") + }.getMessage + assert(message.contains("Can not drop default database")) + } } From 59fa480b6d014369d58db9c5b77d82ddd17ee2a2 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 5 May 2016 14:35:15 -0700 Subject: [PATCH 0090/1470] [SPARK-15072][SQL][REPL][EXAMPLES] Remove SparkSession.withHiveSupport ## What changes were proposed in this pull request? Removing the `withHiveSupport` method of `SparkSession`, instead use `enableHiveSupport` ## How was this patch tested? ran tests locally Author: Sandeep Singh Closes #12851 from techaddict/SPARK-15072. (cherry picked from commit ed6f3f8a5f3a6bf7c53e13c2798de398c9a526a6) Signed-off-by: Andrew Or --- .../spark/examples/sql/hive/HiveFromSpark.scala | 14 +++++++++----- .../scala/org/apache/spark/sql/SparkSession.scala | 13 ------------- .../spark/sql/hive/thriftserver/SparkSQLEnv.scala | 10 ++++++---- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 7 +++++-- 4 files changed, 20 insertions(+), 24 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index ff33091621c14..a15cf5ded0e7f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -36,15 +36,19 @@ object HiveFromSpark { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") - val sc = new SparkContext(sparkConf) // A hive context adds support for finding tables in the MetaStore and writing queries // using HiveQL. Users who do not have an existing Hive deployment can still create a // HiveContext. When not configured by the hive-site.xml, the context automatically // creates metastore_db and warehouse in the current directory. - val sparkSession = SparkSession.withHiveSupport(sc) - import sparkSession.implicits._ - import sparkSession.sql + val spark = SparkSession.builder + .config(sparkConf) + .enableHiveSupport() + .getOrCreate() + val sc = spark.sparkContext + + import spark.implicits._ + import spark.sql sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") @@ -74,7 +78,7 @@ object HiveFromSpark { println("Result of SELECT *:") sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index aa7c335c53d24..9ed3756628c84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -816,17 +816,4 @@ object SparkSession { } } - /** - * Create a new [[SparkSession]] with a catalog backed by Hive. - */ - def withHiveSupport(sc: SparkContext): SparkSession = { - if (hiveClassesArePresent) { - sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive") - new SparkSession(sc) - } else { - throw new IllegalArgumentException( - "Unable to instantiate SparkSession with Hive support because Hive classes are not found.") - } - } - } 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 665a44e51a0c7..8de223f444f70 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 @@ -54,13 +54,15 @@ private[hive] object SparkSQLEnv extends Logging { "spark.kryo.referenceTracking", maybeKryoReferenceTracking.getOrElse("false")) - sparkContext = new SparkContext(sparkConf) - sqlContext = SparkSession.withHiveSupport(sparkContext).wrapped - val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] + val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() + sparkContext = sparkSession.sparkContext + sqlContext = sparkSession.wrapped + + val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState] sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) sessionState.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) sessionState.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) - sqlContext.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) + sparkSession.conf.set("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 77a6a94a6719b..a320011799494 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -290,8 +290,11 @@ object SetWarehouseLocationTest extends Logging { conf.set("spark.sql.warehouse.dir", warehouseLocation.toString) conf.set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString) - val sc = new SparkContext(conf) - val sparkSession = SparkSession.withHiveSupport(sc) + val sparkSession = SparkSession.builder + .config(conf) + .enableHiveSupport() + .getOrCreate() + val catalog = sparkSession.sessionState.catalog sparkSession.sql("drop table if exists testLocation") From e78b31b72d85ffcc596176a2c91b050f3a3ff3ee Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 5 May 2016 14:36:47 -0700 Subject: [PATCH 0091/1470] [SPARK-15135][SQL] Make sure SparkSession thread safe ## What changes were proposed in this pull request? Went through SparkSession and its members and fixed non-thread-safe classes used by SparkSession ## How was this patch tested? Existing unit tests Author: Shixiong Zhu Closes #12915 from zsxwing/spark-session-thread-safe. (cherry picked from commit bb9991dec5dd631b22a05e2e1b83b9082a845e8f) Signed-off-by: Andrew Or --- .../catalyst/analysis/FunctionRegistry.scala | 10 +- .../catalyst/catalog/InMemoryCatalog.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 102 ++++++++++-------- .../spark/sql/ExperimentalMethods.scala | 4 +- .../org/apache/spark/sql/SparkSession.scala | 7 +- .../apache/spark/sql/hive/test/TestHive.scala | 4 +- 6 files changed, 73 insertions(+), 56 deletions(-) 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 1bada2ce67ea2..ac05dd3d0ef9e 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 @@ -28,7 +28,11 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.util.StringKeyHashMap -/** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ +/** + * A catalog for looking up user defined functions, used by an [[Analyzer]]. + * + * Note: The implementation should be thread-safe to allow concurrent access. + */ trait FunctionRegistry { final def registerFunction(name: String, builder: FunctionBuilder): Unit = { @@ -62,7 +66,7 @@ trait FunctionRegistry { class SimpleFunctionRegistry extends FunctionRegistry { - private[sql] val functionBuilders = + protected val functionBuilders = StringKeyHashMap[(ExpressionInfo, FunctionBuilder)](caseSensitive = false) override def registerFunction( @@ -97,7 +101,7 @@ class SimpleFunctionRegistry extends FunctionRegistry { functionBuilders.remove(name).isDefined } - override def clear(): Unit = { + override def clear(): Unit = synchronized { functionBuilders.clear() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 1d2ca2863fae9..c65f4611295e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -340,7 +340,7 @@ class InMemoryCatalog extends ExternalCatalog { catalog(db).functions(funcName) } - override def functionExists(db: String, funcName: String): Boolean = { + override def functionExists(db: String, funcName: String): Boolean = synchronized { requireDbExists(db) catalog(db).functions.contains(funcName) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index eff420eb4c5ac..712770784bf9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.catalog +import javax.annotation.concurrent.GuardedBy + import scala.collection.mutable import org.apache.hadoop.conf.Configuration @@ -37,7 +39,7 @@ import org.apache.spark.sql.catalyst.util.StringUtils * proxy to the underlying metastore (e.g. Hive Metastore) and it also manages temporary * tables and functions of the Spark Session that it belongs to. * - * This class is not thread-safe. + * This class must be thread-safe. */ class SessionCatalog( externalCatalog: ExternalCatalog, @@ -66,12 +68,14 @@ class SessionCatalog( } /** List of temporary tables, mapping from table name to their logical plan. */ + @GuardedBy("this") protected val tempTables = new mutable.HashMap[String, LogicalPlan] // Note: we track current database here because certain operations do not explicitly // specify the database (e.g. DROP TABLE my_table). In these cases we must first // check whether the temporary table or function exists, then, if not, operate on // the corresponding item in the current database. + @GuardedBy("this") protected var currentDb = { val defaultName = "default" val defaultDbDefinition = @@ -137,13 +141,13 @@ class SessionCatalog( externalCatalog.listDatabases(pattern) } - def getCurrentDatabase: String = currentDb + def getCurrentDatabase: String = synchronized { currentDb } def setCurrentDatabase(db: String): Unit = { if (!databaseExists(db)) { throw new AnalysisException(s"Database '$db' does not exist.") } - currentDb = db + synchronized { currentDb = db } } /** @@ -173,7 +177,7 @@ class SessionCatalog( * If no such database is specified, create it in the current database. */ def createTable(tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = { - val db = tableDefinition.identifier.database.getOrElse(currentDb) + val db = tableDefinition.identifier.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableDefinition.identifier.table) val newTableDefinition = tableDefinition.copy(identifier = TableIdentifier(table, Some(db))) externalCatalog.createTable(db, newTableDefinition, ignoreIfExists) @@ -189,7 +193,7 @@ class SessionCatalog( * this becomes a no-op. */ def alterTable(tableDefinition: CatalogTable): Unit = { - val db = tableDefinition.identifier.database.getOrElse(currentDb) + val db = tableDefinition.identifier.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableDefinition.identifier.table) val newTableDefinition = tableDefinition.copy(identifier = TableIdentifier(table, Some(db))) externalCatalog.alterTable(db, newTableDefinition) @@ -201,7 +205,7 @@ class SessionCatalog( * If the specified table is not found in the database then an [[AnalysisException]] is thrown. */ def getTableMetadata(name: TableIdentifier): CatalogTable = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.getTable(db, table) } @@ -212,7 +216,7 @@ class SessionCatalog( * If the specified table is not found in the database then return None if it doesn't exist. */ def getTableMetadataOption(name: TableIdentifier): Option[CatalogTable] = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.getTableOption(db, table) } @@ -227,7 +231,7 @@ class SessionCatalog( loadPath: String, isOverwrite: Boolean, holdDDLTime: Boolean): Unit = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.loadTable(db, table, loadPath, isOverwrite, holdDDLTime) } @@ -245,14 +249,14 @@ class SessionCatalog( holdDDLTime: Boolean, inheritTableSpecs: Boolean, isSkewedStoreAsSubdir: Boolean): Unit = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val table = formatTableName(name.table) externalCatalog.loadPartition(db, table, loadPath, partition, isOverwrite, holdDDLTime, inheritTableSpecs, isSkewedStoreAsSubdir) } def defaultTablePath(tableIdent: TableIdentifier): String = { - val dbName = tableIdent.database.getOrElse(currentDb) + val dbName = tableIdent.database.getOrElse(getCurrentDatabase) val dbLocation = getDatabaseMetadata(dbName).locationUri new Path(new Path(dbLocation), formatTableName(tableIdent.table)).toString @@ -268,7 +272,7 @@ class SessionCatalog( def createTempTable( name: String, tableDefinition: LogicalPlan, - overrideIfExists: Boolean): Unit = { + overrideIfExists: Boolean): Unit = synchronized { val table = formatTableName(name) if (tempTables.contains(table) && !overrideIfExists) { throw new AnalysisException(s"Temporary table '$name' already exists.") @@ -285,7 +289,7 @@ class SessionCatalog( * * This assumes the database specified in `oldName` matches the one specified in `newName`. */ - def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = { + def renameTable(oldName: TableIdentifier, newName: TableIdentifier): Unit = synchronized { val db = oldName.database.getOrElse(currentDb) val newDb = newName.database.getOrElse(currentDb) if (db != newDb) { @@ -310,7 +314,7 @@ class SessionCatalog( * If no database is specified, this will first attempt to drop a temporary table with * the same name, then, if that does not exist, drop the table from the current database. */ - def dropTable(name: TableIdentifier, ignoreIfNotExists: Boolean): Unit = { + def dropTable(name: TableIdentifier, ignoreIfNotExists: Boolean): Unit = synchronized { val db = name.database.getOrElse(currentDb) val table = formatTableName(name.table) if (name.database.isDefined || !tempTables.contains(table)) { @@ -334,19 +338,21 @@ class SessionCatalog( * the same name, then, if that does not exist, return the table from the current database. */ def lookupRelation(name: TableIdentifier, alias: Option[String] = None): LogicalPlan = { - val db = name.database.getOrElse(currentDb) - val table = formatTableName(name.table) - val relation = - if (name.database.isDefined || !tempTables.contains(table)) { - val metadata = externalCatalog.getTable(db, table) - SimpleCatalogRelation(db, metadata, alias) - } else { - tempTables(table) - } - val qualifiedTable = SubqueryAlias(table, relation) - // If an alias was specified by the lookup, wrap the plan in a subquery so that - // attributes are properly qualified with this alias. - alias.map(a => SubqueryAlias(a, qualifiedTable)).getOrElse(qualifiedTable) + synchronized { + val db = name.database.getOrElse(currentDb) + val table = formatTableName(name.table) + val relation = + if (name.database.isDefined || !tempTables.contains(table)) { + val metadata = externalCatalog.getTable(db, table) + SimpleCatalogRelation(db, metadata, alias) + } else { + tempTables(table) + } + val qualifiedTable = SubqueryAlias(table, relation) + // If an alias was specified by the lookup, wrap the plan in a subquery so that + // attributes are properly qualified with this alias. + alias.map(a => SubqueryAlias(a, qualifiedTable)).getOrElse(qualifiedTable) + } } /** @@ -357,7 +363,7 @@ class SessionCatalog( * table with the same name, we will return false if the specified database does not * contain the table. */ - def tableExists(name: TableIdentifier): Boolean = { + def tableExists(name: TableIdentifier): Boolean = synchronized { val db = name.database.getOrElse(currentDb) val table = formatTableName(name.table) if (name.database.isDefined || !tempTables.contains(table)) { @@ -373,7 +379,7 @@ class SessionCatalog( * Note: The temporary table cache is checked only when database is not * explicitly specified. */ - def isTemporaryTable(name: TableIdentifier): Boolean = { + def isTemporaryTable(name: TableIdentifier): Boolean = synchronized { name.database.isEmpty && tempTables.contains(formatTableName(name.table)) } @@ -388,9 +394,11 @@ class SessionCatalog( def listTables(db: String, pattern: String): Seq[TableIdentifier] = { val dbTables = externalCatalog.listTables(db, pattern).map { t => TableIdentifier(t, Some(db)) } - val _tempTables = StringUtils.filterPattern(tempTables.keys.toSeq, pattern) - .map { t => TableIdentifier(t) } - dbTables ++ _tempTables + synchronized { + val _tempTables = StringUtils.filterPattern(tempTables.keys.toSeq, pattern) + .map { t => TableIdentifier(t) } + dbTables ++ _tempTables + } } // TODO: It's strange that we have both refresh and invalidate here. @@ -409,7 +417,7 @@ class SessionCatalog( * Drop all existing temporary tables. * For testing only. */ - def clearTempTables(): Unit = { + def clearTempTables(): Unit = synchronized { tempTables.clear() } @@ -417,7 +425,7 @@ class SessionCatalog( * Return a temporary table exactly as it was stored. * For testing only. */ - private[catalog] def getTempTable(name: String): Option[LogicalPlan] = { + private[catalog] def getTempTable(name: String): Option[LogicalPlan] = synchronized { tempTables.get(name) } @@ -441,7 +449,7 @@ class SessionCatalog( tableName: TableIdentifier, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.createPartitions(db, table, parts, ignoreIfExists) } @@ -454,7 +462,7 @@ class SessionCatalog( tableName: TableIdentifier, parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.dropPartitions(db, table, parts, ignoreIfNotExists) } @@ -469,7 +477,7 @@ class SessionCatalog( tableName: TableIdentifier, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.renamePartitions(db, table, specs, newSpecs) } @@ -484,7 +492,7 @@ class SessionCatalog( * this becomes a no-op. */ def alterPartitions(tableName: TableIdentifier, parts: Seq[CatalogTablePartition]): Unit = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.alterPartitions(db, table, parts) } @@ -494,7 +502,7 @@ class SessionCatalog( * If no database is specified, assume the table is in the current database. */ def getPartition(tableName: TableIdentifier, spec: TablePartitionSpec): CatalogTablePartition = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.getPartition(db, table, spec) } @@ -509,7 +517,7 @@ class SessionCatalog( def listPartitions( tableName: TableIdentifier, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = { - val db = tableName.database.getOrElse(currentDb) + val db = tableName.database.getOrElse(getCurrentDatabase) val table = formatTableName(tableName.table) externalCatalog.listPartitions(db, table, partialSpec) } @@ -532,7 +540,7 @@ class SessionCatalog( * If no such database is specified, create it in the current database. */ def createFunction(funcDefinition: CatalogFunction, ignoreIfExists: Boolean): Unit = { - val db = funcDefinition.identifier.database.getOrElse(currentDb) + val db = funcDefinition.identifier.database.getOrElse(getCurrentDatabase) val identifier = FunctionIdentifier(funcDefinition.identifier.funcName, Some(db)) val newFuncDefinition = funcDefinition.copy(identifier = identifier) if (!functionExists(identifier)) { @@ -547,7 +555,7 @@ class SessionCatalog( * If no database is specified, assume the function is in the current database. */ def dropFunction(name: FunctionIdentifier, ignoreIfNotExists: Boolean): Unit = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) val identifier = name.copy(database = Some(db)) if (functionExists(identifier)) { // TODO: registry should just take in FunctionIdentifier for type safety @@ -571,7 +579,7 @@ class SessionCatalog( * If no database is specified, this will return the function in the current database. */ def getFunctionMetadata(name: FunctionIdentifier): CatalogFunction = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) externalCatalog.getFunction(db, name.funcName) } @@ -579,7 +587,7 @@ class SessionCatalog( * Check if the specified function exists. */ def functionExists(name: FunctionIdentifier): Boolean = { - val db = name.database.getOrElse(currentDb) + val db = name.database.getOrElse(getCurrentDatabase) functionRegistry.functionExists(name.unquotedString) || externalCatalog.functionExists(db, name.funcName) } @@ -644,7 +652,7 @@ class SessionCatalog( /** * Look up the [[ExpressionInfo]] associated with the specified function, assuming it exists. */ - private[spark] def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = { + private[spark] def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = synchronized { // TODO: just make function registry take in FunctionIdentifier instead of duplicating this val qualifiedName = name.copy(database = name.database.orElse(Some(currentDb))) functionRegistry.lookupFunction(name.funcName) @@ -673,7 +681,9 @@ class SessionCatalog( * based on the function class and put the builder into the FunctionRegistry. * The name of this function in the FunctionRegistry will be `databaseName.functionName`. */ - def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { + def lookupFunction( + name: FunctionIdentifier, + children: Seq[Expression]): Expression = synchronized { // Note: the implementation of this function is a little bit convoluted. // We probably shouldn't use a single FunctionRegistry to register all three kinds of functions // (built-in, temp, and external). @@ -741,7 +751,7 @@ class SessionCatalog( * * This is mainly used for tests. */ - private[sql] def reset(): Unit = { + private[sql] def reset(): Unit = synchronized { val default = "default" listDatabases().filter(_ != default).foreach { db => dropDatabase(db, ignoreIfNotExists = false, cascade = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index c5df028485373..a49da6dc2b8b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -42,9 +42,9 @@ class ExperimentalMethods private[sql]() { * @since 1.3.0 */ @Experimental - var extraStrategies: Seq[Strategy] = Nil + @volatile var extraStrategies: Seq[Strategy] = Nil @Experimental - var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil + @volatile var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 9ed3756628c84..2a893c6478d14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -115,14 +115,17 @@ class SparkSession private( @transient private var _wrapped: SQLContext = _ - protected[sql] def wrapped: SQLContext = { + @transient + private val _wrappedLock = new Object + + protected[sql] def wrapped: SQLContext = _wrappedLock.synchronized { if (_wrapped == null) { _wrapped = new SQLContext(self, isRootContext = false) } _wrapped } - protected[sql] def setWrappedContext(sqlContext: SQLContext): Unit = { + protected[sql] def setWrappedContext(sqlContext: SQLContext): Unit = _wrappedLock.synchronized { _wrapped = sqlContext } 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 42746ece3c376..6d418c1dcf461 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 @@ -485,11 +485,11 @@ private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { private val removedFunctions = collection.mutable.ArrayBuffer.empty[(String, (ExpressionInfo, FunctionBuilder))] - def unregisterFunction(name: String): Unit = { + def unregisterFunction(name: String): Unit = synchronized { functionBuilders.remove(name).foreach(f => removedFunctions += name -> f) } - def restore(): Unit = { + def restore(): Unit = synchronized { removedFunctions.foreach { case (name, (info, builder)) => registerFunction(name, info, builder) } From 8b4ab590cb18b926c71c4cb4ec5b184b1b566770 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 5 May 2016 14:37:50 -0700 Subject: [PATCH 0092/1470] [SPARK-15134][EXAMPLE] Indent SparkSession builder patterns and update binary_classification_metrics_example.py ## What changes were proposed in this pull request? This issue addresses the comments in SPARK-15031 and also fix java-linter errors. - Use multiline format in SparkSession builder patterns. - Update `binary_classification_metrics_example.py` to use `SparkSession`. - Fix Java Linter errors (in SPARK-13745, SPARK-15031, and so far) ## How was this patch tested? After passing the Jenkins tests and run `dev/lint-java` manually. Author: Dongjoon Hyun Closes #12911 from dongjoon-hyun/SPARK-15134. (cherry picked from commit 2c170dd3d731bd848d62265431795e1c141d75d7) Signed-off-by: Andrew Or --- .../shuffle/ExternalShuffleBlockHandler.java | 7 +++++-- .../ml/JavaAFTSurvivalRegressionExample.java | 5 ++++- .../apache/spark/examples/ml/JavaALSExample.java | 5 ++++- .../spark/examples/ml/JavaBinarizerExample.java | 8 ++++---- .../examples/ml/JavaBisectingKMeansExample.java | 5 ++++- .../spark/examples/ml/JavaBucketizerExample.java | 5 ++++- .../examples/ml/JavaChiSqSelectorExample.java | 8 ++++---- .../examples/ml/JavaCountVectorizerExample.java | 5 ++++- .../apache/spark/examples/ml/JavaDCTExample.java | 8 ++++---- .../examples/ml/JavaDeveloperApiExample.java | 5 ++++- .../ml/JavaElementwiseProductExample.java | 7 +++---- .../JavaGradientBoostedTreeClassifierExample.java | 10 ++++++---- .../examples/ml/JavaIndexToStringExample.java | 5 ++++- .../spark/examples/ml/JavaKMeansExample.java | 5 ++++- .../apache/spark/examples/ml/JavaLDAExample.java | 5 ++++- .../examples/ml/JavaMaxAbsScalerExample.java | 14 ++++++++++---- .../examples/ml/JavaMinMaxScalerExample.java | 10 ++++++++-- .../spark/examples/ml/JavaNGramExample.java | 5 ++++- .../spark/examples/ml/JavaNaiveBayesExample.java | 5 ++++- .../spark/examples/ml/JavaNormalizerExample.java | 5 ++++- .../examples/ml/JavaOneHotEncoderExample.java | 5 ++++- .../spark/examples/ml/JavaOneVsRestExample.java | 5 ++++- .../apache/spark/examples/ml/JavaPCAExample.java | 5 ++++- .../spark/examples/ml/JavaPipelineExample.java | 5 ++++- .../ml/JavaPolynomialExpansionExample.java | 5 ++++- .../spark/examples/ml/JavaRFormulaExample.java | 5 ++++- .../examples/ml/JavaSQLTransformerExample.java | 5 ++++- .../examples/ml/JavaSimpleParamsExample.java | 5 ++++- .../examples/ml/JavaStandardScalerExample.java | 5 ++++- .../examples/ml/JavaStopWordsRemoverExample.java | 5 ++++- .../examples/ml/JavaStringIndexerExample.java | 5 ++++- .../spark/examples/ml/JavaTfIdfExample.java | 5 ++++- .../spark/examples/ml/JavaTokenizerExample.java | 5 ++++- .../examples/ml/JavaVectorAssemblerExample.java | 5 ++++- .../examples/ml/JavaVectorIndexerExample.java | 5 ++++- .../examples/ml/JavaVectorSlicerExample.java | 5 ++++- .../spark/examples/ml/JavaWord2VecExample.java | 5 ++++- .../apache/spark/examples/sql/JavaSparkSQL.java | 8 ++++++-- .../streaming/JavaSqlNetworkWordCount.java | 5 ++++- examples/src/main/python/ml/als_example.py | 5 ++++- examples/src/main/python/ml/binarizer_example.py | 5 ++++- .../main/python/ml/bisecting_k_means_example.py | 5 ++++- examples/src/main/python/ml/bucketizer_example.py | 5 ++++- .../src/main/python/ml/chisq_selector_example.py | 5 ++++- .../main/python/ml/count_vectorizer_example.py | 5 ++++- examples/src/main/python/ml/cross_validator.py | 5 ++++- examples/src/main/python/ml/dataframe_example.py | 5 ++++- examples/src/main/python/ml/dct_example.py | 5 ++++- .../ml/decision_tree_classification_example.py | 5 ++++- .../python/ml/decision_tree_regression_example.py | 5 ++++- .../main/python/ml/elementwise_product_example.py | 5 ++++- .../ml/estimator_transformer_param_example.py | 5 ++++- .../gradient_boosted_tree_classifier_example.py | 5 ++++- .../ml/gradient_boosted_tree_regressor_example.py | 5 ++++- .../src/main/python/ml/index_to_string_example.py | 5 ++++- examples/src/main/python/ml/kmeans_example.py | 5 ++++- .../ml/linear_regression_with_elastic_net.py | 5 ++++- .../ml/logistic_regression_with_elastic_net.py | 5 ++++- .../src/main/python/ml/max_abs_scaler_example.py | 5 ++++- .../src/main/python/ml/min_max_scaler_example.py | 5 ++++- examples/src/main/python/ml/n_gram_example.py | 5 ++++- .../src/main/python/ml/naive_bayes_example.py | 5 ++++- examples/src/main/python/ml/normalizer_example.py | 5 ++++- .../src/main/python/ml/onehot_encoder_example.py | 5 ++++- examples/src/main/python/ml/pca_example.py | 5 ++++- examples/src/main/python/ml/pipeline_example.py | 5 ++++- .../python/ml/polynomial_expansion_example.py | 5 ++++- .../python/ml/random_forest_classifier_example.py | 5 ++++- .../python/ml/random_forest_regressor_example.py | 5 ++++- examples/src/main/python/ml/rformula_example.py | 5 ++++- .../ml/simple_text_classification_pipeline.py | 5 ++++- examples/src/main/python/ml/sql_transformer.py | 5 ++++- .../src/main/python/ml/standard_scaler_example.py | 5 ++++- .../main/python/ml/stopwords_remover_example.py | 5 ++++- .../src/main/python/ml/string_indexer_example.py | 5 ++++- examples/src/main/python/ml/tf_idf_example.py | 5 ++++- examples/src/main/python/ml/tokenizer_example.py | 5 ++++- .../src/main/python/ml/train_validation_split.py | 5 ++++- .../main/python/ml/vector_assembler_example.py | 5 ++++- .../src/main/python/ml/vector_indexer_example.py | 5 ++++- .../src/main/python/ml/vector_slicer_example.py | 5 ++++- examples/src/main/python/ml/word2vec_example.py | 5 ++++- .../binary_classification_metrics_example.py | 15 ++++++++++----- examples/src/main/python/sql.py | 5 ++++- .../python/streaming/sql_network_wordcount.py | 6 ++++-- .../ml/AFTSurvivalRegressionExample.scala | 5 ++++- .../org/apache/spark/examples/ml/ALSExample.scala | 5 ++++- .../spark/examples/ml/BinarizerExample.scala | 5 ++++- .../spark/examples/ml/BucketizerExample.scala | 5 ++++- .../spark/examples/ml/ChiSqSelectorExample.scala | 5 ++++- .../examples/ml/CountVectorizerExample.scala | 5 ++++- .../org/apache/spark/examples/ml/DCTExample.scala | 5 ++++- .../spark/examples/ml/DataFrameExample.scala | 5 ++++- .../ml/DecisionTreeClassificationExample.scala | 5 ++++- .../spark/examples/ml/DecisionTreeExample.scala | 4 +++- .../ml/DecisionTreeRegressionExample.scala | 5 ++++- .../spark/examples/ml/DeveloperApiExample.scala | 5 ++++- .../examples/ml/ElementwiseProductExample.scala | 5 ++++- .../ml/EstimatorTransformerParamExample.scala | 5 ++++- .../ml/GradientBoostedTreeClassifierExample.scala | 5 ++++- .../ml/GradientBoostedTreeRegressorExample.scala | 5 ++++- .../spark/examples/ml/IndexToStringExample.scala | 5 ++++- .../apache/spark/examples/ml/KMeansExample.scala | 5 ++++- .../org/apache/spark/examples/ml/LDAExample.scala | 5 ++++- .../LinearRegressionWithElasticNetExample.scala | 5 ++++- .../ml/LogisticRegressionSummaryExample.scala | 5 ++++- .../spark/examples/ml/MaxAbsScalerExample.scala | 5 ++++- .../spark/examples/ml/MinMaxScalerExample.scala | 5 ++++- .../MultilayerPerceptronClassifierExample.scala | 5 ++++- .../apache/spark/examples/ml/NGramExample.scala | 5 ++++- .../spark/examples/ml/NaiveBayesExample.scala | 5 ++++- .../spark/examples/ml/NormalizerExample.scala | 5 ++++- .../spark/examples/ml/OneHotEncoderExample.scala | 5 ++++- .../spark/examples/ml/OneVsRestExample.scala | 5 ++++- .../org/apache/spark/examples/ml/PCAExample.scala | 5 ++++- .../spark/examples/ml/PipelineExample.scala | 5 ++++- .../examples/ml/PolynomialExpansionExample.scala | 5 ++++- .../examples/ml/QuantileDiscretizerExample.scala | 5 ++++- .../spark/examples/ml/RFormulaExample.scala | 5 ++++- .../ml/RandomForestClassifierExample.scala | 5 ++++- .../ml/RandomForestRegressorExample.scala | 5 ++++- .../spark/examples/ml/SQLTransformerExample.scala | 5 ++++- .../spark/examples/ml/SimpleParamsExample.scala | 5 ++++- .../ml/SimpleTextClassificationPipeline.scala | 5 ++++- .../spark/examples/ml/StandardScalerExample.scala | 5 ++++- .../examples/ml/StopWordsRemoverExample.scala | 5 ++++- .../spark/examples/ml/StringIndexerExample.scala | 5 ++++- .../apache/spark/examples/ml/TfIdfExample.scala | 5 ++++- .../spark/examples/ml/TokenizerExample.scala | 5 ++++- .../examples/ml/VectorAssemblerExample.scala | 5 ++++- .../spark/examples/ml/VectorIndexerExample.scala | 5 ++++- .../spark/examples/ml/VectorSlicerExample.scala | 5 ++++- .../spark/examples/ml/Word2VecExample.scala | 5 ++++- .../apache/spark/examples/mllib/LDAExample.scala | 4 +++- .../examples/mllib/RankingMetricsExample.scala | 5 ++++- .../examples/mllib/RegressionMetricsExample.scala | 5 ++++- .../apache/spark/examples/sql/RDDRelation.scala | 5 ++++- .../examples/streaming/SqlNetworkWordCount.scala | 5 ++++- .../parquet/VectorizedPlainValuesReader.java | 5 +++-- .../execution/vectorized/OffHeapColumnVector.java | 15 ++++++++------- .../execution/vectorized/OnHeapColumnVector.java | 7 ++++--- .../hive/service/cli/session/SessionManager.java | 2 -- 142 files changed, 585 insertions(+), 178 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index fb1226c09e246..22fd592a321d2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -87,8 +87,11 @@ protected void handleMessage( blocks.add(blockManager.getBlockData(msg.appId, msg.execId, blockId)); } long streamId = streamManager.registerStream(client.getClientId(), blocks.iterator()); - logger.trace("Registered streamId {} with {} buffers for client {} from host {}", streamId, - msg.blockIds.length, client.getClientId(), NettyUtils.getRemoteAddress(client.getChannel())); + logger.trace("Registered streamId {} with {} buffers for client {} from host {}", + streamId, + msg.blockIds.length, + client.getClientId(), + NettyUtils.getRemoteAddress(client.getChannel())); callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); } else if (msgObj instanceof RegisterExecutor) { diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java index ecb7084e03122..2c2aa6df47c77 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java @@ -33,7 +33,10 @@ public class JavaAFTSurvivalRegressionExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaAFTSurvivalRegressionExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaAFTSurvivalRegressionExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 9a9a10489b506..4b13ba6f9cea3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -81,7 +81,10 @@ public static Rating parseRating(String str) { // $example off$ public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaALSExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaALSExample") + .getOrCreate(); // $example on$ JavaRDD ratingsRDD = spark diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java index 88e4298a6191e..5f964aca92096 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -26,7 +24,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.Binarizer; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -38,7 +35,10 @@ public class JavaBinarizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaBinarizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaBinarizerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 51aa35084e845..810ad905c56af 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -42,7 +42,10 @@ public class JavaBisectingKMeansExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaBisectingKMeansExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaBisectingKMeansExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java index 0c24f52cf58ab..691df3887a9bb 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java @@ -35,7 +35,10 @@ public class JavaBucketizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaBucketizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaBucketizerExample") + .getOrCreate(); // $example on$ double[] splits = {Double.NEGATIVE_INFINITY, -0.5, 0.0, 0.5, Double.POSITIVE_INFINITY}; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java index 684cf9a7145d2..f8f2fb14be1f1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java @@ -17,9 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -40,7 +37,10 @@ public class JavaChiSqSelectorExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaChiSqSelectorExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaChiSqSelectorExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java index 0631f9d6d5d74..0a6b13601425b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java @@ -32,7 +32,10 @@ public class JavaCountVectorizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaCountVectorizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaCountVectorizerExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java index ec57a2445138a..eee92c77a8c58 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -26,7 +24,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.DCT; import org.apache.spark.mllib.linalg.VectorUDT; import org.apache.spark.mllib.linalg.Vectors; @@ -39,7 +36,10 @@ public class JavaDCTExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaDCTExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaDCTExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( 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 90023ac06b79b..49bad0afc0b71 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 @@ -49,7 +49,10 @@ public class JavaDeveloperApiExample { public static void main(String[] args) throws Exception { - SparkSession spark = SparkSession.builder().appName("JavaDeveloperApiExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaDeveloperApiExample") + .getOrCreate(); // Prepare training data. List localTraining = Lists.newArrayList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java index a062a6fcd0c19..9126242f9eb78 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; @@ -27,7 +25,6 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.ElementwiseProduct; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.VectorUDT; @@ -42,7 +39,9 @@ public class JavaElementwiseProductExample { public static void main(String[] args) { SparkSession spark = SparkSession - .builder().appName("JavaElementwiseProductExample").getOrCreate(); + .builder() + .appName("JavaElementwiseProductExample") + .getOrCreate(); // $example on$ // Create some vector data; also works for sparse vectors diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java index a7c89b9d19974..baacd796a058f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; // $example on$ import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -35,11 +33,15 @@ public class JavaGradientBoostedTreeClassifierExample { public static void main(String[] args) { SparkSession spark = SparkSession - .builder().appName("JavaGradientBoostedTreeClassifierExample").getOrCreate(); + .builder() + .appName("JavaGradientBoostedTreeClassifierExample") + .getOrCreate(); // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset data = spark.read().format("libsvm") + Dataset data = spark + .read() + .format("libsvm") .load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java index ccd74f2920ce3..0064beb8c8f33 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java @@ -37,7 +37,10 @@ public class JavaIndexToStringExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaIndexToStringExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaIndexToStringExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index e6d82a0513a30..65e29ade299d1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -70,7 +70,10 @@ public static void main(String[] args) { int k = Integer.parseInt(args[1]); // Parses the arguments - SparkSession spark = SparkSession.builder().appName("JavaKMeansExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaKMeansExample") + .getOrCreate(); // $example on$ // Loads data diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java index b8baca5920e8d..1c52f37867a85 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java @@ -65,7 +65,10 @@ public static void main(String[] args) { String inputFile = "data/mllib/sample_lda_data.txt"; // Parses the arguments - SparkSession spark = SparkSession.builder().appName("JavaLDAExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaLDAExample") + .getOrCreate(); // Loads data JavaRDD points = spark.read().text(inputFile).javaRDD().map(new ParseVector()); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java index 80cdd364b9376..9a27b0e9e23b7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java @@ -28,13 +28,19 @@ public class JavaMaxAbsScalerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaMaxAbsScalerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaMaxAbsScalerExample") + .getOrCreate(); // $example on$ - Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); MaxAbsScaler scaler = new MaxAbsScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures"); + .setInputCol("features") + .setOutputCol("scaledFeatures"); // Compute summary statistics and generate MaxAbsScalerModel MaxAbsScalerModel scalerModel = scaler.fit(dataFrame); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java index 022940fd1e67c..37fa1c5434ea6 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java @@ -28,10 +28,16 @@ public class JavaMinMaxScalerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaMinMaxScalerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaMinMaxScalerExample") + .getOrCreate(); // $example on$ - Dataset dataFrame = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset dataFrame = spark + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); MinMaxScaler scaler = new MinMaxScaler() .setInputCol("features") .setOutputCol("scaledFeatures"); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java index 325b7b5874800..899815f57c84b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java @@ -35,7 +35,10 @@ public class JavaNGramExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaNGramExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaNGramExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java index 1f24a23609ebe..50a46a5774fcc 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java @@ -32,7 +32,10 @@ public class JavaNaiveBayesExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaNaiveBayesExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaNaiveBayesExample") + .getOrCreate(); // $example on$ // Load training data diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java index 4b3a718ea92c8..abc38f85ea774 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java @@ -27,7 +27,10 @@ public class JavaNormalizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaNormalizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaNormalizerExample") + .getOrCreate(); // $example on$ Dataset dataFrame = diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java index d6e4d21ead161..5d29e54549213 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -37,7 +37,10 @@ public class JavaOneHotEncoderExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaOneHotEncoderExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaOneHotEncoderExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java index 9cc983bd115df..e0cb752224f75 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java @@ -58,7 +58,10 @@ private static class Params { public static void main(String[] args) { // parse the arguments Params params = parse(args); - SparkSession spark = SparkSession.builder().appName("JavaOneVsRestExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaOneVsRestExample") + .getOrCreate(); // $example on$ // configure the base classifier diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java index 6b1dcb68ba304..ffa979ee013ad 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java @@ -37,7 +37,10 @@ public class JavaPCAExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaPCAExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaPCAExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java index 556a4573263f1..9a43189c91463 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java @@ -36,7 +36,10 @@ */ public class JavaPipelineExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaPipelineExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaPipelineExample") + .getOrCreate(); // $example on$ // Prepare training documents, which are labeled. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java index e328454c70591..7afcd0e50cd95 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -36,7 +36,10 @@ public class JavaPolynomialExpansionExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaPolynomialExpansionExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaPolynomialExpansionExample") + .getOrCreate(); // $example on$ PolynomialExpansion polyExpansion = new PolynomialExpansion() diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java index 8282ce01d36f3..428067e0f7efe 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java @@ -35,7 +35,10 @@ public class JavaRFormulaExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaRFormulaExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaRFormulaExample") + .getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java index 492718bbdba81..2a3d62de41ab7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java @@ -31,7 +31,10 @@ public class JavaSQLTransformerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaSQLTransformerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaSQLTransformerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index f906843640eda..0787079ba4e55 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -40,7 +40,10 @@ public class JavaSimpleParamsExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaSimpleParamsExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaSimpleParamsExample") + .getOrCreate(); // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java index 10f82f223386e..08ea285a0d53d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java @@ -28,7 +28,10 @@ public class JavaStandardScalerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaStandardScalerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaStandardScalerExample") + .getOrCreate(); // $example on$ Dataset dataFrame = diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java index 23ed071c9f6e5..def5994429124 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java @@ -36,7 +36,10 @@ public class JavaStopWordsRemoverExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaStopWordsRemoverExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaStopWordsRemoverExample") + .getOrCreate(); // $example on$ StopWordsRemover remover = new StopWordsRemover() diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java index d4c2cf96a7cf3..7533c1835e325 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java @@ -35,7 +35,10 @@ public class JavaStringIndexerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaStringIndexerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaStringIndexerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index a8169917772c1..6e0753959efd6 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -38,7 +38,10 @@ public class JavaTfIdfExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaTfIdfExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaTfIdfExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java index a65735a5e5143..1cc16bb60d172 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -36,7 +36,10 @@ public class JavaTokenizerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaTokenizerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaTokenizerExample") + .getOrCreate(); // $example on$ List data = Arrays.asList( diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java index 9569bc2412def..41f1d8750ac40 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java @@ -35,7 +35,10 @@ public class JavaVectorAssemblerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaVectorAssemblerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorAssemblerExample") + .getOrCreate(); // $example on$ StructType schema = createStructType(new StructField[]{ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java index 217d5a06d10a4..dd9d757dd6831 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java @@ -30,7 +30,10 @@ public class JavaVectorIndexerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaVectorIndexerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorIndexerExample") + .getOrCreate(); // $example on$ Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java index 4f1ea824a3a9f..24959c0e10f2b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java @@ -37,7 +37,10 @@ public class JavaVectorSlicerExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaVectorSlicerExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorSlicerExample") + .getOrCreate(); // $example on$ Attribute[] attrs = new Attribute[]{ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java index d9b1a79b52548..9be6e6353adcf 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java @@ -32,7 +32,10 @@ public class JavaWord2VecExample { public static void main(String[] args) { - SparkSession spark = SparkSession.builder().appName("JavaWord2VecExample").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaWord2VecExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. 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 ec2142e756955..755b4f538104a 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 @@ -51,7 +51,10 @@ public void setAge(int age) { } public static void main(String[] args) throws Exception { - SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate(); + SparkSession spark = SparkSession + .builder() + .appName("JavaSparkSQL") + .getOrCreate(); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. @@ -147,7 +150,8 @@ public String call(Row row) { // a RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); - JavaRDD anotherPeopleRDD = spark.createDataFrame(jsonData, String.class).toJSON().javaRDD(); + JavaRDD anotherPeopleRDD = spark + .createDataFrame(jsonData, String.class).toJSON().javaRDD(); Dataset peopleFromJsonRDD = spark.read().json(anotherPeopleRDD); // Take a look at the schema of this new DataFrame. diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 44f1e800fe448..57953ef74f795 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -115,7 +115,10 @@ class JavaSparkSessionSingleton { private static transient SparkSession instance = null; public static SparkSession getInstance(SparkConf sparkConf) { if (instance == null) { - instance = SparkSession.builder().config(sparkConf).getOrCreate(); + instance = SparkSession + .builder() + .config(sparkConf) + .getOrCreate(); } return instance; } diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index e36444f18506e..ff0829b0dd45a 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -30,7 +30,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("ALSExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("ALSExample")\ + .getOrCreate() # $example on$ lines = spark.read.text("data/mllib/als/sample_movielens_ratings.txt").rdd diff --git a/examples/src/main/python/ml/binarizer_example.py b/examples/src/main/python/ml/binarizer_example.py index 072187e64564c..4224a27dbef0c 100644 --- a/examples/src/main/python/ml/binarizer_example.py +++ b/examples/src/main/python/ml/binarizer_example.py @@ -23,7 +23,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("BinarizerExample")\ + .getOrCreate() # $example on$ continuousDataFrame = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index 836a89cde0d60..540a4bc3e4bf9 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -30,7 +30,10 @@ """ if __name__ == "__main__": - spark = SparkSession.builder.appName("PythonBisectingKMeansExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PythonBisectingKMeansExample")\ + .getOrCreate() # $example on$ data = spark.read.text("data/mllib/kmeans_data.txt").rdd diff --git a/examples/src/main/python/ml/bucketizer_example.py b/examples/src/main/python/ml/bucketizer_example.py index 288ec62bdfbb5..8177e560ddef1 100644 --- a/examples/src/main/python/ml/bucketizer_example.py +++ b/examples/src/main/python/ml/bucketizer_example.py @@ -23,7 +23,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("BucketizerExample")\ + .getOrCreate() # $example on$ splits = [-float("inf"), -0.5, 0.0, 0.5, float("inf")] diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py index 8f58fc28deb9b..8bafb942e0d27 100644 --- a/examples/src/main/python/ml/chisq_selector_example.py +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -24,7 +24,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("ChiSqSelectorExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/count_vectorizer_example.py b/examples/src/main/python/ml/count_vectorizer_example.py index 9dbf9959d17ef..38cfac82fbe20 100644 --- a/examples/src/main/python/ml/count_vectorizer_example.py +++ b/examples/src/main/python/ml/count_vectorizer_example.py @@ -23,7 +23,10 @@ # $example off$ if __name__ == "__main__": - spark = SparkSession.builder.appName("CountVectorizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("CountVectorizerExample")\ + .getOrCreate() # $example on$ # Input data: Each row is a bag of words with a ID. diff --git a/examples/src/main/python/ml/cross_validator.py b/examples/src/main/python/ml/cross_validator.py index a61d0f63d277c..a41df6cf946fb 100644 --- a/examples/src/main/python/ml/cross_validator.py +++ b/examples/src/main/python/ml/cross_validator.py @@ -35,7 +35,10 @@ """ if __name__ == "__main__": - spark = SparkSession.builder.appName("CrossValidatorExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("CrossValidatorExample")\ + .getOrCreate() # $example on$ # Prepare training documents, which are labeled. training = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py index b3e671038eb7f..a7d8b9056dffe 100644 --- a/examples/src/main/python/ml/dataframe_example.py +++ b/examples/src/main/python/ml/dataframe_example.py @@ -33,7 +33,10 @@ if len(sys.argv) > 2: print("Usage: dataframe_example.py ", file=sys.stderr) exit(-1) - spark = SparkSession.builder.appName("DataFrameExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("DataFrameExample")\ + .getOrCreate() if len(sys.argv) == 2: input = sys.argv[1] else: diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py index 1bf8fc6d140c9..e36fcdeaeed28 100644 --- a/examples/src/main/python/ml/dct_example.py +++ b/examples/src/main/python/ml/dct_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("DCTExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("DCTExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index d2318e24369d8..9b40b701ecaaf 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("decision_tree_classification_example")\ + .getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index 9e8cb382a9bfe..b734d4974a4f6 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("decision_tree_classification_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("decision_tree_classification_example")\ + .getOrCreate() # $example on$ # Load the data stored in LIBSVM format as a DataFrame. diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index 6fa641b772d95..41727edcdb09e 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("ElementwiseProductExample")\ + .getOrCreate() # $example on$ data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py index 4993b5a9846c2..0fcae0e3fc225 100644 --- a/examples/src/main/python/ml/estimator_transformer_param_example.py +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -26,7 +26,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("EstimatorTransformerParamExample")\ + .getOrCreate() # $example on$ # Prepare training data from a list of (label, features) tuples. diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index b09ad41da313c..50026d7b7e3e8 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("gradient_boosted_tree_classifier_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("gradient_boosted_tree_classifier_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py index caa7cfc4e11f5..5dd2272748d70 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("gradient_boosted_tree_regressor_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("gradient_boosted_tree_regressor_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/index_to_string_example.py b/examples/src/main/python/ml/index_to_string_example.py index dd04b2c4b0bfd..523caac00c18a 100644 --- a/examples/src/main/python/ml/index_to_string_example.py +++ b/examples/src/main/python/ml/index_to_string_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("IndexToStringExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame( diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index 7d9d80e6452bd..73823969554fa 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -49,7 +49,10 @@ def parseVector(row): path = sys.argv[1] k = sys.argv[2] - spark = SparkSession.builder.appName("PythonKMeansExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PythonKMeansExample")\ + .getOrCreate() lines = spark.read.text(path).rdd data = lines.map(parseVector) diff --git a/examples/src/main/python/ml/linear_regression_with_elastic_net.py b/examples/src/main/python/ml/linear_regression_with_elastic_net.py index 99b7f7fe99de6..620ab5b87e594 100644 --- a/examples/src/main/python/ml/linear_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/linear_regression_with_elastic_net.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("LinearRegressionWithElasticNet").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("LinearRegressionWithElasticNet")\ + .getOrCreate() # $example on$ # Load training data diff --git a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py index 0d7112e72354f..33d0689f75cd5 100644 --- a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py +++ b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("LogisticRegressionWithElasticNet").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("LogisticRegressionWithElasticNet")\ + .getOrCreate() # $example on$ # Load training data diff --git a/examples/src/main/python/ml/max_abs_scaler_example.py b/examples/src/main/python/ml/max_abs_scaler_example.py index 1cb95a98f08dc..ab91198b083d1 100644 --- a/examples/src/main/python/ml/max_abs_scaler_example.py +++ b/examples/src/main/python/ml/max_abs_scaler_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("MaxAbsScalerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/min_max_scaler_example.py b/examples/src/main/python/ml/min_max_scaler_example.py index 8d91a59e2b0bd..e3e7bc205b1ec 100644 --- a/examples/src/main/python/ml/min_max_scaler_example.py +++ b/examples/src/main/python/ml/min_max_scaler_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("MinMaxScalerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/n_gram_example.py b/examples/src/main/python/ml/n_gram_example.py index b7fecf0d685c0..9ac07f2c8ee20 100644 --- a/examples/src/main/python/ml/n_gram_example.py +++ b/examples/src/main/python/ml/n_gram_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("NGramExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("NGramExample")\ + .getOrCreate() # $example on$ wordDataFrame = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py index e37035542c7e8..89255a2bae64c 100644 --- a/examples/src/main/python/ml/naive_bayes_example.py +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("naive_bayes_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("naive_bayes_example")\ + .getOrCreate() # $example on$ # Load training data diff --git a/examples/src/main/python/ml/normalizer_example.py b/examples/src/main/python/ml/normalizer_example.py index ae2553761928c..19012f51f4023 100644 --- a/examples/src/main/python/ml/normalizer_example.py +++ b/examples/src/main/python/ml/normalizer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("NormalizerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/onehot_encoder_example.py b/examples/src/main/python/ml/onehot_encoder_example.py index 9acc363dc9e25..b9fceef68e703 100644 --- a/examples/src/main/python/ml/onehot_encoder_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("OneHotEncoderExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py index adab151734aa8..f1b3cdec7bd77 100644 --- a/examples/src/main/python/ml/pca_example.py +++ b/examples/src/main/python/ml/pca_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("PCAExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PCAExample")\ + .getOrCreate() # $example on$ data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), diff --git a/examples/src/main/python/ml/pipeline_example.py b/examples/src/main/python/ml/pipeline_example.py index ed9765d96165e..bd10cfd7a252b 100644 --- a/examples/src/main/python/ml/pipeline_example.py +++ b/examples/src/main/python/ml/pipeline_example.py @@ -27,7 +27,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("PipelineExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PipelineExample")\ + .getOrCreate() # $example on$ # Prepare training documents from a list of (id, text, label) tuples. diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 328b559320f85..08882bcb256f4 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PolynomialExpansionExample")\ + .getOrCreate() # $example on$ df = spark\ diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index b0a93e050c54b..c618eaf60c2e9 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("random_forest_classifier_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("random_forest_classifier_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py index 4bb84f0de8ff3..3a793737dba89 100644 --- a/examples/src/main/python/ml/random_forest_regressor_example.py +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -29,7 +29,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("random_forest_regressor_example").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("random_forest_regressor_example")\ + .getOrCreate() # $example on$ # Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/python/ml/rformula_example.py b/examples/src/main/python/ml/rformula_example.py index 45cc116ac2f61..d5df3ce4f5915 100644 --- a/examples/src/main/python/ml/rformula_example.py +++ b/examples/src/main/python/ml/rformula_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("RFormulaExample")\ + .getOrCreate() # $example on$ dataset = spark.createDataFrame( 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 3600c12211d05..886f43c0b08e8 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,10 @@ if __name__ == "__main__": - spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("SimpleTextClassificationPipeline")\ + .getOrCreate() # Prepare training documents, which are labeled. training = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/sql_transformer.py b/examples/src/main/python/ml/sql_transformer.py index 26045db4be6bc..0bf8f35720c95 100644 --- a/examples/src/main/python/ml/sql_transformer.py +++ b/examples/src/main/python/ml/sql_transformer.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("SQLTransformerExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/standard_scaler_example.py b/examples/src/main/python/ml/standard_scaler_example.py index c50804f6bf4e0..c0027480e69b3 100644 --- a/examples/src/main/python/ml/standard_scaler_example.py +++ b/examples/src/main/python/ml/standard_scaler_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("StandardScalerExample")\ + .getOrCreate() # $example on$ dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/stopwords_remover_example.py b/examples/src/main/python/ml/stopwords_remover_example.py index 57362673df786..395fdeffc5379 100644 --- a/examples/src/main/python/ml/stopwords_remover_example.py +++ b/examples/src/main/python/ml/stopwords_remover_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("StopWordsRemoverExample")\ + .getOrCreate() # $example on$ sentenceData = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/string_indexer_example.py b/examples/src/main/python/ml/string_indexer_example.py index aacd4f999bb1b..a328e040f5636 100644 --- a/examples/src/main/python/ml/string_indexer_example.py +++ b/examples/src/main/python/ml/string_indexer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("StringIndexerExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame( diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index 25df8166efc2e..fb4ad992fb809 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("TfIdfExample")\ + .getOrCreate() # $example on$ sentenceData = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/tokenizer_example.py b/examples/src/main/python/ml/tokenizer_example.py index 5be4b4cfe3a52..e61ec920d2281 100644 --- a/examples/src/main/python/ml/tokenizer_example.py +++ b/examples/src/main/python/ml/tokenizer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("TokenizerExample")\ + .getOrCreate() # $example on$ sentenceDataFrame = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/train_validation_split.py b/examples/src/main/python/ml/train_validation_split.py index 2e43a0f8aec54..5f5c52aca8c42 100644 --- a/examples/src/main/python/ml/train_validation_split.py +++ b/examples/src/main/python/ml/train_validation_split.py @@ -31,7 +31,10 @@ """ if __name__ == "__main__": - spark = SparkSession.builder.appName("TrainValidationSplit").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("TrainValidationSplit")\ + .getOrCreate() # $example on$ # Prepare training and test data. data = spark.read.format("libsvm")\ diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py index 019a9ea6f7503..b955ff00a8195 100644 --- a/examples/src/main/python/ml/vector_assembler_example.py +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -24,7 +24,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("VectorAssemblerExample")\ + .getOrCreate() # $example on$ dataset = spark.createDataFrame( diff --git a/examples/src/main/python/ml/vector_indexer_example.py b/examples/src/main/python/ml/vector_indexer_example.py index 3cf5b8ebf17ee..9b00e0f84136c 100644 --- a/examples/src/main/python/ml/vector_indexer_example.py +++ b/examples/src/main/python/ml/vector_indexer_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("VectorIndexerExample")\ + .getOrCreate() # $example on$ data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index 0531bcdb06675..b833a894eb841 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -25,7 +25,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("VectorSlicerExample")\ + .getOrCreate() # $example on$ df = spark.createDataFrame([ diff --git a/examples/src/main/python/ml/word2vec_example.py b/examples/src/main/python/ml/word2vec_example.py index 6766a7b6aa454..66500bee152f7 100644 --- a/examples/src/main/python/ml/word2vec_example.py +++ b/examples/src/main/python/ml/word2vec_example.py @@ -23,7 +23,10 @@ from pyspark.sql import SparkSession if __name__ == "__main__": - spark = SparkSession.builder.appName("Word2VecExample").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("Word2VecExample")\ + .getOrCreate() # $example on$ # Input data: Each row is a bag of words from a sentence or document. diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py index 8f0fc9d45da2b..daf000e38dcd0 100644 --- a/examples/src/main/python/mllib/binary_classification_metrics_example.py +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -18,20 +18,25 @@ Binary Classification Metrics Example. """ from __future__ import print_function -from pyspark import SparkContext +from pyspark.sql import SparkSession # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS from pyspark.mllib.evaluation import BinaryClassificationMetrics -from pyspark.mllib.util import MLUtils +from pyspark.mllib.regression import LabeledPoint # $example off$ if __name__ == "__main__": - sc = SparkContext(appName="BinaryClassificationMetricsExample") + spark = SparkSession\ + .builder\ + .appName("BinaryClassificationMetricsExample")\ + .getOrCreate() # $example on$ # Several of the methods available in scala are currently missing from pyspark # Load training data in LIBSVM format - data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_binary_classification_data.txt") + data = spark\ + .read.format("libsvm").load("data/mllib/sample_binary_classification_data.txt")\ + .rdd.map(lambda row: LabeledPoint(row[0], row[1])) # Split data into training (60%) and test (40%) training, test = data.randomSplit([0.6, 0.4], seed=11L) @@ -53,4 +58,4 @@ print("Area under ROC = %s" % metrics.areaUnderROC) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 59a46cb283c49..5594223465f87 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -25,7 +25,10 @@ if __name__ == "__main__": - spark = SparkSession.builder.appName("PythonSQL").getOrCreate() + spark = SparkSession\ + .builder\ + .appName("PythonSQL")\ + .getOrCreate() # A list of Rows. Infer schema from the first row, create a DataFrame and print the schema rows = [Row(name="John", age=19), Row(name="Smith", age=23), Row(name="Sarah", age=18)] diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index 588cbfee14baf..f8801d4ea63e2 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -38,8 +38,10 @@ def getSparkSessionInstance(sparkConf): if ('sparkSessionSingletonInstance' not in globals()): - globals()['sparkSessionSingletonInstance'] =\ - SparkSession.builder.config(conf=sparkConf).getOrCreate() + globals()['sparkSessionSingletonInstance'] = SparkSession\ + .builder\ + .config(conf=sparkConf)\ + .getOrCreate() return globals()['sparkSessionSingletonInstance'] diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala index 3795af83094a6..2b224d50a0a3c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -30,7 +30,10 @@ import org.apache.spark.sql.SparkSession object AFTSurvivalRegressionExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("AFTSurvivalRegressionExample").getOrCreate() + val spark = SparkSession + .builder + .appName("AFTSurvivalRegressionExample") + .getOrCreate() // $example on$ val training = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index 41750ca779146..7c1cfe293717a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -42,7 +42,10 @@ object ALSExample { // $example off$ def main(args: Array[String]) { - val spark = SparkSession.builder.appName("ALSExample").getOrCreate() + val spark = SparkSession + .builder + .appName("ALSExample") + .getOrCreate() import spark.implicits._ // $example on$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala index 93c153f923232..82bc14789b461 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.{DataFrame, SparkSession} object BinarizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("BinarizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("BinarizerExample") + .getOrCreate() // $example on$ val data = Array((0, 0.1), (1, 0.8), (2, 0.2)) val dataFrame: DataFrame = spark.createDataFrame(data).toDF("label", "feature") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala index 779ad33dbda0a..38cce34bb5091 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object BucketizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("BucketizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("BucketizerExample") + .getOrCreate() // $example on$ val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 84ca1f0b564d2..80f50cd3556af 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object ChiSqSelectorExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("ChiSqSelectorExample").getOrCreate() + val spark = SparkSession + .builder + .appName("ChiSqSelectorExample") + .getOrCreate() import spark.implicits._ // $example on$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index 9ab43a48bff8e..51aa5179fa4a8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object CountVectorizerExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("CounterVectorizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("CounterVectorizerExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala index b415333c710c7..5a888b15eb1c9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object DCTExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("DCTExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DCTExample") + .getOrCreate() // $example on$ val data = Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 2f892f8d72c1a..6cb81cde6f795 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -61,7 +61,10 @@ object DataFrameExample { } def run(params: Params) { - val spark = SparkSession.builder.appName(s"DataFrameExample with $params").getOrCreate() + val spark = SparkSession + .builder + .appName(s"DataFrameExample with $params") + .getOrCreate() // Load input data println(s"Loading LIBSVM file with UDT from ${params.input}.") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala index a0a2e1fb33dce..7f6c8de9679f9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.SparkSession object DecisionTreeClassificationExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("DecisionTreeClassificationExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DecisionTreeClassificationExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index cea1d801aa73a..eadb02ab0d7f7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -167,7 +167,9 @@ object DecisionTreeExample { testInput: String, algo: String, fracTest: Double): (DataFrame, DataFrame) = { - val spark = SparkSession.builder.getOrCreate() + val spark = SparkSession + .builder + .getOrCreate() // Load training data val origExamples: DataFrame = loadData(spark, input, dataFormat) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala index 26b52d0489e1c..799070ef47da6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.SparkSession object DecisionTreeRegressionExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("DecisionTreeRegressionExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DecisionTreeRegressionExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. 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 2aa1ab1ec855f..a522d2127edca 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 @@ -37,7 +37,10 @@ import org.apache.spark.sql.{Dataset, Row, SparkSession} object DeveloperApiExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("DeveloperApiExample").getOrCreate() + val spark = SparkSession + .builder + .appName("DeveloperApiExample") + .getOrCreate() import spark.implicits._ // Prepare training data. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala index f289c28df9b53..b99b76e58cef7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object ElementwiseProductExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("ElementwiseProductExample").getOrCreate() + val spark = SparkSession + .builder + .appName("ElementwiseProductExample") + .getOrCreate() // $example on$ // Create some vector data; also works for sparse vectors diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala index 91076ccbc14d4..972241e76922e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -29,7 +29,10 @@ import org.apache.spark.sql.SparkSession object EstimatorTransformerParamExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("EstimatorTransformerParamExample").getOrCreate() + val spark = SparkSession + .builder + .appName("EstimatorTransformerParamExample") + .getOrCreate() // $example on$ // Prepare training data from a list of (label, features) tuples. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala index 412c54db7d5c1..b6a8baba2d95f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object GradientBoostedTreeClassifierExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("GradientBoostedTreeClassifierExample").getOrCreate() + val spark = SparkSession + .builder + .appName("GradientBoostedTreeClassifierExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala index fd43553cc69df..62285b83cbb9d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object GradientBoostedTreeRegressorExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("GradientBoostedTreeRegressorExample").getOrCreate() + val spark = SparkSession + .builder + .appName("GradientBoostedTreeRegressorExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala index d873618726c6e..950733831c3d5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object IndexToStringExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("IndexToStringExample").getOrCreate() + val spark = SparkSession + .builder + .appName("IndexToStringExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala index d2573fad3596f..2abd588c6f0e4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala @@ -36,7 +36,10 @@ object KMeansExample { def main(args: Array[String]): Unit = { // Creates a Spark context and a SQL context - val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() // $example on$ // Crates a DataFrame diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala index c23adee1a3ead..c2920f6a5d4a1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala @@ -40,7 +40,10 @@ object LDAExample { val input = "data/mllib/sample_lda_data.txt" // Creates a Spark context and a SQL context - val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() // $example on$ // Loads data diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala index cb6e2492f513f..94cf2866238b9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object LinearRegressionWithElasticNetExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("LinearRegressionWithElasticNetExample").getOrCreate() + val spark = SparkSession + .builder + .appName("LinearRegressionWithElasticNetExample") + .getOrCreate() // $example on$ // Load training data diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala index 50670d7b38416..cd8775c942162 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala @@ -27,7 +27,10 @@ import org.apache.spark.sql.functions.max object LogisticRegressionSummaryExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("LogisticRegressionSummaryExample").getOrCreate() + val spark = SparkSession + .builder + .appName("LogisticRegressionSummaryExample") + .getOrCreate() import spark.implicits._ // Load training data diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala index 896d8fadbe67c..572adce657081 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala @@ -24,7 +24,10 @@ import org.apache.spark.sql.SparkSession object MaxAbsScalerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("MaxAbsScalerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("MaxAbsScalerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala index bcdca0fa0440d..d728019a621d4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object MinMaxScalerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("MinMaxScalerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("MinMaxScalerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala index a11fe1b4b2627..0e780fb7d342e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala @@ -30,7 +30,10 @@ import org.apache.spark.sql.SparkSession object MultilayerPerceptronClassifierExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("MultilayerPerceptronClassifierExample").getOrCreate() + val spark = SparkSession + .builder + .appName("MultilayerPerceptronClassifierExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala index 1b71a398905e0..e0b52e7a367fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object NGramExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("NGramExample").getOrCreate() + val spark = SparkSession + .builder + .appName("NGramExample") + .getOrCreate() // $example on$ val wordDataFrame = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala index 8d54555cd3c65..90cdebfcb08de 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object NaiveBayesExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("NaiveBayesExample").getOrCreate() + val spark = SparkSession + .builder + .appName("NaiveBayesExample") + .getOrCreate() // $example on$ // Load the data stored in LIBSVM format as a DataFrame. val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala index 4622d69ef98cc..75ba33a7e7fc1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object NormalizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("NormalizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("NormalizerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala index 338436100c5ce..4aa649b1332c6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object OneHotEncoderExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("OneHotEncoderExample").getOrCreate() + val spark = SparkSession + .builder + .appName("OneHotEncoderExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index e2351c682d753..fc73ae07ff6c6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -109,7 +109,10 @@ object OneVsRestExample { } private def run(params: Params) { - val spark = SparkSession.builder.appName(s"OneVsRestExample with $params").getOrCreate() + val spark = SparkSession + .builder + .appName(s"OneVsRestExample with $params") + .getOrCreate() // $example on$ val inputData = spark.read.format("libsvm").load(params.input) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala index 14394d562468b..7927323b42852 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object PCAExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("PCAExample").getOrCreate() + val spark = SparkSession + .builder + .appName("PCAExample") + .getOrCreate() // $example on$ val data = Array( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala index 61b34aebd9c32..e5e916ac166fb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -30,7 +30,10 @@ import org.apache.spark.sql.SparkSession object PipelineExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("PipelineExample").getOrCreate() + val spark = SparkSession + .builder + .appName("PipelineExample") + .getOrCreate() // $example on$ // Prepare training documents from a list of (id, text, label) tuples. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala index 4d8c672a55c34..94b17a3cd7064 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object PolynomialExpansionExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("PolynomialExpansionExample").getOrCreate() + val spark = SparkSession + .builder + .appName("PolynomialExpansionExample") + .getOrCreate() // $example on$ val data = Array( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala index 0839c609f1d95..1a16515594161 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -24,7 +24,10 @@ import org.apache.spark.sql.SparkSession object QuantileDiscretizerExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("QuantileDiscretizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("QuantileDiscretizerExample") + .getOrCreate() import spark.implicits._ // $example on$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala index 699b621db90af..9ea4920146448 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object RFormulaExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RFormulaExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RFormulaExample") + .getOrCreate() // $example on$ val dataset = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala index 4192a9c737045..ae0bd945d8fed 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object RandomForestClassifierExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RandomForestClassifierExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RandomForestClassifierExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala index 5632f0419a80c..96dc2f05be974 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala @@ -28,7 +28,10 @@ import org.apache.spark.sql.SparkSession object RandomForestRegressorExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RandomForestRegressorExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RandomForestRegressorExample") + .getOrCreate() // $example on$ // Load and parse the data file, converting it to a DataFrame. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala index f03b29ba327c6..bb4587b82cb37 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object SQLTransformerExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("SQLTransformerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("SQLTransformerExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index dff771950715a..3547dd95bdced 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -34,7 +34,10 @@ import org.apache.spark.sql.{Row, SparkSession} object SimpleParamsExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("SimpleParamsExample").getOrCreate() + val spark = SparkSession + .builder + .appName("SimpleParamsExample") + .getOrCreate() import spark.implicits._ // Prepare training data. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 05199007f015a..c78ff2378bc1f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -42,7 +42,10 @@ case class Document(id: Long, text: String) object SimpleTextClassificationPipeline { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("SimpleTextClassificationPipeline").getOrCreate() + val spark = SparkSession + .builder + .appName("SimpleTextClassificationPipeline") + .getOrCreate() import spark.implicits._ // Prepare training documents, which are labeled. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala index 55f777c6e228a..4d668e8ab9670 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object StandardScalerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("StandardScalerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("StandardScalerExample") + .getOrCreate() // $example on$ val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala index 85e79c8cb38ae..fb1a43e962cd5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object StopWordsRemoverExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("StopWordsRemoverExample").getOrCreate() + val spark = SparkSession + .builder + .appName("StopWordsRemoverExample") + .getOrCreate() // $example on$ val remover = new StopWordsRemover() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala index e01a768da988d..63f273e87a209 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object StringIndexerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("StringIndexerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("StringIndexerExample") + .getOrCreate() // $example on$ val df = spark.createDataFrame( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index 910ef62a2670a..33b5daec59783 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object TfIdfExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("TfIdfExample").getOrCreate() + val spark = SparkSession + .builder + .appName("TfIdfExample") + .getOrCreate() // $example on$ val sentenceData = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala index 4f0c47b3c84c7..1c70dc700b91c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object TokenizerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("TokenizerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("TokenizerExample") + .getOrCreate() // $example on$ val sentenceDataFrame = spark.createDataFrame(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala index 56b7263b192e0..8e382ccc4844a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object VectorAssemblerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("VectorAssemblerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("VectorAssemblerExample") + .getOrCreate() // $example on$ val dataset = spark.createDataFrame( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala index 214ad91634ee3..afa761aee0b98 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object VectorIndexerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("VectorIndexerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("VectorIndexerExample") + .getOrCreate() // $example on$ val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala index 716bf023a8087..b1a3997f48c91 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -31,7 +31,10 @@ import org.apache.spark.sql.SparkSession object VectorSlicerExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("VectorSlicerExample").getOrCreate() + val spark = SparkSession + .builder + .appName("VectorSlicerExample") + .getOrCreate() // $example on$ val data = Arrays.asList(Row(Vectors.dense(-2.0, 2.3, 0.0))) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala index 292b6d9f77763..9ac5623607296 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala @@ -25,7 +25,10 @@ import org.apache.spark.sql.SparkSession object Word2VecExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("Word2Vec example").getOrCreate() + val spark = SparkSession + .builder + .appName("Word2Vec example") + .getOrCreate() // $example on$ // Input data: Each row is a bag of words from a sentence or document. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index c2bf1548b5685..7651aade493a0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -189,7 +189,9 @@ object LDAExample { vocabSize: Int, stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { - val spark = SparkSession.builder.getOrCreate() + val spark = SparkSession + .builder + .getOrCreate() import spark.implicits._ // Get dataset of document texts diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala index cd4f0bb0de601..781a934df6637 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.SparkSession object RankingMetricsExample { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("RankingMetricsExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RankingMetricsExample") + .getOrCreate() import spark.implicits._ // $example on$ // Read in the ratings data diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala index 22c47a694d346..abeaaa00b5a4f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -27,7 +27,10 @@ import org.apache.spark.sql.SparkSession object RegressionMetricsExample { def main(args: Array[String]): Unit = { - val spark = SparkSession.builder.appName("RegressionMetricsExample").getOrCreate() + val spark = SparkSession + .builder + .appName("RegressionMetricsExample") + .getOrCreate() // $example on$ // Load the data val data = spark diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index b4118b16e2918..94c378ae4b911 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -26,7 +26,10 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { - val spark = SparkSession.builder.appName("RDDRelation").getOrCreate() + val spark = SparkSession + .builder + .appName("RDDRelation") + .getOrCreate() // Importing the SparkSession gives access to all the SQL functions and implicit conversions. import spark.implicits._ diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 2f0fe704f709b..9aba4a05a89ad 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -93,7 +93,10 @@ object SparkSessionSingleton { def getInstance(sparkConf: SparkConf): SparkSession = { if (instance == null) { - instance = SparkSession.builder.config(sparkConf).getOrCreate() + instance = SparkSession + .builder + .config(sparkConf) + .getOrCreate() } instance } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java index 9475c853a03ff..9def4559d214e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java @@ -34,8 +34,9 @@ public class VectorizedPlainValuesReader extends ValuesReader implements Vectori private int offset; private int bitOffset; // Only used for booleans. private ByteBuffer byteBuffer; // used to wrap the byte array buffer - - private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); public VectorizedPlainValuesReader() { } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index b8dd16227ec17..70b4a683311ca 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -19,8 +19,6 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; -import org.apache.commons.lang.NotImplementedException; - import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; @@ -29,9 +27,10 @@ * Column data backed using offheap memory. */ public final class OffHeapColumnVector extends ColumnVector { - - private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); - + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these two allocations need to maintain binary compatible. We can // directly pass this buffer to external components. private long nulls; @@ -230,7 +229,8 @@ public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4, srcOffset += 4) { - Platform.putInt(null, offset, java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); + Platform.putInt(null, offset, + java.lang.Integer.reverseBytes(Platform.getInt(src, srcOffset))); } } } @@ -276,7 +276,8 @@ public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8, srcOffset += 8) { - Platform.putLong(null, offset, java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); + Platform.putLong(null, offset, + java.lang.Long.reverseBytes(Platform.getLong(src, srcOffset))); } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index b1ffe4c21049b..7fb7617050f2d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -29,9 +29,10 @@ * and a java array for the values. */ public final class OnHeapColumnVector extends ColumnVector { - - private final static boolean bigEndianPlatform = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); - + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + // The data stored in these arrays need to maintain binary compatible. We can // directly pass this buffer to external components. diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java index 0457b3781ebf0..de066dd406c7a 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Date; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; @@ -35,7 +34,6 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.ql.hooks.HookUtils; import org.apache.hive.service.CompositeService; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.cli.SessionHandle; From 19a14e8417cf4ced0dd0fce863d3f4a0bcf414aa Mon Sep 17 00:00:00 2001 From: depend Date: Thu, 5 May 2016 14:39:35 -0700 Subject: [PATCH 0093/1470] [SPARK-15158][CORE] downgrade shouldRollover message to debug level ## What changes were proposed in this pull request? set log level to debug when check shouldRollover ## How was this patch tested? It's tested manually. Author: depend Closes #12931 from depend/master. (cherry picked from commit 5c47db06570e65d3f5544d6f26bbdf893e275b94) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/util/logging/RollingPolicy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index 6e80db2f51f9c..5c4238c0381a1 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -113,7 +113,7 @@ private[spark] class SizeBasedRollingPolicy( /** Should rollover if the next set of bytes is going to exceed the size limit */ def shouldRollover(bytesToBeWritten: Long): Boolean = { - logInfo(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") + logDebug(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") bytesToBeWritten + bytesWrittenSinceRollover > rolloverSizeBytes } From 80a4bfa4d1c86398b90b26c34d8dcbc2355f5a6a Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 5 May 2016 14:40:37 -0700 Subject: [PATCH 0094/1470] [SPARK-9926] Parallelize partition logic in UnionRDD. This patch has the new logic from #8512 that uses a parallel collection to compute partitions in UnionRDD. The rest of #8512 added an alternative code path for calculating splits in S3, but that isn't necessary to get the same speedup. The underlying problem wasn't that bulk listing wasn't used, it was that an extra FileStatus was retrieved for each file. The fix was just committed as [HADOOP-12810](https://issues.apache.org/jira/browse/HADOOP-12810). (I think the original commit also used a single prefix to enumerate all paths, but that isn't always helpful and it was removed in later versions so there is no need for SparkS3Utils.) I tested this using the same table that piapiaozhexiu was using. Calculating splits for a 10-day period took 25 seconds with this change and HADOOP-12810, which is on par with the results from #8512. Author: Ryan Blue Author: Cheolsoo Park Closes #11242 from rdblue/SPARK-9926-parallelize-union-rdd. (cherry picked from commit 08db491265a3b50e31993ac6aa07c3f0dd08cdbb) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/rdd/UnionRDD.scala | 18 +++++++++++++++++- .../scala/org/apache/spark/rdd/RDDSuite.scala | 17 +++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 66cf4369da2ef..8171dcc046379 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -20,6 +20,8 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.collection.parallel.ForkJoinTaskSupport +import scala.concurrent.forkjoin.ForkJoinPool import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} @@ -62,8 +64,22 @@ class UnionRDD[T: ClassTag]( var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies + // visible for testing + private[spark] val isPartitionListingParallel: Boolean = + rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) + + @transient private lazy val partitionEvalTaskSupport = + new ForkJoinTaskSupport(new ForkJoinPool(8)) + override def getPartitions: Array[Partition] = { - val array = new Array[Partition](rdds.map(_.partitions.length).sum) + val parRDDs = if (isPartitionListingParallel) { + val parArray = rdds.par + parArray.tasksupport = partitionEvalTaskSupport + parArray + } else { + rdds + } + val array = new Array[Partition](parRDDs.map(_.partitions.length).seq.sum) var pos = 0 for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) 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 a663dab772bf9..979fb426c9482 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -116,6 +116,23 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) } + test("SparkContext.union parallel partition listing") { + val nums1 = sc.makeRDD(Array(1, 2, 3, 4), 2) + val nums2 = sc.makeRDD(Array(5, 6, 7, 8), 2) + val serialUnion = sc.union(nums1, nums2) + val expected = serialUnion.collect().toList + + assert(serialUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === false) + + sc.conf.set("spark.rdd.parallelListingThreshold", "1") + val parallelUnion = sc.union(nums1, nums2) + val actual = parallelUnion.collect().toList + sc.conf.remove("spark.rdd.parallelListingThreshold") + + assert(parallelUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === true) + assert(expected === actual) + } + test("SparkContext.union creates UnionRDD if at least one RDD has no partitioner") { val rddWithPartitioner = sc.parallelize(Seq(1 -> true)).partitionBy(new HashPartitioner(1)) val rddWithNoPartitioner = sc.parallelize(Seq(2 -> true)) From 1064a3303e72d92db02cb94eb2bb81245ac68fc6 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 5 May 2016 14:44:45 -0700 Subject: [PATCH 0095/1470] [SPARK-14893][SQL] Re-enable HiveSparkSubmitSuite SPARK-8489 test after HiveContext is removed ## What changes were proposed in this pull request? Enable the test that was disabled when HiveContext was removed. ## How was this patch tested? Made sure the enabled test passes with the new jar. Author: Dilip Biswal Closes #12924 from dilipbiswal/spark-14893. (cherry picked from commit 02c07e8999dca545849cb3aa758a624dc51cd1e9) Signed-off-by: Andrew Or --- .../regression-test-SPARK-8489/Main.scala | 12 +++++++----- .../regression-test-SPARK-8489/test-2.10.jar | Bin 6873 -> 6865 bytes .../regression-test-SPARK-8489/test-2.11.jar | Bin 7039 -> 7030 bytes .../spark/sql/hive/HiveSparkSubmitSuite.scala | 3 +-- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala index 10a017df831e0..4fbbbacb76081 100644 --- a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala +++ b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala @@ -15,7 +15,6 @@ * limitations under the License. */ -import org.apache.spark.SparkContext import org.apache.spark.sql.SparkSession /** @@ -33,15 +32,18 @@ object Main { def main(args: Array[String]) { // scalastyle:off println println("Running regression test for SPARK-8489.") - val sc = new SparkContext("local", "testing") - val sparkSession = SparkSession.withHiveSupport(sc) + val spark = SparkSession.builder + .master("local") + .appName("testing") + .enableHiveSupport() + .getOrCreate() // This line should not throw scala.reflect.internal.MissingRequirementError. // See SPARK-8470 for more detail. - val df = sparkSession.createDataFrame(Seq(MyCoolClass("1", "2", "3"))) + val df = spark.createDataFrame(Seq(MyCoolClass("1", "2", "3"))) df.collect() println("Regression test for SPARK-8489 success!") // scalastyle:on println - sc.stop() + spark.stop() } } diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.10.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.10.jar index 26d410f33029bb2a9d2b65d9f6985aa159e924a8..3f28d37b93150ebdeec4c6d803351f8c9e1f6cf2 100644 GIT binary patch delta 2816 zcmZ9Oc{~&TAICRZW-R8&Mkp3(bBt7mOwQcrTv2WjQF4_=SYl{aArh*MxsoHa1T!6Rf}AFbaH-XVXaN8K?ce9$c!v^O-GR~rz-b`jihnTpI}iZyXZsIC z0kR(Z#~1mfTq4XqCuyPrAIfA4;ZQ3V}ve^&b%Faw^Xy$@t&{;y{vkNKX*yV2$iD= zMps=WiIk9>U%Ik4q`0^An4z*zk#c9;-M9(B!fWH5^UMkGd%hZ%BwW^+M;O_jVQDiF zUk%C5&IMDgPS|JjDBZ|vbPNcuEsykgid1hZ>sB|I5gWU+_)t=B`a~D{ux&=AYoT=X zK{V8bv!2c2;M!xIQM?{ibiSJETOQWEVXx;)Qx0X-j(==-R1N<64U%a&wYSLQMMt z>DUQI8iJgN^DKJ@nTgT~81mB0HxUs@rIU8a#`iLjQPJulGSFQE+C zSk?e3i1-3div(0$cbUHaQx{JpblM5b@!6h~bv-MYls2W9)-iLniEF}1u+1Zo>lF3n znQ^A*l@N=|mdXX4KMyThec!^QHLmIprg5(|kJ?%!VpMs+1x8jQYy7WOV;u*yw!--X zhTEA+#0Bp;LEcf|tLVl}fY|ga$%rVnx2R?7wmAAp(R+c5jn0woLM;m>ga-4-PV1&- zXiFD+qN@>jT^9weNa3Kg-HIC+u=AdsjxBB*Pn?QAIYlWDDDsrLz5OUP*KIAgjCjQL zBDP4;*e86h|3YsB&hog3uo$10+_$sWt-dld7&B>h2-ic!@GBqPkRW@0F0LIr6;USd z00&!Ws}Zqh@Hf=~|8{PY`(=5XrNK-rO`cGr-ai1;`^ zx6O38T-{Y|;S^BcLm#N|^z;L=hZiEVt(z2HDnV1KCwJd7xDO({h1pFXz&}C>V7xA) zjrU}ES8~%V^ag{fB}awTL_>SG&fq7E8>}PayCAg&CGG4gPU|-0@+yEkG`0q+Ei>qpOpWd%qc5qMCefZ>u&~0+nFDrqWvqR+UgjjvRV`AG2p>3 zReITH#q%LRc@`l=Og=CFdL8O@ZyU{Gtia~YkXpdQbQvIfQ>ZV*sicw}Rw&B9t z8{cj7dP)Yj;b{veg)4E!$V*@NTTCjFTOol~grm8b`qDmzLK~g^6rK1Sw4<;S?dwW~ z4Ele|)PX*u>_#cs>`bcMl;J-npPwBcRCZj}s4XxR}9y+)T;t`PCy-AGfZua+1Z-U%nf14(&%Va^!P?$`qyUT4;|SyW8^P@L1y8)%sPC zu21VN)N@y%)b0Cu*G}+Gh9@qoR49cwva&EdvL2{$Os0bCL4I2t*&MQJ>t-izP_X3r zv}}k1=aM}gpl%!z31v)bT#B11CeM%uYo*4rCD51ZM-{S-TNPPc=Dh1I52V48chE;JRjEME~GD6n|?z#X49lX)W>j4gj zDk~RH5Bo~)9=inOf1>YlTWLiPF}vvZs!W^H*P|mt<%Y*!KRkj6Kle68Yt_DBn$sE{ zI(x|&%cMvn6^PFRt8%>zMhfO0ht*D@VqUwS)bV+jBP83~DU>`P{*umA))|YRC7>;P zxA^*Be|9%OOOK#c{OEC1F&R*TH2lLaio@NcY}yVz8c)3JNJI?Z(`<3G(Rk4>1;;LM zrDk39{g^{#(aS%LLu1|`>?xagU5;sh+8k26)xX7ajw>cz6I0f&I|>yTr0kGVbyE3<$|eF*I!s+b+Ypmb>>VuyjtYPiHR!G9qw3|lk>9G ztX%MqNh%F1FLt0p=To)3`=bAZH!s6C#z`)$q=FfhjVff_F1>CxPW72@wli3<)Rn2a zMKBrd5%l!felK-At%z`K6RIKAx=zEiAOuUB`>oA!h9=RCN3#8ZZL%L9jAzsY@)dr4 z@NxA?(7Up@XMu-Ip^nm|M8aAUIm3gP&}96uL=J55TknyX z&90$4iFLOFvG|6s?~h5elSR&ZHNZ=v%6c9qJxJ;Npuct+32!2ZVg)&aI7!Rtq`l~0 z0+dS%VTpOWD)l>ah#Vv)IRv_`Jvy)XbS=>UHsBUxrY-cX%og9PhB8QbIXg&b2@#7YeVvba*y`u zuMG@gucXYc#Nw99(>>~ByTtP*tfx_rO@vwNm%4Dp@qO{ew|uqm)4$mFb!OOyc#32P z0HitgB?M#>0Q}!tQbZAF!N0ww{=S=}I3lor9{aA5@(iJHXg~A!#G;HMP=E6K!lm#E v9|k5f|RgQq6j|NdGq$Yy)$?2+&kx-xpQXz|9m4lQ#xRKYjzG%00;yE za1z4O!TRjShm{3)&KG#MJ54D)R~nwvDxPQx007P(?Z3l!@nPvhT5PAd8aaR;fpF|~ z$K~%p0N@(iKZ2-0b}3@M%f71!3f9I>LO-|=a#L2R!hy}GC9#FKX?&4U_@GcMq}dww z`pO&nEZ8b|Zu41e$9h*g!_#Vrr#F1MYv`baKnM_7&bj+0Xz*?SM*m)HM92QXOg{1O z;AITjEo*ETfdOU;bdx zaVUX4;o^&mPt|WR+^)lR{&nuISt6)u_p>JadyAgtvc*L6nhiw(Q|H$meF@ULw|!!& zA!2_J;~wW`QcAq2wp8}~$EaJYwppzVT~lQOj*suW!mxz0DPoto(jyEcRt5?+bAjdU{2he3BS%ZsOsac&4dJDDO$aqy{R{ zNUKlC{i$=jI)`ty)N-r2(w(Z-0;Qr<0crH4{L`Xj>>hR+)=V$Mr47_LX55XcU)@nO zZ+Gn#nCV@2;t}C{Y3&tb)*4TUzBe2%LrQ2TdoXPG%Fz>Jo$qD!@8HvBTTaSgC*a*4 zB+O6hQzF}k1O4cw+ntMA%jYm%r$|ghZ~tyzx?!r1Eoi%23Fe~s#cfdQ-sq^F*=DuW zZd>k*aQE1&m8HsgJmK{{&-!pERwgYM*ejKtT4ka&tsymGIo35ZlZ6emU}i%7rOp=n z^DAU;Rci!2y(8YA>8oK|;p8&=rtV#(knN)I6QZk{50c+8k7Dc-oA{GQ79Bg9u`uBRQCQ&DFk+U1^F|z4h?tx@S=H&mn4hFGi*UuzL6Tz)#b}+%9hDsHbPK`zKX81U4^9 z60=^&KGsbs>EU+HP*FS6$u&}|cwNzJcpewVn#-?14V6L=jFrUbK-dJ%GB#*@$VUe} zQ~4PSy$y}XXAckaZcD1pQd`{otPo{DQm*c}e_=z%FrvdJ6<3#Eu=KEBA>94>6(red z1hax$c)XRIQ60eP;@tRwX<+TlVgKEXh`eG9`(+U!{mZ=H&REU&<`7)OTe-&*Tw6*V ztlEHpE<6SUq%pa;TOBAUgqJ?@+4;gGh`jxhJST1Q@50=H_$o|Hl)w9uHY;>O zp*ydIE-?7-Jd&y`dB zm@379#zkqI2C7PiD#JS`2JW5C=5u!qnV6=~zmCMv*f{rJ-67F}~L*P(4KzgPN#c8m?V^32r~6JflF;oPS#<~O z$Gy-tA1$UFO{i-}?fAb@Mh)jn2EJlCvLBemt*l)li$S;&ImH{P7#^N^)H@~`jo0Ud@e zd4VSUavEY`rI&AgGmS>P9`^A3?QVE;sWt6M)wpb&t^~suRJvZ1eCEw5{IIRjgN4kY zVUZM_IBZ}IW%`a9*P|VW8g*7y_{DR+ZsgySYbO!ya>6l(P*6i?E^e85wfm2;*NX2i$zI}rwmKjd9o=DsEsNi*q|ICef((Mhy3&3nCk zSvL=+FS)O{@I2XLn=I^sWlHs5-q`Iy5520!3%PaVM0wIDKfNA1iO=J0yS$AJN#r&~ zxr}`yL=h=@wPiJ4$TGXLWosL8oKdc>4|5lO(lR4k+KE%TKXOz?AJHqkE(}0Coy~xU zcR&=l$vXFd>f77%58ERlH8p8(|D;9FHYGiZ6tDwJE9}ZA+d_K1Gh7xH*H?|od*XDr zGo?icYl*>fK!$9w@YKc_+w$(nG$Hj6i^Vo1Fo>qz`KNX74s&VvatDw%6K)8d#asm^eH&=E*l9MdB; z8?G*Jd^s|#|M(Sk46c0Q`00OjD{+LOf3H012?^w1HI8LVwU-cM3+1Dd_(Z5TB=~ub XbN~7bHBABvtfH1nAh-nJ$Jze|s16>C diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.11.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.11.jar index f34784752f69f71ce56736d3493cfd55bdc32b4a..5e093697e219a249e6f6dedb1ab9f567c58a2fa9 100644 GIT binary patch delta 3351 zcmZu!c{CJk7av26WsognpD?l;#y$~~-AJ+&NyZY{*NkY&GK6fyAj@l?Y>g#SQd zQJ)=#`m{F}A+4E^q;vrYrb&wd1pol9p6-932|n})AT4_hiN2l+!Z#FisgDW(5C#4l z^_H3smuh)v#csuMA}}U`lHl1disn|#P0X*e(5eVyh zLNvrXN^Q?>eyudCdNR``zvTLY>r?6#N+x0ehkrb*{K-%4kw`PSo;)JXx@cI8?%H7n znd{l~f`$SP2U-1T0?hO7M1XP1Vn5b>f^rH9_Jt^+G1Ry#0lPZLneq4?+?^iSd=WyH*RAvxq}|j_)HmYy9tr zVKw0E{5Ji~i5#;C)}}y{IQP1#jt(5_R|eo3u&dCCi8SxycN^wATD}$pMds#@#xOtm znX*Hh<*CP$!!cq;FX`XNNJ%{jH+IfLCI_nZAbpjIi4NVpEx@u_r~RXdRWOESuRnC^ z>T3@YWe~q|f02!qpQX3MQA%5M)Nk`iN?k-{No%_$9+|*w#K*hi?jFNDtVm;B9f6SOZ0+|RAHT?NLU55 zojWbQ%y`gR@MnOTqEKai78snsfZz(`Q!Nd|nDlOU$Hl>;tm0A}<=KGft;oou@+@Wy zXqEa&mXicidGvReE59dp>P#w}?ZK8T~EPd}|oaHi4-c=w`^tz>Sbg z=R8ql-FlH>es45T?IJ%9l3(@nfwI*!y2av2Y`=U{OjR%S8_G0U%EK@F;N0cBS)LrO zkykFH{!OWsRu=s|?i`;XdoOks%|2`?O%w7Med|GyUP#{zK7OTMywVsh*%?g6Vx8oJ-#hB3>Sa(+xzicBu3n3D&NZOmLXb&4QW_}J%Iu)MI6)??gu-G=bC z8$+t+rE+w^cFD&r3TfDutmcKD+wi;KsqFwob+N9AJy01=iqC5Q2+{(#kQ6Y}Tvm0o zuGUP05KVk^u8pXQi6=b2&$e7EwRF&PRsK4dJ~IEyw-)0)>jEXh962{!>&xa^`$@6N zw3T8QXwz<*u9xU%j{C}Qj)v|;Y1%WkB!@k|`7~GpxPiuCgPa81#QM|q{8W@0i$xOJe;j*Y8-gwEgR;37_laTFk%EU0a{;Vg zhg&7&*IUdpUaj1d^W_Z3Aw>-dEFGQ0{a-OGz);MH;Iyq?`S zpmj;}ms#yWq=RI;n|q_3_r7GjWp?h5m4oMRRo8K-%BiYDT;Hyps$KH}h2{OV)Fv;9 z_1^+irbcuiyt#H>;5h(bf$?80I;l^Y`He+H*lmt{4vC>}*(nkP4k|cKX_F{-os(Fw z@`&Q3R45)|#kCVgd$Ah}wvTN8tm6z!~gtEzkxn*p>CS!-zudZg3xWY@g5(7&jP4Qi$p^j7V)4Na~m(w&lHkdEuq%^2J0e8!;*C$@sx=uw{oM+4TXrIB8M*Xc8d&98Z&;ijFs zbDZ{+Vb5XPu|ZUAFX}^9vxl+7wHpzu=vss^`_D)co7mh%zr%_WxwMIi_p7YVa(Xx4 z(8`5*B2rD=szOm4>bht!IRkGFhdUB^7td#_nlkHHaAU#QNmi|!T+sKkWo zr!QTmk4P6_btbRmHY48X>vUBevE|8@;n$$U$wM45hs$@pIJ@8v!ks5lYc6A=aNM)9T#duJT5AqnyXBb@ENkoIHFasi19a>cI^@N|8X*4Qk3D&3)aF{W!I`DaX-8%Ri zUj}10yG$;-MidY2UuFD!>$6%eV&P7Xqn1F|@=63P*a6o8C$s1f_iKz@JtDabkG?@? zZeV*9ECyPat>zk!e3F*sC^9m<2GbVNhSQ3JDilLZrdMo;>@+;56)SUi%rV=**0&Xg zOH?qnlMihGvFQ+G(*N#5^nkdXr#3uxFp>yAmr{Z3M;deFbW#l!;ml zncLP5FrjBUVBxY(UQVyHn47`_x2u>)A^3IgIr&TY;pZ=;7acdP1wYH9YRn{DqnM0l zGW$78W{udA7Cd?G-@sjY%YQ}iXUV%{v%2WkvL<~|LfRV>a;Q3I8OB%M!vDfY)B2Z% z(PQs-lNaOeHjk1aY9xcV?VpGhRU%FYK2r(TO=W?ebk9PU$Elxd-)p=s?wS?MM}$Al zeW5Jj(9sWbWK#d?NX@=t^RU51lp3Wc?=8Dge%vTYpp^C%JLVG?ed`fA$9{Vfb znyx;AL^Cnhx%cpn(ktpo6>-nIRGQPI5<__TF!!io( z{GYrFWvi%2{J8x*(Dqe~<`Q3yglG0hrjVLGUzAEkb5TjnN;(&d<)XMc%zx~>l|e|I z&Q;-`69w4LwW8WxzR<{#s|_*gIMmf_D^E}!`0gj>0Yg8Wrv4pmURXs5^gU>Oz$WQ# zR1>JNYUJ9%HTd2+)B?9`matiLHdL2Q%)lIe<3oW11(Wpli>iN8FjK--1n^5S)SmQKjKvo+dJfEgco=i2i)l!Ex^zKY^(6e0t)-NzS)};|5Ggzhh{FZC{~p)_mbAA5=4J zjMpIaFmv-dlGMKIQUd_?bY~Ss1>^wyZwer^KvY1dV}Sk{^CwXAs0p1Ze-ixDO8i5b z0fb2&?ptS|KfjpMU*-?_cv|Ubdsj~p83%WJKfiyu)c>;?@ceNN00;slP4W@2JWT%$ z>8ueDGQiaR2}S delta 3411 zcmZ8k2{aV!*B^s1SqfQZ#*$suP_jf~EJK#;`@UrCyA-BjtjUszk#&TOUW70qX=Eom z$!^FlYE+i$<9)yH`}cqEbDrm(=bU?=d+#~-`JLbWcJ=pFrdv0uY1jZ@Fc_eW_RnBa zrao1A2oP=)`py=Tf#ZdyXR}Jc0RVvJ$^MTuRg`~b99ZEj(Hs+F$5zZHK6W=dOCYls zEQ=~wka8{-m+X=`4(szlnEt5h=Zav`#C?U4IFyjQQy%SuNm-AlUCmy_jtuD_#8>v2 zbr6`WE0ps~2Cb|jmoG>>Thd?lS>#^zYJHhUCiS9IsTk!PUfP7^zQ`dQZxXz8*Lz8N z7zFJibYsTPnpUGJyCL40bYsElb{R#Vgo+gDor$(|7?Mq{5z>}1gj6*2Re5LoE=Euj zI~WH!TgSmrjERUqHF&x{tZwIH0ZT6Fa2TvN@E$x zE-aj`4vu@v?R*fg2jc*fTe-q!LRB0=OgqZXVk3Lw1#85F4*r=A*0dOuOJD~|M8Y_@ zbM_|8Krn#qC|zds9xGT@kgwliQauhCl#!g}dLXRfsLeDD|4W#qOWAo*4efnL6SqM% zvWqJU&A<%5G2z$$E?F7BidmG{7f!MBy<}Oj#T#_nls$W!8;ne9^4%fCzBZTanqhhp zY8s{#E5G@HC7SPmm+iYk)$=}7XDRaMEj9ND{Mfzql7j6GtO)10Qdv(33Z4}ab+NXk z(A}xu!xR7R5-;oZ$CKQq!UlguYwh%1;JlkK6&g5&0NV14qbK#rTjEK+tx{i92i2yY z1s2?lR%=(Re#g+Vd|{b!)KY(528F@qQGGdk&bK_UG}VE;rbcJ0m}C>F_rZTF`#;y zde7%k7=9I|&mG?#ZZ&;Aa``?h9roF%bQR;Ow7t0c%UJdGSJr(T2GOU~-;#6C>c7+2 zpWie8`*@G@zTvU>tx12Jg;^}KRS0ShC$j`?*Xv4SD$Eu$f>++JqWL(B&5)&e`OEdS zDC^PFW4#~Av%t5igcM}X+)rSA%|s?}|K71PL|H@GYlx2-^6W+1mk&AbDh(bLxZ14y zl?G+|t3}Ub2nJz2Ip`(zXRc22XsdE0ya2PHv(S0&#*I@MNS&rCr0qaK$ZOOm<7uQJ z@4P~Q+jJFAb66u?4Hi*1)7ckr1WZ!WtkjEQfjN!q&G0o6FFO8L5AXDQiW}o3YP@Y!Ksn~l{n>AP zyl#_F&FZ$D-_RGpF>2*+OGRyuF0VYU!W;2L_Q=*!2Za=pA(_B?F6=ZCC7~*-&~_+P zO>6rv?!Ef;85W_jx9h8fg-DN)1V`Cmb{}5sM3JjLqRYGtxeZC(WE_V}?&w{;A{5a} zFOkY^E)ghtJK_@&m2x?jGv%e5oME}Cg?;y6FM-)`$+_0@y&OA;(2K9v9PNIx*UmVY zRLm%EL{2dr&?(Q!Dk#*CKWb=SGcut_zF!9~$Oc*W6vsQGX%#5v8)OA|(49dNal#*A z=&=lvr!|cpzZXO>pTmAffv2l%W$pua{O9EWGbVZq@RABj`bwLowv+j*j|*-VXJN=g z?XvM?L6gvqqB-VN4mC5wTH*%)7pA$^4u4-gwrgqkQbD=S0M!jtldVj|BL$%8XJ%v4n!!tZ-WL=K?X=dMG1%Yj{oz_0jRuwOPt(mbg zv`r>sLoM&3W_}=5o+@ryqsPaEw^P61@%_b;S=M;*Al!|IxvW;dP2#uM3Dp|xOqv_0_&e@jdTqjU4ab2VR({3!VMr*qwQ zR=1c@%$vgt&+T(bkGgABV7G^^uvwqCR*3WDm3=e2(HL&@v;hs7JXIhKglqF@P+>J;<3hE+fwvWjtd72fR%35st7B%-9+wFP9q|1h-fMRn|s z{KTMuLhMcWC^4_gG(u&P-IH;7uU9N?L%taG@kv)MeI!%s?R(__sQWBumNeY{2 z+`w*G6E{K0-P>}j7Jzx>cHZ`VLw;aeM2d?OxadxtD@3kdRXa2Z{n0aj@kT@Kp0MQ` z6_>(F&Jwc)Z^_Qa<)7Z%zx(%qzbtkezl{p-KKMd$JcV#*^c$(@6Zke`{1=303|;}o zL+WU$h*VHYgZJSe6#yW7g7yDRJmfeeQMOGlB@wTZXV@N4fY=8LJ_HvkTH6W^-9sXl zT3Wh!AnYbaZCS!eW`EwZH%SK;Cxs{Cq1luHza=DLh@4 zQR=DKV?8UNt75-}UUHdGe}y06o=VLN>l*!1Hz~{gN2RbsAMxhrTM}mshQnEDQGI6n z`OEc^p*fG%OL9xw49the;O?em-T1}d%UKJqYs5&im_mNDZ0C`0&gnr19p|P>W#G0^ zRL;HQdE&^5DjOuoU#RKrg3**Mv@$(*x9DM{WV=`)UV`;CV=}%}+Uv41ByvKho-|Z~ z?fXd8ZnEqVqIGFkIH@~WfvKJ!#W#Dlm@~@E(VAeTxn%BCQrjb8_iXalc7?gTQcgwF z12>rD?lF_$N!7!Q2s_h)ev%-Y&xq{U1+9dqznl#pezBTPM$RaIh z@E4`SudtA^HJC_;_GPWx>4VtX5n(j`aD1N@r?bh^Da--VQbf5e}rqFATOYXUkOV{v`s@6iD9VEB<4za(K!in%c zxsjRl-f>HN$Qv$j?jESCqcpToNwAn zd@d9{El|wzb7vU1@@@EIQ2d-rWMpytH5B|^o~0iczwg?1c0?l>rFO$XgG*pwI^78p zzFvuVVc>+B_dFH_ib8(IyuF;lKFBQ9X~ZI$!>2nYW7_^yu-PS}sY@xO;4WL*9tJr1 zEW-8j${7yfetvJue8GtL5(dAKm5%V318tYc>?EWkpUcS3lt2WSg10)Rb4bb5PtRL& z7gWW6g~7X>hR?D)kyw$uwzeYM0;;@wwBe)uO~N1-`Ih9Uj^jXhK3W2{7Cd~n#-niY4Mo9MA?J)74#(Ck%;>L`kNTTT&GkS7w5b6AXS#oXGJl2yD#p6+3(VeXN6OeidJ-z)8K Date: Thu, 5 May 2016 16:34:27 -0700 Subject: [PATCH 0096/1470] [SPARK-15152][DOC][MINOR] Scaladoc and Code style Improvements ## What changes were proposed in this pull request? Minor doc and code style fixes ## How was this patch tested? local build Author: Jacek Laskowski Closes #12928 from jaceklaskowski/SPARK-15152. --- .../scala/org/apache/spark/Accumulator.scala | 9 ++-- .../org/apache/spark/scheduler/Pool.scala | 9 ++-- .../spark/scheduler/SchedulingAlgorithm.scala | 13 ++--- .../spark/util/ShutdownHookManager.scala | 4 +- .../apache/spark/ml/feature/Binarizer.scala | 6 +-- .../org/apache/spark/mllib/util/MLUtils.scala | 5 +- .../sql/catalyst/planning/QueryPlanner.scala | 9 ++-- .../spark/sql/execution/ExpandExec.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 12 +++-- .../sql/execution/WholeStageCodegenExec.scala | 6 +-- .../apache/spark/sql/execution/objects.scala | 2 +- .../streaming/FileStreamSource.scala | 2 +- .../streaming/IncrementalExecution.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 49 ++++++++++--------- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- 15 files changed, 66 insertions(+), 68 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index 23245043e2465..9d1f1d59dbce1 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -24,16 +24,17 @@ package org.apache.spark * They can be used to implement counters (as in MapReduce) or sums. Spark natively supports * accumulators of numeric value types, and programmers can add support for new types. * - * An accumulator is created from an initial value `v` by calling [[SparkContext#accumulator]]. - * Tasks running on the cluster can then add to it using the [[Accumulable#+=]] operator. + * An accumulator is created from an initial value `v` by calling + * [[SparkContext#accumulator SparkContext.accumulator]]. + * Tasks running on the cluster can then add to it using the [[Accumulable#+= +=]] operator. * However, they cannot read its value. Only the driver program can read the accumulator's value, - * using its value method. + * using its [[#value]] method. * * The interpreter session below shows an accumulator being used to add up the elements of an array: * * {{{ * scala> val accum = sc.accumulator(0) - * accum: spark.Accumulator[Int] = 0 + * accum: org.apache.spark.Accumulator[Int] = 0 * * scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) * ... diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index a79e71ec7c9bf..5987cfea2e9b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -26,16 +26,14 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** - * An Schedulable entity that represent collection of Pools or TaskSetManagers + * An Schedulable entity that represents collection of Pools or TaskSetManagers */ - private[spark] class Pool( val poolName: String, val schedulingMode: SchedulingMode, initMinShare: Int, initWeight: Int) - extends Schedulable - with Logging { + extends Schedulable with Logging { val schedulableQueue = new ConcurrentLinkedQueue[Schedulable] val schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable] @@ -56,7 +54,8 @@ private[spark] class Pool( case SchedulingMode.FIFO => new FIFOSchedulingAlgorithm() case _ => - throw new IllegalArgumentException(s"Unsupported spark.scheduler.mode: $schedulingMode") + val msg = "Unsupported scheduling mode: $schedulingMode. Use FAIR or FIFO instead." + throw new IllegalArgumentException(msg) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala index 864941d468af9..18ebbbe78a5b4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala @@ -36,11 +36,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { val stageId2 = s2.stageId res = math.signum(stageId1 - stageId2) } - if (res < 0) { - true - } else { - false - } + res < 0 } } @@ -52,12 +48,12 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0) + val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0) val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble - var compare: Int = 0 + var compare = 0 if (s1Needy && !s2Needy) { return true } else if (!s1Needy && s2Needy) { @@ -67,7 +63,6 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { } else { compare = taskToWeightRatio1.compareTo(taskToWeightRatio2) } - if (compare < 0) { true } else if (compare > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index bd26bfd848ff1..93ac67e5db0d7 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -170,9 +170,7 @@ private [util] class SparkShutdownHookManager { @volatile private var shuttingDown = false /** - * Install a hook to run at shutdown and run all registered hooks in order. Hadoop 1.x does not - * have `ShutdownHookManager`, so in that case we just use the JVM's `Runtime` object and hope for - * the best. + * Install a hook to run at shutdown and run all registered hooks in order. */ def install(): Unit = { val hookTask = new Runnable() { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 898ac2cc8941b..35bc46a5f3435 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -104,9 +104,9 @@ final class Binarizer(override val uid: String) case DoubleType => BinaryAttribute.defaultAttr.withName(outputColName).toStructField() case _: VectorUDT => - new StructField(outputColName, new VectorUDT, true) - case other => - throw new IllegalArgumentException(s"Data type $other is not supported.") + StructField(outputColName, new VectorUDT) + case _ => + throw new IllegalArgumentException(s"Data type $inputType is not supported.") } if (schema.fieldNames.contains(outputColName)) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 774170ff401e9..86ce9705a3125 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -104,7 +104,7 @@ object MLUtils { val (indices, values) = items.tail.filter(_.nonEmpty).map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. - val value = indexAndValue(1).toDouble + val value = indexAndValue(1).toDouble (index, value) }.unzip @@ -119,8 +119,7 @@ object MLUtils { previous = current i += 1 } - - (label, indices.toArray, values.toArray) + (label, indices, values) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 516b41cb138b4..8b1a34f79c42a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode /** - * Given a [[plans.logical.LogicalPlan LogicalPlan]], returns a list of `PhysicalPlan`s that can + * Given a [[LogicalPlan]], returns a list of `PhysicalPlan`s that can * be used for execution. If this strategy does not apply to the give logical operation then an * empty list should be returned. */ @@ -31,9 +31,10 @@ abstract class GenericStrategy[PhysicalPlan <: TreeNode[PhysicalPlan]] extends L } /** - * Abstract class for transforming [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans. - * Child classes are responsible for specifying a list of [[Strategy]] objects that each of which - * can return a list of possible physical plan options. If a given strategy is unable to plan all + * Abstract class for transforming [[LogicalPlan]]s into physical plans. + * Child classes are responsible for specifying a list of [[GenericStrategy]] objects that + * each of which can return a list of possible physical plan options. + * If a given strategy is unable to plan all * of the remaining operators in the tree, it can call [[planLater]], which returns a placeholder * object that will be filled in using other available strategies. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index c201822d4479a..1be41ffc072c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit import org.apache.spark.sql.execution.metric.SQLMetrics /** - * Apply the all of the GroupExpressions to every input row, hence we will get + * Apply all of the GroupExpressions to every input row, hence we will get * multiple output rows for a input row. * @param projections The group of expressions, all of the group expressions should * output the same schema specified bye the parameter `output` 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 0bbe970420707..b94b84d77a502 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 @@ -106,16 +106,20 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil) /** - * Returns the result of this query as an RDD[InternalRow] by delegating to doExecute after - * preparations. Concrete implementations of SparkPlan should override doExecute. + * Returns the result of this query as an RDD[InternalRow] by delegating to `doExecute` after + * preparations. + * + * Concrete implementations of SparkPlan should override `doExecute`. */ final def execute(): RDD[InternalRow] = executeQuery { doExecute() } /** - * Returns the result of this query as a broadcast variable by delegating to doBroadcast after - * preparations. Concrete implementations of SparkPlan should override doBroadcast. + * Returns the result of this query as a broadcast variable by delegating to `doExecuteBroadcast` + * after preparations. + * + * Concrete implementations of SparkPlan should override `doExecuteBroadcast`. */ final def executeBroadcast[T](): broadcast.Broadcast[T] = executeQuery { doExecuteBroadcast() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 15b4abe806678..d6f7b6ed35dbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -105,7 +105,7 @@ trait CodegenSupport extends SparkPlan { protected def doProduce(ctx: CodegenContext): String /** - * Consume the generated columns or row from current SparkPlan, call it's parent's doConsume(). + * Consume the generated columns or row from current SparkPlan, call its parent's `doConsume()`. */ final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { val inputVars = @@ -212,8 +212,8 @@ trait CodegenSupport extends SparkPlan { /** * InputAdapter is used to hide a SparkPlan from a subtree that support codegen. * - * This is the leaf node of a tree with WholeStageCodegen, is used to generate code that consumes - * an RDD iterator of InternalRow. + * This is the leaf node of a tree with WholeStageCodegen that is used to generate code + * that consumes an RDD iterator of InternalRow. */ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 56a39069511d7..bafbbdf65724d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -158,7 +158,7 @@ case class MapPartitionsExec( * Applies the given function to each input object. * The output of its child must be a single-field row containing the input object. * - * This operator is kind of a safe version of [[ProjectExec]], as it's output is custom object, + * This operator is kind of a safe version of [[ProjectExec]], as its output is custom object, * we need to use safe row to contain it. */ case class MapElementsExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 8e66538575b0c..7b4c035bf3c1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -95,7 +95,7 @@ class FileStreamSource( val endId = end.asInstanceOf[LongOffset].offset assert(startId <= endId) - val files = metadataLog.get(Some(startId + 1), Some(endId)).map(_._2).flatten + val files = metadataLog.get(Some(startId + 1), Some(endId)).flatMap(_._2) logInfo(s"Processing ${files.length} files from ${startId + 1}:$endId") logDebug(s"Streaming ${files.mkString(", ")}") dataFrameBuilder(files) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index b89144d727514..e9052a309595a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -47,7 +47,7 @@ class IncrementalExecution( /** * Records the current id for a given stateful operator in the query plan as the `state` - * preperation walks the query plan. + * preparation walks the query plan. */ private var operatorId = 0 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 7d8b8679c5944..6ececb1062ae1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -84,7 +84,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) assert(framework != null, "Checkpoint.framework is null") assert(graph != null, "Checkpoint.graph is null") assert(checkpointTime != null, "Checkpoint.checkpointTime is null") - logInfo("Checkpoint for time " + checkpointTime + " validated") + logInfo(s"Checkpoint for time $checkpointTime validated") } } @@ -103,7 +103,10 @@ object Checkpoint extends Logging { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds + ".bk") } - /** Get checkpoint files present in the give directory, ordered by oldest-first */ + /** + * @param checkpointDir checkpoint directory to read checkpoint files from + * @return checkpoint files from the `checkpointDir` checkpoint directory, ordered by oldest-first + */ def getCheckpointFiles(checkpointDir: String, fsOption: Option[FileSystem] = None): Seq[Path] = { def sortFunc(path1: Path, path2: Path): Boolean = { @@ -121,11 +124,11 @@ object Checkpoint extends Logging { val filtered = paths.filter(p => REGEX.findFirstIn(p.toString).nonEmpty) filtered.sortWith(sortFunc) } else { - logWarning("Listing " + path + " returned null") + logWarning(s"Listing $path returned null") Seq.empty } } else { - logInfo("Checkpoint directory " + path + " does not exist") + logWarning(s"Checkpoint directory $path does not exist") Seq.empty } } @@ -205,7 +208,7 @@ class CheckpointWriter( // time of a batch is greater than the batch interval, checkpointing for completing an old // batch may run after checkpointing of a new batch. If this happens, checkpoint of an old // batch actually has the latest information, so we want to recovery from it. Therefore, we - // also use the latest checkpoint time as the file name, so that we can recovery from the + // also use the latest checkpoint time as the file name, so that we can recover from the // latest checkpoint file. // // Note: there is only one thread writing the checkpoint files, so we don't need to worry @@ -216,8 +219,7 @@ class CheckpointWriter( while (attempts < MAX_ATTEMPTS && !stopped) { attempts += 1 try { - logInfo("Saving checkpoint for time " + checkpointTime + " to file '" + checkpointFile - + "'") + logInfo(s"Saving checkpoint for time $checkpointTime to file '$checkpointFile'") // Write checkpoint to temp file if (fs.exists(tempFile)) { @@ -237,39 +239,38 @@ class CheckpointWriter( fs.delete(backupFile, true) // just in case it exists } if (!fs.rename(checkpointFile, backupFile)) { - logWarning("Could not rename " + checkpointFile + " to " + backupFile) + logWarning(s"Could not rename $checkpointFile to $backupFile") } } // Rename temp file to the final checkpoint file if (!fs.rename(tempFile, checkpointFile)) { - logWarning("Could not rename " + tempFile + " to " + checkpointFile) + logWarning(s"Could not rename $tempFile to $checkpointFile") } // Delete old checkpoint files val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)) if (allCheckpointFiles.size > 10) { allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach { file => - logInfo("Deleting " + file) + logInfo(s"Deleting $file") fs.delete(file, true) } } // All done, print success val finishTime = System.currentTimeMillis() - logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + - "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") + logInfo(s"Checkpoint for time $checkpointTime saved to file '$checkpointFile'" + + s", took ${bytes.length} bytes and ${finishTime - startTime} ms") jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { case ioe: IOException => - logWarning("Error in attempt " + attempts + " of writing checkpoint to " - + checkpointFile, ioe) + val msg = s"Error in attempt $attempts of writing checkpoint to '$checkpointFile'" + logWarning(msg, ioe) fs = null } } - logWarning("Could not write checkpoint for time " + checkpointTime + " to file " - + checkpointFile + "'") + logWarning(s"Could not write checkpoint for time $checkpointTime to file '$checkpointFile'") } } @@ -278,7 +279,7 @@ class CheckpointWriter( val bytes = Checkpoint.serialize(checkpoint, conf) executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bytes, clearCheckpointDataLater)) - logInfo("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") + logInfo(s"Submitted checkpoint of time ${checkpoint.checkpointTime} to writer queue") } catch { case rej: RejectedExecutionException => logError("Could not submit checkpoint task to the thread pool executor", rej) @@ -295,8 +296,8 @@ class CheckpointWriter( executor.shutdownNow() } val endTime = System.currentTimeMillis() - logInfo("CheckpointWriter executor terminated ? " + terminated + - ", waited for " + (endTime - startTime) + " ms.") + logInfo(s"CheckpointWriter executor terminated? $terminated," + + s" waited for ${endTime - startTime} ms.") stopped = true } } @@ -336,20 +337,20 @@ object CheckpointReader extends Logging { } // Try to read the checkpoint files in the order - logInfo("Checkpoint files found: " + checkpointFiles.mkString(",")) + logInfo(s"Checkpoint files found: ${checkpointFiles.mkString(",")}") var readError: Exception = null checkpointFiles.foreach { file => - logInfo("Attempting to load checkpoint from file " + file) + logInfo(s"Attempting to load checkpoint from file $file") try { val fis = fs.open(file) val cp = Checkpoint.deserialize(fis, conf) - logInfo("Checkpoint successfully loaded from file " + file) - logInfo("Checkpoint was generated at time " + cp.checkpointTime) + logInfo(s"Checkpoint successfully loaded from file $file") + logInfo(s"Checkpoint was generated at time ${cp.checkpointTime}") return Some(cp) } catch { case e: Exception => readError = e - logWarning("Error reading checkpoint from file " + file, e) + logWarning(s"Error reading checkpoint from file $file", e) } } 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 b494ef0dd9665..7ea58afb53dc9 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 @@ -158,7 +158,7 @@ private[spark] class Client( val newAppResponse = newApp.getNewApplicationResponse() appId = newAppResponse.getApplicationId() reportLauncherState(SparkAppHandle.State.SUBMITTED) - launcherBackend.setAppId(appId.toString()) + launcherBackend.setAppId(appId.toString) // Verify whether the cluster has enough resources for our AM verifyClusterResources(newAppResponse) @@ -168,7 +168,7 @@ private[spark] class Client( val appContext = createApplicationSubmissionContext(newApp, containerContext) // Finally, submit and monitor the application - logInfo(s"Submitting application ${appId.getId} to ResourceManager") + logInfo(s"Submitting application $appId to ResourceManager") yarnClient.submitApplication(appContext) appId } catch { From 7dc3fb6ae44ba9863eb59c2724c73201c46e5213 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 5 May 2016 16:51:06 -0700 Subject: [PATCH 0097/1470] [HOTFIX] Fix MLUtils compile --- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 86ce9705a3125..f0346e600d239 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -119,7 +119,7 @@ object MLUtils { previous = current i += 1 } - (label, indices, values) + (label, indices.toArray, values.toArray) } /** From 42f2ee6c5d981cdc8bd6b3845f0593a87aae48b6 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 5 May 2016 18:49:43 -0700 Subject: [PATCH 0098/1470] [SPARK-11395][SPARKR] Support over and window specification in SparkR. This PR: 1. Implement WindowSpec S4 class. 2. Implement Window.partitionBy() and Window.orderBy() as utility functions to create WindowSpec objects. 3. Implement over() of Column class. Author: Sun Rui Author: Sun Rui Closes #10094 from sun-rui/SPARK-11395. (cherry picked from commit 157a49aa410dc1870cd171148d317084c5a90d23) Signed-off-by: Shivaram Venkataraman --- R/pkg/DESCRIPTION | 2 + R/pkg/NAMESPACE | 10 ++ R/pkg/R/DataFrame.R | 4 +- R/pkg/R/WindowSpec.R | 188 ++++++++++++++++++++++ R/pkg/R/generics.R | 29 +++- R/pkg/R/pairRDD.R | 4 +- R/pkg/R/window.R | 98 +++++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 36 +++++ 8 files changed, 364 insertions(+), 7 deletions(-) create mode 100644 R/pkg/R/WindowSpec.R create mode 100644 R/pkg/R/window.R diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 7179438efc1d9..963a1bb5806a7 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -26,6 +26,7 @@ Collate: 'pairRDD.R' 'DataFrame.R' 'SQLContext.R' + 'WindowSpec.R' 'backend.R' 'broadcast.R' 'client.R' @@ -38,4 +39,5 @@ Collate: 'stats.R' 'types.R' 'utils.R' + 'window.R' RoxygenNote: 5.0.1 diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 73f7c595f4437..1432ab8a9d1ce 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -216,6 +216,7 @@ exportMethods("%in%", "next_day", "ntile", "otherwise", + "over", "percent_rank", "pmod", "quarter", @@ -315,3 +316,12 @@ export("structField", "structType.jobj", "structType.structField", "print.structType") + +exportClasses("WindowSpec") + +export("partitionBy", + "rowsBetween", + "rangeBetween") + +export("window.partitionBy", + "window.orderBy") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index fcf473ac7b76e..43c46b847446b 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1749,8 +1749,8 @@ setMethod("arrange", #' @export setMethod("orderBy", signature(x = "SparkDataFrame", col = "characterOrColumn"), - function(x, col) { - arrange(x, col) + function(x, col, ...) { + arrange(x, col, ...) }) #' Filter diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R new file mode 100644 index 0000000000000..581176a6c0918 --- /dev/null +++ b/R/pkg/R/WindowSpec.R @@ -0,0 +1,188 @@ +# +# 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. +# + +# WindowSpec.R - WindowSpec class and methods implemented in S4 OO classes + +#' @include generics.R jobj.R column.R +NULL + +#' @title S4 class that represents a WindowSpec +#' @description WindowSpec can be created by using window.partitionBy() +#' or window.orderBy() +#' @rdname WindowSpec +#' @seealso \link{window.partitionBy}, \link{window.orderBy} +#' +#' @param sws A Java object reference to the backing Scala WindowSpec +#' @export +setClass("WindowSpec", + slots = list(sws = "jobj")) + +setMethod("initialize", "WindowSpec", function(.Object, sws) { + .Object@sws <- sws + .Object +}) + +windowSpec <- function(sws) { + stopifnot(class(sws) == "jobj") + new("WindowSpec", sws) +} + +#' @rdname show +setMethod("show", "WindowSpec", + function(object) { + cat("WindowSpec", callJMethod(object@sws, "toString"), "\n") + }) + +#' partitionBy +#' +#' Defines the partitioning columns in a WindowSpec. +#' +#' @param x a WindowSpec +#' @return a WindowSpec +#' @rdname partitionBy +#' @name partitionBy +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' partitionBy(ws, "col1", "col2") +#' partitionBy(ws, df$col1, df$col2) +#' } +setMethod("partitionBy", + signature(x = "WindowSpec"), + function(x, col, ...) { + stopifnot (class(col) %in% c("character", "Column")) + + if (class(col) == "character") { + windowSpec(callJMethod(x@sws, "partitionBy", col, list(...))) + } else { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "partitionBy", jcols)) + } + }) + +#' orderBy +#' +#' Defines the ordering columns in a WindowSpec. +#' +#' @param x a WindowSpec +#' @return a WindowSpec +#' @rdname arrange +#' @name orderBy +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' orderBy(ws, "col1", "col2") +#' orderBy(ws, df$col1, df$col2) +#' } +setMethod("orderBy", + signature(x = "WindowSpec", col = "character"), + function(x, col, ...) { + windowSpec(callJMethod(x@sws, "orderBy", col, list(...))) + }) + +#' @rdname arrange +#' @name orderBy +#' @export +setMethod("orderBy", + signature(x = "WindowSpec", col = "Column"), + function(x, col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "orderBy", jcols)) + }) + +#' rowsBetween +#' +#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' +#' Both `start` and `end` are relative positions from the current row. For example, "0" means +#' "current row", while "-1" means the row before the current row, and "5" means the fifth row +#' after the current row. +#' +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rowsBetween +#' @name rowsBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rowsBetween(ws, 0, 3) +#' } +setMethod("rowsBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rowsBetween", as.integer(start), as.integer(end))) + }) + +#' rangeBetween +#' +#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' +#' Both `start` and `end` are relative from the current row. For example, "0" means "current row", +#' while "-1" means one off before the current row, and "5" means the five off after the +#' current row. + +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rangeBetween +#' @name rangeBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rangeBetween(ws, 0, 3) +#' } +setMethod("rangeBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rangeBetween", as.integer(start), as.integer(end))) + }) + +# Note that over is a method of Column class, but it is placed here to +# avoid Roxygen circular-dependency between class Column and WindowSpec. + +#' over +#' +#' Define a windowing column. +#' +#' @rdname over +#' @name over +#' @family colum_func +#' @export +setMethod("over", + signature(x = "Column", window = "WindowSpec"), + function(x, window) { + column(callJMethod(x@jc, "over", window@sws)) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 3db1ac07666b3..8563be1e64983 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -339,9 +339,9 @@ setGeneric("join", function(x, y, ...) { standardGeneric("join") }) # @export setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) -# @rdname partitionBy -# @export -setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, ...) { standardGeneric("partitionBy") }) # @rdname reduceByKey # @seealso groupByKey @@ -533,7 +533,7 @@ setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) #' @rdname arrange #' @export -setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") }) +setGeneric("orderBy", function(x, col, ...) { standardGeneric("orderBy") }) #' @rdname schema #' @export @@ -733,6 +733,27 @@ setGeneric("when", function(condition, value) { standardGeneric("when") }) #' @export setGeneric("otherwise", function(x, value) { standardGeneric("otherwise") }) +#' @rdname over +#' @export +setGeneric("over", function(x, window) { standardGeneric("over") }) + +###################### WindowSpec Methods ########################## + +#' @rdname rowsBetween +#' @export +setGeneric("rowsBetween", function(x, start, end) { standardGeneric("rowsBetween") }) + +#' @rdname rangeBetween +#' @export +setGeneric("rangeBetween", function(x, start, end) { standardGeneric("rangeBetween") }) + +#' @rdname window.partitionBy +#' @export +setGeneric("window.partitionBy", function(col, ...) { standardGeneric("window.partitionBy") }) + +#' @rdname window.orderBy +#' @export +setGeneric("window.orderBy", function(col, ...) { standardGeneric("window.orderBy") }) ###################### Expression Function Methods ########################## diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 4075ef4377acf..d39775cabef88 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -205,8 +205,10 @@ setMethod("flatMapValues", #' @aliases partitionBy,RDD,integer-method #' @noRd setMethod("partitionBy", - signature(x = "RDD", numPartitions = "numeric"), + signature(x = "RDD"), function(x, numPartitions, partitionFunc = hashCode) { + stopifnot(is.numeric(numPartitions)) + partitionFunc <- cleanClosure(partitionFunc) serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R new file mode 100644 index 0000000000000..7ecf70abc6b50 --- /dev/null +++ b/R/pkg/R/window.R @@ -0,0 +1,98 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# window.R - Utility functions for defining window in DataFrames + +#' window.partitionBy +#' +#' Creates a WindowSpec with the partitioning defined. +#' +#' @rdname window.partitionBy +#' @name window.partitionBy +#' @export +#' @examples +#' \dontrun{ +#' ws <- window.partitionBy("key1", "key2") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- window.partitionBy(df$key1, df$key2) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +setMethod("window.partitionBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + col, + list(...))) + }) + +#' @rdname window.partitionBy +#' @name window.partitionBy +#' @export +setMethod("window.partitionBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + jcols)) + }) + +#' window.orderBy +#' +#' Creates a WindowSpec with the ordering defined. +#' +#' @rdname window.orderBy +#' @name window.orderBy +#' @export +#' @examples +#' \dontrun{ +#' ws <- window.orderBy("key1", "key2") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- window.orderBy(df$key1, df$key2) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +setMethod("window.orderBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + col, + list(...))) + }) + +#' @rdname window.orderBy +#' @name window.orderBy +#' @export +setMethod("window.orderBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + jcols)) + }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 3b6a27c3b86a1..0f67bc2e331d1 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2118,6 +2118,42 @@ test_that("repartition by columns on DataFrame", { expect_equal(nrow(df1), 2) }) +test_that("Window functions on a DataFrame", { + ssc <- callJMethod(sc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, + error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + + df <- createDataFrame(hiveCtx, + list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), + schema = c("key", "value")) + ws <- orderBy(window.partitionBy("key"), "value") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expected <- data.frame(key = c(1L, NA, 2L, NA), + value = c("1", NA, "2", NA), + stringsAsFactors = FALSE) + expect_equal(result, expected) + + ws <- orderBy(window.partitionBy(df$key), df$value) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(window.orderBy("value"), "key") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(window.orderBy(df$value), df$key) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) +}) + unlink(parquetPath) unlink(jsonPath) unlink(jsonPathNa) From 1ee621b1d949ce8e1bb41ef3fe19dfaad4a90ab1 Mon Sep 17 00:00:00 2001 From: Luciano Resende Date: Fri, 6 May 2016 12:25:45 +0100 Subject: [PATCH 0099/1470] [SPARK-14738][BUILD] Separate docker integration tests from main build ## What changes were proposed in this pull request? Create a maven profile for executing the docker integration tests using maven Remove docker integration tests from main sbt build Update documentation on how to run docker integration tests from sbt ## How was this patch tested? Manual test of the docker integration tests as in : mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 compile test ## Other comments Note that the the DB2 Docker Tests are still disabled as there is a kernel version issue on the AMPLab Jenkins slaves and we would need to get them on the right level before enabling those tests. They do run ok locally with the updates from PR #12348 Author: Luciano Resende Closes #12508 from lresende/docker. (cherry picked from commit a03c5e68abd8c066c97ebd388883070d59dce1a7) Signed-off-by: Sean Owen --- docs/building-spark.md | 12 ++++++++++++ .../spark/sql/jdbc/MySQLIntegrationSuite.scala | 3 --- .../spark/sql/jdbc/OracleIntegrationSuite.scala | 5 +---- .../spark/sql/jdbc/PostgresIntegrationSuite.scala | 3 --- pom.xml | 8 +++++++- project/SparkBuild.scala | 3 ++- 6 files changed, 22 insertions(+), 12 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index fec442af95e1b..13c95e4fcb395 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -190,6 +190,18 @@ or Java 8 tests are automatically enabled when a Java 8 JDK is detected. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +# Running Docker based Integration Test Suites + +Running only docker based integration tests and nothing else. + + mvn install -DskipTests + mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + +or + + sbt docker-integration-tests/test + + # Packaging without Hadoop Dependencies for YARN The assembly directory produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with `yarn.application.classpath`. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index aa47228eff3a2..a70ed98b52d5d 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -21,12 +21,9 @@ import java.math.BigDecimal import java.sql.{Connection, Date, Timestamp} import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.tags.DockerTest @DockerTest -@Ignore class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { override val imageName = "mysql:5.7.9" diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 357866b87ca2a..c5e1f8607b333 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.jdbc import java.sql.Connection import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.tags.DockerTest @@ -46,12 +44,11 @@ import org.apache.spark.tags.DockerTest * repository. */ @DockerTest -@Ignore class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLContext { import testImplicits._ override val db = new DatabaseOnDocker { - override val imageName = "wnameless/oracle-xe-11g:latest" + override val imageName = "wnameless/oracle-xe-11g:14.04.4" override val env = Map( "ORACLE_ROOT_PASSWORD" -> "oracle" ) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 6546d4cfd7ce8..79dd70116ecb8 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -20,15 +20,12 @@ package org.apache.spark.sql.jdbc import java.sql.Connection import java.util.Properties -import org.scalatest.Ignore - import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{ArrayType, DecimalType} import org.apache.spark.tags.DockerTest @DockerTest -@Ignore class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { override val imageName = "postgres:9.4.5" diff --git a/pom.xml b/pom.xml index 852136a998aba..3e783fa56e769 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,6 @@ sql/core sql/hive sql/hivecontext-compatibility - external/docker-integration-tests assembly external/flume external/flume-sink @@ -2469,6 +2468,13 @@ + + docker-integration-tests + + external/docker-integration-tests + + + - org.eclipse.jetty.orbit - javax.servlet - ${orbit.version} + org.eclipse.jetty + jetty-servlets + compile + + + javax.servlet + javax.servlet-api + ${javaxservlet.version} @@ -356,7 +359,7 @@ true true - guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security + guava,jetty-io,jetty-servlet,jetty-servlets,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security true diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index 719905a2c901c..be19179b00a49 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -71,7 +71,7 @@ private[spark] case class SSLOptions( keyPassword.foreach(sslContextFactory.setKeyManagerPassword) keyStoreType.foreach(sslContextFactory.setKeyStoreType) if (needClientAuth) { - trustStore.foreach(file => sslContextFactory.setTrustStore(file.getAbsolutePath)) + trustStore.foreach(file => sslContextFactory.setTrustStorePath(file.getAbsolutePath)) trustStorePassword.foreach(sslContextFactory.setTrustStorePassword) trustStoreType.foreach(sslContextFactory.setTrustStoreType) } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 14244ea5714c6..7e93bfc45ebc9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -17,13 +17,12 @@ package org.apache.spark.deploy.rest -import java.net.InetSocketAddress import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.{Server, ServerConnector} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s._ @@ -80,10 +79,15 @@ private[spark] abstract class RestSubmissionServer( * Return a 2-tuple of the started server and the bound port. */ private def doStart(startPort: Int): (Server, Int) = { - val server = new Server(new InetSocketAddress(host, startPort)) val threadPool = new QueuedThreadPool threadPool.setDaemon(true) - server.setThreadPool(threadPool) + val server = new Server(threadPool) + + val connector = new ServerConnector(server) + connector.setHost(host) + connector.setPort(startPort) + server.addConnector(connector) + val mainHandler = new ServletContextHandler mainHandler.setContextPath("/") contextToServlet.foreach { case (prefix, servlet) => @@ -91,7 +95,7 @@ private[spark] abstract class RestSubmissionServer( } server.setHandler(mainHandler) server.start() - val boundPort = server.getConnectors()(0).getLocalPort + val boundPort = connector.getLocalPort (server, boundPort) } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index db24f0319ba05..6854f7baf716d 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -25,11 +25,10 @@ import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import scala.xml.Node -import org.eclipse.jetty.server.{Connector, Request, Server} +import org.eclipse.jetty.server.{Request, Server, ServerConnector} import org.eclipse.jetty.server.handler._ -import org.eclipse.jetty.server.nio.SelectChannelConnector -import org.eclipse.jetty.server.ssl.SslSelectChannelConnector import org.eclipse.jetty.servlet._ +import org.eclipse.jetty.servlets.gzip.GzipHandler import org.eclipse.jetty.util.component.LifeCycle import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s.JValue @@ -243,10 +242,16 @@ private[spark] object JettyUtils extends Logging { // Bind to the given port, or throw a java.net.BindException if the port is occupied def connect(currentPort: Int): (Server, Int) = { - val server = new Server - val connectors = new ArrayBuffer[Connector] + val pool = new QueuedThreadPool + if (serverName.nonEmpty) { + pool.setName(serverName) + } + pool.setDaemon(true) + + val server = new Server(pool) + val connectors = new ArrayBuffer[ServerConnector] // Create a connector on port currentPort to listen for HTTP requests - val httpConnector = new SelectChannelConnector() + val httpConnector = new ServerConnector(server) httpConnector.setPort(currentPort) connectors += httpConnector @@ -260,8 +265,9 @@ private[spark] object JettyUtils extends Logging { } val scheme = "https" // Create a connector on port securePort to listen for HTTPS requests - val connector = new SslSelectChannelConnector(factory) + val connector = new ServerConnector(server, factory) connector.setPort(securePort) + connectors += connector // redirect the HTTP requests to HTTPS port @@ -269,34 +275,27 @@ private[spark] object JettyUtils extends Logging { } gzipHandlers.foreach(collection.addHandler) - connectors.foreach(_.setHost(hostName)) // As each acceptor and each selector will use one thread, the number of threads should at // least be the number of acceptors and selectors plus 1. (See SPARK-13776) var minThreads = 1 - connectors.foreach { c => + connectors.foreach { connector => // Currently we only use "SelectChannelConnector" - val connector = c.asInstanceOf[SelectChannelConnector] // Limit the max acceptor number to 8 so that we don't waste a lot of threads - connector.setAcceptors(math.min(connector.getAcceptors, 8)) + connector.setAcceptQueueSize(math.min(connector.getAcceptors, 8)) + connector.setHost(hostName) // The number of selectors always equals to the number of acceptors minThreads += connector.getAcceptors * 2 } server.setConnectors(connectors.toArray) - - val pool = new QueuedThreadPool - if (serverName.nonEmpty) { - pool.setName(serverName) - } pool.setMaxThreads(math.max(pool.getMaxThreads, minThreads)) - pool.setDaemon(true) - server.setThreadPool(pool) + val errorHandler = new ErrorHandler() errorHandler.setShowStacks(true) server.addBean(errorHandler) server.setHandler(collection) try { server.start() - (server, server.getConnectors.head.getLocalPort) + (server, httpConnector.getLocalPort) } catch { case e: Exception => server.stop() diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index cc934e9e6ff28..83bdd9031903f 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -46,7 +46,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -83,7 +83,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jcl-over-slf4j-1.7.16.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index f85aab3f93d51..121e282618378 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -86,7 +86,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index dfcd35193c4e4..1d5ad271326d9 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar @@ -86,7 +86,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index a29f6700bd598..909b94bde20c8 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar @@ -94,7 +94,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8955b0e97727d..7507599740219 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.10.1.1.jar +derby-10.11.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar @@ -94,7 +94,7 @@ javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar javax.inject-2.4.0-b34.jar -javax.servlet-3.0.0.v201112011016.jar +javax.servlet-api-3.1.0.jar javax.ws.rs-api-2.0.1.jar javolution-5.5.1.jar jaxb-api-2.2.2.jar diff --git a/pom.xml b/pom.xml index d71913c9589c7..9e9aad223a5e1 100644 --- a/pom.xml +++ b/pom.xml @@ -133,11 +133,11 @@ 1.2.1.spark2 1.2.1 - 10.10.1.1 + 10.11.1.1 1.7.0 1.6.0 - 8.1.19.v20160209 - 3.0.0.v201112011016 + 9.2.16.v20160414 + 3.1.0 0.8.0 2.4.0 2.0.8 @@ -328,6 +328,12 @@ ${jetty.version} provided + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + provided + org.eclipse.jetty jetty-util @@ -2264,6 +2270,7 @@ org.eclipse.jetty:jetty-http org.eclipse.jetty:jetty-continuation org.eclipse.jetty:jetty-servlet + org.eclipse.jetty:jetty-servlets org.eclipse.jetty:jetty-plus org.eclipse.jetty:jetty-security org.eclipse.jetty:jetty-util diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 3b57efa38b588..37e4845cceb9e 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -37,8 +37,7 @@ import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.TServlet; -import org.eclipse.jetty.server.nio.SelectChannelConnector; -import org.eclipse.jetty.server.ssl.SslSelectChannelConnector; +import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.ssl.SslContextFactory; @@ -59,9 +58,6 @@ public ThriftHttpCLIService(CLIService cliService) { @Override public void run() { try { - // HTTP Server - httpServer = new org.eclipse.jetty.server.Server(); - // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests String threadPoolName = "HiveServer2-HttpHandler-Pool"; @@ -69,10 +65,12 @@ public void run() { workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), new ThreadFactoryWithGarbageCleanup(threadPoolName)); ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); - httpServer.setThreadPool(threadPool); + + // HTTP Server + httpServer = new org.eclipse.jetty.server.Server(threadPool); // Connector configs - SelectChannelConnector connector = new SelectChannelConnector(); + ServerConnector connector = new ServerConnector(httpServer); boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); String schemeName = useSsl ? "https" : "http"; // Change connector if SSL is used @@ -92,14 +90,14 @@ public void run() { Arrays.toString(sslContextFactory.getExcludeProtocols())); sslContextFactory.setKeyStorePath(keyStorePath); sslContextFactory.setKeyStorePassword(keyStorePassword); - connector = new SslSelectChannelConnector(sslContextFactory); + connector = new ServerConnector(httpServer, sslContextFactory); } connector.setPort(portNum); // Linux:yes, Windows:no connector.setReuseAddress(!Shell.WINDOWS); int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, TimeUnit.MILLISECONDS); - connector.setMaxIdleTime(maxIdleTime); + connector.setIdleTimeout(maxIdleTime); httpServer.addConnector(connector); diff --git a/streaming/pom.xml b/streaming/pom.xml index 7d409c5d3b076..e7415863e3568 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -77,6 +77,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + diff --git a/yarn/pom.xml b/yarn/pom.xml index db7f3e51d3be5..11df2b3f4f761 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -102,6 +102,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + org.glassfish.jersey.core @@ -147,6 +152,15 @@ *:* + + + com.google.protobuf + kinesis.protobuf + + com.google.protobuf.** + + + *:* From 31ea3c7bde94f5bcca1db601f9c16c36c56cef73 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 12 May 2016 13:56:00 -0700 Subject: [PATCH 0205/1470] [SPARK-10605][SQL] Create native collect_list/collect_set aggregates ## What changes were proposed in this pull request? We currently use the Hive implementations for the collect_list/collect_set aggregate functions. This has a few major drawbacks: the use of HiveUDAF (which has quite a bit of overhead) and the lack of support for struct datatypes. This PR adds native implementation of these functions to Spark. The size of the collected list/set may vary, this means we cannot use the fast, Tungsten, aggregation path to perform the aggregation, and that we fallback to the slower sort based path. Another big issue with these operators is that when the size of the collected list/set grows too large, we can start experiencing large GC pauzes and OOMEs. This `collect*` aggregates implemented in this PR rely on the sort based aggregate path for correctness. They maintain their own internal buffer which holds the rows for one group at a time. The sortbased aggregation path is triggered by disabling `partialAggregation` for these aggregates (which is kinda funny); this technique is also employed in `org.apache.spark.sql.hiveHiveUDAFFunction`. I have done some performance testing: ```scala import org.apache.spark.sql.{Dataset, Row} sql("create function collect_list2 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectList'") val df = range(0, 10000000).select($"id", (rand(213123L) * 100000).cast("int").as("grp")) df.select(countDistinct($"grp")).show def benchmark(name: String, plan: Dataset[Row], maxItr: Int = 5): Unit = { // Do not measure planning. plan1.queryExecution.executedPlan // Execute the plan a number of times and average the result. val start = System.nanoTime var i = 0 while (i < maxItr) { plan.rdd.foreach(row => Unit) i += 1 } val time = (System.nanoTime - start) / (maxItr * 1000000L) println(s"[$name] $maxItr iterations completed in an average time of $time ms.") } val plan1 = df.groupBy($"grp").agg(collect_list($"id")) val plan2 = df.groupBy($"grp").agg(callUDF("collect_list2", $"id")) benchmark("Spark collect_list", plan1) ... > [Spark collect_list] 5 iterations completed in an average time of 3371 ms. benchmark("Hive collect_list", plan2) ... > [Hive collect_list] 5 iterations completed in an average time of 9109 ms. ``` Performance is improved by a factor 2-3. ## How was this patch tested? Added tests to `DataFrameAggregateSuite`. Author: Herman van Hovell Closes #12874 from hvanhovell/implode. (cherry picked from commit bb1362eb3b36b553dca246b95f59ba7fd8adcc8a) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../expressions/aggregate/collect.scala | 119 ++++++++++++++++++ .../org/apache/spark/sql/functions.scala | 12 +- .../spark/sql/DataFrameAggregateSuite.scala | 26 ++++ .../spark/sql/hive/HiveSessionCatalog.scala | 16 --- .../hive/HiveDataFrameAnalyticsSuite.scala | 11 -- 6 files changed, 149 insertions(+), 37 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala 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 ac05dd3d0ef9e..c459fe587859e 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 @@ -252,6 +252,8 @@ object FunctionRegistry { expression[VarianceSamp]("variance"), expression[VariancePop]("var_pop"), expression[VarianceSamp]("var_samp"), + expression[CollectList]("collect_list"), + expression[CollectSet]("collect_set"), // string functions expression[Ascii]("ascii"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala new file mode 100644 index 0000000000000..1f4ff9c4b184e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.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.sql.catalyst.expressions.aggregate + +import scala.collection.generic.Growable +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types._ + +/** + * The Collect aggregate function collects all seen expression values into a list of values. + * + * The operator is bound to the slower sort based aggregation path because the number of + * elements (and their memory usage) can not be determined in advance. This also means that the + * collected elements are stored on heap, and that too many elements can cause GC pauses and + * eventually Out of Memory Errors. + */ +abstract class Collect extends ImperativeAggregate { + + val child: Expression + + override def children: Seq[Expression] = child :: Nil + + override def nullable: Boolean = true + + override def dataType: DataType = ArrayType(child.dataType) + + override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType) + + override def supportsPartial: Boolean = false + + override def aggBufferAttributes: Seq[AttributeReference] = Nil + + override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + + override def inputAggBufferAttributes: Seq[AttributeReference] = Nil + + protected[this] val buffer: Growable[Any] with Iterable[Any] + + override def initialize(b: MutableRow): Unit = { + buffer.clear() + } + + override def update(b: MutableRow, input: InternalRow): Unit = { + buffer += child.eval(input) + } + + override def merge(buffer: MutableRow, input: InternalRow): Unit = { + sys.error("Collect cannot be used in partial aggregations.") + } + + override def eval(input: InternalRow): Any = { + new GenericArrayData(buffer.toArray) + } +} + +/** + * Collect a list of elements. + */ +@ExpressionDescription( + usage = "_FUNC_(expr) - Collects and returns a list of non-unique elements.") +case class CollectList( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends Collect { + + def this(child: Expression) = this(child, 0, 0) + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def prettyName: String = "collect_list" + + override protected[this] val buffer: mutable.ArrayBuffer[Any] = mutable.ArrayBuffer.empty +} + +/** + * Collect a list of unique elements. + */ +@ExpressionDescription( + usage = "_FUNC_(expr) - Collects and returns a set of unique elements.") +case class CollectSet( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends Collect { + + def this(child: Expression) = this(child, 0, 0) + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def prettyName: String = "collect_set" + + override protected[this] val buffer: mutable.HashSet[Any] = mutable.HashSet.empty +} 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 3e295c20b6d9f..07f55042eeb40 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 @@ -195,18 +195,14 @@ object functions { /** * Aggregate function: returns a list of objects with duplicates. * - * For now this is an alias for the collect_list Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ - def collect_list(e: Column): Column = callUDF("collect_list", e) + def collect_list(e: Column): Column = withAggregateFunction { CollectList(e.expr) } /** * Aggregate function: returns a list of objects with duplicates. * - * For now this is an alias for the collect_list Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ @@ -215,18 +211,14 @@ object functions { /** * Aggregate function: returns a set of objects with duplicate elements eliminated. * - * For now this is an alias for the collect_set Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ - def collect_set(e: Column): Column = callUDF("collect_set", e) + def collect_set(e: Column): Column = withAggregateFunction { CollectSet(e.expr) } /** * Aggregate function: returns a set of objects with duplicate elements eliminated. * - * For now this is an alias for the collect_set Hive UDAF. - * * @group agg_funcs * @since 1.6.0 */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 8a99866a33c75..69a990789bcfd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -431,6 +431,32 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { Row(null, null, null, null, null)) } + test("collect functions") { + val df = Seq((1, 2), (2, 2), (3, 4)).toDF("a", "b") + checkAnswer( + df.select(collect_list($"a"), collect_list($"b")), + Seq(Row(Seq(1, 2, 3), Seq(2, 2, 4))) + ) + checkAnswer( + df.select(collect_set($"a"), collect_set($"b")), + Seq(Row(Seq(1, 2, 3), Seq(2, 4))) + ) + } + + test("collect functions structs") { + val df = Seq((1, 2, 2), (2, 2, 2), (3, 4, 1)) + .toDF("a", "x", "y") + .select($"a", struct($"x", $"y").as("b")) + checkAnswer( + df.select(collect_list($"a"), sort_array(collect_list($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(2, 2), Row(4, 1)))) + ) + checkAnswer( + df.select(collect_set($"a"), sort_array(collect_set($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(4, 1)))) + ) + } + test("SPARK-14664: Decimal sum/avg over window should work.") { checkAnswer( spark.sql("select sum(a) over () from values 1.0, 2.0, 3.0 T(a)"), diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 75a252ccba569..4f8aac8c2fcdd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -222,20 +222,4 @@ private[sql] class HiveSessionCatalog( } } } - - // Pre-load a few commonly used Hive built-in functions. - HiveSessionCatalog.preloadedHiveBuiltinFunctions.foreach { - case (functionName, clazz) => - val builder = makeFunctionBuilder(functionName, clazz) - val info = new ExpressionInfo(clazz.getCanonicalName, functionName) - createTempFunction(functionName, info, builder, ignoreIfExists = false) - } -} - -private[sql] object HiveSessionCatalog { - // This is the list of Hive's built-in functions that are commonly used and we want to - // pre-load when we create the FunctionRegistry. - val preloadedHiveBuiltinFunctions = - ("collect_set", classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectSet]) :: - ("collect_list", classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectList]) :: Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala index 57f96e725a044..cc41c04c71e16 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala @@ -58,17 +58,6 @@ class HiveDataFrameAnalyticsSuite extends QueryTest with TestHiveSingleton with ) } - test("collect functions") { - checkAnswer( - testData.select(collect_list($"a"), collect_list($"b")), - Seq(Row(Seq(1, 2, 3), Seq(2, 2, 4))) - ) - checkAnswer( - testData.select(collect_set($"a"), collect_set($"b")), - Seq(Row(Seq(1, 2, 3), Seq(2, 4))) - ) - } - test("cube") { checkAnswer( testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), From 0d24fe09aa5bafe85bf694dc4e2e2ebbfb3af250 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 12 May 2016 12:36:18 -0700 Subject: [PATCH 0206/1470] [SPARK-13902][SCHEDULER] Make DAGScheduler not to create duplicate stage. ## What changes were proposed in this pull request? `DAGScheduler`sometimes generate incorrect stage graph. Suppose you have the following DAG: ``` [A] <--(s_A)-- [B] <--(s_B)-- [C] <--(s_C)-- [D] \ / <------------- ``` Note: [] means an RDD, () means a shuffle dependency. Here, RDD `B` has a shuffle dependency on RDD `A`, and RDD `C` has shuffle dependency on both `B` and `A`. The shuffle dependency IDs are numbers in the `DAGScheduler`, but to make the example easier to understand, let's call the shuffled data from `A` shuffle dependency ID `s_A` and the shuffled data from `B` shuffle dependency ID `s_B`. The `getAncestorShuffleDependencies` method in `DAGScheduler` (incorrectly) does not check for duplicates when it's adding ShuffleDependencies to the parents data structure, so for this DAG, when `getAncestorShuffleDependencies` gets called on `C` (previous of the final RDD), `getAncestorShuffleDependencies` will return `s_A`, `s_B`, `s_A` (`s_A` gets added twice: once when the method "visit"s RDD `C`, and once when the method "visit"s RDD `B`). This is problematic because this line of code: https://github.com/apache/spark/blob/8ef3399/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L289 then generates a new shuffle stage for each dependency returned by `getAncestorShuffleDependencies`, resulting in duplicate map stages that compute the map output from RDD `A`. As a result, `DAGScheduler` generates the following stages and their parents for each shuffle: | | stage | parents | |----|----|----| | s_A | ShuffleMapStage 2 | List() | | s_B | ShuffleMapStage 1 | List(ShuffleMapStage 0) | | s_C | ShuffleMapStage 3 | List(ShuffleMapStage 1, ShuffleMapStage 2) | | - | ResultStage 4 | List(ShuffleMapStage 3) | The stage for s_A should be `ShuffleMapStage 0`, but the stage for `s_A` is generated twice as `ShuffleMapStage 2` and `ShuffleMapStage 0` is overwritten by `ShuffleMapStage 2`, and the stage `ShuffleMap Stage1` keeps referring the old stage `ShuffleMapStage 0`. This patch is fixing it. ## How was this patch tested? I added the sample RDD graph to show the illegal stage graph to `DAGSchedulerSuite`. Author: Takuya UESHIN Closes #12655 from ueshin/issues/SPARK-13902. --- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 47 +++++++++++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4dfd532e93624..5291b663667ea 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -286,7 +286,9 @@ class DAGScheduler( case None => // We are going to register ancestor shuffle dependencies getAncestorShuffleDependencies(shuffleDep.rdd).foreach { dep => - shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId) + if (!shuffleToMapStage.contains(dep.shuffleId)) { + shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId) + } } // Then register current shuffleDep val stage = newOrUsedShuffleStage(shuffleDep, firstJobId) 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 e3ed079e4e160..088a476086217 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -325,6 +325,53 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) } + /** + * This test ensures that DAGScheduler build stage graph correctly. + * + * Suppose you have the following DAG: + * + * [A] <--(s_A)-- [B] <--(s_B)-- [C] <--(s_C)-- [D] + * \ / + * <------------- + * + * Here, RDD B has a shuffle dependency on RDD A, and RDD C has shuffle dependency on both + * B and A. The shuffle dependency IDs are numbers in the DAGScheduler, but to make the example + * easier to understand, let's call the shuffled data from A shuffle dependency ID s_A and the + * shuffled data from B shuffle dependency ID s_B. + * + * Note: [] means an RDD, () means a shuffle dependency. + */ + test("[SPARK-13902] Ensure no duplicate stages are created") { + val rddA = new MyRDD(sc, 1, Nil) + val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1)) + val s_A = shuffleDepA.shuffleId + + val rddB = new MyRDD(sc, 1, List(shuffleDepA), tracker = mapOutputTracker) + val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(1)) + val s_B = shuffleDepB.shuffleId + + val rddC = new MyRDD(sc, 1, List(shuffleDepA, shuffleDepB), tracker = mapOutputTracker) + val shuffleDepC = new ShuffleDependency(rddC, new HashPartitioner(1)) + val s_C = shuffleDepC.shuffleId + + val rddD = new MyRDD(sc, 1, List(shuffleDepC), tracker = mapOutputTracker) + + submit(rddD, Array(0)) + + assert(scheduler.shuffleToMapStage.size === 3) + assert(scheduler.activeJobs.size === 1) + + val mapStageA = scheduler.shuffleToMapStage(s_A) + val mapStageB = scheduler.shuffleToMapStage(s_B) + val mapStageC = scheduler.shuffleToMapStage(s_C) + val finalStage = scheduler.activeJobs.head.finalStage + + assert(mapStageA.parents.isEmpty) + assert(mapStageB.parents === List(mapStageA)) + assert(mapStageC.parents === List(mapStageA, mapStageB)) + assert(finalStage.parents === List(mapStageC)) + } + test("zero split job") { var numResults = 0 var failureReason: Option[Exception] = None From 54c04aa5d0a6012eb58efd0e7cf6d1d287818fa8 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 12 May 2016 17:50:55 -0700 Subject: [PATCH 0207/1470] [SPARK-15202][SPARKR] add dapplyCollect() method for DataFrame in SparkR. ## What changes were proposed in this pull request? dapplyCollect() applies an R function on each partition of a SparkDataFrame and collects the result back to R as a data.frame. ``` dapplyCollect(df, function(ldf) {...}) ``` ## How was this patch tested? SparkR unit tests. Author: Sun Rui Closes #12989 from sun-rui/SPARK-15202. (cherry picked from commit b3930f74a0929b2cdcbbe5cbe34f0b1d35eb01cc) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 86 ++++++++++++++++++----- R/pkg/R/generics.R | 4 ++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 21 +++++- 4 files changed, 95 insertions(+), 17 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 1432ab8a9d1ce..239ad065d09ad 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -47,6 +47,7 @@ exportMethods("arrange", "covar_pop", "crosstab", "dapply", + "dapplyCollect", "describe", "dim", "distinct", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 43c46b847446b..0c2a194483b0f 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1153,9 +1153,27 @@ setMethod("summarize", agg(x, ...) }) +dapplyInternal <- function(x, func, schema) { + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "dapply", + x@sdf, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + if (is.null(schema)) { schema } else { schema$jobj }) + dataFrame(sdf) +} + #' dapply #' -#' Apply a function to each partition of a DataFrame. +#' Apply a function to each partition of a SparkDataFrame. #' #' @param x A SparkDataFrame #' @param func A function to be applied to each partition of the SparkDataFrame. @@ -1197,21 +1215,57 @@ setMethod("summarize", setMethod("dapply", signature(x = "SparkDataFrame", func = "function", schema = "structType"), function(x, func, schema) { - packageNamesArr <- serialize(.sparkREnv[[".packages"]], - connection = NULL) - - broadcastArr <- lapply(ls(.broadcastNames), - function(name) { get(name, .broadcastNames) }) - - sdf <- callJStatic( - "org.apache.spark.sql.api.r.SQLUtils", - "dapply", - x@sdf, - serialize(cleanClosure(func), connection = NULL), - packageNamesArr, - broadcastArr, - schema$jobj) - dataFrame(sdf) + dapplyInternal(x, func, schema) + }) + +#' dapplyCollect +#' +#' Apply a function to each partition of a SparkDataFrame and collect the result back +#’ to R as a data.frame. +#' +#' @param x A SparkDataFrame +#' @param func A function to be applied to each partition of the SparkDataFrame. +#' func should have only one parameter, to which a data.frame corresponds +#' to each partition will be passed. +#' The output of func should be a data.frame. +#' @family SparkDataFrame functions +#' @rdname dapply +#' @name dapplyCollect +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame (sqlContext, iris) +#' ldf <- dapplyCollect(df, function(x) { x }) +#' +#' # filter and add a column +#' df <- createDataFrame ( +#' sqlContext, +#' list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), +#' c("a", "b", "c")) +#' ldf <- dapplyCollect( +#' df, +#' function(x) { +#' y <- x[x[1] > 1, ] +#' y <- cbind(y, y[1] + 1L) +#' }) +#' # the result +#' # a b c d +#' # 2 2 2 3 +#' # 3 3 3 4 +#' } +setMethod("dapplyCollect", + signature(x = "SparkDataFrame", func = "function"), + function(x, func) { + df <- dapplyInternal(x, func, NULL) + + content <- callJMethod(df@sdf, "collect") + # content is a list of items of struct type. Each item has a single field + # which is a serialized data.frame corresponds to one partition of the + # SparkDataFrame. + ldfs <- lapply(content, function(x) { unserialize(x[[1]]) }) + ldf <- do.call(rbind, ldfs) + row.names(ldf) <- NULL + ldf }) ############################## RDD Map Functions ################################## diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 8563be1e64983..ed76ad6b73c8b 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -450,6 +450,10 @@ setGeneric("covar_pop", function(col1, col2) {standardGeneric("covar_pop") }) #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) +#' @rdname dapply +#' @export +setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") }) + #' @rdname summary #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 0f67bc2e331d1..6a99b43e5aa59 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2043,7 +2043,7 @@ test_that("Histogram", { expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) }) -test_that("dapply() on a DataFrame", { +test_that("dapply() and dapplyCollect() on a DataFrame", { df <- createDataFrame ( sqlContext, list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), @@ -2053,6 +2053,8 @@ test_that("dapply() on a DataFrame", { result <- collect(df1) expect_identical(ldf, result) + result <- dapplyCollect(df, function(x) { x }) + expect_identical(ldf, result) # Filter and add a column schema <- structType(structField("a", "integer"), structField("b", "double"), @@ -2070,6 +2072,16 @@ test_that("dapply() on a DataFrame", { rownames(expected) <- NULL expect_identical(expected, result) + result <- dapplyCollect( + df, + function(x) { + y <- x[x$a > 1, ] + y <- cbind(y, y$a + 1L) + }) + expected1 <- expected + names(expected1) <- names(result) + expect_identical(expected1, result) + # Remove the added column df2 <- dapply( df1, @@ -2080,6 +2092,13 @@ test_that("dapply() on a DataFrame", { result <- collect(df2) expected <- expected[, c("a", "b", "c")] expect_identical(expected, result) + + result <- dapplyCollect( + df1, + function(x) { + x[, c("a", "b", "c")] + }) + expect_identical(expected, result) }) test_that("repartition by columns on DataFrame", { From d73ce364e301a54e60b336f0fd2d3836d5d0f25a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 12 May 2016 21:35:14 -0700 Subject: [PATCH 0208/1470] [SPARK-15306][SQL] Move object expressions into expressions.objects package ## What changes were proposed in this pull request? This patch moves all the object related expressions into expressions.objects package, for better code organization. ## How was this patch tested? N/A Author: Reynold Xin Closes #13085 from rxin/SPARK-15306. (cherry picked from commit ba169c3230e7d6cb192ec4bd567a1fef7b93b29f) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/sql/Encoders.scala | 3 ++- .../org/apache/spark/sql/catalyst/JavaTypeInference.scala | 1 + .../scala/org/apache/spark/sql/catalyst/ScalaReflection.scala | 1 + .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 1 + .../main/scala/org/apache/spark/sql/catalyst/dsl/package.scala | 1 + .../apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala | 1 + .../org/apache/spark/sql/catalyst/encoders/RowEncoder.scala | 1 + .../sql/catalyst/expressions/ReferenceToExpressions.scala | 1 + .../spark/sql/catalyst/expressions/{ => objects}/objects.scala | 3 ++- .../org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala | 3 ++- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 1 + .../sql/execution/aggregate/TypedAggregateExpression.scala | 1 + .../main/scala/org/apache/spark/sql/execution/objects.scala | 1 + 13 files changed, 16 insertions(+), 3 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{ => objects}/objects.scala (99%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index 3f4df704db755..fa96f8223d179 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -24,7 +24,8 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} -import org.apache.spark.sql.catalyst.expressions.{BoundReference, DecodeUsingSerializer, EncodeUsingSerializer} +import org.apache.spark.sql.catalyst.expressions.objects.{DecodeUsingSerializer, EncodeUsingSerializer} +import org.apache.spark.sql.catalyst.expressions.BoundReference import org.apache.spark.sql.types._ /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 92caf8f4065c2..690758205efff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -27,6 +27,7 @@ import com.google.common.reflect.TypeToken import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String 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 79bb7a701baf8..cb9a62dfd4e81 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String 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 b7884f9b60f31..ce941e3df3e82 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 @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, InMemoryCatalog, import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.objects.NewInstance import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.planning.IntegerIndex import org.apache.spark.sql.catalyst.plans._ 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 b5d10e4a584f3..2ca990d19a2cb 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 @@ -25,6 +25,7 @@ import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 5d294485afd79..f21a39a2d4730 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, JavaTypeInference, ScalaRefle import org.apache.spark.sql.catalyst.analysis.{SimpleAnalyzer, UnresolvedAttribute, UnresolvedDeserializer, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, NewInstance} import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.types.{ObjectType, StructField, StructType} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 33ac1fdab4f6f..ae842a9f87799 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala index c4cc6c39b0477..502d791c6e85c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.objects.LambdaVariable import org.apache.spark.sql.types.DataType /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index dbaff1625ed58..e8a6c742bfe16 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions +package org.apache.spark.sql.catalyst.expressions.objects import java.lang.reflect.Modifier @@ -27,6 +27,7 @@ import org.apache.spark.SparkConf import org.apache.spark.serializer._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.types._ 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 c00e9c7e39561..1b25f5d7d0cc5 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 @@ -23,7 +23,8 @@ import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.typeOf import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, NewInstance, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Literal, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.objects.NewInstance import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 210ad956ff222..02dd6547a4adc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.optimizer.CombineUnions import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index edca816cb1d16..8f94184764c0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedDe import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index bafbbdf65724d..3ff991392dfb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -24,6 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.{DataType, ObjectType} From 51706f8a4dd94e235cf4e2c0627bc3788fec8251 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 12 May 2016 22:18:39 -0700 Subject: [PATCH 0209/1470] [SPARK-14541][SQL] Support IFNULL, NULLIF, NVL and NVL2 ## What changes were proposed in this pull request? This patch adds support for a few SQL functions to improve compatibility with other databases: IFNULL, NULLIF, NVL and NVL2. In order to do this, this patch introduced a RuntimeReplaceable expression trait that allows replacing an unevaluable expression in the optimizer before evaluation. Note that the semantics are not completely identical to other databases in esoteric cases. ## How was this patch tested? Added a new test suite SQLCompatibilityFunctionSuite. Closes #12373. Author: Reynold Xin Closes #13084 from rxin/SPARK-14541. (cherry picked from commit eda2800d44843b6478e22d2c99bca4af7e9c9613) Signed-off-by: Yin Huai --- .../catalyst/analysis/FunctionRegistry.scala | 5 +- .../catalyst/analysis/HiveTypeCoercion.scala | 2 + .../sql/catalyst/expressions/Expression.scala | 27 +++++++ .../expressions/nullExpressions.scala | 78 ++++++++++++++++++- .../sql/catalyst/optimizer/Optimizer.scala | 12 +++ .../spark/sql/DataFrameFunctionsSuite.scala | 6 -- .../sql/SQLCompatibilityFunctionSuite.scala | 72 +++++++++++++++++ .../sql/catalyst/ExpressionToSQLSuite.scala | 1 - 8 files changed, 194 insertions(+), 9 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala 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 c459fe587859e..eca837ccf0a78 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 @@ -165,13 +165,16 @@ object FunctionRegistry { expression[Greatest]("greatest"), expression[If]("if"), expression[IsNaN]("isnan"), + expression[IfNull]("ifnull"), expression[IsNull]("isnull"), expression[IsNotNull]("isnotnull"), expression[Least]("least"), expression[CreateMap]("map"), expression[CreateNamedStruct]("named_struct"), expression[NaNvl]("nanvl"), - expression[Coalesce]("nvl"), + expression[NullIf]("nullif"), + expression[Nvl]("nvl"), + expression[Nvl2]("nvl2"), expression[Rand]("rand"), expression[Randn]("randn"), expression[CreateStruct]("struct"), 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 8319ec0a82f5c..537dda60afbfa 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 @@ -521,6 +521,8 @@ object HiveTypeCoercion { NaNvl(l, Cast(r, DoubleType)) case NaNvl(l, r) if l.dataType == FloatType && r.dataType == DoubleType => NaNvl(Cast(l, DoubleType), r) + + case e: RuntimeReplaceable => e.replaceForTypeCoercion() } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index c26faee2f4876..fab163476f5d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -221,6 +221,33 @@ trait Unevaluable extends Expression { } +/** + * An expression that gets replaced at runtime (currently by the optimizer) into a different + * expression for evaluation. This is mainly used to provide compatibility with other databases. + * For example, we use this to support "nvl" by replacing it with "coalesce". + */ +trait RuntimeReplaceable extends Unevaluable { + /** + * Method for concrete implementations to override that specifies how to construct the expression + * that should replace the current one. + */ + def replaceForEvaluation(): Expression + + /** + * Method for concrete implementations to override that specifies how to coerce the input types. + */ + def replaceForTypeCoercion(): Expression + + /** The expression that should be used during evaluation. */ + lazy val replaced: Expression = replaceForEvaluation() + + override def nullable: Boolean = replaced.nullable + override def foldable: Boolean = replaced.foldable + override def dataType: DataType = replaced.dataType + override def checkInputDataTypes(): TypeCheckResult = replaced.checkInputDataTypes() +} + + /** * Expressions that don't have SQL representation should extend this trait. Examples are * `ScalaUDF`, `ScalaUDAF`, and object expressions like `MapObjects` and `Invoke`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 421200e147b7a..641c81b247a2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{HiveTypeCoercion, TypeCheckResult} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -88,6 +88,82 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } +@ExpressionDescription(usage = "_FUNC_(a,b) - Returns b if a is null, or a otherwise.") +case class IfNull(left: Expression, right: Expression) extends RuntimeReplaceable { + override def children: Seq[Expression] = Seq(left, right) + + override def replaceForEvaluation(): Expression = Coalesce(Seq(left, right)) + + override def replaceForTypeCoercion(): Expression = { + if (left.dataType != right.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + copy(left = Cast(left, dtype), right = Cast(right, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + +@ExpressionDescription(usage = "_FUNC_(a,b) - Returns null if a equals to b, or a otherwise.") +case class NullIf(left: Expression, right: Expression) extends RuntimeReplaceable { + override def children: Seq[Expression] = Seq(left, right) + + override def replaceForEvaluation(): Expression = { + If(EqualTo(left, right), Literal.create(null, left.dataType), left) + } + + override def replaceForTypeCoercion(): Expression = { + if (left.dataType != right.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + copy(left = Cast(left, dtype), right = Cast(right, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + +@ExpressionDescription(usage = "_FUNC_(a,b) - Returns b if a is null, or a otherwise.") +case class Nvl(left: Expression, right: Expression) extends RuntimeReplaceable { + override def children: Seq[Expression] = Seq(left, right) + + override def replaceForEvaluation(): Expression = Coalesce(Seq(left, right)) + + override def replaceForTypeCoercion(): Expression = { + if (left.dataType != right.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + copy(left = Cast(left, dtype), right = Cast(right, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + +@ExpressionDescription(usage = "_FUNC_(a,b,c) - Returns b if a is not null, or c otherwise.") +case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression) + extends RuntimeReplaceable { + + override def replaceForEvaluation(): Expression = If(IsNotNull(expr1), expr2, expr3) + + override def children: Seq[Expression] = Seq(expr1, expr2, expr3) + + override def replaceForTypeCoercion(): Expression = { + if (expr2.dataType != expr3.dataType) { + HiveTypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype => + copy(expr2 = Cast(expr2, dtype), expr3 = Cast(expr3, dtype)) + }.getOrElse(this) + } else { + this + } + } +} + + /** * Evaluates to `true` iff it's NaN. */ 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 928ba213b5931..af7532e0c03ec 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 @@ -49,6 +49,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, EliminateSubqueryAliases, + ReplaceExpressions, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), DistinctAggregationRewriter) :: @@ -1511,6 +1512,17 @@ object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { } } +/** + * Finds all [[RuntimeReplaceable]] expressions and replace them with the expressions that can + * be evaluated. This is mainly used to provide compatibility with other databases. + * For example, we use this to support "nvl" by replacing it with "coalesce". + */ +object ReplaceExpressions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case e: RuntimeReplaceable => e.replaced + } +} + /** * Computes the current date and time to make sure we return the same result in a single query. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 746e25a0c3ec5..73d77651a027e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -152,12 +152,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Row("one", "not_one")) } - test("nvl function") { - checkAnswer( - sql("SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null)"), - Row("x", "y", null)) - } - test("misc md5 function") { val df = Seq(("ABC", Array[Byte](1, 2, 3, 4, 5, 6))).toDF("a", "b") checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala new file mode 100644 index 0000000000000..1e3239550fb81 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext + +/** + * A test suite for functions added for compatibility with other databases such as Oracle, MSSQL. + * These functions are typically implemented using the trait + * [[org.apache.spark.sql.catalyst.expressions.RuntimeReplaceable]]. + */ +class SQLCompatibilityFunctionSuite extends QueryTest with SharedSQLContext { + + test("ifnull") { + checkAnswer( + sql("SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null)"), + Row("x", "y", null)) + + // Type coercion + checkAnswer( + sql("SELECT ifnull(1, 2.1d), ifnull(null, 2.1d)"), + Row(1.0, 2.1)) + } + + test("nullif") { + checkAnswer( + sql("SELECT nullif('x', 'x'), nullif('x', 'y')"), + Row(null, "x")) + + // Type coercion + checkAnswer( + sql("SELECT nullif(1, 2.1d), nullif(1, 1.0d)"), + Row(1.0, null)) + } + + test("nvl") { + checkAnswer( + sql("SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null)"), + Row("x", "y", null)) + + // Type coercion + checkAnswer( + sql("SELECT nvl(1, 2.1d), nvl(null, 2.1d)"), + Row(1.0, 2.1)) + } + + test("nvl2") { + checkAnswer( + sql("SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null)"), + Row("y", "x", null)) + + // Type coercion + checkAnswer( + sql("SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d)"), + Row(2.1, 1.0)) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala index 72736ee55b99b..b4eb50e331cf9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala @@ -102,7 +102,6 @@ class ExpressionToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSqlGeneration("SELECT map(1, 'a', 2, 'b')") checkSqlGeneration("SELECT named_struct('c1',1,'c2',2,'c3',3)") checkSqlGeneration("SELECT nanvl(a, 5), nanvl(b, 10), nanvl(d, c) from t2") - checkSqlGeneration("SELECT nvl(null, 1, 2)") checkSqlGeneration("SELECT rand(1)") checkSqlGeneration("SELECT randn(3)") checkSqlGeneration("SELECT struct(1,2,3)") From 7b925e500a2463b42ddb3ad539343ed633eb76cb Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 12 May 2016 22:31:14 -0700 Subject: [PATCH 0210/1470] [SPARK-13866] [SQL] Handle decimal type in CSV inference at CSV data source. ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-13866 This PR adds the support to infer `DecimalType`. Here are the rules between `IntegerType`, `LongType` and `DecimalType`. #### Infering Types 1. `IntegerType` and then `LongType`are tried first. ```scala Int.MaxValue => IntegerType Long.MaxValue => LongType ``` 2. If it fails, try `DecimalType`. ```scala (Long.MaxValue + 1) => DecimalType(20, 0) ``` This does not try to infer this as `DecimalType` when scale is less than 0. 3. if it fails, try `DoubleType` ```scala 0.1 => DoubleType // This is failed to be inferred as `DecimalType` because it has the scale, 1. ``` #### Compatible Types (Merging Types) For merging types, this is the same with JSON data source. If `DecimalType` is not capable, then it becomes `DoubleType` ## How was this patch tested? Unit tests were used and `./dev/run_tests` for code style test. Author: hyukjinkwon Author: Hyukjin Kwon Closes #11724 from HyukjinKwon/SPARK-13866. (cherry picked from commit 51841d77d99a858f8fa1256e923b0364b9b28fa0) Signed-off-by: Davies Liu --- .../datasources/csv/CSVInferSchema.scala | 50 ++++++++++++++++++- sql/core/src/test/resources/decimal.csv | 7 +++ .../datasources/csv/CSVInferSchemaSuite.scala | 13 ++++- .../execution/datasources/csv/CSVSuite.scala | 15 ++++++ 4 files changed, 81 insertions(+), 4 deletions(-) create mode 100644 sql/core/src/test/resources/decimal.csv diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index cfd66af18892b..05c8d8ee15f66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.csv import java.math.BigDecimal -import java.text.{NumberFormat, SimpleDateFormat} +import java.text.NumberFormat import java.util.Locale import scala.util.control.Exception._ @@ -85,6 +85,7 @@ private[csv] object CSVInferSchema { case NullType => tryParseInteger(field, options) case IntegerType => tryParseInteger(field, options) case LongType => tryParseLong(field, options) + case _: DecimalType => tryParseDecimal(field, options) case DoubleType => tryParseDouble(field, options) case TimestampType => tryParseTimestamp(field, options) case BooleanType => tryParseBoolean(field, options) @@ -107,10 +108,28 @@ private[csv] object CSVInferSchema { if ((allCatch opt field.toLong).isDefined) { LongType } else { - tryParseDouble(field, options) + tryParseDecimal(field, options) } } + private def tryParseDecimal(field: String, options: CSVOptions): DataType = { + val decimalTry = allCatch opt { + // `BigDecimal` conversion can fail when the `field` is not a form of number. + val bigDecimal = new BigDecimal(field) + // Because many other formats do not support decimal, it reduces the cases for + // decimals by disallowing values having scale (eg. `1.1`). + if (bigDecimal.scale <= 0) { + // `DecimalType` conversion can fail when + // 1. The precision is bigger than 38. + // 2. scale is bigger than precision. + DecimalType(bigDecimal.precision, bigDecimal.scale) + } else { + tryParseDouble(field, options) + } + } + decimalTry.getOrElse(tryParseDouble(field, options)) + } + private def tryParseDouble(field: String, options: CSVOptions): DataType = { if ((allCatch opt field.toDouble).isDefined) { DoubleType @@ -170,6 +189,33 @@ private[csv] object CSVInferSchema { val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) Some(numericPrecedence(index)) + // These two cases below deal with when `DecimalType` is larger than `IntegralType`. + case (t1: IntegralType, t2: DecimalType) if t2.isWiderThan(t1) => + Some(t2) + case (t1: DecimalType, t2: IntegralType) if t1.isWiderThan(t2) => + Some(t1) + + // These two cases below deal with when `IntegralType` is larger than `DecimalType`. + case (t1: IntegralType, t2: DecimalType) => + findTightestCommonType(DecimalType.forType(t1), t2) + case (t1: DecimalType, t2: IntegralType) => + findTightestCommonType(t1, DecimalType.forType(t2)) + + // Double support larger range than fixed decimal, DecimalType.Maximum should be enough + // in most case, also have better precision. + case (DoubleType, _: DecimalType) | (_: DecimalType, DoubleType) => + Some(DoubleType) + + case (t1: DecimalType, t2: DecimalType) => + val scale = math.max(t1.scale, t2.scale) + val range = math.max(t1.precision - t1.scale, t2.precision - t2.scale) + if (range + scale > 38) { + // DecimalType can't support precision > 38 + Some(DoubleType) + } else { + Some(DecimalType(range + scale, scale)) + } + case _ => None } } diff --git a/sql/core/src/test/resources/decimal.csv b/sql/core/src/test/resources/decimal.csv new file mode 100644 index 0000000000000..870f6aaf1bb4c --- /dev/null +++ b/sql/core/src/test/resources/decimal.csv @@ -0,0 +1,7 @@ +~ decimal field has integer, integer and decimal values. The last value cannot fit to a long +~ long field has integer, long and integer values. +~ double field has double, double and decimal values. +decimal,long,double +1,1,0.1 +1,9223372036854775807,1.0 +92233720368547758070,1,92233720368547758070 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala index daf85be56f3d2..dbe3af49c90c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.csv -import java.text.SimpleDateFormat - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.types._ @@ -35,6 +33,11 @@ class CSVInferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(NullType, "2015-08-20 15:57:00", options) == TimestampType) assert(CSVInferSchema.inferField(NullType, "True", options) == BooleanType) assert(CSVInferSchema.inferField(NullType, "FAlSE", options) == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(CSVInferSchema.inferField(NullType, textValueOne, options) == expectedTypeOne) } test("String fields types are inferred correctly from other types") { @@ -49,6 +52,11 @@ class CSVInferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(LongType, "True", options) == BooleanType) assert(CSVInferSchema.inferField(IntegerType, "FALSE", options) == BooleanType) assert(CSVInferSchema.inferField(TimestampType, "FALSE", options) == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(CSVInferSchema.inferField(IntegerType, textValueOne, options) == expectedTypeOne) } test("Timestamp field types are inferred correctly via custom data format") { @@ -94,6 +102,7 @@ class CSVInferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(DoubleType, "\\N", options) == DoubleType) assert(CSVInferSchema.inferField(TimestampType, "\\N", options) == TimestampType) assert(CSVInferSchema.inferField(BooleanType, "\\N", options) == BooleanType) + assert(CSVInferSchema.inferField(DecimalType(1, 1), "\\N", options) == DecimalType(1, 1)) } test("Merging Nulltypes should yield Nulltype.") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index ae91e0f606ecc..27d6dc9197d27 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -43,6 +43,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { private val commentsFile = "comments.csv" private val disableCommentsFile = "disable_comments.csv" private val boolFile = "bool.csv" + private val decimalFile = "decimal.csv" private val simpleSparseFile = "simple_sparse.csv" private val numbersFile = "numbers.csv" private val datesFile = "dates.csv" @@ -133,6 +134,20 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(result.schema === expectedSchema) } + test("test inferring decimals") { + val result = sqlContext.read + .format("csv") + .option("comment", "~") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(decimalFile)) + val expectedSchema = StructType(List( + StructField("decimal", DecimalType(20, 0), nullable = true), + StructField("long", LongType, nullable = true), + StructField("double", DoubleType, nullable = true))) + assert(result.schema === expectedSchema) + } + test("test with alternative delimiter and quote") { val cars = spark.read .format("csv") From b6b2c613847779daf2eec8122efdb5f2188fba76 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 13 May 2016 08:39:59 +0200 Subject: [PATCH 0211/1470] [SPARK-15188] Add missing thresholds param to NaiveBayes in PySpark ## What changes were proposed in this pull request? Add missing thresholds param to NiaveBayes ## How was this patch tested? doctests Author: Holden Karau Closes #12963 from holdenk/SPARK-15188-add-missing-naive-bayes-param. (cherry picked from commit d1aadea05ab1c7350e46479cc68d08e11916a751) Signed-off-by: Nick Pentreath --- python/pyspark/ml/classification.py | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index c26c2d7fa5bfc..5c11aa71b4599 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -872,7 +872,7 @@ def featureImportances(self): @inherit_doc class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, - HasRawPredictionCol, JavaMLWritable, JavaMLReadable): + HasRawPredictionCol, HasThresholds, JavaMLWritable, JavaMLReadable): """ Naive Bayes Classifiers. It supports both Multinomial and Bernoulli NB. `Multinomial NB @@ -918,6 +918,11 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H True >>> model.theta == model2.theta True + >>> nb = nb.setThresholds([0.01, 10.00]) + >>> model3 = nb.fit(df) + >>> result = model3.transform(test0).head() + >>> result.prediction + 0.0 .. versionadded:: 1.5.0 """ @@ -931,11 +936,11 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, - modelType="multinomial"): + modelType="multinomial", thresholds=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, \ - modelType="multinomial") + modelType="multinomial", thresholds=None) """ super(NaiveBayes, self).__init__() self._java_obj = self._new_java_obj( @@ -948,11 +953,11 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred @since("1.5.0") def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, - modelType="multinomial"): + modelType="multinomial", thresholds=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, \ - modelType="multinomial") + modelType="multinomial", thresholds=None) Sets params for Naive Bayes. """ kwargs = self.setParams._input_kwargs From 0076bf085b0fbda8a7811e2c961451dfddcd73f8 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 13 May 2016 08:52:06 +0200 Subject: [PATCH 0212/1470] [MINOR][PYSPARK] update _shared_params_code_gen.py ## What changes were proposed in this pull request? 1, add arg-checkings for `tol` and `stepSize` to keep in line with `SharedParamsCodeGen.scala` 2, fix one typo ## How was this patch tested? local build Author: Zheng RuiFeng Closes #12996 from zhengruifeng/py_args_checking. (cherry picked from commit 87d69a01f027aa18718827f94f921b4a1eaa78a5) Signed-off-by: Nick Pentreath --- python/pyspark/ml/clustering.py | 2 +- python/pyspark/ml/param/_shared_params_code_gen.py | 6 +++--- python/pyspark/ml/param/shared.py | 12 ++++++------ 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index b90daf16e2f71..921633164bacb 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -517,7 +517,7 @@ def describeTopics(self, maxTermsPerTopic=10): def estimatedDocConcentration(self): """ Value for :py:attr:`LDA.docConcentration` estimated from data. - If Online LDA was used and :py:attr::`LDA.optimizeDocConcentration` was set to false, + If Online LDA was used and :py:attr:`LDA.optimizeDocConcentration` was set to false, then this returns the fixed (given) value for the :py:attr:`LDA.docConcentration` parameter. """ return self._call_java("estimatedDocConcentration") diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 12f14534f4b88..c32dcc467d492 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -124,12 +124,12 @@ def get$Name(self): "E.g. 10 means that the cache will get checkpointed every 10 iterations.", None, "TypeConverters.toInt"), ("seed", "random seed.", "hash(type(self).__name__)", "TypeConverters.toInt"), - ("tol", "the convergence tolerance for iterative algorithms.", None, + ("tol", "the convergence tolerance for iterative algorithms (>= 0).", None, "TypeConverters.toFloat"), - ("stepSize", "Step size to be used for each iteration of optimization.", None, + ("stepSize", "Step size to be used for each iteration of optimization (>= 0).", None, "TypeConverters.toFloat"), ("handleInvalid", "how to handle invalid entries. Options are skip (which will filter " + - "out rows with bad values), or error (which will throw an errror). More options may be " + + "out rows with bad values), or error (which will throw an error). More options may be " + "added later.", None, "TypeConverters.toString"), ("elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, " + "the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", "0.0", diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 9ed63e48ae40d..c5ccf81540d58 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -328,10 +328,10 @@ def getSeed(self): class HasTol(Params): """ - Mixin for param tol: the convergence tolerance for iterative algorithms. + Mixin for param tol: the convergence tolerance for iterative algorithms (>= 0). """ - tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms.", typeConverter=TypeConverters.toFloat) + tol = Param(Params._dummy(), "tol", "the convergence tolerance for iterative algorithms (>= 0).", typeConverter=TypeConverters.toFloat) def __init__(self): super(HasTol, self).__init__() @@ -351,10 +351,10 @@ def getTol(self): class HasStepSize(Params): """ - Mixin for param stepSize: Step size to be used for each iteration of optimization. + Mixin for param stepSize: Step size to be used for each iteration of optimization (>= 0). """ - stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization.", typeConverter=TypeConverters.toFloat) + stepSize = Param(Params._dummy(), "stepSize", "Step size to be used for each iteration of optimization (>= 0).", typeConverter=TypeConverters.toFloat) def __init__(self): super(HasStepSize, self).__init__() @@ -374,10 +374,10 @@ def getStepSize(self): class HasHandleInvalid(Params): """ - Mixin for param handleInvalid: how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later. + Mixin for param handleInvalid: how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an error). More options may be added later. """ - handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.", typeConverter=TypeConverters.toString) + handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an error). More options may be added later.", typeConverter=TypeConverters.toString) def __init__(self): super(HasHandleInvalid, self).__init__() From 7affde25ba75f83aef33a3355900220b57243ae6 Mon Sep 17 00:00:00 2001 From: sethah Date: Fri, 13 May 2016 09:01:20 +0200 Subject: [PATCH 0213/1470] [SPARK-15181][ML][PYSPARK] Python API for GLR summaries. ## What changes were proposed in this pull request? This patch adds a python API for generalized linear regression summaries (training and test). This helps provide feature parity for Python GLMs. ## How was this patch tested? Added a unit test to `pyspark.ml.tests` Author: sethah Closes #12961 from sethah/GLR_summary. (cherry picked from commit 5b849766ab080c91864ed06ebbfd82ad978d5e4c) Signed-off-by: Nick Pentreath --- .../GeneralizedLinearRegression.scala | 18 +- python/pyspark/ml/regression.py | 201 +++++++++++++++++- python/pyspark/ml/tests.py | 39 +++- 3 files changed, 247 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index c294ef31f90de..05fffa0d97e6b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -848,7 +848,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( import GeneralizedLinearRegression._ /** - * Field in "predictions" which gives the prediction value of each instance. + * Field in "predictions" which gives the predicted value of each instance. * This is set to a new column name if the original model's `predictionCol` is not set. */ @Since("2.0.0") @@ -870,7 +870,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( protected val model: GeneralizedLinearRegressionModel = origModel.copy(ParamMap.empty).setPredictionCol(predictionCol) - /** predictions output by the model's `transform` method */ + /** Predictions output by the model's `transform` method. */ @Since("2.0.0") @transient val predictions: DataFrame = model.transform(dataset) private[regression] lazy val family: Family = Family.fromName(model.getFamily) @@ -880,10 +880,10 @@ class GeneralizedLinearRegressionSummary private[regression] ( family.defaultLink } - /** Number of instances in DataFrame predictions */ + /** Number of instances in DataFrame predictions. */ private[regression] lazy val numInstances: Long = predictions.count() - /** The numeric rank of the fitted linear model */ + /** The numeric rank of the fitted linear model. */ @Since("2.0.0") lazy val rank: Long = if (model.getFitIntercept) { model.coefficients.size + 1 @@ -891,17 +891,17 @@ class GeneralizedLinearRegressionSummary private[regression] ( model.coefficients.size } - /** Degrees of freedom */ + /** Degrees of freedom. */ @Since("2.0.0") lazy val degreesOfFreedom: Long = { numInstances - rank } - /** The residual degrees of freedom */ + /** The residual degrees of freedom. */ @Since("2.0.0") lazy val residualDegreeOfFreedom: Long = degreesOfFreedom - /** The residual degrees of freedom for the null model */ + /** The residual degrees of freedom for the null model. */ @Since("2.0.0") lazy val residualDegreeOfFreedomNull: Long = if (model.getFitIntercept) { numInstances - 1 @@ -944,7 +944,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( } /** - * Get the default residuals(deviance residuals) of the fitted model. + * Get the default residuals (deviance residuals) of the fitted model. */ @Since("2.0.0") def residuals(): DataFrame = devianceResiduals @@ -1000,7 +1000,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( /** * The dispersion of the fitted model. * It is taken as 1.0 for the "binomial" and "poisson" families, and otherwise - * estimated by the residual Pearson's Chi-Squared statistic(which is defined as + * estimated by the residual Pearson's Chi-Squared statistic (which is defined as * sum of the squares of the Pearson residuals) divided by the residual degrees of freedom. */ @Since("2.0.0") diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 0d0eb8ae4618a..fcdc29e69b1d0 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -29,6 +29,7 @@ 'DecisionTreeRegressor', 'DecisionTreeRegressionModel', 'GBTRegressor', 'GBTRegressionModel', 'GeneralizedLinearRegression', 'GeneralizedLinearRegressionModel', + 'GeneralizedLinearRegressionSummary', 'GeneralizedLinearRegressionTrainingSummary', 'IsotonicRegression', 'IsotonicRegressionModel', 'LinearRegression', 'LinearRegressionModel', 'LinearRegressionSummary', 'LinearRegressionTrainingSummary', @@ -1283,7 +1284,7 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha family = Param(Params._dummy(), "family", "The name of family which is a description of " + "the error distribution to be used in the model. Supported options: " + - "gaussian(default), binomial, poisson and gamma.", + "gaussian (default), binomial, poisson and gamma.", typeConverter=TypeConverters.toString) link = Param(Params._dummy(), "link", "The name of link function which provides the " + "relationship between the linear predictor and the mean of the distribution " + @@ -1377,6 +1378,204 @@ def intercept(self): """ return self._call_java("intercept") + @property + @since("2.0.0") + def summary(self): + """ + Gets summary (e.g. residuals, deviance, pValues) of model on + training set. An exception is thrown if + `trainingSummary is None`. + """ + java_glrt_summary = self._call_java("summary") + return GeneralizedLinearRegressionTrainingSummary(java_glrt_summary) + + @property + @since("2.0.0") + def hasSummary(self): + """ + Indicates whether a training summary exists for this model + instance. + """ + return self._call_java("hasSummary") + + @since("2.0.0") + def evaluate(self, dataset): + """ + Evaluates the model on a test dataset. + + :param dataset: + Test dataset to evaluate model on, where dataset is an + instance of :py:class:`pyspark.sql.DataFrame` + """ + if not isinstance(dataset, DataFrame): + raise ValueError("dataset must be a DataFrame but got %s." % type(dataset)) + java_glr_summary = self._call_java("evaluate", dataset) + return GeneralizedLinearRegressionSummary(java_glr_summary) + + +class GeneralizedLinearRegressionSummary(JavaWrapper): + """ + .. note:: Experimental + + Generalized linear regression results evaluated on a dataset. + + .. versionadded:: 2.0.0 + """ + + @property + @since("2.0.0") + def predictions(self): + """ + Predictions output by the model's `transform` method. + """ + return self._call_java("predictions") + + @property + @since("2.0.0") + def predictionCol(self): + """ + Field in :py:attr:`predictions` which gives the predicted value of each instance. + This is set to a new column name if the original model's `predictionCol` is not set. + """ + return self._call_java("predictionCol") + + @property + @since("2.0.0") + def rank(self): + """ + The numeric rank of the fitted linear model. + """ + return self._call_java("rank") + + @property + @since("2.0.0") + def degreesOfFreedom(self): + """ + Degrees of freedom. + """ + return self._call_java("degreesOfFreedom") + + @property + @since("2.0.0") + def residualDegreeOfFreedom(self): + """ + The residual degrees of freedom. + """ + return self._call_java("residualDegreeOfFreedom") + + @property + @since("2.0.0") + def residualDegreeOfFreedomNull(self): + """ + The residual degrees of freedom for the null model. + """ + return self._call_java("residualDegreeOfFreedomNull") + + @since("2.0.0") + def residuals(self, residualsType="deviance"): + """ + Get the residuals of the fitted model by type. + + :param residualsType: The type of residuals which should be returned. + Supported options: deviance (default), pearson, working, and response. + """ + return self._call_java("residuals", residualsType) + + @property + @since("2.0.0") + def nullDeviance(self): + """ + The deviance for the null model. + """ + return self._call_java("nullDeviance") + + @property + @since("2.0.0") + def deviance(self): + """ + The deviance for the fitted model. + """ + return self._call_java("deviance") + + @property + @since("2.0.0") + def dispersion(self): + """ + The dispersion of the fitted model. + It is taken as 1.0 for the "binomial" and "poisson" families, and otherwise + estimated by the residual Pearson's Chi-Squared statistic (which is defined as + sum of the squares of the Pearson residuals) divided by the residual degrees of freedom. + """ + return self._call_java("dispersion") + + @property + @since("2.0.0") + def aic(self): + """ + Akaike's "An Information Criterion"(AIC) for the fitted model. + """ + return self._call_java("aic") + + +@inherit_doc +class GeneralizedLinearRegressionTrainingSummary(GeneralizedLinearRegressionSummary): + """ + .. note:: Experimental + + Generalized linear regression training results. + + .. versionadded:: 2.0.0 + """ + + @property + @since("2.0.0") + def numIterations(self): + """ + Number of training iterations. + """ + return self._call_java("numIterations") + + @property + @since("2.0.0") + def solver(self): + """ + The numeric solver used for training. + """ + return self._call_java("solver") + + @property + @since("2.0.0") + def coefficientStandardErrors(self): + """ + Standard error of estimated coefficients and intercept. + + If :py:attr:`GeneralizedLinearRegression.fitIntercept` is set to True, + then the last element returned corresponds to the intercept. + """ + return self._call_java("coefficientStandardErrors") + + @property + @since("2.0.0") + def tValues(self): + """ + T-statistic of estimated coefficients and intercept. + + If :py:attr:`GeneralizedLinearRegression.fitIntercept` is set to True, + then the last element returned corresponds to the intercept. + """ + return self._call_java("tValues") + + @property + @since("2.0.0") + def pValues(self): + """ + Two-sided p-value of estimated coefficients and intercept. + + If :py:attr:`GeneralizedLinearRegression.fitIntercept` is set to True, + then the last element returned corresponds to the intercept. + """ + return self._call_java("pValues") + if __name__ == "__main__": import doctest diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 49d3a4a332fd1..8e56b0d6fff05 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -52,7 +52,8 @@ from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed from pyspark.ml.recommendation import ALS -from pyspark.ml.regression import LinearRegression, DecisionTreeRegressor +from pyspark.ml.regression import LinearRegression, DecisionTreeRegressor, \ + GeneralizedLinearRegression from pyspark.ml.tuning import * from pyspark.ml.wrapper import JavaParams from pyspark.mllib.common import _java2py @@ -909,6 +910,42 @@ def test_linear_regression_summary(self): sameSummary = model.evaluate(df) self.assertAlmostEqual(sameSummary.explainedVariance, s.explainedVariance) + def test_glr_summary(self): + from pyspark.mllib.linalg import Vectors + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], []))], + ["label", "weight", "features"]) + glr = GeneralizedLinearRegression(family="gaussian", link="identity", weightCol="weight", + fitIntercept=False) + model = glr.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertEqual(s.numIterations, 1) # this should default to a single iteration of WLS + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.predictionCol, "prediction") + self.assertTrue(isinstance(s.residuals(), DataFrame)) + self.assertTrue(isinstance(s.residuals("pearson"), DataFrame)) + coefStdErr = s.coefficientStandardErrors + self.assertTrue(isinstance(coefStdErr, list) and isinstance(coefStdErr[0], float)) + tValues = s.tValues + self.assertTrue(isinstance(tValues, list) and isinstance(tValues[0], float)) + pValues = s.pValues + self.assertTrue(isinstance(pValues, list) and isinstance(pValues[0], float)) + self.assertEqual(s.degreesOfFreedom, 1) + self.assertEqual(s.residualDegreeOfFreedom, 1) + self.assertEqual(s.residualDegreeOfFreedomNull, 2) + self.assertEqual(s.rank, 1) + self.assertTrue(isinstance(s.solver, basestring)) + self.assertTrue(isinstance(s.aic, float)) + self.assertTrue(isinstance(s.deviance, float)) + self.assertTrue(isinstance(s.nullDeviance, float)) + self.assertTrue(isinstance(s.dispersion, float)) + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertAlmostEqual(sameSummary.deviance, s.deviance) + def test_logistic_regression_summary(self): from pyspark.mllib.linalg import Vectors df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), From 86b8f8a9acf88998b10ad86e6438c4ebff17f948 Mon Sep 17 00:00:00 2001 From: BenFradet Date: Fri, 13 May 2016 09:08:04 +0200 Subject: [PATCH 0214/1470] [SPARK-13961][ML] spark.ml ChiSqSelector and RFormula should support other numeric types for label ## What changes were proposed in this pull request? Made ChiSqSelector and RFormula accept all numeric types for label ## How was this patch tested? Unit tests Author: BenFradet Closes #12467 from BenFradet/SPARK-13961. (cherry picked from commit 31f1aebbeb77b4eb1080f22c9bece7fafd8022f8) Signed-off-by: Nick Pentreath --- .../spark/ml/feature/ChiSqSelector.scala | 4 +-- .../apache/spark/ml/feature/RFormula.scala | 4 +-- .../DecisionTreeClassifierSuite.scala | 2 +- .../classification/GBTClassifierSuite.scala | 2 +- .../LogisticRegressionSuite.scala | 2 +- .../MultilayerPerceptronClassifierSuite.scala | 2 +- .../ml/classification/NaiveBayesSuite.scala | 2 +- .../ml/classification/OneVsRestSuite.scala | 2 +- .../RandomForestClassifierSuite.scala | 2 +- .../spark/ml/feature/ChiSqSelectorSuite.scala | 10 ++++++- .../spark/ml/feature/RFormulaSuite.scala | 30 +++++++++++++++---- .../AFTSurvivalRegressionSuite.scala | 2 +- .../DecisionTreeRegressorSuite.scala | 2 +- .../ml/regression/GBTRegressorSuite.scala | 2 +- .../GeneralizedLinearRegressionSuite.scala | 2 +- .../regression/IsotonicRegressionSuite.scala | 2 +- .../ml/regression/LinearRegressionSuite.scala | 2 +- .../RandomForestRegressorSuite.scala | 2 +- .../apache/spark/ml/util/MLTestingUtils.scala | 4 +-- 19 files changed, 53 insertions(+), 27 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index cfecae7e0b152..29f55a7f715ca 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -80,7 +80,7 @@ final class ChiSqSelector(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): ChiSqSelectorModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(labelCol), $(featuresCol)).rdd.map { + val input = dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } @@ -90,7 +90,7 @@ final class ChiSqSelector(override val uid: String) override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(labelCol)) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 5219680be2dc8..a2f3d44132d1d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -256,8 +256,8 @@ class RFormulaModel private[feature]( val columnNames = schema.map(_.name) require(!columnNames.contains($(featuresCol)), "Features column already exists.") require( - !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType == DoubleType, - "Label column already exists and is not of type DoubleType.") + !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType.isInstanceOf[NumericType], + "Label column already exists and is not of type NumericType.") } @Since("2.0.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index f94d336df544d..91a947f44bc31 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -337,7 +337,7 @@ class DecisionTreeClassifierSuite test("should support all NumericType labels and not support other types") { val dt = new DecisionTreeClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[DecisionTreeClassificationModel, DecisionTreeClassifier]( - dt, isClassification = true, spark) { (expected, actual) => + dt, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index c9453aaec255d..5a5e5c15fc59c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -106,7 +106,7 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext test("should support all NumericType labels and not support other types") { val gbt = new GBTClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[GBTClassificationModel, GBTClassifier]( - gbt, isClassification = true, spark) { (expected, actual) => + gbt, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index cb4d087ce5bc2..f127aa217c94d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -938,7 +938,7 @@ class LogisticRegressionSuite test("should support all NumericType labels and not support other types") { val lr = new LogisticRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[LogisticRegressionModel, LogisticRegression]( - lr, isClassification = true, spark) { (expected, actual) => + lr, spark) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients.toArray === actual.coefficients.toArray) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index 876e047db54cf..d5282e07d65c2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -169,7 +169,7 @@ class MultilayerPerceptronClassifierSuite val mpc = new MultilayerPerceptronClassifier().setLayers(layers).setMaxIter(1) MLTestingUtils.checkNumericTypes[ MultilayerPerceptronClassificationModel, MultilayerPerceptronClassifier]( - mpc, isClassification = true, spark) { (expected, actual) => + mpc, spark) { (expected, actual) => assert(expected.layers === actual.layers) assert(expected.weights === actual.weights) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 15d0358c3fc03..2a05c446e5169 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -188,7 +188,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa test("should support all NumericType labels and not support other types") { val nb = new NaiveBayes() MLTestingUtils.checkNumericTypes[NaiveBayesModel, NaiveBayes]( - nb, isClassification = true, spark) { (expected, actual) => + nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 005d609307fbf..5044d40998d66 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -228,7 +228,7 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with Defau test("should support all NumericType labels and not support other types") { val ovr = new OneVsRest().setClassifier(new LogisticRegression().setMaxIter(1)) MLTestingUtils.checkNumericTypes[OneVsRestModel, OneVsRest]( - ovr, isClassification = true, spark) { (expected, actual) => + ovr, spark) { (expected, actual) => val expectedModels = expected.models.map(m => m.asInstanceOf[LogisticRegressionModel]) val actualModels = actual.models.map(m => m.asInstanceOf[LogisticRegressionModel]) assert(expectedModels.length === actualModels.length) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index 97f3feacca077..8002a2f4f29e1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -189,7 +189,7 @@ class RandomForestClassifierSuite test("should support all NumericType labels and not support other types") { val rf = new RandomForestClassifier().setMaxDepth(1) MLTestingUtils.checkNumericTypes[RandomForestClassificationModel, RandomForestClassifier]( - rf, isClassification = true, spark) { (expected, actual) => + rf, spark) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index 4c6d9c5e26097..4fcc9745b738c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint @@ -81,4 +81,12 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext val newInstance = testDefaultReadWrite(instance) assert(newInstance.selectedFeatures === instance.selectedFeatures) } + + test("should support all NumericType labels and not support other types") { + val css = new ChiSqSelector() + MLTestingUtils.checkNumericTypes[ChiSqSelectorModel, ChiSqSelector]( + css, spark) { (expected, actual) => + assert(expected.selectedFeatures === actual.selectedFeatures) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index 46e7495297a6b..c623a6210bdae 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute._ import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.Row +import org.apache.spark.sql.types.DoubleType class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("params") { @@ -68,9 +68,9 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul assert(resultSchema.toString == model.transform(original).schema.toString) } - test("label column already exists but is not double type") { + test("label column already exists but is not numeric type") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") - val original = spark.createDataFrame(Seq((0, 1), (2, 2))).toDF("x", "y") + val original = spark.createDataFrame(Seq((0, true), (2, false))).toDF("x", "y") val model = formula.fit(original) intercept[IllegalArgumentException] { model.transformSchema(original.schema) @@ -134,7 +134,6 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul ).toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val resultSchema = model.transformSchema(original.schema) val expected = spark.createDataFrame( Seq( ("male", "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), @@ -188,7 +187,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul "vec2", Array[Attribute]( NumericAttribute.defaultAttr, - NumericAttribute.defaultAttr)).toMetadata + NumericAttribute.defaultAttr)).toMetadata() val original = base.select(base.col("id"), base.col("vec").as("vec2", metadata)) val model = formula.fit(original) val result = model.transform(original) @@ -309,4 +308,23 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul val newModel = testDefaultReadWrite(model) checkModelData(model, newModel) } + + test("should support all NumericType labels") { + val formula = new RFormula().setFormula("label ~ features") + .setLabelCol("x") + .setFeaturesCol("y") + val dfs = MLTestingUtils.genRegressionDFWithNumericLabelCol(spark) + val expected = formula.fit(dfs(DoubleType)) + val actuals = dfs.keys.filter(_ != DoubleType).map(t => formula.fit(dfs(t))) + actuals.foreach { actual => + assert(expected.pipelineModel.stages.length === actual.pipelineModel.stages.length) + expected.pipelineModel.stages.zip(actual.pipelineModel.stages).foreach { + case (exTransformer, acTransformer) => + assert(exTransformer.params === acTransformer.params) + } + assert(expected.resolvedFormula.label === actual.resolvedFormula.label) + assert(expected.resolvedFormula.terms === actual.resolvedFormula.terms) + assert(expected.resolvedFormula.hasIntercept === actual.resolvedFormula.hasIntercept) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index f8fc775676c0c..e4772df622d18 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -356,7 +356,7 @@ class AFTSurvivalRegressionSuite test("should support all NumericType labels") { val aft = new AFTSurvivalRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[AFTSurvivalRegressionModel, AFTSurvivalRegression]( - aft, isClassification = false, spark) { (expected, actual) => + aft, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index d9f26ad8dc93c..2d30cbf36766b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -120,7 +120,7 @@ class DecisionTreeRegressorSuite test("should support all NumericType labels and not support other types") { val dt = new DecisionTreeRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[DecisionTreeRegressionModel, DecisionTreeRegressor]( - dt, isClassification = false, spark) { (expected, actual) => + dt, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index f6ea5bb741d41..ac833b833d7d8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -115,7 +115,7 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext test("should support all NumericType labels and not support other types") { val gbt = new GBTRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[GBTRegressionModel, GBTRegressor]( - gbt, isClassification = false, spark) { (expected, actual) => + gbt, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index 161f8c80f8df5..3d9aeb8c0a2d0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -1021,7 +1021,7 @@ class GeneralizedLinearRegressionSuite val glr = new GeneralizedLinearRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[ GeneralizedLinearRegressionModel, GeneralizedLinearRegression]( - glr, isClassification = false, spark) { (expected, actual) => + glr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala index 9bf7542b1259b..bed4978b25b37 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala @@ -184,7 +184,7 @@ class IsotonicRegressionSuite test("should support all NumericType labels and not support other types") { val ir = new IsotonicRegression() MLTestingUtils.checkNumericTypes[IsotonicRegressionModel, IsotonicRegression]( - ir, isClassification = false, spark) { (expected, actual) => + ir, spark, isClassification = false) { (expected, actual) => assert(expected.boundaries === actual.boundaries) assert(expected.predictions === actual.predictions) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index 10f547b67375c..a98227d2c14fb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -1010,7 +1010,7 @@ class LinearRegressionSuite test("should support all NumericType labels and not support other types") { val lr = new LinearRegression().setMaxIter(1) MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( - lr, isClassification = false, spark) { (expected, actual) => + lr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index 72f3c65eb8c7d..7a3a3698f950d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -98,7 +98,7 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex test("should support all NumericType labels and not support other types") { val rf = new RandomForestRegressor().setMaxDepth(1) MLTestingUtils.checkNumericTypes[RandomForestRegressionModel, RandomForestRegressor]( - rf, isClassification = false, spark) { (expected, actual) => + rf, spark, isClassification = false) { (expected, actual) => TreeTests.checkEqual(expected, actual) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index 4fe473bbacd42..ad7d2c9b8d40c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -37,8 +37,8 @@ object MLTestingUtils extends SparkFunSuite { def checkNumericTypes[M <: Model[M], T <: Estimator[M]]( estimator: T, - isClassification: Boolean, - spark: SparkSession)(check: (M, M) => Unit): Unit = { + spark: SparkSession, + isClassification: Boolean = true)(check: (M, M) => Unit): Unit = { val dfs = if (isClassification) { genClassifDFWithNumericLabelCol(spark) } else { From 43570c57630bb03aae0b42a4cae65c8323ebfc99 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 13 May 2016 00:15:39 -0700 Subject: [PATCH 0215/1470] [SPARK-15310][SQL] Rename HiveTypeCoercion -> TypeCoercion ## What changes were proposed in this pull request? We originally designed the type coercion rules to match Hive, but over time we have diverged. It does not make sense to call it HiveTypeCoercion anymore. This patch renames it TypeCoercion. ## How was this patch tested? Updated unit tests to reflect the rename. Author: Reynold Xin Closes #13091 from rxin/SPARK-15310. (cherry picked from commit e1dc853737fc1739fbb5377ffe31fb2d89935b1f) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/Analyzer.scala | 6 +- ...eTypeCoercion.scala => TypeCoercion.scala} | 6 +- .../expressions/nullExpressions.scala | 10 +-- ...ionSuite.scala => TypeCoercionSuite.scala} | 72 +++++++++---------- .../datasources/csv/CSVInferSchema.scala | 6 +- .../datasources/json/InferSchema.scala | 4 +- 6 files changed, 50 insertions(+), 54 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/{HiveTypeCoercion.scala => TypeCoercion.scala} (99%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/{HiveTypeCoercionSuite.scala => TypeCoercionSuite.scala} (92%) 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 ce941e3df3e82..1313a011c69c2 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 @@ -105,7 +105,7 @@ class Analyzer( GlobalAggregates :: ResolveAggregateFunctions :: TimeWindowing :: - HiveTypeCoercion.typeCoercionRules ++ + TypeCoercion.typeCoercionRules ++ extendedResolutionRules : _*), Batch("Nondeterministic", Once, PullOutNondeterministic), @@ -1922,8 +1922,8 @@ class Analyzer( } private def illegalNumericPrecedence(from: DataType, to: DataType): Boolean = { - val fromPrecedence = HiveTypeCoercion.numericPrecedence.indexOf(from) - val toPrecedence = HiveTypeCoercion.numericPrecedence.indexOf(to) + val fromPrecedence = TypeCoercion.numericPrecedence.indexOf(from) + val toPrecedence = TypeCoercion.numericPrecedence.indexOf(to) toPrecedence > 0 && fromPrecedence > toPrecedence } 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/TypeCoercion.scala similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 537dda60afbfa..91bdcc3b09a34 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/TypeCoercion.scala @@ -24,7 +24,6 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ @@ -34,9 +33,6 @@ import org.apache.spark.sql.types._ * A collection of [[Rule]] that can be used to coerce differing types that participate in * operations into compatible ones. * - * Most of these rules are based on Hive semantics, but they do not introduce any dependencies on - * the hive codebase. - * * Notes about type widening / tightest common types: Broadly, there are two cases when we need * to widen data types (e.g. union, binary comparison). In case 1, we are looking for a common * data type for two or more data types, and in this case no loss of precision is allowed. Examples @@ -46,7 +42,7 @@ import org.apache.spark.sql.types._ * double's range is larger than decimal, and yet decimal is more precise than double, but in * union we would cast the decimal into double). */ -object HiveTypeCoercion { +object TypeCoercion { val typeCoercionRules = PropagateTypes :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 641c81b247a2d..523fb053972dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{HiveTypeCoercion, TypeCheckResult} +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -96,7 +96,7 @@ case class IfNull(left: Expression, right: Expression) extends RuntimeReplaceabl override def replaceForTypeCoercion(): Expression = { if (left.dataType != right.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => copy(left = Cast(left, dtype), right = Cast(right, dtype)) }.getOrElse(this) } else { @@ -116,7 +116,7 @@ case class NullIf(left: Expression, right: Expression) extends RuntimeReplaceabl override def replaceForTypeCoercion(): Expression = { if (left.dataType != right.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => copy(left = Cast(left, dtype), right = Cast(right, dtype)) }.getOrElse(this) } else { @@ -134,7 +134,7 @@ case class Nvl(left: Expression, right: Expression) extends RuntimeReplaceable { override def replaceForTypeCoercion(): Expression = { if (left.dataType != right.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => copy(left = Cast(left, dtype), right = Cast(right, dtype)) }.getOrElse(this) } else { @@ -154,7 +154,7 @@ case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression) override def replaceForTypeCoercion(): Expression = { if (expr2.dataType != expr3.dataType) { - HiveTypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype => copy(expr2 = Cast(expr2, dtype), expr3 = Cast(expr3, dtype)) }.getOrElse(this) } else { 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/TypeCoercionSuite.scala similarity index 92% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index b591861ac094c..7435399b1492a 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/TypeCoercionSuite.scala @@ -26,11 +26,11 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval -class HiveTypeCoercionSuite extends PlanTest { +class TypeCoercionSuite extends PlanTest { test("eligible implicit type cast") { def shouldCast(from: DataType, to: AbstractDataType, expected: DataType): Unit = { - val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) + val got = TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) assert(got.map(_.dataType) == Option(expected), s"Failed to cast $from to $to") } @@ -101,7 +101,7 @@ class HiveTypeCoercionSuite extends PlanTest { test("ineligible implicit type cast") { def shouldNotCast(from: DataType, to: AbstractDataType): Unit = { - val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) + val got = TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) assert(got.isEmpty, s"Should not be able to cast $from to $to, but got $got") } @@ -129,11 +129,11 @@ class HiveTypeCoercionSuite extends PlanTest { test("tightest common bound for types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { - var found = HiveTypeCoercion.findTightestCommonTypeOfTwo(t1, t2) + var found = TypeCoercion.findTightestCommonTypeOfTwo(t1, t2) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found") // Test both directions to make sure the widening is symmetric. - found = HiveTypeCoercion.findTightestCommonTypeOfTwo(t2, t1) + found = TypeCoercion.findTightestCommonTypeOfTwo(t2, t1) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found") } @@ -206,31 +206,31 @@ class HiveTypeCoercionSuite extends PlanTest { } test("cast NullType for expressions that implement ExpectsInputTypes") { - import HiveTypeCoercionSuite._ + import TypeCoercionSuite._ - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeUnaryExpression(Literal.create(null, NullType)), AnyTypeUnaryExpression(Literal.create(null, NullType))) - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, NumericTypeUnaryExpression(Literal.create(null, NullType)), NumericTypeUnaryExpression(Literal.create(null, DoubleType))) } test("cast NullType for binary operators") { - import HiveTypeCoercionSuite._ + import TypeCoercionSuite._ - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)), AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType))) - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, NumericTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)), NumericTypeBinaryOperator(Literal.create(null, DoubleType), Literal.create(null, DoubleType))) } test("coalesce casts") { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, Coalesce(Literal(1.0) :: Literal(1) :: Literal.create(1.0, FloatType) @@ -239,7 +239,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal(1), DoubleType) :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, Coalesce(Literal(1L) :: Literal(1) :: Literal(new java.math.BigDecimal("1000000000000000000000")) @@ -251,7 +251,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("CreateArray casts") { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateArray(Literal(1.0) :: Literal(1) :: Literal.create(1.0, FloatType) @@ -261,7 +261,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateArray(Literal(1.0) :: Literal(1) :: Literal("a") @@ -274,7 +274,7 @@ class HiveTypeCoercionSuite extends PlanTest { test("CreateMap casts") { // type coercion for map keys - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") :: Literal.create(2.0, FloatType) @@ -286,7 +286,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Literal("b") :: Nil)) // type coercion for map values - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") :: Literal(2) @@ -298,7 +298,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal(3.0), StringType) :: Nil)) // type coercion for both map keys and values - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") :: Literal(2.0) @@ -313,7 +313,7 @@ class HiveTypeCoercionSuite extends PlanTest { test("greatest/least cast") { for (operator <- Seq[(Seq[Expression] => Expression)](Greatest, Least)) { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1.0) :: Literal(1) :: Literal.create(1.0, FloatType) @@ -322,7 +322,7 @@ class HiveTypeCoercionSuite extends PlanTest { :: Cast(Literal(1), DoubleType) :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1L) :: Literal(1) :: Literal(new java.math.BigDecimal("1000000000000000000000")) @@ -335,19 +335,19 @@ class HiveTypeCoercionSuite extends PlanTest { } test("nanvl casts") { - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, NaNvl(Literal.create(1.0, FloatType), Literal.create(1.0, DoubleType)), NaNvl(Cast(Literal.create(1.0, FloatType), DoubleType), Literal.create(1.0, DoubleType))) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, FloatType)), NaNvl(Literal.create(1.0, DoubleType), Cast(Literal.create(1.0, FloatType), DoubleType))) - ruleTest(HiveTypeCoercion.FunctionArgumentConversion, + ruleTest(TypeCoercion.FunctionArgumentConversion, NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, DoubleType)), NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, DoubleType))) } test("type coercion for If") { - val rule = HiveTypeCoercion.IfCoercion + val rule = TypeCoercion.IfCoercion ruleTest(rule, If(Literal(true), Literal(1), Literal(1L)), @@ -367,20 +367,20 @@ class HiveTypeCoercionSuite extends PlanTest { } test("type coercion for CaseKeyWhen") { - ruleTest(HiveTypeCoercion.ImplicitTypeCasts, + ruleTest(TypeCoercion.ImplicitTypeCasts, CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Cast(Literal(1.toShort), IntegerType), Seq(Literal(1), Literal("a"))) ) - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(TypeCoercion.CaseWhenCoercion, CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))) ) - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(TypeCoercion.CaseWhenCoercion, CaseWhen(Seq((Literal(true), Literal(1.2))), Literal.create(1, DecimalType(7, 2))), CaseWhen(Seq((Literal(true), Literal(1.2))), Cast(Literal.create(1, DecimalType(7, 2)), DoubleType)) ) - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(TypeCoercion.CaseWhenCoercion, CaseWhen(Seq((Literal(true), Literal(100L))), Literal.create(1, DecimalType(7, 2))), CaseWhen(Seq((Literal(true), Cast(Literal(100L), DecimalType(22, 2)))), Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2))) @@ -388,7 +388,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("BooleanEquality type cast") { - val be = HiveTypeCoercion.BooleanEquality + val be = TypeCoercion.BooleanEquality // Use something more than a literal to avoid triggering the simplification rules. val one = Add(Literal(Decimal(1)), Literal(Decimal(0))) @@ -414,7 +414,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("BooleanEquality simplification") { - val be = HiveTypeCoercion.BooleanEquality + val be = TypeCoercion.BooleanEquality ruleTest(be, EqualTo(Literal(true), Literal(1)), @@ -473,7 +473,7 @@ class HiveTypeCoercionSuite extends PlanTest { AttributeReference("f", FloatType)(), AttributeReference("l", LongType)()) - val wt = HiveTypeCoercion.WidenSetOperationTypes + val wt = TypeCoercion.WidenSetOperationTypes val expectedTypes = Seq(StringType, DecimalType.SYSTEM_DEFAULT, FloatType, DoubleType) val r1 = wt(Except(firstTable, secondTable)).asInstanceOf[Except] @@ -512,7 +512,7 @@ class HiveTypeCoercionSuite extends PlanTest { AttributeReference("p", ByteType)(), AttributeReference("q", DoubleType)()) - val wt = HiveTypeCoercion.WidenSetOperationTypes + val wt = TypeCoercion.WidenSetOperationTypes val expectedTypes = Seq(StringType, DecimalType.SYSTEM_DEFAULT, FloatType, DoubleType) val unionRelation = wt( @@ -536,7 +536,7 @@ class HiveTypeCoercionSuite extends PlanTest { } } - val dp = HiveTypeCoercion.WidenSetOperationTypes + val dp = TypeCoercion.WidenSetOperationTypes val left1 = LocalRelation( AttributeReference("l", DecimalType(10, 8))()) @@ -584,7 +584,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("rule for date/timestamp operations") { - val dateTimeOperations = HiveTypeCoercion.DateTimeOperations + val dateTimeOperations = TypeCoercion.DateTimeOperations val date = Literal(new java.sql.Date(0L)) val timestamp = Literal(new Timestamp(0L)) val interval = Literal(new CalendarInterval(0, 0)) @@ -615,7 +615,7 @@ class HiveTypeCoercionSuite extends PlanTest { */ test("make sure rules do not fire early") { // InConversion - val inConversion = HiveTypeCoercion.InConversion + val inConversion = TypeCoercion.InConversion ruleTest(inConversion, In(UnresolvedAttribute("a"), Seq(Literal(1))), In(UnresolvedAttribute("a"), Seq(Literal(1))) @@ -633,7 +633,7 @@ class HiveTypeCoercionSuite extends PlanTest { } -object HiveTypeCoercionSuite { +object TypeCoercionSuite { case class AnyTypeUnaryExpression(child: Expression) extends UnaryExpression with ExpectsInputTypes with Unevaluable { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index 05c8d8ee15f66..de3d889621b7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -25,7 +25,7 @@ import scala.util.control.Exception._ import scala.util.Try import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion +import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -171,11 +171,11 @@ private[csv] object CSVInferSchema { StringType } - private val numericPrecedence: IndexedSeq[DataType] = HiveTypeCoercion.numericPrecedence + private val numericPrecedence: IndexedSeq[DataType] = TypeCoercion.numericPrecedence /** * Copied from internal Spark api - * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] + * [[org.apache.spark.sql.catalyst.analysis.TypeCoercion]] */ val findTightestCommonType: (DataType, DataType) => Option[DataType] = { case (t1, t2) if t1 == t2 => Some(t1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala index 42c82625fa53e..579b036417d24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala @@ -22,7 +22,7 @@ import java.util.Comparator import com.fasterxml.jackson.core._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion +import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.execution.datasources.json.JacksonUtils.nextUntil import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -252,7 +252,7 @@ private[sql] object InferSchema { * Returns the most general data type for two given data types. */ def compatibleType(t1: DataType, t2: DataType): DataType = { - HiveTypeCoercion.findTightestCommonTypeOfTwo(t1, t2).getOrElse { + TypeCoercion.findTightestCommonTypeOfTwo(t1, t2).getOrElse { // t1 or t2 is a StructType, ArrayType, or an unexpected type. (t1, t2) match { // Double support larger range than fixed decimal, DecimalType.Maximum should be enough From 3727e28593e4d0784b678a050919c9a85ce208a5 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 13 May 2016 08:29:37 +0100 Subject: [PATCH 0216/1470] [SPARK-14900][ML] spark.ml classification metrics should include accuracy ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Add accuracy to MulticlassMetrics class and add corresponding code in MulticlassClassificationEvaluator. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Scala Unit tests in ml.evaluation Author: wm624@hotmail.com Closes #12882 from wangmiao1981/accuracy. (cherry picked from commit bdff299f9e51b06b809fe505bda466009e759831) Signed-off-by: Sean Owen --- .../MulticlassClassificationEvaluator.scala | 15 +++++---------- .../mllib/evaluation/MulticlassMetrics.scala | 17 ++++++++++++++--- .../evaluation/MulticlassMetricsSuite.scala | 9 +++++---- 3 files changed, 24 insertions(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 3d89843a0b711..8408516751102 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -40,15 +40,15 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid /** * param for metric name in evaluation (supports `"f1"` (default), `"precision"`, `"recall"`, - * `"weightedPrecision"`, `"weightedRecall"`) + * `"weightedPrecision"`, `"weightedRecall"`, `"accuracy"`) * @group param */ @Since("1.5.0") val metricName: Param[String] = { val allowedParams = ParamValidators.inArray(Array("f1", "precision", - "recall", "weightedPrecision", "weightedRecall")) + "recall", "weightedPrecision", "weightedRecall", "accuracy")) new Param(this, "metricName", "metric name in evaluation " + - "(f1|precision|recall|weightedPrecision|weightedRecall)", allowedParams) + "(f1|precision|recall|weightedPrecision|weightedRecall|accuracy)", allowedParams) } /** @group getParam */ @@ -86,18 +86,13 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid case "recall" => metrics.recall case "weightedPrecision" => metrics.weightedPrecision case "weightedRecall" => metrics.weightedRecall + case "accuracy" => metrics.accuracy } metric } @Since("1.5.0") - override def isLargerBetter: Boolean = $(metricName) match { - case "f1" => true - case "precision" => true - case "recall" => true - case "weightedPrecision" => true - case "weightedRecall" => true - } + override def isLargerBetter: Boolean = true @Since("1.5.0") override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 5dde2bdb17f3a..719695a3380df 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -139,7 +139,8 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * Returns precision */ @Since("1.1.0") - lazy val precision: Double = tpByClass.values.sum.toDouble / labelCount + @deprecated("Use accuracy.", "2.0.0") + lazy val precision: Double = accuracy /** * Returns recall @@ -148,14 +149,24 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * of all false negatives) */ @Since("1.1.0") - lazy val recall: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val recall: Double = accuracy /** * Returns f-measure * (equals to precision and recall because precision equals recall) */ @Since("1.1.0") - lazy val fMeasure: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val fMeasure: Double = accuracy + + /** + * Returns accuracy + * (equals to the total number of correctly classified instances + * out of the total number of instances.) + */ + @Since("2.0.0") + lazy val accuracy: Double = tpByClass.values.sum.toDouble / labelCount /** * Returns weighted true positive rate diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index d55bc8c3ec09f..f316c67234f18 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -69,11 +69,12 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(math.abs(metrics.fMeasure(1.0, 2.0) - f2measure1) < delta) assert(math.abs(metrics.fMeasure(2.0, 2.0) - f2measure2) < delta) - assert(math.abs(metrics.recall - + assert(math.abs(metrics.accuracy - (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1))) < delta) - assert(math.abs(metrics.recall - metrics.precision) < delta) - assert(math.abs(metrics.recall - metrics.fMeasure) < delta) - assert(math.abs(metrics.recall - metrics.weightedRecall) < delta) + assert(math.abs(metrics.accuracy - metrics.precision) < delta) + assert(math.abs(metrics.accuracy - metrics.recall) < delta) + assert(math.abs(metrics.accuracy - metrics.fMeasure) < delta) + assert(math.abs(metrics.accuracy - metrics.weightedRecall) < delta) assert(math.abs(metrics.weightedFalsePositiveRate - ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) assert(math.abs(metrics.weightedPrecision - From beaf703d5472a90f529d5194f5cc657b45871087 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 13 May 2016 08:59:18 +0100 Subject: [PATCH 0217/1470] [SPARK-15061][PYSPARK] Upgrade to Py4J 0.10.1 ## What changes were proposed in this pull request? This upgrades to Py4J 0.10.1 which reduces syscal overhead in Java gateway ( see https://github.com/bartdag/py4j/issues/201 ). Related https://issues.apache.org/jira/browse/SPARK-6728 . ## How was this patch tested? Existing doctests & unit tests pass Author: Holden Karau Closes #13064 from holdenk/SPARK-15061-upgrade-to-py4j-0.10.1. (cherry picked from commit 382dbc12bb4b06871850a94f88018e76058a9b52) Signed-off-by: Sean Owen --- LICENSE | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/docs/Makefile | 2 +- python/lib/py4j-0.10.1-src.zip | Bin 0 -> 61356 bytes python/lib/py4j-0.9.2-src.zip | Bin 55521 -> 0 bytes sbin/spark-config.sh | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 4 ++-- .../spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 16 files changed, 15 insertions(+), 15 deletions(-) create mode 100644 python/lib/py4j-0.10.1-src.zip delete mode 100644 python/lib/py4j-0.9.2-src.zip diff --git a/LICENSE b/LICENSE index 9714b3b1e4d17..f40364046b34d 100644 --- a/LICENSE +++ b/LICENSE @@ -263,7 +263,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.9.2 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.1 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) diff --git a/bin/pyspark b/bin/pyspark index d1fe75a08bdac..396a07c9f4134 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -63,7 +63,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.2-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index cb788497ffc79..3e2ff100fb8af 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9.2-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 8584b62c0ecee..c9853528b4e88 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -328,7 +328,7 @@ net.sf.py4j py4j - 0.9.2 + 0.10.1 org.apache.spark diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 8bcd2903fe768..64cf4981714c0 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -32,7 +32,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.9.2-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 83bdd9031903f..2477312d74bf0 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -140,7 +140,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 121e282618378..0181a47a79d3c 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -147,7 +147,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 1d5ad271326d9..f7ff23472b9b5 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -147,7 +147,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 909b94bde20c8..92db55d674a4d 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -155,7 +155,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 7507599740219..44b9b049344ad 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -156,7 +156,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.2.jar +py4j-0.10.1.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/python/docs/Makefile b/python/docs/Makefile index 905e0215c20c2..12e397e4507c5 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -7,7 +7,7 @@ SPHINXBUILD ?= sphinx-build PAPER ?= BUILDDIR ?= _build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.9.2-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.1-src.zip) # User-friendly check for sphinx-build ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) diff --git a/python/lib/py4j-0.10.1-src.zip b/python/lib/py4j-0.10.1-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..a54bcae03afb823da3b2b69814811d192db44630 GIT binary patch literal 61356 zcmb5VV~}XgvMt)SZQHhO+qUi1wr$&XueNR5-K%ZCwf8;e$M@pC*!#_hnlb-W&WNnc zF)Bw^DM$l@Kmq*gK~}Dk`1i;Eyg&hP031A7tmsu$AOV0^teDkB%b3+&J)i*qLC$~y z0RH___&Xit?*s?{amO5qn}#$mRagK3ATa;{jQ^c(W$0$8Z)|U4V`}VTX>aFD=ivEo zi>=ed>i-)5FN;??w)UGGNZ;4`{m#7-+N%TPoAXuZKSQZ< zZJSrwri-dX8gxfY(xyD2H%&C)2aj4upbYh^pP7Rt+?E$$;571sub?ytt`oS7->Y)1{vg?)^8gvqu ze+{{59`UlBAocpk7BJ6wLSJRbCBCvMToYZ!^m@jCk@{sGNmVSdo`~^OjTA@;SyCTYacj=b2?i1S3Yf*8 z{>-78VE7oXg@}zwM)l63q`@TRKmst83`CgQ|4U{|4rHHjk}OGe257$b;-$L#!KOWP z3LL=2ASR#y7O*!RohudxBgDwY>M$tZNX=O@6O{vfxmqQ;L{y6Wveq5-`(^I|i32FE zfl$b6Y0jF^nMjVcd0020&KDOKy{!r>87R3xiltIQc=D|jRF5bkxf`N1MxQ4ekOERi z;HfQ&ZS0Mp(;@1;|m)YSRo*1 zqDSPxy`TiUb3zdK+RZIk-e|kqPtwePo35+6=>_A8AQr_aOqa;POP4T0f?Y^lf8+-_ zV3Y}U9IPdR5T3n!Kv>TR z7Q8IGs$0bYAh;LB{14BYI^bu|pE^7I`!P^@pd&cGkoK;@v`dD^Gsg8#5I7-~M$33C zs_M$MuTRY4AJxcfA(tn9Ge;~0?K5&9i@8wH$PUfPgJ3#y)qy(iYN_yvXHiVipr)vK z{CNk40bOK^AorU=I>JY-XTq2S0?d_d8t=B3+^;D=6CywbqKWz{Qvi1sAi*;6q5xE2 zv~{V(hV3>nNWL7lLXc+ns^)n9+wO!+FRMqZH&&){I`l9Df$jBh+;R(r;TTIAa-Q9> zGQ#|cimWWme$S;NyIEOplxoY3-mMzg?9JwYwsJwYs#=rLwf>WbvR00U8lE)W8_m1ud2N$rz6 zBDw=>iuU~S2~hPDreb!@?+BP#<@ymXN`jQtmG}3#Gg|-Q#2K=Migs6KY`(Ad#9_*` z;eBejs%_o&`cHr>o9NDfu2DZe5x{y-0!`yrp*CQWzT&f&v2Pz5$^Ze*8YGFJQ&+)~ zg&G@d0-1tsNG+j#U3Jx?wZ8oKQGrRu(#~nJV3T}Cd8uL9U}YW_>PdF466sL2JXHYN zl>4uu$;l%A!b5g*bMez&(Nl4A;O9?ydF!-meSe6-wqCp9HcY!@#Vvtg5JHTm5A%(h zio4Hxw?L$`^Cee5yJWn+IkAwaqRE7kql#|C4%T4&G+#;s2wIxCpVht!FCS!z4BO%C zydQC<2y!!Skjp{%a}F!^LwI@lSnxPwKM*KXxAZ(J-Y<>qBZ7lZ?oM;z9N;MK)OtY_ z9DglU^`prNunOB@h@kg(b%O*d8xhbJipkr2R=iJx1L8-<(n_&E`;NM`;P5-0mcsdn zHt<;fOJF8uEdUOKIS8RB>hVr57Hxqbo2g_iviQ>gTzk*BqPMWK6onMkW`vLx===SZ zZDM06ysqCMU9HTD_jJd)1~#Ac^&);4#t=yuzbdI7RKRfrDQRppXf(X2ABJUFTduYh z(thzm;Q3iY)FN6(vBvY6a;#mnn@Xz)Kx(M6kkOKqA+o_}^O6xmSrF6-gos0AcQ6+o zR3zwFCq||{{*f5r4?lR%Z*51_`HanmqPIeMJ--Kn=}<9;Q5sRV%%Bn$6YfxH%RtTt z?J$T4Y#LUB?lKVH@8| z3qBZGaAz0b!p-LABH-oSbD&yMzEysJ)RE4)(3^Z~b9~_lC%(Rr#PJ5@&Tl_nE50UX zXl(#fO`{)rvisQ^eIe|MFieKj6=&{#)2o#=lrCg;m4A7EVOHgUYstsBiBU@&fG5KU zYB8|M4%3MlM=_jLsV7iJokw;}3QB6x5(MnYoVrUy9o=8FtiF!SuN=5qOg@*AuS_&& zW}sIz_w+$6;~_LOX}AByD8N4d!j<_kK`swZ*R3lnv-+~&eY&vugsBIxPI3u-k~xub zY>;)o>E%|-UNrK_lQm7U9euSFg`P2Fd`C4%C*$)u0a` zzqFbPls}c^bGk~WRb%JA489T^!x|`Ea%0qUx$>T`g(f3ESqWcu^?ne4^-+<{WCvqy zyvw=+kyr08B6=h`tjEv33GIj`t2WcIpRWrE<1mNk80>1Ai;pD>14cXxEF@VoAT0b~ zcjH?$5Za`xi?mc+hgDE{%U&amU&nVmNovOPk^@0ik-TQ3P5OIZZb0m=wohYQZZrPQ za>bgX6jtYXvLI!r^HUq=WR-|6k4k((5N@)i@#nW|dg_yf`hhNRPjSGr_A&bvc2)?x zs+D$H*Xw%W(Xt8lgx*#2bP2s*EA_9thg+odcKiaE5&5P7OS0X`{Sl|vZ}I0c^1*qu z62^pe;E$Wrr`_30{L}7lYnsSu_+VeiLG~r2*LAwCQF+?BP|)IFyEFs-u>`l~va_L$ zAoB8a-Ps(3m^3Z9GBp9UDMDUf91|WtXFng74N$E4{&EH3UV0H^e$(Pf_Kfv7PSFaB z?hWfChy&H}5JTJe>qzbDMBUHp=jIttqP?tCFkv1mp5iMiDRl!b+^-Q1NJ~W@$(aa8`|@y&gMDs?2~crWvI(} zMC9_*aC*7!O*-(iC+qBK1^XGttkli|YcY}=lJ@3BE1-7Tm?0_BG0y)n4EaCi_(uermmx-I5o9Ru4uqib&&!9h>+L`%$EY`=xF* zYc|@nA&!p~aOcoA7GRJH21*-h9TR&Wwt2#q7PjG^guSid9b0(8-vShyCQg zL9fS@;wZsDe@K)tcvo4}3~a?tWSuXEYda)wLeXHd)8?je8fL|LMbCP(ua&F=wbt6} z>iAi_&bGlGMYTUQ0=tMP6oqKMfCIxF4reHYsIbM%ecmXl8n0!?hKip}lB%xO4PW+0 zMq40}*P2}lKIjJNB?8H6HFBJEj2(T{mB0*x_63Ky&xX>HQ7ONyn)%}0xNLvf5!?zO z^);o;J#3N*;gci8Y5!yfR5Gh1I=HzY#(cM9JZcP}A5z-?IuXkl>`vSIwvcPz?n}Pj zayQYU=Z3GrOYOQ@yRa?K0WV|kxaMTBW95eTVZzeEi`miH@pkz*rDLnO{q@f>?ZSAo zs|_9iz=#n50O5bjGzTYp7yG|u+J7j|TClADmS^`}exULn!#nLl3t(N)yKaD^irVhD zo95iU`PyU6GMiRU4382QwC}t2p~VzRiAi>j=qR!xGf~Wgo|3J$+LOG*VJzdw{E(8# zHj(iOu$y3vlM*v8ffQpw+#bbuq4W(?Wnlz62}H)&jeJ^7hsV~ULk<3B@rbYDk~6=L zE8XhGC zm_j`8kGzoKOpA0;d%STH*YHdPUn^9zg58O%=~haNiD+{SsQ z!}EqT>kTQxafgIpStdIW)DTW-Vk{+AXr5mXW%-M!8hxE++3syP$&ngm_)QSB{V_~Q z))Q%7vmjl;5S=9|>SO^I$g+1+-#hZfB-4-3h(zh3-M53M)gk{A@c4zGKlyGCRH)HK|Yk%lmoqxX#I%#gj2JI`zhI7H`<$EAZ1IswU#Y zsUD}t5jLINHNPJujoifunBAQbXG+KZ)rw(sx%WG8I4>Og^z4nI{8%pa;bq4L>cC^M zYW3RTJf;?rt*vI?g()Cp-sNEWWHEzV|2Mm#&n8g9R$I`^&OlDDtV z)^*f8H^^((?6K$9$w;T0K8)R{H2in!+Bx~AMh=Xfj!syB{m)N!M-a zjkyp0+#bWVbbP4}l#Mk{PPfXe@n&bH+~~DIVd_esjh8p8+wuh1I=nc0P=i+I2=kOU zR>@TB9?SJs4h;3wQY+X4Cr-BxfFb&|67RKO_I#liob9CyC{0@;7d|<@pTwq-9ANu7el6 zN_)?)>b-qOqdAjG;H(2M4Q+kA_5A?E^y85&`dGpcuSj+)2hWuO zZB;Y%9Ds?I1c!)Qdu;jjKv-Q#Y6ztj&@=TmzMB9Ywh5aYV@^Tv?nT2cGpbG-^Cxsee?m6FlBozvTGUD;!*XE2Yr4w= zJNhwpfeLZ?$mS z32#}XIo0dHh1SVkfRFVW?e`PR)KyP4(|Sm{)_BU;08;8AUN;D0K2$#%`1Q(nGRiNq zGoOoIDZ(0}T^rEJt@^#41yC|4tjLuh*^dpKYAsDeFhoCNeLMg91aE%c70y&Jrd*T*{@-nD7q1&_^w8E4T zJ+%#pkVez|mg`^)OjhgxJ&DD4@#@OnzNdz5FdV;(tn0p2eUi#^5yGlrPxd)aFwT=n zOm`J^tX9`SUBiWi9ua)gmxH)c-Z)P@)nF?DZ6a;0+}%)UXuP={tp!=7ZexR4XG)jw zT3-tVd1{PMK+Uz?D>S`RSyf1?uVmYdZ!;9K-fCWGf-MU7IuEh+?$@t-`o<0 z#}DlWJ-K9bN!L+6=0nD%ehi5)p&>W~@tSHbB))qG-$PDND(1V15d8?nZOkVST*T=N z;EPuZ1~u@6`sOyBs~K4nF6$ttQNdl492VqWqkO8@58QpXqEb`V_>4+ICYij_uo-nz zuWYqMrqqhElP-{5PDOcit;j-`gv#NMfg|MTBcsgksCy)TqA^Wc856+fiYEy!%46MJ z>;j-TfC1Y$Vd1Q2;W;oCdC;LunkA3akO^h;aoZ0X8UE~N$wcYapN^zT!ay7RS^$Gz zld(3-k)P{!O9oYVXd(UF2SdIDC+7|p+?mHdVE(hzxm_Eu2nUW<)2AQ|o((?E+)`*E zzrBDBQ#M%)4Wd0~fpAz2>_ed}4B1ihL8EVA2rN89zyh;lhN^Hi^asvPb)vi%oHgj& zCzCPRcVra(NHZtmDAwePXqWG8`dX?L(^{g~e-b%zJcc|F{bv_hP$EAP6#;c1t^fwN zc_|Vs+BueBXCnoGtK@lKqJ$=Cj@SiHIL`7jA0sH@y3uNPNj?Q>)TdojA=*6KVv;O!4$tdHVQk+xL*eLY4r9 z>9_ePM7%Ub@ErFLzRZ#FgPh#XjVugqMnId)_z} zF}yL}L=_$bWy`Etg|VEDltF2I&j7Tb5W5Qb{>*W7i635B%v_((KM4EOKq=zr{}8d~ zihF&woQ@|WAq#?Ge9*K9am?=G8VQKyqzp~!7?b3;qAcV_C)#GHi?ExSpb9E4BXFnk z35Yq^6$CEGy_FV|bnE=yV^vD!aHPd9FcS6F;O~mHy3mTJP$gLzVh7;)5k-6kt*4GG zNUXgtOaq!HT*Sbd!hnf4eWlVv<5*zm?vz|lEph1AeDl)#TZ~zR0RE1RJ1eM)s?rgZ zh|KkuT0&xNd;a3Pec_Ui$DI(|QidVf%>&&Tp7rQeyYF9bVvyn5Rz;r~s3O7Q>ckmq zrwl9y`{{tl-gJjjKtm+LX<(66@^tLPpW(L}-bzIec7m?#g=Q8r_yS_r{qw8+)xzff z1ETAxVGn%hog%`|W8+7ox7utHmWkqh%{aqwI5}(-3QGV!N)@*=WpZ`};6MsPATD;3 zq(v5E5G zM@-KGl0f9;C$g9C|tbun_jOr!7is-NaEI;_&xp8?~eIn_xj_@h`YD*H|a2T1Jd+(Qd92m z2%4uvqOibz>L$^R1Za=)Tq&Xoz6g|b9$u(DsOnEVNMJg!M&0!4g7cc7l%{b(>=2aL zSTl6|``Y;5@WV8wO=5?!KXqs`Vs_7Eyl;?2)m&`59eU;|sL>W$| z=rnyV6;7F(+D@@=g-+pv&A)w`n735}&abK9f(fFi7oO&mf=I~eWdDWnMj1+>S49qE(lrQ)!zek&uan0 zS*M?T*|!$4-{r!q`7FQe_Oh;NT1(iibKm{>sq=p5>{oEqQcJScqLSL?el7JM&y70V zw|R#Kf_n-K?%79QYs}VIB#@V2<{D>fJqdL4<^6ix$Pb*80+~!vsfS_z|vjU)2XR~j21Z6!e0-4JUQ;0Wj ztQ>@=Z@S2QWua?XLR_uvI$*aqt$nuQG&KzdIA8Z=0`A%CIlZxR6T~qNOu1f}!{Ulq zb|bc~16&r7bW&Vc-NR9Ulk%x0k4^1X5ibHK*_?Nvk)D#QY7~mWsPe8#6@d*j1@#~r zCfvc&(p0n?)1*dn`s2hE3OM*MXU_zAtMLG!Ut}Cu79-yk zR}}QFiV2b(R}Nel^`}pRea9E^ZSwg*L0irLUdJ1zGv0{%I~70lcQctuSr@i$8Ol_6v1O4iZ{yIvcdLE1DlQ?b$(M^QhoYTc$;v%>M!8jc!cSg3 zV?>2 z+q}^KD*(8dI=eXk|7iARY3W)5fdc^i!Ug~!_@@zL8%t9=7ky__CpS|k{eKz$530hI z#^0td8^XVuzFhhweW;nu9}9HDN~@Ad$HU=0bi&#g%`UAON`+!dj@|mb#FXn^jyCLq zVQA@!EJ&X2-K(dunG%Re3M5$9tmei82PzV})k7==%JDTmu2`*Pp?ov#W)&!e6vK&@ z{Y&oDpe++4mae4I=_~mzlM#~Wb#_78UgF})m4o^%1!hE?HO0E*W3e1T#YhFn+%Dd1C z_M|V->Skbm$x8(ks>qb!i6TToOgv+JBsWnNXUAE-x|*y&F_z)`J3C^P`S2=VM?%Fh zm;9_&HO;Tjj}|HZSiBYp!we_#=)6Oz9s*SCBG}`pb4fxlSFGVi@8uSw?q4ozGz|6fiu0m51zg)c4)ugM!ft;A!BWJ~7HD`q;ODH;FzkXDsOkHi;OO-sN=(W|1CT~w^iU9G>o zO?LwA!1X>lISK9VB}!8lcJgucUBFJ)C#z7OkAzVgz++UfxrDf0m%{{h`#!XHGOVe_?WAl7E z3!g0Un_AGEiICFkrwwZ_!-9tqZlrt^jAd1iPK{5CVGr@bDwmw5ABNVpw76!Cu!YrFm;ap~*0Q+O)LFCoDY|i0@AQbP5J3IynHS*)9aS zLwI@?Gkz32Vgld~ZN*d_)JX1buyKKpM-0efjW`d@>VQk>*Vm9O#1Ku$Z)5cG@KDrz+51um+u*h7dWn9;e!;I27n@; zA49wwC6YP9lb=d>oXawFeKy9!2wQBN`?7c*t2)$%pW(XFKJH2lkLSBWyvJ=H-G_&3 z%X(|7SJYaKf;p>KyPE@pdIhWl0PU=y9KxvmK*yP6^<%n@siW|)g`Gt7??E&cry_?9 z0pik8{*cN1eMX<;D8GYTE%ws5qUfxZ}5eRN4!Q7-x7;|KkzDfXf6eFJ~ zR>l2}&C!Rxv1#9CP7qk&szE*H-u*^6ymd`!Gi;A|2E<_INLl}?JuGfUTxXBAxjEM; z+g?N0OvmgWmV(H+0Uy(v3WN=ZG#OYGy9PrR+f!HJn$9;F)Df@Hf`>jB9q0vdf)XWGxjzvN5q!l=;m0S%A z72L!c-GB^#9^f$0gqx4JiY{yR-$wL5m{Rnjd8I&Kop4Q+MTM9h9}*X=r0JBFoT>ae za+9^n*_m zg=L|A(-}Xb25A_7Ldi$^lI(D0=2|h8%i&uA6Y^1=346JolEWKbe7hn1Y>p`DUUnDW zCqH3U<2|32jj*qqYesMLjy*!%k90t}-8MvLA)eaCEZ6iRQW`wJC6Q=Ax>PDsn7sn zWwhINv|P$AR>PYIY6u11v8L?`hTQv%dqT7E2MVflV>m(qWa{OV89e#p4fhBKdT1Ib zGj&TcRWw$4So`tk*6AU>tLkU0?r4uuIlqzz zn28kVWv$`Q(FIS0sWeBce6dXaO2>1(AuaN>iNQWorpU>Rf}CnJqx747YOgPbyh!^z(WK-iQQ;fMly9FJu z?JOwJ3UC=Aitrn}TGQbSUeWS{SjQAq-LSwqwz#@ahf+kXoZq@U>dl5s(chM8@ zN$)qrqLfB(!45~;cPtVP40ZI%-18x~G85`rhAz$sA#uSysACw5+L4HE9XCOzTsY*{ zEuO4kN;w|>%$K=1s|i=}4j#$%bO(e<6LyLd7qcZMmg^0x5V$_7FIlf~Ojw)lbu_?F z*{qfJ#v_0TzV|}GwLwtx^Jj=vM6Y>8n_}ffi4@N!R|)U>(KP6(*UD&i*VRZA+9U_e z=9sAn;EmF`d@vg>b=4eU^VU^3t`CfFn&nN!jy;GLt>UeB=JU$!?N?ZcI^-l^zs8=((xGaFSvSW>bX<@oxd|4}$VS)?^zWvdjDvx-M(| z-IFEntuV*=oBiRW=hPE#6B#NB(*m?BlVAe`fmU65#7@*0KC?9*q7j^?!7Aev&S?R4 z>Bo3J#UcU5n1(ilv3`=tvPA8- z9F)wJ(PW=&D5uGLg%}dAW&IRzJRk}n&Y^gXgU%nZ69*8>{Gi@>I|y50C=wk(Mr4i5 zyw#N~icU?tl`7QB^L_*L(*t$YE-VF2V8(7wU&vQVuBG50ec2su;~A9GiIQWS0}P%S zYC_c0oqSeTH2sz1S^I?sHoNy74_r*`X4SZL(n%(3 zbm*)Gw(&EwE~(vh#U?AUm&SGaakuS$>U_u%OUD8M%$Wq+6$t9hZL)5`6Iu9D$ZB0t zW<%JlDGd7Li<0y$HJR!7+e)@7z*%ic%4%?@QQtH2pX#b$YXR3GmMS{^~Redkr*^=N&kLX zxhkj*#c)chv;a_~5h%`T&G>O^!7P*jY$3&j;jmr*tyZ}SgR35k!V2sMZj{Hwh%g%c z8N0N2v-SerH!6Spej`%&A7SgG5Swd(EPD|0_2Zn4I_QA8k7h%JDfbitl`6e6f<^Pn zwLgx3#tBq*7&#usrJuRnt}v9m$;`>O$W*+>({P{H*|!J$U)Ky-2_8uqU4d518n}6+ zNNG*yU(;z9CHJsm#5Wy-yPHisB1(^Sub}c;OUoB#fmZcx8+f8PQ_e!C8U@I*ZG7T2 zGvg}X5emx^H>8j-CG|>gi%Q7kolG;%q2&sG>kvFSo9Dsg+28h=ii-p~ zU6eBQW1bEu+{~SS-X<`YYRk0GYB`OdK_OsOktN2fJ~E0E6g2hj)iC{z0E~0%%5*cY+oC)3N;yoACUC*h_d72po1T1a8B{6<62Q5v1HdA z1n4`qDP4#&&45l}c^w|pLMUQ6Q>raE;<_$7^VZd*Funo)nLiSU!?#m^Swr+M;$i(W zfBwxH|3eo40px7sWUc>lMd-~pst`GPsN^UeEr=L9#vc}o8!_GEA=B?ZSG9q&Uv&~( zYxzRxJ#~1#1chFdGAZ!!MBVq~H0toTYb?|yxt8U=RMu2jrav50xNQSaVBV+tcT1M3>tD zS*P1980-G&pG7Qr3fsT<@&x^V#}{W)m;VM|w$*l{vIG$PUg|LZL3`lnX_)HaV103(`0o=8L*b=e~;G03n`kS_<5zT2Sk(2&IgLl1q z`wPtDyylb2xT?F}Mf2JSszh)B8xMnqtQD~|QtPHCqX97%mye8*YH`;c)k+Uc)iIkh z`Mq1LH}ow|OcGJu#1MYGLTJLwWIO5R8%1usuW$6CFM7PTU$wiCn$f}C{z6&;8q%1+ zVW}OBkaZurM(hr4r=9ele94ueBb-MBv%1*X_(3$Y)r%> z<^UbK1Xrow+|VW_r$6&RoUft8x%(P3Ex+!tX zw&JvoEXyv)M}Epk>0zh>oJLKkhH`)_TA1Rflt`$G=|G8tmE-`;)Fh@5?@)*p%@d_x zyxXO4G|rGw>k!Y*&x2XxrMk^*v{25WLDNPQt>|EVFPTA@3jvV)J_-7Jv#DRIB9SkA>)-ST_}enWY~}+L{^ioHkaq~ zys)0vy4Q@!>p@e2tazlJd;h{CF9eA>hdx;Xuwt%1tUsLPx~L;)Q(2$J%-rlL6`qx+ zhl*>3j1#==pSu_2hfKs(4O6eckyn(;4h!Z>OJ{c7Ep6ad2r*R92()J;ESL~CPUhsG zCB52nuc5RH^*<|mz)dU_641DP8>IUg8|51GslaC|7CXK1Puzv6A}^Z*W5Y&YJ=}M( zC_4e#k8@oO^5WJg|KM3GTfB1kfZj)T!InoC%VJU7`njX+2*iICGWa@#7A4KOwJTt4 z-U}jS>08Q-R!LV}xyJn=sKG@p$;uXd&;5!_>*w4^TE?OKls@n~^?7`HyZ15X9fkWS z^XVB~T@CkWp-a?@Bioka)7_gNRR5C4LYcF8|EY3^$lHhgXB-t_Q^-91jUyEh006vy z#*vYyi>bb|i<70D`F{h3Au5xxSqv!K52$V9V3m+ob2OC_>&>#%uJdYErCExmoDPs8 zBc;!z$b1#M^Z1vnAVLeH1ZdAsoTGfBahx=IbOfxL6))5tYrplopHm3hN`VOPL$RWS z1T>dNkj8SDD+y72`5|VW#L`60EY6fQ4dk#qyxbH&&uTX|4t?E4y67Wlt%#7ZyzMoKrvq zsz{#h$x6Sb(1bOZwm8(Ie(Y$UGibREV4m#7<6X=ZOw113iCvGdR{!w2V#Mrm+C(h? z{+Ix?92N&ajh~~ANj-_rp04hA&yfP~IW&))b=`<-+s2K{ddo%|9iEo~Q;-Y3TkJb`l? zY`K_Ww=XRnG9N!w1eWVFfUJMFuQ9yc!9jj)#5a-kn^$s+HiIKcDr3DQ)uUmY5}LcO zF0Tn%UW;0Lc38Rl)VJ(2O9eL6&NE2zJ5bUN>Cibem5m>(LMUV-YAeA0u}NgMKZ%m( z4S#DbdKq8e*IbPLdUJL!Y4ihex8*JI^b^HG^DWF^8W2ixTnU=-vdy)d^!J4257De-lY1YTAD_U=+XAI)bIJv2-#P!8RaJ78=*47Qq#dh$ce| zsKC-#8!nWJ1Qn#858R6biPT1ybiJv`ea>C?TDU=mbZi4Srb5&=P_-ixGV;2(Uoe4r zn#XJ#-NbpIy~Mxlf`bS|-nPyH6*ARpnFJ^iMJr;6Q;>Ab1xP0_$!ulTCag3$^J7Hj zx@oA+MJ!pstwXfjXVIr6=j?1-sKUt0cRB0r=nYNVmgHyMT3MB^cWjbqyUbl;{@|_E zGNXWk^g4IHwRD7!XG?Y%eO_Zt@uq_^R|Ak39%?NG7GWO2I4bW*DUEO@$8st9*Xqb< z>B`%i^J3t&;`$eSTZCKOt!6oU`Mr-)>SojNj|?^9ly9rhEy6>!A?0U-_Os_l=#h_- z^XJ?62M%$W;e?2*B_ow5rEX-Jv3Re}ZF+Ovy%*9yRs0CWaYReL^3va(0_-JeOgFF^ zBsG~i`uaL*r7Ag`1gq_@p6)lVaPyE#cA11LxWUr-#tu|_#xK1nHdTEs)xsJ?QYjT$ z(q*(9u}sFd>)8f$tz?fTpHCt=uK>r;jPZ~Wk!`Bwmr2CmjIou=WBc+&ECMQBj-`H9|rOe)dy?n-z>gK(FU%4cu)drm%_e=U;PKwru(nb_i@S zYO2Qe5#Lr`BZgDIALVR}x;Xb*|8E%k~I%k(}BhrC_#S{)w+V|w( zwe#O^4lf_yJNbDECOm)L{Q<@0>n7Tr2g)+4viEaqDONY{-Sz3@`nX+i{ta0DC#H1< zSqxTMxnDZ2*bLJaR(g9WH-_Wkws+2U6zLpUxuAWqMJ?c6tr?YE3`&Ycgv0D6qM3wY!^Kfp%Y;684fS}{0PNE=4dCfZ z08|gC2#sOHN*eICE(r?+mw`>+ObveCknXpWEf=2}(NI?WjrZ_xeG5 z*^)cdXH#}5NpW*usD$MjhZle87p2z${=d-a|Ma^4bte(k{}uFKHx&Oh(Edqv`udi3 zmM;4G|DhlvCQ$HXks?s}+r}bL_$#^@nwvuX69Gf)NV&)`000KO0RZs-`Hz2Z%gha3 zOx+DV{~Hv%;yriT6kGUtLNPuQ7YGxQP-w(mE~`^Zpw!mWrJj&$r9p=x2}7|#$amB* z+p)+q*04P^^p$s)pBgBb$s=Dlgu}*iLi*;(A<@GaMR1kmDO2r&L4IYv?r8i*kw>%J zo3R`!D0MlY)zJ^^KKf-B&VP#hX1PzciesGc9?=#(02yP<5IDqhDo^5r;e5uC?Nb;5 zvcc~XD;Jig%f!X~+Qr=2jd7jl0vbIOy?CcomNGZ&bn+%tcZ4;?qspP*2##6b7Vn?H zU102fF9}3POfP}+&Z-OrlS9Ii`m9si%CD8=(BQ%xFbxaDpMX7OndWilk)psJxrO@O zpd(lW2JC}Iu9#**e?@mUXs~12{v%;2#SImdf!_Z6g*(ROP5e3 z8yeNMn?P>MKq$wdG=bBd7`=QELg%sX$(Q9hTuk4h`1f13SD)EI7~|UQ@vXzp)m@3R zpPS>jWdV#bmvetjb_BsedlIBB862BgdLV+*H3LNZXs^_^MP(+c4;dsOkqdTCUF$AP z5d}!qp7@gV+e0!LyWyiGPk=&S`Q&E0V!q%$Wx?H$t`d1ZJ3;~ASu$9O_#F_@_C5+X zHa2$A{&3em=wjv{L~nBj3>zId@w5wM@59f$P5bPJsQvwU`tAFzq zf13h$^G@!ZySim8&BgQ{4RPRH{*piv+Hn}=^XV)-vI&MK${a~SCc|s|z zWY~+iPIW(dB*9QNH|}~iL^GZ#6xu!7tU&nC6EMQmIkc8Zv!7-M;xUDEf&g)6V(U|G zx(_jsxJ>qT)wVk}$?W9iCJ8r7rw2^?uWThK@8NS;_`l3mxhqP zwn5`0pyS+6+h_ZHGfIp3gkmZ?P|*hVE$r)f;f0LuS(WqyDr4*Te*TZPz4A1{li6_| zW2NcSzD(oJYIU;*?X6t|nnYPLeWGUb#&mV(I2F`EqGhK>H3YrU*eHt}Lps_uIaCdX zqLc9W5NxSQUW$Lf1X^_?jp>9H0IR@Lg1-<9w0+q3VQ()h2fwu zlkA!Cm^2;aGa`pJ6rnO+(bND4P~qv2K=Kq7$?2?6lWYazg zSl$bYEGpk22T}G?RrrnP$M;04;MVDHrE-MWSPa-cv<-`gY$$Y|C_#`>TIBQmI{W2? zsy+J)m=Bn6QZ)s;)v0o2Wl{P8LZb=65J4;(-7!m4_SeU`pbfa~4;M4X)imys-ZB-a z#nxHfPWbb?#1%V-9O=NYqJ7G{!kM+!exejH-g;99Qgl3P*$iBrS&kHBd|@M6XYeMi zG(`vloa!$^bpn-74-uG628+9~#`GK3$QI*A(#dKKhpZCX7;{SaGC~lc;e0}U`GrHH zkqo5Ph_!vcK~P-{9*;?5jYpR9Tt!X#)9GUS(Y!~fpkPT^5XKT&NAMH_Rlp5}i+LB? zz&GEabNKwv;9U$bup)f`cIw6Fq@DXc(*hkLeWdI2p^)fS1?CK(c+ln^l??&!(gF+H z!bz`QiOx#W~lC#sTWDq?N{-VB# zdX$+ex1-oPFc}MN7WLce)rf}YFRQi3s;jsOQy*7-w8nTd7_Fi%G7_mSYd@HzBFd3F zg+Hjw5Nw?bhXVg|##@VY?LEAtppaktsEG9tNayP=WLhUkW|JEltsad&Ze1xgfd*&W z_#Rb|*%=MR{Adn~8404;i3Yk#!`}gwSVaK-8)}cA;rEP06ix|ptHZe0q~riF;WAW>}*mp`wzje=<}iB1U8fn4TTSm4yc49q0nqC zT9Uhb};ab~c^cn74@f_fnp?n|TsFhA!Aj9FB^*WB+y?e3i1t1sAm{K>+I4 zbrT-y7igk|I@GuKO#E;6w#S_kkwW>4lJ0mRpu`w?ZsYmkMG5k}<4uS(L1;Ck~ z&jW*z+2S*?I87|$oj_bqc#N9lt*xcYv361?9QdRx*3E4bC#@}LU*lwb8WD-HKkE^R z1_#ojQKC^sP)7J!s>~d6NrGLzM6Tp}1v{#$GBp{}2<1?iQzLtO-_NLfnfE;vXk3OU>q@i4qa@`;927WCLY@CMdf7M4vuBo)*yj~mexVmenYR4TH7LsrX1hu7GL?$gN>`p0O zhafWE>=zLD%Q+FQ8s>}3uOpoieF`};qtv=G?GcbvucJySpou&>Ux;4pRd=(fFtAd8 zc?8ap=##(joz^*t3OeE?S`NIR9S1f$blE^^s)=`8F55;ZN{=1{$~HBwYj+@#V7~-+ ztPUo7wIrPsUf~xnRo#DHWHgj|-Nu^Y!cjjLc_wK6Ka9O&bZFnU?V0R2*|BZgwr$(C zZQHhO+qP}&7&}(wRJD8G`=7d1?|hiwS6g$nHRl+;_uqKeX-=9;_Lp>@8t4=?E?Qu# z(%06sGOz`|gr%2#lt%OTy`sN1aeKsmYM**si-4G-4vFG5*-DnOVW`!FNAtax&~e<^ zzlA~EXZ9t@k-4wY6=34uy8pY<9$6bi6(YCM5I1@v4erwyy(4lcA&!wDvay{xal==j zTv)5|#CNF9rwV*ClqY3ehg>Se(0+6QwOt;XZObZ>HR!oIBk>* zsNPdqXOJszUXe{`oRb3X$OznYb(U1NBJ-G7Tf`}#Gud{YPuP(+q#4Tnt#>dK#p+?6c!_XWO8M1G?884_iRh;YN(YqN1gt8nepBur_&LE8Wpbo0?AY?>oAF3q(PY>7RLRD(0AecgNIcPlc&9EZf> zIP+qbDW_|6s5Ci&&hHl%$AD*^V(S%$nxgUL$}c*w1|Z7*w_gfPl9_R-<-3Tm{oI3c7S%3H`kus zZ@mbNnnt_SX|vkxm$#v15*wEBeO?*NhE9*7AtiCw;5_Xf#8gJU_^9)!S%mPMNXRA0 zL?P+Rq!*gM$ZI}eX9CR{+A5qUOIgO4qu97y-%BC*tjaUDevD*A1?#}AYbO#GOq=s)`PACPSiCu2@s4>|u@19- z1Y3EUX(Yfu{$%mhUAlvNK0VS`RZ7COgfW>`KQ~T`@wVtcV=nnt6$N@*#z9!m_a17RaGi#_R7POTygRAt}8!XLePwJb(sCK?O2aY#@7^ z*wn-FdBoxHy^900YtaPnXt#Fm8b_7rY2DFdY&B=7%%mXOMA}s2rG9<7uQP}-P0RxM zir8E4Ov6suU&cWvgDZB~yU?{Cu(4!&eeO+@Ge*_B!aq@vAjr+phH(`qIAI$S##&D~ zup_ZW-BWcj0yrpvaUM8anJ`_sKJ3RvJ-xg)ZSi`#@jaGZ_XhUn5d^;mqrRJY#78*r z5e==U)VnO90i__pTQDw0JY85L#BZw(*p%MW2ku$$-F*i@%VP7<0kZAgo!MJIr5X^q z1m1W?(Q`+|X`8o5l1ZZ*E?V|ja7vuEjzy;@b6Ka?05~x<=7&c(2YRKQs*b~9wH;*o z<|e`E`Fq?YxfhIbeFR&EtBERCak&4gs{+Bj2Md)*Ke_a5AJ{u7Gv!eIB0lsxEIVJB zsm>rDoM7K4D1w`5843W>7m>NjCbBt^n$Q`p}8LBp=qCS_C_9tzXfa zKdLNFC5-biMTe_@8v%(S(s)UnB6W^U@pA>9ghdCD!T65cp95K#Ame84(QC}C^{EcB zJjq*b!kcnkhm7b>63M5)->Et@FM-+CyoG2T)eDEO*{d%B+em}bTin6FT!Kmgo#W#< zV=N8|>TE;dbB6n54P@vt0Ox_5hWM~z#PQ=)ICYJa^H%m7-P39_I$K&6#qks$xb!)g z7=On&M*FgK1;*&Y?tQwA%o?br8}wb6@F@>IbG;PMvlAT?deg6aF)Vvs^KI1HT6@6K z`V_&WF}f9rFjX!Sx=Wdsy0IFC>>f+sJI^YC-ep#ttOkb>+LdemX`?}%otR+mMxTgA zAfNQ$m(?Brg~}_=F8Ref)T&OpVR0FfoeObRbv)I&!4~aqlkpE@M+9s0G0df0QE+lE zXe*EyeG89NWzHzz4|2j37%|~vGicn{InM;V)5pZx$O-D$@^9CXnw0_<5Mfqa2)e7{ z$nrqIP7EP*C_Go#qz5Z5?T5}*ChZSNhf3-UH%xn}VL*zpbyp%)3ypWpCTb`&vprbL z%c_FX05Kc&iPQZgLAhHl{zw@PRosKUg`f()?KITXwXjiK;e) zrdyEu+GxWU#1TZE8xOzJK1=kA+qt#`Owo3CpwSN#Heh-^rtc(Ne|&`Jd>+nn<%o%z z6xwN0r;e-yC$BNT7c)_w9Fb^1&DkaaipQBdw|-mK=#;`ejJm-iafK*xxE9juxDcio zmA4#L;^8;~7A@Hg2+ZCu6q_(q$t^CgD(3c*@t^ z_!To40HHfc*-Y2lZ#%7m3#!N&Qq~I`_u*@v8Eq2M!%e4gDKo`@bZuIr6k}~qd*L!o zJ7{U(hX?(@3H1Kjh^j3!&cV*%ymN)yi|v9l=H-Q0RHvIpg3|dZko2o~+x(K@fUreM zU7E-x{UFM=8yONC_*Y?5 znxGkg3mfcRiuNbUM?u<+%#Qot5;845+gp2ij;<+~sY@?@B*XfESqT~h)*#3&=T$ou zcJRWyW-ot}dEXl`ZAL)X9k6=8`)W{P)76$V-frRqRE^5B4l>^dMMeTBAN zGis0{G7k{S*9^3aKTEd7$q=3eS_)w5nHN4S^ zikUkcGT6B|8Qox8ObTGf*HQdXAExEu=E7h_R_zxD(i>L@yRy*;6xB&7KP1PN68>E5 z-`%BaA8qEC)>J#+%bxCCIytrbdX#Tr-DbQ^-ZYyCZx6Y-S%WFAmh(0F^L%`%LIdj7 z1&Yu=y}|f!hPIWupASp!YKIHjH23?1=!syzuliKKxP{$r1)qd#RAuZ)BTo0rRTH1K zno{Lp#8w#@Gu=<*Pha1sU^wVWBa?h|JCz4A58)GJea+Z6ly!>H(+syb5zj@max>D2 zdVR;uiFCiF?Wz{&BO4jCxy3dLRM1cilYTnClsh#u&UJKRHWULgNz&--I}{xx0e}54 zMW8PDyJa8-GVkz}L%G%ob9VNjAYF)(>fH=mUZEax4%xu~#OE6=Fm7f`Cs*HDi#8HE`Cj)$Mt0e4 zs(b%n<*Bc918zw;1Cs3lPxL-{%k(HYYISW8TjhIH)4Tot>TZe(^vwK$JxR;p$JAXE z-%Ah==XWA0B|LLJ1tWu7znSyV$23PAk&1Q#S#L{2W$_ms+>hi#o93@6hA?vM=0z<8 zt+Ru{Z6DM6`&18*oHyj!8t z*7^K9(oL?unasJ53^ZtoR|&Pm%58Ix*(KV#`C7bW7Q!<~`gZ+ie^IqpV6RZ^AX{&3 z$S-Eg4pAXcPP~EO3bYUp!07chh~36R&UKUNPo_0y=k=GT_59cuiwcX z!_l1;kKrXY(o@zS=5K(F9C-teLs~llkZ^q*`dALhcDT_26pY|K@eI?AdjaD2OFsb; z6lO6*hZl1Obu}d9fO-t`M5Og>V`>ZzD&vREdj#!0=%Mfh?fsQ9-;#~diY0!Il0W;w zl*~lA_8w8iBEVlEOsQmev%)ov+!%;-BTooX`!~%Yo2~|L(m7+@+=2ai+7Vc{W6&G8 z?Y43_7Y5}rwemBpvDP1XAe}RWgz%2ODAfTyVvBbrn^f_dzL9yl4V4(Ne1mi51z{(# zP@*SVYk`NnFiUp~Em;$DRW@_XTgxKGkJ3l#1y)W1@5r^a$aRLGGPW{P8iRnn>e*$A z$4q=9-bbXyp>+&^-n>gskE(zRxTpzk zYy97xhSL2$MxmIsB*i{B!h*OQyRFUCnn7a<=k4bN>>feE=Lz@l1iJDJAoi)d^HnPy9ih_bKe zQEGvXbR1!I$iubIWd@YUsv_KEKj_!$eDPS~3F8=sm~erI82bf?^6$7%Y|Pw=wwf=V zYP8b257Y{HnCxk!37R5Z+mw9X5woRwBiS(^VTwegGGY!=OqNPXIK52=LBb;722l{< z0E#5=Xo3;}0zyNTy>O}0^p>Ojn89igCvbLIydI(=mY!Ov+7e2(^USHb5QF~c3Mc)- z9VW?gAyfkO0EsJiywN6}WR^*U5CM%)JbY@BMh0vxSo|p*V2kL#Ur**F$_w&~-B&0(&DTh~{95AP1L71o~r!0&>=aT@IIk z+dgEPkKENK8F}|7xlDxLhG5oO8!wS$jZd}9vPpXlbY?Dlda91+ne6Q?pm{*;G12l@ z>Kb5HM|W0GJy0OqS?~^&f8u7sDwY-Bofy$!usCTbYefgLL<6ZFdS{Njv*<`mX&zl2 z1iI7%?Z}9Edm*XV0Uo_pt{vMoA7%G3fu$Z;^lJ%D2=)r{Zfv^t?&$ooF73HrR9hO* zNEy~{eV3TrWbkt9mMBJ3_@vI@KH4$CoUS^H*wnh|IB6ZKn=C?)B0fchNAt_9IJCYq zx49y5*(P$`zp?VGeQL%u3H?4->so%FK@HY<=(@Sxwd&&J?&;d_r0sa3Z7}oL#JrPi zbO0xpkgaCG4lgCb6;5-XrcacB%cD=SgG=it0`D3$$ivomN+f>GRNn>O5=5f4uT={s zjXEDD3<8rBCy>dFYv)};C5W|9E)00vJvjZ`59Kvnf>mT4Wp2TD+ki=XVQ5y}56JNthd2L3zW^PgmAHU2w0+o5YDh4|4> z!^aBbh}9~+$|Q1WjlJQQN4%j=5RM_t{221}j9reJm~gbR@bx`>~Gmm7<9*x zn4X75h(fF-mNYzd3OJ0Ps1!%%46k0yHAOVsE<}opZk%rvzQ-f2h+;dj|Ik&e*j2J# z5jQS6?&yb!b}zXt=LC7MhHLmGv+utcVZ-IS~-ysx%dj2M;2kpc*S( zJ_M+ZxCihaOhnQnix^HENj7UV3y_ajih@*?Va4^J+w1PHE9H~3_iFUIB}}fan3ZG7 zu%}Aj6_P|kqi{$;%oA&PE;(|jQ@Z#HE+?~#`5Tv9A5N$ocoo1yJSm7SS*Lu_GoiMQ zA*|IH1s*9xTk}x? zKoc;}Ga>pm==jCiXJJ2G2XjRsT9zgBwQgSr41zdoXgl9zH zX($JkP;j&a9|%7 zak|`fZ%|@|ql5c}Ux)L0a*qmrPe-8Y+u5|j+2Ejf!Wu$`>A&i2Yr+Y(8zaI@v%p%9 z`_+8?qw;I#hClr?Ov;bu`%$7ao>=OS6)p3BZ9?`K5&*AbfETZA7}Wp-CM*R6`zFn1 zK#5`HA=ax*{^t|H zNu0aw&6n7(P+m~n4xBnn_%&C6Lf6r~6x#JLiwWn)dd)0#YaVTTfn!VUyV zh$?C=6?4NzCPylgiUyKdV>^Xq@`DkI%?6{A5d)(BWp|-IaFcea}Fbd@el0VsGmYN;K^gjJnkOqpLy$9IXc-F zZ(C-vHuuxC6p~B=2;60eG#b-_E?tv#fwe?yl*c|LBm4Hd8QHd4>*lplOvH7&BL5a= zq6!ckQMZ&A#kbyJM~0;)bpauux#dnMDClSt6HeU(>#qzPnTO1b|oS3Rk$_#0BzL0@61 z)}(B8Q^bb7G#Ka*(9P;MavREV#UBr9N}+r#2jI+fiBMk+X^yBU7~ZRynQ!jq>2}q) z#I91Ip5u5XwIp#QLtbQ7++B(v823sm^Ydn3+rC*#ol72klrZAX1YmxX`lW{~Pv`!+ zFRpB~w27g2F4u*ypy0z=U32v4_@Qf@8KBtKVYo)iYv6~W@YdU^O8M1{7y~cTzWKG- z0G8=->3uO35zQpX)~|+qPoJBYc8nWXqv#Bc-@B9?v(Z)gNT^9DX2zbsIMlkXaoy6E z(-HPwAriw~fRjCh7pVa!9eE%(6T|Z2szqLGAK&D}nb7UH)LLmN|AB&X&fN_2BGEdI zZLa(Snr!`Alm8|CZ54aj+H$!X*ocx!`E)hZbJ70!Q}9FW47CZfo9W>317|IT)6lq) z%w*TvPK?7wjaP~wJM#$t#`q*PkTb-kD)JkAP;qZNUirC&>J{{C%A%MYY}ft4mj-@Z z{{DEGTA3bQo#L=5`p`{E)=AiHh4S|NqTO+h)<{Tz{te9zG4j@T_4)97>paRD^qnpC z<>KuIea^d$Op#Z5M9(;C+16S+ig3UnsHNrx12^GM00yACJy0EijfC&CJ8>|bxvi7rU?WM3%N z5=6v0-Mbr)Oe)hs1U0F48VM(P$}M_B_=iXyDIODOi(f`r_=bP+>jDQ`>#-KxebSOu z^(14a&E7Fiu0|{3s10j9FUwpsM&fGI;wkn|+2!%jQu7e$A zO89XOvUD0?y~G~I3bR_ZP&lJp!Etp2Q-_<+)TJMn-IEVivd6;k+U}C&w=;paPAB5* zha6=D#PBFb)8t-ws>ZKF4MuAvP!wo&h&)6AF_QLXr-+Tt+C(2LETYwDp3q-^z?=Hj ztoTv5J7`*9Ze`_)IPJ$xR(gx83+*XMGyj&<1Q=x2$%Q~tRPd4FxRiVpMwe-AO=r^J zOo9qi$j9&0Rs}LlTNvg!D9R5AiIH1n5pjlN0N*i!1}6R%@PB{3LB_Sh&-OWUZL2=? z%Y3$YjdCLk{#-~Z5LovgD$ib}d48Mm@{DLnF>KP||=KhWs^MW(dQ~QNl!i!corMvzyKe_2e@Bu3@EcvZ#$5jGvNQ{bX zC?9A2mW3$7oKa zg+3BqXdr3Vxp+eS_OGusY;P!2P%`*_k54jEL@IAHCs`HKi+{$szrEFoC^%-5xq$|m z((@M!*crfr)`gMVd++`0(R-Lx4W6B6DYXu1uy*MI14c!@o=2y3LRAVELbg#9#E?Wf zpGAciMF@&Cx(={^5jDgfSa*B&2$K`VKSF7w_~+W|YI>YxSjtkezV?n9#+GfKv#YX; z=eEODco`rWH8t%3+72}RmkhbSk<0<4TRZ1BULR@_0!N&%dpRF)Z`Fz=Sw#iP7S2I# zl)bUQ+JQ8NVAvwl5%)Cyq2cT;!hltBBS^cX8$GDld$*1)prjK}RAmyqtIU1SBWVLQ zq}B|ucCw&rA&&+^1OXCp*ufE~#0|Qw5zZl%V6B75PTUa++513wdva9{S$98r5_KdE zquzIs4d`7%RnlMgdCB=D9+*~L*BTPN3X7X-@1ar!%5B-9CVFO?SSo}WsOv>qrcj>M zl7k6=#a69mP;KRffimb}%|tSJcBtiTjYV`tqeV}@YB)jkveD9W#4T;42{P(Z&K!*J zEMb^+*|W1fe8JMDK^3OI@(*#?ci~76qr`U4;IKW%T>e_ci9OzMAC!hnxr_pjcV4)Y z8-1&-nCPE6GX-m&!sI*RUlgUIIOC)PIyY!zGg0*$c&S1|h;gg04iHP7g>25Xt^|HU zh{7AdhxWQo8Ya{opPw%5qCN%7rU@S@1rSbZ)+&W2!HJ3@zmR9=H~50B>l%< z1TN06HHs13Q3;l|&=6Dy-hqf}h%0t~?X@?8g;`?D+2!{EF$IEVn|yxHmI0EHO>71( z*~=wxE@45zVC?O=-KsZUFs?u2d?S5^pX$ytPub@H8dQ=Ae!vp8|B5e0vq{=AdV*nf z>s%Gl+O3=>;V7f6q2#m70e&WhB#OGv#F7gYC9J<#T?k6Ae}R#Vy6gCQf*2-MV2oypQDgCO@u3F&&VJU}d)A}l(G_pbV{MT_D* zCzQrZ`3Ma7x&0H$!Q$}xF9XG}@bkjLi+L3??ey8$Tz^`PWcjM9CLMOeU02XC_x@g! zjRoSsOwRff>v^kli}2G$l(jH)>1=pR=@AwK5yjAm=uIX@3{JvR*_kVQ6FO(%OZ{?T z*HA5$(U*auyPRzV(fSK7op>&tJmek_A`88IF6u!S7;?ey12!=CD0Xv}LadpwLl(~J zM{)DdEyr@w*c;iGUS@Brlk0{}TCygGCJ!gRMtJY&L?QpRTmp+vZafUyBUPbcN^|SC2DM=ME5k+FDtp~^Tv0kc8sfR zQATg>JH5%FHmCKK3i;J@q(hewhL%Zn8cC2U5TceYiYkg!K5s_?6sb_!?3f;Aa|{ae zV*M>XwBH~HR%F!tjfjcGm75W$#J>*S0u9nU@YJT6|C-7DUJtOzusmzr&qM15v8h|V ztzeoN+mVUX^=9hPg=>KwJF;&O?w1rlzm~l)-Z$0%asJ#L4Ce-`%Mq@4?0S0Xgfc?y zIvM0EU$4RjR_bV8BS)X&WoHLqF;sN8_b?qLtgomgOU;F-2P&?`Y0|k`wqzUl{xEpl zR4T1)0>1ZxNvE%F-i-h^Z#>Z|pL%^!F`U}onMa4`T7iQt+#+^p<=HK}aJKiOcE@eU zR}Z*JTYkMpa!nL~{EpbgzbUdY2=o#?TP3wn6dwVg1vNX1q^2_PD*avN z?P$(D%NVMh*xWxi&wqG9*6;MJdiWValVe2HtG#hhXJxg~;I8Ju_0XoWxk+(}RyYc%v*8rC%IK&)d$CZMJYgzHhLB70O`kC8Q7IG&fIWoh| zom8WMPU=>q%WkaLWxy`5kmz^zuO{Dg(Itnzf8(mN1h;Cp|4hc3h5w)P+T~YMXR=#fSjDq;`F1F>K#I5wLLg#k3*+F&`&+_q`vkzC5F6YK zupxGyuNhY_8Wy8~RPe(p`DlPBcxp=;Bnyg+TO_(dNML=3n6c!eDtW(bVK^fb({A<- z9j!#-p8sCXRnq+j~F#{dc$*XU*Hyq`J#EC-6?IZxS#0(kvz z(+%ZuIl@3%4b_Nfgj*cq=y+%VmXPd&$Sr1BlObO5>=60e`rG?C210Xlz52xyk`Rh% zM<@q+4{Jv}hqyhe2C1snyw#y(X_RvWbH0U^gs!#R;eoS9Y_w`(qN%}rAQFSo8$_v+ zp3PvUaBcNxMukB|W5;8)^rl%OIDGhhr|@cehIP+e=p)FAHC5vn<9rprs{>}j34P!u znE)xNSwVCOc_-U(Vmcys0oKSk>wMo1khyYB^J~e;5EK)RDknp*asMhpYSBXR2+LX% zU{RWpM;KUMkGdn{=lR zVJ7X;(yRgt>i*0kr0e5b3}i<@)WvE4x12yP)V?XIfC0-5`!vep$(x5V%_V0INOI(S z^_*K!6RJL@L_Ti1&7S_1jj%p9Bd^AUp<1UIJ#EQuS!C?#w>Ib?YV|hjYAm#l z%RCiTrm-dmr&YL&5a7W2NcB~BRm*5YX3mI>w`=e4{xKuLE@~b{nV7a)=q#Gf^Jaej z;i_GxOI31_{=0X-lu|17g*>4%5ruWV>@U!zrvy+InYv>*ZlML77n&H0>E{{yS2*#) zh+MSsdwCUmq~9bB=q>Ep>;75=NhavvWk?cjUxIlGU%(hE&kfYKQLF;UH3PAr&qH>y zTUYt%6Wb|=c6^o{oGF7(XFa7Y<6c2XE%;hI0Wu$rXIQcC?Ih#SWKZDDW=O`deeZz(EYI@y;My*KGS+85eO~H+sq&i zWERP>bf8H->}1|(`O#kjLFOw!p%^|ohPF%7piE^$qfkwXB}_N0<&kxG`hL*P@`Deq zTyDk+!_v(G_W-CdB(PNSU)p;Y6VasCXB=&xSq!PSlVU!HmD@Y;We6(gZw-mk1Bn%G zTaFhXnGLq=@S7mNf1bz=K^05Z-6>H4JjCV5$_?%jB`5bG6hr%qDnGc4-r;;y@8b%d z#Y>JB443qGNsHxfBA%Ql5A^hV)j}_e66Q87Cpb%`GiE)nosE;ne#xYZP=Kn4gt({p z8{tI+;9jDxQzu3!=jBHwgzKxuyIbYw19>dRXut~S=PYY9E1cA=lvm!=(^7mCdJ~LU z=GDk^;^!WJD;VEe#Uj1b(7evPPCTao`<5~@<&Lkhp#$QoNI9`gSZe~z4MsO;E@}I6 zkrVUn+}$Kn6i3&qIk3`i0NRuf(GmftyV1B6Nv9e3n1>Ytxfn!pnP;?Du2`g6ei8~(w>+(zHZ z+~YsPZ7!?s{uKN_1QT8SE!0d9#LR&_IJNPWP=406j#Bv`t6a^}hGdC*f#=m5r0=AQ zX7Ns%YFq?y*N*pV&h2p*?NxONJq!f`35OXXcV5DIt}M^< zjH>yuIGXjmUArI~x{SEKp&OEQwFqzja!&45Ay1F#rP=8vxNIgJvXN`Z(j)v{H56As zMmE*Rv&nfQh3Pk?eo*ja+nxdC0E`*$1c8{B8c0_-YC`oO(BUt@#2kStW*-qLFZDB> z?r@)Svb5ko=`JpQ^&AQVWM>Q;S-D4wKK_#g^>LL#dIqVM@dWIiUEw~L&OB}`V{HSoUgqX@3W8BItilZUjix2ORKAx z0}X#FH-zVEP+$b2*B+tJ*aggPG89HH^k53rflYQbf(?G?5dc+t#qKY!#sC7Ohru}_ zT2DTz%xOcs6I=ywX#H#*)CG}gcA3=x*U$Wjy&@nGBSvncUW@q1F)udi0)KOUh`$-= z%})}3k_IA%5fRhw0J2!pe!CS{LtWA%65 z%3TYISLQfzfI!I+>lmLvI^SO+iHkj|KG7p6qCBKTSE7(0Wyr8Gu?04!=%S~x0K797 zZDO^SEJI$jPV@vUnfNYfFsS;E`W@1cEBI~p?)}}Fkuv~$vhH%3&1djlR=Zm7to?U* zE>z(*(ukTZZLjphfh*v(5pI-gbvn;crd{}Jr5;&!-wX0u+o)6jwd8?Q(=Cd={@^#r znt8KSPbQBS0`I>Jo&||2fA&^4F03;`t&(jE_O^!HbRjN64se zEdv!=O?oUtOpwvP|4pBBji%gS{GpKX{|qxl|L4Ek(b&qw(cIL=*oel?*!ur(l4*C! zI!vWl!oMIZyNkpwbj9q{#F6XI(=Rc@ety=CEiiB?P=Wk0|UBQCkuc|~CtugkXzL9_^R zdKPia^4yjJ`~n@O3Cd{lfR{dKs4O7b*Bf#b#cf_a83eoQ(m_`Bj&rKA2s{mnia}Wk z8}SzK@(??1ecx2d8i~_UWei<+q16?^f2nnr%!(!F21mnMefGc-5JPT5aBo!seF_*y zK4-ylO5T9zvFF@4d9Ko)WL9&!4{S@eN$?vrT^kf8`N}FI22H%K-2a<@Lzyz|x+-%^ z@AvUC5i0__wt^I8?uPi+g3LO8MTzL9duq6c0W5w1w7TS=c|Hak`HjH=Im}Obec4|^ zN*zIkXQD>o`CdlZSdHPfG^6yq@xdu?E-LAf^)_;{Uj}_8RP1TU(M_G zrDnw|Hbe{V3}ZE|^$58Se#4seSn;&RC(>?MR!H0q?F0O5B$v!(*wuG=pI zXIJxRn6YqtGkZJKteW@TK$ekP_BJ(u`?;aAf+eK`+V7(bNLb|J6=CA{Yb%W zL8tS`UUofG{;lXw)O!;Ip9*E%O7yMQUk;VsSwny#>S*&_kF0SmJ1$dxybEZqZM1*{%cH zxoKnOe96LQLhv?3^DD{3IYVh@@UFwT-{Ao<5Xde*Z#Jt+kHS4i2!XJFywq}|CJN0J z5VYcavR`Q^G!kNp?3l$^U8qXnn=NjBTLtd-8V=czrY9La?W!xBH(5uZ902|RC;WY( zz`KisQ;JU)AE`|tL^dD^-T7#?63qi)EhL_@8jyr2-!d#El3qF1Fgy>v50@mpX?>ko z@aUy+OhG(~9XY~QKct!BoN>S7CMK2=>O)~y?|*@o*yC_gczjZujO`p@HvaZfB}mgx zc0(^?5=jKZ1~wD0?}D3wlAWhWNqE$>TE6?aN{Q%rC;>;}JL?cA7&|MT<23lfiN)+y zT>}D@BBlZ^3A=b@Q$GVhZ%a?9=vD?J`T;t;5ISsmi43_Q{ zmE7(fIBV0SzliURx84X#7~0A&tNuArj+y25OIwO_d>CnHAOPH0!R0{dZ|J#TY)1KrD*v46)RToB#c2EH7e;8RR#& zw*B`vx{JobTlzFF+?VJavv#j@*y*?Nxv%<4>^`355agHF!%1vzw(2L7V6C}E&f#}? zxGn-aZAXuT;!CaR*WZYIGJ%r^F0)tU-0kXN#8@5jyp{%K3#r9R#(rr>(kxa$-lj6l~x2V zBLo@oS8zJu=Tp(sl;O3vZ$K+3^rq@rPJ=NyTP?ygYUuUzWZXD6E;L`=3D<6>Zdu zB`)R?Q~90-Zbi3qq-1~N_Z&4D*-9$ngcj!;A)mw_=iRErbvL>~e+CNVTu;xTGZuJ zsrg_1(|jJDxOEv9n7W={xSN;wZ0FsKwV$+?dB!zM(k%|{ytpuBb6o2O%J=LMPd?U2 zrvlvG^UMjTDLnGz0l9I1gRkb1Oc_Clv6xVgc=^TRXa5S#GmVIm7e|dB3$%6fA1cEO z?6hfkh-N}#=Zl>4q##7DH)IOE z_lbfv$@++E3876b&NwrNJ!YXY<}*GLvfYXK`ErYokwH3Z=n!yRH^OSFo{%CVR$yJ; z>cWuRNjLt5%4Kn57xUwePM}72;%`JnpSB+4Y}IvAKYt1CTDjmw?rad~63(5mdEi$( zo15-=1|p&D}%dW@KIlN?#O}QN_-nxG?4SWi7nWk1fo9`$fvl!0~biM^E+zomt zn@=&>G`K>A6$ZS!sskJeLdmIL$hS(NqlpAZ)6G~M7i0#U`L*FT9~6)CNL!YTypeju zjSqaShY9*@WFIV6<<3NtcO^eB9oB=)++7N!aGubWmjs%TCC_n9b3Sbq+0%+K2FY`O zlQ@YZ4tgY(5ZpMLRv{F;(do8@if%Bfn^O|Qb3BWL`$_{DPq0XDGLl!}M_1A|7Zp$C z4s%{m>ae2{>rKy!G}6&tf3A{zOkheViYWh9u~gnD3B;sh1au)K6|J#FT9u8SxMD3~ z<4k-!6wdSIzn%Zg>9}D30rtPP427>!j7Se?P5OtlWBarId$i=gb9S)A8CCza{>>1q zr~+Qbbf*9=CN(1?C3`GEIWemo7As3hO-?Nl9jiJvMLkAKAulFHF*8d}Egl_ui$C#K zufe!ncsTvlT}$ikPL4#blv+v+R{V!V_P-wMUl5>`35?ht5C8xejQ{Pi{sqvsvvv4C z&8ApZlZ^eDP4Q!jha1;7?t_oO+AxquJ);z!BOr?d@c{o7Oc}j8B~c`x&{^r5yD!Kf zEZ@Sq4O}J@!t`1C-G+>}263z$Pcb|62(ZxePt!JQvr#!pdZZVN<`5AWKff<`xO z0d^$?9@j9A55;)=p2kMwLHva%%ZNFb7dqb=Mc?M`UdIY^@X!Qf_#uv)peB&5p31Q}K z4B7&*HL|o=76J~a25tg(+_`BJLs-3NE9q>t+LiFOw2&>B0{W8YE$6HHThIXhxJ8FT zT4AWIxpTCptd^5xX5XA&4}8@qO|k?@s#3`zg;)z*Q9N>I?cdE2826}0`o++M+XA)_ zqX_=+)d+U=q~IUd-U2qir8z>;1Tyyyi>=~*7#8@93-+s{$!TU^9%dEk`-=$<@>oHG zV=|sjvle)dTTA%R&N~q-nzJwA)QgCV@9RV2W9g=(Md$?4*xDa@7Xs-8L>@=5Mt?X4 zh1A*;g>8TKo7E$9%G=y@mTqPa1WTqBfI45uoSg1Y8Jyk8{?gn5_!d~P#o<|&Q&JFtB76Y7po8{ggGtY>Tn{~8qW_Nox`pQIg3U21g_z|lM>1DHZ zQj#&ci4$#DSmLG(tw-t){NlKW!JS)GGx#Ddf!jI)gF<2@Z3t1rgHKwjc+}w#qY$($ z|7nlJsR3xSisrxmZDbJ<$zwODn3`hxm`KAp72fiC;8UgL-h*t?BMwHJY>ZGr@n~|4 z=$m^Y7jZGuLDYv|!_>hON@qP8)-r}xA!hG{5LqX)vCj~G`y{?!<=bL&N6D86S_xLr zmO-Q~ae>t7Go;P7sl9o+XDFD+6So`mV@9Z9$1T2Q8r!g$6T%koVd0F;6j`GgJj{+U zkS=xg=e^SA?$d(N`~1PyO&w#v2q7_<(A zPaAc|>{W`UU{%^2W@r0Us`~rtGC3_fI9O`F}1F zM=YT)4q9ZD^T6eD(xKlbskVR%3{R3|>w=@Wmcwn0+NywuCKY?R2NaQ4{U8}Np2((B z8;#f;mAvE1Ftl<$+Sl)^Vnvu%WPJ!uDRI55q%U!#zsY65mzSOt1*G+5mV-ruI)uAb&k9N8>-%rx`i^t+5W zeGx%oV`2ux3KSc^16@fxfNq!EmRX_A#wQNMPmLg-BuP#J)gdo)*BrYRW3ps(RAv#} zd9v1|zDyPWsuiKMJRDu!M6P!pKB#xv(V;c#J(3q%Ijmgex-7|NzqcBol&*6f^ zlq`xweY;6L0smm7+YUOHP5*fgD83a2E&eo67vKN@;Q#AJ@!ze}|3lyXN_9JOQv}X; zRhM21*8)|uhDYs&pEraqk>Dv#HB7YHEj)O9Tv1E!$ z46PMwQpxujKBc#)aLuVB8OKf10P0(Xyi~uDEvr7-fLBAc0ywWF<7}eAy&+cbh zXchE(g7-DTN{+_JRV-JAu2fpvR5`Q&>9R{|nUj_w*^_dBnhw(V*Tf*zjHt|u2bR=O z)Kj-Gdw*)4ygryTdA-xksO-phJg|0D$t02#ENa<(gHt=i4M^AQ8Q7j_06oixs@by_ zxPosBwjo;12oKk=g7~X=b?Zl6?{+Py7m8*e=jH$dGv^vgx_Gn%Or~x{7IPvS&OCm4 zWV++Gp^Cr+w8yEm0=1$_4rWKEH=Ckl|7{RRS>}L9@i5~PopsrTD*aq1b8)y2K3TWa zP7lbmqk=}-OIpG5`2Yc@uMok&^|k`btqAwBrm1$59r9wXzEw6_U>D~9>c14-AAFCD z&<^SE-M5;XxC%J76oe{Otf3H~k}{|VM7zWjl9}AbLNS4HYg`I2l@g-&<_O%Ift0L* zbVnHG6!S6mIKiz5Rk!L$tT<{%hvo8v%;WRQXzPN>&Y20?o$DQAU0g@e6$pZ=rcm_) zZ`z&#rbp||MomH9z{31wQ|yoCKrknrphLULtv1M9A(`^t^EklTWv>pa4ENy@VBmdT zHbMEq9oku7#wF|i#f2b~zVyGaa*RuQLHHSo>Vs^j+XNoBs9y2n0lPzBDq{tU_O8hO ztT}yZv;~;^GeZ&$;3j&G1{m{Ht2!H15y7t~t{_}iBc&?i) z=2spR^2bO>jABh^GJ6qb%$n_mjlVS#GGW2`HZ|jDw0Tw75j;3D>}kNs<0m1TCw?Md zPcQEH_gp;o^C}vC&gl87EOqjkw+snJCcWQeJ~n8>yrn?JKo)o^csug=e{BiFSBuS1>uoM9xmtiHb_&tuYLDG$0Cpr=W;!VlHp z2Y&nrvI+diWB7t!X~&7pf@@G(KOr}i*Z#yJ^Y{N&iT{O)Y*MTef5pT%MgLd0{wLu7 zJ3IU{m^oS4+5UHKAhKju`%nC@+Eq{Q|GlTDN5D%!&Opcb?;-ymn8-GmB5~#LowJC4 znP*k>KRC+&NALbGSK5E6(*Ao9O(P1m|HS`V%pWp3l6|8RrOI8N zWW!-KQ5y-@ZZa!Sz)_qK2oMZ@Vte`L)Ae;4;0Z`%GSUtHU;u64A~)C9P3{QO>u1G@ zjULUS7puu_DQ&KAj6CST-d&X@GrH-6tU{x6*VudJsnZZrt9Xi({s8(*^$|pHqf`Ap zddA&n&Vx^UU2Z~W&{uYd-8S(#seVh(StbQ zC=$&n*c%rn)uh4Hq1*bh^w=SKn|3mj)g#jUkOs7uL<6_ML)c-$;L%CSy;C1?kxF2TyieM%nF{|Z1`7xTp?-hux==mA5x~Ms zQ*os@1#T8_*h3NH0`TNpuQVI>x?X=m4e%+Kgi}4k{HY8yu^)&L2<%?J<-DCf)*)cH zNhgpFm{KQNBq4|=u1;0V`7fwm;BJ|R=2*ji#fLJFR$wnZn*QR-Fe0m@f#G--sf^C> zRI*vrgcJiB9@g8r+T#91vvs>RTriMI3y3}sAlsNmg1dsUN+zMPJ<=p1=}v5p0P&0GM61XXDN}Rhd}=_QnwniKDC?Sk zQLn+;6E=hioHRX3@q3KS4>{T(CClCibbt*BDv;jnA!m~P-N`I|$ z#;jnFdCepGAN&D(_mE9U-R(ngNoj^4nsWo$bC~lN{)4IFiE*Rfo4Y;V&zdfUGH@g3 z$2F1(D-^nG)oEYgWSWe*4*B)0Msrp6;WS%hm-0^{e4vUGw27I>!`2OkLG1>B^W9cB z-F+!D1iUDot;P-2BaC#|NSK%$e=(9dDhbRTC3C00trL`DxX^h*7mMM7b{2AUpF zMfoX5gv{f`Q0pY3*AR8|mP*mujEfIK7brmjg5(ig3Re1mJM_ng4uk*bsW`x?jHyH5 zL2|<3gzRF7-|Jngb7^s3MMl3CP-&9LueybZZAnHd_S9=gYeT`iH*r@)jG zmE3yi`cGfd+yo%s2z?tnc&u0mhDBFANi z$~8R*%AhAgU9<-0*r%z<@%Q2>T=VGs=r|k+u}-p3^RTOyoE^d0wiAyyFOK;KqNCtF z=;*}Or>3;RE&)OuAlBdFii?H$C0BE6k8snA_-=UhXKu^tR^q@l@X-P;u={OFAU*Z)Dt+Ll{VluX=IoL zv6&n2A48ozAqKmhgJP?aYz-KoNO-aOt z+xe^D;qTd zq19(wo#46{2o#s2MLV*zwbws_3BxS>$*$S8C0Hobg<#W7;qT)vVVw6A&OQfY_=lrD(@H7VcYu`X< zm+D$7oK-lU{Y5-j=KGCcoqgVE$|Q)V1qK#Vha9$)M{L9>aP0&X{B550nS3jFOnC1H z3%&a5X28uPauRb`40(t{O@BkvHH zILwnrst-??8m0-$p}4kRW3*COX`O)8672KO4;d-0WS@EPrK0{L;rjqtlp-hK=jaiV zXU5VWVBI;U4Uy~#48<$e)V!M+u5#phNGkPhjTi*Gfd{~7b%6Jm@iSZ`%!Vw((B`bZ z@=V_`qqPM3VneyA@KbyxRlUBDDo0+y@>}?J>2rxMVLVYI!2@l<$;l+UL}kXw-j5Bv zMUNy(dvH+_l&3xdvd9WNk57Cl<9yV_;8BH~OwzfH7xY937BuPg5dK|9H2VD9k{3voe7mP$jXdEZRQnMA{xNd=;Cem9j~xRY zMn$)z^@aqTD0*$5%-HkVV!D<{gupQhDn%$`%oVS!ZXtI(-gX z{Nst%eyaNt6**uba4u7Y!5R=$Si;S3zbrg1eJ`p2x?*)`{>lxtBYuS*x|0RpqBKA) zQnp$Uy6}<6KToY2?Y6_Z+140$0{Uc^WynjeY#^$JRTH33XyR6vs+R4$-v>w*Vv>N> zU*iHhKe8NulMc^^IWui3Dhb+EIm%;CCLs_#3_vY#XVEcRax_5iQjPPo#a&tg&+QhZ z0;7q)_kE6`kg||VUO4(=W}xf>$-32<9>M2;4ft3-COqRr-G*r%djy{3@@c?J!=yhYD+JzGxYLqR;bNn$ONidfhSr};d zKU@>@BZW)AOqKPi@WEnTJUP8xJaKcfb+xg&Ik8_Plc6E~4+kS}YG$Zm^|xUvg!V)O zJyH4c(Muz~TqMcn`y)c^I*xr2&l_W6cDv&ZggaWF6*scjb%f}{j6*M-P}QK&p>bOR z@z{;K@r8cgqGq-YTQ6K_pR%u)=a1kBI5;;mSxZM3g5ZCj+T7j)%>=cCeGqu=F`jCj zUc=BZD$+k2%jKXShM^ui`7@HDK!QtQo$<7Y3)7Ug+YNuUFI9dfa+q{`DX!lB1`mr+YPR%Bk4r zhDWPI^;=2Z{DU41c!KRYGsf}?tJ){x9kSo&&;u94=-w`?c2SV#TjKI`o} zYIWcleBGjWCiY_WbEiy#+}UnuOkDb||HY^8|LN&3Re~TLz&iu@(ZCahQ_D{@>wWa9#LRJ31OHGK(n1bnA%{4d%a)9b@NXo;0Ni6m4qE5i;Hu)!s={A z9%3clz`Pu^f-55}7*zv*67d|kgA7DoD-VT8xTv&ZO~@_tNt0eT3=t_w=$C>!izxPTBTxJ~HM#@g8zxH+#Z!{az_o z-&NH=!?ZgILVIU*-$%HOw=TJhPA3_zWT)tW8jt}!RhAT)-}+jqMN{4HiHgRj$&3Iw z%Qc2&VuO~!!%?Ynp|^<0l3ql@CdKH=ig9w4@2uDfxSc|ca+$&j%zD`}Qjl$-LxyyOnHGZue`^w0_1&^YTKNg^l1e4RK!!SHN!TXU^zt1>-!D&Ppmep zl-S-hXk5O|qkR6$?g#khilf_s-^%pxqM^k$FaEvhx4JAu75Z|Q$HNP*Kn*q7R(GM_ zpPP`%td%l$RIoomGi%E$T?UrT*=QT|3sn|wg5%Kdx;%K)#nko=d9v^$i}gMQ%|2QL z_tPFR-F9!i)~flNcas#=Ws`vDRI93 zhqm*9qeNUg0+x1H=wCyX1n~d2HU6jf;=kJ(rApLl|B3(I*6^`%+Zs*Wcl(BdX*vsj zo)6E-+Q?amZ6ua*EF4bcNz9gBT>}puFAYuy!2n3Eo;dBZtXKhK!bEv z>#MA)s=RuTczm8%GS-}leDq}ZaC(2d;6F;r8PK;^oLPlVXjV~7t|@=A$l6A|kx|t7 zrYN-mmP_J!C+nL@x{K`8Ig2_b$=gLK5kGZlk$%isd%clS`LJ8c$+>&%9Irzwl-SJb zmPsA?E0ZOS?%^~<+ssb%e%h5(Z1JSYmYFuunH|xYq1v*Jsn}HZj^vqxqkK|Bu?3Ww zsUo4#B0*ca;z&7(>Dr4duH=w4A46SDuzynMPLK9Ve10TI`=~mNh(H1!iUw29{^?Ubw3>)hB%~ zXXRBT-qM=V!mi3+uXkaVC2!J3#qEK`)cFOg1=Da}56&r7e`2G+068l_3I)@)cOYW<0wkp9y$1hSMqv99H^jC68Cjb(%fGULI*v^Z%`nD#kb#J%#Z~f+`r^A`4G_kXdk%wR2KutZK z&$QnTC--N^-)-qPuk%A7JosF{pT0%2`@S>a@0Y(D@1)%yUwJ=|X?^Ko{Mx=g zu9-c2{GW07*GPKjNZ_A!x;^XsZ){y0T)%KwyZjs+o<12mczHNEJp+fo%}*a;^t}^w zy}Z6TxY<3PF3$EI-ma28;k_94?^93g9R8xsLv2A`3oxS-`*^<=7kQqvV0ZfM-0d{- z5UM`8b?E+Z?}iR3a(ia)&Qz*4t7uYy=}gXklPb{5GsvKDo8^F3 z{>uhEQ%yrnJHZin@qd9e&I2a_$d=y~0Yu53D#0Y>Ss)D@GP@C$ym!Kj{b6?j8nm~O zIow&KOs%%(B%ragURYOxe&dDDCYXK^^-*P-&_G(MZxvZVcA26sp<-ihQbR1QdjfBl zCphyWP^|$;K8BD)xn=dSTP%%Y9KpynxjkLfTQIcteK6#OQauSy#2hd@VUp zgvt1%Uv+^d&>dqUIb{Ms=dU|VoMPOcpPKaSOQqBHuQLY-Th;qDK)GEsb_SU_0#FIx z-rwni`nYDw7%HZ^W1>xjPe7c{i<+~@bf<#Y{So;6;Nsn#FIIV-Of$MBj)_9}g)$!) zU0sS#73AtMA$bFtv%~-d`sS{XQq9Hxx>rODGu+_(xOYw4LOZah4zmOqja!!*d9ETz zH3q^0l3M|dPm;81L|8D=>x<^ejdOj-Lpla_pJ>?l&2 z#FgCyP^*Z2E%=&RH_y9OpX0s{gK@Y9#33(3b%cGZgB=4TZ$Nv?(ReY^1=d>Fg@KLL zT?K7iL`99ig5JK}i75yD6@CGyI{b%)Y_8K$j5@(Sf8KJ7Zk!+OR^Mkh7YC~ZD0AeM z$6hmJZ=ajO;kci!dvP}6wPc@B>vx$At>o(a07=b|4UGDqQ9_6^Y5RDo0oC z@jtQ_OiDpCHg^RPv&r-w{Z$}GT;mtoUbv}(n=fZ`CNx+4TT%zzsaQ=-Pz!LYAGWQ5 zD^hw*JR|Cqv=-QjkOfgD@~`adtOX3nTP4r~c)DHPo>xJj@?S%HIs|sk?{WzY>;^nw zmKP2*7w5$RfU152c&@=bbc+gVc|q=NctzLeyQ>uGr7L5Pw1!R4z2+4{Qa3{7x{N5k z)N7YqskC|7?fobF_?m`F0E`rh)8ueeYr7s>uEqkyDV(1SLp zMGjQD_6NhJwbK5%n4y2%WZtYw;9 zePc^m(tr})=2MVD^eY*<0}P@3ZOw`@+u#XViO)lu{jh0 zpwxoW-@ldfQU+*Dm%HAe9oW!k35c(s8mzLv&U()K32C0|s4uK2grUON$j%-;$g5Vv^raK)0b+7A`j!B%={ykL zM;Aa%KW-cx1kB-15^9S92nOU}RN8V1@lK^Wn2yr{CO;FF^KE*evsdLatOmneHNcj2 zi3xq|hqoJm3EE2FyvsjqzwfbcMr7mnD=Jka@;NBwjymdY?Cxcz(_|Z98tpbSyZk;- zBe{bTo+@g0BJa*B)x-T4rBqN%tc+8{VsgyLYFAeU#^Mj&3TFTh%$CJ3+SG)ab1fut zmUaD%Wi7n7zJs&@B(`g1)(r3SBLxL!3R=#<-*$XBJ{66kLv9YHagDS54lLqoKgX|Gb@+dV)XBXOgK`9Z4zfAJQW+(&S7KO&j{qf z6Br2YHvF_UOf#dl*pgYK5Vvc_tpkHN6_+#V+synUl%~Af7(1oqX#$xGYqSw9XHaq>3x)aVC1UBJF zO(%v?@<<*(6d9PQ31yqx!^-lK6N>wp^ytH7WXRq-YueepoOm!xv}?jON3>EgbiuAJ zTxGnxqp7e8g(XJYLeQ3B-p{Ib!72Ubvh z{Jbb5UV#GV-ctxiW2Tj{5XWt5iPB}8;Eqobfx9kgw9PE26BEnLWwA&9V%x~5ckj^_ z*|P;xzqFwrUVt39*s-SG4`z$Tj;!b$fCSWLPilwO08`ID;?kk~3OSB>sz~J3}o&$EyCc|NNqFBVjlXX;EPSPE^ zejSo_L?@Pc#ZUE@m<=kilZ+e5t)})w z#F^Wqyw&%>mYQa{(M7{Y0P2pzl3;ol^Mq8fUeA$4aTz&lDXy_(Xzp$^7fvdeEqiBv zuYux?uQEq!b)IZreSeqbV}?m)kK)ygzJP#4nl%|K`cUSp0T6g?1z|(!kE0tVQX8mv zMvM3i-QBujY?e}rOe4R7I~WM`HI(@~ z)kcBu$7tf$B9^mypAWCt(>C4@UeEe$FA!*`c6KgqH6>i&4rI>e&zx}=;TASf9FBDVYIB28Py?|QEO3i(EAA!iYNUXt_`V(VDgg$S2KUjl6VHpHOu3<>=^ zTajv3kAg8mK9weKf>2G>$T>4tMf?8 z-~wo;@#yLqgxL#lJwT1)PiG^jIT17&+l3QoSq2@u z~NC8f}Qxv}& zkW%U8M`$xU2&O}Nm$}~>PIRCpI90lj&+~8R5=#82L1a$P+v$0}n(PStGl9z%{-`v_ z1M>I*l7SdDG}af?jVUVDcUP7KWkZA~L*e&r z&uHuhGAsa1C7R{s19CZ>GOSUxAo|-d#dhOgQ(Ig>U?-ny%i9GY3Q*{HZYd+>sWwlN z0H#GgnVpq$PR$&p6>pb0RhR0?KKQfSlY6Obc!Gpt`!>1pVNN1sUO^C0@MOe>4^-2R z`a+=fKbF6b z=bzvB$#G*~0J?oqU_I56Og;{7CCq)eEx3z_ObHQ%669j!W(O9aE{%1o_JZ`q z)jTh#F%zi9R}X`(BpIJYCJ{v+FYUnN;sJA6u>P}mZu~@tB9Fv@Y159ZzL%62U}xIl z(A3xxrKn;WRcOI}r&jk0zg@rB3H7^fe)Bw#vn`i+mvXU=A{Mb#rGhRCi+#zqnEZir^qL*je#S;Sl|M zBP8svgvD5ywaRRiRrlg_QiRJ2>&3v8KIdoz3eXD=`X{5t%FRaY-=pEqbd#2N;JDqz z;`&wfRutBQ5fMRntC9v)p)42KOiIM`(l?v#%u4lvOfZ@!XFEXMj#zk ziMlmpUv{)zFT)JJle}k3nK5$l9nxgz=yh=I?8p_pdAs2kpVy_9nl&o981A%t$i$x{ zUP5Y0)m=D^G~9+IIl&Ad7gLY8r@8~qs7^0XmKq+?YUzt?Na^&>or>srtKPwUeAuRg zxd2N)1u&!lL;3Ph20~!)>*&F%j)76!DhGz)T_(;cy3!o04`C3fM>$*Ey%w!o-Hj*5Lf0g;Dka9yodcsI4OO4gyY&30wCp<&S zzi$u^>LwwVv`xn zuW}pB6VjhXKrUA@*T0KdxuRxosm`EO{Q)PO802g+P56*=>2c)>3lbC*WyC59L%kyD zJjb}VtgW_emE@K%t1JD4p&sDYn_iTvCf{t5Z{U8e zKk!nbOC%SR@#UEB=-`$mr%)*>G-za_qlF6=)y0clDa@QeLK=@qxoS;X0A2l!9?*3) z)L<&)G25K(<3_yzmHuH0C#eY;U!r_2Ujb~FwkIO zZ#Y`y!;S4R+QLR+3VI3dJg5EEZVo6gYAl`m4nwdHHZ*@Miy4xYwCpvK0bxv9tk8DZ z(F}b)QgY$lx>xnPqcAjz)!~UUm>eV754$>f%q80CAmNrr26IBlVc7>r!Y(91a$?skw@Bw< zrL%<-`GRM|SuU$Z(r99%-`&1o-3%M8F&;%4%fsZhTt)Rv1*|)QvXm4E8uWGT<#s~n^-U5%X z;)B@=lc_>%>yb&8F*dk)Yz;u@)7fZ?`t05To;mwYKUk+##HU&&S*(C-= zU7zFH|AJ@wYqAp{-W6m_z=B`iD$C!voBFx|%RG@boC)86*Df}@7042Iasz5)6s%~J zFEaY*oB=LCLy9lbdBWj*mt|SjwK%8?dUaNl4wIc|a`ZW~Se!ea8*_}A1OCra{2`hg zshn$sdP0>0l@lN?=&Y9fqaUrA@dj0trIzFb3}qkUsV0YYI#HEtoX*WR9i_`U0lQ>J zHUh)cHX?dm@MBWafg;+7HUn%*V_)RA=wOXwlm&DVrQ($6L7OrJ4h?SF6H(T95t^3G z5li}=LO4q+dHB;{QB^V9BjZFFk3z6wsE9hjrCqy)@@YZ<(|pjK z?rxk8stFwbwfq3yqX)<9j2k%@SHelJxOSc)M;*?!5&KAR-EPK8NxP}PnEAOMjZmvY zIHAVdq##$;nhEn%gsCyHn|$NgG>ptTjkjT35#oS*u&3Q7>2?nAub`_ulWKP4DqgE( z6PCR&oSn_`-{Qyggf)N($WoPF?!2=RA|@tX+Oh=hfLFCVpx(ne1&S-Y!`T|R~v!Rva8XzzF- z<_$vDT_=}=P6V70oE(vho;%*uCtA&KVg}N_bNinQz3H=LixLsD9xJC!EcRJ{{^rc> zGk})ZKd|IH;A8>~0EscjsuF=tbk@=KYZP5jBKPmr!?J1KgaR+U(R{dWfYlRv5EX64 z+_K+Tamg@u))b)A_Z#U{?ph3%!F@Rx<$SR+s_3{^=$H>nb@1@ZWs*ikb+w6q};p(qkf$v?Ojo6{_m1K@0;zAgEmtzi{wo`OE!Cz+3%uPtUFC3-<* zcqKwh?7xiPn3wf-RX~_(iSN=*!K{8qlgykcUXHRSb|*Hyzf9bGyq!2zZRlp{*C&r( z=D))ZT#ew^2^-GXrRtt)XI0XzXl4#?C+d>-qi=E@a(^uWbu9j}D+BKqoU~4lhA`3) zt>V^7XVcf2dN``Z`S3^aS}otwytvPj#J;WSvEBF8o0uu8iYzm}bw9iwrWCZFw78n3 zLNhP)av%@4NuXF9LD58*BE8tJ7c^trtg3Qo>I3U_>8RJxKD`1Fzrcb2UCGx{3x!+5 zv!Y0PzfQve=BOwg?wuxi1vg4wI?}2P#}Se32n)dK8fuM4rKyX}sgadOc>`3mj46tE`+SRn@t z{(x8F&!Q=Yy>5gOpSrpLwo>KqeSQ<6A4mhsgdbr#7T8Z?d<<oM%3vCT&X;P zaVnx!-`X&zN5H3Zo1NDZMWy5AJ;9agfqSLeFPKRsBnCRUA{D%^Dd>lXYk5!m9c^cm zt`n!dXzK<7vljb;yLAOjc{5gDj+51;8)A%{_VkAhr)S6K<4fncK51y(t!xH2G@2aq zy!wrEZ;5q$8a_iY8(*iFlYd55Aa?8blA^F>QRj@_F7K}Qsr7Nq8FH;YJh98u49+p# zDScRT{5wXJtzV2r?WPA@s@0YBbdgc*oVB`%3 zmOW({LFYogy220=Rs;z8D@!O7HT6z4WspIj|o6(RR7J$$OG`(ds^5DTC+HSc6t^D)jBUih4S!- z+Ilmr=qQuTT*k!jdCNrL&Fy|NBdDX+!l2#E+$Ul^(^|C3N45dqAO|PMgHqs7%6-*k z1%9*RL`z55uPkE!$4kMCd5$M!uC!w*R0^fi@WG&m^o)Ty-97xstvO+Ck;{Lwat(Qk z!4#;Q@Z~7Vco>Mv+u?@fB4`M(1*6!16FuBavfxC~C7;2OUPEe~y=YwKlgxvHe)=V| zPC8MH)A&7XoY)UR7W6}9&hQ#~OSM611EA21oTI?_lg=R5&489P3qK7x<|sh%uvbJE zEc7X#u&wwOOZc33nkqZWxMN(%x%dOZR97f}^SdH5k8iBB0f+AS+_C!>B3cjt;?!bP zpB!w9t5svEG|T&W+!hr#PgaYZI9yFL(dYjZlcOab zjK%#Ki`?4faBg^7qCTX#H2O0fL(DF%1@8NJXHUC063 zou|sHulBQ>HE$yvGG0H-80~xx}X2IHgjtLO|hII-xdFld`YKo#gGd=yO!| z2bf!Sk4Axl*^{5Ox=N4mykX$}cH>X{1RlwRaV+@@8Da6161UP+XWG|S)a&GNfj*Hw zHq8_gh7h}!N)_uFd-QIN3l!~}2mNi4B_4lBctq4WVqy%diTg0bH=~u|N72jok?l?C zQja?RsDG!IZW6ZbgYqz3zK@OqRhd$-Rn@Fv0foU~HAgbT4pmiriw3E}gYZbL>k;vD z=ovS2Iz*o?bu7*)?YN&*iw^U*)v%^EJk0p1xhu~d(&9RRTCI9$fFJ&YrYvNGG|nV& z0r861X#`EQv5xef7>@1kzd1yEt{VJj881n~_`(!)AZL|Tk{g|r>eatccUa$iy!aI% zvZ3t`;#Hepd5G2KErSRh$)x!dWhyd&Z{{t~vfhX4stC;_3f`PzVZ#IZ@|1P!XGO;R z1H~xS!pWbhI9A3Xh#drlvks$(C}}_?ELe_OpIAF23buQYhkC*e9q?9a+BI6X4&>_W zy$lW$UitPIA7>dfeQt45VBY#>bU#nP(<6Y+yMfR z`g4+VKMi)DX5I{Um7Nxx3U1PmxCMnN+Dr2^;*&~pwkW-FP7xB&t(0bkksLQGD= zSdg=&st2^OSGdx8H1zQw^|>i-rDCXK3X%Ka9BEWweg=+oBN0JMN`0YYODN z^ysE4=`(8u|J~@1HcLN*7Ra zVPj`KjY%Kri^&+aM%wOfltL+Pdel3GsUPh`YcH7l*K4{REXk(Po z)gVbDiPWK6_Mb;fI-z0nu3dL=B@d-1@*hM~Y-tatPn?tI{{&l1rEVKkCN&qQWIQ+H zA{Luls&h{a%ik0gjQJ|>|4v}32repl&l<~`1C5;n33`C(viz@u;p-gXNNy?tyE@5 z^l-c(!k+y*oJhh1IXT{tQmz$!`ZBQO-+I=s5+2h1?gttyhJup( zFop8x*sbZQt(W;aksbxSh>0+5drU zTox|mO1JP+vNf*69ZveYd)w<3oQddu(+2D@4_6-K?GES7R;BiSVl@7~Fr_92o;GGy zrp3^#p8lpW@eNpN-P?05m=xyJ+y~}xjVaOuKT$@e_sB^olhPDQIcs6q_FFB2+vM3LS}*f6ecu;Th5pCA1XWCuD1~S|20Uxb{fwPU z4VG$=W@V+OtX|s)5pS6rqQI&aD!v-QMt(bRWs>`l7FQ`hlQ6{VuJg7c5<@fkE7bb> zU8*N(;Cl)0J>?4MJfwh)hE4gF(S3~ zAVuGKm)zW2SFHgrwq{N#P3M$yt%_s~Uz#e7Mz>vh4TBh;8NzKN!ga7e`{i7J$X zF)FWbzeS}v>GysuBv})TNV~@D7}EpMiubU+rgPmP`L%eIn6z%yNPXuum0K_G2jIkE zA60XmCG=h&4u>WOC$f$w$NT2OFd?f6FUsf1q$*7g4xgHrhHNyxWrtKrDfX8V^$K2z zOwf%>*K}kC{?+kO`6|E)hIbyFSn9ka1m-GexVWo#`;tOxpY`&Xc;+{oY@U+9{tPCXo5j{?>P0Y~h=sk|1ZdFJa1<8FLHoZfhktNzYha zq-U^e1zbHU0R8Vb%1XdlTGt(0gie--S#ok)eBw#7`UVT>79ua(+X{%#A7wWxlyqdA zVQ9eg-C|pZM=;Fu`9puQOr=q1GVnu<0T{d~e zIj&XIOnwQv;v13^)cNbm-T-XX5-6-{5x=fU_lzBL%NYJ>5x!L{Z?=M z-Fxph_sR@2%M5eQ^PJ!NJn!!Cat_%Pra1u>ii+VE_1`@6Qk$H>hoTmhng#RYblW3#Ebn7l@DF%z zlj;SOwMs5OMhhS9mGLxL2GLEQdc`%3UK*&qDj)YoDDmX6*Q_{nQ_J#evp%pv%k-yQ z4yH{x+HDRUl}+4T4`6G(Z>HkSmQy%O!DBxDsb@O~EK7|!E_jc=B1O_JU%(+cpqD6U z{*mJw)nc|G$-bOUJ()gBa4{VbcOP30&ojnBA|T_yT)(GoJNF9KyqL_YEu+xK#fYJk z&E%tnL`H=2=ZcwaQ}X8nwRKS4qH1mSJsw^P;Y(#?*-6_u1B+SrZ1QnF$C--YMWdx@ zHW~jRgq8{rxI?cN7<~)(AbMA26QpQ^xY$TrP1sA$C_fb92L|?1ms;`S7)}j5H#+ugbp{a> zl~2?(o~cC&e3_fmubCM&+Cz@DymqXRujE(AVRSFqD92U*STWQCvNE1~{2HZAy6v+w zI#_6qJ1g43UMUsOOvv;?lsa=4&{&|PMB^C4+iJLOB2t{rP&F^AwCdGoH~H>8Z_zIo~wtG(~&~N$Go?2m3pF%m`Y>9FXFNbxhD+zP4b_{c3*k3T9%wD8FL z74Ih5H6Q15{`;m}>LuB!IB1@qRJCc&`$r}oU9h~(2y{3CFmxfO#|zSoK1(w7F11m+ zL=zxj+RHjX!P*7UVMfJ?2J|xoO6tZnRkGPg&c-A=BU!8$(1#s_Sj^f(7Datyg93{_ zcD*!f!Dfn;o?sm!5H1b9_T1JcTSjjxZSpE7cKHosKaQQsbb_YjNr+jG5yU;#lQcQ02tCfH1B{-sARmf)4c} z|EZ*#oxTV&R37voA#(q~JyyA?$sUi~Yo14iSu}&!@b;o1bX*NI6=Nf&^C6}pw$+3F z+<7}y)j{Yo;pU#Wvi&I$$9ii|%NfPS&T0{KhGGlc#vsPl^X+;y7MSVp6<1w_t9tjI zo%YAu^^ux^<(2&4z&D)j1$3rLSV7YXCo;~AiG(|QxKc`xWU7m}{d$|aK7{-T1Vi!X zSliu_3LDHrl4%|$mKu}On1XxU__Cd*j~-nduZ4bQFyALT0tt`3zg(Y~ap8gnK_`x4 zhq>d1>t611ar7zxgY;u)^-F?yoEt0Lql}-Hf#{RoXlf>X-4uxo>3x+lHy7m+dz%Zo z-Wd~{AB~c$Wo_+om-VW9Ol0t5ka4p@A}4s0JonU8D9A524Q1Ed^h9TnzB?9I36@nL zCK;A2+FoHBazLLtJ-0Dd>P_TW6VOoaVpOFDBq@Do#r80=JI}V`rh!^>L2aldN z<^kFcCYyU77-i|>fNz6TUJgwWb!B461;Mb~EzkS2@yX3bj-ODRnQUpHeJ=~R1~Ao0 zEFFaSgO+){*vagPQ9Z?9H8_UX>iUpve8^*sos~$sGBzS!lfaMA*I4S_PTt+XYL050 z0f+ICQN{*yd8ep_p7P1eTN@5erPp`vgkL(BrC)E{?(O|t)#3Vjr7;lgl-pPC&4gw3 z;C?FI>*)Oc+b5v!D53t#{q-eB?&(;p3eFkuYnT2Ow(Wut+rFSMLO$JT&1#|4 zii|f#Ywd!^?rWoGxg-?3TK=+DeSPwc2=?J~_hw6K5kub*;Q^tjp2I77n0V}*5H@Oq zcCi&L)$OfOQRC9F8gWbEZ*vG(A`g2rGK$hl#v4zG}ey9vqFhD zE2`l4Qh6h9JDXuVTnkEZW6JT?oXUUJwcd5W{D|G|>)@MBHDTiAbmlO7ox> zKcWp~#_Dy$g5^~N;rL~3W5QJ9OO}Bg0Q3^$L;`I~>1_#c;8tOso&`XbWBABUR<^9@ z-VO$b@>R5=?4vN&fF09*M+>1Hip63V@&dyw>IR}fD=KAo=|bCs-u^^#S8paWMnO~g zzG*EK$_+Nu%ox%R)gY>E&dM?On0e?@fHN|tzR`<9tfUAjc$0A^01+gaV^*UN?! z(A`#ipvck(+@)1i(}IckyDGc>QxC-T%ep;r4K_iwq_Q*M4$H6WOD=W|3!6$!D2P=P zt;K`g^&M$8%^Sqw>;uvHUYd}1pMYB+iWqmW9EF+ z>!sDozZ8hs3Go~~Hxl<&S;rg)i)pV%=uwm6Qe*T}f09@!XDFI$50E0${f4z_m7+uW z`KaC4NVP&~;Fic9{^jTP4olYjFInsLJ&Q)5(pcaA;(!AI9n$>ND&yS-!;~&9Vh$Vi zh4XBu=_b%vkwV+2lRiKM70XeGK{uy&o}I3yb-|QQ^1+>Bg?PvLT7>?zYJZVNrmPWq z@`MZ;C;70~R`*{=xIK;ucdp6~>u+e$1wfs=-n|_kt^OP`^cnogj7zlmNu+17F>qYT zqugNI+F0IAf0HqQ+hdUcmpcwpOl8EH$}@T_!zh7QyWGCbx%>$t2m0X(wA#= zlMuu_xs~j7NZt_Ig+0y>(VmuvvJ#RDkK6b?4Ldb@(zWZ8xB|_&WcI% z*E*5oxYA|?oIrtyARi+b1YkFIYTlUqe( z65B#%-0H@jQtQ>M=?;pxte*8a52TSOkn0fD7|y0sZ8<+Jxq5Yyau1wAiiMrr(Gb@c zTVvcuiNO7J=KIrJ(sPkGK@fLb+}?|!t84m4v>Q*PO*A*;Sljz`bmLgj+{Ib~A)7NI zR%m;wGBw@%;oaagIlm6wG&4Puh$B8@Vx|mipQ-7$druI*35TR~wn>r)Ljlt->Zy7V zacbCDQJokX0^JTY&jZF=GnzWh}rd{`I?^m}Dyx5udT>ImLBy8hXJRH>eO< zw9m|*xpH`k?)-ks<1IdYs4PGiL(L+W(>sX`JpC5`P4cJiB~Ti{TjTq_g6{%7Q@R5p z)mq{-O7sOtM$F9Z%7Is|d>kz7Zp7wtsI5F?Kwg4f`I{L!djaEev#R#z--fd?LTLe3Xr6lrMCb4i0U&Fi}&@F3Z|0x1IMFZuB*zyBj+q^;q z+26uv{1^KaB!ZpwtOkseo@(b{*EHhapKL5Wk$2H zgmxy@Jv$zjKsyP=)mh_XWm7UtQvVp7wRw&mjA}O?5g)e|8J7dOWaHBilYG@#|A2Yu zabm@aBbTJwg;v5if6~*}TtyiVC$eb*Y$zK`j4f*Qy=SjpNhXNu-Qn|V9`Xb$8xPj>32I?B(ju$Ol{9ffmh947!8HG zTTqkvB8ho&ASuyL1Dq;p-kFXzpTQnj@&{*Q(QIhR73b!RVh>PP`Y2#r+++6i1Pv8MnBCv z;I4#BkenCeX_YnUoSQY%Znwm{bN;y1=byj^oNv!&BoHeVBc?lRMLsqY} z83O{?ZZxC&3F6et6&|XxuMyDCFkj|B1gh`2G@Q*r924Ivw14P-zUfZ;qU$!=2?$!x z&`A;FWIx+t)Q9iW){7{&!RlJ8nCenp}egMUq#*tZqWTp2oH=hX;>o~k@m z5isAU_0KSAmJ}6+`1t26NJ=1ME}s$tnbb?~7Ze({CvyPAxe}+c%j1>xt(kD97Lgjq zYWQrLRXvLG9?qq+rUhpY+n6uS@N+)ib$F-_SWYlh<(SpJ(0<>qa`jRAk@y?3kC$J< z>Sx)Yj=KK8{mL;JoeCiRi#KP0FYrF12T+ULCLO*0$j-s*|nl6JRG?yS) z>q3y!62^4Ahl3bRHE!^+p9VdTqu(}o{*2M7_X-yoDc46+kPxs?fX#Xa*6g&OOP>^A ze1Z_v=4y50wx8%%PL=+_0bq~UX+v*bxnr;!>lVl`dwnPZdFlPiXcJ}|;^N6LJ<12&&>X71xy!MPC7yucacB%w&MOS zvV$3i&_k1v*~FW#szrFRww1G$-SjK0;S77a)n;F4Nh1Jr5@~f_eQA(cHJs7eKnN?t zbjNgffkCo30e_h>p2*OgVmoW1O-saBO;7jpzF56`$42TsGXq*Ogblcpc$V4N4I-1G zOj-;N@Fs&J(zW6x_H>bY@tPc138d~tD`rW>QR1zjl&&ngspJr99dFYDE^);kX}(R# zCA^=c;}MlnVkf3XBii$NSY84}`HmPyH94{yRe=NDo%T#sRp!dn*NkpOZ@gku^dr!_ z)=`P`9*hH)Q8%%;Jpsf%`QcgG+4$Syp}QuS8^lF#X;={EW8RMfu8rSIRb@qrO37ix z46jdc0W#cdQMuziM$d<1b6wO32qTu_275k3RUjV2sjm7= zMa3W$z)V0ZYZ7fhI}O8s-D~QmbbuqtGU(izg zjE#(b@-4}vr~tG{T^Hc$dUnOvBrYi;>gTZ&gC{k>S6tV7u}XwpRs4|Eq=7*>n6b^d{*;Ur%zEnUGrnL~PFc1lI$Ko3=vJXb^xqQ!UNirUb#$RU04S z@>yxxwTrAgR+3c6fM5ZK@R+`DYWCN~_v#$&B4ebTT7<$vDPAh6cjKz!~z49 z?u2w^&Wn@mR6h~A+|KP$*RBaOZl|L+v)?6DUsJT(-w@wBGokD5E9IFlvSu1LjjxewsZ(V+_n^+c!7+{(C$7DeRI}9q*EbA7Qa;1(D9Sn0H#eyJeJb zD+bz#;bg~_MQ@;Q404+!?L9&kc}_-4yZBt8I$yk?H!LJNf5@lm%xW9)lrHivkIXT{ zn88_rU-v$;zGm-uC4pJ2__yu$IKY0(spY%;z*uc|N>MdLDGE=#2A&5I06MBNJpkk`PTfo*#+}W?8k(vEh&J*JLm5^JcJ2pyFlgQ zX(M&>q=($;tsYQWyMr$f|0x))JQz?no|;yyXcY;{g38OnMnucpG6O84s_;n+)g zU?o6^6jGR@re;j&LoT1C^%1X6qSK?FE?@4sJy5kE1zwl!tfkIR6c@7ZSddUPqROLyjFEJT38g#jjYbcWV;A$OI?F1Y z3fZ1cXvT0Aj0y1Xo|B}jvBxJl@V`o|LrX|D2|UL0nBJoOoUoN^5FHiShY_R4{EZ#L z$m9-XLu%)EQnIrGIqGhyO@5pRSWeN(*O=<}LN?xvVHSrs*VTVEw^vRw38PT42` z2qt)~H;Qrikr07cfH0(dcGNO4?AjeGRTSqQ5>>9?UQ{#eLF(;WUC+wWJukwFp!SO$ zp9ILEX_~Q{53Q|g`cm|v;+B3GR&0;0mDqEDkJy?a@}w!WtryHq7oRR_6Nq7sKiq&5 z9f5YzO()GnIqWK}5B>ADLtZ>=UfWt_(i+7un*ptrK>&qu>a>i2KaMNUH(GcD=8xC0 z-e@obW-C~@*6lwnP+go{E&t7izgBP2d4h+ibwryAEVQFTKY1tkt}pMM6(o#6={2MW z@rjvbOIl*kEKVO@vnQnMv%@YEVC)F9$ziY7)AjY&A`(G5HYO%6IeeEN z`?1x%+aDnNFllubjy}7OlC&byi%qal<@Zqf=?6sEoxEqi;0ag9!zwLQV+l6 zo~xk+4(-&DxfM%P%ew^jhA`n_c5Sw3>y>M*yVisn3S`>OI%T8m77@t`K%P(0bhc?G z+XUJZ61$4msWz{k&GCcpw29H7A8Oi(Di11fSvpYX@yZ8!kReM6?Xk}~-!XZ;a_&xu z@Yp1@+6PR?#L;pNzTl90wg{0!c14w4szShuVR(VY$394FJ>uTh{It$>6&Wx+vSJoy zXdL>5wO|;dkKV-y)w)7-NS~iJuoh!aq$M`95O|llBHma3`hj^%EpdWT>-oE`XFM_Q zSq}&E!!U|lysSV?cF{d_+L3||l=*&SCSE1y1w(`JCv(^L2ILfJ5KWgs4(R92_$G-k zpL4cQMpgP4%8L6e-=YY=(kc&nr@qTt+i{ctKcBEtbhqpP9CGO@9^aYM^9;_!7Pq=M z%v`C({Ay$wa@$pGx6qg&E^Z@0s|96-d7tTq0|)o27PIiiGi_k9oDi(l78_oz-=5P! zlg_42TNK-hTX-*T4tn-f%yVKG!0pox8o0E&RQCmSJ-T~RjQ&kKr{COuqMcoOwRh5U zsX(UsRrQ&46Qi=y}2I$Dk^z$tCxhTs3D~$k-=k zpcaW~r7eZMtMO$W>KCri7uuwmk&bmb7Dhbf?YPcThq7%o5q2vNUp3Y}J&p3`=8BS) z+=e_;XJxdsu!}4Vj9$&&uftUv3ud~5?wrTf@5<@sh~q|O&v8PWNtjWwdb4mD<5Ch+ z7c!z{5l1nF{TyQmEnL+w6bpA8=*dxL8uOyk=QYunkEy||J9H8`r3AXdhh#02#go8E zeTad_s8}M6|1Rv|dfM}ls|UxhxD7-k8sy(RiwU($m0?!Je}17Lkp2D|58*)O`^)b! zDcrg`C5yLo$x4Qhaq`fpgw(Sa)F`$ zE9yVDj|JEM7WMC_AfN}x<3Ed#VPtBD1WRlUphQ5RgDaDR1`hBy-)8gQjGN8Bll*Jh z|C5>#rz&9=CTbi^0Y*5Ma3(kw_?(;H<8AnUfvMJdGglMFQUuG;VSr;AW&U@jUm#Lj zBevRukx;-kJ0%H4dhDdTq^55hE4fOSIwN6^X+NAN?lpOb+8+XXs7 zq0Z_AYv3H~uMI45z~TRBzyk>S)duc;)_-~eE8ZFVdwT276OhCGcLH0W<1b5;Psyrw z4r9={g@8Z<$FL2Xx*s$4+ca4DfPm&6Aa^_GN59&I1SwC33Jf2XvGbR678HQv_#qPr z7BumnX5+V!{=SBYpIo4PpOo+4Bydek2*VMWo7*|tfy~W+;UW!jW)fwFDc|9TfItg} zEhGYm{iFCtmLQ;)rO$tB;D5tiFIG!T`F#>l5fG^1NZx4voy5}J{nsd{rAn16T>2r0 zutTtiEmZhb&8h=OVFh%ywso>}ci{a)B~j(Snb-eyRlBpJQtZKYAON-@;CG;{AsqS- zm8>lt9lu8@{i;m;^M#M>U^0PVM-MMkxg{Kn6W9@C2eNesS_1ylrS!j(cs1KN=?6Q7 zIoJk*pExG~9L0}DzNd)&Y7_TD>P#D8?`KEw|2hql4u59=*t!2I0ep)1&r3ti1&#pz zPVvu~;{RkMj`GjolwWNMK3?}{26QkS!(YR8|62p_xp6%D5_ezk~iQKLe)1KV)b8 zjQn@g%Ab)1Gk-_^`5y2`&HrFu`7`o&ec?ai{)}w7@<+}8`@sAEof!&jZGGRqbO;|{ LH`>be??3$?_=WdZ literal 0 HcmV?d00001 diff --git a/python/lib/py4j-0.9.2-src.zip b/python/lib/py4j-0.9.2-src.zip deleted file mode 100644 index 881bb759d78236931569f9fee632409a38013be7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 55521 zcmdS9W3VXAwk5i3+qP}nwr$&c*|u%l#$L8iGsq)jN4+Zz}dSlC*nT4fp;nb}?C=H=xk zVJRD$>}4w^=Ve)4ot>Q>oEce_Uz(WMRa+mN9FKdjuchIHH}8$$1gI>NZ2X+L5l&W4d9`Wdz8 z91%r4s22oK2zAWrlvJ^hQdiZlynSINQN>pFZIE)gP?pb=uj#p2NlL2U_H>eE?(}{- zyxz|?Pt>&b+JvksCaZT!%mb+FlJoJIR`Ex0#f>+^vCTa@&~lAp6uK`*rr$$eEu9-d zfqOJ50v5VSnJN9KgKphII6dI^vzzKi_Agv*6YABBoe>wyR>!QU(KjILneLviP$BLl zK6lsm$KH+GiGPkmefoNJ5PhomM|kJ7A-NCc?@lNDXss(zDD{f;moPre#Z6NpF5?ue z;0HTgftfn!UN_qjvXle z7PJe$nBXmlv=F2!x3mW)D?oH_R>NSJwgSJBfr#&B9KiMvaK2yjfIZG_lE+18LD$ty zuoeJKS8@YS=jurls6vmt_BV;VLpm1NJgRwEhEOv^fQDgP6ivX)V_!*`Pa?38d$|eJ zw@@N#7rXpu1Y=ELz<2`zVvczmFRgC?#n6l236?nRp-jQFYR!c(Jl;Z%hCcKUMtqXL1v!G%6%iz_$?SU2Eb#n~7q zM41VegBi)F9ai%|zwwRsSYR5fDkaLgjYVH5u||;|$40~j-N;Mii9a`OW(;o-VU9MwTlKLnv$Su*4?5 z6AhG*dBJgYG$q~Q+smRdV}CKxNf9S_XiU!AW!8$|X={lP)^#V6O?&n=f_4#k@nd~h zaxBA~ycm-x23Pmf;8G}~ki_c<-sGHXNJOI}N!0%Ddb4|lP5D}y&oa!+f#S%u0niqR zSW+^amm@fw>@Uq7Kx}~*eFWv1%V1NJl-CZF060!+?Rmx1=iXrMZ80G@wp;I=TX;wQ zzFEiUVfA$MV5~~gq~c|*O89MaDZ6a3PEIz)Fma*_4^P^hsq;kpNmvs9Ftl^4Zh=_L zBXnC&WK=||q6;Z*eDFn2lYl-FY7&aE6)^3UG&KloQQ7hs&`uEbFm zTj{H405Mfo;XA}3``gK6lY<#LH~~X}8FO<_>@q%fI+*tGdxSP*LglPC(^k&dCe-4C z7&7~0Htq$|f1m8vcg42E+)>IOB%MSXSnFW&*7zXWj2ZG4`?S7%{WCOd)QQ^-#xWE0 z@Z;8B7FyfzSrejGh~W{;EmYZ~nS89yu~2UH4Htc~mY&nZh@z}aGL81TaX}}flb>XJ zFvuJM2b&|xX$_cd3c|)Wc3TxX{!BUtqNh#zV~%RYQ*f&|FJIl-q9bm8V|ak44sn;O zb1JEVE06)70BYGRBe;+qx~{9K8}Jm;rD)_$XO_{LQv;D4#n*@Gv>_nb=P$;p3 z)?Qig()_WrYaoXU?5TSh#{*L-8UbI6q>)RQi$hk~6?_PJ+zgnv$?B~TLL-x8Ir)p24hEAoCsr_}g9 zHnM;4u1fe_YxCH_`M1Z{7PG{?-XDm0X`xK`CN)#$L{RFv_c>&)pc`UQTT8vK| zNS>NNJxi0F1ZhJ3$y;;oUX0C_&sCd6^5n~2lleARe$*&NYkfGnx`|rvI(*RVa$vw{ zF?gaZvT<6u%5z(ivzS(oRYhZ_zMjJejV)aii~jMDc?NMeux&r+S~mZe%&tse#r1*$ z0Kj4WPnrE^E%+zR`wz14-&XVfRTuob95A-Gbuj!V#cNU9wp(OC>77-^ab-VOAzC5~ zrt)GW+yjCP5I6vmO-RbF-cjOi%AxgS)bmF_T${09bcKn~Tbgn2eKEE3`u#B_d@I=-K;Ero!OPJkDCDETG5hAVoH?G&>FXpQ6#__1DK!v zl6uXKP^WJ1$q5<2S0}X5(-oPNrsv7^m4-il+fNwS0ZeG80RTFNI1ikOdJ5lo)~T;M zu+3o~*komPPcKYUJM@d9cm{He$X*M~xRz^Lof$u%1@e-!CpNppl!~l2!At!^R==kO zHBqbgX>vH8A!dW(da5W4vZBY`$fe6N`Zs}cB&#)ckvE2pd*xH{^i;zA`Ul&Duoxbo z?yie9#8Wq-PSRgWp$sLFB8tJTl_vj*LjJ~=nDR)U%sDu>;dM=3G!k<%95lJ6Qx&@< zTvu}o&W7PJdcDrxYk~UKG7N!p;Jzr)CXS0c`lZwZx=Qwk^pge_{=EgNb?TvM>&67 z-)eDyT3~;4$!(`tu(2%jL&9He(w&)hH>Sl#e}=m>qrhmRLU`3R<~I?e>%gr`1J2Tp z(`tc?orT~0vxKPK4?SI!5Z{?!ygUVop@C&&NkH^|?w>YznfYK+LI}G;x^V|uM;lm294nf8)NJIe z+WpV1w;21%n6{8vC4)GuWte2hY4C6<%}{9g1yeVgYTcN;de1tPue zj%$pbrHUofC3Qay=Q-{qJf90opiBIxkayWz@PARq-AAkkI4}T!5#)cO4$OZjm;XP% zuK%{gf%sphurQ@|F5_)TS81uT_0U9egWv?PAh7B91fNNkIZj!XCbq z18r$yYv(H6hOWrN;Ln?!uXxh+`|Xa^Frh+nZa}t3iH5B#KEzCY2TE{PUrcK^&bknNa z3LST>9aXZ4Bn1oF_Fv%Cj&TDrHM@p(rI3Zd!_>;-P%TY_zfR?{NGHLM{1s@~lO zQGa&2<~0h%GLUm~fPtBFjig*XTLLChHzSKVk&UMBKRh!%@Y_&DVFKFYR9k^sQKbg6 zqtlzsP_myI1X7kbAW}Td`NU>icc4l?*2r8PZ-bB5taQ=?GVQ6Lk#>`ov3%b_!09VQ zF>rmX!E!6Yy=`c!-Q|Y7nX9i=jOW=!_&@tE#P$Z?A|rG{`g`}RXU8uCj;sWsN|kCT z1gNA9>jBX&@PuV2wy;o4VcZ)R1I(m_>3ui?cc&qxsvtcOhB?K3O+1frD?&A_I}$4n z+tFdUenIB(d1thB!DQ!52kp%Ej!k2;HZ3ySO&0T`01EkSEG$m3symUr05fXC z_RPlL8VQ*&Z*!fRarnD=MZ^g_I5O;Ez}fRVA)6uPwxmW=e_&0Z`SblBdYgNAg+Gh zFZYgIw-&q(4e~RFk>IlW{*pY;3Dd+$W*ED$)6|k6?o~X13k+n8*^o4T`ocRAS4eMlH>! zD0e{p+_yV*D_X3FLoT;TqW~;w>tx1}jmeiySi)mW@8>Z>Z!=J6(MqM(5l)0gpQ@=>uZlI4ESfwV zGUiF-9UW(yS`CFr85w&((~=g-ddBdYT>}O8%yBqLx>hh6deBFmASW{WxCWNyLjgrX zI~a>bwyG4@tn>5Oz(0;At@lb?5PEA5$W?lP_Ij#wUpO`Z)__w7J@AlSx6II>leqkA z*iG|@m+b_p&p)<+c}Cnp+Xa-YXWyC`tMh*Pm3XREe0O%(h(=&Ag`zM7ph9`sFItbr z6!Y0M7Zo!-%CVxCduW}{-U$6NPY?c{XVJKS3!NuNO!~IiY8;e zib5`pnxu>X1XeQrYFaI16<6q0VK3pH{Op=&GRUcQ9}v>t0MK0Ox9lDnL0oj)BgUML zXxA#-K*Us0hXKCM8@@pSiKH7aHDG762z7O{Dwio|FjX~%Tp%Azlubpjz?826(AALL z2@$EF+8~IUVCI04WSvNDmMA_jUP=E%KziI_dw8Khq{)gg=G!q=Ioqc5V5a-S^VkG z9I6S1kMUZF*qCHg?<`6hOi~Uc08`08gt-I1WVYl$_6aA+l2m7a=KC&Qs=FU-+B2uX z0bC4X0t#RO`_j?5VsS7+jBKn9gYu2koHa91InbA@Rgz0YrN}R9-BG_^_8yQpfZ`em zg}j#LtO=coeVIyqtw0lwj< zmt|LVt2hV*_oA5p;dxUB{OtKtXNP}321*Zf1jiTB-ZhkV$?$l_xc&(OC#2G78IMI( zUAgx4iCO%k8hI_`^2BfEh=rhiMh;{#7YZ8Lp*eXFOlPh-Sm#|W6+ZDSiYXe@6g7`O z@4zsqi)<0(eltWz_^9W;LZXhSSDT+fC`MZ zE|u7@-6jUfm%~;F((GQ<9M6B-osj8e^=S3R%2ZB=9%dl0y&jHRZlN#|V@X5KvpZHs zm_JdGm4!LrxpZVVE9;F?ZMo66RRf#7*&NVTF6dTOYcjevP)p#+zGNHWpo-0EsAKCD z2kXcTMpoXsLfQ%46f4{EC@y$lE3xNpjf1xWC?D0fwm-0mxDt%lpW)REW)EU6QnzPI zfp!NC8S6_Q@R) z-GMbldw%%@sQL+0F}vn>1k9{*{ckTyf|S*j_xHIoTK|#68M1|nc2{O>zOVMg5z4fY zeQLO>ZQb_zPk<|%=+1zyQ9nKrzF&S|n>lYB;bsS(*=WgZslNp`Lh=}@&iRRG$Q z`>&$O$s+#3Lw0j>@zXxhQ*m?P=TCWg>$Gcqe~6*BKD**JOuJ;oErDPVLX4&l^NpH{ zyU%*JK%}$tC09SYWW4@4v5=^u$%K-lif+UX)?oZJUrGZATAI0^)&2@EA7qLQ+mY2rTG`y%EMr2uAuC^7@ ze(^%!`B_8MB3ehW#`BqStX;I5N~;J!YN)c1(UO!QvcYKck`Y5$5Y!2Th(ly|Fc%(F zB0$$!d2dX9ITjd8x9qF74y~(#W#}|%p;_C}Z9B)wW{PyFu;%j1t z)&?-uH2UEuyPv($7s9Rx!(>QZapvwfy;@mA=|W~#`Iq+>W>pTjmVAtx7`4PfcruKj z76Y5?FrAoj6vJ7SdIELSd1Tk5prjTpLBO8Osk=nf(fvis>g&k-%E7C}J~SOcX?Zj5>^SKjlr&}8H%E8)wo-VfrhJ}R=A>|m^o zcUgBJ^6LFXM2|#A^!V8~p&ij=)n+>O^K~I%9Om#GgI!H?@v&rKz=&sog(PbRg@qsN zZhVUdLz{GUk(P?-df*NzHg(av-QGlGkjsNqhI@2F31b`!%-ZHskLs zSFAZoVRfD-3sQDEKecgAR*C5HsKh4(;U-%ee|lZhQ=cr<4|IWhiUXdtkJ+!VvqIQa zt+dm+Ue^ncmQAoH^sbtxOX&Stsej!)+#;p7;}^jEmTwBMB-@?b|LyepE&g0aJ~WS3 z!kDlQ{Bd*ov^#r=f7<0g5$0P_7`{M=yfRZ(2Obp0OUsDOzFC zySoZZ8_y*%FnD_gOIqc6WSl0|dTrJ=~y?jc^*O!qs9 z{ID?0`!~FSZX+ANV_D=zXc@s1;o#TAg*-VcG8+2ODabJ z#(qAm7@t&PKOu!EO_QnR`OpJR%^D^xhIA+{_H6EWR%qhle?NsD0A05|4ph z4WKjUtFHs+R)x}+lJoYh%NsMB#4**Vy4z{D0ehgG`1uLI4ej|;XY-tR_Q|;RGSuZf zB69g@IK5o=CLQ?MlXdpAg8htRR%&N~wHV0_Nqh666;L~E%rF&*DoIcw2krDyGb6{S zx-OE~YMIu7tw+Dz^x<3UY9H_ill>|SKeb-UZpn*4s|TYQMWpe=j!k@r{V3AH{ZhA@ zH5+Z(Fy`Eyml-}kJD;z$pA}h|1a-hp>3wX- zR10Uig5@E2zn$vYE#VV(0=w!^v!vXSM z(Caa!I7%?k9}*=D-c=Sg16#2ZS?9}<+78K^P&AnAw7F@VhFNi5(X+nnYbEPIt+n>L zI(`+C{`%M6+)YE8 zmntj(0Fc;!@}$K7;7R{~xVHbc))eLcu%_mQE~f5=p8sS^eXZR$#S+ilzo2NGXEhh3 ziQUM&Z>ndgx*A2urK(3IoVq*k@DYfzkcf~B02CV8*nhrOigvdNC`ZOk+|rU7B=Wo7 zRdsb$b>n6ePiJ$_zUcUPdJsPg-nP=#x}AQC*oUtQr=}%;fJuAGtWoC%6?`L0MR9 z5*w|Ov}7trgp!s2_>jk!9$=Y@`=)9Zas z&dk{TwYbY&&$LOkt5P(Tbi_zg5M0d^u!*M8Gr*bC zK2*|gS>-{Mnu)Gqh?RWV2W}C%$Wmpw;?O193Q6(dwU+TPYvqXAPwMcJ`4J0lv+6|a_dit_;BnJ}-hFeQo0Mdy+ba{vVkz~j) zg=njyz#0jvRc;-F8~$^wJQ!rtKm;br0p-T)0h8&Q(yG2pv!ZmlWB*5TD)LV6$%cex1R(x#{)zetEg%>f(IuA}+n(kM#ZB zV;-8n{A#nOYx)`*dYB&p<+<+pJoqW5-}^=H{eHE6{zlsUevtR`SjL|b#;@b&?3I|4 zllMIe?-oJt8WHrxZm)NR?~|q*J<}H!`T`FpW}dcwJb!ogNQ?G!n$v{e`^nbzdVNO2 z&EflC=HlS#;w97*-h1ZwHu=!T;V;(w_0j`A{}?KH1NdWkp65mCg>TT#-A&+zAL8sJlSY=rkCDq)Aa>76AlqyX4O*a8bW= ze7=8fHz)!>_8vgvB4`SLeBna~K%Ath3QS6o8C=gHs}~{JYbB!84`%PLF@q0*!;?w2 zhB;g05KC)C3GEhtHr2-4cKEu-$bo95|q!4SXo}l~1QMUXD zQY!$`mmg#iE?IqyR!c)DM{zZUVcyAN4_@_}&Q`=*uF%2Id{vu1{L@N89u^#ELPT6r z&#J(Eh)$7_n6d#Ni?`jp4l&->=K*>)u`#9sCm`-S$yI5@s?!ZE!SJj;012LMr|WFC=7}9shs3^oo|#X$E^fK! z60(h$P~6_s2_nFpyQl6T(o+QSBc&PW)Y!5x6Wj7Ahp?7w&?{IOICrd<8q(zu7#KSTkU3Rq^N&**sr$x`GUW+exeWmI zirAOl&&f55ylahF?%NL-$1^}2GICTW*w;GPQ9$wrwAUQVXJb8J?fIQY*jT+4(8dK+ z)c8y2t?RFta?l@6m0b1aI$foxW9+lX5BTWDx!=|4hYV*DV3h!6PTR`Zt48c# zrGA<80p8iqs#jxGm6CVv@5ZXh@1xD-j@FM)jpgVL7}J*Jm;*t-!5kDy&rd(`pZ{@phY|W|e^#qWjHXRVVYDimLD%7aw=$`AgCGi$?fax6xASBS)+rHWkQoJ;>?sG zwSUX>>~?|FPS_-~vw-Cn9oBhb7^4{l=+qNXj}VR^Z$oS+8U|pEDDVH_7Y|UB9M)0` zC?Uw8mEGA6Pc&#bARe<5NX+N@^Vjl{{n6OU;nAobE)pbz%?oPg$t%Q(&Q9`Yj;UmJ zt(2V|Rsp={PJSFc{J=Vz=3Bvu9t92@?jhk&004ozY~jH#e|b%9aH2CSjc4WbQkWw2 z-a5<0&FabOFhqoR92c@?f5DTG5J$(2oQK33V&9B@t`6K^^y&=LfO#S3}o5Ol=S zRIQ<(YYGXpYU%6^V8-N_C0wQhom~ii)Aj&q@|mRuX8J_J)E0m}Lfaa`ocBqyq{!_D z2z&voqChYuArrh=1mV_J(`mL~Ily8Av`Q%tL72pD5McGEl7?}vCm7mKaNHD3CH<+C?Y>9zw64D!AsHwxTHZb_A=S1cO)pF?lQSZtcl15^9QK z&RS)q1A}PA-{g|tgho&NXyeBy-lB(eRNQs2+>sFYMr0fn|@N$9+%dNdiZe|2E zrKwA!t>Sq!E}yA#lBsd9mM}nMLj)v*5oMH_5hS@!UPI0ofRk7Mw!7OW6*F({zyO_XI^|hkoeHO~nkW(Kt^XkX1uc z;sBe{(Fb0l_EIMAYKYDE1MuCbO#FUTm9IoT*E!~KXMJSVi{Ny+Y$I%=!)9jBuWy`4 z-ms*{%33?oHy=$J(b@A-DyTwo#z|oD97a_2s>@yzi8~*~Q-C{WtKw%JYE9+Y77}@@ zx_-v86ryXtLD~QkyVWuqhPSzq-U4%mf*0NrTp$5a;V=(Q#4m$5B;*JvPrcGcH;VrA(jEeF(~R9TaiSIrxWlV4f4w`AFjy%_8q zZ*JIvt)1I249_tNo^M&x7yz?JWkFJQ3P1*EwPcO91OglD9V???{~GOq&~3Y%YwzHoo+*Hc8>DDaVQzQ6s*!HHR9dv-XIdLzzq+Hn zxFUE;3WiTAf5!ORQR2`&bbrU9{Gic`C7 znnC6agG(~9B|`b_47#P=Q^?NWG(BYrotO>z<|+J$=X3UolxV( z2!Ny*Psxc!QICz%(9cy~k4e8mkRpV#MsDS=$$B{{)ZiwO+frLNNqOFlovpFNJrMb2 zp+U7!qFqI64(RqrmoWtk4|qivXgxx=CQU`l7S#62%P>TFJ*@VA>5MLy-{bWNa?^{{ zSEMQIs7#g5+?M((sepxsr(depgK}RQSF89qpiZxWL}?Lub8o7_V-W5}Q#TfBhz)0Z zw#8t<=2^J|l?HFd%fR1t$%KD`*|T^hqc1lgkw#6{nm&~IascE-M^VH`=KXM+h}0G; zp3$-`U5}4uM7x=jB9oYNYEg3+H$Ldb${K)lpb2N$)AtkIRs{C@2XvRpRbO}oK}WEQzFthJ+g9yhs@k$H4$ztyjPNulv6`BZ|AUr<5}DxUJ1D$50M32zD% zFi7(P77=K!SaW2xWX6bvof<*yW|9v()@lKj4mwq2|0kElPvxDF`e0yJmJS9&18q6( zI^_=F(-v*)QshEb-|KFAbMnvU?e<%pHWx^gl)HOpmzq+Ju*c&13ujIjP6`eOsVn8p zd=_?dhM<&;0TKJ{Vxi{D1Ne>CSA5>V%5doz4Ci>=nV5PeDWM`|5f?z4zRd*4>@7lK zsz~jY!j0DOIML8rFz4bpO<;r(N!wS~0L6#8U`O#yiD`}$?&1t~N}=)O!G{vM$wl=d(EeW5*EP)Lhmt$yVQ=QJNk z8Qp&E)b5~EpK^QE=p)$a2f59z}4H_ihO zANBZJJ7ZO0Lx~nEtrB-Cm*C(-jxak2V4BlUTMiaSA&ja^glSDaisEDbXe0xt)y{+4 z3rr+y`sA^h(+krjy}&SJ^$0r97LYF5i`ma`(Vke~l&+mu*^7f)AFF%{z$>+V`AsFl z@dx10A^!c>H-Dfm{}0wQh991NsrY9a|3H7XZ>1%#N7FCIhUIFZqMu_kB`KT2Ygmgu z@45!#XAxk4sLGH`uTPLlVAUZE>p4(9MksfiqjYU?fmt2=s!Z=@0I30>qnQPa<@*{u z$^7VMd1ZIj6`Y!x3aj2u^C~X2lFC0R2~Q3I)xJ6J_+bbkC>k#cjh}f@Mq!Ef((1UQxu(kfx@O;l=HRxUg%? z&NYf|0>GdNhJa5^+3jojERl=EGRA;63%Qj zx0_IjG(D((77v3tRJzd(>8A3MDvO^>1ZY0|4Iv#C-0 zJ+K8icr@81Ui3Kas40qdV0)llH-9bOE9LKY^DOw}Cz$nnfjox9!)j4(&@9$Q_|V3N zA_AB)%H$5D+U?CYZF+y2X{S`D&Vjc{TMV$~!79P7)Y)16JvqXL##%&f19$<-c{`ho zl^q~PW0=tTO-d-Ij9K|K&W(U)W+9jSRXp9iD)<~@)fzAbH-cs|U5eSx+lX_+i*#>e zRa1Z&q9evcxEnL;S_wPv3$e6mOYr*pf-isF1)Cyuq#ID7PO8#+naKJxy1U~p)|Gys z&KH`u`3WZciTBb$Fa>1YGjR9}Xq{!%H*7SYE>rdlJI7`x1V{>qya?7E=+gV1`F5Iy z-A=95O|f#qMvvv{xQ3w70 z%7$Ayfi*K`OblO7jOzl9q;hBVm0G@?`O18;?oSm$f4e*DwZpMb1sswd0|&V+IUQo`0G~{Cl%^Uq=j|D- zER?nOF>owkK2H${lmh04HAn7n<>W=zlE?VW-BtEy08E$*1?ejI{U*t@rYy#eR_bMh z;gy@uT7mc!SUDPlwK9&>kBfoa7(YKO7E{yjTx6DOM!6A@5duQJcy#sm&wS8wOA+j4 zHPbrLgb*F6$_uxaqsiM)96`Khu!Mb7m$KlC=_+t*ITGqp*c-6oGi5wnq}))D9{r>K z>_k+qR!huv@+wjysFuh0Eff^hpr^VRC0i}^2Z1>Bw;D=e-G)$3a`*Uk7!OI5t^;OC z47At$^cKrvXOuUltdZv5H%(<1UOxe`ka1-z+1_*TVm58$-Avcn|YA|=410U zf;7OPshZs5VKu^$^iQYMXfi z>R_;RR(XJ+u+rk?Nre`ha25Y~ZrI6WT!X%h9=!=xojJAx@Ej3m)NJwTFw>cFLB~mJ zqISX9`;fsw)KliVHJGVw2s_$90aq_7(bi;PZ(trH>k!}3JcET?=6nwB+?49nvcP5^ zwUWXwRRib4)Lb$SZF~0?NA&*sbD(L{(^}{sRyA@EAmrDc;Es5!`kB<;_H z=m5_N2I|yK?P`b3;;51(C76MoWiN7iD$8dqHXIs4!8d<*o+}T^nT=_26l}TGkcWL} zKXk-Y)CQoXws+Ojf(bbTRHI+pV*$DbUqOSgL_BVWP=@2UKqi3!6yPo0d1?Zb=+q+N zwoWvMIFO8{I(lQQ4Sl~gQ(~NIx>GK|QN^sL7`8S%Kwo6;I5p`Gdc@E%eiLlid4B&Z zwJ2eKxw2&z;#_jsMLw%VO4##vzlTHbMC({d37vN`wG(jlx|Zw2dr=l#fgyoNsF&|< ze~yjscUO4AFM$2gydPa#DHrPDg-GE1sR3+TUh~BEvD=eZ|3X&~-k<34lT%~T1HC<{v4pbxo4 z%o0HJuy50r<|W(vm1_uzb#W_OMqU6r1ZkxxQVFrXB~ z@5UUeQ+`;oYRnWNW*xkDT{jT8#&F0#tRY_Buf+BN3f-irEU2%m!yS1aQ%l(4jvD;1 zy^2nBI~=?BY4f(}YgA&iWo@tTHt3|P8Xq1OorA2oE=bUsRfctdI+KO|ZnSb56$`Yk zH^`$v=S29`%Qq%FCIm~srlmTYq8P4`s*k-ytw$Hew^v>3goPO9>rQ4^j514gdsNF33i#YBuo^ba?8ZIDB1(|ECEV07ha02tbE;nK4S!nYv zx10U^=kp+KJ5T?cX)Xb$^Mr^D7js8^)KfhV?h7^<7|ynybrMMs!t&LcLfwkq!vJ=~ zdOu4V(^#uX$XcfI6KK15?_~iUBstORiNrTR5oxVIw~)R*8HBC8Tg8xg8{I=&}@wjhqzNkFbB!^hE zY^6q-{=RK4q`aLnfNt2y(D``|UmWzFT~WS$&1nzLu8r8F)XsQ&zUDbEv2fLn@`~;} z-DI@7?<44?YgXSc9M8=O<>R`zqbPnq8EnVwBlK|twrQ=ybe&Km{P$iI%Ra&VJ=}1W zKF1~c#{tlh3=G2ZWr&kenI0-XAZ%)qV7M^2 zu(Hf}M2OdQ1zYh(;Kv=t3HctIj&HR}xr&vdLg7o+b-U~aNZIIABuZAzt#E{YKf0ik3Itu zY!BPI`)v?p^k7*zbWmCI<(_H&H5uD5M}%NTY`PMgKn&?dND*?35J01$E=kT&Nn`cM zGqLSE@u_ZAWme^ohWX$Fak_--eifdJx?)ndZwxiZl6_E{qp4qY2!KboRdZ&Xv#+X^ z#cw-p8HF7IS@%3okFwSTQ4tz`MWS~tY1;{y#JWUuyI)1y4tB8+Z@OhQ4r^$Ghxn&C zZQ_sIKE)WGOD2yTf!)e4z6eA^`Dh`p&Fm=`!{FAfUm67I+77lf@oRwlsE%B{y6JGt z7SkSLtrzc6WwMwPL@;(P$Z$P}b{4Um)&~^I#T$p|@bWbWb}o*}dj*?5h0HkVa32+O zFo(k~w!DWXpgCr_iKs_sbg|FzF1AIw_Ew%FMPB`Kyw>!{t^4{!?0}t_FrJ*Ez=9TX zWij!(wNj+Ki1^uL`a!`4mO{zfq!Q{mRxYSYla47M0kYj8p@%l5p(!WWTpI8yZo|;{26~n-WfoawjVOL9DwtEUi_b3Mcp%G5Uvdj z^zp(7bqo%wUuhY(sj%8wG)WcjURP_~4v8N_PqRof$29b);hfKC$A0QJbeX@bJ#2O0 zVNCYIvvPANA`<#(DZnaDs~BgGlH=X6xt|DKir>qn^QFBs(HHgNGkkXnYJeTMaKRpB zvoML{3sbyA78=+kz4b{aReOe|q59+d^{84LCF|;BJt+bhfW~CzQup%p-xGHk` z)q&Umxge?r;lSHy)r?(YV)#P&@3#|9hF4jVlm}>qyl~XQGx#=mY+MXY;O}JMsp5!H z2D$<;;F8c#$W-5RI6AzWB!yWbIwBl!T@;^!S_=uAMw`!x%MDl(hZuuH!6*Kp*R~=1 z4+G;D-RBwE{yOg52z}&(MzLY}l{Ye#sIz~5d*)N~rKJA3=}NBH;SZWEe%X{+Pi|*?>CxTU zeb(ZYn&SuRmf4fv)da8CGT(G0Wk<2KWA@;f6oV#BhfzCh8$aK+sRxX0+sC&Gmc15S z34LRvBc?rHUQzT88>P09S6Yn{Z-$-S;9@KeU1^W)6HlD*wtQkED8JM)h3>m-y8LIb zRQIk|S8#XI1%ddCqXy^b=?gh6BEspPNDv=;EL0m3pIJB{;N54}DV9n+$VD9adbuDZ zjG4~;%q;R`?sTmo2QYBl4#*CyE>0Rpe=U$Qy+yHf!Y(as-T3Y^wW<*&r5k3~LknkQ zs<7^yQsd6Dn-QCgwS@~uD`@?Sw%Pg2vty15U{rkZ))p2Y*mos)6&JVj&u(L{-!It} zz-rj)b1~#JiYe(lkt6_#je_KvY+P+^Hoc54{o~fkRi!C%o%vPh8o|rKjllXf&1fguv>+g6ut+paF#wr$(CZQHhOProN-?%X?Z-g9Su zJo5+S6B*wp_llJ}_g`C?Vk30GUkB7I(lbySh2Sdmp6i7rK%2uA9gvsV)^O=KN ztjlUXEO+5s4)wdzQI%1N)J+)gAM0jlo8ir%p!wn9ylVL2T@7J%s%ctZHY^Z@In1ZK zAFr}!5J0ZZ8FyP=gHad`81hD+_b>oMzx;ksRwM=UjA`@yR|hh3Cx2hyNK{0ZlGGXKyZvxJy?9;fi8(Z!?wUtucaO z7vk~e3D0ScEa?V3HSk_kM*Hxf$+H@HK(%Jp%ci;?rGCq^lde@SF74lzZ)%QfDHt}Q_=BER7~Lk%PN>-GxgC24!vd2qJ-T3P+l(6V=fCUD?5BZHlF1y zc82F&(m)oM##_ZR(Owk$W8|^taZin=J6W-t6M&&E7yW@)8aTcps z>_%s`?oN@uWtI%FFN6ano!JM?C<;@4(YZ>~Wf+$Y zNXNDzLrjvLB!wy#%7*po!*OvLIA~OdDj%=1J8qwa6-D`0%Jo#zc==MAp6we=J(=qt zU5+A&l8=kW3B!Ifq)d{A$u94tA{~krUn!KIhyKbV0a{FdSTf#k(<8H1{!s z^2($UNtzJ{Lq`V;5p{BCnU_oIwpbVzP5b1Q&Q{*zD`cfvz9~*~&VIcns^3?}M5hz9lI+aJ8|@ zBeZ6>Jh)@y1jkIQ8<%W>@3P)c`EMXz#)|~H1~O|wtr}Jz(BUbv$YfxVQ7)xiO+=0f zs65KCRb&t}1+{A~QAWGh4sE?I-8(a74_#2L1jKHW$?d4-GnbY!4^;H97>{4I$q(qm zTi$3EoZ-0c>$>8*Sbs)SElp5o#SZ%e8qI^Az6OJu%oG!C_=g*3P7TDKU^v zo0!1f4Aw%O46Fs+Bl-guam%Xz93%ii#XjDzDr*k7^TXg`trHK5qIqRgP9UXUXpLkl z&rC&Ao!hle^*^M$BCY#m#+@Ct1%Bs>J>GUBZxdG&bu5_5TR^AKnz|c2^m06L3O%D^ zz%P$8)ySnW4|St)Rj0j2Oe&Lm6s&@xJFHNEu~*O_2kiZ1*8q~u#xf4`dp^cmHC5}S zofveZB@+%B1vRhrXg&P(Oa;!XA*PtA%iAhFYKE+D!Ue82Ts6}p$HfW~P%>Lr+oSlC zld-^PrAiAtT@jgL5{Qq}z(-r9jYF?|jX~IN*h%=Yz>b439?^0WcA@FSX;XQZ(=R@V zO6SB_APh1t8bf?Z+sOdYAJ^c#^sM#)-jEijRGq*YcZ=LAdxhcxN8zJvAj4QKqww*; z)1ZBQU?zXFlWJoDNBA4#;FtT#DwPpz^-dMSId1WAxOxi0Y=rK#9uu zqC`E_6X;&75+jpO@BBiJhAW<6@~t9BV$J+_U(uVLC6&n56gviNCL9qFsN3T;$m6l?R$!T<&wi%_euyzSq8Li+ z-4{E;yvxQ3_s-x2nf;3653o~Ij;)-fZu)E2tD#wvt_!5yk~%xt0M>3*o$>M1_V31q zks?fng@Ws4=n7x(k%FUcChom{9M~a!d-+p3ccLD=i~BSdVX3fA`Gx7UlWe6UdhYmW zO=&vKQCa)CcxIfnr7E-np@cI`V`mJ_YNG9iH5V{ICH-wf&$=qqYf-iwqx8+c(&vRd z&W>2&X#c))Tj8(sDF&<)l*x&aFij~P=&(0-kR zekS1Ock+@Ow);gicPb>lbF0u}WD-w#Yvq&*`y?iCtCVn1HTakC$7J_oTDSGk9?yh(hDk_-nN6uj`hsduX5@nuR#K(`lWs--)!t;F6RY=vcIQ+Jum0o9WAf_kt}u`2ke{sZ_DTi>v^^WsAbJ_+BUdKF% zop95FmD$}Lcr(@Ez^S;X%P?gm3P00Wl4}|w94M4Y_O`g+!J(TR`uL2W^0z&NwgOV1 zr@}=1`)zw8E=HYnFLJWcNKTh6uiX7#Xs`3qwv$1n>JSfyBE+mXwZM4+U}K=-Akvhq z{H@#c&Xj0o$RL#5#jO@03JSL)N;rodv!Z0sVtzb+c!|5przN~S8IV1Iy31us&_47>(VY3QBxHz8*l}zGILZj>L0~o$E4;1Q#`m_X)9|plFijM{hn8kdkFy%gSP}&Ws5kmvaVjIThK(0fYs89kKZe?c+Y!gEMTsD!aEL-4JlSTqUpX6^2H z09!~GRlqMTL0mwGaJ_I%TFJb%A_j0`Z&RlVgrMs@ZLRVmHmT){$uM?Hm|41f? zX>TV2RSaA5O5scga#q~SKK!F)RF3E!-QRkn9AXKp!N74k__MNcea1jnKwL>%bC2?Mp7{krpfxyNpJ8Yi&;TNT)jn*0YzqL6X5=#i*slwkhSgKoAc^NmWO<~1W~ z_DAL?Pe+lUSDC`7lt(%$rC$9i@ZL`2^h&L^-vH@4<<6|GLqVgJaN{lXA|@<)m3y@l zGd{FohK!Yg6Ra0!MmcCl%R!L~q|wxCA=(_aOmFCzv?C!WDCe#z9}5~nO*@XwP*l*= z?`>dbU0v(riA0I4vowyB;7hZblvGbMC~_N0>{Hf8U4nbNhqJJWk^VnTM+-W;J&o`! z=1KI^G6PKK9F<9`>-PP{t`;M>;^>vt62(?Cf9z|JuLhOWwPzL7`E^37c7fU^%axE_ zmuW1<2C6DD>qH@i`>mC}N@*y8*BHRs6$l&Yy(2oMw5hQfNil^p#QG9iHj%dQl>Zd1 z$ny7HmW8UX81j%Y8vqEKA2|0;{_)*1p|JobdQO%?ngd8gXZM>6xghi;+n*@-0|XrT zXQq1G_k~iUS>rEeZOugdq!BgWBWLVsNa__mAxekdhj@a?TLH9u9W$V8qSwCft1a; zf9wNmKYhPdpiaf9dGFbwfile$;W@i#WERT!@$Ok2*b3M8!!9t074(sL%!D)?3;ag< zn)eL=M5N~#S}<*d)D)~LR$0=mY3o&yOer=~phFptdzsR5qJU+T%`uN=@D>s!S=7(v zO#ZMPZ6e@ArC~g@ER3;Ig$DW1RIO7*0Os37}Rn8NJ}KRSZDdPJZ5 zt_QDC+nd>`o-Dn7vM2}@DBuG9!N&PQ+6}5r0|U+4Y5MUqSrUP%P$miqGrh z?LCd(Pe`UQyEg~WI34!Wz8$+i5?i3gre&26?K$;|UAcbd@~J=T!EL1O5tit9H32U~ zO`p(N#oOM`6r{Lzy|5iYa*Zu0Kys( zFV2d^iibs8W}44$qCm5(9w+HOdLoyaYTJqbmdIPSH`*gQ@iP#3+l&muI#+cx%YfmJ zt8n)2QBgQXdMnowGsH22mpHN&sBy4^S27J)w~5(5XzCdRGoB!1X$NaTx!|3!mb@aS z*u`n8`9a1auOkn3EgfYh>jky0fLEI4Kyf9`wj*zFB{>Bxf%Jrhma>L@BEwi3P8?W2 zvWtb}fV3=_x}%cT(eIkO(6xdo$v%#@jj}}ycU>fWm}Isj4K}kR!;KGce{Ig+XAmg6 zXajd5SEmj)93l*mhH5*_{?kaD6cSmF5!6;B3wI1=LSVOt58W1{^#}3!0kMS1Rs)k* zHmN&Vvgsu_23FgTR9dw55ta`LIRj&}soN`tj5!wQ@iM3-hbZmILBH3{JVRIoBHS2+t1tPNe6%=Z%xv}hj0EKYV!YQf$$0YoX-5x)^{nyv z$-O8jbEbw;y=OCgk%Lbh5UAMh4qHVg5BZmo3k@N^XDsiI5E-cxU_KB0$tk>!ft|Tu zzkX#Cr^wtce)T$!4z8Z;oIK9@-T^#B2zrq6CM0rJ(!?eigKN~qzPa&mu)U?+%&*Z2ORPAW%FEdSkzb^y2_y>JDkx?oQ0N@|BzRf+IP<*EQ!R*H6? za*|Ve;{yWUl7ohTHUcb$BE(lKiWWK&m5Tx5$k(1@MlnJmrpA5Zo)z?1%6zIKY6mdo zaW!DN|AGvwUd}kGG7^brCMm>il3u-ly^keQlbo?H`td6|Z!|$zlk|3BA(8r(pqY6H zWT{_4W7+~-Og?w=RF#>il`_^!_W>FTt)AlDpQ)o~SLkT{PaeWs++B9pN8>lgFAj7q zn8o$*dH?IWKV7ETuUhX*Gk<|-gbxtl;a`GCDXi#zc6cVo`OT<=oG?M7<5D1TR`x1F z?3d4$88~T`^dV|Ad6|*CwCxj>x}7oWWrS6o>%QhVv}Y|#nqqlflkGB^OlYGm&9v5* zuSHEuxj-7&SyCGGnDU$9hiK=e>z&y;A9e5d<2oQ#9;YJ@Y`h1XMO$2+C%OYEoOGOn zcKv@=>p7=zs{$Z?_$LKHr(*y``C-+nlawEJw~7*tUyW{*drmF&JsJ57!8{0{J0dj+ z6G>GWTBXcy`1P13F+m8cZJ$X+u+Y;K2|6_}d@MoGbipR2C5{MX1%cfgj!fn%gT#KB zMC1%fE+kl;52{~ULe;hh5!@Q||6b_5Z2$z+uDRO4P6p;6nT+`2JG!c|KlkO(3wdHki>h(oanku6ERXdPgDjwAhMV`^iZ7Bi6j}{&qK&RfONf%YRDqH`qDm z72FdD36~r0U&GyKEkqPJo@+3sPol=p)>rk{^TTg^!NlfGc!4?FZzTT?O>a5L(t~6B z>P9$8aH-Vnd-_lNbeDy!{2yfTd5v5{u)(v@QjYxi#fv2z2vBnwMi9}#VntIwoyAkv<{kq8pS@fJ%8E_gu7UV8>7kX@bX+p+m^gu;y>%6MUfDsJ95}q zbHMC(Za26-Vof#;^wo$Y0m2y5uynWEQ|=5WKw=qv$T-jae0()k>KQPDWW;2de0{63 zv1BP$Q*g|0TsvB|AHk#7iv!*`E~q!U)^fMvOlw(co3!!K=w~5_vV1+qEnB>^H8HD$ z0A}C%*%`k^`u+QDxjP-^@B(gADhmK4gXsrB}PBqMG8Xl`hD}5mIoNGtQ?p9 z(N1b>i92#Sv1zUL=ZnHNd^3BqQ=NE=^e8ux3E#>?eb#})-Av&AOq2S0jyscej#@%j*gULFC=b# z0Z~)aGSkgjQ;!x;Cd#_w2*(yyt$ptJh0EtZEl||Xdo8|wJ8IRAwZZUJ1W^lyQ_h38 zLGN_vXo%a$!wp4TMWbOM(M8hl39JV4Kqvgk%6fo%VKMY)ND>*0Vetwi}^Bwd7T%x@b7hnpkR`Y$M|F{4)GA@0Fn~Vt0?m=C+ z8lIU*CS9%+a{CBi*dj)U6*Vfq*+?GvGv`MBeMBU!>Oy#XY1 z(v=|$QYnPRlcq^j)u>_^p-npSzKi5R6@Xl__#2n$RTS zb`szDPL-`Hi~=zPZIBm+uSX1>Zs&7QD;_y4jrbwXho0UvbQk2uQP^=ly0Xo9LyE}K z5P<9Q1doBTdOD7y(dGp~i9M0WxMu7a7-TdTsH+}x9~n>pZy^}~95UjP5)n%&*A4pO zbD-Vh!#x@8N+h&fJBl#WfTW|3f4~$f7{U6~5^EN7TtdDI7rYtW)^){^G1Q*?@e%&s zf`qGKe5;dk8AZ5IEV+Z-#52gdjvz#YGe-q%%-l2{PZOmEl&Rhp7ZunOE93JhQjOXZ z$pP3=PV#fJ_GE^3ta{{ZO*svhGCb0pxR8+*wC_(+YzTzNQY{+i3{rjII#^ z&z^`p1ZyM)Tmw)^6pX(UB~C@D4lBufTqH(dnRFf`?;)w*QblXf`HKg^o_kA+++|Qq z!`sLi5uGe`8z+}1P>$b`UJWuPhee;wwgsTVpR^#K9}QX_iL8SG6$U=4vE&ml-*Aq~ zmFg?pBuX^*rL_fOl2k{pduC~*9;4D&nbvE@fnVoTfzHY&GdF)0!i`b4u`|i#E>9i~ z_>H9m{mtdJkeOR~;K~iMI--|80OIYUVG1CC6WchwDGxg(kE#PqFNORQ0fKM80sjhV zNRtr+H{b;TFhv0T2jbwLXD{RaT}Z><=NJ6j1~mK!bHK*V+QIrycHln=XK2JQ|C1Sb z*Y5RG=6!Ihg?|pD9dg?Va9B>=8GGI2muHT~NQ30M*(2S9=o$6fwslYenS9*5l`Sfg z)BySvlu6KioW*)mlDjaBX&k8+LLAvD5r64EVFyoZc^K;0$F%@34g!}J2fVg01Jo6%fR zSh1mski&OF0S}Czo-&aoA?YhMHzK^$wHLSpcZ4~IVNp#%^CaVfdSmXNy;Ly|qQA$O z(A)6VmEaU|@(NDrOpYH0?%P(JMt@9(WDls-=#htD5AZ`Xjkm%n!yQvanu;z_Jw3xq z@#Ip}dEWJ~+*z=bBGgIp7{Y1P(2t4M5v$#>BA!7J9L31%qyrU5v2{~F+Ve%m(T!4X zAo*Wb{e!zaDNoav=7r1eh` zktpao+O87HOw&$Lm&-unB|eQJOYz0U2#MH9^>Q+eP+@W!r<1_nc38%ulHYS82nG{P ziot_=)#+)Y9sCy66gayDWw@h!r%{v0PhK?Hk!oHy^TNdL4gCq4)2jkvH1pU;&<+pS5)OZiYOxI;=Z(1^Z+TqLCb$4 z9qtS6hi~}vJ-XLSBg@C&;UL?C@B{SEny=P)oi?;KY{z=9hBcE~-0{Kb;9+%gH`n{& zoeDUHl$CJ&>>YN`g&fo}?MQSJfYb)6|KUla`3`3(Av8$~XD%XUOKbS9?vY!6vOjy( z*CQ&`m*57Q$g~I_wR2rY_r0+K+;F;fr(LjwwL@e}t0i=CJUCTE5u!Cq_SDAfU1sS1 zI!eev7kbA%%GF2XFVwL)XW*r8XyP5a<0uQ19{0Q;~Jm z%s)m)GdM>uobLEn^6-!GigITO=UwXUNj9veUcC)eJgHWsl_eJr*W9e(CP(JXsLftM z>Pp|$hbyy-vP9@Q+$bwhgGPr)lay#C@l?xR(~Z_&=*qto)-fiIZEoxU1GHKs9$P?d zg~QL-8!Fd(CAs9+(>5Y0t7X(?bp>84z2N?@RqS{##Ui>71#gw+T z2EB17L6Mtaw0TpV7YsFsQIZ`goqP2taALF@UWT4Q;j=O75x5E} zS>)jN0Sixd9zQ=#4^`2Q;!qA!-QYTlfy3XiG;k$7zd_oyi8<{MW9DCxaOO1UeFB5O zQU{d-?p<`}vfF0z!G%BR!fvZq)`J1-!eMT#;R85v_3=$G99*MCrU`HXL^ZN?UR-G| zKhVtJ9c4zPE}-I6*oV{Xvp^aPzZU7 zI_Lr=LR2q8Iu{rtzg>)SO!@4SM0a;cHn1gm&R6|YM3-ufo3L}BuQXKZk`K18!zQX-i zy!HZ4E1bzL%+a31WwJGlcn{DaQ>kJYM*xPC)Ta#5FWyIx#@bZZGK^ikdMKBhJ1cHq zi8|E~B!al;h#RB5-?~bAS^J?iG$~AqzK0VZ5b?7%97A;OK(@otHrNG0P4sKFpyrLU zO;8~OGR=E}#8DLq((L4U3AU|2kF#(c+?q2sZy_OBy+^}+XgkD}28FU6_%SM36FjbB z4YDQUQXGYd%acWr*SMe|hvZ&$q`^9+*G}V(Rrd>i)*o6aSu|P>GIaFgwj*fK)~`8@ zylrh1EW22y=d>`hDk|w?ww=&&D(x*3RfzXDO|H}BGFz!-jy? z_%GSOLOY|~c_mg4&%oG8CEAdzN&MbJ7**9B0%r;^6%x5R$MumB6pwxD@I~%KX{z`0 z6O^(i+kEF0heruHB!9V1;4?=z=ubb&s*!cnB}E3k)YX{pKLYVOFE3S<(z~UVmX4w7 ztpyl%QYmXPMI=`bx01+}T1-NEaQu@kNfgMEDI5%kr5lel6RPqS@0L_2W^R}RjV*@I z$3B^MdFly(WUC9rWtE1dih*-WpXXYYFd7UlL^Z^Q$^H9=Z(m0Ji*9Gz*Q$Pd;U-C?%t*NOMfPrej?vSI%NeW%%+9*qq z^M*A8nSW(6AbXFP#u%yRKpe#$T@&i^yi4Cqv|wC{5yej?MKVB<0jB-xCiRP1p-|>k z1mg)}a9EJ1#9)9S_H!{;0Xj)oBSlZFljPsdxv#vWkHkebpqtfF?v?sfB-fktKCQxw zH05WCGlN(#ooXJFvLx{bBNFqkU?%wv5fKt)cOskzUtk#1yV=*6xi~x!-&JKE(*M2W zT1}#p>6pCT%KT<&@3Sm-(2gta>oUtm$u@10 zjcvDgr!y1MG}q?KL{neQqzj(lv{JhfKUe`Oc^_sUeFAa^BZk0zJebEYLl=tgbhw6J z7{$8}PQobnlQ&304Wv(Di?)9o)FQK%dxcTD{0jnbIbtK!Oo;J{x3oMQnPLfb!uBF7 zMoAA8_37-_B&ib%f~J-xIezRJpI@sUua>o{-?}n75fmzKq0-QwnE*#u^RtjL0K$=Bf3`;7aLxW?Fr)YO`FS4Fu_9HDj2!nE{k=6bDPKkgfczr!WUm&;LEi(OCW zU+aad7pBf{59a#{*a>*dheVELnJfZ$I`oIB8Un>}?GHw8v~R6PY(U_TwpnVF64cE? z(iR$*kxVcQUVbIIwqKffy^ZEFGtB%lU#)y41|^8 znUtVw4NeISfsaur$SxCp5EJhn-dR?})(wQ3uYw^{uJymIK2{c0{S1{C(wFL}Iew)L ztDyxbAqx_38w=MCiG)cVn$aE7^`Yv2KC0`94cc9l8f&5ogq#|4KIMny5~75<*>n|z z)zZ>Dgy5Hd09MRPDrs%Ree&&IcJ;fEg&~lnkMZzEd8hFjdT)$_oyrQ-n>(#)vq8}& zo}HOJsbzpZE0s;-frj>cC6DEBT_CK=XZs+;9N& z_P*@}xO)eic~iaNx_ZM4?ebpxmhG-E9*1Tids{I`)f-F*9)`f+g$-B2Zc7}WUgF&o zN8^o-(v5rZirphc&AEg^!U7cDEI#npp2paH7ken)3Vked}r~t;nkavwgDI}@7;S)C(H8_epORn!!kL!APyiZKU~0m(16rwrBvjeR^}(x zkMdy#B1J_Rqw08zwLpb&-tPpW)=Qj^@ZQSY-&$K0>=jfJ?3E~_*16wO zpveNb{iE9#=nyyufYAQk^lb)Q&3Qsu3C1o_wq}bzZr?n<7LT`X`OERVt5d|69hee$ zlzEw;YTXtZR7-KA`3?9=@3!w4r`bl$GXiz8^eJ+@D+1u1{ptQ)B5ORFa$n+>RLYBUm= z!BJ86Adr2`O8hv2+D9VzJ8lDTP zbP!nyN;&R4Y}$7R!X0QTx>0^LtXt+FEUd5o4_FMX4J~c?FD>fKOuKTj^~VtMt}De8k`wkF;!GArL`3NyqCSw{>2UU#x+##% z42N~cBA6c;g`XO2^?&$)Zs}S_31Pwer36}DcYfEb=C~qfZb0W^%%&Lh`B+`xW;oNg z(n_>>>|WaWdxmgYW2jJ>MG$ASO&xUxpI5nYA(%?&+Oq^K4y>L)pOKd%+QN)R(>p7T zUyy(GH=OS39*X|(8)51HRS4xjV@v&u*Z+Tm-o`(NXF~q3@JRwM>g z%iIrz;B`$wl4c7&c8Lu{3UoTp@9eu`qpJ5mxaiB!S0MvXQ#$nRSkr*!I$edtF!F*s*4xnt9VR>D_9%oX2U zj%blP>=xm3fZv7yjr#?@5TmEaBjQgXOR!GzDZMcE{UHm#Bgt|f=-s_7hK8nG5JlmZIQgu(vB8jw^AZuix!sUqiEQ-Af|Tbk1ve}aoo>^%Tb6f zQvPgC{TRc#2{4^cwc3@C2%L=>$Op;t=tI=G-BlgjY-J<8(&rva@8S~MpiX6r7f)R+ zOme9mC5QagQ=MH8DWguQF+HfzaqLY8zuov6cl?3$TkSPicM=dnZd4wU{=C_- zgSczV^c&KDh+KIHXE`%|O@S((n&ZoeP84*3zzk;Y=XsF`@sUkZ zXo=+{91N<9N|iCl2tp(rV-W3|39D8BS2NA1_uFDjmJggh|ArP{W?}8IA~LMx!{TVv z^>prlsOvD~{D!Ve)X^lodH6NHQ-M4=qMKr)8|Sp1u+K)WE<=w1wQL|JkBn@ho@15u zMheqwLVYjq&bB!P!T}gH+Wr%KREf{BieaxWn^GS_kz zk5@N*=~_&N-t47|Cydc;mlqag%j7BClQ~ywC)R5huX+5Zu5S^95D%@6LKZaqvFrez zvwoi8PuPvQCIaP+AKXfsE4T;o}4=eq$ zBHjwD0NA&DwhZV1Pyc<9Q3uz{{D{58&mSd9ZlzX@_{cFUI_w00eYTIk?(fM*62PuH z%iFC5X}LYyZR1Jn0PYf+ki;0D&aIIzxDZHp(b45eawe(Dc6MKeB>H*Uq~8IExS>+E z@8NQPt6MJwCuNr^8Q0JU#Jk|+j#l4DkpqTV$#>ae(HX?xXre*cWmI4JowIb?MBHgwct7h=>li`oz5D1yIx0k7xb~bPc#WG8gfXC7=u%sVtJ5NyZf>W~&ZJP)6 zW*yg>MC#u5Fb;P69NjV;+*)uOTwo(TWt=j91FU7q>AUUI+VKAj)x)8W=8$NG8}38F z2;31%Gg-UiCw{;1;U__17DcpsF=bFwMMCzg#W0OWTFo?~#^9hbx?jIT(At3>2%Xc~ zT`KV|S{p7~;NvJd-3_EOFO{yAn_`W&{1eO5K1E0fZ|{v#>DMJXe_OOp6|3PL zo}*J&juFkr#5?49Kx!0R!vN^Xv+(q&0yw7=pqY0^nXe1fasjZPbLEb7 zN{s?ZTTySlzZq~+7Bsm2T1B`jUy2J0(7~4;P!wMr3KsR?4WBhcNY)WnCoMer>Q?Js zfv9qC>Z(wm5$ycgZdot5$i>(u-awi6_6;)b;3`4L(V>wfaffzJ<0Il5%n?X zNhSmdfjB_LGw7+!Ug@p=>mS6omS2s?m_Psk7$EkoAQVH0`7!A0 z8M_oUK5mamt2rS=nl7(0*dw6?tMq0LPybRS^Fcs{Y&7 z^+k`E*gS$j-RDu?y=wxT8Vo$qmTdmBUk9Pf4=AlzOY(gOI8NG}W3{c_yjPTwFu;%Z z%Z&ROAq9jWh%vt~XT^z7Joumd@+#3%r2~Lkh&ur9fkY%-GKitX;bhZ>(*U`6#VAM> zX%?LKI^C|mI+9*lJFkYX8$#r23K>}@3_B|H9YF~sH1hit#N5#aXA(pE+Qsv)V6xKN zm{7RndT@fJz{>z`VhI6siQ1*}?s3&M3>n1^?A8wDZf$1RTBwfQPlzuiy<4HDiKt@= z)=A!}L}DWzd3es#PrGPd6nLcIEsY0x01d$0&xGikAfx9epZUFXZOmo)Xc^|vS2{gy zFbHA@nm%NDT!Pg)-NM17CyS#9TwLs)EE;)-;BU7lhY}4Q+yeLW@uwEGydpA>=XOo1 zeh(J8hp0SK#r3PX$_Y8Pt)AE^4g!ZQkT7pL5$-t$vx9^FF%;o5xo8v|smrjGs7 zjUBt&^f0wpGc)gTIN$RYuK0+aG)ROF*;l|Z;+z-!~MJYU;DGVvXAmUPX{1so0+si znP4DzLh6DBsX(>XRiOl%^#-#_dy(kgtPb{^E3g)>$ z>kwTA1i-5pV1+AdhE)LmaSMKd-U-ubkfK;Qh_x%SCX_ScoGZvoFd3R}luRd$Y8H)* zXYP)9xv5_O9F~bnV>=CHa$AlfPSgm{VKR&>1bw53=>AF(=!TSNs0k3%L5^N~dBAZ{ z{%amJCeW+9D%M#zU^S9lO#yS@Kxh3|JIOO0NNTlqmUmRn0zdrg&=i)|q)ZgnYr2c5 z05%sNw1L<`&Y26G=y@#%FSvr zH;iU@Og)Tl&NC&>G;V(q#8Y|2_k1KUj&r-Y{u2Ea%maekhEszHzv2v#?>xMdM7tVd zHs<(PtC68*$*pC}e<&linSC!+>DT`VUO}y?Y-&)?WKTs>R!1^zWFx<5w0{o4Qfwlr zP{2N9Otc3#Ru1A)|6~?EBzC$7y$HLwj$d2~qpG5*nOAO)K}e>cLKVdS88&J^kV}M0 zRt?DoTUqiaH;JH2c>EOWn}Jqn1P+{cMe6 zk=<$H>=T}E2-;l6xS3|^jh}TNJZr~twwNGTQyqiJ*qWgc$LXKhYD#|~u%<|r z(#WSoc+YMpBilxE&8!xRv6xOr_(NejDnG#ibyI0UZ1XL4ct~&Zh|>3`C>s#D3lIm z0UVhw5NazSOc50XLc2B6b4^{`T`ubv*pPQtL}IveFtYp50#)FI12^PGVptwr)$sGp z!|SXVW4bM;Y70%JQ%Fe1?DY^263xTt#`05;M9bHz+%Ktbi|C8yri*3&dX!|!r^|t^ z^VUyj@mI*Lffgb5-`aS5z!?jnG&C+GQ<>G)W23MUqvgWLj@&{}7@ws2vIe+R1wQ@v z%C0SkOXAV?x_MpeGAPD-n>DBSQoxUk-ybg%OOwOP6C73rA36z%+Hu>>ke=>ev|Emm z>Tz+9I$jf(FBxT5m{seA7{$v3ZK)Qz`tCk!>?XAF|ojSk>kOGNF7|agTT?P0Yf0ZpNmnPXoi* zVe@GREe#=kw8{Nvg`(grkcMU21`NvwK-tkgw#~C(CZ+sMQ593FJiH;7FE}%!u#4^&g6`ml=k{J}-#GJ`9d9b>3 zQ*SH90tzq#j(Rj)^1I$tHfi4zT@!U9g7qL;(Oblm1|YC%K{PZog7lIKPF`awjHGR| zY!TrydH(so_LO29(G?cIpfOl8e~<wnCLdE||#|KOFp`ix#)LJ>$O={bWiBcREFUXaZV(~FlK>}za zEL`mw2U^?mWH4R~^1>Vff8*^!69g1p^y#z3bH*B3Oy6<~zi`d1@{Dh26@wL~Bl3smzsFlb77s<>u75A=+5m2E}q)X1{i28z@O=?3ehZ1kdN z3krU8!h~2)eZ5H-u5p^v-;(TRLhaW6c1(xbE_B+|QJ5t_%n;lK-c5!dKL<8M^Ld?q13*aHV!A9?T8Ier8uy0m?=@iK%(O%F{Ddm~q00 z<)l%MfCM*GD;-|aZ*_;M;m3`*JxH})IR%G{LdyMPT0hjBm2UJXu*MT4Bif#K(7$x= zZYoEs_)mW(@n805NdC3%{R=7hx0SrVCY=9I$@`zf&3`L#FF~3X_DB2j&9Q4kB9E3V}a2>D|Fz`i@}T8UC_P8;?h4sUpc24ElV!b8z;NfABi^rk))Vh8^yP`89gJ^9bz{V^F9FAXx2KK43;s9{df~tvm$9eq z+(o-Lig^oX!{3{1WUF0S}hWyERD@&-o;LezBmth%3MM?`~D;AMgD#Rp}9( zFl1LhX^`j8k^JOspzH)>mO+{lc*6Vfbc{gvG~{p)$s^GtbP-%C(QOn7Bvec;Rlto} zekrrWmPXqlscY}wBXbeRxJK;7@F(~u+`u|e2Zi!vq2>?4j_E4yqHvy=4{53m>iJsY ze{}biVRa_U)Z;ZCS@?df6Nrs1_sL!LH%g091ieZymz0MEn?o(dmW|O2 zlWNWe4gHO<`QlCN{}uif~ERX8}kL|yK{O-@4KHq4Tiee8qq8MfB>J&TPI4Pi`5(3 z9+Px9RPM8u!i6%TRo69UjVuO7!%9hoGk`@FvT{@L`}dRQ^8$EcBd#~^zS3$98@u9m zrgow-xttb%V#J2P!AL<#2yTRM5ORf0!SdegFFneuC@ZsJt95@-tG`d|)-k(%agE#F zIw|Y1YOa2xcU^Stcz@gER`$aENmynF7)wJLpUB&uM50v58&2@g@n3nc#JmV4q0yGX zi|4nk1L6C>Nt_ez7{Xp3F_OMul`X|^Wpj(!-y>oIi6Bd}*D3eJZqXdJb#zVE7#d(g z;9^GTz6s7k-WtS(INJ7)IH7e%8aEIO&QnY29(Hx^VfHlzH@Z974)})TwncqqiyXV5*jw6v}f%_pw4_*-~gQ~isjblN+SQ+>cont znx#ZRFSKof8jF0&%X3DH4$N~;2ZLr?sb`%YLndd*HQ5vBS&bzI&03*5=eC4Pe3~B-RFQ^er@l;|>%c6`Gj{S&)tjLR_Iz3t&a3fu-=u zlOD}U62EOn(@Tq=;O+713H@6X-2s^Dc;uLv#va>CP3oo1kxNo6{%G9v7hcWTZtc!5 z(NmJ?4xY@b;GRrORJM`$4NN&pjp|r_Dq+1LtX$yKWGH17O9kkLxSo(@Zjd99IUI}YM;$NB zDiGETAekpQMB&ze)W_)@7&^wzvGIp%B^EwhBvoVg^oDmx*FKe>W<$l1Y-oF#1}B9w z(wR~gM4A~X@t!}14)grosueeM!TI^S{nMH=Ol}X5QNpbj*+H0TNV6GIqckjm+C#7(-W+4J+qKnnVpHi_!iS1$)&w=5$r)G{C!>{V9UO z7&`WnEMB63^b_F^IF`J0*dh}`cZw;Br)hn8mQ)K7_AMVzNM2ZK4Rk|k#b|2dSQ#S; zPj5Tin4v+Xb?=&GX4Fm#sUI7le+YeNJ3Oo)@3WUqO9NM$Nh#C$`64JieZ;GtMs#~X zq8?g1iM|37yCR(&&L5jGB33LqiXJapKO0Sm%>f(T(0CZ<*}9vY9|i>Q=G-QP@m(gU zCWMt2vQ0zcCrL?oRnE3$QVkN$uTdM3%K1RJkGAOAKy?)}l={-o=wQ&#iiHGjxKCH^ zNwYlhDUzDBt?|2{vAp}Acn~*;7AWvguris1WUG%cpEbuk7wdhcK$^XfH!(m4(f5&| zimCum&SbcH*`W+jg?}H`DWw-LU89McjA1M(av|`E^2MTvu>A-QOrkac(%9GxX_|PF z1PR6A#oK`3t(kUEmWh64OejuFaLsiA`y}OEh+|^H#geDmQbR)(H1zI@mTmDVAJ%$r zuiFrFom!*X3C^nUhGHbVcc_Y6S7Zw6QztLY`@$72A&|Fgf{;8!t@`D-QHzsEUt%aV z2hPmLmeo}FgV=ldD#+Z4wDzR?Vbj>+NzO*enl^?NgN@OopQUZhl))frl)- zO*nz}4Gz6{aYE9k4({d0ys|fkw;XIsV!GE$6lEuy5>J?pw1joMCp|Adpt}Fz9q-`s>rpBL&EH)g7TXk?9v>P zqm(MVQ1bINl@F@d!KyG{`i!%m%dX_hVW+T$8YrtuNDlb6+(EBJS+qH4HY2plb&-{D zhE*f$--c(VwK}<9yvctdDgwXy46)c-4S+dRSb&B#_*PRox7NKlO-0*+GtyMOdwRv8 zZ}ZEHSmqd|@ByQc(+6c78&N}>scOTo-w>&)j?AWRWwxedHgvAT_2T!UUspjFAP%hW zYt&8fE(-;d9ne*}S%6qHp$}GCI9TX*c%%S08)*u3Fh-22@iO%B(V=Z2eJd(Xup@UY zxw78Jh1!sA;B)8Y9psKU5=-A3&5R`NKYjDfEmvt?>@A5CX`EeIG}DrzaGTfj?nXxH!quM8b?u(e>_oE=h+q_GnTv?fvK2+%U_6GKB6 zdLefw#2Z$SdJIz|>)cjrb@Q~-!{`(Aq@hOQ&ds@^?BU4{yLmBB*^Zh0g!hid4%FLv zSN{)V%?A2*NqGIWDnwg#py1W2?V8@2g&qr$9$?z9Se`o_&?k&8I` zJL{qs!n+uR9G$=-?i$}UCLSuBbO$Krer)H zpTJ#OAXbRYNla4p3JCL??DXC-R<5I%i>qH3{@C`#8z0yLUnwcbas%V~8b2h2ZtFgL zQR~_?w=ZB;F1tf&Ikq1TUL2g2l~$U5Lo6}kI)?6IzdF#kPPHZYRRJG8HqS2S6JJ_R z_BxUDHCjhNl`Prp#{+)`YH5%+@udqOpHxSlR5SCqR92n8(DbuG@vB%Xtpr@?WG6ne z3RDYAgmjw6snoM-o_qV*v+lN@7a_4g4jGpTd#}1_K65E%5@&>FZcpVtl;k!m&BWuZ z;svUa6k%BdnTSx{h-=;Lt9eAd3)}ssHkV!Yj_SRps>YLYadDUC!6nRjXWv9y+ZtN- zsMK1x+(Ee=Ce=_d)7CAv^qUG3ya4$Hon})UwOt->vXUWE>-*!;nE-o7M&wE?Q}fRP zDf9|NtxZ%>n2Q>eMvE$JPKsu$D<=IiUggv(d24)4+Q*F|8-wm56vanW-}?DCJgfz{bqBXmOm%X@F|F{f*~vDjpTrX(aLjj#l6uxA8!x*03UT~M zRL>$43->h*0l*Q_Aj{e>F{3=hFOl=n=u+S2${_B@F)X;cunI)t` zn7&pLIM7hJx_r|wilxXgl+DEvewo(uIiTwb;Q;FXvsN6z+sFkP$H9D$4rcTxc|@BF zk?Q$o$(vhhUzkQ(38%MI_EkpRhAbC#I->kAFKL&)p(F2*(=H6Kg|<;Q=}yD>$^~QM zQWJFSG-P=Tcq+w7iY9qNd^KlOr?b*(=_cfhz;dD^W0mCHPgF2c^UsqT0k8W?NuFIF+TZM~otNMb(Mm|DMw_2htpyoQ5SKR3+= zQ^V^t&eWMY=1ocEhR?^5FuL_&qt(Ls8MU}NRptE9{K`zY`0}fLHvyisC+q{;rN3^iTy}uG|e1?_KXHMsZcix{sFo?kz+>94>FZ@u*@fAS=*&2wo` zNM1m)pDMBe!KpVvx!P=jkuK2v{`_|PB4ImPI66uZx@m}(<}it-xR8DSzMk5$rGNU( zENv1meWCd)kOrNOy#1+N6ln=!s@(BIZ| zy?p8rA+WCXe7IZIz>?~ zsrrB#R3TxSU31WBnH@E61WePz=etjg{sY_!!< z!2#10arIeXi;(*(v1q2ym)`pkGEBP-&Y0D;lJ={V?_1OYQE{q<01(=u_$LJ#w0tRE zbw_D#s~8127iNKm+B1a^n>9uQT39j*J+G|RqlFTSKG3W60IG7BRtP6zxRr(Wyt=m! z@hDH~`^mp$&&cOUJ=k9zIRg7WnSrik3@uMA?I(p$vJ(Lk{iP7w`uEYdgVlKiVl|S#hy1HsIBZ*hOP0 zR5w$fa-_sb+^9PL1Vhw0@ow_|^S<&XPydl389cBE=!KgwmdcRh=%B({g$hoot2cF^ zH<2+u2n?hypbf6rZ-T8~P#xAHt_TB;maN<-yAS4gFSpy4RV_4umx)EmA&2SoFX4Nj zDNu}gVtpv!S`;s;WO7q7O)k-YeUTw!Ax$P#>HH+P%MVWejIQ_il z{pKsO_`dGj)iqt+aN*41UF{?cCJ#tbA@8GVR4eRIE8~ULoP=`*1-5=!4FM%lke+F( zsMB*u2=sA}P9{Fp=e}o@0y>9Jt}{^>Yc@sCLQC0K)yDz}g-QuUjr!9n#cue%^?W7e zxZPirDdBF|sv7GtVvx81PG=o|Ks;W2jyAJTJgizZ+c6218qoB9dIbnNKiJ0*u_@o z0=T$)nnb4iY^ObLHIrw34qXer^Ts2Wkro6kWNaB%_)s`eUvsA<7x$#c-KK89aBuIP z=d*a{gomp+`d9=<%k((SbY8NLWG{P8iPgND*;-y@!~Cd=wG+Uqvqs?_=6Pb?Mi9?N zL${dUCYtwd!~_t2%^{>y;K>HxGstEI^4!cEwG1QVa1;oXbx$^91`whAG@4htN+pB-w_ z{5-k;Uy|Pc*aZI{3cP)NQJ52-t24RckkI`t} zu*bl}zD5;IFAxz~E2H(&@imWBHC}bsq|}&9(2e9PO)LeF&G&r@{+v*9#s+wAvc(ld z)nUXSvy;JwiXx&(7pX)uM@iy#zF9_dmJd`R$V z7X|OjoUy#8#7uM2#HflAg+?-bE<0wUs$W$?QX4VeJk39)9r<=kK$!s^vJLxX0>qT1 zbggu4n0u#$sOHV=Aa=rhV~)_U3?ocf_0uqcqt=nH0xzCM;41_u<>Sf7y-Eu3<%^AD zaahVJ(@l-{{_7e~8kTN6W7@p_mre{%}F?_2@`5 zC}V?D`VbysKpPXc8S!FexNj_bkZDZg2;ePwCOt8A3|1Ko_mP{PuzS{$MXfr=*!gS;;Xy zf)(c|&!>V?HjaxEGhx!>A8|YeWkbesAmcUaRgfAAV{LixoobO(a$K;JpH;`P<@Lwiyo!*7u=BcecTg8J z(E6z8b9qv50+U1#W@WavmWhROE87)WPvuU+@s9)1PP)mhFQ5^dHl732vm={4kZx%8 zSW{U9&yL**`{!DwYjCk|HHR{0+(f7sU*6H=b`$hcwrL(>^$Z1<%@M^4cc6w(U)esL zXvpMrC~+e35=Ir717EYzvR5?>!Rj;_&u);#Yz#WqjM-g6?6~fZ!TL6Xv4OH}^MOdE zc4%bOT2unidnvfPbD)}y;(|#sv4Dl5)OP)ZteE`c64-KG1=Kq;d|B)CmM5$}kjaL( z_rrOhR8)QQJ`*;w5j+!kun0KI%g#qC6}v367XjB7%~HTGC}GHeSFPa52%e{9eW>PV1Kw*Tw+r(z zZWuk`x*aQ**K6*PT+dDB2o9C_-^`REfFK?L4ZoJ{EnLVSxC^_n|%=jI;+;uMW?QS#KiGkd^k5?drU&5f5c z-#GG%Yk@6+UgL?`Id>Gb)jH_l7WDNL7qm3tNQ+iHsVpx_WUr7$k==fSb}*jrEuKvC zE+pl}anY%1wfra#tm`>;ina#ue%R%`(BUPnsrK>%E1+qs-q%dG6mBjRx zmk5V(T}e11(?_nEH+2t=-q;5czR-4yNH=b^+N0FwrOV0)m%RbmuI`_jJ|1R zpN`O!^|&EQ6NcLzcEwo_wSxLI$@ z1hcJ*l{BnI2+PvxqmQgZJ&V%swLDmxw2-P1d)95rMOr0Z?i|SM8mt)UIItHUuXd*T zsmPK}xh}c6YX-2uF4^m*D;!KC`V?rY9iWG75+x^l<30OW@6yD&9x*$m94$HGui_&= z6RyVOWLBN5>!~i?@2ps0Vd;Ip>_<7ce|W7AI)CVfRdC+yiHT!hd(GC}IZ^}eXX+fZ z$SB8U_KD#0O7K#zRqymFFJ(2@)%)jt84z$mOT>p+mbyOfVq4Rcrt&YlKo}v7x8Bp! z>AL5_<+;20arE?at_<3l@D8&CC`4AZj*JSd?NYVcUns5J`ZJ{H(=}_%t(Kab&y~B# z-=s(auUpH-nAXBgnW7kXovy~n?RJIi=DoA$vY4e{L(6 z-zas#Ya|1+=pX$)!P$jUHG@g=Nc59+Pqt&5wNEA)TryatXS~>iZHwpkjXZaze!>md z8Zk@tf7r->w!i&jH}bzK?z7PO;d^kW;*!-ku+ec?OQ%hOMnNvIO+tixq>Eo%+|4W3 zw0z-9J%yC+nw=u#-2Kq(OSOi5b}8}YSnkd1jydm6p^kQAs~GL9|bA7O;N{MTgLPJ6*;}Z=w%%J_{(LS5V3h1-%pkL&Ve}_8v8J*)j4@%!(h`|7G=-hb473Y z50vv$B$>O1t0aVd{Dzc~8!`6^1^cCD&QWGIg^aC1{5Gs~`u%ssj1THN=daT$2+I?w z3RoD$%XHCMU-A-^&KAyp>vrU#1Yi1!jaQguh^xrD&Q^I;v~%okr{^(^Z6Q5D^_EmB z*JG!n7y`ag%6e?c+^FJ(7mSXp^nOqfqR!G>7tetbTe-yzJF6*|%mPHHG1 zISKw6>t(A86NmZmmyHvgS%D3&MLmb`d3?(u$$m4neSz&3;N z+)TPi&X;Jwvpch~n+tq43LFdLu(91_nFG$|rb9b_2aV}?6F~{)9x%Ri|E)>zjDn*+ z8%;eEj@L^qY7>q^*c)!OoowiFSlt^v(J@@mWi%G~FQLV4$*cUz#bcgPd?1fuH8cdP zWqL+SAK2AyBb_o8p(B+1YF{fyAV>|(?7%qSnte^Iy3tA!mGjd=dRJ1d_cpjvuOy>i zo}>lAFzDN0gXi9FwTqvjAGiJ>;; zIHb~8eVfKTI1=tDkNM67!hcU`1G`TVN>3v`4B)+8qIPuCWGB&4*$;3!1uF3ZC9|L5 z%dVG2y{1cC%(c^q7%rXDq)%@$Xz_7uFlQwkP6N-E46+~(ZMi?Rb z{H5xxfJr3XrI+D}pLvZSjy9g#K^y)|TWB^#{FB4>r=^EA6 zy^&h^ zBb+PD1ukqd5se7H7X%rw#0gQB5|xK?JOsCQn3=bX1T~(;OHWmTeLr~#s`4vKU_S|n zsVje4a;>4uVO_IHuGQjTjo-2fe~*^uBfb7)dn~G-K7@+zZl#%QA21S&E$__!jI_%L zq)vlqKjw}HR$qVfeiV%B(w{VCMR|yld%`FD2n{J^L9F|0QX=z-9pJ{k``BrPyEn*c z1w!lW5`iz-n^rjVC9&`P!0IuKybAWI%=Zm{97l%A0_eDBu>S|g{fR;UV;%PojmH)a z=JqD`#&YRw`^o`B-|wTN=S7y!x)8?c3_&TTR!t9{8h z2Ah{Ypi@fMlfm{YZ@x#8*yI&EB;6*HL;HaHWI;e$LF7T^$x7}*x?*VJTx0nhi_zQ zgPwMFb655+2ga&`t*NqDyn)1kS9zL-Dkx4-S7jeG*H9Aq77veXI)pC(Y#X|=RoPqs zgSVcc8sSJrHjBrm*Kn#iyC~C!mNM})r`TIJu~gO{hUOzbC4pnkO=d`*`odr$13^Ee z2#wr_#gamwXM?7CX}0omZNlQz=1F8cfjHpDERa5taDspnBye~XXyBbG%ZNn?GO8lw zn^(f12_3_*T%We4Lhr7oQtF(B3LSgxqhBB2WX54zD%z|^$&V^R%U~N{(0&{(dVN(O zy=BFaFA|7FS!KT}zeSc}zQA5T(frvO7QY2!0kbhIJwFF|wiN-L(408=?Uqw*(VAD{ zh3QqWGnL=%SX74KETAJld79zk+mO3kP&HfOZh{!bVYy+In@DWm4Tzpi)e?Jhl^VT? zCCL1u7e}vCxgC!nBw^`LRYR-}lf_XL?Q@xW*$nP4^htihvLGpbaOe1z2)DhW?%;hy znoBjYVNaCY!bP@Ru8Be~3A&m}?l5y+U}X+y&EAam&%2Y{APRO$+-q4s&q3BB$3)kY z3%B@|L@3hwo7?uzXB(LS-PRtc2J~aWfFHLp|95l#$2usVNu z8~la5PqeI*L_0rH<6#Mw?JH(qb_EzeuPUfk9~f6u752dn%Y@JH6^TC>&_d`3(2pGGKv_qsD1?J2TaS{cK~6V-IXDD;Ch%ZAIz)=V3y2W2i3 zzP%<>4gbi#zTB#J@dT|^@r?q7V*q9r=~x9(`UtT~*Gw_WR2#)`cBE(k1+C4dp&8^` zTRI{R8McV6m+uvj0l{g(%i(-DA(^GPDlMQ{eh7D>4aD~FCgIi)8}TiDcXVpS5$lFc zstg!~xdC3Ec`$3zy1+vm24Bf0ncdTGhUXwz7yC{mLHSHi zsJN;qU87AA4!>T(C3M%f(~5>e`2Th?=@sWH~ZFO~!SgtV(l)!In1k5s=cv z3T5}aa2NTOx~*?;5z;@H-+vGBeMk9hHy0rS%mw=b&GW~m$?(JDlwbY8{;>v1_6r93 z>@O@-O18nWoe6H?4BeN~?RX;JD;G|LN1vt~4WgCP-c$+ECnNIPA*l$pE`qQ1`sZU- zacia$MtPo5zaz;$B~=}JJ0!m_-$wKvtZ}s&R#6LDR|Q# z_QS26qe>Z8i6i?g<@$Dpv-M48%ci06-4!kpFTRf;S!#Ay9I?R{@c?`>gLKG@BvQ-u%>=-1da)}f^xH}H?`7v;d(V>Lb$)!67TxNC7xY}@iI z5$B#s6U1yRA@Cr|aHW=vMhqwU^p%oM3Grx#l{P!YB+rA!3|Lse>uH094zv6UCcHa0 zlWUA)R(gviWLZc$yRE_4sv;}ON!uc;WlC@!Ocl3B%*>{Ue-+c>{4fB?^< z|MkVcFYx`B-+C1ll%aq31X+pikN=OjT^zS>D6q*B1gL!qw15YX16cm^_#+ihXpryz z^!?{wNPn%)`SbzOtw012>GwtdPUb%Xoj-#>cm&b=5aRbqJ;wQ;N%`TbehvdZu?YOZ zeSnh%fB;6%->LkD^9Q-lKhE!eK&Sd}!kQpJKoD^LxKjH-(BH7WD+gIy+5SsS^T$X( zrgsI@8u%lQRDK*%b!0|E2|00M!{Ah0lmVtPW?fZY&{Yo4pe!wmZKw|&b zXuli{ABw~hJb>N@AUQl*y}wk0{k4yp`T=ehK*#Avl>g=s55*@R#*lr0lJg_#zlV?i zDPHlw`v6j}KNJ5wb@hAN`PaD9K0wX_n7{ie_rEj9LoJMl!Dk}lsl>M3ZqciJYWBu5^ z{VIRT`WNvZ9^*YG?OgR|(vQyT{x_ugYJW=l7x5n+VmyxE%yo}Q|J7;5uhD)S!Ot3g zMEmcJ@G!;w@NISu!eZc$X|2~O- zWm11_7K{(jvjEdAe`^;1ZJ>vFK%55y?E}VJ|2&O{3BO+>PyPTi3vki)Q}TZw9{myg zcKaXUe^1{1_?`Pi>0nEtu| diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 97df433a0b675..5f7bf41caf9b4 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -27,4 +27,4 @@ fi export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.2-src.zip:${PYTHONPATH}" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.1-src.zip:${PYTHONPATH}" 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 7ea58afb53dc9..3f6d7b28b5403 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 @@ -1138,9 +1138,9 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), "pyspark.zip not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.9.2-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.10.1-src.zip") require(py4jFile.exists(), - "py4j-0.9.2-src.zip not found; cannot run pyspark application in YARN mode.") + "py4j-0.10.1-src.zip not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) } } 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 7df11ca7608b6..c4656048455a8 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 @@ -197,7 +197,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.9.2-src.zip", + s"$sparkHome/python/lib/py4j-0.10.1-src.zip", s"$sparkHome/python") val extraEnv = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), From 6c5768594fe8b910125f06e1308a8154a199447e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 13 May 2016 09:00:50 +0100 Subject: [PATCH 0218/1470] [SPARK-12972][CORE] Update org.apache.httpcomponents.httpclient ## What changes were proposed in this pull request? - update httpcore/httpclient to latest - centralize version management - remove excludes that are no longer relevant according to SBT/Maven dep graphs - also manage httpmime to match httpclient ## How was this patch tested? Jenkins tests, plus review of dependency graphs from SBT/Maven, and review of test-dependencies.sh output Author: Sean Owen Closes #13049 from srowen/SPARK-12972. (cherry picked from commit c74a6c3f2363f065a4915fdadec5eff665fa02e7) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 4 +- dev/deps/spark-deps-hadoop-2.3 | 4 +- dev/deps/spark-deps-hadoop-2.4 | 4 +- dev/deps/spark-deps-hadoop-2.6 | 4 +- dev/deps/spark-deps-hadoop-2.7 | 4 +- external/docker-integration-tests/pom.xml | 2 - pom.xml | 45 ++++------------------- 7 files changed, 17 insertions(+), 50 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 2477312d74bf0..c3be6b2fee99e 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -69,8 +69,8 @@ hadoop-yarn-server-web-proxy-2.2.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 0181a47a79d3c..61ed4c0889b8e 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.3.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index f7ff23472b9b5..fb014921765f9 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.4.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 92db55d674a4d..0baf4e84fff0a 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 44b9b049344ad..8be218cd68d91 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 53a24f3e06e08..3169a0ebaeb9f 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -85,13 +85,11 @@ org.apache.httpcomponents httpclient - 4.5 test org.apache.httpcomponents httpcore - 4.4.1 test diff --git a/pom.xml b/pom.xml index 9e9aad223a5e1..21b1becd08ea0 100644 --- a/pom.xml +++ b/pom.xml @@ -149,8 +149,8 @@ 0.10.2 - 4.3.2 - 4.3.2 + 4.5.2 + 4.4.4 3.1 3.4.1 @@ -410,6 +410,11 @@ httpclient ${commons.httpclient.version} + + org.apache.httpcomponents + httpmime + ${commons.httpclient.version} + org.apache.httpcomponents httpcore @@ -731,18 +736,6 @@ guava com.google.guava - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - commons-logging - httpclient - commons-logging commons-logging @@ -1448,14 +1441,6 @@ ${hive.group} hive-shims - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.apache.curator curator-framework @@ -1807,14 +1792,6 @@ libthrift ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api @@ -1826,14 +1803,6 @@ libfb303 ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api From 1390eca2c9c51ff862d0d9598ec07d68466b11fc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 13 May 2016 13:51:28 +0100 Subject: [PATCH 0219/1470] Revert "[SPARK-12972][CORE] Update org.apache.httpcomponents.httpclient" This reverts commit 6c5768594fe8b910125f06e1308a8154a199447e. --- dev/deps/spark-deps-hadoop-2.2 | 4 +- dev/deps/spark-deps-hadoop-2.3 | 4 +- dev/deps/spark-deps-hadoop-2.4 | 4 +- dev/deps/spark-deps-hadoop-2.6 | 4 +- dev/deps/spark-deps-hadoop-2.7 | 4 +- external/docker-integration-tests/pom.xml | 2 + pom.xml | 45 +++++++++++++++++++---- 7 files changed, 50 insertions(+), 17 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index c3be6b2fee99e..2477312d74bf0 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -69,8 +69,8 @@ hadoop-yarn-server-web-proxy-2.2.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 61ed4c0889b8e..0181a47a79d3c 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.3.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index fb014921765f9..f7ff23472b9b5 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.4.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 0baf4e84fff0a..92db55d674a4d 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8be218cd68d91..44b9b049344ad 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar -httpclient-4.5.2.jar -httpcore-4.4.4.jar +httpclient-4.3.2.jar +httpcore-4.3.2.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 3169a0ebaeb9f..53a24f3e06e08 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -85,11 +85,13 @@ org.apache.httpcomponents httpclient + 4.5 test org.apache.httpcomponents httpcore + 4.4.1 test diff --git a/pom.xml b/pom.xml index 21b1becd08ea0..9e9aad223a5e1 100644 --- a/pom.xml +++ b/pom.xml @@ -149,8 +149,8 @@ 0.10.2 - 4.5.2 - 4.4.4 + 4.3.2 + 4.3.2 3.1 3.4.1 @@ -410,11 +410,6 @@ httpclient ${commons.httpclient.version} - - org.apache.httpcomponents - httpmime - ${commons.httpclient.version} - org.apache.httpcomponents httpcore @@ -736,6 +731,18 @@ guava com.google.guava + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + + + commons-logging + httpclient + commons-logging commons-logging @@ -1441,6 +1448,14 @@ ${hive.group} hive-shims + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + org.apache.curator curator-framework @@ -1792,6 +1807,14 @@ libthrift ${libthrift.version} + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + org.slf4j slf4j-api @@ -1803,6 +1826,14 @@ libfb303 ${libthrift.version} + + org.apache.httpcomponents + httpclient + + + org.apache.httpcomponents + httpcore + org.slf4j slf4j-api From d3110d8b943b1af2cd44a6408036fc93de1d1aa9 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 13 May 2016 09:04:37 -0700 Subject: [PATCH 0220/1470] [SPARK-15267][SQL] Refactor options for JDBC and ORC data sources and change default compression for ORC ## What changes were proposed in this pull request? Currently, Parquet, JSON and CSV data sources have a class for thier options, (`ParquetOptions`, `JSONOptions` and `CSVOptions`). It is convenient to manage options for sources to gather options into a class. Currently, `JDBC`, `Text`, `libsvm` and `ORC` datasources do not have this class. This might be nicer if these options are in a unified format so that options can be added and This PR refactors the options in Spark internal data sources adding new classes, `OrcOptions`, `TextOptions`, `JDBCOptions` and `LibSVMOptions`. Also, this PR change the default compression codec for ORC from `NONE` to `SNAPPY`. ## How was this patch tested? Existing tests should cover this for refactoring and unittests in `OrcHadoopFsRelationSuite` for changing the default compression codec for ORC. Author: hyukjinkwon Closes #13048 from HyukjinKwon/SPARK-15267. (cherry picked from commit 3ded5bc4db2badc9ff49554e73421021d854306b) Signed-off-by: Reynold Xin --- .../ml/source/libsvm/LibSVMRelation.scala | 2 +- .../datasources/jdbc/DefaultSource.scala | 26 ++++------ .../datasources/jdbc/JDBCOptions.scala | 39 ++++++++++++++ .../datasources/parquet/ParquetOptions.scala | 7 ++- .../spark/sql/hive/orc/OrcOptions.scala | 52 +++++++++++++++++++ .../spark/sql/hive/orc/OrcRelation.scala | 46 ++++------------ .../hive/orc/OrcHadoopFsRelationSuite.scala | 18 +++++-- .../spark/sql/hive/orc/OrcQuerySuite.scala | 8 +-- 8 files changed, 135 insertions(+), 63 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 68a855c99f1b0..39bdd1afadcc8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -32,7 +32,7 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, DataFrameReader, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala index 6ff50a3c61223..6609e5dee3af5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala @@ -30,30 +30,26 @@ class DefaultSource extends RelationProvider with DataSourceRegister { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) - val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) - val partitionColumn = parameters.getOrElse("partitionColumn", null) - val lowerBound = parameters.getOrElse("lowerBound", null) - val upperBound = parameters.getOrElse("upperBound", null) - val numPartitions = parameters.getOrElse("numPartitions", null) - - if (partitionColumn != null - && (lowerBound == null || upperBound == null || numPartitions == null)) { + val jdbcOptions = new JDBCOptions(parameters) + if (jdbcOptions.partitionColumn != null + && (jdbcOptions.lowerBound == null + || jdbcOptions.upperBound == null + || jdbcOptions.numPartitions == null)) { sys.error("Partitioning incompletely specified") } - val partitionInfo = if (partitionColumn == null) { + val partitionInfo = if (jdbcOptions.partitionColumn == null) { null } else { JDBCPartitioningInfo( - partitionColumn, - lowerBound.toLong, - upperBound.toLong, - numPartitions.toInt) + jdbcOptions.partitionColumn, + jdbcOptions.lowerBound.toLong, + jdbcOptions.upperBound.toLong, + jdbcOptions.numPartitions.toInt) } val parts = JDBCRelation.columnPartition(partitionInfo) val properties = new Properties() // Additional properties that we will pass to getConnection parameters.foreach(kv => properties.setProperty(kv._1, kv._2)) - JDBCRelation(url, table, parts, properties)(sqlContext.sparkSession) + JDBCRelation(jdbcOptions.url, jdbcOptions.table, parts, properties)(sqlContext.sparkSession) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala new file mode 100644 index 0000000000000..6c6ec89746ee1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -0,0 +1,39 @@ +/* + * 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.datasources.jdbc + +/** + * Options for the JDBC data source. + */ +private[jdbc] class JDBCOptions( + @transient private val parameters: Map[String, String]) + extends Serializable { + + // a JDBC URL + val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) + // name of table + val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) + // the column used to partition + val partitionColumn = parameters.getOrElse("partitionColumn", null) + // the lower bound of partition column + val lowerBound = parameters.getOrElse("lowerBound", null) + // the upper bound of the partition column + val upperBound = parameters.getOrElse("upperBound", null) + // the number of partitions + val numPartitions = parameters.getOrElse("numPartitions", null) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 00352f23ae660..1ff217cbf0d4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -19,16 +19,15 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.parquet.hadoop.metadata.CompressionCodecName -import org.apache.spark.internal.Logging import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -class ParquetOptions( +private[parquet] class ParquetOptions( @transient private val parameters: Map[String, String], @transient private val sqlConf: SQLConf) - extends Logging with Serializable { + extends Serializable { import ParquetOptions._ @@ -48,7 +47,7 @@ class ParquetOptions( } -object ParquetOptions { +private[parquet] object ParquetOptions { // The parquet compression short names private val shortParquetCompressionCodecNames = Map( "none" -> CompressionCodecName.UNCOMPRESSED, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala new file mode 100644 index 0000000000000..91cf0dc960d58 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.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.sql.hive.orc + +/** + * Options for the ORC data source. + */ +private[orc] class OrcOptions( + @transient private val parameters: Map[String, String]) + extends Serializable { + + import OrcOptions._ + + /** + * Compression codec to use. By default snappy compression. + * Acceptable values are defined in [[shortOrcCompressionCodecNames]]. + */ + val compressionCodec: String = { + val codecName = parameters.getOrElse("compression", "snappy").toLowerCase + if (!shortOrcCompressionCodecNames.contains(codecName)) { + val availableCodecs = shortOrcCompressionCodecNames.keys.map(_.toLowerCase) + throw new IllegalArgumentException(s"Codec [$codecName] " + + s"is not available. Available codecs are ${availableCodecs.mkString(", ")}.") + } + shortOrcCompressionCodecNames(codecName) + } +} + +private[orc] object OrcOptions { + // The ORC compression short names + private val shortOrcCompressionCodecNames = Map( + "none" -> "NONE", + "uncompressed" -> "NONE", + "snappy" -> "SNAPPY", + "zlib" -> "ZLIB", + "lzo" -> "LZO") +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index fed31503043e9..6e55137dd78e1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.io.orc._ -import org.apache.hadoop.hive.ql.io.orc.OrcFile.OrcTableProperties import org.apache.hadoop.hive.serde2.objectinspector.{SettableStructObjectInspector, StructObjectInspector} import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils} import org.apache.hadoop.io.{NullWritable, Writable} @@ -37,7 +36,6 @@ import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.{HiveInspectors, HiveShim} import org.apache.spark.sql.sources.{Filter, _} @@ -66,28 +64,12 @@ private[sql] class DefaultSource job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { - val compressionCodec: Option[String] = options - .get("compression") - .map { codecName => - // Validate if given compression codec is supported or not. - val shortOrcCompressionCodecNames = OrcRelation.shortOrcCompressionCodecNames - if (!shortOrcCompressionCodecNames.contains(codecName.toLowerCase)) { - val availableCodecs = shortOrcCompressionCodecNames.keys.map(_.toLowerCase) - throw new IllegalArgumentException(s"Codec [$codecName] " + - s"is not available. Available codecs are ${availableCodecs.mkString(", ")}.") - } - codecName.toLowerCase - } + val orcOptions = new OrcOptions(options) - compressionCodec.foreach { codecName => - job.getConfiguration.set( - OrcTableProperties.COMPRESSION.getPropName, - OrcRelation - .shortOrcCompressionCodecNames - .getOrElse(codecName, CompressionKind.NONE).name()) - } + val configuration = job.getConfiguration - job.getConfiguration match { + configuration.set(OrcRelation.ORC_COMPRESSION, orcOptions.compressionCodec) + configuration match { case conf: JobConf => conf.setOutputFormat(classOf[OrcOutputFormat]) case conf => @@ -205,7 +187,7 @@ private[orc] class OrcOutputWriter( val partition = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") val compressionExtension = { - val name = conf.get(OrcTableProperties.COMPRESSION.getPropName) + val name = conf.get(OrcRelation.ORC_COMPRESSION) OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") } // It has the `.orc` extension at the end because (de)compression tools @@ -329,21 +311,15 @@ private[orc] object OrcTableScan { } private[orc] object OrcRelation extends HiveInspectors { - // The ORC compression short names - val shortOrcCompressionCodecNames = Map( - "none" -> CompressionKind.NONE, - "uncompressed" -> CompressionKind.NONE, - "snappy" -> CompressionKind.SNAPPY, - "zlib" -> CompressionKind.ZLIB, - "lzo" -> CompressionKind.LZO) + // The references of Hive's classes will be minimized. + val ORC_COMPRESSION = "orc.compress" // The extensions for ORC compression codecs val extensionsForCompressionCodecNames = Map( - CompressionKind.NONE.name -> "", - CompressionKind.SNAPPY.name -> ".snappy", - CompressionKind.ZLIB.name -> ".zlib", - CompressionKind.LZO.name -> ".lzo" - ) + "NONE" -> "", + "SNAPPY" -> ".snappy", + "ZLIB" -> ".zlib", + "LZO" -> ".lzo") def unwrapOrcStructs( conf: Configuration, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala index 965680ff0ddaa..0207b4e8c9775 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive.orc import java.io.File import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.hive.ql.io.orc.{CompressionKind, OrcFile} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.Row @@ -98,9 +97,10 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { val fs = FileSystem.getLocal(conf) val maybeOrcFile = new File(path).listFiles().find(_.getName.endsWith(".zlib.orc")) assert(maybeOrcFile.isDefined) - val orcFilePath = new Path(maybeOrcFile.get.toPath.toString) - val orcReader = OrcFile.createReader(orcFilePath, OrcFile.readerOptions(conf)) - assert(orcReader.getCompression == CompressionKind.ZLIB) + val orcFilePath = maybeOrcFile.get.toPath.toString + val expectedCompressionKind = + OrcFileOperator.getFileReader(orcFilePath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) val copyDf = spark .read @@ -108,4 +108,14 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { checkAnswer(df, copyDf) } } + + test("Default compression codec is snappy for ORC compression") { + withTempPath { file => + spark.range(0, 10).write + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("SNAPPY" === expectedCompressionKind.name()) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 084546f99d495..9a0885822b8d2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -171,7 +171,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "ZLIB") + .option("compression", "ZLIB") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression @@ -180,7 +180,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "SNAPPY") + .option("compression", "SNAPPY") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression @@ -189,7 +189,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "NONE") + .option("compression", "NONE") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression @@ -201,7 +201,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { ignore("LZO compression options for writing to an ORC file not supported in Hive 1.2.1") { withTempPath { file => spark.range(0, 10).write - .option("orc.compress", "LZO") + .option("compression", "LZO") .orc(file.getCanonicalPath) val expectedCompressionKind = OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression From 78bf9a1aad657ca9434700fdd251b1d3b4723e41 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Fri, 13 May 2016 18:10:22 -0700 Subject: [PATCH 0221/1470] [TRIVIAL] Add () to SparkSession's builder function Was trying out `SparkSession` for the first time and the given class doc (when copied as is) did not work over Spark shell: ``` scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate() :27: error: org.apache.spark.sql.SparkSession.Builder does not take parameters SparkSession.builder().master("local").appName("Word Count").getOrCreate() ``` Adding () to the builder method in SparkSession. ``` scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate() res0: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38 scala> SparkSession.builder.master("local").appName("Word Count").getOrCreate() res1: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38 ``` Author: Tejas Patil Closes #13086 from tejasapatil/doc_correction. (cherry picked from commit 4210e2a6b77f922f03eedf781a4230ab03a189cd) Signed-off-by: Reynold Xin --- sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 02c9dc03ae82e..100b43f6b58f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -755,7 +755,7 @@ object SparkSession { * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. * @since 2.0.0 */ - def builder: Builder = new Builder + def builder(): Builder = new Builder private val HIVE_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" private val HIVE_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" From 2d6f3bb4d56a0355625004806d0ed82fa407bce0 Mon Sep 17 00:00:00 2001 From: Nicholas Tietz Date: Sat, 14 May 2016 09:44:20 +0100 Subject: [PATCH 0222/1470] [SPARK-15197][DOCS] Added Scaladoc for countApprox and countByValueApprox parameters This pull request simply adds Scaladoc documentation of the parameters for countApprox and countByValueApprox. This is an important documentation change, as it clarifies what should be passed in for the timeout. Without units, this was previously unclear. I did not open a JIRA ticket per my understanding of the project contribution guidelines; as they state, the description in the ticket would be essentially just what is in the PR. If I should open one, let me know and I will do so. Author: Nicholas Tietz Closes #12955 from ntietz/rdd-countapprox-docs. (cherry picked from commit 0f1f31d3a6669fbac474518cf2a871485e202bdc) Signed-off-by: Sean Owen --- .../apache/spark/api/java/JavaRDDLike.scala | 29 +++++++++++++++++-- .../apache/spark/rdd/PairRDDFunctions.scala | 10 +++++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 16 ++++++++++ 3 files changed, 53 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index c17ca12379824..e4ccd9f11bd14 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -445,6 +445,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) @@ -452,6 +462,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * @param timeout maximum time to wait for the job, in milliseconds */ def countApprox(timeout: Long): PartialResult[BoundedDouble] = rdd.countApprox(timeout) @@ -464,7 +476,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { mapAsSerializableJavaMap(rdd.countByValue()).asInstanceOf[JMap[T, jl.Long]] /** - * (Experimental) Approximate version of countByValue(). + * Approximate version of countByValue(). + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByValueApprox( timeout: Long, @@ -473,7 +495,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { rdd.countByValueApprox(timeout, confidence).map(mapAsSerializableJavaMap) /** - * (Experimental) Approximate version of countByValue(). + * Approximate version of countByValue(). + * + * @param timeout maximum time to wait for the job, in milliseconds + * @return a potentially incomplete result, with error bounds */ def countByValueApprox(timeout: Long): PartialResult[JMap[T, BoundedDouble]] = rdd.countByValueApprox(timeout).map(mapAsSerializableJavaMap) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7936d8e1d45a2..3b12448d63933 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -375,6 +375,16 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = self.withScope { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index d85d0fff46be1..e6db9b3eec8ea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1107,10 +1107,21 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countApprox( timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L while (iter.hasNext) { @@ -1137,10 +1148,15 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of countByValue(). + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByValueApprox(timeout: Long, confidence: Double = 0.95) (implicit ord: Ordering[T] = null) : PartialResult[Map[T, BoundedDouble]] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } From d305f72275255f8d21ebbe62b545ac663d617f3b Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Sat, 14 May 2016 09:45:56 +0100 Subject: [PATCH 0223/1470] [SPARK-15096][ML] LogisticRegression MultiClassSummarizer numClasses can fail if no valid labels are found ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Throw better exception when numClasses is empty and empty.max is thrown. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Add a new unit test, which calls histogram with empty numClasses. Author: wm624@hotmail.com Closes #12969 from wangmiao1981/logisticR. (cherry picked from commit 354f8f11bd4b20fa99bd67a98da3525fd3d75c81) Signed-off-by: Sean Owen --- .../apache/spark/ml/classification/LogisticRegression.scala | 2 +- .../spark/ml/classification/LogisticRegressionSuite.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) 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 d2d4e249b4208..62d68973680a3 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 @@ -744,7 +744,7 @@ private[classification] class MultiClassSummarizer extends Serializable { def countInvalid: Long = totalInvalidCnt /** @return The number of distinct labels in the input dataset. */ - def numClasses: Int = distinctMap.keySet.max + 1 + def numClasses: Int = if (distinctMap.isEmpty) 0 else distinctMap.keySet.max + 1 /** @return The weightSum of each label in the input dataset. */ def histogram: Array[Double] = { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index f127aa217c94d..69650ebb3690a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -256,6 +256,10 @@ class LogisticRegressionSuite assert(summarizer4.countInvalid === 2) assert(summarizer4.numClasses === 4) + val summarizer5 = new MultiClassSummarizer + assert(summarizer5.histogram.isEmpty) + assert(summarizer5.numClasses === 0) + // small map merges large one val summarizerA = summarizer1.merge(summarizer2) assert(summarizerA.hashCode() === summarizer2.hashCode()) From 4f2f96f5e31004045d2ac2cd9f76467325ee9aa1 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Mon, 16 May 2016 10:41:20 +0800 Subject: [PATCH 0224/1470] [SPARK-15253][SQL] Support old table schema config key "spark.sql.sources.schema" for DESCRIBE TABLE ## What changes were proposed in this pull request? "DESCRIBE table" is broken when table schema is stored at key "spark.sql.sources.schema". Originally, we used spark.sql.sources.schema to store the schema of a data source table. After SPARK-6024, we removed this flag. Although we are not using spark.sql.sources.schema any more, we need to still support it. ## How was this patch tested? Unit test. When using spark2.0 to load a table generated by spark 1.2. Before change: `DESCRIBE table` => Schema of this table is inferred at runtime,, After change: `DESCRIBE table` => correct output. Author: Sean Zhong Closes #13073 from clockfly/spark-15253. (cherry picked from commit 4a5ee1954a6fb77231abb492355fe70313f0b35b) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/command/ddl.scala | 29 ++++++++++++------- .../sql/hive/MetastoreDataSourcesSuite.scala | 3 ++ 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 1c1716f050522..49d7fe956f812 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -497,20 +497,27 @@ private[sql] object DDLUtils { // will be inferred at runtime when the table is referenced. def getSchemaFromTableProperties(metadata: CatalogTable): Option[StructType] = { require(isDatasourceTable(metadata)) + val props = metadata.properties + if (props.isDefinedAt("spark.sql.sources.schema")) { + // Originally, we used spark.sql.sources.schema to store the schema of a data source table. + // After SPARK-6024, we removed this flag. + // Although we are not using spark.sql.sources.schema any more, we need to still support. + props.get("spark.sql.sources.schema").map(DataType.fromJson(_).asInstanceOf[StructType]) + } else { + metadata.properties.get("spark.sql.sources.schema.numParts").map { numParts => + val parts = (0 until numParts.toInt).map { index => + val part = metadata.properties.get(s"spark.sql.sources.schema.part.$index").orNull + if (part == null) { + throw new AnalysisException( + "Could not read schema from the metastore because it is corrupted " + + s"(missing part $index of the schema, $numParts parts are expected).") + } - metadata.properties.get("spark.sql.sources.schema.numParts").map { numParts => - val parts = (0 until numParts.toInt).map { index => - val part = metadata.properties.get(s"spark.sql.sources.schema.part.$index").orNull - if (part == null) { - throw new AnalysisException( - "Could not read schema from the metastore because it is corrupted " + - s"(missing part $index of the schema, $numParts parts are expected).") + part } - - part + // Stick all parts back to a single schema string. + DataType.fromJson(parts.mkString).asInstanceOf[StructType] } - // Stick all parts back to a single schema string. - DataType.fromJson(parts.mkString).asInstanceOf[StructType] } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 676fbd0a39b69..b507018e58d1d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -746,6 +746,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sessionState.invalidateTable(tableName) val actualSchema = table(tableName).schema assert(schema === actualSchema) + + // Checks the DESCRIBE output. + checkAnswer(sql("DESCRIBE spark6655"), Row("int", "int", "") :: Nil) } } From 5afde26a0b2e707f84f1be813970d436086ceb5c Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Mon, 16 May 2016 08:22:16 +0200 Subject: [PATCH 0225/1470] [SPARK-15305][ML][DOC] spark.ml document Bisectiong k-means has the incorrect format ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) The generated document has the incorrect format for biseckmeans. ![bug](https://cloud.githubusercontent.com/assets/5033592/15233120/d910098a-185a-11e6-901d-44aeafc8a011.jpg) ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Fix the formatting. ![fix](https://cloud.githubusercontent.com/assets/5033592/15233136/fce2ccd0-185a-11e6-9ded-14d71da4bdab.jpg) Author: wm624@hotmail.com Closes #13083 from wangmiao1981/doc. (cherry picked from commit c1836d66bdc93f80ff9e8852efe8f2d2bc1ca941) Signed-off-by: Nick Pentreath --- docs/ml-clustering.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 0d69bf67dfe46..a0955a3855ce7 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -86,7 +86,6 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. - ## Latent Dirichlet allocation (LDA) `LDA` is implemented as an `Estimator` that supports both `EMLDAOptimizer` and `OnlineLDAOptimizer`, @@ -116,8 +115,8 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. {% include_example python/ml/lda_example.py %} -## Bisecting k-means +## Bisecting k-means Bisecting k-means is a kind of [hierarchical clustering](https://en.wikipedia.org/wiki/Hierarchical_clustering) using a divisive (or "top-down") approach: all observations start in one cluster, and splits are performed recursively as one @@ -148,5 +147,4 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. {% include_example python/ml/bisecting_k_means_example.py %} - From f937ce7663a894f819882ea2df7c52dadd4b3681 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 16 May 2016 09:55:35 +0200 Subject: [PATCH 0226/1470] [SPARK-14979][ML][PYSPARK] Add examples for GeneralizedLinearRegression ## What changes were proposed in this pull request? Add Scala/Java/Python examples for ```GeneralizedLinearRegression```. ## How was this patch tested? They are examples and have been tested offline. Author: Yanbo Liang Closes #12754 from yanboliang/spark-14979. (cherry picked from commit f116a84ef8bf8a201c1a67154fda6990e4222074) Signed-off-by: Nick Pentreath --- ...avaGeneralizedLinearRegressionExample.java | 83 +++++++++++++++++++ .../generalized_linear_regression_example.py | 66 +++++++++++++++ .../GeneralizedLinearRegressionExample.scala | 78 +++++++++++++++++ 3 files changed, 227 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java create mode 100644 examples/src/main/python/ml/generalized_linear_regression_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java new file mode 100644 index 0000000000000..3f072d1e50eb5 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; + +import org.apache.spark.ml.regression.GeneralizedLinearRegression; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionModel; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionTrainingSummary; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * An example demonstrating generalized linear regression. + * Run with + *
    + * bin/run-example ml.JavaGeneralizedLinearRegressionExample
    + * 
    + */ + +public class JavaGeneralizedLinearRegressionExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaGeneralizedLinearRegressionExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset dataset = spark.read().format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt"); + + GeneralizedLinearRegression glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3); + + // Fit the model + GeneralizedLinearRegressionModel model = glr.fit(dataset); + + // Print the coefficients and intercept for generalized linear regression model + System.out.println("Coefficients: " + model.coefficients()); + System.out.println("Intercept: " + model.intercept()); + + // Summarize the model over the training set and print out some metrics + GeneralizedLinearRegressionTrainingSummary summary = model.summary(); + System.out.println("Coefficient Standard Errors: " + + Arrays.toString(summary.coefficientStandardErrors())); + System.out.println("T Values: " + Arrays.toString(summary.tValues())); + System.out.println("P Values: " + Arrays.toString(summary.pValues())); + System.out.println("Dispersion: " + summary.dispersion()); + System.out.println("Null Deviance: " + summary.nullDeviance()); + System.out.println("Residual Degree Of Freedom Null: " + summary.residualDegreeOfFreedomNull()); + System.out.println("Deviance: " + summary.deviance()); + System.out.println("Residual Degree Of Freedom: " + summary.residualDegreeOfFreedom()); + System.out.println("AIC: " + summary.aic()); + System.out.println("Deviance Residuals: "); + summary.residuals().show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/python/ml/generalized_linear_regression_example.py b/examples/src/main/python/ml/generalized_linear_regression_example.py new file mode 100644 index 0000000000000..796752a60f3ab --- /dev/null +++ b/examples/src/main/python/ml/generalized_linear_regression_example.py @@ -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. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.regression import GeneralizedLinearRegression +# $example off$ + +""" +An example demonstrating generalized linear regression. +Run with: + bin/spark-submit examples/src/main/python/ml/generalized_linear_regression_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("GeneralizedLinearRegressionExample")\ + .getOrCreate() + + # $example on$ + # Load training data + dataset = spark.read.format("libsvm")\ + .load("data/mllib/sample_linear_regression_data.txt") + + glr = GeneralizedLinearRegression(family="gaussian", link="identity", maxIter=10, regParam=0.3) + + # Fit the model + model = glr.fit(dataset) + + # Print the coefficients and intercept for generalized linear regression model + print("Coefficients: " + str(model.coefficients)) + print("Intercept: " + str(model.intercept)) + + # Summarize the model over the training set and print out some metrics + summary = model.summary + print("Coefficient Standard Errors: " + str(summary.coefficientStandardErrors)) + print("T Values: " + str(summary.tValues)) + print("P Values: " + str(summary.pValues)) + print("Dispersion: " + str(summary.dispersion)) + print("Null Deviance: " + str(summary.nullDeviance)) + print("Residual Degree Of Freedom Null: " + str(summary.residualDegreeOfFreedomNull)) + print("Deviance: " + str(summary.deviance)) + print("Residual Degree Of Freedom: " + str(summary.residualDegreeOfFreedom)) + print("AIC: " + str(summary.aic)) + print("Deviance Residuals: ") + summary.residuals().show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala new file mode 100644 index 0000000000000..1b86d7cad0b38 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.regression.GeneralizedLinearRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating generalized linear regression. + * Run with + * {{{ + * bin/run-example ml.GeneralizedLinearRegressionExample + * }}} + */ + +object GeneralizedLinearRegressionExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("GeneralizedLinearRegressionExample") + .getOrCreate() + + // $example on$ + // Load training data + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt") + + val glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3) + + // Fit the model + val model = glr.fit(dataset) + + // Print the coefficients and intercept for generalized linear regression model + println(s"Coefficients: ${model.coefficients}") + println(s"Intercept: ${model.intercept}") + + // Summarize the model over the training set and print out some metrics + val summary = model.summary + println(s"Coefficient Standard Errors: ${summary.coefficientStandardErrors.mkString(",")}") + println(s"T Values: ${summary.tValues.mkString(",")}") + println(s"P Values: ${summary.pValues.mkString(",")}") + println(s"Dispersion: ${summary.dispersion}") + println(s"Null Deviance: ${summary.nullDeviance}") + println(s"Residual Degree Of Freedom Null: ${summary.residualDegreeOfFreedomNull}") + println(s"Deviance: ${summary.deviance}") + println(s"Residual Degree Of Freedom: ${summary.residualDegreeOfFreedom}") + println(s"AIC: ${summary.aic}") + println("Deviance Residuals: ") + summary.residuals().show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println From 0dd1f872082f9e5d7712a1f7c1035dd2786b2a39 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Mon, 16 May 2016 12:59:55 -0700 Subject: [PATCH 0227/1470] [SPARK-14942][SQL][STREAMING] Reduce delay between batch construction and execution ## Problem Currently in `StreamExecution`, [we first run the batch, then construct the next](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L165): ```scala if (dataAvailable) runBatch() constructNextBatch() ``` This is good when we run batches ASAP, where data would get processed in the **very next batch**: ![1](https://cloud.githubusercontent.com/assets/15843379/14779964/2786e698-0b0d-11e6-9d2c-bb41513488b2.png) However, when we run batches at trigger like `ProcessTime("1 minute")`, data - such as _y_ below - may not get processed in the very next batch i.e. _batch 1_, but in _batch 2_: ![2](https://cloud.githubusercontent.com/assets/15843379/14779818/6f3bb064-0b0c-11e6-9f16-c1ce4897186b.png) ## What changes were proposed in this pull request? This patch reverses the order of `constructNextBatch()` and `runBatch()`. After this patch, data would get processed in the **very next batch**, i.e. _batch 1_: ![3](https://cloud.githubusercontent.com/assets/15843379/14779816/6f36ee62-0b0c-11e6-9e53-bc8397fade18.png) In addition, this patch alters when we do `currentBatchId += 1`: let's do that when the processing of the current batch's data is completed, so we won't bother passing `currentBatchId + 1` or `currentBatchId - 1` to states or sinks. ## How was this patch tested? New added test case. Also this should be covered by existing test suits, e.g. stress tests and others. Author: Liwei Lin Closes #12725 from lw-lin/construct-before-run-3. (cherry picked from commit 95f4fbae52d26ede94c3ba8248394749f3d95dcc) Signed-off-by: Shixiong Zhu --- .../streaming/IncrementalExecution.scala | 6 +- .../execution/streaming/StreamExecution.scala | 24 ++++-- .../sql/execution/streaming/memory.scala | 4 + .../spark/sql/streaming/StreamSuite.scala | 84 +++++++++++++++++-- 4 files changed, 99 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index e9052a309595a..8b96f65bc31ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -27,12 +27,12 @@ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, * A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]] * plan incrementally. Possibly preserving state in between each execution. */ -class IncrementalExecution( +class IncrementalExecution private[sql]( sparkSession: SparkSession, logicalPlan: LogicalPlan, outputMode: OutputMode, checkpointLocation: String, - currentBatchId: Long) + val currentBatchId: Long) extends QueryExecution(sparkSession, logicalPlan) { // TODO: make this always part of planning. @@ -57,7 +57,7 @@ class IncrementalExecution( case StateStoreSaveExec(keys, None, UnaryExecNode(agg, StateStoreRestoreExec(keys2, None, child))) => - val stateId = OperatorStateId(checkpointLocation, operatorId, currentBatchId - 1) + val stateId = OperatorStateId(checkpointLocation, operatorId, currentBatchId) operatorId += 1 StateStoreSaveExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ea367b699f568..df6304d85fe74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -122,7 +122,7 @@ class StreamExecution( * processing is done. Thus, the Nth record in this log indicated data that is currently being * processed and the N-1th entry indicates which offsets have been durably committed to the sink. */ - private val offsetLog = + private[sql] val offsetLog = new HDFSMetadataLog[CompositeOffset](sparkSession, checkpointFile("offsets")) /** Whether the query is currently active or not */ @@ -174,12 +174,21 @@ class StreamExecution( // While active, repeatedly attempt to run batches. SQLContext.setActive(sparkSession.wrapped) - populateStartOffsets() - logDebug(s"Stream running from $committedOffsets to $availableOffsets") + triggerExecutor.execute(() => { if (isActive) { - if (dataAvailable) runBatch() - constructNextBatch() + if (currentBatchId < 0) { + // We'll do this initialization only once + populateStartOffsets() + logDebug(s"Stream running from $committedOffsets to $availableOffsets") + } else { + constructNextBatch() + } + if (dataAvailable) { + runBatch() + // We'll increase currentBatchId after we complete processing current batch's data + currentBatchId += 1 + } true } else { false @@ -214,7 +223,7 @@ class StreamExecution( offsetLog.getLatest() match { case Some((batchId, nextOffsets)) => logInfo(s"Resuming continuous query, starting with batch $batchId") - currentBatchId = batchId + 1 + currentBatchId = batchId availableOffsets = nextOffsets.toStreamProgress(sources) logDebug(s"Found possibly uncommitted offsets $availableOffsets") @@ -285,7 +294,6 @@ class StreamExecution( offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") } - currentBatchId += 1 logInfo(s"Committed offsets for batch $currentBatchId.") } else { awaitBatchLock.lock() @@ -352,7 +360,7 @@ class StreamExecution( val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) - sink.addBatch(currentBatchId - 1, nextBatch) + sink.addBatch(currentBatchId, nextBatch) awaitBatchLock.lock() try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index a34927ff994a4..bcc33ae8c88bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -124,6 +124,10 @@ class MemorySink(val schema: StructType) extends Sink with Logging { batches.flatten } + def latestBatchId: Option[Int] = synchronized { + if (batches.size == 0) None else Some(batches.size - 1) + } + def lastBatch: Seq[Row] = synchronized { batches.last } def toDebugString: String = synchronized { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 6a8b280174674..013b7316938eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -137,20 +137,88 @@ class StreamSuite extends StreamTest with SharedSQLContext { } } - // This would fail for now -- error is "Timed out waiting for stream" - // Root cause is that data generated in batch 0 may not get processed in batch 1 - // Let's enable this after SPARK-14942: Reduce delay between batch construction and execution - ignore("minimize delay between batch construction and execution") { + test("minimize delay between batch construction and execution") { + + // For each batch, we would retrieve new data's offsets and log them before we run the execution + // This checks whether the key of the offset log is the expected batch id + def CheckOffsetLogLatestBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.offsetLog.getLatest().get._1 == expectedId, + s"offsetLog's latest should be $expectedId") + + // For each batch, we would log the state change during the execution + // This checks whether the key of the state change log is the expected batch id + def CheckIncrementalExecutionCurrentBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.lastExecution.asInstanceOf[IncrementalExecution].currentBatchId == expectedId, + s"lastExecution's currentBatchId should be $expectedId") + + // For each batch, we would log the sink change after the execution + // This checks whether the key of the sink change log is the expected batch id + def CheckSinkLatestBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.sink.asInstanceOf[MemorySink].latestBatchId.get == expectedId, + s"sink's lastBatchId should be $expectedId") + val inputData = MemoryStream[Int] testStream(inputData.toDS())( StartStream(ProcessingTime("10 seconds"), new ManualClock), + /* -- batch 0 ----------------------- */ - AddData(inputData, 1), - AddData(inputData, 2), - AddData(inputData, 3), + // Add some data in batch 0 + AddData(inputData, 1, 2, 3), AdvanceManualClock(10 * 1000), // 10 seconds + /* -- batch 1 ----------------------- */ - CheckAnswer(1, 2, 3)) + // Check the results of batch 0 + CheckAnswer(1, 2, 3), + CheckIncrementalExecutionCurrentBatchId(0), + CheckOffsetLogLatestBatchId(0), + CheckSinkLatestBatchId(0), + // Add some data in batch 1 + AddData(inputData, 4, 5, 6), + AdvanceManualClock(10 * 1000), + + /* -- batch _ ----------------------- */ + // Check the results of batch 1 + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + + AdvanceManualClock(10 * 1000), + AdvanceManualClock(10 * 1000), + AdvanceManualClock(10 * 1000), + + /* -- batch __ ---------------------- */ + // Check the results of batch 1 again; this is to make sure that, when there's no new data, + // the currentId does not get logged (e.g. as 2) even if the clock has advanced many times + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + + /* Stop then restart the Stream */ + StopStream, + StartStream(ProcessingTime("10 seconds"), new ManualClock), + + /* -- batch 1 rerun ----------------- */ + // this batch 1 would re-run because the latest batch id logged in offset log is 1 + AdvanceManualClock(10 * 1000), + + /* -- batch 2 ----------------------- */ + // Check the results of batch 1 + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + // Add some data in batch 2 + AddData(inputData, 7, 8, 9), + AdvanceManualClock(10 * 1000), + + /* -- batch 3 ----------------------- */ + // Check the results of batch 2 + CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8, 9), + CheckIncrementalExecutionCurrentBatchId(2), + CheckOffsetLogLatestBatchId(2), + CheckSinkLatestBatchId(2)) } } From 8e3ee683bb7ecc857480bc347e7a814e5a63ff28 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 17 May 2016 00:08:02 -0700 Subject: [PATCH 0228/1470] [SPARK-14906][ML] Copy linalg in PySpark to new ML package ## What changes were proposed in this pull request? Copy the linalg (Vector/Matrix and VectorUDT/MatrixUDT) in PySpark to new ML package. ## How was this patch tested? Existing tests. Author: Xiangrui Meng Author: Liang-Chi Hsieh Author: Liang-Chi Hsieh Closes #13099 from viirya/move-pyspark-vector-matrix-udt4. (cherry picked from commit 8ad9f08c94e98317a9095dd53d737c1b8df6e29c) Signed-off-by: Xiangrui Meng --- python/docs/pyspark.ml.rst | 8 + python/pyspark/ml/linalg/__init__.py | 1145 ++++++++++++++++++++++++++ python/pyspark/ml/tests.py | 456 +++++++++- 3 files changed, 1564 insertions(+), 45 deletions(-) create mode 100644 python/pyspark/ml/linalg/__init__.py diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst index 86d4186a2c798..26f7415e1a423 100644 --- a/python/docs/pyspark.ml.rst +++ b/python/docs/pyspark.ml.rst @@ -41,6 +41,14 @@ pyspark.ml.clustering module :undoc-members: :inherited-members: +pyspark.ml.linalg module +---------------------------- + +.. automodule:: pyspark.ml.linalg + :members: + :undoc-members: + :inherited-members: + pyspark.ml.recommendation module -------------------------------- diff --git a/python/pyspark/ml/linalg/__init__.py b/python/pyspark/ml/linalg/__init__.py new file mode 100644 index 0000000000000..f42c589b92255 --- /dev/null +++ b/python/pyspark/ml/linalg/__init__.py @@ -0,0 +1,1145 @@ +# +# 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. +# + +""" +MLlib utilities for linear algebra. For dense vectors, MLlib +uses the NumPy C{array} type, so you can simply pass NumPy arrays +around. For sparse vectors, users can construct a L{SparseVector} +object from MLlib or pass SciPy C{scipy.sparse} column vectors if +SciPy is available in their environment. +""" + +import sys +import array +import struct + +if sys.version >= '3': + basestring = str + xrange = range + import copyreg as copy_reg + long = int +else: + from itertools import izip as zip + import copy_reg + +import numpy as np + +from pyspark import since +from pyspark.sql.types import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ + IntegerType, ByteType, BooleanType + + +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', + 'Matrix', 'DenseMatrix', 'SparseMatrix', 'Matrices'] + + +if sys.version_info[:2] == (2, 7): + # speed up pickling array in Python 2.7 + def fast_pickle_array(ar): + return array.array, (ar.typecode, ar.tostring()) + copy_reg.pickle(array.array, fast_pickle_array) + + +# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, +# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. + +try: + import scipy.sparse + _have_scipy = True +except: + # No SciPy in environment, but that's okay + _have_scipy = False + + +def _convert_to_vector(l): + if isinstance(l, Vector): + return l + 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" + csc = l.tocsc() + return SparseVector(l.shape[0], csc.indices, csc.data) + else: + raise TypeError("Cannot convert type %s into Vector" % type(l)) + + +def _vector_size(v): + """ + Returns the size of the vector. + + >>> _vector_size([1., 2., 3.]) + 3 + >>> _vector_size((1., 2., 3.)) + 3 + >>> _vector_size(array.array('d', [1., 2., 3.])) + 3 + >>> _vector_size(np.zeros(3)) + 3 + >>> _vector_size(np.zeros((3, 1))) + 3 + >>> _vector_size(np.zeros((1, 3))) + Traceback (most recent call last): + ... + ValueError: Cannot treat an ndarray of shape (1, 3) as a vector + """ + if isinstance(v, Vector): + return len(v) + 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): + return len(v) + else: + raise ValueError("Cannot treat an ndarray of shape %s as a vector" % str(v.shape)) + elif _have_scipy and scipy.sparse.issparse(v): + assert v.shape[1] == 1, "Expected column vector" + return v.shape[0] + else: + raise TypeError("Cannot treat type %s as a vector" % type(v)) + + +def _format_float(f, digits=4): + s = str(round(f, digits)) + if '.' in s: + s = s[:s.index('.') + 1 + digits] + return s + + +def _format_float_list(l): + return [_format_float(x) for x in l] + + +def _double_to_long_bits(value): + if np.isnan(value): + value = float('nan') + # pack double into 64 bits, then unpack as long int + return struct.unpack('Q', struct.pack('d', value))[0] + + +class VectorUDT(UserDefinedType): + """ + SQL user-defined type (UDT) for Vector. + """ + + @classmethod + def sqlType(cls): + return StructType([ + StructField("type", ByteType(), False), + StructField("size", IntegerType(), True), + StructField("indices", ArrayType(IntegerType(), False), True), + StructField("values", ArrayType(DoubleType(), False), True)]) + + @classmethod + def module(cls): + return "pyspark.ml.linalg" + + @classmethod + def scalaUDT(cls): + return "org.apache.spark.ml.linalg.VectorUDT" + + def serialize(self, obj): + if isinstance(obj, SparseVector): + indices = [int(i) for i in obj.indices] + values = [float(v) for v in obj.values] + return (0, obj.size, indices, values) + elif isinstance(obj, DenseVector): + values = [float(v) for v in obj] + return (1, None, None, values) + else: + raise TypeError("cannot serialize %r of type %r" % (obj, type(obj))) + + def deserialize(self, datum): + assert len(datum) == 4, \ + "VectorUDT.deserialize given row with length %d but requires 4" % len(datum) + tpe = datum[0] + if tpe == 0: + return SparseVector(datum[1], datum[2], datum[3]) + elif tpe == 1: + return DenseVector(datum[3]) + else: + raise ValueError("do not recognize type %r" % tpe) + + def simpleString(self): + return "vector" + + +class MatrixUDT(UserDefinedType): + """ + SQL user-defined type (UDT) for Matrix. + """ + + @classmethod + def sqlType(cls): + return StructType([ + StructField("type", ByteType(), False), + StructField("numRows", IntegerType(), False), + StructField("numCols", IntegerType(), False), + StructField("colPtrs", ArrayType(IntegerType(), False), True), + StructField("rowIndices", ArrayType(IntegerType(), False), True), + StructField("values", ArrayType(DoubleType(), False), True), + StructField("isTransposed", BooleanType(), False)]) + + @classmethod + def module(cls): + return "pyspark.ml.linalg" + + @classmethod + def scalaUDT(cls): + return "org.apache.spark.ml.linalg.MatrixUDT" + + def serialize(self, obj): + if isinstance(obj, SparseMatrix): + colPtrs = [int(i) for i in obj.colPtrs] + rowIndices = [int(i) for i in obj.rowIndices] + values = [float(v) for v in obj.values] + return (0, obj.numRows, obj.numCols, colPtrs, + rowIndices, values, bool(obj.isTransposed)) + elif isinstance(obj, DenseMatrix): + values = [float(v) for v in obj.values] + return (1, obj.numRows, obj.numCols, None, None, values, + bool(obj.isTransposed)) + else: + raise TypeError("cannot serialize type %r" % (type(obj))) + + def deserialize(self, datum): + assert len(datum) == 7, \ + "MatrixUDT.deserialize given row with length %d but requires 7" % len(datum) + tpe = datum[0] + if tpe == 0: + return SparseMatrix(*datum[1:]) + elif tpe == 1: + return DenseMatrix(datum[1], datum[2], datum[5], datum[6]) + else: + raise ValueError("do not recognize type %r" % tpe) + + def simpleString(self): + return "matrix" + + +class Vector(object): + + __UDT__ = VectorUDT() + + """ + Abstract class for DenseVector and SparseVector + """ + def toArray(self): + """ + Convert the vector into an numpy.ndarray + + :return: numpy.ndarray + """ + raise NotImplementedError + + +class DenseVector(Vector): + """ + A dense vector represented by a value array. We use numpy array for + storage and arithmetics will be delegated to the underlying numpy + array. + + >>> v = Vectors.dense([1.0, 2.0]) + >>> u = Vectors.dense([3.0, 4.0]) + >>> v + u + DenseVector([4.0, 6.0]) + >>> 2 - v + DenseVector([1.0, 0.0]) + >>> v / 2 + DenseVector([0.5, 1.0]) + >>> v * u + DenseVector([3.0, 8.0]) + >>> u / v + DenseVector([3.0, 2.0]) + >>> u % 2 + DenseVector([1.0, 0.0]) + """ + def __init__(self, ar): + if isinstance(ar, bytes): + ar = np.frombuffer(ar, dtype=np.float64) + elif not isinstance(ar, np.ndarray): + ar = np.array(ar, dtype=np.float64) + if ar.dtype != np.float64: + ar = ar.astype(np.float64) + self.array = ar + + def __reduce__(self): + return DenseVector, (self.array.tostring(),) + + def numNonzeros(self): + """ + Number of nonzero elements. This scans all active values and count non zeros + """ + return np.count_nonzero(self.array) + + def norm(self, p): + """ + Calculates the norm of a DenseVector. + + >>> a = DenseVector([0, -1, 2, -3]) + >>> a.norm(2) + 3.7... + >>> a.norm(1) + 6.0 + """ + return np.linalg.norm(self.array, p) + + def dot(self, other): + """ + Compute the dot product of two Vectors. We support + (Numpy array, list, SparseVector, or SciPy sparse) + and a target NumPy array that is either 1- or 2-dimensional. + Equivalent to calling numpy.dot of the two vectors. + + >>> dense = DenseVector(array.array('d', [1., 2.])) + >>> dense.dot(dense) + 5.0 + >>> dense.dot(SparseVector(2, [0, 1], [2., 1.])) + 4.0 + >>> dense.dot(range(1, 3)) + 5.0 + >>> dense.dot(np.array(range(1, 3))) + 5.0 + >>> dense.dot([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense.dot(np.reshape([1., 2., 3., 4.], (2, 2), order='F')) + array([ 5., 11.]) + >>> dense.dot(np.reshape([1., 2., 3.], (3, 1), order='F')) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + if type(other) == np.ndarray: + if other.ndim > 1: + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.array, other) + elif _have_scipy and scipy.sparse.issparse(other): + assert len(self) == other.shape[0], "dimension mismatch" + return other.transpose().dot(self.toArray()) + else: + assert len(self) == _vector_size(other), "dimension mismatch" + if isinstance(other, SparseVector): + return other.dot(self) + elif isinstance(other, Vector): + return np.dot(self.toArray(), other.toArray()) + else: + return np.dot(self.toArray(), other) + + def squared_distance(self, other): + """ + Squared distance of two Vectors. + + >>> dense1 = DenseVector(array.array('d', [1., 2.])) + >>> dense1.squared_distance(dense1) + 0.0 + >>> dense2 = np.array([2., 1.]) + >>> dense1.squared_distance(dense2) + 2.0 + >>> dense3 = [2., 1.] + >>> dense1.squared_distance(dense3) + 2.0 + >>> sparse1 = SparseVector(2, [0, 1], [2., 1.]) + >>> dense1.squared_distance(sparse1) + 2.0 + >>> dense1.squared_distance([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense1.squared_distance(SparseVector(1, [0,], [1.,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + assert len(self) == _vector_size(other), "dimension mismatch" + if isinstance(other, SparseVector): + return other.squared_distance(self) + elif _have_scipy and scipy.sparse.issparse(other): + return _convert_to_vector(other).squared_distance(self) + + if isinstance(other, Vector): + other = other.toArray() + elif not isinstance(other, np.ndarray): + other = np.array(other) + diff = self.toArray() - other + return np.dot(diff, diff) + + def toArray(self): + """ + Returns an numpy.ndarray + """ + return self.array + + @property + def values(self): + """ + Returns a list of values + """ + return self.array + + def __getitem__(self, item): + return self.array[item] + + def __len__(self): + return len(self.array) + + def __str__(self): + return "[" + ",".join([str(v) for v in self.array]) + "]" + + def __repr__(self): + return "DenseVector([%s])" % (', '.join(_format_float(i) for i in self.array)) + + def __eq__(self, other): + if isinstance(other, DenseVector): + return np.array_equal(self.array, other.array) + elif isinstance(other, SparseVector): + if len(self) != other.size: + return False + return Vectors._equals(list(xrange(len(self))), self.array, other.indices, other.values) + return False + + def __ne__(self, other): + return not self == other + + def __hash__(self): + size = len(self) + result = 31 + size + nnz = 0 + i = 0 + while i < size and nnz < 128: + if self.array[i] != 0: + result = 31 * result + i + bits = _double_to_long_bits(self.array[i]) + result = 31 * result + (bits ^ (bits >> 32)) + nnz += 1 + i += 1 + return result + + def __getattr__(self, item): + return getattr(self.array, item) + + def _delegate(op): + def func(self, other): + if isinstance(other, DenseVector): + other = other.array + return DenseVector(getattr(self.array, op)(other)) + return func + + __neg__ = _delegate("__neg__") + __add__ = _delegate("__add__") + __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__") + + +class SparseVector(Vector): + """ + A simple sparse vector class for passing data to MLlib. Users may + alternatively pass SciPy's {scipy.sparse} data types. + """ + def __init__(self, size, *args): + """ + Create a sparse vector, using either a dictionary, a list of + (index, value) pairs, or two separate arrays of indices and + values (sorted by index). + + :param size: Size of the vector. + :param args: Active entries, as a dictionary {index: value, ...}, + a list of tuples [(index, value), ...], or a list of strictly + increasing indices and a list of corresponding values [index, ...], + [value, ...]. Inactive entries are treated as zeros. + + >>> 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) + """ Size of the vector. """ + assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" + if len(args) == 1: + pairs = args[0] + if type(pairs) == dict: + pairs = pairs.items() + pairs = sorted(pairs) + self.indices = np.array([p[0] for p in pairs], dtype=np.int32) + """ A list of indices corresponding to active entries. """ + self.values = np.array([p[1] for p in pairs], dtype=np.float64) + """ A list of values corresponding to active entries. """ + else: + 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) + else: + # np.frombuffer() doesn't work well with empty string in older version + self.indices = np.array([], dtype=np.int32) + self.values = np.array([], dtype=np.float64) + else: + self.indices = np.array(args[0], dtype=np.int32) + self.values = np.array(args[1], dtype=np.float64) + assert len(self.indices) == len(self.values), "index and value arrays not same length" + for i in xrange(len(self.indices) - 1): + if self.indices[i] >= self.indices[i + 1]: + raise TypeError( + "Indices %s and %s are not strictly increasing" + % (self.indices[i], self.indices[i + 1])) + + def numNonzeros(self): + """ + Number of nonzero elements. This scans all active values and count non zeros. + """ + return np.count_nonzero(self.values) + + def norm(self, p): + """ + Calculates the norm of a SparseVector. + + >>> a = SparseVector(4, [0, 1], [3., -4.]) + >>> a.norm(1) + 7.0 + >>> a.norm(2) + 5.0 + """ + return np.linalg.norm(self.values, p) + + def __reduce__(self): + return ( + SparseVector, + (self.size, self.indices.tostring(), self.values.tostring())) + + def dot(self, other): + """ + Dot product with a SparseVector or 1- or 2-dimensional Numpy array. + + >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) + >>> a.dot(a) + 25.0 + >>> a.dot(array.array('d', [1., 2., 3., 4.])) + 22.0 + >>> b = SparseVector(4, [2], [1.0]) + >>> a.dot(b) + 0.0 + >>> a.dot(np.array([[1, 1], [2, 2], [3, 3], [4, 4]])) + array([ 22., 22.]) + >>> a.dot([1., 2., 3.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.array([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(DenseVector([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.zeros((3, 2))) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + + if isinstance(other, np.ndarray): + if other.ndim not in [2, 1]: + raise ValueError("Cannot call dot with %d-dimensional array" % other.ndim) + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.values, other[self.indices]) + + assert len(self) == _vector_size(other), "dimension mismatch" + + if isinstance(other, DenseVector): + return np.dot(other.array[self.indices], self.values) + + elif isinstance(other, SparseVector): + # Find out common indices. + self_cmind = np.in1d(self.indices, other.indices, assume_unique=True) + self_values = self.values[self_cmind] + if self_values.size == 0: + return 0.0 + else: + other_cmind = np.in1d(other.indices, self.indices, assume_unique=True) + return np.dot(self_values, other.values[other_cmind]) + + else: + return self.dot(_convert_to_vector(other)) + + def squared_distance(self, other): + """ + Squared distance from a SparseVector or 1-dimensional NumPy array. + + >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) + >>> a.squared_distance(a) + 0.0 + >>> a.squared_distance(array.array('d', [1., 2., 3., 4.])) + 11.0 + >>> a.squared_distance(np.array([1., 2., 3., 4.])) + 11.0 + >>> b = SparseVector(4, [2], [1.0]) + >>> a.squared_distance(b) + 26.0 + >>> b.squared_distance(a) + 26.0 + >>> b.squared_distance([1., 2.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> b.squared_distance(SparseVector(3, [1,], [1.0,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + assert len(self) == _vector_size(other), "dimension mismatch" + + if isinstance(other, np.ndarray) or isinstance(other, DenseVector): + if isinstance(other, np.ndarray) and other.ndim != 1: + raise Exception("Cannot call squared_distance with %d-dimensional array" % + other.ndim) + if isinstance(other, DenseVector): + other = other.array + sparse_ind = np.zeros(other.size, dtype=bool) + sparse_ind[self.indices] = True + dist = other[sparse_ind] - self.values + result = np.dot(dist, dist) + + other_ind = other[~sparse_ind] + result += np.dot(other_ind, other_ind) + return result + + elif isinstance(other, SparseVector): + result = 0.0 + i, j = 0, 0 + while i < len(self.indices) and j < len(other.indices): + if self.indices[i] == other.indices[j]: + diff = self.values[i] - other.values[j] + result += diff * diff + i += 1 + j += 1 + elif self.indices[i] < other.indices[j]: + result += self.values[i] * self.values[i] + i += 1 + else: + result += other.values[j] * other.values[j] + j += 1 + while i < len(self.indices): + result += self.values[i] * self.values[i] + i += 1 + while j < len(other.indices): + result += other.values[j] * other.values[j] + j += 1 + return result + else: + return self.squared_distance(_convert_to_vector(other)) + + def toArray(self): + """ + Returns a copy of this SparseVector as a 1-dimensional NumPy array. + """ + arr = np.zeros((self.size,), dtype=np.float64) + arr[self.indices] = self.values + return arr + + def __len__(self): + return self.size + + def __str__(self): + inds = "[" + ",".join([str(i) for i in self.indices]) + "]" + vals = "[" + ",".join([str(v) for v in self.values]) + "]" + return "(" + ",".join((str(self.size), inds, vals)) + ")" + + def __repr__(self): + inds = self.indices + vals = self.values + entries = ", ".join(["{0}: {1}".format(inds[i], _format_float(vals[i])) + for i in xrange(len(inds))]) + return "SparseVector({0}, {{{1}}})".format(self.size, entries) + + def __eq__(self, other): + if isinstance(other, SparseVector): + return other.size == self.size and np.array_equal(other.indices, self.indices) \ + and np.array_equal(other.values, self.values) + elif isinstance(other, DenseVector): + if self.size != len(other): + return False + return Vectors._equals(self.indices, self.values, list(xrange(len(other))), other.array) + return False + + def __getitem__(self, index): + inds = self.indices + vals = self.values + if not isinstance(index, int): + raise TypeError( + "Indices must be of type integer, got type %s" % type(index)) + + if index >= self.size or index < -self.size: + raise ValueError("Index %d out of bounds." % index) + if index < 0: + index += self.size + + if (inds.size == 0) or (index > inds.item(-1)): + return 0. + + insert_index = np.searchsorted(inds, index) + row_ind = inds[insert_index] + if row_ind == index: + return vals[insert_index] + return 0. + + def __ne__(self, other): + return not self.__eq__(other) + + def __hash__(self): + result = 31 + self.size + nnz = 0 + i = 0 + while i < len(self.values) and nnz < 128: + if self.values[i] != 0: + result = 31 * result + int(self.indices[i]) + bits = _double_to_long_bits(self.values[i]) + result = 31 * result + (bits ^ (bits >> 32)) + nnz += 1 + i += 1 + return result + + +class Vectors(object): + + """ + Factory methods for working with vectors. Note that dense vectors + are simply represented as NumPy array objects, so there is no need + to covert them for use in MLlib. For sparse vectors, the factory + methods in this class create an MLlib-compatible type, or users + can pass in SciPy's C{scipy.sparse} column vectors. + """ + + @staticmethod + def sparse(size, *args): + """ + Create a sparse vector, using either a dictionary, a list of + (index, value) pairs, or two separate arrays of indices and + values (sorted by index). + + :param size: Size of the vector. + :param args: Non-zero entries, as a dictionary, list of tuples, + or two sorted lists containing indices and values. + + >>> 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) + + @staticmethod + def dense(*elements): + """ + Create a dense vector of 64-bit floats from a Python list or numbers. + + >>> Vectors.dense([1, 2, 3]) + DenseVector([1.0, 2.0, 3.0]) + >>> Vectors.dense(1.0, 2.0) + DenseVector([1.0, 2.0]) + """ + if len(elements) == 1 and not isinstance(elements[0], (float, int, long)): + # it's list, numpy.array or other iterable object. + elements = elements[0] + return DenseVector(elements) + + @staticmethod + def squared_distance(v1, v2): + """ + Squared distance between two vectors. + a and b can be of type SparseVector, DenseVector, np.ndarray + or array.array. + + >>> a = Vectors.sparse(4, [(0, 1), (3, 4)]) + >>> b = Vectors.dense([2, 5, 4, 1]) + >>> a.squared_distance(b) + 51.0 + """ + v1, v2 = _convert_to_vector(v1), _convert_to_vector(v2) + return v1.squared_distance(v2) + + @staticmethod + def norm(vector, p): + """ + Find norm of the given vector. + """ + return _convert_to_vector(vector).norm(p) + + @staticmethod + def zeros(size): + return DenseVector(np.zeros(size)) + + @staticmethod + def _equals(v1_indices, v1_values, v2_indices, v2_values): + """ + Check equality between sparse/dense vectors, + v1_indices and v2_indices assume to be strictly increasing. + """ + v1_size = len(v1_values) + v2_size = len(v2_values) + k1 = 0 + k2 = 0 + all_equal = True + while all_equal: + while k1 < v1_size and v1_values[k1] == 0: + k1 += 1 + while k2 < v2_size and v2_values[k2] == 0: + k2 += 1 + + if k1 >= v1_size or k2 >= v2_size: + return k1 >= v1_size and k2 >= v2_size + + all_equal = v1_indices[k1] == v2_indices[k2] and v1_values[k1] == v2_values[k2] + k1 += 1 + k2 += 1 + return all_equal + + +class Matrix(object): + + __UDT__ = MatrixUDT() + + """ + Represents a local matrix. + """ + def __init__(self, numRows, numCols, isTransposed=False): + self.numRows = numRows + self.numCols = numCols + self.isTransposed = isTransposed + + def toArray(self): + """ + Returns its elements in a NumPy ndarray. + """ + 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, bytes): + return np.frombuffer(array_like, dtype=dtype) + return np.asarray(array_like, dtype=dtype) + + +class DenseMatrix(Matrix): + """ + Column-major dense matrix. + """ + def __init__(self, numRows, numCols, values, isTransposed=False): + Matrix.__init__(self, numRows, numCols, isTransposed) + values = self._convert_to_array(values, np.float64) + assert len(values) == numRows * numCols + self.values = values + + def __reduce__(self): + return DenseMatrix, ( + self.numRows, self.numCols, self.values.tostring(), + int(self.isTransposed)) + + def __str__(self): + """ + Pretty printing of a DenseMatrix + + >>> dm = DenseMatrix(2, 2, range(4)) + >>> print(dm) + DenseMatrix([[ 0., 2.], + [ 1., 3.]]) + >>> dm = DenseMatrix(2, 2, range(4), isTransposed=True) + >>> print(dm) + DenseMatrix([[ 0., 1.], + [ 2., 3.]]) + """ + # Inspired by __repr__ in scipy matrices. + array_lines = repr(self.toArray()).splitlines() + + # We need to adjust six spaces which is the difference in number + # of letters between "DenseMatrix" and "array" + x = '\n'.join([(" " * 6 + line) for line in array_lines[1:]]) + return array_lines[0].replace("array", "DenseMatrix") + "\n" + x + + def __repr__(self): + """ + Representation of a DenseMatrix + + >>> dm = DenseMatrix(2, 2, range(4)) + >>> dm + DenseMatrix(2, 2, [0.0, 1.0, 2.0, 3.0], False) + """ + # If the number of values are less than seventeen then return as it is. + # Else return first eight values and last eight values. + if len(self.values) < 17: + entries = _format_float_list(self.values) + else: + entries = ( + _format_float_list(self.values[:8]) + + ["..."] + + _format_float_list(self.values[-8:]) + ) + + entries = ", ".join(entries) + return "DenseMatrix({0}, {1}, [{2}], {3})".format( + self.numRows, self.numCols, entries, self.isTransposed) + + def toArray(self): + """ + Return an numpy.ndarray + + >>> m = DenseMatrix(2, 2, range(4)) + >>> m.toArray() + array([[ 0., 2.], + [ 1., 3.]]) + """ + if self.isTransposed: + return np.asfortranarray( + self.values.reshape((self.numRows, self.numCols))) + else: + return self.values.reshape((self.numRows, self.numCols), order='F') + + def toSparse(self): + """Convert to SparseMatrix""" + if self.isTransposed: + values = np.ravel(self.toArray(), order='F') + else: + values = self.values + indices = np.nonzero(values)[0] + colCounts = np.bincount(indices // self.numRows) + colPtrs = np.cumsum(np.hstack( + (0, colCounts, np.zeros(self.numCols - colCounts.size)))) + values = 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: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j >= self.numCols or j < 0: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + + if self.isTransposed: + return self.values[i * self.numCols + j] + else: + return self.values[i + j * self.numRows] + + def __eq__(self, other): + if (not isinstance(other, DenseMatrix) or + self.numRows != other.numRows or + self.numCols != other.numCols): + return False + + self_values = np.ravel(self.toArray(), order='F') + other_values = np.ravel(other.toArray(), order='F') + return 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, 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 __str__(self): + """ + Pretty printing of a SparseMatrix + + >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + >>> print(sm1) + 2 X 2 CSCMatrix + (0,0) 2.0 + (1,0) 3.0 + (1,1) 4.0 + >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4], True) + >>> print(sm1) + 2 X 2 CSRMatrix + (0,0) 2.0 + (0,1) 3.0 + (1,1) 4.0 + """ + spstr = "{0} X {1} ".format(self.numRows, self.numCols) + if self.isTransposed: + spstr += "CSRMatrix\n" + else: + spstr += "CSCMatrix\n" + + cur_col = 0 + smlist = [] + + # Display first 16 values. + if len(self.values) <= 16: + zipindval = zip(self.rowIndices, self.values) + else: + zipindval = zip(self.rowIndices[:16], self.values[:16]) + for i, (rowInd, value) in enumerate(zipindval): + if self.colPtrs[cur_col + 1] <= i: + cur_col += 1 + if self.isTransposed: + smlist.append('({0},{1}) {2}'.format( + cur_col, rowInd, _format_float(value))) + else: + smlist.append('({0},{1}) {2}'.format( + rowInd, cur_col, _format_float(value))) + spstr += "\n".join(smlist) + + if len(self.values) > 16: + spstr += "\n.." * 2 + return spstr + + def __repr__(self): + """ + Representation of a SparseMatrix + + >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + >>> sm1 + SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2.0, 3.0, 4.0], False) + """ + rowIndices = list(self.rowIndices) + colPtrs = list(self.colPtrs) + + if len(self.values) <= 16: + values = _format_float_list(self.values) + + else: + values = ( + _format_float_list(self.values[:8]) + + ["..."] + + _format_float_list(self.values[-8:]) + ) + rowIndices = rowIndices[:8] + ["..."] + rowIndices[-8:] + + if len(self.colPtrs) > 16: + colPtrs = colPtrs[:8] + ["..."] + colPtrs[-8:] + + values = ", ".join(values) + rowIndices = ", ".join([str(ind) for ind in rowIndices]) + colPtrs = ", ".join([str(ptr) for ptr in colPtrs]) + return "SparseMatrix({0}, {1}, [{2}], [{3}], [{4}], {5})".format( + self.numRows, self.numCols, colPtrs, rowIndices, + values, self.isTransposed) + + def __reduce__(self): + return SparseMatrix, ( + self.numRows, self.numCols, self.colPtrs.tostring(), + self.rowIndices.tostring(), self.values.tostring(), + int(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.ravel(self.toArray(), 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): + """ + Create a DenseMatrix + """ + 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 + (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS) + if failure_count: + exit(-1) + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 8e56b0d6fff05..c5679057597e2 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -18,7 +18,6 @@ """ Unit tests for Spark ML Python APIs. """ -import array import sys if sys.version > '3': xrange = range @@ -40,15 +39,21 @@ from shutil import rmtree import tempfile +import array as pyarray import numpy as np +from numpy import ( + array, array_equal, zeros, inf, random, exp, dot, all, mean, abs, arange, tile, ones) +from numpy import sum as array_sum import inspect -from pyspark import keyword_only +from pyspark import keyword_only, SparkContext from pyspark.ml import Estimator, Model, Pipeline, PipelineModel, Transformer from pyspark.ml.classification import * from pyspark.ml.clustering import * from pyspark.ml.evaluation import BinaryClassificationEvaluator, RegressionEvaluator from pyspark.ml.feature import * +from pyspark.ml.linalg import Vector, SparseVector, DenseVector, VectorUDT,\ + DenseMatrix, SparseMatrix, Vectors, Matrices, MatrixUDT, _convert_to_vector from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed from pyspark.ml.recommendation import ALS @@ -57,13 +62,28 @@ from pyspark.ml.tuning import * from pyspark.ml.wrapper import JavaParams from pyspark.mllib.common import _java2py -from pyspark.mllib.linalg import Vectors, DenseVector, SparseVector +from pyspark.mllib.linalg import SparseVector as OldSparseVector, DenseVector as OldDenseVector,\ + DenseMatrix as OldDenseMatrix, MatrixUDT as OldMatrixUDT, SparseMatrix as OldSparseMatrix,\ + Vectors as OldVectors, VectorUDT as OldVectorUDT +from pyspark.mllib.regression import LabeledPoint +from pyspark.serializers import PickleSerializer from pyspark.sql import DataFrame, Row, SparkSession from pyspark.sql.functions import rand from pyspark.sql.utils import IllegalArgumentException from pyspark.storagelevel import * from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase +ser = PickleSerializer() + + +class MLlibTestCase(unittest.TestCase): + def setUp(self): + self.sc = SparkContext('local[4]', "MLlib tests") + self.spark = SparkSession(self.sc) + + def tearDown(self): + self.spark.stop() + class SparkSessionTestCase(PySparkTestCase): @classmethod @@ -142,23 +162,23 @@ def test_float(self): def test_vector(self): ewp = ElementwiseProduct(scalingVec=[1, 3]) - self.assertEqual(ewp.getScalingVec(), DenseVector([1.0, 3.0])) + self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.0, 3.0])) ewp = ElementwiseProduct(scalingVec=np.array([1.2, 3.4])) - self.assertEqual(ewp.getScalingVec(), DenseVector([1.2, 3.4])) + self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.2, 3.4])) self.assertRaises(TypeError, lambda: ElementwiseProduct(scalingVec=["a", "b"])) def test_list(self): l = [0, 1] - for lst_like in [l, np.array(l), DenseVector(l), SparseVector(len(l), range(len(l)), l), - array.array('l', l), xrange(2), tuple(l)]: + for lst_like in [l, np.array(l), OldDenseVector(l), OldSparseVector(len(l), + range(len(l)), l), pyarray.array('l', l), xrange(2), tuple(l)]: converted = TypeConverters.toList(lst_like) self.assertEqual(type(converted), list) self.assertListEqual(converted, l) def test_list_int(self): - for indices in [[1.0, 2.0], np.array([1.0, 2.0]), DenseVector([1.0, 2.0]), - SparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), - array.array('d', [1.0, 2.0])]: + for indices in [[1.0, 2.0], np.array([1.0, 2.0]), OldDenseVector([1.0, 2.0]), + OldSparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), + pyarray.array('d', [1.0, 2.0])]: vs = VectorSlicer(indices=indices) self.assertListEqual(vs.getIndices(), [1, 2]) self.assertTrue(all([type(v) == int for v in vs.getIndices()])) @@ -390,9 +410,9 @@ def test_binarizer(self): def test_idf(self): dataset = self.spark.createDataFrame([ - (DenseVector([1.0, 2.0]),), - (DenseVector([0.0, 1.0]),), - (DenseVector([3.0, 0.2]),)], ["tf"]) + (OldDenseVector([1.0, 2.0]),), + (OldDenseVector([0.0, 1.0]),), + (OldDenseVector([3.0, 0.2]),)], ["tf"]) idf0 = IDF(inputCol="tf") self.assertListEqual(idf0.params, [idf0.inputCol, idf0.minDocFreq, idf0.outputCol]) idf0m = idf0.fit(dataset, {idf0.outputCol: "idf"}) @@ -437,10 +457,10 @@ def test_stopwordsremover(self): def test_count_vectorizer_with_binary(self): dataset = self.spark.createDataFrame([ - (0, "a a a b b c".split(' '), SparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), - (1, "a a".split(' '), SparseVector(3, {0: 1.0}),), - (2, "a b".split(' '), SparseVector(3, {0: 1.0, 1: 1.0}),), - (3, "c".split(' '), SparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) + (0, "a a a b b c".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), + (1, "a a".split(' '), OldSparseVector(3, {0: 1.0}),), + (2, "a b".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0}),), + (3, "c".split(' '), OldSparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) cv = CountVectorizer(binary=True, inputCol="words", outputCol="features") model = cv.fit(dataset) @@ -561,11 +581,11 @@ def test_save_load(self): # Save/load for CrossValidator will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(Vectors.dense([0.0]), 0.0), - (Vectors.dense([0.4]), 1.0), - (Vectors.dense([0.5]), 0.0), - (Vectors.dense([0.6]), 1.0), - (Vectors.dense([1.0]), 1.0)] * 10, + [(OldVectors.dense([0.0]), 0.0), + (OldVectors.dense([0.4]), 1.0), + (OldVectors.dense([0.5]), 0.0), + (OldVectors.dense([0.6]), 1.0), + (OldVectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -634,11 +654,11 @@ def test_save_load(self): # Save/load for TrainValidationSplit will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(Vectors.dense([0.0]), 0.0), - (Vectors.dense([0.4]), 1.0), - (Vectors.dense([0.5]), 0.0), - (Vectors.dense([0.6]), 1.0), - (Vectors.dense([1.0]), 1.0)] * 10, + [(OldVectors.dense([0.0]), 0.0), + (OldVectors.dense([0.4]), 1.0), + (OldVectors.dense([0.5]), 0.0), + (OldVectors.dense([0.6]), 1.0), + (OldVectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -837,8 +857,8 @@ def _compare(self, m1, m2): def test_persistence(self): # Test save/load for LDA, LocalLDAModel, DistributedLDAModel. df = self.spark.createDataFrame([ - [1, Vectors.dense([0.0, 1.0])], - [2, Vectors.sparse(2, {0: 1.0})], + [1, OldVectors.dense([0.0, 1.0])], + [2, OldVectors.sparse(2, {0: 1.0})], ], ["id", "features"]) # Fit model lda = LDA(k=2, seed=1, optimizer="em") @@ -873,9 +893,8 @@ def test_persistence(self): class TrainingSummaryTest(SparkSessionTestCase): def test_linear_regression_summary(self): - from pyspark.mllib.linalg import Vectors - df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), - (0.0, 2.0, Vectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), + (0.0, 2.0, OldVectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight", fitIntercept=False) @@ -947,9 +966,8 @@ def test_glr_summary(self): self.assertAlmostEqual(sameSummary.deviance, s.deviance) def test_logistic_regression_summary(self): - from pyspark.mllib.linalg import Vectors - df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), - (0.0, 2.0, Vectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), + (0.0, 2.0, OldVectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) model = lr.fit(df) @@ -978,9 +996,9 @@ def test_logistic_regression_summary(self): class OneVsRestTests(SparkSessionTestCase): def test_copy(self): - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), + (1.0, OldVectors.sparse(2, [], [])), + (2.0, OldVectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -992,9 +1010,9 @@ def test_copy(self): self.assertEqual(model1.getPredictionCol(), "indexed") def test_output_columns(self): - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), + (1.0, OldVectors.sparse(2, [], [])), + (2.0, OldVectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1004,9 +1022,9 @@ def test_output_columns(self): def test_save_load(self): temp_path = tempfile.mkdtemp() - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), + (1.0, OldVectors.sparse(2, [], [])), + (2.0, OldVectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1034,7 +1052,7 @@ def test_apply_binary_term_freqs(self): hashingTF.setInputCol("words").setOutputCol("features").setNumFeatures(n).setBinary(True) output = hashingTF.transform(df) features = output.select("features").first().features.toArray() - expected = Vectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() + expected = OldVectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() for i in range(0, n): self.assertAlmostEqual(features[i], expected[i], 14, "Error at " + str(i) + ": expected " + str(expected[i]) + ", got " + str(features[i])) @@ -1109,6 +1127,354 @@ def test_java_params(self): self.check_params(cls()) +def _squared_distance(a, b): + if isinstance(a, Vector): + return a.squared_distance(b) + else: + return b.squared_distance(a) + + +class VectorTests(MLlibTestCase): + + 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(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(bytes(self.sc._jvm.SerDe.dumps(jvecs))) + self.assertEqual(vs, nvs) + + def test_serialize(self): + # Because pickle path still uses old vector/matrix + # TODO: Change this to new vector/matrix when pickle for new vector/matrix is ready. + self._test_serialize(OldDenseVector(range(10))) + self._test_serialize(OldDenseVector(array([1., 2., 3., 4.]))) + self._test_serialize(OldDenseVector(pyarray.array('d', range(10)))) + self._test_serialize(OldSparseVector(4, {1: 1, 3: 2})) + self._test_serialize(OldSparseVector(3, {})) + self._test_serialize(OldDenseMatrix(2, 3, range(6))) + sm1 = OldSparseMatrix( + 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) + self._test_serialize(sm1) + + def test_dot(self): + sv = SparseVector(4, {1: 1, 3: 2}) + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([1, 2, 3, 4]) + mat = array([[1., 2., 3., 4.], + [1., 2., 3., 4.], + [1., 2., 3., 4.], + [1., 2., 3., 4.]]) + arr = pyarray.array('d', [0, 1, 2, 3]) + self.assertEqual(10.0, sv.dot(dv)) + self.assertTrue(array_equal(array([3., 6., 9., 12.]), sv.dot(mat))) + self.assertEqual(30.0, dv.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), dv.dot(mat))) + self.assertEqual(30.0, lst.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), lst.dot(mat))) + self.assertEqual(7.0, sv.dot(arr)) + + def test_squared_distance(self): + sv = SparseVector(4, {1: 1, 3: 2}) + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([4, 3, 2, 1]) + lst1 = [4, 3, 2, 1] + arr = pyarray.array('d', [0, 2, 1, 3]) + narr = array([0, 2, 1, 3]) + self.assertEqual(15.0, _squared_distance(sv, dv)) + self.assertEqual(25.0, _squared_distance(sv, lst)) + self.assertEqual(20.0, _squared_distance(dv, lst)) + self.assertEqual(15.0, _squared_distance(dv, sv)) + self.assertEqual(25.0, _squared_distance(lst, sv)) + self.assertEqual(20.0, _squared_distance(lst, dv)) + self.assertEqual(0.0, _squared_distance(sv, sv)) + self.assertEqual(0.0, _squared_distance(dv, dv)) + self.assertEqual(0.0, _squared_distance(lst, lst)) + self.assertEqual(25.0, _squared_distance(sv, lst1)) + self.assertEqual(3.0, _squared_distance(sv, arr)) + self.assertEqual(3.0, _squared_distance(sv, narr)) + + def test_hash(self): + v1 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v2 = SparseVector(4, [(1, 1.0), (3, 5.5)]) + v3 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v4 = SparseVector(4, [(1, 1.0), (3, 2.5)]) + self.assertEqual(hash(v1), hash(v2)) + self.assertEqual(hash(v1), hash(v3)) + self.assertEqual(hash(v2), hash(v3)) + self.assertFalse(hash(v1) == hash(v4)) + self.assertFalse(hash(v2) == hash(v4)) + + def test_eq(self): + v1 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v2 = SparseVector(4, [(1, 1.0), (3, 5.5)]) + v3 = DenseVector([0.0, 1.0, 0.0, 5.5]) + v4 = SparseVector(6, [(1, 1.0), (3, 5.5)]) + v5 = DenseVector([0.0, 1.0, 0.0, 2.5]) + v6 = SparseVector(4, [(1, 1.0), (3, 2.5)]) + self.assertEqual(v1, v2) + self.assertEqual(v1, v3) + self.assertFalse(v2 == v4) + self.assertFalse(v1 == v5) + self.assertFalse(v1 == v6) + + def test_equals(self): + indices = [1, 2, 4] + values = [1., 3., 2.] + self.assertTrue(Vectors._equals(indices, values, list(range(5)), [0., 1., 3., 0., 2.])) + self.assertFalse(Vectors._equals(indices, values, list(range(5)), [0., 3., 1., 0., 2.])) + self.assertFalse(Vectors._equals(indices, values, list(range(5)), [0., 3., 0., 2.])) + self.assertFalse(Vectors._equals(indices, values, list(range(5)), [0., 1., 3., 2., 2.])) + + def test_conversion(self): + # numpy arrays should be automatically upcast to float64 + # tests for fix of [SPARK-5089] + v = array([1, 2, 3, 4], dtype='float64') + dv = DenseVector(v) + self.assertTrue(dv.array.dtype == 'float64') + v = array([1, 2, 3, 4], dtype='float32') + dv = DenseVector(v) + self.assertTrue(dv.array.dtype == 'float64') + + def test_sparse_vector_indexing(self): + sv = SparseVector(5, {1: 1, 3: 2}) + self.assertEqual(sv[0], 0.) + self.assertEqual(sv[3], 2.) + self.assertEqual(sv[1], 1.) + self.assertEqual(sv[2], 0.) + self.assertEqual(sv[4], 0.) + self.assertEqual(sv[-1], 0.) + self.assertEqual(sv[-2], 2.) + self.assertEqual(sv[-3], 0.) + self.assertEqual(sv[-5], 0.) + for ind in [5, -6]: + self.assertRaises(ValueError, sv.__getitem__, ind) + for ind in [7.8, '1']: + self.assertRaises(TypeError, sv.__getitem__, ind) + + zeros = SparseVector(4, {}) + self.assertEqual(zeros[0], 0.0) + self.assertEqual(zeros[3], 0.0) + for ind in [4, -5]: + self.assertRaises(ValueError, zeros.__getitem__, ind) + + empty = SparseVector(0, {}) + for ind in [-1, 0, 1]: + self.assertRaises(ValueError, empty.__getitem__, ind) + + def test_matrix_indexing(self): + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) + expected = [[0, 6], [1, 8], [4, 10]] + for i in range(3): + for j in range(2): + self.assertEqual(mat[i, j], expected[i][j]) + + def test_repr_dense_matrix(self): + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) + self.assertTrue( + repr(mat), + 'DenseMatrix(3, 2, [0.0, 1.0, 4.0, 6.0, 8.0, 10.0], False)') + + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10], True) + self.assertTrue( + repr(mat), + 'DenseMatrix(3, 2, [0.0, 1.0, 4.0, 6.0, 8.0, 10.0], False)') + + mat = DenseMatrix(6, 3, zeros(18)) + self.assertTrue( + repr(mat), + 'DenseMatrix(6, 3, [0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, ..., \ + 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0], False)') + + def test_repr_sparse_matrix(self): + 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.assertTrue( + repr(sm1t), + 'SparseMatrix(3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], True)') + + indices = tile(arange(6), 3) + values = ones(18) + sm = SparseMatrix(6, 3, [0, 6, 12, 18], indices, values) + self.assertTrue( + repr(sm), "SparseMatrix(6, 3, [0, 6, 12, 18], \ + [0, 1, 2, 3, 4, 5, 0, 1, ..., 4, 5, 0, 1, 2, 3, 4, 5], \ + [1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, ..., \ + 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0], False)") + + self.assertTrue( + str(sm), + "6 X 3 CSCMatrix\n\ + (0,0) 1.0\n(1,0) 1.0\n(2,0) 1.0\n(3,0) 1.0\n(4,0) 1.0\n(5,0) 1.0\n\ + (0,1) 1.0\n(1,1) 1.0\n(2,1) 1.0\n(3,1) 1.0\n(4,1) 1.0\n(5,1) 1.0\n\ + (0,2) 1.0\n(1,2) 1.0\n(2,2) 1.0\n(3,2) 1.0\n..\n..") + + sm = SparseMatrix(1, 18, zeros(19), [], []) + self.assertTrue( + repr(sm), + 'SparseMatrix(1, 18, \ + [0, 0, 0, 0, 0, 0, 0, 0, ..., 0, 0, 0, 0, 0, 0, 0, 0], [], [], False)') + + 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.assertEqual(sm1.numRows, 3) + self.assertEqual(sm1.numCols, 4) + self.assertEqual(sm1.colPtrs.tolist(), [0, 2, 2, 4, 4]) + self.assertEqual(sm1.rowIndices.tolist(), [1, 2, 1, 2]) + self.assertEqual(sm1.values.tolist(), [1.0, 2.0, 4.0, 5.0]) + self.assertTrue( + repr(sm1), + 'SparseMatrix(3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0], False)') + + # 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.assertEqual(expected[i][j], sm1[i, j]) + self.assertTrue(array_equal(sm1.toArray(), expected)) + + # Test conversion to dense and sparse. + smnew = sm1.toDense().toSparse() + self.assertEqual(sm1.numRows, smnew.numRows) + self.assertEqual(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.assertEqual(sm1t.numRows, 3) + self.assertEqual(sm1t.numCols, 4) + self.assertEqual(sm1t.colPtrs.tolist(), [0, 2, 3, 5]) + self.assertEqual(sm1t.rowIndices.tolist(), [0, 1, 2, 0, 2]) + self.assertEqual(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.assertEqual(expected[i][j], sm1t[i, j]) + self.assertTrue(array_equal(sm1t.toArray(), expected)) + + def test_dense_matrix_is_transposed(self): + mat1 = DenseMatrix(3, 2, [0, 4, 1, 6, 3, 9], isTransposed=True) + mat = DenseMatrix(3, 2, [0, 1, 3, 4, 6, 9]) + self.assertEqual(mat1, mat) + + expected = [[0, 4], [1, 6], [3, 9]] + for i in range(3): + for j in range(2): + self.assertEqual(mat1[i, j], expected[i][j]) + self.assertTrue(array_equal(mat1.toArray(), expected)) + + sm = mat1.toSparse() + self.assertTrue(array_equal(sm.rowIndices, [1, 2, 0, 1, 2])) + self.assertTrue(array_equal(sm.colPtrs, [0, 2, 5])) + self.assertTrue(array_equal(sm.values, [1, 3, 4, 6, 9])) + + def test_norms(self): + a = DenseVector([0, 2, 3, -1]) + self.assertAlmostEqual(a.norm(2), 3.742, 3) + self.assertTrue(a.norm(1), 6) + self.assertTrue(a.norm(inf), 3) + a = SparseVector(4, [0, 2], [3, -4]) + self.assertAlmostEqual(a.norm(2), 5) + self.assertTrue(a.norm(1), 7) + self.assertTrue(a.norm(inf), 4) + + tmp = SparseVector(4, [0, 2], [3, 0]) + self.assertEqual(tmp.numNonzeros(), 1) + + +class VectorUDTTests(MLlibTestCase): + + dv0 = DenseVector([]) + dv1 = DenseVector([1.0, 2.0]) + sv0 = SparseVector(2, [], []) + sv1 = SparseVector(2, [1], [2.0]) + udt = VectorUDT() + + old_dv0 = OldDenseVector([]) + old_dv1 = OldDenseVector([1.0, 2.0]) + old_sv0 = OldSparseVector(2, [], []) + old_sv1 = OldSparseVector(2, [1], [2.0]) + old_udt = OldVectorUDT() + + def test_json_schema(self): + self.assertEqual(VectorUDT.fromJson(self.udt.jsonValue()), self.udt) + + def test_serialization(self): + for v in [self.dv0, self.dv1, self.sv0, self.sv1]: + self.assertEqual(v, self.udt.deserialize(self.udt.serialize(v))) + + def test_infer_schema(self): + rdd = self.sc.parallelize([LabeledPoint(1.0, self.old_dv1), + LabeledPoint(0.0, self.old_sv1)]) + df = rdd.toDF() + schema = df.schema + field = [f for f in schema.fields if f.name == "features"][0] + self.assertEqual(field.dataType, self.old_udt) + vectors = df.rdd.map(lambda p: p.features).collect() + self.assertEqual(len(vectors), 2) + for v in vectors: + if isinstance(v, OldSparseVector): + self.assertEqual(v, self.old_sv1) + elif isinstance(v, OldDenseVector): + self.assertEqual(v, self.old_dv1) + else: + raise TypeError("expecting a vector but got %r of type %r" % (v, type(v))) + + +class MatrixUDTTests(MLlibTestCase): + + dm1 = DenseMatrix(3, 2, [0, 1, 4, 5, 9, 10]) + dm2 = DenseMatrix(3, 2, [0, 1, 4, 5, 9, 10], isTransposed=True) + sm1 = SparseMatrix(1, 1, [0, 1], [0], [2.0]) + sm2 = SparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) + udt = MatrixUDT() + + old_dm1 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10]) + old_dm2 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10], isTransposed=True) + old_sm1 = OldSparseMatrix(1, 1, [0, 1], [0], [2.0]) + old_sm2 = OldSparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) + old_udt = OldMatrixUDT() + + def test_json_schema(self): + self.assertEqual(MatrixUDT.fromJson(self.udt.jsonValue()), self.udt) + + def test_serialization(self): + for m in [self.dm1, self.dm2, self.sm1, self.sm2]: + self.assertEqual(m, self.udt.deserialize(self.udt.serialize(m))) + + def test_infer_schema(self): + rdd = self.sc.parallelize([("dense", self.old_dm1), ("sparse", self.old_sm1)]) + df = rdd.toDF() + schema = df.schema + self.assertTrue(schema.fields[1].dataType, self.old_udt) + matrices = df.rdd.map(lambda x: x._2).collect() + self.assertEqual(len(matrices), 2) + for m in matrices: + if isinstance(m, OldDenseMatrix): + self.assertTrue(m, self.old_dm1) + elif isinstance(m, OldSparseMatrix): + self.assertTrue(m, self.old_sm1) + else: + raise ValueError("Expected a matrix but got type %r" % type(m)) + + if __name__ == "__main__": from pyspark.ml.tests import * if xmlrunner: From 0d5e29655f9c3758393794367c0b5d3fd395d1f6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 15 May 2016 15:56:46 +0100 Subject: [PATCH 0229/1470] [SPARK-12972][CORE] Update org.apache.httpcomponents.httpclient ## What changes were proposed in this pull request? (Retry of https://github.com/apache/spark/pull/13049) - update to httpclient 4.5 / httpcore 4.4 - remove some defunct exclusions - manage httpmime version to match - update selenium / httpunit to support 4.5 (possible now that Jetty 9 is used) ## How was this patch tested? Jenkins tests. Also, locally running the same test command of one Jenkins profile that failed: `mvn -Phadoop-2.6 -Pyarn -Phive -Phive-thriftserver -Pkinesis-asl ...` Author: Sean Owen Closes #13117 from srowen/SPARK-12972.2. (cherry picked from commit f5576a052da0bb59343bc2a6b6ce06c6abaac75b) Signed-off-by: Sean Owen --- core/pom.xml | 11 +++-- dev/deps/spark-deps-hadoop-2.2 | 4 +- dev/deps/spark-deps-hadoop-2.3 | 4 +- dev/deps/spark-deps-hadoop-2.4 | 4 +- dev/deps/spark-deps-hadoop-2.6 | 4 +- dev/deps/spark-deps-hadoop-2.7 | 4 +- external/docker-integration-tests/pom.xml | 2 - pom.xml | 54 +++++++---------------- sql/hive-thriftserver/pom.xml | 11 +++-- streaming/pom.xml | 5 +++ 10 files changed, 40 insertions(+), 63 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index c9853528b4e88..4b8fb4e5fb0f9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -275,12 +275,11 @@ org.seleniumhq.selenium selenium-java - - - com.google.guava - guava - - + test + + + org.seleniumhq.selenium + selenium-htmlunit-driver test diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 2477312d74bf0..c3be6b2fee99e 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -69,8 +69,8 @@ hadoop-yarn-server-web-proxy-2.2.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 0181a47a79d3c..61ed4c0889b8e 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.3.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index f7ff23472b9b5..fb014921765f9 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -71,8 +71,8 @@ hadoop-yarn-server-web-proxy-2.4.0.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 92db55d674a4d..0baf4e84fff0a 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.0.4.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 44b9b049344ad..8be218cd68d91 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -77,8 +77,8 @@ hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar htrace-core-3.1.0-incubating.jar -httpclient-4.3.2.jar -httpcore-4.3.2.jar +httpclient-4.5.2.jar +httpcore-4.4.4.jar ivy-2.4.0.jar jackson-annotations-2.5.3.jar jackson-core-2.5.3.jar diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 53a24f3e06e08..3169a0ebaeb9f 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -85,13 +85,11 @@ org.apache.httpcomponents httpclient - 4.5 test org.apache.httpcomponents httpcore - 4.4.1 test diff --git a/pom.xml b/pom.xml index 9e9aad223a5e1..40d9bf5ccccb5 100644 --- a/pom.xml +++ b/pom.xml @@ -149,8 +149,8 @@ 0.10.2 - 4.3.2 - 4.3.2 + 4.5.2 + 4.4.4 3.1 3.4.1 @@ -179,6 +179,7 @@ 0.9.2 4.5.2-1 1.1 + 2.52.0 ${java.home} @@ -410,6 +411,11 @@ httpclient ${commons.httpclient.version}
    + + org.apache.httpcomponents + httpmime + ${commons.httpclient.version} + org.apache.httpcomponents httpcore @@ -418,7 +424,7 @@ org.seleniumhq.selenium selenium-java - 2.45.0 + ${selenium.version} test @@ -431,6 +437,12 @@ + + org.seleniumhq.selenium + selenium-htmlunit-driver + ${selenium.version} + test + xml-apis @@ -731,18 +743,6 @@ guava com.google.guava - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - - - commons-logging - httpclient - commons-logging commons-logging @@ -1448,14 +1448,6 @@ ${hive.group} hive-shims - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.apache.curator curator-framework @@ -1807,14 +1799,6 @@ libthrift ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api @@ -1826,14 +1810,6 @@ libfb303 ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a8b7aaff62a15..5cf16e42b9f86 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -70,6 +70,11 @@ selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + org.apache.spark spark-sql_${scala.binary.version} @@ -106,12 +111,6 @@ - - - - org.codehaus.mojo - build-helper-maven-plugin - add-source generate-sources diff --git a/streaming/pom.xml b/streaming/pom.xml index e7415863e3568..2055cffe751cf 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -97,6 +97,11 @@ selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + org.mockito mockito-core From 6d10b28261e8f1c989d4cab9f59f5f082fd267de Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 16 May 2016 16:27:04 +0100 Subject: [PATCH 0230/1470] [SPARK-12972][CORE][TEST-MAVEN][TEST-HADOOP2.2] Update org.apache.httpcomponents.httpclient, commons-io ## What changes were proposed in this pull request? This is sort of a hot-fix for https://github.com/apache/spark/pull/13117, but, the problem is limited to Hadoop 2.2. The change is to manage `commons-io` to 2.4 for all Hadoop builds, which is only a net change for Hadoop 2.2, which was using 2.1. ## How was this patch tested? Jenkins tests -- normal PR builder, then the `[test-hadoop2.2] [test-maven]` if successful. Author: Sean Owen Closes #13132 from srowen/SPARK-12972.3. (cherry picked from commit fabc8e5b128849a08d820d8c0b3425e39258e02e) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 2 +- pom.xml | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index c3be6b2fee99e..91b333fcae4fe 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -30,7 +30,7 @@ commons-configuration-1.6.jar commons-dbcp-1.4.jar commons-digester-1.8.jar commons-httpclient-3.1.jar -commons-io-2.1.jar +commons-io-2.4.jar commons-lang-2.6.jar commons-lang3-3.3.2.jar commons-logging-1.1.3.jar diff --git a/pom.xml b/pom.xml index 40d9bf5ccccb5..864824d86c3c5 100644 --- a/pom.xml +++ b/pom.xml @@ -166,6 +166,7 @@ 1.1.2 1.2.0-incubating 1.10 + 2.4 2.6 @@ -376,6 +377,11 @@ commons-lang ${commons-lang2.version} + + commons-io + commons-io + ${commons-io.version} + commons-codec commons-codec From 1426235bfb1ecfa55859930913ae45d085912bf7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 17 May 2016 09:55:53 +0100 Subject: [PATCH 0231/1470] [SPARK-15290][BUILD] Move annotations, like @Since / @DeveloperApi, into spark-tags ## What changes were proposed in this pull request? (See https://github.com/apache/spark/pull/12416 where most of this was already reviewed and committed; this is just the module structure and move part. This change does not move the annotations into test scope, which was the apparently problem last time.) Rename `spark-test-tags` -> `spark-tags`; move common annotations like `Since` to `spark-tags` ## How was this patch tested? Jenkins tests. Author: Sean Owen Closes #13074 from srowen/SPARK-15290. (cherry picked from commit 122302cbf5cbf1133067a5acdffd6ab96765dafe) Signed-off-by: Sean Owen --- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 6 +++--- .../spark/annotation/AlphaComponent.java | 0 .../apache/spark/annotation/DeveloperApi.java | 0 .../apache/spark/annotation/Experimental.java | 0 .../org/apache/spark/annotation/Private.java | 0 .../org/apache/spark/annotation/Since.scala | 0 .../apache/spark/annotation/package-info.java | 0 .../org/apache/spark/annotation/package.scala | 0 common/unsafe/pom.xml | 8 ++++---- core/pom.xml | 2 +- dev/sparktestsupport/modules.py | 19 +++++++++++++------ external/docker-integration-tests/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 4 ++++ mllib/pom.xml | 2 +- pom.xml | 3 +-- project/MimaExcludes.scala | 8 ++++++++ project/SparkBuild.scala | 10 +++++----- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- yarn/pom.xml | 2 +- 35 files changed, 59 insertions(+), 41 deletions(-) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/AlphaComponent.java (100%) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/DeveloperApi.java (100%) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/Experimental.java (100%) rename {core => common/tags}/src/main/java/org/apache/spark/annotation/Private.java (100%) rename {core => common/tags}/src/main/scala/org/apache/spark/annotation/Since.scala (100%) rename {core => common/tags}/src/main/scala/org/apache/spark/annotation/package-info.java (100%) rename {core => common/tags}/src/main/scala/org/apache/spark/annotation/package.scala (100%) diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bd507c2cb6c4b..5444ae6d70a92 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -66,7 +66,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.mockito diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 810ec10ca05b3..e736436aec4cf 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -80,7 +80,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} log4j diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index bc83ef24c30ec..1fd3af2e6e622 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -48,7 +48,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 8bc1f52798941..bbbb0bd5aa050 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -38,7 +38,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 8e702b4fefe8c..14e94eca93b22 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -27,12 +27,12 @@ org.apache.spark - spark-test-tags_2.11 + spark-tags_2.11 jar - Spark Project Test Tags + Spark Project Tags http://spark.apache.org/ - test-tags + tags diff --git a/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java b/common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/AlphaComponent.java rename to common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java diff --git a/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java b/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/DeveloperApi.java rename to common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java diff --git a/core/src/main/java/org/apache/spark/annotation/Experimental.java b/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Experimental.java rename to common/tags/src/main/java/org/apache/spark/annotation/Experimental.java diff --git a/core/src/main/java/org/apache/spark/annotation/Private.java b/common/tags/src/main/java/org/apache/spark/annotation/Private.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Private.java rename to common/tags/src/main/java/org/apache/spark/annotation/Private.java diff --git a/core/src/main/scala/org/apache/spark/annotation/Since.scala b/common/tags/src/main/scala/org/apache/spark/annotation/Since.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/Since.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/Since.scala diff --git a/core/src/main/scala/org/apache/spark/annotation/package-info.java b/common/tags/src/main/scala/org/apache/spark/annotation/package-info.java similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/package-info.java rename to common/tags/src/main/scala/org/apache/spark/annotation/package-info.java diff --git a/core/src/main/scala/org/apache/spark/annotation/package.scala b/common/tags/src/main/scala/org/apache/spark/annotation/package.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/package.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/package.scala diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 93b9580f26b86..d0d1da69ea802 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -36,6 +36,10 @@ + + org.apache.spark + spark-tags_${scala.binary.version} + com.twitter chill_${scala.binary.version} @@ -59,10 +63,6 @@ - - org.apache.spark - spark-test-tags_${scala.binary.version} - org.mockito mockito-core diff --git a/core/pom.xml b/core/pom.xml index 4b8fb4e5fb0f9..45f8bfcc05aa1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -331,7 +331,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index f81db8e44a1d4..d7eb44d7a5898 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -92,10 +92,17 @@ def __ne__(self, other): def __hash__(self): return hash(self.name) +tags = Module( + name="tags", + dependencies=[], + source_file_regexes=[ + "common/tags/", + ] +) catalyst = Module( name="catalyst", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "sql/catalyst/", ], @@ -165,7 +172,7 @@ def __hash__(self): sketch = Module( name="sketch", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "common/sketch/", ], @@ -177,7 +184,7 @@ def __hash__(self): graphx = Module( name="graphx", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "graphx/", ], @@ -189,7 +196,7 @@ def __hash__(self): streaming = Module( name="streaming", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "streaming", ], @@ -205,7 +212,7 @@ def __hash__(self): # fail other PRs. streaming_kinesis_asl = Module( name="streaming-kinesis-asl", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "external/kinesis-asl/", "external/kinesis-asl-assembly/", @@ -270,7 +277,7 @@ def __hash__(self): mllib_local = Module( name="mllib-local", - dependencies=[], + dependencies=[tags], source_file_regexes=[ "mllib-local", ], diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 3169a0ebaeb9f..21d40863b77f5 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -126,7 +126,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} ${project.version} test diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index e4effe158c826..016af934bb897 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -92,7 +92,7 @@
    org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/flume/pom.xml b/external/flume/pom.xml index d650dd034d636..f51d334de0876 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -68,7 +68,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1ea9196e9dfe3..60e3ff60df065 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -72,7 +72,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index cccfda3c61a4e..4a20b78917efa 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -88,7 +88,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 935155eb5d362..b5f5ff2854cfb 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -77,7 +77,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/graphx/pom.xml b/graphx/pom.xml index 1813f383cdcba..fc6c700dd1ec8 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -72,7 +72,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/launcher/pom.xml b/launcher/pom.xml index ef731948826ef..e7303853e6565 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -65,7 +65,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 68f15dd905028..1c6ab2b62d8f0 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -53,6 +53,10 @@ mockito-core test + + org.apache.spark + spark-tags_${scala.binary.version} + diff --git a/mllib/pom.xml b/mllib/pom.xml index 24d8274e2222f..c324afb2a2d1b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -116,7 +116,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/pom.xml b/pom.xml index 864824d86c3c5..9c13af17e4ed5 100644 --- a/pom.xml +++ b/pom.xml @@ -286,9 +286,8 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} ${project.version} - test com.twitter diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 69161e0d61f44..1a02f660fddf2 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -716,6 +716,14 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") + ) ++ Seq( + // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Private"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.AlphaComponent"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Experimental"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.DeveloperApi") ) case v if v.startsWith("1.6") => Seq( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d83afa03f5bbf..3ad9873f43679 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -50,10 +50,10 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) val allProjects@Seq( - core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, testTags, sketch, _* + core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, _* ) = Seq( "core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe", - "test-tags", "sketch" + "tags", "sketch" ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(yarn, java8Tests, sparkGangliaLgpl, @@ -340,7 +340,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, hiveCompatibility, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, testTags, sketch, mllibLocal + unsafe, tags, sketch, mllibLocal ).contains(x) } @@ -685,9 +685,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags), // Skip actual catalyst, but include the subproject. // Catalyst is not public API and contains quasiquotes which break scaladoc. diff --git a/repl/pom.xml b/repl/pom.xml index 0f396c9b809bd..c12d121c61156 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -87,7 +87,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.xbean diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 1748fa2778d6a..1923199f4b861 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -55,7 +55,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.spark diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 184fa2f6bd315..2ea980bf20dfe 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -73,7 +73,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.parquet diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5cf16e42b9f86..809d36dc69b99 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -84,7 +84,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} net.sf.jpam diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 177b6884fa13b..c8b20f0afc4ea 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -60,7 +60,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} diff --git a/yarn/pom.xml b/yarn/pom.xml index 11df2b3f4f761..e07b93ab95450 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -54,7 +54,7 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} org.apache.hadoop From c0bcecf914a0e0f6669a62a50e6198af38d4aac6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 17 May 2016 17:02:52 +0800 Subject: [PATCH 0232/1470] [SPARK-15351][SQL] RowEncoder should support array as the external type for ArrayType ## What changes were proposed in this pull request? This PR improves `RowEncoder` and `MapObjects`, to support array as the external type for `ArrayType`. The idea is straightforward, we use `Object` as the external input type for `ArrayType`, and determine its type at runtime in `MapObjects`. ## How was this patch tested? new test in `RowEncoderSuite` Author: Wenchen Fan Closes #13138 from cloud-fan/map-object. (cherry picked from commit c36ca651f9177f8e7a3f6a0098cba5a810ee9deb) Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/sql/Row.scala | 4 +- .../sql/catalyst/encoders/RowEncoder.scala | 22 +++++ .../expressions/objects/objects.scala | 99 +++++++++---------- .../sql/catalyst/util/GenericArrayData.scala | 5 + .../catalyst/encoders/RowEncoderSuite.scala | 17 ++++ 5 files changed, 92 insertions(+), 55 deletions(-) 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 726291b96c29d..a257b831dd1cc 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 @@ -151,7 +151,7 @@ trait Row extends Serializable { * BinaryType -> byte array * ArrayType -> scala.collection.Seq (use getList for java.util.List) * MapType -> scala.collection.Map (use getJavaMap for java.util.Map) - * StructType -> org.apache.spark.sql.Row (or Product) + * StructType -> org.apache.spark.sql.Row * }}} */ def apply(i: Int): Any = get(i) @@ -176,7 +176,7 @@ trait Row extends Serializable { * BinaryType -> byte array * ArrayType -> scala.collection.Seq (use getList for java.util.List) * MapType -> scala.collection.Map (use getJavaMap for java.util.Map) - * StructType -> org.apache.spark.sql.Row (or Product) + * StructType -> org.apache.spark.sql.Row * }}} */ def get(i: Int): Any diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index ae842a9f87799..a5f39aaa2314b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -32,6 +32,26 @@ import org.apache.spark.unsafe.types.UTF8String /** * A factory for constructing encoders that convert external row to/from the Spark SQL * internal binary representation. + * + * The following is a mapping between Spark SQL types and its allowed external types: + * {{{ + * BooleanType -> java.lang.Boolean + * ByteType -> java.lang.Byte + * ShortType -> java.lang.Short + * IntegerType -> java.lang.Integer + * FloatType -> java.lang.Float + * DoubleType -> java.lang.Double + * StringType -> String + * DecimalType -> java.math.BigDecimal or scala.math.BigDecimal or Decimal + * + * DateType -> java.sql.Date + * TimestampType -> java.sql.Timestamp + * + * BinaryType -> byte array + * ArrayType -> scala.collection.Seq or Array + * MapType -> scala.collection.Map + * StructType -> org.apache.spark.sql.Row or Product + * }}} */ object RowEncoder { def apply(schema: StructType): ExpressionEncoder[Row] = { @@ -166,6 +186,8 @@ object RowEncoder { // In order to support both Decimal and java/scala BigDecimal in external row, we make this // as java.lang.Object. case _: DecimalType => ObjectType(classOf[java.lang.Object]) + // In order to support both Array and Seq in external row, we make this as java.lang.Object. + case _: ArrayType => ObjectType(classOf[java.lang.Object]) case _ => externalDataTypeFor(dt) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index e8a6c742bfe16..7df6e06805a54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -376,45 +376,6 @@ case class MapObjects private( lambdaFunction: Expression, inputData: Expression) extends Expression with NonSQLExpression { - @tailrec - private def itemAccessorMethod(dataType: DataType): String => String = dataType match { - case NullType => - val nullTypeClassName = NullType.getClass.getName + ".MODULE$" - (i: String) => s".get($i, $nullTypeClassName)" - case IntegerType => (i: String) => s".getInt($i)" - case LongType => (i: String) => s".getLong($i)" - case FloatType => (i: String) => s".getFloat($i)" - case DoubleType => (i: String) => s".getDouble($i)" - case ByteType => (i: String) => s".getByte($i)" - case ShortType => (i: String) => s".getShort($i)" - case BooleanType => (i: String) => s".getBoolean($i)" - case StringType => (i: String) => s".getUTF8String($i)" - case s: StructType => (i: String) => s".getStruct($i, ${s.size})" - case a: ArrayType => (i: String) => s".getArray($i)" - case _: MapType => (i: String) => s".getMap($i)" - case udt: UserDefinedType[_] => itemAccessorMethod(udt.sqlType) - case DecimalType.Fixed(p, s) => (i: String) => s".getDecimal($i, $p, $s)" - case DateType => (i: String) => s".getInt($i)" - } - - private lazy val (lengthFunction, itemAccessor, primitiveElement) = inputData.dataType match { - case ObjectType(cls) if classOf[Seq[_]].isAssignableFrom(cls) => - (".size()", (i: String) => s".apply($i)", false) - case ObjectType(cls) if cls.isArray => - (".length", (i: String) => s"[$i]", false) - case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) => - (".size()", (i: String) => s".get($i)", false) - case ArrayType(t, _) => - val (sqlType, primitiveElement) = t match { - case m: MapType => (m, false) - case s: StructType => (s, false) - case s: StringType => (s, false) - case udt: UserDefinedType[_] => (udt.sqlType, false) - case o => (o, true) - } - (".numElements()", itemAccessorMethod(sqlType), primitiveElement) - } - override def nullable: Boolean = true override def children: Seq[Expression] = lambdaFunction :: inputData :: Nil @@ -425,7 +386,6 @@ case class MapObjects private( override def dataType: DataType = ArrayType(lambdaFunction.dataType) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val javaType = ctx.javaType(dataType) val elementJavaType = ctx.javaType(loopVar.dataType) ctx.addMutableState("boolean", loopVar.isNull, "") ctx.addMutableState(elementJavaType, loopVar.value, "") @@ -448,27 +408,61 @@ case class MapObjects private( s"new $convertedType[$dataLength]" } - val loopNullCheck = if (primitiveElement) { - s"${loopVar.isNull} = ${genInputData.value}.isNullAt($loopIndex);" - } else { - s"${loopVar.isNull} = ${genInputData.isNull} || ${loopVar.value} == null;" + // In RowEncoder, we use `Object` to represent Array or Seq, so we need to determine the type + // of input collection at runtime for this case. + val seq = ctx.freshName("seq") + val array = ctx.freshName("array") + val determineCollectionType = inputData.dataType match { + case ObjectType(cls) if cls == classOf[Object] => + val seqClass = classOf[Seq[_]].getName + s""" + $seqClass $seq = null; + $elementJavaType[] $array = null; + if (${genInputData.value}.getClass().isArray()) { + $array = ($elementJavaType[]) ${genInputData.value}; + } else { + $seq = ($seqClass) ${genInputData.value}; + } + """ + case _ => "" + } + + + val (getLength, getLoopVar) = inputData.dataType match { + case ObjectType(cls) if classOf[Seq[_]].isAssignableFrom(cls) => + s"${genInputData.value}.size()" -> s"${genInputData.value}.apply($loopIndex)" + case ObjectType(cls) if cls.isArray => + s"${genInputData.value}.length" -> s"${genInputData.value}[$loopIndex]" + case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) => + s"${genInputData.value}.size()" -> s"${genInputData.value}.get($loopIndex)" + case ArrayType(et, _) => + s"${genInputData.value}.numElements()" -> ctx.getValue(genInputData.value, et, loopIndex) + case ObjectType(cls) if cls == classOf[Object] => + s"$seq == null ? $array.length : $seq.size()" -> + s"$seq == null ? $array[$loopIndex] : $seq.apply($loopIndex)" + } + + val loopNullCheck = inputData.dataType match { + case _: ArrayType => s"${loopVar.isNull} = ${genInputData.value}.isNullAt($loopIndex);" + // The element of primitive array will never be null. + case ObjectType(cls) if cls.isArray && cls.getComponentType.isPrimitive => + s"${loopVar.isNull} = false" + case _ => s"${loopVar.isNull} = ${loopVar.value} == null;" } val code = s""" ${genInputData.code} + ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - boolean ${ev.isNull} = ${genInputData.value} == null; - $javaType ${ev.value} = ${ctx.defaultValue(dataType)}; - - if (!${ev.isNull}) { + if (!${genInputData.isNull}) { + $determineCollectionType $convertedType[] $convertedArray = null; - int $dataLength = ${genInputData.value}$lengthFunction; + int $dataLength = $getLength; $convertedArray = $arrayConstructor; int $loopIndex = 0; while ($loopIndex < $dataLength) { - ${loopVar.value} = - ($elementJavaType)${genInputData.value}${itemAccessor(loopIndex)}; + ${loopVar.value} = ($elementJavaType) ($getLoopVar); $loopNullCheck ${genFunction.code} @@ -481,11 +475,10 @@ case class MapObjects private( $loopIndex += 1; } - ${ev.isNull} = false; ${ev.value} = new ${classOf[GenericArrayData].getName}($convertedArray); } """ - ev.copy(code = code) + ev.copy(code = code, isNull = genInputData.isNull) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index 2b8cdc1e23ab3..3a665d370830f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -37,6 +37,11 @@ class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(primitiveArray: Array[Byte]) = this(primitiveArray.toSeq) def this(primitiveArray: Array[Boolean]) = this(primitiveArray.toSeq) + def this(seqOrArray: Any) = this(seqOrArray match { + case seq: Seq[Any] => seq + case array: Array[_] => array.toSeq + }) + override def copy(): ArrayData = new GenericArrayData(array.clone()) override def numElements(): Int = array.length diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 4800e2e26eb52..7bb006c173b07 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -185,6 +185,23 @@ class RowEncoderSuite extends SparkFunSuite { assert(encoder.serializer.head.nullable == false) } + test("RowEncoder should support array as the external type for ArrayType") { + val schema = new StructType() + .add("array", ArrayType(IntegerType)) + .add("nestedArray", ArrayType(ArrayType(StringType))) + .add("deepNestedArray", ArrayType(ArrayType(ArrayType(LongType)))) + val encoder = RowEncoder(schema) + val input = Row( + Array(1, 2, null), + Array(Array("abc", null), null), + Array(Seq(Array(0L, null), null), null)) + val row = encoder.toRow(input) + val convertedBack = encoder.fromRow(row) + assert(convertedBack.getSeq(0) == Seq(1, 2, null)) + assert(convertedBack.getSeq(1) == Seq(Seq("abc", null), null)) + assert(convertedBack.getSeq(2) == Seq(Seq(Seq(0L, null), null), null)) + } + private def encodeDecodeTest(schema: StructType): Unit = { test(s"encode/decode: ${schema.simpleString}") { val encoder = RowEncoder(schema) From b031ea7dc29b3e55dfaf8e8466b6d8f33cb81a3e Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Tue, 17 May 2016 15:20:47 +0200 Subject: [PATCH 0233/1470] [SPARK-14434][ML] User guide doc and examples for GaussianMixture in spark.ml ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Add guide doc and examples for GaussianMixture in Spark.ml in Java, Scala and Python. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manual compile and test all examples Author: wm624@hotmail.com Closes #12788 from wangmiao1981/example. (cherry picked from commit 4134ff0c657efcbf0f61eff0423215afd6132837) Signed-off-by: Nick Pentreath --- docs/ml-clustering.md | 82 +++++++++++++++++++ .../ml/JavaGaussianMixtureExample.java | 64 +++++++++++++++ .../python/ml/gaussian_mixture_example.py | 48 +++++++++++ .../examples/ml/GaussianMixtureExample.scala | 58 +++++++++++++ 4 files changed, 252 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java create mode 100644 examples/src/main/python/ml/gaussian_mixture_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index a0955a3855ce7..33e4b7b0d2cc8 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -148,3 +148,85 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. {% include_example python/ml/bisecting_k_means_example.py %} + +## Gaussian Mixture Model (GMM) + +A [Gaussian Mixture Model](http://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) +represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions, +each with its own probability. The `spark.ml` implementation uses the +[expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) +algorithm to induce the maximum-likelihood model given a set of samples. + +`GaussianMixture` is implemented as an `Estimator` and generates a `GaussianMixtureModel` as the base +model. + +### Input Columns + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    featuresColVector"features"Feature vector
    + +### Output Columns + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    predictionColInt"prediction"Predicted cluster center
    probabilityColVector"probability"Probability of each cluster
    + +### Example + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.GaussianMixture) for more details. + +{% include_example scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/GaussianMixture.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.GaussianMixture) for more details. + +{% include_example python/ml/gaussian_mixture_example.py %} +
    +
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java new file mode 100644 index 0000000000000..79b99095815a1 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java @@ -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.examples.ml; + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture; +import org.apache.spark.ml.clustering.GaussianMixtureModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + + +/** + * An example demonstrating Gaussian Mixture Model. + * Run with + *
    + * bin/run-example ml.JavaGaussianMixtureExample
    + * 
    + */ +public class JavaGaussianMixtureExample { + + public static void main(String[] args) { + + // Creates a SparkSession + SparkSession spark = SparkSession + .builder() + .appName("JavaGaussianMixtureExample") + .getOrCreate(); + + // $example on$ + // Loads data + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); + + // Trains a GaussianMixture model + GaussianMixture gmm = new GaussianMixture() + .setK(2); + GaussianMixtureModel model = gmm.fit(dataset); + + // Output the parameters of the mixture model + for (int i = 0; i < model.getK(); i++) { + System.out.printf("weight=%f\nmu=%s\nsigma=\n%s\n", + model.weights()[i], model.gaussians()[i].mean(), model.gaussians()[i].cov()); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/python/ml/gaussian_mixture_example.py b/examples/src/main/python/ml/gaussian_mixture_example.py new file mode 100644 index 0000000000000..2ca13d68f6890 --- /dev/null +++ b/examples/src/main/python/ml/gaussian_mixture_example.py @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.clustering import GaussianMixture +# $example off$ +from pyspark.sql import SparkSession + +""" +A simple example demonstrating Gaussian Mixture Model (GMM). +Run with: + bin/spark-submit examples/src/main/python/ml/gaussian_mixture_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PythonGuassianMixtureExample")\ + .getOrCreate() + + # $example on$ + # loads data + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + gmm = GaussianMixture().setK(2) + model = gmm.fit(dataset) + + print("Gaussians: ") + model.gaussiansDF.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala new file mode 100644 index 0000000000000..c484ee55569b9 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture +import org.apache.spark.sql.SparkSession +// $example off$ + +/** + * An example demonstrating Gaussian Mixture Model (GMM). + * Run with + * {{{ + * bin/run-example ml.GaussianMixtureExample + * }}} + */ +object GaussianMixtureExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession.builder.appName(s"${this.getClass.getSimpleName}").getOrCreate() + + // $example on$ + // Loads data + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + // Trains Gaussian Mixture Model + val gmm = new GaussianMixture() + .setK(2) + val model = gmm.fit(dataset) + + // output parameters of mixture model model + for (i <- 0 until model.getK) { + println("weight=%f\nmu=%s\nsigma=\n%s\n" format + (model.weights(i), model.gaussians(i).mean, model.gaussians(i).cov)) + } + // $example off$ + + spark.stop() + } +} +// scalastyle:on println From 273f3d05294f8fcd8f3f4e116afcd96bd4b50920 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 17 May 2016 16:40:38 +0100 Subject: [PATCH 0234/1470] [SPARK-15333][DOCS] Reorganize building-spark.md; rationalize vs wiki ## What changes were proposed in this pull request? See JIRA for the motivation. The changes are almost entirely movement of text and edits to sections. Minor changes to text include: - Copying in / merging text from the "Useful Developer Tools" wiki, in areas of - Docker - R - Running one test - standardizing on ./build/mvn not mvn, and likewise for ./build/sbt - correcting some typos - standardizing code block formatting No text has been removed from this doc; text has been imported from the https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools wiki ## How was this patch tested? Jekyll doc build and inspection of resulting HTML in browser. Author: Sean Owen Closes #13124 from srowen/SPARK-15333. (cherry picked from commit 932d8002931d352dd2ec87184e6c84ec5fa859cd) Signed-off-by: Sean Owen --- docs/building-spark.md | 295 ++++++++++++++++++++++------------------- 1 file changed, 156 insertions(+), 139 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 63532c742e39a..2c987cf8346ef 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -7,48 +7,18 @@ redirect_from: "building-with-maven.html" * This will become a table of contents (this text will be scraped). {:toc} -Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+. -The Spark build can supply a suitable Maven binary; see below. - -# Building with `build/mvn` - -Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: - -{% highlight bash %} -build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package -{% endhighlight %} - -Other build examples can be found below. - -**Note:** When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: - - -Xmax-classfile-name - 128 - -and in `project/SparkBuild.scala` add: - - scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), - -to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. - -# Building a Runnable Distribution +# Building Apache Spark -To create a Spark distribution like those distributed by the -[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as -to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured -with Maven profile settings and so on like the direct Maven build. Example: +## Apache Maven - ./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn - -For more information on usage, run `./dev/make-distribution.sh --help` +The Maven-based build is the build of reference for Apache Spark. +Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+. -# Setting up Maven's Memory Usage +### Setting up Maven's Memory Usage You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: -{% highlight bash %} -export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" -{% endhighlight %} + export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" If you don't run this, you may see errors like the following: @@ -65,7 +35,26 @@ You can fix this by setting the `MAVEN_OPTS` variable as discussed before. * For Java 8 and above this step is not required. * If using `build/mvn` with no `MAVEN_OPTS` set, the script will automate this for you. -# Specifying the Hadoop Version +### build/mvn + +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: + + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package + +Other build examples can be found below. + +## Building a Runnable Distribution + +To create a Spark distribution like those distributed by the +[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as +to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured +with Maven profile settings and so on like the direct Maven build. Example: + + ./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn + +For more information on usage, run `./dev/make-distribution.sh --help` + +## Specifying the Hadoop Version Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the `hadoop.version` property. If unset, Spark will build against Hadoop 2.2.0 by default. Note that certain build profiles are required for particular Hadoop versions: @@ -87,87 +76,63 @@ You can enable the `yarn` profile and optionally set the `yarn.version` property Examples: -{% highlight bash %} + # Apache Hadoop 2.2.X + ./build/mvn -Pyarn -Phadoop-2.2 -DskipTests clean package -# Apache Hadoop 2.2.X -mvn -Pyarn -Phadoop-2.2 -DskipTests clean package + # Apache Hadoop 2.3.X + ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package -# Apache Hadoop 2.3.X -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package + # Apache Hadoop 2.4.X or 2.5.X + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package -# Apache Hadoop 2.4.X or 2.5.X -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + # Apache Hadoop 2.6.X + ./build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package -# Apache Hadoop 2.6.X -mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package + # Apache Hadoop 2.7.X and later + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package -# Apache Hadoop 2.7.X and later -mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package + # Different versions of HDFS and YARN. + ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package -# Different versions of HDFS and YARN. -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package -{% endhighlight %} +## Building With Hive and JDBC Support -# Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. By default Spark will build with Hive 1.2.1 bindings. -{% highlight bash %} -# Apache Hadoop 2.4.X with Hive 1.2.1 support -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package -{% endhighlight %} - -# Building for Scala 2.10 -To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: - - ./dev/change-scala-version.sh 2.10 - mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package - -# PySpark Tests with Maven -If you are building PySpark and wish to run the PySpark tests you will need to build Spark with hive support. + # Apache Hadoop 2.4.X with Hive 1.2.1 support + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package -{% highlight bash %} -build/mvn -DskipTests clean package -Phive -./python/run-tests -{% endhighlight %} +## Packaging without Hadoop Dependencies for YARN -The run-tests script also can be limited to a specific Python version or a specific module - - ./python/run-tests --python-executables=python --modules=pyspark-sql - -**Note:** You can also run Python tests with an sbt build, provided you build Spark with hive support. - -# Spark Tests in Maven - -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). +The assembly directory produced by `mvn package` will, by default, include all of Spark's +dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this +causes multiple versions of these to appear on executor classpaths: the version packaged in +the Spark assembly and the version on each node, included with `yarn.application.classpath`. +The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, +like ZooKeeper and Hadoop itself. -Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: - - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package - mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test - -The ScalaTest plugin also supports running only a specific test suite as follows: +## Building for Scala 2.10 +To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: - mvn -Dhadoop.version=... -DwildcardSuites=org.apache.spark.repl.ReplSuite test + ./dev/change-scala-version.sh 2.10 + ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package -# Building submodules individually +## Building submodules individually It's possible to build Spark sub-modules using the `mvn -pl` option. For instance, you can build the Spark Streaming module using: -{% highlight bash %} -mvn -pl :spark-streaming_2.11 clean install -{% endhighlight %} + ./build/mvn -pl :spark-streaming_2.11 clean install where `spark-streaming_2.11` is the `artifactId` as defined in `streaming/pom.xml` file. -# Continuous Compilation +## Continuous Compilation We use the scala-maven-plugin which supports incremental and continuous compilation. E.g. - mvn scala:cc + ./build/mvn scala:cc should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: @@ -182,86 +147,138 @@ the `spark-parent` module). Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: - $ mvn install + $ ./build/mvn install $ cd core - $ mvn scala:cc + $ ../build/mvn scala:cc -# Building Spark with IntelliJ IDEA or Eclipse +## Speeding up Compilation with Zinc -For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the -[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-IDESetup). +[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental +compiler. When run locally as a background process, it speeds up builds of Scala-based projects +like Spark. Developers who regularly recompile Spark with Maven will be the most interested in +Zinc. The project site gives instructions for building and running `zinc`; OS X users can +install it using `brew install zinc`. -# Running Java 8 Test Suites +If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all +builds. This process will auto-start after the first time `build/mvn` is called and bind to port +3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be +shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically +restart whenever `build/mvn` is called. -Running only Java 8 tests and nothing else. +## Building with SBT - mvn install -DskipTests - mvn -pl :java8-tests_2.11 test +Maven is the official build tool recommended for packaging Spark, and is the *build of reference*. +But SBT is supported for day-to-day development since it can provide much faster iterative +compilation. More advanced developers may wish to use SBT. -or +The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables +can be set to control the SBT build. For example: - sbt java8-tests/test + ./build/sbt -Pyarn -Phadoop-2.3 package -Java 8 tests are automatically enabled when a Java 8 JDK is detected. -If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +To avoid the overhead of launching sbt each time you need to re-compile, you can launch sbt +in interactive mode by running `build/sbt`, and then run all build commands at the command +prompt. For more recommendations on reducing build time, refer to the +[wiki page](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-ReducingBuildTimes). -# Running Docker based Integration Test Suites +## Encrypted Filesystems -Running only docker based integration tests and nothing else. +When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: - mvn install -DskipTests - mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + -Xmax-classfile-name + 128 -or +and in `project/SparkBuild.scala` add: - sbt docker-integration-tests/test + scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), +to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. -# Packaging without Hadoop Dependencies for YARN +## IntelliJ IDEA or Eclipse -The assembly directory produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with `yarn.application.classpath`. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. +For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the +[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-IDESetup). -# Building with SBT -Maven is the official build tool recommended for packaging Spark, and is the *build of reference*. -But SBT is supported for day-to-day development since it can provide much faster iterative -compilation. More advanced developers may wish to use SBT. +# Running Tests -The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables -can be set to control the SBT build. For example: +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). - build/sbt -Pyarn -Phadoop-2.3 package +Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: -To avoid the overhead of launching sbt each time you need to re-compile, you can launch sbt -in interactive mode by running `build/sbt`, and then run all build commands at the command -prompt. For more recommendations on reducing build time, refer to the -[wiki page](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-ReducingBuildTimes). + ./build/mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package + ./build/mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + +The ScalaTest plugin also supports running only a specific Scala test suite as follows: + + ./build/mvn -P... -Dtest=none -DwildcardSuites=org.apache.spark.repl.ReplSuite test + ./build/mvn -P... -Dtest=none -DwildcardSuites=org.apache.spark.repl.* test + +or a Java test: -# Testing with SBT + ./build/mvn test -P... -DwildcardSuites=none -Dtest=org.apache.spark.streaming.JavaAPISuite + +## Testing with SBT Some of the tests require Spark to be packaged first, so always run `build/sbt package` the first time. The following is an example of a correct (build, test) sequence: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver package - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver package + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.*" To run test suites of a specific sub project as follows: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test + ./build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test -# Speeding up Compilation with Zinc +## Running Java 8 Test Suites -[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental -compiler. When run locally as a background process, it speeds up builds of Scala-based projects -like Spark. Developers who regularly recompile Spark with Maven will be the most interested in -Zinc. The project site gives instructions for building and running `zinc`; OS X users can -install it using `brew install zinc`. +Running only Java 8 tests and nothing else. -If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all -builds. This process will auto-start after the first time `build/mvn` is called and bind to port -3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be -shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically -restart whenever `build/mvn` is called. + ./build/mvn install -DskipTests + ./build/mvn -pl :java8-tests_2.11 test + +or + + ./build/sbt java8-tests/test + +Java 8 tests are automatically enabled when a Java 8 JDK is detected. +If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. + +## PySpark Tests with Maven + +If you are building PySpark and wish to run the PySpark tests you will need to build Spark with Hive support. + + ./build/mvn -DskipTests clean package -Phive + ./python/run-tests + +The run-tests script also can be limited to a specific Python version or a specific module + + ./python/run-tests --python-executables=python --modules=pyspark-sql + +**Note:** You can also run Python tests with an sbt build, provided you build Spark with Hive support. + +## Running R Tests + +To run the SparkR tests you will need to install the R package `testthat` +(run `install.packages(testthat)` from R shell). You can run just the SparkR tests using +the command: + + ./R/run-tests.sh + +## Running Docker-based Integration Test Suites + +In order to run Docker integration tests, you have to install the `docker` engine on your box. +The instructions for installation can be found at [the Docker site](https://docs.docker.com/engine/installation/). +Once installed, the `docker` service needs to be started, if not already running. +On Linux, this can be done by `sudo service docker start`. + + ./build/mvn install -DskipTests + ./build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + +or + + ./build/sbt docker-integration-tests/test From 670f482225e20d512c2c1c1fccee5b9a7d3745b0 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Tue, 17 May 2016 16:51:01 +0100 Subject: [PATCH 0235/1470] [SPARK-15318][ML][EXAMPLE] spark.ml Collaborative Filtering example does not work in spark-shell ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) copy & paste example in ml-collaborative-filtering.html into spark-shell, we see the following errors. scala> case class Rating(userId: Int, movieId: Int, rating: Float, timestamp: Long) defined class Rating scala> object Rating { def parseRating(str: String): Rating = { | val fields = str.split("::") | assert(fields.size == 4) | Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) | } } :29: error: Rating.type does not take parameters Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) ^ In standard scala repl, it has the same error. Scala/spark-shell repl has some quirks (e.g. packages are also not well supported). The reason of errors is that scala/spark-shell repl discards previous definitions when we define the Object with the same class name. Solution: We can rename the Object Rating. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually test it: 1). ./bin/run-example ALSExample 2). copy & paste example in the generated document. It works fine. Author: wm624@hotmail.com Closes #13110 from wangmiao1981/repl. (cherry picked from commit bebe5f9811f968db92c2d33e2b30c35cfb808a4a) Signed-off-by: Sean Owen --- .../apache/spark/examples/ml/ALSExample.scala | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index 6b151a622e267..da19ea9f10ec6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -24,16 +24,21 @@ import org.apache.spark.ml.recommendation.ALS // $example off$ import org.apache.spark.sql.SparkSession +/** + * An example demonstrating ALS. + * Run with + * {{{ + * bin/run-example ml.ALSExample + * }}} + */ object ALSExample { // $example on$ case class Rating(userId: Int, movieId: Int, rating: Float, timestamp: Long) - object Rating { - def parseRating(str: String): Rating = { - val fields = str.split("::") - assert(fields.size == 4) - Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) - } + def parseRating(str: String): Rating = { + val fields = str.split("::") + assert(fields.size == 4) + Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) } // $example off$ @@ -46,7 +51,7 @@ object ALSExample { // $example on$ val ratings = spark.read.text("data/mllib/als/sample_movielens_ratings.txt") - .map(Rating.parseRating) + .map(parseRating) .toDF() val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2)) From 110876b9afe5e4205062fd8e8979e096e585737d Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 17 May 2016 10:07:01 -0700 Subject: [PATCH 0236/1470] [SPARK-15165] [SQL] Codegen can break because toCommentSafeString is not actually safe ## What changes were proposed in this pull request? toCommentSafeString method replaces "\u" with "\\\\u" to avoid codegen breaking. But if the even number of "\" is put before "u", like "\\\\u", in the string literal in the query, codegen can break. Following code causes compilation error. ``` val df = Seq(...).toDF df.select("'\\\\\\\\u002A/'").show ``` The reason of the compilation error is because "\\\\\\\\\\\\\\\\u002A/" is translated into "*/" (the end of comment). Due to this unsafety, arbitrary code can be injected like as follows. ``` val df = Seq(...).toDF // Inject "System.exit(1)" df.select("'\\\\\\\\u002A/{System.exit(1);}/*'").show ``` ## How was this patch tested? Added new test cases. Author: Kousuke Saruta Author: sarutak Closes #12939 from sarutak/SPARK-15165. (cherry picked from commit c0c3ec35476c756e569a1f34c4b258eb0490585c) Signed-off-by: Davies Liu --- .../spark/sql/catalyst/util/package.scala | 13 +- .../expressions/CodeGenerationSuite.scala | 44 +++ .../org/apache/spark/sql/SQLQuerySuite.scala | 264 ++++++++++++++++++ 3 files changed, 320 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 3d2a624ba3b30..f1d6cab9a5a1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -162,7 +162,18 @@ package object util { def toCommentSafeString(str: String): String = { val len = math.min(str.length, 128) val suffix = if (str.length > len) "..." else "" - str.substring(0, len).replace("*/", "\\*\\/").replace("\\u", "\\\\u") + suffix + + // Unicode literals, like \u0022, should be escaped before + // they are put in code comment to avoid codegen breaking. + // To escape them, single "\" should be prepended to a series of "\" just before "u" + // only when the number of "\" is odd. + // For example, \u0022 should become to \\u0022 + // but \\u0022 should not become to \\\u0022 because the first backslash escapes the second one, + // and \u0022 will remain, means not escaped. + // Otherwise, the runtime Java compiler will fail to compile or code injection can be allowed. + // For details, see SPARK-15165. + str.substring(0, len).replace("*/", "*\\/") + .replaceAll("(^|[^\\\\])(\\\\(\\\\\\\\)*u)", "$1\\\\$2") + suffix } /* FIX ME diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 2082cea0f60f7..db34d12e286fa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -194,4 +194,48 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { true, InternalRow(UTF8String.fromString("\\u"))) } + + test("check compilation error doesn't occur caused by specific literal") { + // The end of comment (*/) should be escaped. + GenerateUnsafeProjection.generate( + Literal.create("*/Compilation error occurs/*", StringType) :: Nil) + + // `\u002A` is `*` and `\u002F` is `/` + // so if the end of comment consists of those characters in queries, we need to escape them. + GenerateUnsafeProjection.generate( + Literal.create("\\u002A/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u002A/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\u002a/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u002a/Compilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\u002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\\\u002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\002fCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("*\\\\002fCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\002A\\002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\002A\\002FCompilation error occurs/*", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\002A\\\\002FCompilation error occurs/*", StringType) :: Nil) + + // \ u002X is an invalid unicode literal so it should be escaped. + GenerateUnsafeProjection.generate( + Literal.create("\\u002X/Compilation error occurs", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u002X/Compilation error occurs", StringType) :: Nil) + + // \ u001 is an invalid unicode literal so it should be escaped. + GenerateUnsafeProjection.generate( + Literal.create("\\u001/Compilation error occurs", StringType) :: Nil) + GenerateUnsafeProjection.generate( + Literal.create("\\\\u001/Compilation error occurs", StringType) :: Nil) + + } } 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 7020841d31717..b67e2bdeb3663 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 @@ -2496,4 +2496,268 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("check code injection is prevented") { + // The end of comment (*/) should be escaped. + var literal = + """|*/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + var expected = + """|*/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + // `\u002A` is `*` and `\u002F` is `/` + // so if the end of comment consists of those characters in queries, we need to escape them. + literal = + """|\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A/"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002a/"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"*\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|*\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"*\\u002f"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|*\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\\\u002A\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002A\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A\\\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + } } From adc1c2685ea0cfbf23716a4199b85c65021d15c6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 17 May 2016 10:12:51 -0700 Subject: [PATCH 0237/1470] [SPARK-14346][SQL][FOLLOW-UP] add tests for CREAT TABLE USING with partition and bucket ## What changes were proposed in this pull request? https://github.com/apache/spark/pull/12781 introduced PARTITIONED BY, CLUSTERED BY, and SORTED BY keywords to CREATE TABLE USING. This PR adds tests to make sure those keywords are handled correctly. This PR also fixes a mistake that we should create non-hive-compatible table if partition or bucket info exists. ## How was this patch tested? N/A Author: Wenchen Fan Closes #13144 from cloud-fan/add-test. (cherry picked from commit 20a89478e168cb6901ef89f4cb6aa79193ed244a) Signed-off-by: Yin Huai --- .../command/createDataSourceTables.scala | 11 +++- .../execution/command/DDLCommandSuite.scala | 53 +++++++++++++++++++ .../sql/execution/command/DDLSuite.scala | 44 +++++++++++++++ 3 files changed, 106 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 7d3c52570f346..70e5108d938b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -399,8 +399,8 @@ object CreateDataSourceTableUtils extends Logging { "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive." (None, message) - case (Some(serde), relation: HadoopFsRelation) - if relation.location.paths.length == 1 && relation.partitionSchema.isEmpty => + case (Some(serde), relation: HadoopFsRelation) if relation.location.paths.length == 1 && + relation.partitionSchema.isEmpty && relation.bucketSpec.isEmpty => val hiveTable = newHiveCompatibleMetastoreTable(relation, serde) val message = s"Persisting data source relation $qualifiedTableName with a single input path " + @@ -415,6 +415,13 @@ object CreateDataSourceTableUtils extends Logging { "Input path(s): " + relation.location.paths.mkString("\n", "\n", "") (None, message) + case (Some(serde), relation: HadoopFsRelation) if relation.bucketSpec.nonEmpty => + val message = + s"Persisting bucketed data source relation $qualifiedTableName into " + + "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " + + "Input path(s): " + relation.location.paths.mkString("\n", "\n", "") + (None, message) + case (Some(serde), relation: HadoopFsRelation) => val message = s"Persisting data source relation $qualifiedTableName with multiple input paths into " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 13df4493e24d7..897170ea57cf1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -24,7 +24,9 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser +import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsing} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { @@ -238,6 +240,57 @@ class DDLCommandSuite extends PlanTest { } } + test("create table using - with partitioned by") { + val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (a)" + val expected = CreateTableUsing( + TableIdentifier("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + "parquet", + false, + Map.empty, + null, + None, + false, + true) + + parser.parsePlan(query) match { + case ct: CreateTableUsing => + // We can't compare array in `CreateTableUsing` directly, so here we compare + // `partitionColumns` ahead, and make `partitionColumns` null before plan comparison. + assert(Seq("a") == ct.partitionColumns.toSeq) + comparePlans(ct.copy(partitionColumns = null), expected) + case other => + fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table using - with bucket") { + val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" + val expected = CreateTableUsing( + TableIdentifier("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + "parquet", + false, + Map.empty, + null, + Some(BucketSpec(5, Seq("a"), Seq("b"))), + false, + true) + + parser.parsePlan(query) match { + case ct: CreateTableUsing => + // `Array.empty == Array.empty` returns false, here we set `partitionColumns` to null before + // plan comparison. + assert(ct.partitionColumns.isEmpty) + comparePlans(ct.copy(partitionColumns = null), expected) + case other => + fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + // ALTER TABLE table_name RENAME TO new_table_name; // ALTER VIEW view_name RENAME TO new_view_name; test("alter table/view: rename table/view") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 82123bec882ee..d72dc092e277c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -29,8 +29,10 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFor import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StructType} class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { private val escapedIdentifier = "`(.+)`".r @@ -350,6 +352,48 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(catalog.getTableMetadata(tableIdent1) === expectedTable) } + test("create table using") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.schema == Seq(CatalogColumn("a", "int"), CatalogColumn("b", "int"))) + assert(table.properties("spark.sql.sources.provider") == "parquet") + } + } + + test("create table using - with partitioned by") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet PARTITIONED BY (a)") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.schema.isEmpty) // partitioned datasource table is not hive-compatible + assert(table.properties("spark.sql.sources.provider") == "parquet") + assert(DDLUtils.getSchemaFromTableProperties(table) == + Some(new StructType().add("a", IntegerType).add("b", IntegerType))) + assert(DDLUtils.getPartitionColumnsFromTableProperties(table) == + Seq("a")) + } + } + + test("create table using - with bucket") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.schema.isEmpty) // partitioned datasource table is not hive-compatible + assert(table.properties("spark.sql.sources.provider") == "parquet") + assert(DDLUtils.getSchemaFromTableProperties(table) == + Some(new StructType().add("a", IntegerType).add("b", IntegerType))) + assert(DDLUtils.getBucketSpecFromTableProperties(table) == + Some(BucketSpec(5, Seq("a"), Seq("b")))) + } + } + test("alter table: rename") { val catalog = spark.sessionState.catalog val tableIdent1 = TableIdentifier("tab1", Some("dbx")) From af37bdd3a7cee5206f98b3a2ba9113e71b53a2f4 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 17 May 2016 11:18:51 -0700 Subject: [PATCH 0238/1470] [SPARK-10216][SQL] Avoid creating empty files during overwriting with group by query ## What changes were proposed in this pull request? Currently, `INSERT INTO` with `GROUP BY` query tries to make at least 200 files (default value of `spark.sql.shuffle.partition`), which results in lots of empty files. This PR makes it avoid creating empty files during overwriting into Hive table and in internal data sources with group by query. This checks whether the given partition has data in it or not and creates/writes file only when it actually has data. ## How was this patch tested? Unittests in `InsertIntoHiveTableSuite` and `HadoopFsRelationTest`. Closes #8411 Author: hyukjinkwon Author: Keuntae Park Closes #12855 from HyukjinKwon/pr/8411. (cherry picked from commit 8d05a7a98bdbd3ce7c81d273e05a375877ebe68f) Signed-off-by: Michael Armbrust --- .../datasources/WriterContainer.scala | 221 +++++++++--------- .../spark/sql/hive/hiveWriterContainers.scala | 24 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 41 +++- .../sql/sources/HadoopFsRelationTest.scala | 22 +- 4 files changed, 182 insertions(+), 126 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 3b064a5bc489f..7e12bbb2128bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -239,48 +239,50 @@ private[sql] class DefaultWriterContainer( extends BaseWriterContainer(relation, job, isAppend) { def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - executorSideSetup(taskContext) - val configuration = taskAttemptContext.getConfiguration - configuration.set("spark.sql.sources.output.path", outputPath) - var writer = newOutputWriter(getWorkPath) - writer.initConverter(dataSchema) - - // If anything below fails, we should abort the task. - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - while (iterator.hasNext) { - val internalRow = iterator.next() - writer.writeInternal(internalRow) - } - commitTask() - }(catchBlock = abortTask()) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } + if (iterator.hasNext) { + executorSideSetup(taskContext) + val configuration = taskAttemptContext.getConfiguration + configuration.set("spark.sql.sources.output.path", outputPath) + var writer = newOutputWriter(getWorkPath) + writer.initConverter(dataSchema) - def commitTask(): Unit = { + // If anything below fails, we should abort the task. try { - if (writer != null) { - writer.close() - writer = null - } - super.commitTask() + Utils.tryWithSafeFinallyAndFailureCallbacks { + while (iterator.hasNext) { + val internalRow = iterator.next() + writer.writeInternal(internalRow) + } + commitTask() + }(catchBlock = abortTask()) } catch { - case cause: Throwable => - // This exception will be handled in `InsertIntoHadoopFsRelation.insert$writeRows`, and - // will cause `abortTask()` to be invoked. - throw new RuntimeException("Failed to commit task", cause) + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) } - } - def abortTask(): Unit = { - try { - if (writer != null) { - writer.close() + def commitTask(): Unit = { + try { + if (writer != null) { + writer.close() + writer = null + } + super.commitTask() + } catch { + case cause: Throwable => + // This exception will be handled in `InsertIntoHadoopFsRelation.insert$writeRows`, and + // will cause `abortTask()` to be invoked. + throw new RuntimeException("Failed to commit task", cause) + } + } + + def abortTask(): Unit = { + try { + if (writer != null) { + writer.close() + } + } finally { + super.abortTask() } - } finally { - super.abortTask() } } } @@ -363,84 +365,87 @@ private[sql] class DynamicPartitionWriterContainer( } def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - executorSideSetup(taskContext) - - // We should first sort by partition columns, then bucket id, and finally sorting columns. - val sortingExpressions: Seq[Expression] = partitionColumns ++ bucketIdExpression ++ sortColumns - val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) - - val sortingKeySchema = StructType(sortingExpressions.map { - case a: Attribute => StructField(a.name, a.dataType, a.nullable) - // The sorting expressions are all `Attribute` except bucket id. - case _ => StructField("bucketId", IntegerType, nullable = false) - }) - - // Returns the data columns to be written given an input row - val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) - - // Returns the partition path given a partition key. - val getPartitionString = - UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) - - // Sorts the data before write, so that we only need one writer at the same time. - // TODO: inject a local sort operator in planning. - val sorter = new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) - - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) - } - logInfo(s"Sorting complete. Writing out partition files one at a time.") - - val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { - identity - } else { - UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { - case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + if (iterator.hasNext) { + executorSideSetup(taskContext) + + // We should first sort by partition columns, then bucket id, and finally sorting columns. + val sortingExpressions: Seq[Expression] = + partitionColumns ++ bucketIdExpression ++ sortColumns + val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) + + val sortingKeySchema = StructType(sortingExpressions.map { + case a: Attribute => StructField(a.name, a.dataType, a.nullable) + // The sorting expressions are all `Attribute` except bucket id. + case _ => StructField("bucketId", IntegerType, nullable = false) }) - } - val sortedIterator = sorter.sortedIterator() + // Returns the data columns to be written given an input row + val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) + + // Returns the partition path given a partition key. + val getPartitionString = + UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) + + // Sorts the data before write, so that we only need one writer at the same time. + // TODO: inject a local sort operator in planning. + val sorter = new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(dataColumns), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + logInfo(s"Sorting complete. Writing out partition files one at a time.") + + val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { + identity + } else { + UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { + case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + }) + } - // If anything below fails, we should abort the task. - var currentWriter: OutputWriter = null - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - var currentKey: UnsafeRow = null - while (sortedIterator.next()) { - val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] - if (currentKey != nextKey) { - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - currentKey = nextKey.copy() - logDebug(s"Writing partition: $currentKey") + val sortedIterator = sorter.sortedIterator() - currentWriter = newOutputWriter(currentKey, getPartitionString) + // If anything below fails, we should abort the task. + var currentWriter: OutputWriter = null + try { + Utils.tryWithSafeFinallyAndFailureCallbacks { + var currentKey: UnsafeRow = null + while (sortedIterator.next()) { + val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + currentKey = nextKey.copy() + logDebug(s"Writing partition: $currentKey") + + currentWriter = newOutputWriter(currentKey, getPartitionString) + } + currentWriter.writeInternal(sortedIterator.getValue) + } + if (currentWriter != null) { + currentWriter.close() + currentWriter = null } - currentWriter.writeInternal(sortedIterator.getValue) - } - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - commitTask() - }(catchBlock = { - if (currentWriter != null) { - currentWriter.close() - } - abortTask() - }) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) + commitTask() + }(catchBlock = { + if (currentWriter != null) { + currentWriter.close() + } + abortTask() + }) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 794fe264ead5d..706fdbc2604fe 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -178,19 +178,21 @@ private[hive] class SparkHiveWriterContainer( // this function is executed on executor side def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() - executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - - iterator.foreach { row => - var i = 0 - while (i < fieldOIs.length) { - outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) - i += 1 + if (iterator.hasNext) { + val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() + executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) + + iterator.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) + i += 1 + } + writer.write(serializer.serialize(outputData, standardOI)) } - writer.write(serializer.serialize(outputData, standardOI)) - } - close() + close() + } } } 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 82d3e49f929d0..883cdac110e0b 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 @@ -19,13 +19,13 @@ package org.apache.spark.sql.hive import java.io.File -import org.apache.hadoop.hive.conf.HiveConf import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException -import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -118,10 +118,10 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef 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) + |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 @@ -216,6 +216,35 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef sql("DROP TABLE hiveTableWithStructValue") } + test("SPARK-10216: Avoid empty files during overwrite into Hive table with group by query") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + val testDataset = hiveContext.sparkContext.parallelize( + (1 to 2).map(i => TestData(i, i.toString))).toDF() + testDataset.createOrReplaceTempView("testDataset") + + val tmpDir = Utils.createTempDir() + sql( + s""" + |CREATE TABLE table1(key int,value string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table1 + |SELECT count(key), value FROM testDataset GROUP BY value + """.stripMargin) + + val overwrittenFiles = tmpDir.listFiles() + .filter(f => f.isFile && !f.getName.endsWith(".crc")) + .sortBy(_.getName) + val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0) + + assert(overwrittenFiles === overwrittenFilesWithoutEmpty) + + sql("DROP TABLE table1") + } + } + test("Reject partitioning that does not match table") { withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index f4d63334b6573..78d2dc28d6b5e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.execution.DataSourceScanExec -import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{FileScanRDD, LocalityTestFileSystem} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -879,6 +879,26 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } } } + + test("SPARK-10216: Avoid empty files during overwriting with group by query") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + withTempPath { path => + val df = spark.range(0, 5) + val groupedDF = df.groupBy("id").count() + groupedDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .save(path.getCanonicalPath) + + val overwrittenFiles = path.listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + .sortBy(_.getName) + val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0) + + assert(overwrittenFiles === overwrittenFilesWithoutEmpty) + } + } + } } // This class is used to test SPARK-8578. We should not use any custom output committer when From 025b3e9f17d511b1768282d9635145fa87378b5b Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Tue, 17 May 2016 20:44:19 +0200 Subject: [PATCH 0239/1470] [SPARK-15182][ML] Copy MLlib doc to ML: ml.feature.tf, idf ## What changes were proposed in this pull request? We should now begin copying algorithm details from the spark.mllib guide to spark.ml as needed, rather than just linking back to the corresponding algorithms in the spark.mllib user guide. ## How was this patch tested? manual review for doc. Author: Yuhao Yang Author: Yuhao Yang Closes #12957 from hhbyyh/tfidfdoc. (cherry picked from commit 3308a862ba0983268c9d5acf9e2a7d2b62d3ec27) Signed-off-by: Nick Pentreath --- docs/ml-features.md | 51 ++++++++++++++++++++++++++------ docs/mllib-feature-extraction.md | 3 ++ 2 files changed, 45 insertions(+), 9 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index c79bcac461797..c44ace91f23f6 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -18,27 +18,60 @@ This section covers algorithms for working with features, roughly divided into t # Feature Extractors -## TF-IDF (HashingTF and IDF) - -[Term Frequency-Inverse Document Frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a common text pre-processing step. In Spark ML, TF-IDF is separate into two parts: TF (+hashing) and IDF. +## TF-IDF + +[Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) +is a feature vectorization method widely used in text mining to reflect the importance of a term +to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. +Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, while +document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. If we only use +term frequency to measure the importance, it is very easy to over-emphasize terms that appear very +often but carry little information about the document, e.g., "a", "the", and "of". If a term appears +very often across the corpus, it means it doesn't carry special information about a particular document. +Inverse document frequency is a numerical measure of how much information a term provides: +`\[ +IDF(t, D) = \log \frac{|D| + 1}{DF(t, D) + 1}, +\]` +where `$|D|$` is the total number of documents in the corpus. Since logarithm is used, if a term +appears in all documents, its IDF value becomes 0. Note that a smoothing term is applied to avoid +dividing by zero for terms outside the corpus. The TF-IDF measure is simply the product of TF and IDF: +`\[ +TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). +\]` +There are several variants on the definition of term frequency and document frequency. +In MLlib, we separate TF and IDF to make them flexible. **TF**: Both `HashingTF` and `CountVectorizer` can be used to generate the term frequency vectors. `HashingTF` is a `Transformer` which takes sets of terms and converts those sets into fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. -The algorithm combines Term Frequency (TF) counts with the -[hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) for dimensionality reduction. +`HashingTF` utilizes the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). +A raw feature is mapped into an index (term) by applying a hash function. Then term frequencies +are calculated based on the mapped indices. This approach avoids the need to compute a global +term-to-index map, which can be expensive for a large corpus, but it suffers from potential hash +collisions, where different raw features may become the same term after hashing. To reduce the +chance of collision, we can increase the target feature dimension, i.e., the number of buckets +of the hash table. Since a simple modulo is used to transform the hash function to a column index, +it is advisable to use a power of two as the feature dimension, otherwise the features will +not be mapped evenly to the columns. The default feature dimension is `$2^{18} = 262,144$`. `CountVectorizer` converts text documents to vectors of term counts. Refer to [CountVectorizer ](ml-features.html#countvectorizer) for more details. **IDF**: `IDF` is an `Estimator` which is fit on a dataset and produces an `IDFModel`. The -`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and scales each column. -Intuitively, it down-weights columns which appear frequently in a corpus. +`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and +scales each column. Intuitively, it down-weights columns which appear frequently in a corpus. -Please refer to the [MLlib user guide on TF-IDF](mllib-feature-extraction.html#tf-idf) for more details on Term Frequency and Inverse Document Frequency. +**Note:** `spark.ml` doesn't provide tools for text segmentation. +We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and +[scalanlp/chalk](https://github.com/scalanlp/chalk). + +**Examples** -In the following code segment, we start with a set of sentences. We split each sentence into words using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance when using text as features. Our feature vectors could then be passed to a learning algorithm. +In the following code segment, we start with a set of sentences. We split each sentence into words +using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into +a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance +when using text as features. Our feature vectors could then be passed to a learning algorithm.
    diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 7a97285032655..4c027c84ec90b 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -10,6 +10,9 @@ displayTitle: Feature Extraction and Transformation - spark.mllib ## TF-IDF +**Note** We recommend using the DataFrame-based API, which is detailed in the [ML user guide on +TF-IDF](ml-features.html#tf-idf). + [Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a feature vectorization method widely used in text mining to reflect the importance of a term to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. From 1ad3bbd0a4c7d4ab4aad0806f345d3904d7cd441 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 May 2016 20:50:22 +0200 Subject: [PATCH 0240/1470] [MINOR][DOCS] Replace remaining 'sqlContext' in ScalaDoc/JavaDoc. ## What changes were proposed in this pull request? According to the recent change, this PR replaces all the remaining `sqlContext` usage with `spark` in ScalaDoc/JavaDoc (.scala/.java files) except `SQLContext.scala`, `SparkPlan.scala', and `DatasetHolder.scala`. ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13125 from dongjoon-hyun/minor_doc_sparksession. (cherry picked from commit 9f176dd3918129a72282a6b7a12e2899cbb6dac9) Signed-off-by: Nick Pentreath --- .../scala/org/apache/spark/ml/feature/package.scala | 2 +- .../scala/org/apache/spark/sql/DataFrameReader.scala | 4 ++-- .../org/apache/spark/sql/DataFrameStatFunctions.scala | 10 +++++----- .../org/apache/spark/sql/ExperimentalMethods.scala | 2 +- .../datasources/PartitioningAwareFileCatalog.scala | 8 ++++---- .../main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala index 4571ab26800c0..b94187ae787cc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.DataFrame * import org.apache.spark.ml.Pipeline * * // a DataFrame with three columns: id (integer), text (string), and rating (double). - * val df = sqlContext.createDataFrame(Seq( + * val df = spark.createDataFrame(Seq( * (0, "Hi I heard about Spark", 3.0), * (1, "I wish Java could use case classes", 4.0), * (2, "Logistic regression models are neat", 4.0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index e1a64dfc5e7b8..011aff4ff6c28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -446,10 +446,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * Each line in the text file is a new row in the resulting Dataset. For example: * {{{ * // Scala: - * sqlContext.read.text("/path/to/spark/README.md") + * spark.read.text("/path/to/spark/README.md") * * // Java: - * sqlContext.read().text("/path/to/spark/README.md") + * spark.read().text("/path/to/spark/README.md") * }}} * * @param paths input path diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index 3eb1f0f0d58ff..1855eab96eaa5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -160,8 +160,8 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @return A DataFrame containing for the contingency table. * * {{{ - * val df = sqlContext.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), - * (3, 3))).toDF("key", "value") + * val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3))) + * .toDF("key", "value") * val ct = df.stat.crosstab("key", "value") * ct.show() * +---------+---+---+---+ @@ -197,7 +197,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * val rows = Seq.tabulate(100) { i => * if (i % 2 == 0) (1, -1.0) else (i, i * -1.0) * } - * val df = sqlContext.createDataFrame(rows).toDF("a", "b") + * val df = spark.createDataFrame(rows).toDF("a", "b") * // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns * // "a" and "b" * val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4) @@ -258,7 +258,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * val rows = Seq.tabulate(100) { i => * if (i % 2 == 0) (1, -1.0) else (i, i * -1.0) * } - * val df = sqlContext.createDataFrame(rows).toDF("a", "b") + * val df = spark.createDataFrame(rows).toDF("a", "b") * // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns * // "a" and "b" * val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4) @@ -314,7 +314,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @return a new [[DataFrame]] that represents the stratified sample * * {{{ - * val df = sqlContext.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), + * val df = spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), * (3, 3))).toDF("key", "value") * val fractions = Map(1 -> 1.0, 3 -> 0.5) * df.stat.sampleBy("key", fractions, 36L).show() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index a49da6dc2b8b2..a435734b0caef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.Rule * regarding binary compatibility and source compatibility of methods here. * * {{{ - * sqlContext.experimental.extraStrategies += ... + * spark.experimental.extraStrategies += ... * }}} * * @since 1.3.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index e0e4ddc30b0f6..406d2e8e81f49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -168,17 +168,17 @@ abstract class PartitioningAwareFileCatalog( * * By default, the paths of the dataset provided by users will be base paths. * Below are three typical examples, - * Case 1) `sqlContext.read.parquet("/path/something=true/")`: the base path will be + * Case 1) `spark.read.parquet("/path/something=true/")`: the base path will be * `/path/something=true/`, and the returned DataFrame will not contain a column of `something`. - * Case 2) `sqlContext.read.parquet("/path/something=true/a.parquet")`: the base path will be + * Case 2) `spark.read.parquet("/path/something=true/a.parquet")`: the base path will be * still `/path/something=true/`, and the returned DataFrame will also not contain a column of * `something`. - * Case 3) `sqlContext.read.parquet("/path/")`: the base path will be `/path/`, and the returned + * Case 3) `spark.read.parquet("/path/")`: the base path will be `/path/`, and the returned * DataFrame will have the column of `something`. * * Users also can override the basePath by setting `basePath` in the options to pass the new base * path to the data source. - * For example, `sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")`, + * For example, `spark.read.option("basePath", "/path/").parquet("/path/something=true/")`, * and the returned DataFrame will have the column of `something`. */ private def basePaths: Set[Path] = { 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 07f55042eeb40..65bc043076759 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 @@ -2952,8 +2952,8 @@ object functions { * 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) + * val spark = df.sparkSession + * spark.udf.register("simpleUDF", (v: Int) => v * v) * df.select($"id", callUDF("simpleUDF", $"value")) * }}} * From ff1cfce18829ccb176f27d4fcc242cbb341a2277 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 17 May 2016 12:51:07 -0700 Subject: [PATCH 0241/1470] [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms ## What changes were proposed in this pull request? Once SPARK-14487 and SPARK-14549 are merged, we will migrate to use the new vector and matrix type in the new ml pipeline based apis. ## How was this patch tested? Unit tests Author: DB Tsai Author: Liang-Chi Hsieh Author: Xiangrui Meng Closes #12627 from dbtsai/SPARK-14615-NewML. (cherry picked from commit e2efe0529acd748f26dbaa41331d1733ed256237) Signed-off-by: Xiangrui Meng --- dev/sparktestsupport/modules.py | 1 + .../ml/JavaBisectingKMeansExample.java | 2 +- .../examples/ml/JavaDeveloperApiExample.java | 8 +- .../ml/JavaElementwiseProductExample.java | 6 +- .../spark/examples/ml/JavaKMeansExample.java | 2 +- .../ml/AFTSurvivalRegressionExample.scala | 2 +- .../examples/ml/ChiSqSelectorExample.scala | 2 +- .../apache/spark/examples/ml/DCTExample.scala | 2 +- .../spark/examples/ml/DataFrameExample.scala | 3 +- .../examples/ml/DecisionTreeExample.scala | 2 +- .../examples/ml/DeveloperApiExample.scala | 4 +- .../ml/ElementwiseProductExample.scala | 2 +- .../ml/EstimatorTransformerParamExample.scala | 2 +- ...elSelectionViaCrossValidationExample.scala | 2 +- .../apache/spark/examples/ml/PCAExample.scala | 2 +- .../spark/examples/ml/PipelineExample.scala | 2 +- .../ml/PolynomialExpansionExample.scala | 2 +- .../examples/ml/SimpleParamsExample.scala | 4 +- .../ml/SimpleTextClassificationPipeline.scala | 2 +- .../examples/ml/VectorAssemblerExample.scala | 2 +- .../examples/ml/VectorSlicerExample.scala | 2 +- .../scala/org/apache/spark/ml/Predictor.scala | 4 +- .../scala/org/apache/spark/ml/ann/Layer.scala | 22 +- .../spark/ml/attribute/AttributeGroup.scala | 2 +- .../spark/ml/classification/Classifier.scala | 4 +- .../DecisionTreeClassifier.scala | 4 +- .../ml/classification/GBTClassifier.scala | 4 +- .../classification/LogisticRegression.scala | 5 +- .../MultilayerPerceptronClassifier.scala | 4 +- .../spark/ml/classification/NaiveBayes.scala | 9 +- .../spark/ml/classification/OneVsRest.scala | 2 +- .../ProbabilisticClassifier.scala | 2 +- .../RandomForestClassifier.scala | 4 +- .../spark/ml/clustering/BisectingKMeans.scala | 16 +- .../spark/ml/clustering/GaussianMixture.scala | 9 +- .../apache/spark/ml/clustering/KMeans.scala | 18 +- .../org/apache/spark/ml/clustering/LDA.scala | 18 +- .../BinaryClassificationEvaluator.scala | 2 +- .../apache/spark/ml/feature/Binarizer.scala | 2 +- .../spark/ml/feature/ChiSqSelector.scala | 21 +- .../spark/ml/feature/CountVectorizer.scala | 2 +- .../org/apache/spark/ml/feature/DCT.scala | 2 +- .../spark/ml/feature/ElementwiseProduct.scala | 6 +- .../apache/spark/ml/feature/HashingTF.scala | 3 +- .../org/apache/spark/ml/feature/IDF.scala | 15 +- .../apache/spark/ml/feature/Instance.scala | 2 +- .../apache/spark/ml/feature/Interaction.scala | 2 +- .../spark/ml/feature/LabeledPoint.scala | 38 ++++ .../spark/ml/feature/MaxAbsScaler.scala | 8 +- .../spark/ml/feature/MinMaxScaler.scala | 9 +- .../apache/spark/ml/feature/Normalizer.scala | 5 +- .../spark/ml/feature/OneHotEncoder.scala | 2 +- .../org/apache/spark/ml/feature/PCA.scala | 21 +- .../ml/feature/PolynomialExpansion.scala | 2 +- .../apache/spark/ml/feature/RFormula.scala | 2 +- .../spark/ml/feature/RFormulaParser.scala | 2 +- .../spark/ml/feature/StandardScaler.scala | 15 +- .../spark/ml/feature/VectorAssembler.scala | 2 +- .../spark/ml/feature/VectorIndexer.scala | 2 +- .../spark/ml/feature/VectorSlicer.scala | 2 +- .../apache/spark/ml/feature/Word2Vec.scala | 3 +- .../apache/spark/ml/linalg/VectorUDT.scala | 2 +- .../IterativelyReweightedLeastSquares.scala | 2 +- .../spark/ml/optim/WeightedLeastSquares.scala | 3 +- .../org/apache/spark/ml/param/params.scala | 7 +- .../ml/regression/AFTSurvivalRegression.scala | 3 +- .../ml/regression/DecisionTreeRegressor.scala | 4 +- .../spark/ml/regression/GBTRegressor.scala | 4 +- .../GeneralizedLinearRegression.scala | 2 +- .../ml/regression/IsotonicRegression.scala | 2 +- .../ml/regression/LinearRegression.scala | 6 +- .../ml/regression/RandomForestRegressor.scala | 4 +- .../ml/source/libsvm/LibSVMRelation.scala | 4 +- .../scala/org/apache/spark/ml/tree/Node.scala | 2 +- .../org/apache/spark/ml/tree/Split.scala | 2 +- .../ml/tree/impl/DecisionTreeMetadata.scala | 2 +- .../ml/tree/impl/GradientBoostedTrees.scala | 4 +- .../spark/ml/tree/impl/RandomForest.scala | 2 +- .../apache/spark/ml/tree/impl/TreePoint.scala | 2 +- .../org/apache/spark/ml/tree/treeModels.scala | 2 +- .../apache/spark/ml/util/MetadataUtils.scala | 2 +- .../mllib/api/python/PythonMLLibAPI.scala | 195 +++++++++++++++++- .../classification/LogisticRegression.scala | 4 +- .../apache/spark/mllib/linalg/Matrices.scala | 22 ++ .../apache/spark/mllib/linalg/Vectors.scala | 22 ++ .../spark/mllib/regression/LabeledPoint.scala | 9 + .../mllib/tree/GradientBoostedTrees.scala | 12 +- .../spark/mllib/tree/RandomForest.scala | 4 +- .../apache/spark/ml/JavaPipelineSuite.java | 4 +- .../JavaDecisionTreeClassifierSuite.java | 4 +- .../JavaGBTClassifierSuite.java | 4 +- .../JavaLogisticRegressionSuite.java | 6 +- ...vaMultilayerPerceptronClassifierSuite.java | 4 +- .../classification/JavaNaiveBayesSuite.java | 4 +- .../ml/classification/JavaOneVsRestSuite.java | 4 +- .../JavaRandomForestClassifierSuite.java | 6 +- .../spark/ml/clustering/JavaKMeansSuite.java | 2 +- .../apache/spark/ml/feature/JavaDCTSuite.java | 6 +- .../spark/ml/feature/JavaHashingTFSuite.java | 2 +- .../spark/ml/feature/JavaNormalizerSuite.java | 2 +- .../apache/spark/ml/feature/JavaPCASuite.java | 24 ++- .../feature/JavaPolynomialExpansionSuite.java | 6 +- .../ml/feature/JavaStandardScalerSuite.java | 2 +- .../ml/feature/JavaVectorAssemblerSuite.java | 6 +- .../ml/feature/JavaVectorIndexerSuite.java | 2 +- .../ml/feature/JavaVectorSlicerSuite.java | 4 +- .../spark/ml/feature/JavaWord2VecSuite.java | 2 +- .../JavaDecisionTreeRegressorSuite.java | 4 +- .../ml/regression/JavaGBTRegressorSuite.java | 4 +- .../regression/JavaLinearRegressionSuite.java | 4 +- .../JavaRandomForestRegressorSuite.java | 6 +- .../libsvm/JavaLibSVMRelationSuite.java | 4 +- .../ml/tuning/JavaCrossValidatorSuite.java | 4 +- .../org/apache/spark/ml/PipelineSuite.scala | 4 +- .../org/apache/spark/ml/ann/ANNSuite.scala | 5 +- .../apache/spark/ml/ann/GradientSuite.scala | 2 +- .../ml/classification/ClassifierSuite.scala | 4 +- .../DecisionTreeClassifierSuite.scala | 18 +- .../classification/GBTClassifierSuite.scala | 10 +- .../LogisticRegressionSuite.scala | 129 +++++++++++- .../MultilayerPerceptronClassifierSuite.scala | 15 +- .../ml/classification/NaiveBayesSuite.scala | 56 ++++- .../ml/classification/OneVsRestSuite.scala | 12 +- .../ProbabilisticClassifierSuite.scala | 2 +- .../RandomForestClassifierSuite.scala | 10 +- .../spark/ml/clustering/KMeansSuite.scala | 2 +- .../apache/spark/ml/clustering/LDASuite.scala | 4 +- .../BinaryClassificationEvaluatorSuite.scala | 2 +- .../evaluation/RegressionEvaluatorSuite.scala | 2 +- .../spark/ml/feature/BinarizerSuite.scala | 2 +- .../spark/ml/feature/BucketizerSuite.scala | 4 +- .../spark/ml/feature/ChiSqSelectorSuite.scala | 5 +- .../ml/feature/CountVectorizerSuite.scala | 4 +- .../apache/spark/ml/feature/DCTSuite.scala | 2 +- .../ml/feature/ElementwiseProductSuite.scala | 2 +- .../spark/ml/feature/HashingTFSuite.scala | 4 +- .../apache/spark/ml/feature/IDFSuite.scala | 5 +- .../spark/ml/feature/InteractionSuite.scala | 2 +- .../spark/ml/feature/MaxAbsScalerSuite.scala | 2 +- .../spark/ml/feature/MinMaxScalerSuite.scala | 2 +- .../spark/ml/feature/NormalizerSuite.scala | 4 +- .../spark/ml/feature/OneHotEncoderSuite.scala | 2 +- .../apache/spark/ml/feature/PCASuite.scala | 9 +- .../ml/feature/PolynomialExpansionSuite.scala | 4 +- .../spark/ml/feature/RFormulaSuite.scala | 2 +- .../ml/feature/StandardScalerSuite.scala | 4 +- .../ml/feature/VectorAssemblerSuite.scala | 2 +- .../spark/ml/feature/VectorIndexerSuite.scala | 2 +- .../spark/ml/feature/VectorSlicerSuite.scala | 2 +- .../spark/ml/feature/Word2VecSuite.scala | 4 +- .../spark/ml/linalg/VectorUDTSuite.scala | 10 +- ...erativelyReweightedLeastSquaresSuite.scala | 4 +- .../ml/optim/WeightedLeastSquaresSuite.scala | 4 +- .../apache/spark/ml/param/ParamsSuite.scala | 4 +- .../spark/ml/recommendation/ALSSuite.scala | 4 +- .../AFTSurvivalRegressionSuite.scala | 4 +- .../DecisionTreeRegressorSuite.scala | 9 +- .../ml/regression/GBTRegressorSuite.scala | 10 +- .../GeneralizedLinearRegressionSuite.scala | 8 +- .../regression/IsotonicRegressionSuite.scala | 2 +- .../ml/regression/LinearRegressionSuite.scala | 16 +- .../RandomForestRegressorSuite.scala | 10 +- .../source/libsvm/LibSVMRelationSuite.scala | 2 +- .../tree/impl/GradientBoostedTreesSuite.scala | 6 +- .../ml/tree/impl/RandomForestSuite.scala | 20 +- .../apache/spark/ml/tree/impl/TreeTests.scala | 4 +- .../spark/ml/tuning/CrossValidatorSuite.scala | 10 +- .../ml/tuning/TrainValidationSplitSuite.scala | 8 +- .../apache/spark/ml/util/MLTestingUtils.scala | 2 +- .../spark/mllib/linalg/MatricesSuite.scala | 51 +++++ .../spark/mllib/linalg/VectorsSuite.scala | 39 ++++ .../mllib/regression/LabeledPointSuite.scala | 13 ++ .../spark/mllib/tree/DecisionTreeSuite.scala | 20 +- project/MimaExcludes.scala | 46 +++++ python/pyspark/ml/classification.py | 14 +- python/pyspark/ml/clustering.py | 8 +- python/pyspark/ml/evaluation.py | 2 +- python/pyspark/ml/feature.py | 26 +-- python/pyspark/ml/param/__init__.py | 2 +- python/pyspark/ml/regression.py | 14 +- python/pyspark/ml/tests.py | 136 ++++++------ python/pyspark/ml/tuning.py | 4 +- 182 files changed, 1176 insertions(+), 484 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index d7eb44d7a5898..0d6aa7422a534 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -413,6 +413,7 @@ def __hash__(self): "pyspark.ml.feature", "pyspark.ml.classification", "pyspark.ml.clustering", + "pyspark.ml.linalg.__init__", "pyspark.ml.recommendation", "pyspark.ml.regression", "pyspark.ml.tuning", diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 62871448e36f5..8c82aaaacca38 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -20,7 +20,7 @@ // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ 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 3265c4d7ec1fa..8971c0ac002b3 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 @@ -24,13 +24,13 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.classification.Classifier; import org.apache.spark.ml.classification.ClassificationModel; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.BLAS; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.ml.param.IntParam; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.util.Identifiable$; -import org.apache.spark.mllib.linalg.BLAS; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java index 9126242f9eb78..d2e70c23babc7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -26,9 +26,9 @@ import java.util.List; import org.apache.spark.ml.feature.ElementwiseProduct; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.DataTypes; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index 2489a9b80b074..d8f948ae38cb3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -20,7 +20,7 @@ // $example on$ import org.apache.spark.ml.clustering.KMeansModel; import org.apache.spark.ml.clustering.KMeans; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; // $example off$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala index b44304d8104a7..b6d7b369162db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -19,8 +19,8 @@ package org.apache.spark.examples.ml // $example on$ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.regression.AFTSurvivalRegression -import org.apache.spark.mllib.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala index 80f50cd3556af..c9394dd9c64b8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.ChiSqSelector -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala index 5a888b15eb1c9..ddc671752872b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.DCT -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 6cb81cde6f795..8ed6367787203 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -24,7 +24,8 @@ import com.google.common.io.Files import scopt.OptionParser import org.apache.spark.examples.mllib.AbstractParams -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.sql.{DataFrame, Row, SparkSession} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index 310418008c219..5e51dbad760f4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -27,10 +27,10 @@ import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage, Transformer} import org.apache.spark.ml.classification.{DecisionTreeClassificationModel, DecisionTreeClassifier} import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.ml.util.MetadataUtils import org.apache.spark.mllib.evaluation.{MulticlassMetrics, RegressionMetrics} -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, SparkSession} 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 b8f47bf12b872..d94d837d10e96 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 @@ -19,10 +19,10 @@ package org.apache.spark.examples.ml import org.apache.spark.ml.classification.{ClassificationModel, Classifier, ClassifierParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.{Dataset, Row, SparkSession} /** diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala index b99b76e58cef7..c0ffc01934b6f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.ElementwiseProduct -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala index a2918d66ea67e..f18d86e1a6921 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -20,8 +20,8 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.sql.Row // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala index c29d36210ab13..c1ff9ef521706 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala @@ -23,8 +23,8 @@ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tuning.{CrossValidator, ParamGridBuilder} -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.Row // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala index 7927323b42852..dca96eea2ba4e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.PCA -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala index e5e916ac166fb..b16692b1fa36f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -22,7 +22,7 @@ package org.apache.spark.examples.ml import org.apache.spark.ml.{Pipeline, PipelineModel} import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.Row // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala index 94b17a3cd7064..54d2e6b36d149 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.PolynomialExpansion -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 83bab5c55758a..3355c8ffa2952 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -19,9 +19,9 @@ package org.apache.spark.examples.ml import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.{Row, SparkSession} /** diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index c78ff2378bc1f..0b2a058bb61aa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -23,7 +23,7 @@ import scala.beans.BeanInfo import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.{Row, SparkSession} @BeanInfo diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala index 8e382ccc4844a..8910470c1cf7a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.VectorAssembler -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors // $example off$ import org.apache.spark.sql.SparkSession diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala index b1a3997f48c91..85dd5c27766c2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -23,7 +23,7 @@ import java.util.Arrays import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} import org.apache.spark.ml.feature.VectorSlicer -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.sql.Row import org.apache.spark.sql.types.StructType // $example off$ diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index 81140d1f7b21f..569a5fb993768 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} 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.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 3588ac1e95bee..889b7f9f719c7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -21,7 +21,9 @@ import java.util.Random import breeze.linalg.{*, axpy => Baxpy, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.optimization._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom @@ -578,10 +580,10 @@ private[ann] object FeedForwardModel { */ private[ann] class ANNGradient(topology: Topology, dataStacker: DataStacker) extends Gradient { override def compute( - data: Vector, + data: OldVector, label: Double, - weights: Vector, - cumGradient: Vector): Double = { + weights: OldVector, + cumGradient: OldVector): Double = { val (input, target, realBatchSize) = dataStacker.unstack(data) val model = topology.model(weights) model.computeGradient(input, target, cumGradient, realBatchSize) @@ -655,15 +657,15 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) private[ann] class ANNUpdater extends Updater { override def compute( - weightsOld: Vector, - gradient: Vector, + weightsOld: OldVector, + gradient: OldVector, stepSize: Double, iter: Int, - regParam: Double): (Vector, Double) = { + regParam: Double): (OldVector, Double) = { val thisIterStepSize = stepSize val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector Baxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) - (Vectors.fromBreeze(brzWeights), 0) + (OldVectors.fromBreeze(brzWeights), 0) } } @@ -806,7 +808,9 @@ private[ml] class FeedForwardTrainer( getWeights } // TODO: deprecate standard optimizer because it needs Vector - val newWeights = optimizer.optimize(dataStacker.stack(data), w) + val newWeights = optimizer.optimize(dataStacker.stack(data).map { v => + (v._1, OldVectors.fromML(v._2)) + }, w) topology.model(newWeights) } 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 2c29eeb01a921..12b9732a4c3d2 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 @@ -20,7 +20,7 @@ package org.apache.spark.ml.attribute import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} /** 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 bc5fe35ad4a5c..e35b04a1cf423 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 @@ -20,10 +20,10 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.shared.HasRawPredictionCol import org.apache.spark.ml.util.{MetadataUtils, SchemaUtils} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 31a69d49a078a..881dcefb79be3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -22,13 +22,13 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index acc04582b8698..f843df449c614 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -24,14 +24,14 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD 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 62d68973680a3..40cff8a86f362 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 @@ -27,12 +27,13 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 72cf55f6bb997..683ae4aaf407e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -24,11 +24,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.ann.{FeedForwardTopology, FeedForwardTrainer} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed, HasStepSize, HasTol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.Dataset /** Params for Multilayer Perceptron. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 267d63b51eb6c..a98bdeca6b723 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -22,14 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.util._ import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes} import org.apache.spark.mllib.classification.{NaiveBayesModel => OldNaiveBayesModel} -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset /** * Params for Naive Bayes Classifiers. @@ -102,7 +102,8 @@ class NaiveBayes @Since("1.5.0") ( setDefault(modelType -> OldNaiveBayes.Multinomial) override protected def train(dataset: Dataset[_]): NaiveBayesModel = { - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val oldDataset: RDD[OldLabeledPoint] = + extractLabeledPoints(dataset).map(OldLabeledPoint.fromML) val oldModel = OldNaiveBayes.train(oldDataset, $(smoothing), $(modelType)) NaiveBayesModel.fromOld(oldModel, this) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index f10c60a78df1a..047a378b79aa7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -32,9 +32,9 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ 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 d00fee12b08c0..59277d0f42b34 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,9 +18,9 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.linalg.{DenseVector, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 1d33ae83c211f..b3c074f839250 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -21,13 +21,13 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 6cc9117da3fea..138e059f94938 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -21,12 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.clustering. - {BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.clustering.{BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -105,7 +107,7 @@ class BisectingKMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) @Since("2.0.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** * Computes the sum of squared distances between the input points and their corresponding cluster @@ -115,7 +117,7 @@ class BisectingKMeansModel private[ml] ( def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) val data = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } - parentModel.computeCost(data) + parentModel.computeCost(data.map(OldVectors.fromML)) } @Since("2.0.0") @@ -216,7 +218,9 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): BisectingKMeansModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val bkm = new MLlibBisectingKMeans() .setK($(k)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index ac86e4ce25e82..63ca812609b7e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -32,6 +32,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{GaussianMixture => MLlibGM} import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors, VectorUDT => OldVectorUDT} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -94,8 +95,8 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - val predUDF = udf((vector: OldVector) => predict(vector.asML)) - val probUDF = udf((vector: OldVector) => OldVectors.fromML(predictProbability(vector.asML))) + val predUDF = udf((vector: Vector) => predict(vector)) + val probUDF = udf((vector: Vector) => predictProbability(vector)) dataset.withColumn($(predictionCol), predUDF(col($(featuresCol)))) .withColumn($(probabilityCol), probUDF(col($(featuresCol)))) } @@ -296,7 +297,9 @@ class GaussianMixture @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): GaussianMixtureModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: OldVector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val algo = new MLlibGM() .setK($(k)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 42a25396adb40..41c0aec0ecf96 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -22,11 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -127,7 +130,7 @@ class KMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) @Since("1.5.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this @@ -137,7 +140,9 @@ class KMeansModel private[ml] ( @Since("2.0.0") def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) - val data = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val data: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } parentModel.computeCost(data) } @@ -210,7 +215,8 @@ object KMeansModel extends MLReadable[KMeansModel] { val dataPath = new Path(path, "data").toString val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] val clusterCenters = data.collect().sortBy(_.clusterIdx).map(_.clusterCenter) - val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) + val model = new KMeansModel(metadata.uid, + new MLlibKMeansModel(clusterCenters.map(OldVectors.fromML))) DefaultParamsReader.getAndSetParams(model, metadata) model @@ -277,7 +283,9 @@ class KMeans @Since("1.5.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): KMeansModel = { - val rdd = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } val instr = Instrumentation.create(this, rdd) instr.logParams(featuresCol, predictionCol, k, initMode, initSteps, maxIter, seed, tol) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 38ecc5a102c12..5a83b2870005d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed} import org.apache.spark.ml.util._ @@ -30,7 +31,10 @@ import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedL LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel, OnlineLDAOptimizer => OldOnlineLDAOptimizer} import org.apache.spark.mllib.impl.PeriodicCheckpointer -import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Vector => OldVector, + Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, monotonicallyIncreasingId, udf} @@ -405,7 +409,11 @@ sealed abstract class LDAModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { if ($(topicDistributionCol).nonEmpty) { - val t = udf(oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext)) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer = oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext) + + val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML } dataset.withColumn($(topicDistributionCol), t(col($(featuresCol)))).toDF } else { logWarning("LDAModel.transform was called without any output columns. Set an output column" + @@ -437,7 +445,7 @@ sealed abstract class LDAModel private[ml] ( * collecting a large amount of data to the driver (on the order of vocabSize x k). */ @Since("1.6.0") - def topicsMatrix: Matrix = oldLocalModel.topicsMatrix + def topicsMatrix: Matrix = oldLocalModel.topicsMatrix.asML /** Indicates whether this instance is of type [[DistributedLDAModel]] */ @Since("1.6.0") @@ -872,13 +880,13 @@ class LDA @Since("1.6.0") ( private[clustering] object LDA extends DefaultParamsReadable[LDA] { /** Get dataset for spark.mllib LDA */ - def getOldDataset(dataset: Dataset[_], featuresCol: String): RDD[(Long, Vector)] = { + def getOldDataset(dataset: Dataset[_], featuresCol: String): RDD[(Long, OldVector)] = { dataset .withColumn("docId", monotonicallyIncreasingId()) .select("docId", featuresCol) .rdd .map { case Row(docId: Long, features: Vector) => - (docId, features) + (docId, OldVectors.fromML(features)) } } 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 0cbc391d96f8a..bff72b20e1c3f 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 @@ -18,11 +18,11 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 35bc46a5f3435..318c8b8b2f7d6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -22,10 +22,10 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.BinaryAttribute +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 29f55a7f715ca..e73a8f5d66087 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -22,12 +22,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ import org.apache.spark.ml.attribute.{AttributeGroup, _} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -80,10 +82,11 @@ final class ChiSqSelector(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): ChiSqSelectorModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - } + val input: RDD[OldLabeledPoint] = + dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + OldLabeledPoint(label, OldVectors.fromML(features)) + } val chiSqSelector = new feature.ChiSqSelector($(numTopFeatures)).fit(input) copyValues(new ChiSqSelectorModel(uid, chiSqSelector).setParent(this)) } @@ -132,7 +135,11 @@ final class ChiSqSelectorModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { val transformedSchema = transformSchema(dataset.schema, logging = true) val newField = transformedSchema.last - val selector = udf { chiSqSelector.transform _ } + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => chiSqSelector.transform(OldVectors.fromML(v)).asML + + val selector = udf(transformer) dataset.withColumn($(outputCol), selector(col($(featuresCol))), newField.metadata) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 3fbfce9d48dd2..fc4885bf4ba84 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -21,10 +21,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.broadcast.Broadcast import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vectors, VectorUDT} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index a6f878151de73..301358ef1226c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -21,9 +21,9 @@ import edu.emory.mathcs.jtransforms.dct._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.BooleanParam import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.types.DataType /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 1b0a9a12e83bc..91989c3d2f5d2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -19,10 +19,12 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.Param import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.types.DataType /** @@ -52,7 +54,7 @@ class ElementwiseProduct(override val uid: String) override protected def createTransformFunc: Vector => Vector = { require(params.contains(scalingVec), s"transformation requires a weight vector") val elemScaler = new feature.ElementwiseProduct($(scalingVec)) - elemScaler.transform + v => elemScaler.transform(v) } override protected def outputDataType: DataType = new VectorUDT() 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 66ae91cfc0970..94e1825ba61e4 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 @@ -86,7 +86,8 @@ class HashingTF(override val uid: String) override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) val hashingTF = new feature.HashingTF($(numFeatures)).setBinary($(binary)) - val t = udf { terms: Seq[_] => hashingTF.transform(terms) } + // TODO: Make the hashingTF.transform natively in ml framework to avoid extra conversion. + val t = udf { terms: Seq[_] => hashingTF.transform(terms).asML } val metadata = outputSchema($(outputCol)).metadata dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 5075b78c9856a..f85f4c65af7d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -21,11 +21,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType @@ -79,7 +81,9 @@ final class IDF(override val uid: String) extends Estimator[IDFModel] with IDFBa @Since("2.0.0") override def fit(dataset: Dataset[_]): IDFModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val idf = new feature.IDF($(minDocFreq)).fit(input) copyValues(new IDFModel(uid, idf).setParent(this)) } @@ -119,7 +123,8 @@ class IDFModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val idf = udf { vec: Vector => idfModel.transform(vec) } + // TODO: Make the idfModel.transform natively in ml framework to avoid extra conversion. + val idf = udf { vec: Vector => idfModel.transform(OldVectors.fromML(vec)).asML } dataset.withColumn($(outputCol), idf(col($(inputCol)))) } @@ -134,7 +139,7 @@ class IDFModel private[ml] ( /** Returns the IDF vector. */ @Since("1.6.0") - def idf: Vector = idfModel.idf + def idf: Vector = idfModel.idf.asML @Since("1.6.0") override def write: MLWriter = new IDFModelWriter(this) @@ -166,7 +171,7 @@ object IDFModel extends MLReadable[IDFModel] { .select("idf") .head() val idf = data.getAs[Vector](0) - val model = new IDFModel(metadata.uid, new feature.IDFModel(idf)) + val model = new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf))) DefaultParamsReader.getAndSetParams(model, metadata) model } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index 12176757aee3d..cce3ca45ccd8f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector /** * Class that represents an instance of weighted data point with label and features. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index 9ca34e9ae22f4..fa65ff987917e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -26,7 +26,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.ml.Transformer -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala new file mode 100644 index 0000000000000..f7f1d42039599 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala @@ -0,0 +1,38 @@ +/* + * 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 + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.Vector + +/** + * Class that represents the features and labels of a data point. + * + * @param label Label for this data point. + * @param features List of features for this data point. + */ +@Since("2.0.0") +@Experimental +@BeanInfo +case class LabeledPoint(@Since("2.0.0") label: Double, @Since("2.0.0") features: Vector) { + override def toString: String = { + s"($label,$features)" + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index e9df600c8a991..0dffba93ac571 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -21,11 +21,13 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.{ParamMap, Params} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -69,7 +71,9 @@ class MaxAbsScaler @Since("2.0.0") (override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): MaxAbsScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val summary = Statistics.colStats(input) val minVals = summary.min.toArray val maxVals = summary.max.toArray diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 125becbb8a5b5..c6ff639f29620 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -21,11 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamMap, Params} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -106,7 +109,9 @@ class MinMaxScaler(override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): MinMaxScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val summary = Statistics.colStats(input) copyValues(new MinMaxScalerModel(uid, summary.min, summary.max).setParent(this)) } 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 a603b3f833202..942ac7ebdb3bd 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 @@ -19,10 +19,11 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamValidators} import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.sql.types.DataType /** @@ -52,7 +53,7 @@ class Normalizer(override val uid: String) override protected def createTransformFunc: Vector => Vector = { val normalizer = new feature.Normalizer($(p)) - normalizer.transform + vector => normalizer.transform(OldVectors.fromML(vector)).asML } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 99357793dbaeb..3d1e6dd818829 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 9cf722e121697..141d3b924b4fe 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -21,11 +21,16 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, DenseVector => OldDenseVector, + Matrices => OldMatrices, Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -71,7 +76,9 @@ class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams @Since("2.0.0") override def fit(dataset: Dataset[_]): PCAModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v} + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val pca = new feature.PCA(k = $(k)) val pcaModel = pca.fit(input) copyValues(new PCAModel(uid, pcaModel.pc, pcaModel.explainedVariance).setParent(this)) @@ -128,8 +135,14 @@ class PCAModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val pcaModel = new feature.PCAModel($(k), pc, explainedVariance) - val pcaOp = udf { pcaModel.transform _ } + val pcaModel = new feature.PCAModel($(k), + OldMatrices.fromML(pc).asInstanceOf[OldDenseMatrix], + OldVectors.fromML(explainedVariance).asInstanceOf[OldDenseVector]) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => pcaModel.transform(OldVectors.fromML(v)).asML + + val pcaOp = udf(transformer) dataset.withColumn($(outputCol), pcaOp(col($(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 0a9b9719c15d3..a01867701bd8b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -21,9 +21,9 @@ import scala.collection.mutable import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.types.DataType /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index a2f3d44132d1d..c0feaa01fc861 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -25,10 +25,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model, Pipeline, PipelineModel, PipelineStage, Transformer} import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.VectorUDT import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala index cf52710ab8cbe..19aecff03885c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable import scala.util.parsing.combinator.RegexParsers -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types._ /** 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 626e97efb47c6..9d084b520c488 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 @@ -21,11 +21,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -93,7 +96,9 @@ class StandardScaler(override val uid: String) extends Estimator[StandardScalerM @Since("2.0.0") override def fit(dataset: Dataset[_]): StandardScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val scaler = new feature.StandardScaler(withMean = $(withMean), withStd = $(withStd)) val scalerModel = scaler.fit(input) copyValues(new StandardScalerModel(uid, scalerModel.std, scalerModel.mean).setParent(this)) @@ -145,7 +150,11 @@ class StandardScalerModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) val scaler = new feature.StandardScalerModel(std, mean, $(withStd), $(withMean)) - val scale = udf { scaler.transform _ } + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => scaler.transform(OldVectors.fromML(v)).asML + + val scale = udf(transformer) dataset.withColumn($(outputCol), scale(col($(inputCol)))) } 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 4d3e46e488c67..1bc24202b7615 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 @@ -23,10 +23,10 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute, UnresolvedAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ 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 68b699d569c7d..2bc9d225ac2f6 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 @@ -27,10 +27,10 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, VectorUDT} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{StructField, StructType} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index 7a9468b87b73e..103738cd91c09 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntArrayParam, ParamMap, StringArrayParam} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index c49e263df0a6d..1469bfd5e857d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -22,11 +22,12 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala index c29f7f86e9f27..0b9b2ff5c5e26 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._ * User-defined type for [[Vector]] in [[mllib-local]] which allows easy interaction with SQL * via [[org.apache.spark.sql.Dataset]]. */ -private[ml] class VectorUDT extends UserDefinedType[Vector] { +private[spark] class VectorUDT extends UserDefinedType[Vector] { override def sqlType: StructType = { // type: 0 = sparse, 1 = dense diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index a2b52835e177a..6ed193cf57d4f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 7d21302f962bf..8f5f4427e1f4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -19,7 +19,8 @@ package org.apache.spark.ml.optim import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ +import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.rdd.RDD /** 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 c368aadd23669..82f2de7ccdfdb 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 @@ -29,8 +29,9 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.JsonVectorConverter import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: DeveloperApi :: @@ -92,7 +93,7 @@ class Param[T](val parent: String, val name: String, val doc: String, val isVali case x: String => compact(render(JString(x))) case v: Vector => - v.toJson + JsonVectorConverter.toJson(v) case _ => throw new NotImplementedError( "The default jsonEncode only supports string and vector. " + @@ -128,7 +129,7 @@ private[ml] object Param { val keys = v.map(_._1) assert(keys.contains("type") && keys.contains("values"), s"Expect a JSON serialized vector but cannot find fields 'type' and 'values' in $json.") - Vectors.fromJson(json).asInstanceOf[T] + JsonVectorConverter.fromJson(json).asInstanceOf[T] case _ => throw new NotImplementedError( "The default jsonDecode only supports string and vector. " + diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 89ba6ab5d2772..cc16c2f038be8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -27,10 +27,11 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors, VectorUDT} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 874d2a81db216..c4df9d11127f4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -23,13 +23,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index c41fb4b0629b2..81f2139f0b421 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -24,13 +24,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.GradientBoostedTrees import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 05fffa0d97e6b..4aa7c2cc0b9b2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -25,11 +25,11 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector} import org.apache.spark.ml.optim._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 7a78ecbdf16de..ba0f59e89bb24 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -22,11 +22,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.regression.IsotonicRegressionModel.IsotonicRegressionModelWriter import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.regression.{IsotonicRegression => MLlibIsotonicRegression} import org.apache.spark.mllib.regression.{IsotonicRegressionModel => MLlibIsotonicRegressionModel} import org.apache.spark.rdd.RDD 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 d13b15fd82f05..3e9a3f9db5f41 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 @@ -28,14 +28,16 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.optim.WeightedLeastSquares import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 9605de72020f1..a6dbf21d55e2b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -22,13 +22,13 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 39bdd1afadcc8..5ba768d551842 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -26,8 +26,8 @@ import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, DataFrameReader, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index f71d28cf59535..d5e5c454605b7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.tree import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{ImpurityStats, InformationGainStats => OldInformationGainStats, Node => OldNode, Predict => OldPredict} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index a4287483d18ed..9704e15cd838f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.tree import java.util.Objects import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} import org.apache.spark.mllib.tree.model.{Split => OldSplit} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index 5f7c40f6071f6..442f52bf0231d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -21,8 +21,8 @@ import scala.collection.mutable import scala.util.Try import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.RandomForestParams -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ import org.apache.spark.mllib.tree.configuration.Strategy diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index b6334762c7a7f..a0faff236e9d5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.configuration.{BoostingStrategy => OldBoostingStrategy} import org.apache.spark.mllib.tree.impurity.{Variance => OldVariance} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 2038a6873db73..be3792eb7732f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -24,10 +24,10 @@ import scala.util.Random import org.apache.spark.internal.Logging import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.util.Instrumentation -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.ImpurityStats diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala index 3a2bf3c725730..a6ac64a0463cc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.tree.impl +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.{ContinuousSplit, Split} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index f38e1ec7c09a8..56c85c9b53e17 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -23,11 +23,11 @@ import org.apache.hadoop.fs.Path import org.json4s._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.{Param, Params} import org.apache.spark.ml.tree.DecisionTreeModelReadWrite.NodeData import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter} import org.apache.spark.ml.util.DefaultParamsReader.Metadata -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala index 96a38a3bde960..f34a8310ddf1c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.util import scala.collection.immutable.HashMap import org.apache.spark.ml.attribute._ -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.StructField 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 8daee7b3aa1e9..90d382753131d 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 @@ -30,6 +30,8 @@ import net.razorvine.pickle._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.ml.feature.{LabeledPoint => MLLabeledPoint} +import org.apache.spark.ml.linalg.{DenseMatrix => NewDenseMatrix, DenseVector => NewDenseVector, SparseMatrix => NewSparseMatrix, SparseVector => NewSparseVector, Vector => NewVector, Vectors => NewVectors} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.evaluation.RankingMetrics @@ -1207,6 +1209,7 @@ private[python] class PythonMLLibAPI extends Serializable { private[spark] object SerDe extends Serializable { val PYSPARK_PACKAGE = "pyspark.mllib" + val PYSPARK_ML_PACKAGE = "pyspark.ml" /** * Base class used for pickle @@ -1214,8 +1217,10 @@ private[spark] object SerDe extends Serializable { private[python] abstract class BasePickler[T: ClassTag] extends IObjectPickler with IObjectConstructor { + protected def packageName: String = PYSPARK_PACKAGE + private val cls = implicitly[ClassTag[T]].runtimeClass - private val module = PYSPARK_PACKAGE + "." + cls.getName.split('.')(4) + private val module = packageName + "." + cls.getName.split('.')(4) private val name = cls.getSimpleName // register this to Pickler and Unpickler @@ -1262,7 +1267,7 @@ private[spark] object SerDe extends Serializable { private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } - // Pickler for DenseVector + // Pickler for (mllib) DenseVector private[python] class DenseVectorPickler extends BasePickler[DenseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1294,7 +1299,41 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for DenseMatrix + // Pickler for (new) DenseVector + private[python] class NewDenseVectorPickler extends BasePickler[NewDenseVector] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val vector: NewDenseVector = obj.asInstanceOf[NewDenseVector] + val bytes = new Array[Byte](8 * vector.size) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + db.put(vector.values) + + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE1) + } + + def construct(args: Array[Object]): Object = { + require(args.length == 1) + if (args.length != 1) { + throw new PickleException("should be 1") + } + val bytes = getBytes(args(0)) + val bb = ByteBuffer.wrap(bytes, 0, bytes.length) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](bytes.length / 8) + db.get(ans) + NewVectors.dense(ans) + } + } + + // Pickler for (mllib) DenseMatrix private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1331,7 +1370,46 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for SparseMatrix + // Pickler for (new) DenseMatrix + private[python] class NewDenseMatrixPickler extends BasePickler[NewDenseMatrix] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val m: NewDenseMatrix = obj.asInstanceOf[NewDenseMatrix] + val bytes = new Array[Byte](8 * m.values.length) + val order = ByteOrder.nativeOrder() + val isTransposed = if (m.isTransposed) 1 else 0 + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 4) { + throw new PickleException("should be 4") + } + val bytes = getBytes(args(2)) + val n = bytes.length / 8 + val values = new Array[Double](n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(3).asInstanceOf[Int] == 1 + new NewDenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) + } + } + + // Pickler for (mllib) SparseMatrix private[python] class SparseMatrixPickler extends BasePickler[SparseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1386,7 +1464,64 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for SparseVector + // Pickler for (new) SparseMatrix + private[python] class NewSparseMatrixPickler extends BasePickler[NewSparseMatrix] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val s = obj.asInstanceOf[NewSparseMatrix] + val order = ByteOrder.nativeOrder() + + val colPtrsBytes = new Array[Byte](4 * s.colPtrs.length) + val indicesBytes = new Array[Byte](4 * s.rowIndices.length) + val valuesBytes = new Array[Byte](8 * s.values.length) + val isTransposed = if (s.isTransposed) 1 else 0 + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().put(s.colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().put(s.rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().put(s.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(colPtrsBytes.length)) + out.write(colPtrsBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indicesBytes.length)) + out.write(indicesBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valuesBytes.length)) + out.write(valuesBytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 6) { + throw new PickleException("should be 6") + } + val order = ByteOrder.nativeOrder() + val colPtrsBytes = getBytes(args(2)) + val indicesBytes = getBytes(args(3)) + val valuesBytes = getBytes(args(4)) + val colPtrs = new Array[Int](colPtrsBytes.length / 4) + val rowIndices = new Array[Int](indicesBytes.length / 4) + val values = new Array[Double](valuesBytes.length / 8) + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().get(colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().get(rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(5).asInstanceOf[Int] == 1 + new NewSparseMatrix( + args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], colPtrs, rowIndices, values, + isTransposed) + } + } + + // Pickler for (mllib) SparseVector private[python] class SparseVectorPickler extends BasePickler[SparseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1428,7 +1563,51 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for LabeledPoint + // Pickler for (new) SparseVector + private[python] class NewSparseVectorPickler extends BasePickler[NewSparseVector] { + + override protected def packageName = PYSPARK_ML_PACKAGE + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val v: NewSparseVector = obj.asInstanceOf[NewSparseVector] + val n = v.indices.length + val indiceBytes = new Array[Byte](4 * n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) + val valueBytes = new Array[Byte](8 * n) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(v.size)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) + out.write(indiceBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valueBytes.length)) + out.write(valueBytes) + out.write(Opcodes.TUPLE3) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + val size = args(0).asInstanceOf[Int] + 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) + if (n > 0) { + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) + } + new NewSparseVector(size, indices, values) + } + } + + // Pickler for MLlib LabeledPoint private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1482,6 +1661,10 @@ private[spark] object SerDe extends Serializable { new DenseMatrixPickler().register() new SparseMatrixPickler().register() new SparseVectorPickler().register() + new NewDenseVectorPickler().register() + new NewDenseMatrixPickler().register() + new NewSparseMatrixPickler().register() + new NewSparseVectorPickler().register() new LabeledPointPickler().register() new RatingPickler().register() initialized = true diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 1d25a58e0f2e8..f3c52f61a3bb5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -431,7 +431,7 @@ class LogisticRegressionWithLBFGS if (userSuppliedWeights) { val uid = Identifiable.randomUID("logreg-static") lr.setInitialModel(new org.apache.spark.ml.classification.LogisticRegressionModel( - uid, initialWeights, 1.0)) + uid, initialWeights.asML, 1.0)) } lr.setFitIntercept(addIntercept) lr.setMaxIter(optimizer.getNumIterations()) @@ -439,7 +439,7 @@ class LogisticRegressionWithLBFGS // Convert our input into a DataFrame val sqlContext = new SQLContext(input.context) import sqlContext.implicits._ - val df = input.toDF() + val df = input.map(_.asML).toDF() // Determine if we should cache the DF val handlePersistence = input.getStorageLevel == StorageLevel.NONE // Train our model 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 076cca6016ecb..5c9a112ca69ca 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 @@ -20,6 +20,7 @@ package org.apache.spark.mllib.linalg import java.util.{Arrays, Random} import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, HashSet => MHashSet} +import scala.language.implicitConversions import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -1212,3 +1213,24 @@ object Matrices { SparseMatrix.fromML(sm) } } + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Matrix]] to + * [[org.apache.spark.ml.linalg.Matrix]] and vice versa. + */ +private[spark] object MatrixImplicits { + + implicit def mllibMatrixToMLMatrix(m: Matrix): newlinalg.Matrix = m.asML + + implicit def mllibDenseMatrixToMLDenseMatrix(m: DenseMatrix): newlinalg.DenseMatrix = m.asML + + implicit def mllibSparseMatrixToMLSparseMatrix(m: SparseMatrix): newlinalg.SparseMatrix = m.asML + + implicit def mlMatrixToMLlibMatrix(m: newlinalg.Matrix): Matrix = Matrices.fromML(m) + + implicit def mlDenseMatrixToMLlibDenseMatrix(m: newlinalg.DenseMatrix): DenseMatrix = + Matrices.fromML(m).asInstanceOf[DenseMatrix] + + implicit def mlSparseMatrixToMLlibSparseMatrix(m: newlinalg.SparseMatrix): SparseMatrix = + Matrices.fromML(m).asInstanceOf[SparseMatrix] +} 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 132e54a8c3de2..1f1cfa0cb28b4 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 @@ -22,6 +22,7 @@ import java.util import scala.annotation.varargs import scala.collection.JavaConverters._ +import scala.language.implicitConversions import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.json4s.DefaultFormats @@ -926,3 +927,24 @@ object SparseVector { new SparseVector(v.size, v.indices, v.values) } } + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Vector]] to + * [[org.apache.spark.ml.linalg.Vector]] and vice versa. + */ +private[spark] object VectorImplicits { + + implicit def mllibVectorToMLVector(v: Vector): newlinalg.Vector = v.asML + + implicit def mllibDenseVectorToMLDenseVector(v: DenseVector): newlinalg.DenseVector = v.asML + + implicit def mllibSparseVectorToMLSparseVector(v: SparseVector): newlinalg.SparseVector = v.asML + + implicit def mlVectorToMLlibVector(v: newlinalg.Vector): Vector = Vectors.fromML(v) + + implicit def mlDenseVectorToMLlibDenseVector(v: newlinalg.DenseVector): DenseVector = + Vectors.fromML(v).asInstanceOf[DenseVector] + + implicit def mlSparseVectorToMLlibSparseVector(v: newlinalg.SparseVector): SparseVector = + Vectors.fromML(v).asInstanceOf[SparseVector] +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 45540f0c5c4ce..f082b16b95e81 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import scala.beans.BeanInfo import org.apache.spark.annotation.Since +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.NumericParser import org.apache.spark.SparkException @@ -38,6 +39,10 @@ case class LabeledPoint @Since("1.0.0") ( override def toString: String = { s"($label,$features)" } + + private[spark] def asML: NewLabeledPoint = { + NewLabeledPoint(label, features.asML) + } } /** @@ -67,4 +72,8 @@ object LabeledPoint { LabeledPoint(label, features) } } + + private[spark] def fromML(point: NewLabeledPoint): LabeledPoint = { + LabeledPoint(point.label, Vectors.fromML(point.features)) + } } 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 7fe60e2d99e4f..ece1e41d986d0 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 @@ -20,6 +20,7 @@ package org.apache.spark.mllib.tree import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.ml.tree.impl.{GradientBoostedTrees => NewGBT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.BoostingStrategy @@ -66,7 +67,9 @@ class GradientBoostedTrees private[spark] ( @Since("1.2.0") def run(input: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.run(input, boostingStrategy, seed.toLong) + val (trees, treeWeights) = NewGBT.run(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } @@ -94,8 +97,11 @@ class GradientBoostedTrees private[spark] ( input: RDD[LabeledPoint], validationInput: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.runWithValidation(input, validationInput, boostingStrategy, - seed.toLong) + val (trees, treeWeights) = NewGBT.runWithValidation(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, validationInput.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index c31ed9c1ce0b8..14f11ce51b878 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -90,8 +90,8 @@ private class RandomForest ( * @return RandomForestModel that can be used for prediction. */ def run(input: RDD[LabeledPoint]): RandomForestModel = { - val trees: Array[NewDTModel] = - NewRandomForest.run(input, strategy, numTrees, featureSubsetStrategy, seed.toLong, None) + val trees: Array[NewDTModel] = NewRandomForest.run(input.map(_.asML), strategy, numTrees, + featureSubsetStrategy, seed.toLong, None) new RandomForestModel(strategy.algo, trees.map(_.toOld)) } diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index e0c4363597da4..46c26e8b929e1 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -24,12 +24,12 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegression; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.feature.StandardScaler; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; /** * Test Pipeline construction and fitting in Java. diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java index 8b899913277fc..c76a1947c64f7 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java index 682371eb9e4d5..4648926c3421f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index e3ff68364e690..98abca221c887 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -27,12 +27,12 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; public class JavaLogisticRegressionSuite implements Serializable { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java index b0624cea3ecd7..48edbc838c818 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java @@ -26,8 +26,8 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java index 3fc3648627153..787909821be9b 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java @@ -26,8 +26,8 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index 486fbbd58c179..58bc5a448aed3 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -29,11 +29,11 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; public class JavaOneVsRestSuite implements Serializable { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java index e3855662fb6de..1ed20b1bfa17d 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java @@ -28,10 +28,10 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java index 3ab09ac27d306..9d07170fa1d5d 100644 --- a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java @@ -28,7 +28,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index 06482d8f0dcd1..9d8c09b30c019 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -27,9 +27,9 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java index 0e21d4a94f24b..3c37441a77beb 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java @@ -25,7 +25,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java index 04b2897b18117..b3e213a49700e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java @@ -25,7 +25,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index 32f6b4375ee72..a4bce2283b860 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -31,9 +31,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -88,9 +88,25 @@ public void testPCA() { ); JavaRDD dataRDD = jsc.parallelize(points, 2); - RowMatrix mat = new RowMatrix(dataRDD.rdd()); + RowMatrix mat = new RowMatrix(dataRDD.map( + new Function() { + public org.apache.spark.mllib.linalg.Vector call(Vector vector) { + return new org.apache.spark.mllib.linalg.DenseVector(vector.toArray()); + } + } + ).rdd()); + Matrix pc = mat.computePrincipalComponents(3); - JavaRDD expected = mat.multiply(pc).rows().toJavaRDD(); + + mat.multiply(pc).rows().toJavaRDD(); + + JavaRDD expected = mat.multiply(pc).rows().toJavaRDD().map( + new Function() { + public Vector call(org.apache.spark.mllib.linalg.Vector vector) { + return vector.asML(); + } + } + ); JavaRDD featuresExpected = dataRDD.zip(expected).map( new Function, VectorPair>() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java index 8f726077a2490..a28f73f10a0b9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java @@ -26,9 +26,9 @@ import org.junit.Test; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java index c7397bdd6878d..8415fdb84f213 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java @@ -25,7 +25,7 @@ import org.junit.Test; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java index 8774cd0c69f19..fedaa77176586 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java @@ -25,9 +25,9 @@ import org.junit.Test; import org.apache.spark.SparkConf; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; 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 index c386c9a45b099..a8dd44608db32 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -29,7 +29,7 @@ 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.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index 59ad3c2f61e85..a565c77af4ab1 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -28,8 +28,8 @@ import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java index 392aabc96d372..bef7eb0f995fb 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java @@ -24,7 +24,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java index bbd59a04ec867..4ea3f2255efa7 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java index 5370b58e8fda5..3b5edf1e15aef 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java @@ -27,9 +27,9 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index 00c59f08b679a..d3ef5f6fca076 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -27,11 +27,11 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; public class JavaLinearRegressionSuite implements Serializable { diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java index fdb41ffc10388..d601e7c540e19 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java @@ -28,10 +28,10 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java index 058f2ddafd866..022dcf94bd006 100644 --- a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java @@ -28,8 +28,8 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.mllib.linalg.DenseVector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.DenseVector; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java index 8b4d034ffea02..b874ccd48b3b1 100644 --- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java @@ -27,13 +27,13 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegression; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; public class JavaCrossValidatorSuite implements Serializable { diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala index 55448325e44df..3b490cdf56018 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -27,9 +27,9 @@ import org.scalatest.mock.MockitoSugar.mock import org.apache.spark.SparkFunSuite import org.apache.spark.ml.Pipeline.SharedReadWrite import org.apache.spark.ml.feature.{HashingTF, MinMaxScaler} -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamPair} +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala index dc91fc5f9e458..35586320cb82b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.ann import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ - class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala index 04cc426c40b5e..f0c0183323c92 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.ann import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class GradientSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala index 98116656ba863..4db5f03fb00b4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.ml.classification import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.classification.ClassifierSuite.MockClassifier +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index 91a947f44bc31..089d30abb5ef9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.{CategoricalSplit, InternalNode, LeafNode} import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD @@ -44,17 +45,18 @@ class DecisionTreeClassifierSuite override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()).map(_.asML) orderedLabeledPointsWithLabel0RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()).map(_.asML) orderedLabeledPointsWithLabel1RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()).map(_.asML) categoricalDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()).map(_.asML) continuousDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()).map(_.asML) categoricalDataPointsForMulticlassForOrderedFeaturesRDD = sc.parallelize( OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures()) + .map(_.asML) } test("params") { @@ -395,7 +397,7 @@ private[ml] object DecisionTreeClassifierSuite extends SparkFunSuite { numClasses: Int): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures, numClasses) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index 5a5e5c15fc59c..8d588ccfd3545 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.classification import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -51,10 +52,13 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } test("params") { @@ -229,7 +233,7 @@ private object GBTClassifierSuite extends SparkFunSuite { val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) - val oldModel = oldGBT.run(data) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 2) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 69650ebb3690a..a1b48539c46e0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.ml.classification +import scala.collection.JavaConverters._ import scala.language.existentials import scala.util.Random +import scala.util.control.Breaks._ import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.{Instance, LabeledPoint} +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.lit @@ -967,4 +968,122 @@ object LogisticRegressionSuite { "standardization" -> false, "threshold" -> 0.6 ) + + def generateLogisticInputAsList( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + generateLogisticInput(offset, scale, nPoints, seed).asJava + } + + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val y = (0 until nPoints).map { i => + val p = 1.0 / (1.0 + math.exp(-(offset + scale * x1(i)))) + if (rnd.nextDouble() < p) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) + testData + } + + /** + * Generates `k` classes multinomial synthetic logistic input in `n` dimensional space given the + * model weights and mean/variance of the features. The synthetic data will be drawn from + * the probability distribution constructed by weights using the following formula. + * + * P(y = 0 | x) = 1 / norm + * P(y = 1 | x) = exp(x * w_1) / norm + * P(y = 2 | x) = exp(x * w_2) / norm + * ... + * P(y = k-1 | x) = exp(x * w_{k-1}) / norm + * where norm = 1 + exp(x * w_1) + exp(x * w_2) + ... + exp(x * w_{k-1}) + * + * @param weights matrix is flatten into a vector; as a result, the dimension of weights vector + * will be (k - 1) * (n + 1) if `addIntercept == true`, and + * if `addIntercept != true`, the dimension will be (k - 1) * n. + * @param xMean the mean of the generated features. Lots of time, if the features are not properly + * standardized, the algorithm with poor implementation will have difficulty + * to converge. + * @param xVariance the variance of the generated features. + * @param addIntercept whether to add intercept. + * @param nPoints the number of instance of generated data. + * @param seed the seed for random generator. For consistent testing result, it will be fixed. + */ + def generateMultinomialLogisticInput( + weights: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + addIntercept: Boolean, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + + val xDim = xMean.length + val xWithInterceptsDim = if (addIntercept) xDim + 1 else xDim + val nClasses = weights.length / xWithInterceptsDim + 1 + + val x = Array.fill[Vector](nPoints)(Vectors.dense(Array.fill[Double](xDim)(rnd.nextGaussian()))) + + x.foreach { vector => + // This doesn't work if `vector` is a sparse vector. + val vectorArray = vector.toArray + var i = 0 + val len = vectorArray.length + while (i < len) { + vectorArray(i) = vectorArray(i) * math.sqrt(xVariance(i)) + xMean(i) + i += 1 + } + } + + val y = (0 until nPoints).map { idx => + val xArray = x(idx).toArray + val margins = Array.ofDim[Double](nClasses) + val probs = Array.ofDim[Double](nClasses) + + for (i <- 0 until nClasses - 1) { + for (j <- 0 until xDim) margins(i + 1) += weights(i * xWithInterceptsDim + j) * xArray(j) + if (addIntercept) margins(i + 1) += weights((i + 1) * xWithInterceptsDim - 1) + } + // Preventing the overflow when we compute the probability + val maxMargin = margins.max + if (maxMargin > 0) for (i <- 0 until nClasses) margins(i) -= maxMargin + + // Computing the probabilities for each class from the margins. + val norm = { + var temp = 0.0 + for (i <- 0 until nClasses) { + probs(i) = math.exp(margins(i)) + temp += probs(i) + } + temp + } + for (i <- 0 until nClasses) probs(i) /= norm + + // Compute the cumulative probability so we can generate a random number and assign a label. + for (i <- 1 until nClasses) probs(i) += probs(i - 1) + val p = rnd.nextDouble() + var y = 0 + breakable { + for (i <- 0 until nClasses) { + if (p < probs(i)) { + y = i + break + } + } + } + y + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), x(i))) + testData + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index d5282e07d65c2..85f325f0765a4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -18,14 +18,16 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{Dataset, Row} class MultilayerPerceptronClassifierSuite @@ -134,12 +136,13 @@ class MultilayerPerceptronClassifierSuite .setNumClasses(numClasses) lr.optimizer.setRegParam(0.0) .setNumIterations(numIterations) - val lrModel = lr.run(rdd) - val lrPredictionAndLabels = lrModel.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val lrModel = lr.run(rdd.map(OldLabeledPoint.fromML)) + val lrPredictionAndLabels = + lrModel.predict(rdd.map(p => OldVectors.fromML(p.features))).zip(rdd.map(_.label)) // MLP's predictions should not differ a lot from LR's. val lrMetrics = new MulticlassMetrics(lrPredictionAndLabels) val mlpMetrics = new MulticlassMetrics(mlpPredictionAndLabels) - assert(mlpMetrics.confusionMatrix ~== lrMetrics.confusionMatrix absTol 100) + assert(mlpMetrics.confusionMatrix.asML ~== lrMetrics.confusionMatrix.asML absTol 100) } test("read/write: MultilayerPerceptronClassifier") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 2a05c446e5169..ff52115ec0ec1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.ml.classification -import breeze.linalg.{Vector => BV} +import scala.util.Random + +import breeze.linalg.{DenseVector => BDV, Vector => BV} +import breeze.stats.distributions.{Multinomial => BrzMultinomial} import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.NaiveBayesSuite._ +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.NaiveBayes.{Bernoulli, Multinomial} -import org.apache.spark.mllib.classification.NaiveBayesSuite._ -import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Dataset, Row} class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -206,4 +210,48 @@ object NaiveBayesSuite { "predictionCol" -> "myPrediction", "smoothing" -> 0.1 ) + + private def calcLabel(p: Double, pi: Array[Double]): Int = { + var sum = 0.0 + for (j <- 0 until pi.length) { + sum += pi(j) + if (p < sum) return j + } + -1 + } + + // Generate input of the form Y = (theta * x).argmax() + def generateNaiveBayesInput( + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int, + modelType: String = Multinomial, + sample: Int = 10): Seq[LabeledPoint] = { + val D = theta(0).length + val rnd = new Random(seed) + val _pi = pi.map(math.pow(math.E, _)) + val _theta = theta.map(row => row.map(math.pow(math.E, _))) + + for (i <- 0 until nPoints) yield { + val y = calcLabel(rnd.nextDouble(), _pi) + val xi = modelType match { + case Bernoulli => Array.tabulate[Double] (D) { j => + if (rnd.nextDouble () < _theta(y)(j) ) 1 else 0 + } + case Multinomial => + val mult = BrzMultinomial(BDV(_theta(y))) + val emptyMap = (0 until D).map(x => (x, 0.0)).toMap + val counts = emptyMap ++ mult.sample(sample).groupBy(x => x).map { + case (index, reps) => (index, reps.size.toDouble) + } + counts.toArray.sortBy(_._1).map(_._2) + case _ => + // This should never happen. + throw new UnknownError(s"Invalid modelType: $modelType.") + } + + LabeledPoint(y, Vectors.dense(xi)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 5044d40998d66..361dd74cb082e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -19,14 +19,16 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.feature.StringIndexer +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{ParamMap, ParamsSuite} import org.apache.spark.ml.util.{DefaultReadWriteTest, MetadataUtils, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD @@ -88,8 +90,8 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with Defau val lr = new LogisticRegressionWithLBFGS().setIntercept(true).setNumClasses(numClasses) lr.optimizer.setRegParam(0.1).setNumIterations(100) - val model = lr.run(rdd) - val results = model.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val model = lr.run(rdd.map(OldLabeledPoint.fromML)) + val results = model.predict(rdd.map(p => OldVectors.fromML(p.features))).zip(rdd.map(_.label)) // determine the #confusion matrix in each class. // bound how much error we allow compared to multinomial logistic regression. val expectedMetrics = new MulticlassMetrics(results) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala index cfa75ecf387cd..b3bd2b3e57b36 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} final class TestProbabilisticClassificationModel( override val uid: String, diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index 8002a2f4f29e1..2e99ee157ae95 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -46,8 +47,10 @@ class RandomForestClassifierSuite super.beforeAll() orderedLabeledPoints50_1000 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + .map(_.asML) orderedLabeledPoints5_20 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 5, 20)) + .map(_.asML) } ///////////////////////////////////////////////////////////////////////////// @@ -233,7 +236,8 @@ private object RandomForestClassifierSuite extends SparkFunSuite { val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, rf.getOldImpurity) val oldModel = OldRandomForest.trainClassifier( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + data.map(OldLabeledPoint.fromML), oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, + rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 2832db2f99156..88f31a1cd26fb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.clustering import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index 34e896428603c..ddfa87555427b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -20,10 +20,10 @@ package org.apache.spark.ml.clustering import org.apache.hadoop.fs.Path import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql._ diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala index a8766f9035241..9ee3df5eb5e33 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class BinaryClassificationEvaluatorSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala index dcc004358db16..42ff8adf6bd65 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala @@ -44,7 +44,7 @@ class RegressionEvaluatorSuite */ val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) /** * Using the following R code to load the data, train the model and evaluate metrics. diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala index e91f758112cf6..9cb84a6ee9b87 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala index 98b2316d78dbf..cd10c78311e1c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.ml.feature import scala.util.Random import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index 4fcc9745b738c..40d5b4881f839 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -18,12 +18,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{Row, SparkSession} class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala index b82e3e90b4f71..a59203c33d814 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index dbd5ae834536f..c02e9610418bf 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -22,8 +22,8 @@ import scala.beans.BeanInfo import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala index fc1c05de233ea..a4cca27be7815 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext class ElementwiseProductSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala index 89d67d8e6f93c..99b800776bb64 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala @@ -19,12 +19,12 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{HashingTF => MLlibHashingTF} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala index 208ea849136c3..09dc8b9b932fd 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{IDFModel => OldIDFModel} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class IDFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala index 340992800729c..3429172a8c903 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala @@ -21,9 +21,9 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.functions.col diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala index 73d69ebfeed08..d6400ee02f951 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala index e495c8e571344..5da84711758c6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row 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 241a1e9fb5c98..b692831714466 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 @@ -18,10 +18,10 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 06ffbc386feb5..7841b4fbc77a9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions.col diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala index 4befa84dbb758..ddb51fb1706a7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class PCASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -45,9 +46,9 @@ class PCASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead val dataRDD = sc.parallelize(data, 2) - val mat = new RowMatrix(dataRDD) + val mat = new RowMatrix(dataRDD.map(OldVectors.fromML)) val pc = mat.computePrincipalComponents(3) - val expected = mat.multiply(pc).rows + val expected = mat.multiply(pc).rows.map(_.asML) val df = spark.createDataFrame(dataRDD.zip(expected)).toDF("features", "expected") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala index e3adbba9d5dfd..8e1f9ddb36cbe 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.ml.feature import org.scalatest.exceptions.TestFailedException import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class PolynomialExpansionSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index c623a6210bdae..c12ab8fe9efe7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.types.DoubleType diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala index d62301be141b5..2243a0f972d32 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index 250011c859bc8..14973e79bf345 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row import org.apache.spark.sql.functions.col 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 d1c0270a02c09..707142332349c 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 @@ -22,9 +22,9 @@ import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -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 diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala index 88a077f9a1a5e..1746ce53107c4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.{StructField, StructType} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index 8cbe0f3defc4b..280a36f56e931 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{Word2VecModel => OldWord2VecModel} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 7b50876d33193..6ddb12cb76aac 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -17,19 +17,11 @@ package org.apache.spark.ml.linalg -import scala.beans.BeanInfo - import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.sql.catalyst.JavaTypeInference import org.apache.spark.sql.types._ -@BeanInfo -case class LabeledPoint(label: Double, features: Vector) { - override def toString: String = { - s"($label,$features)" - } -} - class VectorUDTSuite extends SparkFunSuite { test("preloaded VectorUDT") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala index 604021220a139..b30d995794d4c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala index 0b58a9821f57b..c8de796b2de87 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { 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 a3366c0e5934c..aa9c53ca30eee 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 @@ -17,11 +17,11 @@ package org.apache.spark.ml.param -import java.io.{ByteArrayOutputStream, NotSerializableException, ObjectOutputStream} +import java.io.{ByteArrayOutputStream, ObjectOutputStream} import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.MyParams -import org.apache.spark.mllib.linalg.{Vector, Vectors} class ParamsSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 9da0c32deede1..bbfc415cbb9b7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -31,11 +31,11 @@ import org.apache.commons.io.filefilter.TrueFileFilter import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.recommendation.ALS._ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} import org.apache.spark.sql.{DataFrame, Row, SparkSession} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index e4772df622d18..05aae80c660ea 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.ml.regression import scala.util.Random import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random.{ExponentialGenerator, WeibullGenerator} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class AFTSurvivalRegressionSuite diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index 2d30cbf36766b..9afb742406ec8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -38,7 +39,7 @@ class DecisionTreeRegressorSuite override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints().map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -170,7 +171,7 @@ private[ml] object DecisionTreeRegressorSuite extends SparkFunSuite { categoricalFeatures: Map[Int, Int]): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index ac833b833d7d8..7b5df8f31bb38 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -48,10 +49,13 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } test("Regression with continuous features") { @@ -197,7 +201,7 @@ private object GBTRegressorSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) - val oldModel = oldGBT.run(data) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index 3d9aeb8c0a2d0..a4568e83faca5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -20,15 +20,15 @@ package org.apache.spark.ml.regression import scala.util.Random import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LogisticRegressionSuite._ import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ -import org.apache.spark.mllib.linalg.{BLAS, DenseVector, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random._ -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala index bed4978b25b37..14d8a4e4e3345 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index a98227d2c14fb..332d331a47737 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -21,12 +21,12 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} class LinearRegressionSuite @@ -45,7 +45,7 @@ class LinearRegressionSuite datasetWithDenseFeature = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML)) /* datasetWithDenseFeatureWithoutIntercept is not needed for correctness testing but is useful for illustrating training model without intercept @@ -53,7 +53,7 @@ class LinearRegressionSuite datasetWithDenseFeatureWithoutIntercept = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( intercept = 0.0, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML)) val r = new Random(seed) // When feature size is larger than 4096, normal optimizer is choosed @@ -64,7 +64,7 @@ class LinearRegressionSuite intercept = 0.0, weights = Seq.fill(featureSize)(r.nextDouble()).toArray, xMean = Seq.fill(featureSize)(r.nextDouble()).toArray, xVariance = Seq.fill(featureSize)(r.nextDouble()).toArray, nPoints = 200, - seed, eps = 0.1, sparsity = 0.7), 2)) + seed, eps = 0.1, sparsity = 0.7), 2).map(_.asML)) /* R code: @@ -795,7 +795,7 @@ class LinearRegressionSuite Seq("auto", "l-bfgs", "normal").foreach { solver => val (data, weightedData) = { val activeData = LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1).map(_.asML) val rnd = new Random(8392) val signedData = activeData.map { case p: LabeledPoint => @@ -822,7 +822,7 @@ class LinearRegressionSuite } val noiseData = LinearDataGenerator.generateLinearInput( - 2, Array(1, 3), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) + 2, Array(1, 3), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1).map(_.asML) val weightedNoiseData = noiseData.map { case LabeledPoint(label, features) => Instance(label, weight = 0, features) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index 7a3a3698f950d..c08335f9f84af 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -40,7 +41,8 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex override def beforeAll() { super.beforeAll() orderedLabeledPoints50_1000 = - sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) + .map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -139,8 +141,8 @@ private object RandomForestRegressorSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, rf.getOldImpurity) - val oldModel = OldRandomForest.trainRegressor( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + val oldModel = OldRandomForest.trainRegressor(data.map(OldLabeledPoint.fromML), oldStrategy, + rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala index 7d0e01fd8fb9b..2517de59fed63 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets import com.google.common.io.Files import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{Row, SaveMode} import org.apache.spark.util.Utils diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala index de92b51eb0202..5c50a88c8314a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.mllib.tree.{GradientBoostedTreesSuite => OldGBTSuite} import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ @@ -35,8 +35,8 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext test("runWithValidation stops early and performs better on a validation dataset") { // Set numIterations large enough so that it stops early. val numIterations = 20 - val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2) - val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2) + val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2).map(_.asML) + val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2).map(_.asML) val trainDF = spark.createDataFrame(trainRdd) val validateDF = spark.createDataFrame(validateRdd) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 9739e6c05dcbc..dcc2f305df75a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -21,14 +21,14 @@ import scala.collection.mutable import org.apache.spark.SparkFunSuite import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.tree.{DecisionTreeSuite => OldDTSuite, EnsembleTestHelper} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, QuantileStrategy, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, GiniCalculator} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.collection.OpenHashMap /** @@ -43,7 +43,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { ///////////////////////////////////////////////////////////////////////////// test("Binary classification with continuous features: split calculation") { - val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1() + val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, 3, 2, 100) @@ -55,7 +55,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Binary classification with binary (ordered) categorical features: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -72,7 +72,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { test("Binary classification with 3-ary (ordered) categorical features," + " with no samples for one category: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -148,7 +148,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Multiclass classification with unordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPoints() + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy( @@ -189,7 +189,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Multiclass classification with ordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures().map(_.asML) assert(arr.length === 3000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 100, @@ -334,7 +334,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Second level node building with vs. without groups") { - val arr = OldDTSuite.generateOrderedLabeledPoints() + val arr = OldDTSuite.generateOrderedLabeledPoints().map(_.asML) assert(arr.length === 1000) val rdd = sc.parallelize(arr) // For tree with 1 group @@ -378,7 +378,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: OldStrategy) { val numFeatures = 50 val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) - val rdd = sc.parallelize(arr) + val rdd = sc.parallelize(arr).map(_.asML) // Select feature subset for top nodes. Return true if OK. def checkFeatureSubsetStrategy( diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala index 12ade4c92f85d..8cbd652bacf31 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala @@ -22,9 +22,9 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SparkSession} diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index 85df6da7a16a0..30bd390381e97 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -20,17 +20,17 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model, Pipeline} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} import org.apache.spark.ml.feature.HashingTF +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{ParamMap, ParamPair} import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.types.StructType class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -69,7 +69,7 @@ class CrossValidatorSuite test("cross validation with linear regression") { val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index f8d3de19b0f80..c1e9c2fc1dc11 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -20,15 +20,15 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.DefaultReadWriteTest -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.StructType class TrainValidationSplitSuite @@ -60,7 +60,7 @@ class TrainValidationSplitSuite test("train validation with linear regression") { val dataset = spark.createDataFrame( sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2).map(_.asML)) val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index ad7d2c9b8d40c..6aae625fc83f2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -20,9 +20,9 @@ package org.apache.spark.ml.util import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree.impl.TreeTests -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ 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 b7df02e6c098a..8c5b4bda2518d 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 @@ -560,4 +560,55 @@ class MatricesSuite extends SparkFunSuite { compare(oldSM0, newSM0) compare(oldDM0, newDM0) } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibMatrixToTriple(m: Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibDenseMatrixToTriple(m: DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibSparseMatrixToTriple(m: SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlMatrixToTriple(m: newlinalg.Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlDenseMatrixToTriple(m: newlinalg.DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlSparseMatrixToTriple(m: newlinalg.SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def compare(m1: (Array[Double], Int, Int), m2: (Array[Double], Int, Int)): Unit = { + assert(m1._1 === m2._1) + assert(m1._2 === m2._2) + assert(m1._3 === m2._3) + } + + val dm: DenseMatrix = new DenseMatrix(3, 2, Array(0.0, 0.0, 1.0, 0.0, 2.0, 3.5)) + val sm: SparseMatrix = dm.toSparse + val sm0: Matrix = sm.asInstanceOf[Matrix] + val dm0: Matrix = dm.asInstanceOf[Matrix] + + val newSM: newlinalg.SparseMatrix = sm.asML + val newDM: newlinalg.DenseMatrix = dm.asML + val newSM0: newlinalg.Matrix = sm0.asML + val newDM0: newlinalg.Matrix = dm0.asML + + import org.apache.spark.mllib.linalg.MatrixImplicits._ + + compare(mllibMatrixToTriple(dm0), mllibMatrixToTriple(newDM0)) + compare(mllibMatrixToTriple(sm0), mllibMatrixToTriple(newSM0)) + + compare(mllibDenseMatrixToTriple(dm), mllibDenseMatrixToTriple(newDM)) + compare(mllibSparseMatrixToTriple(sm), mllibSparseMatrixToTriple(newSM)) + + compare(mlMatrixToTriple(dm0), mlMatrixToTriple(newDM)) + compare(mlMatrixToTriple(sm0), mlMatrixToTriple(newSM0)) + + compare(mlDenseMatrixToTriple(dm), mlDenseMatrixToTriple(newDM)) + compare(mlSparseMatrixToTriple(sm), mlSparseMatrixToTriple(newSM)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index a7c1a076044ee..2e9c40ab88ed9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -422,4 +422,43 @@ class VectorsSuite extends SparkFunSuite with Logging { assert(oldSV0.toArray === newSV0.toArray) assert(oldDV0.toArray === newDV0.toArray) } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibVectorToArray(v: Vector): Array[Double] = v.toArray + + def mllibDenseVectorToArray(v: DenseVector): Array[Double] = v.toArray + + def mllibSparseVectorToArray(v: SparseVector): Array[Double] = v.toArray + + def mlVectorToArray(v: newlinalg.Vector): Array[Double] = v.toArray + + def mlDenseVectorToArray(v: newlinalg.DenseVector): Array[Double] = v.toArray + + def mlSparseVectorToArray(v: newlinalg.SparseVector): Array[Double] = v.toArray + + val dv: DenseVector = new DenseVector(Array(1.0, 2.0, 3.5)) + val sv: SparseVector = new SparseVector(5, Array(1, 2, 4), Array(1.1, 2.2, 4.4)) + val sv0: Vector = sv.asInstanceOf[Vector] + val dv0: Vector = dv.asInstanceOf[Vector] + + val newSV: newlinalg.SparseVector = sv.asML + val newDV: newlinalg.DenseVector = dv.asML + val newSV0: newlinalg.Vector = sv0.asML + val newDV0: newlinalg.Vector = dv0.asML + + import org.apache.spark.mllib.linalg.VectorImplicits._ + + assert(mllibVectorToArray(dv0) === mllibVectorToArray(newDV0)) + assert(mllibVectorToArray(sv0) === mllibVectorToArray(newSV0)) + + assert(mllibDenseVectorToArray(dv) === mllibDenseVectorToArray(newDV)) + assert(mllibSparseVectorToArray(sv) === mllibSparseVectorToArray(newSV)) + + assert(mlVectorToArray(dv0) === mlVectorToArray(newDV0)) + assert(mlVectorToArray(sv0) === mlVectorToArray(newSV0)) + + assert(mlDenseVectorToArray(dv) === mlDenseVectorToArray(newDV)) + assert(mlSparseVectorToArray(sv) === mlSparseVectorToArray(newSV)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index f8d0af8820e64..252a068dcd72f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors class LabeledPointSuite extends SparkFunSuite { @@ -40,4 +41,16 @@ class LabeledPointSuite extends SparkFunSuite { val point = LabeledPoint.parse("1.0,1.0 0.0 -2.0") assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.0))) } + + test("conversions between new ml LabeledPoint and mllib LabeledPoint") { + val points: Seq[LabeledPoint] = Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) + + val newPoints: Seq[NewLabeledPoint] = points.map(_.asML) + + points.zip(newPoints).foreach { case (p1, p2) => + assert(p1 === LabeledPoint.fromML(p2)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 49cb7e1f24e35..441d0f7614bf6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -73,7 +73,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -100,7 +100,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1 -> 2)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -116,7 +116,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -133,7 +133,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -150,7 +150,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -167,7 +167,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -183,7 +183,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(strategy.isMulticlassClassification) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -240,7 +240,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = maxBins, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -288,7 +288,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) @@ -310,7 +310,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1a02f660fddf2..45f7297048382 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -716,6 +716,52 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") + ) ++ Seq( + // [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.getOldDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.estimatedDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.topicsMatrix"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.clusterCenters"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.decodeLabel"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.encodeLabeledPoint"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.weights"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.predict"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.theta"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.pi"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.probability2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.getScalingVec"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.setScalingVec"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.PCAModel.pc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMax"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMin"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.IDFModel.idf"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.mean"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.std"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predictQuantiles"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.boundaries"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.this") ) ++ Seq( // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 5c11aa71b4599..a1c3f7298409c 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -53,7 +53,7 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti Currently, this class only supports binary classification. >>> from pyspark.sql import Row - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sc.parallelize([ ... Row(label=1.0, weight=2.0, features=Vectors.dense(1.0)), ... Row(label=0.0, weight=2.0, features=Vectors.sparse(1, [], []))]).toDF() @@ -496,7 +496,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred It supports both binary and multiclass labels, as well as both continuous and categorical features. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), @@ -625,7 +625,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> import numpy >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), @@ -752,7 +752,7 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol `SPARK-4240 `_ >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), @@ -884,7 +884,7 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H The input feature values must be nonnegative. >>> from pyspark.sql import Row - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... Row(label=0.0, features=Vectors.dense([0.0, 0.0])), ... Row(label=0.0, features=Vectors.dense([0.0, 1.0])), @@ -1028,7 +1028,7 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, Number of inputs has to be equal to the size of feature vectors. Number of outputs has to be equal to the total number of labels. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (0.0, Vectors.dense([0.0, 0.0])), ... (1.0, Vectors.dense([0.0, 1.0])), @@ -1193,7 +1193,7 @@ class OneVsRest(Estimator, OneVsRestParams, MLReadable, MLWritable): is picked to label the example. >>> from pyspark.sql import Row - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sc.parallelize([ ... Row(label=0.0, features=Vectors.dense(1.0, 0.8)), ... Row(label=1.0, features=Vectors.sparse(2, [], [])), diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 921633164bacb..ac7183d2efff5 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -65,7 +65,7 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte GaussianMixture clustering. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([-0.1, -0.05 ]),), ... (Vectors.dense([-0.01, -0.1]),), @@ -194,7 +194,7 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol K-means clustering with a k-means++ like initialization mode (the k-means|| algorithm by Bahmani et al). - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] >>> df = sqlContext.createDataFrame(data, ["features"]) @@ -347,7 +347,7 @@ class BisectingKMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte If bisecting all divisible clusters on the bottom level would result more than `k` leaf clusters, larger clusters get higher priority. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] >>> df = sqlContext.createDataFrame(data, ["features"]) @@ -625,7 +625,7 @@ class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInter :py:class:`pyspark.ml.feature.Tokenizer` and :py:class:`pyspark.ml.feature.CountVectorizer` can be useful for converting text to word count vectors. - >>> from pyspark.mllib.linalg import Vectors, SparseVector + >>> from pyspark.ml.linalg import Vectors, SparseVector >>> from pyspark.ml.clustering import LDA >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], ... [2, SparseVector(2, {0: 1.0})],], ["id", "features"]) diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 719c0c7d79fc9..fc9099b7ec172 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -111,7 +111,7 @@ class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPrediction The rawPrediction column can be of type double (binary 0/1 prediction, or probability of label 1) or of type vector (length-2 vector of raw predictions, scores, or label probabilities). - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> scoreAndLabels = map(lambda x: (Vectors.dense([1.0 - x[0], x[0]]), x[1]), ... [(0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)]) >>> dataset = sqlContext.createDataFrame(scoreAndLabels, ["raw", "label"]) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 606a6e7c22b4f..983b6a5301ae1 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -23,11 +23,11 @@ from pyspark import since, keyword_only from pyspark.rdd import ignore_unicode_prefix +from pyspark.ml.linalg import _convert_to_vector from pyspark.ml.param.shared import * from pyspark.ml.util import JavaMLReadable, JavaMLWritable from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer, _jvm from pyspark.mllib.common import inherit_doc -from pyspark.mllib.linalg import _convert_to_vector __all__ = ['Binarizer', 'Bucketizer', @@ -380,7 +380,7 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit .. seealso:: `More information on Wikipedia \ `_. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df1 = sqlContext.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) >>> dct = DCT(inverse=False, inputCol="vec", outputCol="resultVec") >>> df2 = dct.transform(df1) @@ -447,7 +447,7 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada with a provided "weight" vector. In other words, it scales each column of the dataset by a scalar multiplier. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([2.0, 1.0, 3.0]),)], ["values"]) >>> ep = ElementwiseProduct(scalingVec=Vectors.dense([1.0, 2.0, 3.0]), ... inputCol="values", outputCol="eprod") @@ -582,7 +582,7 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab Compute the Inverse Document Frequency (IDF) given a collection of documents. - >>> from pyspark.mllib.linalg import DenseVector + >>> from pyspark.ml.linalg import DenseVector >>> df = sqlContext.createDataFrame([(DenseVector([1.0, 2.0]),), ... (DenseVector([0.0, 1.0]),), (DenseVector([3.0, 0.2]),)], ["tf"]) >>> idf = IDF(minDocFreq=3, inputCol="tf", outputCol="idf") @@ -670,7 +670,7 @@ class MaxAbsScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav absolute value in each feature. It does not shift/center the data, and thus does not destroy any sparsity. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> maScaler = MaxAbsScaler(inputCol="a", outputCol="scaled") >>> model = maScaler.fit(df) @@ -757,7 +757,7 @@ class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav Note that since zero values will probably be transformed to non-zero values, output of the transformer will be DenseVector even for sparse input. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> mmScaler = MinMaxScaler(inputCol="a", outputCol="scaled") >>> model = mmScaler.fit(df) @@ -961,7 +961,7 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav Normalize a vector to have unit norm using the given p-norm. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> svec = Vectors.sparse(4, {1: 4.0, 3: 3.0}) >>> df = sqlContext.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) >>> normalizer = Normalizer(p=2.0, inputCol="dense", outputCol="features") @@ -1114,7 +1114,7 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead multiplication distributes over addition". Take a 2-variable feature vector as an example: `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) >>> px = PolynomialExpansion(degree=2, inputCol="dense", outputCol="expanded") >>> px.transform(df).head().expanded @@ -1459,7 +1459,7 @@ class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, J Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the training set. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") >>> model = standardScaler.fit(df) @@ -1942,7 +1942,7 @@ class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Ja - Add warning if a categorical feature has only 1 category. - Add option for allowing unknown categories. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([(Vectors.dense([-1.0, 0.0]),), ... (Vectors.dense([0.0, 1.0]),), (Vectors.dense([0.0, 2.0]),)], ["a"]) >>> indexer = VectorIndexer(maxCategories=2, inputCol="a", outputCol="indexed") @@ -2062,7 +2062,7 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J The output vector will order features with the selected indices first (in the order given), followed by the selected names (in the order given). - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (Vectors.dense([-2.0, 2.3, 0.0, 0.0, 1.0]),), ... (Vectors.dense([0.0, 0.0, 0.0, 0.0, 0.0]),), @@ -2329,7 +2329,7 @@ class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab PCA trains a model to project vectors to a low-dimensional space using PCA. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), ... (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), ... (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] @@ -2547,7 +2547,7 @@ class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, Ja Chi-Squared feature selection, which selects categorical features to use for predicting a categorical label. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame( ... [(Vectors.dense([0.0, 0.0, 18.0, 1.0]), 1.0), ... (Vectors.dense([0.0, 1.0, 12.0, 0.0]), 0.0), diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index d9513ca5b273d..ade4864e1d785 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -29,8 +29,8 @@ from py4j.java_gateway import JavaObject from pyspark import since +from pyspark.ml.linalg import DenseVector, Vector from pyspark.ml.util import Identifiable -from pyspark.mllib.linalg import DenseVector, Vector __all__ = ['Param', 'Params', 'TypeConverters'] diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index fcdc29e69b1d0..cfcbbfc98e821 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -54,7 +54,7 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction - L1 (Lasso) - L2 + L1 (elastic net) - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, 2.0, Vectors.dense(1.0)), ... (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) @@ -412,7 +412,7 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti Currently implemented using parallelized pool adjacent violators algorithm. Only univariate (single feature) algorithm supported. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -642,7 +642,7 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi learning algorithm for regression. It supports both continuous and categorical features. - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -808,7 +808,7 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi It supports both continuous and categorical features. >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -920,7 +920,7 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, It supports both continuous and categorical features. >>> from numpy import allclose - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) @@ -1055,7 +1055,7 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi .. seealso:: `AFT Model `_ - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(1.0), 1.0), ... (0.0, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) @@ -1252,7 +1252,7 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha .. seealso:: `GLM `_ - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> df = sqlContext.createDataFrame([ ... (1.0, Vectors.dense(0.0, 0.0)), ... (1.0, Vectors.dense(1.0, 2.0)), diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index c5679057597e2..e3511120bdecb 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -62,10 +62,6 @@ from pyspark.ml.tuning import * from pyspark.ml.wrapper import JavaParams from pyspark.mllib.common import _java2py -from pyspark.mllib.linalg import SparseVector as OldSparseVector, DenseVector as OldDenseVector,\ - DenseMatrix as OldDenseMatrix, MatrixUDT as OldMatrixUDT, SparseMatrix as OldSparseMatrix,\ - Vectors as OldVectors, VectorUDT as OldVectorUDT -from pyspark.mllib.regression import LabeledPoint from pyspark.serializers import PickleSerializer from pyspark.sql import DataFrame, Row, SparkSession from pyspark.sql.functions import rand @@ -162,22 +158,22 @@ def test_float(self): def test_vector(self): ewp = ElementwiseProduct(scalingVec=[1, 3]) - self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.0, 3.0])) + self.assertEqual(ewp.getScalingVec(), DenseVector([1.0, 3.0])) ewp = ElementwiseProduct(scalingVec=np.array([1.2, 3.4])) - self.assertEqual(ewp.getScalingVec(), OldDenseVector([1.2, 3.4])) + self.assertEqual(ewp.getScalingVec(), DenseVector([1.2, 3.4])) self.assertRaises(TypeError, lambda: ElementwiseProduct(scalingVec=["a", "b"])) def test_list(self): l = [0, 1] - for lst_like in [l, np.array(l), OldDenseVector(l), OldSparseVector(len(l), + for lst_like in [l, np.array(l), DenseVector(l), SparseVector(len(l), range(len(l)), l), pyarray.array('l', l), xrange(2), tuple(l)]: converted = TypeConverters.toList(lst_like) self.assertEqual(type(converted), list) self.assertListEqual(converted, l) def test_list_int(self): - for indices in [[1.0, 2.0], np.array([1.0, 2.0]), OldDenseVector([1.0, 2.0]), - OldSparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), + for indices in [[1.0, 2.0], np.array([1.0, 2.0]), DenseVector([1.0, 2.0]), + SparseVector(2, {0: 1.0, 1: 2.0}), xrange(1, 3), (1.0, 2.0), pyarray.array('d', [1.0, 2.0])]: vs = VectorSlicer(indices=indices) self.assertListEqual(vs.getIndices(), [1, 2]) @@ -410,9 +406,9 @@ def test_binarizer(self): def test_idf(self): dataset = self.spark.createDataFrame([ - (OldDenseVector([1.0, 2.0]),), - (OldDenseVector([0.0, 1.0]),), - (OldDenseVector([3.0, 0.2]),)], ["tf"]) + (DenseVector([1.0, 2.0]),), + (DenseVector([0.0, 1.0]),), + (DenseVector([3.0, 0.2]),)], ["tf"]) idf0 = IDF(inputCol="tf") self.assertListEqual(idf0.params, [idf0.inputCol, idf0.minDocFreq, idf0.outputCol]) idf0m = idf0.fit(dataset, {idf0.outputCol: "idf"}) @@ -457,10 +453,10 @@ def test_stopwordsremover(self): def test_count_vectorizer_with_binary(self): dataset = self.spark.createDataFrame([ - (0, "a a a b b c".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), - (1, "a a".split(' '), OldSparseVector(3, {0: 1.0}),), - (2, "a b".split(' '), OldSparseVector(3, {0: 1.0, 1: 1.0}),), - (3, "c".split(' '), OldSparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) + (0, "a a a b b c".split(' '), SparseVector(3, {0: 1.0, 1: 1.0, 2: 1.0}),), + (1, "a a".split(' '), SparseVector(3, {0: 1.0}),), + (2, "a b".split(' '), SparseVector(3, {0: 1.0, 1: 1.0}),), + (3, "c".split(' '), SparseVector(3, {2: 1.0}),)], ["id", "words", "expected"]) cv = CountVectorizer(binary=True, inputCol="words", outputCol="features") model = cv.fit(dataset) @@ -581,11 +577,11 @@ def test_save_load(self): # Save/load for CrossValidator will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(OldVectors.dense([0.0]), 0.0), - (OldVectors.dense([0.4]), 1.0), - (OldVectors.dense([0.5]), 0.0), - (OldVectors.dense([0.6]), 1.0), - (OldVectors.dense([1.0]), 1.0)] * 10, + [(Vectors.dense([0.0]), 0.0), + (Vectors.dense([0.4]), 1.0), + (Vectors.dense([0.5]), 0.0), + (Vectors.dense([0.6]), 1.0), + (Vectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -654,11 +650,11 @@ def test_save_load(self): # Save/load for TrainValidationSplit will be added later: SPARK-13786 temp_path = tempfile.mkdtemp() dataset = self.spark.createDataFrame( - [(OldVectors.dense([0.0]), 0.0), - (OldVectors.dense([0.4]), 1.0), - (OldVectors.dense([0.5]), 0.0), - (OldVectors.dense([0.6]), 1.0), - (OldVectors.dense([1.0]), 1.0)] * 10, + [(Vectors.dense([0.0]), 0.0), + (Vectors.dense([0.4]), 1.0), + (Vectors.dense([0.5]), 0.0), + (Vectors.dense([0.6]), 1.0), + (Vectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() @@ -857,8 +853,8 @@ def _compare(self, m1, m2): def test_persistence(self): # Test save/load for LDA, LocalLDAModel, DistributedLDAModel. df = self.spark.createDataFrame([ - [1, OldVectors.dense([0.0, 1.0])], - [2, OldVectors.sparse(2, {0: 1.0})], + [1, Vectors.dense([0.0, 1.0])], + [2, Vectors.sparse(2, {0: 1.0})], ], ["id", "features"]) # Fit model lda = LDA(k=2, seed=1, optimizer="em") @@ -893,8 +889,8 @@ def test_persistence(self): class TrainingSummaryTest(SparkSessionTestCase): def test_linear_regression_summary(self): - df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), - (0.0, 2.0, OldVectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight", fitIntercept=False) @@ -930,7 +926,7 @@ def test_linear_regression_summary(self): self.assertAlmostEqual(sameSummary.explainedVariance, s.explainedVariance) def test_glr_summary(self): - from pyspark.mllib.linalg import Vectors + from pyspark.ml.linalg import Vectors df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) @@ -966,8 +962,8 @@ def test_glr_summary(self): self.assertAlmostEqual(sameSummary.deviance, s.deviance) def test_logistic_regression_summary(self): - df = self.spark.createDataFrame([(1.0, 2.0, OldVectors.dense(1.0)), - (0.0, 2.0, OldVectors.sparse(1, [], []))], + df = self.spark.createDataFrame([(1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) model = lr.fit(df) @@ -996,9 +992,9 @@ def test_logistic_regression_summary(self): class OneVsRestTests(SparkSessionTestCase): def test_copy(self): - df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), - (1.0, OldVectors.sparse(2, [], [])), - (2.0, OldVectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1010,9 +1006,9 @@ def test_copy(self): self.assertEqual(model1.getPredictionCol(), "indexed") def test_output_columns(self): - df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), - (1.0, OldVectors.sparse(2, [], [])), - (2.0, OldVectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1022,9 +1018,9 @@ def test_output_columns(self): def test_save_load(self): temp_path = tempfile.mkdtemp() - df = self.spark.createDataFrame([(0.0, OldVectors.dense(1.0, 0.8)), - (1.0, OldVectors.sparse(2, [], [])), - (2.0, OldVectors.dense(0.5, 0.5))], + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], ["label", "features"]) lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) @@ -1052,7 +1048,7 @@ def test_apply_binary_term_freqs(self): hashingTF.setInputCol("words").setOutputCol("features").setNumFeatures(n).setBinary(True) output = hashingTF.transform(df) features = output.select("features").first().features.toArray() - expected = OldVectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() + expected = Vectors.dense([1.0, 1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0]).toArray() for i in range(0, n): self.assertAlmostEqual(features[i], expected[i], 14, "Error at " + str(i) + ": expected " + str(expected[i]) + ", got " + str(features[i])) @@ -1147,15 +1143,13 @@ def _test_serialize(self, v): self.assertEqual(vs, nvs) def test_serialize(self): - # Because pickle path still uses old vector/matrix - # TODO: Change this to new vector/matrix when pickle for new vector/matrix is ready. - self._test_serialize(OldDenseVector(range(10))) - self._test_serialize(OldDenseVector(array([1., 2., 3., 4.]))) - self._test_serialize(OldDenseVector(pyarray.array('d', range(10)))) - self._test_serialize(OldSparseVector(4, {1: 1, 3: 2})) - self._test_serialize(OldSparseVector(3, {})) - self._test_serialize(OldDenseMatrix(2, 3, range(6))) - sm1 = OldSparseMatrix( + self._test_serialize(DenseVector(range(10))) + self._test_serialize(DenseVector(array([1., 2., 3., 4.]))) + self._test_serialize(DenseVector(pyarray.array('d', range(10)))) + self._test_serialize(SparseVector(4, {1: 1, 3: 2})) + self._test_serialize(SparseVector(3, {})) + self._test_serialize(DenseMatrix(2, 3, range(6))) + sm1 = SparseMatrix( 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) self._test_serialize(sm1) @@ -1407,12 +1401,6 @@ class VectorUDTTests(MLlibTestCase): sv1 = SparseVector(2, [1], [2.0]) udt = VectorUDT() - old_dv0 = OldDenseVector([]) - old_dv1 = OldDenseVector([1.0, 2.0]) - old_sv0 = OldSparseVector(2, [], []) - old_sv1 = OldSparseVector(2, [1], [2.0]) - old_udt = OldVectorUDT() - def test_json_schema(self): self.assertEqual(VectorUDT.fromJson(self.udt.jsonValue()), self.udt) @@ -1421,19 +1409,19 @@ def test_serialization(self): self.assertEqual(v, self.udt.deserialize(self.udt.serialize(v))) def test_infer_schema(self): - rdd = self.sc.parallelize([LabeledPoint(1.0, self.old_dv1), - LabeledPoint(0.0, self.old_sv1)]) + rdd = self.sc.parallelize([Row(label=1.0, features=self.dv1), + Row(label=0.0, features=self.sv1)]) df = rdd.toDF() schema = df.schema field = [f for f in schema.fields if f.name == "features"][0] - self.assertEqual(field.dataType, self.old_udt) + self.assertEqual(field.dataType, self.udt) vectors = df.rdd.map(lambda p: p.features).collect() self.assertEqual(len(vectors), 2) for v in vectors: - if isinstance(v, OldSparseVector): - self.assertEqual(v, self.old_sv1) - elif isinstance(v, OldDenseVector): - self.assertEqual(v, self.old_dv1) + if isinstance(v, SparseVector): + self.assertEqual(v, self.sv1) + elif isinstance(v, DenseVector): + self.assertEqual(v, self.dv1) else: raise TypeError("expecting a vector but got %r of type %r" % (v, type(v))) @@ -1446,12 +1434,6 @@ class MatrixUDTTests(MLlibTestCase): sm2 = SparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) udt = MatrixUDT() - old_dm1 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10]) - old_dm2 = OldDenseMatrix(3, 2, [0, 1, 4, 5, 9, 10], isTransposed=True) - old_sm1 = OldSparseMatrix(1, 1, [0, 1], [0], [2.0]) - old_sm2 = OldSparseMatrix(2, 1, [0, 0, 1], [0], [5.0], isTransposed=True) - old_udt = OldMatrixUDT() - def test_json_schema(self): self.assertEqual(MatrixUDT.fromJson(self.udt.jsonValue()), self.udt) @@ -1460,17 +1442,17 @@ def test_serialization(self): self.assertEqual(m, self.udt.deserialize(self.udt.serialize(m))) def test_infer_schema(self): - rdd = self.sc.parallelize([("dense", self.old_dm1), ("sparse", self.old_sm1)]) + rdd = self.sc.parallelize([("dense", self.dm1), ("sparse", self.sm1)]) df = rdd.toDF() schema = df.schema - self.assertTrue(schema.fields[1].dataType, self.old_udt) + self.assertTrue(schema.fields[1].dataType, self.udt) matrices = df.rdd.map(lambda x: x._2).collect() self.assertEqual(len(matrices), 2) for m in matrices: - if isinstance(m, OldDenseMatrix): - self.assertTrue(m, self.old_dm1) - elif isinstance(m, OldSparseMatrix): - self.assertTrue(m, self.old_sm1) + if isinstance(m, DenseMatrix): + self.assertTrue(m, self.dm1) + elif isinstance(m, SparseMatrix): + self.assertTrue(m, self.sm1) else: raise ValueError("Expected a matrix but got type %r" % type(m)) diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 0920ae6ea144b..75789c4d09184 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -151,7 +151,7 @@ class CrossValidator(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> dataset = sqlContext.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), @@ -310,7 +310,7 @@ class TrainValidationSplit(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator - >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.ml.linalg import Vectors >>> dataset = sqlContext.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), From c0bb77132b9acac951074fd623892abafeb02512 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 May 2016 13:05:07 -0700 Subject: [PATCH 0242/1470] [SPARK-15244] [PYTHON] Type of column name created with createDataFrame is not consistent. ## What changes were proposed in this pull request? **createDataFrame** returns inconsistent types for column names. ```python >>> from pyspark.sql.types import StructType, StructField, StringType >>> schema = StructType([StructField(u"col", StringType())]) >>> df1 = spark.createDataFrame([("a",)], schema) >>> df1.columns # "col" is str ['col'] >>> df2 = spark.createDataFrame([("a",)], [u"col"]) >>> df2.columns # "col" is unicode [u'col'] ``` The reason is only **StructField** has the following code. ``` if not isinstance(name, str): name = name.encode('utf-8') ``` This PR adds the same logic into **createDataFrame** for consistency. ``` if isinstance(schema, list): schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] ``` ## How was this patch tested? Pass the Jenkins test (with new python doctest) Author: Dongjoon Hyun Closes #13097 from dongjoon-hyun/SPARK-15244. (cherry picked from commit 0f576a5748244f7e874b925f8d841f1ca238f087) Signed-off-by: Davies Liu --- python/pyspark/sql/session.py | 2 ++ python/pyspark/sql/tests.py | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index ae314359d512a..0781b442cbcd8 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -465,6 +465,8 @@ def prepare(obj): return (obj, ) schema = StructType().add("value", datatype) else: + if isinstance(schema, list): + schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] prepare = lambda obj: obj if isinstance(data, RDD): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 0c73f58c3b246..0977c43a398da 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -228,6 +228,13 @@ def test_duplicated_column_names(self): self.assertRaises(AnalysisException, lambda: df.select(df.c).first()) self.assertRaises(AnalysisException, lambda: df.select(df["c"]).first()) + def test_column_name_encoding(self): + """Ensure that created columns has `str` type consistently.""" + columns = self.spark.createDataFrame([('Alice', 1)], ['name', u'age']).columns + self.assertEqual(columns, ['name', 'age']) + self.assertTrue(isinstance(columns[0], str)) + self.assertTrue(isinstance(columns[1], str)) + def test_explode(self): from pyspark.sql.functions import explode d = [Row(a=1, intlist=[1, 2, 3], mapfield={"a": "b"})] From 7b62b7c1180dc3414b7d7e07561d0e6b89ff2b37 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 17 May 2016 14:57:21 -0700 Subject: [PATCH 0243/1470] [SPARK-11735][CORE][SQL] Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped ## What changes were proposed in this pull request? Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13154 from zsxwing/check-spark-context-stop. (cherry picked from commit 8e8bc9f957de6c0aefbc6ef4b18c421b486477a6) Signed-off-by: Yin Huai --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 ++ sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e391599336074..e6cdd0d298f37 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -94,7 +94,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] val stopped: AtomicBoolean = new AtomicBoolean(false) - private def assertNotStopped(): Unit = { + private[spark] def assertNotStopped(): Unit = { if (stopped.get()) { val activeContext = SparkContext.activeContext.get() val activeCreationSite = 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 c64e284635ec6..44511885a7db6 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 @@ -67,6 +67,8 @@ class SQLContext private[sql]( self => + sparkSession.sparkContext.assertNotStopped() + // Note: Since Spark 2.0 this class has become a wrapper of SparkSession, where the // real functionality resides. This class remains mainly for backward compatibility. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 100b43f6b58f2..aa974f29cc34d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -68,6 +68,7 @@ class SparkSession private( this(sc, None) } + sparkContext.assertNotStopped() /* ----------------------- * | Session-related state | From 2dddec40d6562d1d16bb242bf7dc730431ee1e3e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 17 May 2016 15:56:44 -0700 Subject: [PATCH 0244/1470] [SPARK-14346][SQL] Native SHOW CREATE TABLE for Hive tables/views ## What changes were proposed in this pull request? This is a follow-up of #12781. It adds native `SHOW CREATE TABLE` support for Hive tables and views. A new field `hasUnsupportedFeatures` is added to `CatalogTable` to indicate whether all table metadata retrieved from the concrete underlying external catalog (i.e. Hive metastore in this case) can be mapped to fields in `CatalogTable`. This flag is useful when the target Hive table contains structures that can't be handled by Spark SQL, e.g., skewed columns and storage handler, etc.. ## How was this patch tested? New test cases are added in `ShowCreateTableSuite` to do round-trip tests. Author: Cheng Lian Closes #13079 from liancheng/spark-14346-show-create-table-for-hive-tables. (cherry picked from commit b674e67c22bf663334e537e35787c00533adbb04) Signed-off-by: Yin Huai --- .../sql/catalyst/catalog/interface.scala | 12 +- .../spark/sql/execution/command/tables.scala | 184 ++++++++++++----- .../sql/hive/client/HiveClientImpl.scala | 10 +- .../spark/sql/hive/ShowCreateTableSuite.scala | 185 +++++++++++++++++- 4 files changed, 333 insertions(+), 58 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index d21565526ec0f..d4f5cbb625850 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -79,6 +79,12 @@ case class CatalogTablePartition( * * Note that Hive's metastore also tracks skewed columns. We should consider adding that in the * future once we have a better understanding of how we want to handle skewed columns. + * + * @param hasUnsupportedFeatures is used to indicate whether all table metadata entries retrieved + * from the concrete underlying external catalog (e.g. Hive metastore) are supported by + * Spark SQL. For example, if the underlying Hive table has skewed columns, this information + * can't be mapped to [[CatalogTable]] since Spark SQL doesn't handle skewed columns for now. + * In this case `hasUnsupportedFeatures` is set to true. By default, it is false. */ case class CatalogTable( identifier: TableIdentifier, @@ -95,7 +101,8 @@ case class CatalogTable( properties: Map[String, String] = Map.empty, viewOriginalText: Option[String] = None, viewText: Option[String] = None, - comment: Option[String] = None) { + comment: Option[String] = None, + hasUnsupportedFeatures: Boolean = false) { // Verify that the provided columns are part of the schema private val colNames = schema.map(_.name).toSet @@ -200,6 +207,7 @@ case class SimpleCatalogRelation( } } - require(metadata.identifier.database == Some(databaseName), + require( + metadata.identifier.database.contains(databaseName), "provided database does not match the one specified in the table definition") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index bb4f1ff4f75f2..1fc02d1d4b469 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -626,40 +626,149 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { showCreateDataSourceTable(tableMetadata) } else { - throw new UnsupportedOperationException( - "SHOW CREATE TABLE only supports Spark SQL data source tables.") + showCreateHiveTable(tableMetadata) } Seq(Row(stmt)) } + private def showCreateHiveTable(metadata: CatalogTable): String = { + def reportUnsupportedError(): Unit = { + throw new UnsupportedOperationException( + s"Failed to execute SHOW CREATE TABLE against table ${metadata.identifier.quotedString}, " + + "because it contains table structure(s) (e.g. skewed columns) that Spark SQL doesn't " + + "support yet." + ) + } + + if (metadata.hasUnsupportedFeatures) { + reportUnsupportedError() + } + + val builder = StringBuilder.newBuilder + + val tableTypeString = metadata.tableType match { + case EXTERNAL => " EXTERNAL TABLE" + case VIEW => " VIEW" + case MANAGED => " TABLE" + case INDEX => reportUnsupportedError() + } + + builder ++= s"CREATE$tableTypeString ${table.quotedString}" + + if (metadata.tableType == VIEW) { + if (metadata.schema.nonEmpty) { + builder ++= metadata.schema.map(_.name).mkString("(", ", ", ")") + } + builder ++= metadata.viewText.mkString(" AS\n", "", "\n") + } else { + showHiveTableHeader(metadata, builder) + showHiveTableNonDataColumns(metadata, builder) + showHiveTableStorageInfo(metadata, builder) + showHiveTableProperties(metadata, builder) + } + + builder.toString() + } + + private def showHiveTableHeader(metadata: CatalogTable, builder: StringBuilder): Unit = { + val columns = metadata.schema.filterNot { column => + metadata.partitionColumnNames.contains(column.name) + }.map(columnToDDLFragment) + + if (columns.nonEmpty) { + builder ++= columns.mkString("(", ", ", ")\n") + } + + metadata + .comment + .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") + .foreach(builder.append) + } + + private def columnToDDLFragment(column: CatalogColumn): String = { + val comment = column.comment.map(escapeSingleQuotedString).map(" COMMENT '" + _ + "'") + s"${quoteIdentifier(column.name)} ${column.dataType}${comment.getOrElse("")}" + } + + private def showHiveTableNonDataColumns(metadata: CatalogTable, builder: StringBuilder): Unit = { + if (metadata.partitionColumns.nonEmpty) { + val partCols = metadata.partitionColumns.map(columnToDDLFragment) + builder ++= partCols.mkString("PARTITIONED BY (", ", ", ")\n") + } + + if (metadata.bucketColumnNames.nonEmpty) { + throw new UnsupportedOperationException( + "Creating Hive table with bucket spec is not supported yet.") + } + } + + private def showHiveTableStorageInfo(metadata: CatalogTable, builder: StringBuilder): Unit = { + val storage = metadata.storage + + storage.serde.foreach { serde => + builder ++= s"ROW FORMAT SERDE '$serde'\n" + + val serdeProps = metadata.storage.serdeProperties.map { + case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + + builder ++= serdeProps.mkString("WITH SERDEPROPERTIES (", ",\n ", "\n)\n") + } + + if (storage.inputFormat.isDefined || storage.outputFormat.isDefined) { + builder ++= "STORED AS\n" + + storage.inputFormat.foreach { format => + builder ++= s" INPUTFORMAT '${escapeSingleQuotedString(format)}'\n" + } + + storage.outputFormat.foreach { format => + builder ++= s" OUTPUTFORMAT '${escapeSingleQuotedString(format)}'\n" + } + } + + if (metadata.tableType == EXTERNAL) { + storage.locationUri.foreach { uri => + builder ++= s"LOCATION '$uri'\n" + } + } + } + + private def showHiveTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { + if (metadata.properties.nonEmpty) { + val filteredProps = metadata.properties.filterNot { + // Skips "EXTERNAL" property for external tables + case (key, _) => key == "EXTERNAL" && metadata.tableType == EXTERNAL + } + + val props = filteredProps.map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + + if (props.nonEmpty) { + builder ++= props.mkString("TBLPROPERTIES (", ",\n ", ")\n") + } + } + } + private def showCreateDataSourceTable(metadata: CatalogTable): String = { val builder = StringBuilder.newBuilder builder ++= s"CREATE TABLE ${table.quotedString} " - showDataSourceTableDataCols(metadata, builder) + showDataSourceTableDataColumns(metadata, builder) showDataSourceTableOptions(metadata, builder) showDataSourceTableNonDataColumns(metadata, builder) builder.toString() } - private def showDataSourceTableDataCols(metadata: CatalogTable, builder: StringBuilder): Unit = { - val props = metadata.properties - val schemaParts = for { - numParts <- props.get("spark.sql.sources.schema.numParts").toSeq - index <- 0 until numParts.toInt - } yield props.getOrElse( - s"spark.sql.sources.schema.part.$index", - throw new AnalysisException( - s"Corrupted schema in catalog: $numParts parts expected, but part $index is missing." - ) - ) - - if (schemaParts.nonEmpty) { - val fields = DataType.fromJson(schemaParts.mkString).asInstanceOf[StructType].fields - val colTypeList = fields.map(f => s"${quoteIdentifier(f.name)} ${f.dataType.sql}") - builder ++= colTypeList.mkString("(", ", ", ")") + private def showDataSourceTableDataColumns( + metadata: CatalogTable, builder: StringBuilder): Unit = { + DDLUtils.getSchemaFromTableProperties(metadata).foreach { schema => + val columns = schema.fields.map(f => s"${quoteIdentifier(f.name)} ${f.dataType.sql}") + builder ++= columns.mkString("(", ", ", ")") } builder ++= "\n" @@ -688,40 +797,21 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman private def showDataSourceTableNonDataColumns( metadata: CatalogTable, builder: StringBuilder): Unit = { - val props = metadata.properties - - def getColumnNamesByType(colType: String, typeName: String): Seq[String] = { - (for { - numCols <- props.get(s"spark.sql.sources.schema.num${colType.capitalize}Cols").toSeq - index <- 0 until numCols.toInt - } yield props.getOrElse( - s"spark.sql.sources.schema.${colType}Col.$index", - throw new AnalysisException( - s"Corrupted $typeName in catalog: $numCols parts expected, but part $index is missing." - ) - )).map(quoteIdentifier) - } - - val partCols = getColumnNamesByType("part", "partitioning columns") + val partCols = DDLUtils.getPartitionColumnsFromTableProperties(metadata) if (partCols.nonEmpty) { builder ++= s"PARTITIONED BY ${partCols.mkString("(", ", ", ")")}\n" } - val bucketCols = getColumnNamesByType("bucket", "bucketing columns") - if (bucketCols.nonEmpty) { - builder ++= s"CLUSTERED BY ${bucketCols.mkString("(", ", ", ")")}\n" - - val sortCols = getColumnNamesByType("sort", "sorting columns") - if (sortCols.nonEmpty) { - builder ++= s"SORTED BY ${sortCols.mkString("(", ", ", ")")}\n" - } + DDLUtils.getBucketSpecFromTableProperties(metadata).foreach { spec => + if (spec.bucketColumnNames.nonEmpty) { + builder ++= s"CLUSTERED BY ${spec.bucketColumnNames.mkString("(", ", ", ")")}\n" - val numBuckets = props.getOrElse( - "spark.sql.sources.schema.numBuckets", - throw new AnalysisException("Corrupted bucket spec in catalog: missing bucket number") - ) + if (spec.sortColumnNames.nonEmpty) { + builder ++= s"SORTED BY ${spec.sortColumnNames.mkString("(", ", ", ")")}\n" + } - builder ++= s"INTO $numBuckets BUCKETS\n" + builder ++= s"INTO ${spec.numBuckets} BUCKETS\n" + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 78c457b6c2af3..a4e9f03b43342 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -339,6 +339,13 @@ private[hive] class HiveClientImpl( // partition columns are part of the schema val partCols = h.getPartCols.asScala.map(fromHiveColumn) val schema = h.getCols.asScala.map(fromHiveColumn) ++ partCols + + // Skew spec, storage handler, and bucketing info can't be mapped to CatalogTable (yet) + val hasUnsupportedFeatures = + !h.getSkewedColNames.isEmpty || + h.getStorageHandler != null || + !h.getBucketCols.isEmpty + CatalogTable( identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), tableType = h.getTableType match { @@ -365,7 +372,8 @@ private[hive] class HiveClientImpl( ), properties = h.getParameters.asScala.toMap, viewOriginalText = Option(h.getViewOriginalText), - viewText = Option(h.getViewExpandedText)) + viewText = Option(h.getViewExpandedText), + hasUnsupportedFeatures = hasUnsupportedFeatures) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala index 12a1ad8987c17..3b8068d3bc478 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -116,24 +116,177 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing .bucketBy(2, "c", "d") .saveAsTable("ddl_test5") - checkCreateTable(TableIdentifier("ddl_test5", Some("default"))) + checkCreateTable("ddl_test5") } } + test("simple hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("simple external hive table") { + withTempDir { dir => + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |LOCATION '$dir' + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + } + + test("partitioned hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |COMMENT 'bla' + |PARTITIONED BY ( + | p1 BIGINT COMMENT 'bla', + | p2 STRING + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with explicit storage info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |COLLECTION ITEMS TERMINATED BY '@' + |MAP KEYS TERMINATED BY '#' + |NULL DEFINED AS 'NaN' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with STORED AS clause") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |STORED AS PARQUET + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with serde info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |WITH SERDEPROPERTIES ( + | 'mapkey.delim' = ',', + | 'field.delim' = ',' + |) + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive view") { + withView("v1") { + sql("CREATE VIEW v1 AS SELECT 1 AS a") + checkCreateView("v1") + } + } + + test("hive view with output columns") { + withView("v1") { + sql("CREATE VIEW v1 (b) AS SELECT 1 AS a") + checkCreateView("v1") + } + } + + test("hive bucketing not supported") { + withTable("t1") { + createRawHiveTable( + s"""CREATE TABLE t1 (a INT, b STRING) + |CLUSTERED BY (a) + |SORTED BY (b) + |INTO 2 BUCKETS + """.stripMargin + ) + + intercept[UnsupportedOperationException] { + sql("SHOW CREATE TABLE t1") + } + } + } + + private def createRawHiveTable(ddl: String): Unit = { + hiveContext.sharedState.metadataHive.runSqlHive(ddl) + } + private def checkCreateTable(table: String): Unit = { - checkCreateTable(TableIdentifier(table, Some("default"))) + checkCreateTableOrView(TableIdentifier(table, Some("default")), "TABLE") + } + + private def checkCreateView(table: String): Unit = { + checkCreateTableOrView(TableIdentifier(table, Some("default")), "VIEW") } - private def checkCreateTable(table: TableIdentifier): Unit = { + private def checkCreateTableOrView(table: TableIdentifier, checkType: String): Unit = { val db = table.database.getOrElse("default") val expected = spark.externalCatalog.getTable(db, table.table) val shownDDL = sql(s"SHOW CREATE TABLE ${table.quotedString}").head().getString(0) - sql(s"DROP TABLE ${table.quotedString}") + sql(s"DROP $checkType ${table.quotedString}") - withTable(table.table) { + try { sql(shownDDL) val actual = spark.externalCatalog.getTable(db, table.table) checkCatalogTables(expected, actual) + } finally { + sql(s"DROP $checkType IF EXISTS ${table.table}") } } @@ -155,15 +308,31 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing "totalSize", "totalNumberFiles", "maxFileSize", - "minFileSize" + "minFileSize", + // EXTERNAL is not non-deterministic, but it is filtered out for external tables. + "EXTERNAL" ) table.copy( createTime = 0L, lastAccessTime = 0L, - properties = table.properties.filterKeys(!nondeterministicProps.contains(_))) + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + // View texts are checked separately + viewOriginalText = None, + viewText = None + ) + } + + // Normalizes attributes auto-generated by Spark SQL for views + def normalizeGeneratedAttributes(str: String): String = { + str.replaceAll("gen_attr_[0-9]+", "gen_attr_0") + } + + // We use expanded canonical view text as original view text of the new table + assertResult(expected.viewText.map(normalizeGeneratedAttributes)) { + actual.viewOriginalText.map(normalizeGeneratedAttributes) } - assert(normalize(expected) == normalize(actual)) + assert(normalize(actual) == normalize(expected)) } } From 1db37417c25429c0001c19d2f10f4a314fe4585c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 17 May 2016 18:02:31 -0700 Subject: [PATCH 0245/1470] [SPARK-14346] Fix scala-2.10 build ## What changes were proposed in this pull request? Scala 2.10 build was broken by #13079. I am reverting the change of that line. Author: Yin Huai Closes #13157 from yhuai/SPARK-14346-fix-scala2.10. (cherry picked from commit 2a5db9c140b9d60a5ec91018be19bec7b80850ee) Signed-off-by: Yin Huai --- .../scala/org/apache/spark/sql/catalyst/catalog/interface.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index d4f5cbb625850..3fdd411ac4cc9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -208,6 +208,6 @@ case class SimpleCatalogRelation( } require( - metadata.identifier.database.contains(databaseName), + metadata.identifier.database == Some(databaseName), "provided database does not match the one specified in the table definition") } From 5f5270ead86d5294af6c871e36112e2a833e9d64 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 18 May 2016 09:01:59 +0800 Subject: [PATCH 0246/1470] [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable ## What changes were proposed in this pull request? Update the unit test code, examples, and documents to remove calls to deprecated method `dataset.registerTempTable`. ## How was this patch tested? This PR only changes the unit test code, examples, and comments. It should be safe. This is a follow up of PR https://github.com/apache/spark/pull/12945 which was merged. Author: Sean Zhong Closes #13098 from clockfly/spark-15171-remove-deprecation. (cherry picked from commit 25b315e6cad7c27b62dcaa2c194293c1115fdfb3) Signed-off-by: Cheng Lian --- docs/sql-programming-guide.md | 48 ++++---- docs/streaming-programming-guide.md | 12 +- .../spark/examples/sql/JavaSparkSQL.java | 8 +- .../streaming/JavaSqlNetworkWordCount.java | 2 +- examples/src/main/python/sql.py | 2 +- .../python/streaming/sql_network_wordcount.py | 2 +- .../spark/examples/sql/RDDRelation.scala | 6 +- .../examples/sql/hive/HiveFromSpark.scala | 4 +- .../streaming/SqlNetworkWordCount.scala | 2 +- .../apache/spark/ml/JavaPipelineSuite.java | 2 +- .../JavaLogisticRegressionSuite.java | 10 +- .../regression/JavaLinearRegressionSuite.java | 4 +- python/pyspark/sql/context.py | 4 +- python/pyspark/sql/readwriter.py | 2 +- python/pyspark/sql/session.py | 2 +- python/pyspark/sql/tests.py | 25 +++-- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/JavaApplySchemaSuite.java | 8 +- .../spark/sql/sources/JavaSaveLoadSuite.java | 2 +- .../apache/spark/sql/CachedTableSuite.scala | 60 +++++----- .../spark/sql/ColumnExpressionSuite.scala | 2 +- .../sql/DataFrameTimeWindowingSuite.scala | 2 +- .../spark/sql/DataFrameWindowSuite.scala | 22 ++-- .../org/apache/spark/sql/JoinSuite.scala | 4 +- .../apache/spark/sql/ListTablesSuite.scala | 4 +- .../apache/spark/sql/SQLContextSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 103 +++++++++--------- .../sql/ScalaReflectionRelationSuite.scala | 10 +- .../org/apache/spark/sql/SubquerySuite.scala | 8 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 12 +- .../spark/sql/UserDefinedTypeSuite.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 10 +- .../benchmark/AggregateBenchmark.scala | 3 +- .../columnar/InMemoryColumnarQuerySuite.scala | 8 +- .../columnar/PartitionBatchPruningSuite.scala | 2 +- .../datasources/json/JsonSuite.scala | 58 +++++----- .../ParquetPartitionDiscoverySuite.scala | 10 +- .../parquet/ParquetQuerySuite.scala | 4 +- .../parquet/ParquetReadBenchmark.scala | 20 ++-- .../datasources/parquet/TPCDSBenchmark.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 8 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../sources/CreateTableAsSelectSuite.scala | 2 +- .../spark/sql/sources/InsertSuite.scala | 6 +- .../spark/sql/sources/SaveLoadSuite.scala | 4 +- .../spark/sql/streaming/StreamSuite.scala | 2 +- .../apache/spark/sql/test/SQLTestData.scala | 46 ++++---- .../spark/sql/hive/JavaDataFrameSuite.java | 2 +- .../hive/JavaMetastoreDataSourcesSuite.java | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 4 +- .../spark/sql/hive/HiveParquetSuite.scala | 4 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 8 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 12 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 8 +- .../hive/ParquetHiveCompatibilitySuite.scala | 2 +- .../spark/sql/hive/QueryPartitionSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../org/apache/spark/sql/hive/UDFSuite.scala | 2 +- .../execution/AggregationQuerySuite.scala | 8 +- .../sql/hive/execution/HiveExplainSuite.scala | 2 +- .../HiveOperatorQueryableSuite.scala | 4 +- .../sql/hive/execution/HivePlanTest.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 18 +-- .../hive/execution/HiveResolutionSuite.scala | 10 +- .../hive/execution/HiveTableScanSuite.scala | 2 +- .../sql/hive/execution/HiveUDFSuite.scala | 22 ++-- .../sql/hive/execution/SQLQuerySuite.scala | 66 +++++------ .../execution/SQLWindowFunctionSuite.scala | 16 +-- .../hive/orc/OrcPartitionDiscoverySuite.scala | 8 +- .../spark/sql/hive/orc/OrcQuerySuite.scala | 12 +- .../spark/sql/hive/orc/OrcSourceSuite.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 9 +- .../sql/sources/HadoopFsRelationTest.scala | 10 +- .../hive/HiveContextCompatibilitySuite.scala | 4 +- 74 files changed, 407 insertions(+), 401 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a16a6bb1d93ef..a9e1f9d5ce46f 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -529,7 +529,7 @@ case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF() -people.registerTempTable("people") +people.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19") @@ -605,7 +605,7 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m // Apply a schema to an RDD of JavaBeans and register it as a table. DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); -schemaPeople.registerTempTable("people"); +schemaPeople.createOrReplaceTempView("people"); // SQL can be run over RDDs that have been registered as tables. DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -643,7 +643,7 @@ people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) # Infer the schema, and register the DataFrame as a table. schemaPeople = sqlContext.createDataFrame(people) -schemaPeople.registerTempTable("people") +schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -703,8 +703,8 @@ val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) // Apply the schema to the RDD. val peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema) -// Register the DataFrames as a table. -peopleDataFrame.registerTempTable("people") +// Creates a temporary view using the DataFrame. +peopleDataFrame.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val results = sqlContext.sql("SELECT name FROM people") @@ -771,10 +771,10 @@ JavaRDD rowRDD = people.map( // Apply the schema to the RDD. DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema); -// Register the DataFrame as a table. -peopleDataFrame.registerTempTable("people"); +// Creates a temporary view using the DataFrame. +peopleDataFrame.createOrReplaceTempView("people"); -// SQL can be run over RDDs that have been registered as tables. +// SQL can be run over a temporary view created using DataFrames. DataFrame results = sqlContext.sql("SELECT name FROM people"); // The results of SQL queries are DataFrames and support all the normal RDD operations. @@ -824,8 +824,8 @@ schema = StructType(fields) # Apply the schema to the RDD. schemaPeople = sqlContext.createDataFrame(people, schema) -# Register the DataFrame as a table. -schemaPeople.registerTempTable("people") +# Creates a temporary view using the DataFrame +schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. results = sqlContext.sql("SELECT name FROM people") @@ -844,7 +844,7 @@ for name in names.collect(): # Data Sources Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. -A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. +A DataFrame can be operated on as normal RDDs and can also be used to create a temporary view. Registering a DataFrame as a table allows you to run SQL queries over its data. This section describes the general methods for loading and saving data using the Spark Data Sources and then goes into specific options that are available for the built-in data sources. @@ -1072,8 +1072,8 @@ people.write.parquet("people.parquet") // The result of loading a Parquet file is also a DataFrame. val parquetFile = sqlContext.read.parquet("people.parquet") -//Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile") +// Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile") val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -1094,8 +1094,8 @@ schemaPeople.write().parquet("people.parquet"); // The result of loading a parquet file is also a DataFrame. DataFrame parquetFile = sqlContext.read().parquet("people.parquet"); -// Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); +// Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile"); DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { @@ -1120,8 +1120,8 @@ schemaPeople.write.parquet("people.parquet") # The result of loading a parquet file is also a DataFrame. parquetFile = sqlContext.read.parquet("people.parquet") -# Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); +# Parquet files can also be used to create a temporary view and then used in SQL statements. +parquetFile.createOrReplaceTempView("parquetFile"); teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) for teenName in teenNames.collect(): @@ -1144,7 +1144,7 @@ write.parquet(schemaPeople, "people.parquet") # The result of loading a parquet file is also a DataFrame. parquetFile <- read.parquet(sqlContext, "people.parquet") -# Parquet files can also be registered as tables and then used in SQL statements. +# Parquet files can also be used to create a temporary view and then used in SQL statements. registerTempTable(parquetFile, "parquetFile") teenagers <- sql(sqlContext, "SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") schema <- structType(structField("name", "string")) @@ -1506,8 +1506,8 @@ people.printSchema() // |-- age: long (nullable = true) // |-- name: string (nullable = true) -// Register this DataFrame as a table. -people.registerTempTable("people") +// Creates a temporary view using the DataFrame +people.createOrReplaceTempView("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -1544,8 +1544,8 @@ people.printSchema(); // |-- age: long (nullable = true) // |-- name: string (nullable = true) -// Register this DataFrame as a table. -people.registerTempTable("people"); +// Creates a temporary view using the DataFrame +people.createOrReplaceTempView("people"); // SQL statements can be run by using the sql methods provided by sqlContext. DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -1582,8 +1582,8 @@ people.printSchema() # |-- age: long (nullable = true) # |-- name: string (nullable = true) -# Register this DataFrame as a table. -people.registerTempTable("people") +# Creates a temporary view using the DataFrame. +people.createOrReplaceTempView("people") # SQL statements can be run by using the sql methods provided by `sqlContext`. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 9ca9b1844c453..4d0a1122dcd12 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1553,8 +1553,8 @@ words.foreachRDD { rdd => // Convert RDD[String] to DataFrame val wordsDataFrame = rdd.toDF("word") - // Register as table - wordsDataFrame.registerTempTable("words") + // Create a temporary view + wordsDataFrame.createOrReplaceTempView("words") // Do word count on DataFrame using SQL and print it val wordCountsDataFrame = @@ -1606,8 +1606,8 @@ words.foreachRDD( }); DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class); - // Register as table - wordsDataFrame.registerTempTable("words"); + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words"); // Do word count on table using SQL and print it DataFrame wordCountsDataFrame = @@ -1646,8 +1646,8 @@ def process(time, rdd): rowRdd = rdd.map(lambda w: Row(word=w)) wordsDataFrame = sqlContext.createDataFrame(rowRdd) - # Register as table - wordsDataFrame.registerTempTable("words") + # Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") 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 cf0167f13a6fb..55e591d0ce166 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 @@ -73,11 +73,11 @@ public Person call(String line) { } }); - // Apply a schema to an RDD of Java Beans and register it as a table. + // Apply a schema to an RDD of Java Beans and create a temporary view Dataset schemaPeople = spark.createDataFrame(people, Person.class); schemaPeople.createOrReplaceTempView("people"); - // SQL can be run over RDDs that have been registered as tables. + // SQL can be run over RDDs which backs a temporary view. Dataset teenagers = spark.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. @@ -101,7 +101,7 @@ public String call(Row row) { // The result of loading a parquet file is also a DataFrame. Dataset parquetFile = spark.read().parquet("people.parquet"); - //Parquet files can also be registered as tables and then used in SQL statements. + // A temporary view can be created by using Parquet files and then used in SQL statements. parquetFile.createOrReplaceTempView("parquetFile"); Dataset teenagers2 = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); @@ -130,7 +130,7 @@ public String call(Row row) { // |-- age: IntegerType // |-- name: StringType - // Register this DataFrame as a table. + // Creates a temporary view using the DataFrame peopleFromJsonFile.createOrReplaceTempView("people"); // SQL statements can be run by using the sql methods provided by `spark` diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 51305227708e2..b8e9e125ba596 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -94,7 +94,7 @@ public JavaRecord call(String word) { }); Dataset wordsDataFrame = spark.createDataFrame(rowRDD, JavaRecord.class); - // Register as table + // Creates a temporary view using the DataFrame wordsDataFrame.createOrReplaceTempView("words"); // Do word count on table using SQL and print it diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 234024063f74e..ac7246938d3b4 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -66,7 +66,7 @@ # |-- age: long (nullable = true) # |-- name: string (nullable = true) - # Register this DataFrame as a temporary table. + # Creates a temporary view using the DataFrame. people.createOrReplaceTempView("people") # SQL statements can be run by using the sql methods provided by `spark` diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index 25e821534e19b..398ac8d2d8f5e 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -70,7 +70,7 @@ def process(time, rdd): rowRdd = rdd.map(lambda w: Row(word=w)) wordsDataFrame = spark.createDataFrame(rowRdd) - # Register as table + # Creates a temporary view using the DataFrame. wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index d1bda0ff841dd..1b019fbb51771 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -35,8 +35,8 @@ object RDDRelation { import spark.implicits._ val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) - // Any RDD containing case classes can be registered as a table. The schema of the table is - // automatically inferred using scala reflection. + // Any RDD containing case classes can be used to create a temporary view. The schema of the + // view is automatically inferred using scala reflection. df.createOrReplaceTempView("records") // Once tables have been registered, you can run SQL queries over them. @@ -66,7 +66,7 @@ object RDDRelation { // Queries can be run using the DSL on parquet files just like the original RDD. parquetFile.where($"key" === 1).select($"value".as("a")).collect().foreach(println) - // These files can also be registered as tables. + // These files can also be used to create a temporary view. parquetFile.createOrReplaceTempView("parquetFile") spark.sql("SELECT * FROM parquetFile").collect().foreach(println) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index a15cf5ded0e7f..7293cb51b2613 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -70,9 +70,9 @@ object HiveFromSpark { case Row(key: Int, value: String) => s"Key: $key, Value: $value" } - // You can also register RDDs as temporary tables within a HiveContext. + // You can also use RDDs to create temporary views within a HiveContext. val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.toDF().registerTempTable("records") + rdd.toDF().createOrReplaceTempView("records") // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 688c5b23c288d..787bbec73b28f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -66,7 +66,7 @@ object SqlNetworkWordCount { // Convert RDD[String] to RDD[case class] to DataFrame val wordsDataFrame = rdd.map(w => Record(w)).toDF() - // Register as table + // Creates a temporary view using the DataFrame wordsDataFrame.createOrReplaceTempView("words") // Do word count on table using SQL and print it diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index 46c26e8b929e1..a81a36d1b1c43 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -68,7 +68,7 @@ public void pipeline() { Pipeline pipeline = new Pipeline() .setStages(new PipelineStage[]{scaler, lr}); PipelineModel model = pipeline.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index 98abca221c887..b8da04c26a49d 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -54,7 +54,7 @@ public void setUp() { List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); + dataset.createOrReplaceTempView("dataset"); } @After @@ -68,7 +68,7 @@ public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); Assert.assertEquals(lr.getLabelCol(), "label"); LogisticRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); // Check defaults @@ -97,14 +97,14 @@ public void logisticRegressionWithSetters() { // Modify model params, and check that the params worked. model.setThreshold(1.0); - model.transform(dataset).registerTempTable("predAllZero"); + model.transform(dataset).createOrReplaceTempView("predAllZero"); Dataset predAllZero = spark.sql("SELECT prediction, myProbability FROM predAllZero"); for (Row r : predAllZero.collectAsList()) { Assert.assertEquals(0.0, r.getDouble(0), eps); } // Call transform with params, and check that the params worked. model.transform(dataset, model.threshold().w(0.0), model.probabilityCol().w("myProb")) - .registerTempTable("predNotAllZero"); + .createOrReplaceTempView("predNotAllZero"); Dataset predNotAllZero = spark.sql("SELECT prediction, myProb FROM predNotAllZero"); boolean foundNonZero = false; for (Row r : predNotAllZero.collectAsList()) { @@ -130,7 +130,7 @@ public void logisticRegressionPredictorClassifierMethods() { LogisticRegressionModel model = lr.fit(dataset); Assert.assertEquals(2, model.numClasses()); - model.transform(dataset).registerTempTable("transformed"); + model.transform(dataset).createOrReplaceTempView("transformed"); Dataset trans1 = spark.sql("SELECT rawPrediction, probability FROM transformed"); for (Row row : trans1.collectAsList()) { Vector raw = (Vector) row.get(0); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index d3ef5f6fca076..126aa6298f3de 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -50,7 +50,7 @@ public void setUp() { List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); + dataset.createOrReplaceTempView("dataset"); } @After @@ -65,7 +65,7 @@ public void linearRegressionDefaultParams() { assertEquals("label", lr.getLabelCol()); assertEquals("auto", lr.getSolver()); LinearRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); Dataset predictions = spark.sql("SELECT label, prediction FROM prediction"); predictions.collect(); // Check defaults diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index ca111ae9bb7d3..e8e60c64121b5 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -57,7 +57,7 @@ def __init__(self, sparkContext, sparkSession=None, jsqlContext=None): ... 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") + >>> df.createOrReplaceTempView("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((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \ @@ -106,7 +106,7 @@ def getOrCreate(cls, sc): def newSession(self): """ Returns a new SQLContext as new session, that has separate SQLConf, - registered temporary tables and UDFs, but shared SparkContext and + registered temporary views and UDFs, but shared SparkContext and table cache. """ return self.__class__(self._sc, self.sparkSession.newSession()) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index c98aef1a0e696..8e6bce90010e3 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -266,7 +266,7 @@ def table(self, tableName): :param tableName: string, name of the table. >>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned') - >>> df.registerTempTable('tmpTable') + >>> df.createOrReplaceTempView('tmpTable') >>> spark.read.table('tmpTable').dtypes [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] """ diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 0781b442cbcd8..257a239c8d7b0 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -186,7 +186,7 @@ def __init__(self, sparkContext, jsparkSession=None): def newSession(self): """ Returns a new SparkSession as new session, that has separate SQLConf, - registered temporary tables and UDFs, but shared SparkContext and + registered temporary views and UDFs, but shared SparkContext and table cache. """ return self.__class__(self._sc, self._jsparkSession.newSession()) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 0977c43a398da..e86f44281d492 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -294,7 +294,8 @@ def test_udf(self): def test_udf2(self): self.spark.catalog.registerFunction("strlen", lambda string: len(string), IntegerType()) - self.spark.createDataFrame(self.sc.parallelize([Row(a="test")])).registerTempTable("test") + self.spark.createDataFrame(self.sc.parallelize([Row(a="test")]))\ + .createOrReplaceTempView("test") [res] = self.spark.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() self.assertEqual(4, res[0]) @@ -320,7 +321,7 @@ def test_multiple_udfs(self): def test_udf_with_array_type(self): d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) - self.spark.createDataFrame(rdd).registerTempTable("test") + self.spark.createDataFrame(rdd).createOrReplaceTempView("test") self.spark.catalog.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) self.spark.catalog.registerFunction("maplen", lambda d: len(d), IntegerType()) [(l1, l2)] = self.spark.sql("select copylist(l), maplen(d) from test").collect() @@ -360,7 +361,7 @@ def test_basic_functions(self): self.assertTrue(df.is_cached) self.assertEqual(2, df.count()) - df.registerTempTable("temp") + df.createOrReplaceTempView("temp") df = self.spark.sql("select foo from temp") df.count() df.collect() @@ -420,7 +421,7 @@ def test_infer_schema(self): df = self.spark.createDataFrame(rdd) self.assertEqual([], df.rdd.map(lambda r: r.l).first()) self.assertEqual([None, ""], df.rdd.map(lambda r: r.s).collect()) - df.registerTempTable("test") + df.createOrReplaceTempView("test") result = self.spark.sql("SELECT l[0].a from test where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) @@ -428,7 +429,7 @@ def test_infer_schema(self): self.assertEqual(df.schema, df2.schema) self.assertEqual({}, df2.rdd.map(lambda r: r.d).first()) self.assertEqual([None, ""], df2.rdd.map(lambda r: r.s).collect()) - df2.registerTempTable("test2") + df2.createOrReplaceTempView("test2") result = self.spark.sql("SELECT l[0].a from test2 where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) @@ -487,7 +488,7 @@ def test_apply_schema(self): datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) self.assertEqual(r, results.first()) - df.registerTempTable("table2") + df.createOrReplaceTempView("table2") r = self.spark.sql("SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + "short1 + 1 AS short1, short2 - 1 AS short2, int1 - 1 AS int1, " + "float1 + 1.5 as float1 FROM table2").first() @@ -515,7 +516,7 @@ def test_convert_row_to_dict(self): row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) self.assertEqual(1, row.asDict()['l'][0].a) df = self.sc.parallelize([row]).toDF() - df.registerTempTable("test") + df.createOrReplaceTempView("test") row = self.spark.sql("select l, d from test").head() self.assertEqual(1, row.asDict()["l"][0].a) self.assertEqual(1.0, row.asDict()['d']['key'].c) @@ -556,7 +557,7 @@ def test_infer_schema_with_udt(self): schema = df.schema field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), ExamplePointUDT) - df.registerTempTable("labeled_point") + df.createOrReplaceTempView("labeled_point") point = self.spark.sql("SELECT point FROM labeled_point").head().point self.assertEqual(point, ExamplePoint(1.0, 2.0)) @@ -565,7 +566,7 @@ def test_infer_schema_with_udt(self): schema = df.schema field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), PythonOnlyUDT) - df.registerTempTable("labeled_point") + df.createOrReplaceTempView("labeled_point") point = self.spark.sql("SELECT point FROM labeled_point").head().point self.assertEqual(point, PythonOnlyPoint(1.0, 2.0)) @@ -1427,7 +1428,7 @@ def test_list_tables(self): spark.sql("CREATE DATABASE some_db") self.assertEquals(spark.catalog.listTables(), []) self.assertEquals(spark.catalog.listTables("some_db"), []) - spark.createDataFrame([(1, 1)]).registerTempTable("temp_tab") + spark.createDataFrame([(1, 1)]).createOrReplaceTempView("temp_tab") spark.sql("CREATE TABLE tab1 (name STRING, age INT)") spark.sql("CREATE TABLE some_db.tab2 (name STRING, age INT)") tables = sorted(spark.catalog.listTables(), key=lambda t: t.name) @@ -1554,8 +1555,8 @@ def test_list_columns(self): def test_cache(self): spark = self.spark - spark.createDataFrame([(2, 2), (3, 3)]).registerTempTable("tab1") - spark.createDataFrame([(2, 2), (3, 3)]).registerTempTable("tab2") + spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab1") + spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab2") self.assertFalse(spark.catalog.isCached("tab1")) self.assertFalse(spark.catalog.isCached("tab2")) spark.catalog.cacheTable("tab1") 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 44511885a7db6..a3e2b49556e4c 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 @@ -376,7 +376,7 @@ class SQLContext private[sql]( * // |-- name: string (nullable = false) * // |-- age: integer (nullable = true) * - * dataFrame.registerTempTable("people") + * dataFrame.createOrReplaceTempView("people") * sqlContext.sql("select name from people").collect.foreach(println) * }}} * diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java index f2ae40e644245..573d0e3594363 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java @@ -108,7 +108,7 @@ public Row call(Person person) throws Exception { StructType schema = DataTypes.createStructType(fields); Dataset df = spark.createDataFrame(rowRDD, schema); - df.registerTempTable("people"); + df.createOrReplaceTempView("people"); List actual = spark.sql("SELECT * FROM people").collectAsList(); List expected = new ArrayList<>(2); @@ -144,7 +144,7 @@ public Row call(Person person) { StructType schema = DataTypes.createStructType(fields); Dataset df = spark.createDataFrame(rowRDD, schema); - df.registerTempTable("people"); + df.createOrReplaceTempView("people"); List actual = spark.sql("SELECT * FROM people").toJavaRDD() .map(new Function() { @Override @@ -202,14 +202,14 @@ public void applySchemaToJSON() { Dataset df1 = spark.read().json(jsonRDD); StructType actualSchema1 = df1.schema(); Assert.assertEquals(expectedSchema, actualSchema1); - df1.registerTempTable("jsonTable1"); + df1.createOrReplaceTempView("jsonTable1"); List actual1 = spark.sql("select * from jsonTable1").collectAsList(); Assert.assertEquals(expectedResult, actual1); Dataset df2 = spark.read().schema(expectedSchema).json(jsonRDD); StructType actualSchema2 = df2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); - df2.registerTempTable("jsonTable2"); + df2.createOrReplaceTempView("jsonTable2"); List actual2 = spark.sql("select * from jsonTable2").collectAsList(); Assert.assertEquals(expectedResult, actual2); } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java index d0435e4d4395e..9840bc46f95f2 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java @@ -72,7 +72,7 @@ public void setUp() throws IOException { } JavaRDD rdd = jsc.parallelize(jsonObjects); df = spark.read().json(rdd); - df.registerTempTable("jsonTable"); + df.createOrReplaceTempView("jsonTable"); } @After 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 6d8de80a11fd6..1c96bdc05cfcd 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 @@ -71,7 +71,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("cache temp table") { - testData.select('key).registerTempTable("tempTable") + testData.select('key).createOrReplaceTempView("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) spark.catalog.cacheTable("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable")) @@ -99,8 +99,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("uncaching temp table") { - testData.select('key).registerTempTable("tempTable1") - testData.select('key).registerTempTable("tempTable2") + testData.select('key).createOrReplaceTempView("tempTable1") + testData.select('key).createOrReplaceTempView("tempTable2") spark.catalog.cacheTable("tempTable1") assertCached(sql("SELECT COUNT(*) FROM tempTable1")) @@ -116,7 +116,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("too big for memory") { val data = "*" * 1000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF() - .registerTempTable("bigData") + .createOrReplaceTempView("bigData") spark.table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(spark.table("bigData").count() === 200000L) spark.table("bigData").unpersist(blocking = true) @@ -191,7 +191,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("SELECT star from cached table") { - sql("SELECT * FROM testData").registerTempTable("selectStar") + sql("SELECT * FROM testData").createOrReplaceTempView("selectStar") spark.catalog.cacheTable("selectStar") checkAnswer( sql("SELECT * FROM selectStar WHERE key = 1"), @@ -286,15 +286,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Drops temporary table") { - testData.select('key).registerTempTable("t1") + testData.select('key).createOrReplaceTempView("t1") spark.table("t1") spark.catalog.dropTempView("t1") intercept[AnalysisException](spark.table("t1")) } test("Drops cached temporary table") { - testData.select('key).registerTempTable("t1") - testData.select('key).registerTempTable("t2") + testData.select('key).createOrReplaceTempView("t1") + testData.select('key).createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") assert(spark.catalog.isCached("t1")) @@ -306,15 +306,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Clear all cache") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") spark.catalog.clearCache() assert(spark.cacheManager.isEmpty) - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") sql("Clear CACHE") @@ -322,8 +322,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("Clear accumulators when uncacheTable to prevent memory leaking") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -350,7 +350,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("SPARK-10327 Cache Table is not working while subquery has alias in its project list") { sparkContext.parallelize((1, 1) :: (2, 2) :: Nil) - .toDF("key", "value").selectExpr("key", "value", "key+1").registerTempTable("abc") + .toDF("key", "value").selectExpr("key", "value", "key+1").createOrReplaceTempView("abc") spark.catalog.cacheTable("abc") val sparkPlan = sql( @@ -371,9 +371,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { val table3x = testData.union(testData).union(testData) - table3x.registerTempTable("testData3x") + table3x.createOrReplaceTempView("testData3x") - sql("SELECT key, value FROM testData3x ORDER BY key").registerTempTable("orderedTable") + sql("SELECT key, value FROM testData3x ORDER BY key").createOrReplaceTempView("orderedTable") spark.catalog.cacheTable("orderedTable") assertCached(spark.table("orderedTable")) // Should not have an exchange as the query is already sorted on the group by key. @@ -388,8 +388,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // different number of partitions. for (numPartitions <- 1 until 10 by 4) { withTempTable("t1", "t2") { - testData.repartition(numPartitions, $"key").registerTempTable("t1") - testData2.repartition(numPartitions, $"a").registerTempTable("t2") + testData.repartition(numPartitions, $"key").createOrReplaceTempView("t1") + testData2.repartition(numPartitions, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -410,8 +410,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // Distribute the tables into non-matching number of partitions. Need to shuffle one side. withTempTable("t1", "t2") { - testData.repartition(6, $"key").registerTempTable("t1") - testData2.repartition(3, $"a").registerTempTable("t2") + testData.repartition(6, $"key").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -427,8 +427,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // One side of join is not partitioned in the desired way. Need to shuffle one side. withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(6, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(6, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -443,8 +443,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(12, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(12, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -462,8 +462,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // the side that has already partitioned is smaller than the side that is not partitioned, // we shuffle both side. withTempTable("t1", "t2") { - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(3, $"a").registerTempTable("t2") + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") @@ -479,7 +479,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // repartition's column ordering is different from group by column ordering. // But they use the same set of columns. withTempTable("t1") { - testData.repartition(6, $"value", $"key").registerTempTable("t1") + testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1") spark.catalog.cacheTable("t1") val query = sql("SELECT value, key from t1 group by key, value") @@ -496,9 +496,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // See PartitioningSuite for more details. withTempTable("t1", "t2") { val df1 = testData - df1.repartition(6, $"value", $"key").registerTempTable("t1") + df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1") val df2 = testData2.select($"a", $"b".cast("string")) - df2.repartition(6, $"a", $"b").registerTempTable("t2") + df2.repartition(6, $"a", $"b").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a5aecca13f616..e89fa32b15143 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -321,7 +321,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { nanvl($"b", $"e"), nanvl($"e", $"f")), Row(null, 3.0, 10.0, null, Double.PositiveInfinity, 3.0, 1.0) ) - testData.registerTempTable("t") + testData.createOrReplaceTempView("t") checkAnswer( sql( "select nanvl(a, 5), nanvl(b, 10), nanvl(10, b), nanvl(c, null), nanvl(d, 10), " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 4ee2006421e1e..a15b4e1221d3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -245,7 +245,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B Seq( ("2016-03-27 19:39:34", 1), ("2016-03-27 19:39:56", 2), - ("2016-03-27 19:39:27", 4)).toDF("time", "value").registerTempTable(tableName) + ("2016-03-27 19:39:27", 4)).toDF("time", "value").createOrReplaceTempView(tableName) try { f(tableName) } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala index 91095af0ddae7..07aad3c406b60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala @@ -49,7 +49,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lead") { val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( @@ -59,7 +59,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lag") { val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( @@ -70,7 +70,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lead with default value") { val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( lead("value", 2, "n/a").over(Window.partitionBy("key").orderBy("value"))), @@ -80,7 +80,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("lag with default value") { val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( lag("value", 2, "n/a").over(Window.partitionBy($"key").orderBy($"value"))), @@ -89,7 +89,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("rank functions in unspecific window") { val df = Seq((1, "1"), (2, "2"), (1, "2"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -112,7 +112,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between") { val df = Seq((1, "1"), (2, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( avg("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 2))), @@ -121,7 +121,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between") { val df = Seq((1, "1"), (1, "1"), (3, "1"), (2, "2"), (2, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( avg("key").over(Window.partitionBy($"value").orderBy($"key").rangeBetween(-1, 1))), @@ -131,7 +131,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between with unbounded") { val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -146,7 +146,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between with unbounded") { val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") checkAnswer( df.select( $"key", @@ -357,7 +357,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and rows between with unbounded + predicate pushdown") { val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") val selectList = Seq($"key", $"value", last("key").over( Window.partitionBy($"value").orderBy($"key").rowsBetween(0, Long.MaxValue)), @@ -372,7 +372,7 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { test("aggregation and range between with unbounded + predicate pushdown") { val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") + df.createOrReplaceTempView("window_table") val selectList = Seq($"key", $"value", last("value").over( Window.partitionBy($"value").orderBy($"key").rangeBetween(-2, -1)).equalTo("2") 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 da567db5eec62..a6b83b3d07504 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 @@ -344,8 +344,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { } test("full outer join") { - upperCaseData.where('N <= 4).registerTempTable("`left`") - upperCaseData.where('N >= 3).registerTempTable("`right`") + upperCaseData.where('N <= 4).createOrReplaceTempView("`left`") + upperCaseData.where('N >= 3).createOrReplaceTempView("`right`") val left = UnresolvedRelation(TableIdentifier("left"), None) val right = UnresolvedRelation(TableIdentifier("right"), None) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index 1c6e6cc15d778..65fe271b69172 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -29,7 +29,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex private lazy val df = (1 to 10).map(i => (i, s"str$i")).toDF("key", "value") before { - df.registerTempTable("listtablessuitetable") + df.createOrReplaceTempView("listtablessuitetable") } after { @@ -74,7 +74,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex case tableDF => assert(expectedSchema === tableDF.schema) - tableDF.registerTempTable("tables") + tableDF.createOrReplaceTempView("tables") checkAnswer( sql( "SELECT isTemporary, tableName from tables WHERE tableName = 'listtablessuitetable'"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 1d5fc570c65d8..38d7b6e25b829 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -60,7 +60,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { // temporary table should not be shared val df = session1.range(10) - df.registerTempTable("test1") + df.createOrReplaceTempView("test1") assert(session1.tableNames().contains("test1")) assert(!session2.tableNames().contains("test1")) 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 b67e2bdeb3663..010dea5b30380 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 @@ -39,7 +39,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { setupTestData() test("having clause") { - Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v").registerTempTable("hav") + Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") + .createOrReplaceTempView("hav") checkAnswer( sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"), Row("one", 6) :: Row("three", 3) :: Nil) @@ -47,7 +48,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-8010: promote numeric to string") { val df = Seq((1, 1)).toDF("key", "value") - df.registerTempTable("src") + df.createOrReplaceTempView("src") val queryCaseWhen = sql("select case when true then 1.0 else '1' end from src ") val queryCoalesce = sql("select coalesce(null, 1, '1') from src ") @@ -100,7 +101,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (83, 0, 38), (26, 0, 79), (43, 81, 24) - ).toDF("a", "b", "c").registerTempTable("cachedData") + ).toDF("a", "b", "c").createOrReplaceTempView("cachedData") spark.catalog.cacheTable("cachedData") checkAnswer( @@ -109,7 +110,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("self join with aliases") { - Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").registerTempTable("df") + Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").createOrReplaceTempView("df") checkAnswer( sql( @@ -137,7 +138,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .toDF("int", "str") .groupBy("str") .agg($"str", count("str").as("strCount")) - .registerTempTable("df") + .createOrReplaceTempView("df") checkAnswer( sql( @@ -195,7 +196,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("grouping on nested fields") { spark.read.json(sparkContext.parallelize( """{"nested": {"attribute": 1}, "value": 2}""" :: Nil)) - .registerTempTable("rows") + .createOrReplaceTempView("rows") checkAnswer( sql( @@ -214,7 +215,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.read.json( sparkContext.parallelize( Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}"))) - .registerTempTable("d") + .createOrReplaceTempView("d") checkAnswer( sql("select * from d where d.a in (1,2)"), @@ -225,7 +226,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.read.json( sparkContext.parallelize( Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", ""))) - .registerTempTable("d") + .createOrReplaceTempView("d") checkAnswer( sql("select count(1) from d"), @@ -261,7 +262,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { spark.table("testData") .union(spark.table("testData")) .union(spark.table("testData")) - .registerTempTable("testData3x") + .createOrReplaceTempView("testData3x") try { // Just to group rows. @@ -391,7 +392,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-3173 Timestamp support in the parser") { - (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").registerTempTable("timestamps") + (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").createOrReplaceTempView("timestamps") checkAnswer(sql( "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"), @@ -746,7 +747,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("count of empty table") { withTempTable("t") { - Seq.empty[(Int, Int)].toDF("a", "b").registerTempTable("t") + Seq.empty[(Int, Int)].toDF("a", "b").createOrReplaceTempView("t") checkAnswer( sql("select count(a) from t"), Row(0)) @@ -891,10 +892,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-3349 partitioning after limit") { sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) - .registerTempTable("subset1") + .createOrReplaceTempView("subset1") sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC") .limit(2) - .registerTempTable("subset2") + .createOrReplaceTempView("subset2") checkAnswer( sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"), Row(3, "c", 3) :: @@ -1111,7 +1112,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df1 = spark.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + df1.createOrReplaceTempView("applySchema1") checkAnswer( sql("SELECT * FROM applySchema1"), Row(1, "A1", true, null) :: @@ -1141,7 +1142,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df2 = spark.createDataFrame(rowRDD2, schema2) - df2.registerTempTable("applySchema2") + df2.createOrReplaceTempView("applySchema2") checkAnswer( sql("SELECT * FROM applySchema2"), Row(Row(1, true), Map("A1" -> null)) :: @@ -1166,7 +1167,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } val df3 = spark.createDataFrame(rowRDD3, schema2) - df3.registerTempTable("applySchema3") + df3.createOrReplaceTempView("applySchema3") checkAnswer( sql("SELECT f1.f11, f2['D4'] FROM applySchema3"), @@ -1214,7 +1215,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { def validateMetadata(rdd: DataFrame): Unit = { assert(rdd.schema("name").metadata.getString(docKey) == docValue) } - personWithMeta.registerTempTable("personWithMeta") + personWithMeta.createOrReplaceTempView("personWithMeta") validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"id", $"name")) @@ -1409,7 +1410,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-3483 Special chars in column names") { val data = sparkContext.parallelize( Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) - spark.read.json(data).registerTempTable("records") + spark.read.json(data).createOrReplaceTempView("records") sql("SELECT `key?number1`, `key.number2` FROM records") } @@ -1451,12 +1452,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-4322 Grouping field with struct field as sub expression") { spark.read.json(sparkContext.makeRDD("""{"a": {"b": [{"c": 1}]}}""" :: Nil)) - .registerTempTable("data") + .createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b[0].c FROM data GROUP BY a.b[0].c"), Row(1)) spark.catalog.dropTempView("data") spark.read.json( - sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).registerTempTable("data") + sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b + 1 FROM data GROUP BY a.b + 1"), Row(2)) spark.catalog.dropTempView("data") } @@ -1478,10 +1479,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1, "1") :: TestData(2, null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) - rdd1.toDF().registerTempTable("nulldata1") + rdd1.toDF().createOrReplaceTempView("nulldata1") val nullCheckData2 = TestData(1, "1") :: TestData(2, null) :: Nil val rdd2 = sparkContext.parallelize((0 to 1).map(i => nullCheckData2(i))) - rdd2.toDF().registerTempTable("nulldata2") + rdd2.toDF().createOrReplaceTempView("nulldata2") checkAnswer(sql("SELECT nulldata1.key FROM nulldata1 join " + "nulldata2 on nulldata1.value <=> nulldata2.value"), (1 to 2).map(i => Row(i))) @@ -1490,7 +1491,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Multi-column COUNT(DISTINCT ...)") { val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF().registerTempTable("distinctData") + rdd.toDF().createOrReplaceTempView("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2)) } @@ -1498,7 +1499,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF().registerTempTable("testTable1") + rdd.toDF().createOrReplaceTempView("testTable1") checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) } } @@ -1506,7 +1507,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-6145: ORDER BY test for nested fields") { spark.read.json(sparkContext.makeRDD( """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)) - .registerTempTable("nestedOrder") + .createOrReplaceTempView("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)) @@ -1517,8 +1518,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-6145: special cases") { - spark.read.json(sparkContext.makeRDD( - """{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)).registerTempTable("t") + spark.read + .json(sparkContext.makeRDD("""{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)) + .createOrReplaceTempView("t") + 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)) } @@ -1526,14 +1529,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-6898: complete support for special chars in column names") { spark.read.json(sparkContext.makeRDD( """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) - .registerTempTable("t") + .createOrReplaceTempView("t") checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } test("SPARK-6583 order by aggregated function") { Seq("1" -> 3, "1" -> 4, "2" -> 7, "2" -> 8, "3" -> 5, "3" -> 6, "4" -> 1, "4" -> 2) - .toDF("a", "b").registerTempTable("orderByData") + .toDF("a", "b").createOrReplaceTempView("orderByData") checkAnswer( sql( @@ -1619,7 +1622,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (0, null, null, false), (1, null, null, false), (null, null, null, true) - ).toDF("i", "b", "r1", "r2").registerTempTable("t") + ).toDF("i", "b", "r1", "r2").createOrReplaceTempView("t") checkAnswer(sql("select i = b from t"), sql("select r1 from t")) checkAnswer(sql("select i <=> b from t"), sql("select r2 from t")) @@ -1629,14 +1632,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-7067: order by queries for complex ExtractValue chain") { withTempTable("t") { spark.read.json(sparkContext.makeRDD( - """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).registerTempTable("t") + """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).createOrReplaceTempView("t") checkAnswer(sql("SELECT a.b FROM t ORDER BY b[0].d"), Row(Seq(Row(1)))) } } test("SPARK-8782: ORDER BY NULL") { withTempTable("t") { - Seq((1, 2), (1, 2)).toDF("a", "b").registerTempTable("t") + Seq((1, 2), (1, 2)).toDF("a", "b").createOrReplaceTempView("t") checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), Row(1, 2))) } } @@ -1645,7 +1648,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withTempTable("t") { val df = Seq(1 -> "a").toDF("count", "sort") checkAnswer(df.filter("count > 0"), Row(1, "a")) - df.registerTempTable("t") + df.createOrReplaceTempView("t") checkAnswer(sql("select count, sort from t"), Row(1, "a")) } } @@ -1759,7 +1762,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { withTempTable("1one") { sparkContext.parallelize(1 to 10).map(i => (i, i.toString)) .toDF("num", "str") - .registerTempTable("1one") + .createOrReplaceTempView("1one") checkAnswer(sql("select count(num) from 1one"), Row(10)) } } @@ -1801,7 +1804,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-10130 type coercion for IF should have children resolved first") { withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer( sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) } @@ -1809,7 +1812,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-10389: order by non-attribute grouping expression on Aggregate") { withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1"), Seq(Row(1), Row(1))) checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY (key + 1) * 2"), @@ -1872,7 +1875,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-11032: resolve having correctly") { withTempTable("src") { - Seq(1 -> "a").toDF("i", "j").registerTempTable("src") + Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("src") checkAnswer( sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t HAVING(COUNT(1) > 0)"), Row(1)) @@ -1910,8 +1913,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1, 1, 1, 1) :: Row(1, 2, 2, 1) :: Row(2, 1, 1, 2) :: Row(2, 2, 2, 2) :: Row(3, 1, 1, 3) :: Row(3, 2, 2, 3) :: Nil) - // Try with a registered table. - sql("select struct(a, b) as record from testData2").registerTempTable("structTable") + // Try with a temporary view + sql("select struct(a, b) as record from testData2").createOrReplaceTempView("structTable") checkAnswer( sql("SELECT record.* FROM structTable"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) @@ -1975,9 +1978,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { nestedStructData.select($"record.r1.*"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) - // Try with a registered table + // Try with a temporary view withTempTable("nestedStructTable") { - nestedStructData.registerTempTable("nestedStructTable") + nestedStructData.createOrReplaceTempView("nestedStructTable") checkAnswer( sql("SELECT record.* FROM nestedStructTable"), nestedStructData.select($"record.*")) @@ -2000,7 +2003,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { | (SELECT struct(a, b) as `col$.a_`, struct(b, a) as `a.b.c.` FROM testData2) tmp """.stripMargin) withTempTable("specialCharacterTable") { - specialCharacterPath.registerTempTable("specialCharacterTable") + specialCharacterPath.createOrReplaceTempView("specialCharacterTable") checkAnswer( specialCharacterPath.select($"`r&&b.c`.*"), nestedStructData.select($"record.*")) @@ -2024,7 +2027,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { // Create a data set that contains a naming conflict val nameConflict = sql("SELECT struct(a, b) as nameConflict, a as a FROM testData2") withTempTable("nameConflict") { - nameConflict.registerTempTable("nameConflict") + nameConflict.createOrReplaceTempView("nameConflict") // Unqualified should resolve to table. checkAnswer(sql("SELECT nameConflict.* FROM nameConflict"), Row(Row(1, 1), 1) :: Row(Row(1, 2), 1) :: Row(Row(2, 1), 2) :: Row(Row(2, 2), 2) :: @@ -2328,7 +2331,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-13056: Null in map value causes NPE") { val df = Seq(1 -> Map("abc" -> "somestring", "cba" -> null)).toDF("key", "value") withTempTable("maptest") { - df.registerTempTable("maptest") + df.createOrReplaceTempView("maptest") // local optimization will by pass codegen code, so we should keep the filter `key=1` checkAnswer(sql("SELECT value['abc'] FROM maptest where key = 1"), Row("somestring")) checkAnswer(sql("SELECT value['cba'] FROM maptest where key = 1"), Row(null)) @@ -2338,7 +2341,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("hash function") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") withTempTable("tbl") { - df.registerTempTable("tbl") + df.createOrReplaceTempView("tbl") checkAnswer( df.select(hash($"i", $"j")), sql("SELECT hash(i, j) from tbl") @@ -2390,8 +2393,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df1 = Seq(("one", 1), ("two", 2), ("three", 3)).toDF("k", "v1") val df2 = Seq(("one", 1), ("two", 22), ("one", 5)).toDF("k", "v2") withTempTable("nt1", "nt2") { - df1.registerTempTable("nt1") - df2.registerTempTable("nt2") + df1.createOrReplaceTempView("nt1") + df2.createOrReplaceTempView("nt2") checkAnswer( sql("SELECT * FROM nt1 natural join nt2 where k = \"one\""), Row("one", 1, 1) :: Row("one", 1, 5) :: Nil) @@ -2418,9 +2421,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df3 = Seq((null, "r1c2", "t3r1c3"), ("r2c1", "r2c2", "t3r2c3"), ("r3c1y", "r3c2", "t3r3c3")).toDF("c1", "c2", "c3") withTempTable("t1", "t2", "t3") { - df1.registerTempTable("t1") - df2.registerTempTable("t2") - df3.registerTempTable("t3") + df1.createOrReplaceTempView("t1") + df2.createOrReplaceTempView("t2") + df3.createOrReplaceTempView("t3") // inner join with one using column checkAnswer( sql("SELECT * FROM t1 join t2 using (c1)"), 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 295f02f9a7b5d..491bdb3ef9db9 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 @@ -78,7 +78,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { 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), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3)) - Seq(data).toDF().registerTempTable("reflectData") + Seq(data).toDF().createOrReplaceTempView("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -88,7 +88,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) - Seq(data).toDF().registerTempTable("reflectNullData") + Seq(data).toDF().createOrReplaceTempView("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -96,7 +96,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) - Seq(data).toDF().registerTempTable("reflectOptionalData") + Seq(data).toDF().createOrReplaceTempView("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -104,7 +104,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { // Equality is broken for Arrays, so we test that separately. test("query binary data") { - Seq(ReflectBinary(Array[Byte](1))).toDF().registerTempTable("reflectBinary") + Seq(ReflectBinary(Array[Byte](1))).toDF().createOrReplaceTempView("reflectBinary") val result = sql("SELECT data FROM reflectBinary") .collect().head(0).asInstanceOf[Array[Byte]] @@ -124,7 +124,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), Nested(None, "abc"))) - Seq(data).toDF().registerTempTable("reflectComplexData") + Seq(data).toDF().createOrReplaceTempView("reflectComplexData") assert(sql("SELECT * FROM reflectComplexData").collect().head === Row( Seq(1, 2, 3), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 17ac0c8c6e496..48196927332f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -49,9 +49,9 @@ class SubquerySuite extends QueryTest with SharedSQLContext { protected override def beforeAll(): Unit = { super.beforeAll() - l.registerTempTable("l") - r.registerTempTable("r") - t.registerTempTable("t") + l.createOrReplaceTempView("l") + r.createOrReplaceTempView("r") + t.createOrReplaceTempView("t") } test("simple uncorrelated scalar subquery") { @@ -99,7 +99,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { test("uncorrelated scalar subquery on a DataFrame generated query") { val df = Seq((1, "one"), (2, "two"), (3, "three")).toDF("key", "value") - df.registerTempTable("subqueryData") + df.createOrReplaceTempView("subqueryData") checkAnswer( sql("select (select key from subqueryData where key > 2 order by key limit 1) + 1"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 922154320c399..547d3c1abe858 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -53,7 +53,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { test("SPARK-8003 spark_partition_id") { val df = Seq((1, "Tearing down the walls that divide us")).toDF("id", "saying") - df.registerTempTable("tmp_table") + df.createOrReplaceTempView("tmp_table") checkAnswer(sql("select spark_partition_id() from tmp_table").toDF(), Row(0)) spark.catalog.dropTempView("tmp_table") } @@ -62,7 +62,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { withTempPath { dir => val data = sparkContext.parallelize(0 to 10, 2).toDF("id") data.write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("test_table") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("test_table") val answer = sql("select input_file_name() from test_table").head().getString(0) assert(answer.contains(dir.getCanonicalPath)) assert(sql("select input_file_name() from test_table").distinct().collect().length >= 2) @@ -107,7 +107,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("integerData") + df.createOrReplaceTempView("integerData") val result = sql("SELECT * FROM integerData WHERE oneArgFilter(key)") @@ -119,7 +119,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -138,7 +138,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -158,7 +158,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( 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 3057e016c14c3..7d7b486530de9 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 @@ -95,7 +95,7 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT test("UDTs and UDFs") { spark.udf.register("testType", (d: UDT.MyDenseVector) => d.isInstanceOf[UDT.MyDenseVector]) - pointsRDD.registerTempTable("points") + pointsRDD.createOrReplaceTempView("points") checkAnswer( sql("SELECT testType(features) from points"), Seq(Row(true), Row(true))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index d2e1ea12fd896..2a5295d0d2231 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -78,7 +78,7 @@ class PlannerSuite extends SharedSQLContext { val schema = StructType(fields) val row = Row.fromSeq(Seq.fill(fields.size)(null)) val rowRDD = sparkContext.parallelize(row :: Nil) - spark.createDataFrame(rowRDD, schema).registerTempTable("testLimit") + spark.createDataFrame(rowRDD, schema).createOrReplaceTempView("testLimit") val planned = sql( """ @@ -132,7 +132,7 @@ class PlannerSuite extends SharedSQLContext { test("InMemoryRelation statistics propagation") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "81920") { withTempTable("tiny") { - testData.limit(3).registerTempTable("tiny") + testData.limit(3).createOrReplaceTempView("tiny") sql("CACHE TABLE tiny") val a = testData.as("a") @@ -199,9 +199,9 @@ class PlannerSuite extends SharedSQLContext { test("PartitioningCollection") { withTempTable("normal", "small", "tiny") { - testData.registerTempTable("normal") - testData.limit(10).registerTempTable("small") - testData.limit(3).registerTempTable("tiny") + testData.createOrReplaceTempView("normal") + testData.limit(10).createOrReplaceTempView("small") + testData.limit(3).createOrReplaceTempView("tiny") // Disable broadcast join withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index b31338e82768d..bf3a39c84b3b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -134,7 +134,8 @@ class AggregateBenchmark extends BenchmarkBase { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N) - sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k").registerTempTable("test") + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") + .createOrReplaceTempView("test") def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 2099d4e1b38f6..e2fb91352d58e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("default size avoids broadcast") { // TODO: Improve this test when we have better statistics sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)) - .toDF().registerTempTable("sizeTst") + .toDF().createOrReplaceTempView("sizeTst") spark.catalog.cacheTable("sizeTst") assert( spark.table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > @@ -92,7 +92,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-2729 regression: timestamp data type") { val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time") - timestamps.registerTempTable("timestamps") + timestamps.createOrReplaceTempView("timestamps") checkAnswer( sql("SELECT time FROM timestamps"), @@ -133,7 +133,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(df.schema.head.dataType === DecimalType(15, 10)) - df.cache().registerTempTable("test_fixed_decimal") + df.cache().createOrReplaceTempView("test_fixed_decimal") checkAnswer( sql("SELECT * FROM test_fixed_decimal"), (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) @@ -179,7 +179,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { (i to i + 10).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, Row((i - 0.25).toFloat, Seq(true, false, null))) } - spark.createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") + spark.createDataFrame(rdd, schema).createOrReplaceTempView("InMemoryCache_different_data_types") // Cache the table. sql("cache table InMemoryCache_different_data_types") // Make sure the table is indeed cached. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index 48c798986b270..a118cec0bbb0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -63,7 +63,7 @@ class PartitionBatchPruningSuite val string = if (((key - 1) / 10) % 2 == 0) null else key.toString TestData(key, string) }, 5).toDF() - pruningData.registerTempTable("pruningData") + pruningData.createOrReplaceTempView("pruningData") spark.catalog.cacheTable("pruningData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 63fe4658d67d5..46213a22ed3d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -239,7 +239,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("nullstr", StringType, true):: Nil) assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select nullstr, headers.Host from jsonTable"), @@ -261,7 +261,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -302,7 +302,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -376,7 +376,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("GetField operation on complex data type") { val jsonDF = spark.read.json(complexFieldAndType1) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -403,7 +403,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -464,7 +464,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ignore("Type conflict in primitive field values (Ignored)") { val jsonDF = spark.read.json(primitiveFieldValueTypeConflict) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Right now, the analyzer does not promote strings in a boolean expression. // Number and Boolean conflict: resolve the type as boolean in this query. @@ -528,7 +528,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -550,7 +550,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -580,7 +580,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") } test("Loading a JSON dataset from a text file") { @@ -601,7 +601,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -633,7 +633,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -674,7 +674,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -759,7 +759,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -885,7 +885,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(schema === jsonDF1.schema) - jsonDF1.registerTempTable("jsonTable1") + jsonDF1.createOrReplaceTempView("jsonTable1") checkAnswer( sql("select * from jsonTable1"), @@ -902,7 +902,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(schema === jsonDF2.schema) - jsonDF2.registerTempTable("jsonTable2") + jsonDF2.createOrReplaceTempView("jsonTable2") checkAnswer( sql("select * from jsonTable2"), @@ -921,7 +921,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) - jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap") + jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") checkAnswer( sql("select `map` from jsonWithSimpleMap"), @@ -949,7 +949,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) - jsonWithComplexMap.registerTempTable("jsonWithComplexMap") + jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") checkAnswer( sql("select `map` from jsonWithComplexMap"), @@ -974,7 +974,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-2096 Correctly parse dot notations") { val jsonDF = spark.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -992,7 +992,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-3390 Complex arrays") { val jsonDF = spark.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -1015,7 +1015,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-3308 Read top level JSON arrays") { val jsonDF = spark.read.json(jsonArray) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -1084,7 +1084,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { withTempTable("jsonTable") { val jsonDF = spark.read.json(corruptRecords) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( StructField("_unparsed", StringType, true) :: StructField("a", StringType, true) :: @@ -1156,7 +1156,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-4068: nulls in arrays") { val jsonDF = spark.read.json(nullsInArrays) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( StructField("field1", @@ -1202,7 +1202,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val df1 = spark.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + df1.createOrReplaceTempView("applySchema1") val df2 = df1.toDF val result = df2.toJSON.collect() // scalastyle:off @@ -1225,7 +1225,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val df3 = spark.createDataFrame(rowRDD2, schema2) - df3.registerTempTable("applySchema2") + df3.createOrReplaceTempView("applySchema2") val df4 = df3.toDF val result2 = df4.toJSON.collect() @@ -1234,7 +1234,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val jsonDF = spark.read.json(primitiveFieldAndType) val primTable = spark.read.json(jsonDF.toJSON.rdd) - primTable.registerTempTable("primitiveTable") + primTable.createOrReplaceTempView("primitiveTable") checkAnswer( sql("select * from primitiveTable"), Row(new java.math.BigDecimal("92233720368547758070"), @@ -1247,7 +1247,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val complexJsonDF = spark.read.json(complexFieldAndType1) val compTable = spark.read.json(complexJsonDF.toJSON.rdd) - compTable.registerTempTable("complexTable") + compTable.createOrReplaceTempView("complexTable") // Access elements of a primitive array. checkAnswer( sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from complexTable"), @@ -1387,7 +1387,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { "col1", "abd") - spark.read.json(root.getAbsolutePath).registerTempTable("test_myjson_with_part") + spark.read.json(root.getAbsolutePath).createOrReplaceTempView("test_myjson_with_part") checkAnswer(sql( "SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abc'"), Row(4)) checkAnswer(sql( @@ -1531,7 +1531,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { { val jsonDF = spark.read.schema(schema).json(additionalCorruptRecords) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // In HiveContext, backticks should be used to access columns starting with a underscore. checkAnswer( @@ -1639,7 +1639,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val schema = (new StructType).add("ts", TimestampType) val jsonDF = spark.read.schema(schema).json(timestampAsLong) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select ts from jsonTable"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 8707e1346144e..847ea6bd523d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -400,7 +400,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha // Introduce _temporary dir to the base dir the robustness of the schema discovery process. new File(base.getCanonicalPath, "_temporary").mkdir() - spark.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -484,7 +484,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - spark.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -533,7 +533,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + parquetRelation.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -573,7 +573,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + parquetRelation.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -609,7 +609,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha .option("mergeSchema", "true") .format("parquet") .load(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index f9f9f80352128..725e14c0fb2af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -46,7 +46,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("appending") { val data = (0 until 10).map(i => (i, i.toString)) - spark.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") // Query appends, don't test with both read modes. withParquetTable(data, "t", false) { sql("INSERT INTO TABLE t SELECT * FROM tmp") @@ -58,7 +58,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) - spark.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withParquetTable(data, "t") { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(spark.table("t"), data.map(Row.fromTuple)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala index 69a600a55b128..487d7a7e5ac88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala @@ -75,10 +75,10 @@ object ParquetReadBenchmark { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast(id as INT) as id from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") sqlBenchmark.addCase("SQL Parquet Vectorized") { iter => spark.sql("select sum(id) from tempTable").collect() @@ -159,10 +159,10 @@ object ParquetReadBenchmark { def intStringScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast(id as INT) as c1, cast(id as STRING) as c2 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("Int and String Scan", values) @@ -193,10 +193,10 @@ object ParquetReadBenchmark { def stringDictionaryScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select cast((id % 200) + 10000 as STRING) as c1 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("String Dictionary", values) @@ -225,10 +225,10 @@ object ParquetReadBenchmark { def partitionTableScanBenchmark(values: Int): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql("select id % 2 as p, cast(id as INT) as id from t1") .write.partitionBy("p").parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("Partitioned Table", values) @@ -260,11 +260,11 @@ object ParquetReadBenchmark { def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { withTempPath { dir => withTempTable("t1", "tempTable") { - spark.range(values).registerTempTable("t1") + spark.range(values).createOrReplaceTempView("t1") spark.sql(s"select IF(rand(1) < $fractionOfNulls, NULL, cast(id as STRING)) as c1, " + s"IF(rand(2) < $fractionOfNulls, NULL, cast(id as STRING)) as c2 from t1") .write.parquet(dir.getCanonicalPath) - spark.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") val benchmark = new Benchmark("String with Nulls Scan", values) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala index 08b7eb3cf7b2f..228ae6f840ccf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala @@ -1187,7 +1187,7 @@ object TPCDSBenchmark { def setupTables(dataLocation: String): Map[String, Long] = { tables.map { tableName => - spark.read.parquet(s"$dataLocation/$tableName").registerTempTable(tableName) + spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) tableName -> spark.table(tableName).count() }.toMap } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 1b827694283e6..08f596f130aa2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -165,7 +165,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // Because SortMergeJoin may skip different rows if the number of partitions is different, this // test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -183,7 +183,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // Because SortMergeJoin may skip different rows if the number of partitions is different, // this test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -237,7 +237,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("BroadcastNestedLoopJoin metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) @@ -265,7 +265,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("CartesianProduct metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") + testDataForJoin.createOrReplaceTempView("testDataForJoin") withTempTable("testDataForJoin") { // Assume the execution plan is // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) 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 44d1b9ddda66a..9c9abfeb2a637 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 @@ -441,7 +441,7 @@ class JDBCSuite extends SparkFunSuite test("test DATE types in cache") { val rows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties).collect() spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) - .cache().registerTempTable("mycached_date") + .cache().createOrReplaceTempView("mycached_date") val cachedRows = sql("select * from mycached_date").collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index c1dc9b983407f..03c18ad009d49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -34,7 +34,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") } override def afterAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 5ac39f54b91ce..854fec5b22f77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -31,7 +31,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") sql( s""" |CREATE TEMPORARY TABLE jsonTable (a int, b string) @@ -111,7 +111,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to less part files. val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 5) - caseInsensitiveContext.read.json(rdd1).registerTempTable("jt1") + caseInsensitiveContext.read.json(rdd1).createOrReplaceTempView("jt1") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt1 @@ -123,7 +123,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to more part files. val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 10) - caseInsensitiveContext.read.json(rdd2).registerTempTable("jt2") + caseInsensitiveContext.read.json(rdd2).createOrReplaceTempView("jt2") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 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 bb2c54aa64977..7738e4107df94 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 @@ -42,7 +42,7 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) df = caseInsensitiveContext.read.json(rdd) - df.registerTempTable("jsonTable") + df.createOrReplaceTempView("jsonTable") } override def afterAll(): Unit = { @@ -123,7 +123,7 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA // verify the append mode df.write.mode(SaveMode.Append).json(path.toString) val df2 = df.union(df) - df2.registerTempTable("jsonTable2") + df2.createOrReplaceTempView("jsonTable2") checkLoad(df2, "jsonTable2") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 013b7316938eb..b742206b58137 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -77,7 +77,7 @@ class StreamSuite extends StreamTest with SharedSQLContext { test("sql queries") { val inputData = MemoryStream[Int] - inputData.toDF().registerTempTable("stream") + inputData.toDF().createOrReplaceTempView("stream") val evens = sql("SELECT * FROM stream WHERE value % 2 = 0") testStream(evens)( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 03369c5a48272..421f6bca7f865 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -41,14 +41,14 @@ private[sql] trait SQLTestData { self => protected lazy val emptyTestData: DataFrame = { val df = spark.sparkContext.parallelize( Seq.empty[Int].map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("emptyTestData") + df.createOrReplaceTempView("emptyTestData") df } protected lazy val testData: DataFrame = { val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("testData") + df.createOrReplaceTempView("testData") df } @@ -60,7 +60,7 @@ private[sql] trait SQLTestData { self => TestData2(2, 2) :: TestData2(3, 1) :: TestData2(3, 2) :: Nil, 2).toDF() - df.registerTempTable("testData2") + df.createOrReplaceTempView("testData2") df } @@ -68,14 +68,14 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( TestData3(1, None) :: TestData3(2, Some(2)) :: Nil).toDF() - df.registerTempTable("testData3") + df.createOrReplaceTempView("testData3") df } protected lazy val negativeData: DataFrame = { val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(-i, (-i).toString))).toDF() - df.registerTempTable("negativeData") + df.createOrReplaceTempView("negativeData") df } @@ -87,7 +87,7 @@ private[sql] trait SQLTestData { self => LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: LargeAndSmallInts(3, 2) :: Nil).toDF() - df.registerTempTable("largeAndSmallInts") + df.createOrReplaceTempView("largeAndSmallInts") df } @@ -99,7 +99,7 @@ private[sql] trait SQLTestData { self => DecimalData(2, 2) :: DecimalData(3, 1) :: DecimalData(3, 2) :: Nil).toDF() - df.registerTempTable("decimalData") + df.createOrReplaceTempView("decimalData") df } @@ -110,7 +110,7 @@ private[sql] trait SQLTestData { self => BinaryData("122".getBytes(StandardCharsets.UTF_8), 3) :: BinaryData("121".getBytes(StandardCharsets.UTF_8), 2) :: BinaryData("123".getBytes(StandardCharsets.UTF_8), 4) :: Nil).toDF() - df.registerTempTable("binaryData") + df.createOrReplaceTempView("binaryData") df } @@ -122,7 +122,7 @@ private[sql] trait SQLTestData { self => UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: UpperCaseData(6, "F") :: Nil).toDF() - df.registerTempTable("upperCaseData") + df.createOrReplaceTempView("upperCaseData") df } @@ -132,7 +132,7 @@ private[sql] trait SQLTestData { self => LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil).toDF() - df.registerTempTable("lowerCaseData") + df.createOrReplaceTempView("lowerCaseData") df } @@ -140,7 +140,7 @@ private[sql] trait SQLTestData { self => val rdd = spark.sparkContext.parallelize( ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: ArrayData(Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) - rdd.toDF().registerTempTable("arrayData") + rdd.toDF().createOrReplaceTempView("arrayData") rdd } @@ -151,13 +151,13 @@ private[sql] trait SQLTestData { self => MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - rdd.toDF().registerTempTable("mapData") + rdd.toDF().createOrReplaceTempView("mapData") rdd } protected lazy val repeatedData: RDD[StringData] = { val rdd = spark.sparkContext.parallelize(List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("repeatedData") + rdd.toDF().createOrReplaceTempView("repeatedData") rdd } @@ -165,7 +165,7 @@ private[sql] trait SQLTestData { self => val rdd = spark.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("nullableRepeatedData") + rdd.toDF().createOrReplaceTempView("nullableRepeatedData") rdd } @@ -175,7 +175,7 @@ private[sql] trait SQLTestData { self => NullInts(2) :: NullInts(3) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("nullInts") + df.createOrReplaceTempView("nullInts") df } @@ -185,7 +185,7 @@ private[sql] trait SQLTestData { self => NullInts(null) :: NullInts(null) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("allNulls") + df.createOrReplaceTempView("allNulls") df } @@ -194,13 +194,13 @@ private[sql] trait SQLTestData { self => NullStrings(1, "abc") :: NullStrings(2, "ABC") :: NullStrings(3, null) :: Nil).toDF() - df.registerTempTable("nullStrings") + df.createOrReplaceTempView("nullStrings") df } protected lazy val tableName: DataFrame = { val df = spark.sparkContext.parallelize(TableName("test") :: Nil).toDF() - df.registerTempTable("tableName") + df.createOrReplaceTempView("tableName") df } @@ -215,7 +215,7 @@ private[sql] trait SQLTestData { self => // An RDD with 4 elements and 8 partitions protected lazy val withEmptyParts: RDD[IntField] = { val rdd = spark.sparkContext.parallelize((1 to 4).map(IntField), 8) - rdd.toDF().registerTempTable("withEmptyParts") + rdd.toDF().createOrReplaceTempView("withEmptyParts") rdd } @@ -223,7 +223,7 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( Person(0, "mike", 30) :: Person(1, "jim", 20) :: Nil).toDF() - df.registerTempTable("person") + df.createOrReplaceTempView("person") df } @@ -231,7 +231,7 @@ private[sql] trait SQLTestData { self => val df = spark.sparkContext.parallelize( Salary(0, 2000.0) :: Salary(1, 1000.0) :: Nil).toDF() - df.registerTempTable("salary") + df.createOrReplaceTempView("salary") df } @@ -240,7 +240,7 @@ private[sql] trait SQLTestData { self => ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1, 1, 1), true) :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2, 2, 2), false) :: Nil).toDF() - df.registerTempTable("complexData") + df.createOrReplaceTempView("complexData") df } @@ -251,7 +251,7 @@ private[sql] trait SQLTestData { self => CourseSales("dotNET", 2012, 5000) :: CourseSales("dotNET", 2013, 48000) :: CourseSales("Java", 2013, 30000) :: Nil).toDF() - df.registerTempTable("courseSales") + df.createOrReplaceTempView("courseSales") df } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java index 64f2ded447a06..f664d5a4cdada 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -57,7 +57,7 @@ public void setUp() throws IOException { jsonObjects.add("{\"key\":" + i + ", \"value\":\"str" + i + "\"}"); } df = hc.read().json(sc.parallelize(jsonObjects)); - df.registerTempTable("window_table"); + df.createOrReplaceTempView("window_table"); } @After diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index f13c32db9d230..e73117c8144ce 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -85,7 +85,7 @@ public void setUp() throws IOException { } JavaRDD rdd = sc.parallelize(jsonObjects); df = sqlContext.read().json(rdd); - df.registerTempTable("jsonTable"); + df.createOrReplaceTempView("jsonTable"); } @After 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 d96eb0169ee8f..d2cb62c617d44 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 @@ -33,8 +33,8 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd if (spark.wrapped.tableNames().contains("src")) { spark.catalog.dropTempView("src") } - Seq((1, "")).toDF("key", "value").registerTempTable("src") - Seq((1, 1, 1)).toDF("a", "a", "b").registerTempTable("dupAttributes") + Seq((1, "")).toDF("key", "value").createOrReplaceTempView("src") + Seq((1, 1, 1)).toDF("a", "a", "b").createOrReplaceTempView("dupAttributes") } override protected def afterEach(): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index b5af758a65b1c..e2304b5397ca1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -51,7 +51,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton test("Converting Hive to Parquet Table via saveAsParquetFile") { withTempPath { dir => sql("SELECT * FROM src").write.parquet(dir.getCanonicalPath) - hiveContext.read.parquet(dir.getCanonicalPath).registerTempTable("p") + hiveContext.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("p") withTempTable("p") { checkAnswer( sql("SELECT * FROM src ORDER BY key"), @@ -65,7 +65,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton withParquetTable((1 to 10).map(i => (i, s"val_$i")), "t", false) { withTempPath { file => sql("SELECT * FROM t LIMIT 1").write.parquet(file.getCanonicalPath) - hiveContext.read.parquet(file.getCanonicalPath).registerTempTable("p") + hiveContext.read.parquet(file.getCanonicalPath).createOrReplaceTempView("p") withTempTable("p") { // let's do three overwrites for good measure sql("INSERT OVERWRITE TABLE p SELECT * FROM t") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index d05a3623ae01f..a4bbe96cf8057 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -355,7 +355,7 @@ object TemporaryHiveUDFTest extends Logging { """.stripMargin) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Actually use the loaded UDF. logInfo("Using the UDF.") val result = hiveContext.sql( @@ -393,7 +393,7 @@ object PermanentHiveUDFTest1 extends Logging { """.stripMargin) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Actually use the loaded UDF. logInfo("Using the UDF.") val result = hiveContext.sql( @@ -429,7 +429,7 @@ object PermanentHiveUDFTest2 extends Logging { hiveContext.sessionState.catalog.createFunction(function, ignoreIfExists = false) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Actually use the loaded UDF. logInfo("Using the UDF.") val result = hiveContext.sql( @@ -491,7 +491,7 @@ object SparkSubmitClassLoaderTest extends Logging { """.stripMargin) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Load a Hive SerDe from the jar. logInfo("Creating a Hive table with a SerDe provided in a jar.") hiveContext.sql( 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 883cdac110e0b..b25684562075c 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 @@ -45,8 +45,8 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef // Since every we are doing tests for DDL statements, // it is better to reset before every test. hiveContext.reset() - // Register the testData, which will be used in every test. - testData.registerTempTable("testData") + // Creates a temporary view with testData, which will be used in all tests. + testData.createOrReplaceTempView("testData") } test("insertInto() HiveTable") { @@ -98,7 +98,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef val rowRDD = hiveContext.sparkContext.parallelize( (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithMapValue") + df.createOrReplaceTempView("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m MAP )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -171,7 +171,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef StructField("a", ArrayType(StringType, containsNull = false)))) val rowRDD = hiveContext.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithArrayValue") + df.createOrReplaceTempView("tableWithArrayValue") sql("CREATE TABLE hiveTableWithArrayValue(a Array )") sql("INSERT OVERWRITE TABLE hiveTableWithArrayValue SELECT a FROM tableWithArrayValue") @@ -188,7 +188,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef val rowRDD = hiveContext.sparkContext.parallelize( (1 to 100).map(i => Row(Map(s"key$i" -> s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithMapValue") + df.createOrReplaceTempView("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m Map )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -205,7 +205,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef val rowRDD = hiveContext.sparkContext.parallelize( (1 to 100).map(i => Row(Row(s"value$i")))) val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithStructValue") + df.createOrReplaceTempView("tableWithStructValue") sql("CREATE TABLE hiveTableWithStructValue(s Struct )") sql("INSERT OVERWRITE TABLE hiveTableWithStructValue SELECT s FROM tableWithStructValue") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index b507018e58d1d..00adb9a44b14e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -80,7 +80,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM jsonTable"), sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM expectedJsonTable")) @@ -110,7 +110,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert(expectedSchema === table("jsonTable").schema) withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT b, ``.`=` FROM jsonTable"), sql("SELECT b, ``.`=` FROM expectedJsonTable")) @@ -248,7 +248,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT * FROM jsonTable"), @@ -554,7 +554,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("scan a parquet table created through a CTAS statement") { withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { withTempTable("jt") { - (1 to 10).map(i => i -> s"str$i").toDF("a", "b").registerTempTable("jt") + (1 to 10).map(i => i -> s"str$i").toDF("a", "b").createOrReplaceTempView("jt") withTable("test_parquet_ctas") { sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index 3f6418cbe85dc..ac89bbbf8e19d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -74,7 +74,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi val schema = spark.table("parquet_compat").schema val rowRDD = spark.sparkContext.parallelize(rows).coalesce(1) - spark.createDataFrame(rowRDD, schema).registerTempTable("data") + spark.createDataFrame(rowRDD, schema).createOrReplaceTempView("data") spark.sql("INSERT INTO TABLE parquet_compat SELECT * FROM data") } } 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 78569c58085cd..cc05e56d66164 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 @@ -32,7 +32,7 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl withSQLConf((SQLConf.HIVE_VERIFY_PARTITION_PATH.key, "true")) { val testData = sparkContext.parallelize( (1 to 10).map(i => TestData(i, i.toString))).toDF() - testData.registerTempTable("testData") + testData.createOrReplaceTempView("testData") val tmpDir = Files.createTempDir() // create the table for test 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 8060ef77e7586..7011cd81221c5 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 @@ -115,7 +115,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { sql("DROP TABLE analyzeTable_part").collect() // Try to analyze a temp table - sql("""SELECT * FROM src""").registerTempTable("tempTable") + sql("""SELECT * FROM src""").createOrReplaceTempView("tempTable") intercept[UnsupportedOperationException] { hiveContext.sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala index d1aa5aa931947..d121bcbe15b35 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -53,7 +53,7 @@ class UDFSuite sql("USE default") testDF = (1 to 10).map(i => s"sTr$i").toDF("value") - testDF.registerTempTable(testTableName) + testDF.createOrReplaceTempView(testTableName) expectedDF = (1 to 10).map(i => s"STR$i").toDF("value") super.beforeAll() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index c97b3f3197588..a2bae2e81f30a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -180,7 +180,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te val emptyDF = spark.createDataFrame( sparkContext.emptyRDD[Row], StructType(StructField("key", StringType) :: StructField("value", IntegerType) :: Nil)) - emptyDF.registerTempTable("emptyTable") + emptyDF.createOrReplaceTempView("emptyTable") // Register UDAFs spark.udf.register("mydoublesum", new MyDoubleSum) @@ -200,7 +200,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te } test("group by function") { - Seq((1, 2)).toDF("a", "b").registerTempTable("data") + Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("data") checkAnswer( sql("SELECT floor(a) AS a, collect_set(b) FROM data GROUP BY floor(a) ORDER BY a"), @@ -783,7 +783,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te (5, 8, 17), (6, 2, 11)).toDF("a", "b", "c") - covar_tab.registerTempTable("covar_tab") + covar_tab.createOrReplaceTempView("covar_tab") checkAnswer( spark.sql( @@ -938,7 +938,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te spark.createDataFrame( sparkContext.parallelize(data, 2), schema) - .registerTempTable("noInputSchemaUDAF") + .createOrReplaceTempView("noInputSchemaUDAF") checkAnswer( spark.sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 17422ca1a02d8..131b06aec8c9c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -80,7 +80,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") { withTempTable("jt") { val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - hiveContext.read.json(rdd).registerTempTable("jt") + hiveContext.read.json(rdd).createOrReplaceTempView("jt") val outputs = sql( s""" |EXPLAIN EXTENDED diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala index b252c6ee2faae..4d2f190b8e6f1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala @@ -29,8 +29,8 @@ class HiveOperatorQueryableSuite extends QueryTest with TestHiveSingleton { test("SPARK-5324 query result of describe command") { hiveContext.loadTestTable("src") - // register a describe command to be a temp table - sql("desc src").registerTempTable("mydesc") + // Creates a temporary view with the output of a describe command + sql("desc src").createOrReplaceTempView("mydesc") checkAnswer( sql("desc mydesc"), Seq( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala index d8d3448adde0b..78c0d1f97e7da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -28,7 +28,7 @@ class HivePlanTest extends QueryTest with TestHiveSingleton { import hiveContext.implicits._ test("udf constant folding") { - Seq.empty[Tuple1[Int]].toDF("a").registerTempTable("t") + Seq.empty[Tuple1[Int]].toDF("a").createOrReplaceTempView("t") val optimized = sql("SELECT cos(null) AS c FROM t").queryExecution.optimizedPlan val correctAnswer = sql("SELECT cast(null as double) AS c FROM t").queryExecution.optimizedPlan 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 19f8cb3877b32..2aaaaadb6afa9 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 @@ -685,12 +685,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("case sensitivity when query Hive table", "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") - test("case sensitivity: registered table") { + test("case sensitivity: created temporary view") { val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) - testData.toDF().registerTempTable("REGisteredTABle") + testData.toDF().createOrReplaceTempView("REGisteredTABle") assertResult(Array(Row(2, "str2"))) { sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + @@ -715,7 +715,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case ((value, attr), key) => HavingRow(key, value, attr)} - TestHive.sparkContext.parallelize(fixture).toDF().registerTempTable("having_test") + TestHive.sparkContext.parallelize(fixture).toDF().createOrReplaceTempView("having_test") val results = sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() @@ -819,12 +819,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .collect() } - // Describe a registered temporary table. + // Describe a temporary view. val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) - testData.toDF().registerTempTable("test_describe_commands2") + testData.toDF().createOrReplaceTempView("test_describe_commands2") assertResult( Array( @@ -996,9 +996,9 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } - test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { - sparkContext.makeRDD(Seq.empty[LogEntry]).toDF().registerTempTable("rawLogs") - sparkContext.makeRDD(Seq.empty[LogFile]).toDF().registerTempTable("logFiles") + test("SPARK-3414 regression: should store analyzed logical plan when creating a temporary view") { + sparkContext.makeRDD(Seq.empty[LogEntry]).toDF().createOrReplaceTempView("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).toDF().createOrReplaceTempView("logFiles") sql( """ @@ -1009,7 +1009,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { FROM logFiles ) files ON rawLogs.filename = files.name - """).registerTempTable("boom") + """).createOrReplaceTempView("boom") // This should be successfully analyzed sql("SELECT * FROM boom").queryExecution.analyzed 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 dd13b8392880a..b2f19d7753956 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 @@ -32,14 +32,14 @@ class HiveResolutionSuite extends HiveComparisonTest { test("SPARK-3698: case insensitive test for nested data") { read.json(sparkContext.makeRDD( - """{"a": [{"a": {"a": 1}}]}""" :: Nil)).registerTempTable("nested") + """{"a": [{"a": {"a": 1}}]}""" :: Nil)).createOrReplaceTempView("nested") // This should be successfully analyzed sql("SELECT a[0].A.A from nested").queryExecution.analyzed } test("SPARK-5278: check ambiguous reference to fields") { read.json(sparkContext.makeRDD( - """{"a": [{"b": 1, "B": 2}]}""" :: Nil)).registerTempTable("nested") + """{"a": [{"b": 1, "B": 2}]}""" :: Nil)).createOrReplaceTempView("nested") // there are 2 filed matching field name "b", we should report Ambiguous reference error val exception = intercept[AnalysisException] { @@ -78,7 +78,7 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1, 2), Seq(Nested(1, 2))) :: Nil) - .toDF().registerTempTable("caseSensitivityTest") + .toDF().createOrReplaceTempView("caseSensitivityTest") val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") assert(query.schema.fields.map(_.name) === Seq("a", "b", "A", "B", "a", "b", "A", "B"), @@ -89,14 +89,14 @@ class HiveResolutionSuite extends HiveComparisonTest { ignore("case insensitivity with scala reflection joins") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1, 2), Seq(Nested(1, 2))) :: Nil) - .toDF().registerTempTable("caseSensitivityTest") + .toDF().createOrReplaceTempView("caseSensitivityTest") sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") { sparkContext.parallelize(Data(1, 2, Nested(1, 2), Seq(Nested(1, 2))) :: Nil) - .toDF().registerTempTable("nestedRepeatedTest") + .toDF().createOrReplaceTempView("nestedRepeatedTest") assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 8c9c37fecec48..60f8be5e0e825 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -84,7 +84,7 @@ class HiveTableScanSuite extends HiveComparisonTest { sql("""insert into table spark_4959 select "hi" from src limit 1""") table("spark_4959").select( 'col1.as("CaseSensitiveColName"), - 'col1.as("CaseSensitiveColName2")).registerTempTable("spark_4959_2") + 'col1.as("CaseSensitiveColName2")).createOrReplaceTempView("spark_4959_2") assert(sql("select CaseSensitiveColName from spark_4959_2").head() === Row("hi")) assert(sql("select casesensitivecolname from spark_4959_2").head() === Row("hi")) 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 521964eb4eb58..23b7f6c75b486 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 @@ -153,7 +153,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFIntegerToString") { val testData = hiveContext.sparkContext.parallelize( IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() - testData.registerTempTable("integerTable") + testData.createOrReplaceTempView("integerTable") val udfName = classOf[UDFIntegerToString].getName sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '$udfName'") @@ -167,7 +167,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToListString") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListString AS '${classOf[UDFToListString].getName}'") val errMsg = intercept[AnalysisException] { @@ -182,7 +182,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToListInt") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListInt AS '${classOf[UDFToListInt].getName}'") val errMsg = intercept[AnalysisException] { @@ -197,7 +197,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToStringIntMap") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToStringIntMap " + s"AS '${classOf[UDFToStringIntMap].getName}'") @@ -213,7 +213,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFToIntIntMap") { val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToIntIntMap " + s"AS '${classOf[UDFToIntIntMap].getName}'") @@ -232,7 +232,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() - testData.registerTempTable("listListIntTable") + testData.createOrReplaceTempView("listListIntTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") checkAnswer( @@ -247,7 +247,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { val testData = hiveContext.sparkContext.parallelize( ListStringCaseClass(Seq("a", "b", "c")) :: ListStringCaseClass(Seq("d", "e")) :: Nil).toDF() - testData.registerTempTable("listStringTable") + testData.createOrReplaceTempView("listStringTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") checkAnswer( @@ -261,7 +261,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("UDFStringString") { val testData = hiveContext.sparkContext.parallelize( StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF() - testData.registerTempTable("stringTable") + testData.createOrReplaceTempView("stringTable") sql(s"CREATE TEMPORARY FUNCTION testStringStringUDF AS '${classOf[UDFStringString].getName}'") checkAnswer( @@ -283,7 +283,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() - testData.registerTempTable("TwoListTable") + testData.createOrReplaceTempView("TwoListTable") sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") checkAnswer( @@ -295,7 +295,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("Hive UDFs with insufficient number of input arguments should trigger an analysis error") { - Seq((1, 2)).toDF("a", "b").registerTempTable("testUDF") + Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("testUDF") { // HiveSimpleUDF @@ -352,7 +352,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("Hive UDF in group by") { withTempTable("tab1") { - Seq(Tuple1(1451400761)).toDF("test_date").registerTempTable("tab1") + Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") val count = sql("select testUDFToDate(cast(test_date as timestamp))" + " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() 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 ac9a3930fd21b..81f3ea8a6e801 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 @@ -102,14 +102,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-6835: udtf in lateral view") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("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-13651: generator outputs shouldn't be resolved from its child's output") { withTempTable("src") { - Seq(("id1", "value1")).toDF("key", "value").registerTempTable("src") + Seq(("id1", "value1")).toDF("key", "value").createOrReplaceTempView("src") val query = sql("SELECT genoutput.* FROM src " + "LATERAL VIEW explode(map('key1', 100, 'key2', 200)) genoutput AS key, value") @@ -135,8 +135,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { 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") + orders.toDF.createOrReplaceTempView("orders1") + orderUpdates.toDF.createOrReplaceTempView("orderupdates1") sql( """CREATE TABLE orders( @@ -305,7 +305,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-5371: union with null and sum") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("table1") val query = sql( """ @@ -329,7 +329,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("CTAS with WITH clause") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("table1") sql( """ @@ -346,7 +346,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("explode nested Field") { - Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.registerTempTable("nestedArray") + Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.createOrReplaceTempView("nestedArray") checkAnswer( sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"), Row(1) :: Row(2) :: Row(3) :: Nil) @@ -543,7 +543,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("specifying the column list for CTAS") { - Seq((1, "111111"), (2, "222222")).toDF("key", "value").registerTempTable("mytable1") + Seq((1, "111111"), (2, "222222")).toDF("key", "value").createOrReplaceTempView("mytable1") sql("create table gen__tmp(a int, b string) as select key, value from mytable1") checkAnswer( @@ -598,7 +598,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("double nested data") { sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil) - .toDF().registerTempTable("nested") + .toDF().createOrReplaceTempView("nested") checkAnswer( sql("SELECT f1.f2.f3 FROM nested"), Row(1)) @@ -682,7 +682,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-4963 DataFrame sample on mutable row return wrong result") { sql("SELECT * FROM src WHERE key % 2 = 0") .sample(withReplacement = false, fraction = 0.3) - .registerTempTable("sampled") + .createOrReplaceTempView("sampled") (1 to 10).foreach { i => checkAnswer( sql("SELECT * FROM sampled WHERE key % 2 = 1"), @@ -707,7 +707,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val rowRdd = sparkContext.parallelize(row :: Nil) - hiveContext.createDataFrame(rowRdd, schema).registerTempTable("testTable") + hiveContext.createDataFrame(rowRdd, schema).createOrReplaceTempView("testTable") sql( """CREATE TABLE nullValuesInInnerComplexTypes @@ -733,14 +733,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-4296 Grouping field with Hive UDF as sub expression") { val rdd = sparkContext.makeRDD( """{"a": "str", "b":"1", "c":"1970-01-01 00:00:00"}""" :: Nil) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer( sql("SELECT concat(a, '-', b), year(c) FROM data GROUP BY concat(a, '-', b), year(c)"), Row("str-1", 1970)) dropTempTable("data") - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer(sql("SELECT year(c) + 1 FROM data GROUP BY year(c) + 1"), Row(1971)) dropTempTable("data") @@ -748,14 +748,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("resolve udtf in projection #1") { val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") val df = sql("SELECT explode(a) AS val FROM data") val col = df("val") } test("resolve udtf in projection #2") { val rdd = sparkContext.makeRDD((1 to 2).map(i => s"""{"a":[$i, ${i + 1}]}""")) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer(sql("SELECT explode(map(1, 1)) FROM data LIMIT 1"), Row(1, 1) :: Nil) checkAnswer(sql("SELECT explode(map(1, 1)) as (k1, k2) FROM data LIMIT 1"), Row(1, 1) :: Nil) intercept[AnalysisException] { @@ -770,7 +770,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // TGF with non-TGF in project is allowed in Spark SQL, but not in Hive test("TGF with non-TGF in projection") { val rdd = sparkContext.makeRDD( """{"a": "1", "b":"1"}""" :: Nil) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") checkAnswer( sql("SELECT explode(map(a, b)) as (k1, k2), a, b FROM data"), Row("1", "1", "1", "1") :: Nil) @@ -784,7 +784,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // 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}]}""")) - read.json(rdd).registerTempTable("data") + read.json(rdd).createOrReplaceTempView("data") val originalConf = sessionState.convertCTAS setConf(HiveUtils.CONVERT_CTAS, false) @@ -824,7 +824,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Seq.empty[(java.math.BigDecimal, java.math.BigDecimal)] .toDF("d1", "d2") .select($"d1".cast(DecimalType(10, 5)).as("d")) - .registerTempTable("dn") + .createOrReplaceTempView("dn") sql("select d from dn union all select d * 2 from dn") .queryExecution.analyzed @@ -832,27 +832,27 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("Star Expansion - script transform") { val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(100000 === sql("SELECT TRANSFORM (*) USING 'cat' FROM script_trans").count()) } test("test script transform for stdout") { val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(100000 === sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans").count()) } test("test script transform for stderr") { val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(0 === sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat 1>&2' AS (a,b,c) FROM script_trans").count()) } test("test script transform data type") { val data = (1 to 5).map { i => (i, i) } - data.toDF("key", "value").registerTempTable("test") + data.toDF("key", "value").createOrReplaceTempView("test") checkAnswer( sql("""FROM |(FROM test SELECT TRANSFORM(key, value) USING 'cat' AS (`thing1` int, thing2 string)) t @@ -864,7 +864,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { withTempTable("data") { spark.range(1, 5) .select(array($"id", $"id" + 1).as("a"), $"id".as("b"), (lit(10) - $"id").as("c")) - .registerTempTable("data") + .createOrReplaceTempView("data") // case 1: missing sort columns are resolvable if join is true checkAnswer( @@ -888,7 +888,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("test case key when") { - (1 to 5).map(i => (i, i.toString)).toDF("k", "v").registerTempTable("t") + (1 to 5).map(i => (i, i.toString)).toDF("k", "v").createOrReplaceTempView("t") checkAnswer( sql("SELECT CASE k WHEN 2 THEN 22 WHEN 4 THEN 44 ELSE 0 END, v FROM t"), Row(0, "1") :: Row(22, "2") :: Row(0, "3") :: Row(44, "4") :: Row(0, "5") :: Nil) @@ -897,7 +897,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-7269 Check analysis failed in case in-sensitive") { Seq(1, 2, 3).map { i => (i.toString, i.toString) - }.toDF("key", "value").registerTempTable("df_analysis") + }.toDF("key", "value").createOrReplaceTempView("df_analysis") sql("SELECT kEy from df_analysis group by key").collect() sql("SELECT kEy+3 from df_analysis group by key+3").collect() sql("SELECT kEy+3, a.kEy, A.kEy from df_analysis A group by key").collect() @@ -1031,7 +1031,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-8588 HiveTypeCoercion.inConversion fires too early") { val df = createDataFrame(Seq((1, "2014-01-01"), (2, "2015-01-01"), (3, "2016-01-01"))) - df.toDF("id", "datef").registerTempTable("test_SPARK8588") + df.toDF("id", "datef").createOrReplaceTempView("test_SPARK8588") checkAnswer( sql( """ @@ -1046,7 +1046,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-9371: fix the support for special chars in column names for hive context") { read.json(sparkContext.makeRDD( """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) - .registerTempTable("t") + .createOrReplaceTempView("t") checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } @@ -1125,7 +1125,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { spark .range(5) .selectExpr("id AS a", "id AS b") - .registerTempTable("test") + .createOrReplaceTempView("test") checkAnswer( sql( @@ -1143,7 +1143,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { spark .range(5) .selectExpr("id AS a", "id AS b") - .registerTempTable("test") + .createOrReplaceTempView("test") val df = sql( """FROM test @@ -1162,7 +1162,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-10741: Sort on Aggregate using parquet") { withTable("test10741") { withTempTable("src") { - Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF().registerTempTable("src") + Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF().createOrReplaceTempView("src") sql("CREATE TABLE test10741(c1 STRING, c2 INT) STORED AS PARQUET AS SELECT * FROM src") } @@ -1374,7 +1374,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { withTempTable("t1") { spark.range(10) .select(array($"id", $"id" + 1).as("arr"), $"id") - .registerTempTable("source") + .createOrReplaceTempView("source") withTable("dest1", "dest2") { sql("CREATE TABLE dest1 (i INT)") sql("CREATE TABLE dest2 (i INT)") @@ -1407,7 +1407,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { withTempTable("t1", "t2") { val path = dir.getCanonicalPath val ds = spark.range(10) - ds.registerTempTable("t1") + ds.createOrReplaceTempView("t1") sql( s"""CREATE TEMPORARY TABLE t2 @@ -1431,7 +1431,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { "shouldn always be used together with PATH data source option" ) { withTempTable("t") { - spark.range(10).registerTempTable("t") + spark.range(10).createOrReplaceTempView("t") val message = intercept[IllegalArgumentException] { sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala index 4d284e10425d6..47ceefb88ebcd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala @@ -40,7 +40,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -112,7 +112,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -139,7 +139,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -182,7 +182,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") val e = intercept[AnalysisException] { sql( @@ -203,7 +203,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 10) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -232,7 +232,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 11) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql("select month, product, sum(product + 1) over() from windowData order by area"), @@ -301,7 +301,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi WindowData(5, "c", 9), WindowData(6, "c", 11) ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") checkAnswer( sql( @@ -322,7 +322,7 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi test("window function: multiple window expressions in a single expression") { val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") - nums.registerTempTable("nums") + nums.createOrReplaceTempView("nums") val expected = Row(1, 1, 1, 55, 1, 57) :: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala index 6161412a49775..fed0d11e9d216 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala @@ -90,7 +90,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - read.orc(base.getCanonicalPath).registerTempTable("t") + read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -137,7 +137,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - read.orc(base.getCanonicalPath).registerTempTable("t") + read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -189,7 +189,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B read .option(ConfVars.DEFAULTPARTITIONNAME.varname, defaultPartitionName) .orc(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -231,7 +231,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B read .option(ConfVars.DEFAULTPARTITIONNAME.varname, defaultPartitionName) .orc(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 9a0885822b8d2..f83b3a3de2e54 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -98,7 +98,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("Creating case class RDD table") { val data = (1 to 100).map(i => (i, s"val_$i")) - sparkContext.parallelize(data).toDF().registerTempTable("t") + sparkContext.parallelize(data).toDF().createOrReplaceTempView("t") withTempTable("t") { checkAnswer(sql("SELECT * FROM t"), data.toDF().collect()) } @@ -223,7 +223,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("appending") { val data = (0 until 10).map(i => (i, i.toString)) - createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withOrcTable(data, "t") { sql("INSERT INTO TABLE t SELECT * FROM tmp") checkAnswer(table("t"), (data ++ data).map(Row.fromTuple)) @@ -233,7 +233,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) - createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withOrcTable(data, "t") { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(table("t"), data.map(Row.fromTuple)) @@ -324,7 +324,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { """.stripMargin) val emptyDF = Seq.empty[(Int, String)].toDF("key", "value").coalesce(1) - emptyDF.registerTempTable("empty") + emptyDF.createOrReplaceTempView("empty") // This creates 1 empty ORC file with Hive ORC SerDe. We are using this trick because // Spark SQL ORC data source always avoids write empty ORC files. @@ -340,7 +340,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { assert(errorMessage.contains("Unable to infer schema for ORC")) val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1) - singleRowDF.registerTempTable("single") + singleRowDF.createOrReplaceTempView("single") spark.sql( s"""INSERT INTO TABLE empty_orc @@ -422,7 +422,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { """.stripMargin) val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1) - singleRowDF.registerTempTable("single") + singleRowDF.createOrReplaceTempView("single") spark.sql( s"""INSERT INTO TABLE dummy_orc diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala index 96a7364437c78..6081d86f4478b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala @@ -51,7 +51,7 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA .makeRDD(1 to 10) .map(i => OrcData(i, s"part-$i")) .toDF() - .registerTempTable(s"orc_temp_table") + .createOrReplaceTempView(s"orc_temp_table") sql( s"""CREATE EXTERNAL TABLE normal_orc( 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 f52c6e48c5760..7fe158c218461 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 @@ -171,8 +171,9 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql(s"ALTER TABLE partitioned_parquet_with_complextypes ADD PARTITION (p=$p)") } - (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").registerTempTable("jt") - (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a").registerTempTable("jt_array") + (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").createOrReplaceTempView("jt") + (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a") + .createOrReplaceTempView("jt_array") setConf(HiveUtils.CONVERT_METASTORE_PARQUET, true) } @@ -541,8 +542,8 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { |STORED AS PARQUET """.stripMargin) - // Temp table to insert data into partitioned table - Seq("foo", "bar").toDF("a").registerTempTable("test_temp") + // Temp view that is used to insert data into partitioned table + Seq("foo", "bar").toDF("a").createOrReplaceTempView("test_temp") sql("INSERT INTO test_added_partitions PARTITION(b='0') SELECT a FROM test_temp") checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 78d2dc28d6b5e..a3183f2977223 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -91,7 +91,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes yield Row(s"val_$i", s"val_$i", s"val_$i", s"val_$i", 1, 1, 1, 1)) // Self-join - df.registerTempTable("t") + df.createOrReplaceTempView("t") withTempTable("t") { checkAnswer( sql( @@ -337,7 +337,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { intercept[AnalysisException] { @@ -347,7 +347,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t") @@ -459,7 +459,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { intercept[AnalysisException] { @@ -474,7 +474,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") withTempTable("t") { partitionedTestDF.write diff --git a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala index 5df674d60e9c4..1c1db72e27ffe 100644 --- a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala +++ b/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -66,7 +66,7 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac val res = df3.collect() val expected = Seq((18, 18, 8)).toDF("a", "x", "b").collect() assert(res.toSeq == expected.toSeq) - df3.registerTempTable("mai_table") + df3.createOrReplaceTempView("mai_table") val df4 = hc.table("mai_table") val res2 = df4.collect() assert(res2.toSeq == expected.toSeq) @@ -82,7 +82,7 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac val databases2 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) assert(databases2.toSet == Set("default", "mee_db")) val df = (1 to 10).map { i => ("bob" + i.toString, i) }.toDF("name", "age") - df.registerTempTable("mee_table") + df.createOrReplaceTempView("mee_table") hc.sql("CREATE TABLE moo_table (name string, age int)") hc.sql("INSERT INTO moo_table SELECT * FROM mee_table") assert( From c8be3da66903899fcd743c425c25e32fc356d981 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 17 May 2016 18:07:59 -0700 Subject: [PATCH 0247/1470] Prepare branch for 2.0.0-preview. --- core/src/main/scala/org/apache/spark/package.scala | 2 +- docs/_config.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index cc5e7ef3ae008..65a3dd2e37e16 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -43,5 +43,5 @@ package org.apache package object spark { // For package docs only - val SPARK_VERSION = "2.0.0-SNAPSHOT" + val SPARK_VERSION = "2.0.0-preview" } diff --git a/docs/_config.yml b/docs/_config.yml index 8bdc68aeeac7f..c0a3be7f8208e 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,7 +14,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.0-SNAPSHOT +SPARK_VERSION: 2.0.0-preview SPARK_VERSION_SHORT: 2.0.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" From 8f5a04b6299e3a47aca13cbb40e72344c0114860 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 17 May 2016 18:15:42 -0700 Subject: [PATCH 0248/1470] Preparing Spark release 2.0.0-preview --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/hivecontext-compatibility/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 33 files changed, 33 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 75ac9262cbae5..958cb453b5544 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5444ae6d70a92..2cb86ea8a4c11 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index e736436aec4cf..58d7879253fb5 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 1fd3af2e6e622..3f8dfe4c20fab 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index bbbb0bd5aa050..03db5b81ff7db 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 14e94eca93b22..e10085136b681 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index d0d1da69ea802..0c2aa8b4b5325 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 45f8bfcc05aa1..ce5eac5140466 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 771da5b9a6e6e..3db074aaa7806 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 21d40863b77f5..b0fc3b943c634 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index ac15b93c048da..51abd3375d493 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 016af934bb897..e22ee74cf6863 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f51d334de0876..1c716ab462888 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 60e3ff60df065..57d25630ccbe2 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 3cc288abeaa22..aff01b6fcf9ee 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 4a20b78917efa..da62aead7c6c3 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 6fb88ebae5b32..1b04ca860a87c 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b5f5ff2854cfb..532fb70ba8d1f 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index bfb92791de3d8..bc12f2b54ee42 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc6c700dd1ec8..51eb281c6fb1f 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e7303853e6565..f536506fc412a 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1c6ab2b62d8f0..e526ff3c1f761 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c324afb2a2d1b..df9681143a6be 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/pom.xml b/pom.xml index 9c13af17e4ed5..6d1f030da4bb1 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index c12d121c61156..5553d36409216 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 1923199f4b861..d0238449ec684 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 2ea980bf20dfe..cb5a499808797 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 809d36dc69b99..a35affa711604 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c8b20f0afc4ea..b9eb5ccca8102 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/sql/hivecontext-compatibility/pom.xml b/sql/hivecontext-compatibility/pom.xml index ed9ef8e27919e..b8f091eff69c0 100644 --- a/sql/hivecontext-compatibility/pom.xml +++ b/sql/hivecontext-compatibility/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 3f6774593644d..89c2f8f7e1bd2 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 9bb20e1381067..001efbca1916f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index e07b93ab95450..fcc94ba0ad4e2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0-preview ../pom.xml From b54500913d49b91949716b2c41bd5f637b1795a7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 17 May 2016 18:15:51 -0700 Subject: [PATCH 0249/1470] Preparing development version 2.0.0-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- sql/hivecontext-compatibility/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 33 files changed, 33 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 958cb453b5544..75ac9262cbae5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2cb86ea8a4c11..5444ae6d70a92 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 58d7879253fb5..e736436aec4cf 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3f8dfe4c20fab..1fd3af2e6e622 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 03db5b81ff7db..bbbb0bd5aa050 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e10085136b681..14e94eca93b22 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 0c2aa8b4b5325..d0d1da69ea802 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ce5eac5140466..45f8bfcc05aa1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 3db074aaa7806..771da5b9a6e6e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index b0fc3b943c634..21d40863b77f5 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 51abd3375d493..ac15b93c048da 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index e22ee74cf6863..016af934bb897 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 1c716ab462888..f51d334de0876 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 57d25630ccbe2..60e3ff60df065 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index aff01b6fcf9ee..3cc288abeaa22 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index da62aead7c6c3..4a20b78917efa 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 1b04ca860a87c..6fb88ebae5b32 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 532fb70ba8d1f..b5f5ff2854cfb 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index bc12f2b54ee42..bfb92791de3d8 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 51eb281c6fb1f..fc6c700dd1ec8 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index f536506fc412a..e7303853e6565 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index e526ff3c1f761..1c6ab2b62d8f0 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index df9681143a6be..c324afb2a2d1b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 6d1f030da4bb1..9c13af17e4ed5 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 5553d36409216..c12d121c61156 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index d0238449ec684..1923199f4b861 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index cb5a499808797..2ea980bf20dfe 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a35affa711604..809d36dc69b99 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b9eb5ccca8102..c8b20f0afc4ea 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hivecontext-compatibility/pom.xml b/sql/hivecontext-compatibility/pom.xml index b8f091eff69c0..ed9ef8e27919e 100644 --- a/sql/hivecontext-compatibility/pom.xml +++ b/sql/hivecontext-compatibility/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 89c2f8f7e1bd2..3f6774593644d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 001efbca1916f..9bb20e1381067 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index fcc94ba0ad4e2..e07b93ab95450 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-preview + 2.0.0-SNAPSHOT ../pom.xml From fc97ff5d7f56f85bc5cf2383d4786005e067a0df Mon Sep 17 00:00:00 2001 From: Takuya Kuwahara Date: Wed, 18 May 2016 08:29:47 +0200 Subject: [PATCH 0250/1470] [SPARK-14978][PYSPARK] PySpark TrainValidationSplitModel should support validationMetrics ## What changes were proposed in this pull request? This pull request includes supporting validationMetrics for TrainValidationSplitModel with Python and test for it. ## How was this patch tested? test in `python/pyspark/ml/tests.py` Author: Takuya Kuwahara Closes #12767 from taku-k/spark-14978. (cherry picked from commit 411c04adb596c514f2634efd5f5d126e12b05df7) Signed-off-by: Nick Pentreath --- python/pyspark/ml/tests.py | 50 ++++++++++++++++++++++++++++++++----- python/pyspark/ml/tuning.py | 13 +++++++--- 2 files changed, 53 insertions(+), 10 deletions(-) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index e3511120bdecb..a7c93ac802726 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -610,17 +610,21 @@ def test_fit_minimize_metric(self): iee = InducedErrorEstimator() evaluator = RegressionEvaluator(metricName="rmse") - grid = (ParamGridBuilder() - .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) - .build()) + grid = ParamGridBuilder() \ + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) \ + .build() tvs = TrainValidationSplit(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) tvsModel = tvs.fit(dataset) bestModel = tvsModel.bestModel bestModelMetric = evaluator.evaluate(bestModel.transform(dataset)) + validationMetrics = tvsModel.validationMetrics self.assertEqual(0.0, bestModel.getOrDefault('inducedError'), "Best model should have zero induced error") self.assertEqual(0.0, bestModelMetric, "Best model has RMSE of 0") + self.assertEqual(len(grid), len(validationMetrics), + "validationMetrics has the same size of grid parameter") + self.assertEqual(0.0, min(validationMetrics)) def test_fit_maximize_metric(self): dataset = self.spark.createDataFrame([ @@ -633,17 +637,21 @@ def test_fit_maximize_metric(self): iee = InducedErrorEstimator() evaluator = RegressionEvaluator(metricName="r2") - grid = (ParamGridBuilder() - .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) - .build()) + grid = ParamGridBuilder() \ + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) \ + .build() tvs = TrainValidationSplit(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) tvsModel = tvs.fit(dataset) bestModel = tvsModel.bestModel bestModelMetric = evaluator.evaluate(bestModel.transform(dataset)) + validationMetrics = tvsModel.validationMetrics self.assertEqual(0.0, bestModel.getOrDefault('inducedError'), "Best model should have zero induced error") self.assertEqual(1.0, bestModelMetric, "Best model has R-squared of 1") + self.assertEqual(len(grid), len(validationMetrics), + "validationMetrics has the same size of grid parameter") + self.assertEqual(1.0, max(validationMetrics)) def test_save_load(self): # This tests saving and loading the trained model only. @@ -669,6 +677,36 @@ def test_save_load(self): self.assertEqual(loadedLrModel.uid, lrModel.uid) self.assertEqual(loadedLrModel.intercept, lrModel.intercept) + def test_copy(self): + dataset = self.spark.createDataFrame([ + (10, 10.0), + (50, 50.0), + (100, 100.0), + (500, 500.0)] * 10, + ["feature", "label"]) + + iee = InducedErrorEstimator() + evaluator = RegressionEvaluator(metricName="r2") + + grid = ParamGridBuilder() \ + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) \ + .build() + tvs = TrainValidationSplit(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) + tvsModel = tvs.fit(dataset) + tvsCopied = tvs.copy() + tvsModelCopied = tvsModel.copy() + + self.assertEqual(tvs.getEstimator().uid, tvsCopied.getEstimator().uid, + "Copied TrainValidationSplit has the same uid of Estimator") + + self.assertEqual(tvsModel.bestModel.uid, tvsModelCopied.bestModel.uid) + self.assertEqual(len(tvsModel.validationMetrics), + len(tvsModelCopied.validationMetrics), + "Copied validationMetrics has the same size of the original") + for index in range(len(tvsModel.validationMetrics)): + self.assertEqual(tvsModel.validationMetrics[index], + tvsModelCopied.validationMetrics[index]) + class PersistenceTest(SparkSessionTestCase): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 75789c4d09184..4f7a6b0f7b73b 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -379,7 +379,7 @@ def _fit(self, dataset): seed = self.getOrDefault(self.seed) randCol = self.uid + "_rand" df = dataset.select("*", rand(seed).alias(randCol)) - metrics = np.zeros(numModels) + metrics = [0.0] * numModels condition = (df[randCol] >= tRatio) validation = df.filter(condition) train = df.filter(~condition) @@ -392,7 +392,7 @@ def _fit(self, dataset): else: bestIndex = np.argmin(metrics) bestModel = est.fit(dataset, epm[bestIndex]) - return self._copyValues(TrainValidationSplitModel(bestModel)) + return self._copyValues(TrainValidationSplitModel(bestModel, metrics)) @since("2.0.0") def copy(self, extra=None): @@ -424,10 +424,12 @@ class TrainValidationSplitModel(Model, ValidatorParams): .. versionadded:: 2.0.0 """ - def __init__(self, bestModel): + def __init__(self, bestModel, validationMetrics=[]): super(TrainValidationSplitModel, self).__init__() #: best model from cross validation self.bestModel = bestModel + #: evaluated validation metrics + self.validationMetrics = validationMetrics def _transform(self, dataset): return self.bestModel.transform(dataset) @@ -439,13 +441,16 @@ def copy(self, extra=None): and some extra params. This copies the underlying bestModel, creates a deep copy of the embedded paramMap, and copies the embedded and extra parameters over. + And, this creates a shallow copy of the validationMetrics. :param extra: Extra parameters to copy to the new instance :return: Copy of this instance """ if extra is None: extra = dict() - return TrainValidationSplitModel(self.bestModel.copy(extra)) + bestModel = self.bestModel.copy(extra) + validationMetrics = list(self.validationMetrics) + return TrainValidationSplitModel(bestModel, validationMetrics) if __name__ == "__main__": From c66da74b1efdc1089b61d79d51cd32c635ad3b5e Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 18 May 2016 16:00:02 +0800 Subject: [PATCH 0251/1470] [SPARK-15334][SQL] HiveClient facade not compatible with Hive 0.12 ## What changes were proposed in this pull request? HiveClient facade is not compatible with Hive 0.12. This PR Fixes the following compatibility issues: 1. `org.apache.spark.sql.hive.client.HiveClientImpl` use `AddPartitionDesc(db, table, ignoreIfExists)` to create partitions, however, Hive 0.12 doesn't have this constructor for `AddPartitionDesc`. 2. `HiveClientImpl` uses `PartitionDropOptions` when dropping partition, however, class `PartitionDropOptions` doesn't exist in Hive 0.12. 3. Hive 0.12 doesn't support adding permanent functions. It is not valid to call `org.apache.hadoop.hive.ql.metadata.Hive.createFunction`, `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction`, and `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction` 4. `org.apache.spark.sql.hive.client.VersionsSuite` doesn't have enough test coverage for different hive versions 0.12, 0.13, 0.14, 1.0.0, 1.1.0, 1.2.0. ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13127 from clockfly/versionSuite. (cherry picked from commit 6e02aec44b9e5bc2ada55cb612f26e6ba000c23e) Signed-off-by: Cheng Lian --- .../sql/hive/client/HiveClientImpl.scala | 74 +--- .../spark/sql/hive/client/HiveShim.scala | 182 +++++++- .../spark/sql/hive/client/VersionsSuite.scala | 412 +++++++++++++++--- 3 files changed, 545 insertions(+), 123 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index a4e9f03b43342..af2850d4f568c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -26,14 +26,10 @@ import scala.language.reflectiveCalls import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.{PartitionDropOptions, TableType => HiveTableType} +import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} -import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType} -import org.apache.hadoop.hive.metastore.api.{NoSuchObjectException, PrincipalType} -import org.apache.hadoop.hive.metastore.api.{ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} -import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.UserGroupInformation @@ -41,13 +37,13 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.util.{CausedBy, CircularBuffer, Utils} +import org.apache.spark.util.{CircularBuffer, Utils} /** * A class that wraps the HiveClient and converts its responses to externally visible classes. @@ -400,11 +396,7 @@ private[hive] class HiveClientImpl( table: String, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = withHiveState { - val addPartitionDesc = new AddPartitionDesc(db, table, ignoreIfExists) - parts.foreach { s => - addPartitionDesc.addPartition(s.spec.asJava, s.storage.locationUri.orNull) - } - client.createPartitions(addPartitionDesc) + shim.createPartitions(client, db, table, parts, ignoreIfExists) } override def dropPartitions( @@ -430,10 +422,9 @@ private[hive] class HiveClientImpl( }.distinct var droppedParts = ArrayBuffer.empty[java.util.List[String]] matchingParts.foreach { partition => - val dropOptions = new PartitionDropOptions - dropOptions.ifExists = ignoreIfNotExists try { - client.dropPartition(db, table, partition, dropOptions) + val deleteData = true + client.dropPartition(db, table, partition, deleteData) } catch { case e: Exception => val remainingParts = matchingParts.toBuffer -- droppedParts @@ -629,37 +620,28 @@ private[hive] class HiveClientImpl( } override def createFunction(db: String, func: CatalogFunction): Unit = withHiveState { - client.createFunction(toHiveFunction(func, db)) + shim.createFunction(client, db, func) } override def dropFunction(db: String, name: String): Unit = withHiveState { - client.dropFunction(db, name) + shim.dropFunction(client, db, name) } override def renameFunction(db: String, oldName: String, newName: String): Unit = withHiveState { - val catalogFunc = getFunction(db, oldName) - .copy(identifier = FunctionIdentifier(newName, Some(db))) - val hiveFunc = toHiveFunction(catalogFunc, db) - client.alterFunction(db, oldName, hiveFunc) + shim.renameFunction(client, db, oldName, newName) } override def alterFunction(db: String, func: CatalogFunction): Unit = withHiveState { - client.alterFunction(db, func.identifier.funcName, toHiveFunction(func, db)) + shim.alterFunction(client, db, func) } override def getFunctionOption( - db: String, - name: String): Option[CatalogFunction] = withHiveState { - try { - Option(client.getFunction(db, name)).map(fromHiveFunction) - } catch { - case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => - None - } + db: String, name: String): Option[CatalogFunction] = withHiveState { + shim.getFunctionOption(client, db, name) } override def listFunctions(db: String, pattern: String): Seq[String] = withHiveState { - client.getFunctions(db, pattern).asScala + shim.listFunctions(client, db, pattern) } def addJar(path: String): Unit = { @@ -708,36 +690,6 @@ private[hive] class HiveClientImpl( Utils.classForName(name) .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] - private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { - val resourceUris = f.resources.map { resource => - new ResourceUri( - ResourceType.valueOf(resource.resourceType.resourceType.toUpperCase()), resource.uri) - } - new HiveFunction( - f.identifier.funcName, - db, - f.className, - null, - PrincipalType.USER, - (System.currentTimeMillis / 1000).toInt, - FunctionType.JAVA, - resourceUris.asJava) - } - - private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { - val name = FunctionIdentifier(hf.getFunctionName, Option(hf.getDbName)) - val resources = hf.getResourceUris.asScala.map { uri => - val resourceType = uri.getResourceType() match { - case ResourceType.ARCHIVE => "archive" - case ResourceType.FILE => "file" - case ResourceType.JAR => "jar" - case r => throw new AnalysisException(s"Unknown resource type: $r") - } - FunctionResource(FunctionResourceType.fromString(resourceType), uri.getUri()) - } - new CatalogFunction(name, hf.getClassName, resources) - } - private def toHiveColumn(c: CatalogColumn): FieldSchema = { new FieldSchema(c.name, c.dataType, c.comment.orNull) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 4ecf866f96395..78713c3f0bace 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -27,15 +27,23 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType, NoSuchObjectException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} +import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException +import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{IntegralType, StringType} +import org.apache.spark.util.CausedBy + /** * A shim that defines the interface between [[HiveClientImpl]] and the underlying Hive library used @@ -73,6 +81,13 @@ private[client] sealed abstract class Shim { def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long + def createPartitions( + hive: Hive, + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit + def loadPartition( hive: Hive, loadPath: Path, @@ -100,6 +115,18 @@ private[client] sealed abstract class Shim { holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit + def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit + + def dropFunction(hive: Hive, db: String, name: String): Unit + + def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit + + def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit + + def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] + + def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] + def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit protected def findStaticMethod(klass: Class[_], name: String, args: Class[_]*): Method = { @@ -112,7 +139,6 @@ private[client] sealed abstract class Shim { protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { klass.getMethod(name, args: _*) } - } private[client] class Shim_v0_12 extends Shim with Logging { @@ -144,6 +170,22 @@ private[client] class Shim_v0_12 extends Shim with Logging { classOf[Driver], "getResults", classOf[JArrayList[String]]) + private lazy val createPartitionMethod = + findMethod( + classOf[Hive], + "createPartition", + classOf[Table], + classOf[JMap[String, String]], + classOf[Path], + classOf[JMap[String, String]], + classOf[String], + classOf[String], + JInteger.TYPE, + classOf[JList[Object]], + classOf[String], + classOf[JMap[String, String]], + classOf[JList[Object]], + classOf[JList[Object]]) private lazy val loadPartitionMethod = findMethod( classOf[Hive], @@ -199,6 +241,42 @@ private[client] class Shim_v0_12 extends Shim with Logging { override def setDataLocation(table: Table, loc: String): Unit = setDataLocationMethod.invoke(table, new URI(loc)) + // Follows exactly the same logic of DDLTask.createPartitions in Hive 0.12 + override def createPartitions( + hive: Hive, + database: String, + tableName: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = { + val table = hive.getTable(database, tableName) + parts.foreach { s => + val location = s.storage.locationUri.map(new Path(table.getPath, _)).orNull + val spec = s.spec.asJava + if (hive.getPartition(table, spec, false) != null && ignoreIfExists) { + // Ignore this partition since it already exists and ignoreIfExists == true + } else { + if (location == null && table.isView()) { + throw new HiveException("LOCATION clause illegal for view partition"); + } + + createPartitionMethod.invoke( + hive, + table, + spec, + location, + null, // partParams + null, // inputFormat + null, // outputFormat + -1: JInteger, // numBuckets + null, // cols + null, // serializationLib + null, // serdeParams + null, // bucketCols + null) // sortCols + } + } + } + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq @@ -265,6 +343,30 @@ private[client] class Shim_v0_12 extends Shim with Logging { dropIndexMethod.invoke(hive, dbName, tableName, indexName, true: JBoolean) } + override def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + throw new AnalysisException("Hive 0.12 doesn't support creating permanent functions. " + + "Please use Hive 0.13 or higher.") + } + + def dropFunction(hive: Hive, db: String, name: String): Unit = { + throw new NoSuchPermanentFunctionException(db, name) + } + + def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit = { + throw new NoSuchPermanentFunctionException(db, oldName) + } + + def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + throw new NoSuchPermanentFunctionException(db, func.identifier.funcName) + } + + def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] = { + None + } + + def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { + Seq.empty[String] + } } private[client] class Shim_v0_13 extends Shim_v0_12 { @@ -308,9 +410,85 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { override def setDataLocation(table: Table, loc: String): Unit = setDataLocationMethod.invoke(table, new Path(loc)) + override def createPartitions( + hive: Hive, + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = { + val addPartitionDesc = new AddPartitionDesc(db, table, ignoreIfExists) + parts.foreach { s => + addPartitionDesc.addPartition(s.spec.asJava, s.storage.locationUri.orNull) + } + hive.createPartitions(addPartitionDesc) + } + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq + private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { + val resourceUris = f.resources.map { resource => + new ResourceUri( + ResourceType.valueOf(resource.resourceType.resourceType.toUpperCase()), resource.uri) + } + new HiveFunction( + f.identifier.funcName, + db, + f.className, + null, + PrincipalType.USER, + (System.currentTimeMillis / 1000).toInt, + FunctionType.JAVA, + resourceUris.asJava) + } + + override def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + hive.createFunction(toHiveFunction(func, db)) + } + + override def dropFunction(hive: Hive, db: String, name: String): Unit = { + hive.dropFunction(db, name) + } + + override def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit = { + val catalogFunc = getFunctionOption(hive, db, oldName) + .getOrElse(throw new NoSuchPermanentFunctionException(db, oldName)) + .copy(identifier = FunctionIdentifier(newName, Some(db))) + val hiveFunc = toHiveFunction(catalogFunc, db) + hive.alterFunction(db, oldName, hiveFunc) + } + + override def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + hive.alterFunction(db, func.identifier.funcName, toHiveFunction(func, db)) + } + + private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { + val name = FunctionIdentifier(hf.getFunctionName, Option(hf.getDbName)) + val resources = hf.getResourceUris.asScala.map { uri => + val resourceType = uri.getResourceType() match { + case ResourceType.ARCHIVE => "archive" + case ResourceType.FILE => "file" + case ResourceType.JAR => "jar" + case r => throw new AnalysisException(s"Unknown resource type: $r") + } + FunctionResource(FunctionResourceType.fromString(resourceType), uri.getUri()) + } + new CatalogFunction(name, hf.getClassName, resources) + } + + override def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] = { + try { + Option(hive.getFunction(db, name)).map(fromHiveFunction) + } catch { + case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + None + } + } + + override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { + hive.getFunctions(db, pattern).asScala + } + /** * Converts catalyst expression to the format that Hive's getPartitionsByFilter() expects, i.e. * a string that represents partition predicates like "str_key=\"value\" and int_key=1 ...". diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index a6a5ab3988fc9..57e8db7e88fb5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -17,21 +17,27 @@ package org.apache.spark.sql.hive.client -import java.io.File +import java.io.{ByteArrayOutputStream, File, PrintStream} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.util.VersionInfo import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal, NamedExpression} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.types.IntegerType import org.apache.spark.tags.ExtendedHiveTest -import org.apache.spark.util.Utils +import org.apache.spark.util.{MutableURLClassLoader, Utils} /** * A simple set of tests that call the methods of a [[HiveClient]], loading different version @@ -97,12 +103,6 @@ class VersionsSuite extends SparkFunSuite with Logging { private val emptyDir = Utils.createTempDir().getCanonicalPath - private def partSpec = { - val hashMap = new java.util.LinkedHashMap[String, String] - hashMap.put("key", "1") - hashMap - } - // Its actually pretty easy to mess things up and have all of your tests "pass" by accidentally // connecting to an auto-populated, in-process metastore. Let's make sure we are getting the // versions right by forcing a known compatibility failure. @@ -122,7 +122,7 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'") } - private val versions = Seq("12", "13", "14", "1.0.0", "1.1.0", "1.2.0") + private val versions = Seq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2") private var client: HiveClient = null @@ -130,110 +130,402 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: create client") { client = null System.gc() // Hack to avoid SEGV on some JVM versions. + val hadoopConf = new Configuration(); + hadoopConf.set("test", "success") client = IsolatedClientLoader.forVersion( hiveMetastoreVersion = version, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf = new Configuration(), + hadoopConf, config = buildConf(), ivyPath = ivyPath).createClient() } + def table(database: String, tableName: String): CatalogTable = { + CatalogTable( + identifier = TableIdentifier(tableName, Some(database)), + tableType = CatalogTableType.MANAGED, + schema = Seq(CatalogColumn("key", "int")), + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = Some(classOf[TextInputFormat].getName), + outputFormat = Some(classOf[HiveIgnoreKeyTextOutputFormat[_, _]].getName), + serde = Some(classOf[LazySimpleSerDe].getName()), + compressed = false, + serdeProperties = Map.empty + )) + } + + /////////////////////////////////////////////////////////////////////////// + // Database related API + /////////////////////////////////////////////////////////////////////////// + + val tempDatabasePath = Utils.createTempDir().getCanonicalPath + test(s"$version: createDatabase") { - val db = CatalogDatabase("default", "desc", "loc", Map()) - client.createDatabase(db, ignoreIfExists = true) + val defaultDB = CatalogDatabase("default", "desc", "loc", Map()) + client.createDatabase(defaultDB, ignoreIfExists = true) + val tempDB = CatalogDatabase( + "temporary", description = "test create", tempDatabasePath, Map()) + client.createDatabase(tempDB, ignoreIfExists = true) + } + + test(s"$version: setCurrentDatabase") { + client.setCurrentDatabase("default") + } + + test(s"$version: getDatabase") { + // No exception should be thrown + client.getDatabase("default") + } + + test(s"$version: getDatabaseOption") { + assert(client.getDatabaseOption("default").isDefined) + assert(client.getDatabaseOption("nonexist") == None) } + test(s"$version: listDatabases") { + assert(client.listDatabases("defau.*") == Seq("default")) + } + + test(s"$version: alterDatabase") { + val database = client.getDatabase("temporary").copy(properties = Map("flag" -> "true")) + client.alterDatabase(database) + assert(client.getDatabase("temporary").properties.contains("flag")) + } + + test(s"$version: dropDatabase") { + assert(client.getDatabaseOption("temporary").isDefined) + client.dropDatabase("temporary", ignoreIfNotExists = false, cascade = true) + assert(client.getDatabaseOption("temporary").isEmpty) + } + + /////////////////////////////////////////////////////////////////////////// + // Table related API + /////////////////////////////////////////////////////////////////////////// + test(s"$version: createTable") { - val table = - CatalogTable( - identifier = TableIdentifier("src", Some("default")), - tableType = CatalogTableType.MANAGED, - schema = Seq(CatalogColumn("key", "int")), - storage = CatalogStorageFormat( - locationUri = None, - inputFormat = Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), - outputFormat = Some( - classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), - serde = Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName()), - compressed = false, - serdeProperties = Map.empty - )) - - client.createTable(table, ignoreIfExists = false) + client.createTable(table("default", tableName = "src"), ignoreIfExists = false) + client.createTable(table("default", "temporary"), ignoreIfExists = false) + } + + test(s"$version: loadTable") { + client.loadTable( + emptyDir, + tableName = "src", + replace = false, + holdDDLTime = false) } test(s"$version: getTable") { + // No exception should be thrown client.getTable("default", "src") } - test(s"$version: listTables") { - assert(client.listTables("default") === Seq("src")) + test(s"$version: getTableOption") { + assert(client.getTableOption("default", "src").isDefined) } - test(s"$version: getDatabase") { - client.getDatabase("default") + test(s"$version: alterTable(table: CatalogTable)") { + val newTable = client.getTable("default", "src").copy(properties = Map("changed" -> "")) + client.alterTable(newTable) + assert(client.getTable("default", "src").properties.contains("changed")) } - test(s"$version: alterTable") { - client.alterTable(client.getTable("default", "src")) + test(s"$version: alterTable(tableName: String, table: CatalogTable)") { + val newTable = client.getTable("default", "src").copy(properties = Map("changedAgain" -> "")) + client.alterTable("src", newTable) + assert(client.getTable("default", "src").properties.contains("changedAgain")) } - test(s"$version: set command") { - client.runSqlHive("SET spark.sql.test.key=1") + test(s"$version: listTables(database)") { + assert(client.listTables("default") === Seq("src", "temporary")) + } + + test(s"$version: listTables(database, pattern)") { + assert(client.listTables("default", pattern = "src") === Seq("src")) + assert(client.listTables("default", pattern = "nonexist").isEmpty) + } + + test(s"$version: dropTable") { + client.dropTable("default", tableName = "temporary", ignoreIfNotExists = false) + assert(client.listTables("default") === Seq("src")) } - test(s"$version: create partitioned table DDL") { - client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key INT)") - client.runSqlHive("ALTER TABLE src_part ADD PARTITION (key = '1')") + /////////////////////////////////////////////////////////////////////////// + // Partition related API + /////////////////////////////////////////////////////////////////////////// + + val storageFormat = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + compressed = false, + serdeProperties = Map.empty) + + test(s"$version: sql create partitioned table") { + client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key1 INT, key2 INT)") } - test(s"$version: getPartitions") { - client.getPartitions(client.getTable("default", "src_part")) + test(s"$version: createPartitions") { + val partition1 = CatalogTablePartition(Map("key1" -> "1", "key2" -> "1"), storageFormat) + val partition2 = CatalogTablePartition(Map("key1" -> "1", "key2" -> "2"), storageFormat) + client.createPartitions( + "default", "src_part", Seq(partition1, partition2), ignoreIfExists = true) + } + + test(s"$version: getPartitions(catalogTable)") { + assert(2 == client.getPartitions(client.getTable("default", "src_part")).size) } test(s"$version: getPartitionsByFilter") { - client.getPartitionsByFilter(client.getTable("default", "src_part"), Seq(EqualTo( - AttributeReference("key", IntegerType, false)(NamedExpression.newExprId), - Literal(1)))) + // Only one partition [1, 1] for key2 == 1 + val result = client.getPartitionsByFilter(client.getTable("default", "src_part"), + Seq(EqualTo(AttributeReference("key2", IntegerType)(), Literal(1)))) + + // Hive 0.12 doesn't support getPartitionsByFilter, it ignores the filter condition. + if (version != "0.12") { + assert(result.size == 1) + } + } + + test(s"$version: getPartition") { + // No exception should be thrown + client.getPartition("default", "src_part", Map("key1" -> "1", "key2" -> "2")) + } + + test(s"$version: getPartitionOption(db: String, table: String, spec: TablePartitionSpec)") { + val partition = client.getPartitionOption( + "default", "src_part", Map("key1" -> "1", "key2" -> "2")) + assert(partition.isDefined) + } + + test(s"$version: getPartitionOption(table: CatalogTable, spec: TablePartitionSpec)") { + val partition = client.getPartitionOption( + client.getTable("default", "src_part"), Map("key1" -> "1", "key2" -> "2")) + assert(partition.isDefined) + } + + test(s"$version: getPartitions(db: String, table: String)") { + assert(2 == client.getPartitions("default", "src_part", None).size) } test(s"$version: loadPartition") { + val partSpec = new java.util.LinkedHashMap[String, String] + partSpec.put("key1", "1") + partSpec.put("key2", "2") + client.loadPartition( emptyDir, "default.src_part", partSpec, - false, - false, - false, - false) - } - - test(s"$version: loadTable") { - client.loadTable( - emptyDir, - "src", - false, - false) + replace = false, + holdDDLTime = false, + inheritTableSpecs = false, + isSkewedStoreAsSubdir = false) } test(s"$version: loadDynamicPartitions") { + val partSpec = new java.util.LinkedHashMap[String, String] + partSpec.put("key1", "1") + partSpec.put("key2", "") // Dynamic partition + client.loadDynamicPartitions( emptyDir, "default.src_part", partSpec, - false, - 1, + replace = false, + numDP = 1, false, false) } - test(s"$version: create index and reset") { + test(s"$version: renamePartitions") { + val oldSpec = Map("key1" -> "1", "key2" -> "1") + val newSpec = Map("key1" -> "1", "key2" -> "3") + client.renamePartitions("default", "src_part", Seq(oldSpec), Seq(newSpec)) + + // Checks the existence of the new partition (key1 = 1, key2 = 3) + assert(client.getPartitionOption("default", "src_part", newSpec).isDefined) + } + + test(s"$version: alterPartitions") { + val spec = Map("key1" -> "1", "key2" -> "2") + val newLocation = Utils.createTempDir().getPath() + val storage = storageFormat.copy(locationUri = Some(newLocation)) + val partition = CatalogTablePartition(spec, storage) + client.alterPartitions("default", "src_part", Seq(partition)) + assert(client.getPartition("default", "src_part", spec) + .storage.locationUri == Some(newLocation)) + } + + test(s"$version: dropPartitions") { + val spec = Map("key1" -> "1", "key2" -> "3") + client.dropPartitions("default", "src_part", Seq(spec), ignoreIfNotExists = true) + assert(client.getPartitionOption("default", "src_part", spec).isEmpty) + } + + /////////////////////////////////////////////////////////////////////////// + // Function related API + /////////////////////////////////////////////////////////////////////////// + + def function(name: String, className: String): CatalogFunction = { + CatalogFunction( + FunctionIdentifier(name, Some("default")), className, Seq.empty[FunctionResource]) + } + + test(s"$version: createFunction") { + val functionClass = "org.apache.spark.MyFunc1" + if (version == "0.12") { + // Hive 0.12 doesn't support creating permanent functions + intercept[AnalysisException] { + client.createFunction("default", function("func1", functionClass)) + } + } else { + client.createFunction("default", function("func1", functionClass)) + } + } + + test(s"$version: functionExists") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.functionExists("default", "func1") == false) + } else { + assert(client.functionExists("default", "func1") == true) + } + } + + test(s"$version: renameFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.renameFunction("default", "func1", "func2") + } + } else { + client.renameFunction("default", "func1", "func2") + assert(client.functionExists("default", "func2") == true) + } + } + + test(s"$version: alterFunction") { + val functionClass = "org.apache.spark.MyFunc2" + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.alterFunction("default", function("func2", functionClass)) + } + } else { + client.alterFunction("default", function("func2", functionClass)) + } + } + + test(s"$version: getFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.getFunction("default", "func2") + } + } else { + // No exception should be thrown + val func = client.getFunction("default", "func2") + assert(func.className == "org.apache.spark.MyFunc2") + } + } + + test(s"$version: getFunctionOption") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.getFunctionOption("default", "func2").isEmpty) + } else { + assert(client.getFunctionOption("default", "func2").isDefined) + } + } + + test(s"$version: listFunctions") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.listFunctions("default", "fun.*").isEmpty) + } else { + assert(client.listFunctions("default", "fun.*").size == 1) + } + } + + test(s"$version: dropFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't support creating permanent functions + intercept[NoSuchPermanentFunctionException] { + client.dropFunction("default", "func2") + } + } else { + // No exception should be thrown + client.dropFunction("default", "func2") + assert(client.listFunctions("default", "fun.*").size == 0) + } + } + + /////////////////////////////////////////////////////////////////////////// + // SQL related API + /////////////////////////////////////////////////////////////////////////// + + test(s"$version: sql set command") { + client.runSqlHive("SET spark.sql.test.key=1") + } + + test(s"$version: sql create index and reset") { client.runSqlHive("CREATE TABLE indexed_table (key INT)") client.runSqlHive("CREATE INDEX index_1 ON TABLE indexed_table(key) " + "as 'COMPACT' WITH DEFERRED REBUILD") + } + + /////////////////////////////////////////////////////////////////////////// + // Miscellaneous API + /////////////////////////////////////////////////////////////////////////// + + test(s"$version: version") { + assert(client.version.fullVersion.startsWith(version)) + } + + test(s"$version: getConf") { + assert("success" === client.getConf("test", null)) + } + + test(s"$version: setOut") { + client.setOut(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: setInfo") { + client.setInfo(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: setError") { + client.setError(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: newSession") { + val newClient = client.newSession() + assert(newClient != null) + } + + test(s"$version: withHiveState and addJar") { + val newClassPath = "." + client.addJar(newClassPath) + client.withHiveState { + // No exception should be thrown. + // withHiveState changes the classloader to MutableURLClassLoader + val classLoader = Thread.currentThread().getContextClassLoader + .asInstanceOf[MutableURLClassLoader] + + val urls = classLoader.getURLs() + urls.contains(new File(newClassPath).toURI.toURL) + } + } + + test(s"$version: reset") { + // Clears all database, tables, functions... client.reset() + assert(client.listTables("default").isEmpty) } } } From 35c25beaca3e9070ccb6dcca33c415a26e6f1f1c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 18:46:57 +0800 Subject: [PATCH 0252/1470] [SPARK-15307][SQL] speed up listing files for data source ## What changes were proposed in this pull request? Currently, listing files is very slow if there is thousands files, especially on local file system, because: 1) FileStatus.getPermission() is very slow on local file system, which is launch a subprocess and parse the stdout. 2) Create an JobConf is very expensive (ClassUtil.findContainingJar() is slow). This PR improve these by: 1) Use another constructor of LocatedFileStatus to avoid calling FileStatus.getPermission, the permissions are not used for data sources. 2) Only create an JobConf once within one task. ## How was this patch tested? Manually tests on a partitioned table with 1828 partitions, decrease the time to load the table from 22 seconds to 1.6 seconds (Most of time are spent in merging schema now). Author: Davies Liu Closes #13094 from davies/listing. (cherry picked from commit 33814f887aea339c99e14ce7f14ca6fcc6875015) Signed-off-by: Cheng Lian --- .../datasources/ListingFileCatalog.scala | 9 +++-- .../datasources/fileSourceInterfaces.scala | 38 ++++++++++++++----- 2 files changed, 33 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index 5cee2b9af68c2..644e5d65d612c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -77,12 +77,12 @@ class ListingFileCatalog( if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext) } else { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) val statuses: Seq[FileStatus] = paths.flatMap { path => val fs = path.getFileSystem(hadoopConf) logInfo(s"Listing $path on driver") - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) val statuses = { val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) @@ -101,7 +101,8 @@ class ListingFileCatalog( // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should a // a big deal since we always use to `listLeafFilesInParallel` when the number of paths // exceeds threshold. - case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + case f => + HadoopFsRelation.createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) } }.filterNot { status => val name = status.getPath.getName diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index b516297115f04..8d332df029166 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -348,28 +348,40 @@ private[sql] object HadoopFsRelation extends Logging { pathName == "_SUCCESS" || pathName == "_temporary" || pathName.startsWith(".") } + /** + * Create a LocatedFileStatus using FileStatus and block locations. + */ + def createLocatedFileStatus(f: FileStatus, locations: Array[BlockLocation]): LocatedFileStatus = { + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), which is + // very slow on some file system (RawLocalFileSystem, which is launch a subprocess and parse the + // stdout). + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + lfs + } + // We don't filter files/directories whose name start with "_" except "_temporary" here, as // specific data sources may take advantages over them (e.g. Parquet _metadata and // _common_metadata files). "_temporary" directories are explicitly ignored since failed // tasks/jobs may leave partial/corrupted data files there. Files and directories whose name // start with "." are also ignored. - def listLeafFiles(fs: FileSystem, status: FileStatus): Array[FileStatus] = { + def listLeafFiles(fs: FileSystem, status: FileStatus, filter: PathFilter): Array[FileStatus] = { logInfo(s"Listing ${status.getPath}") val name = status.getPath.getName.toLowerCase if (shouldFilterOut(name)) { Array.empty } else { - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(fs.getConf, this.getClass()) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) val statuses = { val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) - val stats = files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) - if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats + val stats = files ++ dirs.flatMap(dir => listLeafFiles(fs, dir, filter)) + if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats } statuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { case f: LocatedFileStatus => f - case f => new LocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + case f => createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) } } } @@ -403,9 +415,15 @@ private[sql] object HadoopFsRelation extends Logging { val serializableConfiguration = new SerializableConfiguration(hadoopConf) val serializedPaths = paths.map(_.toString) - val fakeStatuses = sparkContext.parallelize(serializedPaths).map(new Path(_)).flatMap { path => - val fs = path.getFileSystem(serializableConfiguration.value) - Try(listLeafFiles(fs, fs.getFileStatus(path))).getOrElse(Array.empty) + val fakeStatuses = sparkContext.parallelize(serializedPaths).mapPartitions { paths => + // Dummy jobconf to get to the pathFilter defined in configuration + // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) + val jobConf = new JobConf(serializableConfiguration.value, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + paths.map(new Path(_)).flatMap { path => + val fs = path.getFileSystem(serializableConfiguration.value) + Try(listLeafFiles(fs, fs.getFileStatus(path), pathFilter)).getOrElse(Array.empty) + } }.map { status => val blockLocations = status match { case f: LocatedFileStatus => From 14751cd0d0d7ed6f64d6ec26d04ac091e1b72717 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 18 May 2016 11:48:46 +0100 Subject: [PATCH 0253/1470] [SPARK-15322][MLLIB][CORE][SQL] update deprecate accumulator usage into accumulatorV2 in spark project ## What changes were proposed in this pull request? I use Intellj-IDEA to search usage of deprecate SparkContext.accumulator in the whole spark project, and update the code.(except those test code for accumulator method itself) ## How was this patch tested? Exisiting unit tests Author: WeichenXu Closes #13112 from WeichenXu123/update_accuV2_in_mllib. (cherry picked from commit 2f9047b5eb969e0198b8a73e392642ca852ba786) Signed-off-by: Sean Owen --- .../org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 8 ++++---- .../streaming/RecoverableNetworkWordCount.scala | 11 ++++++----- .../scala/org/apache/spark/ml/util/stopwatches.scala | 7 ++++--- .../org/apache/spark/mllib/clustering/KMeans.scala | 4 ++-- .../org/apache/spark/ml/util/StopwatchSuite.scala | 8 ++++---- .../org/apache/spark/sql/DatasetPrimitiveSuite.scala | 8 ++++---- .../scala/org/apache/spark/sql/DatasetSuite.scala | 8 ++++---- 7 files changed, 28 insertions(+), 26 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 8cb0a295b0773..58664e77d24a5 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -65,9 +65,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachAsync") { zeroPartRdd.foreachAsync(i => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 3).foreachAsync { i => - accum += 1 + accum.add(1) }.get() assert(accum.value === 1000) } @@ -75,9 +75,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachPartitionAsync") { zeroPartRdd.foreachPartitionAsync(iter => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 9).foreachPartitionAsync { iter => - accum += 1 + accum.add(1) }.get() assert(accum.value === 9) } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 1bcd85e1d533f..acbcb0c4b7b78 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -23,11 +23,12 @@ import java.nio.charset.Charset import com.google.common.io.Files -import org.apache.spark.{Accumulator, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext, Time} import org.apache.spark.util.IntParam +import org.apache.spark.util.LongAccumulator /** * Use this singleton to get or register a Broadcast variable. @@ -54,13 +55,13 @@ object WordBlacklist { */ object DroppedWordsCounter { - @volatile private var instance: Accumulator[Long] = null + @volatile private var instance: LongAccumulator = null - def getInstance(sc: SparkContext): Accumulator[Long] = { + def getInstance(sc: SparkContext): LongAccumulator = { if (instance == null) { synchronized { if (instance == null) { - instance = sc.accumulator(0L, "WordsInBlacklistCounter") + instance = sc.longAccumulator("WordsInBlacklistCounter") } } } @@ -124,7 +125,7 @@ object RecoverableNetworkWordCount { // Use blacklist to drop words and use droppedWordsCounter to count them val counts = rdd.filter { case (word, count) => if (blacklist.value.contains(word)) { - droppedWordsCounter += count + droppedWordsCounter.add(count) false } else { true diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala index 8d4174124b5c4..e79b1f31643d0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala @@ -19,7 +19,8 @@ package org.apache.spark.ml.util import scala.collection.mutable -import org.apache.spark.{Accumulator, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.util.LongAccumulator; /** * Abstract class for stopwatches. @@ -102,12 +103,12 @@ private[spark] class DistributedStopwatch( sc: SparkContext, override val name: String) extends Stopwatch { - private val elapsedTime: Accumulator[Long] = sc.accumulator(0L, s"DistributedStopwatch($name)") + private val elapsedTime: LongAccumulator = sc.longAccumulator(s"DistributedStopwatch($name)") override def elapsed(): Long = elapsedTime.value override protected def add(duration: Long): Unit = { - elapsedTime += duration + elapsedTime.add(duration) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 60f13d27d0a6e..38728f2693c6c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -279,7 +279,7 @@ class KMeans private ( } val activeCenters = activeRuns.map(r => centers(r)).toArray - val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + val costAccums = activeRuns.map(_ => sc.doubleAccumulator) val bcActiveCenters = sc.broadcast(activeCenters) @@ -296,7 +296,7 @@ class KMeans private ( points.foreach { point => (0 until runs).foreach { i => val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) - costAccums(i) += cost + costAccums(i).add(cost) val sum = sums(i)(bestCenter) axpy(1.0, point.vector, sum) counts(i)(bestCenter) += 1 diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala index 9e6bc7193c13b..141249a427a4c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala @@ -60,9 +60,9 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { test("DistributedStopwatch on executors") { val sw = new DistributedStopwatch(sc, "sw") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => - acc += checkStopwatch(sw) + acc.add(checkStopwatch(sw)) } assert(!sw.isRunning) val elapsed = sw.elapsed() @@ -88,12 +88,12 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { assert(sw.toString === s"{\n local: ${localElapsed}ms,\n spark: ${sparkElapsed}ms\n}") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => sw("local").start() val duration = checkStopwatch(sw("spark")) sw("local").stop() - acc += duration + acc.add(duration) } val localElapsed2 = sw("local").elapsed() assert(localElapsed2 === localElapsed) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index ff022b2dc45ee..a634502e2e4f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -62,15 +62,15 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { test("foreach") { val ds = Seq(1, 2, 3).toDS() - val acc = sparkContext.accumulator(0) - ds.foreach(acc += _) + val acc = sparkContext.longAccumulator + ds.foreach(acc.add(_)) assert(acc.value == 6) } test("foreachPartition") { val ds = Seq(1, 2, 3).toDS() - val acc = sparkContext.accumulator(0) - ds.foreachPartition(_.foreach(acc +=)) + val acc = sparkContext.longAccumulator + ds.foreachPartition(_.foreach(acc.add(_))) assert(acc.value == 6) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 3b9feae4a31c9..b02b714168cb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -207,15 +207,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("foreach") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() - val acc = sparkContext.accumulator(0) - ds.foreach(v => acc += v._2) + val acc = sparkContext.longAccumulator + ds.foreach(v => acc.add(v._2)) assert(acc.value == 6) } test("foreachPartition") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() - val acc = sparkContext.accumulator(0) - ds.foreachPartition(_.foreach(v => acc += v._2)) + val acc = sparkContext.longAccumulator + ds.foreachPartition(_.foreach(v => acc.add(v._2))) assert(acc.value == 6) } From a122a3e709479a6510d97dfa81f3d6219bb79de0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 18 May 2016 18:58:24 +0800 Subject: [PATCH 0254/1470] [SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10 ## What changes were proposed in this pull request? This PR fixes a Scala 2.10 compilation failure introduced in PR #13127. ## How was this patch tested? Jenkins build. Author: Cheng Lian Closes #13166 from liancheng/hotfix-for-scala-2.10. (cherry picked from commit c4a45fd855252f0f7130cdcc08486591fadab7df) Signed-off-by: Cheng Lian --- .../scala/org/apache/spark/sql/hive/client/VersionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 57e8db7e88fb5..d46c4e7b2b50b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -137,7 +137,7 @@ class VersionsSuite extends SparkFunSuite with Logging { hiveMetastoreVersion = version, hadoopVersion = VersionInfo.getVersion, sparkConf = sparkConf, - hadoopConf, + hadoopConf = hadoopConf, config = buildConf(), ivyPath = ivyPath).createClient() } From fe0a0686c50743272a841b909cbfe4534350fc18 Mon Sep 17 00:00:00 2001 From: DLucky Date: Wed, 18 May 2016 12:05:21 +0100 Subject: [PATCH 0255/1470] [SPARK-15346][MLLIB] Reduce duplicate computation in picking initial points mateiz srowen I state that the contribution is my original work and that I license the work to the project under the project's open source license There's some format problems with my last PR, with HyukjinKwon 's help I read the guidance, re-check my code and PR, then run the tests, finally re-submit the PR request here. The related JIRA issue though marked as resolved, this change may relate to it I think. ## Proposed Change After picking each new initial centers, it's unnecessary to compute the distances between all the points and the old ones. Instead this change keeps the distance between all the points and their closest centers, and compare to the distance of them with the new center then update them. ## Test result One can find an easy test way in (https://issues.apache.org/jira/browse/SPARK-6706) I test the KMeans++ method for a small dataset with 16k points, and the whole KMeans|| with a large one with 240k points. The data has 4096 features and I tunes K from 100 to 500. The test environment was on my 4 machine cluster, I also tested a 3M points data on a larger cluster with 25 machines and got similar results, which I would not draw the detail curve. The result of the first two exps are shown below ### Local KMeans++ test: Dataset:4m_ini_center Data_size:16234 Dimension:4096 Lloyd's Iteration = 10 The y-axis is time in sec, the x-axis is tuning the K. ![image](https://cloud.githubusercontent.com/assets/10915169/15175831/d0c92b82-179a-11e6-8b68-4e165fc2fdff.png) ![local_total](https://cloud.githubusercontent.com/assets/10915169/15175957/6b21c3b0-179b-11e6-9741-66dfe4e23eb7.jpg) ### On a larger dataset An improve show in the graph but not commit in this file: In this experiment I also have an improvement for calculation in normalization data (the distance is convert to the cosine distance). As if the data is normalized into (0,1), one improvement in the original vesion for util.MLUtils.fastSauaredDistance would have no effect (the precisionBound 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) will never less then precision in this case). Therefore I design an early terminal method when comparing two distance (used for findClosest). But I don't include this improve in this file, you may only refer to the curves without "normalize" for comparing the results. Dataset:4k24 Data_size:243960 Dimension:4096 Normlize Enlarge Initialize Lloyd's_Iteration NO 1 3 5 YES 10000 3 5 Notice: the normlized data is enlarged to ensure precision The cost time: x-for value of K, y-for time in sec ![4k24_total](https://cloud.githubusercontent.com/assets/10915169/15176635/9a54c0bc-179e-11e6-81c5-238e0c54bce2.jpg) SE for unnormalized data between two version, to ensure the correctness ![4k24_unnorm_se](https://cloud.githubusercontent.com/assets/10915169/15176661/b85dabc8-179e-11e6-9269-fe7d2101dd48.jpg) Here is the SE between normalized data just for reference, it's also correct. ![4k24_norm_se](https://cloud.githubusercontent.com/assets/10915169/15176742/1fbde940-179f-11e6-8290-d24b0dd4a4f7.jpg) Author: DLucky Closes #13133 from mouendless/patch-2. (cherry picked from commit 420b700695fe8bcdda406c34ad48230b9dfc07f1) Signed-off-by: Sean Owen --- .../spark/mllib/clustering/LocalKMeans.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index adf20dc4b8b16..53587670a5db0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -46,17 +46,15 @@ private[mllib] object LocalKMeans extends Logging { // Initialize centers by sampling using the k-means++ procedure. centers(0) = pickWeighted(rand, points, weights).toDense + val costArray = points.map(KMeans.fastSquaredDistance(_, centers(0))) + for (i <- 1 until k) { - // Pick the next center with a probability proportional to cost under current centers - val curCenters = centers.view.take(i) - val sum = points.view.zip(weights).map { case (p, w) => - w * KMeans.pointCost(curCenters, p) - }.sum + val sum = costArray.zip(weights).map(p => p._1 * p._2).sum val r = rand.nextDouble() * sum var cumulativeScore = 0.0 var j = 0 while (j < points.length && cumulativeScore < r) { - cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) + cumulativeScore += weights(j) * costArray(j) j += 1 } if (j == 0) { @@ -66,6 +64,12 @@ private[mllib] object LocalKMeans extends Logging { } else { centers(i) = points(j - 1).toDense } + + // update costArray + for (p <- points.indices) { + costArray(p) = math.min(KMeans.fastSquaredDistance(points(p), centers(i)), costArray(p)) + } + } // Run up to maxIterations iterations of Lloyd's algorithm From 7ae006fdef6bd834067c9e80851116bcd4503b7f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 09:44:21 -0700 Subject: [PATCH 0256/1470] [SPARK-15357] Cooperative spilling should check consumer memory mode ## What changes were proposed in this pull request? Since we support forced spilling for Spillable, which only works in OnHeap mode, different from other SQL operators (could be OnHeap or OffHeap), we should considering the mode of consumer before calling trigger forced spilling. ## How was this patch tested? Add new test. Author: Davies Liu Closes #13151 from davies/fix_mode. (cherry picked from commit 8fb1d1c7f3ed1b62625052a532b7388ebec71bbf) Signed-off-by: Davies Liu --- .../apache/spark/memory/MemoryConsumer.java | 25 ++++-- .../spark/memory/TaskMemoryManager.java | 37 ++++----- .../shuffle/sort/ShuffleExternalSorter.java | 5 +- .../spark/unsafe/map/BytesToBytesMap.java | 2 +- .../unsafe/sort/UnsafeExternalSorter.java | 2 +- .../org/apache/spark/executor/Executor.scala | 10 +-- .../spark/util/collection/Spillable.scala | 4 +- .../spark/memory/TaskMemoryManagerSuite.java | 28 ++++++- .../spark/memory/TestMemoryConsumer.java | 15 ++-- .../sort/PackedRecordPointerSuite.java | 13 +-- .../sort/ShuffleInMemorySorterSuite.java | 4 +- .../sort/UnsafeInMemorySorterSuite.java | 2 +- .../scala/org/apache/spark/FailureSuite.scala | 7 +- .../spark/memory/MemoryManagerSuite.scala | 80 +++++++++++-------- project/MimaExcludes.scala | 1 + .../sql/execution/joins/HashedRelation.scala | 16 ++-- 16 files changed, 145 insertions(+), 106 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 840f13b39464c..38a21a896e1fe 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -31,15 +31,24 @@ public abstract class MemoryConsumer { protected final TaskMemoryManager taskMemoryManager; private final long pageSize; + private final MemoryMode mode; protected long used; - protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize) { + protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize, MemoryMode mode) { this.taskMemoryManager = taskMemoryManager; this.pageSize = pageSize; + this.mode = mode; } protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { - this(taskMemoryManager, taskMemoryManager.pageSizeBytes()); + this(taskMemoryManager, taskMemoryManager.pageSizeBytes(), MemoryMode.ON_HEAP); + } + + /** + * Returns the memory mode, ON_HEAP or OFF_HEAP. + */ + public MemoryMode getMode() { + return mode; } /** @@ -132,19 +141,19 @@ protected void freePage(MemoryBlock page) { } /** - * Allocates a heap memory of `size`. + * Allocates memory of `size`. */ - public long acquireOnHeapMemory(long size) { - long granted = taskMemoryManager.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this); + public long acquireMemory(long size) { + long granted = taskMemoryManager.acquireExecutionMemory(size, this); used += granted; return granted; } /** - * Release N bytes of heap memory. + * Release N bytes of memory. */ - public void freeOnHeapMemory(long size) { - taskMemoryManager.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this); + public void freeMemory(long size) { + taskMemoryManager.releaseExecutionMemory(size, this); used -= size; } } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index a05a79c88df76..a4a571f15a8c0 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -76,9 +76,6 @@ public class TaskMemoryManager { /** Bit mask for the lower 51 bits of a long. */ private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL; - /** Bit mask for the upper 13 bits of a long */ - private static final long MASK_LONG_UPPER_13_BITS = ~MASK_LONG_LOWER_51_BITS; - /** * Similar to an operating system's page table, this array maps page numbers into base object * pointers, allowing us to translate between the hashtable's internal 64-bit address @@ -132,11 +129,10 @@ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { * * @return number of bytes successfully granted (<= N). */ - public long acquireExecutionMemory( - long required, - MemoryMode mode, - MemoryConsumer consumer) { + public long acquireExecutionMemory(long required, MemoryConsumer consumer) { assert(required >= 0); + assert(consumer != null); + MemoryMode mode = consumer.getMode(); // If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap // memory here, then it may not make sense to spill since that would only end up freeing // off-heap memory. This is subject to change, though, so it may be risky to make this @@ -149,10 +145,10 @@ public long acquireExecutionMemory( if (got < required) { // Call spill() on other consumers to release memory for (MemoryConsumer c: consumers) { - if (c != consumer && c.getUsed() > 0) { + if (c != consumer && c.getUsed() > 0 && c.getMode() == mode) { try { long released = c.spill(required - got, consumer); - if (released > 0 && mode == tungstenMemoryMode) { + if (released > 0) { logger.debug("Task {} released {} from {} for {}", taskAttemptId, Utils.bytesToString(released), c, consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); @@ -170,10 +166,10 @@ public long acquireExecutionMemory( } // call spill() on itself - if (got < required && consumer != null) { + if (got < required) { try { long released = consumer.spill(required - got, consumer); - if (released > 0 && mode == tungstenMemoryMode) { + if (released > 0) { logger.debug("Task {} released {} from itself ({})", taskAttemptId, Utils.bytesToString(released), consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); @@ -185,9 +181,7 @@ public long acquireExecutionMemory( } } - if (consumer != null) { - consumers.add(consumer); - } + consumers.add(consumer); logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); return got; } @@ -196,9 +190,9 @@ public long acquireExecutionMemory( /** * Release N bytes of execution memory for a MemoryConsumer. */ - public void releaseExecutionMemory(long size, MemoryMode mode, MemoryConsumer consumer) { + public void releaseExecutionMemory(long size, MemoryConsumer consumer) { logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer); - memoryManager.releaseExecutionMemory(size, taskAttemptId, mode); + memoryManager.releaseExecutionMemory(size, taskAttemptId, consumer.getMode()); } /** @@ -241,12 +235,14 @@ public long pageSizeBytes() { * contains fewer bytes than requested, so callers should verify the size of returned pages. */ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { + assert(consumer != null); + assert(consumer.getMode() == tungstenMemoryMode); if (size > MAXIMUM_PAGE_SIZE_BYTES) { throw new IllegalArgumentException( "Cannot allocate a page with more than " + MAXIMUM_PAGE_SIZE_BYTES + " bytes"); } - long acquired = acquireExecutionMemory(size, tungstenMemoryMode, consumer); + long acquired = acquireExecutionMemory(size, consumer); if (acquired <= 0) { return null; } @@ -255,7 +251,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { synchronized (this) { pageNumber = allocatedPages.nextClearBit(0); if (pageNumber >= PAGE_TABLE_SIZE) { - releaseExecutionMemory(acquired, tungstenMemoryMode, consumer); + releaseExecutionMemory(acquired, consumer); throw new IllegalStateException( "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); } @@ -299,7 +295,7 @@ public void freePage(MemoryBlock page, MemoryConsumer consumer) { } long pageSize = page.size(); memoryManager.tungstenMemoryAllocator().free(page); - releaseExecutionMemory(pageSize, tungstenMemoryMode, consumer); + releaseExecutionMemory(pageSize, consumer); } /** @@ -396,8 +392,7 @@ public long cleanUpAllAllocatedMemory() { Arrays.fill(pageTable, null); } - - // release the memory that is not used by any consumer. + // release the memory that is not used by any consumer (acquired for pages in tungsten mode). memoryManager.releaseExecutionMemory(acquiredButNotUsed, taskAttemptId, tungstenMemoryMode); return memoryManager.releaseAllExecutionMemoryForTask(taskAttemptId); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 2be5a16b2d1e4..014aef86b5cc6 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -104,8 +104,9 @@ final class ShuffleExternalSorter extends MemoryConsumer { int numPartitions, SparkConf conf, ShuffleWriteMetrics writeMetrics) { - super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, - memoryManager.pageSizeBytes())); + super(memoryManager, + (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), + memoryManager.getTungstenMemoryMode()); this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 6807710f9fef1..6c00608302c4e 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -182,7 +182,7 @@ public BytesToBytesMap( double loadFactor, long pageSizeBytes, boolean enablePerfMetrics) { - super(taskMemoryManager, pageSizeBytes); + super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; this.serializerManager = serializerManager; diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 7dc0508784981..e14a23f4a6a83 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -124,7 +124,7 @@ private UnsafeExternalSorter( long pageSizeBytes, @Nullable UnsafeInMemorySorter existingInMemorySorter, boolean canUseRadixSort) { - super(taskMemoryManager, pageSizeBytes); + super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; this.serializerManager = serializerManager; 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 b695aecc13ea1..9a017f29f7d21 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -281,20 +281,20 @@ private[spark] class Executor( val releasedLocks = env.blockManager.releaseAllLocksForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() - if (freedMemory > 0) { + if (freedMemory > 0 && !threwException) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" - if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false) && !threwException) { + if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { throw new SparkException(errMsg) } else { - logError(errMsg) + logWarning(errMsg) } } - if (releasedLocks.nonEmpty) { + if (releasedLocks.nonEmpty && !threwException) { val errMsg = s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" + releasedLocks.mkString("[", ", ", "]") - if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) { + if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false)) { throw new SparkException(errMsg) } else { logWarning(errMsg) diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index bdcbd22fd814b..8183f825592c0 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -83,7 +83,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = acquireOnHeapMemory(amountToRequest) + val granted = acquireMemory(amountToRequest) myMemoryThreshold += granted // If we were granted too little memory to grow further (either tryToAcquire returned 0, // or we already had more memory than myMemoryThreshold), spill the current collection @@ -131,7 +131,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) * Release our memory back to the execution pool so that other tasks can grab it. */ def releaseMemory(): Unit = { - freeOnHeapMemory(myMemoryThreshold - initialMemoryThreshold) + freeMemory(myMemoryThreshold - initialMemoryThreshold) myMemoryThreshold = initialMemoryThreshold } diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index 127789b632b44..ad755529dec64 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -34,7 +34,8 @@ public void leakedPageMemoryIsDetected() { Long.MAX_VALUE, 1), 0); - manager.allocatePage(4096, null); // leak memory + final MemoryConsumer c = new TestMemoryConsumer(manager); + manager.allocatePage(4096, c); // leak memory Assert.assertEquals(4096, manager.getMemoryConsumptionForThisTask()); Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory()); } @@ -45,7 +46,8 @@ public void encodePageNumberAndOffsetOffHeap() { .set("spark.memory.offHeap.enabled", "true") .set("spark.memory.offHeap.size", "1000"); final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); // In off-heap mode, an offset is an absolute address that may require more than 51 bits to // encode. This test exercises that corner-case: final long offset = ((1L << TaskMemoryManager.OFFSET_BITS) + 10); @@ -58,7 +60,8 @@ public void encodePageNumberAndOffsetOffHeap() { public void encodePageNumberAndOffsetOnHeap() { final TaskMemoryManager manager = new TaskMemoryManager( new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, 64); Assert.assertEquals(dataPage.getBaseObject(), manager.getPage(encodedAddress)); Assert.assertEquals(64, manager.getOffsetInPage(encodedAddress)); @@ -106,6 +109,25 @@ public void cooperativeSpilling() { Assert.assertEquals(0, manager.cleanUpAllAllocatedMemory()); } + @Test + public void shouldNotForceSpillingInDifferentModes() { + final TestMemoryManager memoryManager = new TestMemoryManager(new SparkConf()); + memoryManager.limit(100); + final TaskMemoryManager manager = new TaskMemoryManager(memoryManager, 0); + + TestMemoryConsumer c1 = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + TestMemoryConsumer c2 = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + c1.use(80); + Assert.assertEquals(80, c1.getUsed()); + c2.use(80); + Assert.assertEquals(20, c2.getUsed()); // not enough memory + Assert.assertEquals(80, c1.getUsed()); // not spilled + + c2.use(10); + Assert.assertEquals(10, c2.getUsed()); // spilled + Assert.assertEquals(80, c1.getUsed()); // not spilled + } + @Test public void offHeapConfigurationBackwardsCompatibility() { // Tests backwards-compatibility with the old `spark.unsafe.offHeap` configuration, which diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java index e6e16fff80401..db91329c94cb6 100644 --- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -20,8 +20,11 @@ import java.io.IOException; public class TestMemoryConsumer extends MemoryConsumer { + public TestMemoryConsumer(TaskMemoryManager memoryManager, MemoryMode mode) { + super(memoryManager, 1024L, mode); + } public TestMemoryConsumer(TaskMemoryManager memoryManager) { - super(memoryManager); + this(memoryManager, MemoryMode.ON_HEAP); } @Override @@ -32,19 +35,13 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { } void use(long size) { - long got = taskMemoryManager.acquireExecutionMemory( - size, - taskMemoryManager.tungstenMemoryMode, - this); + long got = taskMemoryManager.acquireExecutionMemory(size, this); used += got; } void free(long size) { used -= size; - taskMemoryManager.releaseExecutionMemory( - size, - taskMemoryManager.tungstenMemoryMode, - this); + taskMemoryManager.releaseExecutionMemory(size, this); } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java index fe5abc5c23049..354efe18dbde7 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java @@ -22,8 +22,7 @@ import org.junit.Test; import org.apache.spark.SparkConf; -import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.*; import org.apache.spark.unsafe.memory.MemoryBlock; import static org.apache.spark.shuffle.sort.PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES; @@ -38,8 +37,9 @@ public void heap() throws IOException { final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.ON_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); @@ -59,8 +59,9 @@ public void offHeap() throws IOException { .set("spark.memory.offHeap.size", "10000"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.OFF_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java index 278a827644db7..694352ee2af44 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java @@ -26,6 +26,7 @@ import org.apache.spark.HashPartitioner; import org.apache.spark.SparkConf; +import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; @@ -71,7 +72,8 @@ public void testBasicSorting() throws Exception { final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, c); final Object baseObject = dataPage.getBaseObject(); final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter( consumer, 4, shouldUseRadixSort()); diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 4a2f65a0ed2b2..383c5b3b0884a 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -78,7 +78,7 @@ public void testSortingOnlyByIntegerPrefix() throws Exception { final TaskMemoryManager memoryManager = new TaskMemoryManager( new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer); final Object baseObject = dataPage.getBaseObject(); // Write the records into the data page: long position = dataPage.getBaseOffset(); diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 3def8b0b1850e..333c23bdaf6d6 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{IOException, NotSerializableException, ObjectInputStream} +import org.apache.spark.memory.TestMemoryConsumer import org.apache.spark.util.NonSerializable // Common state shared by FailureSuite-launched tasks. We use a global object @@ -149,7 +150,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // cause is preserved val thrownDueToTaskFailure = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) throw new Exception("intentional task failure") iter }.count() @@ -159,7 +161,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // If the task succeeded but memory was leaked, then the task should fail due to that leak val thrownDueToMemoryLeak = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) iter }.count() } diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 2c4928ab907ad..38bf7e5e5aec3 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -162,39 +162,42 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft test("single task requesting on-heap execution memory") { val manager = createMemoryManager(1000L) val taskMemoryManager = new TaskMemoryManager(manager, 0) + val c = new TestMemoryConsumer(taskMemoryManager) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(200L, MemoryMode.ON_HEAP, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(200L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) - taskMemoryManager.releaseExecutionMemory(500L, MemoryMode.ON_HEAP, null) - assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 300L) - assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 200L) + taskMemoryManager.releaseExecutionMemory(500L, c) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 300L) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 200L) taskMemoryManager.cleanUpAllAllocatedMemory() - assert(taskMemoryManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) === 1000L) - assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(1000L, c) === 1000L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) } test("two tasks requesting full on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 500 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 500L) assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) // Have both tasks each request 500 bytes more; both should immediately return 0 as they are // both now at 1 / N - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } @@ -203,18 +206,20 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 250 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 250L) assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Have both tasks each request 500 bytes more. // We should only grant 250 bytes to each of them on this second request - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t1Result2, futureTimeout) === 250L) assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 250L) } @@ -223,20 +228,22 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) - t1MemManager.releaseExecutionMemory(250L, MemoryMode.ON_HEAP, null) + t1MemManager.releaseExecutionMemory(250L, c1) // The memory freed from t1 should now be granted to t2. assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory. - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, c2) } assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } @@ -244,21 +251,23 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) // t1 releases all of its memory, so t2 should be able to grab all of the memory t1MemManager.cleanUpAllAllocatedMemory() assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 500L) - val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } assert(ThreadUtils.awaitResult(t2Result3, 200.millis) === 0L) } @@ -267,15 +276,17 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, MemoryMode.ON_HEAP, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, c1) } assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 700L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, c2) } assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 300L) - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, c1) } assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) } @@ -285,17 +296,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft maxOffHeapExecutionMemory = 1000L) val tMemManager = new TaskMemoryManager(memoryManager, 1) - val result1 = Future { tMemManager.acquireExecutionMemory(1000L, MemoryMode.OFF_HEAP, null) } + val c = new TestMemoryConsumer(tMemManager, MemoryMode.OFF_HEAP) + val result1 = Future { tMemManager.acquireExecutionMemory(1000L, c) } assert(ThreadUtils.awaitResult(result1, 200.millis) === 1000L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) - val result2 = Future { tMemManager.acquireExecutionMemory(300L, MemoryMode.OFF_HEAP, null) } + val result2 = Future { tMemManager.acquireExecutionMemory(300L, c) } assert(ThreadUtils.awaitResult(result2, 200.millis) === 0L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) - tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + tMemManager.releaseExecutionMemory(500L, c) assert(tMemManager.getMemoryConsumptionForThisTask === 500L) - tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + tMemManager.releaseExecutionMemory(500L, c) assert(tMemManager.getMemoryConsumptionForThisTask === 0L) } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 45f7297048382..4e99a0965780b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -40,6 +40,7 @@ object MimaExcludes { excludePackage("org.spark-project.jetty"), excludePackage("org.apache.spark.unused"), excludePackage("org.apache.spark.unsafe"), + excludePackage("org.apache.spark.memory"), excludePackage("org.apache.spark.util.collection.unsafe"), excludePackage("org.apache.spark.sql.catalyst"), excludePackage("org.apache.spark.sql.execution"), 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 315ef6a8796f0..cb41457b6653f 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 @@ -398,9 +398,9 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap 0) } - private def acquireMemory(size: Long): Unit = { + private def ensureAcquireMemory(size: Long): Unit = { // do not support spilling - val got = mm.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this) + val got = acquireMemory(size) if (got < size) { freeMemory(got) throw new SparkException(s"Can't acquire $size bytes memory to build hash relation, " + @@ -408,15 +408,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } - private def freeMemory(size: Long): Unit = { - mm.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this) - } - private def init(): Unit = { if (mm != null) { var n = 1 while (n < capacity) n *= 2 - acquireMemory(n * 2 * 8 + (1 << 20)) + ensureAcquireMemory(n * 2 * 8 + (1 << 20)) array = new Array[Long](n * 2) mask = n * 2 - 2 page = new Array[Long](1 << 17) // 1M bytes @@ -538,7 +534,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap if (used >= (1 << 30)) { sys.error("Can not build a HashedRelation that is larger than 8G") } - acquireMemory(used * 8L * 2) + ensureAcquireMemory(used * 8L * 2) val newPage = new Array[Long](used * 2) Platform.copyMemory(page, Platform.LONG_ARRAY_OFFSET, newPage, Platform.LONG_ARRAY_OFFSET, cursor - Platform.LONG_ARRAY_OFFSET) @@ -591,7 +587,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap var old_array = array val n = array.length numKeys = 0 - acquireMemory(n * 2 * 8L) + ensureAcquireMemory(n * 2 * 8L) array = new Array[Long](n * 2) mask = n * 2 - 2 var i = 0 @@ -613,7 +609,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap // Convert to dense mode if it does not require more memory or could fit within L1 cache if (range < array.length || range < 1024) { try { - acquireMemory((range + 1) * 8) + ensureAcquireMemory((range + 1) * 8) } catch { case e: SparkException => // there is no enough memory to convert From 67c54721d35b1a97391a552e739ddcde3405bff9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 May 2016 11:51:50 +0100 Subject: [PATCH 0257/1470] [MINOR][SQL] Remove unused pattern matching variables in Optimizers. ## What changes were proposed in this pull request? This PR removes unused pattern matching variable in Optimizers in order to improve readability. ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13145 from dongjoon-hyun/remove_unused_pattern_matching_variables. (cherry picked from commit d2f81df1ba67f5d3fc39719d9a67c52d7903e59e) Signed-off-by: Sean Owen --- .../sql/catalyst/optimizer/Optimizer.scala | 33 +++++++++---------- 1 file changed, 16 insertions(+), 17 deletions(-) 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 af7532e0c03ec..be9f03d4baaaf 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 @@ -151,9 +151,8 @@ object SamplePushDown extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Push down projection into sample - case Project(projectList, s @ Sample(lb, up, replace, seed, child)) => - Sample(lb, up, replace, seed, - Project(projectList, child))() + case Project(projectList, Sample(lb, up, replace, seed, child)) => + Sample(lb, up, replace, seed, Project(projectList, child))() } } @@ -226,7 +225,7 @@ object LimitPushDown extends Rule[LogicalPlan] { private def stripGlobalLimitIfPresent(plan: LogicalPlan): LogicalPlan = { plan match { - case GlobalLimit(expr, child) => child + case GlobalLimit(_, child) => child case _ => plan } } @@ -259,7 +258,7 @@ object LimitPushDown extends Rule[LogicalPlan] { // - If one side is already limited, stack another limit on top if the new limit is smaller. // The redundant limit will be collapsed by the CombineLimits rule. // - If neither side is limited, limit the side that is estimated to be bigger. - case LocalLimit(exp, join @ Join(left, right, joinType, condition)) => + case LocalLimit(exp, join @ Join(left, right, joinType, _)) => val newJoin = joinType match { case RightOuter => join.copy(right = maybePushLimit(exp, right)) case LeftOuter => join.copy(left = maybePushLimit(exp, left)) @@ -408,7 +407,7 @@ object ColumnPruning extends Rule[LogicalPlan] { p.copy(child = g.copy(join = false)) // Eliminate unneeded attributes from right side of a Left Existence Join. - case j @ Join(left, right, LeftExistence(_), condition) => + case j @ Join(_, right, LeftExistence(_), _) => j.copy(right = prunedChild(right, j.references)) // all the columns will be used to compare, so we can't prune them @@ -440,10 +439,10 @@ object ColumnPruning extends Rule[LogicalPlan] { case w: Window if w.windowExpressions.isEmpty => w.child // Eliminate no-op Projects - case p @ Project(projectList, child) if sameOutput(child.output, p.output) => child + case p @ Project(_, child) if sameOutput(child.output, p.output) => child // Can't prune the columns on LeafNode - case p @ Project(_, l: LeafNode) => p + case p @ Project(_, _: LeafNode) => p // for all other logical plans that inherits the output from it's children case p @ Project(_, child) => @@ -541,7 +540,7 @@ object CollapseProject extends Rule[LogicalPlan] { */ object CollapseRepartition extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case r @ Repartition(numPartitions, shuffle, Repartition(_, _, child)) => + case Repartition(numPartitions, shuffle, Repartition(_, _, child)) => Repartition(numPartitions, shuffle, child) } } @@ -917,7 +916,7 @@ object CombineUnions extends Rule[LogicalPlan] { */ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case ff @ Filter(fc, nf @ Filter(nc, grandChild)) => + case Filter(fc, nf @ Filter(nc, grandChild)) => (ExpressionSet(splitConjunctivePredicates(fc)) -- ExpressionSet(splitConjunctivePredicates(nc))).reduceOption(And) match { case Some(ac) => @@ -1071,9 +1070,9 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { } // two filters should be combine together by other rules - case filter @ Filter(_, f: Filter) => filter + case filter @ Filter(_, _: Filter) => filter // should not push predicates through sample, or will generate different results. - case filter @ Filter(_, s: Sample) => filter + case filter @ Filter(_, _: Sample) => filter case filter @ Filter(condition, u: UnaryNode) if u.expressions.forall(_.deterministic) => pushDownPredicate(filter, u.child) { predicate => @@ -1352,11 +1351,11 @@ object RemoveDispensableExpressions extends Rule[LogicalPlan] { */ object CombineLimits extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case ll @ GlobalLimit(le, nl @ GlobalLimit(ne, grandChild)) => + case GlobalLimit(le, GlobalLimit(ne, grandChild)) => GlobalLimit(Least(Seq(ne, le)), grandChild) - case ll @ LocalLimit(le, nl @ LocalLimit(ne, grandChild)) => + case LocalLimit(le, LocalLimit(ne, grandChild)) => LocalLimit(Least(Seq(ne, le)), grandChild) - case ll @ Limit(le, nl @ Limit(ne, grandChild)) => + case Limit(le, Limit(ne, grandChild)) => Limit(Least(Seq(ne, le)), grandChild) } } @@ -1588,7 +1587,7 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] { */ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case f @ Filter(condition, child) => + case Filter(condition, child) => val (withSubquery, withoutSubquery) = splitConjunctivePredicates(condition).partition(PredicateSubquery.hasPredicateSubquery) @@ -1619,7 +1618,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { val replaced = predicate transformUp { case PredicateSubquery(sub, conditions, nullAware, _) => // TODO: support null-aware join - val exists = AttributeReference("exists", BooleanType, false)() + val exists = AttributeReference("exists", BooleanType, nullable = false)() joined = Join(joined, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) exists } From d005f76e6c3a5a01153c0189e774b9717c1a51f9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 18 May 2016 11:18:33 -0700 Subject: [PATCH 0258/1470] [SPARK-15342] [SQL] [PYSPARK] PySpark test for non ascii column name does not actually test with unicode column name ## What changes were proposed in this pull request? The PySpark SQL `test_column_name_with_non_ascii` wants to test non-ascii column name. But it doesn't actually test it. We need to construct an unicode explicitly using `unicode` under Python 2. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13134 from viirya/correct-non-ascii-colname-pytest. (cherry picked from commit 3d1e67f903ab3512fcad82b94b1825578f8117c9) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 11 +++++++++-- python/pyspark/sql/types.py | 3 ++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index e86f44281d492..1790432edd5dc 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1044,8 +1044,15 @@ def test_access_column(self): self.assertRaises(TypeError, lambda: df[{}]) def test_column_name_with_non_ascii(self): - df = self.spark.createDataFrame([(1,)], ["数量"]) - self.assertEqual(StructType([StructField("数量", LongType(), True)]), df.schema) + if sys.version >= '3': + columnName = "数量" + self.assertTrue(isinstance(columnName, str)) + else: + columnName = unicode("数量", "utf-8") + self.assertTrue(isinstance(columnName, unicode)) + schema = StructType([StructField(columnName, LongType(), True)]) + df = self.spark.createDataFrame([(1,)], schema) + self.assertEqual(schema, df.schema) self.assertEqual("DataFrame[数量: bigint]", str(df)) self.assertEqual([("数量", 'bigint')], df.dtypes) self.assertEqual(1, df.select("数量").first()[0]) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 30ab130f29481..7d8d0230b4ab3 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -27,7 +27,7 @@ if sys.version >= "3": long = int - unicode = str + basestring = unicode = str from py4j.protocol import register_input_converter from py4j.java_gateway import JavaClass @@ -401,6 +401,7 @@ def __init__(self, name, dataType, nullable=True, metadata=None): False """ assert isinstance(dataType, DataType), "dataType should be DataType" + assert isinstance(name, basestring), "field name should be string" if not isinstance(name, str): name = name.encode('utf-8') self.name = name From 0da8bce0e3fcf6a7f40b5e23e57ce45795926432 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 18 May 2016 21:13:12 +0200 Subject: [PATCH 0259/1470] [SPARK-14891][ML] Add schema validation for ALS This PR adds schema validation to `ml`'s ALS and ALSModel. Currently, no schema validation was performed as `transformSchema` was never called in `ALS.fit` or `ALSModel.transform`. Furthermore, due to no schema validation, if users passed in Long (or Float etc) ids, they would be silently cast to Int with no warning or error thrown. With this PR, ALS now supports all numeric types for `user`, `item`, and `rating` columns. The rating column is cast to `Float` and the user and item cols are cast to `Int` (as is the case currently) - however for user/item, the cast throws an error if the value is outside integer range. Behavior for rating col is unchanged (as it is not an issue). ## How was this patch tested? New test cases in `ALSSuite`. Author: Nick Pentreath Closes #12762 from MLnick/SPARK-14891-als-validate-schema. (cherry picked from commit e8b79afa024123f9d4ceaf0a1043a7e37d913a8d) Signed-off-by: Nick Pentreath --- .../apache/spark/ml/recommendation/ALS.scala | 55 ++++++++++++----- .../spark/ml/recommendation/ALSSuite.scala | 61 +++++++++++++++++-- .../apache/spark/ml/util/MLTestingUtils.scala | 45 ++++++++++++++ python/pyspark/ml/recommendation.py | 8 +-- 4 files changed, 147 insertions(+), 22 deletions(-) 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 509c944fed74c..f257382d2205c 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 @@ -42,7 +42,7 @@ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} @@ -53,24 +53,43 @@ import org.apache.spark.util.random.XORShiftRandom */ private[recommendation] trait ALSModelParams extends Params with HasPredictionCol { /** - * Param for the column name for user ids. + * Param for the column name for user ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "user" * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids") + val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getUserCol: String = $(userCol) /** - * Param for the column name for item ids. + * Param for the column name for item ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "item" * @group param */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids") + val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getItemCol: String = $(itemCol) + + /** + * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is + * out of integer range. + */ + protected val checkedCast = udf { (n: Double) => + if (n > Int.MaxValue || n < Int.MinValue) { + throw new IllegalArgumentException(s"ALS only supports values in Integer range for columns " + + s"${$(userCol)} and ${$(itemCol)}. Value $n was out of Integer range.") + } else { + n.toInt + } + } } /** @@ -193,10 +212,11 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w * @return output schema */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) - val ratingType = schema($(ratingCol)).dataType - require(ratingType == FloatType || ratingType == DoubleType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) + // rating will be cast to Float + SchemaUtils.checkNumericType(schema, $(ratingCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } } @@ -232,6 +252,7 @@ class ALSModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema) // Register a UDF for DataFrame, and then // create a new column named map(predictionCol) by running the predict UDF. val predict = udf { (userFeatures: Seq[Float], itemFeatures: Seq[Float]) => @@ -242,16 +263,19 @@ class ALSModel private[ml] ( } } dataset - .join(userFactors, dataset($(userCol)) === userFactors("id"), "left") - .join(itemFactors, dataset($(itemCol)) === itemFactors("id"), "left") + .join(userFactors, + checkedCast(dataset($(userCol)).cast(DoubleType)) === userFactors("id"), "left") + .join(itemFactors, + checkedCast(dataset($(itemCol)).cast(DoubleType)) === itemFactors("id"), "left") .select(dataset("*"), predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) } @Since("1.3.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } @@ -430,10 +454,13 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] @Since("2.0.0") override def fit(dataset: Dataset[_]): ALSModel = { + transformSchema(dataset.schema) import dataset.sparkSession.implicits._ + val r = if ($(ratingCol) != "") col($(ratingCol)).cast(FloatType) else lit(1.0f) val ratings = dataset - .select(col($(userCol)).cast(IntegerType), col($(itemCol)).cast(IntegerType), r) + .select(checkedCast(col($(userCol)).cast(DoubleType)), + checkedCast(col($(itemCol)).cast(DoubleType)), r) .rdd .map { row => Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index bbfc415cbb9b7..59b5edc4013e8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types.{FloatType, IntegerType} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -205,7 +206,6 @@ class ALSSuite /** * Generates an explicit feedback dataset for testing ALS. - * * @param numUsers number of users * @param numItems number of items * @param rank rank @@ -246,7 +246,6 @@ class ALSSuite /** * Generates an implicit feedback dataset for testing ALS. - * * @param numUsers number of users * @param numItems number of items * @param rank rank @@ -265,7 +264,6 @@ class ALSSuite /** * Generates random user/item factors, with i.i.d. values drawn from U(a, b). - * * @param size number of users/items * @param rank number of features * @param random random number generator @@ -284,7 +282,6 @@ class ALSSuite /** * Test ALS using the given training/test splits and parameters. - * * @param training training dataset * @param test test dataset * @param rank rank of the matrix factorization @@ -486,6 +483,62 @@ class ALSSuite assert(getFactors(model.userFactors) === getFactors(model2.userFactors)) assert(getFactors(model.itemFactors) === getFactors(model2.itemFactors)) } + + test("input type validation") { + val spark = this.spark + import spark.implicits._ + + // check that ALS can handle all numeric types for rating column + // and user/item columns (when the user/item ids are within Int range) + val als = new ALS().setMaxIter(1).setRank(1) + Seq(("user", IntegerType), ("item", IntegerType), ("rating", FloatType)).foreach { + case (colName, sqlType) => + MLTestingUtils.checkNumericTypesALS(als, spark, colName, sqlType) { + (ex, act) => + ex.userFactors.first().getSeq[Float](1) === act.userFactors.first.getSeq[Float](1) + } { (ex, act, _) => + ex.transform(_: DataFrame).select("prediction").first.getFloat(0) ~== + act.transform(_: DataFrame).select("prediction").first.getFloat(0) absTol 1e-6 + } + } + // check user/item ids falling outside of Int range + val big = Int.MaxValue.toLong + 1 + val small = Int.MinValue.toDouble - 1 + val df = Seq( + (0, 0L, 0d, 1, 1L, 1d, 3.0), + (0, big, small, 0, big, small, 2.0), + (1, 1L, 1d, 0, 0L, 0d, 5.0) + ).toDF("user", "user_big", "user_small", "item", "item_big", "item_small", "rating") + withClue("fit should fail when ids exceed integer range. ") { + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("user_big").as("user"), df("item"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("user_small").as("user"), df("item"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("item_big").as("item"), df("user"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + assert(intercept[IllegalArgumentException] { + als.fit(df.select(df("item_small").as("item"), df("user"), df("rating"))) + }.getMessage.contains("was out of Integer range")) + } + withClue("transform should fail when ids exceed integer range. ") { + val model = als.fit(df) + assert(intercept[SparkException] { + model.transform(df.select(df("user_big").as("user"), df("item"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("user_small").as("user"), df("item"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("item_big").as("item"), df("user"))).first + }.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { + model.transform(df.select(df("item_small").as("item"), df("user"))).first + }.getMessage.contains("was out of Integer range")) + } + } } class ALSCleanerSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index 6aae625fc83f2..80b976914cbdf 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -22,6 +22,7 @@ import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.recommendation.{ALS, ALSModel} import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.functions._ @@ -58,6 +59,30 @@ object MLTestingUtils extends SparkFunSuite { "Column label must be of type NumericType but was actually of type StringType")) } + def checkNumericTypesALS( + estimator: ALS, + spark: SparkSession, + column: String, + baseType: NumericType) + (check: (ALSModel, ALSModel) => Unit) + (check2: (ALSModel, ALSModel, DataFrame) => Unit): Unit = { + val dfs = genRatingsDFWithNumericCols(spark, column) + val expected = estimator.fit(dfs(baseType)) + val actuals = dfs.keys.filter(_ != baseType).map(t => (t, estimator.fit(dfs(t)))) + actuals.foreach { case (_, actual) => check(expected, actual) } + actuals.foreach { case (t, actual) => check2(expected, actual, dfs(t)) } + + val baseDF = dfs(baseType) + val others = baseDF.columns.toSeq.diff(Seq(column)).map(col(_)) + val cols = Seq(col(column).cast(StringType)) ++ others + val strDF = baseDF.select(cols: _*) + val thrown = intercept[IllegalArgumentException] { + estimator.fit(strDF) + } + assert(thrown.getMessage.contains( + s"$column must be of type NumericType but was actually of type StringType")) + } + def checkNumericTypes[T <: Evaluator](evaluator: T, spark: SparkSession): Unit = { val dfs = genEvaluatorDFWithNumericLabelCol(spark, "label", "prediction") val expected = evaluator.evaluate(dfs(DoubleType)) @@ -116,6 +141,26 @@ object MLTestingUtils extends SparkFunSuite { }.toMap } + def genRatingsDFWithNumericCols( + spark: SparkSession, + column: String): Map[NumericType, DataFrame] = { + val df = spark.createDataFrame(Seq( + (0, 10, 1.0), + (1, 20, 2.0), + (2, 30, 3.0), + (3, 40, 4.0), + (4, 50, 5.0) + )).toDF("user", "item", "rating") + + val others = df.columns.toSeq.diff(Seq(column)).map(col(_)) + val types: Seq[NumericType] = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types.map { t => + val cols = Seq(col(column).cast(t)) ++ others + t -> df.select(cols: _*) + }.toMap + } + def genEvaluatorDFWithNumericLabelCol( spark: SparkSession, labelColName: String = "label", diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index d7cb65846574b..86c00d91652d1 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -110,10 +110,10 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha typeConverter=TypeConverters.toBoolean) alpha = Param(Params._dummy(), "alpha", "alpha for implicit preference", typeConverter=TypeConverters.toFloat) - userCol = Param(Params._dummy(), "userCol", "column name for user ids", - typeConverter=TypeConverters.toString) - itemCol = Param(Params._dummy(), "itemCol", "column name for item ids", - typeConverter=TypeConverters.toString) + userCol = Param(Params._dummy(), "userCol", "column name for user ids. Ids must be within " + + "the integer value range.", typeConverter=TypeConverters.toString) + itemCol = Param(Params._dummy(), "itemCol", "column name for item ids. Ids must be within " + + "the integer value range.", typeConverter=TypeConverters.toString) ratingCol = Param(Params._dummy(), "ratingCol", "column name for ratings", typeConverter=TypeConverters.toString) nonnegative = Param(Params._dummy(), "nonnegative", From d65707b7f685b6cee4fff4cf3354b6ea279d92bf Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 May 2016 23:19:55 +0100 Subject: [PATCH 0260/1470] [SPARK-15373][WEB UI] Spark UI should show consistent timezones. ## What changes were proposed in this pull request? Currently, SparkUI shows two timezones in a single page when the timezone of browser is different from the server JVM timezone. The following is an example on Databricks CE which uses 'Etc/UTC' timezone. - The time of `submitted` column of list and pop-up description shows `2016/05/18 00:03:07` - The time of `timeline chart` shows `2016/05/17 17:03:07`. ![Different Timezone](https://issues.apache.org/jira/secure/attachment/12804553/12804553_timezone.png) This PR fixes the **timeline chart** to use the same timezone by the followings. - Upgrade `vis` from 3.9.0(2015-01-16) to 4.16.1(2016-04-18) - Override `moment` of `vis` to get `offset` - Update `AllJobsPage`, `JobPage`, and `StagePage`. ## How was this patch tested? Manual. Run the following command and see the Spark UI's event timelines. ``` $ SPARK_SUBMIT_OPTS="-Dscala.usejavacp=true -Duser.timezone=Etc/UTC" bin/spark-submit --class org.apache.spark.repl.Main ... scala> sql("select 1").head ``` Author: Dongjoon Hyun Closes #13158 from dongjoon-hyun/SPARK-15373. (cherry picked from commit cc6a47dd81fb2fcf2d2ec158c1537bcc27ed1674) Signed-off-by: Sean Owen --- .../apache/spark/ui/static/timeline-view.js | 19 ++++++--- .../org/apache/spark/ui/static/vis.min.css | 2 +- .../org/apache/spark/ui/static/vis.min.js | 41 +++++++++++-------- .../scala/org/apache/spark/ui/UIUtils.scala | 5 ++- .../apache/spark/ui/jobs/AllJobsPage.scala | 2 +- .../org/apache/spark/ui/jobs/JobPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 5 ++- 7 files changed, 49 insertions(+), 28 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index f1beca29c2aa5..9ab5684d901f0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -15,7 +15,7 @@ * limitations under the License. */ -function drawApplicationTimeline(groupArray, eventObjArray, startTime) { +function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $("#application-timeline")[0]; @@ -26,7 +26,10 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime) { editable: false, showCurrentTime: false, min: startTime, - zoomable: false + zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var applicationTimeline = new vis.Timeline(container); @@ -87,7 +90,7 @@ $(function (){ } }); -function drawJobTimeline(groupArray, eventObjArray, startTime) { +function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $('#job-timeline')[0]; @@ -99,6 +102,9 @@ function drawJobTimeline(groupArray, eventObjArray, startTime) { showCurrentTime: false, min: startTime, zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var jobTimeline = new vis.Timeline(container); @@ -159,7 +165,7 @@ $(function (){ } }); -function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, maxFinishTime) { +function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, maxFinishTime, offset) { var groups = new vis.DataSet(groupArray); var items = new vis.DataSet(eventObjArray); var container = $("#task-assignment-timeline")[0] @@ -173,7 +179,10 @@ function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, ma showCurrentTime: false, min: minLaunchTime, max: maxFinishTime, - zoomable: false + zoomable: false, + moment: function (date) { + return vis.moment(date).utcOffset(offset); + } }; var taskTimeline = new vis.Timeline(container) diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.css b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css index a390c40d67574..40d182cfde231 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.min.css +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css @@ -1 +1 @@ -.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}.vis [class*=span]{min-height:0;width:auto}.vis.timeline.root{position:relative;border:1px solid #bfbfbf;overflow:hidden;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel{position:absolute;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right,.vis.timeline .vispanel.top{border:1px #bfbfbf}.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.top{border-left-style:solid;border-right-style:solid}.vis.timeline .background{overflow:hidden}.vis.timeline .vispanel>.content{position:relative}.vis.timeline .vispanel .shadow{position:absolute;width:100%;height:1px;box-shadow:0 0 10px rgba(0,0,0,.8)}.vis.timeline .vispanel .shadow.top{top:-1px;left:0}.vis.timeline .vispanel .shadow.bottom{bottom:-1px;left:0}.vis.timeline .labelset{position:relative;overflow:hidden;box-sizing:border-box}.vis.timeline .labelset .vlabel{position:relative;left:0;top:0;width:100%;color:#4d4d4d;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .labelset .vlabel:last-child{border-bottom:none}.vis.timeline .labelset .vlabel .inner{display:inline-block;padding:5px}.vis.timeline .labelset .vlabel .inner.hidden{padding:0}.vis.timeline .itemset{position:relative;padding:0;margin:0;box-sizing:border-box}.vis.timeline .itemset .background,.vis.timeline .itemset .foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis.timeline .axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis.timeline .foreground .group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .foreground .group:last-child{border-bottom:none}.vis.timeline .item{position:absolute;color:#1A1A1A;border-color:#97B0F8;border-width:1px;background-color:#D5DDF6;display:inline-block;padding:5px}.vis.timeline .item.selected{border-color:#FFC200;background-color:#FFF785;z-index:2}.vis.timeline .editable .item.selected{cursor:move}.vis.timeline .item.point.selected{background-color:#FFF785}.vis.timeline .item.box{text-align:center;border-style:solid;border-radius:2px}.vis.timeline .item.point{background:0 0}.vis.timeline .item.dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis.timeline .item.range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis.timeline .item.background{overflow:hidden;border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis.timeline .item.range .content{position:relative;display:inline-block;max-width:100%;overflow:hidden}.vis.timeline .item.background .content{position:absolute;display:inline-block;overflow:hidden;max-width:100%;margin:5px}.vis.timeline .item.line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis.timeline .item .content{white-space:nowrap;overflow:hidden}.vis.timeline .item .delete{background:url(img/timeline/delete.png) top center no-repeat;position:absolute;width:24px;height:24px;top:0;right:-24px;cursor:pointer}.vis.timeline .item.range .drag-left{position:absolute;width:24px;height:100%;top:0;left:-4px;cursor:w-resize}.vis.timeline .item.range .drag-right{position:absolute;width:24px;height:100%;top:0;right:-4px;cursor:e-resize}.vis.timeline .timeaxis{position:relative;overflow:hidden}.vis.timeline .timeaxis.foreground{top:0;left:0;width:100%}.vis.timeline .timeaxis.background{position:absolute;top:0;left:0;width:100%;height:100%}.vis.timeline .timeaxis .text{position:absolute;color:#4d4d4d;padding:3px;white-space:nowrap}.vis.timeline .timeaxis .text.measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right:0;visibility:hidden}.vis.timeline .timeaxis .grid.vertical{position:absolute;border-left:1px solid}.vis.timeline .timeaxis .grid.minor{border-color:#e5e5e5}.vis.timeline .timeaxis .grid.major{border-color:#bfbfbf}.vis.timeline .currenttime{background-color:#FF7F6E;width:2px;z-index:1}.vis.timeline .customtime{background-color:#6E94FF;width:2px;cursor:move;z-index:1}.vis.timeline .vispanel.background.horizontal .grid.horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis.timeline .vispanel.background.horizontal .grid.minor{border-color:#e5e5e5}.vis.timeline .vispanel.background.horizontal .grid.major{border-color:#bfbfbf}.vis.timeline .dataaxis .yAxis.major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis.timeline .dataaxis .yAxis.major.measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis.timeline .dataaxis .yAxis.minor.measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis.timeline .dataaxis .yAxis.title.measure{padding:0;margin:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title.left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis.timeline .dataaxis .yAxis.title.right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis.timeline .legend{background-color:rgba(247,252,255,.65);padding:5px;border-color:#b3b3b3;border-style:solid;border-width:1px;box-shadow:2px 2px 10px rgba(154,154,154,.55)}.vis.timeline .legendText{white-space:nowrap;display:inline-block}.vis.timeline .graphGroup0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis.timeline .graphGroup1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis.timeline .graphGroup2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis.timeline .graphGroup3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis.timeline .graphGroup4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis.timeline .graphGroup5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis.timeline .graphGroup6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis.timeline .graphGroup7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis.timeline .graphGroup8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#f0f}.vis.timeline .graphGroup9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis.timeline .fill{fill-opacity:.1;stroke:none}.vis.timeline .bar{fill-opacity:.5;stroke-width:1px}.vis.timeline .point{stroke-width:2px;fill-opacity:1}.vis.timeline .legendBackground{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis.timeline .outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis.timeline .iconFill{fill-opacity:.3;stroke:none}div.network-manipulationDiv{border-width:0;border-bottom:1px;border-style:solid;border-color:#d6d9d8;background:#fff;background:-moz-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#fff),color-stop(48%,#fcfcfc),color-stop(50%,#fafafa),color-stop(100%,#fcfcfc));background:-webkit-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-o-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-ms-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:linear-gradient(to bottom,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffff', endColorstr='#fcfcfc', GradientType=0);position:absolute;left:0;top:0;width:100%;height:30px}div.network-manipulation-editMode{position:absolute;left:0;top:0;height:30px;margin-top:20px}div.network-manipulation-closeDiv{position:absolute;right:0;top:0;width:30px;height:30px;background-position:20px 3px;background-repeat:no-repeat;background-image:url(img/network/cross.png);cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-manipulation-closeDiv:hover{opacity:.6}span.network-manipulationUI{font-family:verdana;font-size:12px;-moz-border-radius:15px;border-radius:15px;display:inline-block;background-position:0 0;background-repeat:no-repeat;height:24px;margin:-14px 0 0 10px;vertical-align:middle;cursor:pointer;padding:0 8px;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}span.network-manipulationUI:hover{box-shadow:1px 1px 8px rgba(0,0,0,.2)}span.network-manipulationUI:active{box-shadow:1px 1px 8px rgba(0,0,0,.5)}span.network-manipulationUI.back{background-image:url(img/network/backIcon.png)}span.network-manipulationUI.none:hover{box-shadow:1px 1px 8px transparent;cursor:default}span.network-manipulationUI.none:active{box-shadow:1px 1px 8px transparent}span.network-manipulationUI.none{padding:0}span.network-manipulationUI.notification{margin:2px;font-weight:700}span.network-manipulationUI.add{background-image:url(img/network/addNodeIcon.png)}span.network-manipulationUI.edit{background-image:url(img/network/editIcon.png)}span.network-manipulationUI.edit.editmode{background-color:#fcfcfc;border-style:solid;border-width:1px;border-color:#ccc}span.network-manipulationUI.connect{background-image:url(img/network/connectIcon.png)}span.network-manipulationUI.delete{background-image:url(img/network/deleteIcon.png)}span.network-manipulationLabel{margin:0 0 0 23px;line-height:25px}div.network-seperatorLine{display:inline-block;width:1px;height:20px;background-color:#bdbdbd;margin:5px 7px 0 15px}div.network-navigation_wrapper{position:absolute;left:0;top:0;width:100%;height:100%}div.network-navigation{width:34px;height:34px;-moz-border-radius:17px;border-radius:17px;position:absolute;display:inline-block;background-position:2px 2px;background-repeat:no-repeat;cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-navigation:hover{box-shadow:0 0 3px 3px rgba(56,207,21,.3)}div.network-navigation:active{box-shadow:0 0 1px 3px rgba(56,207,21,.95)}div.network-navigation.up{background-image:url(img/network/upArrow.png);bottom:50px;left:55px}div.network-navigation.down{background-image:url(img/network/downArrow.png);bottom:10px;left:55px}div.network-navigation.left{background-image:url(img/network/leftArrow.png);bottom:10px;left:15px}div.network-navigation.right{background-image:url(img/network/rightArrow.png);bottom:10px;left:95px}div.network-navigation.zoomIn{background-image:url(img/network/plus.png);bottom:10px;right:15px}div.network-navigation.zoomOut{background-image:url(img/network/minus.png);bottom:10px;right:55px}div.network-navigation.zoomExtends{background-image:url(img/network/zoomExtends.png);bottom:50px;right:15px} \ No newline at end of file +.vis-background,.vis-labelset,.vis-timeline{overflow:hidden}.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}.vis [class*=span]{min-height:0;width:auto}div.vis-configuration{position:relative;display:block;float:left;font-size:12px}div.vis-configuration-wrapper{display:block;width:700px}div.vis-configuration-wrapper::after{clear:both;content:"";display:block}div.vis-configuration.vis-config-option-container{display:block;width:495px;background-color:#fff;border:2px solid #f7f8fa;border-radius:4px;margin-top:20px;left:10px;padding-left:5px}div.vis-configuration.vis-config-button{display:block;width:495px;height:25px;vertical-align:middle;line-height:25px;background-color:#f7f8fa;border:2px solid #ceced0;border-radius:4px;margin-top:20px;left:10px;padding-left:5px;cursor:pointer;margin-bottom:30px}div.vis-configuration.vis-config-button.hover{background-color:#4588e6;border:2px solid #214373;color:#fff}div.vis-configuration.vis-config-item{display:block;float:left;width:495px;height:25px;vertical-align:middle;line-height:25px}div.vis-configuration.vis-config-item.vis-config-s2{left:10px;background-color:#f7f8fa;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s3{left:20px;background-color:#e4e9f0;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-item.vis-config-s4{left:30px;background-color:#cfd8e6;padding-left:5px;border-radius:3px}div.vis-configuration.vis-config-header{font-size:18px;font-weight:700}div.vis-configuration.vis-config-label{width:120px;height:25px;line-height:25px}div.vis-configuration.vis-config-label.vis-config-s3{width:110px}div.vis-configuration.vis-config-label.vis-config-s4{width:100px}div.vis-configuration.vis-config-colorBlock{top:1px;width:30px;height:19px;border:1px solid #444;border-radius:2px;padding:0;margin:0;cursor:pointer}input.vis-configuration.vis-config-checkbox{left:-5px}input.vis-configuration.vis-config-rangeinput{position:relative;top:-5px;width:60px;padding:1px;margin:0;pointer-events:none}.vis-panel,.vis-timeline{padding:0;box-sizing:border-box}input.vis-configuration.vis-config-range{-webkit-appearance:none;border:0 solid #fff;background-color:rgba(0,0,0,0);width:300px;height:20px}input.vis-configuration.vis-config-range::-webkit-slider-runnable-track{width:300px;height:5px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(to bottom,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8', GradientType=0 );border:1px solid #999;box-shadow:#aaa 0 0 3px 0;border-radius:3px}input.vis-configuration.vis-config-range::-webkit-slider-thumb{-webkit-appearance:none;border:1px solid #14334b;height:17px;width:17px;border-radius:50%;background:#3876c2;background:-moz-linear-gradient(top,#3876c2 0,#385380 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#3876c2),color-stop(100%,#385380));background:-webkit-linear-gradient(top,#3876c2 0,#385380 100%);background:-o-linear-gradient(top,#3876c2 0,#385380 100%);background:-ms-linear-gradient(top,#3876c2 0,#385380 100%);background:linear-gradient(to bottom,#3876c2 0,#385380 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#3876c2', endColorstr='#385380', GradientType=0 );box-shadow:#111927 0 0 1px 0;margin-top:-7px}input.vis-configuration.vis-config-range:focus{outline:0}input.vis-configuration.vis-config-range:focus::-webkit-slider-runnable-track{background:#9d9d9d;background:-moz-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#9d9d9d),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-o-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#9d9d9d 0,#c8c8c8 99%);background:linear-gradient(to bottom,#9d9d9d 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#9d9d9d', endColorstr='#c8c8c8', GradientType=0 )}input.vis-configuration.vis-config-range::-moz-range-track{width:300px;height:10px;background:#dedede;background:-moz-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#dedede),color-stop(99%,#c8c8c8));background:-webkit-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-o-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:-ms-linear-gradient(top,#dedede 0,#c8c8c8 99%);background:linear-gradient(to bottom,#dedede 0,#c8c8c8 99%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#dedede', endColorstr='#c8c8c8', GradientType=0 );border:1px solid #999;box-shadow:#aaa 0 0 3px 0;border-radius:3px}input.vis-configuration.vis-config-range::-moz-range-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:-moz-focusring{outline:#fff solid 1px;outline-offset:-1px}input.vis-configuration.vis-config-range::-ms-track{width:300px;height:5px;background:0 0;border-color:transparent;border-width:6px 0;color:transparent}input.vis-configuration.vis-config-range::-ms-fill-lower{background:#777;border-radius:10px}input.vis-configuration.vis-config-range::-ms-fill-upper{background:#ddd;border-radius:10px}input.vis-configuration.vis-config-range::-ms-thumb{border:none;height:16px;width:16px;border-radius:50%;background:#385380}input.vis-configuration.vis-config-range:focus::-ms-fill-lower{background:#888}input.vis-configuration.vis-config-range:focus::-ms-fill-upper{background:#ccc}.vis-configuration-popup{position:absolute;background:rgba(57,76,89,.85);border:2px solid #f2faff;line-height:30px;height:30px;width:150px;text-align:center;color:#fff;font-size:14px;border-radius:4px;-webkit-transition:opacity .3s ease-in-out;-moz-transition:opacity .3s ease-in-out;transition:opacity .3s ease-in-out}.vis-configuration-popup:after,.vis-configuration-popup:before{left:100%;top:50%;border:solid transparent;content:" ";height:0;width:0;position:absolute;pointer-events:none}.vis-configuration-popup:after{border-color:rgba(136,183,213,0);border-left-color:rgba(57,76,89,.85);border-width:8px;margin-top:-8px}.vis-configuration-popup:before{border-color:rgba(194,225,245,0);border-left-color:#f2faff;border-width:12px;margin-top:-12px}.vis-timeline{position:relative;border:1px solid #bfbfbf;margin:0}.vis-panel{position:absolute;margin:0}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right,.vis-panel.vis-top{border:1px #bfbfbf}.vis-panel.vis-center,.vis-panel.vis-left,.vis-panel.vis-right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis-panel.vis-bottom,.vis-panel.vis-center,.vis-panel.vis-top{border-left-style:solid;border-right-style:solid}.vis-panel>.vis-content{position:relative}.vis-panel .vis-shadow{position:absolute;width:100%;height:1px;box-shadow:0 0 10px rgba(0,0,0,.8)}.vis-itemset,.vis-labelset,.vis-labelset .vis-label{position:relative;box-sizing:border-box}.vis-panel .vis-shadow.vis-top{top:-1px;left:0}.vis-panel .vis-shadow.vis-bottom{bottom:-1px;left:0}.vis-labelset .vis-label{left:0;top:0;width:100%;color:#4d4d4d;border-bottom:1px solid #bfbfbf}.vis-labelset .vis-label.draggable{cursor:pointer}.vis-labelset .vis-label:last-child{border-bottom:none}.vis-labelset .vis-label .vis-inner{display:inline-block;padding:5px}.vis-labelset .vis-label .vis-inner.vis-hidden{padding:0}.vis-itemset{padding:0;margin:0}.vis-itemset .vis-background,.vis-itemset .vis-foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis-axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis-foreground .vis-group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis-foreground .vis-group:last-child{border-bottom:none}.vis-overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-item{position:absolute;color:#1A1A1A;border-color:#97B0F8;border-width:1px;background-color:#D5DDF6;display:inline-block}.vis-item.vis-point.vis-selected,.vis-item.vis-selected{background-color:#FFF785}.vis-item.vis-selected{border-color:#FFC200;z-index:2}.vis-editable.vis-selected{cursor:move}.vis-item.vis-box{text-align:center;border-style:solid;border-radius:2px}.vis-item.vis-point{background:0 0}.vis-item.vis-dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis-item.vis-range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis-item.vis-background{border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis-item .vis-item-overflow{position:relative;width:100%;height:100%;padding:0;margin:0;overflow:hidden}.vis-item .vis-delete,.vis-item .vis-delete-rtl{background:url(img/timeline/delete.png) center no-repeat;height:24px;top:-4px;cursor:pointer}.vis-item.vis-range .vis-item-content{position:relative;display:inline-block}.vis-item.vis-background .vis-item-content{position:absolute;display:inline-block}.vis-item.vis-line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis-item .vis-item-content{white-space:nowrap;box-sizing:border-box;padding:5px}.vis-item .vis-delete{position:absolute;width:24px;right:-24px}.vis-item .vis-delete-rtl{position:absolute;width:24px;left:-24px}.vis-item.vis-range .vis-drag-left{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0;left:-4px;cursor:w-resize}.vis-item.vis-range .vis-drag-right{position:absolute;width:24px;max-width:20%;min-width:2px;height:100%;top:0;right:-4px;cursor:e-resize}.vis-range.vis-item.vis-readonly .vis-drag-left,.vis-range.vis-item.vis-readonly .vis-drag-right{cursor:auto}.vis-time-axis{position:relative;overflow:hidden}.vis-time-axis.vis-foreground{top:0;left:0;width:100%}.vis-time-axis.vis-background{position:absolute;top:0;left:0;width:100%;height:100%}.vis-time-axis .vis-text{position:absolute;color:#4d4d4d;padding:3px;overflow:hidden;box-sizing:border-box;white-space:nowrap}.vis-time-axis .vis-text.vis-measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right:0;visibility:hidden}.vis-time-axis .vis-grid.vis-vertical{position:absolute;border-left:1px solid}.vis-time-axis .vis-grid.vis-vertical-rtl{position:absolute;border-right:1px solid}.vis-time-axis .vis-grid.vis-minor{border-color:#e5e5e5}.vis-time-axis .vis-grid.vis-major{border-color:#bfbfbf}.vis-current-time{background-color:#FF7F6E;width:2px;z-index:1}.vis-custom-time{background-color:#6E94FF;width:2px;cursor:move;z-index:1}div.vis-network div.vis-close,div.vis-network div.vis-edit-mode div.vis-button,div.vis-network div.vis-manipulation div.vis-button{cursor:pointer;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;-webkit-touch-callout:none;-khtml-user-select:none}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-minor{border-color:#e5e5e5}.vis-panel.vis-background.vis-horizontal .vis-grid.vis-major{border-color:#bfbfbf}.vis-data-axis .vis-y-axis.vis-major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-major.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis-data-axis .vis-y-axis.vis-minor.vis-measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis-data-axis .vis-y-axis.vis-title.vis-measure{padding:0;margin:0;visibility:hidden;width:auto}.vis-data-axis .vis-y-axis.vis-title.vis-left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis-data-axis .vis-y-axis.vis-title.vis-right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis-legend{background-color:rgba(247,252,255,.65);padding:5px;border:1px solid #b3b3b3;box-shadow:2px 2px 10px rgba(154,154,154,.55)}.vis-legend-text{white-space:nowrap;display:inline-block}.vis-graph-group0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis-graph-group1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis-graph-group2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis-graph-group3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis-graph-group4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis-graph-group5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis-graph-group6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis-graph-group7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis-graph-group8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#f0f}.vis-graph-group9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis-timeline .vis-fill{fill-opacity:.1;stroke:none}.vis-timeline .vis-bar{fill-opacity:.5;stroke-width:1px}.vis-timeline .vis-point{stroke-width:2px;fill-opacity:1}.vis-timeline .vis-legend-background{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis-timeline .vis-outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis-timeline .vis-icon-fill{fill-opacity:.3;stroke:none}div.vis-network div.vis-manipulation{border-width:0;border-bottom:1px;border-style:solid;border-color:#d6d9d8;background:#fff;background:-moz-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#fff),color-stop(48%,#fcfcfc),color-stop(50%,#fafafa),color-stop(100%,#fcfcfc));background:-webkit-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-o-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-ms-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:linear-gradient(to bottom,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#ffffff', endColorstr='#fcfcfc', GradientType=0 );padding-top:4px;position:absolute;left:0;top:0;width:100%;height:28px}div.vis-network div.vis-edit-mode{position:absolute;left:0;top:5px;height:30px}div.vis-network div.vis-close{position:absolute;right:0;top:0;width:30px;height:30px;background-position:20px 3px;background-repeat:no-repeat;background-image:url(img/network/cross.png);user-select:none}div.vis-network div.vis-close:hover{opacity:.6}div.vis-network div.vis-edit-mode div.vis-button,div.vis-network div.vis-manipulation div.vis-button{float:left;font-family:verdana;font-size:12px;-moz-border-radius:15px;border-radius:15px;display:inline-block;background-position:0 0;background-repeat:no-repeat;height:24px;margin-left:10px;padding:0 8px;user-select:none}div.vis-network div.vis-manipulation div.vis-button:hover{box-shadow:1px 1px 8px rgba(0,0,0,.2)}div.vis-network div.vis-manipulation div.vis-button:active{box-shadow:1px 1px 8px rgba(0,0,0,.5)}div.vis-network div.vis-manipulation div.vis-button.vis-back{background-image:url(img/network/backIcon.png)}div.vis-network div.vis-manipulation div.vis-button.vis-none:hover{box-shadow:1px 1px 8px transparent;cursor:default}div.vis-network div.vis-manipulation div.vis-button.vis-none:active{box-shadow:1px 1px 8px transparent}div.vis-network div.vis-manipulation div.vis-button.vis-none{padding:0}div.vis-network div.vis-manipulation div.notification{margin:2px;font-weight:700}div.vis-network div.vis-manipulation div.vis-button.vis-add{background-image:url(img/network/addNodeIcon.png)}div.vis-network div.vis-edit-mode div.vis-button.vis-edit,div.vis-network div.vis-manipulation div.vis-button.vis-edit{background-image:url(img/network/editIcon.png)}div.vis-network div.vis-edit-mode div.vis-button.vis-edit.vis-edit-mode{background-color:#fcfcfc;border:1px solid #ccc}div.vis-network div.vis-manipulation div.vis-button.vis-connect{background-image:url(img/network/connectIcon.png)}div.vis-network div.vis-manipulation div.vis-button.vis-delete{background-image:url(img/network/deleteIcon.png)}div.vis-network div.vis-edit-mode div.vis-label,div.vis-network div.vis-manipulation div.vis-label{margin:0 0 0 23px;line-height:25px}div.vis-network div.vis-manipulation div.vis-separator-line{float:left;display:inline-block;width:1px;height:21px;background-color:#bdbdbd;margin:0 7px 0 15px}div.vis-network-tooltip{position:absolute;visibility:hidden;padding:5px;white-space:nowrap;font-family:verdana;font-size:14px;color:#000;background-color:#f5f4ed;-moz-border-radius:3px;-webkit-border-radius:3px;border-radius:3px;border:1px solid #808074;box-shadow:3px 3px 10px rgba(0,0,0,.2);pointer-events:none}div.vis-network div.vis-navigation div.vis-button{width:34px;height:34px;-moz-border-radius:17px;border-radius:17px;position:absolute;display:inline-block;background-position:2px 2px;background-repeat:no-repeat;cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.vis-network div.vis-navigation div.vis-button:hover{box-shadow:0 0 3px 3px rgba(56,207,21,.3)}div.vis-network div.vis-navigation div.vis-button:active{box-shadow:0 0 1px 3px rgba(56,207,21,.95)}div.vis-network div.vis-navigation div.vis-button.vis-up{background-image:url(img/network/upArrow.png);bottom:50px;left:55px}div.vis-network div.vis-navigation div.vis-button.vis-down{background-image:url(img/network/downArrow.png);bottom:10px;left:55px}div.vis-network div.vis-navigation div.vis-button.vis-left{background-image:url(img/network/leftArrow.png);bottom:10px;left:15px}div.vis-network div.vis-navigation div.vis-button.vis-right{background-image:url(img/network/rightArrow.png);bottom:10px;left:95px}div.vis-network div.vis-navigation div.vis-button.vis-zoomIn{background-image:url(img/network/plus.png);bottom:10px;right:15px}div.vis-network div.vis-navigation div.vis-button.vis-zoomOut{background-image:url(img/network/minus.png);bottom:10px;right:55px}div.vis-network div.vis-navigation div.vis-button.vis-zoomExtends{background-image:url(img/network/zoomExtends.png);bottom:50px;right:15px}div.vis-color-picker{position:absolute;top:0;left:30px;margin-top:-140px;margin-left:30px;width:310px;height:444px;z-index:1;padding:10px;border-radius:15px;background-color:#fff;display:none;box-shadow:rgba(0,0,0,.5) 0 0 10px 0}div.vis-color-picker div.vis-arrow{position:absolute;top:147px;left:5px}div.vis-color-picker div.vis-arrow::after,div.vis-color-picker div.vis-arrow::before{right:100%;top:50%;border:solid transparent;content:" ";height:0;width:0;position:absolute;pointer-events:none}div.vis-color-picker div.vis-arrow:after{border-color:rgba(255,255,255,0);border-right-color:#fff;border-width:30px;margin-top:-30px}div.vis-color-picker div.vis-color{position:absolute;width:289px;height:289px;cursor:pointer}div.vis-color-picker div.vis-brightness{position:absolute;top:313px}div.vis-color-picker div.vis-opacity{position:absolute;top:350px}div.vis-color-picker div.vis-selector{position:absolute;top:137px;left:137px;width:15px;height:15px;border-radius:15px;border:1px solid #fff;background:#4c4c4c;background:-moz-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#4c4c4c),color-stop(12%,#595959),color-stop(25%,#666),color-stop(39%,#474747),color-stop(50%,#2c2c2c),color-stop(51%,#000),color-stop(60%,#111),color-stop(76%,#2b2b2b),color-stop(91%,#1c1c1c),color-stop(100%,#131313));background:-webkit-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-o-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:-ms-linear-gradient(top,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);background:linear-gradient(to bottom,#4c4c4c 0,#595959 12%,#666 25%,#474747 39%,#2c2c2c 50%,#000 51%,#111 60%,#2b2b2b 76%,#1c1c1c 91%,#131313 100%);filter:progid:DXImageTransform.Microsoft.gradient( startColorstr='#4c4c4c', endColorstr='#131313', GradientType=0 )}div.vis-color-picker div.vis-initial-color,div.vis-color-picker div.vis-new-color{width:140px;height:20px;top:380px;font-size:10px;color:rgba(0,0,0,.4);line-height:20px;position:absolute;vertical-align:middle}div.vis-color-picker div.vis-new-color{border:1px solid rgba(0,0,0,.1);border-radius:5px;left:159px;text-align:right;padding-right:2px}div.vis-color-picker div.vis-initial-color{border:1px solid rgba(0,0,0,.1);border-radius:5px;left:10px;text-align:left;padding-left:2px}div.vis-color-picker div.vis-label{position:absolute;width:300px;left:10px}div.vis-color-picker div.vis-label.vis-brightness{top:300px}div.vis-color-picker div.vis-label.vis-opacity{top:338px}div.vis-color-picker div.vis-button{position:absolute;width:68px;height:25px;border-radius:10px;vertical-align:middle;text-align:center;line-height:25px;top:410px;border:2px solid #d9d9d9;background-color:#f7f7f7;cursor:pointer}div.vis-color-picker div.vis-button.vis-cancel{left:5px}div.vis-color-picker div.vis-button.vis-load{left:82px}div.vis-color-picker div.vis-button.vis-apply{left:159px}div.vis-color-picker div.vis-button.vis-save{left:236px}div.vis-color-picker input.vis-range{width:290px;height:20px} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js index 2b3b1d60463f7..92b8ed75d85fc 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js @@ -4,11 +4,11 @@ * * A dynamic, browser-based visualization library. * - * @version 3.9.0 - * @date 2015-01-16 + * @version 4.16.1 + * @date 2016-04-18 * * @license - * Copyright (C) 2011-2014 Almende B.V, http://almende.com + * Copyright (C) 2011-2016 Almende B.V, http://almende.com * * Vis.js is dual licensed under both * @@ -22,17 +22,24 @@ * * Vis.js may be distributed under either license. */ -"use strict";!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define(e):"object"==typeof exports?exports.vis=e():t.vis=e()}(this,function(){return function(t){function e(s){if(i[s])return i[s].exports;var o=i[s]={exports:{},id:s,loaded:!1};return t[s].call(o.exports,o,o.exports,e),o.loaded=!0,o.exports}var i={};return e.m=t,e.c=i,e.p="",e(0)}([function(t,e,i){e.util=i(1),e.DOMutil=i(2),e.DataSet=i(3),e.DataView=i(4),e.Queue=i(5),e.Graph3d=i(6),e.graph3d={Camera:i(7),Filter:i(8),Point2d:i(9),Point3d:i(10),Slider:i(11),StepNumber:i(12)},e.Timeline=i(13),e.Graph2d=i(14),e.timeline={DateUtil:i(15),DataStep:i(16),Range:i(17),stack:i(18),TimeStep:i(19),components:{items:{Item:i(31),BackgroundItem:i(32),BoxItem:i(33),PointItem:i(34),RangeItem:i(35)},Component:i(20),CurrentTime:i(21),CustomTime:i(22),DataAxis:i(23),GraphGroup:i(24),Group:i(25),BackgroundGroup:i(26),ItemSet:i(27),Legend:i(28),LineGraph:i(29),TimeAxis:i(30)}},e.Network=i(36),e.network={Edge:i(37),Groups:i(38),Images:i(39),Node:i(40),Popup:i(41),dotparser:i(42),gephiParser:i(43)},e.Graph=function(){throw new Error("Graph is renamed to Network. Please create a graph as new vis.Network(...)")},e.moment=i(44),e.hammer=i(45),e.Hammer=i(45)},function(t,e,i){var s=i(44);e.isNumber=function(t){return t instanceof Number||"number"==typeof t},e.isString=function(t){return t instanceof String||"string"==typeof t},e.isDate=function(t){if(t instanceof Date)return!0;if(e.isString(t)){var i=o.exec(t);if(i)return!0;if(!isNaN(Date.parse(t)))return!0}return!1},e.isDataTable=function(t){return"undefined"!=typeof google&&google.visualization&&google.visualization.DataTable&&t instanceof google.visualization.DataTable},e.randomUUID=function(){var t=function(){return Math.floor(65536*Math.random()).toString(16)};return t()+t()+"-"+t()+"-"+t()+"-"+t()+"-"+t()+t()+t()},e.extend=function(t){for(var e=1,i=arguments.length;i>e;e++){var s=arguments[e];for(var o in s)s.hasOwnProperty(o)&&(t[o]=s[o])}return t},e.selectiveExtend=function(t,e){if(!Array.isArray(t))throw new Error("Array with property names expected as first argument");for(var i=2;ii;i++)if(t[i]!=e[i])return!1;return!0},e.convert=function(t,i){var n;if(void 0===t)return void 0;if(null===t)return null;if(!i)return t;if("string"!=typeof i&&!(i instanceof String))throw new Error("Type must be a string");switch(i){case"boolean":case"Boolean":return Boolean(t);case"number":case"Number":return Number(t.valueOf());case"string":case"String":return String(t);case"Date":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return new Date(t.valueOf());if(s.isMoment(t))return new Date(t.valueOf());if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])):s(t).toDate();throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"Moment":if(e.isNumber(t))return s(t);if(t instanceof Date)return s(t.valueOf());if(s.isMoment(t))return s(t);if(e.isString(t))return n=o.exec(t),s(n?Number(n[1]):t);throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"ISODate":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return t.toISOString();if(s.isMoment(t))return t.toDate().toISOString();if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])).toISOString():new Date(t).toISOString();throw new Error("Cannot convert object of type "+e.getType(t)+" to type ISODate");case"ASPDate":if(e.isNumber(t))return"/Date("+t+")/";if(t instanceof Date)return"/Date("+t.valueOf()+")/";if(e.isString(t)){n=o.exec(t);var r;return r=n?new Date(Number(n[1])).valueOf():new Date(t).valueOf(),"/Date("+r+")/"}throw new Error("Cannot convert object of type "+e.getType(t)+" to type ASPDate");default:throw new Error('Unknown type "'+i+'"')}};var o=/^\/?Date\((\-?\d+)/i;e.getType=function(t){var e=typeof t;return"object"==e?null==t?"null":t instanceof Boolean?"Boolean":t instanceof Number?"Number":t instanceof String?"String":Array.isArray(t)?"Array":t instanceof Date?"Date":"Object":"number"==e?"Number":"boolean"==e?"Boolean":"string"==e?"String":e},e.getAbsoluteLeft=function(t){return t.getBoundingClientRect().left},e.getAbsoluteTop=function(t){return t.getBoundingClientRect().top},e.addClassName=function(t,e){var i=t.className.split(" ");-1==i.indexOf(e)&&(i.push(e),t.className=i.join(" "))},e.removeClassName=function(t,e){var i=t.className.split(" "),s=i.indexOf(e);-1!=s&&(i.splice(s,1),t.className=i.join(" "))},e.forEach=function(t,e){var i,s;if(Array.isArray(t))for(i=0,s=t.length;s>i;i++)e(t[i],i,t);else for(i in t)t.hasOwnProperty(i)&&e(t[i],i,t)},e.toArray=function(t){var e=[];for(var i in t)t.hasOwnProperty(i)&&e.push(t[i]);return e},e.updateProperty=function(t,e,i){return t[e]!==i?(t[e]=i,!0):!1},e.addEventListener=function(t,e,i,s){t.addEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.addEventListener(e,i,s)):t.attachEvent("on"+e,i)},e.removeEventListener=function(t,e,i,s){t.removeEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.removeEventListener(e,i,s)):t.detachEvent("on"+e,i)},e.preventDefault=function(t){t||(t=window.event),t.preventDefault?t.preventDefault():t.returnValue=!1},e.getTarget=function(t){t||(t=window.event);var e;return t.target?e=t.target:t.srcElement&&(e=t.srcElement),void 0!=e.nodeType&&3==e.nodeType&&(e=e.parentNode),e},e.option={},e.option.asBoolean=function(t,e){return"function"==typeof t&&(t=t()),null!=t?0!=t:e||null},e.option.asNumber=function(t,e){return"function"==typeof t&&(t=t()),null!=t?Number(t)||e||null:e||null},e.option.asString=function(t,e){return"function"==typeof t&&(t=t()),null!=t?String(t):e||null},e.option.asSize=function(t,i){return"function"==typeof t&&(t=t()),e.isString(t)?t:e.isNumber(t)?t+"px":i||null},e.option.asElement=function(t,e){return"function"==typeof t&&(t=t()),t||e||null},e.hexToRGB=function(t){var e=/^#?([a-f\d])([a-f\d])([a-f\d])$/i;t=t.replace(e,function(t,e,i,s){return e+e+i+i+s+s});var i=/^#?([a-f\d]{2})([a-f\d]{2})([a-f\d]{2})$/i.exec(t);return i?{r:parseInt(i[1],16),g:parseInt(i[2],16),b:parseInt(i[3],16)}:null},e.RGBToHex=function(t,e,i){return"#"+((1<<24)+(t<<16)+(e<<8)+i).toString(16).slice(1)},e.parseColor=function(t){var i;if(e.isString(t)){if(e.isValidRGB(t)){var s=t.substr(4).substr(0,t.length-5).split(",");t=e.RGBToHex(s[0],s[1],s[2])}if(e.isValidHex(t)){var o=e.hexToHSV(t),n={h:o.h,s:.45*o.s,v:Math.min(1,1.05*o.v)},r={h:o.h,s:Math.min(1,1.25*o.v),v:.6*o.v},a=e.HSVToHex(r.h,r.h,r.v),h=e.HSVToHex(n.h,n.s,n.v);i={background:t,border:a,highlight:{background:h,border:a},hover:{background:h,border:a}}}else i={background:t,border:t,highlight:{background:t,border:t},hover:{background:t,border:t}}}else i={},i.background=t.background||"white",i.border=t.border||i.background,e.isString(t.highlight)?i.highlight={border:t.highlight,background:t.highlight}:(i.highlight={},i.highlight.background=t.highlight&&t.highlight.background||i.background,i.highlight.border=t.highlight&&t.highlight.border||i.border),e.isString(t.hover)?i.hover={border:t.hover,background:t.hover}:(i.hover={},i.hover.background=t.hover&&t.hover.background||i.background,i.hover.border=t.hover&&t.hover.border||i.border);return i},e.RGBToHSV=function(t,e,i){t/=255,e/=255,i/=255;var s=Math.min(t,Math.min(e,i)),o=Math.max(t,Math.max(e,i));if(s==o)return{h:0,s:0,v:s};var n=t==s?e-i:i==s?t-e:i-t,r=t==s?3:i==s?1:5,a=60*(r-n/(o-s))/360,h=(o-s)/o,d=o;return{h:a,s:h,v:d}};var n={split:function(t){var e={};return t.split(";").forEach(function(t){if(""!=t.trim()){var i=t.split(":"),s=i[0].trim(),o=i[1].trim();e[s]=o}}),e},join:function(t){return Object.keys(t).map(function(e){return e+": "+t[e]}).join("; ")}};e.addCssText=function(t,i){var s=n.split(t.style.cssText),o=n.split(i),r=e.extend(s,o);t.style.cssText=n.join(r)},e.removeCssText=function(t,e){var i=n.split(t.style.cssText),s=n.split(e);for(var o in s)s.hasOwnProperty(o)&&delete i[o];t.style.cssText=n.join(i)},e.HSVToRGB=function(t,e,i){var s,o,n,r=Math.floor(6*t),a=6*t-r,h=i*(1-e),d=i*(1-a*e),l=i*(1-(1-a)*e);switch(r%6){case 0:s=i,o=l,n=h;break;case 1:s=d,o=i,n=h;break;case 2:s=h,o=i,n=l;break;case 3:s=h,o=d,n=i;break;case 4:s=l,o=h,n=i;break;case 5:s=i,o=h,n=d}return{r:Math.floor(255*s),g:Math.floor(255*o),b:Math.floor(255*n)}},e.HSVToHex=function(t,i,s){var o=e.HSVToRGB(t,i,s);return e.RGBToHex(o.r,o.g,o.b)},e.hexToHSV=function(t){var i=e.hexToRGB(t);return e.RGBToHSV(i.r,i.g,i.b)},e.isValidHex=function(t){var e=/(^#[0-9A-F]{6}$)|(^#[0-9A-F]{3}$)/i.test(t);return e},e.isValidRGB=function(t){t=t.replace(" ","");var e=/rgb\((\d{1,3}),(\d{1,3}),(\d{1,3})\)/i.test(t);return e},e.selectiveBridgeObject=function(t,i){if("object"==typeof i){for(var s=Object.create(i),o=0;o=r&&o>n;){var h=Math.floor((r+a)/2),d=t[h],l=void 0===s?d[i]:d[i][s],c=e(l);if(0==c)return h;-1==c?r=h+1:a=h-1,n++}return-1},e.binarySearchValue=function(t,e,i,s){for(var o,n,r,a,h=1e4,d=0,l=0,c=t.length-1;c>=l&&h>d;){if(a=Math.floor(.5*(c+l)),o=t[Math.max(0,a-1)][i],n=t[a][i],r=t[Math.min(t.length-1,a+1)][i],n==e)return a;if(e>o&&n>e)return"before"==s?Math.max(0,a-1):a;if(e>n&&r>e)return"before"==s?a:Math.min(t.length-1,a+1);e>n?l=a+1:c=a-1,d++}return-1},e.easeInOutQuad=function(t,e,i,s){var o=i-e;return t/=s/2,1>t?o/2*t*t+e:(t--,-o/2*(t*(t-2)-1)+e)},e.easingFunctions={linear:function(t){return t},easeInQuad:function(t){return t*t},easeOutQuad:function(t){return t*(2-t)},easeInOutQuad:function(t){return.5>t?2*t*t:-1+(4-2*t)*t},easeInCubic:function(t){return t*t*t},easeOutCubic:function(t){return--t*t*t+1},easeInOutCubic:function(t){return.5>t?4*t*t*t:(t-1)*(2*t-2)*(2*t-2)+1},easeInQuart:function(t){return t*t*t*t},easeOutQuart:function(t){return 1- --t*t*t*t},easeInOutQuart:function(t){return.5>t?8*t*t*t*t:1-8*--t*t*t*t},easeInQuint:function(t){return t*t*t*t*t},easeOutQuint:function(t){return 1+--t*t*t*t*t},easeInOutQuint:function(t){return.5>t?16*t*t*t*t*t:1+16*--t*t*t*t*t}}},function(t,e){e.prepareElements=function(t){for(var e in t)t.hasOwnProperty(e)&&(t[e].redundant=t[e].used,t[e].used=[])},e.cleanupElements=function(t){for(var e in t)if(t.hasOwnProperty(e)&&t[e].redundant){for(var i=0;i0?(s=e[t].redundant[0],e[t].redundant.shift()):(s=document.createElementNS("http://www.w3.org/2000/svg",t),i.appendChild(s)):(s=document.createElementNS("http://www.w3.org/2000/svg",t),e[t]={used:[],redundant:[]},i.appendChild(s)),e[t].used.push(s),s},e.getDOMElement=function(t,e,i,s){var o;return e.hasOwnProperty(t)?e[t].redundant.length>0?(o=e[t].redundant[0],e[t].redundant.shift()):(o=document.createElement(t),void 0!==s?i.insertBefore(o,s):i.appendChild(o)):(o=document.createElement(t),e[t]={used:[],redundant:[]},void 0!==s?i.insertBefore(o,s):i.appendChild(o)),e[t].used.push(o),o},e.drawPoint=function(t,i,s,o,n){var r;return"circle"==s.options.drawPoints.style?(r=e.getSVGElement("circle",o,n),r.setAttributeNS(null,"cx",t),r.setAttributeNS(null,"cy",i),r.setAttributeNS(null,"r",.5*s.options.drawPoints.size)):(r=e.getSVGElement("rect",o,n),r.setAttributeNS(null,"x",t-.5*s.options.drawPoints.size),r.setAttributeNS(null,"y",i-.5*s.options.drawPoints.size),r.setAttributeNS(null,"width",s.options.drawPoints.size),r.setAttributeNS(null,"height",s.options.drawPoints.size)),void 0!==s.options.drawPoints.styles&&r.setAttributeNS(null,"style",s.group.options.drawPoints.styles),r.setAttributeNS(null,"class",s.className+" point"),r},e.drawBar=function(t,i,s,o,n,r,a){if(0!=o){0>o&&(o*=-1,i-=o);var h=e.getSVGElement("rect",r,a);h.setAttributeNS(null,"x",t-.5*s),h.setAttributeNS(null,"y",i),h.setAttributeNS(null,"width",s),h.setAttributeNS(null,"height",o),h.setAttributeNS(null,"class",n)}}},function(t,e,i){function s(t,e){if(!t||Array.isArray(t)||o.isDataTable(t)||(e=t,t=null),this._options=e||{},this._data={},this._fieldId=this._options.fieldId||"id",this._type={},this._options.type)for(var i in this._options.type)if(this._options.type.hasOwnProperty(i)){var s=this._options.type[i];this._type[i]="Date"==s||"ISODate"==s||"ASPDate"==s?"Date":s}if(this._options.convert)throw new Error('Option "convert" is deprecated. Use "type" instead.');this._subscribers={},t&&this.add(t),this.setOptions(e)}var o=i(1),n=i(5);s.prototype.setOptions=function(t){t&&void 0!==t.queue&&(t.queue===!1?this._queue&&(this._queue.destroy(),delete this._queue):(this._queue||(this._queue=n.extend(this,{replace:["add","update","remove"]})),"object"==typeof t.queue&&this._queue.setOptions(t.queue)))},s.prototype.on=function(t,e){var i=this._subscribers[t];i||(i=[],this._subscribers[t]=i),i.push({callback:e})},s.prototype.subscribe=s.prototype.on,s.prototype.off=function(t,e){var i=this._subscribers[t];i&&(this._subscribers[t]=i.filter(function(t){return t.callback!=e}))},s.prototype.unsubscribe=s.prototype.off,s.prototype._trigger=function(t,e,i){if("*"==t)throw new Error("Cannot trigger event *");var s=[];t in this._subscribers&&(s=s.concat(this._subscribers[t])),"*"in this._subscribers&&(s=s.concat(this._subscribers["*"]));for(var o=0;or;r++)i=n._addItem(t[r]),s.push(i);else if(o.isDataTable(t))for(var h=this._getColumnNames(t),d=0,l=t.getNumberOfRows();l>d;d++){for(var c={},p=0,u=h.length;u>p;p++){var m=h[p];c[m]=t.getValue(d,p)}i=n._addItem(c),s.push(i)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");i=n._addItem(t),s.push(i)}return s.length&&this._trigger("add",{items:s},e),s},s.prototype.update=function(t,e){var i=[],s=[],n=[],r=this,a=r._fieldId,h=function(t){var e=t[a];r._data[e]?(e=r._updateItem(t),s.push(e),n.push(t)):(e=r._addItem(t),i.push(e))};if(Array.isArray(t))for(var d=0,l=t.length;l>d;d++)h(t[d]);else if(o.isDataTable(t))for(var c=this._getColumnNames(t),p=0,u=t.getNumberOfRows();u>p;p++){for(var m={},f=0,g=c.length;g>f;f++){var v=c[f];m[v]=t.getValue(p,f)}h(m)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");h(t)}return i.length&&this._trigger("add",{items:i},e),s.length&&this._trigger("update",{items:s,data:n},e),i.concat(s)},s.prototype.get=function(){var t,e,i,s,n=this,r=o.getType(arguments[0]);"String"==r||"Number"==r?(t=arguments[0],i=arguments[1],s=arguments[2]):"Array"==r?(e=arguments[0],i=arguments[1],s=arguments[2]):(i=arguments[0],s=arguments[1]);var a;if(i&&i.returnType){var h=["DataTable","Array","Object"];if(a=-1==h.indexOf(i.returnType)?"Array":i.returnType,s&&a!=o.getType(s))throw new Error('Type of parameter "data" ('+o.getType(s)+") does not correspond with specified options.type ("+i.type+")");if("DataTable"==a&&!o.isDataTable(s))throw new Error('Parameter "data" must be a DataTable when options.type is "DataTable"')}else a=s&&"DataTable"==o.getType(s)?"DataTable":"Array";var d,l,c,p,u=i&&i.type||this._options.type,m=i&&i.filter,f=[];if(void 0!=t)d=n._getItem(t,u),m&&!m(d)&&(d=null);else if(void 0!=e)for(c=0,p=e.length;p>c;c++)d=n._getItem(e[c],u),(!m||m(d))&&f.push(d);else for(l in this._data)this._data.hasOwnProperty(l)&&(d=n._getItem(l,u),(!m||m(d))&&f.push(d));if(i&&i.order&&void 0==t&&this._sort(f,i.order),i&&i.fields){var g=i.fields;if(void 0!=t)d=this._filterFields(d,g);else for(c=0,p=f.length;p>c;c++)f[c]=this._filterFields(f[c],g)}if("DataTable"==a){var v=this._getColumnNames(s);if(void 0!=t)n._appendRow(s,v,d);else for(c=0;cc;c++)s.push(f[c]);return s}return f},s.prototype.getIds=function(t){var e,i,s,o,n,r=this._data,a=t&&t.filter,h=t&&t.order,d=t&&t.type||this._options.type,l=[];if(a)if(h){n=[];for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&n.push(o));for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&l.push(o[this._fieldId]));else if(h){n=[];for(s in r)r.hasOwnProperty(s)&&n.push(r[s]);for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=r[s],l.push(o[this._fieldId]));return l},s.prototype.getDataSet=function(){return this},s.prototype.forEach=function(t,e){var i,s,o=e&&e.filter,n=e&&e.type||this._options.type,r=this._data;if(e&&e.order)for(var a=this.get(e),h=0,d=a.length;d>h;h++)i=a[h],s=i[this._fieldId],t(i,s);else for(s in r)r.hasOwnProperty(s)&&(i=this._getItem(s,n),(!o||o(i))&&t(i,s))},s.prototype.map=function(t,e){var i,s=e&&e.filter,o=e&&e.type||this._options.type,n=[],r=this._data;for(var a in r)r.hasOwnProperty(a)&&(i=this._getItem(a,o),(!s||s(i))&&n.push(t(i,a)));return e&&e.order&&this._sort(n,e.order),n},s.prototype._filterFields=function(t,e){var i={};for(var s in t)t.hasOwnProperty(s)&&-1!=e.indexOf(s)&&(i[s]=t[s]);return i},s.prototype._sort=function(t,e){if(o.isString(e)){var i=e;t.sort(function(t,e){var s=t[i],o=e[i];return s>o?1:o>s?-1:0})}else{if("function"!=typeof e)throw new TypeError("Order must be a function or a string");t.sort(e)}},s.prototype.remove=function(t,e){var i,s,o,n=[];if(Array.isArray(t))for(i=0,s=t.length;s>i;i++)o=this._remove(t[i]),null!=o&&n.push(o);else o=this._remove(t),null!=o&&n.push(o);return n.length&&this._trigger("remove",{items:n},e),n},s.prototype._remove=function(t){if(o.isNumber(t)||o.isString(t)){if(this._data[t])return delete this._data[t],t}else if(t instanceof Object){var e=t[this._fieldId];if(e&&this._data[e])return delete this._data[e],e}return null},s.prototype.clear=function(t){var e=Object.keys(this._data);return this._data={},this._trigger("remove",{items:e},t),e},s.prototype.max=function(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||r>s)&&(i=n,s=r)}return i},s.prototype.min=function(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||s>r)&&(i=n,s=r)}return i},s.prototype.distinct=function(t){var e,i=this._data,s=[],n=this._options.type&&this._options.type[t]||null,r=0;for(var a in i)if(i.hasOwnProperty(a)){var h=i[a],d=h[t],l=!1;for(e=0;r>e;e++)if(s[e]==d){l=!0;break}l||void 0===d||(s[r]=d,r++)}if(n)for(e=0;ei;i++)e[i]=t.getColumnId(i)||t.getColumnLabel(i);return e},s.prototype._appendRow=function(t,e,i){for(var s=t.addRow(),o=0,n=e.length;n>o;o++){var r=e[o];t.setValue(s,o,i[r])}},t.exports=s},function(t,e,i){function s(t,e){this._data=null,this._ids={},this._options=e||{},this._fieldId="id",this._subscribers={};var i=this;this.listener=function(){i._onEvent.apply(i,arguments)},this.setData(t)}var o=i(1),n=i(3);s.prototype.setData=function(t){var e,i,s;if(this._data){this._data.unsubscribe&&this._data.unsubscribe("*",this.listener),e=[];for(var o in this._ids)this._ids.hasOwnProperty(o)&&e.push(o);this._ids={},this._trigger("remove",{items:e})}if(this._data=t,this._data){for(this._fieldId=this._options.fieldId||this._data&&this._data.options&&this._data.options.fieldId||"id",e=this._data.getIds({filter:this._options&&this._options.filter}),i=0,s=e.length;s>i;i++)o=e[i],this._ids[o]=!0;this._trigger("add",{items:e}),this._data.on&&this._data.on("*",this.listener)}},s.prototype.get=function(){var t,e,i,s=this,n=o.getType(arguments[0]);"String"==n||"Number"==n||"Array"==n?(t=arguments[0],e=arguments[1],i=arguments[2]):(e=arguments[0],i=arguments[1]);var r=o.extend({},this._options,e);this._options.filter&&e&&e.filter&&(r.filter=function(t){return s._options.filter(t)&&e.filter(t)});var a=[];return void 0!=t&&a.push(t),a.push(r),a.push(i),this._data&&this._data.get.apply(this._data,a)},s.prototype.getIds=function(t){var e;if(this._data){var i,s=this._options.filter;i=t&&t.filter?s?function(e){return s(e)&&t.filter(e)}:t.filter:s,e=this._data.getIds({filter:i,order:t&&t.order})}else e=[];return e},s.prototype.getDataSet=function(){for(var t=this;t instanceof s;)t=t._data;return t||null},s.prototype._onEvent=function(t,e,i){var s,o,n,r,a=e&&e.items,h=this._data,d=[],l=[],c=[];if(a&&h){switch(t){case"add":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r&&(this._ids[n]=!0,d.push(n));break;case"update":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r?this._ids[n]?l.push(n):(this._ids[n]=!0,d.push(n)):this._ids[n]&&(delete this._ids[n],c.push(n));break;case"remove":for(s=0,o=a.length;o>s;s++)n=a[s],this._ids[n]&&(delete this._ids[n],c.push(n))}d.length&&this._trigger("add",{items:d},i),l.length&&this._trigger("update",{items:l},i),c.length&&this._trigger("remove",{items:c},i)}},s.prototype.on=n.prototype.on,s.prototype.off=n.prototype.off,s.prototype._trigger=n.prototype._trigger,s.prototype.subscribe=s.prototype.on,s.prototype.unsubscribe=s.prototype.off,t.exports=s},function(t){function e(t){this.delay=null,this.max=1/0,this._queue=[],this._timeout=null,this._extended=null,this.setOptions(t)}e.prototype.setOptions=function(t){t&&"undefined"!=typeof t.delay&&(this.delay=t.delay),t&&"undefined"!=typeof t.max&&(this.max=t.max),this._flushIfNeeded()},e.extend=function(t,i){var s=new e(i);if(void 0!==t.flush)throw new Error("Target object already has a property flush");t.flush=function(){s.flush()};var o=[{name:"flush",original:void 0}];if(i&&i.replace)for(var n=0;nthis.max&&this.flush(),clearTimeout(this._timeout),this.queue.length>0&&"number"==typeof this.delay){var t=this;this._timeout=setTimeout(function(){t.flush()},this.delay)}},e.prototype.flush=function(){for(;this._queue.length>0;){var t=this._queue.shift();t.fn.apply(t.context||t.fn,t.args||[])}},t.exports=e},function(t,e,i){function s(t,e,i){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");this.containerElement=t,this.width="400px",this.height="400px",this.margin=10,this.defaultXCenter="55%",this.defaultYCenter="50%",this.xLabel="x",this.yLabel="y",this.zLabel="z";var o=function(t){return t};this.xValueLabel=o,this.yValueLabel=o,this.zValueLabel=o,this.filterLabel="time",this.legendLabel="value",this.style=s.STYLE.DOT,this.showPerspective=!0,this.showGrid=!0,this.keepAspectRatio=!0,this.showShadow=!1,this.showGrayBottom=!1,this.showTooltip=!1,this.verticalRatio=.5,this.animationInterval=1e3,this.animationPreload=!1,this.camera=new p,this.eye=new l(0,0,-1),this.dataTable=null,this.dataPoints=null,this.colX=void 0,this.colY=void 0,this.colZ=void 0,this.colValue=void 0,this.colFilter=void 0,this.xMin=0,this.xStep=void 0,this.xMax=1,this.yMin=0,this.yStep=void 0,this.yMax=1,this.zMin=0,this.zStep=void 0,this.zMax=1,this.valueMin=0,this.valueMax=1,this.xBarWidth=1,this.yBarWidth=1,this.colorAxis="#4D4D4D",this.colorGrid="#D3D3D3",this.colorDot="#7DC1FF",this.colorDotBorder="#3267D2",this.create(),this.setOptions(i),e&&this.setData(e)}function o(t){return"clientX"in t?t.clientX:t.targetTouches[0]&&t.targetTouches[0].clientX||0}function n(t){return"clientY"in t?t.clientY:t.targetTouches[0]&&t.targetTouches[0].clientY||0}var r=i(56),a=i(3),h=i(4),d=i(1),l=i(10),c=i(9),p=i(7),u=i(8),m=i(11),f=i(12);r(s.prototype),s.prototype._setScale=function(){this.scale=new l(1/(this.xMax-this.xMin),1/(this.yMax-this.yMin),1/(this.zMax-this.zMin)),this.keepAspectRatio&&(this.scale.x3&&(this.colFilter=3);else{if(this.style!==s.STYLE.DOTCOLOR&&this.style!==s.STYLE.DOTSIZE&&this.style!==s.STYLE.BARCOLOR&&this.style!==s.STYLE.BARSIZE)throw'Unknown style "'+this.style+'"';this.colX=0,this.colY=1,this.colZ=2,this.colValue=3,t.getNumberOfColumns()>4&&(this.colFilter=4)}},s.prototype.getNumberOfRows=function(t){return t.length},s.prototype.getNumberOfColumns=function(t){var e=0;for(var i in t[0])t[0].hasOwnProperty(i)&&e++;return e},s.prototype.getDistinctValues=function(t,e){for(var i=[],s=0;st[s][e]&&(i.min=t[s][e]),i.maxt;t++){var m=(t-p)/(u-p),g=240*m,v=this._hsv2rgb(g,1,1);c.strokeStyle=v,c.beginPath(),c.moveTo(h,r+t),c.lineTo(a,r+t),c.stroke()}c.strokeStyle=this.colorAxis,c.strokeRect(h,r,i,n)}if(this.style===s.STYLE.DOTSIZE&&(c.strokeStyle=this.colorAxis,c.fillStyle=this.colorDot,c.beginPath(),c.moveTo(h,r),c.lineTo(a,r),c.lineTo(a-i+e,d),c.lineTo(h,d),c.closePath(),c.fill(),c.stroke()),this.style===s.STYLE.DOTCOLOR||this.style===s.STYLE.DOTSIZE){var y=5,b=new f(this.valueMin,this.valueMax,(this.valueMax-this.valueMin)/5,!0);for(b.start(),b.getCurrent()0?this.yMin:this.yMax,o=this._convert3Dto2D(new l(x,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(" "+this.xValueLabel(i.getCurrent())+" ",o.x,o.y),i.next()}for(g.lineWidth=1,s=void 0===this.defaultYStep,i=new f(this.yMin,this.yMax,this.yStep,s),i.start(),i.getCurrent()0?this.xMin:this.xMax,o=this._convert3Dto2D(new l(n,i.getCurrent(),this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(" "+this.yValueLabel(i.getCurrent())+" ",o.x,o.y),i.next();for(g.lineWidth=1,s=void 0===this.defaultZStep,i=new f(this.zMin,this.zMax,this.zStep,s),i.start(),i.getCurrent()0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax;!i.end();)t=this._convert3Dto2D(new l(n,r,i.getCurrent())),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(t.x-b,t.y),g.stroke(),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(this.zValueLabel(i.getCurrent())+" ",t.x-5,t.y),i.next();g.lineWidth=1,t=this._convert3Dto2D(new l(n,r,this.zMin)),e=this._convert3Dto2D(new l(n,r,this.zMax)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),g.lineWidth=1,p=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),p=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),g.lineWidth=1,t=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),t=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke();var w=this.xLabel;w.length>0&&(c=.1/this.scale.y,n=(this.xMin+this.xMax)/2,r=Math.cos(_)>0?this.yMin-c:this.yMax+c,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(w,o.x,o.y));var S=this.yLabel;S.length>0&&(d=.1/this.scale.x,n=Math.sin(_)>0?this.xMin-d:this.xMax+d,r=(this.yMin+this.yMax)/2,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(S,o.x,o.y));var M=this.zLabel;M.length>0&&(h=30,n=Math.cos(_)>0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax,a=(this.zMin+this.zMax)/2,o=this._convert3Dto2D(new l(n,r,a)),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(M,o.x-h,o.y))},s.prototype._hsv2rgb=function(t,e,i){var s,o,n,r,a,h;switch(r=i*e,a=Math.floor(t/60),h=r*(1-Math.abs(t/60%2-1)),a){case 0:s=r,o=h,n=0;break;case 1:s=h,o=r,n=0;break;case 2:s=0,o=r,n=h;break;case 3:s=0,o=h,n=r;break;case 4:s=h,o=0,n=r;break;case 5:s=r,o=0,n=h;break;default:s=0,o=0,n=0}return"RGB("+parseInt(255*s)+","+parseInt(255*o)+","+parseInt(255*n)+")"},s.prototype._redrawDataGrid=function(){var t,e,i,o,n,r,a,h,d,c,p,u,m,f=this.frame.canvas,g=f.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(n=0;n0}else r=!0;r?(m=(t.point.z+e.point.z+i.point.z+o.point.z)/4,c=240*(1-(m-this.zMin)*this.scale.z/this.verticalRatio),p=1,this.showShadow?(u=Math.min(1+S.x/M/2,1),a=this._hsv2rgb(c,p,u),h=a):(u=1,a=this._hsv2rgb(c,p,u),h=this.colorAxis)):(a="gray",h=this.colorAxis),d=.5,g.lineWidth=d,g.fillStyle=a,g.strokeStyle=h,g.beginPath(),g.moveTo(t.screen.x,t.screen.y),g.lineTo(e.screen.x,e.screen.y),g.lineTo(o.screen.x,o.screen.y),g.lineTo(i.screen.x,i.screen.y),g.closePath(),g.fill(),g.stroke()}}else for(n=0;np&&(p=0);var u,m,f;this.style===s.STYLE.DOTCOLOR?(u=240*(1-(d.point.value-this.valueMin)*this.scale.value),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)):this.style===s.STYLE.DOTSIZE?(m=this.colorDot,f=this.colorDotBorder):(u=240*(1-(d.point.z-this.zMin)*this.scale.z/this.verticalRatio),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)),i.lineWidth=1,i.strokeStyle=f,i.fillStyle=m,i.beginPath(),i.arc(d.screen.x,d.screen.y,p,0,2*Math.PI,!0),i.fill(),i.stroke()}}},s.prototype._redrawDataBar=function(){var t,e,i,o,n=this.frame.canvas,r=n.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t0&&(t=this.dataPoints[0],s.lineWidth=1,s.strokeStyle="blue",s.beginPath(),s.moveTo(t.screen.x,t.screen.y)),e=1;e0&&s.stroke()}},s.prototype._onMouseDown=function(t){if(t=t||window.event,this.leftButtonDown&&this._onMouseUp(t),this.leftButtonDown=t.which?1===t.which:1===t.button,this.leftButtonDown||this.touchDown){this.startMouseX=o(t),this.startMouseY=n(t),this.startStart=new Date(this.start),this.startEnd=new Date(this.end),this.startArmRotation=this.camera.getArmRotation(),this.frame.style.cursor="move";var e=this;this.onmousemove=function(t){e._onMouseMove(t)},this.onmouseup=function(t){e._onMouseUp(t)},d.addEventListener(document,"mousemove",e.onmousemove),d.addEventListener(document,"mouseup",e.onmouseup),d.preventDefault(t)}},s.prototype._onMouseMove=function(t){t=t||window.event;var e=parseFloat(o(t))-this.startMouseX,i=parseFloat(n(t))-this.startMouseY,s=this.startArmRotation.horizontal+e/200,r=this.startArmRotation.vertical+i/200,a=4,h=Math.sin(a/360*2*Math.PI);Math.abs(Math.sin(s))0?1:0>t?-1:0}var s=e[0],o=e[1],n=e[2],r=i((o.x-s.x)*(t.y-s.y)-(o.y-s.y)*(t.x-s.x)),a=i((n.x-o.x)*(t.y-o.y)-(n.y-o.y)*(t.x-o.x)),h=i((s.x-n.x)*(t.y-n.y)-(s.y-n.y)*(t.x-n.x));return!(0!=r&&0!=a&&r!=a||0!=a&&0!=h&&a!=h||0!=r&&0!=h&&r!=h)},s.prototype._dataPointFromXY=function(t,e){var i,o=100,n=null,r=null,a=null,h=new c(t,e);if(this.style===s.STYLE.BAR||this.style===s.STYLE.BARCOLOR||this.style===s.STYLE.BARSIZE)for(i=this.dataPoints.length-1;i>=0;i--){n=this.dataPoints[i];var d=n.surfaces;if(d)for(var l=d.length-1;l>=0;l--){var p=d[l],u=p.corners,m=[u[0].screen,u[1].screen,u[2].screen],f=[u[2].screen,u[3].screen,u[0].screen];if(this._insideTriangle(h,m)||this._insideTriangle(h,f))return n}}else for(i=0;ib)&&o>b&&(a=b,r=n)}}return r},s.prototype._showTooltip=function(t){var e,i,s;this.tooltip?(e=this.tooltip.dom.content,i=this.tooltip.dom.line,s=this.tooltip.dom.dot):(e=document.createElement("div"),e.style.position="absolute",e.style.padding="10px",e.style.border="1px solid #4d4d4d",e.style.color="#1a1a1a",e.style.background="rgba(255,255,255,0.7)",e.style.borderRadius="2px",e.style.boxShadow="5px 5px 10px rgba(128,128,128,0.5)",i=document.createElement("div"),i.style.position="absolute",i.style.height="40px",i.style.width="0",i.style.borderLeft="1px solid #4d4d4d",s=document.createElement("div"),s.style.position="absolute",s.style.height="0",s.style.width="0",s.style.border="5px solid #4d4d4d",s.style.borderRadius="5px",this.tooltip={dataPoint:null,dom:{content:e,line:i,dot:s}}),this._hideTooltip(),this.tooltip.dataPoint=t,e.innerHTML="function"==typeof this.showTooltip?this.showTooltip(t.point):"
    x:"+t.point.x+"
    y:"+t.point.y+"
    z:"+t.point.z+"
    ",e.style.left="0",e.style.top="0",this.frame.appendChild(e),this.frame.appendChild(i),this.frame.appendChild(s);var o=e.offsetWidth,n=e.offsetHeight,r=i.offsetHeight,a=s.offsetWidth,h=s.offsetHeight,d=t.screen.x-o/2;d=Math.min(Math.max(d,10),this.frame.clientWidth-10-o),i.style.left=t.screen.x+"px",i.style.top=t.screen.y-r+"px",e.style.left=d+"px",e.style.top=t.screen.y-r-n+"px",s.style.left=t.screen.x-a/2+"px",s.style.top=t.screen.y-h/2+"px"},s.prototype._hideTooltip=function(){if(this.tooltip){this.tooltip.dataPoint=null;for(var t in this.tooltip.dom)if(this.tooltip.dom.hasOwnProperty(t)){var e=this.tooltip.dom[t];e&&e.parentNode&&e.parentNode.removeChild(e)}}},t.exports=s},function(t,e,i){function s(){this.armLocation=new o,this.armRotation={},this.armRotation.horizontal=0,this.armRotation.vertical=0,this.armLength=1.7,this.cameraLocation=new o,this.cameraRotation=new o(.5*Math.PI,0,0),this.calculateCameraOrientation()}var o=i(10);s.prototype.setArmLocation=function(t,e,i){this.armLocation.x=t,this.armLocation.y=e,this.armLocation.z=i,this.calculateCameraOrientation()},s.prototype.setArmRotation=function(t,e){void 0!==t&&(this.armRotation.horizontal=t),void 0!==e&&(this.armRotation.vertical=e,this.armRotation.vertical<0&&(this.armRotation.vertical=0),this.armRotation.vertical>.5*Math.PI&&(this.armRotation.vertical=.5*Math.PI)),(void 0!==t||void 0!==e)&&this.calculateCameraOrientation()},s.prototype.getArmRotation=function(){var t={};return t.horizontal=this.armRotation.horizontal,t.vertical=this.armRotation.vertical,t},s.prototype.setArmLength=function(t){void 0!==t&&(this.armLength=t,this.armLength<.71&&(this.armLength=.71),this.armLength>5&&(this.armLength=5),this.calculateCameraOrientation())},s.prototype.getArmLength=function(){return this.armLength},s.prototype.getCameraLocation=function(){return this.cameraLocation},s.prototype.getCameraRotation=function(){return this.cameraRotation},s.prototype.calculateCameraOrientation=function(){this.cameraLocation.x=this.armLocation.x-this.armLength*Math.sin(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.y=this.armLocation.y-this.armLength*Math.cos(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.z=this.armLocation.z+this.armLength*Math.sin(this.armRotation.vertical),this.cameraRotation.x=Math.PI/2-this.armRotation.vertical,this.cameraRotation.y=0,this.cameraRotation.z=-this.armRotation.horizontal},t.exports=s},function(t,e,i){function s(t,e,i){this.data=t,this.column=e,this.graph=i,this.index=void 0,this.value=void 0,this.values=i.getDistinctValues(t.get(),this.column),this.values.sort(function(t,e){return t>e?1:e>t?-1:0}),this.values.length>0&&this.selectValue(0),this.dataPoints=[],this.loaded=!1,this.onLoadCallback=void 0,i.animationPreload?(this.loaded=!1,this.loadInBackground()):this.loaded=!0}var o=i(4);s.prototype.isLoaded=function(){return this.loaded},s.prototype.getLoadedProgress=function(){for(var t=this.values.length,e=0;this.dataPoints[e];)e++;return Math.round(e/t*100)},s.prototype.getLabel=function(){return this.graph.filterLabel},s.prototype.getColumn=function(){return this.column},s.prototype.getSelectedValue=function(){return void 0===this.index?void 0:this.values[this.index]},s.prototype.getValues=function(){return this.values},s.prototype.getValue=function(t){if(t>=this.values.length)throw"Error: index out of range";return this.values[t]},s.prototype._getDataPoints=function(t){if(void 0===t&&(t=this.index),void 0===t)return[];var e;if(this.dataPoints[t])e=this.dataPoints[t];else{var i={};i.column=this.column,i.value=this.values[t];var s=new o(this.data,{filter:function(t){return t[i.column]==i.value}}).get();e=this.graph._getDataPoints(s),this.dataPoints[t]=e}return e},s.prototype.setOnLoadCallback=function(t){this.onLoadCallback=t},s.prototype.selectValue=function(t){if(t>=this.values.length)throw"Error: index out of range";this.index=t,this.value=this.values[t]},s.prototype.loadInBackground=function(t){void 0===t&&(t=0);var e=this.graph.frame;if(t0&&(t--,this.setIndex(t))},s.prototype.next=function(){var t=this.getIndex();t0?this.setIndex(0):this.index=void 0},s.prototype.setIndex=function(t){if(!(ts&&(s=0),s>this.values.length-1&&(s=this.values.length-1),s},s.prototype.indexToLeft=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t/(this.values.length-1)*e,s=i+3;return s},s.prototype._onMouseMove=function(t){var e=t.clientX-this.startClientX,i=this.startSlideX+e,s=this.leftToIndex(i);this.setIndex(s),o.preventDefault()},s.prototype._onMouseUp=function(){this.frame.style.cursor="auto",o.removeEventListener(document,"mousemove",this.onmousemove),o.removeEventListener(document,"mouseup",this.onmouseup),o.preventDefault()},t.exports=s},function(t){function e(t,e,i,s){this._start=0,this._end=0,this._step=1,this.prettyStep=!0,this.precision=5,this._current=0,this.setRange(t,e,i,s)}e.prototype.setRange=function(t,e,i,s){this._start=t?t:0,this._end=e?e:0,this.setStep(i,s)},e.prototype.setStep=function(t,i){void 0===t||0>=t||(void 0!==i&&(this.prettyStep=i),this._step=this.prettyStep===!0?e.calculatePrettyStep(t):t)},e.calculatePrettyStep=function(t){var e=function(t){return Math.log(t)/Math.LN10},i=Math.pow(10,Math.round(e(t))),s=2*Math.pow(10,Math.round(e(t/2))),o=5*Math.pow(10,Math.round(e(t/5))),n=i;return Math.abs(s-t)<=Math.abs(n-t)&&(n=s),Math.abs(o-t)<=Math.abs(n-t)&&(n=o),0>=n&&(n=1),n},e.prototype.getCurrent=function(){return parseFloat(this._current.toPrecision(this.precision))},e.prototype.getStep=function(){return this._step},e.prototype.start=function(){this._current=this._start-this._start%this._step},e.prototype.next=function(){this._current+=this._step},e.prototype.end=function(){return this._current>this._end},t.exports=e},function(t,e,i){function s(t,e,i,r){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var h=r;r=i,i=h}var u=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:u._toScreen.bind(u),toGlobalScreen:u._toGlobalScreen.bind(u),toTime:u._toTime.bind(u),toGlobalTime:u._toGlobalTime.bind(u)}},this.range=new a(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.itemSet=new p(this.body),this.components.push(this.itemSet),this.itemsData=null,this.groupsData=null,r&&this.setOptions(r),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(27);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.itemSet&&this.itemSet.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){if(void 0==this.options.start||void 0==this.options.end)var s=this._getDataRange();var o=void 0!=this.options.start?this.options.start:s.start,a=void 0!=this.options.end?this.options.end:s.end;this.setWindow(o,a,{animate:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.itemSet.setGroups(e)},s.prototype.setSelection=function(t,e){this.itemSet&&this.itemSet.setSelection(t),e&&e.focus&&this.focus(t,e)},s.prototype.getSelection=function(){return this.itemSet&&this.itemSet.getSelection()||[]},s.prototype.focus=function(t,e){if(this.itemsData&&void 0!=t){var i=Array.isArray(t)?t:[t],s=this.itemsData.getDataSet().get(i,{type:{start:"Date",end:"Date"}}),o=null,n=null;if(s.forEach(function(t){var e=t.start.valueOf(),i="end"in t?t.end.valueOf():t.start.valueOf();(null===o||o>e)&&(o=e),(null===n||i>n)&&(n=i)}),null!==o&&null!==n){var r=(o+n)/2,a=Math.max(this.range.end-this.range.start,1.1*(n-o)),h=e&&void 0!==e.animate?e.animate:!0;this.range.setRange(r-a/2,r+a/2,h)}}},s.prototype.getItemRange=function(){var t=this.itemsData.getDataSet(),e=null,i=null;if(t){var s=t.min("start");e=s?o.convert(s.start,"Date").valueOf():null;var n=t.max("start");n&&(i=o.convert(n.start,"Date").valueOf());var r=t.max("end");r&&(i=null==i?o.convert(r.end,"Date").valueOf():Math.max(i,o.convert(r.end,"Date").valueOf()))}return{min:null!=e?new Date(e):null,max:null!=i?new Date(i):null}},t.exports=s},function(t,e,i){function s(t,e,i,s){if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var r=s;s=i,i=r}var h=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:h._toScreen.bind(h),toGlobalScreen:h._toGlobalScreen.bind(h),toTime:h._toTime.bind(h),toGlobalTime:h._toGlobalTime.bind(h)}},this.range=new a(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.linegraph=new p(this.body),this.components.push(this.linegraph),this.itemsData=null,this.groupsData=null,s&&this.setOptions(s),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(29);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.linegraph&&this.linegraph.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){var s=void 0!=this.options.start?this.options.start:null,o=void 0!=this.options.end?this.options.end:null;this.setWindow(s,o,{animate:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.linegraph.setGroups(e)},s.prototype.getLegend=function(t,e,i){return void 0===e&&(e=15),void 0===i&&(i=15),void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].getLegend(e,i):"cannot find group:"+t},s.prototype.isGroupVisible=function(t){return void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].visible&&(void 0===this.linegraph.options.groups.visibility[t]||1==this.linegraph.options.groups.visibility[t]):!1},s.prototype.getItemRange=function(){var t=null,e=null;for(var i in this.linegraph.groups)if(this.linegraph.groups.hasOwnProperty(i)&&1==this.linegraph.groups[i].visible)for(var s=0;sr?r:t,e=null==e?r:r>e?r:e}return{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},t.exports=s},function(t,e,i){var s=i(44);e.convertHiddenOptions=function(t,e){if(t.hiddenDates=[],e&&1==Array.isArray(e)){for(var i=0;i=4*a){var p=0,u=n.clone();switch(i[h].repeat){case"daily":d.day()!=l.day()&&(p=1),d.dayOfYear(o.dayOfYear()),d.year(o.year()),d.subtract(7,"days"),l.dayOfYear(o.dayOfYear()),l.year(o.year()),l.subtract(7-p,"days"),u.add(1,"weeks");break;case"weekly":var m=l.diff(d,"days"),f=d.day();d.date(o.date()),d.month(o.month()),d.year(o.year()),l=d.clone(),d.day(f),l.day(f),l.add(m,"days"),d.subtract(1,"weeks"),l.subtract(1,"weeks"),u.add(1,"weeks");break;case"monthly":d.month()!=l.month()&&(p=1),d.month(o.month()),d.year(o.year()),d.subtract(1,"months"),l.month(o.month()),l.year(o.year()),l.subtract(1,"months"),l.add(p,"months"),u.add(1,"months");break;case"yearly":d.year()!=l.year()&&(p=1),d.year(o.year()),d.subtract(1,"years"),l.year(o.year()),l.subtract(1,"years"),l.add(p,"years"),u.add(1,"years");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}for(;u>d;)switch(t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()}),i[h].repeat){case"daily":d.add(1,"days"),l.add(1,"days");break;case"weekly":d.add(1,"weeks"),l.add(1,"weeks");break;case"monthly":d.add(1,"months"),l.add(1,"months");break;case"yearly":d.add(1,"y"),l.add(1,"y");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()})}}e.removeDuplicates(t);var g=e.isHidden(t.range.start,t.hiddenDates),v=e.isHidden(t.range.end,t.hiddenDates),y=t.range.start,b=t.range.end;1==g.hidden&&(y=1==t.range.startToFront?g.startDate-1:g.endDate+1),1==v.hidden&&(b=1==t.range.endToFront?v.startDate-1:v.endDate+1),(1==g.hidden||1==v.hidden)&&t.range._applyRange(y,b)}},e.removeDuplicates=function(t){for(var e=t.hiddenDates,i=[],s=0;s=e[s].start&&e[o].end<=e[s].end?e[o].remove=!0:e[o].start>=e[s].start&&e[o].start<=e[s].end?(e[s].end=e[o].end,e[o].remove=!0):e[o].end>=e[s].start&&e[o].end<=e[s].end&&(e[s].start=e[o].start,e[o].remove=!0));for(var s=0;s=r&&a>o){i=!0;break}}if(1==i&&o=e&&i>r&&(s+=r-n)}return s},e.correctTimeForHidden=function(t,i,o){return o=s(o).toDate().valueOf(),o-=e.getHiddenDurationBefore(t,i,o)},e.getHiddenDurationBefore=function(t,e,i){var o=0;i=s(i).toDate().valueOf();for(var n=0;n=e.start&&a=a&&(o+=a-r)}return o},e.getAccumulatedHiddenDuration=function(t,e,i){for(var s=0,o=0,n=e.start,r=0;r=e.start&&h=i)break;s+=h-a}}return s},e.snapAwayFromHidden=function(t,i,s,o){var n=e.isHidden(i,t);return 1==n.hidden?0>s?1==o?n.startDate-(n.endDate-i)-1:n.startDate-1:1==o?n.endDate+(i-n.startDate)+1:n.endDate+1:i},e.isHidden=function(t,e){for(var i=0;i=s&&o>t)return{hidden:!0,startDate:s,endDate:o}}return{hidden:!1,startDate:s,endDate:o}}},function(t){function e(t,e,i,s,o,n){this.current=0,this.autoScale=!0,this.stepIndex=0,this.step=1,this.scale=1,this.marginStart,this.marginEnd,this.deadSpace=0,this.majorSteps=[1,2,5,10],this.minorSteps=[.25,.5,1,2],this.alignZeros=n,this.setRange(t,e,i,s,o)}e.prototype.setRange=function(t,e,i,s,o){this._start=void 0===o.min?t:o.min,this._end=void 0===o.max?e:o.max,this._start==this._end&&(this._start-=.75,this._end+=1),1==this.autoScale&&this.setMinimumStep(i,s),this.setFirst(o)},e.prototype.setMinimumStep=function(t,e){var i=this._end-this._start,s=1.2*i,o=t*(s/e),n=Math.round(Math.log(s)/Math.LN10),r=-1,a=Math.pow(10,n),h=0;0>n&&(h=n);for(var d=!1,l=h;Math.abs(l)<=Math.abs(n);l++){a=Math.pow(10,l);for(var c=0;c=o){d=!0,r=c;break}}if(1==d)break}this.stepIndex=r,this.scale=a,this.step=a*this.minorSteps[r]},e.prototype.setFirst=function(t){void 0===t&&(t={});var e=void 0===t.min?this._start-2*this.scale*this.minorSteps[this.stepIndex]:t.min,i=void 0===t.max?this._end+this.scale*this.minorSteps[this.stepIndex]:t.max;this.marginEnd=void 0===t.max?this.roundToMinor(i):t.max,this.marginStart=void 0===t.min?this.roundToMinor(e):t.min,1==this.alignZeros&&(this.marginEnd-this.marginStart)%this.step!=0&&(this.marginEnd+=this.marginEnd%this.step),this.deadSpace=this.roundToMinor(i)-i+this.roundToMinor(e)-e,this.marginRange=this.marginEnd-this.marginStart,this.current=this.marginEnd},e.prototype.roundToMinor=function(t){var e=t-t%(this.scale*this.minorSteps[this.stepIndex]);return t%(this.scale*this.minorSteps[this.stepIndex])>.5*this.scale*this.minorSteps[this.stepIndex]?e+this.scale*this.minorSteps[this.stepIndex]:e},e.prototype.hasNext=function(){return this.current>=this.marginStart},e.prototype.next=function(){var t=this.current;this.current-=this.step,this.current==t&&(this.current=this._end)},e.prototype.previous=function(){this.current+=this.step,this.marginEnd+=this.step,this.marginRange=this.marginEnd-this.marginStart},e.prototype.getCurrent=function(t){var e=Math.abs(this.current)0;s--){if("0"!=i[s]){if("."==i[s]||","==i[s]){i=i.slice(0,s);break}break}i=i.slice(0,s)}}else{var o="",n=i.indexOf("e");if(-1!=n&&(o=i.slice(n),i=i.slice(0,n)),n=Math.max(i.indexOf(","),i.indexOf(".")),-1===n?(0!==t&&(i+="."),n=i.length+t):0!==t&&(n+=t+1),n>i.length)for(var r=n-i.length;r>0;r--)i+="0";else i=i.slice(0,n);i+=o}return i},e.prototype.snap=function(){},e.prototype.isMajor=function(){return this.current%(this.scale*this.majorSteps[this.stepIndex])==0},t.exports=e},function(t,e,i){function s(t,e){var i=a().hours(0).minutes(0).seconds(0).milliseconds(0);this.start=i.clone().add(-3,"days").valueOf(),this.end=i.clone().add(4,"days").valueOf(),this.body=t,this.deltaDifference=0,this.scaleOffset=0,this.startToFront=!1,this.endToFront=!0,this.defaultOptions={start:null,end:null,direction:"horizontal",moveable:!0,zoomable:!0,min:null,max:null,zoomMin:10,zoomMax:31536e10},this.options=r.extend({},this.defaultOptions),this.props={touch:{}},this.animateTimer=null,this.body.emitter.on("panstart",this._onDragStart.bind(this)),this.body.emitter.on("panmove",this._onDrag.bind(this)),this.body.emitter.on("panend",this._onDragEnd.bind(this)),this.body.emitter.on("press",this._onHold.bind(this)),this.body.emitter.on("mousewheel",this._onMouseWheel.bind(this)),this.body.emitter.on("touch",this._onTouch.bind(this)),this.body.emitter.on("pinch",this._onPinch.bind(this)),this.setOptions(e)}function o(t){if("horizontal"!=t&&"vertical"!=t)throw new TypeError('Unknown direction "'+t+'". Choose "horizontal" or "vertical".')}function n(t,e){return{x:t.x-r.getAbsoluteLeft(e),y:t.y-r.getAbsoluteTop(e)}}var r=i(1),a=(i(47),i(44)),h=i(20),d=i(15);s.prototype=new h,s.prototype.setOptions=function(t){if(t){var e=["direction","min","max","zoomMin","zoomMax","moveable","zoomable","activate","hiddenDates"];r.selectiveExtend(e,this.options,t),("start"in t||"end"in t)&&this.setRange(t.start,t.end)}},s.prototype.setRange=function(t,e,i,s){s!==!0&&(s=!1);var o=void 0!=t?r.convert(t,"Date").valueOf():null,n=void 0!=e?r.convert(e,"Date").valueOf():null;if(this._cancelAnimation(),i){var a=this,h=this.start,l=this.end,c="number"==typeof i?i:500,p=(new Date).valueOf(),u=!1,m=function(){if(!a.props.touch.dragging){var t=(new Date).valueOf(),e=t-p,i=e>c,g=i||null===o?o:r.easeInOutQuad(e,h,o,c),v=i||null===n?n:r.easeInOutQuad(e,l,n,c);f=a._applyRange(g,v),d.updateHiddenDates(a.body,a.options.hiddenDates),u=u||f,f&&a.body.emitter.emit("rangechange",{start:new Date(a.start),end:new Date(a.end),byUser:s}),i?u&&a.body.emitter.emit("rangechanged",{start:new Date(a.start),end:new Date(a.end),byUser:s}):a.animateTimer=setTimeout(m,20)}};return m()}var f=this._applyRange(o,n);if(d.updateHiddenDates(this.body,this.options.hiddenDates),f){var g={start:new Date(this.start),end:new Date(this.end),byUser:s};this.body.emitter.emit("rangechange",g),this.body.emitter.emit("rangechanged",g)}},s.prototype._cancelAnimation=function(){this.animateTimer&&(clearTimeout(this.animateTimer),this.animateTimer=null)},s.prototype._applyRange=function(t,e){var i,s=null!=t?r.convert(t,"Date").valueOf():this.start,o=null!=e?r.convert(e,"Date").valueOf():this.end,n=null!=this.options.max?r.convert(this.options.max,"Date").valueOf():null,a=null!=this.options.min?r.convert(this.options.min,"Date").valueOf():null;if(isNaN(s)||null===s)throw new Error('Invalid start "'+t+'"');if(isNaN(o)||null===o)throw new Error('Invalid end "'+e+'"');if(s>o&&(o=s),null!==a&&a>s&&(i=a-s,s+=i,o+=i,null!=n&&o>n&&(o=n)),null!==n&&o>n&&(i=o-n,s-=i,o-=i,null!=a&&a>s&&(s=a)),null!==this.options.zoomMin){var h=parseFloat(this.options.zoomMin);0>h&&(h=0),h>o-s&&(this.end-this.start===h?(s=this.start,o=this.end):(i=h-(o-s),s-=i/2,o+=i/2))}if(null!==this.options.zoomMax){var d=parseFloat(this.options.zoomMax);0>d&&(d=0),o-s>d&&(this.end-this.start===d?(s=this.start,o=this.end):(i=o-s-d,s+=i/2,o-=i/2))}var l=this.start!=s||this.end!=o;return s>=this.start&&s<=this.end||o>=this.start&&o<=this.end||this.start>=s&&this.start<=o||this.end>=s&&this.end<=o||this.body.emitter.emit("checkRangedItems"),this.start=s,this.end=o,l},s.prototype.getRange=function(){return{start:this.start,end:this.end}},s.prototype.conversion=function(t,e){return s.conversion(this.start,this.end,t,e)},s.conversion=function(t,e,i,s){return void 0===s&&(s=0),0!=i&&e-t!=0?{offset:t,scale:i/(e-t-s)}:{offset:0,scale:1}},s.prototype._onDragStart=function(t){this.deltaDifference=0,this.previousDelta=0,this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.dragging=!0,this.body.dom.root&&(this.body.dom.root.style.cursor="move"),t.preventDefault())},s.prototype._onDrag=function(t){if(this.options.moveable&&this.props.touch.allowDragging){var e=this.options.direction;o(e);var i="horizontal"==e?t.deltaX:t.deltaY;i-=this.deltaDifference;var s=this.props.touch.end-this.props.touch.start,n=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end);s-=n;var r="horizontal"==e?this.body.domProps.center.width:this.body.domProps.center.height,a=-i/r*s,h=this.props.touch.start+a,l=this.props.touch.end+a,c=d.snapAwayFromHidden(this.body.hiddenDates,h,this.previousDelta-i,!0),p=d.snapAwayFromHidden(this.body.hiddenDates,l,this.previousDelta-i,!0);if(c!=h||p!=l)return this.deltaDifference+=i,this.props.touch.start=c,this.props.touch.end=p,void this._onDrag(t);this.previousDelta=i,this._applyRange(h,l),this.body.emitter.emit("rangechange",{start:new Date(this.start),end:new Date(this.end),byUser:!0}),t.preventDefault()}},s.prototype._onDragEnd=function(){this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.dragging=!1,this.body.dom.root&&(this.body.dom.root.style.cursor="auto"),this.body.emitter.emit("rangechanged",{start:new Date(this.start),end:new Date(this.end),byUser:!0}))},s.prototype._onMouseWheel=function(t){if(this.options.zoomable&&this.options.moveable){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i;i=0>e?1-e/5:1/(1+e/5);var s=n({x:t.pageX,y:t.pageY},this.body.dom.center),o=this._pointerToDate(s);this.zoom(i,o,e)}t.preventDefault()}},s.prototype._onTouch=function(){this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.allowDragging=!0,this.props.touch.center=null,this.scaleOffset=0,this.deltaDifference=0},s.prototype._onHold=function(){this.props.touch.allowDragging=!1},s.prototype._onPinch=function(t){if(this.options.zoomable&&this.options.moveable){this.props.touch.allowDragging=!1,this.props.touch.center||(this.props.touch.center=n(t.center,this.body.dom.center));var e=1/(t.scale+this.scaleOffset),i=this._pointerToDate(this.props.touch.center),s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),o=d.getHiddenDurationBefore(this.body.hiddenDates,this,i),r=s-o,a=i-o+(this.props.touch.start-(i-o))*e,h=i+r+(this.props.touch.end-(i+r))*e;this.startToFront=0>=1-e,this.endToFront=0>=e-1;var l=d.snapAwayFromHidden(this.body.hiddenDates,a,1-e,!0),c=d.snapAwayFromHidden(this.body.hiddenDates,h,e-1,!0);(l!=a||c!=h)&&(this.props.touch.start=l,this.props.touch.end=c,this.scaleOffset=1-t.scale,a=l,h=c),this.setRange(a,h,!1,!0),this.startToFront=!1,this.endToFront=!0,t.preventDefault()}},s.prototype._pointerToDate=function(t){var e,i=this.options.direction;if(o(i),"horizontal"==i)return this.body.util.toTime(t.x).valueOf();var s=this.body.domProps.center.height;return e=this.conversion(s),t.y/e.scale+e.offset},s.prototype.zoom=function(t,e,i){null==e&&(e=(this.start+this.end)/2);var s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),o=d.getHiddenDurationBefore(this.body.hiddenDates,this,e),n=s-o,r=e-o+(this.start-(e-o))*t,a=e+n+(this.end-(e+n))*t;this.startToFront=i>0?!1:!0,this.endToFront=-i>0?!1:!0;var h=d.snapAwayFromHidden(this.body.hiddenDates,r,i,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,-i,!0);(h!=r||l!=a)&&(r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0},s.prototype.move=function(t){var e=this.end-this.start,i=this.start+e*t,s=this.end+e*t;this.start=i,this.end=s},s.prototype.moveTo=function(t){var e=(this.start+this.end)/2,i=e-t,s=this.start-i,o=this.end-i;this.setRange(s,o)},t.exports=s},function(t,e){var i=.001;e.orderByStart=function(t){t.sort(function(t,e){return t.data.start-e.data.start})},e.orderByEnd=function(t){t.sort(function(t,e){var i="end"in t.data?t.data.end:t.data.start,s="end"in e.data?e.data.end:e.data.start;return i-s})},e.stack=function(t,i,s){var o,n;if(s)for(o=0,n=t.length;n>o;o++)t[o].top=null;for(o=0,n=t.length;n>o;o++){var r=t[o];if(r.stack&&null===r.top){r.top=i.axis;do{for(var a=null,h=0,d=t.length;d>h;h++){var l=t[h];if(null!==l.top&&l!==r&&l.stack&&e.collision(r,l,i.item)){a=l;break}}null!=a&&(r.top=a.top+a.height+i.item.vertical)}while(a)}}},e.nostack=function(t,e,i){var s,o,n;for(s=0,o=t.length;o>s;s++)if(void 0!==t[s].data.subgroup){n=e.axis;for(var r in i)i.hasOwnProperty(r)&&1==i[r].visible&&i[r].indexe.left&&t.top-s.vertical+ie.top}},function(t,e,i){function s(t,e,i,o){this.current=new Date,this._start=new Date,this._end=new Date,this.autoScale=!0,this.scale="day",this.step=1,this.setRange(t,e,i),this.switchedDay=!1,this.switchedMonth=!1,this.switchedYear=!1,this.hiddenDates=o,void 0===o&&(this.hiddenDates=[]),this.format=s.FORMAT}var o=i(44),n=i(15),r=i(1);s.FORMAT={minorLabels:{millisecond:"SSS",second:"s",minute:"HH:mm",hour:"HH:mm",weekday:"ddd D",day:"D",month:"MMM",year:"YYYY"},majorLabels:{millisecond:"HH:mm:ss",second:"D MMMM HH:mm",minute:"ddd D MMMM",hour:"ddd D MMMM",weekday:"MMMM YYYY",day:"MMMM YYYY",month:"YYYY",year:""}},s.prototype.setFormat=function(t){var e=r.deepExtend({},s.FORMAT);this.format=r.deepExtend(e,t)},s.prototype.setRange=function(t,e,i){if(!(t instanceof Date&&e instanceof Date))throw"No legal start or end date in method setRange";this._start=void 0!=t?new Date(t.valueOf()):new Date,this._end=void 0!=e?new Date(e.valueOf()):new Date,this.autoScale&&this.setMinimumStep(i)},s.prototype.first=function(){this.current=new Date(this._start.valueOf()),this.roundToMinor()},s.prototype.roundToMinor=function(){switch(this.scale){case"year":this.current.setFullYear(this.step*Math.floor(this.current.getFullYear()/this.step)),this.current.setMonth(0);case"month":this.current.setDate(1);case"day":case"weekday":this.current.setHours(0);case"hour":this.current.setMinutes(0);case"minute":this.current.setSeconds(0);case"second":this.current.setMilliseconds(0)}if(1!=this.step)switch(this.scale){case"millisecond":this.current.setMilliseconds(this.current.getMilliseconds()-this.current.getMilliseconds()%this.step);break;case"second":this.current.setSeconds(this.current.getSeconds()-this.current.getSeconds()%this.step);break;case"minute":this.current.setMinutes(this.current.getMinutes()-this.current.getMinutes()%this.step);break;case"hour":this.current.setHours(this.current.getHours()-this.current.getHours()%this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()-1-(this.current.getDate()-1)%this.step+1);break;case"month":this.current.setMonth(this.current.getMonth()-this.current.getMonth()%this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()-this.current.getFullYear()%this.step)}},s.prototype.hasNext=function(){return this.current.valueOf()<=this._end.valueOf()},s.prototype.next=function(){var t=this.current.valueOf();if(this.current.getMonth()<6)switch(this.scale){case"millisecond":this.current=new Date(this.current.valueOf()+this.step);break;case"second":this.current=new Date(this.current.valueOf()+1e3*this.step);break;case"minute":this.current=new Date(this.current.valueOf()+1e3*this.step*60);break;case"hour":this.current=new Date(this.current.valueOf()+1e3*this.step*60*60);var e=this.current.getHours();this.current.setHours(e-e%this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()+this.step);break;case"month":this.current.setMonth(this.current.getMonth()+this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()+this.step)}else switch(this.scale){case"millisecond":this.current=new Date(this.current.valueOf()+this.step);break;case"second":this.current.setSeconds(this.current.getSeconds()+this.step);break;case"minute":this.current.setMinutes(this.current.getMinutes()+this.step); -break;case"hour":this.current.setHours(this.current.getHours()+this.step);break;case"weekday":case"day":this.current.setDate(this.current.getDate()+this.step);break;case"month":this.current.setMonth(this.current.getMonth()+this.step);break;case"year":this.current.setFullYear(this.current.getFullYear()+this.step)}if(1!=this.step)switch(this.scale){case"millisecond":this.current.getMilliseconds()0&&(this.step=e),this.autoScale=!1},s.prototype.setAutoScale=function(t){this.autoScale=t},s.prototype.setMinimumStep=function(t){if(void 0!=t){var e=31104e6,i=2592e6,s=864e5,o=36e5,n=6e4,r=1e3,a=1;1e3*e>t&&(this.scale="year",this.step=1e3),500*e>t&&(this.scale="year",this.step=500),100*e>t&&(this.scale="year",this.step=100),50*e>t&&(this.scale="year",this.step=50),10*e>t&&(this.scale="year",this.step=10),5*e>t&&(this.scale="year",this.step=5),e>t&&(this.scale="year",this.step=1),3*i>t&&(this.scale="month",this.step=3),i>t&&(this.scale="month",this.step=1),5*s>t&&(this.scale="day",this.step=5),2*s>t&&(this.scale="day",this.step=2),s>t&&(this.scale="day",this.step=1),s/2>t&&(this.scale="weekday",this.step=1),4*o>t&&(this.scale="hour",this.step=4),o>t&&(this.scale="hour",this.step=1),15*n>t&&(this.scale="minute",this.step=15),10*n>t&&(this.scale="minute",this.step=10),5*n>t&&(this.scale="minute",this.step=5),n>t&&(this.scale="minute",this.step=1),15*r>t&&(this.scale="second",this.step=15),10*r>t&&(this.scale="second",this.step=10),5*r>t&&(this.scale="second",this.step=5),r>t&&(this.scale="second",this.step=1),200*a>t&&(this.scale="millisecond",this.step=200),100*a>t&&(this.scale="millisecond",this.step=100),50*a>t&&(this.scale="millisecond",this.step=50),10*a>t&&(this.scale="millisecond",this.step=10),5*a>t&&(this.scale="millisecond",this.step=5),a>t&&(this.scale="millisecond",this.step=1)}},s.prototype.snap=function(t){var e=new Date(t.valueOf());if("year"==this.scale){var i=e.getFullYear()+Math.round(e.getMonth()/12);e.setFullYear(Math.round(i/this.step)*this.step),e.setMonth(0),e.setDate(0),e.setHours(0),e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("month"==this.scale)e.getDate()>15?(e.setDate(1),e.setMonth(e.getMonth()+1)):e.setDate(1),e.setHours(0),e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0);else if("day"==this.scale){switch(this.step){case 5:case 2:e.setHours(24*Math.round(e.getHours()/24));break;default:e.setHours(12*Math.round(e.getHours()/12))}e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("weekday"==this.scale){switch(this.step){case 5:case 2:e.setHours(12*Math.round(e.getHours()/12));break;default:e.setHours(6*Math.round(e.getHours()/6))}e.setMinutes(0),e.setSeconds(0),e.setMilliseconds(0)}else if("hour"==this.scale){switch(this.step){case 4:e.setMinutes(60*Math.round(e.getMinutes()/60));break;default:e.setMinutes(30*Math.round(e.getMinutes()/30))}e.setSeconds(0),e.setMilliseconds(0)}else if("minute"==this.scale){switch(this.step){case 15:case 10:e.setMinutes(5*Math.round(e.getMinutes()/5)),e.setSeconds(0);break;case 5:e.setSeconds(60*Math.round(e.getSeconds()/60));break;default:e.setSeconds(30*Math.round(e.getSeconds()/30))}e.setMilliseconds(0)}else if("second"==this.scale)switch(this.step){case 15:case 10:e.setSeconds(5*Math.round(e.getSeconds()/5)),e.setMilliseconds(0);break;case 5:e.setMilliseconds(1e3*Math.round(e.getMilliseconds()/1e3));break;default:e.setMilliseconds(500*Math.round(e.getMilliseconds()/500))}else if("millisecond"==this.scale){var s=this.step>5?this.step/2:1;e.setMilliseconds(Math.round(e.getMilliseconds()/s)*s)}return e},s.prototype.isMajor=function(){if(1==this.switchedYear)switch(this.switchedYear=!1,this.scale){case"year":case"month":case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedMonth)switch(this.switchedMonth=!1,this.scale){case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedDay)switch(this.switchedDay=!1,this.scale){case"millisecond":case"second":case"minute":case"hour":return!0;default:return!1}switch(this.scale){case"millisecond":return 0==this.current.getMilliseconds();case"second":return 0==this.current.getSeconds();case"minute":return 0==this.current.getHours()&&0==this.current.getMinutes();case"hour":return 0==this.current.getHours();case"weekday":case"day":return 1==this.current.getDate();case"month":return 0==this.current.getMonth();case"year":return!1;default:return!1}},s.prototype.getLabelMinor=function(t){void 0==t&&(t=this.current);var e=this.format.minorLabels[this.scale];return e&&e.length>0?o(t).format(e):""},s.prototype.getLabelMajor=function(t){void 0==t&&(t=this.current);var e=this.format.majorLabels[this.scale];return e&&e.length>0?o(t).format(e):""},s.prototype.getClassName=function(){function t(t){return t/h%2==0?" even":" odd"}function e(t){return t.isSame(new Date,"day")?" today":t.isSame(o().add(1,"day"),"day")?" tomorrow":t.isSame(o().add(-1,"day"),"day")?" yesterday":""}function i(t){return t.isSame(new Date,"week")?" current-week":""}function s(t){return t.isSame(new Date,"month")?" current-month":""}function n(t){return t.isSame(new Date,"year")?" current-year":""}var r=o(this.current),a=r.locale?r.locale("en"):r.lang("en"),h=this.step;switch(this.scale){case"millisecond":return t(a.milliseconds()).trim();case"second":return t(a.seconds()).trim();case"minute":return t(a.minutes()).trim();case"hour":var d=a.hours();return 4==this.step&&(d=d+"-"+(d+4)),d+"h"+e(a)+t(a.hours());case"weekday":return a.format("dddd").toLowerCase()+e(a)+i(a)+t(a.date());case"day":var l=a.date(),c=a.format("MMMM").toLowerCase();return"day"+l+" "+c+s(a)+t(l-1);case"month":return a.format("MMMM").toLowerCase()+s(a)+t(a.month());case"year":var p=a.year();return"year"+p+n(a)+t(p);default:return""}},t.exports=s},function(t){function e(){this.options=null,this.props=null}e.prototype.setOptions=function(t){t&&util.extend(this.options,t)},e.prototype.redraw=function(){return!1},e.prototype.destroy=function(){},e.prototype._isResized=function(){var t=this.props._previousWidth!==this.props.width||this.props._previousHeight!==this.props.height;return this.props._previousWidth=this.props.width,this.props._previousHeight=this.props.height,t},t.exports=e},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={showCurrentTime:!0,locales:a,locale:"en"},this.options=o.extend({},this.defaultOptions),this.offset=0,this._create(),this.setOptions(e)}var o=i(1),n=i(20),r=i(44),a=i(48);s.prototype=new n,s.prototype._create=function(){var t=document.createElement("div");t.className="currenttime",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t},s.prototype.destroy=function(){this.options.showCurrentTime=!1,this.redraw(),this.body=null},s.prototype.setOptions=function(t){t&&o.selectiveExtend(["showCurrentTime","locale","locales"],this.options,t)},s.prototype.redraw=function(){if(this.options.showCurrentTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar),this.start());var e=new Date((new Date).valueOf()+this.offset),i=this.body.util.toScreen(e),s=this.options.locales[this.options.locale],o=s.current+" "+s.time+": "+r(e).format("dddd, MMMM Do YYYY, H:mm:ss");o=o.charAt(0).toUpperCase()+o.substring(1),this.bar.style.left=i+"px",this.bar.title=o}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),this.stop();return!1},s.prototype.start=function(){function t(){e.stop();var i=e.body.range.conversion(e.body.domProps.center.width).scale,s=1/i/10;30>s&&(s=30),s>1e3&&(s=1e3),e.redraw(),e.currentTimeTimer=setTimeout(t,s)}var e=this;t()},s.prototype.stop=function(){void 0!==this.currentTimeTimer&&(clearTimeout(this.currentTimeTimer),delete this.currentTimeTimer)},s.prototype.setCurrentTime=function(t){var e=o.convert(t,"Date").valueOf(),i=(new Date).valueOf();this.offset=e-i,this.redraw()},s.prototype.getCurrentTime=function(){return new Date((new Date).valueOf()+this.offset)},t.exports=s},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={showCustomTime:!1,locales:h,locale:"en"},this.options=n.extend({},this.defaultOptions),this.customTime=new Date,this.eventParams={},this._create(),this.setOptions(e)}var o=i(45),n=i(1),r=i(20),a=i(44),h=i(48);s.prototype=new r,s.prototype.setOptions=function(t){t&&n.selectiveExtend(["showCustomTime","locale","locales"],this.options,t)},s.prototype._create=function(){var t=document.createElement("div");t.className="customtime",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t;var e=document.createElement("div");e.style.position="relative",e.style.top="0px",e.style.left="-10px",e.style.height="100%",e.style.width="20px",t.appendChild(e),this.hammer=new o(e),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.on("pan",function(t){t.preventDefault()})},s.prototype.destroy=function(){this.options.showCustomTime=!1,this.redraw(),this.hammer.enable(!1),this.hammer=null,this.body=null},s.prototype.redraw=function(){if(this.options.showCustomTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar));var e=this.body.util.toScreen(this.customTime),i=this.options.locales[this.options.locale],s=i.time+": "+a(this.customTime).format("dddd, MMMM Do YYYY, H:mm:ss");s=s.charAt(0).toUpperCase()+s.substring(1),this.bar.style.left=e+"px",this.bar.title=s}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar);return!1},s.prototype.setCustomTime=function(t){this.customTime=n.convert(t,"Date"),this.redraw()},s.prototype.getCustomTime=function(){return new Date(this.customTime.valueOf())},s.prototype._onDragStart=function(t){this.eventParams.dragging=!0,this.eventParams.customTime=this.customTime,t.stopPropagation(),t.preventDefault()},s.prototype._onDrag=function(t){if(this.eventParams.dragging){var e=this.body.util.toScreen(this.eventParams.customTime)+t.deltaX,i=this.body.util.toTime(e);this.setCustomTime(i),this.body.emitter.emit("timechange",{time:new Date(this.customTime.valueOf())}),t.stopPropagation(),t.preventDefault()}},s.prototype._onDragEnd=function(t){this.eventParams.dragging&&(this.body.emitter.emit("timechanged",{time:new Date(this.customTime.valueOf())}),t.stopPropagation(),t.preventDefault())},t.exports=s},function(t,e,i){function s(t,e,i,s){this.id=o.randomUUID(),this.body=t,this.defaultOptions={orientation:"left",showMinorLabels:!0,showMajorLabels:!0,icons:!0,majorLinesOffset:7,minorLinesOffset:4,labelOffsetX:10,labelOffsetY:2,iconWidth:20,width:"40px",visible:!0,alignZeros:!0,customRange:{left:{min:void 0,max:void 0},right:{min:void 0,max:void 0}},title:{left:{text:void 0},right:{text:void 0}},format:{left:{decimals:void 0},right:{decimals:void 0}}},this.linegraphOptions=s,this.linegraphSVG=i,this.props={},this.DOMelements={lines:{},labels:{},title:{}},this.dom={},this.range={start:0,end:0},this.options=o.extend({},this.defaultOptions),this.conversionFactor=1,this.setOptions(e),this.width=Number((""+this.options.width).replace("px","")),this.minWidth=this.width,this.height=this.linegraphSVG.offsetHeight,this.hidden=!1,this.stepPixels=25,this.stepPixelsForced=25,this.zeroCrossing=-1,this.lineOffset=0,this.master=!0,this.svgElements={},this.iconsRemoved=!1,this.groups={},this.amountOfGroups=0,this._create();var n=this;this.body.emitter.on("verticalDrag",function(){n.dom.lineContainer.style.top=n.body.domProps.scrollTop+"px"})}var o=i(1),n=i(2),r=i(20),a=i(16);s.prototype=new r,s.prototype.addGroup=function(t,e){this.groups.hasOwnProperty(t)||(this.groups[t]=e),this.amountOfGroups+=1},s.prototype.updateGroup=function(t,e){this.groups[t]=e},s.prototype.removeGroup=function(t){this.groups.hasOwnProperty(t)&&(delete this.groups[t],this.amountOfGroups-=1)},s.prototype.setOptions=function(t){if(t){var e=!1;this.options.orientation!=t.orientation&&void 0!==t.orientation&&(e=!0);var i=["orientation","showMinorLabels","showMajorLabels","icons","majorLinesOffset","minorLinesOffset","labelOffsetX","labelOffsetY","iconWidth","width","visible","customRange","title","format","alignZeros"];o.selectiveExtend(i,this.options,t),this.minWidth=Number((""+this.options.width).replace("px","")),1==e&&this.dom.frame&&(this.hide(),this.show())}},s.prototype._create=function(){this.dom.frame=document.createElement("div"),this.dom.frame.style.width=this.options.width,this.dom.frame.style.height=this.height,this.dom.lineContainer=document.createElement("div"),this.dom.lineContainer.style.width="100%",this.dom.lineContainer.style.height=this.height,this.dom.lineContainer.style.position="relative",this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="absolute",this.svg.style.top="0px",this.svg.style.height="100%",this.svg.style.width="100%",this.svg.style.display="block",this.dom.frame.appendChild(this.svg)},s.prototype._redrawGroupIcons=function(){n.prepareElements(this.svgElements);var t,e=this.options.iconWidth,i=15,s=4,o=s+.5*i;t="left"==this.options.orientation?s:this.width-e-s;for(var r in this.groups)this.groups.hasOwnProperty(r)&&(1!=this.groups[r].visible||void 0!==this.linegraphOptions.visibility[r]&&1!=this.linegraphOptions.visibility[r]||(this.groups[r].drawIcon(t,o,this.svgElements,this.svg,e,i),o+=i+s));n.cleanupElements(this.svgElements),this.iconsRemoved=!1},s.prototype._cleanupIcons=function(){0==this.iconsRemoved&&(n.prepareElements(this.svgElements),n.cleanupElements(this.svgElements),this.iconsRemoved=!0)},s.prototype.show=function(){this.hidden=!1,this.dom.frame.parentNode||("left"==this.options.orientation?this.body.dom.left.appendChild(this.dom.frame):this.body.dom.right.appendChild(this.dom.frame)),this.dom.lineContainer.parentNode||this.body.dom.backgroundHorizontal.appendChild(this.dom.lineContainer)},s.prototype.hide=function(){this.hidden=!0,this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame),this.dom.lineContainer.parentNode&&this.dom.lineContainer.parentNode.removeChild(this.dom.lineContainer)},s.prototype.setRange=function(t,e){0==this.master&&1==this.options.alignZeros&&-1!=this.zeroCrossing&&t>0&&(t=0),this.range.start=t,this.range.end=e},s.prototype.redraw=function(){var t=!1,e=0;this.dom.lineContainer.style.top=this.body.domProps.scrollTop+"px";for(var i in this.groups)this.groups.hasOwnProperty(i)&&(1!=this.groups[i].visible||void 0!==this.linegraphOptions.visibility[i]&&1!=this.linegraphOptions.visibility[i]||e++);if(0==this.amountOfGroups||0==e)this.hide();else{this.show(),this.height=Number(this.linegraphSVG.style.height.replace("px","")),this.dom.lineContainer.style.height=this.height+"px",this.width=1==this.options.visible?Number((""+this.options.width).replace("px","")):0;var s=this.props,o=this.dom.frame;o.className="dataaxis",this._calculateCharSize();var n=this.options.orientation,r=this.options.showMinorLabels,a=this.options.showMajorLabels;s.minorLabelHeight=r?s.minorCharHeight:0,s.majorLabelHeight=a?s.majorCharHeight:0,s.minorLineWidth=this.body.dom.backgroundHorizontal.offsetWidth-this.lineOffset-this.width+2*this.options.minorLinesOffset,s.minorLineHeight=1,s.majorLineWidth=this.body.dom.backgroundHorizontal.offsetWidth-this.lineOffset-this.width+2*this.options.majorLinesOffset,s.majorLineHeight=1,"left"==n?(o.style.top="0",o.style.left="0",o.style.bottom="",o.style.width=this.width+"px",o.style.height=this.height+"px",this.props.width=this.body.domProps.left.width,this.props.height=this.body.domProps.left.height):(o.style.top="",o.style.bottom="0",o.style.left="0",o.style.width=this.width+"px",o.style.height=this.height+"px",this.props.width=this.body.domProps.right.width,this.props.height=this.body.domProps.right.height),t=this._redrawLabels(),t=this._isResized()||t,1==this.options.icons?this._redrawGroupIcons():this._cleanupIcons(),this._redrawTitle(n)}return t},s.prototype._redrawLabels=function(){var t=!1;n.prepareElements(this.DOMelements.lines),n.prepareElements(this.DOMelements.labels);var e=this.options.orientation,i=this.master?this.props.majorCharHeight||10:this.stepPixelsForced,s=new a(this.range.start,this.range.end,i,this.dom.frame.offsetHeight,this.options.customRange[this.options.orientation],0==this.master&&this.options.alignZeros);this.step=s;var o=(this.dom.frame.offsetHeight-s.deadSpace*(this.dom.frame.offsetHeight/s.marginRange))/((s.marginRange-s.deadSpace)/s.step);this.stepPixels=o;var r=this.height/o,h=0;if(0==this.master){o=this.stepPixelsForced,h=Math.round(this.dom.frame.offsetHeight/o-r);for(var d=0;.5*h>d;d++)s.previous();if(r=this.height/o,-1!=this.zeroCrossing&&1==this.options.alignZeros){var l=s.marginEnd/s.step-this.zeroCrossing;if(l>0)for(var d=0;l>d;d++)s.next();else if(0>l)for(var d=0;-l>d;d++)s.previous()}}else r+=.25;this.valueAtZero=s.marginEnd;var c,p=0,u=1;void 0!==this.options.format[e]&&(c=this.options.format[e].decimals),this.maxLabelSize=0;for(var m=0;u=0&&this._redrawLabel(m-2,s.getCurrent(c),e,"yAxis major",this.props.majorCharHeight),this._redrawLine(m,e,"grid horizontal major",this.options.majorLinesOffset,this.props.majorLineWidth)):this._redrawLine(m,e,"grid horizontal minor",this.options.minorLinesOffset,this.props.minorLineWidth),1==this.master&&0==s.current&&(this.zeroCrossing=u),u++}this.conversionFactor=0==this.master?m/(this.valueAtZero-s.current):this.dom.frame.offsetHeight/s.marginRange;var g=0;void 0!==this.options.title[e]&&void 0!==this.options.title[e].text&&(g=this.props.titleCharHeight);var v=1==this.options.icons?Math.max(this.options.iconWidth,g)+this.options.labelOffsetX+15:g+this.options.labelOffsetX+15;return this.maxLabelSize>this.width-v&&1==this.options.visible?(this.width=this.maxLabelSize+v,this.options.width=this.width+"px",n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),this.redraw(),t=!0):this.maxLabelSizethis.minWidth?(this.width=Math.max(this.minWidth,this.maxLabelSize+v),this.options.width=this.width+"px",n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),this.redraw(),t=!0):(n.cleanupElements(this.DOMelements.lines),n.cleanupElements(this.DOMelements.labels),t=!1),t},s.prototype.convertValue=function(t){var e=this.valueAtZero-t,i=e*this.conversionFactor;return i},s.prototype._redrawLabel=function(t,e,i,s,o){var r=n.getDOMElement("div",this.DOMelements.labels,this.dom.frame);r.className=s,r.innerHTML=e,"left"==i?(r.style.left="-"+this.options.labelOffsetX+"px",r.style.textAlign="right"):(r.style.right="-"+this.options.labelOffsetX+"px",r.style.textAlign="left"),r.style.top=t-.5*o+this.options.labelOffsetY+"px",e+="";var a=Math.max(this.props.majorCharWidth,this.props.minorCharWidth);this.maxLabelSized;d++){var c=this.visibleItems[d];c.repositionY(e)}return s},s.prototype._calculateHeight=function(t){var e,i=this.visibleItems;this.resetSubgroups();var s=this;if(i.length){var n=i[0].top,r=i[0].top+i[0].height;if(o.forEach(i,function(t){n=Math.min(n,t.top),r=Math.max(r,t.top+t.height),void 0!==t.data.subgroup&&(s.subgroups[t.data.subgroup].height=Math.max(s.subgroups[t.data.subgroup].height,t.height),s.subgroups[t.data.subgroup].visible=!0)}),n>t.axis){var a=n-t.axis;r-=a,o.forEach(i,function(t){t.top-=a})}e=r+t.item.vertical/2}else e=t.axis+t.item.vertical;return e=Math.max(e,this.props.label.height)},s.prototype.show=function(){this.dom.label.parentNode||this.itemSet.dom.labelSet.appendChild(this.dom.label),this.dom.foreground.parentNode||this.itemSet.dom.foreground.appendChild(this.dom.foreground),this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background),this.dom.axis.parentNode||this.itemSet.dom.axis.appendChild(this.dom.axis)},s.prototype.hide=function(){var t=this.dom.label;t.parentNode&&t.parentNode.removeChild(t);var e=this.dom.foreground;e.parentNode&&e.parentNode.removeChild(e);var i=this.dom.background;i.parentNode&&i.parentNode.removeChild(i);var s=this.dom.axis;s.parentNode&&s.parentNode.removeChild(s)},s.prototype.add=function(t){if(this.items[t.id]=t,t.setParent(this),void 0!==t.data.subgroup&&(void 0===this.subgroups[t.data.subgroup]&&(this.subgroups[t.data.subgroup]={height:0,visible:!1,index:this.subgroupIndex,items:[]},this.subgroupIndex++),this.subgroups[t.data.subgroup].items.push(t)),this.orderSubgroups(),-1==this.visibleItems.indexOf(t)){var e=this.itemSet.body.range;this._checkIfVisible(t,this.visibleItems,e)}},s.prototype.orderSubgroups=function(){if(void 0!==this.subgroupOrderer){var t=[];if("string"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push({subgroup:e,sortField:this.subgroups[e].items[0].data[this.subgroupOrderer]});t.sort(function(t,e){return t.sortField-e.sortField})}else if("function"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push(this.subgroups[e].items[0].data);t.sort(this.subgroupOrderer)}if(t.length>0)for(var i=0;it?-1:l>=t?0:1};if(e.length>0)for(n=0;nl}),1==this.checkRangedItems)for(this.checkRangedItems=!1,n=0;nl})}for(n=0;n=0&&(n=e[r],!o(n));r--)void 0===s[n.id]&&(s[n.id]=!0,i.push(n));for(r=t+1;rs;s++){var n=this.visibleItems[s];n.repositionY(e)}return i},s.prototype.show=function(){this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background)},t.exports=s},function(t,e,i){function s(t,e){this.body=t,this.defaultOptions={type:null,orientation:"bottom",align:"auto",stack:!0,groupOrder:null,selectable:!0,editable:{updateTime:!1,updateGroup:!1,add:!1,remove:!1},onAdd:function(t,e){e(t)},onUpdate:function(t,e){e(t)},onMove:function(t,e){e(t)},onRemove:function(t,e){e(t)},onMoving:function(t,e){e(t)},margin:{item:{horizontal:10,vertical:10},axis:20},padding:5},this.options=n.extend({},this.defaultOptions),this.itemOptions={type:{start:"Date",end:"Date"}},this.conversion={toScreen:t.util.toScreen,toTime:t.util.toTime},this.dom={},this.props={},this.hammer=null;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e){i._onAdd(e.items)},update:function(t,e){i._onUpdate(e.items)},remove:function(t,e){i._onRemove(e.items)}},this.groupListeners={add:function(t,e){i._onAddGroups(e.items)},update:function(t,e){i._onUpdateGroups(e.items)},remove:function(t,e){i._onRemoveGroups(e.items)}},this.items={},this.groups={},this.groupIds=[],this.selection=[],this.stackDirty=!0,this.touchParams={},this._create(),this.setOptions(e)}var o=i(45),n=i(1),r=i(3),a=i(4),h=i(20),d=i(25),l=i(26),c=i(33),p=i(34),u=i(35),m=i(32),f="__ungrouped__",g="__background__";s.prototype=new h,s.types={background:m,box:c,range:u,point:p},s.prototype._create=function(){var t=document.createElement("div");t.className="itemset",t["timeline-itemset"]=this,this.dom.frame=t;var e=document.createElement("div");e.className="background",t.appendChild(e),this.dom.background=e;var i=document.createElement("div");i.className="foreground",t.appendChild(i),this.dom.foreground=i;var s=document.createElement("div");s.className="axis",this.dom.axis=s;var n=document.createElement("div");n.className="labelset",this.dom.labelSet=n,this._updateUngrouped();var r=new l(g,null,this);r.show(),this.groups[g]=r,this.hammer=new o(this.body.dom.centerContainer),this.hammer.on("hammer.input",function(t){t.isFirst&&this._onTouch(t)}.bind(this)),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.on("tap",this._onSelectItem.bind(this)),this.hammer.on("press",this._onMultiSelectItem.bind(this)),this.hammer.on("doubletap",this._onAddItem.bind(this)),this.show()},s.prototype.setOptions=function(t){if(t){var e=["type","align","orientation","padding","stack","selectable","groupOrder","dataAttributes","template","hide"];n.selectiveExtend(e,this.options,t),"margin"in t&&("number"==typeof t.margin?(this.options.margin.axis=t.margin,this.options.margin.item.horizontal=t.margin,this.options.margin.item.vertical=t.margin):"object"==typeof t.margin&&(n.selectiveExtend(["axis"],this.options.margin,t.margin),"item"in t.margin&&("number"==typeof t.margin.item?(this.options.margin.item.horizontal=t.margin.item,this.options.margin.item.vertical=t.margin.item):"object"==typeof t.margin.item&&n.selectiveExtend(["horizontal","vertical"],this.options.margin.item,t.margin.item)))),"editable"in t&&("boolean"==typeof t.editable?(this.options.editable.updateTime=t.editable,this.options.editable.updateGroup=t.editable,this.options.editable.add=t.editable,this.options.editable.remove=t.editable):"object"==typeof t.editable&&n.selectiveExtend(["updateTime","updateGroup","add","remove"],this.options.editable,t.editable));var i=function(e){var i=t[e];if(i){if(!(i instanceof Function))throw new Error("option "+e+" must be a function "+e+"(item, callback)");this.options[e]=i}}.bind(this);["onAdd","onUpdate","onRemove","onMove","onMoving"].forEach(i),this.markDirty()}},s.prototype.markDirty=function(){this.groupIds=[],this.stackDirty=!0},s.prototype.destroy=function(){this.hide(),this.setItems(null),this.setGroups(null),this.hammer=null,this.body=null,this.conversion=null},s.prototype.hide=function(){this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame),this.dom.axis.parentNode&&this.dom.axis.parentNode.removeChild(this.dom.axis),this.dom.labelSet.parentNode&&this.dom.labelSet.parentNode.removeChild(this.dom.labelSet)},s.prototype.show=function(){this.dom.frame.parentNode||this.body.dom.center.appendChild(this.dom.frame),this.dom.axis.parentNode||this.body.dom.backgroundVertical.appendChild(this.dom.axis),this.dom.labelSet.parentNode||this.body.dom.left.appendChild(this.dom.labelSet)},s.prototype.setSelection=function(t){var e,i,s,o;for(void 0==t&&(t=[]),Array.isArray(t)||(t=[t]),e=0,i=this.selection.length;i>e;e++)s=this.selection[e],o=this.items[s],o&&o.unselect();for(this.selection=[],e=0,i=t.length;i>e;e++)s=t[e],o=this.items[s],o&&(this.selection.push(s),o.select())},s.prototype.getSelection=function(){return this.selection.concat([])},s.prototype.getVisibleItems=function(){var t=this.body.range.getRange(),e=this.body.util.toScreen(t.start),i=this.body.util.toScreen(t.end),s=[];for(var o in this.groups)if(this.groups.hasOwnProperty(o))for(var n=this.groups[o],r=n.visibleItems,a=0;ae&&s.push(h.id)}return s},s.prototype._deselect=function(t){for(var e=this.selection,i=0,s=e.length;s>i;i++)if(e[i]==t){e.splice(i,1);break}},s.prototype.redraw=function(){var t=this.options.margin,e=this.body.range,i=n.option.asSize,s=this.options,o=s.orientation,r=!1,a=this.dom.frame,h=s.editable.updateTime||s.editable.updateGroup;this.props.top=this.body.domProps.top.height+this.body.domProps.border.top,this.props.left=this.body.domProps.left.width+this.body.domProps.border.left,a.className="itemset"+(h?" editable":""),r=this._orderGroups()||r;var d=e.end-e.start,l=d!=this.lastVisibleInterval||this.props.width!=this.props.lastWidth;l&&(this.stackDirty=!0),this.lastVisibleInterval=d,this.props.lastWidth=this.props.width;var c=this.stackDirty,p=this._firstGroup(),u={item:t.item,axis:t.axis},m={item:t.item,axis:t.item.vertical/2},f=0,v=t.axis+t.item.vertical;return this.groups[g].redraw(e,m,c),n.forEach(this.groups,function(t){var i=t==p?u:m,s=t.redraw(e,i,c);r=s||r,f+=t.height}),f=Math.max(f,v),this.stackDirty=!1,a.style.height=i(f),this.props.width=a.offsetWidth,this.props.height=f,this.dom.axis.style.top=i("top"==o?this.body.domProps.top.height+this.body.domProps.border.top:this.body.domProps.top.height+this.body.domProps.centerContainer.height),this.dom.axis.style.left="0",r=this._isResized()||r},s.prototype._firstGroup=function(){var t="top"==this.options.orientation?0:this.groupIds.length-1,e=this.groupIds[t],i=this.groups[e]||this.groups[f];return i||null},s.prototype._updateUngrouped=function(){{var t,e,i=this.groups[f];this.groups[g]}if(this.groupsData){if(i){i.hide(),delete this.groups[f];for(e in this.items)if(this.items.hasOwnProperty(e)){t=this.items[e],t.parent&&t.parent.remove(t);var s=this._getGroupId(t.data),o=this.groups[s];o&&o.add(t)||t.hide()}}}else if(!i){var n=null,r=null;i=new d(n,r,this),this.groups[f]=i;for(e in this.items)this.items.hasOwnProperty(e)&&(t=this.items[e],i.add(t));i.show()}},s.prototype.getLabelSet=function(){return this.dom.labelSet},s.prototype.setItems=function(t){var e,i=this,s=this.itemsData;if(t){if(!(t instanceof r||t instanceof a))throw new TypeError("Data must be an instance of DataSet or DataView");this.itemsData=t}else this.itemsData=null;if(s&&(n.forEach(this.itemListeners,function(t,e){s.off(e,t)}),e=s.getIds(),this._onRemove(e)),this.itemsData){var o=this.id;n.forEach(this.itemListeners,function(t,e){i.itemsData.on(e,t,o)}),e=this.itemsData.getIds(),this._onAdd(e),this._updateUngrouped()}},s.prototype.getItems=function(){return this.itemsData},s.prototype.setGroups=function(t){var e,i=this;if(this.groupsData&&(n.forEach(this.groupListeners,function(t,e){i.groupsData.unsubscribe(e,t)}),e=this.groupsData.getIds(),this.groupsData=null,this._onRemoveGroups(e)),t){if(!(t instanceof r||t instanceof a))throw new TypeError("Data must be an instance of DataSet or DataView");this.groupsData=t}else this.groupsData=null;if(this.groupsData){var s=this.id;n.forEach(this.groupListeners,function(t,e){i.groupsData.on(e,t,s)}),e=this.groupsData.getIds(),this._onAddGroups(e)}this._updateUngrouped(),this._order(),this.body.emitter.emit("change",{queue:!0})},s.prototype.getGroups=function(){return this.groupsData},s.prototype.removeItem=function(t){var e=this.itemsData.get(t),i=this.itemsData.getDataSet();e&&this.options.onRemove(e,function(e){e&&i.remove(t)})},s.prototype._getType=function(t){return t.type||this.options.type||(t.end?"range":"box")},s.prototype._getGroupId=function(t){var e=this._getType(t);return"background"==e&&void 0==t.group?g:this.groupsData?t.group:f},s.prototype._onUpdate=function(t){var e=this;t.forEach(function(t){var i=e.itemsData.get(t,e.itemOptions),o=e.items[t],n=e._getType(i),r=s.types[n];if(o&&(r&&o instanceof r?e._updateItem(o,i):(e._removeItem(o),o=null)),!o){if(!r)throw new TypeError("rangeoverflow"==n?'Item type "rangeoverflow" is deprecated. Use css styling instead: .vis.timeline .item.range .content {overflow: visible;}':'Unknown item type "'+n+'"');o=new r(i,e.conversion,e.options),o.id=t,e._addItem(o)}}),this._order(),this.stackDirty=!0,this.body.emitter.emit("change",{queue:!0})},s.prototype._onAdd=s.prototype._onUpdate,s.prototype._onRemove=function(t){var e=0,i=this;t.forEach(function(t){var s=i.items[t];s&&(e++,i._removeItem(s))}),e&&(this._order(),this.stackDirty=!0,this.body.emitter.emit("change",{queue:!0}))},s.prototype._order=function(){n.forEach(this.groups,function(t){t.order()})},s.prototype._onUpdateGroups=function(t){this._onAddGroups(t)},s.prototype._onAddGroups=function(t){var e=this;t.forEach(function(t){var i=e.groupsData.get(t),s=e.groups[t];if(s)s.setData(i);else{if(t==f||t==g)throw new Error("Illegal group id. "+t+" is a reserved id.");var o=Object.create(e.options);n.extend(o,{height:null}),s=new d(t,i,e),e.groups[t]=s;for(var r in e.items)if(e.items.hasOwnProperty(r)){var a=e.items[r];a.data.group==t&&s.add(a)}s.order(),s.show()}}),this.body.emitter.emit("change",{queue:!0})},s.prototype._onRemoveGroups=function(t){var e=this.groups;t.forEach(function(t){var i=e[t];i&&(i.hide(),delete e[t])}),this.markDirty(),this.body.emitter.emit("change",{queue:!0})},s.prototype._orderGroups=function(){if(this.groupsData){var t=this.groupsData.getIds({order:this.options.groupOrder}),e=!n.equalArray(t,this.groupIds);if(e){var i=this.groups;t.forEach(function(t){i[t].hide()}),t.forEach(function(t){i[t].show()}),this.groupIds=t}return e}return!1},s.prototype._addItem=function(t){this.items[t.id]=t;var e=this._getGroupId(t.data),i=this.groups[e];i&&i.add(t)},s.prototype._updateItem=function(t,e){var i=t.data.group;if(t.setData(e),i!=t.data.group){var s=this.groups[i];s&&s.remove(t);var o=this._getGroupId(t.data),n=this.groups[o];n&&n.add(t)}},s.prototype._removeItem=function(t){t.hide(),delete this.items[t.id];var e=this.selection.indexOf(t.id);-1!=e&&this.selection.splice(e,1),t.parent&&t.parent.remove(t)},s.prototype._constructByEndArray=function(t){for(var e=[],i=0;i0||o.length>0)&&this.body.emitter.emit("select",{items:a})}},s.prototype._onAddItem=function(t){if(this.options.selectable&&this.options.editable.add){var e=this,i=this.body.util.snap||null,o=s.itemFromTarget(t);if(o){var r=e.itemsData.get(o.id);this.options.onUpdate(r,function(t){t&&e.itemsData.getDataSet().update(t)})}else{var a=n.getAbsoluteLeft(this.dom.frame),h=t.center.x-a,d=this.body.util.toTime(h),l={start:i?i(d):d,content:"new item"};if("range"===this.options.type){var c=this.body.util.toTime(h+this.props.width/5);l.end=i?i(c):c}l[this.itemsData._fieldId]=n.randomUUID();var p=s.groupFromTarget(t);p&&(l.group=p.groupId),this.options.onAdd(l,function(t){t&&e.itemsData.getDataSet().add(t)})}}},s.prototype._onMultiSelectItem=function(t){if(this.options.selectable){var e,i=s.itemFromTarget(t);if(i){e=this.getSelection();var o=t.srcEvent&&t.srcEvent.shiftKey||!1;if(o){e.push(i.id);var n=s._getItemRange(this.itemsData.get(e,this.itemOptions));e=[];for(var r in this.items)if(this.items.hasOwnProperty(r)){var a=this.items[r],h=a.data.start,d=void 0!==a.data.end?a.data.end:h;h>=n.min&&d<=n.max&&e.push(a.id)}}else{var l=e.indexOf(i.id);-1==l?e.push(i.id):e.splice(l,1)}this.setSelection(e),this.body.emitter.emit("select",{items:this.getSelection()})}}},s._getItemRange=function(t){var e=null,i=null;return t.forEach(function(t){(null==i||t.starte)&&(e=t.end):(null==e||t.start>e)&&(e=t.start)}),{min:i,max:e}},s.itemFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-item"))return e["timeline-item"];e=e.parentNode}return null},s.groupFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-group"))return e["timeline-group"];e=e.parentNode}return null},s.itemSetFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-itemset"))return e["timeline-itemset"];e=e.parentNode}return null},t.exports=s},function(t,e,i){function s(t,e,i,s){this.body=t,this.defaultOptions={enabled:!0,icons:!0,iconSize:20,iconSpacing:6,left:{visible:!0,position:"top-left"},right:{visible:!0,position:"top-left"}},this.side=i,this.options=o.extend({},this.defaultOptions),this.linegraphOptions=s,this.svgElements={},this.dom={},this.groups={},this.amountOfGroups=0,this._create(),this.setOptions(e)}var o=i(1),n=i(2),r=i(20);s.prototype=new r,s.prototype.clear=function(){this.groups={},this.amountOfGroups=0},s.prototype.addGroup=function(t,e){this.groups.hasOwnProperty(t)||(this.groups[t]=e),this.amountOfGroups+=1},s.prototype.updateGroup=function(t,e){this.groups[t]=e},s.prototype.removeGroup=function(t){this.groups.hasOwnProperty(t)&&(delete this.groups[t],this.amountOfGroups-=1)},s.prototype._create=function(){this.dom.frame=document.createElement("div"),this.dom.frame.className="legend",this.dom.frame.style.position="absolute",this.dom.frame.style.top="10px",this.dom.frame.style.display="block",this.dom.textArea=document.createElement("div"),this.dom.textArea.className="legendText",this.dom.textArea.style.position="relative",this.dom.textArea.style.top="0px",this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="absolute",this.svg.style.top="0px",this.svg.style.width=this.options.iconSize+5+"px",this.svg.style.height="100%",this.dom.frame.appendChild(this.svg),this.dom.frame.appendChild(this.dom.textArea)},s.prototype.hide=function(){this.dom.frame.parentNode&&this.dom.frame.parentNode.removeChild(this.dom.frame)},s.prototype.show=function(){this.dom.frame.parentNode||this.body.dom.center.appendChild(this.dom.frame)},s.prototype.setOptions=function(t){var e=["enabled","orientation","icons","left","right"];o.selectiveDeepExtend(e,this.options,t)},s.prototype.redraw=function(){var t=0;for(var e in this.groups)this.groups.hasOwnProperty(e)&&(1!=this.groups[e].visible||void 0!==this.linegraphOptions.visibility[e]&&1!=this.linegraphOptions.visibility[e]||t++);if(0==this.options[this.side].visible||0==this.amountOfGroups||0==this.options.enabled||0==t)this.hide();else{if(this.show(),"top-left"==this.options[this.side].position||"bottom-left"==this.options[this.side].position?(this.dom.frame.style.left="4px",this.dom.frame.style.textAlign="left",this.dom.textArea.style.textAlign="left",this.dom.textArea.style.left=this.options.iconSize+15+"px",this.dom.textArea.style.right="",this.svg.style.left="0px",this.svg.style.right=""):(this.dom.frame.style.right="4px",this.dom.frame.style.textAlign="right",this.dom.textArea.style.textAlign="right",this.dom.textArea.style.right=this.options.iconSize+15+"px",this.dom.textArea.style.left="",this.svg.style.right="0px",this.svg.style.left=""),"top-left"==this.options[this.side].position||"top-right"==this.options[this.side].position)this.dom.frame.style.top=4-Number(this.body.dom.center.style.top.replace("px",""))+"px",this.dom.frame.style.bottom="";else{var i=this.body.domProps.center.height-this.body.domProps.centerContainer.height;this.dom.frame.style.bottom=4+i+Number(this.body.dom.center.style.top.replace("px",""))+"px",this.dom.frame.style.top=""}0==this.options.icons?(this.dom.frame.style.width=this.dom.textArea.offsetWidth+10+"px",this.dom.textArea.style.right="",this.dom.textArea.style.left="",this.svg.style.width="0px"):(this.dom.frame.style.width=this.options.iconSize+15+this.dom.textArea.offsetWidth+10+"px",this.drawLegendIcons());var s="";for(var e in this.groups)this.groups.hasOwnProperty(e)&&(1!=this.groups[e].visible||void 0!==this.linegraphOptions.visibility[e]&&1!=this.linegraphOptions.visibility[e]||(s+=this.groups[e].content+"
    "));this.dom.textArea.innerHTML=s,this.dom.textArea.style.lineHeight=.75*this.options.iconSize+this.options.iconSpacing+"px"}},s.prototype.drawLegendIcons=function(){if(this.dom.frame.parentNode){n.prepareElements(this.svgElements);var t=window.getComputedStyle(this.dom.frame).paddingTop,e=Number(t.replace("px","")),i=e,s=this.options.iconSize,o=.75*this.options.iconSize,r=e+.5*o+3;this.svg.style.width=s+5+e+"px";for(var a in this.groups)this.groups.hasOwnProperty(a)&&(1!=this.groups[a].visible||void 0!==this.linegraphOptions.visibility[a]&&1!=this.linegraphOptions.visibility[a]||(this.groups[a].drawIcon(i,r,this.svgElements,this.svg,s,o),r+=o+this.options.iconSpacing));n.cleanupElements(this.svgElements)}},t.exports=s},function(t,e,i){function s(t,e){this.id=o.randomUUID(),this.body=t,this.defaultOptions={yAxisOrientation:"left",defaultGroup:"default",sort:!0,sampling:!0,graphHeight:"400px",shaded:{enabled:!1,orientation:"bottom"},style:"line",barChart:{width:50,handleOverlap:"overlap",align:"center"},catmullRom:{enabled:!0,parametrization:"centripetal",alpha:.5},drawPoints:{enabled:!0,size:6,style:"square"},dataAxis:{showMinorLabels:!0,showMajorLabels:!0,icons:!1,width:"40px",visible:!0,alignZeros:!0,customRange:{left:{min:void 0,max:void 0},right:{min:void 0,max:void 0}}},legend:{enabled:!1,icons:!0,left:{visible:!0,position:"top-left"},right:{visible:!0,position:"top-right"}},groups:{visibility:{}}},this.options=o.extend({},this.defaultOptions),this.dom={},this.props={},this.hammer=null,this.groups={},this.abortedGraphUpdate=!1,this.updateSVGheight=!1,this.updateSVGheightOnResize=!1;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e){i._onAdd(e.items)},update:function(t,e){i._onUpdate(e.items)},remove:function(t,e){i._onRemove(e.items)}},this.groupListeners={add:function(t,e){i._onAddGroups(e.items)},update:function(t,e){i._onUpdateGroups(e.items)},remove:function(t,e){i._onRemoveGroups(e.items)}},this.items={},this.selection=[],this.lastStart=this.body.range.start,this.touchParams={},this.svgElements={},this.setOptions(e),this.groupsUsingDefaultStyles=[0],this.COUNTER=0,this.body.emitter.on("rangechanged",function(){i.lastStart=i.body.range.start,i.svg.style.left=o.option.asSize(-i.props.width),i.redraw.call(i,!0)}),this._create(),this.framework={svg:this.svg,svgElements:this.svgElements,options:this.options,groups:this.groups},this.body.emitter.emit("change")}var o=i(1),n=i(2),r=i(3),a=i(4),h=i(20),d=i(23),l=i(24),c=i(28),p=i(52),u="__ungrouped__";s.prototype=new h,s.prototype._create=function(){var t=document.createElement("div");t.className="LineGraph",this.dom.frame=t,this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="relative",this.svg.style.height=(""+this.options.graphHeight).replace("px","")+"px",this.svg.style.display="block",t.appendChild(this.svg),this.options.dataAxis.orientation="left",this.yAxisLeft=new d(this.body,this.options.dataAxis,this.svg,this.options.groups),this.options.dataAxis.orientation="right",this.yAxisRight=new d(this.body,this.options.dataAxis,this.svg,this.options.groups),delete this.options.dataAxis.orientation,this.legendLeft=new c(this.body,this.options.legend,"left",this.options.groups),this.legendRight=new c(this.body,this.options.legend,"right",this.options.groups),this.show()},s.prototype.setOptions=function(t){if(t){var e=["sampling","defaultGroup","height","graphHeight","yAxisOrientation","style","barChart","dataAxis","sort","groups"];void 0===t.graphHeight&&void 0!==t.height&&void 0!==this.body.domProps.centerContainer.height?(this.updateSVGheight=!0,this.updateSVGheightOnResize=!0):void 0!==this.body.domProps.centerContainer.height&&void 0!==t.graphHeight&&parseInt((t.graphHeight+"").replace("px",""))0){var d=this.body.util.toGlobalTime(-this.body.domProps.root.width),l=this.body.util.toGlobalTime(2*this.body.domProps.root.width),c={};for(this._getRelevantData(a,c,d,l),this._applySampling(a,c),e=0;eu&&console.log("WARNING: there may be an infinite loop in the _updateGraph emitter cycle."),this.COUNTER=0,this.abortedGraphUpdate=!1,e=0;e0)for(r=0;rs){d.push(h);break}d.push(h)}}else for(a=0;ai&&h.x0)for(var s=0;s0){var n=1,r=o.length,a=this.body.util.toGlobalScreen(o[o.length-1].x)-this.body.util.toGlobalScreen(o[0].x),h=r/a;n=Math.min(Math.ceil(.2*r),Math.max(1,Math.round(h)));for(var d=[],l=0;r>l;l+=n)d.push(o[l]);e[t[s]]=d}}},s.prototype._getYRanges=function(t,e,i){var s,o,n,r,a=[],h=[];if(t.length>0){for(n=0;n0&&(o=this.groups[t[n]],"stack"==r.barChart.handleOverlap&&"bar"==r.style?"left"==r.yAxisOrientation?a=a.concat(o.getYRange(s)):h=h.concat(o.getYRange(s)):i[t[n]]=o.getYRange(s,t[n]));p.getStackedBarYRange(a,i,t,"__barchartLeft","left"),p.getStackedBarYRange(h,i,t,"__barchartRight","right")}},s.prototype._updateYAxis=function(t,e){var i,s,o=!1,n=!1,r=!1,a=1e9,h=1e9,d=-1e9,l=-1e9;if(t.length>0){for(var c=0;ci?i:a,d=s>d?s:d):(r=!0,h=h>i?i:h,l=s>l?s:l));1==n&&this.yAxisLeft.setRange(a,d),1==r&&this.yAxisRight.setRange(h,l)}return o=this._toggleAxisVisiblity(n,this.yAxisLeft)||o,o=this._toggleAxisVisiblity(r,this.yAxisRight)||o,1==r&&1==n?(this.yAxisLeft.drawIcons=!0,this.yAxisRight.drawIcons=!0):(this.yAxisLeft.drawIcons=!1,this.yAxisRight.drawIcons=!1),this.yAxisRight.master=!n,0==this.yAxisRight.master?(this.yAxisLeft.lineOffset=1==r?this.yAxisRight.width:0,o=this.yAxisLeft.redraw()||o,this.yAxisRight.stepPixelsForced=this.yAxisLeft.stepPixels,this.yAxisRight.zeroCrossing=this.yAxisLeft.zeroCrossing,o=this.yAxisRight.redraw()||o):o=this.yAxisRight.redraw()||o,-1!=t.indexOf("__barchartLeft")&&t.splice(t.indexOf("__barchartLeft"),1),-1!=t.indexOf("__barchartRight")&&t.splice(t.indexOf("__barchartRight"),1),o},s.prototype._toggleAxisVisiblity=function(t,e){var i=!1;return 0==t?e.dom.frame.parentNode&&0==e.hidden&&(e.hide(),i=!0):e.dom.frame.parentNode||1!=e.hidden||(e.show(),i=!0),i},s.prototype._convertXcoordinates=function(t){for(var e,i,s=[],o=this.body.util.toScreen,n=0;ny;)y++,l=h.getCurrent(),c=h.isMajor(),u=h.getClassName(),f=m,m=this.body.util.toScreen(l),g=m-f,p&&(p.style.width=g+"px"),this.options.showMinorLabels&&this._repaintMinorText(m,h.getLabelMinor(),t,u),c&&this.options.showMajorLabels?(m>0&&(void 0==v&&(v=m),this._repaintMajorText(m,h.getLabelMajor(),t,u)),p=this._repaintMajorLine(m,t,u)):p=this._repaintMinorLine(m,t,u),h.next();if(this.options.showMajorLabels){var b=this.body.util.toTime(0),_=h.getLabelMajor(b),x=_.length*(this.props.majorCharWidth||10)+10;(void 0==v||v>x)&&this._repaintMajorText(0,_,t,u)}o.forEach(this.dom.redundant,function(t){for(;t.length;){var e=t.pop();e&&e.parentNode&&e.parentNode.removeChild(e)}})},s.prototype._repaintMinorText=function(t,e,i,s){var o=this.dom.redundant.minorTexts.shift();if(!o){var n=document.createTextNode("");o=document.createElement("div"),o.appendChild(n),this.dom.foreground.appendChild(o)}this.dom.minorTexts.push(o),o.childNodes[0].nodeValue=e,o.style.top="top"==i?this.props.majorLabelHeight+"px":"0",o.style.left=t+"px",o.className="text minor "+s},s.prototype._repaintMajorText=function(t,e,i,s){var o=this.dom.redundant.majorTexts.shift();if(!o){var n=document.createTextNode(e);o=document.createElement("div"),o.appendChild(n),this.dom.foreground.appendChild(o)}this.dom.majorTexts.push(o),o.childNodes[0].nodeValue=e,o.className="text major "+s,o.style.top="top"==i?"0":this.props.minorLabelHeight+"px",o.style.left=t+"px"},s.prototype._repaintMinorLine=function(t,e,i){var s=this.dom.redundant.lines.shift();s||(s=document.createElement("div"),this.dom.background.appendChild(s)),this.dom.lines.push(s);var o=this.props;return s.style.top="top"==e?o.majorLabelHeight+"px":this.body.domProps.top.height+"px",s.style.height=o.minorLineHeight+"px",s.style.left=t-o.minorLineWidth/2+"px",s.className="grid vertical minor "+i,s},s.prototype._repaintMajorLine=function(t,e,i){var s=this.dom.redundant.lines.shift();s||(s=document.createElement("div"),this.dom.background.appendChild(s)),this.dom.lines.push(s);var o=this.props;return s.style.top="top"==e?"0":this.body.domProps.top.height+"px",s.style.left=t-o.majorLineWidth/2+"px",s.style.height=o.majorLineHeight+"px",s.className="grid vertical major "+i,s},s.prototype._calculateCharSize=function(){this.dom.measureCharMinor||(this.dom.measureCharMinor=document.createElement("DIV"),this.dom.measureCharMinor.className="text minor measure",this.dom.measureCharMinor.style.position="absolute",this.dom.measureCharMinor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMinor)),this.props.minorCharHeight=this.dom.measureCharMinor.clientHeight,this.props.minorCharWidth=this.dom.measureCharMinor.clientWidth,this.dom.measureCharMajor||(this.dom.measureCharMajor=document.createElement("DIV"),this.dom.measureCharMajor.className="text major measure",this.dom.measureCharMajor.style.position="absolute",this.dom.measureCharMajor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMajor)),this.props.majorCharHeight=this.dom.measureCharMajor.clientHeight,this.props.majorCharWidth=this.dom.measureCharMajor.clientWidth},s.prototype.snap=function(t){return this.step.snap(t)},t.exports=s},function(t,e,i){function s(t,e,i){this.id=null,this.parent=null,this.data=t,this.dom=null,this.conversion=e||{},this.options=i||{},this.selected=!1,this.displayed=!1,this.dirty=!0,this.top=null,this.left=null,this.width=null,this.height=null}var o=i(45),n=i(1);s.prototype.stack=!0,s.prototype.select=function(){this.selected=!0,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.unselect=function(){this.selected=!1,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.setData=function(t){this.data=t,this.dirty=!0,this.displayed&&this.redraw()},s.prototype.setParent=function(t){this.displayed?(this.hide(),this.parent=t,this.parent&&this.show()):this.parent=t},s.prototype.isVisible=function(){return!1},s.prototype.show=function(){return!1},s.prototype.hide=function(){return!1},s.prototype.redraw=function(){},s.prototype.repositionX=function(){},s.prototype.repositionY=function(){},s.prototype._repaintDeleteButton=function(t){if(this.selected&&this.options.editable.remove&&!this.dom.deleteButton){var e=this,i=document.createElement("div");i.className="delete",i.title="Delete this item",new o(i).on("tap",function(t){e.parent.removeFromDataSet(e),t.stopPropagation(),t.preventDefault()}),t.appendChild(i),this.dom.deleteButton=i}else!this.selected&&this.dom.deleteButton&&(this.dom.deleteButton.parentNode&&this.dom.deleteButton.parentNode.removeChild(this.dom.deleteButton),this.dom.deleteButton=null)},s.prototype._updateContents=function(t){var e;if(this.options.template){var i=this.parent.itemSet.itemsData.get(this.id);e=this.options.template(i)}else e=this.data.content;if(e!==this.content){if(e instanceof Element)t.innerHTML="",t.appendChild(e);else if(void 0!=e)t.innerHTML=e;else if("background"!=this.data.type||void 0!==this.data.content)throw new Error('Property "content" missing in item '+this.id);this.content=e}},s.prototype._updateTitle=function(t){null!=this.data.title?t.title=this.data.title||"":t.removeAttribute("title")},s.prototype._updateDataAttributes=function(t){if(this.options.dataAttributes&&this.options.dataAttributes.length>0){var e=[];if(Array.isArray(this.options.dataAttributes))e=this.options.dataAttributes;else{if("all"!=this.options.dataAttributes)return;e=Object.keys(this.data)}for(var i=0;it.start},s.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.content=document.createElement("div"),t.content.className="content",t.box.appendChild(t.content),this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.background;if(!e)throw new Error("Cannot redraw item: parent has no background container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.content),this._updateDataAttributes(this.dom.content),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.props.content.width=this.dom.content.offsetWidth,this.height=0,this.dirty=!1}},s.prototype.show=r.prototype.show,s.prototype.hide=r.prototype.hide,s.prototype.repositionX=r.prototype.repositionX,s.prototype.repositionY=function(t){var e="top"===this.options.orientation;this.dom.content.style.top=e?"":"0",this.dom.content.style.bottom=e?"0":"";var i;if(void 0!==this.data.subgroup){var s=this.data.subgroup,o=this.parent.subgroups,r=o[s].index;if(1==e){i=this.parent.subgroups[s].height+t.item.vertical,i+=0==r?t.axis-.5*t.item.vertical:0;var a=this.parent.top;for(var h in o)o.hasOwnProperty(h)&&1==o[h].visible&&o[h].indexr&&(a+=o[h].height+t.item.vertical);i=this.parent.subgroups[s].height+t.item.vertical,this.dom.box.style.top=a+"px",this.dom.box.style.bottom=""}}else this.parent instanceof n?(i=Math.max(this.parent.height,this.parent.itemSet.body.domProps.center.height,this.parent.itemSet.body.domProps.centerContainer.height),this.dom.box.style.top=e?"0":"",this.dom.box.style.bottom=e?"":"0"):(i=this.parent.height,this.dom.box.style.top=this.parent.top+"px",this.dom.box.style.bottom="");this.dom.box.style.height=i+"px"},t.exports=s},function(t,e,i){function s(t,e,i){if(this.props={dot:{width:0,height:0},line:{width:0,height:0}},t&&void 0==t.start)throw new Error('Property "start" missing in item '+t);o.call(this,t,e,i)}{var o=i(31);i(1)}s.prototype=new o(null,null,null),s.prototype.isVisible=function(t){var e=(t.end-t.start)/4;return this.data.start>t.start-e&&this.data.startt.start-e&&this.data.startt.start},s.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.content=document.createElement("div"),t.content.className="content",t.box.appendChild(t.content),t.box["timeline-item"]=this,this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.foreground;if(!e)throw new Error("Cannot redraw item: parent has no foreground container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.box),this._updateDataAttributes(this.dom.box),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.dom.content.style.maxWidth="none",this.props.content.width=this.dom.content.offsetWidth,this.height=this.dom.box.offsetHeight,this.dom.content.style.maxWidth="",this.dirty=!1}this._repaintDeleteButton(t.box),this._repaintDragLeft(),this._repaintDragRight()},s.prototype.show=function(){this.displayed||this.redraw()},s.prototype.hide=function(){if(this.displayed){var t=this.dom.box;t.parentNode&&t.parentNode.removeChild(t),this.top=null,this.left=null,this.displayed=!1}},s.prototype.repositionX=function(){var t,e,i=this.parent.width,s=this.conversion.toScreen(this.data.start),o=this.conversion.toScreen(this.data.end);-i>s&&(s=-i),o>2*i&&(o=2*i);var n=Math.max(o-s,1);switch(this.overflow?(this.left=s,this.width=n+this.props.content.width,e=this.props.content.width):(this.left=s,this.width=n,e=Math.min(o-s-2*this.options.padding,this.props.content.width)),this.dom.box.style.left=this.left+"px",this.dom.box.style.width=n+"px",this.options.align){case"left":this.dom.content.style.left="0";break;case"right":this.dom.content.style.left=Math.max(n-e-2*this.options.padding,0)+"px";break;case"center":this.dom.content.style.left=Math.max((n-e-2*this.options.padding)/2,0)+"px";break;default:t=this.overflow?o>0?Math.max(-s,0):-e:0>s?Math.min(-s,o-s-e-2*this.options.padding):0,this.dom.content.style.left=t+"px"}},s.prototype.repositionY=function(){var t=this.options.orientation,e=this.dom.box;e.style.top="top"==t?this.top+"px":this.parent.height-this.top-this.height+"px"},s.prototype._repaintDragLeft=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragLeft){var t=document.createElement("div");t.className="drag-left",t.dragLeftItem=this,this.dom.box.appendChild(t),this.dom.dragLeft=t}else!this.selected&&this.dom.dragLeft&&(this.dom.dragLeft.parentNode&&this.dom.dragLeft.parentNode.removeChild(this.dom.dragLeft),this.dom.dragLeft=null)},s.prototype._repaintDragRight=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragRight){var t=document.createElement("div");t.className="drag-right",t.dragRightItem=this,this.dom.box.appendChild(t),this.dom.dragRight=t}else!this.selected&&this.dom.dragRight&&(this.dom.dragRight.parentNode&&this.dom.dragRight.parentNode.removeChild(this.dom.dragRight),this.dom.dragRight=null)},t.exports=s},function(t,e,i){function s(t,e,i){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");this._determineBrowserMethod(),this._initializeMixinLoaders(),this.containerElement=t,this.renderRefreshRate=60,this.renderTimestep=1e3/this.renderRefreshRate,this.renderTime=0,this.physicsTime=0,this.runDoubleSpeed=!1,this.physicsDiscreteStepsize=.5,this.initializing=!0,this.triggerFunctions={add:null,edit:null,editEdge:null,connect:null,del:null},this.defaultOptions={nodes:{mass:1,radiusMin:10,radiusMax:30,radius:10,shape:"ellipse",image:void 0,widthMin:16,widthMax:64,fontColor:"black",fontSize:14,fontFace:"verdana",fontFill:void 0,fontStrokeWidth:0,fontStrokeColor:"white",level:-1,color:{border:"#2B7CE9",background:"#97C2FC",highlight:{border:"#2B7CE9",background:"#D2E5FF"},hover:{border:"#2B7CE9",background:"#D2E5FF"}},group:void 0,borderWidth:1,borderWidthSelected:void 0},edges:{widthMin:1,widthMax:15,width:1,widthSelectionMultiplier:2,hoverWidth:1.5,style:"line",color:{color:"#848484",highlight:"#848484",hover:"#848484"},fontColor:"#343434",fontSize:14,fontFace:"arial",fontFill:"white",fontStrokeWidth:0,fontStrokeColor:"white",labelAlignment:"horizontal",arrowScaleFactor:1,dash:{length:10,gap:5,altLength:void 0},inheritColor:"from"},configurePhysics:!1,physics:{barnesHut:{enabled:!0,thetaInverted:2,gravitationalConstant:-2e3,centralGravity:.3,springLength:95,springConstant:.04,damping:.09},repulsion:{centralGravity:0,springLength:200,springConstant:.05,nodeDistance:100,damping:.09},hierarchicalRepulsion:{enabled:!1,centralGravity:0,springLength:100,springConstant:.01,nodeDistance:150,damping:.09},damping:null,centralGravity:null,springLength:null,springConstant:null},clustering:{enabled:!1,initialMaxNodes:100,clusterThreshold:500,reduceToNodes:300,chainThreshold:.4,clusterEdgeThreshold:20,sectorThreshold:100,screenSizeThreshold:.2,fontSizeMultiplier:4,maxFontSize:1e3,forceAmplification:.1,distanceAmplification:.1,edgeGrowth:20,nodeScaling:{width:1,height:1,radius:1},maxNodeSizeIncrements:600,activeAreaBoxSize:80,clusterLevelDifference:2},navigation:{enabled:!1},keyboard:{enabled:!1,speed:{x:10,y:10,zoom:.02}},dataManipulation:{enabled:!1,initiallyVisible:!1},hierarchicalLayout:{enabled:!1,levelSeparation:150,nodeSpacing:100,direction:"UD",layout:"hubsize"},freezeForStabilization:!1,smoothCurves:{enabled:!0,dynamic:!0,type:"continuous",roundness:.5},maxVelocity:30,minVelocity:.1,stabilize:!0,stabilizationIterations:1e3,zoomExtentOnStabilize:!0,locale:"en",locales:_,tooltip:{delay:300,fontColor:"black",fontSize:14,fontFace:"verdana",color:{border:"#666",background:"#FFFFC6"}},dragNetwork:!0,dragNodes:!0,zoomable:!0,hover:!1,hideEdgesOnDrag:!1,hideNodesOnDrag:!1,width:"100%",height:"100%",selectable:!0},this.constants=a.extend({},this.defaultOptions),this.pixelRatio=1,this.hoverObj={nodes:{},edges:{}},this.controlNodesActive=!1,this.navigationHammers={existing:[],_new:[]},this.animationSpeed=1/this.renderRefreshRate,this.animationEasingFunction="easeInOutQuint",this.easingTime=0,this.sourceScale=0,this.targetScale=0,this.sourceTranslation=0,this.targetTranslation=0,this.lockedOnNodeId=null,this.lockedOnNodeOffset=null,this.touchTime=0;var o=this;this.groups=new u,this.images=new m,this.images.setOnloadCallback(function(){o._redraw()}),this.xIncrement=0,this.yIncrement=0,this.zoomIncrement=0,this._loadPhysicsSystem(),this._create(),this._loadSectorSystem(),this._loadClusterSystem(),this._loadSelectionSystem(),this._loadHierarchySystem(),this._setTranslation(this.frame.clientWidth/2,this.frame.clientHeight/2),this._setScale(1),this.setOptions(i),this.freezeSimulation=!1,this.cachedFunctions={},this.startedStabilization=!1,this.stabilized=!1,this.stabilizationIterations=null,this.draggingNodes=!1,this.calculationNodes={},this.calculationNodeIndices=[],this.nodeIndices=[],this.nodes={},this.edges={},this.canvasTopLeft={x:0,y:0},this.canvasBottomRight={x:0,y:0},this.pointerPosition={x:0,y:0},this.areaCenter={},this.scale=1,this.previousScale=this.scale,this.nodesData=null,this.edgesData=null,this.nodesListeners={add:function(t,e){o._addNodes(e.items),o.start()},update:function(t,e){o._updateNodes(e.items,e.data),o.start()},remove:function(t,e){o._removeNodes(e.items),o.start()}},this.edgesListeners={add:function(t,e){o._addEdges(e.items),o.start()},update:function(t,e){o._updateEdges(e.items),o.start()},remove:function(t,e){o._removeEdges(e.items),o.start()}},this.moving=!0,this.timer=void 0,this.setData(e,this.constants.clustering.enabled||this.constants.hierarchicalLayout.enabled),this.initializing=!1,1==this.constants.hierarchicalLayout.enabled?this._setupHierarchicalLayout():0==this.constants.stabilize&&this.zoomExtent(void 0,!0,this.constants.clustering.enabled),this.constants.clustering.enabled&&this.startWithClustering()}var o=i(56),n=i(45),r=i(58),a=i(1),h=i(47),d=i(3),l=i(4),c=i(42),p=i(43),u=i(38),m=i(39),f=i(40),g=i(37),v=i(41),y=i(54),b=i(55),_=i(49);i(50),o(s.prototype),s.prototype._determineBrowserMethod=function(){var t=navigator.userAgent.toLowerCase();this.requiresTimeout=!1,-1!=t.indexOf("msie 9.0")?this.requiresTimeout=!0:-1!=t.indexOf("safari")&&t.indexOf("chrome")<=-1&&(this.requiresTimeout=!0)},s.prototype._getScriptPath=function(){for(var t=document.getElementsByTagName("script"),e=0;et.boundingBox.left&&(s=t.boundingBox.left),ot.boundingBox.bottom&&(e=t.boundingBox.bottom),i=this.constants.clustering.initialMaxNodes?49.07548/(n+142.05338)+91444e-8:12.662/(n+7.4147)+.0964822:1==this.constants.clustering.enabled&&n>=this.constants.clustering.initialMaxNodes?77.5271985/(n+187.266146)+476710517e-13:30.5062972/(n+19.93597763)+.08413486;var r=Math.min(this.frame.canvas.clientWidth/600,this.frame.canvas.clientHeight/600);s*=r}else{var a=1.1*Math.abs(o.maxX-o.minX),h=1.1*Math.abs(o.maxY-o.minY),d=this.frame.canvas.clientWidth/a,l=this.frame.canvas.clientHeight/h;s=l>=d?d:l}s>1&&(s=1);var c=this._findCenter(o);if(0==i){var p={position:c,scale:s,animation:t};this.moveTo(p),this.moving=!0,this.start()}else c.x*=s,c.y*=s,c.x-=.5*this.frame.canvas.clientWidth,c.y-=.5*this.frame.canvas.clientHeight,this._setScale(s),this._setTranslation(-c.x,-c.y)},s.prototype._updateNodeIndexList=function(){this._clearNodeIndexList();for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&this.nodeIndices.push(t)},s.prototype.setData=function(t,e){if(void 0===e&&(e=!1),this.initializing=!0,t&&t.dot&&(t.nodes||t.edges))throw new SyntaxError('Data must contain either parameter "dot" or parameter pair "nodes" and "edges", but not both.');if(1==this.constants.dataManipulation.enabled&&this._createManipulatorBar(),this.setOptions(t&&t.options),t&&t.dot){if(t&&t.dot){var i=c.DOTToGraph(t.dot);return void this.setData(i)}}else if(t&&t.gephi){if(t&&t.gephi){var s=p.parseGephi(t.gephi);return void this.setData(s)}}else this._setNodes(t&&t.nodes),this._setEdges(t&&t.edges);this._putDataInSector(),0==e&&(1==this.constants.hierarchicalLayout.enabled?(this._resetLevels(),this._setupHierarchicalLayout()):this.constants.stabilize&&this._stabilize(),this.start()),this.initializing=!1},s.prototype.setOptions=function(t){if(t){var e,i=["nodes","edges","smoothCurves","hierarchicalLayout","clustering","navigation","keyboard","dataManipulation","onAdd","onEdit","onEditEdge","onConnect","onDelete","clickToUse"];if(a.selectiveNotDeepExtend(i,this.constants,t),a.selectiveNotDeepExtend(["color"],this.constants.nodes,t.nodes),a.selectiveNotDeepExtend(["color","length"],this.constants.edges,t.edges),t.physics&&(a.mergeOptions(this.constants.physics,t.physics,"barnesHut"),a.mergeOptions(this.constants.physics,t.physics,"repulsion"),t.physics.hierarchicalRepulsion)){this.constants.hierarchicalLayout.enabled=!0,this.constants.physics.hierarchicalRepulsion.enabled=!0,this.constants.physics.barnesHut.enabled=!1;for(e in t.physics.hierarchicalRepulsion)t.physics.hierarchicalRepulsion.hasOwnProperty(e)&&(this.constants.physics.hierarchicalRepulsion[e]=t.physics.hierarchicalRepulsion[e]) -}if(t.onAdd&&(this.triggerFunctions.add=t.onAdd),t.onEdit&&(this.triggerFunctions.edit=t.onEdit),t.onEditEdge&&(this.triggerFunctions.editEdge=t.onEditEdge),t.onConnect&&(this.triggerFunctions.connect=t.onConnect),t.onDelete&&(this.triggerFunctions.del=t.onDelete),a.mergeOptions(this.constants,t,"smoothCurves"),a.mergeOptions(this.constants,t,"hierarchicalLayout"),a.mergeOptions(this.constants,t,"clustering"),a.mergeOptions(this.constants,t,"navigation"),a.mergeOptions(this.constants,t,"keyboard"),a.mergeOptions(this.constants,t,"dataManipulation"),t.dataManipulation&&(this.editMode=this.constants.dataManipulation.initiallyVisible),t.edges&&(void 0!==t.edges.color&&(a.isString(t.edges.color)?(this.constants.edges.color={},this.constants.edges.color.color=t.edges.color,this.constants.edges.color.highlight=t.edges.color,this.constants.edges.color.hover=t.edges.color):(void 0!==t.edges.color.color&&(this.constants.edges.color.color=t.edges.color.color),void 0!==t.edges.color.highlight&&(this.constants.edges.color.highlight=t.edges.color.highlight),void 0!==t.edges.color.hover&&(this.constants.edges.color.hover=t.edges.color.hover)),this.constants.edges.inheritColor=!1),t.edges.fontColor||void 0!==t.edges.color&&(a.isString(t.edges.color)?this.constants.edges.fontColor=t.edges.color:void 0!==t.edges.color.color&&(this.constants.edges.fontColor=t.edges.color.color))),t.nodes&&t.nodes.color){var s=a.parseColor(t.nodes.color);this.constants.nodes.color.background=s.background,this.constants.nodes.color.border=s.border,this.constants.nodes.color.highlight.background=s.highlight.background,this.constants.nodes.color.highlight.border=s.highlight.border,this.constants.nodes.color.hover.background=s.hover.background,this.constants.nodes.color.hover.border=s.hover.border}if(t.groups)for(var o in t.groups)if(t.groups.hasOwnProperty(o)){var n=t.groups[o];this.groups.add(o,n)}if(t.tooltip){for(e in t.tooltip)t.tooltip.hasOwnProperty(e)&&(this.constants.tooltip[e]=t.tooltip[e]);t.tooltip.color&&(this.constants.tooltip.color=a.parseColor(t.tooltip.color))}if("clickToUse"in t&&(t.clickToUse?this.activator||(this.activator=new b(this.frame),this.activator.on("change",this._createKeyBinds.bind(this))):this.activator&&(this.activator.destroy(),delete this.activator)),t.labels)throw new Error('Option "labels" is deprecated. Use options "locale" and "locales" instead.');this._loadPhysicsSystem(),this._loadNavigationControls(),this._loadManipulationSystem(),this._configureSmoothCurves(),this._createKeyBinds(),this.setSize(this.constants.width,this.constants.height),this.moving=!0,this.start()}},s.prototype._create=function(){for(;this.containerElement.hasChildNodes();)this.containerElement.removeChild(this.containerElement.firstChild);if(this.frame=document.createElement("div"),this.frame.className="vis network-frame",this.frame.style.position="relative",this.frame.style.overflow="hidden",this.frame.canvas=document.createElement("canvas"),this.frame.canvas.style.position="relative",this.frame.appendChild(this.frame.canvas),this.frame.canvas.getContext){var t=this.frame.canvas.getContext("2d");this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1),this.frame.canvas.getContext("2d").setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0)}else{var e=document.createElement("DIV");e.style.color="red",e.style.fontWeight="bold",e.style.padding="10px",e.innerHTML="Error: your browser does not support HTML canvas",this.frame.canvas.appendChild(e)}var i=this;this.drag={},this.pinch={},this.hammer=new n(this.frame.canvas),this.hammer.get("pinch").set({enable:!0}),this.hammer.on("tap",i._onTap.bind(i)),this.hammer.on("doubletap",i._onDoubleTap.bind(i)),this.hammer.on("press",i._onHold.bind(i)),this.hammer.on("pinch",i._onPinch.bind(i)),h.onTouch(this.hammer,i._onTouch.bind(i)),this.hammer.on("panstart",i._onDragStart.bind(i)),this.hammer.on("panmove",i._onDrag.bind(i)),this.hammer.on("panend",i._onDragEnd.bind(i)),this.frame.canvas.addEventListener("mousemove",i._onMouseMoveTitle.bind(i)),this.frame.canvas.addEventListener("mousewheel",i._onMouseWheel.bind(i)),this.frame.canvas.addEventListener("DOMMouseScroll",i._onMouseWheel.bind(i)),this.containerElement.appendChild(this.frame)},s.prototype._createKeyBinds=function(){var t=this;void 0!==this.keycharm&&this.keycharm.destroy(),this.keycharm=r(),this.keycharm.reset(),this.constants.keyboard.enabled&&this.isActive()&&(this.keycharm.bind("up",this._moveUp.bind(t),"keydown"),this.keycharm.bind("up",this._yStopMoving.bind(t),"keyup"),this.keycharm.bind("down",this._moveDown.bind(t),"keydown"),this.keycharm.bind("down",this._yStopMoving.bind(t),"keyup"),this.keycharm.bind("left",this._moveLeft.bind(t),"keydown"),this.keycharm.bind("left",this._xStopMoving.bind(t),"keyup"),this.keycharm.bind("right",this._moveRight.bind(t),"keydown"),this.keycharm.bind("right",this._xStopMoving.bind(t),"keyup"),this.keycharm.bind("=",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("=",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("num+",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("num+",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("num-",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("num-",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("-",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("-",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("[",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("[",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("]",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("]",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("pageup",this._zoomIn.bind(t),"keydown"),this.keycharm.bind("pageup",this._stopZoom.bind(t),"keyup"),this.keycharm.bind("pagedown",this._zoomOut.bind(t),"keydown"),this.keycharm.bind("pagedown",this._stopZoom.bind(t),"keyup")),1==this.constants.dataManipulation.enabled&&(this.keycharm.bind("esc",this._createManipulatorBar.bind(t)),this.keycharm.bind("delete",this._deleteSelected.bind(t)))},s.prototype.destroy=function(){this.start=function(){},this.redraw=function(){},this.timer=!1,this._cleanupPhysicsConfiguration(),this.keycharm.reset(),this.hammer.destroy(),this.off(),this._recursiveDOMDelete(this.containerElement)},s.prototype._recursiveDOMDelete=function(t){for(;1==t.hasChildNodes();)this._recursiveDOMDelete(t.firstChild),t.removeChild(t.firstChild)},s.prototype._getPointer=function(t){return{x:t.x-a.getAbsoluteLeft(this.frame.canvas),y:t.y-a.getAbsoluteTop(this.frame.canvas)}},s.prototype._onTouch=function(t){(new Date).valueOf()-this.touchTime>100&&(this.drag.pointer=this._getPointer(t.center),this.drag.pinched=!1,this.pinch.scale=this._getScale(),this.touchTime=(new Date).valueOf(),this._handleTouch(this.drag.pointer))},s.prototype._onDragStart=function(t){this._handleDragStart(t)},s.prototype._handleDragStart=function(t){void 0===this.drag.pointer&&this._onTouch(t);var e=this._getNodeAt(this.drag.pointer);if(this.drag.dragging=!0,this.drag.selection=[],this.drag.translation=this._getTranslation(),this.drag.nodeId=null,this.draggingNodes=!1,null!=e&&1==this.constants.dragNodes){this.draggingNodes=!0,this.drag.nodeId=e.id,e.isSelected()||this._selectObject(e,!1),this.emit("dragStart",{nodeIds:this.getSelection().nodes});for(var i in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(i)){var s=this.selectionObj.nodes[i],o={id:s.id,node:s,x:s.x,y:s.y,xFixed:s.xFixed,yFixed:s.yFixed};s.xFixed=!0,s.yFixed=!0,this.drag.selection.push(o)}}t.preventDefault()},s.prototype._onDrag=function(t){this._handleOnDrag(t)},s.prototype._handleOnDrag=function(t){if(!this.drag.pinched){this.releaseNode();var e=this._getPointer(t.center),i=this,s=this.drag,o=s.selection;if(o&&o.length&&1==this.constants.dragNodes){var n=e.x-s.pointer.x,r=e.y-s.pointer.y;o.forEach(function(t){var e=t.node;t.xFixed||(e.x=i._XconvertDOMtoCanvas(i._XconvertCanvasToDOM(t.x)+n)),t.yFixed||(e.y=i._YconvertDOMtoCanvas(i._YconvertCanvasToDOM(t.y)+r))}),this.moving||(this.moving=!0,this.start())}else if(1==this.constants.dragNetwork){if(void 0===this.drag.pointer)return void this._handleDragStart(t);var a=e.x-this.drag.pointer.x,h=e.y-this.drag.pointer.y;this._setTranslation(this.drag.translation.x+a,this.drag.translation.y+h),this._redraw()}t.preventDefault()}},s.prototype._onDragEnd=function(t){this._handleDragEnd(t)},s.prototype._handleDragEnd=function(t){this.drag.dragging=!1;var e=this.drag.selection;e&&e.length?(e.forEach(function(t){t.node.xFixed=t.xFixed,t.node.yFixed=t.yFixed}),this.moving=!0,this.start()):this._redraw(),0==this.draggingNodes?this.emit("dragEnd",{nodeIds:[]}):this.emit("dragEnd",{nodeIds:this.getSelection().nodes}),t.preventDefault()},s.prototype._onTap=function(t){var e=this._getPointer(t.center);this.pointerPosition=e,this._handleTap(e)},s.prototype._onDoubleTap=function(t){var e=this._getPointer(t.center);this._handleDoubleTap(e)},s.prototype._onHold=function(t){var e=this._getPointer(t.center);this.pointerPosition=e,this._handleOnHold(e)},s.prototype._onRelease=function(t){var e=this._getPointer(t.center);this._handleOnRelease(e)},s.prototype._onPinch=function(t){var e=this._getPointer(t.center);this.drag.pinched=!0,"scale"in this.pinch||(this.pinch.scale=1);var i=this.pinch.scale*t.scale;this._zoom(i,e)},s.prototype._zoom=function(t,e){if(1==this.constants.zoomable){var i=this._getScale();1e-5>t&&(t=1e-5),t>10&&(t=10);var s=null;void 0!==this.drag&&1==this.drag.dragging&&(s=this.DOMtoCanvas(this.drag.pointer));var o=this._getTranslation(),n=t/i,r=(1-n)*e.x+o.x*n,a=(1-n)*e.y+o.y*n;if(this.areaCenter={x:this._XconvertDOMtoCanvas(e.x),y:this._YconvertDOMtoCanvas(e.y)},this._setScale(t),this._setTranslation(r,a),this.updateClustersDefault(),null!=s){var h=this.canvasToDOM(s);this.drag.pointer.x=h.x,this.drag.pointer.y=h.y}return this._redraw(),t>i?this.emit("zoom",{direction:"+"}):this.emit("zoom",{direction:"-"}),t}},s.prototype._onMouseWheel=function(t){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i=this._getScale(),s=e/10;0>e&&(s/=1-s),i*=1+s;var o=this._getPointer({x:t.pageX,y:t.pageY});this._zoom(i,o)}t.preventDefault()},s.prototype._onMouseMoveTitle=function(t){var e=this._getPointer({x:t.pageX,y:t.pageY});this.popupObj&&this._checkHidePopup(e);var i=this,s=function(){i._checkShowPopup(e)};if(this.popupTimer&&clearInterval(this.popupTimer),this.drag.dragging||(this.popupTimer=setTimeout(s,this.constants.tooltip.delay)),1==this.constants.hover){for(var o in this.hoverObj.edges)this.hoverObj.edges.hasOwnProperty(o)&&(this.hoverObj.edges[o].hover=!1,delete this.hoverObj.edges[o]);var n=this._getNodeAt(e);null==n&&(n=this._getEdgeAt(e)),null!=n&&this._hoverObject(n);for(var r in this.hoverObj.nodes)this.hoverObj.nodes.hasOwnProperty(r)&&(n instanceof f&&n.id!=r||n instanceof g||null==n)&&(this._blurObject(this.hoverObj.nodes[r]),delete this.hoverObj.nodes[r]);this.redraw()}},s.prototype._checkShowPopup=function(t){var e,i={left:this._XconvertDOMtoCanvas(t.x),top:this._YconvertDOMtoCanvas(t.y),right:this._XconvertDOMtoCanvas(t.x),bottom:this._YconvertDOMtoCanvas(t.y)},s=this.popupObj,o=!1;if(void 0==this.popupObj){var n=this.nodes,r=[];for(e in n)if(n.hasOwnProperty(e)){var a=n[e];a.isOverlappingWith(i)&&void 0!==a.getTitle()&&r.push(e)}r.length>0&&(this.popupObj=this.nodes[r[r.length-1]],o=!0)}if(void 0===this.popupObj&&0==o){var h=this.edges,d=[];for(e in h)if(h.hasOwnProperty(e)){var l=h[e];l.connected&&void 0!==l.getTitle()&&l.isOverlappingWith(i)&&d.push(e)}d.length>0&&(this.popupObj=this.edges[d[d.length-1]])}if(this.popupObj){if(this.popupObj!=s){var c=this;c.popup||(c.popup=new v(c.frame,c.constants.tooltip)),c.popup.setPosition(t.x-3,t.y-3),c.popup.setText(c.popupObj.getTitle()),c.popup.show()}}else this.popup&&this.popup.hide()},s.prototype._checkHidePopup=function(t){this.popupObj&&this._getNodeAt(t)||(this.popupObj=void 0,this.popup&&this.popup.hide())},s.prototype.setSize=function(t,e){var i=!1,s=this.frame.canvas.width,o=this.frame.canvas.height;t!=this.constants.width||e!=this.constants.height||this.frame.style.width!=t||this.frame.style.height!=e?(this.frame.style.width=t,this.frame.style.height=e,this.frame.canvas.style.width="100%",this.frame.canvas.style.height="100%",this.frame.canvas.width=this.frame.canvas.clientWidth*this.pixelRatio,this.frame.canvas.height=this.frame.canvas.clientHeight*this.pixelRatio,this.constants.width=t,this.constants.height=e,i=!0):(this.frame.canvas.width!=this.frame.canvas.clientWidth*this.pixelRatio&&(this.frame.canvas.width=this.frame.canvas.clientWidth*this.pixelRatio,i=!0),this.frame.canvas.height!=this.frame.canvas.clientHeight*this.pixelRatio&&(this.frame.canvas.height=this.frame.canvas.clientHeight*this.pixelRatio,i=!0)),1==i&&this.emit("resize",{width:this.frame.canvas.width*this.pixelRatio,height:this.frame.canvas.height*this.pixelRatio,oldWidth:s*this.pixelRatio,oldHeight:o*this.pixelRatio})},s.prototype._setNodes=function(t){var e=this.nodesData;if(t instanceof d||t instanceof l)this.nodesData=t;else if(Array.isArray(t))this.nodesData=new d,this.nodesData.add(t);else{if(t)throw new TypeError("Array or DataSet expected");this.nodesData=new d}if(e&&a.forEach(this.nodesListeners,function(t,i){e.off(i,t)}),this.nodes={},this.nodesData){var i=this;a.forEach(this.nodesListeners,function(t,e){i.nodesData.on(e,t)});var s=this.nodesData.getIds();this._addNodes(s)}this._updateSelection()},s.prototype._addNodes=function(t){for(var e,i=0,s=t.length;s>i;i++){e=t[i];var o=this.nodesData.get(e),n=new f(o,this.images,this.groups,this.constants);if(this.nodes[e]=n,!(0!=n.xFixed&&0!=n.yFixed||null!==n.x&&null!==n.y)){var r=1*t.length+10,a=2*Math.PI*Math.random();0==n.xFixed&&(n.x=r*Math.cos(a)),0==n.yFixed&&(n.y=r*Math.sin(a))}this.moving=!0}this._updateNodeIndexList(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes(),this._reconnectEdges(),this._updateValueRange(this.nodes),this.updateLabels()},s.prototype._updateNodes=function(t,e){for(var i=this.nodes,s=0,o=t.length;o>s;s++){var n=t[s],r=i[n],a=e[s];r?r.setProperties(a,this.constants):(r=new f(properties,this.images,this.groups,this.constants),i[n]=r)}this.moving=!0,1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateNodeIndexList(),this._updateValueRange(i)},s.prototype._removeNodes=function(t){for(var e=this.nodes,i=0,s=t.length;s>i;i++){var o=t[i];delete e[o]}this._updateNodeIndexList(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes(),this._reconnectEdges(),this._updateSelection(),this._updateValueRange(e)},s.prototype._setEdges=function(t){var e=this.edgesData;if(t instanceof d||t instanceof l)this.edgesData=t;else if(Array.isArray(t))this.edgesData=new d,this.edgesData.add(t);else{if(t)throw new TypeError("Array or DataSet expected");this.edgesData=new d}if(e&&a.forEach(this.edgesListeners,function(t,i){e.off(i,t)}),this.edges={},this.edgesData){var i=this;a.forEach(this.edgesListeners,function(t,e){i.edgesData.on(e,t)});var s=this.edgesData.getIds();this._addEdges(s)}this._reconnectEdges()},s.prototype._addEdges=function(t){for(var e=this.edges,i=this.edgesData,s=0,o=t.length;o>s;s++){var n=t[s],r=e[n];r&&r.disconnect();var a=i.get(n,{showInternalIds:!0});e[n]=new g(a,this,this.constants)}this.moving=!0,this._updateValueRange(e),this._createBezierNodes(),this._updateCalculationNodes(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout())},s.prototype._updateEdges=function(t){for(var e=this.edges,i=this.edgesData,s=0,o=t.length;o>s;s++){var n=t[s],r=i.get(n),a=e[n];a?(a.disconnect(),a.setProperties(r,this.constants),a.connect()):(a=new g(r,this,this.constants),this.edges[n]=a)}this._createBezierNodes(),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this.moving=!0,this._updateValueRange(e)},s.prototype._removeEdges=function(t){for(var e=this.edges,i=0,s=t.length;s>i;i++){var o=t[i],n=e[o];n&&(null!=n.via&&delete this.sectors.support.nodes[n.via.id],n.disconnect(),delete e[o])}this.moving=!0,this._updateValueRange(e),1==this.constants.hierarchicalLayout.enabled&&0==this.initializing&&(this._resetLevels(),this._setupHierarchicalLayout()),this._updateCalculationNodes()},s.prototype._reconnectEdges=function(){var t,e=this.nodes,i=this.edges;for(t in e)e.hasOwnProperty(t)&&(e[t].edges=[],e[t].dynamicEdges=[]);for(t in i)if(i.hasOwnProperty(t)){var s=i[t];s.from=null,s.to=null,s.connect()}},s.prototype._updateValueRange=function(t){var e,i=void 0,s=void 0;for(e in t)if(t.hasOwnProperty(e)){var o=t[e].getValue();void 0!==o&&(i=void 0===i?o:Math.min(o,i),s=void 0===s?o:Math.max(o,s))}if(void 0!==i&&void 0!==s)for(e in t)t.hasOwnProperty(e)&&t[e].setValueRange(i,s)},s.prototype.redraw=function(){this.setSize(this.constants.width,this.constants.height),this._redraw()},s.prototype._redraw=function(t){var e=this.frame.canvas.getContext("2d");e.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0);var i=this.frame.canvas.width*this.pixelRatio,s=this.frame.canvas.height*this.pixelRatio;e.clearRect(0,0,i,s),e.save(),e.translate(this.translation.x,this.translation.y),e.scale(this.scale,this.scale),this.canvasTopLeft={x:this._XconvertDOMtoCanvas(0),y:this._YconvertDOMtoCanvas(0)},this.canvasBottomRight={x:this._XconvertDOMtoCanvas(this.frame.canvas.clientWidth*this.pixelRatio),y:this._YconvertDOMtoCanvas(this.frame.canvas.clientHeight*this.pixelRatio)},1!=t&&(this._doInAllSectors("_drawAllSectorNodes",e),(0==this.drag.dragging||void 0===this.drag.dragging||0==this.constants.hideEdgesOnDrag)&&this._doInAllSectors("_drawEdges",e)),(0==this.drag.dragging||void 0===this.drag.dragging||0==this.constants.hideNodesOnDrag)&&this._doInAllSectors("_drawNodes",e,!1),1!=t&&1==this.controlNodesActive&&this._doInAllSectors("_drawControlNodes",e),e.restore(),1==t&&e.clearRect(0,0,i,s)},s.prototype._setTranslation=function(t,e){void 0===this.translation&&(this.translation={x:0,y:0}),void 0!==t&&(this.translation.x=t),void 0!==e&&(this.translation.y=e),this.emit("viewChanged")},s.prototype._getTranslation=function(){return{x:this.translation.x,y:this.translation.y}},s.prototype._setScale=function(t){this.scale=t},s.prototype._getScale=function(){return this.scale},s.prototype._XconvertDOMtoCanvas=function(t){return(t-this.translation.x)/this.scale},s.prototype._XconvertCanvasToDOM=function(t){return t*this.scale+this.translation.x},s.prototype._YconvertDOMtoCanvas=function(t){return(t-this.translation.y)/this.scale},s.prototype._YconvertCanvasToDOM=function(t){return t*this.scale+this.translation.y},s.prototype.canvasToDOM=function(t){return{x:this._XconvertCanvasToDOM(t.x),y:this._YconvertCanvasToDOM(t.y)}},s.prototype.DOMtoCanvas=function(t){return{x:this._XconvertDOMtoCanvas(t.x),y:this._YconvertDOMtoCanvas(t.y)}},s.prototype._drawNodes=function(t,e){void 0===e&&(e=!1);var i=this.nodes,s=[];for(var o in i)i.hasOwnProperty(o)&&(i[o].setScaleAndPos(this.scale,this.canvasTopLeft,this.canvasBottomRight),i[o].isSelected()?s.push(o):(i[o].inArea()||e)&&i[o].draw(t));for(var n=0,r=s.length;r>n;n++)(i[s[n]].inArea()||e)&&i[s[n]].draw(t)},s.prototype._drawEdges=function(t){var e=this.edges;for(var i in e)if(e.hasOwnProperty(i)){var s=e[i];s.setScale(this.scale),s.connected&&e[i].draw(t)}},s.prototype._drawControlNodes=function(t){var e=this.edges;for(var i in e)e.hasOwnProperty(i)&&e[i]._drawControlNodes(t)},s.prototype._stabilize=function(){1==this.constants.freezeForStabilization&&this._freezeDefinedNodes();for(var t=0;this.moving&&t0)for(t in i)i.hasOwnProperty(t)&&(i[t].discreteStepLimited(e,this.constants.maxVelocity),s=!0);else for(t in i)i.hasOwnProperty(t)&&(i[t].discreteStep(e),s=!0);if(1==s){var o=this.constants.minVelocity/Math.max(this.scale,.05);return o>.5*this.constants.maxVelocity?!0:this._isMoving(o)}return!1},s.prototype._revertPhysicsState=function(){var t=this.nodes;for(var e in t)t.hasOwnProperty(e)&&t[e].revertPosition()},s.prototype._revertPhysicsTick=function(){this._doInAllActiveSectors("_revertPhysicsState"),1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic&&this._doInSupportSector("_revertPhysicsState")},s.prototype._physicsTick=function(){if(!this.freezeSimulation&&1==this.moving){var t=!1,e=!1;this._doInAllActiveSectors("_initializeForceCalculation");var i=this._doInAllActiveSectors("_discreteStepNodes");1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic&&(e=this._doInSupportSector("_discreteStepNodes"));for(var s=0;s2*e||1==this.runDoubleSpeed)&&1==this.moving&&(this._physicsTick(),0!=this.renderTime&&(this.runDoubleSpeed=!0));var i=Date.now();this._redraw(),this.renderTime=Date.now()-i,this.start()},"undefined"!=typeof window&&(window.requestAnimationFrame=window.requestAnimationFrame||window.mozRequestAnimationFrame||window.webkitRequestAnimationFrame||window.msRequestAnimationFrame),s.prototype.start=function(){if(1==this.moving||0!=this.xIncrement||0!=this.yIncrement||0!=this.zoomIncrement)this.timer||(this.timer=1==this.requiresTimeout?window.setTimeout(this._animationStep.bind(this),this.renderTimestep):window.requestAnimationFrame(this._animationStep.bind(this)));else if(this._redraw(),this.stabilizationIterations>1){var t=this,e={iterations:t.stabilizationIterations};this.stabilizationIterations=0,this.startedStabilization=!1,setTimeout(function(){t.emit("stabilized",e)},0)}else this.stabilizationIterations=0},s.prototype._handleNavigation=function(){if(0!=this.xIncrement||0!=this.yIncrement){var t=this._getTranslation();this._setTranslation(t.x+this.xIncrement,t.y+this.yIncrement)}if(0!=this.zoomIncrement){var e={x:this.frame.canvas.clientWidth/2,y:this.frame.canvas.clientHeight/2};this._zoom(this.scale*(1+this.zoomIncrement),e)}},s.prototype.toggleFreeze=function(){0==this.freezeSimulation?this.freezeSimulation=!0:(this.freezeSimulation=!1,this.start())},s.prototype._configureSmoothCurves=function(t){if(void 0===t&&(t=!0),1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic){this._createBezierNodes();for(var e in this.sectors.support.nodes)this.sectors.support.nodes.hasOwnProperty(e)&&void 0===this.edges[this.sectors.support.nodes[e].parentEdgeId]&&delete this.sectors.support.nodes[e]}else{this.sectors.support.nodes={};for(var i in this.edges)this.edges.hasOwnProperty(i)&&(this.edges[i].via=null)}this._updateCalculationNodes(),t||(this.moving=!0,this.start())},s.prototype._createBezierNodes=function(){if(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic)for(var t in this.edges)if(this.edges.hasOwnProperty(t)){var e=this.edges[t];if(null==e.via){var i="edgeId:".concat(e.id);this.sectors.support.nodes[i]=new f({id:i,mass:1,shape:"circle",image:"",internalMultiplier:1},{},{},this.constants),e.via=this.sectors.support.nodes[i],e.via.parentEdgeId=e.id,e.positionBezierNode()}}},s.prototype._initializeMixinLoaders=function(){for(var t in y)y.hasOwnProperty(t)&&(s.prototype[t]=y[t])},s.prototype.storePosition=function(){console.log("storePosition is deprecated: use .storePositions() from now on."),this.storePositions()},s.prototype.storePositions=function(){var t=[];for(var e in this.nodes)if(this.nodes.hasOwnProperty(e)){var i=this.nodes[e],s=!this.nodes.xFixed,o=!this.nodes.yFixed;(this.nodesData._data[e].x!=Math.round(i.x)||this.nodesData._data[e].y!=Math.round(i.y))&&t.push({id:e,x:Math.round(i.x),y:Math.round(i.y),allowedToMoveX:s,allowedToMoveY:o})}this.nodesData.update(t)},s.prototype.getPositions=function(t){var e={};if(void 0!==t){if(1==Array.isArray(t)){for(var i=0;i=1&&(this.easingTime=0,this._redraw=null!=this.lockedOnNodeId?this._lockedRedraw:this._classicRedraw,this.emit("animationFinished"))},s.prototype._classicRedraw=function(){},s.prototype.isActive=function(){return!this.activator||this.activator.active},s.prototype.setScale=function(){return this._setScale()},s.prototype.getScale=function(){return this._getScale()},s.prototype.getCenterCoordinates=function(){return this.DOMtoCanvas({x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight})},s.prototype.getBoundingBox=function(t){return void 0!==this.nodes[t]?this.nodes[t].boundingBox:void 0},t.exports=s},function(t,e,i){function s(t,e,i){if(!e)throw"No network provided";var s=["edges","physics"],n=o.selectiveBridgeObject(s,i);this.options=n.edges,this.physics=n.physics,this.options.smoothCurves=i.smoothCurves,this.network=e,this.id=void 0,this.fromId=void 0,this.toId=void 0,this.title=void 0,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier,this.value=void 0,this.selected=!1,this.hover=!1,this.labelDimensions={top:0,left:0,width:0,height:0,yLine:0},this.dirtyLabel=!0,this.from=null,this.to=null,this.via=null,this.fromBackup=null,this.toBackup=null,this.originalFromId=[],this.originalToId=[],this.connected=!1,this.widthFixed=!1,this.lengthFixed=!1,this.setProperties(t),this.controlNodesEnabled=!1,this.controlNodes={from:null,to:null,positions:{}},this.connectedNode=null}var o=i(1),n=i(40);s.prototype.setProperties=function(t){if(t){var e=["style","fontSize","fontFace","fontColor","fontFill","fontStrokeWidth","fontStrokeColor","width","widthSelectionMultiplier","hoverWidth","arrowScaleFactor","dash","inheritColor","labelAlignment"];switch(o.selectiveDeepExtend(e,this.options,t),void 0!==t.from&&(this.fromId=t.from),void 0!==t.to&&(this.toId=t.to),void 0!==t.id&&(this.id=t.id),void 0!==t.label&&(this.label=t.label,this.dirtyLabel=!0),void 0!==t.title&&(this.title=t.title),void 0!==t.value&&(this.value=t.value),void 0!==t.length&&(this.physics.springLength=t.length),void 0!==t.color&&(this.options.inheritColor=!1,o.isString(t.color)?(this.options.color.color=t.color,this.options.color.highlight=t.color):(void 0!==t.color.color&&(this.options.color.color=t.color.color),void 0!==t.color.highlight&&(this.options.color.highlight=t.color.highlight),void 0!==t.color.hover&&(this.options.color.hover=t.color.hover))),this.connect(),this.widthFixed=this.widthFixed||void 0!==t.width,this.lengthFixed=this.lengthFixed||void 0!==t.length,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier,this.options.style){case"line":this.draw=this._drawLine;break;case"arrow":this.draw=this._drawArrow;break;case"arrow-center":this.draw=this._drawArrowCenter;break;case"dash-line":this.draw=this._drawDashLine;break;default:this.draw=this._drawLine}}},s.prototype.connect=function(){this.disconnect(),this.from=this.network.nodes[this.fromId]||null,this.to=this.network.nodes[this.toId]||null,this.connected=this.from&&this.to,this.connected?(this.from.attachEdge(this),this.to.attachEdge(this)):(this.from&&this.from.detachEdge(this),this.to&&this.to.detachEdge(this))},s.prototype.disconnect=function(){this.from&&(this.from.detachEdge(this),this.from=null),this.to&&(this.to.detachEdge(this),this.to=null),this.connected=!1},s.prototype.getTitle=function(){return"function"==typeof this.title?this.title():this.title -},s.prototype.getValue=function(){return this.value},s.prototype.setValueRange=function(t,e){if(!this.widthFixed&&void 0!==this.value){var i=(this.options.widthMax-this.options.widthMin)/(e-t);this.options.width=(this.value-t)*i+this.options.widthMin,this.widthSelected=this.options.width*this.options.widthSelectionMultiplier}},s.prototype.draw=function(){throw"Method draw not initialized in edge"},s.prototype.isOverlappingWith=function(t){if(this.connected){var e=10,i=this.from.x,s=this.from.y,o=this.to.x,n=this.to.y,r=t.left,a=t.top,h=this._getDistanceToEdge(i,s,o,n,r,a);return e>h}return!1},s.prototype._getColor=function(){var t=this.options.color;return"to"==this.options.inheritColor?t={highlight:this.to.options.color.highlight.border,hover:this.to.options.color.hover.border,color:this.to.options.color.border}:("from"==this.options.inheritColor||1==this.options.inheritColor)&&(t={highlight:this.from.options.color.highlight.border,hover:this.from.options.color.hover.border,color:this.from.options.color.border}),1==this.selected?t.highlight:1==this.hover?t.hover:t.color},s.prototype._drawLine=function(t){if(t.strokeStyle=this._getColor(),t.lineWidth=this._getLineWidth(),this.from!=this.to){var e,i=this._line(t);if(this.label){if(1==this.options.smoothCurves.enabled&&null!=i){var s=.5*(.5*(this.from.x+i.x)+.5*(this.to.x+i.x)),o=.5*(.5*(this.from.y+i.y)+.5*(this.to.y+i.y));e={x:s,y:o}}else e=this._pointOnLine(.5);this._label(t,this.label,e.x,e.y)}}else{var n,r,a=this.physics.springLength/4,h=this.from;h.width||h.resize(t),h.width>h.height?(n=h.x+h.width/2,r=h.y-a):(n=h.x+a,r=h.y-h.height/2),this._circle(t,n,r,a),e=this._pointOnCircle(n,r,a,.5),this._label(t,this.label,e.x,e.y)}},s.prototype._getLineWidth=function(){return 1==this.selected?Math.max(Math.min(this.widthSelected,this.options.widthMax),.3*this.networkScaleInv):1==this.hover?Math.max(Math.min(this.options.hoverWidth,this.options.widthMax),.3*this.networkScaleInv):Math.max(this.options.width,.3*this.networkScaleInv)},s.prototype._getViaCoordinates=function(){if(1==this.options.smoothCurves.dynamic&&1==this.options.smoothCurves.enabled)return this.via;if(0==this.options.smoothCurves.enabled)return{x:0,y:0};var t=null,e=null,i=this.options.smoothCurves.roundness,s=this.options.smoothCurves.type,o=Math.abs(this.from.x-this.to.x),n=Math.abs(this.from.y-this.to.y);return"discrete"==s||"diagonalCross"==s?Math.abs(this.from.x-this.to.x)this.to.y?this.from.xthis.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n)),"discrete"==s&&(t=i*n>o?this.from.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>this.to.y?this.from.xthis.to.x&&(t=this.from.x-i*o,e=this.from.y-i*o):this.from.ythis.to.x&&(t=this.from.x-i*o,e=this.from.y+i*o)),"discrete"==s&&(e=i*o>n?this.from.y:e)):"straightCross"==s?Math.abs(this.from.x-this.to.x)Math.abs(this.from.y-this.to.y)&&(t=this.from.xthis.to.y?this.from.xthis.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n,t=this.to.x>t?this.to.x:t):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n,t=this.to.x>t?this.to.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>this.to.y?this.from.xe?this.to.y:e):this.from.x>this.to.x&&(t=this.from.x-i*o,e=this.from.y-i*o,e=this.to.y>e?this.to.y:e):this.from.ythis.to.x&&(t=this.from.x-i*o,e=this.from.y+i*o,e=this.to.yd;d++){var l=t.measureText(n[d]).width;h=l>h?l:h}var c=this.options.fontSize*r,p=i-h/2,u=s-c/2;this.labelDimensions={top:u,left:p,width:h,height:c,yLine:o}}var o=this.labelDimensions.yLine;t.save(),"horizontal"!=this.options.labelAlignment&&(t.translate(i,o),this._rotateForLabelAlignment(t),i=0,o=0),this._drawLabelRect(t),this._drawLabelText(t,i,o,n,r,a),t.restore()}},s.prototype._rotateForLabelAlignment=function(t){var e=this.from.y-this.to.y,i=this.from.x-this.to.x,s=Math.atan2(e,i);(-1>s&&0>i||s>0&&0>i)&&(s+=Math.PI),t.rotate(s)},s.prototype._drawLabelRect=function(t){if(void 0!==this.options.fontFill&&null!==this.options.fontFill&&"none"!==this.options.fontFill){t.fillStyle=this.options.fontFill;var e=2;"line-center"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,.5*-this.labelDimensions.height,this.labelDimensions.width,this.labelDimensions.height):"line-above"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,-(this.labelDimensions.height+e),this.labelDimensions.width,this.labelDimensions.height):"line-below"==this.options.labelAlignment?t.fillRect(.5*-this.labelDimensions.width,e,this.labelDimensions.width,this.labelDimensions.height):t.fillRect(this.labelDimensions.left,this.labelDimensions.top,this.labelDimensions.width,this.labelDimensions.height)}},s.prototype._drawLabelText=function(t,e,i,s,o,n){if(t.fillStyle=this.options.fontColor||"black",t.textAlign="center","horizontal"!=this.options.labelAlignment){var r=2;"line-above"==this.options.labelAlignment?(t.textBaseline="alphabetic",i-=2*r):"line-below"==this.options.labelAlignment?(t.textBaseline="hanging",i+=2*r):t.textBaseline="middle"}else t.textBaseline="middle";this.options.fontStrokeWidth>0&&(t.lineWidth=this.options.fontStrokeWidth,t.strokeStyle=this.options.fontStrokeColor,t.lineJoin="round");for(var a=0;o>a;a++)this.options.fontStrokeWidth>0&&t.strokeText(s[a],e,i),t.fillText(s[a],e,i),i+=n},s.prototype._drawDashLine=function(t){t.strokeStyle=this._getColor(),t.lineWidth=this._getLineWidth();var e=null;if(void 0!==t.setLineDash){t.save();var i=[0];i=void 0!==this.options.dash.length&&void 0!==this.options.dash.gap?[this.options.dash.length,this.options.dash.gap]:[5,5],t.setLineDash(i),t.lineDashOffset=0,e=this._line(t),t.setLineDash([0]),t.lineDashOffset=0,t.restore()}else t.beginPath(),t.lineCap="round",void 0!==this.options.dash.altLength?t.dashedLine(this.from.x,this.from.y,this.to.x,this.to.y,[this.options.dash.length,this.options.dash.gap,this.options.dash.altLength,this.options.dash.gap]):void 0!==this.options.dash.length&&void 0!==this.options.dash.gap?t.dashedLine(this.from.x,this.from.y,this.to.x,this.to.y,[this.options.dash.length,this.options.dash.gap]):(t.moveTo(this.from.x,this.from.y),t.lineTo(this.to.x,this.to.y)),t.stroke();if(this.label){var s;if(1==this.options.smoothCurves.enabled&&null!=e){var o=.5*(.5*(this.from.x+e.x)+.5*(this.to.x+e.x)),n=.5*(.5*(this.from.y+e.y)+.5*(this.to.y+e.y));s={x:o,y:n}}else s=this._pointOnLine(.5);this._label(t,this.label,s.x,s.y)}},s.prototype._pointOnLine=function(t){return{x:(1-t)*this.from.x+t*this.to.x,y:(1-t)*this.from.y+t*this.to.y}},s.prototype._pointOnCircle=function(t,e,i,s){var o=2*(s-3/8)*Math.PI;return{x:t+i*Math.cos(o),y:e-i*Math.sin(o)}},s.prototype._drawArrowCenter=function(t){var e;if(t.strokeStyle=this._getColor(),t.fillStyle=t.strokeStyle,t.lineWidth=this._getLineWidth(),this.from!=this.to){var i=this._line(t),s=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),o=(10+5*this.options.width)*this.options.arrowScaleFactor;if(1==this.options.smoothCurves.enabled&&null!=i){var n=.5*(.5*(this.from.x+i.x)+.5*(this.to.x+i.x)),r=.5*(.5*(this.from.y+i.y)+.5*(this.to.y+i.y));e={x:n,y:r}}else e=this._pointOnLine(.5);t.arrow(e.x,e.y,s,o),t.fill(),t.stroke(),this.label&&this._label(t,this.label,e.x,e.y)}else{var a,h,d=.25*Math.max(100,this.physics.springLength),l=this.from;l.width||l.resize(t),l.width>l.height?(a=l.x+.5*l.width,h=l.y-d):(a=l.x+d,h=l.y-.5*l.height),this._circle(t,a,h,d);var s=.2*Math.PI,o=(10+5*this.options.width)*this.options.arrowScaleFactor;e=this._pointOnCircle(a,h,d,.5),t.arrow(e.x,e.y,s,o),t.fill(),t.stroke(),this.label&&(e=this._pointOnCircle(a,h,d,.5),this._label(t,this.label,e.x,e.y))}},s.prototype._pointOnBezier=function(t){var e=this._getViaCoordinates(),i=Math.pow(1-t,2)*this.from.x+2*t*(1-t)*e.x+Math.pow(t,2)*this.to.x,s=Math.pow(1-t,2)*this.from.y+2*t*(1-t)*e.y+Math.pow(t,2)*this.to.y;return{x:i,y:s}},s.prototype._findBorderPosition=function(t,e){var i,s,o,n,r,a=10,h=0,d=0,l=1,c=.2,p=this.to;for(1==t&&(p=this.from);l>=d&&a>h;){var u=.5*(d+l);if(i=this._pointOnBezier(u),s=Math.atan2(p.y-i.y,p.x-i.x),o=p.distanceToBorder(e,s),n=Math.sqrt(Math.pow(i.x-p.x,2)+Math.pow(i.y-p.y,2)),r=o-n,Math.abs(r)r?0==t?d=u:l=u:0==t?l=u:d=u,h++}return i.t=u,i},s.prototype._drawArrow=function(t){t.strokeStyle=this._getColor(),t.fillStyle=t.strokeStyle,t.lineWidth=this._getLineWidth();var e,i,s;if(this.from!=this.to){if(this._line(t),1==this.options.smoothCurves.enabled){var o=this._getViaCoordinates();s=this._findBorderPosition(!1,t);var n=this._pointOnBezier(Math.max(0,s.t-.1));e=Math.atan2(s.y-n.y,s.x-n.x)}else{e=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x);var r=this.to.x-this.from.x,a=this.to.y-this.from.y,h=Math.sqrt(r*r+a*a),d=this.to.distanceToBorder(t,e),l=(h-d)/h;s={},s.x=(1-l)*this.from.x+l*this.to.x,s.y=(1-l)*this.from.y+l*this.to.y}if(i=(10+5*this.options.width)*this.options.arrowScaleFactor,t.arrow(s.x,s.y,e,i),t.fill(),t.stroke(),this.label){var c;c=1==this.options.smoothCurves.enabled&&null!=o?this._pointOnBezier(.5):this._pointOnLine(.5),this._label(t,this.label,c.x,c.y)}}else{var p,u,m,f=this.from,g=.25*Math.max(100,this.physics.springLength);f.width||f.resize(t),f.width>f.height?(p=f.x+.5*f.width,u=f.y-g,m={x:p,y:f.y,angle:.9*Math.PI}):(p=f.x+g,u=f.y-.5*f.height,m={x:f.x,y:u,angle:.6*Math.PI}),t.beginPath(),t.arc(p,u,g,0,2*Math.PI,!1),t.stroke();var i=(10+5*this.options.width)*this.options.arrowScaleFactor;t.arrow(m.x,m.y,m.angle,i),t.fill(),t.stroke(),this.label&&(c=this._pointOnCircle(p,u,g,.5),this._label(t,this.label,c.x,c.y))}},s.prototype._getDistanceToEdge=function(t,e,i,s,o,n){var r=0;if(this.from!=this.to)if(1==this.options.smoothCurves.enabled){var a,h;if(1==this.options.smoothCurves.enabled&&1==this.options.smoothCurves.dynamic)a=this.via.x,h=this.via.y;else{var d=this._getViaCoordinates();a=d.x,h=d.y}var l,c,p,u,m,f,g,v=1e9;for(c=0;10>c;c++)p=.1*c,u=Math.pow(1-p,2)*t+2*p*(1-p)*a+Math.pow(p,2)*i,m=Math.pow(1-p,2)*e+2*p*(1-p)*h+Math.pow(p,2)*s,c>0&&(l=this._getDistanceToLine(f,g,u,m,o,n),v=v>l?l:v),f=u,g=m;r=v}else r=this._getDistanceToLine(t,e,i,s,o,n);else{var u,m,y,b,_=.25*this.physics.springLength,x=this.from;x.width>x.height?(u=x.x+.5*x.width,m=x.y-_):(u=x.x+_,m=x.y-.5*x.height),y=u-o,b=m-n,r=Math.abs(Math.sqrt(y*y+b*b)-_)}return this.labelDimensions.lefto&&this.labelDimensions.topn?0:r},s.prototype._getDistanceToLine=function(t,e,i,s,o,n){var r=i-t,a=s-e,h=r*r+a*a,d=((o-t)*r+(n-e)*a)/h;d>1?d=1:0>d&&(d=0);var l=t+d*r,c=e+d*a,p=l-o,u=c-n;return Math.sqrt(p*p+u*u)},s.prototype.setScale=function(t){this.networkScaleInv=1/t},s.prototype.select=function(){this.selected=!0},s.prototype.unselect=function(){this.selected=!1},s.prototype.positionBezierNode=function(){null!==this.via&&null!==this.from&&null!==this.to?(this.via.x=.5*(this.from.x+this.to.x),this.via.y=.5*(this.from.y+this.to.y)):(this.via.x=0,this.via.y=0)},s.prototype._drawControlNodes=function(t){if(1==this.controlNodesEnabled){if(null===this.controlNodes.from&&null===this.controlNodes.to){var e="edgeIdFrom:".concat(this.id),i="edgeIdTo:".concat(this.id),s={nodes:{group:"",radius:7,borderWidth:2,borderWidthSelected:2},physics:{damping:0},clustering:{maxNodeSizeIncrements:0,nodeScaling:{width:0,height:0,radius:0}}};this.controlNodes.from=new n({id:e,shape:"dot",color:{background:"#ff0000",border:"#3c3c3c",highlight:{background:"#07f968"}}},{},{},s),this.controlNodes.to=new n({id:i,shape:"dot",color:{background:"#ff0000",border:"#3c3c3c",highlight:{background:"#07f968"}}},{},{},s)}this.controlNodes.positions={},0==this.controlNodes.from.selected&&(this.controlNodes.positions.from=this.getControlNodeFromPosition(t),this.controlNodes.from.x=this.controlNodes.positions.from.x,this.controlNodes.from.y=this.controlNodes.positions.from.y),0==this.controlNodes.to.selected&&(this.controlNodes.positions.to=this.getControlNodeToPosition(t),this.controlNodes.to.x=this.controlNodes.positions.to.x,this.controlNodes.to.y=this.controlNodes.positions.to.y),this.controlNodes.from.draw(t),this.controlNodes.to.draw(t)}else this.controlNodes={from:null,to:null,positions:{}}},s.prototype._enableControlNodes=function(){this.fromBackup=this.from,this.toBackup=this.to,this.controlNodesEnabled=!0},s.prototype._disableControlNodes=function(){this.fromId=this.from.id,this.toId=this.to.id,this.fromId!=this.fromBackup.id?this.fromBackup.detachEdge(this):this.toId!=this.toBackup.id&&this.toBackup.detachEdge(this),this.fromBackup=null,this.toBackup=null,this.controlNodesEnabled=!1},s.prototype._getSelectedControlNode=function(t,e){var i=this.controlNodes.positions,s=Math.sqrt(Math.pow(t-i.from.x,2)+Math.pow(e-i.from.y,2)),o=Math.sqrt(Math.pow(t-i.to.x,2)+Math.pow(e-i.to.y,2));return 15>s?(this.connectedNode=this.from,this.from=this.controlNodes.from,this.controlNodes.from):15>o?(this.connectedNode=this.to,this.to=this.controlNodes.to,this.controlNodes.to):null},s.prototype._restoreControlNodes=function(){1==this.controlNodes.from.selected?(this.from=this.connectedNode,this.connectedNode=null,this.controlNodes.from.unselect()):1==this.controlNodes.to.selected&&(this.to=this.connectedNode,this.connectedNode=null,this.controlNodes.to.unselect())},s.prototype.getControlNodeFromPosition=function(t){var e;if(1==this.options.smoothCurves.enabled)e=this._findBorderPosition(!0,t);else{var i=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),s=this.to.x-this.from.x,o=this.to.y-this.from.y,n=Math.sqrt(s*s+o*o),r=this.from.distanceToBorder(t,i+Math.PI),a=(n-r)/n;e={},e.x=a*this.from.x+(1-a)*this.to.x,e.y=a*this.from.y+(1-a)*this.to.y}return e},s.prototype.getControlNodeToPosition=function(t){var e;if(1==this.options.smoothCurves.enabled)e=this._findBorderPosition(!1,t);else{var i=Math.atan2(this.to.y-this.from.y,this.to.x-this.from.x),s=this.to.x-this.from.x,o=this.to.y-this.from.y,n=Math.sqrt(s*s+o*o),r=this.to.distanceToBorder(t,i),a=(n-r)/n;e={},e.x=(1-a)*this.from.x+a*this.to.x,e.y=(1-a)*this.from.y+a*this.to.y}return e},t.exports=s},function(t,e,i){function s(){this.clear(),this.defaultIndex=0}i(1);s.DEFAULT=[{border:"#2B7CE9",background:"#97C2FC",highlight:{border:"#2B7CE9",background:"#D2E5FF"},hover:{border:"#2B7CE9",background:"#D2E5FF"}},{border:"#FFA500",background:"#FFFF00",highlight:{border:"#FFA500",background:"#FFFFA3"},hover:{border:"#FFA500",background:"#FFFFA3"}},{border:"#FA0A10",background:"#FB7E81",highlight:{border:"#FA0A10",background:"#FFAFB1"},hover:{border:"#FA0A10",background:"#FFAFB1"}},{border:"#41A906",background:"#7BE141",highlight:{border:"#41A906",background:"#A1EC76"},hover:{border:"#41A906",background:"#A1EC76"}},{border:"#E129F0",background:"#EB7DF4",highlight:{border:"#E129F0",background:"#F0B3F5"},hover:{border:"#E129F0",background:"#F0B3F5"}},{border:"#7C29F0",background:"#AD85E4",highlight:{border:"#7C29F0",background:"#D3BDF0"},hover:{border:"#7C29F0",background:"#D3BDF0"}},{border:"#C37F00",background:"#FFA807",highlight:{border:"#C37F00",background:"#FFCA66"},hover:{border:"#C37F00",background:"#FFCA66"}},{border:"#4220FB",background:"#6E6EFD",highlight:{border:"#4220FB",background:"#9B9BFD"},hover:{border:"#4220FB",background:"#9B9BFD"}},{border:"#FD5A77",background:"#FFC0CB",highlight:{border:"#FD5A77",background:"#FFD1D9"},hover:{border:"#FD5A77",background:"#FFD1D9"}},{border:"#4AD63A",background:"#C2FABC",highlight:{border:"#4AD63A",background:"#E6FFE3"},hover:{border:"#4AD63A",background:"#E6FFE3"}}],s.prototype.clear=function(){this.groups={},this.groups.length=function(){var t=0;for(var e in this)this.hasOwnProperty(e)&&t++;return t}},s.prototype.get=function(t){var e=this.groups[t];if(void 0==e){var i=this.defaultIndex%s.DEFAULT.length;this.defaultIndex++,e={},e.color=s.DEFAULT[i],this.groups[t]=e}return e},s.prototype.add=function(t,e){return this.groups[t]=e,e},t.exports=s},function(t){function e(){this.images={},this.imageBroken={},this.callback=void 0}e.prototype.setOnloadCallback=function(t){this.callback=t},e.prototype.load=function(t,e){var i=this.images[t];if(void 0===i){var s=this;i=new Image,i.onload=function(){0==this.width&&(document.body.appendChild(this),this.width=this.offsetWidth,this.height=this.offsetHeight,document.body.removeChild(this)),s.callback&&(s.images[t]=i,s.callback(this))},i.onerror=function(){void 0===e?(console.error("Could not load image:",t),delete this.src,s.callback&&s.callback(this)):s.imageBroken[t]===!0?(console.error("Could not load brokenImage:",e),delete this.src,s.callback&&s.callback(this)):(this.src=e,s.imageBroken[t]=!0)},i.src=t}return i},t.exports=e},function(t,e,i){function s(t,e,i,s){var n=o.selectiveBridgeObject(["nodes"],s);this.options=n.nodes,this.selected=!1,this.hover=!1,this.edges=[],this.dynamicEdges=[],this.reroutedEdges={},this.fontDrawThreshold=3,this.id=void 0,this.allowedToMoveX=!1,this.allowedToMoveY=!1,this.xFixed=!1,this.yFixed=!1,this.horizontalAlignLeft=!0,this.verticalAlignTop=!0,this.baseRadiusValue=s.nodes.radius,this.radiusFixed=!1,this.level=-1,this.preassignedLevel=!1,this.hierarchyEnumerated=!1,this.labelDimensions={top:0,left:0,width:0,height:0,yLine:0},this.boundingBox={top:0,left:0,right:0,bottom:0},this.imagelist=e,this.grouplist=i,this.fx=0,this.fy=0,this.vx=0,this.vy=0,this.x=null,this.y=null,this.previousState={vx:0,vy:0,x:0,y:0},this.damping=s.physics.damping,this.fixedData={x:null,y:null},this.setProperties(t,n),this.resetCluster(),this.dynamicEdgesLength=0,this.clusterSession=0,this.clusterSizeWidthFactor=s.clustering.nodeScaling.width,this.clusterSizeHeightFactor=s.clustering.nodeScaling.height,this.clusterSizeRadiusFactor=s.clustering.nodeScaling.radius,this.maxNodeSizeIncrements=s.clustering.maxNodeSizeIncrements,this.growthIndicator=0,this.networkScaleInv=1,this.networkScale=1,this.canvasTopLeft={x:-300,y:-300},this.canvasBottomRight={x:300,y:300},this.parentEdgeId=null}var o=i(1);s.prototype.revertPosition=function(){this.x=this.previousState.x,this.y=this.previousState.y,this.vx=this.previousState.vx,this.vy=this.previousState.vy},s.prototype.resetCluster=function(){this.formationScale=void 0,this.clusterSize=1,this.containedNodes={},this.containedEdges={},this.clusterSessions=[]},s.prototype.attachEdge=function(t){-1==this.edges.indexOf(t)&&this.edges.push(t),-1==this.dynamicEdges.indexOf(t)&&this.dynamicEdges.push(t),this.dynamicEdgesLength=this.dynamicEdges.length},s.prototype.detachEdge=function(t){var e=this.edges.indexOf(t);-1!=e&&this.edges.splice(e,1),e=this.dynamicEdges.indexOf(t),-1!=e&&this.dynamicEdges.splice(e,1),this.dynamicEdgesLength=this.dynamicEdges.length},s.prototype.setProperties=function(t,e){if(t){var i=["borderWidth","borderWidthSelected","shape","image","brokenImage","radius","fontColor","fontSize","fontFace","fontFill","fontStrokeWidth","fontStrokeColor","group","mass"];if(o.selectiveDeepExtend(i,this.options,t),void 0!==t.id&&(this.id=t.id),void 0!==t.label&&(this.label=t.label,this.originalLabel=t.label),void 0!==t.title&&(this.title=t.title),void 0!==t.x&&(this.x=t.x),void 0!==t.y&&(this.y=t.y),void 0!==t.value&&(this.value=t.value),void 0!==t.level&&(this.level=t.level,this.preassignedLevel=!0),void 0!==t.horizontalAlignLeft&&(this.horizontalAlignLeft=t.horizontalAlignLeft),void 0!==t.verticalAlignTop&&(this.verticalAlignTop=t.verticalAlignTop),void 0!==t.triggerFunction&&(this.triggerFunction=t.triggerFunction),void 0===this.id)throw"Node must have an id";if("number"==typeof this.options.group||"string"==typeof this.options.group&&""!=this.options.group){var s=this.grouplist.get(this.options.group);o.deepExtend(this.options,s),this.options.color=o.parseColor(this.options.color)}if(void 0!==t.radius&&(this.baseRadiusValue=this.options.radius),void 0!==t.color&&(this.options.color=o.parseColor(t.color)),void 0!==this.options.image&&""!=this.options.image){if(!this.imagelist)throw"No imagelist provided";this.imageObj=this.imagelist.load(this.options.image,this.options.brokenImage)}switch(void 0!==t.allowedToMoveX?(this.xFixed=!t.allowedToMoveX,this.allowedToMoveX=t.allowedToMoveX):void 0!==t.x&&0==this.allowedToMoveX&&(this.xFixed=!0),void 0!==t.allowedToMoveY?(this.yFixed=!t.allowedToMoveY,this.allowedToMoveY=t.allowedToMoveY):void 0!==t.y&&0==this.allowedToMoveY&&(this.yFixed=!0),this.radiusFixed=this.radiusFixed||void 0!==t.radius,("image"===this.options.shape||"circularImage"===this.options.shape)&&(this.options.radiusMin=e.nodes.widthMin,this.options.radiusMax=e.nodes.widthMax),this.options.shape){case"database":this.draw=this._drawDatabase,this.resize=this._resizeDatabase;break;case"box":this.draw=this._drawBox,this.resize=this._resizeBox;break;case"circle":this.draw=this._drawCircle,this.resize=this._resizeCircle;break;case"ellipse":this.draw=this._drawEllipse,this.resize=this._resizeEllipse;break;case"image":this.draw=this._drawImage,this.resize=this._resizeImage;break;case"circularImage":this.draw=this._drawCircularImage,this.resize=this._resizeCircularImage;break;case"text":this.draw=this._drawText,this.resize=this._resizeText;break;case"dot":this.draw=this._drawDot,this.resize=this._resizeShape;break;case"square":this.draw=this._drawSquare,this.resize=this._resizeShape;break;case"triangle":this.draw=this._drawTriangle,this.resize=this._resizeShape;break;case"triangleDown":this.draw=this._drawTriangleDown,this.resize=this._resizeShape;break;case"star":this.draw=this._drawStar,this.resize=this._resizeShape;break;default:this.draw=this._drawEllipse,this.resize=this._resizeEllipse}this._reset()}},s.prototype.select=function(){this.selected=!0,this._reset()},s.prototype.unselect=function(){this.selected=!1,this._reset()},s.prototype.clearSizeCache=function(){this._reset()},s.prototype._reset=function(){this.width=void 0,this.height=void 0},s.prototype.getTitle=function(){return"function"==typeof this.title?this.title():this.title},s.prototype.distanceToBorder=function(t,e){var i=1;switch(this.width||this.resize(t),this.options.shape){case"circle":case"dot":return this.options.radius+i;case"ellipse":var s=this.width/2,o=this.height/2,n=Math.sin(e)*s,r=Math.cos(e)*o;return s*o/Math.sqrt(n*n+r*r);case"box":case"image":case"text":default:return this.width?Math.min(Math.abs(this.width/2/Math.cos(e)),Math.abs(this.height/2/Math.sin(e)))+i:0}},s.prototype._setForce=function(t,e){this.fx=t,this.fy=e},s.prototype._addForce=function(t,e){this.fx+=t,this.fy+=e},s.prototype.storeState=function(){this.previousState.x=this.x,this.previousState.y=this.y,this.previousState.vx=this.vx,this.previousState.vy=this.vy},s.prototype.discreteStep=function(t){if(this.storeState(),this.xFixed)this.fx=0,this.vx=0;else{var e=this.damping*this.vx,i=(this.fx-e)/this.options.mass;this.vx+=i*t,this.x+=this.vx*t}if(this.yFixed)this.fy=0,this.vy=0;else{var s=this.damping*this.vy,o=(this.fy-s)/this.options.mass;this.vy+=o*t,this.y+=this.vy*t}},s.prototype.discreteStepLimited=function(t,e){if(this.storeState(),this.xFixed)this.fx=0,this.vx=0;else{var i=this.damping*this.vx,s=(this.fx-i)/this.options.mass;this.vx+=s*t,this.vx=Math.abs(this.vx)>e?this.vx>0?e:-e:this.vx,this.x+=this.vx*t}if(this.yFixed)this.fy=0,this.vy=0;else{var o=this.damping*this.vy,n=(this.fy-o)/this.options.mass;this.vy+=n*t,this.vy=Math.abs(this.vy)>e?this.vy>0?e:-e:this.vy,this.y+=this.vy*t}},s.prototype.isFixed=function(){return this.xFixed&&this.yFixed},s.prototype.isMoving=function(t){var e=Math.sqrt(Math.pow(this.vx,2)+Math.pow(this.vy,2));return e>t},s.prototype.isSelected=function(){return this.selected},s.prototype.getValue=function(){return this.value},s.prototype.getDistance=function(t,e){var i=this.x-t,s=this.y-e;return Math.sqrt(i*i+s*s)},s.prototype.setValueRange=function(t,e){if(!this.radiusFixed&&void 0!==this.value)if(e==t)this.options.radius=(this.options.radiusMin+this.options.radiusMax)/2;else{var i=(this.options.radiusMax-this.options.radiusMin)/(e-t);this.options.radius=(this.value-t)*i+this.options.radiusMin}this.baseRadiusValue=this.options.radius},s.prototype.draw=function(){throw"Draw method not initialized for node"},s.prototype.resize=function(){throw"Resize method not initialized for node"},s.prototype.isOverlappingWith=function(t){return this.leftt.left&&this.topt.top},s.prototype._resizeImage=function(){if(!this.width||!this.height){var t,e;if(this.value){this.options.radius=this.baseRadiusValue;var i=this.imageObj.height/this.imageObj.width;void 0!==i?(t=this.options.radius||this.imageObj.width,e=this.options.radius*i||this.imageObj.height):(t=0,e=0)}else t=this.imageObj.width,e=this.imageObj.height;this.width=t,this.height=e,this.growthIndicator=0,this.width>0&&this.height>0&&(this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-t)}},s.prototype._drawImageAtPosition=function(t){if(0!=this.imageObj.width){if(this.clusterSize>1){var e=this.clusterSize>1?10:0;e*=this.networkScaleInv,e=Math.min(.2*this.width,e),t.globalAlpha=.5,t.drawImage(this.imageObj,this.left-e,this.top-e,this.width+2*e,this.height+2*e)}t.globalAlpha=1,t.drawImage(this.imageObj,this.left,this.top,this.width,this.height)}},s.prototype._drawImageLabel=function(t){var e,i=0;if(this.height){i=this.height/2;var s=this.getTextSize(t);s.lineCount>=1&&(i+=s.height/2,i+=3)}e=this.y+i,this._label(t,this.label,this.x,e,void 0)},s.prototype._drawImage=function(t){this._resizeImage(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._drawImageAtPosition(t),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._drawImageLabel(t),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height)},s.prototype._resizeCircularImage=function(t){if(this.imageObj.src&&this.imageObj.width&&this.imageObj.height)this._swapToImageResizeWhenImageLoaded&&(this.width=0,this.height=0,delete this._swapToImageResizeWhenImageLoaded),this._resizeImage(t);else if(!this.width){var e=2*this.options.radius;this.width=e,this.height=e,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.options.radius-.5*e,this._swapToImageResizeWhenImageLoaded=!0}},s.prototype._drawCircularImage=function(t){this._resizeCircularImage(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=this.left+this.width/2,i=this.top+this.height/2,s=Math.abs(this.height/2);this._drawRawCircle(t,e,i,s),t.save(),t.circle(this.x,this.y,s),t.stroke(),t.clip(),this._drawImageAtPosition(t),t.restore(),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this._drawImageLabel(t),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height)},s.prototype._resizeBox=function(t){if(!this.width){var e=5,i=this.getTextSize(t);this.width=i.width+2*e,this.height=i.height+2*e,this.width+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.growthIndicator=this.width-(i.width+2*e)}},s.prototype._drawBox=function(t){this._resizeBox(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=2.5,i=this.options.borderWidth,s=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.roundRect(this.left-2*t.lineWidth,this.top-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth,this.options.radius),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.roundRect(this.left,this.top,this.width,this.height,this.options.radius),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y)},s.prototype._resizeDatabase=function(t){if(!this.width){var e=5,i=this.getTextSize(t),s=i.width+2*e;this.width=s,this.height=s,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-s}},s.prototype._drawDatabase=function(t){this._resizeDatabase(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var e=2.5,i=this.options.borderWidth,s=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.database(this.x-this.width/2-2*t.lineWidth,this.y-.5*this.height-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.database(this.x-this.width/2,this.y-.5*this.height,this.width,this.height),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y) -},s.prototype._resizeCircle=function(t){if(!this.width){var e=5,i=this.getTextSize(t),s=Math.max(i.width,i.height)+2*e;this.options.radius=s/2,this.width=s,this.height=s,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.options.radius-.5*s}},s.prototype._drawRawCircle=function(t,e,i,s){var o=2.5,n=this.options.borderWidth,r=this.options.borderWidthSelected||2*this.options.borderWidth;t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?r:n)+(this.clusterSize>1?o:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.circle(e,i,s+2*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?r:n)+(this.clusterSize>1?o:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.circle(this.x,this.y,s),t.fill(),t.stroke()},s.prototype._drawCircle=function(t){this._resizeCircle(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._drawRawCircle(t,this.x,this.y,this.options.radius),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this._label(t,this.label,this.x,this.y)},s.prototype._resizeEllipse=function(t){if(!this.width){var e=this.getTextSize(t);this.width=1.5*e.width,this.height=2*e.height,this.width1&&(t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.ellipse(this.left-2*t.lineWidth,this.top-2*t.lineWidth,this.width+4*t.lineWidth,this.height+4*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?s:i)+(this.clusterSize>1?e:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t.ellipse(this.left,this.top,this.width,this.height),t.fill(),t.stroke(),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,this._label(t,this.label,this.x,this.y)},s.prototype._drawDot=function(t){this._drawShape(t,"circle")},s.prototype._drawTriangle=function(t){this._drawShape(t,"triangle")},s.prototype._drawTriangleDown=function(t){this._drawShape(t,"triangleDown")},s.prototype._drawSquare=function(t){this._drawShape(t,"square")},s.prototype._drawStar=function(t){this._drawShape(t,"star")},s.prototype._resizeShape=function(){if(!this.width){this.options.radius=this.baseRadiusValue;var t=2*this.options.radius;this.width=t,this.height=t,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=.5*Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-t}},s.prototype._drawShape=function(t,e){this._resizeShape(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2;var i=2.5,s=this.options.borderWidth,o=this.options.borderWidthSelected||2*this.options.borderWidth,n=2;switch(e){case"dot":n=2;break;case"square":n=2;break;case"triangle":n=3;break;case"triangleDown":n=3;break;case"star":n=4}t.strokeStyle=this.selected?this.options.color.highlight.border:this.hover?this.options.color.hover.border:this.options.color.border,this.clusterSize>1&&(t.lineWidth=(this.selected?o:s)+(this.clusterSize>1?i:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t[e](this.x,this.y,this.options.radius+n*t.lineWidth),t.stroke()),t.lineWidth=(this.selected?o:s)+(this.clusterSize>1?i:0),t.lineWidth*=this.networkScaleInv,t.lineWidth=Math.min(this.width,t.lineWidth),t.fillStyle=this.selected?this.options.color.highlight.background:this.hover?this.options.color.hover.background:this.options.color.background,t[e](this.x,this.y,this.options.radius),t.fill(),t.stroke(),this.boundingBox.top=this.y-this.options.radius,this.boundingBox.left=this.x-this.options.radius,this.boundingBox.right=this.x+this.options.radius,this.boundingBox.bottom=this.y+this.options.radius,this.label&&(this._label(t,this.label,this.x,this.y+this.height/2,void 0,"hanging",!0),this.boundingBox.left=Math.min(this.boundingBox.left,this.labelDimensions.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelDimensions.left+this.labelDimensions.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelDimensions.height))},s.prototype._resizeText=function(t){if(!this.width){var e=5,i=this.getTextSize(t);this.width=i.width+2*e,this.height=i.height+2*e,this.width+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeWidthFactor,this.height+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeHeightFactor,this.options.radius+=Math.min(this.clusterSize-1,this.maxNodeSizeIncrements)*this.clusterSizeRadiusFactor,this.growthIndicator=this.width-(i.width+2*e)}},s.prototype._drawText=function(t){this._resizeText(t),this.left=this.x-this.width/2,this.top=this.y-this.height/2,this._label(t,this.label,this.x,this.y),this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height},s.prototype._label=function(t,e,i,s,o,n,r){if(e&&Number(this.options.fontSize)*this.networkScale>this.fontDrawThreshold){t.font=(this.selected?"bold ":"")+this.options.fontSize+"px "+this.options.fontFace;var a=e.split("\n"),h=a.length,d=Number(this.options.fontSize),l=s+(1-h)/2*d;1==r&&(l=s+(1-h)/(2*d));for(var c=t.measureText(a[0]).width,p=1;h>p;p++){var u=t.measureText(a[p]).width;c=u>c?u:c}var m=this.options.fontSize*h,f=i-c/2,g=s-m/2;"hanging"==n&&(g+=.5*d,g+=4,l+=4),this.labelDimensions={top:g,left:f,width:c,height:m,yLine:l},void 0!==this.options.fontFill&&null!==this.options.fontFill&&"none"!==this.options.fontFill&&(t.fillStyle=this.options.fontFill,t.fillRect(f,g,c,m)),t.fillStyle=this.options.fontColor||"black",t.textAlign=o||"center",t.textBaseline=n||"middle",this.options.fontStrokeWidth>0&&(t.lineWidth=this.options.fontStrokeWidth,t.strokeStyle=this.options.fontStrokeColor,t.lineJoin="round");for(var p=0;h>p;p++)this.options.fontStrokeWidth&&t.strokeText(a[p],i,l),t.fillText(a[p],i,l),l+=d}},s.prototype.getTextSize=function(t){if(void 0!==this.label){t.font=(this.selected?"bold ":"")+this.options.fontSize+"px "+this.options.fontFace;for(var e=this.label.split("\n"),i=(Number(this.options.fontSize)+4)*e.length,s=0,o=0,n=e.length;n>o;o++)s=Math.max(s,t.measureText(e[o]).width);return{width:s,height:i,lineCount:e.length}}return{width:0,height:0,lineCount:0}},s.prototype.inArea=function(){return void 0!==this.width?this.x+this.width*this.networkScaleInv>=this.canvasTopLeft.x&&this.x-this.width*this.networkScaleInv=this.canvasTopLeft.y&&this.y-this.height*this.networkScaleInv=this.canvasTopLeft.x&&this.x=this.canvasTopLeft.y&&this.ys&&(n=s-e-this.padding),no&&(r=o-i-this.padding),ri;i++)if(e.id===r.nodes[i].id){o=r.nodes[i];break}for(o||(o={id:e.id},t.node&&(o.attr=a(o.attr,t.node))),i=n.length-1;i>=0;i--){var h=n[i];h.nodes||(h.nodes=[]),-1==h.nodes.indexOf(o)&&h.nodes.push(o)}e.attr&&(o.attr=a(o.attr,e.attr))}function l(t,e){if(t.edges||(t.edges=[]),t.edges.push(e),t.edge){var i=a({},t.edge);e.attr=a(i,e.attr)}}function c(t,e,i,s,o){var n={from:e,to:i,type:s};return t.edge&&(n.attr=a({},t.edge)),n.attr=a(n.attr||{},o),n}function p(){for(N=D.NULL,k="";" "==E||" "==E||"\n"==E||"\r"==E;)o();do{var t=!1;if("#"==E){for(var e=O-1;" "==T.charAt(e)||" "==T.charAt(e);)e--;if("\n"==T.charAt(e)||""==T.charAt(e)){for(;""!=E&&"\n"!=E;)o();t=!0}}if("/"==E&&"/"==n()){for(;""!=E&&"\n"!=E;)o();t=!0}if("/"==E&&"*"==n()){for(;""!=E;){if("*"==E&&"/"==n()){o(),o();break}o()}t=!0}for(;" "==E||" "==E||"\n"==E||"\r"==E;)o()}while(t);if(""==E)return void(N=D.DELIMITER);var i=E+n();if(C[i])return N=D.DELIMITER,k=i,o(),void o();if(C[E])return N=D.DELIMITER,k=E,void o();if(r(E)||"-"==E){for(k+=E,o();r(E);)k+=E,o();return"false"==k?k=!1:"true"==k?k=!0:isNaN(Number(k))||(k=Number(k)),void(N=D.IDENTIFIER)}if('"'==E){for(o();""!=E&&('"'!=E||'"'==E&&'"'==n());)k+=E,'"'==E&&o(),o();if('"'!=E)throw x('End of string " expected');return o(),void(N=D.IDENTIFIER)}for(N=D.UNKNOWN;""!=E;)k+=E,o();throw new SyntaxError('Syntax error in part "'+w(k,30)+'"')}function u(){var t={};if(s(),p(),"strict"==k&&(t.strict=!0,p()),("graph"==k||"digraph"==k)&&(t.type=k,p()),N==D.IDENTIFIER&&(t.id=k,p()),"{"!=k)throw x("Angle bracket { expected");if(p(),m(t),"}"!=k)throw x("Angle bracket } expected");if(p(),""!==k)throw x("End of file expected");return p(),delete t.node,delete t.edge,delete t.graph,t}function m(t){for(;""!==k&&"}"!=k;)f(t),";"==k&&p()}function f(t){var e=g(t);if(e)return void b(t,e);var i=v(t);if(!i){if(N!=D.IDENTIFIER)throw x("Identifier expected");var s=k;if(p(),"="==k){if(p(),N!=D.IDENTIFIER)throw x("Identifier expected");t[s]=k,p()}else y(t,s)}}function g(t){var e=null;if("subgraph"==k&&(e={},e.type="subgraph",p(),N==D.IDENTIFIER&&(e.id=k,p())),"{"==k){if(p(),e||(e={}),e.parent=t,e.node=t.node,e.edge=t.edge,e.graph=t.graph,m(e),"}"!=k)throw x("Angle bracket } expected");p(),delete e.node,delete e.edge,delete e.graph,delete e.parent,t.subgraphs||(t.subgraphs=[]),t.subgraphs.push(e)}return e}function v(t){return"node"==k?(p(),t.node=_(),"node"):"edge"==k?(p(),t.edge=_(),"edge"):"graph"==k?(p(),t.graph=_(),"graph"):null}function y(t,e){var i={id:e},s=_();s&&(i.attr=s),d(t,i),b(t,e)}function b(t,e){for(;"->"==k||"--"==k;){var i,s=k;p();var o=g(t);if(o)i=o;else{if(N!=D.IDENTIFIER)throw x("Identifier or subgraph expected");i=k,d(t,{id:i}),p()}var n=_(),r=c(t,e,i,s,n);l(t,r),e=i}}function _(){for(var t=null;"["==k;){for(p(),t={};""!==k&&"]"!=k;){if(N!=D.IDENTIFIER)throw x("Attribute name expected");var e=k;if(p(),"="!=k)throw x("Equal sign = expected");if(p(),N!=D.IDENTIFIER)throw x("Attribute value expected");var i=k;h(t,e,i),p(),","==k&&p()}if("]"!=k)throw x("Bracket ] expected");p()}return t}function x(t){return new SyntaxError(t+', got "'+w(k,30)+'" (char '+O+")")}function w(t,e){return t.length<=e?t:t.substr(0,27)+"..."}function S(t,e,i){Array.isArray(t)?t.forEach(function(t){Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}):Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}function M(t){var e=i(t),s={nodes:[],edges:[],options:{}};if(e.nodes&&e.nodes.forEach(function(t){var e={id:t.id,label:String(t.label||t.id)};a(e,t.attr),e.image&&(e.shape="image"),s.nodes.push(e)}),e.edges){var o=function(t){var e={from:t.from,to:t.to};return a(e,t.attr),e.style="->"==t.type?"arrow":"line",e};e.edges.forEach(function(t){var e,i;e=t.from instanceof Object?t.from.nodes:{id:t.from},i=t.to instanceof Object?t.to.nodes:{id:t.to},t.from instanceof Object&&t.from.edges&&t.from.edges.forEach(function(t){var e=o(t);s.edges.push(e)}),S(e,i,function(e,i){var n=c(s,e.id,i.id,t.type,t.attr),r=o(n);s.edges.push(r)}),t.to instanceof Object&&t.to.edges&&t.to.edges.forEach(function(t){var e=o(t);s.edges.push(e)})})}return e.attr&&(s.options=e.attr),s}var D={NULL:0,DELIMITER:1,IDENTIFIER:2,UNKNOWN:3},C={"{":!0,"}":!0,"[":!0,"]":!0,";":!0,"=":!0,",":!0,"->":!0,"--":!0},T="",O=0,E="",k="",N=D.NULL,I=/[a-zA-Z_0-9.:#]/;e.parseDOT=i,e.DOTToGraph=M},function(t,e){function i(t,e){var i=[],s=[];this.options={edges:{inheritColor:!0},nodes:{allowedToMove:!1,parseColor:!1}},void 0!==e&&(this.options.nodes.allowedToMove=e.allowedToMove|!1,this.options.nodes.parseColor=e.parseColor|!1,this.options.edges.inheritColor=e.inheritColor|!0);for(var o=t.edges,n=t.nodes,r=0;r=s&&(s=864e5),e=new Date(e.valueOf()-.05*s),i=new Date(i.valueOf()+.05*s)}return{start:e,end:i}},s.prototype.setWindow=function(t,e,i){var s=i&&void 0!==i.animate?i.animate:!0;if(1==arguments.length){var o=arguments[0];this.range.setRange(o.start,o.end,s)}else this.range.setRange(t,e,s)},s.prototype.moveTo=function(t,e){var i=this.range.end-this.range.start,s=r.convert(t,"Date").valueOf(),o=s-i/2,n=s+i/2,a=e&&void 0!==e.animate?e.animate:!0;this.range.setRange(o,n,a)},s.prototype.getWindow=function(){var t=this.range.getRange();return{start:new Date(t.start),end:new Date(t.end)}},s.prototype.redraw=function(){var t=!1,e=this.options,i=this.props,s=this.dom;if(s){h.updateHiddenDates(this.body,this.options.hiddenDates),"top"==e.orientation?(r.addClassName(s.root,"top"),r.removeClassName(s.root,"bottom")):(r.removeClassName(s.root,"top"),r.addClassName(s.root,"bottom")),s.root.style.maxHeight=r.option.asSize(e.maxHeight,""),s.root.style.minHeight=r.option.asSize(e.minHeight,""),s.root.style.width=r.option.asSize(e.width,""),i.border.left=(s.centerContainer.offsetWidth-s.centerContainer.clientWidth)/2,i.border.right=i.border.left,i.border.top=(s.centerContainer.offsetHeight-s.centerContainer.clientHeight)/2,i.border.bottom=i.border.top;var o=s.root.offsetHeight-s.root.clientHeight,n=s.root.offsetWidth-s.root.clientWidth;0===s.centerContainer.clientHeight&&(i.border.left=i.border.top,i.border.right=i.border.left),0===s.root.clientHeight&&(n=o),i.center.height=s.center.offsetHeight,i.left.height=s.left.offsetHeight,i.right.height=s.right.offsetHeight,i.top.height=s.top.clientHeight||-i.border.top,i.bottom.height=s.bottom.clientHeight||-i.border.bottom;var a=Math.max(i.left.height,i.center.height,i.right.height),d=i.top.height+a+i.bottom.height+o+i.border.top+i.border.bottom;s.root.style.height=r.option.asSize(e.height,d+"px"),i.root.height=s.root.offsetHeight,i.background.height=i.root.height-o;var l=i.root.height-i.top.height-i.bottom.height-o;i.centerContainer.height=l,i.leftContainer.height=l,i.rightContainer.height=i.leftContainer.height,i.root.width=s.root.offsetWidth,i.background.width=i.root.width-n,i.left.width=s.leftContainer.clientWidth||-i.border.left,i.leftContainer.width=i.left.width,i.right.width=s.rightContainer.clientWidth||-i.border.right,i.rightContainer.width=i.right.width;var c=i.root.width-i.left.width-i.right.width-n;i.center.width=c,i.centerContainer.width=c,i.top.width=c,i.bottom.width=c,s.background.style.height=i.background.height+"px",s.backgroundVertical.style.height=i.background.height+"px",s.backgroundHorizontal.style.height=i.centerContainer.height+"px",s.centerContainer.style.height=i.centerContainer.height+"px",s.leftContainer.style.height=i.leftContainer.height+"px",s.rightContainer.style.height=i.rightContainer.height+"px",s.background.style.width=i.background.width+"px",s.backgroundVertical.style.width=i.centerContainer.width+"px",s.backgroundHorizontal.style.width=i.background.width+"px",s.centerContainer.style.width=i.center.width+"px",s.top.style.width=i.top.width+"px",s.bottom.style.width=i.bottom.width+"px",s.background.style.left="0",s.background.style.top="0",s.backgroundVertical.style.left=i.left.width+i.border.left+"px",s.backgroundVertical.style.top="0",s.backgroundHorizontal.style.left="0",s.backgroundHorizontal.style.top=i.top.height+"px",s.centerContainer.style.left=i.left.width+"px",s.centerContainer.style.top=i.top.height+"px",s.leftContainer.style.left="0",s.leftContainer.style.top=i.top.height+"px",s.rightContainer.style.left=i.left.width+i.center.width+"px",s.rightContainer.style.top=i.top.height+"px",s.top.style.left=i.left.width+"px",s.top.style.top="0",s.bottom.style.left=i.left.width+"px",s.bottom.style.top=i.top.height+i.centerContainer.height+"px",this._updateScrollTop();var p=this.props.scrollTop;"bottom"==e.orientation&&(p+=Math.max(this.props.centerContainer.height-this.props.center.height-this.props.border.top-this.props.border.bottom,0)),s.center.style.left="0",s.center.style.top=p+"px",s.left.style.left="0",s.left.style.top=p+"px",s.right.style.left="0",s.right.style.top=p+"px";var u=0==this.props.scrollTop?"hidden":"",m=this.props.scrollTop==this.props.scrollTopMin?"hidden":"";if(s.shadowTop.style.visibility=u,s.shadowBottom.style.visibility=m,s.shadowTopLeft.style.visibility=u,s.shadowBottomLeft.style.visibility=m,s.shadowTopRight.style.visibility=u,s.shadowBottomRight.style.visibility=m,this.components.forEach(function(e){t=e.redraw()||t}),t){var f=3;this.redrawCount0&&(this.props.scrollTop=0),this.props.scrollTops;s++){var o=s%2===0?1.3*i:.5*i;this.lineTo(t+o*Math.sin(2*s*Math.PI/10),e-o*Math.cos(2*s*Math.PI/10))}this.closePath()},CanvasRenderingContext2D.prototype.roundRect=function(t,e,i,s,o){var n=Math.PI/180;0>i-2*o&&(o=i/2),0>s-2*o&&(o=s/2),this.beginPath(),this.moveTo(t+o,e),this.lineTo(t+i-o,e),this.arc(t+i-o,e+o,o,270*n,360*n,!1),this.lineTo(t+i,e+s-o),this.arc(t+i-o,e+s-o,o,0,90*n,!1),this.lineTo(t+o,e+s),this.arc(t+o,e+s-o,o,90*n,180*n,!1),this.lineTo(t,e+o),this.arc(t+o,e+o,o,180*n,270*n,!1)},CanvasRenderingContext2D.prototype.ellipse=function(t,e,i,s){var o=.5522848,n=i/2*o,r=s/2*o,a=t+i,h=e+s,d=t+i/2,l=e+s/2; -this.beginPath(),this.moveTo(t,l),this.bezierCurveTo(t,l-r,d-n,e,d,e),this.bezierCurveTo(d+n,e,a,l-r,a,l),this.bezierCurveTo(a,l+r,d+n,h,d,h),this.bezierCurveTo(d-n,h,t,l+r,t,l)},CanvasRenderingContext2D.prototype.database=function(t,e,i,s){var o=1/3,n=i,r=s*o,a=.5522848,h=n/2*a,d=r/2*a,l=t+n,c=e+r,p=t+n/2,u=e+r/2,m=e+(s-r/2),f=e+s;this.beginPath(),this.moveTo(l,u),this.bezierCurveTo(l,u+d,p+h,c,p,c),this.bezierCurveTo(p-h,c,t,u+d,t,u),this.bezierCurveTo(t,u-d,p-h,e,p,e),this.bezierCurveTo(p+h,e,l,u-d,l,u),this.lineTo(l,m),this.bezierCurveTo(l,m+d,p+h,f,p,f),this.bezierCurveTo(p-h,f,t,m+d,t,m),this.lineTo(t,u)},CanvasRenderingContext2D.prototype.arrow=function(t,e,i,s){var o=t-s*Math.cos(i),n=e-s*Math.sin(i),r=t-.9*s*Math.cos(i),a=e-.9*s*Math.sin(i),h=o+s/3*Math.cos(i+.5*Math.PI),d=n+s/3*Math.sin(i+.5*Math.PI),l=o+s/3*Math.cos(i-.5*Math.PI),c=n+s/3*Math.sin(i-.5*Math.PI);this.beginPath(),this.moveTo(t,e),this.lineTo(h,d),this.lineTo(r,a),this.lineTo(l,c),this.closePath()},CanvasRenderingContext2D.prototype.dashedLine=function(t,e,i,s,o){o||(o=[10,5]),0==p&&(p=.001);var n=o.length;this.moveTo(t,e);for(var r=i-t,a=s-e,h=a/r,d=Math.sqrt(r*r+a*a),l=0,c=!0;d>=.1;){var p=o[l++%n];p>d&&(p=d);var u=Math.sqrt(p*p/(1+h*h));0>r&&(u=-u),t+=u,e+=h*u,this[c?"lineTo":"moveTo"](t,e),d-=p,c=!c}})},function(t,e,i){function s(t,e){this.groupId=t,this.options=e}var o=i(2),n=i(53);s.prototype.getYRange=function(t){for(var e=t[0].y,i=t[0].y,s=0;st[s].y?t[s].y:e,i=i0){var r,a,h=Number(i.svg.style.height.replace("px",""));if(r=o.getSVGElement("path",i.svgElements,i.svg),r.setAttributeNS(null,"class",e.className),void 0!==e.style&&r.setAttributeNS(null,"style",e.style),a=1==e.options.catmullRom.enabled?s._catmullRom(t,e):s._linear(t),1==e.options.shaded.enabled){var d,l=o.getSVGElement("path",i.svgElements,i.svg);d="top"==e.options.shaded.orientation?"M"+t[0].x+",0 "+a+"L"+t[t.length-1].x+",0":"M"+t[0].x+","+h+" "+a+"L"+t[t.length-1].x+","+h,l.setAttributeNS(null,"class",e.className+" fill"),void 0!==e.options.shaded.style&&l.setAttributeNS(null,"style",e.options.shaded.style),l.setAttributeNS(null,"d",d)}r.setAttributeNS(null,"d","M"+a),1==e.options.drawPoints.enabled&&n.draw(t,e,i)}},s._catmullRomUniform=function(t){for(var e,i,s,o,n,r,a=Math.round(t[0].x)+","+Math.round(t[0].y)+" ",h=1/6,d=t.length,l=0;d-1>l;l++)e=0==l?t[0]:t[l-1],i=t[l],s=t[l+1],o=d>l+2?t[l+2]:s,n={x:(-e.x+6*i.x+s.x)*h,y:(-e.y+6*i.y+s.y)*h},r={x:(i.x+6*s.x-o.x)*h,y:(i.y+6*s.y-o.y)*h},a+="C"+n.x+","+n.y+" "+r.x+","+r.y+" "+s.x+","+s.y+" ";return a},s._catmullRom=function(t,e){var i=e.options.catmullRom.alpha;if(0==i||void 0===i)return this._catmullRomUniform(t);for(var s,o,n,r,a,h,d,l,c,p,u,m,f,g,v,y,b,_,x,w=Math.round(t[0].x)+","+Math.round(t[0].y)+" ",S=t.length,M=0;S-1>M;M++)s=0==M?t[0]:t[M-1],o=t[M],n=t[M+1],r=S>M+2?t[M+2]:n,d=Math.sqrt(Math.pow(s.x-o.x,2)+Math.pow(s.y-o.y,2)),l=Math.sqrt(Math.pow(o.x-n.x,2)+Math.pow(o.y-n.y,2)),c=Math.sqrt(Math.pow(n.x-r.x,2)+Math.pow(n.y-r.y,2)),g=Math.pow(c,i),y=Math.pow(c,2*i),v=Math.pow(l,i),b=Math.pow(l,2*i),x=Math.pow(d,i),_=Math.pow(d,2*i),p=2*_+3*x*v+b,u=2*y+3*g*v+b,m=3*x*(x+v),m>0&&(m=1/m),f=3*g*(g+v),f>0&&(f=1/f),a={x:(-b*s.x+p*o.x+_*n.x)*m,y:(-b*s.y+p*o.y+_*n.y)*m},h={x:(y*o.x+u*n.x-b*r.x)*f,y:(y*o.y+u*n.y-b*r.y)*f},0==a.x&&0==a.y&&(a=o),0==h.x&&0==h.y&&(h=n),w+="C"+a.x+","+a.y+" "+h.x+","+h.y+" "+n.x+","+n.y+" ";return w},s._linear=function(t){for(var e="",i=0;it[s].y?t[s].y:e,i=i0&&(n=Math.min(n,Math.abs(c[d-1].x-r))),a=s._getSafeDrawData(n,h,m);else{var g=d+(p[r].amount-p[r].resolved),v=d-(p[r].resolved+1);g0&&(n=Math.min(n,Math.abs(c[v].x-r))),a=s._getSafeDrawData(n,h,m),p[r].resolved+=1,"stack"==h.options.barChart.handleOverlap?(f=p[r].accumulated,p[r].accumulated+=h.zeroPosition-c[d].y):"sideBySide"==h.options.barChart.handleOverlap&&(a.width=a.width/p[r].amount,a.offset+=p[r].resolved*a.width-.5*a.width*(p[r].amount+1),"left"==h.options.barChart.align?a.offset-=.5*a.width:"right"==h.options.barChart.align&&(a.offset+=.5*a.width))}o.drawBar(c[d].x+a.offset,c[d].y-f,a.width,h.zeroPosition-c[d].y,h.className+" bar",i.svgElements,i.svg),1==h.options.drawPoints.enabled&&o.drawPoint(c[d].x+a.offset,c[d].y,h,i.svgElements,i.svg)}},s._getDataIntersections=function(t,e){for(var i,s=0;s0&&(i=Math.min(i,Math.abs(e[s-1].x-e[s].x))),0==i&&(void 0===t[e[s].x]&&(t[e[s].x]={amount:0,resolved:0,accumulated:0}),t[e[s].x].amount+=1)},s._getSafeDrawData=function(t,e,i){var s,o;return t0?(s=i>t?i:t,o=0,"left"==e.options.barChart.align?o-=.5*t:"right"==e.options.barChart.align&&(o+=.5*t)):(s=e.options.barChart.width,o=0,"left"==e.options.barChart.align?o-=.5*e.options.barChart.width:"right"==e.options.barChart.align&&(o+=.5*e.options.barChart.width)),{width:s,offset:o}},s.getStackedBarYRange=function(t,e,i,o,n){if(t.length>0){t.sort(function(t,e){return t.x==e.x?t.groupId-e.groupId:t.x-e.x});var r={};s._getDataIntersections(r,t),e[o]=s._getStackedBarYRange(r,t),e[o].yAxisOrientation=n,i.push(o)}},s._getStackedBarYRange=function(t,e){for(var i,s=e[0].y,o=e[0].y,n=0;ne[n].y?e[n].y:s,o=ot[r].accumulated?t[r].accumulated:s,o=ot[s].y?t[s].y:e,i=is;++s)i[s].apply(this,e)}return this},e.prototype.listeners=function(t){return this._callbacks=this._callbacks||{},this._callbacks[t]||[]},e.prototype.hasListeners=function(t){return!!this.listeners(t).length}},function(t,e,i){var s;(function(t,o){(function(n){function r(t,e,i){switch(arguments.length){case 2:return null!=t?t:e;case 3:return null!=t?t:null!=e?e:i;default:throw new Error("Implement me")}}function a(t,e){return Ie.call(t,e)}function h(){return{empty:!1,unusedTokens:[],unusedInput:[],overflow:-2,charsLeftOver:0,nullInput:!1,invalidMonth:null,invalidFormat:!1,userInvalidated:!1,iso:!1}}function d(t){Ce.suppressDeprecationWarnings===!1&&"undefined"!=typeof console&&console.warn&&console.warn("Deprecation warning: "+t)}function l(t,e){var i=!0;return b(function(){return i&&(d(t),i=!1),e.apply(this,arguments)},e)}function c(t,e){Si[t]||(d(e),Si[t]=!0)}function p(t,e){return function(i){return w(t.call(this,i),e)}}function u(t,e){return function(i){return this.localeData().ordinal(t.call(this,i),e)}}function m(t,e){var i,s,o=12*(e.year()-t.year())+(e.month()-t.month()),n=t.clone().add(o,"months");return 0>e-n?(i=t.clone().add(o-1,"months"),s=(e-n)/(n-i)):(i=t.clone().add(o+1,"months"),s=(e-n)/(i-n)),-(o+s)}function f(t,e,i){var s;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?(s=t.isPM(i),s&&12>e&&(e+=12),s||12!==e||(e=0),e):e}function g(){}function v(t,e){e!==!1&&F(t),_(this,t),this._d=new Date(+t._d),Di===!1&&(Di=!0,Ce.updateOffset(this),Di=!1)}function y(t){var e=N(t),i=e.year||0,s=e.quarter||0,o=e.month||0,n=e.week||0,r=e.day||0,a=e.hour||0,h=e.minute||0,d=e.second||0,l=e.millisecond||0;this._milliseconds=+l+1e3*d+6e4*h+36e5*a,this._days=+r+7*n,this._months=+o+3*s+12*i,this._data={},this._locale=Ce.localeData(),this._bubble()}function b(t,e){for(var i in e)a(e,i)&&(t[i]=e[i]);return a(e,"toString")&&(t.toString=e.toString),a(e,"valueOf")&&(t.valueOf=e.valueOf),t}function _(t,e){var i,s,o;if("undefined"!=typeof e._isAMomentObject&&(t._isAMomentObject=e._isAMomentObject),"undefined"!=typeof e._i&&(t._i=e._i),"undefined"!=typeof e._f&&(t._f=e._f),"undefined"!=typeof e._l&&(t._l=e._l),"undefined"!=typeof e._strict&&(t._strict=e._strict),"undefined"!=typeof e._tzm&&(t._tzm=e._tzm),"undefined"!=typeof e._isUTC&&(t._isUTC=e._isUTC),"undefined"!=typeof e._offset&&(t._offset=e._offset),"undefined"!=typeof e._pf&&(t._pf=e._pf),"undefined"!=typeof e._locale&&(t._locale=e._locale),Ye.length>0)for(i in Ye)s=Ye[i],o=e[s],"undefined"!=typeof o&&(t[s]=o);return t}function x(t){return 0>t?Math.ceil(t):Math.floor(t)}function w(t,e,i){for(var s=""+Math.abs(t),o=t>=0;s.lengths;s++)(i&&t[s]!==e[s]||!i&&L(t[s])!==L(e[s]))&&r++;return r+n}function k(t){if(t){var e=t.toLowerCase().replace(/(.)s$/,"$1");t=gi[t]||vi[e]||e}return t}function N(t){var e,i,s={};for(i in t)a(t,i)&&(e=k(i),e&&(s[e]=t[i]));return s}function I(t){var e,i;if(0===t.indexOf("week"))e=7,i="day";else{if(0!==t.indexOf("month"))return;e=12,i="month"}Ce[t]=function(s,o){var r,a,h=Ce._locale[t],d=[];if("number"==typeof s&&(o=s,s=n),a=function(t){var e=Ce().utc().set(i,t);return h.call(Ce._locale,e,s||"")},null!=o)return a(o);for(r=0;e>r;r++)d.push(a(r));return d}}function L(t){var e=+t,i=0;return 0!==e&&isFinite(e)&&(i=e>=0?Math.floor(e):Math.ceil(e)),i}function z(t,e){return new Date(Date.UTC(t,e+1,0)).getUTCDate()}function P(t,e,i){return me(Ce([t,11,31+e-i]),e,i).week}function A(t){return R(t)?366:365}function R(t){return t%4===0&&t%100!==0||t%400===0}function F(t){var e;t._a&&-2===t._pf.overflow&&(e=t._a[ze]<0||t._a[ze]>11?ze:t._a[Pe]<1||t._a[Pe]>z(t._a[Le],t._a[ze])?Pe:t._a[Ae]<0||t._a[Ae]>24||24===t._a[Ae]&&(0!==t._a[Re]||0!==t._a[Fe]||0!==t._a[He])?Ae:t._a[Re]<0||t._a[Re]>59?Re:t._a[Fe]<0||t._a[Fe]>59?Fe:t._a[He]<0||t._a[He]>999?He:-1,t._pf._overflowDayOfYear&&(Le>e||e>Pe)&&(e=Pe),t._pf.overflow=e)}function H(t){return null==t._isValid&&(t._isValid=!isNaN(t._d.getTime())&&t._pf.overflow<0&&!t._pf.empty&&!t._pf.invalidMonth&&!t._pf.nullInput&&!t._pf.invalidFormat&&!t._pf.userInvalidated,t._strict&&(t._isValid=t._isValid&&0===t._pf.charsLeftOver&&0===t._pf.unusedTokens.length&&t._pf.bigHour===n)),t._isValid}function B(t){return t?t.toLowerCase().replace("_","-"):t}function Y(t){for(var e,i,s,o,n=0;n0;){if(s=W(o.slice(0,e).join("-")))return s;if(i&&i.length>=e&&E(o,i,!0)>=e-1)break;e--}n++}return null}function W(t){var e=null;if(!Be[t]&&We)try{e=Ce.locale(),!function(){var t=new Error('Cannot find module "./locale"');throw t.code="MODULE_NOT_FOUND",t}(),Ce.locale(e)}catch(i){}return Be[t]}function G(t,e){var i,s;return e._isUTC?(i=e.clone(),s=(Ce.isMoment(t)||O(t)?+t:+Ce(t))-+i,i._d.setTime(+i._d+s),Ce.updateOffset(i,!1),i):Ce(t).local()}function j(t){return t.match(/\[[\s\S]/)?t.replace(/^\[|\]$/g,""):t.replace(/\\/g,"")}function U(t){var e,i,s=t.match(Ve);for(e=0,i=s.length;i>e;e++)s[e]=wi[s[e]]?wi[s[e]]:j(s[e]);return function(o){var n="";for(e=0;i>e;e++)n+=s[e]instanceof Function?s[e].call(o,t):s[e];return n}}function V(t,e){return t.isValid()?(e=X(e,t.localeData()),yi[e]||(yi[e]=U(e)),yi[e](t)):t.localeData().invalidDate()}function X(t,e){function i(t){return e.longDateFormat(t)||t}var s=5;for(Xe.lastIndex=0;s>=0&&Xe.test(t);)t=t.replace(Xe,i),Xe.lastIndex=0,s-=1;return t}function q(t,e){var i,s=e._strict;switch(t){case"Q":return oi;case"DDDD":return ri;case"YYYY":case"GGGG":case"gggg":return s?ai:Qe;case"Y":case"G":case"g":return di;case"YYYYYY":case"YYYYY":case"GGGGG":case"ggggg":return s?hi:Ke;case"S":if(s)return oi;case"SS":if(s)return ni;case"SSS":if(s)return ri;case"DDD":return Ze;case"MMM":case"MMMM":case"dd":case"ddd":case"dddd":return Je;case"a":case"A":return e._locale._meridiemParse;case"x":return ii;case"X":return si;case"Z":case"ZZ":return ti;case"T":return ei;case"SSSS":return $e;case"MM":case"DD":case"YY":case"GG":case"gg":case"HH":case"hh":case"mm":case"ss":case"ww":case"WW":return s?ni:qe;case"M":case"D":case"d":case"H":case"h":case"m":case"s":case"w":case"W":case"e":case"E":return qe;case"Do":return s?e._locale._ordinalParse:e._locale._ordinalParseLenient;default:return i=new RegExp(se(ie(t.replace("\\","")),"i"))}}function Z(t){t=t||"";var e=t.match(ti)||[],i=e[e.length-1]||[],s=(i+"").match(mi)||["-",0,0],o=+(60*s[1])+L(s[2]);return"+"===s[0]?o:-o}function Q(t,e,i){var s,o=i._a;switch(t){case"Q":null!=e&&(o[ze]=3*(L(e)-1));break;case"M":case"MM":null!=e&&(o[ze]=L(e)-1);break;case"MMM":case"MMMM":s=i._locale.monthsParse(e,t,i._strict),null!=s?o[ze]=s:i._pf.invalidMonth=e;break;case"D":case"DD":null!=e&&(o[Pe]=L(e));break;case"Do":null!=e&&(o[Pe]=L(parseInt(e.match(/\d{1,2}/)[0],10)));break;case"DDD":case"DDDD":null!=e&&(i._dayOfYear=L(e));break;case"YY":o[Le]=Ce.parseTwoDigitYear(e);break;case"YYYY":case"YYYYY":case"YYYYYY":o[Le]=L(e);break;case"a":case"A":i._meridiem=e;break;case"h":case"hh":i._pf.bigHour=!0;case"H":case"HH":o[Ae]=L(e);break;case"m":case"mm":o[Re]=L(e);break;case"s":case"ss":o[Fe]=L(e);break;case"S":case"SS":case"SSS":case"SSSS":o[He]=L(1e3*("0."+e));break;case"x":i._d=new Date(L(e));break;case"X":i._d=new Date(1e3*parseFloat(e));break;case"Z":case"ZZ":i._useUTC=!0,i._tzm=Z(e);break;case"dd":case"ddd":case"dddd":s=i._locale.weekdaysParse(e),null!=s?(i._w=i._w||{},i._w.d=s):i._pf.invalidWeekday=e;break;case"w":case"ww":case"W":case"WW":case"d":case"e":case"E":t=t.substr(0,1);case"gggg":case"GGGG":case"GGGGG":t=t.substr(0,2),e&&(i._w=i._w||{},i._w[t]=L(e));break;case"gg":case"GG":i._w=i._w||{},i._w[t]=Ce.parseTwoDigitYear(e)}}function K(t){var e,i,s,o,n,a,h;e=t._w,null!=e.GG||null!=e.W||null!=e.E?(n=1,a=4,i=r(e.GG,t._a[Le],me(Ce(),1,4).year),s=r(e.W,1),o=r(e.E,1)):(n=t._locale._week.dow,a=t._locale._week.doy,i=r(e.gg,t._a[Le],me(Ce(),n,a).year),s=r(e.w,1),null!=e.d?(o=e.d,n>o&&++s):o=null!=e.e?e.e+n:n),h=fe(i,s,o,a,n),t._a[Le]=h.year,t._dayOfYear=h.dayOfYear}function $(t){var e,i,s,o,n=[];if(!t._d){for(s=te(t),t._w&&null==t._a[Pe]&&null==t._a[ze]&&K(t),t._dayOfYear&&(o=r(t._a[Le],s[Le]),t._dayOfYear>A(o)&&(t._pf._overflowDayOfYear=!0),i=le(o,0,t._dayOfYear),t._a[ze]=i.getUTCMonth(),t._a[Pe]=i.getUTCDate()),e=0;3>e&&null==t._a[e];++e)t._a[e]=n[e]=s[e];for(;7>e;e++)t._a[e]=n[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[Ae]&&0===t._a[Re]&&0===t._a[Fe]&&0===t._a[He]&&(t._nextDay=!0,t._a[Ae]=0),t._d=(t._useUTC?le:de).apply(null,n),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[Ae]=24)}}function J(t){var e;t._d||(e=N(t._i),t._a=[e.year,e.month,e.day||e.date,e.hour,e.minute,e.second,e.millisecond],$(t))}function te(t){var e=new Date;return t._useUTC?[e.getUTCFullYear(),e.getUTCMonth(),e.getUTCDate()]:[e.getFullYear(),e.getMonth(),e.getDate()]}function ee(t){if(t._f===Ce.ISO_8601)return void ne(t);t._a=[],t._pf.empty=!0;var e,i,s,o,r,a=""+t._i,h=a.length,d=0;for(s=X(t._f,t._locale).match(Ve)||[],e=0;e0&&t._pf.unusedInput.push(r),a=a.slice(a.indexOf(i)+i.length),d+=i.length),wi[o]?(i?t._pf.empty=!1:t._pf.unusedTokens.push(o),Q(o,i,t)):t._strict&&!i&&t._pf.unusedTokens.push(o);t._pf.charsLeftOver=h-d,a.length>0&&t._pf.unusedInput.push(a),t._pf.bigHour===!0&&t._a[Ae]<=12&&(t._pf.bigHour=n),t._a[Ae]=f(t._locale,t._a[Ae],t._meridiem),$(t),F(t)}function ie(t){return t.replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,function(t,e,i,s,o){return e||i||s||o})}function se(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}function oe(t){var e,i,s,o,n;if(0===t._f.length)return t._pf.invalidFormat=!0,void(t._d=new Date(0/0));for(o=0;on)&&(s=n,i=e));b(t,i||e)}function ne(t){var e,i,s=t._i,o=li.exec(s);if(o){for(t._pf.iso=!0,e=0,i=pi.length;i>e;e++)if(pi[e][1].exec(s)){t._f=pi[e][0]+(o[6]||" ");break}for(e=0,i=ui.length;i>e;e++)if(ui[e][1].exec(s)){t._f+=ui[e][0];break}s.match(ti)&&(t._f+="Z"),ee(t)}else t._isValid=!1}function re(t){ne(t),t._isValid===!1&&(delete t._isValid,Ce.createFromInputFallback(t))}function ae(t,e){var i,s=[];for(i=0;it&&a.setFullYear(t),a}function le(t){var e=new Date(Date.UTC.apply(null,arguments));return 1970>t&&e.setUTCFullYear(t),e}function ce(t,e){if("string"==typeof t)if(isNaN(t)){if(t=e.weekdaysParse(t),"number"!=typeof t)return null}else t=parseInt(t,10);return t}function pe(t,e,i,s,o){return o.relativeTime(e||1,!!i,t,s)}function ue(t,e,i){var s=Ce.duration(t).abs(),o=Ne(s.as("s")),n=Ne(s.as("m")),r=Ne(s.as("h")),a=Ne(s.as("d")),h=Ne(s.as("M")),d=Ne(s.as("y")),l=o0,l[4]=i,pe.apply({},l)}function me(t,e,i){var s,o=i-e,n=i-t.day();return n>o&&(n-=7),o-7>n&&(n+=7),s=Ce(t).add(n,"d"),{week:Math.ceil(s.dayOfYear()/7),year:s.year()}}function fe(t,e,i,s,o){var n,r,a=le(t,0,1).getUTCDay();return a=0===a?7:a,i=null!=i?i:o,n=o-a+(a>s?7:0)-(o>a?7:0),r=7*(e-1)+(i-o)+n+1,{year:r>0?t:t-1,dayOfYear:r>0?r:A(t-1)+r}}function ge(t){var e,i=t._i,s=t._f;return t._locale=t._locale||Ce.localeData(t._l),null===i||s===n&&""===i?Ce.invalid({nullInput:!0}):("string"==typeof i&&(t._i=i=t._locale.preparse(i)),Ce.isMoment(i)?new v(i,!0):(s?T(s)?oe(t):ee(t):he(t),e=new v(t),e._nextDay&&(e.add(1,"d"),e._nextDay=n),e))}function ve(t,e){var i,s;if(1===e.length&&T(e[0])&&(e=e[0]),!e.length)return Ce();for(i=e[0],s=1;s=0?"+":"-";return e+w(Math.abs(t),6)},gg:function(){return w(this.weekYear()%100,2)},gggg:function(){return w(this.weekYear(),4)},ggggg:function(){return w(this.weekYear(),5)},GG:function(){return w(this.isoWeekYear()%100,2)},GGGG:function(){return w(this.isoWeekYear(),4)},GGGGG:function(){return w(this.isoWeekYear(),5)},e:function(){return this.weekday()},E:function(){return this.isoWeekday()},a:function(){return this.localeData().meridiem(this.hours(),this.minutes(),!0)},A:function(){return this.localeData().meridiem(this.hours(),this.minutes(),!1)},H:function(){return this.hours()},h:function(){return this.hours()%12||12},m:function(){return this.minutes()},s:function(){return this.seconds()},S:function(){return L(this.milliseconds()/100)},SS:function(){return w(L(this.milliseconds()/10),2)},SSS:function(){return w(this.milliseconds(),3)},SSSS:function(){return w(this.milliseconds(),3)},Z:function(){var t=this.utcOffset(),e="+";return 0>t&&(t=-t,e="-"),e+w(L(t/60),2)+":"+w(L(t)%60,2)},ZZ:function(){var t=this.utcOffset(),e="+";return 0>t&&(t=-t,e="-"),e+w(L(t/60),2)+w(L(t)%60,2)},z:function(){return this.zoneAbbr()},zz:function(){return this.zoneName()},x:function(){return this.valueOf()},X:function(){return this.unix()},Q:function(){return this.quarter()}},Si={},Mi=["months","monthsShort","weekdays","weekdaysShort","weekdaysMin"],Di=!1;_i.length;)Oe=_i.pop(),wi[Oe+"o"]=u(wi[Oe],Oe);for(;xi.length;)Oe=xi.pop(),wi[Oe+Oe]=p(wi[Oe],2);wi.DDDD=p(wi.DDD,3),b(g.prototype,{set:function(t){var e,i;for(i in t)e=t[i],"function"==typeof e?this[i]=e:this["_"+i]=e;this._ordinalParseLenient=new RegExp(this._ordinalParse.source+"|"+/\d{1,2}/.source)},_months:"January_February_March_April_May_June_July_August_September_October_November_December".split("_"),months:function(t){return this._months[t.month()]},_monthsShort:"Jan_Feb_Mar_Apr_May_Jun_Jul_Aug_Sep_Oct_Nov_Dec".split("_"),monthsShort:function(t){return this._monthsShort[t.month()]},monthsParse:function(t,e,i){var s,o,n;for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),s=0;12>s;s++){if(o=Ce.utc([2e3,s]),i&&!this._longMonthsParse[s]&&(this._longMonthsParse[s]=new RegExp("^"+this.months(o,"").replace(".","")+"$","i"),this._shortMonthsParse[s]=new RegExp("^"+this.monthsShort(o,"").replace(".","")+"$","i")),i||this._monthsParse[s]||(n="^"+this.months(o,"")+"|^"+this.monthsShort(o,""),this._monthsParse[s]=new RegExp(n.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[s].test(t))return s;if(i&&"MMM"===e&&this._shortMonthsParse[s].test(t))return s;if(!i&&this._monthsParse[s].test(t))return s}},_weekdays:"Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),weekdays:function(t){return this._weekdays[t.day()]},_weekdaysShort:"Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),weekdaysShort:function(t){return this._weekdaysShort[t.day()]},_weekdaysMin:"Su_Mo_Tu_We_Th_Fr_Sa".split("_"),weekdaysMin:function(t){return this._weekdaysMin[t.day()]},weekdaysParse:function(t){var e,i,s;for(this._weekdaysParse||(this._weekdaysParse=[]),e=0;7>e;e++)if(this._weekdaysParse[e]||(i=Ce([2e3,1]).day(e),s="^"+this.weekdays(i,"")+"|^"+this.weekdaysShort(i,"")+"|^"+this.weekdaysMin(i,""),this._weekdaysParse[e]=new RegExp(s.replace(".",""),"i")),this._weekdaysParse[e].test(t))return e},_longDateFormat:{LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY LT",LLLL:"dddd, MMMM D, YYYY LT"},longDateFormat:function(t){var e=this._longDateFormat[t]; -return!e&&this._longDateFormat[t.toUpperCase()]&&(e=this._longDateFormat[t.toUpperCase()].replace(/MMMM|MM|DD|dddd/g,function(t){return t.slice(1)}),this._longDateFormat[t]=e),e},isPM:function(t){return"p"===(t+"").toLowerCase().charAt(0)},_meridiemParse:/[ap]\.?m?\.?/i,meridiem:function(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"},_calendar:{sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},calendar:function(t,e,i){var s=this._calendar[t];return"function"==typeof s?s.apply(e,[i]):s},_relativeTime:{future:"in %s",past:"%s ago",s:"a few seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},relativeTime:function(t,e,i,s){var o=this._relativeTime[i];return"function"==typeof o?o(t,e,i,s):o.replace(/%d/i,t)},pastFuture:function(t,e){var i=this._relativeTime[t>0?"future":"past"];return"function"==typeof i?i(e):i.replace(/%s/i,e)},ordinal:function(t){return this._ordinal.replace("%d",t)},_ordinal:"%d",_ordinalParse:/\d{1,2}/,preparse:function(t){return t},postformat:function(t){return t},week:function(t){return me(t,this._week.dow,this._week.doy).week},_week:{dow:0,doy:6},firstDayOfWeek:function(){return this._week.dow},firstDayOfYear:function(){return this._week.doy},_invalidDate:"Invalid date",invalidDate:function(){return this._invalidDate}}),Ce=function(t,e,i,s){var o;return"boolean"==typeof i&&(s=i,i=n),o={},o._isAMomentObject=!0,o._i=t,o._f=e,o._l=i,o._strict=s,o._isUTC=!1,o._pf=h(),ge(o)},Ce.suppressDeprecationWarnings=!1,Ce.createFromInputFallback=l("moment construction falls back to js Date. This is discouraged and will be removed in upcoming major release. Please refer to https://github.com/moment/moment/issues/1407 for more info.",function(t){t._d=new Date(t._i+(t._useUTC?" UTC":""))}),Ce.min=function(){var t=[].slice.call(arguments,0);return ve("isBefore",t)},Ce.max=function(){var t=[].slice.call(arguments,0);return ve("isAfter",t)},Ce.utc=function(t,e,i,s){var o;return"boolean"==typeof i&&(s=i,i=n),o={},o._isAMomentObject=!0,o._useUTC=!0,o._isUTC=!0,o._l=i,o._i=t,o._f=e,o._strict=s,o._pf=h(),ge(o).utc()},Ce.unix=function(t){return Ce(1e3*t)},Ce.duration=function(t,e){var i,s,o,n,r=t,h=null;return Ce.isDuration(t)?r={ms:t._milliseconds,d:t._days,M:t._months}:"number"==typeof t?(r={},e?r[e]=t:r.milliseconds=t):(h=je.exec(t))?(i="-"===h[1]?-1:1,r={y:0,d:L(h[Pe])*i,h:L(h[Ae])*i,m:L(h[Re])*i,s:L(h[Fe])*i,ms:L(h[He])*i}):(h=Ue.exec(t))?(i="-"===h[1]?-1:1,o=function(t){var e=t&&parseFloat(t.replace(",","."));return(isNaN(e)?0:e)*i},r={y:o(h[2]),M:o(h[3]),d:o(h[4]),h:o(h[5]),m:o(h[6]),s:o(h[7]),w:o(h[8])}):null==r?r={}:"object"==typeof r&&("from"in r||"to"in r)&&(n=M(Ce(r.from),Ce(r.to)),r={},r.ms=n.milliseconds,r.M=n.months),s=new y(r),Ce.isDuration(t)&&a(t,"_locale")&&(s._locale=t._locale),s},Ce.version=Ee,Ce.defaultFormat=ci,Ce.ISO_8601=function(){},Ce.momentProperties=Ye,Ce.updateOffset=function(){},Ce.relativeTimeThreshold=function(t,e){return bi[t]===n?!1:e===n?bi[t]:(bi[t]=e,!0)},Ce.lang=l("moment.lang is deprecated. Use moment.locale instead.",function(t,e){return Ce.locale(t,e)}),Ce.locale=function(t,e){var i;return t&&(i="undefined"!=typeof e?Ce.defineLocale(t,e):Ce.localeData(t),i&&(Ce.duration._locale=Ce._locale=i)),Ce._locale._abbr},Ce.defineLocale=function(t,e){return null!==e?(e.abbr=t,Be[t]||(Be[t]=new g),Be[t].set(e),Ce.locale(t),Be[t]):(delete Be[t],null)},Ce.langData=l("moment.langData is deprecated. Use moment.localeData instead.",function(t){return Ce.localeData(t)}),Ce.localeData=function(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return Ce._locale;if(!T(t)){if(e=W(t))return e;t=[t]}return Y(t)},Ce.isMoment=function(t){return t instanceof v||null!=t&&a(t,"_isAMomentObject")},Ce.isDuration=function(t){return t instanceof y};for(Oe=Mi.length-1;Oe>=0;--Oe)I(Mi[Oe]);Ce.normalizeUnits=function(t){return k(t)},Ce.invalid=function(t){var e=Ce.utc(0/0);return null!=t?b(e._pf,t):e._pf.userInvalidated=!0,e},Ce.parseZone=function(){return Ce.apply(null,arguments).parseZone()},Ce.parseTwoDigitYear=function(t){return L(t)+(L(t)>68?1900:2e3)},Ce.isDate=O,b(Ce.fn=v.prototype,{clone:function(){return Ce(this)},valueOf:function(){return+this._d-6e4*(this._offset||0)},unix:function(){return Math.floor(+this/1e3)},toString:function(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")},toDate:function(){return this._offset?new Date(+this):this._d},toISOString:function(){var t=Ce(this).utc();return 00:!1},parsingFlags:function(){return b({},this._pf)},invalidAt:function(){return this._pf.overflow},utc:function(t){return this.utcOffset(0,t)},local:function(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(this._dateUtcOffset(),"m")),this},format:function(t){var e=V(this,t||Ce.defaultFormat);return this.localeData().postformat(e)},add:D(1,"add"),subtract:D(-1,"subtract"),diff:function(t,e,i){var s,o,n=G(t,this),r=6e4*(n.utcOffset()-this.utcOffset());return e=k(e),"year"===e||"month"===e||"quarter"===e?(o=m(this,n),"quarter"===e?o/=3:"year"===e&&(o/=12)):(s=this-n,o="second"===e?s/1e3:"minute"===e?s/6e4:"hour"===e?s/36e5:"day"===e?(s-r)/864e5:"week"===e?(s-r)/6048e5:s),i?o:x(o)},from:function(t,e){return Ce.duration({to:this,from:t}).locale(this.locale()).humanize(!e)},fromNow:function(t){return this.from(Ce(),t)},calendar:function(t){var e=t||Ce(),i=G(e,this).startOf("day"),s=this.diff(i,"days",!0),o=-6>s?"sameElse":-1>s?"lastWeek":0>s?"lastDay":1>s?"sameDay":2>s?"nextDay":7>s?"nextWeek":"sameElse";return this.format(this.localeData().calendar(o,this,Ce(e)))},isLeapYear:function(){return R(this.year())},isDST:function(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()},day:function(t){var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=ce(t,this.localeData()),this.add(t-e,"d")):e},month:xe("Month",!0),startOf:function(t){switch(t=k(t)){case"year":this.month(0);case"quarter":case"month":this.date(1);case"week":case"isoWeek":case"day":this.hours(0);case"hour":this.minutes(0);case"minute":this.seconds(0);case"second":this.milliseconds(0)}return"week"===t?this.weekday(0):"isoWeek"===t&&this.isoWeekday(1),"quarter"===t&&this.month(3*Math.floor(this.month()/3)),this},endOf:function(t){return t=k(t),t===n||"millisecond"===t?this:this.startOf(t).add(1,"isoWeek"===t?"week":t).subtract(1,"ms")},isAfter:function(t,e){var i;return e=k("undefined"!=typeof e?e:"millisecond"),"millisecond"===e?(t=Ce.isMoment(t)?t:Ce(t),+this>+t):(i=Ce.isMoment(t)?+t:+Ce(t),i<+this.clone().startOf(e))},isBefore:function(t,e){var i;return e=k("undefined"!=typeof e?e:"millisecond"),"millisecond"===e?(t=Ce.isMoment(t)?t:Ce(t),+t>+this):(i=Ce.isMoment(t)?+t:+Ce(t),+this.clone().endOf(e)t?this:t}),max:l("moment().max is deprecated, use moment.max instead. https://github.com/moment/moment/issues/1548",function(t){return t=Ce.apply(null,arguments),t>this?this:t}),zone:l("moment().zone is deprecated, use moment().utcOffset instead. https://github.com/moment/moment/issues/1779",function(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()}),utcOffset:function(t,e){var i,s=this._offset||0;return null!=t?("string"==typeof t&&(t=Z(t)),Math.abs(t)<16&&(t=60*t),!this._isUTC&&e&&(i=this._dateUtcOffset()),this._offset=t,this._isUTC=!0,null!=i&&this.add(i,"m"),s!==t&&(!e||this._changeInProgress?C(this,Ce.duration(t-s,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,Ce.updateOffset(this,!0),this._changeInProgress=null)),this):this._isUTC?s:this._dateUtcOffset()},isLocal:function(){return!this._isUTC},isUtcOffset:function(){return this._isUTC},isUtc:function(){return this._isUTC&&0===this._offset},zoneAbbr:function(){return this._isUTC?"UTC":""},zoneName:function(){return this._isUTC?"Coordinated Universal Time":""},parseZone:function(){return this._tzm?this.utcOffset(this._tzm):"string"==typeof this._i&&this.utcOffset(Z(this._i)),this},hasAlignedHourOffset:function(t){return t=t?Ce(t).utcOffset():0,(this.utcOffset()-t)%60===0},daysInMonth:function(){return z(this.year(),this.month())},dayOfYear:function(t){var e=Ne((Ce(this).startOf("day")-Ce(this).startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")},quarter:function(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)},weekYear:function(t){var e=me(this,this.localeData()._week.dow,this.localeData()._week.doy).year;return null==t?e:this.add(t-e,"y")},isoWeekYear:function(t){var e=me(this,1,4).year;return null==t?e:this.add(t-e,"y")},week:function(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")},isoWeek:function(t){var e=me(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")},weekday:function(t){var e=(this.day()+7-this.localeData()._week.dow)%7;return null==t?e:this.add(t-e,"d")},isoWeekday:function(t){return null==t?this.day()||7:this.day(this.day()%7?t:t-7)},isoWeeksInYear:function(){return P(this.year(),1,4)},weeksInYear:function(){var t=this.localeData()._week;return P(this.year(),t.dow,t.doy)},get:function(t){return t=k(t),this[t]()},set:function(t,e){var i;if("object"==typeof t)for(i in t)this.set(i,t[i]);else t=k(t),"function"==typeof this[t]&&this[t](e);return this},locale:function(t){var e;return t===n?this._locale._abbr:(e=Ce.localeData(t),null!=e&&(this._locale=e),this)},lang:l("moment().lang() is deprecated. Instead, use moment().localeData() to get the language configuration. Use moment().locale() to change languages.",function(t){return t===n?this.localeData():this.locale(t)}),localeData:function(){return this._locale},_dateUtcOffset:function(){return 15*-Math.round(this._d.getTimezoneOffset()/15)}}),Ce.fn.millisecond=Ce.fn.milliseconds=xe("Milliseconds",!1),Ce.fn.second=Ce.fn.seconds=xe("Seconds",!1),Ce.fn.minute=Ce.fn.minutes=xe("Minutes",!1),Ce.fn.hour=Ce.fn.hours=xe("Hours",!0),Ce.fn.date=xe("Date",!0),Ce.fn.dates=l("dates accessor is deprecated. Use date instead.",xe("Date",!0)),Ce.fn.year=xe("FullYear",!0),Ce.fn.years=l("years accessor is deprecated. Use year instead.",xe("FullYear",!0)),Ce.fn.days=Ce.fn.day,Ce.fn.months=Ce.fn.month,Ce.fn.weeks=Ce.fn.week,Ce.fn.isoWeeks=Ce.fn.isoWeek,Ce.fn.quarters=Ce.fn.quarter,Ce.fn.toJSON=Ce.fn.toISOString,Ce.fn.isUTC=Ce.fn.isUtc,b(Ce.duration.fn=y.prototype,{_bubble:function(){var t,e,i,s=this._milliseconds,o=this._days,n=this._months,r=this._data,a=0;r.milliseconds=s%1e3,t=x(s/1e3),r.seconds=t%60,e=x(t/60),r.minutes=e%60,i=x(e/60),r.hours=i%24,o+=x(i/24),a=x(we(o)),o-=x(Se(a)),n+=x(o/30),o%=30,a+=x(n/12),n%=12,r.days=o,r.months=n,r.years=a},abs:function(){return this._milliseconds=Math.abs(this._milliseconds),this._days=Math.abs(this._days),this._months=Math.abs(this._months),this._data.milliseconds=Math.abs(this._data.milliseconds),this._data.seconds=Math.abs(this._data.seconds),this._data.minutes=Math.abs(this._data.minutes),this._data.hours=Math.abs(this._data.hours),this._data.months=Math.abs(this._data.months),this._data.years=Math.abs(this._data.years),this},weeks:function(){return x(this.days()/7)},valueOf:function(){return this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*L(this._months/12)},humanize:function(t){var e=ue(this,!t,this.localeData());return t&&(e=this.localeData().pastFuture(+this,e)),this.localeData().postformat(e)},add:function(t,e){var i=Ce.duration(t,e);return this._milliseconds+=i._milliseconds,this._days+=i._days,this._months+=i._months,this._bubble(),this},subtract:function(t,e){var i=Ce.duration(t,e);return this._milliseconds-=i._milliseconds,this._days-=i._days,this._months-=i._months,this._bubble(),this},get:function(t){return t=k(t),this[t.toLowerCase()+"s"]()},as:function(t){var e,i;if(t=k(t),"month"===t||"year"===t)return e=this._days+this._milliseconds/864e5,i=this._months+12*we(e),"month"===t?i:i/12;switch(e=this._days+Math.round(Se(this._months/12)),t){case"week":return e/7+this._milliseconds/6048e5;case"day":return e+this._milliseconds/864e5;case"hour":return 24*e+this._milliseconds/36e5;case"minute":return 24*e*60+this._milliseconds/6e4;case"second":return 24*e*60*60+this._milliseconds/1e3;case"millisecond":return Math.floor(24*e*60*60*1e3)+this._milliseconds;default:throw new Error("Unknown unit "+t)}},lang:Ce.fn.lang,locale:Ce.fn.locale,toIsoString:l("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",function(){return this.toISOString()}),toISOString:function(){var t=Math.abs(this.years()),e=Math.abs(this.months()),i=Math.abs(this.days()),s=Math.abs(this.hours()),o=Math.abs(this.minutes()),n=Math.abs(this.seconds()+this.milliseconds()/1e3);return this.asSeconds()?(this.asSeconds()<0?"-":"")+"P"+(t?t+"Y":"")+(e?e+"M":"")+(i?i+"D":"")+(s||o||n?"T":"")+(s?s+"H":"")+(o?o+"M":"")+(n?n+"S":""):"P0D"},localeData:function(){return this._locale},toJSON:function(){return this.toISOString()}}),Ce.duration.fn.toString=Ce.duration.fn.toISOString;for(Oe in fi)a(fi,Oe)&&Me(Oe.toLowerCase());Ce.duration.fn.asMilliseconds=function(){return this.as("ms")},Ce.duration.fn.asSeconds=function(){return this.as("s")},Ce.duration.fn.asMinutes=function(){return this.as("m")},Ce.duration.fn.asHours=function(){return this.as("h")},Ce.duration.fn.asDays=function(){return this.as("d")},Ce.duration.fn.asWeeks=function(){return this.as("weeks")},Ce.duration.fn.asMonths=function(){return this.as("M")},Ce.duration.fn.asYears=function(){return this.as("y")},Ce.locale("en",{ordinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10,i=1===L(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th";return t+i}}),We?o.exports=Ce:(s=function(t,e,i){return i.config&&i.config()&&i.config().noGlobal===!0&&(ke.moment=Te),Ce}.call(e,i,e,o),!(s!==n&&(o.exports=s)),De(!0))}).call(this)}).call(e,function(){return this}(),i(72)(t))},function(t,e){var i,s,o;!function(n,r){s=[],i=r,o="function"==typeof i?i.apply(e,s):i,!(void 0!==o&&(t.exports=o))}(this,function(){function t(t){var e,i=t&&t.preventDefault||!1,s=t&&t.container||window,o={},n={keydown:{},keyup:{}},r={};for(e=97;122>=e;e++)r[String.fromCharCode(e)]={code:65+(e-97),shift:!1};for(e=65;90>=e;e++)r[String.fromCharCode(e)]={code:e,shift:!0};for(e=0;9>=e;e++)r[""+e]={code:48+e,shift:!1};for(e=1;12>=e;e++)r["F"+e]={code:111+e,shift:!1};for(e=0;9>=e;e++)r["num"+e]={code:96+e,shift:!1};r["num*"]={code:106,shift:!1},r["num+"]={code:107,shift:!1},r["num-"]={code:109,shift:!1},r["num/"]={code:111,shift:!1},r["num."]={code:110,shift:!1},r.left={code:37,shift:!1},r.up={code:38,shift:!1},r.right={code:39,shift:!1},r.down={code:40,shift:!1},r.space={code:32,shift:!1},r.enter={code:13,shift:!1},r.shift={code:16,shift:void 0},r.esc={code:27,shift:!1},r.backspace={code:8,shift:!1},r.tab={code:9,shift:!1},r.ctrl={code:17,shift:!1},r.alt={code:18,shift:!1},r["delete"]={code:46,shift:!1},r.pageup={code:33,shift:!1},r.pagedown={code:34,shift:!1},r["="]={code:187,shift:!1},r["-"]={code:189,shift:!1},r["]"]={code:221,shift:!1},r["["]={code:219,shift:!1};var a=function(t){d(t,"keydown")},h=function(t){d(t,"keyup")},d=function(t,e){if(void 0!==n[e][t.keyCode]){for(var s=n[e][t.keyCode],o=0;o0?i._handlers[t]=s:(i._off(t,o),delete i._handlers[t]))}),i},i.destroy=function(){var t=i.element;delete t.hammer,i._handlers={},i._destroy()},i}})},function(t,e,i){var s;!function(o,n,r,a){function h(t,e,i){return setTimeout(m(t,i),e)}function d(t,e,i){return Array.isArray(t)?(l(t,i[e],i),!0):!1}function l(t,e,i){var s;if(t)if(t.forEach)t.forEach(e,i);else if(t.length!==a)for(s=0;s-1}function x(t){return t.trim().split(/\s+/g)}function w(t,e,i){if(t.indexOf&&!i)return t.indexOf(e);for(var s=0;si[e]}):s.sort()),s}function D(t,e){for(var i,s,o=e[0].toUpperCase()+e.slice(1),n=0;n1&&!i.firstMultiple?i.firstMultiple=z(e):1===o&&(i.firstMultiple=!1);var n=i.firstInput,r=i.firstMultiple,a=r?r.center:n.center,h=e.center=P(s);e.timeStamp=ve(),e.deltaTime=e.timeStamp-n.timeStamp,e.angle=H(a,h),e.distance=F(a,h),I(i,e),e.offsetDirection=R(e.deltaX,e.deltaY),e.scale=r?Y(r.pointers,s):1,e.rotation=r?B(r.pointers,s):0,L(i,e);var d=t.element;b(e.srcEvent.target,d)&&(d=e.srcEvent.target),e.target=d}function I(t,e){var i=e.center,s=t.offsetDelta||{},o=t.prevDelta||{},n=t.prevInput||{};(e.eventType===Oe||n.eventType===ke)&&(o=t.prevDelta={x:n.deltaX||0,y:n.deltaY||0},s=t.offsetDelta={x:i.x,y:i.y}),e.deltaX=o.x+(i.x-s.x),e.deltaY=o.y+(i.y-s.y)}function L(t,e){var i,s,o,n,r=t.lastInterval||e,h=e.timeStamp-r.timeStamp;if(e.eventType!=Ne&&(h>Te||r.velocity===a)){var d=r.deltaX-e.deltaX,l=r.deltaY-e.deltaY,c=A(h,d,l);s=c.x,o=c.y,i=ge(c.x)>ge(c.y)?c.x:c.y,n=R(d,l),t.lastInterval=e}else i=r.velocity,s=r.velocityX,o=r.velocityY,n=r.direction;e.velocity=i,e.velocityX=s,e.velocityY=o,e.direction=n}function z(t){for(var e=[],i=0;io;)i+=t[o].clientX,s+=t[o].clientY,o++;return{x:fe(i/e),y:fe(s/e)}}function A(t,e,i){return{x:e/t||0,y:i/t||0}}function R(t,e){return t===e?Ie:ge(t)>=ge(e)?t>0?Le:ze:e>0?Pe:Ae}function F(t,e,i){i||(i=Be);var s=e[i[0]]-t[i[0]],o=e[i[1]]-t[i[1]];return Math.sqrt(s*s+o*o)}function H(t,e,i){i||(i=Be);var s=e[i[0]]-t[i[0]],o=e[i[1]]-t[i[1]];return 180*Math.atan2(o,s)/Math.PI}function B(t,e){return H(e[1],e[0],Ye)-H(t[1],t[0],Ye)}function Y(t,e){return F(e[0],e[1],Ye)/F(t[0],t[1],Ye)}function W(){this.evEl=Ge,this.evWin=je,this.allow=!0,this.pressed=!1,O.apply(this,arguments)}function G(){this.evEl=Xe,this.evWin=qe,O.apply(this,arguments),this.store=this.manager.session.pointerEvents=[]}function j(){this.evTarget=Qe,this.evWin=Ke,this.started=!1,O.apply(this,arguments)}function U(t,e){var i=S(t.touches),s=S(t.changedTouches);return e&(ke|Ne)&&(i=M(i.concat(s),"identifier",!0)),[i,s]}function V(){this.evTarget=Je,this.targetIds={},O.apply(this,arguments)}function X(t,e){var i=S(t.touches),s=this.targetIds;if(e&(Oe|Ee)&&1===i.length)return s[i[0].identifier]=!0,[i,i];var o,n,r=S(t.changedTouches),a=[],h=this.target;if(n=i.filter(function(t){return b(t.target,h)}),e===Oe)for(o=0;oa&&(e.push(t),a=e.length-1):o&(ke|Ne)&&(i=!0),0>a||(e[a]=t,this.callback(this.manager,o,{pointers:e,changedPointers:[t],pointerType:n,srcEvent:t}),i&&e.splice(a,1))}});var Ze={touchstart:Oe,touchmove:Ee,touchend:ke,touchcancel:Ne},Qe="touchstart",Ke="touchstart touchmove touchend touchcancel";u(j,O,{handler:function(t){var e=Ze[t.type];if(e===Oe&&(this.started=!0),this.started){var i=U.call(this,t,e);e&(ke|Ne)&&i[0].length-i[1].length===0&&(this.started=!1),this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Se,srcEvent:t})}}});var $e={touchstart:Oe,touchmove:Ee,touchend:ke,touchcancel:Ne},Je="touchstart touchmove touchend touchcancel";u(V,O,{handler:function(t){var e=$e[t.type],i=X.call(this,t,e);i&&this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Se,srcEvent:t})}}),u(q,O,{handler:function(t,e,i){var s=i.pointerType==Se,o=i.pointerType==De;if(s)this.mouse.allow=!1;else if(o&&!this.mouse.allow)return;e&(ke|Ne)&&(this.mouse.allow=!0),this.callback(t,e,i)},destroy:function(){this.touch.destroy(),this.mouse.destroy()}});var ti=D(ue.style,"touchAction"),ei=ti!==a,ii="compute",si="auto",oi="manipulation",ni="none",ri="pan-x",ai="pan-y";Z.prototype={set:function(t){t==ii&&(t=this.compute()),ei&&(this.manager.element.style[ti]=t),this.actions=t.toLowerCase().trim()},update:function(){this.set(this.manager.options.touchAction)},compute:function(){var t=[];return l(this.manager.recognizers,function(e){f(e.options.enable,[e])&&(t=t.concat(e.getTouchAction()))}),Q(t.join(" "))},preventDefaults:function(t){if(!ei){var e=t.srcEvent,i=t.offsetDirection;if(this.manager.session.prevented)return void e.preventDefault();var s=this.actions,o=_(s,ni),n=_(s,ai),r=_(s,ri);return o||n&&i&Re||r&&i&Fe?this.preventSrc(e):void 0}},preventSrc:function(t){this.manager.session.prevented=!0,t.preventDefault()}};var hi=1,di=2,li=4,ci=8,pi=ci,ui=16,mi=32;K.prototype={defaults:{},set:function(t){return c(this.options,t),this.manager&&this.manager.touchAction.update(),this},recognizeWith:function(t){if(d(t,"recognizeWith",this))return this;var e=this.simultaneous;return t=te(t,this),e[t.id]||(e[t.id]=t,t.recognizeWith(this)),this},dropRecognizeWith:function(t){return d(t,"dropRecognizeWith",this)?this:(t=te(t,this),delete this.simultaneous[t.id],this)},requireFailure:function(t){if(d(t,"requireFailure",this))return this;var e=this.requireFail;return t=te(t,this),-1===w(e,t)&&(e.push(t),t.requireFailure(this)),this},dropRequireFailure:function(t){if(d(t,"dropRequireFailure",this))return this;t=te(t,this);var e=w(this.requireFail,t);return e>-1&&this.requireFail.splice(e,1),this},hasRequireFailures:function(){return this.requireFail.length>0},canRecognizeWith:function(t){return!!this.simultaneous[t.id]},emit:function(t){function e(e){i.manager.emit(i.options.event+(e?$(s):""),t)}var i=this,s=this.state;ci>s&&e(!0),e(),s>=ci&&e(!0)},tryEmit:function(t){return this.canEmit()?this.emit(t):void(this.state=mi)},canEmit:function(){for(var t=0;tn?Le:ze,i=n!=this.pX,s=Math.abs(t.deltaX)):(o=0===r?Ie:0>r?Pe:Ae,i=r!=this.pY,s=Math.abs(t.deltaY))),t.direction=o,i&&s>e.threshold&&o&e.direction},attrTest:function(t){return ee.prototype.attrTest.call(this,t)&&(this.state&di||!(this.state&di)&&this.directionTest(t))},emit:function(t){this.pX=t.deltaX,this.pY=t.deltaY;var e=J(t.direction);e&&this.manager.emit(this.options.event+e,t),this._super.emit.call(this,t)}}),u(se,ee,{defaults:{event:"pinch",threshold:0,pointers:2},getTouchAction:function(){return[ni]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.scale-1)>this.options.threshold||this.state&di)},emit:function(t){if(this._super.emit.call(this,t),1!==t.scale){var e=t.scale<1?"in":"out";this.manager.emit(this.options.event+e,t)}}}),u(oe,K,{defaults:{event:"press",pointers:1,time:500,threshold:5},getTouchAction:function(){return[si]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,s=t.distancee.time;if(this._input=t,!s||!i||t.eventType&(ke|Ne)&&!o)this.reset();else if(t.eventType&Oe)this.reset(),this._timer=h(function(){this.state=pi,this.tryEmit() -},e.time,this);else if(t.eventType&ke)return pi;return mi},reset:function(){clearTimeout(this._timer)},emit:function(t){this.state===pi&&(t&&t.eventType&ke?this.manager.emit(this.options.event+"up",t):(this._input.timeStamp=ve(),this.manager.emit(this.options.event,this._input)))}}),u(ne,ee,{defaults:{event:"rotate",threshold:0,pointers:2},getTouchAction:function(){return[ni]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.rotation)>this.options.threshold||this.state&di)}}),u(re,ee,{defaults:{event:"swipe",threshold:10,velocity:.65,direction:Re|Fe,pointers:1},getTouchAction:function(){return ie.prototype.getTouchAction.call(this)},attrTest:function(t){var e,i=this.options.direction;return i&(Re|Fe)?e=t.velocity:i&Re?e=t.velocityX:i&Fe&&(e=t.velocityY),this._super.attrTest.call(this,t)&&i&t.direction&&t.distance>this.options.threshold&&ge(e)>this.options.velocity&&t.eventType&ke},emit:function(t){var e=J(t.direction);e&&this.manager.emit(this.options.event+e,t),this.manager.emit(this.options.event,t)}}),u(ae,K,{defaults:{event:"tap",pointers:1,taps:1,interval:300,time:250,threshold:2,posThreshold:10},getTouchAction:function(){return[oi]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,s=t.distancet&&s>o;)o%3==0?(this.forceAggregateHubs(!0),this.normalizeClusterLevels()):this.increaseClusterLevel(),i=this.nodeIndices.length,o+=1;o>0&&1==e&&this.repositionNodes(),this._updateCalculationNodes()},e.openCluster=function(t){var e=this.moving;if(t.clusterSize>this.constants.clustering.sectorThreshold&&this._nodeInActiveArea(t)&&("default"!=this._sector()||1!=this.nodeIndices.length)){this._addSector(t);for(var i=0;this.nodeIndices.lengthi;)this.decreaseClusterLevel(),i+=1}else this._expandClusterNode(t,!1,!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this._updateCalculationNodes(),this.updateLabels();this.moving!=e&&this.start()},e.updateClustersDefault=function(){1==this.constants.clustering.enabled&&this.updateClusters(0,!1,!1)},e.increaseClusterLevel=function(){this.updateClusters(-1,!1,!0)},e.decreaseClusterLevel=function(){this.updateClusters(1,!1,!0)},e.updateClusters=function(t,e,i,s){var o=this.moving,n=this.nodeIndices.length;this.previousScale>this.scale&&0==t&&this._collapseSector(),this.previousScale>this.scale||-1==t?this._formClusters(i):(this.previousScalethis.scale||-1==t)&&(this._aggregateHubs(i),this._updateNodeIndexList()),(this.previousScale>this.scale||-1==t)&&(this.handleChains(),this._updateNodeIndexList()),this.previousScale=this.scale,this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.lengththis.constants.clustering.chainThreshold&&this._reduceAmountOfChains(1-this.constants.clustering.chainThreshold/t)},e._aggregateHubs=function(t){this._getHubSize(),this._formClustersByHub(t,!1)},e.forceAggregateHubs=function(t){var e=this.moving,i=this.nodeIndices.length;this._aggregateHubs(!0),this._updateNodeIndexList(),this._updateDynamicEdges(),this.updateLabels(),this.nodeIndices.length!=i&&(this.clusterSession+=1),(0==t||void 0===t)&&this.moving!=e&&this.start()},e._openClustersBySize=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];1==e.inView()&&(e.width*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientWidth||e.height*this.scale>this.constants.clustering.screenSizeThreshold*this.frame.canvas.clientHeight)&&this.openCluster(e)}},e._openClusters=function(t,e){for(var i=0;i1&&(t.clusterSizei)){var r=n.from,a=n.to;n.to.options.mass>n.from.options.mass&&(r=n.to,a=n.from),1==a.dynamicEdgesLength?this._addToCluster(r,a,!1):1==r.dynamicEdgesLength&&this._addToCluster(a,r,!1)}}},e._forceClustersByZoom=function(){for(var t in this.nodes)if(this.nodes.hasOwnProperty(t)){var e=this.nodes[t];if(1==e.dynamicEdgesLength&&0!=e.dynamicEdges.length){var i=e.dynamicEdges[0],s=i.toId==e.id?this.nodes[i.fromId]:this.nodes[i.toId];e.id!=s.id&&(s.options.mass>e.options.mass?this._addToCluster(s,e,!0):this._addToCluster(e,s,!0))}}},e._clusterToSmallestNeighbour=function(t){for(var e=-1,i=null,s=0;so.clusterSessions.length&&(e=o.clusterSessions.length,i=o)}null!=o&&void 0!==this.nodes[o.id]&&this._addToCluster(o,t,!0)},e._formClustersByHub=function(t,e){for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&this._formClusterFromHub(this.nodes[i],t,e)},e._formClusterFromHub=function(t,e,i,s){if(void 0===s&&(s=0),t.dynamicEdgesLength>=this.hubThreshold&&0==i||t.dynamicEdgesLength==this.hubThreshold&&1==i){for(var o,n,r,a=this.constants.clustering.clusterEdgeThreshold/this.scale,h=!1,d=[],l=t.dynamicEdges.length,c=0;l>c;c++)d.push(t.dynamicEdges[c].id);if(0==e)for(h=!1,c=0;l>c;c++){var p=this.edges[d[c]];if(void 0!==p&&p.connected&&p.toId!=p.fromId&&(o=p.to.x-p.from.x,n=p.to.y-p.from.y,r=Math.sqrt(o*o+n*n),a>r)){h=!0;break}}if(!e&&h||e)for(c=0;l>c;c++)if(p=this.edges[d[c]],void 0!==p){var u=this.nodes[p.fromId==t.id?p.toId:p.fromId];u.dynamicEdges.length<=this.hubThreshold+s&&u.id!=t.id&&this._addToCluster(t,u,e)}}},e._addToCluster=function(t,e,i){t.containedNodes[e.id]=e;for(var s=0;s1)for(var s=0;s1&&(e.label="[".concat(String(e.clusterSize),"]"))}for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(e=this.nodes[t],1==e.clusterSize&&(e.label=void 0!==e.originalLabel?e.originalLabel:String(e.id)))},e.normalizeClusterLevels=function(){var t,e=0,i=1e9,s=0;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&(s=this.nodes[t].clusterSessions.length,s>e&&(e=s),i>s&&(i=s));if(e-i>this.constants.clustering.clusterLevelDifference){var o=this.nodeIndices.length,n=e-this.constants.clustering.clusterLevelDifference;for(t in this.nodes)this.nodes.hasOwnProperty(t)&&this.nodes[t].clusterSessions.lengths&&(s=n.dynamicEdgesLength),t+=n.dynamicEdgesLength,e+=Math.pow(n.dynamicEdgesLength,2),i+=1}t/=i,e/=i;var r=e-Math.pow(t,2),a=Math.sqrt(r);this.hubThreshold=Math.floor(t+2*a),this.hubThreshold>s&&(this.hubThreshold=s)},e._reduceAmountOfChains=function(t){this.hubThreshold=2;var e=Math.floor(this.nodeIndices.length*t);for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&e>0&&(this._formClusterFromHub(this.nodes[i],!0,!0,1),e-=1)},e._getChainFraction=function(){var t=0,e=0;for(var i in this.nodes)this.nodes.hasOwnProperty(i)&&(2==this.nodes[i].dynamicEdgesLength&&this.nodes[i].dynamicEdges.length>=2&&(t+=1),e+=1);return t/e}},function(t,e,i){var s=i(1),o=i(40);e._putDataInSector=function(){this.sectors.active[this._sector()].nodes=this.nodes,this.sectors.active[this._sector()].edges=this.edges,this.sectors.active[this._sector()].nodeIndices=this.nodeIndices},e._switchToSector=function(t,e){void 0===e||"active"==e?this._switchToActiveSector(t):this._switchToFrozenSector(t)},e._switchToActiveSector=function(t){this.nodeIndices=this.sectors.active[t].nodeIndices,this.nodes=this.sectors.active[t].nodes,this.edges=this.sectors.active[t].edges},e._switchToSupportSector=function(){this.nodeIndices=this.sectors.support.nodeIndices,this.nodes=this.sectors.support.nodes,this.edges=this.sectors.support.edges},e._switchToFrozenSector=function(t){this.nodeIndices=this.sectors.frozen[t].nodeIndices,this.nodes=this.sectors.frozen[t].nodes,this.edges=this.sectors.frozen[t].edges},e._loadLatestSector=function(){this._switchToSector(this._sector())},e._sector=function(){return this.activeSector[this.activeSector.length-1]},e._previousSector=function(){if(this.activeSector.length>1)return this.activeSector[this.activeSector.length-2];throw new TypeError("there are not enough sectors in the this.activeSector array.")},e._setActiveSector=function(t){this.activeSector.push(t)},e._forgetLastSector=function(){this.activeSector.pop()},e._createNewSector=function(t){this.sectors.active[t]={nodes:{},edges:{},nodeIndices:[],formationScale:this.scale,drawingNode:void 0},this.sectors.active[t].drawingNode=new o({id:t,color:{background:"#eaefef",border:"495c5e"}},{},{},this.constants),this.sectors.active[t].drawingNode.clusterSize=2},e._deleteActiveSector=function(t){delete this.sectors.active[t]},e._deleteFrozenSector=function(t){delete this.sectors.frozen[t]},e._freezeSector=function(t){this.sectors.frozen[t]=this.sectors.active[t],this._deleteActiveSector(t)},e._activateSector=function(t){this.sectors.active[t]=this.sectors.frozen[t],this._deleteFrozenSector(t)},e._mergeThisWithFrozen=function(t){for(var e in this.nodes)this.nodes.hasOwnProperty(e)&&(this.sectors.frozen[t].nodes[e]=this.nodes[e]);for(var i in this.edges)this.edges.hasOwnProperty(i)&&(this.sectors.frozen[t].edges[i]=this.edges[i]);for(var s=0;s1?this[t](o[0],o[1]):this[t](e))}return this._loadLatestSector(),i},e._doInSupportSector=function(t,e){var i=!1;if(void 0===e)this._switchToSupportSector(),i=this[t]();else{this._switchToSupportSector();var s=Array.prototype.splice.call(arguments,1);i=s.length>1?this[t](s[0],s[1]):this[t](e)}return this._loadLatestSector(),i},e._doInAllFrozenSectors=function(t,e){if(void 0===e)for(var i in this.sectors.frozen)this.sectors.frozen.hasOwnProperty(i)&&(this._switchToFrozenSector(i),this[t]());else for(var i in this.sectors.frozen)if(this.sectors.frozen.hasOwnProperty(i)){this._switchToFrozenSector(i);var s=Array.prototype.splice.call(arguments,1);s.length>1?this[t](s[0],s[1]):this[t](e)}this._loadLatestSector()},e._doInAllSectors=function(t,e){var i=Array.prototype.splice.call(arguments,1);void 0===e?(this._doInAllActiveSectors(t),this._doInAllFrozenSectors(t)):i.length>1?(this._doInAllActiveSectors(t,i[0],i[1]),this._doInAllFrozenSectors(t,i[0],i[1])):(this._doInAllActiveSectors(t,e),this._doInAllFrozenSectors(t,e))},e._clearNodeIndexList=function(){var t=this._sector();this.sectors.active[t].nodeIndices=[],this.nodeIndices=this.sectors.active[t].nodeIndices},e._drawSectorNodes=function(t,e){var i,s=1e9,o=-1e9,n=1e9,r=-1e9;for(var a in this.sectors[e])if(this.sectors[e].hasOwnProperty(a)&&void 0!==this.sectors[e][a].drawingNode){this._switchToSector(a,e),s=1e9,o=-1e9,n=1e9,r=-1e9;for(var h in this.nodes)this.nodes.hasOwnProperty(h)&&(i=this.nodes[h],i.resize(t),n>i.x-.5*i.width&&(n=i.x-.5*i.width),ri.y-.5*i.height&&(s=i.y-.5*i.height),o0?this.nodes[i[i.length-1]]:null},e._getEdgesOverlappingWith=function(t,e){var i=this.edges;for(var s in i)i.hasOwnProperty(s)&&i[s].isOverlappingWith(t)&&e.push(s)},e._getAllEdgesOverlappingWith=function(t){var e=[];return this._doInAllActiveSectors("_getEdgesOverlappingWith",t,e),e},e._getEdgeAt=function(t){var e=this._pointerToPositionObject(t),i=this._getAllEdgesOverlappingWith(e);return i.length>0?this.edges[i[i.length-1]]:null},e._addToSelection=function(t){t instanceof s?this.selectionObj.nodes[t.id]=t:this.selectionObj.edges[t.id]=t},e._addToHover=function(t){t instanceof s?this.hoverObj.nodes[t.id]=t:this.hoverObj.edges[t.id]=t},e._removeFromSelection=function(t){t instanceof s?delete this.selectionObj.nodes[t.id]:delete this.selectionObj.edges[t.id]},e._unselectAll=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].unselect();for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&this.selectionObj.edges[i].unselect();this.selectionObj={nodes:{},edges:{}},0==t&&this.emit("select",this.getSelection())},e._unselectClusters=function(t){void 0===t&&(t=!1);for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&this.selectionObj.nodes[e].clusterSize>1&&(this.selectionObj.nodes[e].unselect(),this._removeFromSelection(this.selectionObj.nodes[e]));0==t&&this.emit("select",this.getSelection())},e._getSelectedNodeCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedNode=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return this.selectionObj.nodes[t];return null},e._getSelectedEdge=function(){for(var t in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(t))return this.selectionObj.edges[t];return null},e._getSelectedEdgeCount=function(){var t=0;for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(t+=1);return t},e._getSelectedObjectCount=function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&(t+=1);return t},e._selectionIsEmpty=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return!1;for(var e in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(e))return!1;return!0},e._clusterInSelection=function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].clusterSize>1)return!0;return!1},e._selectConnectedEdges=function(t){for(var e=0;ei;i++){o=t[i];var n=this.nodes[o];if(!n)throw new RangeError('Node with id "'+o+'" not found');this._selectObject(n,!0,!0,e,!0)}this.redraw()},e.selectEdges=function(t){var e,i,s;if(!t||void 0==t.length)throw"Selection must be an array with ids";for(this._unselectAll(!0),e=0,i=t.length;i>e;e++){s=t[e];var o=this.edges[s];if(!o)throw new RangeError('Edge with id "'+s+'" not found');this._selectObject(o,!0,!0,!1,!0)}this.redraw()},e._updateSelection=function(){for(var t in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(t)&&(this.nodes.hasOwnProperty(t)||delete this.selectionObj.nodes[t]);for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(this.edges.hasOwnProperty(e)||delete this.selectionObj.edges[e])}},function(t,e,i){var s=i(1),o=i(40),n=i(37);e._clearManipulatorBar=function(){this._recursiveDOMDelete(this.manipulationDiv),this.manipulationDOM={},this._manipulationReleaseOverload=function(){},delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode,this.controlNodesActive=!1,this.freezeSimulation=!1},e._restoreOverloadedFunctions=function(){for(var t in this.cachedFunctions)this.cachedFunctions.hasOwnProperty(t)&&(this[t]=this.cachedFunctions[t],delete this.cachedFunctions[t])},e._toggleEditMode=function(){this.editMode=!this.editMode;var t=this.manipulationDiv,e=this.closeDiv,i=this.editModeDiv;1==this.editMode?(t.style.display="block",e.style.display="block",i.style.display="none",e.onclick=this._toggleEditMode.bind(this)):(t.style.display="none",e.style.display="none",i.style.display="block",e.onclick=null),this._createManipulatorBar()},e._createManipulatorBar=function(){this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];if(void 0!==this.edgeBeingEdited&&(this.edgeBeingEdited._disableControlNodes(),this.edgeBeingEdited=void 0,this.selectedControlNode=null,this.controlNodesActive=!1,this._redraw()),this._restoreOverloadedFunctions(),this.freezeSimulation=!1,this.blockConnectingEdgeSelection=!1,this.forceAppendSelection=!1,this.manipulationDOM={},1==this.editMode){for(;this.manipulationDiv.hasChildNodes();)this.manipulationDiv.removeChild(this.manipulationDiv.firstChild);this.manipulationDOM.addNodeSpan=document.createElement("span"),this.manipulationDOM.addNodeSpan.className="network-manipulationUI add",this.manipulationDOM.addNodeLabelSpan=document.createElement("span"),this.manipulationDOM.addNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addNodeLabelSpan.innerHTML=t.addNode,this.manipulationDOM.addNodeSpan.appendChild(this.manipulationDOM.addNodeLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.addEdgeSpan=document.createElement("span"),this.manipulationDOM.addEdgeSpan.className="network-manipulationUI connect",this.manipulationDOM.addEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.addEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.addEdgeLabelSpan.innerHTML=t.addEdge,this.manipulationDOM.addEdgeSpan.appendChild(this.manipulationDOM.addEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.addNodeSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.addEdgeSpan),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?(this.manipulationDOM.seperatorLineDiv2=document.createElement("div"),this.manipulationDOM.seperatorLineDiv2.className="network-seperatorLine",this.manipulationDOM.editNodeSpan=document.createElement("span"),this.manipulationDOM.editNodeSpan.className="network-manipulationUI edit",this.manipulationDOM.editNodeLabelSpan=document.createElement("span"),this.manipulationDOM.editNodeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editNodeLabelSpan.innerHTML=t.editNode,this.manipulationDOM.editNodeSpan.appendChild(this.manipulationDOM.editNodeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv2),this.manipulationDiv.appendChild(this.manipulationDOM.editNodeSpan)):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.seperatorLineDiv3=document.createElement("div"),this.manipulationDOM.seperatorLineDiv3.className="network-seperatorLine",this.manipulationDOM.editEdgeSpan=document.createElement("span"),this.manipulationDOM.editEdgeSpan.className="network-manipulationUI edit",this.manipulationDOM.editEdgeLabelSpan=document.createElement("span"),this.manipulationDOM.editEdgeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editEdgeLabelSpan.innerHTML=t.editEdge,this.manipulationDOM.editEdgeSpan.appendChild(this.manipulationDOM.editEdgeLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv3),this.manipulationDiv.appendChild(this.manipulationDOM.editEdgeSpan)),0==this._selectionIsEmpty()&&(this.manipulationDOM.seperatorLineDiv4=document.createElement("div"),this.manipulationDOM.seperatorLineDiv4.className="network-seperatorLine",this.manipulationDOM.deleteSpan=document.createElement("span"),this.manipulationDOM.deleteSpan.className="network-manipulationUI delete",this.manipulationDOM.deleteLabelSpan=document.createElement("span"),this.manipulationDOM.deleteLabelSpan.className="network-manipulationLabel",this.manipulationDOM.deleteLabelSpan.innerHTML=t.del,this.manipulationDOM.deleteSpan.appendChild(this.manipulationDOM.deleteLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv4),this.manipulationDiv.appendChild(this.manipulationDOM.deleteSpan)),this.manipulationDOM.addNodeSpan.onclick=this._createAddNodeToolbar.bind(this),this.manipulationDOM.addEdgeSpan.onclick=this._createAddEdgeToolbar.bind(this),1==this._getSelectedNodeCount()&&this.triggerFunctions.edit?this.manipulationDOM.editNodeSpan.onclick=this._editNode.bind(this):1==this._getSelectedEdgeCount()&&0==this._getSelectedNodeCount()&&(this.manipulationDOM.editEdgeSpan.onclick=this._createEditEdgeToolbar.bind(this)),0==this._selectionIsEmpty()&&(this.manipulationDOM.deleteSpan.onclick=this._deleteSelected.bind(this)),this.closeDiv.onclick=this._toggleEditMode.bind(this); -var e=this;this.boundFunction=e._createManipulatorBar,this.on("select",this.boundFunction)}else{for(;this.editModeDiv.hasChildNodes();)this.editModeDiv.removeChild(this.editModeDiv.firstChild);this.manipulationDOM.editModeSpan=document.createElement("span"),this.manipulationDOM.editModeSpan.className="network-manipulationUI edit editmode",this.manipulationDOM.editModeLabelSpan=document.createElement("span"),this.manipulationDOM.editModeLabelSpan.className="network-manipulationLabel",this.manipulationDOM.editModeLabelSpan.innerHTML=t.edit,this.manipulationDOM.editModeSpan.appendChild(this.manipulationDOM.editModeLabelSpan),this.editModeDiv.appendChild(this.manipulationDOM.editModeSpan),this.manipulationDOM.editModeSpan.onclick=this._toggleEditMode.bind(this)}},e._createAddNodeToolbar=function(){this._clearManipulatorBar(),this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.addDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._addNode,this.on("select",this.boundFunction)},e._createAddEdgeToolbar=function(){this._clearManipulatorBar(),this._unselectAll(!0),this.freezeSimulation=!0,this.boundFunction&&this.off("select",this.boundFunction);var t=this.constants.locales[this.constants.locale];this._unselectAll(),this.forceAppendSelection=!1,this.blockConnectingEdgeSelection=!0,this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.edgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this);var e=this;this.boundFunction=e._handleConnect,this.on("select",this.boundFunction),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleDragEnd=this._handleDragEnd,this._handleTouch=this._handleConnect,this._manipulationReleaseOverload=function(){},this._handleDragStart=function(){},this._handleDragEnd=this._finishConnect,this._redraw()},e._createEditEdgeToolbar=function(){this._clearManipulatorBar(),this.controlNodesActive=!0,this.boundFunction&&this.off("select",this.boundFunction),this.edgeBeingEdited=this._getSelectedEdge(),this.edgeBeingEdited._enableControlNodes();var t=this.constants.locales[this.constants.locale];this.manipulationDOM={},this.manipulationDOM.backSpan=document.createElement("span"),this.manipulationDOM.backSpan.className="network-manipulationUI back",this.manipulationDOM.backLabelSpan=document.createElement("span"),this.manipulationDOM.backLabelSpan.className="network-manipulationLabel",this.manipulationDOM.backLabelSpan.innerHTML=t.back,this.manipulationDOM.backSpan.appendChild(this.manipulationDOM.backLabelSpan),this.manipulationDOM.seperatorLineDiv1=document.createElement("div"),this.manipulationDOM.seperatorLineDiv1.className="network-seperatorLine",this.manipulationDOM.descriptionSpan=document.createElement("span"),this.manipulationDOM.descriptionSpan.className="network-manipulationUI none",this.manipulationDOM.descriptionLabelSpan=document.createElement("span"),this.manipulationDOM.descriptionLabelSpan.className="network-manipulationLabel",this.manipulationDOM.descriptionLabelSpan.innerHTML=t.editEdgeDescription,this.manipulationDOM.descriptionSpan.appendChild(this.manipulationDOM.descriptionLabelSpan),this.manipulationDiv.appendChild(this.manipulationDOM.backSpan),this.manipulationDiv.appendChild(this.manipulationDOM.seperatorLineDiv1),this.manipulationDiv.appendChild(this.manipulationDOM.descriptionSpan),this.manipulationDOM.backSpan.onclick=this._createManipulatorBar.bind(this),this.cachedFunctions._handleTouch=this._handleTouch,this.cachedFunctions._manipulationReleaseOverload=this._manipulationReleaseOverload,this.cachedFunctions._handleTap=this._handleTap,this.cachedFunctions._handleDragStart=this._handleDragStart,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleTouch=this._selectControlNode,this._handleTap=function(){},this._handleOnDrag=this._controlNodeDrag,this._handleDragStart=function(){},this._manipulationReleaseOverload=this._releaseControlNode,this._redraw()},e._selectControlNode=function(t){this.edgeBeingEdited.controlNodes.from.unselect(),this.edgeBeingEdited.controlNodes.to.unselect(),this.selectedControlNode=this.edgeBeingEdited._getSelectedControlNode(this._XconvertDOMtoCanvas(t.x),this._YconvertDOMtoCanvas(t.y)),null!==this.selectedControlNode&&(this.selectedControlNode.select(),this.freezeSimulation=!0),this._redraw()},e._controlNodeDrag=function(t){var e=this._getPointer(t.center);null!==this.selectedControlNode&&void 0!==this.selectedControlNode&&(this.selectedControlNode.x=this._XconvertDOMtoCanvas(e.x),this.selectedControlNode.y=this._YconvertDOMtoCanvas(e.y)),this._redraw()},e._releaseControlNode=function(t){var e=this._getNodeAt(t);null!==e?(1==this.edgeBeingEdited.controlNodes.from.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(e.id,this.edgeBeingEdited.to.id),this.edgeBeingEdited.controlNodes.from.unselect()),1==this.edgeBeingEdited.controlNodes.to.selected&&(this.edgeBeingEdited._restoreControlNodes(),this._editEdge(this.edgeBeingEdited.from.id,e.id),this.edgeBeingEdited.controlNodes.to.unselect())):this.edgeBeingEdited._restoreControlNodes(),this.freezeSimulation=!1,this._redraw()},e._handleConnect=function(t){if(0==this._getSelectedNodeCount()){var e=this._getNodeAt(t);if(null!=e)if(e.clusterSize>1)alert(this.constants.locales[this.constants.locale].createEdgeError);else{this._selectObject(e,!1);var i=this.sectors.support.nodes;i.targetNode=new o({id:"targetNode"},{},{},this.constants);var s=i.targetNode;s.x=e.x,s.y=e.y,this.edges.connectionEdge=new n({id:"connectionEdge",from:e.id,to:s.id},this,this.constants);var r=this.edges.connectionEdge;r.from=e,r.connected=!0,r.options.smoothCurves={enabled:!0,dynamic:!1,type:"continuous",roundness:.5},r.selected=!0,r.to=s,this.cachedFunctions._handleOnDrag=this._handleOnDrag,this._handleOnDrag=function(t){var e=this._getPointer(t.center),i=this.edges.connectionEdge;i.to.x=this._XconvertDOMtoCanvas(e.x),i.to.y=this._YconvertDOMtoCanvas(e.y)},this.moving=!0,this.start()}}},e._finishConnect=function(t){if(1==this._getSelectedNodeCount()){var e=this._getPointer(t.center);this._handleOnDrag=this.cachedFunctions._handleOnDrag,delete this.cachedFunctions._handleOnDrag;var i=this.edges.connectionEdge.fromId;delete this.edges.connectionEdge,delete this.sectors.support.nodes.targetNode,delete this.sectors.support.nodes.targetViaNode;var s=this._getNodeAt(e);null!=s&&(s.clusterSize>1?alert(this.constants.locales[this.constants.locale].createEdgeError):(this._createEdge(i,s.id),this._createManipulatorBar())),this._unselectAll()}},e._addNode=function(){if(this._selectionIsEmpty()&&1==this.editMode){var t=this._pointerToPositionObject(this.pointerPosition),e={id:s.randomUUID(),x:t.left,y:t.top,label:"new",allowedToMoveX:!0,allowedToMoveY:!0};if(this.triggerFunctions.add){if(2!=this.triggerFunctions.add.length)throw new Error("The function for add does not support two arguments (data,callback)");var i=this;this.triggerFunctions.add(e,function(t){i.nodesData.add(t),i._createManipulatorBar(),i.moving=!0,i.start()})}else this.nodesData.add(e),this._createManipulatorBar(),this.moving=!0,this.start()}},e._createEdge=function(t,e){if(1==this.editMode){var i={from:t,to:e};if(this.triggerFunctions.connect){if(2!=this.triggerFunctions.connect.length)throw new Error("The function for connect does not support two arguments (data,callback)");var s=this;this.triggerFunctions.connect(i,function(t){s.edgesData.add(t),s.moving=!0,s.start()})}else this.edgesData.add(i),this.moving=!0,this.start()}},e._editEdge=function(t,e){if(1==this.editMode){var i={id:this.edgeBeingEdited.id,from:t,to:e};if(this.triggerFunctions.editEdge){if(2!=this.triggerFunctions.editEdge.length)throw new Error("The function for edit does not support two arguments (data, callback)");var s=this;this.triggerFunctions.editEdge(i,function(t){s.edgesData.update(t),s.moving=!0,s.start()})}else this.edgesData.update(i),this.moving=!0,this.start()}},e._editNode=function(){if(!this.triggerFunctions.edit||1!=this.editMode)throw new Error("No edit function has been bound to this button");var t=this._getSelectedNode(),e={id:t.id,label:t.label,group:t.options.group,shape:t.options.shape,color:{background:t.options.color.background,border:t.options.color.border,highlight:{background:t.options.color.highlight.background,border:t.options.color.highlight.border}}};if(2!=this.triggerFunctions.edit.length)throw new Error("The function for edit does not support two arguments (data, callback)");var i=this;this.triggerFunctions.edit(e,function(t){i.nodesData.update(t),i._createManipulatorBar(),i.moving=!0,i.start()})},e._deleteSelected=function(){if(!this._selectionIsEmpty()&&1==this.editMode)if(this._clusterInSelection())alert(this.constants.locales[this.constants.locale].deleteClusterError);else{var t=this.getSelectedNodes(),e=this.getSelectedEdges();if(this.triggerFunctions.del){var i=this,s={nodes:t,edges:e};if(2!=this.triggerFunctions.del.length)throw new Error("The function for delete does not support two arguments (data, callback)");this.triggerFunctions.del(s,function(t){i.edgesData.remove(t.edges),i.nodesData.remove(t.nodes),i._unselectAll(),i.moving=!0,i.start()})}else this.edgesData.remove(e),this.nodesData.remove(t),this._unselectAll(),this.moving=!0,this.start()}}},function(t,e,i){var s=(i(1),i(47)),o=i(45);e._cleanNavigation=function(){if(0!=this.navigationHammers.existing.length){for(var t=0;t0){var t,e,i=0,s=!1,o=!1;for(e in this.nodes)this.nodes.hasOwnProperty(e)&&(t=this.nodes[e],-1!=t.level?s=!0:o=!0,is&&(n.xFixed=!1,n.x=i[n.level].minPos,r=!0):n.yFixed&&n.level>s&&(n.yFixed=!1,n.y=i[n.level].minPos,r=!0),1==r&&(i[n.level].minPos+=i[n.level].nodeSpacing,n.edges.length>1&&this._placeBranchNodes(n.edges,n.id,i,n.level))}},e._setLevel=function(t,e,i){for(var s=0;st)&&(o.level=t,o.edges.length>1&&this._setLevel(t+1,o.edges,o.id))}},e._setLevelDirected=function(t,e,i){this.nodes[i].hierarchyEnumerated=!0;for(var s,o,n=0;n1&&s.hierarchyEnumerated===!1&&this._setLevelDirected(s.level,s.edges,s.id)},e._restoreNodes=function(){for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.nodes[t].xFixed=!1,this.nodes[t].yFixed=!1)}},function(t,e,i){function s(){this.constants.smoothCurves.enabled=!this.constants.smoothCurves.enabled;var t=document.getElementById("graph_toggleSmooth");t.style.background=1==this.constants.smoothCurves.enabled?"#A4FF56":"#FF8532",this._configureSmoothCurves(!1)}function o(){for(var t in this.calculationNodes)this.calculationNodes.hasOwnProperty(t)&&(this.calculationNodes[t].vx=0,this.calculationNodes[t].vy=0,this.calculationNodes[t].fx=0,this.calculationNodes[t].fy=0);1==this.constants.hierarchicalLayout.enabled?(this._setupHierarchicalLayout(),a.call(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),a.call(this,"graph_H_cg",1,"physics_centralGravity"),a.call(this,"graph_H_sc",1,"physics_springConstant"),a.call(this,"graph_H_sl",1,"physics_springLength"),a.call(this,"graph_H_damp",1,"physics_damping")):this.repositionNodes(),this.moving=!0,this.start()}function n(){var t="No options are required, default values used.",e=[],i=document.getElementById("graph_physicsMethod1"),s=document.getElementById("graph_physicsMethod2");if(1==i.checked){if(this.constants.physics.barnesHut.gravitationalConstant!=this.backupConstants.physics.barnesHut.gravitationalConstant&&e.push("gravitationalConstant: "+this.constants.physics.barnesHut.gravitationalConstant),this.constants.physics.centralGravity!=this.backupConstants.physics.barnesHut.centralGravity&&e.push("centralGravity: "+this.constants.physics.centralGravity),this.constants.physics.springLength!=this.backupConstants.physics.barnesHut.springLength&&e.push("springLength: "+this.constants.physics.springLength),this.constants.physics.springConstant!=this.backupConstants.physics.barnesHut.springConstant&&e.push("springConstant: "+this.constants.physics.springConstant),this.constants.physics.damping!=this.backupConstants.physics.barnesHut.damping&&e.push("damping: "+this.constants.physics.damping),0!=e.length){t="var options = {",t+="physics: {barnesHut: {";for(var o=0;othis.constants.clustering.clusterThreshold&&1==this.constants.clustering.enabled&&this.clusterToFit(this.constants.clustering.reduceToNodes,!1),this._calculateForces())},e._calculateForces=function(){this._calculateGravitationalForces(),this._calculateNodeForces(),this.constants.physics.springConstant>0&&(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic?this._calculateSpringForcesWithSupport():1==this.constants.physics.hierarchicalRepulsion.enabled?this._calculateHierarchicalSpringForces():this._calculateSpringForces())},e._updateCalculationNodes=function(){if(1==this.constants.smoothCurves.enabled&&1==this.constants.smoothCurves.dynamic){this.calculationNodes={},this.calculationNodeIndices=[];for(var t in this.nodes)this.nodes.hasOwnProperty(t)&&(this.calculationNodes[t]=this.nodes[t]);var e=this.sectors.support.nodes;for(var i in e)e.hasOwnProperty(i)&&(this.edges.hasOwnProperty(e[i].parentEdgeId)?this.calculationNodes[i]=e[i]:e[i]._setForce(0,0));for(var s in this.calculationNodes)this.calculationNodes.hasOwnProperty(s)&&this.calculationNodeIndices.push(s)}else this.calculationNodes=this.nodes,this.calculationNodeIndices=this.nodeIndices},e._calculateGravitationalForces=function(){var t,e,i,s,o,n=this.calculationNodes,r=this.constants.physics.centralGravity,a=0;for(o=0;oSimulation Mode:Barnes HutRepulsionHierarchical
    Options:
    ',this.containerElement.parentElement.insertBefore(this.physicsConfiguration,this.containerElement),this.optionsDiv=document.createElement("div"),this.optionsDiv.style.fontSize="14px",this.optionsDiv.style.fontFamily="verdana",this.containerElement.parentElement.insertBefore(this.optionsDiv,this.containerElement); -var e;e=document.getElementById("graph_BH_gc"),e.onchange=a.bind(this,"graph_BH_gc",-1,"physics_barnesHut_gravitationalConstant"),e=document.getElementById("graph_BH_cg"),e.onchange=a.bind(this,"graph_BH_cg",1,"physics_centralGravity"),e=document.getElementById("graph_BH_sc"),e.onchange=a.bind(this,"graph_BH_sc",1,"physics_springConstant"),e=document.getElementById("graph_BH_sl"),e.onchange=a.bind(this,"graph_BH_sl",1,"physics_springLength"),e=document.getElementById("graph_BH_damp"),e.onchange=a.bind(this,"graph_BH_damp",1,"physics_damping"),e=document.getElementById("graph_R_nd"),e.onchange=a.bind(this,"graph_R_nd",1,"physics_repulsion_nodeDistance"),e=document.getElementById("graph_R_cg"),e.onchange=a.bind(this,"graph_R_cg",1,"physics_centralGravity"),e=document.getElementById("graph_R_sc"),e.onchange=a.bind(this,"graph_R_sc",1,"physics_springConstant"),e=document.getElementById("graph_R_sl"),e.onchange=a.bind(this,"graph_R_sl",1,"physics_springLength"),e=document.getElementById("graph_R_damp"),e.onchange=a.bind(this,"graph_R_damp",1,"physics_damping"),e=document.getElementById("graph_H_nd"),e.onchange=a.bind(this,"graph_H_nd",1,"physics_hierarchicalRepulsion_nodeDistance"),e=document.getElementById("graph_H_cg"),e.onchange=a.bind(this,"graph_H_cg",1,"physics_centralGravity"),e=document.getElementById("graph_H_sc"),e.onchange=a.bind(this,"graph_H_sc",1,"physics_springConstant"),e=document.getElementById("graph_H_sl"),e.onchange=a.bind(this,"graph_H_sl",1,"physics_springLength"),e=document.getElementById("graph_H_damp"),e.onchange=a.bind(this,"graph_H_damp",1,"physics_damping"),e=document.getElementById("graph_H_direction"),e.onchange=a.bind(this,"graph_H_direction",t,"hierarchicalLayout_direction"),e=document.getElementById("graph_H_levsep"),e.onchange=a.bind(this,"graph_H_levsep",1,"hierarchicalLayout_levelSeparation"),e=document.getElementById("graph_H_nspac"),e.onchange=a.bind(this,"graph_H_nspac",1,"hierarchicalLayout_nodeSpacing");var i=document.getElementById("graph_physicsMethod1"),d=document.getElementById("graph_physicsMethod2"),l=document.getElementById("graph_physicsMethod3");d.checked=!0,this.constants.physics.barnesHut.enabled&&(i.checked=!0),this.constants.hierarchicalLayout.enabled&&(l.checked=!0);var c=document.getElementById("graph_toggleSmooth"),p=document.getElementById("graph_repositionNodes"),u=document.getElementById("graph_generateOptions");c.onclick=s.bind(this),p.onclick=o.bind(this),u.onclick=n.bind(this),c.style.background=1==this.constants.smoothCurves&&0==this.constants.dynamicSmoothCurves?"#A4FF56":"#FF8532",r.apply(this),i.onchange=r.bind(this),d.onchange=r.bind(this),l.onchange=r.bind(this)}},e._overWriteGraphConstants=function(t,e){var i=t.split("_");1==i.length?this.constants[i[0]]=e:2==i.length?this.constants[i[0]][i[1]]=e:3==i.length&&(this.constants[i[0]][i[1]][i[2]]=e)}},function(t){function e(t){throw new Error("Cannot find module '"+t+"'.")}e.keys=function(){return[]},e.resolve=e,t.exports=e,e.id=68},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l,c=this.calculationNodes,p=this.calculationNodeIndices,u=-2/3,m=4/3,f=this.constants.physics.repulsion.nodeDistance,g=f;for(d=0;di&&(r=.5*g>i?1:v*i+m,r*=0==n?1:1+n*this.constants.clustering.forceAmplification,r/=Math.max(i,.01*g),s=t*r,o=e*r,a.fx-=s,a.fy-=o,h.fx+=s,h.fy+=o)}}},function(t,e){e._calculateNodeForces=function(){var t,e,i,s,o,n,r,a,h,d,l=this.calculationNodes,c=this.calculationNodeIndices,p=this.constants.physics.hierarchicalRepulsion.nodeDistance;for(h=0;hi?-Math.pow(u*i,2)+Math.pow(u*p,2):0,0==i?i=.01:n/=i,s=t*n,o=e*n,r.fx-=s,r.fy-=o,a.fx+=s,a.fy+=o}},e._calculateHierarchicalSpringForces=function(){for(var t,e,i,s,o,n,r,a,h,d=this.edges,l=this.calculationNodes,c=this.calculationNodeIndices,p=0;pn;n++)t=e[i[n]],t.options.mass>0&&(this._getForceContribution(o.root.children.NW,t),this._getForceContribution(o.root.children.NE,t),this._getForceContribution(o.root.children.SW,t),this._getForceContribution(o.root.children.SE,t))}},e._getForceContribution=function(t,e){if(t.childrenCount>0){var i,s,o;if(i=t.centerOfMass.x-e.x,s=t.centerOfMass.y-e.y,o=Math.sqrt(i*i+s*s),o*t.calcSize>this.constants.physics.barnesHut.thetaInverted){0==o&&(o=.1*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}else if(4==t.childrenCount)this._getForceContribution(t.children.NW,e),this._getForceContribution(t.children.NE,e),this._getForceContribution(t.children.SW,e),this._getForceContribution(t.children.SE,e);else if(t.children.data.id!=e.id){0==o&&(o=.5*Math.random(),i=o);var n=this.constants.physics.barnesHut.gravitationalConstant*t.mass*e.options.mass/(o*o*o),r=i*n,a=s*n;e.fx+=r,e.fy+=a}}},e._formBarnesHutTree=function(t,e){for(var i,s=e.length,o=Number.MAX_VALUE,n=Number.MAX_VALUE,r=-Number.MAX_VALUE,a=-Number.MAX_VALUE,h=0;s>h;h++){var d=t[e[h]].x,l=t[e[h]].y;t[e[h]].options.mass>0&&(o>d&&(o=d),d>r&&(r=d),n>l&&(n=l),l>a&&(a=l))}var c=Math.abs(r-o)-Math.abs(a-n);c>0?(n-=.5*c,a+=.5*c):(o+=.5*c,r-=.5*c);var p=1e-5,u=Math.max(p,Math.abs(r-o)),m=.5*u,f=.5*(o+r),g=.5*(n+a),v={root:{centerOfMass:{x:0,y:0},mass:0,range:{minX:f-m,maxX:f+m,minY:g-m,maxY:g+m},size:u,calcSize:1/u,children:{data:null},maxWidth:0,level:0,childrenCount:4}};for(this._splitBranch(v.root),h=0;s>h;h++)i=t[e[h]],i.options.mass>0&&this._placeInTree(v.root,i);this.barnesHutTree=v},e._updateBranchMass=function(t,e){var i=t.mass+e.options.mass,s=1/i;t.centerOfMass.x=t.centerOfMass.x*t.mass+e.x*e.options.mass,t.centerOfMass.x*=s,t.centerOfMass.y=t.centerOfMass.y*t.mass+e.y*e.options.mass,t.centerOfMass.y*=s,t.mass=i;var o=Math.max(Math.max(e.height,e.radius),e.width);t.maxWidth=t.maxWidthe.x?t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NW"):this._placeInRegion(t,e,"SW"):t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NE"):this._placeInRegion(t,e,"SE")},e._placeInRegion=function(t,e,i){switch(t.children[i].childrenCount){case 0:t.children[i].children.data=e,t.children[i].childrenCount=1,this._updateBranchMass(t.children[i],e);break;case 1:t.children[i].children.data.x==e.x&&t.children[i].children.data.y==e.y?(e.x+=Math.random(),e.y+=Math.random()):(this._splitBranch(t.children[i]),this._placeInTree(t.children[i],e));break;case 4:this._placeInTree(t.children[i],e)}},e._splitBranch=function(t){var e=null;1==t.childrenCount&&(e=t.children.data,t.mass=0,t.centerOfMass.x=0,t.centerOfMass.y=0),t.childrenCount=4,t.children.data=null,this._insertRegion(t,"NW"),this._insertRegion(t,"NE"),this._insertRegion(t,"SW"),this._insertRegion(t,"SE"),null!=e&&this._placeInTree(t,e)},e._insertRegion=function(t,e){var i,s,o,n,r=.5*t.size;switch(e){case"NW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY,n=t.range.minY+r;break;case"NE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY,n=t.range.minY+r;break;case"SW":i=t.range.minX,s=t.range.minX+r,o=t.range.minY+r,n=t.range.maxY;break;case"SE":i=t.range.minX+r,s=t.range.maxX,o=t.range.minY+r,n=t.range.maxY}t.children[e]={centerOfMass:{x:0,y:0},mass:0,range:{minX:i,maxX:s,minY:o,maxY:n},size:.5*t.size,calcSize:2*t.calcSize,children:{data:null},maxWidth:0,level:t.level+1,childrenCount:0}},e._drawTree=function(t,e){void 0!==this.barnesHutTree&&(t.lineWidth=1,this._drawBranch(this.barnesHutTree.root,t,e))},e._drawBranch=function(t,e,i){void 0===i&&(i="#FF0000"),4==t.childrenCount&&(this._drawBranch(t.children.NW,e),this._drawBranch(t.children.NE,e),this._drawBranch(t.children.SE,e),this._drawBranch(t.children.SW,e)),e.strokeStyle=i,e.beginPath(),e.moveTo(t.range.minX,t.range.minY),e.lineTo(t.range.maxX,t.range.minY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.minY),e.lineTo(t.range.maxX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.maxY),e.lineTo(t.range.minX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.minX,t.range.maxY),e.lineTo(t.range.minX,t.range.minY),e.stroke()}},function(t){t.exports=function(t){return t.webpackPolyfill||(t.deprecate=function(){},t.paths=[],t.children=[],t.webpackPolyfill=1),t}},function(t,e){(function(e){t.exports=e}).call(e,{})}])}); +"use strict";!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define([],e):"object"==typeof exports?exports.vis=e():t.vis=e()}(this,function(){return function(t){function e(o){if(i[o])return i[o].exports;var n=i[o]={exports:{},id:o,loaded:!1};return t[o].call(n.exports,n,n.exports,e),n.loaded=!0,n.exports}var i={};return e.m=t,e.c=i,e.p="",e(0)}([function(t,e,i){var o=i(1);o.extend(e,i(7)),o.extend(e,i(24)),o.extend(e,i(60))},function(t,e,i){var o="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},n=i(2),s=i(6);e.isNumber=function(t){return t instanceof Number||"number"==typeof t},e.recursiveDOMDelete=function(t){if(t)for(;t.hasChildNodes()===!0;)e.recursiveDOMDelete(t.firstChild),t.removeChild(t.firstChild)},e.giveRange=function(t,e,i,o){if(e==t)return.5;var n=1/(e-t);return Math.max(0,(o-t)*n)},e.isString=function(t){return t instanceof String||"string"==typeof t},e.isDate=function(t){if(t instanceof Date)return!0;if(e.isString(t)){var i=r.exec(t);if(i)return!0;if(!isNaN(Date.parse(t)))return!0}return!1},e.randomUUID=function(){return s.v4()},e.assignAllKeys=function(t,e){for(var i in t)t.hasOwnProperty(i)&&"object"!==o(t[i])&&(t[i]=e)},e.fillIfDefined=function(t,i){var n=arguments.length<=2||void 0===arguments[2]?!1:arguments[2];for(var s in t)void 0!==i[s]&&("object"!==o(i[s])?void 0!==i[s]&&null!==i[s]||void 0===t[s]||n!==!0?t[s]=i[s]:delete t[s]:"object"===o(t[s])&&e.fillIfDefined(t[s],i[s],n))},e.protoExtend=function(t,e){for(var i=1;ii;i++)if(t[i]!=e[i])return!1;return!0},e.convert=function(t,i){var o;if(void 0!==t){if(null===t)return null;if(!i)return t;if("string"!=typeof i&&!(i instanceof String))throw new Error("Type must be a string");switch(i){case"boolean":case"Boolean":return Boolean(t);case"number":case"Number":return Number(t.valueOf());case"string":case"String":return String(t);case"Date":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return new Date(t.valueOf());if(n.isMoment(t))return new Date(t.valueOf());if(e.isString(t))return o=r.exec(t),o?new Date(Number(o[1])):n(t).toDate();throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"Moment":if(e.isNumber(t))return n(t);if(t instanceof Date)return n(t.valueOf());if(n.isMoment(t))return n(t);if(e.isString(t))return o=r.exec(t),n(o?Number(o[1]):t);throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"ISODate":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return t.toISOString();if(n.isMoment(t))return t.toDate().toISOString();if(e.isString(t))return o=r.exec(t),o?new Date(Number(o[1])).toISOString():new Date(t).toISOString();throw new Error("Cannot convert object of type "+e.getType(t)+" to type ISODate");case"ASPDate":if(e.isNumber(t))return"/Date("+t+")/";if(t instanceof Date)return"/Date("+t.valueOf()+")/";if(e.isString(t)){o=r.exec(t);var s;return s=o?new Date(Number(o[1])).valueOf():new Date(t).valueOf(),"/Date("+s+")/"}throw new Error("Cannot convert object of type "+e.getType(t)+" to type ASPDate");default:throw new Error('Unknown type "'+i+'"')}}};var r=/^\/?Date\((\-?\d+)/i;e.getType=function(t){var e="undefined"==typeof t?"undefined":o(t);return"object"==e?null===t?"null":t instanceof Boolean?"Boolean":t instanceof Number?"Number":t instanceof String?"String":Array.isArray(t)?"Array":t instanceof Date?"Date":"Object":"number"==e?"Number":"boolean"==e?"Boolean":"string"==e?"String":void 0===e?"undefined":e},e.copyAndExtendArray=function(t,e){for(var i=[],o=0;oi;i++)e(t[i],i,t);else for(i in t)t.hasOwnProperty(i)&&e(t[i],i,t)},e.toArray=function(t){var e=[];for(var i in t)t.hasOwnProperty(i)&&e.push(t[i]);return e},e.updateProperty=function(t,e,i){return t[e]!==i?(t[e]=i,!0):!1},e.throttle=function(t,e){var i=null,o=!1;return function n(){i?o=!0:(o=!1,t(),i=setTimeout(function(){i=null,o&&n()},e))}},e.addEventListener=function(t,e,i,o){t.addEventListener?(void 0===o&&(o=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.addEventListener(e,i,o)):t.attachEvent("on"+e,i)},e.removeEventListener=function(t,e,i,o){t.removeEventListener?(void 0===o&&(o=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.removeEventListener(e,i,o)):t.detachEvent("on"+e,i)},e.preventDefault=function(t){t||(t=window.event),t.preventDefault?t.preventDefault():t.returnValue=!1},e.getTarget=function(t){t||(t=window.event);var e;return t.target?e=t.target:t.srcElement&&(e=t.srcElement),void 0!=e.nodeType&&3==e.nodeType&&(e=e.parentNode),e},e.hasParent=function(t,e){for(var i=t;i;){if(i===e)return!0;i=i.parentNode}return!1},e.option={},e.option.asBoolean=function(t,e){return"function"==typeof t&&(t=t()),null!=t?0!=t:e||null},e.option.asNumber=function(t,e){return"function"==typeof t&&(t=t()),null!=t?Number(t)||e||null:e||null},e.option.asString=function(t,e){return"function"==typeof t&&(t=t()),null!=t?String(t):e||null},e.option.asSize=function(t,i){return"function"==typeof t&&(t=t()),e.isString(t)?t:e.isNumber(t)?t+"px":i||null},e.option.asElement=function(t,e){return"function"==typeof t&&(t=t()),t||e||null},e.hexToRGB=function(t){var e=/^#?([a-f\d])([a-f\d])([a-f\d])$/i;t=t.replace(e,function(t,e,i,o){return e+e+i+i+o+o});var i=/^#?([a-f\d]{2})([a-f\d]{2})([a-f\d]{2})$/i.exec(t);return i?{r:parseInt(i[1],16),g:parseInt(i[2],16),b:parseInt(i[3],16)}:null},e.overrideOpacity=function(t,i){if(-1!=t.indexOf("rgba"))return t;if(-1!=t.indexOf("rgb")){var o=t.substr(t.indexOf("(")+1).replace(")","").split(",");return"rgba("+o[0]+","+o[1]+","+o[2]+","+i+")"}var o=e.hexToRGB(t);return null==o?t:"rgba("+o.r+","+o.g+","+o.b+","+i+")"},e.RGBToHex=function(t,e,i){return"#"+((1<<24)+(t<<16)+(e<<8)+i).toString(16).slice(1)},e.parseColor=function(t){var i;if(e.isString(t)===!0){if(e.isValidRGB(t)===!0){var o=t.substr(4).substr(0,t.length-5).split(",").map(function(t){return parseInt(t)});t=e.RGBToHex(o[0],o[1],o[2])}if(e.isValidHex(t)===!0){var n=e.hexToHSV(t),s={h:n.h,s:.8*n.s,v:Math.min(1,1.02*n.v)},r={h:n.h,s:Math.min(1,1.25*n.s),v:.8*n.v},a=e.HSVToHex(r.h,r.s,r.v),h=e.HSVToHex(s.h,s.s,s.v);i={background:t,border:a,highlight:{background:h,border:a},hover:{background:h,border:a}}}else i={background:t,border:t,highlight:{background:t,border:t},hover:{background:t,border:t}}}else i={},i.background=t.background||void 0,i.border=t.border||void 0,e.isString(t.highlight)?i.highlight={border:t.highlight,background:t.highlight}:(i.highlight={},i.highlight.background=t.highlight&&t.highlight.background||void 0,i.highlight.border=t.highlight&&t.highlight.border||void 0),e.isString(t.hover)?i.hover={border:t.hover,background:t.hover}:(i.hover={},i.hover.background=t.hover&&t.hover.background||void 0,i.hover.border=t.hover&&t.hover.border||void 0);return i},e.RGBToHSV=function(t,e,i){t/=255,e/=255,i/=255;var o=Math.min(t,Math.min(e,i)),n=Math.max(t,Math.max(e,i));if(o==n)return{h:0,s:0,v:o};var s=t==o?e-i:i==o?t-e:i-t,r=t==o?3:i==o?1:5,a=60*(r-s/(n-o))/360,h=(n-o)/n,d=n;return{h:a,s:h,v:d}};var a={split:function(t){var e={};return t.split(";").forEach(function(t){if(""!=t.trim()){var i=t.split(":"),o=i[0].trim(),n=i[1].trim();e[o]=n}}),e},join:function(t){return Object.keys(t).map(function(e){return e+": "+t[e]}).join("; ")}};e.addCssText=function(t,i){var o=a.split(t.style.cssText),n=a.split(i),s=e.extend(o,n);t.style.cssText=a.join(s)},e.removeCssText=function(t,e){var i=a.split(t.style.cssText),o=a.split(e);for(var n in o)o.hasOwnProperty(n)&&delete i[n];t.style.cssText=a.join(i)},e.HSVToRGB=function(t,e,i){var o,n,s,r=Math.floor(6*t),a=6*t-r,h=i*(1-e),d=i*(1-a*e),l=i*(1-(1-a)*e);switch(r%6){case 0:o=i,n=l,s=h;break;case 1:o=d,n=i,s=h;break;case 2:o=h,n=i,s=l;break;case 3:o=h,n=d,s=i;break;case 4:o=l,n=h,s=i;break;case 5:o=i,n=h,s=d}return{r:Math.floor(255*o),g:Math.floor(255*n),b:Math.floor(255*s)}},e.HSVToHex=function(t,i,o){var n=e.HSVToRGB(t,i,o);return e.RGBToHex(n.r,n.g,n.b)},e.hexToHSV=function(t){var i=e.hexToRGB(t);return e.RGBToHSV(i.r,i.g,i.b)},e.isValidHex=function(t){var e=/(^#[0-9A-F]{6}$)|(^#[0-9A-F]{3}$)/i.test(t);return e},e.isValidRGB=function(t){t=t.replace(" ","");var e=/rgb\((\d{1,3}),(\d{1,3}),(\d{1,3})\)/i.test(t);return e},e.isValidRGBA=function(t){t=t.replace(" ","");var e=/rgba\((\d{1,3}),(\d{1,3}),(\d{1,3}),(.{1,3})\)/i.test(t);return e},e.selectiveBridgeObject=function(t,i){if("object"==("undefined"==typeof i?"undefined":o(i))){for(var n=Object.create(i),s=0;s0&&e(o,t[n-1])<0;n--)t[n]=t[n-1];t[n]=o}return t},e.mergeOptions=function(t,e,i){var o=(arguments.length<=3||void 0===arguments[3]?!1:arguments[3],arguments.length<=4||void 0===arguments[4]?{}:arguments[4]);if(null===e[i])t[i]=Object.create(o[i]);else if(void 0!==e[i])if("boolean"==typeof e[i])t[i].enabled=e[i];else{void 0===e[i].enabled&&(t[i].enabled=!0);for(var n in e[i])e[i].hasOwnProperty(n)&&(t[i][n]=e[i][n])}},e.binarySearchCustom=function(t,e,i,o){for(var n=1e4,s=0,r=0,a=t.length-1;a>=r&&n>s;){var h=Math.floor((r+a)/2),d=t[h],l=void 0===o?d[i]:d[i][o],c=e(l);if(0==c)return h;-1==c?r=h+1:a=h-1,s++}return-1},e.binarySearchValue=function(t,e,i,o,n){for(var s,r,a,h,d=1e4,l=0,c=0,u=t.length-1,n=void 0!=n?n:function(t,e){return t==e?0:e>t?-1:1};u>=c&&d>l;){if(h=Math.floor(.5*(u+c)),s=t[Math.max(0,h-1)][i],r=t[h][i],a=t[Math.min(t.length-1,h+1)][i],0==n(r,e))return h;if(n(s,e)<0&&n(r,e)>0)return"before"==o?Math.max(0,h-1):h;if(n(r,e)<0&&n(a,e)>0)return"before"==o?h:Math.min(t.length-1,h+1);n(r,e)<0?c=h+1:u=h-1,l++}return-1},e.easingFunctions={linear:function(t){return t},easeInQuad:function(t){return t*t},easeOutQuad:function(t){return t*(2-t)},easeInOutQuad:function(t){return.5>t?2*t*t:-1+(4-2*t)*t},easeInCubic:function(t){return t*t*t},easeOutCubic:function(t){return--t*t*t+1},easeInOutCubic:function(t){return.5>t?4*t*t*t:(t-1)*(2*t-2)*(2*t-2)+1},easeInQuart:function(t){return t*t*t*t},easeOutQuart:function(t){return 1- --t*t*t*t},easeInOutQuart:function(t){return.5>t?8*t*t*t*t:1-8*--t*t*t*t},easeInQuint:function(t){return t*t*t*t*t},easeOutQuint:function(t){return 1+--t*t*t*t*t},easeInOutQuint:function(t){return.5>t?16*t*t*t*t*t:1+16*--t*t*t*t*t}}},function(t,e,i){t.exports="undefined"!=typeof window&&window.moment||i(3)},function(t,e,i){(function(t){!function(e,i){t.exports=i()}(this,function(){function e(){return ro.apply(null,arguments)}function i(t){ro=t}function o(t){return t instanceof Array||"[object Array]"===Object.prototype.toString.call(t)}function n(t){return t instanceof Date||"[object Date]"===Object.prototype.toString.call(t)}function s(t,e){var i,o=[];for(i=0;i0)for(i in ho)o=ho[i],n=e[o],p(n)||(t[o]=n);return t}function m(t){f(this,t),this._d=new Date(null!=t._d?t._d.getTime():NaN),lo===!1&&(lo=!0,e.updateOffset(this),lo=!1)}function v(t){return t instanceof m||null!=t&&null!=t._isAMomentObject}function g(t){return 0>t?Math.ceil(t):Math.floor(t)}function y(t){var e=+t,i=0;return 0!==e&&isFinite(e)&&(i=g(e)),i}function b(t,e,i){var o,n=Math.min(t.length,e.length),s=Math.abs(t.length-e.length),r=0;for(o=0;n>o;o++)(i&&t[o]!==e[o]||!i&&y(t[o])!==y(e[o]))&&r++;return r+s}function w(t){e.suppressDeprecationWarnings===!1&&"undefined"!=typeof console&&console.warn&&console.warn("Deprecation warning: "+t)}function _(t,i){var o=!0;return a(function(){return null!=e.deprecationHandler&&e.deprecationHandler(null,t),o&&(w(t+"\nArguments: "+Array.prototype.slice.call(arguments).join(", ")+"\n"+(new Error).stack),o=!1),i.apply(this,arguments)},i)}function x(t,i){null!=e.deprecationHandler&&e.deprecationHandler(t,i),co[t]||(w(i),co[t]=!0)}function k(t){return t instanceof Function||"[object Function]"===Object.prototype.toString.call(t)}function O(t){return"[object Object]"===Object.prototype.toString.call(t)}function M(t){var e,i;for(i in t)e=t[i],k(e)?this[i]=e:this["_"+i]=e;this._config=t,this._ordinalParseLenient=new RegExp(this._ordinalParse.source+"|"+/\d{1,2}/.source)}function D(t,e){var i,o=a({},t);for(i in e)r(e,i)&&(O(t[i])&&O(e[i])?(o[i]={},a(o[i],t[i]),a(o[i],e[i])):null!=e[i]?o[i]=e[i]:delete o[i]);return o}function S(t){null!=t&&this.set(t)}function C(t){return t?t.toLowerCase().replace("_","-"):t}function T(t){for(var e,i,o,n,s=0;s0;){if(o=E(n.slice(0,e).join("-")))return o;if(i&&i.length>=e&&b(n,i,!0)>=e-1)break;e--}s++}return null}function E(e){var i=null;if(!mo[e]&&"undefined"!=typeof t&&t&&t.exports)try{i=po._abbr,!function(){var t=new Error('Cannot find module "./locale"');throw t.code="MODULE_NOT_FOUND",t}(),P(i)}catch(o){}return mo[e]}function P(t,e){var i;return t&&(i=p(e)?R(t):I(t,e),i&&(po=i)),po._abbr}function I(t,e){return null!==e?(e.abbr=t,null!=mo[t]?(x("defineLocaleOverride","use moment.updateLocale(localeName, config) to change an existing locale. moment.defineLocale(localeName, config) should only be used for creating a new locale"),e=D(mo[t]._config,e)):null!=e.parentLocale&&(null!=mo[e.parentLocale]?e=D(mo[e.parentLocale]._config,e):x("parentLocaleUndefined","specified parentLocale is not defined yet")),mo[t]=new S(e),P(t),mo[t]):(delete mo[t],null)}function N(t,e){if(null!=e){var i;null!=mo[t]&&(e=D(mo[t]._config,e)),i=new S(e),i.parentLocale=mo[t],mo[t]=i,P(t)}else null!=mo[t]&&(null!=mo[t].parentLocale?mo[t]=mo[t].parentLocale:null!=mo[t]&&delete mo[t]);return mo[t]}function R(t){var e;if(t&&t._locale&&t._locale._abbr&&(t=t._locale._abbr),!t)return po;if(!o(t)){if(e=E(t))return e;t=[t]}return T(t)}function z(){return uo(mo)}function L(t,e){var i=t.toLowerCase();vo[i]=vo[i+"s"]=vo[e]=t}function A(t){return"string"==typeof t?vo[t]||vo[t.toLowerCase()]:void 0}function B(t){var e,i,o={};for(i in t)r(t,i)&&(e=A(i),e&&(o[e]=t[i]));return o}function F(t,i){return function(o){return null!=o?(H(this,t,o),e.updateOffset(this,i),this):j(this,t)}}function j(t,e){return t.isValid()?t._d["get"+(t._isUTC?"UTC":"")+e]():NaN}function H(t,e,i){t.isValid()&&t._d["set"+(t._isUTC?"UTC":"")+e](i)}function W(t,e){var i;if("object"==typeof t)for(i in t)this.set(i,t[i]);else if(t=A(t),k(this[t]))return this[t](e);return this}function Y(t,e,i){var o=""+Math.abs(t),n=e-o.length,s=t>=0;return(s?i?"+":"":"-")+Math.pow(10,Math.max(0,n)).toString().substr(1)+o}function G(t,e,i,o){var n=o;"string"==typeof o&&(n=function(){return this[o]()}),t&&(wo[t]=n),e&&(wo[e[0]]=function(){return Y(n.apply(this,arguments),e[1],e[2])}),i&&(wo[i]=function(){return this.localeData().ordinal(n.apply(this,arguments),t)})}function V(t){return t.match(/\[[\s\S]/)?t.replace(/^\[|\]$/g,""):t.replace(/\\/g,"")}function U(t){var e,i,o=t.match(go);for(e=0,i=o.length;i>e;e++)wo[o[e]]?o[e]=wo[o[e]]:o[e]=V(o[e]);return function(e){var n,s="";for(n=0;i>n;n++)s+=o[n]instanceof Function?o[n].call(e,t):o[n];return s}}function q(t,e){return t.isValid()?(e=X(e,t.localeData()),bo[e]=bo[e]||U(e),bo[e](t)):t.localeData().invalidDate()}function X(t,e){function i(t){return e.longDateFormat(t)||t}var o=5;for(yo.lastIndex=0;o>=0&&yo.test(t);)t=t.replace(yo,i),yo.lastIndex=0,o-=1;return t}function Z(t,e,i){Bo[t]=k(e)?e:function(t,o){return t&&i?i:e}}function K(t,e){return r(Bo,t)?Bo[t](e._strict,e._locale):new RegExp(J(t))}function J(t){return Q(t.replace("\\","").replace(/\\(\[)|\\(\])|\[([^\]\[]*)\]|\\(.)/g,function(t,e,i,o,n){return e||i||o||n}))}function Q(t){return t.replace(/[-\/\\^$*+?.()|[\]{}]/g,"\\$&")}function $(t,e){var i,o=e;for("string"==typeof t&&(t=[t]),"number"==typeof e&&(o=function(t,i){i[e]=y(t)}),i=0;io;++o)s=h([2e3,o]),this._shortMonthsParse[o]=this.monthsShort(s,"").toLocaleLowerCase(),this._longMonthsParse[o]=this.months(s,"").toLocaleLowerCase();return i?"MMM"===e?(n=fo.call(this._shortMonthsParse,r),-1!==n?n:null):(n=fo.call(this._longMonthsParse,r),-1!==n?n:null):"MMM"===e?(n=fo.call(this._shortMonthsParse,r),-1!==n?n:(n=fo.call(this._longMonthsParse,r),-1!==n?n:null)):(n=fo.call(this._longMonthsParse,r),-1!==n?n:(n=fo.call(this._shortMonthsParse,r),-1!==n?n:null))}function rt(t,e,i){var o,n,s;if(this._monthsParseExact)return st.call(this,t,e,i);for(this._monthsParse||(this._monthsParse=[],this._longMonthsParse=[],this._shortMonthsParse=[]),o=0;12>o;o++){if(n=h([2e3,o]),i&&!this._longMonthsParse[o]&&(this._longMonthsParse[o]=new RegExp("^"+this.months(n,"").replace(".","")+"$","i"),this._shortMonthsParse[o]=new RegExp("^"+this.monthsShort(n,"").replace(".","")+"$","i")),i||this._monthsParse[o]||(s="^"+this.months(n,"")+"|^"+this.monthsShort(n,""),this._monthsParse[o]=new RegExp(s.replace(".",""),"i")),i&&"MMMM"===e&&this._longMonthsParse[o].test(t))return o;if(i&&"MMM"===e&&this._shortMonthsParse[o].test(t))return o;if(!i&&this._monthsParse[o].test(t))return o}}function at(t,e){var i;if(!t.isValid())return t;if("string"==typeof e)if(/^\d+$/.test(e))e=y(e);else if(e=t.localeData().monthsParse(e),"number"!=typeof e)return t;return i=Math.min(t.date(),it(t.year(),e)),t._d["set"+(t._isUTC?"UTC":"")+"Month"](e,i),t}function ht(t){return null!=t?(at(this,t),e.updateOffset(this,!0),this):j(this,"Month")}function dt(){return it(this.year(),this.month())}function lt(t){return this._monthsParseExact?(r(this,"_monthsRegex")||ut.call(this),t?this._monthsShortStrictRegex:this._monthsShortRegex):this._monthsShortStrictRegex&&t?this._monthsShortStrictRegex:this._monthsShortRegex}function ct(t){return this._monthsParseExact?(r(this,"_monthsRegex")||ut.call(this),t?this._monthsStrictRegex:this._monthsRegex):this._monthsStrictRegex&&t?this._monthsStrictRegex:this._monthsRegex}function ut(){function t(t,e){return e.length-t.length}var e,i,o=[],n=[],s=[];for(e=0;12>e;e++)i=h([2e3,e]),o.push(this.monthsShort(i,"")),n.push(this.months(i,"")),s.push(this.months(i,"")),s.push(this.monthsShort(i,""));for(o.sort(t),n.sort(t),s.sort(t),e=0;12>e;e++)o[e]=Q(o[e]),n[e]=Q(n[e]),s[e]=Q(s[e]);this._monthsRegex=new RegExp("^("+s.join("|")+")","i"),this._monthsShortRegex=this._monthsRegex,this._monthsStrictRegex=new RegExp("^("+n.join("|")+")","i"),this._monthsShortStrictRegex=new RegExp("^("+o.join("|")+")","i")}function pt(t){var e,i=t._a;return i&&-2===l(t).overflow&&(e=i[Ho]<0||i[Ho]>11?Ho:i[Wo]<1||i[Wo]>it(i[jo],i[Ho])?Wo:i[Yo]<0||i[Yo]>24||24===i[Yo]&&(0!==i[Go]||0!==i[Vo]||0!==i[Uo])?Yo:i[Go]<0||i[Go]>59?Go:i[Vo]<0||i[Vo]>59?Vo:i[Uo]<0||i[Uo]>999?Uo:-1,l(t)._overflowDayOfYear&&(jo>e||e>Wo)&&(e=Wo),l(t)._overflowWeeks&&-1===e&&(e=qo),l(t)._overflowWeekday&&-1===e&&(e=Xo),l(t).overflow=e),t}function ft(t){var e,i,o,n,s,r,a=t._i,h=tn.exec(a)||en.exec(a);if(h){for(l(t).iso=!0,e=0,i=nn.length;i>e;e++)if(nn[e][1].exec(h[1])){n=nn[e][0],o=nn[e][2]!==!1;break}if(null==n)return void(t._isValid=!1);if(h[3]){for(e=0,i=sn.length;i>e;e++)if(sn[e][1].exec(h[3])){s=(h[2]||" ")+sn[e][0];break}if(null==s)return void(t._isValid=!1)}if(!o&&null!=s)return void(t._isValid=!1);if(h[4]){if(!on.exec(h[4]))return void(t._isValid=!1);r="Z"}t._f=n+(s||"")+(r||""),Tt(t)}else t._isValid=!1}function mt(t){var i=rn.exec(t._i);return null!==i?void(t._d=new Date(+i[1])):(ft(t),void(t._isValid===!1&&(delete t._isValid,e.createFromInputFallback(t))))}function vt(t,e,i,o,n,s,r){var a=new Date(t,e,i,o,n,s,r);return 100>t&&t>=0&&isFinite(a.getFullYear())&&a.setFullYear(t),a}function gt(t){var e=new Date(Date.UTC.apply(null,arguments));return 100>t&&t>=0&&isFinite(e.getUTCFullYear())&&e.setUTCFullYear(t),e}function yt(t){return bt(t)?366:365}function bt(t){return t%4===0&&t%100!==0||t%400===0}function wt(){return bt(this.year())}function _t(t,e,i){var o=7+e-i,n=(7+gt(t,0,o).getUTCDay()-e)%7;return-n+o-1}function xt(t,e,i,o,n){var s,r,a=(7+i-o)%7,h=_t(t,o,n),d=1+7*(e-1)+a+h;return 0>=d?(s=t-1,r=yt(s)+d):d>yt(t)?(s=t+1,r=d-yt(t)):(s=t,r=d),{year:s,dayOfYear:r}}function kt(t,e,i){var o,n,s=_t(t.year(),e,i),r=Math.floor((t.dayOfYear()-s-1)/7)+1;return 1>r?(n=t.year()-1,o=r+Ot(n,e,i)):r>Ot(t.year(),e,i)?(o=r-Ot(t.year(),e,i),n=t.year()+1):(n=t.year(),o=r),{week:o,year:n}}function Ot(t,e,i){var o=_t(t,e,i),n=_t(t+1,e,i);return(yt(t)-o+n)/7}function Mt(t,e,i){return null!=t?t:null!=e?e:i}function Dt(t){var i=new Date(e.now());return t._useUTC?[i.getUTCFullYear(),i.getUTCMonth(),i.getUTCDate()]:[i.getFullYear(),i.getMonth(),i.getDate()]}function St(t){var e,i,o,n,s=[];if(!t._d){for(o=Dt(t),t._w&&null==t._a[Wo]&&null==t._a[Ho]&&Ct(t),t._dayOfYear&&(n=Mt(t._a[jo],o[jo]),t._dayOfYear>yt(n)&&(l(t)._overflowDayOfYear=!0),i=gt(n,0,t._dayOfYear),t._a[Ho]=i.getUTCMonth(),t._a[Wo]=i.getUTCDate()),e=0;3>e&&null==t._a[e];++e)t._a[e]=s[e]=o[e];for(;7>e;e++)t._a[e]=s[e]=null==t._a[e]?2===e?1:0:t._a[e];24===t._a[Yo]&&0===t._a[Go]&&0===t._a[Vo]&&0===t._a[Uo]&&(t._nextDay=!0,t._a[Yo]=0),t._d=(t._useUTC?gt:vt).apply(null,s),null!=t._tzm&&t._d.setUTCMinutes(t._d.getUTCMinutes()-t._tzm),t._nextDay&&(t._a[Yo]=24)}}function Ct(t){var e,i,o,n,s,r,a,h;e=t._w,null!=e.GG||null!=e.W||null!=e.E?(s=1,r=4,i=Mt(e.GG,t._a[jo],kt(At(),1,4).year),o=Mt(e.W,1),n=Mt(e.E,1),(1>n||n>7)&&(h=!0)):(s=t._locale._week.dow,r=t._locale._week.doy,i=Mt(e.gg,t._a[jo],kt(At(),s,r).year),o=Mt(e.w,1),null!=e.d?(n=e.d,(0>n||n>6)&&(h=!0)):null!=e.e?(n=e.e+s,(e.e<0||e.e>6)&&(h=!0)):n=s),1>o||o>Ot(i,s,r)?l(t)._overflowWeeks=!0:null!=h?l(t)._overflowWeekday=!0:(a=xt(i,o,n,s,r),t._a[jo]=a.year,t._dayOfYear=a.dayOfYear)}function Tt(t){if(t._f===e.ISO_8601)return void ft(t);t._a=[],l(t).empty=!0;var i,o,n,s,r,a=""+t._i,h=a.length,d=0;for(n=X(t._f,t._locale).match(go)||[],i=0;i0&&l(t).unusedInput.push(r),a=a.slice(a.indexOf(o)+o.length),d+=o.length),wo[s]?(o?l(t).empty=!1:l(t).unusedTokens.push(s),et(s,o,t)):t._strict&&!o&&l(t).unusedTokens.push(s);l(t).charsLeftOver=h-d,a.length>0&&l(t).unusedInput.push(a),l(t).bigHour===!0&&t._a[Yo]<=12&&t._a[Yo]>0&&(l(t).bigHour=void 0),l(t).parsedDateParts=t._a.slice(0),l(t).meridiem=t._meridiem,t._a[Yo]=Et(t._locale,t._a[Yo],t._meridiem),St(t),pt(t)}function Et(t,e,i){var o;return null==i?e:null!=t.meridiemHour?t.meridiemHour(e,i):null!=t.isPM?(o=t.isPM(i),o&&12>e&&(e+=12),o||12!==e||(e=0),e):e}function Pt(t){var e,i,o,n,s;if(0===t._f.length)return l(t).invalidFormat=!0,void(t._d=new Date(NaN));for(n=0;ns)&&(o=s,i=e));a(t,i||e)}function It(t){if(!t._d){var e=B(t._i);t._a=s([e.year,e.month,e.day||e.date,e.hour,e.minute,e.second,e.millisecond],function(t){return t&&parseInt(t,10)}),St(t)}}function Nt(t){var e=new m(pt(Rt(t)));return e._nextDay&&(e.add(1,"d"),e._nextDay=void 0),e}function Rt(t){var e=t._i,i=t._f;return t._locale=t._locale||R(t._l),null===e||void 0===i&&""===e?u({nullInput:!0}):("string"==typeof e&&(t._i=e=t._locale.preparse(e)),v(e)?new m(pt(e)):(o(i)?Pt(t):i?Tt(t):n(e)?t._d=e:zt(t),c(t)||(t._d=null),t))}function zt(t){var i=t._i;void 0===i?t._d=new Date(e.now()):n(i)?t._d=new Date(i.valueOf()):"string"==typeof i?mt(t):o(i)?(t._a=s(i.slice(0),function(t){return parseInt(t,10)}),St(t)):"object"==typeof i?It(t):"number"==typeof i?t._d=new Date(i):e.createFromInputFallback(t)}function Lt(t,e,i,o,n){var s={};return"boolean"==typeof i&&(o=i,i=void 0),s._isAMomentObject=!0,s._useUTC=s._isUTC=n,s._l=i,s._i=t,s._f=e,s._strict=o,Nt(s)}function At(t,e,i,o){return Lt(t,e,i,o,!1)}function Bt(t,e){var i,n;if(1===e.length&&o(e[0])&&(e=e[0]),!e.length)return At();for(i=e[0],n=1;nt&&(t=-t,i="-"),i+Y(~~(t/60),2)+e+Y(~~t%60,2)})}function Gt(t,e){var i=(e||"").match(t)||[],o=i[i.length-1]||[],n=(o+"").match(cn)||["-",0,0],s=+(60*n[1])+y(n[2]);return"+"===n[0]?s:-s}function Vt(t,i){var o,s;return i._isUTC?(o=i.clone(),s=(v(t)||n(t)?t.valueOf():At(t).valueOf())-o.valueOf(),o._d.setTime(o._d.valueOf()+s),e.updateOffset(o,!1),o):At(t).local()}function Ut(t){return 15*-Math.round(t._d.getTimezoneOffset()/15)}function qt(t,i){var o,n=this._offset||0;return this.isValid()?null!=t?("string"==typeof t?t=Gt(zo,t):Math.abs(t)<16&&(t=60*t),!this._isUTC&&i&&(o=Ut(this)),this._offset=t,this._isUTC=!0,null!=o&&this.add(o,"m"),n!==t&&(!i||this._changeInProgress?le(this,ne(t-n,"m"),1,!1):this._changeInProgress||(this._changeInProgress=!0,e.updateOffset(this,!0),this._changeInProgress=null)),this):this._isUTC?n:Ut(this):null!=t?this:NaN}function Xt(t,e){return null!=t?("string"!=typeof t&&(t=-t),this.utcOffset(t,e),this):-this.utcOffset()}function Zt(t){return this.utcOffset(0,t)}function Kt(t){return this._isUTC&&(this.utcOffset(0,t),this._isUTC=!1,t&&this.subtract(Ut(this),"m")),this}function Jt(){return this._tzm?this.utcOffset(this._tzm):"string"==typeof this._i&&this.utcOffset(Gt(Ro,this._i)),this}function Qt(t){return this.isValid()?(t=t?At(t).utcOffset():0,(this.utcOffset()-t)%60===0):!1}function $t(){return this.utcOffset()>this.clone().month(0).utcOffset()||this.utcOffset()>this.clone().month(5).utcOffset()}function te(){if(!p(this._isDSTShifted))return this._isDSTShifted;var t={};if(f(t,this),t=Rt(t),t._a){var e=t._isUTC?h(t._a):At(t._a);this._isDSTShifted=this.isValid()&&b(t._a,e.toArray())>0}else this._isDSTShifted=!1;return this._isDSTShifted}function ee(){return this.isValid()?!this._isUTC:!1}function ie(){return this.isValid()?this._isUTC:!1}function oe(){return this.isValid()?this._isUTC&&0===this._offset:!1}function ne(t,e){var i,o,n,s=t,a=null;return Wt(t)?s={ms:t._milliseconds,d:t._days,M:t._months}:"number"==typeof t?(s={},e?s[e]=t:s.milliseconds=t):(a=un.exec(t))?(i="-"===a[1]?-1:1,s={y:0,d:y(a[Wo])*i,h:y(a[Yo])*i,m:y(a[Go])*i,s:y(a[Vo])*i,ms:y(a[Uo])*i}):(a=pn.exec(t))?(i="-"===a[1]?-1:1,s={y:se(a[2],i),M:se(a[3],i),w:se(a[4],i),d:se(a[5],i),h:se(a[6],i),m:se(a[7],i),s:se(a[8],i)}):null==s?s={}:"object"==typeof s&&("from"in s||"to"in s)&&(n=ae(At(s.from),At(s.to)),s={},s.ms=n.milliseconds,s.M=n.months),o=new Ht(s),Wt(t)&&r(t,"_locale")&&(o._locale=t._locale),o}function se(t,e){var i=t&&parseFloat(t.replace(",","."));return(isNaN(i)?0:i)*e}function re(t,e){var i={milliseconds:0,months:0};return i.months=e.month()-t.month()+12*(e.year()-t.year()),t.clone().add(i.months,"M").isAfter(e)&&--i.months, +i.milliseconds=+e-+t.clone().add(i.months,"M"),i}function ae(t,e){var i;return t.isValid()&&e.isValid()?(e=Vt(e,t),t.isBefore(e)?i=re(t,e):(i=re(e,t),i.milliseconds=-i.milliseconds,i.months=-i.months),i):{milliseconds:0,months:0}}function he(t){return 0>t?-1*Math.round(-1*t):Math.round(t)}function de(t,e){return function(i,o){var n,s;return null===o||isNaN(+o)||(x(e,"moment()."+e+"(period, number) is deprecated. Please use moment()."+e+"(number, period)."),s=i,i=o,o=s),i="string"==typeof i?+i:i,n=ne(i,o),le(this,n,t),this}}function le(t,i,o,n){var s=i._milliseconds,r=he(i._days),a=he(i._months);t.isValid()&&(n=null==n?!0:n,s&&t._d.setTime(t._d.valueOf()+s*o),r&&H(t,"Date",j(t,"Date")+r*o),a&&at(t,j(t,"Month")+a*o),n&&e.updateOffset(t,r||a))}function ce(t,e){var i=t||At(),o=Vt(i,this).startOf("day"),n=this.diff(o,"days",!0),s=-6>n?"sameElse":-1>n?"lastWeek":0>n?"lastDay":1>n?"sameDay":2>n?"nextDay":7>n?"nextWeek":"sameElse",r=e&&(k(e[s])?e[s]():e[s]);return this.format(r||this.localeData().calendar(s,this,At(i)))}function ue(){return new m(this)}function pe(t,e){var i=v(t)?t:At(t);return this.isValid()&&i.isValid()?(e=A(p(e)?"millisecond":e),"millisecond"===e?this.valueOf()>i.valueOf():i.valueOf()e-s?(i=t.clone().add(n-1,"months"),o=(e-s)/(s-i)):(i=t.clone().add(n+1,"months"),o=(e-s)/(i-s)),-(n+o)||0}function _e(){return this.clone().locale("en").format("ddd MMM DD YYYY HH:mm:ss [GMT]ZZ")}function xe(){var t=this.clone().utc();return 0s&&(e=s),Xe.call(this,t,e,i,o,n))}function Xe(t,e,i,o,n){var s=xt(t,e,i,o,n),r=gt(s.year,0,s.dayOfYear);return this.year(r.getUTCFullYear()),this.month(r.getUTCMonth()),this.date(r.getUTCDate()),this}function Ze(t){return null==t?Math.ceil((this.month()+1)/3):this.month(3*(t-1)+this.month()%3)}function Ke(t){return kt(t,this._week.dow,this._week.doy).week}function Je(){return this._week.dow}function Qe(){return this._week.doy}function $e(t){var e=this.localeData().week(this);return null==t?e:this.add(7*(t-e),"d")}function ti(t){var e=kt(this,1,4).week;return null==t?e:this.add(7*(t-e),"d")}function ei(t,e){return"string"!=typeof t?t:isNaN(t)?(t=e.weekdaysParse(t),"number"==typeof t?t:null):parseInt(t,10)}function ii(t,e){return o(this._weekdays)?this._weekdays[t.day()]:this._weekdays[this._weekdays.isFormat.test(e)?"format":"standalone"][t.day()]}function oi(t){return this._weekdaysShort[t.day()]}function ni(t){return this._weekdaysMin[t.day()]}function si(t,e,i){var o,n,s,r=t.toLocaleLowerCase();if(!this._weekdaysParse)for(this._weekdaysParse=[],this._shortWeekdaysParse=[],this._minWeekdaysParse=[],o=0;7>o;++o)s=h([2e3,1]).day(o),this._minWeekdaysParse[o]=this.weekdaysMin(s,"").toLocaleLowerCase(),this._shortWeekdaysParse[o]=this.weekdaysShort(s,"").toLocaleLowerCase(),this._weekdaysParse[o]=this.weekdays(s,"").toLocaleLowerCase();return i?"dddd"===e?(n=fo.call(this._weekdaysParse,r),-1!==n?n:null):"ddd"===e?(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:null):(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null):"dddd"===e?(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null))):"ddd"===e?(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:null))):(n=fo.call(this._minWeekdaysParse,r),-1!==n?n:(n=fo.call(this._weekdaysParse,r),-1!==n?n:(n=fo.call(this._shortWeekdaysParse,r),-1!==n?n:null)))}function ri(t,e,i){var o,n,s;if(this._weekdaysParseExact)return si.call(this,t,e,i);for(this._weekdaysParse||(this._weekdaysParse=[],this._minWeekdaysParse=[],this._shortWeekdaysParse=[],this._fullWeekdaysParse=[]),o=0;7>o;o++){if(n=h([2e3,1]).day(o),i&&!this._fullWeekdaysParse[o]&&(this._fullWeekdaysParse[o]=new RegExp("^"+this.weekdays(n,"").replace(".",".?")+"$","i"),this._shortWeekdaysParse[o]=new RegExp("^"+this.weekdaysShort(n,"").replace(".",".?")+"$","i"),this._minWeekdaysParse[o]=new RegExp("^"+this.weekdaysMin(n,"").replace(".",".?")+"$","i")),this._weekdaysParse[o]||(s="^"+this.weekdays(n,"")+"|^"+this.weekdaysShort(n,"")+"|^"+this.weekdaysMin(n,""),this._weekdaysParse[o]=new RegExp(s.replace(".",""),"i")),i&&"dddd"===e&&this._fullWeekdaysParse[o].test(t))return o;if(i&&"ddd"===e&&this._shortWeekdaysParse[o].test(t))return o;if(i&&"dd"===e&&this._minWeekdaysParse[o].test(t))return o;if(!i&&this._weekdaysParse[o].test(t))return o}}function ai(t){if(!this.isValid())return null!=t?this:NaN;var e=this._isUTC?this._d.getUTCDay():this._d.getDay();return null!=t?(t=ei(t,this.localeData()),this.add(t-e,"d")):e}function hi(t){if(!this.isValid())return null!=t?this:NaN;var e=(this.day()+7-this.localeData()._week.dow)%7;return null==t?e:this.add(t-e,"d")}function di(t){return this.isValid()?null==t?this.day()||7:this.day(this.day()%7?t:t-7):null!=t?this:NaN}function li(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysStrictRegex:this._weekdaysRegex):this._weekdaysStrictRegex&&t?this._weekdaysStrictRegex:this._weekdaysRegex}function ci(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex):this._weekdaysShortStrictRegex&&t?this._weekdaysShortStrictRegex:this._weekdaysShortRegex}function ui(t){return this._weekdaysParseExact?(r(this,"_weekdaysRegex")||pi.call(this),t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex):this._weekdaysMinStrictRegex&&t?this._weekdaysMinStrictRegex:this._weekdaysMinRegex}function pi(){function t(t,e){return e.length-t.length}var e,i,o,n,s,r=[],a=[],d=[],l=[];for(e=0;7>e;e++)i=h([2e3,1]).day(e),o=this.weekdaysMin(i,""),n=this.weekdaysShort(i,""),s=this.weekdays(i,""),r.push(o),a.push(n),d.push(s),l.push(o),l.push(n),l.push(s);for(r.sort(t),a.sort(t),d.sort(t),l.sort(t),e=0;7>e;e++)a[e]=Q(a[e]),d[e]=Q(d[e]),l[e]=Q(l[e]);this._weekdaysRegex=new RegExp("^("+l.join("|")+")","i"),this._weekdaysShortRegex=this._weekdaysRegex,this._weekdaysMinRegex=this._weekdaysRegex,this._weekdaysStrictRegex=new RegExp("^("+d.join("|")+")","i"),this._weekdaysShortStrictRegex=new RegExp("^("+a.join("|")+")","i"),this._weekdaysMinStrictRegex=new RegExp("^("+r.join("|")+")","i")}function fi(t){var e=Math.round((this.clone().startOf("day")-this.clone().startOf("year"))/864e5)+1;return null==t?e:this.add(t-e,"d")}function mi(){return this.hours()%12||12}function vi(){return this.hours()||24}function gi(t,e){G(t,0,0,function(){return this.localeData().meridiem(this.hours(),this.minutes(),e)})}function yi(t,e){return e._meridiemParse}function bi(t){return"p"===(t+"").toLowerCase().charAt(0)}function wi(t,e,i){return t>11?i?"pm":"PM":i?"am":"AM"}function _i(t,e){e[Uo]=y(1e3*("0."+t))}function xi(){return this._isUTC?"UTC":""}function ki(){return this._isUTC?"Coordinated Universal Time":""}function Oi(t){return At(1e3*t)}function Mi(){return At.apply(null,arguments).parseZone()}function Di(t,e,i){var o=this._calendar[t];return k(o)?o.call(e,i):o}function Si(t){var e=this._longDateFormat[t],i=this._longDateFormat[t.toUpperCase()];return e||!i?e:(this._longDateFormat[t]=i.replace(/MMMM|MM|DD|dddd/g,function(t){return t.slice(1)}),this._longDateFormat[t])}function Ci(){return this._invalidDate}function Ti(t){return this._ordinal.replace("%d",t)}function Ei(t){return t}function Pi(t,e,i,o){var n=this._relativeTime[i];return k(n)?n(t,e,i,o):n.replace(/%d/i,t)}function Ii(t,e){var i=this._relativeTime[t>0?"future":"past"];return k(i)?i(e):i.replace(/%s/i,e)}function Ni(t,e,i,o){var n=R(),s=h().set(o,e);return n[i](s,t)}function Ri(t,e,i){if("number"==typeof t&&(e=t,t=void 0),t=t||"",null!=e)return Ni(t,e,i,"month");var o,n=[];for(o=0;12>o;o++)n[o]=Ni(t,o,i,"month");return n}function zi(t,e,i,o){"boolean"==typeof t?("number"==typeof e&&(i=e,e=void 0),e=e||""):(e=t,i=e,t=!1,"number"==typeof e&&(i=e,e=void 0),e=e||"");var n=R(),s=t?n._week.dow:0;if(null!=i)return Ni(e,(i+s)%7,o,"day");var r,a=[];for(r=0;7>r;r++)a[r]=Ni(e,(r+s)%7,o,"day");return a}function Li(t,e){return Ri(t,e,"months")}function Ai(t,e){return Ri(t,e,"monthsShort")}function Bi(t,e,i){return zi(t,e,i,"weekdays")}function Fi(t,e,i){return zi(t,e,i,"weekdaysShort")}function ji(t,e,i){return zi(t,e,i,"weekdaysMin")}function Hi(){var t=this._data;return this._milliseconds=jn(this._milliseconds),this._days=jn(this._days),this._months=jn(this._months),t.milliseconds=jn(t.milliseconds),t.seconds=jn(t.seconds),t.minutes=jn(t.minutes),t.hours=jn(t.hours),t.months=jn(t.months),t.years=jn(t.years),this}function Wi(t,e,i,o){var n=ne(e,i);return t._milliseconds+=o*n._milliseconds,t._days+=o*n._days,t._months+=o*n._months,t._bubble()}function Yi(t,e){return Wi(this,t,e,1)}function Gi(t,e){return Wi(this,t,e,-1)}function Vi(t){return 0>t?Math.floor(t):Math.ceil(t)}function Ui(){var t,e,i,o,n,s=this._milliseconds,r=this._days,a=this._months,h=this._data;return s>=0&&r>=0&&a>=0||0>=s&&0>=r&&0>=a||(s+=864e5*Vi(Xi(a)+r),r=0,a=0),h.milliseconds=s%1e3,t=g(s/1e3),h.seconds=t%60,e=g(t/60),h.minutes=e%60,i=g(e/60),h.hours=i%24,r+=g(i/24),n=g(qi(r)),a+=n,r-=Vi(Xi(n)),o=g(a/12),a%=12,h.days=r,h.months=a,h.years=o,this}function qi(t){return 4800*t/146097}function Xi(t){return 146097*t/4800}function Zi(t){var e,i,o=this._milliseconds;if(t=A(t),"month"===t||"year"===t)return e=this._days+o/864e5,i=this._months+qi(e),"month"===t?i:i/12;switch(e=this._days+Math.round(Xi(this._months)),t){case"week":return e/7+o/6048e5;case"day":return e+o/864e5;case"hour":return 24*e+o/36e5;case"minute":return 1440*e+o/6e4;case"second":return 86400*e+o/1e3;case"millisecond":return Math.floor(864e5*e)+o;default:throw new Error("Unknown unit "+t)}}function Ki(){return this._milliseconds+864e5*this._days+this._months%12*2592e6+31536e6*y(this._months/12)}function Ji(t){return function(){return this.as(t)}}function Qi(t){return t=A(t),this[t+"s"]()}function $i(t){return function(){return this._data[t]}}function to(){return g(this.days()/7)}function eo(t,e,i,o,n){return n.relativeTime(e||1,!!i,t,o)}function io(t,e,i){var o=ne(t).abs(),n=is(o.as("s")),s=is(o.as("m")),r=is(o.as("h")),a=is(o.as("d")),h=is(o.as("M")),d=is(o.as("y")),l=n=s&&["m"]||s=r&&["h"]||r=a&&["d"]||a=h&&["M"]||h=d&&["y"]||["yy",d];return l[2]=e,l[3]=+t>0,l[4]=i,eo.apply(null,l)}function oo(t,e){return void 0===os[t]?!1:void 0===e?os[t]:(os[t]=e,!0)}function no(t){var e=this.localeData(),i=io(this,!t,e);return t&&(i=e.pastFuture(+this,i)),e.postformat(i)}function so(){var t,e,i,o=ns(this._milliseconds)/1e3,n=ns(this._days),s=ns(this._months);t=g(o/60),e=g(t/60),o%=60,t%=60,i=g(s/12),s%=12;var r=i,a=s,h=n,d=e,l=t,c=o,u=this.asSeconds();return u?(0>u?"-":"")+"P"+(r?r+"Y":"")+(a?a+"M":"")+(h?h+"D":"")+(d||l||c?"T":"")+(d?d+"H":"")+(l?l+"M":"")+(c?c+"S":""):"P0D"}var ro,ao;ao=Array.prototype.some?Array.prototype.some:function(t){for(var e=Object(this),i=e.length>>>0,o=0;i>o;o++)if(o in e&&t.call(this,e[o],o,e))return!0;return!1};var ho=e.momentProperties=[],lo=!1,co={};e.suppressDeprecationWarnings=!1,e.deprecationHandler=null;var uo;uo=Object.keys?Object.keys:function(t){var e,i=[];for(e in t)r(t,e)&&i.push(e);return i};var po,fo,mo={},vo={},go=/(\[[^\[]*\])|(\\)?([Hh]mm(ss)?|Mo|MM?M?M?|Do|DDDo|DD?D?D?|ddd?d?|do?|w[o|w]?|W[o|W]?|Qo?|YYYYYY|YYYYY|YYYY|YY|gg(ggg?)?|GG(GGG?)?|e|E|a|A|hh?|HH?|kk?|mm?|ss?|S{1,9}|x|X|zz?|ZZ?|.)/g,yo=/(\[[^\[]*\])|(\\)?(LTS|LT|LL?L?L?|l{1,4})/g,bo={},wo={},_o=/\d/,xo=/\d\d/,ko=/\d{3}/,Oo=/\d{4}/,Mo=/[+-]?\d{6}/,Do=/\d\d?/,So=/\d\d\d\d?/,Co=/\d\d\d\d\d\d?/,To=/\d{1,3}/,Eo=/\d{1,4}/,Po=/[+-]?\d{1,6}/,Io=/\d+/,No=/[+-]?\d+/,Ro=/Z|[+-]\d\d:?\d\d/gi,zo=/Z|[+-]\d\d(?::?\d\d)?/gi,Lo=/[+-]?\d+(\.\d{1,3})?/,Ao=/[0-9]*['a-z\u00A0-\u05FF\u0700-\uD7FF\uF900-\uFDCF\uFDF0-\uFFEF]+|[\u0600-\u06FF\/]+(\s*?[\u0600-\u06FF]+){1,2}/i,Bo={},Fo={},jo=0,Ho=1,Wo=2,Yo=3,Go=4,Vo=5,Uo=6,qo=7,Xo=8;fo=Array.prototype.indexOf?Array.prototype.indexOf:function(t){var e;for(e=0;e=t?""+t:"+"+t}),G(0,["YY",2],0,function(){return this.year()%100}),G(0,["YYYY",4],0,"year"),G(0,["YYYYY",5],0,"year"),G(0,["YYYYYY",6,!0],0,"year"),L("year","y"),Z("Y",No),Z("YY",Do,xo),Z("YYYY",Eo,Oo),Z("YYYYY",Po,Mo),Z("YYYYYY",Po,Mo),$(["YYYYY","YYYYYY"],jo),$("YYYY",function(t,i){i[jo]=2===t.length?e.parseTwoDigitYear(t):y(t)}),$("YY",function(t,i){i[jo]=e.parseTwoDigitYear(t)}),$("Y",function(t,e){e[jo]=parseInt(t,10)}),e.parseTwoDigitYear=function(t){return y(t)+(y(t)>68?1900:2e3)};var an=F("FullYear",!0);e.ISO_8601=function(){};var hn=_("moment().min is deprecated, use moment.max instead. https://github.com/moment/moment/issues/1548",function(){var t=At.apply(null,arguments);return this.isValid()&&t.isValid()?this>t?this:t:u()}),dn=_("moment().max is deprecated, use moment.min instead. https://github.com/moment/moment/issues/1548",function(){var t=At.apply(null,arguments);return this.isValid()&&t.isValid()?t>this?this:t:u()}),ln=function(){return Date.now?Date.now():+new Date};Yt("Z",":"),Yt("ZZ",""),Z("Z",zo),Z("ZZ",zo),$(["Z","ZZ"],function(t,e,i){i._useUTC=!0,i._tzm=Gt(zo,t)});var cn=/([\+\-]|\d\d)/gi;e.updateOffset=function(){};var un=/^(\-)?(?:(\d*)[. ])?(\d+)\:(\d+)(?:\:(\d+)\.?(\d{3})?\d*)?$/,pn=/^(-)?P(?:(-?[0-9,.]*)Y)?(?:(-?[0-9,.]*)M)?(?:(-?[0-9,.]*)W)?(?:(-?[0-9,.]*)D)?(?:T(?:(-?[0-9,.]*)H)?(?:(-?[0-9,.]*)M)?(?:(-?[0-9,.]*)S)?)?$/;ne.fn=Ht.prototype;var fn=de(1,"add"),mn=de(-1,"subtract");e.defaultFormat="YYYY-MM-DDTHH:mm:ssZ",e.defaultFormatUtc="YYYY-MM-DDTHH:mm:ss[Z]";var vn=_("moment().lang() is deprecated. Instead, use moment().localeData() to get the language configuration. Use moment().locale() to change languages.",function(t){return void 0===t?this.localeData():this.locale(t)});G(0,["gg",2],0,function(){return this.weekYear()%100}),G(0,["GG",2],0,function(){return this.isoWeekYear()%100}),We("gggg","weekYear"),We("ggggg","weekYear"),We("GGGG","isoWeekYear"),We("GGGGG","isoWeekYear"),L("weekYear","gg"),L("isoWeekYear","GG"),Z("G",No),Z("g",No),Z("GG",Do,xo),Z("gg",Do,xo),Z("GGGG",Eo,Oo),Z("gggg",Eo,Oo),Z("GGGGG",Po,Mo),Z("ggggg",Po,Mo),tt(["gggg","ggggg","GGGG","GGGGG"],function(t,e,i,o){e[o.substr(0,2)]=y(t)}),tt(["gg","GG"],function(t,i,o,n){i[n]=e.parseTwoDigitYear(t)}),G("Q",0,"Qo","quarter"),L("quarter","Q"),Z("Q",_o),$("Q",function(t,e){e[Ho]=3*(y(t)-1)}),G("w",["ww",2],"wo","week"),G("W",["WW",2],"Wo","isoWeek"),L("week","w"),L("isoWeek","W"),Z("w",Do),Z("ww",Do,xo),Z("W",Do),Z("WW",Do,xo),tt(["w","ww","W","WW"],function(t,e,i,o){e[o.substr(0,1)]=y(t)});var gn={dow:0,doy:6};G("D",["DD",2],"Do","date"),L("date","D"),Z("D",Do),Z("DD",Do,xo),Z("Do",function(t,e){return t?e._ordinalParse:e._ordinalParseLenient}),$(["D","DD"],Wo),$("Do",function(t,e){e[Wo]=y(t.match(Do)[0],10)});var yn=F("Date",!0);G("d",0,"do","day"),G("dd",0,0,function(t){return this.localeData().weekdaysMin(this,t)}),G("ddd",0,0,function(t){return this.localeData().weekdaysShort(this,t)}),G("dddd",0,0,function(t){return this.localeData().weekdays(this,t)}),G("e",0,0,"weekday"),G("E",0,0,"isoWeekday"),L("day","d"),L("weekday","e"),L("isoWeekday","E"),Z("d",Do),Z("e",Do),Z("E",Do),Z("dd",function(t,e){return e.weekdaysMinRegex(t)}),Z("ddd",function(t,e){return e.weekdaysShortRegex(t)}),Z("dddd",function(t,e){return e.weekdaysRegex(t)}),tt(["dd","ddd","dddd"],function(t,e,i,o){var n=i._locale.weekdaysParse(t,o,i._strict);null!=n?e.d=n:l(i).invalidWeekday=t}),tt(["d","e","E"],function(t,e,i,o){e[o]=y(t)});var bn="Sunday_Monday_Tuesday_Wednesday_Thursday_Friday_Saturday".split("_"),wn="Sun_Mon_Tue_Wed_Thu_Fri_Sat".split("_"),_n="Su_Mo_Tu_We_Th_Fr_Sa".split("_"),xn=Ao,kn=Ao,On=Ao;G("DDD",["DDDD",3],"DDDo","dayOfYear"),L("dayOfYear","DDD"),Z("DDD",To),Z("DDDD",ko),$(["DDD","DDDD"],function(t,e,i){i._dayOfYear=y(t)}),G("H",["HH",2],0,"hour"),G("h",["hh",2],0,mi),G("k",["kk",2],0,vi),G("hmm",0,0,function(){return""+mi.apply(this)+Y(this.minutes(),2)}),G("hmmss",0,0,function(){return""+mi.apply(this)+Y(this.minutes(),2)+Y(this.seconds(),2)}),G("Hmm",0,0,function(){return""+this.hours()+Y(this.minutes(),2)}),G("Hmmss",0,0,function(){return""+this.hours()+Y(this.minutes(),2)+Y(this.seconds(),2)}),gi("a",!0),gi("A",!1),L("hour","h"),Z("a",yi),Z("A",yi),Z("H",Do),Z("h",Do),Z("HH",Do,xo),Z("hh",Do,xo),Z("hmm",So),Z("hmmss",Co),Z("Hmm",So),Z("Hmmss",Co),$(["H","HH"],Yo),$(["a","A"],function(t,e,i){i._isPm=i._locale.isPM(t),i._meridiem=t}),$(["h","hh"],function(t,e,i){e[Yo]=y(t),l(i).bigHour=!0}),$("hmm",function(t,e,i){var o=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o)),l(i).bigHour=!0}),$("hmmss",function(t,e,i){var o=t.length-4,n=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o,2)),e[Vo]=y(t.substr(n)),l(i).bigHour=!0}),$("Hmm",function(t,e,i){var o=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o))}),$("Hmmss",function(t,e,i){var o=t.length-4,n=t.length-2;e[Yo]=y(t.substr(0,o)),e[Go]=y(t.substr(o,2)),e[Vo]=y(t.substr(n))});var Mn=/[ap]\.?m?\.?/i,Dn=F("Hours",!0);G("m",["mm",2],0,"minute"),L("minute","m"),Z("m",Do),Z("mm",Do,xo),$(["m","mm"],Go);var Sn=F("Minutes",!1);G("s",["ss",2],0,"second"),L("second","s"),Z("s",Do),Z("ss",Do,xo),$(["s","ss"],Vo);var Cn=F("Seconds",!1);G("S",0,0,function(){return~~(this.millisecond()/100)}),G(0,["SS",2],0,function(){return~~(this.millisecond()/10)}),G(0,["SSS",3],0,"millisecond"),G(0,["SSSS",4],0,function(){return 10*this.millisecond()}),G(0,["SSSSS",5],0,function(){return 100*this.millisecond()}),G(0,["SSSSSS",6],0,function(){return 1e3*this.millisecond()}),G(0,["SSSSSSS",7],0,function(){return 1e4*this.millisecond()}),G(0,["SSSSSSSS",8],0,function(){return 1e5*this.millisecond()}),G(0,["SSSSSSSSS",9],0,function(){return 1e6*this.millisecond()}),L("millisecond","ms"),Z("S",To,_o),Z("SS",To,xo),Z("SSS",To,ko);var Tn;for(Tn="SSSS";Tn.length<=9;Tn+="S")Z(Tn,Io);for(Tn="S";Tn.length<=9;Tn+="S")$(Tn,_i);var En=F("Milliseconds",!1);G("z",0,0,"zoneAbbr"),G("zz",0,0,"zoneName");var Pn=m.prototype;Pn.add=fn,Pn.calendar=ce,Pn.clone=ue,Pn.diff=be,Pn.endOf=Pe,Pn.format=ke,Pn.from=Oe,Pn.fromNow=Me,Pn.to=De,Pn.toNow=Se,Pn.get=W,Pn.invalidAt=je,Pn.isAfter=pe,Pn.isBefore=fe,Pn.isBetween=me,Pn.isSame=ve,Pn.isSameOrAfter=ge,Pn.isSameOrBefore=ye,Pn.isValid=Be,Pn.lang=vn,Pn.locale=Ce,Pn.localeData=Te,Pn.max=dn,Pn.min=hn,Pn.parsingFlags=Fe,Pn.set=W,Pn.startOf=Ee,Pn.subtract=mn,Pn.toArray=ze,Pn.toObject=Le,Pn.toDate=Re,Pn.toISOString=xe,Pn.toJSON=Ae,Pn.toString=_e,Pn.unix=Ne,Pn.valueOf=Ie,Pn.creationData=He,Pn.year=an,Pn.isLeapYear=wt,Pn.weekYear=Ye,Pn.isoWeekYear=Ge,Pn.quarter=Pn.quarters=Ze,Pn.month=ht,Pn.daysInMonth=dt,Pn.week=Pn.weeks=$e,Pn.isoWeek=Pn.isoWeeks=ti,Pn.weeksInYear=Ue,Pn.isoWeeksInYear=Ve,Pn.date=yn,Pn.day=Pn.days=ai,Pn.weekday=hi,Pn.isoWeekday=di,Pn.dayOfYear=fi,Pn.hour=Pn.hours=Dn,Pn.minute=Pn.minutes=Sn,Pn.second=Pn.seconds=Cn,Pn.millisecond=Pn.milliseconds=En,Pn.utcOffset=qt,Pn.utc=Zt,Pn.local=Kt,Pn.parseZone=Jt,Pn.hasAlignedHourOffset=Qt,Pn.isDST=$t,Pn.isDSTShifted=te,Pn.isLocal=ee,Pn.isUtcOffset=ie,Pn.isUtc=oe,Pn.isUTC=oe,Pn.zoneAbbr=xi,Pn.zoneName=ki,Pn.dates=_("dates accessor is deprecated. Use date instead.",yn),Pn.months=_("months accessor is deprecated. Use month instead",ht),Pn.years=_("years accessor is deprecated. Use year instead",an),Pn.zone=_("moment().zone is deprecated, use moment().utcOffset instead. https://github.com/moment/moment/issues/1779",Xt);var In=Pn,Nn={sameDay:"[Today at] LT",nextDay:"[Tomorrow at] LT",nextWeek:"dddd [at] LT",lastDay:"[Yesterday at] LT",lastWeek:"[Last] dddd [at] LT",sameElse:"L"},Rn={LTS:"h:mm:ss A",LT:"h:mm A",L:"MM/DD/YYYY",LL:"MMMM D, YYYY",LLL:"MMMM D, YYYY h:mm A",LLLL:"dddd, MMMM D, YYYY h:mm A"},zn="Invalid date",Ln="%d",An=/\d{1,2}/,Bn={future:"in %s",past:"%s ago",s:"a few seconds",m:"a minute",mm:"%d minutes",h:"an hour",hh:"%d hours",d:"a day",dd:"%d days",M:"a month",MM:"%d months",y:"a year",yy:"%d years"},Fn=S.prototype;Fn._calendar=Nn,Fn.calendar=Di,Fn._longDateFormat=Rn,Fn.longDateFormat=Si,Fn._invalidDate=zn,Fn.invalidDate=Ci,Fn._ordinal=Ln,Fn.ordinal=Ti,Fn._ordinalParse=An,Fn.preparse=Ei,Fn.postformat=Ei,Fn._relativeTime=Bn,Fn.relativeTime=Pi,Fn.pastFuture=Ii,Fn.set=M,Fn.months=ot,Fn._months=Ko,Fn.monthsShort=nt,Fn._monthsShort=Jo,Fn.monthsParse=rt,Fn._monthsRegex=$o,Fn.monthsRegex=ct,Fn._monthsShortRegex=Qo,Fn.monthsShortRegex=lt,Fn.week=Ke,Fn._week=gn,Fn.firstDayOfYear=Qe,Fn.firstDayOfWeek=Je,Fn.weekdays=ii,Fn._weekdays=bn,Fn.weekdaysMin=ni,Fn._weekdaysMin=_n,Fn.weekdaysShort=oi,Fn._weekdaysShort=wn,Fn.weekdaysParse=ri,Fn._weekdaysRegex=xn,Fn.weekdaysRegex=li,Fn._weekdaysShortRegex=kn,Fn.weekdaysShortRegex=ci,Fn._weekdaysMinRegex=On,Fn.weekdaysMinRegex=ui,Fn.isPM=bi,Fn._meridiemParse=Mn,Fn.meridiem=wi,P("en",{ordinalParse:/\d{1,2}(th|st|nd|rd)/,ordinal:function(t){var e=t%10,i=1===y(t%100/10)?"th":1===e?"st":2===e?"nd":3===e?"rd":"th";return t+i}}),e.lang=_("moment.lang is deprecated. Use moment.locale instead.",P),e.langData=_("moment.langData is deprecated. Use moment.localeData instead.",R);var jn=Math.abs,Hn=Ji("ms"),Wn=Ji("s"),Yn=Ji("m"),Gn=Ji("h"),Vn=Ji("d"),Un=Ji("w"),qn=Ji("M"),Xn=Ji("y"),Zn=$i("milliseconds"),Kn=$i("seconds"),Jn=$i("minutes"),Qn=$i("hours"),$n=$i("days"),ts=$i("months"),es=$i("years"),is=Math.round,os={s:45,m:45,h:22,d:26,M:11},ns=Math.abs,ss=Ht.prototype;ss.abs=Hi,ss.add=Yi,ss.subtract=Gi,ss.as=Zi,ss.asMilliseconds=Hn,ss.asSeconds=Wn,ss.asMinutes=Yn,ss.asHours=Gn,ss.asDays=Vn,ss.asWeeks=Un,ss.asMonths=qn,ss.asYears=Xn,ss.valueOf=Ki,ss._bubble=Ui,ss.get=Qi,ss.milliseconds=Zn,ss.seconds=Kn,ss.minutes=Jn,ss.hours=Qn,ss.days=$n,ss.weeks=to,ss.months=ts,ss.years=es,ss.humanize=no,ss.toISOString=so,ss.toString=so,ss.toJSON=so,ss.locale=Ce,ss.localeData=Te,ss.toIsoString=_("toIsoString() is deprecated. Please use toISOString() instead (notice the capitals)",so),ss.lang=vn,G("X",0,0,"unix"),G("x",0,0,"valueOf"),Z("x",No),Z("X",Lo),$("X",function(t,e,i){i._d=new Date(1e3*parseFloat(t,10))}),$("x",function(t,e,i){i._d=new Date(y(t))}),e.version="2.13.0",i(At),e.fn=In,e.min=Ft,e.max=jt,e.now=ln,e.utc=h,e.unix=Oi,e.months=Li,e.isDate=n,e.locale=P,e.invalid=u,e.duration=ne,e.isMoment=v,e.weekdays=Bi,e.parseZone=Mi,e.localeData=R,e.isDuration=Wt,e.monthsShort=Ai,e.weekdaysMin=ji,e.defineLocale=I,e.updateLocale=N,e.locales=z,e.weekdaysShort=Fi,e.normalizeUnits=A,e.relativeTimeThreshold=oo,e.prototype=In;var rs=e;return rs})}).call(e,i(4)(t))},function(t,e){t.exports=function(t){return t.webpackPolyfill||(t.deprecate=function(){},t.paths=[],t.children=[],t.webpackPolyfill=1),t}},function(t,e){function i(t){throw new Error("Cannot find module '"+t+"'.")}i.keys=function(){return[]},i.resolve=i,t.exports=i,i.id=5},function(t,e){(function(e){function i(t,e,i){var o=e&&i||0,n=0;for(e=e||[],t.toLowerCase().replace(/[0-9a-f]{2}/g,function(t){16>n&&(e[o+n++]=c[t])});16>n;)e[o+n++]=0;return e}function o(t,e){var i=e||0,o=l;return o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+"-"+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]+o[t[i++]]}function n(t,e,i){var n=e&&i||0,s=e||[];t=t||{};var r=void 0!==t.clockseq?t.clockseq:m,a=void 0!==t.msecs?t.msecs:(new Date).getTime(),h=void 0!==t.nsecs?t.nsecs:g+1,d=a-v+(h-g)/1e4;if(0>d&&void 0===t.clockseq&&(r=r+1&16383),(0>d||a>v)&&void 0===t.nsecs&&(h=0),h>=1e4)throw new Error("uuid.v1(): Can't create more than 10M uuids/sec");v=a,g=h,m=r,a+=122192928e5;var l=(1e4*(268435455&a)+h)%4294967296;s[n++]=l>>>24&255,s[n++]=l>>>16&255,s[n++]=l>>>8&255,s[n++]=255&l;var c=a/4294967296*1e4&268435455;s[n++]=c>>>8&255,s[n++]=255&c,s[n++]=c>>>24&15|16,s[n++]=c>>>16&255,s[n++]=r>>>8|128,s[n++]=255&r;for(var u=t.node||f,p=0;6>p;p++)s[n+p]=u[p];return e?e:o(s)}function s(t,e,i){var n=e&&i||0;"string"==typeof t&&(e="binary"==t?new Array(16):null,t=null),t=t||{};var s=t.random||(t.rng||r)();if(s[6]=15&s[6]|64,s[8]=63&s[8]|128,e)for(var a=0;16>a;a++)e[n+a]=s[a];return e||o(s)}var r,a="undefined"!=typeof window?window:"undefined"!=typeof e?e:null;if(a&&a.crypto&&crypto.getRandomValues){var h=new Uint8Array(16);r=function(){return crypto.getRandomValues(h),h}}if(!r){var d=new Array(16);r=function(){for(var t,e=0;16>e;e++)0===(3&e)&&(t=4294967296*Math.random()),d[e]=t>>>((3&e)<<3)&255;return d}}for(var l=[],c={},u=0;256>u;u++)l[u]=(u+256).toString(16).substr(1),c[l[u]]=u;var p=r(),f=[1|p[0],p[1],p[2],p[3],p[4],p[5]],m=16383&(p[6]<<8|p[7]),v=0,g=0,y=s;y.v1=n,y.v4=s,y.parse=i,y.unparse=o,t.exports=y}).call(e,function(){return this}())},function(t,e,i){e.util=i(1),e.DOMutil=i(8),e.DataSet=i(9),e.DataView=i(11),e.Queue=i(10),e.Graph3d=i(12),e.graph3d={Camera:i(16),Filter:i(17),Point2d:i(15),Point3d:i(14),Slider:i(18),StepNumber:i(19)},e.moment=i(2),e.Hammer=i(20),e.keycharm=i(23)},function(t,e){e.prepareElements=function(t){for(var e in t)t.hasOwnProperty(e)&&(t[e].redundant=t[e].used,t[e].used=[])},e.cleanupElements=function(t){for(var e in t)if(t.hasOwnProperty(e)&&t[e].redundant){for(var i=0;i0?(o=e[t].redundant[0],e[t].redundant.shift()):(o=document.createElementNS("http://www.w3.org/2000/svg",t),i.appendChild(o)):(o=document.createElementNS("http://www.w3.org/2000/svg",t),e[t]={used:[],redundant:[]},i.appendChild(o)),e[t].used.push(o),o},e.getDOMElement=function(t,e,i,o){var n;return e.hasOwnProperty(t)?e[t].redundant.length>0?(n=e[t].redundant[0],e[t].redundant.shift()):(n=document.createElement(t),void 0!==o?i.insertBefore(n,o):i.appendChild(n)):(n=document.createElement(t),e[t]={used:[],redundant:[]},void 0!==o?i.insertBefore(n,o):i.appendChild(n)),e[t].used.push(n),n},e.drawPoint=function(t,i,o,n,s,r){var a;if("circle"==o.style?(a=e.getSVGElement("circle",n,s),a.setAttributeNS(null,"cx",t),a.setAttributeNS(null,"cy",i),a.setAttributeNS(null,"r",.5*o.size)):(a=e.getSVGElement("rect",n,s),a.setAttributeNS(null,"x",t-.5*o.size),a.setAttributeNS(null,"y",i-.5*o.size),a.setAttributeNS(null,"width",o.size),a.setAttributeNS(null,"height",o.size)),void 0!==o.styles&&a.setAttributeNS(null,"style",o.styles),a.setAttributeNS(null,"class",o.className+" vis-point"),r){var h=e.getSVGElement("text",n,s); +r.xOffset&&(t+=r.xOffset),r.yOffset&&(i+=r.yOffset),r.content&&(h.textContent=r.content),r.className&&h.setAttributeNS(null,"class",r.className+" vis-label"),h.setAttributeNS(null,"x",t),h.setAttributeNS(null,"y",i)}return a},e.drawBar=function(t,i,o,n,s,r,a,h){if(0!=n){0>n&&(n*=-1,i-=n);var d=e.getSVGElement("rect",r,a);d.setAttributeNS(null,"x",t-.5*o),d.setAttributeNS(null,"y",i),d.setAttributeNS(null,"width",o),d.setAttributeNS(null,"height",n),d.setAttributeNS(null,"class",s),h&&d.setAttributeNS(null,"style",h)}}},function(t,e,i){function o(t,e){if(t&&!Array.isArray(t)&&(e=t,t=null),this._options=e||{},this._data={},this.length=0,this._fieldId=this._options.fieldId||"id",this._type={},this._options.type)for(var i=Object.keys(this._options.type),o=0,n=i.length;n>o;o++){var s=i[o],r=this._options.type[s];"Date"==r||"ISODate"==r||"ASPDate"==r?this._type[s]="Date":this._type[s]=r}if(this._options.convert)throw new Error('Option "convert" is deprecated. Use "type" instead.');this._subscribers={},t&&this.add(t),this.setOptions(e)}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(10);o.prototype.setOptions=function(t){t&&void 0!==t.queue&&(t.queue===!1?this._queue&&(this._queue.destroy(),delete this._queue):(this._queue||(this._queue=r.extend(this,{replace:["add","update","remove"]})),"object"===n(t.queue)&&this._queue.setOptions(t.queue)))},o.prototype.on=function(t,e){var i=this._subscribers[t];i||(i=[],this._subscribers[t]=i),i.push({callback:e})},o.prototype.subscribe=function(){throw new Error("DataSet.subscribe is deprecated. Use DataSet.on instead.")},o.prototype.off=function(t,e){var i=this._subscribers[t];i&&(this._subscribers[t]=i.filter(function(t){return t.callback!=e}))},o.prototype.unsubscribe=function(){throw new Error("DataSet.unsubscribe is deprecated. Use DataSet.off instead.")},o.prototype._trigger=function(t,e,i){if("*"==t)throw new Error("Cannot trigger event *");var o=[];t in this._subscribers&&(o=o.concat(this._subscribers[t])),"*"in this._subscribers&&(o=o.concat(this._subscribers["*"]));for(var n=0,s=o.length;s>n;n++){var r=o[n];r.callback&&r.callback(t,e,i||null)}},o.prototype.add=function(t,e){var i,o=[],n=this;if(Array.isArray(t))for(var s=0,r=t.length;r>s;s++)i=n._addItem(t[s]),o.push(i);else{if(!(t instanceof Object))throw new Error("Unknown dataType");i=n._addItem(t),o.push(i)}return o.length&&this._trigger("add",{items:o},e),o},o.prototype.update=function(t,e){var i=[],o=[],n=[],r=[],a=this,h=a._fieldId,d=function(t){var e=t[h];if(a._data[e]){var d=s.extend({},a._data[e]);e=a._updateItem(t),o.push(e),r.push(t),n.push(d)}else e=a._addItem(t),i.push(e)};if(Array.isArray(t))for(var l=0,c=t.length;c>l;l++)t[l]instanceof Object?d(t[l]):console.warn("Ignoring input item, which is not an object at index "+l);else{if(!(t instanceof Object))throw new Error("Unknown dataType");d(t)}if(i.length&&this._trigger("add",{items:i},e),o.length){var u={items:o,oldData:n,data:r};this._trigger("update",u,e)}return i.concat(o)},o.prototype.get=function(t){var e,i,o,n=this,r=s.getType(arguments[0]);"String"==r||"Number"==r?(e=arguments[0],o=arguments[1]):"Array"==r?(i=arguments[0],o=arguments[1]):o=arguments[0];var a;if(o&&o.returnType){var h=["Array","Object"];a=-1==h.indexOf(o.returnType)?"Array":o.returnType}else a="Array";var d,l,c,u,p,f=o&&o.type||this._options.type,m=o&&o.filter,v=[];if(void 0!=e)d=n._getItem(e,f),d&&m&&!m(d)&&(d=null);else if(void 0!=i)for(u=0,p=i.length;p>u;u++)d=n._getItem(i[u],f),m&&!m(d)||v.push(d);else for(l=Object.keys(this._data),u=0,p=l.length;p>u;u++)c=l[u],d=n._getItem(c,f),m&&!m(d)||v.push(d);if(o&&o.order&&void 0==e&&this._sort(v,o.order),o&&o.fields){var g=o.fields;if(void 0!=e)d=this._filterFields(d,g);else for(u=0,p=v.length;p>u;u++)v[u]=this._filterFields(v[u],g)}if("Object"==a){var y,b={};for(u=0,p=v.length;p>u;u++)y=v[u],b[y.id]=y;return b}return void 0!=e?d:v},o.prototype.getIds=function(t){var e,i,o,n,s,r=this._data,a=t&&t.filter,h=t&&t.order,d=t&&t.type||this._options.type,l=Object.keys(r),c=[];if(a)if(h){for(s=[],e=0,i=l.length;i>e;e++)o=l[e],n=this._getItem(o,d),a(n)&&s.push(n);for(this._sort(s,h),e=0,i=s.length;i>e;e++)c.push(s[e][this._fieldId])}else for(e=0,i=l.length;i>e;e++)o=l[e],n=this._getItem(o,d),a(n)&&c.push(n[this._fieldId]);else if(h){for(s=[],e=0,i=l.length;i>e;e++)o=l[e],s.push(r[o]);for(this._sort(s,h),e=0,i=s.length;i>e;e++)c.push(s[e][this._fieldId])}else for(e=0,i=l.length;i>e;e++)o=l[e],n=r[o],c.push(n[this._fieldId]);return c},o.prototype.getDataSet=function(){return this},o.prototype.forEach=function(t,e){var i,o,n,s,r=e&&e.filter,a=e&&e.type||this._options.type,h=this._data,d=Object.keys(h);if(e&&e.order){var l=this.get(e);for(i=0,o=l.length;o>i;i++)n=l[i],s=n[this._fieldId],t(n,s)}else for(i=0,o=d.length;o>i;i++)s=d[i],n=this._getItem(s,a),r&&!r(n)||t(n,s)},o.prototype.map=function(t,e){var i,o,n,s,r=e&&e.filter,a=e&&e.type||this._options.type,h=[],d=this._data,l=Object.keys(d);for(i=0,o=l.length;o>i;i++)n=l[i],s=this._getItem(n,a),r&&!r(s)||h.push(t(s,n));return e&&e.order&&this._sort(h,e.order),h},o.prototype._filterFields=function(t,e){if(!t)return t;var i,o,n={},s=Object.keys(t),r=s.length;if(Array.isArray(e))for(i=0;r>i;i++)o=s[i],-1!=e.indexOf(o)&&(n[o]=t[o]);else for(i=0;r>i;i++)o=s[i],e.hasOwnProperty(o)&&(n[e[o]]=t[o]);return n},o.prototype._sort=function(t,e){if(s.isString(e)){var i=e;t.sort(function(t,e){var o=t[i],n=e[i];return o>n?1:n>o?-1:0})}else{if("function"!=typeof e)throw new TypeError("Order must be a function or a string");t.sort(e)}},o.prototype.remove=function(t,e){var i,o,n,s=[];if(Array.isArray(t))for(i=0,o=t.length;o>i;i++)n=this._remove(t[i]),null!=n&&s.push(n);else n=this._remove(t),null!=n&&s.push(n);return s.length&&this._trigger("remove",{items:s},e),s},o.prototype._remove=function(t){if(s.isNumber(t)||s.isString(t)){if(this._data[t])return delete this._data[t],this.length--,t}else if(t instanceof Object){var e=t[this._fieldId];if(void 0!==e&&this._data[e])return delete this._data[e],this.length--,e}return null},o.prototype.clear=function(t){var e=Object.keys(this._data);return this._data={},this.length=0,this._trigger("remove",{items:e},t),e},o.prototype.max=function(t){var e,i,o=this._data,n=Object.keys(o),s=null,r=null;for(e=0,i=n.length;i>e;e++){var a=n[e],h=o[a],d=h[t];null!=d&&(!s||d>r)&&(s=h,r=d)}return s},o.prototype.min=function(t){var e,i,o=this._data,n=Object.keys(o),s=null,r=null;for(e=0,i=n.length;i>e;e++){var a=n[e],h=o[a],d=h[t];null!=d&&(!s||r>d)&&(s=h,r=d)}return s},o.prototype.distinct=function(t){var e,i,o,n=this._data,r=Object.keys(n),a=[],h=this._options.type&&this._options.type[t]||null,d=0;for(e=0,o=r.length;o>e;e++){var l=r[e],c=n[l],u=c[t],p=!1;for(i=0;d>i;i++)if(a[i]==u){p=!0;break}p||void 0===u||(a[d]=u,d++)}if(h)for(e=0,o=a.length;o>e;e++)a[e]=s.convert(a[e],h);return a},o.prototype._addItem=function(t){var e=t[this._fieldId];if(void 0!=e){if(this._data[e])throw new Error("Cannot add item: item with id "+e+" already exists")}else e=s.randomUUID(),t[this._fieldId]=e;var i,o,n={},r=Object.keys(t);for(i=0,o=r.length;o>i;i++){var a=r[i],h=this._type[a];n[a]=s.convert(t[a],h)}return this._data[e]=n,this.length++,e},o.prototype._getItem=function(t,e){var i,o,n,r,a=this._data[t];if(!a)return null;var h={},d=Object.keys(a);if(e)for(n=0,r=d.length;r>n;n++)i=d[n],o=a[i],h[i]=s.convert(o,e[i]);else for(n=0,r=d.length;r>n;n++)i=d[n],o=a[i],h[i]=o;return h},o.prototype._updateItem=function(t){var e=t[this._fieldId];if(void 0==e)throw new Error("Cannot update item: item has no id (item: "+JSON.stringify(t)+")");var i=this._data[e];if(!i)throw new Error("Cannot update item: no item with id "+e+" found");for(var o=Object.keys(t),n=0,r=o.length;r>n;n++){var a=o[n],h=this._type[a];i[a]=s.convert(t[a],h)}return e},t.exports=o},function(t,e){function i(t){this.delay=null,this.max=1/0,this._queue=[],this._timeout=null,this._extended=null,this.setOptions(t)}i.prototype.setOptions=function(t){t&&"undefined"!=typeof t.delay&&(this.delay=t.delay),t&&"undefined"!=typeof t.max&&(this.max=t.max),this._flushIfNeeded()},i.extend=function(t,e){var o=new i(e);if(void 0!==t.flush)throw new Error("Target object already has a property flush");t.flush=function(){o.flush()};var n=[{name:"flush",original:void 0}];if(e&&e.replace)for(var s=0;sthis.max&&this.flush(),clearTimeout(this._timeout),this.queue.length>0&&"number"==typeof this.delay){var t=this;this._timeout=setTimeout(function(){t.flush()},this.delay)}},i.prototype.flush=function(){for(;this._queue.length>0;){var t=this._queue.shift();t.fn.apply(t.context||t.fn,t.args||[])}},t.exports=i},function(t,e,i){function o(t,e){this._data=null,this._ids={},this.length=0,this._options=e||{},this._fieldId="id",this._subscribers={};var i=this;this.listener=function(){i._onEvent.apply(i,arguments)},this.setData(t)}var n=i(1),s=i(9);o.prototype.setData=function(t){var e,i,o,n;if(this._data&&(this._data.off&&this._data.off("*",this.listener),e=Object.keys(this._ids),this._ids={},this.length=0,this._trigger("remove",{items:e})),this._data=t,this._data){for(this._fieldId=this._options.fieldId||this._data&&this._data.options&&this._data.options.fieldId||"id",e=this._data.getIds({filter:this._options&&this._options.filter}),o=0,n=e.length;n>o;o++)i=e[o],this._ids[i]=!0;this.length=e.length,this._trigger("add",{items:e}),this._data.on&&this._data.on("*",this.listener)}},o.prototype.refresh=function(){var t,e,i,o=this._data.getIds({filter:this._options&&this._options.filter}),n=Object.keys(this._ids),s={},r=[],a=[];for(e=0,i=o.length;i>e;e++)t=o[e],s[t]=!0,this._ids[t]||(r.push(t),this._ids[t]=!0);for(e=0,i=n.length;i>e;e++)t=n[e],s[t]||(a.push(t),delete this._ids[t]);this.length+=r.length-a.length,r.length&&this._trigger("add",{items:r}),a.length&&this._trigger("remove",{items:a})},o.prototype.get=function(t){var e,i,o,s=this,r=n.getType(arguments[0]);"String"==r||"Number"==r||"Array"==r?(e=arguments[0],i=arguments[1],o=arguments[2]):(i=arguments[0],o=arguments[1]);var a=n.extend({},this._options,i);this._options.filter&&i&&i.filter&&(a.filter=function(t){return s._options.filter(t)&&i.filter(t)});var h=[];return void 0!=e&&h.push(e),h.push(a),h.push(o),this._data&&this._data.get.apply(this._data,h)},o.prototype.getIds=function(t){var e;if(this._data){var i,o=this._options.filter;i=t&&t.filter?o?function(e){return o(e)&&t.filter(e)}:t.filter:o,e=this._data.getIds({filter:i,order:t&&t.order})}else e=[];return e},o.prototype.map=function(t,e){var i=[];if(this._data){var o,n=this._options.filter;o=e&&e.filter?n?function(t){return n(t)&&e.filter(t)}:e.filter:n,i=this._data.map(t,{filter:o,order:e&&e.order})}else i=[];return i},o.prototype.getDataSet=function(){for(var t=this;t instanceof o;)t=t._data;return t||null},o.prototype._onEvent=function(t,e,i){var o,n,s,r,a=e&&e.items,h=this._data,d=[],l=[],c=[],u=[];if(a&&h){switch(t){case"add":for(o=0,n=a.length;n>o;o++)s=a[o],r=this.get(s),r&&(this._ids[s]=!0,l.push(s));break;case"update":for(o=0,n=a.length;n>o;o++)s=a[o],r=this.get(s),r?this._ids[s]?(c.push(s),d.push(e.data[o])):(this._ids[s]=!0,l.push(s)):this._ids[s]&&(delete this._ids[s],u.push(s));break;case"remove":for(o=0,n=a.length;n>o;o++)s=a[o],this._ids[s]&&(delete this._ids[s],u.push(s))}this.length+=l.length-u.length,l.length&&this._trigger("add",{items:l},i),c.length&&this._trigger("update",{items:c,data:d},i),u.length&&this._trigger("remove",{items:u},i)}},o.prototype.on=s.prototype.on,o.prototype.off=s.prototype.off,o.prototype._trigger=s.prototype._trigger,o.prototype.subscribe=o.prototype.on,o.prototype.unsubscribe=o.prototype.off,t.exports=o},function(t,e,i){function o(t,e,i){if(!(this instanceof o))throw new SyntaxError("Constructor must be called with the new operator");this.containerElement=t,this.width="400px",this.height="400px",this.margin=10,this.defaultXCenter="55%",this.defaultYCenter="50%",this.xLabel="x",this.yLabel="y",this.zLabel="z";var n=function(t){return t};this.xValueLabel=n,this.yValueLabel=n,this.zValueLabel=n,this.filterLabel="time",this.legendLabel="value",this.style=o.STYLE.DOT,this.showPerspective=!0,this.showGrid=!0,this.keepAspectRatio=!0,this.showShadow=!1,this.showGrayBottom=!1,this.showTooltip=!1,this.verticalRatio=.5,this.animationInterval=1e3,this.animationPreload=!1,this.camera=new p,this.camera.setArmRotation(1,.5),this.camera.setArmLength(1.7),this.eye=new c(0,0,-1),this.dataTable=null,this.dataPoints=null,this.colX=void 0,this.colY=void 0,this.colZ=void 0,this.colValue=void 0,this.colFilter=void 0,this.xMin=0,this.xStep=void 0,this.xMax=1,this.yMin=0,this.yStep=void 0,this.yMax=1,this.zMin=0,this.zStep=void 0,this.zMax=1,this.valueMin=0,this.valueMax=1,this.xBarWidth=1,this.yBarWidth=1,this.axisColor="#4D4D4D",this.gridColor="#D3D3D3",this.dataColor={fill:"#7DC1FF",stroke:"#3267D2",strokeWidth:1},this.dotSizeRatio=.02,this.create(),this.setOptions(i),e&&this.setData(e)}function n(t){return"clientX"in t?t.clientX:t.targetTouches[0]&&t.targetTouches[0].clientX||0}function s(t){return"clientY"in t?t.clientY:t.targetTouches[0]&&t.targetTouches[0].clientY||0}var r="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},a=i(13),h=i(9),d=i(11),l=i(1),c=i(14),u=i(15),p=i(16),f=i(17),m=i(18),v=i(19);a(o.prototype),o.prototype._setScale=function(){this.scale=new c(1/(this.xMax-this.xMin),1/(this.yMax-this.yMin),1/(this.zMax-this.zMin)),this.keepAspectRatio&&(this.scale.x3&&(this.colFilter=3);else{if(this.style!==o.STYLE.DOTCOLOR&&this.style!==o.STYLE.DOTSIZE&&this.style!==o.STYLE.BARCOLOR&&this.style!==o.STYLE.BARSIZE)throw'Unknown style "'+this.style+'"';this.colX=0,this.colY=1,this.colZ=2,this.colValue=3,t.getNumberOfColumns()>4&&(this.colFilter=4)}},o.prototype.getNumberOfRows=function(t){return t.length},o.prototype.getNumberOfColumns=function(t){var e=0;for(var i in t[0])t[0].hasOwnProperty(i)&&e++;return e},o.prototype.getDistinctValues=function(t,e){for(var i=[],o=0;ot[o][e]&&(i.min=t[o][e]),i.maxt;t++){var f=(t-u)/(p-u),m=240*f,g=this._hsv2rgb(m,1,1);c.strokeStyle=g,c.beginPath(),c.moveTo(h,r+t),c.lineTo(a,r+t),c.stroke()}c.strokeStyle=this.axisColor,c.strokeRect(h,r,i,s)}if(this.style===o.STYLE.DOTSIZE&&(c.strokeStyle=this.axisColor,c.fillStyle=this.dataColor.fill,c.beginPath(),c.moveTo(h,r),c.lineTo(a,r),c.lineTo(a-i+e,d),c.lineTo(h,d),c.closePath(),c.fill(),c.stroke()),this.style===o.STYLE.DOTCOLOR||this.style===o.STYLE.DOTSIZE){var y=5,b=new v(this.valueMin,this.valueMax,(this.valueMax-this.valueMin)/5,!0);for(b.start(),b.getCurrent()0?this.yMin:this.yMax,n=this._convert3Dto2D(new c(_,r,this.zMin)),Math.cos(2*w)>0?(m.textAlign="center",m.textBaseline="top",n.y+=b):Math.sin(2*w)<0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(" "+this.xValueLabel(i.getCurrent())+" ",n.x,n.y),i.next()}for(m.lineWidth=1,o=void 0===this.defaultYStep,i=new v(this.yMin,this.yMax,this.yStep,o),i.start(),i.getCurrent()0?this.xMin:this.xMax,n=this._convert3Dto2D(new c(s,i.getCurrent(),this.zMin)),Math.cos(2*w)<0?(m.textAlign="center",m.textBaseline="top",n.y+=b):Math.sin(2*w)>0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(" "+this.yValueLabel(i.getCurrent())+" ",n.x,n.y),i.next();for(m.lineWidth=1,o=void 0===this.defaultZStep,i=new v(this.zMin,this.zMax,this.zStep,o),i.start(),i.getCurrent()0?this.xMin:this.xMax,r=Math.sin(w)<0?this.yMin:this.yMax;!i.end();)t=this._convert3Dto2D(new c(s,r,i.getCurrent())),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(t.x-b,t.y),m.stroke(),m.textAlign="right",m.textBaseline="middle",m.fillStyle=this.axisColor,m.fillText(this.zValueLabel(i.getCurrent())+" ",t.x-5,t.y),i.next();m.lineWidth=1,t=this._convert3Dto2D(new c(s,r,this.zMin)),e=this._convert3Dto2D(new c(s,r,this.zMax)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke(),m.lineWidth=1,u=this._convert3Dto2D(new c(this.xMin,this.yMin,this.zMin)),p=this._convert3Dto2D(new c(this.xMax,this.yMin,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(u.x,u.y),m.lineTo(p.x,p.y),m.stroke(),u=this._convert3Dto2D(new c(this.xMin,this.yMax,this.zMin)),p=this._convert3Dto2D(new c(this.xMax,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(u.x,u.y),m.lineTo(p.x,p.y),m.stroke(),m.lineWidth=1,t=this._convert3Dto2D(new c(this.xMin,this.yMin,this.zMin)),e=this._convert3Dto2D(new c(this.xMin,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke(),t=this._convert3Dto2D(new c(this.xMax,this.yMin,this.zMin)),e=this._convert3Dto2D(new c(this.xMax,this.yMax,this.zMin)),m.strokeStyle=this.axisColor,m.beginPath(),m.moveTo(t.x,t.y),m.lineTo(e.x,e.y),m.stroke();var x=this.xLabel;x.length>0&&(l=.1/this.scale.y,s=(this.xMin+this.xMax)/2,r=Math.cos(w)>0?this.yMin-l:this.yMax+l,n=this._convert3Dto2D(new c(s,r,this.zMin)),Math.cos(2*w)>0?(m.textAlign="center",m.textBaseline="top"):Math.sin(2*w)<0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(x,n.x,n.y));var k=this.yLabel;k.length>0&&(d=.1/this.scale.x,s=Math.sin(w)>0?this.xMin-d:this.xMax+d,r=(this.yMin+this.yMax)/2,n=this._convert3Dto2D(new c(s,r,this.zMin)),Math.cos(2*w)<0?(m.textAlign="center",m.textBaseline="top"):Math.sin(2*w)>0?(m.textAlign="right",m.textBaseline="middle"):(m.textAlign="left",m.textBaseline="middle"),m.fillStyle=this.axisColor,m.fillText(k,n.x,n.y));var O=this.zLabel;O.length>0&&(h=30,s=Math.cos(w)>0?this.xMin:this.xMax,r=Math.sin(w)<0?this.yMin:this.yMax,a=(this.zMin+this.zMax)/2,n=this._convert3Dto2D(new c(s,r,a)),m.textAlign="right",m.textBaseline="middle",m.fillStyle=this.axisColor,m.fillText(O,n.x-h,n.y))},o.prototype._hsv2rgb=function(t,e,i){var o,n,s,r,a,h;switch(r=i*e,a=Math.floor(t/60),h=r*(1-Math.abs(t/60%2-1)),a){case 0:o=r,n=h,s=0;break;case 1:o=h,n=r,s=0;break;case 2:o=0,n=r,s=h;break;case 3:o=0,n=h,s=r;break;case 4:o=h,n=0,s=r;break;case 5:o=r,n=0,s=h;break;default:o=0,n=0,s=0}return"RGB("+parseInt(255*o)+","+parseInt(255*n)+","+parseInt(255*s)+")"},o.prototype._redrawDataGrid=function(){var t,e,i,n,s,r,a,h,d,l,u,p,f=this.frame.canvas,m=f.getContext("2d");if(m.lineJoin="round",m.lineCap="round",!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(s=0;s0}else r=!0;r?(p=(t.point.z+e.point.z+i.point.z+n.point.z)/4,d=240*(1-(p-this.zMin)*this.scale.z/this.verticalRatio),l=1,this.showShadow?(u=Math.min(1+x.x/k/2,1),a=this._hsv2rgb(d,l,u),h=a):(u=1,a=this._hsv2rgb(d,l,u),h=this.axisColor)):(a="gray",h=this.axisColor),m.lineWidth=this._getStrokeWidth(t),m.fillStyle=a,m.strokeStyle=h,m.beginPath(),m.moveTo(t.screen.x,t.screen.y),m.lineTo(e.screen.x,e.screen.y),m.lineTo(n.screen.x,n.screen.y),m.lineTo(i.screen.x,i.screen.y),m.closePath(),m.fill(),m.stroke()}}else for(s=0;su&&(u=0);var p,f,m;this.style===o.STYLE.DOTCOLOR?(p=240*(1-(d.point.value-this.valueMin)*this.scale.value),f=this._hsv2rgb(p,1,1),m=this._hsv2rgb(p,1,.8)):this.style===o.STYLE.DOTSIZE?(f=this.dataColor.fill,m=this.dataColor.stroke):(p=240*(1-(d.point.z-this.zMin)*this.scale.z/this.verticalRatio),f=this._hsv2rgb(p,1,1),m=this._hsv2rgb(p,1,.8)),i.lineWidth=this._getStrokeWidth(d),i.strokeStyle=m,i.fillStyle=f,i.beginPath(),i.arc(d.screen.x,d.screen.y,u,0,2*Math.PI,!0),i.fill(),i.stroke()}}},o.prototype._redrawDataBar=function(){var t,e,i,n,s=this.frame.canvas,r=s.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t0){for(t=this.dataPoints[0],o.lineWidth=this._getStrokeWidth(t),o.lineJoin="round",o.lineCap="round",o.strokeStyle=this.dataColor.stroke,o.beginPath(),o.moveTo(t.screen.x,t.screen.y),e=1;e0?1:0>t?-1:0}var o=e[0],n=e[1],s=e[2],r=i((n.x-o.x)*(t.y-o.y)-(n.y-o.y)*(t.x-o.x)),a=i((s.x-n.x)*(t.y-n.y)-(s.y-n.y)*(t.x-n.x)),h=i((o.x-s.x)*(t.y-s.y)-(o.y-s.y)*(t.x-s.x));return!(0!=r&&0!=a&&r!=a||0!=a&&0!=h&&a!=h||0!=r&&0!=h&&r!=h)},o.prototype._dataPointFromXY=function(t,e){var i,n=100,s=null,r=null,a=null,h=new u(t,e);if(this.style===o.STYLE.BAR||this.style===o.STYLE.BARCOLOR||this.style===o.STYLE.BARSIZE)for(i=this.dataPoints.length-1;i>=0;i--){s=this.dataPoints[i];var d=s.surfaces;if(d)for(var l=d.length-1;l>=0;l--){var c=d[l],p=c.corners,f=[p[0].screen,p[1].screen,p[2].screen],m=[p[2].screen,p[3].screen,p[0].screen];if(this._insideTriangle(h,f)||this._insideTriangle(h,m))return s}}else for(i=0;ib)&&n>b&&(a=b,r=s)}}return r},o.prototype._showTooltip=function(t){var e,i,o;this.tooltip?(e=this.tooltip.dom.content,i=this.tooltip.dom.line,o=this.tooltip.dom.dot):(e=document.createElement("div"),e.style.position="absolute",e.style.padding="10px",e.style.border="1px solid #4d4d4d",e.style.color="#1a1a1a",e.style.background="rgba(255,255,255,0.7)",e.style.borderRadius="2px",e.style.boxShadow="5px 5px 10px rgba(128,128,128,0.5)",i=document.createElement("div"),i.style.position="absolute",i.style.height="40px",i.style.width="0",i.style.borderLeft="1px solid #4d4d4d",o=document.createElement("div"),o.style.position="absolute",o.style.height="0",o.style.width="0",o.style.border="5px solid #4d4d4d",o.style.borderRadius="5px",this.tooltip={dataPoint:null,dom:{content:e,line:i,dot:o}}),this._hideTooltip(),this.tooltip.dataPoint=t,"function"==typeof this.showTooltip?e.innerHTML=this.showTooltip(t.point):e.innerHTML="
    "+this.xLabel+":"+t.point.x+"
    "+this.yLabel+":"+t.point.y+"
    "+this.zLabel+":"+t.point.z+"
    ",e.style.left="0",e.style.top="0",this.frame.appendChild(e),this.frame.appendChild(i),this.frame.appendChild(o);var n=e.offsetWidth,s=e.offsetHeight,r=i.offsetHeight,a=o.offsetWidth,h=o.offsetHeight,d=t.screen.x-n/2;d=Math.min(Math.max(d,10),this.frame.clientWidth-10-n),i.style.left=t.screen.x+"px",i.style.top=t.screen.y-r+"px",e.style.left=d+"px",e.style.top=t.screen.y-r-s+"px",o.style.left=t.screen.x-a/2+"px",o.style.top=t.screen.y-h/2+"px"},o.prototype._hideTooltip=function(){if(this.tooltip){this.tooltip.dataPoint=null;for(var t in this.tooltip.dom)if(this.tooltip.dom.hasOwnProperty(t)){var e=this.tooltip.dom[t];e&&e.parentNode&&e.parentNode.removeChild(e)}}},t.exports=o},function(t,e){function i(t){return t?o(t):void 0}function o(t){for(var e in i.prototype)t[e]=i.prototype[e];return t}t.exports=i,i.prototype.on=i.prototype.addEventListener=function(t,e){return this._callbacks=this._callbacks||{},(this._callbacks[t]=this._callbacks[t]||[]).push(e),this},i.prototype.once=function(t,e){function i(){o.off(t,i),e.apply(this,arguments)}var o=this;return this._callbacks=this._callbacks||{},i.fn=e,this.on(t,i),this},i.prototype.off=i.prototype.removeListener=i.prototype.removeAllListeners=i.prototype.removeEventListener=function(t,e){if(this._callbacks=this._callbacks||{},0==arguments.length)return this._callbacks={},this;var i=this._callbacks[t];if(!i)return this;if(1==arguments.length)return delete this._callbacks[t],this;for(var o,n=0;no;++o)i[o].apply(this,e)}return this},i.prototype.listeners=function(t){return this._callbacks=this._callbacks||{},this._callbacks[t]||[]},i.prototype.hasListeners=function(t){return!!this.listeners(t).length}},function(t,e){function i(t,e,i){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0,this.z=void 0!==i?i:0}i.subtract=function(t,e){var o=new i;return o.x=t.x-e.x,o.y=t.y-e.y,o.z=t.z-e.z,o},i.add=function(t,e){var o=new i;return o.x=t.x+e.x,o.y=t.y+e.y,o.z=t.z+e.z,o},i.avg=function(t,e){return new i((t.x+e.x)/2,(t.y+e.y)/2,(t.z+e.z)/2)},i.crossProduct=function(t,e){var o=new i;return o.x=t.y*e.z-t.z*e.y,o.y=t.z*e.x-t.x*e.z,o.z=t.x*e.y-t.y*e.x,o},i.prototype.length=function(){return Math.sqrt(this.x*this.x+this.y*this.y+this.z*this.z)},t.exports=i},function(t,e){function i(t,e){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0}t.exports=i},function(t,e,i){function o(){this.armLocation=new n,this.armRotation={},this.armRotation.horizontal=0,this.armRotation.vertical=0,this.armLength=1.7,this.cameraLocation=new n,this.cameraRotation=new n(.5*Math.PI,0,0),this.calculateCameraOrientation()}var n=i(14);o.prototype.setArmLocation=function(t,e,i){this.armLocation.x=t,this.armLocation.y=e,this.armLocation.z=i,this.calculateCameraOrientation()},o.prototype.setArmRotation=function(t,e){void 0!==t&&(this.armRotation.horizontal=t),void 0!==e&&(this.armRotation.vertical=e,this.armRotation.vertical<0&&(this.armRotation.vertical=0),this.armRotation.vertical>.5*Math.PI&&(this.armRotation.vertical=.5*Math.PI)),void 0===t&&void 0===e||this.calculateCameraOrientation()},o.prototype.getArmRotation=function(){var t={};return t.horizontal=this.armRotation.horizontal,t.vertical=this.armRotation.vertical,t},o.prototype.setArmLength=function(t){void 0!==t&&(this.armLength=t,this.armLength<.71&&(this.armLength=.71),this.armLength>5&&(this.armLength=5),this.calculateCameraOrientation())},o.prototype.getArmLength=function(){return this.armLength},o.prototype.getCameraLocation=function(){return this.cameraLocation},o.prototype.getCameraRotation=function(){return this.cameraRotation},o.prototype.calculateCameraOrientation=function(){this.cameraLocation.x=this.armLocation.x-this.armLength*Math.sin(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.y=this.armLocation.y-this.armLength*Math.cos(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.z=this.armLocation.z+this.armLength*Math.sin(this.armRotation.vertical),this.cameraRotation.x=Math.PI/2-this.armRotation.vertical,this.cameraRotation.y=0,this.cameraRotation.z=-this.armRotation.horizontal},t.exports=o},function(t,e,i){function o(t,e,i){this.data=t,this.column=e,this.graph=i,this.index=void 0,this.value=void 0,this.values=i.getDistinctValues(t.get(),this.column),this.values.sort(function(t,e){return t>e?1:e>t?-1:0}),this.values.length>0&&this.selectValue(0),this.dataPoints=[],this.loaded=!1,this.onLoadCallback=void 0,i.animationPreload?(this.loaded=!1,this.loadInBackground()):this.loaded=!0}var n=i(11);o.prototype.isLoaded=function(){return this.loaded},o.prototype.getLoadedProgress=function(){for(var t=this.values.length,e=0;this.dataPoints[e];)e++;return Math.round(e/t*100)},o.prototype.getLabel=function(){return this.graph.filterLabel},o.prototype.getColumn=function(){return this.column},o.prototype.getSelectedValue=function(){return void 0!==this.index?this.values[this.index]:void 0},o.prototype.getValues=function(){return this.values},o.prototype.getValue=function(t){if(t>=this.values.length)throw"Error: index out of range";return this.values[t]},o.prototype._getDataPoints=function(t){if(void 0===t&&(t=this.index),void 0===t)return[];var e;if(this.dataPoints[t])e=this.dataPoints[t];else{var i={};i.column=this.column,i.value=this.values[t];var o=new n(this.data,{filter:function(t){return t[i.column]==i.value}}).get();e=this.graph._getDataPoints(o),this.dataPoints[t]=e}return e},o.prototype.setOnLoadCallback=function(t){this.onLoadCallback=t},o.prototype.selectValue=function(t){if(t>=this.values.length)throw"Error: index out of range";this.index=t,this.value=this.values[t]},o.prototype.loadInBackground=function(t){void 0===t&&(t=0);var e=this.graph.frame;if(t0&&(t--,this.setIndex(t))},o.prototype.next=function(){var t=this.getIndex();t0?this.setIndex(0):this.index=void 0},o.prototype.setIndex=function(t){if(!(to&&(o=0),o>this.values.length-1&&(o=this.values.length-1),o},o.prototype.indexToLeft=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t/(this.values.length-1)*e,o=i+3;return o},o.prototype._onMouseMove=function(t){var e=t.clientX-this.startClientX,i=this.startSlideX+e,o=this.leftToIndex(i);this.setIndex(o),n.preventDefault()},o.prototype._onMouseUp=function(t){this.frame.style.cursor="auto",n.removeEventListener(document,"mousemove",this.onmousemove),n.removeEventListener(document,"mouseup",this.onmouseup),n.preventDefault()},t.exports=o},function(t,e){function i(t,e,i,o){this._start=0,this._end=0,this._step=1,this.prettyStep=!0,this.precision=5,this._current=0,this.setRange(t,e,i,o)}i.prototype.setRange=function(t,e,i,o){this._start=t?t:0,this._end=e?e:0,this.setStep(i,o)},i.prototype.setStep=function(t,e){void 0===t||0>=t||(void 0!==e&&(this.prettyStep=e),this.prettyStep===!0?this._step=i.calculatePrettyStep(t):this._step=t)},i.calculatePrettyStep=function(t){var e=function(t){return Math.log(t)/Math.LN10},i=Math.pow(10,Math.round(e(t))),o=2*Math.pow(10,Math.round(e(t/2))),n=5*Math.pow(10,Math.round(e(t/5))),s=i;return Math.abs(o-t)<=Math.abs(s-t)&&(s=o),Math.abs(n-t)<=Math.abs(s-t)&&(s=n),0>=s&&(s=1),s},i.prototype.getCurrent=function(){return parseFloat(this._current.toPrecision(this.precision))},i.prototype.getStep=function(){return this._step},i.prototype.start=function(){this._current=this._start-this._start%this._step},i.prototype.next=function(){this._current+=this._step},i.prototype.end=function(){return this._current>this._end},t.exports=i},function(t,e,i){if("undefined"!=typeof window){var o=i(21),n=window.Hammer||i(22);t.exports=o(n,{preventDefault:"mouse"})}else t.exports=function(){throw Error("hammer.js is only available in a browser, not in node.js.")}},function(t,e,i){var o,n,s;!function(i){n=[],o=i,s="function"==typeof o?o.apply(e,n):o,!(void 0!==s&&(t.exports=s))}(function(){var t=null;return function e(i,o){function n(t){return t.match(/[^ ]+/g)}function s(e){if("hammer.input"!==e.type){if(e.srcEvent._handled||(e.srcEvent._handled={}),e.srcEvent._handled[e.type])return;e.srcEvent._handled[e.type]=!0}var i=!1;e.stopPropagation=function(){i=!0};var o=e.srcEvent.stopPropagation.bind(e.srcEvent);"function"==typeof o&&(e.srcEvent.stopPropagation=function(){o(),e.stopPropagation()}),e.firstTarget=t;for(var n=t;n&&!i;){var s=n.hammer;if(s)for(var r,a=0;a0?d._handlers[t]=o:(i.off(t,s),delete d._handlers[t]))}),d},d.emit=function(e,o){t=o.target,i.emit(e,o)},d.destroy=function(){var t=i.element.hammer,e=t.indexOf(d);-1!==e&&t.splice(e,1),t.length||delete i.element.hammer,d._handlers={},i.destroy()},d}})},function(t,e,i){var o;!function(n,s,r,a){function h(t,e,i){return setTimeout(p(t,i),e)}function d(t,e,i){return Array.isArray(t)?(l(t,i[e],i),!0):!1}function l(t,e,i){var o;if(t)if(t.forEach)t.forEach(e,i);else if(t.length!==a)for(o=0;o\s*\(/gm,"{anonymous}()@"):"Unknown Stack Trace",s=n.console&&(n.console.warn||n.console.log);return s&&s.call(n.console,o,i),t.apply(this,arguments)}}function u(t,e,i){var o,n=e.prototype;o=t.prototype=Object.create(n),o.constructor=t,o._super=n,i&&ct(o,i)}function p(t,e){return function(){return t.apply(e,arguments)}}function f(t,e){return typeof t==ft?t.apply(e?e[0]||a:a,e):t}function m(t,e){return t===a?e:t}function v(t,e,i){l(w(e),function(e){t.addEventListener(e,i,!1)})}function g(t,e,i){l(w(e),function(e){t.removeEventListener(e,i,!1)})}function y(t,e){for(;t;){if(t==e)return!0;t=t.parentNode}return!1}function b(t,e){return t.indexOf(e)>-1}function w(t){return t.trim().split(/\s+/g)}function _(t,e,i){if(t.indexOf&&!i)return t.indexOf(e);for(var o=0;oi[e]}):o.sort()),o}function O(t,e){for(var i,o,n=e[0].toUpperCase()+e.slice(1),s=0;s1&&!i.firstMultiple?i.firstMultiple=N(e):1===n&&(i.firstMultiple=!1);var s=i.firstInput,r=i.firstMultiple,a=r?r.center:s.center,h=e.center=R(o);e.timeStamp=gt(),e.deltaTime=e.timeStamp-s.timeStamp,e.angle=B(a,h),e.distance=A(a,h),P(i,e),e.offsetDirection=L(e.deltaX,e.deltaY);var d=z(e.deltaTime,e.deltaX,e.deltaY);e.overallVelocityX=d.x,e.overallVelocityY=d.y,e.overallVelocity=vt(d.x)>vt(d.y)?d.x:d.y,e.scale=r?j(r.pointers,o):1,e.rotation=r?F(r.pointers,o):0,e.maxPointers=i.prevInput?e.pointers.length>i.prevInput.maxPointers?e.pointers.length:i.prevInput.maxPointers:e.pointers.length,I(i,e);var l=t.element;y(e.srcEvent.target,l)&&(l=e.srcEvent.target),e.target=l}function P(t,e){var i=e.center,o=t.offsetDelta||{},n=t.prevDelta||{},s=t.prevInput||{};e.eventType!==Et&&s.eventType!==It||(n=t.prevDelta={x:s.deltaX||0,y:s.deltaY||0},o=t.offsetDelta={x:i.x,y:i.y}),e.deltaX=n.x+(i.x-o.x),e.deltaY=n.y+(i.y-o.y)}function I(t,e){var i,o,n,s,r=t.lastInterval||e,h=e.timeStamp-r.timeStamp;if(e.eventType!=Nt&&(h>Tt||r.velocity===a)){var d=e.deltaX-r.deltaX,l=e.deltaY-r.deltaY,c=z(h,d,l);o=c.x,n=c.y,i=vt(c.x)>vt(c.y)?c.x:c.y,s=L(d,l),t.lastInterval=e}else i=r.velocity,o=r.velocityX,n=r.velocityY,s=r.direction;e.velocity=i,e.velocityX=o,e.velocityY=n,e.direction=s}function N(t){for(var e=[],i=0;in;)i+=t[n].clientX,o+=t[n].clientY,n++;return{x:mt(i/e),y:mt(o/e)}}function z(t,e,i){return{x:e/t||0,y:i/t||0}}function L(t,e){return t===e?Rt:vt(t)>=vt(e)?0>t?zt:Lt:0>e?At:Bt}function A(t,e,i){i||(i=Wt);var o=e[i[0]]-t[i[0]],n=e[i[1]]-t[i[1]];return Math.sqrt(o*o+n*n)}function B(t,e,i){i||(i=Wt);var o=e[i[0]]-t[i[0]],n=e[i[1]]-t[i[1]];return 180*Math.atan2(n,o)/Math.PI}function F(t,e){return B(e[1],e[0],Yt)+B(t[1],t[0],Yt)}function j(t,e){return A(e[0],e[1],Yt)/A(t[0],t[1],Yt)}function H(){this.evEl=Vt,this.evWin=Ut,this.allow=!0,this.pressed=!1,S.apply(this,arguments)}function W(){this.evEl=Zt,this.evWin=Kt,S.apply(this,arguments),this.store=this.manager.session.pointerEvents=[]}function Y(){this.evTarget=Qt,this.evWin=$t,this.started=!1,S.apply(this,arguments)}function G(t,e){var i=x(t.touches),o=x(t.changedTouches);return e&(It|Nt)&&(i=k(i.concat(o),"identifier",!0)),[i,o]}function V(){this.evTarget=ee,this.targetIds={},S.apply(this,arguments)}function U(t,e){var i=x(t.touches),o=this.targetIds;if(e&(Et|Pt)&&1===i.length)return o[i[0].identifier]=!0,[i,i];var n,s,r=x(t.changedTouches),a=[],h=this.target;if(s=i.filter(function(t){return y(t.target,h)}),e===Et)for(n=0;na&&(e.push(t),a=e.length-1):n&(It|Nt)&&(i=!0),0>a||(e[a]=t,this.callback(this.manager,n,{pointers:e,changedPointers:[t],pointerType:s,srcEvent:t}),i&&e.splice(a,1))}});var Jt={touchstart:Et,touchmove:Pt,touchend:It,touchcancel:Nt},Qt="touchstart",$t="touchstart touchmove touchend touchcancel";u(Y,S,{handler:function(t){var e=Jt[t.type];if(e===Et&&(this.started=!0),this.started){var i=G.call(this,t,e);e&(It|Nt)&&i[0].length-i[1].length===0&&(this.started=!1),this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Mt,srcEvent:t})}}});var te={touchstart:Et,touchmove:Pt,touchend:It,touchcancel:Nt},ee="touchstart touchmove touchend touchcancel";u(V,S,{handler:function(t){var e=te[t.type],i=U.call(this,t,e);i&&this.callback(this.manager,e,{pointers:i[0],changedPointers:i[1],pointerType:Mt,srcEvent:t})}}),u(q,S,{handler:function(t,e,i){var o=i.pointerType==Mt,n=i.pointerType==St;if(o)this.mouse.allow=!1;else if(n&&!this.mouse.allow)return;e&(It|Nt)&&(this.mouse.allow=!0),this.callback(t,e,i)},destroy:function(){this.touch.destroy(),this.mouse.destroy()}});var ie=O(pt.style,"touchAction"),oe=ie!==a,ne="compute",se="auto",re="manipulation",ae="none",he="pan-x",de="pan-y";X.prototype={set:function(t){t==ne&&(t=this.compute()),oe&&this.manager.element.style&&(this.manager.element.style[ie]=t),this.actions=t.toLowerCase().trim()},update:function(){this.set(this.manager.options.touchAction)},compute:function(){var t=[];return l(this.manager.recognizers,function(e){f(e.options.enable,[e])&&(t=t.concat(e.getTouchAction()))}),Z(t.join(" "))},preventDefaults:function(t){if(!oe){var e=t.srcEvent,i=t.offsetDirection;if(this.manager.session.prevented)return void e.preventDefault();var o=this.actions,n=b(o,ae),s=b(o,de),r=b(o,he);if(n){var a=1===t.pointers.length,h=t.distance<2,d=t.deltaTime<250;if(a&&h&&d)return}if(!r||!s)return n||s&&i&Ft||r&&i&jt?this.preventSrc(e):void 0}},preventSrc:function(t){this.manager.session.prevented=!0,t.preventDefault()}};var le=1,ce=2,ue=4,pe=8,fe=pe,me=16,ve=32;K.prototype={defaults:{},set:function(t){return ct(this.options,t),this.manager&&this.manager.touchAction.update(),this},recognizeWith:function(t){if(d(t,"recognizeWith",this))return this;var e=this.simultaneous;return t=$(t,this),e[t.id]||(e[t.id]=t,t.recognizeWith(this)),this},dropRecognizeWith:function(t){return d(t,"dropRecognizeWith",this)?this:(t=$(t,this),delete this.simultaneous[t.id],this)},requireFailure:function(t){if(d(t,"requireFailure",this))return this;var e=this.requireFail;return t=$(t,this),-1===_(e,t)&&(e.push(t),t.requireFailure(this)),this},dropRequireFailure:function(t){if(d(t,"dropRequireFailure",this))return this;t=$(t,this);var e=_(this.requireFail,t);return e>-1&&this.requireFail.splice(e,1),this},hasRequireFailures:function(){return this.requireFail.length>0},canRecognizeWith:function(t){return!!this.simultaneous[t.id]},emit:function(t){function e(e){i.manager.emit(e,t)}var i=this,o=this.state;pe>o&&e(i.options.event+J(o)),e(i.options.event),t.additionalEvent&&e(t.additionalEvent),o>=pe&&e(i.options.event+J(o))},tryEmit:function(t){return this.canEmit()?this.emit(t):void(this.state=ve)},canEmit:function(){for(var t=0;ts?zt:Lt,i=s!=this.pX,o=Math.abs(t.deltaX)):(n=0===r?Rt:0>r?At:Bt,i=r!=this.pY,o=Math.abs(t.deltaY))),t.direction=n,i&&o>e.threshold&&n&e.direction},attrTest:function(t){return tt.prototype.attrTest.call(this,t)&&(this.state&ce||!(this.state&ce)&&this.directionTest(t))},emit:function(t){this.pX=t.deltaX,this.pY=t.deltaY;var e=Q(t.direction);e&&(t.additionalEvent=this.options.event+e),this._super.emit.call(this,t)}}),u(it,tt,{defaults:{event:"pinch",threshold:0,pointers:2},getTouchAction:function(){return[ae]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.scale-1)>this.options.threshold||this.state&ce)},emit:function(t){if(1!==t.scale){var e=t.scale<1?"in":"out";t.additionalEvent=this.options.event+e}this._super.emit.call(this,t)}}),u(ot,K,{defaults:{event:"press",pointers:1,time:251,threshold:9},getTouchAction:function(){return[se]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,o=t.distancee.time;if(this._input=t,!o||!i||t.eventType&(It|Nt)&&!n)this.reset();else if(t.eventType&Et)this.reset(),this._timer=h(function(){this.state=fe,this.tryEmit()},e.time,this);else if(t.eventType&It)return fe;return ve},reset:function(){clearTimeout(this._timer)},emit:function(t){this.state===fe&&(t&&t.eventType&It?this.manager.emit(this.options.event+"up",t):(this._input.timeStamp=gt(),this.manager.emit(this.options.event,this._input)))}}),u(nt,tt,{defaults:{event:"rotate",threshold:0,pointers:2},getTouchAction:function(){return[ae]},attrTest:function(t){return this._super.attrTest.call(this,t)&&(Math.abs(t.rotation)>this.options.threshold||this.state&ce)}}),u(st,tt,{defaults:{event:"swipe",threshold:10,velocity:.3,direction:Ft|jt,pointers:1},getTouchAction:function(){return et.prototype.getTouchAction.call(this)},attrTest:function(t){var e,i=this.options.direction;return i&(Ft|jt)?e=t.overallVelocity:i&Ft?e=t.overallVelocityX:i&jt&&(e=t.overallVelocityY),this._super.attrTest.call(this,t)&&i&t.offsetDirection&&t.distance>this.options.threshold&&t.maxPointers==this.options.pointers&&vt(e)>this.options.velocity&&t.eventType&It},emit:function(t){var e=Q(t.offsetDirection);e&&this.manager.emit(this.options.event+e,t),this.manager.emit(this.options.event,t)}}),u(rt,K,{defaults:{event:"tap",pointers:1,taps:1,interval:300,time:250,threshold:9,posThreshold:10},getTouchAction:function(){return[re]},process:function(t){var e=this.options,i=t.pointers.length===e.pointers,o=t.distance=e;e++)r[String.fromCharCode(e)]={code:65+(e-97),shift:!1};for(e=65;90>=e;e++)r[String.fromCharCode(e)]={code:e,shift:!0};for(e=0;9>=e;e++)r[""+e]={code:48+e,shift:!1};for(e=1;12>=e;e++)r["F"+e]={code:111+e,shift:!1};for(e=0;9>=e;e++)r["num"+e]={code:96+e,shift:!1};r["num*"]={code:106,shift:!1},r["num+"]={code:107,shift:!1},r["num-"]={code:109,shift:!1},r["num/"]={code:111,shift:!1},r["num."]={code:110,shift:!1},r.left={code:37,shift:!1},r.up={code:38,shift:!1},r.right={code:39,shift:!1},r.down={code:40,shift:!1},r.space={code:32,shift:!1},r.enter={code:13,shift:!1},r.shift={code:16,shift:void 0},r.esc={code:27,shift:!1},r.backspace={code:8,shift:!1},r.tab={code:9,shift:!1},r.ctrl={code:17,shift:!1},r.alt={code:18,shift:!1},r["delete"]={code:46,shift:!1},r.pageup={code:33,shift:!1},r.pagedown={code:34,shift:!1},r["="]={code:187,shift:!1},r["-"]={code:189,shift:!1},r["]"]={code:221,shift:!1},r["["]={code:219,shift:!1};var a=function(t){d(t,"keydown")},h=function(t){d(t,"keyup")},d=function(t,e){if(void 0!==s[e][t.keyCode]){for(var o=s[e][t.keyCode],n=0;ne)&&(n=e),(null===s||i>s)&&(s=i)}),null!==n&&null!==s){var r=(n+s)/2,a=Math.max(this.range.end-this.range.start,1.1*(s-n)),h=e&&void 0!==e.animation?e.animation:!0;this.range.setRange(r-a/2,r+a/2,h)}}},n.prototype.fit=function(t){var e,i=t&&void 0!==t.animation?t.animation:!0,o=this.itemsData&&this.itemsData.getDataSet();1===o.length&&void 0===o.get()[0].end?(e=this.getDataRange(),this.moveTo(e.min.valueOf(),{animation:i})):(e=this.getItemRange(),this.range.setRange(e.min,e.max,i))},n.prototype.getItemRange=function(){var t=this,e=this.getDataRange(),i=null!==e.min?e.min.valueOf():null,o=null!==e.max?e.max.valueOf():null,n=null,s=null;if(null!=i&&null!=o){var r,a,h,d,c;!function(){var e=function(t){return l.convert(t.data.start,"Date").valueOf()},u=function(t){var e=void 0!=t.data.end?t.data.end:t.data.start;return l.convert(e,"Date").valueOf()};r=o-i,0>=r&&(r=10),a=r/t.props.center.width,l.forEach(t.itemSet.items,function(t){t.show(),t.repositionX();var r=e(t),h=u(t);if(this.options.rtl)var d=r-(t.getWidthRight()+10)*a,l=h+(t.getWidthLeft()+10)*a;else var d=r-(t.getWidthLeft()+10)*a,l=h+(t.getWidthRight()+10)*a;i>d&&(i=d,n=t),l>o&&(o=l,s=t)}.bind(t)),n&&s&&(h=n.getWidthLeft()+10,d=s.getWidthRight()+10,c=t.props.center.width-h-d,c>0&&(t.options.rtl?(i=e(n)-d*r/c,o=u(s)+h*r/c):(i=e(n)-h*r/c,o=u(s)+d*r/c)))}()}return{min:null!=i?new Date(i):null,max:null!=o?new Date(o):null}},n.prototype.getDataRange=function(){var t=null,e=null,i=this.itemsData&&this.itemsData.getDataSet();return i&&i.forEach(function(i){var o=l.convert(i.start,"Date").valueOf(),n=l.convert(void 0!=i.end?i.end:i.start,"Date").valueOf();(null===t||t>o)&&(t=o),(null===e||n>e)&&(e=n)}),{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},n.prototype.getEventProperties=function(t){var e=t.center?t.center.x:t.clientX,i=t.center?t.center.y:t.clientY;if(this.options.rtl)var o=l.getAbsoluteRight(this.dom.centerContainer)-e;else var o=e-l.getAbsoluteLeft(this.dom.centerContainer);var n=i-l.getAbsoluteTop(this.dom.centerContainer),s=this.itemSet.itemFromTarget(t),r=this.itemSet.groupFromTarget(t),a=g.customTimeFromTarget(t),h=this.itemSet.options.snap||null,d=this.body.util.getScale(),c=this.body.util.getStep(),u=this._toTime(o),p=h?h(u,d,c):u,f=l.getTarget(t),m=null;return null!=s?m="item":null!=a?m="custom-time":l.hasParent(f,this.timeAxis.dom.foreground)?m="axis":this.timeAxis2&&l.hasParent(f,this.timeAxis2.dom.foreground)?m="axis":l.hasParent(f,this.itemSet.dom.labelSet)?m="group-label":l.hasParent(f,this.currentTime.bar)?m="current-time":l.hasParent(f,this.dom.center)&&(m="background"),{event:t,item:s?s.id:null,group:r?r.groupId:null,what:m,pageX:t.srcEvent?t.srcEvent.pageX:t.pageX,pageY:t.srcEvent?t.srcEvent.pageY:t.pageY,x:o,y:n,time:u,snappedTime:p}},t.exports=n},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i0&&this._makeItem([]),this._makeHeader(n),this._handleObject(this.configureOptions[n],[n])),i++);this.options.showButton===!0&&!function(){var e=document.createElement("div");e.className="vis-configuration vis-config-button",e.innerHTML="generate options",e.onclick=function(){t._printOptions()},e.onmouseover=function(){e.className="vis-configuration vis-config-button hover"},e.onmouseout=function(){e.className="vis-configuration vis-config-button"},t.optionsContainer=document.createElement("div"),t.optionsContainer.className="vis-configuration vis-config-option-container",t.domElements.push(t.optionsContainer),t.domElements.push(e)}(),this._push()}},{key:"_push",value:function(){this.wrapper=document.createElement("div"),this.wrapper.className="vis-configuration-wrapper",this.container.appendChild(this.wrapper);for(var t=0;t1?o-1:0),r=1;o>r;r++)n[r-1]=e[r];return n.forEach(function(t){s.appendChild(t)}),i.domElements.push(s),{v:i.domElements.length}}();if("object"===("undefined"==typeof a?"undefined":s(a)))return a.v}return 0}},{key:"_makeHeader",value:function(t){var e=document.createElement("div");e.className="vis-configuration vis-config-header",e.innerHTML=t,this._makeItem([],e)}},{key:"_makeLabel",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=document.createElement("div");return o.className="vis-configuration vis-config-label vis-config-s"+e.length,i===!0?o.innerHTML=""+t+":":o.innerHTML=t+":",o}},{key:"_makeDropdown",value:function(t,e,i){var o=document.createElement("select");o.className="vis-configuration vis-config-select";var n=0;void 0!==e&&-1!==t.indexOf(e)&&(n=t.indexOf(e));for(var s=0;se&&n>e*c?(a.min=Math.ceil(e*c),l=a.min,d="range increased"):n>e/c&&(a.min=Math.ceil(e/c),l=a.min,d="range increased"),e*c>s&&1!==s&&(a.max=Math.ceil(e*c),l=a.max,d="range increased"),a.value=e}else a.value=o;var u=document.createElement("input");u.className="vis-configuration vis-config-rangeinput",u.value=a.value;var p=this;a.onchange=function(){u.value=this.value,p._update(Number(this.value),i)},a.oninput=function(){u.value=this.value};var f=this._makeLabel(i[i.length-1],i),m=this._makeItem(i,f,a,u);""!==d&&this.popupHistory[m]!==l&&(this.popupHistory[m]=l,this._setupPopup(d,m))}},{key:"_setupPopup",value:function(t,e){var i=this;if(this.initialized===!0&&this.allowCreation===!0&&this.popupCountervar options = "+JSON.stringify(t,null,2)+""}},{key:"getOptions",value:function(){for(var t={},e=0;es;s++)for(r=0;rp?p+1:p;var f=l/this.r,m=a.RGBToHSV(this.color.r,this.color.g,this.color.b);m.h=p,m.s=f;var v=a.HSVToRGB(m.h,m.s,m.v);v.a=this.color.a,this.color=v,this.initialColorDiv.style.backgroundColor="rgba("+this.initialColor.r+","+this.initialColor.g+","+this.initialColor.b+","+this.initialColor.a+")",this.newColorDiv.style.backgroundColor="rgba("+this.color.r+","+this.color.g+","+this.color.b+","+this.color.a+")"}}]),t}();e["default"]=h},function(t,e,i){i(20);e.onTouch=function(t,e){e.inputHandler=function(t){t.isFirst&&e(t)},t.on("hammer.input",e.inputHandler)},e.onRelease=function(t,e){return e.inputHandler=function(t){t.isFinal&&e(t)},t.on("hammer.input",e.inputHandler)},e.offTouch=function(t,e){t.off("hammer.input",e.inputHandler)},e.offRelease=e.offTouch,e.disablePreventDefaultVertically=function(t){var e="pan-y";return t.getTouchAction=function(){return[e]},t}},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=function(){function t(t,e){for(var i=0;is.distance?console.log('%cUnknown option detected: "'+e+'" in '+t.printLocation(n.path,e,"")+"Perhaps it was misplaced? Matching option found at: "+t.printLocation(s.path,s.closestMatch,""),d):n.distance<=r?console.log('%cUnknown option detected: "'+e+'". Did you mean "'+n.closestMatch+'"?'+t.printLocation(n.path,e),d):console.log('%cUnknown option detected: "'+e+'". Did you mean one of these: '+t.print(Object.keys(i))+t.printLocation(o,e),d),a=!0}},{key:"findInOptions",value:function(e,i,o){var n=arguments.length<=3||void 0===arguments[3]?!1:arguments[3],s=1e9,a="",h=[],d=e.toLowerCase(),l=void 0;for(var c in i){var u=void 0;if(void 0!==i[c].__type__&&n===!0){var p=t.findInOptions(e,i[c],r.copyAndExtendArray(o,c));s>p.distance&&(a=p.closestMatch,h=p.path,s=p.distance,l=p.indexMatch)}else-1!==c.toLowerCase().indexOf(d)&&(l=c),u=t.levenshteinDistance(e,c),s>u&&(a=c,h=r.copyArray(o),s=u)}return{closestMatch:a,path:h,distance:s,indexMatch:l}}},{key:"printLocation",value:function(t,e){for(var i=arguments.length<=2||void 0===arguments[2]?"Problem value found at: \n":arguments[2],o="\n\n"+i+"options = {\n",n=0;ns;s++)o+=" ";o+=t[n]+": {\n"}for(var r=0;ru,r=s||null===n?n:l+(n-l)*i,p=s||null===a?a:c+(a-c)*i;y=h._applyRange(r,p),d.updateHiddenDates(h.options.moment,h.body,h.options.hiddenDates),v=v||y,y&&h.body.emitter.emit("rangechange",{start:new Date(h.start),end:new Date(h.end),byUser:o}),s?v&&h.body.emitter.emit("rangechanged",{start:new Date(h.start),end:new Date(h.end),byUser:o}):h.animationTimer=setTimeout(w,20)}};return g()}var y=this._applyRange(n,a);if(d.updateHiddenDates(this.options.moment,this.body,this.options.hiddenDates),y){var b={start:new Date(this.start),end:new Date(this.end),byUser:o};this.body.emitter.emit("rangechange",b),this.body.emitter.emit("rangechanged",b)}},o.prototype._cancelAnimation=function(){this.animationTimer&&(clearTimeout(this.animationTimer),this.animationTimer=null)},o.prototype._applyRange=function(t,e){var i,o=null!=t?r.convert(t,"Date").valueOf():this.start,n=null!=e?r.convert(e,"Date").valueOf():this.end,s=null!=this.options.max?r.convert(this.options.max,"Date").valueOf():null,a=null!=this.options.min?r.convert(this.options.min,"Date").valueOf():null;if(isNaN(o)||null===o)throw new Error('Invalid start "'+t+'"');if(isNaN(n)||null===n)throw new Error('Invalid end "'+e+'"');if(o>n&&(n=o),null!==a&&a>o&&(i=a-o,o+=i,n+=i,null!=s&&n>s&&(n=s)),null!==s&&n>s&&(i=n-s,o-=i,n-=i,null!=a&&a>o&&(o=a)),null!==this.options.zoomMin){var h=parseFloat(this.options.zoomMin);0>h&&(h=0),h>n-o&&(this.end-this.start===h&&o>this.start&&nd&&(d=0),n-o>d&&(this.end-this.start===d&&othis.end?(o=this.start,n=this.end):(i=n-o-d,o+=i/2,n-=i/2))}var l=this.start!=o||this.end!=n;return o>=this.start&&o<=this.end||n>=this.start&&n<=this.end||this.start>=o&&this.start<=n||this.end>=o&&this.end<=n||this.body.emitter.emit("checkRangedItems"),this.start=o,this.end=n,l},o.prototype.getRange=function(){return{start:this.start,end:this.end}},o.prototype.conversion=function(t,e){return o.conversion(this.start,this.end,t,e)},o.conversion=function(t,e,i,o){return void 0===o&&(o=0),0!=i&&e-t!=0?{offset:t,scale:i/(e-t-o)}:{offset:0,scale:1}},o.prototype._onDragStart=function(t){this.deltaDifference=0,this.previousDelta=0,this.options.moveable&&this._isInsideRange(t)&&this.props.touch.allowDragging&&(this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.dragging=!0,this.body.dom.root&&(this.body.dom.root.style.cursor="move"))},o.prototype._onDrag=function(t){if(this.props.touch.dragging&&this.options.moveable&&this.props.touch.allowDragging){var e=this.options.direction;n(e);var i="horizontal"==e?t.deltaX:t.deltaY;i-=this.deltaDifference;var o=this.props.touch.end-this.props.touch.start,s=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end);o-=s;var r="horizontal"==e?this.body.domProps.center.width:this.body.domProps.center.height;if(this.options.rtl)var a=i/r*o;else var a=-i/r*o;var h=this.props.touch.start+a,l=this.props.touch.end+a,c=d.snapAwayFromHidden(this.body.hiddenDates,h,this.previousDelta-i,!0),u=d.snapAwayFromHidden(this.body.hiddenDates,l,this.previousDelta-i,!0);if(c!=h||u!=l)return this.deltaDifference+=i,this.props.touch.start=c,this.props.touch.end=u,void this._onDrag(t);this.previousDelta=i,this._applyRange(h,l);var p=new Date(this.start),f=new Date(this.end);this.body.emitter.emit("rangechange",{start:p,end:f,byUser:!0})}},o.prototype._onDragEnd=function(t){this.props.touch.dragging&&this.options.moveable&&this.props.touch.allowDragging&&(this.props.touch.dragging=!1,this.body.dom.root&&(this.body.dom.root.style.cursor="auto"),this.body.emitter.emit("rangechanged",{start:new Date(this.start),end:new Date(this.end),byUser:!0}))},o.prototype._onMouseWheel=function(t){if(this.options.zoomable&&this.options.moveable&&this._isInsideRange(t)&&(!this.options.zoomKey||t[this.options.zoomKey])){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),e){var i;i=0>e?1-e/5:1/(1+e/5);var o=this.getPointer({x:t.clientX,y:t.clientY},this.body.dom.center),n=this._pointerToDate(o);this.zoom(i,n,e)}t.preventDefault()}},o.prototype._onTouch=function(t){this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.allowDragging=!0,this.props.touch.center=null,this.scaleOffset=0,this.deltaDifference=0},o.prototype._onPinch=function(t){if(this.options.zoomable&&this.options.moveable){this.props.touch.allowDragging=!1,this.props.touch.center||(this.props.touch.center=this.getPointer(t.center,this.body.dom.center));var e=1/(t.scale+this.scaleOffset),i=this._pointerToDate(this.props.touch.center),o=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),n=d.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this,i),s=o-n,r=i-n+(this.props.touch.start-(i-n))*e,a=i+s+(this.props.touch.end-(i+s))*e; +this.startToFront=0>=1-e,this.endToFront=0>=e-1;var h=d.snapAwayFromHidden(this.body.hiddenDates,r,1-e,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,e-1,!0);h==r&&l==a||(this.props.touch.start=h,this.props.touch.end=l,this.scaleOffset=1-t.scale,r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0}},o.prototype._isInsideRange=function(t){var e=t.center?t.center.x:t.clientX;if(this.options.rtl)var i=e-r.getAbsoluteLeft(this.body.dom.centerContainer);else var i=r.getAbsoluteRight(this.body.dom.centerContainer)-e;var o=this.body.util.toTime(i);return o>=this.start&&o<=this.end},o.prototype._pointerToDate=function(t){var e,i=this.options.direction;if(n(i),"horizontal"==i)return this.body.util.toTime(t.x).valueOf();var o=this.body.domProps.center.height;return e=this.conversion(o),t.y/e.scale+e.offset},o.prototype.getPointer=function(t,e){return this.options.rtl?{x:r.getAbsoluteRight(e)-t.x,y:t.y-r.getAbsoluteTop(e)}:{x:t.x-r.getAbsoluteLeft(e),y:t.y-r.getAbsoluteTop(e)}},o.prototype.zoom=function(t,e,i){null==e&&(e=(this.start+this.end)/2);var o=d.getHiddenDurationBetween(this.body.hiddenDates,this.start,this.end),n=d.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this,e),s=o-n,r=e-n+(this.start-(e-n))*t,a=e+s+(this.end-(e+s))*t;this.startToFront=!(i>0),this.endToFront=!(-i>0);var h=d.snapAwayFromHidden(this.body.hiddenDates,r,i,!0),l=d.snapAwayFromHidden(this.body.hiddenDates,a,-i,!0);h==r&&l==a||(r=h,a=l),this.setRange(r,a,!1,!0),this.startToFront=!1,this.endToFront=!0},o.prototype.move=function(t){var e=this.end-this.start,i=this.start+e*t,o=this.end+e*t;this.start=i,this.end=o},o.prototype.moveTo=function(t){var e=(this.start+this.end)/2,i=e-t,o=this.start-i,n=this.end-i;this.setRange(o,n)},t.exports=o},function(t,e){function i(t,e){this.options=null,this.props=null}i.prototype.setOptions=function(t){t&&util.extend(this.options,t)},i.prototype.redraw=function(){return!1},i.prototype.destroy=function(){},i.prototype._isResized=function(){var t=this.props._previousWidth!==this.props.width||this.props._previousHeight!==this.props.height;return this.props._previousWidth=this.props.width,this.props._previousHeight=this.props.height,t},t.exports=i},function(t,e){e.convertHiddenOptions=function(t,i,o){if(o&&!Array.isArray(o))return e.convertHiddenOptions(t,i,[o]);if(i.hiddenDates=[],o&&1==Array.isArray(o)){for(var n=0;n=4*a){var u=0,p=s.clone();switch(o[h].repeat){case"daily":d.day()!=l.day()&&(u=1),d.dayOfYear(n.dayOfYear()),d.year(n.year()),d.subtract(7,"days"),l.dayOfYear(n.dayOfYear()),l.year(n.year()),l.subtract(7-u,"days"),p.add(1,"weeks");break;case"weekly":var f=l.diff(d,"days"),m=d.day();d.date(n.date()),d.month(n.month()),d.year(n.year()),l=d.clone(),d.day(m),l.day(m),l.add(f,"days"),d.subtract(1,"weeks"),l.subtract(1,"weeks"),p.add(1,"weeks");break;case"monthly":d.month()!=l.month()&&(u=1),d.month(n.month()),d.year(n.year()),d.subtract(1,"months"),l.month(n.month()),l.year(n.year()),l.subtract(1,"months"),l.add(u,"months"),p.add(1,"months");break;case"yearly":d.year()!=l.year()&&(u=1),d.year(n.year()),d.subtract(1,"years"),l.year(n.year()),l.subtract(1,"years"),l.add(u,"years"),p.add(1,"years");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",o[h].repeat)}for(;p>d;)switch(i.hiddenDates.push({start:d.valueOf(),end:l.valueOf()}),o[h].repeat){case"daily":d.add(1,"days"),l.add(1,"days");break;case"weekly":d.add(1,"weeks"),l.add(1,"weeks");break;case"monthly":d.add(1,"months"),l.add(1,"months");break;case"yearly":d.add(1,"y"),l.add(1,"y");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",o[h].repeat)}i.hiddenDates.push({start:d.valueOf(),end:l.valueOf()})}}e.removeDuplicates(i);var v=e.isHidden(i.range.start,i.hiddenDates),g=e.isHidden(i.range.end,i.hiddenDates),y=i.range.start,b=i.range.end;1==v.hidden&&(y=1==i.range.startToFront?v.startDate-1:v.endDate+1),1==g.hidden&&(b=1==i.range.endToFront?g.startDate-1:g.endDate+1),1!=v.hidden&&1!=g.hidden||i.range._applyRange(y,b)}},e.removeDuplicates=function(t){for(var e=t.hiddenDates,i=[],o=0;o=e[o].start&&e[n].end<=e[o].end?e[n].remove=!0:e[n].start>=e[o].start&&e[n].start<=e[o].end?(e[o].end=e[n].end,e[n].remove=!0):e[n].end>=e[o].start&&e[n].end<=e[o].end&&(e[o].start=e[n].start,e[n].remove=!0));for(var o=0;o=r&&a>n){o=!0;break}}if(1==o&&n=e&&i>r&&(o+=r-s)}return o},e.correctTimeForHidden=function(t,i,o,n){return n=t(n).toDate().valueOf(),n-=e.getHiddenDurationBefore(t,i,o,n)},e.getHiddenDurationBefore=function(t,e,i,o){var n=0;o=t(o).toDate().valueOf();for(var s=0;s=i.start&&a=a&&(n+=a-r)}return n},e.getAccumulatedHiddenDuration=function(t,e,i){for(var o=0,n=0,s=e.start,r=0;r=e.start&&h=i)break;o+=h-a}}return o},e.snapAwayFromHidden=function(t,i,o,n){var s=e.isHidden(i,t);return 1==s.hidden?0>o?1==n?s.startDate-(s.endDate-i)-1:s.startDate-1:1==n?s.endDate+(i-s.startDate)+1:s.endDate+1:i},e.isHidden=function(t,e){for(var i=0;i=o&&n>t)return{hidden:!0,startDate:o,endDate:n}}return{hidden:!1,startDate:o,endDate:n}}},function(t,e,i){function o(){}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(13),r=i(20),a=i(28),h=i(1),d=(i(9),i(11),i(30),i(34),i(44)),l=i(45),c=i(32),u=i(46);s(o.prototype),o.prototype._create=function(t){function e(t){i.isActive()&&i.emit("mousewheel",t)}this.dom={},this.dom.container=t,this.dom.root=document.createElement("div"),this.dom.background=document.createElement("div"),this.dom.backgroundVertical=document.createElement("div"),this.dom.backgroundHorizontal=document.createElement("div"),this.dom.centerContainer=document.createElement("div"),this.dom.leftContainer=document.createElement("div"),this.dom.rightContainer=document.createElement("div"),this.dom.center=document.createElement("div"),this.dom.left=document.createElement("div"),this.dom.right=document.createElement("div"),this.dom.top=document.createElement("div"),this.dom.bottom=document.createElement("div"),this.dom.shadowTop=document.createElement("div"),this.dom.shadowBottom=document.createElement("div"),this.dom.shadowTopLeft=document.createElement("div"),this.dom.shadowBottomLeft=document.createElement("div"),this.dom.shadowTopRight=document.createElement("div"),this.dom.shadowBottomRight=document.createElement("div"),this.dom.root.className="vis-timeline",this.dom.background.className="vis-panel vis-background",this.dom.backgroundVertical.className="vis-panel vis-background vis-vertical",this.dom.backgroundHorizontal.className="vis-panel vis-background vis-horizontal",this.dom.centerContainer.className="vis-panel vis-center",this.dom.leftContainer.className="vis-panel vis-left",this.dom.rightContainer.className="vis-panel vis-right",this.dom.top.className="vis-panel vis-top",this.dom.bottom.className="vis-panel vis-bottom",this.dom.left.className="vis-content",this.dom.center.className="vis-content",this.dom.right.className="vis-content",this.dom.shadowTop.className="vis-shadow vis-top",this.dom.shadowBottom.className="vis-shadow vis-bottom",this.dom.shadowTopLeft.className="vis-shadow vis-top",this.dom.shadowBottomLeft.className="vis-shadow vis-bottom",this.dom.shadowTopRight.className="vis-shadow vis-top",this.dom.shadowBottomRight.className="vis-shadow vis-bottom",this.dom.root.appendChild(this.dom.background),this.dom.root.appendChild(this.dom.backgroundVertical),this.dom.root.appendChild(this.dom.backgroundHorizontal),this.dom.root.appendChild(this.dom.centerContainer),this.dom.root.appendChild(this.dom.leftContainer),this.dom.root.appendChild(this.dom.rightContainer),this.dom.root.appendChild(this.dom.top),this.dom.root.appendChild(this.dom.bottom),this.dom.centerContainer.appendChild(this.dom.center),this.dom.leftContainer.appendChild(this.dom.left),this.dom.rightContainer.appendChild(this.dom.right),this.dom.centerContainer.appendChild(this.dom.shadowTop),this.dom.centerContainer.appendChild(this.dom.shadowBottom),this.dom.leftContainer.appendChild(this.dom.shadowTopLeft),this.dom.leftContainer.appendChild(this.dom.shadowBottomLeft),this.dom.rightContainer.appendChild(this.dom.shadowTopRight),this.dom.rightContainer.appendChild(this.dom.shadowBottomRight),this.on("rangechange",function(){this.initialDrawDone===!0&&this._redraw()}.bind(this)),this.on("touch",this._onTouch.bind(this)),this.on("pan",this._onDrag.bind(this));var i=this;this.on("_change",function(t){t&&1==t.queue?i._redrawTimer||(i._redrawTimer=setTimeout(function(){i._redrawTimer=null,i._redraw()},0)):i._redraw()}),this.hammer=new r(this.dom.root);var o=this.hammer.get("pinch").set({enable:!0});a.disablePreventDefaultVertically(o),this.hammer.get("pan").set({threshold:5,direction:r.DIRECTION_HORIZONTAL}),this.listeners={};var n=["tap","doubletap","press","pinch","pan","panstart","panmove","panend"];if(n.forEach(function(t){var e=function(e){i.isActive()&&i.emit(t,e)};i.hammer.on(t,e),i.listeners[t]=e}),a.onTouch(this.hammer,function(t){i.emit("touch",t)}.bind(this)),a.onRelease(this.hammer,function(t){i.emit("release",t)}.bind(this)),this.dom.root.addEventListener("mousewheel",e),this.dom.root.addEventListener("DOMMouseScroll",e),this.props={root:{},background:{},centerContainer:{},leftContainer:{},rightContainer:{},center:{},left:{},right:{},top:{},bottom:{},border:{},scrollTop:0,scrollTopMin:0},this.customTimes=[],this.touch={},this.redrawCount=0,this.initialDrawDone=!1,!t)throw new Error("No container provided");t.appendChild(this.dom.root)},o.prototype.setOptions=function(t){if(t){var e=["width","height","minHeight","maxHeight","autoResize","start","end","clickToUse","dataAttributes","hiddenDates","locale","locales","moment","rtl","throttleRedraw"];if(h.selectiveExtend(e,this.options,t),this.options.rtl){var i=this.dom.leftContainer;this.dom.leftContainer=this.dom.rightContainer,this.dom.rightContainer=i,this.dom.container.style.direction="rtl",this.dom.backgroundVertical.className="vis-panel vis-background vis-vertical-rtl"}if(this.options.orientation={item:void 0,axis:void 0},"orientation"in t&&("string"==typeof t.orientation?this.options.orientation={item:t.orientation,axis:t.orientation}:"object"===n(t.orientation)&&("item"in t.orientation&&(this.options.orientation.item=t.orientation.item),"axis"in t.orientation&&(this.options.orientation.axis=t.orientation.axis))),"both"===this.options.orientation.axis){if(!this.timeAxis2){var o=this.timeAxis2=new d(this.body);o.setOptions=function(t){var e=t?h.extend({},t):{};e.orientation="top",d.prototype.setOptions.call(o,e)},this.components.push(o)}}else if(this.timeAxis2){var s=this.components.indexOf(this.timeAxis2);-1!==s&&this.components.splice(s,1),this.timeAxis2.destroy(),this.timeAxis2=null}if("function"==typeof t.drawPoints&&(t.drawPoints={onRender:t.drawPoints}),"hiddenDates"in this.options&&c.convertHiddenOptions(this.options.moment,this.body,this.options.hiddenDates),"clickToUse"in t&&(t.clickToUse?this.activator||(this.activator=new l(this.dom.root)):this.activator&&(this.activator.destroy(),delete this.activator)),"showCustomTime"in t)throw new Error("Option `showCustomTime` is deprecated. Create a custom time bar via timeline.addCustomTime(time [, id])");this._initAutoResize()}if(this.components.forEach(function(e){return e.setOptions(t)}),"configure"in t){this.configurator||(this.configurator=this._createConfigurator()),this.configurator.setOptions(t.configure);var r=h.deepExtend({},this.options);this.components.forEach(function(t){h.deepExtend(r,t.options)}),this.configurator.setModuleOptions({global:r})}this._origRedraw?this._redraw():(this._origRedraw=this._redraw.bind(this),this._redraw=h.throttle(this._origRedraw,this.options.throttleRedraw))},o.prototype.isActive=function(){return!this.activator||this.activator.active},o.prototype.destroy=function(){this.setItems(null),this.setGroups(null),this.off(),this._stopAutoResize(),this.dom.root.parentNode&&this.dom.root.parentNode.removeChild(this.dom.root),this.dom=null,this.activator&&(this.activator.destroy(),delete this.activator);for(var t in this.listeners)this.listeners.hasOwnProperty(t)&&delete this.listeners[t];this.listeners=null,this.hammer=null,this.components.forEach(function(t){return t.destroy()}),this.body=null},o.prototype.setCustomTime=function(t,e){var i=this.customTimes.filter(function(t){return e===t.options.id});if(0===i.length)throw new Error("No custom time bar found with id "+JSON.stringify(e));i.length>0&&i[0].setCustomTime(t)},o.prototype.getCustomTime=function(t){var e=this.customTimes.filter(function(e){return e.options.id===t});if(0===e.length)throw new Error("No custom time bar found with id "+JSON.stringify(t));return e[0].getCustomTime()},o.prototype.setCustomTimeTitle=function(t,e){var i=this.customTimes.filter(function(t){return t.options.id===e});if(0===i.length)throw new Error("No custom time bar found with id "+JSON.stringify(e));return i.length>0?i[0].setCustomTitle(t):void 0},o.prototype.getEventProperties=function(t){return{event:t}},o.prototype.addCustomTime=function(t,e){var i=void 0!==t?h.convert(t,"Date").valueOf():new Date,o=this.customTimes.some(function(t){return t.options.id===e});if(o)throw new Error("A custom time with id "+JSON.stringify(e)+" already exists");var n=new u(this.body,h.extend({},this.options,{time:i,id:e}));return this.customTimes.push(n),this.components.push(n),this._redraw(),e},o.prototype.removeCustomTime=function(t){var e=this.customTimes.filter(function(e){return e.options.id===t});if(0===e.length)throw new Error("No custom time bar found with id "+JSON.stringify(t));e.forEach(function(t){this.customTimes.splice(this.customTimes.indexOf(t),1),this.components.splice(this.components.indexOf(t),1),t.destroy()}.bind(this))},o.prototype.getVisibleItems=function(){return this.itemSet&&this.itemSet.getVisibleItems()||[]},o.prototype.fit=function(t){var e=this.getDataRange();if(null!==e.min||null!==e.max){var i=e.max-e.min,o=new Date(e.min.valueOf()-.01*i),n=new Date(e.max.valueOf()+.01*i),s=t&&void 0!==t.animation?t.animation:!0;this.range.setRange(o,n,s)}},o.prototype.getDataRange=function(){throw new Error("Cannot invoke abstract method getDataRange")},o.prototype.setWindow=function(t,e,i){var o;if(1==arguments.length){var n=arguments[0];o=void 0!==n.animation?n.animation:!0,this.range.setRange(n.start,n.end,o)}else o=i&&void 0!==i.animation?i.animation:!0,this.range.setRange(t,e,o)},o.prototype.moveTo=function(t,e){var i=this.range.end-this.range.start,o=h.convert(t,"Date").valueOf(),n=o-i/2,s=o+i/2,r=e&&void 0!==e.animation?e.animation:!0;this.range.setRange(n,s,r)},o.prototype.getWindow=function(){var t=this.range.getRange();return{start:new Date(t.start),end:new Date(t.end)}},o.prototype.redraw=function(){this._redraw()},o.prototype._redraw=function(){this.redrawCount++;var t=!1,e=this.options,i=this.props,o=this.dom;if(o&&o.container&&0!=o.root.offsetWidth){c.updateHiddenDates(this.options.moment,this.body,this.options.hiddenDates),"top"==e.orientation?(h.addClassName(o.root,"vis-top"),h.removeClassName(o.root,"vis-bottom")):(h.removeClassName(o.root,"vis-top"),h.addClassName(o.root,"vis-bottom")),o.root.style.maxHeight=h.option.asSize(e.maxHeight,""),o.root.style.minHeight=h.option.asSize(e.minHeight,""),o.root.style.width=h.option.asSize(e.width,""),i.border.left=(o.centerContainer.offsetWidth-o.centerContainer.clientWidth)/2,i.border.right=i.border.left,i.border.top=(o.centerContainer.offsetHeight-o.centerContainer.clientHeight)/2,i.border.bottom=i.border.top;var n=o.root.offsetHeight-o.root.clientHeight,s=o.root.offsetWidth-o.root.clientWidth;0===o.centerContainer.clientHeight&&(i.border.left=i.border.top,i.border.right=i.border.left),0===o.root.clientHeight&&(s=n),i.center.height=o.center.offsetHeight,i.left.height=o.left.offsetHeight,i.right.height=o.right.offsetHeight,i.top.height=o.top.clientHeight||-i.border.top,i.bottom.height=o.bottom.clientHeight||-i.border.bottom;var a=Math.max(i.left.height,i.center.height,i.right.height),d=i.top.height+a+i.bottom.height+n+i.border.top+i.border.bottom;o.root.style.height=h.option.asSize(e.height,d+"px"),i.root.height=o.root.offsetHeight,i.background.height=i.root.height-n;var l=i.root.height-i.top.height-i.bottom.height-n;i.centerContainer.height=l,i.leftContainer.height=l,i.rightContainer.height=i.leftContainer.height,i.root.width=o.root.offsetWidth,i.background.width=i.root.width-s,i.left.width=o.leftContainer.clientWidth||-i.border.left,i.leftContainer.width=i.left.width,i.right.width=o.rightContainer.clientWidth||-i.border.right,i.rightContainer.width=i.right.width;var u=i.root.width-i.left.width-i.right.width-s;i.center.width=u,i.centerContainer.width=u,i.top.width=u,i.bottom.width=u,o.background.style.height=i.background.height+"px",o.backgroundVertical.style.height=i.background.height+"px",o.backgroundHorizontal.style.height=i.centerContainer.height+"px",o.centerContainer.style.height=i.centerContainer.height+"px",o.leftContainer.style.height=i.leftContainer.height+"px",o.rightContainer.style.height=i.rightContainer.height+"px",o.background.style.width=i.background.width+"px",o.backgroundVertical.style.width=i.centerContainer.width+"px",o.backgroundHorizontal.style.width=i.background.width+"px",o.centerContainer.style.width=i.center.width+"px",o.top.style.width=i.top.width+"px",o.bottom.style.width=i.bottom.width+"px",o.background.style.left="0",o.background.style.top="0",o.backgroundVertical.style.left=i.left.width+i.border.left+"px",o.backgroundVertical.style.top="0",o.backgroundHorizontal.style.left="0",o.backgroundHorizontal.style.top=i.top.height+"px",o.centerContainer.style.left=i.left.width+"px",o.centerContainer.style.top=i.top.height+"px",o.leftContainer.style.left="0",o.leftContainer.style.top=i.top.height+"px",o.rightContainer.style.left=i.left.width+i.center.width+"px",o.rightContainer.style.top=i.top.height+"px",o.top.style.left=i.left.width+"px",o.top.style.top="0",o.bottom.style.left=i.left.width+"px",o.bottom.style.top=i.top.height+i.centerContainer.height+"px",this._updateScrollTop();var p=this.props.scrollTop;"top"!=e.orientation.item&&(p+=Math.max(this.props.centerContainer.height-this.props.center.height-this.props.border.top-this.props.border.bottom,0)),o.center.style.left="0",o.center.style.top=p+"px",o.left.style.left="0",o.left.style.top=p+"px",o.right.style.left="0",o.right.style.top=p+"px";var f=0==this.props.scrollTop?"hidden":"",m=this.props.scrollTop==this.props.scrollTopMin?"hidden":"";o.shadowTop.style.visibility=f,o.shadowBottom.style.visibility=m,o.shadowTopLeft.style.visibility=f,o.shadowBottomLeft.style.visibility=m,o.shadowTopRight.style.visibility=f,o.shadowBottomRight.style.visibility=m;var v=this.props.center.height>this.props.centerContainer.height;this.hammer.get("pan").set({direction:v?r.DIRECTION_ALL:r.DIRECTION_HORIZONTAL}),this.components.forEach(function(e){t=e.redraw()||t});var g=5;if(t){if(this.redrawCount0&&(this.props.scrollTop=0),this.props.scrollTope;e++)o=this.selection[e],n=this.items[o],n&&n.unselect();for(this.selection=[],e=0,i=t.length;i>e;e++)o=t[e],n=this.items[o],n&&(this.selection.push(o),n.select())},o.prototype.getSelection=function(){return this.selection.concat([])},o.prototype.getVisibleItems=function(){var t=this.body.range.getRange();if(this.options.rtl)var e=this.body.util.toScreen(t.start),i=this.body.util.toScreen(t.end);else var i=this.body.util.toScreen(t.start),e=this.body.util.toScreen(t.end);var o=[];for(var n in this.groups)if(this.groups.hasOwnProperty(n))for(var s=this.groups[n],r=s.visibleItems,a=0;ae&&o.push(h.id):h.lefti&&o.push(h.id)}return o},o.prototype._deselect=function(t){for(var e=this.selection,i=0,o=e.length;o>i;i++)if(e[i]==t){e.splice(i,1);break}},o.prototype.redraw=function(){var t=this.options.margin,e=this.body.range,i=r.option.asSize,o=this.options,n=o.orientation.item,s=!1,a=this.dom.frame;this.props.top=this.body.domProps.top.height+this.body.domProps.border.top,this.options.rtl?this.props.right=this.body.domProps.right.width+this.body.domProps.border.right:this.props.left=this.body.domProps.left.width+this.body.domProps.border.left,a.className="vis-itemset",s=this._orderGroups()||s;var h=e.end-e.start,d=h!=this.lastVisibleInterval||this.props.width!=this.props.lastWidth;d&&(this.stackDirty=!0), +this.lastVisibleInterval=h,this.props.lastWidth=this.props.width;var l=this.stackDirty,c=this._firstGroup(),u={item:t.item,axis:t.axis},p={item:t.item,axis:t.item.vertical/2},f=0,m=t.axis+t.item.vertical;return this.groups[y].redraw(e,p,l),r.forEach(this.groups,function(t){var i=t==c?u:p,o=t.redraw(e,i,l);s=o||s,f+=t.height}),f=Math.max(f,m),this.stackDirty=!1,a.style.height=i(f),this.props.width=a.offsetWidth,this.props.height=f,this.dom.axis.style.top=i("top"==n?this.body.domProps.top.height+this.body.domProps.border.top:this.body.domProps.top.height+this.body.domProps.centerContainer.height),this.options.rtl?this.dom.axis.style.right="0":this.dom.axis.style.left="0",s=this._isResized()||s},o.prototype._firstGroup=function(){var t="top"==this.options.orientation.item?0:this.groupIds.length-1,e=this.groupIds[t],i=this.groups[e]||this.groups[g];return i||null},o.prototype._updateUngrouped=function(){var t,e,i=this.groups[g];this.groups[y];if(this.groupsData){if(i){i.hide(),delete this.groups[g];for(e in this.items)if(this.items.hasOwnProperty(e)){t=this.items[e],t.parent&&t.parent.remove(t);var o=this._getGroupId(t.data),n=this.groups[o];n&&n.add(t)||t.hide()}}}else if(!i){var s=null,r=null;i=new c(s,r,this),this.groups[g]=i;for(e in this.items)this.items.hasOwnProperty(e)&&(t=this.items[e],i.add(t));i.show()}},o.prototype.getLabelSet=function(){return this.dom.labelSet},o.prototype.setItems=function(t){var e,i=this,o=this.itemsData;if(t){if(!(t instanceof a||t instanceof h))throw new TypeError("Data must be an instance of DataSet or DataView");this.itemsData=t}else this.itemsData=null;if(o&&(r.forEach(this.itemListeners,function(t,e){o.off(e,t)}),e=o.getIds(),this._onRemove(e)),this.itemsData){var n=this.id;r.forEach(this.itemListeners,function(t,e){i.itemsData.on(e,t,n)}),e=this.itemsData.getIds(),this._onAdd(e),this._updateUngrouped()}this.body.emitter.emit("_change",{queue:!0})},o.prototype.getItems=function(){return this.itemsData},o.prototype.setGroups=function(t){var e,i=this;if(this.groupsData&&(r.forEach(this.groupListeners,function(t,e){i.groupsData.off(e,t)}),e=this.groupsData.getIds(),this.groupsData=null,this._onRemoveGroups(e)),t){if(!(t instanceof a||t instanceof h))throw new TypeError("Data must be an instance of DataSet or DataView");this.groupsData=t}else this.groupsData=null;if(this.groupsData){var o=this.id;r.forEach(this.groupListeners,function(t,e){i.groupsData.on(e,t,o)}),e=this.groupsData.getIds(),this._onAddGroups(e)}this._updateUngrouped(),this._order(),this.body.emitter.emit("_change",{queue:!0})},o.prototype.getGroups=function(){return this.groupsData},o.prototype.removeItem=function(t){var e=this.itemsData.get(t),i=this.itemsData.getDataSet();e&&this.options.onRemove(e,function(e){e&&i.remove(t)})},o.prototype._getType=function(t){return t.type||this.options.type||(t.end?"range":"box")},o.prototype._getGroupId=function(t){var e=this._getType(t);return"background"==e&&void 0==t.group?y:this.groupsData?t.group:g},o.prototype._onUpdate=function(t){var e=this;t.forEach(function(t){var i,n=e.itemsData.get(t,e.itemOptions),s=e.items[t],r=e._getType(n),a=o.types[r];if(s&&(a&&s instanceof a?e._updateItem(s,n):(i=s.selected,e._removeItem(s),s=null)),!s){if(!a)throw"rangeoverflow"==r?new TypeError('Item type "rangeoverflow" is deprecated. Use css styling instead: .vis-item.vis-range .vis-item-content {overflow: visible;}'):new TypeError('Unknown item type "'+r+'"');s=new a(n,e.conversion,e.options),s.id=t,e._addItem(s),i&&(this.selection.push(t),s.select())}}.bind(this)),this._order(),this.stackDirty=!0,this.body.emitter.emit("_change",{queue:!0})},o.prototype._onAdd=o.prototype._onUpdate,o.prototype._onRemove=function(t){var e=0,i=this;t.forEach(function(t){var o=i.items[t];o&&(e++,i._removeItem(o))}),e&&(this._order(),this.stackDirty=!0,this.body.emitter.emit("_change",{queue:!0}))},o.prototype._order=function(){r.forEach(this.groups,function(t){t.order()})},o.prototype._onUpdateGroups=function(t){this._onAddGroups(t)},o.prototype._onAddGroups=function(t){var e=this;t.forEach(function(t){var i=e.groupsData.get(t),o=e.groups[t];if(o)o.setData(i);else{if(t==g||t==y)throw new Error("Illegal group id. "+t+" is a reserved id.");var n=Object.create(e.options);r.extend(n,{height:null}),o=new c(t,i,e),e.groups[t]=o;for(var s in e.items)if(e.items.hasOwnProperty(s)){var a=e.items[s];a.data.group==t&&o.add(a)}o.order(),o.show()}}),this.body.emitter.emit("_change",{queue:!0})},o.prototype._onRemoveGroups=function(t){var e=this.groups;t.forEach(function(t){var i=e[t];i&&(i.hide(),delete e[t])}),this.markDirty(),this.body.emitter.emit("_change",{queue:!0})},o.prototype._orderGroups=function(){if(this.groupsData){var t=this.groupsData.getIds({order:this.options.groupOrder}),e=!r.equalArray(t,this.groupIds);if(e){var i=this.groups;t.forEach(function(t){i[t].hide()}),t.forEach(function(t){i[t].show()}),this.groupIds=t}return e}return!1},o.prototype._addItem=function(t){this.items[t.id]=t;var e=this._getGroupId(t.data),i=this.groups[e];i&&i.add(t)},o.prototype._updateItem=function(t,e){var i=t.data.group,o=t.data.subgroup;if(t.setData(e),i!=t.data.group||o!=t.data.subgroup){var n=this.groups[i];n&&n.remove(t);var s=this._getGroupId(t.data),r=this.groups[s];r&&r.add(t)}},o.prototype._removeItem=function(t){t.hide(),delete this.items[t.id];var e=this.selection.indexOf(t.id);-1!=e&&this.selection.splice(e,1),t.parent&&t.parent.remove(t)},o.prototype._constructByEndArray=function(t){for(var e=[],i=0;in+s)return}else{var a=e.height;if(n+a-s>o)return}}if(e&&e!=this.groupTouchParams.group){var h=this.groupsData,d=h.get(e.groupId),l=h.get(this.groupTouchParams.group.groupId);l&&d&&(this.options.groupOrderSwap(l,d,this.groupsData),this.groupsData.update(l),this.groupsData.update(d));var c=this.groupsData.getIds({order:this.options.groupOrder});if(!r.equalArray(c,this.groupTouchParams.originalOrder))for(var h=this.groupsData,u=this.groupTouchParams.originalOrder,p=this.groupTouchParams.group.groupId,f=Math.min(u.length,c.length),m=0,v=0,g=0;f>m;){for(;f>m+v&&f>m+g&&c[m+v]==u[m+g];)m++;if(m+v>=f)break;if(c[m+v]!=p)if(u[m+g]!=p){var y=c.indexOf(u[m+g]),b=h.get(c[m+v]),w=h.get(u[m+g]);this.options.groupOrderSwap(b,w,h),h.update(b),h.update(w);var _=c[m+v];c[m+v]=u[m+g],c[y]=_,m++}else g=1;else v=1}}}},o.prototype._onGroupDragEnd=function(t){if(this.options.groupEditable.order&&this.groupTouchParams.group){t.stopPropagation();var e=this,i=e.groupTouchParams.group.groupId,o=e.groupsData.getDataSet(),n=r.extend({},o.get(i));e.options.onMoveGroup(n,function(t){if(t)t[o._fieldId]=i,o.update(t);else{var n=o.getIds({order:e.options.groupOrder});if(!r.equalArray(n,e.groupTouchParams.originalOrder))for(var s=e.groupTouchParams.originalOrder,a=Math.min(s.length,n.length),h=0;a>h;){for(;a>h&&n[h]==s[h];)h++;if(h>=a)break;var d=n.indexOf(s[h]),l=o.get(n[h]),c=o.get(s[h]);e.options.groupOrderSwap(l,c,o),groupsData.update(l),groupsData.update(c);var u=n[h];n[h]=s[h],n[d]=u,h++}}}),e.body.emitter.emit("groupDragged",{groupId:i})}},o.prototype._onSelectItem=function(t){if(this.options.selectable){var e=t.srcEvent&&(t.srcEvent.ctrlKey||t.srcEvent.metaKey),i=t.srcEvent&&t.srcEvent.shiftKey;if(e||i)return void this._onMultiSelectItem(t);var o=this.getSelection(),n=this.itemFromTarget(t),s=n?[n.id]:[];this.setSelection(s);var r=this.getSelection();(r.length>0||o.length>0)&&this.body.emitter.emit("select",{items:r,event:t})}},o.prototype._onAddItem=function(t){if(this.options.selectable&&this.options.editable.add){var e=this,i=this.options.snap||null,o=this.itemFromTarget(t);if(o){var n=e.itemsData.get(o.id);this.options.onUpdate(n,function(t){t&&e.itemsData.getDataSet().update(t)})}else{if(this.options.rtl)var s=r.getAbsoluteRight(this.dom.frame),a=s-t.center.x;else var s=r.getAbsoluteLeft(this.dom.frame),a=t.center.x-s;var h=this.body.util.toTime(a),d=this.body.util.getScale(),l=this.body.util.getStep(),c={start:i?i(h,d,l):h,content:"new item"};if("range"===this.options.type){var u=this.body.util.toTime(a+this.props.width/5);c.end=i?i(u,d,l):u}c[this.itemsData._fieldId]=r.randomUUID();var p=this.groupFromTarget(t);p&&(c.group=p.groupId),c=this._cloneItemData(c),this.options.onAdd(c,function(t){t&&e.itemsData.getDataSet().add(t)})}}},o.prototype._onMultiSelectItem=function(t){if(this.options.selectable){var e=this.itemFromTarget(t);if(e){var i=this.options.multiselect?this.getSelection():[],n=t.srcEvent&&t.srcEvent.shiftKey||!1;if(n&&this.options.multiselect){var s=this.itemsData.get(e.id).group,r=void 0;this.options.multiselectPerGroup&&i.length>0&&(r=this.itemsData.get(i[0]).group),this.options.multiselectPerGroup&&void 0!=r&&r!=s||i.push(e.id);var a=o._getItemRange(this.itemsData.get(i,this.itemOptions));if(!this.options.multiselectPerGroup||r==s){i=[];for(var h in this.items)if(this.items.hasOwnProperty(h)){var d=this.items[h],l=d.data.start,c=void 0!==d.data.end?d.data.end:l;!(l>=a.min&&c<=a.max)||this.options.multiselectPerGroup&&r!=this.itemsData.get(d.id).group||d instanceof v||i.push(d.id)}}}else{var u=i.indexOf(e.id);-1==u?i.push(e.id):i.splice(u,1)}this.setSelection(i),this.body.emitter.emit("select",{items:this.getSelection(),event:t})}}},o._getItemRange=function(t){var e=null,i=null;return t.forEach(function(t){(null==i||t.starte)&&(e=t.end):(null==e||t.start>e)&&(e=t.start)}),{min:i,max:e}},o.prototype.itemFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("timeline-item"))return e["timeline-item"];e=e.parentNode}return null},o.prototype.groupFromTarget=function(t){for(var e=t.center?t.center.y:t.clientY,i=0;ia&&ea)return n}else if(0===i&&e0?t.step:1,this.autoScale=!1)},o.prototype.setAutoScale=function(t){this.autoScale=t},o.prototype.setMinimumStep=function(t){if(void 0!=t){var e=31104e6,i=2592e6,o=864e5,n=36e5,s=6e4,r=1e3,a=1;1e3*e>t&&(this.scale="year",this.step=1e3),500*e>t&&(this.scale="year",this.step=500),100*e>t&&(this.scale="year",this.step=100),50*e>t&&(this.scale="year",this.step=50),10*e>t&&(this.scale="year",this.step=10),5*e>t&&(this.scale="year",this.step=5),e>t&&(this.scale="year",this.step=1),3*i>t&&(this.scale="month",this.step=3),i>t&&(this.scale="month",this.step=1),5*o>t&&(this.scale="day",this.step=5),2*o>t&&(this.scale="day",this.step=2),o>t&&(this.scale="day",this.step=1),o/2>t&&(this.scale="weekday",this.step=1),4*n>t&&(this.scale="hour",this.step=4),n>t&&(this.scale="hour",this.step=1),15*s>t&&(this.scale="minute",this.step=15),10*s>t&&(this.scale="minute",this.step=10),5*s>t&&(this.scale="minute",this.step=5),s>t&&(this.scale="minute",this.step=1),15*r>t&&(this.scale="second",this.step=15),10*r>t&&(this.scale="second",this.step=10),5*r>t&&(this.scale="second",this.step=5),r>t&&(this.scale="second",this.step=1),200*a>t&&(this.scale="millisecond",this.step=200),100*a>t&&(this.scale="millisecond",this.step=100),50*a>t&&(this.scale="millisecond",this.step=50),10*a>t&&(this.scale="millisecond",this.step=10),5*a>t&&(this.scale="millisecond",this.step=5),a>t&&(this.scale="millisecond",this.step=1)}},o.snap=function(t,e,i){var o=n(t);if("year"==e){var s=o.year()+Math.round(o.month()/12);o.year(Math.round(s/i)*i),o.month(0),o.date(0),o.hours(0),o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("month"==e)o.date()>15?(o.date(1),o.add(1,"month")):o.date(1),o.hours(0),o.minutes(0),o.seconds(0),o.milliseconds(0);else if("day"==e){switch(i){case 5:case 2:o.hours(24*Math.round(o.hours()/24));break;default:o.hours(12*Math.round(o.hours()/12))}o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("weekday"==e){switch(i){case 5:case 2:o.hours(12*Math.round(o.hours()/12));break;default:o.hours(6*Math.round(o.hours()/6))}o.minutes(0),o.seconds(0),o.milliseconds(0)}else if("hour"==e){switch(i){case 4:o.minutes(60*Math.round(o.minutes()/60));break;default:o.minutes(30*Math.round(o.minutes()/30))}o.seconds(0),o.milliseconds(0)}else if("minute"==e){switch(i){case 15:case 10:o.minutes(5*Math.round(o.minutes()/5)),o.seconds(0);break;case 5:o.seconds(60*Math.round(o.seconds()/60));break;default:o.seconds(30*Math.round(o.seconds()/30))}o.milliseconds(0)}else if("second"==e)switch(i){case 15:case 10:o.seconds(5*Math.round(o.seconds()/5)),o.milliseconds(0);break;case 5:o.milliseconds(1e3*Math.round(o.milliseconds()/1e3));break;default:o.milliseconds(500*Math.round(o.milliseconds()/500))}else if("millisecond"==e){var r=i>5?i/2:1;o.milliseconds(Math.round(o.milliseconds()/r)*r)}return o},o.prototype.isMajor=function(){if(1==this.switchedYear)switch(this.switchedYear=!1,this.scale){case"year":case"month":case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedMonth)switch(this.switchedMonth=!1,this.scale){case"weekday":case"day":case"hour":case"minute":case"second":case"millisecond":return!0;default:return!1}else if(1==this.switchedDay)switch(this.switchedDay=!1,this.scale){case"millisecond":case"second":case"minute":case"hour":return!0;default:return!1}var t=this.moment(this.current);switch(this.scale){case"millisecond":return 0==t.milliseconds();case"second":return 0==t.seconds();case"minute":return 0==t.hours()&&0==t.minutes();case"hour":return 0==t.hours();case"weekday":case"day":return 1==t.date();case"month":return 0==t.month();case"year":return!1;default:return!1}},o.prototype.getLabelMinor=function(t){void 0==t&&(t=this.current);var e=this.format.minorLabels[this.scale];return e&&e.length>0?this.moment(t).format(e):""},o.prototype.getLabelMajor=function(t){void 0==t&&(t=this.current);var e=this.format.majorLabels[this.scale];return e&&e.length>0?this.moment(t).format(e):""},o.prototype.getClassName=function(){function t(t){return t/h%2==0?" vis-even":" vis-odd"}function e(t){return t.isSame(new Date,"day")?" vis-today":t.isSame(s().add(1,"day"),"day")?" vis-tomorrow":t.isSame(s().add(-1,"day"),"day")?" vis-yesterday":""}function i(t){return t.isSame(new Date,"week")?" vis-current-week":""}function o(t){return t.isSame(new Date,"month")?" vis-current-month":""}function n(t){return t.isSame(new Date,"year")?" vis-current-year":""}var s=this.moment,r=this.moment(this.current),a=r.locale?r.locale("en"):r.lang("en"),h=this.step;switch(this.scale){case"millisecond":return t(a.milliseconds()).trim();case"second":return t(a.seconds()).trim();case"minute":return t(a.minutes()).trim();case"hour":var d=a.hours();return 4==this.step&&(d=d+"-h"+(d+4)),"vis-h"+d+e(a)+t(a.hours());case"weekday":return"vis-"+a.format("dddd").toLowerCase()+e(a)+i(a)+t(a.date());case"day":var l=a.date(),c=a.format("MMMM").toLowerCase();return"vis-day"+l+" vis-"+c+o(a)+t(l-1);case"month":return"vis-"+a.format("MMMM").toLowerCase()+o(a)+t(a.month());case"year":var u=a.year();return"vis-year"+u+n(a)+t(u);default:return""}},t.exports=o},function(t,e,i){function o(t,e,i){this.groupId=t,this.subgroups={},this.subgroupIndex=0,this.subgroupOrderer=e&&e.subgroupOrder,this.itemSet=i,this.dom={},this.props={label:{width:0,height:0}},this.className=null,this.items={},this.visibleItems=[],this.orderedItems={byStart:[],byEnd:[]},this.checkRangedItems=!1;var o=this;this.itemSet.body.emitter.on("checkRangedItems",function(){o.checkRangedItems=!0}),this._create(),this.setData(e)}var n=i(1),s=i(37);i(38);o.prototype._create=function(){var t=document.createElement("div");this.itemSet.options.groupEditable.order?t.className="vis-label draggable":t.className="vis-label",this.dom.label=t;var e=document.createElement("div");e.className="vis-inner",t.appendChild(e),this.dom.inner=e;var i=document.createElement("div");i.className="vis-group",i["timeline-group"]=this,this.dom.foreground=i,this.dom.background=document.createElement("div"),this.dom.background.className="vis-group",this.dom.axis=document.createElement("div"),this.dom.axis.className="vis-group",this.dom.marker=document.createElement("div"),this.dom.marker.style.visibility="hidden",this.dom.marker.innerHTML="?",this.dom.background.appendChild(this.dom.marker)},o.prototype.setData=function(t){var e;if(e=this.itemSet.options&&this.itemSet.options.groupTemplate?this.itemSet.options.groupTemplate(t):t&&t.content,e instanceof Element){for(this.dom.inner.appendChild(e);this.dom.inner.firstChild;)this.dom.inner.removeChild(this.dom.inner.firstChild);this.dom.inner.appendChild(e)}else void 0!==e&&null!==e?this.dom.inner.innerHTML=e:this.dom.inner.innerHTML=this.groupId||"";this.dom.label.title=t&&t.title||"",this.dom.inner.firstChild?n.removeClassName(this.dom.inner,"vis-hidden"):n.addClassName(this.dom.inner,"vis-hidden");var i=t&&t.className||null;i!=this.className&&(this.className&&(n.removeClassName(this.dom.label,this.className),n.removeClassName(this.dom.foreground,this.className),n.removeClassName(this.dom.background,this.className),n.removeClassName(this.dom.axis,this.className)),n.addClassName(this.dom.label,i),n.addClassName(this.dom.foreground,i),n.addClassName(this.dom.background,i),n.addClassName(this.dom.axis,i),this.className=i),this.style&&(n.removeCssText(this.dom.label,this.style),this.style=null),t&&t.style&&(n.addCssText(this.dom.label,t.style),this.style=t.style)},o.prototype.getLabelWidth=function(){return this.props.label.width},o.prototype.redraw=function(t,e,i){var o=!1,r=this.dom.marker.clientHeight;if(r!=this.lastMarkerHeight&&(this.lastMarkerHeight=r,n.forEach(this.items,function(t){t.dirty=!0,t.displayed&&t.redraw()}),i=!0),this._calculateSubGroupHeights(),"function"==typeof this.itemSet.options.order){if(i){var a=this,h=!1;n.forEach(this.items,function(t){t.displayed||(t.redraw(),a.visibleItems.push(t)),t.repositionX(h)});var d=this.orderedItems.byStart.slice().sort(function(t,e){return a.itemSet.options.order(t.data,e.data)});s.stack(d,e,!0)}this.visibleItems=this._updateVisibleItems(this.orderedItems,this.visibleItems,t)}else this.visibleItems=this._updateVisibleItems(this.orderedItems,this.visibleItems,t),this.itemSet.options.stack?s.stack(this.visibleItems,e,i):s.nostack(this.visibleItems,e,this.subgroups);var l=this._calculateHeight(e),c=this.dom.foreground;this.top=c.offsetTop,this.right=c.offsetLeft,this.width=c.offsetWidth,o=n.updateProperty(this,"height",l)||o,o=n.updateProperty(this.props.label,"width",this.dom.inner.clientWidth)||o,o=n.updateProperty(this.props.label,"height",this.dom.inner.clientHeight)||o,this.dom.background.style.height=l+"px",this.dom.foreground.style.height=l+"px",this.dom.label.style.height=l+"px";for(var u=0,p=this.visibleItems.length;p>u;u++){var f=this.visibleItems[u];f.repositionY(e)}return o},o.prototype._calculateSubGroupHeights=function(){if(Object.keys(this.subgroups).length>0){var t=this;this.resetSubgroups(),n.forEach(this.visibleItems,function(e){void 0!==e.data.subgroup&&(t.subgroups[e.data.subgroup].height=Math.max(t.subgroups[e.data.subgroup].height,e.height),t.subgroups[e.data.subgroup].visible=!0)})}},o.prototype._calculateHeight=function(t){var e,i=this.visibleItems;if(i.length>0){var o=i[0].top,s=i[0].top+i[0].height;if(n.forEach(i,function(t){o=Math.min(o,t.top),s=Math.max(s,t.top+t.height)}),o>t.axis){var r=o-t.axis;s-=r,n.forEach(i,function(t){t.top-=r})}e=s+t.item.vertical/2}else e=0;return e=Math.max(e,this.props.label.height)},o.prototype.show=function(){this.dom.label.parentNode||this.itemSet.dom.labelSet.appendChild(this.dom.label),this.dom.foreground.parentNode||this.itemSet.dom.foreground.appendChild(this.dom.foreground),this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background),this.dom.axis.parentNode||this.itemSet.dom.axis.appendChild(this.dom.axis)},o.prototype.hide=function(){var t=this.dom.label;t.parentNode&&t.parentNode.removeChild(t);var e=this.dom.foreground;e.parentNode&&e.parentNode.removeChild(e);var i=this.dom.background;i.parentNode&&i.parentNode.removeChild(i);var o=this.dom.axis;o.parentNode&&o.parentNode.removeChild(o)},o.prototype.add=function(t){if(this.items[t.id]=t,t.setParent(this),void 0!==t.data.subgroup&&(void 0===this.subgroups[t.data.subgroup]&&(this.subgroups[t.data.subgroup]={height:0,visible:!1,index:this.subgroupIndex,items:[]},this.subgroupIndex++),this.subgroups[t.data.subgroup].items.push(t)),this.orderSubgroups(),-1==this.visibleItems.indexOf(t)){var e=this.itemSet.body.range;this._checkIfVisible(t,this.visibleItems,e)}},o.prototype.orderSubgroups=function(){if(void 0!==this.subgroupOrderer){var t=[];if("string"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push({subgroup:e,sortField:this.subgroups[e].items[0].data[this.subgroupOrderer]});t.sort(function(t,e){return t.sortField-e.sortField})}else if("function"==typeof this.subgroupOrderer){for(var e in this.subgroups)t.push(this.subgroups[e].items[0].data);t.sort(this.subgroupOrderer)}if(t.length>0)for(var i=0;it?-1:l>=t?0:1};if(e.length>0)for(s=0;sl}),1==this.checkRangedItems)for(this.checkRangedItems=!1,s=0;sl})}for(s=0;s=0&&(s=e[r],!n(s));r--)void 0===o[s.id]&&(o[s.id]=!0,i.push(s));for(r=t+1;rn;n++)t[n].top=null;for(n=0,s=t.length;s>n;n++){var r=t[n];if(r.stack&&null===r.top){r.top=i.axis;do{for(var a=null,h=0,d=t.length;d>h;h++){var l=t[h];if(null!==l.top&&l!==r&&l.stack&&e.collision(r,l,i.item,l.options.rtl)){a=l;break}}null!=a&&(r.top=a.top+a.height+i.item.vertical)}while(a)}}},e.nostack=function(t,e,i){var o,n,s;for(o=0,n=t.length;n>o;o++)if(void 0!==t[o].data.subgroup){s=e.axis;for(var r in i)i.hasOwnProperty(r)&&1==i[r].visible&&i[r].indexe.right&&t.top-o.vertical+ie.top:t.left-o.horizontal+ie.left&&t.top-o.vertical+ie.top}},function(t,e,i){function o(t,e,i){if(this.props={content:{width:0}},this.overflow=!1,this.options=i,t){if(void 0==t.start)throw new Error('Property "start" missing in item '+t.id);if(void 0==t.end)throw new Error('Property "end" missing in item '+t.id)}n.call(this,t,e,i)}var n=(i(20),i(39));o.prototype=new n(null,null,null),o.prototype.baseClassName="vis-item vis-range",o.prototype.isVisible=function(t){return this.data.startt.start},o.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.frame=document.createElement("div"),t.frame.className="vis-item-overflow",t.box.appendChild(t.frame),t.content=document.createElement("div"),t.content.className="vis-item-content",t.frame.appendChild(t.content),t.box["timeline-item"]=this,this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.foreground;if(!e)throw new Error("Cannot redraw item: parent has no foreground container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.box),this._updateDataAttributes(this.dom.box),this._updateStyle(this.dom.box);var i=(this.options.editable.updateTime||this.options.editable.updateGroup||this.editable===!0)&&this.editable!==!1,o=(this.data.className?" "+this.data.className:"")+(this.selected?" vis-selected":"")+(i?" vis-editable":" vis-readonly");t.box.className=this.baseClassName+o,this.overflow="hidden"!==window.getComputedStyle(t.frame).overflow,this.dom.content.style.maxWidth="none",this.props.content.width=this.dom.content.offsetWidth,this.height=this.dom.box.offsetHeight,this.dom.content.style.maxWidth="",this.dirty=!1}this._repaintDeleteButton(t.box),this._repaintDragLeft(),this._repaintDragRight()},o.prototype.show=function(){this.displayed||this.redraw()},o.prototype.hide=function(){if(this.displayed){var t=this.dom.box;t.parentNode&&t.parentNode.removeChild(t),this.displayed=!1}},o.prototype.repositionX=function(t){var e,i,o=this.parent.width,n=this.conversion.toScreen(this.data.start),s=this.conversion.toScreen(this.data.end);void 0!==t&&t!==!0||(-o>n&&(n=-o),s>2*o&&(s=2*o));var r=Math.max(s-n,1);switch(this.overflow?(this.options.rtl?this.right=n:this.left=n,this.width=r+this.props.content.width,i=this.props.content.width):(this.options.rtl?this.right=n:this.left=n,this.width=r,i=Math.min(s-n,this.props.content.width)),this.options.rtl?this.dom.box.style.right=this.right+"px":this.dom.box.style.left=this.left+"px",this.dom.box.style.width=r+"px",this.options.align){case"left":this.options.rtl?this.dom.content.style.right="0":this.dom.content.style.left="0";break;case"right":this.options.rtl?this.dom.content.style.right=Math.max(r-i,0)+"px":this.dom.content.style.left=Math.max(r-i,0)+"px";break;case"center":this.options.rtl?this.dom.content.style.right=Math.max((r-i)/2,0)+"px":this.dom.content.style.left=Math.max((r-i)/2,0)+"px";break;default:e=this.overflow?s>0?Math.max(-n,0):-i:0>n?-n:0,this.options.rtl?this.dom.content.style.right=e+"px":this.dom.content.style.left=e+"px"}},o.prototype.repositionY=function(){var t=this.options.orientation.item,e=this.dom.box;"top"==t?e.style.top=this.top+"px":e.style.top=this.parent.height-this.top-this.height+"px"},o.prototype._repaintDragLeft=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragLeft){var t=document.createElement("div");t.className="vis-drag-left",t.dragLeftItem=this,this.dom.box.appendChild(t),this.dom.dragLeft=t}else!this.selected&&this.dom.dragLeft&&(this.dom.dragLeft.parentNode&&this.dom.dragLeft.parentNode.removeChild(this.dom.dragLeft),this.dom.dragLeft=null)},o.prototype._repaintDragRight=function(){if(this.selected&&this.options.editable.updateTime&&!this.dom.dragRight){var t=document.createElement("div");t.className="vis-drag-right",t.dragRightItem=this,this.dom.box.appendChild(t),this.dom.dragRight=t}else!this.selected&&this.dom.dragRight&&(this.dom.dragRight.parentNode&&this.dom.dragRight.parentNode.removeChild(this.dom.dragRight),this.dom.dragRight=null)},t.exports=o},function(t,e,i){function o(t,e,i){this.id=null,this.parent=null,this.data=t,this.dom=null,this.conversion=e||{},this.options=i||{},this.selected=!1,this.displayed=!1,this.dirty=!0,this.top=null,this.right=null,this.left=null,this.width=null,this.height=null,this.editable=null,this.data&&this.data.hasOwnProperty("editable")&&"boolean"==typeof this.data.editable&&(this.editable=t.editable)}var n=i(20),s=i(1);o.prototype.stack=!0,o.prototype.select=function(){this.selected=!0,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.unselect=function(){this.selected=!1,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.setData=function(t){var e=void 0!=t.group&&this.data.group!=t.group;e&&this.parent.itemSet._moveToGroup(this,t.group),t.hasOwnProperty("editable")&&"boolean"==typeof t.editable&&(this.editable=t.editable),this.data=t,this.dirty=!0,this.displayed&&this.redraw()},o.prototype.setParent=function(t){this.displayed?(this.hide(),this.parent=t,this.parent&&this.show()):this.parent=t},o.prototype.isVisible=function(t){return!1},o.prototype.show=function(){return!1},o.prototype.hide=function(){return!1},o.prototype.redraw=function(){},o.prototype.repositionX=function(){},o.prototype.repositionY=function(){},o.prototype._repaintDeleteButton=function(t){var e=(this.options.editable.remove||this.data.editable===!0)&&this.data.editable!==!1;if(this.selected&&e&&!this.dom.deleteButton){var i=this,o=document.createElement("div");this.options.rtl?o.className="vis-delete-rtl":o.className="vis-delete",o.title="Delete this item",new n(o).on("tap",function(t){t.stopPropagation(),i.parent.removeFromDataSet(i)}),t.appendChild(o),this.dom.deleteButton=o}else!this.selected&&this.dom.deleteButton&&(this.dom.deleteButton.parentNode&&this.dom.deleteButton.parentNode.removeChild(this.dom.deleteButton),this.dom.deleteButton=null)},o.prototype._updateContents=function(t){var e;if(this.options.template){var i=this.parent.itemSet.itemsData.get(this.id);e=this.options.template(i)}else e=this.data.content;var o=this._contentToString(this.content)!==this._contentToString(e);if(o){if(e instanceof Element)t.innerHTML="",t.appendChild(e);else if(void 0!=e)t.innerHTML=e;else if("background"!=this.data.type||void 0!==this.data.content)throw new Error('Property "content" missing in item '+this.id);this.content=e}},o.prototype._updateTitle=function(t){null!=this.data.title?t.title=this.data.title||"":t.removeAttribute("vis-title")},o.prototype._updateDataAttributes=function(t){if(this.options.dataAttributes&&this.options.dataAttributes.length>0){var e=[];if(Array.isArray(this.options.dataAttributes))e=this.options.dataAttributes;else{if("all"!=this.options.dataAttributes)return;e=Object.keys(this.data)}for(var i=0;in;n++){var r=this.visibleItems[n];r.repositionY(e)}return o},o.prototype.show=function(){this.dom.background.parentNode||this.itemSet.dom.background.appendChild(this.dom.background)},t.exports=o},function(t,e,i){function o(t,e,i){if(this.props={dot:{width:0,height:0},line:{width:0,height:0}},this.options=i,t&&void 0==t.start)throw new Error('Property "start" missing in item '+t);n.call(this,t,e,i)}var n=i(39);i(1);o.prototype=new n(null,null,null),o.prototype.isVisible=function(t){var e=(t.end-t.start)/4;return this.data.start>t.start-e&&this.data.startt.start-e&&this.data.startt.start},o.prototype.redraw=function(){var t=this.dom;if(t||(this.dom={},t=this.dom,t.box=document.createElement("div"),t.frame=document.createElement("div"),t.frame.className="vis-item-overflow",t.box.appendChild(t.frame),t.content=document.createElement("div"),t.content.className="vis-item-content",t.frame.appendChild(t.content),this.dirty=!0),!this.parent)throw new Error("Cannot redraw item: no parent attached");if(!t.box.parentNode){var e=this.parent.dom.background;if(!e)throw new Error("Cannot redraw item: parent has no background container element");e.appendChild(t.box)}if(this.displayed=!0,this.dirty){this._updateContents(this.dom.content),this._updateTitle(this.dom.content),this._updateDataAttributes(this.dom.content),this._updateStyle(this.dom.box);var i=(this.data.className?" "+this.data.className:"")+(this.selected?" vis-selected":"");t.box.className=this.baseClassName+i,this.overflow="hidden"!==window.getComputedStyle(t.content).overflow,this.props.content.width=this.dom.content.offsetWidth,this.height=0,this.dirty=!1}},o.prototype.show=r.prototype.show,o.prototype.hide=r.prototype.hide,o.prototype.repositionX=r.prototype.repositionX,o.prototype.repositionY=function(t){var e="top"===this.options.orientation.item;this.dom.content.style.top=e?"":"0",this.dom.content.style.bottom=e?"0":"";var i;if(void 0!==this.data.subgroup){var o=this.data.subgroup,n=this.parent.subgroups,r=n[o].index;if(1==e){i=this.parent.subgroups[o].height+t.item.vertical,i+=0==r?t.axis-.5*t.item.vertical:0;var a=this.parent.top;for(var h in n)n.hasOwnProperty(h)&&1==n[h].visible&&n[h].indexr&&(a+=l)}i=this.parent.subgroups[o].height+t.item.vertical,this.dom.box.style.top=this.parent.height-d+a+"px",this.dom.box.style.bottom=""}}else this.parent instanceof s?(i=Math.max(this.parent.height,this.parent.itemSet.body.domProps.center.height,this.parent.itemSet.body.domProps.centerContainer.height),this.dom.box.style.top=e?"0":"",this.dom.box.style.bottom=e?"":"0"):(i=this.parent.height,this.dom.box.style.top=this.parent.top+"px",this.dom.box.style.bottom="");this.dom.box.style.height=i+"px"},t.exports=o},function(t,e,i){function o(t,e){this.dom={foreground:null,lines:[],majorTexts:[],minorTexts:[],redundant:{lines:[],majorTexts:[],minorTexts:[]}},this.props={range:{start:0,end:0,minimumStep:0},lineTop:0},this.defaultOptions={orientation:{axis:"bottom"},showMinorLabels:!0,showMajorLabels:!0,maxMinorChars:7,format:a.FORMAT,moment:d,timeAxis:null},this.options=s.extend({},this.defaultOptions),this.body=t,this._create(),this.setOptions(e)}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(31),a=i(35),h=i(32),d=i(2);o.prototype=new r,o.prototype.setOptions=function(t){t&&(s.selectiveExtend(["showMinorLabels","showMajorLabels","maxMinorChars","hiddenDates","timeAxis","moment","rtl"],this.options,t),s.selectiveDeepExtend(["format"],this.options,t),"orientation"in t&&("string"==typeof t.orientation?this.options.orientation.axis=t.orientation:"object"===n(t.orientation)&&"axis"in t.orientation&&(this.options.orientation.axis=t.orientation.axis)),"locale"in t&&("function"==typeof d.locale?d.locale(t.locale):d.lang(t.locale)))},o.prototype._create=function(){this.dom.foreground=document.createElement("div"),this.dom.background=document.createElement("div"),this.dom.foreground.className="vis-time-axis vis-foreground",this.dom.background.className="vis-time-axis vis-background"},o.prototype.destroy=function(){this.dom.foreground.parentNode&&this.dom.foreground.parentNode.removeChild(this.dom.foreground),this.dom.background.parentNode&&this.dom.background.parentNode.removeChild(this.dom.background),this.body=null},o.prototype.redraw=function(){var t=this.props,e=this.dom.foreground,i=this.dom.background,o="top"==this.options.orientation.axis?this.body.dom.top:this.body.dom.bottom,n=e.parentNode!==o;this._calculateCharSize();var s=this.options.showMinorLabels&&"none"!==this.options.orientation.axis,r=this.options.showMajorLabels&&"none"!==this.options.orientation.axis;t.minorLabelHeight=s?t.minorCharHeight:0,t.majorLabelHeight=r?t.majorCharHeight:0,t.height=t.minorLabelHeight+t.majorLabelHeight,t.width=e.offsetWidth,t.minorLineHeight=this.body.domProps.root.height-t.majorLabelHeight-("top"==this.options.orientation.axis?this.body.domProps.bottom.height:this.body.domProps.top.height),t.minorLineWidth=1,t.majorLineHeight=t.minorLineHeight+t.majorLabelHeight,t.majorLineWidth=1;var a=e.nextSibling,h=i.nextSibling;return e.parentNode&&e.parentNode.removeChild(e),i.parentNode&&i.parentNode.removeChild(i),e.style.height=this.props.height+"px",this._repaintLabels(),a?o.insertBefore(e,a):o.appendChild(e),h?this.body.dom.backgroundVertical.insertBefore(i,h):this.body.dom.backgroundVertical.appendChild(i),this._isResized()||n},o.prototype._repaintLabels=function(){var t=this.options.orientation.axis,e=s.convert(this.body.range.start,"Number"),i=s.convert(this.body.range.end,"Number"),o=this.body.util.toTime((this.props.minorCharWidth||10)*this.options.maxMinorChars).valueOf(),n=o-h.getHiddenDurationBefore(this.options.moment,this.body.hiddenDates,this.body.range,o);n-=this.body.util.toTime(0).valueOf();var r=new a(new Date(e),new Date(i),n,this.body.hiddenDates);r.setMoment(this.options.moment),this.options.format&&r.setFormat(this.options.format),this.options.timeAxis&&r.setScale(this.options.timeAxis),this.step=r;var d=this.dom;d.redundant.lines=d.lines,d.redundant.majorTexts=d.majorTexts,d.redundant.minorTexts=d.minorTexts,d.lines=[],d.majorTexts=[],d.minorTexts=[];var c,u,p,f,m,v,g,y,b,w,_=0,x=void 0,k=0,O=1e3;for(r.start(),u=r.getCurrent(),f=this.body.util.toScreen(u);r.hasNext()&&O>k;){k++,m=r.isMajor(),w=r.getClassName(),b=r.getLabelMinor(),c=u,p=f,r.next(),u=r.getCurrent(),v=r.isMajor(),f=this.body.util.toScreen(u),g=_,_=f-p;var M=_>=.4*g;if(this.options.showMinorLabels&&M){var D=this._repaintMinorText(p,b,t,w);D.style.width=_+"px"}m&&this.options.showMajorLabels?(p>0&&(void 0==x&&(x=p),D=this._repaintMajorText(p,r.getLabelMajor(),t,w)),y=this._repaintMajorLine(p,_,t,w)):M?y=this._repaintMinorLine(p,_,t,w):y&&(y.style.width=parseInt(y.style.width)+_+"px")}if(k!==O||l||(console.warn("Something is wrong with the Timeline scale. Limited drawing of grid lines to "+O+" lines."),l=!0),this.options.showMajorLabels){var S=this.body.util.toTime(0),C=r.getLabelMajor(S),T=C.length*(this.props.majorCharWidth||10)+10;(void 0==x||x>T)&&this._repaintMajorText(0,C,t,w)}s.forEach(this.dom.redundant,function(t){for(;t.length;){var e=t.pop();e&&e.parentNode&&e.parentNode.removeChild(e)}})},o.prototype._repaintMinorText=function(t,e,i,o){var n=this.dom.redundant.minorTexts.shift();if(!n){var s=document.createTextNode("");n=document.createElement("div"),n.appendChild(s),this.dom.foreground.appendChild(n)}return this.dom.minorTexts.push(n),n.childNodes[0].nodeValue=e,n.style.top="top"==i?this.props.majorLabelHeight+"px":"0",this.options.rtl?(n.style.left="",n.style.right=t+"px"):n.style.left=t+"px",n.className="vis-text vis-minor "+o,n},o.prototype._repaintMajorText=function(t,e,i,o){var n=this.dom.redundant.majorTexts.shift();if(!n){var s=document.createTextNode(e);n=document.createElement("div"),n.appendChild(s),this.dom.foreground.appendChild(n)}return this.dom.majorTexts.push(n),n.childNodes[0].nodeValue=e,n.className="vis-text vis-major "+o,n.style.top="top"==i?"0":this.props.minorLabelHeight+"px",this.options.rtl?(n.style.left="",n.style.right=t+"px"):n.style.left=t+"px",n},o.prototype._repaintMinorLine=function(t,e,i,o){var n=this.dom.redundant.lines.shift();n||(n=document.createElement("div"),this.dom.background.appendChild(n)),this.dom.lines.push(n);var s=this.props;return"top"==i?n.style.top=s.majorLabelHeight+"px":n.style.top=this.body.domProps.top.height+"px",n.style.height=s.minorLineHeight+"px",this.options.rtl?(n.style.left="",n.style.right=t-s.minorLineWidth/2+"px",n.className="vis-grid vis-vertical-rtl vis-minor "+o):(n.style.left=t-s.minorLineWidth/2+"px",n.className="vis-grid vis-vertical vis-minor "+o),n.style.width=e+"px",n},o.prototype._repaintMajorLine=function(t,e,i,o){var n=this.dom.redundant.lines.shift();n||(n=document.createElement("div"),this.dom.background.appendChild(n)),this.dom.lines.push(n);var s=this.props;return"top"==i?n.style.top="0":n.style.top=this.body.domProps.top.height+"px",this.options.rtl?(n.style.left="",n.style.right=t-s.majorLineWidth/2+"px",n.className="vis-grid vis-vertical-rtl vis-major "+o):(n.style.left=t-s.majorLineWidth/2+"px",n.className="vis-grid vis-vertical vis-major "+o),n.style.height=s.majorLineHeight+"px",n.style.width=e+"px",n},o.prototype._calculateCharSize=function(){this.dom.measureCharMinor||(this.dom.measureCharMinor=document.createElement("DIV"),this.dom.measureCharMinor.className="vis-text vis-minor vis-measure",this.dom.measureCharMinor.style.position="absolute",this.dom.measureCharMinor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMinor)),this.props.minorCharHeight=this.dom.measureCharMinor.clientHeight,this.props.minorCharWidth=this.dom.measureCharMinor.clientWidth,this.dom.measureCharMajor||(this.dom.measureCharMajor=document.createElement("DIV"),this.dom.measureCharMajor.className="vis-text vis-major vis-measure",this.dom.measureCharMajor.style.position="absolute",this.dom.measureCharMajor.appendChild(document.createTextNode("0")),this.dom.foreground.appendChild(this.dom.measureCharMajor)),this.props.majorCharHeight=this.dom.measureCharMajor.clientHeight,this.props.majorCharWidth=this.dom.measureCharMajor.clientWidth};var l=!1;t.exports=o},function(t,e,i){function o(t){this.active=!1,this.dom={container:t},this.dom.overlay=document.createElement("div"),this.dom.overlay.className="vis-overlay",this.dom.container.appendChild(this.dom.overlay),this.hammer=a(this.dom.overlay),this.hammer.on("tap",this._onTapOverlay.bind(this));var e=this,i=["tap","doubletap","press","pinch","pan","panstart","panmove","panend"];i.forEach(function(t){e.hammer.on(t,function(t){t.stopPropagation()})}),document&&document.body&&(this.onClick=function(i){n(i.target,t)||e.deactivate()},document.body.addEventListener("click",this.onClick)),void 0!==this.keycharm&&this.keycharm.destroy(),this.keycharm=s(),this.escListener=this.deactivate.bind(this)}function n(t,e){for(;t;){if(t===e)return!0;t=t.parentNode}return!1}var s=i(23),r=i(13),a=i(20),h=i(1);r(o.prototype),o.current=null,o.prototype.destroy=function(){this.deactivate(),this.dom.overlay.parentNode.removeChild(this.dom.overlay),this.onClick&&document.body.removeEventListener("click",this.onClick),this.hammer.destroy(),this.hammer=null},o.prototype.activate=function(){o.current&&o.current.deactivate(),o.current=this,this.active=!0,this.dom.overlay.style.display="none",h.addClassName(this.dom.container,"vis-active"),this.emit("change"),this.emit("activate"),this.keycharm.bind("esc",this.escListener)},o.prototype.deactivate=function(){this.active=!1,this.dom.overlay.style.display="",h.removeClassName(this.dom.container,"vis-active"),this.keycharm.unbind("esc",this.escListener),this.emit("change"),this.emit("deactivate")},o.prototype._onTapOverlay=function(t){this.activate(),t.stopPropagation()},t.exports=o},function(t,e,i){function o(t,e){this.body=t,this.defaultOptions={moment:a,locales:h,locale:"en",id:void 0,title:void 0},this.options=s.extend({},this.defaultOptions),e&&e.time?this.customTime=e.time:this.customTime=new Date,this.eventParams={},this.setOptions(e),this._create()}var n=i(20),s=i(1),r=i(31),a=i(2),h=i(47);o.prototype=new r,o.prototype.setOptions=function(t){t&&s.selectiveExtend(["moment","locale","locales","id"],this.options,t)},o.prototype._create=function(){var t=document.createElement("div");t["custom-time"]=this,t.className="vis-custom-time "+(this.options.id||""),t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t;var e=document.createElement("div");e.style.position="relative",e.style.top="0px",e.style.left="-10px",e.style.height="100%",e.style.width="20px",t.appendChild(e),this.hammer=new n(e),this.hammer.on("panstart",this._onDragStart.bind(this)),this.hammer.on("panmove",this._onDrag.bind(this)),this.hammer.on("panend",this._onDragEnd.bind(this)),this.hammer.get("pan").set({threshold:5,direction:n.DIRECTION_HORIZONTAL})},o.prototype.destroy=function(){this.hide(),this.hammer.destroy(),this.hammer=null,this.body=null},o.prototype.redraw=function(){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar));var e=this.body.util.toScreen(this.customTime),i=this.options.locales[this.options.locale];i||(this.warned||(console.log("WARNING: options.locales['"+this.options.locale+"'] not found. See http://visjs.org/docs/timeline.html#Localization"),this.warned=!0),i=this.options.locales.en);var o=this.options.title;return void 0===o&&(o=i.time+": "+this.options.moment(this.customTime).format("dddd, MMMM Do YYYY, H:mm:ss"),o=o.charAt(0).toUpperCase()+o.substring(1)),this.bar.style.left=e+"px",this.bar.title=o,!1},o.prototype.hide=function(){this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar)},o.prototype.setCustomTime=function(t){this.customTime=s.convert(t,"Date"),this.redraw()},o.prototype.getCustomTime=function(){return new Date(this.customTime.valueOf())},o.prototype.setCustomTitle=function(t){this.options.title=t},o.prototype._onDragStart=function(t){this.eventParams.dragging=!0,this.eventParams.customTime=this.customTime,t.stopPropagation()},o.prototype._onDrag=function(t){if(this.eventParams.dragging){var e=this.body.util.toScreen(this.eventParams.customTime)+t.deltaX,i=this.body.util.toTime(e);this.setCustomTime(i),this.body.emitter.emit("timechange",{id:this.options.id,time:new Date(this.customTime.valueOf()) +}),t.stopPropagation()}},o.prototype._onDragEnd=function(t){this.eventParams.dragging&&(this.body.emitter.emit("timechanged",{id:this.options.id,time:new Date(this.customTime.valueOf())}),t.stopPropagation())},o.customTimeFromTarget=function(t){for(var e=t.target;e;){if(e.hasOwnProperty("custom-time"))return e["custom-time"];e=e.parentNode}return null},t.exports=o},function(t,e){e.en={current:"current",time:"time"},e.en_EN=e.en,e.en_US=e.en,e.nl={current:"huidige",time:"tijd"},e.nl_NL=e.nl,e.nl_BE=e.nl},function(t,e,i){function o(t,e){this.body=t,this.defaultOptions={rtl:!1,showCurrentTime:!0,moment:r,locales:a,locale:"en"},this.options=n.extend({},this.defaultOptions),this.offset=0,this._create(),this.setOptions(e)}var n=i(1),s=i(31),r=i(2),a=i(47);o.prototype=new s,o.prototype._create=function(){var t=document.createElement("div");t.className="vis-current-time",t.style.position="absolute",t.style.top="0px",t.style.height="100%",this.bar=t},o.prototype.destroy=function(){this.options.showCurrentTime=!1,this.redraw(),this.body=null},o.prototype.setOptions=function(t){t&&n.selectiveExtend(["rtl","showCurrentTime","moment","locale","locales"],this.options,t)},o.prototype.redraw=function(){if(this.options.showCurrentTime){var t=this.body.dom.backgroundVertical;this.bar.parentNode!=t&&(this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),t.appendChild(this.bar),this.start());var e=this.options.moment((new Date).valueOf()+this.offset),i=this.body.util.toScreen(e),o=this.options.locales[this.options.locale];o||(this.warned||(console.log("WARNING: options.locales['"+this.options.locale+"'] not found. See http://visjs.org/docs/timeline/#Localization"),this.warned=!0),o=this.options.locales.en);var n=o.current+" "+o.time+": "+e.format("dddd, MMMM Do YYYY, H:mm:ss");n=n.charAt(0).toUpperCase()+n.substring(1),this.options.rtl?this.bar.style.right=i+"px":this.bar.style.left=i+"px",this.bar.title=n}else this.bar.parentNode&&this.bar.parentNode.removeChild(this.bar),this.stop();return!1},o.prototype.start=function(){function t(){e.stop();var i=e.body.range.conversion(e.body.domProps.center.width).scale,o=1/i/10;30>o&&(o=30),o>1e3&&(o=1e3),e.redraw(),e.body.emitter.emit("currentTimeTick"),e.currentTimeTimer=setTimeout(t,o)}var e=this;t()},o.prototype.stop=function(){void 0!==this.currentTimeTimer&&(clearTimeout(this.currentTimeTimer),delete this.currentTimeTimer)},o.prototype.setCurrentTime=function(t){var e=n.convert(t,"Date").valueOf(),i=(new Date).valueOf();this.offset=e-i,this.redraw()},o.prototype.getCurrentTime=function(){return new Date((new Date).valueOf()+this.offset)},t.exports=o},function(t,e){Object.defineProperty(e,"__esModule",{value:!0});var i="string",o="boolean",n="number",s="array",r="date",a="object",h="dom",d="moment",l="any",c={configure:{enabled:{"boolean":o},filter:{"boolean":o,"function":"function"},container:{dom:h},__type__:{object:a,"boolean":o,"function":"function"}},align:{string:i},rtl:{"boolean":o,undefined:"undefined"},autoResize:{"boolean":o},throttleRedraw:{number:n},clickToUse:{"boolean":o},dataAttributes:{string:i,array:s},editable:{add:{"boolean":o,undefined:"undefined"},remove:{"boolean":o,undefined:"undefined"},updateGroup:{"boolean":o,undefined:"undefined"},updateTime:{"boolean":o,undefined:"undefined"},__type__:{"boolean":o,object:a}},end:{number:n,date:r,string:i,moment:d},format:{minorLabels:{millisecond:{string:i,undefined:"undefined"},second:{string:i,undefined:"undefined"},minute:{string:i,undefined:"undefined"},hour:{string:i,undefined:"undefined"},weekday:{string:i,undefined:"undefined"},day:{string:i,undefined:"undefined"},month:{string:i,undefined:"undefined"},year:{string:i,undefined:"undefined"},__type__:{object:a}},majorLabels:{millisecond:{string:i,undefined:"undefined"},second:{string:i,undefined:"undefined"},minute:{string:i,undefined:"undefined"},hour:{string:i,undefined:"undefined"},weekday:{string:i,undefined:"undefined"},day:{string:i,undefined:"undefined"},month:{string:i,undefined:"undefined"},year:{string:i,undefined:"undefined"},__type__:{object:a}},__type__:{object:a}},moment:{"function":"function"},groupOrder:{string:i,"function":"function"},groupEditable:{add:{"boolean":o,undefined:"undefined"},remove:{"boolean":o,undefined:"undefined"},order:{"boolean":o,undefined:"undefined"},__type__:{"boolean":o,object:a}},groupOrderSwap:{"function":"function"},height:{string:i,number:n},hiddenDates:{start:{date:r,number:n,string:i,moment:d},end:{date:r,number:n,string:i,moment:d},repeat:{string:i},__type__:{object:a,array:s}},itemsAlwaysDraggable:{"boolean":o},locale:{string:i},locales:{__any__:{any:l},__type__:{object:a}},margin:{axis:{number:n},item:{horizontal:{number:n,undefined:"undefined"},vertical:{number:n,undefined:"undefined"},__type__:{object:a,number:n}},__type__:{object:a,number:n}},max:{date:r,number:n,string:i,moment:d},maxHeight:{number:n,string:i},maxMinorChars:{number:n},min:{date:r,number:n,string:i,moment:d},minHeight:{number:n,string:i},moveable:{"boolean":o},multiselect:{"boolean":o},multiselectPerGroup:{"boolean":o},onAdd:{"function":"function"},onUpdate:{"function":"function"},onMove:{"function":"function"},onMoving:{"function":"function"},onRemove:{"function":"function"},onAddGroup:{"function":"function"},onMoveGroup:{"function":"function"},onRemoveGroup:{"function":"function"},order:{"function":"function"},orientation:{axis:{string:i,undefined:"undefined"},item:{string:i,undefined:"undefined"},__type__:{string:i,object:a}},selectable:{"boolean":o},showCurrentTime:{"boolean":o},showMajorLabels:{"boolean":o},showMinorLabels:{"boolean":o},stack:{"boolean":o},snap:{"function":"function","null":"null"},start:{date:r,number:n,string:i,moment:d},template:{"function":"function"},groupTemplate:{"function":"function"},timeAxis:{scale:{string:i,undefined:"undefined"},step:{number:n,undefined:"undefined"},__type__:{object:a}},type:{string:i},width:{string:i,number:n},zoomable:{"boolean":o},zoomKey:{string:["ctrlKey","altKey","metaKey",""]},zoomMax:{number:n},zoomMin:{number:n},__type__:{object:a}},u={global:{align:["center","left","right"],direction:!1,autoResize:!0,throttleRedraw:[10,0,1e3,10],clickToUse:!1,editable:{add:!1,remove:!1,updateGroup:!1,updateTime:!1},end:"",format:{minorLabels:{millisecond:"SSS",second:"s",minute:"HH:mm",hour:"HH:mm",weekday:"ddd D",day:"D",month:"MMM",year:"YYYY"},majorLabels:{millisecond:"HH:mm:ss",second:"D MMMM HH:mm",minute:"ddd D MMMM",hour:"ddd D MMMM",weekday:"MMMM YYYY",day:"MMMM YYYY",month:"YYYY",year:""}},groupsDraggable:!1,height:"",locale:"",margin:{axis:[20,0,100,1],item:{horizontal:[10,0,100,1],vertical:[10,0,100,1]}},max:"",maxHeight:"",maxMinorChars:[7,0,20,1],min:"",minHeight:"",moveable:!1,multiselect:!1,multiselectPerGroup:!1,orientation:{axis:["both","bottom","top"],item:["bottom","top"]},selectable:!0,showCurrentTime:!1,showMajorLabels:!0,showMinorLabels:!0,stack:!0,start:"",type:["box","point","range","background"],width:"100%",zoomable:!0,zoomKey:["ctrlKey","altKey","metaKey",""],zoomMax:[31536e10,10,31536e10,1],zoomMin:[10,10,31536e10,1]}};e.allOptions=c,e.configureOptions=u},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e,i,o){if(!(Array.isArray(i)||i instanceof c||i instanceof u)&&i instanceof Object){var n=o;o=i,i=n}var s=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:{axis:"bottom",item:"bottom"},moment:d,width:null,height:null,maxHeight:null,minHeight:null},this.options=l.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{toScreen:s._toScreen.bind(s),toGlobalScreen:s._toGlobalScreen.bind(s),toTime:s._toTime.bind(s),toGlobalTime:s._toGlobalTime.bind(s)}},this.range=new p(this.body),this.components.push(this.range),this.body.range=this.range,this.timeAxis=new m(this.body),this.components.push(this.timeAxis),this.currentTime=new v(this.body),this.components.push(this.currentTime),this.linegraph=new y(this.body),this.components.push(this.linegraph),this.itemsData=null,this.groupsData=null,this.on("tap",function(t){s.emit("click",s.getEventProperties(t))}),this.on("doubletap",function(t){s.emit("doubleClick",s.getEventProperties(t))}),this.dom.root.oncontextmenu=function(t){s.emit("contextmenu",s.getEventProperties(t))},o&&this.setOptions(o),i&&this.setGroups(i),e&&this.setItems(e),this._redraw()}var s=i(26),r=o(s),a=i(29),h=o(a),d=(i(13),i(20),i(2)),l=i(1),c=i(9),u=i(11),p=i(30),f=i(33),m=i(44),v=i(48),g=i(46),y=i(51),b=i(29).printStyle,w=i(59).allOptions,_=i(59).configureOptions;n.prototype=new f,n.prototype.setOptions=function(t){var e=h["default"].validate(t,w);e===!0&&console.log("%cErrors have been found in the supplied options object.",b),f.prototype.setOptions.call(this,t)},n.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof c||t instanceof u?t:new c(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.linegraph&&this.linegraph.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){var o=void 0!=this.options.start?this.options.start:null,n=void 0!=this.options.end?this.options.end:null;this.setWindow(o,n,{animation:!1})}else this.fit({animation:!1})},n.prototype.setGroups=function(t){var e;e=t?t instanceof c||t instanceof u?t:new c(t):null,this.groupsData=e,this.linegraph.setGroups(e)},n.prototype.getLegend=function(t,e,i){return void 0===e&&(e=15),void 0===i&&(i=15),void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].getLegend(e,i):"cannot find group:'"+t+"'"},n.prototype.isGroupVisible=function(t){return void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].visible&&(void 0===this.linegraph.options.groups.visibility[t]||1==this.linegraph.options.groups.visibility[t]):!1},n.prototype.getDataRange=function(){var t=null,e=null;for(var i in this.linegraph.groups)if(this.linegraph.groups.hasOwnProperty(i)&&1==this.linegraph.groups[i].visible)for(var o=0;os?s:t,e=null==e?s:s>e?s:e}return{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},n.prototype.getEventProperties=function(t){var e=t.center?t.center.x:t.clientX,i=t.center?t.center.y:t.clientY,o=e-l.getAbsoluteLeft(this.dom.centerContainer),n=i-l.getAbsoluteTop(this.dom.centerContainer),s=this._toTime(o),r=g.customTimeFromTarget(t),a=l.getTarget(t),h=null;l.hasParent(a,this.timeAxis.dom.foreground)?h="axis":this.timeAxis2&&l.hasParent(a,this.timeAxis2.dom.foreground)?h="axis":l.hasParent(a,this.linegraph.yAxisLeft.dom.frame)?h="data-axis":l.hasParent(a,this.linegraph.yAxisRight.dom.frame)?h="data-axis":l.hasParent(a,this.linegraph.legendLeft.dom.frame)?h="legend":l.hasParent(a,this.linegraph.legendRight.dom.frame)?h="legend":null!=r?h="custom-time":l.hasParent(a,this.currentTime.bar)?h="current-time":l.hasParent(a,this.dom.center)&&(h="background");var d=[],c=this.linegraph.yAxisLeft,u=this.linegraph.yAxisRight;return c.hidden||d.push(c.screenToValue(n)),u.hidden||d.push(u.screenToValue(n)),{event:t,what:h,pageX:t.srcEvent?t.srcEvent.pageX:t.pageX,pageY:t.srcEvent?t.srcEvent.pageY:t.pageY,x:o,y:n,time:s,value:d}},n.prototype._createConfigurator=function(){return new r["default"](this,this.dom.container,_)},t.exports=n},function(t,e,i){function o(t,e){this.id=s.randomUUID(),this.body=t,this.defaultOptions={yAxisOrientation:"left",defaultGroup:"default",sort:!0,sampling:!0,stack:!1,graphHeight:"400px",shaded:{enabled:!1,orientation:"bottom"},style:"line",barChart:{width:50,sideBySide:!1,align:"center"},interpolation:{enabled:!0,parametrization:"centripetal",alpha:.5},drawPoints:{enabled:!0,size:6,style:"square"},dataAxis:{},legend:{},groups:{visibility:{}}},this.options=s.extend({},this.defaultOptions),this.dom={},this.props={},this.hammer=null,this.groups={},this.abortedGraphUpdate=!1,this.updateSVGheight=!1,this.updateSVGheightOnResize=!1,this.forceGraphUpdate=!0;var i=this;this.itemsData=null,this.groupsData=null,this.itemListeners={add:function(t,e,o){i._onAdd(e.items)},update:function(t,e,o){i._onUpdate(e.items)},remove:function(t,e,o){i._onRemove(e.items)}},this.groupListeners={add:function(t,e,o){i._onAddGroups(e.items)},update:function(t,e,o){i._onUpdateGroups(e.items)},remove:function(t,e,o){i._onRemoveGroups(e.items)}},this.items={},this.selection=[],this.lastStart=this.body.range.start,this.touchParams={},this.svgElements={},this.setOptions(e),this.groupsUsingDefaultStyles=[0],this.body.emitter.on("rangechanged",function(){i.lastStart=i.body.range.start,i.svg.style.left=s.option.asSize(-i.props.width),i.forceGraphUpdate=!0,i.redraw.call(i)}),this._create(),this.framework={svg:this.svg,svgElements:this.svgElements,options:this.options,groups:this.groups}}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=i(8),a=i(9),h=i(11),d=i(31),l=i(52),c=i(54),u=i(58),p=i(55),f=i(57),m=i(56),v="__ungrouped__";o.prototype=new d,o.prototype._create=function(){var t=document.createElement("div");t.className="vis-line-graph",this.dom.frame=t,this.svg=document.createElementNS("http://www.w3.org/2000/svg","svg"),this.svg.style.position="relative",this.svg.style.height=(""+this.options.graphHeight).replace("px","")+"px",this.svg.style.display="block",t.appendChild(this.svg),this.options.dataAxis.orientation="left",this.yAxisLeft=new l(this.body,this.options.dataAxis,this.svg,this.options.groups),this.options.dataAxis.orientation="right",this.yAxisRight=new l(this.body,this.options.dataAxis,this.svg,this.options.groups),delete this.options.dataAxis.orientation,this.legendLeft=new u(this.body,this.options.legend,"left",this.options.groups),this.legendRight=new u(this.body,this.options.legend,"right",this.options.groups),this.show()},o.prototype.setOptions=function(t){if(t){var e=["sampling","defaultGroup","stack","height","graphHeight","yAxisOrientation","style","barChart","dataAxis","sort","groups"];void 0===t.graphHeight&&void 0!==t.height?(this.updateSVGheight=!0,this.updateSVGheightOnResize=!0):void 0!==this.body.domProps.centerContainer.height&&void 0!==t.graphHeight&&parseInt((t.graphHeight+"").replace("px",""))i?-1:1});for(var o=new Array(t.length),n=0;n0){var h={};for(this._getRelevantData(a,h,n,s),this._applySampling(a,h),e=0;e0)switch(t.options.style){case"line":l.hasOwnProperty(a[e])||(l[a[e]]=f.calcPath(h[a[e]],t)),f.draw(l[a[e]],t,this.framework);case"point":case"points":"point"!=t.options.style&&"points"!=t.options.style&&1!=t.options.drawPoints.enabled||m.draw(h[a[e]],t,this.framework);break;case"bar":}}}return r.cleanupElements(this.svgElements),!1},o.prototype._stack=function(t,e){var i,o,n,s,r;i=0;for(var a=0;at[a].x){r=e[h],s=0==h?r:e[h-1],i=h;break}}void 0===r&&(s=e[e.length-1],r=e[e.length-1]),o=r.x-s.x,n=r.y-s.y,0==o?t[a].y=t[a].orginalY+r.y:t[a].y=t[a].orginalY+n/o*(t[a].x-s.x)+s.y}},o.prototype._getRelevantData=function(t,e,i,o){var n,r,a,h;if(t.length>0)for(r=0;rt?-1:1},c=Math.max(0,s.binarySearchValue(d,i,"x","before",l)),u=Math.min(d.length,s.binarySearchValue(d,o,"x","after",l)+1);0>=u&&(u=d.length);var p=new Array(u-c);for(a=c;u>a;a++)h=n.itemsData[a],p[a-c]=h;e[t[r]]=p}else e[t[r]]=n.itemsData}},o.prototype._applySampling=function(t,e){var i;if(t.length>0)for(var o=0;o0){var s=1,r=n.length,a=this.body.util.toGlobalScreen(n[n.length-1].x)-this.body.util.toGlobalScreen(n[0].x),h=r/a;s=Math.min(Math.ceil(.2*r),Math.max(1,Math.round(h)));for(var d=new Array(r),l=0;r>l;l+=s){var c=Math.round(l/s);d[c]=n[l]}e[t[o]]=d.splice(0,Math.round(r/s))}}},o.prototype._getYRanges=function(t,e,i){var o,n,s,r,a=[],h=[];if(t.length>0){for(s=0;s0&&(n=this.groups[t[s]],r.stack===!0&&"bar"===r.style?"left"===r.yAxisOrientation?a=a.concat(n.getItems()):h=h.concat(n.getItems()):i[t[s]]=n.getYRange(o,t[s]));p.getStackedYRange(a,i,t,"__barStackLeft","left"),p.getStackedYRange(h,i,t,"__barStackRight","right")}},o.prototype._updateYAxis=function(t,e){var i,o,n=!1,s=!1,r=!1,a=1e9,h=1e9,d=-1e9,l=-1e9;if(t.length>0){for(var c=0;ci?i:a,d=o>d?o:d):(r=!0,h=h>i?i:h,l=o>l?o:l));1==s&&this.yAxisLeft.setRange(a,d),1==r&&this.yAxisRight.setRange(h,l)}n=this._toggleAxisVisiblity(s,this.yAxisLeft)||n,n=this._toggleAxisVisiblity(r,this.yAxisRight)||n,1==r&&1==s?(this.yAxisLeft.drawIcons=!0,this.yAxisRight.drawIcons=!0):(this.yAxisLeft.drawIcons=!1,this.yAxisRight.drawIcons=!1),this.yAxisRight.master=!s,this.yAxisRight.masterAxis=this.yAxisLeft,0==this.yAxisRight.master?(1==r?this.yAxisLeft.lineOffset=this.yAxisRight.width:this.yAxisLeft.lineOffset=0,n=this.yAxisLeft.redraw()||n,n=this.yAxisRight.redraw()||n):n=this.yAxisRight.redraw()||n;for(var p=["__barStackLeft","__barStackRight","__lineStackLeft","__lineStackRight"],c=0;ct?-1:1});for(var a=0;a=0&&t._redrawLabel(o-2,e.val,i,"vis-y-axis vis-major",t.props.majorCharHeight),t.master===!0&&(n?t._redrawLine(o,i,"vis-grid vis-horizontal vis-major",t.options.majorLinesOffset,t.props.majorLineWidth):t._redrawLine(o,i,"vis-grid vis-horizontal vis-minor",t.options.minorLinesOffset,t.props.minorLineWidth))});var d=0;void 0!==this.options[i].title&&void 0!==this.options[i].title.text&&(d=this.props.titleCharHeight);var l=this.options.icons===!0?Math.max(this.options.iconWidth,d)+this.options.labelOffsetX+15:d+this.options.labelOffsetX+15;return this.maxLabelSize>this.width-l&&this.options.visible===!0?(this.width=this.maxLabelSize+l,this.options.width=this.width+"px",s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),this.redraw(),e=!0):this.maxLabelSizethis.minWidth?(this.width=Math.max(this.minWidth,this.maxLabelSize+l),this.options.width=this.width+"px",s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),this.redraw(),e=!0):(s.cleanupElements(this.DOMelements.lines),s.cleanupElements(this.DOMelements.labels),e=!1),e},o.prototype.convertValue=function(t){return this.scale.convertValue(t)},o.prototype.screenToValue=function(t){return this.scale.screenToValue(t)},o.prototype._redrawLabel=function(t,e,i,o,n){var r=s.getDOMElement("div",this.DOMelements.labels,this.dom.frame);r.className=o,r.innerHTML=e,"left"===i?(r.style.left="-"+this.options.labelOffsetX+"px",r.style.textAlign="right"):(r.style.right="-"+this.options.labelOffsetX+"px",r.style.textAlign="left"),r.style.top=t-.5*n+this.options.labelOffsetY+"px",e+="";var a=Math.max(this.props.majorCharWidth,this.props.minorCharWidth);this.maxLabelSize.5*(h.magnitudefactor*h.minorSteps[h.minorStepIdx])?e+h.magnitudefactor*h.minorSteps[h.minorStepIdx]:e};i&&(this._start-=2*this.magnitudefactor*this.minorSteps[this.minorStepIdx],this._start=d(this._start)),o&&(this._end+=this.magnitudefactor*this.minorSteps[this.minorStepIdx],this._end=d(this._end)),this.determineScale()}}i.prototype.setCharHeight=function(t){this.majorCharHeight=t},i.prototype.setHeight=function(t){this.containerHeight=t},i.prototype.determineScale=function(){var t=this._end-this._start;this.scale=this.containerHeight/t;var e=this.majorCharHeight/this.scale,i=t>0?Math.round(Math.log(t)/Math.LN10):0;this.minorStepIdx=-1,this.magnitudefactor=Math.pow(10,i);var o=0;0>i&&(o=i);for(var n=!1,s=o;Math.abs(s)<=Math.abs(i);s++){this.magnitudefactor=Math.pow(10,s);for(var r=0;r=e){n=!0,this.minorStepIdx=r;break}}if(n===!0)break}},i.prototype.is_major=function(t){return t%(this.magnitudefactor*this.majorSteps[this.minorStepIdx])===0},i.prototype.getStep=function(){return this.magnitudefactor*this.minorSteps[this.minorStepIdx]},i.prototype.getFirstMajor=function(){var t=this.magnitudefactor*this.majorSteps[this.minorStepIdx];return this.convertValue(this._start+(t-this._start%t)%t)},i.prototype.formatValue=function(t){var e=t.toPrecision(5);return"function"==typeof this.formattingFunction&&(e=this.formattingFunction(t)),"number"==typeof e?""+e:"string"==typeof e?e:t.toPrecision(5)},i.prototype.getLines=function(){for(var t=[],e=this.getStep(),i=(e-this._start%e)%e,o=this._start+i;this._end-o>1e-5;o+=e)o!=this._start&&t.push({major:this.is_major(o),y:this.convertValue(o),val:this.formatValue(o)});return t},i.prototype.followScale=function(t){var e=this.minorStepIdx,i=this._start,o=this._end,n=this,s=function(){n.magnitudefactor*=2},r=function(){n.magnitudefactor/=2};t.minorStepIdx<=1&&this.minorStepIdx<=1||t.minorStepIdx>1&&this.minorStepIdx>1||(t.minorStepIdxo+1e-5)r(),d=!1;else{if(!this.autoScaleStart&&this._start=0)){r(),d=!1;continue}console.warn("Can't adhere to given 'min' range, due to zeroalign")}this.autoScaleStart&&this.autoScaleEnd&&o-i>c?(s(),d=!1):d=!0}}},i.prototype.convertValue=function(t){return this.containerHeight-(t-this._start)*this.scale},i.prototype.screenToValue=function(t){return(this.containerHeight-t)/this.scale+this._start},t.exports=i},function(t,e,i){function o(t,e,i,o){this.id=e;var n=["sampling","style","sort","yAxisOrientation","barChart","drawPoints","shaded","interpolation","zIndex","excludeFromStacking","excludeFromLegend"];this.options=s.selectiveBridgeObject(n,i),this.usingDefaultStyle=void 0===t.className,this.groupsUsingDefaultStyles=o,this.zeroPosition=0,this.update(t),1==this.usingDefaultStyle&&(this.groupsUsingDefaultStyles[0]+=1),this.itemsData=[],this.visible=void 0===t.visible?!0:t.visible}var n="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},s=i(1),r=(i(8),i(55)),a=i(57),h=i(56);o.prototype.setItems=function(t){null!=t?(this.itemsData=t,1==this.options.sort&&s.insertSort(this.itemsData,function(t,e){return t.x>e.x?1:-1})):this.itemsData=[]},o.prototype.getItems=function(){return this.itemsData},o.prototype.setZeroPosition=function(t){this.zeroPosition=t},o.prototype.setOptions=function(t){if(void 0!==t){var e=["sampling","style","sort","yAxisOrientation","barChart","zIndex","excludeFromStacking","excludeFromLegend"];s.selectiveDeepExtend(e,this.options,t),"function"==typeof t.drawPoints&&(t.drawPoints={onRender:t.drawPoints}),s.mergeOptions(this.options,t,"interpolation"),s.mergeOptions(this.options,t,"drawPoints"),s.mergeOptions(this.options,t,"shaded"),t.interpolation&&"object"==n(t.interpolation)&&t.interpolation.parametrization&&("uniform"==t.interpolation.parametrization?this.options.interpolation.alpha=0:"chordal"==t.interpolation.parametrization?this.options.interpolation.alpha=1:(this.options.interpolation.parametrization="centripetal",this.options.interpolation.alpha=.5))}},o.prototype.update=function(t){this.group=t,this.content=t.content||"graph",this.className=t.className||this.className||"vis-graph-group"+this.groupsUsingDefaultStyles[0]%10,this.visible=void 0===t.visible?!0:t.visible,this.style=t.style,this.setOptions(t.options)},o.prototype.getLegend=function(t,e,i,o,n){if(void 0==i||null==i){var s=document.createElementNS("http://www.w3.org/2000/svg","svg");i={svg:s,svgElements:{},options:this.options,groups:[this]}}switch(void 0!=o&&null!=o||(o=0),void 0!=n&&null!=n||(n=.5*e),this.options.style){case"line":a.drawIcon(this,o,n,t,e,i);break;case"points":case"point":h.drawIcon(this,o,n,t,e,i);break;case"bar":r.drawIcon(this,o,n,t,e,i)}return{icon:i.svg,label:this.content,orientation:this.options.yAxisOrientation}},o.prototype.getYRange=function(t){for(var e=t[0].y,i=t[0].y,o=0;ot[o].y?t[o].y:e,i=i0&&(i=Math.min(i,Math.abs(e[o-1].screen_x-e[o].screen_x))),0===i&&(void 0===t[e[o].screen_x]&&(t[e[o].screen_x]={amount:0,resolved:0,accumulatedPositive:0,accumulatedNegative:0}),t[e[o].screen_x].amount+=1)},o._getSafeDrawData=function(t,e,i){var o,n;return t0?(o=i>t?i:t,n=0,"left"===e.options.barChart.align?n-=.5*t:"right"===e.options.barChart.align&&(n+=.5*t)):(o=e.options.barChart.width,n=0,"left"===e.options.barChart.align?n-=.5*e.options.barChart.width:"right"===e.options.barChart.align&&(n+=.5*e.options.barChart.width)),{width:o,offset:n}},o.getStackedYRange=function(t,e,i,n,s){if(t.length>0){t.sort(function(t,e){return t.screen_x===e.screen_x?t.groupIde[s].screen_y?e[s].screen_y:o,n=nt[r].accumulatedNegative?t[r].accumulatedNegative:o,o=o>t[r].accumulatedPositive?t[r].accumulatedPositive:o,n=n0){var i=[];return i=1==e.options.interpolation.enabled?o._catmullRom(t,e):o._linear(t)}},o.drawIcon=function(t,e,i,o,s,r){var a,h,d=.5*s,l=n.getSVGElement("rect",r.svgElements,r.svg);if(l.setAttributeNS(null,"x",e),l.setAttributeNS(null,"y",i-d),l.setAttributeNS(null,"width",o),l.setAttributeNS(null,"height",2*d),l.setAttributeNS(null,"class","vis-outline"),a=n.getSVGElement("path",r.svgElements,r.svg),a.setAttributeNS(null,"class",t.className),void 0!==t.style&&a.setAttributeNS(null,"style",t.style),a.setAttributeNS(null,"d","M"+e+","+i+" L"+(e+o)+","+i),1==t.options.shaded.enabled&&(h=n.getSVGElement("path",r.svgElements,r.svg),"top"==t.options.shaded.orientation?h.setAttributeNS(null,"d","M"+e+", "+(i-d)+"L"+e+","+i+" L"+(e+o)+","+i+" L"+(e+o)+","+(i-d)):h.setAttributeNS(null,"d","M"+e+","+i+" L"+e+","+(i+d)+" L"+(e+o)+","+(i+d)+"L"+(e+o)+","+i),h.setAttributeNS(null,"class",t.className+" vis-icon-fill"),void 0!==t.options.shaded.style&&""!==t.options.shaded.style&&h.setAttributeNS(null,"style",t.options.shaded.style)),1==t.options.drawPoints.enabled){var c={style:t.options.drawPoints.style,styles:t.options.drawPoints.styles,size:t.options.drawPoints.size,className:t.className};n.drawPoint(e+.5*o,i,c,r.svgElements,r.svg)}},o.drawShading=function(t,e,i,o){if(1==e.options.shaded.enabled){var s=Number(o.svg.style.height.replace("px","")),r=n.getSVGElement("path",o.svgElements,o.svg),a="L";1==e.options.interpolation.enabled&&(a="C");var h,d=0;d="top"==e.options.shaded.orientation?0:"bottom"==e.options.shaded.orientation?s:Math.min(Math.max(0,e.zeroPosition),s),h="group"==e.options.shaded.orientation&&null!=i&&void 0!=i?"M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,a,!1)+" L"+i[i.length-1][0]+","+i[i.length-1][1]+" "+this.serializePath(i,a,!0)+i[0][0]+","+i[0][1]+" Z":"M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,a,!1)+" V"+d+" H"+t[0][0]+" Z",r.setAttributeNS(null,"class",e.className+" vis-fill"),void 0!==e.options.shaded.style&&r.setAttributeNS(null,"style",e.options.shaded.style),r.setAttributeNS(null,"d",h)}},o.draw=function(t,e,i){if(null!=t&&void 0!=t){var o=n.getSVGElement("path",i.svgElements,i.svg);o.setAttributeNS(null,"class",e.className),void 0!==e.style&&o.setAttributeNS(null,"style",e.style);var s="L";1==e.options.interpolation.enabled&&(s="C"),o.setAttributeNS(null,"d","M"+t[0][0]+","+t[0][1]+" "+this.serializePath(t,s,!1))}},o.serializePath=function(t,e,i){if(t.length<2)return"";var o=e;if(i)for(var n=t.length-2;n>0;n--)o+=t[n][0]+","+t[n][1]+" ";else for(var n=1;nl;l++)e=0==l?t[0]:t[l-1],i=t[l],o=t[l+1],n=d>l+2?t[l+2]:o,s={screen_x:(-e.screen_x+6*i.screen_x+o.screen_x)*h,screen_y:(-e.screen_y+6*i.screen_y+o.screen_y)*h},r={screen_x:(i.screen_x+6*o.screen_x-n.screen_x)*h,screen_y:(i.screen_y+6*o.screen_y-n.screen_y)*h},a.push([s.screen_x,s.screen_y]),a.push([r.screen_x,r.screen_y]),a.push([o.screen_x,o.screen_y]);return a},o._catmullRom=function(t,e){var i=e.options.interpolation.alpha;if(0==i||void 0===i)return this._catmullRomUniform(t);var o,n,s,r,a,h,d,l,c,u,p,f,m,v,g,y,b,w,_,x=[];x.push([Math.round(t[0].screen_x),Math.round(t[0].screen_y)]);for(var k=t.length,O=0;k-1>O;O++)o=0==O?t[0]:t[O-1],n=t[O],s=t[O+1],r=k>O+2?t[O+2]:s,d=Math.sqrt(Math.pow(o.screen_x-n.screen_x,2)+Math.pow(o.screen_y-n.screen_y,2)),l=Math.sqrt(Math.pow(n.screen_x-s.screen_x,2)+Math.pow(n.screen_y-s.screen_y,2)),c=Math.sqrt(Math.pow(s.screen_x-r.screen_x,2)+Math.pow(s.screen_y-r.screen_y,2)),v=Math.pow(c,i),y=Math.pow(c,2*i),g=Math.pow(l,i),b=Math.pow(l,2*i),_=Math.pow(d,i),w=Math.pow(d,2*i),u=2*w+3*_*g+b,p=2*y+3*v*g+b,f=3*_*(_+g),f>0&&(f=1/f),m=3*v*(v+g),m>0&&(m=1/m),a={screen_x:(-b*o.screen_x+u*n.screen_x+w*s.screen_x)*f,screen_y:(-b*o.screen_y+u*n.screen_y+w*s.screen_y)*f},h={screen_x:(y*n.screen_x+p*s.screen_x-b*r.screen_x)*m,screen_y:(y*n.screen_y+p*s.screen_y-b*r.screen_y)*m},0==a.screen_x&&0==a.screen_y&&(a=n),0==h.screen_x&&0==h.screen_y&&(h=s),x.push([a.screen_x,a.screen_y]),x.push([h.screen_x,h.screen_y]),x.push([s.screen_x,s.screen_y]);return x},o._linear=function(t){for(var e=[],i=0;it?-1:1});for(var i=0;i")}this.dom.textArea.innerHTML=s,this.dom.textArea.style.lineHeight=.75*this.options.iconSize+this.options.iconSpacing+"px"}},o.prototype.drawLegendIcons=function(){if(this.dom.frame.parentNode){var t=Object.keys(this.groups);t.sort(function(t,e){return e>t?-1:1}),s.resetElements(this.svgElements);var e=window.getComputedStyle(this.dom.frame).paddingTop,i=Number(e.replace("px","")),o=i,n=this.options.iconSize,r=.75*this.options.iconSize,a=i+.5*r+3;this.svg.style.width=n+5+i+"px";for(var h=0;h0){var i=this.groupIndex%this.groupsArray.length;this.groupIndex++,e={},e.color=this.groups[this.groupsArray[i]],this.groups[t]=e}else{var o=this.defaultIndex%this.defaultGroups.length;this.defaultIndex++,e={},e.color=this.defaultGroups[o],this.groups[t]=e}return e}},{key:"add",value:function(t,e){return this.groups[t]=e,this.groupsArray.push(t),e}}]),t}();e["default"]=r},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;it.left&&this.shape.topt.top}},{key:"isBoundingBoxOverlappingWith",value:function(t){return this.shape.boundingBox.leftt.left&&this.shape.boundingBox.topt.top}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=arguments.length<=3||void 0===arguments[3]?{}:arguments[3],n=["color","font","fixed","shadow"];if(A.selectiveNotDeepExtend(n,t,e,i),A.mergeOptions(t,e,"shadow",i,o),void 0!==e.color&&null!==e.color){var s=A.parseColor(e.color);A.fillIfDefined(t.color,s)}else i===!0&&null===e.color&&(t.color=A.bridgeObject(o.color));void 0!==e.fixed&&null!==e.fixed&&("boolean"==typeof e.fixed?(t.fixed.x=e.fixed,t.fixed.y=e.fixed):(void 0!==e.fixed.x&&"boolean"==typeof e.fixed.x&&(t.fixed.x=e.fixed.x),void 0!==e.fixed.y&&"boolean"==typeof e.fixed.y&&(t.fixed.y=e.fixed.y))),void 0!==e.font&&null!==e.font?a["default"].parseOptions(t.font,e):i===!0&&null===e.font&&(t.font=A.bridgeObject(o.font)),void 0!==e.scaling&&A.mergeOptions(t.scaling,e.scaling,"label",i,o.scaling)}}]),t}();e["default"]=B},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i=this.nodeOptions.scaling.label.maxVisible&&(r=Number(this.nodeOptions.scaling.label.maxVisible)/this.body.view.scale);var h=this.size.yLine,d=this._getColor(a),l=n(d,2),c=l[0],u=l[1],p=this._setAlignment(t,i,h,s),f=n(p,2);i=f[0],h=f[1],t.font=(e&&this.nodeOptions.labelHighlightBold?"bold ":"")+r+"px "+this.fontOptions.face,t.fillStyle=c,this.isEdgeLabel||"left"!==this.fontOptions.align?t.textAlign="center":(t.textAlign=this.fontOptions.align,i-=.5*this.size.width),this.fontOptions.strokeWidth>0&&(t.lineWidth=this.fontOptions.strokeWidth,t.strokeStyle=u,t.lineJoin="round");for(var m=0;m0&&t.strokeText(this.lines[m],i,h),t.fillText(this.lines[m],i,h),h+=r}},{key:"_setAlignment",value:function(t,e,i,o){if(this.isEdgeLabel&&"horizontal"!==this.fontOptions.align&&this.pointToSelf===!1){e=0,i=0;var n=2;"top"===this.fontOptions.align?(t.textBaseline="alphabetic",i-=2*n):"bottom"===this.fontOptions.align?(t.textBaseline="hanging",i+=2*n):t.textBaseline="middle"}else t.textBaseline=o;return[e,i]}},{key:"_getColor",value:function(t){var e=this.fontOptions.color||"#000000",i=this.fontOptions.strokeColor||"#ffffff";if(t<=this.nodeOptions.scaling.label.drawThreshold){var o=Math.max(0,Math.min(1,1-(this.nodeOptions.scaling.label.drawThreshold-t)));e=a.overrideOpacity(e,o),i=a.overrideOpacity(i,o)}return[e,i]}},{key:"getTextSize",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?!1:arguments[1],i={width:this._processLabel(t,e),height:this.fontOptions.size*this.lineCount,lineCount:this.lineCount};return i}},{key:"calculateLabelSize",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?0:arguments[2],o=arguments.length<=3||void 0===arguments[3]?0:arguments[3],n=arguments.length<=4||void 0===arguments[4]?"middle":arguments[4];this.labelDirty===!0&&(this.size.width=this._processLabel(t,e)),this.size.height=this.fontOptions.size*this.lineCount,this.size.left=i-.5*this.size.width,this.size.top=o-.5*this.size.height,this.size.yLine=o+.5*(1-this.lineCount)*this.fontOptions.size,"hanging"===n&&(this.size.top+=.5*this.fontOptions.size,this.size.top+=4,this.size.yLine+=4),this.labelDirty=!1}},{key:"_processLabel",value:function(t,e){var i=0,o=[""],n=0;if(void 0!==this.nodeOptions.label){o=String(this.nodeOptions.label).split("\n"),n=o.length,t.font=(e&&this.nodeOptions.labelHighlightBold?"bold ":"")+this.fontOptions.size+"px "+this.fontOptions.face,i=t.measureText(o[0]).width;for(var s=1;n>s;s++){var r=t.measureText(o[s]).width;i=r>i?r:i}}return this.lines=o,this.lineCount=n,i}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2];if("string"==typeof e.font){var o=e.font.split(" ");t.size=o[0].replace("px",""),t.face=o[1],t.color=o[2]}else"object"===s(e.font)&&a.fillIfDefined(t,e.font,i);t.size=Number(t.size)}}]),t}();e["default"]=h},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height;var n=this.options.shapeProperties.borderRadius;this.boundingBox.left=this.left-n,this.boundingBox.top=this.top-n,this.boundingBox.bottom=this.top+this.height+n,this.boundingBox.right=this.left+this.width+n}},{key:"distanceToBorder",value:function(t,e){this.resize(t);var i=this.options.borderWidth;return Math.min(Math.abs(this.width/2/Math.cos(e)),Math.abs(this.height/2/Math.sin(e)))+i}}]),e}(d["default"]);e["default"]=l},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ithis.imageObj.height?(o=this.imageObj.width/this.imageObj.height,e=2*this.options.size*o||this.imageObj.width,i=2*this.options.size||this.imageObj.height):(o=this.imageObj.width&&this.imageObj.height?this.imageObj.height/this.imageObj.width:1,e=2*this.options.size,i=2*this.options.size*o):(e=this.imageObj.width,i=this.imageObj.height),this.width=e,this.height=i,this.radius=.5*this.width}}},{key:"_drawRawCircle",value:function(t,e,i,o,n,s){var r=this.options.borderWidth,a=this.options.borderWidthSelected||2*this.options.borderWidth,h=(o?a:r)/this.body.view.scale;t.lineWidth=Math.min(this.width,h),t.strokeStyle=o?this.options.color.highlight.border:n?this.options.color.hover.border:this.options.color.border,t.fillStyle=o?this.options.color.highlight.background:n?this.options.color.hover.background:this.options.color.background,t.circle(e,i,s),this.enableShadow(t),t.fill(),this.disableShadow(t),t.save(),h>0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore()}},{key:"_drawImageAtPosition",value:function(t){if(0!=this.imageObj.width){t.globalAlpha=1,this.enableShadow(t);var e=this.imageObj.width/this.width/this.body.view.scale;if(e>2&&this.options.shapeProperties.interpolation===!0){var i=this.imageObj.width,o=this.imageObj.height,n=document.createElement("canvas");n.width=i,n.height=i;var s=n.getContext("2d");e*=.5,i*=.5,o*=.5,s.drawImage(this.imageObj,0,0,i,o);for(var r=0,a=1;e>2&&4>a;)s.drawImage(n,r,0,i,o,r+i,0,i/2,o/2),r+=i,e*=.5,i*=.5,o*=.5,a+=1;t.drawImage(n,r,0,i,o,this.left,this.top,this.width,this.height)}else t.drawImage(this.imageObj,this.left,this.top,this.width,this.height);this.disableShadow(t)}}},{key:"_drawImageLabel",value:function(t,e,i,o){var n,s=0;if(void 0!==this.height){s=.5*this.height;var r=this.labelModule.getTextSize(t);r.lineCount>=1&&(s+=r.height/2)}n=i+s,this.options.label&&(this.labelOffset=s),this.labelModule.draw(t,e,n,o,"hanging")}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height,this.boundingBox.left=this.left,this.boundingBox.top=this.top,this.boundingBox.bottom=this.top+this.height,this.boundingBox.right=this.left+this.width}},{key:"distanceToBorder",value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),void 0!==this.options.label){var l=n+.5*this.height+3;this.labelModule.draw(t,o,l,s,"hanging")}this.updateBoundingBox(o,n)}},{key:"updateBoundingBox",value:function(t,e){this.boundingBox.top=e-this.options.size,this.boundingBox.left=t-this.options.size,this.boundingBox.right=t+this.options.size,this.boundingBox.bottom=e+this.options.size,void 0!==this.options.label&&this.labelModule.size.width>0&&(this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelModule.size.height+3))}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),this.updateBoundingBox(e,i,t,o),this.labelModule.draw(t,e,i,o)}},{key:"updateBoundingBox",value:function(t,e,i,o){this.resize(i,o),this.left=t-.5*this.width,this.top=e-.5*this.height,this.boundingBox.left=this.left,this.boundingBox.top=this.top,this.boundingBox.bottom=this.top+this.height,this.boundingBox.right=this.left+this.width}},{key:"distanceToBorder",value:function(t,e){this.resize(t);var i=.5*this.width,o=.5*this.height,n=Math.sin(e)*i,s=Math.cos(e)*o;return i*o/Math.sqrt(n*n+s*s)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0){var i=5;this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelModule.size.height+i)}}},{key:"_icon",value:function(t,e,i,o){var n=Number(this.options.icon.size);void 0!==this.options.icon.code?(t.font=(o?"bold ":"")+n+"px "+this.options.icon.face,t.fillStyle=this.options.icon.color||"black",t.textAlign="center",t.textBaseline="middle",this.enableShadow(t),t.fillText(this.options.icon.code,e,i),this.disableShadow(t)):console.error("When using the icon shape, you need to define the code in the icon options object. This can be done per node or globally.")}},{key:"distanceToBorder",value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i0&&(this.enableBorderDashes(t),t.stroke(),this.disableBorderDashes(t)),t.restore(),t.closePath()}this._drawImageAtPosition(t),this._drawImageLabel(t,e,i,o||n),this.updateBoundingBox(e,i)}},{key:"updateBoundingBox",value:function(t,e){this.resize(),this.left=t-this.width/2,this.top=e-this.height/2,this.boundingBox.top=this.top,this.boundingBox.left=this.left,this.boundingBox.right=this.left+this.width,this.boundingBox.bottom=this.top+this.height,void 0!==this.options.label&&this.labelModule.size.width>0&&(this.boundingBox.left=Math.min(this.boundingBox.left,this.labelModule.size.left),this.boundingBox.right=Math.max(this.boundingBox.right,this.labelModule.size.left+this.labelModule.size.width),this.boundingBox.bottom=Math.max(this.boundingBox.bottom,this.boundingBox.bottom+this.labelOffset))}},{key:"distanceToBorder", +value:function(t,e){return this._distanceToBorder(t,e)}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;ii.shape.height?(r=i.x+.5*i.shape.width,a=i.y-h):(r=i.x+h,a=i.y-.5*i.shape.height),s=this._pointOnCircle(r,a,h,.125),this.labelModule.draw(t,s.x,s.y,n)}}}},{key:"isOverlappingWith",value:function(t){if(this.connected){var e=10,i=this.from.x,o=this.from.y,n=this.to.x,s=this.to.y,r=t.left,a=t.top,h=this.edgeType.getDistanceToEdge(i,o,n,s,r,a);return e>h}return!1}},{key:"_rotateForLabelAlignment",value:function(t){var e=this.from.y-this.to.y,i=this.from.x-this.to.x,o=Math.atan2(e,i);(-1>o&&0>i||o>0&&0>i)&&(o+=Math.PI),t.rotate(o)}},{key:"_pointOnCircle",value:function(t,e,i,o){var n=2*o*Math.PI;return{x:t+i*Math.cos(n),y:e-i*Math.sin(n)}}},{key:"select",value:function(){this.selected=!0}},{key:"unselect",value:function(){this.selected=!1}},{key:"cleanup",value:function(){return this.edgeType.cleanup()}}],[{key:"parseOptions",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=arguments.length<=3||void 0===arguments[3]?{}:arguments[3],n=["arrowStrikethrough","id","from","hidden","hoverWidth","label","labelHighlightBold","length","line","opacity","physics","scaling","selectionWidth","selfReferenceSize","to","title","value","width"];if(g.selectiveDeepExtend(n,t,e,i),g.mergeOptions(t,e,"smooth",i,o),g.mergeOptions(t,e,"shadow",i,o),void 0!==e.dashes&&null!==e.dashes?t.dashes=e.dashes:i===!0&&null===e.dashes&&(t.dashes=Object.create(o.dashes)),void 0!==e.scaling&&null!==e.scaling?(void 0!==e.scaling.min&&(t.scaling.min=e.scaling.min),void 0!==e.scaling.max&&(t.scaling.max=e.scaling.max),g.mergeOptions(t.scaling,e.scaling,"label",i,o.scaling)):i===!0&&null===e.scaling&&(t.scaling=Object.create(o.scaling)),void 0!==e.arrows&&null!==e.arrows)if("string"==typeof e.arrows){var r=e.arrows.toLowerCase();t.arrows.to.enabled=-1!=r.indexOf("to"),t.arrows.middle.enabled=-1!=r.indexOf("middle"),t.arrows.from.enabled=-1!=r.indexOf("from")}else{if("object"!==s(e.arrows))throw new Error("The arrow newOptions can only be an object or a string. Refer to the documentation. You used:"+JSON.stringify(e.arrows));g.mergeOptions(t.arrows,e.arrows,"to",i,o.arrows),g.mergeOptions(t.arrows,e.arrows,"middle",i,o.arrows),g.mergeOptions(t.arrows,e.arrows,"from",i,o.arrows)}else i===!0&&null===e.arrows&&(t.arrows=Object.create(o.arrows));if(void 0!==e.color&&null!==e.color)if(t.color=g.deepExtend({},t.color,!0),g.isString(e.color))t.color.color=e.color,t.color.highlight=e.color,t.color.hover=e.color,t.color.inherit=!1;else{var a=!1;void 0!==e.color.color&&(t.color.color=e.color.color,a=!0),void 0!==e.color.highlight&&(t.color.highlight=e.color.highlight,a=!0),void 0!==e.color.hover&&(t.color.hover=e.color.hover,a=!0),void 0!==e.color.inherit&&(t.color.inherit=e.color.inherit),void 0!==e.color.opacity&&(t.color.opacity=Math.min(1,Math.max(0,e.color.opacity))),void 0===e.color.inherit&&a===!0&&(t.color.inherit=!1)}else i===!0&&null===e.color&&(t.color=g.bridgeObject(o.color));void 0!==e.font&&null!==e.font?h["default"].parseOptions(t.font,e):i===!0&&null===e.font&&(t.font=g.bridgeObject(o.font))}}]),t}();e["default"]=y},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),h=function(){function t(t,e){for(var i=0;iMath.abs(e)||this.options.smooth.forceDirection===!0||"horizontal"===this.options.smooth.forceDirection)&&"vertical"!==this.options.smooth.forceDirection?(o=this.from.y,s=this.to.y,i=this.from.x-r*t,n=this.to.x+r*t):(o=this.from.y-r*e,s=this.to.y+r*e,i=this.from.x,n=this.to.x),[{x:i,y:o},{x:n,y:s}]}},{key:"getViaNode",value:function(){return this._getViaCoordinates()}},{key:"_findBorderPosition",value:function(t,e){return this._findBorderPositionBezier(t,e)}},{key:"_getDistanceToEdge",value:function(t,e,i,o,n,s){var r=arguments.length<=6||void 0===arguments[6]?this._getViaCoordinates():arguments[6],h=a(r,2),d=h[0],l=h[1];return this._getDistanceToBezierEdge(t,e,i,o,n,s,d,l)}},{key:"getPoint",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?this._getViaCoordinates():arguments[1],i=a(e,2),o=i[0],n=i[1],s=t,r=[];r[0]=Math.pow(1-s,3),r[1]=3*s*Math.pow(1-s,2),r[2]=3*Math.pow(s,2)*(1-s),r[3]=Math.pow(s,3);var h=r[0]*this.fromPoint.x+r[1]*o.x+r[2]*n.x+r[3]*this.toPoint.x,d=r[0]*this.fromPoint.y+r[1]*o.y+r[2]*n.y+r[3]*this.toPoint.y;return{x:h,y:d}}}]),e}(l["default"]);e["default"]=c},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;il;l++)c=.1*l,v[0]=Math.pow(1-c,3),v[1]=3*c*Math.pow(1-c,2),v[2]=3*Math.pow(c,2)*(1-c),v[3]=Math.pow(c,3),u=v[0]*t+v[1]*r.x+v[2]*a.x+v[3]*i,p=v[0]*e+v[1]*r.y+v[2]*a.y+v[3]*o,l>0&&(d=this._getDistanceToLine(f,m,u,p,n,s),h=h>d?d:h),f=u,m=p;return h}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i=l&&h>d;){var m=.5*(l+c);if(i=this.getPoint(m,a),o=Math.atan2(p.y-i.y,p.x-i.x),n=p.distanceToBorder(e,o),s=Math.sqrt(Math.pow(i.x-p.x,2)+Math.pow(i.y-p.y,2)),r=n-s,Math.abs(r)r?f===!1?l=m:c=m:f===!1?c=m:l=m,d++}return i.t=m,i}},{key:"_getDistanceToBezierEdge",value:function(t,e,i,o,n,s,r){var a=1e9,h=void 0,d=void 0,l=void 0,c=void 0,u=void 0,p=t,f=e;for(d=1;10>d;d++)l=.1*d,c=Math.pow(1-l,2)*t+2*l*(1-l)*r.x+Math.pow(l,2)*i,u=Math.pow(1-l,2)*e+2*l*(1-l)*r.y+Math.pow(l,2)*o,d>0&&(h=this._getDistanceToLine(p,f,c,u,n,s),a=a>h?h:a),p=c,f=u;return a}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),s=function(){function t(t,e){for(var i=0;io.shape.height?(e=o.x+.5*o.shape.width,i=o.y-n):(e=o.x+n,i=o.y-.5*o.shape.height),[e,i,n]}},{key:"_pointOnCircle",value:function(t,e,i,o){var n=2*o*Math.PI;return{x:t+i*Math.cos(n),y:e-i*Math.sin(n)}}},{key:"_findBorderPositionCircle",value:function(t,e,i){for(var o=i.x,n=i.y,s=i.low,r=i.high,a=i.direction,h=10,d=0,l=this.options.selfReferenceSize,c=void 0,u=void 0,p=void 0,f=void 0,m=void 0,v=.05,g=.5*(s+r);r>=s&&h>d&&(g=.5*(s+r),c=this._pointOnCircle(o,n,l,g),u=Math.atan2(t.y-c.y,t.x-c.x),p=t.distanceToBorder(e,u),f=Math.sqrt(Math.pow(c.x-t.x,2)+Math.pow(c.y-t.y,2)), +m=p-f,!(Math.abs(m)0?a>0?s=g:r=g:a>0?r=g:s=g,d++;return c.t=g,c}},{key:"getLineWidth",value:function(t,e){return t===!0?Math.max(this.selectionWidth,.3/this.body.view.scale):e===!0?Math.max(this.hoverWidth,.3/this.body.view.scale):Math.max(this.options.width,.3/this.body.view.scale)}},{key:"getColor",value:function(t,e,i){var o=this.options.color;if(o.inherit!==!1){if("both"===o.inherit&&this.from.id!==this.to.id){var n=t.createLinearGradient(this.from.x,this.from.y,this.to.x,this.to.y),s=void 0,a=void 0;return s=this.from.options.color.highlight.border,a=this.to.options.color.highlight.border,this.from.selected===!1&&this.to.selected===!1?(s=r.overrideOpacity(this.from.options.color.border,this.options.color.opacity),a=r.overrideOpacity(this.to.options.color.border,this.options.color.opacity)):this.from.selected===!0&&this.to.selected===!1?a=this.to.options.color.border:this.from.selected===!1&&this.to.selected===!0&&(s=this.from.options.color.border),n.addColorStop(0,s),n.addColorStop(1,a),n}this.colorDirty===!0&&("to"===o.inherit?(this.color.highlight=this.to.options.color.highlight.border,this.color.hover=this.to.options.color.hover.border,this.color.color=r.overrideOpacity(this.to.options.color.border,o.opacity)):(this.color.highlight=this.from.options.color.highlight.border,this.color.hover=this.from.options.color.hover.border,this.color.color=r.overrideOpacity(this.from.options.color.border,o.opacity)))}else this.colorDirty===!0&&(this.color.highlight=o.highlight,this.color.hover=o.hover,this.color.color=r.overrideOpacity(o.color,o.opacity));return this.colorDirty=!1,e===!0?this.color.highlight:i===!0?this.color.hover:this.color.color}},{key:"_circle",value:function(t,e,i,o){this.enableShadow(t),t.beginPath(),t.arc(e,i,o,0,2*Math.PI,!1),t.stroke(),this.disableShadow(t)}},{key:"getDistanceToEdge",value:function(t,e,i,o,s,r,a){var h=0;if(this.from!=this.to)h=this._getDistanceToEdge(t,e,i,o,s,r,a);else{var d=this._getCircleData(),l=n(d,3),c=l[0],u=l[1],p=l[2],f=c-s,m=u-r;h=Math.abs(Math.sqrt(f*f+m*m)-p)}return this.labelModule.size.lefts&&this.labelModule.size.topr?0:h}},{key:"_getDistanceToLine",value:function(t,e,i,o,n,s){var r=i-t,a=o-e,h=r*r+a*a,d=((n-t)*r+(s-e)*a)/h;d>1?d=1:0>d&&(d=0);var l=t+d*r,c=e+d*a,u=l-n,p=c-s;return Math.sqrt(u*u+p*p)}},{key:"getArrowData",value:function(t,e,i,o,s){var r=void 0,a=void 0,h=void 0,d=void 0,l=void 0,c=void 0,u=this.getLineWidth(o,s);if("from"===e?(h=this.from,d=this.to,l=.1,c=this.options.arrows.from.scaleFactor):"to"===e?(h=this.to,d=this.from,l=-.1,c=this.options.arrows.to.scaleFactor):(h=this.to,d=this.from,c=this.options.arrows.middle.scaleFactor),h!=d)if("middle"!==e)if(this.options.smooth.enabled===!0){a=this.findBorderPosition(h,t,{via:i});var p=this.getPoint(Math.max(0,Math.min(1,a.t+l)),i);r=Math.atan2(a.y-p.y,a.x-p.x)}else r=Math.atan2(h.y-d.y,h.x-d.x),a=this.findBorderPosition(h,t);else r=Math.atan2(h.y-d.y,h.x-d.x),a=this.getPoint(.5,i);else{var f=this._getCircleData(t),m=n(f,3),v=m[0],g=m[1],y=m[2];"from"===e?(a=this.findBorderPosition(this.from,t,{x:v,y:g,low:.25,high:.6,direction:-1}),r=-2*a.t*Math.PI+1.5*Math.PI+.1*Math.PI):"to"===e?(a=this.findBorderPosition(this.from,t,{x:v,y:g,low:.6,high:1,direction:1}),r=-2*a.t*Math.PI+1.5*Math.PI-1.1*Math.PI):(a=this._pointOnCircle(v,g,y,.175),r=3.9269908169872414)}var b=15*c+3*u,w=a.x-.9*b*Math.cos(r),_=a.y-.9*b*Math.sin(r),x={x:w,y:_};return{point:a,core:x,angle:r,length:b}}},{key:"drawArrowHead",value:function(t,e,i,o){t.strokeStyle=this.getColor(t,e,i),t.fillStyle=t.strokeStyle,t.lineWidth=this.getLineWidth(e,i),t.arrow(o.point.x,o.point.y,o.angle,o.length),this.enableShadow(t),t.fill(),this.disableShadow(t)}},{key:"enableShadow",value:function(t){this.options.shadow.enabled===!0&&(t.shadowColor=this.options.shadow.color,t.shadowBlur=this.options.shadow.size,t.shadowOffsetX=this.options.shadow.x,t.shadowOffsetY=this.options.shadow.y)}},{key:"disableShadow",value:function(t){this.options.shadow.enabled===!0&&(t.shadowColor="rgba(0,0,0,0)",t.shadowBlur=0,t.shadowOffsetX=0,t.shadowOffsetY=0)}}]),t}();e["default"]=a},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function s(t,e){if(!t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return!e||"object"!=typeof e&&"function"!=typeof e?t:e}function r(t,e){if("function"!=typeof e&&null!==e)throw new TypeError("Super expression must either be null or a function, not "+typeof e);t.prototype=Object.create(e&&e.prototype,{constructor:{value:t,enumerable:!1,writable:!0,configurable:!0}}),e&&(Object.setPrototypeOf?Object.setPrototypeOf(t,e):t.__proto__=e)}Object.defineProperty(e,"__esModule",{value:!0});var a=function(){function t(t,e){for(var i=0;i=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*s,e=this.from.y-i*s):this.from.x>this.to.x&&(t=this.from.x-i*s,e=this.from.y-i*s):this.from.ythis.to.x&&(t=this.from.x-i*s,e=this.from.y+i*s)),"discrete"===o&&(t=i*s>n?this.from.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*n,e=this.from.y-i*n):this.from.x>this.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n)),"discrete"===o&&(e=i*n>s?this.from.y:e));else if("straightCross"===o)Math.abs(this.from.x-this.to.x)<=Math.abs(this.from.y-this.to.y)?(t=this.from.x,e=this.from.yMath.abs(this.from.y-this.to.y)&&(t=this.from.x=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*s,e=this.from.y-i*s,t=this.to.xthis.to.x&&(t=this.from.x-i*s,e=this.from.y-i*s,t=this.to.x>t?this.to.x:t):this.from.ythis.to.x&&(t=this.from.x-i*s,e=this.from.y+i*s,t=this.to.x>t?this.to.x:t)):Math.abs(this.from.x-this.to.x)>Math.abs(this.from.y-this.to.y)&&(this.from.y>=this.to.y?this.from.x<=this.to.x?(t=this.from.x+i*n,e=this.from.y-i*n,e=this.to.y>e?this.to.y:e):this.from.x>this.to.x&&(t=this.from.x-i*n,e=this.from.y-i*n,e=this.to.y>e?this.to.y:e):this.from.ythis.to.x&&(t=this.from.x-i*n,e=this.from.y+i*n,e=this.to.y1||this.startedStabilization===!0)&&setTimeout(function(){t.body.emitter.emit("stabilized",{iterations:e}),t.startedStabilization=!1,t.stabilizationIterations=0},0)}},{key:"physicsTick",value:function(){if(this.startedStabilization===!1&&(this.body.emitter.emit("startStabilizing"),this.startedStabilization=!0),this.stabilized===!1){if(this.adaptiveTimestep===!0&&this.adaptiveTimestepEnabled===!0){var t=1.2;this.adaptiveCounter%this.adaptiveInterval===0?(this.timestep=2*this.timestep,this.calculateForces(),this.moveNodes(),this.revert(),this.timestep=.5*this.timestep,this.calculateForces(),this.moveNodes(),this.calculateForces(),this.moveNodes(),this._evaluateStepQuality()===!0?this.timestep=t*this.timestep:this.timestep/ts))return!1;return!0}},{key:"moveNodes",value:function(){for(var t=this.physicsBody.physicsNodeIndices,e=this.options.maxVelocity?this.options.maxVelocity:1e9,i=0,o=0,n=5,s=0;se?s[t].x>0?e:-e:s[t].x,i.x+=s[t].x*o}else n[t].x=0,s[t].x=0;if(i.options.fixed.y===!1){var h=this.modelOptions.damping*s[t].y,d=(n[t].y-h)/i.options.mass;s[t].y+=d*o,s[t].y=Math.abs(s[t].y)>e?s[t].y>0?e:-e:s[t].y,i.y+=s[t].y*o}else n[t].y=0,s[t].y=0;var l=Math.sqrt(Math.pow(s[t].x,2)+Math.pow(s[t].y,2));return l}},{key:"calculateForces",value:function(){this.gravitySolver.solve(),this.nodesSolver.solve(),this.edgesSolver.solve()}},{key:"_freezeNodes",value:function(){var t=this.body.nodes;for(var e in t)t.hasOwnProperty(e)&&t[e].x&&t[e].y&&(this.freezeCache[e]={x:t[e].options.fixed.x,y:t[e].options.fixed.y},t[e].options.fixed.x=!0,t[e].options.fixed.y=!0)}},{key:"_restoreFrozenNodes",value:function(){var t=this.body.nodes;for(var e in t)t.hasOwnProperty(e)&&void 0!==this.freezeCache[e]&&(t[e].options.fixed.x=this.freezeCache[e].x,t[e].options.fixed.y=this.freezeCache[e].y);this.freezeCache={}}},{key:"stabilize",value:function(){var t=this,e=arguments.length<=0||void 0===arguments[0]?this.options.stabilization.iterations:arguments[0];return"number"!=typeof e&&(console.log("The stabilize method needs a numeric amount of iterations. Switching to default: ",this.options.stabilization.iterations),e=this.options.stabilization.iterations),0===this.physicsBody.physicsNodeIndices.length?void(this.ready=!0):(this.adaptiveTimestep=this.options.adaptiveTimestep,this.body.emitter.emit("_resizeNodes"),this.stopSimulation(),this.stabilized=!1,this.body.emitter.emit("_blockRedraw"),this.targetIterations=e,this.options.stabilization.onlyDynamicEdges===!0&&this._freezeNodes(),this.stabilizationIterations=0,void setTimeout(function(){return t._stabilizationBatch()},0))}},{key:"_stabilizationBatch",value:function(){this.startedStabilization===!1&&(this.body.emitter.emit("startStabilizing"),this.startedStabilization=!0);for(var t=0;this.stabilized===!1&&t0){var t=void 0,e=this.body.nodes,i=this.physicsBody.physicsNodeIndices,o=i.length,n=this._formBarnesHutTree(e,i);this.barnesHutTree=n;for(var s=0;o>s;s++)t=e[i[s]],t.options.mass>0&&(this._getForceContribution(n.root.children.NW,t),this._getForceContribution(n.root.children.NE,t),this._getForceContribution(n.root.children.SW,t),this._getForceContribution(n.root.children.SE,t))}}},{key:"_getForceContribution",value:function(t,e){if(t.childrenCount>0){var i=void 0,o=void 0,n=void 0;i=t.centerOfMass.x-e.x,o=t.centerOfMass.y-e.y,n=Math.sqrt(i*i+o*o),n*t.calcSize>this.thetaInversed?this._calculateForces(n,i,o,e,t):4===t.childrenCount?(this._getForceContribution(t.children.NW,e),this._getForceContribution(t.children.NE,e),this._getForceContribution(t.children.SW,e),this._getForceContribution(t.children.SE,e)):t.children.data.id!=e.id&&this._calculateForces(n,i,o,e,t)}}},{key:"_calculateForces",value:function(t,e,i,o,n){0===t&&(t=.1,e=t),this.overlapAvoidanceFactor<1&&(t=Math.max(.1+this.overlapAvoidanceFactor*o.shape.radius,t-o.shape.radius));var s=this.options.gravitationalConstant*n.mass*o.options.mass/Math.pow(t,3),r=e*s,a=i*s;this.physicsBody.forces[o.id].x+=r,this.physicsBody.forces[o.id].y+=a}},{key:"_formBarnesHutTree",value:function(t,e){for(var i=void 0,o=e.length,n=t[e[0]].x,s=t[e[0]].y,r=t[e[0]].x,a=t[e[0]].y,h=1;o>h;h++){var d=t[e[h]].x,l=t[e[h]].y;t[e[h]].options.mass>0&&(n>d&&(n=d),d>r&&(r=d),s>l&&(s=l),l>a&&(a=l))}var c=Math.abs(r-n)-Math.abs(a-s);c>0?(s-=.5*c,a+=.5*c):(n+=.5*c,r-=.5*c);var u=1e-5,p=Math.max(u,Math.abs(r-n)),f=.5*p,m=.5*(n+r),v=.5*(s+a),g={root:{centerOfMass:{x:0,y:0},mass:0,range:{minX:m-f,maxX:m+f,minY:v-f,maxY:v+f},size:p,calcSize:1/p,children:{data:null},maxWidth:0,level:0,childrenCount:4}};this._splitBranch(g.root);for(var y=0;o>y;y++)i=t[e[y]],i.options.mass>0&&this._placeInTree(g.root,i);return g}},{key:"_updateBranchMass",value:function(t,e){var i=t.mass+e.options.mass,o=1/i;t.centerOfMass.x=t.centerOfMass.x*t.mass+e.x*e.options.mass,t.centerOfMass.x*=o,t.centerOfMass.y=t.centerOfMass.y*t.mass+e.y*e.options.mass,t.centerOfMass.y*=o,t.mass=i;var n=Math.max(Math.max(e.height,e.radius),e.width);t.maxWidth=t.maxWidthe.x?t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NW"):this._placeInRegion(t,e,"SW"):t.children.NW.range.maxY>e.y?this._placeInRegion(t,e,"NE"):this._placeInRegion(t,e,"SE")}},{key:"_placeInRegion",value:function(t,e,i){switch(t.children[i].childrenCount){case 0:t.children[i].children.data=e,t.children[i].childrenCount=1,this._updateBranchMass(t.children[i],e);break;case 1:t.children[i].children.data.x===e.x&&t.children[i].children.data.y===e.y?(e.x+=this.seededRandom(),e.y+=this.seededRandom()):(this._splitBranch(t.children[i]),this._placeInTree(t.children[i],e));break;case 4:this._placeInTree(t.children[i],e)}}},{key:"_splitBranch",value:function(t){var e=null;1===t.childrenCount&&(e=t.children.data,t.mass=0,t.centerOfMass.x=0,t.centerOfMass.y=0),t.childrenCount=4,t.children.data=null,this._insertRegion(t,"NW"),this._insertRegion(t,"NE"),this._insertRegion(t,"SW"),this._insertRegion(t,"SE"),null!=e&&this._placeInTree(t,e)}},{key:"_insertRegion",value:function(t,e){var i=void 0,o=void 0,n=void 0,s=void 0,r=.5*t.size;switch(e){case"NW":i=t.range.minX,o=t.range.minX+r,n=t.range.minY,s=t.range.minY+r;break;case"NE":i=t.range.minX+r,o=t.range.maxX,n=t.range.minY,s=t.range.minY+r;break;case"SW":i=t.range.minX,o=t.range.minX+r,n=t.range.minY+r,s=t.range.maxY;break;case"SE":i=t.range.minX+r,o=t.range.maxX,n=t.range.minY+r,s=t.range.maxY}t.children[e]={centerOfMass:{x:0,y:0},mass:0,range:{minX:i,maxX:o,minY:n,maxY:s},size:.5*t.size,calcSize:2*t.calcSize,children:{data:null},maxWidth:0,level:t.level+1,childrenCount:0}}},{key:"_debug",value:function(t,e){void 0!==this.barnesHutTree&&(t.lineWidth=1,this._drawBranch(this.barnesHutTree.root,t,e))}},{key:"_drawBranch",value:function(t,e,i){void 0===i&&(i="#FF0000"),4===t.childrenCount&&(this._drawBranch(t.children.NW,e),this._drawBranch(t.children.NE,e),this._drawBranch(t.children.SE,e),this._drawBranch(t.children.SW,e)),e.strokeStyle=i,e.beginPath(),e.moveTo(t.range.minX,t.range.minY),e.lineTo(t.range.maxX,t.range.minY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.minY),e.lineTo(t.range.maxX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.maxX,t.range.maxY), +e.lineTo(t.range.minX,t.range.maxY),e.stroke(),e.beginPath(),e.moveTo(t.range.minX,t.range.maxY),e.lineTo(t.range.minX,t.range.minY),e.stroke()}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ii&&(s=.5*c>i?1:u*i+p,s/=i,o=t*s,n=e*s,l[r.id].x-=o,l[r.id].y-=n,l[a.id].x+=o,l[a.id].y+=n)}}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;ii?-Math.pow(f*i,2)+Math.pow(f*p,2):0,0===i?i=.01:s/=i,o=t*s,n=e*s,u[r.id].x-=o,u[r.id].y-=n,u[a.id].x+=o,u[a.id].y+=n}}}]),t}();e["default"]=n},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;i0){var s=n.edges.length+1,r=this.options.centralGravity*s*n.options.mass;o[n.id].x=e*r,o[n.id].y=i*r}}}]),e}(d["default"]);e["default"]=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol?"symbol":typeof t},r=function(){function t(t,e){for(var i=0;i=t&&i.push(n.id)}for(var r=0;r0&&Object.keys(p).length>0&&m===!0&&o.push({nodes:u,edges:p})}}}for(var b=0;bo?r.x:o,n=r.ys?r.y:s;return{x:.5*(i+o),y:.5*(n+s)}}},{key:"openCluster",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!0:arguments[2];if(void 0===t)throw new Error("No clusterNodeId supplied to openCluster.");if(void 0===this.body.nodes[t])throw new Error("The clusterNodeId supplied to openCluster does not exist.");if(void 0===this.body.nodes[t].containedNodes)return void console.log("The node:"+t+" is not a cluster.");var o=this.body.nodes[t],n=o.containedNodes,s=o.containedEdges;if(void 0!==e&&void 0!==e.releaseFunction&&"function"==typeof e.releaseFunction){var r={},a={x:o.x,y:o.y};for(var d in n)if(n.hasOwnProperty(d)){var l=this.body.nodes[d];r[d]={x:l.x,y:l.y}}var u=e.releaseFunction(a,r);for(var p in n)if(n.hasOwnProperty(p)){var f=this.body.nodes[p];void 0!==u[p]&&(f.x=void 0===u[p].x?o.x:u[p].x,f.y=void 0===u[p].y?o.y:u[p].y)}}else for(var m in n)if(n.hasOwnProperty(m)){var v=this.body.nodes[m];v=n[m],v.options.fixed.x===!1&&(v.x=o.x),v.options.fixed.y===!1&&(v.y=o.y)}for(var g in n)if(n.hasOwnProperty(g)){var y=this.body.nodes[g];y.vx=o.vx,y.vy=o.vy,y.setOptions({hidden:!1,physics:!0}),delete this.clusteredNodes[g]}for(var b=[],w=0;wo;)e.push(this.body.nodes[t].id),t=this.clusteredNodes[t].clusterId,o++;return e.push(this.body.nodes[t].id),e.reverse(),e}},{key:"_getConnectedId",value:function(t,e){return t.toId!=e?t.toId:t.fromId!=e?t.fromId:t.fromId}},{key:"_getHubSize",value:function(){for(var t=0,e=0,i=0,o=0,n=0;no&&(o=s.edges.length),t+=s.edges.length,e+=Math.pow(s.edges.length,2),i+=1}t/=i,e/=i;var r=e-Math.pow(t,2),a=Math.sqrt(r),h=Math.floor(t+2*a);return h>o&&(h=o),h}}]),t}();e["default"]=u},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i0)for(var a=0;ae.shape.boundingBox.left&&(s=e.shape.boundingBox.left),re.shape.boundingBox.top&&(o=e.shape.boundingBox.top),n0)for(var a=0;ae.x&&(s=e.x),re.y&&(o=e.y),n0,t.renderTimer=void 0}),this.body.emitter.on("destroy",function(){t.renderRequests=0,t.allowRedraw=!1,t.renderingActive=!1,t.requiresTimeout===!0?clearTimeout(t.renderTimer):cancelAnimationFrame(t.renderTimer),t.body.emitter.off()})}},{key:"setOptions",value:function(t){if(void 0!==t){var e=["hideEdgesOnDrag","hideNodesOnDrag"];s.selectiveDeepExtend(e,this.options,t)}}},{key:"_startRendering",value:function(){this.renderingActive===!0&&void 0===this.renderTimer&&(this.requiresTimeout===!0?this.renderTimer=window.setTimeout(this._renderStep.bind(this),this.simulationInterval):this.renderTimer=window.requestAnimationFrame(this._renderStep.bind(this)))}},{key:"_renderStep",value:function(){this.renderingActive===!0&&(this.renderTimer=void 0,this.requiresTimeout===!0&&this._startRendering(),this._redraw(),this.requiresTimeout===!1&&this._startRendering())}},{key:"redraw",value:function(){this.body.emitter.emit("setSize"),this._redraw()}},{key:"_requestRedraw",value:function(){var t=this;this.redrawRequested!==!0&&this.renderingActive===!1&&this.allowRedraw===!0&&(this.redrawRequested=!0,this.requiresTimeout===!0?window.setTimeout(function(){t._redraw(!1)},0):window.requestAnimationFrame(function(){t._redraw(!1)}))}},{key:"_redraw",value:function(){var t=arguments.length<=0||void 0===arguments[0]?!1:arguments[0];if(this.allowRedraw===!0){this.body.emitter.emit("initRedraw"),this.redrawRequested=!1;var e=this.canvas.frame.canvas.getContext("2d");0!==this.canvas.frame.canvas.width&&0!==this.canvas.frame.canvas.height||this.canvas.setSize(),this.pixelRatio=(window.devicePixelRatio||1)/(e.webkitBackingStorePixelRatio||e.mozBackingStorePixelRatio||e.msBackingStorePixelRatio||e.oBackingStorePixelRatio||e.backingStorePixelRatio||1),e.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0);var i=this.canvas.frame.canvas.clientWidth,o=this.canvas.frame.canvas.clientHeight;if(e.clearRect(0,0,i,o),0===this.canvas.frame.clientWidth)return;e.save(),e.translate(this.body.view.translation.x,this.body.view.translation.y),e.scale(this.body.view.scale,this.body.view.scale),e.beginPath(),this.body.emitter.emit("beforeDrawing",e),e.closePath(),t===!1&&(this.dragging===!1||this.dragging===!0&&this.options.hideEdgesOnDrag===!1)&&this._drawEdges(e),(this.dragging===!1||this.dragging===!0&&this.options.hideNodesOnDrag===!1)&&this._drawNodes(e,t),e.beginPath(),this.body.emitter.emit("afterDrawing",e),e.closePath(),e.restore(),t===!0&&e.clearRect(0,0,i,o)}}},{key:"_resizeNodes",value:function(){var t=this.canvas.frame.canvas.getContext("2d");void 0===this.pixelRatio&&(this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1)),t.setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0),t.save(),t.translate(this.body.view.translation.x,this.body.view.translation.y),t.scale(this.body.view.scale,this.body.view.scale);var e=this.body.nodes,i=void 0;for(var o in e)e.hasOwnProperty(o)&&(i=e[o],i.resize(t),i.updateBoundingBox(t,i.selected));t.restore()}},{key:"_drawNodes",value:function(t){for(var e=arguments.length<=1||void 0===arguments[1]?!1:arguments[1],i=this.body.nodes,o=this.body.nodeIndices,n=void 0,s=[],r=20,a=this.canvas.DOMtoCanvas({x:-r,y:-r}),h=this.canvas.DOMtoCanvas({x:this.canvas.frame.canvas.clientWidth+r,y:this.canvas.frame.canvas.clientHeight+r}),d={top:a.y,left:a.x,bottom:h.y,right:h.x},l=0;l0){var t=this.frame.canvas.width/this.pixelRatio/this.cameraState.previousWidth,e=this.frame.canvas.height/this.pixelRatio/this.cameraState.previousHeight,i=this.cameraState.scale;1!=t&&1!=e?i=.5*this.cameraState.scale*(t+e):1!=t?i=this.cameraState.scale*t:1!=e&&(i=this.cameraState.scale*e),this.body.view.scale=i;var o=this.DOMtoCanvas({x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight}),n={x:o.x-this.cameraState.position.x,y:o.y-this.cameraState.position.y};this.body.view.translation.x+=n.x*this.body.view.scale,this.body.view.translation.y+=n.y*this.body.view.scale}}},{key:"_prepareValue",value:function(t){if("number"==typeof t)return t+"px";if("string"==typeof t){if(-1!==t.indexOf("%")||-1!==t.indexOf("px"))return t;if(-1===t.indexOf("%"))return t+"px"}throw new Error("Could not use the value supplied for width or height:"+t)}},{key:"_create",value:function(){for(;this.body.container.hasChildNodes();)this.body.container.removeChild(this.body.container.firstChild);if(this.frame=document.createElement("div"),this.frame.className="vis-network",this.frame.style.position="relative",this.frame.style.overflow="hidden",this.frame.tabIndex=900,this.frame.canvas=document.createElement("canvas"),this.frame.canvas.style.position="relative",this.frame.appendChild(this.frame.canvas),this.frame.canvas.getContext){var t=this.frame.canvas.getContext("2d");this.pixelRatio=(window.devicePixelRatio||1)/(t.webkitBackingStorePixelRatio||t.mozBackingStorePixelRatio||t.msBackingStorePixelRatio||t.oBackingStorePixelRatio||t.backingStorePixelRatio||1), +this.frame.canvas.getContext("2d").setTransform(this.pixelRatio,0,0,this.pixelRatio,0,0)}else{var e=document.createElement("DIV");e.style.color="red",e.style.fontWeight="bold",e.style.padding="10px",e.innerHTML="Error: your browser does not support HTML canvas",this.frame.canvas.appendChild(e)}this.body.container.appendChild(this.frame),this.body.view.scale=1,this.body.view.translation={x:.5*this.frame.canvas.clientWidth,y:.5*this.frame.canvas.clientHeight},this._bindHammer()}},{key:"_bindHammer",value:function(){var t=this;void 0!==this.hammer&&this.hammer.destroy(),this.drag={},this.pinch={},this.hammer=new s(this.frame.canvas),this.hammer.get("pinch").set({enable:!0}),this.hammer.get("pan").set({threshold:5,direction:s.DIRECTION_ALL}),r.onTouch(this.hammer,function(e){t.body.eventListeners.onTouch(e)}),this.hammer.on("tap",function(e){t.body.eventListeners.onTap(e)}),this.hammer.on("doubletap",function(e){t.body.eventListeners.onDoubleTap(e)}),this.hammer.on("press",function(e){t.body.eventListeners.onHold(e)}),this.hammer.on("panstart",function(e){t.body.eventListeners.onDragStart(e)}),this.hammer.on("panmove",function(e){t.body.eventListeners.onDrag(e)}),this.hammer.on("panend",function(e){t.body.eventListeners.onDragEnd(e)}),this.hammer.on("pinch",function(e){t.body.eventListeners.onPinch(e)}),this.frame.canvas.addEventListener("mousewheel",function(e){t.body.eventListeners.onMouseWheel(e)}),this.frame.canvas.addEventListener("DOMMouseScroll",function(e){t.body.eventListeners.onMouseWheel(e)}),this.frame.canvas.addEventListener("mousemove",function(e){t.body.eventListeners.onMouseMove(e)}),this.frame.canvas.addEventListener("contextmenu",function(e){t.body.eventListeners.onContext(e)}),this.hammerFrame=new s(this.frame),r.onRelease(this.hammerFrame,function(e){t.body.eventListeners.onRelease(e)})}},{key:"setSize",value:function(){var t=arguments.length<=0||void 0===arguments[0]?this.options.width:arguments[0],e=arguments.length<=1||void 0===arguments[1]?this.options.height:arguments[1];t=this._prepareValue(t),e=this._prepareValue(e);var i=!1,o=this.frame.canvas.width,n=this.frame.canvas.height,s=this.frame.canvas.getContext("2d"),r=this.pixelRatio;return this.pixelRatio=(window.devicePixelRatio||1)/(s.webkitBackingStorePixelRatio||s.mozBackingStorePixelRatio||s.msBackingStorePixelRatio||s.oBackingStorePixelRatio||s.backingStorePixelRatio||1),t!=this.options.width||e!=this.options.height||this.frame.style.width!=t||this.frame.style.height!=e?(this._getCameraState(r),this.frame.style.width=t,this.frame.style.height=e,this.frame.canvas.style.width="100%",this.frame.canvas.style.height="100%",this.frame.canvas.width=Math.round(this.frame.canvas.clientWidth*this.pixelRatio),this.frame.canvas.height=Math.round(this.frame.canvas.clientHeight*this.pixelRatio),this.options.width=t,this.options.height=e,i=!0):(this.frame.canvas.width==Math.round(this.frame.canvas.clientWidth*this.pixelRatio)&&this.frame.canvas.height==Math.round(this.frame.canvas.clientHeight*this.pixelRatio)||this._getCameraState(r),this.frame.canvas.width!=Math.round(this.frame.canvas.clientWidth*this.pixelRatio)&&(this.frame.canvas.width=Math.round(this.frame.canvas.clientWidth*this.pixelRatio),i=!0),this.frame.canvas.height!=Math.round(this.frame.canvas.clientHeight*this.pixelRatio)&&(this.frame.canvas.height=Math.round(this.frame.canvas.clientHeight*this.pixelRatio),i=!0)),i===!0&&(this.body.emitter.emit("resize",{width:Math.round(this.frame.canvas.width/this.pixelRatio),height:Math.round(this.frame.canvas.height/this.pixelRatio),oldWidth:Math.round(o/this.pixelRatio),oldHeight:Math.round(n/this.pixelRatio)}),this._setCameraState()),this.initialized=!0,i}},{key:"_XconvertDOMtoCanvas",value:function(t){return(t-this.body.view.translation.x)/this.body.view.scale}},{key:"_XconvertCanvasToDOM",value:function(t){return t*this.body.view.scale+this.body.view.translation.x}},{key:"_YconvertDOMtoCanvas",value:function(t){return(t-this.body.view.translation.y)/this.body.view.scale}},{key:"_YconvertCanvasToDOM",value:function(t){return t*this.body.view.scale+this.body.view.translation.y}},{key:"canvasToDOM",value:function(t){return{x:this._XconvertCanvasToDOM(t.x),y:this._YconvertCanvasToDOM(t.y)}}},{key:"DOMtoCanvas",value:function(t){return{x:this._XconvertDOMtoCanvas(t.x),y:this._YconvertDOMtoCanvas(t.y)}}}]),t}();e["default"]=h},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;i.5*this.body.nodeIndices.length)return void this.fit(t,!1);i=a["default"].getRange(this.body.nodes,t.nodes);var h=this.body.nodeIndices.length;o=12.662/(h+7.4147)+.0964822;var d=Math.min(this.canvas.frame.canvas.clientWidth/600,this.canvas.frame.canvas.clientHeight/600);o*=d}else{this.body.emitter.emit("_resizeNodes"),i=a["default"].getRange(this.body.nodes,t.nodes);var l=1.1*Math.abs(i.maxX-i.minX),c=1.1*Math.abs(i.maxY-i.minY),u=this.canvas.frame.canvas.clientWidth/l,p=this.canvas.frame.canvas.clientHeight/c;o=p>=u?u:p}o>1?o=1:0===o&&(o=1);var f=a["default"].findCenter(i),m={position:f,scale:o,animation:t.animation};this.moveTo(m)}},{key:"focus",value:function(t){var e=arguments.length<=1||void 0===arguments[1]?{}:arguments[1];if(void 0!==this.body.nodes[t]){var i={x:this.body.nodes[t].x,y:this.body.nodes[t].y};e.position=i,e.lockedOnNode=t,this.moveTo(e)}else console.log("Node: "+t+" cannot be found.")}},{key:"moveTo",value:function(t){return void 0===t?void(t={}):(void 0===t.offset&&(t.offset={x:0,y:0}),void 0===t.offset.x&&(t.offset.x=0),void 0===t.offset.y&&(t.offset.y=0),void 0===t.scale&&(t.scale=this.body.view.scale),void 0===t.position&&(t.position=this.getViewPosition()),void 0===t.animation&&(t.animation={duration:0}),t.animation===!1&&(t.animation={duration:0}),t.animation===!0&&(t.animation={}),void 0===t.animation.duration&&(t.animation.duration=1e3),void 0===t.animation.easingFunction&&(t.animation.easingFunction="easeInOutQuad"),void this.animateView(t))}},{key:"animateView",value:function(t){if(void 0!==t){this.animationEasingFunction=t.animation.easingFunction,this.releaseNode(),t.locked===!0&&(this.lockedOnNodeId=t.lockedOnNode,this.lockedOnNodeOffset=t.offset),0!=this.easingTime&&this._transitionRedraw(!0),this.sourceScale=this.body.view.scale,this.sourceTranslation=this.body.view.translation,this.targetScale=t.scale,this.body.view.scale=this.targetScale;var e=this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight}),i={x:e.x-t.position.x,y:e.y-t.position.y};this.targetTranslation={x:this.sourceTranslation.x+i.x*this.targetScale+t.offset.x,y:this.sourceTranslation.y+i.y*this.targetScale+t.offset.y},0===t.animation.duration?void 0!=this.lockedOnNodeId?(this.viewFunction=this._lockedRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction)):(this.body.view.scale=this.targetScale,this.body.view.translation=this.targetTranslation,this.body.emitter.emit("_requestRedraw")):(this.animationSpeed=1/(60*t.animation.duration*.001)||1/60,this.animationEasingFunction=t.animation.easingFunction,this.viewFunction=this._transitionRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction),this.body.emitter.emit("_startRendering"))}}},{key:"_lockedRedraw",value:function(){var t={x:this.body.nodes[this.lockedOnNodeId].x,y:this.body.nodes[this.lockedOnNodeId].y},e=this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight}),i={x:e.x-t.x,y:e.y-t.y},o=this.body.view.translation,n={x:o.x+i.x*this.body.view.scale+this.lockedOnNodeOffset.x,y:o.y+i.y*this.body.view.scale+this.lockedOnNodeOffset.y};this.body.view.translation=n}},{key:"releaseNode",value:function(){void 0!==this.lockedOnNodeId&&void 0!==this.viewFunction&&(this.body.emitter.off("initRedraw",this.viewFunction),this.lockedOnNodeId=void 0,this.lockedOnNodeOffset=void 0)}},{key:"_transitionRedraw",value:function(){var t=arguments.length<=0||void 0===arguments[0]?!1:arguments[0];this.easingTime+=this.animationSpeed,this.easingTime=t===!0?1:this.easingTime;var e=h.easingFunctions[this.animationEasingFunction](this.easingTime);this.body.view.scale=this.sourceScale+(this.targetScale-this.sourceScale)*e,this.body.view.translation={x:this.sourceTranslation.x+(this.targetTranslation.x-this.sourceTranslation.x)*e,y:this.sourceTranslation.y+(this.targetTranslation.y-this.sourceTranslation.y)*e},this.easingTime>=1&&(this.body.emitter.off("initRedraw",this.viewFunction),this.easingTime=0,void 0!=this.lockedOnNodeId&&(this.viewFunction=this._lockedRedraw.bind(this),this.body.emitter.on("initRedraw",this.viewFunction)),this.body.emitter.emit("animationFinished"))}},{key:"getScale",value:function(){return this.body.view.scale}},{key:"getViewPosition",value:function(){return this.canvas.DOMtoCanvas({x:.5*this.canvas.frame.canvas.clientWidth,y:.5*this.canvas.frame.canvas.clientHeight})}}]),t}();e["default"]=d},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){for(var i=0;i50&&(this.drag.pointer=this.getPointer(t.center),this.drag.pinched=!1,this.pinch.scale=this.body.view.scale,this.touchTime=(new Date).valueOf())}},{key:"onTap",value:function(t){var e=this.getPointer(t.center),i=this.selectionHandler.options.multiselect&&(t.changedPointers[0].ctrlKey||t.changedPointers[0].metaKey);this.checkSelectionChanges(e,t,i),this.selectionHandler._generateClickEvent("click",t,e)}},{key:"onDoubleTap",value:function(t){var e=this.getPointer(t.center);this.selectionHandler._generateClickEvent("doubleClick",t,e)}},{key:"onHold",value:function(t){var e=this.getPointer(t.center),i=this.selectionHandler.options.multiselect;this.checkSelectionChanges(e,t,i),this.selectionHandler._generateClickEvent("click",t,e),this.selectionHandler._generateClickEvent("hold",t,e)}},{key:"onRelease",value:function(t){if((new Date).valueOf()-this.touchTime>10){var e=this.getPointer(t.center);this.selectionHandler._generateClickEvent("release",t,e),this.touchTime=(new Date).valueOf()}}},{key:"onContext",value:function(t){var e=this.getPointer({x:t.clientX,y:t.clientY});this.selectionHandler._generateClickEvent("oncontext",t,e)}},{key:"checkSelectionChanges",value:function(t,e){var i=arguments.length<=2||void 0===arguments[2]?!1:arguments[2],o=this.selectionHandler._getSelectedEdgeCount(),n=this.selectionHandler._getSelectedNodeCount(),s=this.selectionHandler.getSelection(),r=void 0;r=i===!0?this.selectionHandler.selectAdditionalOnPoint(t):this.selectionHandler.selectOnPoint(t);var a=this.selectionHandler._getSelectedEdgeCount(),h=this.selectionHandler._getSelectedNodeCount(),d=this.selectionHandler.getSelection(),l=this._determineIfDifferent(s,d),c=l.nodesChanged,u=l.edgesChanged,p=!1;h-n>0?(this.selectionHandler._generateClickEvent("selectNode",e,t),r=!0,p=!0):c===!0&&h>0?(this.selectionHandler._generateClickEvent("deselectNode",e,t,s),this.selectionHandler._generateClickEvent("selectNode",e,t),p=!0,r=!0):0>h-n&&(this.selectionHandler._generateClickEvent("deselectNode",e,t,s),r=!0),a-o>0&&p===!1?(this.selectionHandler._generateClickEvent("selectEdge",e,t),r=!0):a>0&&u===!0?(this.selectionHandler._generateClickEvent("deselectEdge",e,t,s),this.selectionHandler._generateClickEvent("selectEdge",e,t),r=!0):0>a-o&&(this.selectionHandler._generateClickEvent("deselectEdge",e,t,s),r=!0),r===!0&&this.selectionHandler._generateClickEvent("select",e,t)}},{key:"_determineIfDifferent",value:function(t,e){for(var i=!1,o=!1,n=0;nt&&(t=1e-5),t>10&&(t=10);var o=void 0;void 0!==this.drag&&this.drag.dragging===!0&&(o=this.canvas.DOMtoCanvas(this.drag.pointer));var n=this.body.view.translation,s=t/i,r=(1-s)*e.x+n.x*s,a=(1-s)*e.y+n.y*s;if(this.body.view.scale=t,this.body.view.translation={x:r,y:a},void 0!=o){var h=this.canvas.canvasToDOM(o);this.drag.pointer.x=h.x,this.drag.pointer.y=h.y}this.body.emitter.emit("_requestRedraw"),t>i?this.body.emitter.emit("zoom",{direction:"+",scale:this.body.view.scale}):this.body.emitter.emit("zoom",{direction:"-",scale:this.body.view.scale})}}},{key:"onMouseWheel",value:function(t){if(this.options.zoomView===!0){var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3),0!==e){var i=this.body.view.scale,o=e/10;0>e&&(o/=1-o),i*=1+o;var n=this.getPointer({x:t.clientX,y:t.clientY});this.zoom(i,n)}t.preventDefault()}}},{key:"onMouseMove",value:function(t){var e=this,i=this.getPointer({x:t.clientX,y:t.clientY}),o=!1;if(void 0!==this.popup&&(this.popup.hidden===!1&&this._checkHidePopup(i),this.popup.hidden===!1&&(o=!0,this.popup.setPosition(i.x+3,i.y-5),this.popup.show())),this.options.keyboard.bindToWindow===!1&&this.options.keyboard.enabled===!0&&this.canvas.frame.focus(),o===!1&&(void 0!==this.popupTimer&&(clearInterval(this.popupTimer),this.popupTimer=void 0),this.drag.dragging||(this.popupTimer=setTimeout(function(){return e._checkShowPopup(i)},this.options.tooltipDelay))),this.options.hover===!0){var n=this.selectionHandler.getNodeAt(i);void 0===n&&(n=this.selectionHandler.getEdgeAt(i)),this.selectionHandler.hoverObject(n)}}},{key:"_checkShowPopup",value:function(t){var e=this.canvas._XconvertDOMtoCanvas(t.x),i=this.canvas._YconvertDOMtoCanvas(t.y),o={left:e,top:i,right:e,bottom:i},n=void 0===this.popupObj?void 0:this.popupObj.id,s=!1,r="node";if(void 0===this.popupObj){for(var a=this.body.nodeIndices,h=this.body.nodes,l=void 0,c=[],u=0;u0&&(this.popupObj=h[c[c.length-1]],s=!0)}if(void 0===this.popupObj&&s===!1){for(var p=this.body.edgeIndices,f=this.body.edges,m=void 0,v=[],g=0;g0&&(this.popupObj=f[v[v.length-1]],r="edge")}void 0!==this.popupObj?this.popupObj.id!==n&&(void 0===this.popup&&(this.popup=new d["default"](this.canvas.frame)),this.popup.popupTargetType=r,this.popup.popupTargetId=this.popupObj.id,this.popup.setPosition(t.x+3,t.y-5),this.popup.setText(this.popupObj.getTitle()),this.popup.show(),this.body.emitter.emit("showPopup",this.popupObj.id)):void 0!==this.popup&&(this.popup.hide(),this.body.emitter.emit("hidePopup"))}},{key:"_checkHidePopup",value:function(t){var e=this.selectionHandler._pointerToPositionObject(t),i=!1;if("node"===this.popup.popupTargetType){if(void 0!==this.body.nodes[this.popup.popupTargetId]&&(i=this.body.nodes[this.popup.popupTargetId].isOverlappingWith(e),i===!0)){var o=this.selectionHandler.getNodeAt(t);i=o.id===this.popup.popupTargetId}}else void 0===this.selectionHandler.getNodeAt(t)&&void 0!==this.body.edges[this.popup.popupTargetId]&&(i=this.body.edges[this.popup.popupTargetId].isOverlappingWith(e));i===!1&&(this.popupObj=void 0,this.popup.hide(),this.body.emitter.emit("hidePopup"))}}]),t}();e["default"]=c},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i700&&(this.body.emitter.emit("fit",{duration:700}),this.touchTime=(new Date).valueOf())}},{key:"_stopMovement",value:function(){for(var t in this.boundFunctions)this.boundFunctions.hasOwnProperty(t)&&(this.body.emitter.off("initRedraw",this.boundFunctions[t]),this.body.emitter.emit("_stopRendering"));this.boundFunctions={}}},{key:"_moveUp",value:function(){this.body.view.translation.y+=this.options.keyboard.speed.y}},{key:"_moveDown",value:function(){this.body.view.translation.y-=this.options.keyboard.speed.y}},{key:"_moveLeft",value:function(){this.body.view.translation.x+=this.options.keyboard.speed.x}},{key:"_moveRight",value:function(){this.body.view.translation.x-=this.options.keyboard.speed.x}},{key:"_zoomIn",value:function(){this.body.view.scale*=1+this.options.keyboard.speed.zoom,this.body.emitter.emit("zoom",{direction:"+",scale:this.body.view.scale})}},{key:"_zoomOut",value:function(){this.body.view.scale/=1+this.options.keyboard.speed.zoom,this.body.emitter.emit("zoom",{direction:"-",scale:this.body.view.scale})}},{key:"configureKeyboardBindings",value:function(){var t=this;void 0!==this.keycharm&&this.keycharm.destroy(),this.options.keyboard.enabled===!0&&(this.options.keyboard.bindToWindow===!0?this.keycharm=a({container:window,preventDefault:!0}):this.keycharm=a({container:this.canvas.frame,preventDefault:!0}),this.keycharm.reset(),this.activated===!0&&(this.keycharm.bind("up",function(){t.bindToRedraw("_moveUp")},"keydown"),this.keycharm.bind("down",function(){t.bindToRedraw("_moveDown")},"keydown"),this.keycharm.bind("left",function(){t.bindToRedraw("_moveLeft")},"keydown"),this.keycharm.bind("right",function(){t.bindToRedraw("_moveRight")},"keydown"),this.keycharm.bind("=",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("num+",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("num-",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("-",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("[",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("]",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("pageup",function(){t.bindToRedraw("_zoomIn")},"keydown"),this.keycharm.bind("pagedown",function(){t.bindToRedraw("_zoomOut")},"keydown"),this.keycharm.bind("up",function(){t.unbindFromRedraw("_moveUp")},"keyup"),this.keycharm.bind("down",function(){t.unbindFromRedraw("_moveDown")},"keyup"),this.keycharm.bind("left",function(){t.unbindFromRedraw("_moveLeft")},"keyup"),this.keycharm.bind("right",function(){t.unbindFromRedraw("_moveRight")},"keyup"),this.keycharm.bind("=",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("num+",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("num-",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("-",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("[",function(){t.unbindFromRedraw("_zoomOut")},"keyup"),this.keycharm.bind("]",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("pageup",function(){t.unbindFromRedraw("_zoomIn")},"keyup"),this.keycharm.bind("pagedown",function(){t.unbindFromRedraw("_zoomOut")},"keyup")))}}]),t}();e["default"]=h},function(t,e){function i(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var o=function(){function t(t,e){for(var i=0;io&&(s=o-e-this.padding),sn&&(r=n-i-this.padding),r0?e===!0?this.body.nodes[o[o.length-1]]:o[o.length-1]:void 0}},{key:"_getEdgesOverlappingWith",value:function(t,e){for(var i=this.body.edges,o=0;o0?e===!0?this.body.edges[o[o.length-1]]:o[o.length-1]:void 0}},{key:"_addToSelection",value:function(t){t instanceof a["default"]?this.selectionObj.nodes[t.id]=t:this.selectionObj.edges[t.id]=t}},{key:"_addToHover",value:function(t){t instanceof a["default"]?this.hoverObj.nodes[t.id]=t:this.hoverObj.edges[t.id]=t}},{key:"_removeFromSelection",value:function(t){t instanceof a["default"]?(delete this.selectionObj.nodes[t.id],this._unselectConnectedEdges(t)):delete this.selectionObj.edges[t.id]}},{key:"unselectAll",value:function(){for(var t in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].unselect();for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&this.selectionObj.edges[e].unselect();this.selectionObj={nodes:{},edges:{}}}},{key:"_getSelectedNodeCount",value:function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);return t}},{key:"_getSelectedNode",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return this.selectionObj.nodes[t]}},{key:"_getSelectedEdge",value:function(){for(var t in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(t))return this.selectionObj.edges[t]}},{key:"_getSelectedEdgeCount",value:function(){var t=0;for(var e in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(e)&&(t+=1);return t}},{key:"_getSelectedObjectCount",value:function(){var t=0;for(var e in this.selectionObj.nodes)this.selectionObj.nodes.hasOwnProperty(e)&&(t+=1);for(var i in this.selectionObj.edges)this.selectionObj.edges.hasOwnProperty(i)&&(t+=1);return t}},{key:"_selectionIsEmpty",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t))return!1;for(var e in this.selectionObj.edges)if(this.selectionObj.edges.hasOwnProperty(e))return!1;return!0}},{key:"_clusterInSelection",value:function(){for(var t in this.selectionObj.nodes)if(this.selectionObj.nodes.hasOwnProperty(t)&&this.selectionObj.nodes[t].clusterSize>1)return!0;return!1}},{key:"_selectConnectedEdges",value:function(t){for(var e=0;e0&&(this.options.hierarchical.levelSeparation*=-1):this.options.hierarchical.levelSeparation<0&&(this.options.hierarchical.levelSeparation*=-1),this.body.emitter.emit("_resetHierarchicalLayout"),this.adaptAllOptionsForHierarchicalLayout(e);if(i===!0)return this.body.emitter.emit("refresh"),l.deepExtend(e,this.optionsBackup)}return e}},{key:"adaptAllOptionsForHierarchicalLayout",value:function(t){if(this.options.hierarchical.enabled===!0){void 0===t.physics||t.physics===!0?(t.physics={enabled:void 0===this.optionsBackup.physics.enabled?!0:this.optionsBackup.physics.enabled,solver:"hierarchicalRepulsion"},this.optionsBackup.physics.enabled=void 0===this.optionsBackup.physics.enabled?!0:this.optionsBackup.physics.enabled,this.optionsBackup.physics.solver=this.optionsBackup.physics.solver||"barnesHut"):"object"===r(t.physics)?(this.optionsBackup.physics.enabled=void 0===t.physics.enabled?!0:t.physics.enabled,this.optionsBackup.physics.solver=t.physics.solver||"barnesHut",t.physics.solver="hierarchicalRepulsion"):t.physics!==!1&&(this.optionsBackup.physics.solver="barnesHut",t.physics={solver:"hierarchicalRepulsion"});var e="horizontal";"RL"!==this.options.hierarchical.direction&&"LR"!==this.options.hierarchical.direction||(e="vertical"),void 0===t.edges?(this.optionsBackup.edges={smooth:{enabled:!0,type:"dynamic"}},t.edges={smooth:!1}):void 0===t.edges.smooth?(this.optionsBackup.edges={smooth:{enabled:!0,type:"dynamic"}},t.edges.smooth=!1):"boolean"==typeof t.edges.smooth?(this.optionsBackup.edges={smooth:t.edges.smooth},t.edges.smooth={enabled:t.edges.smooth,type:e}):(void 0!==t.edges.smooth.type&&"dynamic"!==t.edges.smooth.type&&(e=t.edges.smooth.type),this.optionsBackup.edges={smooth:void 0===t.edges.smooth.enabled?!0:t.edges.smooth.enabled,type:void 0===t.edges.smooth.type?"dynamic":t.edges.smooth.type,roundness:void 0===t.edges.smooth.roundness?.5:t.edges.smooth.roundness,forceDirection:void 0===t.edges.smooth.forceDirection?!1:t.edges.smooth.forceDirection},t.edges.smooth={enabled:void 0===t.edges.smooth.enabled?!0:t.edges.smooth.enabled,type:e,roundness:void 0===t.edges.smooth.roundness?.5:t.edges.smooth.roundness,forceDirection:void 0===t.edges.smooth.forceDirection?!1:t.edges.smooth.forceDirection}),this.body.emitter.emit("_forceDisableDynamicCurves",e)}return t}},{key:"seededRandom",value:function(){var t=1e4*Math.sin(this.randomSeed++);return t-Math.floor(t)}},{key:"positionInitially",value:function(t){if(this.options.hierarchical.enabled!==!0){this.randomSeed=this.initialRandomSeed;for(var e=0;es){for(var r=this.body.nodeIndices.length;this.body.nodeIndices.length>s;){n+=1;var a=this.body.nodeIndices.length;n%3===0?this.body.modules.clustering.clusterBridges():this.body.modules.clustering.clusterOutliers();var h=this.body.nodeIndices.length;if(a==h&&n%3!==0||n>o)return this._declusterAll(),this.body.emitter.emit("_layoutFailed"),void console.info("This network could not be positioned by this version of the improved layout algorithm. Please disable improvedLayout for better performance.")}this.body.modules.kamadaKawai.setOptions({springLength:Math.max(150,2*r)})}this.body.modules.kamadaKawai.solve(this.body.nodeIndices,this.body.edgeIndices,!0),this._shiftToCenter();for(var d=70,l=0;l0){var t=void 0,e=void 0,i=!1,o=!0,n=!1;this.hierarchicalLevels={},this.lastNodeOnLevel={},this.hierarchicalChildrenReference={},this.hierarchicalParentReference={},this.hierarchicalTrees={},this.treeIndex=-1,this.distributionOrdering={},this.distributionIndex={},this.distributionOrderingPresence={};for(e in this.body.nodes)this.body.nodes.hasOwnProperty(e)&&(t=this.body.nodes[e],void 0===t.options.x&&void 0===t.options.y&&(o=!1),void 0!==t.options.level?(i=!0,this.hierarchicalLevels[e]=t.options.level):n=!0);if(n===!0&&i===!0)throw new Error("To use the hierarchical layout, nodes require either no predefined levels or levels have to be defined for all nodes.");n===!0&&("hubsize"===this.options.hierarchical.sortMethod?this._determineLevelsByHubsize():"directed"===this.options.hierarchical.sortMethod?this._determineLevelsDirected():"custom"===this.options.hierarchical.sortMethod&&this._determineLevelsCustomCallback());for(var s in this.body.nodes)this.body.nodes.hasOwnProperty(s)&&void 0===this.hierarchicalLevels[s]&&(this.hierarchicalLevels[s]=0);var r=this._getDistribution();this._generateMap(),this._placeNodesByHierarchy(r),this._condenseHierarchy(),this._shiftToCenter()}}},{key:"_condenseHierarchy",value:function(){var t=this,e=!1,i={},o=function(){for(var e=a(),i=0;i0)for(var n=0;n=l&&(r=Math.min(c,r),a=Math.max(c,a))}return[r,a,o,n]},l=function _(e){var i=t.hierarchicalLevels[e];if(t.hierarchicalChildrenReference[e]){var o=t.hierarchicalChildrenReference[e];if(o.length>0)for(var n=0;n1)for(var a=0;at.options.hierarchical.nodeSpacing){var u={};u[i.id]=!0;var p={};p[o.id]=!0,h(i,u),h(o,p);var f=c(i,o),m=d(u,f),v=s(m,4),g=(v[0],v[1]),y=(v[2],v[3],d(p,f)),b=s(y,4),w=b[0],_=(b[1],b[2]),x=(b[3],Math.abs(g-w));if(x>t.options.hierarchical.nodeSpacing){var k=g-w+t.options.hierarchical.nodeSpacing;k<-_+t.options.hierarchical.nodeSpacing&&(k=-_+t.options.hierarchical.nodeSpacing),0>k&&(t._shiftBlock(o.id,k),e=!0,n===!0&&t._centerParent(o))}}},m=function(o,n){for(var r=n.id,a=n.edges,l=t.hierarchicalLevels[n.id],c=t.options.hierarchical.levelSeparation*t.options.hierarchical.levelSeparation,u={},p=[],f=0;fr;r++){var a=g(o,i),h=y(o,i),d=40,l=Math.max(-d,Math.min(d,Math.round(a/h)));if(o-=l,void 0!==s[o])break;s[o]=r}return o},w=function(o){var r=t._getPositionForHierarchy(n);if(void 0===i[n.id]){var a={};a[n.id]=!0,h(n,a),i[n.id]=a}var l=d(i[n.id]),c=s(l,4),u=(c[0],c[1],c[2]),p=c[3],f=o-r,m=0;f>0?m=Math.min(f,p-t.options.hierarchical.nodeSpacing):0>f&&(m=-Math.min(-f,u-t.options.hierarchical.nodeSpacing)),0!=m&&(t._shiftBlock(n.id,m),e=!0)},_=function(i){var o=t._getPositionForHierarchy(n),r=t._getSpaceAroundNode(n),a=s(r,2),h=a[0],d=a[1],l=i-o,c=o;l>0?c=Math.min(o+(d-t.options.hierarchical.nodeSpacing),i):0>l&&(c=Math.max(o-(h-t.options.hierarchical.nodeSpacing),i)),c!==o&&(t._setPositionForHierarchy(n,c,void 0,!0),e=!0)},x=b(o,p);w(x),x=b(o,a),_(x)},v=function(i){var o=Object.keys(t.distributionOrdering);o=o.reverse();for(var n=0;i>n;n++){e=!1;for(var s=0;sn&&(e=!1,p(f,o,!0),e===!0);n++);},y=function(){for(var e in t.body.nodes)t.body.nodes.hasOwnProperty(e)&&t._centerParent(t.body.nodes[e])},b=function(){var e=Object.keys(t.distributionOrdering);e=e.reverse();for(var i=0;i0)for(var d=0;dg&&Math.abs(g)0&&Math.abs(g)0&&(r=this._getPositionForHierarchy(i[n-1])+this.options.hierarchical.nodeSpacing),this._setPositionForHierarchy(s,r,e),this._validataPositionAndContinue(s,e,r),o++}}}}},{key:"_placeBranchNodes",value:function(t,e){if(void 0!==this.hierarchicalChildrenReference[t]){for(var i=[],o=0;oe&&void 0===this.positionedNodes[s.id]))return;var a=void 0;a=0===n?this._getPositionForHierarchy(this.body.nodes[t]):this._getPositionForHierarchy(i[n-1])+this.options.hierarchical.nodeSpacing,this._setPositionForHierarchy(s,a,r),this._validataPositionAndContinue(s,r,a)}for(var h=1e9,d=-1e9,l=0;l0&&(e=this._getHubSize(),0!==e);)for(var o in this.body.nodes)if(this.body.nodes.hasOwnProperty(o)){var n=this.body.nodes[o];n.edges.length===e&&this._crawlNetwork(i,o)}}},{key:"_determineLevelsCustomCallback",value:function(){var t=this,e=1e5,i=function(t,e,i){},o=function(o,n,s){var r=t.hierarchicalLevels[o.id];void 0===r&&(t.hierarchicalLevels[o.id]=e);var a=i(d["default"].cloneOptions(o,"node"),d["default"].cloneOptions(n,"node"),d["default"].cloneOptions(s,"edge"));t.hierarchicalLevels[n.id]=t.hierarchicalLevels[o.id]+a};this._crawlNetwork(o),this._setMinLevelToZero()}},{key:"_determineLevelsDirected",value:function(){var t=this,e=1e4,i=function(i,o,n){var s=t.hierarchicalLevels[i.id];void 0===s&&(t.hierarchicalLevels[i.id]=e),n.toId==o.id?t.hierarchicalLevels[o.id]=t.hierarchicalLevels[i.id]+1:t.hierarchicalLevels[o.id]=t.hierarchicalLevels[i.id]-1};this._crawlNetwork(i),this._setMinLevelToZero()}},{key:"_setMinLevelToZero",value:function(){var t=1e9;for(var e in this.body.nodes)this.body.nodes.hasOwnProperty(e)&&void 0!==this.hierarchicalLevels[e]&&(t=Math.min(this.hierarchicalLevels[e],t));for(var i in this.body.nodes)this.body.nodes.hasOwnProperty(i)&&void 0!==this.hierarchicalLevels[i]&&(this.hierarchicalLevels[i]-=t)}},{key:"_generateMap",value:function(){var t=this,e=function(e,i){if(t.hierarchicalLevels[i.id]>t.hierarchicalLevels[e.id]){var o=e.id,n=i.id;void 0===t.hierarchicalChildrenReference[o]&&(t.hierarchicalChildrenReference[o]=[]),t.hierarchicalChildrenReference[o].push(n),void 0===t.hierarchicalParentReference[n]&&(t.hierarchicalParentReference[n]=[]),t.hierarchicalParentReference[n].push(o)}};this._crawlNetwork(e)}},{key:"_crawlNetwork",value:function(){var t=this,e=arguments.length<=0||void 0===arguments[0]?function(){}:arguments[0],i=arguments[1],o={},n=0,s=function d(i,n){if(void 0===o[i.id]){void 0===t.hierarchicalTrees[i.id]&&(t.hierarchicalTrees[i.id]=n,t.treeIndex=Math.max(n,t.treeIndex)),o[i.id]=!0;for(var s=void 0,r=0;r1&&("UD"===this.options.hierarchical.direction||"DU"===this.options.hierarchical.direction?t.sort(function(t,e){return t.x-e.x}):t.sort(function(t,e){return t.y-e.y}))}}]),t}();e["default"]=c},function(t,e,i){function o(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var n=function(){function t(t,e){for(var i=0;i0&&this.options.deleteNode!==!1?(n===!0&&this._createSeperator(4),this._createDeleteButton(o)):0===t&&this.options.deleteEdge!==!1&&(n===!0&&this._createSeperator(4),this._createDeleteButton(o))),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this)),this._temporaryBindEvent("select",this.showManipulatorToolbar.bind(this))}this.body.emitter.emit("_redraw")}},{key:"addNodeMode",value:function(){if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="addNode",this.guiEnabled===!0){var t=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(t),this._createSeperator(),this._createDescription(t.addDescription||this.options.locales.en.addDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this._temporaryBindEvent("click",this._performAddNode.bind(this))}},{key:"editNode",value:function(){var t=this;this.editMode!==!0&&this.enableEditMode(),this._clean();var e=this.selectionHandler._getSelectedNode();if(void 0!==e){if(this.inMode="editNode","function"!=typeof this.options.editNode)throw new Error("No function has been configured to handle the editing of nodes.");if(e.isCluster!==!0){var i=s.deepExtend({},e.options,!1); +if(i.x=e.x,i.y=e.y,2!==this.options.editNode.length)throw new Error("The function for edit does not support two arguments (data, callback)");this.options.editNode(i,function(e){null!==e&&void 0!==e&&"editNode"===t.inMode&&t.body.data.nodes.getDataSet().update(e),t.showManipulatorToolbar()})}else alert(this.options.locales[this.options.locale].editClusterError||this.options.locales.en.editClusterError)}else this.showManipulatorToolbar()}},{key:"addEdgeMode",value:function(){if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="addEdge",this.guiEnabled===!0){var t=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(t),this._createSeperator(),this._createDescription(t.edgeDescription||this.options.locales.en.edgeDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this._temporaryBindUI("onTouch",this._handleConnect.bind(this)),this._temporaryBindUI("onDragEnd",this._finishConnect.bind(this)),this._temporaryBindUI("onDrag",this._dragControlNode.bind(this)),this._temporaryBindUI("onRelease",this._finishConnect.bind(this)),this._temporaryBindUI("onDragStart",function(){}),this._temporaryBindUI("onHold",function(){})}},{key:"editEdgeMode",value:function(){var t=this;if(this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="editEdge",this.guiEnabled===!0){var e=this.options.locales[this.options.locale];this.manipulationDOM={},this._createBackButton(e),this._createSeperator(),this._createDescription(e.editEdgeDescription||this.options.locales.en.editEdgeDescription),this._bindHammerToDiv(this.closeDiv,this.toggleEditMode.bind(this))}this.edgeBeingEditedId=this.selectionHandler.getSelectedEdges()[0],void 0!==this.edgeBeingEditedId?!function(){var e=t.body.edges[t.edgeBeingEditedId],i=t._getNewTargetNode(e.from.x,e.from.y),o=t._getNewTargetNode(e.to.x,e.to.y);t.temporaryIds.nodes.push(i.id),t.temporaryIds.nodes.push(o.id),t.body.nodes[i.id]=i,t.body.nodeIndices.push(i.id),t.body.nodes[o.id]=o,t.body.nodeIndices.push(o.id),t._temporaryBindUI("onTouch",t._controlNodeTouch.bind(t)),t._temporaryBindUI("onTap",function(){}),t._temporaryBindUI("onHold",function(){}),t._temporaryBindUI("onDragStart",t._controlNodeDragStart.bind(t)),t._temporaryBindUI("onDrag",t._controlNodeDrag.bind(t)),t._temporaryBindUI("onDragEnd",t._controlNodeDragEnd.bind(t)),t._temporaryBindUI("onMouseMove",function(){}),t._temporaryBindEvent("beforeDrawing",function(t){var n=e.edgeType.findBorderPositions(t);i.selected===!1&&(i.x=n.from.x,i.y=n.from.y),o.selected===!1&&(o.x=n.to.x,o.y=n.to.y)}),t.body.emitter.emit("_redraw")}():this.showManipulatorToolbar()}},{key:"deleteSelected",value:function(){var t=this;this.editMode!==!0&&this.enableEditMode(),this._clean(),this.inMode="delete";var e=this.selectionHandler.getSelectedNodes(),i=this.selectionHandler.getSelectedEdges(),o=void 0;if(e.length>0){for(var n=0;n0&&"function"==typeof this.options.deleteEdge&&(o=this.options.deleteEdge);if("function"==typeof o){var s={nodes:e,edges:i};if(2!==o.length)throw new Error("The function for delete does not support two arguments (data, callback)");o(s,function(e){null!==e&&void 0!==e&&"delete"===t.inMode?(t.body.data.edges.getDataSet().remove(e.edges),t.body.data.nodes.getDataSet().remove(e.nodes),t.body.emitter.emit("startSimulation"),t.showManipulatorToolbar()):(t.body.emitter.emit("startSimulation"),t.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().remove(i),this.body.data.nodes.getDataSet().remove(e),this.body.emitter.emit("startSimulation"),this.showManipulatorToolbar()}},{key:"_setup",value:function(){this.options.enabled===!0?(this.guiEnabled=!0,this._createWrappers(),this.editMode===!1?this._createEditButton():this.showManipulatorToolbar()):(this._removeManipulationDOM(),this.guiEnabled=!1)}},{key:"_createWrappers",value:function(){void 0===this.manipulationDiv&&(this.manipulationDiv=document.createElement("div"),this.manipulationDiv.className="vis-manipulation",this.editMode===!0?this.manipulationDiv.style.display="block":this.manipulationDiv.style.display="none",this.canvas.frame.appendChild(this.manipulationDiv)),void 0===this.editModeDiv&&(this.editModeDiv=document.createElement("div"),this.editModeDiv.className="vis-edit-mode",this.editMode===!0?this.editModeDiv.style.display="none":this.editModeDiv.style.display="block",this.canvas.frame.appendChild(this.editModeDiv)),void 0===this.closeDiv&&(this.closeDiv=document.createElement("div"),this.closeDiv.className="vis-close",this.closeDiv.style.display=this.manipulationDiv.style.display,this.canvas.frame.appendChild(this.closeDiv))}},{key:"_getNewTargetNode",value:function(t,e){var i=s.deepExtend({},this.options.controlNodeStyle);i.id="targetNode"+s.randomUUID(),i.hidden=!1,i.physics=!1,i.x=t,i.y=e;var o=this.body.functions.createNode(i);return o.shape.boundingBox={left:t,right:t,top:e,bottom:e},o}},{key:"_createEditButton",value:function(){this._clean(),this.manipulationDOM={},s.recursiveDOMDelete(this.editModeDiv);var t=this.options.locales[this.options.locale],e=this._createButton("editMode","vis-button vis-edit vis-edit-mode",t.edit||this.options.locales.en.edit);this.editModeDiv.appendChild(e),this._bindHammerToDiv(e,this.toggleEditMode.bind(this))}},{key:"_clean",value:function(){this.inMode=!1,this.guiEnabled===!0&&(s.recursiveDOMDelete(this.editModeDiv),s.recursiveDOMDelete(this.manipulationDiv),this._cleanManipulatorHammers()),this._cleanupTemporaryNodesAndEdges(),this._unbindTemporaryUIs(),this._unbindTemporaryEvents(),this.body.emitter.emit("restorePhysics")}},{key:"_cleanManipulatorHammers",value:function(){if(0!=this.manipulationHammers.length){for(var t=0;t=0;r--)if(n[r]!==this.selectedControlNode.id){s=this.body.nodes[n[r]];break}if(void 0!==s&&void 0!==this.selectedControlNode)if(s.isCluster===!0)alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError);else{var a=this.body.nodes[this.temporaryIds.nodes[0]];this.selectedControlNode.id===a.id?this._performEditEdge(s.id,o.to.id):this._performEditEdge(o.from.id,s.id)}else o.updateEdgeType(),this.body.emitter.emit("restorePhysics");this.body.emitter.emit("_redraw")}}},{key:"_handleConnect",value:function(t){if((new Date).valueOf()-this.touchTime>100){this.lastTouch=this.body.functions.getPointer(t.center),this.lastTouch.translation=s.extend({},this.body.view.translation);var e=this.lastTouch,i=this.selectionHandler.getNodeAt(e);if(void 0!==i)if(i.isCluster===!0)alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError);else{var o=this._getNewTargetNode(i.x,i.y);this.body.nodes[o.id]=o,this.body.nodeIndices.push(o.id);var n=this.body.functions.createEdge({id:"connectionEdge"+s.randomUUID(),from:i.id,to:o.id,physics:!1,smooth:{enabled:!0,type:"continuous",roundness:.5}});this.body.edges[n.id]=n,this.body.edgeIndices.push(n.id),this.temporaryIds.nodes.push(o.id),this.temporaryIds.edges.push(n.id)}this.touchTime=(new Date).valueOf()}}},{key:"_dragControlNode",value:function(t){var e=this.body.functions.getPointer(t.center);if(void 0!==this.temporaryIds.nodes[0]){var i=this.body.nodes[this.temporaryIds.nodes[0]];i.x=this.canvas._XconvertDOMtoCanvas(e.x),i.y=this.canvas._YconvertDOMtoCanvas(e.y),this.body.emitter.emit("_redraw")}else{var o=e.x-this.lastTouch.x,n=e.y-this.lastTouch.y;this.body.view.translation={x:this.lastTouch.translation.x+o,y:this.lastTouch.translation.y+n}}}},{key:"_finishConnect",value:function(t){var e=this.body.functions.getPointer(t.center),i=this.selectionHandler._pointerToPositionObject(e),o=void 0;void 0!==this.temporaryIds.edges[0]&&(o=this.body.edges[this.temporaryIds.edges[0]].fromId);for(var n=this.selectionHandler._getAllNodesOverlappingWith(i),s=void 0,r=n.length-1;r>=0;r--)if(-1===this.temporaryIds.nodes.indexOf(n[r])){s=this.body.nodes[n[r]];break}this._cleanupTemporaryNodesAndEdges(),void 0!==s&&(s.isCluster===!0?alert(this.options.locales[this.options.locale].createEdgeError||this.options.locales.en.createEdgeError):void 0!==this.body.nodes[o]&&void 0!==this.body.nodes[s.id]&&this._performAddEdge(o,s.id)),this.body.emitter.emit("_redraw")}},{key:"_performAddNode",value:function(t){var e=this,i={id:s.randomUUID(),x:t.pointer.canvas.x,y:t.pointer.canvas.y,label:"new"};if("function"==typeof this.options.addNode){if(2!==this.options.addNode.length)throw new Error("The function for add does not support two arguments (data,callback)");this.options.addNode(i,function(t){null!==t&&void 0!==t&&"addNode"===e.inMode&&(e.body.data.nodes.getDataSet().add(t),e.showManipulatorToolbar())})}else this.body.data.nodes.getDataSet().add(i),this.showManipulatorToolbar()}},{key:"_performAddEdge",value:function(t,e){var i=this,o={from:t,to:e};if("function"==typeof this.options.addEdge){if(2!==this.options.addEdge.length)throw new Error("The function for connect does not support two arguments (data,callback)");this.options.addEdge(o,function(t){null!==t&&void 0!==t&&"addEdge"===i.inMode&&(i.body.data.edges.getDataSet().add(t),i.selectionHandler.unselectAll(),i.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().add(o),this.selectionHandler.unselectAll(),this.showManipulatorToolbar()}},{key:"_performEditEdge",value:function(t,e){var i=this,o={id:this.edgeBeingEditedId,from:t,to:e};if("function"==typeof this.options.editEdge){if(2!==this.options.editEdge.length)throw new Error("The function for edit does not support two arguments (data, callback)");this.options.editEdge(o,function(t){null===t||void 0===t||"editEdge"!==i.inMode?(i.body.edges[o.id].updateEdgeType(),i.body.emitter.emit("_redraw")):(i.body.data.edges.getDataSet().update(t),i.selectionHandler.unselectAll(),i.showManipulatorToolbar())})}else this.body.data.edges.getDataSet().update(o),this.selectionHandler.unselectAll(),this.showManipulatorToolbar()}}]),t}();e["default"]=h},function(t,e){Object.defineProperty(e,"__esModule",{value:!0});var i="string",o="boolean",n="number",s="array",r="object",a="dom",h="any",d={configure:{enabled:{"boolean":o},filter:{"boolean":o,string:i,array:s,"function":"function"},container:{dom:a},showButton:{"boolean":o},__type__:{object:r,"boolean":o,string:i,array:s,"function":"function"}},edges:{arrows:{to:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},middle:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},from:{enabled:{"boolean":o},scaleFactor:{number:n},__type__:{object:r,"boolean":o}},__type__:{string:["from","to","middle"],object:r}},arrowStrikethrough:{"boolean":o},color:{color:{string:i},highlight:{string:i},hover:{string:i},inherit:{string:["from","to","both"],"boolean":o},opacity:{number:n},__type__:{object:r,string:i}},dashes:{"boolean":o,array:s},font:{color:{string:i},size:{number:n},face:{string:i},background:{string:i},strokeWidth:{number:n},strokeColor:{string:i},align:{string:["horizontal","top","middle","bottom"]},__type__:{object:r,string:i}},hidden:{"boolean":o},hoverWidth:{"function":"function",number:n},label:{string:i,undefined:"undefined"},labelHighlightBold:{"boolean":o},length:{number:n,undefined:"undefined"},physics:{"boolean":o},scaling:{min:{number:n},max:{number:n},label:{enabled:{"boolean":o},min:{number:n},max:{number:n},maxVisible:{number:n},drawThreshold:{number:n},__type__:{object:r,"boolean":o}},customScalingFunction:{"function":"function"},__type__:{object:r}},selectionWidth:{"function":"function",number:n},selfReferenceSize:{number:n},shadow:{enabled:{"boolean":o},color:{string:i},size:{number:n},x:{number:n},y:{number:n},__type__:{object:r,"boolean":o}},smooth:{enabled:{"boolean":o},type:{string:["dynamic","continuous","discrete","diagonalCross","straightCross","horizontal","vertical","curvedCW","curvedCCW","cubicBezier"]},roundness:{number:n},forceDirection:{string:["horizontal","vertical","none"],"boolean":o},__type__:{object:r,"boolean":o}},title:{string:i,undefined:"undefined"},width:{number:n},value:{number:n,undefined:"undefined"},__type__:{object:r}},groups:{useDefaultGroups:{"boolean":o},__any__:"get from nodes, will be overwritten below",__type__:{object:r}},interaction:{dragNodes:{"boolean":o},dragView:{"boolean":o},hideEdgesOnDrag:{"boolean":o},hideNodesOnDrag:{"boolean":o},hover:{"boolean":o},keyboard:{enabled:{"boolean":o},speed:{x:{number:n},y:{number:n},zoom:{number:n},__type__:{object:r}},bindToWindow:{"boolean":o},__type__:{object:r,"boolean":o}},multiselect:{"boolean":o},navigationButtons:{"boolean":o},selectable:{"boolean":o},selectConnectedEdges:{"boolean":o},hoverConnectedEdges:{"boolean":o},tooltipDelay:{number:n},zoomView:{"boolean":o},__type__:{object:r}},layout:{randomSeed:{undefined:"undefined",number:n},improvedLayout:{"boolean":o},hierarchical:{enabled:{"boolean":o},levelSeparation:{number:n},nodeSpacing:{number:n},treeSpacing:{number:n},blockShifting:{"boolean":o},edgeMinimization:{"boolean":o},parentCentralization:{"boolean":o},direction:{string:["UD","DU","LR","RL"]},sortMethod:{string:["hubsize","directed"]},__type__:{object:r,"boolean":o}},__type__:{object:r}},manipulation:{enabled:{"boolean":o},initiallyActive:{"boolean":o},addNode:{"boolean":o,"function":"function"},addEdge:{"boolean":o,"function":"function"},editNode:{"function":"function"},editEdge:{"boolean":o,"function":"function"},deleteNode:{"boolean":o,"function":"function"},deleteEdge:{"boolean":o,"function":"function"},controlNodeStyle:"get from nodes, will be overwritten below",__type__:{object:r,"boolean":o}},nodes:{borderWidth:{number:n},borderWidthSelected:{number:n,undefined:"undefined"},brokenImage:{string:i,undefined:"undefined"},color:{border:{string:i},background:{string:i},highlight:{border:{string:i},background:{string:i},__type__:{object:r,string:i}},hover:{border:{string:i},background:{string:i},__type__:{object:r,string:i}},__type__:{object:r,string:i}},fixed:{x:{"boolean":o},y:{"boolean":o},__type__:{object:r,"boolean":o}},font:{align:{string:i},color:{string:i},size:{number:n},face:{string:i},background:{string:i},strokeWidth:{number:n},strokeColor:{string:i},__type__:{object:r,string:i}},group:{string:i,number:n,undefined:"undefined"},hidden:{"boolean":o},icon:{face:{string:i},code:{string:i},size:{number:n},color:{string:i},__type__:{object:r}},id:{string:i,number:n},image:{string:i,undefined:"undefined"},label:{string:i,undefined:"undefined"},labelHighlightBold:{"boolean":o},level:{number:n,undefined:"undefined"},mass:{number:n},physics:{"boolean":o},scaling:{min:{number:n},max:{number:n},label:{enabled:{"boolean":o},min:{number:n},max:{number:n},maxVisible:{number:n},drawThreshold:{number:n},__type__:{object:r,"boolean":o}},customScalingFunction:{"function":"function"},__type__:{object:r}},shadow:{enabled:{"boolean":o},color:{string:i},size:{number:n},x:{number:n},y:{number:n},__type__:{object:r,"boolean":o}},shape:{string:["ellipse","circle","database","box","text","image","circularImage","diamond","dot","star","triangle","triangleDown","square","icon"]},shapeProperties:{borderDashes:{"boolean":o,array:s},borderRadius:{number:n},interpolation:{"boolean":o},useImageSize:{"boolean":o},useBorderWithImage:{"boolean":o},__type__:{object:r}},size:{number:n},title:{string:i,undefined:"undefined"},value:{number:n,undefined:"undefined"},x:{number:n},y:{number:n},__type__:{object:r}},physics:{enabled:{"boolean":o},barnesHut:{gravitationalConstant:{number:n},centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},damping:{number:n},avoidOverlap:{number:n},__type__:{object:r}},forceAtlas2Based:{gravitationalConstant:{number:n},centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},damping:{number:n},avoidOverlap:{number:n},__type__:{object:r}},repulsion:{centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},nodeDistance:{number:n},damping:{number:n},__type__:{object:r}},hierarchicalRepulsion:{centralGravity:{number:n},springLength:{number:n},springConstant:{number:n},nodeDistance:{number:n},damping:{number:n},__type__:{object:r}},maxVelocity:{number:n},minVelocity:{number:n},solver:{string:["barnesHut","repulsion","hierarchicalRepulsion","forceAtlas2Based"]},stabilization:{enabled:{"boolean":o},iterations:{number:n},updateInterval:{number:n},onlyDynamicEdges:{"boolean":o},fit:{"boolean":o},__type__:{object:r,"boolean":o}},timestep:{number:n},adaptiveTimestep:{"boolean":o},__type__:{object:r,"boolean":o}},autoResize:{"boolean":o},clickToUse:{"boolean":o},locale:{string:i},locales:{__any__:{any:h},__type__:{object:r}},height:{string:i},width:{string:i},__type__:{object:r}};d.groups.__any__=d.nodes,d.manipulation.controlNodeStyle=d.nodes;var l={nodes:{borderWidth:[1,0,10,1],borderWidthSelected:[2,0,10,1],color:{border:["color","#2B7CE9"],background:["color","#97C2FC"],highlight:{border:["color","#2B7CE9"],background:["color","#D2E5FF"]},hover:{border:["color","#2B7CE9"],background:["color","#D2E5FF"]}},fixed:{x:!1,y:!1},font:{color:["color","#343434"],size:[14,0,100,1],face:["arial","verdana","tahoma"],background:["color","none"],strokeWidth:[0,0,50,1],strokeColor:["color","#ffffff"]},hidden:!1,labelHighlightBold:!0,physics:!0,scaling:{min:[10,0,200,1],max:[30,0,200,1],label:{enabled:!1,min:[14,0,200,1],max:[30,0,200,1],maxVisible:[30,0,200,1],drawThreshold:[5,0,20,1]}},shadow:{enabled:!1,color:"rgba(0,0,0,0.5)",size:[10,0,20,1],x:[5,-30,30,1],y:[5,-30,30,1]},shape:["ellipse","box","circle","database","diamond","dot","square","star","text","triangle","triangleDown"],shapeProperties:{borderDashes:!1,borderRadius:[6,0,20,1],interpolation:!0,useImageSize:!1},size:[25,0,200,1]},edges:{arrows:{to:{enabled:!1,scaleFactor:[1,0,3,.05]},middle:{enabled:!1,scaleFactor:[1,0,3,.05]},from:{enabled:!1,scaleFactor:[1,0,3,.05]}},arrowStrikethrough:!0,color:{color:["color","#848484"],highlight:["color","#848484"],hover:["color","#848484"],inherit:["from","to","both",!0,!1],opacity:[1,0,1,.05]},dashes:!1,font:{color:["color","#343434"],size:[14,0,100,1],face:["arial","verdana","tahoma"],background:["color","none"],strokeWidth:[2,0,50,1],strokeColor:["color","#ffffff"],align:["horizontal","top","middle","bottom"]},hidden:!1,hoverWidth:[1.5,0,5,.1],labelHighlightBold:!0,physics:!0,scaling:{min:[1,0,100,1],max:[15,0,100,1],label:{enabled:!0,min:[14,0,200,1],max:[30,0,200,1],maxVisible:[30,0,200,1],drawThreshold:[5,0,20,1]}},selectionWidth:[1.5,0,5,.1],selfReferenceSize:[20,0,200,1],shadow:{enabled:!1,color:"rgba(0,0,0,0.5)",size:[10,0,20,1],x:[5,-30,30,1],y:[5,-30,30,1]},smooth:{enabled:!0,type:["dynamic","continuous","discrete","diagonalCross","straightCross","horizontal","vertical","curvedCW","curvedCCW","cubicBezier"],forceDirection:["horizontal","vertical","none"],roundness:[.5,0,1,.05]},width:[1,0,30,1]},layout:{hierarchical:{enabled:!1,levelSeparation:[150,20,500,5],nodeSpacing:[100,20,500,5],treeSpacing:[200,20,500,5],blockShifting:!0,edgeMinimization:!0,parentCentralization:!0,direction:["UD","DU","LR","RL"],sortMethod:["hubsize","directed"]}},interaction:{dragNodes:!0,dragView:!0,hideEdgesOnDrag:!1,hideNodesOnDrag:!1,hover:!1,keyboard:{enabled:!1,speed:{x:[10,0,40,1],y:[10,0,40,1],zoom:[.02,0,.1,.005]},bindToWindow:!0},multiselect:!1,navigationButtons:!1,selectable:!0,selectConnectedEdges:!0,hoverConnectedEdges:!0,tooltipDelay:[300,0,1e3,25],zoomView:!0},manipulation:{enabled:!1,initiallyActive:!1},physics:{enabled:!0,barnesHut:{gravitationalConstant:[-2e3,-3e4,0,50],centralGravity:[.3,0,10,.05],springLength:[95,0,500,5],springConstant:[.04,0,1.2,.005],damping:[.09,0,1,.01],avoidOverlap:[0,0,1,.01]},forceAtlas2Based:{gravitationalConstant:[-50,-500,0,1],centralGravity:[.01,0,1,.005],springLength:[95,0,500,5],springConstant:[.08,0,1.2,.005],damping:[.4,0,1,.01],avoidOverlap:[0,0,1,.01]},repulsion:{centralGravity:[.2,0,10,.05],springLength:[200,0,500,5],springConstant:[.05,0,1.2,.005],nodeDistance:[100,0,500,5],damping:[.09,0,1,.01]},hierarchicalRepulsion:{centralGravity:[.2,0,10,.05],springLength:[100,0,500,5],springConstant:[.01,0,1.2,.005],nodeDistance:[120,0,500,5],damping:[.09,0,1,.01]},maxVelocity:[50,0,150,1],minVelocity:[.1,.01,.5,.01],solver:["barnesHut","forceAtlas2Based","repulsion","hierarchicalRepulsion"],timestep:[.5,.01,1,.01]},global:{locale:["en","nl"]}};e.allOptions=d,e.configureOptions=l},function(t,e,i){function o(t){return t&&t.__esModule?t:{"default":t}}function n(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}Object.defineProperty(e,"__esModule",{value:!0});var s=function(){function t(t,e){var i=[],o=!0,n=!1,s=void 0;try{for(var r,a=t[Symbol.iterator]();!(o=(r=a.next()).done)&&(i.push(r.value),!e||i.length!==e);o=!0);}catch(h){n=!0,s=h}finally{try{!o&&a["return"]&&a["return"]()}finally{if(n)throw s}}return i}return function(e,i){if(Array.isArray(e))return e;if(Symbol.iterator in Object(e))return t(e,i);throw new TypeError("Invalid attempt to destructure non-iterable instance")}}(),r=function(){function t(t,e){for(var i=0;in&&h>a;){a+=1;var v=this._getHighestEnergyNode(i),g=s(v,4);for(c=g[0],l=g[1],u=g[2],p=g[3],f=l,m=0;f>r&&d>m;){m+=1,this._moveNode(c,u,p);var y=this._getEnergy(c),b=s(y,3);f=b[0],u=b[1],p=b[2]}}}},{key:"_getHighestEnergyNode",value:function(t){for(var e=this.body.nodeIndices,i=this.body.nodes,o=0,n=e[0],r=0,a=0,h=0;ho&&(o=u,n=d,r=p,a=f)}}return[n,o,r,a]}},{key:"_getEnergy",value:function(t){for(var e=this.body.nodeIndices,i=this.body.nodes,o=i[t].x,n=i[t].y,s=0,r=0,a=0;al;l++)for(var c=0;d-1>c;c++)for(var u=c+1;d>u;u++)o[e[c]][e[u]]=Math.min(o[e[c]][e[u]],o[e[c]][e[l]]+o[e[l]][e[u]]),o[e[u]][e[c]]=o[e[c]][e[u]];return o}}]),t}();e["default"]=n},function(t,e){"undefined"!=typeof CanvasRenderingContext2D&&(CanvasRenderingContext2D.prototype.circle=function(t,e,i){this.beginPath(),this.arc(t,e,i,0,2*Math.PI,!1),this.closePath()},CanvasRenderingContext2D.prototype.square=function(t,e,i){this.beginPath(),this.rect(t-i,e-i,2*i,2*i),this.closePath()},CanvasRenderingContext2D.prototype.triangle=function(t,e,i){this.beginPath(),i*=1.15,e+=.275*i;var o=2*i,n=o/2,s=Math.sqrt(3)/6*o,r=Math.sqrt(o*o-n*n);this.moveTo(t,e-(r-s)),this.lineTo(t+n,e+s),this.lineTo(t-n,e+s),this.lineTo(t,e-(r-s)),this.closePath()},CanvasRenderingContext2D.prototype.triangleDown=function(t,e,i){this.beginPath(),i*=1.15,e-=.275*i;var o=2*i,n=o/2,s=Math.sqrt(3)/6*o,r=Math.sqrt(o*o-n*n);this.moveTo(t,e+(r-s)), +this.lineTo(t+n,e-s),this.lineTo(t-n,e-s),this.lineTo(t,e+(r-s)),this.closePath()},CanvasRenderingContext2D.prototype.star=function(t,e,i){this.beginPath(),i*=.82,e+=.1*i;for(var o=0;10>o;o++){var n=o%2===0?1.3*i:.5*i;this.lineTo(t+n*Math.sin(2*o*Math.PI/10),e-n*Math.cos(2*o*Math.PI/10))}this.closePath()},CanvasRenderingContext2D.prototype.diamond=function(t,e,i){this.beginPath(),this.lineTo(t,e+i),this.lineTo(t+i,e),this.lineTo(t,e-i),this.lineTo(t-i,e),this.closePath()},CanvasRenderingContext2D.prototype.roundRect=function(t,e,i,o,n){var s=Math.PI/180;0>i-2*n&&(n=i/2),0>o-2*n&&(n=o/2),this.beginPath(),this.moveTo(t+n,e),this.lineTo(t+i-n,e),this.arc(t+i-n,e+n,n,270*s,360*s,!1),this.lineTo(t+i,e+o-n),this.arc(t+i-n,e+o-n,n,0,90*s,!1),this.lineTo(t+n,e+o),this.arc(t+n,e+o-n,n,90*s,180*s,!1),this.lineTo(t,e+n),this.arc(t+n,e+n,n,180*s,270*s,!1),this.closePath()},CanvasRenderingContext2D.prototype.ellipse=function(t,e,i,o){var n=.5522848,s=i/2*n,r=o/2*n,a=t+i,h=e+o,d=t+i/2,l=e+o/2;this.beginPath(),this.moveTo(t,l),this.bezierCurveTo(t,l-r,d-s,e,d,e),this.bezierCurveTo(d+s,e,a,l-r,a,l),this.bezierCurveTo(a,l+r,d+s,h,d,h),this.bezierCurveTo(d-s,h,t,l+r,t,l),this.closePath()},CanvasRenderingContext2D.prototype.database=function(t,e,i,o){var n=1/3,s=i,r=o*n,a=.5522848,h=s/2*a,d=r/2*a,l=t+s,c=e+r,u=t+s/2,p=e+r/2,f=e+(o-r/2),m=e+o;this.beginPath(),this.moveTo(l,p),this.bezierCurveTo(l,p+d,u+h,c,u,c),this.bezierCurveTo(u-h,c,t,p+d,t,p),this.bezierCurveTo(t,p-d,u-h,e,u,e),this.bezierCurveTo(u+h,e,l,p-d,l,p),this.lineTo(l,f),this.bezierCurveTo(l,f+d,u+h,m,u,m),this.bezierCurveTo(u-h,m,t,f+d,t,f),this.lineTo(t,p)},CanvasRenderingContext2D.prototype.arrow=function(t,e,i,o){var n=t-o*Math.cos(i),s=e-o*Math.sin(i),r=t-.9*o*Math.cos(i),a=e-.9*o*Math.sin(i),h=n+o/3*Math.cos(i+.5*Math.PI),d=s+o/3*Math.sin(i+.5*Math.PI),l=n+o/3*Math.cos(i-.5*Math.PI),c=s+o/3*Math.sin(i-.5*Math.PI);this.beginPath(),this.moveTo(t,e),this.lineTo(h,d),this.lineTo(r,a),this.lineTo(l,c),this.closePath()},CanvasRenderingContext2D.prototype.dashedLine=function(t,e,i,o,n){this.beginPath(),this.moveTo(t,e);for(var s=n.length,r=i-t,a=o-e,h=a/r,d=Math.sqrt(r*r+a*a),l=0,c=!0,u=0,p=n[0];d>=.1;)p=n[l++%s],p>d&&(p=d),u=Math.sqrt(p*p/(1+h*h)),u=0>r?-u:u,t+=u,e+=h*u,c===!0?this.lineTo(t,e):this.moveTo(t,e),d-=p,c=!c})},function(t,e){function i(t){return P=t,p()}function o(){I=0,N=P.charAt(0)}function n(){I++,N=P.charAt(I)}function s(){return P.charAt(I+1)}function r(t){return L.test(t)}function a(t,e){if(t||(t={}),e)for(var i in e)e.hasOwnProperty(i)&&(t[i]=e[i]);return t}function h(t,e,i){for(var o=e.split("."),n=t;o.length;){var s=o.shift();o.length?(n[s]||(n[s]={}),n=n[s]):n[s]=i}}function d(t,e){for(var i,o,n=null,s=[t],r=t;r.parent;)s.push(r.parent),r=r.parent;if(r.nodes)for(i=0,o=r.nodes.length;o>i;i++)if(e.id===r.nodes[i].id){n=r.nodes[i];break}for(n||(n={id:e.id},t.node&&(n.attr=a(n.attr,t.node))),i=s.length-1;i>=0;i--){var h=s[i];h.nodes||(h.nodes=[]),-1===h.nodes.indexOf(n)&&h.nodes.push(n)}e.attr&&(n.attr=a(n.attr,e.attr))}function l(t,e){if(t.edges||(t.edges=[]),t.edges.push(e),t.edge){var i=a({},t.edge);e.attr=a(i,e.attr)}}function c(t,e,i,o,n){var s={from:e,to:i,type:o};return t.edge&&(s.attr=a({},t.edge)),s.attr=a(s.attr||{},n),s}function u(){for(z=T.NULL,R="";" "===N||" "===N||"\n"===N||"\r"===N;)n();do{var t=!1;if("#"===N){for(var e=I-1;" "===P.charAt(e)||" "===P.charAt(e);)e--;if("\n"===P.charAt(e)||""===P.charAt(e)){for(;""!=N&&"\n"!=N;)n();t=!0}}if("/"===N&&"/"===s()){for(;""!=N&&"\n"!=N;)n();t=!0}if("/"===N&&"*"===s()){for(;""!=N;){if("*"===N&&"/"===s()){n(),n();break}n()}t=!0}for(;" "===N||" "===N||"\n"===N||"\r"===N;)n()}while(t);if(""===N)return void(z=T.DELIMITER);var i=N+s();if(E[i])return z=T.DELIMITER,R=i,n(),void n();if(E[N])return z=T.DELIMITER,R=N,void n();if(r(N)||"-"===N){for(R+=N,n();r(N);)R+=N,n();return"false"===R?R=!1:"true"===R?R=!0:isNaN(Number(R))||(R=Number(R)),void(z=T.IDENTIFIER)}if('"'===N){for(n();""!=N&&('"'!=N||'"'===N&&'"'===s());)R+=N,'"'===N&&n(),n();if('"'!=N)throw _('End of string " expected');return n(),void(z=T.IDENTIFIER)}for(z=T.UNKNOWN;""!=N;)R+=N,n();throw new SyntaxError('Syntax error in part "'+x(R,30)+'"')}function p(){var t={};if(o(),u(),"strict"===R&&(t.strict=!0,u()),"graph"!==R&&"digraph"!==R||(t.type=R,u()),z===T.IDENTIFIER&&(t.id=R,u()),"{"!=R)throw _("Angle bracket { expected");if(u(),f(t),"}"!=R)throw _("Angle bracket } expected");if(u(),""!==R)throw _("End of file expected");return u(),delete t.node,delete t.edge,delete t.graph,t}function f(t){for(;""!==R&&"}"!=R;)m(t),";"===R&&u()}function m(t){var e=v(t);if(e)return void b(t,e);var i=g(t);if(!i){if(z!=T.IDENTIFIER)throw _("Identifier expected");var o=R;if(u(),"="===R){if(u(),z!=T.IDENTIFIER)throw _("Identifier expected");t[o]=R,u()}else y(t,o)}}function v(t){var e=null;if("subgraph"===R&&(e={},e.type="subgraph",u(),z===T.IDENTIFIER&&(e.id=R,u())),"{"===R){if(u(),e||(e={}),e.parent=t,e.node=t.node,e.edge=t.edge,e.graph=t.graph,f(e),"}"!=R)throw _("Angle bracket } expected");u(),delete e.node,delete e.edge,delete e.graph,delete e.parent,t.subgraphs||(t.subgraphs=[]),t.subgraphs.push(e)}return e}function g(t){return"node"===R?(u(),t.node=w(),"node"):"edge"===R?(u(),t.edge=w(),"edge"):"graph"===R?(u(),t.graph=w(),"graph"):null}function y(t,e){var i={id:e},o=w();o&&(i.attr=o),d(t,i),b(t,e)}function b(t,e){for(;"->"===R||"--"===R;){var i,o=R;u();var n=v(t);if(n)i=n;else{if(z!=T.IDENTIFIER)throw _("Identifier or subgraph expected");i=R,d(t,{id:i}),u()}var s=w(),r=c(t,e,i,o,s);l(t,r),e=i}}function w(){for(var t=null;"["===R;){for(u(),t={};""!==R&&"]"!=R;){if(z!=T.IDENTIFIER)throw _("Attribute name expected");var e=R;if(u(),"="!=R)throw _("Equal sign = expected");if(u(),z!=T.IDENTIFIER)throw _("Attribute value expected");var i=R;h(t,e,i),u(),","==R&&u()}if("]"!=R)throw _("Bracket ] expected");u()}return t}function _(t){return new SyntaxError(t+', got "'+x(R,30)+'" (char '+I+")")}function x(t,e){return t.length<=e?t:t.substr(0,27)+"..."}function k(t,e,i){Array.isArray(t)?t.forEach(function(t){Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}):Array.isArray(e)?e.forEach(function(e){i(t,e)}):i(t,e)}function O(t,e,i){for(var o=e.split("."),n=o.pop(),s=t,r=0;r":!0,"--":!0},P="",I=0,N="",R="",z=T.NULL,L=/[a-zA-Z_0-9.:#]/;e.parseDOT=i,e.DOTToGraph=D},function(t,e){function i(t,e){var i=[],o=[],n={edges:{inheritColor:!1},nodes:{fixed:!1,parseColor:!1}};void 0!==e&&(void 0!==e.fixed&&(n.nodes.fixed=e.fixed),void 0!==e.parseColor&&(n.nodes.parseColor=e.parseColor),void 0!==e.inheritColor&&(n.edges.inheritColor=e.inheritColor));for(var s=t.edges,r=t.nodes,a=0;a ++ } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 22ee13b6e78c1..99f2bd8bc1f22 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -179,7 +179,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
    ++ } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 5d1928ac6b2ca..6ddabfd8ef089 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -26,7 +26,7 @@ import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.{InternalAccumulator, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} import org.apache.spark.ui._ @@ -746,7 +746,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ++ } From 4c0af3bbd5a6e88b1fedf0d9c624bbdb82c1aa40 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 15:45:59 -0700 Subject: [PATCH 0261/1470] [SPARK-15392][SQL] fix default value of size estimation of logical plan ## What changes were proposed in this pull request? We use autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD. This PR change the default value to Long.MaxValue. ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #13179 from davies/fix_default_size. (cherry picked from commit fc29b896dae08b957ed15fa681b46162600a4050) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 3 +-- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 +++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7933d12e284f1..a7f461381b505 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -605,8 +605,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) - def defaultSizeInBytes: Long = - getConf(DEFAULT_SIZE_IN_BYTES, autoBroadcastJoinThreshold + 1L) + def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, Long.MaxValue) def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING) 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 f573abf859610..df029e44c54e5 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 @@ -1476,4 +1476,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { getMessage() assert(e1.startsWith("Path does not exist")) } + + test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { + val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) + val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) + assert(df.queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + } } From 36acf8856c94f93f3b16f4592b6d5fb64acda39d Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Wed, 18 May 2016 16:15:09 -0700 Subject: [PATCH 0262/1470] [SPARK-15323][SPARK-14463][SQL] Fix reading of partitioned format=text datasets https://issues.apache.org/jira/browse/SPARK-15323 I was using partitioned text datasets in Spark 1.6.1 but it broke in Spark 2.0.0. It would be logical if you could also write those, but not entirely sure how to solve this with the new DataSet implementation. Also it doesn't work using `sqlContext.read.text`, since that method returns a `DataSet[String]`. See https://issues.apache.org/jira/browse/SPARK-14463 for that issue. Author: Jurriaan Pruis Closes #13104 from jurriaan/fix-partitioned-text-reads. (cherry picked from commit 32be51fba45f5e07a2a3520293c12dc7765a364d) Signed-off-by: Reynold Xin --- .../apache/spark/sql/DataFrameReader.scala | 3 ++- .../datasources/text/DefaultSource.scala | 14 ------------- .../text-partitioned/year=2014/data.txt | 1 + .../text-partitioned/year=2015/data.txt | 1 + .../datasources/text/TextSuite.scala | 20 +++++++++++++++++++ 5 files changed, 24 insertions(+), 15 deletions(-) create mode 100644 sql/core/src/test/resources/text-partitioned/year=2014/data.txt create mode 100644 sql/core/src/test/resources/text-partitioned/year=2015/data.txt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 011aff4ff6c28..e33fd831ab471 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -457,7 +457,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { */ @scala.annotation.varargs def text(paths: String*): Dataset[String] = { - format("text").load(paths : _*).as[String](sparkSession.implicits.newStringEncoder) + format("text").load(paths : _*).select("value") + .as[String](sparkSession.implicits.newStringEncoder) } /////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index f22c0241d930e..f091615a9a714 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -83,19 +83,6 @@ class DefaultSource extends FileFormat with DataSourceRegister { } } - override private[sql] def buildReaderWithPartitionValues( - sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { - // Text data source doesn't support partitioning. Here we simply delegate to `buildReader`. - buildReader( - sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf) - } - override def buildReader( sparkSession: SparkSession, dataSchema: StructType, @@ -152,4 +139,3 @@ class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemp recordWriter.close(context) } } - diff --git a/sql/core/src/test/resources/text-partitioned/year=2014/data.txt b/sql/core/src/test/resources/text-partitioned/year=2014/data.txt new file mode 100644 index 0000000000000..e2719428bb28e --- /dev/null +++ b/sql/core/src/test/resources/text-partitioned/year=2014/data.txt @@ -0,0 +1 @@ +2014-test diff --git a/sql/core/src/test/resources/text-partitioned/year=2015/data.txt b/sql/core/src/test/resources/text-partitioned/year=2015/data.txt new file mode 100644 index 0000000000000..b8c03daa8c199 --- /dev/null +++ b/sql/core/src/test/resources/text-partitioned/year=2015/data.txt @@ -0,0 +1 @@ +2015-test diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index f61fce5d4102d..b5e51e963f1b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -65,6 +65,26 @@ class TextSuite extends QueryTest with SharedSQLContext { } } + test("reading partitioned data using read.text()") { + val partitionedData = Thread.currentThread().getContextClassLoader + .getResource("text-partitioned").toString + val df = spark.read.text(partitionedData) + val data = df.collect() + + assert(df.schema == new StructType().add("value", StringType)) + assert(data.length == 2) + } + + test("support for partitioned reading") { + val partitionedData = Thread.currentThread().getContextClassLoader + .getResource("text-partitioned").toString + val df = spark.read.format("text").load(partitionedData) + val data = df.filter("year = '2015'").select("value").collect() + + assert(data(0) == Row("2015-test")) + assert(data.length == 1) + } + test("SPARK-13503 Support to specify the option for compression codec for TEXT") { val testDf = spark.read.text(testFile) val extensionNameMap = Map("bzip2" -> ".bz2", "deflate" -> ".deflate", "gzip" -> ".gz") From f5784459e84e84d6641725bf0d0fb31db742456c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 May 2016 18:06:38 -0700 Subject: [PATCH 0263/1470] [SPARK-15192][SQL] null check for SparkSession.createDataFrame ## What changes were proposed in this pull request? This PR adds null check in `SparkSession.createDataFrame`, so that we can make sure the passed in rows matches the given schema. ## How was this patch tested? new tests in `DatasetSuite` Author: Wenchen Fan Closes #13008 from cloud-fan/row-encoder. (cherry picked from commit ebfe3a1f2c77e6869c3c36ba67afb7fabe6a94f5) Signed-off-by: Yin Huai --- .../scala/org/apache/spark/mllib/fpm/FPGrowth.scala | 2 +- .../apache/spark/sql/catalyst/ScalaReflection.scala | 4 ++-- .../spark/sql/catalyst/encoders/RowEncoder.scala | 10 +++------- .../sql/catalyst/expressions/BoundAttribute.scala | 2 +- .../sql/catalyst/expressions/objects/objects.scala | 4 +++- .../scala/org/apache/spark/sql/SparkSession.scala | 4 ++-- .../scala/org/apache/spark/sql/api/r/SQLUtils.scala | 5 ++++- .../scala/org/apache/spark/sql/DatasetSuite.scala | 13 +++++++++++-- .../org/apache/spark/sql/test/SQLTestUtils.scala | 6 +----- 9 files changed, 28 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 9166faa54de56..28e4966f918a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -116,7 +116,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { StructField("freq", LongType)) val schema = StructType(fields) val rowDataRDD = model.freqItemsets.map { x => - Row(x.items, x.freq) + Row(x.items.toSeq, x.freq) } sqlContext.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) } 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 cb9a62dfd4e81..c0fa220d34bb6 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 @@ -113,8 +113,8 @@ object ScalaReflection extends ScalaReflection { * Returns true if the value of this data type is same between internal and external. */ def isNativeType(dt: DataType): Boolean = dt match { - case BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | BinaryType => true + case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType | BinaryType | CalendarIntervalType => true case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index a5f39aaa2314b..71b39c54fa0c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -70,8 +70,7 @@ object RowEncoder { private def serializerFor( inputObject: Expression, inputType: DataType): Expression = inputType match { - case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | BinaryType | CalendarIntervalType => inputObject + case dt if ScalaReflection.isNativeType(dt) => inputObject case p: PythonUserDefinedType => serializerFor(inputObject, p.sqlType) @@ -151,7 +150,7 @@ object RowEncoder { case StructType(fields) => val convertedFields = fields.zipWithIndex.map { case (f, i) => val fieldValue = serializerFor( - GetExternalRowField(inputObject, i, externalDataTypeForInput(f.dataType)), + GetExternalRowField(inputObject, i, f.name, externalDataTypeForInput(f.dataType)), f.dataType ) if (f.nullable) { @@ -193,7 +192,6 @@ object RowEncoder { private def externalDataTypeFor(dt: DataType): DataType = dt match { case _ if ScalaReflection.isNativeType(dt) => dt - case CalendarIntervalType => dt case TimestampType => ObjectType(classOf[java.sql.Timestamp]) case DateType => ObjectType(classOf[java.sql.Date]) case _: DecimalType => ObjectType(classOf[java.math.BigDecimal]) @@ -202,7 +200,6 @@ object RowEncoder { case _: MapType => ObjectType(classOf[scala.collection.Map[_, _]]) case _: StructType => ObjectType(classOf[Row]) case udt: UserDefinedType[_] => ObjectType(udt.userClass) - case _: NullType => ObjectType(classOf[java.lang.Object]) } private def deserializerFor(schema: StructType): Expression = { @@ -222,8 +219,7 @@ object RowEncoder { } private def deserializerFor(input: Expression): Expression = input.dataType match { - case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType | BinaryType | CalendarIntervalType => input + case dt if ScalaReflection.isNativeType(dt) => input case udt: UserDefinedType[_] => val annotation = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 99f156a935b50..a38f1ec09156d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) extends LeafExpression { - override def toString: String = s"input[$ordinal, ${dataType.simpleString}]" + override def toString: String = s"input[$ordinal, ${dataType.simpleString}, $nullable]" // Use special getter for primitive types (for UnsafeRow) override def eval(input: InternalRow): Any = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 7df6e06805a54..fc38369f38c33 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -693,6 +693,7 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) case class GetExternalRowField( child: Expression, index: Int, + fieldName: String, dataType: DataType) extends UnaryExpression with NonSQLExpression { override def nullable: Boolean = false @@ -716,7 +717,8 @@ case class GetExternalRowField( } if (${row.value}.isNullAt($index)) { - throw new RuntimeException("The ${index}th field of input row cannot be null."); + throw new RuntimeException("The ${index}th field '$fieldName' of input row " + + "cannot be null."); } final ${ctx.javaType(dataType)} ${ev.value} = $getField; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index aa974f29cc34d..8ef6cd3c35572 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -478,8 +478,8 @@ class SparkSession private( // 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) { - val converter = CatalystTypeConverters.createToCatalystConverter(schema) - rowRDD.map(converter(_).asInstanceOf[InternalRow]) + val encoder = RowEncoder(schema) + rowRDD.map(encoder.toRow) } else { rowRDD.map{r: Row => InternalRow.fromSeq(r.toSeq)} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 36173a49250b5..2021cddab4404 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.api.r import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} +import scala.collection.JavaConverters._ import scala.util.matching.Regex import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} @@ -110,6 +111,8 @@ private[sql] object SQLUtils { data match { case d: java.lang.Double if dataType == FloatType => new java.lang.Float(d) + // Scala Map is the only allowed external type of map type in Row. + case m: java.util.Map[_, _] => m.asScala case _ => data } } @@ -120,7 +123,7 @@ private[sql] object SQLUtils { val num = SerDe.readInt(dis) Row.fromSeq((0 until num).map { i => doConversion(SerDe.readObject(dis), schema.fields(i).dataType) - }.toSeq) + }) } private[sql] def rowToRBytes(row: Row): Array[Byte] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index b02b714168cb0..1935e41185ac0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -507,7 +507,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val schema = StructType(Seq( StructField("f", StructType(Seq( StructField("a", StringType, nullable = true), - StructField("b", IntegerType, nullable = false) + StructField("b", IntegerType, nullable = true) )), nullable = true) )) @@ -684,7 +684,16 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val message = intercept[Exception] { df.collect() }.getMessage - assert(message.contains("The 0th field of input row cannot be null")) + assert(message.contains("The 0th field 'i' of input row cannot be null")) + } + + test("row nullability mismatch") { + val schema = new StructType().add("a", StringType, true).add("b", StringType, false) + val rdd = sqlContext.sparkContext.parallelize(Row(null, "123") :: Row("234", null) :: Nil) + val message = intercept[Exception] { + sqlContext.createDataFrame(rdd, schema).collect() + }.getMessage + assert(message.contains("The 1th field 'b' of input row cannot be null")) } test("createTempView") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 45a9c9dc47815..51538eca644f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -217,11 +217,7 @@ private[sql] trait SQLTestUtils case FilterExec(_, child) => child } - val childRDD = withoutFilters - .execute() - .map(row => Row.fromSeq(row.copy().toSeq(schema))) - - spark.createDataFrame(childRDD, schema) + spark.internalCreateDataFrame(withoutFilters.execute(), schema) } /** From 760e7ac8161bc4a847abde6fa93ce21e14de4163 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 19 May 2016 10:05:53 +0800 Subject: [PATCH 0264/1470] [SPARK-15297][SQL] Fix Set -V Command #### What changes were proposed in this pull request? The command `SET -v` always outputs the default values even if we set the parameter. This behavior is incorrect. Instead, if users override it, we should output the user-specified value. In addition, the output schema of `SET -v` is wrong. We should use the column `value` instead of `default` for the parameter value. This PR is to fix the above two issues. #### How was this patch tested? Added a test case. Author: gatorsmile Closes #13081 from gatorsmile/setVcommand. (cherry picked from commit 9c2a376e413b0701097b0784bd725e4ca87cd837) Signed-off-by: Wenchen Fan --- .../sql/execution/command/SetCommand.scala | 4 +--- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../spark/sql/internal/SQLConfSuite.scala | 23 ++++++++++++++++++- 3 files changed, 24 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 2409b5d203f40..282f26ce998fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.command -import java.util.NoSuchElementException - import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute @@ -88,7 +86,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm } val schema = StructType( StructField("key", StringType, nullable = false) :: - StructField("default", StringType, nullable = false) :: + StructField("value", StringType, nullable = false) :: StructField("meaning", StringType, nullable = false) :: Nil) (schema.toAttributes, runFunc) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index a7f461381b505..248c6e3335f23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -751,7 +751,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { */ def getAllDefinedConfs: Seq[(String, String, String)] = sqlConfEntries.synchronized { sqlConfEntries.values.asScala.filter(_.isPublic).map { entry => - (entry.key, entry.defaultValueString, entry.doc) + (entry.key, getConfString(entry.key, entry.defaultValueString), entry.doc) }.toSeq } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 7ead97bbf6937..81bc973be74a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal -import org.apache.spark.sql.{QueryTest, SparkSession, SQLContext} +import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} class SQLConfSuite extends QueryTest with SharedSQLContext { @@ -75,6 +75,27 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { spark.wrapped.conf.clear() } + test("set command for display") { + spark.wrapped.conf.clear() + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Nil) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "true")) + + sql("SET spark.sql.groupByOrdinal=false") + + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + } + test("deprecated property") { spark.wrapped.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) From 595ed8de60c2d0cfde4aaf8aafe44f734d26631a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 18 May 2016 19:16:28 -0700 Subject: [PATCH 0265/1470] [SPARK-14463][SQL] Document the semantics for read.text ## What changes were proposed in this pull request? This patch is a follow-up to https://github.com/apache/spark/pull/13104 and adds documentation to clarify the semantics of read.text with respect to partitioning. ## How was this patch tested? N/A Author: Reynold Xin Closes #13184 from rxin/SPARK-14463. (cherry picked from commit 4987f39ac7a694e1c8b8b82246eb4fbd863201c4) Signed-off-by: Reynold Xin --- R/pkg/R/SQLContext.R | 2 ++ python/pyspark/sql/readwriter.py | 3 +++ .../main/scala/org/apache/spark/sql/DataFrameReader.scala | 8 ++++++-- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 3824e0a99557c..6b7a341bee889 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -298,6 +298,8 @@ parquetFile <- function(sqlContext, ...) { #' Create a SparkDataFrame from a text file. #' #' Loads a text file and returns a SparkDataFrame with a single string column named "value". +#' If the directory structure of the text files contains partitioning information, those are +#' ignored in the resulting DataFrame. #' Each line in the text file is a new row in the resulting SparkDataFrame. #' #' @param sqlContext SQLContext to use diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 8e6bce90010e3..855c9d666f0bc 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -286,6 +286,9 @@ def parquet(self, *paths): @since(1.6) def text(self, paths): """Loads a text file and returns a [[DataFrame]] with a single string column named "value". + If the directory structure of the text files contains partitioning information, + those are ignored in the resulting DataFrame. To include partitioning information as + columns, use ``read.format('text').load(...)``. Each line in the text file is a new row in the resulting DataFrame. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index e33fd831ab471..57a2091fe8c77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -440,10 +440,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } /** - * Loads a text file and returns a [[Dataset]] of String. The underlying schema of the Dataset + * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset * contains a single string column named "value". * - * Each line in the text file is a new row in the resulting Dataset. For example: + * If the directory structure of the text files contains partitioning information, those are + * ignored in the resulting Dataset. To include partitioning information as columns, use + * `read.format("text").load("...")`. + * + * Each line in the text files is a new element in the resulting Dataset. For example: * {{{ * // Scala: * spark.read.text("/path/to/spark/README.md") From a1948a0af1523ab6b14b391c4208cd458bccd2c1 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 18 May 2016 20:15:00 -0700 Subject: [PATCH 0266/1470] [SPARK-15395][CORE] Use getHostString to create RpcAddress ## What changes were proposed in this pull request? Right now the netty RPC uses `InetSocketAddress.getHostName` to create `RpcAddress` for network events. If we use an IP address to connect, then the RpcAddress's host will be a host name (if the reverse lookup successes) instead of the IP address. However, some places need to compare the original IP address and the RpcAddress in `onDisconnect` (e.g., CoarseGrainedExecutorBackend), and this behavior will make the check incorrect. This PR uses `getHostString` to resolve the issue. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13185 from zsxwing/host-string. (cherry picked from commit 5c9117a3ed373461529f9f9306668ed4149c63fb) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 7d7b4c82fa392..89d2fb9b47971 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -574,7 +574,7 @@ private[netty] class NettyRpcHandler( private def internalReceive(client: TransportClient, message: ByteBuffer): RequestMessage = { val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) val requestMessage = nettyEnv.deserialize[RequestMessage](client, message) if (requestMessage.senderAddress == null) { // Create a new message with the socket address of the client as the sender. @@ -595,7 +595,7 @@ private[netty] class NettyRpcHandler( override def exceptionCaught(cause: Throwable, client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) dispatcher.postToAll(RemoteProcessConnectionError(cause, clientAddr)) // If the remove RpcEnv listens to some address, we should also fire a // RemoteProcessConnectionError for the remote RpcEnv listening address @@ -614,14 +614,14 @@ private[netty] class NettyRpcHandler( override def channelActive(client: TransportClient): Unit = { val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) dispatcher.postToAll(RemoteProcessConnected(clientAddr)) } override def channelInactive(client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) nettyEnv.removeOutbox(clientAddr) dispatcher.postToAll(RemoteProcessDisconnected(clientAddr)) val remoteEnvAddress = remoteAddresses.remove(clientAddr) From 34c743c4b8f009f8549d17f724a1a6b6ba8a59b4 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 May 2016 21:43:07 -0700 Subject: [PATCH 0267/1470] [SPARK-15381] [SQL] physical object operator should define reference correctly ## What changes were proposed in this pull request? Whole Stage Codegen depends on `SparkPlan.reference` to do some optimization. For physical object operators, they should be consistent with their logical version and set the `reference` correctly. ## How was this patch tested? new test in DatasetSuite Author: Wenchen Fan Closes #13167 from cloud-fan/bug. (cherry picked from commit 661c21049b62ebfaf788dcbc31d62a09e206265b) Signed-off-by: Davies Liu --- .../sql/catalyst/plans/logical/object.scala | 10 +- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../apache/spark/sql/execution/objects.scala | 94 +++++++++++-------- .../org/apache/spark/sql/DatasetSuite.scala | 5 + 4 files changed, 64 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 84339f439a666..98ce5dd2efd91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -94,7 +94,7 @@ case class DeserializeToObject( */ case class SerializeFromObject( serializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode with ObjectConsumer { + child: LogicalPlan) extends ObjectConsumer { override def output: Seq[Attribute] = serializer.map(_.toAttribute) } @@ -118,7 +118,7 @@ object MapPartitions { case class MapPartitions( func: Iterator[Any] => Iterator[Any], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer + child: LogicalPlan) extends ObjectConsumer with ObjectProducer object MapPartitionsInR { def apply( @@ -152,7 +152,7 @@ case class MapPartitionsInR( inputSchema: StructType, outputSchema: StructType, outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer { + child: LogicalPlan) extends ObjectConsumer with ObjectProducer { override lazy val schema = outputSchema } @@ -175,7 +175,7 @@ object MapElements { case class MapElements( func: AnyRef, outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer + child: LogicalPlan) extends ObjectConsumer with ObjectProducer /** Factory for constructing new `AppendColumn` nodes. */ object AppendColumns { @@ -215,7 +215,7 @@ case class AppendColumnsWithObject( func: Any => Any, childSerializer: Seq[NamedExpression], newColumnsSerializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode with ObjectConsumer { + child: LogicalPlan) extends ObjectConsumer { override def output: Seq[Attribute] = (childSerializer ++ newColumnsSerializer).map(_.toAttribute) } 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 faf359f54838e..5cfb6d5363a3b 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 @@ -303,7 +303,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { "logical except operator should have been replaced by anti-join in the optimizer") case logical.DeserializeToObject(deserializer, objAttr, child) => - execution.DeserializeToObject(deserializer, objAttr, planLater(child)) :: Nil + execution.DeserializeToObjectExec(deserializer, objAttr, planLater(child)) :: Nil case logical.SerializeFromObject(serializer, child) => execution.SerializeFromObjectExec(serializer, planLater(child)) :: Nil case logical.MapPartitions(f, objAttr, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 3ff991392dfb4..5fced940b38d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -28,17 +28,41 @@ import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.{DataType, ObjectType} + +/** + * Physical version of `ObjectProducer`. + */ +trait ObjectProducerExec extends SparkPlan { + // The attribute that reference to the single object field this operator outputs. + protected def outputObjAttr: Attribute + + override def output: Seq[Attribute] = outputObjAttr :: Nil + + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + + def outputObjectType: DataType = outputObjAttr.dataType +} + +/** + * Physical version of `ObjectConsumer`. + */ +trait ObjectConsumerExec extends UnaryExecNode { + assert(child.output.length == 1) + + // This operator always need all columns of its child, even it doesn't reference to. + override def references: AttributeSet = child.outputSet + + def inputObjectType: DataType = child.output.head.dataType +} + /** * Takes the input row from child and turns it into object using the given deserializer expression. * The output of this operator is a single-field safe row containing the deserialized object. */ -case class DeserializeToObject( +case class DeserializeToObjectExec( deserializer: Expression, outputObjAttr: Attribute, - child: SparkPlan) extends UnaryExecNode with CodegenSupport { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + child: SparkPlan) extends UnaryExecNode with ObjectProducerExec with CodegenSupport { override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() @@ -70,7 +94,7 @@ case class DeserializeToObject( */ case class SerializeFromObjectExec( serializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode with CodegenSupport { + child: SparkPlan) extends ObjectConsumerExec with CodegenSupport { override def output: Seq[Attribute] = serializer.map(_.toAttribute) @@ -102,7 +126,7 @@ case class SerializeFromObjectExec( /** * Helper functions for physical operators that work with user defined objects. */ -trait ObjectOperator extends SparkPlan { +object ObjectOperator { def deserializeRowToObject( deserializer: Expression, inputSchema: Seq[Attribute]): InternalRow => Any = { @@ -141,15 +165,12 @@ case class MapPartitionsExec( func: Iterator[Any] => Iterator[Any], outputObjAttr: Attribute, child: SparkPlan) - extends UnaryExecNode with ObjectOperator { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + extends ObjectConsumerExec with ObjectProducerExec { override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getObject = unwrapObjectFromRow(child.output.head.dataType) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) func(iter.map(getObject)).map(outputObject) } } @@ -166,10 +187,7 @@ case class MapElementsExec( func: AnyRef, outputObjAttr: Attribute, child: SparkPlan) - extends UnaryExecNode with ObjectOperator with CodegenSupport { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + extends ObjectConsumerExec with ObjectProducerExec with CodegenSupport { override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() @@ -202,8 +220,8 @@ case class MapElementsExec( } child.execute().mapPartitionsInternal { iter => - val getObject = unwrapObjectFromRow(child.output.head.dataType) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) iter.map(row => outputObject(callFunc(getObject(row)))) } } @@ -218,7 +236,7 @@ case class AppendColumnsExec( func: Any => Any, deserializer: Expression, serializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode with ObjectOperator { + child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = child.output ++ serializer.map(_.toAttribute) @@ -226,9 +244,9 @@ case class AppendColumnsExec( override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getObject = deserializeRowToObject(deserializer, child.output) + val getObject = ObjectOperator.deserializeRowToObject(deserializer, child.output) val combiner = GenerateUnsafeRowJoiner.create(child.schema, newColumnSchema) - val outputObject = serializeObjectToRow(serializer) + val outputObject = ObjectOperator.serializeObjectToRow(serializer) iter.map { row => val newColumns = outputObject(func(getObject(row))) @@ -246,7 +264,7 @@ case class AppendColumnsWithObjectExec( func: Any => Any, inputSerializer: Seq[NamedExpression], newColumnsSerializer: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode with ObjectOperator { + child: SparkPlan) extends ObjectConsumerExec { override def output: Seq[Attribute] = (inputSerializer ++ newColumnsSerializer).map(_.toAttribute) @@ -255,9 +273,9 @@ case class AppendColumnsWithObjectExec( override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getChildObject = unwrapObjectFromRow(child.output.head.dataType) - val outputChildObject = serializeObjectToRow(inputSerializer) - val outputNewColumnOjb = serializeObjectToRow(newColumnsSerializer) + val getChildObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) + val outputChildObject = ObjectOperator.serializeObjectToRow(inputSerializer) + val outputNewColumnOjb = ObjectOperator.serializeObjectToRow(newColumnsSerializer) val combiner = GenerateUnsafeRowJoiner.create(inputSchema, newColumnSchema) iter.map { row => @@ -280,10 +298,7 @@ case class MapGroupsExec( groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: SparkPlan) extends UnaryExecNode with ObjectOperator { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(groupingAttributes) :: Nil @@ -295,9 +310,9 @@ case class MapGroupsExec( child.execute().mapPartitionsInternal { iter => val grouped = GroupedIterator(iter, groupingAttributes, child.output) - val getKey = deserializeRowToObject(keyDeserializer, groupingAttributes) - val getValue = deserializeRowToObject(valueDeserializer, dataAttributes) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) + val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) grouped.flatMap { case (key, rowIter) => val result = func( @@ -325,10 +340,7 @@ case class CoGroupExec( rightAttr: Seq[Attribute], outputObjAttr: Attribute, left: SparkPlan, - right: SparkPlan) extends BinaryExecNode with ObjectOperator { - - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + right: SparkPlan) extends BinaryExecNode with ObjectProducerExec { override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil @@ -341,10 +353,10 @@ case class CoGroupExec( val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) - val getKey = deserializeRowToObject(keyDeserializer, leftGroup) - val getLeft = deserializeRowToObject(leftDeserializer, leftAttr) - val getRight = deserializeRowToObject(rightDeserializer, rightAttr) - val outputObject = wrapObjectToRow(outputObjAttr.dataType) + val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, leftGroup) + val getLeft = ObjectOperator.deserializeRowToObject(leftDeserializer, leftAttr) + val getRight = ObjectOperator.deserializeRowToObject(rightDeserializer, rightAttr) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup).flatMap { case (key, leftResult, rightResult) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 1935e41185ac0..52e706285c7ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -711,6 +711,11 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(e.message.contains("already exists")) dataset.sparkSession.catalog.dropTempView("tempView") } + + test("SPARK-15381: physical object operator should define `reference` correctly") { + val df = Seq(1 -> 2).toDF("a", "b") + checkAnswer(df.map(row => row)(RowEncoder(df.schema)).select("b", "a"), Row(2, 1)) + } } case class Generic[T](id: T, value: Double) From b2a4dac2d92e906460fe3ca0a38fc672a82eb6cb Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 19 May 2016 08:52:41 +0200 Subject: [PATCH 0268/1470] [SPARK-15031][EXAMPLES][FOLLOW-UP] Make Python param example working with SparkSession ## What changes were proposed in this pull request? It seems most of Python examples were changed to use SparkSession by https://github.com/apache/spark/pull/12809. This PR said both examples below: - `simple_params_example.py` - `aft_survival_regression.py` are not changed because it dose not work. It seems `aft_survival_regression.py` is changed by https://github.com/apache/spark/pull/13050 but `simple_params_example.py` is not yet. This PR corrects the example and make this use SparkSession. In more detail, it seems `threshold` is replaced to `thresholds` here and there by https://github.com/apache/spark/commit/5a23213c148bfe362514f9c71f5273ebda0a848a. However, when it calls `lr.fit(training, paramMap)` this overwrites the values. So, `threshold` was 5 and `thresholds` becomes 5.5 (by `1 / (1 + thresholds(0) / thresholds(1)`). According to the comment below. this is not allowed, https://github.com/apache/spark/blob/354f8f11bd4b20fa99bd67a98da3525fd3d75c81/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L58-L61. So, in this PR, it sets the equivalent value so that this does not throw an exception. ## How was this patch tested? Manully (`mvn package -DskipTests && spark-submit simple_params_example.py`) Author: hyukjinkwon Closes #13135 from HyukjinKwon/SPARK-15031. (cherry picked from commit e2ec32dab8530aa21ec95a27d60b1c22f3d1a18c) Signed-off-by: Nick Pentreath --- .../examples/ml/JavaSimpleParamsExample.java | 2 +- .../main/python/ml/simple_params_example.py | 24 +++++++++---------- .../examples/ml/SimpleParamsExample.scala | 2 +- 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index ff1eb07dc6058..ca80d0d8bba57 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -77,7 +77,7 @@ public static void main(String[] args) { ParamMap paramMap = new ParamMap(); paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. - double[] thresholds = {0.45, 0.55}; + double[] thresholds = {0.5, 0.5}; paramMap.put(lr.regParam().w(0.1), lr.thresholds().w(thresholds)); // Specify multiple Params. // One can also combine ParamMaps. diff --git a/examples/src/main/python/ml/simple_params_example.py b/examples/src/main/python/ml/simple_params_example.py index 2d6d115d54d02..c57e59d01b547 100644 --- a/examples/src/main/python/ml/simple_params_example.py +++ b/examples/src/main/python/ml/simple_params_example.py @@ -20,11 +20,10 @@ import pprint import sys -from pyspark import SparkContext from pyspark.ml.classification import LogisticRegression from pyspark.mllib.linalg import DenseVector from pyspark.mllib.regression import LabeledPoint -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession """ A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -33,21 +32,20 @@ """ if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: simple_params_example", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonSimpleParamsExample") - sqlContext = SQLContext(sc) + spark = SparkSession \ + .builder \ + .appName("SimpleTextClassificationPipeline") \ + .getOrCreate() # prepare training data. # We create an RDD of LabeledPoints and convert them into a DataFrame. # A LabeledPoint is an Object with two fields named label and features # and Spark SQL identifies these fields and creates the schema appropriately. - training = sc.parallelize([ + training = spark.createDataFrame([ LabeledPoint(1.0, DenseVector([0.0, 1.1, 0.1])), LabeledPoint(0.0, DenseVector([2.0, 1.0, -1.0])), LabeledPoint(0.0, DenseVector([2.0, 1.3, 1.0])), - LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]).toDF() + LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]) # Create a LogisticRegression instance with maxIter = 10. # This instance is an Estimator. @@ -70,7 +68,7 @@ # We may alternatively specify parameters using a parameter map. # paramMap overrides all lr parameters set earlier. - paramMap = {lr.maxIter: 20, lr.thresholds: [0.45, 0.55], lr.probabilityCol: "myProbability"} + paramMap = {lr.maxIter: 20, lr.thresholds: [0.5, 0.5], lr.probabilityCol: "myProbability"} # Now learn a new model using the new parameters. model2 = lr.fit(training, paramMap) @@ -78,10 +76,10 @@ pprint.pprint(model2.extractParamMap()) # prepare test data. - test = sc.parallelize([ + test = spark.createDataFrame([ LabeledPoint(1.0, DenseVector([-1.0, 1.5, 1.3])), LabeledPoint(0.0, DenseVector([3.0, 2.0, -0.1])), - LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]).toDF() + LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]) # Make predictions on test data using the Transformer.transform() method. # LogisticRegressionModel.transform will only use the 'features' column. @@ -95,4 +93,4 @@ print("features=%s,label=%s -> prob=%s, prediction=%s" % (row.features, row.label, row.myProbability, row.prediction)) - sc.stop() + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 3355c8ffa2952..29f1f509608a7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -70,7 +70,7 @@ object SimpleParamsExample { // which supports several methods for specifying parameters. val paramMap = ParamMap(lr.maxIter -> 20) paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. - paramMap.put(lr.regParam -> 0.1, lr.thresholds -> Array(0.45, 0.55)) // Specify multiple Params. + paramMap.put(lr.regParam -> 0.1, lr.thresholds -> Array(0.5, 0.5)) // Specify multiple Params. // One can also combine ParamMaps. val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name From ff115f515d5d0469a1d25c3cf3eaec8c9f5dd495 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 May 2016 15:57:44 +0800 Subject: [PATCH 0269/1470] [SPARK-14939][SQL] Add FoldablePropagation optimizer ## What changes were proposed in this pull request? This PR aims to add new **FoldablePropagation** optimizer that propagates foldable expressions by replacing all attributes with the aliases of original foldable expression. Other optimizations will take advantage of the propagated foldable expressions: e.g. `EliminateSorts` optimizer now can handle the following Case 2 and 3. (Case 1 is the previous implementation.) 1. Literals and foldable expression, e.g. "ORDER BY 1.0, 'abc', Now()" 2. Foldable ordinals, e.g. "SELECT 1.0, 'abc', Now() ORDER BY 1, 2, 3" 3. Foldable aliases, e.g. "SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, z" This PR has been generalized based on cloud-fan 's key ideas many times; he should be credited for the work he did. **Before** ``` scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain == Physical Plan == WholeStageCodegen : +- Sort [1.0#5 ASC,x#0 ASC], true, 0 : +- INPUT +- Exchange rangepartitioning(1.0#5 ASC, x#0 ASC, 200), None +- WholeStageCodegen : +- Project [1.0 AS 1.0#5,1461873043577000 AS x#0] : +- INPUT +- Scan OneRowRelation[] ``` **After** ``` scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain == Physical Plan == WholeStageCodegen : +- Project [1.0 AS 1.0#5,1461873079484000 AS x#0] : +- INPUT +- Scan OneRowRelation[] ``` ## How was this patch tested? Pass the Jenkins tests including a new test case. Author: Dongjoon Hyun Closes #12719 from dongjoon-hyun/SPARK-14939. (cherry picked from commit 5907ebfc11aac8029cfc6d5f8e91cd5f53af54fe) Signed-off-by: Wenchen Fan --- .../expressions/objects/objects.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 42 +++++- .../optimizer/AggregateOptimizeSuite.scala | 21 +-- .../optimizer/EliminateSortsSuite.scala | 15 +- .../optimizer/FoldablePropagationSuite.scala | 131 ++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 8 ++ 6 files changed, 208 insertions(+), 11 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index fc38369f38c33..5e17f8920901a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -656,7 +656,7 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) extends UnaryExpression with NonSQLExpression { override def dataType: DataType = child.dataType - + override def foldable: Boolean = false override def nullable: Boolean = false override def eval(input: InternalRow): Any = 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 be9f03d4baaaf..6825b65e2b282 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 @@ -22,7 +22,7 @@ import scala.collection.immutable.HashSet import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} -import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, DistinctAggregationRewriter, EliminateSubqueryAliases, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -91,6 +91,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) CombineUnions, // Constant folding and strength reduction NullPropagation, + FoldablePropagation, OptimizeIn(conf), ConstantFolding, LikeSimplification, @@ -657,6 +658,45 @@ object NullPropagation extends Rule[LogicalPlan] { } } +/** + * Propagate foldable expressions: + * Replace attributes with aliases of the original foldable expressions if possible. + * Other optimizations will take advantage of the propagated foldable expressions. + * + * {{{ + * SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, 3 + * ==> SELECT 1.0 x, 'abc' y, Now() z ORDER BY 1.0, 'abc', Now() + * }}} + */ +object FoldablePropagation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val foldableMap = AttributeMap(plan.flatMap { + case Project(projectList, _) => projectList.collect { + case a: Alias if a.resolved && a.child.foldable => (a.toAttribute, a) + } + case _ => Nil + }) + + if (foldableMap.isEmpty) { + plan + } else { + var stop = false + CleanupAliases(plan.transformUp { + case u: Union => + stop = true + u + case c: Command => + stop = true + c + case p: LogicalPlan if !stop => p.transformExpressions { + case a: AttributeReference if foldableMap.contains(a) => + foldableMap(a) + } + }) + } + } +} + /** * Generate a list of additional filters from an operator's existing constraint but remove those * that are either already part of the operator's condition or are part of the operator's child diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala index c94dcb33546f8..4c26c184b7b5b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala @@ -34,29 +34,34 @@ class AggregateOptimizeSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Aggregate", FixedPoint(100), + FoldablePropagation, RemoveLiteralFromGroupExpressions, RemoveRepetitionFromGroupExpressions) :: Nil } + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + test("remove literals in grouping expression") { - val input = LocalRelation('a.int, 'b.int) + val query = testRelation.groupBy('a, Literal("1"), Literal(1) + Literal(2))(sum('b)) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = testRelation.groupBy('a)(sum('b)).analyze - val query = - input.groupBy('a, Literal(1), Literal(1) + Literal(2))(sum('b)) - val optimized = Optimize.execute(query) + comparePlans(optimized, correctAnswer) + } - val correctAnswer = input.groupBy('a)(sum('b)) + test("Remove aliased literals") { + val query = testRelation.select('a, Literal(1).as('y)).groupBy('a, 'y)(sum('b)) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = testRelation.select('a, Literal(1).as('y)).groupBy('a)(sum('b)).analyze comparePlans(optimized, correctAnswer) } test("remove repetition in grouping expression") { val input = LocalRelation('a.int, 'b.int, 'c.int) - val query = input.groupBy('a + 1, 'b + 2, Literal(1) + 'A, Literal(2) + 'B)(sum('c)) val optimized = Optimize.execute(analyzer.execute(query)) - - val correctAnswer = analyzer.execute(input.groupBy('a + 1, 'b + 2)(sum('c))) + val correctAnswer = input.groupBy('a + 1, 'b + 2)(sum('c)).analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala index 8c92ad82ac5be..7402918c1bbba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala @@ -34,7 +34,8 @@ class EliminateSortsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Eliminate Sorts", Once, + Batch("Eliminate Sorts", FixedPoint(10), + FoldablePropagation, EliminateSorts) :: Nil } @@ -69,4 +70,16 @@ class EliminateSortsSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("Remove no-op alias") { + val x = testRelation + + val query = x.select('a.as('x), Year(CurrentDate()).as('y), 'b) + .orderBy('x.asc, 'y.asc, 'b.desc) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = analyzer.execute( + x.select('a.as('x), Year(CurrentDate()).as('y), 'b).orderBy('x.asc, 'b.desc)) + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala new file mode 100644 index 0000000000000..355b3fc4aa637 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + +class FoldablePropagationSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Foldable Propagation", FixedPoint(20), + FoldablePropagation) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int) + + test("Propagate from subquery") { + val query = OneRowRelation + .select(Literal(1).as('a), Literal(2).as('b)) + .subquery('T) + .select('a, 'b) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = OneRowRelation + .select(Literal(1).as('a), Literal(2).as('b)) + .subquery('T) + .select(Literal(1).as('a), Literal(2).as('b)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to select clause") { + val query = testRelation + .select('a.as('x), "str".as('y), 'b.as('z)) + .select('x, 'y, 'z) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), "str".as('y), 'b.as('z)) + .select('x, "str".as('y), 'z).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to where clause") { + val query = testRelation + .select("str".as('y)) + .where('y === "str" && "str" === 'y) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select("str".as('y)) + .where("str".as('y) === "str" && "str" === "str".as('y)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to orderBy clause") { + val query = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .orderBy('x.asc, 'y.asc, 'b.desc) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .orderBy('x.asc, SortOrder(Year(CurrentDate()), Ascending), 'b.desc).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate to groupBy clause") { + val query = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .groupBy('x, 'y, 'b)(sum('x), avg('y).as('AVG), count('b)) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .groupBy('x, Year(CurrentDate()).as('y), 'b)(sum('x), avg(Year(CurrentDate())).as('AVG), + count('b)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate in a complex query") { + val query = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .where('x > 1 && 'y === 2016 && 'b > 1) + .groupBy('x, 'y, 'b)(sum('x), avg('y).as('AVG), count('b)) + .orderBy('x.asc, 'AVG.asc) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = testRelation + .select('a.as('x), Year(CurrentDate()).as('y), 'b) + .where('x > 1 && Year(CurrentDate()).as('y) === 2016 && 'b > 1) + .groupBy('x, Year(CurrentDate()).as("y"), 'b)(sum('x), avg(Year(CurrentDate())).as('AVG), + count('b)) + .orderBy('x.asc, 'AVG.asc).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Propagate in subqueries of Union queries") { + val query = Union( + Seq( + testRelation.select(Literal(1).as('x), 'a).select('x + 'a), + testRelation.select(Literal(2).as('x), 'a).select('x + 'a))) + .select('x) + val optimized = Optimize.execute(query.analyze) + val correctAnswer = Union( + Seq( + testRelation.select(Literal(1).as('x), 'a).select((Literal(1).as('x) + 'a).as("(x + a)")), + testRelation.select(Literal(2).as('x), 'a).select((Literal(2).as('x) + 'a).as("(x + a)")))) + .select('x).analyze + + comparePlans(optimized, correctAnswer) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 010dea5b30380..743a27aa7a21d 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 @@ -2499,6 +2499,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("Eliminate noop ordinal ORDER BY") { + withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "true") { + val plan1 = sql("SELECT 1.0, 'abc', year(current_date()) ORDER BY 1, 2, 3") + val plan2 = sql("SELECT 1.0, 'abc', year(current_date())") + comparePlans(plan1.queryExecution.optimizedPlan, plan2.queryExecution.optimizedPlan) + } + } + test("check code injection is prevented") { // The end of comment (*/) should be escaped. var literal = From 282a2a79cf4cbfec5697991d52577c6419bdda6c Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 10:25:33 -0700 Subject: [PATCH 0270/1470] [SPARK-15362][ML] Make spark.ml KMeansModel load backwards compatible ## What changes were proposed in this pull request? [SPARK-14646](https://issues.apache.org/jira/browse/SPARK-14646) makes ```KMeansModel``` store the cluster centers one per row. ```KMeansModel.load()``` method needs to be updated in order to load models saved with Spark 1.6. ## How was this patch tested? Since ```save/load``` is ```Experimental``` for 1.6, I think offline test for backwards compatibility is enough. Author: Yanbo Liang Closes #13149 from yanboliang/spark-15362. (cherry picked from commit 1052d3644d7eb0e784eb883293ce63a352a3b123) Signed-off-by: Joseph K. Bradley --- .../apache/spark/ml/clustering/KMeans.scala | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 41c0aec0ecf96..986f7e0fb0a5e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -185,6 +185,12 @@ object KMeansModel extends MLReadable[KMeansModel] { /** Helper class for storing model data */ private case class Data(clusterIdx: Int, clusterCenter: Vector) + /** + * We store all cluster centers in a single row and use this class to store model data by + * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. + */ + private case class OldData(clusterCenters: Array[OldVector]) + /** [[MLWriter]] instance for [[KMeansModel]] */ private[KMeansModel] class KMeansModelWriter(instance: KMeansModel) extends MLWriter { @@ -211,13 +217,19 @@ object KMeansModel extends MLReadable[KMeansModel] { import sqlContext.implicits._ val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] - val clusterCenters = data.collect().sortBy(_.clusterIdx).map(_.clusterCenter) - val model = new KMeansModel(metadata.uid, - new MLlibKMeansModel(clusterCenters.map(OldVectors.fromML))) + val versionRegex = "([0-9]+)\\.(.+)".r + val versionRegex(major, _) = metadata.sparkVersion + + val clusterCenters = if (major.toInt >= 2) { + val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] + data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) + } else { + // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. + sqlContext.read.parquet(dataPath).as[OldData].head().clusterCenters + } + val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) DefaultParamsReader.getAndSetParams(model, metadata) model } From 9f2730b0c7cefe83747e5dbc8a4e6ccbe5f72167 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 10:27:17 -0700 Subject: [PATCH 0271/1470] [SPARK-15292][ML] ML 2.0 QA: Scala APIs audit for classification ## What changes were proposed in this pull request? Audit Scala API for classification, almost all issues were related ```MultilayerPerceptronClassifier``` in this section. * Fix one wrong param getter function: ```getOptimizer``` -> ```getSolver``` * Add missing setter function for ```solver``` and ```stepSize```. * Make ```GD``` solver take effect. * Update docs, annotations and fix other minor issues. ## How was this patch tested? Existing unit tests. Author: Yanbo Liang Closes #13076 from yanboliang/spark-15292. (cherry picked from commit 8ecf7f77b2be0a178a8d94d60477876d4ab7517a) Signed-off-by: Joseph K. Bradley --- .../MultilayerPerceptronClassifier.scala | 121 +++++++++++++----- .../MultilayerPerceptronClassifierSuite.scala | 5 +- 2 files changed, 91 insertions(+), 35 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 683ae4aaf407e..c4e882240ffd2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -32,22 +32,22 @@ import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset /** Params for Multilayer Perceptron. */ -private[ml] trait MultilayerPerceptronParams extends PredictorParams +private[classification] trait MultilayerPerceptronParams extends PredictorParams with HasSeed with HasMaxIter with HasTol with HasStepSize { /** * Layer sizes including input size and output size. - * Default: Array(1, 1) * * @group param */ + @Since("1.5.0") final val layers: IntArrayParam = new IntArrayParam(this, "layers", - "Sizes of layers from input layer to output layer" + - " E.g., Array(780, 100, 10) means 780 inputs, " + + "Sizes of layers from input layer to output layer. " + + "E.g., Array(780, 100, 10) means 780 inputs, " + "one hidden layer with 100 neurons and output layer of 10 neurons.", - (t: Array[Int]) => t.forall(ParamValidators.gt(0)) && t.length > 1 - ) + (t: Array[Int]) => t.forall(ParamValidators.gt(0)) && t.length > 1) /** @group getParam */ + @Since("1.5.0") final def getLayers: Array[Int] = $(layers) /** @@ -59,42 +59,49 @@ private[ml] trait MultilayerPerceptronParams extends PredictorParams * * @group expertParam */ + @Since("1.5.0") final val blockSize: IntParam = new IntParam(this, "blockSize", "Block size for stacking input data in matrices. Data is stacked within partitions." + " If block size is more than remaining data in a partition then " + "it is adjusted to the size of this data. Recommended size is between 10 and 1000", ParamValidators.gt(0)) - /** @group getParam */ + /** @group expertGetParam */ + @Since("1.5.0") final def getBlockSize: Int = $(blockSize) /** - * Allows setting the solver: minibatch gradient descent (gd) or l-bfgs. - * l-bfgs is the default one. + * The solver algorithm for optimization. + * Supported options: "gd" (minibatch gradient descent) or "l-bfgs". + * Default: "l-bfgs" * * @group expertParam */ + @Since("2.0.0") final val solver: Param[String] = new Param[String](this, "solver", - " Allows setting the solver: minibatch gradient descent (gd) or l-bfgs. " + - " l-bfgs is the default one.", - ParamValidators.inArray[String](Array("gd", "l-bfgs"))) + "The solver algorithm for optimization. Supported options: " + + s"${MultilayerPerceptronClassifier.supportedSolvers.mkString(", ")}. (Default l-bfgs)", + ParamValidators.inArray[String](MultilayerPerceptronClassifier.supportedSolvers)) - /** @group getParam */ - final def getOptimizer: String = $(solver) + /** @group expertGetParam */ + @Since("2.0.0") + final def getSolver: String = $(solver) /** - * Model weights. Can be returned either after training or after explicit setting + * The initial weights of the model. * * @group expertParam */ - final val weights: Param[Vector] = new Param[Vector](this, "weights", - " Sets the weights of the model ") - - /** @group getParam */ - final def getWeights: Vector = $(weights) + @Since("2.0.0") + final val initialWeights: Param[Vector] = new Param[Vector](this, "initialWeights", + "The initial weights of the model") + /** @group expertGetParam */ + @Since("2.0.0") + final def getInitialWeights: Vector = $(initialWeights) - setDefault(maxIter -> 100, tol -> 1e-4, blockSize -> 128, solver -> "l-bfgs", stepSize -> 0.03) + setDefault(maxIter -> 100, tol -> 1e-4, blockSize -> 128, + solver -> MultilayerPerceptronClassifier.LBFGS, stepSize -> 0.03) } /** Label to vector converter. */ @@ -145,14 +152,32 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( @Since("1.5.0") def this() = this(Identifiable.randomUID("mlpc")) - /** @group setParam */ + /** + * Sets the value of param [[layers]]. + * + * @group setParam + */ @Since("1.5.0") def setLayers(value: Array[Int]): this.type = set(layers, value) - /** @group setParam */ + /** + * Sets the value of param [[blockSize]]. + * Default is 128. + * + * @group expertSetParam + */ @Since("1.5.0") def setBlockSize(value: Int): this.type = set(blockSize, value) + /** + * Sets the value of param [[solver]]. + * Default is "l-bfgs". + * + * @group expertSetParam + */ + @Since("2.0.0") + def setSolver(value: String): this.type = set(solver, value) + /** * Set the maximum number of iterations. * Default is 100. @@ -181,12 +206,21 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( def setSeed(value: Long): this.type = set(seed, value) /** - * Sets the model weights. + * Sets the value of param [[initialWeights]]. * - * @group expertParam + * @group expertSetParam + */ + @Since("2.0.0") + def setInitialWeights(value: Vector): this.type = set(initialWeights, value) + + /** + * Sets the value of param [[stepSize]] (applicable only for solver "gd"). + * Default is 0.03. + * + * @group setParam */ @Since("2.0.0") - def setWeights(value: Vector): this.type = set(weights, value) + def setStepSize(value: Double): this.type = set(stepSize, value) @Since("1.5.0") override def copy(extra: ParamMap): MultilayerPerceptronClassifier = defaultCopy(extra) @@ -204,16 +238,26 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( val labels = myLayers.last val lpData = extractLabeledPoints(dataset) val data = lpData.map(lp => LabelConverter.encodeLabeledPoint(lp, labels)) - val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, true) + val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, softmaxOnTop = true) val trainer = new FeedForwardTrainer(topology, myLayers(0), myLayers.last) - if (isDefined(weights)) { - trainer.setWeights($(weights)) + if (isDefined(initialWeights)) { + trainer.setWeights($(initialWeights)) } else { trainer.setSeed($(seed)) } - trainer.LBFGSOptimizer - .setConvergenceTol($(tol)) - .setNumIterations($(maxIter)) + if ($(solver) == MultilayerPerceptronClassifier.LBFGS) { + trainer.LBFGSOptimizer + .setConvergenceTol($(tol)) + .setNumIterations($(maxIter)) + } else if ($(solver) == MultilayerPerceptronClassifier.GD) { + trainer.SGDOptimizer + .setNumIterations($(maxIter)) + .setConvergenceTol($(tol)) + .setStepSize($(stepSize)) + } else { + throw new IllegalArgumentException( + s"The solver $solver is not supported by MultilayerPerceptronClassifier.") + } trainer.setStackSize($(blockSize)) val mlpModel = trainer.train(data) new MultilayerPerceptronClassificationModel(uid, myLayers, mlpModel.weights) @@ -224,6 +268,15 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( object MultilayerPerceptronClassifier extends DefaultParamsReadable[MultilayerPerceptronClassifier] { + /** String name for "l-bfgs" solver. */ + private[classification] val LBFGS = "l-bfgs" + + /** String name for "gd" (minibatch gradient descent) solver. */ + private[classification] val GD = "gd" + + /** Set of solvers that MultilayerPerceptronClassifier supports. */ + private[classification] val supportedSolvers = Array(LBFGS, GD) + @Since("2.0.0") override def load(path: String): MultilayerPerceptronClassifier = super.load(path) } @@ -250,7 +303,9 @@ class MultilayerPerceptronClassificationModel private[ml] ( @Since("1.6.0") override val numFeatures: Int = layers.head - private val mlpModel = FeedForwardTopology.multiLayerPerceptron(layers, true).model(weights) + private val mlpModel = FeedForwardTopology + .multiLayerPerceptron(layers, softmaxOnTop = true) + .model(weights) /** * Returns layers in a Java List. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index 85f325f0765a4..e809dd4092afa 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -70,6 +70,7 @@ class MultilayerPerceptronClassifierSuite .setBlockSize(1) .setSeed(123L) .setMaxIter(100) + .setSolver("l-bfgs") val model = trainer.fit(dataset) val result = model.transform(dataset) val predictionAndLabels = result.select("prediction", "label").collect() @@ -93,9 +94,9 @@ class MultilayerPerceptronClassifierSuite .setMaxIter(1) .setTol(1e-6) val initialWeights = trainer.fit(dataFrame).weights - trainer.setWeights(initialWeights.copy) + trainer.setInitialWeights(initialWeights.copy) val weights1 = trainer.fit(dataFrame).weights - trainer.setWeights(initialWeights.copy) + trainer.setInitialWeights(initialWeights.copy) val weights2 = trainer.fit(dataFrame).weights assert(weights1 ~== weights2 absTol 10e-5, "Training should produce the same weights given equal initial weights and number of steps") From bd609b0b79ea9b9aa67b16d64007c242df0f1291 Mon Sep 17 00:00:00 2001 From: Pravin Gadakh Date: Thu, 19 May 2016 10:59:07 -0700 Subject: [PATCH 0272/1470] [SPARK-14613][ML] Add @Since into the matrix and vector classes in spark-mllib-local ## What changes were proposed in this pull request? This PR add `Since` annotations in `Vectors.scala` and `Matrices.scala` of spark-mllib-local. ## How was this patch tested? Scala Style Checks. Author: Pravin Gadakh Closes #13191 from pravingadakh/SPARK-14613. (cherry picked from commit 31f63ac25da43746fdef2a9477f6a79ac046112f) Signed-off-by: DB Tsai --- .../org/apache/spark/ml/linalg/Matrices.scala | 71 ++++++++++++++++--- .../org/apache/spark/ml/linalg/Vectors.scala | 37 ++++++++-- 2 files changed, 94 insertions(+), 14 deletions(-) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index 8204b5af02cff..a47526d36f1a1 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -24,21 +24,28 @@ import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, Has import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.annotation.Since + /** * Trait for a local matrix. */ +@Since("2.0.0") sealed trait Matrix extends Serializable { /** Number of rows. */ + @Since("2.0.0") def numRows: Int /** Number of columns. */ + @Since("2.0.0") def numCols: Int /** Flag that keeps track whether the matrix is transposed or not. False by default. */ + @Since("2.0.0") val isTransposed: Boolean = false /** Converts to a dense array in column major. */ + @Since("2.0.0") def toArray: Array[Double] = { val newArray = new Array[Double](numRows * numCols) foreachActive { (i, j, v) => @@ -51,18 +58,21 @@ sealed trait Matrix extends Serializable { * Returns an iterator of column vectors. * This operation could be expensive, depending on the underlying storage. */ + @Since("2.0.0") def colIter: Iterator[Vector] /** * Returns an iterator of row vectors. * This operation could be expensive, depending on the underlying storage. */ + @Since("2.0.0") def rowIter: Iterator[Vector] = this.transpose.colIter /** Converts to a breeze matrix. */ private[ml] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ + @Since("2.0.0") def apply(i: Int, j: Int): Double /** Return the index for the (i, j)-th element in the backing array. */ @@ -72,12 +82,15 @@ sealed trait Matrix extends Serializable { private[ml] def update(i: Int, j: Int, v: Double): Unit /** Get a deep copy of the matrix. */ + @Since("2.0.0") def copy: Matrix /** Transpose the Matrix. Returns a new `Matrix` instance sharing the same underlying data. */ + @Since("2.0.0") def transpose: Matrix /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + @Since("2.0.0") def multiply(y: DenseMatrix): DenseMatrix = { val C: DenseMatrix = DenseMatrix.zeros(numRows, y.numCols) BLAS.gemm(1.0, this, y, 0.0, C) @@ -85,11 +98,13 @@ sealed trait Matrix extends Serializable { } /** Convenience method for `Matrix`-`DenseVector` multiplication. For binary compatibility. */ + @Since("2.0.0") def multiply(y: DenseVector): DenseVector = { multiply(y.asInstanceOf[Vector]) } /** Convenience method for `Matrix`-`Vector` multiplication. */ + @Since("2.0.0") def multiply(y: Vector): DenseVector = { val output = new DenseVector(new Array[Double](numRows)) BLAS.gemv(1.0, this, y, 0.0, output) @@ -100,6 +115,7 @@ sealed trait Matrix extends Serializable { override def toString: String = toBreeze.toString() /** A human readable representation of the matrix with maximum lines and width */ + @Since("2.0.0") def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) /** @@ -129,11 +145,13 @@ sealed trait Matrix extends Serializable { /** * Find the number of non-zero active values. */ + @Since("2.0.0") def numNonzeros: Int /** * Find the number of values stored explicitly. These values can be zero as well. */ + @Since("2.0.0") def numActives: Int } @@ -154,10 +172,11 @@ sealed trait Matrix extends Serializable { * @param isTransposed whether the matrix is transposed. If true, `values` stores the matrix in * row major. */ -class DenseMatrix ( - val numRows: Int, - val numCols: Int, - val values: Array[Double], +@Since("2.0.0") +class DenseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val values: Array[Double], override val isTransposed: Boolean) extends Matrix { require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + @@ -178,6 +197,7 @@ class DenseMatrix ( * @param numCols number of columns * @param values matrix entries in column major */ + @Since("2.0.0") def this(numRows: Int, numCols: Int, values: Array[Double]) = this(numRows, numCols, values, false) @@ -266,6 +286,7 @@ class DenseMatrix ( * Generate a `SparseMatrix` from the given `DenseMatrix`. The new matrix will have isTransposed * set to false. */ + @Since("2.0.0") def toSparse: SparseMatrix = { val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble val colPtrs: Array[Int] = new Array[Int](numCols + 1) @@ -307,6 +328,7 @@ class DenseMatrix ( /** * Factory methods for [[org.apache.spark.ml.linalg.DenseMatrix]]. */ +@Since("2.0.0") object DenseMatrix { /** @@ -315,6 +337,7 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros */ + @Since("2.0.0") def zeros(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -327,6 +350,7 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones */ + @Since("2.0.0") def ones(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -338,6 +362,7 @@ object DenseMatrix { * @param n number of rows and columns of the matrix * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def eye(n: Int): DenseMatrix = { val identity = DenseMatrix.zeros(n, n) var i = 0 @@ -355,6 +380,7 @@ object DenseMatrix { * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -368,6 +394,7 @@ object DenseMatrix { * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -380,6 +407,7 @@ object DenseMatrix { * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` * on the diagonal */ + @Since("2.0.0") def diag(vector: Vector): DenseMatrix = { val n = vector.size val matrix = DenseMatrix.zeros(n, n) @@ -415,12 +443,13 @@ object DenseMatrix { * Compressed Sparse Row (CSR) format, where `colPtrs` behaves as rowPtrs, * and `rowIndices` behave as colIndices, and `values` are stored in row major. */ -class SparseMatrix ( - val numRows: Int, - val numCols: Int, - val colPtrs: Array[Int], - val rowIndices: Array[Int], - val values: Array[Double], +@Since("2.0.0") +class SparseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val colPtrs: Array[Int], + @Since("2.0.0") val rowIndices: Array[Int], + @Since("2.0.0") val values: Array[Double], override val isTransposed: Boolean) extends Matrix { require(values.length == rowIndices.length, "The number of row indices and values don't match! " + @@ -451,6 +480,7 @@ class SparseMatrix ( * order for each column * @param values non-zero matrix entries in column major */ + @Since("2.0.0") def this( numRows: Int, numCols: Int, @@ -550,6 +580,7 @@ class SparseMatrix ( * Generate a `DenseMatrix` from the given `SparseMatrix`. The new matrix will have isTransposed * set to false. */ + @Since("2.0.0") def toDense: DenseMatrix = { new DenseMatrix(numRows, numCols, toArray) } @@ -594,6 +625,7 @@ class SparseMatrix ( /** * Factory methods for [[org.apache.spark.ml.linalg.SparseMatrix]]. */ +@Since("2.0.0") object SparseMatrix { /** @@ -605,6 +637,7 @@ object SparseMatrix { * @param entries Array of (i, j, value) tuples * @return The corresponding `SparseMatrix` */ + @Since("2.0.0") def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = { val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1)) val numEntries = sortedEntries.size @@ -653,6 +686,7 @@ object SparseMatrix { * @param n number of rows and columns of the matrix * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def speye(n: Int): SparseMatrix = { new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0)) } @@ -722,6 +756,7 @@ object SparseMatrix { * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextDouble()) @@ -735,6 +770,7 @@ object SparseMatrix { * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextGaussian()) @@ -746,6 +782,7 @@ object SparseMatrix { * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero * `values` on the diagonal */ + @Since("2.0.0") def spdiag(vector: Vector): SparseMatrix = { val n = vector.size vector match { @@ -762,6 +799,7 @@ object SparseMatrix { /** * Factory methods for [[org.apache.spark.ml.linalg.Matrix]]. */ +@Since("2.0.0") object Matrices { /** @@ -771,6 +809,7 @@ object Matrices { * @param numCols number of columns * @param values matrix entries in column major */ + @Since("2.0.0") def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { new DenseMatrix(numRows, numCols, values) } @@ -784,6 +823,7 @@ object Matrices { * @param rowIndices the row index of the entry * @param values non-zero matrix entries in column major */ + @Since("2.0.0") def sparse( numRows: Int, numCols: Int, @@ -825,6 +865,7 @@ object Matrices { * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of zeros */ + @Since("2.0.0") def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols) /** @@ -833,6 +874,7 @@ object Matrices { * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of ones */ + @Since("2.0.0") def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols) /** @@ -840,6 +882,7 @@ object Matrices { * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def eye(n: Int): Matrix = DenseMatrix.eye(n) /** @@ -847,6 +890,7 @@ object Matrices { * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ + @Since("2.0.0") def speye(n: Int): Matrix = SparseMatrix.speye(n) /** @@ -856,6 +900,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def rand(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.rand(numRows, numCols, rng) @@ -867,6 +912,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ + @Since("2.0.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprand(numRows, numCols, density, rng) @@ -877,6 +923,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def randn(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.randn(numRows, numCols, rng) @@ -888,6 +935,7 @@ object Matrices { * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ + @Since("2.0.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprandn(numRows, numCols, density, rng) @@ -897,6 +945,7 @@ object Matrices { * @return Square `Matrix` with size `values.length` x `values.length` and `values` * on the diagonal */ + @Since("2.0.0") def diag(vector: Vector): Matrix = DenseMatrix.diag(vector) /** @@ -906,6 +955,7 @@ object Matrices { * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were horizontally concatenated */ + @Since("2.0.0") def horzcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) @@ -964,6 +1014,7 @@ object Matrices { * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were vertically concatenated */ + @Since("2.0.0") def vertcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index c0d112d2c53d8..59f9c2adba5fc 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -25,21 +25,26 @@ import scala.collection.JavaConverters._ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} +import org.apache.spark.annotation.Since + /** * Represents a numeric vector, whose index type is Int and value type is Double. * * Note: Users should not implement this interface. */ +@Since("2.0.0") sealed trait Vector extends Serializable { /** * Size of the vector. */ + @Since("2.0.0") def size: Int /** * Converts the instance to a double array. */ + @Since("2.0.0") def toArray: Array[Double] override def equals(other: Any): Boolean = { @@ -93,11 +98,13 @@ sealed trait Vector extends Serializable { * Gets the value of the ith element. * @param i index */ + @Since("2.0.0") def apply(i: Int): Double = toBreeze(i) /** * Makes a deep copy of this vector. */ + @Since("2.0.0") def copy: Vector = { throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") } @@ -109,32 +116,38 @@ sealed trait Vector extends Serializable { * the vector with type `Int`, and the second parameter is the corresponding value * with type `Double`. */ + @Since("2.0.0") def foreachActive(f: (Int, Double) => Unit): Unit /** * Number of active entries. An "active entry" is an element which is explicitly stored, * regardless of its value. Note that inactive entries have value 0. */ + @Since("2.0.0") def numActives: Int /** * Number of nonzero elements. This scans all active values and count nonzeros. */ + @Since("2.0.0") def numNonzeros: Int /** * Converts this vector to a sparse vector with all explicit zeros removed. */ + @Since("2.0.0") def toSparse: SparseVector /** * Converts this vector to a dense vector. */ + @Since("2.0.0") def toDense: DenseVector = new DenseVector(this.toArray) /** * Returns a vector in either dense or sparse format, whichever uses less storage. */ + @Since("2.0.0") def compressed: Vector = { val nnz = numNonzeros // A dense vector needs 8 * size + 8 bytes, while a sparse vector needs 12 * nnz + 20 bytes. @@ -149,6 +162,7 @@ sealed trait Vector extends Serializable { * Find the index of a maximal element. Returns the first maximal element in case of a tie. * Returns -1 if vector has length 0. */ + @Since("2.0.0") def argmax: Int } @@ -157,12 +171,14 @@ sealed trait Vector extends Serializable { * We don't use the name `Vector` because Scala imports * [[scala.collection.immutable.Vector]] by default. */ +@Since("2.0.0") object Vectors { /** * Creates a dense vector from its values. */ @varargs + @Since("2.0.0") def dense(firstValue: Double, otherValues: Double*): Vector = new DenseVector((firstValue +: otherValues).toArray) @@ -170,6 +186,7 @@ object Vectors { /** * Creates a dense vector from a double array. */ + @Since("2.0.0") def dense(values: Array[Double]): Vector = new DenseVector(values) /** @@ -179,6 +196,7 @@ object Vectors { * @param indices index array, must be strictly increasing. * @param values value array, must have the same length as indices. */ + @Since("2.0.0") def sparse(size: Int, indices: Array[Int], values: Array[Double]): Vector = new SparseVector(size, indices, values) @@ -188,6 +206,7 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ + @Since("2.0.0") def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { require(size > 0, "The size of the requested sparse vector must be greater than 0.") @@ -209,6 +228,7 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ + @Since("2.0.0") def sparse(size: Int, elements: JavaIterable[(JavaInteger, JavaDouble)]): Vector = { sparse(size, elements.asScala.map { case (i, x) => (i.intValue(), x.doubleValue()) @@ -221,6 +241,7 @@ object Vectors { * @param size vector size * @return a zero vector */ + @Since("2.0.0") def zeros(size: Int): Vector = { new DenseVector(new Array[Double](size)) } @@ -253,6 +274,7 @@ object Vectors { * @param p norm. * @return norm in L^p^ space. */ + @Since("2.0.0") def norm(vector: Vector, p: Double): Double = { 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.") @@ -305,6 +327,7 @@ object Vectors { * @param v2 second Vector. * @return squared distance between two Vectors. */ + @Since("2.0.0") def sqdist(v1: Vector, v2: Vector): Double = { require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + s"=${v2.size}.") @@ -421,7 +444,8 @@ object Vectors { /** * A dense vector represented by a value array. */ -class DenseVector (val values: Array[Double]) extends Vector { +@Since("2.0.0") +class DenseVector @Since("2.0.0") ( @Since("2.0.0") val values: Array[Double]) extends Vector { override def size: Int = values.length @@ -515,9 +539,11 @@ class DenseVector (val values: Array[Double]) extends Vector { } } +@Since("2.0.0") object DenseVector { /** Extracts the value array from a dense vector. */ + @Since("2.0.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) } @@ -528,10 +554,11 @@ object DenseVector { * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ -class SparseVector ( +@Since("2.0.0") +class SparseVector @Since("2.0.0") ( override val size: Int, - val indices: Array[Int], - val values: Array[Double]) extends Vector { + @Since("2.0.0") val indices: Array[Int], + @Since("2.0.0") val values: Array[Double]) extends Vector { require(indices.length == values.length, "Sparse vectors require that the dimension of the" + s" indices match the dimension of the values. You provided ${indices.length} indices and " + @@ -693,7 +720,9 @@ class SparseVector ( } } +@Since("2.0.0") object SparseVector { + @Since("2.0.0") def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) } From 2604eadcfad20bebe6bd73fa8da36cd631e92e55 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 11:45:18 -0700 Subject: [PATCH 0273/1470] [SPARK-15390] fix broadcast with 100 millions rows ## What changes were proposed in this pull request? When broadcast a table with more than 100 millions rows (should not ideally), the size of needed memory will overflow. This PR fix the overflow by converting it to Long when calculating the size of memory. Also add more checking in broadcast to show reasonable messages. ## How was this patch tested? Add test. Author: Davies Liu Closes #13182 from davies/fix_broadcast. (cherry picked from commit 9308bf119204015c8733fab0c2aef70ff2e41d74) Signed-off-by: Davies Liu --- .../exchange/BroadcastExchangeExec.scala | 13 +++++++++++-- .../sql/execution/joins/HashedRelation.scala | 5 +++-- .../sql/execution/joins/HashedRelationSuite.scala | 15 +++++++++++++++ 3 files changed, 29 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index b6ecd3cb065ae..d3081ba7accd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.exchange import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import org.apache.spark.broadcast +import org.apache.spark.{broadcast, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow @@ -72,9 +72,18 @@ case class BroadcastExchangeExec( val beforeCollect = System.nanoTime() // Note that we use .executeCollect() because we don't want to convert data to Scala types val input: Array[InternalRow] = child.executeCollect() + if (input.length >= 512000000) { + throw new SparkException( + s"Cannot broadcast the table with more than 512 millions rows: ${input.length} rows") + } val beforeBuild = System.nanoTime() longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 - longMetric("dataSize") += input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum + val dataSize = input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum + longMetric("dataSize") += dataSize + if (dataSize >= (8L << 30)) { + throw new SparkException( + s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") + } // Construct and broadcast the relation. val relation = mode.transform(input) 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 cb41457b6653f..cd6b97a855412 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 @@ -410,9 +410,10 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private def init(): Unit = { if (mm != null) { + require(capacity < 512000000, "Cannot broadcast more than 512 millions rows") var n = 1 while (n < capacity) n *= 2 - ensureAcquireMemory(n * 2 * 8 + (1 << 20)) + ensureAcquireMemory(n * 2L * 8 + (1 << 20)) array = new Array[Long](n * 2) mask = n * 2 - 2 page = new Array[Long](1 << 17) // 1M bytes @@ -788,7 +789,7 @@ private[joins] object LongHashedRelation { sizeEstimate: Int, taskMemoryManager: TaskMemoryManager): LongHashedRelation = { - val map: LongToUnsafeRowMap = new LongToUnsafeRowMap(taskMemoryManager, sizeEstimate) + val map = new LongToUnsafeRowMap(taskMemoryManager, sizeEstimate) val keyGenerator = UnsafeProjection.create(key) // Create a mapping of key -> rows diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index b7b08dc4b126f..a5b56541c90f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -212,4 +212,19 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { assert(longRelation.estimatedSize > (2L << 30)) longRelation.close() } + + test("build HashedRelation with more than 100 millions rows") { + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, IntegerType, false), + BoundReference(1, StringType, true))) + val unsafeRow = unsafeProj(InternalRow(0, UTF8String.fromString(" " * 100))) + val key = Seq(BoundReference(0, IntegerType, false)) + val rows = (0 until (1 << 10)).iterator.map { i => + unsafeRow.setInt(0, i % 1000000) + unsafeRow.setInt(1, i) + unsafeRow + } + val m = LongHashedRelation(rows, key, 100 << 20, mm) + m.close() + } } From 496f6d0fc189d81c3a354df37356d5518e22f35b Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 19 May 2016 11:46:11 -0700 Subject: [PATCH 0274/1470] [SPARK-14603][SQL][FOLLOWUP] Verification of Metadata Operations by Session Catalog #### What changes were proposed in this pull request? This follow-up PR is to address the remaining comments in https://github.com/apache/spark/pull/12385 The major change in this PR is to issue better error messages in PySpark by using the mechanism that was proposed by davies in https://github.com/apache/spark/pull/7135 For example, in PySpark, if we input the following statement: ```python >>> l = [('Alice', 1)] >>> df = sqlContext.createDataFrame(l) >>> df.createTempView("people") >>> df.createTempView("people") ``` Before this PR, the exception we will get is like ``` Traceback (most recent call last): File "", line 1, in File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView self._jdf.createTempView(name) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o35.createTempView. : org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException: Temporary table 'people' already exists; at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTempView(SessionCatalog.scala:324) at org.apache.spark.sql.SparkSession.createTempView(SparkSession.scala:523) at org.apache.spark.sql.Dataset.createTempView(Dataset.scala:2328) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) at py4j.Gateway.invoke(Gateway.java:280) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:211) at java.lang.Thread.run(Thread.java:745) ``` After this PR, the exception we will get become cleaner: ``` Traceback (most recent call last): File "", line 1, in File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView self._jdf.createTempView(name) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 75, in deco raise AnalysisException(s.split(': ', 1)[1], stackTrace) pyspark.sql.utils.AnalysisException: u"Temporary table 'people' already exists;" ``` #### How was this patch tested? Fixed an existing PySpark test case Author: gatorsmile Closes #13126 from gatorsmile/followup-14684. --- python/pyspark/sql/dataframe.py | 3 +-- python/pyspark/sql/utils.py | 10 +++++++++- .../spark/sql/catalyst/catalog/InMemoryCatalog.scala | 4 ++-- .../spark/sql/catalyst/catalog/SessionCatalog.scala | 4 ++-- 4 files changed, 14 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index a0264ce1acf35..a68ef33d3999c 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -144,8 +144,7 @@ def createTempView(self, name): >>> df.createTempView("people") # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... - Py4JJavaError: ... - : org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException... + AnalysisException: u"Temporary table 'people' already exists;" >>> spark.catalog.dropTempView("people") """ diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 36c93228b94af..8c8768f50bfde 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -51,6 +51,12 @@ class ContinuousQueryException(CapturedException): """ +class QueryExecutionException(CapturedException): + """ + Failed to execute a query. + """ + + def capture_sql_exception(f): def deco(*a, **kw): try: @@ -61,12 +67,14 @@ def deco(*a, **kw): e.java_exception.getStackTrace())) if s.startswith('org.apache.spark.sql.AnalysisException: '): raise AnalysisException(s.split(': ', 1)[1], stackTrace) - if s.startswith('org.apache.spark.sql.catalyst.analysis.NoSuchTableException: '): + if s.startswith('org.apache.spark.sql.catalyst.analysis'): raise AnalysisException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.catalyst.parser.ParseException: '): raise ParseException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.ContinuousQueryException: '): raise ContinuousQueryException(s.split(': ', 1)[1], stackTrace) + if s.startswith('org.apache.spark.sql.execution.QueryExecutionException: '): + raise QueryExecutionException(s.split(': ', 1)[1], stackTrace) if s.startswith('java.lang.IllegalArgumentException: '): raise IllegalArgumentException(s.split(': ', 1)[1], stackTrace) raise diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 21da55cbc375c..489a1c8c3facd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -87,7 +87,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E db: String, table: String, specs: Seq[TablePartitionSpec]): Unit = { - specs foreach { s => + specs.foreach { s => if (!partitionExists(db, table, s)) { throw new NoSuchPartitionException(db = db, table = table, spec = s) } @@ -98,7 +98,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E db: String, table: String, specs: Seq[TablePartitionSpec]): Unit = { - specs foreach { s => + specs.foreach { s => if (partitionExists(db, table, s)) { throw new PartitionAlreadyExistsException(db = db, table = table, spec = s) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 54b30d3898746..cf9286e6b97a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -111,13 +111,13 @@ class SessionCatalog( fs.makeQualified(hadoopPath) } - protected[this] def requireDbExists(db: String): Unit = { + private def requireDbExists(db: String): Unit = { if (!databaseExists(db)) { throw new NoSuchDatabaseException(db) } } - protected[this] def requireTableExists(name: TableIdentifier): Unit = { + private def requireTableExists(name: TableIdentifier): Unit = { if (!tableExists(name)) { val db = name.database.getOrElse(currentDb) throw new NoSuchTableException(db = db, table = name.table) From 96a473a1187b5d21c02d4af01470347a942cc854 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 11:47:17 -0700 Subject: [PATCH 0275/1470] [SPARK-15300] Fix writer lock conflict when remove a block ## What changes were proposed in this pull request? A writer lock could be acquired when 1) create a new block 2) remove a block 3) evict a block to disk. 1) and 3) could happen in the same time within the same task, all of them could happen in the same time outside a task. It's OK that when someone try to grab the write block for a block, but the block is acquired by another one that has the same task attempt id. This PR remove the check. ## How was this patch tested? Updated existing tests. Author: Davies Liu Closes #13082 from davies/write_lock_conflict. (cherry picked from commit ad182086cc3bd7951aaf82693d9bcb56815b43e4) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/storage/BlockInfoManager.scala | 5 +---- .../org/apache/spark/storage/BlockInfoManagerSuite.scala | 6 ++---- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index ca53534b61c4a..20ffe1342e509 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -228,10 +228,7 @@ private[storage] class BlockInfoManager extends Logging { infos.get(blockId) match { case None => return None case Some(info) => - if (info.writerTask == currentTaskAttemptId) { - throw new IllegalStateException( - s"Task $currentTaskAttemptId has already locked $blockId for writing") - } else if (info.writerTask == BlockInfo.NO_WRITER && info.readerCount == 0) { + if (info.writerTask == BlockInfo.NO_WRITER && info.readerCount == 0) { info.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 9ee83b76e71dc..1b325801e27fc 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -208,16 +208,14 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } } - test("cannot call lockForWriting while already holding a write lock") { + test("cannot grab a writer lock while already holding a write lock") { withTaskId(0) { assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) blockInfoManager.unlock("block") } withTaskId(1) { assert(blockInfoManager.lockForWriting("block").isDefined) - intercept[IllegalStateException] { - blockInfoManager.lockForWriting("block") - } + assert(blockInfoManager.lockForWriting("block", false).isEmpty) blockInfoManager.assertBlockIsLockedForWriting("block") } } From 9c817d027713859cac483b4baaaf8b53c040ad93 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 19 May 2016 11:51:59 -0700 Subject: [PATCH 0276/1470] [SPARK-15387][SQL] SessionCatalog in SimpleAnalyzer does not need to make database directory. ## What changes were proposed in this pull request? After #12871 is fixed, we are forced to make `/user/hive/warehouse` when SimpleAnalyzer is used but SimpleAnalyzer may not need the directory. ## How was this patch tested? Manual test. Author: Kousuke Saruta Closes #13175 from sarutak/SPARK-15387. --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++-- 1 file changed, 4 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 1313a011c69c2..2c269478ee7ef 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 @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogRelation, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -45,7 +45,9 @@ object SimpleAnalyzer extends Analyzer( new SessionCatalog( new InMemoryCatalog, EmptyFunctionRegistry, - new SimpleCatalystConf(caseSensitiveAnalysis = true)), + new SimpleCatalystConf(caseSensitiveAnalysis = true)) { + override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean) {} + }, new SimpleCatalystConf(caseSensitiveAnalysis = true)) /** From 554e0f30a7fd9fae4282d93ec4c0f1c6dbdffabe Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 19 May 2016 11:54:50 -0700 Subject: [PATCH 0277/1470] [SPARK-15322][SQL][FOLLOW-UP] Update deprecated accumulator usage into accumulatorV2 ## What changes were proposed in this pull request? This PR corrects another case that uses deprecated `accumulableCollection` to use `listAccumulator`, which seems the previous PR missed. Since `ArrayBuffer[InternalRow].asJava` is `java.util.List[InternalRow]`, it seems ok to replace the usage. ## How was this patch tested? Related existing tests `InMemoryColumnarQuerySuite` and `CachedTableSuite`. Author: hyukjinkwon Closes #13187 from HyukjinKwon/SPARK-15322. (cherry picked from commit f5065abf49dea0eac04b0ec219f2d832a0f6730a) Signed-off-by: Andrew Or --- .../columnar/InMemoryTableScanExec.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 009fbaa006574..ba61940b3d5a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.execution.columnar -import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ import org.apache.commons.lang.StringUtils -import org.apache.spark.{Accumulable, Accumulator} +import org.apache.spark.Accumulator import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.AccumulatorContext +import org.apache.spark.util.{AccumulatorContext, ListAccumulator} private[sql] object InMemoryRelation { @@ -67,14 +67,14 @@ private[sql] case class InMemoryRelation( tableName: Option[String])( @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, @transient private[sql] var _statistics: Statistics = null, - private[sql] var _batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] = null) + private[sql] var _batchStats: ListAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { override def producedAttributes: AttributeSet = outputSet - private[sql] val batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] = + private[sql] val batchStats: ListAccumulator[InternalRow] = if (_batchStats == null) { - child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[InternalRow]) + child.sqlContext.sparkContext.listAccumulator[InternalRow] } else { _batchStats } @@ -87,7 +87,7 @@ private[sql] case class InMemoryRelation( output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), partitionStatistics.schema) - batchStats.value.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum + batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum } // Statistics propagation contracts: @@ -169,7 +169,7 @@ private[sql] case class InMemoryRelation( val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) .flatMap(_.values)) - batchStats += stats + batchStats.add(stats) CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) From 97fd9a09ce1313ad7b9569fc3ca8e944d36d0ce9 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 19 May 2016 20:59:19 +0200 Subject: [PATCH 0278/1470] [SPARK-15316][PYSPARK][ML] Add linkPredictionCol to GeneralizedLinearRegression ## What changes were proposed in this pull request? Add linkPredictionCol to GeneralizedLinearRegression and fix the PyDoc to generate the bullet list ## How was this patch tested? doctests & built docs locally Author: Holden Karau Closes #13106 from holdenk/SPARK-15316-add-linkPredictionCol-toGeneralizedLinearRegression. (cherry picked from commit e71cd96bf733f0440f818c6efc7a04b68d7cbe45) Signed-off-by: Nick Pentreath --- python/pyspark/ml/regression.py | 46 +++++++++++++++++++++++++-------- 1 file changed, 35 insertions(+), 11 deletions(-) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index cfcbbfc98e821..25640b1f85250 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1245,10 +1245,14 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha predictor (link function) and a description of the error distribution (family). It supports "gaussian", "binomial", "poisson" and "gamma" as family. Valid link functions for each family is listed below. The first link function of each family is the default one. - - "gaussian" -> "identity", "log", "inverse" - - "binomial" -> "logit", "probit", "cloglog" - - "poisson" -> "log", "identity", "sqrt" - - "gamma" -> "inverse", "identity", "log" + + * "gaussian" -> "identity", "log", "inverse" + + * "binomial" -> "logit", "probit", "cloglog" + + * "poisson" -> "log", "identity", "sqrt" + + * "gamma" -> "inverse", "identity", "log" .. seealso:: `GLM `_ @@ -1258,9 +1262,12 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha ... (1.0, Vectors.dense(1.0, 2.0)), ... (2.0, Vectors.dense(0.0, 0.0)), ... (2.0, Vectors.dense(1.0, 1.0)),], ["label", "features"]) - >>> glr = GeneralizedLinearRegression(family="gaussian", link="identity") + >>> glr = GeneralizedLinearRegression(family="gaussian", link="identity", linkPredictionCol="p") >>> model = glr.fit(df) - >>> abs(model.transform(df).head().prediction - 1.5) < 0.001 + >>> transformed = model.transform(df) + >>> abs(transformed.head().prediction - 1.5) < 0.001 + True + >>> abs(transformed.head().p - 1.5) < 0.001 True >>> model.coefficients DenseVector([1.5..., -1.0...]) @@ -1290,20 +1297,23 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha "relationship between the linear predictor and the mean of the distribution " + "function. Supported options: identity, log, inverse, logit, probit, cloglog " + "and sqrt.", typeConverter=TypeConverters.toString) + linkPredictionCol = Param(Params._dummy(), "linkPredictionCol", "link prediction (linear " + + "predictor) column name", typeConverter=TypeConverters.toString) @keyword_only def __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, - regParam=0.0, weightCol=None, solver="irls"): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=""): """ __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ - regParam=0.0, weightCol=None, solver="irls") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol="") """ super(GeneralizedLinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.GeneralizedLinearRegression", self.uid) - self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls") + self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", + linkPredictionCol="") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -1311,11 +1321,11 @@ def __init__(self, labelCol="label", featuresCol="features", predictionCol="pred @since("2.0.0") def setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, - regParam=0.0, weightCol=None, solver="irls"): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=""): """ setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ - regParam=0.0, weightCol=None, solver="irls") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol="") Sets params for generalized linear regression. """ kwargs = self.setParams._input_kwargs @@ -1338,6 +1348,20 @@ def getFamily(self): """ return self.getOrDefault(self.family) + @since("2.0.0") + def setLinkPredictionCol(self, value): + """ + Sets the value of :py:attr:`linkPredictionCol`. + """ + return self._set(linkPredictionCol=value) + + @since("2.0.0") + def getLinkPredictionCol(self): + """ + Gets the value of linkPredictionCol or its default value. + """ + return self.getOrDefault(self.linkPredictionCol) + @since("2.0.0") def setLink(self, value): """ From 4f8639f9d4144a5e2b1eb24a98e4c752c2dc00b8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 19 May 2016 12:02:41 -0700 Subject: [PATCH 0279/1470] [SPARK-14346][SQL] Lists unsupported Hive features in SHOW CREATE TABLE output ## What changes were proposed in this pull request? This PR is a follow-up of #13079. It replaces `hasUnsupportedFeatures: Boolean` in `CatalogTable` with `unsupportedFeatures: Seq[String]`, which contains unsupported Hive features of the underlying Hive table. In this way, we can accurately report all unsupported Hive features in the exception message. ## How was this patch tested? Updated existing test case to check exception message. Author: Cheng Lian Closes #13173 from liancheng/spark-14346-follow-up. (cherry picked from commit 6ac1c3a040f88fae15c46acd73e7e3691f7d3619) Signed-off-by: Andrew Or --- .../sql/catalyst/catalog/interface.scala | 9 +++------ .../spark/sql/execution/command/tables.scala | 14 +++++++------- .../sql/hive/client/HiveClientImpl.scala | 19 ++++++++++++++----- .../spark/sql/hive/ShowCreateTableSuite.scala | 8 +++++--- 4 files changed, 29 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 3fdd411ac4cc9..4a073d11893c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -80,11 +80,8 @@ case class CatalogTablePartition( * Note that Hive's metastore also tracks skewed columns. We should consider adding that in the * future once we have a better understanding of how we want to handle skewed columns. * - * @param hasUnsupportedFeatures is used to indicate whether all table metadata entries retrieved - * from the concrete underlying external catalog (e.g. Hive metastore) are supported by - * Spark SQL. For example, if the underlying Hive table has skewed columns, this information - * can't be mapped to [[CatalogTable]] since Spark SQL doesn't handle skewed columns for now. - * In this case `hasUnsupportedFeatures` is set to true. By default, it is false. + * @param unsupportedFeatures is a list of string descriptions of features that are used by the + * underlying table but not supported by Spark SQL yet. */ case class CatalogTable( identifier: TableIdentifier, @@ -102,7 +99,7 @@ case class CatalogTable( viewOriginalText: Option[String] = None, viewText: Option[String] = None, comment: Option[String] = None, - hasUnsupportedFeatures: Boolean = false) { + unsupportedFeatures: Seq[String] = Seq.empty) { // Verify that the provided columns are part of the schema private val colNames = schema.map(_.name).toSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 1fc02d1d4b469..a3472745371f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -633,16 +633,16 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman } private def showCreateHiveTable(metadata: CatalogTable): String = { - def reportUnsupportedError(): Unit = { - throw new UnsupportedOperationException( + def reportUnsupportedError(features: Seq[String]): Unit = { + throw new AnalysisException( s"Failed to execute SHOW CREATE TABLE against table ${metadata.identifier.quotedString}, " + - "because it contains table structure(s) (e.g. skewed columns) that Spark SQL doesn't " + - "support yet." + "which is created by Hive and uses the following unsupported feature(s)\n" + + features.map(" - " + _).mkString("\n") ) } - if (metadata.hasUnsupportedFeatures) { - reportUnsupportedError() + if (metadata.unsupportedFeatures.nonEmpty) { + reportUnsupportedError(metadata.unsupportedFeatures) } val builder = StringBuilder.newBuilder @@ -651,7 +651,7 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman case EXTERNAL => " EXTERNAL TABLE" case VIEW => " VIEW" case MANAGED => " TABLE" - case INDEX => reportUnsupportedError() + case INDEX => reportUnsupportedError(Seq("index table")) } builder ++= s"CREATE$tableTypeString ${table.quotedString}" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index af2850d4f568c..0f0c1b0702fb1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -337,10 +337,19 @@ private[hive] class HiveClientImpl( val schema = h.getCols.asScala.map(fromHiveColumn) ++ partCols // Skew spec, storage handler, and bucketing info can't be mapped to CatalogTable (yet) - val hasUnsupportedFeatures = - !h.getSkewedColNames.isEmpty || - h.getStorageHandler != null || - !h.getBucketCols.isEmpty + val unsupportedFeatures = ArrayBuffer.empty[String] + + if (!h.getSkewedColNames.isEmpty) { + unsupportedFeatures += "skewed columns" + } + + if (h.getStorageHandler != null) { + unsupportedFeatures += "storage handler" + } + + if (!h.getBucketCols.isEmpty) { + unsupportedFeatures += "bucketing" + } CatalogTable( identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), @@ -369,7 +378,7 @@ private[hive] class HiveClientImpl( properties = h.getParameters.asScala.toMap, viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), - hasUnsupportedFeatures = hasUnsupportedFeatures) + unsupportedFeatures = unsupportedFeatures) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala index 3b8068d3bc478..dedc8f55f01ba 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -247,7 +247,7 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing } } - test("hive bucketing not supported") { + test("hive bucketing is not supported") { withTable("t1") { createRawHiveTable( s"""CREATE TABLE t1 (a INT, b STRING) @@ -257,9 +257,11 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing """.stripMargin ) - intercept[UnsupportedOperationException] { + val cause = intercept[AnalysisException] { sql("SHOW CREATE TABLE t1") } + + assert(cause.getMessage.contains(" - bucketing")) } } From 62e5158f167425b9565a2b01fc494e593c57cae3 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 12:05:17 -0700 Subject: [PATCH 0280/1470] [SPARK-15317][CORE] Don't store accumulators for every task in listeners ## What changes were proposed in this pull request? In general, the Web UI doesn't need to store the Accumulator/AccumulableInfo for every task. It only needs the Accumulator values. In this PR, it creates new UIData classes to store the necessary fields and make `JobProgressListener` store only these new classes, so that `JobProgressListener` won't store Accumulator/AccumulableInfo and the size of `JobProgressListener` becomes pretty small. I also eliminates `AccumulableInfo` from `SQLListener` so that we don't keep any references for those unused `AccumulableInfo`s. ## How was this patch tested? I ran two tests reported in JIRA locally: The first one is: ``` val data = spark.range(0, 10000, 1, 10000) data.cache().count() ``` The retained size of JobProgressListener decreases from 60.7M to 6.9M. The second one is: ``` import org.apache.spark.ml.CC import org.apache.spark.sql.SQLContext val sqlContext = SQLContext.getOrCreate(sc) CC.runTest(sqlContext) ``` This test won't cause OOM after applying this patch. Author: Shixiong Zhu Closes #13153 from zsxwing/memory. (cherry picked from commit 4e3cb7a5d965fd490390398ecfe35f1fc05e8511) Signed-off-by: Andrew Or --- .../status/api/v1/AllStagesResource.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 12 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../org/apache/spark/ui/jobs/UIData.scala | 136 +++++++++++++++++- .../org/apache/spark/util/AccumulatorV2.scala | 3 + .../api/v1/AllStagesResourceSuite.scala | 4 +- .../ui/jobs/JobProgressListenerSuite.scala | 29 +++- .../sql/execution/metric/SQLMetrics.scala | 8 +- .../spark/sql/execution/ui/SQLListener.scala | 13 +- .../execution/metric/SQLMetricsSuite.scala | 7 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- 11 files changed, 186 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index eddc36edc9611..7d63a8f734f0e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -20,10 +20,10 @@ import java.util.{Arrays, Date, List => JList} import javax.ws.rs.{GET, Produces, QueryParam} import javax.ws.rs.core.MediaType -import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} +import org.apache.spark.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} import org.apache.spark.util.Distribution @Produces(Array(MediaType.APPLICATION_JSON)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 945830c8bf242..842f42b4c98db 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -332,7 +332,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { new StageUIData }) stageData.numActiveTasks += 1 - stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo, Some(metrics))) + stageData.taskData.put(taskInfo.taskId, TaskUIData(taskInfo, Some(metrics))) } for ( activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); @@ -395,9 +395,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) } - val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) - taskData.taskInfo = info - taskData.metrics = taskMetrics + val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) + taskData.updateTaskInfo(info) + taskData.updateTaskMetrics(taskMetrics) taskData.errorMessage = errorMessage for ( @@ -425,7 +425,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData: StageUIData, execId: String, taskMetrics: TaskMetrics, - oldMetrics: Option[TaskMetrics]) { + oldMetrics: Option[TaskMetricsUIData]) { val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) val shuffleWriteDelta = @@ -503,7 +503,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { if (!t.taskInfo.finished) { updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.metrics) // Overwrite task metrics - t.metrics = Some(metrics) + t.updateTaskMetrics(Some(metrics)) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 6ddabfd8ef089..d986a55959b82 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -768,7 +768,7 @@ private[ui] object StagePage { } private[ui] def getSchedulerDelay( - info: TaskInfo, metrics: TaskMetrics, currentTime: Long): Long = { + info: TaskInfo, metrics: TaskMetricsUIData, currentTime: Long): Long = { if (info.finished) { val totalExecutionTime = info.finishTime - info.launchTime val executorOverhead = (metrics.executorDeserializeTime + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index b454ef1b204b2..d76a0e657c288 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -21,8 +21,10 @@ import scala.collection.mutable import scala.collection.mutable.HashMap import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.util.AccumulatorContext import org.apache.spark.util.collection.OpenHashSet private[spark] object UIData { @@ -105,13 +107,137 @@ private[spark] object UIData { /** * These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation. */ - class TaskUIData( - var taskInfo: TaskInfo, - var metrics: Option[TaskMetrics] = None, - var errorMessage: Option[String] = None) + class TaskUIData private( + private var _taskInfo: TaskInfo, + private var _metrics: Option[TaskMetricsUIData]) { + + var errorMessage: Option[String] = None + + def taskInfo: TaskInfo = _taskInfo + + def metrics: Option[TaskMetricsUIData] = _metrics + + def updateTaskInfo(taskInfo: TaskInfo): Unit = { + _taskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) + } + + def updateTaskMetrics(metrics: Option[TaskMetrics]): Unit = { + _metrics = TaskUIData.toTaskMetricsUIData(metrics) + } + } + + object TaskUIData { + def apply(taskInfo: TaskInfo, metrics: Option[TaskMetrics]): TaskUIData = { + new TaskUIData(dropInternalAndSQLAccumulables(taskInfo), toTaskMetricsUIData(metrics)) + } + + private def toTaskMetricsUIData(metrics: Option[TaskMetrics]): Option[TaskMetricsUIData] = { + metrics.map { m => + TaskMetricsUIData( + executorDeserializeTime = m.executorDeserializeTime, + executorRunTime = m.executorRunTime, + resultSize = m.resultSize, + jvmGCTime = m.jvmGCTime, + resultSerializationTime = m.resultSerializationTime, + memoryBytesSpilled = m.memoryBytesSpilled, + diskBytesSpilled = m.diskBytesSpilled, + peakExecutionMemory = m.peakExecutionMemory, + updatedBlockStatuses = m.updatedBlockStatuses.toList, + inputMetrics = InputMetricsUIData(m.inputMetrics.bytesRead, m.inputMetrics.recordsRead), + outputMetrics = + OutputMetricsUIData(m.outputMetrics.bytesWritten, m.outputMetrics.recordsWritten), + shuffleReadMetrics = ShuffleReadMetricsUIData(m.shuffleReadMetrics), + shuffleWriteMetrics = ShuffleWriteMetricsUIData(m.shuffleWriteMetrics)) + } + } + + /** + * We don't need to store internal or SQL accumulables as their values will be shown in other + * places, so drop them to reduce the memory usage. + */ + private[spark] def dropInternalAndSQLAccumulables(taskInfo: TaskInfo): TaskInfo = { + val newTaskInfo = new TaskInfo( + taskId = taskInfo.taskId, + index = taskInfo.index, + attemptNumber = taskInfo.attemptNumber, + launchTime = taskInfo.launchTime, + executorId = taskInfo.executorId, + host = taskInfo.host, + taskLocality = taskInfo.taskLocality, + speculative = taskInfo.speculative + ) + newTaskInfo.gettingResultTime = taskInfo.gettingResultTime + newTaskInfo.accumulables ++= taskInfo.accumulables.filter { + accum => !accum.internal && accum.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) + } + newTaskInfo.finishTime = taskInfo.finishTime + newTaskInfo.failed = taskInfo.failed + newTaskInfo + } + } class ExecutorUIData( val startTime: Long, var finishTime: Option[Long] = None, var finishReason: Option[String] = None) + + case class TaskMetricsUIData( + executorDeserializeTime: Long, + executorRunTime: Long, + resultSize: Long, + jvmGCTime: Long, + resultSerializationTime: Long, + memoryBytesSpilled: Long, + diskBytesSpilled: Long, + peakExecutionMemory: Long, + updatedBlockStatuses: Seq[(BlockId, BlockStatus)], + inputMetrics: InputMetricsUIData, + outputMetrics: OutputMetricsUIData, + shuffleReadMetrics: ShuffleReadMetricsUIData, + shuffleWriteMetrics: ShuffleWriteMetricsUIData) + + case class InputMetricsUIData(bytesRead: Long, recordsRead: Long) + + case class OutputMetricsUIData(bytesWritten: Long, recordsWritten: Long) + + case class ShuffleReadMetricsUIData( + remoteBlocksFetched: Long, + localBlocksFetched: Long, + remoteBytesRead: Long, + localBytesRead: Long, + fetchWaitTime: Long, + recordsRead: Long, + totalBytesRead: Long, + totalBlocksFetched: Long) + + object ShuffleReadMetricsUIData { + def apply(metrics: ShuffleReadMetrics): ShuffleReadMetricsUIData = { + new ShuffleReadMetricsUIData( + remoteBlocksFetched = metrics.remoteBlocksFetched, + localBlocksFetched = metrics.localBlocksFetched, + remoteBytesRead = metrics.remoteBytesRead, + localBytesRead = metrics.localBytesRead, + fetchWaitTime = metrics.fetchWaitTime, + recordsRead = metrics.recordsRead, + totalBytesRead = metrics.totalBytesRead, + totalBlocksFetched = metrics.totalBlocksFetched + ) + } + } + + case class ShuffleWriteMetricsUIData( + bytesWritten: Long, + recordsWritten: Long, + writeTime: Long) + + object ShuffleWriteMetricsUIData { + def apply(metrics: ShuffleWriteMetrics): ShuffleWriteMetricsUIData = { + new ShuffleWriteMetricsUIData( + bytesWritten = metrics.bytesWritten, + recordsWritten = metrics.recordsWritten, + writeTime = metrics.writeTime + ) + } + } + } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 0cf9df084fdbe..13cb6a28c3465 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -255,6 +255,9 @@ private[spark] object AccumulatorContext { def clear(): Unit = { originals.clear() } + + // Identifier for distinguishing SQL metrics from other accumulators + private[spark] val SQL_ACCUM_IDENTIFIER = "sql" } diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala index d223af1496a4b..f684e16c25f7c 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala @@ -30,8 +30,8 @@ class AllStagesResourceSuite extends SparkFunSuite { def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = { val tasks = new HashMap[Long, TaskUIData] taskLaunchTimes.zipWithIndex.foreach { case (time, idx) => - tasks(idx.toLong) = new TaskUIData( - new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None) + tasks(idx.toLong) = TaskUIData( + new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None) } val stageUiData = new StageUIData() 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 ce7d51d1c371b..6f7c9f282a586 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 @@ -25,7 +25,8 @@ import org.apache.spark._ import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor._ import org.apache.spark.scheduler._ -import org.apache.spark.util.Utils +import org.apache.spark.ui.jobs.UIData.TaskUIData +import org.apache.spark.util.{AccumulatorContext, Utils} class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers { @@ -359,4 +360,30 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with assert( stage1Data.taskData.get(1237L).get.metrics.get.shuffleReadMetrics.totalBlocksFetched == 402) } + + test("drop internal and sql accumulators") { + val taskInfo = new TaskInfo(0, 0, 0, 0, "", "", TaskLocality.ANY, false) + val internalAccum = + AccumulableInfo(id = 1, name = Some("internal"), None, None, internal = true, false) + val sqlAccum = AccumulableInfo( + id = 2, + name = Some("sql"), + None, + None, + internal = false, + countFailedValues = false, + metadata = Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) + val userAccum = AccumulableInfo( + id = 3, + name = Some("user"), + None, + None, + internal = false, + countFailedValues = false, + metadata = None) + taskInfo.accumulables ++= Seq(internalAccum, sqlAccum, userAccum) + + val newTaskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) + assert(newTaskInfo.accumulables === Seq(userAccum)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index d6de15494fefa..e63c7c581e9b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -21,7 +21,7 @@ import java.text.NumberFormat import org.apache.spark.SparkContext import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.util.{AccumulatorV2, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, Utils} class SQLMetric(val metricType: String, initValue: Long = 0L) extends AccumulatorV2[Long, Long] { @@ -56,15 +56,13 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato // Provide special identifier as metadata so we can tell that this is a `SQLMetric` later private[spark] override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { - new AccumulableInfo(id, name, update, value, true, true, Some(SQLMetrics.ACCUM_IDENTIFIER)) + new AccumulableInfo( + id, name, update, value, true, true, Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) } } private[sql] object SQLMetrics { - // Identifier for distinguishing SQL metrics from other accumulators - private[sql] val ACCUM_IDENTIFIER = "sql" - private[sql] val SUM_METRIC = "sum" private[sql] val SIZE_METRIC = "size" private[sql] val TIMING_METRIC = "timing" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 510a2ee3bfafb..03b532664a0c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -26,6 +26,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric._ import org.apache.spark.ui.SparkUI +import org.apache.spark.util.AccumulatorContext @DeveloperApi case class SparkListenerSQLExecutionStart( @@ -177,8 +178,10 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi taskId: Long, stageId: Int, stageAttemptID: Int, - accumulatorUpdates: Seq[AccumulableInfo], + _accumulatorUpdates: Seq[AccumulableInfo], finishTask: Boolean): Unit = { + val accumulatorUpdates = + _accumulatorUpdates.filter(_.update.isDefined).map(accum => (accum.id, accum.update.get)) _stageIdToStageMetrics.get(stageId) match { case Some(stageMetrics) => @@ -290,9 +293,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi stageMetrics <- _stageIdToStageMetrics.get(stageId).toIterable; taskMetrics <- stageMetrics.taskIdToMetricUpdates.values; accumulatorUpdate <- taskMetrics.accumulatorUpdates) yield { - assert(accumulatorUpdate.update.isDefined, s"accumulator update from " + - s"task did not have a partial value: ${accumulatorUpdate.name}") - (accumulatorUpdate.id, accumulatorUpdate.update.get) + (accumulatorUpdate._1, accumulatorUpdate._2) } }.filter { case (id, _) => executionUIData.accumulatorMetrics.contains(id) } mergeAccumulatorUpdates(accumulatorUpdates, accumulatorId => @@ -336,7 +337,7 @@ private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) taskEnd.taskInfo.accumulables.flatMap { a => // Filter out accumulators that are not SQL metrics // For now we assume all SQL metrics are Long's that have been JSON serialized as String's - if (a.metadata == Some(SQLMetrics.ACCUM_IDENTIFIER)) { + if (a.metadata == Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) { val newValue = a.update.map(_.toString.toLong).getOrElse(0L) Some(a.copy(update = Some(newValue))) } else { @@ -418,4 +419,4 @@ private[ui] class SQLStageMetrics( private[ui] class SQLTaskMetrics( val attemptId: Long, // TODO not used yet var finished: Boolean, - var accumulatorUpdates: Seq[AccumulableInfo]) + var accumulatorUpdates: Seq[(Long, Any)]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 08f596f130aa2..7a89b484eb85b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -29,9 +29,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.{AccumulatorContext, JsonProtocol, Utils} class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { @@ -308,7 +307,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { case Some(v) => fail(s"metric value was not a Long: ${v.getClass.getName}") case _ => fail("metric update is missing") } - assert(metricInfo.metadata === Some(SQLMetrics.ACCUM_IDENTIFIER)) + assert(metricInfo.metadata === Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) // After serializing to JSON, the original value type is lost, but we can still // identify that it's a SQL metric from the metadata val metricInfoJson = JsonProtocol.accumulableInfoToJson(metricInfo) @@ -318,7 +317,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { case Some(v) => fail(s"deserialized metric value was not a string: ${v.getClass.getName}") case _ => fail("deserialized metric update is missing") } - assert(metricInfoDeser.metadata === Some(SQLMetrics.ACCUM_IDENTIFIER)) + assert(metricInfoDeser.metadata === Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 9eff42ab2d559..1c467137baa86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -383,7 +383,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { } // Listener tracks only SQL metrics, not other accumulators assert(trackedAccums.size === 1) - assert(trackedAccums.head === sqlMetricInfo) + assert(trackedAccums.head === (sqlMetricInfo.id, sqlMetricInfo.update.get)) } } From d1b5df83d789510340e20a98bee8fd3e0e55b8f8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 19 May 2016 12:12:42 -0700 Subject: [PATCH 0281/1470] [SPARK-15392][SQL] fix default value of size estimation of logical plan ## What changes were proposed in this pull request? We use autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD. This PR change the default value to Long.MaxValue. ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #13183 from davies/fix_default_size. (cherry picked from commit 5ccecc078aa757d3f1f6632aa6df5659490f602f) Signed-off-by: Andrew Or --- python/pyspark/sql/dataframe.py | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 6 ++-- .../org/apache/spark/sql/JoinSuite.scala | 2 +- .../apache/spark/sql/StatisticsSuite.scala | 34 +++++++++++++++++++ 4 files changed, 39 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index a68ef33d3999c..4fa799ac55bdf 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -576,7 +576,7 @@ def alias(self, alias): >>> df_as2 = df.alias("df_as2") >>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner') >>> joined_df.select("df_as1.name", "df_as2.name", "df_as2.age").collect() - [Row(name=u'Alice', name=u'Alice', age=2), Row(name=u'Bob', name=u'Bob', age=5)] + [Row(name=u'Bob', name=u'Bob', age=5), Row(name=u'Alice', name=u'Alice', age=2)] """ assert isinstance(alias, basestring), "alias should be a string" return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 248c6e3335f23..5d1868980163d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -120,8 +120,8 @@ object SQLConf { "nodes when performing a join. By setting this value to -1 broadcasting can be disabled. " + "Note that currently statistics are only supported for Hive Metastore tables where the " + "commandANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run.") - .intConf - .createWithDefault(10 * 1024 * 1024) + .longConf + .createWithDefault(10L * 1024 * 1024) val DEFAULT_SIZE_IN_BYTES = SQLConfigBuilder("spark.sql.defaultSizeInBytes") .internal() @@ -599,7 +599,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def subexpressionEliminationEnabled: Boolean = getConf(SUBEXPRESSION_ELIMINATION_ENABLED) - def autoBroadcastJoinThreshold: Int = getConf(AUTO_BROADCASTJOIN_THRESHOLD) + def autoBroadcastJoinThreshold: Long = getConf(AUTO_BROADCASTJOIN_THRESHOLD) def preferSortMergeJoin: Boolean = getConf(PREFER_SORTMERGEJOIN) 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 a6b83b3d07504..a5d8cb19eadc9 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 @@ -438,7 +438,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { spark.cacheManager.clearCache() sql("CACHE TABLE testData") - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000000000") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString) { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[BroadcastHashJoinExec]), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala new file mode 100644 index 0000000000000..9523f6f9f5bbf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ + +class StatisticsSuite extends QueryTest with SharedSQLContext { + + test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { + val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) + val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) + assert(df.queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > + spark.wrapped.conf.autoBroadcastJoinThreshold) + } + +} From 4257ba37203b406fa63b978db101e578be075bfb Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 12:32:26 -0700 Subject: [PATCH 0282/1470] Fix the compiler error introduced by #13153 for Scala 2.10 --- .../spark/ui/jobs/JobProgressListenerSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 6f7c9f282a586..1fa9b28edf4be 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 @@ -364,20 +364,20 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with test("drop internal and sql accumulators") { val taskInfo = new TaskInfo(0, 0, 0, 0, "", "", TaskLocality.ANY, false) val internalAccum = - AccumulableInfo(id = 1, name = Some("internal"), None, None, internal = true, false) + AccumulableInfo(id = 1, name = Some("internal"), None, None, true, false, None) val sqlAccum = AccumulableInfo( id = 2, name = Some("sql"), - None, - None, + update = None, + value = None, internal = false, countFailedValues = false, metadata = Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) val userAccum = AccumulableInfo( id = 3, name = Some("user"), - None, - None, + update = None, + value = None, internal = false, countFailedValues = false, metadata = None) From 833dbf9263f90b2b61dd5148249b8ec49524a8c4 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 19 May 2016 13:10:51 -0700 Subject: [PATCH 0283/1470] [SPARK-15411][ML] Add @since to ml.stat.MultivariateOnlineSummarizer.scala ## What changes were proposed in this pull request? Add since to ml.stat.MultivariateOnlineSummarizer.scala ## How was this patch tested? unit tests Author: DB Tsai Closes #13197 from dbtsai/cleanup. (cherry picked from commit 5255e55c843c7b67fcb2abb4284b8b1a09bd6672) Signed-off-by: DB Tsai --- .../ml/stat/distribution/MultivariateGaussian.scala | 11 ++++++++--- .../stat/distribution/MultivariateGaussian.scala | 4 ++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala index c62a1eab2016b..383d6d96e8d7c 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.stat.distribution import breeze.linalg.{diag, eigSym, max, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.ml.impl.Utils import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} @@ -32,9 +33,11 @@ import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} * @param mean The mean vector of the distribution * @param cov The covariance matrix of the distribution */ -class MultivariateGaussian( - val mean: Vector, - val cov: Matrix) extends Serializable { +@Since("2.0.0") +@DeveloperApi +class MultivariateGaussian @Since("2.0.0") ( + @Since("2.0.0") val mean: Vector, + @Since("2.0.0") val cov: Matrix) extends Serializable { require(cov.numCols == cov.numRows, "Covariance matrix must be square") require(mean.size == cov.numCols, "Mean vector length must match covariance matrix size") @@ -56,6 +59,7 @@ class MultivariateGaussian( /** * Returns density of this multivariate Gaussian at given point, x */ + @Since("2.0.0") def pdf(x: Vector): Double = { pdf(x.toBreeze) } @@ -63,6 +67,7 @@ class MultivariateGaussian( /** * Returns the log-density of this multivariate Gaussian at given point, x */ + @Since("2.0.0") def logpdf(x: Vector): Double = { logpdf(x.toBreeze) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index 6c6e9fb7c6b3d..c806d6bdf6b63 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -64,7 +64,7 @@ class MultivariateGaussian @Since("1.3.0") ( /** * Returns density of this multivariate Gaussian at given point, x */ - @Since("1.3.0") + @Since("1.3.0") def pdf(x: Vector): Double = { pdf(x.toBreeze) } @@ -72,7 +72,7 @@ class MultivariateGaussian @Since("1.3.0") ( /** * Returns the log-density of this multivariate Gaussian at given point, x */ - @Since("1.3.0") + @Since("1.3.0") def logpdf(x: Vector): Double = { logpdf(x.toBreeze) } From ebf30eddd5f72b3785cfa0042719b2a983e3788e Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 13:26:41 -0700 Subject: [PATCH 0284/1470] [SPARK-15361][ML] ML 2.0 QA: Scala APIs audit for ml.clustering ## What changes were proposed in this pull request? Audit Scala API for ml.clustering. Fix some wrong API documentations and update outdated one. ## How was this patch tested? Existing unit tests. Author: Yanbo Liang Closes #13148 from yanboliang/spark-15361. (cherry picked from commit 59e6c5560d13def686091391aabe024ecb43174b) Signed-off-by: Joseph K. Bradley --- .../spark/ml/clustering/BisectingKMeans.scala | 22 +++++++++++-------- .../spark/ml/clustering/GaussianMixture.scala | 22 ++++++++++++++++--- .../apache/spark/ml/clustering/KMeans.scala | 14 +++++++----- .../org/apache/spark/ml/clustering/LDA.scala | 6 ++--- 4 files changed, 43 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 138e059f94938..afb1080b9b7d5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -41,23 +41,27 @@ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter with HasFeaturesCol with HasSeed with HasPredictionCol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * The desired number of leaf clusters. Must be > 1. Default: 4. + * The actual number could be smaller if there are no divisible leaf clusters. * @group param */ @Since("2.0.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "The desired number of leaf clusters. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("2.0.0") def getK: Int = $(k) - /** @group expertParam */ + /** + * The minimum number of points (if >= 1.0) or the minimum proportion + * of points (if < 1.0) of a divisible cluster (default: 1.0). + * @group expertParam + */ @Since("2.0.0") - final val minDivisibleClusterSize = new DoubleParam( - this, - "minDivisibleClusterSize", - "the minimum number of points (if >= 1.0) or the minimum proportion", - (value: Double) => value > 0) + final val minDivisibleClusterSize = new DoubleParam(this, "minDivisibleClusterSize", + "The minimum number of points (if >= 1.0) or the minimum proportion " + + "of points (if < 1.0) of a divisible cluster.", ParamValidators.gt(0.0)) /** @group expertGetParam */ @Since("2.0.0") @@ -78,7 +82,7 @@ private[clustering] trait BisectingKMeansParams extends Params * :: Experimental :: * Model fitted by BisectingKMeans. * - * @param parentModel a model trained by spark.mllib.clustering.BisectingKMeans. + * @param parentModel a model trained by [[org.apache.spark.mllib.clustering.BisectingKMeans]]. */ @Since("2.0.0") @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 63ca812609b7e..d81b337607556 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -25,7 +25,7 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.impl.Utils.EPSILON import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param.{IntParam, ParamMap, Params} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.stat.distribution.MultivariateGaussian import org.apache.spark.ml.util._ @@ -45,11 +45,12 @@ private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter w with HasSeed with HasPredictionCol with HasProbabilityCol with HasTol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * Number of independent Gaussians in the mixture model. Must be > 1. Default: 2. * @group param */ @Since("2.0.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "Number of independent Gaussians in the mixture model. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("2.0.0") @@ -249,6 +250,21 @@ object GaussianMixtureModel extends MLReadable[GaussianMixtureModel] { /** * :: Experimental :: * Gaussian Mixture clustering. + * + * This class performs expectation maximization for multivariate Gaussian + * Mixture Models (GMMs). A GMM represents a composite distribution of + * independent Gaussian distributions with associated "mixing" weights + * specifying each's contribution to the composite. + * + * Given a set of sample points, this class will maximize the log-likelihood + * for a mixture of k Gaussians, iterating until the log-likelihood changes by + * less than convergenceTol, or until it has reached the max number of iterations. + * While this process is generally guaranteed to converge, it is not guaranteed + * to find a global optimum. + * + * Note: For high-dimensional data (with many features), this algorithm may perform poorly. + * This is due to high-dimensional data (a) making it difficult to cluster at all (based + * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. */ @Since("2.0.0") @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 986f7e0fb0a5e..0ab370e3b476e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -23,7 +23,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{Vector, VectorUDT} -import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} @@ -41,11 +41,12 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe with HasSeed with HasPredictionCol with HasTol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * The number of clusters to create (k). Must be > 1. Default: 2. * @group param */ @Since("1.5.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "The number of clusters to create. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("1.5.0") @@ -58,7 +59,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * @group expertParam */ @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "initialization algorithm", + final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + + "Supported options: 'random' and 'k-means||'.", (value: String) => MLlibKMeans.validateInitMode(value)) /** @group expertGetParam */ @@ -71,8 +73,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * @group expertParam */ @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "number of steps for k-means||", - (value: Int) => value > 0) + final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + + "initialization mode. Must be > 0.", ParamValidators.gt(0)) /** @group expertGetParam */ @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 5a83b2870005d..ec60991af64ff 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -50,8 +50,8 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM * @group param */ @Since("1.6.0") - final val k = new IntParam(this, "k", "number of topics (clusters) to infer", - ParamValidators.gt(1)) + final val k = new IntParam(this, "k", "The number of topics (clusters) to infer. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("1.6.0") @@ -165,7 +165,7 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM */ @Since("1.6.0") final val optimizer = new Param[String](this, "optimizer", "Optimizer or inference" + - " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "), + " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "), (o: String) => ParamValidators.inArray(supportedOptimizers).apply(o.toLowerCase)) /** @group getParam */ From 758253f7c7493df60e417ed5c3ee360015cfd166 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 17:24:42 -0700 Subject: [PATCH 0285/1470] [SPARK-15414][MLLIB] Make the mllib,ml linalg type conversion APIs public ## What changes were proposed in this pull request? Open up APIs for converting between new, old linear algebra types (in spark.mllib.linalg): `Sparse`/`Dense` X `Vector`/`Matrices` `.asML` and `.fromML` ## How was this patch tested? Existing Tests Author: Sandeep Singh Closes #13202 from techaddict/SPARK-15414. (cherry picked from commit ef43a5fe51614eecce2d144cc13b33004a47533a) Signed-off-by: Xiangrui Meng --- .../apache/spark/mllib/linalg/Matrices.scala | 30 +++++++++++++------ .../apache/spark/mllib/linalg/Vectors.scala | 30 +++++++++++++------ 2 files changed, 42 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 5c9a112ca69ca..ee1956c2d4135 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 @@ -164,7 +164,8 @@ sealed trait Matrix extends Serializable { * Convert this matrix to the new mllib-local representation. * This does NOT copy the data; it copies references. */ - private[spark] def asML: newlinalg.Matrix + @Since("2.0.0") + def asML: newlinalg.Matrix } private[spark] class MatrixUDT extends UserDefinedType[Matrix] { @@ -427,7 +428,8 @@ class DenseMatrix @Since("1.3.0") ( } } - private[spark] override def asML: newlinalg.DenseMatrix = { + @Since("2.0.0") + override def asML: newlinalg.DenseMatrix = { new newlinalg.DenseMatrix(numRows, numCols, values, isTransposed) } } @@ -527,8 +529,11 @@ object DenseMatrix { matrix } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.DenseMatrix): DenseMatrix = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.DenseMatrix): DenseMatrix = { new DenseMatrix(m.numRows, m.numCols, m.values, m.isTransposed) } } @@ -740,7 +745,8 @@ class SparseMatrix @Since("1.3.0") ( } } - private[spark] override def asML: newlinalg.SparseMatrix = { + @Since("2.0.0") + override def asML: newlinalg.SparseMatrix = { new newlinalg.SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) } } @@ -918,8 +924,11 @@ object SparseMatrix { } } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.SparseMatrix): SparseMatrix = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.SparseMatrix): SparseMatrix = { new SparseMatrix(m.numRows, m.numCols, m.colPtrs, m.rowIndices, m.values, m.isTransposed) } } @@ -1205,8 +1214,11 @@ object Matrices { } } - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(m: newlinalg.Matrix): Matrix = m match { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.Matrix): Matrix = m match { case dm: newlinalg.DenseMatrix => DenseMatrix.fromML(dm) case sm: newlinalg.SparseMatrix => 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 1f1cfa0cb28b4..7ebcd297bd549 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 @@ -186,7 +186,8 @@ sealed trait Vector extends Serializable { * Convert this vector to the new mllib-local representation. * This does NOT copy the data; it copies references. */ - private[spark] def asML: newlinalg.Vector + @Since("2.0.0") + def asML: newlinalg.Vector } /** @@ -581,8 +582,11 @@ object Vectors { /** Max number of nonzero entries used in computing hash code. */ private[linalg] val MAX_HASH_NNZ = 128 - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.Vector): Vector = v match { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.Vector): Vector = v match { case dv: newlinalg.DenseVector => DenseVector.fromML(dv) case sv: newlinalg.SparseVector => @@ -704,7 +708,8 @@ class DenseVector @Since("1.0.0") ( compact(render(jValue)) } - private[spark] override def asML: newlinalg.DenseVector = { + @Since("2.0.0") + override def asML: newlinalg.DenseVector = { new newlinalg.DenseVector(values) } } @@ -716,8 +721,11 @@ object DenseVector { @Since("1.3.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.DenseVector): DenseVector = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.DenseVector): DenseVector = { new DenseVector(v.values) } } @@ -911,7 +919,8 @@ class SparseVector @Since("1.0.0") ( compact(render(jValue)) } - private[spark] override def asML: newlinalg.SparseVector = { + @Since("2.0.0") + override def asML: newlinalg.SparseVector = { new newlinalg.SparseVector(size, indices, values) } } @@ -922,8 +931,11 @@ object SparseVector { def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) - /** Convert new linalg type to spark.mllib type. Light copy; only copies references */ - private[spark] def fromML(v: newlinalg.SparseVector): SparseVector = { + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.SparseVector): SparseVector = { new SparseVector(v.size, v.indices, v.values) } } From 2c939e541db67a339002204e9e0c22a611ccac85 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 19 May 2016 17:42:59 -0700 Subject: [PATCH 0286/1470] [SPARK-15375][SQL][STREAMING] Add ConsoleSink to structure streaming ## What changes were proposed in this pull request? Add ConsoleSink to structure streaming, user could use it to display dataframes on the console (useful for debugging and demostrating), similar to the functionality of `DStream#print`, to use it: ``` val query = result.write .format("console") .trigger(ProcessingTime("2 seconds")) .startStream() ``` ## How was this patch tested? local verified. Not sure it is suitable to add into structure streaming, please review and help to comment, thanks a lot. Author: jerryshao Closes #13162 from jerryshao/SPARK-15375. (cherry picked from commit dcf407de676ce99462a62f17cd98c9df56f057df) Signed-off-by: Reynold Xin --- ...pache.spark.sql.sources.DataSourceRegister | 1 + .../sql/execution/streaming/console.scala | 60 +++++++++++++++++++ .../DataFrameReaderWriterSuite.scala | 15 +++++ 3 files changed, 76 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 226d59d0eae88..ef9255794b6ab 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -3,3 +3,4 @@ org.apache.spark.sql.execution.datasources.jdbc.DefaultSource org.apache.spark.sql.execution.datasources.json.DefaultSource org.apache.spark.sql.execution.datasources.parquet.DefaultSource org.apache.spark.sql.execution.datasources.text.DefaultSource +org.apache.spark.sql.execution.streaming.ConsoleSinkProvider diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala new file mode 100644 index 0000000000000..f11a3fb969db6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.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.sql.execution.streaming + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} + +class ConsoleSink(options: Map[String, String]) extends Sink with Logging { + // Number of rows to display, by default 20 rows + private val numRowsToShow = options.get("numRows").map(_.toInt).getOrElse(20) + + // Truncate the displayed data if it is too long, by default it is true + private val isTruncated = options.get("truncate").map(_.toBoolean).getOrElse(true) + + // Track the batch id + private var lastBatchId = -1L + + override def addBatch(batchId: Long, data: DataFrame): Unit = synchronized { + val batchIdStr = if (batchId <= lastBatchId) { + s"Rerun batch: $batchId" + } else { + lastBatchId = batchId + s"Batch: $batchId" + } + + // scalastyle:off println + println("-------------------------------------------") + println(batchIdStr) + println("-------------------------------------------") + // scalastyle:off println + data.show(numRowsToShow, isTruncated) + } +} + +class ConsoleSinkProvider extends StreamSinkProvider with DataSourceRegister { + def createSink( + sqlContext: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String]): Sink = { + new ConsoleSink(parameters) + } + + def shortName(): String = "console" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala index cb53b2b1aac1d..ff53505549330 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala @@ -524,4 +524,19 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B val e = intercept[AnalysisException](w.csv("non_exist_path")) assert(e.getMessage == "csv() can only be called on non-continuous queries;") } + + test("ConsoleSink can be correctly loaded") { + LastOptions.clear() + val df = spark.read + .format("org.apache.spark.sql.streaming.test") + .stream() + + val cq = df.write + .format("console") + .option("checkpointLocation", newMetadataDir) + .trigger(ProcessingTime(2.seconds)) + .startStream() + + cq.awaitTermination(2000L) + } } From b0aff55dd4ed4edd1dd64bc42fcacdf505a6db56 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 17:54:18 -0700 Subject: [PATCH 0287/1470] [SPARK-15341][DOC][ML] Add documentation for "model.write" to clarify "summary" was not saved ## What changes were proposed in this pull request? Currently in ```model.write```, we don't save ```summary```(if applicable). We should add documentation to clarify it. We fixed the incorrect link ```[[MLWriter]]``` to ```[[org.apache.spark.ml.util.MLWriter]]``` BTW. ## How was this patch tested? Documentation update, no unit test. Author: Yanbo Liang Closes #13131 from yanboliang/spark-15341. (cherry picked from commit f8107c7846c9fcabbe2579867574305c7f2028e7) Signed-off-by: Xiangrui Meng --- .../spark/ml/classification/LogisticRegression.scala | 2 +- .../org/apache/spark/ml/clustering/GaussianMixture.scala | 7 +++++++ .../main/scala/org/apache/spark/ml/clustering/KMeans.scala | 7 +++++++ .../spark/ml/regression/GeneralizedLinearRegression.scala | 7 +++++++ .../org/apache/spark/ml/regression/LinearRegression.scala | 2 +- 5 files changed, 23 insertions(+), 2 deletions(-) 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 40cff8a86f362..27380ca5c0a7c 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 @@ -613,7 +613,7 @@ class LogisticRegressionModel private[spark] ( } /** - * Returns a [[MLWriter]] instance for this ML instance. + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. * * For [[LogisticRegressionModel]], this does NOT currently save the training [[summary]]. * An option to save [[summary]] may be added in the future. diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index d81b337607556..88b6b27e62d97 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -139,6 +139,13 @@ class GaussianMixtureModel private[ml] ( sqlContext.createDataFrame(modelGaussians).toDF("mean", "cov") } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GaussianMixtureModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ @Since("2.0.0") override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 0ab370e3b476e..790ef1fe8dc94 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -148,6 +148,13 @@ class KMeansModel private[ml] ( parentModel.computeCost(data) } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[KMeansModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ @Since("1.6.0") override def write: MLWriter = new KMeansModel.KMeansModelWriter(this) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 4aa7c2cc0b9b2..e8474d035ee62 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -776,6 +776,13 @@ class GeneralizedLinearRegressionModel private[ml] ( .setParent(parent) } + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GeneralizedLinearRegressionModel]], this does NOT currently save the + * training [[summary]]. An option to save [[summary]] may be added in the future. + * + */ @Since("2.0.0") override def write: MLWriter = new GeneralizedLinearRegressionModel.GeneralizedLinearRegressionModelWriter(this) 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 3e9a3f9db5f41..a702f02c9143c 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 @@ -449,7 +449,7 @@ class LinearRegressionModel private[ml] ( } /** - * Returns a [[MLWriter]] instance for this ML instance. + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. * * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. * An option to save [[summary]] may be added in the future. From e53a8f2188b853e02716d36dd4bfc01b9d7abc74 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 17:56:21 -0700 Subject: [PATCH 0288/1470] [MINOR][ML][PYSPARK] ml.evaluation Scala and Python API sync ## What changes were proposed in this pull request? ```ml.evaluation``` Scala and Python API sync. ## How was this patch tested? Only API docs change, no new tests. Author: Yanbo Liang Closes #13195 from yanboliang/evaluation-doc. (cherry picked from commit 664367781786df7ec52e39950dccd5a09681602c) Signed-off-by: Xiangrui Meng --- .../ml/evaluation/MulticlassClassificationEvaluator.scala | 2 +- python/pyspark/ml/evaluation.py | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 8408516751102..390e9b6444c74 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.DoubleType /** * :: Experimental :: - * Evaluator for multiclass classification, which expects two input columns: score and label. + * Evaluator for multiclass classification, which expects two input columns: prediction and label. */ @Since("1.5.0") @Experimental diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index fc9099b7ec172..16029dc34863a 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -193,9 +193,6 @@ class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): .. versionadded:: 1.4.0 """ - # Because we will maximize evaluation value (ref: `CrossValidator`), - # when we evaluate a metric that is needed to minimize (e.g., `"rmse"`, `"mse"`, `"mae"`), - # we take and output the negative of this metric. metricName = Param(Params._dummy(), "metricName", """metric name in evaluation - one of: rmse - root mean squared error (default) @@ -270,7 +267,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio """ metricName = Param(Params._dummy(), "metricName", "metric name in evaluation " - "(f1|precision|recall|weightedPrecision|weightedRecall)", + "(f1|precision|recall|weightedPrecision|weightedRecall|accuracy)", typeConverter=TypeConverters.toString) @keyword_only From 7e25131a93f2f13ecb525179bbe4ad77def84292 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 19 May 2016 18:31:05 -0700 Subject: [PATCH 0289/1470] [SPARK-15416][SQL] Display a better message for not finding classes removed in Spark 2.0 ## What changes were proposed in this pull request? If finding `NoClassDefFoundError` or `ClassNotFoundException`, check if the class name is removed in Spark 2.0. If so, the user must be using an incompatible library and we can provide a better message. ## How was this patch tested? 1. Run `bin/pyspark --packages com.databricks:spark-avro_2.10:2.0.1` 2. type `sqlContext.read.format("com.databricks.spark.avro").load("src/test/resources/episodes.avro")`. It will show `java.lang.ClassNotFoundException: org.apache.spark.sql.sources.HadoopFsRelationProvider is removed in Spark 2.0. Please check if your library is compatible with Spark 2.0` Author: Shixiong Zhu Closes #13201 from zsxwing/better-message. (cherry picked from commit 16ba71aba4e68bbb892d4ceb38d6d1d135d63fd3) Signed-off-by: Michael Armbrust --- .../execution/datasources/DataSource.scala | 61 +++++++++++++------ 1 file changed, 44 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index ce45168a13198..ccad9b3fd52fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -83,6 +83,14 @@ case class DataSource( "com.databricks.spark.csv" -> classOf[csv.DefaultSource].getCanonicalName ) + /** + * Class that were removed in Spark 2.0. Used to detect incompatibility libraries for Spark 2.0. + */ + private val spark2RemovedClasses = Set( + "org.apache.spark.sql.DataFrame", + "org.apache.spark.sql.sources.HadoopFsRelationProvider", + "org.apache.spark.Logging") + /** Given a provider name, look up the data source class definition. */ private def lookupDataSource(provider0: String): Class[_] = { val provider = backwardCompatibilityMap.getOrElse(provider0, provider0) @@ -93,26 +101,45 @@ case class DataSource( serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList match { // the provider format did not match any given registered aliases case Nil => - Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match { - case Success(dataSource) => - // Found the data source using fully qualified path - dataSource - case Failure(error) => - if (provider.startsWith("org.apache.spark.sql.hive.orc")) { - throw new ClassNotFoundException( - "The ORC data source must be used with Hive support enabled.", error) - } else { - if (provider == "avro" || provider == "com.databricks.spark.avro") { + try { + Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match { + case Success(dataSource) => + // Found the data source using fully qualified path + dataSource + case Failure(error) => + if (error.isInstanceOf[ClassNotFoundException]) { + val className = error.getMessage + if (spark2RemovedClasses.contains(className)) { + throw new ClassNotFoundException(s"$className is removed in Spark 2.0. " + + "Please check if your library is compatible with Spark 2.0") + } + } + if (provider.startsWith("org.apache.spark.sql.hive.orc")) { throw new ClassNotFoundException( - s"Failed to find data source: $provider. Please use Spark package " + - "http://spark-packages.org/package/databricks/spark-avro", - error) + "The ORC data source must be used with Hive support enabled.", error) } else { - throw new ClassNotFoundException( - s"Failed to find data source: $provider. Please find packages at " + - "http://spark-packages.org", - error) + if (provider == "avro" || provider == "com.databricks.spark.avro") { + throw new ClassNotFoundException( + s"Failed to find data source: $provider. Please use Spark package " + + "http://spark-packages.org/package/databricks/spark-avro", + error) + } else { + throw new ClassNotFoundException( + s"Failed to find data source: $provider. Please find packages at " + + "http://spark-packages.org", + error) + } } + } + } catch { + case e: NoClassDefFoundError => // This one won't be caught by Scala NonFatal + // NoClassDefFoundError's class name uses "/" rather than "." for packages + val className = e.getMessage.replaceAll("/", ".") + if (spark2RemovedClasses.contains(className)) { + throw new ClassNotFoundException(s"$className was removed in Spark 2.0. " + + "Please check if your library is compatible with Spark 2.0", e) + } else { + throw e } } case head :: Nil => From 5fa23956b23a9488c1382ee17a5b3c5113e73c87 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 20:38:44 -0700 Subject: [PATCH 0290/1470] [SPARK-15296][MLLIB] Refactor All Java Tests that use SparkSession ## What changes were proposed in this pull request? Refactor All Java Tests that use SparkSession, to extend SharedSparkSesion ## How was this patch tested? Existing Tests Author: Sandeep Singh Closes #13101 from techaddict/SPARK-15296. (cherry picked from commit 01cf649c4f96f64fb4bd09e0e1811cabcc5ead2e) Signed-off-by: Xiangrui Meng --- .../ml/JavaGaussianMixtureExample.java | 2 +- .../org/apache/spark/SharedSparkSession.java | 48 +++++++++++++++++++ .../apache/spark/ml/JavaPipelineSuite.java | 27 +++-------- .../JavaDecisionTreeClassifierSuite.java | 27 +---------- .../JavaGBTClassifierSuite.java | 28 +---------- .../JavaLogisticRegressionSuite.java | 28 +++-------- ...vaMultilayerPerceptronClassifierSuite.java | 23 +-------- .../classification/JavaNaiveBayesSuite.java | 23 +-------- .../ml/classification/JavaOneVsRestSuite.java | 30 +++--------- .../JavaRandomForestClassifierSuite.java | 28 +---------- .../spark/ml/clustering/JavaKMeansSuite.java | 27 +++-------- .../spark/ml/feature/JavaBucketizerSuite.java | 21 +------- .../apache/spark/ml/feature/JavaDCTSuite.java | 21 +------- .../spark/ml/feature/JavaHashingTFSuite.java | 21 +------- .../spark/ml/feature/JavaNormalizerSuite.java | 24 +--------- .../apache/spark/ml/feature/JavaPCASuite.java | 26 ++-------- .../feature/JavaPolynomialExpansionSuite.java | 24 +--------- .../ml/feature/JavaStandardScalerSuite.java | 24 +--------- .../ml/feature/JavaStopWordsRemoverSuite.java | 22 +-------- .../ml/feature/JavaStringIndexerSuite.java | 26 ++-------- .../spark/ml/feature/JavaTokenizerSuite.java | 24 +--------- .../ml/feature/JavaVectorAssemblerSuite.java | 26 ++-------- .../ml/feature/JavaVectorIndexerSuite.java | 25 +--------- .../ml/feature/JavaVectorSlicerSuite.java | 21 +------- .../spark/ml/feature/JavaWord2VecSuite.java | 21 +------- .../spark/ml/param/JavaParamsSuite.java | 23 --------- .../JavaDecisionTreeRegressorSuite.java | 26 +--------- .../ml/regression/JavaGBTRegressorSuite.java | 26 +--------- .../regression/JavaLinearRegressionSuite.java | 28 +++-------- .../JavaRandomForestRegressorSuite.java | 26 +--------- .../libsvm/JavaLibSVMRelationSuite.java | 20 +++----- .../ml/tuning/JavaCrossValidatorSuite.java | 33 ++++--------- .../ml/util/JavaDefaultReadWriteSuite.java | 31 +++--------- .../JavaLogisticRegressionSuite.java | 25 +--------- .../classification/JavaNaiveBayesSuite.java | 25 +--------- .../mllib/classification/JavaSVMSuite.java | 25 +--------- .../JavaStreamingLogisticRegressionSuite.java | 3 +- .../clustering/JavaBisectingKMeansSuite.java | 26 +--------- .../clustering/JavaGaussianMixtureSuite.java | 25 +--------- .../mllib/clustering/JavaKMeansSuite.java | 25 +--------- .../spark/mllib/clustering/JavaLDASuite.java | 29 +++-------- .../clustering/JavaStreamingKMeansSuite.java | 3 +- .../evaluation/JavaRankingMetricsSuite.java | 28 +++-------- .../spark/mllib/feature/JavaTfIdfSuite.java | 25 +--------- .../mllib/feature/JavaWord2VecSuite.java | 25 +--------- .../mllib/fpm/JavaAssociationRulesSuite.java | 25 +--------- .../spark/mllib/fpm/JavaFPGrowthSuite.java | 25 +--------- .../spark/mllib/fpm/JavaPrefixSpanSuite.java | 24 +--------- .../spark/mllib/linalg/JavaMatricesSuite.java | 3 +- .../spark/mllib/linalg/JavaVectorsSuite.java | 3 +- .../mllib/random/JavaRandomRDDsSuite.java | 24 +--------- .../mllib/recommendation/JavaALSSuite.java | 25 +--------- .../JavaIsotonicRegressionSuite.java | 25 +--------- .../mllib/regression/JavaLassoSuite.java | 25 +--------- .../regression/JavaLinearRegressionSuite.java | 25 +--------- .../regression/JavaRidgeRegressionSuite.java | 25 +--------- .../JavaStreamingLinearRegressionSuite.java | 3 +- .../spark/mllib/stat/JavaStatisticsSuite.java | 3 +- .../mllib/tree/JavaDecisionTreeSuite.java | 26 +--------- 59 files changed, 207 insertions(+), 1148 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/SharedSparkSession.java diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java index 79b99095815a1..526bed93fbd24 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java @@ -37,7 +37,7 @@ public class JavaGaussianMixtureExample { public static void main(String[] args) { - // Creates a SparkSession + // Creates a SparkSession SparkSession spark = SparkSession .builder() .appName("JavaGaussianMixtureExample") diff --git a/mllib/src/test/java/org/apache/spark/SharedSparkSession.java b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java new file mode 100644 index 0000000000000..43779878890db --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java @@ -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. + */ + +package org.apache.spark; + +import java.io.IOException; +import java.io.Serializable; + +import org.junit.After; +import org.junit.Before; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +public abstract class SharedSparkSession implements Serializable { + + protected transient SparkSession spark; + protected transient JavaSparkContext jsc; + + @Before + public void setUp() throws IOException { + spark = SparkSession.builder() + .master("local[2]") + .appName(getClass().getSimpleName()) + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } +} diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index a81a36d1b1c43..9b209006bc369 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -17,47 +17,34 @@ package org.apache.spark.ml; -import org.junit.After; -import org.junit.Before; +import java.io.IOException; + import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegression; import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.feature.StandardScaler; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; /** * Test Pipeline construction and fitting in Java. */ -public class JavaPipelineSuite { +public class JavaPipelineSuite extends SharedSparkSession { - private transient SparkSession spark; - private transient JavaSparkContext jsc; private transient Dataset dataset; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPipelineSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); + @Override + public void setUp() throws IOException { + super.setUp(); JavaRDD points = jsc.parallelize(generateLogisticInputAsList(1.0, 1.0, 100, 42), 2); dataset = spark.createDataFrame(points, LabeledPoint.class); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void pipeline() { StandardScaler scaler = new StandardScaler() diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java index c76a1947c64f7..5aba4e8f7de07 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -17,42 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaDecisionTreeClassifierSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDecisionTreeClassifierSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDecisionTreeClassifierSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java index 4648926c3421f..74bb46bd217a9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java @@ -17,43 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -public class JavaGBTClassifierSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaGBTClassifierSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaGBTClassifierSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index b8da04c26a49d..004102103d52c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -17,52 +17,36 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaLogisticRegressionSuite implements Serializable { +public class JavaLogisticRegressionSuite extends SharedSparkSession { - private transient SparkSession spark; - private transient JavaSparkContext jsc; private transient Dataset dataset; private transient JavaRDD datasetRDD; private double eps = 1e-5; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); dataset.createOrReplaceTempView("dataset"); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java index 48edbc838c818..6d0604d8f9a5a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java @@ -17,38 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaMultilayerPerceptronClassifierSuite implements Serializable { - - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaMultilayerPerceptronClassifierSuite extends SharedSparkSession { @Test public void testMLPC() { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java index 787909821be9b..c2a9e7b58b470 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java @@ -17,43 +17,24 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaNaiveBayesSuite implements Serializable { - - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaNaiveBayesSuite extends SharedSparkSession { public void validatePrediction(Dataset predictionAndLabels) { for (Row r : predictionAndLabels.collectAsList()) { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index 58bc5a448aed3..6194167bda354 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -17,39 +17,29 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; +import java.io.IOException; import java.util.List; import scala.collection.JavaConverters; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; -public class JavaOneVsRestSuite implements Serializable { +public class JavaOneVsRestSuite extends SharedSparkSession { - private transient SparkSession spark; - private transient JavaSparkContext jsc; private transient Dataset dataset; private transient JavaRDD datasetRDD; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLOneVsRestSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - + @Override + public void setUp() throws IOException { + super.setUp(); int nPoints = 3; // The following coefficients and xMean/xVariance are computed from iris dataset with @@ -68,12 +58,6 @@ public void setUp() { dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void oneVsRestDefaultParams() { OneVsRest ova = new OneVsRest(); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java index 1ed20b1bfa17d..dd98513f37ecf 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java @@ -17,45 +17,21 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -public class JavaRandomForestClassifierSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRandomForestClassifierSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRandomForestClassifierSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java index 9d07170fa1d5d..1be6f96f4c942 100644 --- a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java @@ -17,43 +17,30 @@ package org.apache.spark.ml.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; +import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaKMeansSuite implements Serializable { +public class JavaKMeansSuite extends SharedSparkSession { private transient int k = 5; private transient Dataset dataset; - private transient SparkSession spark; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaKMeansSuite") - .getOrCreate(); + @Override + public void setUp() throws IOException { + super.setUp(); dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void fitAndTransform() { KMeans kmeans = new KMeans().setK(k).setSeed(1); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java index a96b43de15779..87639380bdcf4 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java @@ -20,36 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaBucketizerSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaBucketizerSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaBucketizerSuite extends SharedSparkSession { @Test public void bucketizerTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index 9d8c09b30c019..b7956b6fd3e9a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -22,38 +22,21 @@ import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaDCTSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDCTSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDCTSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java index 3c37441a77beb..57696d0150a8b 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java @@ -20,38 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaHashingTFSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaHashingTFSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaHashingTFSuite extends SharedSparkSession { @Test public void hashingTF() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java index b3e213a49700e..6f877b566875c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java @@ -19,35 +19,15 @@ import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaNormalizerSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaNormalizerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaNormalizerSuite extends SharedSparkSession { @Test public void normalizer() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index a4bce2283b860..ac479c08418ce 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -23,13 +23,11 @@ import scala.Tuple2; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.Vectors; @@ -37,26 +35,8 @@ import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - -public class JavaPCASuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaPCASuite extends SharedSparkSession { public static class VectorPair implements Serializable { private Vector features = Vectors.dense(0.0); @@ -95,7 +75,7 @@ public org.apache.spark.mllib.linalg.Vector call(Vector vector) { } } ).rdd()); - + Matrix pc = mat.computePrincipalComponents(3); mat.multiply(pc).rows().toJavaRDD(); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java index a28f73f10a0b9..df5d34fbe94e6 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java @@ -20,41 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaPolynomialExpansionSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPolynomialExpansionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaPolynomialExpansionSuite extends SharedSparkSession { @Test public void polynomialExpansionTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java index 8415fdb84f213..dbc0b1db5c002 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java @@ -20,34 +20,14 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaStandardScalerSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaStandardScalerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaStandardScalerSuite extends SharedSparkSession { @Test public void standardScaler() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java index 2b156f3bca5b1..6480b57e1f796 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java @@ -20,37 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaStopWordsRemoverSuite { - - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaStopWordsRemoverSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaStopWordsRemoverSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java index 52c0bde8f3677..c1928a26b609e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java @@ -20,37 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.SparkConf; +import org.apache.spark.SharedSparkSession; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaStringIndexerSuite { - private transient SparkSession spark; - @Before - public void setUp() { - SparkConf sparkConf = new SparkConf(); - sparkConf.setMaster("local"); - sparkConf.setAppName("JavaStringIndexerSuite"); - - spark = SparkSession.builder().config(sparkConf).getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaStringIndexerSuite extends SharedSparkSession { @Test public void testStringIndexer() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java index 0bac2839e179d..27550a3d5c373 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java @@ -20,35 +20,15 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaTokenizerSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaTokenizerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaTokenizerSuite extends SharedSparkSession { @Test public void regexTokenizer() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java index fedaa77176586..583652badb8fb 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java @@ -19,40 +19,22 @@ import java.util.Arrays; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.SparkConf; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.VectorUDT; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaVectorAssemblerSuite { - private transient SparkSession spark; - @Before - public void setUp() { - SparkConf sparkConf = new SparkConf(); - sparkConf.setMaster("local"); - sparkConf.setAppName("JavaVectorAssemblerSuite"); - - spark = SparkSession.builder().config(sparkConf).getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaVectorAssemblerSuite extends SharedSparkSession { @Test public void testVectorAssembler() { 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 index a8dd44608db32..ca8fae3a48b9d 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -17,42 +17,21 @@ package org.apache.spark.ml.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaVectorIndexerSuite implements Serializable { - private transient SparkSession spark; - private JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaVectorIndexerSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaVectorIndexerSuite extends SharedSparkSession { @Test public void vectorIndexerAPI() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index a565c77af4ab1..3dc2e1f896143 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -20,11 +20,10 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; @@ -33,26 +32,10 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.StructType; -public class JavaVectorSlicerSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaVectorSlicerSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaVectorSlicerSuite extends SharedSparkSession { @Test public void vectorSlice() { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java index bef7eb0f995fb..d0a849fd11c7e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java @@ -19,34 +19,17 @@ import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; -public class JavaWord2VecSuite { - private transient SparkSession spark; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaWord2VecSuite") - .getOrCreate(); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test public void testJavaWord2Vec() { diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java index a5b5dd4088ff8..1077e103a3b89 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java @@ -19,37 +19,14 @@ import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; - /** * Test Param and related classes in Java */ public class JavaParamsSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaParamsSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void testParams() { JavaTestParams testParams = new JavaTestParams(); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java index 4ea3f2255efa7..1da85ed9dab4e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -17,43 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaDecisionTreeRegressorSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDecisionTreeRegressorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDecisionTreeRegressorSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java index 3b5edf1e15aef..7fd9b1feb7f83 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java @@ -17,43 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaGBTRegressorSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaGBTRegressorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaGBTRegressorSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index 126aa6298f3de..6cdcdda1a6480 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -17,48 +17,32 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaLinearRegressionSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; +public class JavaLinearRegressionSuite extends SharedSparkSession { private transient Dataset dataset; private transient JavaRDD datasetRDD; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLinearRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); dataset.createOrReplaceTempView("dataset"); } - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - @Test public void linearRegressionDefaultParams() { LinearRegression lr = new LinearRegression(); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java index d601e7c540e19..4ba13e2e06c8d 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java @@ -17,45 +17,23 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionSuite; import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.tree.impl.TreeTests; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -public class JavaRandomForestRegressorSuite implements Serializable { - - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRandomForestRegressorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRandomForestRegressorSuite extends SharedSparkSession { @Test public void runDT() { diff --git a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java index 022dcf94bd006..fa39f4560c8aa 100644 --- a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java @@ -23,35 +23,28 @@ import com.google.common.io.Files; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.linalg.DenseVector; import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; import org.apache.spark.util.Utils; /** * Test LibSVMRelation in Java. */ -public class JavaLibSVMRelationSuite { - private transient SparkSession spark; +public class JavaLibSVMRelationSuite extends SharedSparkSession { private File tempDir; private String path; - @Before + @Override public void setUp() throws IOException { - spark = SparkSession.builder() - .master("local") - .appName("JavaLibSVMRelationSuite") - .getOrCreate(); - + super.setUp(); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource"); File file = new File(tempDir, "part-00000"); String s = "1 1:1.0 3:2.0 5:3.0\n0\n0 2:4.0 4:5.0 6:6.0"; @@ -59,10 +52,9 @@ public void setUp() throws IOException { path = tempDir.toURI().toString(); } - @After + @Override public void tearDown() { - spark.stop(); - spark = null; + super.tearDown(); Utils.deleteRecursively(tempDir); } diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java index b874ccd48b3b1..692d5ad591e84 100644 --- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java @@ -17,48 +17,33 @@ package org.apache.spark.ml.tuning; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.classification.LogisticRegression; -import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; -import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; -public class JavaCrossValidatorSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - private transient Dataset dataset; +public class JavaCrossValidatorSuite extends SharedSparkSession { - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaCrossValidatorSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); + private transient Dataset dataset; + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); dataset = spark.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); } - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - @Test public void crossValidationWithLogisticRegression() { LogisticRegression lr = new LogisticRegression(); diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java index 7151e27cde839..da623d1d15702 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -20,42 +20,25 @@ import java.io.File; import java.io.IOException; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.SharedSparkSession; import org.apache.spark.util.Utils; -public class JavaDefaultReadWriteSuite { - - JavaSparkContext jsc = null; - SparkSession spark = null; +public class JavaDefaultReadWriteSuite extends SharedSparkSession { File tempDir = null; - @Before - public void setUp() { - SQLContext.clearActive(); - spark = SparkSession.builder() - .master("local[2]") - .appName("JavaDefaultReadWriteSuite") - .getOrCreate(); - SQLContext.setActive(spark.wrapped()); - + @Override + public void setUp() throws IOException { + super.setUp(); tempDir = Utils.createTempDir( System.getProperty("java.io.tmpdir"), "JavaDefaultReadWriteSuite"); } - @After + @Override public void tearDown() { - SQLContext.clearActive(); - if (spark != null) { - spark.stop(); - spark = null; - } + super.tearDown(); Utils.deleteRecursively(tempDir); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index 2f10d14da5e4a..c04e2e69541ba 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -17,37 +17,16 @@ package org.apache.spark.mllib.classification; -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 org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.SparkSession; -public class JavaLogisticRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLogisticRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaLogisticRegressionSuite extends SharedSparkSession { int validatePrediction(List validationData, LogisticRegressionModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 5e212e2fc5b3b..6ded42e928250 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -17,42 +17,21 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.SparkSession; -public class JavaNaiveBayesSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaNaiveBayesSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaNaiveBayesSuite extends SharedSparkSession { private static final List POINTS = Arrays.asList( new LabeledPoint(0, Vectors.dense(1.0, 0.0, 0.0)), diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 2a090c054fe2b..0f54e684e447d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -17,37 +17,16 @@ package org.apache.spark.mllib.classification; -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 org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.SparkSession; -public class JavaSVMSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaSVMSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaSVMSuite extends SharedSparkSession { int validatePrediction(List validationData, SVMModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java index 62c6d9b7e390a..8c6bced52dd74 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -37,7 +36,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLogisticRegressionSuite implements Serializable { +public class JavaStreamingLogisticRegressionSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java index 7f29b050479fc..3d62b273d2210 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java @@ -17,39 +17,17 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; - import com.google.common.collect.Lists; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; - -public class JavaBisectingKMeansSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaBisectingKMeansSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaBisectingKMeansSuite extends SharedSparkSession { @Test public void twoDimensionalData() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java index 20edd08a2172d..bf76719937772 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java @@ -17,40 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import static org.junit.Assert.assertEquals; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; -public class JavaGaussianMixtureSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaGaussianMixture") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaGaussianMixtureSuite extends SharedSparkSession { @Test public void runGaussianMixture() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 4e5b87f588e3d..270e636f82117 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -17,40 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import static org.junit.Assert.assertEquals; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; -public class JavaKMeansSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaKMeans") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaKMeansSuite extends SharedSparkSession { @Test public void runKMeansUsingStaticMethods() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index f16585aff4f5d..08d6713ab2bc3 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -17,39 +17,28 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import scala.Tuple2; import scala.Tuple3; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.*; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.SparkSession; - -public class JavaLDASuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLDASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); +public class JavaLDASuite extends SharedSparkSession { + @Override + public void setUp() throws IOException { + super.setUp(); ArrayList> tinyCorpus = new ArrayList<>(); for (int i = 0; i < LDASuite.tinyCorpus().length; i++) { tinyCorpus.add(new Tuple2<>((Long) LDASuite.tinyCorpus()[i]._1(), @@ -59,12 +48,6 @@ public void setUp() { corpus = JavaPairRDD.fromJavaRDD(tmpCorpus); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void localLDAModel() { Matrix topics = LDASuite.tinyTopics(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java index d1d618f7de2d8..d41fc0e4dca96 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -36,7 +35,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingKMeansSuite implements Serializable { +public class JavaStreamingKMeansSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java index 6a096d6386550..e9d7e4fdbe8ce 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java @@ -17,35 +17,25 @@ package org.apache.spark.mllib.evaluation; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; import scala.Tuple2; import scala.Tuple2$; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -public class JavaRankingMetricsSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; +public class JavaRankingMetricsSuite extends SharedSparkSession { private transient JavaRDD, List>> predictionAndLabels; - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - + @Override + public void setUp() throws IOException { + super.setUp(); predictionAndLabels = jsc.parallelize(Arrays.asList( Tuple2$.MODULE$.apply( Arrays.asList(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Arrays.asList(1, 2, 3, 4, 5)), @@ -55,12 +45,6 @@ public void setUp() { Arrays.asList(1, 2, 3, 4, 5), Arrays.asList())), 2); } - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void rankingMetrics() { @SuppressWarnings("unchecked") diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index de50fb8c4fdb2..05128ea343420 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -17,38 +17,17 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.SparkSession; -public class JavaTfIdfSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaTfIdfSuite extends SharedSparkSession { @Test public void tfIdf() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java index 64885cc8425d3..3e3abddbee638 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -25,33 +24,13 @@ import scala.Tuple2; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -public class JavaWord2VecSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPCASuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test @SuppressWarnings("unchecked") diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java index fdc19a5b3dc47..3451e0773759b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java @@ -16,36 +16,15 @@ */ package org.apache.spark.mllib.fpm; -import java.io.Serializable; import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset; -import org.apache.spark.sql.SparkSession; -public class JavaAssociationRulesSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaAssociationRulesSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaAssociationRulesSuite extends SharedSparkSession { @Test public void runAssociationRules() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java index f235251e61d42..46e9dd8b59828 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java @@ -18,39 +18,18 @@ package org.apache.spark.mllib.fpm; import java.io.File; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import static org.junit.Assert.assertEquals; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; import org.apache.spark.util.Utils; -public class JavaFPGrowthSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaFPGrowth") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaFPGrowthSuite extends SharedSparkSession { @Test public void runFPGrowth() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java index bf7f1fc71b08e..75b0ec6480196 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java @@ -21,35 +21,15 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.PrefixSpan.FreqSequence; -import org.apache.spark.sql.SparkSession; import org.apache.spark.util.Utils; -public class JavaPrefixSpanSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaPrefixSpan") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaPrefixSpanSuite extends SharedSparkSession { @Test public void runPrefixSpan() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java index 92fc57871cdbf..f427846b9ad10 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.linalg; -import java.io.Serializable; import java.util.Random; import static org.junit.Assert.assertArrayEquals; @@ -25,7 +24,7 @@ import org.junit.Test; -public class JavaMatricesSuite implements Serializable { +public class JavaMatricesSuite { @Test public void randMatrixConstruction() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index 817b962c75007..f67f555e418a7 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.linalg; -import java.io.Serializable; import java.util.Arrays; import static org.junit.Assert.assertArrayEquals; @@ -26,7 +25,7 @@ import org.junit.Test; -public class JavaVectorsSuite implements Serializable { +public class JavaVectorsSuite { @Test public void denseArrayConstruction() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java index b449108a9b83e..6d114024c31be 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -20,36 +20,16 @@ import java.io.Serializable; import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.SparkSession; import static org.apache.spark.mllib.random.RandomRDDs.*; -public class JavaRandomRDDsSuite { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRandomRDDsSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRandomRDDsSuite extends SharedSparkSession { @Test public void testUniformRDD() { diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index aa784054d551e..363ab42546d11 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -17,41 +17,20 @@ package org.apache.spark.mllib.recommendation; -import java.io.Serializable; import java.util.ArrayList; import java.util.List; import scala.Tuple2; import scala.Tuple3; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; - -public class JavaALSSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaALS") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaALSSuite extends SharedSparkSession { private void validatePrediction( MatrixFactorizationModel model, diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java index 8b05675d65a1d..dbd4cbfd2b746 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java @@ -17,26 +17,20 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import scala.Tuple3; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -public class JavaIsotonicRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; +public class JavaIsotonicRegressionSuite extends SharedSparkSession { private static List> generateIsotonicInput(double[] labels) { List> input = new ArrayList<>(labels.length); @@ -55,21 +49,6 @@ private IsotonicRegressionModel runIsotonicRegression(double[] labels) { return new IsotonicRegression().run(trainRDD); } - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLinearRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } - @Test public void testIsotonicRegressionJavaRDD() { IsotonicRegressionModel model = diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index 098bac3bedfff..1458cc72bc17f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -17,37 +17,16 @@ package org.apache.spark.mllib.regression; -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 org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -import org.apache.spark.sql.SparkSession; -public class JavaLassoSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLassoSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaLassoSuite extends SharedSparkSession { int validatePrediction(List validationData, LassoModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 35087a5e461df..a46b1321b3ca2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -17,39 +17,18 @@ package org.apache.spark.mllib.regression; -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 org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.util.LinearDataGenerator; -import org.apache.spark.sql.SparkSession; -public class JavaLinearRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaLinearRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaLinearRegressionSuite extends SharedSparkSession { int validatePrediction(List validationData, LinearRegressionModel model) { int numAccurate = 0; diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index b2efb2e72e374..cb00977412345 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -17,38 +17,17 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; import java.util.Random; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -import org.apache.spark.sql.SparkSession; - -public class JavaRidgeRegressionSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaRidgeRegressionSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaRidgeRegressionSuite extends SharedSparkSession { private static double predictionError(List validationData, RidgeRegressionModel model) { diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java index ea0ccd7448986..ab554475d59a1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -36,7 +35,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLinearRegressionSuite implements Serializable { +public class JavaStreamingLinearRegressionSuite { protected transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java index 373417d3ba7c0..1abaa39eadc22 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.stat; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -42,7 +41,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStatisticsSuite implements Serializable { +public class JavaStatisticsSuite { private transient SparkSession spark; private transient JavaSparkContext jsc; private transient JavaStreamingContext ssc; diff --git a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java index 5b464a4722d92..1dcbbcaa0223c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java @@ -17,17 +17,14 @@ package org.apache.spark.mllib.tree; -import java.io.Serializable; import java.util.HashMap; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.regression.LabeledPoint; @@ -35,27 +32,8 @@ import org.apache.spark.mllib.tree.configuration.Strategy; import org.apache.spark.mllib.tree.impurity.Gini; import org.apache.spark.mllib.tree.model.DecisionTreeModel; -import org.apache.spark.sql.SparkSession; - -public class JavaDecisionTreeSuite implements Serializable { - private transient SparkSession spark; - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - spark = SparkSession.builder() - .master("local") - .appName("JavaDecisionTreeSuite") - .getOrCreate(); - jsc = new JavaSparkContext(spark.sparkContext()); - } - - @After - public void tearDown() { - spark.stop(); - spark = null; - } +public class JavaDecisionTreeSuite extends SharedSparkSession { int validatePrediction(List validationData, DecisionTreeModel model) { int numCorrect = 0; From c21c69187652936e5c74ed7571f07a0b423e9193 Mon Sep 17 00:00:00 2001 From: Sumedh Mungee Date: Fri, 20 May 2016 12:30:04 +0800 Subject: [PATCH 0291/1470] [SPARK-15321] Fix bug where Array[Timestamp] cannot be encoded/decoded correctly ## What changes were proposed in this pull request? Fix `MapObjects.itemAccessorMethod` to handle `TimestampType`. Without this fix, `Array[Timestamp]` cannot be properly encoded or decoded. To reproduce this, in `ExpressionEncoderSuite`, if you add the following test case: `encodeDecodeTest(Array(Timestamp.valueOf("2016-01-29 10:00:00")), "array of timestamp") ` ... you will see that (without this fix) it fails with the following output: ``` - encode/decode for array of timestamp: [Ljava.sql.Timestamp;fd9ebde *** FAILED *** Exception thrown while decoding Converted: [0,1000000010,800000001,52a7ccdc36800] Schema: value#61615 root -- value: array (nullable = true) |-- element: timestamp (containsNull = true) Encoder: class[value[0]: array] (ExpressionEncoderSuite.scala:312) ``` ## How was this patch tested? Existing tests Author: Sumedh Mungee Closes #13108 from smungee/fix-itemAccessorMethod. (cherry picked from commit d5c47f8ff8c09ff017e896835db044661ee60909) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 177b1390b2f49..227e835e7ed51 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -115,6 +115,7 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { encodeDecodeTest("hello", "string") encodeDecodeTest(Date.valueOf("2012-12-23"), "date") encodeDecodeTest(Timestamp.valueOf("2016-01-29 10:00:00"), "timestamp") + encodeDecodeTest(Array(Timestamp.valueOf("2016-01-29 10:00:00")), "array of timestamp") encodeDecodeTest(Array[Byte](13, 21, -23), "binary") encodeDecodeTest(Seq(31, -123, 4), "seq of int") From e6810e9cd74cd8e8999ea5eca622367f55e268b1 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Fri, 20 May 2016 12:41:14 +0800 Subject: [PATCH 0292/1470] [SPARK-11827][SQL] Adding java.math.BigInteger support in Java type inference for POJOs and Java collections Hello : Can you help check this PR? I am adding support for the java.math.BigInteger for java bean code path. I saw internally spark is converting the BigInteger to BigDecimal in ColumnType.scala and CatalystRowConverter.scala. I use the similar way and convert the BigInteger to the BigDecimal. . Author: Kevin Yu Closes #10125 from kevinyu98/working_on_spark-11827. (cherry picked from commit 17591d90e6873f30a042112f56a1686726ccbd60) Signed-off-by: Wenchen Fan --- .../sql/catalyst/CatalystTypeConverters.scala | 2 ++ .../sql/catalyst/JavaTypeInference.scala | 1 + .../spark/sql/catalyst/ScalaReflection.scala | 24 +++++++++++++++ .../org/apache/spark/sql/types/Decimal.scala | 29 ++++++++++++++++++- .../apache/spark/sql/types/DecimalType.scala | 1 + .../encoders/ExpressionEncoderSuite.scala | 4 ++- .../apache/spark/sql/JavaDataFrameSuite.java | 11 ++++++- .../sql/ScalaReflectionRelationSuite.scala | 10 +++++-- 8 files changed, 76 insertions(+), 6 deletions(-) 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 9bfc381639140..9cc7b2ac79205 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst import java.lang.{Iterable => JavaIterable} import java.math.{BigDecimal => JavaBigDecimal} +import java.math.{BigInteger => JavaBigInteger} import java.sql.{Date, Timestamp} import java.util.{Map => JavaMap} import javax.annotation.Nullable @@ -326,6 +327,7 @@ object CatalystTypeConverters { val decimal = scalaValue match { case d: BigDecimal => Decimal(d) case d: JavaBigDecimal => Decimal(d) + case d: JavaBigInteger => Decimal(d) case d: Decimal => d } if (decimal.changePrecision(dataType.precision, dataType.scale)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 690758205efff..1fe143494abad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -89,6 +89,7 @@ object JavaTypeInference { case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType.SYSTEM_DEFAULT, true) + case c: Class[_] if c == classOf[java.math.BigInteger] => (DecimalType.BigIntDecimal, true) case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) 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 c0fa220d34bb6..58df651da2942 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 @@ -259,6 +259,12 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[BigDecimal] => Invoke(getPath, "toBigDecimal", ObjectType(classOf[BigDecimal])) + case t if t <:< localTypeOf[java.math.BigInteger] => + Invoke(getPath, "toJavaBigInteger", ObjectType(classOf[java.math.BigInteger])) + + case t if t <:< localTypeOf[scala.math.BigInt] => + Invoke(getPath, "toScalaBigInt", ObjectType(classOf[scala.math.BigInt])) + case t if t <:< localTypeOf[Array[_]] => val TypeRef(_, _, Seq(elementType)) = t @@ -592,6 +598,20 @@ object ScalaReflection extends ScalaReflection { "apply", inputObject :: Nil) + case t if t <:< localTypeOf[java.math.BigInteger] => + StaticInvoke( + Decimal.getClass, + DecimalType.BigIntDecimal, + "apply", + inputObject :: Nil) + + case t if t <:< localTypeOf[scala.math.BigInt] => + StaticInvoke( + Decimal.getClass, + DecimalType.BigIntDecimal, + "apply", + inputObject :: Nil) + case t if t <:< localTypeOf[java.lang.Integer] => Invoke(inputObject, "intValue", IntegerType) case t if t <:< localTypeOf[java.lang.Long] => @@ -736,6 +756,10 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if t <:< localTypeOf[java.math.BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) + case t if t <:< localTypeOf[java.math.BigInteger] => + Schema(DecimalType.BigIntDecimal, nullable = true) + case t if t <:< localTypeOf[scala.math.BigInt] => + Schema(DecimalType.BigIntDecimal, nullable = true) case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if t <:< localTypeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) case t if t <:< localTypeOf[java.lang.Long] => Schema(LongType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 2f7422b7420d0..b907f62802175 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import java.math.{MathContext, RoundingMode} +import java.math.{BigInteger, MathContext, RoundingMode} import org.apache.spark.annotation.DeveloperApi @@ -128,6 +128,23 @@ final class Decimal extends Ordered[Decimal] with Serializable { this } + /** + * Set this Decimal to the given BigInteger value. Will have precision 38 and scale 0. + */ + def set(bigintval: BigInteger): Decimal = { + try { + this.decimalVal = null + this.longVal = bigintval.longValueExact() + this._precision = DecimalType.MAX_PRECISION + this._scale = 0 + this + } + catch { + case e: ArithmeticException => + throw new IllegalArgumentException(s"BigInteger ${bigintval} too large for decimal") + } + } + /** * Set this Decimal to the given Decimal value. */ @@ -155,6 +172,10 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } + def toScalaBigInt: BigInt = BigInt(toLong) + + def toJavaBigInteger: java.math.BigInteger = java.math.BigInteger.valueOf(toLong) + def toUnscaledLong: Long = { if (decimalVal.ne(null)) { decimalVal.underlying().unscaledValue().longValue() @@ -371,6 +392,10 @@ object Decimal { def apply(value: java.math.BigDecimal): Decimal = new Decimal().set(value) + def apply(value: java.math.BigInteger): Decimal = new Decimal().set(value) + + def apply(value: scala.math.BigInt): Decimal = new Decimal().set(value.bigInteger) + def apply(value: BigDecimal, precision: Int, scale: Int): Decimal = new Decimal().set(value, precision, scale) @@ -387,6 +412,8 @@ object Decimal { value match { case j: java.math.BigDecimal => apply(j) case d: BigDecimal => apply(d) + case k: scala.math.BigInt => apply(k) + case l: java.math.BigInteger => apply(l) case d: Decimal => d } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 9c1319c1c5e6f..6b7e3714e0b04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { private[sql] val LongDecimal = DecimalType(20, 0) private[sql] val FloatDecimal = DecimalType(14, 7) private[sql] val DoubleDecimal = DecimalType(30, 15) + private[sql] val BigIntDecimal = DecimalType(38, 0) private[sql] def forType(dataType: DataType): DecimalType = dataType match { case ByteType => ByteDecimal diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 227e835e7ed51..d4387890b403b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.encoders +import java.math.BigInteger import java.sql.{Date, Timestamp} import java.util.Arrays @@ -109,7 +110,8 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { encodeDecodeTest(BigDecimal("32131413.211321313"), "scala decimal") encodeDecodeTest(new java.math.BigDecimal("231341.23123"), "java decimal") - + encodeDecodeTest(BigInt("23134123123"), "scala biginteger") + encodeDecodeTest(new BigInteger("23134123123"), "java BigInteger") encodeDecodeTest(Decimal("32131413.211321313"), "catalyst decimal") encodeDecodeTest("hello", "string") 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 324ebbae38767..35a9f44feca64 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 @@ -21,6 +21,8 @@ import java.net.URISyntaxException; import java.net.URL; import java.util.*; +import java.math.BigInteger; +import java.math.BigDecimal; import scala.collection.JavaConverters; import scala.collection.Seq; @@ -130,6 +132,7 @@ public static class Bean implements Serializable { private Integer[] b = { 0, 1 }; private Map c = ImmutableMap.of("hello", new int[] { 1, 2 }); private List d = Arrays.asList("floppy", "disk"); + private BigInteger e = new BigInteger("1234567"); public double getA() { return a; @@ -146,6 +149,8 @@ public Map getC() { public List getD() { return d; } + + public BigInteger getE() { return e; } } void validateDataFrameWithBeans(Bean bean, Dataset df) { @@ -163,7 +168,9 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { Assert.assertEquals( new StructField("d", new ArrayType(DataTypes.StringType, true), true, Metadata.empty()), schema.apply("d")); - Row first = df.select("a", "b", "c", "d").first(); + Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, Metadata.empty()), + schema.apply("e")); + Row first = df.select("a", "b", "c", "d", "e").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); // Now Java lists and maps are converted to Scala Seq's and Map's. Once we get a Seq below, // verify that it has the expected length, and contains expected elements. @@ -182,6 +189,8 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { for (int i = 0; i < d.length(); i++) { Assert.assertEquals(bean.getD().get(i), d.apply(i)); } + // Java.math.BigInteger is equavient to Spark Decimal(38,0) + Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); } @Test 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 491bdb3ef9db9..c9bd05d0e4e36 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 @@ -34,7 +34,9 @@ case class ReflectData( decimalField: java.math.BigDecimal, date: Date, timestampField: Timestamp, - seqInt: Seq[Int]) + seqInt: Seq[Int], + javaBigInt: java.math.BigInteger, + scalaBigInt: scala.math.BigInt) case class NullReflectData( intField: java.lang.Integer, @@ -77,13 +79,15 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { 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), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3)) + new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3), + new java.math.BigInteger("1"), scala.math.BigInt(1)) Seq(data).toDF().createOrReplaceTempView("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), - new Timestamp(12345), Seq(1, 2, 3))) + new Timestamp(12345), Seq(1, 2, 3), new java.math.BigDecimal(1), + new java.math.BigDecimal(1))) } test("query case class RDD with nulls") { From 52b967fe66ad07b4eed4b5a5861e5358b7c4af1d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 21:53:26 -0700 Subject: [PATCH 0293/1470] [SPARK-15075][SPARK-15345][SQL] Clean up SparkSession builder and propagate config options to existing sessions if specified ## What changes were proposed in this pull request? Currently SparkSession.Builder use SQLContext.getOrCreate. It should probably the the other way around, i.e. all the core logic goes in SparkSession, and SQLContext just calls that. This patch does that. This patch also makes sure config options specified in the builder are propagated to the existing (and of course the new) SparkSession. ## How was this patch tested? Updated tests to reflect the change, and also introduced a new SparkSessionBuilderSuite that should cover all the branches. Author: Reynold Xin Closes #13200 from rxin/SPARK-15075. (cherry picked from commit f2ee0ed4b7ecb2855cc4928a9613a07d45446f4e) Signed-off-by: Reynold Xin --- .../ml/util/JavaDefaultReadWriteSuite.java | 2 +- python/pyspark/sql/context.py | 5 +- python/pyspark/sql/session.py | 17 +- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 124 ++------------ .../org/apache/spark/sql/SparkSession.scala | 155 ++++++++++++++---- .../spark/sql/execution/ExistingRDD.scala | 3 +- .../spark/sql/execution/QueryExecution.scala | 4 +- .../spark/sql/execution/SparkPlan.scala | 6 +- .../execution/datasources/DataSource.scala | 13 +- .../datasources/fileSourceInterfaces.scala | 2 +- .../datasources/jdbc/JDBCRelation.scala | 2 +- .../execution/streaming/StreamExecution.scala | 2 +- .../org/apache/spark/sql/functions.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 10 -- .../spark/sql/internal/SharedState.scala | 19 ++- .../apache/spark/sql/ListTablesSuite.scala | 13 +- .../spark/sql/MultiSQLContextsSuite.scala | 100 ----------- .../apache/spark/sql/SQLContextSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 8 +- .../apache/spark/sql/SerializationSuite.scala | 2 +- .../spark/sql/SparkSessionBuilderSuite.scala | 93 +++++++++++ .../apache/spark/sql/StatisticsSuite.scala | 4 +- .../execution/ExchangeCoordinatorSuite.scala | 16 +- .../spark/sql/execution/PlannerSuite.scala | 2 +- .../spark/sql/execution/SparkPlanTest.scala | 4 +- .../datasources/parquet/ParquetTest.scala | 2 +- .../streaming/state/StateStoreRDDSuite.scala | 14 +- .../sql/execution/ui/SQLListenerSuite.scala | 6 +- .../spark/sql/internal/SQLConfSuite.scala | 18 +- .../spark/sql/sources/DDLTestSuite.scala | 2 +- .../spark/sql/sources/FilteredScanSuite.scala | 2 +- .../spark/sql/sources/PrunedScanSuite.scala | 2 +- .../spark/sql/sources/TableScanSuite.scala | 4 +- .../DataFrameReaderWriterSuite.scala | 4 +- .../apache/spark/sql/test/SQLTestData.scala | 2 +- .../apache/spark/sql/test/SQLTestUtils.scala | 2 +- .../spark/sql/test/SharedSQLContext.scala | 4 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/MultiDatabaseSuite.scala | 42 ++--- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- .../apache/spark/sql/hive/orc/OrcTest.scala | 2 +- 43 files changed, 367 insertions(+), 357 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java index da623d1d15702..7bda219243bf5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -56,7 +56,7 @@ public void testDefaultReadWrite() throws IOException { } catch (IOException e) { // expected } - instance.write().context(spark.wrapped()).overwrite().save(outputPath); + instance.write().context(spark.sqlContext()).overwrite().save(outputPath); MyParams newInstance = MyParams.load(outputPath); Assert.assertEquals("UID should match.", instance.uid(), newInstance.uid()); Assert.assertEquals("Params should be preserved.", diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e8e60c64121b5..486733a390a0c 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -34,7 +34,10 @@ class SQLContext(object): - """Wrapper around :class:`SparkSession`, the main entry point to Spark SQL functionality. + """The entry point for working with structured data (rows and columns) in Spark, in Spark 1.x. + + As of Spark 2.0, this is replaced by :class:`SparkSession`. However, we are keeping the class + here for backward compatibility. A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 257a239c8d7b0..0e04b88265fa1 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -120,6 +120,8 @@ def master(self, master): def appName(self, name): """Sets a name for the application, which will be shown in the Spark web UI. + If no application name is set, a randomly generated name will be used. + :param name: an application name """ return self.config("spark.app.name", name) @@ -133,8 +135,17 @@ def enableHiveSupport(self): @since(2.0) def getOrCreate(self): - """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a new - one based on the options set in this builder. + """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a + new one based on the options set in this builder. + + This method first checks whether there is a valid thread-local SparkSession, + and if yes, return that one. It then checks whether there is a valid global + default SparkSession, and if yes, return that one. If no valid global default + SparkSession exists, the method creates a new SparkSession and assigns the + newly created SparkSession as the global default. + + In case an existing SparkSession is returned, the config options specified + in this builder will be applied to the existing SparkSession. """ with self._lock: from pyspark.conf import SparkConf @@ -175,7 +186,7 @@ def __init__(self, sparkContext, jsparkSession=None): if jsparkSession is None: jsparkSession = self._jvm.SparkSession(self._jsc.sc()) self._jsparkSession = jsparkSession - self._jwrapped = self._jsparkSession.wrapped() + self._jwrapped = self._jsparkSession.sqlContext() self._wrapped = SQLContext(self._sc, self, self._jwrapped) _monkey_patch_RDD(self) install_exception_handler() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 02dd6547a4adc..78a167eef2e4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -213,7 +213,7 @@ class Dataset[T] private[sql]( private implicit def classTag = unresolvedTEncoder.clsTag // sqlContext must be val because a stable identifier is expected when you import implicits - @transient lazy val sqlContext: SQLContext = sparkSession.wrapped + @transient lazy val sqlContext: SQLContext = sparkSession.sqlContext protected[sql] def resolve(colName: String): NamedExpression = { queryExecution.analyzed.resolveQuoted(colName, sparkSession.sessionState.analyzer.resolver) 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 a3e2b49556e4c..14d12d30bc0b3 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 @@ -19,25 +19,22 @@ package org.apache.spark.sql import java.beans.BeanInfo import java.util.Properties -import java.util.concurrent.atomic.AtomicReference import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.rdd.RDD -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ShowTablesCommand -import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ @@ -46,8 +43,8 @@ import org.apache.spark.sql.util.ExecutionListenerManager /** * The entry point for working with structured data (rows and columns) in Spark, in Spark 1.x. * - * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class here - * for backward compatibility. + * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class + * here for backward compatibility. * * @groupname basic Basic Operations * @groupname ddl_ops Persistent Catalog DDL @@ -76,42 +73,21 @@ class SQLContext private[sql]( this(sparkSession, true) } + @deprecated("Use SparkSession.builder instead", "2.0.0") def this(sc: SparkContext) = { this(new SparkSession(sc)) } + @deprecated("Use SparkSession.builder instead", "2.0.0") def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) // TODO: move this logic into SparkSession - // If spark.sql.allowMultipleContexts is true, we will throw an exception if a user - // wants to create a new root SQLContext (a SQLContext that is not created by newSession). - private val allowMultipleContexts = - sparkContext.conf.getBoolean( - SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, - SQLConf.ALLOW_MULTIPLE_CONTEXTS.defaultValue.get) - - // Assert no root SQLContext is running when allowMultipleContexts is false. - { - if (!allowMultipleContexts && isRootContext) { - SQLContext.getInstantiatedContextOption() match { - case Some(rootSQLContext) => - val errMsg = "Only one SQLContext/HiveContext may be running in this JVM. " + - s"It is recommended to use SQLContext.getOrCreate to get the instantiated " + - s"SQLContext/HiveContext. To ignore this error, " + - s"set ${SQLConf.ALLOW_MULTIPLE_CONTEXTS.key} = true in SparkConf." - throw new SparkException(errMsg) - case None => // OK - } - } - } - protected[sql] def sessionState: SessionState = sparkSession.sessionState protected[sql] def sharedState: SharedState = sparkSession.sharedState protected[sql] def conf: SQLConf = sessionState.conf protected[sql] def runtimeConf: RuntimeConfig = sparkSession.conf protected[sql] def cacheManager: CacheManager = sparkSession.cacheManager - protected[sql] def listener: SQLListener = sparkSession.listener protected[sql] def externalCatalog: ExternalCatalog = sparkSession.externalCatalog def sparkContext: SparkContext = sparkSession.sparkContext @@ -123,7 +99,7 @@ class SQLContext private[sql]( * * @since 1.6.0 */ - def newSession(): SQLContext = sparkSession.newSession().wrapped + def newSession(): SQLContext = sparkSession.newSession().sqlContext /** * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s @@ -760,21 +736,6 @@ class SQLContext private[sql]( schema: StructType): DataFrame = { sparkSession.applySchemaToPythonRDD(rdd, schema) } - - // TODO: move this logic into SparkSession - - // Register a successfully instantiated context to the singleton. This should be at the end of - // the class definition so that the singleton is updated only if there is no exception in the - // construction of the instance. - sparkContext.addSparkListener(new SparkListener { - override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { - SQLContext.clearInstantiatedContext() - SQLContext.clearSqlListener() - } - }) - - sparkSession.setWrappedContext(self) - SQLContext.setInstantiatedContext(self) } /** @@ -787,19 +748,6 @@ class SQLContext private[sql]( */ object SQLContext { - /** - * The active SQLContext for the current thread. - */ - private val activeContext: InheritableThreadLocal[SQLContext] = - new InheritableThreadLocal[SQLContext] - - /** - * Reference to the created SQLContext. - */ - @transient private val instantiatedContext = new AtomicReference[SQLContext]() - - @transient private val sqlListener = new AtomicReference[SQLListener]() - /** * Get the singleton SQLContext if it exists or create a new one using the given SparkContext. * @@ -811,41 +759,9 @@ object SQLContext { * * @since 1.5.0 */ + @deprecated("Use SparkSession.builder instead", "2.0.0") def getOrCreate(sparkContext: SparkContext): SQLContext = { - val ctx = activeContext.get() - if (ctx != null && !ctx.sparkContext.isStopped) { - return ctx - } - - synchronized { - val ctx = instantiatedContext.get() - if (ctx == null || ctx.sparkContext.isStopped) { - new SQLContext(sparkContext) - } else { - ctx - } - } - } - - private[sql] def clearInstantiatedContext(): Unit = { - instantiatedContext.set(null) - } - - private[sql] def setInstantiatedContext(sqlContext: SQLContext): Unit = { - synchronized { - val ctx = instantiatedContext.get() - if (ctx == null || ctx.sparkContext.isStopped) { - instantiatedContext.set(sqlContext) - } - } - } - - private[sql] def getInstantiatedContextOption(): Option[SQLContext] = { - Option(instantiatedContext.get()) - } - - private[sql] def clearSqlListener(): Unit = { - sqlListener.set(null) + SparkSession.builder().sparkContext(sparkContext).getOrCreate().sqlContext } /** @@ -855,8 +771,9 @@ object SQLContext { * * @since 1.6.0 */ + @deprecated("Use SparkSession.setActiveSession instead", "2.0.0") def setActive(sqlContext: SQLContext): Unit = { - activeContext.set(sqlContext) + SparkSession.setActiveSession(sqlContext.sparkSession) } /** @@ -865,12 +782,9 @@ object SQLContext { * * @since 1.6.0 */ + @deprecated("Use SparkSession.clearActiveSession instead", "2.0.0") def clearActive(): Unit = { - activeContext.remove() - } - - private[sql] def getActive(): Option[SQLContext] = { - Option(activeContext.get()) + SparkSession.clearActiveSession() } /** @@ -894,20 +808,6 @@ object SQLContext { } } - /** - * Create a SQLListener then add it into SparkContext, and create an SQLTab if there is SparkUI. - */ - private[sql] def createListenerAndUI(sc: SparkContext): SQLListener = { - if (sqlListener.get() == null) { - val listener = new SQLListener(sc.conf) - if (sqlListener.compareAndSet(null, listener)) { - sc.addSparkListener(listener) - sc.ui.foreach(new SQLTab(listener, _)) - } - } - sqlListener.get() - } - /** * Extract `spark.sql.*` properties from the conf and return them as a [[Properties]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 8ef6cd3c35572..ab751f0c04115 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.beans.Introspector +import java.util.concurrent.atomic.AtomicReference import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -30,6 +31,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ @@ -98,24 +100,10 @@ class SparkSession private( } /** - * A wrapped version of this session in the form of a [[SQLContext]]. + * A wrapped version of this session in the form of a [[SQLContext]], for backward compatibility. */ @transient - private var _wrapped: SQLContext = _ - - @transient - private val _wrappedLock = new Object - - protected[sql] def wrapped: SQLContext = _wrappedLock.synchronized { - if (_wrapped == null) { - _wrapped = new SQLContext(self, isRootContext = false) - } - _wrapped - } - - protected[sql] def setWrappedContext(sqlContext: SQLContext): Unit = _wrappedLock.synchronized { - _wrapped = sqlContext - } + private[sql] val sqlContext: SQLContext = new SQLContext(this) protected[sql] def cacheManager: CacheManager = sharedState.cacheManager protected[sql] def listener: SQLListener = sharedState.listener @@ -238,7 +226,7 @@ class SparkSession private( */ @Experimental def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { - SQLContext.setActive(wrapped) + SparkSession.setActiveSession(this) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes val rowRDD = RDDConversions.productToRowRdd(rdd, schema.map(_.dataType)) @@ -254,7 +242,7 @@ class SparkSession private( */ @Experimental def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { - SQLContext.setActive(wrapped) + SparkSession.setActiveSession(this) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes Dataset.ofRows(self, LocalRelation.fromProduct(attributeSeq, data)) @@ -573,7 +561,7 @@ class SparkSession private( */ @Experimental object implicits extends SQLImplicits with Serializable { - protected override def _sqlContext: SQLContext = wrapped + protected override def _sqlContext: SQLContext = SparkSession.this.sqlContext } // scalastyle:on @@ -649,8 +637,16 @@ object SparkSession { private[this] val options = new scala.collection.mutable.HashMap[String, String] + private[this] var userSuppliedContext: Option[SparkContext] = None + + private[sql] def sparkContext(sparkContext: SparkContext): Builder = synchronized { + userSuppliedContext = Option(sparkContext) + this + } + /** * Sets a name for the application, which will be shown in the Spark web UI. + * If no application name is set, a randomly generated name will be used. * * @since 2.0.0 */ @@ -735,29 +731,130 @@ object SparkSession { } /** - * Gets an existing [[SparkSession]] or, if there is no existing one, creates a new one - * based on the options set in this builder. + * Gets an existing [[SparkSession]] or, if there is no existing one, creates a new + * one based on the options set in this builder. + * + * This method first checks whether there is a valid thread-local SparkSession, + * and if yes, return that one. It then checks whether there is a valid global + * default SparkSession, and if yes, return that one. If no valid global default + * SparkSession exists, the method creates a new SparkSession and assigns the + * newly created SparkSession as the global default. + * + * In case an existing SparkSession is returned, the config options specified in + * this builder will be applied to the existing SparkSession. * * @since 2.0.0 */ def getOrCreate(): SparkSession = synchronized { - // Step 1. Create a SparkConf - // Step 2. Get a SparkContext - // Step 3. Get a SparkSession - val sparkConf = new SparkConf() - options.foreach { case (k, v) => sparkConf.set(k, v) } - val sparkContext = SparkContext.getOrCreate(sparkConf) - - SQLContext.getOrCreate(sparkContext).sparkSession + // Get the session from current thread's active session. + var session = activeThreadSession.get() + if ((session ne null) && !session.sparkContext.isStopped) { + options.foreach { case (k, v) => session.conf.set(k, v) } + return session + } + + // Global synchronization so we will only set the default session once. + SparkSession.synchronized { + // If the current thread does not have an active session, get it from the global session. + session = defaultSession.get() + if ((session ne null) && !session.sparkContext.isStopped) { + options.foreach { case (k, v) => session.conf.set(k, v) } + return session + } + + // No active nor global default session. Create a new one. + val sparkContext = userSuppliedContext.getOrElse { + // set app name if not given + if (!options.contains("spark.app.name")) { + options += "spark.app.name" -> java.util.UUID.randomUUID().toString + } + + val sparkConf = new SparkConf() + options.foreach { case (k, v) => sparkConf.set(k, v) } + SparkContext.getOrCreate(sparkConf) + } + session = new SparkSession(sparkContext) + options.foreach { case (k, v) => session.conf.set(k, v) } + defaultSession.set(session) + + // Register a successfully instantiated context to the singleton. This should be at the + // end of the class definition so that the singleton is updated only if there is no + // exception in the construction of the instance. + sparkContext.addSparkListener(new SparkListener { + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { + defaultSession.set(null) + sqlListener.set(null) + } + }) + } + + return session } } /** * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. + * * @since 2.0.0 */ def builder(): Builder = new Builder + /** + * Changes the SparkSession that will be returned in this thread and its children when + * SparkSession.getOrCreate() is called. This can be used to ensure that a given thread receives + * a SparkSession with an isolated session, instead of the global (first created) context. + * + * @since 2.0.0 + */ + def setActiveSession(session: SparkSession): Unit = { + activeThreadSession.set(session) + } + + /** + * Clears the active SparkSession for current thread. Subsequent calls to getOrCreate will + * return the first created context instead of a thread-local override. + * + * @since 2.0.0 + */ + def clearActiveSession(): Unit = { + activeThreadSession.remove() + } + + /** + * Sets the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def setDefaultSession(session: SparkSession): Unit = { + defaultSession.set(session) + } + + /** + * Clears the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def clearDefaultSession(): Unit = { + defaultSession.set(null) + } + + private[sql] def getActiveSession: Option[SparkSession] = Option(activeThreadSession.get) + + private[sql] def getDefaultSession: Option[SparkSession] = Option(defaultSession.get) + + /** A global SQL listener used for the SQL UI. */ + private[sql] val sqlListener = new AtomicReference[SQLListener]() + + //////////////////////////////////////////////////////////////////////////////////////// + // Private methods from now on + //////////////////////////////////////////////////////////////////////////////////////// + + /** The active SparkSession for the current thread. */ + private val activeThreadSession = new InheritableThreadLocal[SparkSession] + + /** Reference to the root SparkSession. */ + private val defaultSession = new AtomicReference[SparkSession] + private val HIVE_SHARED_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSharedState" private val HIVE_SESSION_STATE_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionState" 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 85af4faf4d090..d8911f88b0004 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 @@ -157,7 +157,8 @@ private[sql] case class RowDataSourceScanExec( val outputUnsafeRows = relation match { case r: HadoopFsRelation if r.fileFormat.isInstanceOf[ParquetSource] => - !SQLContext.getActive().get.conf.getConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED) + !SparkSession.getActiveSession.get.sessionState.conf.getConf( + SQLConf.PARQUET_VECTORIZED_READER_ENABLED) case _: HadoopFsRelation => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index cb3c46a98bfb4..34187b9a1ae7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -60,7 +60,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { } lazy val analyzed: LogicalPlan = { - SQLContext.setActive(sparkSession.wrapped) + SparkSession.setActiveSession(sparkSession) sparkSession.sessionState.analyzer.execute(logical) } @@ -73,7 +73,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { lazy val optimizedPlan: LogicalPlan = sparkSession.sessionState.optimizer.execute(withCachedData) lazy val sparkPlan: SparkPlan = { - SQLContext.setActive(sparkSession.wrapped) + SparkSession.setActiveSession(sparkSession) planner.plan(ReturnAnswer(optimizedPlan)).next() } 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 b94b84d77a502..045ccc7bd6eae 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 @@ -27,7 +27,7 @@ import org.apache.spark.{broadcast, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd.{RDD, RDDOperationScope} -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -50,7 +50,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ * populated by the query planning infrastructure. */ @transient - protected[spark] final val sqlContext = SQLContext.getActive().orNull + final val sqlContext = SparkSession.getActiveSession.map(_.sqlContext).orNull protected def sparkContext = sqlContext.sparkContext @@ -65,7 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** Overridden make copy also propagates sqlContext to copied plan. */ override def makeCopy(newArgs: Array[AnyRef]): SparkPlan = { - SQLContext.setActive(sqlContext) + SparkSession.setActiveSession(sqlContext.sparkSession) super.makeCopy(newArgs) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index ccad9b3fd52fd..2e17b763a5370 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -178,7 +178,7 @@ case class DataSource( providingClass.newInstance() match { case s: StreamSourceProvider => val (name, schema) = s.sourceSchema( - sparkSession.wrapped, userSpecifiedSchema, className, options) + sparkSession.sqlContext, userSpecifiedSchema, className, options) SourceInfo(name, schema) case format: FileFormat => @@ -198,7 +198,8 @@ case class DataSource( def createSource(metadataPath: String): Source = { providingClass.newInstance() match { case s: StreamSourceProvider => - s.createSource(sparkSession.wrapped, metadataPath, userSpecifiedSchema, className, options) + s.createSource( + sparkSession.sqlContext, metadataPath, userSpecifiedSchema, className, options) case format: FileFormat => val path = new CaseInsensitiveMap(options).getOrElse("path", { @@ -215,7 +216,7 @@ case class DataSource( /** Returns a sink that can be used to continually write data. */ def createSink(): Sink = { providingClass.newInstance() match { - case s: StreamSinkProvider => s.createSink(sparkSession.wrapped, options, partitionColumns) + case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, options, partitionColumns) case parquet: parquet.DefaultSource => val caseInsensitiveOptions = new CaseInsensitiveMap(options) @@ -265,9 +266,9 @@ case class DataSource( val relation = (providingClass.newInstance(), userSpecifiedSchema) match { // TODO: Throw when too much is given. case (dataSource: SchemaRelationProvider, Some(schema)) => - dataSource.createRelation(sparkSession.wrapped, caseInsensitiveOptions, schema) + dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions, schema) case (dataSource: RelationProvider, None) => - dataSource.createRelation(sparkSession.wrapped, caseInsensitiveOptions) + dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) case (_: SchemaRelationProvider, None) => throw new AnalysisException(s"A schema needs to be specified when using $className.") case (_: RelationProvider, Some(_)) => @@ -383,7 +384,7 @@ case class DataSource( providingClass.newInstance() match { case dataSource: CreatableRelationProvider => - dataSource.createRelation(sparkSession.wrapped, mode, options, data) + dataSource.createRelation(sparkSession.sqlContext, mode, options, data) case format: FileFormat => // Don't glob path for the write path. The contracts here are: // 1. Only one output path can be specified on the write path; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 8d332df029166..88125a2b4da78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -142,7 +142,7 @@ case class HadoopFsRelation( fileFormat: FileFormat, options: Map[String, String]) extends BaseRelation with FileRelation { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext val schema: StructType = { val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index bcf70fdc4a497..233b7891d664c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -92,7 +92,7 @@ private[sql] case class JDBCRelation( with PrunedFilteredScan with InsertableRelation { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override val needConversion: Boolean = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index df6304d85fe74..7d09bdcebdc3d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -173,7 +173,7 @@ class StreamExecution( startLatch.countDown() // While active, repeatedly attempt to run batches. - SQLContext.setActive(sparkSession.wrapped) + SparkSession.setActiveSession(sparkSession) triggerExecutor.execute(() => { if (isActive) { 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 65bc043076759..0b490fe71c526 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 @@ -1168,7 +1168,7 @@ object functions { * @group normal_funcs */ def expr(expr: String): Column = { - val parser = SQLContext.getActive().map(_.sessionState.sqlParser).getOrElse { + val parser = SparkSession.getActiveSession.map(_.sessionState.sqlParser).getOrElse { new SparkSqlParser(new SQLConf) } Column(parser.parseExpression(expr)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5d1868980163d..35d67ca2d8c5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -70,16 +70,6 @@ object SQLConf { .intConf .createWithDefault(10) - val ALLOW_MULTIPLE_CONTEXTS = SQLConfigBuilder("spark.sql.allowMultipleContexts") - .doc("When set to true, creating multiple SQLContexts/HiveContexts is allowed. " + - "When set to false, only one SQLContext/HiveContext is allowed to be created " + - "through the constructor (new SQLContexts/HiveContexts created through newSession " + - "method is allowed). Please note that this conf needs to be set in Spark Conf. Once " + - "a SQLContext/HiveContext has been created, changing the value of this conf will not " + - "have effect.") - .booleanConf - .createWithDefault(true) - val COMPRESS_CACHED = SQLConfigBuilder("spark.sql.inMemoryColumnarStorage.compressed") .internal() .doc("When set to true Spark SQL will automatically select a compression codec for each " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index eaf993aaed4d2..9f6137d6e3c7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.internal import org.apache.spark.SparkContext -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} import org.apache.spark.sql.execution.CacheManager -import org.apache.spark.sql.execution.ui.SQLListener +import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.util.MutableURLClassLoader @@ -38,7 +38,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) { /** * A listener for SQL-specific [[org.apache.spark.scheduler.SparkListenerEvent]]s. */ - val listener: SQLListener = SQLContext.createListenerAndUI(sparkContext) + val listener: SQLListener = createListenerAndUI(sparkContext) /** * A catalog that interacts with external systems. @@ -51,6 +51,19 @@ private[sql] class SharedState(val sparkContext: SparkContext) { val jarClassLoader = new NonClosableMutableURLClassLoader( org.apache.spark.util.Utils.getContextOrSparkClassLoader) + /** + * Create a SQLListener then add it into SparkContext, and create an SQLTab if there is SparkUI. + */ + private def createListenerAndUI(sc: SparkContext): SQLListener = { + if (SparkSession.sqlListener.get() == null) { + val listener = new SQLListener(sc.conf) + if (SparkSession.sqlListener.compareAndSet(null, listener)) { + sc.addSparkListener(listener) + sc.ui.foreach(new SQLTab(listener, _)) + } + } + SparkSession.sqlListener.get() + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index 65fe271b69172..b447006761f45 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -39,7 +39,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex test("get all tables") { checkAnswer( - spark.wrapped.tables().filter("tableName = 'listtablessuitetable'"), + spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'"), Row("listtablessuitetable", true)) checkAnswer( @@ -48,12 +48,12 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex spark.sessionState.catalog.dropTable( TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) - assert(spark.wrapped.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + assert(spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) } test("getting all tables with a database name has no impact on returned table names") { checkAnswer( - spark.wrapped.tables("default").filter("tableName = 'listtablessuitetable'"), + spark.sqlContext.tables("default").filter("tableName = 'listtablessuitetable'"), Row("listtablessuitetable", true)) checkAnswer( @@ -62,7 +62,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex spark.sessionState.catalog.dropTable( TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) - assert(spark.wrapped.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + assert(spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) } test("query the returned DataFrame of tables") { @@ -70,7 +70,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex StructField("tableName", StringType, false) :: StructField("isTemporary", BooleanType, false) :: Nil) - Seq(spark.wrapped.tables(), sql("SHOW TABLes")).foreach { + Seq(spark.sqlContext.tables(), sql("SHOW TABLes")).foreach { case tableDF => assert(expectedSchema === tableDF.schema) @@ -81,7 +81,8 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContex Row(true, "listtablessuitetable") ) checkAnswer( - spark.wrapped.tables().filter("tableName = 'tables'").select("tableName", "isTemporary"), + spark.sqlContext.tables() + .filter("tableName = 'tables'").select("tableName", "isTemporary"), Row("tables", true)) spark.catalog.dropTempView("tables") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala deleted file mode 100644 index 0b5a92c256e57..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark._ -import org.apache.spark.sql.internal.SQLConf - -class MultiSQLContextsSuite extends SparkFunSuite with BeforeAndAfterAll { - - private var originalActiveSQLContext: Option[SQLContext] = _ - private var originalInstantiatedSQLContext: Option[SQLContext] = _ - private var sparkConf: SparkConf = _ - - override protected def beforeAll(): Unit = { - originalActiveSQLContext = SQLContext.getActive() - originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption() - - SQLContext.clearActive() - SQLContext.clearInstantiatedContext() - sparkConf = - new SparkConf(false) - .setMaster("local[*]") - .setAppName("test") - .set("spark.ui.enabled", "false") - .set("spark.driver.allowMultipleContexts", "true") - } - - override protected def afterAll(): Unit = { - // Set these states back. - originalActiveSQLContext.foreach(ctx => SQLContext.setActive(ctx)) - originalInstantiatedSQLContext.foreach(ctx => SQLContext.setInstantiatedContext(ctx)) - } - - def testNewSession(rootSQLContext: SQLContext): Unit = { - // Make sure we can successfully create new Session. - rootSQLContext.newSession() - - // Reset the state. It is always safe to clear the active context. - SQLContext.clearActive() - } - - def testCreatingNewSQLContext(allowsMultipleContexts: Boolean): Unit = { - val conf = - sparkConf - .clone - .set(SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, allowsMultipleContexts.toString) - val sparkContext = new SparkContext(conf) - - try { - if (allowsMultipleContexts) { - new SQLContext(sparkContext) - SQLContext.clearActive() - } else { - // If allowsMultipleContexts is false, make sure we can get the error. - val message = intercept[SparkException] { - new SQLContext(sparkContext) - }.getMessage - assert(message.contains("Only one SQLContext/HiveContext may be running")) - } - } finally { - sparkContext.stop() - } - } - - test("test the flag to disallow creating multiple root SQLContext") { - Seq(false, true).foreach { allowMultipleSQLContexts => - val conf = - sparkConf - .clone - .set(SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, allowMultipleSQLContexts.toString) - val sc = new SparkContext(conf) - try { - val rootSQLContext = new SQLContext(sc) - testNewSession(rootSQLContext) - testNewSession(rootSQLContext) - testCreatingNewSQLContext(allowMultipleSQLContexts) - } finally { - sc.stop() - SQLContext.clearInstantiatedContext() - } - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 38d7b6e25b829..c9594a7e9ab28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -40,7 +40,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { val newSession = sqlContext.newSession() assert(SQLContext.getOrCreate(sc).eq(sqlContext), "SQLContext.getOrCreate after explicitly created SQLContext did not return the context") - SQLContext.setActive(newSession) + SparkSession.setActiveSession(newSession.sparkSession) assert(SQLContext.getOrCreate(sc).eq(newSession), "SQLContext.getOrCreate after explicitly setActive() did not return the active context") } 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 743a27aa7a21d..460e34a5ff308 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 @@ -1042,7 +1042,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SET commands semantics using sql()") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() val testKey = "test.key.0" val testVal = "test.val.0" val nonexistentKey = "nonexistent" @@ -1083,17 +1083,17 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SET $nonexistentKey"), Row(nonexistentKey, "") ) - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("SET commands with illegal or inappropriate argument") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() // Set negative mapred.reduce.tasks for automatically determining // the number of reducers is not supported intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-2")) - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("apply schema") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala index b489b74fec074..cd6b2647e0be6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -25,6 +25,6 @@ class SerializationSuite extends SparkFunSuite with SharedSQLContext { test("[SPARK-5235] SQLContext should be serializable") { val spark = SparkSession.builder.getOrCreate() - new JavaSerializer(new SparkConf()).newInstance().serialize(spark.wrapped) + new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sqlContext) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala new file mode 100644 index 0000000000000..ec6a2b3575869 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -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. + */ + +package org.apache.spark.sql + +import org.apache.spark.{SparkContext, SparkFunSuite} + +/** + * Test cases for the builder pattern of [[SparkSession]]. + */ +class SparkSessionBuilderSuite extends SparkFunSuite { + + private var initialSession: SparkSession = _ + + private lazy val sparkContext: SparkContext = { + initialSession = SparkSession.builder() + .master("local") + .config("spark.ui.enabled", value = false) + .config("some-config", "v2") + .getOrCreate() + initialSession.sparkContext + } + + test("create with config options and propagate them to SparkContext and SparkSession") { + // Creating a new session with config - this works by just calling the lazy val + sparkContext + assert(initialSession.sparkContext.conf.get("some-config") == "v2") + assert(initialSession.conf.get("some-config") == "v2") + SparkSession.clearDefaultSession() + } + + test("use global default session") { + val session = SparkSession.builder().getOrCreate() + assert(SparkSession.builder().getOrCreate() == session) + SparkSession.clearDefaultSession() + } + + test("config options are propagated to existing SparkSession") { + val session1 = SparkSession.builder().config("spark-config1", "a").getOrCreate() + assert(session1.conf.get("spark-config1") == "a") + val session2 = SparkSession.builder().config("spark-config1", "b").getOrCreate() + assert(session1 == session2) + assert(session1.conf.get("spark-config1") == "b") + SparkSession.clearDefaultSession() + } + + test("use session from active thread session and propagate config options") { + val defaultSession = SparkSession.builder().getOrCreate() + val activeSession = defaultSession.newSession() + SparkSession.setActiveSession(activeSession) + val session = SparkSession.builder().config("spark-config2", "a").getOrCreate() + + assert(activeSession != defaultSession) + assert(session == activeSession) + assert(session.conf.get("spark-config2") == "a") + SparkSession.clearActiveSession() + + assert(SparkSession.builder().getOrCreate() == defaultSession) + SparkSession.clearDefaultSession() + } + + test("create a new session if the default session has been stopped") { + val defaultSession = SparkSession.builder().getOrCreate() + SparkSession.setDefaultSession(defaultSession) + defaultSession.stop() + val newSession = SparkSession.builder().master("local").getOrCreate() + assert(newSession != defaultSession) + newSession.stop() + } + + test("create a new session if the active thread session has been stopped") { + val activeSession = SparkSession.builder().master("local").getOrCreate() + SparkSession.setActiveSession(activeSession) + activeSession.stop() + val newSession = SparkSession.builder().master("local").getOrCreate() + assert(newSession != activeSession) + newSession.stop() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index 9523f6f9f5bbf..4de3cf605caa1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -26,9 +26,9 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 70a00a43f7db4..2f45db3925a00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -27,21 +27,21 @@ import org.apache.spark.sql.internal.SQLConf class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { - private var originalActiveSQLContext: Option[SQLContext] = _ - private var originalInstantiatedSQLContext: Option[SQLContext] = _ + private var originalActiveSQLContext: Option[SparkSession] = _ + private var originalInstantiatedSQLContext: Option[SparkSession] = _ override protected def beforeAll(): Unit = { - originalActiveSQLContext = SQLContext.getActive() - originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption() + originalActiveSQLContext = SparkSession.getActiveSession + originalInstantiatedSQLContext = SparkSession.getDefaultSession - SQLContext.clearActive() - SQLContext.clearInstantiatedContext() + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() } override protected def afterAll(): Unit = { // Set these states back. - originalActiveSQLContext.foreach(ctx => SQLContext.setActive(ctx)) - originalInstantiatedSQLContext.foreach(ctx => SQLContext.setInstantiatedContext(ctx)) + originalActiveSQLContext.foreach(ctx => SparkSession.setActiveSession(ctx)) + originalInstantiatedSQLContext.foreach(ctx => SparkSession.setDefaultSession(ctx)) } private def checkEstimation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 2a5295d0d2231..8243470b19334 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -155,7 +155,7 @@ class PlannerSuite extends SharedSQLContext { val path = file.getCanonicalPath testData.write.parquet(path) val df = spark.read.parquet(path) - spark.wrapped.registerDataFrameAsTable(df, "testPushed") + spark.sqlContext.registerDataFrameAsTable(df, "testPushed") withTempTable("testPushed") { val exp = sql("select * from testPushed where key = 15").queryExecution.sparkPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala index d7eae21f9f556..9fe0e9646e31e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala @@ -91,7 +91,7 @@ private[sql] abstract class SparkPlanTest extends SparkFunSuite { expectedAnswer: Seq[Row], sortAnswers: Boolean = true): Unit = { SparkPlanTest - .checkAnswer(input, planFunction, expectedAnswer, sortAnswers, spark.wrapped) match { + .checkAnswer(input, planFunction, expectedAnswer, sortAnswers, spark.sqlContext) match { case Some(errorMessage) => fail(errorMessage) case None => } @@ -115,7 +115,7 @@ private[sql] abstract class SparkPlanTest extends SparkFunSuite { expectedPlanFunction: SparkPlan => SparkPlan, sortAnswers: Boolean = true): Unit = { SparkPlanTest.checkAnswer( - input, planFunction, expectedPlanFunction, sortAnswers, spark.wrapped) match { + input, planFunction, expectedPlanFunction, sortAnswers, spark.sqlContext) match { case Some(errorMessage) => fail(errorMessage) case None => } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index b5fc51603e160..1753b84ba6af6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -90,7 +90,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { (data: Seq[T], tableName: String, testVectorized: Boolean = true) (f: => Unit): Unit = { withParquetDataFrame(data, testVectorized) { df => - spark.wrapped.registerDataFrameAsTable(df, tableName) + spark.sqlContext.registerDataFrameAsTable(df, tableName) withTempTable(tableName)(f) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala index 4fa1754253aff..bd197be655d58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala @@ -60,13 +60,13 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn val opId = 0 val rdd1 = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( - spark.wrapped, path, opId, storeVersion = 0, keySchema, valueSchema)( + spark.sqlContext, path, opId, storeVersion = 0, keySchema, valueSchema)( increment) assert(rdd1.collect().toSet === Set("a" -> 2, "b" -> 1)) // Generate next version of stores val rdd2 = makeRDD(spark.sparkContext, Seq("a", "c")).mapPartitionsWithStateStore( - spark.wrapped, path, opId, storeVersion = 1, keySchema, valueSchema)(increment) + spark.sqlContext, path, opId, storeVersion = 1, keySchema, valueSchema)(increment) assert(rdd2.collect().toSet === Set("a" -> 3, "b" -> 1, "c" -> 1)) // Make sure the previous RDD still has the same data. @@ -82,7 +82,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn spark: SparkSession, seq: Seq[String], storeVersion: Int): RDD[(String, Int)] = { - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext makeRDD(spark.sparkContext, Seq("a")).mapPartitionsWithStateStore( sqlContext, path, opId, storeVersion, keySchema, valueSchema)(increment) } @@ -102,7 +102,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn test("usage with iterators - only gets and only puts") { withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString val opId = 0 @@ -131,7 +131,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn } val rddOfGets1 = makeRDD(spark.sparkContext, Seq("a", "b", "c")).mapPartitionsWithStateStore( - spark.wrapped, path, opId, storeVersion = 0, keySchema, valueSchema)(iteratorOfGets) + spark.sqlContext, path, opId, storeVersion = 0, keySchema, valueSchema)(iteratorOfGets) assert(rddOfGets1.collect().toSet === Set("a" -> None, "b" -> None, "c" -> None)) val rddOfPuts = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( @@ -150,7 +150,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext val coordinatorRef = sqlContext.streams.stateStoreCoordinator coordinatorRef.reportActiveInstance(StateStoreId(path, opId, 0), "host1", "exec1") coordinatorRef.reportActiveInstance(StateStoreId(path, opId, 1), "host2", "exec2") @@ -183,7 +183,7 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn SparkSession.builder .config(sparkConf.setMaster("local-cluster[2, 1, 1024]")) .getOrCreate()) { spark => - implicit val sqlContext = spark.wrapped + implicit val sqlContext = spark.sqlContext val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString val opId = 0 val rdd1 = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 1c467137baa86..2374ffaaa5036 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -24,7 +24,7 @@ import org.mockito.Mockito.mock import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -400,8 +400,8 @@ class SQLListenerMemoryLeakSuite extends SparkFunSuite { .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly val sc = new SparkContext(conf) try { - SQLContext.clearSqlListener() - val spark = new SQLContext(sc) + SparkSession.sqlListener.set(null) + val spark = new SparkSession(sc) import spark.implicits._ // Run 100 successful executions and 100 failed executions. // Each execution only has one job and one stage. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 81bc973be74a3..0296229100a24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -35,7 +35,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // Set a conf first. spark.conf.set(testKey, testVal) // Clear the conf. - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() // After clear, only overrideConfs used by unit test should be in the SQLConf. assert(spark.conf.getAll === TestSQLContext.overrideConfs) @@ -50,11 +50,11 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { assert(spark.conf.get(testKey, testVal + "_") === testVal) assert(spark.conf.getAll.contains(testKey)) - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("parse SQL set commands") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() sql(s"set $testKey=$testVal") assert(spark.conf.get(testKey, testVal + "_") === testVal) assert(spark.conf.get(testKey, testVal + "_") === testVal) @@ -72,11 +72,11 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { sql(s"set $key=") assert(spark.conf.get(key, "0") === "") - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("set command for display") { - spark.wrapped.conf.clear() + spark.sessionState.conf.clear() checkAnswer( sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), Nil) @@ -97,7 +97,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("deprecated property") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) try{ sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") @@ -108,7 +108,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("invalid conf value") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() val e = intercept[IllegalArgumentException] { sql(s"set ${SQLConf.CASE_SENSITIVE.key}=10") } @@ -116,7 +116,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("Test SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE's method") { - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "100") assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === 100) @@ -144,7 +144,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-90000000000g") } - spark.wrapped.conf.clear() + spark.sqlContext.conf.clear() } test("SparkSession can access configs set in SparkConf") { 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 612cfc7ec7bd6..a34f70ed65b5c 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 @@ -41,7 +41,7 @@ case class SimpleDDLScan( table: String)(@transient val sparkSession: SparkSession) extends BaseRelation with TableScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType(Seq( 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 51d04f2f4efc6..f969660ddd322 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 @@ -40,7 +40,7 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sparkSession: S extends BaseRelation with PrunedFilteredScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType( 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 cd0256db43aad..9cdf7dea7663e 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 @@ -37,7 +37,7 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sparkSession: Spa extends BaseRelation with PrunedScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType( 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 34b8726a922fb..cddf4a1884fa8 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 @@ -38,7 +38,7 @@ class SimpleScanSource extends RelationProvider { case class SimpleScan(from: Int, to: Int)(@transient val sparkSession: SparkSession) extends BaseRelation with TableScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = StructType(StructField("i", IntegerType, nullable = false) :: Nil) @@ -70,7 +70,7 @@ case class AllDataTypesScan( extends BaseRelation with TableScan { - override def sqlContext: SQLContext = sparkSession.wrapped + override def sqlContext: SQLContext = sparkSession.sqlContext override def schema: StructType = userSpecifiedSchema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala index ff53505549330..e6c0ce95e7b57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala @@ -355,14 +355,14 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B q.stop() verify(LastOptions.mockStreamSourceProvider).createSource( - spark.wrapped, + spark.sqlContext, checkpointLocation + "/sources/0", None, "org.apache.spark.sql.streaming.test", Map.empty) verify(LastOptions.mockStreamSourceProvider).createSource( - spark.wrapped, + spark.sqlContext, checkpointLocation + "/sources/1", None, "org.apache.spark.sql.streaming.test", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 421f6bca7f865..0cfe260e52152 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -30,7 +30,7 @@ private[sql] trait SQLTestData { self => // Helper object to import SQL implicits without a concrete SQLContext private object internalImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.spark.wrapped + protected override def _sqlContext: SQLContext = self.spark.sqlContext } import internalImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 51538eca644f6..853dd0ff3f601 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -66,7 +66,7 @@ private[sql] trait SQLTestUtils * but the implicits import is needed in the constructor. */ protected object testImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.spark.wrapped + protected override def _sqlContext: SQLContext = self.spark.sqlContext } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala index 620bfa995aa20..79c37faa4e9ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala @@ -44,13 +44,13 @@ trait SharedSQLContext extends SQLTestUtils { /** * The [[TestSQLContext]] to use for all tests in this suite. */ - protected implicit def sqlContext: SQLContext = _spark.wrapped + protected implicit def sqlContext: SQLContext = _spark.sqlContext /** * Initialize the [[TestSparkSession]]. */ protected override def beforeAll(): Unit = { - SQLContext.clearSqlListener() + SparkSession.sqlListener.set(null) if (_spark == null) { _spark = new TestSparkSession(sparkConf) } 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 8de223f444f70..638911599aad3 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 @@ -56,7 +56,7 @@ private[hive] object SparkSQLEnv extends Logging { val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() sparkContext = sparkSession.sparkContext - sqlContext = sparkSession.wrapped + sqlContext = sparkSession.sqlContext val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState] sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) 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 d2cb62c617d44..7c74a0308d483 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 @@ -30,7 +30,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd override protected def beforeEach(): Unit = { super.beforeEach() - if (spark.wrapped.tableNames().contains("src")) { + if (spark.sqlContext.tableNames().contains("src")) { spark.catalog.dropTempView("src") } Seq((1, "")).toDF("key", "value").createOrReplaceTempView("src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 6c9ce208dbd6a..622b043581c5d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -36,11 +36,11 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) checkAnswer(spark.table("t"), df) } - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") @@ -50,7 +50,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle test(s"saveAsTable() to non-default database - without USE - Overwrite") { withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") @@ -65,7 +65,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").mode(SaveMode.Overwrite).save(path) spark.catalog.createExternalTable("t", path, "parquet") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table("t"), df) sql( @@ -76,7 +76,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle | path '$path' |) """.stripMargin) - assert(spark.wrapped.tableNames(db).contains("t1")) + assert(spark.sqlContext.tableNames(db).contains("t1")) checkAnswer(spark.table("t1"), df) } } @@ -90,7 +90,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").mode(SaveMode.Overwrite).save(path) spark.catalog.createExternalTable(s"$db.t", path, "parquet") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df) sql( @@ -101,7 +101,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle | path '$path' |) """.stripMargin) - assert(spark.wrapped.tableNames(db).contains("t1")) + assert(spark.sqlContext.tableNames(db).contains("t1")) checkAnswer(spark.table(s"$db.t1"), df) } } @@ -112,11 +112,11 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") df.write.mode(SaveMode.Append).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) checkAnswer(spark.table("t"), df.union(df)) } - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") @@ -127,7 +127,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") df.write.mode(SaveMode.Append).saveAsTable(s"$db.t") - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") @@ -138,7 +138,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) df.write.insertInto(s"$db.t") checkAnswer(spark.table(s"$db.t"), df.union(df)) @@ -150,10 +150,10 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.wrapped.tableNames().contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) } - assert(spark.wrapped.tableNames(db).contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) df.write.insertInto(s"$db.t") checkAnswer(spark.table(s"$db.t"), df.union(df)) @@ -175,21 +175,21 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { sql(s"CREATE TABLE t (key INT)") - assert(spark.wrapped.tableNames().contains("t")) - assert(!spark.wrapped.tableNames("default").contains("t")) + assert(spark.sqlContext.tableNames().contains("t")) + assert(!spark.sqlContext.tableNames("default").contains("t")) } - assert(!spark.wrapped.tableNames().contains("t")) - assert(spark.wrapped.tableNames(db).contains("t")) + assert(!spark.sqlContext.tableNames().contains("t")) + assert(spark.sqlContext.tableNames(db).contains("t")) activateDatabase(db) { sql(s"DROP TABLE t") - assert(!spark.wrapped.tableNames().contains("t")) - assert(!spark.wrapped.tableNames("default").contains("t")) + assert(!spark.sqlContext.tableNames().contains("t")) + assert(!spark.sqlContext.tableNames("default").contains("t")) } - assert(!spark.wrapped.tableNames().contains("t")) - assert(!spark.wrapped.tableNames(db).contains("t")) + assert(!spark.sqlContext.tableNames().contains("t")) + assert(!spark.sqlContext.tableNames(db).contains("t")) } } 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 81f3ea8a6e801..8a31a49d97f08 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 @@ -1417,7 +1417,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) checkAnswer( - spark.wrapped.tables().select('isTemporary).filter('tableName === "t2"), + spark.sqlContext.tables().select('isTemporary).filter('tableName === "t2"), Row(true) ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala index aba60da33fe3e..bb351e20c5e99 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala @@ -61,7 +61,7 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton { (data: Seq[T], tableName: String) (f: => Unit): Unit = { withOrcDataFrame(data) { df => - spark.wrapped.registerDataFrameAsTable(df, tableName) + spark.sqlContext.registerDataFrameAsTable(df, tableName) withTempTable(tableName)(f) } } From c08739afb89ef67e696b2a86ef88c988ad9f4f5d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 22:14:10 -0700 Subject: [PATCH 0294/1470] [SPARK-14990][SQL] Fix checkForSameTypeInputExpr (ignore nullability) ## What changes were proposed in this pull request? This patch fixes a bug in TypeUtils.checkForSameTypeInputExpr. Previously the code was testing on strict equality, which does not taking nullability into account. This is based on https://github.com/apache/spark/pull/12768. This patch fixed a bug there (with empty expression) and added a test case. ## How was this patch tested? Added a new test suite and test case. Closes #12768. Author: Reynold Xin Author: Oleg Danilov Closes #13208 from rxin/SPARK-14990. (cherry picked from commit 3ba34d435c1f61435c2dddc28650cd111e7c1f33) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/util/TypeUtils.scala | 14 ++++-- .../sql/catalyst/util/TypeUtilsSuite.scala | 46 +++++++++++++++++++ 2 files changed, 56 insertions(+), 4 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala index f603cbfb0cc21..7101ca5a17de9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala @@ -42,11 +42,17 @@ object TypeUtils { } def checkForSameTypeInputExpr(types: Seq[DataType], caller: String): TypeCheckResult = { - if (types.distinct.size > 1) { - TypeCheckResult.TypeCheckFailure( - s"input to $caller should all be the same type, but it's " + - types.map(_.simpleString).mkString("[", ", ", "]")) + if (types.size <= 1) { + TypeCheckResult.TypeCheckSuccess } else { + val firstType = types.head + types.foreach { t => + if (!t.sameType(firstType)) { + return TypeCheckResult.TypeCheckFailure( + s"input to $caller should all be the same type, but it's " + + types.map(_.simpleString).mkString("[", ", ", "]")) + } + } TypeCheckResult.TypeCheckSuccess } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala new file mode 100644 index 0000000000000..bc6852ca7e1fd --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.types._ + +class TypeUtilsSuite extends SparkFunSuite { + + private def typeCheckPass(types: Seq[DataType]): Unit = { + assert(TypeUtils.checkForSameTypeInputExpr(types, "a") == TypeCheckSuccess) + } + + private def typeCheckFail(types: Seq[DataType]): Unit = { + assert(TypeUtils.checkForSameTypeInputExpr(types, "a").isInstanceOf[TypeCheckFailure]) + } + + test("checkForSameTypeInputExpr") { + typeCheckPass(Nil) + typeCheckPass(StringType :: Nil) + typeCheckPass(StringType :: StringType :: Nil) + + typeCheckFail(StringType :: IntegerType :: Nil) + typeCheckFail(StringType :: IntegerType :: Nil) + + // Should also work on arrays. See SPARK-14990 + typeCheckPass(ArrayType(StringType, containsNull = true) :: + ArrayType(StringType, containsNull = false) :: Nil) + } +} From 7bb33352f2409493bd7e3880d98508dcb1be888d Mon Sep 17 00:00:00 2001 From: Oleg Danilov Date: Thu, 19 May 2016 22:23:28 -0700 Subject: [PATCH 0295/1470] [SPARK-14261][SQL] Memory leak in Spark Thrift Server Fixed memory leak (HiveConf in the CommandProcessorFactory) Author: Oleg Danilov Closes #12932 from dosoft/SPARK-14261. (cherry picked from commit e384c7fbb94cef3c18e8fa8d06159b76b88b5167) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 0f0c1b0702fb1..71d5c9960a70c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -545,12 +545,14 @@ private[hive] class HiveClientImpl( // Throw an exception if there is an error in query processing. if (response.getResponseCode != 0) { driver.close() + CommandProcessorFactory.clean(conf) throw new QueryExecutionException(response.getErrorMessage) } driver.setMaxRows(maxRows) val results = shim.getDriverResults(driver) driver.close() + CommandProcessorFactory.clean(conf) results case _ => From dcf36ad54598118408c1425e81aa6552f42328c8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 May 2016 14:02:04 +0100 Subject: [PATCH 0296/1470] [SPARK-15057][GRAPHX] Remove stale TODO comment for making `enum` in GraphGenerators This PR removes a stale TODO comment in `GraphGenerators.scala` Just comment removed. Author: Dongjoon Hyun Closes #12839 from dongjoon-hyun/SPARK-15057. (cherry picked from commit 46965cd014fd4ba68bdec15156ec9bcc27d9b217) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 80c6b6838faf5..4da1ecb2a9af3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -209,7 +209,6 @@ object GraphGenerators extends Logging { } } - // TODO(crankshaw) turn result into an enum (or case class for pattern matching} private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { if (a + b + c + d != 1.0) { throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a + b + c + d) From 1dc30f189ac30f070068ca5f60b7b4c85f2adc9e Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 19 May 2016 04:48:36 +0200 Subject: [PATCH 0297/1470] [DOC][MINOR] ml.feature Scala and Python API sync I reviewed Scala and Python APIs for ml.feature and corrected discrepancies. Built docs locally, ran style checks Author: Bryan Cutler Closes #13159 from BryanCutler/ml.feature-api-sync. (cherry picked from commit b1bc5ebdd52ed12aea3fdc7b8f2fa2d00ea09c6b) Signed-off-by: Reynold Xin --- .../org/apache/spark/ml/feature/IDF.scala | 4 +- .../org/apache/spark/ml/feature/PCA.scala | 5 ++- .../apache/spark/ml/feature/RFormula.scala | 4 +- .../spark/ml/feature/VectorIndexer.scala | 3 +- python/pyspark/ml/feature.py | 39 ++++++++++++------- 5 files changed, 36 insertions(+), 19 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index f85f4c65af7d8..08beda6d7515d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -38,12 +38,12 @@ import org.apache.spark.sql.types.StructType private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol { /** - * The minimum of documents in which a term should appear. + * The minimum number of documents in which a term should appear. * Default: 0 * @group param */ final val minDocFreq = new IntParam( - this, "minDocFreq", "minimum of documents in which a term should appear for filtering") + this, "minDocFreq", "minimum number of documents in which a term should appear for filtering") setDefault(minDocFreq -> 0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 141d3b924b4fe..dbbaa5aa46f49 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -53,7 +53,8 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC /** * :: Experimental :: - * PCA trains a model to project vectors to a low-dimensional space using PCA. + * PCA trains a model to project vectors to a lower dimensional space of the top [[PCA!.k]] + * principal components. */ @Experimental class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams @@ -106,7 +107,7 @@ object PCA extends DefaultParamsReadable[PCA] { /** * :: Experimental :: - * Model fitted by [[PCA]]. + * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. * * @param pc A principal components Matrix. Each column is one principal component. * @param explainedVariance A vector of proportions of variance explained by diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index c0feaa01fc861..2916b6d9df3be 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -194,7 +194,9 @@ object RFormula extends DefaultParamsReadable[RFormula] { /** * :: Experimental :: - * A fitted RFormula. Fitting is required to determine the factor levels of formula terms. + * Model fitted by [[RFormula]]. Fitting is required to determine the factor levels of + * formula terms. + * * @param resolvedFormula the fitted R formula. * @param pipelineModel the fitted feature model, including factor to index mappings. */ 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 2bc9d225ac2f6..d814528ec48d1 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 @@ -240,7 +240,8 @@ object VectorIndexer extends DefaultParamsReadable[VectorIndexer] { /** * :: Experimental :: - * Transform categorical features to use 0-based indices instead of their original values. + * Model fitted by [[VectorIndexer]]. 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), diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 983b6a5301ae1..497f2ad68ee51 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -352,7 +352,7 @@ class CountVectorizerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by CountVectorizer. + Model fitted by :py:class:`CountVectorizer`. .. versionadded:: 1.6.0 """ @@ -609,7 +609,7 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab """ minDocFreq = Param(Params._dummy(), "minDocFreq", - "minimum of documents in which a term should appear for filtering", + "minimum number of documents in which a term should appear for filtering", typeConverter=TypeConverters.toInt) @keyword_only @@ -655,7 +655,7 @@ class IDFModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by IDF. + Model fitted by :py:class:`IDF`. .. versionadded:: 1.4.0 """ @@ -1302,7 +1302,8 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, minTokenLength = Param(Params._dummy(), "minTokenLength", "minimum token length (>= 0)", typeConverter=TypeConverters.toInt) - gaps = Param(Params._dummy(), "gaps", "whether regex splits on gaps (True) or matches tokens") + gaps = Param(Params._dummy(), "gaps", "whether regex splits on gaps (True) or matches tokens " + + "(False)") pattern = Param(Params._dummy(), "pattern", "regex pattern (Java dialect) used for tokenizing", typeConverter=TypeConverters.toString) toLowercase = Param(Params._dummy(), "toLowercase", "whether to convert all characters to " + @@ -1549,7 +1550,7 @@ class StandardScalerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by StandardScaler. + Model fitted by :py:class:`StandardScaler`. .. versionadded:: 1.4.0 """ @@ -1641,7 +1642,7 @@ class StringIndexerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by StringIndexer. + Model fitted by :py:class:`StringIndexer`. .. versionadded:: 1.4.0 """ @@ -1907,7 +1908,7 @@ class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Ja """ .. note:: Experimental - Class for indexing categorical feature columns in a dataset of [[Vector]]. + Class for indexing categorical feature columns in a dataset of `Vector`. This has 2 usage modes: - Automatically identify categorical features (default behavior) @@ -2023,7 +2024,17 @@ class VectorIndexerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by VectorIndexer. + Model fitted by :py:class:`VectorIndexer`. + + 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). + Non-ML metadata is not carried over from the input to the output column. + + This maintains vector sparsity. .. versionadded:: 1.4.0 """ @@ -2296,7 +2307,7 @@ class Word2VecModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by Word2Vec. + Model fitted by :py:class:`Word2Vec`. .. versionadded:: 1.4.0 """ @@ -2327,7 +2338,8 @@ class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab """ .. note:: Experimental - PCA trains a model to project vectors to a low-dimensional space using PCA. + PCA trains a model to project vectors to a lower dimensional space of the + top :py:attr:`k` principal components. >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), @@ -2401,7 +2413,7 @@ class PCAModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by PCA. + Model fitted by :py:class:`PCA`. Transforms vectors to a lower dimensional space. .. versionadded:: 1.5.0 """ @@ -2532,7 +2544,8 @@ class RFormulaModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by :py:class:`RFormula`. + Model fitted by :py:class:`RFormula`. Fitting is required to determine the + factor levels of formula terms. .. versionadded:: 1.5.0 """ @@ -2624,7 +2637,7 @@ class ChiSqSelectorModel(JavaModel, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental - Model fitted by ChiSqSelector. + Model fitted by :py:class:`ChiSqSelector`. .. versionadded:: 2.0.0 """ From 642f00980f1de13a0f6d1dc8bc7ed5b0547f3a9d Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sun, 15 May 2016 15:59:49 +0100 Subject: [PATCH 0298/1470] [MINOR] Fix Typos 1,Rename matrix args in BreezeUtil to upper to match the doc 2,Fix several typos in ML and SQL manual tests Author: Zheng RuiFeng Closes #13078 from zhengruifeng/fix_ann. (cherry picked from commit c7efc56c7b6fc99c005b35c335716ff676856c6c) Signed-off-by: Reynold Xin --- docs/ml-guide.md | 2 +- .../org/apache/spark/ml/ann/BreezeUtil.scala | 33 ++++++++--------- .../scala/org/apache/spark/ml/ann/Layer.scala | 36 ++++++++++--------- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 8 ++--- .../sql/expressions/scalalang/typed.scala | 2 +- 6 files changed, 42 insertions(+), 41 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 99167873cd02d..cc353df1ecd00 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -257,7 +257,7 @@ Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/ The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) -for binary data, or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) +for binary data, or a [`MulticlassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) for multiclass problems. The default metric used to choose the best `ParamMap` can be overridden by the `setMetricName` method in each of these evaluators. diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala index 7429f9d652ac5..6bbe7e1cb2134 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala @@ -26,38 +26,39 @@ import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} private[ann] object BreezeUtil { // TODO: switch to MLlib BLAS interface - private def transposeString(a: BDM[Double]): String = if (a.isTranspose) "T" else "N" + private def transposeString(A: BDM[Double]): String = if (A.isTranspose) "T" else "N" /** * DGEMM: C := alpha * A * B + beta * C * @param alpha alpha - * @param a A - * @param b B + * @param A A + * @param B B * @param beta beta - * @param c C + * @param C C */ - def dgemm(alpha: Double, a: BDM[Double], b: BDM[Double], beta: Double, c: BDM[Double]): Unit = { + def dgemm(alpha: Double, A: BDM[Double], B: BDM[Double], beta: Double, C: BDM[Double]): Unit = { // TODO: add code if matrices isTranspose!!! - require(a.cols == b.rows, "A & B Dimension mismatch!") - require(a.rows == c.rows, "A & C Dimension mismatch!") - require(b.cols == c.cols, "A & C Dimension mismatch!") - NativeBLAS.dgemm(transposeString(a), transposeString(b), c.rows, c.cols, a.cols, - alpha, a.data, a.offset, a.majorStride, b.data, b.offset, b.majorStride, - beta, c.data, c.offset, c.rows) + require(A.cols == B.rows, "A & B Dimension mismatch!") + require(A.rows == C.rows, "A & C Dimension mismatch!") + require(B.cols == C.cols, "A & C Dimension mismatch!") + NativeBLAS.dgemm(transposeString(A), transposeString(B), C.rows, C.cols, A.cols, + alpha, A.data, A.offset, A.majorStride, B.data, B.offset, B.majorStride, + beta, C.data, C.offset, C.rows) } /** * DGEMV: y := alpha * A * x + beta * y * @param alpha alpha - * @param a A + * @param A A * @param x x * @param beta beta * @param y y */ - def dgemv(alpha: Double, a: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { - require(a.cols == x.length, "A & b Dimension mismatch!") - NativeBLAS.dgemv(transposeString(a), a.rows, a.cols, - alpha, a.data, a.offset, a.majorStride, x.data, x.offset, x.stride, + def dgemv(alpha: Double, A: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { + require(A.cols == x.length, "A & x Dimension mismatch!") + require(A.rows == y.length, "A & y Dimension mismatch!") + NativeBLAS.dgemv(transposeString(A), A.rows, A.cols, + alpha, A.data, A.offset, A.majorStride, x.data, x.offset, x.stride, beta, y.data, y.offset, y.stride) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 889b7f9f719c7..0a569c49179a6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -66,8 +66,9 @@ private[ann] trait Layer extends Serializable { * @return the layer model */ def createModel(initialWeights: BDV[Double]): LayerModel + /** - * Returns the instance of the layer with random generated weights + * Returns the instance of the layer with random generated weights. * * @param weights vector for weights initialization, must be equal to weightSize * @param random random number generator @@ -85,11 +86,11 @@ private[ann] trait LayerModel extends Serializable { val weights: BDV[Double] /** - * Evaluates the data (process the data through the layer) + * Evaluates the data (process the data through the layer). * Output is allocated based on the size provided by the - * LayerModel implementation and the stack (batch) size + * LayerModel implementation and the stack (batch) size. * Developer is responsible for checking the size of output - * when writing to it + * when writing to it. * * @param data data * @param output output (modified in place) @@ -97,11 +98,11 @@ private[ann] trait LayerModel extends Serializable { def eval(data: BDM[Double], output: BDM[Double]): Unit /** - * Computes the delta for back propagation + * Computes the delta for back propagation. * Delta is allocated based on the size provided by the - * LayerModel implementation and the stack (batch) size + * LayerModel implementation and the stack (batch) size. * Developer is responsible for checking the size of - * prevDelta when writing to it + * prevDelta when writing to it. * * @param delta delta of this layer * @param output output of this layer @@ -110,10 +111,10 @@ private[ann] trait LayerModel extends Serializable { def computePrevDelta(delta: BDM[Double], output: BDM[Double], prevDelta: BDM[Double]): Unit /** - * Computes the gradient - * cumGrad is a wrapper on the part of the weight vector - * size of cumGrad is based on weightSize provided by - * implementation of LayerModel + * Computes the gradient. + * cumGrad is a wrapper on the part of the weight vector. + * Size of cumGrad is based on weightSize provided by + * implementation of LayerModel. * * @param delta delta for this layer * @param input input data @@ -199,11 +200,11 @@ private[ann] object AffineLayerModel { } /** - * Initialize weights randomly in the interval - * Uses [Bottou-88] heuristic [-a/sqrt(in); a/sqrt(in)] - * where a is chosen in a such way that the weight variance corresponds + * Initialize weights randomly in the interval. + * Uses [Bottou-88] heuristic [-a/sqrt(in); a/sqrt(in)], + * where `a` is chosen in such a way that the weight variance corresponds * to the points to the maximal curvature of the activation function - * (which is approximately 2.38 for a standard sigmoid) + * (which is approximately 2.38 for a standard sigmoid). * * @param numIn number of inputs * @param numOut number of outputs @@ -308,7 +309,7 @@ private[ann] class FunctionalLayer (val activationFunction: ActivationFunction) /** * Functional layer model. Holds no weights. * - * @param layer functiona layer + * @param layer functional layer */ private[ann] class FunctionalLayerModel private[ann] (val layer: FunctionalLayer) extends LayerModel { @@ -354,6 +355,7 @@ private[ann] trait TopologyModel extends Serializable { * Array of layer models */ val layerModels: Array[LayerModel] + /** * Forward propagation * @@ -412,7 +414,7 @@ private[ml] object FeedForwardTopology { * Creates a multi-layer perceptron * * @param layerSizes sizes of layers including input and output size - * @param softmaxOnTop wether to use SoftMax or Sigmoid function for an output layer. + * @param softmaxOnTop whether to use SoftMax or Sigmoid function for an output layer. * Softmax is default * @return multilayer perceptron topology */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index ab751f0c04115..f697769bdcdb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.ui.SQLListener -import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState, SQLConf} +import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, LongType, StructType} import org.apache.spark.sql.util.ExecutionListenerManager diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 2021cddab4404..486a440b6f9a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -27,9 +27,7 @@ import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext} -import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.Encoder import org.apache.spark.sql.types._ private[sql] object SQLUtils { @@ -76,7 +74,7 @@ private[sql] object SQLUtils { org.apache.spark.sql.types.MapType(getSQLDataType(keyType), getSQLDataType(valueType)) case r"\Astruct<(.+)${fieldsStr}>\Z" => if (fieldsStr(fieldsStr.length - 1) == ',') { - throw new IllegalArgumentException(s"Invaid type $dataType") + throw new IllegalArgumentException(s"Invalid type $dataType") } val fields = fieldsStr.split(",") val structFields = fields.map { field => @@ -84,11 +82,11 @@ private[sql] object SQLUtils { case r"\A(.+)${fieldName}:(.+)${fieldType}\Z" => createStructField(fieldName, fieldType, true) - case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + case _ => throw new IllegalArgumentException(s"Invalid type $dataType") } } createStructType(structFields) - case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + case _ => throw new IllegalArgumentException(s"Invalid type $dataType") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala index f46a4a7879788..60d7b7d0894d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala @@ -38,7 +38,7 @@ object typed { // The reason we have separate files for Java and Scala is because in the Scala version, we can // use tighter types (primitive types) for return types, whereas in the Java version we can only // use boxed primitive types. - // For example, avg in the Scala veresion returns Scala primitive Double, whose bytecode + // For example, avg in the Scala version returns Scala primitive Double, whose bytecode // signature is just a java.lang.Object; avg in the Java version returns java.lang.Double. // TODO: This is pretty hacky. Maybe we should have an object for implicit encoders. From 2126fb0c2b2bb8ac4c5338df15182fcf8713fb2f Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Thu, 19 May 2016 10:44:26 +0100 Subject: [PATCH 0299/1470] [CORE][MINOR] Remove redundant set master in OutputCommitCoordinatorIntegrationSuite Remove redundant set master in OutputCommitCoordinatorIntegrationSuite, as we are already setting it in SparkContext below on line 43. existing tests Author: Sandeep Singh Closes #13168 from techaddict/minor-1. (cherry picked from commit 3facca5152e685d9c7da96bff5102169740a4a06) Signed-off-by: Reynold Xin --- .../scheduler/OutputCommitCoordinatorIntegrationSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala index 601f1c378c41f..32cdf16dd3318 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala @@ -37,7 +37,6 @@ class OutputCommitCoordinatorIntegrationSuite override def beforeAll(): Unit = { super.beforeAll() val conf = new SparkConf() - .set("master", "local[2,4]") .set("spark.hadoop.outputCommitCoordination.enabled", "true") .set("spark.hadoop.mapred.output.committer.class", classOf[ThrowExceptionOnFirstAttemptOutputCommitter].getCanonicalName) From 1fc0f95eb8abbb9cc8ede2139670e493e6939317 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 22:40:03 -0700 Subject: [PATCH 0300/1470] [HOTFIX] Test compilation error from 52b967f --- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 df029e44c54e5..9b7eac8e0f9c2 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 @@ -1481,8 +1481,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) + spark.sessionState.conf.autoBroadcastJoinThreshold) } } From dd0c7fb39cac44e8f0d73f9884fd1582c25e9cf4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 May 2016 22:46:08 -0700 Subject: [PATCH 0301/1470] Revert "[HOTFIX] Test compilation error from 52b967f" This reverts commit 1fc0f95eb8abbb9cc8ede2139670e493e6939317. --- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 9b7eac8e0f9c2..df029e44c54e5 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 @@ -1481,8 +1481,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.sessionState.conf.autoBroadcastJoinThreshold) + spark.wrapped.conf.autoBroadcastJoinThreshold) assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.sessionState.conf.autoBroadcastJoinThreshold) + spark.wrapped.conf.autoBroadcastJoinThreshold) } } From f8d0177c31d43eab59a7535945f3dfa24e906273 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 May 2016 16:02:52 -0700 Subject: [PATCH 0302/1470] Revert "[SPARK-15392][SQL] fix default value of size estimation of logical plan" This reverts commit fc29b896dae08b957ed15fa681b46162600a4050. (cherry picked from commit 84b23453ddb0a97e3d81306de0a5dcb64f88bdd0) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 --------- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 35d67ca2d8c5d..d1c83f2cebfad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -595,7 +595,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) - def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, Long.MaxValue) + def defaultSizeInBytes: Long = + getConf(DEFAULT_SIZE_IN_BYTES, autoBroadcastJoinThreshold + 1L) def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING) 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 df029e44c54e5..f573abf859610 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 @@ -1476,13 +1476,4 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { getMessage() assert(e1.startsWith("Path does not exist")) } - - test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { - val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) - val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) - assert(df.queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) - assert(df.selectExpr("a").queryExecution.analyzed.statistics.sizeInBytes > - spark.wrapped.conf.autoBroadcastJoinThreshold) - } } From 2ef645724a7f229309a87c5053b0fbdf45d06f52 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 19 May 2016 22:55:44 -0700 Subject: [PATCH 0303/1470] [SPARK-15313][SQL] EmbedSerializerInFilter rule should keep exprIds of output of surrounded SerializeFromObject. ## What changes were proposed in this pull request? The following code: ``` val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() ds.filter(_._1 == "b").select(expr("_1").as[String]).foreach(println(_)) ``` throws an Exception: ``` org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: _1#420 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87) ... Cause: java.lang.RuntimeException: Couldn't find _1#420 in [_1#416,_2#417] at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:94) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87) ... ``` This is because `EmbedSerializerInFilter` rule drops the `exprId`s of output of surrounded `SerializeFromObject`. The analyzed and optimized plans of the above example are as follows: ``` == Analyzed Logical Plan == _1: string Project [_1#420] +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421] +- Filter .apply +- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2 +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] == Optimized Logical Plan == !Project [_1#420] +- Filter .apply +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] ``` This PR fixes `EmbedSerializerInFilter` rule to keep `exprId`s of output of surrounded `SerializeFromObject`. The plans after this patch are as follows: ``` == Analyzed Logical Plan == _1: string Project [_1#420] +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421] +- Filter .apply +- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2 +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] == Optimized Logical Plan == Project [_1#416] +- Filter .apply +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]] ``` ## How was this patch tested? Existing tests and I added a test to check if `filter and then select` works. Author: Takuya UESHIN Closes #13096 from ueshin/issues/SPARK-15313. (cherry picked from commit d5e1c5acde95158db38448526c8afad4a6d21dc2) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 12 ++++++++++-- .../optimizer/TypedFilterOptimizationSuite.scala | 2 +- .../scala/org/apache/spark/sql/DatasetSuite.scala | 7 +++++++ 3 files changed, 18 insertions(+), 3 deletions(-) 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 6825b65e2b282..a6fb34cbfb110 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 @@ -109,7 +109,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: Batch("Typed Filter Optimization", fixedPoint, - EmbedSerializerInFilter) :: + EmbedSerializerInFilter, + RemoveAliasOnlyProject) :: Batch("LocalRelation", fixedPoint, ConvertToLocalRelation) :: Batch("OptimizeCodegen", Once, @@ -1611,7 +1612,14 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] { val newCondition = condition transform { case a: Attribute if a == d.output.head => d.deserializer } - Filter(newCondition, d.child) + val filter = Filter(newCondition, d.child) + + // Adds an extra Project here, to preserve the output expr id of `SerializeFromObject`. + // We will remove it later in RemoveAliasOnlyProject rule. + val objAttrs = filter.output.zip(s.output).map { case (fout, sout) => + Alias(fout, fout.name)(exprId = sout.exprId) + } + Project(objAttrs, filter) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala index 1fae64e3bc6b1..289c16aef47aa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala @@ -67,7 +67,7 @@ class TypedFilterOptimizationSuite extends PlanTest { val deserializer = UnresolvedDeserializer(encoderFor[(Int, Int)].deserializer) val condition = callFunction(f, BooleanType, deserializer) - val expected = input.where(condition).analyze + val expected = input.where(condition).select('_1.as("_1"), '_2.as("_2")).analyze comparePlans(optimized, expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 52e706285c7ab..0ffbd6db12be6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -205,6 +205,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("b", 2)) } + test("filter and then select") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( + ds.filter(_._1 == "b").select(expr("_1").as[String]), + ("b")) + } + test("foreach") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.longAccumulator From 612866473503cbf4f025ae9678cef0f75a94aba8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 22:55:29 -0700 Subject: [PATCH 0304/1470] [HOTFIX] Add back intended change from SPARK-15392 This was accidentally reverted in f8d0177. --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d1c83f2cebfad..35d67ca2d8c5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -595,8 +595,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) - def defaultSizeInBytes: Long = - getConf(DEFAULT_SIZE_IN_BYTES, autoBroadcastJoinThreshold + 1L) + def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, Long.MaxValue) def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING) From 47feebd13dca730c7769bcdc64a0ecc5b6c6c563 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 19 May 2016 23:03:59 -0700 Subject: [PATCH 0305/1470] [SPARK-15335][SQL] Implement TRUNCATE TABLE Command ## What changes were proposed in this pull request? Like TRUNCATE TABLE Command in Hive, TRUNCATE TABLE is also supported by Hive. See the link: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL Below is the related Hive JIRA: https://issues.apache.org/jira/browse/HIVE-446 This PR is to implement such a command for truncate table excluded column truncation(HIVE-4005). ## How was this patch tested? Added a test case. Author: Lianhui Wang Closes #13170 from lianhuiwang/truncate. (cherry picked from commit 09a00510c4759ff87abb0b2fdf1630ddf36ca12c) Signed-off-by: Reynold Xin --- .../spark/sql/execution/SparkSqlParser.scala | 19 +++++ .../spark/sql/execution/command/tables.scala | 53 +++++++++++++ .../sql/hive/execution/HiveCommandSuite.scala | 79 +++++++++++++++++++ 3 files changed, 151 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3045f3af360bc..8af6d07719c9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -350,6 +350,25 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ) } + /** + * Create a [[TruncateTable]] command. + * + * For example: + * {{{ + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * [COLUMNS (col1, col2)] + * }}} + */ + override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { + if (ctx.identifierList != null) { + throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) + } + TruncateTable( + visitTableIdentifier(ctx.tableIdentifier), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) + ) + } + /** * Convert a table property list into a key-value map. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index a3472745371f1..d13492e55070b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -22,6 +22,9 @@ import java.net.URI import java.util.Date import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal + +import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier @@ -270,6 +273,56 @@ case class LoadData( } } +/** + * A command to truncate table. + * + * The syntax of this command is: + * {{{ + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * }}} + */ +case class TruncateTable( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + if (!catalog.tableExists(tableName)) { + logError(s"table '$tableName' in TRUNCATE TABLE does not exist.") + } else if (catalog.isTemporaryTable(tableName)) { + logError(s"table '$tableName' in TRUNCATE TABLE is a temporary table.") + } else { + val locations = if (partitionSpec.isDefined) { + catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) + } else { + val table = catalog.getTableMetadata(tableName) + if (table.partitionColumnNames.nonEmpty) { + catalog.listPartitions(tableName).map(_.storage.locationUri) + } else { + Seq(table.storage.locationUri) + } + } + val hadoopConf = sparkSession.sessionState.newHadoopConf() + locations.foreach { location => + if (location.isDefined) { + val path = new Path(location.get) + try { + val fs = path.getFileSystem(hadoopConf) + fs.delete(path, true) + fs.mkdirs(path) + } catch { + case NonFatal(e) => + throw new AnalysisException( + s"Failed to truncate table '$tableName' when removing data of the path: $path " + + s"because of ${e.toString}") + } + } + } + } + Seq.empty[Row] + } +} + /** * Command that looks like * {{{ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 8225bd69c1c7c..df62ba08b8018 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -269,6 +270,84 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } + test("Truncate Table") { + withTable("non_part_table", "part_table") { + sql( + """ + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + val testData = hiveContext.getHiveFile("data/files/employee.dat").getCanonicalPath + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE non_part_table""") + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Nil) + + val testResults = sql("SELECT * FROM non_part_table").collect() + + intercept[ParseException] { + sql("TRUNCATE TABLE non_part_table COLUMNS (employeeID)") + } + + sql("TRUNCATE TABLE non_part_table") + checkAnswer(sql("SELECT * FROM non_part_table"), Seq.empty[Row]) + + sql( + """ + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="1", d="1")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '1'"), + testResults) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="1", d="2")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + testResults) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="2", d="2")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '2'"), + testResults) + + intercept[ParseException] { + sql("TRUNCATE TABLE part_table PARTITION(c='1', d='1') COLUMNS (employeeID)") + } + + sql("TRUNCATE TABLE part_table PARTITION(c='1', d='1')") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '1'"), + Seq.empty[Row]) + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + testResults) + + intercept[ParseException] { + sql("TRUNCATE TABLE part_table PARTITION(c='1') COLUMNS (employeeID)") + } + + sql("TRUNCATE TABLE part_table PARTITION(c='1')") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1'"), + Seq.empty[Row]) + + sql("TRUNCATE TABLE part_table") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table"), + Seq.empty[Row]) + } + } + test("show columns") { checkAnswer( sql("SHOW COLUMNS IN parquet_tab3"), From 8fb087772d7e226a188e2f3298abb603fd3909ed Mon Sep 17 00:00:00 2001 From: dding3 Date: Mon, 9 May 2016 09:43:07 +0100 Subject: [PATCH 0306/1470] [SPARK-15172][ML] Explicitly tell user initial coefficients is ignored when size mismatch happened in LogisticRegression ## What changes were proposed in this pull request? Explicitly tell user initial coefficients is ignored if its size doesn't match expected size in LogisticRegression ## How was this patch tested? local build Author: dding3 Closes #12948 from dding3/master. (cherry picked from commit a78fbfa619a13421b294328b80c82510ca7efed0) Signed-off-by: Xiangrui Meng --- .../apache/spark/ml/classification/LogisticRegression.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 27380ca5c0a7c..0ab4459bdb9df 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 @@ -367,9 +367,10 @@ class LogisticRegression @Since("1.2.0") ( Vectors.zeros(if ($(fitIntercept)) numFeatures + 1 else numFeatures) if (optInitialModel.isDefined && optInitialModel.get.coefficients.size != numFeatures) { - val vec = optInitialModel.get.coefficients + val vecSize = optInitialModel.get.coefficients.size logWarning( - s"Initial coefficients provided $vec did not match the expected size $numFeatures") + s"Initial coefficients will be ignored!! As its size $vecSize did not match the " + + s"expected size $numFeatures") } if (optInitialModel.isDefined && optInitialModel.get.coefficients.size == numFeatures) { From e4e3e9867e3aba6f3c32bc2c2d060bc681d829c9 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Thu, 19 May 2016 23:21:17 -0700 Subject: [PATCH 0307/1470] [SPARK-15363][ML][EXAMPLE] Example code shouldn't use VectorImplicits._, asML/fromML ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) In this DataFrame example, we use VectorImplicits._, which is private API. Since Vectors object has public API, we use Vectors.fromML instead of implicts. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually run the example. Author: wm624@hotmail.com Closes #13213 from wangmiao1981/ml. (cherry picked from commit 4c7a6b385c79f4de07a89495afce4f8e73b06086) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/examples/ml/DataFrameExample.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 8ed6367787203..c69027babba8d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -25,7 +25,7 @@ import scopt.OptionParser import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.linalg.Vector -import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.sql.{DataFrame, Row, SparkSession} @@ -81,7 +81,7 @@ object DataFrameExample { // Convert features column to an RDD of vectors. val features = df.select("features").rdd.map { case Row(v: Vector) => v } val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())( - (summary, feat) => summary.add(feat), + (summary, feat) => summary.add(Vectors.fromML(feat)), (sum1, sum2) => sum1.merge(sum2)) println(s"Selected features column with average values:\n ${featureSummary.mean.toString}") From 539dfa205dacea72188642f15773a30a99f8e8ac Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Thu, 19 May 2016 23:26:11 -0700 Subject: [PATCH 0308/1470] [SPARK-15398][ML] Update the warning message to recommend ML usage ## What changes were proposed in this pull request? MLlib are not recommended to use, and some methods are even deprecated. Update the warning message to recommend ML usage. ``` def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS |for more conventional use. """.stripMargin) } ``` To ``` def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } ``` ## How was this patch tested? local build Author: Zheng RuiFeng Closes #13190 from zhengruifeng/update_recd. (cherry picked from commit 47a2940da97caa55bbb8bb8ec1d51c9f6d5041c6) Signed-off-by: Xiangrui Meng --- .../main/java/org/apache/spark/examples/JavaHdfsLR.java | 6 ++---- examples/src/main/python/als.py | 4 ++-- examples/src/main/python/kmeans.py | 8 ++++---- examples/src/main/python/logistic_regression.py | 7 ++++--- .../main/scala/org/apache/spark/examples/LocalALS.scala | 4 ++-- .../scala/org/apache/spark/examples/LocalFileLR.scala | 6 ++---- .../scala/org/apache/spark/examples/LocalKMeans.scala | 4 ++-- .../main/scala/org/apache/spark/examples/LocalLR.scala | 6 ++---- .../main/scala/org/apache/spark/examples/SparkALS.scala | 4 ++-- .../scala/org/apache/spark/examples/SparkHdfsLR.scala | 6 ++---- .../scala/org/apache/spark/examples/SparkKMeans.scala | 4 ++-- .../main/scala/org/apache/spark/examples/SparkLR.scala | 6 ++---- 12 files changed, 28 insertions(+), 37 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 31a79ddd3fff1..f64155ce3c0cf 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -32,8 +32,7 @@ * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ public final class JavaHdfsLR { @@ -43,8 +42,7 @@ public final class JavaHdfsLR { static void showWarning() { String warning = "WARN: This is a naive implementation of Logistic Regression " + "and is given as an example!\n" + - "Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD " + - "or org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS " + + "Please use org.apache.spark.ml.classification.LogisticRegression " + "for more conventional use."; System.err.println(warning); } diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 205ca02962bee..f07020b50397b 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -17,7 +17,7 @@ """ This is an example implementation of ALS for learning how to use Spark. Please refer to -ALS in pyspark.mllib.recommendation for more conventional use. +pyspark.ml.recommendation.ALS for more conventional use. This example requires numpy (http://www.numpy.org/) """ @@ -59,7 +59,7 @@ def update(i, vec, mat, ratings): """ 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 + example. Please use pyspark.ml.recommendation.ALS for more conventional use.""", file=sys.stderr) sc = SparkContext(appName="PythonALS") diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 0ea7cfb7025a0..3426e491dc74a 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -17,8 +17,8 @@ """ The K-means algorithm written from scratch against PySpark. In practice, -one may prefer to use the KMeans algorithm in MLlib, as shown in -examples/src/main/python/mllib/kmeans.py. +one may prefer to use the KMeans algorithm in ML, as shown in +examples/src/main/python/ml/kmeans_example.py. This example requires NumPy (http://www.numpy.org/). """ @@ -52,8 +52,8 @@ def closestPoint(p, centers): exit(-1) 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.""", file=sys.stderr) + as an example! Please refer to examples/src/main/python/ml/kmeans_example.py for an + example on how to use ML's KMeans implementation.""", file=sys.stderr) sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index b318b7d87bfdc..7d33be7e81d73 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -20,7 +20,7 @@ to act on batches of input data using efficient matrix operations. In practice, one may prefer to use the LogisticRegression algorithm in -MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. +ML, as shown in examples/src/main/python/ml/logistic_regression_with_elastic_net.py. """ from __future__ import print_function @@ -51,8 +51,9 @@ def readPointBatch(iterator): exit(-1) 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.""", file=sys.stderr) + given as an example! + Please refer to examples/src/main/python/ml/logistic_regression_with_elastic_net.py + to see how ML's implementation is used.""", file=sys.stderr) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index fa1010195551a..97aefac025e55 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -24,7 +24,7 @@ import org.apache.commons.math3.linear._ * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object LocalALS { @@ -96,7 +96,7 @@ object LocalALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index bec89f7c3dff0..3d02ce05619ad 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -26,8 +26,7 @@ import breeze.linalg.{DenseVector, Vector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalFileLR { val D = 10 // Number of dimensions @@ -43,8 +42,7 @@ object LocalFileLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index f8961847f3df2..fca585c2a362b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -29,7 +29,7 @@ import breeze.linalg.{squaredDistance, DenseVector, Vector} * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object LocalKMeans { val N = 1000 @@ -66,7 +66,7 @@ object LocalKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 0baf6db607ad9..13ccc2ae7c3d8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -26,8 +26,7 @@ import breeze.linalg.{DenseVector, Vector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalLR { val N = 10000 // Number of data points @@ -50,8 +49,7 @@ object LocalLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 4263680c6fde3..b06c629802006 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -26,7 +26,7 @@ import org.apache.spark._ * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object SparkALS { @@ -81,7 +81,7 @@ object SparkALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 7463b868ff19b..c514eb0fa51ae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -31,8 +31,7 @@ import org.apache.spark._ * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkHdfsLR { val D = 10 // Number of dimensions @@ -54,8 +53,7 @@ object SparkHdfsLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index d9f94a42b1a0b..676164806e196 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkConf, SparkContext} * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object SparkKMeans { @@ -52,7 +52,7 @@ object SparkKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index acd8656b65a69..718f84f6450e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -31,8 +31,7 @@ import org.apache.spark._ * Usage: SparkLR [slices] * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkLR { val N = 10000 // Number of data points @@ -55,8 +54,7 @@ object SparkLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } From 5f73f627f966926ac477663642903f175cad54d0 Mon Sep 17 00:00:00 2001 From: sethah Date: Thu, 19 May 2016 23:29:37 -0700 Subject: [PATCH 0309/1470] [SPARK-15394][ML][DOCS] User guide typos and grammar audit ## What changes were proposed in this pull request? Correct some typos and incorrectly worded sentences. ## How was this patch tested? Doc changes only. Note that many of these changes were identified by whomfire01 Author: sethah Closes #13180 from sethah/ml_guide_audit. (cherry picked from commit 5e203505f1a092e5849ebd01d9ff9e4fc6cdc34a) Signed-off-by: Xiangrui Meng --- docs/ml-classification-regression.md | 28 ++++++++--------- docs/ml-clustering.md | 2 +- docs/ml-collaborative-filtering.md | 6 ++-- docs/ml-features.md | 47 ++++++++++++++-------------- docs/ml-guide.md | 8 ++--- 5 files changed, 45 insertions(+), 46 deletions(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index f6a6937e29f04..f1a21f436f133 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -236,9 +236,9 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). MLPC consists of multiple layers of nodes. -Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes maps inputs to the outputs -by performing linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. -It can be written in matrix form for MLPC with `$K+1$` layers as follows: +Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes map inputs to outputs +by a linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. +This can be written in matrix form for MLPC with `$K+1$` layers as follows: `\[ \mathrm{y}(\x) = \mathrm{f_K}(...\mathrm{f_2}(\wv_2^T\mathrm{f_1}(\wv_1^T \x+b_1)+b_2)...+b_K) \]` @@ -252,7 +252,7 @@ Nodes in the output layer use softmax function: \]` The number of nodes `$N$` in the output layer corresponds to the number of classes. -MLPC employs backpropagation for learning the model. We use logistic loss function for optimization and L-BFGS as optimization routine. +MLPC employs backpropagation for learning the model. We use the logistic loss function for optimization and L-BFGS as an optimization routine. **Example** @@ -311,9 +311,9 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat ## Naive Bayes -[Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple +[Naive Bayes classifiers](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) are a family of simple probabilistic classifiers based on applying Bayes' theorem with strong (naive) independence -assumptions between the features. The spark.ml implementation currently supports both [multinomial +assumptions between the features. The `spark.ml` implementation currently supports both [multinomial naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). More information can be found in the section on [Naive Bayes in MLlib](mllib-naive-bayes.html#naive-bayes-sparkmllib). @@ -482,11 +482,11 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression. In `spark.ml`, we implement the [Accelerated failure time (AFT)](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) model which is a parametric survival regression model for censored data. -It describes a model for the log of survival time, so it's often called -log-linear model for survival analysis. Different from +It describes a model for the log of survival time, so it's often called a +log-linear model for survival analysis. Different from a [Proportional hazards](https://en.wikipedia.org/wiki/Proportional_hazards_model) model -designed for the same purpose, the AFT model is more easily to parallelize -because each instance contribute to the objective function independently. +designed for the same purpose, the AFT model is easier to parallelize +because each instance contributes to the objective function independently. Given the values of the covariates $x^{'}$, for random lifetime $t_{i}$ of subjects i = 1, ..., n, with possible right-censoring, @@ -501,10 +501,10 @@ assumes the form: \iota(\beta,\sigma)=\sum_{i=1}^{n}[-\delta_{i}\log\sigma+\delta_{i}\log{f_{0}}(\epsilon_{i})+(1-\delta_{i})\log{S_{0}(\epsilon_{i})}] \]` Where $S_{0}(\epsilon_{i})$ is the baseline survivor function, -and $f_{0}(\epsilon_{i})$ is corresponding density function. +and $f_{0}(\epsilon_{i})$ is the corresponding density function. The most commonly used AFT model is based on the Weibull distribution of the survival time. -The Weibull distribution for lifetime corresponding to extreme value distribution for +The Weibull distribution for lifetime corresponds to the extreme value distribution for the log of the lifetime, and the $S_{0}(\epsilon)$ function is: `\[ S_{0}(\epsilon_{i})=\exp(-e^{\epsilon_{i}}) @@ -513,7 +513,7 @@ the $f_{0}(\epsilon_{i})$ function is: `\[ f_{0}(\epsilon_{i})=e^{\epsilon_{i}}\exp(-e^{\epsilon_{i}}) \]` -The log-likelihood function for AFT model with Weibull distribution of lifetime is: +The log-likelihood function for AFT model with a Weibull distribution of lifetime is: `\[ \iota(\beta,\sigma)= -\sum_{i=1}^n[\delta_{i}\log\sigma-\delta_{i}\epsilon_{i}+e^{\epsilon_{i}}] \]` @@ -529,7 +529,7 @@ The gradient functions for $\beta$ and $\log\sigma$ respectively are: The AFT model can be formulated as a convex optimization problem, i.e. the task of finding a minimizer of a convex function $-\iota(\beta,\sigma)$ -that depends coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. +that depends on the coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. The optimization algorithm underlying the implementation is L-BFGS. The implementation matches the result from R's survival function [survreg](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html) diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 33e4b7b0d2cc8..8656eb4001f4b 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -89,7 +89,7 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering. ## Latent Dirichlet allocation (LDA) `LDA` is implemented as an `Estimator` that supports both `EMLDAOptimizer` and `OnlineLDAOptimizer`, -and generates a `LDAModel` as the base models. Expert users may cast a `LDAModel` generated by +and generates a `LDAModel` as the base model. Expert users may cast a `LDAModel` generated by `EMLDAOptimizer` to a `DistributedLDAModel` if needed.
    diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md index 4514a358e12f2..bd3d527d9a0e3 100644 --- a/docs/ml-collaborative-filtering.md +++ b/docs/ml-collaborative-filtering.md @@ -60,7 +60,7 @@ best parameter learned from a sampled subset to the full dataset and expect simi
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are @@ -91,7 +91,7 @@ val als = new ALS()
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are @@ -122,7 +122,7 @@ ALS als = new ALS()
    -In the following example, we load rating data from the +In the following example, we load ratings data from the [MovieLens dataset](http://grouplens.org/datasets/movielens/), each row consisting of a user, a movie, a rating and a timestamp. We then train an ALS model which assumes, by default, that the ratings are diff --git a/docs/ml-features.md b/docs/ml-features.md index c44ace91f23f6..3db24a3840599 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -26,7 +26,7 @@ to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and th Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, while document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. If we only use term frequency to measure the importance, it is very easy to over-emphasize terms that appear very -often but carry little information about the document, e.g., "a", "the", and "of". If a term appears +often but carry little information about the document, e.g. "a", "the", and "of". If a term appears very often across the corpus, it means it doesn't carry special information about a particular document. Inverse document frequency is a numerical measure of how much information a term provides: `\[ @@ -50,7 +50,7 @@ A raw feature is mapped into an index (term) by applying a hash function. Then t are calculated based on the mapped indices. This approach avoids the need to compute a global term-to-index map, which can be expensive for a large corpus, but it suffers from potential hash collisions, where different raw features may become the same term after hashing. To reduce the -chance of collision, we can increase the target feature dimension, i.e., the number of buckets +chance of collision, we can increase the target feature dimension, i.e. the number of buckets of the hash table. Since a simple modulo is used to transform the hash function to a column index, it is advisable to use a power of two as the feature dimension, otherwise the features will not be mapped evenly to the columns. The default feature dimension is `$2^{18} = 262,144$`. @@ -104,7 +104,7 @@ the [IDF Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.IDF) for mor `Word2Vec` is an `Estimator` which takes sequences of words representing documents and trains a `Word2VecModel`. The model maps each word to a unique fixed-size vector. The `Word2VecModel` transforms each document into a vector using the average of all words in the document; this vector -can then be used for as features for prediction, document similarity calculations, etc. +can then be used as features for prediction, document similarity calculations, etc. Please refer to the [MLlib user guide on Word2Vec](mllib-feature-extraction.html#word2vec) for more details. @@ -140,12 +140,12 @@ for more details on the API. `CountVectorizer` and `CountVectorizerModel` aim to help convert a collection of text documents to vectors of token counts. When an a-priori dictionary is not available, `CountVectorizer` can - be used as an `Estimator` to extract the vocabulary and generates a `CountVectorizerModel`. The + be used as an `Estimator` to extract the vocabulary, and generates a `CountVectorizerModel`. The model produces sparse representations for the documents over the vocabulary, which can then be passed to other algorithms like LDA. During the fitting process, `CountVectorizer` will select the top `vocabSize` words ordered by - term frequency across the corpus. An optional parameter "minDF" also affect the fitting process + term frequency across the corpus. An optional parameter "minDF" also affects the fitting process by specifying the minimum number (or fraction if < 1.0) of documents a term must appear in to be included in the vocabulary. @@ -161,8 +161,8 @@ Assume that we have the following DataFrame with columns `id` and `texts`: ~~~~ each row in `texts` is a document of type Array[String]. -Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c), -then the output column "vector" after transformation contains: +Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c). +Then the output column "vector" after transformation contains: ~~~~ id | texts | vector @@ -171,7 +171,7 @@ then the output column "vector" after transformation contains: 1 | Array("a", "b", "b", "c", "a") | (3,[0,1,2],[2.0,2.0,1.0]) ~~~~ -each vector represents the token counts of the document over the vocabulary. +Each vector represents the token counts of the document over the vocabulary.
    @@ -477,8 +477,7 @@ for more details on the API. ## StringIndexer `StringIndexer` encodes a string column of labels to a column of label indices. -The indices are in `[0, numLabels)`, ordered by label frequencies. -So the most frequent label gets index `0`. +The indices are in `[0, numLabels)`, ordered by label frequencies, so the most frequent label gets index `0`. If the input column is numeric, we cast it to string and index the string values. When downstream pipeline components such as `Estimator` or `Transformer` make use of this string-indexed label, you must set the input @@ -585,7 +584,7 @@ for more details on the API. ## IndexToString Symmetrically to `StringIndexer`, `IndexToString` maps a column of label indices -back to a column containing the original labels as strings. The common use case +back to a column containing the original labels as strings. A common use case is to produce indices from labels with `StringIndexer`, train a model with those indices and retrieve the original labels from the column of predicted indices with `IndexToString`. However, you are free to supply your own labels. @@ -652,7 +651,7 @@ for more details on the API. ## OneHotEncoder -[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features +[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features.
    @@ -888,7 +887,7 @@ for more details on the API. * `splits`: Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which also includes y. Splits should be strictly increasing. Values at -inf, inf must be explicitly provided to cover all Double values; Otherwise, values outside the splits specified will be treated as errors. Two examples of `splits` are `Array(Double.NegativeInfinity, 0.0, 1.0, Double.PositiveInfinity)` and `Array(0.0, 1.0, 2.0)`. -Note that if you have no idea of the upper bound and lower bound of the targeted column, you would better add the `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potential out of Bucketizer bounds exception. +Note that if you have no idea of the upper and lower bounds of the targeted column, you should add `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potential out of Bucketizer bounds exception. Note also that the splits that you provided have to be in strictly increasing order, i.e. `s0 < s1 < s2 < ... < sn`. @@ -976,7 +975,7 @@ for more details on the API. Currently we only support SQL syntax like `"SELECT ... FROM __THIS__ ..."` where `"__THIS__"` represents the underlying table of the input dataset. The select clause specifies the fields, constants, and expressions to display in -the output, it can be any select clause that Spark SQL supports. Users can also +the output, and can be any select clause that Spark SQL supports. Users can also use Spark SQL built-in function and UDFs to operate on these selected columns. For example, `SQLTransformer` supports statements like: @@ -1121,7 +1120,7 @@ Assume that we have a DataFrame with the columns `id`, `hour`: ~~~ `hour` is a continuous feature with `Double` type. We want to turn the continuous feature into -categorical one. Given `numBuckets = 3`, we should get the following DataFrame: +a categorical one. Given `numBuckets = 3`, we should get the following DataFrame: ~~~ id | hour | result @@ -1171,19 +1170,19 @@ for more details on the API. `VectorSlicer` is a transformer that takes a feature vector and outputs a new feature vector with a sub-array of the original features. It is useful for extracting features from a vector column. -`VectorSlicer` accepts a vector column with a specified indices, then outputs a new vector column +`VectorSlicer` accepts a vector column with specified indices, then outputs a new vector column whose values are selected via those indices. There are two types of indices, - 1. Integer indices that represents the indices into the vector, `setIndices()`; + 1. Integer indices that represent the indices into the vector, `setIndices()`. - 2. String indices that represents the names of features into the vector, `setNames()`. + 2. String indices that represent the names of features into the vector, `setNames()`. *This requires the vector column to have an `AttributeGroup` since the implementation matches on the name field of an `Attribute`.* Specification by integer and string are both acceptable. Moreover, you can use integer index and string name simultaneously. At least one feature must be selected. Duplicate features are not allowed, so there can be no overlap between selected indices and names. Note that if names of -features are selected, an exception will be threw out when encountering with empty input attributes. +features are selected, an exception will be thrown if empty input attributes are encountered. The output vector will order features with the selected indices first (in the order given), followed by the selected names (in the order given). @@ -1198,8 +1197,8 @@ Suppose that we have a DataFrame with the column `userFeatures`: [0.0, 10.0, 0.5] ~~~ -`userFeatures` is a vector column that contains three user features. Assuming that the first column -of `userFeatures` are all zeros, so we want to remove it and only the last two columns are selected. +`userFeatures` is a vector column that contains three user features. Assume that the first column +of `userFeatures` are all zeros, so we want to remove it and select only the last two columns. The `VectorSlicer` selects the last two elements with `setIndices(1, 2)` then produces a new vector column named `features`: @@ -1209,7 +1208,7 @@ column named `features`: [0.0, 10.0, 0.5] | [10.0, 0.5] ~~~ -Suppose also that we have a potential input attributes for the `userFeatures`, i.e. +Suppose also that we have potential input attributes for the `userFeatures`, i.e. `["f1", "f2", "f3"]`, then we can use `setNames("f2", "f3")` to select them. ~~~ @@ -1337,8 +1336,8 @@ id | features | clicked 9 | [1.0, 0.0, 15.0, 0.1] | 0.0 ~~~ -If we use `ChiSqSelector` with a `numTopFeatures = 1`, then according to our label `clicked` the -last column in our `features` chosen as the most useful feature: +If we use `ChiSqSelector` with `numTopFeatures = 1`, then according to our label `clicked` the +last column in our `features` is chosen as the most useful feature: ~~~ id | features | clicked | selectedFeatures diff --git a/docs/ml-guide.md b/docs/ml-guide.md index cc353df1ecd00..dae86d84804d0 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -47,7 +47,7 @@ mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. * **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. -E.g., an ML model is a `Transformer` which transforms `DataFrame` with features into a `DataFrame` with predictions. +E.g., an ML model is a `Transformer` which transforms a `DataFrame` with features into a `DataFrame` with predictions. * **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. @@ -292,13 +292,13 @@ However, it is also a well-established method for choosing parameters which is m ## Example: model selection via train validation split In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. -`TrainValidationSplit` only evaluates each combination of parameters once as opposed to k times in - case of `CrossValidator`. It is therefore less expensive, +`TrainValidationSplit` only evaluates each combination of parameters once, as opposed to k times in + the case of `CrossValidator`. It is therefore less expensive, but will not produce as reliable results when the training dataset is not sufficiently large. `TrainValidationSplit` takes an `Estimator`, a set of `ParamMap`s provided in the `estimatorParamMaps` parameter, and an `Evaluator`. -It begins by splitting the dataset into two parts using `trainRatio` parameter +It begins by splitting the dataset into two parts using the `trainRatio` parameter which are used as separate training and test datasets. For example with `$trainRatio=0.75$` (default), `TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. Similar to `CrossValidator`, `TrainValidationSplit` also iterates through the set of `ParamMap`s. From 9963fd4398d7ef6c632fc9851ef64bd71a87aa12 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 May 2016 23:35:20 -0700 Subject: [PATCH 0310/1470] [SPARK-15339][ML] ML 2.0 QA: Scala APIs and code audit for regression ## What changes were proposed in this pull request? * ```GeneralizedLinearRegression``` API docs enhancement. * The default value of ```GeneralizedLinearRegression``` ```linkPredictionCol``` is not set rather than empty. This will consistent with other similar params such as ```weightCol``` * Make some methods more private. * Fix a minor bug of LinearRegression. * Fix some other issues. ## How was this patch tested? Existing tests. Author: Yanbo Liang Closes #13129 from yanboliang/spark-15339. (cherry picked from commit c94b34ebbf4c6ce353c899c571beb34e8db98917) Signed-off-by: Xiangrui Meng --- .../ml/regression/AFTSurvivalRegression.scala | 4 +- .../GeneralizedLinearRegression.scala | 74 ++++++++++--------- .../ml/regression/IsotonicRegression.scala | 4 +- .../ml/regression/LinearRegression.scala | 8 +- .../ml/regression/LinearRegressionSuite.scala | 15 +++- 5 files changed, 58 insertions(+), 47 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index cc16c2f038be8..e63eb71080604 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -89,8 +89,8 @@ private[regression] trait AFTSurvivalRegressionParams extends Params def getQuantilesCol: String = $(quantilesCol) /** Checks whether the input has quantiles column name. */ - protected[regression] def hasQuantilesCol: Boolean = { - isDefined(quantilesCol) && $(quantilesCol) != "" + private[regression] def hasQuantilesCol: Boolean = { + isDefined(quantilesCol) && $(quantilesCol).nonEmpty } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index e8474d035ee62..adbdd345e92ec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -43,6 +43,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam with HasFitIntercept with HasMaxIter with HasTol with HasRegParam with HasWeightCol with HasSolver with Logging { + import GeneralizedLinearRegression._ + /** * Param for the name of family which is a description of the error distribution * to be used in the model. @@ -54,8 +56,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") final val family: Param[String] = new Param(this, "family", "The name of family which is a description of the error distribution to be used in the " + - "model. Supported options: gaussian(default), binomial, poisson and gamma.", - ParamValidators.inArray[String](GeneralizedLinearRegression.supportedFamilyNames.toArray)) + s"model. Supported options: ${supportedFamilyNames.mkString(", ")}.", + ParamValidators.inArray[String](supportedFamilyNames.toArray)) /** @group getParam */ @Since("2.0.0") @@ -71,9 +73,8 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam @Since("2.0.0") final val link: Param[String] = new Param(this, "link", "The name of link function " + "which provides the relationship between the linear predictor and the mean of the " + - "distribution function. Supported options: identity, log, inverse, logit, probit, " + - "cloglog and sqrt.", - ParamValidators.inArray[String](GeneralizedLinearRegression.supportedLinkNames.toArray)) + s"distribution function. Supported options: ${supportedLinkNames.mkString(", ")}", + ParamValidators.inArray[String](supportedLinkNames.toArray)) /** @group getParam */ @Since("2.0.0") @@ -81,19 +82,23 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam /** * Param for link prediction (linear predictor) column name. - * Default is empty, which means we do not output link prediction. + * Default is not set, which means we do not output link prediction. * * @group param */ @Since("2.0.0") final val linkPredictionCol: Param[String] = new Param[String](this, "linkPredictionCol", "link prediction (linear predictor) column name") - setDefault(linkPredictionCol, "") /** @group getParam */ @Since("2.0.0") def getLinkPredictionCol: String = $(linkPredictionCol) + /** Checks whether we should output link prediction. */ + private[regression] def hasLinkPredictionCol: Boolean = { + isDefined(linkPredictionCol) && $(linkPredictionCol).nonEmpty + } + import GeneralizedLinearRegression._ @Since("2.0.0") @@ -107,7 +112,7 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam s"with ${$(family)} family does not support ${$(link)} link function.") } val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) - if ($(linkPredictionCol).nonEmpty) { + if (hasLinkPredictionCol) { SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) } else { newSchema @@ -205,7 +210,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val /** * Sets the value of param [[weightCol]]. * If this is not set or empty, we treat all instance weights as 1.0. - * Default is empty, so all instances have weight one. + * Default is not set, so all instances have weight one. * * @group setParam */ @@ -214,7 +219,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val /** * Sets the solver algorithm used for optimization. - * Currently only support "irls" which is also the default solver. + * Currently only supports "irls" which is also the default solver. * * @group setParam */ @@ -239,10 +244,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val } val familyAndLink = new FamilyAndLink(familyObj, linkObj) - val numFeatures = dataset.select(col($(featuresCol))).limit(1).rdd - .map { case Row(features: Vector) => - features.size - }.first() + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size if (numFeatures > WeightedLeastSquares.MAX_NUM_FEATURES) { val msg = "Currently, GeneralizedLinearRegression only supports number of features" + s" <= ${WeightedLeastSquares.MAX_NUM_FEATURES}. Found $numFeatures in the input dataset." @@ -294,7 +296,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def load(path: String): GeneralizedLinearRegression = super.load(path) /** Set of family and link pairs that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedFamilyAndLinkPairs = Set( + private[regression] lazy val supportedFamilyAndLinkPairs = Set( Gaussian -> Identity, Gaussian -> Log, Gaussian -> Inverse, Binomial -> Logit, Binomial -> Probit, Binomial -> CLogLog, Poisson -> Log, Poisson -> Identity, Poisson -> Sqrt, @@ -302,17 +304,17 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine ) /** Set of family names that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name) + private[regression] lazy val supportedFamilyNames = supportedFamilyAndLinkPairs.map(_._1.name) /** Set of link names that GeneralizedLinearRegression supports. */ - private[ml] lazy val supportedLinkNames = supportedFamilyAndLinkPairs.map(_._2.name) + private[regression] lazy val supportedLinkNames = supportedFamilyAndLinkPairs.map(_._2.name) - private[ml] val epsilon: Double = 1E-16 + private[regression] val epsilon: Double = 1E-16 /** * Wrapper of family and link combination used in the model. */ - private[ml] class FamilyAndLink(val family: Family, val link: Link) extends Serializable { + private[regression] class FamilyAndLink(val family: Family, val link: Link) extends Serializable { /** Linear predictor based on given mu. */ def predict(mu: Double): Double = link.link(family.project(mu)) @@ -359,7 +361,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * * @param name the name of the family. */ - private[ml] abstract class Family(val name: String) extends Serializable { + private[regression] abstract class Family(val name: String) extends Serializable { /** The default link instance of this family. */ val defaultLink: Link @@ -391,7 +393,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine def project(mu: Double): Double = mu } - private[ml] object Family { + private[regression] object Family { /** * Gets the [[Family]] object from its name. @@ -412,7 +414,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Gaussian exponential family distribution. * The default link for the Gaussian family is the identity link. */ - private[ml] object Gaussian extends Family("gaussian") { + private[regression] object Gaussian extends Family("gaussian") { val defaultLink: Link = Identity @@ -448,7 +450,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Binomial exponential family distribution. * The default link for the Binomial family is the logit link. */ - private[ml] object Binomial extends Family("binomial") { + private[regression] object Binomial extends Family("binomial") { val defaultLink: Link = Logit @@ -492,7 +494,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Poisson exponential family distribution. * The default link for the Poisson family is the log link. */ - private[ml] object Poisson extends Family("poisson") { + private[regression] object Poisson extends Family("poisson") { val defaultLink: Link = Log @@ -533,7 +535,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * Gamma exponential family distribution. * The default link for the Gamma family is the inverse link. */ - private[ml] object Gamma extends Family("gamma") { + private[regression] object Gamma extends Family("gamma") { val defaultLink: Link = Inverse @@ -578,7 +580,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine * * @param name the name of link function. */ - private[ml] abstract class Link(val name: String) extends Serializable { + private[regression] abstract class Link(val name: String) extends Serializable { /** The link function. */ def link(mu: Double): Double @@ -590,7 +592,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine def unlink(eta: Double): Double } - private[ml] object Link { + private[regression] object Link { /** * Gets the [[Link]] object from its name. @@ -611,7 +613,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine } } - private[ml] object Identity extends Link("identity") { + private[regression] object Identity extends Link("identity") { override def link(mu: Double): Double = mu @@ -620,7 +622,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = eta } - private[ml] object Logit extends Link("logit") { + private[regression] object Logit extends Link("logit") { override def link(mu: Double): Double = math.log(mu / (1.0 - mu)) @@ -629,7 +631,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 / (1.0 + math.exp(-1.0 * eta)) } - private[ml] object Log extends Link("log") { + private[regression] object Log extends Link("log") { override def link(mu: Double): Double = math.log(mu) @@ -638,7 +640,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = math.exp(eta) } - private[ml] object Inverse extends Link("inverse") { + private[regression] object Inverse extends Link("inverse") { override def link(mu: Double): Double = 1.0 / mu @@ -647,7 +649,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 / eta } - private[ml] object Probit extends Link("probit") { + private[regression] object Probit extends Link("probit") { override def link(mu: Double): Double = dist.Gaussian(0.0, 1.0).icdf(mu) @@ -658,7 +660,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = dist.Gaussian(0.0, 1.0).cdf(eta) } - private[ml] object CLogLog extends Link("cloglog") { + private[regression] object CLogLog extends Link("cloglog") { override def link(mu: Double): Double = math.log(-1.0 * math.log(1 - mu)) @@ -667,7 +669,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine override def unlink(eta: Double): Double = 1.0 - math.exp(-1.0 * math.exp(eta)) } - private[ml] object Sqrt extends Link("sqrt") { + private[regression] object Sqrt extends Link("sqrt") { override def link(mu: Double): Double = math.sqrt(mu) @@ -732,7 +734,7 @@ class GeneralizedLinearRegressionModel private[ml] ( if ($(predictionCol).nonEmpty) { output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) } - if ($(linkPredictionCol).nonEmpty) { + if (hasLinkPredictionCol) { output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)))) } output.toDF() @@ -860,7 +862,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( */ @Since("2.0.0") val predictionCol: String = { - if (origModel.isDefined(origModel.predictionCol) && origModel.getPredictionCol != "") { + if (origModel.isDefined(origModel.predictionCol) && origModel.getPredictionCol.nonEmpty) { origModel.getPredictionCol } else { "prediction_" + java.util.UUID.randomUUID.toString diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index ba0f59e89bb24..d16e8e3f6b259 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -69,8 +69,8 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures setDefault(isotonic -> true, featureIndex -> 0) /** Checks whether the input has weight column. */ - protected[ml] def hasWeightCol: Boolean = { - isDefined(weightCol) && $(weightCol) != "" + private[regression] def hasWeightCol: Boolean = { + isDefined(weightCol) && $(weightCol).nonEmpty } /** 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 a702f02c9143c..ff1038cbf185f 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 @@ -161,9 +161,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String override protected def train(dataset: Dataset[_]): LinearRegressionModel = { // Extract the number of features before deciding optimization solver. - val numFeatures = dataset.select(col($(featuresCol))).limit(1).rdd.map { - case Row(features: Vector) => features.size - }.first() + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) if (($(solver) == "auto" && $(elasticNetParam) == 0.0 && @@ -242,7 +240,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val coefficients = Vectors.sparse(numFeatures, Seq()) val intercept = yMean - val model = new LinearRegressionModel(uid, coefficients, intercept) + val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) // Handle possible missing or invalid prediction columns val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() @@ -254,7 +252,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String model, Array(0D), Array(0D)) - return copyValues(model.setSummary(trainingSummary)) + return model.setSummary(trainingSummary) } else { require($(regParam) == 0.0, "The standard deviation of the label is zero. " + "Model cannot be regularized.") diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index 332d331a47737..265f2f45c45fe 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -610,20 +610,31 @@ class LinearRegressionSuite val model1 = new LinearRegression() .setFitIntercept(fitIntercept) .setWeightCol("weight") + .setPredictionCol("myPrediction") .setSolver(solver) .fit(datasetWithWeightConstantLabel) val actual1 = Vectors.dense(model1.intercept, model1.coefficients(0), model1.coefficients(1)) assert(actual1 ~== expected(idx) absTol 1e-4) + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightConstantLabel.schema.fieldNames.toSet + model1.getPredictionCol) + .subsetOf(model1.summary.predictions.schema.fieldNames.toSet)) + val model2 = new LinearRegression() .setFitIntercept(fitIntercept) .setWeightCol("weight") + .setPredictionCol("myPrediction") .setSolver(solver) .fit(datasetWithWeightZeroLabel) val actual2 = Vectors.dense(model2.intercept, model2.coefficients(0), model2.coefficients(1)) assert(actual2 ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1e-4) + + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightZeroLabel.schema.fieldNames.toSet + model2.getPredictionCol) + .subsetOf(model2.summary.predictions.schema.fieldNames.toSet)) + idx += 1 } } @@ -672,7 +683,7 @@ class LinearRegressionSuite test("linear regression model training summary") { Seq("auto", "l-bfgs", "normal").foreach { solver => - val trainer = new LinearRegression().setSolver(solver) + val trainer = new LinearRegression().setSolver(solver).setPredictionCol("myPrediction") val model = trainer.fit(datasetWithDenseFeature) val trainerNoPredictionCol = trainer.setPredictionCol("") val modelNoPredictionCol = trainerNoPredictionCol.fit(datasetWithDenseFeature) @@ -682,7 +693,7 @@ class LinearRegressionSuite assert(modelNoPredictionCol.hasSummary) // Schema should be a superset of the input dataset - assert((datasetWithDenseFeature.schema.fieldNames.toSet + "prediction").subsetOf( + assert((datasetWithDenseFeature.schema.fieldNames.toSet + model.getPredictionCol).subsetOf( model.summary.predictions.schema.fieldNames.toSet)) // Validate that we re-insert a prediction column for evaluation val modelNoPredictionColFieldNames From 4d13348f861fd391c64433a1691c1b7f33a36db1 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 20 May 2016 14:38:25 +0800 Subject: [PATCH 0311/1470] [SPARK-15367][SQL] Add refreshTable back #### What changes were proposed in this pull request? `refreshTable` was a method in `HiveContext`. It was deleted accidentally while we were migrating the APIs. This PR is to add it back to `HiveContext`. In addition, in `SparkSession`, we put it under the catalog namespace (`SparkSession.catalog.refreshTable`). #### How was this patch tested? Changed the existing test cases to use the function `refreshTable`. Also added a test case for refreshTable in `hivecontext-compatibility` Author: gatorsmile Closes #13156 from gatorsmile/refreshTable. (cherry picked from commit 39fd469078271aa12f3163606000e06e382d35dc) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalog/Catalog.scala | 13 +++++++++ .../spark/sql/execution/datasources/ddl.scala | 21 +++------------ .../spark/sql/internal/CatalogImpl.scala | 27 +++++++++++++++++++ .../spark/sql/internal/SessionState.scala | 4 --- .../sql/hive/MetastoreDataSourcesSuite.scala | 4 +-- .../spark/sql/hive/MultiDatabaseSuite.scala | 4 +-- .../apache/spark/sql/hive/HiveContext.scala | 12 +++++++++ 7 files changed, 59 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 49c07427617f3..a99bc3bff6eea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -211,4 +211,17 @@ abstract class Catalog { */ def clearCache(): Unit + /** + * 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. + * + * If this table is cached as an InMemoryRelation, drop the original cached version and make the + * new version cached lazily. + * + * @since 2.0.0 + */ + def refreshTable(tableName: String): Unit + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 68238dbb46e9f..78b1db16826e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -126,24 +126,9 @@ case class RefreshTable(tableIdent: TableIdentifier) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - // Refresh the given table's metadata first. - sparkSession.sessionState.catalog.refreshTable(tableIdent) - - // If this table is cached as a InMemoryColumnarRelation, drop the original - // cached version and make the new version cached lazily. - val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent) - // Use lookupCachedData directly since RefreshTable also takes databaseName. - val isCached = sparkSession.cacheManager.lookupCachedData(logicalPlan).nonEmpty - if (isCached) { - // Create a data frame to represent the table. - // TODO: Use uncacheTable once it supports database name. - val df = Dataset.ofRows(sparkSession, logicalPlan) - // Uncache the logicalPlan. - sparkSession.cacheManager.tryUncacheQuery(df, blocking = true) - // Cache it again. - sparkSession.cacheManager.cacheQuery(df, Some(tableIdent.table)) - } - + // Refresh the given table's metadata. If this table is cached as an InMemoryRelation, + // drop the original cached version and make the new version cached lazily. + sparkSession.catalog.refreshTable(tableIdent.quotedString) Seq.empty[Row] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 473e827f4db18..1371abe189f84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -345,6 +345,33 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { sparkSession.cacheManager.lookupCachedData(qName).nonEmpty } + /** + * Refresh the cache entry for a table, if any. For Hive metastore table, the metadata + * is refreshed. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def refreshTable(tableName: String): Unit = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + sessionCatalog.refreshTable(tableIdent) + + // If this table is cached as a InMemoryRelation, drop the original + // cached version and make the new version cached lazily. + val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent) + // Use lookupCachedData directly since RefreshTable also takes databaseName. + val isCached = sparkSession.cacheManager.lookupCachedData(logicalPlan).nonEmpty + if (isCached) { + // Create a data frame to represent the table. + // TODO: Use uncacheTable once it supports database name. + val df = Dataset.ofRows(sparkSession, logicalPlan) + // Uncache the logicalPlan. + sparkSession.cacheManager.tryUncacheQuery(df, blocking = true) + // Cache it again. + sparkSession.cacheManager.cacheQuery(df, Some(tableIdent.table)) + } + } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index f0b8a83dee8ca..8f7c6f5d0ca44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -162,10 +162,6 @@ private[sql] class SessionState(sparkSession: SparkSession) { def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan) - def refreshTable(tableName: String): Unit = { - catalog.refreshTable(sqlParser.parseTableIdentifier(tableName)) - } - def invalidateTable(tableName: String): Unit = { catalog.invalidateTable(sqlParser.parseTableIdentifier(tableName)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 00adb9a44b14e..686c63065dfcc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -622,7 +622,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("arrayInParquet") - sessionState.refreshTable("arrayInParquet") + sparkSession.catalog.refreshTable("arrayInParquet") checkAnswer( sql("SELECT a FROM arrayInParquet"), @@ -681,7 +681,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("mapInParquet") - sessionState.refreshTable("mapInParquet") + sparkSession.catalog.refreshTable("mapInParquet") checkAnswer( sql("SELECT a FROM mapInParquet"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 622b043581c5d..5b706b0432418 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -217,7 +217,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql("ALTER TABLE t ADD PARTITION (p=2)") - hiveContext.sessionState.refreshTable("t") + spark.catalog.refreshTable("t") checkAnswer( spark.table("t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) @@ -249,7 +249,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql(s"ALTER TABLE $db.t ADD PARTITION (p=2)") - hiveContext.sessionState.refreshTable(s"$db.t") + spark.catalog.refreshTable(s"$db.t") checkAnswer( spark.table(s"$db.t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) diff --git a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 75166f6beaa87..415d4c0049d40 100644 --- a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -58,4 +58,16 @@ class HiveContext private[hive]( sparkSession.sharedState.asInstanceOf[HiveSharedState] } + /** + * 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. + * + * @since 1.3.0 + */ + def refreshTable(tableName: String): Unit = { + sparkSession.catalog.refreshTable(tableName) + } + } From 4e25d6e8ce9ce88a58fc0ea0e00cc7b68370a62d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 23:43:01 -0700 Subject: [PATCH 0312/1470] [SPARK-15421][SQL] Validate DDL property values ## What changes were proposed in this pull request? When we parse DDLs involving table or database properties, we need to validate the values. E.g. if we alter a database's property without providing a value: ``` ALTER DATABASE my_db SET DBPROPERTIES('some_key') ``` Then we'll ignore it with Hive, but override the property with the in-memory catalog. Inconsistencies like these arise because we don't validate the property values. In such cases, we should throw exceptions instead. ## How was this patch tested? `DDLCommandSuite` Author: Andrew Or Closes #13205 from andrewor14/ddl-prop-values. (cherry picked from commit 257375019266ab9e3c320e33026318cc31f58ada) Signed-off-by: Andrew Or --- .../spark/sql/execution/SparkSqlParser.scala | 45 +++++++++++++++---- .../execution/command/DDLCommandSuite.scala | 41 +++++++++++++++++ 2 files changed, 77 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8af6d07719c9f..ee12bfa7251fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -293,7 +293,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (external) { throw operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } - val options = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) + val options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val provider = ctx.tableProvider.qualifiedName.getText val partitionColumnNames = Option(ctx.partitionColumnNames) @@ -371,6 +371,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { /** * Convert a table property list into a key-value map. + * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. */ override def visitTablePropertyList( ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { @@ -381,6 +382,32 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { }.toMap } + /** + * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified. + */ + private def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v == null }.keys + if (badKeys.nonEmpty) { + throw operationNotAllowed( + s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props + } + + /** + * Parse a list of keys from a [[TablePropertyListContext]], assuming no values are specified. + */ + private def visitPropertyKeys(ctx: TablePropertyListContext): Seq[String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v != null }.keys + if (badKeys.nonEmpty) { + throw operationNotAllowed( + s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props.keys.toSeq + } + /** * A table property key can either be String or a collection of dot separated elements. This * function extracts the property key based on whether its a string literal or a table property @@ -409,7 +436,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.EXISTS != null, Option(ctx.locationSpec).map(visitLocationSpec), Option(ctx.comment).map(string), - Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty)) + Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } /** @@ -424,7 +451,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterDatabaseProperties( ctx.identifier.getText, - visitTablePropertyList(ctx.tablePropertyList)) + visitPropertyKeyValues(ctx.tablePropertyList)) } /** @@ -540,7 +567,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterTableSetProperties( visitTableIdentifier(ctx.tableIdentifier), - visitTablePropertyList(ctx.tablePropertyList), + visitPropertyKeyValues(ctx.tablePropertyList), ctx.VIEW != null) } @@ -557,7 +584,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterTableUnsetProperties( visitTableIdentifier(ctx.tableIdentifier), - visitTablePropertyList(ctx.tablePropertyList).keys.toSeq, + visitPropertyKeys(ctx.tablePropertyList), ctx.EXISTS != null, ctx.VIEW != null) } @@ -575,7 +602,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { AlterTableSerDeProperties( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.STRING).map(string), - Option(ctx.tablePropertyList).map(visitTablePropertyList), + Option(ctx.tablePropertyList).map(visitPropertyKeyValues), // TODO a partition spec is allowed to have optional values. This is currently violated. Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } @@ -783,7 +810,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val comment = Option(ctx.STRING).map(string) val partitionCols = Option(ctx.partitionColumns).toSeq.flatMap(visitCatalogColumns) val cols = Option(ctx.columns).toSeq.flatMap(visitCatalogColumns) - val properties = Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty) + val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val selectQuery = Option(ctx.query).map(plan) // Note: Hive requires partition columns to be distinct from the schema, so we need @@ -944,7 +971,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { import ctx._ EmptyStorageFormat.copy( serde = Option(string(name)), - serdeProperties = Option(tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty)) + serdeProperties = Option(tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } /** @@ -1001,7 +1028,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { comment = Option(ctx.STRING).map(string), schema, ctx.query, - Option(ctx.tablePropertyList).map(visitTablePropertyList).getOrElse(Map.empty), + Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty), ctx.EXISTS != null, ctx.REPLACE != null, ctx.TEMPORARY != null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 897170ea57cf1..0925a51310e69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -57,6 +57,12 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed, expected) } + test("create database - property values must be set") { + assertUnsupported( + sql = "CREATE DATABASE my_db WITH DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("drop database") { val sql1 = "DROP DATABASE IF EXISTS database_name RESTRICT" val sql2 = "DROP DATABASE IF EXISTS database_name CASCADE" @@ -121,6 +127,12 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed2, expected2) } + test("alter database - property values must be set") { + assertUnsupported( + sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("describe database") { // DESCRIBE DATABASE [EXTENDED] db_name; val sql1 = "DESCRIBE DATABASE EXTENDED db_name" @@ -228,6 +240,16 @@ class DDLCommandSuite extends PlanTest { } } + test("create table - property values must be set") { + assertUnsupported( + sql = "CREATE TABLE my_tab TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + assertUnsupported( + sql = "CREATE TABLE my_tab ROW FORMAT SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + test("create table - location implies external") { val query = "CREATE TABLE my_tab LOCATION '/something/anything'" parser.parsePlan(query) match { @@ -349,6 +371,18 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed3_view, expected3_view) } + test("alter table - property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("alter table unset properties - property values must NOT be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_with_value")) + } + test("alter table: SerDe properties") { val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" val sql2 = @@ -404,6 +438,13 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed5, expected5) } + test("alter table - SerDe property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + // ALTER TABLE table_name ADD [IF NOT EXISTS] PARTITION partition_spec // [LOCATION 'location1'] partition_spec [LOCATION 'location2'] ...; test("alter table: add partition") { From 53c09f065fac9cabe479cd1f205810230eda110d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 May 2016 23:44:10 -0700 Subject: [PATCH 0313/1470] [SPARK-15417][SQL][PYTHON] PySpark shell always uses in-memory catalog ## What changes were proposed in this pull request? There is no way to use the Hive catalog in `pyspark-shell`. This is because we used to create a `SparkContext` before calling `SparkSession.enableHiveSupport().getOrCreate()`, which just gets the existing `SparkContext` instead of creating a new one. As a result, `spark.sql.catalogImplementation` was never propagated. ## How was this patch tested? Manual. Author: Andrew Or Closes #13203 from andrewor14/fix-pyspark-shell. (cherry picked from commit c32b1b162e7e5ecc5c823f79ba9f23cbd1407dbf) Signed-off-by: Andrew Or --- python/pyspark/shell.py | 8 +++++--- python/pyspark/sql/session.py | 6 ++++++ 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index adaa3b5a79938..ef46d3065e5e8 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,12 +35,11 @@ if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) -sc = SparkContext() -atexit.register(lambda: sc.stop()) +SparkContext._ensure_initialized() try: # Try to access HiveConf, it will raise exception if Hive is not added - sc._jvm.org.apache.hadoop.hive.conf.HiveConf() + SparkContext._jvm.org.apache.hadoop.hive.conf.HiveConf() spark = SparkSession.builder\ .enableHiveSupport()\ .getOrCreate() @@ -49,6 +48,9 @@ except TypeError: spark = SparkSession(sc) +sc = spark.sparkContext +atexit.register(lambda: sc.stop()) + # for compatibility sqlContext = spark._wrapped sqlCtx = sqlContext diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 0e04b88265fa1..241947537fa29 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -202,6 +202,12 @@ def newSession(self): """ return self.__class__(self._sc, self._jsparkSession.newSession()) + @property + @since(2.0) + def sparkContext(self): + """Returns the underlying :class:`SparkContext`.""" + return self._sc + @property @since(2.0) def conf(self): From 1346f3cd6cf78c940f646bb2b808ae3b22f936b3 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 20 May 2016 13:40:13 +0200 Subject: [PATCH 0314/1470] [SPARK-15444][PYSPARK][ML][HOTFIX] Default value mismatch of param linkPredictionCol for GeneralizedLinearRegression ## What changes were proposed in this pull request? Default value mismatch of param linkPredictionCol for GeneralizedLinearRegression between PySpark and Scala. That is because default value conflict between #13106 and #13129. This causes ml.tests failed. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13220 from viirya/hotfix-regresstion. (cherry picked from commit 4e739331187f2acdd84a5e65857edb62e58a0f8f) Signed-off-by: Nick Pentreath --- python/pyspark/ml/regression.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 25640b1f85250..e21dd83923adb 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1303,17 +1303,16 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha @keyword_only def __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, - regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=""): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None): """ __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ - regParam=0.0, weightCol=None, solver="irls", linkPredictionCol="") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None) """ super(GeneralizedLinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.GeneralizedLinearRegression", self.uid) - self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls", - linkPredictionCol="") + self._setDefault(family="gaussian", maxIter=25, tol=1e-6, regParam=0.0, solver="irls") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -1321,11 +1320,11 @@ def __init__(self, labelCol="label", featuresCol="features", predictionCol="pred @since("2.0.0") def setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, - regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=""): + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None): """ setParams(self, labelCol="label", featuresCol="features", predictionCol="prediction", \ family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, \ - regParam=0.0, weightCol=None, solver="irls", linkPredictionCol="") + regParam=0.0, weightCol=None, solver="irls", linkPredictionCol=None) Sets params for generalized linear regression. """ kwargs = self.setParams._input_kwargs From 93f9f928e45988371d7d86f080b4e9971c03fbc9 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Fri, 20 May 2016 08:17:19 -0500 Subject: [PATCH 0315/1470] [SPARK-15203][DEPLOY] The spark daemon shell script error, daemon process start successfully but script output fail message ## What changes were proposed in this pull request? fix the bug: The spark daemon shell script error, daemon process start successfully but script output fail message ## How was this patch tested? existing test. Author: WeichenXu Closes #13172 from WeichenXu123/fix-spark-15203. (cherry picked from commit a3ceb875c64421ced8e52db6d8e51aec9b758e3e) Signed-off-by: Sean Owen --- sbin/spark-daemon.sh | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 6ab57df409529..59823571124f1 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -162,6 +162,16 @@ run_command() { esac echo "$newpid" > "$pid" + + #Poll for up to 5 seconds for the java process to start + for i in {1..10} + do + if [[ $(ps -p "$newpid" -o comm=) =~ "java" ]]; then + break + fi + sleep 0.5 + done + sleep 2 # Check if the process has died; in that case we'll tail the log so the user can see if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then From 0066d35cc909361460fa99f3791453741dfd707e Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 20 May 2016 09:30:20 -0700 Subject: [PATCH 0316/1470] [SPARK-15222][SPARKR][ML] SparkR ML examples update in 2.0 ## What changes were proposed in this pull request? Update example code in examples/src/main/r/ml.R to reflect the new algorithms. * spark.glm and glm * spark.survreg * spark.naiveBayes * spark.kmeans ## How was this patch tested? Offline test. Author: Yanbo Liang Closes #13000 from yanboliang/spark-15222. (cherry picked from commit 9a9c6f5c22248c5a891e9d3b788ff12b6b4718b2) Signed-off-by: Xiangrui Meng --- examples/src/main/r/ml.R | 129 +++++++++++++++++++++++++++++++++------ 1 file changed, 112 insertions(+), 17 deletions(-) diff --git a/examples/src/main/r/ml.R b/examples/src/main/r/ml.R index a0c903939cbbb..fd35936635334 100644 --- a/examples/src/main/r/ml.R +++ b/examples/src/main/r/ml.R @@ -16,7 +16,7 @@ # # To run this example use -# ./bin/sparkR examples/src/main/r/ml.R +# ./bin/spark-submit examples/src/main/r/ml.R # Load SparkR library into your R session library(SparkR) @@ -25,30 +25,125 @@ library(SparkR) sc <- sparkR.init(appName="SparkR-ML-example") sqlContext <- sparkRSQL.init(sc) -# Train GLM of family 'gaussian' -training1 <- suppressWarnings(createDataFrame(sqlContext, iris)) -test1 <- training1 -model1 <- glm(Sepal_Length ~ Sepal_Width + Species, training1, family = "gaussian") +############################ spark.glm and glm ############################################## + +irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +# Fit a generalized linear model of family "gaussian" with spark.glm +gaussianDF <- irisDF +gaussianTestDF <- irisDF +gaussianGLM <- spark.glm(gaussianDF, Sepal_Length ~ Sepal_Width + Species, family = "gaussian") # Model summary -summary(model1) +summary(gaussianGLM) # Prediction -predictions1 <- predict(model1, test1) -head(select(predictions1, "Sepal_Length", "prediction")) +gaussianPredictions <- predict(gaussianGLM, gaussianTestDF) +showDF(gaussianPredictions) + +# Fit a generalized linear model with glm (R-compliant) +gaussianGLM2 <- glm(Sepal_Length ~ Sepal_Width + Species, gaussianDF, family = "gaussian") +summary(gaussianGLM2) + +# Fit a generalized linear model of family "binomial" with spark.glm +binomialDF <- filter(irisDF, irisDF$Species != "setosa") +binomialTestDF <- binomialDF +binomialGLM <- spark.glm(binomialDF, Species ~ Sepal_Length + Sepal_Width, family = "binomial") + +# Model summary +summary(binomialGLM) + +# Prediction +binomialPredictions <- predict(binomialGLM, binomialTestDF) +showDF(binomialPredictions) + +############################ spark.survreg ############################################## + +# Use the ovarian dataset available in R survival package +library(survival) -# Train GLM of family 'binomial' -training2 <- filter(training1, training1$Species != "setosa") -test2 <- training2 -model2 <- glm(Species ~ Sepal_Length + Sepal_Width, data = training2, family = "binomial") +# Fit an accelerated failure time (AFT) survival regression model with spark.survreg +ovarianDF <- suppressWarnings(createDataFrame(sqlContext, ovarian)) +aftDF <- ovarianDF +aftTestDF <- ovarianDF +aftModel <- spark.survreg(aftDF, Surv(futime, fustat) ~ ecog_ps + rx) # Model summary -summary(model2) +summary(aftModel) + +# Prediction +aftPredictions <- predict(aftModel, aftTestDF) +showDF(aftPredictions) + +############################ spark.naiveBayes ############################################## + +# Fit a Bernoulli naive Bayes model with spark.naiveBayes +titanic <- as.data.frame(Titanic) +titanicDF <- suppressWarnings(createDataFrame(sqlContext, titanic[titanic$Freq > 0, -5])) +nbDF <- titanicDF +nbTestDF <- titanicDF +nbModel <- spark.naiveBayes(nbDF, Survived ~ Class + Sex + Age) + +# Model summary +summary(nbModel) + +# Prediction +nbPredictions <- predict(nbModel, nbTestDF) +showDF(nbPredictions) + +############################ spark.kmeans ############################################## + +# Fit a k-means model with spark.kmeans +irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +kmeansDF <- irisDF +kmeansTestDF <- irisDF +kmeansModel <- spark.kmeans(kmeansDF, ~ Sepal_Length + Sepal_Width + Petal_Length + Petal_Width, + k = 3) + +# Model summary +summary(kmeansModel) + +# Get fitted result from the k-means model +showDF(fitted(kmeansModel)) + +# Prediction +kmeansPredictions <- predict(kmeansModel, kmeansTestDF) +showDF(kmeansPredictions) + +############################ model read/write ############################################## + +irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +# Fit a generalized linear model of family "gaussian" with spark.glm +gaussianDF <- irisDF +gaussianTestDF <- irisDF +gaussianGLM <- spark.glm(gaussianDF, Sepal_Length ~ Sepal_Width + Species, family = "gaussian") + +# Save and then load a fitted MLlib model +modelPath <- tempfile(pattern = "ml", fileext = ".tmp") +write.ml(gaussianGLM, modelPath) +gaussianGLM2 <- read.ml(modelPath) + +# Check model summary +summary(gaussianGLM2) + +# Check model prediction +gaussianPredictions <- predict(gaussianGLM2, gaussianTestDF) +showDF(gaussianPredictions) + +unlink(modelPath) + +############################ fit models with spark.lapply ##################################### + +# Perform distributed training of multiple models with spark.lapply +families <- c("gaussian", "poisson") +train <- function(family) { + model <- glm(Sepal.Length ~ Sepal.Width + Species, iris, family = family) + summary(model) +} +model.summaries <- spark.lapply(sc, families, train) + +# Print the summary of each model +print(model.summaries) -# Prediction (Currently the output of prediction for binomial GLM is the indexed label, -# we need to transform back to the original string label later) -predictions2 <- predict(model2, test2) -head(select(predictions2, "Species", "prediction")) # Stop the SparkContext now sparkR.stop() From 78c8825bd4b5b86596ccf260c15bf97a9689b6ac Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 20 May 2016 09:34:55 -0700 Subject: [PATCH 0317/1470] [SPARK-15308][SQL] RowEncoder should preserve nested column name. ## What changes were proposed in this pull request? The following code generates wrong schema: ``` val schema = new StructType().add( "struct", new StructType() .add("i", IntegerType, nullable = false) .add( "s", new StructType().add("int", IntegerType, nullable = false), nullable = false), nullable = false) val ds = sqlContext.range(10).map(l => Row(l, Row(l)))(RowEncoder(schema)) ds.printSchema() ``` This should print as follows: ``` root |-- struct: struct (nullable = false) | |-- i: integer (nullable = false) | |-- s: struct (nullable = false) | | |-- int: integer (nullable = false) ``` but the result is: ``` root |-- struct: struct (nullable = false) | |-- col1: integer (nullable = false) | |-- col2: struct (nullable = false) | | |-- col1: integer (nullable = false) ``` This PR fixes `RowEncoder` to preserve nested column name. ## How was this patch tested? Existing tests and I added a test to check if `RowEncoder` preserves nested column name. Author: Takuya UESHIN Closes #13090 from ueshin/issues/SPARK-15308. (cherry picked from commit d2e1aa97ef5bf7cfffc777a178f44ab8fa775266) Signed-off-by: Reynold Xin --- .../sql/catalyst/encoders/RowEncoder.scala | 22 ++++++++++--------- .../catalyst/encoders/RowEncoderSuite.scala | 22 +++++++++++++++++++ 2 files changed, 34 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 71b39c54fa0c2..2f8ba33f3520f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -62,7 +62,7 @@ object RowEncoder { new ExpressionEncoder[Row]( schema, flat = false, - serializer.asInstanceOf[CreateStruct].children, + serializer.asInstanceOf[CreateNamedStruct].flatten, deserializer, ClassTag(cls)) } @@ -148,28 +148,30 @@ object RowEncoder { dataType = t) case StructType(fields) => - val convertedFields = fields.zipWithIndex.map { case (f, i) => + val nonNullOutput = CreateNamedStruct(fields.zipWithIndex.flatMap { case (field, index) => val fieldValue = serializerFor( - GetExternalRowField(inputObject, i, f.name, externalDataTypeForInput(f.dataType)), - f.dataType + GetExternalRowField( + inputObject, index, field.name, externalDataTypeForInput(field.dataType)), + field.dataType ) - if (f.nullable) { + val convertedField = if (field.nullable) { If( - Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil), - Literal.create(null, f.dataType), + Invoke(inputObject, "isNullAt", BooleanType, Literal(index) :: Nil), + Literal.create(null, field.dataType), fieldValue ) } else { fieldValue } - } + Literal(field.name) :: convertedField :: Nil + }) if (inputObject.nullable) { If(IsNull(inputObject), Literal.create(null, inputType), - CreateStruct(convertedFields)) + nonNullOutput) } else { - CreateStruct(convertedFields) + nonNullOutput } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 7bb006c173b07..39fcc7225b37e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -185,6 +185,28 @@ class RowEncoderSuite extends SparkFunSuite { assert(encoder.serializer.head.nullable == false) } + test("RowEncoder should preserve nested column name") { + val schema = new StructType().add( + "struct", + new StructType() + .add("i", IntegerType, nullable = false) + .add( + "s", + new StructType().add("int", IntegerType, nullable = false), + nullable = false), + nullable = false) + val encoder = RowEncoder(schema) + assert(encoder.serializer.length == 1) + assert(encoder.serializer.head.dataType == + new StructType() + .add("i", IntegerType, nullable = false) + .add( + "s", + new StructType().add("int", IntegerType, nullable = false), + nullable = false)) + assert(encoder.serializer.head.nullable == false) + } + test("RowEncoder should support array as the external type for ArrayType") { val schema = new StructType() .add("array", ArrayType(IntegerType)) From a879e7c32e41326387e0754095a5f14d781e1cf1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 May 2016 09:36:14 -0700 Subject: [PATCH 0318/1470] [SPARK-15435][SQL] Append Command to all commands ## What changes were proposed in this pull request? We started this convention to append Command suffix to all SQL commands. However, not all commands follow that convention. This patch adds Command suffix to all RunnableCommands. ## How was this patch tested? Updated test cases to reflect the renames. Author: Reynold Xin Closes #13215 from rxin/SPARK-15435. (cherry picked from commit e8adc552df80af413e1d31b020489612d13a8770) Signed-off-by: Reynold Xin --- .../spark/sql/execution/SparkSqlParser.scala | 91 ++++++++++--------- .../spark/sql/execution/SparkStrategies.scala | 6 +- ...eTable.scala => AnalyzeTableCommand.scala} | 6 +- .../command/createDataSourceTables.scala | 3 +- .../spark/sql/execution/command/ddl.scala | 26 +++--- .../sql/execution/command/functions.scala | 12 +-- .../sql/execution/command/resources.scala | 4 +- .../spark/sql/execution/command/tables.scala | 37 ++++---- .../execution/datasources/DataSource.scala | 2 +- .../datasources/DataSourceStrategy.scala | 4 +- ...cala => InsertIntoDataSourceCommand.scala} | 2 +- ...> InsertIntoHadoopFsRelationCommand.scala} | 12 +-- .../spark/sql/execution/datasources/ddl.scala | 2 +- .../spark/sql/internal/SessionState.scala | 4 +- .../execution/command/DDLCommandSuite.scala | 90 +++++++++--------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- ...scala => CreateTableAsSelectCommand.scala} | 2 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 12 +-- .../spark/sql/hive/StatisticsSuite.scala | 16 ++-- .../apache/spark/sql/hive/parquetSuites.scala | 10 +- 20 files changed, 173 insertions(+), 170 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/command/{AnalyzeTable.scala => AnalyzeTableCommand.scala} (95%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{InsertIntoDataSource.scala => InsertIntoDataSourceCommand.scala} (97%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{InsertIntoHadoopFsRelation.scala => InsertIntoHadoopFsRelationCommand.scala} (93%) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/{CreateTableAsSelect.scala => CreateTableAsSelectCommand.scala} (99%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ee12bfa7251fd..2966eefd07c77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -76,8 +76,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AnalyzeTable]] command. This currently only implements the NOSCAN option (other - * options are passed on to Hive) e.g.: + * Create an [[AnalyzeTableCommand]] command. This currently only implements the NOSCAN + * option (other options are passed on to Hive) e.g.: * {{{ * ANALYZE TABLE table COMPUTE STATISTICS NOSCAN; * }}} @@ -86,11 +86,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (ctx.partitionSpec == null && ctx.identifier != null && ctx.identifier.getText.toLowerCase == "noscan") { - AnalyzeTable(visitTableIdentifier(ctx.tableIdentifier).toString) + AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString) } else { // Always just run the no scan analyze. We should fix this and implement full analyze // command in the future. - AnalyzeTable(visitTableIdentifier(ctx.tableIdentifier).toString) + AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString) } } @@ -332,7 +332,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[LoadData]] command. + * Create a [[LoadDataCommand]] command. * * For example: * {{{ @@ -341,7 +341,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { - LoadData( + LoadDataCommand( table = visitTableIdentifier(ctx.tableIdentifier), path = string(ctx.path), isLocal = ctx.LOCAL != null, @@ -351,7 +351,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[TruncateTable]] command. + * Create a [[TruncateTableCommand]] command. * * For example: * {{{ @@ -363,7 +363,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (ctx.identifierList != null) { throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) } - TruncateTable( + TruncateTableCommand( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) ) @@ -422,7 +422,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[CreateDatabase]] command. + * Create a [[CreateDatabaseCommand]] command. * * For example: * {{{ @@ -431,7 +431,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitCreateDatabase(ctx: CreateDatabaseContext): LogicalPlan = withOrigin(ctx) { - CreateDatabase( + CreateDatabaseCommand( ctx.identifier.getText, ctx.EXISTS != null, Option(ctx.locationSpec).map(visitLocationSpec), @@ -440,7 +440,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AlterDatabaseProperties]] command. + * Create an [[AlterDatabasePropertiesCommand]] command. * * For example: * {{{ @@ -449,13 +449,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitSetDatabaseProperties( ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterDatabaseProperties( + AlterDatabasePropertiesCommand( ctx.identifier.getText, visitPropertyKeyValues(ctx.tablePropertyList)) } /** - * Create a [[DropDatabase]] command. + * Create a [[DropDatabaseCommand]] command. * * For example: * {{{ @@ -463,11 +463,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitDropDatabase(ctx: DropDatabaseContext): LogicalPlan = withOrigin(ctx) { - DropDatabase(ctx.identifier.getText, ctx.EXISTS != null, ctx.CASCADE != null) + DropDatabaseCommand(ctx.identifier.getText, ctx.EXISTS != null, ctx.CASCADE != null) } /** - * Create a [[DescribeDatabase]] command. + * Create a [[DescribeDatabaseCommand]] command. * * For example: * {{{ @@ -475,11 +475,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitDescribeDatabase(ctx: DescribeDatabaseContext): LogicalPlan = withOrigin(ctx) { - DescribeDatabase(ctx.identifier.getText, ctx.EXTENDED != null) + DescribeDatabaseCommand(ctx.identifier.getText, ctx.EXTENDED != null) } /** - * Create a [[CreateFunction]] command. + * Create a [[CreateFunctionCommand]] command. * * For example: * {{{ @@ -500,7 +500,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Extract database, name & alias. val functionIdentifier = visitFunctionName(ctx.qualifiedName) - CreateFunction( + CreateFunctionCommand( functionIdentifier.database, functionIdentifier.funcName, string(ctx.className), @@ -509,7 +509,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[DropFunction]] command. + * Create a [[DropFunctionCommand]] command. * * For example: * {{{ @@ -518,7 +518,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitDropFunction(ctx: DropFunctionContext): LogicalPlan = withOrigin(ctx) { val functionIdentifier = visitFunctionName(ctx.qualifiedName) - DropFunction( + DropFunctionCommand( functionIdentifier.database, functionIdentifier.funcName, ctx.EXISTS != null, @@ -526,20 +526,20 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create a [[DropTable]] command. + * Create a [[DropTableCommand]] command. */ override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { if (ctx.PURGE != null) { throw operationNotAllowed("DROP TABLE ... PURGE", ctx) } - DropTable( + DropTableCommand( visitTableIdentifier(ctx.tableIdentifier), ctx.EXISTS != null, ctx.VIEW != null) } /** - * Create a [[AlterTableRename]] command. + * Create a [[AlterTableRenameCommand]] command. * * For example: * {{{ @@ -548,14 +548,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { - AlterTableRename( + AlterTableRenameCommand( visitTableIdentifier(ctx.from), visitTableIdentifier(ctx.to), ctx.VIEW != null) } /** - * Create an [[AlterTableSetProperties]] command. + * Create an [[AlterTableSetPropertiesCommand]] command. * * For example: * {{{ @@ -565,14 +565,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitSetTableProperties( ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetProperties( + AlterTableSetPropertiesCommand( visitTableIdentifier(ctx.tableIdentifier), visitPropertyKeyValues(ctx.tablePropertyList), ctx.VIEW != null) } /** - * Create an [[AlterTableUnsetProperties]] command. + * Create an [[AlterTableUnsetPropertiesCommand]] command. * * For example: * {{{ @@ -582,7 +582,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitUnsetTableProperties( ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterTableUnsetProperties( + AlterTableUnsetPropertiesCommand( visitTableIdentifier(ctx.tableIdentifier), visitPropertyKeys(ctx.tablePropertyList), ctx.EXISTS != null, @@ -590,7 +590,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AlterTableSerDeProperties]] command. + * Create an [[AlterTableSerDePropertiesCommand]] command. * * For example: * {{{ @@ -599,7 +599,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitSetTableSerDe(ctx: SetTableSerDeContext): LogicalPlan = withOrigin(ctx) { - AlterTableSerDeProperties( + AlterTableSerDePropertiesCommand( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.STRING).map(string), Option(ctx.tablePropertyList).map(visitPropertyKeyValues), @@ -608,7 +608,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AlterTableAddPartition]] command. + * Create an [[AlterTableAddPartitionCommand]] command. * * For example: * {{{ @@ -636,14 +636,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Alter View: the location clauses are not allowed. ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec(_) -> None) } - AlterTableAddPartition( + AlterTableAddPartitionCommand( visitTableIdentifier(ctx.tableIdentifier), specsAndLocs, ctx.EXISTS != null) } /** - * Create an [[AlterTableRenamePartition]] command + * Create an [[AlterTableRenamePartitionCommand]] command * * For example: * {{{ @@ -652,14 +652,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitRenameTablePartition( ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenamePartition( + AlterTableRenamePartitionCommand( visitTableIdentifier(ctx.tableIdentifier), visitNonOptionalPartitionSpec(ctx.from), visitNonOptionalPartitionSpec(ctx.to)) } /** - * Create an [[AlterTableDropPartition]] command + * Create an [[AlterTableDropPartitionCommand]] command * * For example: * {{{ @@ -678,14 +678,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { if (ctx.PURGE != null) { throw operationNotAllowed("ALTER TABLE ... DROP PARTITION ... PURGE", ctx) } - AlterTableDropPartition( + AlterTableDropPartitionCommand( visitTableIdentifier(ctx.tableIdentifier), ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec), ctx.EXISTS != null) } /** - * Create an [[AlterTableSetLocation]] command + * Create an [[AlterTableSetLocationCommand]] command * * For example: * {{{ @@ -693,7 +693,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetLocation( + AlterTableSetLocationCommand( visitTableIdentifier(ctx.tableIdentifier), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), visitLocationSpec(ctx.locationSpec)) @@ -759,18 +759,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AddJar]] or [[AddFile]] command depending on the requested resource. + * Create an [[AddJarCommand]] or [[AddFileCommand]] command depending on the requested resource. */ override def visitAddResource(ctx: AddResourceContext): LogicalPlan = withOrigin(ctx) { ctx.identifier.getText.toLowerCase match { - case "file" => AddFile(remainder(ctx.identifier).trim) - case "jar" => AddJar(remainder(ctx.identifier).trim) + case "file" => AddFileCommand(remainder(ctx.identifier).trim) + case "jar" => AddJarCommand(remainder(ctx.identifier).trim) case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) } } /** - * Create a table, returning either a [[CreateTable]] or a [[CreateTableAsSelectLogicalPlan]]. + * Create a table, returning either a [[CreateTableCommand]] or a + * [[CreateTableAsSelectLogicalPlan]]. * * This is not used to create datasource tables, which is handled through * "CREATE TABLE ... USING ...". @@ -868,12 +869,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { selectQuery match { case Some(q) => CreateTableAsSelectLogicalPlan(tableDesc, q, ifNotExists) - case None => CreateTable(tableDesc, ifNotExists) + case None => CreateTableCommand(tableDesc, ifNotExists) } } /** - * Create a [[CreateTableLike]] command. + * Create a [[CreateTableLikeCommand]] command. * * For example: * {{{ @@ -884,7 +885,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitCreateTableLike(ctx: CreateTableLikeContext): LogicalPlan = withOrigin(ctx) { val targetTable = visitTableIdentifier(ctx.target) val sourceTable = visitTableIdentifier(ctx.source) - CreateTableLike(targetTable, sourceTable, ctx.EXISTS != null) + CreateTableLikeCommand(targetTable, sourceTable, ctx.EXISTS != null) } /** 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 5cfb6d5363a3b..3343039ae1cf3 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 @@ -398,7 +398,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { sys.error("Cannot create temporary partitioned table.") case c: CreateTableUsingAsSelect if c.temporary => - val cmd = CreateTempTableUsingAsSelect( + val cmd = CreateTempTableUsingAsSelectCommand( c.tableIdent, c.provider, Array.empty[String], c.mode, c.options, c.child) ExecutedCommandExec(cmd) :: Nil @@ -415,10 +415,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ExecutedCommandExec(cmd) :: Nil case logical.ShowFunctions(db, pattern) => - ExecutedCommandExec(ShowFunctions(db, pattern)) :: Nil + ExecutedCommandExec(ShowFunctionsCommand(db, pattern)) :: Nil case logical.DescribeFunction(function, extended) => - ExecutedCommandExec(DescribeFunction(function, extended)) :: Nil + ExecutedCommandExec(DescribeFunctionCommand(function, extended)) :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index de2db44b0e1b2..a469d4da8613b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} * Right now, it only supports Hive tables and it only updates the size of a Hive table * in the Hive metastore. */ -case class AnalyzeTable(tableName: String) extends RunnableCommand { +case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val sessionState = sparkSession.sessionState @@ -95,7 +95,7 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { sessionState.catalog.alterTable( catalogTable.copy( properties = relation.catalogTable.properties + - (AnalyzeTable.TOTAL_SIZE_FIELD -> newTotalSize.toString))) + (AnalyzeTableCommand.TOTAL_SIZE_FIELD -> newTotalSize.toString))) } case otherRelation => @@ -106,6 +106,6 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { } } -object AnalyzeTable { +object AnalyzeTableCommand { val TOTAL_SIZE_FIELD = "totalSize" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 70e5108d938b9..6ca66a22df12d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.types._ /** * A command used to create a data source table. * - * Note: This is different from [[CreateTable]]. Please check the syntax for difference. + * Note: This is different from [[CreateTableCommand]]. Please check the syntax for difference. * This is not intended for temporary tables. * * The syntax of using this command in SQL is: @@ -253,6 +253,7 @@ case class CreateDataSourceTableAsSelectCommand( } } + object CreateDataSourceTableUtils extends Logging { /** * Checks if the given name conforms the Hive standard ("[a-zA-z_0-9]+"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 49d7fe956f812..dd3f17d525abd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.types._ * [WITH DBPROPERTIES (property_name=property_value, ...)]; * }}} */ -case class CreateDatabase( +case class CreateDatabaseCommand( databaseName: String, ifNotExists: Boolean, path: Option[String], @@ -85,7 +85,7 @@ case class CreateDatabase( * DROP DATABASE [IF EXISTS] database_name [RESTRICT|CASCADE]; * }}} */ -case class DropDatabase( +case class DropDatabaseCommand( databaseName: String, ifExists: Boolean, cascade: Boolean) @@ -108,7 +108,7 @@ case class DropDatabase( * ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...) * }}} */ -case class AlterDatabaseProperties( +case class AlterDatabasePropertiesCommand( databaseName: String, props: Map[String, String]) extends RunnableCommand { @@ -134,7 +134,7 @@ case class AlterDatabaseProperties( * DESCRIBE DATABASE [EXTENDED] db_name * }}} */ -case class DescribeDatabase( +case class DescribeDatabaseCommand( databaseName: String, extended: Boolean) extends RunnableCommand { @@ -175,7 +175,7 @@ case class DescribeDatabase( * DROP VIEW [IF EXISTS] [db_name.]view_name; * }}} */ -case class DropTable( +case class DropTableCommand( tableName: TableIdentifier, ifExists: Boolean, isView: Boolean) extends RunnableCommand { @@ -220,7 +220,7 @@ case class DropTable( * ALTER VIEW view1 SET TBLPROPERTIES ('key1' = 'val1', 'key2' = 'val2', ...); * }}} */ -case class AlterTableSetProperties( +case class AlterTableSetPropertiesCommand( tableName: TableIdentifier, properties: Map[String, String], isView: Boolean) @@ -251,7 +251,7 @@ case class AlterTableSetProperties( * ALTER VIEW view1 UNSET TBLPROPERTIES [IF EXISTS] ('key1', 'key2', ...); * }}} */ -case class AlterTableUnsetProperties( +case class AlterTableUnsetPropertiesCommand( tableName: TableIdentifier, propKeys: Seq[String], ifExists: Boolean, @@ -291,7 +291,7 @@ case class AlterTableUnsetProperties( * ALTER TABLE table [PARTITION spec] SET SERDEPROPERTIES serde_properties; * }}} */ -case class AlterTableSerDeProperties( +case class AlterTableSerDePropertiesCommand( tableName: TableIdentifier, serdeClassName: Option[String], serdeProperties: Option[Map[String, String]], @@ -330,7 +330,7 @@ case class AlterTableSerDeProperties( * ALTER TABLE table ADD [IF NOT EXISTS] PARTITION spec [LOCATION 'loc1'] * }}} */ -case class AlterTableAddPartition( +case class AlterTableAddPartitionCommand( tableName: TableIdentifier, partitionSpecsAndLocs: Seq[(TablePartitionSpec, Option[String])], ifNotExists: Boolean) @@ -361,7 +361,7 @@ case class AlterTableAddPartition( * ALTER TABLE table PARTITION spec1 RENAME TO PARTITION spec2; * }}} */ -case class AlterTableRenamePartition( +case class AlterTableRenamePartitionCommand( tableName: TableIdentifier, oldPartition: TablePartitionSpec, newPartition: TablePartitionSpec) @@ -389,7 +389,7 @@ case class AlterTableRenamePartition( * ALTER TABLE table DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE]; * }}} */ -case class AlterTableDropPartition( +case class AlterTableDropPartitionCommand( tableName: TableIdentifier, specs: Seq[TablePartitionSpec], ifExists: Boolean) @@ -420,7 +420,7 @@ case class AlterTableDropPartition( * ALTER TABLE table_name [PARTITION partition_spec] SET LOCATION "loc"; * }}} */ -case class AlterTableSetLocation( +case class AlterTableSetLocationCommand( tableName: TableIdentifier, partitionSpec: Option[TablePartitionSpec], location: String) @@ -459,7 +459,7 @@ case class AlterTableSetLocation( } -private[sql] object DDLUtils { +object DDLUtils { def isDatasourceTable(props: Map[String, String]): Boolean = { props.contains("spark.sql.sources.provider") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 1ea9bc5299968..d2d8e3ddeae26 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} * AS className [USING JAR\FILE 'uri' [, JAR|FILE 'uri']] * }}} */ -case class CreateFunction( +case class CreateFunctionCommand( databaseName: Option[String], functionName: String, className: String, @@ -81,7 +81,7 @@ case class CreateFunction( * DESCRIBE FUNCTION [EXTENDED] upper; * }}} */ -case class DescribeFunction( +case class DescribeFunctionCommand( functionName: FunctionIdentifier, isExtended: Boolean) extends RunnableCommand { @@ -142,7 +142,7 @@ case class DescribeFunction( * ifExists: returns an error if the function doesn't exist, unless this is true. * isTemp: indicates if it is a temporary function. */ -case class DropFunction( +case class DropFunctionCommand( databaseName: Option[String], functionName: String, ifExists: Boolean, @@ -180,10 +180,10 @@ case class DropFunction( * For the pattern, '*' matches any sequence of characters (including no characters) and * '|' is for alternation. * For example, "show functions like 'yea*|windo*'" will return "window" and "year". - * - * TODO currently we are simply ignore the db */ -case class ShowFunctions(db: Option[String], pattern: Option[String]) extends RunnableCommand { +case class ShowFunctionsCommand(db: Option[String], pattern: Option[String]) + extends RunnableCommand { + override val output: Seq[Attribute] = { val schema = StructType(StructField("function", StringType, nullable = false) :: Nil) schema.toAttributes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala index 29bcb30592516..162d493c1f8ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.types.{IntegerType, StructField, StructType} /** * Adds a jar to the current session so it can be used (for UDFs or serdes). */ -case class AddJar(path: String) extends RunnableCommand { +case class AddJarCommand(path: String) extends RunnableCommand { override val output: Seq[Attribute] = { val schema = StructType( StructField("result", IntegerType, nullable = false) :: Nil) @@ -40,7 +40,7 @@ case class AddJar(path: String) extends RunnableCommand { /** * Adds a file to the current session so it can be used. */ -case class AddFile(path: String) extends RunnableCommand { +case class AddFileCommand(path: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sparkContext.addFile(path) Seq.empty[Row] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index d13492e55070b..13e63a1befb20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -39,9 +39,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils case class CreateTableAsSelectLogicalPlan( - tableDesc: CatalogTable, - child: LogicalPlan, - allowExisting: Boolean) extends UnaryNode with Command { + tableDesc: CatalogTable, + child: LogicalPlan, + allowExisting: Boolean) extends UnaryNode with Command { override def output: Seq[Attribute] = Seq.empty[Attribute] @@ -63,7 +63,7 @@ case class CreateTableAsSelectLogicalPlan( * LIKE [other_db_name.]existing_table_name * }}} */ -case class CreateTableLike( +case class CreateTableLikeCommand( targetTable: TableIdentifier, sourceTable: TableIdentifier, ifNotExists: Boolean) extends RunnableCommand { @@ -115,7 +115,7 @@ case class CreateTableLike( * [AS select_statement]; * }}} */ -case class CreateTable(table: CatalogTable, ifNotExists: Boolean) extends RunnableCommand { +case class CreateTableCommand(table: CatalogTable, ifNotExists: Boolean) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sessionState.catalog.createTable(table, ifNotExists) @@ -134,7 +134,7 @@ case class CreateTable(table: CatalogTable, ifNotExists: Boolean) extends Runnab * ALTER VIEW view1 RENAME TO view2; * }}} */ -case class AlterTableRename( +case class AlterTableRenameCommand( oldName: TableIdentifier, newName: TableIdentifier, isView: Boolean) @@ -159,7 +159,7 @@ case class AlterTableRename( * [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ -case class LoadData( +case class LoadDataCommand( table: TableIdentifier, path: String, isLocal: Boolean, @@ -281,7 +281,7 @@ case class LoadData( * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ -case class TruncateTable( +case class TruncateTableCommand( tableName: TableIdentifier, partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand { @@ -444,16 +444,17 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF append(buffer, "Sort Columns:", sortColumns.mkString("[", ", ", "]"), "") } - DDLUtils.getBucketSpecFromTableProperties(metadata).map { bucketSpec => - appendBucketInfo( - bucketSpec.numBuckets, - bucketSpec.bucketColumnNames, - bucketSpec.sortColumnNames) - }.getOrElse { - appendBucketInfo( - metadata.numBuckets, - metadata.bucketColumnNames, - metadata.sortColumnNames) + DDLUtils.getBucketSpecFromTableProperties(metadata) match { + case Some(bucketSpec) => + appendBucketInfo( + bucketSpec.numBuckets, + bucketSpec.bucketColumnNames, + bucketSpec.sortColumnNames) + case None => + appendBucketInfo( + metadata.numBuckets, + metadata.bucketColumnNames, + metadata.sortColumnNames) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 2e17b763a5370..e5dd4d81d6779 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -433,7 +433,7 @@ case class DataSource( // ordering of data.logicalPlan (partition columns are all moved after data column). This // will be adjusted within InsertIntoHadoopFsRelation. val plan = - InsertIntoHadoopFsRelation( + InsertIntoHadoopFsRelationCommand( outputPath, partitionColumns.map(UnresolvedAttribute.quoted), bucketSpec, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 0494fafb0e424..a3d87cd38bb8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -66,7 +66,7 @@ private[sql] object DataSourceAnalysis extends Rule[LogicalPlan] { "Cannot overwrite a path that is also being read from.") } - InsertIntoHadoopFsRelation( + InsertIntoHadoopFsRelationCommand( outputPath, t.partitionSchema.fields.map(_.name).map(UnresolvedAttribute(_)), t.bucketSpec, @@ -153,7 +153,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), part, query, overwrite, false) if part.isEmpty => - ExecutedCommandExec(InsertIntoDataSource(l, query, overwrite)) :: Nil + ExecutedCommandExec(InsertIntoDataSourceCommand(l, query, overwrite)) :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index 7b15e496414e3..c3e07f7d00557 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.sources.InsertableRelation /** * Inserts the results of `query` in to a relation that extends [[InsertableRelation]]. */ -private[sql] case class InsertIntoDataSource( +private[sql] case class InsertIntoDataSourceCommand( logicalRelation: LogicalRelation, query: LogicalPlan, overwrite: Boolean) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala similarity index 93% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 4921e4ca6bb7d..1426dcf4697ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -35,11 +35,11 @@ import org.apache.spark.sql.internal.SQLConf /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. - * Writing to dynamic partitions is also supported. Each [[InsertIntoHadoopFsRelation]] issues a - * single write job, and owns a UUID that identifies this job. Each concrete implementation of - * [[HadoopFsRelation]] should use this UUID together with task id to generate unique file path for - * each task output file. This UUID is passed to executor side via a property named - * `spark.sql.sources.writeJobUUID`. + * Writing to dynamic partitions is also supported. Each [[InsertIntoHadoopFsRelationCommand]] + * issues a single write job, and owns a UUID that identifies this job. Each concrete + * implementation of [[HadoopFsRelation]] should use this UUID together with task id to generate + * unique file path for each task output file. This UUID is passed to executor side via a + * property named `spark.sql.sources.writeJobUUID`. * * Different writer containers, [[DefaultWriterContainer]] and [[DynamicPartitionWriterContainer]] * are used to write to normal tables and tables with dynamic partitions. @@ -55,7 +55,7 @@ import org.apache.spark.sql.internal.SQLConf * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is * thrown during job commitment, also aborts the job. */ -private[sql] case class InsertIntoHadoopFsRelation( +private[sql] case class InsertIntoHadoopFsRelationCommand( outputPath: Path, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 78b1db16826e2..edbccde214c61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -91,7 +91,7 @@ case class CreateTempTableUsing( } } -case class CreateTempTableUsingAsSelect( +case class CreateTempTableUsingAsSelectCommand( tableIdent: TableIdentifier, provider: String, partitionColumns: Array[String], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 8f7c6f5d0ca44..939b9195cae99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.command.AnalyzeTable +import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, FindDataSourceTable, PreInsertCastAndRename, ResolveDataSource} import org.apache.spark.sql.util.ExecutionListenerManager @@ -189,6 +189,6 @@ private[sql] class SessionState(sparkSession: SparkSession) { * in the external catalog. */ def analyze(tableName: String): Unit = { - AnalyzeTable(tableName).run(sparkSession) + AnalyzeTableCommand(tableName).run(sparkSession) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 0925a51310e69..708b878c843a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -48,7 +48,7 @@ class DDLCommandSuite extends PlanTest { |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') """.stripMargin val parsed = parser.parsePlan(sql) - val expected = CreateDatabase( + val expected = CreateDatabaseCommand( "database_name", ifNotExists = true, Some("/home/user/db"), @@ -82,19 +82,19 @@ class DDLCommandSuite extends PlanTest { val parsed6 = parser.parsePlan(sql6) val parsed7 = parser.parsePlan(sql7) - val expected1 = DropDatabase( + val expected1 = DropDatabaseCommand( "database_name", ifExists = true, cascade = false) - val expected2 = DropDatabase( + val expected2 = DropDatabaseCommand( "database_name", ifExists = true, cascade = true) - val expected3 = DropDatabase( + val expected3 = DropDatabaseCommand( "database_name", ifExists = false, cascade = false) - val expected4 = DropDatabase( + val expected4 = DropDatabaseCommand( "database_name", ifExists = false, cascade = true) @@ -116,10 +116,10 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterDatabaseProperties( + val expected1 = AlterDatabasePropertiesCommand( "database_name", Map("a" -> "a", "b" -> "b", "c" -> "c")) - val expected2 = AlterDatabaseProperties( + val expected2 = AlterDatabasePropertiesCommand( "database_name", Map("a" -> "a")) @@ -141,10 +141,10 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = DescribeDatabase( + val expected1 = DescribeDatabaseCommand( "db_name", extended = true) - val expected2 = DescribeDatabase( + val expected2 = DescribeDatabaseCommand( "db_name", extended = false) @@ -167,7 +167,7 @@ class DDLCommandSuite extends PlanTest { """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = CreateFunction( + val expected1 = CreateFunctionCommand( None, "helloworld", "com.matthewrathbone.example.SimpleUDFExample", @@ -175,7 +175,7 @@ class DDLCommandSuite extends PlanTest { FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"), FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")), isTemp = true) - val expected2 = CreateFunction( + val expected2 = CreateFunctionCommand( Some("hello"), "world", "com.matthewrathbone.example.SimpleUDFExample", @@ -198,22 +198,22 @@ class DDLCommandSuite extends PlanTest { val parsed3 = parser.parsePlan(sql3) val parsed4 = parser.parsePlan(sql4) - val expected1 = DropFunction( + val expected1 = DropFunctionCommand( None, "helloworld", ifExists = false, isTemp = true) - val expected2 = DropFunction( + val expected2 = DropFunctionCommand( None, "helloworld", ifExists = true, isTemp = true) - val expected3 = DropFunction( + val expected3 = DropFunctionCommand( Some("hello"), "world", ifExists = false, isTemp = false) - val expected4 = DropFunction( + val expected4 = DropFunctionCommand( Some("hello"), "world", ifExists = true, @@ -231,11 +231,11 @@ class DDLCommandSuite extends PlanTest { containsThesePhrases = Seq("create external table", "location")) val query = "CREATE EXTERNAL TABLE my_tab LOCATION '/something/anything'" parser.parsePlan(query) match { - case ct: CreateTable => + case ct: CreateTableCommand => assert(ct.table.tableType == CatalogTableType.EXTERNAL) assert(ct.table.storage.locationUri == Some("/something/anything")) case other => - fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -253,11 +253,11 @@ class DDLCommandSuite extends PlanTest { test("create table - location implies external") { val query = "CREATE TABLE my_tab LOCATION '/something/anything'" parser.parsePlan(query) match { - case ct: CreateTable => + case ct: CreateTableCommand => assert(ct.table.tableType == CatalogTableType.EXTERNAL) assert(ct.table.storage.locationUri == Some("/something/anything")) case other => - fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -282,7 +282,7 @@ class DDLCommandSuite extends PlanTest { assert(Seq("a") == ct.partitionColumns.toSeq) comparePlans(ct.copy(partitionColumns = null), expected) case other => - fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -308,7 +308,7 @@ class DDLCommandSuite extends PlanTest { assert(ct.partitionColumns.isEmpty) comparePlans(ct.copy(partitionColumns = null), expected) case other => - fail(s"Expected to parse ${classOf[CreateTable].getClass.getName} from query," + + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + s"got ${other.getClass.getName}: $query") } } @@ -320,11 +320,11 @@ class DDLCommandSuite extends PlanTest { val sql_view = sql_table.replace("TABLE", "VIEW") val parsed_table = parser.parsePlan(sql_table) val parsed_view = parser.parsePlan(sql_view) - val expected_table = AlterTableRename( + val expected_table = AlterTableRenameCommand( TableIdentifier("table_name", None), TableIdentifier("new_table_name", None), isView = false) - val expected_view = AlterTableRename( + val expected_view = AlterTableRenameCommand( TableIdentifier("table_name", None), TableIdentifier("new_table_name", None), isView = true) @@ -353,11 +353,11 @@ class DDLCommandSuite extends PlanTest { val parsed3_view = parser.parsePlan(sql3_view) val tableIdent = TableIdentifier("table_name", None) - val expected1_table = AlterTableSetProperties( + val expected1_table = AlterTableSetPropertiesCommand( tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false) - val expected2_table = AlterTableUnsetProperties( + val expected2_table = AlterTableUnsetPropertiesCommand( tableIdent, Seq("comment", "test"), ifExists = false, isView = false) - val expected3_table = AlterTableUnsetProperties( + val expected3_table = AlterTableUnsetPropertiesCommand( tableIdent, Seq("comment", "test"), ifExists = true, isView = false) val expected1_view = expected1_table.copy(isView = true) val expected2_view = expected2_table.copy(isView = true) @@ -412,21 +412,21 @@ class DDLCommandSuite extends PlanTest { val parsed4 = parser.parsePlan(sql4) val parsed5 = parser.parsePlan(sql5) val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableSerDeProperties( + val expected1 = AlterTableSerDePropertiesCommand( tableIdent, Some("org.apache.class"), None, None) - val expected2 = AlterTableSerDeProperties( + val expected2 = AlterTableSerDePropertiesCommand( tableIdent, Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) - val expected3 = AlterTableSerDeProperties( + val expected3 = AlterTableSerDePropertiesCommand( tableIdent, None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) - val expected4 = AlterTableSerDeProperties( + val expected4 = AlterTableSerDePropertiesCommand( tableIdent, Some("org.apache.class"), Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), Some(Map("test" -> null, "dt" -> "2008-08-08", "country" -> "us"))) - val expected5 = AlterTableSerDeProperties( + val expected5 = AlterTableSerDePropertiesCommand( tableIdent, None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), @@ -459,13 +459,13 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) - val expected1 = AlterTableAddPartition( + val expected1 = AlterTableAddPartitionCommand( TableIdentifier("table_name", None), Seq( (Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")), (Map("dt" -> "2009-09-09", "country" -> "uk"), None)), ifNotExists = true) - val expected2 = AlterTableAddPartition( + val expected2 = AlterTableAddPartitionCommand( TableIdentifier("table_name", None), Seq((Map("dt" -> "2008-08-08"), Some("loc"))), ifNotExists = false) @@ -490,7 +490,7 @@ class DDLCommandSuite extends PlanTest { |RENAME TO PARTITION (dt='2008-09-09', country='uk') """.stripMargin val parsed = parser.parsePlan(sql) - val expected = AlterTableRenamePartition( + val expected = AlterTableRenamePartitionCommand( TableIdentifier("table_name", None), Map("dt" -> "2008-08-08", "country" -> "us"), Map("dt" -> "2008-09-09", "country" -> "uk")) @@ -529,7 +529,7 @@ class DDLCommandSuite extends PlanTest { assertUnsupported(sql2_view) val tableIdent = TableIdentifier("table_name", None) - val expected1_table = AlterTableDropPartition( + val expected1_table = AlterTableDropPartitionCommand( tableIdent, Seq( Map("dt" -> "2008-08-08", "country" -> "us"), @@ -565,11 +565,11 @@ class DDLCommandSuite extends PlanTest { val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableSetLocation( + val expected1 = AlterTableSetLocationCommand( tableIdent, None, "new location") - val expected2 = AlterTableSetLocation( + val expected2 = AlterTableSetLocationCommand( tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")), "new location") @@ -676,13 +676,13 @@ class DDLCommandSuite extends PlanTest { assertUnsupported(s"DROP TABLE IF EXISTS $tableName2 PURGE") val expected1 = - DropTable(TableIdentifier("tab", Option("db")), ifExists = false, isView = false) + DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = false, isView = false) val expected2 = - DropTable(TableIdentifier("tab", Option("db")), ifExists = true, isView = false) + DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = true, isView = false) val expected3 = - DropTable(TableIdentifier("tab", None), ifExists = false, isView = false) + DropTableCommand(TableIdentifier("tab", None), ifExists = false, isView = false) val expected4 = - DropTable(TableIdentifier("tab", None), ifExists = true, isView = false) + DropTableCommand(TableIdentifier("tab", None), ifExists = true, isView = false) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -700,13 +700,13 @@ class DDLCommandSuite extends PlanTest { val parsed4 = parser.parsePlan(s"DROP VIEW IF EXISTS $viewName2") val expected1 = - DropTable(TableIdentifier("view", Option("db")), ifExists = false, isView = true) + DropTableCommand(TableIdentifier("view", Option("db")), ifExists = false, isView = true) val expected2 = - DropTable(TableIdentifier("view", Option("db")), ifExists = true, isView = true) + DropTableCommand(TableIdentifier("view", Option("db")), ifExists = true, isView = true) val expected3 = - DropTable(TableIdentifier("view", None), ifExists = false, isView = true) + DropTableCommand(TableIdentifier("view", None), ifExists = false, isView = true) val expected4 = - DropTable(TableIdentifier("view", None), ifExists = true, isView = true) + DropTableCommand(TableIdentifier("view", None), ifExists = true, isView = true) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) 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 4c528fbbbeef7..86ab152402625 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 @@ -485,7 +485,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) - execution.CreateTableAsSelect( + execution.CreateTableAsSelectCommand( desc.copy(identifier = TableIdentifier(tblName, Some(dbName))), child, allowExisting) 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/CreateTableAsSelectCommand.scala similarity index 99% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala index 9dfbafae872f6..3fc900961e645 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/CreateTableAsSelectCommand.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.hive.MetastoreRelation * raise exception */ private[hive] -case class CreateTableAsSelect( +case class CreateTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, allowExisting: Boolean) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 2d8b1f325ac32..30ad392969b4e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Generate, ScriptTransformation} -import org.apache.spark.sql.execution.command.{CreateTable, CreateTableAsSelectLogicalPlan, CreateTableLike, CreateViewCommand, LoadData} +import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.hive.test.TestHive class HiveDDLCommandSuite extends PlanTest { @@ -37,7 +37,7 @@ class HiveDDLCommandSuite extends PlanTest { private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { parser.parsePlan(sql).collect { - case CreateTable(desc, allowExisting) => (desc, allowExisting) + case CreateTableCommand(desc, allowExisting) => (desc, allowExisting) case CreateTableAsSelectLogicalPlan(desc, _, allowExisting) => (desc, allowExisting) case CreateViewCommand(desc, _, allowExisting, _, _, _) => (desc, allowExisting) }.head @@ -555,7 +555,7 @@ class HiveDDLCommandSuite extends PlanTest { test("create table like") { val v1 = "CREATE TABLE table1 LIKE table2" val (target, source, exists) = parser.parsePlan(v1).collect { - case CreateTableLike(t, s, allowExisting) => (t, s, allowExisting) + case CreateTableLikeCommand(t, s, allowExisting) => (t, s, allowExisting) }.head assert(exists == false) assert(target.database.isEmpty) @@ -565,7 +565,7 @@ class HiveDDLCommandSuite extends PlanTest { val v2 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2" val (target2, source2, exists2) = parser.parsePlan(v2).collect { - case CreateTableLike(t, s, allowExisting) => (t, s, allowExisting) + case CreateTableLikeCommand(t, s, allowExisting) => (t, s, allowExisting) }.head assert(exists2) assert(target2.database.isEmpty) @@ -577,7 +577,7 @@ class HiveDDLCommandSuite extends PlanTest { test("load data") { val v1 = "LOAD DATA INPATH 'path' INTO TABLE table1" val (table, path, isLocal, isOverwrite, partition) = parser.parsePlan(v1).collect { - case LoadData(t, path, l, o, partition) => (t, path, l, o, partition) + case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) }.head assert(table.database.isEmpty) assert(table.table == "table1") @@ -588,7 +588,7 @@ class HiveDDLCommandSuite extends PlanTest { val v2 = "LOAD DATA LOCAL INPATH 'path' OVERWRITE INTO TABLE table1 PARTITION(c='1', d='2')" val (table2, path2, isLocal2, isOverwrite2, partition2) = parser.parsePlan(v2).collect { - case LoadData(t, path, l, o, partition) => (t, path, l, o, partition) + case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) }.head assert(table2.database.isEmpty) assert(table2.table == "table1") 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 7011cd81221c5..1a7b6c0112279 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 @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.command.AnalyzeTable +import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -33,7 +33,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { val parsed = hiveContext.parseSql(analyzeCommand) val operators = parsed.collect { - case a: AnalyzeTable => a + case a: AnalyzeTableCommand => a case o => o } @@ -49,23 +49,23 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { assertAnalyzeCommand( "ANALYZE TABLE Table1 COMPUTE STATISTICS", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS noscan", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) assertAnalyzeCommand( "ANALYZE TABLE Table1 COMPUTE STATISTICS nOscAn", - classOf[AnalyzeTable]) + classOf[AnalyzeTableCommand]) } ignore("analyze MetastoreRelations") { 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 7fe158c218461..3e5140fe578de 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 @@ -23,7 +23,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.command.ExecutedCommandExec -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand, LogicalRelation} import org.apache.spark.sql.hive.execution.HiveTableScanExec import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -308,10 +308,10 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt") df.queryExecution.sparkPlan match { - case ExecutedCommandExec(_: InsertIntoHadoopFsRelation) => // OK + case ExecutedCommandExec(_: InsertIntoHadoopFsRelationCommand) => // OK case o => fail("test_insert_parquet should be converted to a " + s"${classOf[HadoopFsRelation ].getCanonicalName} and " + - s"${classOf[InsertIntoDataSource].getCanonicalName} is expected as the SparkPlan. " + + s"${classOf[InsertIntoDataSourceCommand].getCanonicalName} should have been SparkPlan. " + s"However, found a ${o.toString} ") } @@ -338,10 +338,10 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array") df.queryExecution.sparkPlan match { - case ExecutedCommandExec(_: InsertIntoHadoopFsRelation) => // OK + case ExecutedCommandExec(_: InsertIntoHadoopFsRelationCommand) => // OK case o => fail("test_insert_parquet should be converted to a " + s"${classOf[HadoopFsRelation ].getCanonicalName} and " + - s"${classOf[InsertIntoDataSource].getCanonicalName} is expected as the SparkPlan." + + s"${classOf[InsertIntoDataSourceCommand].getCanonicalName} should have been SparkPlan." + s"However, found a ${o.toString} ") } From 0dd3bdc2738a8ddaa69c471b2f31fd6f3d41ce46 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 20 May 2016 09:38:34 -0700 Subject: [PATCH 0319/1470] [SPARK-15400][SQL] CreateNamedStruct and CreateNamedStructUnsafe should preserve metadata of value expressions if it is NamedExpression. ## What changes were proposed in this pull request? `CreateNamedStruct` and `CreateNamedStructUnsafe` should preserve metadata of value expressions if it is `NamedExpression` like `CreateStruct` or `CreateStructUnsafe` are doing. ## How was this patch tested? Existing tests. Author: Takuya UESHIN Closes #13193 from ueshin/issues/SPARK-15400. (cherry picked from commit 2cbe96e64d5f84474b2eb59bed9ce3ab543d8aff) Signed-off-by: Reynold Xin --- .../expressions/complexTypeCreator.scala | 17 ++++++++++++----- .../expressions/ComplexTypeSuite.scala | 18 ++++++++++++++++++ 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index d986d9dca6ed4..f60d278c54873 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -252,9 +252,13 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { private lazy val names = nameExprs.map(_.eval(EmptyRow)) override lazy val dataType: StructType = { - val fields = names.zip(valExprs).map { case (name, valExpr) => - StructField(name.asInstanceOf[UTF8String].toString, - valExpr.dataType, valExpr.nullable, Metadata.empty) + val fields = names.zip(valExprs).map { + case (name, valExpr: NamedExpression) => + StructField(name.asInstanceOf[UTF8String].toString, + valExpr.dataType, valExpr.nullable, valExpr.metadata) + case (name, valExpr) => + StructField(name.asInstanceOf[UTF8String].toString, + valExpr.dataType, valExpr.nullable, Metadata.empty) } StructType(fields) } @@ -365,8 +369,11 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression private lazy val names = nameExprs.map(_.eval(EmptyRow).toString) override lazy val dataType: StructType = { - val fields = names.zip(valExprs).map { case (name, valExpr) => - StructField(name, valExpr.dataType, valExpr.nullable, Metadata.empty) + val fields = names.zip(valExprs).map { + case (name, valExpr: NamedExpression) => + StructField(name, valExpr.dataType, valExpr.nullable, valExpr.metadata) + case (name, valExpr) => + StructField(name, valExpr.dataType, valExpr.nullable, Metadata.empty) } StructType(fields) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 7c009a7360b6f..ec7be4d4b849d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -228,4 +228,22 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { checkErrorMessage(structType, IntegerType, "Field name should be String Literal") checkErrorMessage(otherType, StringType, "Can't extract value from") } + + test("ensure to preserve metadata") { + val metadata = new MetadataBuilder() + .putString("key", "value") + .build() + + def checkMetadata(expr: Expression): Unit = { + assert(expr.dataType.asInstanceOf[StructType]("a").metadata === metadata) + assert(expr.dataType.asInstanceOf[StructType]("b").metadata === Metadata.empty) + } + + val a = AttributeReference("a", IntegerType, metadata = metadata)() + val b = AttributeReference("b", IntegerType)() + checkMetadata(CreateStruct(Seq(a, b))) + checkMetadata(CreateNamedStruct(Seq("a", a, "b", b))) + checkMetadata(CreateStructUnsafe(Seq(a, b))) + checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) + } } From 96e41dc6a5d8ad44f0756255e15452affabb079b Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 20 May 2016 10:27:41 -0700 Subject: [PATCH 0320/1470] [SPARK-15360][SPARK-SUBMIT] Should print spark-submit usage when no arguments is specified (Please fill in changes proposed in this fix) In 2.0, ./bin/spark-submit doesn't print out usage, but it raises an exception. In this PR, an exception handling is added in the Main.java when the exception is thrown. In the handling code, if there is no additional argument, it prints out usage. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually tested. ./bin/spark-submit Usage: spark-submit [options] [app arguments] Usage: spark-submit --kill [submission ID] --master [spark://...] Usage: spark-submit --status [submission ID] --master [spark://...] Usage: spark-submit run-example [options] example-class [example args] Options: --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or on one of the worker machines inside the cluster ("cluster") (Default: client). --class CLASS_NAME Your application's main class (for Java / Scala apps). --name NAME A name of your application. --jars JARS Comma-separated list of local jars to include on the driver and executor classpaths. --packages Comma-separated list of maven coordinates of jars to include on the driver and executor classpaths. Will search the local maven repo, then maven central and any additional remote repositories given by --repositories. The format for the coordinates should be groupId:artifactId:version. Author: wm624@hotmail.com Closes #13163 from wangmiao1981/submit. (cherry picked from commit fe2fcb48039ac897242e2cfaed31703fa6116db7) Signed-off-by: Marcelo Vanzin --- .../launcher/SparkSubmitCommandBuilder.java | 47 +++++++++++-------- .../SparkSubmitCommandBuilderSuite.java | 13 +++++ 2 files changed, 41 insertions(+), 19 deletions(-) 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 76897c4f75199..824500d980eee 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -107,28 +107,37 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { SparkSubmitCommandBuilder(List args) { this.allowsMixedArguments = false; - + this.sparkArgs = new ArrayList<>(); boolean isExample = false; List submitArgs = args; - if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { - this.allowsMixedArguments = true; - appResource = PYSPARK_SHELL; - submitArgs = args.subList(1, args.size()); - } else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) { - this.allowsMixedArguments = true; - appResource = SPARKR_SHELL; - submitArgs = args.subList(1, args.size()); - } else if (args.size() > 0 && args.get(0).equals(RUN_EXAMPLE)) { - isExample = true; - submitArgs = args.subList(1, args.size()); - } - this.sparkArgs = new ArrayList<>(); - this.isExample = isExample; + if (args.size() > 0) { + switch (args.get(0)) { + case PYSPARK_SHELL: + this.allowsMixedArguments = true; + appResource = PYSPARK_SHELL; + submitArgs = args.subList(1, args.size()); + break; + + case SPARKR_SHELL: + this.allowsMixedArguments = true; + appResource = SPARKR_SHELL; + submitArgs = args.subList(1, args.size()); + break; + + case RUN_EXAMPLE: + isExample = true; + submitArgs = args.subList(1, args.size()); + } - OptionParser parser = new OptionParser(); - parser.parse(submitArgs); - this.printInfo = parser.infoRequested; + this.isExample = isExample; + OptionParser parser = new OptionParser(); + parser.parse(submitArgs); + this.printInfo = parser.infoRequested; + } else { + this.isExample = isExample; + this.printInfo = true; + } } @Override @@ -147,7 +156,7 @@ List buildSparkSubmitArgs() { List args = new ArrayList<>(); SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); - if (!allowsMixedArguments) { + if (!allowsMixedArguments && !printInfo) { checkArgument(appResource != null, "Missing application resource."); } 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 c16f46a360277..ffe11bc51b722 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -58,6 +58,19 @@ public void testClusterCmdBuilder() throws Exception { testCmdBuilder(false, false); } + @Test + public void testCliHelpAndNoArg() throws Exception { + List helpArgs = Arrays.asList(parser.HELP); + Map env = new HashMap<>(); + List cmd = buildCommand(helpArgs, env); + assertTrue("--help should be contained in the final cmd.", cmd.contains(parser.HELP)); + + List sparkEmptyArgs = Collections.emptyList(); + cmd = buildCommand(sparkEmptyArgs, env); + assertTrue("org.apache.spark.deploy.SparkSubmit should be contained in the final cmd of empty input.", + cmd.contains("org.apache.spark.deploy.SparkSubmit")); + } + @Test public void testCliParser() throws Exception { List sparkSubmitArgs = Arrays.asList( From 1677fd31937fde19fdfc8323cdb33b44f3a67204 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 20 May 2016 10:44:26 -0700 Subject: [PATCH 0321/1470] [HOTFIX] disable stress test --- .../apache/spark/sql/execution/joins/HashedRelationSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index a5b56541c90f7..40864c80ebc81 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -213,7 +213,8 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { longRelation.close() } - test("build HashedRelation with more than 100 millions rows") { + // This test require 4G heap to run, should run it manually + ignore("build HashedRelation with more than 100 millions rows") { val unsafeProj = UnsafeProjection.create( Seq(BoundReference(0, IntegerType, false), BoundReference(1, StringType, true))) From e99b22080b47e0596254cad4ac6eb28b8c4c69a0 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 20 May 2016 10:56:35 -0700 Subject: [PATCH 0322/1470] [SPARK-15165] [SPARK-15205] [SQL] Introduce place holder for comments in generated code ## What changes were proposed in this pull request? This PR introduce place holder for comment in generated code and the purpose is same for #12939 but much safer. Generated code to be compiled doesn't include actual comments but includes place holder instead. Place holders in generated code will be replaced with actual comments only at the time of logging. Also, this PR can resolve SPARK-15205. ## How was this patch tested? Existing tests. Author: Kousuke Saruta Closes #12979 from sarutak/SPARK-15205. (cherry picked from commit 22947cd0213856442025baf653be588c6c707e36) Signed-off-by: Davies Liu --- .../sql/catalyst/expressions/Expression.scala | 6 +- .../expressions/codegen/CodeFormatter.scala | 12 +++- .../expressions/codegen/CodeGenerator.scala | 59 +++++++++++++++++-- .../expressions/codegen/CodegenFallback.scala | 6 +- .../codegen/GenerateMutableProjection.scala | 3 +- .../codegen/GenerateOrdering.scala | 3 +- .../codegen/GeneratePredicate.scala | 3 +- .../codegen/GenerateSafeProjection.scala | 3 +- .../codegen/GenerateUnsafeProjection.scala | 3 +- .../codegen/GenerateUnsafeRowJoiner.scala | 4 +- .../spark/sql/catalyst/util/package.scala | 21 ------- .../codegen/CodeFormatterSuite.scala | 5 +- .../spark/sql/execution/ExistingRDD.scala | 3 +- .../sql/execution/WholeStageCodegenExec.scala | 16 +++-- .../columnar/GenerateColumnAccessor.scala | 5 +- 15 files changed, 95 insertions(+), 57 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index fab163476f5d1..b4fe151f277a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -21,7 +21,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.types._ //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -97,15 +96,14 @@ abstract class Expression extends TreeNode[Expression] { ctx.subExprEliminationExprs.get(this).map { subExprState => // This expression is repeated which means that the code to evaluate it has already been added // as a function before. In that case, we just re-use it. - val code = s"/* ${toCommentSafeString(this.toString)} */" - ExprCode(code, subExprState.isNull, subExprState.value) + ExprCode(ctx.registerComment(this.toString), subExprState.isNull, subExprState.value) }.getOrElse { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") val ve = doGenCode(ctx, ExprCode("", isNull, value)) if (ve.code.nonEmpty) { // Add `this` in the comment. - ve.copy(s"/* ${toCommentSafeString(this.toString)} */\n" + ve.code.trim) + ve.copy(code = s"${ctx.registerComment(this.toString)}\n" + ve.code.trim) } else { ve } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index ab4831f7abdd0..c7410925da6c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.codegen +import org.apache.commons.lang3.StringUtils + /** * An utility class that indents a block of code based on the curly braces and parentheses. * This is used to prettify generated code when in debug mode (or exceptions). @@ -24,7 +26,15 @@ package org.apache.spark.sql.catalyst.expressions.codegen * Written by Matei Zaharia. */ object CodeFormatter { - def format(code: String): String = new CodeFormatter().addLines(code).result() + def format(code: CodeAndComment): String = { + new CodeFormatter().addLines( + StringUtils.replaceEach( + code.body, + code.comment.keys.toArray, + code.comment.values.toArray) + ).result + } + def stripExtraNewLines(input: String): String = { val code = new StringBuilder var lastLine: String = "dummy" 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 67f671926561a..8b74d606dbb26 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 @@ -199,6 +199,11 @@ class CodegenContext { */ var freshNamePrefix = "" + /** + * The map from a place holder to a corresponding comment + */ + private val placeHolderToComments = new mutable.HashMap[String, String] + /** * Returns a term name that is unique within this instance of a `CodegenContext`. */ @@ -706,6 +711,35 @@ class CodegenContext { if (doSubexpressionElimination) subexpressionElimination(expressions) expressions.map(e => e.genCode(this)) } + + /** + * get a map of the pair of a place holder and a corresponding comment + */ + def getPlaceHolderToComments(): collection.Map[String, String] = placeHolderToComments + + /** + * Register a multi-line comment and return the corresponding place holder + */ + private def registerMultilineComment(text: String): String = { + val placeHolder = s"/*${freshName("c")}*/" + val comment = text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") + placeHolderToComments += (placeHolder -> comment) + placeHolder + } + + /** + * Register a comment and return the corresponding place holder + */ + def registerComment(text: String): String = { + if (text.contains("\n") || text.contains("\r")) { + registerMultilineComment(text) + } else { + val placeHolder = s"/*${freshName("c")}*/" + val safeComment = s"// $text" + placeHolderToComments += (placeHolder -> safeComment) + placeHolder + } + } } /** @@ -716,6 +750,19 @@ abstract class GeneratedClass { def generate(references: Array[Any]): Any } +/** + * A wrapper for the source code to be compiled by [[CodeGenerator]]. + */ +class CodeAndComment(val body: String, val comment: collection.Map[String, String]) + extends Serializable { + override def equals(that: Any): Boolean = that match { + case t: CodeAndComment if t.body == body => true + case _ => false + } + + override def hashCode(): Int = body.hashCode +} + /** * A base class for generators of byte code to perform expression evaluation. Includes a set of * helpers for referring to Catalyst types and building trees that perform evaluation of individual @@ -760,14 +807,14 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. */ - def compile(code: String): GeneratedClass = { + def compile(code: CodeAndComment): GeneratedClass = { cache.get(code) } /** * Compile the Java source code into a Java class, using Janino. */ - private[this] def doCompile(code: String): GeneratedClass = { + private[this] def doCompile(code: CodeAndComment): GeneratedClass = { val evaluator = new ClassBodyEvaluator() evaluator.setParentClassLoader(Utils.getContextOrSparkClassLoader) // Cannot be under package codegen, or fail with java.lang.InstantiationException @@ -788,7 +835,7 @@ object CodeGenerator extends Logging { )) evaluator.setExtendedClass(classOf[GeneratedClass]) - def formatted = CodeFormatter.format(code) + lazy val formatted = CodeFormatter.format(code) logDebug({ // Only add extra debugging info to byte code when we are going to print the source code. @@ -797,7 +844,7 @@ object CodeGenerator extends Logging { }) try { - evaluator.cook("generated.java", code) + evaluator.cook("generated.java", code.body) } catch { case e: Exception => val msg = s"failed to compile: $e\n$formatted" @@ -819,8 +866,8 @@ object CodeGenerator extends Logging { private val cache = CacheBuilder.newBuilder() .maximumSize(100) .build( - new CacheLoader[String, GeneratedClass]() { - override def load(code: String): GeneratedClass = { + new CacheLoader[CodeAndComment, GeneratedClass]() { + override def load(code: CodeAndComment): GeneratedClass = { val startTime = System.nanoTime() val result = doCompile(code) val endTime = System.nanoTime() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala index 2bd77c65c31cf..6a5a3e7933eea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression, Nondeterministic} -import org.apache.spark.sql.catalyst.util.toCommentSafeString /** * A trait that can be used to provide a fallback mode for expression code generation. @@ -36,9 +35,10 @@ trait CodegenFallback extends Expression { val idx = ctx.references.length ctx.references += this val objectTerm = ctx.freshName("obj") + val placeHolder = ctx.registerComment(this.toString) if (nullable) { ev.copy(code = s""" - /* expression: ${toCommentSafeString(this.toString)} */ + $placeHolder Object $objectTerm = ((Expression) references[$idx]).eval($input); boolean ${ev.isNull} = $objectTerm == null; ${ctx.javaType(this.dataType)} ${ev.value} = ${ctx.defaultValue(this.dataType)}; @@ -47,7 +47,7 @@ trait CodegenFallback extends Expression { }""") } else { ev.copy(code = s""" - /* expression: ${toCommentSafeString(this.toString)} */ + $placeHolder Object $objectTerm = ((Expression) references[$idx]).eval($input); ${ctx.javaType(this.dataType)} ${ev.value} = (${ctx.boxedType(this.dataType)}) $objectTerm; """, isNull = "false") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index f143b40443836..1305289e783c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -94,7 +94,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP val allProjections = ctx.splitExpressions(ctx.INPUT_ROW, projectionCodes) val allUpdates = ctx.splitExpressions(ctx.INPUT_ROW, updates) - val code = s""" + val codeBody = s""" public java.lang.Object generate(Object[] references) { return new SpecificMutableProjection(references); } @@ -133,6 +133,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP } """ + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) 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 dc4825cdd8765..1c53d62a5e98a 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 @@ -113,7 +113,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR protected def create(ordering: Seq[SortOrder]): BaseOrdering = { val ctx = newCodeGenContext() val comparisons = genComparisons(ctx, ordering) - val code = s""" + val codeBody = s""" public SpecificOrdering generate(Object[] references) { return new SpecificOrdering(references); } @@ -136,6 +136,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR } }""" + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index dd8e2a289a661..ef44e6b46b798 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -40,7 +40,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool protected def create(predicate: Expression): ((InternalRow) => Boolean) = { val ctx = newCodeGenContext() val eval = predicate.genCode(ctx) - val code = s""" + val codeBody = s""" public SpecificPredicate generate(Object[] references) { return new SpecificPredicate(references); } @@ -61,6 +61,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool } }""" + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") val p = CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index ee1a363145dad..b0b121255357a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -155,7 +155,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] """ } val allExpressions = ctx.splitExpressions(ctx.INPUT_ROW, expressionCodes) - val code = s""" + val codeBody = s""" public java.lang.Object generate(Object[] references) { return new SpecificSafeProjection(references); } @@ -181,6 +181,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] } """ + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 6aa9cbf08bdb9..102f276e9b3ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -362,7 +362,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro val ctx = newCodeGenContext() val eval = createCode(ctx, expressions, subexpressionEliminationEnabled) - val code = s""" + val codeBody = s""" public java.lang.Object generate(Object[] references) { return new SpecificUnsafeProjection(references); } @@ -390,6 +390,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } """ + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index b1ffbaa3e94ec..4dc1678ff67bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -157,7 +157,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U }.mkString("\n") // ------------------------ Finally, put everything together --------------------------- // - val code = s""" + val codeBody = s""" |public java.lang.Object generate(Object[] references) { | return new SpecificUnsafeRowJoiner(); |} @@ -193,7 +193,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | } |} """.stripMargin - + val code = new CodeAndComment(codeBody, Map.empty) logDebug(s"SpecificUnsafeRowJoiner($schema1, $schema2):\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index f1d6cab9a5a1c..4005087dad05a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -155,27 +155,6 @@ package object util { def toPrettySQL(e: Expression): String = usePrettyExpression(e).sql - /** - * Returns the string representation of this expression that is safe to be put in - * code comments of generated code. The length is capped at 128 characters. - */ - def toCommentSafeString(str: String): String = { - val len = math.min(str.length, 128) - val suffix = if (str.length > len) "..." else "" - - // Unicode literals, like \u0022, should be escaped before - // they are put in code comment to avoid codegen breaking. - // To escape them, single "\" should be prepended to a series of "\" just before "u" - // only when the number of "\" is odd. - // For example, \u0022 should become to \\u0022 - // but \\u0022 should not become to \\\u0022 because the first backslash escapes the second one, - // and \u0022 will remain, means not escaped. - // Otherwise, the runtime Java compiler will fail to compile or code injection can be allowed. - // For details, see SPARK-15165. - str.substring(0, len).replace("*/", "*\\/") - .replaceAll("(^|[^\\\\])(\\\\(\\\\\\\\)*u)", "$1\\\\$2") + suffix - } - /* FIX ME implicit class debugLogging(a: Any) { def debugLogging() { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala index f57b82bb96399..6022f2dbbe948 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala @@ -25,11 +25,12 @@ class CodeFormatterSuite extends SparkFunSuite { def testCase(name: String)(input: String)(expected: String): Unit = { test(name) { - if (CodeFormatter.format(input).trim !== expected.trim) { + val sourceCode = new CodeAndComment(input, Map.empty) + if (CodeFormatter.format(sourceCode).trim !== expected.trim) { fail( s""" |== FAIL: Formatted code doesn't match === - |${sideBySide(CodeFormatter.format(input).trim, expected.trim).mkString("\n")} + |${sideBySide(CodeFormatter.format(sourceCode).trim, expected.trim).mkString("\n")} """.stripMargin) } } 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 d8911f88b0004..ec23a9c41a029 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 @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} -import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -253,7 +252,7 @@ private[sql] case class BatchedDataSourceScanExec( val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } val valueVar = ctx.freshName("value") val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = s"/* ${toCommentSafeString(str)} */\n" + (if (nullable) { + val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { s""" boolean ${isNullVar} = ${columnVar}.isNullAt($ordinal); $javaType ${valueVar} = ${isNullVar} ? ${ctx.defaultValue(dataType)} : ($value); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index d6f7b6ed35dbd..37fdc362b5e4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.toCommentSafeString import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -79,7 +78,7 @@ trait CodegenSupport extends SparkPlan { this.parent = parent ctx.freshNamePrefix = variablePrefix s""" - |/*** PRODUCE: ${toCommentSafeString(this.simpleString)} */ + |${ctx.registerComment(s"PRODUCE: ${this.simpleString}")} |${doProduce(ctx)} """.stripMargin } @@ -147,8 +146,7 @@ trait CodegenSupport extends SparkPlan { ctx.freshNamePrefix = parent.variablePrefix val evaluated = evaluateRequiredVariables(output, inputVars, parent.usedInputs) s""" - | - |/*** CONSUME: ${toCommentSafeString(parent.simpleString)} */ + |${ctx.registerComment(s"CONSUME: ${parent.simpleString}")} |$evaluated |${parent.doConsume(ctx, inputVars, rowVar)} """.stripMargin @@ -299,7 +297,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co * * @return the tuple of the codegen context and the actual generated source. */ - def doCodeGen(): (CodegenContext, String) = { + def doCodeGen(): (CodegenContext, CodeAndComment) = { val ctx = new CodegenContext val code = child.asInstanceOf[CodegenSupport].produce(ctx, this) val source = s""" @@ -307,9 +305,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co return new GeneratedIterator(references); } - /** Codegened pipeline for: - * ${toCommentSafeString(child.treeString.trim)} - */ + ${ctx.registerComment(s"""Codegend pipeline for\n${child.treeString.trim}""")} final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { private Object[] references; @@ -333,7 +329,9 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co """.trim // try to compile, helpful for debug - val cleanedSource = CodeFormatter.stripExtraNewLines(source) + val cleanedSource = + new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments()) + logDebug(s"\n${CodeFormatter.format(cleanedSource)}") CodeGenerator.compile(cleanedSource) (ctx, cleanedSource) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index bd5cb800dde3c..e0b48119f61dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator, UnsafeRowWriter} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, UnsafeRowWriter} import org.apache.spark.sql.types._ /** @@ -150,7 +150,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera (0 to groupedAccessorsLength - 1).map { i => s"extractors$i();" }.mkString("\n")) } - val code = s""" + val codeBody = s""" import java.nio.ByteBuffer; import java.nio.ByteOrder; import scala.collection.Iterator; @@ -224,6 +224,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera } }""" + val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] From 42e63c35a60dc256759cb42260ba1113df05c74b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 20 May 2016 12:38:46 -0700 Subject: [PATCH 0323/1470] [SPARK-15190][SQL] Support using SQLUserDefinedType for case classes ## What changes were proposed in this pull request? Right now inferring the schema for case classes happens before searching the SQLUserDefinedType annotation, so the SQLUserDefinedType annotation for case classes doesn't work. This PR simply changes the inferring order to resolve it. I also reenabled the java.math.BigDecimal test and added two tests for `List`. ## How was this patch tested? `encodeDecodeTest(UDTCaseClass(new java.net.URI("http://spark.apache.org/")), "udt with case class")` Author: Shixiong Zhu Closes #12965 from zsxwing/SPARK-15190. (cherry picked from commit dfa61f7b136ae060bbe04e3c0da1148da41018c7) Signed-off-by: Michael Armbrust --- .../spark/sql/catalyst/ScalaReflection.scala | 70 +++++++++---------- .../encoders/ExpressionEncoderSuite.scala | 28 +++++++- 2 files changed, 62 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 58df651da2942..36989a20cb5d5 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 @@ -348,6 +348,23 @@ object ScalaReflection extends ScalaReflection { "toScalaMap", keyData :: valueData :: Nil) + case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + val obj = NewInstance( + udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), + Nil, + dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) + Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) + + case t if UDTRegistration.exists(getClassNameFromType(t)) => + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() + .asInstanceOf[UserDefinedType[_]] + val obj = NewInstance( + udt.getClass, + Nil, + dataType = ObjectType(udt.getClass)) + Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) + case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) @@ -388,23 +405,6 @@ object ScalaReflection extends ScalaReflection { } else { newInstance } - - case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() - val obj = NewInstance( - udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), - Nil, - dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) - - case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] - val obj = NewInstance( - udt.getClass, - Nil, - dataType = ObjectType(udt.getClass)) - Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) } } @@ -522,17 +522,6 @@ object ScalaReflection extends ScalaReflection { val TypeRef(_, _, Seq(elementType)) = t toCatalystArray(inputObject, elementType) - case t if definedByConstructorParams(t) => - val params = getConstructorParameters(t) - val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => - val fieldValue = Invoke(inputObject, fieldName, dataTypeFor(fieldType)) - val clsName = getClassNameFromType(fieldType) - val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath - expressions.Literal(fieldName) :: serializerFor(fieldValue, fieldType, newPath) :: Nil - }) - val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) - expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) - case t if t <:< localTypeOf[Array[_]] => val TypeRef(_, _, Seq(elementType)) = t toCatalystArray(inputObject, elementType) @@ -645,6 +634,17 @@ object ScalaReflection extends ScalaReflection { dataType = ObjectType(udt.getClass)) Invoke(obj, "serialize", udt.sqlType, inputObject :: Nil) + case t if definedByConstructorParams(t) => + val params = getConstructorParameters(t) + val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => + val fieldValue = Invoke(inputObject, fieldName, dataTypeFor(fieldType)) + val clsName = getClassNameFromType(fieldType) + val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath + expressions.Literal(fieldName) :: serializerFor(fieldValue, fieldType, newPath) :: Nil + }) + val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) + expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) + case other => throw new UnsupportedOperationException( s"No Encoder found for $tpe\n" + walkedTypePath.mkString("\n")) @@ -743,13 +743,6 @@ object ScalaReflection extends ScalaReflection { val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if definedByConstructorParams(t) => - val params = getConstructorParameters(t) - Schema(StructType( - params.map { case (fieldName, fieldType) => - val Schema(dataType, nullable) = schemaFor(fieldType) - StructField(fieldName, dataType, nullable) - }), nullable = true) case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) @@ -775,6 +768,13 @@ object ScalaReflection extends ScalaReflection { case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) + case t if definedByConstructorParams(t) => + val params = getConstructorParameters(t) + Schema(StructType( + params.map { case (fieldName, fieldType) => + val Schema(dataType, nullable) = schemaFor(fieldType) + StructField(fieldName, dataType, nullable) + }), nullable = true) case other => throw new UnsupportedOperationException(s"Schema for type $other is not supported") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index d4387890b403b..3d97113b52e39 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.ArrayData -import org.apache.spark.sql.types.{ArrayType, Decimal, ObjectType, StructType} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String case class RepeatedStruct(s: Seq[PrimitiveData]) @@ -86,6 +87,25 @@ class JavaSerializable(val value: Int) extends Serializable { } } +/** For testing UDT for a case class */ +@SQLUserDefinedType(udt = classOf[UDTForCaseClass]) +case class UDTCaseClass(uri: java.net.URI) + +class UDTForCaseClass extends UserDefinedType[UDTCaseClass] { + + override def sqlType: DataType = StringType + + override def serialize(obj: UDTCaseClass): UTF8String = { + UTF8String.fromString(obj.uri.toString) + } + + override def userClass: Class[UDTCaseClass] = classOf[UDTCaseClass] + + override def deserialize(datum: Any): UDTCaseClass = datum match { + case uri: UTF8String => UDTCaseClass(new java.net.URI(uri.toString)) + } +} + class ExpressionEncoderSuite extends PlanTest with AnalysisTest { OuterScopes.addOuterScope(this) @@ -147,6 +167,12 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { encodeDecodeTest(Tuple1[Seq[Int]](null), "null seq in tuple") encodeDecodeTest(Tuple1[Map[String, String]](null), "null map in tuple") + encodeDecodeTest(List(1, 2), "list of int") + encodeDecodeTest(List("a", null), "list with String and null") + + encodeDecodeTest( + UDTCaseClass(new java.net.URI("http://spark.apache.org/")), "udt with case class") + // Kryo encoders encodeDecodeTest("hello", "kryo string")(encoderFor(Encoders.kryo[String])) encodeDecodeTest(new KryoSerializable(15), "kryo object")( From 3ed9ba6e1a8b84f69b21c4d17d0edb574de5c176 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 20 May 2016 13:00:29 -0700 Subject: [PATCH 0324/1470] =?UTF-8?q?[SPARK-10216][SQL]=20Revert=20"[]=20A?= =?UTF-8?q?void=20creating=20empty=20files=20during=20overwrit=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 8d05a7a from #12855, which seems to have caused regressions when working with empty DataFrames. Author: Michael Armbrust Closes #13181 from marmbrus/revert12855. (cherry picked from commit 2ba3ff044900d16d5f6331523526f785864c1e62) Signed-off-by: Michael Armbrust --- .../datasources/WriterContainer.scala | 221 +++++++++--------- .../spark/sql/hive/hiveWriterContainers.scala | 24 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 41 +--- .../sql/sources/HadoopFsRelationTest.scala | 22 +- 4 files changed, 126 insertions(+), 182 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 7e12bbb2128bf..3b064a5bc489f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -239,50 +239,48 @@ private[sql] class DefaultWriterContainer( extends BaseWriterContainer(relation, job, isAppend) { def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - if (iterator.hasNext) { - executorSideSetup(taskContext) - val configuration = taskAttemptContext.getConfiguration - configuration.set("spark.sql.sources.output.path", outputPath) - var writer = newOutputWriter(getWorkPath) - writer.initConverter(dataSchema) + executorSideSetup(taskContext) + val configuration = taskAttemptContext.getConfiguration + configuration.set("spark.sql.sources.output.path", outputPath) + var writer = newOutputWriter(getWorkPath) + writer.initConverter(dataSchema) - // If anything below fails, we should abort the task. - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - while (iterator.hasNext) { - val internalRow = iterator.next() - writer.writeInternal(internalRow) - } - commitTask() - }(catchBlock = abortTask()) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } + // If anything below fails, we should abort the task. + try { + Utils.tryWithSafeFinallyAndFailureCallbacks { + while (iterator.hasNext) { + val internalRow = iterator.next() + writer.writeInternal(internalRow) + } + commitTask() + }(catchBlock = abortTask()) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } - def commitTask(): Unit = { - try { - if (writer != null) { - writer.close() - writer = null - } - super.commitTask() - } catch { - case cause: Throwable => - // This exception will be handled in `InsertIntoHadoopFsRelation.insert$writeRows`, and - // will cause `abortTask()` to be invoked. - throw new RuntimeException("Failed to commit task", cause) + def commitTask(): Unit = { + try { + if (writer != null) { + writer.close() + writer = null } + super.commitTask() + } catch { + case cause: Throwable => + // This exception will be handled in `InsertIntoHadoopFsRelation.insert$writeRows`, and + // will cause `abortTask()` to be invoked. + throw new RuntimeException("Failed to commit task", cause) } + } - def abortTask(): Unit = { - try { - if (writer != null) { - writer.close() - } - } finally { - super.abortTask() + def abortTask(): Unit = { + try { + if (writer != null) { + writer.close() } + } finally { + super.abortTask() } } } @@ -365,87 +363,84 @@ private[sql] class DynamicPartitionWriterContainer( } def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - if (iterator.hasNext) { - executorSideSetup(taskContext) - - // We should first sort by partition columns, then bucket id, and finally sorting columns. - val sortingExpressions: Seq[Expression] = - partitionColumns ++ bucketIdExpression ++ sortColumns - val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) - - val sortingKeySchema = StructType(sortingExpressions.map { - case a: Attribute => StructField(a.name, a.dataType, a.nullable) - // The sorting expressions are all `Attribute` except bucket id. - case _ => StructField("bucketId", IntegerType, nullable = false) - }) + executorSideSetup(taskContext) + + // We should first sort by partition columns, then bucket id, and finally sorting columns. + val sortingExpressions: Seq[Expression] = partitionColumns ++ bucketIdExpression ++ sortColumns + val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) + + val sortingKeySchema = StructType(sortingExpressions.map { + case a: Attribute => StructField(a.name, a.dataType, a.nullable) + // The sorting expressions are all `Attribute` except bucket id. + case _ => StructField("bucketId", IntegerType, nullable = false) + }) + + // Returns the data columns to be written given an input row + val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) + + // Returns the partition path given a partition key. + val getPartitionString = + UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) + + // Sorts the data before write, so that we only need one writer at the same time. + // TODO: inject a local sort operator in planning. + val sorter = new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(dataColumns), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + logInfo(s"Sorting complete. Writing out partition files one at a time.") - // Returns the data columns to be written given an input row - val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) - - // Returns the partition path given a partition key. - val getPartitionString = - UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) - - // Sorts the data before write, so that we only need one writer at the same time. - // TODO: inject a local sort operator in planning. - val sorter = new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) - - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) - } - logInfo(s"Sorting complete. Writing out partition files one at a time.") - - val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { - identity - } else { - UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { - case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) - }) - } + val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { + identity + } else { + UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { + case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + }) + } - val sortedIterator = sorter.sortedIterator() + val sortedIterator = sorter.sortedIterator() - // If anything below fails, we should abort the task. - var currentWriter: OutputWriter = null - try { - Utils.tryWithSafeFinallyAndFailureCallbacks { - var currentKey: UnsafeRow = null - while (sortedIterator.next()) { - val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] - if (currentKey != nextKey) { - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - currentKey = nextKey.copy() - logDebug(s"Writing partition: $currentKey") - - currentWriter = newOutputWriter(currentKey, getPartitionString) + // If anything below fails, we should abort the task. + var currentWriter: OutputWriter = null + try { + Utils.tryWithSafeFinallyAndFailureCallbacks { + var currentKey: UnsafeRow = null + while (sortedIterator.next()) { + val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null } - currentWriter.writeInternal(sortedIterator.getValue) - } - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } + currentKey = nextKey.copy() + logDebug(s"Writing partition: $currentKey") - commitTask() - }(catchBlock = { - if (currentWriter != null) { - currentWriter.close() + currentWriter = newOutputWriter(currentKey, getPartitionString) } - abortTask() - }) - } catch { - case t: Throwable => - throw new SparkException("Task failed while writing rows", t) - } + currentWriter.writeInternal(sortedIterator.getValue) + } + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + + commitTask() + }(catchBlock = { + if (currentWriter != null) { + currentWriter.close() + } + abortTask() + }) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 706fdbc2604fe..794fe264ead5d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -178,21 +178,19 @@ private[hive] class SparkHiveWriterContainer( // this function is executed on executor side def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { - if (iterator.hasNext) { - val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() - executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - - iterator.foreach { row => - var i = 0 - while (i < fieldOIs.length) { - outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) - i += 1 - } - writer.write(serializer.serialize(outputData, standardOI)) - } + val (serializer, standardOI, fieldOIs, dataTypes, wrappers, outputData) = prepareForWrite() + executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - close() + iterator.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) + i += 1 + } + writer.write(serializer.serialize(outputData, standardOI)) } + + close() } } 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 b25684562075c..4a55bcc3b19d0 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 @@ -19,13 +19,13 @@ package org.apache.spark.sql.hive import java.io.File +import org.apache.hadoop.hive.conf.HiveConf import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException -import org.apache.spark.sql._ +import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -118,10 +118,10 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef 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) + |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 @@ -216,35 +216,6 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef sql("DROP TABLE hiveTableWithStructValue") } - test("SPARK-10216: Avoid empty files during overwrite into Hive table with group by query") { - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { - val testDataset = hiveContext.sparkContext.parallelize( - (1 to 2).map(i => TestData(i, i.toString))).toDF() - testDataset.createOrReplaceTempView("testDataset") - - val tmpDir = Utils.createTempDir() - sql( - s""" - |CREATE TABLE table1(key int,value string) - |location '${tmpDir.toURI.toString}' - """.stripMargin) - sql( - """ - |INSERT OVERWRITE TABLE table1 - |SELECT count(key), value FROM testDataset GROUP BY value - """.stripMargin) - - val overwrittenFiles = tmpDir.listFiles() - .filter(f => f.isFile && !f.getName.endsWith(".crc")) - .sortBy(_.getName) - val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0) - - assert(overwrittenFiles === overwrittenFilesWithoutEmpty) - - sql("DROP TABLE table1") - } - } - test("Reject partitioning that does not match table") { withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index a3183f2977223..62998572eaf4b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.execution.DataSourceScanExec -import org.apache.spark.sql.execution.datasources.{FileScanRDD, LocalityTestFileSystem} +import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -879,26 +879,6 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } } } - - test("SPARK-10216: Avoid empty files during overwriting with group by query") { - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { - withTempPath { path => - val df = spark.range(0, 5) - val groupedDF = df.groupBy("id").count() - groupedDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .save(path.getCanonicalPath) - - val overwrittenFiles = path.listFiles() - .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) - .sortBy(_.getName) - val overwrittenFilesWithoutEmpty = overwrittenFiles.filter(_.length > 0) - - assert(overwrittenFiles === overwrittenFilesWithoutEmpty) - } - } - } } // This class is used to test SPARK-8578. We should not use any custom output committer when From 89e29870bb73dac9dfebd3c3663320e4fdc6c03a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 20 May 2016 13:21:53 -0700 Subject: [PATCH 0325/1470] [SPARK-15438][SQL] improve explain of whole stage codegen ## What changes were proposed in this pull request? Currently, the explain of a query with whole-stage codegen looks like this ``` >>> df = sqlCtx.range(1000);df2 = sqlCtx.range(1000);df.join(pyspark.sql.functions.broadcast(df2), 'id').explain() == Physical Plan == WholeStageCodegen : +- Project [id#1L] : +- BroadcastHashJoin [id#1L], [id#4L], Inner, BuildRight, None : :- Range 0, 1, 4, 1000, [id#1L] : +- INPUT +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint])) +- WholeStageCodegen : +- Range 0, 1, 4, 1000, [id#4L] ``` The problem is that the plan looks much different than logical plan, make us hard to understand the plan (especially when the logical plan is not showed together). This PR will change it to: ``` >>> df = sqlCtx.range(1000);df2 = sqlCtx.range(1000);df.join(pyspark.sql.functions.broadcast(df2), 'id').explain() == Physical Plan == *Project [id#0L] +- *BroadcastHashJoin [id#0L], [id#3L], Inner, BuildRight, None :- *Range 0, 1, 4, 1000, [id#0L] +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false])) +- *Range 0, 1, 4, 1000, [id#3L] ``` The `*`before the plan means that it's part of whole-stage codegen, it's easy to understand. ## How was this patch tested? Manually ran some queries and check the explain. Author: Davies Liu Closes #13204 from davies/explain_codegen. (cherry picked from commit 0e70fd61b4bc92bd744fc44dd3cbe91443207c72) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/trees/TreeNode.scala | 57 +++---------------- .../sql/execution/WholeStageCodegenExec.scala | 29 +++++----- .../sql/execution/exchange/Exchange.scala | 3 - 3 files changed, 22 insertions(+), 67 deletions(-) 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 5eb8fdf048e28..e8e2a7bbabcd4 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 @@ -467,50 +467,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } /** - * All the nodes that will be used to generate tree string. - * - * For example: - * - * WholeStageCodegen - * +-- SortMergeJoin - * |-- InputAdapter - * | +-- Sort - * +-- InputAdapter - * +-- Sort - * - * the treeChildren of WholeStageCodegen will be Seq(Sort, Sort), it will generate a tree string - * like this: - * - * WholeStageCodegen - * : +- SortMergeJoin - * : :- INPUT - * : :- INPUT - * :- Sort - * :- Sort - */ - protected def treeChildren: Seq[BaseType] = children - - /** - * All the nodes that are parts of this node. - * - * For example: - * - * WholeStageCodegen - * +- SortMergeJoin - * |-- InputAdapter - * | +-- Sort - * +-- InputAdapter - * +-- Sort - * - * the innerChildren of WholeStageCodegen will be Seq(SortMergeJoin), it will generate a tree - * string like this: - * - * WholeStageCodegen - * : +- SortMergeJoin - * : :- INPUT - * : :- INPUT - * :- Sort - * :- Sort + * All the nodes that are parts of this node, this is used by subquries. */ protected def innerChildren: Seq[BaseType] = Nil @@ -522,7 +479,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { * `lastChildren` for the root node should be empty. */ def generateTreeString( - depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder): StringBuilder = { + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + prefix: String = ""): StringBuilder = { if (depth > 0) { lastChildren.init.foreach { isLast => val prefixFragment = if (isLast) " " else ": " @@ -533,6 +493,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { builder.append(branch) } + builder.append(prefix) builder.append(simpleString) builder.append("\n") @@ -542,9 +503,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { innerChildren.last.generateTreeString(depth + 2, lastChildren :+ false :+ true, builder) } - if (treeChildren.nonEmpty) { - treeChildren.init.foreach(_.generateTreeString(depth + 1, lastChildren :+ false, builder)) - treeChildren.last.generateTreeString(depth + 1, lastChildren :+ true, builder) + if (children.nonEmpty) { + children.init.foreach(_.generateTreeString(depth + 1, lastChildren :+ false, builder, prefix)) + children.last.generateTreeString(depth + 1, lastChildren :+ true, builder, prefix) } builder diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 37fdc362b5e4e..2a1ce735b74ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -245,9 +245,13 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp """.stripMargin } - override def simpleString: String = "INPUT" - - override def treeChildren: Seq[SparkPlan] = Nil + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + prefix: String = ""): StringBuilder = { + child.generateTreeString(depth, lastChildren, builder, "") + } } object WholeStageCodegenExec { @@ -398,20 +402,13 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co """.stripMargin.trim } - override def innerChildren: Seq[SparkPlan] = { - child :: Nil - } - - private def collectInputs(plan: SparkPlan): Seq[SparkPlan] = plan match { - case InputAdapter(c) => c :: Nil - case other => other.children.flatMap(collectInputs) + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + builder: StringBuilder, + prefix: String = ""): StringBuilder = { + child.generateTreeString(depth, lastChildren, builder, "*") } - - override def treeChildren: Seq[SparkPlan] = { - collectInputs(child) - } - - override def simpleString: String = "WholeStageCodegen" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala index 9da9df617405d..9a9597d3733e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala @@ -60,9 +60,6 @@ case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchan override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { child.executeBroadcast() } - - // Do not repeat the same tree in explain. - override def treeChildren: Seq[SparkPlan] = Nil } /** From ea92f7e7e5fbc28747b528c8c60982f8ce7f739f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 May 2016 14:49:54 -0700 Subject: [PATCH 0326/1470] [SPARK-15454][SQL] Filter out files starting with _ ## What changes were proposed in this pull request? Many other systems (e.g. Impala) uses _xxx as staging, and Spark should not be reading those files. ## How was this patch tested? Added a unit test case. Author: Reynold Xin Closes #13227 from rxin/SPARK-15454. (cherry picked from commit dcac8e6f49918a809fb3f2b8bf666582c479a6eb) Signed-off-by: Reynold Xin --- .../execution/datasources/fileSourceInterfaces.scala | 10 +++++----- .../execution/datasources/HadoopFsRelationSuite.scala | 11 +++++++++++ 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 88125a2b4da78..e0e569bca48b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -341,11 +341,11 @@ private[sql] object HadoopFsRelation extends Logging { /** Checks if we should filter out this path name. */ def shouldFilterOut(pathName: String): Boolean = { - // TODO: We should try to filter out all files/dirs starting with "." or "_". - // The only reason that we are not doing it now is that Parquet needs to find those - // metadata files from leaf files returned by this methods. We should refactor - // this logic to not mix metadata files with data files. - pathName == "_SUCCESS" || pathName == "_temporary" || pathName.startsWith(".") + // We filter everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + (pathName.startsWith("_") || pathName.startsWith(".")) && + !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala index 89d57653adcbd..3c68dc8bb98d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala @@ -39,4 +39,15 @@ class HadoopFsRelationSuite extends QueryTest with SharedSQLContext { assert(df.queryExecution.logical.statistics.sizeInBytes === BigInt(totalSize)) } } + + test("file filtering") { + assert(!HadoopFsRelation.shouldFilterOut("abcd")) + assert(HadoopFsRelation.shouldFilterOut(".ab")) + assert(HadoopFsRelation.shouldFilterOut("_cd")) + + assert(!HadoopFsRelation.shouldFilterOut("_metadata")) + assert(!HadoopFsRelation.shouldFilterOut("_common_metadata")) + assert(HadoopFsRelation.shouldFilterOut("_ab_metadata")) + assert(HadoopFsRelation.shouldFilterOut("_cd_common_metadata")) + } } From f31c24a6afd964bdf6dd334454e50bbd64106290 Mon Sep 17 00:00:00 2001 From: tedyu Date: Fri, 20 May 2016 18:13:18 -0500 Subject: [PATCH 0327/1470] [SPARK-15273] YarnSparkHadoopUtil#getOutOfMemoryErrorArgument should respect OnOutOfMemoryError parameter given by user ## What changes were proposed in this pull request? As Nirav reported in this thread: http://search-hadoop.com/m/q3RTtdF3yNLMd7u YarnSparkHadoopUtil#getOutOfMemoryErrorArgument previously specified 'kill %p' unconditionally. We should respect the parameter given by user. ## How was this patch tested? Existing tests Author: tedyu Closes #13057 from tedyu/master. (cherry picked from commit 06c9f520714e07259c6f8ce6f9ea5a230a278cb5) Signed-off-by: Sean Owen --- .../spark/deploy/yarn/ExecutorRunnable.scala | 9 ++------ .../deploy/yarn/YarnSparkHadoopUtil.scala | 23 ++++++++++++------- 2 files changed, 17 insertions(+), 15 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 3d370e6d71426..fc753b7e750cb 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 @@ -211,15 +211,10 @@ private[yarn] class ExecutorRunnable( Seq("--user-class-path", "file:" + absPath) }.toSeq + YarnSparkHadoopUtil.addOutOfMemoryErrorArgument(javaOpts) val commands = prefixEnv ++ Seq( YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", - "-server", - // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in - // an inconsistent state. - // TODO: If the OOM is not recoverable by rescheduling it on different node, then do - // 'something' to fail job ... akin to blacklisting trackers in mapred ? - YarnSparkHadoopUtil.getOutOfMemoryErrorArgument) ++ + "-server") ++ javaOpts ++ Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", "--driver-url", masterAddress.toString, diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 44181610d770d..de6cd946137cb 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -24,7 +24,8 @@ import java.security.PrivilegedExceptionAction import java.util.regex.Matcher import java.util.regex.Pattern -import scala.collection.mutable.HashMap +import scala.collection.JavaConverters._ +import scala.collection.mutable.{HashMap, ListBuffer} import scala.reflect.runtime._ import scala.util.Try @@ -405,6 +406,12 @@ object YarnSparkHadoopUtil { } /** + * Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. + * Not killing the task leaves various aspects of the executor and (to some extent) the jvm in + * an inconsistent state. + * TODO: If the OOM is not recoverable by rescheduling it on different node, then do + * 'something' to fail job ... akin to blacklisting trackers in mapred ? + * * The handler if an OOM Exception is thrown by the JVM must be configured on Windows * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'. * @@ -415,14 +422,14 @@ object YarnSparkHadoopUtil { * the behavior of '%' in a .cmd file: it gets interpreted as an incomplete environment * variable. Windows .cmd files escape a '%' by '%%'. Thus, the correct way of writing * '%%p' in an escaped way is '%%%%p'. - * - * @return The correct OOM Error handler JVM option, platform dependent. */ - def getOutOfMemoryErrorArgument: String = { - if (Utils.isWindows) { - escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") - } else { - "-XX:OnOutOfMemoryError='kill %p'" + private[yarn] def addOutOfMemoryErrorArgument(javaOpts: ListBuffer[String]): Unit = { + if (!javaOpts.exists(_.contains("-XX:OnOutOfMemoryError"))) { + if (Utils.isWindows) { + javaOpts += escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") + } else { + javaOpts += "-XX:OnOutOfMemoryError='kill %p'" + } } } From 68416786206d7f6c48f6517ff8da1e76927f0713 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 20 May 2016 16:40:33 -0700 Subject: [PATCH 0328/1470] [SPARK-15031][EXAMPLE] Use SparkSession in examples ## What changes were proposed in this pull request? Use `SparkSession` according to [SPARK-15031](https://issues.apache.org/jira/browse/SPARK-15031) `MLLLIB` is not recommended to use now, so examples in `MLLIB` are ignored in this PR. `StreamingContext` can not be directly obtained from `SparkSession`, so example in `Streaming` are ignored too. cc andrewor14 ## How was this patch tested? manual tests with spark-submit Author: Zheng RuiFeng Closes #13164 from zhengruifeng/use_sparksession_ii. (cherry picked from commit 127bf1bb07967e2e4f99ad7abaa7f6fab3b3f407) Signed-off-by: Andrew Or --- .../org/apache/spark/examples/JavaHdfsLR.java | 14 ++++++++------ .../apache/spark/examples/JavaLogQuery.java | 12 ++++++++---- .../apache/spark/examples/JavaPageRank.java | 13 +++++++------ .../apache/spark/examples/JavaSparkPi.java | 12 ++++++++---- .../spark/examples/JavaStatusTrackerDemo.java | 19 ++++++++++++------- .../org/apache/spark/examples/JavaTC.java | 15 ++++++++++----- .../apache/spark/examples/JavaWordCount.java | 15 +++++++++------ examples/src/main/python/als.py | 12 +++++++++--- examples/src/main/python/avro_inputformat.py | 12 +++++++++--- examples/src/main/python/kmeans.py | 12 ++++++++---- .../src/main/python/logistic_regression.py | 13 +++++++++---- examples/src/main/python/pagerank.py | 11 +++++++---- .../src/main/python/parquet_inputformat.py | 12 +++++++++--- examples/src/main/python/pi.py | 12 +++++++++--- examples/src/main/python/sort.py | 13 +++++++++---- .../src/main/python/transitive_closure.py | 12 +++++++++--- examples/src/main/python/wordcount.py | 13 +++++++++---- .../apache/spark/examples/BroadcastTest.scala | 16 ++++++++++++---- .../spark/examples/DFSReadWriteTest.scala | 13 ++++++++----- .../examples/ExceptionHandlingTest.scala | 12 ++++++++---- .../apache/spark/examples/GroupByTest.scala | 12 ++++++++---- .../org/apache/spark/examples/HdfsTest.scala | 12 +++++++----- .../spark/examples/MultiBroadcastTest.scala | 13 +++++++++---- .../examples/SimpleSkewedGroupByTest.scala | 13 ++++++++----- .../spark/examples/SkewedGroupByTest.scala | 13 +++++++++---- .../org/apache/spark/examples/SparkALS.scala | 12 ++++++++---- .../apache/spark/examples/SparkHdfsLR.scala | 16 +++++++++------- .../apache/spark/examples/SparkKMeans.scala | 15 +++++++++------ .../org/apache/spark/examples/SparkLR.scala | 13 +++++++++---- .../apache/spark/examples/SparkPageRank.scala | 13 ++++++++----- .../org/apache/spark/examples/SparkPi.scala | 11 +++++++---- .../org/apache/spark/examples/SparkTC.scala | 11 +++++++---- .../examples/sql/hive/HiveFromSpark.scala | 2 +- .../apache/spark/sql/JavaDataFrameSuite.java | 6 +++--- 34 files changed, 279 insertions(+), 146 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index f64155ce3c0cf..ded442096c257 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -17,11 +17,10 @@ package org.apache.spark.examples; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; +import org.apache.spark.sql.SparkSession; import java.io.Serializable; import java.util.Arrays; @@ -122,9 +121,12 @@ public static void main(String[] args) { showWarning(); - SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - JavaRDD lines = sc.textFile(args[0]); + SparkSession spark = SparkSession + .builder() + .appName("JavaHdfsLR") + .getOrCreate(); + + JavaRDD lines = spark.read().text(args[0]).javaRDD(); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[1]); @@ -152,6 +154,6 @@ public static void main(String[] args) { System.out.print("Final w: "); printWeights(w); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index ebb0687b14ae0..7775443861661 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -20,12 +20,13 @@ import com.google.common.collect.Lists; import scala.Tuple2; import scala.Tuple3; -import org.apache.spark.SparkConf; + import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; import java.io.Serializable; import java.util.List; @@ -99,9 +100,12 @@ public static Stats extractStats(String line) { } public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaLogQuery") + .getOrCreate(); - SparkConf sparkConf = new SparkConf().setAppName("JavaLogQuery"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); JavaRDD dataSet = (args.length == 1) ? jsc.textFile(args[0]) : jsc.parallelize(exampleApacheLogs); @@ -123,6 +127,6 @@ public Stats call(Stats stats, Stats stats2) { for (Tuple2 t : output) { System.out.println(t._1() + "\t" + t._2()); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 229d1234414e5..128b5ab17c8d4 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -26,14 +26,13 @@ import com.google.common.collect.Iterables; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; /** * Computes the PageRank of URLs from an input file. Input file should @@ -73,15 +72,17 @@ public static void main(String[] args) throws Exception { showWarning(); - SparkConf sparkConf = new SparkConf().setAppName("JavaPageRank"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaPageRank") + .getOrCreate(); // Loads in input file. It should be in format of: // URL neighbor URL // URL neighbor URL // URL neighbor URL // ... - JavaRDD lines = ctx.textFile(args[0], 1); + JavaRDD lines = spark.read().text(args[0]).javaRDD(); // Loads all URLs from input file and initialize their neighbors. JavaPairRDD> links = lines.mapToPair( @@ -132,6 +133,6 @@ public Double call(Double sum) { System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 04a57a6bfb58b..7df145e3117b8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -17,11 +17,11 @@ package org.apache.spark.examples; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; +import org.apache.spark.sql.SparkSession; import java.util.ArrayList; import java.util.List; @@ -33,8 +33,12 @@ public final class JavaSparkPi { public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName("JavaSparkPi"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaSparkPi") + .getOrCreate(); + + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); int slices = (args.length == 1) ? Integer.parseInt(args[0]) : 2; int n = 100000 * slices; @@ -61,6 +65,6 @@ public Integer call(Integer integer, Integer integer2) { System.out.println("Pi is roughly " + 4.0 * count / n); - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java index e68ec74c3ed54..6f899c772eb98 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java @@ -17,13 +17,14 @@ package org.apache.spark.examples; -import org.apache.spark.SparkConf; import org.apache.spark.SparkJobInfo; import org.apache.spark.SparkStageInfo; import org.apache.spark.api.java.JavaFutureAction; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; +import org.apache.spark.sql.SparkSession; + import java.util.Arrays; import java.util.List; @@ -44,11 +45,15 @@ public T call(T x) throws Exception { } public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName(APP_NAME); - final JavaSparkContext sc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName(APP_NAME) + .getOrCreate(); + + final JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); // Example of implementing a progress reporter for a simple job. - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( + JavaRDD rdd = jsc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( new IdentityWithDelay()); JavaFutureAction> jobFuture = rdd.collectAsync(); while (!jobFuture.isDone()) { @@ -58,13 +63,13 @@ public static void main(String[] args) throws Exception { continue; } int currentJobId = jobIds.get(jobIds.size() - 1); - SparkJobInfo jobInfo = sc.statusTracker().getJobInfo(currentJobId); - SparkStageInfo stageInfo = sc.statusTracker().getStageInfo(jobInfo.stageIds()[0]); + SparkJobInfo jobInfo = jsc.statusTracker().getJobInfo(currentJobId); + SparkStageInfo stageInfo = jsc.statusTracker().getStageInfo(jobInfo.stageIds()[0]); System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() + " active, " + stageInfo.numCompletedTasks() + " complete"); } System.out.println("Job results are: " + jobFuture.get()); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index ca10384212da2..f12ca77ed1eb0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -25,10 +25,10 @@ import scala.Tuple2; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; /** * Transitive closure on a graph, implemented in Java. @@ -64,10 +64,15 @@ public Tuple2 call(Tuple2> t } public static void main(String[] args) { - SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaTC") + .getOrCreate(); + + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); + Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; - JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); + JavaPairRDD tc = jsc.parallelizePairs(generateGraph(), slices).cache(); // Linear transitive closure: each round grows paths by one edge, // by joining the graph's edges with the already-discovered paths. @@ -94,6 +99,6 @@ public Tuple2 call(Tuple2 e) { } while (nextCount != oldCount); System.out.println("TC has " + tc.count() + " edges."); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 3ff5412b934f0..1caee60e348d2 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -18,13 +18,13 @@ package org.apache.spark.examples; import scala.Tuple2; -import org.apache.spark.SparkConf; + import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; import java.util.Arrays; import java.util.Iterator; @@ -41,9 +41,12 @@ public static void main(String[] args) throws Exception { System.exit(1); } - SparkConf sparkConf = new SparkConf().setAppName("JavaWordCount"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); - JavaRDD lines = ctx.textFile(args[0], 1); + SparkSession spark = SparkSession + .builder() + .appName("JavaWordCount") + .getOrCreate(); + + JavaRDD lines = spark.read().text(args[0]).javaRDD(); JavaRDD words = lines.flatMap(new FlatMapFunction() { @Override @@ -72,6 +75,6 @@ public Integer call(Integer i1, Integer i2) { for (Tuple2 tuple : output) { System.out.println(tuple._1() + ": " + tuple._2()); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index f07020b50397b..81562e20a927d 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -28,7 +28,7 @@ import numpy as np from numpy.random import rand from numpy import matrix -from pyspark import SparkContext +from pyspark.sql import SparkSession LAMBDA = 0.01 # regularization np.random.seed(42) @@ -62,7 +62,13 @@ def update(i, vec, mat, ratings): example. Please use pyspark.ml.recommendation.ALS for more conventional use.""", file=sys.stderr) - sc = SparkContext(appName="PythonALS") + spark = SparkSession\ + .builder\ + .appName("PythonALS")\ + .getOrCreate() + + sc = spark._sc + M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 F = int(sys.argv[3]) if len(sys.argv) > 3 else 10 @@ -99,4 +105,4 @@ def update(i, vec, mat, ratings): print("Iteration %d:" % i) print("\nRMSE: %5.4f\n" % error) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index da368ac628a49..3f65e8f79a65c 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -19,8 +19,8 @@ import sys -from pyspark import SparkContext from functools import reduce +from pyspark.sql import SparkSession """ Read data file users.avro in local Spark distro: @@ -64,7 +64,13 @@ exit(-1) path = sys.argv[1] - sc = SparkContext(appName="AvroKeyInputFormat") + + spark = SparkSession\ + .builder\ + .appName("AvroKeyInputFormat")\ + .getOrCreate() + + sc = spark._sc conf = None if len(sys.argv) == 3: @@ -82,4 +88,4 @@ for k in output: print(k) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 3426e491dc74a..92e0a3ae2ee60 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -27,7 +27,7 @@ import sys import numpy as np -from pyspark import SparkContext +from pyspark.sql import SparkSession def parseVector(line): @@ -55,8 +55,12 @@ def closestPoint(p, centers): as an example! Please refer to examples/src/main/python/ml/kmeans_example.py for an example on how to use ML's KMeans implementation.""", file=sys.stderr) - sc = SparkContext(appName="PythonKMeans") - lines = sc.textFile(sys.argv[1]) + spark = SparkSession\ + .builder\ + .appName("PythonKMeans")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) data = lines.map(parseVector).cache() K = int(sys.argv[2]) convergeDist = float(sys.argv[3]) @@ -79,4 +83,4 @@ def closestPoint(p, centers): print("Final centers: " + str(kPoints)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 7d33be7e81d73..01c938454b108 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -27,7 +27,7 @@ import sys import numpy as np -from pyspark import SparkContext +from pyspark.sql import SparkSession D = 10 # Number of dimensions @@ -55,8 +55,13 @@ def readPointBatch(iterator): Please refer to examples/src/main/python/ml/logistic_regression_with_elastic_net.py to see how ML's implementation is used.""", file=sys.stderr) - sc = SparkContext(appName="PythonLR") - points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() + spark = SparkSession\ + .builder\ + .appName("PythonLR")\ + .getOrCreate() + + points = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0])\ + .mapPartitions(readPointBatch).cache() iterations = int(sys.argv[2]) # Initialize w to a random value @@ -80,4 +85,4 @@ def add(x, y): print("Final w: " + str(w)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index 2fdc9773d4eb1..a399a9c37c5d5 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -25,7 +25,7 @@ import sys from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession def computeContribs(urls, rank): @@ -51,14 +51,17 @@ def parseNeighbors(urls): file=sys.stderr) # Initialize the spark context. - sc = SparkContext(appName="PythonPageRank") + spark = SparkSession\ + .builder\ + .appName("PythonPageRank")\ + .getOrCreate() # Loads in input file. It should be in format of: # URL neighbor URL # URL neighbor URL # URL neighbor URL # ... - lines = sc.textFile(sys.argv[1], 1) + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) # Loads all URLs from input file and initialize their neighbors. links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() @@ -79,4 +82,4 @@ def parseNeighbors(urls): for (link, rank) in ranks.collect(): print("%s has rank: %s." % (link, rank)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index e1fd85b082c08..2f09f4d5730e8 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -18,7 +18,7 @@ import sys -from pyspark import SparkContext +from pyspark.sql import SparkSession """ Read data file users.parquet in local Spark distro: @@ -47,7 +47,13 @@ exit(-1) path = sys.argv[1] - sc = SparkContext(appName="ParquetInputFormat") + + spark = SparkSession\ + .builder\ + .appName("ParquetInputFormat")\ + .getOrCreate() + + sc = spark._sc parquet_rdd = sc.newAPIHadoopFile( path, @@ -59,4 +65,4 @@ for k in output: print(k) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 92e5cf45abc8b..5db03e4a21091 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -20,14 +20,20 @@ from random import random from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": """ Usage: pi [partitions] """ - sc = SparkContext(appName="PythonPi") + spark = SparkSession\ + .builder\ + .appName("PythonPi")\ + .getOrCreate() + + sc = spark._sc + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * partitions @@ -39,4 +45,4 @@ def f(_): count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) print("Pi is roughly %f" % (4.0 * count / n)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index b6c2916254056..81898cf6d5ce6 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -19,15 +19,20 @@ import sys -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": if len(sys.argv) != 2: print("Usage: sort ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="PythonSort") - lines = sc.textFile(sys.argv[1], 1) + + spark = SparkSession\ + .builder\ + .appName("PythonSort")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) sortedCount = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (int(x), 1)) \ .sortByKey() @@ -37,4 +42,4 @@ for (num, unitcount) in output: print(num) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 3d61250d8b230..37c41dcd03dc6 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -20,7 +20,7 @@ import sys from random import Random -from pyspark import SparkContext +from pyspark.sql import SparkSession numEdges = 200 numVertices = 100 @@ -41,7 +41,13 @@ def generateGraph(): """ Usage: transitive_closure [partitions] """ - sc = SparkContext(appName="PythonTransitiveClosure") + spark = SparkSession\ + .builder\ + .appName("PythonTransitiveClosure")\ + .getOrCreate() + + sc = spark._sc + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 tc = sc.parallelize(generateGraph(), partitions).cache() @@ -67,4 +73,4 @@ def generateGraph(): print("TC has %i edges" % tc.count()) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index 7c0143607b61d..3d5e44d5b2df1 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -20,15 +20,20 @@ import sys from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": if len(sys.argv) != 2: print("Usage: wordcount ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="PythonWordCount") - lines = sc.textFile(sys.argv[1], 1) + + spark = SparkSession\ + .builder\ + .appName("PythonWordCount")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) counts = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (x, 1)) \ .reduceByKey(add) @@ -36,4 +41,4 @@ for (word, count) in output: print("%s: %i" % (word, count)) - sc.stop() + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index af5a815f6ec76..c50f25d951947 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -18,7 +18,8 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession /** * Usage: BroadcastTest [slices] [numElem] [blockSize] @@ -28,9 +29,16 @@ object BroadcastTest { val blockSize = if (args.length > 2) args(2) else "4096" - val sparkConf = new SparkConf().setAppName("Broadcast Test") + val sparkConf = new SparkConf() .set("spark.broadcast.blockSize", blockSize) - val sc = new SparkContext(sparkConf) + + val spark = SparkSession + .builder + .config(sparkConf) + .appName("Broadcast Test") + .getOrCreate() + + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 @@ -48,7 +56,7 @@ object BroadcastTest { println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6)) } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index 7bf023667dcae..4b5e36c73642d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -22,7 +22,7 @@ import java.io.File import scala.io.Source._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Simple test for reading and writing to a distributed @@ -101,11 +101,14 @@ object DFSReadWriteTest { val fileContents = readFile(localFilePath.toString()) val localWordCount = runLocalWordCount(fileContents) - println("Creating SparkConf") - val conf = new SparkConf().setAppName("DFS Read Write Test") + println("Creating SparkSession") + val spark = SparkSession + .builder + .appName("DFS Read Write Test") + .getOrCreate() println("Creating SparkContext") - val sc = new SparkContext(conf) + val sc = spark.sparkContext println("Writing local file to DFS") val dfsFilename = dfsDirPath + "/dfs_read_write_test" @@ -124,7 +127,7 @@ object DFSReadWriteTest { .values .sum - sc.stop() + spark.stop() if (localWordCount == dfsWordCount) { println(s"Success! Local Word Count ($localWordCount) " + diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index d42f63e87052e..6a1bbed290235 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -17,18 +17,22 @@ package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession object ExceptionHandlingTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("ExceptionHandlingTest") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("ExceptionHandlingTest") + .getOrCreate() + val sc = spark.sparkContext + sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") } } - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 4db229b5dec32..0cb61d7495fa6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -20,20 +20,24 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] */ object GroupByTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("GroupBy Test") + val spark = SparkSession + .builder + .appName("GroupBy Test") + .getOrCreate() + var numMappers = if (args.length > 0) args(0).toInt else 2 var numKVPairs = if (args.length > 1) args(1).toInt else 1000 var valSize = if (args.length > 2) args(2).toInt else 1000 var numReducers = if (args.length > 3) args(3).toInt else numMappers - val sc = new SparkContext(sparkConf) + val sc = spark.sparkContext val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random @@ -50,7 +54,7 @@ object GroupByTest { println(pairs1.groupByKey(numReducers).count()) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index 124dc9af6390f..aa8de69839e28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -18,7 +18,7 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark._ +import org.apache.spark.sql.SparkSession object HdfsTest { @@ -29,9 +29,11 @@ object HdfsTest { System.err.println("Usage: HdfsTest ") System.exit(1) } - val sparkConf = new SparkConf().setAppName("HdfsTest") - val sc = new SparkContext(sparkConf) - val file = sc.textFile(args(0)) + val spark = SparkSession + .builder + .appName("HdfsTest") + .getOrCreate() + val file = spark.read.text(args(0)).rdd val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { val start = System.currentTimeMillis() @@ -39,7 +41,7 @@ object HdfsTest { val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 3eb0c2772337a..961ab99200d29 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -18,8 +18,9 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession + /** * Usage: MultiBroadcastTest [slices] [numElem] @@ -27,8 +28,12 @@ import org.apache.spark.rdd.RDD object MultiBroadcastTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("Multi-Broadcast Test") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("Multi-Broadcast Test") + .getOrCreate() + + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 @@ -51,7 +56,7 @@ object MultiBroadcastTest { // Collect the small RDD so we can print the observed sizes locally. observedSizes.collect().foreach(i => println(i)) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index ec07e6323ee9a..255c2bfcee669 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -20,23 +20,26 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Usage: SimpleSkewedGroupByTest [numMappers] [numKVPairs] [valSize] [numReducers] [ratio] */ object SimpleSkewedGroupByTest { def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("SimpleSkewedGroupByTest") + .getOrCreate() + + val sc = spark.sparkContext - val sparkConf = new SparkConf().setAppName("SimpleSkewedGroupByTest") var numMappers = if (args.length > 0) args(0).toInt else 2 var numKVPairs = if (args.length > 1) args(1).toInt else 1000 var valSize = if (args.length > 2) args(2).toInt else 1000 var numReducers = if (args.length > 3) args(3).toInt else numMappers var ratio = if (args.length > 4) args(4).toInt else 5.0 - val sc = new SparkContext(sparkConf) - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random var result = new Array[(Int, Array[Byte])](numKVPairs) @@ -64,7 +67,7 @@ object SimpleSkewedGroupByTest { // .map{case (k,v) => (k, v.size)} // .collectAsMap) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 8e4c2b6229755..efd40147f7ca7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -20,20 +20,25 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] */ object SkewedGroupByTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("GroupBy Test") + val spark = SparkSession + .builder + .appName("GroupBy Test") + .getOrCreate() + + val sc = spark.sparkContext + var numMappers = if (args.length > 0) args(0).toInt else 2 var numKVPairs = if (args.length > 1) args(1).toInt else 1000 var valSize = if (args.length > 2) args(2).toInt else 1000 var numReducers = if (args.length > 3) args(3).toInt else numMappers - val sc = new SparkContext(sparkConf) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random @@ -54,7 +59,7 @@ object SkewedGroupByTest { println(pairs1.groupByKey(numReducers).count()) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index b06c629802006..8a3d08f459783 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples import org.apache.commons.math3.linear._ -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Alternating least squares matrix factorization. @@ -108,8 +108,12 @@ object SparkALS { println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS") - val sparkConf = new SparkConf().setAppName("SparkALS") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkALS") + .getOrCreate() + + val sc = spark.sparkContext val R = generateR() @@ -135,7 +139,7 @@ object SparkALS { println() } - sc.stop() + spark.stop() } private def randomVector(n: Int): RealVector = diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index c514eb0fa51ae..84f133e0116a2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -23,9 +23,8 @@ import java.util.Random import scala.math.exp import breeze.linalg.{DenseVector, Vector} -import org.apache.hadoop.conf.Configuration -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Logistic regression based classification. @@ -67,11 +66,14 @@ object SparkHdfsLR { showWarning() - val sparkConf = new SparkConf().setAppName("SparkHdfsLR") + val spark = SparkSession + .builder + .appName("SparkHdfsLR") + .getOrCreate() + val inputPath = args(0) - val conf = new Configuration() - val sc = new SparkContext(sparkConf) - val lines = sc.textFile(inputPath) + val lines = spark.read.text(inputPath).rdd + val points = lines.map(parsePoint).cache() val ITERATIONS = args(1).toInt @@ -88,7 +90,7 @@ object SparkHdfsLR { } println("Final w: " + w) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 676164806e196..aa93c93c441f4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples import breeze.linalg.{squaredDistance, DenseVector, Vector} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * K-means clustering. @@ -66,14 +66,17 @@ object SparkKMeans { showWarning() - val sparkConf = new SparkConf().setAppName("SparkKMeans") - val sc = new SparkContext(sparkConf) - val lines = sc.textFile(args(0)) + val spark = SparkSession + .builder + .appName("SparkKMeans") + .getOrCreate() + + val lines = spark.read.text(args(0)).rdd val data = lines.map(parseVector _).cache() val K = args(1).toInt val convergeDist = args(2).toDouble - val kPoints = data.takeSample(withReplacement = false, K, 42).toArray + val kPoints = data.takeSample(withReplacement = false, K, 42) var tempDist = 1.0 while(tempDist > convergeDist) { @@ -97,7 +100,7 @@ object SparkKMeans { println("Final centers:") kPoints.foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 718f84f6450e0..8ef3aab6570d6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -24,7 +24,7 @@ import scala.math.exp import breeze.linalg.{DenseVector, Vector} -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Logistic regression based classification. @@ -63,8 +63,13 @@ object SparkLR { showWarning() - val sparkConf = new SparkConf().setAppName("SparkLR") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkLR") + .getOrCreate() + + val sc = spark.sparkContext + val numSlices = if (args.length > 0) args(0).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() @@ -82,7 +87,7 @@ object SparkLR { println("Final w: " + w) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 2664ddbb87d23..b7c363c7d4fae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -18,7 +18,7 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Computes the PageRank of URLs from an input file. Input file should @@ -50,10 +50,13 @@ object SparkPageRank { showWarning() - val sparkConf = new SparkConf().setAppName("PageRank") + val spark = SparkSession + .builder + .appName("SparkPageRank") + .getOrCreate() + val iters = if (args.length > 1) args(1).toInt else 10 - val ctx = new SparkContext(sparkConf) - val lines = ctx.textFile(args(0), 1) + val lines = spark.read.text(args(0)).rdd val links = lines.map{ s => val parts = s.split("\\s+") (parts(0), parts(1)) @@ -71,7 +74,7 @@ object SparkPageRank { val output = ranks.collect() output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + ".")) - ctx.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 818d4f2b81f82..5be8f3b073410 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -20,16 +20,19 @@ package org.apache.spark.examples import scala.math.random -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** Computes an approximation to pi */ object SparkPi { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("Spark Pi") - val spark = new SparkContext(conf) + val spark = SparkSession + .builder + .appName("Spark Pi") + .getOrCreate() + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow - val count = spark.parallelize(1 until n, slices).map { i => + val count = sc.parallelize(1 until n, slices).map { i => val x = random * 2 - 1 val y = random * 2 - 1 if (x*x + y*y < 1) 1 else 0 diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index fc7a1f859f602..46aa68b8b8195 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -21,7 +21,7 @@ package org.apache.spark.examples import scala.collection.mutable import scala.util.Random -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Transitive closure on a graph. @@ -42,10 +42,13 @@ object SparkTC { } def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("SparkTC") - val spark = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkTC") + .getOrCreate() + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 - var tc = spark.parallelize(generateGraph, slices).cache() + var tc = sc.parallelize(generateGraph, slices).cache() // Linear transitive closure: each round grows paths by one edge, // by joining the graph's edges with the already-discovered paths. diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 7293cb51b2613..59bdfa09ad0f8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -22,7 +22,7 @@ import java.io.File import com.google.common.io.{ByteStreams, Files} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkConf import org.apache.spark.sql._ object HiveFromSpark { 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 35a9f44feca64..1e8f1062c53ee 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 @@ -168,8 +168,8 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { Assert.assertEquals( new StructField("d", new ArrayType(DataTypes.StringType, true), true, Metadata.empty()), schema.apply("d")); - Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, Metadata.empty()), - schema.apply("e")); + Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, + Metadata.empty()), schema.apply("e")); Row first = df.select("a", "b", "c", "d", "e").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); // Now Java lists and maps are converted to Scala Seq's and Map's. Once we get a Seq below, @@ -189,7 +189,7 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { for (int i = 0; i < d.length(); i++) { Assert.assertEquals(bean.getD().get(i), d.apply(i)); } - // Java.math.BigInteger is equavient to Spark Decimal(38,0) + // Java.math.BigInteger is equavient to Spark Decimal(38,0) Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); } From c7e013f1888f084d6a02d6dd905a77c4f20d1db4 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Fri, 20 May 2016 16:41:57 -0700 Subject: [PATCH 0329/1470] [SPARK-15456][PYSPARK] Fixed PySpark shell context initialization when HiveConf not present ## What changes were proposed in this pull request? When PySpark shell cannot find HiveConf, it will fallback to create a SparkSession from a SparkContext. This fixes a bug caused by using a variable to SparkContext before it was initialized. ## How was this patch tested? Manually starting PySpark shell and using the SparkContext Author: Bryan Cutler Closes #13237 from BryanCutler/pyspark-shell-session-context-SPARK-15456. (cherry picked from commit 021c19702c720b4466b016498917d47f99000e13) Signed-off-by: Andrew Or --- python/pyspark/shell.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index ef46d3065e5e8..ac5ce87a3f0fd 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -44,9 +44,9 @@ .enableHiveSupport()\ .getOrCreate() except py4j.protocol.Py4JError: - spark = SparkSession(sc) + spark = SparkSession.builder.getOrCreate() except TypeError: - spark = SparkSession(sc) + spark = SparkSession.builder.getOrCreate() sc = spark.sparkContext atexit.register(lambda: sc.stop()) From c0cc921a577c90f058ef84367ae2a92ecae8677a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 May 2016 22:01:55 -0700 Subject: [PATCH 0330/1470] [SPARK-15424][SPARK-15437][SPARK-14807][SQL] Revert Create a hivecontext-compatibility module ## What changes were proposed in this pull request? I initially asked to create a hivecontext-compatibility module to put the HiveContext there. But we are so close to Spark 2.0 release and there is only a single class in it. It seems overkill to have an entire package, which makes it more inconvenient, for a single class. ## How was this patch tested? Tests were moved. Author: Reynold Xin Closes #13207 from rxin/SPARK-15424. (cherry picked from commit 45b7557e61d440612d4ce49c31b5ef242fdefa54) Signed-off-by: Reynold Xin --- dev/run-tests.py | 2 +- dev/sparktestsupport/modules.py | 12 ---- pom.xml | 1 - project/SparkBuild.scala | 6 +- .../apache/spark/sql/hive/HiveContext.scala | 0 .../hive/HiveContextCompatibilitySuite.scala | 5 +- sql/hivecontext-compatibility/pom.xml | 57 ------------------- 7 files changed, 6 insertions(+), 77 deletions(-) rename sql/{hivecontext-compatibility => hive}/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala (100%) rename sql/{hivecontext-compatibility => hive}/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala (95%) delete mode 100644 sql/hivecontext-compatibility/pom.xml diff --git a/dev/run-tests.py b/dev/run-tests.py index 7b3269752bac6..2030c4ab2308b 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -110,7 +110,7 @@ def determine_modules_to_test(changed_modules): ['graphx', 'examples'] >>> x = [x.name for x in determine_modules_to_test([modules.sql])] >>> x # doctest: +NORMALIZE_WHITESPACE - ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver', 'hivecontext-compatibility', + ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver', 'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] """ modules_to_test = set() diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 0d6aa7422a534..8e2364d2f71cb 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -158,18 +158,6 @@ def __hash__(self): ) -hivecontext_compatibility = Module( - name="hivecontext-compatibility", - dependencies=[hive], - source_file_regexes=[ - "sql/hivecontext-compatibility/", - ], - sbt_test_goals=[ - "hivecontext-compatibility/test" - ] -) - - sketch = Module( name="sketch", dependencies=[tags], diff --git a/pom.xml b/pom.xml index 9c13af17e4ed5..e778f77fa3151 100644 --- a/pom.xml +++ b/pom.xml @@ -100,7 +100,6 @@ sql/catalyst sql/core sql/hive - sql/hivecontext-compatibility assembly external/flume external/flume-sink diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 3ad9873f43679..f08ca7001f34d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -39,8 +39,8 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, hiveCompatibility) = Seq( - "catalyst", "sql", "hive", "hive-thriftserver", "hivecontext-compatibility" + val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer) = Seq( + "catalyst", "sql", "hive", "hive-thriftserver" ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( @@ -339,7 +339,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( - spark, hive, hiveThriftServer, hiveCompatibility, catalyst, repl, networkCommon, networkShuffle, networkYarn, + spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, unsafe, tags, sketch, mllibLocal ).contains(x) } diff --git a/sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala similarity index 100% rename from sql/hivecontext-compatibility/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala diff --git a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala similarity index 95% rename from sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala index 1c1db72e27ffe..3aa8174702513 100644 --- a/sql/hivecontext-compatibility/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive import org.scalatest.BeforeAndAfterEach -import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEach { @@ -29,7 +29,7 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac override def beforeAll(): Unit = { super.beforeAll() - sc = new SparkContext("local[4]", "test") + sc = SparkContext.getOrCreate(new SparkConf().setMaster("local").setAppName("test")) HiveUtils.newTemporaryConfiguration(useInMemoryDerby = true).foreach { case (k, v) => sc.hadoopConfiguration.set(k, v) } @@ -47,7 +47,6 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac override def afterAll(): Unit = { try { - sc.stop() sc = null hc = null } finally { diff --git a/sql/hivecontext-compatibility/pom.xml b/sql/hivecontext-compatibility/pom.xml deleted file mode 100644 index ed9ef8e27919e..0000000000000 --- a/sql/hivecontext-compatibility/pom.xml +++ /dev/null @@ -1,57 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-hivecontext-compatibility_2.11 - jar - Spark Project HiveContext Compatibility - http://spark.apache.org/ - - hivecontext-compatibility - - - - - org.apache.spark - spark-hive_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - From a93f04dd43669ddac3ab0b298036b1a5e8e263f1 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Sat, 21 May 2016 06:39:47 -0500 Subject: [PATCH 0331/1470] [SPARK-15445][SQL] Build fails for java 1.7 after adding java.mathBigInteger support ## What changes were proposed in this pull request? Using longValue() and then checking whether the value is in the range for a long manually. ## How was this patch tested? Existing tests Author: Sandeep Singh Closes #13223 from techaddict/SPARK-15445. (cherry picked from commit 666bf2e8357cb9a61eaa2270c414e635d9c331ce) Signed-off-by: Sean Owen --- .../org/apache/spark/sql/types/Decimal.scala | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index b907f62802175..31604bad0ff80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.types +import java.lang.{Long => JLong} import java.math.{BigInteger, MathContext, RoundingMode} import org.apache.spark.annotation.DeveloperApi @@ -132,17 +133,15 @@ final class Decimal extends Ordered[Decimal] with Serializable { * Set this Decimal to the given BigInteger value. Will have precision 38 and scale 0. */ def set(bigintval: BigInteger): Decimal = { - try { - this.decimalVal = null - this.longVal = bigintval.longValueExact() - this._precision = DecimalType.MAX_PRECISION - this._scale = 0 - this - } - catch { - case e: ArithmeticException => - throw new IllegalArgumentException(s"BigInteger ${bigintval} too large for decimal") - } + // TODO: Remove this once we migrate to java8 and use longValueExact() instead. + require( + bigintval.compareTo(LONG_MAX_BIG_INT) <= 0 && bigintval.compareTo(LONG_MIN_BIG_INT) >= 0, + s"BigInteger $bigintval too large for decimal") + this.decimalVal = null + this.longVal = bigintval.longValue() + this._precision = DecimalType.MAX_PRECISION + this._scale = 0 + this } /** @@ -382,6 +381,9 @@ object Decimal { private[sql] val ZERO = Decimal(0) private[sql] val ONE = Decimal(1) + private val LONG_MAX_BIG_INT = BigInteger.valueOf(JLong.MAX_VALUE) + private val LONG_MIN_BIG_INT = BigInteger.valueOf(JLong.MIN_VALUE) + def apply(value: Double): Decimal = new Decimal().set(value) def apply(value: Long): Decimal = new Decimal().set(value) From 69f39552d66eb3fe1cd096d6f8380f3f66de91e8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 21 May 2016 08:11:14 -0700 Subject: [PATCH 0332/1470] [SPARK-15462][SQL][TEST] unresolved === false` is enough in testcases. ## What changes were proposed in this pull request? In only `catalyst` module, there exists 8 evaluation test cases on unresolved expressions. But, in real-world situation, those cases doesn't happen since they occurs exceptions before evaluations. ```scala scala> sql("select format_number(null, 3)") res0: org.apache.spark.sql.DataFrame = [format_number(CAST(NULL AS DOUBLE), 3): string] scala> sql("select format_number(cast(null as NULL), 3)") org.apache.spark.sql.catalyst.parser.ParseException: DataType null() is not supported.(line 1, pos 34) ``` This PR makes those testcases more realistic. ```scala - checkEvaluation(FormatNumber(Literal.create(null, NullType), Literal(3)), null) + assert(FormatNumber(Literal.create(null, NullType), Literal(3)).resolved === false) ``` Also, this PR also removes redundant `resolved` checking in `FoldablePropagation` optimizer. ## How was this patch tested? Pass the modified Jenkins tests. Author: Dongjoon Hyun Closes #13241 from dongjoon-hyun/SPARK-15462. (cherry picked from commit f39621c998a0fe91a5115f3f843c3ca8dd71c1ab) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- .../apache/spark/sql/catalyst/expressions/CastSuite.scala | 7 ------- .../sql/catalyst/expressions/DateExpressionsSuite.scala | 6 +++--- .../sql/catalyst/expressions/StringExpressionsSuite.scala | 2 +- 4 files changed, 5 insertions(+), 12 deletions(-) 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 a6fb34cbfb110..5e998d61880ea 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 @@ -673,7 +673,7 @@ object FoldablePropagation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { val foldableMap = AttributeMap(plan.flatMap { case Project(projectList, _) => projectList.collect { - case a: Alias if a.resolved && a.child.foldable => (a.toAttribute, a) + case a: Alias if a.child.foldable => (a.toAttribute, a) } case _ => Nil }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 43af3592070fe..dfda7c50f2c05 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -548,7 +548,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = false)) assert(ret.resolved === false) - checkEvaluation(ret, Seq(null, true, false)) } { @@ -607,7 +606,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = false)) assert(ret.resolved === false) - checkEvaluation(ret, Map("a" -> null, "b" -> true, "c" -> false)) } { val ret = cast(map_notNull, MapType(IntegerType, StringType, valueContainsNull = true)) @@ -714,7 +712,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("b", BooleanType, nullable = true), StructField("c", BooleanType, nullable = false)))) assert(ret.resolved === false) - checkEvaluation(ret, InternalRow(null, true, false)) } { @@ -755,10 +752,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("l", LongType, nullable = true))))))) assert(ret.resolved === false) - checkEvaluation(ret, Row( - Seq(123, null, null), - Map("a" -> null, "b" -> true, "c" -> false), - Row(0L))) } test("cast between string and interval") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 53c66d8a754ed..6118a34d29eaa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -143,7 +143,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Seconds") { - checkEvaluation(Second(Literal.create(null, DateType)), null) + assert(Second(Literal.create(null, DateType)).resolved === false) checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) checkEvaluation(Second(Literal(ts)), 15) @@ -176,7 +176,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Hour") { - checkEvaluation(Hour(Literal.create(null, DateType)), null) + assert(Hour(Literal.create(null, DateType)).resolved === false) checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) checkEvaluation(Hour(Literal(ts)), 13) @@ -195,7 +195,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Minute") { - checkEvaluation(Minute(Literal.create(null, DateType)), null) + assert(Minute(Literal.create(null, DateType)).resolved === false) checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) checkEvaluation(Minute(Literal(ts)), 10) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 2cf8ca7000edc..c09c64fd6bf62 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -688,7 +688,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { Literal(Decimal(123123324123L) * Decimal(123123.21234d)), Literal(4)), "15,159,339,180,002,773.2778") checkEvaluation(FormatNumber(Literal.create(null, IntegerType), Literal(3)), null) - checkEvaluation(FormatNumber(Literal.create(null, NullType), Literal(3)), null) + assert(FormatNumber(Literal.create(null, NullType), Literal(3)).resolved === false) } test("find in set") { From 9a8df0c9a46babb55dd8301e11721025cdfb8af5 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Sat, 21 May 2016 08:36:08 -0700 Subject: [PATCH 0333/1470] [SPARK-15114][SQL] Column name generated by typed aggregate is super verbose ## What changes were proposed in this pull request? Generate a shorter default alias for `AggregateExpression `, In this PR, aggregate function name along with a index is used for generating the alias name. ```SQL val ds = Seq(1, 3, 2, 5).toDS() ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)).show() ``` Output before change. ```SQL +-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+ |typedsumdouble(unresolveddeserializer(upcast(input[0, int], IntegerType, - root class: "scala.Int"), value#1), upcast(value))|typedaverage(unresolveddeserializer(upcast(input[0, int], IntegerType, - root class: "scala.Int"), value#1), newInstance(class scala.Tuple2))| +-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+ | 11.0| 2.75| +-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+ ``` Output after change: ```SQL +-----------------+---------------+ |typedsumdouble_c1|typedaverage_c2| +-----------------+---------------+ | 11.0| 2.75| +-----------------+---------------+ ``` Note: There is one test in ParquetSuites.scala which shows that that the system picked alias name is not usable and is rejected. [test](https://github.com/apache/spark/blob/master/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala#L672-#L687) ## How was this patch tested? A new test was added in DataSetAggregatorSuite. Author: Dilip Biswal Closes #13045 from dilipbiswal/spark-15114. (cherry picked from commit 5e1ee28984b169eaab5d2f832921d32cf09de915) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++-- .../spark/sql/catalyst/analysis/unresolved.scala | 7 +++++-- .../main/scala/org/apache/spark/sql/Column.scala | 16 +++++++++++++++- .../spark/sql/RelationalGroupedDataset.scala | 3 +++ .../spark/sql/DatasetAggregatorSuite.scala | 12 ++++++++++++ 5 files changed, 39 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 2c269478ee7ef..9a92330f75f6f 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 @@ -177,14 +177,16 @@ class Analyzer( private def assignAliases(exprs: Seq[NamedExpression]) = { exprs.zipWithIndex.map { case (expr, i) => - expr.transformUp { case u @ UnresolvedAlias(child, optionalAliasName) => + expr.transformUp { case u @ UnresolvedAlias(child, optGenAliasFunc) => child match { case ne: NamedExpression => ne case e if !e.resolved => u case g: Generator => MultiAlias(g, Nil) case c @ Cast(ne: NamedExpression, _) => Alias(c, ne.name)() case e: ExtractValue => Alias(e, toPrettySQL(e))() - case e => Alias(e, optionalAliasName.getOrElse(toPrettySQL(e)))() + case e if optGenAliasFunc.isDefined => + Alias(child, optGenAliasFunc.get.apply(e))() + case e => Alias(e, toPrettySQL(e))() } } }.asInstanceOf[Seq[NamedExpression]] 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 1f1897dc36df2..e953eda7843c9 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 @@ -325,10 +325,13 @@ case class UnresolvedExtractValue(child: Expression, extraction: Expression) * Holds the expression that has yet to be aliased. * * @param child The computation that is needs to be resolved during analysis. - * @param aliasName The name if specified to be associated with the result of computing [[child]] + * @param aliasFunc The function if specified to be called to generate an alias to associate + * with the result of computing [[child]] * */ -case class UnresolvedAlias(child: Expression, aliasName: Option[String] = None) +case class UnresolvedAlias( + child: Expression, + aliasFunc: Option[Expression => String] = None) extends UnaryExpression with NamedExpression with Unevaluable { override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") 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 9b8334d334e4d..204af719b2c59 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 @@ -24,6 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression @@ -37,6 +38,14 @@ private[sql] object Column { def apply(expr: Expression): Column = new Column(expr) def unapply(col: Column): Option[Expression] = Some(col.expr) + + private[sql] def generateAlias(e: Expression): String = { + e match { + case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => + a.aggregateFunction.toString + case expr => usePrettyExpression(expr).sql + } + } } /** @@ -145,7 +154,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { case jt: JsonTuple => MultiAlias(jt, Nil) - case func: UnresolvedFunction => UnresolvedAlias(func, Some(usePrettyExpression(func).sql)) + case func: UnresolvedFunction => UnresolvedAlias(func, Some(Column.generateAlias)) // If we have a top level Cast, there is a chance to give it a better alias, if there is a // NamedExpression under this Cast. @@ -156,9 +165,14 @@ class Column(protected[sql] val expr: Expression) extends Logging { case other => Alias(expr, usePrettyExpression(expr).sql)() } + case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => + UnresolvedAlias(a, Some(Column.generateAlias)) + case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } + + override def toString: String = usePrettyExpression(expr).sql override def equals(that: Any): Boolean = that match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 4f5bf633fab2e..b0e48a6553a45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Pivot} import org.apache.spark.sql.catalyst.util.usePrettyExpression +import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.NumericType @@ -73,6 +74,8 @@ class RelationalGroupedDataset protected[sql]( private[this] def alias(expr: Expression): NamedExpression = expr match { case u: UnresolvedAttribute => UnresolvedAlias(u) case expr: NamedExpression => expr + case a: AggregateExpression if (a.aggregateFunction.isInstanceOf[TypedAggregateExpression]) => + UnresolvedAlias(a, Some(Column.generateAlias)) case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index f1585ca3ff318..ead7bd9642eca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -240,4 +240,16 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { val df2 = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") checkAnswer(df2.agg(RowAgg.toColumn as "b").select("b"), Row(6) :: Nil) } + + test("spark-15114 shorter system generated alias names") { + val ds = Seq(1, 3, 2, 5).toDS() + assert(ds.select(typed.sum((i: Int) => i)).columns.head === "TypedSumDouble(int)") + val ds2 = ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)) + assert(ds2.columns.head === "TypedSumDouble(int)") + assert(ds2.columns.last === "TypedAverage(int)") + val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") + assert(df.groupBy($"j").agg(RowAgg.toColumn).columns.last == + "RowAgg(org.apache.spark.sql.Row)") + assert(df.groupBy($"j").agg(RowAgg.toColumn as "agg1").columns.last == "agg1") + } } From d7bf318a0c41ad6ab9b84bb3dd922560080837cb Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 20 May 2016 15:19:28 -0700 Subject: [PATCH 0334/1470] [SPARK-15078] [SQL] Add all TPCDS 1.4 benchmark queries for SparkSQL Now that SparkSQL supports all TPC-DS queries, this patch adds all 99 benchmark queries inside SparkSQL. Benchmark only Author: Sameer Agarwal Closes #13188 from sameeragarwal/tpcds-all. (cherry picked from commit a78d6ce376edf2a8836e01f47b9dff5371058d4c) Signed-off-by: Reynold Xin --- dev/.rat-excludes | 1 + sql/core/src/test/resources/tpcds/q1.sql | 19 + sql/core/src/test/resources/tpcds/q10.sql | 57 + sql/core/src/test/resources/tpcds/q11.sql | 68 + sql/core/src/test/resources/tpcds/q12.sql | 22 + sql/core/src/test/resources/tpcds/q13.sql | 49 + sql/core/src/test/resources/tpcds/q14a.sql | 120 ++ sql/core/src/test/resources/tpcds/q14b.sql | 95 ++ sql/core/src/test/resources/tpcds/q15.sql | 15 + sql/core/src/test/resources/tpcds/q16.sql | 23 + sql/core/src/test/resources/tpcds/q17.sql | 33 + sql/core/src/test/resources/tpcds/q18.sql | 28 + sql/core/src/test/resources/tpcds/q19.sql | 19 + sql/core/src/test/resources/tpcds/q2.sql | 81 ++ sql/core/src/test/resources/tpcds/q20.sql | 18 + sql/core/src/test/resources/tpcds/q21.sql | 25 + sql/core/src/test/resources/tpcds/q22.sql | 14 + sql/core/src/test/resources/tpcds/q23a.sql | 53 + sql/core/src/test/resources/tpcds/q23b.sql | 68 + sql/core/src/test/resources/tpcds/q24a.sql | 34 + sql/core/src/test/resources/tpcds/q24b.sql | 34 + sql/core/src/test/resources/tpcds/q25.sql | 33 + sql/core/src/test/resources/tpcds/q26.sql | 19 + sql/core/src/test/resources/tpcds/q27.sql | 21 + sql/core/src/test/resources/tpcds/q28.sql | 56 + sql/core/src/test/resources/tpcds/q29.sql | 32 + sql/core/src/test/resources/tpcds/q3.sql | 13 + sql/core/src/test/resources/tpcds/q30.sql | 35 + sql/core/src/test/resources/tpcds/q31.sql | 60 + sql/core/src/test/resources/tpcds/q32.sql | 15 + sql/core/src/test/resources/tpcds/q33.sql | 65 + sql/core/src/test/resources/tpcds/q34.sql | 32 + sql/core/src/test/resources/tpcds/q35.sql | 46 + sql/core/src/test/resources/tpcds/q36.sql | 26 + sql/core/src/test/resources/tpcds/q37.sql | 15 + sql/core/src/test/resources/tpcds/q38.sql | 30 + sql/core/src/test/resources/tpcds/q39a.sql | 47 + sql/core/src/test/resources/tpcds/q39b.sql | 48 + sql/core/src/test/resources/tpcds/q4.sql | 120 ++ sql/core/src/test/resources/tpcds/q40.sql | 25 + sql/core/src/test/resources/tpcds/q41.sql | 49 + sql/core/src/test/resources/tpcds/q42.sql | 18 + sql/core/src/test/resources/tpcds/q43.sql | 33 + sql/core/src/test/resources/tpcds/q44.sql | 46 + sql/core/src/test/resources/tpcds/q45.sql | 21 + sql/core/src/test/resources/tpcds/q46.sql | 32 + sql/core/src/test/resources/tpcds/q47.sql | 63 + sql/core/src/test/resources/tpcds/q48.sql | 63 + sql/core/src/test/resources/tpcds/q49.sql | 126 ++ sql/core/src/test/resources/tpcds/q5.sql | 131 ++ sql/core/src/test/resources/tpcds/q50.sql | 47 + sql/core/src/test/resources/tpcds/q51.sql | 55 + sql/core/src/test/resources/tpcds/q52.sql | 14 + sql/core/src/test/resources/tpcds/q53.sql | 30 + sql/core/src/test/resources/tpcds/q54.sql | 61 + sql/core/src/test/resources/tpcds/q55.sql | 13 + sql/core/src/test/resources/tpcds/q56.sql | 65 + sql/core/src/test/resources/tpcds/q57.sql | 56 + sql/core/src/test/resources/tpcds/q58.sql | 59 + sql/core/src/test/resources/tpcds/q59.sql | 75 + sql/core/src/test/resources/tpcds/q6.sql | 21 + sql/core/src/test/resources/tpcds/q60.sql | 62 + sql/core/src/test/resources/tpcds/q61.sql | 33 + sql/core/src/test/resources/tpcds/q62.sql | 35 + sql/core/src/test/resources/tpcds/q63.sql | 31 + sql/core/src/test/resources/tpcds/q64.sql | 92 ++ sql/core/src/test/resources/tpcds/q65.sql | 33 + sql/core/src/test/resources/tpcds/q66.sql | 240 ++++ sql/core/src/test/resources/tpcds/q67.sql | 38 + sql/core/src/test/resources/tpcds/q68.sql | 34 + sql/core/src/test/resources/tpcds/q69.sql | 38 + sql/core/src/test/resources/tpcds/q7.sql | 19 + sql/core/src/test/resources/tpcds/q70.sql | 38 + sql/core/src/test/resources/tpcds/q71.sql | 44 + sql/core/src/test/resources/tpcds/q72.sql | 33 + sql/core/src/test/resources/tpcds/q73.sql | 30 + sql/core/src/test/resources/tpcds/q74.sql | 58 + sql/core/src/test/resources/tpcds/q75.sql | 76 + sql/core/src/test/resources/tpcds/q76.sql | 47 + sql/core/src/test/resources/tpcds/q77.sql | 100 ++ sql/core/src/test/resources/tpcds/q78.sql | 64 + sql/core/src/test/resources/tpcds/q79.sql | 27 + sql/core/src/test/resources/tpcds/q8.sql | 87 ++ sql/core/src/test/resources/tpcds/q80.sql | 94 ++ sql/core/src/test/resources/tpcds/q81.sql | 38 + sql/core/src/test/resources/tpcds/q82.sql | 15 + sql/core/src/test/resources/tpcds/q83.sql | 56 + sql/core/src/test/resources/tpcds/q84.sql | 19 + sql/core/src/test/resources/tpcds/q85.sql | 82 ++ sql/core/src/test/resources/tpcds/q86.sql | 24 + sql/core/src/test/resources/tpcds/q87.sql | 28 + sql/core/src/test/resources/tpcds/q88.sql | 122 ++ sql/core/src/test/resources/tpcds/q89.sql | 30 + sql/core/src/test/resources/tpcds/q9.sql | 48 + sql/core/src/test/resources/tpcds/q90.sql | 19 + sql/core/src/test/resources/tpcds/q91.sql | 23 + sql/core/src/test/resources/tpcds/q92.sql | 16 + sql/core/src/test/resources/tpcds/q93.sql | 19 + sql/core/src/test/resources/tpcds/q94.sql | 23 + sql/core/src/test/resources/tpcds/q95.sql | 29 + sql/core/src/test/resources/tpcds/q96.sql | 11 + sql/core/src/test/resources/tpcds/q97.sql | 30 + sql/core/src/test/resources/tpcds/q98.sql | 21 + sql/core/src/test/resources/tpcds/q99.sql | 34 + .../benchmark/TPCDSQueryBenchmark.scala | 126 ++ .../datasources/parquet/TPCDSBenchmark.scala | 1226 ----------------- 106 files changed, 4858 insertions(+), 1226 deletions(-) create mode 100755 sql/core/src/test/resources/tpcds/q1.sql create mode 100755 sql/core/src/test/resources/tpcds/q10.sql create mode 100755 sql/core/src/test/resources/tpcds/q11.sql create mode 100755 sql/core/src/test/resources/tpcds/q12.sql create mode 100755 sql/core/src/test/resources/tpcds/q13.sql create mode 100755 sql/core/src/test/resources/tpcds/q14a.sql create mode 100755 sql/core/src/test/resources/tpcds/q14b.sql create mode 100755 sql/core/src/test/resources/tpcds/q15.sql create mode 100755 sql/core/src/test/resources/tpcds/q16.sql create mode 100755 sql/core/src/test/resources/tpcds/q17.sql create mode 100755 sql/core/src/test/resources/tpcds/q18.sql create mode 100755 sql/core/src/test/resources/tpcds/q19.sql create mode 100755 sql/core/src/test/resources/tpcds/q2.sql create mode 100755 sql/core/src/test/resources/tpcds/q20.sql create mode 100755 sql/core/src/test/resources/tpcds/q21.sql create mode 100755 sql/core/src/test/resources/tpcds/q22.sql create mode 100755 sql/core/src/test/resources/tpcds/q23a.sql create mode 100755 sql/core/src/test/resources/tpcds/q23b.sql create mode 100755 sql/core/src/test/resources/tpcds/q24a.sql create mode 100755 sql/core/src/test/resources/tpcds/q24b.sql create mode 100755 sql/core/src/test/resources/tpcds/q25.sql create mode 100755 sql/core/src/test/resources/tpcds/q26.sql create mode 100755 sql/core/src/test/resources/tpcds/q27.sql create mode 100755 sql/core/src/test/resources/tpcds/q28.sql create mode 100755 sql/core/src/test/resources/tpcds/q29.sql create mode 100755 sql/core/src/test/resources/tpcds/q3.sql create mode 100755 sql/core/src/test/resources/tpcds/q30.sql create mode 100755 sql/core/src/test/resources/tpcds/q31.sql create mode 100755 sql/core/src/test/resources/tpcds/q32.sql create mode 100755 sql/core/src/test/resources/tpcds/q33.sql create mode 100755 sql/core/src/test/resources/tpcds/q34.sql create mode 100755 sql/core/src/test/resources/tpcds/q35.sql create mode 100755 sql/core/src/test/resources/tpcds/q36.sql create mode 100755 sql/core/src/test/resources/tpcds/q37.sql create mode 100755 sql/core/src/test/resources/tpcds/q38.sql create mode 100755 sql/core/src/test/resources/tpcds/q39a.sql create mode 100755 sql/core/src/test/resources/tpcds/q39b.sql create mode 100755 sql/core/src/test/resources/tpcds/q4.sql create mode 100755 sql/core/src/test/resources/tpcds/q40.sql create mode 100755 sql/core/src/test/resources/tpcds/q41.sql create mode 100755 sql/core/src/test/resources/tpcds/q42.sql create mode 100755 sql/core/src/test/resources/tpcds/q43.sql create mode 100755 sql/core/src/test/resources/tpcds/q44.sql create mode 100755 sql/core/src/test/resources/tpcds/q45.sql create mode 100755 sql/core/src/test/resources/tpcds/q46.sql create mode 100755 sql/core/src/test/resources/tpcds/q47.sql create mode 100755 sql/core/src/test/resources/tpcds/q48.sql create mode 100755 sql/core/src/test/resources/tpcds/q49.sql create mode 100755 sql/core/src/test/resources/tpcds/q5.sql create mode 100755 sql/core/src/test/resources/tpcds/q50.sql create mode 100755 sql/core/src/test/resources/tpcds/q51.sql create mode 100755 sql/core/src/test/resources/tpcds/q52.sql create mode 100755 sql/core/src/test/resources/tpcds/q53.sql create mode 100755 sql/core/src/test/resources/tpcds/q54.sql create mode 100755 sql/core/src/test/resources/tpcds/q55.sql create mode 100755 sql/core/src/test/resources/tpcds/q56.sql create mode 100755 sql/core/src/test/resources/tpcds/q57.sql create mode 100755 sql/core/src/test/resources/tpcds/q58.sql create mode 100755 sql/core/src/test/resources/tpcds/q59.sql create mode 100755 sql/core/src/test/resources/tpcds/q6.sql create mode 100755 sql/core/src/test/resources/tpcds/q60.sql create mode 100755 sql/core/src/test/resources/tpcds/q61.sql create mode 100755 sql/core/src/test/resources/tpcds/q62.sql create mode 100755 sql/core/src/test/resources/tpcds/q63.sql create mode 100755 sql/core/src/test/resources/tpcds/q64.sql create mode 100755 sql/core/src/test/resources/tpcds/q65.sql create mode 100755 sql/core/src/test/resources/tpcds/q66.sql create mode 100755 sql/core/src/test/resources/tpcds/q67.sql create mode 100755 sql/core/src/test/resources/tpcds/q68.sql create mode 100755 sql/core/src/test/resources/tpcds/q69.sql create mode 100755 sql/core/src/test/resources/tpcds/q7.sql create mode 100755 sql/core/src/test/resources/tpcds/q70.sql create mode 100755 sql/core/src/test/resources/tpcds/q71.sql create mode 100755 sql/core/src/test/resources/tpcds/q72.sql create mode 100755 sql/core/src/test/resources/tpcds/q73.sql create mode 100755 sql/core/src/test/resources/tpcds/q74.sql create mode 100755 sql/core/src/test/resources/tpcds/q75.sql create mode 100755 sql/core/src/test/resources/tpcds/q76.sql create mode 100755 sql/core/src/test/resources/tpcds/q77.sql create mode 100755 sql/core/src/test/resources/tpcds/q78.sql create mode 100755 sql/core/src/test/resources/tpcds/q79.sql create mode 100755 sql/core/src/test/resources/tpcds/q8.sql create mode 100755 sql/core/src/test/resources/tpcds/q80.sql create mode 100755 sql/core/src/test/resources/tpcds/q81.sql create mode 100755 sql/core/src/test/resources/tpcds/q82.sql create mode 100755 sql/core/src/test/resources/tpcds/q83.sql create mode 100755 sql/core/src/test/resources/tpcds/q84.sql create mode 100755 sql/core/src/test/resources/tpcds/q85.sql create mode 100755 sql/core/src/test/resources/tpcds/q86.sql create mode 100755 sql/core/src/test/resources/tpcds/q87.sql create mode 100755 sql/core/src/test/resources/tpcds/q88.sql create mode 100755 sql/core/src/test/resources/tpcds/q89.sql create mode 100755 sql/core/src/test/resources/tpcds/q9.sql create mode 100755 sql/core/src/test/resources/tpcds/q90.sql create mode 100755 sql/core/src/test/resources/tpcds/q91.sql create mode 100755 sql/core/src/test/resources/tpcds/q92.sql create mode 100755 sql/core/src/test/resources/tpcds/q93.sql create mode 100755 sql/core/src/test/resources/tpcds/q94.sql create mode 100755 sql/core/src/test/resources/tpcds/q95.sql create mode 100755 sql/core/src/test/resources/tpcds/q96.sql create mode 100755 sql/core/src/test/resources/tpcds/q97.sql create mode 100755 sql/core/src/test/resources/tpcds/q98.sql create mode 100755 sql/core/src/test/resources/tpcds/q99.sql create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 67a241806d7cb..fb582dec56d51 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -98,3 +98,4 @@ spark-deps-.* .*csv .*tsv org.apache.spark.scheduler.ExternalClusterManager +.*\.sql diff --git a/sql/core/src/test/resources/tpcds/q1.sql b/sql/core/src/test/resources/tpcds/q1.sql new file mode 100755 index 0000000000000..4d20faad8ef58 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q1.sql @@ -0,0 +1,19 @@ +WITH customer_total_return AS +( SELECT + sr_customer_sk AS ctr_customer_sk, + sr_store_sk AS ctr_store_sk, + sum(sr_return_amt) AS ctr_total_return + FROM store_returns, date_dim + WHERE sr_returned_date_sk = d_date_sk AND d_year = 2000 + GROUP BY sr_customer_sk, sr_store_sk) +SELECT c_customer_id +FROM customer_total_return ctr1, store, customer +WHERE ctr1.ctr_total_return > + (SELECT avg(ctr_total_return) * 1.2 + FROM customer_total_return ctr2 + WHERE ctr1.ctr_store_sk = ctr2.ctr_store_sk) + AND s_store_sk = ctr1.ctr_store_sk + AND s_state = 'TN' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q10.sql b/sql/core/src/test/resources/tpcds/q10.sql new file mode 100755 index 0000000000000..5500e1aea1552 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q10.sql @@ -0,0 +1,57 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_county IN ('Rush County', 'Toole County', 'Jefferson County', + 'Dona Ana County', 'La Porte County') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3)) +GROUP BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +ORDER BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q11.sql b/sql/core/src/test/resources/tpcds/q11.sql new file mode 100755 index 0000000000000..3618fb14fa39c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q11.sql @@ -0,0 +1,68 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ss_ext_list_price - ss_ext_discount_amt) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id + , c_first_name + , c_last_name + , d_year + , c_preferred_cust_flag + , c_birth_country + , c_login + , c_email_address + , d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ws_ext_list_price - ws_ext_discount_amt) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + GROUP BY + c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, + c_login, c_email_address, d_year) +SELECT t_s_secyear.customer_preferred_cust_flag +FROM year_total t_s_firstyear + , year_total t_s_secyear + , year_total t_w_firstyear + , year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY t_s_secyear.customer_preferred_cust_flag +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q12.sql b/sql/core/src/test/resources/tpcds/q12.sql new file mode 100755 index 0000000000000..0382737f5aa2c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q12.sql @@ -0,0 +1,22 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) AS itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + web_sales, item, date_dim +WHERE + ws_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q13.sql b/sql/core/src/test/resources/tpcds/q13.sql new file mode 100755 index 0000000000000..32dc9e26097ba --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q13.sql @@ -0,0 +1,49 @@ +SELECT + avg(ss_quantity), + avg(ss_ext_sales_price), + avg(ss_ext_wholesale_cost), + sum(ss_ext_wholesale_cost) +FROM store_sales + , store + , customer_demographics + , household_demographics + , customer_address + , date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND ((ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'M' + AND cd_education_status = 'Advanced Degree' + AND ss_sales_price BETWEEN 100.00 AND 150.00 + AND hd_dep_count = 3 +) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'S' + AND cd_education_status = 'College' + AND ss_sales_price BETWEEN 50.00 AND 100.00 + AND hd_dep_count = 1 + ) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'W' + AND cd_education_status = '2 yr Degree' + AND ss_sales_price BETWEEN 150.00 AND 200.00 + AND hd_dep_count = 1 + )) + AND ((ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('TX', 'OH', 'TX') + AND ss_net_profit BETWEEN 100 AND 200 +) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('OR', 'NM', 'KY') + AND ss_net_profit BETWEEN 150 AND 300 + ) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('VA', 'TX', 'MS') + AND ss_net_profit BETWEEN 50 AND 250 + )) diff --git a/sql/core/src/test/resources/tpcds/q14a.sql b/sql/core/src/test/resources/tpcds/q14a.sql new file mode 100755 index 0000000000000..954ddd41be0e6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q14a.sql @@ -0,0 +1,120 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM ( + SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 2001 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT + channel, + i_brand_id, + i_class_id, + i_category_id, + sum(sales), + sum(number_sales) +FROM ( + SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales) + UNION ALL + SELECT + 'catalog' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(cs_quantity * cs_list_price) sales, + count(*) number_sales + FROM catalog_sales, item, date_dim + WHERE cs_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(cs_quantity * cs_list_price) > (SELECT average_sales FROM avg_sales) + UNION ALL + SELECT + 'web' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ws_quantity * ws_list_price) sales, + count(*) number_sales + FROM web_sales, item, date_dim + WHERE ws_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ws_quantity * ws_list_price) > (SELECT average_sales + FROM avg_sales) + ) y +GROUP BY ROLLUP (channel, i_brand_id, i_class_id, i_category_id) +ORDER BY channel, i_brand_id, i_class_id, i_category_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q14b.sql b/sql/core/src/test/resources/tpcds/q14b.sql new file mode 100755 index 0000000000000..929a8484bf9b4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q14b.sql @@ -0,0 +1,95 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM (SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT * +FROM + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 + 1 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) this_year, + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) last_year +WHERE this_year.i_brand_id = last_year.i_brand_id + AND this_year.i_class_id = last_year.i_class_id + AND this_year.i_category_id = last_year.i_category_id +ORDER BY this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q15.sql b/sql/core/src/test/resources/tpcds/q15.sql new file mode 100755 index 0000000000000..b8182e23b0195 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q15.sql @@ -0,0 +1,15 @@ +SELECT + ca_zip, + sum(cs_sales_price) +FROM catalog_sales, customer, customer_address, date_dim +WHERE cs_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND (substr(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', + '85392', '85460', '80348', '81792') + OR ca_state IN ('CA', 'WA', 'GA') + OR cs_sales_price > 500) + AND cs_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip +ORDER BY ca_zip +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q16.sql b/sql/core/src/test/resources/tpcds/q16.sql new file mode 100755 index 0000000000000..732ad0d848071 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q16.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT cs_order_number) AS `order count `, + sum(cs_ext_ship_cost) AS `total shipping cost `, + sum(cs_net_profit) AS `total net profit ` +FROM + catalog_sales cs1, date_dim, customer_address, call_center +WHERE + d_date BETWEEN '2002-02-01' AND (CAST('2002-02-01' AS DATE) + INTERVAL 60 days) + AND cs1.cs_ship_date_sk = d_date_sk + AND cs1.cs_ship_addr_sk = ca_address_sk + AND ca_state = 'GA' + AND cs1.cs_call_center_sk = cc_call_center_sk + AND cc_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + AND EXISTS(SELECT * + FROM catalog_sales cs2 + WHERE cs1.cs_order_number = cs2.cs_order_number + AND cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM catalog_returns cr1 + WHERE cs1.cs_order_number = cr1.cr_order_number) +ORDER BY count(DISTINCT cs_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q17.sql b/sql/core/src/test/resources/tpcds/q17.sql new file mode 100755 index 0000000000000..4d647f7956004 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q17.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_state, + count(ss_quantity) AS store_sales_quantitycount, + avg(ss_quantity) AS store_sales_quantityave, + stddev_samp(ss_quantity) AS store_sales_quantitystdev, + stddev_samp(ss_quantity) / avg(ss_quantity) AS store_sales_quantitycov, + count(sr_return_quantity) as_store_returns_quantitycount, + avg(sr_return_quantity) as_store_returns_quantityave, + stddev_samp(sr_return_quantity) as_store_returns_quantitystdev, + stddev_samp(sr_return_quantity) / avg(sr_return_quantity) AS store_returns_quantitycov, + count(cs_quantity) AS catalog_sales_quantitycount, + avg(cs_quantity) AS catalog_sales_quantityave, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitystdev, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitycov +FROM store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, store, item +WHERE d1.d_quarter_name = '2001Q1' + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') +GROUP BY i_item_id, i_item_desc, s_state +ORDER BY i_item_id, i_item_desc, s_state +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q18.sql b/sql/core/src/test/resources/tpcds/q18.sql new file mode 100755 index 0000000000000..4055c80fdef51 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q18.sql @@ -0,0 +1,28 @@ +SELECT + i_item_id, + ca_country, + ca_state, + ca_county, + avg(cast(cs_quantity AS DECIMAL(12, 2))) agg1, + avg(cast(cs_list_price AS DECIMAL(12, 2))) agg2, + avg(cast(cs_coupon_amt AS DECIMAL(12, 2))) agg3, + avg(cast(cs_sales_price AS DECIMAL(12, 2))) agg4, + avg(cast(cs_net_profit AS DECIMAL(12, 2))) agg5, + avg(cast(c_birth_year AS DECIMAL(12, 2))) agg6, + avg(cast(cd1.cd_dep_count AS DECIMAL(12, 2))) agg7 +FROM catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd1.cd_demo_sk AND + cs_bill_customer_sk = c_customer_sk AND + cd1.cd_gender = 'F' AND + cd1.cd_education_status = 'Unknown' AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_addr_sk = ca_address_sk AND + c_birth_month IN (1, 6, 8, 9, 12, 2) AND + d_year = 1998 AND + ca_state IN ('MS', 'IN', 'ND', 'OK', 'NM', 'VA', 'MS') +GROUP BY ROLLUP (i_item_id, ca_country, ca_state, ca_county) +ORDER BY ca_country, ca_state, ca_county, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q19.sql b/sql/core/src/test/resources/tpcds/q19.sql new file mode 100755 index 0000000000000..e38ab7f2683f4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q19.sql @@ -0,0 +1,19 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + i_manufact_id, + i_manufact, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item, customer, customer_address, store +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 8 + AND d_moy = 11 + AND d_year = 1998 + AND ss_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5) + AND ss_store_sk = s_store_sk +GROUP BY i_brand, i_brand_id, i_manufact_id, i_manufact +ORDER BY ext_price DESC, brand, brand_id, i_manufact_id, i_manufact +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q2.sql b/sql/core/src/test/resources/tpcds/q2.sql new file mode 100755 index 0000000000000..52c0e90c46740 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q2.sql @@ -0,0 +1,81 @@ +WITH wscs AS +( SELECT + sold_date_sk, + sales_price + FROM (SELECT + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + FROM web_sales) x + UNION ALL + (SELECT + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + FROM catalog_sales)), + wswscs AS + ( SELECT + d_week_seq, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN sales_price + ELSE NULL END) + sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN sales_price + ELSE NULL END) + mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN sales_price + ELSE NULL END) + tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN sales_price + ELSE NULL END) + wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN sales_price + ELSE NULL END) + thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN sales_price + ELSE NULL END) + fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN sales_price + ELSE NULL END) + sat_sales + FROM wscs, date_dim + WHERE d_date_sk = sold_date_sk + GROUP BY d_week_seq) +SELECT + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +FROM + (SELECT + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001) y, + (SELECT + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1) z +WHERE d_week_seq1 = d_week_seq2 - 53 +ORDER BY d_week_seq1 diff --git a/sql/core/src/test/resources/tpcds/q20.sql b/sql/core/src/test/resources/tpcds/q20.sql new file mode 100755 index 0000000000000..7ac6c7a75d8ea --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q20.sql @@ -0,0 +1,18 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(cs_ext_sales_price) AS itemrevenue, + sum(cs_ext_sales_price) * 100 / sum(sum(cs_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM catalog_sales, item, date_dim +WHERE cs_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) +AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q21.sql b/sql/core/src/test/resources/tpcds/q21.sql new file mode 100755 index 0000000000000..550881143f809 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q21.sql @@ -0,0 +1,25 @@ +SELECT * +FROM ( + SELECT + w_warehouse_name, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_after + FROM inventory, warehouse, item, date_dim + WHERE i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = inv_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) + GROUP BY w_warehouse_name, i_item_id) x +WHERE (CASE WHEN inv_before > 0 + THEN inv_after / inv_before + ELSE NULL + END) BETWEEN 2.0 / 3.0 AND 3.0 / 2.0 +ORDER BY w_warehouse_name, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q22.sql b/sql/core/src/test/resources/tpcds/q22.sql new file mode 100755 index 0000000000000..add3b41f7c76c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q22.sql @@ -0,0 +1,14 @@ +SELECT + i_product_name, + i_brand, + i_class, + i_category, + avg(inv_quantity_on_hand) qoh +FROM inventory, date_dim, item, warehouse +WHERE inv_date_sk = d_date_sk + AND inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 +GROUP BY ROLLUP (i_product_name, i_brand, i_class, i_category) +ORDER BY qoh, i_product_name, i_brand, i_class, i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q23a.sql b/sql/core/src/test/resources/tpcds/q23a.sql new file mode 100755 index 0000000000000..37791f643375c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q23a.sql @@ -0,0 +1,53 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales, customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT sum(sales) +FROM ((SELECT cs_quantity * cs_list_price sales +FROM catalog_sales, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer)) + UNION ALL + (SELECT ws_quantity * ws_list_price sales + FROM web_sales, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer))) y +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q23b.sql b/sql/core/src/test/resources/tpcds/q23b.sql new file mode 100755 index 0000000000000..01150197af2ba --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q23b.sql @@ -0,0 +1,68 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales + , customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT + c_last_name, + c_first_name, + sales +FROM ((SELECT + c_last_name, + c_first_name, + sum(cs_quantity * cs_list_price) sales +FROM catalog_sales, customer, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer) + AND cs_bill_customer_sk = c_customer_sk +GROUP BY c_last_name, c_first_name) + UNION ALL + (SELECT + c_last_name, + c_first_name, + sum(ws_quantity * ws_list_price) sales + FROM web_sales, customer, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer) + AND ws_bill_customer_sk = c_customer_sk + GROUP BY c_last_name, c_first_name)) y +ORDER BY c_last_name, c_first_name, sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q24a.sql b/sql/core/src/test/resources/tpcds/q24a.sql new file mode 100755 index 0000000000000..bcc189486634d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q24a.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, + i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'pale' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/sql/core/src/test/resources/tpcds/q24b.sql b/sql/core/src/test/resources/tpcds/q24b.sql new file mode 100755 index 0000000000000..830eb670bcdd2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q24b.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, + i_color, i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'chiffon' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/sql/core/src/test/resources/tpcds/q25.sql b/sql/core/src/test/resources/tpcds/q25.sql new file mode 100755 index 0000000000000..a4d78a3c56adc --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q25.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_net_profit) AS store_sales_profit, + sum(sr_net_loss) AS store_returns_loss, + sum(cs_net_profit) AS catalog_sales_profit +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, + store, item +WHERE + d1.d_moy = 4 + AND d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 4 AND 10 + AND d2.d_year = 2001 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_moy BETWEEN 4 AND 10 + AND d3.d_year = 2001 +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 \ No newline at end of file diff --git a/sql/core/src/test/resources/tpcds/q26.sql b/sql/core/src/test/resources/tpcds/q26.sql new file mode 100755 index 0000000000000..6d395a1d791dd --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q26.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 +FROM catalog_sales, customer_demographics, date_dim, item, promotion +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd_demo_sk AND + cs_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q27.sql b/sql/core/src/test/resources/tpcds/q27.sql new file mode 100755 index 0000000000000..b0e2fd95fd159 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q27.sql @@ -0,0 +1,21 @@ +SELECT + i_item_id, + s_state, + grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, store, item +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_store_sk = s_store_sk AND + ss_cdemo_sk = cd_demo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + d_year = 2002 AND + s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_item_id, s_state) +ORDER BY i_item_id, s_state +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q28.sql b/sql/core/src/test/resources/tpcds/q28.sql new file mode 100755 index 0000000000000..f34c2bb0e34e1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q28.sql @@ -0,0 +1,56 @@ +SELECT * +FROM (SELECT + avg(ss_list_price) B1_LP, + count(ss_list_price) B1_CNT, + count(DISTINCT ss_list_price) B1_CNTD +FROM store_sales +WHERE ss_quantity BETWEEN 0 AND 5 + AND (ss_list_price BETWEEN 8 AND 8 + 10 + OR ss_coupon_amt BETWEEN 459 AND 459 + 1000 + OR ss_wholesale_cost BETWEEN 57 AND 57 + 20)) B1, + (SELECT + avg(ss_list_price) B2_LP, + count(ss_list_price) B2_CNT, + count(DISTINCT ss_list_price) B2_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 6 AND 10 + AND (ss_list_price BETWEEN 90 AND 90 + 10 + OR ss_coupon_amt BETWEEN 2323 AND 2323 + 1000 + OR ss_wholesale_cost BETWEEN 31 AND 31 + 20)) B2, + (SELECT + avg(ss_list_price) B3_LP, + count(ss_list_price) B3_CNT, + count(DISTINCT ss_list_price) B3_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 11 AND 15 + AND (ss_list_price BETWEEN 142 AND 142 + 10 + OR ss_coupon_amt BETWEEN 12214 AND 12214 + 1000 + OR ss_wholesale_cost BETWEEN 79 AND 79 + 20)) B3, + (SELECT + avg(ss_list_price) B4_LP, + count(ss_list_price) B4_CNT, + count(DISTINCT ss_list_price) B4_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 16 AND 20 + AND (ss_list_price BETWEEN 135 AND 135 + 10 + OR ss_coupon_amt BETWEEN 6071 AND 6071 + 1000 + OR ss_wholesale_cost BETWEEN 38 AND 38 + 20)) B4, + (SELECT + avg(ss_list_price) B5_LP, + count(ss_list_price) B5_CNT, + count(DISTINCT ss_list_price) B5_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 25 + AND (ss_list_price BETWEEN 122 AND 122 + 10 + OR ss_coupon_amt BETWEEN 836 AND 836 + 1000 + OR ss_wholesale_cost BETWEEN 17 AND 17 + 20)) B5, + (SELECT + avg(ss_list_price) B6_LP, + count(ss_list_price) B6_CNT, + count(DISTINCT ss_list_price) B6_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 26 AND 30 + AND (ss_list_price BETWEEN 154 AND 154 + 10 + OR ss_coupon_amt BETWEEN 7326 AND 7326 + 1000 + OR ss_wholesale_cost BETWEEN 7 AND 7 + 20)) B6 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q29.sql b/sql/core/src/test/resources/tpcds/q29.sql new file mode 100755 index 0000000000000..3f1fd553f6da8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q29.sql @@ -0,0 +1,32 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_quantity) AS store_sales_quantity, + sum(sr_return_quantity) AS store_returns_quantity, + sum(cs_quantity) AS catalog_sales_quantity +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, + date_dim d3, store, item +WHERE + d1.d_moy = 9 + AND d1.d_year = 1999 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 9 AND 9 + 3 + AND d2.d_year = 1999 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_year IN (1999, 1999 + 1, 1999 + 2) +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q3.sql b/sql/core/src/test/resources/tpcds/q3.sql new file mode 100755 index 0000000000000..181509df9deb7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q3.sql @@ -0,0 +1,13 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + SUM(ss_ext_sales_price) sum_agg +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manufact_id = 128 + AND dt.d_moy = 11 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, sum_agg DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q30.sql b/sql/core/src/test/resources/tpcds/q30.sql new file mode 100755 index 0000000000000..986bef566d2c8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q30.sql @@ -0,0 +1,35 @@ +WITH customer_total_return AS +(SELECT + wr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(wr_return_amt) AS ctr_total_return + FROM web_returns, date_dim, customer_address + WHERE wr_returned_date_sk = d_date_sk + AND d_year = 2002 + AND wr_returning_addr_sk = ca_address_sk + GROUP BY wr_returning_customer_sk, ca_state) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_day, + c_birth_month, + c_birth_year, + c_birth_country, + c_login, + c_email_address, + c_last_review_date, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag + , c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address + , c_last_review_date, ctr_total_return +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q31.sql b/sql/core/src/test/resources/tpcds/q31.sql new file mode 100755 index 0000000000000..3e543d5436407 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q31.sql @@ -0,0 +1,60 @@ +WITH ss AS +(SELECT + ca_county, + d_qoy, + d_year, + sum(ss_ext_sales_price) AS store_sales + FROM store_sales, date_dim, customer_address + WHERE ss_sold_date_sk = d_date_sk + AND ss_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year), + ws AS + (SELECT + ca_county, + d_qoy, + d_year, + sum(ws_ext_sales_price) AS web_sales + FROM web_sales, date_dim, customer_address + WHERE ws_sold_date_sk = d_date_sk + AND ws_bill_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year) +SELECT + ss1.ca_county, + ss1.d_year, + ws2.web_sales / ws1.web_sales web_q1_q2_increase, + ss2.store_sales / ss1.store_sales store_q1_q2_increase, + ws3.web_sales / ws2.web_sales web_q2_q3_increase, + ss3.store_sales / ss2.store_sales store_q2_q3_increase +FROM + ss ss1, ss ss2, ss ss3, ws ws1, ws ws2, ws ws3 +WHERE + ss1.d_qoy = 1 + AND ss1.d_year = 2000 + AND ss1.ca_county = ss2.ca_county + AND ss2.d_qoy = 2 + AND ss2.d_year = 2000 + AND ss2.ca_county = ss3.ca_county + AND ss3.d_qoy = 3 + AND ss3.d_year = 2000 + AND ss1.ca_county = ws1.ca_county + AND ws1.d_qoy = 1 + AND ws1.d_year = 2000 + AND ws1.ca_county = ws2.ca_county + AND ws2.d_qoy = 2 + AND ws2.d_year = 2000 + AND ws1.ca_county = ws3.ca_county + AND ws3.d_qoy = 3 + AND ws3.d_year = 2000 + AND CASE WHEN ws1.web_sales > 0 + THEN ws2.web_sales / ws1.web_sales + ELSE NULL END + > CASE WHEN ss1.store_sales > 0 + THEN ss2.store_sales / ss1.store_sales + ELSE NULL END + AND CASE WHEN ws2.web_sales > 0 + THEN ws3.web_sales / ws2.web_sales + ELSE NULL END + > CASE WHEN ss2.store_sales > 0 + THEN ss3.store_sales / ss2.store_sales + ELSE NULL END +ORDER BY ss1.ca_county diff --git a/sql/core/src/test/resources/tpcds/q32.sql b/sql/core/src/test/resources/tpcds/q32.sql new file mode 100755 index 0000000000000..1a907961e74bb --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q32.sql @@ -0,0 +1,15 @@ +SELECT 1 AS `excess discount amount ` +FROM + catalog_sales, item, date_dim +WHERE + i_manufact_id = 977 + AND i_item_sk = cs_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk + AND cs_ext_discount_amt > ( + SELECT 1.3 * avg(cs_ext_discount_amt) + FROM catalog_sales, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27]' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q33.sql b/sql/core/src/test/resources/tpcds/q33.sql new file mode 100755 index 0000000000000..d24856aa5c1eb --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q33.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_manufact_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), cs AS +(SELECT + i_manufact_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN ( + SELECT i_manufact_id + FROM item + WHERE + i_category IN ('Electronics')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), + ws AS ( + SELECT + i_manufact_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id) +SELECT + i_manufact_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_manufact_id +ORDER BY total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q34.sql b/sql/core/src/test/resources/tpcds/q34.sql new file mode 100755 index 0000000000000..33396bf16e574 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q34.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (date_dim.d_dom BETWEEN 1 AND 3 OR date_dim.d_dom BETWEEN 25 AND 28) + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND (CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL + END) > 1.2 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', + 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + GROUP BY ss_ticket_number, ss_customer_sk) dn, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 15 AND 20 +ORDER BY c_last_name, c_first_name, c_salutation, c_preferred_cust_flag DESC diff --git a/sql/core/src/test/resources/tpcds/q35.sql b/sql/core/src/test/resources/tpcds/q35.sql new file mode 100755 index 0000000000000..cfe4342d8be86 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q35.sql @@ -0,0 +1,46 @@ +SELECT + ca_state, + cd_gender, + cd_marital_status, + count(*) cnt1, + min(cd_dep_count), + max(cd_dep_count), + avg(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + min(cd_dep_employed_count), + max(cd_dep_employed_count), + avg(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + min(cd_dep_college_count), + max(cd_dep_college_count), + avg(cd_dep_college_count) +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4)) +GROUP BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +ORDER BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q36.sql b/sql/core/src/test/resources/tpcds/q36.sql new file mode 100755 index 0000000000000..a8f93df76a34b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q36.sql @@ -0,0 +1,26 @@ +SELECT + sum(ss_net_profit) / sum(ss_ext_sales_price) AS gross_margin, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ss_net_profit) / sum(ss_ext_sales_price) ASC) AS rank_within_parent +FROM + store_sales, date_dim d1, item, store +WHERE + d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN i_category END + , rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q37.sql b/sql/core/src/test/resources/tpcds/q37.sql new file mode 100755 index 0000000000000..11b3821fa48b8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q37.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, catalog_sales +WHERE i_current_price BETWEEN 68 AND 68 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-02-01' AS DATE) AND (cast('2000-02-01' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (677, 940, 694, 808) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND cs_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q38.sql b/sql/core/src/test/resources/tpcds/q38.sql new file mode 100755 index 0000000000000..1c8d53ee2bbfc --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q38.sql @@ -0,0 +1,30 @@ +SELECT count(*) +FROM ( + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM store_sales, date_dim, customer + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + ) hot_cust +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q39a.sql b/sql/core/src/test/resources/tpcds/q39a.sql new file mode 100755 index 0000000000000..9fc4c1701cf21 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q39a.sql @@ -0,0 +1,47 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/sql/core/src/test/resources/tpcds/q39b.sql b/sql/core/src/test/resources/tpcds/q39b.sql new file mode 100755 index 0000000000000..6f8493029fab4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q39b.sql @@ -0,0 +1,48 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 + AND inv1.cov > 1.5 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/sql/core/src/test/resources/tpcds/q4.sql b/sql/core/src/test/resources/tpcds/q4.sql new file mode 100755 index 0000000000000..b9f27fbc9a4a6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q4.sql @@ -0,0 +1,120 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + + ss_ext_sales_price) / 2) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + + cs_ext_sales_price) / 2)) year_total, + 'c' sale_type + FROM customer, catalog_sales, date_dim + WHERE c_customer_sk = cs_bill_customer_sk AND cs_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / + 2)) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk AND ws_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +FROM year_total t_s_firstyear, year_total t_s_secyear, year_total t_c_firstyear, + year_total t_c_secyear, year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_c_secyear.customer_id + AND t_s_firstyear.customer_id = t_c_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_c_firstyear.sale_type = 'c' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_c_secyear.sale_type = 'c' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_c_firstyear.dyear = 2001 + AND t_c_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_c_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END +ORDER BY + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q40.sql b/sql/core/src/test/resources/tpcds/q40.sql new file mode 100755 index 0000000000000..66d8b73ac1c15 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q40.sql @@ -0,0 +1,25 @@ +SELECT + w_state, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_after +FROM + catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + , warehouse, item, date_dim +WHERE + i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = cs_item_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) +GROUP BY w_state, i_item_id +ORDER BY w_state, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q41.sql b/sql/core/src/test/resources/tpcds/q41.sql new file mode 100755 index 0000000000000..25e317e0e201a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q41.sql @@ -0,0 +1,49 @@ +SELECT DISTINCT (i_product_name) +FROM item i1 +WHERE i_manufact_id BETWEEN 738 AND 738 + 40 + AND (SELECT count(*) AS item_cnt +FROM item +WHERE (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'powder' OR i_color = 'khaki') AND + (i_units = 'Ounce' OR i_units = 'Oz') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'brown' OR i_color = 'honeydew') AND + (i_units = 'Bunch' OR i_units = 'Ton') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'floral' OR i_color = 'deep') AND + (i_units = 'N/A' OR i_units = 'Dozen') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'light' OR i_color = 'cornflower') AND + (i_units = 'Box' OR i_units = 'Pound') AND + (i_size = 'medium' OR i_size = 'extra large') + ))) OR + (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'midnight' OR i_color = 'snow') AND + (i_units = 'Pallet' OR i_units = 'Gross') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'cyan' OR i_color = 'papaya') AND + (i_units = 'Cup' OR i_units = 'Dram') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'orange' OR i_color = 'frosted') AND + (i_units = 'Each' OR i_units = 'Tbl') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'forest' OR i_color = 'ghost') AND + (i_units = 'Lb' OR i_units = 'Bundle') AND + (i_size = 'medium' OR i_size = 'extra large') + )))) > 0 +ORDER BY i_product_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q42.sql b/sql/core/src/test/resources/tpcds/q42.sql new file mode 100755 index 0000000000000..4d2e71760d870 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q42.sql @@ -0,0 +1,18 @@ +SELECT + dt.d_year, + item.i_category_id, + item.i_category, + sum(ss_ext_sales_price) +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year + , item.i_category_id + , item.i_category +ORDER BY sum(ss_ext_sales_price) DESC, dt.d_year + , item.i_category_id + , item.i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q43.sql b/sql/core/src/test/resources/tpcds/q43.sql new file mode 100755 index 0000000000000..45411772c1b54 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q43.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + s_store_id, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales +FROM date_dim, store_sales, store +WHERE d_date_sk = ss_sold_date_sk AND + s_store_sk = ss_store_sk AND + s_gmt_offset = -5 AND + d_year = 2000 +GROUP BY s_store_name, s_store_id +ORDER BY s_store_name, s_store_id, sun_sales, mon_sales, tue_sales, wed_sales, + thu_sales, fri_sales, sat_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q44.sql b/sql/core/src/test/resources/tpcds/q44.sql new file mode 100755 index 0000000000000..379e604788625 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q44.sql @@ -0,0 +1,46 @@ +SELECT + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +FROM (SELECT * +FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col ASC) rnk +FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +FROM store_sales ss1 +WHERE ss_store_sk = 4 +GROUP BY ss_item_sk +HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col +FROM store_sales +WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL +GROUP BY ss_store_sk)) V1) V11 +WHERE rnk < 11) asceding, + (SELECT * + FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col DESC) rnk + FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + FROM store_sales ss1 + WHERE ss_store_sk = 4 + GROUP BY ss_item_sk + HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col + FROM store_sales + WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL + GROUP BY ss_store_sk)) V2) V21 + WHERE rnk < 11) descending, + item i1, item i2 +WHERE asceding.rnk = descending.rnk + AND i1.i_item_sk = asceding.item_sk + AND i2.i_item_sk = descending.item_sk +ORDER BY asceding.rnk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q45.sql b/sql/core/src/test/resources/tpcds/q45.sql new file mode 100755 index 0000000000000..907438f196c4c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q45.sql @@ -0,0 +1,21 @@ +SELECT + ca_zip, + ca_city, + sum(ws_sales_price) +FROM web_sales, customer, customer_address, date_dim, item +WHERE ws_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND ws_item_sk = i_item_sk + AND (substr(ca_zip, 1, 5) IN + ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') + OR + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) +) + AND ws_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip, ca_city +ORDER BY ca_zip, ca_city +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q46.sql b/sql/core/src/test/resources/tpcds/q46.sql new file mode 100755 index 0000000000000..0911677dff206 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q46.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics, customer_address + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_dow IN (6, 0) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Fairview', 'Midway', 'Fairview', 'Fairview', 'Fairview') + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q47.sql b/sql/core/src/test/resources/tpcds/q47.sql new file mode 100755 index 0000000000000..cfc37a4cece66 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q47.sql @@ -0,0 +1,63 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + s_store_name, + s_company_name, + d_year, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name + ORDER BY d_year, d_moy) rn + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.s_store_name, + v1.s_company_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.s_store_name = v1_lag.s_store_name AND + v1.s_store_name = v1_lead.s_store_name AND + v1.s_company_name = v1_lag.s_company_name AND + v1.s_company_name = v1_lead.s_company_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q48.sql b/sql/core/src/test/resources/tpcds/q48.sql new file mode 100755 index 0000000000000..fdb9f38e294f7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q48.sql @@ -0,0 +1,63 @@ +SELECT sum(ss_quantity) +FROM store_sales, store, customer_demographics, customer_address, date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND + ( + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'M' + AND + cd_education_status = '4 yr Degree' + AND + ss_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'D' + AND + cd_education_status = '2 yr Degree' + AND + ss_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'S' + AND + cd_education_status = 'College' + AND + ss_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('CO', 'OH', 'TX') + AND ss_net_profit BETWEEN 0 AND 2000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('OR', 'MN', 'KY') + AND ss_net_profit BETWEEN 150 AND 3000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('VA', 'CA', 'MS') + AND ss_net_profit BETWEEN 50 AND 25000 + ) + ) diff --git a/sql/core/src/test/resources/tpcds/q49.sql b/sql/core/src/test/resources/tpcds/q49.sql new file mode 100755 index 0000000000000..9568d8b92d10a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q49.sql @@ -0,0 +1,126 @@ +SELECT + 'web' AS channel, + web.item, + web.return_ratio, + web.return_rank, + web.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + ws.ws_item_sk AS item, + (cast(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + web_sales ws LEFT OUTER JOIN web_returns wr + ON (ws.ws_order_number = wr.wr_order_number AND + ws.ws_item_sk = wr.wr_item_sk) + , date_dim + WHERE + wr.wr_return_amt > 10000 + AND ws.ws_net_profit > 1 + AND ws.ws_net_paid > 0 + AND ws.ws_quantity > 0 + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY ws.ws_item_sk + ) in_web + ) web +WHERE (web.return_rank <= 10 OR web.currency_rank <= 10) +UNION +SELECT + 'catalog' AS channel, + catalog.item, + catalog.return_ratio, + catalog.return_rank, + catalog.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + cs.cs_item_sk AS item, + (cast(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + catalog_sales cs LEFT OUTER JOIN catalog_returns cr + ON (cs.cs_order_number = cr.cr_order_number AND + cs.cs_item_sk = cr.cr_item_sk) + , date_dim + WHERE + cr.cr_return_amount > 10000 + AND cs.cs_net_profit > 1 + AND cs.cs_net_paid > 0 + AND cs.cs_quantity > 0 + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY cs.cs_item_sk + ) in_cat + ) catalog +WHERE (catalog.return_rank <= 10 OR catalog.currency_rank <= 10) +UNION +SELECT + 'store' AS channel, + store.item, + store.return_ratio, + store.return_rank, + store.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + sts.ss_item_sk AS item, + (cast(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + store_sales sts LEFT OUTER JOIN store_returns sr + ON (sts.ss_ticket_number = sr.sr_ticket_number AND sts.ss_item_sk = sr.sr_item_sk) + , date_dim + WHERE + sr.sr_return_amt > 10000 + AND sts.ss_net_profit > 1 + AND sts.ss_net_paid > 0 + AND sts.ss_quantity > 0 + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY sts.ss_item_sk + ) in_store + ) store +WHERE (store.return_rank <= 10 OR store.currency_rank <= 10) +ORDER BY 1, 4, 5 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q5.sql b/sql/core/src/test/resources/tpcds/q5.sql new file mode 100755 index 0000000000000..b87cf3a44827b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q5.sql @@ -0,0 +1,131 @@ +WITH ssr AS +( SELECT + s_store_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ss_store_sk AS store_sk, + ss_sold_date_sk AS date_sk, + ss_ext_sales_price AS sales_price, + ss_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM store_sales + UNION ALL + SELECT + sr_store_sk AS store_sk, + sr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + sr_return_amt AS return_amt, + sr_net_loss AS net_loss + FROM store_returns) + salesreturns, date_dim, store + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND store_sk = s_store_sk + GROUP BY s_store_id), + csr AS + ( SELECT + cp_catalog_page_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + cs_catalog_page_sk AS page_sk, + cs_sold_date_sk AS date_sk, + cs_ext_sales_price AS sales_price, + cs_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM catalog_sales + UNION ALL + SELECT + cr_catalog_page_sk AS page_sk, + cr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + cr_return_amount AS return_amt, + cr_net_loss AS net_loss + FROM catalog_returns + ) salesreturns, date_dim, catalog_page + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND page_sk = cp_catalog_page_sk + GROUP BY cp_catalog_page_id) + , + wsr AS + ( SELECT + web_site_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ws_web_site_sk AS wsr_web_site_sk, + ws_sold_date_sk AS date_sk, + ws_ext_sales_price AS sales_price, + ws_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM web_sales + UNION ALL + SELECT + ws_web_site_sk AS wsr_web_site_sk, + wr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + wr_return_amt AS return_amt, + wr_net_loss AS net_loss + FROM web_returns + LEFT OUTER JOIN web_sales ON + (wr_item_sk = ws_item_sk + AND wr_order_number = ws_order_number) + ) salesreturns, date_dim, web_site + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND wsr_web_site_sk = web_site_sk + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + concat('store', s_store_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', cp_catalog_page_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM wsr + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q50.sql b/sql/core/src/test/resources/tpcds/q50.sql new file mode 100755 index 0000000000000..f1d4b15449edd --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q50.sql @@ -0,0 +1,47 @@ +SELECT + s_store_name, + s_company_id, + s_street_number, + s_street_name, + s_street_type, + s_suite_number, + s_city, + s_county, + s_state, + s_zip, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND + (sr_returned_date_sk - ss_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND + (sr_returned_date_sk - ss_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 90) AND + (sr_returned_date_sk - ss_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + store_sales, store_returns, store, date_dim d1, date_dim d2 +WHERE + d2.d_year = 2001 + AND d2.d_moy = 8 + AND ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND sr_returned_date_sk = d2.d_date_sk + AND ss_customer_sk = sr_customer_sk + AND ss_store_sk = s_store_sk +GROUP BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +ORDER BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q51.sql b/sql/core/src/test/resources/tpcds/q51.sql new file mode 100755 index 0000000000000..62b003eb67b9b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q51.sql @@ -0,0 +1,55 @@ +WITH web_v1 AS ( + SELECT + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + OVER (PARTITION BY ws_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_item_sk IS NOT NULL + GROUP BY ws_item_sk, d_date), + store_v1 AS ( + SELECT + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + OVER (PARTITION BY ss_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ss_item_sk IS NOT NULL + GROUP BY ss_item_sk, d_date) +SELECT * +FROM (SELECT + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) web_cumulative, + max(store_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) store_cumulative +FROM (SELECT + CASE WHEN web.item_sk IS NOT NULL + THEN web.item_sk + ELSE store.item_sk END item_sk, + CASE WHEN web.d_date IS NOT NULL + THEN web.d_date + ELSE store.d_date END d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +FROM web_v1 web FULL OUTER JOIN store_v1 store ON (web.item_sk = store.item_sk + AND web.d_date = store.d_date) + ) x) y +WHERE web_cumulative > store_cumulative +ORDER BY item_sk, d_date +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q52.sql b/sql/core/src/test/resources/tpcds/q52.sql new file mode 100755 index 0000000000000..467d1ae050455 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q52.sql @@ -0,0 +1,14 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, ext_price DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q53.sql b/sql/core/src/test/resources/tpcds/q53.sql new file mode 100755 index 0000000000000..b42c68dcf871b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q53.sql @@ -0,0 +1,30 @@ +SELECT * +FROM + (SELECT + i_manufact_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manufact_id) avg_quarterly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, + 1200 + 7, 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) AND + ((i_category IN ('Books', 'Children', 'Electronics') AND + i_class IN ('personal', 'portable', 'reference', 'self-help') AND + i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR + (i_category IN ('Women', 'Music', 'Men') AND + i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND + i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) + GROUP BY i_manufact_id, d_qoy) tmp1 +WHERE CASE WHEN avg_quarterly_sales > 0 + THEN abs(sum_sales - avg_quarterly_sales) / avg_quarterly_sales + ELSE NULL END > 0.1 +ORDER BY avg_quarterly_sales, + sum_sales, + i_manufact_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q54.sql b/sql/core/src/test/resources/tpcds/q54.sql new file mode 100755 index 0000000000000..897237fb6e10b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q54.sql @@ -0,0 +1,61 @@ +WITH my_customers AS ( + SELECT DISTINCT + c_customer_sk, + c_current_addr_sk + FROM + (SELECT + cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales + UNION ALL + SELECT + ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + FROM web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + WHERE sold_date_sk = d_date_sk + AND item_sk = i_item_sk + AND i_category = 'Women' + AND i_class = 'maternity' + AND c_customer_sk = cs_or_ws_sales.customer_sk + AND d_moy = 12 + AND d_year = 1998 +) + , my_revenue AS ( + SELECT + c_customer_sk, + sum(ss_ext_sales_price) AS revenue + FROM my_customers, + store_sales, + customer_address, + store, + date_dim + WHERE c_current_addr_sk = ca_address_sk + AND ca_county = s_county + AND ca_state = s_state + AND ss_sold_date_sk = d_date_sk + AND c_customer_sk = ss_customer_sk + AND d_month_seq BETWEEN (SELECT DISTINCT d_month_seq + 1 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + AND (SELECT DISTINCT d_month_seq + 3 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + GROUP BY c_customer_sk +) + , segments AS +(SELECT cast((revenue / 50) AS INT) AS segment + FROM my_revenue) +SELECT + segment, + count(*) AS num_customers, + segment * 50 AS segment_base +FROM segments +GROUP BY segment +ORDER BY segment, num_customers +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q55.sql b/sql/core/src/test/resources/tpcds/q55.sql new file mode 100755 index 0000000000000..bc5d888c9ac58 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q55.sql @@ -0,0 +1,13 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 28 + AND d_moy = 11 + AND d_year = 1999 +GROUP BY i_brand, i_brand_id +ORDER BY ext_price DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q56.sql b/sql/core/src/test/resources/tpcds/q56.sql new file mode 100755 index 0000000000000..2fa1738dcfee6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q56.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM + catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q57.sql b/sql/core/src/test/resources/tpcds/q57.sql new file mode 100755 index 0000000000000..cf70d4b905b55 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q57.sql @@ -0,0 +1,56 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + cc_name, + d_year, + d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) + OVER + (PARTITION BY i_category, i_brand, cc_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, cc_name + ORDER BY d_year, d_moy) rn + FROM item, catalog_sales, date_dim, call_center + WHERE cs_item_sk = i_item_sk AND + cs_sold_date_sk = d_date_sk AND + cc_call_center_sk = cs_call_center_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + cc_name, d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.cc_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.cc_name = v1_lag.cc_name AND + v1.cc_name = v1_lead.cc_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q58.sql b/sql/core/src/test/resources/tpcds/q58.sql new file mode 100755 index 0000000000000..5f63f33dc927c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q58.sql @@ -0,0 +1,59 @@ +WITH ss_items AS +(SELECT + i_item_id item_id, + sum(ss_ext_sales_price) ss_item_rev + FROM store_sales, item, date_dim + WHERE ss_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ss_sold_date_sk = d_date_sk + GROUP BY i_item_id), + cs_items AS + (SELECT + i_item_id item_id, + sum(cs_ext_sales_price) cs_item_rev + FROM catalog_sales, item, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND cs_sold_date_sk = d_date_sk + GROUP BY i_item_id), + ws_items AS + (SELECT + i_item_id item_id, + sum(ws_ext_sales_price) ws_item_rev + FROM web_sales, item, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ws_sold_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + ss_items.item_id, + ss_item_rev, + ss_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ss_dev, + cs_item_rev, + cs_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 cs_dev, + ws_item_rev, + ws_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ws_dev, + (ss_item_rev + cs_item_rev + ws_item_rev) / 3 average +FROM ss_items, cs_items, ws_items +WHERE ss_items.item_id = cs_items.item_id + AND ss_items.item_id = ws_items.item_id + AND ss_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev + AND ss_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND cs_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND cs_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND ws_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND ws_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev +ORDER BY item_id, ss_item_rev +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q59.sql b/sql/core/src/test/resources/tpcds/q59.sql new file mode 100755 index 0000000000000..3cef2027680b0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q59.sql @@ -0,0 +1,75 @@ +WITH wss AS +(SELECT + d_week_seq, + ss_store_sk, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + GROUP BY d_week_seq, ss_store_sk +) +SELECT + s_store_name1, + s_store_id1, + d_week_seq1, + sun_sales1 / sun_sales2, + mon_sales1 / mon_sales2, + tue_sales1 / tue_sales2, + wed_sales1 / wed_sales2, + thu_sales1 / thu_sales2, + fri_sales1 / fri_sales2, + sat_sales1 / sat_sales2 +FROM + (SELECT + s_store_name s_store_name1, + wss.d_week_seq d_week_seq1, + s_store_id s_store_id1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 AND 1212 + 11) y, + (SELECT + s_store_name s_store_name2, + wss.d_week_seq d_week_seq2, + s_store_id s_store_id2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 + 12 AND 1212 + 23) x +WHERE s_store_id1 = s_store_id2 + AND d_week_seq1 = d_week_seq2 - 52 +ORDER BY s_store_name1, s_store_id1, d_week_seq1 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q6.sql b/sql/core/src/test/resources/tpcds/q6.sql new file mode 100755 index 0000000000000..f0f5cf05aebda --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q6.sql @@ -0,0 +1,21 @@ +SELECT + a.ca_state state, + count(*) cnt +FROM + customer_address a, customer c, store_sales s, date_dim d, item i +WHERE a.ca_address_sk = c.c_current_addr_sk + AND c.c_customer_sk = s.ss_customer_sk + AND s.ss_sold_date_sk = d.d_date_sk + AND s.ss_item_sk = i.i_item_sk + AND d.d_month_seq = + (SELECT DISTINCT (d_month_seq) + FROM date_dim + WHERE d_year = 2000 AND d_moy = 1) + AND i.i_current_price > 1.2 * + (SELECT avg(j.i_current_price) + FROM item j + WHERE j.i_category = i.i_category) +GROUP BY a.ca_state +HAVING count(*) >= 10 +ORDER BY cnt +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q60.sql b/sql/core/src/test/resources/tpcds/q60.sql new file mode 100755 index 0000000000000..41b963f44ba13 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q60.sql @@ -0,0 +1,62 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY i_item_id, total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q61.sql b/sql/core/src/test/resources/tpcds/q61.sql new file mode 100755 index 0000000000000..b0a872b4b80e1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q61.sql @@ -0,0 +1,33 @@ +SELECT + promotions, + total, + cast(promotions AS DECIMAL(15, 4)) / cast(total AS DECIMAL(15, 4)) * 100 +FROM + (SELECT sum(ss_ext_sales_price) promotions + FROM store_sales, store, promotion, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_promo_sk = p_promo_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) promotional_sales, + (SELECT sum(ss_ext_sales_price) total + FROM store_sales, store, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) all_sales +ORDER BY promotions, total +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q62.sql b/sql/core/src/test/resources/tpcds/q62.sql new file mode 100755 index 0000000000000..8a414f154bdc8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q62.sql @@ -0,0 +1,35 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + web_name, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND + (ws_ship_date_sk - ws_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND + (ws_ship_date_sk - ws_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND + (ws_ship_date_sk - ws_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + web_sales, warehouse, ship_mode, web_site, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_ship_date_sk = d_date_sk + AND ws_warehouse_sk = w_warehouse_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND ws_web_site_sk = web_site_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +ORDER BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q63.sql b/sql/core/src/test/resources/tpcds/q63.sql new file mode 100755 index 0000000000000..ef6867e0a9451 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q63.sql @@ -0,0 +1,31 @@ +SELECT * +FROM (SELECT + i_manager_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manager_id) avg_monthly_sales +FROM item + , store_sales + , date_dim + , store +WHERE ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, 1200 + 7, + 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) + AND ((i_category IN ('Books', 'Children', 'Electronics') + AND i_class IN ('personal', 'portable', 'refernece', 'self-help') + AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR (i_category IN ('Women', 'Music', 'Men') + AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') + AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) +GROUP BY i_manager_id, d_moy) tmp1 +WHERE CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY i_manager_id + , avg_monthly_sales + , sum_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q64.sql b/sql/core/src/test/resources/tpcds/q64.sql new file mode 100755 index 0000000000000..8ec1d31b61afe --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q64.sql @@ -0,0 +1,92 @@ +WITH cs_ui AS +(SELECT + cs_item_sk, + sum(cs_ext_list_price) AS sale, + sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit) AS refund + FROM catalog_sales + , catalog_returns + WHERE cs_item_sk = cr_item_sk + AND cs_order_number = cr_order_number + GROUP BY cs_item_sk + HAVING sum(cs_ext_list_price) > 2 * sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit)), + cross_sales AS + (SELECT + i_product_name product_name, + i_item_sk item_sk, + s_store_name store_name, + s_zip store_zip, + ad1.ca_street_number b_street_number, + ad1.ca_street_name b_streen_name, + ad1.ca_city b_city, + ad1.ca_zip b_zip, + ad2.ca_street_number c_street_number, + ad2.ca_street_name c_street_name, + ad2.ca_city c_city, + ad2.ca_zip c_zip, + d1.d_year AS syear, + d2.d_year AS fsyear, + d3.d_year s2year, + count(*) cnt, + sum(ss_wholesale_cost) s1, + sum(ss_list_price) s2, + sum(ss_coupon_amt) s3 + FROM store_sales, store_returns, cs_ui, date_dim d1, date_dim d2, date_dim d3, + store, customer, customer_demographics cd1, customer_demographics cd2, + promotion, household_demographics hd1, household_demographics hd2, + customer_address ad1, customer_address ad2, income_band ib1, income_band ib2, item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk = cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk AND + ss_item_sk = i_item_sk AND + ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number AND + ss_item_sk = cs_ui.cs_item_sk AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk AND + c_first_sales_date_sk = d2.d_date_sk AND + c_first_shipto_date_sk = d3.d_date_sk AND + ss_promo_sk = p_promo_sk AND + hd1.hd_income_band_sk = ib1.ib_income_band_sk AND + hd2.hd_income_band_sk = ib2.ib_income_band_sk AND + cd1.cd_marital_status <> cd2.cd_marital_status AND + i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium') AND + i_current_price BETWEEN 64 AND 64 + 10 AND + i_current_price BETWEEN 64 + 1 AND 64 + 15 + GROUP BY i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, + ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, + ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year + ) +SELECT + cs1.product_name, + cs1.store_name, + cs1.store_zip, + cs1.b_street_number, + cs1.b_streen_name, + cs1.b_city, + cs1.b_zip, + cs1.c_street_number, + cs1.c_street_name, + cs1.c_city, + cs1.c_zip, + cs1.syear, + cs1.cnt, + cs1.s1, + cs1.s2, + cs1.s3, + cs2.s1, + cs2.s2, + cs2.s3, + cs2.syear, + cs2.cnt +FROM cross_sales cs1, cross_sales cs2 +WHERE cs1.item_sk = cs2.item_sk AND + cs1.syear = 1999 AND + cs2.syear = 1999 + 1 AND + cs2.cnt <= cs1.cnt AND + cs1.store_name = cs2.store_name AND + cs1.store_zip = cs2.store_zip +ORDER BY cs1.product_name, cs1.store_name, cs2.cnt diff --git a/sql/core/src/test/resources/tpcds/q65.sql b/sql/core/src/test/resources/tpcds/q65.sql new file mode 100755 index 0000000000000..aad04be1bcdf0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q65.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand +FROM store, item, + (SELECT + ss_store_sk, + avg(revenue) AS ave + FROM + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sa + GROUP BY ss_store_sk) sb, + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sc +WHERE sb.ss_store_sk = sc.ss_store_sk AND + sc.revenue <= 0.1 * sb.ave AND + s_store_sk = sc.ss_store_sk AND + i_item_sk = sc.ss_item_sk +ORDER BY s_store_name, i_item_desc +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q66.sql b/sql/core/src/test/resources/tpcds/q66.sql new file mode 100755 index 0000000000000..f826b4164372a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q66.sql @@ -0,0 +1,240 @@ +SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + ship_carriers, + year, + sum(jan_sales) AS jan_sales, + sum(feb_sales) AS feb_sales, + sum(mar_sales) AS mar_sales, + sum(apr_sales) AS apr_sales, + sum(may_sales) AS may_sales, + sum(jun_sales) AS jun_sales, + sum(jul_sales) AS jul_sales, + sum(aug_sales) AS aug_sales, + sum(sep_sales) AS sep_sales, + sum(oct_sales) AS oct_sales, + sum(nov_sales) AS nov_sales, + sum(dec_sales) AS dec_sales, + sum(jan_sales / w_warehouse_sq_ft) AS jan_sales_per_sq_foot, + sum(feb_sales / w_warehouse_sq_ft) AS feb_sales_per_sq_foot, + sum(mar_sales / w_warehouse_sq_ft) AS mar_sales_per_sq_foot, + sum(apr_sales / w_warehouse_sq_ft) AS apr_sales_per_sq_foot, + sum(may_sales / w_warehouse_sq_ft) AS may_sales_per_sq_foot, + sum(jun_sales / w_warehouse_sq_ft) AS jun_sales_per_sq_foot, + sum(jul_sales / w_warehouse_sq_ft) AS jul_sales_per_sq_foot, + sum(aug_sales / w_warehouse_sq_ft) AS aug_sales_per_sq_foot, + sum(sep_sales / w_warehouse_sq_ft) AS sep_sales_per_sq_foot, + sum(oct_sales / w_warehouse_sq_ft) AS oct_sales_per_sq_foot, + sum(nov_sales / w_warehouse_sq_ft) AS nov_sales_per_sq_foot, + sum(dec_sales / w_warehouse_sq_ft) AS dec_sales_per_sq_foot, + sum(jan_net) AS jan_net, + sum(feb_net) AS feb_net, + sum(mar_net) AS mar_net, + sum(apr_net) AS apr_net, + sum(may_net) AS may_net, + sum(jun_net) AS jun_net, + sum(jul_net) AS jul_net, + sum(aug_net) AS aug_net, + sum(sep_net) AS sep_net, + sum(oct_net) AS oct_net, + sum(nov_net) AS nov_net, + sum(dec_net) AS dec_net +FROM ( + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS dec_net + FROM + web_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + ws_warehouse_sk = w_warehouse_sk + AND ws_sold_date_sk = d_date_sk + AND ws_sold_time_sk = t_time_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year) + UNION ALL + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS dec_net + FROM + catalog_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND cs_sold_time_sk = t_time_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year + ) + ) x +GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, + ship_carriers, year +ORDER BY w_warehouse_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q67.sql b/sql/core/src/test/resources/tpcds/q67.sql new file mode 100755 index 0000000000000..f66e2252bdbd4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q67.sql @@ -0,0 +1,38 @@ +SELECT * +FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sumsales, + rank() + OVER (PARTITION BY i_category + ORDER BY sumsales DESC) rk + FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sum(coalesce(ss_sales_price * ss_quantity, 0)) sumsales + FROM store_sales, date_dim, store, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ROLLUP (i_category, i_class, i_brand, i_product_name, d_year, d_qoy, + d_moy, s_store_id)) dw1) dw2 +WHERE rk <= 100 +ORDER BY + i_category, i_class, i_brand, i_product_name, d_year, + d_qoy, d_moy, s_store_id, sumsales, rk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q68.sql b/sql/core/src/test/resources/tpcds/q68.sql new file mode 100755 index 0000000000000..adb8a7189dad7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q68.sql @@ -0,0 +1,34 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + extended_price, + extended_tax, + list_price +FROM (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_ext_sales_price) extended_price, + sum(ss_ext_list_price) list_price, + sum(ss_ext_tax) extended_tax +FROM store_sales, date_dim, store, household_demographics, customer_address +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Midway', 'Fairview') +GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, + customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, ss_ticket_number +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q69.sql b/sql/core/src/test/resources/tpcds/q69.sql new file mode 100755 index 0000000000000..1f0ee64f565a3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q69.sql @@ -0,0 +1,38 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_state IN ('KY', 'GA', 'NM') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + (NOT exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + NOT exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2)) +GROUP BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +ORDER BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q7.sql b/sql/core/src/test/resources/tpcds/q7.sql new file mode 100755 index 0000000000000..6630a00548403 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q7.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, item, promotion +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_cdemo_sk = cd_demo_sk AND + ss_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q70.sql b/sql/core/src/test/resources/tpcds/q70.sql new file mode 100755 index 0000000000000..625011b212fe0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q70.sql @@ -0,0 +1,38 @@ +SELECT + sum(ss_net_profit) AS total_sum, + s_state, + s_county, + grouping(s_state) + grouping(s_county) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(s_state) + grouping(s_county), + CASE WHEN grouping(s_county) = 0 + THEN s_state END + ORDER BY sum(ss_net_profit) DESC) AS rank_within_parent +FROM + store_sales, date_dim d1, store +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + AND s_state IN + (SELECT s_state + FROM + (SELECT + s_state AS s_state, + rank() + OVER (PARTITION BY s_state + ORDER BY sum(ss_net_profit) DESC) AS ranking + FROM store_sales, store, date_dim + WHERE d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + GROUP BY s_state) tmp1 + WHERE ranking <= 5) +GROUP BY ROLLUP (s_state, s_county) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN s_state END + , rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q71.sql b/sql/core/src/test/resources/tpcds/q71.sql new file mode 100755 index 0000000000000..8d724b9244e11 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q71.sql @@ -0,0 +1,44 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + t_hour, + t_minute, + sum(ext_price) ext_price +FROM item, + (SELECT + ws_ext_sales_price AS ext_price, + ws_sold_date_sk AS sold_date_sk, + ws_item_sk AS sold_item_sk, + ws_sold_time_sk AS time_sk + FROM web_sales, date_dim + WHERE d_date_sk = ws_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + cs_ext_sales_price AS ext_price, + cs_sold_date_sk AS sold_date_sk, + cs_item_sk AS sold_item_sk, + cs_sold_time_sk AS time_sk + FROM catalog_sales, date_dim + WHERE d_date_sk = cs_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + ss_ext_sales_price AS ext_price, + ss_sold_date_sk AS sold_date_sk, + ss_item_sk AS sold_item_sk, + ss_sold_time_sk AS time_sk + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + ) AS tmp, time_dim +WHERE + sold_item_sk = i_item_sk + AND i_manager_id = 1 + AND time_sk = t_time_sk + AND (t_meal_time = 'breakfast' OR t_meal_time = 'dinner') +GROUP BY i_brand, i_brand_id, t_hour, t_minute +ORDER BY ext_price DESC, brand_id diff --git a/sql/core/src/test/resources/tpcds/q72.sql b/sql/core/src/test/resources/tpcds/q72.sql new file mode 100755 index 0000000000000..99b3eee54aa1a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q72.sql @@ -0,0 +1,33 @@ +SELECT + i_item_desc, + w_warehouse_name, + d1.d_week_seq, + count(CASE WHEN p_promo_sk IS NULL + THEN 1 + ELSE 0 END) no_promo, + count(CASE WHEN p_promo_sk IS NOT NULL + THEN 1 + ELSE 0 END) promo, + count(*) total_cnt +FROM catalog_sales + JOIN inventory ON (cs_item_sk = inv_item_sk) + JOIN warehouse ON (w_warehouse_sk = inv_warehouse_sk) + JOIN item ON (i_item_sk = cs_item_sk) + JOIN customer_demographics ON (cs_bill_cdemo_sk = cd_demo_sk) + JOIN household_demographics ON (cs_bill_hdemo_sk = hd_demo_sk) + JOIN date_dim d1 ON (cs_sold_date_sk = d1.d_date_sk) + JOIN date_dim d2 ON (inv_date_sk = d2.d_date_sk) + JOIN date_dim d3 ON (cs_ship_date_sk = d3.d_date_sk) + LEFT OUTER JOIN promotion ON (cs_promo_sk = p_promo_sk) + LEFT OUTER JOIN catalog_returns ON (cr_item_sk = cs_item_sk AND cr_order_number = cs_order_number) +WHERE d1.d_week_seq = d2.d_week_seq + AND inv_quantity_on_hand < cs_quantity + AND d3.d_date > (cast(d1.d_date AS DATE) + interval 5 days) + AND hd_buy_potential = '>10000' + AND d1.d_year = 1999 + AND hd_buy_potential = '>10000' + AND cd_marital_status = 'D' + AND d1.d_year = 1999 +GROUP BY i_item_desc, w_warehouse_name, d1.d_week_seq +ORDER BY total_cnt DESC, i_item_desc, w_warehouse_name, d_week_seq +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q73.sql b/sql/core/src/test/resources/tpcds/q73.sql new file mode 100755 index 0000000000000..881be2e9024d7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q73.sql @@ -0,0 +1,30 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN + household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL END > 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN ('Williamson County', 'Franklin Parish', 'Bronx County', 'Orange County') + GROUP BY ss_ticket_number, ss_customer_sk) dj, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 1 AND 5 +ORDER BY cnt DESC diff --git a/sql/core/src/test/resources/tpcds/q74.sql b/sql/core/src/test/resources/tpcds/q74.sql new file mode 100755 index 0000000000000..154b26d6802a3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q74.sql @@ -0,0 +1,58 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ss_net_paid) year_total, + 's' sale_type + FROM + customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ws_net_paid) year_total, + 'w' sale_type + FROM + customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name +FROM + year_total t_s_firstyear, year_total t_s_secyear, + year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.year = 2001 + AND t_s_secyear.year = 2001 + 1 + AND t_w_firstyear.year = 2001 + AND t_w_secyear.year = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY 1, 1, 1 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q75.sql b/sql/core/src/test/resources/tpcds/q75.sql new file mode 100755 index 0000000000000..2a143232b5196 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q75.sql @@ -0,0 +1,76 @@ +WITH all_sales AS ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + SUM(sales_cnt) AS sales_cnt, + SUM(sales_amt) AS sales_amt + FROM ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + cs_quantity - COALESCE(cr_return_quantity, 0) AS sales_cnt, + cs_ext_sales_price - COALESCE(cr_return_amount, 0.0) AS sales_amt + FROM catalog_sales + JOIN item ON i_item_sk = cs_item_sk + JOIN date_dim ON d_date_sk = cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ss_quantity - COALESCE(sr_return_quantity, 0) AS sales_cnt, + ss_ext_sales_price - COALESCE(sr_return_amt, 0.0) AS sales_amt + FROM store_sales + JOIN item ON i_item_sk = ss_item_sk + JOIN date_dim ON d_date_sk = ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ws_quantity - COALESCE(wr_return_quantity, 0) AS sales_cnt, + ws_ext_sales_price - COALESCE(wr_return_amt, 0.0) AS sales_amt + FROM web_sales + JOIN item ON i_item_sk = ws_item_sk + JOIN date_dim ON d_date_sk = ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number = wr_order_number + AND ws_item_sk = wr_item_sk) + WHERE i_category = 'Books') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) +SELECT + prev_yr.d_year AS prev_year, + curr_yr.d_year AS year, + curr_yr.i_brand_id, + curr_yr.i_class_id, + curr_yr.i_category_id, + curr_yr.i_manufact_id, + prev_yr.sales_cnt AS prev_yr_cnt, + curr_yr.sales_cnt AS curr_yr_cnt, + curr_yr.sales_cnt - prev_yr.sales_cnt AS sales_cnt_diff, + curr_yr.sales_amt - prev_yr.sales_amt AS sales_amt_diff +FROM all_sales curr_yr, all_sales prev_yr +WHERE curr_yr.i_brand_id = prev_yr.i_brand_id + AND curr_yr.i_class_id = prev_yr.i_class_id + AND curr_yr.i_category_id = prev_yr.i_category_id + AND curr_yr.i_manufact_id = prev_yr.i_manufact_id + AND curr_yr.d_year = 2002 + AND prev_yr.d_year = 2002 - 1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17, 2)) / CAST(prev_yr.sales_cnt AS DECIMAL(17, 2)) < 0.9 +ORDER BY sales_cnt_diff +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q76.sql b/sql/core/src/test/resources/tpcds/q76.sql new file mode 100755 index 0000000000000..815fa922be19d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q76.sql @@ -0,0 +1,47 @@ +SELECT + channel, + col_name, + d_year, + d_qoy, + i_category, + COUNT(*) sales_cnt, + SUM(ext_sales_price) sales_amt +FROM ( + SELECT + 'store' AS channel, + ss_store_sk col_name, + d_year, + d_qoy, + i_category, + ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_store_sk IS NULL + AND ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + UNION ALL + SELECT + 'web' AS channel, + ws_ship_customer_sk col_name, + d_year, + d_qoy, + i_category, + ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_ship_customer_sk IS NULL + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk = i_item_sk + UNION ALL + SELECT + 'catalog' AS channel, + cs_ship_addr_sk col_name, + d_year, + d_qoy, + i_category, + cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_ship_addr_sk IS NULL + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk = i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q77.sql b/sql/core/src/test/resources/tpcds/q77.sql new file mode 100755 index 0000000000000..7830f96e76515 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q77.sql @@ -0,0 +1,100 @@ +WITH ss AS +(SELECT + s_store_sk, + sum(ss_ext_sales_price) AS sales, + sum(ss_net_profit) AS profit + FROM store_sales, date_dim, store + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + GROUP BY s_store_sk), + sr AS + (SELECT + s_store_sk, + sum(sr_return_amt) AS returns, + sum(sr_net_loss) AS profit_loss + FROM store_returns, date_dim, store + WHERE sr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND sr_store_sk = s_store_sk + GROUP BY s_store_sk), + cs AS + (SELECT + cs_call_center_sk, + sum(cs_ext_sales_price) AS sales, + sum(cs_net_profit) AS profit + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + GROUP BY cs_call_center_sk), + cr AS + (SELECT + sum(cr_return_amount) AS returns, + sum(cr_net_loss) AS profit_loss + FROM catalog_returns, date_dim + WHERE cr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03]' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days)), + ws AS + (SELECT + wp_web_page_sk, + sum(ws_ext_sales_price) AS sales, + sum(ws_net_profit) AS profit + FROM web_sales, date_dim, web_page + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ws_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk), + wr AS + (SELECT + wp_web_page_sk, + sum(wr_return_amt) AS returns, + sum(wr_net_loss) AS profit_loss + FROM web_returns, date_dim, web_page + WHERE wr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND wr_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + ss.s_store_sk AS id, + sales, + coalesce(returns, 0) AS returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ss + LEFT JOIN sr + ON ss.s_store_sk = sr.s_store_sk + UNION ALL + SELECT + 'catalog channel' AS channel, + cs_call_center_sk AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM cs, cr + UNION ALL + SELECT + 'web channel' AS channel, + ws.wp_web_page_sk AS id, + sales, + coalesce(returns, 0) returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ws + LEFT JOIN wr + ON ws.wp_web_page_sk = wr.wp_web_page_sk + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q78.sql b/sql/core/src/test/resources/tpcds/q78.sql new file mode 100755 index 0000000000000..07b0940e26882 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q78.sql @@ -0,0 +1,64 @@ +WITH ws AS +(SELECT + d_year AS ws_sold_year, + ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + FROM web_sales + LEFT JOIN web_returns ON wr_order_number = ws_order_number AND ws_item_sk = wr_item_sk + JOIN date_dim ON ws_sold_date_sk = d_date_sk + WHERE wr_order_number IS NULL + GROUP BY d_year, ws_item_sk, ws_bill_customer_sk +), + cs AS + (SELECT + d_year AS cs_sold_year, + cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + FROM catalog_sales + LEFT JOIN catalog_returns ON cr_order_number = cs_order_number AND cs_item_sk = cr_item_sk + JOIN date_dim ON cs_sold_date_sk = d_date_sk + WHERE cr_order_number IS NULL + GROUP BY d_year, cs_item_sk, cs_bill_customer_sk + ), + ss AS + (SELECT + d_year AS ss_sold_year, + ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + FROM store_sales + LEFT JOIN store_returns ON sr_ticket_number = ss_ticket_number AND ss_item_sk = sr_item_sk + JOIN date_dim ON ss_sold_date_sk = d_date_sk + WHERE sr_ticket_number IS NULL + GROUP BY d_year, ss_item_sk, ss_customer_sk + ) +SELECT + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) ratio, + ss_qty store_qty, + ss_wc store_wholesale_cost, + ss_sp store_sales_price, + coalesce(ws_qty, 0) + coalesce(cs_qty, 0) other_chan_qty, + coalesce(ws_wc, 0) + coalesce(cs_wc, 0) other_chan_wholesale_cost, + coalesce(ws_sp, 0) + coalesce(cs_sp, 0) other_chan_sales_price +FROM ss + LEFT JOIN ws + ON (ws_sold_year = ss_sold_year AND ws_item_sk = ss_item_sk AND ws_customer_sk = ss_customer_sk) + LEFT JOIN cs + ON (cs_sold_year = ss_sold_year AND cs_item_sk = ss_item_sk AND cs_customer_sk = ss_customer_sk) +WHERE coalesce(ws_qty, 0) > 0 AND coalesce(cs_qty, 0) > 0 AND ss_sold_year = 2000 +ORDER BY + ratio, + ss_qty DESC, ss_wc DESC, ss_sp DESC, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q79.sql b/sql/core/src/test/resources/tpcds/q79.sql new file mode 100755 index 0000000000000..08f86dc2032aa --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q79.sql @@ -0,0 +1,27 @@ +SELECT + c_last_name, + c_first_name, + substr(s_city, 1, 30), + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + store.s_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (household_demographics.hd_dep_count = 6 OR + household_demographics.hd_vehicle_count > 2) + AND date_dim.d_dow = 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_number_employees BETWEEN 200 AND 295 + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city) ms, customer +WHERE ss_customer_sk = c_customer_sk +ORDER BY c_last_name, c_first_name, substr(s_city, 1, 30), profit +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q8.sql b/sql/core/src/test/resources/tpcds/q8.sql new file mode 100755 index 0000000000000..497725111f4f3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q8.sql @@ -0,0 +1,87 @@ +SELECT + s_store_name, + sum(ss_net_profit) +FROM store_sales, date_dim, store, + (SELECT ca_zip + FROM ( + (SELECT substr(ca_zip, 1, 5) ca_zip + FROM customer_address + WHERE substr(ca_zip, 1, 5) IN ( + '24128','76232','65084','87816','83926','77556','20548', + '26231','43848','15126','91137','61265','98294','25782', + '17920','18426','98235','40081','84093','28577','55565', + '17183','54601','67897','22752','86284','18376','38607', + '45200','21756','29741','96765','23932','89360','29839', + '25989','28898','91068','72550','10390','18845','47770', + '82636','41367','76638','86198','81312','37126','39192', + '88424','72175','81426','53672','10445','42666','66864', + '66708','41248','48583','82276','18842','78890','49448', + '14089','38122','34425','79077','19849','43285','39861', + '66162','77610','13695','99543','83444','83041','12305', + '57665','68341','25003','57834','62878','49130','81096', + '18840','27700','23470','50412','21195','16021','76107', + '71954','68309','18119','98359','64544','10336','86379', + '27068','39736','98569','28915','24206','56529','57647', + '54917','42961','91110','63981','14922','36420','23006', + '67467','32754','30903','20260','31671','51798','72325', + '85816','68621','13955','36446','41766','68806','16725', + '15146','22744','35850','88086','51649','18270','52867', + '39972','96976','63792','11376','94898','13595','10516', + '90225','58943','39371','94945','28587','96576','57855', + '28488','26105','83933','25858','34322','44438','73171', + '30122','34102','22685','71256','78451','54364','13354', + '45375','40558','56458','28286','45266','47305','69399', + '83921','26233','11101','15371','69913','35942','15882', + '25631','24610','44165','99076','33786','70738','26653', + '14328','72305','62496','22152','10144','64147','48425', + '14663','21076','18799','30450','63089','81019','68893', + '24996','51200','51211','45692','92712','70466','79994', + '22437','25280','38935','71791','73134','56571','14060', + '19505','72425','56575','74351','68786','51650','20004', + '18383','76614','11634','18906','15765','41368','73241', + '76698','78567','97189','28545','76231','75691','22246', + '51061','90578','56691','68014','51103','94167','57047', + '14867','73520','15734','63435','25733','35474','24676', + '94627','53535','17879','15559','53268','59166','11928', + '59402','33282','45721','43933','68101','33515','36634', + '71286','19736','58058','55253','67473','41918','19515', + '36495','19430','22351','77191','91393','49156','50298', + '87501','18652','53179','18767','63193','23968','65164', + '68880','21286','72823','58470','67301','13394','31016', + '70372','67030','40604','24317','45748','39127','26065', + '77721','31029','31880','60576','24671','45549','13376', + '50016','33123','19769','22927','97789','46081','72151', + '15723','46136','51949','68100','96888','64528','14171', + '79777','28709','11489','25103','32213','78668','22245', + '15798','27156','37930','62971','21337','51622','67853', + '10567','38415','15455','58263','42029','60279','37125', + '56240','88190','50308','26859','64457','89091','82136', + '62377','36233','63837','58078','17043','30010','60099', + '28810','98025','29178','87343','73273','30469','64034', + '39516','86057','21309','90257','67875','40162','11356', + '73650','61810','72013','30431','22461','19512','13375', + '55307','30625','83849','68908','26689','96451','38193', + '46820','88885','84935','69035','83144','47537','56616', + '94983','48033','69952','25486','61547','27385','61860', + '58048','56910','16807','17871','35258','31387','35458', + '35576')) + INTERSECT + (SELECT ca_zip + FROM + (SELECT + substr(ca_zip, 1, 5) ca_zip, + count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk AND + c_preferred_cust_flag = 'Y' + GROUP BY ca_zip + HAVING count(*) > 10) A1) + ) A2 + ) V1 +WHERE ss_store_sk = s_store_sk + AND ss_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 1998 + AND (substr(s_zip, 1, 2) = substr(V1.ca_zip, 1, 2)) +GROUP BY s_store_name +ORDER BY s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q80.sql b/sql/core/src/test/resources/tpcds/q80.sql new file mode 100755 index 0000000000000..433db87d2a858 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q80.sql @@ -0,0 +1,94 @@ +WITH ssr AS +(SELECT + s_store_id AS store_id, + sum(ss_ext_sales_price) AS sales, + sum(coalesce(sr_return_amt, 0)) AS returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) AS profit + FROM store_sales + LEFT OUTER JOIN store_returns ON + (ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number) + , + date_dim, store, item, promotion + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + AND ss_item_sk = i_item_sk + AND i_current_price > 50 + AND ss_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY s_store_id), + csr AS + (SELECT + cp_catalog_page_id AS catalog_page_id, + sum(cs_ext_sales_price) AS sales, + sum(coalesce(cr_return_amount, 0)) AS returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) AS profit + FROM catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_item_sk = cr_item_sk AND + cs_order_number = cr_order_number) + , + date_dim, catalog_page, item, promotion + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND cs_catalog_page_sk = cp_catalog_page_sk + AND cs_item_sk = i_item_sk + AND i_current_price > 50 + AND cs_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY cp_catalog_page_id), + wsr AS + (SELECT + web_site_id, + sum(ws_ext_sales_price) AS sales, + sum(coalesce(wr_return_amt, 0)) AS returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) AS profit + FROM web_sales + LEFT OUTER JOIN web_returns ON + (ws_item_sk = wr_item_sk AND ws_order_number = wr_order_number) + , + date_dim, web_site, item, promotion + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ws_web_site_sk = web_site_sk + AND ws_item_sk = i_item_sk + AND i_current_price > 50 + AND ws_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM (SELECT + 'store channel' AS channel, + concat('store', store_id) AS id, + sales, + returns, + profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', catalog_page_id) AS id, + sales, + returns, + profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + profit + FROM wsr) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q81.sql b/sql/core/src/test/resources/tpcds/q81.sql new file mode 100755 index 0000000000000..18f0ffa7e8f4c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q81.sql @@ -0,0 +1,38 @@ +WITH customer_total_return AS +(SELECT + cr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(cr_return_amt_inc_tax) AS ctr_total_return + FROM catalog_returns, date_dim, customer_address + WHERE cr_returned_date_sk = d_date_sk + AND d_year = 2000 + AND cr_returning_addr_sk = ca_address_sk + GROUP BY cr_returning_customer_sk, ca_state ) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + ca_street_number, + ca_street_name, + ca_street_type, + ca_suite_number, + ca_city, + ca_county, + ca_state, + ca_zip, + ca_country, + ca_gmt_offset, + ca_location_type, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name + , ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset + , ca_location_type, ctr_total_return +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q82.sql b/sql/core/src/test/resources/tpcds/q82.sql new file mode 100755 index 0000000000000..20942cfeb0787 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q82.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, store_sales +WHERE i_current_price BETWEEN 62 AND 62 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-05-25' AS DATE) AND (cast('2000-05-25' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (129, 270, 821, 423) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND ss_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q83.sql b/sql/core/src/test/resources/tpcds/q83.sql new file mode 100755 index 0000000000000..53c10c7ded6c1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q83.sql @@ -0,0 +1,56 @@ +WITH sr_items AS +(SELECT + i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + FROM store_returns, item, date_dim + WHERE sr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND sr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + cr_items AS + (SELECT + i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + FROM catalog_returns, item, date_dim + WHERE cr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND cr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + wr_items AS + (SELECT + i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + FROM web_returns, item, date_dim + WHERE wr_item_sk = i_item_sk AND d_date IN + (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND wr_returned_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + sr_items.item_id, + sr_item_qty, + sr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 sr_dev, + cr_item_qty, + cr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 cr_dev, + wr_item_qty, + wr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 wr_dev, + (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 average +FROM sr_items, cr_items, wr_items +WHERE sr_items.item_id = cr_items.item_id + AND sr_items.item_id = wr_items.item_id +ORDER BY sr_items.item_id, sr_item_qty +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q84.sql b/sql/core/src/test/resources/tpcds/q84.sql new file mode 100755 index 0000000000000..a1076b57ced5c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q84.sql @@ -0,0 +1,19 @@ +SELECT + c_customer_id AS customer_id, + concat(c_last_name, ', ', c_first_name) AS customername +FROM customer + , customer_address + , customer_demographics + , household_demographics + , income_band + , store_returns +WHERE ca_city = 'Edgewood' + AND c_current_addr_sk = ca_address_sk + AND ib_lower_bound >= 38128 + AND ib_upper_bound <= 38128 + 50000 + AND ib_income_band_sk = hd_income_band_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND sr_cdemo_sk = cd_demo_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q85.sql b/sql/core/src/test/resources/tpcds/q85.sql new file mode 100755 index 0000000000000..cf718b0f8adec --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q85.sql @@ -0,0 +1,82 @@ +SELECT + substr(r_reason_desc, 1, 20), + avg(ws_quantity), + avg(wr_refunded_cash), + avg(wr_fee) +FROM web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason +WHERE ws_web_page_sk = wp_web_page_sk + AND ws_item_sk = wr_item_sk + AND ws_order_number = wr_order_number + AND ws_sold_date_sk = d_date_sk AND d_year = 2000 + AND cd1.cd_demo_sk = wr_refunded_cdemo_sk + AND cd2.cd_demo_sk = wr_returning_cdemo_sk + AND ca_address_sk = wr_refunded_addr_sk + AND r_reason_sk = wr_reason_sk + AND + ( + ( + cd1.cd_marital_status = 'M' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'Advanced Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd1.cd_marital_status = 'S' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'College' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd1.cd_marital_status = 'W' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = '2 yr Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ca_country = 'United States' + AND + ca_state IN ('IN', 'OH', 'NJ') + AND ws_net_profit BETWEEN 100 AND 200 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('WI', 'CT', 'KY') + AND ws_net_profit BETWEEN 150 AND 300 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('LA', 'IA', 'AR') + AND ws_net_profit BETWEEN 50 AND 250 + ) + ) +GROUP BY r_reason_desc +ORDER BY substr(r_reason_desc, 1, 20) + , avg(ws_quantity) + , avg(wr_refunded_cash) + , avg(wr_fee) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q86.sql b/sql/core/src/test/resources/tpcds/q86.sql new file mode 100755 index 0000000000000..789a4abf7b5f7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q86.sql @@ -0,0 +1,24 @@ +SELECT + sum(ws_net_paid) AS total_sum, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ws_net_paid) DESC) AS rank_within_parent +FROM + web_sales, date_dim d1, item +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ws_sold_date_sk + AND i_item_sk = ws_item_sk +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC, + CASE WHEN lochierarchy = 0 + THEN i_category END, + rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q87.sql b/sql/core/src/test/resources/tpcds/q87.sql new file mode 100755 index 0000000000000..4aaa9f39dce9e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q87.sql @@ -0,0 +1,28 @@ +SELECT count(*) +FROM ((SELECT DISTINCT + c_last_name, + c_first_name, + d_date +FROM store_sales, date_dim, customer +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + ) cool_cust diff --git a/sql/core/src/test/resources/tpcds/q88.sql b/sql/core/src/test/resources/tpcds/q88.sql new file mode 100755 index 0000000000000..25bcd90f41ab6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q88.sql @@ -0,0 +1,122 @@ +SELECT * +FROM + (SELECT count(*) h8_30_to_9 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 8 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s1, + (SELECT count(*) h9_to_9_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s2, + (SELECT count(*) h9_30_to_10 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s3, + (SELECT count(*) h10_to_10_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s4, + (SELECT count(*) h10_30_to_11 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s5, + (SELECT count(*) h11_to_11_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s6, + (SELECT count(*) h11_30_to_12 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s7, + (SELECT count(*) h12_to_12_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 12 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s8 diff --git a/sql/core/src/test/resources/tpcds/q89.sql b/sql/core/src/test/resources/tpcds/q89.sql new file mode 100755 index 0000000000000..75408cb0323f8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q89.sql @@ -0,0 +1,30 @@ +SELECT * +FROM ( + SELECT + i_category, + i_class, + i_brand, + s_store_name, + s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_year IN (1999) AND + ((i_category IN ('Books', 'Electronics', 'Sports') AND + i_class IN ('computers', 'stereo', 'football')) + OR (i_category IN ('Men', 'Jewelry', 'Women') AND + i_class IN ('shirts', 'birdal', 'dresses'))) + GROUP BY i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +WHERE CASE WHEN (avg_monthly_sales <> 0) + THEN (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q9.sql b/sql/core/src/test/resources/tpcds/q9.sql new file mode 100755 index 0000000000000..de3db9d988f1e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q9.sql @@ -0,0 +1,48 @@ +SELECT + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) > 62316685 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) END bucket1, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) > 19045798 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) END bucket2, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) > 365541424 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) END bucket3, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) > 216357808 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) END bucket4, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) > 184483884 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) END bucket5 +FROM reason +WHERE r_reason_sk = 1 diff --git a/sql/core/src/test/resources/tpcds/q90.sql b/sql/core/src/test/resources/tpcds/q90.sql new file mode 100755 index 0000000000000..85e35bf8bf8ec --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q90.sql @@ -0,0 +1,19 @@ +SELECT cast(amc AS DECIMAL(15, 4)) / cast(pmc AS DECIMAL(15, 4)) am_pm_ratio +FROM (SELECT count(*) amc +FROM web_sales, household_demographics, time_dim, web_page +WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 8 AND 8 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) at, + (SELECT count(*) pmc + FROM web_sales, household_demographics, time_dim, web_page + WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 19 AND 19 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) pt +ORDER BY am_pm_ratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q91.sql b/sql/core/src/test/resources/tpcds/q91.sql new file mode 100755 index 0000000000000..9ca7ce00ac775 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q91.sql @@ -0,0 +1,23 @@ +SELECT + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +FROM + call_center, catalog_returns, date_dim, customer, customer_address, + customer_demographics, household_demographics +WHERE + cr_call_center_sk = cc_call_center_sk + AND cr_returned_date_sk = d_date_sk + AND cr_returning_customer_sk = c_customer_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND ca_address_sk = c_current_addr_sk + AND d_year = 1998 + AND d_moy = 11 + AND ((cd_marital_status = 'M' AND cd_education_status = 'Unknown') + OR (cd_marital_status = 'W' AND cd_education_status = 'Advanced Degree')) + AND hd_buy_potential LIKE 'Unknown%' + AND ca_gmt_offset = -7 +GROUP BY cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status +ORDER BY sum(cr_net_loss) DESC diff --git a/sql/core/src/test/resources/tpcds/q92.sql b/sql/core/src/test/resources/tpcds/q92.sql new file mode 100755 index 0000000000000..99129c3bd9e5b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q92.sql @@ -0,0 +1,16 @@ +SELECT sum(ws_ext_discount_amt) AS `Excess Discount Amount ` +FROM web_sales, item, date_dim +WHERE i_manufact_id = 350 + AND i_item_sk = ws_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + AND ws_ext_discount_amt > + ( + SELECT 1.3 * avg(ws_ext_discount_amt) + FROM web_sales, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + ) +ORDER BY sum(ws_ext_discount_amt) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q93.sql b/sql/core/src/test/resources/tpcds/q93.sql new file mode 100755 index 0000000000000..222dc31c1f561 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q93.sql @@ -0,0 +1,19 @@ +SELECT + ss_customer_sk, + sum(act_sales) sumsales +FROM (SELECT + ss_item_sk, + ss_ticket_number, + ss_customer_sk, + CASE WHEN sr_return_quantity IS NOT NULL + THEN (ss_quantity - sr_return_quantity) * ss_sales_price + ELSE (ss_quantity * ss_sales_price) END act_sales +FROM store_sales + LEFT OUTER JOIN store_returns + ON (sr_item_sk = ss_item_sk AND sr_ticket_number = ss_ticket_number) + , + reason +WHERE sr_reason_sk = r_reason_sk AND r_reason_desc = 'reason 28') t +GROUP BY ss_customer_sk +ORDER BY sumsales, ss_customer_sk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q94.sql b/sql/core/src/test/resources/tpcds/q94.sql new file mode 100755 index 0000000000000..d6de3d75b82d2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q94.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 days) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND EXISTS(SELECT * + FROM web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM web_returns wr1 + WHERE ws1.ws_order_number = wr1.wr_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q95.sql b/sql/core/src/test/resources/tpcds/q95.sql new file mode 100755 index 0000000000000..df71f00bd6c0b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q95.sql @@ -0,0 +1,29 @@ +WITH ws_wh AS +(SELECT + ws1.ws_order_number, + ws1.ws_warehouse_sk wh1, + ws2.ws_warehouse_sk wh2 + FROM web_sales ws1, web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 DAY) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND ws1.ws_order_number IN (SELECT ws_order_number + FROM ws_wh) + AND ws1.ws_order_number IN (SELECT wr_order_number + FROM web_returns, ws_wh + WHERE wr_order_number = ws_wh.ws_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q96.sql b/sql/core/src/test/resources/tpcds/q96.sql new file mode 100755 index 0000000000000..7ab17e7bc4597 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q96.sql @@ -0,0 +1,11 @@ +SELECT count(*) +FROM store_sales, household_demographics, time_dim, store +WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 20 + AND time_dim.t_minute >= 30 + AND household_demographics.hd_dep_count = 7 + AND store.s_store_name = 'ese' +ORDER BY count(*) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q97.sql b/sql/core/src/test/resources/tpcds/q97.sql new file mode 100755 index 0000000000000..e7e0b1a05259d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q97.sql @@ -0,0 +1,30 @@ +WITH ssci AS ( + SELECT + ss_customer_sk customer_sk, + ss_item_sk item_sk + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ss_customer_sk, ss_item_sk), + csci AS ( + SELECT + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY cs_bill_customer_sk, cs_item_sk) +SELECT + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL + THEN 1 + ELSE 0 END) store_only, + sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) catalog_only, + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) store_and_catalog +FROM ssci + FULL OUTER JOIN csci ON (ssci.customer_sk = csci.customer_sk + AND ssci.item_sk = csci.item_sk) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q98.sql b/sql/core/src/test/resources/tpcds/q98.sql new file mode 100755 index 0000000000000..bb10d4bf8da23 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q98.sql @@ -0,0 +1,21 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ss_ext_sales_price) AS itemrevenue, + sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + store_sales, item, date_dim +WHERE + ss_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio diff --git a/sql/core/src/test/resources/tpcds/q99.sql b/sql/core/src/test/resources/tpcds/q99.sql new file mode 100755 index 0000000000000..f1a3d4d2b7fe9 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q99.sql @@ -0,0 +1,34 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + cc_name, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND + (cs_ship_date_sk - cs_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND + (cs_ship_date_sk - cs_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND + (cs_ship_date_sk - cs_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + catalog_sales, warehouse, ship_mode, call_center, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND cs_ship_date_sk = d_date_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND cs_call_center_sk = cc_call_center_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, cc_name +ORDER BY substr(w_warehouse_name, 1, 20), sm_type, cc_name +LIMIT 100 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala new file mode 100644 index 0000000000000..398d8d98be9b3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.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.sql.execution.benchmark + +import java.io.File + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure TPCDS query performance. + * To run this: + * spark-submit --class --jars + */ +object TPCDSQueryBenchmark { + val conf = + new SparkConf() + .setMaster("local[1]") + .setAppName("test-sql-context") + .set("spark.sql.parquet.compression.codec", "snappy") + .set("spark.sql.shuffle.partitions", "4") + .set("spark.driver.memory", "3g") + .set("spark.executor.memory", "3g") + .set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) + + val spark = SparkSession.builder.config(conf).getOrCreate() + + val tables = Seq("catalog_page", "catalog_returns", "customer", "customer_address", + "customer_demographics", "date_dim", "household_demographics", "inventory", "item", + "promotion", "store", "store_returns", "catalog_sales", "web_sales", "store_sales", + "web_returns", "web_site", "reason", "call_center", "warehouse", "ship_mode", "income_band", + "time_dim", "web_page") + + def setupTables(dataLocation: String): Map[String, Long] = { + tables.map { tableName => + spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) + tableName -> spark.table(tableName).count() + }.toMap + } + + def tpcdsAll(dataLocation: String, queries: Seq[String]): Unit = { + require(dataLocation.nonEmpty, + "please modify the value of dataLocation to point to your local TPCDS data") + val tableSizes = setupTables(dataLocation) + spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + queries.foreach { name => + val queryString = fileToString(new File(Thread.currentThread().getContextClassLoader + .getResource(s"tpcds/$name.sql").getFile)) + + // This is an indirect hack to estimate the size of each query's input by traversing the + // logical plan and adding up the sizes of all tables that appear in the plan. Note that this + // currently doesn't take WITH subqueries into account which might lead to fairly inaccurate + // per-row processing time for those cases. + val queryRelations = scala.collection.mutable.HashSet[String]() + spark.sql(queryString).queryExecution.logical.map { + case ur @ UnresolvedRelation(t: TableIdentifier, _) => + queryRelations.add(t.table) + case lp: LogicalPlan => + lp.expressions.foreach { _ foreach { + case subquery: SubqueryExpression => + subquery.plan.foreach { + case ur @ UnresolvedRelation(t: TableIdentifier, _) => + queryRelations.add(t.table) + case _ => + } + case _ => + } + } + case _ => + } + val numRows = queryRelations.map(tableSizes.getOrElse(_, 0L)).sum + val benchmark = new Benchmark(s"TPCDS Snappy", numRows, 5) + benchmark.addCase(name) { i => + spark.sql(queryString).collect() + } + benchmark.run() + } + } + + def main(args: Array[String]): Unit = { + + // List of all TPC-DS queries + val tpcdsQueries = Seq( + "q1", "q2", "q3", "q4", "q5", "q6", "q7", "q8", "q9", "q10", "q11", + "q12", "q13", "q14a", "q14b", "q15", "q16", "q17", "q18", "q19", "q20", + "q21", "q22", "q23a", "q23b", "q24a", "q24b", "q25", "q26", "q27", "q28", "q29", "q30", + "q31", "q32", "q33", "q34", "q35", "q36", "q37", "q38", "q39a", "q39b", "q40", + "q41", "q42", "q43", "q44", "q45", "q46", "q47", "q48", "q49", "q50", + "q51", "q52", "q53", "q54", "q55", "q56", "q57", "q58", "q59", "q60", + "q61", "q62", "q63", "q64", "q65", "q66", "q67", "q68", "q69", "q70", + "q71", "q72", "q73", "q74", "q75", "q76", "q77", "q78", "q79", "q80", + "q81", "q82", "q83", "q84", "q85", "q86", "q87", "q88", "q89", "q90", + "q91", "q92", "q93", "q94", "q95", "q96", "q97", "q98", "q99") + + // In order to run this benchmark, please follow the instructions at + // https://github.com/databricks/spark-sql-perf/blob/master/README.md to generate the TPCDS data + // locally (preferably with a scale factor of 5 for benchmarking). Thereafter, the value of + // dataLocation below needs to be set to the location where the generated data is stored. + val dataLocation = "" + + tpcdsAll(dataLocation, queries = tpcdsQueries) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala deleted file mode 100644 index 228ae6f840ccf..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TPCDSBenchmark.scala +++ /dev/null @@ -1,1226 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.parquet - -import org.apache.spark.SparkConf -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.util.Benchmark - -/** - * Benchmark to measure TPCDS query performance. - * To run this: - * spark-submit --class --jars - */ -object TPCDSBenchmark { - val conf = new SparkConf() - conf.set("spark.sql.parquet.compression.codec", "snappy") - conf.set("spark.sql.shuffle.partitions", "4") - conf.set("spark.driver.memory", "3g") - conf.set("spark.executor.memory", "3g") - conf.set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) - conf.setMaster("local[1]") - conf.setAppName("test-sql-context") - - val spark = SparkSession.builder.config(conf).getOrCreate() - - // These queries a subset of the TPCDS benchmark queries and are taken from - // https://github.com/databricks/spark-sql-perf/blob/master/src/main/scala/com/databricks/spark/ - // sql/perf/tpcds/ImpalaKitQueries.scala - val tpcds = Seq( - ("q19", """ - |select - | i_brand_id, - | i_brand, - | i_manufact_id, - | i_manufact, - | sum(ss_ext_sales_price) ext_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join customer on (store_sales.ss_customer_sk = customer.c_customer_sk) - | join customer_address on - | (customer.c_current_addr_sk = customer_address.ca_address_sk) - |where - | ss_sold_date_sk between 2451484 and 2451513 - | and d_moy = 11 - | and d_year = 1999 - | and i_manager_id = 7 - | and substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5) - |group by - | i_brand, - | i_brand_id, - | i_manufact_id, - | i_manufact - |order by - | ext_price desc, - | i_brand, - | i_brand_id, - | i_manufact_id, - | i_manufact - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q19 1710 / 1858 8.7 114.5 1.0X - */ - - ("q27", """ - |select - | i_item_id, - | s_state, - | avg(ss_quantity) agg1, - | avg(ss_list_price) agg2, - | avg(ss_coupon_amt) agg3, - | avg(ss_sales_price) agg4 - |from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join customer_demographics on - | (store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | ss_sold_date_sk between 2450815 and 2451179 -- partition key filter - | and d_year = 1998 - | and cd_gender = 'F' - | and cd_marital_status = 'W' - | and cd_education_status = 'Primary' - | and s_state in ('WI', 'CA', 'TX', 'FL', 'WA', 'TN') - |group by - | i_item_id, - | s_state - |order by - | i_item_id, - | s_state - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q27 2016 / 2180 8.2 122.6 1.0X - */ - - ("q3", """ - |select - | dt.d_year, - | item.i_brand_id brand_id, - | item.i_brand brand, - | sum(ss_ext_sales_price) sum_agg - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim dt on (dt.d_date_sk = store_sales.ss_sold_date_sk) - |where - | item.i_manufact_id = 436 - | and dt.d_moy = 12 - | and (ss_sold_date_sk between 2451149 and 2451179 - | or ss_sold_date_sk between 2451514 and 2451544 - | or ss_sold_date_sk between 2451880 and 2451910 - | or ss_sold_date_sk between 2452245 and 2452275 - | or ss_sold_date_sk between 2452610 and 2452640) - |group by - | d_year, - | item.i_brand, - | item.i_brand_id - |order by - | d_year, - | sum_agg desc, - | brand_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q3 1073 / 1140 13.5 73.9 1.0X - */ - - ("q34", """ - |select - | c_last_name, - | c_first_name, - | c_salutation, - | c_preferred_cust_flag, - | ss_ticket_number, - | cnt - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | count(*) cnt - | from - | store_sales - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | date_dim.d_year in (1998, 1998 + 1, 1998 + 2) - | and (date_dim.d_dom between 1 and 3 - | or date_dim.d_dom between 25 and 28) - | and (household_demographics.hd_buy_potential = '>10000' - | or household_demographics.hd_buy_potential = 'unknown') - | and household_demographics.hd_vehicle_count > 0 - | and (case when household_demographics.hd_vehicle_count > 0 then - | household_demographics.hd_dep_count / household_demographics.hd_vehicle_count - | else null end) > 1.2 - | and ss_sold_date_sk between 2450816 and 2451910 -- partition key filter - | group by - | ss_ticket_number, - | ss_customer_sk - | ) dn - |join customer on (dn.ss_customer_sk = customer.c_customer_sk) - |where - | cnt between 15 and 20 - |order by - | c_last_name, - | c_first_name, - | c_salutation, - | c_preferred_cust_flag desc, - | ss_ticket_number, - | cnt - |limit 1000 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q34 1482 / 1734 10.0 100.4 1.0X - */ - - ("q42", """ - |select - | d_year, - | i_category_id, - | i_category, - | sum(ss_ext_sales_price) as total_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim dt on (dt.d_date_sk = store_sales.ss_sold_date_sk) - |where - | item.i_manager_id = 1 - | and dt.d_moy = 12 - | and dt.d_year = 1998 - | and ss_sold_date_sk between 2451149 and 2451179 -- partition key filter - |group by - | d_year, - | i_category_id, - | i_category - |order by - | total_price desc, - | d_year, - | i_category_id, - | i_category - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q42 1125 / 1357 12.9 77.4 1.0X - */ - - ("q43", """ - |select - | s_store_name, - | s_store_id, - | sum(case when (d_day_name = 'Sunday') then ss_sales_price else null end) sun_sales, - | sum(case when (d_day_name = 'Monday') then ss_sales_price else null end) mon_sales, - | sum(case when (d_day_name = 'Tuesday') then - | ss_sales_price else null end) tue_sales, - | sum(case when (d_day_name = 'Wednesday') then - | ss_sales_price else null end) wed_sales, - | sum(case when (d_day_name = 'Thursday') then - | ss_sales_price else null end) thu_sales, - | sum(case when (d_day_name = 'Friday') then ss_sales_price else null end) fri_sales, - | sum(case when (d_day_name = 'Saturday') then - | ss_sales_price else null end) sat_sales - |from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | s_gmt_offset = -5 - | and d_year = 1998 - | and ss_sold_date_sk between 2450816 and 2451179 -- partition key filter - |group by - | s_store_name, - | s_store_id - |order by - | s_store_name, - | s_store_id, - | sun_sales, - | mon_sales, - | tue_sales, - | wed_sales, - | thu_sales, - | fri_sales, - | sat_sales - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q43 1681 / 1985 8.6 116.1 1.0X - */ - - ("q46", """ - |select - | c_last_name, - | c_first_name, - | ca_city, - | bought_city, - | ss_ticket_number, - | amt, - | profit - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | ca_city bought_city, - | sum(ss_coupon_amt) amt, - | sum(ss_net_profit) profit - | from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join customer_address on - | (store_sales.ss_addr_sk = customer_address.ca_address_sk) - | where - | store.s_city in ('Midway', 'Concord', 'Spring Hill', 'Brownsville', 'Greenville') - | and (household_demographics.hd_dep_count = 5 - | or household_demographics.hd_vehicle_count = 3) - | and date_dim.d_dow in (6, 0) - | and date_dim.d_year in (1999, 1999 + 1, 1999 + 2) - | group by - | ss_ticket_number, - | ss_customer_sk, - | ss_addr_sk, - | ca_city - | ) dn - | join customer on (dn.ss_customer_sk = customer.c_customer_sk) - | join customer_address current_addr on - | (customer.c_current_addr_sk = current_addr.ca_address_sk) - |where - | current_addr.ca_city <> bought_city - |order by - | c_last_name, - | c_first_name, - | ca_city, - | bought_city, - | ss_ticket_number - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q46 2948 / 3218 5.1 196.1 1.0X - */ - - ("q52", """ - |select - | d_year, - | i_brand_id, - | i_brand, - | sum(ss_ext_sales_price) ext_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim dt on (store_sales.ss_sold_date_sk = dt.d_date_sk) - |where - | i_manager_id = 1 - | and d_moy = 12 - | and d_year = 1998 - | and ss_sold_date_sk between 2451149 and 2451179 -- partition key filter - |group by - | d_year, - | i_brand, - | i_brand_id - |order by - | d_year, - | ext_price desc, - | i_brand_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q52 1099 / 1228 13.2 75.7 1.0X - */ - - ("q53", """ - |select - | * - |from - | (select - | i_manufact_id, - | sum(ss_sales_price) sum_sales - | from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq in(1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, - | 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11) - | and ( - | (i_category in('Books', 'Children', 'Electronics') - | and i_class in('personal', 'portable', 'reference', 'self-help') - | and i_brand in('scholaramalgamalg #14', 'scholaramalgamalg #7', - | 'exportiunivamalg #9', 'scholaramalgamalg #9') - | ) - | or - | (i_category in('Women', 'Music', 'Men') - | and i_class in('accessories', 'classical', 'fragrances', 'pants') - | and i_brand in('amalgimporto #1', 'edu packscholar #1', - | 'exportiimporto #1', 'importoamalg #1') - | ) - | ) - | group by - | i_manufact_id, - | d_qoy - | ) tmp1 - |order by - | sum_sales, - | i_manufact_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q53 968 / 1020 15.0 66.6 1.0X - */ - - ("q55", """ - |select - | i_brand_id, - | i_brand, - | sum(ss_ext_sales_price) ext_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | i_manager_id = 36 - | and d_moy = 12 - | and d_year = 2001 - | and ss_sold_date_sk between 2452245 and 2452275 -- partition key filter - |group by - | i_brand, - | i_brand_id - |order by - | ext_price desc, - | i_brand_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q55 1002 / 1020 14.5 69.0 1.0X - */ - - ("q59", """ - |select - | s_store_name1, - | s_store_id1, - | d_week_seq1, - | sun_sales1 / sun_sales2, - | mon_sales1 / mon_sales2, - | tue_sales1 / tue_sales2, - | wed_sales1 / wed_sales2, - | thu_sales1 / thu_sales2, - | fri_sales1 / fri_sales2, - | sat_sales1 / sat_sales2 - |from - | (select - | s_store_name s_store_name1, - | wss.d_week_seq d_week_seq1, - | s_store_id s_store_id1, - | sun_sales sun_sales1, - | mon_sales mon_sales1, - | tue_sales tue_sales1, - | wed_sales wed_sales1, - | thu_sales thu_sales1, - | fri_sales fri_sales1, - | sat_sales sat_sales1 - | from - | (select - | d_week_seq, - | ss_store_sk, - | sum(case when(d_day_name = 'Sunday') then - | ss_sales_price else null end) sun_sales, - | sum(case when(d_day_name = 'Monday') then - | ss_sales_price else null end) mon_sales, - | sum(case when(d_day_name = 'Tuesday') then - | ss_sales_price else null end) tue_sales, - | sum(case when(d_day_name = 'Wednesday') then - | ss_sales_price else null end) wed_sales, - | sum(case when(d_day_name = 'Thursday') then - | ss_sales_price else null end) thu_sales, - | sum(case when(d_day_name = 'Friday') then - | ss_sales_price else null end) fri_sales, - | sum(case when(d_day_name = 'Saturday') then - | ss_sales_price else null end) sat_sales - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451088 and 2451452 - | group by - | d_week_seq, - | ss_store_sk - | ) wss - | join store on (wss.ss_store_sk = store.s_store_sk) - | join date_dim d on (wss.d_week_seq = d.d_week_seq) - | where - | d_month_seq between 1185 and 1185 + 11 - | ) y - | join - | (select - | s_store_name s_store_name2, - | wss.d_week_seq d_week_seq2, - | s_store_id s_store_id2, - | sun_sales sun_sales2, - | mon_sales mon_sales2, - | tue_sales tue_sales2, - | wed_sales wed_sales2, - | thu_sales thu_sales2, - | fri_sales fri_sales2, - | sat_sales sat_sales2 - | from - | (select - | d_week_seq, - | ss_store_sk, - | sum(case when(d_day_name = 'Sunday') then - | ss_sales_price else null end) sun_sales, - | sum(case when(d_day_name = 'Monday') then - | ss_sales_price else null end) mon_sales, - | sum(case when(d_day_name = 'Tuesday') then - | ss_sales_price else null end) tue_sales, - | sum(case when(d_day_name = 'Wednesday') then - | ss_sales_price else null end) wed_sales, - | sum(case when(d_day_name = 'Thursday') then - | ss_sales_price else null end) thu_sales, - | sum(case when(d_day_name = 'Friday') then - | ss_sales_price else null end) fri_sales, - | sum(case when(d_day_name = 'Saturday') then - | ss_sales_price else null end) sat_sales - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451088 and 2451452 - | group by - | d_week_seq, - | ss_store_sk - | ) wss - | join store on (wss.ss_store_sk = store.s_store_sk) - | join date_dim d on (wss.d_week_seq = d.d_week_seq) - | where - | d_month_seq between 1185 + 12 and 1185 + 23 - | ) x - | on (y.s_store_id1 = x.s_store_id2) - |where - | d_week_seq1 = d_week_seq2 - 52 - |order by - | s_store_name1, - | s_store_id1, - | d_week_seq1 - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q59 1624 / 1663 17.9 55.8 1.0X - */ - - ("q63", """ - |select - | * - |from - | (select - | i_manager_id, - | sum(ss_sales_price) sum_sales - | from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq in (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, - | 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11) - | and ( - | (i_category in('Books', 'Children', 'Electronics') - | and i_class in('personal', 'portable', 'refernece', 'self-help') - | and i_brand in('scholaramalgamalg #14', 'scholaramalgamalg #7', - | 'exportiunivamalg #9', 'scholaramalgamalg #9') - | ) - | or - | (i_category in('Women', 'Music', 'Men') - | and i_class in('accessories', 'classical', 'fragrances', 'pants') - | and i_brand in('amalgimporto #1', 'edu packscholar #1', - | 'exportiimporto #1', 'importoamalg #1') - | ) - | ) - | group by - | i_manager_id, - | d_moy - | ) tmp1 - |order by - | i_manager_id, - | sum_sales - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q63 979 / 1006 14.8 67.4 1.0X - */ - - ("q65", """ - |select - | s_store_name, - | i_item_desc, - | sc.revenue, - | i_current_price, - | i_wholesale_cost, - | i_brand - |from - | (select - | ss_store_sk, - | ss_item_sk, - | sum(ss_sales_price) as revenue - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq between 1212 and 1212 + 11 - | group by - | ss_store_sk, - | ss_item_sk - | ) sc - | join item on (sc.ss_item_sk = item.i_item_sk) - | join store on (sc.ss_store_sk = store.s_store_sk) - | join - | (select - | ss_store_sk, - | avg(revenue) as ave - | from - | (select - | ss_store_sk, - | ss_item_sk, - | sum(ss_sales_price) as revenue - | from - | store_sales - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451911 and 2452275 -- partition key filter - | and d_month_seq between 1212 and 1212 + 11 - | group by - | ss_store_sk, - | ss_item_sk - | ) sa - | group by - | ss_store_sk - | ) sb on (sc.ss_store_sk = sb.ss_store_sk) -- 676 rows - |where - | sc.revenue <= 0.1 * sb.ave - |order by - | s_store_name, - | i_item_desc - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q65 7770 / 8097 3.7 267.9 1.0X - */ - - ("q68", """ - |select - | c_last_name, - | c_first_name, - | ca_city, - | bought_city, - | ss_ticket_number, - | extended_price, - | extended_tax, - | list_price - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | ca_city bought_city, - | sum(ss_ext_sales_price) extended_price, - | sum(ss_ext_list_price) list_price, - | sum(ss_ext_tax) extended_tax - | from - | store_sales - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join customer_address on - | (store_sales.ss_addr_sk = customer_address.ca_address_sk) - | where - | store.s_city in('Midway', 'Fairview') - | --and date_dim.d_dom between 1 and 2 - | --and date_dim.d_year in(1999, 1999 + 1, 1999 + 2) - | -- and ss_date between '1999-01-01' and '2001-12-31' - | -- and dayofmonth(ss_date) in (1,2) - | and (household_demographics.hd_dep_count = 5 - | or household_demographics.hd_vehicle_count = 3) - | and d_date between '1999-01-01' and '1999-03-31' - | and ss_sold_date_sk between 2451180 and 2451269 - | -- partition key filter (3 months) - | group by - | ss_ticket_number, - | ss_customer_sk, - | ss_addr_sk, - | ca_city - | ) dn - | join customer on (dn.ss_customer_sk = customer.c_customer_sk) - | join customer_address current_addr on - | (customer.c_current_addr_sk = current_addr.ca_address_sk) - |where - | current_addr.ca_city <> bought_city - |order by - | c_last_name, - | ss_ticket_number - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q68 3105 / 3405 4.8 206.5 1.0X - */ - - ("q7", """ - |select - | i_item_id, - | avg(ss_quantity) agg1, - | avg(ss_list_price) agg2, - | avg(ss_coupon_amt) agg3, - | avg(ss_sales_price) agg4 - |from - | store_sales - | join customer_demographics on - | (store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join promotion on (store_sales.ss_promo_sk = promotion.p_promo_sk) - | join date_dim on (ss_sold_date_sk = d_date_sk) - |where - | cd_gender = 'F' - | and cd_marital_status = 'W' - | and cd_education_status = 'Primary' - | and (p_channel_email = 'N' - | or p_channel_event = 'N') - | and d_year = 1998 - | and ss_sold_date_sk between 2450815 and 2451179 -- partition key filter - |group by - | i_item_id - |order by - | i_item_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q7 2042 / 2333 8.1 124.2 1.0X - */ - - ("q73", """ - |select - | c_last_name, - | c_first_name, - | c_salutation, - | c_preferred_cust_flag, - | ss_ticket_number, - | cnt - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | count(*) cnt - | from - | store_sales - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | -- join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | store.s_county in - | ('Williamson County','Franklin Parish','Bronx County','Orange County') - | -- and date_dim.d_dom between 1 and 2 - | -- and date_dim.d_year in(1998, 1998 + 1, 1998 + 2) - | -- and ss_date between '1999-01-01' and '2001-12-02' - | -- and dayofmonth(ss_date) in (1,2) - | -- partition key filter - | -- and ss_sold_date_sk in (2450816, 2450846, 2450847, 2450874, 2450875, 2450905, - | -- 2450906, 2450935, 2450936, 2450966, 2450967, - | -- 2450996, 2450997, 2451027, 2451028, 2451058, 2451059, - | -- 2451088, 2451089, 2451119, 2451120, 2451149, - | -- 2451150, 2451180, 2451181, 2451211, 2451212, 2451239, - | -- 2451240, 2451270, 2451271, 2451300, 2451301, - | -- 2451331, 2451332, 2451361, 2451362, 2451392, 2451393, - | -- 2451423, 2451424, 2451453, 2451454, 2451484, - | -- 2451485, 2451514, 2451515, 2451545, 2451546, 2451576, - | -- 2451577, 2451605, 2451606, 2451636, 2451637, - | -- 2451666, 2451667, 2451697, 2451698, 2451727, 2451728, - | -- 2451758, 2451759, 2451789, 2451790, 2451819, - | -- 2451820, 2451850, 2451851, 2451880, 2451881) - | and (household_demographics.hd_buy_potential = '>10000' - | or household_demographics.hd_buy_potential = 'unknown') - | and household_demographics.hd_vehicle_count > 0 - | and case when household_demographics.hd_vehicle_count > 0 then - | household_demographics.hd_dep_count / household_demographics.hd_vehicle_count - | else null end > 1 - | and ss_sold_date_sk between 2451180 and 2451269 - | -- partition key filter (3 months) - | group by - | ss_ticket_number, - | ss_customer_sk - | ) dj - | join customer on (dj.ss_customer_sk = customer.c_customer_sk) - |where - | cnt between 1 and 5 - |order by - | cnt desc - |limit 1000 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q73 1124 / 1221 13.1 76.5 1.0X - */ - - ("q79", """ - |select - | c_last_name, - | c_first_name, - | substr(s_city, 1, 30) as city, - | ss_ticket_number, - | amt, - | profit - |from - | (select - | ss_ticket_number, - | ss_customer_sk, - | s_city, - | sum(ss_coupon_amt) amt, - | sum(ss_net_profit) profit - | from - | store_sales - | join household_demographics on - | (store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | where - | store.s_number_employees between 200 and 295 - | and (household_demographics.hd_dep_count = 8 - | or household_demographics.hd_vehicle_count > 0) - | and date_dim.d_dow = 1 - | and date_dim.d_year in (1998, 1998 + 1, 1998 + 2) - | -- and ss_date between '1998-01-01' and '2000-12-25' - | -- 156 days - | and d_date between '1999-01-01' and '1999-03-31' - | and ss_sold_date_sk between 2451180 and 2451269 -- partition key filter - | group by - | ss_ticket_number, - | ss_customer_sk, - | ss_addr_sk, - | s_city - | ) ms - | join customer on (ms.ss_customer_sk = customer.c_customer_sk) - |order by - | c_last_name, - | c_first_name, - | city, - | profit - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q79 2029 / 2488 7.3 137.5 1.0X - */ - - ("q8", - """ - |select s_store_name - | ,sum(ss_net_profit) - | from store_sales - | ,date_dim - | ,store, - | (select distinct a01.ca_zip - | from - | (SELECT substr(ca_zip,1,5) ca_zip - | FROM customer_address - | WHERE substr(ca_zip,1,5) IN ('89436', '30868', '65085', '22977', '83927', '77557', - | '58429', '40697', '80614', '10502', '32779', - | '91137', '61265', '98294', '17921', '18427', '21203', '59362', '87291', '84093', - | '21505', '17184', '10866', '67898', '25797', - | '28055', '18377', '80332', '74535', '21757', '29742', '90885', '29898', '17819', - | '40811', '25990', '47513', '89531', '91068', - | '10391', '18846', '99223', '82637', '41368', '83658', '86199', '81625', '26696', - | '89338', '88425', '32200', '81427', '19053', - | '77471', '36610', '99823', '43276', '41249', '48584', '83550', '82276', '18842', - | '78890', '14090', '38123', '40936', '34425', - | '19850', '43286', '80072', '79188', '54191', '11395', '50497', '84861', '90733', - | '21068', '57666', '37119', '25004', '57835', - | '70067', '62878', '95806', '19303', '18840', '19124', '29785', '16737', '16022', - | '49613', '89977', '68310', '60069', '98360', - | '48649', '39050', '41793', '25002', '27413', '39736', '47208', '16515', '94808', - | '57648', '15009', '80015', '42961', '63982', - | '21744', '71853', '81087', '67468', '34175', '64008', '20261', '11201', '51799', - | '48043', '45645', '61163', '48375', '36447', - | '57042', '21218', '41100', '89951', '22745', '35851', '83326', '61125', '78298', - | '80752', '49858', '52940', '96976', '63792', - | '11376', '53582', '18717', '90226', '50530', '94203', '99447', '27670', '96577', - | '57856', '56372', '16165', '23427', '54561', - | '28806', '44439', '22926', '30123', '61451', '92397', '56979', '92309', '70873', - | '13355', '21801', '46346', '37562', '56458', - | '28286', '47306', '99555', '69399', '26234', '47546', '49661', '88601', '35943', - | '39936', '25632', '24611', '44166', '56648', - | '30379', '59785', '11110', '14329', '93815', '52226', '71381', '13842', '25612', - | '63294', '14664', '21077', '82626', '18799', - | '60915', '81020', '56447', '76619', '11433', '13414', '42548', '92713', '70467', - | '30884', '47484', '16072', '38936', '13036', - | '88376', '45539', '35901', '19506', '65690', '73957', '71850', '49231', '14276', - | '20005', '18384', '76615', '11635', '38177', - | '55607', '41369', '95447', '58581', '58149', '91946', '33790', '76232', '75692', - | '95464', '22246', '51061', '56692', '53121', - | '77209', '15482', '10688', '14868', '45907', '73520', '72666', '25734', '17959', - | '24677', '66446', '94627', '53535', '15560', - | '41967', '69297', '11929', '59403', '33283', '52232', '57350', '43933', '40921', - | '36635', '10827', '71286', '19736', '80619', - | '25251', '95042', '15526', '36496', '55854', '49124', '81980', '35375', '49157', - | '63512', '28944', '14946', '36503', '54010', - | '18767', '23969', '43905', '66979', '33113', '21286', '58471', '59080', '13395', - | '79144', '70373', '67031', '38360', '26705', - | '50906', '52406', '26066', '73146', '15884', '31897', '30045', '61068', '45550', - | '92454', '13376', '14354', '19770', '22928', - | '97790', '50723', '46081', '30202', '14410', '20223', '88500', '67298', '13261', - | '14172', '81410', '93578', '83583', '46047', - | '94167', '82564', '21156', '15799', '86709', '37931', '74703', '83103', '23054', - | '70470', '72008', '35709', '91911', '69998', - | '20961', '70070', '63197', '54853', '88191', '91830', '49521', '19454', '81450', - | '89091', '62378', '31904', '61869', '51744', - | '36580', '85778', '36871', '48121', '28810', '83712', '45486', '67393', '26935', - | '42393', '20132', '55349', '86057', '21309', - | '80218', '10094', '11357', '48819', '39734', '40758', '30432', '21204', '29467', - | '30214', '61024', '55307', '74621', '11622', - | '68908', '33032', '52868', '99194', '99900', '84936', '69036', '99149', '45013', - | '32895', '59004', '32322', '14933', '32936', - | '33562', '72550', '27385', '58049', '58200', '16808', '21360', '32961', '18586', - | '79307', '15492')) a01 - | inner join - | (select ca_zip - | from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt - | FROM customer_address, customer - | WHERE ca_address_sk = c_current_addr_sk and - | c_preferred_cust_flag='Y' - | group by ca_zip - | having count(*) > 10)A1 - | ) b11 - | on (a01.ca_zip = b11.ca_zip )) A2 - | where ss_store_sk = s_store_sk - | and ss_sold_date_sk = d_date_sk - | and ss_sold_date_sk between 2451271 and 2451361 - | and d_qoy = 2 and d_year = 1999 - | and (substr(s_zip,1,2) = substr(a2.ca_zip,1,2)) - | group by s_store_name - | order by s_store_name - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q8 1737 / 2197 8.7 115.6 1.0X - */ - - ("q82", """ - |select - | i_item_id, - | i_item_desc, - | i_current_price - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join inventory on (item.i_item_sk = inventory.inv_item_sk) - | join date_dim on (inventory.inv_date_sk = date_dim.d_date_sk) - |where - | i_current_price between 30 and 30 + 30 - | and i_manufact_id in (437, 129, 727, 663) - | and inv_quantity_on_hand between 100 and 500 - |group by - | i_item_id, - | i_item_desc, - | i_current_price - |order by - | i_item_id - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q82 9399 / 10245 6.8 147.2 1.0X - */ - - ("q89", """ - |select - | * - |from - | (select - | i_category, - | i_class, - | i_brand, - | s_store_name, - | s_company_name, - | d_moy, - | sum(ss_sales_price) sum_sales - | from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join store on (store_sales.ss_store_sk = store.s_store_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - | where - | ss_sold_date_sk between 2451545 and 2451910 -- partition key filter - | and d_year in (2000) - | and ((i_category in('Home', 'Books', 'Electronics') - | and i_class in('wallpaper', 'parenting', 'musical')) - | or (i_category in('Shoes', 'Jewelry', 'Men') - | and i_class in('womens', 'birdal', 'pants')) - | ) - | group by - | i_category, - | i_class, - | i_brand, - | s_store_name, - | s_company_name, - | d_moy - | ) tmp1 - |order by - | sum_sales, - | s_store_name - |limit 100 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q89 1122 / 1274 12.9 77.2 1.0X - */ - - ("q98", """ - |select - | i_item_desc, - | i_category, - | i_class, - | i_current_price, - | sum(ss_ext_sales_price) as itemrevenue - |from - | store_sales - | join item on (store_sales.ss_item_sk = item.i_item_sk) - | join date_dim on (store_sales.ss_sold_date_sk = date_dim.d_date_sk) - |where - | ss_sold_date_sk between 2451911 and 2451941 - | -- partition key filter (1 calendar month) - | and d_date between '2001-01-01' and '2001-01-31' - | and i_category in('Jewelry', 'Sports', 'Books') - |group by - | i_item_id, - | i_item_desc, - | i_category, - | i_class, - | i_current_price - |order by - | i_category, - | i_class, - | i_item_id, - | i_item_desc - | -- revenueratio - |limit 1000 - """.stripMargin), - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - q98 1235 / 1542 11.8 85.0 1.0X - */ - - ("ss_max", """ - |select - | count(*) as total, - | max(ss_sold_date_sk) as max_ss_sold_date_sk, - | max(ss_sold_time_sk) as max_ss_sold_time_sk, - | max(ss_item_sk) as max_ss_item_sk, - | max(ss_customer_sk) as max_ss_customer_sk, - | max(ss_cdemo_sk) as max_ss_cdemo_sk, - | max(ss_hdemo_sk) as max_ss_hdemo_sk, - | max(ss_addr_sk) as max_ss_addr_sk, - | max(ss_store_sk) as max_ss_store_sk, - | max(ss_promo_sk) as max_ss_promo_sk - |from store_sales - """.stripMargin) - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - TPCDS Snappy (scale = 5): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - ss_max 2305 / 2731 6.2 160.0 1.0X - */ - - ).toArray - - val tables = Seq("customer", "customer_address", "customer_demographics", "date_dim", - "household_demographics", "inventory", "item", "promotion", "store", "catalog_sales", - "web_sales", "store_sales") - - def setupTables(dataLocation: String): Map[String, Long] = { - tables.map { tableName => - spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) - tableName -> spark.table(tableName).count() - }.toMap - } - - def tpcdsAll(dataLocation: String): Unit = { - require(dataLocation.nonEmpty, - "please modify the value of dataLocation to point to your local TPCDS data") - val tableSizes = setupTables(dataLocation) - spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - tpcds.filter(q => q._1 != "").foreach { - case (name: String, query: String) => - val numRows = spark.sql(query).queryExecution.logical.map { - case ur@UnresolvedRelation(t: TableIdentifier, _) => - tableSizes.getOrElse(t.table, throw new RuntimeException(s"${t.table} not found.")) - case _ => 0L - }.sum - val benchmark = new Benchmark("TPCDS Snappy (scale = 5)", numRows, 5) - benchmark.addCase(name) { i => - spark.sql(query).collect() - } - benchmark.run() - } - } - - def main(args: Array[String]): Unit = { - - // In order to run this benchmark, please follow the instructions at - // https://github.com/databricks/spark-sql-perf/blob/master/README.md to generate the TPCDS data - // locally (preferably with a scale factor of 5 for benchmarking). Thereafter, the value of - // dataLocation below needs to be set to the location where the generated data is stored. - val dataLocation = "" - - tpcdsAll(dataLocation) - } -} From 4148a9c2c3856ff7dbd914dd235df7623f18df93 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 21 May 2016 12:46:25 -0700 Subject: [PATCH 0335/1470] [SPARK-15452][SQL] Mark aggregator API as experimental ## What changes were proposed in this pull request? The Aggregator API was introduced in 2.0 for Dataset. All typed Dataset APIs should still be marked as experimental in 2.0. ## How was this patch tested? N/A - annotation only change. Author: Reynold Xin Closes #13226 from rxin/SPARK-15452. (cherry picked from commit 201a51f36682726d78d9d2fe2c388093bb860ee0) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/expressions/Aggregator.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala index baae9dd2d5e3e..51179a528c503 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.expressions +import org.apache.spark.annotation.Experimental import org.apache.spark.sql.{Dataset, Encoder, TypedColumn} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression /** + * :: Experimental :: * A base class for user-defined aggregations, which can be used in [[Dataset]] operations to take * all of the elements of a group and reduce them to a single value. * @@ -48,6 +50,7 @@ import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression * @tparam OUT The type of the final output result. * @since 1.6.0 */ +@Experimental abstract class Aggregator[-IN, BUF, OUT] extends Serializable { /** From 6871deb937fd6d6185b1d2a7a2ea36535ce303ea Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Sat, 21 May 2016 16:08:31 -0700 Subject: [PATCH 0336/1470] [SPARK-15280] Input/Output] Refactored OrcOutputWriter and moved serialization to a new class. ## What changes were proposed in this pull request? Refactoring: Separated ORC serialization logic from OrcOutputWriter and moved to a new class called OrcSerializer. ## How was this patch tested? Manual tests & existing tests. Author: Ergin Seyfe Closes #13066 from seyfe/orc_serializer. (cherry picked from commit c18fa464f404ed2612f8c4d355cb0544b355975b) Signed-off-by: Yin Huai --- .../spark/sql/hive/orc/OrcRelation.scala | 84 ++++++++++--------- 1 file changed, 45 insertions(+), 39 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 6e55137dd78e1..38f50c112a236 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -149,39 +149,70 @@ private[sql] class DefaultSource } } -private[orc] class OrcOutputWriter( - path: String, - bucketId: Option[Int], - dataSchema: StructType, - context: TaskAttemptContext) - extends OutputWriter with HiveInspectors { +private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) + extends HiveInspectors { + + def serialize(row: InternalRow): Writable = { + wrapOrcStruct(cachedOrcStruct, structOI, row) + serializer.serialize(cachedOrcStruct, structOI) + } - private val serializer = { + private[this] val serializer = { val table = new Properties() table.setProperty("columns", dataSchema.fieldNames.mkString(",")) table.setProperty("columns.types", dataSchema.map(_.dataType.catalogString).mkString(":")) val serde = new OrcSerde - val configuration = context.getConfiguration - serde.initialize(configuration, table) + serde.initialize(conf, table) serde } - // Object inspector converted from the schema of the relation to be written. - private val structOI = { + // Object inspector converted from the schema of the relation to be serialized. + private[this] val structOI = { val typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(dataSchema.catalogString) OrcStruct.createObjectInspector(typeInfo.asInstanceOf[StructTypeInfo]) .asInstanceOf[SettableStructObjectInspector] } + private[this] val cachedOrcStruct = structOI.create().asInstanceOf[OrcStruct] + + private[this] def wrapOrcStruct( + struct: OrcStruct, + oi: SettableStructObjectInspector, + row: InternalRow): Unit = { + val fieldRefs = oi.getAllStructFieldRefs + var i = 0 + while (i < fieldRefs.size) { + + oi.setStructFieldData( + struct, + fieldRefs.get(i), + wrap( + row.get(i, dataSchema(i).dataType), + fieldRefs.get(i).getFieldObjectInspector, + dataSchema(i).dataType)) + i += 1 + } + } +} + +private[orc] class OrcOutputWriter( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext) + extends OutputWriter { + + private[this] val conf = context.getConfiguration + + private[this] val serializer = new OrcSerializer(dataSchema, conf) + // `OrcRecordWriter.close()` creates an empty file if no rows are written at all. We use this // flag to decide whether `OrcRecordWriter.close()` needs to be called. private var recordWriterInstantiated = false private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - - val conf = context.getConfiguration val uniqueWriteJobId = conf.get("spark.sql.sources.writeJobUUID") val taskAttemptId = context.getTaskAttemptID val partition = taskAttemptId.getTaskID.getId @@ -206,33 +237,8 @@ private[orc] class OrcOutputWriter( override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") - private def wrapOrcStruct( - struct: OrcStruct, - oi: SettableStructObjectInspector, - row: InternalRow): Unit = { - val fieldRefs = oi.getAllStructFieldRefs - var i = 0 - while (i < fieldRefs.size) { - - oi.setStructFieldData( - struct, - fieldRefs.get(i), - wrap( - row.get(i, dataSchema(i).dataType), - fieldRefs.get(i).getFieldObjectInspector, - dataSchema(i).dataType)) - i += 1 - } - } - - val cachedOrcStruct = structOI.create().asInstanceOf[OrcStruct] - override protected[sql] def writeInternal(row: InternalRow): Unit = { - wrapOrcStruct(cachedOrcStruct, structOI, row) - - recordWriter.write( - NullWritable.get(), - serializer.serialize(cachedOrcStruct, structOI)) + recordWriter.write(NullWritable.get(), serializer.serialize(row)) } override def close(): Unit = { From 9c20c7a33b86c1db5899900b21e36f80f08a1b8a Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 21 May 2016 20:07:34 -0700 Subject: [PATCH 0337/1470] [SPARK-15330][SQL] Implement Reset Command #### What changes were proposed in this pull request? Like `Set` Command in Hive, `Reset` is also supported by Hive. See the link: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli Below is the related Hive JIRA: https://issues.apache.org/jira/browse/HIVE-3202 This PR is to implement such a command for resetting the SQL-related configuration to the default values. One of the use case shown in HIVE-3202 is listed below: > For the purpose of optimization we set various configs per query. It's worthy but all those configs should be reset every time for next query. #### How was this patch tested? Added a test case. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13121 from gatorsmile/resetCommand. (cherry picked from commit 8f0a3d5bcba313dc3b70d4aa9a8ba2aa2d276062) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 12 +++++ .../sql/execution/command/SetCommand.scala | 16 ++++++ .../spark/sql/internal/SQLConfSuite.scala | 49 ++++++++++++++++++- .../hive/thriftserver/SparkSQLCLIDriver.scala | 6 +-- 5 files changed, 82 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 06ac37b7f83ed..848c59e3b8a53 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -120,6 +120,7 @@ statement | ADD identifier .*? #addResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration + | RESET #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -633,7 +634,7 @@ nonReserved | GROUPING | CUBE | ROLLUP | EXPLAIN | FORMAT | LOGICAL | FORMATTED | CODEGEN | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF - | SET + | SET | RESET | VIEW | REPLACE | IF | NO | DATA @@ -748,6 +749,7 @@ MAP: 'MAP'; STRUCT: 'STRUCT'; COMMENT: 'COMMENT'; SET: 'SET'; +RESET: 'RESET'; DATA: 'DATA'; START: 'START'; TRANSACTION: 'TRANSACTION'; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2966eefd07c77..2e3ac9706daf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -75,6 +75,18 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } } + /** + * Create a [[ResetCommand]] logical plan. + * Example SQL : + * {{{ + * RESET; + * }}} + */ + override def visitResetConfiguration( + ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { + ResetCommand + } + /** * Create an [[AnalyzeTableCommand]] command. This currently only implements the NOSCAN * option (other options are passed on to Hive) e.g.: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 282f26ce998fe..b0e2d03af070d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -116,3 +116,19 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm override def run(sparkSession: SparkSession): Seq[Row] = runFunc(sparkSession) } + +/** + * This command is for resetting SQLConf to the default values. Command that runs + * {{{ + * reset; + * }}} + */ +case object ResetCommand extends RunnableCommand with Logging { + + override def run(sparkSession: SparkSession): Seq[Row] = { + sparkSession.sessionState.conf.clear() + Seq.empty[Row] + } + + override val output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 0296229100a24..f8227e3bd6ee8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -99,7 +99,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { test("deprecated property") { spark.sqlContext.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) - try{ + try { sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") assert(spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) === 10) } finally { @@ -107,6 +107,53 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + test("reset - public conf") { + spark.sqlContext.conf.clear() + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + try { + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) + sql(s"reset") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + } + } + + test("reset - internal conf") { + spark.sqlContext.conf.clear() + val original = spark.conf.get(SQLConf.NATIVE_VIEW) + try { + assert(spark.conf.get(SQLConf.NATIVE_VIEW) === true) + sql(s"set ${SQLConf.NATIVE_VIEW.key}=false") + assert(spark.conf.get(SQLConf.NATIVE_VIEW) === false) + assert(sql(s"set").where(s"key = '${SQLConf.NATIVE_VIEW.key}'").count() == 1) + sql(s"reset") + assert(spark.conf.get(SQLConf.NATIVE_VIEW) === true) + assert(sql(s"set").where(s"key = '${SQLConf.NATIVE_VIEW.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.NATIVE_VIEW}=$original") + } + } + + test("reset - user-defined conf") { + spark.sqlContext.conf.clear() + val userDefinedConf = "x.y.z.reset" + try { + assert(spark.conf.getOption(userDefinedConf).isEmpty) + sql(s"set $userDefinedConf=false") + assert(spark.conf.get(userDefinedConf) === "false") + assert(sql(s"set").where(s"key = '$userDefinedConf'").count() == 1) + sql(s"reset") + assert(spark.conf.getOption(userDefinedConf).isEmpty) + } finally { + spark.conf.unset(userDefinedConf) + } + } + test("invalid conf value") { spark.sqlContext.conf.clear() val e = intercept[IllegalArgumentException] { 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 1402e0a687290..33ff8aee79969 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 @@ -32,8 +32,8 @@ 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.{AddResourceProcessor, CommandProcessor, - CommandProcessorFactory, SetProcessor} +import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, CommandProcessor} +import org.apache.hadoop.hive.ql.processors.{CommandProcessorFactory, ResetProcessor, SetProcessor} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket @@ -312,7 +312,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (proc != null) { // scalastyle:off println if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || - proc.isInstanceOf[AddResourceProcessor]) { + proc.isInstanceOf[AddResourceProcessor] || proc.isInstanceOf[ResetProcessor]) { val driver = new SparkSQLDriver driver.init() From 9a08c9f1c1ad4d1c0d2411e6fdf93b98925f8480 Mon Sep 17 00:00:00 2001 From: xin Wu Date: Sat, 21 May 2016 21:41:12 -0700 Subject: [PATCH 0338/1470] [SPARK-15206][SQL] add testcases for distinct aggregate in having clause ## What changes were proposed in this pull request? Add new test cases for including distinct aggregate in having clause in 2.0 branch. This is a followup PR for [#12974](https://github.com/apache/spark/pull/12974), which is for 1.6 branch. Author: xin Wu Closes #12984 from xwu0226/SPARK-15206. (cherry picked from commit df9adb5ec994f054b2fa58e492867bbc5a60c234) Signed-off-by: Wenchen Fan --- .../execution/AggregationQuerySuite.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index a2bae2e81f30a..9fc5628b28dbb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -958,6 +958,37 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(11) :: Nil) } } + + test("SPARK-15206: single distinct aggregate function in having clause") { + checkAnswer( + sql( + """ + |select key, count(distinct value1) + |from agg2 group by key + |having count(distinct value1) > 0 + """.stripMargin), + Seq( + Row(null, 3), + Row(1, 2), + Row(2, 2) + ) + ) + } + + test("SPARK-15206: multiple distinct aggregate function in having clause") { + checkAnswer( + sql( + """ + |select key, count(distinct value1), count(distinct value2) + |from agg2 group by key + |having count(distinct value1) > 0 and count(distinct value2) = 3 + """.stripMargin), + Seq( + Row(null, 3, 3), + Row(1, 2, 3) + ) + ) + } } From fd7e83119948187212ce75f4837cd7487d8a128a Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Sat, 21 May 2016 23:01:14 -0700 Subject: [PATCH 0339/1470] [SPARK-15415][SQL] Fix BroadcastHint when autoBroadcastJoinThreshold is 0 or -1 ## What changes were proposed in this pull request? This PR makes BroadcastHint more deterministic by using a special isBroadcastable property instead of setting the sizeInBytes to 1. See https://issues.apache.org/jira/browse/SPARK-15415 ## How was this patch tested? Added testcases to test if the broadcast hash join is included in the plan when the BroadcastHint is supplied and also tests for propagation of the joins. Author: Jurriaan Pruis Closes #13244 from jurriaan/broadcast-hint. (cherry picked from commit 223f6339088434eb3590c2f42091a38f05f1e5db) Signed-off-by: Reynold Xin --- .../catalyst/plans/logical/LogicalPlan.scala | 3 +- .../catalyst/plans/logical/Statistics.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 29 +++-- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../execution/joins/BroadcastJoinSuite.scala | 103 +++++++++++++++--- 5 files changed, 114 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 45ac126a72f5c..4984f235b412c 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 @@ -313,7 +313,8 @@ abstract class UnaryNode extends LogicalPlan { // (product of children). sizeInBytes = 1 } - Statistics(sizeInBytes = sizeInBytes) + + child.statistics.copy(sizeInBytes = sizeInBytes) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index 9ac4c3a2a56c8..63f86ad09412c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -32,4 +32,4 @@ package org.apache.spark.sql.catalyst.plans.logical * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it * defaults to the product of children's `sizeInBytes`. */ -private[sql] case class Statistics(sizeInBytes: BigInt) +private[sql] case class Statistics(sizeInBytes: BigInt, isBroadcastable: Boolean = false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 732b0d7919c39..bed48b6f6101b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -163,7 +163,9 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation val leftSize = left.statistics.sizeInBytes val rightSize = right.statistics.sizeInBytes val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize - Statistics(sizeInBytes = sizeInBytes) + val isBroadcastable = left.statistics.isBroadcastable || right.statistics.isBroadcastable + + Statistics(sizeInBytes = sizeInBytes, isBroadcastable = isBroadcastable) } } @@ -183,7 +185,7 @@ case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(le duplicateResolved override def statistics: Statistics = { - Statistics(sizeInBytes = left.statistics.sizeInBytes) + left.statistics.copy() } } @@ -330,6 +332,16 @@ case class Join( case UsingJoin(_, _) => false case _ => resolvedExceptNatural } + + override def statistics: Statistics = joinType match { + case LeftAnti | LeftSemi => + // LeftSemi and LeftAnti won't ever be bigger than left + left.statistics.copy() + case _ => + // make sure we don't propagate isBroadcastable in other joins, because + // they could explode the size. + super.statistics.copy(isBroadcastable = false) + } } /** @@ -338,9 +350,8 @@ case class Join( case class BroadcastHint(child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output - // We manually set statistics of BroadcastHint to smallest value to make sure - // the plan wrapped by BroadcastHint will be considered to broadcast later. - override def statistics: Statistics = Statistics(sizeInBytes = 1) + // set isBroadcastable to true so the child will be broadcasted + override def statistics: Statistics = super.statistics.copy(isBroadcastable = true) } case class InsertIntoTable( @@ -465,7 +476,7 @@ case class Aggregate( override def statistics: Statistics = { if (groupingExpressions.isEmpty) { - Statistics(sizeInBytes = 1) + super.statistics.copy(sizeInBytes = 1) } else { super.statistics } @@ -638,7 +649,7 @@ case class GlobalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryN override lazy val statistics: Statistics = { val limit = limitExpr.eval().asInstanceOf[Int] val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum - Statistics(sizeInBytes = sizeInBytes) + child.statistics.copy(sizeInBytes = sizeInBytes) } } @@ -653,7 +664,7 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo override lazy val statistics: Statistics = { val limit = limitExpr.eval().asInstanceOf[Int] val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum - Statistics(sizeInBytes = sizeInBytes) + child.statistics.copy(sizeInBytes = sizeInBytes) } } @@ -690,7 +701,7 @@ case class Sample( if (sizeInBytes == 0) { sizeInBytes = 1 } - Statistics(sizeInBytes = sizeInBytes) + child.statistics.copy(sizeInBytes = sizeInBytes) } override protected def otherCopyArgs: Seq[AnyRef] = isTableSample :: Nil 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 3343039ae1cf3..664e7f566145b 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 @@ -92,7 +92,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * Matches a plan whose output should be small enough to be used in broadcast join. */ private def canBroadcast(plan: LogicalPlan): Boolean = { - plan.statistics.sizeInBytes <= conf.autoBroadcastJoinThreshold + plan.statistics.isBroadcastable || + plan.statistics.sizeInBytes <= conf.autoBroadcastJoinThreshold } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 730ec43556c95..e681b88685b58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -22,9 +22,12 @@ import scala.reflect.ClassTag import org.scalatest.BeforeAndAfterAll import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext} -import org.apache.spark.sql.{QueryTest, SparkSession} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} import org.apache.spark.sql.execution.exchange.EnsureRequirements +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils /** * Test various broadcast join operators. @@ -33,7 +36,9 @@ import org.apache.spark.sql.functions._ * unsafe map in [[org.apache.spark.sql.execution.joins.UnsafeHashedRelation]] is not triggered * without serializing the hashed relation, which does not happen in local mode. */ -class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { +class BroadcastJoinSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + protected var spark: SparkSession = null /** @@ -56,30 +61,100 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { /** * Test whether the specified broadcast join updates the peak execution memory accumulator. */ - private def testBroadcastJoin[T: ClassTag](name: String, joinType: String): Unit = { + private def testBroadcastJoinPeak[T: ClassTag](name: String, joinType: String): Unit = { AccumulatorSuite.verifyPeakExecutionMemorySet(spark.sparkContext, name) { - val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") - val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") - // Comparison at the end is for broadcast left semi join - val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") - val df3 = df1.join(broadcast(df2), joinExpression, joinType) - val plan = - EnsureRequirements(spark.sessionState.conf).apply(df3.queryExecution.sparkPlan) - assert(plan.collect { case p: T => p }.size === 1) + val plan = testBroadcastJoin[T](joinType) plan.executeCollect() } } + private def testBroadcastJoin[T: ClassTag](joinType: String, + forceBroadcast: Boolean = false): SparkPlan = { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + var df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") + + // Comparison at the end is for broadcast left semi join + val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") + val df3 = if (forceBroadcast) { + df1.join(broadcast(df2), joinExpression, joinType) + } else { + df1.join(df2, joinExpression, joinType) + } + val plan = + EnsureRequirements(spark.sessionState.conf).apply(df3.queryExecution.sparkPlan) + assert(plan.collect { case p: T => p }.size === 1) + + return plan + } + test("unsafe broadcast hash join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast hash join", "inner") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast hash join", "inner") } test("unsafe broadcast hash outer join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast hash outer join", "left_outer") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast hash outer join", "left_outer") } test("unsafe broadcast left semi join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoinExec]("unsafe broadcast left semi join", "leftsemi") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast left semi join", "leftsemi") + } + + test("broadcast hint isn't bothered by authBroadcastJoinThreshold set to low values") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + testBroadcastJoin[BroadcastHashJoinExec]("inner", true) + } + } + + test("broadcast hint isn't bothered by a disabled authBroadcastJoinThreshold") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + testBroadcastJoin[BroadcastHashJoinExec]("inner", true) + } + } + + test("broadcast hint isn't propagated after a join") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") + val df3 = df1.join(broadcast(df2), Seq("key"), "inner").drop(df2("key")) + + val df4 = spark.createDataFrame(Seq((1, "5"), (2, "5"))).toDF("key", "value") + val df5 = df4.join(df3, Seq("key"), "inner") + + val plan = + EnsureRequirements(spark.sessionState.conf).apply(df5.queryExecution.sparkPlan) + + assert(plan.collect { case p: BroadcastHashJoinExec => p }.size === 1) + assert(plan.collect { case p: SortMergeJoinExec => p }.size === 1) + } } + private def assertBroadcastJoin(df : Dataset[Row]) : Unit = { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + val joined = df1.join(df, Seq("key"), "inner") + + val plan = + EnsureRequirements(spark.sessionState.conf).apply(joined.queryExecution.sparkPlan) + + assert(plan.collect { case p: BroadcastHashJoinExec => p }.size === 1) + } + + test("broadcast hint is propagated correctly") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"), (3, "2"))).toDF("key", "value") + val broadcasted = broadcast(df2) + val df3 = spark.createDataFrame(Seq((2, "2"), (3, "3"))).toDF("key", "value") + + val cases = Seq(broadcasted.limit(2), + broadcasted.filter("value < 10"), + broadcasted.sample(true, 0.5), + broadcasted.distinct(), + broadcasted.groupBy("value").agg(min($"key").as("key")), + // except and intersect are semi/anti-joins which won't return more data then + // their left argument, so the broadcast hint should be propagated here + broadcasted.except(df3), + broadcasted.intersect(df3)) + + cases.foreach(assertBroadcastJoin) + } + } } From da5d2300edec800377e6f0fc3a6f066d67638d05 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 21 May 2016 23:12:27 -0700 Subject: [PATCH 0340/1470] [SPARK-15396][SQL][DOC] It can't connect hive metastore database #### What changes were proposed in this pull request? The `hive.metastore.warehouse.dir` property in hive-site.xml is deprecated since Spark 2.0.0. Users might not be able to connect to the existing metastore if they do not use the new conf parameter `spark.sql.warehouse.dir`. This PR is to update the document and example for explaining the latest changes in the configuration of default location of database. Below is the screenshot of the latest generated docs: screenshot 2016-05-20 08 38 10 screenshot 2016-05-20 08 53 26 screenshot 2016-05-20 08 53 37 No change is made in the R's example. screenshot 2016-05-20 08 54 38 #### How was this patch tested? N/A Author: gatorsmile Closes #13225 from gatorsmile/document. (cherry picked from commit 6cb8f836da197eec17d33e4a547340c15e59d091) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 72 +++++++++++-------- .../examples/sql/hive/HiveFromSpark.scala | 11 +-- 2 files changed, 50 insertions(+), 33 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a9e1f9d5ce46f..940c1d77045ad 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1663,43 +1663,50 @@ Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (
    -When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can still create a `HiveContext`. When not configured by the -hive-site.xml, the context automatically creates `metastore_db` in the current directory and -creates `warehouse` directory indicated by HiveConf, which defaults to `/user/hive/warehouse`. -Note that you may need to grant write privilege on `/user/hive/warehouse` to the user who starts -the spark application. +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the spark application. {% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc) +// warehouse_location points to the default location for managed databases and tables +val conf = new SparkConf().setAppName("HiveFromSpark").set("spark.sql.warehouse.dir", warehouse_location) +val spark = SparkSession.builder.config(conf).enableHiveSupport().getOrCreate() -sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -sqlContext.sql("FROM src SELECT key, value").collect().foreach(println) +spark.sql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
    -When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to -the `sql` method a `HiveContext` also provides an `hql` method, which allows queries to be -expressed in HiveQL. +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the spark application. {% highlight java %} -// sc is an existing JavaSparkContext. -HiveContext sqlContext = new org.apache.spark.sql.hive.HiveContext(sc.sc); +SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate(); -sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); +spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = sqlContext.sql("FROM src SELECT key, value").collect(); +Row[] results = spark.sql("FROM src SELECT key, value").collect(); {% endhighlight %} @@ -1707,18 +1714,25 @@ Row[] results = sqlContext.sql("FROM src SELECT key, value").collect();
    -When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in the MetaStore and writing queries using HiveQL. +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the spark application. + {% highlight python %} -# sc is an existing SparkContext. -from pyspark.sql import HiveContext -sqlContext = HiveContext(sc) +from pyspark.sql import SparkSession +spark = SparkSession.builder.enableHiveSupport().getOrCreate() -sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = sqlContext.sql("FROM src SELECT key, value").collect() +results = spark.sql("FROM src SELECT key, value").collect() {% endhighlight %} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 59bdfa09ad0f8..d3bb7e4398cd3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -37,10 +37,13 @@ object HiveFromSpark { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") - // A hive context adds support for finding tables in the MetaStore and writing queries - // using HiveQL. Users who do not have an existing Hive deployment can still create a - // HiveContext. When not configured by the hive-site.xml, the context automatically - // creates metastore_db and warehouse in the current directory. + // When working with Hive, one must instantiate `SparkSession` with Hive support, including + // connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined + // functions. Users who do not have an existing Hive deployment can still enable Hive support. + // When not configured by the hive-site.xml, the context automatically creates `metastore_db` + // in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, + // which defaults to the directory `spark-warehouse` in the current directory that the spark + // application is started. val spark = SparkSession.builder .config(sparkConf) .enableHiveSupport() From 199bac8fa4dfec54292fe1ba9141edb87c8ee09f Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 21 May 2016 23:56:10 -0700 Subject: [PATCH 0341/1470] [SPARK-15312][SQL] Detect Duplicate Key in Partition Spec and Table Properties #### What changes were proposed in this pull request? When there are duplicate keys in the partition specs or table properties, we always use the last value and ignore all the previous values. This is caused by the function call `toMap`. partition specs or table properties are widely used in multiple DDL statements. This PR is to detect the duplicates and issue an exception if found. #### How was this patch tested? Added test cases in DDLSuite Author: gatorsmile Closes #13095 from gatorsmile/detectDuplicate. (cherry picked from commit a11175eecacd4a57325dab29fff9ebfad819f22f) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/AstBuilder.scala | 15 ++++++--------- .../spark/sql/catalyst/parser/ParserUtils.scala | 7 +++++++ .../sql/catalyst/parser/PlanParserSuite.scala | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 7 +++++-- .../sql/execution/command/DDLCommandSuite.scala | 15 +++++++++++++++ 5 files changed, 34 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 2d7d0f9032956..cace026701384 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -125,14 +125,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val namedQuery = visitNamedQuery(nCtx) (namedQuery.alias, namedQuery) } - // Check for duplicate names. - ctes.groupBy(_._1).filter(_._2.size > 1).foreach { - case (name, _) => - throw new ParseException( - s"Name '$name' is used for multiple common table expressions", ctx) - } - + checkDuplicateKeys(ctes, ctx) With(query, ctes.toMap) } } @@ -220,11 +214,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { */ override def visitPartitionSpec( ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) { - ctx.partitionVal.asScala.map { pVal => + val parts = ctx.partitionVal.asScala.map { pVal => val name = pVal.identifier.getText.toLowerCase val value = Option(pVal.constant).map(visitStringConstant) name -> value - }.toMap + } + // Check for duplicate partition columns in one spec. + checkDuplicateKeys(parts, ctx) + parts.toMap } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 58e2bdb6e24fb..9619884edeafe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -43,6 +43,13 @@ object ParserUtils { new ParseException(s"Operation not allowed: $message", ctx) } + /** Check if duplicate keys exist in a set of key-value pairs. */ + def checkDuplicateKeys[T](keyPairs: Seq[(String, T)], ctx: ParserRuleContext): Unit = { + keyPairs.groupBy(_._1).filter(_._2.size > 1).foreach { case (key, _) => + throw new ParseException(s"Found duplicate keys '$key'.", ctx) + } + } + /** Get the code that creates the given node. */ def source(ctx: ParserRuleContext): String = { val stream = ctx.getStart.getInputStream diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 25d87d93bec42..5811d32cd9e62 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -108,7 +108,7 @@ class PlanParserSuite extends PlanTest { "cte2" -> table("cte1").select(star()))) intercept( "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1", - "Name 'cte1' is used for multiple common table expressions") + "Found duplicate keys 'cte1'") } test("simple select query") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2e3ac9706daf8..c517b8b55fadb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -387,11 +387,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitTablePropertyList( ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { - ctx.tableProperty.asScala.map { property => + val properties = ctx.tableProperty.asScala.map { property => val key = visitTablePropertyKey(property.key) val value = Option(property.value).map(string).orNull key -> value - }.toMap + } + // Check for duplicate property names. + checkDuplicateKeys(properties, ctx) + properties.toMap } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 708b878c843a7..54f98a6232e91 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -665,6 +665,21 @@ class DDLCommandSuite extends PlanTest { assert(parsed.isInstanceOf[Project]) } + test("duplicate keys in table properties") { + val e = intercept[ParseException] { + parser.parsePlan("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2')") + }.getMessage + assert(e.contains("Found duplicate keys 'key1'")) + } + + test("duplicate columns in partition specs") { + val e = intercept[ParseException] { + parser.parsePlan( + "ALTER TABLE dbx.tab1 PARTITION (a='1', a='2') RENAME TO PARTITION (a='100', a='200')") + }.getMessage + assert(e.contains("Found duplicate keys 'a'")) + } + test("drop table") { val tableName1 = "db.tab" val tableName2 = "tab" From 9bfb16a6b042cdf8217df2e335027fa6c67366cf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 22 May 2016 00:03:37 -0700 Subject: [PATCH 0342/1470] [SPARK-15459][SQL] Make Range logical and physical explain consistent ## What changes were proposed in this pull request? This patch simplifies the implementation of Range operator and make the explain string consistent between logical plan and physical plan. To do this, I changed RangeExec to embed a Range logical plan in it. Before this patch (note that the logical Range and physical Range actually output different information): ``` == Optimized Logical Plan == Range 0, 100, 2, 2, [id#8L] == Physical Plan == *Range 0, 2, 2, 50, [id#8L] ``` After this patch: If step size is 1: ``` == Optimized Logical Plan == Range(0, 100, splits=2) == Physical Plan == *Range(0, 100, splits=2) ``` If step size is not 1: ``` == Optimized Logical Plan == Range (0, 100, step=2, splits=2) == Physical Plan == *Range (0, 100, step=2, splits=2) ``` ## How was this patch tested? N/A Author: Reynold Xin Closes #13239 from rxin/SPARK-15459. (cherry picked from commit 845e447fa03bf0a53ed79fa7e240af94dc152d2c) Signed-off-by: Reynold Xin --- .../plans/logical/basicLogicalOperators.scala | 18 ++++++++++--- .../catalog/SessionCatalogSuite.scala | 16 ++++++------ .../spark/sql/execution/SparkStrategies.scala | 4 +-- .../execution/basicPhysicalOperators.scala | 26 +++++++++---------- .../spark/sql/internal/CatalogSuite.scala | 2 +- 5 files changed, 37 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index bed48b6f6101b..b1b3e00de1456 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -431,8 +431,11 @@ case class Range( end: Long, step: Long, numSlices: Int, - output: Seq[Attribute]) extends LeafNode with MultiInstanceRelation { - require(step != 0, "step cannot be 0") + output: Seq[Attribute]) + extends LeafNode with MultiInstanceRelation { + + require(step != 0, s"step ($step) cannot be 0") + val numElements: BigInt = { val safeStart = BigInt(start) val safeEnd = BigInt(end) @@ -444,13 +447,20 @@ case class Range( } } - override def newInstance(): Range = - Range(start, end, step, numSlices, output.map(_.newInstance())) + override def newInstance(): Range = copy(output = output.map(_.newInstance())) override def statistics: Statistics = { val sizeInBytes = LongType.defaultSize * numElements Statistics( sizeInBytes = sizeInBytes ) } + + override def simpleString: String = { + if (step == 1) { + s"Range ($start, $end, splits=$numSlices)" + } else { + s"Range ($start, $end, step=$step, splits=$numSlices)" + } + } } case class Aggregate( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 91e2e077cf3ce..a4dc03cd8b260 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -197,8 +197,8 @@ class SessionCatalogSuite extends SparkFunSuite { test("create temp table") { val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable1 = Range(1, 10, 1, 10, Seq()) - val tempTable2 = Range(1, 20, 2, 10, Seq()) + val tempTable1 = Range(1, 10, 1, 10) + val tempTable2 = Range(1, 20, 2, 10) catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) catalog.createTempView("tbl2", tempTable2, overrideIfExists = false) assert(catalog.getTempTable("tbl1") == Option(tempTable1)) @@ -243,7 +243,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("drop temp table") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) @@ -304,7 +304,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("rename temp table") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") assert(sessionCatalog.getTempTable("tbl1") == Option(tempTable)) @@ -383,7 +383,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("lookup table relation") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - val tempTable1 = Range(1, 10, 1, 10, Seq()) + val tempTable1 = Range(1, 10, 1, 10) val metastoreTable1 = externalCatalog.getTable("db2", "tbl1") sessionCatalog.createTempView("tbl1", tempTable1, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") @@ -422,7 +422,7 @@ class SessionCatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(TableIdentifier("tbl1", Some("db1")))) assert(!catalog.tableExists(TableIdentifier("tbl2", Some("db1")))) // If database is explicitly specified, do not check temporary tables - val tempTable = Range(1, 10, 1, 10, Seq()) + val tempTable = Range(1, 10, 1, 10) catalog.createTempView("tbl3", tempTable, overrideIfExists = false) assert(!catalog.tableExists(TableIdentifier("tbl3", Some("db2")))) // If database is not explicitly specified, check the current database @@ -434,7 +434,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("list tables without pattern") { val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.createTempView("tbl4", tempTable, overrideIfExists = false) assert(catalog.listTables("db1").toSet == @@ -451,7 +451,7 @@ class SessionCatalogSuite extends SparkFunSuite { test("list tables with pattern") { val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable = Range(1, 10, 2, 10, Seq()) + val tempTable = Range(1, 10, 2, 10) catalog.createTempView("tbl1", tempTable, overrideIfExists = false) catalog.createTempView("tbl4", tempTable, overrideIfExists = false) assert(catalog.listTables("db1", "*").toSet == catalog.listTables("db1").toSet) 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 664e7f566145b..555a2f4c01d0e 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 @@ -360,8 +360,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { generator, join = join, outer = outer, g.output, planLater(child)) :: Nil case logical.OneRowRelation => execution.RDDScanExec(Nil, singleRowRdd, "OneRowRelation") :: Nil - case r @ logical.Range(start, end, step, numSlices, output) => - execution.RangeExec(start, step, numSlices, r.numElements, output) :: Nil + case r : logical.Range => + execution.RangeExec(r) :: Nil case logical.RepartitionByExpression(expressions, child, nPartitions) => exchange.ShuffleExchange(HashPartitioning( expressions, nPartitions.getOrElse(numPartitions)), planLater(child)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index d492fa7c412df..89bde6ad73f3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, ExpressionCanonicalizer} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.LongType +import org.apache.spark.sql.types.{LongType, StructField, StructType} import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} /** Physical plan for Project. */ @@ -305,22 +305,18 @@ case class SampleExec( /** - * Physical plan for range (generating a range of 64 bit numbers. - * - * @param start first number in the range, inclusive. - * @param step size of the step increment. - * @param numSlices number of partitions. - * @param numElements total number of elements to output. - * @param output output attributes. + * Physical plan for range (generating a range of 64 bit numbers). */ -case class RangeExec( - start: Long, - step: Long, - numSlices: Int, - numElements: BigInt, - output: Seq[Attribute]) +case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) extends LeafExecNode with CodegenSupport { + def start: Long = range.start + def step: Long = range.step + def numSlices: Int = range.numSlices + def numElements: BigInt = range.numElements + + override val output: Seq[Attribute] = range.output + private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -458,6 +454,8 @@ case class RangeExec( } } } + + override def simpleString: String = range.simpleString } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index e4d4cecd5b5dc..cd434f7887db6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -58,7 +58,7 @@ class CatalogSuite } private def createTempTable(name: String): Unit = { - sessionCatalog.createTempView(name, Range(1, 2, 3, 4, Seq()), overrideIfExists = true) + sessionCatalog.createTempView(name, Range(1, 2, 3, 4), overrideIfExists = true) } private def dropTable(name: String, db: Option[String] = None): Unit = { From 24ea16598d4d912ac6a4e7961be1b66e82c2c23f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 22 May 2016 02:08:18 -0700 Subject: [PATCH 0343/1470] [SPARK-15428][SQL] Disable multiple streaming aggregations ## What changes were proposed in this pull request? Incrementalizing plans of with multiple streaming aggregation is tricky and we dont have the necessary support for "delta" to implement correctly. So disabling the support for multiple streaming aggregations. ## How was this patch tested? Additional unit tests Author: Tathagata Das Closes #13210 from tdas/SPARK-15428. (cherry picked from commit 1ffa608ba5a849739a56047bda8b157b86b08650) Signed-off-by: Tathagata Das --- .../UnsupportedOperationChecker.scala | 18 +++++++-- .../analysis/UnsupportedOperationsSuite.scala | 39 +++++++++++++------ .../streaming/StreamingAggregationSuite.scala | 19 --------- 3 files changed, 41 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index aadc1d31bd4b2..0e08bf013c8d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -55,10 +55,20 @@ object UnsupportedOperationChecker { case _: InsertIntoTable => throwError("InsertIntoTable is not supported with streaming DataFrames/Datasets") - case Aggregate(_, _, child) if child.isStreaming && outputMode == Append => - throwError( - "Aggregations are not supported on streaming DataFrames/Datasets in " + - "Append output mode. Consider changing output mode to Update.") + case Aggregate(_, _, child) if child.isStreaming => + if (outputMode == Append) { + throwError( + "Aggregations are not supported on streaming DataFrames/Datasets in " + + "Append output mode. Consider changing output mode to Update.") + } + val moreStreamingAggregates = child.find { + case Aggregate(_, _, grandchild) if grandchild.isStreaming => true + case _ => false + } + if (moreStreamingAggregates.nonEmpty) { + throwError("Multiple streaming aggregations are not supported with " + + "streaming DataFrames/Datasets") + } case Join(left, right, joinType, _) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 50baebe8bf4de..674277bdbe15d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.IntegerType @@ -95,6 +96,26 @@ class UnsupportedOperationsSuite extends SparkFunSuite { outputMode = Append, Seq("aggregation", "append output mode")) + // Multiple streaming aggregations not supported + def aggExprs(name: String): Seq[NamedExpression] = Seq(Count("*").as(name)) + + assertSupportedInStreamingPlan( + "aggregate - multiple batch aggregations", + Aggregate(Nil, aggExprs("c"), Aggregate(Nil, aggExprs("d"), batchRelation)), + Update) + + assertSupportedInStreamingPlan( + "aggregate - multiple aggregations but only one streaming aggregation", + Aggregate(Nil, aggExprs("c"), batchRelation).join( + Aggregate(Nil, aggExprs("d"), streamRelation), joinType = Inner), + Update) + + assertNotSupportedInStreamingPlan( + "aggregate - multiple streaming aggregations", + Aggregate(Nil, aggExprs("c"), Aggregate(Nil, aggExprs("d"), streamRelation)), + outputMode = Update, + expectedMsgs = Seq("multiple streaming aggregations")) + // Inner joins: Stream-stream not supported testBinaryOperationInStreamingPlan( "inner join", @@ -354,17 +375,11 @@ class UnsupportedOperationsSuite extends SparkFunSuite { val e = intercept[AnalysisException] { testBody } - - if (!expectedMsgs.map(_.toLowerCase).forall(e.getMessage.toLowerCase.contains)) { - fail( - s"""Exception message should contain the following substrings: - | - | ${expectedMsgs.mkString("\n ")} - | - |Actual exception message: - | - | ${e.getMessage} - """.stripMargin) + expectedMsgs.foreach { m => + if (!e.getMessage.toLowerCase.contains(m.toLowerCase)) { + fail(s"Exception message should contain: '$m', " + + s"actual exception message:\n\t'${e.getMessage}'") + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 0f5fc9ca72d98..7104d01c4a2a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -84,25 +84,6 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be ) } - test("multiple aggregations") { - val inputData = MemoryStream[Int] - - val aggregated = - inputData.toDF() - .groupBy($"value") - .agg(count("*") as 'count) - .groupBy($"value" % 2) - .agg(sum($"count")) - .as[(Int, Long)] - - testStream(aggregated)( - AddData(inputData, 1, 2, 3, 4), - CheckLastBatch((0, 2), (1, 2)), - AddData(inputData, 1, 3, 5), - CheckLastBatch((1, 5)) - ) - } - testQuietly("midbatch failure") { val inputData = MemoryStream[Int] FailureSinglton.firstTime = true From 831c7c085d9714c488ebc6876e0a8404a52c0e37 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 22 May 2016 08:08:46 -0500 Subject: [PATCH 0344/1470] [SPARK-15430][SQL] Fix potential ConcurrentModificationException for ListAccumulator ## What changes were proposed in this pull request? In `ListAccumulator` we create an unmodifiable view for underlying list. However, it doesn't prevent the underlying to be modified further. So as we access the unmodifiable list, the underlying list can be modified in the same time. It could cause `java.util.ConcurrentModificationException`. We can observe such exception in recent tests. To fix it, we can copy a list of the underlying list and then create the unmodifiable view of this list instead. ## How was this patch tested? The exception might be difficult to test. Existing tests should be passed. Author: Liang-Chi Hsieh Closes #13211 from viirya/fix-concurrentmodify. (cherry picked from commit 7920296bf8f313e010205937d3ebcbbc7b1a1d9e) Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/util/AccumulatorV2.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 13cb6a28c3465..21ba46024d3b4 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.{lang => jl} import java.io.ObjectInputStream +import java.util.ArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong @@ -415,7 +416,7 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { - private val _list: java.util.List[T] = new java.util.ArrayList[T] + private val _list: java.util.List[T] = new ArrayList[T] override def isZero: Boolean = _list.isEmpty @@ -437,7 +438,9 @@ class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } - override def value: java.util.List[T] = java.util.Collections.unmodifiableList(_list) + override def value: java.util.List[T] = _list.synchronized { + java.util.Collections.unmodifiableList(new ArrayList[T](_list)) + } private[spark] def setValue(newValue: java.util.List[T]): Unit = { _list.clear() From d39eb4bc3c2e1142413e9e49bf5b9028b5e0a1fb Mon Sep 17 00:00:00 2001 From: Bo Meng Date: Sun, 22 May 2016 08:10:54 -0500 Subject: [PATCH 0345/1470] [SPARK-15468][SQL] fix some typos ## What changes were proposed in this pull request? Fix some typos while browsing the codes. ## How was this patch tested? None and obvious. Author: Bo Meng Author: bomeng Closes #13246 from bomeng/typo. (cherry picked from commit 72288fd67edc00f56e2e47eab2ef58fe4ff8c177) Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala | 2 +- .../catalyst/expressions/codegen/GenerateSafeProjection.scala | 2 +- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../org/apache/spark/sql/catalyst/planning/patterns.scala | 4 ++-- .../scala/org/apache/spark/sql/types/UDTRegistration.scala | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 21ba46024d3b4..0b9a47c1124b1 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -55,7 +55,7 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { /** * Returns true if this accumulator has been registered. Note that all accumulators must be - * registered before ues, or it will throw exception. + * registered before use, or it will throw exception. */ final def isRegistered: Boolean = metadata != null && AccumulatorContext.get(metadata.id).isDefined diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index b0b121255357a..214dc40641f85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -48,7 +48,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] val tmp = ctx.freshName("tmp") val output = ctx.freshName("safeRow") val values = ctx.freshName("values") - // These expressions could be splitted into multiple functions + // These expressions could be split into multiple functions ctx.addMutableState("Object[]", values, s"this.$values = null;") val rowClass = classOf[GenericInternalRow].getName diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index cace026701384..a13c03a529f37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -771,7 +771,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * ******************************************************************************************** */ /** * Create an expression from the given context. This method just passes the context on to the - * vistor and only takes care of typing (We assume that the visitor returns an Expression here). + * visitor and only takes care of typing (We assume that the visitor returns an Expression here). */ protected def expression(ctx: ParserRuleContext): Expression = typedVisit(ctx) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 00656191354f2..f42e67ca6ec20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -227,8 +227,8 @@ object IntegerIndex { * - Unnamed grouping expressions are named so that they can be referred to across phases of * aggregation * - Aggregations that appear multiple times are deduplicated. - * - The compution of the aggregations themselves is separated from the final result. For example, - * the `count` in `count + 1` will be split into an [[AggregateExpression]] and a final + * - The computation of the aggregations themselves is separated from the final result. For + * example, the `count` in `count + 1` will be split into an [[AggregateExpression]] and a final * computation that computes `count.resultAttribute + 1`. */ object PhysicalAggregation { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala index 0f24e51ed2b76..20ec75c70615b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UDTRegistration.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils * This object keeps the mappings between user classes and their User Defined Types (UDTs). * Previously we use the annotation `SQLUserDefinedType` to register UDTs for user classes. * However, by doing this, we add SparkSQL dependency on user classes. This object provides - * alterntive approach to register UDTs for user classes. + * alternative approach to register UDTs for user classes. */ private[spark] object UDTRegistration extends Serializable with Logging { From 92c87f74d8d78d605e4fe95552c5ae3c1cb9076f Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Sun, 22 May 2016 09:19:28 -0500 Subject: [PATCH 0346/1470] [MINOR] More than 100 chars in line in SparkSubmitCommandBuilderSuite ## What changes were proposed in this pull request? More than 100 chars in line. ## How was this patch tested? Author: Sandeep Singh Closes #13249 from techaddict/fix-1. (cherry picked from commit 3eff65f82d5fc29b21f330966ac763489d217f4d) Signed-off-by: Sean Owen --- .../apache/spark/launcher/SparkSubmitCommandBuilderSuite.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 ffe11bc51b722..d451651555768 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -67,7 +67,8 @@ public void testCliHelpAndNoArg() throws Exception { List sparkEmptyArgs = Collections.emptyList(); cmd = buildCommand(sparkEmptyArgs, env); - assertTrue("org.apache.spark.deploy.SparkSubmit should be contained in the final cmd of empty input.", + assertTrue( + "org.apache.spark.deploy.SparkSubmit should be contained in the final cmd of empty input.", cmd.contains("org.apache.spark.deploy.SparkSubmit")); } From ddac9f26293d7bf508522aede103ae2034e03606 Mon Sep 17 00:00:00 2001 From: wangyang Date: Sun, 22 May 2016 19:30:14 -0700 Subject: [PATCH 0347/1470] [SPARK-15379][SQL] check special invalid date ## What changes were proposed in this pull request? When invalid date string like "2015-02-29 00:00:00" are cast as date or timestamp using spark sql, it used to not return null but another valid date (2015-03-01 in this case). In this pr, invalid date string like "2016-02-29" and "2016-04-31" are returned as null when cast as date or timestamp. ## How was this patch tested? Unit tests are added. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: wangyang Closes #13169 from wangyang1992/invalid_date. (cherry picked from commit fc44b694bf5162b3a044768da4627b9969909829) Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/DateTimeUtils.scala | 27 ++++++++++++++++--- .../catalyst/util/DateTimeUtilsSuite.scala | 19 +++++++++++++ 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index f84c6592c6ae5..e08328a32079e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -58,6 +58,7 @@ object DateTimeUtils { final val YearZero = -17999 final val toYearZero = to2001 + 7304850 final val TimeZoneGMT = TimeZone.getTimeZone("GMT") + final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) @transient lazy val defaultTimeZone = TimeZone.getDefault @@ -333,8 +334,7 @@ object DateTimeUtils { digitsMilli += 1 } - if (!justTime && (segments(0) < 0 || segments(0) > 9999 || segments(1) < 1 || - segments(1) > 12 || segments(2) < 1 || segments(2) > 31)) { + if (!justTime && isInvalidDate(segments(0), segments(1), segments(2))) { return None } @@ -414,10 +414,10 @@ object DateTimeUtils { return None } segments(i) = currentSegmentValue - if (segments(0) < 0 || segments(0) > 9999 || segments(1) < 1 || segments(1) > 12 || - segments(2) < 1 || segments(2) > 31) { + if (isInvalidDate(segments(0), segments(1), segments(2))) { return None } + val c = threadLocalGmtCalendar.get() c.clear() c.set(segments(0), segments(1) - 1, segments(2), 0, 0, 0) @@ -425,6 +425,25 @@ object DateTimeUtils { Some((c.getTimeInMillis / MILLIS_PER_DAY).toInt) } + /** + * Return true if the date is invalid. + */ + private def isInvalidDate(year: Int, month: Int, day: Int): Boolean = { + if (year < 0 || year > 9999 || month < 1 || month > 12 || day < 1 || day > 31) { + return true + } + if (month == 2) { + if (isLeapYear(year) && day > 29) { + return true + } else if (!isLeapYear(year) && day > 28) { + return true + } + } else if (!MonthOf31Days.contains(month) && day > 30) { + return true + } + false + } + /** * Returns the microseconds since year zero (-17999) from microseconds since epoch. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 6745b4b6c3c67..28e30c2219e3f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -353,6 +353,25 @@ class DateTimeUtilsSuite extends SparkFunSuite { c.getTimeInMillis * 1000 + 123456) } + test("SPARK-15379: special invalid date string") { + // Test stringToDate + assert(stringToDate( + UTF8String.fromString("2015-02-29 00:00:00")).isEmpty) + assert(stringToDate( + UTF8String.fromString("2015-04-31 00:00:00")).isEmpty) + assert(stringToDate(UTF8String.fromString("2015-02-29")).isEmpty) + assert(stringToDate(UTF8String.fromString("2015-04-31")).isEmpty) + + + // Test stringToTimestamp + assert(stringToTimestamp( + UTF8String.fromString("2015-02-29 00:00:00")).isEmpty) + assert(stringToTimestamp( + UTF8String.fromString("2015-04-31 00:00:00")).isEmpty) + assert(stringToTimestamp(UTF8String.fromString("2015-02-29")).isEmpty) + assert(stringToTimestamp(UTF8String.fromString("2015-04-31")).isEmpty) + } + test("hours") { val c = Calendar.getInstance() c.set(2015, 2, 18, 13, 2, 11) From 4462da7071462084c5b55cc414c7faa0e1396a18 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Sun, 22 May 2016 23:32:39 -0700 Subject: [PATCH 0348/1470] [SPARK-15425][SQL] Disallow cross joins by default ## What changes were proposed in this pull request? In order to prevent users from inadvertently writing queries with cartesian joins, this patch introduces a new conf `spark.sql.crossJoin.enabled` (set to `false` by default) that if not set, results in a `SparkException` if the query contains one or more cartesian products. ## How was this patch tested? Added a test to verify the new behavior in `JoinSuite`. Additionally, `SQLQuerySuite` and `SQLMetricsSuite` were modified to explicitly enable cartesian products. Author: Sameer Agarwal Closes #13209 from sameeragarwal/disallow-cartesian. (cherry picked from commit dafcb05c2ef8e09f45edfb7eabf58116c23975a0) Signed-off-by: Reynold Xin --- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../joins/BroadcastNestedLoopJoinExec.scala | 14 +++++- .../joins/CartesianProductExec.scala | 11 +++++ .../apache/spark/sql/internal/SQLConf.scala | 9 +++- .../org/apache/spark/sql/JoinSuite.scala | 31 +++++++++---- .../org/apache/spark/sql/SQLQuerySuite.scala | 32 +++++++------ .../sql/execution/joins/InnerJoinSuite.scala | 3 +- .../execution/metric/SQLMetricsSuite.scala | 46 ++++++++++--------- .../execution/HiveCompatibilitySuite.scala | 4 ++ .../sql/hive/execution/HiveQuerySuite.scala | 6 +++ 10 files changed, 113 insertions(+), 46 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 555a2f4c01d0e..c46cecc71f37e 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 @@ -190,7 +190,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } // This join could be very slow or OOM joins.BroadcastNestedLoopJoinExec( - planLater(left), planLater(right), buildSide, joinType, condition) :: Nil + planLater(left), planLater(right), buildSide, joinType, condition, + withinBroadcastThreshold = false) :: Nil // --- Cases where this strategy does not apply --------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index 2a250ecce6de4..4d43765f8fcd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow 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.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.collection.{BitSet, CompactBuffer} case class BroadcastNestedLoopJoinExec( @@ -32,7 +34,8 @@ case class BroadcastNestedLoopJoinExec( right: SparkPlan, buildSide: BuildSide, joinType: JoinType, - condition: Option[Expression]) extends BinaryExecNode { + condition: Option[Expression], + withinBroadcastThreshold: Boolean = true) extends BinaryExecNode { override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -337,6 +340,15 @@ case class BroadcastNestedLoopJoinExec( ) } + protected override def doPrepare(): Unit = { + if (!withinBroadcastThreshold && !sqlContext.conf.crossJoinEnabled) { + throw new AnalysisException("Both sides of this join are outside the broadcasting " + + "threshold and computing it could be prohibitively expensive. To explicitly enable it, " + + s"please set ${SQLConf.CROSS_JOINS_ENABLED.key} = true") + } + super.doPrepare() + } + protected override def doExecute(): RDD[InternalRow] = { val broadcastedRelation = broadcast.executeBroadcast[Array[InternalRow]]() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 8d7ecc442a9e1..88f78a7a73bce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -19,11 +19,13 @@ package org.apache.spark.sql.execution.joins import org.apache.spark._ import org.apache.spark.rdd.{CartesianPartition, CartesianRDD, RDD} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter @@ -88,6 +90,15 @@ case class CartesianProductExec( override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + protected override def doPrepare(): Unit = { + if (!sqlContext.conf.crossJoinEnabled) { + throw new AnalysisException("Cartesian joins could be prohibitively expensive and are " + + "disabled by default. To explicitly enable them, please set " + + s"${SQLConf.CROSS_JOINS_ENABLED.key} = true") + } + super.doPrepare() + } + protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 35d67ca2d8c5d..f3064eb6ac6d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -338,9 +338,14 @@ object SQLConf { .booleanConf .createWithDefault(true) + val CROSS_JOINS_ENABLED = SQLConfigBuilder("spark.sql.crossJoin.enabled") + .doc("When false, we will throw an error if a query contains a cross join") + .booleanConf + .createWithDefault(false) + val ORDER_BY_ORDINAL = SQLConfigBuilder("spark.sql.orderByOrdinal") .doc("When true, the ordinal numbers are treated as the position in the select list. " + - "When false, the ordinal numbers in order/sort By clause are ignored.") + "When false, the ordinal numbers in order/sort by clause are ignored.") .booleanConf .createWithDefault(true) @@ -622,6 +627,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def bucketingEnabled: Boolean = getConf(SQLConf.BUCKETING_ENABLED) + def crossJoinEnabled: Boolean = getConf(SQLConf.CROSS_JOINS_ENABLED) + // Do not use a value larger than 4000 as the default value of this property. // See the comments of SCHEMA_STRING_LENGTH_THRESHOLD above for more information. def schemaStringLengthThreshold: Int = getConf(SCHEMA_STRING_LENGTH_THRESHOLD) 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 a5d8cb19eadc9..55836737083de 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 @@ -62,7 +62,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("join operator selection") { spark.cacheManager.clearCache() - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0") { + withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[SortMergeJoinExec]), @@ -204,13 +205,27 @@ class JoinSuite extends QueryTest with SharedSQLContext { testData.rdd.flatMap(row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq) } - test("cartisian product join") { - checkAnswer( - testData3.join(testData3), - Row(1, null, 1, null) :: - Row(1, null, 2, 2) :: - Row(2, 2, 1, null) :: - Row(2, 2, 2, 2) :: Nil) + test("cartesian product join") { + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "false") { + val e = intercept[Exception] { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } + assert(e.getMessage.contains("Cartesian joins could be prohibitively expensive and are " + + "disabled by default")) + } } test("left outer join") { 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 460e34a5ff308..b1f848fdc89a3 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 @@ -104,9 +104,11 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ).toDF("a", "b", "c").createOrReplaceTempView("cachedData") spark.catalog.cacheTable("cachedData") - checkAnswer( - sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"), - Row(0) :: Row(81) :: Nil) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"), + Row(0) :: Row(81) :: Nil) + } } test("self join with aliases") { @@ -435,10 +437,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("left semi greater than predicate") { - checkAnswer( - sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"), - Seq(Row(3, 1), Row(3, 2)) - ) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"), + Seq(Row(3, 1), Row(3, 2)) + ) + } } test("left semi greater than predicate and equal operator") { @@ -824,12 +828,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("cartesian product join") { - checkAnswer( - testData3.join(testData3), - Row(1, null, 1, null) :: - Row(1, null, 2, 2) :: - Row(2, 2, 1, null) :: - Row(2, 2, 2, 2) :: Nil) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } } test("left outer join") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 7caeb3be549d9..27f6abcd9509a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -187,7 +187,8 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using CartesianProduct") { - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => CartesianProductExec(left, right, Some(condition())), expectedAnswer.map(Row.fromTuple), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 7a89b484eb85b..12940c86fe650 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{AccumulatorContext, JsonProtocol, Utils} @@ -237,16 +238,18 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("BroadcastNestedLoopJoin metrics") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) testDataForJoin.createOrReplaceTempView("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = spark.sql( - "SELECT * FROM testData2 left JOIN testDataForJoin ON " + - "testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a") - testSparkPlanMetrics(df, 3, Map( - 1L -> ("BroadcastNestedLoopJoin", Map( - "number of output rows" -> 12L))) - ) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + withTempTable("testDataForJoin") { + // Assume the execution plan is + // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = spark.sql( + "SELECT * FROM testData2 left JOIN testDataForJoin ON " + + "testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a") + testSparkPlanMetrics(df, 3, Map( + 1L -> ("BroadcastNestedLoopJoin", Map( + "number of output rows" -> 12L))) + ) + } } } @@ -263,17 +266,18 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } test("CartesianProduct metrics") { - val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.createOrReplaceTempView("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = spark.sql( - "SELECT * FROM testData2 JOIN testDataForJoin") - testSparkPlanMetrics(df, 1, Map( - 0L -> ("CartesianProduct", Map( - "number of output rows" -> 12L))) - ) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) + testDataForJoin.createOrReplaceTempView("testDataForJoin") + withTempTable("testDataForJoin") { + // Assume the execution plan is + // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = spark.sql( + "SELECT * FROM testData2 JOIN testDataForJoin") + testSparkPlanMetrics(df, 1, Map( + 0L -> ("CartesianProduct", Map("number of output rows" -> 12L))) + ) + } } } 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 54fb440b3374b..a8645f7cd31d7 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -40,6 +40,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalColumnBatchSize = TestHive.conf.columnBatchSize private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalConvertMetastoreOrc = TestHive.sessionState.convertMetastoreOrc + private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -61,6 +62,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Ensures that the plans generation use metastore relation and not OrcRelation // Was done because SqlBuilder does not work with plans having logical relation TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, false) + // Ensures that cross joins are enabled so that we can test them + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) RuleExecutor.resetTime() } @@ -72,6 +75,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, originalConvertMetastoreOrc) + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) TestHive.sessionState.functionRegistry.restore() // For debugging dump some statistics about how much time was spent in various optimizer rules diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2aaaaadb6afa9..e179021491a69 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 @@ -35,6 +35,7 @@ import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.internal.SQLConf case class TestData(a: Int, b: String) @@ -48,6 +49,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ + private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled + override def beforeAll() { super.beforeAll() TestHive.setCacheTables(true) @@ -55,6 +58,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting Locale.setDefault(Locale.US) + // Ensures that cross joins are enabled so that we can test them + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) } override def afterAll() { @@ -63,6 +68,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) } finally { super.afterAll() } From 6eb8ec6f41d7e460ba7f706590d2da62c045aead Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 23 May 2016 10:48:25 -0700 Subject: [PATCH 0349/1470] [SPARK-14031][SQL] speedup CSV writer ## What changes were proposed in this pull request? Currently, we create an CSVWriter for every row, it's very expensive and memory hungry, took about 15 seconds to write out 1 mm rows (two columns). This PR will write the rows in batch mode, create a CSVWriter for every 1k rows, which could write out 1 mm rows in about 1 seconds (15X faster). ## How was this patch tested? Manually benchmark it. Author: Davies Liu Closes #13229 from davies/csv_writer. (cherry picked from commit 80091b8a6840b562cf76341926e5b828d4def7e2) Signed-off-by: Cheng Lian --- .../execution/datasources/csv/CSVParser.scala | 19 +++++++++++---- .../datasources/csv/CSVRelation.scala | 23 ++++++++++++------- 2 files changed, 29 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index ae797a1e0749f..111995da9cbad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -76,17 +76,26 @@ private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten writerSettings.setQuoteAllFields(false) writerSettings.setHeaders(headers: _*) - def writeRow(row: Seq[String], includeHeader: Boolean): String = { - val buffer = new ByteArrayOutputStream() - val outputWriter = new OutputStreamWriter(buffer, StandardCharsets.UTF_8) - val writer = new CsvWriter(outputWriter, writerSettings) + private var buffer = new ByteArrayOutputStream() + private var writer = new CsvWriter( + new OutputStreamWriter(buffer, StandardCharsets.UTF_8), + writerSettings) + def writeRow(row: Seq[String], includeHeader: Boolean): Unit = { if (includeHeader) { writer.writeHeaders() } writer.writeRow(row.toArray: _*) + } + + def flush(): String = { writer.close() - buffer.toString.stripLineEnd + val lines = buffer.toString.stripLineEnd + buffer = new ByteArrayOutputStream() + writer = new CsvWriter( + new OutputStreamWriter(buffer, StandardCharsets.UTF_8), + writerSettings) + lines } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 4f2d4387b180f..9849484dcec85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -176,8 +176,8 @@ private[sql] class CsvOutputWriter( }.getRecordWriter(context) } - private var firstRow: Boolean = params.headerFlag - + private val FLUSH_BATCH_SIZE = 1024L + private var records: Long = 0L private val csvWriter = new LineCsvWriter(params, dataSchema.fieldNames.toSeq) private def rowToString(row: Seq[Any]): Seq[String] = row.map { field => @@ -191,16 +191,23 @@ private[sql] class CsvOutputWriter( override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") override protected[sql] def writeInternal(row: InternalRow): Unit = { - // TODO: Instead of converting and writing every row, we should use the univocity buffer - val resultString = csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), firstRow) - if (firstRow) { - firstRow = false + csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), records == 0L && params.headerFlag) + records += 1 + if (records % FLUSH_BATCH_SIZE == 0) { + flush() + } + } + + private def flush(): Unit = { + val lines = csvWriter.flush() + if (lines.nonEmpty) { + text.set(lines) + recordWriter.write(NullWritable.get(), text) } - text.set(resultString) - recordWriter.write(NullWritable.get(), text) } override def close(): Unit = { + flush() recordWriter.close(context) } } From 655d88293ee554f631aabb355b8c24cc38e23332 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 23 May 2016 11:13:27 -0700 Subject: [PATCH 0350/1470] [SPARK-15471][SQL] ScalaReflection cleanup ## What changes were proposed in this pull request? 1. simplify the logic of deserializing option type. 2. simplify the logic of serializing array type, and remove silentSchemaFor 3. remove some unnecessary code. ## How was this patch tested? existing tests Author: Wenchen Fan Closes #13250 from cloud-fan/encoder. (cherry picked from commit 07c36a2f07fcf5da6fb395f830ebbfc10eb27dcc) Signed-off-by: Michael Armbrust --- .../spark/sql/catalyst/ScalaReflection.scala | 105 ++++-------------- .../expressions/objects/objects.scala | 4 +- 2 files changed, 21 insertions(+), 88 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 36989a20cb5d5..bdd40f340235b 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 @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** @@ -72,6 +72,7 @@ object ScalaReflection extends ScalaReflection { case t if t <:< definitions.ByteTpe => ByteType case t if t <:< definitions.BooleanTpe => BooleanType case t if t <:< localTypeOf[Array[Byte]] => BinaryType + case t if t <:< localTypeOf[CalendarInterval] => CalendarIntervalType case t if t <:< localTypeOf[Decimal] => DecimalType.SYSTEM_DEFAULT case _ => val className = getClassNameFromType(tpe) @@ -189,7 +190,6 @@ object ScalaReflection extends ScalaReflection { case _ => UpCast(expr, expected, walkedTypePath) } - val className = getClassNameFromType(tpe) tpe match { case t if !dataTypeFor(t).isInstanceOf[ObjectType] => getPath @@ -239,16 +239,14 @@ object ScalaReflection extends ScalaReflection { DateTimeUtils.getClass, ObjectType(classOf[java.sql.Date]), "toJavaDate", - getPath :: Nil, - propagateNull = true) + getPath :: Nil) case t if t <:< localTypeOf[java.sql.Timestamp] => StaticInvoke( DateTimeUtils.getClass, ObjectType(classOf[java.sql.Timestamp]), "toJavaTimestamp", - getPath :: Nil, - propagateNull = true) + getPath :: Nil) case t if t <:< localTypeOf[java.lang.String] => Invoke(getPath, "toString", ObjectType(classOf[String])) @@ -437,17 +435,17 @@ object ScalaReflection extends ScalaReflection { walkedTypePath: Seq[String]): Expression = ScalaReflectionLock.synchronized { def toCatalystArray(input: Expression, elementType: `Type`): Expression = { - val externalDataType = dataTypeFor(elementType) - val Schema(catalystType, nullable) = silentSchemaFor(elementType) - if (isNativeType(externalDataType)) { - NewInstance( - classOf[GenericArrayData], - input :: Nil, - dataType = ArrayType(catalystType, nullable)) - } else { - val clsName = getClassNameFromType(elementType) - val newPath = s"""- array element class: "$clsName"""" +: walkedTypePath - MapObjects(serializerFor(_, elementType, newPath), input, externalDataType) + dataTypeFor(elementType) match { + case dt: ObjectType => + val clsName = getClassNameFromType(elementType) + val newPath = s"""- array element class: "$clsName"""" +: walkedTypePath + MapObjects(serializerFor(_, elementType, newPath), input, dt) + + case dt => + NewInstance( + classOf[GenericArrayData], + input :: Nil, + dataType = ArrayType(dt, schemaFor(elementType).nullable)) } } @@ -457,63 +455,10 @@ object ScalaReflection extends ScalaReflection { tpe match { case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t - optType match { - // For primitive types we must manually unbox the value of the object. - case t if t <:< definitions.IntTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Integer]), inputObject), - "intValue", - IntegerType) - case t if t <:< definitions.LongTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Long]), inputObject), - "longValue", - LongType) - case t if t <:< definitions.DoubleTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Double]), inputObject), - "doubleValue", - DoubleType) - case t if t <:< definitions.FloatTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Float]), inputObject), - "floatValue", - FloatType) - case t if t <:< definitions.ShortTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Short]), inputObject), - "shortValue", - ShortType) - case t if t <:< definitions.ByteTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Byte]), inputObject), - "byteValue", - ByteType) - case t if t <:< definitions.BooleanTpe => - Invoke( - UnwrapOption(ObjectType(classOf[java.lang.Boolean]), inputObject), - "booleanValue", - BooleanType) - - // For non-primitives, we can just extract the object from the Option and then recurse. - case other => - val className = getClassNameFromType(optType) - val newPath = s"""- option value class: "$className"""" +: walkedTypePath - - val optionObjectType: DataType = other match { - // Special handling is required for arrays, as getClassFromType() will fail - // since Scala Arrays map to native Java constructs. E.g. "Array[Int]" will map to - // the Java type "[I". - case arr if arr <:< localTypeOf[Array[_]] => arrayClassFor(t) - case cls => ObjectType(getClassFromType(cls)) - } - val unwrapped = UnwrapOption(optionObjectType, inputObject) - - expressions.If( - IsNull(unwrapped), - expressions.Literal.create(null, silentSchemaFor(optType).dataType), - serializerFor(unwrapped, optType, newPath)) - } + val className = getClassNameFromType(optType) + val newPath = s"""- option value class: "$className"""" +: walkedTypePath + val unwrapped = UnwrapOption(dataTypeFor(optType), inputObject) + serializerFor(unwrapped, optType, newPath) // Since List[_] also belongs to localTypeOf[Product], we put this case before // "case t if definedByConstructorParams(t)" to make sure it will match to the @@ -704,18 +649,6 @@ object ScalaReflection extends ScalaReflection { /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor[T: TypeTag]: Schema = schemaFor(localTypeOf[T]) - /** - * Returns a catalyst DataType and its nullability for the given Scala Type using reflection. - * - * Unlike `schemaFor`, this method won't throw exception for un-supported type, it will return - * `NullType` silently instead. - */ - def silentSchemaFor(tpe: `Type`): Schema = try { - schemaFor(tpe) - } catch { - case _: UnsupportedOperationException => Schema(NullType, nullable = true) - } - /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized { tpe match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 5e17f8920901a..2f2323fa3a25f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -289,8 +289,8 @@ case class UnwrapOption( ${inputObject.code} final boolean ${ev.isNull} = ${inputObject.isNull} || ${inputObject.value}.isEmpty(); - $javaType ${ev.value} = - ${ev.isNull} ? ${ctx.defaultValue(javaType)} : ($javaType) ${inputObject.value}.get(); + $javaType ${ev.value} = ${ev.isNull} ? + ${ctx.defaultValue(javaType)} : (${ctx.boxedType(javaType)}) ${inputObject.value}.get(); """ ev.copy(code = code) } From c55a39c973ca617122e5f8af40316e259399b9c9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 23 May 2016 11:55:03 -0700 Subject: [PATCH 0351/1470] [SPARK-15279][SQL] Catch conflicting SerDe when creating table ## What changes were proposed in this pull request? The user may do something like: ``` CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS PARQUET CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS ... SERDE 'myserde' CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ORC CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ... SERDE 'myserde' ``` None of these should be allowed because the SerDe's conflict. As of this patch: - `ROW FORMAT DELIMITED` is only compatible with `TEXTFILE` - `ROW FORMAT SERDE` is only compatible with `TEXTFILE`, `RCFILE` and `SEQUENCEFILE` ## How was this patch tested? New tests in `DDLCommandSuite`. Author: Andrew Or Closes #13068 from andrewor14/row-format-conflict. (cherry picked from commit 2585d2b322f3b6b85a0a12ddf7dcde957453000d) Signed-off-by: Andrew Or --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 60 ++++++++++-- .../execution/command/DDLCommandSuite.scala | 94 +++++++++++++++---- .../spark/sql/hive/HiveDDLCommandSuite.scala | 4 +- 4 files changed, 129 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 848c59e3b8a53..8ea8f766295ec 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -267,8 +267,8 @@ createFileFormat ; fileFormat - : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING (SERDE serdeCls=STRING)? #tableFileFormat - | identifier #genericFileFormat + : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING #tableFileFormat + | identifier #genericFileFormat ; storageHandler diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index c517b8b55fadb..6e4af9500c3c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -796,14 +796,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * Expected format: * {{{ - * CREATE [TEMPORARY] [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name - * [(col1 data_type [COMMENT col_comment], ...)] + * CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name + * [(col1[:] data_type [COMMENT col_comment], ...)] * [COMMENT table_comment] - * [PARTITIONED BY (col3 data_type [COMMENT col_comment], ...)] - * [CLUSTERED BY (col1, ...) [SORTED BY (col1 [ASC|DESC], ...)] INTO num_buckets BUCKETS] - * [SKEWED BY (col1, col2, ...) ON ((col_value, col_value, ...), ...) [STORED AS DIRECTORIES]] + * [PARTITIONED BY (col2[:] data_type [COMMENT col_comment], ...)] * [ROW FORMAT row_format] - * [STORED AS file_format | STORED BY storage_handler_class [WITH SERDEPROPERTIES (...)]] + * [STORED AS file_format] * [LOCATION path] * [TBLPROPERTIES (property_name=property_value, ...)] * [AS select_statement]; @@ -849,6 +847,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { compressed = false, serdeProperties = Map()) } + validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) .getOrElse(EmptyStorageFormat) val rowStorage = Option(ctx.rowFormat).map(visitRowFormat).getOrElse(EmptyStorageFormat) @@ -905,6 +904,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { /** * Create a [[CatalogStorageFormat]] for creating tables. + * + * Format: STORED AS ... */ override def visitCreateFileFormat( ctx: CreateFileFormatContext): CatalogStorageFormat = withOrigin(ctx) { @@ -932,9 +933,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx: TableFileFormatContext): CatalogStorageFormat = withOrigin(ctx) { EmptyStorageFormat.copy( inputFormat = Option(string(ctx.inFmt)), - outputFormat = Option(string(ctx.outFmt)), - serde = Option(ctx.serdeCls).map(string) - ) + outputFormat = Option(string(ctx.outFmt))) } /** @@ -1018,6 +1017,49 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { EmptyStorageFormat.copy(serdeProperties = entries.toMap) } + /** + * Throw a [[ParseException]] if the user specified incompatible SerDes through ROW FORMAT + * and STORED AS. + * + * The following are allowed. Anything else is not: + * ROW FORMAT SERDE ... STORED AS [SEQUENCEFILE | RCFILE | TEXTFILE] + * ROW FORMAT DELIMITED ... STORED AS TEXTFILE + * ROW FORMAT ... STORED AS INPUTFORMAT ... OUTPUTFORMAT ... + */ + private def validateRowFormatFileFormat( + rowFormatCtx: RowFormatContext, + createFileFormatCtx: CreateFileFormatContext, + parentCtx: ParserRuleContext): Unit = { + if (rowFormatCtx == null || createFileFormatCtx == null) { + return + } + (rowFormatCtx, createFileFormatCtx.fileFormat) match { + case (_, ffTable: TableFileFormatContext) => // OK + case (rfSerde: RowFormatSerdeContext, ffGeneric: GenericFileFormatContext) => + ffGeneric.identifier.getText.toLowerCase match { + case ("sequencefile" | "textfile" | "rcfile") => // OK + case fmt => + throw operationNotAllowed( + s"ROW FORMAT SERDE is incompatible with format '$fmt', which also specifies a serde", + parentCtx) + } + case (rfDelimited: RowFormatDelimitedContext, ffGeneric: GenericFileFormatContext) => + ffGeneric.identifier.getText.toLowerCase match { + case "textfile" => // OK + case fmt => throw operationNotAllowed( + s"ROW FORMAT DELIMITED is only compatible with 'textfile', not '$fmt'", parentCtx) + } + case _ => + // should never happen + def str(ctx: ParserRuleContext): String = { + (0 until ctx.getChildCount).map { i => ctx.getChild(i).getText }.mkString(" ") + } + throw operationNotAllowed( + s"Unexpected combination of ${str(rowFormatCtx)} and ${str(createFileFormatCtx)}", + parentCtx) + } + } + /** * Create or replace a view. This creates a [[CreateViewCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 54f98a6232e91..eab1f55712f6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.command +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, FunctionResource} import org.apache.spark.sql.catalyst.catalog.FunctionResourceType @@ -25,9 +27,10 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsing} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { private val parser = new SparkSqlParser(new SQLConf) @@ -40,6 +43,15 @@ class DDLCommandSuite extends PlanTest { containsThesePhrases.foreach { p => assert(e.getMessage.toLowerCase.contains(p.toLowerCase)) } } + private def parseAs[T: ClassTag](query: String): T = { + parser.parsePlan(query) match { + case t: T => t + case other => + fail(s"Expected to parse ${classTag[T].runtimeClass} from query," + + s"got ${other.getClass.getName}: $query") + } + } + test("create database") { val sql = """ @@ -225,19 +237,69 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed4, expected4) } + test("create table - row format and table file format") { + val createTableStart = "CREATE TABLE my_tab ROW FORMAT" + val fileFormat = s"STORED AS INPUTFORMAT 'inputfmt' OUTPUTFORMAT 'outputfmt'" + val query1 = s"$createTableStart SERDE 'anything' $fileFormat" + val query2 = s"$createTableStart DELIMITED FIELDS TERMINATED BY ' ' $fileFormat" + + // No conflicting serdes here, OK + val parsed1 = parseAs[CreateTableCommand](query1) + assert(parsed1.table.storage.serde == Some("anything")) + assert(parsed1.table.storage.inputFormat == Some("inputfmt")) + assert(parsed1.table.storage.outputFormat == Some("outputfmt")) + val parsed2 = parseAs[CreateTableCommand](query2) + assert(parsed2.table.storage.serde.isEmpty) + assert(parsed2.table.storage.inputFormat == Some("inputfmt")) + assert(parsed2.table.storage.outputFormat == Some("outputfmt")) + } + + test("create table - row format serde and generic file format") { + val allSources = Seq("parquet", "orc", "avro", "sequencefile", "rcfile", "textfile") + val supportedSources = Set("sequencefile", "rcfile", "textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS $s" + if (supportedSources.contains(s)) { + val ct = parseAs[CreateTableCommand](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s, new SQLConf) + assert(hiveSerde.isDefined) + assert(ct.table.storage.serde == Some("anything")) + assert(ct.table.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.table.storage.outputFormat == hiveSerde.get.outputFormat) + } else { + assertUnsupported(query, Seq("row format serde", "incompatible", s)) + } + } + } + + test("create table - row format delimited and generic file format") { + val allSources = Seq("parquet", "orc", "avro", "sequencefile", "rcfile", "textfile") + val supportedSources = Set("textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS $s" + if (supportedSources.contains(s)) { + val ct = parseAs[CreateTableCommand](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s, new SQLConf) + assert(hiveSerde.isDefined) + assert(ct.table.storage.serde == hiveSerde.get.serde) + assert(ct.table.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.table.storage.outputFormat == hiveSerde.get.outputFormat) + } else { + assertUnsupported(query, Seq("row format delimited", "only compatible with 'textfile'", s)) + } + } + } + test("create external table - location must be specified") { assertUnsupported( sql = "CREATE EXTERNAL TABLE my_tab", containsThesePhrases = Seq("create external table", "location")) val query = "CREATE EXTERNAL TABLE my_tab LOCATION '/something/anything'" - parser.parsePlan(query) match { - case ct: CreateTableCommand => - assert(ct.table.tableType == CatalogTableType.EXTERNAL) - assert(ct.table.storage.locationUri == Some("/something/anything")) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") - } + val ct = parseAs[CreateTableCommand](query) + assert(ct.table.tableType == CatalogTableType.EXTERNAL) + assert(ct.table.storage.locationUri == Some("/something/anything")) } test("create table - property values must be set") { @@ -252,14 +314,9 @@ class DDLCommandSuite extends PlanTest { test("create table - location implies external") { val query = "CREATE TABLE my_tab LOCATION '/something/anything'" - parser.parsePlan(query) match { - case ct: CreateTableCommand => - assert(ct.table.tableType == CatalogTableType.EXTERNAL) - assert(ct.table.storage.locationUri == Some("/something/anything")) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") - } + val ct = parseAs[CreateTableCommand](query) + assert(ct.table.tableType == CatalogTableType.EXTERNAL) + assert(ct.table.storage.locationUri == Some("/something/anything")) } test("create table using - with partitioned by") { @@ -551,8 +608,7 @@ class DDLCommandSuite extends PlanTest { test("alter table: set file format (not allowed)") { assertUnsupported( - "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + - "OUTPUTFORMAT 'test' SERDE 'test'") + "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' OUTPUTFORMAT 'test'") assertUnsupported( "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + "SET FILEFORMAT PARQUET") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 30ad392969b4e..96c8fa6b70501 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -61,7 +61,7 @@ class HiveDDLCommandSuite extends PlanTest { |country STRING COMMENT 'country of origination') |COMMENT 'This is the staging page view table' |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') - |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\054' STORED AS RCFILE + |STORED AS RCFILE |LOCATION '/user/external/page_view' |TBLPROPERTIES ('p1'='v1', 'p2'='v2') |AS SELECT * FROM src""".stripMargin @@ -88,8 +88,6 @@ class HiveDDLCommandSuite extends PlanTest { assert(desc.partitionColumns == CatalogColumn("dt", "string", comment = Some("date type")) :: CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) - assert(desc.storage.serdeProperties == - Map((serdeConstants.SERIALIZATION_FORMAT, "\u002C"), (serdeConstants.FIELD_DELIM, "\u002C"))) assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) assert(desc.storage.serde == From 80bf4ce3074414d57e35e7a1234f5030973c72bf Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 23 May 2016 14:19:25 -0700 Subject: [PATCH 0352/1470] [MINOR][SQL][DOCS] Add notes of the deterministic assumption on UDF functions ## What changes were proposed in this pull request? Spark assumes that UDF functions are deterministic. This PR adds explicit notes about that. ## How was this patch tested? It's only about docs. Author: Dongjoon Hyun Closes #13087 from dongjoon-hyun/SPARK-15282. (cherry picked from commit 37c617e4f580482b59e1abbe3c0c27c7125cf605) Signed-off-by: Michael Armbrust --- python/pyspark/sql/functions.py | 3 +++ .../org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 1 + sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 3 +++ .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 3 +++ .../src/main/scala/org/apache/spark/sql/UDFRegistration.scala | 1 + .../org/apache/spark/sql/expressions/UserDefinedFunction.scala | 3 +++ .../scala/org/apache/spark/sql/internal/SessionState.scala | 1 + 7 files changed, 15 insertions(+) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index dac842c0ce8c0..716b16fdc9530 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1756,6 +1756,9 @@ def __call__(self, *cols): @since(1.3) def udf(f, returnType=StringType()): """Creates a :class:`Column` expression representing a user defined function (UDF). + Note that the user-defined functions must be deterministic. Due to optimization, + duplicate invocations may be eliminated or the function may even be invoked more times than + it is present in the query. >>> from pyspark.sql.types import IntegerType >>> slen = udf(lambda s: len(s), IntegerType()) 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 0038cf65e2993..21390644bc0b6 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.types.DataType /** * User-defined function. + * Note that the user-defined functions must be deterministic. * @param function The user defined scala function to run. * Note that if you use primitive parameters, you are not able to check if it is * null or not, and the UDF will return null for you if the primitive input is 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 14d12d30bc0b3..7013e316ead83 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 @@ -199,6 +199,9 @@ class SQLContext private[sql]( /** * A collection of methods for registering user-defined functions (UDF). + * Note that the user-defined functions must be deterministic. Due to optimization, + * duplicate invocations may be eliminated or the function may even be invoked more times than + * it is present in the query. * * The following example registers a Scala closure as UDF: * {{{ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index f697769bdcdb5..5c87c844185c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -145,6 +145,9 @@ class SparkSession private( /** * A collection of methods for registering user-defined functions (UDF). + * Note that the user-defined functions must be deterministic. Due to optimization, + * duplicate invocations may be eliminated or the function may even be invoked more times than + * it is present in the query. * * The following example registers a Scala closure as UDF: * {{{ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 3a043dcc6af22..b006236481a29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.types.DataType /** * Functions for registering user-defined functions. Use [[SQLContext.udf]] to access this. + * Note that the user-defined functions must be deterministic. * * @since 1.3.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index bd35d19aa20bb..49fdec57558e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -25,6 +25,9 @@ import org.apache.spark.sql.types.DataType /** * A user-defined function. To create one, use the `udf` functions in [[functions]]. + * Note that the user-defined functions must be deterministic. Due to optimization, + * duplicate invocations may be eliminated or the function may even be invoked more times than + * it is present in the query. * As an example: * {{{ * // Defined a UDF that returns true or false based on some numeric score. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 939b9195cae99..c9cc2ba04a413 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -100,6 +100,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { /** * Interface exposed to the user for registering user-defined functions. + * Note that the user-defined functions must be deterministic. */ lazy val udf: UDFRegistration = new UDFRegistration(functionRegistry) From 4673b88b4249b2bf574abf98ace8632b06357ea6 Mon Sep 17 00:00:00 2001 From: sureshthalamati Date: Mon, 23 May 2016 17:15:19 -0700 Subject: [PATCH 0353/1470] [SPARK-15315][SQL] Adding error check to the CSV datasource writer for unsupported complex data types. ## What changes were proposed in this pull request? Adds error handling to the CSV writer for unsupported complex data types. Currently garbage gets written to the output csv files if the data frame schema has complex data types. ## How was this patch tested? Added new unit test case. Author: sureshthalamati Closes #13105 from sureshthalamati/csv_complex_types_SPARK-15315. (cherry picked from commit 03c7b7c4b9374f0cb6a29aeaf495bd21c2563de4) Signed-off-by: Wenchen Fan --- .../datasources/csv/DefaultSource.scala | 14 +++++++++++- .../execution/datasources/csv/CSVSuite.scala | 22 +++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala index f47ed76cba765..057bde1a75f07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration /** @@ -86,6 +86,7 @@ class DefaultSource extends FileFormat with DataSourceRegister { job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { + verifySchema(dataSchema) val conf = job.getConfiguration val csvOptions = new CSVOptions(options) csvOptions.compressionCodec.foreach { codec => @@ -172,4 +173,15 @@ class DefaultSource extends FileFormat with DataSourceRegister { .mapPartitions(_.map(pair => new String(pair._2.getBytes, 0, pair._2.getLength, charset))) } } + + private def verifySchema(schema: StructType): Unit = { + schema.foreach { field => + field.dataType match { + case _: ArrayType | _: MapType | _: StructType => + throw new UnsupportedOperationException( + s"CSV data source does not support ${field.dataType.simpleString} data type.") + case _ => + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 27d6dc9197d27..bae290776f6e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -31,6 +31,8 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { + import testImplicits._ + private val carsFile = "cars.csv" private val carsMalformedFile = "cars-malformed.csv" private val carsFile8859 = "cars_iso-8859-1.csv" @@ -582,4 +584,24 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(numbers.count() == 8) } + + test("error handling for unsupported data types.") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + var msg = intercept[UnsupportedOperationException] { + Seq((1, "Tesla")).toDF("a", "b").selectExpr("struct(a, b)").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support struct data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, Map("Tesla" -> 3))).toDF("id", "cars").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support map data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, Array("Tesla", "Chevy", "Ford"))).toDF("id", "brands").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support array data type")) + } + } } From ca271c79279fc2e4d4005aaf50426578d824ac92 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 23 May 2016 17:20:29 -0700 Subject: [PATCH 0354/1470] [MINOR][SPARKR][DOC] Add a description for running unit tests in Windows ## What changes were proposed in this pull request? This PR adds the description for running unit tests in Windows. ## How was this patch tested? On a bare machine (Window 7, 32bits), this was manually built and tested. Author: hyukjinkwon Closes #13217 from HyukjinKwon/minor-r-doc. (cherry picked from commit a8e97d17b91684e68290d9f18a43622232aa94e7) Signed-off-by: Shivaram Venkataraman --- R/README.md | 8 +++++++- R/WINDOWS.md | 20 ++++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/R/README.md b/R/README.md index 810bfc14e977e..044f95312ae8e 100644 --- a/R/README.md +++ b/R/README.md @@ -1,11 +1,13 @@ # R on Spark SparkR is an R package that provides a light-weight frontend to use Spark from R. + ### Installing sparkR Libraries of sparkR need to be created in `$SPARK_HOME/R/lib`. This can be done by running the script `$SPARK_HOME/R/install-dev.sh`. By default the above script uses the system wide installation of R. However, this can be changed to any user installed location of R by setting the environment variable `R_HOME` the full path of the base directory where R is installed, before running install-dev.sh script. Example: + ``` # where /home/username/R is where R is installed and /home/username/R/bin contains the files R and RScript export R_HOME=/home/username/R @@ -17,6 +19,7 @@ export R_HOME=/home/username/R #### Build Spark Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run + ``` build/mvn -DskipTests -Psparkr package ``` @@ -38,6 +41,7 @@ To set other options like driver memory, executor memory etc. you can pass in th #### 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/username/spark") @@ -64,13 +68,15 @@ To run one of them, use `./bin/spark-submit `. For example: ./bin/spark-submit examples/src/main/r/dataframe.R -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): +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` 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/dataframe.R diff --git a/R/WINDOWS.md b/R/WINDOWS.md index 3f889c0ca3d1e..f948ed3974794 100644 --- a/R/WINDOWS.md +++ b/R/WINDOWS.md @@ -11,3 +11,23 @@ include Rtools and R in `PATH`. 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` + +## Unit tests + +To run the SparkR unit tests on Windows, the following steps are required —assuming you are in the Spark root directory and do not have Apache Hadoop installed already: + +1. Create a folder to download Hadoop related files for Windows. For example, `cd ..` and `mkdir hadoop`. + +2. Download the relevant Hadoop bin package from [steveloughran/winutils](https://github.com/steveloughran/winutils). While these are not official ASF artifacts, they are built from the ASF release git hashes by a Hadoop PMC member on a dedicated Windows VM. For further reading, consult [Windows Problems on the Hadoop wiki](https://wiki.apache.org/hadoop/WindowsProblems). + +3. Install the files into `hadoop\bin`; make sure that `winutils.exe` and `hadoop.dll` are present. + +4. Set the environment variable `HADOOP_HOME` to the full path to the newly created `hadoop` directory. + +5. Run unit tests for SparkR by running the command below. 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')" + .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defualt.name="file:///" R\pkg\tests\run-all.R + ``` + From 3def56120e0a57238c95f64e7bb84f973b22c065 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Mon, 23 May 2016 17:32:01 -0700 Subject: [PATCH 0355/1470] [SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively ## What changes were proposed in this pull request? Currently command `ADD FILE|JAR ` is supported natively in SparkSQL. However, when this command is run, the file/jar is added to the resources that can not be looked up by `LIST FILE(s)|JAR(s)` command because the `LIST` command is passed to Hive command processor in Spark-SQL or simply not supported in Spark-shell. There is no way users can find out what files/jars are added to the spark context. Refer to [Hive commands](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli) This PR is to support following commands: `LIST (FILE[s] [filepath ...] | JAR[s] [jarfile ...])` ### For example: ##### LIST FILE(s) ``` scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt") res1: org.apache.spark.sql.DataFrame = [] scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt") res2: org.apache.spark.sql.DataFrame = [] scala> spark.sql("list file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt").show(false) +----------------------------------------------+ |result | +----------------------------------------------+ |hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt| +----------------------------------------------+ scala> spark.sql("list files").show(false) +----------------------------------------------+ |result | +----------------------------------------------+ |hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt| |hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt | +----------------------------------------------+ ``` ##### LIST JAR(s) ``` scala> spark.sql("add jar /Users/xinwu/spark/core/src/test/resources/TestUDTF.jar") res9: org.apache.spark.sql.DataFrame = [result: int] scala> spark.sql("list jar TestUDTF.jar").show(false) +---------------------------------------------+ |result | +---------------------------------------------+ |spark://192.168.1.234:50131/jars/TestUDTF.jar| +---------------------------------------------+ scala> spark.sql("list jars").show(false) +---------------------------------------------+ |result | +---------------------------------------------+ |spark://192.168.1.234:50131/jars/TestUDTF.jar| +---------------------------------------------+ ``` ## How was this patch tested? New test cases are added for Spark-SQL, Spark-Shell and SparkContext API code path. Author: Xin Wu Author: xin Wu Closes #13212 from xwu0226/list_command. (cherry picked from commit 01659bc50cd3d53815d205d005c3678e714c08e0) Signed-off-by: Cheng Lian --- .../scala/org/apache/spark/SparkContext.scala | 10 +++ core/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../org/apache/spark/SparkContextSuite.scala | 14 ++++- .../spark/sql/catalyst/parser/SqlBase.g4 | 5 +- .../spark/sql/execution/SparkSqlParser.scala | 39 ++++++++++-- .../sql/execution/command/resources.scala | 57 +++++++++++++++++- .../hive/thriftserver/SparkSQLCLIDriver.scala | 10 ++- .../src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../sql/hive/thriftserver/CliSuite.scala | 19 ++++++ .../sql/hive/execution/HiveQuerySuite.scala | 12 ++++ 10 files changed, 149 insertions(+), 17 deletions(-) create mode 100644 core/src/test/resources/TestUDTF.jar create mode 100644 sql/hive-thriftserver/src/test/resources/TestUDTF.jar diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6cdd0d298f37..351024bea47e6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1386,6 +1386,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli addFile(path, false) } + /** + * Returns a list of file paths that are added to resources. + */ + def listFiles(): Seq[String] = addedFiles.keySet.toSeq + /** * Add a file to be downloaded with this Spark job on every node. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported @@ -1724,6 +1729,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli postEnvironmentUpdate() } + /** + * Returns a list of jar files that are added to resources. + */ + def listJars(): Seq[String] = addedJars.keySet.toSeq + // Shut down the SparkContext. def stop() { if (LiveListenerBus.withinListenerThread.value) { diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d | JAR[S] ) + * LIST (FILE[S] [filepath ...] | JAR[S] [jarpath ...]) + * }}} */ - override def visitAddResource(ctx: AddResourceContext): LogicalPlan = withOrigin(ctx) { - ctx.identifier.getText.toLowerCase match { - case "file" => AddFileCommand(remainder(ctx.identifier).trim) - case "jar" => AddJarCommand(remainder(ctx.identifier).trim) - case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) + override def visitManageResource(ctx: ManageResourceContext): LogicalPlan = withOrigin(ctx) { + val mayebePaths = remainder(ctx.identifier).trim + ctx.op.getType match { + case SqlBaseParser.ADD => + ctx.identifier.getText.toLowerCase match { + case "file" => AddFileCommand(mayebePaths) + case "jar" => AddJarCommand(mayebePaths) + case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) + } + case SqlBaseParser.LIST => + ctx.identifier.getText.toLowerCase match { + case "files" | "file" => + if (mayebePaths.length > 0) { + ListFilesCommand(mayebePaths.split("\\s+")) + } else { + ListFilesCommand() + } + case "jars" | "jar" => + if (mayebePaths.length > 0) { + ListJarsCommand(mayebePaths.split("\\s+")) + } else { + ListJarsCommand() + } + case other => throw operationNotAllowed(s"LIST with resource type '$other'", ctx) + } + case _ => throw operationNotAllowed(s"Other types of operation on resources", ctx) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala index 162d493c1f8ce..20b08946675d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala @@ -17,9 +17,14 @@ package org.apache.spark.sql.execution.command +import java.io.File +import java.net.URI + +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} /** * Adds a jar to the current session so it can be used (for UDFs or serdes). @@ -46,3 +51,51 @@ case class AddFileCommand(path: String) extends RunnableCommand { Seq.empty[Row] } } + +/** + * Returns a list of file paths that are added to resources. + * If file paths are provided, return the ones that are added to resources. + */ +case class ListFilesCommand(files: Seq[String] = Seq.empty[String]) extends RunnableCommand { + override val output: Seq[Attribute] = { + AttributeReference("Results", StringType, nullable = false)() :: Nil + } + override def run(sparkSession: SparkSession): Seq[Row] = { + val fileList = sparkSession.sparkContext.listFiles() + if (files.size > 0) { + files.map { f => + val uri = new URI(f) + val schemeCorrectedPath = uri.getScheme match { + case null | "local" => new File(f).getCanonicalFile.toURI.toString + case _ => f + } + new Path(schemeCorrectedPath).toUri.toString + }.collect { + case f if fileList.contains(f) => f + }.map(Row(_)) + } else { + fileList.map(Row(_)) + } + } +} + +/** + * Returns a list of jar files that are added to resources. + * If jar files are provided, return the ones that are added to resources. + */ +case class ListJarsCommand(jars: Seq[String] = Seq.empty[String]) extends RunnableCommand { + override val output: Seq[Attribute] = { + AttributeReference("Results", StringType, nullable = false)() :: Nil + } + override def run(sparkSession: SparkSession): Seq[Row] = { + val jarList = sparkSession.sparkContext.listJars() + if (jars.nonEmpty) { + for { + jarName <- jars.map(f => new Path(f).getName) + jarPath <- jarList if jarPath.contains(jarName) + } yield Row(jarPath) + } else { + jarList.map(Row(_)) + } + } +} 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 33ff8aee79969..7389e18aefb1b 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 @@ -32,8 +32,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.{AddResourceProcessor, CommandProcessor} -import org.apache.hadoop.hive.ql.processors.{CommandProcessorFactory, ResetProcessor, SetProcessor} +import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket @@ -295,9 +294,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { System.exit(0) } if (tokens(0).toLowerCase(Locale.ENGLISH).equals("source") || - cmd_trimmed.startsWith("!") || - tokens(0).toLowerCase.equals("list") || - isRemoteMode) { + cmd_trimmed.startsWith("!") || isRemoteMode) { val start = System.currentTimeMillis() super.processCmd(cmd) val end = System.currentTimeMillis() @@ -312,7 +309,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (proc != null) { // scalastyle:off println if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || - proc.isInstanceOf[AddResourceProcessor] || proc.isInstanceOf[ResetProcessor]) { + proc.isInstanceOf[AddResourceProcessor] || proc.isInstanceOf[ListResourceProcessor] || + proc.isInstanceOf[ResetProcessor] ) { val driver = new SparkSQLDriver driver.init() diff --git a/sql/hive-thriftserver/src/test/resources/TestUDTF.jar b/sql/hive-thriftserver/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d "This is a test for Spark-11624") } + + test("list jars") { + val jarFile = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + runCliWithin(2.minute)( + s"ADD JAR $jarFile" -> "", + s"LIST JARS" -> "TestUDTF.jar", + s"List JAR $jarFile" -> "TestUDTF.jar" + ) + } + + test("list files") { + val dataFilePath = Thread.currentThread().getContextClassLoader + .getResource("data/files/small_kv.txt") + runCliWithin(2.minute)( + s"ADD FILE $dataFilePath" -> "", + s"LIST FILES" -> "small_kv.txt", + s"LIST FILE $dataFilePath" -> "small_kv.txt" + ) + } } 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 e179021491a69..e0f6ccf04dd33 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 @@ -876,6 +876,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") sql("select * from src join t1 on src.key = t1.a") sql("DROP TABLE t1") + assert(sql("list jars"). + filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + assert(sql("list jar"). + filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + val testJar2 = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath + sql(s"ADD JAR $testJar2") + assert(sql(s"list jar $testJar").count() == 1) } test("CREATE TEMPORARY FUNCTION") { @@ -899,6 +906,11 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } assert(checkAddFileRDD.first()) + assert(sql("list files"). + filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file"). + filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) } createQueryTest("dynamic_partition", From 220b9a08ef483e9dfd42c69d2822662d013d19c0 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 23 May 2016 18:03:45 -0700 Subject: [PATCH 0356/1470] [SPARK-15311][SQL] Disallow DML on Regular Tables when Using In-Memory Catalog #### What changes were proposed in this pull request? So far, when using In-Memory Catalog, we allow DDL operations for the tables. However, the corresponding DML operations are not supported for the tables that are neither temporary nor data source tables. For example, ```SQL CREATE TABLE tabName(i INT, j STRING) SELECT * FROM tabName INSERT OVERWRITE TABLE tabName SELECT 1, 'a' ``` In the above example, before this PR fix, we will get very confusing exception messages for either `SELECT` or `INSERT` ``` org.apache.spark.sql.AnalysisException: unresolved operator 'SimpleCatalogRelation default, CatalogTable(`default`.`tbl`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(i,int,true,None), CatalogColumn(j,string,true,None)),List(),List(),List(),-1,,1463928681802,-1,Map(),None,None,None,List()), None; ``` This PR is to issue appropriate exceptions in this case. The message will be like ``` org.apache.spark.sql.AnalysisException: Please enable Hive support when operating non-temporary tables: `tbl`; ``` #### How was this patch tested? Added a test case in `DDLSuite`. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13093 from gatorsmile/selectAfterCreate. (cherry picked from commit 5afd927a47aa7ede3039234f2f7262e2247aa2ae) Signed-off-by: Andrew Or --- .../sql/catalyst/analysis/CheckAnalysis.scala | 15 ++++++ .../plans/logical/basicLogicalOperators.scala | 3 +- .../sql/execution/command/DDLSuite.scala | 48 +++++++++++++++++++ 3 files changed, 65 insertions(+), 1 deletion(-) 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 28aa249888b30..cd242d78a4123 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.catalog.SimpleCatalogRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.UsingJoin @@ -305,6 +306,20 @@ trait CheckAnalysis extends PredicateHelper { |Conflicting attributes: ${conflictingAttributes.mkString(",")} """.stripMargin) + case s: SimpleCatalogRelation => + failAnalysis( + s""" + |Please enable Hive support when selecting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + + case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => + failAnalysis( + s""" + |Please enable Hive support when inserting the regular tables: + |${s.catalogTable.identifier} + """.stripMargin) + case o if !o.resolved => failAnalysis( s"unresolved operator ${operator.simpleString}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b1b3e00de1456..ca0096eeb2087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -377,7 +377,8 @@ case class InsertIntoTable( } assert(overwrite || !ifNotExists) - override lazy val resolved: Boolean = childrenResolved && expectedColumns.forall { expected => + override lazy val resolved: Boolean = + childrenResolved && table.resolved && expectedColumns.forall { expected => child.output.size == expected.size && child.output.zip(expected).forall { case (childAttr, tableAttr) => DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index d72dc092e277c..64f5a4ac479a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach +import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, NoSuchPartitionException, NoSuchTableException} @@ -1044,6 +1045,53 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { ) } + test("select/insert into the managed table") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + val tabName = "tbl" + withTable(tabName) { + sql(s"CREATE TABLE $tabName (i INT, j STRING)") + val catalogTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + assert(catalogTable.tableType == CatalogTableType.MANAGED) + + var message = intercept[AnalysisException] { + sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") + }.getMessage + assert(message.contains("Please enable Hive support when inserting the regular tables")) + message = intercept[AnalysisException] { + sql(s"SELECT * FROM $tabName") + }.getMessage + assert(message.contains("Please enable Hive support when selecting the regular tables")) + } + } + + test("select/insert into external table") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + withTempDir { tempDir => + val tabName = "tbl" + withTable(tabName) { + sql( + s""" + |CREATE EXTERNAL TABLE $tabName (i INT, j STRING) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |LOCATION '$tempDir' + """.stripMargin) + val catalogTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + assert(catalogTable.tableType == CatalogTableType.EXTERNAL) + + var message = intercept[AnalysisException] { + sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") + }.getMessage + assert(message.contains("Please enable Hive support when inserting the regular tables")) + message = intercept[AnalysisException] { + sql(s"SELECT * FROM $tabName") + }.getMessage + assert(message.contains("Please enable Hive support when selecting the regular tables")) + } + } + } + test("drop default database") { Seq("true", "false").foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { From f3162b96da4f61524c11150904916734c0e949ab Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Mon, 23 May 2016 18:14:48 -0700 Subject: [PATCH 0357/1470] [SPARK-15464][ML][MLLIB][SQL][TESTS] Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code ## What changes were proposed in this pull request? Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code. ## How was this patch tested? Existing test. Author: WeichenXu Closes #13242 from WeichenXu123/python_doctest_update_sparksession. (cherry picked from commit a15ca5533db91fefaf3248255a59c4d94eeda1a9) Signed-off-by: Andrew Or --- python/pyspark/ml/classification.py | 38 ++--- python/pyspark/ml/clustering.py | 22 +-- python/pyspark/ml/evaluation.py | 20 +-- python/pyspark/ml/feature.py | 66 ++++----- python/pyspark/ml/recommendation.py | 18 +-- python/pyspark/ml/regression.py | 46 ++++--- python/pyspark/ml/tuning.py | 18 +-- python/pyspark/mllib/classification.py | 10 +- python/pyspark/mllib/evaluation.py | 10 +- python/pyspark/mllib/feature.py | 10 +- python/pyspark/mllib/fpm.py | 9 +- python/pyspark/mllib/linalg/distributed.py | 12 +- python/pyspark/mllib/random.py | 10 +- python/pyspark/mllib/regression.py | 10 +- python/pyspark/mllib/stat/_statistics.py | 10 +- python/pyspark/mllib/tree.py | 9 +- python/pyspark/mllib/util.py | 10 +- python/pyspark/sql/catalog.py | 14 +- python/pyspark/sql/column.py | 12 +- python/pyspark/sql/conf.py | 11 +- python/pyspark/sql/functions.py | 153 +++++++++++---------- python/pyspark/sql/group.py | 12 +- 22 files changed, 298 insertions(+), 232 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index a1c3f7298409c..ea660d7808fa4 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -498,7 +498,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -512,7 +512,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred 1 >>> model.featureImportances SparseVector(1, {0: 1.0}) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -520,7 +520,7 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred DenseVector([1.0, 0.0]) >>> result.rawPrediction DenseVector([1.0, 0.0]) - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 @@ -627,7 +627,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -639,7 +639,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0, 1.0]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -647,7 +647,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred 0 >>> numpy.argmax(result.rawPrediction) 0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> rfc_path = temp_path + "/rfc" @@ -754,7 +754,7 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.feature import StringIndexer - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed") @@ -766,10 +766,10 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> gbtc_path = temp_path + "gbtc" @@ -885,7 +885,7 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H >>> from pyspark.sql import Row >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... Row(label=0.0, features=Vectors.dense([0.0, 0.0])), ... Row(label=0.0, features=Vectors.dense([0.0, 1.0])), ... Row(label=1.0, features=Vectors.dense([1.0, 0.0]))]) @@ -1029,7 +1029,7 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, Number of outputs has to be equal to the total number of labels. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (0.0, Vectors.dense([0.0, 0.0])), ... (1.0, Vectors.dense([0.0, 1.0])), ... (1.0, Vectors.dense([1.0, 0.0])), @@ -1040,7 +1040,7 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, [2, 5, 2] >>> model.weights.size 27 - >>> testDF = sqlContext.createDataFrame([ + >>> testDF = spark.createDataFrame([ ... (Vectors.dense([1.0, 0.0]),), ... (Vectors.dense([0.0, 0.0]),)], ["features"]) >>> model.transform(testDF).show() @@ -1467,21 +1467,23 @@ def _to_java(self): if __name__ == "__main__": import doctest import pyspark.ml.classification - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.classification.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.classification tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.classification tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index ac7183d2efff5..a457904e7880a 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -73,7 +73,7 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte ... (Vectors.dense([0.75, 0.935]),), ... (Vectors.dense([-0.83, -0.68]),), ... (Vectors.dense([-0.91, -0.76]),)] - >>> df = sqlContext.createDataFrame(data, ["features"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> gm = GaussianMixture(k=3, tol=0.0001, ... maxIter=10, seed=10) >>> model = gm.fit(df) @@ -197,7 +197,7 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] - >>> df = sqlContext.createDataFrame(data, ["features"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> kmeans = KMeans(k=2, seed=1) >>> model = kmeans.fit(df) >>> centers = model.clusterCenters() @@ -350,7 +350,7 @@ class BisectingKMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] - >>> df = sqlContext.createDataFrame(data, ["features"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> bkm = BisectingKMeans(k=2, minDivisibleClusterSize=1.0) >>> model = bkm.fit(df) >>> centers = model.clusterCenters() @@ -627,7 +627,7 @@ class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInter >>> from pyspark.ml.linalg import Vectors, SparseVector >>> from pyspark.ml.clustering import LDA - >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], + >>> df = spark.createDataFrame([[1, Vectors.dense([0.0, 1.0])], ... [2, SparseVector(2, {0: 1.0})],], ["id", "features"]) >>> lda = LDA(k=2, seed=1, optimizer="em") >>> model = lda.fit(df) @@ -933,21 +933,23 @@ def getKeepLastCheckpoint(self): if __name__ == "__main__": import doctest import pyspark.ml.clustering - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.clustering.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.clustering tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.clustering tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 16029dc34863a..b8b2b37af553d 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -114,7 +114,7 @@ class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPrediction >>> from pyspark.ml.linalg import Vectors >>> scoreAndLabels = map(lambda x: (Vectors.dense([1.0 - x[0], x[0]]), x[1]), ... [(0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)]) - >>> dataset = sqlContext.createDataFrame(scoreAndLabels, ["raw", "label"]) + >>> dataset = spark.createDataFrame(scoreAndLabels, ["raw", "label"]) ... >>> evaluator = BinaryClassificationEvaluator(rawPredictionCol="raw") >>> evaluator.evaluate(dataset) @@ -181,7 +181,7 @@ class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): >>> scoreAndLabels = [(-28.98343821, -27.0), (20.21491975, 21.5), ... (-25.98418959, -22.0), (30.69731842, 33.0), (74.69283752, 71.0)] - >>> dataset = sqlContext.createDataFrame(scoreAndLabels, ["raw", "label"]) + >>> dataset = spark.createDataFrame(scoreAndLabels, ["raw", "label"]) ... >>> evaluator = RegressionEvaluator(predictionCol="raw") >>> evaluator.evaluate(dataset) @@ -253,7 +253,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio >>> scoreAndLabels = [(0.0, 0.0), (0.0, 1.0), (0.0, 0.0), ... (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)] - >>> dataset = sqlContext.createDataFrame(scoreAndLabels, ["prediction", "label"]) + >>> dataset = spark.createDataFrame(scoreAndLabels, ["prediction", "label"]) ... >>> evaluator = MulticlassClassificationEvaluator(predictionCol="prediction") >>> evaluator.evaluate(dataset) @@ -313,17 +313,19 @@ def setParams(self, predictionCol="prediction", labelCol="label", if __name__ == "__main__": import doctest - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.evaluation tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.evaluation tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 497f2ad68ee51..93745c70c4fe6 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -66,7 +66,7 @@ class Binarizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java Binarize a column of continuous features given a threshold. - >>> df = sqlContext.createDataFrame([(0.5,)], ["values"]) + >>> df = spark.createDataFrame([(0.5,)], ["values"]) >>> binarizer = Binarizer(threshold=1.0, inputCol="values", outputCol="features") >>> binarizer.transform(df).head().features 0.0 @@ -131,7 +131,7 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav Maps a column of continuous features to a column of feature buckets. - >>> df = sqlContext.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) + >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], ... inputCol="values", outputCol="buckets") >>> bucketed = bucketizer.transform(df).collect() @@ -206,7 +206,7 @@ class CountVectorizer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Extracts a vocabulary from document collections and generates a :py:attr:`CountVectorizerModel`. - >>> df = sqlContext.createDataFrame( + >>> df = spark.createDataFrame( ... [(0, ["a", "b", "c"]), (1, ["a", "b", "b", "c", "a"])], ... ["label", "raw"]) >>> cv = CountVectorizer(inputCol="raw", outputCol="vectors") @@ -381,7 +381,7 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit `_. >>> from pyspark.ml.linalg import Vectors - >>> df1 = sqlContext.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) + >>> df1 = spark.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) >>> dct = DCT(inverse=False, inputCol="vec", outputCol="resultVec") >>> df2 = dct.transform(df1) >>> df2.head().resultVec @@ -448,7 +448,7 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada by a scalar multiplier. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([2.0, 1.0, 3.0]),)], ["values"]) + >>> df = spark.createDataFrame([(Vectors.dense([2.0, 1.0, 3.0]),)], ["values"]) >>> ep = ElementwiseProduct(scalingVec=Vectors.dense([1.0, 2.0, 3.0]), ... inputCol="values", outputCol="eprod") >>> ep.transform(df).head().eprod @@ -516,7 +516,7 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, Java it is advisable to use a power of two as the numFeatures parameter; otherwise the features will not be mapped evenly to the columns. - >>> df = sqlContext.createDataFrame([(["a", "b", "c"],)], ["words"]) + >>> df = spark.createDataFrame([(["a", "b", "c"],)], ["words"]) >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") >>> hashingTF.transform(df).head().features SparseVector(10, {0: 1.0, 1: 1.0, 2: 1.0}) @@ -583,7 +583,7 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab Compute the Inverse Document Frequency (IDF) given a collection of documents. >>> from pyspark.ml.linalg import DenseVector - >>> df = sqlContext.createDataFrame([(DenseVector([1.0, 2.0]),), + >>> df = spark.createDataFrame([(DenseVector([1.0, 2.0]),), ... (DenseVector([0.0, 1.0]),), (DenseVector([3.0, 0.2]),)], ["tf"]) >>> idf = IDF(minDocFreq=3, inputCol="tf", outputCol="idf") >>> model = idf.fit(df) @@ -671,7 +671,7 @@ class MaxAbsScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav any sparsity. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> df = spark.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> maScaler = MaxAbsScaler(inputCol="a", outputCol="scaled") >>> model = maScaler.fit(df) >>> model.transform(df).show() @@ -758,7 +758,7 @@ class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Jav transformer will be DenseVector even for sparse input. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> df = spark.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> mmScaler = MinMaxScaler(inputCol="a", outputCol="scaled") >>> model = mmScaler.fit(df) >>> model.originalMin @@ -889,7 +889,7 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr When the input array length is less than n (number of elements per n-gram), no n-grams are returned. - >>> df = sqlContext.createDataFrame([Row(inputTokens=["a", "b", "c", "d", "e"])]) + >>> df = spark.createDataFrame([Row(inputTokens=["a", "b", "c", "d", "e"])]) >>> ngram = NGram(n=2, inputCol="inputTokens", outputCol="nGrams") >>> ngram.transform(df).head() Row(inputTokens=[u'a', u'b', u'c', u'd', u'e'], nGrams=[u'a b', u'b c', u'c d', u'd e']) @@ -963,7 +963,7 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav >>> from pyspark.ml.linalg import Vectors >>> svec = Vectors.sparse(4, {1: 4.0, 3: 3.0}) - >>> df = sqlContext.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) + >>> df = spark.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) >>> normalizer = Normalizer(p=2.0, inputCol="dense", outputCol="features") >>> normalizer.transform(df).head().features DenseVector([0.6, -0.8]) @@ -1115,7 +1115,7 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) + >>> df = spark.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) >>> px = PolynomialExpansion(degree=2, inputCol="dense", outputCol="expanded") >>> px.transform(df).head().expanded DenseVector([0.5, 0.25, 2.0, 1.0, 4.0]) @@ -1182,7 +1182,7 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav covering all real values. This attempts to find numBuckets partitions based on a sample of data, but it may find fewer depending on the data sample values. - >>> df = sqlContext.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) + >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) >>> qds = QuantileDiscretizer(numBuckets=2, ... inputCol="values", outputCol="buckets", seed=123) >>> qds.getSeed() @@ -1272,7 +1272,7 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, length. It returns an array of strings that can be empty. - >>> df = sqlContext.createDataFrame([("A B c",)], ["text"]) + >>> df = spark.createDataFrame([("A B c",)], ["text"]) >>> reTokenizer = RegexTokenizer(inputCol="text", outputCol="words") >>> reTokenizer.transform(df).head() Row(text=u'A B c', words=[u'a', u'b', u'c']) @@ -1400,7 +1400,7 @@ class SQLTransformer(JavaTransformer, JavaMLReadable, JavaMLWritable): Currently we only support SQL syntax like 'SELECT ... FROM __THIS__' where '__THIS__' represents the underlying table of the input dataset. - >>> df = sqlContext.createDataFrame([(0, 1.0, 3.0), (2, 2.0, 5.0)], ["id", "v1", "v2"]) + >>> df = spark.createDataFrame([(0, 1.0, 3.0), (2, 2.0, 5.0)], ["id", "v1", "v2"]) >>> sqlTrans = SQLTransformer( ... statement="SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__") >>> sqlTrans.transform(df).head() @@ -1461,7 +1461,7 @@ class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, J statistics on the samples in the training set. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) + >>> df = spark.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") >>> model = standardScaler.fit(df) >>> model.mean @@ -1718,7 +1718,7 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadabl A feature transformer that filters out stop words from input. Note: null values from input array are preserved unless adding null to stopWords explicitly. - >>> df = sqlContext.createDataFrame([(["a", "b", "c"],)], ["text"]) + >>> df = spark.createDataFrame([(["a", "b", "c"],)], ["text"]) >>> remover = StopWordsRemover(inputCol="text", outputCol="words", stopWords=["b"]) >>> remover.transform(df).head().words == ['a', 'c'] True @@ -1810,7 +1810,7 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java A tokenizer that converts the input string to lowercase and then splits it by white spaces. - >>> df = sqlContext.createDataFrame([("a b c",)], ["text"]) + >>> df = spark.createDataFrame([("a b c",)], ["text"]) >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) @@ -1864,7 +1864,7 @@ class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol, JavaMLReadabl A feature transformer that merges multiple columns into a vector column. - >>> df = sqlContext.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) + >>> df = spark.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) >>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features") >>> vecAssembler.transform(df).head().features DenseVector([1.0, 0.0, 3.0]) @@ -1944,7 +1944,7 @@ class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, Ja - Add option for allowing unknown categories. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([(Vectors.dense([-1.0, 0.0]),), + >>> df = spark.createDataFrame([(Vectors.dense([-1.0, 0.0]),), ... (Vectors.dense([0.0, 1.0]),), (Vectors.dense([0.0, 2.0]),)], ["a"]) >>> indexer = VectorIndexer(maxCategories=2, inputCol="a", outputCol="indexed") >>> model = indexer.fit(df) @@ -2074,7 +2074,7 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J followed by the selected names (in the order given). >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (Vectors.dense([-2.0, 2.3, 0.0, 0.0, 1.0]),), ... (Vectors.dense([0.0, 0.0, 0.0, 0.0, 0.0]),), ... (Vectors.dense([0.6, -1.1, -3.0, 4.5, 3.3]),)], ["features"]) @@ -2161,7 +2161,7 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, Has natural language processing or machine learning process. >>> sent = ("a b " * 100 + "a c " * 10).split(" ") - >>> doc = sqlContext.createDataFrame([(sent,), (sent,)], ["sentence"]) + >>> doc = spark.createDataFrame([(sent,), (sent,)], ["sentence"]) >>> word2Vec = Word2Vec(vectorSize=5, seed=42, inputCol="sentence", outputCol="model") >>> model = word2Vec.fit(doc) >>> model.getVectors().show() @@ -2345,7 +2345,7 @@ class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab >>> data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), ... (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), ... (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] - >>> df = sqlContext.createDataFrame(data,["features"]) + >>> df = spark.createDataFrame(data,["features"]) >>> pca = PCA(k=2, inputCol="features", outputCol="pca_features") >>> model = pca.fit(df) >>> model.transform(df).collect()[0].pca_features @@ -2447,7 +2447,7 @@ class RFormula(JavaEstimator, HasFeaturesCol, HasLabelCol, JavaMLReadable, JavaM operators, including '~', '.', ':', '+', and '-'. Also see the `R formula docs `_. - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, 1.0, "a"), ... (0.0, 2.0, "b"), ... (0.0, 0.0, "a") @@ -2561,7 +2561,7 @@ class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, Ja categorical label. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame( + >>> df = spark.createDataFrame( ... [(Vectors.dense([0.0, 0.0, 18.0, 1.0]), 1.0), ... (Vectors.dense([0.0, 1.0, 12.0, 0.0]), 0.0), ... (Vectors.dense([1.0, 0.0, 15.0, 0.1]), 0.0)], @@ -2656,8 +2656,7 @@ def selectedFeatures(self): import tempfile import pyspark.ml.feature - from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext + from pyspark.sql import Row, SparkSession globs = globals().copy() features = pyspark.ml.feature.__dict__.copy() @@ -2665,19 +2664,22 @@ def selectedFeatures(self): # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.feature tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.feature tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark testData = sc.parallelize([Row(id=0, label="a"), Row(id=1, label="b"), Row(id=2, label="c"), Row(id=3, label="a"), Row(id=4, label="a"), Row(id=5, label="c")], 2) - globs['stringIndDf'] = sqlContext.createDataFrame(testData) + globs['stringIndDf'] = spark.createDataFrame(testData) temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 86c00d91652d1..bac2a3029fd71 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -65,7 +65,7 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha indicated user preferences rather than explicit ratings given to items. - >>> df = sqlContext.createDataFrame( + >>> df = spark.createDataFrame( ... [(0, 0, 4.0), (0, 1, 2.0), (1, 1, 3.0), (1, 2, 4.0), (2, 1, 1.0), (2, 2, 5.0)], ... ["user", "item", "rating"]) >>> als = ALS(rank=10, maxIter=5) @@ -74,7 +74,7 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha 10 >>> model.userFactors.orderBy("id").collect() [Row(id=0, features=[...]), Row(id=1, ...), Row(id=2, ...)] - >>> test = sqlContext.createDataFrame([(0, 2), (1, 0), (2, 0)], ["user", "item"]) + >>> test = spark.createDataFrame([(0, 2), (1, 0), (2, 0)], ["user", "item"]) >>> predictions = sorted(model.transform(test).collect(), key=lambda r: r[0]) >>> predictions[0] Row(user=0, item=2, prediction=-0.13807615637779236) @@ -370,21 +370,23 @@ def itemFactors(self): if __name__ == "__main__": import doctest import pyspark.ml.recommendation - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.recommendation.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.recommendation tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.recommendation tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index e21dd83923adb..8f58594a66360 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -55,19 +55,19 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction - L2 + L1 (elastic net) >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, 2.0, Vectors.dense(1.0)), ... (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) >>> lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight") >>> model = lr.fit(df) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> abs(model.transform(test0).head().prediction - (-1.0)) < 0.001 True >>> abs(model.coefficients[0] - 1.0) < 0.001 True >>> abs(model.intercept - 0.0) < 0.001 True - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> abs(model.transform(test1).head().prediction - 1.0) < 0.001 True >>> lr.setParams("vector") @@ -413,12 +413,12 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti Only univariate (single feature) algorithm supported. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> ir = IsotonicRegression() >>> model = ir.fit(df) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 >>> model.boundaries @@ -643,7 +643,7 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi It supports both continuous and categorical features. >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> dt = DecisionTreeRegressor(maxDepth=2, varianceCol="variance") @@ -654,10 +654,10 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi 3 >>> model.featureImportances SparseVector(1, {0: 1.0}) - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> dtr_path = temp_path + "/dtr" @@ -809,7 +809,7 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2, seed=42) @@ -818,10 +818,10 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 0.5 >>> rfr_path = temp_path + "/rfr" @@ -921,7 +921,7 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, >>> from numpy import allclose >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> gbt = GBTRegressor(maxIter=5, maxDepth=2, seed=42) @@ -932,10 +932,10 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True - >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 - >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) + >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 >>> gbtr_path = temp_path + "gbtr" @@ -1056,7 +1056,7 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi .. seealso:: `AFT Model `_ >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0), 1.0), ... (0.0, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) >>> aftsr = AFTSurvivalRegression() @@ -1257,7 +1257,7 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha .. seealso:: `GLM `_ >>> from pyspark.ml.linalg import Vectors - >>> df = sqlContext.createDataFrame([ + >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(0.0, 0.0)), ... (1.0, Vectors.dense(1.0, 2.0)), ... (2.0, Vectors.dense(0.0, 0.0)), @@ -1603,21 +1603,23 @@ def pValues(self): if __name__ == "__main__": import doctest import pyspark.ml.regression - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.regression.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.regression tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.regression tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 4f7a6b0f7b73b..fe87b6cdb9369 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -152,7 +152,7 @@ class CrossValidator(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator >>> from pyspark.ml.linalg import Vectors - >>> dataset = sqlContext.createDataFrame( + >>> dataset = spark.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), ... (Vectors.dense([0.5]), 0.0), @@ -311,7 +311,7 @@ class TrainValidationSplit(Estimator, ValidatorParams): >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator >>> from pyspark.ml.linalg import Vectors - >>> dataset = sqlContext.createDataFrame( + >>> dataset = spark.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), ... (Vectors.dense([0.5]), 0.0), @@ -456,17 +456,19 @@ def copy(self, extra=None): if __name__ == "__main__": import doctest - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.tuning tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.tuning tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = sqlContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - sc.stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index fe5b6844bfceb..f186217031193 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -756,12 +756,16 @@ def update(rdd): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession import pyspark.mllib.classification globs = pyspark.mllib.classification.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.classification tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 22e68ea5b4511..5f32f092c7cbe 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -516,12 +516,16 @@ def accuracy(self): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession import pyspark.mllib.evaluation globs = pyspark.mllib.evaluation.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.evaluation tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 90559f6cfbe43..e31c75c1e8675 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -732,11 +732,15 @@ def transform(self, vector): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.feature tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index f339e50891166..ab4066f7d68ba 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -183,16 +183,21 @@ class FreqSequence(namedtuple("FreqSequence", ["sequence", "freq"])): def _test(): import doctest + from pyspark.sql import SparkSession import pyspark.mllib.fpm globs = pyspark.mllib.fpm.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.fpm tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext import tempfile temp_path = tempfile.mkdtemp() globs['temp_path'] = temp_path try: (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() finally: from shutil import rmtree try: diff --git a/python/pyspark/mllib/linalg/distributed.py b/python/pyspark/mllib/linalg/distributed.py index af34ce346b0ca..ea4f27cf4ffe9 100644 --- a/python/pyspark/mllib/linalg/distributed.py +++ b/python/pyspark/mllib/linalg/distributed.py @@ -1184,16 +1184,18 @@ def toCoordinateMatrix(self): def _test(): import doctest - from pyspark import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession from pyspark.mllib.linalg import Matrices import pyspark.mllib.linalg.distributed globs = pyspark.mllib.linalg.distributed.__dict__.copy() - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) - globs['sqlContext'] = SQLContext(globs['sc']) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.linalg.distributed tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext globs['Matrices'] = Matrices (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 6a3c643b66417..61213ddf62e8b 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -409,13 +409,17 @@ def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed= def _test(): import doctest - from pyspark.context import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.random tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 639c5eabaa23b..43d9072a24e4d 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -824,12 +824,16 @@ def update(rdd): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession import pyspark.mllib.regression globs = pyspark.mllib.regression.__dict__.copy() - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.regression tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index 36c8f48a4a882..b0a85240b289a 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -306,11 +306,15 @@ def kolmogorovSmirnovTest(data, distName="norm", *params): def _test(): import doctest - from pyspark import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.stat.statistics tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index f7ea466b43291..8be76fcefe542 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -657,9 +657,14 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, def _test(): import doctest globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + from pyspark.sql import SparkSession + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("mllib.tree tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 39bc6586dd582..a316ee1ad45ff 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -347,13 +347,17 @@ def generateLinearRDD(sc, nexamples, nfeatures, eps, def _test(): import doctest - from pyspark.context import SparkContext + from pyspark.sql import SparkSession globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("mllib.util tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 812dbba59e277..3033f147bc969 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -244,21 +244,23 @@ def _reset(self): def _test(): import os import doctest - from pyspark.context import SparkContext - from pyspark.sql.session import SparkSession + from pyspark.sql import SparkSession import pyspark.sql.catalog os.chdir(os.environ["SPARK_HOME"]) globs = pyspark.sql.catalog.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') - globs['sc'] = sc - globs['spark'] = SparkSession(sc) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.catalog tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod( pyspark.sql.catalog, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 5b26e9469801e..4b99f3058b75c 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -434,13 +434,15 @@ def __repr__(self): def _test(): import doctest - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession import pyspark.sql.column globs = pyspark.sql.column.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.column tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) @@ -448,7 +450,7 @@ def _test(): (failure_count, test_count) = doctest.testmod( pyspark.sql.column, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index 609d882a95a32..792c420ca6386 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -71,11 +71,14 @@ def _test(): os.chdir(os.environ["SPARK_HOME"]) globs = pyspark.sql.conf.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') - globs['sc'] = sc - globs['spark'] = SparkSession(sc) + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.conf tests")\ + .getOrCreate() + globs['sc'] = spark.sparkContext + globs['spark'] = spark (failure_count, test_count) = doctest.testmod(pyspark.sql.conf, globs=globs) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 716b16fdc9530..1f15eec645382 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -212,7 +212,7 @@ def broadcast(df): def coalesce(*cols): """Returns the first column that is not null. - >>> cDf = sqlContext.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b")) + >>> cDf = spark.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b")) >>> cDf.show() +----+----+ | a| b| @@ -252,7 +252,7 @@ def corr(col1, col2): >>> a = range(20) >>> b = [2 * x for x in range(20)] - >>> df = sqlContext.createDataFrame(zip(a, b), ["a", "b"]) + >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) >>> df.agg(corr("a", "b").alias('c')).collect() [Row(c=1.0)] """ @@ -267,7 +267,7 @@ def covar_pop(col1, col2): >>> a = [1] * 10 >>> b = [1] * 10 - >>> df = sqlContext.createDataFrame(zip(a, b), ["a", "b"]) + >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) >>> df.agg(covar_pop("a", "b").alias('c')).collect() [Row(c=0.0)] """ @@ -282,7 +282,7 @@ def covar_samp(col1, col2): >>> a = [1] * 10 >>> b = [1] * 10 - >>> df = sqlContext.createDataFrame(zip(a, b), ["a", "b"]) + >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) >>> df.agg(covar_samp("a", "b").alias('c')).collect() [Row(c=0.0)] """ @@ -373,7 +373,7 @@ def input_file_name(): def isnan(col): """An expression that returns true iff the column is NaN. - >>> df = sqlContext.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) + >>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) >>> df.select(isnan("a").alias("r1"), isnan(df.a).alias("r2")).collect() [Row(r1=False, r2=False), Row(r1=True, r2=True)] """ @@ -385,7 +385,7 @@ def isnan(col): def isnull(col): """An expression that returns true iff the column is null. - >>> df = sqlContext.createDataFrame([(1, None), (None, 2)], ("a", "b")) + >>> df = spark.createDataFrame([(1, None), (None, 2)], ("a", "b")) >>> df.select(isnull("a").alias("r1"), isnull(df.a).alias("r2")).collect() [Row(r1=False, r2=False), Row(r1=True, r2=True)] """ @@ -432,7 +432,7 @@ def nanvl(col1, col2): Both inputs should be floating point columns (DoubleType or FloatType). - >>> df = sqlContext.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) + >>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) >>> df.select(nanvl("a", "b").alias("r1"), nanvl(df.a, df.b).alias("r2")).collect() [Row(r1=1.0, r2=1.0), Row(r1=2.0, r2=2.0)] """ @@ -470,7 +470,7 @@ def round(col, scale=0): Round the given value to `scale` decimal places using HALF_UP rounding mode if `scale` >= 0 or at integral part when `scale` < 0. - >>> sqlContext.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect() + >>> spark.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect() [Row(r=3.0)] """ sc = SparkContext._active_spark_context @@ -483,7 +483,7 @@ def bround(col, scale=0): Round the given value to `scale` decimal places using HALF_EVEN rounding mode if `scale` >= 0 or at integral part when `scale` < 0. - >>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect() + >>> spark.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect() [Row(r=2.0)] """ sc = SparkContext._active_spark_context @@ -494,7 +494,7 @@ def bround(col, scale=0): def shiftLeft(col, numBits): """Shift the given value numBits left. - >>> sqlContext.createDataFrame([(21,)], ['a']).select(shiftLeft('a', 1).alias('r')).collect() + >>> spark.createDataFrame([(21,)], ['a']).select(shiftLeft('a', 1).alias('r')).collect() [Row(r=42)] """ sc = SparkContext._active_spark_context @@ -505,7 +505,7 @@ def shiftLeft(col, numBits): def shiftRight(col, numBits): """Shift the given value numBits right. - >>> sqlContext.createDataFrame([(42,)], ['a']).select(shiftRight('a', 1).alias('r')).collect() + >>> spark.createDataFrame([(42,)], ['a']).select(shiftRight('a', 1).alias('r')).collect() [Row(r=21)] """ sc = SparkContext._active_spark_context @@ -517,7 +517,7 @@ def shiftRight(col, numBits): def shiftRightUnsigned(col, numBits): """Unsigned shift the given value numBits right. - >>> df = sqlContext.createDataFrame([(-42,)], ['a']) + >>> df = spark.createDataFrame([(-42,)], ['a']) >>> df.select(shiftRightUnsigned('a', 1).alias('r')).collect() [Row(r=9223372036854775787)] """ @@ -575,7 +575,7 @@ def greatest(*cols): Returns the greatest value of the list of column names, skipping null values. This function takes at least 2 parameters. It will return null iff all parameters are null. - >>> df = sqlContext.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) + >>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) >>> df.select(greatest(df.a, df.b, df.c).alias("greatest")).collect() [Row(greatest=4)] """ @@ -591,7 +591,7 @@ def least(*cols): Returns the least value of the list of column names, skipping null values. This function takes at least 2 parameters. It will return null iff all parameters are null. - >>> df = sqlContext.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) + >>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) >>> df.select(least(df.a, df.b, df.c).alias("least")).collect() [Row(least=1)] """ @@ -647,7 +647,7 @@ def log(arg1, arg2=None): def log2(col): """Returns the base-2 logarithm of the argument. - >>> sqlContext.createDataFrame([(4,)], ['a']).select(log2('a').alias('log2')).collect() + >>> spark.createDataFrame([(4,)], ['a']).select(log2('a').alias('log2')).collect() [Row(log2=2.0)] """ sc = SparkContext._active_spark_context @@ -660,7 +660,7 @@ def conv(col, fromBase, toBase): """ Convert a number in a string column from one base to another. - >>> df = sqlContext.createDataFrame([("010101",)], ['n']) + >>> df = spark.createDataFrame([("010101",)], ['n']) >>> df.select(conv(df.n, 2, 16).alias('hex')).collect() [Row(hex=u'15')] """ @@ -673,7 +673,7 @@ def factorial(col): """ Computes the factorial of the given value. - >>> df = sqlContext.createDataFrame([(5,)], ['n']) + >>> df = spark.createDataFrame([(5,)], ['n']) >>> df.select(factorial(df.n).alias('f')).collect() [Row(f=120)] """ @@ -765,7 +765,7 @@ def date_format(date, format): NOTE: Use when ever possible specialized functions like `year`. These benefit from a specialized implementation. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(date_format('a', 'MM/dd/yyy').alias('date')).collect() [Row(date=u'04/08/2015')] """ @@ -778,7 +778,7 @@ def year(col): """ Extract the year of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(year('a').alias('year')).collect() [Row(year=2015)] """ @@ -791,7 +791,7 @@ def quarter(col): """ Extract the quarter of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(quarter('a').alias('quarter')).collect() [Row(quarter=2)] """ @@ -804,7 +804,7 @@ def month(col): """ Extract the month of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(month('a').alias('month')).collect() [Row(month=4)] """ @@ -817,7 +817,7 @@ def dayofmonth(col): """ Extract the day of the month of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(dayofmonth('a').alias('day')).collect() [Row(day=8)] """ @@ -830,7 +830,7 @@ def dayofyear(col): """ Extract the day of the year of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(dayofyear('a').alias('day')).collect() [Row(day=98)] """ @@ -843,7 +843,7 @@ def hour(col): """ Extract the hours of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df.select(hour('a').alias('hour')).collect() [Row(hour=13)] """ @@ -856,7 +856,7 @@ def minute(col): """ Extract the minutes of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df.select(minute('a').alias('minute')).collect() [Row(minute=8)] """ @@ -869,7 +869,7 @@ def second(col): """ Extract the seconds of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df.select(second('a').alias('second')).collect() [Row(second=15)] """ @@ -882,7 +882,7 @@ def weekofyear(col): """ Extract the week number of a given date as integer. - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['a']) >>> df.select(weekofyear(df.a).alias('week')).collect() [Row(week=15)] """ @@ -895,7 +895,7 @@ def date_add(start, days): """ Returns the date that is `days` days after `start` - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['d']) >>> df.select(date_add(df.d, 1).alias('d')).collect() [Row(d=datetime.date(2015, 4, 9))] """ @@ -908,7 +908,7 @@ def date_sub(start, days): """ Returns the date that is `days` days before `start` - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['d']) >>> df.select(date_sub(df.d, 1).alias('d')).collect() [Row(d=datetime.date(2015, 4, 7))] """ @@ -921,7 +921,7 @@ def datediff(end, start): """ Returns the number of days from `start` to `end`. - >>> df = sqlContext.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2']) + >>> df = spark.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2']) >>> df.select(datediff(df.d2, df.d1).alias('diff')).collect() [Row(diff=32)] """ @@ -934,7 +934,7 @@ def add_months(start, months): """ Returns the date that is `months` months after `start` - >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d']) + >>> df = spark.createDataFrame([('2015-04-08',)], ['d']) >>> df.select(add_months(df.d, 1).alias('d')).collect() [Row(d=datetime.date(2015, 5, 8))] """ @@ -947,7 +947,7 @@ def months_between(date1, date2): """ Returns the number of months between date1 and date2. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['t', 'd']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['t', 'd']) >>> df.select(months_between(df.t, df.d).alias('months')).collect() [Row(months=3.9495967...)] """ @@ -960,7 +960,7 @@ def to_date(col): """ Converts the column of StringType or TimestampType into DateType. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_date(df.t).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] """ @@ -975,7 +975,7 @@ def trunc(date, format): :param format: 'year', 'YYYY', 'yy' or 'month', 'mon', 'mm' - >>> df = sqlContext.createDataFrame([('1997-02-28',)], ['d']) + >>> df = spark.createDataFrame([('1997-02-28',)], ['d']) >>> df.select(trunc(df.d, 'year').alias('year')).collect() [Row(year=datetime.date(1997, 1, 1))] >>> df.select(trunc(df.d, 'mon').alias('month')).collect() @@ -993,7 +993,7 @@ def next_day(date, dayOfWeek): Day of the week parameter is case insensitive, and accepts: "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". - >>> df = sqlContext.createDataFrame([('2015-07-27',)], ['d']) + >>> df = spark.createDataFrame([('2015-07-27',)], ['d']) >>> df.select(next_day(df.d, 'Sun').alias('date')).collect() [Row(date=datetime.date(2015, 8, 2))] """ @@ -1006,7 +1006,7 @@ def last_day(date): """ Returns the last day of the month which the given date belongs to. - >>> df = sqlContext.createDataFrame([('1997-02-10',)], ['d']) + >>> df = spark.createDataFrame([('1997-02-10',)], ['d']) >>> df.select(last_day(df.d).alias('date')).collect() [Row(date=datetime.date(1997, 2, 28))] """ @@ -1045,7 +1045,7 @@ def from_utc_timestamp(timestamp, tz): """ Assumes given timestamp is UTC and converts to given timezone. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(from_utc_timestamp(df.t, "PST").alias('t')).collect() [Row(t=datetime.datetime(1997, 2, 28, 2, 30))] """ @@ -1058,7 +1058,7 @@ def to_utc_timestamp(timestamp, tz): """ Assumes given timestamp is in given timezone and converts to UTC. - >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00',)], ['t']) + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_utc_timestamp(df.t, "PST").alias('t')).collect() [Row(t=datetime.datetime(1997, 2, 28, 18, 30))] """ @@ -1087,7 +1087,7 @@ def window(timeColumn, windowDuration, slideDuration=None, startTime=None): The output column will be a struct called 'window' by default with the nested columns 'start' and 'end', where 'start' and 'end' will be of `TimestampType`. - >>> df = sqlContext.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val") + >>> df = spark.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val") >>> w = df.groupBy(window("date", "5 seconds")).agg(sum("val").alias("sum")) >>> w.select(w.window.start.cast("string").alias("start"), ... w.window.end.cast("string").alias("end"), "sum").collect() @@ -1124,7 +1124,7 @@ def crc32(col): Calculates the cyclic redundancy check value (CRC32) of a binary column and returns the value as a bigint. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(crc32('a').alias('crc32')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(crc32('a').alias('crc32')).collect() [Row(crc32=2743272264)] """ sc = SparkContext._active_spark_context @@ -1136,7 +1136,7 @@ def crc32(col): def md5(col): """Calculates the MD5 digest and returns the value as a 32 character hex string. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect() [Row(hash=u'902fbdd2b1df0c4f70b4a5d23525e932')] """ sc = SparkContext._active_spark_context @@ -1149,7 +1149,7 @@ def md5(col): def sha1(col): """Returns the hex string result of SHA-1. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect() [Row(hash=u'3c01bdbb26f358bab27f267924aa2c9a03fcfdb8')] """ sc = SparkContext._active_spark_context @@ -1179,7 +1179,7 @@ def sha2(col, numBits): def hash(*cols): """Calculates the hash code of given columns, and returns the result as a int column. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect() [Row(hash=-757602832)] """ sc = SparkContext._active_spark_context @@ -1215,7 +1215,7 @@ def concat(*cols): """ Concatenates multiple input string columns together into a single string column. - >>> df = sqlContext.createDataFrame([('abcd','123')], ['s', 'd']) + >>> df = spark.createDataFrame([('abcd','123')], ['s', 'd']) >>> df.select(concat(df.s, df.d).alias('s')).collect() [Row(s=u'abcd123')] """ @@ -1230,7 +1230,7 @@ def concat_ws(sep, *cols): Concatenates multiple input string columns together into a single string column, using the given separator. - >>> df = sqlContext.createDataFrame([('abcd','123')], ['s', 'd']) + >>> df = spark.createDataFrame([('abcd','123')], ['s', 'd']) >>> df.select(concat_ws('-', df.s, df.d).alias('s')).collect() [Row(s=u'abcd-123')] """ @@ -1268,7 +1268,7 @@ def format_number(col, d): :param col: the column name of the numeric value to be formatted :param d: the N decimal places - >>> sqlContext.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect() + >>> spark.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect() [Row(v=u'5.0000')] """ sc = SparkContext._active_spark_context @@ -1284,7 +1284,7 @@ def format_string(format, *cols): :param col: the column name of the numeric value to be formatted :param d: the N decimal places - >>> df = sqlContext.createDataFrame([(5, "hello")], ['a', 'b']) + >>> df = spark.createDataFrame([(5, "hello")], ['a', 'b']) >>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect() [Row(v=u'5 hello')] """ @@ -1301,7 +1301,7 @@ def instr(str, substr): NOTE: The position is not zero based, but 1 based index, returns 0 if substr could not be found in str. - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(instr(df.s, 'b').alias('s')).collect() [Row(s=2)] """ @@ -1317,7 +1317,7 @@ def substring(str, pos, len): returns the slice of byte array that starts at `pos` in byte and is of length `len` when str is Binary type - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(substring(df.s, 1, 2).alias('s')).collect() [Row(s=u'ab')] """ @@ -1334,7 +1334,7 @@ def substring_index(str, delim, count): returned. If count is negative, every to the right of the final delimiter (counting from the right) is returned. substring_index performs a case-sensitive match when searching for delim. - >>> df = sqlContext.createDataFrame([('a.b.c.d',)], ['s']) + >>> df = spark.createDataFrame([('a.b.c.d',)], ['s']) >>> df.select(substring_index(df.s, '.', 2).alias('s')).collect() [Row(s=u'a.b')] >>> df.select(substring_index(df.s, '.', -3).alias('s')).collect() @@ -1349,7 +1349,7 @@ def substring_index(str, delim, count): def levenshtein(left, right): """Computes the Levenshtein distance of the two given strings. - >>> df0 = sqlContext.createDataFrame([('kitten', 'sitting',)], ['l', 'r']) + >>> df0 = spark.createDataFrame([('kitten', 'sitting',)], ['l', 'r']) >>> df0.select(levenshtein('l', 'r').alias('d')).collect() [Row(d=3)] """ @@ -1370,7 +1370,7 @@ def locate(substr, str, pos=0): :param str: a Column of StringType :param pos: start position (zero based) - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(locate('b', df.s, 1).alias('s')).collect() [Row(s=2)] """ @@ -1384,7 +1384,7 @@ def lpad(col, len, pad): """ Left-pad the string column to width `len` with `pad`. - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(lpad(df.s, 6, '#').alias('s')).collect() [Row(s=u'##abcd')] """ @@ -1398,7 +1398,7 @@ def rpad(col, len, pad): """ Right-pad the string column to width `len` with `pad`. - >>> df = sqlContext.createDataFrame([('abcd',)], ['s',]) + >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(rpad(df.s, 6, '#').alias('s')).collect() [Row(s=u'abcd##')] """ @@ -1412,7 +1412,7 @@ def repeat(col, n): """ Repeats a string column n times, and returns it as a new string column. - >>> df = sqlContext.createDataFrame([('ab',)], ['s',]) + >>> df = spark.createDataFrame([('ab',)], ['s',]) >>> df.select(repeat(df.s, 3).alias('s')).collect() [Row(s=u'ababab')] """ @@ -1428,7 +1428,7 @@ def split(str, pattern): NOTE: pattern is a string represent the regular expression. - >>> df = sqlContext.createDataFrame([('ab12cd',)], ['s',]) + >>> df = spark.createDataFrame([('ab12cd',)], ['s',]) >>> df.select(split(df.s, '[0-9]+').alias('s')).collect() [Row(s=[u'ab', u'cd'])] """ @@ -1441,7 +1441,7 @@ def split(str, pattern): def regexp_extract(str, pattern, idx): """Extract a specific(idx) group identified by a java regex, from the specified string column. - >>> df = sqlContext.createDataFrame([('100-200',)], ['str']) + >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_extract('str', '(\d+)-(\d+)', 1).alias('d')).collect() [Row(d=u'100')] """ @@ -1455,7 +1455,7 @@ def regexp_extract(str, pattern, idx): def regexp_replace(str, pattern, replacement): """Replace all substrings of the specified string value that match regexp with rep. - >>> df = sqlContext.createDataFrame([('100-200',)], ['str']) + >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_replace('str', '(\\d+)', '--').alias('d')).collect() [Row(d=u'-----')] """ @@ -1469,7 +1469,7 @@ def regexp_replace(str, pattern, replacement): def initcap(col): """Translate the first letter of each word to upper case in the sentence. - >>> sqlContext.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect() + >>> spark.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect() [Row(v=u'Ab Cd')] """ sc = SparkContext._active_spark_context @@ -1482,7 +1482,7 @@ def soundex(col): """ Returns the SoundEx encoding for a string - >>> df = sqlContext.createDataFrame([("Peters",),("Uhrbach",)], ['name']) + >>> df = spark.createDataFrame([("Peters",),("Uhrbach",)], ['name']) >>> df.select(soundex(df.name).alias("soundex")).collect() [Row(soundex=u'P362'), Row(soundex=u'U612')] """ @@ -1509,7 +1509,7 @@ def hex(col): """Computes hex value of the given column, which could be StringType, BinaryType, IntegerType or LongType. - >>> sqlContext.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() + >>> spark.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() [Row(hex(a)=u'414243', hex(b)=u'3')] """ sc = SparkContext._active_spark_context @@ -1523,7 +1523,7 @@ def unhex(col): """Inverse of hex. Interprets each pair of characters as a hexadecimal number and converts to the byte representation of number. - >>> sqlContext.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect() + >>> spark.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect() [Row(unhex(a)=bytearray(b'ABC'))] """ sc = SparkContext._active_spark_context @@ -1535,7 +1535,7 @@ def unhex(col): def length(col): """Calculates the length of a string or binary expression. - >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(length('a').alias('length')).collect() + >>> spark.createDataFrame([('ABC',)], ['a']).select(length('a').alias('length')).collect() [Row(length=3)] """ sc = SparkContext._active_spark_context @@ -1550,7 +1550,7 @@ def translate(srcCol, matching, replace): The translate will happen when any character in the string matching with the character in the `matching`. - >>> sqlContext.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123")\ + >>> spark.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123")\ .alias('r')).collect() [Row(r=u'1a2s3ae')] """ @@ -1608,7 +1608,7 @@ def array_contains(col, value): :param col: name of column containing array :param value: value to check for in array - >>> df = sqlContext.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) + >>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) >>> df.select(array_contains(df.data, "a")).collect() [Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)] """ @@ -1621,7 +1621,7 @@ def explode(col): """Returns a new row for each element in the given array or map. >>> from pyspark.sql import Row - >>> eDF = sqlContext.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})]) + >>> eDF = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})]) >>> eDF.select(explode(eDF.intlist).alias("anInt")).collect() [Row(anInt=1), Row(anInt=2), Row(anInt=3)] @@ -1648,7 +1648,7 @@ def get_json_object(col, path): :param path: path to the json object to extract >>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')] - >>> df = sqlContext.createDataFrame(data, ("key", "jstring")) + >>> df = spark.createDataFrame(data, ("key", "jstring")) >>> df.select(df.key, get_json_object(df.jstring, '$.f1').alias("c0"), \ get_json_object(df.jstring, '$.f2').alias("c1") ).collect() [Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)] @@ -1667,7 +1667,7 @@ def json_tuple(col, *fields): :param fields: list of fields to extract >>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')] - >>> df = sqlContext.createDataFrame(data, ("key", "jstring")) + >>> df = spark.createDataFrame(data, ("key", "jstring")) >>> df.select(df.key, json_tuple(df.jstring, 'f1', 'f2')).collect() [Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)] """ @@ -1683,7 +1683,7 @@ def size(col): :param col: name of column or expression - >>> df = sqlContext.createDataFrame([([1, 2, 3],),([1],),([],)], ['data']) + >>> df = spark.createDataFrame([([1, 2, 3],),([1],),([],)], ['data']) >>> df.select(size(df.data)).collect() [Row(size(data)=3), Row(size(data)=1), Row(size(data)=0)] """ @@ -1698,7 +1698,7 @@ def sort_array(col, asc=True): :param col: name of column or expression - >>> df = sqlContext.createDataFrame([([2, 1, 3],),([1],),([],)], ['data']) + >>> df = spark.createDataFrame([([2, 1, 3],),([1],),([],)], ['data']) >>> df.select(sort_array(df.data).alias('r')).collect() [Row(r=[1, 2, 3]), Row(r=[1]), Row(r=[])] >>> df.select(sort_array(df.data, asc=False).alias('r')).collect() @@ -1775,18 +1775,21 @@ def udf(f, returnType=StringType()): def _test(): import doctest - from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext + from pyspark.sql import Row, SparkSession import pyspark.sql.functions globs = pyspark.sql.functions.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.functions tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = SQLContext(sc) + globs['spark'] = spark 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, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index ee734cb439287..6987af69cfb50 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -195,13 +195,15 @@ def pivot(self, pivot_col, values=None): def _test(): import doctest - from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext + from pyspark.sql import Row, SparkSession import pyspark.sql.group globs = pyspark.sql.group.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') + spark = SparkSession.builder\ + .master("local[4]")\ + .appName("sql.group tests")\ + .getOrCreate() + sc = spark.sparkContext globs['sc'] = sc - globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')]) \ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) @@ -216,7 +218,7 @@ def _test(): (failure_count, test_count) = doctest.testmod( pyspark.sql.group, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) - globs['sc'].stop() + spark.stop() if failure_count: exit(-1) From c189be976454be8ab878b4a8b44e7d7112cca6ab Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 23 May 2016 21:07:14 -0700 Subject: [PATCH 0358/1470] [SPARK-15485][SQL][DOCS] Spark SQL Configuration #### What changes were proposed in this pull request? So far, the page Configuration in the official documentation does not have a section for Spark SQL. http://spark.apache.org/docs/latest/configuration.html For Spark users, the information and default values of these public configuration parameters are very useful. This PR is to add this missing section to the configuration.html. rxin yhuai marmbrus #### How was this patch tested? Below is the generated webpage. screenshot 2016-05-23 11 35 57 screenshot 2016-05-23 11 37 38 screenshot 2016-05-23 11 36 11 screenshot 2016-05-23 11 36 18 Author: gatorsmile Closes #13263 from gatorsmile/configurationSQL. (cherry picked from commit d207716451f722c899b3845ee454f1e16c094125) Signed-off-by: Reynold Xin --- docs/configuration.md | 42 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index d23f0fe1a1c46..d6471a8cc7873 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1480,6 +1480,48 @@ Apart from these, the following properties are also available, and may be useful +#### Spark SQL +Running the SET -v command will show the entire list of the SQL configuration. + +
    +
    + +{% highlight scala %} +// spark is an existing SparkSession +spark.sql("SET -v").show(numRows = 200, truncate = false) +{% endhighlight %} + +
    + +
    + +{% highlight java %} +// spark is an existing SparkSession +spark.sql("SET -v").show(200, false); +{% endhighlight %} +
    + +
    + +{% highlight python %} +# spark is an existing SparkSession +spark.sql("SET -v").show(n=200, truncate=False) +{% endhighlight %} + +
    + +
    + +{% highlight r %} +# sqlContext is an existing sqlContext. +properties <- sql(sqlContext, "SET -v") +showDF(properties, numRows = 200, truncate = FALSE) +{% endhighlight %} + +
    +
    + + #### Spark Streaming From d0bcec157d2bd2ed4eff848f831841bef4745904 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Mon, 23 May 2016 21:12:34 -0700 Subject: [PATCH 0359/1470] [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB ## What changes were proposed in this pull request? This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method. ## How was this patch tested? Added new tests Author: Kazuaki Ishizaki Closes #13243 from kiszk/SPARK-15285. (cherry picked from commit fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf) Signed-off-by: Wenchen Fan --- .../expressions/objects/objects.scala | 32 +++++++++++++---- .../spark/sql/DataFrameComplexTypeSuite.scala | 35 +++++++++++++++++++ 2 files changed, 61 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 2f2323fa3a25f..cc094f2a96a8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -232,27 +232,47 @@ case class NewInstance( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) - val argGen = arguments.map(_.genCode(ctx)) - val argString = argGen.map(_.value).mkString(", ") + val argIsNulls = ctx.freshName("argIsNulls") + ctx.addMutableState("boolean[]", argIsNulls, + s"$argIsNulls = new boolean[${arguments.size}];") + val argValues = arguments.zipWithIndex.map { case (e, i) => + val argValue = ctx.freshName("argValue") + ctx.addMutableState(ctx.javaType(e.dataType), argValue, "") + argValue + } + + val argCodes = arguments.zipWithIndex.map { case (e, i) => + val expr = e.genCode(ctx) + expr.code + s""" + $argIsNulls[$i] = ${expr.isNull}; + ${argValues(i)} = ${expr.value}; + """ + } + val argCode = ctx.splitExpressions(ctx.INPUT_ROW, argCodes) val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) var isNull = ev.isNull val setIsNull = if (propagateNull && arguments.nonEmpty) { - s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" + s""" + boolean $isNull = false; + for (int idx = 0; idx < ${arguments.length}; idx++) { + if ($argIsNulls[idx]) { $isNull = true; break; } + } + """ } else { isNull = "false" "" } val constructorCall = outer.map { gen => - s"""${gen.value}.new ${cls.getSimpleName}($argString)""" + s"""${gen.value}.new ${cls.getSimpleName}(${argValues.mkString(", ")})""" }.getOrElse { - s"new $className($argString)" + s"new $className(${argValues.mkString(", ")})" } val code = s""" - ${argGen.map(_.code).mkString("\n")} + $argCode ${outer.map(_.code).getOrElse("")} $setIsNull final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 72f676e6225ee..07fbaba0f819c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -58,4 +58,39 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { val nullIntRow = df.selectExpr("i[1]").collect()(0) assert(nullIntRow == org.apache.spark.sql.Row(null)) } + + test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { + val ds100_5 = Seq(S100_5()).toDS() + ds100_5.rdd.count + } } + +case class S100( + s1: String = "1", s2: String = "2", s3: String = "3", s4: String = "4", + s5: String = "5", s6: String = "6", s7: String = "7", s8: String = "8", + s9: String = "9", s10: String = "10", s11: String = "11", s12: String = "12", + s13: String = "13", s14: String = "14", s15: String = "15", s16: String = "16", + s17: String = "17", s18: String = "18", s19: String = "19", s20: String = "20", + s21: String = "21", s22: String = "22", s23: String = "23", s24: String = "24", + s25: String = "25", s26: String = "26", s27: String = "27", s28: String = "28", + s29: String = "29", s30: String = "30", s31: String = "31", s32: String = "32", + s33: String = "33", s34: String = "34", s35: String = "35", s36: String = "36", + s37: String = "37", s38: String = "38", s39: String = "39", s40: String = "40", + s41: String = "41", s42: String = "42", s43: String = "43", s44: String = "44", + s45: String = "45", s46: String = "46", s47: String = "47", s48: String = "48", + s49: String = "49", s50: String = "50", s51: String = "51", s52: String = "52", + s53: String = "53", s54: String = "54", s55: String = "55", s56: String = "56", + s57: String = "57", s58: String = "58", s59: String = "59", s60: String = "60", + s61: String = "61", s62: String = "62", s63: String = "63", s64: String = "64", + s65: String = "65", s66: String = "66", s67: String = "67", s68: String = "68", + s69: String = "69", s70: String = "70", s71: String = "71", s72: String = "72", + s73: String = "73", s74: String = "74", s75: String = "75", s76: String = "76", + s77: String = "77", s78: String = "78", s79: String = "79", s80: String = "80", + s81: String = "81", s82: String = "82", s83: String = "83", s84: String = "84", + s85: String = "85", s86: String = "86", s87: String = "87", s88: String = "88", + s89: String = "89", s90: String = "90", s91: String = "91", s92: String = "92", + s93: String = "93", s94: String = "94", s95: String = "95", s96: String = "96", + s97: String = "97", s98: String = "98", s99: String = "99", s100: String = "100") + +case class S100_5( + s1: S100 = S100(), s2: S100 = S100(), s3: S100 = S100(), s4: S100 = S100(), s5: S100 = S100()) From 1890f5fdf501104171f93c759e20835a61c8927b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 23 May 2016 21:43:34 -0700 Subject: [PATCH 0360/1470] Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB" This reverts commit d0bcec157d2bd2ed4eff848f831841bef4745904. --- .../expressions/objects/objects.scala | 32 ++++------------- .../spark/sql/DataFrameComplexTypeSuite.scala | 35 ------------------- 2 files changed, 6 insertions(+), 61 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index cc094f2a96a8a..2f2323fa3a25f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -232,47 +232,27 @@ case class NewInstance( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) - val argIsNulls = ctx.freshName("argIsNulls") - ctx.addMutableState("boolean[]", argIsNulls, - s"$argIsNulls = new boolean[${arguments.size}];") - val argValues = arguments.zipWithIndex.map { case (e, i) => - val argValue = ctx.freshName("argValue") - ctx.addMutableState(ctx.javaType(e.dataType), argValue, "") - argValue - } - - val argCodes = arguments.zipWithIndex.map { case (e, i) => - val expr = e.genCode(ctx) - expr.code + s""" - $argIsNulls[$i] = ${expr.isNull}; - ${argValues(i)} = ${expr.value}; - """ - } - val argCode = ctx.splitExpressions(ctx.INPUT_ROW, argCodes) + val argGen = arguments.map(_.genCode(ctx)) + val argString = argGen.map(_.value).mkString(", ") val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) var isNull = ev.isNull val setIsNull = if (propagateNull && arguments.nonEmpty) { - s""" - boolean $isNull = false; - for (int idx = 0; idx < ${arguments.length}; idx++) { - if ($argIsNulls[idx]) { $isNull = true; break; } - } - """ + s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" } else { isNull = "false" "" } val constructorCall = outer.map { gen => - s"""${gen.value}.new ${cls.getSimpleName}(${argValues.mkString(", ")})""" + s"""${gen.value}.new ${cls.getSimpleName}($argString)""" }.getOrElse { - s"new $className(${argValues.mkString(", ")})" + s"new $className($argString)" } val code = s""" - $argCode + ${argGen.map(_.code).mkString("\n")} ${outer.map(_.code).getOrElse("")} $setIsNull final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 07fbaba0f819c..72f676e6225ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -58,39 +58,4 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { val nullIntRow = df.selectExpr("i[1]").collect()(0) assert(nullIntRow == org.apache.spark.sql.Row(null)) } - - test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { - val ds100_5 = Seq(S100_5()).toDS() - ds100_5.rdd.count - } } - -case class S100( - s1: String = "1", s2: String = "2", s3: String = "3", s4: String = "4", - s5: String = "5", s6: String = "6", s7: String = "7", s8: String = "8", - s9: String = "9", s10: String = "10", s11: String = "11", s12: String = "12", - s13: String = "13", s14: String = "14", s15: String = "15", s16: String = "16", - s17: String = "17", s18: String = "18", s19: String = "19", s20: String = "20", - s21: String = "21", s22: String = "22", s23: String = "23", s24: String = "24", - s25: String = "25", s26: String = "26", s27: String = "27", s28: String = "28", - s29: String = "29", s30: String = "30", s31: String = "31", s32: String = "32", - s33: String = "33", s34: String = "34", s35: String = "35", s36: String = "36", - s37: String = "37", s38: String = "38", s39: String = "39", s40: String = "40", - s41: String = "41", s42: String = "42", s43: String = "43", s44: String = "44", - s45: String = "45", s46: String = "46", s47: String = "47", s48: String = "48", - s49: String = "49", s50: String = "50", s51: String = "51", s52: String = "52", - s53: String = "53", s54: String = "54", s55: String = "55", s56: String = "56", - s57: String = "57", s58: String = "58", s59: String = "59", s60: String = "60", - s61: String = "61", s62: String = "62", s63: String = "63", s64: String = "64", - s65: String = "65", s66: String = "66", s67: String = "67", s68: String = "68", - s69: String = "69", s70: String = "70", s71: String = "71", s72: String = "72", - s73: String = "73", s74: String = "74", s75: String = "75", s76: String = "76", - s77: String = "77", s78: String = "78", s79: String = "79", s80: String = "80", - s81: String = "81", s82: String = "82", s83: String = "83", s84: String = "84", - s85: String = "85", s86: String = "86", s87: String = "87", s88: String = "88", - s89: String = "89", s90: String = "90", s91: String = "91", s92: String = "92", - s93: String = "93", s94: String = "94", s95: String = "95", s96: String = "96", - s97: String = "97", s98: String = "98", s99: String = "99", s100: String = "100") - -case class S100_5( - s1: S100 = S100(), s2: S100 = S100(), s3: S100 = S100(), s4: S100 = S100(), s5: S100 = S100()) From 6adbc061361bd4fd66bedb5b5a9997cf489d6439 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 23 May 2016 23:29:15 -0700 Subject: [PATCH 0361/1470] [SPARK-15397][SQL] fix string udf locate as hive ## What changes were proposed in this pull request? in hive, `locate("aa", "aaa", 0)` would yield 0, `locate("aa", "aaa", 1)` would yield 1 and `locate("aa", "aaa", 2)` would yield 2, while in Spark, `locate("aa", "aaa", 0)` would yield 1, `locate("aa", "aaa", 1)` would yield 2 and `locate("aa", "aaa", 2)` would yield 0. This results from the different understanding of the third parameter in udf `locate`. It means the starting index and starts from 1, so when we use 0, the return would always be 0. ## How was this patch tested? tested with modified `StringExpressionsSuite` and `StringFunctionsSuite` Author: Daoyuan Wang Closes #13186 from adrian-wang/locate. (cherry picked from commit d642b273544bb77ef7f584326aa2d214649ac61b) Signed-off-by: Andrew Or --- R/pkg/R/functions.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- python/pyspark/sql/functions.py | 2 +- .../expressions/stringExpressions.scala | 19 +++++++++++++------ .../expressions/StringExpressionsSuite.scala | 16 +++++++++------- .../spark/sql/StringFunctionsSuite.scala | 10 +++++----- 6 files changed, 30 insertions(+), 21 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 4a0bdf331523e..2665d1d477802 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2226,7 +2226,7 @@ setMethod("window", signature(x = "Column"), #' @export #' @examples \dontrun{locate('b', df$c, 1)} setMethod("locate", signature(substr = "character", str = "Column"), - function(substr, str, pos = 0) { + function(substr, str, pos = 1) { jc <- callJStatic("org.apache.spark.sql.functions", "locate", substr, str@jc, as.integer(pos)) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 6a99b43e5aa59..b2d769f2acd0c 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1152,7 +1152,7 @@ test_that("string operators", { l2 <- list(list(a = "aaads")) df2 <- createDataFrame(sqlContext, l2) expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) - expect_equal(collect(select(df2, locate("aa", df2$a, 1)))[1, 1], 2) + expect_equal(collect(select(df2, locate("aa", df2$a, 2)))[1, 1], 2) expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") # nolint expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") # nolint diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 1f15eec645382..64b8bc442dad0 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1359,7 +1359,7 @@ def levenshtein(left, right): @since(1.5) -def locate(substr, str, pos=0): +def locate(substr, str, pos=1): """ Locate the position of the first occurrence of substr in a string column, after position pos. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 78e846d3f580e..44ff7fda8ef49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -494,7 +494,7 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) extends TernaryExpression with ImplicitCastInputTypes { def this(substr: Expression, str: Expression) = { - this(substr, str, Literal(0)) + this(substr, str, Literal(1)) } override def children: Seq[Expression] = substr :: str :: start :: Nil @@ -516,9 +516,14 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) if (l == null) { null } else { - l.asInstanceOf[UTF8String].indexOf( - r.asInstanceOf[UTF8String], - s.asInstanceOf[Int]) + 1 + val sVal = s.asInstanceOf[Int] + if (sVal < 1) { + 0 + } else { + l.asInstanceOf[UTF8String].indexOf( + r.asInstanceOf[UTF8String], + s.asInstanceOf[Int] - 1) + 1 + } } } } @@ -537,8 +542,10 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) if (!${substrGen.isNull}) { ${strGen.code} if (!${strGen.isNull}) { - ${ev.value} = ${strGen.value}.indexOf(${substrGen.value}, - ${startGen.value}) + 1; + if (${startGen.value} > 0) { + ${ev.value} = ${strGen.value}.indexOf(${substrGen.value}, + ${startGen.value} - 1) + 1; + } } else { ${ev.isNull} = true; } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index c09c64fd6bf62..29bf15bf524b9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -508,16 +508,18 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val s2 = 'b.string.at(1) val s3 = 'c.string.at(2) val s4 = 'd.int.at(3) - val row1 = create_row("aaads", "aa", "zz", 1) - val row2 = create_row(null, "aa", "zz", 0) - val row3 = create_row("aaads", null, "zz", 0) - val row4 = create_row(null, null, null, 0) + val row1 = create_row("aaads", "aa", "zz", 2) + val row2 = create_row(null, "aa", "zz", 1) + val row3 = create_row("aaads", null, "zz", 1) + val row4 = create_row(null, null, null, 1) checkEvaluation(new StringLocate(Literal("aa"), Literal("aaads")), 1, row1) - checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(1)), 2, row1) - checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(2)), 0, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(0)), 0, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(1)), 1, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(2)), 2, row1) + checkEvaluation(StringLocate(Literal("aa"), Literal("aaads"), Literal(3)), 0, row1) checkEvaluation(new StringLocate(Literal("de"), Literal("aaads")), 0, row1) - checkEvaluation(StringLocate(Literal("de"), Literal("aaads"), 1), 0, row1) + checkEvaluation(StringLocate(Literal("de"), Literal("aaads"), 2), 0, row1) checkEvaluation(new StringLocate(s2, s1), 1, row1) checkEvaluation(StringLocate(s2, s1, s4), 2, row1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index c7b95c2683d81..1de2d9b5adab1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -189,15 +189,15 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { } test("string locate function") { - val df = Seq(("aaads", "aa", "zz", 1)).toDF("a", "b", "c", "d") + val df = Seq(("aaads", "aa", "zz", 2)).toDF("a", "b", "c", "d") checkAnswer( - df.select(locate("aa", $"a"), locate("aa", $"a", 1)), - Row(1, 2)) + df.select(locate("aa", $"a"), locate("aa", $"a", 2), locate("aa", $"a", 0)), + Row(1, 2, 0)) checkAnswer( - df.selectExpr("locate(b, a)", "locate(b, a, d)"), - Row(1, 2)) + df.selectExpr("locate(b, a)", "locate(b, a, d)", "locate(b, a, 3)"), + Row(1, 2, 0)) } test("string padding functions") { From 6ee1583c34a29f3e0c6e99b1e7ba4fd59f4cdfd9 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 24 May 2016 10:02:10 +0200 Subject: [PATCH 0362/1470] [SPARK-15442][ML][PYSPARK] Add 'relativeError' param to PySpark QuantileDiscretizer This PR adds the `relativeError` param to PySpark's `QuantileDiscretizer` to match Scala. Also cleaned up a duplication of `numBuckets` where the param is both a class and instance attribute (I removed the instance attr to match the style of params throughout `ml`). Finally, cleaned up the docs for `QuantileDiscretizer` to reflect that it now uses `approxQuantile`. ## How was this patch tested? A little doctest and built API docs locally to check HTML doc generation. Author: Nick Pentreath Closes #13228 from MLnick/SPARK-15442-py-relerror-param. (cherry picked from commit 6075f5b4d8e98483d26c31576f58e2229024b4f4) Signed-off-by: Nick Pentreath --- .../ml/feature/QuantileDiscretizer.scala | 13 +++-- python/pyspark/ml/feature.py | 51 +++++++++++++------ 2 files changed, 44 insertions(+), 20 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 5a6daa06ef546..61483590cde38 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -50,13 +50,13 @@ private[feature] trait QuantileDiscretizerBase extends Params /** * Relative error (see documentation for * [[org.apache.spark.sql.DataFrameStatFunctions.approxQuantile approxQuantile]] for description) - * Must be a number in [0, 1]. + * Must be in the range [0, 1]. * default: 0.001 * @group param */ val relativeError = new DoubleParam(this, "relativeError", "The relative target precision " + - "for approxQuantile", - ParamValidators.inRange(0.0, 1.0)) + "for the approximate quantile algorithm used to generate buckets. " + + "Must be in the range [0, 1].", ParamValidators.inRange(0.0, 1.0)) setDefault(relativeError -> 0.001) /** @group getParam */ @@ -66,8 +66,11 @@ private[feature] trait QuantileDiscretizerBase extends Params /** * :: Experimental :: * `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned - * categorical features. The bin ranges are chosen by taking a sample of the data and dividing it - * into roughly equal parts. The lower and upper bin bounds will be -Infinity and +Infinity, + * categorical features. The number of bins can be set using the `numBuckets` parameter. + * The bin ranges are chosen using an approximate algorithm (see the documentation for + * [[org.apache.spark.sql.DataFrameStatFunctions.approxQuantile approxQuantile]] + * for a detailed description). The precision of the approximation can be controlled with the + * `relativeError` parameter. The lower and upper bin bounds will be `-Infinity` and `+Infinity`, * covering all real values. */ @Experimental diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 93745c70c4fe6..eb555cb940931 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1177,16 +1177,20 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav .. note:: Experimental `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned - categorical features. The bin ranges are chosen by taking a sample of the data and dividing it - into roughly equal parts. The lower and upper bin bounds will be -Infinity and +Infinity, - covering all real values. This attempts to find numBuckets partitions based on a sample of data, - but it may find fewer depending on the data sample values. + categorical features. The number of bins can be set using the :py:attr:`numBuckets` parameter. + The bin ranges are chosen using an approximate algorithm (see the documentation for + :py:meth:`~.DataFrameStatFunctions.approxQuantile` for a detailed description). + The precision of the approximation can be controlled with the + :py:attr:`relativeError` parameter. + The lower and upper bin bounds will be `-Infinity` and `+Infinity`, covering all real values. >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) >>> qds = QuantileDiscretizer(numBuckets=2, - ... inputCol="values", outputCol="buckets", seed=123) + ... inputCol="values", outputCol="buckets", seed=123, relativeError=0.01) >>> qds.getSeed() 123 + >>> qds.getRelativeError() + 0.01 >>> bucketizer = qds.fit(df) >>> splits = bucketizer.getSplits() >>> splits[0] @@ -1205,32 +1209,35 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav .. versionadded:: 2.0.0 """ - # a placeholder to make it appear in the generated doc numBuckets = Param(Params._dummy(), "numBuckets", "Maximum number of buckets (quantiles, or " + - "categories) into which data points are grouped. Must be >= 2. Default 2.", + "categories) into which data points are grouped. Must be >= 2.", typeConverter=TypeConverters.toInt) + relativeError = Param(Params._dummy(), "relativeError", "The relative target precision for " + + "the approximate quantile algorithm used to generate buckets. " + + "Must be in the range [0, 1].", + typeConverter=TypeConverters.toFloat) + @keyword_only - def __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None): + def __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, relativeError=0.001): """ - __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None) + __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, relativeError=0.001) """ super(QuantileDiscretizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.QuantileDiscretizer", self.uid) - self.numBuckets = Param(self, "numBuckets", - "Maximum number of buckets (quantiles, or " + - "categories) into which data points are grouped. Must be >= 2.") - self._setDefault(numBuckets=2) + self._setDefault(numBuckets=2, relativeError=0.001) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("2.0.0") - def setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None): + def setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, + relativeError=0.001): """ - setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None) + setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, \ + relativeError=0.001) Set the params for the QuantileDiscretizer """ kwargs = self.setParams._input_kwargs @@ -1250,6 +1257,20 @@ def getNumBuckets(self): """ return self.getOrDefault(self.numBuckets) + @since("2.0.0") + def setRelativeError(self, value): + """ + Sets the value of :py:attr:`relativeError`. + """ + return self._set(relativeError=value) + + @since("2.0.0") + def getRelativeError(self): + """ + Gets the value of relativeError or its default value. + """ + return self.getOrDefault(self.relativeError) + def _create_model(self, java_model): """ Private method to convert the java_model to a Python model. From 9f55951a134baf5916226b1889db45b33a59e5b4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 24 May 2016 09:43:39 -0700 Subject: [PATCH 0363/1470] [SPARK-11753][SQL][TEST-HADOOP2.2] Make allowNonNumericNumbers option work ## What changes were proposed in this pull request? Jackson suppprts `allowNonNumericNumbers` option to parse non-standard non-numeric numbers such as "NaN", "Infinity", "INF". Currently used Jackson version (2.5.3) doesn't support it all. This patch upgrades the library and make the two ignored tests in `JsonParsingOptionsSuite` passed. ## How was this patch tested? `JsonParsingOptionsSuite`. Author: Liang-Chi Hsieh Author: Liang-Chi Hsieh Closes #9759 from viirya/fix-json-nonnumric. (cherry picked from commit c24b6b679c3efa053f7de19be73eb36dc70d9930) Signed-off-by: Wenchen Fan --- dev/deps/spark-deps-hadoop-2.2 | 11 ++-- dev/deps/spark-deps-hadoop-2.3 | 11 ++-- dev/deps/spark-deps-hadoop-2.4 | 11 ++-- dev/deps/spark-deps-hadoop-2.6 | 11 ++-- dev/deps/spark-deps-hadoop-2.7 | 11 ++-- pom.xml | 8 ++- python/pyspark/sql/readwriter.py | 3 + .../apache/spark/sql/DataFrameReader.scala | 2 + .../datasources/json/JacksonParser.scala | 28 +++++---- .../json/JsonParsingOptionsSuite.scala | 59 ++++++++++++++----- 10 files changed, 102 insertions(+), 53 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 91b333fcae4fe..a9068da8b2096 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -72,12 +72,13 @@ hk2-utils-2.4.0-b34.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.5.3.jar -jackson-core-2.5.3.jar +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar janino-2.7.8.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar @@ -127,7 +128,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 61ed4c0889b8e..7e60a313ae8fe 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -74,12 +74,13 @@ hk2-utils-2.4.0-b34.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.5.3.jar -jackson-core-2.5.3.jar +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar @@ -134,7 +135,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index fb014921765f9..70d33b4f4812c 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -74,12 +74,13 @@ hk2-utils-2.4.0-b34.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.5.3.jar -jackson-core-2.5.3.jar +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar @@ -134,7 +135,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 0baf4e84fff0a..a80f6bc2a4061 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -80,13 +80,14 @@ htrace-core-3.0.4.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.5.3.jar -jackson-core-2.5.3.jar +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar @@ -142,7 +143,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8be218cd68d91..c0b53f73cd499 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -80,13 +80,14 @@ htrace-core-3.1.0-incubating.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.5.3.jar -jackson-core-2.5.3.jar +jackson-annotations-2.7.3.jar +jackson-core-2.7.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.5.3.jar +jackson-databind-2.7.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.11-2.5.3.jar +jackson-module-paranamer-2.7.3.jar +jackson-module-scala_2.11-2.7.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar @@ -143,7 +144,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.6.jar +paranamer-2.8.jar parquet-column-1.7.0.jar parquet-common-1.7.0.jar parquet-encoding-1.7.0.jar diff --git a/pom.xml b/pom.xml index e778f77fa3151..f28aa14adc55f 100644 --- a/pom.xml +++ b/pom.xml @@ -160,7 +160,7 @@ ${scala.version} org.scala-lang 1.9.13 - 2.5.3 + 2.7.3 1.1.2.4 1.1.2 1.2.0-incubating @@ -180,6 +180,7 @@ 4.5.2-1 1.1 2.52.0 + 2.8 ${java.home} @@ -1825,6 +1826,11 @@ antlr4-runtime ${antlr4.version} + + com.thoughtworks.paranamer + paranamer + ${paranamer.version} + diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 855c9d666f0bc..6f788cf50c7d7 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -193,6 +193,9 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, set, it uses the default value, ``true``. :param allowNumericLeadingZero: allows leading zeros in numbers (e.g. 00012). If None is set, it uses the default value, ``false``. + :param allowNonNumericNumbers: allows using non-numeric numbers such as "NaN", "Infinity", + "-Infinity", "INF", "-INF", which are convertd to floating + point numbers, ``true``. :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character using backslash quoting mechanism. If None is set, it uses the default value, ``false``. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 57a2091fe8c77..0fed9171a8bf3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -293,6 +293,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * *
  • `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers * (e.g. 00012)
  • + *
  • `allowNonNumericNumbers` (default `true`): allows using non-numeric numbers such as "NaN", + * "Infinity", "-Infinity", "INF", "-INF", which are convertd to floating point numbers.
  • *
  • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala index aeee2600a19ee..cafca323187d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala @@ -129,13 +129,15 @@ object JacksonParser extends Logging { case (VALUE_STRING, FloatType) => // Special case handling for NaN and Infinity. val value = parser.getText - val lowerCaseValue = value.toLowerCase() - if (lowerCaseValue.equals("nan") || - lowerCaseValue.equals("infinity") || - lowerCaseValue.equals("-infinity") || - lowerCaseValue.equals("inf") || - lowerCaseValue.equals("-inf")) { + if (value.equals("NaN") || + value.equals("Infinity") || + value.equals("+Infinity") || + value.equals("-Infinity")) { value.toFloat + } else if (value.equals("+INF") || value.equals("INF")) { + Float.PositiveInfinity + } else if (value.equals("-INF")) { + Float.NegativeInfinity } else { throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.") } @@ -146,13 +148,15 @@ object JacksonParser extends Logging { case (VALUE_STRING, DoubleType) => // Special case handling for NaN and Infinity. val value = parser.getText - val lowerCaseValue = value.toLowerCase() - if (lowerCaseValue.equals("nan") || - lowerCaseValue.equals("infinity") || - lowerCaseValue.equals("-infinity") || - lowerCaseValue.equals("inf") || - lowerCaseValue.equals("-inf")) { + if (value.equals("NaN") || + value.equals("Infinity") || + value.equals("+Infinity") || + value.equals("-Infinity")) { value.toDouble + } else if (value.equals("+INF") || value.equals("INF")) { + Double.PositiveInfinity + } else if (value.equals("-INF")) { + Double.NegativeInfinity } else { throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index c31dffedbdf67..2aab955c1ecbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.json import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} /** * Test cases for various [[JSONOptions]]. @@ -93,23 +94,51 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSQLContext { assert(df.first().getLong(0) == 18) } - // The following two tests are not really working - need to look into Jackson's - // JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS. - ignore("allowNonNumericNumbers off") { - val str = """{"age": NaN}""" - val rdd = spark.sparkContext.parallelize(Seq(str)) - val df = spark.read.json(rdd) - - assert(df.schema.head.name == "_corrupt_record") + test("allowNonNumericNumbers off") { + // non-quoted non-numeric numbers don't work if allowNonNumericNumbers is off. + var testCases: Seq[String] = Seq("""{"age": NaN}""", """{"age": Infinity}""", + """{"age": +Infinity}""", """{"age": -Infinity}""", """{"age": INF}""", + """{"age": +INF}""", """{"age": -INF}""") + testCases.foreach { str => + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.option("allowNonNumericNumbers", "false").json(rdd) + + assert(df.schema.head.name == "_corrupt_record") + } + + // quoted non-numeric numbers should still work even allowNonNumericNumbers is off. + testCases = Seq("""{"age": "NaN"}""", """{"age": "Infinity"}""", """{"age": "+Infinity"}""", + """{"age": "-Infinity"}""", """{"age": "INF"}""", """{"age": "+INF"}""", + """{"age": "-INF"}""") + val tests: Seq[Double => Boolean] = Seq(_.isNaN, _.isPosInfinity, _.isPosInfinity, + _.isNegInfinity, _.isPosInfinity, _.isPosInfinity, _.isNegInfinity) + val schema = StructType(StructField("age", DoubleType, true) :: Nil) + + testCases.zipWithIndex.foreach { case (str, idx) => + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.option("allowNonNumericNumbers", "false").schema(schema).json(rdd) + + assert(df.schema.head.name == "age") + assert(tests(idx)(df.first().getDouble(0))) + } } - ignore("allowNonNumericNumbers on") { - val str = """{"age": NaN}""" - val rdd = spark.sparkContext.parallelize(Seq(str)) - val df = spark.read.option("allowNonNumericNumbers", "true").json(rdd) - - assert(df.schema.head.name == "age") - assert(df.first().getDouble(0).isNaN) + test("allowNonNumericNumbers on") { + val testCases: Seq[String] = Seq("""{"age": NaN}""", """{"age": Infinity}""", + """{"age": +Infinity}""", """{"age": -Infinity}""", """{"age": +INF}""", + """{"age": -INF}""", """{"age": "NaN"}""", """{"age": "Infinity"}""", + """{"age": "-Infinity"}""") + val tests: Seq[Double => Boolean] = Seq(_.isNaN, _.isPosInfinity, _.isPosInfinity, + _.isNegInfinity, _.isPosInfinity, _.isNegInfinity, _.isNaN, _.isPosInfinity, + _.isNegInfinity, _.isPosInfinity, _.isNegInfinity) + val schema = StructType(StructField("age", DoubleType, true) :: Nil) + testCases.zipWithIndex.foreach { case (str, idx) => + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.option("allowNonNumericNumbers", "true").schema(schema).json(rdd) + + assert(df.schema.head.name == "age") + assert(tests(idx)(df.first().getDouble(0))) + } } test("allowBackslashEscapingAnyCharacter off") { From 4e2a53ba4f0fcd1b5aff0b6428fe8babfd9bb842 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 24 May 2016 10:08:14 -0700 Subject: [PATCH 0364/1470] [SPARK-13135] [SQL] Don't print expressions recursively in generated code ## What changes were proposed in this pull request? This PR is an up-to-date and a little bit improved version of #11019 of rxin for - (1) preventing recursive printing of expressions in generated code. Since the major function of this PR is indeed the above, he should be credited for the work he did. In addition to #11019, this PR improves the followings in code generation. - (2) Improve multiline comment indentation. - (3) Reduce the number of empty lines (mainly consecutive empty lines). - (4) Remove all space characters on empty lines. **Example** ```scala spark.range(1, 1000).select('id+1+2+3, 'id+4+5+6) ``` **Before** ``` Generated code: /* 001 */ public Object generate(Object[] references) { ... /* 005 */ /** /* 006 */ * Codegend pipeline for /* 007 */ * Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 008 */ * +- Range 1, 1, 8, 999, [id#0L] /* 009 */ */ ... /* 075 */ // PRODUCE: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 076 */ /* 077 */ // PRODUCE: Range 1, 1, 8, 999, [id#0L] /* 078 */ /* 079 */ // initialize Range ... /* 092 */ // CONSUME: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 093 */ /* 094 */ // CONSUME: WholeStageCodegen /* 095 */ /* 096 */ // (((input[0, bigint, false] + 1) + 2) + 3) /* 097 */ // ((input[0, bigint, false] + 1) + 2) /* 098 */ // (input[0, bigint, false] + 1) ... /* 107 */ // (((input[0, bigint, false] + 4) + 5) + 6) /* 108 */ // ((input[0, bigint, false] + 4) + 5) /* 109 */ // (input[0, bigint, false] + 4) ... /* 126 */ } ``` **After** ``` Generated code: /* 001 */ public Object generate(Object[] references) { ... /* 005 */ /** /* 006 */ * Codegend pipeline for /* 007 */ * Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 008 */ * +- Range 1, 1, 8, 999, [id#0L] /* 009 */ */ ... /* 075 */ // PRODUCE: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 076 */ // PRODUCE: Range 1, 1, 8, 999, [id#0L] /* 077 */ // initialize Range ... /* 090 */ // CONSUME: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L] /* 091 */ // CONSUME: WholeStageCodegen /* 092 */ // (((input[0, bigint, false] + 1) + 2) + 3) ... /* 101 */ // (((input[0, bigint, false] + 4) + 5) + 6) ... /* 118 */ } ``` ## How was this patch tested? Pass the Jenkins tests and see the result of the following command manually. ```scala scala> spark.range(1, 1000).select('id+1+2+3, 'id+4+5+6).queryExecution.debug.codegen() ``` Author: Dongjoon Hyun Author: Reynold Xin Author: Dongjoon Hyun Closes #13192 from dongjoon-hyun/SPARK-13135. --- .../expressions/codegen/CodeFormatter.scala | 27 ++++++++++-- .../codegen/GenerateMutableProjection.scala | 3 +- .../codegen/GenerateOrdering.scala | 3 +- .../codegen/GeneratePredicate.scala | 3 +- .../codegen/GenerateSafeProjection.scala | 3 +- .../codegen/GenerateUnsafeProjection.scala | 3 +- .../codegen/GenerateUnsafeRowJoiner.scala | 2 +- .../codegen/CodeFormatterSuite.scala | 43 +++++++++++++++++++ .../sql/execution/WholeStageCodegenExec.scala | 4 +- .../columnar/GenerateColumnAccessor.scala | 3 +- 10 files changed, 82 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index c7410925da6c8..855ae6432da28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -40,7 +40,7 @@ object CodeFormatter { var lastLine: String = "dummy" input.split('\n').foreach { l => val line = l.trim() - val skip = line == "" && (lastLine == "" || lastLine.endsWith("{")) + val skip = line == "" && (lastLine == "" || lastLine.endsWith("{") || lastLine.endsWith("*/")) if (!skip) { code.append(line) code.append("\n") @@ -49,6 +49,24 @@ object CodeFormatter { } code.result() } + + def stripOverlappingComments(codeAndComment: CodeAndComment): CodeAndComment = { + val code = new StringBuilder + val map = codeAndComment.comment + var lastLine: String = "dummy" + codeAndComment.body.split('\n').foreach { l => + val line = l.trim() + val skip = lastLine.startsWith("/*") && lastLine.endsWith("*/") && + line.startsWith("/*") && line.endsWith("*/") && + map(lastLine).substring(3).contains(map(line).substring(3)) + if (!skip) { + code.append(line) + code.append("\n") + } + lastLine = line + } + new CodeAndComment(code.result().trim(), map) + } } private class CodeFormatter { @@ -100,8 +118,11 @@ private class CodeFormatter { indentString } code.append(f"/* ${currentLine}%03d */ ") - code.append(thisLineIndent) - code.append(line) + if (line.trim().length > 0) { + code.append(thisLineIndent) + if (inCommentBlock && line.startsWith("*") || line.startsWith("*/")) code.append(" ") + code.append(line) + } code.append("\n") indentLevel = newIndentLevel indentString = " " * (indentSize * newIndentLevel) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 1305289e783c6..0f82d2e613c73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -133,7 +133,8 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP } """ - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) 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 1c53d62a5e98a..c10829d4f14f3 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 @@ -136,7 +136,8 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR } }""" - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated Ordering by ${ordering.mkString(",")}:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[BaseOrdering] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index ef44e6b46b798..106bb27964cab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -61,7 +61,8 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool } }""" - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") val p = CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index 214dc40641f85..b891f94673752 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -181,7 +181,8 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] } """ - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 102f276e9b3ff..5efba4b3a6087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -390,7 +390,8 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro } """ - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index 4dc1678ff67bf..4aa5ec82471ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -193,7 +193,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U | } |} """.stripMargin - val code = new CodeAndComment(codeBody, Map.empty) + val code = CodeFormatter.stripOverlappingComments(new CodeAndComment(codeBody, Map.empty)) logDebug(s"SpecificUnsafeRowJoiner($schema1, $schema2):\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala index 6022f2dbbe948..76afc2e8ecbac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala @@ -36,6 +36,22 @@ class CodeFormatterSuite extends SparkFunSuite { } } + test("removing overlapping comments") { + val code = new CodeAndComment( + """/*project_c4*/ + |/*project_c3*/ + |/*project_c2*/ + """.stripMargin, + Map( + "/*project_c4*/" -> "// (((input[0, bigint, false] + 1) + 2) + 3))", + "/*project_c3*/" -> "// ((input[0, bigint, false] + 1) + 2)", + "/*project_c2*/" -> "// (input[0, bigint, false] + 1)" + )) + + val reducedCode = CodeFormatter.stripOverlappingComments(code) + assert(reducedCode.body === "/*project_c4*/") + } + testCase("basic example") { """class A { |blahblah; @@ -147,4 +163,31 @@ class CodeFormatterSuite extends SparkFunSuite { |/* 006 */ } """.stripMargin } + + // scalastyle:off whitespace.end.of.line + testCase("reduce empty lines") { + CodeFormatter.stripExtraNewLines( + """class A { + | + | + | /*** comment1 */ + | + | class body; + | + | + | if (c) {duh;} + | else {boo;} + |}""".stripMargin) + }{ + """ + |/* 001 */ class A { + |/* 002 */ /*** comment1 */ + |/* 003 */ class body; + |/* 004 */ + |/* 005 */ if (c) {duh;} + |/* 006 */ else {boo;} + |/* 007 */ } + """.stripMargin + } + // scalastyle:on whitespace.end.of.line } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 2a1ce735b74ea..908e22de73bdc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -333,8 +333,8 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co """.trim // try to compile, helpful for debug - val cleanedSource = - new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments()) + val cleanedSource = CodeFormatter.stripOverlappingComments( + new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments())) logDebug(s"\n${CodeFormatter.format(cleanedSource)}") CodeGenerator.compile(cleanedSource) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index e0b48119f61dc..1041bab9d5d6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -224,7 +224,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera } }""" - val code = new CodeAndComment(codeBody, ctx.getPlaceHolderToComments()) + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated ColumnarIterator:\n${CodeFormatter.format(code)}") CodeGenerator.compile(code).generate(Array.empty).asInstanceOf[ColumnarIterator] From 988d4dbf427a18b69a7eba25a9b786bc48cea637 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 24 May 2016 10:26:55 -0700 Subject: [PATCH 0365/1470] [SPARK-15405][YARN] Remove unnecessary upload of config archive. We only need one copy of it. The client code that was uploading the second copy just needs to be modified to update the metadata in the cache, so that the AM knows where to find the configuration. Tested by running app on YARN and verifying in the logs only one archive is uploaded. Author: Marcelo Vanzin Closes #13232 from vanzin/SPARK-15405. (cherry picked from commit a313a5ae74ae4e7686283657ba56076222317595) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/deploy/yarn/Client.scala | 9 +++++---- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 12 ++++++++---- 2 files changed, 13 insertions(+), 8 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 3f6d7b28b5403..a12391d08180d 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 @@ -591,10 +591,11 @@ private[spark] class Client( copyFileToRemote(destDir, localConfArchive, replication, force = true, destName = Some(LOCALIZED_CONF_ARCHIVE)) - val (_, confLocalizedPath) = distribute(createConfArchive().toURI().getPath(), - resType = LocalResourceType.ARCHIVE, - destName = Some(LOCALIZED_CONF_DIR)) - require(confLocalizedPath != null) + // Manually add the config archive to the cache manager so that the AM is launched with + // the proper files set up. + distCacheMgr.addResource( + remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE, + LOCALIZED_CONF_DIR, statCache, appMasterOnly = false) // Clear the cache-related entries from the configuration to avoid them polluting the // UI's environment page. This works for client mode; for cluster mode, this is handled 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 a408c48d1d231..0a4f291e25fb0 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 @@ -144,9 +144,16 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll .set(SPARK_JARS, Seq(SPARK)) .set("spark.yarn.dist.jars", ADDED) val client = createClient(sparkConf, args = Array("--jar", USER)) + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), + any(classOf[Path]), anyShort(), anyBoolean(), any()) val tempDir = Utils.createTempDir() try { + // Because we mocked "copyFileToRemote" above to avoid having to create fake local files, + // we need to create a fake config archive in the temp dir to avoid having + // prepareLocalResources throw an exception. + new FileOutputStream(new File(tempDir, LOCALIZED_CONF_ARCHIVE)).close() + client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil) sparkConf.get(APP_JAR) should be (Some(USER)) @@ -384,10 +391,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll conf: Configuration = new Configuration(), args: Array[String] = Array()): Client = { val clientArgs = new ClientArguments(args) - val client = spy(new Client(clientArgs, conf, sparkConf)) - doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), - any(classOf[Path]), anyShort(), anyBoolean(), any()) - client + spy(new Client(clientArgs, conf, sparkConf)) } private def classpath(client: Client): Array[String] = { From 1bb0aa4b0a5b680e535cac6305b9eac856606ef4 Mon Sep 17 00:00:00 2001 From: wangyang Date: Tue, 24 May 2016 11:03:12 -0700 Subject: [PATCH 0366/1470] [SPARK-15388][SQL] Fix spark sql CREATE FUNCTION with hive 1.2.1 ## What changes were proposed in this pull request? spark.sql("CREATE FUNCTION myfunc AS 'com.haizhi.bdp.udf.UDFGetGeoCode'") throws "org.apache.hadoop.hive.ql.metadata.HiveException:MetaException(message:NoSuchObjectException(message:Function default.myfunc does not exist))" with hive 1.2.1. I think it is introduced by pr #12853. Fixing it by catching Exception (not NoSuchObjectException) and string matching. ## How was this patch tested? added a unit test and also tested it manually Author: wangyang Closes #13177 from wangyang1992/fixCreateFunc2. (cherry picked from commit 784cc07d1675eb9e0a387673cf86874e1bfc10f9) Signed-off-by: Andrew Or --- .../apache/spark/sql/hive/client/HiveShim.scala | 14 ++++++++++++-- .../spark/sql/hive/client/VersionsSuite.scala | 1 + 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 78713c3f0bace..9df4a26d55a27 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -24,6 +24,7 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JS import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf @@ -42,7 +43,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{IntegralType, StringType} -import org.apache.spark.util.CausedBy /** @@ -480,11 +480,21 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { try { Option(hive.getFunction(db, name)).map(fromHiveFunction) } catch { - case CausedBy(ex: NoSuchObjectException) if ex.getMessage.contains(name) => + case NonFatal(e) if isCausedBy(e, s"$name does not exist") => None } } + private def isCausedBy(e: Throwable, matchMassage: String): Boolean = { + if (e.getMessage.contains(matchMassage)) { + true + } else if (e.getCause != null) { + isCausedBy(e.getCause, matchMassage) + } else { + false + } + } + override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { hive.getFunctions(db, pattern).asScala } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index d46c4e7b2b50b..8ae4535f4ebf4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -440,6 +440,7 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.getFunctionOption("default", "func2").isEmpty) } else { assert(client.getFunctionOption("default", "func2").isDefined) + assert(client.getFunctionOption("default", "the_func_not_exists").isEmpty) } } From 2574abea088be725b6ecf2c473819535cd1cf0a5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 24 May 2016 11:09:54 -0700 Subject: [PATCH 0367/1470] [MINOR][CORE][TEST] Update obsolete `takeSample` test case. ## What changes were proposed in this pull request? This PR fixes some obsolete comments and assertion in `takeSample` testcase of `RDDSuite.scala`. ## How was this patch tested? This fixes the testcase only. Author: Dongjoon Hyun Closes #13260 from dongjoon-hyun/SPARK-15481. (cherry picked from commit be99a99fe7976419d727c0cc92e872aa4af58bf1) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/rdd/RDDSuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) 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 979fb426c9482..a4992fe8ac556 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -678,27 +678,26 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } { val sample = data.takeSample(withReplacement = true, num = 20) - assert(sample.size === 20) // Got exactly 100 elements - assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.size === 20) // Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } { val sample = data.takeSample(withReplacement = true, num = n) - assert(sample.size === n) // Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n) // Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, n, seed) - assert(sample.size === n) // Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n) // Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, 2 * n, seed) - assert(sample.size === 2 * n) // Got exactly 200 elements - // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.size === 2 * n) // Got exactly 2 * n elements + // Chance of getting all distinct elements is still quite low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } } From 31fb5fa4042eb46c541e5726a3f14da6f9f6bc2d Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 24 May 2016 11:34:06 -0700 Subject: [PATCH 0368/1470] [SPARK-15502][DOC][ML][PYSPARK] add guide note that ALS only supports integer ids This PR adds a note to clarify that the ML API for ALS only supports integers for user/item ids, and that other types for these columns can be used but the ids must fall within integer range. (Refer [SPARK-14891](https://issues.apache.org/jira/browse/SPARK-14891)). Also cleaned up a reference to `mllib` in the ML doc. ## How was this patch tested? Built and viewed User Guide doc locally. Author: Nick Pentreath Closes #13278 from MLnick/SPARK-15502-als-int-id-doc-note. (cherry picked from commit 20900e5feced76e87f0a12823d0e3f07e082105f) Signed-off-by: Joseph K. Bradley --- docs/ml-collaborative-filtering.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md index bd3d527d9a0e3..8bd75f3bcf7a7 100644 --- a/docs/ml-collaborative-filtering.md +++ b/docs/ml-collaborative-filtering.md @@ -29,6 +29,10 @@ following parameters: *baseline* confidence in preference observations (defaults to 1.0). * *nonnegative* specifies whether or not to use nonnegative constraints for least squares (defaults to `false`). +**Note:** The DataFrame-based API for ALS currently only supports integers for user and item ids. +Other numeric types are supported for the user and item id columns, +but the ids must be within the integer value range. + ### Explicit vs. implicit feedback The standard approach to matrix factorization based collaborative filtering treats @@ -36,7 +40,7 @@ the entries in the user-item matrix as *explicit* preferences given by the user for example, users giving ratings to movies. It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, -clicks, purchases, likes, shares etc.). The approach used in `spark.mllib` to deal with such data is taken +clicks, purchases, likes, shares etc.). The approach used in `spark.ml` to deal with such data is taken from [Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). Essentially, instead of trying to model the matrix of ratings directly, this approach treats the data as numbers representing the *strength* in observations of user actions (such as the number of clicks, From 1fb7b3a0a2e3a5c5f784aab662df93fcc1449c36 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 24 May 2016 14:27:39 -0700 Subject: [PATCH 0369/1470] [SPARK-15458][SQL][STREAMING] Disable schema inference for streaming datasets on file streams ## What changes were proposed in this pull request? If the user relies on the schema to be inferred in file streams can break easily for multiple reasons - accidentally running on a directory which has no data - schema changing underneath - on restart, the query will infer schema again, and may unexpectedly infer incorrect schema, as the file in the directory may be different at the time of the restart. To avoid these complicated scenarios, for Spark 2.0, we are going to disable schema inferencing by default with a config, so that user is forced to consider explicitly what is the schema it wants, rather than the system trying to infer it and run into weird corner cases. In this PR, I introduce a SQLConf that determines whether schema inference for file streams is allowed or not. It is disabled by default. ## How was this patch tested? Updated unit tests that test error behavior with and without schema inference enabled. Author: Tathagata Das Closes #13238 from tdas/SPARK-15458. (cherry picked from commit e631b819fe348729aab062207a452b8f1d1511bd) Signed-off-by: Tathagata Das --- .../execution/datasources/DataSource.scala | 11 + .../apache/spark/sql/internal/SQLConf.scala | 7 + .../sql/streaming/FileStreamSourceSuite.scala | 239 +++++++++++------- 3 files changed, 166 insertions(+), 91 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index e5dd4d81d6779..d0853f67b97ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{CalendarIntervalType, StructType} import org.apache.spark.util.Utils @@ -186,6 +187,16 @@ case class DataSource( val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") }) + val isSchemaInferenceEnabled = sparkSession.conf.get(SQLConf.STREAMING_SCHEMA_INFERENCE) + val isTextSource = providingClass == classOf[text.DefaultSource] + // If the schema inference is disabled, only text sources require schema to be specified + if (!isSchemaInferenceEnabled && !isTextSource && userSpecifiedSchema.isEmpty) { + throw new IllegalArgumentException( + "Schema must be specified when creating a streaming source DataFrame. " + + "If some files already exist in the directory, then depending on the file format " + + "you may be able to create a static DataFrame on that directory with " + + "'spark.read.load(directory)' and infer schema from it.") + } SourceInfo(s"FileSource[$path]", inferFileFormatSchema(format)) case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f3064eb6ac6d6..b91518acce33a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -516,6 +516,13 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(60 * 1000L) // 10 minutes + val STREAMING_SCHEMA_INFERENCE = + SQLConfigBuilder("spark.sql.streaming.schemaInference") + .internal() + .doc("Whether file-based streaming sources will infer its own schema") + .booleanConf + .createWithDefault(false) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index c97304c0ec1c1..1d784f1f4ee85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -23,6 +23,7 @@ import java.util.UUID import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -165,19 +166,36 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { .collect { case s @ StreamingRelation(dataSource, _, _) => s.schema }.head } + // ============= Basic parameter exists tests ================ + test("FileStreamSource schema: no path") { - val e = intercept[IllegalArgumentException] { - createFileStreamSourceAndGetSchema(format = None, path = None, schema = None) + def testError(): Unit = { + val e = intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema(format = None, path = None, schema = None) + } + assert(e.getMessage.contains("path")) // reason is path, not schema } - assert("'path' is not specified" === e.getMessage) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { testError() } + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { testError() } } - test("FileStreamSource schema: path doesn't exist") { - intercept[AnalysisException] { + test("FileStreamSource schema: path doesn't exist, no schema") { + val e = intercept[IllegalArgumentException] { createFileStreamSourceAndGetSchema(format = None, path = Some("/a/b/c"), schema = None) } + assert(e.getMessage.toLowerCase.contains("schema")) // reason is schema absence, not the path + } + + test("FileStreamSource schema: path doesn't exist, with schema") { + val userSchema = new StructType().add(new StructField("value", IntegerType)) + val schema = createFileStreamSourceAndGetSchema( + format = None, path = Some("/a/b/c"), schema = Some(userSchema)) + assert(schema === userSchema) } + + // =============== Text file stream schema tests ================ + test("FileStreamSource schema: text, no existing files, no schema") { withTempDir { src => val schema = createFileStreamSourceAndGetSchema( @@ -205,13 +223,19 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== Parquet file stream schema tests ================ + test("FileStreamSource schema: parquet, no existing files, no schema") { withTempDir { src => - val e = intercept[AnalysisException] { - createFileStreamSourceAndGetSchema( - format = Some("parquet"), path = Some(new File(src, "1").getCanonicalPath), schema = None) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val e = intercept[AnalysisException] { + createFileStreamSourceAndGetSchema( + format = Some("parquet"), + path = Some(new File(src, "1").getCanonicalPath), + schema = None) + } + assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } - assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } } @@ -220,9 +244,21 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { Seq("a", "b", "c").toDS().as("userColumn").toDF().write .mode(org.apache.spark.sql.SaveMode.Overwrite) .parquet(src.getCanonicalPath) - val schema = createFileStreamSourceAndGetSchema( - format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) - assert(schema === new StructType().add("value", StringType)) + + // Without schema inference, should throw error + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { + intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema( + format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) + } + } + + // With schema inference, should infer correct schema + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val schema = createFileStreamSourceAndGetSchema( + format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("value", StringType)) + } } } @@ -237,22 +273,39 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== JSON file stream schema tests ================ + test("FileStreamSource schema: json, no existing files, no schema") { withTempDir { src => - val e = intercept[AnalysisException] { - createFileStreamSourceAndGetSchema( - format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + + val e = intercept[AnalysisException] { + createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + } + assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } - assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) } } test("FileStreamSource schema: json, existing files, no schema") { withTempDir { src => - stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c': '3'}") - val schema = createFileStreamSourceAndGetSchema( - format = Some("json"), path = Some(src.getCanonicalPath), schema = None) - assert(schema === new StructType().add("c", StringType)) + + // Without schema inference, should throw error + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { + intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + } + } + + // With schema inference, should infer correct schema + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c': '3'}") + val schema = createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("c", StringType)) + } } } @@ -266,6 +319,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== Text file stream tests ================ + test("read from text files") { withTempDirs { case (src, tmp) => val textStream = createFileStream("text", src.getCanonicalPath) @@ -284,6 +339,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== JSON file stream tests ================ + test("read from json files") { withTempDirs { case (src, tmp) => val fileStream = createFileStream("json", src.getCanonicalPath, Some(valueSchema)) @@ -313,74 +370,82 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { test("read from json files with inferring schema") { withTempDirs { case (src, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { - // Add a file so that we can infer its schema - stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") - val fileStream = createFileStream("json", src.getCanonicalPath) - assert(fileStream.schema === StructType(Seq(StructField("c", StringType)))) + val fileStream = createFileStream("json", src.getCanonicalPath) + assert(fileStream.schema === StructType(Seq(StructField("c", StringType)))) - // FileStreamSource should infer the column "c" - val filtered = fileStream.filter($"c" contains "keep") + // FileStreamSource should infer the column "c" + val filtered = fileStream.filter($"c" contains "keep") - testStream(filtered)( - AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), - CheckAnswer("keep2", "keep3", "keep5", "keep6") - ) + testStream(filtered)( + AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6") + ) + } } } test("reading from json files inside partitioned directory") { withTempDirs { case (baseSrc, tmp) => - val src = new File(baseSrc, "type=X") - src.mkdirs() + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val src = new File(baseSrc, "type=X") + src.mkdirs() - // Add a file so that we can infer its schema - stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") - val fileStream = createFileStream("json", src.getCanonicalPath) + val fileStream = createFileStream("json", src.getCanonicalPath) - // FileStreamSource should infer the column "c" - val filtered = fileStream.filter($"c" contains "keep") + // FileStreamSource should infer the column "c" + val filtered = fileStream.filter($"c" contains "keep") - testStream(filtered)( - AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), - CheckAnswer("keep2", "keep3", "keep5", "keep6") - ) + testStream(filtered)( + AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6") + ) + } } } test("reading from json files with changing schema") { withTempDirs { case (src, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { - // Add a file so that we can infer its schema - stringToFile(new File(src, "existing"), "{'k': 'value0'}") + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'k': 'value0'}") - val fileStream = createFileStream("json", src.getCanonicalPath) + val fileStream = createFileStream("json", src.getCanonicalPath) - // FileStreamSource should infer the column "k" - assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) + // FileStreamSource should infer the column "k" + assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) - // After creating DF and before starting stream, add data with different schema - // Should not affect the inferred schema any more - stringToFile(new File(src, "existing2"), "{'k': 'value1', 'v': 'new'}") + // After creating DF and before starting stream, add data with different schema + // Should not affect the inferred schema any more + stringToFile(new File(src, "existing2"), "{'k': 'value1', 'v': 'new'}") - testStream(fileStream)( + testStream(fileStream)( - // Should not pick up column v in the file added before start - AddTextFileData("{'k': 'value2'}", src, tmp), - CheckAnswer("value0", "value1", "value2"), + // Should not pick up column v in the file added before start + AddTextFileData("{'k': 'value2'}", src, tmp), + CheckAnswer("value0", "value1", "value2"), - // Should read data in column k, and ignore v - AddTextFileData("{'k': 'value3', 'v': 'new'}", src, tmp), - CheckAnswer("value0", "value1", "value2", "value3"), + // Should read data in column k, and ignore v + AddTextFileData("{'k': 'value3', 'v': 'new'}", src, tmp), + CheckAnswer("value0", "value1", "value2", "value3"), - // Should ignore rows that do not have the necessary k column - AddTextFileData("{'v': 'value4'}", src, tmp), - CheckAnswer("value0", "value1", "value2", "value3", null)) + // Should ignore rows that do not have the necessary k column + AddTextFileData("{'v': 'value4'}", src, tmp), + CheckAnswer("value0", "value1", "value2", "value3", null)) + } } } + // =============== Parquet file stream tests ================ + test("read from parquet files") { withTempDirs { case (src, tmp) => val fileStream = createFileStream("parquet", src.getCanonicalPath, Some(valueSchema)) @@ -402,49 +467,39 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { test("read from parquet files with changing schema") { withTempDirs { case (src, tmp) => - // Add a file so that we can infer its schema - AddParquetFileData.writeToFile(Seq("value0").toDF("k"), src, tmp) - - val fileStream = createFileStream("parquet", src.getCanonicalPath) + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { - // FileStreamSource should infer the column "k" - assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) + // Add a file so that we can infer its schema + AddParquetFileData.writeToFile(Seq("value0").toDF("k"), src, tmp) - // After creating DF and before starting stream, add data with different schema - // Should not affect the inferred schema any more - AddParquetFileData.writeToFile(Seq(("value1", 0)).toDF("k", "v"), src, tmp) + val fileStream = createFileStream("parquet", src.getCanonicalPath) - testStream(fileStream)( - // Should not pick up column v in the file added before start - AddParquetFileData(Seq("value2").toDF("k"), src, tmp), - CheckAnswer("value0", "value1", "value2"), + // FileStreamSource should infer the column "k" + assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) - // Should read data in column k, and ignore v - AddParquetFileData(Seq(("value3", 1)).toDF("k", "v"), src, tmp), - CheckAnswer("value0", "value1", "value2", "value3"), + // After creating DF and before starting stream, add data with different schema + // Should not affect the inferred schema any more + AddParquetFileData.writeToFile(Seq(("value1", 0)).toDF("k", "v"), src, tmp) - // Should ignore rows that do not have the necessary k column - AddParquetFileData(Seq("value5").toDF("v"), src, tmp), - CheckAnswer("value0", "value1", "value2", "value3", null) - ) - } - } + testStream(fileStream)( + // Should not pick up column v in the file added before start + AddParquetFileData(Seq("value2").toDF("k"), src, tmp), + CheckAnswer("value0", "value1", "value2"), - test("file stream source without schema") { - withTempDir { src => - // Only "text" doesn't need a schema - createFileStream("text", src.getCanonicalPath) + // Should read data in column k, and ignore v + AddParquetFileData(Seq(("value3", 1)).toDF("k", "v"), src, tmp), + CheckAnswer("value0", "value1", "value2", "value3"), - // Both "json" and "parquet" require a schema if no existing file to infer - intercept[AnalysisException] { - createFileStream("json", src.getCanonicalPath) - } - intercept[AnalysisException] { - createFileStream("parquet", src.getCanonicalPath) + // Should ignore rows that do not have the necessary k column + AddParquetFileData(Seq("value5").toDF("v"), src, tmp), + CheckAnswer("value0", "value1", "value2", "value3", null) + ) } } } + // =============== file stream globbing tests ================ + test("read new files in nested directories with globbing") { withTempDirs { case (dir, tmp) => @@ -518,6 +573,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } + // =============== other tests ================ + test("fault tolerance") { withTempDirs { case (src, tmp) => val fileStream = createFileStream("text", src.getCanonicalPath) From 1de3446d9240c4ee7513d3b2f3be2a77344c2e70 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 24 May 2016 18:55:23 -0700 Subject: [PATCH 0370/1470] [SPARK-15512][CORE] repartition(0) should raise IllegalArgumentException ## What changes were proposed in this pull request? Previously, SPARK-8893 added the constraints on positive number of partitions for repartition/coalesce operations in general. This PR adds one missing part for that and adds explicit two testcases. **Before** ```scala scala> sc.parallelize(1 to 5).coalesce(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> sc.parallelize(1 to 5).repartition(0).collect() res1: Array[Int] = Array() // empty scala> spark.sql("select 1").coalesce(0) res2: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int] scala> spark.sql("select 1").coalesce(0).collect() java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. scala> spark.sql("select 1").repartition(0) res3: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int] scala> spark.sql("select 1").repartition(0).collect() res4: Array[org.apache.spark.sql.Row] = Array() // empty ``` **After** ```scala scala> sc.parallelize(1 to 5).coalesce(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> sc.parallelize(1 to 5).repartition(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> spark.sql("select 1").coalesce(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... scala> spark.sql("select 1").repartition(0) java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive. ... ``` ## How was this patch tested? Pass the Jenkins tests with new testcases. Author: Dongjoon Hyun Closes #13282 from dongjoon-hyun/SPARK-15512. (cherry picked from commit f08bf587b1913c6cc8ecb34c45331cf4750961c9) Signed-off-by: Reynold Xin --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 + core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 8 ++++++++ .../catalyst/plans/logical/basicLogicalOperators.scala | 1 + .../spark/sql/catalyst/plans/logical/partitioning.scala | 2 +- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 8 ++++++++ 6 files changed, 27 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e6db9b3eec8ea..e251421c48fca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -437,6 +437,7 @@ abstract class RDD[T: ClassTag]( partitionCoalescer: Option[PartitionCoalescer] = Option.empty) (implicit ord: Ordering[T] = null) : RDD[T] = withScope { + require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") if (shuffle) { /** Distributes elements evenly across output partitions, starting from a random partition. */ val distributePartition = (index: Int, items: Iterator[T]) => { 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 a4992fe8ac556..ad56715656c85 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -276,6 +276,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { test("repartitioned RDDs") { val data = sc.parallelize(1 to 1000, 10) + intercept[IllegalArgumentException] { + data.repartition(0) + } + // Coalesce partitions val repartitioned1 = data.repartition(2) assert(repartitioned1.partitions.size == 2) @@ -329,6 +333,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { test("coalesced RDDs") { val data = sc.parallelize(1 to 10, 10) + intercept[IllegalArgumentException] { + data.coalesce(0) + } + val coalesced1 = data.coalesce(2) assert(coalesced1.collect().toList === (1 to 10).toList) assert(coalesced1.glom().collect().map(_.toList).toList === diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index ca0096eeb2087..0a9250b71fb61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -734,6 +734,7 @@ case class Distinct(child: LogicalPlan) extends UnaryNode { */ case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan) extends UnaryNode { + require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") override def output: Seq[Attribute] = child.output } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index a5bdee1b854ce..28cbce8748fcd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -43,7 +43,7 @@ case class RepartitionByExpression( child: LogicalPlan, numPartitions: Option[Int] = None) extends RedistributeData { numPartitions match { - case Some(n) => require(n > 0, "numPartitions must be greater than 0.") + case Some(n) => require(n > 0, s"Number of partitions ($n) must be positive.") case None => // Ok } } 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 f573abf859610..0614747352c11 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 @@ -259,12 +259,20 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("repartition") { + intercept[IllegalArgumentException] { + testData.select('key).repartition(0) + } + checkAnswer( testData.select('key).repartition(10).select('key), testData.select('key).collect().toSeq) } test("coalesce") { + intercept[IllegalArgumentException] { + testData.select('key).coalesce(0) + } + assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 0ffbd6db12be6..05de79eb2f1d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -81,6 +81,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val data = (1 to 100).map(i => ClassData(i.toString, i)) val ds = data.toDS() + intercept[IllegalArgumentException] { + ds.coalesce(0) + } + + intercept[IllegalArgumentException] { + ds.repartition(0) + } + assert(ds.repartition(10).rdd.partitions.length == 10) checkDataset( ds.repartition(10), From 5504f60e8a6cccd7361ab7787adfb0ffa904bc6c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 24 May 2016 20:55:47 -0700 Subject: [PATCH 0371/1470] [SPARK-15518] Rename various scheduler backend for consistency ## What changes were proposed in this pull request? This patch renames various scheduler backends to make them consistent: - LocalScheduler -> LocalSchedulerBackend - AppClient -> StandaloneAppClient - AppClientListener -> StandaloneAppClientListener - SparkDeploySchedulerBackend -> StandaloneSchedulerBackend - CoarseMesosSchedulerBackend -> MesosCoarseGrainedSchedulerBackend - MesosSchedulerBackend -> MesosFineGrainedSchedulerBackend ## How was this patch tested? Updated test cases to reflect the name change. Author: Reynold Xin Closes #13288 from rxin/SPARK-15518. (cherry picked from commit 14494da87bdf057d2d2f796b962a4d8bc4747d31) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/SparkContext.scala | 23 +++++----- ...Client.scala => StandaloneAppClient.scala} | 11 +++-- ...cala => StandaloneAppClientListener.scala} | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 11 +++-- ...scala => StandaloneSchedulerBackend.scala} | 15 +++--- ... MesosCoarseGrainedSchedulerBackend.scala} | 46 ++++++++++--------- ...=> MesosFineGrainedSchedulerBackend.scala} | 34 +++++++------- .../mesos/MesosSchedulerBackendUtil.scala | 2 +- ...la => LocalSchedulerBackendEndpoint.scala} | 18 ++++---- .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- .../SparkContextSchedulerCreationSuite.scala | 33 +++++++------ .../spark/deploy/client/AppClientSuite.scala | 4 +- ...sCoarseGrainedSchedulerBackendSuite.scala} | 12 +++-- ...sosFineGrainedSchedulerBackendSuite.scala} | 15 +++--- 14 files changed, 123 insertions(+), 107 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/client/{AppClient.scala => StandaloneAppClient.scala} (97%) rename core/src/main/scala/org/apache/spark/deploy/client/{AppClientListener.scala => StandaloneAppClientListener.scala} (96%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/{SparkDeploySchedulerBackend.scala => StandaloneSchedulerBackend.scala} (94%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{CoarseMesosSchedulerBackend.scala => MesosCoarseGrainedSchedulerBackend.scala} (93%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{MesosSchedulerBackend.scala => MesosFineGrainedSchedulerBackend.scala} (93%) rename core/src/main/scala/org/apache/spark/scheduler/local/{LocalBackend.scala => LocalSchedulerBackendEndpoint.scala} (87%) rename core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/{CoarseMesosSchedulerBackendSuite.scala => MesosCoarseGrainedSchedulerBackendSuite.scala} (96%) rename core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/{MesosSchedulerBackendSuite.scala => MesosFineGrainedSchedulerBackendSuite.scala} (95%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 351024bea47e6..36aa3becb4fc3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -56,10 +56,9 @@ import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, - SparkDeploySchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.scheduler.local.LocalBackend +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend} +import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump import org.apache.spark.ui.{ConsoleProgressBar, SparkUI} @@ -2426,7 +2425,7 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(sc.getConf, scheduler, 1) + val backend = new LocalSchedulerBackendEndpoint(sc.getConf, scheduler, 1) scheduler.initialize(backend) (backend, scheduler) @@ -2438,7 +2437,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(sc.getConf, scheduler, threadCount) + val backend = new LocalSchedulerBackendEndpoint(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) @@ -2448,14 +2447,14 @@ 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(sc.getConf, scheduler, threadCount) + val backend = new LocalSchedulerBackendEndpoint(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) + val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) (backend, scheduler) @@ -2472,9 +2471,9 @@ object SparkContext extends Logging { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt, sc.conf) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) + val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) - backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { + backend.shutdownCallback = (backend: StandaloneSchedulerBackend) => { localCluster.stop() } (backend, scheduler) @@ -2484,9 +2483,9 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc) val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", defaultValue = true) val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, sc, mesosUrl, sc.env.securityManager) + new MesosCoarseGrainedSchedulerBackend(scheduler, sc, mesosUrl, sc.env.securityManager) } else { - new MesosSchedulerBackend(scheduler, sc, mesosUrl) + new MesosFineGrainedSchedulerBackend(scheduler, sc, mesosUrl) } scheduler.initialize(backend) (backend, scheduler) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala rename to core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index 43b17e5d49bf1..a9df732df93ca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -32,17 +32,18 @@ import org.apache.spark.rpc._ import org.apache.spark.util.{RpcUtils, ThreadUtils} /** - * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, - * an app description, and a listener for cluster events, and calls back the listener when various - * events occur. + * Interface allowing applications to speak with a Spark standalone cluster manager. + * + * Takes a master URL, an app description, and a listener for cluster events, and calls + * back the listener when various events occur. * * @param masterUrls Each url should look like spark://host:port. */ -private[spark] class AppClient( +private[spark] class StandaloneAppClient( rpcEnv: RpcEnv, masterUrls: Array[String], appDescription: ApplicationDescription, - listener: AppClientListener, + listener: StandaloneAppClientListener, conf: SparkConf) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala rename to core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala index 94506a0cbb27c..370b16ce4213a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala @@ -24,7 +24,7 @@ package org.apache.spark.deploy.client * * Users of this API should *not* block inside the callback methods. */ -private[spark] trait AppClientListener { +private[spark] trait StandaloneAppClientListener { def connected(appId: String): Unit /** Disconnection may be a temporary state, as we fail over to a new Master. */ 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 371fb8602f785..01e85ca405587 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -33,21 +33,22 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality +import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. - * It can also work with a local setup by using a LocalBackend and setting isLocal to true. - * It handles common logic, like determining a scheduling order across jobs, waking up to launch - * speculative tasks, etc. + * It can also work with a local setup by using a [[LocalSchedulerBackendEndpoint]] and setting + * isLocal to true. It handles common logic, like determining a scheduling order across jobs, waking + * up to launch speculative tasks, etc. * * Clients should first call initialize() and start(), then submit task sets through the * runTasks method. * - * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple + * THREADING: [[SchedulerBackend]]s and task-submitting clients can call this class from multiple * threads, so it needs locks in public API methods to maintain its state. In addition, some - * SchedulerBackends synchronize on themselves when they want to send events here, and then + * [[SchedulerBackend]]s synchronize on themselves when they want to send events here, and then * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 85d002011d64c..8382fbe9ddb80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -21,28 +21,31 @@ import java.util.concurrent.Semaphore import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} -import org.apache.spark.deploy.client.{AppClient, AppClientListener} +import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} import org.apache.spark.internal.Logging import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ import org.apache.spark.util.Utils -private[spark] class SparkDeploySchedulerBackend( +/** + * A [[SchedulerBackend]] implementation for Spark's standalone cluster manager. + */ +private[spark] class StandaloneSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, masters: Array[String]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with AppClientListener + with StandaloneAppClientListener with Logging { - private var client: AppClient = null + private var client: StandaloneAppClient = null private var stopping = false private val launcherBackend = new LauncherBackend() { override protected def onStopRequest(): Unit = stop(SparkAppHandle.State.KILLED) } - @volatile var shutdownCallback: SparkDeploySchedulerBackend => Unit = _ + @volatile var shutdownCallback: StandaloneSchedulerBackend => Unit = _ @volatile private var appId: String = _ private val registrationBarrier = new Semaphore(0) @@ -100,7 +103,7 @@ private[spark] class SparkDeploySchedulerBackend( } val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor, initialExecutorLimit) - client = new AppClient(sc.env.rpcEnv, masters, appDesc, this, conf) + client = new StandaloneAppClient(sc.env.rpcEnv, masters, appDesc, this, conf) client.start() launcherBackend.setState(SparkAppHandle.State.SUBMITTED) waitForRegistration() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 2c5be1f528894..e88e4ad4750d7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -25,13 +25,12 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.{Buffer, HashMap, HashSet} -import org.apache.mesos.{Scheduler => MScheduler, SchedulerDriver} import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskState} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient -import org.apache.spark.rpc.{RpcEndpointAddress} +import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -43,16 +42,16 @@ import org.apache.spark.util.Utils * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable * latency. * - * Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to - * remove this. + * Unfortunately this has a bit of duplication from [[MesosFineGrainedSchedulerBackend]], + * but it seems hard to remove this. */ -private[spark] class CoarseMesosSchedulerBackend( +private[spark] class MesosCoarseGrainedSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String, securityManager: SecurityManager) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with MScheduler + with org.apache.mesos.Scheduler with MesosSchedulerUtils { val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures @@ -149,7 +148,7 @@ private[spark] class CoarseMesosSchedulerBackend( super.start() val driver = createSchedulerDriver( master, - CoarseMesosSchedulerBackend.this, + MesosCoarseGrainedSchedulerBackend.this, sc.sparkUser, sc.appName, sc.conf, @@ -239,9 +238,10 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} + override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + override def registered( + d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { appId = frameworkId.getValue mesosExternalShuffleClient.foreach(_.init(appId)) logInfo("Registered as framework ID " + appId) @@ -252,15 +252,15 @@ private[spark] class CoarseMesosSchedulerBackend( totalCoresAcquired >= maxCores * minRegisteredRatio } - override def disconnected(d: SchedulerDriver) {} + override def disconnected(d: org.apache.mesos.SchedulerDriver) {} - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} + override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {} /** * Method called by Mesos to offer resources on slaves. We respond by launching an executor, * unless we've already launched more than we wanted to. */ - override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { stateLock.synchronized { if (stopCalled) { logDebug("Ignoring offers during shutdown") @@ -282,7 +282,8 @@ private[spark] class CoarseMesosSchedulerBackend( } } - private def declineUnmatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { + private def declineUnmatchedOffers( + d: org.apache.mesos.SchedulerDriver, offers: Buffer[Offer]): Unit = { offers.foreach { offer => declineOffer(d, offer, Some("unmet constraints"), Some(rejectOfferDurationForUnmetConstraints)) @@ -290,7 +291,7 @@ private[spark] class CoarseMesosSchedulerBackend( } private def declineOffer( - d: SchedulerDriver, + d: org.apache.mesos.SchedulerDriver, offer: Offer, reason: Option[String] = None, refuseSeconds: Option[Long] = None): Unit = { @@ -319,7 +320,8 @@ private[spark] class CoarseMesosSchedulerBackend( * @param d SchedulerDriver * @param offers Mesos offers that match attribute constraints */ - private def handleMatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { + private def handleMatchedOffers( + d: org.apache.mesos.SchedulerDriver, offers: Buffer[Offer]): Unit = { val tasks = buildMesosTasks(offers) for (offer <- offers) { val offerAttributes = toAttributeMap(offer.getAttributesList) @@ -440,7 +442,7 @@ private[spark] class CoarseMesosSchedulerBackend( math.min(offerCPUs, maxCores - totalCoresAcquired)) } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { val taskId = status.getTaskId.getValue val slaveId = status.getSlaveId.getValue val state = TaskState.fromMesos(status.getState) @@ -498,7 +500,7 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def error(d: SchedulerDriver, message: String) { + override def error(d: org.apache.mesos.SchedulerDriver, message: String) { logError(s"Mesos error: $message") scheduler.error(message) } @@ -538,14 +540,15 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + override def frameworkMessage( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} /** * Called when a slave is lost or a Mesos task finished. Updates local view on * what tasks are running. It also notifies the driver that an executor was removed. */ private def executorTerminated( - d: SchedulerDriver, + d: org.apache.mesos.SchedulerDriver, slaveId: String, taskId: String, reason: String): Unit = { @@ -555,11 +558,12 @@ private[spark] class CoarseMesosSchedulerBackend( } } - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID): Unit = { + override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = { logInfo(s"Mesos slave lost: ${slaveId.getValue}") } - override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { + override def executorLost( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { logInfo("Mesos executor lost: %s".format(e.getValue)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index 1a94aee2ca30c..e08dc3b5957bb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -23,7 +23,6 @@ import java.util.{ArrayList => JArrayList, Collections, List => JList} import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, HashSet} -import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.protobuf.ByteString @@ -38,12 +37,12 @@ import org.apache.spark.util.Utils * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks * from multiple apps can run on different cores) and in time (a core can switch ownership). */ -private[spark] class MesosSchedulerBackend( +private[spark] class MesosFineGrainedSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String) extends SchedulerBackend - with MScheduler + with org.apache.mesos.Scheduler with MesosSchedulerUtils { // Stores the slave ids that has launched a Mesos executor. @@ -74,7 +73,7 @@ private[spark] class MesosSchedulerBackend( classLoader = Thread.currentThread.getContextClassLoader val driver = createSchedulerDriver( master, - MesosSchedulerBackend.this, + MesosFineGrainedSchedulerBackend.this, sc.sparkUser, sc.appName, sc.conf, @@ -175,9 +174,10 @@ private[spark] class MesosSchedulerBackend( execArgs } - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} + override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + override def registered( + d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { inClassLoader() { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) @@ -195,9 +195,9 @@ private[spark] class MesosSchedulerBackend( } } - override def disconnected(d: SchedulerDriver) {} + override def disconnected(d: org.apache.mesos.SchedulerDriver) {} - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} + override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {} private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = { val builder = new StringBuilder @@ -216,7 +216,7 @@ private[spark] class MesosSchedulerBackend( * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ - override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { inClassLoader() { // Fail first on offers with unmet constraints val (offersMatchingConstraints, offersNotMatchingConstraints) = @@ -355,7 +355,7 @@ private[spark] class MesosSchedulerBackend( (taskInfo, finalResources.asJava) } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { inClassLoader() { val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) @@ -373,7 +373,7 @@ private[spark] class MesosSchedulerBackend( } } - override def error(d: SchedulerDriver, message: String) { + override def error(d: org.apache.mesos.SchedulerDriver, message: String) { inClassLoader() { logError("Mesos error: " + message) markErr() @@ -391,7 +391,8 @@ private[spark] class MesosSchedulerBackend( mesosDriver.reviveOffers() } - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + override def frameworkMessage( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} /** * Remove executor associated with slaveId in a thread safe manner. @@ -403,7 +404,8 @@ private[spark] class MesosSchedulerBackend( } } - private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { + private def recordSlaveLost( + d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { inClassLoader() { logInfo("Mesos slave lost: " + slaveId.getValue) removeExecutor(slaveId.getValue, reason.toString) @@ -411,12 +413,12 @@ private[spark] class MesosSchedulerBackend( } } - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { + override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID) { recordSlaveLost(d, slaveId, SlaveLost()) } - override def executorLost(d: SchedulerDriver, executorId: ExecutorID, - slaveId: SlaveID, status: Int) { + override def executorLost( + d: org.apache.mesos.SchedulerDriver, executorId: ExecutorID, slaveId: SlaveID, status: Int) { logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue, slaveId.getValue)) recordSlaveLost(d, slaveId, ExecutorExited(status, exitCausedByApp = true)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala index 1b7ac172defb9..05b2b08944098 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -25,7 +25,7 @@ import org.apache.spark.internal.Logging /** * A collection of utility functions which can be used by both the - * MesosSchedulerBackend and the CoarseMesosSchedulerBackend. + * MesosSchedulerBackend and the [[MesosFineGrainedSchedulerBackend]]. */ private[mesos] object MesosSchedulerBackendUtil extends Logging { /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala similarity index 87% rename from core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala index 3473ef21b39a4..ee0658837997f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala @@ -39,15 +39,15 @@ private case class KillTask(taskId: Long, interruptThread: Boolean) private case class StopExecutor() /** - * Calls to LocalBackend are all serialized through LocalEndpoint. Using an RpcEndpoint makes the - * calls on LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend - * and the TaskSchedulerImpl. + * Calls to [[LocalSchedulerBackendEndpoint]] are all serialized through LocalEndpoint. Using an + * RpcEndpoint makes the calls on [[LocalSchedulerBackendEndpoint]] asynchronous, which is necessary + * to prevent deadlock between [[LocalSchedulerBackendEndpoint]] and the [[TaskSchedulerImpl]]. */ private[spark] class LocalEndpoint( override val rpcEnv: RpcEnv, userClassPath: Seq[URL], scheduler: TaskSchedulerImpl, - executorBackend: LocalBackend, + executorBackend: LocalSchedulerBackendEndpoint, private val totalCores: Int) extends ThreadSafeRpcEndpoint with Logging { @@ -91,11 +91,11 @@ private[spark] class LocalEndpoint( } /** - * LocalBackend is used when running a local version of Spark where the executor, backend, and - * 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. + * Used when running a local version of Spark where the executor, backend, and master all run in + * the same JVM. It sits behind a [[TaskSchedulerImpl]] and handles launching tasks on a single + * Executor (created by the [[LocalSchedulerBackendEndpoint]]) running locally. */ -private[spark] class LocalBackend( +private[spark] class LocalSchedulerBackendEndpoint( conf: SparkConf, scheduler: TaskSchedulerImpl, val totalCores: Int) @@ -124,7 +124,7 @@ private[spark] class LocalBackend( override def start() { val rpcEnv = SparkEnv.get.rpcEnv val executorEndpoint = new LocalEndpoint(rpcEnv, userClassPath, scheduler, this, totalCores) - localEndpoint = rpcEnv.setupEndpoint("LocalBackendEndpoint", executorEndpoint) + localEndpoint = rpcEnv.setupEndpoint("LocalSchedulerBackendEndpoint", executorEndpoint) listenerBus.post(SparkListenerExecutorAdded( System.currentTimeMillis, executorEndpoint.localExecutorId, diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 1adc90ab1e9dd..81b94b57219db 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -242,8 +242,8 @@ class HeartbeatReceiverSuite } private def getTrackedExecutors: Map[String, Long] = { - // We may receive undesired SparkListenerExecutorAdded from LocalBackend, so exclude it from - // the map. See SPARK-10800. + // We may receive undesired SparkListenerExecutorAdded from LocalSchedulerBackend, + // so exclude it from the map. See SPARK-10800. heartbeatReceiver.invokePrivate(_executorLastSeen()). filterKeys(_ != SparkContext.DRIVER_IDENTIFIER) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 213d70f4e5840..11be40abca099 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -21,10 +21,10 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.scheduler.local.LocalBackend -import org.apache.spark.util.Utils +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend} +import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint + class SparkContextSchedulerCreationSuite extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging { @@ -58,7 +58,7 @@ class SparkContextSchedulerCreationSuite test("local") { val sched = createTaskScheduler("local") sched.backend match { - case s: LocalBackend => assert(s.totalCores === 1) + case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 1) case _ => fail() } } @@ -66,7 +66,8 @@ class SparkContextSchedulerCreationSuite test("local-*") { val sched = createTaskScheduler("local[*]") sched.backend match { - case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case s: LocalSchedulerBackendEndpoint => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } } @@ -75,7 +76,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) sched.backend match { - case s: LocalBackend => assert(s.totalCores === 5) + case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 5) case _ => fail() } } @@ -84,7 +85,8 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[* ,2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case s: LocalSchedulerBackendEndpoint => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } } @@ -93,7 +95,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[4, 2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalBackend => assert(s.totalCores === 4) + case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 4) case _ => fail() } } @@ -117,14 +119,14 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local", "client", conf) sched.backend match { - case s: LocalBackend => assert(s.defaultParallelism() === 16) + case s: LocalSchedulerBackendEndpoint => assert(s.defaultParallelism() === 16) case _ => fail() } } test("local-cluster") { createTaskScheduler("local-cluster[3, 14, 1024]").backend match { - case s: SparkDeploySchedulerBackend => // OK + case s: StandaloneSchedulerBackend => // OK case _ => fail() } } @@ -143,19 +145,20 @@ class SparkContextSchedulerCreationSuite } test("mesos fine-grained") { - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) + testMesos("mesos://localhost:1234", classOf[MesosFineGrainedSchedulerBackend], coarse = false) } test("mesos coarse-grained") { - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) + testMesos("mesos://localhost:1234", classOf[MesosCoarseGrainedSchedulerBackend], coarse = true) } test("mesos with zookeeper") { testMesos("mesos://zk://localhost:1234,localhost:2345", - classOf[MesosSchedulerBackend], coarse = false) + classOf[MesosFineGrainedSchedulerBackend], coarse = false) } test("mesos with zookeeper and Master URL starting with zk://") { - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) + testMesos("zk://localhost:1234,localhost:2345", + classOf[MesosFineGrainedSchedulerBackend], coarse = false) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index 7b46f9101d89b..f6ef9d15ddee2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -166,7 +166,7 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd } /** Application Listener to collect events */ - private class AppClientCollector extends AppClientListener with Logging { + private class AppClientCollector extends StandaloneAppClientListener with Logging { val connectedIdList = new ConcurrentLinkedQueue[String]() @volatile var disconnectedCount: Int = 0 val deadReasonList = new ConcurrentLinkedQueue[String]() @@ -208,7 +208,7 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd List(), Map(), Seq(), Seq(), Seq()) private val desc = new ApplicationDescription("AppClientSuite", Some(1), 512, cmd, "ignored") val listener = new AppClientCollector - val client = new AppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) + val client = new StandaloneAppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala similarity index 96% rename from core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 15d59e7052912..7f21d4c623afc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.TaskSchedulerImpl -class CoarseMesosSchedulerBackendSuite extends SparkFunSuite +class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with BeforeAndAfter { @@ -44,7 +44,7 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite private var sparkConf: SparkConf = _ private var driver: SchedulerDriver = _ private var taskScheduler: TaskSchedulerImpl = _ - private var backend: CoarseMesosSchedulerBackend = _ + private var backend: MesosCoarseGrainedSchedulerBackend = _ private var externalShuffleClient: MesosExternalShuffleClient = _ private var driverEndpoint: RpcEndpointRef = _ @@ -230,7 +230,8 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) val securityManager = mock[SecurityManager] - val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master", securityManager) { + val backend = new MesosCoarseGrainedSchedulerBackend( + taskScheduler, sc, "master", securityManager) { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, @@ -323,10 +324,11 @@ class CoarseMesosSchedulerBackendSuite extends SparkFunSuite taskScheduler: TaskSchedulerImpl, driver: SchedulerDriver, shuffleClient: MesosExternalShuffleClient, - endpoint: RpcEndpointRef): CoarseMesosSchedulerBackend = { + endpoint: RpcEndpointRef): MesosCoarseGrainedSchedulerBackend = { val securityManager = mock[SecurityManager] - val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master", securityManager) { + val backend = new MesosCoarseGrainedSchedulerBackend( + taskScheduler, sc, "master", securityManager) { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, 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/MesosFineGrainedSchedulerBackendSuite.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index 7d6b7bde68253..41693b1191a3c 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/MesosFineGrainedSchedulerBackendSuite.scala @@ -40,7 +40,8 @@ import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.ExecutorInfo -class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { +class MesosFineGrainedSchedulerBackendSuite + extends SparkFunSuite with LocalSparkContext with MockitoSugar { test("weburi is set in created scheduler driver") { val conf = new SparkConf @@ -56,7 +57,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val driver = mock[SchedulerDriver] when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") { + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, @@ -96,7 +97,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val resources = Arrays.asList( mesosSchedulerBackend.createResource("cpus", 4), @@ -127,7 +128,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val mesosSchedulerBackend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val resources = Arrays.asList( mesosSchedulerBackend.createResource("cpus", 4), @@ -163,7 +164,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(sc.conf).thenReturn(conf) when(sc.listenerBus).thenReturn(listenerBus) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val (execInfo, _) = backend.createExecutorInfo( Arrays.asList(backend.createResource("cpus", 4)), "mockExecutor") @@ -222,7 +223,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(sc.conf).thenReturn(new SparkConf) when(sc.listenerBus).thenReturn(listenerBus) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val minMem = backend.executorMemory(sc) val minCpu = 4 @@ -333,7 +334,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val mesosOffers = new java.util.ArrayList[Offer] mesosOffers.add(offer) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master") val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1) expectedWorkerOffers.append(new WorkerOffer( From e13cfd6d265cd47365cda1a0452133434e0515df Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 24 May 2016 20:58:20 -0700 Subject: [PATCH 0372/1470] [SPARK-15365][SQL] When table size statistics are not available from metastore, we should fallback to HDFS ## What changes were proposed in this pull request? Currently if a table is used in join operation we rely on Metastore returned size to calculate if we can convert the operation to Broadcast join. This optimization only kicks in for table's that have the statistics available in metastore. Hive generally rolls over to HDFS if the statistics are not available directly from metastore and this seems like a reasonable choice to adopt given the optimization benefit of using broadcast joins. ## How was this patch tested? I have executed queries locally to test. Author: Parth Brahmbhatt Closes #13150 from Parth-Brahmbhatt/SPARK-15365. (cherry picked from commit 4acababcaba567c85f3be0d5e939d99119b82d1d) Signed-off-by: Reynold Xin --- .../apache/spark/sql/internal/SQLConf.scala | 9 ++++ .../spark/sql/hive/MetastoreRelation.scala | 33 +++++++++--- .../spark/sql/hive/StatisticsSuite.scala | 50 ++++++++++++++++++- 3 files changed, 83 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b91518acce33a..4efefdacabf22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -113,6 +113,13 @@ object SQLConf { .longConf .createWithDefault(10L * 1024 * 1024) + val ENABLE_FALL_BACK_TO_HDFS_FOR_STATS = + SQLConfigBuilder("spark.sql.enableFallBackToHdfsForStats") + .doc("If the table statistics are not available from table metadata enable fall back to hdfs." + + " This is useful in determining if a table is small enough to use auto broadcast joins.") + .booleanConf + .createWithDefault(false) + val DEFAULT_SIZE_IN_BYTES = SQLConfigBuilder("spark.sql.defaultSizeInBytes") .internal() .doc("The default table size used in query planning. By default, it is set to a larger " + @@ -603,6 +610,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def autoBroadcastJoinThreshold: Long = getConf(AUTO_BROADCASTJOIN_THRESHOLD) + def fallBackToHdfsForStatsEnabled: Boolean = getConf(ENABLE_FALL_BACK_TO_HDFS_FOR_STATS) + def preferSortMergeJoin: Boolean = getConf(PREFER_SORTMERGEJOIN) def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 1671228fd9b42..9c820144aee12 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -17,9 +17,12 @@ package org.apache.spark.sql.hive +import java.io.IOException + import scala.collection.JavaConverters._ import com.google.common.base.Objects +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.FieldSchema @@ -114,17 +117,31 @@ private[hive] case class MetastoreRelation( val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) // TODO: check if this estimate is valid for tables after partition pruning. // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be - // relatively cheap if parameters for the table are populated into the metastore. An - // alternative would be going through Hadoop's FileSystem API, which can be expensive if a lot - // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, - // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. + // relatively cheap if parameters for the table are populated into the metastore. + // Besides `totalSize`, there are also `numFiles`, `numRows`, `rawDataSize` keys + // (see StatsSetupConst in Hive) that we can look at in the future. BigInt( // When table is external,`totalSize` is always zero, which will influence join strategy // so when `totalSize` is zero, use `rawDataSize` instead - // if the size is still less than zero, we use default size - Option(totalSize).map(_.toLong).filter(_ > 0) - .getOrElse(Option(rawDataSize).map(_.toLong).filter(_ > 0) - .getOrElse(sparkSession.sessionState.conf.defaultSizeInBytes))) + // if the size is still less than zero, we try to get the file size from HDFS. + // given this is only needed for optimization, if the HDFS call fails we return the default. + if (totalSize != null && totalSize.toLong > 0L) { + totalSize.toLong + } else if (rawDataSize != null && rawDataSize.toLong > 0) { + rawDataSize.toLong + } else if (sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled) { + try { + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val fs: FileSystem = hiveQlTable.getPath.getFileSystem(hadoopConf) + fs.getContentSummary(hiveQlTable.getPath).getLength + } catch { + case e: IOException => + logWarning("Failed to get table size from hdfs.", e) + sparkSession.sessionState.conf.defaultSizeInBytes + } + } else { + sparkSession.sessionState.conf.defaultSizeInBytes + }) } ) 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 1a7b6c0112279..f8e00a35a31e2 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.io.{File, PrintWriter} + import scala.reflect.ClassTag import org.apache.spark.sql.{QueryTest, Row} @@ -25,8 +27,9 @@ import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils -class StatisticsSuite extends QueryTest with TestHiveSingleton { +class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { import hiveContext.sql test("parse analyze commands") { @@ -68,6 +71,51 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { classOf[AnalyzeTableCommand]) } + test("MetastoreRelations fallback to HDFS for size estimation") { + val enableFallBackToHdfsForStats = hiveContext.conf.fallBackToHdfsForStatsEnabled + try { + withTempDir { tempDir => + + // EXTERNAL OpenCSVSerde table pointing to LOCATION + + val file1 = new File(tempDir + "/data1") + val writer1 = new PrintWriter(file1) + writer1.write("1,2") + writer1.close() + + val file2 = new File(tempDir + "/data2") + val writer2 = new PrintWriter(file2) + writer2.write("1,2") + writer2.close() + + sql( + s"""CREATE EXTERNAL TABLE csv_table(page_id INT, impressions INT) + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' + WITH SERDEPROPERTIES ( + \"separatorChar\" = \",\", + \"quoteChar\" = \"\\\"\", + \"escapeChar\" = \"\\\\\") + LOCATION '$tempDir' + """) + + hiveContext.setConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS, true) + + val relation = hiveContext.sessionState.catalog.lookupRelation(TableIdentifier("csv_table")) + .asInstanceOf[MetastoreRelation] + + val properties = relation.hiveQlTable.getParameters + assert(properties.get("totalSize").toLong <= 0, "external table totalSize must be <= 0") + assert(properties.get("rawDataSize").toLong <= 0, "external table rawDataSize must be <= 0") + + val sizeInBytes = relation.statistics.sizeInBytes + assert(sizeInBytes === BigInt(file1.length() + file2.length())) + } + } finally { + hiveContext.setConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS, enableFallBackToHdfsForStats) + sql("DROP TABLE csv_table ") + } + } + ignore("analyze MetastoreRelations") { def queryTotalSize(tableName: String): BigInt = hiveContext.sessionState.catalog.lookupRelation( From fb7b90f61a7390b5ddb182413f628530cc72bee0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 24 May 2016 21:23:39 -0700 Subject: [PATCH 0373/1470] [SPARK-15498][TESTS] fix slow tests ## What changes were proposed in this pull request? This PR fixes 3 slow tests: 1. `ParquetQuerySuite.read/write wide table`: This is not a good unit test as it runs more than 5 minutes. This PR removes it and add a new regression test in `CodeGenerationSuite`, which is more "unit". 2. `ParquetQuerySuite.returning batch for wide table`: reduce the threshold and use smaller data size. 3. `DatasetSuite.SPARK-14554: Dataset.map may generate wrong java code for wide table`: Improve `CodeFormatter.format`(introduced at https://github.com/apache/spark/pull/12979) can dramatically speed this it up. ## How was this patch tested? N/A Author: Wenchen Fan Closes #13273 from cloud-fan/test. (cherry picked from commit 50b660d725269dc0c11e0d350ddd7fc8b19539a0) Signed-off-by: Cheng Lian --- .../expressions/codegen/CodeFormatter.scala | 48 +++++--- .../expressions/codegen/CodeGenerator.scala | 22 +--- .../expressions/CodeGenerationSuite.scala | 14 +++ .../codegen/CodeFormatterSuite.scala | 107 ++++++++++++------ .../parquet/ParquetQuerySuite.scala | 16 +-- 5 files changed, 128 insertions(+), 79 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index 855ae6432da28..05b7c96e44c02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import org.apache.commons.lang3.StringUtils +import java.util.regex.Matcher /** * An utility class that indents a block of code based on the curly braces and parentheses. @@ -26,13 +26,17 @@ import org.apache.commons.lang3.StringUtils * Written by Matei Zaharia. */ object CodeFormatter { + val commentHolder = """\/\*(.+?)\*\/""".r + def format(code: CodeAndComment): String = { - new CodeFormatter().addLines( - StringUtils.replaceEach( - code.body, - code.comment.keys.toArray, - code.comment.values.toArray) - ).result + val formatter = new CodeFormatter + code.body.split("\n").foreach { line => + val commentReplaced = commentHolder.replaceAllIn( + line.trim, + m => code.comment.get(m.group(1)).map(Matcher.quoteReplacement).getOrElse(m.group(0))) + formatter.addLine(commentReplaced) + } + formatter.result() } def stripExtraNewLines(input: String): String = { @@ -53,16 +57,28 @@ object CodeFormatter { def stripOverlappingComments(codeAndComment: CodeAndComment): CodeAndComment = { val code = new StringBuilder val map = codeAndComment.comment + + def getComment(line: String): Option[String] = { + if (line.startsWith("/*") && line.endsWith("*/")) { + map.get(line.substring(2, line.length - 2)) + } else { + None + } + } + var lastLine: String = "dummy" codeAndComment.body.split('\n').foreach { l => val line = l.trim() - val skip = lastLine.startsWith("/*") && lastLine.endsWith("*/") && - line.startsWith("/*") && line.endsWith("*/") && - map(lastLine).substring(3).contains(map(line).substring(3)) + + val skip = getComment(lastLine).zip(getComment(line)).exists { + case (lastComment, currentComment) => + lastComment.substring(3).contains(currentComment.substring(3)) + } + if (!skip) { - code.append(line) - code.append("\n") + code.append(line).append("\n") } + lastLine = line } new CodeAndComment(code.result().trim(), map) @@ -117,8 +133,9 @@ private class CodeFormatter { } else { indentString } - code.append(f"/* ${currentLine}%03d */ ") + code.append(f"/* ${currentLine}%03d */") if (line.trim().length > 0) { + code.append(" ") // add a space after the line number comment. code.append(thisLineIndent) if (inCommentBlock && line.startsWith("*") || line.startsWith("*/")) code.append(" ") code.append(line) @@ -129,10 +146,5 @@ private class CodeFormatter { currentLine += 1 } - private def addLines(code: String): CodeFormatter = { - code.split('\n').foreach(s => addLine(s.trim())) - this - } - private def result(): String = code.result() } 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 8b74d606dbb26..2706c38d9e0a1 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 @@ -717,28 +717,18 @@ class CodegenContext { */ def getPlaceHolderToComments(): collection.Map[String, String] = placeHolderToComments - /** - * Register a multi-line comment and return the corresponding place holder - */ - private def registerMultilineComment(text: String): String = { - val placeHolder = s"/*${freshName("c")}*/" - val comment = text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") - placeHolderToComments += (placeHolder -> comment) - placeHolder - } - /** * Register a comment and return the corresponding place holder */ def registerComment(text: String): String = { - if (text.contains("\n") || text.contains("\r")) { - registerMultilineComment(text) + val name = freshName("c") + val comment = if (text.contains("\n") || text.contains("\r")) { + text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") } else { - val placeHolder = s"/*${freshName("c")}*/" - val safeComment = s"// $text" - placeHolderToComments += (placeHolder -> safeComment) - placeHolder + s"// $text" } + placeHolderToComments += (name -> comment) + s"/*$name*/" } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index db34d12e286fa..8ffe39084c7f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.objects.CreateExternalRow import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -137,6 +138,19 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("SPARK-14224: split wide external row creation into blocks due to JVM code size limit") { + val length = 5000 + val schema = StructType(Seq.fill(length)(StructField("int", IntegerType))) + val expressions = Seq(CreateExternalRow(Seq.fill(length)(Literal(1)), schema)) + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) + val expected = Seq(Row.fromSeq(Seq.fill(length)(1))) + + if (!checkResult(actual, expected)) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + } + } + test("test generated safe and unsafe projection") { val schema = new StructType(Array( StructField("a", StringType, true), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala index 76afc2e8ecbac..bc5a8f078234a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatterSuite.scala @@ -23,9 +23,10 @@ import org.apache.spark.sql.catalyst.util._ class CodeFormatterSuite extends SparkFunSuite { - def testCase(name: String)(input: String)(expected: String): Unit = { + def testCase(name: String)( + input: String, comment: Map[String, String] = Map.empty)(expected: String): Unit = { test(name) { - val sourceCode = new CodeAndComment(input, Map.empty) + val sourceCode = new CodeAndComment(input.trim, comment) if (CodeFormatter.format(sourceCode).trim !== expected.trim) { fail( s""" @@ -43,9 +44,9 @@ class CodeFormatterSuite extends SparkFunSuite { |/*project_c2*/ """.stripMargin, Map( - "/*project_c4*/" -> "// (((input[0, bigint, false] + 1) + 2) + 3))", - "/*project_c3*/" -> "// ((input[0, bigint, false] + 1) + 2)", - "/*project_c2*/" -> "// (input[0, bigint, false] + 1)" + "project_c4" -> "// (((input[0, bigint, false] + 1) + 2) + 3))", + "project_c3" -> "// ((input[0, bigint, false] + 1) + 2)", + "project_c2" -> "// (input[0, bigint, false] + 1)" )) val reducedCode = CodeFormatter.stripOverlappingComments(code) @@ -53,9 +54,11 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("basic example") { - """class A { + """ + |class A { |blahblah; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ class A { @@ -65,11 +68,13 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("nested example") { - """class A { + """ + |class A { | if (c) { |duh; |} - |}""".stripMargin + |} + """.stripMargin } { """ |/* 001 */ class A { @@ -81,9 +86,11 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("single line") { - """class A { + """ + |class A { | if (c) {duh;} - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ class A { @@ -93,9 +100,11 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("if else on the same line") { - """class A { + """ + |class A { | if (c) {duh;} else {boo;} - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ class A { @@ -105,10 +114,12 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("function calls") { - """foo( + """ + |foo( |a, |b, - |c)""".stripMargin + |c) + """.stripMargin }{ """ |/* 001 */ foo( @@ -119,10 +130,12 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("single line comments") { - """// This is a comment about class A { { { ( ( + """ + |// This is a comment about class A { { { ( ( |class A { |class body; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ // This is a comment about class A { { { ( ( @@ -133,10 +146,12 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("single line comments /* */ ") { - """/** This is a comment about class A { { { ( ( */ + """ + |/** This is a comment about class A { { { ( ( */ |class A { |class body; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ /** This is a comment about class A { { { ( ( */ @@ -147,12 +162,14 @@ class CodeFormatterSuite extends SparkFunSuite { } testCase("multi-line comments") { - """ /* This is a comment about + """ + | /* This is a comment about |class A { |class body; ...*/ |class A { |class body; - |}""".stripMargin + |} + """.stripMargin }{ """ |/* 001 */ /* This is a comment about @@ -164,30 +181,56 @@ class CodeFormatterSuite extends SparkFunSuite { """.stripMargin } - // scalastyle:off whitespace.end.of.line testCase("reduce empty lines") { CodeFormatter.stripExtraNewLines( - """class A { + """ + |class A { | | - | /*** comment1 */ + | /* + | * multi + | * line + | * comment + | */ | | class body; | | | if (c) {duh;} | else {boo;} - |}""".stripMargin) + |} + """.stripMargin.trim) }{ """ |/* 001 */ class A { - |/* 002 */ /*** comment1 */ - |/* 003 */ class body; - |/* 004 */ - |/* 005 */ if (c) {duh;} - |/* 006 */ else {boo;} - |/* 007 */ } + |/* 002 */ /* + |/* 003 */ * multi + |/* 004 */ * line + |/* 005 */ * comment + |/* 006 */ */ + |/* 007 */ class body; + |/* 008 */ + |/* 009 */ if (c) {duh;} + |/* 010 */ else {boo;} + |/* 011 */ } + """.stripMargin + } + + testCase("comment place holder")( + """ + |/*c1*/ + |class A + |/*c2*/ + |class B + |/*c1*//*c2*/ + """.stripMargin, Map("c1" -> "/*abc*/", "c2" -> "/*xyz*/") + ) { + """ + |/* 001 */ /*abc*/ + |/* 002 */ class A + |/* 003 */ /*xyz*/ + |/* 004 */ class B + |/* 005 */ /*abc*//*xyz*/ """.stripMargin } - // scalastyle:on whitespace.end.of.line } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 725e14c0fb2af..0a2fb0ef50661 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -581,21 +581,11 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext assert(CatalystReadSupport.expandUDT(schema) === expected) } - test("read/write wide table") { - withTempPath { dir => - val path = dir.getCanonicalPath - - val df = spark.range(1000).select(Seq.tabulate(1000) {i => ('id + i).as(s"c$i")} : _*) - df.write.mode(SaveMode.Overwrite).parquet(path) - checkAnswer(spark.read.parquet(path), df) - } - } - test("returning batch for wide table") { - withSQLConf("spark.sql.codegen.maxFields" -> "100") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "10") { withTempPath { dir => val path = dir.getCanonicalPath - val df = spark.range(100).select(Seq.tabulate(110) {i => ('id + i).as(s"c$i")} : _*) + val df = spark.range(10).select(Seq.tabulate(11) {i => ('id + i).as(s"c$i")} : _*) df.write.mode(SaveMode.Overwrite).parquet(path) // donot return batch, because whole stage codegen is disabled for wide table (>200 columns) @@ -605,7 +595,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext checkAnswer(df2, df) // return batch - val columns = Seq.tabulate(90) {i => s"c$i"} + val columns = Seq.tabulate(9) {i => s"c$i"} val df3 = df2.selectExpr(columns : _*) assert( df3.queryExecution.sparkPlan.find(_.isInstanceOf[BatchedDataSourceScanExec]).isDefined, From 6f22ba3e16c93c83f38ef01cbe511792fe0772ef Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 24 May 2016 22:01:40 -0700 Subject: [PATCH 0374/1470] [SPARK-15508][STREAMING][TESTS] Fix flaky test: JavaKafkaStreamSuite.testKafkaStream ## What changes were proposed in this pull request? `JavaKafkaStreamSuite.testKafkaStream` assumes when `sent.size == result.size`, the contents of `sent` and `result` should be same. However, that's not true. The content of `result` may not be the final content. This PR modified the test to always retry the assertions even if the contents of `sent` and `result` are not same. Here is the failure in Jenkins: http://spark-tests.appspot.com/tests/org.apache.spark.streaming.kafka.JavaKafkaStreamSuite/testKafkaStream ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13281 from zsxwing/flaky-kafka-test. (cherry picked from commit c9c1c0e54d34773ac2cf5457fe5925559ece36c7) Signed-off-by: Shixiong Zhu --- .../streaming/kafka/JavaKafkaStreamSuite.java | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 868df64e8c944..98fe38e826afb 100644 --- a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -122,14 +122,23 @@ public void call(JavaPairRDD rdd) { ssc.start(); long startTime = System.currentTimeMillis(); - boolean sizeMatches = false; - while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { - sizeMatches = sent.size() == result.size(); + AssertionError lastError = null; + while (System.currentTimeMillis() - startTime < 20000) { + try { + Assert.assertEquals(sent.size(), result.size()); + for (Map.Entry e : sent.entrySet()) { + Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); + } + return; + } catch (AssertionError e) { + lastError = e; + } Thread.sleep(200); } - Assert.assertEquals(sent.size(), result.size()); - for (Map.Entry e : sent.entrySet()) { - Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); + if (lastError != null) { + throw lastError; + } else { + Assert.fail("timeout"); } } } From 402995e5de360a630a88c43282a946f0d473b47a Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 24 May 2016 22:20:00 -0700 Subject: [PATCH 0375/1470] [SPARK-15412][PYSPARK][SPARKR][DOCS] Improve linear isotonic regression pydoc & doc build insturctions ## What changes were proposed in this pull request? PySpark: Add links to the predictors from the models in regression.py, improve linear and isotonic pydoc in minor ways. User guide / R: Switch the installed package list to be enough to build the R docs on a "fresh" install on ubuntu and add sudo to match the rest of the commands. User Guide: Add a note about using gem2.0 for systems with both 1.9 and 2.0 (e.g. some ubuntu but maybe more). ## How was this patch tested? built pydocs locally, tested new user build instructions Author: Holden Karau Closes #13199 from holdenk/SPARK-15412-improve-linear-isotonic-regression-pydoc. (cherry picked from commit cd9f16906cabd012b7676eb0f524e68a9cbe4db1) Signed-off-by: Shivaram Venkataraman --- docs/README.md | 4 +++- python/pyspark/ml/regression.py | 30 +++++++++++++++++------------- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/docs/README.md b/docs/README.md index bcea93e1f3b6d..8b515e187379c 100644 --- a/docs/README.md +++ b/docs/README.md @@ -20,8 +20,10 @@ installed. Also install the following libraries: $ sudo pip install Pygments # Following is needed only for generating API docs $ sudo pip install sphinx - $ Rscript -e 'install.packages(c("knitr", "devtools"), repos="http://cran.stat.ucla.edu/")' + $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "roxygen2", "testthat"), repos="http://cran.stat.ucla.edu/")' ``` +(Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0) + ## Generating the Documentation HTML We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 8f58594a66360..1b7af7ef597c2 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -48,11 +48,15 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction The learning objective is to minimize the squared error, with regularization. The specific squared error loss function used is: L = 1/2n ||A coefficients - y||^2^ - This support multiple types of regularization: - - none (a.k.a. ordinary least squares) - - L2 (ridge regression) - - L1 (Lasso) - - L2 + L1 (elastic net) + This supports multiple types of regularization: + + * none (a.k.a. ordinary least squares) + + * L2 (ridge regression) + + * L1 (Lasso) + + * L2 + L1 (elastic net) >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([ @@ -128,7 +132,7 @@ class LinearRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by LinearRegression. + Model fitted by :class:`LinearRegression`. .. versionadded:: 1.4.0 """ @@ -503,13 +507,13 @@ class IsotonicRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by IsotonicRegression. + Model fitted by :class:`IsotonicRegression`. """ @property def boundaries(self): """ - Model boundaries. + Boundaries in increasing order for which predictions are known. """ return self._call_java("boundaries") @@ -769,7 +773,7 @@ class DecisionTreeRegressionModel(DecisionTreeModel, JavaMLWritable, JavaMLReada """ .. note:: Experimental - Model fitted by DecisionTreeRegressor. + Model fitted by :class:`DecisionTreeRegressor`. .. versionadded:: 1.4.0 """ @@ -887,7 +891,7 @@ class RandomForestRegressionModel(TreeEnsembleModels, JavaMLWritable, JavaMLRead """ .. note:: Experimental - Model fitted by RandomForestRegressor. + Model fitted by :class:`RandomForestRegressor`. .. versionadded:: 1.4.0 """ @@ -1021,7 +1025,7 @@ class GBTRegressionModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by GBTRegressor. + Model fitted by :class:`GBTRegressor`. .. versionadded:: 1.4.0 """ @@ -1190,7 +1194,7 @@ class AFTSurvivalRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ .. note:: Experimental - Model fitted by AFTSurvivalRegression. + Model fitted by :class:`AFTSurvivalRegression`. .. versionadded:: 1.6.0 """ @@ -1380,7 +1384,7 @@ class GeneralizedLinearRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable """ .. note:: Experimental - Model fitted by GeneralizedLinearRegression. + Model fitted by :class:`GeneralizedLinearRegression`. .. versionadded:: 2.0.0 """ From 1dad1a8913a62eb17f0208c72bd336bba5149452 Mon Sep 17 00:00:00 2001 From: Krishna Kalyan Date: Tue, 24 May 2016 22:21:52 -0700 Subject: [PATCH 0376/1470] [SPARK-12071][DOC] Document the behaviour of NA in R ## What changes were proposed in this pull request? Under Upgrading From SparkR 1.5.x to 1.6.x section added the information, SparkSQL converts `NA` in R to `null`. ## How was this patch tested? Document update, no tests. Author: Krishna Kalyan Closes #13268 from krishnakalyan3/spark-12071-1. (cherry picked from commit 9082b7968ad952e05fc6f4feb499febef6aa45a7) Signed-off-by: Shivaram Venkataraman --- docs/sparkr.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sparkr.md b/docs/sparkr.md index 9b5eaa1ec7232..6b2ca6d6a55f4 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -386,6 +386,7 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma ## Upgrading From SparkR 1.5.x to 1.6.x - Before Spark 1.6.0, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. + - SparkSQL converts `NA` in R to `null` and vice-versa. ## Upgrading From SparkR 1.6.x to 2.0 From f63ba2210a27df1843f5007d367036dae0e0139f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 May 2016 19:17:53 +0200 Subject: [PATCH 0377/1470] [SPARK-15436][SQL] Remove DescribeFunction and ShowFunctions ## What changes were proposed in this pull request? This patch removes the last two commands defined in the catalyst module: DescribeFunction and ShowFunctions. They were unnecessary since the parser could just generate DescribeFunctionCommand and ShowFunctionsCommand directly. ## How was this patch tested? Created a new SparkSqlParserSuite. Author: Reynold Xin Closes #13292 from rxin/SPARK-15436. (cherry picked from commit 4f27b8dd58a66fca7ddd4c239e02b90c34b1cebd) Signed-off-by: Herman van Hovell --- .../sql/catalyst/parser/AstBuilder.scala | 32 +-------- .../sql/catalyst/plans/logical/Command.scala | 25 +++++++ .../sql/catalyst/plans/logical/commands.scala | 55 --------------- .../analysis/UnsupportedOperationsSuite.scala | 9 ++- .../sql/catalyst/parser/PlanParserSuite.scala | 30 +++----- .../spark/sql/execution/SparkSqlParser.scala | 33 ++++++++- .../spark/sql/execution/SparkStrategies.scala | 6 -- .../sql/execution/SparkSqlParserSuite.scala | 68 +++++++++++++++++++ .../execution/command/DDLCommandSuite.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 6 +- 10 files changed, 145 insertions(+), 121 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index a13c03a529f37..3473feec3209c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} @@ -81,37 +82,6 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * ******************************************************************************************** */ protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree) - /** - * Create a plan for a SHOW FUNCTIONS command. - */ - override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { - import ctx._ - if (qualifiedName != null) { - val name = visitFunctionName(qualifiedName) - ShowFunctions(name.database, Some(name.funcName)) - } else if (pattern != null) { - ShowFunctions(None, Some(string(pattern))) - } else { - ShowFunctions(None, None) - } - } - - /** - * Create a plan for a DESCRIBE FUNCTION command. - */ - override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) { - import ctx._ - val functionName = - if (describeFuncName.STRING() != null) { - FunctionIdentifier(string(describeFuncName.STRING()), database = None) - } else if (describeFuncName.qualifiedName() != null) { - visitFunctionName(describeFuncName.qualifiedName) - } else { - FunctionIdentifier(describeFuncName.getText, database = None) - } - DescribeFunction(functionName, EXTENDED != null) - } - /** * Create a top-level plan with Common Table Expressions. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala new file mode 100644 index 0000000000000..75a5b10d9ed04 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical + +/** + * A logical node that represents a non-query command to be executed by the system. For example, + * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are + * eagerly executed. + */ +trait Command diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala deleted file mode 100644 index 0ec3ff3c25a00..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.plans.logical - -import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.types.StringType - -/** - * A logical node that represents a non-query command to be executed by the system. For example, - * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are - * eagerly executed. - */ -trait Command - -/** - * Returned for the "DESCRIBE FUNCTION [EXTENDED] functionName" command. - * - * @param functionName The function to be described. - * @param isExtended True if "DESCRIBE FUNCTION EXTENDED" is used. Otherwise, false. - */ -private[sql] case class DescribeFunction( - functionName: FunctionIdentifier, - isExtended: Boolean) extends LogicalPlan with Command { - - override def children: Seq[LogicalPlan] = Seq.empty - override val output: Seq[Attribute] = Seq( - AttributeReference("function_desc", StringType, nullable = false)()) -} - -/** - * Returned for the "SHOW FUNCTIONS" command, which will list all of the - * registered function list. - */ -private[sql] case class ShowFunctions( - db: Option[String], pattern: Option[String]) extends LogicalPlan with Command { - override def children: Seq[LogicalPlan] = Seq.empty - override val output: Seq[Attribute] = Seq( - AttributeReference("function", StringType, nullable = false)()) -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 674277bdbe15d..aaeee0f2a41c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -29,6 +28,12 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.IntegerType +/** A dummy command for testing unsupported operations. */ +case class DummyCommand() extends LogicalPlan with Command { + override def output: Seq[Attribute] = Nil + override def children: Seq[LogicalPlan] = Nil +} + class UnsupportedOperationsSuite extends SparkFunSuite { val attribute = AttributeReference("a", IntegerType, nullable = true)() @@ -70,7 +75,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite { // Commands assertNotSupportedInStreamingPlan( "commmands", - DescribeFunction(FunctionIdentifier("func", database = None), true), + DummyCommand(), outputMode = Append, expectedMsgs = "commands" :: Nil) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 5811d32cd9e62..a6fad2d8a0398 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.catalyst.parser import org.apache.spark.sql.Row @@ -24,17 +25,21 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.IntegerType - +/** + * Parser test cases for rules defined in [[CatalystSqlParser]] / [[AstBuilder]]. + * + * There is also SparkSqlParserSuite in sql/core module for parser rules defined in sql/core module. + */ class PlanParserSuite extends PlanTest { import CatalystSqlParser._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ - def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { + private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { comparePlans(parsePlan(sqlCommand), plan) } - def intercept(sqlCommand: String, messages: String*): Unit = { + private def intercept(sqlCommand: String, messages: String*): Unit = { val e = intercept[ParseException](parsePlan(sqlCommand)) messages.foreach { message => assert(e.message.contains(message)) @@ -53,25 +58,6 @@ class PlanParserSuite extends PlanTest { intercept("EXPLAIN formatted SELECT 1", "Unsupported SQL statement") } - test("show functions") { - assertEqual("show functions", ShowFunctions(None, None)) - assertEqual("show functions foo", ShowFunctions(None, Some("foo"))) - assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar"))) - assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*"))) - intercept("show functions foo.bar.baz", "Unsupported function name") - } - - test("describe function") { - assertEqual("describe function bar", - DescribeFunction(FunctionIdentifier("bar", database = None), isExtended = false)) - assertEqual("describe function extended bar", - DescribeFunction(FunctionIdentifier("bar", database = None), isExtended = true)) - assertEqual("describe function foo.bar", - DescribeFunction(FunctionIdentifier("bar", database = Option("foo")), isExtended = false)) - assertEqual("describe function extended f.bar", - DescribeFunction(FunctionIdentifier("bar", database = Option("f")), isExtended = true)) - } - test("set operations") { val a = table("a").select(star()) val b = table("b").select(star()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index f85d6062e8d35..57f534cd9eb30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -24,7 +24,7 @@ import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ @@ -493,6 +493,37 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { DescribeDatabaseCommand(ctx.identifier.getText, ctx.EXTENDED != null) } + /** + * Create a plan for a DESCRIBE FUNCTION command. + */ + override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) { + import ctx._ + val functionName = + if (describeFuncName.STRING() != null) { + FunctionIdentifier(string(describeFuncName.STRING()), database = None) + } else if (describeFuncName.qualifiedName() != null) { + visitFunctionName(describeFuncName.qualifiedName) + } else { + FunctionIdentifier(describeFuncName.getText, database = None) + } + DescribeFunctionCommand(functionName, EXTENDED != null) + } + + /** + * Create a plan for a SHOW FUNCTIONS command. + */ + override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { + import ctx._ + if (qualifiedName != null) { + val name = visitFunctionName(qualifiedName) + ShowFunctionsCommand(name.database, Some(name.funcName)) + } else if (pattern != null) { + ShowFunctionsCommand(None, Some(string(pattern))) + } else { + ShowFunctionsCommand(None, None) + } + } + /** * Create a [[CreateFunctionCommand]] command. * 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 c46cecc71f37e..e40525287a0a1 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 @@ -416,12 +416,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { c.child) ExecutedCommandExec(cmd) :: Nil - case logical.ShowFunctions(db, pattern) => - ExecutedCommandExec(ShowFunctionsCommand(db, pattern)) :: Nil - - case logical.DescribeFunction(function, extended) => - ExecutedCommandExec(DescribeFunctionCommand(function, extended)) :: Nil - case _ => Nil } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala new file mode 100644 index 0000000000000..e2858bb475401 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.command.{DescribeFunctionCommand, ShowFunctionsCommand} +import org.apache.spark.sql.internal.SQLConf + +/** + * Parser test cases for rules defined in [[SparkSqlParser]]. + * + * See [[org.apache.spark.sql.catalyst.parser.PlanParserSuite]] for rules + * defined in the Catalyst module. + */ +class SparkSqlParserSuite extends PlanTest { + + private lazy val parser = new SparkSqlParser(new SQLConf) + + private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { + comparePlans(parser.parsePlan(sqlCommand), plan) + } + + private def intercept(sqlCommand: String, messages: String*): Unit = { + val e = intercept[ParseException](parser.parsePlan(sqlCommand)) + messages.foreach { message => + assert(e.message.contains(message)) + } + } + + test("show functions") { + assertEqual("show functions", ShowFunctionsCommand(None, None)) + assertEqual("show functions foo", ShowFunctionsCommand(None, Some("foo"))) + assertEqual("show functions foo.bar", ShowFunctionsCommand(Some("foo"), Some("bar"))) + assertEqual("show functions 'foo\\\\.*'", ShowFunctionsCommand(None, Some("foo\\.*"))) + intercept("show functions foo.bar.baz", "Unsupported function name") + } + + test("describe function") { + assertEqual("describe function bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = false)) + assertEqual("describe function extended bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = true)) + assertEqual("describe function foo.bar", + DescribeFunctionCommand( + FunctionIdentifier("bar", database = Option("foo")), isExtended = false)) + assertEqual("describe function extended f.bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = Option("f")), isExtended = true)) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index eab1f55712f6f..850fca585216d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructType} // TODO: merge this with DDLSuite (SPARK-14441) class DDLCommandSuite extends PlanTest { - private val parser = new SparkSqlParser(new SQLConf) + private lazy val parser = new SparkSqlParser(new SQLConf) private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { val e = intercept[ParseException] { 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 b12f3aafefb8b..65d53debd67cf 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.SQLBuilder import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExplainCommand, SetCommand, ShowColumnsCommand} +import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable} import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} @@ -414,8 +414,8 @@ abstract class HiveComparisonTest // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && - (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && - (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive From 69327667d5a14b12de8055d752fbe3abb8d6671c Mon Sep 17 00:00:00 2001 From: Lukasz Date: Wed, 25 May 2016 10:24:21 -0700 Subject: [PATCH 0378/1470] [SPARK-9044] Fix "Storage" tab in UI so that it reflects RDD name change. ## What changes were proposed in this pull request? 1. Making 'name' field of RDDInfo mutable. 2. In StorageListener: catching the fact that RDD's name was changed and updating it in RDDInfo. ## How was this patch tested? 1. Manual verification - the 'Storage' tab now behaves as expected. 2. The commit also contains a new unit test which verifies this. Author: Lukasz Closes #13264 from lgieron/SPARK-9044. (cherry picked from commit b120fba6ae26186b3fa0dfbb1637046f4e76c2b0) Signed-off-by: Shixiong Zhu --- .../org/apache/spark/storage/RDDInfo.scala | 2 +- .../apache/spark/ui/storage/StorageTab.scala | 2 +- .../spark/ui/storage/StorageTabSuite.scala | 17 +++++++++++++++++ 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 083d78b59ebee..e5abbf745cc41 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.Utils @DeveloperApi class RDDInfo( val id: Int, - val name: String, + var name: String, val numPartitions: Int, var storageLevel: StorageLevel, val parentIds: Seq[Int], diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 50095831b4a53..c212362557be6 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -59,7 +59,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val rddInfos = stageSubmitted.stageInfo.rddInfos - rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } + rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info).name = info.name } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { 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 7d77deeb60618..411a0ddebeb77 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 @@ -179,6 +179,23 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { assert(storageListener.rddInfoList.size === 2) } + test("verify StorageTab still contains a renamed RDD") { + val rddInfo = new RDDInfo(0, "original_name", 1, memOnly, Seq(4)) + val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo), Seq.empty, "details") + bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + val blockUpdateInfos1 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(0, 1), memOnly, 100L, 0L)) + postUpdateBlocks(bus, blockUpdateInfos1) + assert(storageListener.rddInfoList.size == 1) + + val newName = "new_name" + val rddInfoRenamed = new RDDInfo(0, newName, 1, memOnly, Seq(4)) + val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfoRenamed), Seq.empty, "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) + assert(storageListener.rddInfoList.size == 1) + assert(storageListener.rddInfoList.head.name == newName) + } + private def postUpdateBlocks( bus: SparkListenerBus, blockUpdateInfos: Seq[BlockUpdatedInfo]): Unit = { blockUpdateInfos.foreach { blockUpdateInfo => From 27f26a39db021735bcf75a1f1b89b9481b199341 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Wed, 25 May 2016 10:46:51 -0700 Subject: [PATCH 0379/1470] [SPARK-15345][SQL][PYSPARK] SparkSession's conf doesn't take effect when this already an existing SparkContext ## What changes were proposed in this pull request? Override the existing SparkContext is the provided SparkConf is different. PySpark part hasn't been fixed yet, will do that after the first round of review to ensure this is the correct approach. ## How was this patch tested? Manually verify it in spark-shell. rxin Please help review it, I think this is a very critical issue for spark 2.0 Author: Jeff Zhang Closes #13160 from zjffdu/SPARK-15345. (cherry picked from commit 01e7b9c85bb84924e279021f9748774dce9702c8) Signed-off-by: Andrew Or --- .../main/scala/org/apache/spark/SparkContext.scala | 3 +++ .../scala/org/apache/spark/sql/SparkSession.scala | 14 ++++++++++++-- .../spark/sql/SparkSessionBuilderSuite.scala | 14 +++++++++++++- 3 files changed, 28 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 36aa3becb4fc3..5018eb38d91c0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2254,6 +2254,9 @@ object SparkContext extends Logging { if (activeContext.get() == null) { setActiveContext(new SparkContext(config), allowMultipleContexts = false) } + if (config.getAll.nonEmpty) { + logWarning("Use an existing SparkContext, some configuration may not take effect.") + } activeContext.get() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5c87c844185c5..86c97b92dfc3d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -636,7 +636,7 @@ object SparkSession { /** * Builder for [[SparkSession]]. */ - class Builder { + class Builder extends Logging { private[this] val options = new scala.collection.mutable.HashMap[String, String] @@ -753,6 +753,9 @@ object SparkSession { var session = activeThreadSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } + if (options.nonEmpty) { + logWarning("Use an existing SparkSession, some configuration may not take effect.") + } return session } @@ -762,6 +765,9 @@ object SparkSession { session = defaultSession.get() if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } + if (options.nonEmpty) { + logWarning("Use an existing SparkSession, some configuration may not take effect.") + } return session } @@ -774,7 +780,11 @@ object SparkSession { val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } - SparkContext.getOrCreate(sparkConf) + val sc = SparkContext.getOrCreate(sparkConf) + // maybe this is an existing SparkContext, update its SparkConf which maybe used + // by SparkSession + options.foreach { case (k, v) => sc.conf.set(k, v) } + sc } session = new SparkSession(sparkContext) options.foreach { case (k, v) => session.conf.set(k, v) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index ec6a2b3575869..786956df8a555 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} /** * Test cases for the builder pattern of [[SparkSession]]. @@ -90,4 +90,16 @@ class SparkSessionBuilderSuite extends SparkFunSuite { assert(newSession != activeSession) newSession.stop() } + + test("create SparkContext first then SparkSession") { + sparkContext.stop() + val conf = new SparkConf().setAppName("test").setMaster("local").set("key1", "value1") + val sparkContext2 = new SparkContext(conf) + val session = SparkSession.builder().config("key2", "value2").getOrCreate() + assert(session.conf.get("key1") == "value1") + assert(session.conf.get("key2") == "value2") + assert(session.sparkContext.conf.get("key1") == "value1") + assert(session.sparkContext.conf.get("key2") == "value2") + session.stop() + } } From c75ec5eaa6c95b3647c80b6595902d16ab3165fa Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 25 May 2016 10:49:11 -0700 Subject: [PATCH 0380/1470] [SPARK-15520][SQL] SparkSession builder in python should also allow overriding confs of existing sessions ## What changes were proposed in this pull request? This fixes the python SparkSession builder to allow setting confs correctly. This was a leftover TODO from https://github.com/apache/spark/pull/13200. ## How was this patch tested? Python doc tests. cc andrewor14 Author: Eric Liang Closes #13289 from ericl/spark-15520. (cherry picked from commit 8239fdcb9b54ab6d13c31ad9916b8334dd1462c2) Signed-off-by: Andrew Or --- python/pyspark/sql/session.py | 35 ++++++++++++++++++++++++----------- 1 file changed, 24 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 241947537fa29..52e7f3d348ab9 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -138,24 +138,37 @@ def getOrCreate(self): """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a new one based on the options set in this builder. - This method first checks whether there is a valid thread-local SparkSession, - and if yes, return that one. It then checks whether there is a valid global - default SparkSession, and if yes, return that one. If no valid global default - SparkSession exists, the method creates a new SparkSession and assigns the - newly created SparkSession as the global default. + This method first checks whether there is a valid global default SparkSession, and if + yes, return that one. If no valid global default SparkSession exists, the method + creates a new SparkSession and assigns the newly created SparkSession as the global + default. + + >>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() + >>> s1.conf.get("k1") == "v1" + True In case an existing SparkSession is returned, the config options specified in this builder will be applied to the existing SparkSession. + + >>> s2 = SparkSession.builder.config("k2", "v2").getOrCreate() + >>> s1.conf.get("k1") == s2.conf.get("k1") + True + >>> s1.conf.get("k2") == s2.conf.get("k2") + True """ with self._lock: - from pyspark.conf import SparkConf from pyspark.context import SparkContext - from pyspark.sql.context import SQLContext - sparkConf = SparkConf() + from pyspark.conf import SparkConf + session = SparkSession._instantiatedContext + if session is None: + sparkConf = SparkConf() + for key, value in self._options.items(): + sparkConf.set(key, value) + sc = SparkContext.getOrCreate(sparkConf) + session = SparkSession(sc) for key, value in self._options.items(): - sparkConf.set(key, value) - sparkContext = SparkContext.getOrCreate(sparkConf) - return SQLContext.getOrCreate(sparkContext).sparkSession + session.conf.set(key, value) + return session builder = Builder() From 4009ddafd810f91f699e52d7822c8c959fe7761e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 25 May 2016 10:51:33 -0700 Subject: [PATCH 0381/1470] [MINOR][CORE] Fix a HadoopRDD log message and remove unused imports in rdd files. ## What changes were proposed in this pull request? This PR fixes the following typos in log message and comments of `HadoopRDD.scala`. Also, this removes unused imports. ```scala - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + ... - // since its not removed yet + // since it's not removed yet ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13294 from dongjoon-hyun/minor_rdd_fix_log_message. (cherry picked from commit d6d3e50719b01005aa0e77349fc9a6ff88fecce3) Signed-off-by: Andrew Or --- core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 5 ++--- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 - .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala | 1 - 5 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index be0cb175f5340..41832e8354741 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.hadoop.conf.{ Configurable, Configuration } +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.JobContextImpl diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index b22134af45b30..515fd6f4e278c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -43,7 +43,6 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.Logging import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} @@ -70,7 +69,7 @@ private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: Inp val envVars: Map[String, String] = if (inputSplit.value.isInstanceOf[FileSplit]) { val is: FileSplit = inputSplit.value.asInstanceOf[FileSplit] // map_input_file is deprecated in favor of mapreduce_map_input_file but set both - // since its not removed yet + // since it's not removed yet Map("map_input_file" -> is.getPath().toString(), "mapreduce_map_input_file" -> is.getPath().toString()) } else { @@ -335,7 +334,7 @@ class HadoopRDD[K, V]( override def persist(storageLevel: StorageLevel): this.type = { if (storageLevel.deserialized) { - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + " Use a map transformation to make copies of the records.") } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index ad7c2216a042f..189dc7b331337 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.task.{JobContextImpl, TaskAttemptContextImpl} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod import org.apache.spark.internal.Logging import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.storage.StorageLevel diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 3b12448d63933..a7142376017d5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -40,7 +40,7 @@ import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} +import org.apache.spark.executor.OutputMetrics import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 45616856fd7a5..49625b7042d94 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -31,7 +31,6 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.util.control.NonFatal import org.apache.spark.{Partition, SparkEnv, TaskContext} import org.apache.spark.util.Utils From 6fc367e50c2de9f68237763919eae12241a2179f Mon Sep 17 00:00:00 2001 From: lfzCarlosC Date: Wed, 25 May 2016 10:53:53 -0700 Subject: [PATCH 0382/1470] [MINOR][MLLIB][STREAMING][SQL] Fix typos fixed typos for source code for components [mllib] [streaming] and [SQL] None and obvious. Author: lfzCarlosC Closes #13298 from lfzCarlosC/master. (cherry picked from commit 02c8072eea72425e89256347e1f373a3e76e6eba) Signed-off-by: Andrew Or --- mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala | 2 +- .../apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala | 2 +- .../sql/catalyst/plans/logical/basicLogicalOperators.scala | 2 +- .../apache/spark/sql/execution/vectorized/ColumnVector.java | 2 +- .../spark/sql/execution/streaming/state/StateStoreSuite.scala | 2 +- sql/hive-thriftserver/if/TCLIService.thrift | 4 ++-- .../org/apache/hive/service/ServiceStateChangeListener.java | 2 +- .../org/apache/hive/service/cli/operation/SQLOperation.java | 2 +- .../hive/service/cli/session/HiveSessionHookContext.java | 2 +- .../scala/org/apache/spark/sql/hive/HiveSessionState.scala | 2 +- .../spark/streaming/util/WriteAheadLogRecordHandle.java | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index 9457c6e9e35f2..bb4b37ef21a84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -204,7 +204,7 @@ private object IDFModel { * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector * * @param idf an IDF vector - * @param v a term frequence vector + * @param v a term frequency vector * @return a TF-IDF vector */ def transform(idf: Vector, v: Vector): Vector = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala index 9748fbf2c97b6..c3de5d75f4f7d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala @@ -45,7 +45,7 @@ import org.apache.spark.rdd.RDD * many elements are in each partition. Once these three values have been returned for every * partition, we can collect and operate locally. Locally, we can now adjust each distance by the * appropriate constant (the cumulative sum of number of elements in the prior partitions divided by - * thedata set size). Finally, we take the maximum absolute value, and this is the statistic. + * the data set size). Finally, we take the maximum absolute value, and this is the statistic. */ private[stat] object KolmogorovSmirnovTest extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 0a9250b71fb61..8b7e21b67982d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -591,7 +591,7 @@ case class Expand( } // This operator can reuse attributes (for example making them null when doing a roll up) so - // the contraints of the child may no longer be valid. + // the constraints of the child may no longer be valid. override protected def validConstraints: Set[Expression] = Set.empty[Expression] } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index e7dccd1b2261f..3f94255256699 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -504,7 +504,7 @@ public ColumnarBatch.Row getStruct(int rowId) { /** * Returns a utility object to get structs. - * provided to keep API compabilitity with InternalRow for code generation + * provided to keep API compatibility with InternalRow for code generation */ public ColumnarBatch.Row getStruct(int rowId, int size) { resultStruct.rowId = rowId; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index f8f8bc7d6ff84..984b84fd13fbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -188,7 +188,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth provider.getStore(-1) } - // Prepare some data in the stoer + // Prepare some data in the store val store = provider.getStore(0) put(store, "a", 1) assert(store.commit() === 1) diff --git a/sql/hive-thriftserver/if/TCLIService.thrift b/sql/hive-thriftserver/if/TCLIService.thrift index baf583fb3ecd4..7cd6fa37cec37 100644 --- a/sql/hive-thriftserver/if/TCLIService.thrift +++ b/sql/hive-thriftserver/if/TCLIService.thrift @@ -661,7 +661,7 @@ union TGetInfoValue { // The function returns general information about the data source // using the same keys as ODBC. struct TGetInfoReq { - // The sesssion to run this request against + // The session to run this request against 1: required TSessionHandle sessionHandle 2: required TGetInfoType infoType @@ -1032,7 +1032,7 @@ enum TFetchOrientation { FETCH_PRIOR, // Return the rowset at the given fetch offset relative - // to the curren rowset. + // to the current rowset. // NOT SUPPORTED FETCH_RELATIVE, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java index 16ad9a991e921..d1aadad04cf67 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java @@ -29,7 +29,7 @@ public interface ServiceStateChangeListener { * have changed state before this callback is invoked. * * This operation is invoked on the thread that initiated the state change, - * while the service itself in in a sychronized section. + * while the service itself in in a synchronized section. *
      *
    1. Any long-lived operation here will prevent the service state * change from completing in a timely manner.
    2. diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java index a35405484a765..5014cedd870b6 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -456,7 +456,7 @@ private SerDe getSerDe() throws SQLException { private HiveConf getConfigForOperation() throws HiveSQLException { HiveConf sqlOperationConf = getParentSession().getHiveConf(); if (!getConfOverlay().isEmpty() || shouldRunAsync()) { - // clone the partent session config for this query + // clone the parent session config for this query sqlOperationConf = new HiveConf(sqlOperationConf); // apply overlay query specific settings, if any diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java index 0a10dba8b479d..c56a107d42466 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java @@ -22,7 +22,7 @@ /** * HiveSessionHookContext. * Interface passed to the HiveServer2 session hook execution. This enables - * the hook implementation to accesss session config, user and session handle + * the hook implementation to access session config, user and session handle */ public interface HiveSessionHookContext { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 46579ecd85caa..081d85acb9084 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -143,7 +143,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) * converted to a data source table, using the data source set by spark.sql.sources.default. * The table in CTAS statement will be converted when it meets any of the following conditions: * - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or - * a Storage Hanlder (STORED BY), and the value of hive.default.fileformat in hive-site.xml + * a Storage Handler (STORED BY), and the value of hive.default.fileformat in hive-site.xml * is either TextFile or SequenceFile. * - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe * is specified (no ROW FORMAT SERDE clause). diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java index 662889e779fb2..3c5cc7e2cae1b 100644 --- a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java @@ -23,7 +23,7 @@ * This abstract class represents a handle that refers to a record written in a * {@link org.apache.spark.streaming.util.WriteAheadLog WriteAheadLog}. * It must contain all the information necessary for the record to be read and returned by - * an implemenation of the WriteAheadLog class. + * an implementation of the WriteAheadLog class. * * @see org.apache.spark.streaming.util.WriteAheadLog */ From 409eb28f7219778d2aba36079c20c84c8d31b604 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 25 May 2016 20:41:53 +0200 Subject: [PATCH 0383/1470] [SPARK-15500][DOC][ML][PYSPARK] Remove default value in Param doc field in ALS Remove "Default: MEMORY_AND_DISK" from `Param` doc field in ALS storage level params. This fixes up the output of `explainParam(s)` so that default values are not displayed twice. We can revisit in the case that [SPARK-15130](https://issues.apache.org/jira/browse/SPARK-15130) moves ahead with adding defaults in some way to PySpark param doc fields. Tests N/A. Author: Nick Pentreath Closes #13277 from MLnick/SPARK-15500-als-remove-default-storage-param. (cherry picked from commit 1cb347fbc446092b478ae0578fc7d1b0626a9294) Signed-off-by: Nick Pentreath --- .../main/scala/org/apache/spark/ml/recommendation/ALS.scala | 4 ++-- python/pyspark/ml/recommendation.py | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) 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 f257382d2205c..8dc7437d4747c 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 @@ -180,7 +180,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w * @group expertParam */ val intermediateStorageLevel = new Param[String](this, "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'. Default: 'MEMORY_AND_DISK'.", + "StorageLevel for intermediate datasets. Cannot be 'NONE'.", (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE") /** @group expertGetParam */ @@ -194,7 +194,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w * @group expertParam */ val finalStorageLevel = new Param[String](this, "finalStorageLevel", - "StorageLevel for ALS model factors. Default: 'MEMORY_AND_DISK'.", + "StorageLevel for ALS model factors.", (s: String) => Try(StorageLevel.fromString(s)).isSuccess) /** @group expertGetParam */ diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index bac2a3029fd71..1778bfe938b1e 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -120,12 +120,10 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha "whether to use nonnegative constraint for least squares", typeConverter=TypeConverters.toBoolean) intermediateStorageLevel = Param(Params._dummy(), "intermediateStorageLevel", - "StorageLevel for intermediate datasets. Cannot be 'NONE'. " + - "Default: 'MEMORY_AND_DISK'.", + "StorageLevel for intermediate datasets. Cannot be 'NONE'.", typeConverter=TypeConverters.toString) finalStorageLevel = Param(Params._dummy(), "finalStorageLevel", - "StorageLevel for ALS model factors. " + - "Default: 'MEMORY_AND_DISK'.", + "StorageLevel for ALS model factors.", typeConverter=TypeConverters.toString) @keyword_only From 20cc2eb1b8bf9fbeda9d00a6d02da8f624973742 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 25 May 2016 12:02:07 -0700 Subject: [PATCH 0384/1470] [SPARK-15483][SQL] IncrementalExecution should use extra strategies. ## What changes were proposed in this pull request? Extra strategies does not work for streams because `IncrementalExecution` uses modified planner with stateful operations but it does not include extra strategies. This pr fixes `IncrementalExecution` to include extra strategies to use them. ## How was this patch tested? I added a test to check if extra strategies work for streams. Author: Takuya UESHIN Closes #13261 from ueshin/issues/SPARK-15483. (cherry picked from commit 4b88067416ce922ae15a1445cf953fb9b5c43427) Signed-off-by: Michael Armbrust --- .../streaming/IncrementalExecution.scala | 3 ++- .../apache/spark/sql/streaming/StreamSuite.scala | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 8b96f65bc31ac..fe5f36e1cdeeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -36,7 +36,8 @@ class IncrementalExecution private[sql]( extends QueryExecution(sparkSession, logicalPlan) { // TODO: make this always part of planning. - val stateStrategy = sparkSession.sessionState.planner.StatefulAggregationStrategy :: Nil + val stateStrategy = sparkSession.sessionState.planner.StatefulAggregationStrategy +: + sparkSession.sessionState.experimentalMethods.extraStrategies // Modified planner with stateful operations. override def planner: SparkPlanner = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index b742206b58137..ae89a6887a6d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -220,6 +220,21 @@ class StreamSuite extends StreamTest with SharedSQLContext { CheckOffsetLogLatestBatchId(2), CheckSinkLatestBatchId(2)) } + + test("insert an extraStrategy") { + try { + spark.experimental.extraStrategies = TestStrategy :: Nil + + val inputData = MemoryStream[(String, Int)] + val df = inputData.toDS().map(_._1).toDF("a") + + testStream(df)( + AddData(inputData, ("so slow", 1)), + CheckAnswer("so fast")) + } finally { + spark.experimental.extraStrategies = Nil + } + } } /** From 8629537cc7d89c97ef2038326f1138d166596315 Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Wed, 25 May 2016 12:40:16 -0700 Subject: [PATCH 0385/1470] [SPARK-15493][SQL] default QuoteEscapingEnabled flag to true when writing CSV ## What changes were proposed in this pull request? Default QuoteEscapingEnabled flag to true when writing CSV and add an escapeQuotes option to be able to change this. See https://github.com/uniVocity/univocity-parsers/blob/f3eb2af26374940e60d91d1703bde54619f50c51/src/main/java/com/univocity/parsers/csv/CsvWriterSettings.java#L231-L247 This change is needed to be able to write RFC 4180 compatible CSV files (https://tools.ietf.org/html/rfc4180#section-2) https://issues.apache.org/jira/browse/SPARK-15493 ## How was this patch tested? Added a test that verifies the output is quoted correctly. Author: Jurriaan Pruis Closes #13267 from jurriaan/quote-escaping. (cherry picked from commit c875d81a3de3f209b9eb03adf96b7c740b2c7b52) Signed-off-by: Reynold Xin --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/pyspark/sql/readwriter.py | 7 ++- sql/core/pom.xml | 2 +- .../apache/spark/sql/DataFrameWriter.scala | 3 ++ .../datasources/csv/CSVOptions.scala | 2 + .../execution/datasources/csv/CSVParser.scala | 1 + .../execution/datasources/csv/CSVSuite.scala | 51 +++++++++++++++++++ 11 files changed, 69 insertions(+), 7 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index a9068da8b2096..0ac1c00154838 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -159,7 +159,7 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 7e60a313ae8fe..fa35fa70516f3 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -167,7 +167,7 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 70d33b4f4812c..99dffa93bbcf7 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -167,7 +167,7 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index a80f6bc2a4061..a3bee36ce535a 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -175,7 +175,7 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index c0b53f73cd499..dbd7a8e0bf21c 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -176,7 +176,7 @@ stax-api-1.0.1.jar stream-2.7.0.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.1.0.jar +univocity-parsers-2.1.1.jar validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 6f788cf50c7d7..73d2b81b6bd04 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -769,7 +769,7 @@ def text(self, path, compression=None): @since(2.0) def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, - header=None, nullValue=None): + header=None, nullValue=None, escapeQuotes=None): """Saves the content of the [[DataFrame]] in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -790,6 +790,9 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No value, ``"``. :param escape: sets the single character used for escaping quotes inside an already quoted value. If None is set, it uses the default value, ``\`` + :param escapeQuotes: A flag indicating whether values containing quotes should always + be enclosed in quotes. If None is set, it uses the default value + ``true``, escaping all values containing a quote character. :param header: writes the names of columns as the first line. If None is set, it uses the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses @@ -810,6 +813,8 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No self.option("header", header) if nullValue is not None: self.option("nullValue", nullValue) + if escapeQuotes is not None: + self.option("escapeQuotes", nullValue) self._jwrite.csv(path) @since(1.5) diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 2ea980bf20dfe..b833b9369ec64 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -39,7 +39,7 @@ com.univocity univocity-parsers - 2.1.0 + 2.1.1 jar diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 6f5fb69ea3778..3aacce7d7f383 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -684,6 +684,9 @@ final class DataFrameWriter private[sql](df: DataFrame) { * the separator can be part of the value. *
    3. `escape` (default `\`): sets the single character used for escaping quotes inside * an already quoted value.
    4. + *
    5. `escapeQuotes` (default `true`): a flag indicating whether values containing + * quotes should always be enclosed in quotes. Default is to escape all values containing + * a quote character.
    6. *
    7. `header` (default `false`): writes the names of columns as the first line.
    8. *
    9. `nullValue` (default empty string): sets the string representation of a null value.
    10. *
    11. `compression` (default `null`): compression codec to use when saving to file. This can be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index e4fd09462fabd..9f4ce8358b045 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -111,6 +111,8 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str val maxCharsPerColumn = getInt("maxCharsPerColumn", 1000000) + val escapeQuotes = getBool("escapeQuotes", true) + val inputBufferSize = 128 val isCommentSet = this.comment != '\u0000' diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index 111995da9cbad..b06f12369dd0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -75,6 +75,7 @@ private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten writerSettings.setSkipEmptyLines(true) writerSettings.setQuoteAllFields(false) writerSettings.setHeaders(headers: _*) + writerSettings.setQuoteEscapingEnabled(params.escapeQuotes) private var buffer = new ByteArrayOutputStream() private var writer = new CsvWriter( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index bae290776f6e1..ad7c05c12eb9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -366,6 +366,57 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } + test("save csv with quote escaping enabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escape", "\"") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "\"test \"\"quote\"\"\",123,\"it \"\"works\"\"!\",\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + + test("save csv with quote escaping disabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escapeQuotes", "false") + .option("escape", "\"") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "test \"quote\",123,it \"works\"!,\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + test("commented lines in CSV data") { val results = spark.read .format("csv") From f7158c4828320e18aedd4369832da16c759b90fb Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Wed, 25 May 2016 14:29:14 -0700 Subject: [PATCH 0386/1470] [MINOR] [PYSPARK] [EXAMPLES] Changed examples to use SparkSession.sparkContext instead of _sc ## What changes were proposed in this pull request? Some PySpark examples need a SparkContext and get it by accessing _sc directly from the session. These examples should use the provided property `sparkContext` in `SparkSession` instead. ## How was this patch tested? Ran modified examples Author: Bryan Cutler Closes #13303 from BryanCutler/pyspark-session-sparkContext-MINOR. (cherry picked from commit 9c297df3d4d5fa4bbfdffdaad15f362586db384b) Signed-off-by: Davies Liu --- examples/src/main/python/als.py | 2 +- examples/src/main/python/avro_inputformat.py | 2 +- examples/src/main/python/parquet_inputformat.py | 2 +- examples/src/main/python/pi.py | 2 +- examples/src/main/python/transitive_closure.py | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 81562e20a927d..80290e7de9b06 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -67,7 +67,7 @@ def update(i, vec, mat, ratings): .appName("PythonALS")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index 3f65e8f79a65c..4422f9e7a9589 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -70,7 +70,7 @@ .appName("AvroKeyInputFormat")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext conf = None if len(sys.argv) == 3: diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index 2f09f4d5730e8..29a1ac274eccf 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -53,7 +53,7 @@ .appName("ParquetInputFormat")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext parquet_rdd = sc.newAPIHadoopFile( path, diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 5db03e4a21091..b39d710540b1b 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -32,7 +32,7 @@ .appName("PythonPi")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * partitions diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 37c41dcd03dc6..d88ea94e41c27 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -46,7 +46,7 @@ def generateGraph(): .appName("PythonTransitiveClosure")\ .getOrCreate() - sc = spark._sc + sc = spark.sparkContext partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 tc = sc.parallelize(generateGraph(), partitions).cache() From 0064a4dcbed1d91732a29c2cede464b8d148aeca Mon Sep 17 00:00:00 2001 From: Gio Borje Date: Wed, 25 May 2016 16:52:31 -0500 Subject: [PATCH 0387/1470] Log warnings for numIterations * miniBatchFraction < 1.0 ## What changes were proposed in this pull request? Add a warning log for the case that `numIterations * miniBatchFraction <1.0` during gradient descent. If the product of those two numbers is less than `1.0`, then not all training examples will be used during optimization. To put this concretely, suppose that `numExamples = 100`, `miniBatchFraction = 0.2` and `numIterations = 3`. Then, 3 iterations will occur each sampling approximately 6 examples each. In the best case, each of the 6 examples are unique; hence 18/100 examples are used. This may be counter-intuitive to most users and led to the issue during the development of another Spark ML model: https://github.com/zhengruifeng/spark-libFM/issues/11. If a user actually does not require the training data set, it would be easier and more intuitive to use `RDD.sample`. ## How was this patch tested? `build/mvn -DskipTests clean package` build succeeds Author: Gio Borje Closes #13265 from Hydrotoast/master. (cherry picked from commit 589cce93c821ac28e9090a478f6e7465398b7c30) Signed-off-by: Sean Owen --- .../apache/spark/mllib/optimization/GradientDescent.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index a67ea836e5681..735e780909ef2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -197,6 +197,11 @@ object GradientDescent extends Logging { "< 1.0 can be unstable because of the stochasticity in sampling.") } + if (numIterations * miniBatchFraction < 1.0) { + logWarning("Not all examples will be used if numIterations * miniBatchFraction < 1.0: " + + s"numIterations=$numIterations and miniBatchFraction=$miniBatchFraction") + } + val stochasticLossHistory = new ArrayBuffer[Double](numIterations) // Record previous weight and current one to calculate solution vector difference From 347acc4ea411cd430cfb1d8b8b596463f5b9dd3c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 May 2016 15:08:39 -0700 Subject: [PATCH 0388/1470] [SPARK-15534][SPARK-15535][SQL] Truncate table fixes ## What changes were proposed in this pull request? Two changes: - When things fail, `TRUNCATE TABLE` just returns nothing. Instead, we should throw exceptions. - Remove `TRUNCATE TABLE ... COLUMN`, which was never supported by either Spark or Hive. ## How was this patch tested? Jenkins. Author: Andrew Or Closes #13302 from andrewor14/truncate-table. (cherry picked from commit ee682fe293b47988056b540ee46ca49861309982) Signed-off-by: Andrew Or --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 3 +-- .../apache/spark/sql/execution/SparkSqlParser.scala | 7 +------ .../apache/spark/sql/execution/command/tables.scala | 7 ++++--- .../spark/sql/hive/execution/HiveCommandSuite.scala | 12 ------------ 4 files changed, 6 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 403191af5e5be..b0e71c7e7c7d1 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -115,8 +115,7 @@ statement | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData - | TRUNCATE TABLE tableIdentifier partitionSpec? - (COLUMNS identifierList)? #truncateTable + | TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 57f534cd9eb30..cfebfc6a5ce71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -368,17 +368,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * For example: * {{{ * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] - * [COLUMNS (col1, col2)] * }}} */ override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { - if (ctx.identifierList != null) { - throw operationNotAllowed("TRUNCATE TABLE ... COLUMNS", ctx) - } TruncateTableCommand( visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ) + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 13e63a1befb20..bef4c9222c296 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -278,7 +278,7 @@ case class LoadDataCommand( * * The syntax of this command is: * {{{ - * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] + * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ case class TruncateTableCommand( @@ -288,9 +288,10 @@ case class TruncateTableCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog if (!catalog.tableExists(tableName)) { - logError(s"table '$tableName' in TRUNCATE TABLE does not exist.") + throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does not exist.") } else if (catalog.isTemporaryTable(tableName)) { - logError(s"table '$tableName' in TRUNCATE TABLE is a temporary table.") + throw new AnalysisException( + s"Operation not allowed: TRUNCATE TABLE on temporary tables: '$tableName'") } else { val locations = if (partitionSpec.isDefined) { catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index df62ba08b8018..6f374d713bfc8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -289,10 +289,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val testResults = sql("SELECT * FROM non_part_table").collect() - intercept[ParseException] { - sql("TRUNCATE TABLE non_part_table COLUMNS (employeeID)") - } - sql("TRUNCATE TABLE non_part_table") checkAnswer(sql("SELECT * FROM non_part_table"), Seq.empty[Row]) @@ -320,10 +316,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '2'"), testResults) - intercept[ParseException] { - sql("TRUNCATE TABLE part_table PARTITION(c='1', d='1') COLUMNS (employeeID)") - } - sql("TRUNCATE TABLE part_table PARTITION(c='1', d='1')") checkAnswer( sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '1'"), @@ -332,10 +324,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), testResults) - intercept[ParseException] { - sql("TRUNCATE TABLE part_table PARTITION(c='1') COLUMNS (employeeID)") - } - sql("TRUNCATE TABLE part_table PARTITION(c='1')") checkAnswer( sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1'"), From 733cb44e3321744d521f0791e241cc0fe062d01a Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 25 May 2016 15:35:38 -0700 Subject: [PATCH 0389/1470] [SPARK-15525][SQL][BUILD] Upgrade ANTLR4 SBT plugin ## What changes were proposed in this pull request? The ANTLR4 SBT plugin has been moved from its own repo to one on bintray. The version was also changed from `0.7.10` to `0.7.11`. The latter actually broke our build (ihji has fixed this by also adding `0.7.10` and others to the bin-tray repo). This PR upgrades the SBT-ANTLR4 plugin and ANTLR4 to their most recent versions (`0.7.11`/`4.5.3`). I have also removed a few obsolete build configurations. ## How was this patch tested? Manually running SBT/Maven builds. Author: Herman van Hovell Closes #13299 from hvanhovell/SPARK-15525. (cherry picked from commit 527499b624e743583fe0f93ea0b487031891ac3a) Signed-off-by: Reynold Xin --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- pom.xml | 7 +------ project/plugins.sbt | 5 +---- 7 files changed, 7 insertions(+), 15 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 0ac1c00154838..0d6b18e8b72d8 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -3,7 +3,7 @@ RoaringBitmap-0.5.11.jar ST4-4.0.4.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index fa35fa70516f3..a3597f4ec70e5 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 99dffa93bbcf7..3ca44c5abc8f0 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index a3bee36ce535a..01e75517081d5 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index dbd7a8e0bf21c..402fd05376ee5 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -4,7 +4,7 @@ ST4-4.0.4.jar activation-1.1.1.jar antlr-2.7.7.jar antlr-runtime-3.4.jar -antlr4-runtime-4.5.2-1.jar +antlr4-runtime-4.5.3.jar aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar diff --git a/pom.xml b/pom.xml index f28aa14adc55f..3fa0eeb5f0387 100644 --- a/pom.xml +++ b/pom.xml @@ -177,7 +177,7 @@ 3.5.2 1.3.9 0.9.2 - 4.5.2-1 + 4.5.3 1.1 2.52.0 2.8 @@ -1952,11 +1952,6 @@ - - org.antlr - antlr3-maven-plugin - 3.5.2 - org.antlr antlr4-maven-plugin diff --git a/project/plugins.sbt b/project/plugins.sbt index 44ec3a12ae709..4578b56247aa9 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -20,7 +20,4 @@ libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3" libraryDependencies += "org.ow2.asm" % "asm-commons" % "5.0.3" -// TODO I am not sure we want such a dep. -resolvers += "simplytyped" at "http://simplytyped.github.io/repo/releases" - -addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.10") +addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.11") From 15a2dba6619438c5fb228437eaab50f399839473 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Wed, 25 May 2016 19:10:57 -0700 Subject: [PATCH 0390/1470] [SPARK-15533][SQL] Deprecate Dataset.explode ## What changes were proposed in this pull request? This patch deprecates `Dataset.explode` and documents appropriate workarounds to use `flatMap()` or `functions.explode()` instead. ## How was this patch tested? N/A Author: Sameer Agarwal Closes #13312 from sameeragarwal/deprecate. (cherry picked from commit 06ed1fa3e45adfc11b0f615cb8b97c99fadc735f) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/Dataset.scala | 33 ++++++++++++------- 1 file changed, 22 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 78a167eef2e4e..e5140fcf1337e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1556,30 +1556,33 @@ class Dataset[T] private[sql]( } /** - * :: Experimental :: * (Scala-specific) Returns a new [[Dataset]] where each row has been expanded to zero or more * rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of * the input row are implicitly joined with each row that is output by the function. * - * The following example uses this function to count the number of books which contain - * a given word: + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()` or `flatMap()`. The following example uses these alternatives to count + * the number of books that contain a given word: * * {{{ * case class Book(title: String, words: String) * val ds: Dataset[Book] * - * case class Word(word: String) - * val allWords = ds.explode('words) { - * case Row(words: String) => words.split(" ").map(Word(_)) - * } + * val allWords = ds.select('title, explode(split('words, " ")).as("word")) * * val bookCountPerWord = allWords.groupBy("word").agg(countDistinct("title")) * }}} * + * Using `flatMap()` this can similarly be exploded as: + * + * {{{ + * ds.flatMap(_.words.split(" ")) + * }}} + * * @group untypedrel * @since 2.0.0 */ - @Experimental + @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { val elementSchema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] @@ -1596,19 +1599,27 @@ class Dataset[T] private[sql]( } /** - * :: Experimental :: * (Scala-specific) Returns a new [[Dataset]] where a single column has been expanded to zero * or more rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. All * columns of the input row are implicitly joined with each value that is output by the function. * + * Given that this is deprecated, as an alternative, you can explode columns either using + * `functions.explode()`: + * + * {{{ + * ds.select(explode(split('words, " ")).as("word")) + * }}} + * + * or `flatMap()`: + * * {{{ - * ds.explode("words", "word") {words: String => words.split(" ")} + * ds.flatMap(_.words.split(" ")) * }}} * * @group untypedrel * @since 2.0.0 */ - @Experimental + @deprecated("use flatMap() or select() with functions.explode() instead", "2.0.0") def explode[A, B : TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) : DataFrame = { val dataType = ScalaReflection.schemaFor[B].dataType From bcad1d13f58a119948e3374072824f70a14a6d34 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Wed, 25 May 2016 21:08:03 -0700 Subject: [PATCH 0391/1470] [SPARK-15439][SPARKR] Failed to run unit test in SparkR ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) There are some failures when running SparkR unit tests. In this PR, I fixed two of these failures in test_context.R and test_sparkSQL.R The first one is due to different masked name. I added missed names in the expected arrays. The second one is because one PR removed the logic of a previous fix of missing subset method. The file privilege issue is still there. I am debugging it. SparkR shell can run the test case successfully. test_that("pipeRDD() on RDDs", { actual <- collect(pipeRDD(rdd, "more")) When using run-test script, it complains no such directories as below: cannot open file '/tmp/Rtmp4FQbah/filee2273f9d47f7': No such file or directory ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually test it Author: wm624@hotmail.com Closes #13284 from wangmiao1981/R. (cherry picked from commit 06bae8af17d9478c889d206a4556a697b5d629e7) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 6 +++++- R/pkg/inst/tests/testthat/test_context.R | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 0c2a194483b0f..f719173607fd9 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1445,7 +1445,11 @@ setMethod("[", signature(x = "SparkDataFrame"), #' } setMethod("subset", signature(x = "SparkDataFrame"), function(x, subset, select, drop = F, ...) { - x[subset, select, drop = drop] + if (missing(subset)) { + x[, select, drop = drop, ...] + } else { + x[subset, select, drop = drop, ...] + } }) #' Select diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 0e5e15c0a96c9..95258babbf101 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -27,6 +27,11 @@ test_that("Check masked functions", { namesOfMasked <- c("describe", "cov", "filter", "lag", "na.omit", "predict", "sd", "var", "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", "summary", "transform", "drop", "window", "as.data.frame") + namesOfMaskedCompletely <- c("cov", "filter", "sample") + if (as.numeric(R.version$major) == 3 && as.numeric(R.version$minor) > 2) { + namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) + namesOfMaskedCompletely <- c("endsWith", "startsWith", namesOfMaskedCompletely) + } expect_equal(length(maskedBySparkR), length(namesOfMasked)) expect_equal(sort(maskedBySparkR), sort(namesOfMasked)) # above are those reported as masked when `library(SparkR)` @@ -36,7 +41,6 @@ test_that("Check masked functions", { any(grepl("=\"ANY\"", capture.output(showMethods(x)[-1]))) })) maskedCompletely <- masked[!funcHasAny] - namesOfMaskedCompletely <- c("cov", "filter", "sample") expect_equal(length(maskedCompletely), length(namesOfMaskedCompletely)) expect_equal(sort(maskedCompletely), sort(namesOfMaskedCompletely)) }) From b3ee53b8424bf7be4f3111744e07a6100a01e4da Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 May 2016 23:54:24 -0700 Subject: [PATCH 0392/1470] [SPARK-15543][SQL] Rename DefaultSources to make them more self-describing ## What changes were proposed in this pull request? This patch renames various DefaultSources to make their names more self-describing. The choice of "DefaultSource" was from the days when we did not have a good way to specify short names. They are now named: - LibSVMFileFormat - CSVFileFormat - JdbcRelationProvider - JsonFileFormat - ParquetFileFormat - TextFileFormat Backward compatibility is maintained through aliasing. ## How was this patch tested? Updated relevant test cases too. Author: Reynold Xin Closes #13311 from rxin/SPARK-15543. (cherry picked from commit 361ebc282b2d09dc6dcf21419a53c5c617b1b6bd) Signed-off-by: Reynold Xin --- ...pache.spark.sql.sources.DataSourceRegister | 2 +- .../ml/source/libsvm/LibSVMRelation.scala | 8 +++- project/MimaExcludes.scala | 4 +- ...pache.spark.sql.sources.DataSourceRegister | 10 ++--- .../spark/sql/execution/ExistingRDD.scala | 2 +- .../execution/datasources/DataSource.scala | 45 ++++++++++++++----- ...efaultSource.scala => CSVFileFormat.scala} | 6 +-- ...ource.scala => JdbcRelationProvider.scala} | 2 +- ...SONRelation.scala => JsonFileFormat.scala} | 4 +- ...Relation.scala => ParquetFileFormat.scala} | 14 +++--- ...faultSource.scala => TextFileFormat.scala} | 2 +- .../datasources/json/JsonSuite.scala | 4 +- .../parquet/ParquetSchemaSuite.scala | 12 ++--- .../sql/sources/ResolvedDataSourceSuite.scala | 18 ++++---- .../sql/streaming/FileStreamSinkSuite.scala | 4 +- .../DataFrameReaderWriterSuite.scala | 1 - ...pache.spark.sql.sources.DataSourceRegister | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 ++++---- ...{OrcRelation.scala => OrcFileFormat.scala} | 8 +++- .../hive/orc/OrcHadoopFsRelationSuite.scala | 2 +- 20 files changed, 99 insertions(+), 69 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/{DefaultSource.scala => CSVFileFormat.scala} (96%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/{DefaultSource.scala => JdbcRelationProvider.scala} (96%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/{JSONRelation.scala => JsonFileFormat.scala} (98%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{ParquetRelation.scala => ParquetFileFormat.scala} (98%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/{DefaultSource.scala => TextFileFormat.scala} (98%) rename sql/core/src/test/scala/org/apache/spark/sql/streaming/{ => test}/DataFrameReaderWriterSuite.scala (99%) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/{OrcRelation.scala => OrcFileFormat.scala} (98%) diff --git a/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index f632dd603c449..a865cbe19b184 100644 --- a/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1 @@ -org.apache.spark.ml.source.libsvm.DefaultSource +org.apache.spark.ml.source.libsvm.LibSVMFileFormat diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 5ba768d551842..64ebf0c982346 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -90,7 +90,7 @@ private[libsvm] class LibSVMOutputWriter( * .load("data/mllib/sample_libsvm_data.txt") * * // Java - * DataFrame df = spark.read().format("libsvm") + * Dataset df = spark.read().format("libsvm") * .option("numFeatures, "780") * .load("data/mllib/sample_libsvm_data.txt"); * }}} @@ -105,9 +105,13 @@ private[libsvm] class LibSVMOutputWriter( * - "vectorType": feature vector type, "sparse" (default) or "dense". * * @see [[https://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/ LIBSVM datasets]] + * + * Note that this class is public for documentation purpose. Please don't use this class directly. + * Rather, use the data source API as illustrated above. */ +// If this is moved or renamed, please update DataSource's backwardCompatibilityMap. @Since("1.6.0") -class DefaultSource extends FileFormat with DataSourceRegister { +class LibSVMFileFormat extends FileFormat with DataSourceRegister { @Since("1.6.0") override def shortName(): String = "libsvm" diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4e99a0965780b..08c575aaeec78 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -68,7 +68,9 @@ object MimaExcludes { // SPARK-13664 Replace HadoopFsRelation with FileFormat ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.LibSVMRelation"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelationProvider"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache") + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache"), + // SPARK-15543 Rename DefaultSources to make them more self-describing + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.DefaultSource") ) ++ Seq( ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.SparkContext.emptyRDD"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.broadcast.HttpBroadcastFactory"), diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index ef9255794b6ab..9f8bb5d38fd6c 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1,6 +1,6 @@ -org.apache.spark.sql.execution.datasources.csv.DefaultSource -org.apache.spark.sql.execution.datasources.jdbc.DefaultSource -org.apache.spark.sql.execution.datasources.json.DefaultSource -org.apache.spark.sql.execution.datasources.parquet.DefaultSource -org.apache.spark.sql.execution.datasources.text.DefaultSource +org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider +org.apache.spark.sql.execution.datasources.json.JsonFileFormat +org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +org.apache.spark.sql.execution.datasources.text.TextFileFormat org.apache.spark.sql.execution.streaming.ConsoleSinkProvider 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 ec23a9c41a029..412f5fa87ee7b 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCo import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetSource} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.BaseRelation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index d0853f67b97ec..dfe06478fccb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -30,6 +30,10 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider +import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -74,15 +78,34 @@ case class DataSource( lazy val sourceInfo = sourceSchema() /** A map to maintain backward compatibility in case we move data sources around. */ - private val backwardCompatibilityMap = Map( - "org.apache.spark.sql.jdbc" -> classOf[jdbc.DefaultSource].getCanonicalName, - "org.apache.spark.sql.jdbc.DefaultSource" -> classOf[jdbc.DefaultSource].getCanonicalName, - "org.apache.spark.sql.json" -> classOf[json.DefaultSource].getCanonicalName, - "org.apache.spark.sql.json.DefaultSource" -> classOf[json.DefaultSource].getCanonicalName, - "org.apache.spark.sql.parquet" -> classOf[parquet.DefaultSource].getCanonicalName, - "org.apache.spark.sql.parquet.DefaultSource" -> classOf[parquet.DefaultSource].getCanonicalName, - "com.databricks.spark.csv" -> classOf[csv.DefaultSource].getCanonicalName - ) + private val backwardCompatibilityMap: Map[String, String] = { + val jdbc = classOf[JdbcRelationProvider].getCanonicalName + val json = classOf[JsonFileFormat].getCanonicalName + val parquet = classOf[ParquetFileFormat].getCanonicalName + val csv = classOf[CSVFileFormat].getCanonicalName + val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat" + val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat" + + Map( + "org.apache.spark.sql.jdbc" -> jdbc, + "org.apache.spark.sql.jdbc.DefaultSource" -> jdbc, + "org.apache.spark.sql.execution.datasources.jdbc.DefaultSource" -> jdbc, + "org.apache.spark.sql.execution.datasources.jdbc" -> jdbc, + "org.apache.spark.sql.json" -> json, + "org.apache.spark.sql.json.DefaultSource" -> json, + "org.apache.spark.sql.execution.datasources.json" -> json, + "org.apache.spark.sql.execution.datasources.json.DefaultSource" -> json, + "org.apache.spark.sql.parquet" -> parquet, + "org.apache.spark.sql.parquet.DefaultSource" -> parquet, + "org.apache.spark.sql.execution.datasources.parquet" -> parquet, + "org.apache.spark.sql.execution.datasources.parquet.DefaultSource" -> parquet, + "org.apache.spark.sql.hive.orc.DefaultSource" -> orc, + "org.apache.spark.sql.hive.orc" -> orc, + "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm, + "org.apache.spark.ml.source.libsvm" -> libsvm, + "com.databricks.spark.csv" -> csv + ) + } /** * Class that were removed in Spark 2.0. Used to detect incompatibility libraries for Spark 2.0. @@ -188,7 +211,7 @@ case class DataSource( throw new IllegalArgumentException("'path' is not specified") }) val isSchemaInferenceEnabled = sparkSession.conf.get(SQLConf.STREAMING_SCHEMA_INFERENCE) - val isTextSource = providingClass == classOf[text.DefaultSource] + val isTextSource = providingClass == classOf[text.TextFileFormat] // If the schema inference is disabled, only text sources require schema to be specified if (!isSchemaInferenceEnabled && !isTextSource && userSpecifiedSchema.isEmpty) { throw new IllegalArgumentException( @@ -229,7 +252,7 @@ case class DataSource( providingClass.newInstance() match { case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, options, partitionColumns) - case parquet: parquet.DefaultSource => + case parquet: parquet.ParquetFileFormat => val caseInsensitiveOptions = new CaseInsensitiveMap(options) val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 057bde1a75f07..4d36b760568cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -28,8 +28,6 @@ import org.apache.hadoop.mapreduce._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -38,7 +36,7 @@ import org.apache.spark.util.SerializableConfiguration /** * Provides access to CSV data from pure SQL statements. */ -class DefaultSource extends FileFormat with DataSourceRegister { +class CSVFileFormat extends FileFormat with DataSourceRegister { override def shortName(): String = "csv" @@ -46,7 +44,7 @@ class DefaultSource extends FileFormat with DataSourceRegister { override def hashCode(): Int = getClass.hashCode() - override def equals(other: Any): Boolean = other.isInstanceOf[DefaultSource] + override def equals(other: Any): Boolean = other.isInstanceOf[CSVFileFormat] override def inferSchema( sparkSession: SparkSession, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index 6609e5dee3af5..106ed1d440102 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -22,7 +22,7 @@ import java.util.Properties import org.apache.spark.sql.SQLContext import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, RelationProvider} -class DefaultSource extends RelationProvider with DataSourceRegister { +class JdbcRelationProvider extends RelationProvider with DataSourceRegister { override def shortName(): String = "jdbc" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 4c97abed53ce6..35f247692ffdc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -class DefaultSource extends FileFormat with DataSourceRegister { +class JsonFileFormat extends FileFormat with DataSourceRegister { override def shortName(): String = "json" @@ -151,7 +151,7 @@ class DefaultSource extends FileFormat with DataSourceRegister { override def hashCode(): Int = getClass.hashCode() - override def equals(other: Any): Boolean = other.isInstanceOf[DefaultSource] + override def equals(other: Any): Boolean = other.isInstanceOf[JsonFileFormat] } private[json] class JsonOutputWriter( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index cf5c8e94f468d..b47d41e1661f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -50,7 +50,7 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration -private[sql] class DefaultSource +private[sql] class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging @@ -62,7 +62,7 @@ private[sql] class DefaultSource override def hashCode(): Int = getClass.hashCode() - override def equals(other: Any): Boolean = other.isInstanceOf[DefaultSource] + override def equals(other: Any): Boolean = other.isInstanceOf[ParquetFileFormat] override def prepareWrite( sparkSession: SparkSession, @@ -141,7 +141,7 @@ private[sql] class DefaultSource // Should we merge schemas from all Parquet part-files? val shouldMergeSchemas = parameters - .get(ParquetRelation.MERGE_SCHEMA) + .get(ParquetFileFormat.MERGE_SCHEMA) .map(_.toBoolean) .getOrElse(sparkSession.conf.get(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED)) @@ -217,7 +217,7 @@ private[sql] class DefaultSource .orElse(filesByType.data.headOption) .toSeq } - ParquetRelation.mergeSchemasInParallel(filesToTouch, sparkSession) + ParquetFileFormat.mergeSchemasInParallel(filesToTouch, sparkSession) } case class FileTypes( @@ -543,7 +543,7 @@ private[sql] class ParquetOutputWriter( override def close(): Unit = recordWriter.close(context) } -private[sql] object ParquetRelation extends Logging { +private[sql] object ParquetFileFormat extends Logging { // Whether we should merge schemas collected from all Parquet part-files. private[sql] val MERGE_SCHEMA = "mergeSchema" @@ -822,9 +822,9 @@ private[sql] object ParquetRelation extends Logging { if (footers.isEmpty) { Iterator.empty } else { - var mergedSchema = ParquetRelation.readSchemaFromFooter(footers.head, converter) + var mergedSchema = ParquetFileFormat.readSchemaFromFooter(footers.head, converter) footers.tail.foreach { footer => - val schema = ParquetRelation.readSchemaFromFooter(footer, converter) + val schema = ParquetFileFormat.readSchemaFromFooter(footer, converter) try { mergedSchema = mergedSchema.merge(schema) } catch { case cause: SparkException => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index f091615a9a714..d9525efe6d9be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.SerializableConfiguration /** * A data source for reading text files. */ -class DefaultSource extends FileFormat with DataSourceRegister { +class TextFileFormat extends FileFormat with DataSourceRegister { override def shortName(): String = "text" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 46213a22ed3d3..500d8ff55a9a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -1320,7 +1320,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { userSpecifiedSchema = None, partitionColumns = Array.empty[String], bucketSpec = None, - className = classOf[DefaultSource].getCanonicalName, + className = classOf[JsonFileFormat].getCanonicalName, options = Map("path" -> path)).resolveRelation() val d2 = DataSource( @@ -1328,7 +1328,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { userSpecifiedSchema = None, partitionColumns = Array.empty[String], bucketSpec = None, - className = classOf[DefaultSource].getCanonicalName, + className = classOf[JsonFileFormat].getCanonicalName, options = Map("path" -> path)).resolveRelation() assert(d1 === d2) }) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index c43b142de2cf1..6db649228210d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -375,7 +375,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { StructField("lowerCase", StringType), StructField("UPPERCase", DoubleType, nullable = false)))) { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("lowercase", StringType), StructField("uppercase", DoubleType, nullable = false))), @@ -390,7 +390,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { StructType(Seq( StructField("UPPERCase", DoubleType, nullable = false)))) { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("uppercase", DoubleType, nullable = false))), @@ -401,7 +401,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { // Metastore schema contains additional non-nullable fields. assert(intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("uppercase", DoubleType, nullable = false), StructField("lowerCase", BinaryType, nullable = false))), @@ -412,7 +412,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { // Conflicting non-nullable field names intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq(StructField("lower", StringType, nullable = false))), StructType(Seq(StructField("lowerCase", BinaryType)))) } @@ -426,7 +426,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { StructField("firstField", StringType, nullable = true), StructField("secondField", StringType, nullable = true), StructField("thirdfield", StringType, nullable = true)))) { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("firstfield", StringType, nullable = true), StructField("secondfield", StringType, nullable = true), @@ -439,7 +439,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { // Merge should fail if the Metastore contains any additional fields that are not // nullable. assert(intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( + ParquetFileFormat.mergeMetastoreParquetSchema( StructType(Seq( StructField("firstfield", StringType, nullable = true), StructField("secondfield", StringType, nullable = true), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 4f6df5441736e..320aaea1e4eeb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -27,37 +27,37 @@ class ResolvedDataSourceSuite extends SparkFunSuite { test("jdbc") { assert( getProvidingClass("jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) assert( getProvidingClass("org.apache.spark.sql.execution.datasources.jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) assert( getProvidingClass("org.apache.spark.sql.jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) } test("json") { assert( getProvidingClass("json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) assert( getProvidingClass("org.apache.spark.sql.execution.datasources.json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) assert( getProvidingClass("org.apache.spark.sql.json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) } test("parquet") { assert( getProvidingClass("parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) assert( getProvidingClass("org.apache.spark.sql.execution.datasources.parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) assert( getProvidingClass("org.apache.spark.sql.parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) } test("error message for unknown data sources") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 6238b74ffac56..f3262f772b160 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -41,7 +41,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { path.delete() val hadoopConf = spark.sparkContext.hadoopConfiguration - val fileFormat = new parquet.DefaultSource() + val fileFormat = new parquet.ParquetFileFormat() def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { val df = spark @@ -73,7 +73,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { path.delete() val hadoopConf = spark.sparkContext.hadoopConfiguration - val fileFormat = new parquet.DefaultSource() + val fileFormat = new parquet.ParquetFileFormat() def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { val df = spark diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index e6c0ce95e7b57..288f6dc59741e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -101,7 +101,6 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { } class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { - import testImplicits._ private def newMetadataDir = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath diff --git a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 4a774fbf1fdf8..32aa13ff257a6 100644 --- a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1 @@ -org.apache.spark.sql.hive.orc.DefaultSource +org.apache.spark.sql.hive.orc.OrcFileFormat 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 86ab152402625..b377a20e3943b 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 @@ -32,8 +32,8 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.CreateTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} -import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetDefaultSource, ParquetRelation} -import org.apache.spark.sql.hive.orc.{DefaultSource => OrcDefaultSource} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ @@ -281,7 +281,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val inferredSchema = defaultSource.inferSchema(sparkSession, options, fileCatalog.allFiles()) inferredSchema.map { inferred => - ParquetRelation.mergeMetastoreParquetSchema(metastoreSchema, inferred) + ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, inferred) }.getOrElse(metastoreSchema) } else { defaultSource.inferSchema(sparkSession, options, fileCatalog.allFiles()).get @@ -348,13 +348,13 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } private def convertToParquetRelation(relation: MetastoreRelation): LogicalRelation = { - val defaultSource = new ParquetDefaultSource() - val fileFormatClass = classOf[ParquetDefaultSource] + val defaultSource = new ParquetFileFormat() + val fileFormatClass = classOf[ParquetFileFormat] val mergeSchema = sessionState.convertMetastoreParquetWithSchemaMerging val options = Map( - ParquetRelation.MERGE_SCHEMA -> mergeSchema.toString, - ParquetRelation.METASTORE_TABLE_NAME -> TableIdentifier( + ParquetFileFormat.MERGE_SCHEMA -> mergeSchema.toString, + ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier( relation.tableName, Some(relation.databaseName) ).unquotedString @@ -400,8 +400,8 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } private def convertToOrcRelation(relation: MetastoreRelation): LogicalRelation = { - val defaultSource = new OrcDefaultSource() - val fileFormatClass = classOf[OrcDefaultSource] + val defaultSource = new OrcFileFormat() + val fileFormatClass = classOf[OrcFileFormat] val options = Map[String, String]() convertToLogicalRelation(relation, options, defaultSource, fileFormatClass, "orc") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala similarity index 98% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 38f50c112a236..f1198179a0994 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -42,7 +42,11 @@ import org.apache.spark.sql.sources.{Filter, _} import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -private[sql] class DefaultSource +/** + * [[FileFormat]] for reading ORC files. If this is moved or renamed, please update + * [[DataSource]]'s backwardCompatibilityMap. + */ +private[sql] class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable { override def shortName(): String = "orc" @@ -262,7 +266,7 @@ private[orc] case class OrcTableScan( // Figure out the actual schema from the ORC source (without partition columns) so that we // can pick the correct ordinals. Note that this assumes that all files have the same schema. - val orcFormat = new DefaultSource + val orcFormat = new OrcFileFormat val dataSchema = orcFormat .inferSchema(sparkSession, Map.empty, inputPaths) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala index 0207b4e8c9775..5dfa58f673bf2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.types._ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { import testImplicits._ - override val dataSourceName: String = classOf[DefaultSource].getCanonicalName + override val dataSourceName: String = classOf[OrcFileFormat].getCanonicalName // ORC does not play well with NullType and UDT. override protected def supportsDataType(dataType: DataType): Boolean = dataType match { From 36acd53e8be9755a42aed8a810a17f067222120e Mon Sep 17 00:00:00 2001 From: Bo Meng Date: Thu, 26 May 2016 00:22:47 -0700 Subject: [PATCH 0393/1470] [SPARK-15537][SQL] fix dir delete issue ## What changes were proposed in this pull request? For some of the test cases, e.g. `OrcSourceSuite`, it will create temp folders and temp files inside them. But after tests finish, the folders are not removed. This will cause lots of temp files created and space occupied, if we keep running the test cases. The reason is dir.delete() won't work if dir is not empty. We need to recursively delete the content before deleting the folder. ## How was this patch tested? Manually checked the temp folder to make sure the temp files were deleted. Author: Bo Meng Closes #13304 from bomeng/SPARK-15537. (cherry picked from commit 53d4abe9e996e53c1bdcd5ac4cb8cbf08b9ec8b5) Signed-off-by: Reynold Xin --- .../spark/sql/hive/orc/OrcQuerySuite.scala | 6 ------ .../spark/sql/hive/orc/OrcSourceSuite.scala | 18 +++--------------- 2 files changed, 3 insertions(+), 21 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index f83b3a3de2e54..9771b2314a08d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -53,12 +53,6 @@ case class Person(name: String, age: Int, contacts: Seq[Contact]) class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { - def getTempFilePath(prefix: String, suffix: String = ""): File = { - val tempFile = File.createTempFile(prefix, suffix) - tempFile.delete() - tempFile - } - test("Read/write All Types") { val data = (0 to 255).map { i => (s"$i", i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala index 6081d86f4478b..4cac334859ba1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils case class OrcData(intField: Int, stringField: String) @@ -37,14 +38,10 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA override def beforeAll(): Unit = { super.beforeAll() - orcTableAsDir = File.createTempFile("orctests", "sparksql") - orcTableAsDir.delete() - orcTableAsDir.mkdir() + orcTableAsDir = Utils.createTempDir("orctests", "sparksql") // Hack: to prepare orc data files using hive external tables - orcTableDir = File.createTempFile("orctests", "sparksql") - orcTableDir.delete() - orcTableDir.mkdir() + orcTableDir = Utils.createTempDir("orctests", "sparksql") import org.apache.spark.sql.hive.test.TestHive.implicits._ sparkContext @@ -68,15 +65,6 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA """.stripMargin) } - override def afterAll(): Unit = { - try { - orcTableDir.delete() - orcTableAsDir.delete() - } finally { - super.afterAll() - } - } - test("create temporary orc table") { checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10)) From c54a073483be22cf0a3adb47cfbce19ee79dbb6b Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Thu, 26 May 2016 12:36:36 +0200 Subject: [PATCH 0394/1470] [SPARK-15492][ML][DOC] Binarization scala example copy & paste to spark-shell error ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) The Binarization scala example val dataFrame : Dataframe = spark.createDataFrame(data).toDF("label", "feature"), which can't be pasted in the spark-shell as Dataframe is not imported. Compared with other examples, this explicit type is not required. So I removed Dataframe in the code. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually tested Author: wm624@hotmail.com Closes #13266 from wangmiao1981/unit. (cherry picked from commit e451f7f0c3857cdbbca98e66928a97f797f2fc6b) Signed-off-by: Nick Pentreath --- .../scala/org/apache/spark/examples/ml/BinarizerExample.scala | 4 ++-- .../scala/org/apache/spark/examples/ml/OneVsRestExample.scala | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala index 82bc14789b461..5cd13ad64ca44 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala @@ -21,7 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.Binarizer // $example off$ -import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.{SparkSession} object BinarizerExample { def main(args: Array[String]): Unit = { @@ -31,7 +31,7 @@ object BinarizerExample { .getOrCreate() // $example on$ val data = Array((0, 0.1), (1, 0.8), (2, 0.2)) - val dataFrame: DataFrame = spark.createDataFrame(data).toDF("label", "feature") + val dataFrame = spark.createDataFrame(data).toDF("label", "feature") val binarizer: Binarizer = new Binarizer() .setInputCol("feature") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index 0b333cf629419..0da8e3137a597 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -21,7 +21,6 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.classification.{LogisticRegression, OneVsRest} import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator -import org.apache.spark.sql.DataFrame // $example off$ import org.apache.spark.sql.SparkSession @@ -43,7 +42,7 @@ object OneVsRestExample { // $example on$ // load data file. - val inputData: DataFrame = spark.read.format("libsvm") + val inputData = spark.read.format("libsvm") .load("data/mllib/sample_multiclass_classification_data.txt") // generate the train/test split. From 87374de43576df28df7ae752204af1d1149d926d Mon Sep 17 00:00:00 2001 From: Villu Ruusmann Date: Thu, 26 May 2016 08:11:34 -0500 Subject: [PATCH 0395/1470] [SPARK-15523][ML][MLLIB] Update JPMML to 1.2.15 ## What changes were proposed in this pull request? See https://issues.apache.org/jira/browse/SPARK-15523 This PR replaces PR #13293. It's isolated to a new branch, and contains some more squashed changes. ## How was this patch tested? 1. Executed `mvn clean package` in `mllib` directory 2. Executed `dev/test-dependencies.sh --replace-manifest` in the root directory. Author: Villu Ruusmann Closes #13297 from vruusmann/update-jpmml. (cherry picked from commit 6d506c9ae9a2519d1a93e788ae5316d4f942d35d) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 5 ++--- dev/deps/spark-deps-hadoop-2.3 | 5 ++--- dev/deps/spark-deps-hadoop-2.4 | 5 ++--- dev/deps/spark-deps-hadoop-2.6 | 5 ++--- dev/deps/spark-deps-hadoop-2.7 | 5 ++--- mllib/pom.xml | 10 +++------- 6 files changed, 13 insertions(+), 22 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 0d6b18e8b72d8..578691cc9395d 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -137,9 +137,8 @@ parquet-generator-1.7.0.jar parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.7.0.jar -pmml-agent-1.2.7.jar -pmml-model-1.2.7.jar -pmml-schema-1.2.7.jar +pmml-model-1.2.15.jar +pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.1.jar pyrolite-4.9.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index a3597f4ec70e5..fc6306f366815 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -144,9 +144,8 @@ parquet-generator-1.7.0.jar parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.7.0.jar -pmml-agent-1.2.7.jar -pmml-model-1.2.7.jar -pmml-schema-1.2.7.jar +pmml-model-1.2.15.jar +pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.1.jar pyrolite-4.9.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 3ca44c5abc8f0..dee1417c79af1 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -144,9 +144,8 @@ parquet-generator-1.7.0.jar parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.7.0.jar -pmml-agent-1.2.7.jar -pmml-model-1.2.7.jar -pmml-schema-1.2.7.jar +pmml-model-1.2.15.jar +pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.1.jar pyrolite-4.9.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 01e75517081d5..9695661b9c92c 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -152,9 +152,8 @@ parquet-generator-1.7.0.jar parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.7.0.jar -pmml-agent-1.2.7.jar -pmml-model-1.2.7.jar -pmml-schema-1.2.7.jar +pmml-model-1.2.15.jar +pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.1.jar pyrolite-4.9.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 402fd05376ee5..18c136ed63cb2 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -153,9 +153,8 @@ parquet-generator-1.7.0.jar parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.7.0.jar -pmml-agent-1.2.7.jar -pmml-model-1.2.7.jar -pmml-schema-1.2.7.jar +pmml-model-1.2.15.jar +pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.1.jar pyrolite-4.9.jar diff --git a/mllib/pom.xml b/mllib/pom.xml index c324afb2a2d1b..2a59fcdff7aae 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -102,15 +102,11 @@ org.jpmml pmml-model - 1.2.7 + 1.2.15 - com.sun.xml.fastinfoset - FastInfoset - - - com.sun.istack - istack-commons-runtime + org.jpmml + pmml-agent From 9cf34727c82e5289703777017b9764452b090414 Mon Sep 17 00:00:00 2001 From: felixcheung Date: Thu, 26 May 2016 11:20:20 -0700 Subject: [PATCH 0396/1470] [SPARK-10903][SPARKR] R - Simplify SQLContext method signatures and use a singleton Eliminate the need to pass sqlContext to method since it is a singleton - and we don't want to support multiple contexts in a R session. Changes are done in a back compat way with deprecation warning added. Method signature for S3 methods are added in a concise, clean approach such that in the next release the deprecated signature can be taken out easily/cleanly (just delete a few lines per method). Custom method dispatch is implemented to allow for multiple JVM reference types that are all 'jobj' in R and to avoid having to add 30 new exports. Author: felixcheung Closes #9192 from felixcheung/rsqlcontext. (cherry picked from commit c76457c8e422ce6fbf72a8fe5db94565783b12d0) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 20 +- R/pkg/R/SQLContext.R | 298 +++++++++++----- R/pkg/R/jobj.R | 5 + R/pkg/inst/tests/testthat/test_context.R | 2 +- R/pkg/inst/tests/testthat/test_mllib.R | 30 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 392 +++++++++++----------- 6 files changed, 450 insertions(+), 297 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index f719173607fd9..d54ee54cd8539 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2213,13 +2213,7 @@ setMethod("write.df", signature(df = "SparkDataFrame", path = "character"), function(df, path, source = NULL, mode = "error", ...){ if (is.null(source)) { - if (exists(".sparkRSQLsc", envir = .sparkREnv)) { - sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv) - } else if (exists(".sparkRHivesc", envir = .sparkREnv)) { - sqlContext <- get(".sparkRHivesc", envir = .sparkREnv) - } else { - stop("sparkRHive or sparkRSQL context has to be specified") - } + sqlContext <- getSqlContext() source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", "org.apache.spark.sql.parquet") } @@ -2281,15 +2275,9 @@ setMethod("saveAsTable", signature(df = "SparkDataFrame", tableName = "character"), function(df, tableName, source = NULL, mode="error", ...){ if (is.null(source)) { - if (exists(".sparkRSQLsc", envir = .sparkREnv)) { - sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv) - } else if (exists(".sparkRHivesc", envir = .sparkREnv)) { - sqlContext <- get(".sparkRHivesc", envir = .sparkREnv) - } else { - stop("sparkRHive or sparkRSQL context has to be specified") - } - source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", - "org.apache.spark.sql.parquet") + sqlContext <- getSqlContext() + source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", + "org.apache.spark.sql.parquet") } jmode <- convertToJSaveMode(mode) options <- varargsToEnv(...) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 6b7a341bee889..584bbbf0e4c2d 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -37,6 +37,45 @@ getInternalType <- function(x) { stop(paste("Unsupported type for SparkDataFrame:", class(x)))) } +#' Temporary function to reroute old S3 Method call to new +#' This function is specifically implemented to remove SQLContext from the parameter list. +#' It determines the target to route the call by checking the parent of this callsite (say 'func'). +#' The target should be called 'func.default'. +#' We need to check the class of x to ensure it is SQLContext/HiveContext before dispatching. +#' @param newFuncSig name of the function the user should call instead in the deprecation message +#' @param x the first parameter of the original call +#' @param ... the rest of parameter to pass along +#' @return whatever the target returns +#' @noRd +dispatchFunc <- function(newFuncSig, x, ...) { + funcName <- as.character(sys.call(sys.parent())[[1]]) + f <- get(paste0(funcName, ".default")) + # Strip sqlContext from list of parameters and then pass the rest along. + contextNames <- c("org.apache.spark.sql.SQLContext", + "org.apache.spark.sql.hive.HiveContext", + "org.apache.spark.sql.hive.test.TestHiveContext") + if (missing(x) && length(list(...)) == 0) { + f() + } else if (class(x) == "jobj" && + any(grepl(paste(contextNames, collapse = "|"), getClassName.jobj(x)))) { + .Deprecated(newFuncSig, old = paste0(funcName, "(sqlContext...)")) + f(...) + } else { + f(x, ...) + } +} + +#' return the SQL Context +getSqlContext <- function() { + if (exists(".sparkRHivesc", envir = .sparkREnv)) { + get(".sparkRHivesc", envir = .sparkREnv) + } else if (exists(".sparkRSQLsc", envir = .sparkREnv)) { + get(".sparkRSQLsc", envir = .sparkREnv) + } else { + stop("SQL context not initialized") + } +} + #' infer the SQL type infer_type <- function(x) { if (is.null(x)) { @@ -74,7 +113,6 @@ infer_type <- function(x) { #' #' Converts R data.frame or list into SparkDataFrame. #' -#' @param sqlContext A SQLContext #' @param data An RDD or list or data.frame #' @param schema a list of column names or named list (StructType), optional #' @return a SparkDataFrame @@ -84,13 +122,16 @@ infer_type <- function(x) { #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df1 <- as.DataFrame(sqlContext, iris) -#' df2 <- as.DataFrame(sqlContext, list(3,4,5,6)) -#' df3 <- createDataFrame(sqlContext, iris) +#' df1 <- as.DataFrame(iris) +#' df2 <- as.DataFrame(list(3,4,5,6)) +#' df3 <- createDataFrame(iris) #' } +#' @name createDataFrame +#' @method createDataFrame default # TODO(davies): support sampling and infer type from NA -createDataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0) { +createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { + sqlContext <- getSqlContext() if (is.data.frame(data)) { # get the names of columns, they will be put into RDD if (is.null(schema)) { @@ -164,11 +205,21 @@ createDataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0 dataFrame(sdf) } +createDataFrame <- function(x, ...) { + dispatchFunc("createDataFrame(data, schema = NULL, samplingRatio = 1.0)", x, ...) +} + #' @rdname createDataFrame #' @aliases createDataFrame #' @export -as.DataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0) { - createDataFrame(sqlContext, data, schema, samplingRatio) +#' @method as.DataFrame default + +as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { + createDataFrame(data, schema, samplingRatio) +} + +as.DataFrame <- function(x, ...) { + dispatchFunc("as.DataFrame(data, schema = NULL, samplingRatio = 1.0)", x, ...) } #' toDF @@ -190,14 +241,7 @@ setGeneric("toDF", function(x, ...) { standardGeneric("toDF") }) setMethod("toDF", signature(x = "RDD"), function(x, ...) { - sqlContext <- 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(sqlContext, x, ...) + createDataFrame(x, ...) }) #' Create a SparkDataFrame from a JSON file. @@ -205,21 +249,23 @@ setMethod("toDF", signature(x = "RDD"), #' Loads a JSON file (one object per line), returning the result as a SparkDataFrame #' It goes through the entire dataset once to determine the schema. #' -#' @param sqlContext SQLContext to use #' @param path Path of file to read. A vector of multiple paths is allowed. #' @return SparkDataFrame #' @rdname read.json -#' @name read.json #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) +#' df <- jsonFile(path) #' } -read.json <- function(sqlContext, path) { +#' @name read.json +#' @method read.json default + +read.json.default <- function(path) { + sqlContext <- getSqlContext() # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sqlContext, "read") @@ -227,14 +273,23 @@ read.json <- function(sqlContext, path) { dataFrame(sdf) } +read.json <- function(x, ...) { + dispatchFunc("read.json(path)", x, ...) +} + #' @rdname read.json #' @name jsonFile #' @export -jsonFile <- function(sqlContext, path) { +#' @method jsonFile default + +jsonFile.default <- function(path) { .Deprecated("read.json") - read.json(sqlContext, path) + read.json(path) } +jsonFile <- function(x, ...) { + dispatchFunc("jsonFile(path)", x, ...) +} #' JSON RDD #' @@ -254,6 +309,7 @@ jsonFile <- function(sqlContext, path) { #' df <- jsonRDD(sqlContext, rdd) #'} +# TODO: remove - this method is no longer exported # TODO: support schema jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { .Deprecated("read.json") @@ -272,13 +328,15 @@ jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { #' #' Loads a Parquet file, returning the result as a SparkDataFrame. #' -#' @param sqlContext SQLContext to use #' @param path Path of file to read. A vector of multiple paths is allowed. #' @return SparkDataFrame #' @rdname read.parquet -#' @name read.parquet #' @export -read.parquet <- function(sqlContext, path) { +#' @name read.parquet +#' @method read.parquet default + +read.parquet.default <- function(path) { + sqlContext <- getSqlContext() # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sqlContext, "read") @@ -286,13 +344,22 @@ read.parquet <- function(sqlContext, path) { dataFrame(sdf) } +read.parquet <- function(x, ...) { + dispatchFunc("read.parquet(...)", x, ...) +} + #' @rdname read.parquet #' @name parquetFile #' @export -# TODO: Implement saveasParquetFile and write examples for both -parquetFile <- function(sqlContext, ...) { +#' @method parquetFile default + +parquetFile.default <- function(...) { .Deprecated("read.parquet") - read.parquet(sqlContext, unlist(list(...))) + read.parquet(unlist(list(...))) +} + +parquetFile <- function(x, ...) { + dispatchFunc("parquetFile(...)", x, ...) } #' Create a SparkDataFrame from a text file. @@ -302,20 +369,22 @@ parquetFile <- function(sqlContext, ...) { #' ignored in the resulting DataFrame. #' Each line in the text file is a new row in the resulting SparkDataFrame. #' -#' @param sqlContext SQLContext to use #' @param path Path of file to read. A vector of multiple paths is allowed. #' @return SparkDataFrame #' @rdname read.text -#' @name read.text #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.txt" -#' df <- read.text(sqlContext, path) +#' df <- read.text(path) #' } -read.text <- function(sqlContext, path) { +#' @name read.text +#' @method read.text default + +read.text.default <- function(path) { + sqlContext <- getSqlContext() # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sqlContext, "read") @@ -323,27 +392,38 @@ read.text <- function(sqlContext, path) { dataFrame(sdf) } +read.text <- function(x, ...) { + dispatchFunc("read.text(path)", x, ...) +} + #' SQL Query #' #' Executes a SQL query using Spark, returning the result as a SparkDataFrame. #' -#' @param sqlContext SQLContext to use #' @param sqlQuery A character vector containing the SQL query #' @return SparkDataFrame +#' @rdname sql #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' registerTempTable(df, "table") -#' new_df <- sql(sqlContext, "SELECT * FROM table") +#' new_df <- sql("SELECT * FROM table") #' } +#' @name sql +#' @method sql default + +sql.default <- function(sqlQuery) { + sqlContext <- getSqlContext() + sdf <- callJMethod(sqlContext, "sql", sqlQuery) + dataFrame(sdf) +} -sql <- function(sqlContext, sqlQuery) { - sdf <- callJMethod(sqlContext, "sql", sqlQuery) - dataFrame(sdf) +sql <- function(x, ...) { + dispatchFunc("sql(sqlQuery)", x, ...) } #' Create a SparkDataFrame from a SparkSQL Table @@ -351,7 +431,6 @@ sql <- function(sqlContext, sqlQuery) { #' Returns the specified Table as a SparkDataFrame. The Table must have already been registered #' in the SQLContext. #' -#' @param sqlContext SQLContext to use #' @param tableName The SparkSQL Table to convert to a SparkDataFrame. #' @return SparkDataFrame #' @rdname tableToDF @@ -362,12 +441,14 @@ sql <- function(sqlContext, sqlQuery) { #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' registerTempTable(df, "table") -#' new_df <- tableToDF(sqlContext, "table") +#' new_df <- tableToDF("table") #' } +#' @note since 2.0.0 -tableToDF <- function(sqlContext, tableName) { +tableToDF <- function(tableName) { + sqlContext <- getSqlContext() sdf <- callJMethod(sqlContext, "table", tableName) dataFrame(sdf) } @@ -376,18 +457,21 @@ tableToDF <- function(sqlContext, tableName) { #' #' Returns a SparkDataFrame containing names of tables in the given database. #' -#' @param sqlContext SQLContext to use #' @param databaseName name of the database #' @return a SparkDataFrame +#' @rdname tables #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' tables(sqlContext, "hive") +#' tables("hive") #' } +#' @name tables +#' @method tables default -tables <- function(sqlContext, databaseName = NULL) { +tables.default <- function(databaseName = NULL) { + sqlContext <- getSqlContext() jdf <- if (is.null(databaseName)) { callJMethod(sqlContext, "tables") } else { @@ -396,23 +480,29 @@ tables <- function(sqlContext, databaseName = NULL) { dataFrame(jdf) } +tables <- function(x, ...) { + dispatchFunc("tables(databaseName = NULL)", x, ...) +} #' Table Names #' #' Returns the names of tables in the given database as an array. #' -#' @param sqlContext SQLContext to use #' @param databaseName name of the database #' @return a list of table names +#' @rdname tableNames #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' tableNames(sqlContext, "hive") +#' tableNames("hive") #' } +#' @name tableNames +#' @method tableNames default -tableNames <- function(sqlContext, databaseName = NULL) { +tableNames.default <- function(databaseName = NULL) { + sqlContext <- getSqlContext() if (is.null(databaseName)) { callJMethod(sqlContext, "tableNames") } else { @@ -420,88 +510,121 @@ tableNames <- function(sqlContext, databaseName = NULL) { } } +tableNames <- function(x, ...) { + dispatchFunc("tableNames(databaseName = NULL)", x, ...) +} #' Cache Table #' #' Caches the specified table in-memory. #' -#' @param sqlContext SQLContext to use #' @param tableName The name of the table being cached #' @return SparkDataFrame +#' @rdname cacheTable #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' registerTempTable(df, "table") -#' cacheTable(sqlContext, "table") +#' cacheTable("table") #' } +#' @name cacheTable +#' @method cacheTable default -cacheTable <- function(sqlContext, tableName) { +cacheTable.default <- function(tableName) { + sqlContext <- getSqlContext() callJMethod(sqlContext, "cacheTable", tableName) } +cacheTable <- function(x, ...) { + dispatchFunc("cacheTable(tableName)", x, ...) +} + #' Uncache Table #' #' Removes the specified table from the in-memory cache. #' -#' @param sqlContext SQLContext to use #' @param tableName The name of the table being uncached #' @return SparkDataFrame +#' @rdname uncacheTable #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' registerTempTable(df, "table") -#' uncacheTable(sqlContext, "table") +#' uncacheTable("table") #' } +#' @name uncacheTable +#' @method uncacheTable default -uncacheTable <- function(sqlContext, tableName) { +uncacheTable.default <- function(tableName) { + sqlContext <- getSqlContext() callJMethod(sqlContext, "uncacheTable", tableName) } +uncacheTable <- function(x, ...) { + dispatchFunc("uncacheTable(tableName)", x, ...) +} + #' Clear Cache #' #' Removes all cached tables from the in-memory cache. #' -#' @param sqlContext SQLContext to use +#' @rdname clearCache +#' @export #' @examples #' \dontrun{ -#' clearCache(sqlContext) +#' clearCache() #' } +#' @name clearCache +#' @method clearCache default -clearCache <- function(sqlContext) { +clearCache.default <- function() { + sqlContext <- getSqlContext() callJMethod(sqlContext, "clearCache") } +clearCache <- function() { + dispatchFunc("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 sqlContext SQLContext to use #' @param tableName The name of the SparkSQL table to be dropped. +#' @rdname dropTempTable +#' @export #' @examples #' \dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df <- read.df(sqlContext, path, "parquet") +#' df <- read.df(path, "parquet") #' registerTempTable(df, "table") -#' dropTempTable(sqlContext, "table") +#' dropTempTable("table") #' } +#' @name dropTempTable +#' @method dropTempTable default -dropTempTable <- function(sqlContext, tableName) { +dropTempTable.default <- function(tableName) { + sqlContext <- getSqlContext() if (class(tableName) != "character") { stop("tableName must be a string.") } callJMethod(sqlContext, "dropTempTable", tableName) } +dropTempTable <- function(x, ...) { + dispatchFunc("dropTempTable(tableName)", x, ...) +} + #' Load a SparkDataFrame #' #' Returns the dataset in a data source as a SparkDataFrame @@ -510,7 +633,6 @@ dropTempTable <- function(sqlContext, tableName) { #' If `source` is not specified, the default data source configured by #' "spark.sql.sources.default" will be used. #' -#' @param sqlContext SQLContext to use #' @param path The path of files to load #' @param source The name of external data source #' @param schema The data schema defined in structType @@ -522,20 +644,22 @@ dropTempTable <- function(sqlContext, tableName) { #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df1 <- read.df(sqlContext, "path/to/file.json", source = "json") +#' df1 <- read.df("path/to/file.json", source = "json") #' schema <- structType(structField("name", "string"), #' structField("info", "map")) -#' df2 <- read.df(sqlContext, mapTypeJsonPath, "json", schema) -#' df3 <- loadDF(sqlContext, "data/test_table", "parquet", mergeSchema = "true") +#' df2 <- read.df(mapTypeJsonPath, "json", schema) +#' df3 <- loadDF("data/test_table", "parquet", mergeSchema = "true") #' } +#' @name read.df +#' @method read.df default -read.df <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) { +read.df.default <- function(path = NULL, source = NULL, schema = NULL, ...) { + sqlContext <- getSqlContext() options <- varargsToEnv(...) if (!is.null(path)) { options[["path"]] <- path } if (is.null(source)) { - sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv) source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", "org.apache.spark.sql.parquet") } @@ -549,10 +673,20 @@ read.df <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) dataFrame(sdf) } +read.df <- function(x, ...) { + dispatchFunc("read.df(path = NULL, source = NULL, schema = NULL, ...)", x, ...) +} + #' @rdname read.df #' @name loadDF -loadDF <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) { - read.df(sqlContext, path, source, schema, ...) +#' @method loadDF default + +loadDF.default <- function(path = NULL, source = NULL, schema = NULL, ...) { + read.df(path, source, schema, ...) +} + +loadDF <- function(x, ...) { + dispatchFunc("loadDF(path = NULL, source = NULL, schema = NULL, ...)", x, ...) } #' Create an external table @@ -564,20 +698,23 @@ loadDF <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) { #' If `source` is not specified, the default data source configured by #' "spark.sql.sources.default" will be used. #' -#' @param sqlContext 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 SparkDataFrame +#' @rdname createExternalTable #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df <- sparkRSQL.createExternalTable(sqlContext, "myjson", path="path/to/json", source="json") +#' df <- sparkRSQL.createExternalTable("myjson", path="path/to/json", source="json") #' } +#' @name createExternalTable +#' @method createExternalTable default -createExternalTable <- function(sqlContext, tableName, path = NULL, source = NULL, ...) { +createExternalTable.default <- function(tableName, path = NULL, source = NULL, ...) { + sqlContext <- getSqlContext() options <- varargsToEnv(...) if (!is.null(path)) { options[["path"]] <- path @@ -586,6 +723,10 @@ createExternalTable <- function(sqlContext, tableName, path = NULL, source = NUL dataFrame(sdf) } +createExternalTable <- function(x, ...) { + dispatchFunc("createExternalTable(tableName, path = NULL, source = NULL, ...)", x, ...) +} + #' Create a SparkDataFrame representing the database table accessible via JDBC URL #' #' Additional JDBC database connection properties can be set (...) @@ -596,7 +737,6 @@ createExternalTable <- function(sqlContext, tableName, path = NULL, source = NUL #' Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash #' your external database systems. #' -#' @param sqlContext SQLContext to use #' @param url JDBC database url of the form `jdbc:subprotocol:subname` #' @param tableName the name of the table in the external database #' @param partitionColumn the name of a column of integral type that will be used for partitioning @@ -616,12 +756,13 @@ createExternalTable <- function(sqlContext, tableName, path = NULL, source = NUL #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" -#' df <- read.jdbc(sqlContext, jdbcUrl, "table", predicates = list("field<=123"), user = "username") -#' df2 <- read.jdbc(sqlContext, jdbcUrl, "table2", partitionColumn = "index", lowerBound = 0, +#' df <- read.jdbc(jdbcUrl, "table", predicates = list("field<=123"), user = "username") +#' df2 <- read.jdbc(jdbcUrl, "table2", partitionColumn = "index", lowerBound = 0, #' upperBound = 10000, user = "username", password = "password") #' } +#' @note since 2.0.0 -read.jdbc <- function(sqlContext, url, tableName, +read.jdbc <- function(url, tableName, partitionColumn = NULL, lowerBound = NULL, upperBound = NULL, numPartitions = 0L, predicates = list(), ...) { jprops <- varargsToJProperties(...) @@ -629,6 +770,7 @@ read.jdbc <- function(sqlContext, url, tableName, read <- callJMethod(sqlContext, "read") if (!is.null(partitionColumn)) { if (is.null(numPartitions) || numPartitions == 0) { + sqlContext <- getSqlContext() sc <- callJMethod(sqlContext, "sparkContext") numPartitions <- callJMethod(sc, "defaultParallelism") } else { diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R index 0838a7bb35e0d..898e80648fc2b 100644 --- a/R/pkg/R/jobj.R +++ b/R/pkg/R/jobj.R @@ -77,6 +77,11 @@ print.jobj <- function(x, ...) { cat("Java ref type", name, "id", x$id, "\n", sep = " ") } +getClassName.jobj <- function(x) { + cls <- callJMethod(x, "getClass") + callJMethod(cls, "getName") +} + cleanup.jobj <- function(jobj) { if (isValidJobj(jobj)) { objId <- jobj$id diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 95258babbf101..c1f1a8932d98f 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -58,7 +58,7 @@ test_that("repeatedly starting and stopping SparkR SQL", { for (i in 1:4) { sc <- sparkR.init() sqlContext <- sparkRSQL.init(sc) - df <- createDataFrame(sqlContext, data.frame(a = 1:20)) + df <- createDataFrame(data.frame(a = 1:20)) expect_equal(count(df), 20) sparkR.stop() } diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index 5f8a27d4e094c..59ef15c1e9fd5 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -26,7 +26,7 @@ sc <- sparkR.init() sqlContext <- sparkRSQL.init(sc) test_that("formula of spark.glm", { - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) # directly calling the spark API # dot minus and intercept vs native glm model <- spark.glm(training, Sepal_Width ~ . - Species + 0) @@ -41,7 +41,7 @@ test_that("formula of spark.glm", { expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) # glm should work with long formula - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) training$LongLongLongLongLongName <- training$Sepal_Width training$VeryLongLongLongLonLongName <- training$Sepal_Length training$AnotherLongLongLongLongName <- training$Species @@ -53,7 +53,7 @@ test_that("formula of spark.glm", { }) test_that("spark.glm and predict", { - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) # gaussian family model <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species) prediction <- predict(model, training) @@ -80,7 +80,7 @@ test_that("spark.glm and predict", { test_that("spark.glm summary", { # gaussian family - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) stats <- summary(spark.glm(training, Sepal_Width ~ Sepal_Length + Species)) rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) @@ -99,7 +99,7 @@ test_that("spark.glm summary", { expect_equal(stats$aic, rStats$aic) # binomial family - df <- suppressWarnings(createDataFrame(sqlContext, iris)) + df <- suppressWarnings(createDataFrame(iris)) training <- df[df$Species %in% c("versicolor", "virginica"), ] stats <- summary(spark.glm(training, Species ~ Sepal_Length + Sepal_Width, family = binomial(link = "logit"))) @@ -128,7 +128,7 @@ test_that("spark.glm summary", { }) test_that("spark.glm save/load", { - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) m <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species) s <- summary(m) @@ -157,7 +157,7 @@ test_that("spark.glm save/load", { test_that("formula of glm", { - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) # dot minus and intercept vs native glm model <- glm(Sepal_Width ~ . - Species + 0, data = training) vals <- collect(select(predict(model, training), "prediction")) @@ -171,7 +171,7 @@ test_that("formula of glm", { expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) # glm should work with long formula - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) training$LongLongLongLongLongName <- training$Sepal_Width training$VeryLongLongLongLonLongName <- training$Sepal_Length training$AnotherLongLongLongLongName <- training$Species @@ -183,7 +183,7 @@ test_that("formula of glm", { }) test_that("glm and predict", { - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) # gaussian family model <- glm(Sepal_Width ~ Sepal_Length + Species, data = training) prediction <- predict(model, training) @@ -210,7 +210,7 @@ test_that("glm and predict", { test_that("glm summary", { # gaussian family - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training)) rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) @@ -229,7 +229,7 @@ test_that("glm summary", { expect_equal(stats$aic, rStats$aic) # binomial family - df <- suppressWarnings(createDataFrame(sqlContext, iris)) + df <- suppressWarnings(createDataFrame(iris)) training <- df[df$Species %in% c("versicolor", "virginica"), ] stats <- summary(glm(Species ~ Sepal_Length + Sepal_Width, data = training, family = binomial(link = "logit"))) @@ -258,7 +258,7 @@ test_that("glm summary", { }) test_that("glm save/load", { - training <- suppressWarnings(createDataFrame(sqlContext, iris)) + training <- suppressWarnings(createDataFrame(iris)) m <- glm(Sepal_Width ~ Sepal_Length + Species, data = training) s <- summary(m) @@ -287,7 +287,7 @@ test_that("glm save/load", { test_that("spark.kmeans", { newIris <- iris newIris$Species <- NULL - training <- suppressWarnings(createDataFrame(sqlContext, newIris)) + training <- suppressWarnings(createDataFrame(newIris)) take(training, 1) @@ -365,7 +365,7 @@ test_that("spark.naiveBayes", { t <- as.data.frame(Titanic) t1 <- t[t$Freq > 0, -5] - df <- suppressWarnings(createDataFrame(sqlContext, t1)) + df <- suppressWarnings(createDataFrame(t1)) m <- spark.naiveBayes(df, Survived ~ .) s <- summary(m) expect_equal(as.double(s$apriori[1, "Yes"]), 0.5833333, tolerance = 1e-6) @@ -420,7 +420,7 @@ test_that("spark.survreg", { # data <- list(list(4, 1, 0, 0), list(3, 1, 2, 0), list(1, 1, 1, 0), list(1, 0, 1, 0), list(2, 1, 1, 1), list(2, 1, 0, 1), list(3, 0, 0, 1)) - df <- createDataFrame(sqlContext, data, c("time", "status", "x", "sex")) + df <- createDataFrame(data, c("time", "status", "x", "sex")) model <- spark.survreg(df, Surv(time, status) ~ x + sex) stats <- summary(model) coefs <- as.vector(stats$coefficients[, 1]) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index b2d769f2acd0c..94fa363d7ef4e 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -32,6 +32,22 @@ markUtf8 <- function(s) { s } +setHiveContext <- function(sc) { + ssc <- callJMethod(sc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, + error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv) + hiveCtx +} + +unsetHiveContext <- function() { + remove(".sparkRHivesc", envir = .sparkREnv) +} + # Tests for SparkSQL functions in SparkR sc <- sparkR.init() @@ -99,8 +115,8 @@ test_that("structType and structField", { test_that("create DataFrame from RDD", { rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) - df <- createDataFrame(sqlContext, rdd, list("a", "b")) - dfAsDF <- as.DataFrame(sqlContext, rdd, list("a", "b")) + df <- createDataFrame(rdd, list("a", "b")) + dfAsDF <- as.DataFrame(rdd, list("a", "b")) expect_is(df, "SparkDataFrame") expect_is(dfAsDF, "SparkDataFrame") expect_equal(count(df), 10) @@ -116,8 +132,8 @@ test_that("create DataFrame from RDD", { expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) expect_equal(dtypes(dfAsDF), list(c("a", "int"), c("b", "string"))) - df <- createDataFrame(sqlContext, rdd) - dfAsDF <- as.DataFrame(sqlContext, rdd) + df <- createDataFrame(rdd) + dfAsDF <- as.DataFrame(rdd) expect_is(df, "SparkDataFrame") expect_is(dfAsDF, "SparkDataFrame") expect_equal(columns(df), c("_1", "_2")) @@ -125,13 +141,13 @@ test_that("create DataFrame from RDD", { schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), structField(x = "b", type = "string", nullable = TRUE)) - df <- createDataFrame(sqlContext, rdd, schema) + df <- createDataFrame(rdd, schema) expect_is(df, "SparkDataFrame") 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(sqlContext, rdd) + df <- createDataFrame(rdd) expect_is(df, "SparkDataFrame") expect_equal(count(df), 10) expect_equal(columns(df), c("a", "b")) @@ -139,9 +155,9 @@ test_that("create DataFrame from RDD", { schema <- structType(structField("name", "string"), structField("age", "integer"), structField("height", "float")) - df <- read.df(sqlContext, jsonPathNa, "json", schema) - df2 <- createDataFrame(sqlContext, toRDD(df), schema) - df2AsDF <- as.DataFrame(sqlContext, toRDD(df), schema) + df <- read.df(jsonPathNa, "json", schema) + df2 <- createDataFrame(toRDD(df), schema) + df2AsDF <- as.DataFrame(toRDD(df), schema) expect_equal(columns(df2), c("name", "age", "height")) expect_equal(columns(df2AsDF), c("name", "age", "height")) expect_equal(dtypes(df2), list(c("name", "string"), c("age", "int"), c("height", "float"))) @@ -154,7 +170,7 @@ test_that("create DataFrame from RDD", { localDF <- data.frame(name = c("John", "Smith", "Sarah"), age = c(19L, 23L, 18L), height = c(176.5, 181.4, 173.7)) - df <- createDataFrame(sqlContext, localDF, schema) + df <- createDataFrame(localDF, schema) expect_is(df, "SparkDataFrame") expect_equal(count(df), 3) expect_equal(columns(df), c("name", "age", "height")) @@ -162,55 +178,50 @@ test_that("create DataFrame from RDD", { expect_equal(as.list(collect(where(df, df$name == "John"))), list(name = "John", age = 19L, height = 176.5)) - ssc <- callJMethod(sc, "sc") - hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) - }, - error = function(err) { - skip("Hive is not build with SparkSQL, skipped") - }) - sql(hiveCtx, "CREATE TABLE people (name string, age double, height float)") - df <- read.df(hiveCtx, jsonPathNa, "json", schema) + setHiveContext(sc) + sql("CREATE TABLE people (name string, age double, height float)") + df <- read.df(jsonPathNa, "json", schema) invisible(insertInto(df, "people")) - expect_equal(collect(sql(hiveCtx, "SELECT age from people WHERE name = 'Bob'"))$age, + expect_equal(collect(sql("SELECT age from people WHERE name = 'Bob'"))$age, c(16)) - expect_equal(collect(sql(hiveCtx, "SELECT height from people WHERE name ='Bob'"))$height, + expect_equal(collect(sql("SELECT height from people WHERE name ='Bob'"))$height, c(176.5)) + unsetHiveContext() }) test_that("convert NAs to null type in DataFrames", { rdd <- parallelize(sc, list(list(1L, 2L), list(NA, 4L))) - df <- createDataFrame(sqlContext, rdd, list("a", "b")) + df <- createDataFrame(rdd, list("a", "b")) expect_true(is.na(collect(df)[2, "a"])) expect_equal(collect(df)[2, "b"], 4L) l <- data.frame(x = 1L, y = c(1L, NA_integer_, 3L)) - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_equal(collect(df)[2, "x"], 1L) expect_true(is.na(collect(df)[2, "y"])) rdd <- parallelize(sc, list(list(1, 2), list(NA, 4))) - df <- createDataFrame(sqlContext, rdd, list("a", "b")) + df <- createDataFrame(rdd, list("a", "b")) expect_true(is.na(collect(df)[2, "a"])) expect_equal(collect(df)[2, "b"], 4) l <- data.frame(x = 1, y = c(1, NA_real_, 3)) - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_equal(collect(df)[2, "x"], 1) expect_true(is.na(collect(df)[2, "y"])) l <- list("a", "b", NA, "d") - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_true(is.na(collect(df)[3, "_1"])) expect_equal(collect(df)[4, "_1"], "d") l <- list("a", "b", NA_character_, "d") - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_true(is.na(collect(df)[3, "_1"])) expect_equal(collect(df)[4, "_1"], "d") l <- list(TRUE, FALSE, NA, TRUE) - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_true(is.na(collect(df)[3, "_1"])) expect_equal(collect(df)[4, "_1"], TRUE) }) @@ -244,40 +255,40 @@ test_that("toDF", { test_that("create DataFrame from list or data.frame", { l <- list(list(1, 2), list(3, 4)) - df <- createDataFrame(sqlContext, l, c("a", "b")) + df <- createDataFrame(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(sqlContext, l) + df <- createDataFrame(l) expect_equal(columns(df), c("a", "b")) a <- 1:3 b <- c("a", "b", "c") ldf <- data.frame(a, b) - df <- createDataFrame(sqlContext, ldf) + df <- createDataFrame(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) - irisdf <- suppressWarnings(createDataFrame(sqlContext, iris)) + irisdf <- suppressWarnings(createDataFrame(iris)) iris_collected <- collect(irisdf) expect_equivalent(iris_collected[, -5], iris[, -5]) expect_equal(iris_collected$Species, as.character(iris$Species)) - mtcarsdf <- createDataFrame(sqlContext, mtcars) + mtcarsdf <- createDataFrame(mtcars) expect_equivalent(collect(mtcarsdf), mtcars) bytes <- as.raw(c(1, 2, 3)) - df <- createDataFrame(sqlContext, list(list(bytes))) + df <- createDataFrame(list(list(bytes))) expect_equal(collect(df)[[1]][[1]], bytes) }) 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(sqlContext, list(l)) + df <- createDataFrame(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) @@ -291,7 +302,7 @@ test_that("create DataFrame with complex types", { s <- listToStruct(list(a = "aa", b = 3L)) l <- list(as.list(1:10), list("a", "b"), e, s) - df <- createDataFrame(sqlContext, list(l), c("a", "b", "c", "d")) + df <- createDataFrame(list(l), c("a", "b", "c", "d")) expect_equal(dtypes(df), list(c("a", "array"), c("b", "array"), c("c", "map"), @@ -318,7 +329,7 @@ test_that("create DataFrame from a data.frame with complex types", { ldf$a_list <- list(list(1, 2), list(3, 4)) ldf$an_envir <- c(as.environment(list(a = 1, b = 2)), as.environment(list(c = 3))) - sdf <- createDataFrame(sqlContext, ldf) + sdf <- createDataFrame(ldf) collected <- collect(sdf) expect_identical(ldf[, 1, FALSE], collected[, 1, FALSE]) @@ -334,7 +345,7 @@ writeLines(mockLinesMapType, mapTypeJsonPath) test_that("Collect DataFrame with complex types", { # ArrayType - df <- read.json(sqlContext, complexTypeJsonPath) + df <- read.json(complexTypeJsonPath) ldf <- collect(df) expect_equal(nrow(ldf), 3) expect_equal(ncol(ldf), 3) @@ -346,7 +357,7 @@ test_that("Collect DataFrame with complex types", { # MapType schema <- structType(structField("name", "string"), structField("info", "map")) - df <- read.df(sqlContext, mapTypeJsonPath, "json", schema) + df <- read.df(mapTypeJsonPath, "json", schema) expect_equal(dtypes(df), list(c("name", "string"), c("info", "map"))) ldf <- collect(df) @@ -360,7 +371,7 @@ test_that("Collect DataFrame with complex types", { expect_equal(bob$height, 176.5) # StructType - df <- read.json(sqlContext, mapTypeJsonPath) + df <- read.json(mapTypeJsonPath) expect_equal(dtypes(df), list(c("info", "struct"), c("name", "string"))) ldf <- collect(df) @@ -376,7 +387,7 @@ test_that("Collect DataFrame with complex types", { test_that("read/write json files", { # Test read.df - df <- read.df(sqlContext, jsonPath, "json") + df <- read.df(jsonPath, "json") expect_is(df, "SparkDataFrame") expect_equal(count(df), 3) @@ -384,17 +395,17 @@ test_that("read/write json files", { schema <- structType(structField("name", type = "string"), structField("age", type = "double")) - df1 <- read.df(sqlContext, jsonPath, "json", schema) + df1 <- read.df(jsonPath, "json", schema) expect_is(df1, "SparkDataFrame") expect_equal(dtypes(df1), list(c("name", "string"), c("age", "double"))) # Test loadDF - df2 <- loadDF(sqlContext, jsonPath, "json", schema) + df2 <- loadDF(jsonPath, "json", schema) expect_is(df2, "SparkDataFrame") expect_equal(dtypes(df2), list(c("name", "string"), c("age", "double"))) # Test read.json - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expect_is(df, "SparkDataFrame") expect_equal(count(df), 3) @@ -407,11 +418,11 @@ test_that("read/write json files", { write.json(df, jsonPath3) # Test read.json()/jsonFile() works with multiple input paths - jsonDF1 <- read.json(sqlContext, c(jsonPath2, jsonPath3)) + jsonDF1 <- read.json(c(jsonPath2, jsonPath3)) expect_is(jsonDF1, "SparkDataFrame") expect_equal(count(jsonDF1), 6) # Suppress warnings because jsonFile is deprecated - jsonDF2 <- suppressWarnings(jsonFile(sqlContext, c(jsonPath2, jsonPath3))) + jsonDF2 <- suppressWarnings(jsonFile(c(jsonPath2, jsonPath3))) expect_is(jsonDF2, "SparkDataFrame") expect_equal(count(jsonDF2), 6) @@ -433,82 +444,82 @@ test_that("jsonRDD() on a RDD with json string", { }) test_that("test cache, uncache and clearCache", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) registerTempTable(df, "table1") - cacheTable(sqlContext, "table1") - uncacheTable(sqlContext, "table1") - clearCache(sqlContext) - dropTempTable(sqlContext, "table1") + cacheTable("table1") + uncacheTable("table1") + clearCache() + dropTempTable("table1") }) test_that("test tableNames and tables", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) registerTempTable(df, "table1") - expect_equal(length(tableNames(sqlContext)), 1) - df <- tables(sqlContext) + expect_equal(length(tableNames()), 1) + df <- tables() expect_equal(count(df), 1) - dropTempTable(sqlContext, "table1") + dropTempTable("table1") }) test_that("registerTempTable() results in a queryable table and sql() results in a new DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) registerTempTable(df, "table1") - newdf <- sql(sqlContext, "SELECT * FROM table1 where name = 'Michael'") + newdf <- sql("SELECT * FROM table1 where name = 'Michael'") expect_is(newdf, "SparkDataFrame") expect_equal(count(newdf), 1) - dropTempTable(sqlContext, "table1") + dropTempTable("table1") }) test_that("insertInto() on a registered table", { - df <- read.df(sqlContext, jsonPath, "json") + df <- read.df(jsonPath, "json") write.df(df, parquetPath, "parquet", "overwrite") - dfParquet <- read.df(sqlContext, parquetPath, "parquet") + dfParquet <- read.df(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 <- read.df(sqlContext, jsonPath2, "json") + df2 <- read.df(jsonPath2, "json") write.df(df2, parquetPath2, "parquet", "overwrite") - dfParquet2 <- read.df(sqlContext, parquetPath2, "parquet") + dfParquet2 <- read.df(parquetPath2, "parquet") registerTempTable(dfParquet, "table1") insertInto(dfParquet2, "table1") - expect_equal(count(sql(sqlContext, "select * from table1")), 5) - expect_equal(first(sql(sqlContext, "select * from table1 order by age"))$name, "Michael") - dropTempTable(sqlContext, "table1") + expect_equal(count(sql("select * from table1")), 5) + expect_equal(first(sql("select * from table1 order by age"))$name, "Michael") + dropTempTable("table1") registerTempTable(dfParquet, "table1") insertInto(dfParquet2, "table1", overwrite = TRUE) - expect_equal(count(sql(sqlContext, "select * from table1")), 2) - expect_equal(first(sql(sqlContext, "select * from table1 order by age"))$name, "Bob") - dropTempTable(sqlContext, "table1") + expect_equal(count(sql("select * from table1")), 2) + expect_equal(first(sql("select * from table1 order by age"))$name, "Bob") + dropTempTable("table1") unlink(jsonPath2) unlink(parquetPath2) }) test_that("tableToDF() returns a new DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) registerTempTable(df, "table1") - tabledf <- tableToDF(sqlContext, "table1") + tabledf <- tableToDF("table1") expect_is(tabledf, "SparkDataFrame") expect_equal(count(tabledf), 3) - tabledf2 <- tableToDF(sqlContext, "table1") + tabledf2 <- tableToDF("table1") expect_equal(count(tabledf2), 3) - dropTempTable(sqlContext, "table1") + dropTempTable("table1") }) test_that("toRDD() returns an RRDD", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) testRDD <- toRDD(df) expect_is(testRDD, "RDD") expect_equal(count(testRDD), 3) }) test_that("union on two RDDs created from DataFrames returns an RRDD", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) RDD1 <- toRDD(df) RDD2 <- toRDD(df) unioned <- unionRDD(RDD1, RDD2) @@ -530,7 +541,7 @@ test_that("union on mixed serialization types correctly returns a byte RRDD", { writeLines(textLines, textPath) textRDD <- textFile(sc, textPath) - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) dfRDD <- toRDD(df) unionByte <- unionRDD(rdd, dfRDD) @@ -548,7 +559,7 @@ test_that("union on mixed serialization types correctly returns a byte RRDD", { test_that("objectFile() works with row serialization", { objectPath <- tempfile(pattern = "spark-test", fileext = ".tmp") - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) dfRDD <- toRDD(df) saveAsObjectFile(coalesce(dfRDD, 1L), objectPath) objectIn <- objectFile(sc, objectPath) @@ -559,7 +570,7 @@ test_that("objectFile() works with row serialization", { }) test_that("lapply() on a DataFrame returns an RDD with the correct columns", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) testRDD <- lapply(df, function(row) { row$newCol <- row$age + 5 row @@ -571,7 +582,7 @@ test_that("lapply() on a DataFrame returns an RDD with the correct columns", { }) test_that("collect() returns a data.frame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) rdf <- collect(df) expect_true(is.data.frame(rdf)) expect_equal(names(rdf)[1], "age") @@ -587,20 +598,20 @@ test_that("collect() returns a data.frame", { expect_equal(ncol(rdf), 2) # collect() correctly handles multiple columns with same name - df <- createDataFrame(sqlContext, list(list(1, 2)), schema = c("name", "name")) + df <- createDataFrame(list(list(1, 2)), schema = c("name", "name")) ldf <- collect(df) expect_equal(names(ldf), c("name", "name")) }) test_that("limit() returns DataFrame with the correct number of rows", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) dfLimited <- limit(df, 2) expect_is(dfLimited, "SparkDataFrame") expect_equal(count(dfLimited), 2) }) test_that("collect() and take() on a DataFrame return the same number of rows and columns", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expect_equal(nrow(collect(df)), nrow(take(df, 10))) expect_equal(ncol(collect(df)), ncol(take(df, 10))) }) @@ -614,7 +625,7 @@ test_that("collect() support Unicode characters", { jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPath) - df <- read.df(sqlContext, jsonPath, "json") + df <- read.df(jsonPath, "json") rdf <- collect(df) expect_true(is.data.frame(rdf)) expect_equal(rdf$name[1], markUtf8("안녕하세요")) @@ -622,12 +633,12 @@ test_that("collect() support Unicode characters", { expect_equal(rdf$name[3], markUtf8("こんにちは")) expect_equal(rdf$name[4], markUtf8("Xin chào")) - df1 <- createDataFrame(sqlContext, rdf) + df1 <- createDataFrame(rdf) expect_equal(collect(where(df1, df1$name == markUtf8("您好")))$name, markUtf8("您好")) }) test_that("multiple pipeline transformations result in an RDD with the correct values", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) first <- lapply(df, function(row) { row$age <- row$age + 5 row @@ -644,7 +655,7 @@ test_that("multiple pipeline transformations result in an RDD with the correct v }) test_that("cache(), persist(), and unpersist() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expect_false(df@env$isCached) cache(df) expect_true(df@env$isCached) @@ -663,7 +674,7 @@ test_that("cache(), persist(), and unpersist() on a DataFrame", { }) test_that("schema(), dtypes(), columns(), names() return the correct values/format", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) testSchema <- schema(df) expect_equal(length(testSchema$fields()), 2) expect_equal(testSchema$fields()[[1]]$dataType.toString(), "LongType") @@ -684,7 +695,7 @@ test_that("schema(), dtypes(), columns(), names() return the correct values/form }) test_that("names() colnames() set the column names", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) names(df) <- c("col1", "col2") expect_equal(colnames(df)[2], "col2") @@ -699,7 +710,7 @@ test_that("names() colnames() set the column names", { expect_error(colnames(df) <- c("1", NA), "Column names cannot be NA.") # Note: if this test is broken, remove check for "." character on colnames<- method - irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) + irisDF <- suppressWarnings(createDataFrame(iris)) expect_equal(names(irisDF)[1], "Sepal_Length") # Test base::colnames base::names @@ -715,7 +726,7 @@ test_that("names() colnames() set the column names", { }) test_that("head() and first() return the correct data", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) testHead <- head(df) expect_equal(nrow(testHead), 3) expect_equal(ncol(testHead), 2) @@ -748,7 +759,7 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { jsonPathWithDup <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPathWithDup) - df <- read.json(sqlContext, jsonPathWithDup) + df <- read.json(jsonPathWithDup) uniques <- distinct(df) expect_is(uniques, "SparkDataFrame") expect_equal(count(uniques), 3) @@ -759,7 +770,6 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { # Test dropDuplicates() df <- createDataFrame( - sqlContext, list( list(2, 1, 2), list(1, 1, 1), list(1, 2, 1), list(2, 1, 2), @@ -795,7 +805,7 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { }) test_that("sample on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) sampled <- sample(df, FALSE, 1.0) expect_equal(nrow(collect(sampled)), count(df)) expect_is(sampled, "SparkDataFrame") @@ -817,7 +827,7 @@ test_that("sample on a DataFrame", { }) test_that("select operators", { - df <- select(read.json(sqlContext, jsonPath), "name", "age") + df <- select(read.json(jsonPath), "name", "age") expect_is(df$name, "Column") expect_is(df[[2]], "Column") expect_is(df[["age"]], "Column") @@ -846,7 +856,7 @@ test_that("select operators", { }) test_that("select with column", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) df1 <- select(df, "name") expect_equal(columns(df1), c("name")) expect_equal(count(df1), 3) @@ -869,7 +879,7 @@ test_that("select with column", { }) test_that("drop column", { - df <- select(read.json(sqlContext, jsonPath), "name", "age") + df <- select(read.json(jsonPath), "name", "age") df1 <- drop(df, "name") expect_equal(columns(df1), c("age")) @@ -891,7 +901,7 @@ test_that("drop column", { test_that("subsetting", { # read.json returns columns in random order - df <- select(read.json(sqlContext, jsonPath), "name", "age") + df <- select(read.json(jsonPath), "name", "age") filtered <- df[df$age > 20, ] expect_equal(count(filtered), 1) expect_equal(columns(filtered), c("name", "age")) @@ -928,7 +938,7 @@ test_that("subsetting", { }) test_that("selectExpr() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) selected <- selectExpr(df, "age * 2") expect_equal(names(selected), "(age * 2)") expect_equal(collect(selected), collect(select(df, df$age * 2L))) @@ -939,12 +949,12 @@ test_that("selectExpr() on a DataFrame", { }) test_that("expr() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expect_equal(collect(select(df, expr("abs(-123)")))[1, 1], 123) }) test_that("column calculation", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) d <- collect(select(df, alias(df$age + 1, "age2"))) expect_equal(names(d), c("age2")) df2 <- select(df, lower(df$name), abs(df$age)) @@ -953,40 +963,35 @@ test_that("column calculation", { }) test_that("test HiveContext", { - ssc <- callJMethod(sc, "sc") - 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") + setHiveContext(sc) + df <- createExternalTable("json", jsonPath, "json") expect_is(df, "SparkDataFrame") expect_equal(count(df), 3) - df2 <- sql(hiveCtx, "select * from json") + df2 <- sql("select * from json") expect_is(df2, "SparkDataFrame") expect_equal(count(df2), 3) jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") invisible(saveAsTable(df, "json2", "json", "append", path = jsonPath2)) - df3 <- sql(hiveCtx, "select * from json2") + df3 <- sql("select * from json2") expect_is(df3, "SparkDataFrame") expect_equal(count(df3), 3) unlink(jsonPath2) hivetestDataPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") invisible(saveAsTable(df, "hivetestbl", path = hivetestDataPath)) - df4 <- sql(hiveCtx, "select * from hivetestbl") + df4 <- sql("select * from hivetestbl") expect_is(df4, "SparkDataFrame") expect_equal(count(df4), 3) unlink(hivetestDataPath) parquetDataPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") invisible(saveAsTable(df, "parquetest", "parquet", mode = "overwrite", path = parquetDataPath)) - df5 <- sql(hiveCtx, "select * from parquetest") + df5 <- sql("select * from parquetest") expect_is(df5, "SparkDataFrame") expect_equal(count(df5), 3) unlink(parquetDataPath) + unsetHiveContext() }) test_that("column operators", { @@ -1025,7 +1030,7 @@ test_that("column functions", { expect_equal(class(rank())[[1]], "Column") expect_equal(rank(1:3), as.numeric(c(1:3))) - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) df2 <- select(df, between(df$age, c(20, 30)), between(df$age, c(10, 20))) expect_equal(collect(df2)[[2, 1]], TRUE) expect_equal(collect(df2)[[2, 2]], FALSE) @@ -1044,11 +1049,11 @@ test_that("column functions", { expect_true(abs(collect(select(df, stddev(df$age)))[1, 1] - 7.778175) < 1e-6) expect_equal(collect(select(df, var_pop(df$age)))[1, 1], 30.25) - df5 <- createDataFrame(sqlContext, list(list(a = "010101"))) + df5 <- createDataFrame(list(list(a = "010101"))) expect_equal(collect(select(df5, conv(df5$a, 2, 16)))[1, 1], "15") # Test array_contains() and sort_array() - df <- createDataFrame(sqlContext, list(list(list(1L, 2L, 3L)), list(list(6L, 5L, 4L)))) + df <- createDataFrame(list(list(list(1L, 2L, 3L)), list(list(6L, 5L, 4L)))) result <- collect(select(df, array_contains(df[[1]], 1L)))[[1]] expect_equal(result, c(TRUE, FALSE)) @@ -1061,8 +1066,7 @@ test_that("column functions", { expect_equal(length(lag(ldeaths, 12)), 72) # Test struct() - df <- createDataFrame(sqlContext, - list(list(1L, 2L, 3L), list(4L, 5L, 6L)), + df <- createDataFrame(list(list(1L, 2L, 3L), list(4L, 5L, 6L)), schema = c("a", "b", "c")) result <- collect(select(df, struct("a", "c"))) expected <- data.frame(row.names = 1:2) @@ -1078,15 +1082,14 @@ test_that("column functions", { # Test encode(), decode() bytes <- as.raw(c(0xe5, 0xa4, 0xa7, 0xe5, 0x8d, 0x83, 0xe4, 0xb8, 0x96, 0xe7, 0x95, 0x8c)) - df <- createDataFrame(sqlContext, - list(list(markUtf8("大千世界"), "utf-8", bytes)), + df <- createDataFrame(list(list(markUtf8("大千世界"), "utf-8", bytes)), schema = c("a", "b", "c")) result <- collect(select(df, encode(df$a, "utf-8"), decode(df$c, "utf-8"))) expect_equal(result[[1]][[1]], bytes) expect_equal(result[[2]], markUtf8("大千世界")) # Test first(), last() - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expect_equal(collect(select(df, first(df$age)))[[1]], NA) expect_equal(collect(select(df, first(df$age, TRUE)))[[1]], 30) expect_equal(collect(select(df, first("age")))[[1]], NA) @@ -1097,7 +1100,7 @@ test_that("column functions", { expect_equal(collect(select(df, last("age", TRUE)))[[1]], 19) # Test bround() - df <- createDataFrame(sqlContext, data.frame(x = c(2.5, 3.5))) + df <- createDataFrame(data.frame(x = c(2.5, 3.5))) expect_equal(collect(select(df, bround(df$x, 0)))[[1]][1], 2) expect_equal(collect(select(df, bround(df$x, 0)))[[1]][2], 4) }) @@ -1109,7 +1112,7 @@ test_that("column binary mathfunctions", { "{\"a\":4, \"b\":8}") jsonPathWithDup <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPathWithDup) - df <- read.json(sqlContext, jsonPathWithDup) + df <- read.json(jsonPathWithDup) expect_equal(collect(select(df, atan2(df$a, df$b)))[1, "ATAN2(a, b)"], atan2(1, 5)) expect_equal(collect(select(df, atan2(df$a, df$b)))[2, "ATAN2(a, b)"], atan2(2, 6)) expect_equal(collect(select(df, atan2(df$a, df$b)))[3, "ATAN2(a, b)"], atan2(3, 7)) @@ -1130,7 +1133,7 @@ test_that("column binary mathfunctions", { }) test_that("string operators", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(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") @@ -1150,14 +1153,14 @@ test_that("string operators", { expect_equal(collect(select(df, regexp_replace(df$name, "(n.y)", "ydn")))[2, 1], "Aydn") l2 <- list(list(a = "aaads")) - df2 <- createDataFrame(sqlContext, l2) + df2 <- createDataFrame(l2) expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) expect_equal(collect(select(df2, locate("aa", df2$a, 2)))[1, 1], 2) expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") # nolint expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") # nolint l3 <- list(list(a = "a.b.c.d")) - df3 <- createDataFrame(sqlContext, l3) + df3 <- createDataFrame(l3) expect_equal(collect(select(df3, substring_index(df3$a, ".", 2)))[1, 1], "a.b") expect_equal(collect(select(df3, substring_index(df3$a, ".", -3)))[1, 1], "b.c.d") expect_equal(collect(select(df3, translate(df3$a, "bc", "12")))[1, 1], "a.1.2.d") @@ -1169,7 +1172,7 @@ test_that("date functions on a DataFrame", { l <- list(list(a = 1L, b = as.Date("2012-12-13")), list(a = 2L, b = as.Date("2013-12-14")), list(a = 3L, b = as.Date("2014-12-15"))) - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_equal(collect(select(df, dayofmonth(df$b)))[, 1], c(13, 14, 15)) expect_equal(collect(select(df, dayofyear(df$b)))[, 1], c(348, 348, 349)) expect_equal(collect(select(df, weekofyear(df$b)))[, 1], c(50, 50, 51)) @@ -1189,7 +1192,7 @@ test_that("date functions on a DataFrame", { l2 <- list(list(a = 1L, b = as.POSIXlt("2012-12-13 12:34:00", tz = "UTC")), list(a = 2L, b = as.POSIXlt("2014-12-15 01:24:34", tz = "UTC"))) - df2 <- createDataFrame(sqlContext, l2) + df2 <- createDataFrame(l2) expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24)) expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34)) expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1], @@ -1201,7 +1204,7 @@ test_that("date functions on a DataFrame", { expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) l3 <- list(list(a = 1000), list(a = -1000)) - df3 <- createDataFrame(sqlContext, l3) + df3 <- createDataFrame(l3) result31 <- collect(select(df3, from_unixtime(df3$a))) expect_equal(grep("\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", result31[, 1], perl = TRUE), c(1, 2)) @@ -1212,13 +1215,13 @@ test_that("date functions on a DataFrame", { test_that("greatest() and least() on a DataFrame", { l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_equal(collect(select(df, greatest(df$a, df$b)))[, 1], c(2, 4)) expect_equal(collect(select(df, least(df$a, df$b)))[, 1], c(1, 3)) }) test_that("time windowing (window()) with all inputs", { - df <- createDataFrame(sqlContext, data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) df$window <- window(df$t, "5 seconds", "5 seconds", "0 seconds") local <- collect(df)$v # Not checking time windows because of possible time zone issues. Just checking that the function @@ -1227,7 +1230,7 @@ test_that("time windowing (window()) with all inputs", { }) test_that("time windowing (window()) with slide duration", { - df <- createDataFrame(sqlContext, data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) df$window <- window(df$t, "5 seconds", "2 seconds") local <- collect(df)$v # Not checking time windows because of possible time zone issues. Just checking that the function @@ -1236,7 +1239,7 @@ test_that("time windowing (window()) with slide duration", { }) test_that("time windowing (window()) with start time", { - df <- createDataFrame(sqlContext, data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) df$window <- window(df$t, "5 seconds", startTime = "2 seconds") local <- collect(df)$v # Not checking time windows because of possible time zone issues. Just checking that the function @@ -1245,7 +1248,7 @@ test_that("time windowing (window()) with start time", { }) test_that("time windowing (window()) with just window duration", { - df <- createDataFrame(sqlContext, data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) df$window <- window(df$t, "5 seconds") local <- collect(df)$v # Not checking time windows because of possible time zone issues. Just checking that the function @@ -1255,7 +1258,7 @@ test_that("time windowing (window()) with just window duration", { test_that("when(), otherwise() and ifelse() on a DataFrame", { l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, 1)))[, 1], c(NA, 1)) expect_equal(collect(select(df, otherwise(when(df$a > 1, 1), 0)))[, 1], c(0, 1)) expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, 0, 1)))[, 1], c(1, 0)) @@ -1263,14 +1266,14 @@ test_that("when(), otherwise() and ifelse() on a DataFrame", { test_that("when(), otherwise() and ifelse() with column on a DataFrame", { l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) - df <- createDataFrame(sqlContext, l) + df <- createDataFrame(l) expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, lit(1))))[, 1], c(NA, 1)) expect_equal(collect(select(df, otherwise(when(df$a > 1, lit(1)), lit(0))))[, 1], c(0, 1)) expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, lit(0), lit(1))))[, 1], c(1, 0)) }) test_that("group by, agg functions", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) df1 <- agg(df, name = "max", age = "sum") expect_equal(1, count(df1)) df1 <- agg(df, age2 = max(df$age)) @@ -1315,7 +1318,7 @@ test_that("group by, agg functions", { "{\"name\":\"ID2\", \"value\": \"-3\"}") jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines2, jsonPath2) - gd2 <- groupBy(read.json(sqlContext, jsonPath2), "name") + gd2 <- groupBy(read.json(jsonPath2), "name") df6 <- agg(gd2, value = "sum") df6_local <- collect(df6) expect_equal(42, df6_local[df6_local$name == "ID1", ][1, 2]) @@ -1332,7 +1335,7 @@ test_that("group by, agg functions", { "{\"name\":\"Justin\", \"age\":1}") jsonPath3 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines3, jsonPath3) - df8 <- read.json(sqlContext, jsonPath3) + df8 <- read.json(jsonPath3) gd3 <- groupBy(df8, "name") gd3_local <- collect(sum(gd3)) expect_equal(60, gd3_local[gd3_local$name == "Andy", ][1, 2]) @@ -1351,7 +1354,7 @@ test_that("group by, agg functions", { }) test_that("arrange() and orderBy() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) sorted <- arrange(df, df$age) expect_equal(collect(sorted)[1, 2], "Michael") @@ -1377,7 +1380,7 @@ test_that("arrange() and orderBy() on a DataFrame", { }) test_that("filter() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) filtered <- filter(df, "age > 20") expect_equal(count(filtered), 1) expect_equal(collect(filtered)$name, "Andy") @@ -1400,7 +1403,7 @@ test_that("filter() on a DataFrame", { }) test_that("join() and merge() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", "{\"name\":\"Andy\", \"test\": \"no\"}", @@ -1408,7 +1411,7 @@ test_that("join() and merge() on a DataFrame", { "{\"name\":\"Bob\", \"test\": \"yes\"}") jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines2, jsonPath2) - df2 <- read.json(sqlContext, jsonPath2) + df2 <- read.json(jsonPath2) joined <- join(df, df2) expect_equal(names(joined), c("age", "name", "name", "test")) @@ -1483,7 +1486,7 @@ test_that("join() and merge() on a DataFrame", { "{\"name\":\"Bob\", \"name_y\":\"Bob\", \"test\": \"yes\"}") jsonPath3 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines3, jsonPath3) - df3 <- read.json(sqlContext, jsonPath3) + df3 <- read.json(jsonPath3) expect_error(merge(df, df3), paste("The following column name: name_y occurs more than once in the 'DataFrame'.", "Please use different suffixes for the intersected columns.", sep = "")) @@ -1493,7 +1496,7 @@ test_that("join() and merge() on a DataFrame", { }) test_that("toJSON() returns an RDD of the correct values", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) testRDD <- toJSON(df) expect_is(testRDD, "RDD") expect_equal(getSerializedMode(testRDD), "string") @@ -1501,7 +1504,7 @@ test_that("toJSON() returns an RDD of the correct values", { }) test_that("showDF()", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expected <- paste("+----+-------+\n", "| age| name|\n", "+----+-------+\n", @@ -1513,19 +1516,19 @@ test_that("showDF()", { }) test_that("isLocal()", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expect_false(isLocal(df)) }) test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(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 <- read.df(sqlContext, jsonPath2, "json") + df2 <- read.df(jsonPath2, "json") unioned <- arrange(unionAll(df, df2), df$age) expect_is(unioned, "SparkDataFrame") @@ -1557,7 +1560,7 @@ test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", { }) test_that("withColumn() and withColumnRenamed()", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) newDF <- withColumn(df, "newAge", df$age + 2) expect_equal(length(columns(newDF)), 3) expect_equal(columns(newDF)[3], "newAge") @@ -1574,7 +1577,7 @@ test_that("withColumn() and withColumnRenamed()", { }) test_that("mutate(), transform(), rename() and names()", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) newDF <- mutate(df, newAge = df$age + 2) expect_equal(length(columns(newDF)), 3) expect_equal(columns(newDF)[3], "newAge") @@ -1622,10 +1625,10 @@ test_that("mutate(), transform(), rename() and names()", { }) test_that("read/write Parquet files", { - df <- read.df(sqlContext, jsonPath, "json") + df <- read.df(jsonPath, "json") # Test write.df and read.df write.df(df, parquetPath, "parquet", mode = "overwrite") - df2 <- read.df(sqlContext, parquetPath, "parquet") + df2 <- read.df(parquetPath, "parquet") expect_is(df2, "SparkDataFrame") expect_equal(count(df2), 3) @@ -1634,10 +1637,10 @@ test_that("read/write Parquet files", { write.parquet(df, parquetPath2) parquetPath3 <- tempfile(pattern = "parquetPath3", fileext = ".parquet") suppressWarnings(saveAsParquetFile(df, parquetPath3)) - parquetDF <- read.parquet(sqlContext, c(parquetPath2, parquetPath3)) + parquetDF <- read.parquet(c(parquetPath2, parquetPath3)) expect_is(parquetDF, "SparkDataFrame") expect_equal(count(parquetDF), count(df) * 2) - parquetDF2 <- suppressWarnings(parquetFile(sqlContext, parquetPath2, parquetPath3)) + parquetDF2 <- suppressWarnings(parquetFile(parquetPath2, parquetPath3)) expect_is(parquetDF2, "SparkDataFrame") expect_equal(count(parquetDF2), count(df) * 2) @@ -1654,7 +1657,7 @@ test_that("read/write Parquet files", { test_that("read/write text files", { # Test write.df and read.df - df <- read.df(sqlContext, jsonPath, "text") + df <- read.df(jsonPath, "text") expect_is(df, "SparkDataFrame") expect_equal(colnames(df), c("value")) expect_equal(count(df), 3) @@ -1664,7 +1667,7 @@ test_that("read/write text files", { # Test write.text and read.text textPath2 <- tempfile(pattern = "textPath2", fileext = ".txt") write.text(df, textPath2) - df2 <- read.text(sqlContext, c(textPath, textPath2)) + df2 <- read.text(c(textPath, textPath2)) expect_is(df2, "SparkDataFrame") expect_equal(colnames(df2), c("value")) expect_equal(count(df2), count(df) * 2) @@ -1674,7 +1677,7 @@ test_that("read/write text files", { }) test_that("describe() and summarize() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) stats <- describe(df, "age") expect_equal(collect(stats)[1, "summary"], "count") expect_equal(collect(stats)[2, "age"], "24.5") @@ -1692,7 +1695,7 @@ test_that("describe() and summarize() on a DataFrame", { }) test_that("dropna() and na.omit() on a DataFrame", { - df <- read.json(sqlContext, jsonPathNa) + df <- read.json(jsonPathNa) rows <- collect(df) # drop with columns @@ -1778,7 +1781,7 @@ test_that("dropna() and na.omit() on a DataFrame", { }) test_that("fillna() on a DataFrame", { - df <- read.json(sqlContext, jsonPathNa) + df <- read.json(jsonPathNa) rows <- collect(df) # fill with value @@ -1829,7 +1832,7 @@ test_that("crosstab() on a DataFrame", { test_that("cov() and corr() on a DataFrame", { l <- lapply(c(0:9), function(x) { list(x, x * 2.0) }) - df <- createDataFrame(sqlContext, l, c("singles", "doubles")) + df <- createDataFrame(l, c("singles", "doubles")) result <- cov(df, "singles", "doubles") expect_true(abs(result - 55.0 / 3) < 1e-12) @@ -1847,7 +1850,7 @@ test_that("freqItems() on a DataFrame", { rdf <- data.frame(numbers = input, letters = as.character(input), negDoubles = input * -1.0, stringsAsFactors = F) rdf[ input %% 3 == 0, ] <- c(1, "1", -1) - df <- createDataFrame(sqlContext, rdf) + df <- createDataFrame(rdf) multiColResults <- freqItems(df, c("numbers", "letters"), support = 0.1) expect_true(1 %in% multiColResults$numbers[[1]]) expect_true("1" %in% multiColResults$letters[[1]]) @@ -1857,7 +1860,7 @@ test_that("freqItems() on a DataFrame", { l <- lapply(c(0:99), function(i) { if (i %% 2 == 0) { list(1L, -1.0) } else { list(i, i * -1.0) }}) - df <- createDataFrame(sqlContext, l, c("a", "b")) + df <- createDataFrame(l, c("a", "b")) result <- freqItems(df, c("a", "b"), 0.4) expect_identical(result[[1]], list(list(1L, 99L))) expect_identical(result[[2]], list(list(-1, -99))) @@ -1865,7 +1868,7 @@ test_that("freqItems() on a DataFrame", { test_that("sampleBy() on a DataFrame", { l <- lapply(c(0:99), function(i) { as.character(i %% 3) }) - df <- createDataFrame(sqlContext, l, "key") + df <- createDataFrame(l, "key") fractions <- list("0" = 0.1, "1" = 0.2) sample <- sampleBy(df, "key", fractions, 0) result <- collect(orderBy(count(groupBy(sample, "key")), "key")) @@ -1875,19 +1878,19 @@ test_that("sampleBy() on a DataFrame", { test_that("approxQuantile() on a DataFrame", { l <- lapply(c(0:99), function(i) { i }) - df <- createDataFrame(sqlContext, l, "key") + df <- createDataFrame(l, "key") quantiles <- approxQuantile(df, "key", c(0.5, 0.8), 0.0) expect_equal(quantiles[[1]], 50) expect_equal(quantiles[[2]], 80) }) test_that("SQL error message is returned from JVM", { - retError <- tryCatch(sql(sqlContext, "select * from blah"), error = function(e) e) + retError <- tryCatch(sql("select * from blah"), error = function(e) e) expect_equal(grepl("Table or view not found", retError), TRUE) expect_equal(grepl("blah", retError), TRUE) }) -irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +irisDF <- suppressWarnings(createDataFrame(iris)) test_that("Method as.data.frame as a synonym for collect()", { expect_equal(as.data.frame(irisDF), collect(irisDF)) @@ -1899,7 +1902,7 @@ test_that("Method as.data.frame as a synonym for collect()", { }) test_that("attach() on a DataFrame", { - df <- read.json(sqlContext, jsonPath) + df <- read.json(jsonPath) expect_error(age) attach(df) expect_is(age, "SparkDataFrame") @@ -1919,7 +1922,7 @@ test_that("attach() on a DataFrame", { }) test_that("with() on a DataFrame", { - df <- suppressWarnings(createDataFrame(sqlContext, iris)) + df <- suppressWarnings(createDataFrame(iris)) expect_error(Sepal_Length) sum1 <- with(df, list(summary(Sepal_Length), summary(Sepal_Width))) expect_equal(collect(sum1[[1]])[1, "Sepal_Length"], "150") @@ -1939,15 +1942,15 @@ test_that("Method coltypes() to get and set R's data types of a DataFrame", { structField("c4", "timestamp")) # Test primitive types - DF <- createDataFrame(sqlContext, data, schema) + DF <- createDataFrame(data, schema) expect_equal(coltypes(DF), c("integer", "logical", "POSIXct")) # Test complex types - x <- createDataFrame(sqlContext, list(list(as.environment( + x <- createDataFrame(list(list(as.environment( list("a" = "b", "c" = "d", "e" = "f"))))) expect_equal(coltypes(x), "map") - df <- selectExpr(read.json(sqlContext, jsonPath), "name", "(age * 1.21) as age") + df <- selectExpr(read.json(jsonPath), "name", "(age * 1.21) as age") expect_equal(dtypes(df), list(c("name", "string"), c("age", "decimal(24,2)"))) df1 <- select(df, cast(df$age, "integer")) @@ -1971,7 +1974,7 @@ test_that("Method str()", { iris2 <- iris colnames(iris2) <- c("Sepal_Length", "Sepal_Width", "Petal_Length", "Petal_Width", "Species") iris2$col <- TRUE - irisDF2 <- createDataFrame(sqlContext, iris2) + irisDF2 <- createDataFrame(iris2) out <- capture.output(str(irisDF2)) expect_equal(length(out), 7) @@ -1989,7 +1992,7 @@ test_that("Method str()", { # number of returned rows x <- runif(200, 1, 10) df <- data.frame(t(as.matrix(data.frame(x, x, x, x, x, x, x, x, x)))) - DF <- createDataFrame(sqlContext, df) + DF <- createDataFrame(df) out <- capture.output(str(DF)) expect_equal(length(out), 103) @@ -2039,13 +2042,12 @@ test_that("Histogram", { histogram(irisDF, "Sepal_Width", 12)$counts), T) # Test when there are zero counts - df <- as.DataFrame(sqlContext, data.frame(x = c(1, 2, 3, 4, 100))) + df <- as.DataFrame(data.frame(x = c(1, 2, 3, 4, 100))) expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) }) test_that("dapply() and dapplyCollect() on a DataFrame", { - df <- createDataFrame ( - sqlContext, + df <- createDataFrame( list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), c("a", "b", "c")) ldf <- collect(df) @@ -2102,8 +2104,7 @@ test_that("dapply() and dapplyCollect() on a DataFrame", { }) test_that("repartition by columns on DataFrame", { - df <- createDataFrame ( - sqlContext, + df <- createDataFrame( list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), c("a", "b", "c", "d")) @@ -2138,16 +2139,8 @@ test_that("repartition by columns on DataFrame", { }) test_that("Window functions on a DataFrame", { - ssc <- callJMethod(sc, "sc") - hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) - }, - error = function(err) { - skip("Hive is not build with SparkSQL, skipped") - }) - - df <- createDataFrame(hiveCtx, - list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), + setHiveContext(sc) + df <- createDataFrame(list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), schema = c("key", "value")) ws <- orderBy(window.partitionBy("key"), "value") result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) @@ -2171,6 +2164,31 @@ test_that("Window functions on a DataFrame", { result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) names(result) <- c("key", "value") expect_equal(result, expected) + unsetHiveContext() +}) + +test_that("createDataFrame sqlContext parameter backward compatibility", { + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + df <- suppressWarnings(createDataFrame(sqlContext, 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) + + df2 <- suppressWarnings(createDataFrame(sqlContext, iris)) + expect_equal(count(df2), 150) + expect_equal(ncol(df2), 5) + + df3 <- suppressWarnings(read.df(sqlContext, jsonPath, "json")) + expect_is(df3, "SparkDataFrame") + expect_equal(count(df3), 3) + + before <- suppressWarnings(createDataFrame(sqlContext, iris)) + after <- suppressWarnings(createDataFrame(iris)) + expect_equal(collect(before), collect(after)) }) unlink(parquetPath) From 0cb69a918220304ec46c5c1f4032796fda0d449d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 26 May 2016 13:55:22 -0500 Subject: [PATCH 0397/1470] [SPARK-13148][YARN] document zero-keytab Oozie application launch; add diagnostics This patch provides detail on what to do for keytabless Oozie launches of spark apps, and adds some debug-level diagnostics of what credentials have been submitted Author: Steve Loughran Author: Steve Loughran Closes #11033 from steveloughran/stevel/feature/SPARK-13148-oozie. (cherry picked from commit 01b350a4f7c17d6516b27b6cd27ba8390834d40c) Signed-off-by: Tom Graves --- .../apache/spark/deploy/SparkHadoopUtil.scala | 51 +++++++++- docs/running-on-yarn.md | 96 +++++++++++++++++++ .../org/apache/spark/deploy/yarn/Client.scala | 3 + 3 files changed, 148 insertions(+), 2 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 2e9e45a1550fd..7a5fc866bb8bf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -17,10 +17,11 @@ package org.apache.spark.deploy -import java.io.{ByteArrayInputStream, DataInputStream} +import java.io.{ByteArrayInputStream, DataInputStream, IOException} import java.lang.reflect.Method import java.security.PrivilegedExceptionAction -import java.util.{Arrays, Comparator} +import java.text.DateFormat +import java.util.{Arrays, Comparator, Date} import scala.collection.JavaConverters._ import scala.concurrent.duration._ @@ -34,6 +35,8 @@ import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi @@ -357,6 +360,50 @@ class SparkHadoopUtil extends Logging { newConf.setBoolean(confKey, true) newConf } + + /** + * Dump the credentials' tokens to string values. + * + * @param credentials credentials + * @return an iterator over the string values. If no credentials are passed in: an empty list + */ + private[spark] def dumpTokens(credentials: Credentials): Iterable[String] = { + if (credentials != null) { + credentials.getAllTokens.asScala.map(tokenToString) + } else { + Seq() + } + } + + /** + * Convert a token to a string for logging. + * If its an abstract delegation token, attempt to unmarshall it and then + * print more details, including timestamps in human-readable form. + * + * @param token token to convert to a string + * @return a printable string value. + */ + private[spark] def tokenToString(token: Token[_ <: TokenIdentifier]): String = { + val df = DateFormat.getDateTimeInstance(DateFormat.SHORT, DateFormat.SHORT) + val buffer = new StringBuilder(128) + buffer.append(token.toString) + try { + val ti = token.decodeIdentifier + buffer.append("; ").append(ti) + ti match { + case dt: AbstractDelegationTokenIdentifier => + // include human times and the renewer, which the HDFS tokens toString omits + buffer.append("; Renewer: ").append(dt.getRenewer) + buffer.append("; Issued: ").append(df.format(new Date(dt.getIssueDate))) + buffer.append("; Max Date: ").append(df.format(new Date(dt.getMaxDate))) + case _ => + } + } catch { + case e: IOException => + logDebug("Failed to decode $token: $e", e) + } + buffer.toString + } } object SparkHadoopUtil { diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 3bd16bf60c818..f2fbe3ca56a1f 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -476,3 +476,99 @@ If you need a reference to the proper location to put log files in the YARN so t - In `cluster` mode, the local directories used by the Spark executors and the Spark driver will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. In `client` mode, the Spark executors will use the local directories configured for YARN while the Spark driver will use those defined in `spark.local.dir`. This is because the Spark driver does not run on the YARN cluster in `client` mode, only the Spark executors do. - The `--files` and `--archives` options support specifying file names with the # similar to Hadoop. For example you can specify: `--files localtest.txt#appSees.txt` and this will upload the file you have locally named `localtest.txt` into HDFS but this will be linked to by the name `appSees.txt`, and your application should use the name as `appSees.txt` to reference it when running on YARN. - The `--jars` option allows the `SparkContext.addJar` function to work if you are using it with local files and running in `cluster` mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. + +# Running in a Secure Cluster + +As covered in [security](security.html), Kerberos is used in a secure Hadoop cluster to +authenticate principals associated with services and clients. This allows clients to +make requests of these authenticated services; the services to grant rights +to the authenticated principals. + +Hadoop services issue *hadoop tokens* to grant access to the services and data. +Clients must first acquire tokens for the services they will access and pass them along with their +application as it is launched in the YARN cluster. + +For a Spark application to interact with HDFS, HBase and Hive, it must acquire the relevant tokens +using the Kerberos credentials of the user launching the application +—that is, the principal whose identity will become that of the launched Spark application. + +This is normally done at launch time: in a secure cluster Spark will automatically obtain a +token for the cluster's HDFS filesystem, and potentially for HBase and Hive. + +An HBase token will be obtained if HBase is in on classpath, the HBase configuration declares +the application is secure (i.e. `hbase-site.xml` sets `hbase.security.authentication` to `kerberos`), +and `spark.yarn.security.tokens.hbase.enabled` is not set to `false`. + +Similarly, a Hive token will be obtained if Hive is on the classpath, its configuration +includes a URI of the metadata store in `"hive.metastore.uris`, and +`spark.yarn.security.tokens.hive.enabled` is not set to `false`. + +If an application needs to interact with other secure HDFS clusters, then +the tokens needed to access these clusters must be explicitly requested at +launch time. This is done by listing them in the `spark.yarn.access.namenodes` property. + +``` +spark.yarn.access.namenodes hdfs://ireland.example.org:8020/,hdfs://frankfurt.example.org:8020/ +``` + +## Launching your application with Apache Oozie + +Apache Oozie can launch Spark applications as part of a workflow. +In a secure cluster, the launched application will need the relevant tokens to access the cluster's +services. If Spark is launched with a keytab, this is automatic. +However, if Spark is to be launched without a keytab, the responsibility for setting up security +must be handed over to Oozie. + +The details of configuring Oozie for secure clusters and obtaining +credentials for a job can be found on the [Oozie web site](http://oozie.apache.org/) +in the "Authentication" section of the specific release's documentation. + +For Spark applications, the Oozie workflow must be set up for Oozie to request all tokens which +the application needs, including: + +- The YARN resource manager. +- The local HDFS filesystem. +- Any remote HDFS filesystems used as a source or destination of I/O. +- Hive —if used. +- HBase —if used. +- The YARN timeline server, if the application interacts with this. + +To avoid Spark attempting —and then failing— to obtain Hive, HBase and remote HDFS tokens, +the Spark configuration must be set to disable token collection for the services. + +The Spark configuration must include the lines: + +``` +spark.yarn.security.tokens.hive.enabled false +spark.yarn.security.tokens.hbase.enabled false +``` + +The configuration option `spark.yarn.access.namenodes` must be unset. + +## Troubleshooting Kerberos + +Debugging Hadoop/Kerberos problems can be "difficult". One useful technique is to +enable extra logging of Kerberos operations in Hadoop by setting the `HADOOP_JAAS_DEBUG` +environment variable. + +```bash +export HADOOP_JAAS_DEBUG=true +``` + +The JDK classes can be configured to enable extra logging of their Kerberos and +SPNEGO/REST authentication via the system properties `sun.security.krb5.debug` +and `sun.security.spnego.debug=true` + +``` +-Dsun.security.krb5.debug=true -Dsun.security.spnego.debug=true +``` + +All these options can be enabled in the Application Master: + +``` +spark.yarn.appMasterEnv.HADOOP_JAAS_DEBUG true +spark.yarn.am.extraJavaOptions -Dsun.security.krb5.debug=true -Dsun.security.spnego.debug=true +``` + +Finally, if the log level for `org.apache.spark.deploy.yarn.Client` is set to `DEBUG`, the log +will include a list of all tokens obtained, and their expiry details 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 a12391d08180d..598eb17d5d4a0 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 @@ -375,6 +375,9 @@ private[spark] class Client( val distributedNames = new HashSet[String] YarnSparkHadoopUtil.get.obtainTokenForHiveMetastore(sparkConf, hadoopConf, credentials) YarnSparkHadoopUtil.get.obtainTokenForHBase(sparkConf, hadoopConf, credentials) + if (credentials != null) { + logDebug(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n")) + } val replication = sparkConf.get(STAGING_FILE_REPLICATION).map(_.toShort) .getOrElse(fs.getDefaultReplication(destDir)) From 69b3e9ceeaba8b1b836e93c0164566ed2e25a84e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 May 2016 12:04:18 -0700 Subject: [PATCH 0398/1470] [SPARK-15539][SQL] DROP TABLE throw exception if table doesn't exist ## What changes were proposed in this pull request? Same as #13302, but for DROP TABLE. ## How was this patch tested? `DDLSuite` Author: Andrew Or Closes #13307 from andrewor14/drop-table. (cherry picked from commit 2b1ac6cea882246ef0e655bb2c134ef1656a5068) Signed-off-by: Andrew Or --- .../spark/sql/execution/command/ddl.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 10 ++-- .../execution/HiveCompatibilitySuite.scala | 52 ++++++++++--------- .../HiveWindowFunctionQuerySuite.scala | 10 ++-- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../spark/sql/hive/QueryPartitionSuite.scala | 4 +- 6 files changed, 42 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index dd3f17d525abd..ffea6285528a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -185,7 +185,7 @@ case class DropTableCommand( if (!catalog.tableExists(tableName)) { if (!ifExists) { val objectName = if (isView) "View" else "Table" - logError(s"$objectName '${tableName.quotedString}' does not exist") + throw new AnalysisException(s"$objectName to drop '$tableName' does not exist") } } else { // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view @@ -202,7 +202,7 @@ case class DropTableCommand( try { sparkSession.cacheManager.tryUncacheQuery(sparkSession.table(tableName.quotedString)) } catch { - case NonFatal(e) => log.warn(s"${e.getMessage}", e) + case NonFatal(e) => log.warn(e.toString, e) } catalog.invalidateTable(tableName) catalog.dropTable(tableName, ifExists) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 64f5a4ac479a7..bddd3f2119ae2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -741,14 +741,12 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("DROP TABLE dbx.tab1") assert(catalog.listTables("dbx") == Nil) sql("DROP TABLE IF EXISTS dbx.tab1") - // no exception will be thrown - sql("DROP TABLE dbx.tab1") + intercept[AnalysisException] { + sql("DROP TABLE dbx.tab1") + } } - test("drop view in SQLContext") { - // SQLContext does not support create view. Log an error message, if tab1 does not exists - sql("DROP VIEW tab1") - + test("drop view") { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) createDatabase(catalog, "dbx") 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 a8645f7cd31d7..2d5a970c12006 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 @@ -515,7 +515,33 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "plan_json", // This test uses CREATE EXTERNAL TABLE without specifying LOCATION - "alter2" + "alter2", + + // These tests DROP TABLE that don't exist (but do not specify IF EXISTS) + "alter_rename_partition1", + "date_1", + "date_4", + "date_join1", + "date_serde", + "insert_compressed", + "lateral_view_cp", + "leftsemijoin", + "mapjoin_subquery2", + "nomore_ambiguous_table_col", + "partition_date", + "partition_varchar1", + "ppd_repeated_alias", + "push_or", + "reducesink_dedup", + "subquery_in", + "subquery_notin_having", + "timestamp_3", + "timestamp_lazy", + "udaf_covar_pop", + "union31", + "union_date", + "varchar_2", + "varchar_join1" ) /** @@ -529,7 +555,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "add_partition_with_whitelist", "alias_casted_column", "alter_partition_with_whitelist", - "alter_rename_partition", "ambiguous_col", "annotate_stats_join", "annotate_stats_limit", @@ -606,12 +631,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "database_drop", "database_location", "database_properties", - "date_1", "date_2", - "date_4", "date_comparison", - "date_join1", - "date_serde", "decimal_1", "decimal_4", "decimal_join", @@ -737,7 +758,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "insert1", "insert1_overwrite_partitions", "insert2_overwrite_partitions", - "insert_compressed", "join1", "join10", "join11", @@ -793,10 +813,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join_reorder4", "join_star", "lateral_view", - "lateral_view_cp", "lateral_view_noalias", "lateral_view_ppd", - "leftsemijoin", "leftsemijoin_mr", "limit_pushdown_negative", "lineage1", @@ -824,7 +842,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "mapjoin_filter_on_outerjoin", "mapjoin_mapjoin", "mapjoin_subquery", - "mapjoin_subquery2", "mapjoin_test_outer", "mapreduce1", "mapreduce2", @@ -846,7 +863,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "multi_join_union", "multigroupby_singlemr", "noalias_subq1", - "nomore_ambiguous_table_col", "nonblock_op_deduplicate", "notable_alias1", "notable_alias2", @@ -870,10 +886,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "part_inherit_tbl_props", "part_inherit_tbl_props_with_star", "partcols1", - "partition_date", "partition_serde_format", "partition_type_check", - "partition_varchar1", "partition_wise_fileformat9", "ppd1", "ppd2", @@ -893,7 +907,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ppd_outer_join4", "ppd_outer_join5", "ppd_random", - "ppd_repeated_alias", "ppd_udf_col", "ppd_union", "ppr_allchildsarenull", @@ -901,7 +914,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ppr_pushdown2", "ppr_pushdown3", "progress_1", - "push_or", "query_with_semi", "quote1", "quote2", @@ -913,7 +925,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "reduce_deduplicate_exclude_gby", "reduce_deduplicate_exclude_join", "reduce_deduplicate_extended", - "reducesink_dedup", "router_join_ppr", "select_as_omitted", "select_unquote_and", @@ -936,20 +947,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "subquery_exists_having", "subquery_notexists", "subquery_notexists_having", - "subquery_in", "subquery_in_having", - "subquery_notin_having", "tablename_with_select", - "timestamp_3", "timestamp_comparison", - "timestamp_lazy", "timestamp_null", "transform_ppr1", "transform_ppr2", "type_cast_1", "type_widening", "udaf_collect_set", - "udaf_covar_pop", "udaf_histogram_numeric", "udf2", "udf5", @@ -1113,7 +1119,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union29", "union3", "union30", - "union31", "union33", "union34", "union4", @@ -1122,13 +1127,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union7", "union8", "union9", - "union_date", "union_lateralview", "union_ppr", "union_remove_6", "union_script", - "varchar_2", - "varchar_join1", "varchar_union1", "view", "view_cast", diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index de592f8d937dd..6c3978154d4b6 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -826,15 +826,17 @@ class HiveWindowFunctionQueryFileSuite "windowing_ntile", "windowing_udaf", "windowing_windowspec", - "windowing_rank" - ) + "windowing_rank", - override def whiteList: Seq[String] = Seq( - "windowing_udaf2", + // These tests DROP TABLE that don't exist (but do not specify IF EXISTS) "windowing_columnPruning", "windowing_adjust_rowcontainer_sz" ) + override def whiteList: Seq[String] = Seq( + "windowing_udaf2" + ) + // Only run those query tests in the realWhileList (do not try other ignored query files). override def testCases: Seq[(String, File)] = super.testCases.filter { case (name, _) => realWhiteList.contains(name) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 686c63065dfcc..153b0c3c72a78 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -548,7 +548,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv }.getMessage.contains("Unable to infer schema"), "We should complain that path is not specified.") - sql("DROP TABLE createdJsonTable") + sql("DROP TABLE IF EXISTS createdJsonTable") } test("scan a parquet table created through a CTAS 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 cc05e56d66164..266fdd6c1f19f 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 @@ -61,8 +61,8 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl checkAnswer(sql("select key,value from table_with_partition"), testData.toDF.collect ++ testData.toDF.collect ++ testData.toDF.collect) - sql("DROP TABLE table_with_partition") - sql("DROP TABLE createAndInsertTest") + sql("DROP TABLE IF EXISTS table_with_partition") + sql("DROP TABLE IF EXISTS createAndInsertTest") } } } From eb0c49799880acf2c35b95984b17fcb0ad7b5eca Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 26 May 2016 12:05:47 -0700 Subject: [PATCH 0399/1470] [SPARK-15520][SQL] Also set sparkContext confs when using SparkSession builder in pyspark ## What changes were proposed in this pull request? Also sets confs in the underlying sc when using SparkSession.builder.getOrCreate(). This is a bug-fix from a post-merge comment in https://github.com/apache/spark/pull/13289 ## How was this patch tested? Python doc-tests. Author: Eric Liang Closes #13309 from ericl/spark-15520-1. (cherry picked from commit 594a1bf200fea8d6bcf25839a49186f66f922bc8) Signed-off-by: Andrew Or --- python/pyspark/sql/session.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 52e7f3d348ab9..8f7dcb54a7a2a 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -144,7 +144,7 @@ def getOrCreate(self): default. >>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate() - >>> s1.conf.get("k1") == "v1" + >>> s1.conf.get("k1") == s1.sparkContext.getConf().get("k1") == "v1" True In case an existing SparkSession is returned, the config options specified @@ -168,6 +168,8 @@ def getOrCreate(self): session = SparkSession(sc) for key, value in self._options.items(): session.conf.set(key, value) + for key, value in self._options.items(): + session.sparkContext._conf.set(key, value) return session builder = Builder() From 6eea33ec3ea805a16b812287141d22ce1aa659f9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 26 May 2016 13:03:07 -0700 Subject: [PATCH 0400/1470] [SPARK-15552][SQL] Remove unnecessary private[sql] methods in SparkSession ## What changes were proposed in this pull request? SparkSession has a list of unnecessary private[sql] methods. These methods cause some trouble because private[sql] doesn't apply in Java. In the cases that they are easy to remove, we can simply remove them. This patch does that. As part of this pull request, I also replaced a bunch of protected[sql] with private[sql], to tighten up visibility. ## How was this patch tested? Updated test cases to reflect the changes. Author: Reynold Xin Closes #13319 from rxin/SPARK-15552. (cherry picked from commit 0f61d6efb45b9ee94fa663f67c4489fbdae2eded) Signed-off-by: Andrew Or --- .../apache/spark/sql/DataFrameWriter.scala | 4 +- .../scala/org/apache/spark/sql/Dataset.scala | 32 +++++------ .../org/apache/spark/sql/SQLContext.scala | 49 +++-------------- .../org/apache/spark/sql/SparkSession.scala | 54 ++++++++----------- .../spark/sql/execution/CacheManager.scala | 2 +- .../spark/sql/execution/QueryExecution.scala | 2 +- .../sql/execution/command/commands.scala | 2 +- .../spark/sql/execution/command/ddl.scala | 3 +- .../spark/sql/execution/command/views.scala | 6 +-- .../execution/datasources/DataSource.scala | 2 +- .../InsertIntoDataSourceCommand.scala | 2 +- .../spark/sql/internal/CatalogImpl.scala | 24 ++++----- .../spark/sql/internal/SessionState.scala | 4 +- .../apache/spark/sql/CachedTableSuite.scala | 16 +++--- .../spark/sql/DataFramePivotSuite.scala | 2 +- .../apache/spark/sql/DatasetCacheSuite.scala | 12 +++-- .../org/apache/spark/sql/JoinSuite.scala | 8 +-- .../columnar/InMemoryColumnarQuerySuite.scala | 8 +-- .../execution/metric/SQLMetricsSuite.scala | 18 ++++--- .../sql/execution/ui/SQLListenerSuite.scala | 16 +++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../hive/thriftserver/SparkSQLDriver.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 ++- .../CreateTableAsSelectCommand.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/ShowCreateTableSuite.scala | 4 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../hive/execution/ConcurrentHiveSuite.scala | 6 +-- 29 files changed, 129 insertions(+), 168 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 3aacce7d7f383..2e85e3676747a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -402,7 +402,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { Project(inputDataCols ++ inputPartCols, df.logicalPlan) }.getOrElse(df.logicalPlan) - df.sparkSession.executePlan( + df.sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdent), partitions.getOrElse(Map.empty[String, Option[String]]), @@ -524,7 +524,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { mode, extraOptions.toMap, df.logicalPlan) - df.sparkSession.executePlan(cmd).toRdd + df.sparkSession.sessionState.executePlan(cmd).toRdd } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index e5140fcf1337e..961ae32b0b881 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -58,7 +58,7 @@ private[sql] object Dataset { } def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = { - val qe = sparkSession.executePlan(logicalPlan) + val qe = sparkSession.sessionState.executePlan(logicalPlan) qe.assertAnalyzed() new Dataset[Row](sparkSession, logicalPlan, RowEncoder(qe.analyzed.schema)) } @@ -165,14 +165,14 @@ class Dataset[T] private[sql]( // you wrap it with `withNewExecutionId` if this actions doesn't call other action. def this(sparkSession: SparkSession, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { - this(sparkSession, sparkSession.executePlan(logicalPlan), encoder) + this(sparkSession, sparkSession.sessionState.executePlan(logicalPlan), encoder) } def this(sqlContext: SQLContext, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { this(sqlContext.sparkSession, logicalPlan, encoder) } - @transient protected[sql] val logicalPlan: LogicalPlan = { + @transient private[sql] val logicalPlan: LogicalPlan = { def hasSideEffects(plan: LogicalPlan): Boolean = plan match { case _: Command | _: InsertIntoTable | @@ -215,7 +215,7 @@ class Dataset[T] private[sql]( // sqlContext must be val because a stable identifier is expected when you import implicits @transient lazy val sqlContext: SQLContext = sparkSession.sqlContext - protected[sql] def resolve(colName: String): NamedExpression = { + private[sql] def resolve(colName: String): NamedExpression = { queryExecution.analyzed.resolveQuoted(colName, sparkSession.sessionState.analyzer.resolver) .getOrElse { throw new AnalysisException( @@ -223,7 +223,7 @@ class Dataset[T] private[sql]( } } - protected[sql] def numericColumns: Seq[Expression] = { + private[sql] def numericColumns: Seq[Expression] = { schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => queryExecution.analyzed.resolveQuoted(n.name, sparkSession.sessionState.analyzer.resolver).get } @@ -417,7 +417,7 @@ class Dataset[T] private[sql]( */ def explain(extended: Boolean): Unit = { val explain = ExplainCommand(queryExecution.logical, extended = extended) - sparkSession.executePlan(explain).executedPlan.executeCollect().foreach { + sparkSession.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { // scalastyle:off println r => println(r.getString(0)) // scalastyle:on println @@ -641,7 +641,7 @@ class Dataset[T] private[sql]( def join(right: Dataset[_], usingColumns: Seq[String], joinType: String): DataFrame = { // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right // by creating a new instance for one of the branch. - val joined = sparkSession.executePlan( + val joined = sparkSession.sessionState.executePlan( Join(logicalPlan, right.logicalPlan, joinType = JoinType(joinType), None)) .analyzed.asInstanceOf[Join] @@ -757,7 +757,7 @@ class Dataset[T] private[sql]( val left = this.logicalPlan val right = other.logicalPlan - val joined = sparkSession.executePlan(Join(left, right, joinType = + val joined = sparkSession.sessionState.executePlan(Join(left, right, joinType = JoinType(joinType), Some(condition.expr))) val leftOutput = joined.analyzed.output.take(left.output.length) val rightOutput = joined.analyzed.output.takeRight(right.output.length) @@ -1263,7 +1263,7 @@ class Dataset[T] private[sql]( def groupByKey[K: Encoder](func: T => K): KeyValueGroupedDataset[K, T] = { val inputPlan = logicalPlan val withGroupingKey = AppendColumns(func, inputPlan) - val executed = sparkSession.executePlan(withGroupingKey) + val executed = sparkSession.sessionState.executePlan(withGroupingKey) new KeyValueGroupedDataset( encoderFor[K], @@ -2238,7 +2238,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def persist(): this.type = { - sparkSession.cacheManager.cacheQuery(this) + sparkSession.sharedState.cacheManager.cacheQuery(this) this } @@ -2260,7 +2260,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def persist(newLevel: StorageLevel): this.type = { - sparkSession.cacheManager.cacheQuery(this, None, newLevel) + sparkSession.sharedState.cacheManager.cacheQuery(this, None, newLevel) this } @@ -2273,7 +2273,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def unpersist(blocking: Boolean): this.type = { - sparkSession.cacheManager.tryUncacheQuery(this, blocking) + sparkSession.sharedState.cacheManager.tryUncacheQuery(this, blocking) this } @@ -2294,7 +2294,7 @@ class Dataset[T] private[sql]( lazy val rdd: RDD[T] = { val objectType = unresolvedTEncoder.deserializer.dataType val deserialized = CatalystSerde.deserialize[T](logicalPlan) - sparkSession.executePlan(deserialized).toRdd.mapPartitions { rows => + sparkSession.sessionState.executePlan(deserialized).toRdd.mapPartitions { rows => rows.map(_.get(0, objectType).asInstanceOf[T]) } } @@ -2417,19 +2417,19 @@ class Dataset[T] private[sql]( /** * Converts a JavaRDD to a PythonRDD. */ - protected[sql] def javaToPython: JavaRDD[Array[Byte]] = { + private[sql] def javaToPython: JavaRDD[Array[Byte]] = { val structType = schema // capture it for closure val rdd = queryExecution.toRdd.map(EvaluatePython.toJava(_, structType)) EvaluatePython.javaToPython(rdd) } - protected[sql] def collectToPython(): Int = { + private[sql] def collectToPython(): Int = { withNewExecutionId { PythonRDD.collectAndServe(javaToPython.rdd) } } - protected[sql] def toPythonIterator(): Int = { + private[sql] def toPythonIterator(): Int = { withNewExecutionId { PythonRDD.toLocalIteratorAndServe(javaToPython.rdd) } 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 7013e316ead83..b17fb8a839487 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 @@ -83,12 +83,9 @@ class SQLContext private[sql]( // TODO: move this logic into SparkSession - protected[sql] def sessionState: SessionState = sparkSession.sessionState - protected[sql] def sharedState: SharedState = sparkSession.sharedState - protected[sql] def conf: SQLConf = sessionState.conf - protected[sql] def runtimeConf: RuntimeConfig = sparkSession.conf - protected[sql] def cacheManager: CacheManager = sparkSession.cacheManager - protected[sql] def externalCatalog: ExternalCatalog = sparkSession.externalCatalog + private[sql] def sessionState: SessionState = sparkSession.sessionState + private[sql] def sharedState: SharedState = sparkSession.sharedState + private[sql] def conf: SQLConf = sessionState.conf def sparkContext: SparkContext = sparkSession.sparkContext @@ -167,14 +164,6 @@ class SQLContext private[sql]( sparkSession.conf.getAll } - protected[sql] def parseSql(sql: String): LogicalPlan = sparkSession.parseSql(sql) - - protected[sql] def executeSql(sql: String): QueryExecution = sparkSession.executeSql(sql) - - protected[sql] def executePlan(plan: LogicalPlan): QueryExecution = { - sparkSession.executePlan(plan) - } - /** * :: Experimental :: * A collection of methods that are considered experimental, but can be used to hook into @@ -240,15 +229,6 @@ class SQLContext private[sql]( sparkSession.catalog.isCached(tableName) } - /** - * Returns true if the [[Dataset]] is currently cached in-memory. - * @group cachemgmt - * @since 1.3.0 - */ - private[sql] def isCached(qName: Dataset[_]): Boolean = { - sparkSession.cacheManager.lookupCachedData(qName).nonEmpty - } - /** * Caches the specified table in-memory. * @group cachemgmt @@ -718,26 +698,9 @@ class SQLContext private[sql]( * have the same format as the one generated by `toString` in scala. * It is only used by PySpark. */ - protected[sql] def parseDataType(dataTypeString: String): DataType = { - sparkSession.parseDataType(dataTypeString) - } - - /** - * Apply a schema defined by the schemaString to an RDD. It is only used by PySpark. - */ - protected[sql] def applySchemaToPythonRDD( - rdd: RDD[Array[Any]], - schemaString: String): DataFrame = { - sparkSession.applySchemaToPythonRDD(rdd, schemaString) - } - - /** - * Apply a schema defined by the schema to an RDD. It is only used by PySpark. - */ - protected[sql] def applySchemaToPythonRDD( - rdd: RDD[Array[Any]], - schema: StructType): DataFrame = { - sparkSession.applySchemaToPythonRDD(rdd, schema) + // TODO: Remove this function (would require updating PySpark). + private[sql] def parseDataType(dataTypeString: String): DataType = { + DataType.fromJson(dataTypeString) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 86c97b92dfc3d..a36368afe22a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -34,10 +34,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.ui.SQLListener @@ -51,7 +50,14 @@ import org.apache.spark.util.Utils /** * The entry point to programming Spark with the Dataset and DataFrame API. * - * To create a SparkSession, use the following builder pattern: + * In environments that this has been created upfront (e.g. REPL, notebooks), use the builder + * to get an existing session: + * + * {{{ + * SparkSession.builder().getOrCreate() + * }}} + * + * The builder can also be used to create a new session: * * {{{ * SparkSession.builder() @@ -81,7 +87,7 @@ class SparkSession private( * and a catalog that interacts with external systems. */ @transient - protected[sql] lazy val sharedState: SharedState = { + private[sql] lazy val sharedState: SharedState = { existingSharedState.getOrElse( SparkSession.reflect[SharedState, SparkContext]( SparkSession.sharedStateClassName(sparkContext.conf), @@ -93,7 +99,7 @@ class SparkSession private( * functions, and everything else that accepts a [[org.apache.spark.sql.internal.SQLConf]]. */ @transient - protected[sql] lazy val sessionState: SessionState = { + private[sql] lazy val sessionState: SessionState = { SparkSession.reflect[SessionState, SparkSession]( SparkSession.sessionStateClassName(sparkContext.conf), self) @@ -105,10 +111,6 @@ class SparkSession private( @transient private[sql] val sqlContext: SQLContext = new SQLContext(this) - protected[sql] def cacheManager: CacheManager = sharedState.cacheManager - protected[sql] def listener: SQLListener = sharedState.listener - protected[sql] def externalCatalog: ExternalCatalog = sharedState.externalCatalog - /** * Runtime configuration interface for Spark. * @@ -178,12 +180,14 @@ class SparkSession private( def udf: UDFRegistration = sessionState.udf /** + * :: Experimental :: * Returns a [[ContinuousQueryManager]] that allows managing all the * [[org.apache.spark.sql.ContinuousQuery ContinuousQueries]] active on `this`. * * @group basic * @since 2.0.0 */ + @Experimental def streams: ContinuousQueryManager = sessionState.continuousQueryManager /** @@ -208,13 +212,11 @@ class SparkSession private( * --------------------------------- */ /** - * :: Experimental :: * Returns a [[DataFrame]] with no rows or columns. * * @group dataframes * @since 2.0.0 */ - @Experimental @transient lazy val emptyDataFrame: DataFrame = { createDataFrame(sparkContext.emptyRDD[Row], StructType(Nil)) @@ -449,7 +451,7 @@ class SparkSession private( * Creates a [[DataFrame]] from an RDD[Row]. * User can specify whether the input rows should be converted to Catalyst rows. */ - protected[sql] def internalCreateDataFrame( + private[sql] def internalCreateDataFrame( catalystRows: RDD[InternalRow], schema: StructType): DataFrame = { // TODO: use MutableProjection when rowRDD is another DataFrame and the applied @@ -462,7 +464,7 @@ class SparkSession private( * Creates a [[DataFrame]] from an RDD[Row]. * User can specify whether the input rows should be converted to Catalyst rows. */ - protected[sql] def createDataFrame( + private[sql] def createDataFrame( rowRDD: RDD[Row], schema: StructType, needsConversion: Boolean) = { @@ -502,7 +504,7 @@ class SparkSession private( table(sessionState.sqlParser.parseTableIdentifier(tableName)) } - protected[sql] def table(tableIdent: TableIdentifier): DataFrame = { + private[sql] def table(tableIdent: TableIdentifier): DataFrame = { Dataset.ofRows(self, sessionState.catalog.lookupRelation(tableIdent)) } @@ -510,7 +512,7 @@ class SparkSession private( * Creates a temporary view with a DataFrame. The lifetime of this temporary view is tied to * this [[SparkSession]]. */ - protected[sql] def createTempView( + private[sql] def createTempView( viewName: String, df: DataFrame, replaceIfExists: Boolean) = { sessionState.catalog.createTempView( sessionState.sqlParser.parseTableIdentifier(viewName).table, @@ -529,11 +531,10 @@ class SparkSession private( * @since 2.0.0 */ def sql(sqlText: String): DataFrame = { - Dataset.ofRows(self, parseSql(sqlText)) + Dataset.ofRows(self, sessionState.sqlParser.parsePlan(sqlText)) } /** - * :: Experimental :: * Returns a [[DataFrameReader]] that can be used to read data and streams in as a [[DataFrame]]. * {{{ * sparkSession.read.parquet("/path/to/file.parquet") @@ -543,7 +544,6 @@ class SparkSession private( * @group genericdata * @since 2.0.0 */ - @Experimental def read: DataFrameReader = new DataFrameReader(self) @@ -577,18 +577,6 @@ class SparkSession private( sparkContext.stop() } - protected[sql] def parseSql(sql: String): LogicalPlan = { - sessionState.sqlParser.parsePlan(sql) - } - - protected[sql] def executeSql(sql: String): QueryExecution = { - executePlan(parseSql(sql)) - } - - protected[sql] def executePlan(plan: LogicalPlan): QueryExecution = { - sessionState.executePlan(plan) - } - /** * Parses the data type in our internal string representation. The data type string should * have the same format as the one generated by `toString` in scala. @@ -601,17 +589,17 @@ class SparkSession private( /** * Apply a schema defined by the schemaString to an RDD. It is only used by PySpark. */ - protected[sql] def applySchemaToPythonRDD( + private[sql] def applySchemaToPythonRDD( rdd: RDD[Array[Any]], schemaString: String): DataFrame = { - val schema = parseDataType(schemaString).asInstanceOf[StructType] + val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] applySchemaToPythonRDD(rdd, schema) } /** * Apply a schema defined by the schema to an RDD. It is only used by PySpark. */ - protected[sql] def applySchemaToPythonRDD( + private[sql] def applySchemaToPythonRDD( rdd: RDD[Array[Any]], schema: StructType): DataFrame = { val rowRdd = rdd.map(r => python.EvaluatePython.fromJava(r, schema).asInstanceOf[InternalRow]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index f601138a9d72b..c8bdb0d22c9f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -95,7 +95,7 @@ private[sql] class CacheManager extends Logging { sparkSession.sessionState.conf.useCompression, sparkSession.sessionState.conf.columnBatchSize, storageLevel, - sparkSession.executePlan(planToCache).executedPlan, + sparkSession.sessionState.executePlan(planToCache).executedPlan, tableName)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 34187b9a1ae7f..330459c11ea98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -67,7 +67,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { lazy val withCachedData: LogicalPlan = { assertAnalyzed() assertSupported() - sparkSession.cacheManager.useCachedData(analyzed) + sparkSession.sharedState.cacheManager.useCachedData(analyzed) } lazy val optimizedPlan: LogicalPlan = sparkSession.sessionState.optimizer.execute(withCachedData) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index d5aaccc4bdd90..642a95a99262d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -98,7 +98,7 @@ case class ExplainCommand( // Run through the optimizer to generate the physical plan. override def run(sparkSession: SparkSession): Seq[Row] = try { - val queryExecution = sparkSession.executePlan(logicalPlan) + val queryExecution = sparkSession.sessionState.executePlan(logicalPlan) val outputString = if (codegen) { codegenString(queryExecution.executedPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index ffea6285528a3..7ce7bb903de46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -200,7 +200,8 @@ case class DropTableCommand( case _ => }) try { - sparkSession.cacheManager.tryUncacheQuery(sparkSession.table(tableName.quotedString)) + sparkSession.sharedState.cacheManager.tryUncacheQuery( + sparkSession.table(tableName.quotedString)) } catch { case NonFatal(e) => log.warn(e.toString, e) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 075849afde08c..84990119c9c0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -79,7 +79,7 @@ case class CreateViewCommand( override def run(sparkSession: SparkSession): Seq[Row] = { // If the plan cannot be analyzed, throw an exception and don't proceed. - val qe = sparkSession.executePlan(child) + val qe = sparkSession.sessionState.executePlan(child) qe.assertAnalyzed() val analyzedPlan = qe.analyzed @@ -132,7 +132,7 @@ case class CreateViewCommand( val projectList = analyzedPlan.output.zip(tableDesc.schema).map { case (attr, col) => Alias(attr, col.name)() } - sparkSession.executePlan(Project(projectList, analyzedPlan)).analyzed + sparkSession.sessionState.executePlan(Project(projectList, analyzedPlan)).analyzed } } @@ -153,7 +153,7 @@ case class CreateViewCommand( val projectList = analyzedPlan.output.zip(tableDesc.schema).map { case (attr, col) => Alias(attr, col.name)() } - sparkSession.executePlan(Project(projectList, analyzedPlan)).analyzed + sparkSession.sessionState.executePlan(Project(projectList, analyzedPlan)).analyzed } new SQLBuilder(logicalPlan).toSQL } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index dfe06478fccb2..b3beb6c85f8ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -476,7 +476,7 @@ case class DataSource( options, data.logicalPlan, mode) - sparkSession.executePlan(plan).toRdd + sparkSession.sessionState.executePlan(plan).toRdd case _ => sys.error(s"${providingClass.getCanonicalName} does not allow create table as select.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index c3e07f7d00557..25b901f2db8d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -40,7 +40,7 @@ private[sql] case class InsertIntoDataSourceCommand( relation.insert(df, overwrite) // Invalidate the cache. - sparkSession.cacheManager.invalidateCache(logicalRelation) + sparkSession.sharedState.cacheManager.invalidateCache(logicalRelation) Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 1371abe189f84..f3f36efda599b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -230,7 +230,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { bucketSpec = None, allowExisting = false, managedIfNoPath = false) - sparkSession.executePlan(cmd).toRdd + sparkSession.sessionState.executePlan(cmd).toRdd sparkSession.table(tableIdent) } @@ -278,7 +278,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { bucketSpec = None, allowExisting = false, managedIfNoPath = false) - sparkSession.executePlan(cmd).toRdd + sparkSession.sessionState.executePlan(cmd).toRdd sparkSession.table(tableIdent) } @@ -291,7 +291,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def dropTempView(viewName: String): Unit = { - sparkSession.cacheManager.tryUncacheQuery(sparkSession.table(viewName)) + sparkSession.sharedState.cacheManager.tryUncacheQuery(sparkSession.table(viewName)) sessionCatalog.dropTable(TableIdentifier(viewName), ignoreIfNotExists = true) } @@ -302,7 +302,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def isCached(tableName: String): Boolean = { - sparkSession.cacheManager.lookupCachedData(sparkSession.table(tableName)).nonEmpty + sparkSession.sharedState.cacheManager.lookupCachedData(sparkSession.table(tableName)).nonEmpty } /** @@ -312,7 +312,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def cacheTable(tableName: String): Unit = { - sparkSession.cacheManager.cacheQuery(sparkSession.table(tableName), Some(tableName)) + sparkSession.sharedState.cacheManager.cacheQuery(sparkSession.table(tableName), Some(tableName)) } /** @@ -322,7 +322,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def uncacheTable(tableName: String): Unit = { - sparkSession.cacheManager.uncacheQuery(sparkSession.table(tableName)) + sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession.table(tableName)) } /** @@ -332,7 +332,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def clearCache(): Unit = { - sparkSession.cacheManager.clearCache() + sparkSession.sharedState.cacheManager.clearCache() } /** @@ -342,7 +342,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ protected[sql] def isCached(qName: Dataset[_]): Boolean = { - sparkSession.cacheManager.lookupCachedData(qName).nonEmpty + sparkSession.sharedState.cacheManager.lookupCachedData(qName).nonEmpty } /** @@ -360,15 +360,15 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // cached version and make the new version cached lazily. val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent) // Use lookupCachedData directly since RefreshTable also takes databaseName. - val isCached = sparkSession.cacheManager.lookupCachedData(logicalPlan).nonEmpty + val isCached = sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty if (isCached) { // Create a data frame to represent the table. // TODO: Use uncacheTable once it supports database name. val df = Dataset.ofRows(sparkSession, logicalPlan) // Uncache the logicalPlan. - sparkSession.cacheManager.tryUncacheQuery(df, blocking = true) + sparkSession.sharedState.cacheManager.tryUncacheQuery(df, blocking = true) // Cache it again. - sparkSession.cacheManager.cacheQuery(df, Some(tableIdent.table)) + sparkSession.sharedState.cacheManager.cacheQuery(df, Some(tableIdent.table)) } } @@ -383,7 +383,7 @@ private[sql] object CatalogImpl { val enc = ExpressionEncoder[T]() val encoded = data.map(d => enc.toRow(d).copy()) val plan = new LocalRelation(enc.schema.toAttributes, encoded) - val queryExecution = sparkSession.executePlan(plan) + val queryExecution = sparkSession.sessionState.executePlan(plan) new Dataset[T](sparkSession, queryExecution, enc) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index c9cc2ba04a413..4c7bbf04bc72a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -92,7 +92,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { * Internal catalog for managing table and database states. */ lazy val catalog = new SessionCatalog( - sparkSession.externalCatalog, + sparkSession.sharedState.externalCatalog, functionResourceLoader, functionRegistry, conf, @@ -161,6 +161,8 @@ private[sql] class SessionState(sparkSession: SparkSession) { // Helper methods, partially leftover from pre-2.0 days // ------------------------------------------------------ + def executeSql(sql: String): QueryExecution = executePlan(sqlParser.parsePlan(sql)) + def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan) def invalidateTable(tableName: String): Unit = { 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 1c96bdc05cfcd..e08a9ab7e6914 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 @@ -79,17 +79,17 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("unpersist an uncached table will not raise exception") { - assert(None == spark.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = true) - assert(None == spark.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = false) - assert(None == spark.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.persist() - assert(None != spark.cacheManager.lookupCachedData(testData)) + assert(None != spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = true) - assert(None == spark.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = false) - assert(None == spark.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) } test("cache table as select") { @@ -311,14 +311,14 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") spark.catalog.clearCache() - assert(spark.cacheManager.isEmpty) + assert(spark.sharedState.cacheManager.isEmpty) sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") spark.catalog.cacheTable("t2") sql("Clear CACHE") - assert(spark.cacheManager.isEmpty) + assert(spark.sharedState.cacheManager.isEmpty) } test("Clear accumulators when uncacheTable to prevent memory leaking") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index fa8fa0690721b..d5cb5e15688e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -104,7 +104,7 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ // pivot with extra columns to trigger optimization .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) .agg(sum($"earnings")) - val queryExecution = spark.executePlan(df.queryExecution.logical) + val queryExecution = spark.sessionState.executePlan(df.queryExecution.logical) assert(queryExecution.simpleString.contains("pivotfirst")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 8c0906b74600d..ac9f6c2f38537 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -39,7 +39,8 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { 2, 3, 4) // Drop the cache. cached.unpersist() - assert(spark.cacheManager.lookupCachedData(cached).isEmpty, "The Dataset should not be cached.") + assert(spark.sharedState.cacheManager.lookupCachedData(cached).isEmpty, + "The Dataset should not be cached.") } test("persist and then rebind right encoder when join 2 datasets") { @@ -56,10 +57,10 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { assertCached(joined, 2) ds1.unpersist() - assert(spark.cacheManager.lookupCachedData(ds1).isEmpty, + assert(spark.sharedState.cacheManager.lookupCachedData(ds1).isEmpty, "The Dataset ds1 should not be cached.") ds2.unpersist() - assert(spark.cacheManager.lookupCachedData(ds2).isEmpty, + assert(spark.sharedState.cacheManager.lookupCachedData(ds2).isEmpty, "The Dataset ds2 should not be cached.") } @@ -75,9 +76,10 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { assertCached(agged.filter(_._1 == "b")) ds.unpersist() - assert(spark.cacheManager.lookupCachedData(ds).isEmpty, "The Dataset ds should not be cached.") + assert(spark.sharedState.cacheManager.lookupCachedData(ds).isEmpty, + "The Dataset ds should not be cached.") agged.unpersist() - assert(spark.cacheManager.lookupCachedData(agged).isEmpty, + assert(spark.sharedState.cacheManager.lookupCachedData(agged).isEmpty, "The Dataset agged should not be cached.") } } 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 55836737083de..cbf4a8a612594 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 @@ -60,7 +60,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { } test("join operator selection") { - spark.cacheManager.clearCache() + spark.sharedState.cacheManager.clearCache() withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { @@ -113,7 +113,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { // } test("broadcasted hash join operator selection") { - spark.cacheManager.clearCache() + spark.sharedState.cacheManager.clearCache() sql("CACHE TABLE testData") Seq( ("SELECT * FROM testData join testData2 ON key = a", @@ -127,7 +127,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { } test("broadcasted hash outer join operator selection") { - spark.cacheManager.clearCache() + spark.sharedState.cacheManager.clearCache() sql("CACHE TABLE testData") sql("CACHE TABLE testData2") Seq( @@ -450,7 +450,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { } test("broadcasted existence join operator selection") { - spark.cacheManager.clearCache() + spark.sharedState.cacheManager.clearCache() sql("CACHE TABLE testData") withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index e2fb91352d58e..af3ed14c122d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -33,7 +33,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { setupTestData() test("simple columnar query") { - val plan = spark.executePlan(testData.logicalPlan).sparkPlan + val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) checkAnswer(scan, testData.collect().toSeq) @@ -50,7 +50,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } test("projection") { - val plan = spark.executePlan(testData.select('value, 'key).logicalPlan).sparkPlan + val plan = spark.sessionState.executePlan(testData.select('value, 'key).logicalPlan).sparkPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) checkAnswer(scan, testData.collect().map { @@ -59,7 +59,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { - val plan = spark.executePlan(testData.logicalPlan).sparkPlan + val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) checkAnswer(scan, testData.collect().toSeq) @@ -202,7 +202,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { cached.count() // Make sure, the DataFrame is indeed cached. - assert(spark.cacheManager.lookupCachedData(cached).nonEmpty) + assert(spark.sharedState.cacheManager.lookupCachedData(cached).nonEmpty) // Check result. checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 12940c86fe650..7e9160febdec7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -71,21 +71,22 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { df: DataFrame, expectedNumOfJobs: Int, expectedMetrics: Map[Long, (String, Map[String, Any])]): Unit = { - val previousExecutionIds = spark.listener.executionIdToData.keySet + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet withSQLConf("spark.sql.codegen.wholeStage" -> "false") { df.collect() } sparkContext.listenerBus.waitUntilEmpty(10000) - val executionIds = spark.listener.executionIdToData.keySet.diff(previousExecutionIds) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) assert(executionIds.size === 1) val executionId = executionIds.head - val jobs = spark.listener.getExecution(executionId).get.jobs + val jobs = spark.sharedState.listener.getExecution(executionId).get.jobs // Use "<=" because there is a race condition that we may miss some jobs // TODO Change it to "=" once we fix the race condition that missing the JobStarted event. assert(jobs.size <= expectedNumOfJobs) if (jobs.size == expectedNumOfJobs) { // If we can track all jobs, check the metric values - val metricValues = spark.listener.getExecutionMetrics(executionId) + val metricValues = spark.sharedState.listener.getExecutionMetrics(executionId) val actualMetrics = SparkPlanGraph(SparkPlanInfo.fromSparkPlan( df.queryExecution.executedPlan)).allNodes.filter { node => expectedMetrics.contains(node.id) @@ -283,19 +284,20 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("save metrics") { withTempPath { file => - val previousExecutionIds = spark.listener.executionIdToData.keySet + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet // Assume the execution plan is // PhysicalRDD(nodeId = 0) person.select('name).write.format("json").save(file.getAbsolutePath) sparkContext.listenerBus.waitUntilEmpty(10000) - val executionIds = spark.listener.executionIdToData.keySet.diff(previousExecutionIds) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) assert(executionIds.size === 1) val executionId = executionIds.head - val jobs = spark.listener.getExecution(executionId).get.jobs + val jobs = spark.sharedState.listener.getExecution(executionId).get.jobs // Use "<=" because there is a race condition that we may miss some jobs // TODO Change "<=" to "=" once we fix the race condition that missing the JobStarted event. assert(jobs.size <= 1) - val metricValues = spark.listener.getExecutionMetrics(executionId) + val metricValues = spark.sharedState.listener.getExecutionMetrics(executionId) // Because "save" will create a new DataFrame internally, we cannot get the real metric id. // However, we still can check the value. assert(metricValues.values.toSeq.exists(_ === "2")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 2374ffaaa5036..cf7e976acc65f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -340,16 +340,16 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { } test("SPARK-11126: no memory leak when running non SQL jobs") { - val previousStageNumber = spark.listener.stageIdToStageMetrics.size + val previousStageNumber = spark.sharedState.listener.stageIdToStageMetrics.size spark.sparkContext.parallelize(1 to 10).foreach(i => ()) spark.sparkContext.listenerBus.waitUntilEmpty(10000) // listener should ignore the non SQL stage - assert(spark.listener.stageIdToStageMetrics.size == previousStageNumber) + assert(spark.sharedState.listener.stageIdToStageMetrics.size == previousStageNumber) spark.sparkContext.parallelize(1 to 10).toDF().foreach(i => ()) spark.sparkContext.listenerBus.waitUntilEmpty(10000) // listener should save the SQL stage - assert(spark.listener.stageIdToStageMetrics.size == previousStageNumber + 1) + assert(spark.sharedState.listener.stageIdToStageMetrics.size == previousStageNumber + 1) } test("SPARK-13055: history listener only tracks SQL metrics") { @@ -418,12 +418,12 @@ class SQLListenerMemoryLeakSuite extends SparkFunSuite { } } sc.listenerBus.waitUntilEmpty(10000) - assert(spark.listener.getCompletedExecutions.size <= 50) - assert(spark.listener.getFailedExecutions.size <= 50) + assert(spark.sharedState.listener.getCompletedExecutions.size <= 50) + assert(spark.sharedState.listener.getFailedExecutions.size <= 50) // 50 for successful executions and 50 for failed executions - assert(spark.listener.executionIdToData.size <= 100) - assert(spark.listener.jobIdToExecutionId.size <= 100) - assert(spark.listener.stageIdToStageMetrics.size <= 100) + assert(spark.sharedState.listener.executionIdToData.size <= 100) + assert(spark.sharedState.listener.jobIdToExecutionId.size <= 100) + assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 100) } finally { sc.stop() } 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 9c9abfeb2a637..abb7918ae607b 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 @@ -639,7 +639,7 @@ class JDBCSuite extends SparkFunSuite test("test credentials in the properties are not in plan output") { val df = sql("SELECT * FROM parts") val explain = ExplainCommand(df.queryExecution.logical, extended = true) - spark.executePlan(explain).executedPlan.executeCollect().foreach { + spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { r => assert(!List("testPass", "testUser").exists(r.toString.contains)) } // test the JdbcRelation toString output @@ -651,7 +651,7 @@ class JDBCSuite extends SparkFunSuite test("test credentials in the connection url are not in the plan output") { val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties) val explain = ExplainCommand(df.queryExecution.logical, extended = true) - spark.executePlan(explain).executedPlan.executeCollect().foreach { + spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { r => assert(!List("testPass", "testUser").exists(r.toString.contains)) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index c24e474d9ca46..0d5dc7af5f522 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -59,7 +59,7 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont // TODO unify the error code try { context.sparkContext.setJobDescription(command) - val execution = context.executePlan(context.sql(command).logicalPlan) + val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) hiveResponse = execution.hiveResultString() tableSchema = getResultSetSchema(execution) new CommandProcessorResponse(0) 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 b377a20e3943b..ea721e4d9b6ec 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 @@ -177,8 +177,11 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." - case None => SubqueryAlias(table.identifier.table, sparkSession.parseSql(viewText)) - case Some(aliasText) => SubqueryAlias(aliasText, sparkSession.parseSql(viewText)) + case None => + SubqueryAlias(table.identifier.table, + sparkSession.sessionState.sqlParser.parsePlan(viewText)) + case Some(aliasText) => + SubqueryAlias(aliasText, sessionState.sqlParser.parsePlan(viewText)) } } else { MetastoreRelation( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala index 3fc900961e645..cfe614909532b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala @@ -86,7 +86,7 @@ case class CreateTableAsSelectCommand( throw new AnalysisException(s"$tableIdentifier already exists.") } } else { - sparkSession.executePlan(InsertIntoTable( + sparkSession.sessionState.executePlan(InsertIntoTable( metastoreRelation, Map(), query, overwrite = true, ifNotExists = false)).toRdd } 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 3805674d39589..9e8ff9317c108 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 @@ -296,7 +296,7 @@ case class InsertIntoHiveTable( } // Invalidate the cache. - sqlContext.cacheManager.invalidateCache(table) + sqlContext.sharedState.cacheManager.invalidateCache(table) // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which 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 7c74a0308d483..dc8f374eb178f 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 @@ -130,7 +130,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = hiveContext.parseSql(query) + def ast = hiveContext.sessionState.sqlParser.parsePlan(query) def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala index dedc8f55f01ba..f789d88d5dd4a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -279,13 +279,13 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing private def checkCreateTableOrView(table: TableIdentifier, checkType: String): Unit = { val db = table.database.getOrElse("default") - val expected = spark.externalCatalog.getTable(db, table.table) + val expected = spark.sharedState.externalCatalog.getTable(db, table.table) val shownDDL = sql(s"SHOW CREATE TABLE ${table.quotedString}").head().getString(0) sql(s"DROP $checkType ${table.quotedString}") try { sql(shownDDL) - val actual = spark.externalCatalog.getTable(db, table.table) + val actual = spark.sharedState.externalCatalog.getTable(db, table.table) checkCatalogTables(expected, actual) } finally { sql(s"DROP $checkType IF EXISTS ${table.table}") 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 f8e00a35a31e2..73b1a7850d6fb 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 @@ -34,7 +34,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { - val parsed = hiveContext.parseSql(analyzeCommand) + val parsed = hiveContext.sessionState.sqlParser.parsePlan(analyzeCommand) val operators = parsed.collect { case a: AnalyzeTableCommand => a case o => o diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index f5cd73d45ed75..1583a448efaf7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -30,9 +30,9 @@ class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { conf.set("spark.ui.enabled", "false") val ts = new TestHiveContext(new SparkContext("local", s"TestSQLContext$i", conf)) - ts.executeSql("SHOW TABLES").toRdd.collect() - ts.executeSql("SELECT * FROM src").toRdd.collect() - ts.executeSql("SHOW TABLES").toRdd.collect() + ts.sessionState.executeSql("SHOW TABLES").toRdd.collect() + ts.sessionState.executeSql("SELECT * FROM src").toRdd.collect() + ts.sessionState.executeSql("SHOW TABLES").toRdd.collect() } } } From 216e39505ef8861d12e31d5117fad90e57bed885 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 26 May 2016 14:25:28 -0700 Subject: [PATCH 0401/1470] [SPARK-15457][MLLIB][ML] Eliminate some warnings from MLlib about deprecations ## What changes were proposed in this pull request? Several classes and methods have been deprecated and are creating lots of build warnings in branch-2.0. This issue is to identify and fix those items: * WithSGD classes: Change to make class not deprecated, object deprecated, and public class constructor deprecated. Any public use will require a deprecated API. We need to keep a non-deprecated private API since we cannot eliminate certain uses: Python API, streaming algs, and examples. * Use in PythonMLlibAPI: Change to using private constructors * Streaming algs: No warnings after we un-deprecate the classes * Examples: Deprecate or change ones which use deprecated APIs * MulticlassMetrics fields (precision, etc.) * LinearRegressionSummary.model field ## How was this patch tested? Existing tests. Checked for warnings manually. Author: Sean Owen Author: Joseph K. Bradley Closes #13314 from jkbradley/warning-cleanups. (cherry picked from commit b0a03feef2cf4daa7642ec7f4dc479dbd473b581) Signed-off-by: Joseph K. Bradley --- ...avaLogisticRegressionWithLBFGSExample.java | 4 +- ...ulticlassClassificationMetricsExample.java | 4 +- .../examples/ml/DecisionTreeExample.scala | 2 +- .../examples/mllib/DecisionTreeRunner.scala | 10 ++-- .../mllib/GradientBoostedTreesRunner.scala | 5 +- .../examples/mllib/LinearRegression.scala | 1 + .../LinearRegressionWithSGDExample.scala | 1 + .../LogisticRegressionWithLBFGSExample.scala | 4 +- .../mllib/MulticlassMetricsExample.scala | 8 +-- .../spark/examples/mllib/PCAExample.scala | 1 + .../mllib/RegressionMetricsExample.scala | 2 + .../MulticlassClassificationEvaluator.scala | 4 +- .../ml/regression/LinearRegression.scala | 53 ++++++++++--------- .../mllib/api/python/PythonMLLibAPI.scala | 8 +-- .../classification/LogisticRegression.scala | 4 +- .../apache/spark/mllib/regression/Lasso.scala | 6 +-- .../mllib/regression/LinearRegression.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 6 +-- 18 files changed, 63 insertions(+), 62 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java index 9d8e4a90dbc99..7fc371ec0f990 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java @@ -65,8 +65,8 @@ public Tuple2 call(LabeledPoint p) { // Get evaluation metrics. MulticlassMetrics metrics = new MulticlassMetrics(predictionAndLabels.rdd()); - double precision = metrics.precision(); - System.out.println("Precision = " + precision); + double accuracy = metrics.accuracy(); + System.out.println("Accuracy = " + accuracy); // Save and load model model.save(sc, "target/tmp/javaLogisticRegressionWithLBFGSModel"); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java index 5247c9c748618..e84a3a712df14 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java @@ -68,9 +68,7 @@ public Tuple2 call(LabeledPoint p) { System.out.println("Confusion matrix: \n" + confusion); // Overall statistics - System.out.println("Precision = " + metrics.precision()); - System.out.println("Recall = " + metrics.recall()); - System.out.println("F1 Score = " + metrics.fMeasure()); + System.out.println("Accuracy = " + metrics.accuracy()); // Stats by labels for (int i = 0; i < metrics.labels().length; i++) { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index 5e51dbad760f4..de4474555d2d3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -321,7 +321,7 @@ object DecisionTreeExample { case None => throw new RuntimeException( "Unknown failure when indexing labels for classification.") } - val accuracy = new MulticlassMetrics(predictions.zip(labels)).precision + val accuracy = new MulticlassMetrics(predictions.zip(labels)).accuracy println(s" Accuracy ($numClasses classes): $accuracy") } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index ee811d3aa1015..a85aa2cac9e1b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -295,11 +295,10 @@ object DecisionTreeRunner { } if (params.algo == Classification) { val trainAccuracy = - new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) - .precision + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Train accuracy = $trainAccuracy") val testAccuracy = - new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Test accuracy = $testAccuracy") } if (params.algo == Regression) { @@ -322,11 +321,10 @@ object DecisionTreeRunner { println(model) // Print model summary. } val trainAccuracy = - new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) - .precision + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Train accuracy = $trainAccuracy") val testAccuracy = - new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Test accuracy = $testAccuracy") } if (params.algo == Regression) { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index b0144ef533133..90e4687c1f444 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -120,11 +120,10 @@ object GradientBoostedTreesRunner { println(model) // Print model summary. } val trainAccuracy = - new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) - .precision + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Train accuracy = $trainAccuracy") val testAccuracy = - new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Test accuracy = $testAccuracy") } else if (params.algo == "Regression") { val startTime = System.nanoTime() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index f87611f5d4613..a70203028c858 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -34,6 +34,7 @@ import org.apache.spark.mllib.util.MLUtils * A synthetic dataset can be found at `data/mllib/sample_linear_regression_data.txt`. * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ +@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") object LinearRegression { object RegType extends Enumeration { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala index 669868787e8f0..d399618094487 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala @@ -26,6 +26,7 @@ import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.LinearRegressionWithSGD // $example off$ +@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") object LinearRegressionWithSGDExample { def main(args: Array[String]): Unit = { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala index 632a2d537e5bc..31ba740ad4af0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala @@ -54,8 +54,8 @@ object LogisticRegressionWithLBFGSExample { // Get evaluation metrics. val metrics = new MulticlassMetrics(predictionAndLabels) - val precision = metrics.precision - println("Precision = " + precision) + val accuracy = metrics.accuracy + println(s"Accuracy = $accuracy") // Save and load model model.save(sc, "target/tmp/scalaLogisticRegressionWithLBFGSModel") diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala index 4f925ede24d82..12394c867e973 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala @@ -59,13 +59,9 @@ object MulticlassMetricsExample { println(metrics.confusionMatrix) // Overall Statistics - val precision = metrics.precision - val recall = metrics.recall // same as true positive rate - val f1Score = metrics.fMeasure + val accuracy = metrics.accuracy println("Summary Statistics") - println(s"Precision = $precision") - println(s"Recall = $recall") - println(s"F1 Score = $f1Score") + println(s"Accuracy = $accuracy") // Precision by label val labels = metrics.labels diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala index f7a813695304f..eb36697d94ba1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala @@ -26,6 +26,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} // $example off$ +@deprecated("Deprecated since LinearRegressionWithSGD is deprecated. Use ml.feature.PCA", "2.0.0") object PCAExample { def main(args: Array[String]): Unit = { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala index abeaaa00b5a4f..76cfb804e18f3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -25,6 +25,8 @@ import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} // $example off$ import org.apache.spark.sql.SparkSession +@deprecated("Use ml.regression.LinearRegression and the resulting model summary for metrics", + "2.0.0") object RegressionMetricsExample { def main(args: Array[String]): Unit = { val spark = SparkSession diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 390e9b6444c74..0b84e0a3fa784 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -82,8 +82,8 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid val metrics = new MulticlassMetrics(predictionAndLabels) val metric = $(metricName) match { case "f1" => metrics.weightedFMeasure - case "precision" => metrics.precision - case "recall" => metrics.recall + case "precision" => metrics.accuracy + case "recall" => metrics.accuracy case "weightedPrecision" => metrics.weightedPrecision case "weightedRecall" => metrics.weightedRecall case "accuracy" => metrics.accuracy 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 ff1038cbf185f..37552194c57d3 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 @@ -558,16 +558,18 @@ class LinearRegressionSummary private[regression] ( val predictionCol: String, val labelCol: String, val featuresCol: String, - @deprecated("The model field is deprecated and will be removed in 2.1.0.", "2.0.0") - val model: LinearRegressionModel, + private val privateModel: LinearRegressionModel, private val diagInvAtWA: Array[Double]) extends Serializable { + @deprecated("The model field is deprecated and will be removed in 2.1.0.", "2.0.0") + val model: LinearRegressionModel = privateModel + @transient private val metrics = new RegressionMetrics( predictions .select(col(predictionCol), col(labelCol).cast(DoubleType)) .rdd .map { case Row(pred: Double, label: Double) => (pred, label) }, - !model.getFitIntercept) + !privateModel.getFitIntercept) /** * Returns the explained variance regression score. @@ -631,10 +633,10 @@ class LinearRegressionSummary private[regression] ( lazy val numInstances: Long = predictions.count() /** Degrees of freedom */ - private val degreesOfFreedom: Long = if (model.getFitIntercept) { - numInstances - model.coefficients.size - 1 + private val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { + numInstances - privateModel.coefficients.size - 1 } else { - numInstances - model.coefficients.size + numInstances - privateModel.coefficients.size } /** @@ -642,13 +644,15 @@ class LinearRegressionSummary private[regression] ( * the square root of the instance weights. */ lazy val devianceResiduals: Array[Double] = { - val weighted = if (!model.isDefined(model.weightCol) || model.getWeightCol.isEmpty) { - lit(1.0) - } else { - sqrt(col(model.getWeightCol)) - } - val dr = predictions.select(col(model.getLabelCol).minus(col(model.getPredictionCol)) - .multiply(weighted).as("weightedResiduals")) + val weighted = + if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { + lit(1.0) + } else { + sqrt(col(privateModel.getWeightCol)) + } + val dr = predictions + .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) + .multiply(weighted).as("weightedResiduals")) .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) .first() Array(dr.getDouble(0), dr.getDouble(1)) @@ -668,14 +672,15 @@ class LinearRegressionSummary private[regression] ( throw new UnsupportedOperationException( "No Std. Error of coefficients available for this LinearRegressionModel") } else { - val rss = if (!model.isDefined(model.weightCol) || model.getWeightCol.isEmpty) { - meanSquaredError * numInstances - } else { - val t = udf { (pred: Double, label: Double, weight: Double) => - math.pow(label - pred, 2.0) * weight } - predictions.select(t(col(model.getPredictionCol), col(model.getLabelCol), - col(model.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) - } + val rss = + if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { + meanSquaredError * numInstances + } else { + val t = udf { (pred: Double, label: Double, weight: Double) => + math.pow(label - pred, 2.0) * weight } + predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), + col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) + } val sigma2 = rss / degreesOfFreedom diagInvAtWA.map(_ * sigma2).map(math.sqrt) } @@ -695,10 +700,10 @@ class LinearRegressionSummary private[regression] ( throw new UnsupportedOperationException( "No t-statistic available for this LinearRegressionModel") } else { - val estimate = if (model.getFitIntercept) { - Array.concat(model.coefficients.toArray, Array(model.intercept)) + val estimate = if (privateModel.getFitIntercept) { + Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) } else { - model.coefficients.toArray + privateModel.coefficients.toArray } estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } } 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 90d382753131d..667290ece3514 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 @@ -152,7 +152,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val lrAlg = new LinearRegressionWithSGD() + val lrAlg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0) lrAlg.setIntercept(intercept) .setValidateData(validateData) lrAlg.optimizer @@ -181,7 +181,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val lassoAlg = new LassoWithSGD() + val lassoAlg = new LassoWithSGD(1.0, 100, 0.01, 1.0) lassoAlg.setIntercept(intercept) .setValidateData(validateData) lassoAlg.optimizer @@ -209,7 +209,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val ridgeAlg = new RidgeRegressionWithSGD() + val ridgeAlg = new RidgeRegressionWithSGD(1.0, 100, 0.01, 1.0) ridgeAlg.setIntercept(intercept) .setValidateData(validateData) ridgeAlg.optimizer @@ -268,7 +268,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val LogRegAlg = new LogisticRegressionWithSGD() + val LogRegAlg = new LogisticRegressionWithSGD(1.0, 100, 0.01, 1.0) LogRegAlg.setIntercept(intercept) .setValidateData(validateData) LogRegAlg.optimizer diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index f3c52f61a3bb5..adbcdd302aba9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -200,13 +200,12 @@ object LogisticRegressionModel extends Loader[LogisticRegressionModel] { /** * Train a classification model for Binary Logistic Regression * using Stochastic Gradient Descent. By default L2 regularization is used, - * which can be changed via [[LogisticRegressionWithSGD.optimizer]]. + * which can be changed via `LogisticRegressionWithSGD.optimizer`. * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} * for k classes multi-label classification problem. * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ @Since("0.8.0") -@deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0") class LogisticRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, @@ -229,6 +228,7 @@ class LogisticRegressionWithSGD private[mllib] ( * numIterations: 100, regParm: 0.01, miniBatchFraction: 1.0}. */ @Since("0.8.0") + @deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0") def this() = this(1.0, 100, 0.01, 1.0) override protected[mllib] def createModel(weights: Vector, intercept: Double) = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index ef8c80f0cb807..e14bddf97d0f0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -85,9 +85,7 @@ object LassoModel extends Loader[LassoModel] { * See also the documentation for the precise formulation. */ @Since("0.8.0") -@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " + - "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0") -class LassoWithSGD private ( +class LassoWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, private var regParam: Double, @@ -108,6 +106,8 @@ class LassoWithSGD private ( * regParam: 0.01, miniBatchFraction: 1.0}. */ @Since("0.8.0") + @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " + + "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0") def this() = this(1.0, 100, 0.01, 1.0) override protected def createModel(weights: Vector, intercept: Double) = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 9e9d98bc5e41b..2ceac4b8cc319 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -86,7 +86,6 @@ object LinearRegressionModel extends Loader[LinearRegressionModel] { * See also the documentation for the precise formulation. */ @Since("0.8.0") -@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") class LinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, @@ -108,6 +107,7 @@ class LinearRegressionWithSGD private[mllib] ( * numIterations: 100, miniBatchFraction: 1.0}. */ @Since("0.8.0") + @deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") def this() = this(1.0, 100, 0.0, 1.0) override protected[mllib] def createModel(weights: Vector, intercept: Double) = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 512fb9a712b7a..301f02fd98155 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -86,9 +86,7 @@ object RidgeRegressionModel extends Loader[RidgeRegressionModel] { * See also the documentation for the precise formulation. */ @Since("0.8.0") -@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " + - "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0") -class RidgeRegressionWithSGD private ( +class RidgeRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, private var regParam: Double, @@ -109,6 +107,8 @@ class RidgeRegressionWithSGD private ( * regParam: 0.01, miniBatchFraction: 1.0}. */ @Since("0.8.0") + @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " + + "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0") def this() = this(1.0, 100, 0.01, 1.0) override protected def createModel(weights: Vector, intercept: Double) = { From 7393449dbe94c613ca6808adf35588087ba0d7fa Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Thu, 26 May 2016 14:50:00 -0700 Subject: [PATCH 0402/1470] [SPARK-13445][SQL] Improves error message and add test coverage for Window function ## What changes were proposed in this pull request? Add more verbose error message when order by clause is missed when using Window function. ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13333 from clockfly/spark-13445. (cherry picked from commit b5859e0bb8cc147858cb28d8bdb5ca3b4a2cec77) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 +++- .../scala/org/apache/spark/sql/DataFrameWindowSuite.scala | 8 ++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9a92330f75f6f..bf221e0d7cfc4 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 @@ -1795,7 +1795,9 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan transform { case logical: LogicalPlan => logical transformExpressions { case WindowExpression(wf: WindowFunction, spec) if spec.orderSpec.isEmpty => - failAnalysis(s"WindowFunction $wf requires window to be ordered") + failAnalysis(s"Window function $wf requires window to be ordered, please add ORDER BY " + + s"clause. For example SELECT $wf(value_expr) OVER (PARTITION BY window_partition " + + s"ORDER BY window_ordering) from table") case WindowExpression(rank: RankLike, spec) if spec.resolved => val order = spec.orderSpec.map(_.child) WindowExpression(rank.withOrder(order), spec) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala index 07aad3c406b60..9a1aa46947c91 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala @@ -110,6 +110,14 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { Row(2, 2, 1, 5.0d / 3.0d, 3, 5, 2, 3, 2, 2, 1.0d, 0.5d) :: Nil) } + test("window function should fail if order by clause is not specified") { + val df = Seq((1, "1"), (2, "2"), (1, "2"), (2, "2")).toDF("key", "value") + val e = intercept[AnalysisException]( + // Here we missed .orderBy("key")! + df.select(row_number().over(Window.partitionBy("value"))).collect()) + assert(e.message.contains("requires window to be ordered")) + } + test("aggregation and rows between") { val df = Seq((1, "1"), (2, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") df.createOrReplaceTempView("window_table") From 29681cca109425e1773dccd9331a09b022e8723a Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Thu, 26 May 2016 15:49:16 -0700 Subject: [PATCH 0403/1470] [SPARK-8428][SPARK-13850] Fix integer overflows in TimSort ## What changes were proposed in this pull request? This patch fixes a few integer overflows in `UnsafeSortDataFormat.copyRange()` and `ShuffleSortDataFormat copyRange()` that seems to be the most likely cause behind a number of `TimSort` contract violation errors seen in Spark 2.0 and Spark 1.6 while sorting large datasets. ## How was this patch tested? Added a test in `ExternalSorterSuite` that instantiates a large array of the form of [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999] that triggers a `copyRange` in `TimSort.mergeLo` or `TimSort.mergeHi`. Note that the input dataset should contain at least 268.43 million rows with a certain data distribution for an overflow to occur. Author: Sameer Agarwal Closes #13336 from sameeragarwal/timsort-bug. (cherry picked from commit fe6de16f781ff659b34e0ddda427d371d3d94536) Signed-off-by: Reynold Xin --- .../shuffle/sort/ShuffleSortDataFormat.java | 6 ++--- .../unsafe/sort/UnsafeSortDataFormat.java | 6 ++--- .../util/collection/ExternalSorterSuite.scala | 24 +++++++++++++++++++ 3 files changed, 30 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 8f4e3229976dc..1e924d2aec442 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -61,10 +61,10 @@ public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { Platform.copyMemory( src.getBaseObject(), - src.getBaseOffset() + srcPos * 8, + src.getBaseOffset() + srcPos * 8L, dst.getBaseObject(), - dst.getBaseOffset() + dstPos * 8, - length * 8 + dst.getBaseOffset() + dstPos * 8L, + length * 8L ); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java index d19b71fbc1bcb..7bda76907f4c3 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java @@ -75,10 +75,10 @@ public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { Platform.copyMemory( src.getBaseObject(), - src.getBaseOffset() + srcPos * 16, + src.getBaseOffset() + srcPos * 16L, dst.getBaseObject(), - dst.getBaseOffset() + dstPos * 16, - length * 16); + dst.getBaseOffset() + dstPos * 16L, + length * 16L); } @Override 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 4dd8e31c27351..699f7fa1f2727 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 @@ -17,12 +17,17 @@ package org.apache.spark.util.collection +import java.util.Comparator + import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark._ import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.unsafe.array.LongArray +import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordPointerAndKeyPrefix, UnsafeSortDataFormat} class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { import TestUtils.{assertNotSpilled, assertSpilled} @@ -93,6 +98,25 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { testWithMultipleSer("sort without breaking sorting contracts", loadDefaults = true)( sortWithoutBreakingSortingContracts) + // This test is ignored by default as it requires a fairly large heap size (16GB) + ignore("sort without breaking timsort contracts for large arrays") { + val size = 300000000 + // To manifest the bug observed in SPARK-8428 and SPARK-13850, we explicitly use an array of + // the form [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999] + // that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi() + val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i } + val buf = new LongArray(MemoryBlock.fromLongArray(ref)) + + new Sorter(UnsafeSortDataFormat.INSTANCE).sort( + buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] { + override def compare( + r1: RecordPointerAndKeyPrefix, + r2: RecordPointerAndKeyPrefix): Int = { + PrefixComparators.LONG.compare(r1.keyPrefix, r2.keyPrefix) + } + }) + } + test("spilling with hash collisions") { val size = 1000 val conf = createSparkConf(loadDefaults = true, kryo = false) From 4939c2a12c854a87a020a7e759e4f87810f16710 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 26 May 2016 16:23:48 -0700 Subject: [PATCH 0404/1470] [SPARK-15550][SQL] Dataset.show() should show contents nested products as rows ## What changes were proposed in this pull request? This PR addresses two related issues: 1. `Dataset.showString()` should show case classes/Java beans at all levels as rows, while master code only handles top level ones. 2. `Dataset.showString()` should show full contents produced the underlying query plan Dataset is only a view of the underlying query plan. Columns not referred by the encoder are still reachable using methods like `Dataset.col`. So it probably makes more sense to show full contents of the query plan. ## How was this patch tested? Two new test cases are added in `DatasetSuite` to check `.showString()` output. Author: Cheng Lian Closes #13331 from liancheng/spark-15550-ds-show. (cherry picked from commit e7082caeb4a53c1ee172d136894eece1ac880f65) Signed-off-by: Cheng Lian --- .../scala/org/apache/spark/sql/Dataset.scala | 10 +-- .../org/apache/spark/sql/DatasetSuite.scala | 68 ++++++++++++++----- 2 files changed, 52 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 961ae32b0b881..85f0cf8a60415 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -237,19 +237,13 @@ class Dataset[T] private[sql]( */ private[sql] def showString(_numRows: Int, truncate: Boolean = true): String = { val numRows = _numRows.max(0) - val takeResult = take(numRows + 1) + val takeResult = toDF().take(numRows + 1) val hasMoreData = takeResult.length > numRows val data = takeResult.take(numRows) // For array values, replace Seq and Array with square brackets // For cells that are beyond 20 characters, replace it with the first 17 and "..." - val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { - case r: Row => r - case tuple: Product => Row.fromTuple(tuple) - case definedByCtor: DefinedByConstructorParams => - Row.fromSeq(ScalaReflection.getConstructorParameterValues(definedByCtor)) - case o => Row(o) - }.map { row => + val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { row => row.toSeq.map { cell => val str = cell match { case null => "null" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 05de79eb2f1d0..32320a6435acb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -22,9 +22,8 @@ import java.sql.{Date, Timestamp} import scala.language.postfixOps -import org.scalatest.words.MatcherWords.be - import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} +import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext @@ -217,7 +216,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkDataset( ds.filter(_._1 == "b").select(expr("_1").as[String]), - ("b")) + "b") } test("foreach") { @@ -436,20 +435,6 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(ds.toString == "[_1: int, _2: int]") } - test("showString: Kryo encoder") { - implicit val kryoEncoder = Encoders.kryo[KryoData] - val ds = Seq(KryoData(1), KryoData(2)).toDS() - - val expectedAnswer = """+-----------+ - || value| - |+-----------+ - ||KryoData(1)| - ||KryoData(2)| - |+-----------+ - |""".stripMargin - assert(ds.showString(10) === expectedAnswer) - } - test("Kryo encoder") { implicit val kryoEncoder = Encoders.kryo[KryoData] val ds = Seq(KryoData(1), KryoData(2)).toDS() @@ -677,7 +662,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("dataset.rdd with generic case class") { - val ds = Seq(Generic(1, 1.0), Generic(2, 2.0)).toDS + val ds = Seq(Generic(1, 1.0), Generic(2, 2.0)).toDS() val ds2 = ds.map(g => Generic(g.id, g.value)) assert(ds.rdd.map(r => r.id).count === 2) assert(ds2.rdd.map(r => r.id).count === 2) @@ -731,6 +716,53 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val df = Seq(1 -> 2).toDF("a", "b") checkAnswer(df.map(row => row)(RowEncoder(df.schema)).select("b", "a"), Row(2, 1)) } + + private def checkShowString[T](ds: Dataset[T], expected: String): Unit = { + val numRows = expected.split("\n").length - 4 + val actual = ds.showString(numRows, truncate = true) + + if (expected != actual) { + fail( + "Dataset.showString() gives wrong result:\n\n" + sideBySide( + "== Expected ==\n" + expected, + "== Actual ==\n" + actual + ).mkString("\n") + ) + } + } + + test("SPARK-15550 Dataset.show() should show contents of the underlying logical plan") { + val df = Seq((1, "foo", "extra"), (2, "bar", "extra")).toDF("b", "a", "c") + val ds = df.as[ClassData] + val expected = + """+---+---+-----+ + || b| a| c| + |+---+---+-----+ + || 1|foo|extra| + || 2|bar|extra| + |+---+---+-----+ + |""".stripMargin + + checkShowString(ds, expected) + } + + test("SPARK-15550 Dataset.show() should show inner nested products as rows") { + val ds = Seq( + NestedStruct(ClassData("foo", 1)), + NestedStruct(ClassData("bar", 2)) + ).toDS() + + val expected = + """+-------+ + || f| + |+-------+ + ||[foo,1]| + ||[bar,2]| + |+-------+ + |""".stripMargin + + checkShowString(ds, expected) + } } case class Generic[T](id: T, value: Double) From d3cd579d7cdc1d2642186041dfbe296b266e4069 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 26 May 2016 16:53:31 -0700 Subject: [PATCH 0405/1470] [SPARK-15532][SQL] SQLContext/HiveContext's public constructors should use SparkSession.build.getOrCreate ## What changes were proposed in this pull request? This PR changes SQLContext/HiveContext's public constructor to use SparkSession.build.getOrCreate and removes isRootContext from SQLContext. ## How was this patch tested? Existing tests. Author: Yin Huai Closes #13310 from yhuai/SPARK-15532. (cherry picked from commit 3ac2363d757cc9cebc627974f17ecda3a263efdf) Signed-off-by: Yin Huai --- .../mllib/classification/LogisticRegression.scala | 2 +- project/MimaExcludes.scala | 2 ++ .../main/scala/org/apache/spark/sql/SQLContext.scala | 10 ++-------- .../main/scala/org/apache/spark/sql/SparkSession.scala | 4 ++-- .../scala/org/apache/spark/sql/hive/HiveContext.scala | 10 ++++------ .../org/apache/spark/sql/hive/test/TestHive.scala | 9 ++++----- 6 files changed, 15 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index adbcdd302aba9..4bba2ea057af7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -437,7 +437,7 @@ class LogisticRegressionWithLBFGS lr.setMaxIter(optimizer.getNumIterations()) lr.setTol(optimizer.getConvergenceTol()) // Convert our input into a DataFrame - val sqlContext = new SQLContext(input.context) + val sqlContext = SQLContext.getOrCreate(input.context) import sqlContext.implicits._ val df = input.map(_.asML).toDF() // Determine if we should cache the DF diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 08c575aaeec78..73debe9da4277 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -54,6 +54,8 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.coalesce"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.rdd.PartitionCoalescer$LocationIterator"), ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.rdd.PartitionCoalescer"), + // SPARK-15532 Remove isRootContext flag from SQLContext. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.isRootContext"), // SPARK-12600 Remove SQL deprecated methods ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$QueryExecution"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$SparkPlanner"), 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 b17fb8a839487..66d9aa2c85799 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 @@ -57,9 +57,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager * @groupname Ungrouped Support functions for language integrated queries * @since 1.0.0 */ -class SQLContext private[sql]( - val sparkSession: SparkSession, - val isRootContext: Boolean) +class SQLContext private[sql](val sparkSession: SparkSession) extends Logging with Serializable { self => @@ -69,13 +67,9 @@ class SQLContext private[sql]( // Note: Since Spark 2.0 this class has become a wrapper of SparkSession, where the // real functionality resides. This class remains mainly for backward compatibility. - private[sql] def this(sparkSession: SparkSession) = { - this(sparkSession, true) - } - @deprecated("Use SparkSession.builder instead", "2.0.0") def this(sc: SparkContext) = { - this(new SparkSession(sc)) + this(SparkSession.builder().sparkContext(sc).getOrCreate()) } @deprecated("Use SparkSession.builder instead", "2.0.0") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index a36368afe22a7..5dabe0e83c1cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -25,7 +25,7 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging @@ -40,7 +40,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.ui.SQLListener -import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState} +import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, LongType, StructType} import org.apache.spark.sql.util.ExecutionListenerManager diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 415d4c0049d40..3cfe93234f24b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -28,15 +28,13 @@ import org.apache.spark.sql.{SparkSession, SQLContext} * Configuration for Hive is read from hive-site.xml on the classpath. */ @deprecated("Use SparkSession.builder.enableHiveSupport instead", "2.0.0") -class HiveContext private[hive]( - _sparkSession: SparkSession, - isRootContext: Boolean) - extends SQLContext(_sparkSession, isRootContext) with Logging { +class HiveContext private[hive](_sparkSession: SparkSession) + extends SQLContext(_sparkSession) with Logging { self => def this(sc: SparkContext) = { - this(new SparkSession(HiveUtils.withHiveExternalCatalog(sc)), true) + this(SparkSession.builder().sparkContext(HiveUtils.withHiveExternalCatalog(sc)).getOrCreate()) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -47,7 +45,7 @@ class HiveContext private[hive]( * and Hive client (both of execution and metadata) with existing HiveContext. */ override def newSession(): HiveContext = { - new HiveContext(sparkSession.newSession(), isRootContext = false) + new HiveContext(sparkSession.newSession()) } protected[sql] override def sessionState: HiveSessionState = { 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 2f6aa36f95827..81964db5477c6 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 @@ -70,16 +70,15 @@ object TestHive * test cases that rely on TestHive must be serialized. */ class TestHiveContext( - @transient override val sparkSession: TestHiveSparkSession, - isRootContext: Boolean) - extends SQLContext(sparkSession, isRootContext) { + @transient override val sparkSession: TestHiveSparkSession) + extends SQLContext(sparkSession) { def this(sc: SparkContext) { - this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc)), true) + this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc))) } override def newSession(): TestHiveContext = { - new TestHiveContext(sparkSession.newSession(), false) + new TestHiveContext(sparkSession.newSession()) } override def sharedState: TestHiveSharedState = sparkSession.sharedState From c99ad9215cea7f0f983ce06e9cbdbcfa7bd092bf Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 May 2016 19:01:41 -0700 Subject: [PATCH 0406/1470] [SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2 ## What changes were proposed in this pull request? Two more changes: (1) Fix truncate table for data source tables (only for cases without `PARTITION`) (2) Disallow truncating external tables or views ## How was this patch tested? `DDLSuite` Author: Andrew Or Closes #13315 from andrewor14/truncate-table. --- .../spark/sql/execution/command/tables.scala | 78 ++++++++++++------- .../sql/execution/command/DDLSuite.scala | 34 ++++++++ 2 files changed, 86 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index bef4c9222c296..e34beec33d560 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -285,41 +285,67 @@ case class TruncateTableCommand( tableName: TableIdentifier, partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand { - override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog + override def run(spark: SparkSession): Seq[Row] = { + val catalog = spark.sessionState.catalog if (!catalog.tableExists(tableName)) { throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does not exist.") - } else if (catalog.isTemporaryTable(tableName)) { + } + if (catalog.isTemporaryTable(tableName)) { throw new AnalysisException( s"Operation not allowed: TRUNCATE TABLE on temporary tables: '$tableName'") - } else { - val locations = if (partitionSpec.isDefined) { - catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) + } + val table = catalog.getTableMetadata(tableName) + if (table.tableType == CatalogTableType.EXTERNAL) { + throw new AnalysisException( + s"Operation not allowed: TRUNCATE TABLE on external tables: '$tableName'") + } + if (table.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( + s"Operation not allowed: TRUNCATE TABLE on views: '$tableName'") + } + val isDatasourceTable = DDLUtils.isDatasourceTable(table) + if (isDatasourceTable && partitionSpec.isDefined) { + throw new AnalysisException( + s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + + s"for tables created using the data sources API: '$tableName'") + } + if (table.partitionColumnNames.isEmpty && partitionSpec.isDefined) { + throw new AnalysisException( + s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + + s"for tables that are not partitioned: '$tableName'") + } + val locations = + if (isDatasourceTable || table.partitionColumnNames.isEmpty) { + Seq(table.storage.locationUri) } else { - val table = catalog.getTableMetadata(tableName) - if (table.partitionColumnNames.nonEmpty) { - catalog.listPartitions(tableName).map(_.storage.locationUri) - } else { - Seq(table.storage.locationUri) - } + catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) } - val hadoopConf = sparkSession.sessionState.newHadoopConf() - locations.foreach { location => - if (location.isDefined) { - val path = new Path(location.get) - try { - val fs = path.getFileSystem(hadoopConf) - fs.delete(path, true) - fs.mkdirs(path) - } catch { - case NonFatal(e) => - throw new AnalysisException( - s"Failed to truncate table '$tableName' when removing data of the path: $path " + - s"because of ${e.toString}") - } + val hadoopConf = spark.sessionState.newHadoopConf() + locations.foreach { location => + if (location.isDefined) { + val path = new Path(location.get) + try { + val fs = path.getFileSystem(hadoopConf) + fs.delete(path, true) + fs.mkdirs(path) + } catch { + case NonFatal(e) => + throw new AnalysisException( + s"Failed to truncate table '$tableName' when removing data of the path: $path " + + s"because of ${e.toString}") } } } + // After deleting the data, invalidate the table to make sure we don't keep around a stale + // file relation in the metastore cache. + spark.sessionState.invalidateTable(tableName.unquotedString) + // Also try to drop the contents of the table from the columnar cache + try { + spark.sharedState.cacheManager.tryUncacheQuery(spark.table(tableName.quotedString)) + } catch { + case NonFatal(e) => + log.warn(s"Exception when attempting to uncache table '$tableName'", e) + } Seq.empty[Row] } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index bddd3f2119ae2..6c038c7735b4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.util.Utils class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { private val escapedIdentifier = "`(.+)`".r @@ -1109,4 +1110,37 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } } + + test("truncate table - datasource table") { + import testImplicits._ + val data = (1 to 10).map { i => (i, i) }.toDF("width", "length") + data.write.saveAsTable("rectangles") + spark.catalog.cacheTable("rectangles") + assume(spark.table("rectangles").collect().nonEmpty, "bad test; table was empty to begin with") + assume(spark.catalog.isCached("rectangles"), "bad test; table was not cached to begin with") + sql("TRUNCATE TABLE rectangles") + assert(spark.table("rectangles").collect().isEmpty) + assert(!spark.catalog.isCached("rectangles")) + // truncating partitioned data source tables is not supported + data.write.partitionBy("length").saveAsTable("rectangles2") + assertUnsupported("TRUNCATE TABLE rectangles PARTITION (width=1)") + assertUnsupported("TRUNCATE TABLE rectangles2 PARTITION (width=1)") + } + + test("truncate table - external table, temporary table, view (not allowed)") { + import testImplicits._ + val path = Utils.createTempDir().getAbsolutePath + (1 to 10).map { i => (i, i) }.toDF("a", "b").createTempView("my_temp_tab") + sql(s"CREATE EXTERNAL TABLE my_ext_tab LOCATION '$path'") + sql(s"CREATE VIEW my_view AS SELECT 1") + assertUnsupported("TRUNCATE TABLE my_temp_tab") + assertUnsupported("TRUNCATE TABLE my_ext_tab") + assertUnsupported("TRUNCATE TABLE my_view") + } + + test("truncate table - non-partitioned table (not allowed)") { + sql("CREATE TABLE my_tab (age INT, name STRING)") + assertUnsupported("TRUNCATE TABLE my_tab PARTITION (age=10)") + } + } From 702755f9292ec2b4cd2a216a931bed6ba29ac549 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Thu, 26 May 2016 21:25:13 -0500 Subject: [PATCH 0407/1470] [SPARK-15542][SPARKR] Make error message clear for script './R/install-dev.sh' when R is missing on Mac https://issues.apache.org/jira/browse/SPARK-15542 ## What changes were proposed in this pull request? When running`./R/install-dev.sh` in **Mac OS EI Captain** environment, I got ``` mbp185-xr:spark xin$ ./R/install-dev.sh usage: dirname path ``` This message is very confusing to me, and then I found R is not properly configured on my Mac when this script is using `$(which R)` to get R home. I tried similar situation on CentOS with R missing, and it's giving me very clear error message while MacOS is not. on CentOS: ``` [rootip-xxx-31-9-xx spark]# which R /usr/bin/which: no R in (/usr/local/sbin:/usr/local/bin:/sbin:/bin:/usr/sbin:/usr/bin:/usr/lib/jvm/java-1.7.0-openjdk.x86_64/bin:/root/bin) ``` but on Mac, if not found then nothing returned and this is causing the confusing message for R build failure and running R/install-dev.sh: ``` mbp185-xr:spark xin$ which R mbp185-xr:spark xin$ ``` Here I just added a clear message for this miss configuration for R when running `R/install-dev.sh`. ``` mbp185-xr:spark xin$ ./R/install-dev.sh Cannot find R home by running 'which R', please make sure R is properly installed. ``` ## How was this patch tested? Manually tested on local machine. Author: Xin Ren Closes #13308 from keypointt/SPARK-15542. (cherry picked from commit 6ab973ec51f3db72de1766ad8d8316b7a58ed0a0) Signed-off-by: Sean Owen --- R/install-dev.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/R/install-dev.sh b/R/install-dev.sh index befd413c4cd26..ada6303a722b7 100755 --- a/R/install-dev.sh +++ b/R/install-dev.sh @@ -38,7 +38,12 @@ pushd $FWDIR > /dev/null if [ ! -z "$R_HOME" ] then R_SCRIPT_PATH="$R_HOME/bin" - else + else + # if system wide R_HOME is not found, then exit + if [ ! `command -v R` ]; then + echo "Cannot find 'R_HOME'. Please specify 'R_HOME' or make sure R is properly installed." + exit 1 + fi R_SCRIPT_PATH="$(dirname $(which R))" fi echo "USING R_HOME = $R_HOME" From 8e26b74fccc8e7b52db1011f6d6e295c6ba0c5aa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 May 2016 20:11:09 -0700 Subject: [PATCH 0408/1470] [SPARK-15583][SQL] Disallow altering datasource properties ## What changes were proposed in this pull request? Certain table properties (and SerDe properties) are in the protected namespace `spark.sql.sources.`, which we use internally for datasource tables. The user should not be allowed to (1) Create a Hive table setting these properties (2) Alter these properties in an existing table Previously, we threw an exception if the user tried to alter the properties of an existing datasource table. However, this is overly restrictive for datasource tables and does not do anything for Hive tables. ## How was this patch tested? DDLSuite Author: Andrew Or Closes #13341 from andrewor14/alter-table-props. (cherry picked from commit 3fca635b4ed322208debcd89a539e42cdde6bbd4) Signed-off-by: Yin Huai --- .../command/createDataSourceTables.scala | 17 ++ .../spark/sql/execution/command/ddl.scala | 37 +++-- .../spark/sql/execution/command/tables.scala | 2 + .../sql/execution/command/DDLSuite.scala | 148 +++++++++++------- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 5 files changed, 139 insertions(+), 67 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 6ca66a22df12d..deedb68a7817b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -255,6 +255,23 @@ case class CreateDataSourceTableAsSelectCommand( object CreateDataSourceTableUtils extends Logging { + + // TODO: Actually replace usages with these variables (SPARK-15584) + + val DATASOURCE_PREFIX = "spark.sql.sources." + val DATASOURCE_PROVIDER = DATASOURCE_PREFIX + "provider" + val DATASOURCE_WRITEJOBUUID = DATASOURCE_PREFIX + "writeJobUUID" + val DATASOURCE_OUTPUTPATH = DATASOURCE_PREFIX + "output.path" + val DATASOURCE_SCHEMA_PREFIX = DATASOURCE_PREFIX + "schema." + val DATASOURCE_SCHEMA_NUMPARTS = DATASOURCE_SCHEMA_PREFIX + "numParts" + val DATASOURCE_SCHEMA_NUMPARTCOLS = DATASOURCE_SCHEMA_PREFIX + "numPartCols" + val DATASOURCE_SCHEMA_NUMBUCKETS = DATASOURCE_SCHEMA_PREFIX + "numBuckets" + val DATASOURCE_SCHEMA_NUMBUCKETCOLS = DATASOURCE_SCHEMA_PREFIX + "numBucketCols" + val DATASOURCE_SCHEMA_PART_PREFIX = DATASOURCE_SCHEMA_PREFIX + "part." + val DATASOURCE_SCHEMA_PARTCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "partCol." + val DATASOURCE_SCHEMA_BUCKETCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "bucketCol." + val DATASOURCE_SCHEMA_SORTCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "sortCol." + /** * Checks if the given name conforms the Hive standard ("[a-zA-z_0-9]+"), * i.e. if this name only contains characters, numbers, and _. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 7ce7bb903de46..15eba3b0110b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, CatalogTableType, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils.DATASOURCE_PREFIX import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.types._ @@ -228,15 +229,13 @@ case class AlterTableSetPropertiesCommand( extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { + val ident = if (isView) "VIEW" else "TABLE" val catalog = sparkSession.sessionState.catalog DDLUtils.verifyAlterTableType(catalog, tableName, isView) + DDLUtils.verifyTableProperties(properties.keys.toSeq, s"ALTER $ident") val table = catalog.getTableMetadata(tableName) - val newProperties = table.properties ++ properties - if (DDLUtils.isDatasourceTable(newProperties)) { - throw new AnalysisException("ALTER TABLE SET TBLPROPERTIES is not supported for " + - "tables defined using the datasource API") - } - val newTable = table.copy(properties = newProperties) + // This overrides old properties + val newTable = table.copy(properties = table.properties ++ properties) catalog.alterTable(newTable) Seq.empty[Row] } @@ -260,18 +259,16 @@ case class AlterTableUnsetPropertiesCommand( extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { + val ident = if (isView) "VIEW" else "TABLE" val catalog = sparkSession.sessionState.catalog DDLUtils.verifyAlterTableType(catalog, tableName, isView) + DDLUtils.verifyTableProperties(propKeys, s"ALTER $ident") val table = catalog.getTableMetadata(tableName) - if (DDLUtils.isDatasourceTable(table)) { - throw new AnalysisException( - "alter table properties is not supported for datasource tables") - } if (!ifExists) { propKeys.foreach { k => if (!table.properties.contains(k)) { throw new AnalysisException( - s"attempted to unset non-existent property '$k' in table '$tableName'") + s"Attempted to unset non-existent property '$k' in table '$tableName'") } } } @@ -304,11 +301,15 @@ case class AlterTableSerDePropertiesCommand( "ALTER TABLE attempted to set neither serde class name nor serde properties") override def run(sparkSession: SparkSession): Seq[Row] = { + DDLUtils.verifyTableProperties( + serdeProperties.toSeq.flatMap(_.keys.toSeq), + "ALTER TABLE SERDEPROPERTIES") val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) // Do not support setting serde for datasource tables if (serdeClassName.isDefined && DDLUtils.isDatasourceTable(table)) { - throw new AnalysisException("ALTER TABLE SET SERDE is not supported for datasource tables") + throw new AnalysisException("Operation not allowed: ALTER TABLE SET SERDE is " + + "not supported for tables created with the datasource API") } val newTable = table.withNewStorage( serde = serdeClassName.orElse(table.storage.serde), @@ -489,6 +490,18 @@ object DDLUtils { }) } + /** + * If the given table properties (or SerDe properties) contains datasource properties, + * throw an exception. + */ + def verifyTableProperties(propKeys: Seq[String], operation: String): Unit = { + val datasourceKeys = propKeys.filter(_.startsWith(DATASOURCE_PREFIX)) + if (datasourceKeys.nonEmpty) { + throw new AnalysisException(s"Operation not allowed: $operation property keys may not " + + s"start with '$DATASOURCE_PREFIX': ${datasourceKeys.mkString("[", ", ", "]")}") + } + } + def isTablePartitioned(table: CatalogTable): Boolean = { table.partitionColumns.nonEmpty || table.properties.contains("spark.sql.sources.schema.numPartCols") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index e34beec33d560..d1024090d312d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -118,6 +118,8 @@ case class CreateTableLikeCommand( case class CreateTableCommand(table: CatalogTable, ifNotExists: Boolean) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { + DDLUtils.verifyTableProperties(table.properties.keys.toSeq, "CREATE TABLE") + DDLUtils.verifyTableProperties(table.storage.serdeProperties.keys.toSeq, "CREATE TABLE") sparkSession.sessionState.catalog.createTable(table, ifNotExists) Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 6c038c7735b4d..ff56749387f56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFor import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils.DATASOURCE_PREFIX import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -489,63 +490,19 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("alter table: set properties") { - val catalog = spark.sessionState.catalog - val tableIdent = TableIdentifier("tab1", Some("dbx")) - createDatabase(catalog, "dbx") - createTable(catalog, tableIdent) - assert(catalog.getTableMetadata(tableIdent).properties.isEmpty) - // set table properties - sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('andrew' = 'or14', 'kor' = 'bel')") - assert(catalog.getTableMetadata(tableIdent).properties == - Map("andrew" -> "or14", "kor" -> "bel")) - // set table properties without explicitly specifying database - catalog.setCurrentDatabase("dbx") - sql("ALTER TABLE tab1 SET TBLPROPERTIES ('kor' = 'belle', 'kar' = 'bol')") - assert(catalog.getTableMetadata(tableIdent).properties == - Map("andrew" -> "or14", "kor" -> "belle", "kar" -> "bol")) - // table to alter does not exist - intercept[AnalysisException] { - sql("ALTER TABLE does_not_exist SET TBLPROPERTIES ('winner' = 'loser')") - } - // throw exception for datasource tables - convertToDatasourceTable(catalog, tableIdent) - val e = intercept[AnalysisException] { - sql("ALTER TABLE tab1 SET TBLPROPERTIES ('sora' = 'bol')") - } - assert(e.getMessage.contains("datasource")) + testSetProperties(isDatasourceTable = false) + } + + test("alter table: set properties (datasource table)") { + testSetProperties(isDatasourceTable = true) } test("alter table: unset properties") { - val catalog = spark.sessionState.catalog - val tableIdent = TableIdentifier("tab1", Some("dbx")) - createDatabase(catalog, "dbx") - createTable(catalog, tableIdent) - // unset table properties - sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('j' = 'am', 'p' = 'an', 'c' = 'lan')") - sql("ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('j')") - assert(catalog.getTableMetadata(tableIdent).properties == Map("p" -> "an", "c" -> "lan")) - // unset table properties without explicitly specifying database - catalog.setCurrentDatabase("dbx") - sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('p')") - assert(catalog.getTableMetadata(tableIdent).properties == Map("c" -> "lan")) - // table to alter does not exist - intercept[AnalysisException] { - sql("ALTER TABLE does_not_exist UNSET TBLPROPERTIES ('c' = 'lan')") - } - // property to unset does not exist - val e = intercept[AnalysisException] { - sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('c', 'xyz')") - } - assert(e.getMessage.contains("xyz")) - // property to unset does not exist, but "IF EXISTS" is specified - sql("ALTER TABLE tab1 UNSET TBLPROPERTIES IF EXISTS ('c', 'xyz')") - assert(catalog.getTableMetadata(tableIdent).properties.isEmpty) - // throw exception for datasource tables - convertToDatasourceTable(catalog, tableIdent) - val e1 = intercept[AnalysisException] { - sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('sora')") - } - assert(e1.getMessage.contains("datasource")) + testUnsetProperties(isDatasourceTable = false) + } + + test("alter table: unset properties (datasource table)") { + testUnsetProperties(isDatasourceTable = true) } test("alter table: set serde") { @@ -768,6 +725,78 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { properties = Map("spark.sql.sources.provider" -> "csv"))) } + private def testSetProperties(isDatasourceTable: Boolean): Unit = { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + if (isDatasourceTable) { + convertToDatasourceTable(catalog, tableIdent) + } + def getProps: Map[String, String] = { + catalog.getTableMetadata(tableIdent).properties.filterKeys { k => + !isDatasourceTable || !k.startsWith(DATASOURCE_PREFIX) + } + } + assert(getProps.isEmpty) + // set table properties + sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('andrew' = 'or14', 'kor' = 'bel')") + assert(getProps == Map("andrew" -> "or14", "kor" -> "bel")) + // set table properties without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 SET TBLPROPERTIES ('kor' = 'belle', 'kar' = 'bol')") + assert(getProps == Map("andrew" -> "or14", "kor" -> "belle", "kar" -> "bol")) + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist SET TBLPROPERTIES ('winner' = 'loser')") + } + // datasource table property keys are not allowed + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE tab1 SET TBLPROPERTIES ('${DATASOURCE_PREFIX}foo' = 'loser')") + } + assert(e.getMessage.contains(DATASOURCE_PREFIX + "foo")) + } + + private def testUnsetProperties(isDatasourceTable: Boolean): Unit = { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + if (isDatasourceTable) { + convertToDatasourceTable(catalog, tableIdent) + } + def getProps: Map[String, String] = { + catalog.getTableMetadata(tableIdent).properties.filterKeys { k => + !isDatasourceTable || !k.startsWith(DATASOURCE_PREFIX) + } + } + // unset table properties + sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('j' = 'am', 'p' = 'an', 'c' = 'lan', 'x' = 'y')") + sql("ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('j')") + assert(getProps == Map("p" -> "an", "c" -> "lan", "x" -> "y")) + // unset table properties without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('p')") + assert(getProps == Map("c" -> "lan", "x" -> "y")) + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist UNSET TBLPROPERTIES ('c' = 'lan')") + } + // property to unset does not exist + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('c', 'xyz')") + } + assert(e.getMessage.contains("xyz")) + // property to unset does not exist, but "IF EXISTS" is specified + sql("ALTER TABLE tab1 UNSET TBLPROPERTIES IF EXISTS ('c', 'xyz')") + assert(getProps == Map("x" -> "y")) + // datasource table property keys are not allowed + val e2 = intercept[AnalysisException] { + sql(s"ALTER TABLE tab1 UNSET TBLPROPERTIES ('${DATASOURCE_PREFIX}foo')") + } + assert(e2.getMessage.contains(DATASOURCE_PREFIX + "foo")) + } + private def testSetLocation(isDatasourceTable: Boolean): Unit = { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) @@ -870,6 +899,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { intercept[AnalysisException] { sql("ALTER TABLE does_not_exist SET SERDEPROPERTIES ('x' = 'y')") } + // serde properties must not be a datasource property + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE tab1 SET SERDEPROPERTIES ('${DATASOURCE_PREFIX}foo'='wah')") + } + assert(e.getMessage.contains(DATASOURCE_PREFIX + "foo")) } private def testAddPartitions(isDatasourceTable: Boolean): Unit = { @@ -1091,6 +1125,12 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("create table with datasource properties (not allowed)") { + assertUnsupported("CREATE TABLE my_tab TBLPROPERTIES ('spark.sql.sources.me'='anything')") + assertUnsupported("CREATE TABLE my_tab ROW FORMAT SERDE 'serde' " + + "WITH SERDEPROPERTIES ('spark.sql.sources.me'='anything')") + } + test("drop default database") { Seq("true", "false").foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index e2cef38556ed8..80e6f4ec70220 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -287,7 +287,7 @@ class HiveDDLSuite sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") }.getMessage assert(message.contains( - "attempted to unset non-existent property 'p' in table '`view1`'")) + "Attempted to unset non-existent property 'p' in table '`view1`'")) } } } From 4f66bf5fba6befdb49ef2f8e5e3037cc3e601508 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 26 May 2016 20:55:06 -0700 Subject: [PATCH 0409/1470] [SPARK-8603][SPARKR] Use shell() instead of system2() for SparkR on Windows ## What changes were proposed in this pull request? This PR corrects SparkR to use `shell()` instead of `system2()` on Windows. Using `system2(...)` on Windows does not process windows file separator `\`. `shell(tralsate = TRUE, ...)` can treat this problem. So, this was changed to be chosen according to OS. Existing tests were failed on Windows due to this problem. For example, those were failed. ``` 8. Failure: sparkJars tag in SparkContext (test_includeJAR.R#34) 9. Failure: sparkJars tag in SparkContext (test_includeJAR.R#36) ``` The cases above were due to using of `system2`. In addition, this PR also fixes some tests failed on Windows. ``` 5. Failure: sparkJars sparkPackages as comma-separated strings (test_context.R#128) 6. Failure: sparkJars sparkPackages as comma-separated strings (test_context.R#131) 7. Failure: sparkJars sparkPackages as comma-separated strings (test_context.R#134) ``` The cases above were due to a weird behaviour of `normalizePath()`. On Linux, if the path does not exist, it just prints out the input but it prints out including the current path on Windows. ```r # On Linus path <- normalizePath("aa") print(path) [1] "aa" # On Windows path <- normalizePath("aa") print(path) [1] "C:\\Users\\aa" ``` ## How was this patch tested? Jenkins tests and manually tested in a Window machine as below: Here is the [stdout](https://gist.github.com/HyukjinKwon/4bf35184f3a30f3bce987a58ec2bbbab) of testing. Closes #7025 Author: hyukjinkwon Author: Hyukjin Kwon Author: Prakash PC Closes #13165 from HyukjinKwon/pr/7025. (cherry picked from commit 1c403733b89258e57daf7b8b0a2011981ad7ed8a) Signed-off-by: Shivaram Venkataraman --- R/WINDOWS.md | 2 +- R/pkg/R/client.R | 4 ++-- R/pkg/R/utils.R | 9 +++++++ R/pkg/inst/tests/testthat/test_Windows.R | 26 +++++++++++++++++++++ R/pkg/inst/tests/testthat/test_context.R | 6 ++--- R/pkg/inst/tests/testthat/test_includeJAR.R | 7 +++--- 6 files changed, 44 insertions(+), 10 deletions(-) create mode 100644 R/pkg/inst/tests/testthat/test_Windows.R diff --git a/R/WINDOWS.md b/R/WINDOWS.md index f948ed3974794..f67a1c51d1785 100644 --- a/R/WINDOWS.md +++ b/R/WINDOWS.md @@ -28,6 +28,6 @@ To run the SparkR unit tests on Windows, the following steps are required —ass ``` R -e "install.packages('testthat', repos='http://cran.us.r-project.org')" - .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defualt.name="file:///" R\pkg\tests\run-all.R + .\bin\spark-submit2.cmd --conf spark.hadoop.fs.default.name="file:///" R\pkg\tests\run-all.R ``` diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R index 25e99390a9c89..2d341d836c133 100644 --- a/R/pkg/R/client.R +++ b/R/pkg/R/client.R @@ -38,7 +38,7 @@ determineSparkSubmitBin <- function() { if (.Platform$OS.type == "unix") { sparkSubmitBinName <- "spark-submit" } else { - sparkSubmitBinName <- "spark-submit.cmd" + sparkSubmitBinName <- "spark-submit2.cmd" } sparkSubmitBinName } @@ -69,5 +69,5 @@ launchBackend <- function(args, sparkHome, jars, sparkSubmitOpts, packages) { } combinedArgs <- generateSparkSubmitArgs(args, sparkHome, jars, sparkSubmitOpts, packages) cat("Launching java with spark-submit command", sparkSubmitBin, combinedArgs, "\n") - invisible(system2(sparkSubmitBin, combinedArgs, wait = F)) + invisible(launchScript(sparkSubmitBin, combinedArgs)) } diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 784f7371807e5..e734366131943 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -664,3 +664,12 @@ varargsToJProperties <- function(...) { } props } + +launchScript <- function(script, combinedArgs, capture = FALSE) { + if (.Platform$OS.type == "windows") { + scriptWithArgs <- paste(script, combinedArgs, sep = " ") + shell(scriptWithArgs, translate = TRUE, wait = capture, intern = capture) # nolint + } else { + system2(script, combinedArgs, wait = capture, stdout = capture) + } +} diff --git a/R/pkg/inst/tests/testthat/test_Windows.R b/R/pkg/inst/tests/testthat/test_Windows.R new file mode 100644 index 0000000000000..8813e18a1fa4d --- /dev/null +++ b/R/pkg/inst/tests/testthat/test_Windows.R @@ -0,0 +1,26 @@ +# +# 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("Windows-specific tests") + +test_that("sparkJars tag in SparkContext", { + if (.Platform$OS.type != "windows") { + skip("This test is only for Windows, skipped") + } + testOutput <- launchScript("ECHO", "a/b/c", capture = TRUE) + abcPath <- testOutput[1] + expect_equal(abcPath, "a\\b\\c") +}) diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index c1f1a8932d98f..15915e2d13c29 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -129,13 +129,13 @@ test_that("getClientModeSparkSubmitOpts() returns spark-submit args from whiteli test_that("sparkJars sparkPackages as comma-separated strings", { expect_warning(processSparkJars(" a, b ")) jars <- suppressWarnings(processSparkJars(" a, b ")) - expect_equal(jars, c("a", "b")) + expect_equal(lapply(jars, basename), list("a", "b")) jars <- suppressWarnings(processSparkJars(" abc ,, def ")) - expect_equal(jars, c("abc", "def")) + expect_equal(lapply(jars, basename), list("abc", "def")) jars <- suppressWarnings(processSparkJars(c(" abc ,, def ", "", "xyz", " ", "a,b"))) - expect_equal(jars, c("abc", "def", "xyz", "a", "b")) + expect_equal(lapply(jars, basename), list("abc", "def", "xyz", "a", "b")) p <- processSparkPackages(c("ghi", "lmn")) expect_equal(p, c("ghi", "lmn")) diff --git a/R/pkg/inst/tests/testthat/test_includeJAR.R b/R/pkg/inst/tests/testthat/test_includeJAR.R index f89aa8e507fd5..512dd39cb29ff 100644 --- a/R/pkg/inst/tests/testthat/test_includeJAR.R +++ b/R/pkg/inst/tests/testthat/test_includeJAR.R @@ -21,10 +21,9 @@ runScript <- function() { sparkTestJarPath <- "R/lib/SparkR/test_support/sparktestjar_2.10-1.0.jar" jarPath <- paste("--jars", shQuote(file.path(sparkHome, sparkTestJarPath))) scriptPath <- file.path(sparkHome, "R/lib/SparkR/tests/testthat/jarTest.R") - submitPath <- file.path(sparkHome, "bin/spark-submit") - res <- system2(command = submitPath, - args = c(jarPath, scriptPath), - stdout = TRUE) + submitPath <- file.path(sparkHome, paste("bin/", determineSparkSubmitBin(), sep = "")) + combinedArgs <- paste(jarPath, scriptPath, sep = " ") + res <- launchScript(submitPath, combinedArgs, capture = TRUE) tail(res, 2) } From c1468447e7d532c3e810f715080aae35d6215fae Mon Sep 17 00:00:00 2001 From: felixcheung Date: Thu, 26 May 2016 21:42:36 -0700 Subject: [PATCH 0410/1470] [SPARK-10903] followup - update API doc for SqlContext ## What changes were proposed in this pull request? Follow up on the earlier PR - in here we are fixing up roxygen2 doc examples. Also add to the programming guide migration section. ## How was this patch tested? SparkR tests Author: felixcheung Closes #13340 from felixcheung/sqlcontextdoc. (cherry picked from commit c82883239eadc4615a3aba907cd4633cb7aed26e) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 141 ++++++++++++++++++++++---------------------- R/pkg/R/jobj.R | 3 +- R/pkg/R/mllib.R | 10 ++-- R/pkg/R/stats.R | 12 ++-- docs/sparkr.md | 1 + 5 files changed, 82 insertions(+), 85 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index d54ee54cd8539..30a567523fa56 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -39,7 +39,7 @@ setOldClass("structType") #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df <- createDataFrame(sqlContext, faithful) +#' df <- createDataFrame(faithful) #'} setClass("SparkDataFrame", slots = list(env = "environment", @@ -78,7 +78,7 @@ dataFrame <- function(sdf, isCached = FALSE) { #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' printSchema(df) #'} setMethod("printSchema", @@ -103,7 +103,7 @@ setMethod("printSchema", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' dfSchema <- schema(df) #'} setMethod("schema", @@ -127,7 +127,7 @@ setMethod("schema", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' explain(df, TRUE) #'} setMethod("explain", @@ -158,7 +158,7 @@ setMethod("explain", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' isLocal(df) #'} setMethod("isLocal", @@ -183,7 +183,7 @@ setMethod("isLocal", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' showDF(df) #'} setMethod("showDF", @@ -208,7 +208,7 @@ setMethod("showDF", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' df #'} setMethod("show", "SparkDataFrame", @@ -235,7 +235,7 @@ setMethod("show", "SparkDataFrame", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' dtypes(df) #'} setMethod("dtypes", @@ -262,7 +262,7 @@ setMethod("dtypes", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' columns(df) #' colnames(df) #'} @@ -342,7 +342,7 @@ setMethod("colnames<-", #' @export #' @examples #'\dontrun{ -#' irisDF <- createDataFrame(sqlContext, iris) +#' irisDF <- createDataFrame(iris) #' coltypes(irisDF) #'} setMethod("coltypes", @@ -397,7 +397,7 @@ setMethod("coltypes", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' coltypes(df) <- c("character", "integer") #' coltypes(df) <- c(NA, "numeric") #'} @@ -444,9 +444,9 @@ setMethod("coltypes<-", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' registerTempTable(df, "json_df") -#' new_df <- sql(sqlContext, "SELECT * FROM json_df") +#' new_df <- sql("SELECT * FROM json_df") #'} setMethod("registerTempTable", signature(x = "SparkDataFrame", tableName = "character"), @@ -471,8 +471,8 @@ setMethod("registerTempTable", #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df <- read.df(sqlContext, path, "parquet") -#' df2 <- read.df(sqlContext, path2, "parquet") +#' df <- read.df(path, "parquet") +#' df2 <- read.df(path2, "parquet") #' registerTempTable(df, "table1") #' insertInto(df2, "table1", overwrite = TRUE) #'} @@ -500,7 +500,7 @@ setMethod("insertInto", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' cache(df) #'} setMethod("cache", @@ -528,7 +528,7 @@ setMethod("cache", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' persist(df, "MEMORY_AND_DISK") #'} setMethod("persist", @@ -556,7 +556,7 @@ setMethod("persist", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' persist(df, "MEMORY_AND_DISK") #' unpersist(df) #'} @@ -591,7 +591,7 @@ setMethod("unpersist", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' newDF <- repartition(df, 2L) #' newDF <- repartition(df, numPartitions = 2L) #' newDF <- repartition(df, col = df$"col1", df$"col2") @@ -636,7 +636,7 @@ setMethod("repartition", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' newRDD <- toJSON(df) #'} setMethod("toJSON", @@ -664,7 +664,7 @@ setMethod("toJSON", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' write.json(df, "/tmp/sparkr-tmp/") #'} setMethod("write.json", @@ -691,7 +691,7 @@ setMethod("write.json", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' write.parquet(df, "/tmp/sparkr-tmp1/") #' saveAsParquetFile(df, "/tmp/sparkr-tmp2/") #'} @@ -730,7 +730,7 @@ setMethod("saveAsParquetFile", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.txt" -#' df <- read.text(sqlContext, path) +#' df <- read.text(path) #' write.text(df, "/tmp/sparkr-tmp/") #'} setMethod("write.text", @@ -755,7 +755,7 @@ setMethod("write.text", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' distinctDF <- distinct(df) #'} setMethod("distinct", @@ -791,7 +791,7 @@ setMethod("unique", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' collect(sample(df, FALSE, 0.5)) #' collect(sample(df, TRUE, 0.5)) #'} @@ -834,7 +834,7 @@ setMethod("sample_frac", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' count(df) #' } setMethod("count", @@ -864,7 +864,7 @@ setMethod("nrow", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' ncol(df) #' } setMethod("ncol", @@ -885,7 +885,7 @@ setMethod("ncol", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' dim(df) #' } setMethod("dim", @@ -909,7 +909,7 @@ setMethod("dim", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' collected <- collect(df) #' firstName <- collected[[1]]$name #' } @@ -981,7 +981,7 @@ setMethod("collect", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' limitedDF <- limit(df, 10) #' } setMethod("limit", @@ -1002,7 +1002,7 @@ setMethod("limit", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' take(df, 2) #' } setMethod("take", @@ -1031,7 +1031,7 @@ setMethod("take", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' head(df) #' } setMethod("head", @@ -1054,7 +1054,7 @@ setMethod("head", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' first(df) #' } setMethod("first", @@ -1075,7 +1075,7 @@ setMethod("first", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' rdd <- toRDD(df) #'} setMethod("toRDD", @@ -1188,13 +1188,12 @@ dapplyInternal <- function(x, func, schema) { #' @export #' @examples #' \dontrun{ -#' df <- createDataFrame (sqlContext, iris) +#' df <- createDataFrame(iris) #' df1 <- dapply(df, function(x) { x }, schema(df)) #' collect(df1) #' #' # filter and add a column -#' df <- createDataFrame ( -#' sqlContext, +#' df <- createDataFrame( #' list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), #' c("a", "b", "c")) #' schema <- structType(structField("a", "integer"), structField("b", "double"), @@ -1234,12 +1233,11 @@ setMethod("dapply", #' @export #' @examples #' \dontrun{ -#' df <- createDataFrame (sqlContext, iris) +#' df <- createDataFrame(iris) #' ldf <- dapplyCollect(df, function(x) { x }) #' #' # filter and add a column -#' df <- createDataFrame ( -#' sqlContext, +#' df <- createDataFrame( #' list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), #' c("a", "b", "c")) #' ldf <- dapplyCollect( @@ -1533,7 +1531,7 @@ setMethod("select", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' selectExpr(df, "col1", "(col2 * 5) as newCol") #' } setMethod("selectExpr", @@ -1563,7 +1561,7 @@ setMethod("selectExpr", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' newDF <- withColumn(df, "newCol", df$col1 * 5) #' # Replace an existing column #' newDF2 <- withColumn(newDF, "newCol", newDF$col1) @@ -1592,13 +1590,12 @@ setMethod("withColumn", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' newDF <- mutate(df, newCol = df$col1 * 5, newCol2 = df$col1 * 2) #' names(newDF) # Will contain newCol, newCol2 #' newDF2 <- transform(df, newCol = df$col1 / 5, newCol2 = df$col1 * 2) #' -#' df <- createDataFrame(sqlContext, -#' list(list("Andy", 30L), list("Justin", 19L)), c("name", "age")) +#' df <- createDataFrame(list(list("Andy", 30L), list("Justin", 19L)), c("name", "age")) #' # Replace the "age" column #' df1 <- mutate(df, age = df$age + 1L) #' } @@ -1684,7 +1681,7 @@ setMethod("transform", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' newDF <- withColumnRenamed(df, "col1", "newCol1") #' } setMethod("withColumnRenamed", @@ -1709,7 +1706,7 @@ setMethod("withColumnRenamed", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' newDF <- rename(df, col1 = df$newCol1) #' } setMethod("rename", @@ -1753,7 +1750,7 @@ setClassUnion("characterOrColumn", c("character", "Column")) #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' arrange(df, df$col1) #' arrange(df, asc(df$col1), desc(abs(df$col2))) #' arrange(df, "col1", decreasing = TRUE) @@ -1829,7 +1826,7 @@ setMethod("orderBy", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' filter(df, "col1 > 0") #' filter(df, df$col2 != "abcdefg") #' } @@ -1869,7 +1866,7 @@ setMethod("where", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' dropDuplicates(df) #' dropDuplicates(df, c("col1", "col2")) #' } @@ -1903,8 +1900,8 @@ setMethod("dropDuplicates", #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df1 <- read.json(sqlContext, path) -#' df2 <- read.json(sqlContext, path2) +#' df1 <- read.json(path) +#' df2 <- read.json(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") @@ -1960,8 +1957,8 @@ setMethod("join", #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df1 <- read.json(sqlContext, path) -#' df2 <- read.json(sqlContext, path2) +#' df1 <- read.json(path) +#' df2 <- read.json(path2) #' merge(df1, df2) # Performs a Cartesian #' merge(df1, df2, by = "col1") # Performs an inner join based on expression #' merge(df1, df2, by.x = "col1", by.y = "col2", all.y = TRUE) @@ -2093,8 +2090,8 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df1 <- read.json(sqlContext, path) -#' df2 <- read.json(sqlContext, path2) +#' df1 <- read.json(path) +#' df2 <- read.json(path2) #' unioned <- unionAll(df, df2) #' } setMethod("unionAll", @@ -2136,8 +2133,8 @@ setMethod("rbind", #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df1 <- read.json(sqlContext, path) -#' df2 <- read.json(sqlContext, path2) +#' df1 <- read.json(path) +#' df2 <- read.json(path2) #' intersectDF <- intersect(df, df2) #' } setMethod("intersect", @@ -2163,8 +2160,8 @@ setMethod("intersect", #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) -#' df1 <- read.json(sqlContext, path) -#' df2 <- read.json(sqlContext, path2) +#' df1 <- read.json(path) +#' df2 <- read.json(path2) #' exceptDF <- except(df, df2) #' } #' @rdname except @@ -2205,7 +2202,7 @@ setMethod("except", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' write.df(df, "myfile", "parquet", "overwrite") #' saveDF(df, parquetPath2, "parquet", mode = saveMode, mergeSchema = mergeSchema) #' } @@ -2268,7 +2265,7 @@ setMethod("saveDF", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' saveAsTable(df, "myfile") #' } setMethod("saveAsTable", @@ -2307,7 +2304,7 @@ setMethod("saveAsTable", #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" -#' df <- read.json(sqlContext, path) +#' df <- read.json(path) #' describe(df) #' describe(df, "col1") #' describe(df, "col1", "col2") @@ -2476,7 +2473,7 @@ setMethod("fillna", #' @rdname as.data.frame #' @examples \dontrun{ #' -#' irisDF <- createDataFrame(sqlContext, iris) +#' irisDF <- createDataFrame(iris) #' df <- as.data.frame(irisDF[irisDF$Species == "setosa", ]) #' } setMethod("as.data.frame", @@ -2543,7 +2540,7 @@ setMethod("with", #' @param object a SparkDataFrame #' @examples \dontrun{ #' # Create a SparkDataFrame from the Iris dataset -#' irisDF <- createDataFrame(sqlContext, iris) +#' irisDF <- createDataFrame(iris) #' #' # Show the structure of the SparkDataFrame #' str(irisDF) @@ -2651,7 +2648,7 @@ setMethod("drop", }) #' This function computes a histogram for a given SparkR Column. -#' +#' #' @name histogram #' @title Histogram #' @param nbins the number of bins (optional). Default value is 10. @@ -2661,12 +2658,12 @@ setMethod("drop", #' @rdname histogram #' @family SparkDataFrame functions #' @export -#' @examples +#' @examples #' \dontrun{ -#' +#' #' # Create a SparkDataFrame from the Iris dataset -#' irisDF <- createDataFrame(sqlContext, iris) -#' +#' irisDF <- createDataFrame(iris) +#' #' # Compute histogram statistics #' histStats <- histogram(irisDF, irisDF$Sepal_Length, nbins = 12) #' @@ -2676,8 +2673,8 @@ setMethod("drop", #' require(ggplot2) #' plot <- ggplot(histStats, aes(x = centroids, y = counts)) + #' geom_bar(stat = "identity") + -#' xlab("Sepal_Length") + ylab("Frequency") -#' } +#' xlab("Sepal_Length") + ylab("Frequency") +#' } setMethod("histogram", signature(df = "SparkDataFrame", col = "characterOrColumn"), function(df, col, nbins = 10) { diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R index 898e80648fc2b..5b9142bbd338e 100644 --- a/R/pkg/R/jobj.R +++ b/R/pkg/R/jobj.R @@ -72,8 +72,7 @@ jobj <- function(objId) { #' @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") + name <- getClassName.jobj(x) cat("Java ref type", name, "id", x$id, "\n", sep = " ") } diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index f46681149d5ac..d4152b43b6f5f 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -66,7 +66,7 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' data(iris) -#' df <- createDataFrame(sqlContext, iris) +#' df <- createDataFrame(iris) #' model <- spark.glm(df, Sepal_Length ~ Sepal_Width, family="gaussian") #' summary(model) #' } @@ -114,7 +114,7 @@ setMethod( #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' data(iris) -#' df <- createDataFrame(sqlContext, iris) +#' df <- createDataFrame(iris) #' model <- glm(Sepal_Length ~ Sepal_Width, df, family="gaussian") #' summary(model) #' } @@ -391,7 +391,7 @@ setMethod("predict", signature(object = "KMeansModel"), #' @export #' @examples #' \dontrun{ -#' df <- createDataFrame(sqlContext, infert) +#' df <- createDataFrame(infert) #' model <- spark.naiveBayes(df, education ~ ., laplace = 0) #'} setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "formula"), @@ -414,7 +414,7 @@ setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "form #' @export #' @examples #' \dontrun{ -#' df <- createDataFrame(sqlContext, infert) +#' df <- createDataFrame(infert) #' model <- spark.naiveBayes(df, education ~ ., laplace = 0) #' path <- "path/to/model" #' write.ml(model, path) @@ -545,7 +545,7 @@ read.ml <- function(path) { #' @export #' @examples #' \dontrun{ -#' df <- createDataFrame(sqlContext, ovarian) +#' df <- createDataFrame(ovarian) #' model <- spark.survreg(df, Surv(futime, fustat) ~ ecog_ps + rx) #' } setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula"), diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index 879b664421316..6b53517873a72 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -37,7 +37,7 @@ setOldClass("jobj") #' @export #' @examples #' \dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' ct <- crosstab(df, "title", "gender") #' } setMethod("crosstab", @@ -62,7 +62,7 @@ setMethod("crosstab", #' @export #' @examples #'\dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' cov <- cov(df, "title", "gender") #' } setMethod("cov", @@ -91,7 +91,7 @@ setMethod("cov", #' @export #' @examples #'\dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' corr <- corr(df, "title", "gender") #' corr <- corr(df, "title", "gender", method = "pearson") #' } @@ -120,7 +120,7 @@ setMethod("corr", #' @export #' @examples #' \dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' fi = freqItems(df, c("title", "gender")) #' } setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), @@ -157,7 +157,7 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), #' @export #' @examples #' \dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' quantiles <- approxQuantile(df, "key", c(0.5, 0.8), 0.0) #' } setMethod("approxQuantile", @@ -185,7 +185,7 @@ setMethod("approxQuantile", #' @export #' @examples #'\dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' sample <- sampleBy(df, "key", fractions, 36) #' } setMethod("sampleBy", diff --git a/docs/sparkr.md b/docs/sparkr.md index 6b2ca6d6a55f4..59e486d1929f0 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -392,3 +392,4 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma - The method `table` has been removed and replaced by `tableToDF`. - The class `DataFrame` has been renamed to `SparkDataFrame` to avoid name conflicts. + - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable` From 64d477cd4d6fef7220077ab9cfa7feca461e7a9a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 26 May 2016 22:36:43 -0700 Subject: [PATCH 0411/1470] [MINOR][CORE] Fixed doc for Accumulator2.add ## What changes were proposed in this pull request? Scala doc used outdated ```+=```. Replaced with ```add```. ## How was this patch tested? N/A Author: Joseph K. Bradley Closes #13346 from jkbradley/accum-doc. (cherry picked from commit ee3609a2ef55ae5a2797e5ffe06c2849cbd11e15) Signed-off-by: Reynold Xin --- core/src/main/scala/org/apache/spark/SparkContext.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5018eb38d91c0..5aba2a8c94691 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1304,7 +1304,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** - * Create and register a long accumulator, which starts with 0 and accumulates inputs by `+=`. + * Create and register a long accumulator, which starts with 0 and accumulates inputs by `add`. */ def longAccumulator: LongAccumulator = { val acc = new LongAccumulator @@ -1313,7 +1313,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** - * Create and register a long accumulator, which starts with 0 and accumulates inputs by `+=`. + * Create and register a long accumulator, which starts with 0 and accumulates inputs by `add`. */ def longAccumulator(name: String): LongAccumulator = { val acc = new LongAccumulator @@ -1322,7 +1322,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** - * Create and register a double accumulator, which starts with 0 and accumulates inputs by `+=`. + * Create and register a double accumulator, which starts with 0 and accumulates inputs by `add`. */ def doubleAccumulator: DoubleAccumulator = { val acc = new DoubleAccumulator @@ -1331,7 +1331,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** - * Create and register a double accumulator, which starts with 0 and accumulates inputs by `+=`. + * Create and register a double accumulator, which starts with 0 and accumulates inputs by `add`. */ def doubleAccumulator(name: String): DoubleAccumulator = { val acc = new DoubleAccumulator From 9c7e6ad28e0c0c111c3381adb3c942ea42c96f90 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Thu, 26 May 2016 22:39:14 -0700 Subject: [PATCH 0412/1470] [MINOR] Fix Typos 'a -> an' ## What changes were proposed in this pull request? `a` -> `an` I use regex to generate potential error lines: `grep -in ' a [aeiou]' mllib/src/main/scala/org/apache/spark/ml/*/*scala` and review them line by line. ## How was this patch tested? local build `lint-java` checking Author: Zheng RuiFeng Closes #13317 from zhengruifeng/a_an. (cherry picked from commit 6b1a6180e7bd45b0a0ec47de9f7c7956543f4dfa) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/HeartbeatReceiver.scala | 2 +- .../scala/org/apache/spark/deploy/master/Master.scala | 2 +- .../org/apache/spark/memory/ExecutionMemoryPool.scala | 2 +- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 4 ++-- .../main/scala/org/apache/spark/rpc/netty/Inbox.scala | 2 +- .../apache/spark/scheduler/ExecutorLossReason.scala | 2 +- .../apache/spark/serializer/SerializerManager.scala | 2 +- docs/streaming-custom-receivers.md | 2 +- docs/streaming-programming-guide.md | 4 ++-- .../spark/examples/streaming/JavaCustomReceiver.java | 2 +- .../spark/examples/streaming/CustomReceiver.scala | 2 +- .../spark/ml/classification/LogisticRegression.scala | 4 ++-- .../main/scala/org/apache/spark/ml/param/params.scala | 2 +- .../spark/ml/regression/AFTSurvivalRegression.scala | 2 +- .../apache/spark/ml/regression/LinearRegression.scala | 2 +- .../org/apache/spark/ml/tuning/ParamGridBuilder.scala | 2 +- .../main/scala/org/apache/spark/mllib/fpm/FPTree.scala | 2 +- .../mllib/stat/MultivariateOnlineSummarizer.scala | 2 +- python/pyspark/mllib/regression.py | 6 +++--- python/pyspark/sql/functions.py | 2 +- python/pyspark/sql/readwriter.py | 2 +- python/pyspark/streaming/dstream.py | 2 +- python/pyspark/streaming/kafka.py | 2 +- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../spark/sql/catalyst/analysis/TypeCheckResult.scala | 2 +- .../spark/sql/catalyst/analysis/TypeCoercion.scala | 4 ++-- .../spark/sql/catalyst/catalog/functionResources.scala | 2 +- .../sql/catalyst/expressions/aggregate/First.scala | 2 +- .../sql/catalyst/expressions/aggregate/Last.scala | 2 +- .../catalyst/expressions/aggregate/PivotFirst.scala | 2 +- .../spark/sql/catalyst/expressions/literals.scala | 2 +- .../apache/spark/sql/catalyst/expressions/rows.scala | 2 +- .../spark/sql/catalyst/expressions/subquery.scala | 2 +- .../spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- .../apache/spark/sql/catalyst/parser/ParseDriver.scala | 2 +- .../catalyst/plans/logical/basicLogicalOperators.scala | 4 ++-- .../org/apache/spark/sql/catalyst/plans/package.scala | 2 +- .../sql/catalyst/plans/physical/partitioning.scala | 2 +- .../scala/org/apache/spark/sql/types/Decimal.scala | 2 +- .../scala/org/apache/spark/sql/types/DecimalType.scala | 2 +- .../scala/org/apache/spark/sql/DataFrameWriter.scala | 2 +- .../org/apache/spark/sql/execution/ExpandExec.scala | 2 +- .../apache/spark/sql/execution/ShuffledRowRDD.scala | 2 +- .../spark/sql/execution/WholeStageCodegenExec.scala | 2 +- .../aggregate/TungstenAggregationIterator.scala | 6 +++--- .../spark/sql/execution/basicPhysicalOperators.scala | 6 +++--- .../sql/execution/datasources/WriterContainer.scala | 2 +- .../apache/spark/sql/execution/streaming/Offset.scala | 2 +- .../main/scala/org/apache/spark/sql/functions.scala | 2 +- .../org/apache/spark/sql/internal/CatalogImpl.scala | 2 +- .../sql/hive/thriftserver/HiveThriftServer2.scala | 2 +- .../scala/org/apache/spark/sql/hive/HiveUtils.scala | 2 +- .../spark/sql/hive/execution/InsertIntoHiveTable.scala | 4 ++-- .../apache/spark/sql/hive/orc/OrcFileOperator.scala | 4 ++-- .../scala/org/apache/spark/streaming/Checkpoint.scala | 2 +- .../org/apache/spark/streaming/StreamingContext.scala | 10 +++++----- .../spark/streaming/receiver/ReceiverSupervisor.scala | 2 +- .../streaming/receiver/ReceiverSupervisorImpl.scala | 2 +- .../scheduler/ExecutorAllocationManager.scala | 2 +- .../spark/streaming/scheduler/InputInfoTracker.scala | 2 +- .../org/apache/spark/streaming/ui/BatchPage.scala | 2 +- 61 files changed, 78 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 73495a8d7d7fd..c3764ac671afb 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -166,7 +166,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) } /** - * Send ExecutorRemoved to the event loop to remove a executor. Only for test. + * Send ExecutorRemoved to the event loop to remove an executor. Only for test. * * @return if HeartbeatReceiver is stopped, return None. Otherwise, return a Some(Future) that * indicate if this operation is successful. 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 faed4f4dc9573..f8aac3008cefa 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 @@ -929,7 +929,7 @@ private[deploy] class Master( exec.state = ExecutorState.KILLED } - /** Generate a new app ID given a app's submission date */ + /** Generate a new app ID given an app's submission date */ private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index f8167074c6dfa..f1915857ea43a 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.apache.spark.internal.Logging /** - * Implements policies and bookkeeping for sharing a adjustable-sized pool of memory between tasks. + * Implements policies and bookkeeping for sharing an adjustable-sized pool of memory between tasks. * * Tries to ensure that each task gets a reasonable share of memory, instead of some task ramping up * to a large amount first and then causing others to spill to disk repeatedly. diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index a7142376017d5..104e0cb37155f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -1054,7 +1054,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val warningMessage = s"$outputCommitterClass may be an output committer that writes data directly to " + "the final location. Because speculation is enabled, this output committer may " + - "cause data loss (see the case in SPARK-10063). If possible, please use a output " + + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + "committer that does not have this behavior (e.g. FileOutputCommitter)." logWarning(warningMessage) } @@ -1142,7 +1142,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val warningMessage = s"$outputCommitterClass may be an output committer that writes data directly to " + "the final location. Because speculation is enabled, this output committer may " + - "cause data loss (see the case in SPARK-10063). If possible, please use a output " + + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + "committer that does not have this behavior (e.g. FileOutputCommitter)." logWarning(warningMessage) } diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index fffbd5cd44a23..ae4a6003517cc 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -52,7 +52,7 @@ private[netty] case class RemoteProcessConnectionError(cause: Throwable, remoteA extends InboxMessage /** - * A inbox that stores messages for an [[RpcEndpoint]] and posts messages to it thread-safely. + * An inbox that stores messages for an [[RpcEndpoint]] and posts messages to it thread-safely. */ private[netty] class Inbox( val endpointRef: NettyRpcEndpointRef, diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index 7e1197d742802..642bf81ac087e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark.executor.ExecutorExitCode /** - * Represents an explanation for a executor or whole slave failing or exiting. + * Represents an explanation for an executor or whole slave failing or exiting. */ private[spark] class ExecutorLossReason(val message: String) extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 745ef126913f5..9dc274c9fe288 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -134,7 +134,7 @@ private[spark] class SerializerManager(defaultSerializer: Serializer, conf: Spar } /** - * Deserializes a InputStream into an iterator of values and disposes of it when the end of + * Deserializes an InputStream into an iterator of values and disposes of it when the end of * the iterator is reached. */ def dataDeserializeStream[T: ClassTag]( diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index a4e17fd24eac2..479140f519103 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -36,7 +36,7 @@ Any exception in the receiving threads should be caught and handled properly to failures of the receiver. `restart()` will restart the receiver by asynchronously calling `onStop()` and then calling `onStart()` after a delay. `stop()` will call `onStop()` and terminate the receiver. Also, `reportError()` -reports a error message to the driver (visible in the logs and UI) without stopping / restarting +reports an error message to the driver (visible in the logs and UI) without stopping / restarting the receiver. The following is a custom receiver that receives a stream of text over a socket. It treats diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 4d0a1122dcd12..d7eafff38f35b 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -612,7 +612,7 @@ as well as to run the receiver(s). - When running a Spark Streaming program locally, do not use "local" or "local[1]" as the master URL. Either of these means that only one thread will be used for running tasks locally. If you are using - a input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will + an input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will be used to run the receiver, leaving no thread for processing the received data. Hence, when running locally, always use "local[*n*]" as the master URL, where *n* > number of receivers to run (see [Spark Properties](configuration.html#spark-properties) for information on how to set @@ -1788,7 +1788,7 @@ This example appends the word counts of network data into a file. This behavior is made simple by using `JavaStreamingContext.getOrCreate`. This is used as follows. {% highlight java %} -// Create a factory object that can create a and setup a new JavaStreamingContext +// Create a factory object that can create and setup a new JavaStreamingContext JavaStreamingContextFactory contextFactory = new JavaStreamingContextFactory() { @Override public JavaStreamingContext create() { JavaStreamingContext jssc = new JavaStreamingContext(...); // new context diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index 1cba565b38c2a..e20b94d5b03f2 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -70,7 +70,7 @@ public static void main(String[] args) throws Exception { SparkConf sparkConf = new SparkConf().setAppName("JavaCustomReceiver"); JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); - // Create a input stream with the custom receiver on target ip:port and count the + // Create an input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') JavaReceiverInputDStream lines = ssc.receiverStream( new JavaCustomReceiver(args[0], Integer.parseInt(args[1]))); diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 1d144db9864bd..f70975eeb5c99 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -50,7 +50,7 @@ object CustomReceiver { val sparkConf = new SparkConf().setAppName("CustomReceiver") val ssc = new StreamingContext(sparkConf, Seconds(1)) - // Create a input stream with the custom receiver on target ip:port and count the + // Create an input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') val lines = ssc.receiverStream(new CustomReceiver(args(0), args(1).toInt)) val words = lines.flatMap(_.split(" ")) 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 0ab4459bdb9df..61e355ab9fba1 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 @@ -686,7 +686,7 @@ object LogisticRegressionModel extends MLReadable[LogisticRegressionModel] { /** * MultiClassSummarizer computes the number of distinct labels and corresponding counts, * and validates the data to see if the labels used for k class multi-label classification - * are in the range of {0, 1, ..., k - 1} in a online fashion. + * are in the range of {0, 1, ..., k - 1} in an online fashion. * * Two MultilabelSummarizer can be merged together to have a statistical summary of the * corresponding joint dataset. @@ -923,7 +923,7 @@ class BinaryLogisticRegressionSummary private[classification] ( /** * LogisticAggregator computes the gradient and loss for binary logistic loss function, as used - * in binary classification for instances in sparse or dense vector in a online fashion. + * in binary classification for instances in sparse or dense vector in an online fashion. * * Note that multinomial logistic loss is not supported yet! * 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 82f2de7ccdfdb..ecec61a72f823 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 @@ -790,7 +790,7 @@ trait Params extends Identifiable with Serializable { * :: DeveloperApi :: * Java-friendly wrapper for [[Params]]. * Java developers who need to extend [[Params]] should use this class instead. - * If you need to extend a abstract class which already extends [[Params]], then that abstract + * If you need to extend an abstract class which already extends [[Params]], then that abstract * class should be Java-friendly as well. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index e63eb71080604..00ef6ccc74d28 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -396,7 +396,7 @@ object AFTSurvivalRegressionModel extends MLReadable[AFTSurvivalRegressionModel] /** * AFTAggregator computes the gradient and loss for a AFT loss function, - * as used in AFT survival regression for samples in sparse or dense vector in a online fashion. + * as used in AFT survival regression for samples in sparse or dense vector in an online fashion. * * The loss function and likelihood function under the AFT model based on: * Lawless, J. F., Statistical Models and Methods for Lifetime Data, 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 37552194c57d3..4d66b0eb37ab4 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 @@ -731,7 +731,7 @@ class LinearRegressionSummary private[regression] ( /** * LeastSquaresAggregator computes the gradient and loss for a Least-squared loss function, - * as used in linear regression for samples in sparse or dense vector in a online fashion. + * as used in linear regression for samples in sparse or dense vector in an online fashion. * * Two LeastSquaresAggregator can be merged together to have a summary of loss and gradient of * the corresponding joint dataset. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala index b836d2a2340e6..7d12f447f7963 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala @@ -74,7 +74,7 @@ class ParamGridBuilder @Since("1.2.0") { } /** - * Adds a int param with multiple values. + * Adds an int param with multiple values. */ @Since("1.2.0") def addGrid(param: IntParam, values: Array[Int]): this.type = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala index 1d2d777c00793..b0fa287473c3c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala @@ -126,7 +126,7 @@ private[fpm] object FPTree { def isRoot: Boolean = parent == null } - /** Summary of a item in an FP-Tree. */ + /** Summary of an item in an FP-Tree. */ private class Summary[T] extends Serializable { var count: Long = 0L val nodes: ListBuffer[Node[T]] = ListBuffer.empty diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 98404be2603c7..d4de0fd7d5f4d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -24,7 +24,7 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} * :: DeveloperApi :: * MultivariateOnlineSummarizer implements [[MultivariateStatisticalSummary]] to compute the mean, * variance, minimum, maximum, counts, and nonzero counts for instances in sparse or dense vector - * format in a online fashion. + * format in an online fashion. * * Two MultivariateOnlineSummarizer can be merged together to have a statistical summary of * the corresponding joint dataset. diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 43d9072a24e4d..705022934e41b 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -648,7 +648,7 @@ def predict(self, x): @since("1.4.0") def save(self, sc, path): - """Save a IsotonicRegressionModel.""" + """Save an IsotonicRegressionModel.""" java_boundaries = _py2java(sc, self.boundaries.tolist()) java_predictions = _py2java(sc, self.predictions.tolist()) java_model = sc._jvm.org.apache.spark.mllib.regression.IsotonicRegressionModel( @@ -658,7 +658,7 @@ def save(self, sc, path): @classmethod @since("1.4.0") def load(cls, sc, path): - """Load a IsotonicRegressionModel.""" + """Load an IsotonicRegressionModel.""" java_model = sc._jvm.org.apache.spark.mllib.regression.IsotonicRegressionModel.load( sc._jsc.sc(), path) py_boundaries = _java2py(sc, java_model.boundaryVector()).toArray() @@ -694,7 +694,7 @@ class IsotonicRegression(object): @since("1.4.0") def train(cls, data, isotonic=True): """ - Train a isotonic regression model on the given data. + Train an isotonic regression model on the given data. :param data: RDD of (label, feature, weight) tuples. diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 64b8bc442dad0..15cefc8cf1125 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1177,7 +1177,7 @@ def sha2(col, numBits): @since(2.0) def hash(*cols): - """Calculates the hash code of given columns, and returns the result as a int column. + """Calculates the hash code of given columns, and returns the result as an int column. >>> spark.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect() [Row(hash=-757602832)] diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 73d2b81b6bd04..13d21d7143d28 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -847,7 +847,7 @@ def orc(self, path, mode=None, partitionBy=None, compression=None): @since(1.4) def jdbc(self, url, table, mode=None, properties=None): - """Saves the content of the :class:`DataFrame` to a external database table via JDBC. + """Saves the content of the :class:`DataFrame` to an external database table via JDBC. .. note:: Don't create too many partitions in parallel on a large cluster; \ otherwise Spark might crash your external database systems. diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 67a0819601efa..fb6c66f2df997 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -623,7 +623,7 @@ def __init__(self, prev, func): self._jdstream_val = None # Using type() to avoid folding the functions and compacting the DStreams which is not - # not strictly a object of TransformedDStream. + # not strictly an object of TransformedDStream. # Changed here is to avoid bug in KafkaTransformedDStream when calling offsetRanges(). if (type(prev) is TransformedDStream and not prev.is_cached and not prev.is_checkpointed): diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index 015ca77dbf1fb..2c1a667fc80c4 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -228,7 +228,7 @@ class OffsetRange(object): def __init__(self, topic, partition, fromOffset, untilOffset): """ - Create a OffsetRange to represent range of offsets + Create an OffsetRange to represent range of offsets :param topic: Kafka topic name. :param partition: Kafka partition id. :param fromOffset: Inclusive starting offset. 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 cd242d78a4123..c5f221d7830f3 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 @@ -215,7 +215,7 @@ trait CheckAnalysis extends PredicateHelper { if (!RowOrdering.isOrderable(expr.dataType)) { failAnalysis( s"expression ${expr.sql} cannot be used as a grouping expression " + - s"because its data type ${expr.dataType.simpleString} is not a orderable " + + s"because its data type ${expr.dataType.simpleString} is not an orderable " + s"data type.") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala index 79c3528a522d3..d4350598f478c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCheckResult.scala @@ -37,7 +37,7 @@ object TypeCheckResult { /** * Represents the failing result of `Expression.checkInputDataTypes`, - * with a error message to show the reason of failure. + * with an error message to show the reason of failure. */ case class TypeCheckFailure(message: String) extends TypeCheckResult { def isSuccess: Boolean = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 91bdcc3b09a34..387e5552549e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -178,8 +178,8 @@ object TypeCoercion { q transformExpressions { case a: AttributeReference => inputMap.get(a.exprId) match { - // This can happen when a Attribute reference is born in a non-leaf node, for example - // due to a call to an external script like in the Transform operator. + // This can happen when an Attribute reference is born in a non-leaf node, for + // example due to a call to an external script like in the Transform operator. // TODO: Perhaps those should actually be aliases? case None => a // Leave the same if the dataTypes match. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala index 7da1fe93c6c7c..c66d08d2cf8fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala @@ -26,7 +26,7 @@ object JarResource extends FunctionResourceType("jar") object FileResource extends FunctionResourceType("file") -// We do not allow users to specify a archive because it is YARN specific. +// We do not allow users to specify an archive because it is YARN specific. // When loading resources, we will throw an exception and ask users to // use --archive with spark submit. object ArchiveResource extends FunctionResourceType("archive") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala index b8ab0364dd8f3..946b3d446a40f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.types._ /** * Returns the first value of `child` for a group of rows. If the first value of `child` - * is `null`, it returns `null` (respecting nulls). Even if [[First]] is used on a already + * is `null`, it returns `null` (respecting nulls). Even if [[First]] is used on an already * sorted column, if we do partial aggregation and final aggregation (when mergeExpression * is used) its result will not be deterministic (unless the input table is sorted and has * a single partition, and we use a single reducer to do the aggregation.). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index b05d74b49b591..53b4b761ae514 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.types._ /** * Returns the last value of `child` for a group of rows. If the last value of `child` - * is `null`, it returns `null` (respecting nulls). Even if [[Last]] is used on a already + * is `null`, it returns `null` (respecting nulls). Even if [[Last]] is used on an already * sorted column, if we do partial aggregation and final aggregation (when mergeExpression * is used) its result will not be deterministic (unless the input table is sorted and has * a single partition, and we use a single reducer to do the aggregation.). diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala index 9ead571c5374a..16c03c500ad08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala @@ -51,7 +51,7 @@ object PivotFirst { } /** - * PivotFirst is a aggregate function used in the second phase of a two phase pivot to do the + * PivotFirst is an aggregate function used in the second phase of a two phase pivot to do the * required rearrangement of values into pivoted form. * * For example on an input of 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 7e3683e482df1..95ed68fbb0528 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 @@ -182,7 +182,7 @@ case class Literal protected (value: Any, dataType: DataType) override protected def jsonFields: List[JField] = { // Turns all kinds of literal values to string in json field, as the type info is hard to - // retain in json format, e.g. {"a": 123} can be a int, or double, or decimal, etc. + // retain in json format, e.g. {"a": 123} can be an int, or double, or decimal, etc. val jsonValue = (value, dataType) match { case (null, _) => JNull case (i: Int, DateType) => JString(DateTimeUtils.toJavaDate(i).toString) 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 93a8278528697..e036982e70f99 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 @@ -214,7 +214,7 @@ class GenericRowWithSchema(values: Array[Any], override val schema: StructType) } /** - * A internal row implementation that uses an array of objects as the underlying storage. + * An internal row implementation that uses an array of objects as the underlying storage. * Note that, while the array is not copied, and thus could technically be mutated after creation, * this is not allowed. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 5001f9a41e07e..08cb6c0134e3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -129,7 +129,7 @@ object PredicateSubquery { /** * A [[ListQuery]] expression defines the query which we want to search in an IN subquery - * expression. It should and can only be used in conjunction with a IN expression. + * expression. It should and can only be used in conjunction with an IN expression. * * For example (SQL): * {{{ 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 5e998d61880ea..48d70099b6d88 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 @@ -926,7 +926,7 @@ object SimplifyConditionals extends Rule[LogicalPlan] with PredicateHelper { case e @ CaseWhen(branches, _) if branches.headOption.map(_._1) == Some(TrueLiteral) => // If the first branch is a true literal, remove the entire CaseWhen and use the value // from that. Note that CaseWhen.branches should never be empty, and as a result the - // headOption (rather than head) added above is just a extra (and unnecessary) safeguard. + // headOption (rather than head) added above is just an extra (and unnecessary) safeguard. branches.head._2 } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index d042e191a9e86..d687a85c18b63 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -59,7 +59,7 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { } } - /** Get the builder (visitor) which converts a ParseTree into a AST. */ + /** Get the builder (visitor) which converts a ParseTree into an AST. */ protected def astBuilder: AstBuilder protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 8b7e21b67982d..898784dab1d98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -530,7 +530,7 @@ private[sql] object Expand { /** * Apply the all of the GroupExpressions to every input row, hence we will get - * multiple output rows for a input row. + * multiple output rows for an input row. * * @param bitmasks The bitmask set represents the grouping sets * @param groupByAliases The aliased original group by expressions @@ -572,7 +572,7 @@ private[sql] object Expand { /** * Apply a number of projections to every input row, hence we will get multiple output rows for - * a input row. + * an input row. * * @param projections to apply * @param output of all projections. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index 42bdab42b79ff..b46f7a6d5a138 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst /** - * A a collection of common abstractions for query plans as well as + * A collection of common abstractions for query plans as well as * a base logical plan representation. */ package object plans 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 d449088498c8e..51d78dd1233fe 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 @@ -67,7 +67,7 @@ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distributi case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { require( ordering != Nil, - "The ordering expressions of a OrderedDistribution should not be Nil. " + + "The ordering expressions of an OrderedDistribution should not be Nil. " + "An AllTuples should be used to represent a distribution that only has " + "a single partition.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 31604bad0ff80..52e021070e940 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -366,7 +366,7 @@ object Decimal { val ROUND_CEILING = BigDecimal.RoundingMode.CEILING val ROUND_FLOOR = BigDecimal.RoundingMode.FLOOR - /** Maximum number of decimal digits a Int can represent */ + /** Maximum number of decimal digits an Int can represent */ val MAX_INT_DIGITS = 9 /** Maximum number of decimal digits a Long can represent */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 6b7e3714e0b04..6500875f95e54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -152,7 +152,7 @@ object DecimalType extends AbstractDataType { } /** - * Returns if dt is a DecimalType that fits inside a int + * Returns if dt is a DecimalType that fits inside an int */ def is32BitDecimalType(dt: DataType): Boolean = { dt match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 2e85e3676747a..c7b887ecd4d3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -529,7 +529,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { } /** - * Saves the content of the [[DataFrame]] to a external database table via JDBC. In the case the + * Saves the content of the [[DataFrame]] to an external database table via JDBC. In the case the * table already exists in the external database, behavior of this function depends on the * save mode, specified by the `mode` function (default to throwing an exception). * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index 1be41ffc072c1..4c046f7bdca48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics /** * Apply all of the GroupExpressions to every input row, hence we will get - * multiple output rows for a input row. + * multiple output rows for an input row. * @param projections The group of expressions, all of the group expressions should * output the same schema specified bye the parameter `output` * @param output The output Schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index e81cd28ea34d1..5f0c26441692d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -95,7 +95,7 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A * interfaces / internals. * * This RDD takes a [[ShuffleDependency]] (`dependency`), - * and a optional array of partition start indices as input arguments + * and an optional array of partition start indices as input arguments * (`specifiedPartitionStartIndices`). * * The `dependency` has the parent RDD of this RDD, which represents the dataset before shuffle diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 908e22de73bdc..2aec9318941a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -445,7 +445,7 @@ case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { } /** - * Inserts a InputAdapter on top of those that do not support codegen. + * Inserts an InputAdapter on top of those that do not support codegen. */ private def insertInputAdapter(plan: SparkPlan): SparkPlan = plan match { case j @ SortMergeJoinExec(_, _, _, _, left, right) if j.supportCodegen => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 243aa15deba3d..4b8adf5230717 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -41,7 +41,7 @@ import org.apache.spark.unsafe.KVIterator * - Step 0: Do hash-based aggregation. * - Step 1: Sort all entries of the hash map based on values of grouping expressions and * spill them to disk. - * - Step 2: Create a external sorter based on the spilled sorted map entries and reset the map. + * - Step 2: Create an external sorter based on the spilled sorted map entries and reset the map. * - Step 3: Get a sorted [[KVIterator]] from the external sorter. * - Step 4: Repeat step 0 until no more input. * - Step 5: Initialize sort-based aggregation on the sorted iterator. @@ -434,12 +434,12 @@ class TungstenAggregationIterator( /////////////////////////////////////////////////////////////////////////// /** - * Generate a output row when there is no input and there is no grouping expression. + * Generate an output row when there is no input and there is no grouping expression. */ def outputForEmptyGroupingKeyWithoutInput(): UnsafeRow = { if (groupingExpressions.isEmpty) { sortBasedAggregationBuffer.copyFrom(initialAggregationBuffer) - // We create a output row and copy it. So, we can free the map. + // We create an output row and copy it. So, we can free the map. val resultCopy = generateOutput(UnsafeRow.createFromByteArray(0, 0), sortBasedAggregationBuffer).copy() hashMap.free() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 89bde6ad73f3b..185c79f899e68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -142,9 +142,9 @@ case class FilterExec(condition: Expression, child: SparkPlan) // To generate the predicates we will follow this algorithm. // For each predicate that is not IsNotNull, we will generate them one by one loading attributes - // as necessary. For each of both attributes, if there is a IsNotNull predicate we will generate - // that check *before* the predicate. After all of these predicates, we will generate the - // remaining IsNotNull checks that were not part of other predicates. + // as necessary. For each of both attributes, if there is an IsNotNull predicate we will + // generate that check *before* the predicate. After all of these predicates, we will generate + // the remaining IsNotNull checks that were not part of other predicates. // This has the property of not doing redundant IsNotNull checks and taking better advantage of // short-circuiting, not loading attributes until they are needed. // This is very perf sensitive. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 3b064a5bc489f..61dcbebd644f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -176,7 +176,7 @@ private[sql] abstract class BaseWriterContainer( val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) ctor.newInstance(new Path(outputPath), context) } else { - // The specified output committer is just a OutputCommitter. + // The specified output committer is just an OutputCommitter. // So, we will use the no-argument constructor. val ctor = clazz.getDeclaredConstructor() ctor.newInstance() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala index 0f5d6445b1e2b..2cc012840dcaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.streaming /** - * A offset is a monotonically increasing metric used to track progress in the computation of a + * An offset is a monotonically increasing metric used to track progress in the computation of a * stream. An [[Offset]] must be comparable, and the result of `compareTo` must be consistent * with `equals` and `hashcode`. */ 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 0b490fe71c526..d89e98645b367 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 @@ -1996,7 +1996,7 @@ object functions { /** * Computes the numeric value of the first character of the string column, and returns the - * result as a int column. + * result as an int column. * * @group string_funcs * @since 1.5.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index f3f36efda599b..ceb68622752ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -356,7 +356,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) sessionCatalog.refreshTable(tableIdent) - // If this table is cached as a InMemoryRelation, drop the original + // If this table is cached as an InMemoryRelation, drop the original // cached version and make the new version cached lazily. val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent) // Use lookupCachedData directly since RefreshTable also takes databaseName. diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index cef5912c62409..de70fdc14ef34 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -157,7 +157,7 @@ object HiveThriftServer2 extends Logging { /** - * A inner sparkListener called in sc.stop to clean up the HiveThriftServer2 + * An inner sparkListener called in sc.stop to clean up the HiveThriftServer2 */ private[thriftserver] class HiveThriftServer2Listener( val server: HiveServer2, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index d033b05d4806e..88f4a2d2b20ba 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -391,7 +391,7 @@ private[spark] object HiveUtils extends Logging { // Remote means that the metastore server is running in its own process. // When the mode is remote, configurations like "javax.jdo.option.ConnectionURL" will not be // used (because they are used by remote metastore server that talks to the database). - // Because execution Hive should always connects to a embedded derby metastore. + // Because execution Hive should always connects to an embedded derby metastore. // We have to remove the value of hive.metastore.uris. So, the execution Hive client connects // to the actual embedded derby metastore instead of the remote metastore. // You can search HiveConf.ConfVars.METASTOREURIS in the code of HiveConf (in Hive's repo). 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 9e8ff9317c108..b3896484da178 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 @@ -211,7 +211,7 @@ case class InsertIntoHiveTable( val warningMessage = s"$outputCommitterClass may be an output committer that writes data directly to " + "the final location. Because speculation is enabled, this output committer may " + - "cause data loss (see the case in SPARK-10063). If possible, please use a output " + + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + "committer that does not have this behavior (e.g. FileOutputCommitter)." logWarning(warningMessage) } @@ -250,7 +250,7 @@ case class InsertIntoHiveTable( orderedPartitionSpec.put(entry.getName, partitionSpec.getOrElse(entry.getName, "")) } - // inheritTableSpecs is set to true. It should be set to false for a IMPORT query + // inheritTableSpecs is set to true. It should be set to false for an IMPORT query // which is currently considered as a Hive native command. val inheritTableSpecs = true // TODO: Correctly set isSkewedStoreAsSubdir. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index 13d2bed606a79..0589c8ece3910 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.types.StructType private[orc] object OrcFileOperator extends Logging { /** - * Retrieves a ORC file reader from a given path. The path can point to either a directory or a + * Retrieves an ORC file reader from a given path. The path can point to either a directory or a * single ORC file. If it points to an directory, it picks any non-empty ORC file within that * directory. * @@ -42,7 +42,7 @@ private[orc] object OrcFileOperator extends Logging { * ORC file if the file contains zero rows. This is OK for Hive since the schema of the * table is managed by metastore. But this becomes a problem when reading ORC files * directly from HDFS via Spark SQL, because we have to discover the schema from raw ORC - * files. So this method always tries to find a ORC file whose schema is non-empty, and + * files. So this method always tries to find an ORC file whose schema is non-empty, and * create the result reader from that file. If no such file is found, it returns `None`. * @todo Needs to consider all files when schema evolution is taken into account. */ 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 6ececb1062ae1..0b11026863199 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -154,7 +154,7 @@ object Checkpoint extends Logging { Utils.tryWithSafeFinally { // 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 + // to find classes, which maybe the wrong class loader. Hence, an 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) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 928739a416f0f..b524af9578b72 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -322,7 +322,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream from network source hostname:port, where data is received + * Create an input stream from network source hostname:port, where data is received * as serialized blocks (serialized using the Spark's serializer) that can be directly * pushed into the block manager without deserializing them. This is the most efficient * way to receive data. @@ -341,7 +341,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * Files must be written to the monitored directory by "moving" them from another * location within the same file system. File names starting with . are ignored. @@ -359,7 +359,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * Files must be written to the monitored directory by "moving" them from another * location within the same file system. @@ -379,7 +379,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * Files must be written to the monitored directory by "moving" them from another * location within the same file system. File names starting with . are ignored. @@ -403,7 +403,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them as text files (using key as LongWritable, value * as Text and input format as TextInputFormat). Files must be written to the * monitored directory by "moving" them from another location within the same diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 42fc84c19b971..faf6db82d5b18 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -79,7 +79,7 @@ private[streaming] abstract class ReceiverSupervisor( optionalBlockId: Option[StreamBlockId] ): Unit - /** Store a iterator of received data as a data block into Spark's memory. */ + /** Store an iterator of received data as a data block into Spark's memory. */ def pushIterator( iterator: Iterator[_], optionalMetadata: Option[Any], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 4fb0f8caacbb6..5ba09a54af18d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -129,7 +129,7 @@ private[streaming] class ReceiverSupervisorImpl( pushAndReportBlock(ArrayBufferBlock(arrayBuffer), metadataOption, blockIdOption) } - /** Store a iterator of received data as a data block into Spark's memory. */ + /** Store an iterator of received data as a data block into Spark's memory. */ def pushIterator( iterator: Iterator[_], metadataOption: Option[Any], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala index f7b6584893c6e..fb5587edeccee 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala @@ -38,7 +38,7 @@ import org.apache.spark.util.{Clock, Utils} * - Periodically take the average batch completion times and compare with the batch interval * - If (avg. proc. time / batch interval) >= scaling up ratio, then request more executors. * The number of executors requested is based on the ratio = (avg. proc. time / batch interval). - * - If (avg. proc. time / batch interval) <= scaling down ratio, then try to kill a executor that + * - If (avg. proc. time / batch interval) <= scaling down ratio, then try to kill an executor that * is not running a receiver. * * This features should ideally be used in conjunction with backpressure, as backpressure ensures diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala index 4f124a1356b5a..8e1a090618433 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala @@ -67,7 +67,7 @@ private[streaming] class InputInfoTracker(ssc: StreamingContext) extends Logging if (inputInfos.contains(inputInfo.inputStreamId)) { throw new IllegalStateException(s"Input stream ${inputInfo.inputStreamId} for batch" + - s"$batchTime is already added into InputInfoTracker, this is a illegal state") + s"$batchTime is already added into InputInfoTracker, this is an illegal state") } inputInfos += ((inputInfo.inputStreamId, inputInfo)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 1ef26d2f865da..60122b48135f7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -86,7 +86,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { /** * Generate a row for a Spark Job. Because duplicated output op infos needs to be collapsed into - * one cell, we use "rowspan" for the first row of a output op. + * one cell, we use "rowspan" for the first row of an output op. */ private def generateNormalJobRow( outputOpData: OutputOperationUIData, From b3845fedeb9ab87249b669be651c7396d0aaa824 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 26 May 2016 22:40:57 -0700 Subject: [PATCH 0413/1470] [SPARK-15529][SQL] Replace SQLContext and HiveContext with SparkSession in Test #### What changes were proposed in this pull request? This PR is to use the new entrance `Sparksession` to replace the existing `SQLContext` and `HiveContext` in SQL test suites. No change is made in the following suites: - `ListTablesSuite` is to test the APIs of `SQLContext`. - `SQLContextSuite` is to test `SQLContext` - `HiveContextCompatibilitySuite` is to test `HiveContext` **Update**: Move tests in `ListTableSuite` to `SQLContextSuite` #### How was this patch tested? N/A Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13337 from gatorsmile/sparkSessionTest. (cherry picked from commit d5911d1173fe0872f21cae6c47abf8ff479345a4) Signed-off-by: Reynold Xin --- .../apache/spark/sql/DataFrameReader.scala | 2 +- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 6 +- .../apache/spark/sql/ListTablesSuite.scala | 90 ------------------- .../apache/spark/sql/SQLContextSuite.scala | 61 +++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 8 +- .../execution/ExchangeCoordinatorSuite.scala | 22 ++--- .../spark/sql/execution/PlannerSuite.scala | 2 +- .../spark/sql/execution/SortSuite.scala | 2 +- .../TakeOrderedAndProjectSuite.scala | 2 +- .../sql/execution/command/DDLSuite.scala | 18 ++-- .../datasources/FileCatalogSuite.scala | 6 +- .../execution/datasources/csv/CSVSuite.scala | 2 +- .../ParquetPartitionDiscoverySuite.scala | 10 +-- .../datasources/parquet/ParquetTest.scala | 2 +- .../execution/joins/BroadcastJoinSuite.scala | 2 +- .../execution/joins/ExistenceJoinSuite.scala | 6 +- .../spark/sql/internal/SQLConfSuite.scala | 22 ++--- .../sources/CreateTableAsSelectSuite.scala | 12 +-- .../sql/sources/DDLSourceLoadSuite.scala | 10 +-- .../spark/sql/sources/DDLTestSuite.scala | 2 +- .../spark/sql/sources/DataSourceTest.scala | 9 +- .../spark/sql/sources/FilteredScanSuite.scala | 8 +- .../spark/sql/sources/InsertSuite.scala | 22 ++--- .../spark/sql/sources/PrunedScanSuite.scala | 6 +- .../spark/sql/sources/SaveLoadSuite.scala | 31 +++---- .../spark/sql/sources/TableScanSuite.scala | 8 +- .../sql/streaming/FileStreamSinkSuite.scala | 2 +- .../spark/sql/hive/CachedTableSuite.scala | 6 +- .../spark/sql/hive/ErrorPositionSuite.scala | 8 +- .../hive/HiveDataFrameAnalyticsSuite.scala | 8 +- .../sql/hive/HiveDataFrameJoinSuite.scala | 2 +- .../spark/sql/hive/HiveDataFrameSuite.scala | 10 +-- .../sql/hive/HiveMetastoreCatalogSuite.scala | 4 +- .../spark/sql/hive/HiveParquetSuite.scala | 4 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 20 ++--- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../spark/sql/hive/MultiDatabaseSuite.scala | 53 ++++++----- .../spark/sql/hive/QueryPartitionSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 36 ++++---- .../org/apache/spark/sql/hive/UDFSuite.scala | 14 +-- .../sql/hive/execution/HiveDDLSuite.scala | 24 +++-- .../sql/hive/execution/HiveExplainSuite.scala | 2 +- .../HiveOperatorQueryableSuite.scala | 2 +- .../sql/hive/execution/HivePlanTest.scala | 4 +- .../sql/hive/execution/HiveUDFSuite.scala | 22 ++--- .../sql/hive/execution/SQLQuerySuite.scala | 8 +- .../sql/hive/execution/SQLViewSuite.scala | 2 +- .../execution/SQLWindowFunctionSuite.scala | 2 +- .../execution/ScriptTransformationSuite.scala | 2 +- .../hive/orc/OrcHadoopFsRelationSuite.scala | 2 +- .../hive/orc/OrcPartitionDiscoverySuite.scala | 6 +- .../spark/sql/hive/orc/OrcSourceSuite.scala | 6 +- .../apache/spark/sql/hive/orc/OrcTest.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 4 +- .../spark/sql/sources/BucketedReadSuite.scala | 17 ++-- .../sql/sources/BucketedWriteSuite.scala | 2 +- .../sources/JsonHadoopFsRelationSuite.scala | 10 +-- .../ParquetHadoopFsRelationSuite.scala | 8 +- .../SimpleTextHadoopFsRelationSuite.scala | 2 +- 61 files changed, 319 insertions(+), 354 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 0fed9171a8bf3..2057878028383 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -426,7 +426,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * @param path input path * @since 1.5.0 - * @note Currently, this method can only be used together with `HiveContext`. + * @note Currently, this method can only be used after enabling Hive support. */ def orc(path: String): DataFrame = format("orc").load(path) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index c7b887ecd4d3e..f2ba2dfc086c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -639,7 +639,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * This will overwrite `orc.compress`.
    12. * * @since 1.5.0 - * @note Currently, this method can only be used together with `HiveContext`. + * @note Currently, this method can only be used after enabling Hive support */ def orc(path: String): Unit = { assertNotStreaming("orc() can only be called on non-continuous queries") 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 0614747352c11..e2dc4d86395ee 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 @@ -515,7 +515,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { ) } - test("callUDF in SQLContext") { + test("callUDF without Hive Support") { val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") df.sparkSession.udf.register("simpleUDF", (v: Int) => v * v) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 32320a6435acb..2a6591653e644 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -673,7 +673,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("runtime null check for RowEncoder") { val schema = new StructType().add("i", IntegerType, nullable = false) - val df = sqlContext.range(10).map(l => { + val df = spark.range(10).map(l => { if (l % 5 == 0) { Row(null) } else { @@ -689,9 +689,9 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("row nullability mismatch") { val schema = new StructType().add("a", StringType, true).add("b", StringType, false) - val rdd = sqlContext.sparkContext.parallelize(Row(null, "123") :: Row("234", null) :: Nil) + val rdd = spark.sparkContext.parallelize(Row(null, "123") :: Row("234", null) :: Nil) val message = intercept[Exception] { - sqlContext.createDataFrame(rdd, schema).collect() + spark.createDataFrame(rdd, schema).collect() }.getMessage assert(message.contains("The 1th field 'b' of input row cannot be null")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala deleted file mode 100644 index b447006761f45..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ /dev/null @@ -1,90 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import org.scalatest.BeforeAndAfter - -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} - -class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { - import testImplicits._ - - private lazy val df = (1 to 10).map(i => (i, s"str$i")).toDF("key", "value") - - before { - df.createOrReplaceTempView("listtablessuitetable") - } - - after { - spark.sessionState.catalog.dropTable( - TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) - } - - test("get all tables") { - checkAnswer( - spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) - - checkAnswer( - sql("SHOW tables").filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) - - spark.sessionState.catalog.dropTable( - TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) - assert(spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) - } - - test("getting all tables with a database name has no impact on returned table names") { - checkAnswer( - spark.sqlContext.tables("default").filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) - - checkAnswer( - sql("show TABLES in default").filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) - - spark.sessionState.catalog.dropTable( - TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) - assert(spark.sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) - } - - test("query the returned DataFrame of tables") { - val expectedSchema = StructType( - StructField("tableName", StringType, false) :: - StructField("isTemporary", BooleanType, false) :: Nil) - - Seq(spark.sqlContext.tables(), sql("SHOW TABLes")).foreach { - case tableDF => - assert(expectedSchema === tableDF.schema) - - tableDF.createOrReplaceTempView("tables") - checkAnswer( - sql( - "SELECT isTemporary, tableName from tables WHERE tableName = 'listtablessuitetable'"), - Row(true, "listtablessuitetable") - ) - checkAnswer( - spark.sqlContext.tables() - .filter("tableName = 'tables'").select("tableName", "isTemporary"), - Row("tables", true)) - spark.catalog.dropTempView("tables") - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index c9594a7e9ab28..417d09e238c62 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql import org.apache.spark.{SharedSparkContext, SparkFunSuite} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} class SQLContextSuite extends SparkFunSuite with SharedSparkContext { @@ -79,4 +81,63 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { assert(sqlContext.sessionState.optimizer.batches.flatMap(_.rules).contains(DummyRule)) } + test("get all tables") { + val sqlContext = SQLContext.getOrCreate(sc) + val df = sqlContext.range(10) + df.createOrReplaceTempView("listtablessuitetable") + assert( + sqlContext.tables().filter("tableName = 'listtablessuitetable'").collect().toSeq == + Row("listtablessuitetable", true) :: Nil) + + assert( + sqlContext.sql("SHOW tables").filter("tableName = 'listtablessuitetable'").collect().toSeq == + Row("listtablessuitetable", true) :: Nil) + + sqlContext.sessionState.catalog.dropTable( + TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) + assert(sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + } + + test("getting all tables with a database name has no impact on returned table names") { + val sqlContext = SQLContext.getOrCreate(sc) + val df = sqlContext.range(10) + df.createOrReplaceTempView("listtablessuitetable") + assert( + sqlContext.tables("default").filter("tableName = 'listtablessuitetable'").collect().toSeq == + Row("listtablessuitetable", true) :: Nil) + + assert( + sqlContext.sql("show TABLES in default").filter("tableName = 'listtablessuitetable'") + .collect().toSeq == Row("listtablessuitetable", true) :: Nil) + + sqlContext.sessionState.catalog.dropTable( + TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true) + assert(sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + } + + test("query the returned DataFrame of tables") { + val sqlContext = SQLContext.getOrCreate(sc) + val df = sqlContext.range(10) + df.createOrReplaceTempView("listtablessuitetable") + + val expectedSchema = StructType( + StructField("tableName", StringType, false) :: + StructField("isTemporary", BooleanType, false) :: Nil) + + Seq(sqlContext.tables(), sqlContext.sql("SHOW TABLes")).foreach { + case tableDF => + assert(expectedSchema === tableDF.schema) + + tableDF.createOrReplaceTempView("tables") + assert( + sqlContext.sql( + "SELECT isTemporary, tableName from tables WHERE tableName = 'listtablessuitetable'") + .collect().toSeq == Row(true, "listtablessuitetable") :: Nil) + assert( + sqlContext.tables().filter("tableName = 'tables'").select("tableName", "isTemporary") + .collect().toSeq == Row("tables", true) :: Nil) + sqlContext.dropTempTable("tables") + } + } + } 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 b1f848fdc89a3..1ddb586d60fd9 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 @@ -1048,7 +1048,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SET commands semantics using sql()") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() val testKey = "test.key.0" val testVal = "test.val.0" val nonexistentKey = "nonexistent" @@ -1089,17 +1089,17 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SET $nonexistentKey"), Row(nonexistentKey, "") ) - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() } test("SET commands with illegal or inappropriate argument") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() // Set negative mapred.reduce.tasks for automatically determining // the number of reducers is not supported intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-2")) - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() } test("apply schema") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 2f45db3925a00..2803b62462417 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -27,12 +27,12 @@ import org.apache.spark.sql.internal.SQLConf class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { - private var originalActiveSQLContext: Option[SparkSession] = _ - private var originalInstantiatedSQLContext: Option[SparkSession] = _ + private var originalActiveSparkSession: Option[SparkSession] = _ + private var originalInstantiatedSparkSession: Option[SparkSession] = _ override protected def beforeAll(): Unit = { - originalActiveSQLContext = SparkSession.getActiveSession - originalInstantiatedSQLContext = SparkSession.getDefaultSession + originalActiveSparkSession = SparkSession.getActiveSession + originalInstantiatedSparkSession = SparkSession.getDefaultSession SparkSession.clearActiveSession() SparkSession.clearDefaultSession() @@ -40,8 +40,8 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { override protected def afterAll(): Unit = { // Set these states back. - originalActiveSQLContext.foreach(ctx => SparkSession.setActiveSession(ctx)) - originalInstantiatedSQLContext.foreach(ctx => SparkSession.setDefaultSession(ctx)) + originalActiveSparkSession.foreach(ctx => SparkSession.setActiveSession(ctx)) + originalInstantiatedSparkSession.foreach(ctx => SparkSession.setDefaultSession(ctx)) } private def checkEstimation( @@ -249,7 +249,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - def withSQLContext( + def withSparkSession( f: SparkSession => Unit, targetNumPostShufflePartitions: Int, minNumPostShufflePartitions: Option[Int]): Unit = { @@ -322,7 +322,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - withSQLContext(test, 2000, minNumPostShufflePartitions) + withSparkSession(test, 2000, minNumPostShufflePartitions) } test(s"determining the number of reducers: join operator$testNameNote") { @@ -373,7 +373,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - withSQLContext(test, 16384, minNumPostShufflePartitions) + withSparkSession(test, 16384, minNumPostShufflePartitions) } test(s"determining the number of reducers: complex query 1$testNameNote") { @@ -425,7 +425,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - withSQLContext(test, 6644, minNumPostShufflePartitions) + withSparkSession(test, 6644, minNumPostShufflePartitions) } test(s"determining the number of reducers: complex query 2$testNameNote") { @@ -477,7 +477,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - withSQLContext(test, 6144, minNumPostShufflePartitions) + withSparkSession(test, 6144, minNumPostShufflePartitions) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 8243470b19334..c96239e682018 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -155,7 +155,7 @@ class PlannerSuite extends SharedSQLContext { val path = file.getCanonicalPath testData.write.parquet(path) val df = spark.read.parquet(path) - spark.sqlContext.registerDataFrameAsTable(df, "testPushed") + df.createOrReplaceTempView("testPushed") withTempTable("testPushed") { val exp = sql("select * from testPushed where key = 15").queryExecution.sparkPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index ebeb39b690e69..c3acf29c2d36f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -95,7 +95,7 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { ) { test(s"sorting on $dataType with nullable=$nullable, sortOrder=$sortOrder") { val inputData = Seq.fill(1000)(randomDataGenerator()) - val inputDf = sqlContext.createDataFrame( + val inputDf = spark.createDataFrame( sparkContext.parallelize(Random.shuffle(inputData).map(v => Row(v))), StructType(StructField("a", dataType, nullable = true) :: Nil) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index fba04d0cb2653..3217e34bd8ad3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -42,7 +42,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { .add("a", IntegerType, nullable = false) .add("b", IntegerType, nullable = false) val inputData = Seq.fill(10000)(Row(rand.nextInt(), rand.nextInt())) - sqlContext.createDataFrame(sparkContext.parallelize(Random.shuffle(inputData), 10), schema) + spark.createDataFrame(sparkContext.parallelize(Random.shuffle(inputData), 10), schema) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index ff56749387f56..e32521aaafecd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -132,7 +132,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE db2") val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri assert("file" === pathInCatalog.getScheme) - val expectedPath = appendTrailingSlash(sqlContext.conf.warehousePath) + "db2.db" + val expectedPath = appendTrailingSlash(spark.sessionState.conf.warehousePath) + "db2.db" assert(expectedPath === pathInCatalog.getPath) } @@ -145,7 +145,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { withTempDir { tmpDir => val path = tmpDir.toString withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") databaseNames.foreach { dbName => @@ -172,7 +172,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("Create/Drop Database - location") { - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => val path = tmpDir.toString @@ -200,7 +200,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { withTempDir { tmpDir => val path = tmpDir.toString withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") databaseNames.foreach { dbName => @@ -231,7 +231,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { withTempDir { tmpDir => val path = tmpDir.toString withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") databaseNames.foreach { dbName => @@ -300,7 +300,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("drop non-empty database in restrict mode") { - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog val dbName = "db1" sql(s"CREATE DATABASE $dbName") @@ -322,7 +322,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("drop non-empty database in cascade mode") { - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog val dbName = "db1" sql(s"CREATE DATABASE $dbName") @@ -441,7 +441,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { "RENAME TEMPORARY TABLE from '`tab1`' to '`default`.`tab2`': " + "cannot specify database name 'default' in the destination table")) - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"))) } } @@ -476,7 +476,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(e.getMessage.contains( "RENAME TEMPORARY TABLE from '`tab1`' to '`tab2`': destination table already exists")) - val catalog = sqlContext.sessionState.catalog + val catalog = spark.sessionState.catalog assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"), TableIdentifier("tab2"))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index dab5c76200f0c..85c2e8ba5529d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -32,7 +32,7 @@ class FileCatalogSuite extends SharedSQLContext { stringToFile(file, "text") val path = new Path(file.getCanonicalPath) - val catalog = new ListingFileCatalog(sqlContext.sparkSession, Seq(path), Map.empty, None) { + val catalog = new ListingFileCatalog(spark, Seq(path), Map.empty, None) { def leafFilePaths: Seq[Path] = leafFiles.keys.toSeq def leafDirPaths: Seq[Path] = leafDirToChildrenFiles.keys.toSeq } @@ -56,11 +56,11 @@ class FileCatalogSuite extends SharedSQLContext { require(qualifiedFilePath.toString.startsWith("file:")) val catalog1 = new ListingFileCatalog( - sqlContext.sparkSession, Seq(unqualifiedDirPath), Map.empty, None) + spark, Seq(unqualifiedDirPath), Map.empty, None) assert(catalog1.allFiles.map(_.getPath) === Seq(qualifiedFilePath)) val catalog2 = new ListingFileCatalog( - sqlContext.sparkSession, Seq(unqualifiedFilePath), Map.empty, None) + spark, Seq(unqualifiedFilePath), Map.empty, None) assert(catalog2.allFiles.map(_.getPath) === Seq(qualifiedFilePath)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index ad7c05c12eb9a..bc95446387956 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -137,7 +137,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } test("test inferring decimals") { - val result = sqlContext.read + val result = spark.read .format("csv") .option("comment", "~") .option("header", "true") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 847ea6bd523d0..e19345529e93b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -445,17 +445,17 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => ParquetData(i, i.toString)), path) // when the input is the base path containing partitioning directories - val baseDf = sqlContext.read.parquet(base.getCanonicalPath) + val baseDf = spark.read.parquet(base.getCanonicalPath) assert(baseDf.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) // when the input is a path to the leaf directory containing a parquet file - val partDf = sqlContext.read.parquet(path.getCanonicalPath) + val partDf = spark.read.parquet(path.getCanonicalPath) assert(partDf.schema.map(_.name) === Seq("intField", "stringField")) path.listFiles().foreach { f => if (f.getName.toLowerCase().endsWith(".parquet")) { // when the input is a path to a parquet file - val df = sqlContext.read.parquet(f.getCanonicalPath) + val df = spark.read.parquet(f.getCanonicalPath) assert(df.schema.map(_.name) === Seq("intField", "stringField")) } } @@ -464,7 +464,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha if (f.getName.toLowerCase().endsWith(".parquet")) { // when the input is a path to a parquet file but `basePath` is overridden to // the base path containing partitioning directories - val df = sqlContext + val df = spark .read.option("basePath", base.getCanonicalPath) .parquet(f.getCanonicalPath) assert(df.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) @@ -780,7 +780,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha .parquet(dir.getCanonicalPath) def check(path: String, basePath: String, expectedDf: DataFrame): Unit = { - val testDf = sqlContext.read + val testDf = spark.read .option("basePath", basePath) .parquet(path) checkAnswer(testDf, expectedDf) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index 1753b84ba6af6..1953d6fa5a7af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -90,7 +90,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { (data: Seq[T], tableName: String, testVectorized: Boolean = true) (f: => Unit): Unit = { withParquetDataFrame(data, testVectorized) { df => - spark.sqlContext.registerDataFrameAsTable(df, tableName) + df.createOrReplaceTempView(tableName) withTempTable(tableName)(f) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index e681b88685b58..2069652adebd0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -42,7 +42,7 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { protected var spark: SparkSession = null /** - * Create a new [[SQLContext]] running in local-cluster mode with unsafe and codegen enabled. + * Create a new [[SparkSession]] running in local-cluster mode with unsafe and codegen enabled. */ override def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index 8093054b6dd19..38377164c10e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StructT class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { - private lazy val left = sqlContext.createDataFrame( + private lazy val left = spark.createDataFrame( sparkContext.parallelize(Seq( Row(1, 2.0), Row(1, 2.0), @@ -42,7 +42,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { Row(6, null) )), new StructType().add("a", IntegerType).add("b", DoubleType)) - private lazy val right = sqlContext.createDataFrame( + private lazy val right = spark.createDataFrame( sparkContext.parallelize(Seq( Row(2, 3.0), Row(2, 3.0), @@ -53,7 +53,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { Row(6, null) )), new StructType().add("c", IntegerType).add("d", DoubleType)) - private lazy val rightUniqueKey = sqlContext.createDataFrame( + private lazy val rightUniqueKey = spark.createDataFrame( sparkContext.parallelize(Seq( Row(2, 3.0), Row(3, 2.0), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index f8227e3bd6ee8..ad5365a35ea5c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -35,7 +35,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // Set a conf first. spark.conf.set(testKey, testVal) // Clear the conf. - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() // After clear, only overrideConfs used by unit test should be in the SQLConf. assert(spark.conf.getAll === TestSQLContext.overrideConfs) @@ -50,11 +50,11 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { assert(spark.conf.get(testKey, testVal + "_") === testVal) assert(spark.conf.getAll.contains(testKey)) - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() } test("parse SQL set commands") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() sql(s"set $testKey=$testVal") assert(spark.conf.get(testKey, testVal + "_") === testVal) assert(spark.conf.get(testKey, testVal + "_") === testVal) @@ -72,7 +72,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { sql(s"set $key=") assert(spark.conf.get(key, "0") === "") - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() } test("set command for display") { @@ -97,7 +97,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("deprecated property") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) try { sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") @@ -108,7 +108,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("reset - public conf") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) try { assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) @@ -124,7 +124,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("reset - internal conf") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.NATIVE_VIEW) try { assert(spark.conf.get(SQLConf.NATIVE_VIEW) === true) @@ -140,7 +140,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("reset - user-defined conf") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() val userDefinedConf = "x.y.z.reset" try { assert(spark.conf.getOption(userDefinedConf).isEmpty) @@ -155,7 +155,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("invalid conf value") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() val e = intercept[IllegalArgumentException] { sql(s"set ${SQLConf.CASE_SENSITIVE.key}=10") } @@ -163,7 +163,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("Test SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE's method") { - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "100") assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === 100) @@ -191,7 +191,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-90000000000g") } - spark.sqlContext.conf.clear() + spark.sessionState.conf.clear() } test("SparkSession can access configs set in SparkConf") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 03c18ad009d49..cbddb0643b26d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -27,19 +27,19 @@ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ private var path: File = null override def beforeAll(): Unit = { super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") + spark.read.json(rdd).createOrReplaceTempView("jt") } override def afterAll(): Unit = { try { - caseInsensitiveContext.dropTempTable("jt") + spark.catalog.dropTempView("jt") } finally { super.afterAll() } @@ -64,7 +64,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with sql("SELECT a, b FROM jsonTable"), sql("SELECT a, b FROM jt").collect()) - caseInsensitiveContext.dropTempTable("jsonTable") + spark.catalog.dropTempView("jsonTable") } test("CREATE TEMPORARY TABLE AS SELECT based on the file without write permission") { @@ -132,7 +132,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with sql("SELECT * FROM jsonTable"), sql("SELECT a * 4 FROM jt").collect()) - caseInsensitiveContext.dropTempTable("jsonTable") + spark.catalog.dropTempView("jsonTable") // Explicitly delete the data. if (path.exists()) Utils.deleteRecursively(path) @@ -150,7 +150,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with sql("SELECT * FROM jsonTable"), sql("SELECT b FROM jt").collect()) - caseInsensitiveContext.dropTempTable("jsonTable") + spark.catalog.dropTempView("jsonTable") } test("CREATE TEMPORARY TABLE AS SELECT with IF NOT EXISTS is not allowed") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala index 853707c036c9a..f07c33042a72e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala @@ -27,23 +27,23 @@ class DDLSourceLoadSuite extends DataSourceTest with SharedSQLContext { test("data sources with the same name") { intercept[RuntimeException] { - caseInsensitiveContext.read.format("Fluet da Bomb").load() + spark.read.format("Fluet da Bomb").load() } } test("load data source from format alias") { - caseInsensitiveContext.read.format("gathering quorum").load().schema == + spark.read.format("gathering quorum").load().schema == StructType(Seq(StructField("stringType", StringType, nullable = false))) } test("specify full classname with duplicate formats") { - caseInsensitiveContext.read.format("org.apache.spark.sql.sources.FakeSourceOne") + spark.read.format("org.apache.spark.sql.sources.FakeSourceOne") .load().schema == StructType(Seq(StructField("stringType", StringType, nullable = false))) } - test("should fail to load ORC without HiveContext") { + test("should fail to load ORC without Hive Support") { intercept[ClassNotFoundException] { - caseInsensitiveContext.read.format("orc").load() + spark.read.format("orc").load() } } } 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 a34f70ed65b5c..5a7a9073fb3a2 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 @@ -78,7 +78,7 @@ case class SimpleDDLScan( } class DDLTestSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ override def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 754aa32a300b9..206d03ea98e69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -22,16 +22,9 @@ import org.apache.spark.sql.internal.SQLConf private[sql] abstract class DataSourceTest extends QueryTest { - // We want to test some edge cases. - protected lazy val caseInsensitiveContext: SQLContext = { - val ctx = new SQLContext(spark.sparkContext) - ctx.setConf(SQLConf.CASE_SENSITIVE, false) - ctx - } - protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row]) { test(sqlString) { - checkAnswer(caseInsensitiveContext.sql(sqlString), expectedAnswer) + checkAnswer(spark.sql(sqlString), expectedAnswer) } } 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 f969660ddd322..45e737f5ed047 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 @@ -133,7 +133,7 @@ object ColumnsRequired { } class FilteredScanSuite extends DataSourceTest with SharedSQLContext with PredicateHelper { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ override def beforeAll(): Unit = { super.beforeAll() @@ -310,7 +310,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic test(s"PushDown Returns $expectedCount: $sqlString") { // These tests check a particular plan, disable whole stage codegen. - caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, false) + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, false) try { val queryExecution = sql(sqlString).queryExecution val rawPlan = queryExecution.executedPlan.collect { @@ -322,7 +322,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic val rawCount = rawPlan.execute().count() assert(ColumnsRequired.set === requiredColumnNames) - val table = caseInsensitiveContext.table("oneToTenFiltered") + val table = spark.table("oneToTenFiltered") val relation = table.queryExecution.logical.collectFirst { case LogicalRelation(r, _, _) => r }.get @@ -337,7 +337,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic queryExecution) } } finally { - caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, SQLConf.WHOLESTAGE_CODEGEN_ENABLED.defaultValue.get) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 854fec5b22f77..4780eb473d79b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -24,14 +24,14 @@ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils class InsertSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ private var path: File = null override def beforeAll(): Unit = { super.beforeAll() path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - caseInsensitiveContext.read.json(rdd).createOrReplaceTempView("jt") + spark.read.json(rdd).createOrReplaceTempView("jt") sql( s""" |CREATE TEMPORARY TABLE jsonTable (a int, b string) @@ -44,8 +44,8 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { override def afterAll(): Unit = { try { - caseInsensitiveContext.dropTempTable("jsonTable") - caseInsensitiveContext.dropTempTable("jt") + spark.catalog.dropTempView("jsonTable") + spark.catalog.dropTempView("jt") Utils.deleteRecursively(path) } finally { super.afterAll() @@ -111,7 +111,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to less part files. val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 5) - caseInsensitiveContext.read.json(rdd1).createOrReplaceTempView("jt1") + spark.read.json(rdd1).createOrReplaceTempView("jt1") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt1 @@ -123,7 +123,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to more part files. val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 10) - caseInsensitiveContext.read.json(rdd2).createOrReplaceTempView("jt2") + spark.read.json(rdd2).createOrReplaceTempView("jt2") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 @@ -142,8 +142,8 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { (1 to 10).map(i => Row(i * 10, s"str$i")) ) - caseInsensitiveContext.dropTempTable("jt1") - caseInsensitiveContext.dropTempTable("jt2") + spark.catalog.dropTempView("jt1") + spark.catalog.dropTempView("jt2") } test("INSERT INTO JSONRelation for now") { @@ -185,7 +185,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt """.stripMargin) // Cached Query Execution - caseInsensitiveContext.cacheTable("jsonTable") + spark.catalog.cacheTable("jsonTable") assertCached(sql("SELECT * FROM jsonTable")) checkAnswer( sql("SELECT * FROM jsonTable"), @@ -226,7 +226,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // sql("SELECT a * 2, b FROM jt").collect()) // // // Verify uncaching -// caseInsensitiveContext.uncacheTable("jsonTable") +// spark.catalog.uncacheTable("jsonTable") // assertCached(sql("SELECT * FROM jsonTable"), 0) } @@ -257,6 +257,6 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { "It is not allowed to insert into a table that is not an InsertableRelation." ) - caseInsensitiveContext.dropTempTable("oneToTen") + spark.catalog.dropTempView("oneToTen") } } 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 9cdf7dea7663e..207f89d3eaea0 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 @@ -56,7 +56,7 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sparkSession: Spa } class PrunedScanSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ override def beforeAll(): Unit = { super.beforeAll() @@ -122,7 +122,7 @@ class PrunedScanSuite extends DataSourceTest with SharedSQLContext { test(s"Columns output ${expectedColumns.mkString(",")}: $sqlString") { // These tests check a particular plan, disable whole stage codegen. - caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, false) + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, false) try { val queryExecution = sql(sqlString).queryExecution val rawPlan = queryExecution.executedPlan.collect { @@ -145,7 +145,7 @@ class PrunedScanSuite extends DataSourceTest with SharedSQLContext { fail(s"Wrong output row. Got $rawOutput\n$queryExecution") } } finally { - caseInsensitiveContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, SQLConf.WHOLESTAGE_CODEGEN_ENABLED.defaultValue.get) } } 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 7738e4107df94..b1756c27fae0a 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 @@ -28,26 +28,26 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ private var originalDefaultSource: String = null private var path: File = null private var df: DataFrame = null override def beforeAll(): Unit = { super.beforeAll() - originalDefaultSource = caseInsensitiveContext.conf.defaultDataSourceName + originalDefaultSource = spark.sessionState.conf.defaultDataSourceName path = Utils.createTempDir() path.delete() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - df = caseInsensitiveContext.read.json(rdd) + df = spark.read.json(rdd) df.createOrReplaceTempView("jsonTable") } override def afterAll(): Unit = { try { - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, originalDefaultSource) } finally { super.afterAll() } @@ -58,45 +58,42 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA } def checkLoad(expectedDF: DataFrame = df, tbl: String = "jsonTable"): Unit = { - caseInsensitiveContext.conf.setConf( - SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") - checkAnswer(caseInsensitiveContext.read.load(path.toString), expectedDF.collect()) + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "org.apache.spark.sql.json") + checkAnswer(spark.read.load(path.toString), expectedDF.collect()) // Test if we can pick up the data source name passed in load. - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") - checkAnswer(caseInsensitiveContext.read.format("json").load(path.toString), + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "not a source name") + checkAnswer(spark.read.format("json").load(path.toString), expectedDF.collect()) - checkAnswer(caseInsensitiveContext.read.format("json").load(path.toString), + checkAnswer(spark.read.format("json").load(path.toString), expectedDF.collect()) val schema = StructType(StructField("b", StringType, true) :: Nil) checkAnswer( - caseInsensitiveContext.read.format("json").schema(schema).load(path.toString), + spark.read.format("json").schema(schema).load(path.toString), sql(s"SELECT b FROM $tbl").collect()) } test("save with path and load") { - caseInsensitiveContext.conf.setConf( - SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "org.apache.spark.sql.json") df.write.save(path.toString) checkLoad() } test("save with string mode and path, and load") { - caseInsensitiveContext.conf.setConf( - SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "org.apache.spark.sql.json") path.createNewFile() df.write.mode("overwrite").save(path.toString) checkLoad() } test("save with path and datasource, and load") { - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "not a source name") df.write.json(path.toString) checkLoad() } test("save with data source and options, and load") { - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "not a source name") df.write.mode(SaveMode.ErrorIfExists).json(path.toString) checkLoad() } 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 cddf4a1884fa8..93116d84ced71 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 @@ -106,7 +106,7 @@ case class AllDataTypesScan( } class TableScanSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ private lazy val tableWithSchemaExpected = (1 to 10).map { i => Row( @@ -241,7 +241,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { Nil ) - assert(expectedSchema == caseInsensitiveContext.table("tableWithSchema").schema) + assert(expectedSchema == spark.table("tableWithSchema").schema) checkAnswer( sql( @@ -297,7 +297,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { test("Caching") { // Cached Query Execution - caseInsensitiveContext.cacheTable("oneToTen") + spark.catalog.cacheTable("oneToTen") assertCached(sql("SELECT * FROM oneToTen")) checkAnswer( sql("SELECT * FROM oneToTen"), @@ -325,7 +325,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { (2 to 10).map(i => Row(i, i - 1)).toSeq) // Verify uncaching - caseInsensitiveContext.uncacheTable("oneToTen") + spark.catalog.uncacheTable("oneToTen") assertCached(sql("SELECT * FROM oneToTen"), 0) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index f3262f772b160..3d8dcaf5a5322 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -173,7 +173,7 @@ class FileStreamSinkSuite extends StreamTest with SharedSQLContext { query.processAllAvailable() } - val outputDf = sqlContext.read.parquet(outputDir) + val outputDf = spark.read.parquet(outputDir) val expectedSchema = new StructType() .add(StructField("value", IntegerType)) .add(StructField("id", IntegerType)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 4ca5619603220..52ba90f02c945 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -97,18 +97,18 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { test("correct error on uncache of non-cached table") { intercept[IllegalArgumentException] { - hiveContext.uncacheTable("src") + spark.catalog.uncacheTable("src") } } test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") { sql("CACHE TABLE src") assertCached(table("src")) - assert(hiveContext.isCached("src"), "Table 'src' should be cached") + assert(spark.catalog.isCached("src"), "Table 'src' should be cached") sql("UNCACHE TABLE src") assertCached(table("src"), 0) - assert(!hiveContext.isCached("src"), "Table 'src' should not be cached") + assert(!spark.catalog.isCached("src"), "Table 'src' should not be cached") } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { 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 dc8f374eb178f..aa1973de7f678 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 @@ -26,11 +26,11 @@ import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHiveSingleton class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterEach { - import hiveContext.implicits._ + import spark.implicits._ override protected def beforeEach(): Unit = { super.beforeEach() - if (spark.sqlContext.tableNames().contains("src")) { + if (spark.catalog.listTables().collect().map(_.name).contains("src")) { spark.catalog.dropTempView("src") } Seq((1, "")).toDF("key", "value").createOrReplaceTempView("src") @@ -130,12 +130,12 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = hiveContext.sessionState.sqlParser.parsePlan(query) + def ast = spark.sessionState.sqlParser.parsePlan(query) def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { val error = intercept[AnalysisException] { - quietly(hiveContext.sql(query)) + quietly(spark.sql(query)) } assert(!error.getMessage.contains("Seq(")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala index cc41c04c71e16..6477974fe713a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala @@ -27,20 +27,20 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton // `hive` package is optional in compiling, however, `SQLContext.sql` doesn't // support the `cube` or `rollup` yet. class HiveDataFrameAnalyticsSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { - import hiveContext.implicits._ - import hiveContext.sql + import spark.implicits._ + import spark.sql private var testData: DataFrame = _ override def beforeAll() { super.beforeAll() testData = Seq((1, 2), (2, 2), (3, 4)).toDF("a", "b") - hiveContext.registerDataFrameAsTable(testData, "mytable") + testData.createOrReplaceTempView("mytable") } override def afterAll(): Unit = { try { - hiveContext.dropTempTable("mytable") + spark.catalog.dropTempView("mytable") } finally { super.afterAll() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala index 63cf5030ab8b6..cdc259d75b139 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.hive.test.TestHiveSingleton class HiveDataFrameJoinSuite extends QueryTest with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ // We should move this into SQL package if we make case sensitivity configurable in SQL. test("join - self join auto resolve ambiguity with case insensitivity") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala index 7fdc5d71937ff..1b31caa76d3a5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala @@ -23,10 +23,10 @@ import org.apache.spark.sql.QueryTest class HiveDataFrameSuite extends QueryTest with TestHiveSingleton { test("table name with schema") { // regression test for SPARK-11778 - hiveContext.sql("create schema usrdb") - hiveContext.sql("create table usrdb.test(c int)") - hiveContext.read.table("usrdb.test") - hiveContext.sql("drop table usrdb.test") - hiveContext.sql("drop schema usrdb") + spark.sql("create schema usrdb") + spark.sql("create table usrdb.test(c int)") + spark.read.table("usrdb.test") + spark.sql("drop table usrdb.test") + spark.sql("drop schema usrdb") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index b043d291aa65f..b420781e51bd3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} import org.apache.spark.sql.types.{DecimalType, StringType, StructType} class HiveMetastoreCatalogSuite extends TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ test("struct field should accept underscore in sub-column name") { val hiveTypeStr = "struct" @@ -45,7 +45,7 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton { } test("duplicated metastore relations") { - val df = hiveContext.sql("SELECT * FROM src") + val df = spark.sql("SELECT * FROM src") logInfo(df.queryExecution.toString) df.as('a).join(df.as('b), $"a.key" === $"b.key") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index e2304b5397ca1..33252ad07add9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -51,7 +51,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton test("Converting Hive to Parquet Table via saveAsParquetFile") { withTempPath { dir => sql("SELECT * FROM src").write.parquet(dir.getCanonicalPath) - hiveContext.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("p") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("p") withTempTable("p") { checkAnswer( sql("SELECT * FROM src ORDER BY key"), @@ -65,7 +65,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton withParquetTable((1 to 10).map(i => (i, s"val_$i")), "t", false) { withTempPath { file => sql("SELECT * FROM t LIMIT 1").write.parquet(file.getCanonicalPath) - hiveContext.read.parquet(file.getCanonicalPath).createOrReplaceTempView("p") + spark.read.parquet(file.getCanonicalPath).createOrReplaceTempView("p") withTempTable("p") { // let's do three overwrites for good measure sql("INSERT OVERWRITE TABLE p SELECT * FROM t") 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 4a55bcc3b19d0..fae59001b98e1 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 @@ -36,9 +36,9 @@ case class ThreeCloumntable(key: Int, value: String, key1: String) class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter with SQLTestUtils { - import hiveContext.implicits._ + import spark.implicits._ - override lazy val testData = hiveContext.sparkContext.parallelize( + override lazy val testData = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() before { @@ -95,9 +95,9 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef test("SPARK-4052: scala.collection.Map as value type of MapType") { val schema = StructType(StructField("m", MapType(StringType, StringType), true) :: Nil) - val rowRDD = hiveContext.sparkContext.parallelize( + val rowRDD = spark.sparkContext.parallelize( (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) + val df = spark.createDataFrame(rowRDD, schema) df.createOrReplaceTempView("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m MAP )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -169,8 +169,8 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef test("Insert ArrayType.containsNull == false") { val schema = StructType(Seq( StructField("a", ArrayType(StringType, containsNull = false)))) - val rowRDD = hiveContext.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) + val rowRDD = spark.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i")))) + val df = spark.createDataFrame(rowRDD, schema) df.createOrReplaceTempView("tableWithArrayValue") sql("CREATE TABLE hiveTableWithArrayValue(a Array )") sql("INSERT OVERWRITE TABLE hiveTableWithArrayValue SELECT a FROM tableWithArrayValue") @@ -185,9 +185,9 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef test("Insert MapType.valueContainsNull == false") { val schema = StructType(Seq( StructField("m", MapType(StringType, StringType, valueContainsNull = false)))) - val rowRDD = hiveContext.sparkContext.parallelize( + val rowRDD = spark.sparkContext.parallelize( (1 to 100).map(i => Row(Map(s"key$i" -> s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) + val df = spark.createDataFrame(rowRDD, schema) df.createOrReplaceTempView("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m Map )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -202,9 +202,9 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef test("Insert StructType.fields.exists(_.nullable == false)") { val schema = StructType(Seq( StructField("s", StructType(Seq(StructField("f", StringType, nullable = false)))))) - val rowRDD = hiveContext.sparkContext.parallelize( + val rowRDD = spark.sparkContext.parallelize( (1 to 100).map(i => Row(Row(s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) + val df = spark.createDataFrame(rowRDD, schema) df.createOrReplaceTempView("tableWithStructValue") sql("CREATE TABLE hiveTableWithStructValue(s Struct )") sql("INSERT OVERWRITE TABLE hiveTableWithStructValue SELECT s FROM tableWithStructValue") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 153b0c3c72a78..1e6de463b3eba 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -40,7 +40,7 @@ import org.apache.spark.util.Utils */ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ - import hiveContext.implicits._ + import spark.implicits._ var jsonFilePath: String = _ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 5b706b0432418..83f1b192f7c94 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -25,22 +25,29 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle private lazy val df = spark.range(10).coalesce(1).toDF() private def checkTablePath(dbName: String, tableName: String): Unit = { - val metastoreTable = hiveContext.sharedState.externalCatalog.getTable(dbName, tableName) + val metastoreTable = spark.sharedState.externalCatalog.getTable(dbName, tableName) val expectedPath = - hiveContext.sharedState.externalCatalog.getDatabase(dbName).locationUri + "/" + tableName + spark.sharedState.externalCatalog.getDatabase(dbName).locationUri + "/" + tableName assert(metastoreTable.storage.serdeProperties("path") === expectedPath) } + private def getTableNames(dbName: Option[String] = None): Array[String] = { + dbName match { + case Some(db) => spark.catalog.listTables(db).collect().map(_.name) + case None => spark.catalog.listTables().collect().map(_.name) + } + } + test(s"saveAsTable() to non-default database - with USE - Overwrite") { withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.sqlContext.tableNames().contains("t")) + assert(getTableNames().contains("t")) checkAnswer(spark.table("t"), df) } - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") @@ -50,7 +57,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle test(s"saveAsTable() to non-default database - without USE - Overwrite") { withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") @@ -65,7 +72,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").mode(SaveMode.Overwrite).save(path) spark.catalog.createExternalTable("t", path, "parquet") - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table("t"), df) sql( @@ -76,7 +83,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle | path '$path' |) """.stripMargin) - assert(spark.sqlContext.tableNames(db).contains("t1")) + assert(getTableNames(Option(db)).contains("t1")) checkAnswer(spark.table("t1"), df) } } @@ -90,7 +97,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.format("parquet").mode(SaveMode.Overwrite).save(path) spark.catalog.createExternalTable(s"$db.t", path, "parquet") - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table(s"$db.t"), df) sql( @@ -101,7 +108,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle | path '$path' |) """.stripMargin) - assert(spark.sqlContext.tableNames(db).contains("t1")) + assert(getTableNames(Option(db)).contains("t1")) checkAnswer(spark.table(s"$db.t1"), df) } } @@ -112,11 +119,11 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") df.write.mode(SaveMode.Append).saveAsTable("t") - assert(spark.sqlContext.tableNames().contains("t")) + assert(getTableNames().contains("t")) checkAnswer(spark.table("t"), df.union(df)) } - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") @@ -127,7 +134,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") df.write.mode(SaveMode.Append).saveAsTable(s"$db.t") - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") @@ -138,7 +145,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.sqlContext.tableNames().contains("t")) + assert(getTableNames().contains("t")) df.write.insertInto(s"$db.t") checkAnswer(spark.table(s"$db.t"), df.union(df)) @@ -150,10 +157,10 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(spark.sqlContext.tableNames().contains("t")) + assert(getTableNames().contains("t")) } - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) df.write.insertInto(s"$db.t") checkAnswer(spark.table(s"$db.t"), df.union(df)) @@ -175,21 +182,21 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { sql(s"CREATE TABLE t (key INT)") - assert(spark.sqlContext.tableNames().contains("t")) - assert(!spark.sqlContext.tableNames("default").contains("t")) + assert(getTableNames().contains("t")) + assert(!getTableNames(Option("default")).contains("t")) } - assert(!spark.sqlContext.tableNames().contains("t")) - assert(spark.sqlContext.tableNames(db).contains("t")) + assert(!getTableNames().contains("t")) + assert(getTableNames(Option(db)).contains("t")) activateDatabase(db) { sql(s"DROP TABLE t") - assert(!spark.sqlContext.tableNames().contains("t")) - assert(!spark.sqlContext.tableNames("default").contains("t")) + assert(!getTableNames().contains("t")) + assert(!getTableNames(Option("default")).contains("t")) } - assert(!spark.sqlContext.tableNames().contains("t")) - assert(!spark.sqlContext.tableNames(db).contains("t")) + assert(!getTableNames().contains("t")) + assert(!getTableNames(Option(db)).contains("t")) } } 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 266fdd6c1f19f..f7650e001a8b9 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.util.Utils class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ test("SPARK-5068: query data when path doesn't exist") { withSQLConf((SQLConf.HIVE_VERIFY_PARTITION_PATH.key, "true")) { 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 73b1a7850d6fb..666a8da0dab85 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 @@ -30,11 +30,10 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { - import hiveContext.sql test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { - val parsed = hiveContext.sessionState.sqlParser.parsePlan(analyzeCommand) + val parsed = spark.sessionState.sqlParser.parsePlan(analyzeCommand) val operators = parsed.collect { case a: AnalyzeTableCommand => a case o => o @@ -72,7 +71,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils } test("MetastoreRelations fallback to HDFS for size estimation") { - val enableFallBackToHdfsForStats = hiveContext.conf.fallBackToHdfsForStatsEnabled + val enableFallBackToHdfsForStats = spark.sessionState.conf.fallBackToHdfsForStatsEnabled try { withTempDir { tempDir => @@ -98,9 +97,9 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils LOCATION '$tempDir' """) - hiveContext.setConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS, true) + spark.conf.set(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS.key, true) - val relation = hiveContext.sessionState.catalog.lookupRelation(TableIdentifier("csv_table")) + val relation = spark.sessionState.catalog.lookupRelation(TableIdentifier("csv_table")) .asInstanceOf[MetastoreRelation] val properties = relation.hiveQlTable.getParameters @@ -111,15 +110,14 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils assert(sizeInBytes === BigInt(file1.length() + file2.length())) } } finally { - hiveContext.setConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS, enableFallBackToHdfsForStats) + spark.conf.set(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS.key, enableFallBackToHdfsForStats) sql("DROP TABLE csv_table ") } } ignore("analyze MetastoreRelations") { def queryTotalSize(tableName: String): BigInt = - hiveContext.sessionState.catalog.lookupRelation( - TableIdentifier(tableName)).statistics.sizeInBytes + spark.sessionState.catalog.lookupRelation(TableIdentifier(tableName)).statistics.sizeInBytes // Non-partitioned table sql("CREATE TABLE analyzeTable (key STRING, value STRING)").collect() @@ -153,7 +151,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils |SELECT * FROM src """.stripMargin).collect() - assert(queryTotalSize("analyzeTable_part") === hiveContext.conf.defaultSizeInBytes) + assert(queryTotalSize("analyzeTable_part") === spark.sessionState.conf.defaultSizeInBytes) sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") @@ -165,9 +163,9 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils // Try to analyze a temp table sql("""SELECT * FROM src""").createOrReplaceTempView("tempTable") intercept[UnsupportedOperationException] { - hiveContext.sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") + sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") } - hiveContext.sessionState.catalog.dropTable( + spark.sessionState.catalog.dropTable( TableIdentifier("tempTable"), ignoreIfNotExists = true) } @@ -196,8 +194,8 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils val sizes = df.queryExecution.analyzed.collect { case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes } - assert(sizes.size === 2 && sizes(0) <= hiveContext.conf.autoBroadcastJoinThreshold - && sizes(1) <= hiveContext.conf.autoBroadcastJoinThreshold, + assert(sizes.size === 2 && sizes(0) <= spark.sessionState.conf.autoBroadcastJoinThreshold + && sizes(1) <= spark.sessionState.conf.autoBroadcastJoinThreshold, s"query should contain two relations, each of which has size smaller than autoConvertSize") // Using `sparkPlan` because for relevant patterns in HashJoin to be @@ -208,8 +206,8 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils checkAnswer(df, expectedAnswer) // check correctness of output - hiveContext.conf.settings.synchronized { - val tmp = hiveContext.conf.autoBroadcastJoinThreshold + spark.sessionState.conf.settings.synchronized { + val tmp = spark.sessionState.conf.autoBroadcastJoinThreshold sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1""") df = sql(query) @@ -252,8 +250,8 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils .isAssignableFrom(r.getClass) => r.statistics.sizeInBytes } - assert(sizes.size === 2 && sizes(1) <= hiveContext.conf.autoBroadcastJoinThreshold - && sizes(0) <= hiveContext.conf.autoBroadcastJoinThreshold, + assert(sizes.size === 2 && sizes(1) <= spark.sessionState.conf.autoBroadcastJoinThreshold + && sizes(0) <= spark.sessionState.conf.autoBroadcastJoinThreshold, s"query should contain two relations, each of which has size smaller than autoConvertSize") // Using `sparkPlan` because for relevant patterns in HashJoin to be @@ -266,8 +264,8 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils checkAnswer(df, answer) // check correctness of output - hiveContext.conf.settings.synchronized { - val tmp = hiveContext.conf.autoBroadcastJoinThreshold + spark.sessionState.conf.settings.synchronized { + val tmp = spark.sessionState.conf.autoBroadcastJoinThreshold sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1") df = sql(leftSemiJoinQuery) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala index d121bcbe15b35..88cc42efd0fe3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -36,7 +36,7 @@ class UDFSuite with TestHiveSingleton with BeforeAndAfterEach { - import hiveContext.implicits._ + import spark.implicits._ private[this] val functionName = "myUPper" private[this] val functionNameUpper = "MYUPPER" @@ -64,12 +64,12 @@ class UDFSuite } test("UDF case insensitive") { - hiveContext.udf.register("random0", () => { Math.random() }) - hiveContext.udf.register("RANDOM1", () => { Math.random() }) - hiveContext.udf.register("strlenScala", (_: String).length + (_: Int)) - assert(hiveContext.sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0) - assert(hiveContext.sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0) - assert(hiveContext.sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5) + spark.udf.register("random0", () => { Math.random() }) + spark.udf.register("RANDOM1", () => { Math.random() }) + spark.udf.register("strlenScala", (_: String).length + (_: Int)) + assert(sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0) + assert(sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0) + assert(sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5) } test("temporary function: create and drop") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 80e6f4ec70220..a98d469880736 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.test.SQLTestUtils class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { - import hiveContext.implicits._ + import spark.implicits._ override def afterEach(): Unit = { try { @@ -52,7 +52,7 @@ class HiveDDLSuite new Path(new Path(dbPath.get), tableIdentifier.table).toString } val filesystemPath = new Path(expectedTablePath) - val fs = filesystemPath.getFileSystem(hiveContext.sessionState.newHadoopConf()) + val fs = filesystemPath.getFileSystem(spark.sessionState.newHadoopConf()) fs.exists(filesystemPath) } @@ -86,8 +86,7 @@ class HiveDDLSuite """.stripMargin) val hiveTable = - hiveContext.sessionState.catalog - .getTableMetadata(TableIdentifier(tabName, Some("default"))) + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) assert(hiveTable.tableType == CatalogTableType.EXTERNAL) assert(tmpDir.listFiles.nonEmpty) @@ -113,8 +112,7 @@ class HiveDDLSuite } val hiveTable = - hiveContext.sessionState.catalog - .getTableMetadata(TableIdentifier(tabName, Some("default"))) + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) // This data source table is external table assert(hiveTable.tableType == CatalogTableType.EXTERNAL) @@ -127,7 +125,7 @@ class HiveDDLSuite } test("create table and view with comment") { - val catalog = hiveContext.sessionState.catalog + val catalog = spark.sessionState.catalog val tabName = "tab1" withTable(tabName) { sql(s"CREATE TABLE $tabName(c1 int) COMMENT 'BLABLA'") @@ -143,7 +141,7 @@ class HiveDDLSuite } test("add/drop partitions - external table") { - val catalog = hiveContext.sessionState.catalog + val catalog = spark.sessionState.catalog withTempDir { tmpDir => val basePath = tmpDir.getCanonicalPath val partitionPath_1stCol_part1 = new File(basePath + "/ds=2008-04-08") @@ -242,7 +240,7 @@ class HiveDDLSuite val oldViewName = "view1" val newViewName = "view2" withView(oldViewName, newViewName) { - val catalog = hiveContext.sessionState.catalog + val catalog = spark.sessionState.catalog sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName") assert(catalog.tableExists(TableIdentifier(oldViewName))) @@ -260,7 +258,7 @@ class HiveDDLSuite spark.range(10).write.saveAsTable(tabName) val viewName = "view1" withView(viewName) { - val catalog = hiveContext.sessionState.catalog + val catalog = spark.sessionState.catalog sql(s"CREATE VIEW $viewName AS SELECT * FROM $tabName") assert(catalog.getTableMetadata(TableIdentifier(viewName)) @@ -299,7 +297,7 @@ class HiveDDLSuite val oldViewName = "view1" val newViewName = "view2" withView(oldViewName, newViewName) { - val catalog = hiveContext.sessionState.catalog + val catalog = spark.sessionState.catalog sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName") assert(catalog.tableExists(TableIdentifier(tabName))) @@ -391,7 +389,7 @@ class HiveDDLSuite val catalog = spark.sessionState.catalog val dbName = "db1" val tabName = "tab1" - val fs = new Path(tmpDir.toString).getFileSystem(hiveContext.sessionState.newHadoopConf()) + val fs = new Path(tmpDir.toString).getFileSystem(spark.sessionState.newHadoopConf()) withTable(tabName) { if (dirExists) { assert(tmpDir.listFiles.isEmpty) @@ -441,7 +439,7 @@ class HiveDDLSuite val path = tmpDir.toString withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val dbName = "db1" - val fs = new Path(path).getFileSystem(hiveContext.sessionState.newHadoopConf()) + val fs = new Path(path).getFileSystem(spark.sessionState.newHadoopConf()) val dbPath = new Path(path) // the database directory does not exist assert(!fs.exists(dbPath)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 131b06aec8c9c..0d08f7edc8ea8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -80,7 +80,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") { withTempTable("jt") { val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - hiveContext.read.json(rdd).createOrReplaceTempView("jt") + spark.read.json(rdd).createOrReplaceTempView("jt") val outputs = sql( s""" |EXPLAIN EXTENDED diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala index 4d2f190b8e6f1..0e89e990e564e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton * A set of tests that validates commands can also be queried by like a table */ class HiveOperatorQueryableSuite extends QueryTest with TestHiveSingleton { - import hiveContext._ + import spark._ test("SPARK-5324 query result of describe command") { hiveContext.loadTestTable("src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala index 78c0d1f97e7da..89e6edb6b1577 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.hive.test.TestHiveSingleton class HivePlanTest extends QueryTest with TestHiveSingleton { - import hiveContext.sql - import hiveContext.implicits._ + import spark.sql + import spark.implicits._ test("udf constant folding") { Seq.empty[Tuple1[Int]].toDF("a").createOrReplaceTempView("t") 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 23b7f6c75b486..ffeed63695dfb 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 @@ -47,8 +47,8 @@ case class ListStringCaseClass(l: Seq[String]) */ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { - import hiveContext.udf - import hiveContext.implicits._ + import spark.udf + import spark.implicits._ test("spark sql udf test that returns a struct") { udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) @@ -151,7 +151,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFIntegerToString") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() testData.createOrReplaceTempView("integerTable") @@ -166,7 +166,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFToListString") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListString AS '${classOf[UDFToListString].getName}'") @@ -181,7 +181,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFToListInt") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListInt AS '${classOf[UDFToListInt].getName}'") @@ -196,7 +196,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFToStringIntMap") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToStringIntMap " + @@ -212,7 +212,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFToIntIntMap") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToIntIntMap " + @@ -228,7 +228,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFListListInt") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() @@ -244,7 +244,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFListString") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( ListStringCaseClass(Seq("a", "b", "c")) :: ListStringCaseClass(Seq("d", "e")) :: Nil).toDF() testData.createOrReplaceTempView("listStringTable") @@ -259,7 +259,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFStringString") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF() testData.createOrReplaceTempView("stringTable") @@ -278,7 +278,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("UDFTwoListList") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: 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 8a31a49d97f08..4b51f021bfa0c 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 @@ -58,7 +58,7 @@ case class Order( */ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ - import hiveContext.implicits._ + import spark.implicits._ test("UDTF") { withUserDefinedFunction("udtf_count2" -> true) { @@ -690,7 +690,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("SPARK-4699 HiveContext should be case insensitive by default") { + test("SPARK-4699 SparkSession with Hive Support should be case insensitive by default") { checkAnswer( sql("SELECT KEY FROM Src ORDER BY value"), sql("SELECT key FROM src ORDER BY value").collect().toSeq) @@ -707,7 +707,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val rowRdd = sparkContext.parallelize(row :: Nil) - hiveContext.createDataFrame(rowRdd, schema).createOrReplaceTempView("testTable") + spark.createDataFrame(rowRdd, schema).createOrReplaceTempView("testTable") sql( """CREATE TABLE nullValuesInInnerComplexTypes @@ -1417,7 +1417,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) checkAnswer( - spark.sqlContext.tables().select('isTemporary).filter('tableName === "t2"), + spark.sql("SHOW TABLES").select('isTemporary).filter('tableName === "t2"), Row(true) ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 42dbe188fb193..72db3618e0870 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.test.SQLTestUtils * A suite for testing view related functionality. */ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ override def beforeAll(): Unit = { // Create a simple table with two columns: id and id1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala index 47ceefb88ebcd..77e97dff8c221 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala @@ -29,7 +29,7 @@ case class WindowData(month: Int, area: String, product: Int) * Test suite for SQL window functions. */ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ test("window function: udaf with aggregate expression") { val data = Seq( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 19e8025d6b7c9..6f8062240732a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types.StringType class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ private val noSerdeIOSchema = HiveScriptIOSchema( inputRowFormat = Seq.empty, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala index 5dfa58f673bf2..463c368fc42b1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala @@ -59,7 +59,7 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.options(Map( + spark.read.options(Map( "path" -> file.getCanonicalPath, "dataSchema" -> dataSchemaWithPartition.json)).format(dataSourceName).load()) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala index fed0d11e9d216..d1ce3f1e2f058 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala @@ -37,8 +37,8 @@ case class OrcParDataWithKey(intField: Int, pi: Int, stringField: String, ps: St // TODO This test suite duplicates ParquetPartitionDiscoverySuite a lot class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { - import hiveContext._ - import hiveContext.implicits._ + import spark._ + import spark.implicits._ val defaultPartitionName = ConfVars.DEFAULTPARTITIONNAME.defaultStrVal @@ -59,7 +59,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B } protected def withTempTable(tableName: String)(f: => Unit): Unit = { - try f finally hiveContext.dropTempTable(tableName) + try f finally spark.catalog.dropTempView(tableName) } protected def makePartitionDir( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala index 4cac334859ba1..871b9e02eb382 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.Utils case class OrcData(intField: Int, stringField: String) abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { - import hiveContext._ + import spark._ var orcTableDir: File = null var orcTableAsDir: File = null @@ -152,7 +152,7 @@ class OrcSourceSuite extends OrcSuite { override def beforeAll(): Unit = { super.beforeAll() - hiveContext.sql( + spark.sql( s"""CREATE TEMPORARY TABLE normal_orc_source |USING org.apache.spark.sql.hive.orc |OPTIONS ( @@ -160,7 +160,7 @@ class OrcSourceSuite extends OrcSuite { |) """.stripMargin) - hiveContext.sql( + spark.sql( s"""CREATE TEMPORARY TABLE normal_orc_as_source |USING org.apache.spark.sql.hive.orc |OPTIONS ( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala index bb351e20c5e99..2a647115b7e01 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala @@ -61,7 +61,7 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton { (data: Seq[T], tableName: String) (f: => Unit): Unit = { withOrcDataFrame(data) { df => - spark.sqlContext.registerDataFrameAsTable(df, tableName) + df.createOrReplaceTempView(tableName) withTempTable(tableName)(f) } } 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 3e5140fe578de..06b74da196854 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 @@ -57,7 +57,7 @@ case class ParquetDataWithKeyAndComplexTypes( */ class ParquetMetastoreSuite extends ParquetPartitioningTest { import hiveContext._ - import hiveContext.implicits._ + import spark.implicits._ override def beforeAll(): Unit = { super.beforeAll() @@ -571,7 +571,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { */ class ParquetSourceSuite extends ParquetPartitioningTest { import testImplicits._ - import hiveContext._ + import spark._ override def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index d271e55467c6f..f9891ac5717e0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -51,7 +51,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet .saveAsTable("bucketed_table") for (i <- 0 until 5) { - val table = hiveContext.table("bucketed_table").filter($"i" === i) + val table = spark.table("bucketed_table").filter($"i" === i) val query = table.queryExecution val output = query.analyzed.output val rdd = query.toRdd @@ -80,7 +80,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet originalDataFrame: DataFrame): Unit = { // This test verifies parts of the plan. Disable whole stage codegen. withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { - val bucketedDataFrame = hiveContext.table("bucketed_table").select("i", "j", "k") + val bucketedDataFrame = spark.table("bucketed_table").select("i", "j", "k") val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec // Limit: bucket pruning only works when the bucket column has one and only one column assert(bucketColumnNames.length == 1) @@ -252,8 +252,8 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { - val t1 = hiveContext.table("bucketed_table1") - val t2 = hiveContext.table("bucketed_table2") + val t1 = spark.table("bucketed_table1") + val t2 = spark.table("bucketed_table2") val joined = t1.join(t2, joinCondition(t1, t2, joinColumns)) // First check the result is corrected. @@ -321,7 +321,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet test("avoid shuffle when grouping keys are equal to bucket keys") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("bucketed_table") - val tbl = hiveContext.table("bucketed_table") + val tbl = spark.table("bucketed_table") val agged = tbl.groupBy("i", "j").agg(max("k")) checkAnswer( @@ -335,7 +335,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet test("avoid shuffle when grouping keys are a super-set of bucket keys") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") - val tbl = hiveContext.table("bucketed_table") + val tbl = spark.table("bucketed_table") val agged = tbl.groupBy("i", "j").agg(max("k")) checkAnswer( @@ -349,11 +349,12 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") - val tableDir = new File(hiveContext.sparkSession.warehousePath, "bucketed_table") + val tableDir = new File(hiveContext + .sparkSession.warehousePath, "bucketed_table") Utils.deleteRecursively(tableDir) df1.write.parquet(tableDir.getAbsolutePath) - val agged = hiveContext.table("bucketed_table").groupBy("i").count() + val agged = spark.table("bucketed_table").groupBy("i").count() val error = intercept[RuntimeException] { agged.count() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 8bf6f224a4b98..ff44c6f29497d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -69,7 +69,7 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") def tableDir: File = { - val identifier = hiveContext.sessionState.sqlParser.parseTableIdentifier("bucketed_table") + val identifier = spark.sessionState.sqlParser.parseTableIdentifier("bucketed_table") new File(URI.create(hiveContext.sessionState.catalog.hiveDefaultTableFilePath(identifier))) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala index ef37787137d07..d79edee5b1a4c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala @@ -53,7 +53,7 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.format(dataSourceName) + spark.read.format(dataSourceName) .option("dataSchema", dataSchemaWithPartition.json) .load(file.getCanonicalPath)) } @@ -71,14 +71,14 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { val data = Row(Seq(1L, 2L, 3L), Map("m1" -> Row(4L))) :: Row(Seq(5L, 6L, 7L), Map("m2" -> Row(10L))) :: Nil - val df = hiveContext.createDataFrame(sparkContext.parallelize(data), schema) + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) // Write the data out. df.write.format(dataSourceName).save(file.getCanonicalPath) // Read it back and check the result. checkAnswer( - hiveContext.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), + spark.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), df ) } @@ -96,14 +96,14 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { Row(new BigDecimal("10.02")) :: Row(new BigDecimal("20000.99")) :: Row(new BigDecimal("10000")) :: Nil - val df = hiveContext.createDataFrame(sparkContext.parallelize(data), schema) + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) // Write the data out. df.write.format(dataSourceName).save(file.getCanonicalPath) // Read it back and check the result. checkAnswer( - hiveContext.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), + spark.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), df ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index 4b4852c1d793a..f9a1d16d9094d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -58,7 +58,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.format(dataSourceName) + spark.read.format(dataSourceName) .option("dataSchema", dataSchemaWithPartition.json) .load(file.getCanonicalPath)) } @@ -76,7 +76,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { .format("parquet") .save(s"${dir.getCanonicalPath}/_temporary") - checkAnswer(hiveContext.read.format("parquet").load(dir.getCanonicalPath), df.collect()) + checkAnswer(spark.read.format("parquet").load(dir.getCanonicalPath), df.collect()) } } @@ -104,7 +104,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { // This shouldn't throw anything. df.write.format("parquet").mode(SaveMode.Overwrite).save(path) - checkAnswer(hiveContext.read.format("parquet").load(path), df) + checkAnswer(spark.read.format("parquet").load(path), df) } } @@ -114,7 +114,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { // Parquet doesn't allow field names with spaces. Here we are intentionally making an // exception thrown from the `ParquetRelation2.prepareForWriteJob()` method to trigger // the bug. Please refer to spark-8079 for more details. - hiveContext.range(1, 10) + spark.range(1, 10) .withColumnRenamed("id", "a b") .write .format("parquet") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala index fa64c7dcfab6e..a47a2246ddc3c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala @@ -60,7 +60,7 @@ class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest with Predicat StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.format(dataSourceName) + spark.read.format(dataSourceName) .option("dataSchema", dataSchemaWithPartition.json) .load(file.getCanonicalPath)) } From b430aa98caa16978cd53dd354423cac45410c284 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 27 May 2016 08:54:14 -0700 Subject: [PATCH 0414/1470] [SPARK-15431][SQL][HOTFIX] ignore 'list' command testcase from CliSuite for now ## What changes were proposed in this pull request? The test cases for `list` command added in `CliSuite` by PR #13212 can not run in some jenkins jobs after being merged. However, some jenkins jobs can pass: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.6/ https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/ https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.2/ https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/ https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.3/ Others failed on this test case. But the failures on those jobs are at slightly different checkpoints among different jobs too. So it seems that CliSuite's output capture is flaky for list commands to check for expected output. There are test cases already in `HiveQuerySuite` and `SparkContextSuite` to cover the cases. So I am ignoring 2 test cases added by PR #13212 . Author: Xin Wu Closes #13276 from xwu0226/SPARK-15431-clisuite. (cherry picked from commit 6f95c6c030db0057de213733c2bd3453463bc6f2) Signed-off-by: Yin Huai --- .../org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 2bf0221c782e2..656fe973b0cae 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 @@ -239,7 +239,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { "" -> "This is a test for Spark-11624") } - test("list jars") { + ignore("list jars") { val jarFile = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") runCliWithin(2.minute)( s"ADD JAR $jarFile" -> "", @@ -248,7 +248,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { ) } - test("list files") { + ignore("list files") { val dataFilePath = Thread.currentThread().getContextClassLoader .getResource("data/files/small_kv.txt") runCliWithin(2.minute)( From 2cb84dd2356e782b9e606cd126057726fcf6f228 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 27 May 2016 09:54:31 -0700 Subject: [PATCH 0415/1470] [SPARK-15565][SQL] Add the File Scheme to the Default Value of WAREHOUSE_PATH #### What changes were proposed in this pull request? The default value of `spark.sql.warehouse.dir` is `System.getProperty("user.dir")/spark-warehouse`. Since `System.getProperty("user.dir")` is a local dir, we should explicitly set the scheme to local filesystem. cc yhuai #### How was this patch tested? Added two test cases Author: gatorsmile Closes #13348 from gatorsmile/addSchemeToDefaultWarehousePath. (cherry picked from commit c17272902c95290beca274ee6316a8a98fd7a725) Signed-off-by: Yin Huai --- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/execution/command/DDLSuite.scala | 25 +++++++++++++++++++ .../spark/sql/internal/SQLConfSuite.scala | 12 +++++++++ 3 files changed, 38 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4efefdacabf22..d1db0dd800a33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -55,7 +55,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("${system:user.dir}/spark-warehouse") + .createWithDefault("file:${system:user.dir}/spark-warehouse") val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e32521aaafecd..e975756685dec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -171,6 +171,31 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("Create Database using Default Warehouse Path") { + withSQLConf(SQLConf.WAREHOUSE_PATH.key -> "") { + // Will use the default location if and only if we unset the conf + spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) + val catalog = spark.sessionState.catalog + val dbName = "db1" + try { + sql(s"CREATE DATABASE $dbName") + val db1 = catalog.getDatabaseMetadata(dbName) + val expectedLocation = + "file:" + appendTrailingSlash(System.getProperty("user.dir")) + + s"spark-warehouse/$dbName.db" + assert(db1 == CatalogDatabase( + dbName, + "", + expectedLocation, + Map.empty)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.databaseExists(dbName)) + } finally { + catalog.reset() + } + } + } + test("Create/Drop Database - location") { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index ad5365a35ea5c..3d4fc75e83bbb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -207,4 +207,16 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + test("default value of WAREHOUSE_PATH") { + val original = spark.conf.get(SQLConf.WAREHOUSE_PATH) + try { + // to get the default value, always unset it + spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) + assert(spark.sessionState.conf.warehousePath + === s"file:${System.getProperty("user.dir")}/spark-warehouse") + } finally { + sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") + } + } + } From a355edeefa16988da8b05d2539a91277e75e823c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 27 May 2016 11:09:15 -0700 Subject: [PATCH 0416/1470] [SPARK-15603][MLLIB] Replace SQLContext with SparkSession in ML/MLLib ## What changes were proposed in this pull request? This PR replaces all deprecated `SQLContext` occurrences with `SparkSession` in `ML/MLLib` module except the following two classes. These two classes use `SQLContext` in their function signatures. - ReadWrite.scala - TreeModels.scala ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13352 from dongjoon-hyun/SPARK-15603. (cherry picked from commit d24e251572d39a453293cabfe14e4aed25a55208) Signed-off-by: Andrew Or --- .../spark/ml/clustering/GaussianMixture.scala | 7 ++-- .../spark/ml/feature/SQLTransformer.scala | 14 ++++---- .../apache/spark/ml/feature/Word2Vec.scala | 16 ++++----- .../mllib/api/python/PythonMLLibAPI.scala | 23 ++++++------ .../classification/LogisticRegression.scala | 19 +++++----- .../mllib/classification/NaiveBayes.scala | 24 ++++++------- .../impl/GLMClassificationModel.scala | 18 +++++----- .../clustering/BisectingKMeansModel.scala | 12 +++---- .../clustering/GaussianMixtureModel.scala | 16 ++++----- .../spark/mllib/clustering/KMeansModel.scala | 13 ++++--- .../spark/mllib/clustering/LDAModel.scala | 36 +++++++++---------- .../clustering/PowerIterationClustering.scala | 12 +++---- .../spark/mllib/feature/ChiSqSelector.scala | 13 ++++--- .../apache/spark/mllib/feature/Word2Vec.scala | 13 +++---- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 10 +++--- .../apache/spark/mllib/fpm/PrefixSpan.scala | 10 +++--- .../MatrixFactorizationModel.scala | 12 +++---- .../mllib/regression/IsotonicRegression.scala | 12 +++---- .../regression/impl/GLMRegressionModel.scala | 18 +++++----- .../mllib/tree/model/DecisionTreeModel.scala | 20 +++++------ .../mllib/tree/model/treeEnsembleModels.scala | 17 +++++---- .../ml/feature/QuantileDiscretizerSuite.scala | 14 ++++---- .../mllib/util/MLlibTestSparkContext.scala | 6 ++-- 23 files changed, 160 insertions(+), 195 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 88b6b27e62d97..773e50e24549c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -20,7 +20,6 @@ package org.apache.spark.ml.clustering import breeze.linalg.{DenseVector => BDV} import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.impl.Utils.EPSILON @@ -33,7 +32,7 @@ import org.apache.spark.mllib.clustering.{GaussianMixture => MLlibGM} import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors, VectorUDT => OldVectorUDT} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} @@ -134,9 +133,7 @@ class GaussianMixtureModel private[ml] ( val modelGaussians = gaussians.map { gaussian => (OldVectors.fromML(gaussian.mean), OldMatrices.fromML(gaussian.cov)) } - val sc = SparkContext.getOrCreate() - val sqlContext = SQLContext.getOrCreate(sc) - sqlContext.createDataFrame(modelGaussians).toDF("mean", "cov") + SparkSession.builder().getOrCreate().createDataFrame(modelGaussians).toDF("mean", "cov") } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 2d4cac6dc42c0..bd8f9494fb193 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -17,12 +17,11 @@ package org.apache.spark.ml.feature -import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.Transformer import org.apache.spark.ml.util._ -import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.types.StructType /** @@ -74,15 +73,14 @@ class SQLTransformer @Since("1.6.0") (override val uid: String) extends Transfor @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - val sc = SparkContext.getOrCreate() - val sqlContext = SQLContext.getOrCreate(sc) - val dummyRDD = sc.parallelize(Seq(Row.empty)) - val dummyDF = sqlContext.createDataFrame(dummyRDD, schema) + val spark = SparkSession.builder().getOrCreate() + val dummyRDD = spark.sparkContext.parallelize(Seq(Row.empty)) + val dummyDF = spark.createDataFrame(dummyRDD, schema) val tableName = Identifiable.randomUID(uid) val realStatement = $(statement).replace(tableIdentifier, tableName) dummyDF.createOrReplaceTempView(tableName) - val outputSchema = sqlContext.sql(realStatement).schema - sqlContext.dropTempTable(tableName) + val outputSchema = spark.sql(realStatement).schema + spark.catalog.dropTempView(tableName) outputSchema } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 1469bfd5e857d..1b929cdfffe36 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -28,7 +28,7 @@ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -183,11 +183,9 @@ class Word2VecModel private[ml] ( * and the vector the DenseVector that it is mapped to. */ @transient lazy val getVectors: DataFrame = { - val sc = SparkContext.getOrCreate() - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().getOrCreate() val wordVec = wordVectors.getVectors.mapValues(vec => Vectors.dense(vec.map(_.toDouble))) - sc.parallelize(wordVec.toSeq).toDF("word", "vector") + spark.createDataFrame(wordVec.toSeq).toDF("word", "vector") } /** @@ -205,10 +203,8 @@ class Word2VecModel private[ml] ( * synonyms and the given word vector. */ def findSynonyms(word: Vector, num: Int): DataFrame = { - val sc = SparkContext.getOrCreate() - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ - sc.parallelize(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") + val spark = SparkSession.builder().getOrCreate() + spark.createDataFrame(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") } /** @group setParam */ @@ -230,7 +226,7 @@ class Word2VecModel private[ml] ( val bVectors = dataset.sparkSession.sparkContext.broadcast(vectors) val d = $(vectorSize) val word2Vec = udf { sentence: Seq[String] => - if (sentence.size == 0) { + if (sentence.isEmpty) { Vectors.sparse(d, Array.empty[Int], Array.empty[Double]) } else { val sum = Vectors.zeros(d) 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 667290ece3514..6e0ed374c7ee1 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 @@ -30,8 +30,7 @@ import net.razorvine.pickle._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil -import org.apache.spark.ml.feature.{LabeledPoint => MLLabeledPoint} -import org.apache.spark.ml.linalg.{DenseMatrix => NewDenseMatrix, DenseVector => NewDenseVector, SparseMatrix => NewSparseMatrix, SparseVector => NewSparseVector, Vector => NewVector, Vectors => NewVectors} +import org.apache.spark.ml.linalg.{DenseMatrix => NewDenseMatrix, DenseVector => NewDenseVector, SparseMatrix => NewSparseMatrix, SparseVector => NewSparseVector, Vectors => NewVectors} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.evaluation.RankingMetrics @@ -43,8 +42,7 @@ 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.{ - KernelDensity, MultivariateStatisticalSummary, Statistics} +import org.apache.spark.mllib.stat.{KernelDensity, 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, KolmogorovSmirnovTestResult} @@ -56,7 +54,7 @@ import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTree RandomForestModel} import org.apache.spark.mllib.util.{LinearDataGenerator, MLUtils} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -1178,8 +1176,9 @@ private[python] class PythonMLLibAPI extends Serializable { def getIndexedRows(indexedRowMatrix: IndexedRowMatrix): DataFrame = { // We use DataFrames for serialization of IndexedRows to Python, // so return a DataFrame. - val sqlContext = SQLContext.getOrCreate(indexedRowMatrix.rows.sparkContext) - sqlContext.createDataFrame(indexedRowMatrix.rows) + val sc = indexedRowMatrix.rows.sparkContext + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + spark.createDataFrame(indexedRowMatrix.rows) } /** @@ -1188,8 +1187,9 @@ private[python] class PythonMLLibAPI extends Serializable { def getMatrixEntries(coordinateMatrix: CoordinateMatrix): DataFrame = { // We use DataFrames for serialization of MatrixEntry entries to // Python, so return a DataFrame. - val sqlContext = SQLContext.getOrCreate(coordinateMatrix.entries.sparkContext) - sqlContext.createDataFrame(coordinateMatrix.entries) + val sc = coordinateMatrix.entries.sparkContext + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + spark.createDataFrame(coordinateMatrix.entries) } /** @@ -1198,8 +1198,9 @@ private[python] class PythonMLLibAPI extends Serializable { def getMatrixBlocks(blockMatrix: BlockMatrix): DataFrame = { // We use DataFrames for serialization of sub-matrix blocks to // Python, so return a DataFrame. - val sqlContext = SQLContext.getOrCreate(blockMatrix.blocks.sparkContext) - sqlContext.createDataFrame(blockMatrix.blocks) + val sc = blockMatrix.blocks.sparkContext + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + spark.createDataFrame(blockMatrix.blocks) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 4bba2ea057af7..b186ca37703d3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{DataValidators, Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.storage.StorageLevel /** @@ -422,7 +422,7 @@ class LogisticRegressionWithLBFGS LogisticRegressionModel = { // ml's Logistic regression only supports binary classification currently. if (numOfLinearPredictor == 1) { - def runWithMlLogisitcRegression(elasticNetParam: Double) = { + def runWithMlLogisticRegression(elasticNetParam: Double) = { // Prepare the ml LogisticRegression based on our settings val lr = new org.apache.spark.ml.classification.LogisticRegression() lr.setRegParam(optimizer.getRegParam()) @@ -437,20 +437,19 @@ class LogisticRegressionWithLBFGS lr.setMaxIter(optimizer.getNumIterations()) lr.setTol(optimizer.getConvergenceTol()) // Convert our input into a DataFrame - val sqlContext = SQLContext.getOrCreate(input.context) - import sqlContext.implicits._ - val df = input.map(_.asML).toDF() + val spark = SparkSession.builder().config(input.context.getConf).getOrCreate() + val df = spark.createDataFrame(input.map(_.asML)) // Determine if we should cache the DF val handlePersistence = input.getStorageLevel == StorageLevel.NONE // Train our model - val mlLogisticRegresionModel = lr.train(df, handlePersistence) + val mlLogisticRegressionModel = lr.train(df, handlePersistence) // convert the model - val weights = Vectors.dense(mlLogisticRegresionModel.coefficients.toArray) - createModel(weights, mlLogisticRegresionModel.intercept) + val weights = Vectors.dense(mlLogisticRegressionModel.coefficients.toArray) + createModel(weights, mlLogisticRegressionModel.intercept) } optimizer.getUpdater() match { - case x: SquaredL2Updater => runWithMlLogisitcRegression(0.0) - case x: L1Updater => runWithMlLogisitcRegression(1.0) + case x: SquaredL2Updater => runWithMlLogisticRegression(0.0) + case x: L1Updater => runWithMlLogisticRegression(1.0) case _ => super.run(input, initialWeights) } } else { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index eb3ee41f7cf4f..452802f043abf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -31,7 +31,7 @@ import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, SparseVect import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.SparkSession /** * Model for Naive Bayes Classifiers. @@ -193,8 +193,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { modelType: String) def save(sc: SparkContext, path: String, data: Data): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -203,15 +202,14 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) // Create Parquet data. - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF() - dataRDD.write.parquet(dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath(path)) } @Since("1.3.0") def load(sc: SparkContext, path: String): NaiveBayesModel = { - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // Load Parquet data. - val dataRDD = sqlContext.read.parquet(dataPath(path)) + val dataRDD = spark.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("labels", "pi", "theta", "modelType").take(1) @@ -240,8 +238,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { theta: Array[Array[Double]]) def save(sc: SparkContext, path: String, data: Data): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -250,14 +247,13 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) // Create Parquet data. - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF() - dataRDD.write.parquet(dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath(path)) } def load(sc: SparkContext, path: String): NaiveBayesModel = { - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // Load Parquet data. - val dataRDD = sqlContext.read.parquet(dataPath(path)) + val dataRDD = spark.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("labels", "pi", "theta").take(1) @@ -327,7 +323,7 @@ class NaiveBayes private ( @Since("0.9.0") def setLambda(lambda: Double): NaiveBayes = { require(lambda >= 0, - s"Smoothing parameter must be nonnegative but got ${lambda}") + s"Smoothing parameter must be nonnegative but got $lambda") this.lambda = lambda this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala index 4308ae04ee84d..32e323d080af8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala @@ -23,7 +23,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.Loader -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * Helper class for import/export of GLM classification models. @@ -51,8 +51,7 @@ private[classification] object GLMClassificationModel { weights: Vector, intercept: Double, threshold: Option[Double]): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -62,7 +61,7 @@ private[classification] object GLMClassificationModel { // Create Parquet data. val data = Data(weights, intercept, threshold) - sc.parallelize(Seq(data), 1).toDF().write.parquet(Loader.dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(Loader.dataPath(path)) } /** @@ -73,13 +72,13 @@ private[classification] object GLMClassificationModel { * @param modelClass String name for model class (used for error messages) */ def loadData(sc: SparkContext, path: String, modelClass: String): Data = { - val datapath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) - val dataRDD = sqlContext.read.parquet(datapath) + val dataPath = Loader.dataPath(path) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataRDD = spark.read.parquet(dataPath) val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1) - assert(dataArray.length == 1, s"Unable to load $modelClass data from: $datapath") + assert(dataArray.length == 1, s"Unable to load $modelClass data from: $dataPath") val data = dataArray(0) - assert(data.size == 3, s"Unable to load $modelClass data from: $datapath") + assert(data.size == 3, s"Unable to load $modelClass data from: $dataPath") val (weights, intercept) = data match { case Row(weights: Vector, intercept: Double, _) => (weights, intercept) @@ -92,5 +91,4 @@ private[classification] object GLMClassificationModel { Data(weights, intercept, threshold) } } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index c3b5b8b7900f5..510a91b5a77fd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -29,7 +29,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * Clustering model produced by [[BisectingKMeans]]. @@ -144,8 +144,7 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val thisClassName = "org.apache.spark.mllib.clustering.BisectingKMeansModel" def save(sc: SparkContext, model: BisectingKMeansModel, path: String): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rootId" -> model.root.index))) @@ -154,8 +153,7 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val data = getNodes(model.root).map(node => Data(node.index, node.size, node.centerWithNorm.vector, node.centerWithNorm.norm, node.cost, node.height, node.children.map(_.index))) - val dataRDD = sc.parallelize(data).toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + spark.createDataFrame(data).write.parquet(Loader.dataPath(path)) } private def getNodes(node: ClusteringTreeNode): Array[ClusteringTreeNode] = { @@ -167,8 +165,8 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { } def load(sc: SparkContext, path: String, rootId: Int): BisectingKMeansModel = { - val sqlContext = SQLContext.getOrCreate(sc) - val rows = sqlContext.read.parquet(Loader.dataPath(path)) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val rows = spark.read.parquet(Loader.dataPath(path)) Loader.checkSchema[Data](rows.schema) val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index f87613cc72f9a..4b068164b8502 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -29,7 +29,7 @@ import org.apache.spark.mllib.linalg.{Matrix, Vector} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian 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.sql.{Row, SparkSession} /** * Multivariate Gaussian Mixture Model (GMM) consisting of k Gaussians, where points @@ -143,9 +143,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { path: String, weights: Array[Double], gaussians: Array[MultivariateGaussian]): Unit = { - - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // Create JSON metadata. val metadata = compact(render @@ -156,13 +154,13 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { val dataArray = Array.tabulate(weights.length) { i => Data(weights(i), gaussians(i).mu, gaussians(i).sigma) } - sc.parallelize(dataArray, 1).toDF().write.parquet(Loader.dataPath(path)) + spark.createDataFrame(dataArray).repartition(1).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): GaussianMixtureModel = { val dataPath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) - val dataFrame = sqlContext.read.parquet(dataPath) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataFrame = spark.read.parquet(dataPath) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[Data](dataFrame.schema) val dataArray = dataFrame.select("weight", "mu", "sigma").collect() @@ -172,7 +170,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { (weight, new MultivariateGaussian(mu, sigma)) }.unzip - new GaussianMixtureModel(weights.toArray, gaussians.toArray) + new GaussianMixtureModel(weights, gaussians) } } @@ -189,7 +187,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { s"GaussianMixtureModel requires weights of length $k " + s"got weights of length ${model.weights.length}") require(model.gaussians.length == k, - s"GaussianMixtureModel requires gaussians of length $k" + + s"GaussianMixtureModel requires gaussians of length $k " + s"got gaussians of length ${model.gaussians.length}") model case _ => throw new Exception( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 439e4f8672242..5f939c1a218fb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -30,7 +30,7 @@ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. @@ -123,25 +123,24 @@ object KMeansModel extends Loader[KMeansModel] { val thisClassName = "org.apache.spark.mllib.clustering.KMeansModel" def save(sc: SparkContext, model: KMeansModel, path: String): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) val dataRDD = sc.parallelize(model.clusterCenters.zipWithIndex).map { case (point, id) => Cluster(id, point) - }.toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + } + spark.createDataFrame(dataRDD).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): KMeansModel = { implicit val formats = DefaultFormats - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val k = (metadata \ "k").extract[Int] - val centroids = sqlContext.read.parquet(Loader.dataPath(path)) + val centroids = spark.read.parquet(Loader.dataPath(path)) Loader.checkSchema[Cluster](centroids.schema) val localCentroids = centroids.rdd.map(Cluster.apply).collect() assert(k == localCentroids.length) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 4913c0287a1d0..0a515f893dc0b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -31,7 +31,7 @@ import org.apache.spark.graphx.{Edge, EdgeContext, Graph, VertexId} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.util.BoundedPriorityQueue /** @@ -446,9 +446,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { docConcentration: Vector, topicConcentration: Double, gammaShape: Double): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ - + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val k = topicsMatrix.numCols val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ @@ -461,8 +459,8 @@ object LocalLDAModel extends Loader[LocalLDAModel] { val topicsDenseMatrix = topicsMatrix.toBreeze.toDenseMatrix val topics = Range(0, k).map { topicInd => Data(Vectors.dense((topicsDenseMatrix(::, topicInd).toArray)), topicInd) - }.toSeq - sc.parallelize(topics, 1).toDF().write.parquet(Loader.dataPath(path)) + } + spark.createDataFrame(topics).repartition(1).write.parquet(Loader.dataPath(path)) } def load( @@ -472,8 +470,8 @@ object LocalLDAModel extends Loader[LocalLDAModel] { topicConcentration: Double, gammaShape: Double): LocalLDAModel = { val dataPath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) - val dataFrame = sqlContext.read.parquet(dataPath) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataFrame = spark.read.parquet(dataPath) Loader.checkSchema[Data](dataFrame.schema) val topics = dataFrame.collect() @@ -853,8 +851,7 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { topicConcentration: Double, iterationTimes: Array[Double], gammaShape: Double): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ @@ -866,18 +863,17 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) val newPath = new Path(Loader.dataPath(path), "globalTopicTotals").toUri.toString - sc.parallelize(Seq(Data(Vectors.fromBreeze(globalTopicTotals)))).toDF() - .write.parquet(newPath) + spark.createDataFrame(Seq(Data(Vectors.fromBreeze(globalTopicTotals)))).write.parquet(newPath) val verticesPath = new Path(Loader.dataPath(path), "topicCounts").toUri.toString - graph.vertices.map { case (ind, vertex) => + spark.createDataFrame(graph.vertices.map { case (ind, vertex) => VertexData(ind, Vectors.fromBreeze(vertex)) - }.toDF().write.parquet(verticesPath) + }).write.parquet(verticesPath) val edgesPath = new Path(Loader.dataPath(path), "tokenCounts").toUri.toString - graph.edges.map { case Edge(srcId, dstId, prop) => + spark.createDataFrame(graph.edges.map { case Edge(srcId, dstId, prop) => EdgeData(srcId, dstId, prop) - }.toDF().write.parquet(edgesPath) + }).write.parquet(edgesPath) } def load( @@ -891,10 +887,10 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { val dataPath = new Path(Loader.dataPath(path), "globalTopicTotals").toUri.toString val vertexDataPath = new Path(Loader.dataPath(path), "topicCounts").toUri.toString val edgeDataPath = new Path(Loader.dataPath(path), "tokenCounts").toUri.toString - val sqlContext = SQLContext.getOrCreate(sc) - val dataFrame = sqlContext.read.parquet(dataPath) - val vertexDataFrame = sqlContext.read.parquet(vertexDataPath) - val edgeDataFrame = sqlContext.read.parquet(edgeDataPath) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataFrame = spark.read.parquet(dataPath) + val vertexDataFrame = spark.read.parquet(vertexDataPath) + val edgeDataFrame = spark.read.parquet(edgeDataPath) Loader.checkSchema[Data](dataFrame.schema) Loader.checkSchema[VertexData](vertexDataFrame.schema) 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 2e257ff9b7def..51077bd630a15 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 @@ -29,7 +29,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors 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.sql.{Row, SparkSession} import org.apache.spark.util.random.XORShiftRandom /** @@ -70,28 +70,26 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode @Since("1.4.0") def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() 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.write.parquet(Loader.dataPath(path)) + spark.createDataFrame(model.assignments).write.parquet(Loader.dataPath(path)) } @Since("1.4.0") def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { implicit val formats = DefaultFormats - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val k = (metadata \ "k").extract[Int] - val assignments = sqlContext.read.parquet(Loader.dataPath(path)) + val assignments = spark.read.parquet(Loader.dataPath(path)) Loader.checkSchema[PowerIterationClustering.Assignment](assignments.schema) val assignmentsRDD = assignments.rdd.map { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index 4f0e13feae086..13decefcd6695 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -30,7 +30,7 @@ import org.apache.spark.mllib.stat.Statistics import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * Chi Squared selector model. @@ -134,8 +134,8 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { val thisClassName = "org.apache.spark.mllib.feature.ChiSqSelectorModel" def save(sc: SparkContext, model: ChiSqSelectorModel, path: String): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) @@ -144,18 +144,17 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { val dataArray = Array.tabulate(model.selectedFeatures.length) { i => Data(model.selectedFeatures(i)) } - sc.parallelize(dataArray, 1).toDF().write.parquet(Loader.dataPath(path)) - + spark.createDataFrame(dataArray).repartition(1).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): ChiSqSelectorModel = { implicit val formats = DefaultFormats - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) - val dataFrame = sqlContext.read.parquet(Loader.dataPath(path)) + val dataFrame = spark.read.parquet(Loader.dataPath(path)) val dataArray = dataFrame.select("feature") // Check schema explicitly since erasure makes it hard to use match-case for checking. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 7e6c3679704c5..9bd79aa7c627e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom @@ -609,9 +609,8 @@ object Word2VecModel extends Loader[Word2VecModel] { case class Data(word: String, vector: Array[Float]) def load(sc: SparkContext, path: String): Word2VecModel = { - val dataPath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) - val dataFrame = sqlContext.read.parquet(dataPath) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataFrame = spark.read.parquet(Loader.dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[Data](dataFrame.schema) @@ -621,9 +620,7 @@ object Word2VecModel extends Loader[Word2VecModel] { } def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]): Unit = { - - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val vectorSize = model.values.head.length val numWords = model.size @@ -641,7 +638,7 @@ object Word2VecModel extends Loader[Word2VecModel] { val approxSize = 4L * numWords * vectorSize val nPartitions = ((approxSize / partitionSize) + 1).toInt val dataArray = model.toSeq.map { case (w, v) => Data(w, v) } - sc.parallelize(dataArray.toSeq, nPartitions).toDF().write.parquet(Loader.dataPath(path)) + spark.createDataFrame(dataArray).repartition(nPartitions).write.parquet(Loader.dataPath(path)) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 28e4966f918a0..8c0639baeaca4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -37,7 +37,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.fpm.FPGrowth._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel @@ -99,7 +99,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { def save(model: FPGrowthModel[_], path: String): Unit = { val sc = model.freqItemsets.sparkContext - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) @@ -118,18 +118,18 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { val rowDataRDD = model.freqItemsets.map { x => Row(x.items.toSeq, x.freq) } - sqlContext.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) + spark.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): FPGrowthModel[_] = { implicit val formats = DefaultFormats - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) - val freqItemsets = sqlContext.read.parquet(Loader.dataPath(path)) + val freqItemsets = spark.read.parquet(Loader.dataPath(path)) val sample = freqItemsets.select("items").head().get(0) loadImpl(freqItemsets, sample) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index 4344ab1bade9a..10bbcd2a3d924 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -36,7 +36,7 @@ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.internal.Logging import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel @@ -616,7 +616,7 @@ object PrefixSpanModel extends Loader[PrefixSpanModel[_]] { def save(model: PrefixSpanModel[_], path: String): Unit = { val sc = model.freqSequences.sparkContext - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) @@ -635,18 +635,18 @@ object PrefixSpanModel extends Loader[PrefixSpanModel[_]] { val rowDataRDD = model.freqSequences.map { x => Row(x.sequence, x.freq) } - sqlContext.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) + spark.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): PrefixSpanModel[_] = { implicit val formats = DefaultFormats - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) - val freqSequences = sqlContext.read.parquet(Loader.dataPath(path)) + val freqSequences = spark.read.parquet(Loader.dataPath(path)) val sample = freqSequences.select("sequence").head().get(0) loadImpl(freqSequences, sample) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 6f780b0da71f5..450025f477f19 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -37,7 +37,7 @@ import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.storage.StorageLevel /** @@ -354,8 +354,8 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { */ def save(model: MatrixFactorizationModel, path: String): Unit = { val sc = model.userFeatures.sparkContext - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + import spark.implicits._ val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rank" -> model.rank))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) @@ -365,16 +365,16 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { def load(sc: SparkContext, path: String): MatrixFactorizationModel = { implicit val formats = DefaultFormats - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val (className, formatVersion, metadata) = loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val rank = (metadata \ "rank").extract[Int] - val userFeatures = sqlContext.read.parquet(userPath(path)).rdd.map { + val userFeatures = spark.read.parquet(userPath(path)).rdd.map { case Row(id: Int, features: Seq[_]) => (id, features.asInstanceOf[Seq[Double]].toArray) } - val productFeatures = sqlContext.read.parquet(productPath(path)).rdd.map { + val productFeatures = spark.read.parquet(productPath(path)).rdd.map { case Row(id: Int, features: Seq[_]) => (id, features.asInstanceOf[Seq[Double]].toArray) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index abdd7981970fa..215a799b9646a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -34,7 +34,7 @@ import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * Regression model for isotonic regression. @@ -185,21 +185,21 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { boundaries: Array[Double], predictions: Array[Double], isotonic: Boolean): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("isotonic" -> isotonic))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) - sqlContext.createDataFrame( + spark.createDataFrame( boundaries.toSeq.zip(predictions).map { case (b, p) => Data(b, p) } ).write.parquet(dataPath(path)) } def load(sc: SparkContext, path: String): (Array[Double], Array[Double]) = { - val sqlContext = SQLContext.getOrCreate(sc) - val dataRDD = sqlContext.read.parquet(dataPath(path)) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataRDD = spark.read.parquet(dataPath(path)) checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("boundary", "prediction").collect() @@ -221,7 +221,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { val (boundaries, predictions) = SaveLoadV1_0.load(sc, path) new IsotonicRegressionModel(boundaries, predictions, isotonic) case _ => throw new Exception( - s"IsotonicRegressionModel.load did not recognize model with (className, format version):" + + s"IsotonicRegressionModel.load did not recognize model with (className, format version): " + s"($loadedClassName, $version). Supported:\n" + s" ($classNameV1_0, 1.0)" ) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala index 7696fdf2dcbed..3c7bbc52446d9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala @@ -23,7 +23,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.Loader -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * Helper methods for import/export of GLM regression models. @@ -47,7 +47,7 @@ private[regression] object GLMRegressionModel { modelClass: String, weights: Vector, intercept: Double): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -57,7 +57,7 @@ private[regression] object GLMRegressionModel { // Create Parquet data. val data = Data(weights, intercept) - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(Loader.dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(Loader.dataPath(path)) } /** @@ -67,17 +67,17 @@ private[regression] object GLMRegressionModel { * The length of the weights vector should equal numFeatures. */ def loadData(sc: SparkContext, path: String, modelClass: String, numFeatures: Int): Data = { - val datapath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) - val dataRDD = sqlContext.read.parquet(datapath) + val dataPath = Loader.dataPath(path) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataRDD = spark.read.parquet(dataPath) val dataArray = dataRDD.select("weights", "intercept").take(1) - assert(dataArray.length == 1, s"Unable to load $modelClass data from: $datapath") + assert(dataArray.length == 1, s"Unable to load $modelClass data from: $dataPath") val data = dataArray(0) - assert(data.size == 2, s"Unable to load $modelClass data from: $datapath") + assert(data.size == 2, s"Unable to load $modelClass data from: $dataPath") data match { case Row(weights: Vector, intercept: Double) => assert(weights.size == numFeatures, s"Expected $numFeatures features, but" + - s" found ${weights.size} features when loading $modelClass weights from $datapath") + s" found ${weights.size} features when loading $modelClass weights from $dataPath") Data(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index c13b9a66c4e24..72663188a98ae 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -32,7 +32,7 @@ import org.apache.spark.mllib.tree.configuration.{Algo, FeatureType} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.util.Utils /** @@ -202,9 +202,6 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { } def save(sc: SparkContext, path: String, model: DecisionTreeModel): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ - // SPARK-6120: We do a hacky check here so users understand why save() is failing // when they run the ML guide example. // TODO: Fix this issue for real. @@ -235,17 +232,16 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { // Create Parquet data. val nodes = model.topNode.subtreeIterator.toSeq - val dataRDD: DataFrame = sc.parallelize(nodes) - .map(NodeData.apply(0, _)) - .toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + val dataRDD = sc.parallelize(nodes).map(NodeData.apply(0, _)) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + spark.createDataFrame(dataRDD).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String, algo: String, numNodes: Int): DecisionTreeModel = { - val datapath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) // Load Parquet data. - val dataRDD = sqlContext.read.parquet(datapath) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val dataPath = Loader.dataPath(path) + val dataRDD = spark.read.parquet(dataPath) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[NodeData](dataRDD.schema) val nodes = dataRDD.rdd.map(NodeData.apply) @@ -254,7 +250,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { assert(trees.length == 1, "Decision tree should contain exactly one tree but got ${trees.size} trees.") val model = new DecisionTreeModel(trees(0), Algo.fromString(algo)) - assert(model.numNodes == numNodes, s"Unable to load DecisionTreeModel data from: $datapath." + + assert(model.numNodes == numNodes, s"Unable to load DecisionTreeModel data from: $dataPath." + s" Expected $numNodes nodes but found ${model.numNodes}") model } 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 cbf49b6d5821a..c653b988e21e7 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 @@ -36,7 +36,7 @@ import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._ import org.apache.spark.mllib.tree.loss.Loss import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils /** @@ -413,8 +413,7 @@ private[tree] object TreeEnsembleModel extends Logging { case class EnsembleNodeData(treeId: Int, node: NodeData) def save(sc: SparkContext, path: String, model: TreeEnsembleModel, className: String): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() // SPARK-6120: We do a hacky check here so users understand why save() is failing // when they run the ML guide example. @@ -450,8 +449,8 @@ private[tree] object TreeEnsembleModel extends Logging { // Create Parquet data. val dataRDD = sc.parallelize(model.trees.zipWithIndex).flatMap { case (tree, treeId) => tree.topNode.subtreeIterator.toSeq.map(node => NodeData(treeId, node)) - }.toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + } + spark.createDataFrame(dataRDD).write.parquet(Loader.dataPath(path)) } /** @@ -472,10 +471,10 @@ private[tree] object TreeEnsembleModel extends Logging { sc: SparkContext, path: String, treeAlgo: String): Array[DecisionTreeModel] = { - val datapath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) - val nodes = sqlContext.read.parquet(datapath).rdd.map(NodeData.apply) - val trees = constructTrees(nodes) + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + import spark.implicits._ + val nodes = spark.read.parquet(Loader.dataPath(path)).map(NodeData.apply) + val trees = constructTrees(nodes.rdd) trees.map(new DecisionTreeModel(_, Algo.fromString(treeAlgo))) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala index 8895d630a0879..621c13a8e5ac6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala @@ -20,15 +20,15 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions.udf class QuantileDiscretizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("Test observed number of buckets and their sizes match expected values") { - val sqlCtx = SQLContext.getOrCreate(sc) - import sqlCtx.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + import spark.implicits._ val datasetSize = 100000 val numBuckets = 5 @@ -53,8 +53,8 @@ class QuantileDiscretizerSuite } test("Test transform method on unseen data") { - val sqlCtx = SQLContext.getOrCreate(sc) - import sqlCtx.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + import spark.implicits._ val trainDF = sc.parallelize(1.0 to 100.0 by 1.0).map(Tuple1.apply).toDF("input") val testDF = sc.parallelize(-10.0 to 110.0 by 1.0).map(Tuple1.apply).toDF("input") @@ -82,8 +82,8 @@ class QuantileDiscretizerSuite } test("Verify resulting model has parent") { - val sqlCtx = SQLContext.getOrCreate(sc) - import sqlCtx.implicits._ + val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + import spark.implicits._ val df = sc.parallelize(1 to 100).map(Tuple1.apply).toDF("input") val discretizer = new QuantileDiscretizer() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala index ba8d36f45f55c..db56aff63102c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala @@ -21,9 +21,9 @@ import java.io.File import org.scalatest.Suite -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.ml.util.TempDirectory -import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils trait MLlibTestSparkContext extends TempDirectory { self: Suite => @@ -46,7 +46,7 @@ trait MLlibTestSparkContext extends TempDirectory { self: Suite => override def afterAll() { try { Utils.deleteRecursively(new File(checkpointDir)) - SQLContext.clearActive() + SparkSession.clearActiveSession() if (spark != null) { spark.stop() } From a14c88acce0733f3db8b0508ae8b0417822e08d8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 27 May 2016 11:10:31 -0700 Subject: [PATCH 0417/1470] [SPARK-15584][SQL] Abstract duplicate code: `spark.sql.sources.` properties ## What changes were proposed in this pull request? This PR replaces `spark.sql.sources.` strings with `CreateDataSourceTableUtils.*` constant variables. ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13349 from dongjoon-hyun/SPARK-15584. (cherry picked from commit 4538443e276597530a27c6922e48503677b13956) Signed-off-by: Andrew Or --- .../ml/source/libsvm/LibSVMRelation.scala | 3 +- .../command/createDataSourceTables.scala | 28 +++++---- .../spark/sql/execution/command/ddl.scala | 19 +++--- .../spark/sql/execution/command/tables.scala | 4 +- .../datasources/DataSourceStrategy.scala | 2 +- .../datasources/WriterContainer.scala | 10 ++-- .../datasources/csv/CSVRelation.scala | 3 +- .../datasources/json/JsonFileFormat.scala | 5 +- .../parquet/ParquetFileFormat.scala | 4 +- .../datasources/text/TextFileFormat.scala | 3 +- .../sql/execution/command/DDLSuite.scala | 10 ++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 +++--- .../spark/sql/hive/orc/OrcFileFormat.scala | 3 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 58 +++++++++---------- .../sql/hive/execution/HiveCommandSuite.scala | 16 +++-- .../sql/sources/SimpleTextRelation.scala | 3 +- 16 files changed, 95 insertions(+), 94 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 64ebf0c982346..7629369ab1379 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -51,7 +52,7 @@ private[libsvm] class LibSVMOutputWriter( new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val uniqueWriteJobId = configuration.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index deedb68a7817b..4b9aab612e7c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -256,15 +256,15 @@ case class CreateDataSourceTableAsSelectCommand( object CreateDataSourceTableUtils extends Logging { - // TODO: Actually replace usages with these variables (SPARK-15584) - val DATASOURCE_PREFIX = "spark.sql.sources." val DATASOURCE_PROVIDER = DATASOURCE_PREFIX + "provider" val DATASOURCE_WRITEJOBUUID = DATASOURCE_PREFIX + "writeJobUUID" val DATASOURCE_OUTPUTPATH = DATASOURCE_PREFIX + "output.path" - val DATASOURCE_SCHEMA_PREFIX = DATASOURCE_PREFIX + "schema." + val DATASOURCE_SCHEMA = DATASOURCE_PREFIX + "schema" + val DATASOURCE_SCHEMA_PREFIX = DATASOURCE_SCHEMA + "." val DATASOURCE_SCHEMA_NUMPARTS = DATASOURCE_SCHEMA_PREFIX + "numParts" val DATASOURCE_SCHEMA_NUMPARTCOLS = DATASOURCE_SCHEMA_PREFIX + "numPartCols" + val DATASOURCE_SCHEMA_NUMSORTCOLS = DATASOURCE_SCHEMA_PREFIX + "numSortCols" val DATASOURCE_SCHEMA_NUMBUCKETS = DATASOURCE_SCHEMA_PREFIX + "numBuckets" val DATASOURCE_SCHEMA_NUMBUCKETCOLS = DATASOURCE_SCHEMA_PREFIX + "numBucketCols" val DATASOURCE_SCHEMA_PART_PREFIX = DATASOURCE_SCHEMA_PREFIX + "part." @@ -296,7 +296,7 @@ object CreateDataSourceTableUtils extends Logging { options: Map[String, String], isExternal: Boolean): Unit = { val tableProperties = new mutable.HashMap[String, String] - tableProperties.put("spark.sql.sources.provider", provider) + tableProperties.put(DATASOURCE_PROVIDER, provider) // Saves optional user specified schema. Serialized JSON schema string may be too long to be // stored into a single metastore SerDe property. In this case, we split the JSON string and @@ -306,34 +306,32 @@ object CreateDataSourceTableUtils extends Logging { val schemaJsonString = schema.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq - tableProperties.put("spark.sql.sources.schema.numParts", parts.size.toString) + tableProperties.put(DATASOURCE_SCHEMA_NUMPARTS, parts.size.toString) parts.zipWithIndex.foreach { case (part, index) => - tableProperties.put(s"spark.sql.sources.schema.part.$index", part) + tableProperties.put(s"$DATASOURCE_SCHEMA_PART_PREFIX$index", part) } } if (userSpecifiedSchema.isDefined && partitionColumns.length > 0) { - tableProperties.put("spark.sql.sources.schema.numPartCols", partitionColumns.length.toString) + tableProperties.put(DATASOURCE_SCHEMA_NUMPARTCOLS, partitionColumns.length.toString) partitionColumns.zipWithIndex.foreach { case (partCol, index) => - tableProperties.put(s"spark.sql.sources.schema.partCol.$index", partCol) + tableProperties.put(s"$DATASOURCE_SCHEMA_PARTCOL_PREFIX$index", partCol) } } if (userSpecifiedSchema.isDefined && bucketSpec.isDefined) { val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames) = bucketSpec.get - tableProperties.put("spark.sql.sources.schema.numBuckets", numBuckets.toString) - tableProperties.put("spark.sql.sources.schema.numBucketCols", - bucketColumnNames.length.toString) + tableProperties.put(DATASOURCE_SCHEMA_NUMBUCKETS, numBuckets.toString) + tableProperties.put(DATASOURCE_SCHEMA_NUMBUCKETCOLS, bucketColumnNames.length.toString) bucketColumnNames.zipWithIndex.foreach { case (bucketCol, index) => - tableProperties.put(s"spark.sql.sources.schema.bucketCol.$index", bucketCol) + tableProperties.put(s"$DATASOURCE_SCHEMA_BUCKETCOL_PREFIX$index", bucketCol) } if (sortColumnNames.nonEmpty) { - tableProperties.put("spark.sql.sources.schema.numSortCols", - sortColumnNames.length.toString) + tableProperties.put(DATASOURCE_SCHEMA_NUMSORTCOLS, sortColumnNames.length.toString) sortColumnNames.zipWithIndex.foreach { case (sortCol, index) => - tableProperties.put(s"spark.sql.sources.schema.sortCol.$index", sortCol) + tableProperties.put(s"$DATASOURCE_SCHEMA_SORTCOL_PREFIX$index", sortCol) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 15eba3b0110b6..95bac94996577 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, CatalogTableType, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils.DATASOURCE_PREFIX +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.types._ @@ -464,7 +464,7 @@ case class AlterTableSetLocationCommand( object DDLUtils { def isDatasourceTable(props: Map[String, String]): Boolean = { - props.contains("spark.sql.sources.provider") + props.contains(DATASOURCE_PROVIDER) } def isDatasourceTable(table: CatalogTable): Boolean = { @@ -503,8 +503,7 @@ object DDLUtils { } def isTablePartitioned(table: CatalogTable): Boolean = { - table.partitionColumns.nonEmpty || - table.properties.contains("spark.sql.sources.schema.numPartCols") + table.partitionColumns.nonEmpty || table.properties.contains(DATASOURCE_SCHEMA_NUMPARTCOLS) } // A persisted data source table may not store its schema in the catalog. In this case, its schema @@ -512,15 +511,15 @@ object DDLUtils { def getSchemaFromTableProperties(metadata: CatalogTable): Option[StructType] = { require(isDatasourceTable(metadata)) val props = metadata.properties - if (props.isDefinedAt("spark.sql.sources.schema")) { + if (props.isDefinedAt(DATASOURCE_SCHEMA)) { // Originally, we used spark.sql.sources.schema to store the schema of a data source table. // After SPARK-6024, we removed this flag. // Although we are not using spark.sql.sources.schema any more, we need to still support. - props.get("spark.sql.sources.schema").map(DataType.fromJson(_).asInstanceOf[StructType]) + props.get(DATASOURCE_SCHEMA).map(DataType.fromJson(_).asInstanceOf[StructType]) } else { - metadata.properties.get("spark.sql.sources.schema.numParts").map { numParts => + metadata.properties.get(DATASOURCE_SCHEMA_NUMPARTS).map { numParts => val parts = (0 until numParts.toInt).map { index => - val part = metadata.properties.get(s"spark.sql.sources.schema.part.$index").orNull + val part = metadata.properties.get(s"$DATASOURCE_SCHEMA_PART_PREFIX$index").orNull if (part == null) { throw new AnalysisException( "Could not read schema from the metastore because it is corrupted " + @@ -543,7 +542,7 @@ object DDLUtils { numCols <- props.get(s"spark.sql.sources.schema.num${colType.capitalize}Cols").toSeq index <- 0 until numCols.toInt } yield props.getOrElse( - s"spark.sql.sources.schema.${colType}Col.$index", + s"$DATASOURCE_SCHEMA_PREFIX${colType}Col.$index", throw new AnalysisException( s"Corrupted $typeName in catalog: $numCols parts expected, but part $index is missing." ) @@ -556,7 +555,7 @@ object DDLUtils { def getBucketSpecFromTableProperties(metadata: CatalogTable): Option[BucketSpec] = { if (isDatasourceTable(metadata)) { - metadata.properties.get("spark.sql.sources.schema.numBuckets").map { numBuckets => + metadata.properties.get(DATASOURCE_SCHEMA_NUMBUCKETS).map { numBuckets => BucketSpec( numBuckets.toInt, getColumnNamesByType(metadata.properties, "bucket", "bucketing columns"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index d1024090d312d..2d6a3b48603d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -443,7 +443,7 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF table.properties.filterNot { // Hides schema properties that hold user-defined schema, partition columns, and bucketing // information since they are already extracted and shown in other parts. - case (key, _) => key.startsWith("spark.sql.sources.schema") + case (key, _) => key.startsWith(CreateDataSourceTableUtils.DATASOURCE_SCHEMA) }.foreach { case (key, value) => append(buffer, s" $key", value, "") } @@ -860,7 +860,7 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman private def showDataSourceTableOptions(metadata: CatalogTable, builder: StringBuilder): Unit = { val props = metadata.properties - builder ++= s"USING ${props("spark.sql.sources.provider")}\n" + builder ++= s"USING ${props(CreateDataSourceTableUtils.DATASOURCE_PROVIDER)}\n" val dataSourceOptions = metadata.storage.serdeProperties.filterNot { case (key, value) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index a3d87cd38bb8c..2b4786542c72f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -101,7 +101,7 @@ private[sql] class FindDataSourceTable(sparkSession: SparkSession) extends Rule[ userSpecifiedSchema = userSpecifiedSchema, partitionColumns = partitionColumns, bucketSpec = bucketSpec, - className = table.properties("spark.sql.sources.provider"), + className = table.properties(CreateDataSourceTableUtils.DATASOURCE_PROVIDER), options = options) LogicalRelation( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 61dcbebd644f6..f56b50a54385a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.UnsafeKVExternalSorter +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -91,7 +92,7 @@ private[sql] abstract class BaseWriterContainer( // This UUID is sent to executor side together with the serialized `Configuration` object within // the `Job` instance. `OutputWriters` on the executor side should use this UUID to generate // unique task output files. - job.getConfiguration.set("spark.sql.sources.writeJobUUID", uniqueWriteJobId.toString) + job.getConfiguration.set(DATASOURCE_WRITEJOBUUID, uniqueWriteJobId.toString) // Order of the following two lines is important. For Hadoop 1, TaskAttemptContext constructor // clones the Configuration object passed in. If we initialize the TaskAttemptContext first, @@ -241,7 +242,7 @@ private[sql] class DefaultWriterContainer( def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { executorSideSetup(taskContext) val configuration = taskAttemptContext.getConfiguration - configuration.set("spark.sql.sources.output.path", outputPath) + configuration.set(DATASOURCE_OUTPUTPATH, outputPath) var writer = newOutputWriter(getWorkPath) writer.initConverter(dataSchema) @@ -349,11 +350,10 @@ private[sql] class DynamicPartitionWriterContainer( val configuration = taskAttemptContext.getConfiguration val path = if (partitionColumns.nonEmpty) { val partitionPath = getPartitionString(key).getString(0) - configuration.set( - "spark.sql.sources.output.path", new Path(outputPath, partitionPath).toString) + configuration.set(DATASOURCE_OUTPUTPATH, new Path(outputPath, partitionPath).toString) new Path(getWorkPath, partitionPath).toString } else { - configuration.set("spark.sql.sources.output.path", outputPath) + configuration.set(DATASOURCE_OUTPUTPATH, outputPath) getWorkPath } val bucketId = getBucketIdFromKey(key) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 9849484dcec85..d72c8b9ac2e7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -30,6 +30,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.types._ @@ -168,7 +169,7 @@ private[sql] class CsvOutputWriter( new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val uniqueWriteJobId = configuration.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.csv$extension") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 35f247692ffdc..c7c528119660c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -32,8 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -170,7 +169,7 @@ private[json] class JsonOutputWriter( new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val uniqueWriteJobId = configuration.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index b47d41e1661f0..ff7962df22452 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -44,6 +44,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -521,7 +522,8 @@ private[sql] class ParquetOutputWriter( // partitions in the case of dynamic partitioning. override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val uniqueWriteJobId = configuration.get( + CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index d9525efe6d9be..1e5bce4a75978 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeRowWriter} +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructType} @@ -119,7 +120,7 @@ class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemp new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val uniqueWriteJobId = configuration.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.txt$extension") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e975756685dec..ccb4006483906 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFor import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils.DATASOURCE_PREFIX +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -387,7 +387,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val table = catalog.getTableMetadata(TableIdentifier("tbl")) assert(table.tableType == CatalogTableType.MANAGED) assert(table.schema == Seq(CatalogColumn("a", "int"), CatalogColumn("b", "int"))) - assert(table.properties("spark.sql.sources.provider") == "parquet") + assert(table.properties(DATASOURCE_PROVIDER) == "parquet") } } @@ -398,7 +398,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val table = catalog.getTableMetadata(TableIdentifier("tbl")) assert(table.tableType == CatalogTableType.MANAGED) assert(table.schema.isEmpty) // partitioned datasource table is not hive-compatible - assert(table.properties("spark.sql.sources.provider") == "parquet") + assert(table.properties(DATASOURCE_PROVIDER) == "parquet") assert(DDLUtils.getSchemaFromTableProperties(table) == Some(new StructType().add("a", IntegerType).add("b", IntegerType))) assert(DDLUtils.getPartitionColumnsFromTableProperties(table) == @@ -414,7 +414,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val table = catalog.getTableMetadata(TableIdentifier("tbl")) assert(table.tableType == CatalogTableType.MANAGED) assert(table.schema.isEmpty) // partitioned datasource table is not hive-compatible - assert(table.properties("spark.sql.sources.provider") == "parquet") + assert(table.properties(DATASOURCE_PROVIDER) == "parquet") assert(DDLUtils.getSchemaFromTableProperties(table) == Some(new StructType().add("a", IntegerType).add("b", IntegerType))) assert(DDLUtils.getBucketSpecFromTableProperties(table) == @@ -747,7 +747,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog: SessionCatalog, tableIdent: TableIdentifier): Unit = { catalog.alterTable(catalog.getTableMetadata(tableIdent).copy( - properties = Map("spark.sql.sources.provider" -> "csv"))) + properties = Map(DATASOURCE_PROVIDER -> "csv"))) } private def testSetProperties(isDatasourceTable: Boolean): Unit = { 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 ea721e4d9b6ec..ff395f39b7052 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 @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.command.CreateTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -74,9 +75,9 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // TODO: the following code is duplicated with FindDataSourceTable.readDataSourceTable def schemaStringFromParts: Option[String] = { - table.properties.get("spark.sql.sources.schema.numParts").map { numParts => + table.properties.get(DATASOURCE_SCHEMA_NUMPARTS).map { numParts => val parts = (0 until numParts.toInt).map { index => - val part = table.properties.get(s"spark.sql.sources.schema.part.$index").orNull + val part = table.properties.get(s"$DATASOURCE_SCHEMA_PART_PREFIX$index").orNull if (part == null) { throw new AnalysisException( "Could not read schema from the metastore because it is corrupted " + @@ -91,9 +92,9 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } def getColumnNames(colType: String): Seq[String] = { - table.properties.get(s"spark.sql.sources.schema.num${colType.capitalize}Cols").map { + table.properties.get(s"$DATASOURCE_SCHEMA.num${colType.capitalize}Cols").map { numCols => (0 until numCols.toInt).map { index => - table.properties.getOrElse(s"spark.sql.sources.schema.${colType}Col.$index", + table.properties.getOrElse(s"$DATASOURCE_SCHEMA_PREFIX${colType}Col.$index", throw new AnalysisException( s"Could not read $colType columns from the metastore because it is corrupted " + s"(missing part $index of it, $numCols parts are expected).")) @@ -104,8 +105,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // Originally, we used spark.sql.sources.schema to store the schema of a data source table. // After SPARK-6024, we removed this flag. // Although we are not using spark.sql.sources.schema any more, we need to still support. - val schemaString = - table.properties.get("spark.sql.sources.schema").orElse(schemaStringFromParts) + val schemaString = table.properties.get(DATASOURCE_SCHEMA).orElse(schemaStringFromParts) val userSpecifiedSchema = schemaString.map(s => DataType.fromJson(s).asInstanceOf[StructType]) @@ -115,7 +115,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // from userSpecifiedSchema. val partitionColumns = getColumnNames("part") - val bucketSpec = table.properties.get("spark.sql.sources.schema.numBuckets").map { n => + val bucketSpec = table.properties.get(DATASOURCE_SCHEMA_NUMBUCKETS).map { n => BucketSpec(n.toInt, getColumnNames("bucket"), getColumnNames("sort")) } @@ -126,7 +126,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log userSpecifiedSchema = userSpecifiedSchema, partitionColumns = partitionColumns, bucketSpec = bucketSpec, - className = table.properties("spark.sql.sources.provider"), + className = table.properties(DATASOURCE_PROVIDER), options = options) LogicalRelation( @@ -166,7 +166,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val qualifiedTableName = getQualifiedTableName(tableIdent) val table = client.getTable(qualifiedTableName.database, qualifiedTableName.name) - if (table.properties.get("spark.sql.sources.provider").isDefined) { + if (table.properties.get(DATASOURCE_PROVIDER).isDefined) { val dataSourceTable = cachedDataSourceTables(qualifiedTableName) val qualifiedTable = SubqueryAlias(qualifiedTableName.name, dataSourceTable) // Then, if alias is specified, wrap the table with a Subquery using the alias. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index f1198179a0994..0e8c37df88ee5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -36,6 +36,7 @@ import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.{HiveInspectors, HiveShim} import org.apache.spark.sql.sources.{Filter, _} @@ -217,7 +218,7 @@ private[orc] class OrcOutputWriter( private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - val uniqueWriteJobId = conf.get("spark.sql.sources.writeJobUUID") + val uniqueWriteJobId = conf.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val partition = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 1e6de463b3eba..2c50cc88cc4cc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -700,7 +700,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType, true))) // Manually create a metastore data source table. - CreateDataSourceTableUtils.createDataSourceTable( + createDataSourceTable( sparkSession = spark, tableIdent = TableIdentifier("wide_schema"), userSpecifiedSchema = Some(schema), @@ -737,8 +737,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv "path" -> sessionState.catalog.hiveDefaultTableFilePath(TableIdentifier(tableName))) ), properties = Map( - "spark.sql.sources.provider" -> "json", - "spark.sql.sources.schema" -> schema.json, + DATASOURCE_PROVIDER -> "json", + DATASOURCE_SCHEMA -> schema.json, "EXTERNAL" -> "FALSE")) sharedState.externalCatalog.createTable("default", hiveTable, ignoreIfExists = false) @@ -762,13 +762,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val metastoreTable = sharedState.externalCatalog.getTable("default", tableName) val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) - val numPartCols = metastoreTable.properties("spark.sql.sources.schema.numPartCols").toInt + val numPartCols = metastoreTable.properties(DATASOURCE_SCHEMA_NUMPARTCOLS).toInt assert(numPartCols == 2) val actualPartitionColumns = StructType( (0 until numPartCols).map { index => - df.schema(metastoreTable.properties(s"spark.sql.sources.schema.partCol.$index")) + df.schema(metastoreTable.properties(s"$DATASOURCE_SCHEMA_PARTCOL_PREFIX$index")) }) // Make sure partition columns are correctly stored in metastore. assert( @@ -798,19 +798,19 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val expectedBucketByColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val expectedSortByColumns = StructType(df.schema("c") :: Nil) - val numBuckets = metastoreTable.properties("spark.sql.sources.schema.numBuckets").toInt + val numBuckets = metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt assert(numBuckets == 8) - val numBucketCols = metastoreTable.properties("spark.sql.sources.schema.numBucketCols").toInt + val numBucketCols = metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt assert(numBucketCols == 2) - val numSortCols = metastoreTable.properties("spark.sql.sources.schema.numSortCols").toInt + val numSortCols = metastoreTable.properties(DATASOURCE_SCHEMA_NUMSORTCOLS).toInt assert(numSortCols == 1) val actualBucketByColumns = StructType( (0 until numBucketCols).map { index => - df.schema(metastoreTable.properties(s"spark.sql.sources.schema.bucketCol.$index")) + df.schema(metastoreTable.properties(s"$DATASOURCE_SCHEMA_BUCKETCOL_PREFIX$index")) }) // Make sure bucketBy columns are correctly stored in metastore. assert( @@ -821,7 +821,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val actualSortByColumns = StructType( (0 until numSortCols).map { index => - df.schema(metastoreTable.properties(s"spark.sql.sources.schema.sortCol.$index")) + df.schema(metastoreTable.properties(s"$DATASOURCE_SCHEMA_SORTCOL_PREFIX$index")) }) // Make sure sortBy columns are correctly stored in metastore. assert( @@ -913,7 +913,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv withTempDir { tempPath => val schema = StructType((1 to 5).map(i => StructField(s"c_$i", StringType))) - CreateDataSourceTableUtils.createDataSourceTable( + createDataSourceTable( sparkSession = spark, tableIdent = TableIdentifier("not_skip_hive_metadata"), userSpecifiedSchema = Some(schema), @@ -928,7 +928,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert(sharedState.externalCatalog.getTable("default", "not_skip_hive_metadata").schema .forall(column => CatalystSqlParser.parseDataType(column.dataType) == StringType)) - CreateDataSourceTableUtils.createDataSourceTable( + createDataSourceTable( sparkSession = spark, tableIdent = TableIdentifier("skip_hive_metadata"), userSpecifiedSchema = Some(schema), @@ -960,10 +960,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv ) val metastoreTable = sharedState.externalCatalog.getTable("default", "t") - assert(metastoreTable.properties("spark.sql.sources.schema.numPartCols").toInt === 1) - assert(!metastoreTable.properties.contains("spark.sql.sources.schema.numBuckets")) - assert(!metastoreTable.properties.contains("spark.sql.sources.schema.numBucketCols")) - assert(!metastoreTable.properties.contains("spark.sql.sources.schema.numSortCols")) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMPARTCOLS).toInt === 1) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMBUCKETS)) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMBUCKETCOLS)) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMSORTCOLS)) checkAnswer(table("t"), Row(2, 1)) } @@ -984,10 +984,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv ) val metastoreTable = sharedState.externalCatalog.getTable("default", "t") - assert(!metastoreTable.properties.contains("spark.sql.sources.schema.numPartCols")) - assert(metastoreTable.properties("spark.sql.sources.schema.numBuckets").toInt === 2) - assert(metastoreTable.properties("spark.sql.sources.schema.numBucketCols").toInt === 1) - assert(metastoreTable.properties("spark.sql.sources.schema.numSortCols").toInt === 1) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMPARTCOLS)) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt === 2) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt === 1) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMSORTCOLS).toInt === 1) checkAnswer(table("t"), Row(1, 2)) } @@ -1006,10 +1006,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv ) val metastoreTable = sharedState.externalCatalog.getTable("default", "t") - assert(!metastoreTable.properties.contains("spark.sql.sources.schema.numPartCols")) - assert(metastoreTable.properties("spark.sql.sources.schema.numBuckets").toInt === 2) - assert(metastoreTable.properties("spark.sql.sources.schema.numBucketCols").toInt === 1) - assert(!metastoreTable.properties.contains("spark.sql.sources.schema.numSortCols")) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMPARTCOLS)) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt === 2) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt === 1) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMSORTCOLS)) checkAnswer(table("t"), Row(1, 2)) } @@ -1031,10 +1031,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv ) val metastoreTable = sharedState.externalCatalog.getTable("default", "t") - assert(metastoreTable.properties("spark.sql.sources.schema.numPartCols").toInt === 1) - assert(metastoreTable.properties("spark.sql.sources.schema.numBuckets").toInt === 2) - assert(metastoreTable.properties("spark.sql.sources.schema.numBucketCols").toInt === 1) - assert(metastoreTable.properties("spark.sql.sources.schema.numSortCols").toInt === 1) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMPARTCOLS).toInt === 1) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt === 2) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt === 1) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMSORTCOLS).toInt === 1) checkAnswer(table("t"), Row(2, 3, 1)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 6f374d713bfc8..741abcb7513cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException -import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -101,24 +101,22 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto test("show tblproperties of data source tables - basic") { checkAnswer( - sql("SHOW TBLPROPERTIES parquet_tab1") - .filter(s"key = 'spark.sql.sources.provider'"), - Row("spark.sql.sources.provider", "org.apache.spark.sql.parquet.DefaultSource") :: Nil + sql("SHOW TBLPROPERTIES parquet_tab1").filter(s"key = '$DATASOURCE_PROVIDER'"), + Row(DATASOURCE_PROVIDER, "org.apache.spark.sql.parquet.DefaultSource") :: Nil ) checkAnswer( - sql("SHOW TBLPROPERTIES parquet_tab1(spark.sql.sources.provider)"), + sql(s"SHOW TBLPROPERTIES parquet_tab1($DATASOURCE_PROVIDER)"), Row("org.apache.spark.sql.parquet.DefaultSource") :: Nil ) checkAnswer( - sql("SHOW TBLPROPERTIES parquet_tab1") - .filter(s"key = 'spark.sql.sources.schema.numParts'"), - Row("spark.sql.sources.schema.numParts", "1") :: Nil + sql("SHOW TBLPROPERTIES parquet_tab1").filter(s"key = '$DATASOURCE_SCHEMA_NUMPARTS'"), + Row(DATASOURCE_SCHEMA_NUMPARTS, "1") :: Nil ) checkAnswer( - sql("SHOW TBLPROPERTIES parquet_tab1('spark.sql.sources.schema.numParts')"), + sql(s"SHOW TBLPROPERTIES parquet_tab1('$DATASOURCE_SCHEMA_NUMPARTS')"), Row("1")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 0fa18414154ce..1fb777ade4b22 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.{sources, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedPredicate, InterpretedProjection, JoinedRow, Literal} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.SerializableConfiguration @@ -144,7 +145,7 @@ class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullW override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val uniqueWriteJobId = configuration.get(DATASOURCE_WRITEJOBUUID) val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId val name = FileOutputFormat.getOutputName(context) From e69639f4334aae3ace5e50452603dd667467ea9a Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 27 May 2016 11:11:31 -0700 Subject: [PATCH 0418/1470] [SPARK-15599][SQL][DOCS] API docs for `createDataset` functions in SparkSession ## What changes were proposed in this pull request? Adds API docs and usage examples for the 3 `createDataset` calls in `SparkSession` ## How was this patch tested? N/A Author: Sameer Agarwal Closes #13345 from sameeragarwal/dataset-doc. (cherry picked from commit 635fb30f83a66cc56f5fecfed5bff77873bf49a6) Signed-off-by: Andrew Or --- .../org/apache/spark/sql/SparkSession.scala | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5dabe0e83c1cf..aa60048405532 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -376,6 +376,40 @@ class SparkSession private( Dataset.ofRows(self, LogicalRelation(baseRelation)) } + /* ------------------------------- * + | Methods for creating DataSets | + * ------------------------------- */ + + /** + * :: Experimental :: + * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Example == + * + * {{{ + * + * import spark.implicits._ + * case class Person(name: String, age: Long) + * val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19)) + * val ds = spark.createDataset(data) + * + * ds.show() + * // +-------+---+ + * // | name|age| + * // +-------+---+ + * // |Michael| 29| + * // | Andy| 30| + * // | Justin| 19| + * // +-------+---+ + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -384,6 +418,17 @@ class SparkSession private( Dataset[T](self, plan) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from an RDD of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -392,6 +437,24 @@ class SparkSession private( Dataset[T](self, plan) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from a [[java.util.List]] of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Java Example == + * + * {{{ + * List data = Arrays.asList("hello", "world"); + * Dataset ds = spark.createDataset(data, Encoders.STRING()); + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } From f52a9524865b8c56058a65b29a1aaacffb709f69 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 27 May 2016 11:13:09 -0700 Subject: [PATCH 0419/1470] [SPARK-15597][SQL] Add SparkSession.emptyDataset ## What changes were proposed in this pull request? This patch adds a new function emptyDataset to SparkSession, for creating an empty dataset. ## How was this patch tested? Added a test case. Author: Reynold Xin Closes #13344 from rxin/SPARK-15597. (cherry picked from commit a52e6813392ba4bdb1b818694b7ced8f6caa6a2b) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/sql/SparkSession.scala | 12 ++++++++++++ .../scala/org/apache/spark/sql/DatasetSuite.scala | 6 ++++++ 2 files changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index aa60048405532..c9276cf140d65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -222,6 +222,18 @@ class SparkSession private( createDataFrame(sparkContext.emptyRDD[Row], StructType(Nil)) } + /** + * :: Experimental :: + * Creates a new [[Dataset]] of type T containing zero elements. + * + * @return 2.0.0 + */ + @Experimental + def emptyDataset[T: Encoder]: Dataset[T] = { + val encoder = implicitly[Encoder[T]] + new Dataset(self, LocalRelation(encoder.schema.toAttributes), encoder) + } + /** * :: Experimental :: * Creates a [[DataFrame]] from an RDD of Product (e.g. case classes, tuples). diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 2a6591653e644..e395007999592 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -46,6 +46,12 @@ class DatasetSuite extends QueryTest with SharedSQLContext { 1, 1, 1) } + test("emptyDataset") { + val ds = spark.emptyDataset[Int] + assert(ds.count() == 0L) + assert(ds.collect() sameElements Array.empty[Int]) + } + test("range") { assert(spark.range(10).map(_ + 1).reduce(_ + _) == 55) assert(spark.range(10).map{ case i: java.lang.Long => i + 1 }.reduce(_ + _) == 55) From 89fdb6972d5410f250bc56f8a834c939ee6653d2 Mon Sep 17 00:00:00 2001 From: Xinh Huynh Date: Fri, 27 May 2016 11:13:53 -0700 Subject: [PATCH 0420/1470] [MINOR][DOCS] Typo fixes in Dataset scaladoc ## What changes were proposed in this pull request? Minor typo fixes in Dataset scaladoc * Corrected context type as SparkSession, not SQLContext. liancheng rxin andrewor14 ## How was this patch tested? Compiled locally Author: Xinh Huynh Closes #13330 from xinhhuynh/fix-dataset-typos. (cherry picked from commit 5bdbedf2201efa6c34392aa9eff709761f027e1d) Signed-off-by: Andrew Or --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 85f0cf8a60415..abd16f21499e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -121,7 +121,7 @@ private[sql] object Dataset { * * A more concrete example in Scala: * {{{ - * // To create Dataset[Row] using SQLContext + * // To create Dataset[Row] using SparkSession * val people = spark.read.parquet("...") * val department = spark.read.parquet("...") * @@ -133,7 +133,7 @@ private[sql] object Dataset { * * and in Java: * {{{ - * // To create Dataset using SQLContext + * // To create Dataset using SparkSession * Dataset people = spark.read().parquet("..."); * Dataset department = spark.read().parquet("..."); * From 30e87b55b6f59ca029778087710effc768fafc35 Mon Sep 17 00:00:00 2001 From: Sital Kedia Date: Fri, 27 May 2016 11:22:39 -0700 Subject: [PATCH 0421/1470] =?UTF-8?q?[SPARK-15569]=20Reduce=20frequency=20?= =?UTF-8?q?of=20updateBytesWritten=20function=20in=20Disk=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Profiling a Spark job spilling large amount of intermediate data we found that significant portion of time is being spent in DiskObjectWriter.updateBytesWritten function. Looking at the code, we see that the function is being called too frequently to update the number of bytes written to disk. We should reduce the frequency to avoid this. ## How was this patch tested? Tested by running the job on cluster and saw 20% CPU gain by this change. Author: Sital Kedia Closes #13332 from sitalkedia/DiskObjectWriter. (cherry picked from commit ce756daa4f012ebdc5a41bf5a89ff11b6dfdab8c) Signed-off-by: Andrew Or --- .../apache/spark/storage/DiskBlockObjectWriter.scala | 3 +-- .../spark/storage/DiskBlockObjectWriterSuite.scala | 12 ++++++------ 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index ab97d2e4b8b78..5b493f470b50a 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -203,8 +203,7 @@ private[spark] class DiskBlockObjectWriter( numRecordsWritten += 1 writeMetrics.incRecordsWritten(1) - // TODO: call updateBytesWritten() less frequently. - if (numRecordsWritten % 32 == 0) { + if (numRecordsWritten % 16384 == 0) { updateBytesWritten() } } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index 8eff3c297035d..ec4ef4b2fcbf0 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -53,13 +53,13 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.bytesWritten == 0) - // After 32 writes, metrics should update - for (i <- 0 until 32) { + // After 16384 writes, metrics should update + for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.bytesWritten > 0) - assert(writeMetrics.recordsWritten === 33) + assert(writeMetrics.recordsWritten === 16385) writer.commitAndClose() assert(file.length() == writeMetrics.bytesWritten) } @@ -75,13 +75,13 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.bytesWritten == 0) - // After 32 writes, metrics should update - for (i <- 0 until 32) { + // After 16384 writes, metrics should update + for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.bytesWritten > 0) - assert(writeMetrics.recordsWritten === 33) + assert(writeMetrics.recordsWritten === 16385) writer.revertPartialWritesAndClose() assert(writeMetrics.bytesWritten == 0) assert(writeMetrics.recordsWritten == 0) From 074989af945b8ebb2779f94b8714752b67f3e82f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 27 May 2016 11:28:28 -0700 Subject: [PATCH 0422/1470] [SPARK-15531][DEPLOY] spark-class tries to use too much memory when running Launcher ## What changes were proposed in this pull request? Explicitly limit launcher JVM memory to modest 128m ## How was this patch tested? Jenkins tests. Author: Sean Owen Closes #13360 from srowen/SPARK-15531. (cherry picked from commit 623aae5907f4ba8b7079c21328e0c0b5fef7bb00) Signed-off-by: Marcelo Vanzin --- bin/spark-class | 2 +- bin/spark-class2.cmd | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 23a60c6ee44fe..658e076bc0462 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -68,7 +68,7 @@ fi # The exit code of the launcher is appended to the output, so the parent shell removes it from the # command array and checks the value to see if the launcher succeeded. build_command() { - "$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@" + "$RUNNER" -Xmx128m -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@" printf "%d\0" $? } diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index db680218dc964..869c0b202f7f3 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -55,7 +55,7 @@ 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. set LAUNCHER_OUTPUT=%temp%\spark-class-launcher-output-%RANDOM%.txt -"%RUNNER%" -cp "%LAUNCH_CLASSPATH%" org.apache.spark.launcher.Main %* > %LAUNCHER_OUTPUT% +"%RUNNER%" -Xmx128m -cp "%LAUNCH_CLASSPATH%" org.apache.spark.launcher.Main %* > %LAUNCHER_OUTPUT% for /f "tokens=*" %%i in (%LAUNCHER_OUTPUT%) do ( set SPARK_CMD=%%i ) From 17f43cc87ed8b3c77b7c34163340da8e2da48eb1 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 27 May 2016 11:31:25 -0700 Subject: [PATCH 0423/1470] [YARN][DOC][MINOR] Remove several obsolete env variables and update the doc ## What changes were proposed in this pull request? Remove several obsolete env variables not supported for Spark on YARN now, also updates the docs to include several changes with 2.0. ## How was this patch tested? N/A CC vanzin tgravescs Author: jerryshao Closes #13296 from jerryshao/yarn-doc. (cherry picked from commit 1b98fa2e4382d3d8385cf1ac25d7fd3ae5650475) Signed-off-by: Marcelo Vanzin --- conf/spark-env.sh.template | 4 ---- docs/running-on-yarn.md | 4 ++++ 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index a031cd6a722f9..9cffdc30c20fe 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -40,10 +40,6 @@ # - SPARK_EXECUTOR_CORES, Number of cores for the executors (Default: 1). # - SPARK_EXECUTOR_MEMORY, Memory per Executor (e.g. 1000M, 2G) (Default: 1G) # - SPARK_DRIVER_MEMORY, Memory for Driver (e.g. 1000M, 2G) (Default: 1G) -# - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) -# - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: 'default') -# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. -# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job. # Options for the daemons used in the standalone deploy mode # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index f2fbe3ca56a1f..9833806716aab 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -60,6 +60,8 @@ Running Spark on YARN requires a binary distribution of Spark which is built wit Binary distributions can be downloaded from the [downloads page](http://spark.apache.org/downloads.html) of the project website. To build Spark yourself, refer to [Building Spark](building-spark.html). +To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](running-on-yarn.html#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. + # Configuration Most of the configs are the same for Spark on YARN as for other deployment modes. See the [configuration page](configuration.html) for more information on those. These are configs that are specific to Spark on YARN. @@ -99,6 +101,8 @@ to the same log file). If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use `spark.yarn.app.container.log.dir` in your `log4j.properties`. For example, `log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log`. For streaming applications, configuring `RollingFileAppender` and setting file location to YARN's log directory will avoid disk overflow caused by large log files, and logs can be accessed using YARN's log utility. +To use a custom metrics.properties for the application master and executors, update the `$SPARK_CONF_DIR/metrics.properties` file. It will automatically be uploaded with other configurations, so you don't need to specify it manually with `--files`. + #### Spark Properties
  • Property NameDefaultMeaning
    From 5ea58898cc9413fd0b04b60db230c8894d8bb9ef Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 27 May 2016 11:43:01 -0700 Subject: [PATCH 0424/1470] [HOTFIX] Scala 2.10 compile GaussianMixtureModel --- .../apache/spark/mllib/clustering/GaussianMixtureModel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 4b068164b8502..f470b0f3c3690 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -170,7 +170,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { (weight, new MultivariateGaussian(mu, sigma)) }.unzip - new GaussianMixtureModel(weights, gaussians) + new GaussianMixtureModel(weights.toArray, gaussians.toArray) } } From d76e066d3355a9942af3ae4f23d81a948c236e5e Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Fri, 27 May 2016 12:05:11 -0700 Subject: [PATCH 0425/1470] [SPARK-14400][SQL] ScriptTransformation does not fail the job for bad user command ## What changes were proposed in this pull request? - Refer to the Jira for the problem: jira : https://issues.apache.org/jira/browse/SPARK-14400 - The fix is to check if the process has exited with a non-zero exit code in `hasNext()`. I have moved this and checking of writer thread exception to a separate method. ## How was this patch tested? - Ran a job which had incorrect transform script command and saw that the job fails - Existing unit tests for `ScriptTransformationSuite`. Added a new unit test Author: Tejas Patil Closes #12194 from tejasapatil/script_transform. (cherry picked from commit a96e4151a9d429cfaf457c07b4ce174890a3b39b) Signed-off-by: Reynold Xin --- .../spark/sql/execution/SparkPlanTest.scala | 7 +- .../hive/execution/ScriptTransformation.scala | 90 ++++++++++++------- .../execution/ScriptTransformationSuite.scala | 18 +++- 3 files changed, 81 insertions(+), 34 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala index 9fe0e9646e31e..b29e822add8bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala @@ -231,7 +231,12 @@ object SparkPlanTest { } } - private def executePlan(outputPlan: SparkPlan, spark: SQLContext): Seq[Row] = { + /** + * Runs the plan + * @param outputPlan SparkPlan to be executed + * @param spark SqlContext used for execution of the plan + */ + def executePlan(outputPlan: SparkPlan, spark: SQLContext): Seq[Row] = { val execution = new QueryExecution(spark.sparkSession, null) { override lazy val sparkPlan: SparkPlan = outputPlan transform { case plan: SparkPlan => 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 f6e6a75c3ee58..9e25e1d40ce81 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 @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.io.Writable -import org.apache.spark.TaskContext +import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} @@ -127,45 +127,71 @@ case class ScriptTransformation( } val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + private def checkFailureAndPropagate(cause: Throwable = null): Unit = { + if (writerThread.exception.isDefined) { + throw writerThread.exception.get + } + + // Checks if the proc is still alive (incase the command ran was bad) + // The ideal way to do this is to use Java 8's Process#isAlive() + // but it cannot be used because Spark still supports Java 7. + // Following is a workaround used to check if a process is alive in Java 7 + // TODO: Once builds are switched to Java 8, this can be changed + try { + val exitCode = proc.exitValue() + if (exitCode != 0) { + logError(stderrBuffer.toString) // log the stderr circular buffer + throw new SparkException(s"Subprocess exited with status $exitCode. " + + s"Error: ${stderrBuffer.toString}", cause) + } + } catch { + case _: IllegalThreadStateException => + // This means that the process is still alive. Move ahead + } + } + override def hasNext: Boolean = { - if (outputSerde == null) { - if (curLine == null) { - curLine = reader.readLine() + try { + if (outputSerde == null) { if (curLine == null) { - if (writerThread.exception.isDefined) { - throw writerThread.exception.get + curLine = reader.readLine() + if (curLine == null) { + checkFailureAndPropagate() + return false } - false - } else { - true } - } else { - true - } - } else if (scriptOutputWritable == null) { - scriptOutputWritable = reusedWritableObject + } else if (scriptOutputWritable == null) { + scriptOutputWritable = reusedWritableObject - if (scriptOutputReader != null) { - if (scriptOutputReader.next(scriptOutputWritable) <= 0) { - writerThread.exception.foreach(throw _) - false + if (scriptOutputReader != null) { + if (scriptOutputReader.next(scriptOutputWritable) <= 0) { + checkFailureAndPropagate() + return false + } } else { - true - } - } else { - try { - scriptOutputWritable.readFields(scriptOutputStream) - true - } catch { - case _: EOFException => - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } - false + try { + scriptOutputWritable.readFields(scriptOutputStream) + } catch { + case _: EOFException => + // This means that the stdout of `proc` (ie. TRANSFORM process) has exhausted. + // Ideally the proc should *not* be alive at this point but + // there can be a lag between EOF being written out and the process + // being terminated. So explicitly waiting for the process to be done. + proc.waitFor() + checkFailureAndPropagate() + return false + } } } - } else { + true + } catch { + case NonFatal(e) => + // If this exception is due to abrupt / unclean termination of `proc`, + // then detect it and propagate a better exception message for end users + checkFailureAndPropagate(e) + + throw e } } @@ -284,7 +310,6 @@ private class ScriptTransformationWriterThread( } } } - outputStream.close() threwException = false } catch { case NonFatal(e) => @@ -295,6 +320,7 @@ private class ScriptTransformationWriterThread( throw e } finally { try { + outputStream.close() if (proc.waitFor() != 0) { logError(stderrBuffer.toString) // log the stderr circular buffer } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 6f8062240732a..a8e81d7a3c42a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.scalatest.exceptions.TestFailedException -import org.apache.spark.TaskContext +import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} @@ -109,6 +109,22 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { } assert(e.getMessage().contains("intentional exception")) } + + test("SPARK-14400 script transformation should fail for bad script command") { + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") + + val e = intercept[SparkException] { + val plan = + new ScriptTransformation( + input = Seq(rowsDf.col("a").expr), + script = "some_non_existent_command", + output = Seq(AttributeReference("a", StringType)()), + child = rowsDf.queryExecution.sparkPlan, + ioschema = serdeIOSchema) + SparkPlanTest.executePlan(plan, hiveContext) + } + assert(e.getMessage.contains("Subprocess exited with status")) + } } private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExecNode { From 5dd1423f462f03b7ae625a93cdaf9d882969afb6 Mon Sep 17 00:00:00 2001 From: sethah Date: Fri, 27 May 2016 12:55:48 -0700 Subject: [PATCH 0426/1470] [SPARK-15186][ML][DOCS] Add user guide for generalized linear regression ## What changes were proposed in this pull request? This patch adds a user guide section for generalized linear regression and includes the examples from [#12754](https://github.com/apache/spark/pull/12754). ## How was this patch tested? Documentation only, no tests required. ## Approach In general, it is a bit unclear what level of detail ought to be included in the user guide since there is a lot of variability within the current user guide. I tried to give a fairly brief mathematical introduction to GLMs, and cover what types of problems they could be used for. Additionally, I included a brief blurb on the IRLS solver. The input/output columns are given in a table as is found elsewhere in the docs (though, again, these appear rather intermittently in the current docs), as well as a table providing the supported families and their link functions. Author: sethah Closes #13139 from sethah/SPARK-15186. (cherry picked from commit c96244f5acd8b335e34694c171bab32d92e6e0fb) Signed-off-by: Joseph K. Bradley --- docs/ml-classification-regression.md | 132 +++++++++++++++++++++++++++ 1 file changed, 132 insertions(+) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index f1a21f436f133..ff8dec6d2d3a4 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -374,6 +374,138 @@ regression model and extracting model summary statistics. +## Generalized linear regression + +Contrasted with linear regression where the output is assumed to follow a Gaussian +distribution, [generalized linear models](https://en.wikipedia.org/wiki/Generalized_linear_model) (GLMs) are specifications of linear models where the response variable $Y_i$ follows some +distribution from the [exponential family of distributions](https://en.wikipedia.org/wiki/Exponential_family). +Spark's `GeneralizedLinearRegression` interface +allows for flexible specification of GLMs which can be used for various types of +prediction problems including linear regression, Poisson regression, logistic regression, and others. +Currently in `spark.ml`, only a subset of the exponential family distributions are supported and they are listed +[below](#available-families). + +**NOTE**: Spark currently only supports up to 4096 features through its `GeneralizedLinearRegression` +interface, and will throw an exception if this constraint is exceeded. See the [advanced section](ml-advanced) for more details. + Still, for linear and logistic regression, models with an increased number of features can be trained + using the `LinearRegression` and `LogisticRegression` estimators. + +GLMs require exponential family distributions that can be written in their "canonical" or "natural" form, aka +[natural exponential family distributions](https://en.wikipedia.org/wiki/Natural_exponential_family). The form of a natural exponential family distribution is given as: + +$$ +f_Y(y|\theta, \tau) = h(y, \tau)\exp{\left( \frac{\theta \cdot y - A(\theta)}{d(\tau)} \right)} +$$ + +where $\theta$ is the parameter of interest and $\tau$ is a dispersion parameter. In a GLM the response variable $Y_i$ is assumed to be drawn from a natural exponential family distribution: + +$$ +Y_i \sim f\left(\cdot|\theta_i, \tau \right) +$$ + +where the parameter of interest $\theta_i$ is related to the expected value of the response variable $\mu_i$ by + +$$ +\mu_i = A'(\theta_i) +$$ + +Here, $A'(\theta_i)$ is defined by the form of the distribution selected. GLMs also allow specification +of a link function, which defines the relationship between the expected value of the response variable $\mu_i$ +and the so called _linear predictor_ $\eta_i$: + +$$ +g(\mu_i) = \eta_i = \vec{x_i}^T \cdot \vec{\beta} +$$ + +Often, the link function is chosen such that $A' = g^{-1}$, which yields a simplified relationship +between the parameter of interest $\theta$ and the linear predictor $\eta$. In this case, the link +function $g(\mu)$ is said to be the "canonical" link function. + +$$ +\theta_i = A'^{-1}(\mu_i) = g(g^{-1}(\eta_i)) = \eta_i +$$ + +A GLM finds the regression coefficients $\vec{\beta}$ which maximize the likelihood function. + +$$ +\max_{\vec{\beta}} \mathcal{L}(\vec{\theta}|\vec{y},X) = +\prod_{i=1}^{N} h(y_i, \tau) \exp{\left(\frac{y_i\theta_i - A(\theta_i)}{d(\tau)}\right)} +$$ + +where the parameter of interest $\theta_i$ is related to the regression coefficients $\vec{\beta}$ +by + +$$ +\theta_i = A'^{-1}(g^{-1}(\vec{x_i} \cdot \vec{\beta})) +$$ + +Spark's generalized linear regression interface also provides summary statistics for diagnosing the +fit of GLM models, including residuals, p-values, deviances, the Akaike information criterion, and +others. + +[See here](http://data.princeton.edu/wws509/notes/) for a more comprehensive review of GLMs and their applications. + +### Available families + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    FamilyResponse TypeSupported Links
    GaussianContinuousIdentity*, Log, Inverse
    BinomialBinaryLogit*, Probit, CLogLog
    PoissonCountLog*, Identity, Sqrt
    GammaContinuousInverse*, Idenity, Log
    * Canonical Link
    + +**Example** + +The following example demonstrates training a GLM with a Gaussian response and identity link +function and extracting model summary statistics. + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.GeneralizedLinearRegression) for more details. + +{% include_example scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/GeneralizedLinearRegression.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.GeneralizedLinearRegression) for more details. + +{% include_example python/ml/generalized_linear_regression_example.py %} +
    + +
    + ## Decision tree regression From e6e2f293d6830ce118050e789773a09b3888fd30 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 27 May 2016 13:16:22 -0700 Subject: [PATCH 0427/1470] [SPARK-11959][SPARK-15484][DOC][ML] Document WLS and IRLS ## What changes were proposed in this pull request? * Document ```WeightedLeastSquares```(normal equation) and ```IterativelyReweightedLeastSquares```. * Copy ```L-BFGS``` documents from ```spark.mllib``` to ```spark.ml```. Due to the session ```Optimization of linear methods``` is used for developers, I think we should provide the brief introduction of the optimization method, necessary references and how it implements in Spark. It's not necessary to paste all mathematical formula and derivation here. If developers/users want to learn more, they can track reference. ## How was this patch tested? Document update, no tests. Author: Yanbo Liang Closes #13262 from yanboliang/spark-15484. (cherry picked from commit a3550e3747e21c79a5110132dc127ee83879062a) Signed-off-by: Joseph K. Bradley --- docs/ml-advanced.md | 85 +++++++++++++++++-- .../IterativelyReweightedLeastSquares.scala | 2 +- 2 files changed, 81 insertions(+), 6 deletions(-) diff --git a/docs/ml-advanced.md b/docs/ml-advanced.md index 91731d78a2d43..1c5f844b08e40 100644 --- a/docs/ml-advanced.md +++ b/docs/ml-advanced.md @@ -4,10 +4,85 @@ title: Advanced topics - spark.ml displayTitle: Advanced topics - spark.ml --- -# Optimization of linear methods +* Table of contents +{:toc} + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +# Optimization of linear methods (developer) + +## Limited-memory BFGS (L-BFGS) +[L-BFGS](http://en.wikipedia.org/wiki/Limited-memory_BFGS) is an optimization +algorithm in the family of quasi-Newton methods to solve the optimization problems of the form +`$\min_{\wv \in\R^d} \; f(\wv)$`. The L-BFGS method approximates the objective function locally as a +quadratic without evaluating the second partial derivatives of the objective function to construct the +Hessian matrix. The Hessian matrix is approximated by previous gradient evaluations, so there is no +vertical scalability issue (the number of training features) unlike computing the Hessian matrix +explicitly in Newton's method. As a result, L-BFGS often achieves faster convergence compared with +other first-order optimizations. -The optimization algorithm underlying the implementation is called [Orthant-Wise Limited-memory -QuasiNewton](http://research-srv.microsoft.com/en-us/um/people/jfgao/paper/icml07scalable.pdf) -(OWL-QN). It is an extension of L-BFGS that can effectively handle L1 -regularization and elastic net. +Quasi-Newton](http://research-srv.microsoft.com/en-us/um/people/jfgao/paper/icml07scalable.pdf) +(OWL-QN) is an extension of L-BFGS that can effectively handle L1 and elastic net regularization. + +L-BFGS is used as a solver for [LinearRegression](api/scala/index.html#org.apache.spark.ml.regression.LinearRegression), +[LogisticRegression](api/scala/index.html#org.apache.spark.ml.classification.LogisticRegression), +[AFTSurvivalRegression](api/scala/index.html#org.apache.spark.ml.regression.AFTSurvivalRegression) +and [MultilayerPerceptronClassifier](api/scala/index.html#org.apache.spark.ml.classification.MultilayerPerceptronClassifier). + +MLlib L-BFGS solver calls the corresponding implementation in [breeze](https://github.com/scalanlp/breeze/blob/master/math/src/main/scala/breeze/optimize/LBFGS.scala). + +## Normal equation solver for weighted least squares + +MLlib implements normal equation solver for [weighted least squares](https://en.wikipedia.org/wiki/Least_squares#Weighted_least_squares) by [WeightedLeastSquares](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala). + +Given $n$ weighted observations $(w_i, a_i, b_i)$: + +* $w_i$ the weight of i-th observation +* $a_i$ the features vector of i-th observation +* $b_i$ the label of i-th observation + +The number of features for each observation is $m$. We use the following weighted least squares formulation: +`\[ +minimize_{x}\frac{1}{2} \sum_{i=1}^n \frac{w_i(a_i^T x -b_i)^2}{\sum_{k=1}^n w_k} + \frac{1}{2}\frac{\lambda}{\delta}\sum_{j=1}^m(\sigma_{j} x_{j})^2 +\]` +where $\lambda$ is the regularization parameter, $\delta$ is the population standard deviation of the label +and $\sigma_j$ is the population standard deviation of the j-th feature column. + +This objective function has an analytic solution and it requires only one pass over the data to collect necessary statistics to solve. +Unlike the original dataset which can only be stored in a distributed system, +these statistics can be loaded into memory on a single machine if the number of features is relatively small, and then we can solve the objective function through Cholesky factorization on the driver. + +WeightedLeastSquares only supports L2 regularization and provides options to enable or disable regularization and standardization. +In order to make the normal equation approach efficient, WeightedLeastSquares requires that the number of features be no more than 4096. For larger problems, use L-BFGS instead. + +## Iteratively reweighted least squares (IRLS) + +MLlib implements [iteratively reweighted least squares (IRLS)](https://en.wikipedia.org/wiki/Iteratively_reweighted_least_squares) by [IterativelyReweightedLeastSquares](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala). +It can be used to find the maximum likelihood estimates of a generalized linear model (GLM), find M-estimator in robust regression and other optimization problems. +Refer to [Iteratively Reweighted Least Squares for Maximum Likelihood Estimation, and some Robust and Resistant Alternatives](http://www.jstor.org/stable/2345503) for more information. + +It solves certain optimization problems iteratively through the following procedure: + +* linearize the objective at current solution and update corresponding weight. +* solve a weighted least squares (WLS) problem by WeightedLeastSquares. +* repeat above steps until convergence. + +Since it involves solving a weighted least squares (WLS) problem by WeightedLeastSquares in each iteration, +it also requires the number of features to be no more than 4096. +Currently IRLS is used as the default solver of [GeneralizedLinearRegression](api/scala/index.html#org.apache.spark.ml.regression.GeneralizedLinearRegression). diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index 6ed193cf57d4f..d732f53029e8c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -38,7 +38,7 @@ private[ml] class IterativelyReweightedLeastSquaresModel( /** * Implements the method of iteratively reweighted least squares (IRLS) which is used to solve * certain optimization problems by an iterative method. In each step of the iterations, it - * involves solving a weighted lease squares (WLS) problem by [[WeightedLeastSquares]]. + * involves solving a weighted least squares (WLS) problem by [[WeightedLeastSquares]]. * It can be used to find maximum likelihood estimates of a generalized linear model (GLM), * find M-estimator in robust regression and other optimization problems. * From a778d3c90599eb76e6bca87b7aa3c0f9910f24c5 Mon Sep 17 00:00:00 2001 From: yinxusen Date: Fri, 27 May 2016 13:18:29 -0700 Subject: [PATCH 0428/1470] [SPARK-15008][ML][PYSPARK] Add integration test for OneVsRest ## What changes were proposed in this pull request? 1. Add `_transfer_param_map_to/from_java` for OneVsRest; 2. Add `_compare_params` in ml/tests.py to help compare params. 3. Add `test_onevsrest` as the integration test for OneVsRest. ## How was this patch tested? Python unit test. Author: yinxusen Closes #12875 from yinxusen/SPARK-15008. (cherry picked from commit 130b8d07b8eb08f2ad522081a95032b90247094d) Signed-off-by: Joseph K. Bradley --- python/pyspark/ml/tests.py | 69 +++++++++++++++++++++++++------------- 1 file changed, 46 insertions(+), 23 deletions(-) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index a7c93ac802726..4358175a57e63 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -747,12 +747,32 @@ def test_logistic_regression(self): except OSError: pass + def _compare_params(self, m1, m2, param): + """ + Compare 2 ML Params instances for the given param, and assert both have the same param value + and parent. The param must be a parameter of m1. + """ + # Prevent key not found error in case of some param in neither paramMap nor defaultParamMap. + if m1.isDefined(param): + paramValue1 = m1.getOrDefault(param) + paramValue2 = m2.getOrDefault(m2.getParam(param.name)) + if isinstance(paramValue1, Params): + self._compare_pipelines(paramValue1, paramValue2) + else: + self.assertEqual(paramValue1, paramValue2) # for general types param + # Assert parents are equal + self.assertEqual(param.parent, m2.getParam(param.name).parent) + else: + # If m1 is not defined param, then m2 should not, too. See SPARK-14931. + self.assertFalse(m2.isDefined(m2.getParam(param.name))) + def _compare_pipelines(self, m1, m2): """ Compare 2 ML types, asserting that they are equivalent. This currently supports: - basic types - Pipeline, PipelineModel + - OneVsRest, OneVsRestModel This checks: - uid - type @@ -763,8 +783,7 @@ def _compare_pipelines(self, m1, m2): if isinstance(m1, JavaParams): self.assertEqual(len(m1.params), len(m2.params)) for p in m1.params: - self.assertEqual(m1.getOrDefault(p), m2.getOrDefault(p)) - self.assertEqual(p.parent, m2.getParam(p.name).parent) + self._compare_params(m1, m2, p) elif isinstance(m1, Pipeline): self.assertEqual(len(m1.getStages()), len(m2.getStages())) for s1, s2 in zip(m1.getStages(), m2.getStages()): @@ -773,6 +792,13 @@ def _compare_pipelines(self, m1, m2): self.assertEqual(len(m1.stages), len(m2.stages)) for s1, s2 in zip(m1.stages, m2.stages): self._compare_pipelines(s1, s2) + elif isinstance(m1, OneVsRest) or isinstance(m1, OneVsRestModel): + for p in m1.params: + self._compare_params(m1, m2, p) + if isinstance(m1, OneVsRestModel): + self.assertEqual(len(m1.models), len(m2.models)) + for x, y in zip(m1.models, m2.models): + self._compare_pipelines(x, y) else: raise RuntimeError("_compare_pipelines does not yet support type: %s" % type(m1)) @@ -833,6 +859,24 @@ def test_nested_pipeline_persistence(self): except OSError: pass + def test_onevsrest(self): + temp_path = tempfile.mkdtemp() + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))] * 10, + ["label", "features"]) + lr = LogisticRegression(maxIter=5, regParam=0.01) + ovr = OneVsRest(classifier=lr) + model = ovr.fit(df) + ovrPath = temp_path + "/ovr" + ovr.save(ovrPath) + loadedOvr = OneVsRest.load(ovrPath) + self._compare_pipelines(ovr, loadedOvr) + modelPath = temp_path + "/ovrModel" + model.save(modelPath) + loadedModel = OneVsRestModel.load(modelPath) + self._compare_pipelines(model, loadedModel) + def test_decisiontree_classifier(self): dt = DecisionTreeClassifier(maxDepth=1) path = tempfile.mkdtemp() @@ -1054,27 +1098,6 @@ def test_output_columns(self): output = model.transform(df) self.assertEqual(output.columns, ["label", "features", "prediction"]) - def test_save_load(self): - temp_path = tempfile.mkdtemp() - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))], - ["label", "features"]) - lr = LogisticRegression(maxIter=5, regParam=0.01) - ovr = OneVsRest(classifier=lr) - model = ovr.fit(df) - ovrPath = temp_path + "/ovr" - ovr.save(ovrPath) - loadedOvr = OneVsRest.load(ovrPath) - self.assertEqual(loadedOvr.getFeaturesCol(), ovr.getFeaturesCol()) - self.assertEqual(loadedOvr.getLabelCol(), ovr.getLabelCol()) - self.assertEqual(loadedOvr.getClassifier().uid, ovr.getClassifier().uid) - modelPath = temp_path + "/ovrModel" - model.save(modelPath) - loadedModel = OneVsRestModel.load(modelPath) - for m, n in zip(model.models, loadedModel.models): - self.assertEqual(m.uid, n.uid) - class HashingTFTest(SparkSessionTestCase): From dcf498e8aafd2b53c5680cf7f3ada31829686b62 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Fri, 27 May 2016 14:02:39 -0700 Subject: [PATCH 0429/1470] [SPARK-15413][ML][MLLIB] Change `toBreeze` to `asBreeze` in Vector and Matrix ## What changes were proposed in this pull request? We're using `asML` to convert the mllib vector/matrix to ml vector/matrix now. Using `as` is more correct given that this conversion actually shares the same underline data structure. As a result, in this PR, `toBreeze` will be changed to `asBreeze`. This is a private API, as a result, it will not affect any user's application. ## How was this patch tested? unit tests Author: DB Tsai Closes #13198 from dbtsai/minor. (cherry picked from commit 21b2605dc4900894ea7a911e039781ecc2a18c14) Signed-off-by: Joseph K. Bradley --- .../org/apache/spark/ml/linalg/Matrices.scala | 16 ++++++------ .../org/apache/spark/ml/linalg/Vectors.scala | 8 +++--- .../distribution/MultivariateGaussian.scala | 8 +++--- .../linalg/BreezeMatrixConversionSuite.scala | 4 +-- .../linalg/BreezeVectorConversionSuite.scala | 4 +-- .../spark/ml/linalg/MatricesSuite.scala | 14 +++++----- .../apache/spark/ml/linalg/VectorsSuite.scala | 2 +- .../scala/org/apache/spark/ml/ann/Layer.scala | 8 +++--- .../classification/LogisticRegression.scala | 2 +- .../spark/ml/clustering/GaussianMixture.scala | 2 +- .../spark/ml/feature/MaxAbsScaler.scala | 2 +- .../spark/ml/feature/MinMaxScaler.scala | 2 +- .../ml/regression/AFTSurvivalRegression.scala | 2 +- .../ml/regression/LinearRegression.scala | 2 +- .../spark/mllib/classification/SVM.scala | 2 +- .../mllib/clustering/GaussianMixture.scala | 2 +- .../clustering/GaussianMixtureModel.scala | 4 +-- .../spark/mllib/clustering/LDAModel.scala | 26 +++++++++---------- .../spark/mllib/clustering/LDAOptimizer.scala | 6 ++--- .../mllib/clustering/StreamingKMeans.scala | 4 +-- .../apache/spark/mllib/linalg/Matrices.scala | 16 ++++++------ .../apache/spark/mllib/linalg/Vectors.scala | 8 +++--- .../linalg/distributed/BlockMatrix.scala | 8 +++--- .../mllib/linalg/distributed/RowMatrix.scala | 16 ++++++------ .../mllib/optimization/GradientDescent.scala | 4 +-- .../spark/mllib/optimization/LBFGS.scala | 4 +-- .../spark/mllib/optimization/Updater.scala | 14 +++++----- .../apache/spark/mllib/regression/Lasso.scala | 2 +- .../mllib/regression/LinearRegression.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 2 +- .../stat/correlation/PearsonCorrelation.scala | 2 +- .../distribution/MultivariateGaussian.scala | 8 +++--- .../spark/mllib/stat/test/ChiSqTest.scala | 2 +- .../ml/classification/NaiveBayesSuite.scala | 6 ++--- .../LogisticRegressionSuite.scala | 4 +-- .../classification/NaiveBayesSuite.scala | 4 +-- .../spark/mllib/clustering/LDASuite.scala | 4 +-- .../clustering/StreamingKMeansSuite.scala | 2 +- .../spark/mllib/feature/NormalizerSuite.scala | 16 ++++++------ .../linalg/BreezeMatrixConversionSuite.scala | 4 +-- .../linalg/BreezeVectorConversionSuite.scala | 4 +-- .../spark/mllib/linalg/MatricesSuite.scala | 14 +++++----- .../spark/mllib/linalg/VectorsSuite.scala | 2 +- .../linalg/distributed/BlockMatrixSuite.scala | 2 +- .../distributed/IndexedRowMatrixSuite.scala | 10 +++---- .../linalg/distributed/RowMatrixSuite.scala | 14 +++++----- .../spark/mllib/stat/CorrelationSuite.scala | 6 ++--- .../spark/mllib/util/MLUtilsSuite.scala | 6 ++--- project/MimaExcludes.scala | 3 +++ 49 files changed, 156 insertions(+), 153 deletions(-) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index a47526d36f1a1..0ea687bbccc54 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -69,7 +69,7 @@ sealed trait Matrix extends Serializable { def rowIter: Iterator[Vector] = this.transpose.colIter /** Converts to a breeze matrix. */ - private[ml] def toBreeze: BM[Double] + private[ml] def asBreeze: BM[Double] /** Gets the (i, j)-th element. */ @Since("2.0.0") @@ -112,11 +112,11 @@ sealed trait Matrix extends Serializable { } /** A human readable representation of the matrix */ - override def toString: String = toBreeze.toString() + override def toString: String = asBreeze.toString() /** A human readable representation of the matrix with maximum lines and width */ @Since("2.0.0") - def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) + def toString(maxLines: Int, maxLineWidth: Int): String = asBreeze.toString(maxLines, maxLineWidth) /** * Map the values of this matrix using a function. Generates a new matrix. Performs the @@ -202,7 +202,7 @@ class DenseMatrix @Since("2.0.0") ( this(numRows, numCols, values, false) override def equals(o: Any): Boolean = o match { - case m: Matrix => toBreeze == m.toBreeze + case m: Matrix => asBreeze == m.asBreeze case _ => false } @@ -210,7 +210,7 @@ class DenseMatrix @Since("2.0.0") ( Seq(numRows, numCols, toArray).## } - private[ml] def toBreeze: BM[Double] = { + private[ml] def asBreeze: BM[Double] = { if (!isTransposed) { new BDM[Double](numRows, numCols, values) } else { @@ -488,14 +488,14 @@ class SparseMatrix @Since("2.0.0") ( rowIndices: Array[Int], values: Array[Double]) = this(numRows, numCols, colPtrs, rowIndices, values, false) - override def hashCode(): Int = toBreeze.hashCode() + override def hashCode(): Int = asBreeze.hashCode() override def equals(o: Any): Boolean = o match { - case m: Matrix => toBreeze == m.toBreeze + case m: Matrix => asBreeze == m.asBreeze case _ => false } - private[ml] def toBreeze: BM[Double] = { + private[ml] def asBreeze: BM[Double] = { if (!isTransposed) { new BSM[Double](values, numRows, numCols, colPtrs, rowIndices) } else { diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index 59f9c2adba5fc..909fec1c06653 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -92,14 +92,14 @@ sealed trait Vector extends Serializable { /** * Converts the instance to a breeze vector. */ - private[spark] def toBreeze: BV[Double] + private[spark] def asBreeze: BV[Double] /** * Gets the value of the ith element. * @param i index */ @Since("2.0.0") - def apply(i: Int): Double = toBreeze(i) + def apply(i: Int): Double = asBreeze(i) /** * Makes a deep copy of this vector. @@ -453,7 +453,7 @@ class DenseVector @Since("2.0.0") ( @Since("2.0.0") val values: Array[Double]) e override def toArray: Array[Double] = values - private[spark] override def toBreeze: BV[Double] = new BDV[Double](values) + private[spark] override def asBreeze: BV[Double] = new BDV[Double](values) override def apply(i: Int): Double = values(i) @@ -584,7 +584,7 @@ class SparseVector @Since("2.0.0") ( new SparseVector(size, indices.clone(), values.clone()) } - private[spark] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) + private[spark] override def asBreeze: BV[Double] = new BSV[Double](indices, values, size) override def foreachActive(f: (Int, Double) => Unit): Unit = { var i = 0 diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala index 383d6d96e8d7c..0be28677eff31 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala @@ -47,7 +47,7 @@ class MultivariateGaussian @Since("2.0.0") ( this(Vectors.fromBreeze(mean), Matrices.fromBreeze(cov)) } - private val breezeMu = mean.toBreeze.toDenseVector + private val breezeMu = mean.asBreeze.toDenseVector /** * Compute distribution dependent constants: @@ -61,7 +61,7 @@ class MultivariateGaussian @Since("2.0.0") ( */ @Since("2.0.0") def pdf(x: Vector): Double = { - pdf(x.toBreeze) + pdf(x.asBreeze) } /** @@ -69,7 +69,7 @@ class MultivariateGaussian @Since("2.0.0") ( */ @Since("2.0.0") def logpdf(x: Vector): Double = { - logpdf(x.toBreeze) + logpdf(x.asBreeze) } /** Returns density of this multivariate Gaussian at given point, x */ @@ -113,7 +113,7 @@ class MultivariateGaussian @Since("2.0.0") ( * relation to the maximum singular value (same tolerance used by, e.g., Octave). */ private def calculateCovarianceConstants: (BDM[Double], Double) = { - val eigSym.EigSym(d, u) = eigSym(cov.toBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t + val eigSym.EigSym(d, u) = eigSym(cov.asBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t // For numerical stability, values are considered to be non-zero only if they exceed tol. // This prevents any inverted value from exceeding (eps * n * max(d))^-1 diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeMatrixConversionSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeMatrixConversionSuite.scala index 70a21e41bfbd1..f07ed20cf0e77 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeMatrixConversionSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeMatrixConversionSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.SparkMLFunSuite class BreezeMatrixConversionSuite extends SparkMLFunSuite { test("dense matrix to breeze") { val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) - val breeze = mat.toBreeze.asInstanceOf[BDM[Double]] + val breeze = mat.asBreeze.asInstanceOf[BDM[Double]] assert(breeze.rows === mat.numRows) assert(breeze.cols === mat.numCols) assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") @@ -48,7 +48,7 @@ class BreezeMatrixConversionSuite extends SparkMLFunSuite { val colPtrs = Array(0, 2, 4) val rowIndices = Array(1, 2, 1, 2) val mat = Matrices.sparse(3, 2, colPtrs, rowIndices, values) - val breeze = mat.toBreeze.asInstanceOf[BSM[Double]] + val breeze = mat.asBreeze.asInstanceOf[BSM[Double]] assert(breeze.rows === mat.numRows) assert(breeze.cols === mat.numCols) assert(breeze.data.eq(mat.asInstanceOf[SparseMatrix].values), "should not copy data") diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeVectorConversionSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeVectorConversionSuite.scala index 00c9ee79eb175..4c9740b6bca76 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeVectorConversionSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeVectorConversionSuite.scala @@ -33,12 +33,12 @@ class BreezeVectorConversionSuite extends SparkMLFunSuite { test("dense to breeze") { val vec = Vectors.dense(arr) - assert(vec.toBreeze === new BDV[Double](arr)) + assert(vec.asBreeze === new BDV[Double](arr)) } test("sparse to breeze") { val vec = Vectors.sparse(n, indices, values) - assert(vec.toBreeze === new BSV[Double](indices, values, n)) + assert(vec.asBreeze === new BSV[Double](indices, values, n)) } test("dense breeze to vector") { diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala index 5c69c5ed7bdd2..2796fcf2cbc22 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala @@ -61,7 +61,7 @@ class MatricesSuite extends SparkMLFunSuite { (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0)) val mat2 = SparseMatrix.fromCOO(m, n, entries) - assert(mat.toBreeze === mat2.toBreeze) + assert(mat.asBreeze === mat2.asBreeze) assert(mat2.values.length == 4) } @@ -174,8 +174,8 @@ class MatricesSuite extends SparkMLFunSuite { val spMat2 = deMat1.toSparse val deMat2 = spMat1.toDense - assert(spMat1.toBreeze === spMat2.toBreeze) - assert(deMat1.toBreeze === deMat2.toBreeze) + assert(spMat1.asBreeze === spMat2.asBreeze) + assert(deMat1.asBreeze === deMat2.asBreeze) } test("map, update") { @@ -209,8 +209,8 @@ class MatricesSuite extends SparkMLFunSuite { val sATexpected = new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) - assert(dAT.toBreeze === dATexpected.toBreeze) - assert(sAT.toBreeze === sATexpected.toBreeze) + assert(dAT.asBreeze === dATexpected.asBreeze) + assert(sAT.asBreeze === sATexpected.asBreeze) assert(dA(1, 0) === dAT(0, 1)) assert(dA(2, 1) === dAT(1, 2)) assert(sA(1, 0) === sAT(0, 1)) @@ -219,8 +219,8 @@ class MatricesSuite extends SparkMLFunSuite { assert(!dA.toArray.eq(dAT.toArray), "has to have a new array") assert(dA.values.eq(dAT.transpose.asInstanceOf[DenseMatrix].values), "should not copy array") - assert(dAT.toSparse.toBreeze === sATexpected.toBreeze) - assert(sAT.toDense.toBreeze === dATexpected.toBreeze) + assert(dAT.toSparse.asBreeze === sATexpected.asBreeze) + assert(sAT.toDense.asBreeze === dATexpected.asBreeze) } test("foreachActive") { diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala index 887814b5e731a..614be460a414a 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala @@ -230,7 +230,7 @@ class VectorsSuite extends SparkMLFunSuite { val denseVector1 = Vectors.dense(sparseVector1.toArray) val denseVector2 = Vectors.dense(sparseVector2.toArray) - val squaredDist = breezeSquaredDistance(sparseVector1.toBreeze, sparseVector2.toBreeze) + val squaredDist = breezeSquaredDistance(sparseVector1.asBreeze, sparseVector2.asBreeze) // SparseVector vs. SparseVector assert(Vectors.sqdist(sparseVector1, sparseVector2) ~== squaredDist relTol 1E-8) diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 0a569c49179a6..576584c62797d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -616,8 +616,8 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) data.map { v => (0.0, Vectors.fromBreeze(BDV.vertcat( - v._1.toBreeze.toDenseVector, - v._2.toBreeze.toDenseVector)) + v._1.asBreeze.toDenseVector, + v._2.asBreeze.toDenseVector)) ) } } else { data.mapPartitions { it => @@ -665,8 +665,8 @@ private[ann] class ANNUpdater extends Updater { iter: Int, regParam: Double): (OldVector, Double) = { val thisIterStepSize = stepSize - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector - Baxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector + Baxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) (OldVectors.fromBreeze(brzWeights), 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 61e355ab9fba1..cecf3c43dfad4 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 @@ -400,7 +400,7 @@ class LogisticRegression @Since("1.2.0") ( } val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialCoefficientsWithIntercept.toBreeze.toDenseVector) + initialCoefficientsWithIntercept.asBreeze.toDenseVector) /* Note that in Logistic Regression, the objective history (loss + regularization) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 773e50e24549c..563a3b14e9a6f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -113,7 +113,7 @@ class GaussianMixtureModel private[ml] ( private[clustering] def predictProbability(features: Vector): Vector = { val probs: Array[Double] = - GaussianMixtureModel.computeProbabilities(features.toBreeze.toDenseVector, gaussians, weights) + GaussianMixtureModel.computeProbabilities(features.asBreeze.toDenseVector, gaussians, weights) Vectors.dense(probs) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index 0dffba93ac571..1b5159902eeb5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -122,7 +122,7 @@ class MaxAbsScalerModel private[ml] ( // TODO: this looks hack, we may have to handle sparse and dense vectors separately. val maxAbsUnzero = Vectors.dense(maxAbs.toArray.map(x => if (x == 0) 1 else x)) val reScale = udf { (vector: Vector) => - val brz = vector.toBreeze / maxAbsUnzero.toBreeze + val brz = vector.asBreeze / maxAbsUnzero.asBreeze Vectors.fromBreeze(brz) } dataset.withColumn($(outputCol), reScale(col($(inputCol)))) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index c6ff639f29620..d15f1b8563b24 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -162,7 +162,7 @@ class MinMaxScalerModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - val originalRange = (originalMax.toBreeze - originalMin.toBreeze).toArray + val originalRange = (originalMax.asBreeze - originalMin.asBreeze).toArray val minArray = originalMin.toArray val reScale = udf { (vector: Vector) => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 00ef6ccc74d28..c4400738426b5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -223,7 +223,7 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S val initialParameters = Vectors.zeros(numFeatures + 2) val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialParameters.toBreeze.toDenseVector) + initialParameters.asBreeze.toDenseVector) val parameters = { val arrayBuilder = mutable.ArrayBuilder.make[Double] 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 4d66b0eb37ab4..6be2584785bd3 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 @@ -297,7 +297,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val initialCoefficients = Vectors.zeros(numFeatures) val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialCoefficients.toBreeze.toDenseVector) + initialCoefficients.asBreeze.toDenseVector) val (coefficients, objectiveHistory) = { /* diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index a8d3fd4177a23..783c1c835b3b7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -72,7 +72,7 @@ class SVMModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + val margin = weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept threshold match { case Some(t) => if (margin > t) 1.0 else 0.0 case None => margin diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index f04c87259c941..a214b1a26f443 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -166,7 +166,7 @@ class GaussianMixture private ( val sc = data.sparkContext // we will operate on the data as breeze data - val breezeData = data.map(_.toBreeze).cache() + val breezeData = data.map(_.asBreeze).cache() // Get length of the input vectors val d = breezeData.first().length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index f470b0f3c3690..31ad56dba6aef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -96,7 +96,7 @@ class GaussianMixtureModel @Since("1.3.0") ( val bcDists = sc.broadcast(gaussians) val bcWeights = sc.broadcast(weights) points.map { x => - computeSoftAssignments(x.toBreeze.toDenseVector, bcDists.value, bcWeights.value, k) + computeSoftAssignments(x.asBreeze.toDenseVector, bcDists.value, bcWeights.value, k) } } @@ -105,7 +105,7 @@ class GaussianMixtureModel @Since("1.3.0") ( */ @Since("1.4.0") def predictSoft(point: Vector): Array[Double] = { - computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) + computeSoftAssignments(point.asBreeze.toDenseVector, gaussians, weights, k) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 0a515f893dc0b..1b66013d543ad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -205,7 +205,7 @@ class LocalLDAModel private[spark] ( @Since("1.3.0") override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { - val brzTopics = topics.toBreeze.toDenseMatrix + val brzTopics = topics.asBreeze.toDenseMatrix Range(0, k).map { topicIndex => val topic = normalize(brzTopics(::, topicIndex), 1.0) val (termWeights, terms) = @@ -233,7 +233,7 @@ class LocalLDAModel private[spark] ( */ @Since("1.5.0") def logLikelihood(documents: RDD[(Long, Vector)]): Double = logLikelihoodBound(documents, - docConcentration, topicConcentration, topicsMatrix.toBreeze.toDenseMatrix, gammaShape, k, + docConcentration, topicConcentration, topicsMatrix.asBreeze.toDenseMatrix, gammaShape, k, vocabSize) /** @@ -291,7 +291,7 @@ class LocalLDAModel private[spark] ( gammaShape: Double, k: Int, vocabSize: Long): Double = { - val brzAlpha = alpha.toBreeze.toDenseVector + val brzAlpha = alpha.asBreeze.toDenseVector // transpose because dirichletExpectation normalizes by row and we need to normalize // by topic (columns of lambda) val Elogbeta = LDAUtils.dirichletExpectation(lambda.t).t @@ -344,9 +344,9 @@ class LocalLDAModel private[spark] ( def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = { // Double transpose because dirichletExpectation normalizes by row and we need to normalize // by topic (columns of lambda) - val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.toBreeze.toDenseMatrix.t).t) + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) val expElogbetaBc = documents.sparkContext.broadcast(expElogbeta) - val docConcentrationBrz = this.docConcentration.toBreeze + val docConcentrationBrz = this.docConcentration.asBreeze val gammaShape = this.gammaShape val k = this.k @@ -367,9 +367,9 @@ class LocalLDAModel private[spark] ( /** Get a method usable as a UDF for [[topicDistributions()]] */ private[spark] def getTopicDistributionMethod(sc: SparkContext): Vector => Vector = { - val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.toBreeze.toDenseMatrix.t).t) + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) val expElogbetaBc = sc.broadcast(expElogbeta) - val docConcentrationBrz = this.docConcentration.toBreeze + val docConcentrationBrz = this.docConcentration.asBreeze val gammaShape = this.gammaShape val k = this.k @@ -399,14 +399,14 @@ class LocalLDAModel private[spark] ( */ @Since("2.0.0") def topicDistribution(document: Vector): Vector = { - val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.toBreeze.toDenseMatrix.t).t) + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) if (document.numNonzeros == 0) { Vectors.zeros(this.k) } else { val (gamma, _, _) = OnlineLDAOptimizer.variationalTopicInference( document, expElogbeta, - this.docConcentration.toBreeze, + this.docConcentration.asBreeze, gammaShape, this.k) Vectors.dense(normalize(gamma, 1.0).toArray) @@ -456,7 +456,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { ("gammaShape" -> gammaShape))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) - val topicsDenseMatrix = topicsMatrix.toBreeze.toDenseMatrix + val topicsDenseMatrix = topicsMatrix.asBreeze.toDenseMatrix val topics = Range(0, k).map { topicInd => Data(Vectors.dense((topicsDenseMatrix(::, topicInd).toArray)), topicInd) } @@ -480,7 +480,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { val brzTopics = BDM.zeros[Double](vocabSize, k) topics.foreach { case Row(vec: Vector, ind: Int) => - brzTopics(::, ind) := vec.toBreeze + brzTopics(::, ind) := vec.asBreeze } val topicsMat = Matrices.fromBreeze(brzTopics) @@ -896,9 +896,9 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { Loader.checkSchema[VertexData](vertexDataFrame.schema) Loader.checkSchema[EdgeData](edgeDataFrame.schema) val globalTopicTotals: LDA.TopicCounts = - dataFrame.first().getAs[Vector](0).toBreeze.toDenseVector + dataFrame.first().getAs[Vector](0).asBreeze.toDenseVector val vertices: RDD[(VertexId, LDA.TopicCounts)] = vertexDataFrame.rdd.map { - case Row(ind: Long, vec: Vector) => (ind, vec.toBreeze.toDenseVector) + case Row(ind: Long, vec: Vector) => (ind, vec.asBreeze.toDenseVector) } val edges: RDD[Edge[LDA.TokenCount]] = edgeDataFrame.rdd.map { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index 1b3e2f600d028..2436efba32489 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -137,7 +137,7 @@ final class EMLDAOptimizer extends LDAOptimizer { // For each document, create an edge (Document -> Term) for each unique term in the document. val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => // Add edges for terms with non-zero counts. - termCounts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + termCounts.asBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => Edge(docID, term2index(term), cnt) } } @@ -457,7 +457,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { val vocabSize = this.vocabSize val expElogbeta = exp(LDAUtils.dirichletExpectation(lambda)).t val expElogbetaBc = batch.sparkContext.broadcast(expElogbeta) - val alpha = this.alpha.toBreeze + val alpha = this.alpha.asBreeze val gammaShape = this.gammaShape val stats: RDD[(BDM[Double], List[BDV[Double]])] = batch.mapPartitions { docs => @@ -507,7 +507,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { private def updateAlpha(gammat: BDM[Double]): Unit = { val weight = rho() val N = gammat.rows.toDouble - val alpha = this.alpha.toBreeze.toDenseVector + val alpha = this.alpha.asBreeze.toDenseVector val logphat: BDM[Double] = sum(LDAUtils.dirichletExpectation(gammat)(::, breeze.linalg.*)) / N val gradf = N * (-LDAUtils.dirichletExpectation(alpha) + logphat.toDenseVector) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index 24e1cff0dcc6b..52bdccb919a61 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -135,8 +135,8 @@ class StreamingKMeansModel @Since("1.2.0") ( while (j < dim) { val x = largestClusterCenter(j) val p = 1e-14 * math.max(math.abs(x), 1.0) - largestClusterCenter.toBreeze(j) = x + p - smallestClusterCenter.toBreeze(j) = x - p + largestClusterCenter.asBreeze(j) = x + p + smallestClusterCenter.asBreeze(j) = x - p j += 1 } } 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 ee1956c2d4135..e8f34388cd9fe 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 @@ -75,7 +75,7 @@ sealed trait Matrix extends Serializable { def rowIter: Iterator[Vector] = this.transpose.colIter /** Converts to a breeze matrix. */ - private[mllib] def toBreeze: BM[Double] + private[mllib] def asBreeze: BM[Double] /** Gets the (i, j)-th element. */ @Since("1.3.0") @@ -118,11 +118,11 @@ sealed trait Matrix extends Serializable { } /** A human readable representation of the matrix */ - override def toString: String = toBreeze.toString() + override def toString: String = asBreeze.toString() /** A human readable representation of the matrix with maximum lines and width */ @Since("1.4.0") - def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) + def toString(maxLines: Int, maxLineWidth: Int): String = asBreeze.toString(maxLines, maxLineWidth) /** * Map the values of this matrix using a function. Generates a new matrix. Performs the @@ -300,7 +300,7 @@ class DenseMatrix @Since("1.3.0") ( this(numRows, numCols, values, false) override def equals(o: Any): Boolean = o match { - case m: Matrix => toBreeze == m.toBreeze + case m: Matrix => asBreeze == m.asBreeze case _ => false } @@ -308,7 +308,7 @@ class DenseMatrix @Since("1.3.0") ( com.google.common.base.Objects.hashCode(numRows: Integer, numCols: Integer, toArray) } - private[mllib] def toBreeze: BM[Double] = { + private[mllib] def asBreeze: BM[Double] = { if (!isTransposed) { new BDM[Double](numRows, numCols, values) } else { @@ -607,13 +607,13 @@ class SparseMatrix @Since("1.3.0") ( values: Array[Double]) = this(numRows, numCols, colPtrs, rowIndices, values, false) override def equals(o: Any): Boolean = o match { - case m: Matrix => toBreeze == m.toBreeze + case m: Matrix => asBreeze == m.asBreeze case _ => false } - override def hashCode(): Int = toBreeze.hashCode + override def hashCode(): Int = asBreeze.hashCode - private[mllib] def toBreeze: BM[Double] = { + private[mllib] def asBreeze: BM[Double] = { if (!isTransposed) { new BSM[Double](values, numRows, numCols, colPtrs, rowIndices) } else { 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 7ebcd297bd549..02fd60da7d590 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 @@ -103,14 +103,14 @@ sealed trait Vector extends Serializable { /** * Converts the instance to a breeze vector. */ - private[spark] def toBreeze: BV[Double] + private[spark] def asBreeze: BV[Double] /** * Gets the value of the ith element. * @param i index */ @Since("1.1.0") - def apply(i: Int): Double = toBreeze(i) + def apply(i: Int): Double = asBreeze(i) /** * Makes a deep copy of this vector. @@ -610,7 +610,7 @@ class DenseVector @Since("1.0.0") ( @Since("1.0.0") override def toArray: Array[Double] = values - private[spark] override def toBreeze: BV[Double] = new BDV[Double](values) + private[spark] override def asBreeze: BV[Double] = new BDV[Double](values) @Since("1.0.0") override def apply(i: Int): Double = values(i) @@ -770,7 +770,7 @@ class SparseVector @Since("1.0.0") ( new SparseVector(size, indices.clone(), values.clone()) } - private[spark] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) + private[spark] override def asBreeze: BV[Double] = new BSV[Double](indices, values, size) @Since("1.6.0") override def foreachActive(f: (Int, Double) => Unit): Unit = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 580d7a98fb362..7a24617781ece 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -275,7 +275,7 @@ class BlockMatrix @Since("1.3.0") ( val rows = blocks.flatMap { case ((blockRowIdx, blockColIdx), mat) => mat.rowIter.zipWithIndex.map { case (vector, rowIdx) => - blockRowIdx * rowsPerBlock + rowIdx -> (blockColIdx, vector.toBreeze) + blockRowIdx * rowsPerBlock + rowIdx -> (blockColIdx, vector.asBreeze) } }.groupByKey().map { case (rowIdx, vectors) => val numberNonZeroPerRow = vectors.map(_._2.activeSize).sum.toDouble / cols.toDouble @@ -367,12 +367,12 @@ class BlockMatrix @Since("1.3.0") ( } if (a.isEmpty) { val zeroBlock = BM.zeros[Double](b.head.numRows, b.head.numCols) - val result = binMap(zeroBlock, b.head.toBreeze) + val result = binMap(zeroBlock, b.head.asBreeze) new MatrixBlock((blockRowIndex, blockColIndex), Matrices.fromBreeze(result)) } else if (b.isEmpty) { new MatrixBlock((blockRowIndex, blockColIndex), a.head) } else { - val result = binMap(a.head.toBreeze, b.head.toBreeze) + val result = binMap(a.head.asBreeze, b.head.asBreeze) new MatrixBlock((blockRowIndex, blockColIndex), Matrices.fromBreeze(result)) } } @@ -479,7 +479,7 @@ class BlockMatrix @Since("1.3.0") ( case _ => throw new SparkException(s"Unrecognized matrix type ${rightBlock.getClass}.") } - ((leftRowIndex, rightColIndex), C.toBreeze) + ((leftRowIndex, rightColIndex), C.asBreeze) } } }.reduceByKey(resultPartitioner, (a, b) => a + b).mapValues(Matrices.fromBreeze) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 4b8ed301eb3cb..cd5209d0ebe20 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -92,7 +92,7 @@ class RowMatrix @Since("1.0.0") ( val vbr = rows.context.broadcast(v) rows.treeAggregate(BDV.zeros[Double](n))( seqOp = (U, r) => { - val rBrz = r.toBreeze + val rBrz = r.asBreeze val a = rBrz.dot(vbr.value) rBrz match { // use specialized axpy for better performance @@ -250,12 +250,12 @@ class RowMatrix @Since("1.0.0") ( val (sigmaSquares: BDV[Double], u: BDM[Double]) = computeMode match { case SVDMode.LocalARPACK => require(k < n, s"k must be smaller than n in local-eigs mode but got k=$k and n=$n.") - val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + val G = computeGramianMatrix().asBreeze.asInstanceOf[BDM[Double]] EigenValueDecomposition.symmetricEigs(v => G * v, n, k, tol, maxIter) case SVDMode.LocalLAPACK => // breeze (v0.10) svd latent constraint, 7 * n * n + 4 * n < Int.MaxValue require(n < 17515, s"$n exceeds the breeze svd capability") - val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + val G = computeGramianMatrix().asBreeze.asInstanceOf[BDM[Double]] val brzSvd.SVD(uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G) (sigmaSquaresFull, uFull) case SVDMode.DistARPACK => @@ -338,7 +338,7 @@ class RowMatrix @Since("1.0.0") ( // large but Cov(X, Y) is small, but it is good for sparse computation. // TODO: find a fast and stable way for sparse data. - val G = computeGramianMatrix().toBreeze + val G = computeGramianMatrix().asBreeze var i = 0 var j = 0 @@ -381,7 +381,7 @@ class RowMatrix @Since("1.0.0") ( val n = numCols().toInt require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") - val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] + val Cov = computeCovariance().asBreeze.asInstanceOf[BDM[Double]] val brzSvd.SVD(u: BDM[Double], s: BDV[Double], _) = brzSvd(Cov) @@ -436,14 +436,14 @@ class RowMatrix @Since("1.0.0") ( require(B.isInstanceOf[DenseMatrix], s"Only support dense matrix at this time but found ${B.getClass.getName}.") - val Bb = rows.context.broadcast(B.toBreeze.asInstanceOf[BDM[Double]].toDenseVector.toArray) + val Bb = rows.context.broadcast(B.asBreeze.asInstanceOf[BDM[Double]].toDenseVector.toArray) val AB = rows.mapPartitions { iter => val Bi = Bb.value iter.map { row => val v = BDV.zeros[Double](k) var i = 0 while (i < k) { - v(i) = row.toBreeze.dot(new BDV(Bi, i * n, 1, n)) + v(i) = row.asBreeze.dot(new BDV(Bi, i * n, 1, n)) i += 1 } Vectors.fromBreeze(v) @@ -541,7 +541,7 @@ class RowMatrix @Since("1.0.0") ( val bdm = BDM.zeros[Double](partRows.length, col) var i = 0 partRows.foreach { row => - bdm(i, ::) := row.toBreeze.t + bdm(i, ::) := row.asBreeze.t i += 1 } breeze.linalg.qr.reduced(bdm).r diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 735e780909ef2..480a64548cb70 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -301,8 +301,8 @@ object GradientDescent extends Logging { currentWeights: Vector, convergenceTol: Double): Boolean = { // To compare with convergence tolerance. - val previousBDV = previousWeights.toBreeze.toDenseVector - val currentBDV = currentWeights.toBreeze.toDenseVector + val previousBDV = previousWeights.asBreeze.toDenseVector + val currentBDV = currentWeights.asBreeze.toDenseVector // This represents the difference of updated weights in the iteration. val solutionVecDiff: Double = norm(previousBDV - currentBDV) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 74e2cad76c8f5..ec6ffe6e19439 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -200,7 +200,7 @@ object LBFGS extends Logging { val lbfgs = new BreezeLBFGS[BDV[Double]](maxNumIterations, numCorrections, convergenceTol) val states = - lbfgs.iterations(new CachedDiffFunction(costFun), initialWeights.toBreeze.toDenseVector) + lbfgs.iterations(new CachedDiffFunction(costFun), initialWeights.asBreeze.toDenseVector) /** * NOTE: lossSum and loss is computed using the weights from the previous iteration @@ -281,7 +281,7 @@ object LBFGS extends Logging { // gradientTotal = gradientSum / numExamples + gradientTotal axpy(1.0 / numExamples, gradientSum, gradientTotal) - (loss, gradientTotal.toBreeze.asInstanceOf[BDV[Double]]) + (loss, gradientTotal.asBreeze.asInstanceOf[BDV[Double]]) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 03c01e0553d78..67d484575db52 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -75,8 +75,8 @@ class SimpleUpdater extends Updater { iter: Int, regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector - brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) (Vectors.fromBreeze(brzWeights), 0) } @@ -87,7 +87,7 @@ class SimpleUpdater extends Updater { * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. - + * * Instead of subgradient of the regularizer, the proximal operator for the * L1 regularization is applied after the gradient step. This is known to * result in better sparsity of the intermediate solution. @@ -111,8 +111,8 @@ class L1Updater extends Updater { regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) // Take gradient step - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector - brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) // Apply proximal operator (soft thresholding) val shrinkageVal = regParam * thisIterStepSize var i = 0 @@ -146,9 +146,9 @@ class SquaredL2Updater extends Updater { // w' = w - thisIterStepSize * (gradient + regParam * w) // w' = (1 - thisIterStepSize * regParam) * w - thisIterStepSize * gradient val thisIterStepSize = stepSize / math.sqrt(iter) - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector brzWeights :*= (1.0 - thisIterStepSize * regParam) - brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + brzAxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) val norm = brzNorm(brzWeights, 2.0) (Vectors.fromBreeze(brzWeights), 0.5 * regParam * norm * norm) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index e14bddf97d0f0..cef1b4f51b843 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -44,7 +44,7 @@ class LassoModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double = { - weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept } @Since("1.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 2ceac4b8cc319..60262fdc497a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -44,7 +44,7 @@ class LinearRegressionModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double = { - weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept } @Since("1.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 301f02fd98155..52977ac4f062a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -45,7 +45,7 @@ class RidgeRegressionModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double = { - weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept } @Since("1.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala index 515be0b817835..e478c31bc9a05 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala @@ -55,7 +55,7 @@ private[stat] object PearsonCorrelation extends Correlation with Logging { * 0 variance results in a correlation value of Double.NaN. */ def computeCorrelationMatrixFromCovariance(covarianceMatrix: Matrix): Matrix = { - val cov = covarianceMatrix.toBreeze.asInstanceOf[BDM[Double]] + val cov = covarianceMatrix.asBreeze.asInstanceOf[BDM[Double]] val n = cov.cols // Compute the standard deviation on the diagonals first diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index c806d6bdf6b63..39c3644450d6d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -42,7 +42,7 @@ class MultivariateGaussian @Since("1.3.0") ( require(sigma.numCols == sigma.numRows, "Covariance matrix must be square") require(mu.size == sigma.numCols, "Mean vector length must match covariance matrix size") - private val breezeMu = mu.toBreeze.toDenseVector + private val breezeMu = mu.asBreeze.toDenseVector /** * private[mllib] constructor @@ -66,7 +66,7 @@ class MultivariateGaussian @Since("1.3.0") ( */ @Since("1.3.0") def pdf(x: Vector): Double = { - pdf(x.toBreeze) + pdf(x.asBreeze) } /** @@ -74,7 +74,7 @@ class MultivariateGaussian @Since("1.3.0") ( */ @Since("1.3.0") def logpdf(x: Vector): Double = { - logpdf(x.toBreeze) + logpdf(x.asBreeze) } /** Returns density of this multivariate Gaussian at given point, x */ @@ -118,7 +118,7 @@ class MultivariateGaussian @Since("1.3.0") ( * relation to the maximum singular value (same tolerance used by, e.g., Octave). */ private def calculateCovarianceConstants: (DBM[Double], Double) = { - val eigSym.EigSym(d, u) = eigSym(sigma.toBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t + val eigSym.EigSym(d, u) = eigSym(sigma.asBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t // For numerical stability, values are considered to be non-zero only if they exceed tol. // This prevents any inverted value from exceeding (eps * n * max(d))^-1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 76ca6a8abd032..da5df9bf45e5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -110,7 +110,7 @@ private[stat] object ChiSqTest extends Logging { } i += 1 distinctLabels += label - val brzFeatures = features.toBreeze + val brzFeatures = features.asBreeze (startCol until endCol).map { col => val feature = brzFeatures(col) allDistinctFeatures(col) += feature diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index ff52115ec0ec1..04c010bd13e1e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -69,7 +69,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa } def expectedMultinomialProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { - val logClassProbs: BV[Double] = model.pi.toBreeze + model.theta.multiply(feature).toBreeze + val logClassProbs: BV[Double] = model.pi.asBreeze + model.theta.multiply(feature).asBreeze val classProbs = logClassProbs.toArray.map(math.exp) val classProbsSum = classProbs.sum Vectors.dense(classProbs.map(_ / classProbsSum)) @@ -78,8 +78,8 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with Defa def expectedBernoulliProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { val negThetaMatrix = model.theta.map(v => math.log(1.0 - math.exp(v))) val negFeature = Vectors.dense(feature.toArray.map(v => 1.0 - v)) - val piTheta: BV[Double] = model.pi.toBreeze + model.theta.multiply(feature).toBreeze - val logClassProbs: BV[Double] = piTheta + negThetaMatrix.multiply(negFeature).toBreeze + val piTheta: BV[Double] = model.pi.asBreeze + model.theta.multiply(feature).asBreeze + val logClassProbs: BV[Double] = piTheta + negThetaMatrix.multiply(negFeature).asBreeze val classProbs = logClassProbs.toArray.map(math.exp) val classProbsSum = classProbs.sum Vectors.dense(classProbs.map(_ / classProbsSum)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 28fada7053d65..5cf4377768516 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -411,10 +411,10 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w val testRDD1 = sc.parallelize(testData, 2) val testRDD2 = sc.parallelize( - testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E3))), 2) + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.asBreeze * 1.0E3))), 2) val testRDD3 = sc.parallelize( - testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E6))), 2) + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.asBreeze * 1.0E6))), 2) testRDD1.cache() testRDD2.cache() 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 ab54cb06d5aab..0c0aefc52b9bf 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 @@ -182,7 +182,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val piVector = new BDV(model.pi) // model.theta is row-major; treat it as col-major representation of transpose, and transpose: val thetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten).t - val logClassProbs: BV[Double] = piVector + (thetaMatrix * testData.toBreeze) + val logClassProbs: BV[Double] = piVector + (thetaMatrix * testData.asBreeze) val classProbs = logClassProbs.toArray.map(math.exp) val classProbsSum = classProbs.sum classProbs.map(_ / classProbsSum) @@ -234,7 +234,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val thetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten).t val negThetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten.map(v => math.log(1.0 - math.exp(v)))).t - val testBreeze = testData.toBreeze + val testBreeze = testData.asBreeze val negTestBreeze = new BDV(Array.fill(testBreeze.size)(1.0)) - testBreeze val piTheta: BV[Double] = piVector + (thetaMatrix * testBreeze) val logClassProbs: BV[Double] = piTheta + (negThetaMatrix * negTestBreeze) 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 ea23196d2c801..eb050158d48fe 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 @@ -116,7 +116,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { case (docId, (topicDistribution, (indices, weights))) => assert(indices.length == 2) assert(weights.length == 2) - val bdvTopicDist = topicDistribution.toBreeze + val bdvTopicDist = topicDistribution.asBreeze val top2Indices = argtopk(bdvTopicDist, 2) assert(top2Indices.toArray === indices) assert(bdvTopicDist(top2Indices).toArray === weights) @@ -369,7 +369,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { val actualPredictions = ldaModel.topicDistributions(docs).cache() val topTopics = actualPredictions.map { case (id, topics) => // convert results to expectedPredictions format, which only has highest probability topic - val topicsBz = topics.toBreeze.toDenseVector + val topicsBz = topics.asBreeze.toDenseVector (id, (argmax(topicsBz), max(topicsBz))) }.sortByKey() .values 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 65e37c64d404e..fdaa098345d13 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 @@ -67,7 +67,7 @@ class StreamingKMeansSuite extends SparkFunSuite 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.asBreeze).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/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala index 34122d6ed2e95..10f7bafd6cf5b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala @@ -51,10 +51,10 @@ class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(brzNorm(data1(0).toBreeze, 1) ~== 1.0 absTol 1E-5) - assert(brzNorm(data1(2).toBreeze, 1) ~== 1.0 absTol 1E-5) - assert(brzNorm(data1(3).toBreeze, 1) ~== 1.0 absTol 1E-5) - assert(brzNorm(data1(4).toBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(0).asBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(2).asBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(3).asBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(4).asBreeze, 1) ~== 1.0 absTol 1E-5) assert(data1(0) ~== Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))) absTol 1E-5) assert(data1(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) @@ -78,10 +78,10 @@ class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(brzNorm(data2(0).toBreeze, 2) ~== 1.0 absTol 1E-5) - assert(brzNorm(data2(2).toBreeze, 2) ~== 1.0 absTol 1E-5) - assert(brzNorm(data2(3).toBreeze, 2) ~== 1.0 absTol 1E-5) - assert(brzNorm(data2(4).toBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(0).asBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(2).asBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(3).asBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(4).asBreeze, 2) ~== 1.0 absTol 1E-5) assert(data2(0) ~== Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))) absTol 1E-5) assert(data2(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index de2c3c13bd923..9e4735afdd59f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.SparkFunSuite class BreezeMatrixConversionSuite extends SparkFunSuite { test("dense matrix to breeze") { val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) - val breeze = mat.toBreeze.asInstanceOf[BDM[Double]] + val breeze = mat.asBreeze.asInstanceOf[BDM[Double]] assert(breeze.rows === mat.numRows) assert(breeze.cols === mat.numCols) assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") @@ -48,7 +48,7 @@ class BreezeMatrixConversionSuite extends SparkFunSuite { val colPtrs = Array(0, 2, 4) val rowIndices = Array(1, 2, 1, 2) val mat = Matrices.sparse(3, 2, colPtrs, rowIndices, values) - val breeze = mat.toBreeze.asInstanceOf[BSM[Double]] + val breeze = mat.asBreeze.asInstanceOf[BSM[Double]] assert(breeze.rows === mat.numRows) assert(breeze.cols === mat.numCols) assert(breeze.data.eq(mat.asInstanceOf[SparseMatrix].values), "should not copy data") diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala index 3772c9235ad3a..996f621f18c80 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala @@ -33,12 +33,12 @@ class BreezeVectorConversionSuite extends SparkFunSuite { test("dense to breeze") { val vec = Vectors.dense(arr) - assert(vec.toBreeze === new BDV[Double](arr)) + assert(vec.asBreeze === new BDV[Double](arr)) } test("sparse to breeze") { val vec = Vectors.sparse(n, indices, values) - assert(vec.toBreeze === new BSV[Double](indices, values, n)) + assert(vec.asBreeze === new BSV[Double](indices, values, n)) } test("dense breeze to vector") { 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 8c5b4bda2518d..d0c4dd28e14ee 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 @@ -63,7 +63,7 @@ class MatricesSuite extends SparkFunSuite { (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0)) val mat2 = SparseMatrix.fromCOO(m, n, entries) - assert(mat.toBreeze === mat2.toBreeze) + assert(mat.asBreeze === mat2.asBreeze) assert(mat2.values.length == 4) } @@ -176,8 +176,8 @@ class MatricesSuite extends SparkFunSuite { val spMat2 = deMat1.toSparse val deMat2 = spMat1.toDense - assert(spMat1.toBreeze === spMat2.toBreeze) - assert(deMat1.toBreeze === deMat2.toBreeze) + assert(spMat1.asBreeze === spMat2.asBreeze) + assert(deMat1.asBreeze === deMat2.asBreeze) } test("map, update") { @@ -211,8 +211,8 @@ class MatricesSuite extends SparkFunSuite { val sATexpected = new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) - assert(dAT.toBreeze === dATexpected.toBreeze) - assert(sAT.toBreeze === sATexpected.toBreeze) + assert(dAT.asBreeze === dATexpected.asBreeze) + assert(sAT.asBreeze === sATexpected.asBreeze) assert(dA(1, 0) === dAT(0, 1)) assert(dA(2, 1) === dAT(1, 2)) assert(sA(1, 0) === sAT(0, 1)) @@ -221,8 +221,8 @@ class MatricesSuite extends SparkFunSuite { assert(!dA.toArray.eq(dAT.toArray), "has to have a new array") assert(dA.values.eq(dAT.transpose.asInstanceOf[DenseMatrix].values), "should not copy array") - assert(dAT.toSparse.toBreeze === sATexpected.toBreeze) - assert(sAT.toDense.toBreeze === dATexpected.toBreeze) + assert(dAT.toSparse.asBreeze === sATexpected.asBreeze) + assert(sAT.toDense.asBreeze === dATexpected.asBreeze) } test("foreachActive") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 2e9c40ab88ed9..71a3ceac1b947 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -269,7 +269,7 @@ class VectorsSuite extends SparkFunSuite with Logging { val denseVector1 = Vectors.dense(sparseVector1.toArray) val denseVector2 = Vectors.dense(sparseVector2.toArray) - val squaredDist = breezeSquaredDistance(sparseVector1.toBreeze, sparseVector2.toBreeze) + val squaredDist = breezeSquaredDistance(sparseVector1.asBreeze, sparseVector2.asBreeze) // SparseVector vs. SparseVector assert(Vectors.sqdist(sparseVector1, sparseVector2) ~== squaredDist relTol 1E-8) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index f37eaf225ab88..e5a2cbbb588df 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -152,7 +152,7 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val C = B.toIndexedRowMatrix.rows.collect - (C(0).vector.toBreeze, C(1).vector.toBreeze) match { + (C(0).vector.asBreeze, C(1).vector.asBreeze) match { case (denseVector: BDV[Double], sparseVector: BSV[Double]) => assert(denseVector.length === sparseVector.length) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala index 5b7ccb90158b0..99af5fa10d999 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -108,7 +108,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val C = A.multiply(B) val localA = A.toBreeze() val localC = C.toBreeze() - val expected = localA * B.toBreeze.asInstanceOf[BDM[Double]] + val expected = localA * B.asBreeze.asInstanceOf[BDM[Double]] assert(localC === expected) } @@ -119,7 +119,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { (90.0, 12.0, 24.0), (12.0, 17.0, 22.0), (24.0, 22.0, 30.0)) - assert(G.toBreeze === expected) + assert(G.asBreeze === expected) } test("svd") { @@ -128,8 +128,8 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(svd.U.isInstanceOf[IndexedRowMatrix]) val localA = A.toBreeze() val U = svd.U.toBreeze() - val s = svd.s.toBreeze.asInstanceOf[BDV[Double]] - val V = svd.V.toBreeze.asInstanceOf[BDM[Double]] + val s = svd.s.asBreeze.asInstanceOf[BDV[Double]] + val V = svd.V.asBreeze.asInstanceOf[BDM[Double]] assert(closeToZero(U.t * U - BDM.eye[Double](n))) assert(closeToZero(V.t * V - BDM.eye[Double](n))) assert(closeToZero(U * brzDiag(s) * V.t - localA)) @@ -155,7 +155,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { test("similar columns") { val A = new IndexedRowMatrix(indexedRows) - val gram = A.computeGramianMatrix().toBreeze.toDenseMatrix + val gram = A.computeGramianMatrix().asBreeze.toDenseMatrix val G = A.columnSimilarities().toBreeze() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 2dff52c601d81..7c4c6d8409c6c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -96,7 +96,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) for (mat <- Seq(denseMat, sparseMat)) { val G = mat.computeGramianMatrix() - assert(G.toBreeze === expected.toBreeze) + assert(G.asBreeze === expected.asBreeze) } } @@ -153,8 +153,8 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(V.numRows === n) assert(V.numCols === k) assertColumnEqualUpToSign(U.toBreeze(), localU, k) - assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) - assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + assertColumnEqualUpToSign(V.asBreeze.asInstanceOf[BDM[Double]], localV, k) + assert(closeToZero(s.asBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) } } val svdWithoutU = mat.computeSVD(1, computeU = false, 1e-9, 300, 1e-10, mode) @@ -207,7 +207,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val (pc, expVariance) = mat.computePrincipalComponentsAndExplainedVariance(k) assert(pc.numRows === n) assert(pc.numCols === k) - assertColumnEqualUpToSign(pc.toBreeze.asInstanceOf[BDM[Double]], principalComponents, k) + assertColumnEqualUpToSign(pc.asBreeze.asInstanceOf[BDM[Double]], principalComponents, k) assert( closeToZero(BDV(expVariance.toArray) - BDV(Arrays.copyOfRange(explainedVariance.data, 0, k)))) @@ -256,12 +256,12 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val calcQ = result.Q val calcR = result.R assert(closeToZero(abs(expected.q) - abs(calcQ.toBreeze()))) - assert(closeToZero(abs(expected.r) - abs(calcR.toBreeze.asInstanceOf[BDM[Double]]))) + assert(closeToZero(abs(expected.r) - abs(calcR.asBreeze.asInstanceOf[BDM[Double]]))) assert(closeToZero(calcQ.multiply(calcR).toBreeze - mat.toBreeze())) // Decomposition without computing Q val rOnly = mat.tallSkinnyQR(computeQ = false) assert(rOnly.Q == null) - assert(closeToZero(abs(expected.r) - abs(rOnly.R.toBreeze.asInstanceOf[BDM[Double]]))) + assert(closeToZero(abs(expected.r) - abs(rOnly.R.asBreeze.asInstanceOf[BDM[Double]]))) } } @@ -269,7 +269,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { for (mat <- Seq(denseMat, sparseMat)) { val result = mat.computeCovariance() val expected = breeze.linalg.cov(mat.toBreeze()) - assert(closeToZero(abs(expected) - abs(result.toBreeze.asInstanceOf[BDM[Double]]))) + assert(closeToZero(abs(expected) - abs(result.asBreeze.asInstanceOf[BDM[Double]]))) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala index 700f803490c08..e32767edb17a8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -104,8 +104,8 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log (Double.NaN, Double.NaN, 1.00000000, Double.NaN), (0.40047142, 0.91359586, Double.NaN, 1.0000000)) // scalastyle:on - assert(matrixApproxEqual(defaultMat.toBreeze, expected)) - assert(matrixApproxEqual(pearsonMat.toBreeze, expected)) + assert(matrixApproxEqual(defaultMat.asBreeze, expected)) + assert(matrixApproxEqual(pearsonMat.asBreeze, expected)) } test("corr(X) spearman") { @@ -118,7 +118,7 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log (Double.NaN, Double.NaN, 1.00000000, Double.NaN), (0.4000000, 0.9486833, Double.NaN, 1.0000000)) // scalastyle:on - assert(matrixApproxEqual(spearmanMat.toBreeze, expected)) + assert(matrixApproxEqual(spearmanMat.asBreeze, expected)) } test("method identification") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 0c6aabf1926e9..7b6bfee00cb1f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -53,13 +53,13 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { val norm2 = Vectors.norm(v2, 2.0) val v3 = Vectors.sparse(n, indices, indices.map(i => a(i) + 0.5)) val norm3 = Vectors.norm(v3, 2.0) - val squaredDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) + val squaredDist = breezeSquaredDistance(v1.asBreeze, v2.asBreeze) val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision) assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m") val fastSquaredDist2 = fastSquaredDistance(v1, norm1, Vectors.dense(v2.toArray), norm2, precision) assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") - val squaredDist2 = breezeSquaredDistance(v2.toBreeze, v3.toBreeze) + val squaredDist2 = breezeSquaredDistance(v2.asBreeze, v3.asBreeze) val fastSquaredDist3 = fastSquaredDistance(v2, norm2, v3, norm3, precision) assert((fastSquaredDist3 - squaredDist2) <= precision * squaredDist2, s"failed with m = $m") @@ -67,7 +67,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { val v4 = Vectors.sparse(n, indices.slice(0, m - 10), indices.map(i => a(i) + 0.5).slice(0, m - 10)) val norm4 = Vectors.norm(v4, 2.0) - val squaredDist = breezeSquaredDistance(v2.toBreeze, v4.toBreeze) + val squaredDist = breezeSquaredDistance(v2.asBreeze, v4.asBreeze) val fastSquaredDist = fastSquaredDistance(v2, norm2, v4, norm4, precision) assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m") diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 73debe9da4277..9d0d9b1be0771 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -775,6 +775,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.AlphaComponent"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Experimental"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.DeveloperApi") + ) ++ Seq( + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asBreeze"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asBreeze") ) case v if v.startsWith("1.6") => Seq( From 9c137b2e361ad80845dbf086c173bc430c53d2a2 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 27 May 2016 14:07:12 -0700 Subject: [PATCH 0430/1470] [SPARK-15431][SQL][BRANCH-2.0-TEST] rework the clisuite test cases ## What changes were proposed in this pull request? This PR reworks on the CliSuite test cases for `LIST FILES/JARS` commands. CC yhuai Thanks! Author: Xin Wu Closes #13361 from xwu0226/SPARK-15431-clisuite-new. (cherry picked from commit 019afd9c78a9f40e1d07f0a74868010206e90ed5) Signed-off-by: Yin Huai --- .../sql/hive/thriftserver/CliSuite.scala | 37 +++++++++++++------ 1 file changed, 26 insertions(+), 11 deletions(-) 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 656fe973b0cae..75535cad1b18e 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 @@ -62,13 +62,13 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { /** * Run a CLI operation and expect all the queries and expected answers to be returned. + * * @param timeout maximum time for the commands to complete * @param extraArgs any extra arguments * @param errorResponses a sequence of strings whose presence in the stdout of the forked process * is taken as an immediate error condition. That is: if a line containing * with one of these strings is found, fail the test immediately. * The default value is `Seq("Error:")` - * * @param queriesAndExpectedAnswers one or more tuples of query + answer */ def runCliWithin( @@ -239,22 +239,37 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { "" -> "This is a test for Spark-11624") } - ignore("list jars") { + test("list jars") { val jarFile = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") runCliWithin(2.minute)( - s"ADD JAR $jarFile" -> "", - s"LIST JARS" -> "TestUDTF.jar", - s"List JAR $jarFile" -> "TestUDTF.jar" + s"ADD JAR $jarFile;" -> "", + s"LIST JARS;" -> "TestUDTF.jar" + ) + } + + test("list jar ") { + val jarFile = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + runCliWithin(2.minute)( + s"ADD JAR $jarFile;" -> "", + s"List JAR $jarFile;" -> "TestUDTF.jar" + ) + } + + test("list files") { + val dataFilePath = Thread.currentThread(). + getContextClassLoader.getResource("data/files/small_kv.txt") + runCliWithin(2.minute)( + s"ADD FILE $dataFilePath;" -> "", + s"LIST FILES;" -> "small_kv.txt" ) } - ignore("list files") { - val dataFilePath = Thread.currentThread().getContextClassLoader - .getResource("data/files/small_kv.txt") + test("list file ") { + val dataFilePath = Thread.currentThread(). + getContextClassLoader.getResource("data/files/small_kv.txt") runCliWithin(2.minute)( - s"ADD FILE $dataFilePath" -> "", - s"LIST FILES" -> "small_kv.txt", - s"LIST FILE $dataFilePath" -> "small_kv.txt" + s"ADD FILE $dataFilePath;" -> "", + s"LIST FILE $dataFilePath;" -> "small_kv.txt" ) } } From dc6e94157ce08df91aa1a31db8e5ec733a1ab0c5 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Fri, 27 May 2016 16:59:38 -0700 Subject: [PATCH 0431/1470] [SPARK-9876][SQL] Update Parquet to 1.8.1. ## What changes were proposed in this pull request? This includes minimal changes to get Spark using the current release of Parquet, 1.8.1. ## How was this patch tested? This uses the existing Parquet tests. Author: Ryan Blue Closes #13280 from rdblue/SPARK-9876-update-parquet. (cherry picked from commit 776d183c82b424ef7c3cae30537d8afe9b9eee83) Signed-off-by: Cheng Lian --- dev/deps/spark-deps-hadoop-2.2 | 11 ++- dev/deps/spark-deps-hadoop-2.3 | 11 ++- dev/deps/spark-deps-hadoop-2.4 | 11 ++- dev/deps/spark-deps-hadoop-2.6 | 11 ++- dev/deps/spark-deps-hadoop-2.7 | 11 ++- pom.xml | 2 +- .../SpecificParquetRecordReaderBase.java | 20 +++-- .../parquet/CatalystReadSupport.scala | 12 ++- .../parquet/CatalystSchemaConverter.scala | 16 ++++ .../datasources/parquet/ParquetFilters.scala | 83 ++++--------------- .../parquet/ParquetSchemaSuite.scala | 20 +++-- 11 files changed, 91 insertions(+), 117 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 578691cc9395d..deec033c21dae 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -129,14 +129,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.7.0.jar -parquet-common-1.7.0.jar -parquet-encoding-1.7.0.jar +parquet-column-1.8.1.jar +parquet-common-1.8.1.jar +parquet-encoding-1.8.1.jar parquet-format-2.3.0-incubating.jar -parquet-generator-1.7.0.jar -parquet-hadoop-1.7.0.jar +parquet-hadoop-1.8.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.7.0.jar +parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index fc6306f366815..43c7dd35805dc 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -136,14 +136,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.7.0.jar -parquet-common-1.7.0.jar -parquet-encoding-1.7.0.jar +parquet-column-1.8.1.jar +parquet-common-1.8.1.jar +parquet-encoding-1.8.1.jar parquet-format-2.3.0-incubating.jar -parquet-generator-1.7.0.jar -parquet-hadoop-1.7.0.jar +parquet-hadoop-1.8.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.7.0.jar +parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index dee1417c79af1..7186b305a8af7 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -136,14 +136,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.7.0.jar -parquet-common-1.7.0.jar -parquet-encoding-1.7.0.jar +parquet-column-1.8.1.jar +parquet-common-1.8.1.jar +parquet-encoding-1.8.1.jar parquet-format-2.3.0-incubating.jar -parquet-generator-1.7.0.jar -parquet-hadoop-1.7.0.jar +parquet-hadoop-1.8.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.7.0.jar +parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 9695661b9c92c..3e4ed74cc6b25 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -144,14 +144,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.7.0.jar -parquet-common-1.7.0.jar -parquet-encoding-1.7.0.jar +parquet-column-1.8.1.jar +parquet-common-1.8.1.jar +parquet-encoding-1.8.1.jar parquet-format-2.3.0-incubating.jar -parquet-generator-1.7.0.jar -parquet-hadoop-1.7.0.jar +parquet-hadoop-1.8.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.7.0.jar +parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 18c136ed63cb2..6b999538a3dd8 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -145,14 +145,13 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.7.0.jar -parquet-common-1.7.0.jar -parquet-encoding-1.7.0.jar +parquet-column-1.8.1.jar +parquet-common-1.8.1.jar +parquet-encoding-1.8.1.jar parquet-format-2.3.0-incubating.jar -parquet-generator-1.7.0.jar -parquet-hadoop-1.7.0.jar +parquet-hadoop-1.8.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.7.0.jar +parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/pom.xml b/pom.xml index 3fa0eeb5f0387..ce9aa9aa00cc0 100644 --- a/pom.xml +++ b/pom.xml @@ -133,7 +133,7 @@ 1.2.1 10.11.1.1 - 1.7.0 + 1.8.1 1.6.0 9.2.16.v20160414 3.1.0 diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index cbe8f78164ae7..3f7a872ff635f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -58,6 +58,8 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.hadoop.util.ConfigurationUtil; import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; import org.apache.spark.sql.types.StructType; @@ -186,15 +188,19 @@ protected void initialize(String path, List columns) throws IOException if (columns == null) { this.requestedSchema = fileSchema; } else { - Types.MessageTypeBuilder builder = Types.buildMessage(); - for (String s: columns) { - if (!fileSchema.containsField(s)) { - throw new IOException("Can only project existing columns. Unknown field: " + s + - " File schema:\n" + fileSchema); + if (columns.size() > 0) { + Types.MessageTypeBuilder builder = Types.buildMessage(); + for (String s: columns) { + if (!fileSchema.containsField(s)) { + throw new IOException("Can only project existing columns. Unknown field: " + s + + " File schema:\n" + fileSchema); + } + builder.addFields(fileSchema.getType(s)); } - builder.addFields(fileSchema.getType(s)); + this.requestedSchema = builder.named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME()); + } else { + this.requestedSchema = CatalystSchemaConverter.EMPTY_MESSAGE(); } - this.requestedSchema = builder.named("spark_schema"); } this.sparkSchema = new CatalystSchemaConverter(config).convert(requestedSchema); this.reader = new ParquetFileReader(config, file, blocks, requestedSchema.getColumns()); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index 850e807b8677e..9c885b252f01b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -109,10 +109,14 @@ private[parquet] object CatalystReadSupport { */ def clipParquetSchema(parquetSchema: MessageType, catalystSchema: StructType): MessageType = { val clippedParquetFields = clipParquetGroupFields(parquetSchema.asGroupType(), catalystSchema) - Types - .buildMessage() - .addFields(clippedParquetFields: _*) - .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + if (clippedParquetFields.isEmpty) { + CatalystSchemaConverter.EMPTY_MESSAGE + } else { + Types + .buildMessage() + .addFields(clippedParquetFields: _*) + .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + } } private def clipParquetType(parquetType: Type, catalystType: DataType): Type = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala index 6f6340f541ada..3688c3e2b57e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala @@ -538,6 +538,22 @@ private[parquet] class CatalystSchemaConverter( private[parquet] object CatalystSchemaConverter { val SPARK_PARQUET_SCHEMA_NAME = "spark_schema" + // !! HACK ALERT !! + // + // PARQUET-363 & PARQUET-278: parquet-mr 1.8.1 doesn't allow constructing empty GroupType, + // which prevents us to avoid selecting any columns for queries like `SELECT COUNT(*) FROM t`. + // This issue has been fixed in parquet-mr 1.8.2-SNAPSHOT. + // + // To workaround this problem, here we first construct a `MessageType` with a single dummy + // field, and then remove the field to obtain an empty `MessageType`. + // + // TODO Reverts this change after upgrading parquet-mr to 1.8.2+ + val EMPTY_MESSAGE = Types + .buildMessage() + .required(PrimitiveType.PrimitiveTypeName.INT32).named("dummy") + .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + EMPTY_MESSAGE.getFields.clear() + def checkFieldName(name: String): Unit = { // ,;{}()\n\t= and space are special characters in Parquet schema checkConversionRequirement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 95afdc789f322..624081250113a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -22,8 +22,6 @@ import java.io.Serializable import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.io.api.Binary -import org.apache.parquet.schema.OriginalType -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.spark.sql.sources import org.apache.spark.sql.types._ @@ -53,18 +51,15 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) - // See https://issues.apache.org/jira/browse/SPARK-11153 - /* // Binary.fromString and Binary.fromByteArray don't accept null values case StringType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[String].getBytes("utf-8"))).orNull) + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) case BinaryType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(b => Binary.fromByteArray(v.asInstanceOf[Array[Byte]])).orNull) - */ + Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) } private val makeNotEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -79,17 +74,14 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.notEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) - // See https://issues.apache.org/jira/browse/SPARK-11153 - /* case StringType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[String].getBytes("utf-8"))).orNull) + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) case BinaryType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(b => Binary.fromByteArray(v.asInstanceOf[Array[Byte]])).orNull) - */ + Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) } private val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -102,16 +94,13 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) - // See https://issues.apache.org/jira/browse/SPARK-11153 - /* case StringType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), - Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) + Binary.fromString(v.asInstanceOf[String])) case BinaryType => (n: String, v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) - */ + FilterApi.lt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) } private val makeLtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -124,16 +113,13 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) - // See https://issues.apache.org/jira/browse/SPARK-11153 - /* case StringType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), - Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) + Binary.fromString(v.asInstanceOf[String])) case BinaryType => (n: String, v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) - */ + FilterApi.ltEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) } private val makeGt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -147,15 +133,13 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) // See https://issues.apache.org/jira/browse/SPARK-11153 - /* case StringType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), - Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) + Binary.fromString(v.asInstanceOf[String])) case BinaryType => (n: String, v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) - */ + FilterApi.gt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) } private val makeGtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -168,16 +152,13 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) - // See https://issues.apache.org/jira/browse/SPARK-11153 - /* case StringType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), - Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) + Binary.fromString(v.asInstanceOf[String])) case BinaryType => (n: String, v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) - */ + FilterApi.gtEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) } private val makeInSet: PartialFunction[DataType, (String, Set[Any]) => FilterPredicate] = { @@ -194,17 +175,14 @@ private[sql] object ParquetFilters { (n: String, v: Set[Any]) => FilterApi.userDefined(doubleColumn(n), SetInFilter(v.asInstanceOf[Set[java.lang.Double]])) - // See https://issues.apache.org/jira/browse/SPARK-11153 - /* case StringType => (n: String, v: Set[Any]) => FilterApi.userDefined(binaryColumn(n), - SetInFilter(v.map(s => Binary.fromByteArray(s.asInstanceOf[String].getBytes("utf-8"))))) + SetInFilter(v.map(s => Binary.fromString(s.asInstanceOf[String])))) case BinaryType => (n: String, v: Set[Any]) => FilterApi.userDefined(binaryColumn(n), - SetInFilter(v.map(e => Binary.fromByteArray(e.asInstanceOf[Array[Byte]])))) - */ + SetInFilter(v.map(e => Binary.fromReusedByteArray(e.asInstanceOf[Array[Byte]])))) } /** @@ -228,8 +206,6 @@ private[sql] object ParquetFilters { def createFilter(schema: StructType, predicate: sources.Filter): Option[FilterPredicate] = { val dataTypeOf = getFieldMap(schema).toMap - relaxParquetValidTypeMap - // NOTE: // // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, @@ -299,35 +275,4 @@ private[sql] object ParquetFilters { case _ => None } } - - // !! HACK ALERT !! - // - // This lazy val is a workaround for PARQUET-201, and should be removed once we upgrade to - // parquet-mr 1.8.1 or higher versions. - // - // In Parquet, not all types of columns can be used for filter push-down optimization. The set - // of valid column types is controlled by `ValidTypeMap`. Unfortunately, in parquet-mr 1.7.0 and - // prior versions, the limitation is too strict, and doesn't allow `BINARY (ENUM)` columns to be - // pushed down. - // - // This restriction is problematic for Spark SQL, because Spark SQL doesn't have a type that maps - // to Parquet original type `ENUM` directly, and always converts `ENUM` to `StringType`. Thus, - // a predicate involving a `ENUM` field can be pushed-down as a string column, which is perfectly - // legal except that it fails the `ValidTypeMap` check. - // - // Here we add `BINARY (ENUM)` into `ValidTypeMap` lazily via reflection to workaround this issue. - private lazy val relaxParquetValidTypeMap: Unit = { - val constructor = Class - .forName(classOf[ValidTypeMap].getCanonicalName + "$FullTypeDescriptor") - .getDeclaredConstructor(classOf[PrimitiveTypeName], classOf[OriginalType]) - - constructor.setAccessible(true) - val enumTypeDescriptor = constructor - .newInstance(PrimitiveTypeName.BINARY, OriginalType.ENUM) - .asInstanceOf[AnyRef] - - val addMethod = classOf[ValidTypeMap].getDeclaredMethods.find(_.getName == "add").get - addMethod.setAccessible(true) - addMethod.invoke(null, classOf[Binary], enumTypeDescriptor) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 6db649228210d..0b5038cb82803 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.ScalaReflection @@ -1065,18 +1065,26 @@ class ParquetSchemaSuite extends ParquetSchemaTest { parquetSchema: String, catalystSchema: StructType, expectedSchema: String): Unit = { + testSchemaClipping(testName, parquetSchema, catalystSchema, + MessageTypeParser.parseMessageType(expectedSchema)) + } + + private def testSchemaClipping( + testName: String, + parquetSchema: String, + catalystSchema: StructType, + expectedSchema: MessageType): Unit = { test(s"Clipping - $testName") { - val expected = MessageTypeParser.parseMessageType(expectedSchema) val actual = CatalystReadSupport.clipParquetSchema( MessageTypeParser.parseMessageType(parquetSchema), catalystSchema) try { - expected.checkContains(actual) - actual.checkContains(expected) + expectedSchema.checkContains(actual) + actual.checkContains(expectedSchema) } catch { case cause: Throwable => fail( s"""Expected clipped schema: - |$expected + |$expectedSchema |Actual clipped schema: |$actual """.stripMargin, @@ -1429,7 +1437,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { catalystSchema = new StructType(), - expectedSchema = "message root {}") + expectedSchema = CatalystSchemaConverter.EMPTY_MESSAGE) testSchemaClipping( "disjoint field sets", From 80a40e8e2cc198c34dabbc431d4ca302319fbbad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 27 May 2016 17:27:24 -0700 Subject: [PATCH 0432/1470] [SPARK-15594][SQL] ALTER TABLE SERDEPROPERTIES does not respect partition spec ## What changes were proposed in this pull request? These commands ignore the partition spec and change the storage properties of the table itself: ``` ALTER TABLE table_name PARTITION (a=1, b=2) SET SERDE 'my_serde' ALTER TABLE table_name PARTITION (a=1, b=2) SET SERDEPROPERTIES ('key1'='val1') ``` Now they change the storage properties of the specified partition. ## How was this patch tested? DDLSuite Author: Andrew Or Closes #13343 from andrewor14/alter-table-serdeproperties. (cherry picked from commit 4a2fb8b87ca4517e0f4a1d7a1a1b3c08c1c1294d) Signed-off-by: Yin Huai --- .../spark/sql/execution/command/ddl.scala | 26 ++++++-- .../sql/execution/command/DDLSuite.scala | 64 +++++++++++++++++++ 2 files changed, 84 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 95bac94996577..5fd0b83cf0a5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -293,7 +293,7 @@ case class AlterTableSerDePropertiesCommand( tableName: TableIdentifier, serdeClassName: Option[String], serdeProperties: Option[Map[String, String]], - partition: Option[Map[String, String]]) + partSpec: Option[TablePartitionSpec]) extends RunnableCommand { // should never happen if we parsed things correctly @@ -306,15 +306,29 @@ case class AlterTableSerDePropertiesCommand( "ALTER TABLE SERDEPROPERTIES") val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) - // Do not support setting serde for datasource tables + // For datasource tables, disallow setting serde or specifying partition + if (partSpec.isDefined && DDLUtils.isDatasourceTable(table)) { + throw new AnalysisException("Operation not allowed: ALTER TABLE SET " + + "[SERDE | SERDEPROPERTIES] for a specific partition is not supported " + + "for tables created with the datasource API") + } if (serdeClassName.isDefined && DDLUtils.isDatasourceTable(table)) { throw new AnalysisException("Operation not allowed: ALTER TABLE SET SERDE is " + "not supported for tables created with the datasource API") } - val newTable = table.withNewStorage( - serde = serdeClassName.orElse(table.storage.serde), - serdeProperties = table.storage.serdeProperties ++ serdeProperties.getOrElse(Map())) - catalog.alterTable(newTable) + if (partSpec.isEmpty) { + val newTable = table.withNewStorage( + serde = serdeClassName.orElse(table.storage.serde), + serdeProperties = table.storage.serdeProperties ++ serdeProperties.getOrElse(Map())) + catalog.alterTable(newTable) + } else { + val spec = partSpec.get + val part = catalog.getPartition(tableName, spec) + val newPart = part.copy(storage = part.storage.copy( + serde = serdeClassName.orElse(part.storage.serde), + serdeProperties = part.storage.serdeProperties ++ serdeProperties.getOrElse(Map()))) + catalog.alterPartitions(tableName, Seq(newPart)) + } Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index ccb4006483906..5d45cfb501c73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -538,6 +538,14 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { testSetSerde(isDatasourceTable = true) } + test("alter table: set serde partition") { + testSetSerdePartition(isDatasourceTable = false) + } + + test("alter table: set serde partition (datasource table)") { + testSetSerdePartition(isDatasourceTable = true) + } + test("alter table: bucketing is not supported") { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) @@ -931,6 +939,62 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(e.getMessage.contains(DATASOURCE_PREFIX + "foo")) } + private def testSetSerdePartition(isDatasourceTable: Boolean): Unit = { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + val spec = Map("a" -> "1", "b" -> "2") + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + createTablePartition(catalog, spec, tableIdent) + createTablePartition(catalog, Map("a" -> "1", "b" -> "3"), tableIdent) + createTablePartition(catalog, Map("a" -> "2", "b" -> "2"), tableIdent) + createTablePartition(catalog, Map("a" -> "2", "b" -> "3"), tableIdent) + if (isDatasourceTable) { + convertToDatasourceTable(catalog, tableIdent) + } + assert(catalog.getPartition(tableIdent, spec).storage.serde.isEmpty) + assert(catalog.getPartition(tableIdent, spec).storage.serdeProperties.isEmpty) + // set table serde and/or properties (should fail on datasource tables) + if (isDatasourceTable) { + val e1 = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'whatever'") + } + val e2 = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'org.apache.madoop' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + } + assert(e1.getMessage.contains("datasource")) + assert(e2.getMessage.contains("datasource")) + } else { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'org.apache.jadoop'") + assert(catalog.getPartition(tableIdent, spec).storage.serde == Some("org.apache.jadoop")) + assert(catalog.getPartition(tableIdent, spec).storage.serdeProperties.isEmpty) + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'org.apache.madoop' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + assert(catalog.getPartition(tableIdent, spec).storage.serde == Some("org.apache.madoop")) + assert(catalog.getPartition(tableIdent, spec).storage.serdeProperties == + Map("k" -> "v", "kay" -> "vee")) + } + // set serde properties only + maybeWrapException(isDatasourceTable) { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) " + + "SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')") + assert(catalog.getPartition(tableIdent, spec).storage.serdeProperties == + Map("k" -> "vvv", "kay" -> "vee")) + } + // set things without explicitly specifying database + catalog.setCurrentDatabase("dbx") + maybeWrapException(isDatasourceTable) { + sql("ALTER TABLE tab1 PARTITION (a=1, b=2) SET SERDEPROPERTIES ('kay' = 'veee')") + assert(catalog.getPartition(tableIdent, spec).storage.serdeProperties == + Map("k" -> "vvv", "kay" -> "veee")) + } + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist SET SERDEPROPERTIES ('x' = 'y')") + } + } + private def testAddPartitions(isDatasourceTable: Boolean): Unit = { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) From 8467e2102886da1cefb43f2aaa69864375fe91bc Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 27 May 2016 20:59:24 -0500 Subject: [PATCH 0433/1470] [SPARK-15449][MLLIB][EXAMPLE] Wrong Data Format - Documentation Issue ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) In the MLLib naivebayes example, scala and python example doesn't use libsvm data, but Java does. I make changes in scala and python example to use the libsvm data as the same as Java example. ## How was this patch tested? Manual tests Author: wm624@hotmail.com Closes #13301 from wangmiao1981/example. (cherry picked from commit 5d4dafe8fdea49dcbd6b0e4c23e3791fa30c8911) Signed-off-by: Sean Owen --- data/mllib/sample_naive_bayes_data.txt | 12 ------------ .../examples/mllib/JavaNaiveBayesExample.java | 4 ++-- .../src/main/python/mllib/naive_bayes_example.py | 13 ++++--------- .../spark/examples/mllib/NaiveBayesExample.scala | 14 ++++---------- 4 files changed, 10 insertions(+), 33 deletions(-) delete mode 100644 data/mllib/sample_naive_bayes_data.txt diff --git a/data/mllib/sample_naive_bayes_data.txt b/data/mllib/sample_naive_bayes_data.txt deleted file mode 100644 index bd22bea3a59d6..0000000000000 --- a/data/mllib/sample_naive_bayes_data.txt +++ /dev/null @@ -1,12 +0,0 @@ -0,1 0 0 -0,2 0 0 -0,3 0 0 -0,4 0 0 -1,0 1 0 -1,0 2 0 -1,0 3 0 -1,0 4 0 -2,0 0 1 -2,0 0 2 -2,0 0 3 -2,0 0 4 \ No newline at end of file diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java index 2b17dbb96365e..f4ec04b0c677c 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java @@ -36,9 +36,9 @@ public static void main(String[] args) { SparkConf sparkConf = new SparkConf().setAppName("JavaNaiveBayesExample"); JavaSparkContext jsc = new JavaSparkContext(sparkConf); // $example on$ - String path = "data/mllib/sample_naive_bayes_data.txt"; + String path = "data/mllib/sample_libsvm_data.txt"; JavaRDD inputData = MLUtils.loadLibSVMFile(jsc.sc(), path).toJavaRDD(); - JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 0.4}, 12345); + JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 0.4}); JavaRDD training = tmp[0]; // training set JavaRDD test = tmp[1]; // test set final NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); diff --git a/examples/src/main/python/mllib/naive_bayes_example.py b/examples/src/main/python/mllib/naive_bayes_example.py index 35724f7d6a92d..749353b20eb3e 100644 --- a/examples/src/main/python/mllib/naive_bayes_example.py +++ b/examples/src/main/python/mllib/naive_bayes_example.py @@ -29,15 +29,9 @@ from pyspark import SparkContext # $example on$ from pyspark.mllib.classification import NaiveBayes, NaiveBayesModel -from pyspark.mllib.linalg import Vectors -from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.util import MLUtils -def parseLine(line): - parts = line.split(',') - label = float(parts[0]) - features = Vectors.dense([float(x) for x in parts[1].split(' ')]) - return LabeledPoint(label, features) # $example off$ if __name__ == "__main__": @@ -45,10 +39,11 @@ def parseLine(line): sc = SparkContext(appName="PythonNaiveBayesExample") # $example on$ - data = sc.textFile('data/mllib/sample_naive_bayes_data.txt').map(parseLine) + # Load and parse the data file. + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") # Split data approximately into training (60%) and test (40%) - training, test = data.randomSplit([0.6, 0.4], seed=0) + training, test = data.randomSplit([0.6, 0.4]) # Train a naive Bayes model. model = NaiveBayes.train(training, 1.0) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala index 0187ad603a654..b321d8e127aa5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala @@ -21,8 +21,7 @@ package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.classification.{NaiveBayes, NaiveBayesModel} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils // $example off$ object NaiveBayesExample { @@ -31,16 +30,11 @@ object NaiveBayesExample { val conf = new SparkConf().setAppName("NaiveBayesExample") val sc = new SparkContext(conf) // $example on$ - val data = sc.textFile("data/mllib/sample_naive_bayes_data.txt") - val parsedData = data.map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) - } + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") // Split data into training (60%) and test (40%). - val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L) - val training = splits(0) - val test = splits(1) + val Array(training, test) = data.randomSplit(Array(0.6, 0.4)) val model = NaiveBayes.train(training, lambda = 1.0, modelType = "multinomial") From 6d82e0c1b8b4368e91aeebfc80430a61762c7e88 Mon Sep 17 00:00:00 2001 From: dding3 Date: Fri, 27 May 2016 21:01:50 -0500 Subject: [PATCH 0434/1470] [SPARK-15562][ML] Delete temp directory after program exit in DataFrameExample ## What changes were proposed in this pull request? Temp directory used to save records is not deleted after program exit in DataFrameExample. Although it called deleteOnExit, it doesn't work as the directory is not empty. Similar things happend in ContextCleanerSuite. Update the code to make sure temp directory is deleted after program exit. ## How was this patch tested? unit tests and local build. Author: dding3 Closes #13328 from dding3/master. (cherry picked from commit 88c9c467a31630c558719679ca0894873a268b27) Signed-off-by: Sean Owen --- .../src/test/scala/org/apache/spark/ContextCleanerSuite.scala | 4 ++-- .../scala/org/apache/spark/examples/ml/DataFrameExample.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 69ff6c7c28ee9..6724af952505f 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage._ +import org.apache.spark.util.Utils /** * An abstract base class for context cleaner tests, which sets up a context with a config @@ -206,8 +207,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { } test("automatically cleanup normal checkpoint") { - val checkpointDir = java.io.File.createTempFile("temp", "") - checkpointDir.deleteOnExit() + val checkpointDir = Utils.createTempDir() checkpointDir.delete() var rdd = newPairRDD() sc.setCheckpointDir(checkpointDir.toString) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index c69027babba8d..11faa6192b3fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.util.Utils /** * An example of how to use [[org.apache.spark.sql.DataFrame]] for ML. Run with @@ -86,8 +87,7 @@ object DataFrameExample { println(s"Selected features column with average values:\n ${featureSummary.mean.toString}") // Save the records in a parquet file. - val tmpDir = Files.createTempDir() - tmpDir.deleteOnExit() + val tmpDir = Utils.createTempDir() val outputDir = new File(tmpDir, "dataframe").toString println(s"Saving to $outputDir as Parquet file.") df.write.parquet(outputDir) From 3801fb4f35ba1ffb8dbaf8326eff927b738551f2 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 27 May 2016 21:57:41 -0500 Subject: [PATCH 0435/1470] [SPARK-15610][ML] update error message for k in pca ## What changes were proposed in this pull request? Fix the wrong bound of `k` in `PCA` `require(k <= sources.first().size, ...` -> `require(k < sources.first().size` BTW, remove unused import in `ml.ElementwiseProduct` ## How was this patch tested? manual tests Author: Zheng RuiFeng Closes #13356 from zhengruifeng/fix_pca. (cherry picked from commit 9893dc975784551a62f65bbd709f8972e0204b2a) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/feature/ElementwiseProduct.scala | 1 - .../src/main/scala/org/apache/spark/mllib/feature/PCA.scala | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 91989c3d2f5d2..9d2e60fa3f1e4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -23,7 +23,6 @@ import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.Param import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.types.DataType diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala index 30c403e547bee..15b72205ac17a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala @@ -40,8 +40,9 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { */ @Since("1.4.0") def fit(sources: RDD[Vector]): PCAModel = { - require(k <= sources.first().size, - s"source vector size is ${sources.first().size} must be greater than k=$k") + val numFeatures = sources.first().size + require(k <= numFeatures, + s"source vector size $numFeatures must be no less than k=$k") val mat = new RowMatrix(sources) val (pc, explainedVariance) = mat.computePrincipalComponentsAndExplainedVariance(k) @@ -58,7 +59,6 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { case m => throw new IllegalArgumentException("Unsupported matrix format. Expected " + s"SparseMatrix or DenseMatrix. Instead got: ${m.getClass}") - } val denseExplainedVariance = explainedVariance match { case dv: DenseVector => From ada319844a42b0b76e92d62faec258bfd0bb10ac Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 27 May 2016 21:20:02 -0700 Subject: [PATCH 0436/1470] [SPARK-15633][MINOR] Make package name for Java tests consistent ## What changes were proposed in this pull request? This is a simple patch that makes package names for Java 8 test suites consistent. I moved everything to test.org.apache.spark to we can test package private APIs properly. Also added "java8" as the package name so we can easily run all the tests related to Java 8. ## How was this patch tested? This is a test only change. Author: Reynold Xin Closes #13364 from rxin/SPARK-15633. (cherry picked from commit 73178c75565e20f53e6ee1478f3d976732c64438) Signed-off-by: Reynold Xin --- core/src/test/scala/org/apache/spark/SparkFunSuite.scala | 2 +- .../org/apache/spark/java8/Java8RDDAPISuite.java} | 8 +++++--- .../org/apache/spark/java8/dstream}/Java8APISuite.java | 3 ++- .../spark/java8}/sql/Java8DatasetAggregatorSuite.java | 7 ++++--- .../org/apache/spark/java8}/JDK8ScalaSuite.scala | 5 ++++- .../sql/{sources => }/JavaDatasetAggregatorSuite.java | 2 +- .../sql/{sources => }/JavaDatasetAggregatorSuiteBase.java | 2 +- .../apache/spark/sql/{sources => }/JavaSaveLoadSuite.java | 2 +- 8 files changed, 19 insertions(+), 12 deletions(-) rename external/java8-tests/src/test/java/{org/apache/spark/Java8APISuite.java => test/org/apache/spark/java8/Java8RDDAPISuite.java} (98%) rename external/java8-tests/src/test/java/{org/apache/spark/streaming => test/org/apache/spark/java8/dstream}/Java8APISuite.java (99%) rename external/java8-tests/src/test/java/{org/apache/spark => test/org/apache/spark/java8}/sql/Java8DatasetAggregatorSuite.java (88%) rename external/java8-tests/src/test/scala/{org/apache/spark => test/org/apache/spark/java8}/JDK8ScalaSuite.scala (89%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources => }/JavaDatasetAggregatorSuite.java (99%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources => }/JavaDatasetAggregatorSuiteBase.java (98%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources => }/JavaSaveLoadSuite.java (98%) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 0081bca63959c..cd876807f890e 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.AccumulatorContext /** * Base abstract class for all unit tests in Spark for handling common functionality. */ -private[spark] abstract class SparkFunSuite +abstract class SparkFunSuite extends FunSuite with BeforeAndAfterAll with Logging { diff --git a/external/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java similarity index 98% rename from external/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java rename to external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java index 6ac5ca9cf56af..8ee0e7e4156bf 100644 --- a/external/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark; +package test.org.apache.spark.java8; import java.io.File; import java.io.Serializable; @@ -33,6 +33,8 @@ import org.junit.Before; import org.junit.Test; +import org.apache.spark.Accumulator; +import org.apache.spark.AccumulatorParam; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -45,8 +47,8 @@ * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8 * lambda syntax. */ -public class Java8APISuite implements Serializable { - static int foreachCalls = 0; +public class Java8RDDAPISuite implements Serializable { + private static int foreachCalls = 0; private transient JavaSparkContext sc; @Before diff --git a/external/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java similarity index 99% rename from external/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java rename to external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java index d0fed303e659c..cf5607f5e814e 100644 --- a/external/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java +++ b/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming; +package test.org.apache.spark.java8.dstream; import java.io.Serializable; import java.util.*; @@ -33,6 +33,7 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaMapWithStateDStream; diff --git a/external/java8-tests/src/test/java/org/apache/spark/sql/Java8DatasetAggregatorSuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/sql/Java8DatasetAggregatorSuite.java similarity index 88% rename from external/java8-tests/src/test/java/org/apache/spark/sql/Java8DatasetAggregatorSuite.java rename to external/java8-tests/src/test/java/test/org/apache/spark/java8/sql/Java8DatasetAggregatorSuite.java index 1a2aea67d9d03..10d25fa4458a4 100644 --- a/external/java8-tests/src/test/java/org/apache/spark/sql/Java8DatasetAggregatorSuite.java +++ b/external/java8-tests/src/test/java/test/org/apache/spark/java8/sql/Java8DatasetAggregatorSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources; +package test.org.apache.spark.java8.sql; import java.util.Arrays; @@ -26,6 +26,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.KeyValueGroupedDataset; import org.apache.spark.sql.expressions.javalang.typed; +import test.org.apache.spark.sql.JavaDatasetAggregatorSuiteBase; /** * Suite that replicates tests in JavaDatasetAggregatorSuite using lambda syntax. @@ -42,7 +43,7 @@ public void testTypedAggregationAverage() { public void testTypedAggregationCount() { KeyValueGroupedDataset> grouped = generateGroupedDataset(); Dataset> agged = grouped.agg(typed.count(v -> v)); - Assert.assertEquals(Arrays.asList(tuple2("a", 2), tuple2("b", 1)), agged.collectAsList()); + Assert.assertEquals(Arrays.asList(tuple2("a", 2L), tuple2("b", 1L)), agged.collectAsList()); } @Test @@ -56,6 +57,6 @@ public void testTypedAggregationSumDouble() { public void testTypedAggregationSumLong() { KeyValueGroupedDataset> grouped = generateGroupedDataset(); Dataset> agged = grouped.agg(typed.sumLong(v -> (long)v._2())); - Assert.assertEquals(Arrays.asList(tuple2("a", 3), tuple2("b", 3)), agged.collectAsList()); + Assert.assertEquals(Arrays.asList(tuple2("a", 3L), tuple2("b", 3L)), agged.collectAsList()); } } diff --git a/external/java8-tests/src/test/scala/org/apache/spark/JDK8ScalaSuite.scala b/external/java8-tests/src/test/scala/test/org/apache/spark/java8/JDK8ScalaSuite.scala similarity index 89% rename from external/java8-tests/src/test/scala/org/apache/spark/JDK8ScalaSuite.scala rename to external/java8-tests/src/test/scala/test/org/apache/spark/java8/JDK8ScalaSuite.scala index fa0681db41088..c4042e47e84e8 100644 --- a/external/java8-tests/src/test/scala/org/apache/spark/JDK8ScalaSuite.scala +++ b/external/java8-tests/src/test/scala/test/org/apache/spark/java8/JDK8ScalaSuite.scala @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.spark +package test.org.apache.spark.java8 + +import org.apache.spark.SharedSparkContext +import org.apache.spark.SparkFunSuite /** * Test cases where JDK8-compiled Scala user code is used with Spark. diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaDatasetAggregatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java similarity index 99% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaDatasetAggregatorSuite.java rename to sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java index f9842e130b5d0..fe863715162f5 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaDatasetAggregatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources; +package test.org.apache.spark.sql; import java.util.Arrays; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaDatasetAggregatorSuiteBase.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java similarity index 98% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaDatasetAggregatorSuiteBase.java rename to sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java index 059c2d9f2cde5..8fc4eff55ddd0 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaDatasetAggregatorSuiteBase.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources; +package test.org.apache.spark.sql; import java.io.Serializable; import java.util.Arrays; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java similarity index 98% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java rename to sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java index 9840bc46f95f2..6941c86dfcd4b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources; +package test.org.apache.spark.sql; import java.io.File; import java.io.IOException; From 36045106d43b3952c55bae4439dbc86892399b3c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 27 May 2016 21:24:08 -0700 Subject: [PATCH 0437/1470] [SPARK-15553][SQL] Dataset.createTempView should use CreateViewCommand ## What changes were proposed in this pull request? Let `Dataset.createTempView` and `Dataset.createOrReplaceTempView` use `CreateViewCommand`, rather than calling `SparkSession.createTempView`. Besides, this patch also removes `SparkSession.createTempView`. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13327 from viirya/dataset-createtempview. (cherry picked from commit f1b220eeeed1d4d12121fe0b3b175da44488da68) Signed-off-by: Reynold Xin --- .../sql/catalyst/catalog/interface.scala | 5 ++++ .../scala/org/apache/spark/sql/Dataset.scala | 23 +++++++++++++++---- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../org/apache/spark/sql/SparkSession.scala | 11 --------- .../spark/sql/execution/SparkSqlParser.scala | 18 +++++++-------- .../spark/sql/execution/command/cache.scala | 3 +-- .../spark/sql/execution/command/views.scala | 8 +++++-- 7 files changed, 39 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 4a073d11893c4..77731b1f26eed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -50,6 +50,11 @@ case class CatalogStorageFormat( compressed: Boolean, serdeProperties: Map[String, String]) +object CatalogStorageFormat { + /** Empty storage format for default values and copies. */ + val EmptyStorageFormat = CatalogStorageFormat(locationUri = None, inputFormat = None, + outputFormat = None, serde = None, compressed = false, serdeProperties = Map.empty) +} /** * A column in a table. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index abd16f21499e3..7aeec20c4954f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -35,6 +35,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -44,7 +45,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} -import org.apache.spark.sql.execution.command.ExplainCommand +import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand} import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython @@ -2329,8 +2330,14 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @throws[AnalysisException] - def createTempView(viewName: String): Unit = { - sparkSession.createTempView(viewName, toDF(), replaceIfExists = false) + def createTempView(viewName: String): Unit = withPlan { + val tableDesc = CatalogTable( + identifier = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName), + tableType = CatalogTableType.VIEW, + schema = Seq.empty[CatalogColumn], + storage = CatalogStorageFormat.EmptyStorageFormat) + CreateViewCommand(tableDesc, logicalPlan, allowExisting = false, replace = false, + isTemporary = true, sql = "") } /** @@ -2340,8 +2347,14 @@ class Dataset[T] private[sql]( * @group basic * @since 2.0.0 */ - def createOrReplaceTempView(viewName: String): Unit = { - sparkSession.createTempView(viewName, toDF(), replaceIfExists = true) + def createOrReplaceTempView(viewName: String): Unit = withPlan { + val tableDesc = CatalogTable( + identifier = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName), + tableType = CatalogTableType.VIEW, + schema = Seq.empty[CatalogColumn], + storage = CatalogStorageFormat.EmptyStorageFormat) + CreateViewCommand(tableDesc, logicalPlan, allowExisting = false, replace = true, + isTemporary = true, sql = "") } /** 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 66d9aa2c85799..af419fcf95b64 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 @@ -552,7 +552,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * only during the lifetime of this instance of SQLContext. */ private[sql] def registerDataFrameAsTable(df: DataFrame, tableName: String): Unit = { - sparkSession.createTempView(tableName, df, replaceIfExists = true) + df.createOrReplaceTempView(tableName) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index c9276cf140d65..20e22baa351a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -583,17 +583,6 @@ class SparkSession private( Dataset.ofRows(self, sessionState.catalog.lookupRelation(tableIdent)) } - /** - * Creates a temporary view with a DataFrame. The lifetime of this temporary view is tied to - * this [[SparkSession]]. - */ - private[sql] def createTempView( - viewName: String, df: DataFrame, replaceIfExists: Boolean) = { - sessionState.catalog.createTempView( - sessionState.sqlParser.parseTableIdentifier(viewName).table, - df.logicalPlan, replaceIfExists) - } - /* ----------------- * | Everything else | * ----------------- */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index cfebfc6a5ce71..48fb95b519d88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -902,8 +902,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) - .getOrElse(EmptyStorageFormat) - val rowStorage = Option(ctx.rowFormat).map(visitRowFormat).getOrElse(EmptyStorageFormat) + .getOrElse(CatalogStorageFormat.EmptyStorageFormat) + val rowStorage = Option(ctx.rowFormat).map(visitRowFormat) + .getOrElse(CatalogStorageFormat.EmptyStorageFormat) val location = Option(ctx.locationSpec).map(visitLocationSpec) // If we are creating an EXTERNAL table, then the LOCATION field is required if (external && location.isEmpty) { @@ -976,15 +977,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } } - /** Empty storage format for default values and copies. */ - private val EmptyStorageFormat = CatalogStorageFormat(None, None, None, None, false, Map.empty) - /** * Create a [[CatalogStorageFormat]]. */ override def visitTableFileFormat( ctx: TableFileFormatContext): CatalogStorageFormat = withOrigin(ctx) { - EmptyStorageFormat.copy( + CatalogStorageFormat.EmptyStorageFormat.copy( inputFormat = Option(string(ctx.inFmt)), outputFormat = Option(string(ctx.outFmt))) } @@ -997,7 +995,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val source = ctx.identifier.getText HiveSerDe.sourceToSerDe(source, conf) match { case Some(s) => - EmptyStorageFormat.copy( + CatalogStorageFormat.EmptyStorageFormat.copy( inputFormat = s.inputFormat, outputFormat = s.outputFormat, serde = s.serde) @@ -1037,7 +1035,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitRowFormatSerde( ctx: RowFormatSerdeContext): CatalogStorageFormat = withOrigin(ctx) { import ctx._ - EmptyStorageFormat.copy( + CatalogStorageFormat.EmptyStorageFormat.copy( serde = Option(string(name)), serdeProperties = Option(tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } @@ -1067,7 +1065,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx) "line.delim" -> value } - EmptyStorageFormat.copy(serdeProperties = entries.toMap) + CatalogStorageFormat.EmptyStorageFormat.copy(serdeProperties = entries.toMap) } /** @@ -1181,7 +1179,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { identifier = visitTableIdentifier(name), tableType = CatalogTableType.VIEW, schema = schema, - storage = EmptyStorageFormat, + storage = CatalogStorageFormat.EmptyStorageFormat, properties = properties, viewOriginalText = sql, viewText = sql, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 31dc016a01543..b1290a4759a25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -30,8 +30,7 @@ case class CacheTableCommand( override def run(sparkSession: SparkSession): Seq[Row] = { plan.foreach { logicalPlan => - sparkSession.createTempView( - tableName, Dataset.ofRows(sparkSession, logicalPlan), replaceIfExists = true) + Dataset.ofRows(sparkSession, logicalPlan).createOrReplaceTempView(tableName) } sparkSession.catalog.cacheTable(tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 84990119c9c0f..6468916cdcc13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -57,8 +57,12 @@ case class CreateViewCommand( override def output: Seq[Attribute] = Seq.empty[Attribute] - require(tableDesc.tableType == CatalogTableType.VIEW) - require(tableDesc.viewText.isDefined) + require(tableDesc.tableType == CatalogTableType.VIEW, + "The type of the table to created with CREATE VIEW must be 'CatalogTableType.VIEW'.") + if (!isTemporary) { + require(tableDesc.viewText.isDefined, + "The table to created with CREATE VIEW must have 'viewText'.") + } if (allowExisting && replace) { throw new AnalysisException("CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed.") From 2c1b6b58d161dadfab1208e05d4ef549cc2e735c Mon Sep 17 00:00:00 2001 From: Yadong Qi Date: Sat, 28 May 2016 10:19:29 -0700 Subject: [PATCH 0438/1470] [SPARK-15549][SQL] Disable bucketing when the output doesn't contain all bucketing columns ## What changes were proposed in this pull request? I create a bucketed table bucketed_table with bucket column i, ```scala case class Data(i: Int, j: Int, k: Int) sc.makeRDD(Array((1, 2, 3))).map(x => Data(x._1, x._2, x._3)).toDF.write.bucketBy(2, "i").saveAsTable("bucketed_table") ``` and I run the following SQLs: ```sql SELECT j FROM bucketed_table; Error in query: bucket column i not found in existing columns (j); SELECT j, MAX(k) FROM bucketed_table GROUP BY j; Error in query: bucket column i not found in existing columns (j, k); ``` I think we should add a check that, we only enable bucketing when it satisfies all conditions below: 1. the conf is enabled 2. the relation is bucketed 3. the output contains all bucketing columns ## How was this patch tested? Updated test cases to reflect the changes. Author: Yadong Qi Closes #13321 from watermen/SPARK-15549. (cherry picked from commit b4c32c4952f7af2733258aa4e27f21e8832c8a3a) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/ExistingRDD.scala | 13 ++++++------- .../spark/sql/sources/BucketedReadSuite.scala | 11 +++++++++++ 2 files changed, 17 insertions(+), 7 deletions(-) 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 412f5fa87ee7b..fef3255c739ab 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 @@ -347,15 +347,14 @@ private[sql] object DataSourceScanExec { case _ => None } - def toAttribute(colName: String): Attribute = output.find(_.name == colName).getOrElse { - throw new AnalysisException(s"bucket column $colName not found in existing columns " + - s"(${output.map(_.name).mkString(", ")})") - } - bucketSpec.map { spec => val numBuckets = spec.numBuckets - val bucketColumns = spec.bucketColumnNames.map(toAttribute) - HashPartitioning(bucketColumns, numBuckets) + val bucketColumns = spec.bucketColumnNames.flatMap { n => output.find(_.name == n) } + if (bucketColumns.size == spec.bucketColumnNames.size) { + HashPartitioning(bucketColumns, numBuckets) + } else { + UnknownPartitioning(0) + } }.getOrElse { UnknownPartitioning(0) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index f9891ac5717e0..bab0092c37d34 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -362,4 +362,15 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet assert(error.toString contains "Invalid bucket file") } } + + test("disable bucketing when the output doesn't contain all bucketing columns") { + withTable("bucketed_table") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") + + checkAnswer(hiveContext.table("bucketed_table").select("j"), df1.select("j")) + + checkAnswer(hiveContext.table("bucketed_table").groupBy("j").agg(max("k")), + df1.groupBy("j").agg(max("k"))) + } + } } From a2f68ded22d5d8727311fb039714400cbf48156a Mon Sep 17 00:00:00 2001 From: felixcheung Date: Sat, 28 May 2016 10:32:40 -0700 Subject: [PATCH 0439/1470] [SPARK-15637][SPARKR] fix R tests on R 3.2.2 ## What changes were proposed in this pull request? Change version check in R tests ## How was this patch tested? R tests shivaram Author: felixcheung Closes #13369 from felixcheung/rversioncheck. (cherry picked from commit 74c1b79f3f82751d166bccba877501a8cabc9b7c) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_context.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 15915e2d13c29..1d56ced399a9e 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -28,7 +28,7 @@ test_that("Check masked functions", { "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", "summary", "transform", "drop", "window", "as.data.frame") namesOfMaskedCompletely <- c("cov", "filter", "sample") - if (as.numeric(R.version$major) == 3 && as.numeric(R.version$minor) > 2) { + if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) namesOfMaskedCompletely <- c("endsWith", "startsWith", namesOfMaskedCompletely) } From f3570bcea697704f6f10fa62109300ce3cf6b28b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 28 May 2016 14:14:36 -0700 Subject: [PATCH 0440/1470] [SPARK-15636][SQL] Make aggregate expressions more concise in explain ## What changes were proposed in this pull request? This patch reduces the verbosity of aggregate expressions in explain (but does not actually remove any information). As an example, for the following command: ``` spark.range(10).selectExpr("sum(id) + 1", "count(distinct id)").explain(true) ``` Output before this patch: ``` == Physical Plan == *TungstenAggregate(key=[], functions=[(sum(id#0L),mode=Final,isDistinct=false),(count(id#0L),mode=Final,isDistinct=true)], output=[(sum(id) + 1)#3L,count(DISTINCT id)#16L]) +- Exchange SinglePartition, None +- *TungstenAggregate(key=[], functions=[(sum(id#0L),mode=PartialMerge,isDistinct=false),(count(id#0L),mode=Partial,isDistinct=true)], output=[sum#18L,count#21L]) +- *TungstenAggregate(key=[id#0L], functions=[(sum(id#0L),mode=PartialMerge,isDistinct=false)], output=[id#0L,sum#18L]) +- Exchange hashpartitioning(id#0L, 5), None +- *TungstenAggregate(key=[id#0L], functions=[(sum(id#0L),mode=Partial,isDistinct=false)], output=[id#0L,sum#18L]) +- *Range (0, 10, splits=2) ``` Output after this patch: ``` == Physical Plan == *TungstenAggregate(key=[], functions=[sum(id#0L),count(distinct id#0L)], output=[(sum(id) + 1)#3L,count(DISTINCT id)#16L]) +- Exchange SinglePartition, None +- *TungstenAggregate(key=[], functions=[merge_sum(id#0L),partial_count(distinct id#0L)], output=[sum#18L,count#21L]) +- *TungstenAggregate(key=[id#0L], functions=[merge_sum(id#0L)], output=[id#0L,sum#18L]) +- Exchange hashpartitioning(id#0L, 5), None +- *TungstenAggregate(key=[id#0L], functions=[partial_sum(id#0L)], output=[id#0L,sum#18L]) +- *Range (0, 10, splits=2) ``` Note the change from `(sum(id#0L),mode=PartialMerge,isDistinct=false)` to `merge_sum(id#0L)`. In general aggregate explain is still very verbose, but further work will be done as follow-up pull requests. ## How was this patch tested? Tested manually. Author: Reynold Xin Closes #13367 from rxin/SPARK-15636. (cherry picked from commit 472f16181d199684996a156b0e429bc525d65a57) Signed-off-by: Yin Huai --- .../sql/catalyst/expressions/Expression.scala | 2 +- .../expressions/aggregate/interfaces.scala | 15 ++++++++++++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index b4fe151f277a2..2ec46216e1cdb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -185,7 +185,7 @@ abstract class Expression extends TreeNode[Expression] { */ def prettyName: String = nodeName.toLowerCase - private def flatArguments = productIterator.flatMap { + protected def flatArguments = productIterator.flatMap { case t: Traversable[_] => t case single => single :: Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index d31ccf9985360..504cea52797de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -126,7 +126,14 @@ private[sql] case class AggregateExpression( AttributeSet(childReferences) } - override def toString: String = s"($aggregateFunction,mode=$mode,isDistinct=$isDistinct)" + override def toString: String = { + val prefix = mode match { + case Partial => "partial_" + case PartialMerge => "merge_" + case Final | Complete => "" + } + prefix + aggregateFunction.toAggString(isDistinct) + } override def sql: String = aggregateFunction.sql(isDistinct) } @@ -203,6 +210,12 @@ sealed abstract class AggregateFunction extends Expression with ImplicitCastInpu val distinct = if (isDistinct) "DISTINCT " else "" s"$prettyName($distinct${children.map(_.sql).mkString(", ")})" } + + /** String representation used in explain plans. */ + def toAggString(isDistinct: Boolean): String = { + val start = if (isDistinct) "(distinct " else "(" + prettyName + flatArguments.mkString(start, ", ", ")") + } } /** From be2d23dfd97ae8b04c4960784e1223a54fd736bc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 29 May 2016 16:48:14 -0500 Subject: [PATCH 0441/1470] [MINOR] Resolve a number of miscellaneous build warnings This change resolves a number of build warnings that have accumulated, before 2.x. It does not address a large number of deprecation warnings, especially related to the Accumulator API. That will happen separately. Jenkins Author: Sean Owen Closes #13377 from srowen/BuildWarnings. (cherry picked from commit ce1572d16f03d383071bcc1f30ede551e8ded49f) Signed-off-by: Sean Owen --- .../src/main/java/org/apache/spark/unsafe/Platform.java | 4 ++-- .../java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java | 4 +++- .../scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala | 2 ++ .../org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala | 2 +- .../spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala | 3 ++- sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala | 2 ++ .../spark/sql/execution/datasources/FileCatalogSuite.scala | 2 ++ .../apache/spark/network/yarn/YarnShuffleServiceSuite.scala | 1 + 8 files changed, 15 insertions(+), 5 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index bdf52f32c6fe1..77c8c398be955 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -155,8 +155,8 @@ public static long reallocateMemory(long address, long oldSize, long newSize) { @SuppressWarnings("unchecked") public static ByteBuffer allocateDirectBuffer(int size) { try { - Class cls = Class.forName("java.nio.DirectByteBuffer"); - Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); + Class cls = Class.forName("java.nio.DirectByteBuffer"); + Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); constructor.setAccessible(true); Field cleanerField = cls.getDeclaredField("cleaner"); cleanerField.setAccessible(true); diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java index 75b0ec6480196..32d3141149a74 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java @@ -72,7 +72,9 @@ public void runPrefixSpanSaveLoad() { try { model.save(spark.sparkContext(), outputPath); - PrefixSpanModel newModel = PrefixSpanModel.load(spark.sparkContext(), outputPath); + @SuppressWarnings("unchecked") + PrefixSpanModel newModel = + (PrefixSpanModel) PrefixSpanModel.load(spark.sparkContext(), outputPath); JavaRDD> freqSeqs = newModel.freqSequences().toJavaRDD(); List> localFreqSeqs = freqSeqs.collect(); Assert.assertEquals(5, localFreqSeqs.size()); diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala index 6d8c7b47d8373..4c2376376dd2a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.mllib.fpm +import scala.language.existentials + import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.Utils 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 1b25f5d7d0cc5..85563ddedc165 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 @@ -288,7 +288,7 @@ class ScalaReflectionSuite extends SparkFunSuite { assert(serializer.children.head.asInstanceOf[Literal].value === UTF8String.fromString("value")) assert(serializer.children.last.isInstanceOf[NewInstance]) assert(serializer.children.last.asInstanceOf[NewInstance] - .cls.isInstanceOf[Class[org.apache.spark.sql.catalyst.util.GenericArrayData]]) + .cls.isAssignableFrom(classOf[org.apache.spark.sql.catalyst.util.GenericArrayData])) } private val dataTypeForComplexData = dataTypeFor[ComplexData] diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 3d97113b52e39..232dcc9ee51ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -365,7 +365,8 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { Arrays.deepEquals(b1.asInstanceOf[Array[AnyRef]], b2.asInstanceOf[Array[AnyRef]]) case (b1: Array[_], b2: Array[_]) => Arrays.equals(b1.asInstanceOf[Array[AnyRef]], b2.asInstanceOf[Array[AnyRef]]) - case (left: Comparable[Any], right: Comparable[Any]) => left.compareTo(right) == 0 + case (left: Comparable[_], right: Comparable[_]) => + left.asInstanceOf[Comparable[Any]].compareTo(right) == 0 case _ => input == convertedBack } 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 cbf4a8a612594..44889d92ee306 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql +import scala.language.existentials + import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.joins._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 85c2e8ba5529d..0d9ea512729bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources import java.io.File +import scala.language.reflectiveCalls + import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.util._ diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 0e433f6c1b0be..c33a9e6bbe255 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.network.yarn import java.io.{DataOutputStream, File, FileOutputStream} import scala.annotation.tailrec +import scala.language.postfixOps import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.api.records.ApplicationId From 6dae027a6cdd7c862963f71e1ea08f7f1b4b3506 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 29 May 2016 23:19:12 -0700 Subject: [PATCH 0442/1470] [SPARK-15112][SQL] Disables EmbedSerializerInFilter for plan fragments that change schema ## What changes were proposed in this pull request? `EmbedSerializerInFilter` implicitly assumes that the plan fragment being optimized doesn't change plan schema, which is reasonable because `Dataset.filter` should never change the schema. However, due to another issue involving `DeserializeToObject` and `SerializeFromObject`, typed filter *does* change plan schema (see [SPARK-15632][1]). This breaks `EmbedSerializerInFilter` and causes corrupted data. This PR disables `EmbedSerializerInFilter` when there's a schema change to avoid data corruption. The schema change issue should be addressed in follow-up PRs. ## How was this patch tested? New test case added in `DatasetSuite`. [1]: https://issues.apache.org/jira/browse/SPARK-15632 Author: Cheng Lian Closes #13362 from liancheng/spark-15112-corrupted-filter. (cherry picked from commit 1360a6d636dd812a27955fc85df8e0255db60dfa) Signed-off-by: Cheng Lian --- .../sql/catalyst/optimizer/Optimizer.scala | 21 ++++++++++++++++++- .../org/apache/spark/sql/DatasetSuite.scala | 16 +++++++++++++- 2 files changed, 35 insertions(+), 2 deletions(-) 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 48d70099b6d88..688c77d3ca64f 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 @@ -1597,7 +1597,19 @@ case class GetCurrentDatabase(sessionCatalog: SessionCatalog) extends Rule[Logic */ object EmbedSerializerInFilter extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) => + case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) + // SPARK-15632: Conceptually, filter operator should never introduce schema change. This + // optimization rule also relies on this assumption. However, Dataset typed filter operator + // does introduce schema changes in some cases. Thus, we only enable this optimization when + // + // 1. either input and output schemata are exactly the same, or + // 2. both input and output schemata are single-field schema and share the same type. + // + // The 2nd case is included because encoders for primitive types always have only a single + // field with hard-coded field name "value". + // TODO Cleans this up after fixing SPARK-15632. + if s.schema == d.child.schema || samePrimitiveType(s.schema, d.child.schema) => + val numObjects = condition.collect { case a: Attribute if a == d.output.head => a }.length @@ -1622,6 +1634,13 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] { Project(objAttrs, filter) } } + + def samePrimitiveType(lhs: StructType, rhs: StructType): Boolean = { + (lhs, rhs) match { + case (StructType(Array(f1)), StructType(Array(f2))) => f1.dataType == f2.dataType + case _ => false + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index e395007999592..8fc4dc9f17466 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -706,7 +706,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val dataset = Seq(1, 2, 3).toDS() dataset.createOrReplaceTempView("tempView") - // Overrrides the existing temporary view with same name + // Overrides the existing temporary view with same name // No exception should be thrown here. dataset.createOrReplaceTempView("tempView") @@ -769,6 +769,20 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkShowString(ds, expected) } + + test( + "SPARK-15112: EmbedDeserializerInFilter should not optimize plan fragment that changes schema" + ) { + val ds = Seq(1 -> "foo", 2 -> "bar").toDF("b", "a").as[ClassData] + + assertResult(Seq(ClassData("foo", 1), ClassData("bar", 2))) { + ds.collect().toSeq + } + + assertResult(Seq(ClassData("bar", 2))) { + ds.filter(_.b > 1).collect().toSeq + } + } } case class Generic[T](id: T, value: Double) From 7076b3707b530e19a8e8353dd8a4fd42dd148eab Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Mon, 30 May 2016 08:40:03 -0500 Subject: [PATCH 0443/1470] [SPARK-15645][STREAMING] Fix some typos of Streaming module ## What changes were proposed in this pull request? No code change, just some typo fixing. ## How was this patch tested? Manually run project build with testing, and build is successful. Author: Xin Ren Closes #13385 from keypointt/codeWalkThroughStreaming. (cherry picked from commit 5728aa558e44f056f3e5a7f8726ab174d3830103) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/rdd/ParallelCollectionRDD.scala | 4 ++-- .../apache/spark/examples/streaming/CustomReceiver.scala | 2 +- .../org/apache/spark/examples/streaming/QueueStream.scala | 2 +- .../scala/org/apache/spark/streaming/dstream/DStream.scala | 2 +- .../apache/spark/streaming/receiver/BlockGenerator.scala | 6 +++--- .../org/apache/spark/streaming/receiver/Receiver.scala | 6 +++--- .../org/apache/spark/streaming/scheduler/JobGenerator.scala | 3 +-- .../apache/spark/streaming/scheduler/ReceiverTracker.scala | 2 +- 8 files changed, 13 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 34a1c112cbcd0..e9092739b298a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -32,8 +32,8 @@ import org.apache.spark.util.Utils private[spark] class ParallelCollectionPartition[T: ClassTag]( var rddId: Long, var slice: Int, - var values: Seq[T]) - extends Partition with Serializable { + var values: Seq[T] + ) extends Partition with Serializable { def iterator: Iterator[T] = values.iterator diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index f70975eeb5c99..43044d01b1204 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.receiver.Receiver /** - * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * Custom Receiver that receives data over a socket. Received bytes are interpreted as * text and \n delimited lines are considered as records. They are then counted and printed. * * To run this on your local machine, you need to first run a Netcat server diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index 5455aed22085d..19bacd449787b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -43,7 +43,7 @@ object QueueStream { reducedStream.print() ssc.start() - // Create and push some RDDs into + // Create and push some RDDs into rddQueue for (i <- 1 to 30) { rddQueue.synchronized { rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 01dcfcf24b0fe..147e8c129034b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -52,7 +52,7 @@ import org.apache.spark.util.{CallSite, Utils} * `join`. These operations are automatically available on any DStream of pairs * (e.g., DStream[(Int, Int)] through implicit conversions. * - * DStreams internally is characterized by a few basic properties: + * A DStream internally is characterized by a few basic properties: * - A list of other DStreams that the DStream depends on * - A time interval at which the DStream generates an RDD * - A function that is used to generate an RDD after each time interval 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 4592e015ed9a0..90309c0145ae1 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 @@ -86,13 +86,13 @@ private[streaming] class BlockGenerator( /** * The BlockGenerator can be in 5 possible states, in the order as follows. * - * - Initialized: Nothing has been started + * - Initialized: Nothing has been started. * - Active: start() has been called, and it is generating blocks on added data. * - StoppedAddingData: stop() has been called, the adding of data has been stopped, * but blocks are still being generated and pushed. * - StoppedGeneratingBlocks: Generating of blocks has been stopped, but * they are still being pushed. - * - StoppedAll: Everything has stopped, and the BlockGenerator object can be GCed. + * - StoppedAll: Everything has been stopped, and the BlockGenerator object can be GCed. */ private object GeneratorState extends Enumeration { type GeneratorState = Value @@ -148,7 +148,7 @@ private[streaming] class BlockGenerator( blockIntervalTimer.stop(interruptTimer = false) synchronized { state = StoppedGeneratingBlocks } - // Wait for the queue to drain and mark generated as stopped + // Wait for the queue to drain and mark state as StoppedAll logInfo("Waiting for block pushing thread to terminate") blockPushingThread.join() synchronized { state = StoppedAll } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 5157ca62dc449..d91a64df321a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -32,7 +32,7 @@ import org.apache.spark.storage.StorageLevel * should define the setup steps necessary to start receiving data, * and `onStop()` should define the cleanup steps necessary to stop receiving data. * Exceptions while receiving can be handled either by restarting the receiver with `restart(...)` - * or stopped completely by `stop(...)` or + * or stopped completely by `stop(...)`. * * A custom receiver in Scala would look like this. * @@ -45,7 +45,7 @@ import org.apache.spark.storage.StorageLevel * // Call store(...) in those threads to store received data into Spark's memory. * * // Call stop(...), restart(...) or reportError(...) on any thread based on how - * // different errors needs to be handled. + * // different errors need to be handled. * * // See corresponding method documentation for more details * } @@ -71,7 +71,7 @@ import org.apache.spark.storage.StorageLevel * // Call store(...) in those threads to store received data into Spark's memory. * * // Call stop(...), restart(...) or reportError(...) on any thread based on how - * // different errors needs to be handled. + * // different errors need to be handled. * * // See corresponding method documentation for more details * } 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 8f9421fc098ba..19c88f1ee0114 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 @@ -19,7 +19,6 @@ package org.apache.spark.streaming.scheduler import scala.util.{Failure, Success, Try} -import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} @@ -239,7 +238,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Restarted JobGenerator at " + restartTime) } - /** Generate jobs and perform checkpoint for the given `time`. */ + /** Generate jobs and perform checkpointing for the given `time`. */ private def generateJobs(time: Time) { // Checkpoint all RDDs marked for checkpointing to ensure their lineages are // truncated periodically. Otherwise, we may run into stack overflows (SPARK-6847). diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 9aa2f0bbb9952..b9d898a72362e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.scheduler import java.util.concurrent.{CountDownLatch, TimeUnit} import scala.collection.mutable.HashMap -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.ExecutionContext import scala.language.existentials import scala.util.{Failure, Success} From f208a9dcd5814077cd231fdbc4e49257b9169c1f Mon Sep 17 00:00:00 2001 From: Matthew Wise Date: Mon, 30 May 2016 09:12:02 -0500 Subject: [PATCH 0444/1470] [DOCS] fix example code issues in documentation ## What changes were proposed in this pull request? Fixed broken java code examples in streaming documentation Attn: tdas Author: Matthew Wise Closes #13388 from mawise/fix_docs_java_streaming_example. (cherry picked from commit 2d34183b273af1125181f04c49725efc2fa351af) Signed-off-by: Sean Owen --- docs/streaming-kafka-integration.md | 2 +- docs/streaming-programming-guide.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 0f1e32212eb40..e0d3f4f69be8f 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -111,7 +111,7 @@ Next, we discuss how to use this approach in your streaming application.
    import org.apache.spark.streaming.kafka.*; - JavaPairReceiverInputDStream directKafkaStream = + JavaPairInputDStream directKafkaStream = KafkaUtils.createDirectStream(streamingContext, [key class], [value class], [key decoder class], [value decoder class], [map of Kafka parameters], [set of topics to consume]); diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index d7eafff38f35b..6550fcc0521c3 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -145,8 +145,8 @@ import org.apache.spark.streaming.api.java.*; import scala.Tuple2; // Create a local StreamingContext with two working thread and batch interval of 1 second -SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") -JavaStreamingContext jssc = new JavaStreamingContext(conf, Durations.seconds(1)) +SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount"); +JavaStreamingContext jssc = new JavaStreamingContext(conf, Durations.seconds(1)); {% endhighlight %} Using this context, we can create a DStream that represents streaming data from a TCP From 2b35ce818650d91a25a14cb40121ae22521de4b5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 30 May 2016 22:47:58 -0700 Subject: [PATCH 0445/1470] [SPARK-15638][SQL] Audit Dataset, SparkSession, and SQLContext ## What changes were proposed in this pull request? This patch contains a list of changes as a result of my auditing Dataset, SparkSession, and SQLContext. The patch audits the categorization of experimental APIs, function groups, and deprecations. For the detailed list of changes, please see the diff. ## How was this patch tested? N/A Author: Reynold Xin Closes #13370 from rxin/SPARK-15638. (cherry picked from commit 675921040ee4802aa9914457de62af746bc3657d) Signed-off-by: Reynold Xin --- .../spark/memory/TaskMemoryManager.java | 2 +- .../spark/scheduler/SchedulableBuilder.scala | 3 +- .../apache/spark/storage/BlockManager.scala | 6 +-- .../spark/storage/BlockManagerMaster.scala | 4 +- .../org/apache/spark/util/ListenerBus.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../sql/catalyst/catalog/interface.scala | 2 +- .../scala/org/apache/spark/sql/Column.scala | 35 +++++++-------- .../scala/org/apache/spark/sql/Dataset.scala | 45 +++++++++---------- .../spark/sql/KeyValueGroupedDataset.scala | 8 ++-- .../spark/sql/RelationalGroupedDataset.scala | 4 +- .../org/apache/spark/sql/SQLContext.scala | 18 +------- .../spark/sql/execution/SparkSqlParser.scala | 16 +++---- .../spark/sql/execution/command/views.scala | 4 +- .../apache/spark/sql/SQLContextSuite.scala | 1 + .../spark/sql/hive/HiveDDLCommandSuite.scala | 8 ++-- 16 files changed, 69 insertions(+), 93 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index a4a571f15a8c0..867c4a1050677 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -182,7 +182,7 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { } consumers.add(consumer); - logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); + logger.debug("Task {} acquired {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); return got; } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 100ed76ecb6d6..96325a0329f89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -112,7 +112,8 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) schedulingMode = SchedulingMode.withName(xmlSchedulingMode) } catch { case e: NoSuchElementException => - logWarning("Error xml schedulingMode, using default schedulingMode") + logWarning(s"Unsupported schedulingMode: $xmlSchedulingMode, " + + s"using the default schedulingMode: $schedulingMode") } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c56e451c11cb5..2f9473aedc2dc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -92,9 +92,9 @@ private[spark] class BlockManager( private[spark] val diskStore = new DiskStore(conf, diskBlockManager) memoryManager.setMemoryStore(memoryStore) - // Note: depending on the memory manager, `maxStorageMemory` may actually vary over time. + // Note: depending on the memory manager, `maxMemory` may actually vary over time. // However, since we use this only for reporting and logging, what we actually want here is - // the absolute maximum value that `maxStorageMemory` can ever possibly reach. We may need + // the absolute maximum value that `maxMemory` can ever possibly reach. We may need // to revisit whether reporting this value as the "max" is intuitive to the user. private val maxMemory = memoryManager.maxOnHeapStorageMemory @@ -231,7 +231,7 @@ private[spark] class BlockManager( */ def reregister(): Unit = { // TODO: We might need to rate limit re-registering. - logInfo("BlockManager re-registering with master") + logInfo(s"BlockManager $blockManagerId re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) reportAllBlocks() } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index c22d2e0fb61fa..52db45bd488b9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -45,9 +45,9 @@ class BlockManagerMaster( /** Register the BlockManager's id with the driver. */ def registerBlockManager( blockManagerId: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef): Unit = { - logInfo("Trying to register BlockManager") + logInfo(s"Registering BlockManager $blockManagerId") tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint)) - logInfo("Registered BlockManager") + logInfo(s"Registered BlockManager $blockManagerId") } def updateBlockInfo( 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 436c1951dee2f..79fc2e94599c7 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -54,7 +54,7 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { */ final def postToAll(event: E): Unit = { // JavaConverters can create a JIterableWrapper if we use asScala. - // However, this method will be called frequently. To avoid the wrapper cost, here ewe use + // However, this method will be called frequently. To avoid the wrapper cost, here we use // Java Iterator directly. val iter = listeners.iterator while (iter.hasNext) { @@ -70,7 +70,7 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { /** * Post an event to the specified listener. `onPostEvent` is guaranteed to be called in the same - * thread. + * thread for all listeners. */ protected def doPostEvent(listener: L, event: E): Unit 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 a8bb0002a7b25..7e204fa21852c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1169,7 +1169,7 @@ private[spark] object Utils extends Logging { } /** - * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * Execute a block of code that evaluates to Unit, stop SparkContext if there is any uncaught * exception * * NOTE: This method is to be called by the driver-side components to avoid stopping the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 77731b1f26eed..b59195770c17e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -52,7 +52,7 @@ case class CatalogStorageFormat( object CatalogStorageFormat { /** Empty storage format for default values and copies. */ - val EmptyStorageFormat = CatalogStorageFormat(locationUri = None, inputFormat = None, + val empty = CatalogStorageFormat(locationUri = None, inputFormat = None, outputFormat = None, serde = None, compressed = false, serdeProperties = Map.empty) } 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 204af719b2c59..713f7941beeb2 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 @@ -92,7 +92,6 @@ class TypedColumn[-T, U]( } /** - * :: Experimental :: * A column that will be computed based on the data in a [[DataFrame]]. * * A new column is constructed based on the input columns present in a dataframe: @@ -121,7 +120,6 @@ class TypedColumn[-T, U]( * * @since 1.3.0 */ -@Experimental class Column(protected[sql] val expr: Expression) extends Logging { def this(name: String) = this(name match { @@ -132,6 +130,15 @@ class Column(protected[sql] val expr: Expression) extends Logging { case _ => UnresolvedAttribute.quotedString(name) }) + override def toString: String = usePrettyExpression(expr).sql + + override def equals(that: Any): Boolean = that match { + case that: Column => that.expr.equals(this.expr) + case _ => false + } + + override def hashCode: Int = this.expr.hashCode() + /** Creates a column based on the given expression. */ private def withExpr(newExpr: Expression): Column = new Column(newExpr) @@ -158,12 +165,13 @@ class Column(protected[sql] val expr: Expression) extends Logging { // If we have a top level Cast, there is a chance to give it a better alias, if there is a // NamedExpression under this Cast. - case c: Cast => c.transformUp { - case Cast(ne: NamedExpression, to) => UnresolvedAlias(Cast(ne, to)) - } match { - case ne: NamedExpression => ne - case other => Alias(expr, usePrettyExpression(expr).sql)() - } + case c: Cast => + c.transformUp { + case Cast(ne: NamedExpression, to) => UnresolvedAlias(Cast(ne, to)) + } match { + case ne: NamedExpression => ne + case other => Alias(expr, usePrettyExpression(expr).sql)() + } case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => UnresolvedAlias(a, Some(Column.generateAlias)) @@ -171,17 +179,6 @@ class Column(protected[sql] val expr: Expression) extends Logging { case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } - - - override def toString: String = usePrettyExpression(expr).sql - - override def equals(that: Any): Boolean = that match { - case that: Column => that.expr.equals(this.expr) - case _ => false - } - - override def hashCode: Int = this.expr.hashCode - /** * Provides a type hint about the expected return value of this column. This information can * be used by operations such as `select` on a [[Dataset]] to automatically convert the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 7aeec20c4954f..8b6662ab1fae6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -146,11 +146,8 @@ private[sql] object Dataset { * * @groupname basic Basic Dataset functions * @groupname action Actions - * @groupname untypedrel Untyped Language Integrated Relational Queries - * @groupname typedrel Typed Language Integrated Relational Queries - * @groupname func Functional Transformations - * @groupname rdd RDD Operations - * @groupname output Output Operations + * @groupname untypedrel Untyped transformations + * @groupname typedrel Typed transformations * * @since 1.6.0 */ @@ -1892,7 +1889,7 @@ class Dataset[T] private[sql]( * .transform(...) * }}} * - * @group func + * @group typedrel * @since 1.6.0 */ def transform[U](t: Dataset[T] => Dataset[U]): Dataset[U] = t(this) @@ -1902,7 +1899,7 @@ class Dataset[T] private[sql]( * (Scala-specific) * Returns a new [[Dataset]] that only contains elements where `func` returns `true`. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -1919,7 +1916,7 @@ class Dataset[T] private[sql]( * (Java-specific) * Returns a new [[Dataset]] that only contains elements where `func` returns `true`. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -1936,7 +1933,7 @@ class Dataset[T] private[sql]( * (Scala-specific) * Returns a new [[Dataset]] that contains the result of applying `func` to each element. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -1949,7 +1946,7 @@ class Dataset[T] private[sql]( * (Java-specific) * Returns a new [[Dataset]] that contains the result of applying `func` to each element. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -1963,7 +1960,7 @@ class Dataset[T] private[sql]( * (Scala-specific) * Returns a new [[Dataset]] that contains the result of applying `func` to each partition. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -1979,7 +1976,7 @@ class Dataset[T] private[sql]( * (Java-specific) * Returns a new [[Dataset]] that contains the result of applying `f` to each partition. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -1991,8 +1988,6 @@ class Dataset[T] private[sql]( /** * Returns a new [[DataFrame]] that contains the result of applying a serialized R function * `func` to each partition. - * - * @group func */ private[sql] def mapPartitionsInR( func: Array[Byte], @@ -2011,7 +2006,7 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] by first applying a function to all elements of this [[Dataset]], * and then flattening the results. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -2024,7 +2019,7 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] by first applying a function to all elements of this [[Dataset]], * and then flattening the results. * - * @group func + * @group typedrel * @since 1.6.0 */ @Experimental @@ -2207,7 +2202,7 @@ class Dataset[T] private[sql]( * 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 + * @group typedrel * @since 1.6.0 */ def coalesce(numPartitions: Int): Dataset[T] = withTypedPlan { @@ -2283,7 +2278,7 @@ class Dataset[T] private[sql]( /** * Represents the content of the [[Dataset]] as an [[RDD]] of [[T]]. * - * @group rdd + * @group basic * @since 1.6.0 */ lazy val rdd: RDD[T] = { @@ -2296,14 +2291,14 @@ class Dataset[T] private[sql]( /** * Returns the content of the [[Dataset]] as a [[JavaRDD]] of [[Row]]s. - * @group rdd + * @group basic * @since 1.6.0 */ def toJavaRDD: JavaRDD[T] = rdd.toJavaRDD() /** * Returns the content of the [[Dataset]] as a [[JavaRDD]] of [[Row]]s. - * @group rdd + * @group basic * @since 1.6.0 */ def javaRDD: JavaRDD[T] = toJavaRDD @@ -2335,9 +2330,9 @@ class Dataset[T] private[sql]( identifier = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName), tableType = CatalogTableType.VIEW, schema = Seq.empty[CatalogColumn], - storage = CatalogStorageFormat.EmptyStorageFormat) + storage = CatalogStorageFormat.empty) CreateViewCommand(tableDesc, logicalPlan, allowExisting = false, replace = false, - isTemporary = true, sql = "") + isTemporary = true) } /** @@ -2352,16 +2347,16 @@ class Dataset[T] private[sql]( identifier = sparkSession.sessionState.sqlParser.parseTableIdentifier(viewName), tableType = CatalogTableType.VIEW, schema = Seq.empty[CatalogColumn], - storage = CatalogStorageFormat.EmptyStorageFormat) + storage = CatalogStorageFormat.empty) CreateViewCommand(tableDesc, logicalPlan, allowExisting = false, replace = true, - isTemporary = true, sql = "") + isTemporary = true) } /** * :: Experimental :: * Interface for saving the content of the [[Dataset]] out into external storage or streams. * - * @group output + * @group basic * @since 1.6.0 */ @Experimental diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 3a5ea19b8ad14..53f4ea647c853 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -29,8 +29,8 @@ import org.apache.spark.sql.execution.QueryExecution /** * :: Experimental :: * A [[Dataset]] has been logically grouped by a user specified grouping key. Users should not - * construct a [[KeyValueGroupedDataset]] directly, but should instead call `groupBy` on an existing - * [[Dataset]]. + * construct a [[KeyValueGroupedDataset]] directly, but should instead call `groupByKey` on + * an existing [[Dataset]]. * * @since 2.0.0 */ @@ -73,7 +73,8 @@ class KeyValueGroupedDataset[K, V] private[sql]( groupingAttributes) /** - * Returns a [[Dataset]] that contains each unique key. + * Returns a [[Dataset]] that contains each unique key. This is equivalent to doing mapping + * over the Dataset to extract the keys and then running a distinct operation on those. * * @since 1.6.0 */ @@ -204,7 +205,6 @@ class KeyValueGroupedDataset[K, V] private[sql]( * Internal helper function for building typed aggregations that return tuples. For simplicity * and code reuse, we do this without the help of the type system and then use helper functions * that cast appropriately for the user facing interface. - * TODO: does not handle aggregations that return nonflat results, */ protected def aggUntyped(columns: TypedColumn[_, _]*): Dataset[_] = { val encoders = columns.map(_.encoder) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index b0e48a6553a45..58850a7d4b49f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -35,6 +35,8 @@ import org.apache.spark.sql.types.NumericType * The main method is the agg function, which has multiple variants. This class also contains * convenience some first order statistics such as mean, sum for convenience. * + * This class was named `GroupedData` in Spark 1.x. + * * @since 2.0.0 */ class RelationalGroupedDataset protected[sql]( @@ -74,7 +76,7 @@ class RelationalGroupedDataset protected[sql]( private[this] def alias(expr: Expression): NamedExpression = expr match { case u: UnresolvedAttribute => UnresolvedAlias(u) case expr: NamedExpression => expr - case a: AggregateExpression if (a.aggregateFunction.isInstanceOf[TypedAggregateExpression]) => + case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => UnresolvedAlias(a, Some(Column.generateAlias)) case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } 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 af419fcf95b64..0dc70c0b1c7fe 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 @@ -41,7 +41,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ExecutionListenerManager /** - * The entry point for working with structured data (rows and columns) in Spark, in Spark 1.x. + * The entry point for working with structured data (rows and columns) in Spark 1.x. * * As of Spark 2.0, this is replaced by [[SparkSession]]. However, we are keeping the class * here for backward compatibility. @@ -171,13 +171,11 @@ class SQLContext private[sql](val sparkSession: SparkSession) def experimental: ExperimentalMethods = sparkSession.experimental /** - * :: Experimental :: * Returns a [[DataFrame]] with no rows or columns. * * @group basic * @since 1.3.0 */ - @Experimental def emptyDataFrame: DataFrame = sparkSession.emptyDataFrame /** @@ -437,7 +435,6 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * :: Experimental :: * Returns a [[DataFrameReader]] that can be used to read data and streams in as a [[DataFrame]]. * {{{ * sqlContext.read.parquet("/path/to/file.parquet") @@ -447,31 +444,26 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group genericdata * @since 1.4.0 */ - @Experimental def read: DataFrameReader = sparkSession.read /** - * :: Experimental :: * Creates an external table from the given path and returns the corresponding DataFrame. * It will use the default data source configured by spark.sql.sources.default. * * @group ddl_ops * @since 1.3.0 */ - @Experimental def createExternalTable(tableName: String, path: String): DataFrame = { sparkSession.catalog.createExternalTable(tableName, path) } /** - * :: Experimental :: * Creates an external table from the given path based on a data source * and returns the corresponding DataFrame. * * @group ddl_ops * @since 1.3.0 */ - @Experimental def createExternalTable( tableName: String, path: String, @@ -480,14 +472,12 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * :: Experimental :: * Creates an external table from the given path based on a data source and a set of options. * Then, returns the corresponding DataFrame. * * @group ddl_ops * @since 1.3.0 */ - @Experimental def createExternalTable( tableName: String, source: String, @@ -496,7 +486,6 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * :: Experimental :: * (Scala-specific) * Creates an external table from the given path based on a data source and a set of options. * Then, returns the corresponding DataFrame. @@ -504,7 +493,6 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group ddl_ops * @since 1.3.0 */ - @Experimental def createExternalTable( tableName: String, source: String, @@ -513,14 +501,12 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * :: Experimental :: * Create an external table from the given path based on a data source, a schema and * a set of options. Then, returns the corresponding DataFrame. * * @group ddl_ops * @since 1.3.0 */ - @Experimental def createExternalTable( tableName: String, source: String, @@ -530,7 +516,6 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * :: Experimental :: * (Scala-specific) * Create an external table from the given path based on a data source, a schema and * a set of options. Then, returns the corresponding DataFrame. @@ -538,7 +523,6 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group ddl_ops * @since 1.3.0 */ - @Experimental def createExternalTable( tableName: String, source: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 48fb95b519d88..6c19bf02dc11e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -902,9 +902,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) - .getOrElse(CatalogStorageFormat.EmptyStorageFormat) + .getOrElse(CatalogStorageFormat.empty) val rowStorage = Option(ctx.rowFormat).map(visitRowFormat) - .getOrElse(CatalogStorageFormat.EmptyStorageFormat) + .getOrElse(CatalogStorageFormat.empty) val location = Option(ctx.locationSpec).map(visitLocationSpec) // If we are creating an EXTERNAL table, then the LOCATION field is required if (external && location.isEmpty) { @@ -982,7 +982,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitTableFileFormat( ctx: TableFileFormatContext): CatalogStorageFormat = withOrigin(ctx) { - CatalogStorageFormat.EmptyStorageFormat.copy( + CatalogStorageFormat.empty.copy( inputFormat = Option(string(ctx.inFmt)), outputFormat = Option(string(ctx.outFmt))) } @@ -995,7 +995,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val source = ctx.identifier.getText HiveSerDe.sourceToSerDe(source, conf) match { case Some(s) => - CatalogStorageFormat.EmptyStorageFormat.copy( + CatalogStorageFormat.empty.copy( inputFormat = s.inputFormat, outputFormat = s.outputFormat, serde = s.serde) @@ -1035,7 +1035,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitRowFormatSerde( ctx: RowFormatSerdeContext): CatalogStorageFormat = withOrigin(ctx) { import ctx._ - CatalogStorageFormat.EmptyStorageFormat.copy( + CatalogStorageFormat.empty.copy( serde = Option(string(name)), serdeProperties = Option(tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } @@ -1065,7 +1065,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx) "line.delim" -> value } - CatalogStorageFormat.EmptyStorageFormat.copy(serdeProperties = entries.toMap) + CatalogStorageFormat.empty.copy(serdeProperties = entries.toMap) } /** @@ -1179,12 +1179,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { identifier = visitTableIdentifier(name), tableType = CatalogTableType.VIEW, schema = schema, - storage = CatalogStorageFormat.EmptyStorageFormat, + storage = CatalogStorageFormat.empty, properties = properties, viewOriginalText = sql, viewText = sql, comment = comment) - CreateViewCommand(tableDesc, plan(query), allowExist, replace, isTemporary, command(ctx)) + CreateViewCommand(tableDesc, plan(query), allowExist, replace, isTemporary) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 6468916cdcc13..20c02786ecc5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -41,15 +41,13 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} * at the end of current Spark session. Existing permanent relations with the same * name are not visible to the current session while the temporary view exists, * unless they are specified with full qualified table name with database prefix. - * @param sql the original sql */ case class CreateViewCommand( tableDesc: CatalogTable, child: LogicalPlan, allowExisting: Boolean, replace: Boolean, - isTemporary: Boolean, - sql: String) + isTemporary: Boolean) extends RunnableCommand { // TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 417d09e238c62..e57c1716a5bfd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} +@deprecated("This suite is deprecated to silent compiler deprecation warnings", "2.0.0") class SQLContextSuite extends SparkFunSuite with SharedSparkContext { object DummyRule extends Rule[LogicalPlan] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 96c8fa6b70501..3297a6f6c3461 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.hive -import org.apache.hadoop.hive.serde.serdeConstants - import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} @@ -37,9 +35,9 @@ class HiveDDLCommandSuite extends PlanTest { private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { parser.parsePlan(sql).collect { - case CreateTableCommand(desc, allowExisting) => (desc, allowExisting) - case CreateTableAsSelectLogicalPlan(desc, _, allowExisting) => (desc, allowExisting) - case CreateViewCommand(desc, _, allowExisting, _, _, _) => (desc, allowExisting) + case c: CreateTableCommand => (c.table, c.ifNotExists) + case c: CreateTableAsSelectLogicalPlan => (c.tableDesc, c.allowExisting) + case c: CreateViewCommand => (c.tableDesc, c.allowExisting) }.head } From 3d00c7015d0c73b9673d286cb230c17889922470 Mon Sep 17 00:00:00 2001 From: catapan Date: Tue, 31 May 2016 06:55:07 -0500 Subject: [PATCH 0446/1470] [SPARK-15641] HistoryServer to not show invalid date for incomplete application ## What changes were proposed in this pull request? For incomplete applications in HistoryServer, the complete column will show "-" instead of incorrect date. ## How was this patch tested? manually tested. Author: catapan Author: Ziying Pan Closes #13396 from catapan/SPARK-15641_fix_completed_column. (cherry picked from commit 6878f3e2eafe0bd2e19a14ddffe6e0c1e545a42d) Signed-off-by: Sean Owen --- .../main/resources/org/apache/spark/ui/static/historypage.js | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index ef89a9a86f093..d2161662d5679 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -54,7 +54,8 @@ function makeIdNumeric(id) { } function formatDate(date) { - return date.split(".")[0].replace("T", " "); + if (date <= 0) return "-"; + else return date.split(".")[0].replace("T", " "); } function getParameterByName(name, searchString) { From 2e3ead20cdf54b7cefd5e53e86c8e5a7984f157b Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Tue, 31 May 2016 07:25:16 -0500 Subject: [PATCH 0447/1470] [SPARK-15528][SQL] Fix race condition in NumberConverter ## What changes were proposed in this pull request? A local variable in NumberConverter is wrongly shared between threads. This pr fixes the race condition. ## How was this patch tested? Manually checked. Author: Takeshi YAMAMURO Closes #13391 from maropu/SPARK-15528. (cherry picked from commit 95db8a44f3e2d79913cbe0d29297796b4c3b0d1b) Signed-off-by: Sean Owen --- .../sql/catalyst/util/NumberConverter.scala | 38 +++++++++---------- 1 file changed, 17 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index da90ddbd63afb..9c3f6b7c5d245 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -21,8 +21,6 @@ import org.apache.spark.unsafe.types.UTF8String object NumberConverter { - private val value = new Array[Byte](64) - /** * Divide x by m as if x is an unsigned 64-bit integer. Examples: * unsignedLongDiv(-1, 2) == Long.MAX_VALUE unsignedLongDiv(6, 3) == 2 @@ -49,7 +47,7 @@ object NumberConverter { * @param v is treated as an unsigned 64-bit integer * @param radix must be between MIN_RADIX and MAX_RADIX */ - private def decode(v: Long, radix: Int): Unit = { + private def decode(v: Long, radix: Int, value: Array[Byte]): Unit = { var tmpV = v java.util.Arrays.fill(value, 0.asInstanceOf[Byte]) var i = value.length - 1 @@ -69,11 +67,9 @@ object NumberConverter { * @param fromPos is the first element that should be considered * @return the result should be treated as an unsigned 64-bit integer. */ - private def encode(radix: Int, fromPos: Int): Long = { + private def encode(radix: Int, fromPos: Int, value: Array[Byte]): Long = { var v: Long = 0L val bound = unsignedLongDiv(-1 - radix, radix) // Possible overflow once - // val - // exceeds this value var i = fromPos while (i < value.length && value(i) >= 0) { if (v >= bound) { @@ -94,7 +90,7 @@ object NumberConverter { * @param radix must be between MIN_RADIX and MAX_RADIX * @param fromPos is the first nonzero element */ - private def byte2char(radix: Int, fromPos: Int): Unit = { + private def byte2char(radix: Int, fromPos: Int, value: Array[Byte]): Unit = { var i = fromPos while (i < value.length) { value(i) = Character.toUpperCase(Character.forDigit(value(i), radix)).asInstanceOf[Byte] @@ -109,9 +105,9 @@ object NumberConverter { * @param radix must be between MIN_RADIX and MAX_RADIX * @param fromPos is the first nonzero element */ - private def char2byte(radix: Int, fromPos: Int): Unit = { + private def char2byte(radix: Int, fromPos: Int, value: Array[Byte]): Unit = { var i = fromPos - while ( i < value.length) { + while (i < value.length) { value(i) = Character.digit(value(i), radix).asInstanceOf[Byte] i += 1 } @@ -124,8 +120,8 @@ object NumberConverter { */ def convert(n: Array[Byte], fromBase: Int, toBase: Int ): UTF8String = { if (fromBase < Character.MIN_RADIX || fromBase > Character.MAX_RADIX - || Math.abs(toBase) < Character.MIN_RADIX - || Math.abs(toBase) > Character.MAX_RADIX) { + || Math.abs(toBase) < Character.MIN_RADIX + || Math.abs(toBase) > Character.MAX_RADIX) { return null } @@ -136,15 +132,16 @@ object NumberConverter { var (negative, first) = if (n(0) == '-') (true, 1) else (false, 0) // Copy the digits in the right side of the array + val temp = new Array[Byte](64) var i = 1 while (i <= n.length - first) { - value(value.length - i) = n(n.length - i) + temp(temp.length - i) = n(n.length - i) i += 1 } - char2byte(fromBase, value.length - n.length + first) + char2byte(fromBase, temp.length - n.length + first, temp) // Do the conversion by going through a 64 bit integer - var v = encode(fromBase, value.length - n.length + first) + var v = encode(fromBase, temp.length - n.length + first, temp) if (negative && toBase > 0) { if (v < 0) { v = -1 @@ -156,21 +153,20 @@ object NumberConverter { v = -v negative = true } - decode(v, Math.abs(toBase)) + decode(v, Math.abs(toBase), temp) // Find the first non-zero digit or the last digits if all are zero. val firstNonZeroPos = { - val firstNonZero = value.indexWhere( _ != 0) - if (firstNonZero != -1) firstNonZero else value.length - 1 + val firstNonZero = temp.indexWhere( _ != 0) + if (firstNonZero != -1) firstNonZero else temp.length - 1 } - - byte2char(Math.abs(toBase), firstNonZeroPos) + byte2char(Math.abs(toBase), firstNonZeroPos, temp) var resultStartPos = firstNonZeroPos if (negative && toBase < 0) { resultStartPos = firstNonZeroPos - 1 - value(resultStartPos) = '-' + temp(resultStartPos) = '-' } - UTF8String.fromBytes(java.util.Arrays.copyOfRange(value, resultStartPos, value.length)) + UTF8String.fromBytes(java.util.Arrays.copyOfRange(temp, resultStartPos, temp.length)) } } From e110464571554942bc261ab93ee9e6503bb12516 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 31 May 2016 09:21:51 -0700 Subject: [PATCH 0448/1470] [SPARK-15649][SQL] Avoid to serialize MetastoreRelation in HiveTableScanExec ## What changes were proposed in this pull request? in HiveTableScanExec, schema is lazy and is related with relation.attributeMap. So it needs to serialize MetastoreRelation when serializing task binary bytes.It can avoid to serialize MetastoreRelation. ## How was this patch tested? Author: Lianhui Wang Closes #13397 from lianhuiwang/avoid-serialize. (cherry picked from commit 2bfc4f15214a870b3e067f06f37eb506b0070a1f) Signed-off-by: Reynold Xin --- .../apache/spark/sql/hive/execution/HiveTableScanExec.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index e29864f996e2e..cc3e74b4e8ccc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -152,8 +152,10 @@ case class HiveTableScanExec( } } val numOutputRows = longMetric("numOutputRows") + // Avoid to serialize MetastoreRelation because schema is lazy. (see SPARK-15649) + val outputSchema = schema rdd.mapPartitionsInternal { iter => - val proj = UnsafeProjection.create(schema) + val proj = UnsafeProjection.create(outputSchema) iter.map { r => numOutputRows += 1 proj(r) From 6347ff512d1e11106e44609a59be25a296aef731 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 31 May 2016 10:08:00 -0700 Subject: [PATCH 0449/1470] [SPARK-15647][SQL] Fix Boundary Cases in OptimizeCodegen Rule #### What changes were proposed in this pull request? The following condition in the Optimizer rule `OptimizeCodegen` is not right. ```Scala branches.size < conf.maxCaseBranchesForCodegen ``` - The number of branches in case when clause should be `branches.size + elseBranch.size`. - `maxCaseBranchesForCodegen` is the maximum boundary for enabling codegen. Thus, we should use `<=` instead of `<`. This PR is to fix this boundary case and also add missing test cases for verifying the conf `MAX_CASES_BRANCHES`. #### How was this patch tested? Added test cases in `SQLConfSuite` Author: gatorsmile Closes #13392 from gatorsmile/maxCaseWhen. (cherry picked from commit d67c82e4b647dacd0b789d72c9eaf4dc7d329dbd) Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 8 +++-- .../spark/sql/internal/SQLConfSuite.scala | 29 +++++++++++++++++++ 2 files changed, 35 insertions(+), 2 deletions(-) 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 688c77d3ca64f..93762ad1b91c2 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 @@ -937,8 +937,12 @@ object SimplifyConditionals extends Rule[LogicalPlan] with PredicateHelper { */ case class OptimizeCodegen(conf: CatalystConf) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case e @ CaseWhen(branches, _) if branches.size < conf.maxCaseBranchesForCodegen => - e.toCodegen() + case e: CaseWhen if canCodegen(e) => e.toCodegen() + } + + private def canCodegen(e: CaseWhen): Boolean = { + val numBranches = e.branches.size + e.elseValue.size + numBranches <= conf.maxCaseBranchesForCodegen } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 3d4fc75e83bbb..2cd3f475b6c0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.internal import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} +import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} class SQLConfSuite extends QueryTest with SharedSQLContext { @@ -219,4 +220,32 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + test("MAX_CASES_BRANCHES") { + withTable("tab1") { + spark.range(10).write.saveAsTable("tab1") + val sql_one_branch_caseWhen = "SELECT CASE WHEN id = 1 THEN 1 END FROM tab1" + val sql_two_branch_caseWhen = "SELECT CASE WHEN id = 1 THEN 1 ELSE 0 END FROM tab1" + + withSQLConf(SQLConf.MAX_CASES_BRANCHES.key -> "0") { + assert(!sql(sql_one_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + assert(!sql(sql_two_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + } + + withSQLConf(SQLConf.MAX_CASES_BRANCHES.key -> "1") { + assert(sql(sql_one_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + assert(!sql(sql_two_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + } + + withSQLConf(SQLConf.MAX_CASES_BRANCHES.key -> "2") { + assert(sql(sql_one_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + assert(sql(sql_two_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + } + } + } } From 29b94fdb380e78ea173585db8bd63def92c4684c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 31 May 2016 11:00:38 -0700 Subject: [PATCH 0450/1470] [SPARK-15658][SQL] UDT serializer should declare its data type as udt instead of udt.sqlType ## What changes were proposed in this pull request? When we build serializer for UDT object, we should declare its data type as udt instead of udt.sqlType, or if we deserialize it again, we lose the information that it's a udt object and throw analysis exception. ## How was this patch tested? new test in `UserDefiendTypeSuite` Author: Wenchen Fan Closes #13402 from cloud-fan/udt. (cherry picked from commit 2bfed1a0c5be7d0718fd574a4dad90f4f6b44be7) Signed-off-by: Yin Huai --- .../scala/org/apache/spark/sql/catalyst/ScalaReflection.scala | 4 ++-- .../org/apache/spark/sql/catalyst/encoders/RowEncoder.scala | 2 +- .../scala/org/apache/spark/sql/UserDefinedTypeSuite.scala | 4 ++++ 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index bdd40f340235b..052cc486e8cb2 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 @@ -568,7 +568,7 @@ object ScalaReflection extends ScalaReflection { udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), Nil, dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "serialize", udt.sqlType, inputObject :: Nil) + Invoke(obj, "serialize", udt, inputObject :: Nil) case t if UDTRegistration.exists(getClassNameFromType(t)) => val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() @@ -577,7 +577,7 @@ object ScalaReflection extends ScalaReflection { udt.getClass, Nil, dataType = ObjectType(udt.getClass)) - Invoke(obj, "serialize", udt.sqlType, inputObject :: Nil) + Invoke(obj, "serialize", udt, inputObject :: Nil) case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 2f8ba33f3520f..0de9166aa29aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -88,7 +88,7 @@ object RowEncoder { udtClass, Nil, dataType = ObjectType(udtClass), false) - Invoke(obj, "serialize", udt.sqlType, inputObject :: Nil) + Invoke(obj, "serialize", udt, inputObject :: Nil) case TimestampType => StaticInvoke( 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 7d7b486530de9..474f17ff7afbc 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 @@ -188,6 +188,10 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT val toCatalystConverter = CatalystTypeConverters.createToCatalystConverter(udt) assert(toCatalystConverter(null) === null) + } + test("SPARK-15658: Analysis exception if Dataset.map returns UDT object") { + // call `collect` to make sure this query can pass analysis. + pointsRDD.as[MyLabeledPoint].map(_.copy(label = 2.0)).collect() } } From 45472f8e081a623905fab15932213de6b96cd144 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 31 May 2016 12:30:34 -0700 Subject: [PATCH 0451/1470] [SPARK-15622][SQL] Wrap the parent classloader of Janino's classloader in the ParentClassLoader. ## What changes were proposed in this pull request? At https://github.com/aunkrig/janino/blob/janino_2.7.8/janino/src/org/codehaus/janino/ClassLoaderIClassLoader.java#L80-L85, Janino's classloader throws the exception when its parent throws a ClassNotFoundException with a cause set. However, it does not throw the exception when there is no cause set. Seems we need to use a special ClassLoader to wrap the actual parent classloader set to Janino handle this behavior. ## How was this patch tested? I have reverted the workaround made by https://issues.apache.org/jira/browse/SPARK-11636 ( https://github.com/apache/spark/compare/master...yhuai:SPARK-15622?expand=1#diff-bb538fda94224dd0af01d0fd7e1b4ea0R81) and `test-only *ReplSuite -- -z "SPARK-2576 importing implicits"` still passes the test (without the change in `CodeGenerator`, this test does not pass with the change in `ExecutorClassLoader `). Author: Yin Huai Closes #13366 from yhuai/SPARK-15622. (cherry picked from commit c6de5832bfad423e7d6f7e0a92a48170265f25cd) Signed-off-by: Yin Huai --- .../apache/spark/repl/ExecutorClassLoader.scala | 8 +------- .../expressions/codegen/CodeGenerator.scala | 16 +++++++++++++--- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 4a15d52b570a4..2f07395edf8d1 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -79,13 +79,7 @@ class ExecutorClassLoader( case e: ClassNotFoundException => val classOption = findClassLocally(name) classOption match { - case None => - // If this class has a cause, it will break the internal assumption of Janino - // (the compiler used for Spark SQL code-gen). - // See org.codehaus.janino.ClassLoaderIClassLoader's findIClass, you will see - // its behavior will be changed if there is a cause and the compilation - // of generated class will fail. - throw new ClassNotFoundException(name) + case None => throw new ClassNotFoundException(name, e) case Some(a) => a } } 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 2706c38d9e0a1..86883d7593412 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 @@ -19,10 +19,10 @@ package org.apache.spark.sql.catalyst.expressions.codegen import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.language.existentials import com.google.common.cache.{CacheBuilder, CacheLoader} import org.codehaus.janino.ClassBodyEvaluator +import scala.language.existentials import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.types._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ParentClassLoader, Utils} /** * Java source for evaluating an [[Expression]] given a [[InternalRow]] of input. @@ -806,7 +806,17 @@ object CodeGenerator extends Logging { */ private[this] def doCompile(code: CodeAndComment): GeneratedClass = { val evaluator = new ClassBodyEvaluator() - evaluator.setParentClassLoader(Utils.getContextOrSparkClassLoader) + + // A special classloader used to wrap the actual parent classloader of + // [[org.codehaus.janino.ClassBodyEvaluator]] (see CodeGenerator.doCompile). This classloader + // does not throw a ClassNotFoundException with a cause set (i.e. exception.getCause returns + // a null). This classloader is needed because janino will throw the exception directly if + // the parent classloader throws a ClassNotFoundException with cause set instead of trying to + // find other possible classes (see org.codehaus.janinoClassLoaderIClassLoader's + // findIClass method). Please also see https://issues.apache.org/jira/browse/SPARK-15622 and + // https://issues.apache.org/jira/browse/SPARK-11636. + val parentClassLoader = new ParentClassLoader(Utils.getContextOrSparkClassLoader) + evaluator.setParentClassLoader(parentClassLoader) // Cannot be under package codegen, or fail with java.lang.InstantiationException evaluator.setClassName("org.apache.spark.sql.catalyst.expressions.GeneratedClass") evaluator.setDefaultImports(Array( From 20a07e443bbb4c6cfa03423a2bccfb626cb87a67 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 31 May 2016 14:50:07 -0700 Subject: [PATCH 0452/1470] Revert "[SPARK-11753][SQL][TEST-HADOOP2.2] Make allowNonNumericNumbers option work ## What changes were proposed in this pull request? This reverts commit c24b6b679c3efa053f7de19be73eb36dc70d9930. Sent a PR to run Jenkins tests due to the revert conflicts of `dev/deps/spark-deps-hadoop*`. ## How was this patch tested? Jenkins unit tests, integration tests, manual tests) Author: Shixiong Zhu Closes #13417 from zsxwing/revert-SPARK-11753. (cherry picked from commit 9a74de18a13d84805e1a448f858bb05ce30de87e) Signed-off-by: Shixiong Zhu --- dev/deps/spark-deps-hadoop-2.2 | 12 ++-- dev/deps/spark-deps-hadoop-2.3 | 12 ++-- dev/deps/spark-deps-hadoop-2.4 | 12 ++-- dev/deps/spark-deps-hadoop-2.6 | 12 ++-- dev/deps/spark-deps-hadoop-2.7 | 12 ++-- pom.xml | 8 +-- python/pyspark/sql/readwriter.py | 3 - .../apache/spark/sql/DataFrameReader.scala | 2 - .../datasources/json/JacksonParser.scala | 28 ++++----- .../json/JsonParsingOptionsSuite.scala | 59 +++++-------------- 10 files changed, 58 insertions(+), 102 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index deec033c21dae..b5c38a6c056ec 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -72,13 +72,13 @@ hk2-utils-2.4.0-b34.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.7.3.jar -jackson-core-2.7.3.jar +jackson-annotations-2.6.5.jar +jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.7.3.jar +jackson-databind-2.6.5.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.7.3.jar -jackson-module-scala_2.11-2.7.3.jar +jackson-module-paranamer-2.6.5.jar +jackson-module-scala_2.11-2.6.5.jar janino-2.7.8.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar @@ -128,7 +128,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar +paranamer-2.3.jar parquet-column-1.8.1.jar parquet-common-1.8.1.jar parquet-encoding-1.8.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 43c7dd35805dc..969df0495d4c9 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -74,13 +74,13 @@ hk2-utils-2.4.0-b34.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.7.3.jar -jackson-core-2.7.3.jar +jackson-annotations-2.6.5.jar +jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.7.3.jar +jackson-databind-2.6.5.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.7.3.jar -jackson-module-scala_2.11-2.7.3.jar +jackson-module-paranamer-2.6.5.jar +jackson-module-scala_2.11-2.6.5.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar @@ -135,7 +135,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar +paranamer-2.3.jar parquet-column-1.8.1.jar parquet-common-1.8.1.jar parquet-encoding-1.8.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 7186b305a8af7..f0491ece7c2b4 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -74,13 +74,13 @@ hk2-utils-2.4.0-b34.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.7.3.jar -jackson-core-2.7.3.jar +jackson-annotations-2.6.5.jar +jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.7.3.jar +jackson-databind-2.6.5.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.7.3.jar -jackson-module-scala_2.11-2.7.3.jar +jackson-module-paranamer-2.6.5.jar +jackson-module-scala_2.11-2.6.5.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar @@ -135,7 +135,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar +paranamer-2.3.jar parquet-column-1.8.1.jar parquet-common-1.8.1.jar parquet-encoding-1.8.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 3e4ed74cc6b25..b3dced63b9e7f 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -80,14 +80,14 @@ htrace-core-3.0.4.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.7.3.jar -jackson-core-2.7.3.jar +jackson-annotations-2.6.5.jar +jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.7.3.jar +jackson-databind-2.6.5.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.7.3.jar -jackson-module-scala_2.11-2.7.3.jar +jackson-module-paranamer-2.6.5.jar +jackson-module-scala_2.11-2.6.5.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar @@ -143,7 +143,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar +paranamer-2.3.jar parquet-column-1.8.1.jar parquet-common-1.8.1.jar parquet-encoding-1.8.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 6b999538a3dd8..16f60f29ffbb2 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -80,14 +80,14 @@ htrace-core-3.1.0-incubating.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.7.3.jar -jackson-core-2.7.3.jar +jackson-annotations-2.6.5.jar +jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.7.3.jar +jackson-databind-2.6.5.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.7.3.jar -jackson-module-scala_2.11-2.7.3.jar +jackson-module-paranamer-2.6.5.jar +jackson-module-scala_2.11-2.6.5.jar jackson-xc-1.9.13.jar janino-2.7.8.jar java-xmlbuilder-1.0.jar @@ -144,7 +144,7 @@ objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar +paranamer-2.3.jar parquet-column-1.8.1.jar parquet-common-1.8.1.jar parquet-encoding-1.8.1.jar diff --git a/pom.xml b/pom.xml index ce9aa9aa00cc0..fff5560afea2a 100644 --- a/pom.xml +++ b/pom.xml @@ -160,7 +160,7 @@ ${scala.version} org.scala-lang 1.9.13 - 2.7.3 + 2.6.5 1.1.2.4 1.1.2 1.2.0-incubating @@ -180,7 +180,6 @@ 4.5.3 1.1 2.52.0 - 2.8 ${java.home} @@ -1826,11 +1825,6 @@ antlr4-runtime ${antlr4.version} - - com.thoughtworks.paranamer - paranamer - ${paranamer.version} - diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 13d21d7143d28..73105f881b464 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -193,9 +193,6 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, set, it uses the default value, ``true``. :param allowNumericLeadingZero: allows leading zeros in numbers (e.g. 00012). If None is set, it uses the default value, ``false``. - :param allowNonNumericNumbers: allows using non-numeric numbers such as "NaN", "Infinity", - "-Infinity", "INF", "-INF", which are convertd to floating - point numbers, ``true``. :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character using backslash quoting mechanism. If None is set, it uses the default value, ``false``. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 2057878028383..88fa5cd21d58f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -293,8 +293,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * *
  • `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers * (e.g. 00012)
  • - *
  • `allowNonNumericNumbers` (default `true`): allows using non-numeric numbers such as "NaN", - * "Infinity", "-Infinity", "INF", "-INF", which are convertd to floating point numbers.
  • *
  • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala index cafca323187d6..aeee2600a19ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala @@ -129,15 +129,13 @@ object JacksonParser extends Logging { case (VALUE_STRING, FloatType) => // Special case handling for NaN and Infinity. val value = parser.getText - if (value.equals("NaN") || - value.equals("Infinity") || - value.equals("+Infinity") || - value.equals("-Infinity")) { + val lowerCaseValue = value.toLowerCase() + if (lowerCaseValue.equals("nan") || + lowerCaseValue.equals("infinity") || + lowerCaseValue.equals("-infinity") || + lowerCaseValue.equals("inf") || + lowerCaseValue.equals("-inf")) { value.toFloat - } else if (value.equals("+INF") || value.equals("INF")) { - Float.PositiveInfinity - } else if (value.equals("-INF")) { - Float.NegativeInfinity } else { throw new SparkSQLJsonProcessingException(s"Cannot parse $value as FloatType.") } @@ -148,15 +146,13 @@ object JacksonParser extends Logging { case (VALUE_STRING, DoubleType) => // Special case handling for NaN and Infinity. val value = parser.getText - if (value.equals("NaN") || - value.equals("Infinity") || - value.equals("+Infinity") || - value.equals("-Infinity")) { + val lowerCaseValue = value.toLowerCase() + if (lowerCaseValue.equals("nan") || + lowerCaseValue.equals("infinity") || + lowerCaseValue.equals("-infinity") || + lowerCaseValue.equals("inf") || + lowerCaseValue.equals("-inf")) { value.toDouble - } else if (value.equals("+INF") || value.equals("INF")) { - Double.PositiveInfinity - } else if (value.equals("-INF")) { - Double.NegativeInfinity } else { throw new SparkSQLJsonProcessingException(s"Cannot parse $value as DoubleType.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index 2aab955c1ecbd..c31dffedbdf67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.datasources.json import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} /** * Test cases for various [[JSONOptions]]. @@ -94,51 +93,23 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSQLContext { assert(df.first().getLong(0) == 18) } - test("allowNonNumericNumbers off") { - // non-quoted non-numeric numbers don't work if allowNonNumericNumbers is off. - var testCases: Seq[String] = Seq("""{"age": NaN}""", """{"age": Infinity}""", - """{"age": +Infinity}""", """{"age": -Infinity}""", """{"age": INF}""", - """{"age": +INF}""", """{"age": -INF}""") - testCases.foreach { str => - val rdd = spark.sparkContext.parallelize(Seq(str)) - val df = spark.read.option("allowNonNumericNumbers", "false").json(rdd) - - assert(df.schema.head.name == "_corrupt_record") - } - - // quoted non-numeric numbers should still work even allowNonNumericNumbers is off. - testCases = Seq("""{"age": "NaN"}""", """{"age": "Infinity"}""", """{"age": "+Infinity"}""", - """{"age": "-Infinity"}""", """{"age": "INF"}""", """{"age": "+INF"}""", - """{"age": "-INF"}""") - val tests: Seq[Double => Boolean] = Seq(_.isNaN, _.isPosInfinity, _.isPosInfinity, - _.isNegInfinity, _.isPosInfinity, _.isPosInfinity, _.isNegInfinity) - val schema = StructType(StructField("age", DoubleType, true) :: Nil) - - testCases.zipWithIndex.foreach { case (str, idx) => - val rdd = spark.sparkContext.parallelize(Seq(str)) - val df = spark.read.option("allowNonNumericNumbers", "false").schema(schema).json(rdd) - - assert(df.schema.head.name == "age") - assert(tests(idx)(df.first().getDouble(0))) - } + // The following two tests are not really working - need to look into Jackson's + // JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS. + ignore("allowNonNumericNumbers off") { + val str = """{"age": NaN}""" + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.json(rdd) + + assert(df.schema.head.name == "_corrupt_record") } - test("allowNonNumericNumbers on") { - val testCases: Seq[String] = Seq("""{"age": NaN}""", """{"age": Infinity}""", - """{"age": +Infinity}""", """{"age": -Infinity}""", """{"age": +INF}""", - """{"age": -INF}""", """{"age": "NaN"}""", """{"age": "Infinity"}""", - """{"age": "-Infinity"}""") - val tests: Seq[Double => Boolean] = Seq(_.isNaN, _.isPosInfinity, _.isPosInfinity, - _.isNegInfinity, _.isPosInfinity, _.isNegInfinity, _.isNaN, _.isPosInfinity, - _.isNegInfinity, _.isPosInfinity, _.isNegInfinity) - val schema = StructType(StructField("age", DoubleType, true) :: Nil) - testCases.zipWithIndex.foreach { case (str, idx) => - val rdd = spark.sparkContext.parallelize(Seq(str)) - val df = spark.read.option("allowNonNumericNumbers", "true").schema(schema).json(rdd) - - assert(df.schema.head.name == "age") - assert(tests(idx)(df.first().getDouble(0))) - } + ignore("allowNonNumericNumbers on") { + val str = """{"age": NaN}""" + val rdd = spark.sparkContext.parallelize(Seq(str)) + val df = spark.read.option("allowNonNumericNumbers", "true").json(rdd) + + assert(df.schema.head.name == "age") + assert(df.first().getDouble(0).isNaN) } test("allowBackslashEscapingAnyCharacter off") { From 7feb7908563431090eabdee1369531dc1ba3e35b Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 31 May 2016 14:56:43 -0700 Subject: [PATCH 0453/1470] [MINOR][DOC][ML] ml.clustering scala & python api doc sync ## What changes were proposed in this pull request? Since we done Scala API audit for ml.clustering at #13148, we should also fix and update the corresponding Python API docs to keep them in sync. ## How was this patch tested? Docs change, no tests. Author: Yanbo Liang Closes #13291 from yanboliang/spark-15361-followup. (cherry picked from commit 594484cd8343f870c53fbc829ed4fb889016a8cf) Signed-off-by: Nick Pentreath --- python/pyspark/ml/clustering.py | 35 +++++++++++++++++++++++---------- 1 file changed, 25 insertions(+), 10 deletions(-) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index a457904e7880a..92df19e804374 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -64,6 +64,21 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte .. note:: Experimental GaussianMixture clustering. + This class performs expectation maximization for multivariate Gaussian + Mixture Models (GMMs). A GMM represents a composite distribution of + independent Gaussian distributions with associated "mixing" weights + specifying each's contribution to the composite. + + Given a set of sample points, this class will maximize the log-likelihood + for a mixture of k Gaussians, iterating until the log-likelihood changes by + less than convergenceTol, or until it has reached the max number of iterations. + While this process is generally guaranteed to converge, it is not guaranteed + to find a global optimum. + + Note: For high-dimensional data (with many features), this algorithm may perform poorly. + This is due to high-dimensional data (a) making it difficult to cluster at all + (based on statistical/theoretical arguments) and (b) numerical issues with + Gaussian distributions. >>> from pyspark.ml.linalg import Vectors @@ -118,8 +133,8 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte .. versionadded:: 2.0.0 """ - k = Param(Params._dummy(), "k", "number of clusters to create", - typeConverter=TypeConverters.toInt) + k = Param(Params._dummy(), "k", "Number of independent Gaussians in the mixture model. " + + "Must be > 1.", typeConverter=TypeConverters.toInt) @keyword_only def __init__(self, featuresCol="features", predictionCol="prediction", k=2, @@ -227,15 +242,15 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol .. versionadded:: 1.5.0 """ - k = Param(Params._dummy(), "k", "number of clusters to create", + k = Param(Params._dummy(), "k", "The number of clusters to create. Must be > 1.", typeConverter=TypeConverters.toInt) initMode = Param(Params._dummy(), "initMode", - "the initialization algorithm. This can be either \"random\" to " + + "The initialization algorithm. This can be either \"random\" to " + "choose random points as initial cluster centers, or \"k-means||\" " + "to use a parallel variant of k-means++", typeConverter=TypeConverters.toString) - initSteps = Param(Params._dummy(), "initSteps", "steps for k-means initialization mode", - typeConverter=TypeConverters.toInt) + initSteps = Param(Params._dummy(), "initSteps", "The number of steps for k-means|| " + + "initialization mode. Must be > 0.", typeConverter=TypeConverters.toInt) @keyword_only def __init__(self, featuresCol="features", predictionCol="prediction", k=2, @@ -380,11 +395,11 @@ class BisectingKMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte .. versionadded:: 2.0.0 """ - k = Param(Params._dummy(), "k", "number of clusters to create", + k = Param(Params._dummy(), "k", "The desired number of leaf clusters. Must be > 1.", typeConverter=TypeConverters.toInt) minDivisibleClusterSize = Param(Params._dummy(), "minDivisibleClusterSize", - "the minimum number of points (if >= 1.0) " + - "or the minimum proportion", + "The minimum number of points (if >= 1.0) or the minimum " + + "proportion of points (if < 1.0) of a divisible cluster.", typeConverter=TypeConverters.toFloat) @keyword_only @@ -661,7 +676,7 @@ class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInter .. versionadded:: 2.0.0 """ - k = Param(Params._dummy(), "k", "number of topics (clusters) to infer", + k = Param(Params._dummy(), "k", "The number of topics (clusters) to infer. Must be > 1.", typeConverter=TypeConverters.toInt) optimizer = Param(Params._dummy(), "optimizer", "Optimizer or inference algorithm used to estimate the LDA model. " From acf56c5db803f327a341b524d9fbf178eb3dc711 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 31 May 2016 15:36:02 -0700 Subject: [PATCH 0454/1470] [SPARK-15327] [SQL] fix split expression in whole stage codegen ## What changes were proposed in this pull request? Right now, we will split the code for expressions into multiple functions when it exceed 64k, which requires that the the expressions are using Row object, but this is not true for whole-state codegen, it will fail to compile after splitted. This PR will not split the code in whole-stage codegen. ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #13235 from davies/fix_nested_codegen. (cherry picked from commit 2df6ca848e99b90acd11c3a3de342fa4d77015d6) Signed-off-by: Davies Liu --- .../expressions/codegen/CodeGenerator.scala | 4 ++++ .../sql/execution/WholeStageCodegenExec.scala | 1 + .../aggregate/TungstenAggregate.scala | 2 ++ .../org/apache/spark/sql/SQLQuerySuite.scala | 24 +++++++++++++++++++ 4 files changed, 31 insertions(+) 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 86883d7593412..93e477e73236e 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 @@ -560,6 +560,10 @@ class CodegenContext { * @param expressions the codes to evaluate expressions. */ def splitExpressions(row: String, expressions: Seq[String]): String = { + if (row == null) { + // Cannot split these expressions because they are not created from a row object. + return expressions.mkString("\n") + } val blocks = new ArrayBuffer[String]() val blockBuilder = new StringBuilder() for (code <- expressions) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 2aec9318941a7..cd9ba7c75b91d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -130,6 +130,7 @@ trait CodegenSupport extends SparkPlan { } val evaluateInputs = evaluateVariables(outputVars) // generate the code to create a UnsafeRow + ctx.INPUT_ROW = row ctx.currentVars = outputVars val ev = GenerateUnsafeProjection.createCode(ctx, colExprs, false) val code = s""" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index d2dc80a7e42eb..905e93c15dce6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -599,6 +599,8 @@ case class TungstenAggregate( // create grouping key ctx.currentVars = input + // make sure that the generated code will not be splitted as multiple functions + ctx.INPUT_ROW = null val unsafeRowKeyCode = GenerateUnsafeProjection.createCode( ctx, groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) val vectorizedRowKeys = ctx.generateExpressions( 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 1ddb586d60fd9..91d93022df377 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 @@ -2483,6 +2483,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("SPARK-15327: fail to compile generated code with complex data structure") { + withTempDir{ dir => + val json = + """ + |{"h": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "d": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"c": {"b": {"c": [{"e": "adfgd"}], "a": [{"count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "a": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"e": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "g": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"f": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "b": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}}' + | + """.stripMargin + val rdd = sparkContext.parallelize(Array(json)) + spark.read.json(rdd).write.mode("overwrite").parquet(dir.toString) + spark.read.parquet(dir.toString).collect() + } + } + test("SPARK-14986: Outer lateral view with empty generate expression") { checkAnswer( sql("select nil from (select 1 as x ) x lateral view outer explode(array()) n as nil"), From 8657942ce601195ccde9bcef7402d9f944c6a0b9 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Tue, 31 May 2016 15:49:45 -0700 Subject: [PATCH 0455/1470] [SPARK-15557] [SQL] cast the string into DoubleType when it's used together with decimal In this case, the result type of the expression becomes DECIMAL(38, 36) as we promote the individual string literals to DECIMAL(38, 18) when we handle string promotions for `BinaryArthmaticExpression`. I think we need to cast the string literals to Double type instead. I looked at the history and found that this was changed to use decimal instead of double to avoid potential loss of precision when we cast decimal to double. To double check i ran the query against hive, mysql. This query returns non NULL result for both the databases and both promote the expression to use double. Here is the output. - Hive ```SQL hive> create table l2 as select (cast(99 as decimal(19,6)) + '2') from l1; OK hive> describe l2; OK _c0 double ``` - MySQL ```SQL mysql> create table foo2 as select (cast(99 as decimal(19,6)) + '2') from test; Query OK, 1 row affected (0.01 sec) Records: 1 Duplicates: 0 Warnings: 0 mysql> describe foo2; +-----------------------------------+--------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-----------------------------------+--------+------+-----+---------+-------+ | (cast(99 as decimal(19,6)) + '2') | double | NO | | 0 | | +-----------------------------------+--------+------+-----+---------+-------+ ``` ## How was this patch tested? Added a new test in SQLQuerySuite Author: Dilip Biswal Closes #13368 from dilipbiswal/spark-15557. (cherry picked from commit dfe2cbeb437a4fa69bec3eca4ac9242f3eb51c81) Signed-off-by: Davies Liu --- .../sql/catalyst/analysis/TypeCoercion.scala | 5 ----- .../sql/catalyst/analysis/AnalysisSuite.scala | 3 +-- .../datasources/json/JsonSuite.scala | 4 ++-- .../sql/hive/execution/SQLQuerySuite.scala | 19 +++++++++++++++++++ 4 files changed, 22 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 387e5552549e6..a5b5b91e4ab3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -290,11 +290,6 @@ object TypeCoercion { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - case a @ BinaryArithmetic(left @ StringType(), right @ DecimalType.Expression(_, _)) => - a.makeCopy(Array(Cast(left, DecimalType.SYSTEM_DEFAULT), right)) - case a @ BinaryArithmetic(left @ DecimalType.Expression(_, _), right @ StringType()) => - a.makeCopy(Array(left, Cast(right, DecimalType.SYSTEM_DEFAULT))) - case a @ BinaryArithmetic(left @ StringType(), right) => a.makeCopy(Array(Cast(left, DoubleType), right)) case a @ BinaryArithmetic(left, right @ StringType()) => 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 a63d1770f3255..77ea29ead92cc 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 @@ -182,8 +182,7 @@ class AnalysisSuite extends AnalysisTest { assert(pl(0).dataType == DoubleType) assert(pl(1).dataType == DoubleType) assert(pl(2).dataType == DoubleType) - // StringType will be promoted into Decimal(38, 18) - assert(pl(3).dataType == DecimalType(38, 22)) + assert(pl(3).dataType == DoubleType) assert(pl(4).dataType == DoubleType) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 500d8ff55a9a8..9f35c02d48762 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -446,13 +446,13 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // Number and String conflict: resolve the type as number in this query. checkAnswer( sql("select num_str + 1.2 from jsonTable where num_str > 14"), - Row(BigDecimal("92233720368547758071.2")) + Row(92233720368547758071.2) ) // Number and String conflict: resolve the type as number in this query. checkAnswer( sql("select num_str + 1.2 from jsonTable where num_str >= 92233720368547758060"), - Row(new java.math.BigDecimal("92233720368547758071.2")) + Row(new java.math.BigDecimal("92233720368547758071.2").doubleValue) ) // String and Boolean conflict: resolve the type as string. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 4b51f021bfa0c..2a9b06b75efa1 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 @@ -1560,4 +1560,23 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer(sql("SELECT * FROM tbl"), Row(1, "a")) } } + + test("spark-15557 promote string test") { + withTable("tbl") { + sql("CREATE TABLE tbl(c1 string, c2 string)") + sql("insert into tbl values ('3', '2.3')") + checkAnswer( + sql("select (cast (99 as decimal(19,6)) + cast('3' as decimal)) * cast('2.3' as decimal)"), + Row(204.0) + ) + checkAnswer( + sql("select (cast(99 as decimal(19,6)) + '3') *'2.3' from tbl"), + Row(234.6) + ) + checkAnswer( + sql("select (cast(99 as decimal(19,6)) + c1) * c2 from tbl"), + Row(234.6) + ) + } + } } From df4f87106d33acc3475185468f6bef86492b01e1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 31 May 2016 15:57:01 -0700 Subject: [PATCH 0456/1470] [SPARK-15517][SQL][STREAMING] Add support for complete output mode in Structure Streaming ## What changes were proposed in this pull request? Currently structured streaming only supports append output mode. This PR adds the following. - Added support for Complete output mode in the internal state store, analyzer and planner. - Added public API in Scala and Python for users to specify output mode - Added checks for unsupported combinations of output mode and DF operations - Plans with no aggregation should support only Append mode - Plans with aggregation should support only Update and Complete modes - Default output mode is Append mode (**Question: should we change this to automatically set to Complete mode when there is aggregation?**) - Added support for Complete output mode in Memory Sink. So Memory Sink internally supports append and complete, update. But from public API only Complete and Append output modes are supported. ## How was this patch tested? Unit tests in various test suites - StreamingAggregationSuite: tests for complete mode - MemorySinkSuite: tests for checking behavior in Append and Complete modes. - UnsupportedOperationSuite: tests for checking unsupported combinations of DF ops and output modes - DataFrameReaderWriterSuite: tests for checking that output mode cannot be called on static DFs - Python doc test and existing unit tests modified to call write.outputMode. Author: Tathagata Das Closes #13286 from tdas/complete-mode. (cherry picked from commit 90b11439b3d4540f48985e87dcc99749f0369287) Signed-off-by: Michael Armbrust --- python/pyspark/sql/readwriter.py | 20 ++ python/pyspark/sql/tests.py | 7 +- .../java/org/apache/spark/sql/OutputMode.java | 54 ++++++ .../spark/sql/InternalOutputModes.scala | 45 +++++ .../UnsupportedOperationChecker.scala | 49 ++--- .../spark/sql/JavaOutputModeSuite.java} | 16 +- .../analysis/UnsupportedOperationsSuite.scala | 63 +++--- .../spark/sql/ContinuousQueryManager.scala | 8 +- .../apache/spark/sql/DataFrameWriter.scala | 50 ++++- .../spark/sql/execution/aggregate/utils.scala | 6 +- .../execution/datasources/DataSource.scala | 9 +- .../streaming/IncrementalExecution.scala | 8 +- .../streaming/StatefulAggregate.scala | 70 +++++-- .../execution/streaming/StreamExecution.scala | 1 - .../sql/execution/streaming/console.scala | 5 +- .../sql/execution/streaming/memory.scala | 40 ++-- .../apache/spark/sql/sources/interfaces.scala | 3 +- .../org/apache/spark/sql/StreamTest.scala | 16 +- .../ContinuousQueryManagerSuite.scala | 22 +-- .../spark/sql/streaming/MemorySinkSuite.scala | 183 +++++++++++++++++- .../spark/sql/streaming/StreamSuite.scala | 7 + .../streaming/StreamingAggregationSuite.scala | 56 +++++- .../test/DataFrameReaderWriterSuite.scala | 38 +++- 23 files changed, 634 insertions(+), 142 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/OutputMode.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala rename sql/catalyst/src/{main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala => test/java/org/apache/spark/sql/JavaOutputModeSuite.java} (70%) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 73105f881b464..9208a527d29c3 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -497,6 +497,26 @@ def mode(self, saveMode): self._jwrite = self._jwrite.mode(saveMode) return self + @since(2.0) + def outputMode(self, outputMode): + """Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + + Options include: + + * `append`:Only the new rows in the streaming DataFrame/Dataset will be written to + the sink + * `complete`:All the rows in the streaming DataFrame/Dataset will be written to the sink + every time these is some updates + + .. note:: Experimental. + + >>> writer = sdf.write.outputMode('append') + """ + if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0: + raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode) + self._jwrite = self._jwrite.outputMode(outputMode) + return self + @since(1.4) def format(self, source): """Specifies the underlying output data source. diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 1790432edd5dc..0d9dd5ea2a364 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -926,7 +926,7 @@ def test_stream_save_options(self): out = os.path.join(tmpPath, 'out') chk = os.path.join(tmpPath, 'chk') cq = df.write.option('checkpointLocation', chk).queryName('this_query') \ - .format('parquet').option('path', out).startStream() + .format('parquet').outputMode('append').option('path', out).startStream() try: self.assertEqual(cq.name, 'this_query') self.assertTrue(cq.isActive) @@ -952,8 +952,9 @@ def test_stream_save_options_overwrite(self): fake1 = os.path.join(tmpPath, 'fake1') fake2 = os.path.join(tmpPath, 'fake2') cq = df.write.option('checkpointLocation', fake1).format('memory').option('path', fake2) \ - .queryName('fake_query').startStream(path=out, format='parquet', queryName='this_query', - checkpointLocation=chk) + .queryName('fake_query').outputMode('append') \ + .startStream(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) + try: self.assertEqual(cq.name, 'this_query') self.assertTrue(cq.isActive) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/OutputMode.java b/sql/catalyst/src/main/java/org/apache/spark/sql/OutputMode.java new file mode 100644 index 0000000000000..1936d53e5e835 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/OutputMode.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql; + +import org.apache.spark.annotation.Experimental; + +/** + * :: Experimental :: + * + * OutputMode is used to what data will be written to a streaming sink when there is + * new data available in a streaming DataFrame/Dataset. + * + * @since 2.0.0 + */ +@Experimental +public class OutputMode { + + /** + * OutputMode in which only the new rows in the streaming DataFrame/Dataset will be + * written to the sink. This output mode can be only be used in queries that do not + * contain any aggregation. + * + * @since 2.0.0 + */ + public static OutputMode Append() { + return InternalOutputModes.Append$.MODULE$; + } + + /** + * OutputMode in which all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time these is some updates. This output mode can only be used in queries + * that contain aggregations. + * + * @since 2.0.0 + */ + public static OutputMode Complete() { + return InternalOutputModes.Complete$.MODULE$; + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala new file mode 100644 index 0000000000000..8ef5d9a653278 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala @@ -0,0 +1,45 @@ +/* + * 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 + +/** + * Internal helper class to generate objects representing various [[OutputMode]]s, + */ +private[sql] object InternalOutputModes { + + /** + * OutputMode in which only the new rows in the streaming DataFrame/Dataset will be + * written to the sink. This output mode can be only be used in queries that do not + * contain any aggregation. + */ + case object Append extends OutputMode + + /** + * OutputMode in which all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time these is some updates. This output mode can only be used in queries + * that contain aggregations. + */ + case object Complete extends OutputMode + + /** + * OutputMode in which only the rows in the streaming DataFrame/Dataset that were updated will be + * written to the sink every time these is some updates. This output mode can only be used in + * queries that contain aggregations. + */ + case object Update extends OutputMode +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 0e08bf013c8d9..f4c03476096d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, InternalOutputModes, OutputMode} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -29,8 +29,7 @@ object UnsupportedOperationChecker { def checkForBatch(plan: LogicalPlan): Unit = { plan.foreachUp { case p if p.isStreaming => - throwError( - "Queries with streaming sources must be executed with write.startStream()")(p) + throwError("Queries with streaming sources must be executed with write.startStream()")(p) case _ => } @@ -43,10 +42,10 @@ object UnsupportedOperationChecker { "Queries without streaming sources cannot be executed with write.startStream()")(plan) } - plan.foreachUp { implicit plan => + plan.foreachUp { implicit subPlan => // Operations that cannot exists anywhere in a streaming plan - plan match { + subPlan match { case _: Command => throwError("Commands like CreateTable*, AlterTable*, Show* are not supported with " + @@ -55,21 +54,6 @@ object UnsupportedOperationChecker { case _: InsertIntoTable => throwError("InsertIntoTable is not supported with streaming DataFrames/Datasets") - case Aggregate(_, _, child) if child.isStreaming => - if (outputMode == Append) { - throwError( - "Aggregations are not supported on streaming DataFrames/Datasets in " + - "Append output mode. Consider changing output mode to Update.") - } - val moreStreamingAggregates = child.find { - case Aggregate(_, _, grandchild) if grandchild.isStreaming => true - case _ => false - } - if (moreStreamingAggregates.nonEmpty) { - throwError("Multiple streaming aggregations are not supported with " + - "streaming DataFrames/Datasets") - } - case Join(left, right, joinType, _) => joinType match { @@ -119,10 +103,10 @@ object UnsupportedOperationChecker { case GroupingSets(_, _, child, _) if child.isStreaming => throwError("GroupingSets is not supported on streaming DataFrames/Datasets") - case GlobalLimit(_, _) | LocalLimit(_, _) if plan.children.forall(_.isStreaming) => + case GlobalLimit(_, _) | LocalLimit(_, _) if subPlan.children.forall(_.isStreaming) => throwError("Limits are not supported on streaming DataFrames/Datasets") - case Sort(_, _, _) | SortPartitions(_, _) if plan.children.forall(_.isStreaming) => + case Sort(_, _, _) | SortPartitions(_, _) if subPlan.children.forall(_.isStreaming) => throwError("Sorting is not supported on streaming DataFrames/Datasets") case Sample(_, _, _, _, child) if child.isStreaming => @@ -138,6 +122,27 @@ object UnsupportedOperationChecker { case _ => } } + + // Checks related to aggregations + val aggregates = plan.collect { case a @ Aggregate(_, _, _) if a.isStreaming => a } + outputMode match { + case InternalOutputModes.Append if aggregates.nonEmpty => + throwError( + s"$outputMode output mode not supported when there are streaming aggregations on " + + s"streaming DataFrames/DataSets")(plan) + + case InternalOutputModes.Complete | InternalOutputModes.Update if aggregates.isEmpty => + throwError( + s"$outputMode output mode not supported when there are no streaming aggregations on " + + s"streaming DataFrames/Datasets")(plan) + + case _ => + } + if (aggregates.size > 1) { + throwError( + "Multiple streaming aggregations are not supported with " + + "streaming DataFrames/Datasets")(plan) + } } private def throwErrorIf( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala b/sql/catalyst/src/test/java/org/apache/spark/sql/JavaOutputModeSuite.java similarity index 70% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala rename to sql/catalyst/src/test/java/org/apache/spark/sql/JavaOutputModeSuite.java index a4d387eae3c80..1764f3348d8fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/JavaOutputModeSuite.java @@ -15,9 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.analysis +package org.apache.spark.sql; -sealed trait OutputMode +import org.junit.Test; -case object Append extends OutputMode -case object Update extends OutputMode +public class JavaOutputModeSuite { + + @Test + public void testOutputModes() { + OutputMode o1 = OutputMode.Append(); + assert(o1.toString().toLowerCase().contains("append")); + OutputMode o2 = OutputMode.Complete(); + assert (o2.toString().toLowerCase().contains("complete")); + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index aaeee0f2a41c4..c2e3d474506de 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, OutputMode} +import org.apache.spark.sql.InternalOutputModes._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -79,35 +80,13 @@ class UnsupportedOperationsSuite extends SparkFunSuite { outputMode = Append, expectedMsgs = "commands" :: Nil) - // Aggregates: Not supported on streams in Append mode - assertSupportedInStreamingPlan( - "aggregate - batch with update output mode", - batchRelation.groupBy("a")("count(*)"), - outputMode = Update) - - assertSupportedInStreamingPlan( - "aggregate - batch with append output mode", - batchRelation.groupBy("a")("count(*)"), - outputMode = Append) - - assertSupportedInStreamingPlan( - "aggregate - stream with update output mode", - streamRelation.groupBy("a")("count(*)"), - outputMode = Update) - - assertNotSupportedInStreamingPlan( - "aggregate - stream with append output mode", - streamRelation.groupBy("a")("count(*)"), - outputMode = Append, - Seq("aggregation", "append output mode")) - // Multiple streaming aggregations not supported def aggExprs(name: String): Seq[NamedExpression] = Seq(Count("*").as(name)) assertSupportedInStreamingPlan( "aggregate - multiple batch aggregations", Aggregate(Nil, aggExprs("c"), Aggregate(Nil, aggExprs("d"), batchRelation)), - Update) + Append) assertSupportedInStreamingPlan( "aggregate - multiple aggregations but only one streaming aggregation", @@ -209,7 +188,6 @@ class UnsupportedOperationsSuite extends SparkFunSuite { _.intersect(_), streamStreamSupported = false) - // Unary operations testUnaryOperatorInStreamingPlan("sort", Sort(Nil, true, _)) testUnaryOperatorInStreamingPlan("sort partitions", SortPartitions(Nil, _), expectedMsg = "sort") @@ -218,6 +196,10 @@ class UnsupportedOperationsSuite extends SparkFunSuite { testUnaryOperatorInStreamingPlan( "window", Window(Nil, Nil, Nil, _), expectedMsg = "non-time-based windows") + // Output modes with aggregation and non-aggregation plans + testOutputMode(Append, shouldSupportAggregation = false) + testOutputMode(Update, shouldSupportAggregation = true) + testOutputMode(Complete, shouldSupportAggregation = true) /* ======================================================================================= @@ -316,6 +298,37 @@ class UnsupportedOperationsSuite extends SparkFunSuite { outputMode) } + def testOutputMode( + outputMode: OutputMode, + shouldSupportAggregation: Boolean): Unit = { + + // aggregation + if (shouldSupportAggregation) { + assertNotSupportedInStreamingPlan( + s"$outputMode output mode - no aggregation", + streamRelation.where($"a" > 1), + outputMode = outputMode, + Seq("aggregation", s"$outputMode output mode")) + + assertSupportedInStreamingPlan( + s"$outputMode output mode - aggregation", + streamRelation.groupBy("a")("count(*)"), + outputMode = outputMode) + + } else { + assertSupportedInStreamingPlan( + s"$outputMode output mode - no aggregation", + streamRelation.where($"a" > 1), + outputMode = outputMode) + + assertNotSupportedInStreamingPlan( + s"$outputMode output mode - aggregation", + streamRelation.groupBy("a")("count(*)"), + outputMode = outputMode, + Seq("aggregation", s"$outputMode output mode")) + } + } + /** * Assert that the logical plan is supported as subplan insider a streaming plan. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala index eab557443d1d3..c686400150b2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.collection.mutable import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.catalyst.analysis.{Append, OutputMode, UnsupportedOperationChecker} +import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf @@ -35,7 +35,7 @@ import org.apache.spark.util.{Clock, SystemClock} * @since 2.0.0 */ @Experimental -class ContinuousQueryManager(sparkSession: SparkSession) { +class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { private[sql] val stateStoreCoordinator = StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) @@ -175,9 +175,9 @@ class ContinuousQueryManager(sparkSession: SparkSession) { checkpointLocation: String, df: DataFrame, sink: Sink, + outputMode: OutputMode, trigger: Trigger = ProcessingTime(0), - triggerClock: Clock = new SystemClock(), - outputMode: OutputMode = Append): ContinuousQuery = { + triggerClock: Clock = new SystemClock()): ContinuousQuery = { activeQueriesLock.synchronized { if (activeQueries.contains(name)) { throw new IllegalArgumentException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index f2ba2dfc086c2..291b8250c9051 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -77,7 +77,47 @@ final class DataFrameWriter private[sql](df: DataFrame) { case "ignore" => SaveMode.Ignore case "error" | "default" => SaveMode.ErrorIfExists case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " + - "Accepted modes are 'overwrite', 'append', 'ignore', 'error'.") + "Accepted save modes are 'overwrite', 'append', 'ignore', 'error'.") + } + this + } + + /** + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be + * written to the sink + * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time these is some updates + * + * @since 2.0.0 + */ + @Experimental + def outputMode(outputMode: OutputMode): DataFrameWriter = { + assertStreaming("outputMode() can only be called on continuous queries") + this.outputMode = outputMode + this + } + + /** + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to + * the sink + * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink + * every time these is some updates + * + * @since 2.0.0 + */ + @Experimental + def outputMode(outputMode: String): DataFrameWriter = { + assertStreaming("outputMode() can only be called on continuous queries") + this.outputMode = outputMode.toLowerCase match { + case "append" => + OutputMode.Append + case "complete" => + OutputMode.Complete + case _ => + throw new IllegalArgumentException(s"Unknown output mode $outputMode. " + + "Accepted output modes are 'append' and 'complete'") } this } @@ -319,7 +359,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { checkpointPath.toUri.toString } - val sink = new MemorySink(df.schema) + val sink = new MemorySink(df.schema, outputMode) val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink)) resultDf.createOrReplaceTempView(queryName) val continuousQuery = df.sparkSession.sessionState.continuousQueryManager.startQuery( @@ -327,6 +367,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { checkpointLocation, df, sink, + outputMode, trigger) continuousQuery } else { @@ -352,7 +393,8 @@ final class DataFrameWriter private[sql](df: DataFrame) { queryName, checkpointLocation, df, - dataSource.createSink(), + dataSource.createSink(outputMode), + outputMode, trigger) } } @@ -708,6 +750,8 @@ final class DataFrameWriter private[sql](df: DataFrame) { private var mode: SaveMode = SaveMode.ErrorIfExists + private var outputMode: OutputMode = OutputMode.Append + private var trigger: Trigger = ProcessingTime(0L) private var extraOptions = new scala.collection.mutable.HashMap[String, String] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala index f93c446007422..d617a048130e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala @@ -311,8 +311,10 @@ object Utils { aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes), child = restored) } - - val saved = StateStoreSaveExec(groupingAttributes, None, partialMerged2) + // Note: stateId and returnAllStates are filled in later with preparation rules + // in IncrementalExecution. + val saved = StateStoreSaveExec( + groupingAttributes, stateId = None, returnAllStates = None, partialMerged2) val finalAndCompleteAggregate: SparkPlan = { val finalAggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = Final)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index b3beb6c85f8ed..814880b0e0f40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -248,15 +248,20 @@ case class DataSource( } /** Returns a sink that can be used to continually write data. */ - def createSink(): Sink = { + def createSink(outputMode: OutputMode): Sink = { providingClass.newInstance() match { - case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, options, partitionColumns) + case s: StreamSinkProvider => + s.createSink(sparkSession.sqlContext, options, partitionColumns, outputMode) case parquet: parquet.ParquetFileFormat => val caseInsensitiveOptions = new CaseInsensitiveMap(options) val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") }) + if (outputMode != OutputMode.Append) { + throw new IllegalArgumentException( + s"Data source $className does not support $outputMode output mode") + } new FileStreamSink(sparkSession, path, parquet, partitionColumns, options) case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index fe5f36e1cdeeb..5c86049851163 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.analysis.OutputMode +import org.apache.spark.sql.{InternalOutputModes, OutputMode, SparkSession} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} @@ -54,16 +53,19 @@ class IncrementalExecution private[sql]( /** Locates save/restore pairs surrounding aggregation. */ val state = new Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan transform { - case StateStoreSaveExec(keys, None, + case StateStoreSaveExec(keys, None, None, UnaryExecNode(agg, StateStoreRestoreExec(keys2, None, child))) => val stateId = OperatorStateId(checkpointLocation, operatorId, currentBatchId) + val returnAllStates = if (outputMode == InternalOutputModes.Complete) true else false operatorId += 1 StateStoreSaveExec( keys, Some(stateId), + Some(returnAllStates), agg.withNewChildren( StateStoreRestoreExec( keys, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index d5e4dd8f78ac2..4d0283fbef1d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -82,10 +82,14 @@ case class StateStoreRestoreExec( case class StateStoreSaveExec( keyExpressions: Seq[Attribute], stateId: Option[OperatorStateId], + returnAllStates: Option[Boolean], child: SparkPlan) extends execution.UnaryExecNode with StatefulOperator { override protected def doExecute(): RDD[InternalRow] = { + assert(returnAllStates.nonEmpty, + "Incorrect planning in IncrementalExecution, returnAllStates have not been set") + val saveAndReturnFunc = if (returnAllStates.get) saveAndReturnAll _ else saveAndReturnUpdated _ child.execute().mapPartitionsWithStateStore( getStateId.checkpointLocation, operatorId = getStateId.operatorId, @@ -93,29 +97,57 @@ case class StateStoreSaveExec( keyExpressions.toStructType, child.output.toStructType, sqlContext.sessionState, - Some(sqlContext.streams.stateStoreCoordinator)) { case (store, iter) => - new Iterator[InternalRow] { - private[this] val baseIterator = iter - private[this] val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) + Some(sqlContext.streams.stateStoreCoordinator) + )(saveAndReturnFunc) + } + + override def output: Seq[Attribute] = child.output - override def hasNext: Boolean = { - if (!baseIterator.hasNext) { - store.commit() - false - } else { - true - } - } + /** + * Save all the rows to the state store, and return all the rows in the state store. + * Note that this returns an iterator that pipelines the saving to store with downstream + * processing. + */ + private def saveAndReturnUpdated( + store: StateStore, + iter: Iterator[InternalRow]): Iterator[InternalRow] = { + new Iterator[InternalRow] { + private[this] val baseIterator = iter + private[this] val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) - override def next(): InternalRow = { - val row = baseIterator.next().asInstanceOf[UnsafeRow] - val key = getKey(row) - store.put(key.copy(), row.copy()) - row - } + override def hasNext: Boolean = { + if (!baseIterator.hasNext) { + store.commit() + false + } else { + true } + } + + override def next(): InternalRow = { + val row = baseIterator.next().asInstanceOf[UnsafeRow] + val key = getKey(row) + store.put(key.copy(), row.copy()) + row + } } } - override def output: Seq[Attribute] = child.output + /** + * Save all the rows to the state store, and return all the rows in the state store. + * Note that the saving to store is blocking; only after all the rows have been saved + * is the iterator on the update store data is generated. + */ + private def saveAndReturnAll( + store: StateStore, + iter: Iterator[InternalRow]): Iterator[InternalRow] = { + val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) + while (iter.hasNext) { + val row = iter.next().asInstanceOf[UnsafeRow] + val key = getKey(row) + store.put(key.copy(), row.copy()) + } + store.commit() + store.iterator().map(_._2.asInstanceOf[InternalRow]) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 7d09bdcebdc3d..ab0900d7f6ea6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.OutputMode import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index f11a3fb969db6..391f1e54b7542 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, OutputMode, SQLContext} import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} class ConsoleSink(options: Map[String, String]) extends Sink with Logging { @@ -52,7 +52,8 @@ class ConsoleSinkProvider extends StreamSinkProvider with DataSourceRegister { def createSink( sqlContext: SQLContext, parameters: Map[String, String], - partitionColumns: Seq[String]): Sink = { + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { new ConsoleSink(parameters) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index bcc33ae8c88bd..e4a95e7335303 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row, SQLContext} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LeafNode @@ -114,35 +114,49 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) * A sink that stores the results in memory. This [[Sink]] is primarily intended for use in unit * tests and does not provide durability. */ -class MemorySink(val schema: StructType) extends Sink with Logging { +class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink with Logging { + + private case class AddedData(batchId: Long, data: Array[Row]) + /** An order list of batches that have been written to this [[Sink]]. */ @GuardedBy("this") - private val batches = new ArrayBuffer[Array[Row]]() + private val batches = new ArrayBuffer[AddedData]() /** Returns all rows that are stored in this [[Sink]]. */ def allData: Seq[Row] = synchronized { - batches.flatten + batches.map(_.data).flatten } - def latestBatchId: Option[Int] = synchronized { - if (batches.size == 0) None else Some(batches.size - 1) + def latestBatchId: Option[Long] = synchronized { + batches.lastOption.map(_.batchId) } - def lastBatch: Seq[Row] = synchronized { batches.last } + def latestBatchData: Seq[Row] = synchronized { batches.lastOption.toSeq.flatten(_.data) } def toDebugString: String = synchronized { - batches.zipWithIndex.map { case (b, i) => - val dataStr = try b.mkString(" ") catch { + batches.map { case AddedData(batchId, data) => + val dataStr = try data.mkString(" ") catch { case NonFatal(e) => "[Error converting to string]" } - s"$i: $dataStr" + s"$batchId: $dataStr" }.mkString("\n") } override def addBatch(batchId: Long, data: DataFrame): Unit = synchronized { - if (batchId == batches.size) { - logDebug(s"Committing batch $batchId") - batches.append(data.collect()) + if (latestBatchId.isEmpty || batchId > latestBatchId.get) { + logDebug(s"Committing batch $batchId to $this") + outputMode match { + case InternalOutputModes.Append | InternalOutputModes.Update => + batches.append(AddedData(batchId, data.collect())) + + case InternalOutputModes.Complete => + batches.clear() + batches.append(AddedData(batchId, data.collect())) + + case _ => + throw new IllegalArgumentException( + s"Output mode $outputMode is not supported by MemorySink") + } } else { logDebug(s"Skipping already committed batch: $batchId") } 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 26285bde31ad0..3d4edbb93d698 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 @@ -137,7 +137,8 @@ trait StreamSinkProvider { def createSink( sqlContext: SQLContext, parameters: Map[String, String], - partitionColumns: Seq[String]): Sink + partitionColumns: Seq[String], + outputMode: OutputMode): Sink } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala index 1ab562f873341..b033725f18b34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala @@ -33,7 +33,6 @@ import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.catalyst.analysis.{Append, OutputMode} import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ @@ -69,8 +68,6 @@ trait StreamTest extends QueryTest with Timeouts { /** How long to wait for an active stream to catch up when checking a result. */ val streamingTimeout = 10.seconds - val outputMode: OutputMode = Append - /** A trait for actions that can be performed while testing a streaming DataFrame. */ trait StreamAction @@ -191,14 +188,17 @@ trait StreamTest extends QueryTest with Timeouts { * Note that if the stream is not explicitly started before an action that requires it to be * running then it will be automatically started before performing any other actions. */ - def testStream(_stream: Dataset[_])(actions: StreamAction*): Unit = { + def testStream( + _stream: Dataset[_], + outputMode: OutputMode = OutputMode.Append)(actions: StreamAction*): Unit = { + val stream = _stream.toDF() var pos = 0 var currentPlan: LogicalPlan = stream.logicalPlan var currentStream: StreamExecution = null var lastStream: StreamExecution = null val awaiting = new mutable.HashMap[Int, Offset]() // source index -> offset to wait for - val sink = new MemorySink(stream.schema) + val sink = new MemorySink(stream.schema, outputMode) @volatile var streamDeathCause: Throwable = null @@ -297,9 +297,9 @@ trait StreamTest extends QueryTest with Timeouts { metadataRoot, stream, sink, + outputMode, trigger, - triggerClock, - outputMode = outputMode) + triggerClock) .asInstanceOf[StreamExecution] currentStream.microBatchThread.setUncaughtExceptionHandler( new UncaughtExceptionHandler { @@ -429,7 +429,7 @@ trait StreamTest extends QueryTest with Timeouts { } } - val sparkAnswer = try if (lastOnly) sink.lastBatch else sink.allData catch { + val sparkAnswer = try if (lastOnly) sink.latestBatchData else sink.allData catch { case e: Exception => failTest("Exception while getting data from sink", e) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala index a743cdde408fc..b75c3ea106e4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException -import org.apache.spark.sql.{ContinuousQuery, Dataset, StreamTest} +import org.apache.spark.sql.{ContinuousQuery, Dataset, OutputMode, StreamTest} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils @@ -232,20 +232,20 @@ class ContinuousQueryManagerSuite extends StreamTest with SharedSQLContext with private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[ContinuousQuery] => Unit): Unit = { failAfter(streamingTimeout) { val queries = withClue("Error starting queries") { - datasets.map { ds => + datasets.zipWithIndex.map { case (ds, i) => @volatile var query: StreamExecution = null try { val df = ds.toDF val metadataRoot = - Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath - query = spark - .streams - .startQuery( - StreamExecution.nextName, - metadataRoot, - df, - new MemorySink(df.schema)) - .asInstanceOf[StreamExecution] + Utils.createTempDir(namePrefix = "streaming.checkpoint").getCanonicalPath + query = + df.write + .format("memory") + .queryName(s"query$i") + .option("checkpointLocation", metadataRoot) + .outputMode("append") + .startStream() + .asInstanceOf[StreamExecution] } catch { case NonFatal(e) => if (query != null) query.stop() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala index 09c35bbf2c34b..e5bd0b47443eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala @@ -17,27 +17,132 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql.{AnalysisException, Row, StreamTest} +import scala.language.implicitConversions + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils -class MemorySinkSuite extends StreamTest with SharedSQLContext { +class MemorySinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { + import testImplicits._ - test("registering as a table") { - testRegisterAsTable() + after { + sqlContext.streams.active.foreach(_.stop()) } - ignore("stress test") { - // Ignore the stress test as it takes several minutes to run - (0 until 1000).foreach(_ => testRegisterAsTable()) + test("directly add data in Append output mode") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, InternalOutputModes.Append) + + // Before adding data, check output + assert(sink.latestBatchId === None) + checkAnswer(sink.latestBatchData, Seq.empty) + checkAnswer(sink.allData, Seq.empty) + + // Add batch 0 and check outputs + sink.addBatch(0, 1 to 3) + assert(sink.latestBatchId === Some(0)) + checkAnswer(sink.latestBatchData, 1 to 3) + checkAnswer(sink.allData, 1 to 3) + + // Add batch 1 and check outputs + sink.addBatch(1, 4 to 6) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) // new data should get appended to old data + + // Re-add batch 1 with different data, should not be added and outputs should not be changed + sink.addBatch(1, 7 to 9) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) + + // Add batch 2 and check outputs + sink.addBatch(2, 7 to 9) + assert(sink.latestBatchId === Some(2)) + checkAnswer(sink.latestBatchData, 7 to 9) + checkAnswer(sink.allData, 1 to 9) } - private def testRegisterAsTable(): Unit = { + test("directly add data in Update output mode") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, InternalOutputModes.Update) + + // Before adding data, check output + assert(sink.latestBatchId === None) + checkAnswer(sink.latestBatchData, Seq.empty) + checkAnswer(sink.allData, Seq.empty) + + // Add batch 0 and check outputs + sink.addBatch(0, 1 to 3) + assert(sink.latestBatchId === Some(0)) + checkAnswer(sink.latestBatchData, 1 to 3) + checkAnswer(sink.allData, 1 to 3) + + // Add batch 1 and check outputs + sink.addBatch(1, 4 to 6) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) // new data should get appended to old data + + // Re-add batch 1 with different data, should not be added and outputs should not be changed + sink.addBatch(1, 7 to 9) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) + + // Add batch 2 and check outputs + sink.addBatch(2, 7 to 9) + assert(sink.latestBatchId === Some(2)) + checkAnswer(sink.latestBatchData, 7 to 9) + checkAnswer(sink.allData, 1 to 9) + } + + test("directly add data in Complete output mode") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, InternalOutputModes.Complete) + + // Before adding data, check output + assert(sink.latestBatchId === None) + checkAnswer(sink.latestBatchData, Seq.empty) + checkAnswer(sink.allData, Seq.empty) + + // Add batch 0 and check outputs + sink.addBatch(0, 1 to 3) + assert(sink.latestBatchId === Some(0)) + checkAnswer(sink.latestBatchData, 1 to 3) + checkAnswer(sink.allData, 1 to 3) + + // Add batch 1 and check outputs + sink.addBatch(1, 4 to 6) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 4 to 6) // new data should replace old data + + // Re-add batch 1 with different data, should not be added and outputs should not be changed + sink.addBatch(1, 7 to 9) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 4 to 6) + + // Add batch 2 and check outputs + sink.addBatch(2, 7 to 9) + assert(sink.latestBatchId === Some(2)) + checkAnswer(sink.latestBatchData, 7 to 9) + checkAnswer(sink.allData, 7 to 9) + } + + + test("registering as a table in Append output mode") { val input = MemoryStream[Int] val query = input.toDF().write .format("memory") + .outputMode("append") .queryName("memStream") .startStream() input.addData(1, 2, 3) @@ -56,6 +161,57 @@ class MemorySinkSuite extends StreamTest with SharedSQLContext { query.stop() } + test("registering as a table in Complete output mode") { + val input = MemoryStream[Int] + val query = input.toDF() + .groupBy("value") + .count() + .write + .format("memory") + .outputMode("complete") + .queryName("memStream") + .startStream() + input.addData(1, 2, 3) + query.processAllAvailable() + + checkDataset( + spark.table("memStream").as[(Int, Long)], + (1, 1L), (2, 1L), (3, 1L)) + + input.addData(4, 5, 6) + query.processAllAvailable() + checkDataset( + spark.table("memStream").as[(Int, Long)], + (1, 1L), (2, 1L), (3, 1L), (4, 1L), (5, 1L), (6, 1L)) + + query.stop() + } + + ignore("stress test") { + // Ignore the stress test as it takes several minutes to run + (0 until 1000).foreach { _ => + val input = MemoryStream[Int] + val query = input.toDF().write + .format("memory") + .queryName("memStream") + .startStream() + input.addData(1, 2, 3) + query.processAllAvailable() + + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3) + + input.addData(4, 5, 6) + query.processAllAvailable() + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3, 4, 5, 6) + + query.stop() + } + } + test("error when no name is specified") { val error = intercept[AnalysisException] { val input = MemoryStream[Int] @@ -88,4 +244,15 @@ class MemorySinkSuite extends StreamTest with SharedSQLContext { .startStream() } } + + private def checkAnswer(rows: Seq[Row], expected: Seq[Int])(implicit schema: StructType): Unit = { + checkAnswer( + sqlContext.createDataFrame(sparkContext.makeRDD(rows), schema), + intsToDF(expected)(schema)) + } + + private implicit def intsToDF(seq: Seq[Int])(implicit schema: StructType): DataFrame = { + require(schema.fields.size === 1) + sqlContext.createDataset(seq).toDF(schema.fieldNames.head) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index ae89a6887a6d1..c17cb1de6ce9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -235,6 +235,13 @@ class StreamSuite extends StreamTest with SharedSQLContext { spark.experimental.extraStrategies = Nil } } + + test("output mode API in Scala") { + val o1 = OutputMode.Append + assert(o1 === InternalOutputModes.Append) + val o2 = OutputMode.Complete + assert(o2 === InternalOutputModes.Complete) + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 7104d01c4a2a1..322bbb9ea0a57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.streaming import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException -import org.apache.spark.sql.StreamTest -import org.apache.spark.sql.catalyst.analysis.Update +import org.apache.spark.sql.{AnalysisException, StreamTest} +import org.apache.spark.sql.InternalOutputModes._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scalalang.typed @@ -41,9 +41,7 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be import testImplicits._ - override val outputMode = Update - - test("simple count") { + test("simple count, update mode") { val inputData = MemoryStream[Int] val aggregated = @@ -52,7 +50,7 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be .agg(count("*")) .as[(Int, Long)] - testStream(aggregated)( + testStream(aggregated, Update)( AddData(inputData, 3), CheckLastBatch((3, 1)), AddData(inputData, 3, 2), @@ -67,6 +65,46 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be ) } + test("simple count, complete mode") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + testStream(aggregated, Complete)( + AddData(inputData, 3), + CheckLastBatch((3, 1)), + AddData(inputData, 2), + CheckLastBatch((3, 1), (2, 1)), + StopStream, + StartStream(), + AddData(inputData, 3, 2, 1), + CheckLastBatch((3, 2), (2, 2), (1, 1)), + AddData(inputData, 4, 4, 4, 4), + CheckLastBatch((4, 4), (3, 2), (2, 2), (1, 1)) + ) + } + + test("simple count, append mode") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + val e = intercept[AnalysisException] { + testStream(aggregated, Append)() + } + Seq("append", "not supported").foreach { m => + assert(e.getMessage.toLowerCase.contains(m.toLowerCase)) + } + } + test("multiple keys") { val inputData = MemoryStream[Int] @@ -76,7 +114,7 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be .agg(count("*")) .as[(Int, Int, Long)] - testStream(aggregated)( + testStream(aggregated, Update)( AddData(inputData, 1, 2), CheckLastBatch((1, 2, 1), (2, 3, 1)), AddData(inputData, 1, 2), @@ -101,7 +139,7 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be .agg(count("*")) .as[(Int, Long)] - testStream(aggregated)( + testStream(aggregated, Update)( StartStream(), AddData(inputData, 1, 2, 3, 4), ExpectFailure[SparkException](), @@ -114,7 +152,7 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext with Be val inputData = MemoryStream[(String, Int)] val aggregated = inputData.toDS().groupByKey(_._1).agg(typed.sumLong(_._2)) - testStream(aggregated)( + testStream(aggregated, Update)( AddData(inputData, ("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)), CheckLastBatch(("a", 30), ("b", 3), ("c", 1)) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index 288f6dc59741e..38a0534ab6be4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -90,10 +90,11 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { override def createSink( spark: SQLContext, parameters: Map[String, String], - partitionColumns: Seq[String]): Sink = { + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { LastOptions.parameters = parameters LastOptions.partitionColumns = partitionColumns - LastOptions.mockStreamSinkProvider.createSink(spark, parameters, partitionColumns) + LastOptions.mockStreamSinkProvider.createSink(spark, parameters, partitionColumns, outputMode) new Sink { override def addBatch(batchId: Long, data: DataFrame): Unit = {} } @@ -416,6 +417,39 @@ class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with B assert(e.getMessage == "mode() can only be called on non-continuous queries;") } + test("check outputMode(OutputMode) can only be called on continuous queries") { + val df = spark.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.outputMode(OutputMode.Append)) + Seq("outputmode", "continuous queries").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + + test("check outputMode(string) can only be called on continuous queries") { + val df = spark.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val e = intercept[AnalysisException](w.outputMode("append")) + Seq("outputmode", "continuous queries").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + + test("check outputMode(string) throws exception on unsupported modes") { + def testError(outputMode: String): Unit = { + val df = spark.read + .format("org.apache.spark.sql.streaming.test") + .stream() + val w = df.write + val e = intercept[IllegalArgumentException](w.outputMode(outputMode)) + Seq("output mode", "unknown", outputMode).foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + testError("Update") + testError("Xyz") + } + test("check bucketBy() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") From ef8d16ec4d566c004a1fd526dfe65b5df7a27b62 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 31 May 2016 16:54:34 -0700 Subject: [PATCH 0457/1470] [SPARK-15451][BUILD] Use jdk7's rt.jar when available. This helps with preventing jdk8-specific calls being checked in, because PR builders are running the compiler with the wrong settings. If the JAVA_7_HOME env variable is set, assume it points at a jdk7 and use its rt.jar when invoking javac. For zinc, just run it with jdk7, and disable it when building jdk8-specific code. A big note for sbt usage: adding the bootstrap options forces sbt to fork the compiler, and that disables incremental compilation. That means that it's really not convenient to use for normal development, but should be ok for automated builds. Tested with JAVA_HOME=jdk8 and JAVA_7_HOME=jdk7: - mvn + zinc - mvn sans zinc - sbt Verified that in all cases, jdk8-specific library calls fail to compile. Author: Marcelo Vanzin Closes #13272 from vanzin/SPARK-15451. (cherry picked from commit 57adb77e6b0208dc6d29488a674b6ad78702262f) Signed-off-by: Marcelo Vanzin --- build/mvn | 6 +++++- external/java8-tests/pom.xml | 1 + pom.xml | 39 ++++++++++++++++++++++++++++++++++++ project/SparkBuild.scala | 16 +++++++++++++-- 4 files changed, 59 insertions(+), 3 deletions(-) diff --git a/build/mvn b/build/mvn index a78b93a685689..c3ab62da36868 100755 --- a/build/mvn +++ b/build/mvn @@ -141,9 +141,13 @@ cd "${_CALLING_DIR}" # Now that zinc is ensured to be installed, check its status and, if its # not running or just installed, start it if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}`" ]; then + ZINC_JAVA_HOME= + if [ -n "$JAVA_7_HOME" ]; then + ZINC_JAVA_HOME="env JAVA_HOME=$JAVA_7_HOME" + fi export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} - "${ZINC_BIN}" -start -port ${ZINC_PORT} \ + $ZINC_JAVA_HOME "${ZINC_BIN}" -start -port ${ZINC_PORT} \ -scala-compiler "${SCALA_COMPILER}" \ -scala-library "${SCALA_LIBRARY}" &>/dev/null fi diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 60e3ff60df065..74a3ee1ce11e2 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -106,6 +106,7 @@ net.alchim31.maven scala-maven-plugin + ${useZincForJdk8} -source 1.8 diff --git a/pom.xml b/pom.xml index fff5560afea2a..60c8c8dc7a727 100644 --- a/pom.xml +++ b/pom.xml @@ -184,6 +184,9 @@ ${java.home} + + true + org.spark_project @@ -2576,6 +2579,42 @@ + + java7 + + env.JAVA_7_HOME + + + false + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + -bootclasspath + ${env.JAVA_7_HOME}/jre/lib/rt.jar + + + + + net.alchim31.maven + scala-maven-plugin + + + -javabootclasspath + ${env.JAVA_7_HOME}/jre/lib/rt.jar + + + + + + + + scala-2.11 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f08ca7001f34d..744f57c5177a3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -277,12 +277,24 @@ object SparkBuild extends PomBuild { // additional discussion and explanation. javacOptions in (Compile, compile) ++= Seq( "-target", javacJVMVersion.value - ), + ) ++ sys.env.get("JAVA_7_HOME").toSeq.flatMap { jdk7 => + if (javacJVMVersion.value == "1.7") { + Seq("-bootclasspath", s"$jdk7/jre/lib/rt.jar") + } else { + Nil + } + }, scalacOptions in Compile ++= Seq( s"-target:jvm-${scalacJVMVersion.value}", "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath // Required for relative source links in scaladoc - ), + ) ++ sys.env.get("JAVA_7_HOME").toSeq.flatMap { jdk7 => + if (javacJVMVersion.value == "1.7") { + Seq("-javabootclasspath", s"$jdk7/jre/lib/rt.jar") + } else { + Nil + } + }, // Implements -Xfatal-warnings, ignoring deprecation warnings. // Code snippet taken from https://issues.scala-lang.org/browse/SI-8410. From 1f4c4def946fd990393aba93aa76b5ac187fc111 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Tue, 31 May 2016 19:21:25 -0500 Subject: [PATCH 0458/1470] [CORE][MINOR][DOC] Removing incorrect scaladoc ## What changes were proposed in this pull request? I don't think the method will ever throw an exception so removing a false comment. Sorry srowen and rxin again -- I simply couldn't resist. I wholeheartedly support merging the change with a bigger one (and trashing this PR). ## How was this patch tested? Manual build Author: Jacek Laskowski Closes #13384 from jaceklaskowski/blockinfomanager. (cherry picked from commit 6954704299386cdcfdbc3f7fa6967c0880c9b7b8) Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 20ffe1342e509..dd8f5bacb9f6e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -211,9 +211,6 @@ private[storage] class BlockInfoManager extends Logging { * If another task has already locked this block for either reading or writing, then this call * will block until the other locks are released or will return immediately if `blocking = false`. * - * If this is called by a task which already holds the block's exclusive write lock, then this - * method will throw an exception. - * * @param blockId the block to lock. * @param blocking if true (default), this call will block until the lock is acquired. If false, * this call will return immediately if the lock acquisition fails. From 978f54e76bab2e03e034e2dfcc2c1ebaffefc5e0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 31 May 2016 17:29:10 -0700 Subject: [PATCH 0459/1470] [SPARK-15662][SQL] Add since annotation for classes in sql.catalog ## What changes were proposed in this pull request? This patch does a few things: 1. Adds since version annotation to methods and classes in sql.catalog. 2. Fixed a typo in FilterFunction and a whitespace issue in spark/api/java/function/package.scala 3. Added "database" field to Function class. ## How was this patch tested? Updated unit test case for "database" field in Function class. Author: Reynold Xin Closes #13406 from rxin/SPARK-15662. (cherry picked from commit 223f1d58c4f4b6eb0f0037a118a0bb635ae20bb1) Signed-off-by: Andrew Or --- .../api/java/function/FilterFunction.java | 2 +- .../spark/api/java/function/package.scala | 2 +- .../apache/spark/sql/catalog/Catalog.scala | 2 + .../apache/spark/sql/catalog/interface.scala | 42 ++++++++++++++++++- .../spark/sql/internal/CatalogImpl.scala | 1 + .../spark/sql/internal/CatalogSuite.scala | 22 +++++++--- 6 files changed, 62 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java index e8d999dd00135..462ca3f6f6d19 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java @@ -22,7 +22,7 @@ /** * Base interface for a function used in Dataset's filter function. * - * If the function returns true, the element is discarded in the returned Dataset. + * If the function returns true, the element is included in the returned Dataset. */ public interface FilterFunction extends Serializable { boolean call(T value) throws Exception; diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala index 0f9bac7164162..e19f12fdac090 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/package.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -22,4 +22,4 @@ package org.apache.spark.api.java * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's * Java programming guide for more details. */ -package object function +package object function diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index a99bc3bff6eea..6ddb1a7a1f1a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -24,6 +24,8 @@ import org.apache.spark.sql.types.StructType /** * Catalog interface for Spark. To access this, use `SparkSession.catalog`. + * + * @since 2.0.0 */ abstract class Catalog { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala index 0f7feb8eee7be..33032f07f7bea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala @@ -25,6 +25,14 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams // Note: all classes here are expected to be wrapped in Datasets and so must extend // DefinedByConstructorParams for the catalog to be able to create encoders for them. +/** + * A database in Spark, as returned by the `listDatabases` method defined in [[Catalog]]. + * + * @param name name of the database. + * @param description description of the database. + * @param locationUri path (in the form of a uri) to data files. + * @since 2.0.0 + */ class Database( val name: String, @Nullable val description: String, @@ -41,6 +49,16 @@ class Database( } +/** + * A table in Spark, as returned by the `listTables` method in [[Catalog]]. + * + * @param name name of the table. + * @param database name of the database the table belongs to. + * @param description description of the table. + * @param tableType type of the table (e.g. view, table). + * @param isTemporary whether the table is a temporary table. + * @since 2.0.0 + */ class Table( val name: String, @Nullable val database: String, @@ -61,6 +79,17 @@ class Table( } +/** + * A column in Spark, as returned by `listColumns` method in [[Catalog]]. + * + * @param name name of the column. + * @param description description of the column. + * @param dataType data type of the column. + * @param nullable whether the column is nullable. + * @param isPartition whether the column is a partition column. + * @param isBucket whether the column is a bucket column. + * @since 2.0.0 + */ class Column( val name: String, @Nullable val description: String, @@ -83,9 +112,19 @@ class Column( } -// TODO(andrew): should we include the database here? +/** + * A user-defined function in Spark, as returned by `listFunctions` method in [[Catalog]]. + * + * @param name name of the function. + * @param database name of the database the function belongs to. + * @param description description of the function; description can be null. + * @param className the fully qualified class name of the function. + * @param isTemporary whether the function is a temporary function or not. + * @since 2.0.0 + */ class Function( val name: String, + @Nullable val database: String, @Nullable val description: String, val className: String, val isTemporary: Boolean) @@ -94,6 +133,7 @@ class Function( override def toString: String = { "Function[" + s"name='$name', " + + Option(database).map { d => s"database='$d', " }.getOrElse("") + Option(description).map { d => s"description='$d', " }.getOrElse("") + s"className='$className', " + s"isTemporary='$isTemporary']" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index ceb68622752ea..70e17b10ac3cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -125,6 +125,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { val metadata = sessionCatalog.lookupFunctionInfo(funcIdent) new Function( name = funcIdent.identifier, + database = funcIdent.database.orNull, description = null, // for now, this is always undefined className = metadata.getClassName, isTemporary = funcIdent.database.isEmpty) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index cd434f7887db6..aec0312c4003e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.internal import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.{Column, Database, Function, Table} import org.apache.spark.sql.catalyst.{FunctionIdentifier, ScalaReflection, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ @@ -207,6 +207,14 @@ class CatalogSuite assert(!funcNames2.contains("my_func1")) assert(funcNames2.contains("my_func2")) assert(funcNames2.contains("my_temp_func")) + + // Make sure database is set properly. + assert( + spark.catalog.listFunctions("my_db1").collect().map(_.database).toSet == Set("my_db1", null)) + assert( + spark.catalog.listFunctions("my_db2").collect().map(_.database).toSet == Set("my_db2", null)) + + // Remove the function and make sure they no longer appear. dropFunction("my_func1", Some("my_db1")) dropTempFunction("my_temp_func") val funcNames1b = spark.catalog.listFunctions("my_db1").collect().map(_.name).toSet @@ -248,9 +256,11 @@ class CatalogSuite } test("Function.toString") { - assert(new Function("nama", "commenta", "classNameAh", isTemporary = true).toString == - "Function[name='nama', description='commenta', className='classNameAh', isTemporary='true']") - assert(new Function("nama", null, "classNameAh", isTemporary = false).toString == + assert( + new Function("nama", "databasa", "commenta", "classNameAh", isTemporary = true).toString == + "Function[name='nama', database='databasa', description='commenta', " + + "className='classNameAh', isTemporary='true']") + assert(new Function("nama", null, null, "classNameAh", isTemporary = false).toString == "Function[name='nama', className='classNameAh', isTemporary='false']") } @@ -268,7 +278,7 @@ class CatalogSuite test("catalog classes format in Dataset.show") { val db = new Database("nama", "descripta", "locata") val table = new Table("nama", "databasa", "descripta", "typa", isTemporary = false) - val function = new Function("nama", "descripta", "classa", isTemporary = false) + val function = new Function("nama", "databasa", "descripta", "classa", isTemporary = false) val column = new Column( "nama", "descripta", "typa", nullable = false, isPartition = true, isBucket = true) val dbFields = ScalaReflection.getConstructorParameterValues(db) @@ -277,7 +287,7 @@ class CatalogSuite val columnFields = ScalaReflection.getConstructorParameterValues(column) assert(dbFields == Seq("nama", "descripta", "locata")) assert(tableFields == Seq("nama", "databasa", "descripta", "typa", false)) - assert(functionFields == Seq("nama", "descripta", "classa", false)) + assert(functionFields == Seq("nama", "databasa", "descripta", "classa", false)) assert(columnFields == Seq("nama", "descripta", "typa", false, true, true)) val dbString = CatalogImpl.makeDataset(Seq(db), spark).showString(10) val tableString = CatalogImpl.makeDataset(Seq(table), spark).showString(10) From f0e8738c1ec0e4c5526aeada6f50cf76428f9afd Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 31 May 2016 17:30:03 -0700 Subject: [PATCH 0460/1470] [SPARK-15680][SQL] Disable comments in generated code in order to avoid perf. issues ## What changes were proposed in this pull request? In benchmarks involving tables with very wide and complex schemas (thousands of columns, deep nesting), I noticed that significant amounts of time (order of tens of seconds per task) were being spent generating comments during the code generation phase. The root cause of the performance problem stems from the fact that calling toString() on a complex expression can involve thousands of string concatenations, resulting in huge amounts (tens of gigabytes) of character array allocation and copying. In the long term, we can avoid this problem by passing StringBuilders down the tree and using them to accumulate output. As a short-term workaround, this patch guards comment generation behind a flag and disables comments by default (for wide tables / complex queries, these comments were being truncated prior to display and thus were not very useful). ## How was this patch tested? This was tested manually by running a Spark SQL query over an empty table with a very wide schema obtained from a real workload. Disabling comments brought the per-task time down from about 16 seconds to 600 milliseconds. Author: Josh Rosen Closes #13421 from JoshRosen/disable-line-comments-in-codegen. (cherry picked from commit 8ca01a6feb4935b1a3815cfbff1b90ccc6f60984) Signed-off-by: Reynold Xin --- .../expressions/codegen/CodeGenerator.scala | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) 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 93e477e73236e..9657f26402c01 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 @@ -24,6 +24,7 @@ import com.google.common.cache.{CacheBuilder, CacheLoader} import org.codehaus.janino.ClassBodyEvaluator import scala.language.existentials +import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -724,15 +725,23 @@ class CodegenContext { /** * Register a comment and return the corresponding place holder */ - def registerComment(text: String): String = { - val name = freshName("c") - val comment = if (text.contains("\n") || text.contains("\r")) { - text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") + def registerComment(text: => String): String = { + // By default, disable comments in generated code because computing the comments themselves can + // be extremely expensive in certain cases, such as deeply-nested expressions which operate over + // inputs with wide schemas. For more details on the performance issues that motivated this + // flat, see SPARK-15680. + if (SparkEnv.get != null && SparkEnv.get.conf.getBoolean("spark.sql.codegen.comments", false)) { + val name = freshName("c") + val comment = if (text.contains("\n") || text.contains("\r")) { + text.split("(\r\n)|\r|\n").mkString("/**\n * ", "\n * ", "\n */") + } else { + s"// $text" + } + placeHolderToComments += (name -> comment) + s"/*$name*/" } else { - s"// $text" + "" } - placeHolderToComments += (name -> comment) - s"/*$name*/" } } From 00fca2a058d9803fe2b07d7c5827d51e821e523e Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Tue, 31 May 2016 17:32:37 -0700 Subject: [PATCH 0461/1470] [CORE][DOC][MINOR] typos + links ## What changes were proposed in this pull request? A very tiny change to javadoc (which I don't mind if gets merged with a bigger change). I've just found it annoying and couldn't resist proposing a pull request. Sorry srowen and rxin. ## How was this patch tested? Manual build Author: Jacek Laskowski Closes #13383 from jaceklaskowski/memory-consumer. (cherry picked from commit 0f24713468088fa7617d208572179d558e1f286b) Signed-off-by: Andrew Or --- .../src/main/java/org/apache/spark/memory/MemoryConsumer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 38a21a896e1fe..fc1f3a80239ba 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -23,7 +23,7 @@ import org.apache.spark.unsafe.memory.MemoryBlock; /** - * An memory consumer of TaskMemoryManager, which support spilling. + * A memory consumer of {@link TaskMemoryManager} that supports spilling. * * Note: this only supports allocation / spilling of Tungsten memory. */ @@ -45,7 +45,7 @@ protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { } /** - * Returns the memory mode, ON_HEAP or OFF_HEAP. + * Returns the memory mode, {@link MemoryMode#ON_HEAP} or {@link MemoryMode#OFF_HEAP}. */ public MemoryMode getMode() { return mode; From 7f240eaeed78b42c23973678c5f643e486d4cfa3 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 31 May 2016 17:34:34 -0700 Subject: [PATCH 0462/1470] [SPARK-15670][JAVA API][SPARK CORE] label_accumulator_deprecate_in_java_spark_context ## What changes were proposed in this pull request? Add deprecate annotation for acumulator V1 interface in JavaSparkContext class ## How was this patch tested? N/A Author: WeichenXu Closes #13412 from WeichenXu123/label_accumulator_deprecate_in_java_spark_context. (cherry picked from commit dad5a68818436eb7feaeb762b72433248eff298f) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index fb6323413e3ea..bfb6a35f5bb93 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -596,6 +596,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `add` method. Only the master can access the accumulator's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) @@ -605,6 +606,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulator[T](initialValue: T, name: String, accumulatorParam: AccumulatorParam[T]) : Accumulator[T] = sc.accumulator(initialValue, name)(accumulatorParam) @@ -613,6 +615,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks * can "add" values with `add`. Only the master can access the accumuable's `value`. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) @@ -622,6 +625,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use AccumulatorV2", "2.0.0") def accumulable[T, R](initialValue: T, name: String, param: AccumulableParam[T, R]) : Accumulable[T, R] = sc.accumulable(initialValue, name)(param) From b8de4ad7d2ab88dfe5505711a55ef0ba666781b8 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Tue, 31 May 2016 17:34:10 -0700 Subject: [PATCH 0463/1470] [SPARK-12988][SQL] Can't drop top level columns that contain dots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Fixes "Can't drop top level columns that contain dots". This work is based on dilipbiswal's https://github.com/apache/spark/pull/10943. This PR fixes problems like: ``` scala> Seq((1, 2)).toDF("a.b", "a.c").drop("a.b") org.apache.spark.sql.AnalysisException: cannot resolve '`a.c`' given input columns: [a.b, a.c]; ``` `drop(columnName)` can only be used to drop top level column, so, we should parse the column name literally WITHOUT interpreting dot "." We should also NOT interpret back tick "`", otherwise it is hard to understand what ``` ​```aaa```bbb`` ``` actually means. ## How was this patch tested? Unit tests. Author: Sean Zhong Closes #13306 from clockfly/fix_drop_column. (cherry picked from commit 06514d689cb346e2bcfe3f37a037049d4410befa) Signed-off-by: Yin Huai --- .../scala/org/apache/spark/sql/Dataset.scala | 18 +++++++++++----- .../org/apache/spark/sql/DataFrameSuite.scala | 21 +++++++++++++++++++ 2 files changed, 34 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 8b6662ab1fae6..31000dc41be6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1703,8 +1703,11 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with a column dropped. - * This is a no-op if schema doesn't contain column name. + * Returns a new [[Dataset]] with a column dropped. This is a no-op if schema doesn't contain + * column name. + * + * This method can only be used to drop top level columns. the colName string is treated + * literally without further interpretation. * * @group untypedrel * @since 2.0.0 @@ -1717,15 +1720,20 @@ class Dataset[T] private[sql]( * Returns a new [[Dataset]] with columns dropped. * This is a no-op if schema doesn't contain column name(s). * + * This method can only be used to drop top level columns. the colName string is treated literally + * without further interpretation. + * * @group untypedrel * @since 2.0.0 */ @scala.annotation.varargs def drop(colNames: String*): DataFrame = { val resolver = sparkSession.sessionState.analyzer.resolver - val remainingCols = - schema.filter(f => colNames.forall(n => !resolver(f.name, n))).map(f => Column(f.name)) - if (remainingCols.size == this.schema.size) { + val allColumns = queryExecution.analyzed.output + val remainingCols = allColumns.filter { attribute => + colNames.forall(n => !resolver(attribute.name, n)) + }.map(attribute => Column(attribute)) + if (remainingCols.size == allColumns.size) { toDF() } else { this.select(remainingCols: _*) 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 e2dc4d86395ee..0e18ade09cbe5 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 @@ -609,6 +609,27 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df("id") == person("id")) } + test("drop top level columns that contains dot") { + val df1 = Seq((1, 2)).toDF("a.b", "a.c") + checkAnswer(df1.drop("a.b"), Row(2)) + + // Creates data set: {"a.b": 1, "a": {"b": 3}} + val df2 = Seq((1)).toDF("a.b").withColumn("a", struct(lit(3) as "b")) + // Not like select(), drop() parses the column name "a.b" literally without interpreting "." + checkAnswer(df2.drop("a.b").select("a.b"), Row(3)) + + // "`" is treated as a normal char here with no interpreting, "`a`b" is a valid column name. + assert(df2.drop("`a.b`").columns.size == 2) + } + + test("drop(name: String) search and drop all top level columns that matchs the name") { + val df1 = Seq((1, 2)).toDF("a", "b") + val df2 = Seq((3, 4)).toDF("a", "b") + checkAnswer(df1.join(df2), Row(1, 2, 3, 4)) + // Finds and drops all columns that match the name (case insensitive). + checkAnswer(df1.join(df2).drop("A"), Row(2, 4)) + } + test("withColumnRenamed") { val df = testData.toDF().withColumn("newCol", col("key") + 1) .withColumnRenamed("value", "valueRenamed") From 4b19c97764489a48abccab75e1b132b469383f44 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 31 May 2016 17:36:24 -0700 Subject: [PATCH 0464/1470] [MINOR][SQL][DOCS] Fix docs of Dataset.scala and SQLImplicits.scala. This PR fixes a sample code, a description, and indentations in docs. Manual. Author: Dongjoon Hyun Closes #13420 from dongjoon-hyun/minor_fix_dataset_doc. (cherry picked from commit 196a0d82730e78b573a64a791a6ad873aa9ec74d) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/sql/Dataset.scala | 36 +++++++++---------- .../org/apache/spark/sql/SQLImplicits.scala | 2 +- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 31000dc41be6a..7be49b174983f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql @@ -93,14 +93,14 @@ private[sql] object Dataset { * to some files on storage systems, using the `read` function available on a `SparkSession`. * {{{ * val people = spark.read.parquet("...").as[Person] // Scala - * Dataset people = spark.read().parquet("...").as(Encoders.bean(Person.class) // Java + * Dataset people = spark.read().parquet("...").as(Encoders.bean(Person.class)); // Java * }}} * * Datasets can also be created through transformations available on existing Datasets. For example, * the following creates a new Dataset by applying a filter on the existing one: * {{{ * val names = people.map(_.name) // in Scala; names is a Dataset[String] - * Dataset names = people.map((Person p) -> p.name, Encoders.STRING)) // in Java 8 + * Dataset names = people.map((Person p) -> p.name, Encoders.STRING)); // in Java 8 * }}} * * Dataset operations can also be untyped, through various domain-specific-language (DSL) @@ -110,7 +110,7 @@ private[sql] object Dataset { * To select a column from the Dataset, use `apply` method in Scala and `col` in Java. * {{{ * val ageCol = people("age") // in Scala - * Column ageCol = people.col("age") // in Java + * Column ageCol = people.col("age"); // in Java * }}} * * Note that the [[Column]] type can also be manipulated through its various functions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index f423e7d6b5765..b7ea2a89175fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -24,7 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder /** - * A collection of implicit methods for converting common Scala objects into [[DataFrame]]s. + * A collection of implicit methods for converting common Scala objects into [[Dataset]]s. * * @since 1.6.0 */ From ac4cb17182615dba56e7fb5978150e3eb69a113a Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 31 May 2016 17:39:03 -0700 Subject: [PATCH 0465/1470] [MINOR] Slightly better error message when attempting to query hive tables w/in-mem catalog andrewor14 Author: Eric Liang Closes #13427 from ericl/better-error-msg. (cherry picked from commit 93e97147eb499dde1e54e07ba113eebcbe25508a) Signed-off-by: Andrew Or --- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c5f221d7830f3..7b451baaa02b9 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 @@ -309,14 +309,14 @@ trait CheckAnalysis extends PredicateHelper { case s: SimpleCatalogRelation => failAnalysis( s""" - |Please enable Hive support when selecting the regular tables: + |Hive support is required to select over the following tables: |${s.catalogTable.identifier} """.stripMargin) case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => failAnalysis( s""" - |Please enable Hive support when inserting the regular tables: + |Hive support is required to insert into the following tables: |${s.catalogTable.identifier} """.stripMargin) From 459fd34a0b011589d924e318925c97657e71930d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 31 May 2016 17:40:44 -0700 Subject: [PATCH 0466/1470] [SPARK-15618][SQL][MLLIB] Use SparkSession.builder.sparkContext if applicable. This PR changes function `SparkSession.builder.sparkContext(..)` from **private[sql]** into **private[spark]**, and uses it if applicable like the followings. ``` - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() ``` Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13365 from dongjoon-hyun/SPARK-15618. (cherry picked from commit 85d6b0db9f5bd425c36482ffcb1c3b9fd0fcdb31) Signed-off-by: Andrew Or --- .../apache/spark/examples/BroadcastTest.scala | 7 +--- .../spark/examples/mllib/LDAExample.scala | 1 + .../examples/sql/hive/HiveFromSpark.scala | 5 +-- .../mllib/api/python/PythonMLLibAPI.scala | 6 +-- .../classification/LogisticRegression.scala | 2 +- .../mllib/classification/NaiveBayes.scala | 8 ++-- .../impl/GLMClassificationModel.scala | 4 +- .../clustering/BisectingKMeansModel.scala | 4 +- .../clustering/GaussianMixtureModel.scala | 4 +- .../spark/mllib/clustering/KMeansModel.scala | 4 +- .../spark/mllib/clustering/LDAModel.scala | 8 ++-- .../clustering/PowerIterationClustering.scala | 4 +- .../spark/mllib/feature/ChiSqSelector.scala | 4 +- .../apache/spark/mllib/feature/Word2Vec.scala | 4 +- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 4 +- .../apache/spark/mllib/fpm/PrefixSpan.scala | 4 +- .../MatrixFactorizationModel.scala | 4 +- .../mllib/regression/IsotonicRegression.scala | 4 +- .../regression/impl/GLMRegressionModel.scala | 4 +- .../mllib/tree/model/DecisionTreeModel.scala | 4 +- .../mllib/tree/model/treeEnsembleModels.scala | 4 +- .../spark/ml/feature/ChiSqSelectorSuite.scala | 8 +--- .../ml/feature/QuantileDiscretizerSuite.scala | 6 +-- .../spark/ml/recommendation/ALSSuite.scala | 3 +- .../apache/spark/ml/tree/impl/TreeTests.scala | 3 +- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../execution/joins/BroadcastJoinSuite.scala | 39 +++++++++---------- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 13 +++---- 28 files changed, 78 insertions(+), 89 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index c50f25d951947..a68fd0285f567 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -29,13 +29,10 @@ object BroadcastTest { val blockSize = if (args.length > 2) args(2) else "4096" - val sparkConf = new SparkConf() - .set("spark.broadcast.blockSize", blockSize) - val spark = SparkSession - .builder - .config(sparkConf) + .builder() .appName("Broadcast Test") + .config("spark.broadcast.blockSize", blockSize) .getOrCreate() val sc = spark.sparkContext diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 7651aade493a0..3fbf8e03339e8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -191,6 +191,7 @@ object LDAExample { val spark = SparkSession .builder + .sparkContext(sc) .getOrCreate() import spark.implicits._ diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index d3bb7e4398cd3..2d7a01a95d830 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -22,7 +22,6 @@ import java.io.File import com.google.common.io.{ByteStreams, Files} -import org.apache.spark.SparkConf import org.apache.spark.sql._ object HiveFromSpark { @@ -35,8 +34,6 @@ object HiveFromSpark { ByteStreams.copy(kv1Stream, Files.newOutputStreamSupplier(kv1File)) def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("HiveFromSpark") - // When working with Hive, one must instantiate `SparkSession` with Hive support, including // connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined // functions. Users who do not have an existing Hive deployment can still enable Hive support. @@ -45,7 +42,7 @@ object HiveFromSpark { // which defaults to the directory `spark-warehouse` in the current directory that the spark // application is started. val spark = SparkSession.builder - .config(sparkConf) + .appName("HiveFromSpark") .enableHiveSupport() .getOrCreate() val sc = spark.sparkContext 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 6e0ed374c7ee1..e43469bf1cf86 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 @@ -1177,7 +1177,7 @@ private[python] class PythonMLLibAPI extends Serializable { // We use DataFrames for serialization of IndexedRows to Python, // so return a DataFrame. val sc = indexedRowMatrix.rows.sparkContext - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() spark.createDataFrame(indexedRowMatrix.rows) } @@ -1188,7 +1188,7 @@ private[python] class PythonMLLibAPI extends Serializable { // We use DataFrames for serialization of MatrixEntry entries to // Python, so return a DataFrame. val sc = coordinateMatrix.entries.sparkContext - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() spark.createDataFrame(coordinateMatrix.entries) } @@ -1199,7 +1199,7 @@ private[python] class PythonMLLibAPI extends Serializable { // We use DataFrames for serialization of sub-matrix blocks to // Python, so return a DataFrame. val sc = blockMatrix.blocks.sparkContext - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() spark.createDataFrame(blockMatrix.blocks) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index b186ca37703d3..e4cc784cfe421 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -437,7 +437,7 @@ class LogisticRegressionWithLBFGS lr.setMaxIter(optimizer.getNumIterations()) lr.setTol(optimizer.getConvergenceTol()) // Convert our input into a DataFrame - val spark = SparkSession.builder().config(input.context.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(input.context).getOrCreate() val df = spark.createDataFrame(input.map(_.asML)) // Determine if we should cache the DF val handlePersistence = input.getStorageLevel == StorageLevel.NONE diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 452802f043abf..593a86f69ad51 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -193,7 +193,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { modelType: String) def save(sc: SparkContext, path: String, data: Data): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -207,7 +207,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { @Since("1.3.0") def load(sc: SparkContext, path: String): NaiveBayesModel = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Load Parquet data. val dataRDD = spark.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. @@ -238,7 +238,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { theta: Array[Array[Double]]) def save(sc: SparkContext, path: String, data: Data): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -251,7 +251,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { } def load(sc: SparkContext, path: String): NaiveBayesModel = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Load Parquet data. val dataRDD = spark.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala index 32e323d080af8..84491181d077f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala @@ -51,7 +51,7 @@ private[classification] object GLMClassificationModel { weights: Vector, intercept: Double, threshold: Option[Double]): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -73,7 +73,7 @@ private[classification] object GLMClassificationModel { */ def loadData(sc: SparkContext, path: String, modelClass: String): Data = { val dataPath = Loader.dataPath(path) - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataRDD = spark.read.parquet(dataPath) val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1) assert(dataArray.length == 1, s"Unable to load $modelClass data from: $dataPath") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index 510a91b5a77fd..b3546a1ee3677 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -144,7 +144,7 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val thisClassName = "org.apache.spark.mllib.clustering.BisectingKMeansModel" def save(sc: SparkContext, model: BisectingKMeansModel, path: String): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rootId" -> model.root.index))) @@ -165,7 +165,7 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { } def load(sc: SparkContext, path: String, rootId: Int): BisectingKMeansModel = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val rows = spark.read.parquet(Loader.dataPath(path)) Loader.checkSchema[Data](rows.schema) val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 31ad56dba6aef..c30cc3e2398e4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -143,7 +143,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { path: String, weights: Array[Double], gaussians: Array[MultivariateGaussian]): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render @@ -159,7 +159,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { def load(sc: SparkContext, path: String): GaussianMixtureModel = { val dataPath = Loader.dataPath(path) - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataFrame = spark.read.parquet(dataPath) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[Data](dataFrame.schema) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 5f939c1a218fb..aa78149699a27 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -123,7 +123,7 @@ object KMeansModel extends Loader[KMeansModel] { val thisClassName = "org.apache.spark.mllib.clustering.KMeansModel" def save(sc: SparkContext, model: KMeansModel, path: String): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) @@ -135,7 +135,7 @@ object KMeansModel extends Loader[KMeansModel] { def load(sc: SparkContext, path: String): KMeansModel = { implicit val formats = DefaultFormats - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 1b66013d543ad..4f07236225cd2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -446,7 +446,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { docConcentration: Vector, topicConcentration: Double, gammaShape: Double): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val k = topicsMatrix.numCols val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ @@ -470,7 +470,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { topicConcentration: Double, gammaShape: Double): LocalLDAModel = { val dataPath = Loader.dataPath(path) - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataFrame = spark.read.parquet(dataPath) Loader.checkSchema[Data](dataFrame.schema) @@ -851,7 +851,7 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { topicConcentration: Double, iterationTimes: Array[Double], gammaShape: Double): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ @@ -887,7 +887,7 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { val dataPath = new Path(Loader.dataPath(path), "globalTopicTotals").toUri.toString val vertexDataPath = new Path(Loader.dataPath(path), "topicCounts").toUri.toString val edgeDataPath = new Path(Loader.dataPath(path), "tokenCounts").toUri.toString - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataFrame = spark.read.parquet(dataPath) val vertexDataFrame = spark.read.parquet(vertexDataPath) val edgeDataFrame = spark.read.parquet(edgeDataPath) 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 51077bd630a15..c760ddd6ad40b 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 @@ -70,7 +70,7 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode @Since("1.4.0") def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) @@ -82,7 +82,7 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode @Since("1.4.0") def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { implicit val formats = DefaultFormats - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index 13decefcd6695..c8c2823bbaf04 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -134,7 +134,7 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { val thisClassName = "org.apache.spark.mllib.feature.ChiSqSelectorModel" def save(sc: SparkContext, model: ChiSqSelectorModel, path: String): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) @@ -149,7 +149,7 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { def load(sc: SparkContext, path: String): ChiSqSelectorModel = { implicit val formats = DefaultFormats - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 9bd79aa7c627e..2f52825c6cb01 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -609,7 +609,7 @@ object Word2VecModel extends Loader[Word2VecModel] { case class Data(word: String, vector: Array[Float]) def load(sc: SparkContext, path: String): Word2VecModel = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataFrame = spark.read.parquet(Loader.dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[Data](dataFrame.schema) @@ -620,7 +620,7 @@ object Word2VecModel extends Loader[Word2VecModel] { } def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val vectorSize = model.values.head.length val numWords = model.size diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 8c0639baeaca4..0f7fbe9556c5d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -99,7 +99,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { def save(model: FPGrowthModel[_], path: String): Unit = { val sc = model.freqItemsets.sparkContext - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) @@ -123,7 +123,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { def load(sc: SparkContext, path: String): FPGrowthModel[_] = { implicit val formats = DefaultFormats - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index 10bbcd2a3d924..c13c794775fec 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -616,7 +616,7 @@ object PrefixSpanModel extends Loader[PrefixSpanModel[_]] { def save(model: PrefixSpanModel[_], path: String): Unit = { val sc = model.freqSequences.sparkContext - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) @@ -640,7 +640,7 @@ object PrefixSpanModel extends Loader[PrefixSpanModel[_]] { def load(sc: SparkContext, path: String): PrefixSpanModel[_] = { implicit val formats = DefaultFormats - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 450025f477f19..c642573ccba6d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -354,7 +354,7 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { */ def save(model: MatrixFactorizationModel, path: String): Unit = { val sc = model.userFeatures.sparkContext - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() import spark.implicits._ val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rank" -> model.rank))) @@ -365,7 +365,7 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { def load(sc: SparkContext, path: String): MatrixFactorizationModel = { implicit val formats = DefaultFormats - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 215a799b9646a..1cd6f2a8969a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -185,7 +185,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { boundaries: Array[Double], predictions: Array[Double], isotonic: Boolean): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ @@ -198,7 +198,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { } def load(sc: SparkContext, path: String): (Array[Double], Array[Double]) = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataRDD = spark.read.parquet(dataPath(path)) checkSchema[Data](dataRDD.schema) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala index 3c7bbc52446d9..cd90e97cc5388 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala @@ -47,7 +47,7 @@ private[regression] object GLMRegressionModel { modelClass: String, weights: Vector, intercept: Double): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -68,7 +68,7 @@ private[regression] object GLMRegressionModel { */ def loadData(sc: SparkContext, path: String, modelClass: String, numFeatures: Int): Data = { val dataPath = Loader.dataPath(path) - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataRDD = spark.read.parquet(dataPath) val dataArray = dataRDD.select("weights", "intercept").take(1) assert(dataArray.length == 1, s"Unable to load $modelClass data from: $dataPath") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 72663188a98ae..a1562384b0a7e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -233,13 +233,13 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { // Create Parquet data. val nodes = model.topNode.subtreeIterator.toSeq val dataRDD = sc.parallelize(nodes).map(NodeData.apply(0, _)) - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() spark.createDataFrame(dataRDD).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String, algo: String, numNodes: Int): DecisionTreeModel = { // Load Parquet data. - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val dataPath = Loader.dataPath(path) val dataRDD = spark.read.parquet(dataPath) // Check schema explicitly since erasure makes it hard to use match-case for checking. 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 c653b988e21e7..f7d9b22b6f424 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 @@ -413,7 +413,7 @@ private[tree] object TreeEnsembleModel extends Logging { case class EnsembleNodeData(treeId: Int, node: NodeData) def save(sc: SparkContext, path: String, model: TreeEnsembleModel, className: String): Unit = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // SPARK-6120: We do a hacky check here so users understand why save() is failing // when they run the ML guide example. @@ -471,7 +471,7 @@ private[tree] object TreeEnsembleModel extends Logging { sc: SparkContext, path: String, treeAlgo: String): Array[DecisionTreeModel] = { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() import spark.implicits._ val nodes = spark.read.parquet(Loader.dataPath(path)).map(NodeData.apply) val trees = constructTrees(nodes.rdd) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index 40d5b4881f839..3558290b23ae0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -23,18 +23,14 @@ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.Row class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("Test Chi-Square selector") { - val spark = SparkSession.builder - .master("local[2]") - .appName("ChiSqSelectorSuite") - .getOrCreate() + val spark = this.spark import spark.implicits._ - val data = Seq( LabeledPoint(0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0)))), LabeledPoint(1.0, Vectors.sparse(3, Array((1, 9.0), (2, 6.0)))), diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala index 621c13a8e5ac6..b73dbd62328cf 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala @@ -27,7 +27,7 @@ class QuantileDiscretizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("Test observed number of buckets and their sizes match expected values") { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = this.spark import spark.implicits._ val datasetSize = 100000 @@ -53,7 +53,7 @@ class QuantileDiscretizerSuite } test("Test transform method on unseen data") { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = this.spark import spark.implicits._ val trainDF = sc.parallelize(1.0 to 100.0 by 1.0).map(Tuple1.apply).toDF("input") @@ -82,7 +82,7 @@ class QuantileDiscretizerSuite } test("Verify resulting model has parent") { - val spark = SparkSession.builder().config(sc.getConf).getOrCreate() + val spark = this.spark import spark.implicits._ val df = sc.parallelize(1 to 100).map(Tuple1.apply).toDF("input") diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 59b5edc4013e8..e8ed50acf877c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -591,6 +591,7 @@ class ALSCleanerSuite extends SparkFunSuite { val spark = SparkSession.builder .master("local[2]") .appName("ALSCleanerSuite") + .sparkContext(sc) .getOrCreate() import spark.implicits._ val als = new ALS() @@ -606,7 +607,7 @@ class ALSCleanerSuite extends SparkFunSuite { val pattern = "shuffle_(\\d+)_.+\\.data".r val rddIds = resultingFiles.flatMap { f => pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } - assert(rddIds.toSet.size === 4) + assert(rddIds.size === 4) } finally { sc.stop() } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala index 8cbd652bacf31..d2fa8d0d6335d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala @@ -42,9 +42,10 @@ private[ml] object TreeTests extends SparkFunSuite { data: RDD[LabeledPoint], categoricalFeatures: Map[Int, Int], numClasses: Int): DataFrame = { - val spark = SparkSession.builder + val spark = SparkSession.builder() .master("local[2]") .appName("TreeTests") + .sparkContext(data.sparkContext) .getOrCreate() import spark.implicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 20e22baa351a9..dc4b72a6fb33a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -694,7 +694,7 @@ object SparkSession { private[this] var userSuppliedContext: Option[SparkContext] = None - private[sql] def sparkContext(sparkContext: SparkContext): Builder = synchronized { + private[spark] def sparkContext(sparkContext: SparkContext): Builder = synchronized { userSuppliedContext = Option(sparkContext) this } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 2069652adebd0..b679e3bb328af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.execution.joins @@ -46,11 +46,10 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { */ override def beforeAll(): Unit = { super.beforeAll() - val conf = new SparkConf() - .setMaster("local-cluster[2,1,1024]") - .setAppName("testing") - val sc = new SparkContext(conf) - spark = SparkSession.builder.getOrCreate() + spark = SparkSession.builder() + .master("local-cluster[2,1,1024]") + .appName("testing") + .getOrCreate() } override def afterAll(): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index a4bbe96cf8057..d56bede0cc2fd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} +import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, FunctionResource, JarResource} import org.apache.spark.sql.expressions.Window @@ -282,15 +282,12 @@ object SetWarehouseLocationTest extends Logging { val hiveWarehouseLocation = Utils.createTempDir() hiveWarehouseLocation.delete() - val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") // We will use the value of spark.sql.warehouse.dir override the // value of hive.metastore.warehouse.dir. - conf.set("spark.sql.warehouse.dir", warehouseLocation.toString) - conf.set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString) - - val sparkSession = SparkSession.builder - .config(conf) + val sparkSession = SparkSession.builder() + .config("spark.ui.enabled", "false") + .config("spark.sql.warehouse.dir", warehouseLocation.toString) + .config("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString) .enableHiveSupport() .getOrCreate() From db96f398be338a937ef369515615f7fe7bb439a5 Mon Sep 17 00:00:00 2001 From: xin Wu Date: Tue, 31 May 2016 17:42:47 -0700 Subject: [PATCH 0467/1470] [SPARK-15236][SQL][SPARK SHELL] Add spark-defaults property to switch to use InMemoryCatalog ## What changes were proposed in this pull request? This PR change REPL/Main to check this property `spark.sql.catalogImplementation` to decide if `enableHiveSupport `should be called. If `spark.sql.catalogImplementation` is set to `hive`, and hive classes are built, Spark will use Hive support. Other wise, Spark will create a SparkSession with in-memory catalog support. ## How was this patch tested? Run the REPL component test. Author: xin Wu Author: Xin Wu Closes #13088 from xwu0226/SPARK-15236. (cherry picked from commit 04f925ede851fc77add9ef1cacb79fb3a617f650) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/repl/Main.scala | 20 ++++++-- .../org/apache/spark/repl/ReplSuite.scala | 50 ++++++++++++++++++- 2 files changed, 66 insertions(+), 4 deletions(-) diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 005edda2bee76..771670fa559a0 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -22,6 +22,7 @@ import java.io.File import scala.tools.nsc.GenericRunnerSettings import org.apache.spark._ +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils @@ -88,10 +89,23 @@ object Main extends Logging { } val builder = SparkSession.builder.config(conf) - if (SparkSession.hiveClassesArePresent) { - sparkSession = builder.enableHiveSupport().getOrCreate() - logInfo("Created Spark session with Hive support") + if (conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase == "hive") { + if (SparkSession.hiveClassesArePresent) { + // In the case that the property is not set at all, builder's config + // does not have this value set to 'hive' yet. The original default + // behavior is that when there are hive classes, we use hive catalog. + sparkSession = builder.enableHiveSupport().getOrCreate() + logInfo("Created Spark session with Hive support") + } else { + // Need to change it back to 'in-memory' if no hive classes are found + // in the case that the property is set to hive in spark-defaults.conf + builder.config(CATALOG_IMPLEMENTATION.key, "in-memory") + sparkSession = builder.getOrCreate() + logInfo("Created Spark session") + } } else { + // In the case that the property is set but not to 'hive', the internal + // default is 'in-memory'. So the sparkSession will use in-memory catalog. sparkSession = builder.getOrCreate() logInfo("Created Spark session") } diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index af82e7a111fa8..125686030c01f 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -21,9 +21,11 @@ import java.io._ import java.net.URLClassLoader import scala.collection.mutable.ArrayBuffer - import org.apache.commons.lang3.StringEscapeUtils +import org.apache.log4j.{Level, LogManager} import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils class ReplSuite extends SparkFunSuite { @@ -99,6 +101,52 @@ class ReplSuite extends SparkFunSuite { System.clearProperty("spark.driver.port") } + test("SPARK-15236: use Hive catalog") { + // turn on the INFO log so that it is possible the code will dump INFO + // entry for using "HiveMetastore" + val rootLogger = LogManager.getRootLogger() + val logLevel = rootLogger.getLevel + rootLogger.setLevel(Level.INFO) + try { + Main.conf.set(CATALOG_IMPLEMENTATION.key, "hive") + val output = runInterpreter("local", + """ + |spark.sql("drop table if exists t_15236") + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + // only when the config is set to hive and + // hive classes are built, we will use hive catalog. + // Then log INFO entry will show things using HiveMetastore + if (SparkSession.hiveClassesArePresent) { + assertContains("HiveMetaStore", output) + } else { + // If hive classes are not built, in-memory catalog will be used + assertDoesNotContain("HiveMetaStore", output) + } + } finally { + rootLogger.setLevel(logLevel) + } + } + + test("SPARK-15236: use in-memory catalog") { + val rootLogger = LogManager.getRootLogger() + val logLevel = rootLogger.getLevel + rootLogger.setLevel(Level.INFO) + try { + Main.conf.set(CATALOG_IMPLEMENTATION.key, "in-memory") + val output = runInterpreter("local", + """ + |spark.sql("drop table if exists t_16236") + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertDoesNotContain("HiveMetaStore", output) + } finally { + rootLogger.setLevel(logLevel) + } + } + test("simple foreach with accumulator") { val output = runInterpreter("local", """ From d34c0fc1056af08c226eea3d2927838518465a51 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Tue, 31 May 2016 19:52:22 -0500 Subject: [PATCH 0468/1470] [SPARK-15601][CORE] CircularBuffer's toString() to print only the contents written if buffer isn't full ## What changes were proposed in this pull request? 1. The class allocated 4x space than needed as it was using `Int` to store the `Byte` values 2. If CircularBuffer isn't full, currently toString() will print some garbage chars along with the content written as is tries to print the entire array allocated for the buffer. The fix is to keep track of buffer getting full and don't print the tail of the buffer if it isn't full (suggestion by sameeragarwal over https://github.com/apache/spark/pull/12194#discussion_r64495331) 3. Simplified `toString()` ## How was this patch tested? Added new test case Author: Tejas Patil Closes #13351 from tejasapatil/circular_buffer. (cherry picked from commit ac38bdc756c25632069e7887a657250fe2fd6d82) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/util/Utils.scala | 31 +++++++--------- .../org/apache/spark/util/UtilsSuite.scala | 37 +++++++++++++++---- 2 files changed, 43 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 7e204fa21852c..1a9dbcae8c083 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2344,29 +2344,24 @@ private[spark] class RedirectThread( * the toString method. */ private[spark] class CircularBuffer(sizeInBytes: Int = 10240) extends java.io.OutputStream { - var pos: Int = 0 - var buffer = new Array[Int](sizeInBytes) + private var pos: Int = 0 + private var isBufferFull = false + private val buffer = new Array[Byte](sizeInBytes) - def write(i: Int): Unit = { - buffer(pos) = i + def write(input: Int): Unit = { + buffer(pos) = input.toByte pos = (pos + 1) % buffer.length + isBufferFull = isBufferFull || (pos == 0) } override def toString: String = { - val (end, start) = buffer.splitAt(pos) - val input = new java.io.InputStream { - val iterator = (start ++ end).iterator - - def read(): Int = if (iterator.hasNext) iterator.next() else -1 - } - val reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8)) - val stringBuilder = new StringBuilder - var line = reader.readLine() - while (line != null) { - stringBuilder.append(line) - stringBuilder.append("\n") - line = reader.readLine() + if (!isBufferFull) { + return new String(buffer, 0, pos, StandardCharsets.UTF_8) } - stringBuilder.toString() + + val nonCircularBuffer = new Array[Byte](sizeInBytes) + System.arraycopy(buffer, pos, nonCircularBuffer, 0, buffer.length - pos) + System.arraycopy(buffer, 0, nonCircularBuffer, buffer.length - pos, pos) + new String(nonCircularBuffer, StandardCharsets.UTF_8) } } 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 4aa4854c36f3a..66987498669d4 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File, FileOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File, FileOutputStream, PrintStream} import java.lang.{Double => JDouble, Float => JFloat} import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} @@ -681,14 +681,37 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(!Utils.isInDirectory(nullFile, childFile3)) } - test("circular buffer") { + test("circular buffer: if nothing was written to the buffer, display nothing") { + val buffer = new CircularBuffer(4) + assert(buffer.toString === "") + } + + test("circular buffer: if the buffer isn't full, print only the contents written") { + val buffer = new CircularBuffer(10) + val stream = new PrintStream(buffer, true, "UTF-8") + stream.print("test") + assert(buffer.toString === "test") + } + + test("circular buffer: data written == size of the buffer") { + val buffer = new CircularBuffer(4) + val stream = new PrintStream(buffer, true, "UTF-8") + + // fill the buffer to its exact size so that it just hits overflow + stream.print("test") + assert(buffer.toString === "test") + + // add more data to the buffer + stream.print("12") + assert(buffer.toString === "st12") + } + + test("circular buffer: multiple overflow") { val buffer = new CircularBuffer(25) - val stream = new java.io.PrintStream(buffer, true, "UTF-8") + val stream = new PrintStream(buffer, true, "UTF-8") - // scalastyle:off println - stream.println("test circular test circular test circular test circular test circular") - // scalastyle:on println - assert(buffer.toString === "t circular test circular\n") + stream.print("test circular test circular test circular test circular test circular") + assert(buffer.toString === "st circular test circular") } test("nanSafeCompareDoubles") { From 0ade44cc45ca01899e81cc2bc377cc5fa63e914b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 31 May 2016 20:06:08 -0700 Subject: [PATCH 0469/1470] [HOTFIX] DDLSuite was broken by 93e9714 --- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 5d45cfb501c73..dd1f59880701d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1179,11 +1179,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { var message = intercept[AnalysisException] { sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") }.getMessage - assert(message.contains("Please enable Hive support when inserting the regular tables")) + assert(message.contains("Hive support is required to insert into the following tables")) message = intercept[AnalysisException] { sql(s"SELECT * FROM $tabName") }.getMessage - assert(message.contains("Please enable Hive support when selecting the regular tables")) + assert(message.contains("Hive support is required to select over the following tables")) } } @@ -1205,11 +1205,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { var message = intercept[AnalysisException] { sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1, 'a'") }.getMessage - assert(message.contains("Please enable Hive support when inserting the regular tables")) + assert(message.contains("Hive support is required to insert into the following tables")) message = intercept[AnalysisException] { sql(s"SELECT * FROM $tabName") }.getMessage - assert(message.contains("Please enable Hive support when selecting the regular tables")) + assert(message.contains("Hive support is required to select over the following tables")) } } } From 29a1cdfc4fee9c8e4527bf7d8b4d8e974417f810 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 1 Jun 2016 08:28:19 -0500 Subject: [PATCH 0470/1470] [SPARK-15659][SQL] Ensure FileSystem is gotten from path ## What changes were proposed in this pull request? Currently `spark.sql.warehouse.dir` is pointed to local dir by default, which will throw exception when HADOOP_CONF_DIR is configured and default FS is hdfs. ``` java.lang.IllegalArgumentException: Wrong FS: file:/Users/sshao/projects/apache-spark/spark-warehouse, expected: hdfs://localhost:8020 ``` So we should always get the `FileSystem` from `Path` to avoid wrong FS problem. ## How was this patch tested? Local test. Author: jerryshao Closes #13405 from jerryshao/SPARK-15659. (cherry picked from commit e4ce1bc4f3ca088365ff199e563f23a552dc88ef) Signed-off-by: Sean Owen --- .../sql/catalyst/catalog/InMemoryCatalog.scala | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 489a1c8c3facd..60525794edc5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -22,7 +22,7 @@ import java.io.IOException import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException @@ -105,8 +105,6 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E } } - private val fs = FileSystem.get(hadoopConfig) - // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- @@ -120,7 +118,9 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E } } else { try { - fs.mkdirs(new Path(dbDefinition.locationUri)) + val location = new Path(dbDefinition.locationUri) + val fs = location.getFileSystem(hadoopConfig) + fs.mkdirs(location) } catch { case e: IOException => throw new SparkException(s"Unable to create database ${dbDefinition.name} as failed " + @@ -147,7 +147,9 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E // Remove the database. val dbDefinition = catalog(db).db try { - fs.delete(new Path(dbDefinition.locationUri), true) + val location = new Path(dbDefinition.locationUri) + val fs = location.getFileSystem(hadoopConfig) + fs.delete(location, true) } catch { case e: IOException => throw new SparkException(s"Unable to drop database ${dbDefinition.name} as failed " + @@ -203,6 +205,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E if (tableDefinition.tableType == CatalogTableType.MANAGED) { val dir = new Path(catalog(db).db.locationUri, table) try { + val fs = dir.getFileSystem(hadoopConfig) fs.mkdirs(dir) } catch { case e: IOException => @@ -223,6 +226,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E if (getTable(db, table).tableType == CatalogTableType.MANAGED) { val dir = new Path(catalog(db).db.locationUri, table) try { + val fs = dir.getFileSystem(hadoopConfig) fs.delete(dir, true) } catch { case e: IOException => @@ -248,6 +252,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E val oldDir = new Path(catalog(db).db.locationUri, oldName) val newDir = new Path(catalog(db).db.locationUri, newName) try { + val fs = oldDir.getFileSystem(hadoopConfig) fs.rename(oldDir, newDir) } catch { case e: IOException => @@ -338,6 +343,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E p.spec.get(col).map(col + "=" + _) }.mkString("/") try { + val fs = tableDir.getFileSystem(hadoopConfig) fs.mkdirs(new Path(tableDir, partitionPath)) } catch { case e: IOException => @@ -373,6 +379,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E p.get(col).map(col + "=" + _) }.mkString("/") try { + val fs = tableDir.getFileSystem(hadoopConfig) fs.delete(new Path(tableDir, partitionPath), true) } catch { case e: IOException => @@ -409,6 +416,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E newSpec.get(col).map(col + "=" + _) }.mkString("/") try { + val fs = tableDir.getFileSystem(hadoopConfig) fs.rename(new Path(tableDir, oldPath), new Path(tableDir, newPath)) } catch { case e: IOException => From 8fb125bdf20c2c9ad824d05a893af4f492ced0f1 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 1 Jun 2016 08:30:38 -0500 Subject: [PATCH 0471/1470] [SPARK-15664][MLLIB] Replace FileSystem.get(conf) with path.getFileSystem(conf) when removing CheckpointFile in MLlib ## What changes were proposed in this pull request? if sparkContext.set CheckpointDir to another Dir that is not default FileSystem, it will throw exception when removing CheckpointFile in MLlib. So we should always get the FileSystem from Path to avoid wrong FS problem. ## How was this patch tested? N/A Author: Lianhui Wang Closes #13408 from lianhuiwang/SPARK-15664. (cherry picked from commit 6563d72b168c39115376e73788b48a2d60803d4e) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/clustering/LDA.scala | 6 +++--- .../apache/spark/ml/tree/impl/NodeIdCache.scala | 14 +++++++++----- .../spark/mllib/impl/PeriodicCheckpointer.scala | 15 +++++++++------ .../impl/PeriodicGraphCheckpointerSuite.scala | 8 +++++--- .../mllib/impl/PeriodicRDDCheckpointerSuite.scala | 8 +++++--- 5 files changed, 31 insertions(+), 20 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index ec60991af64ff..5aec692c98e6e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.clustering -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging @@ -696,8 +696,8 @@ class DistributedLDAModel private[ml] ( @DeveloperApi @Since("2.0.0") def deleteCheckpointFiles(): Unit = { - val fs = FileSystem.get(sparkSession.sparkContext.hadoopConfiguration) - _checkpointFiles.foreach(PeriodicCheckpointer.removeCheckpointFile(_, fs)) + val hadoopConf = sparkSession.sparkContext.hadoopConfiguration + _checkpointFiles.foreach(PeriodicCheckpointer.removeCheckpointFile(_, hadoopConf)) _checkpointFiles = Array.empty[String] } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala index 800430f96c5b1..a7c5f489dea86 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala @@ -21,7 +21,7 @@ import java.io.IOException import scala.collection.mutable -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.ml.tree.{LearningNode, Split} @@ -77,8 +77,8 @@ private[spark] class NodeIdCache( // Indicates whether we can checkpoint private val canCheckpoint = nodeIdsForInstances.sparkContext.getCheckpointDir.nonEmpty - // FileSystem instance for deleting checkpoints as needed - private val fs = FileSystem.get(nodeIdsForInstances.sparkContext.hadoopConfiguration) + // Hadoop Configuration for deleting checkpoints as needed + private val hadoopConf = nodeIdsForInstances.sparkContext.hadoopConfiguration /** * Update the node index values in the cache. @@ -130,7 +130,9 @@ private[spark] class NodeIdCache( val old = checkpointQueue.dequeue() // Since the old checkpoint is not deleted by Spark, we'll manually delete it here. try { - fs.delete(new Path(old.getCheckpointFile.get), true) + val path = new Path(old.getCheckpointFile.get) + val fs = path.getFileSystem(hadoopConf) + fs.delete(path, true) } catch { case e: IOException => logError("Decision Tree learning using cacheNodeIds failed to remove checkpoint" + @@ -154,7 +156,9 @@ private[spark] class NodeIdCache( val old = checkpointQueue.dequeue() if (old.getCheckpointFile.isDefined) { try { - fs.delete(new Path(old.getCheckpointFile.get), true) + val path = new Path(old.getCheckpointFile.get) + val fs = path.getFileSystem(hadoopConf) + fs.delete(path, true) } catch { case e: IOException => logError("Decision Tree learning using cacheNodeIds failed to remove checkpoint" + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala index 5c12c9305b99c..4dd498cd91b4e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.impl import scala.collection.mutable -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext import org.apache.spark.internal.Logging @@ -160,21 +161,23 @@ private[mllib] abstract class PeriodicCheckpointer[T]( private def removeCheckpointFile(): Unit = { val old = checkpointQueue.dequeue() // Since the old checkpoint is not deleted by Spark, we manually delete it. - val fs = FileSystem.get(sc.hadoopConfiguration) - getCheckpointFiles(old).foreach(PeriodicCheckpointer.removeCheckpointFile(_, fs)) + getCheckpointFiles(old).foreach( + PeriodicCheckpointer.removeCheckpointFile(_, sc.hadoopConfiguration)) } } private[spark] object PeriodicCheckpointer extends Logging { /** Delete a checkpoint file, and log a warning if deletion fails. */ - def removeCheckpointFile(path: String, fs: FileSystem): Unit = { + def removeCheckpointFile(checkpointFile: String, conf: Configuration): Unit = { try { - fs.delete(new Path(path), true) + val path = new Path(checkpointFile) + val fs = path.getFileSystem(conf) + fs.delete(path, true) } catch { case e: Exception => logWarning("PeriodicCheckpointer could not remove old checkpoint file: " + - path) + checkpointFile) } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala index e331c75989187..a13e7f63a9296 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.impl -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.graphx.{Edge, Graph} @@ -140,9 +140,11 @@ private object PeriodicGraphCheckpointerSuite { // Instead, we check for the presence of the checkpoint files. // This test should continue to work even after this graph.isCheckpointed issue // is fixed (though it can then be simplified and not look for the files). - val fs = FileSystem.get(graph.vertices.sparkContext.hadoopConfiguration) + val hadoopConf = graph.vertices.sparkContext.hadoopConfiguration graph.getCheckpointFiles.foreach { checkpointFile => - assert(!fs.exists(new Path(checkpointFile)), + val path = new Path(checkpointFile) + val fs = path.getFileSystem(hadoopConf) + assert(!fs.exists(path), "Graph checkpoint file should have been removed") } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala index b2a459a68b5fa..14adf8c29fc6b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.impl -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -127,9 +127,11 @@ private object PeriodicRDDCheckpointerSuite { // Instead, we check for the presence of the checkpoint files. // This test should continue to work even after this rdd.isCheckpointed issue // is fixed (though it can then be simplified and not look for the files). - val fs = FileSystem.get(rdd.sparkContext.hadoopConfiguration) + val hadoopConf = rdd.sparkContext.hadoopConfiguration rdd.getCheckpointFile.foreach { checkpointFile => - assert(!fs.exists(new Path(checkpointFile)), "RDD checkpoint file should have been removed") + val path = new Path(checkpointFile) + val fs = path.getFileSystem(hadoopConf) + assert(!fs.exists(path), "RDD checkpoint file should have been removed") } } From cb254ecb1f351c00f7fe4c3c9cc41c46beda90b4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 1 Jun 2016 07:30:55 -0700 Subject: [PATCH 0472/1470] [SPARK-14343][SQL] Proper column pruning for text data source ## What changes were proposed in this pull request? Text data source ignores requested schema, and may give wrong result when the only data column is not requested. This may happen when only partitioning column(s) are requested for a partitioned text table. ## How was this patch tested? New test case added in `TextSuite`. Author: Cheng Lian Closes #13431 from liancheng/spark-14343-partitioned-text-table. (cherry picked from commit 1f43562daf9454428796317203d9dcc9030a46eb) Signed-off-by: Cheng Lian --- .../datasources/text/TextFileFormat.scala | 31 +++++++++++++------ .../datasources/text/TextSuite.scala | 17 ++++++++-- 2 files changed, 35 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index 1e5bce4a75978..9c03ab28dd769 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -92,20 +92,31 @@ class TextFileFormat extends FileFormat with DataSourceRegister { filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + assert( + requiredSchema.length <= 1, + "Text data source only produces a single data column named \"value\".") + val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) (file: PartitionedFile) => { - val unsafeRow = new UnsafeRow(1) - val bufferHolder = new BufferHolder(unsafeRow) - val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1) - - new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map { line => - // Writes to an UnsafeRow directly - bufferHolder.reset() - unsafeRowWriter.write(0, line.getBytes, 0, line.getLength) - unsafeRow.setTotalSize(bufferHolder.totalSize()) - unsafeRow + val reader = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value) + + if (requiredSchema.isEmpty) { + val emptyUnsafeRow = new UnsafeRow(0) + reader.map(_ => emptyUnsafeRow) + } else { + val unsafeRow = new UnsafeRow(1) + val bufferHolder = new BufferHolder(unsafeRow) + val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1) + + reader.map { line => + // Writes to an UnsafeRow directly + bufferHolder.reset() + unsafeRowWriter.write(0, line.getBytes, 0, line.getLength) + unsafeRow.setTotalSize(bufferHolder.totalSize()) + unsafeRow + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index b5e51e963f1b6..7b6981f95e9dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark.sql.execution.datasources.text import java.io.File -import scala.collection.JavaConverters._ - -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec @@ -31,6 +28,7 @@ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.Utils class TextSuite extends QueryTest with SharedSQLContext { + import testImplicits._ test("reading text file") { verifyFrame(spark.read.format("text").load(testFile)) @@ -126,6 +124,19 @@ class TextSuite extends QueryTest with SharedSQLContext { } } + test("SPARK-14343: select partitioning column") { + withTempPath { dir => + val path = dir.getCanonicalPath + val ds1 = spark.range(1).selectExpr("CONCAT('val_', id)") + ds1.write.text(s"$path/part=a") + ds1.write.text(s"$path/part=b") + + checkDataset( + spark.read.format("text").load(path).select($"part"), + Row("a"), Row("b")) + } + } + private def testFile: String = { Thread.currentThread().getContextClassLoader.getResource("text-suite.txt").toString } From 9406a3c9a990bfc953e3bdc1ce558ae220176fa8 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 1 Jun 2016 09:58:01 -0700 Subject: [PATCH 0473/1470] [SPARK-15495][SQL] Improve the explain output for Aggregation operator ## What changes were proposed in this pull request? This PR improves the explain output of Aggregator operator. SQL: ``` Seq((1,2,3)).toDF("a", "b", "c").createTempView("df1") spark.sql("cache table df1") spark.sql("select count(a), count(c), b from df1 group by b").explain() ``` **Before change:** ``` *TungstenAggregate(key=[b#8], functions=[count(1),count(1)], output=[count(a)#79L,count(c)#80L,b#8]) +- Exchange hashpartitioning(b#8, 200), None +- *TungstenAggregate(key=[b#8], functions=[partial_count(1),partial_count(1)], output=[b#8,count#98L,count#99L]) +- InMemoryTableScan [b#8], InMemoryRelation [a#7,b#8,c#9], true, 10000, StorageLevel(disk=true, memory=true, offheap=false, deserialized=true, replication=1), LocalTableScan [a#7,b#8,c#9], [[1,2,3]], Some(df1) `````` **After change:** ``` *Aggregate(key=[b#8], functions=[count(1),count(1)], output=[count(a)#79L,count(c)#80L,b#8]) +- Exchange hashpartitioning(b#8, 200), None +- *Aggregate(key=[b#8], functions=[partial_count(1),partial_count(1)], output=[b#8,count#98L,count#99L]) +- InMemoryTableScan [b#8], InMemoryRelation [a#7,b#8,c#9], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), LocalTableScan [a#7,b#8,c#9], [[1,2,3]], Some(df1) ``` ## How was this patch tested? Manual test and existing UT. Author: Sean Zhong Closes #13363 from clockfly/verbose3. (cherry picked from commit d5012c274036463c47a751cfe9977ade3a68b668) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/storage/StorageLevel.scala | 10 ++++++++-- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../execution/aggregate/SortBasedAggregateExec.scala | 2 +- .../sql/execution/aggregate/TungstenAggregate.scala | 4 ++-- 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 216ec0793492f..fad0404bebc36 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -120,8 +120,14 @@ class StorageLevel private( private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) override def toString: String = { - s"StorageLevel(disk=$useDisk, memory=$useMemory, offheap=$useOffHeap, " + - s"deserialized=$deserialized, replication=$replication)" + val disk = if (useDisk) "disk" else "" + val memory = if (useMemory) "memory" else "" + val heap = if (useOffHeap) "offheap" else "" + val deserialize = if (deserialized) "deserialized" else "" + + val output = + Seq(disk, memory, heap, deserialize, s"$replication replicas").filter(_.nonEmpty) + s"StorageLevel(${output.mkString(", ")})" } override def hashCode(): Int = toInt * 41 + replication 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 e8e2a7bbabcd4..d87e6c76ed734 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 @@ -434,7 +434,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case other => other :: Nil }.mkString(", ") - /** String representation of this node without any children. */ + /** ONE line description of this node. */ def simpleString: String = s"$nodeName $argString".trim override def toString: String = treeString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala index 2e74d59c5f5b6..af1fb4c604c81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala @@ -106,6 +106,6 @@ case class SortBasedAggregateExec( val keyString = groupingExpressions.mkString("[", ",", "]") val functionString = allAggregateExpressions.mkString("[", ",", "]") val outputString = output.mkString("[", ",", "]") - s"SortBasedAggregate(key=$keyString, functions=$functionString, output=$outputString)" + s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 905e93c15dce6..091177959bedb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -769,9 +769,9 @@ case class TungstenAggregate( val keyString = groupingExpressions.mkString("[", ",", "]") val functionString = allAggregateExpressions.mkString("[", ",", "]") val outputString = output.mkString("[", ",", "]") - s"TungstenAggregate(key=$keyString, functions=$functionString, output=$outputString)" + s"Aggregate(key=$keyString, functions=$functionString, output=$outputString)" case Some(fallbackStartsAt) => - s"TungstenAggregateWithControlledFallback $groupingExpressions " + + s"AggregateWithControlledFallback $groupingExpressions " + s"$allAggregateExpressions $resultExpressions fallbackStartsAt=$fallbackStartsAt" } } From a780848afb0dc47069189c43ea91cfd4c148097f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 Jun 2016 10:14:40 -0700 Subject: [PATCH 0474/1470] [SPARK-15686][SQL] Move user-facing streaming classes into sql.streaming ## What changes were proposed in this pull request? This patch moves all user-facing structured streaming classes into sql.streaming. As part of this, I also added some since version annotation to methods and classes that don't have them. ## How was this patch tested? Updated tests to reflect the moves. Author: Reynold Xin Closes #13429 from rxin/SPARK-15686. (cherry picked from commit a71d1364ae87aa388128da34dd0b9b02ff85e458) Signed-off-by: Michael Armbrust --- python/pyspark/sql/streaming.py | 3 +- python/pyspark/sql/utils.py | 2 +- .../spark/sql/{ => streaming}/OutputMode.java | 3 +- .../spark/sql/InternalOutputModes.scala | 2 ++ .../UnsupportedOperationChecker.scala | 3 +- .../{ => streaming}/JavaOutputModeSuite.java | 2 +- .../analysis/UnsupportedOperationsSuite.scala | 3 +- .../apache/spark/sql/DataFrameWriter.scala | 1 + .../scala/org/apache/spark/sql/Dataset.scala | 1 + .../org/apache/spark/sql/SQLContext.scala | 6 ++-- .../org/apache/spark/sql/SparkSession.scala | 7 ++-- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../execution/datasources/DataSource.scala | 1 + .../ContinuousQueryListenerBus.scala | 11 +++--- .../streaming/IncrementalExecution.scala | 3 +- .../execution/streaming/StreamExecution.scala | 5 +-- .../execution/streaming/TriggerExecutor.scala | 2 +- .../sql/execution/streaming/console.scala | 3 +- .../sql/execution/streaming/memory.scala | 1 + .../spark/sql/internal/SessionState.scala | 3 +- .../apache/spark/sql/sources/interfaces.scala | 1 + .../sql/{ => streaming}/ContinuousQuery.scala | 3 +- .../ContinuousQueryException.scala | 2 +- .../ContinuousQueryListener.scala | 36 ++++++++++++++----- .../ContinuousQueryManager.scala | 9 +++-- .../sql/{ => streaming}/SinkStatus.scala | 2 +- .../sql/{ => streaming}/SourceStatus.scala | 2 +- .../spark/sql/{ => streaming}/Trigger.scala | 18 ++++++++-- .../spark/sql/ProcessingTimeSuite.scala | 1 + .../ProcessingTimeExecutorSuite.scala | 2 +- .../ContinuousQueryListenerSuite.scala | 9 +++-- .../ContinuousQueryManagerSuite.scala | 5 ++- .../sql/streaming/ContinuousQuerySuite.scala | 5 ++- .../sql/streaming/FileStreamSinkSuite.scala | 2 +- .../sql/streaming/FileStreamSourceSuite.scala | 4 +-- .../spark/sql/streaming/FileStressSuite.scala | 4 +-- .../spark/sql/streaming/MemorySinkSuite.scala | 2 +- .../streaming/MemorySourceStressSuite.scala | 4 +-- .../spark/sql/streaming/StreamSuite.scala | 2 +- .../sql/{ => streaming}/StreamTest.scala | 8 +++-- .../streaming/StreamingAggregationSuite.scala | 5 ++- .../test/DataFrameReaderWriterSuite.scala | 4 +-- 42 files changed, 121 insertions(+), 74 deletions(-) rename sql/catalyst/src/main/java/org/apache/spark/sql/{ => streaming}/OutputMode.java (95%) rename sql/catalyst/src/test/java/org/apache/spark/sql/{ => streaming}/JavaOutputModeSuite.java (96%) rename sql/core/src/main/scala/org/apache/spark/sql/{ => streaming}/ContinuousQuery.scala (97%) rename sql/core/src/main/scala/org/apache/spark/sql/{ => streaming}/ContinuousQueryException.scala (98%) rename sql/core/src/main/scala/org/apache/spark/sql/{util => streaming}/ContinuousQueryListener.scala (81%) rename sql/core/src/main/scala/org/apache/spark/sql/{ => streaming}/ContinuousQueryManager.scala (96%) rename sql/core/src/main/scala/org/apache/spark/sql/{ => streaming}/SinkStatus.scala (97%) rename sql/core/src/main/scala/org/apache/spark/sql/{ => streaming}/SourceStatus.scala (97%) rename sql/core/src/main/scala/org/apache/spark/sql/{ => streaming}/Trigger.scala (94%) rename sql/core/src/test/scala/org/apache/spark/sql/{util => streaming}/ContinuousQueryListenerSuite.scala (95%) rename sql/core/src/test/scala/org/apache/spark/sql/{ => streaming}/StreamTest.scala (98%) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 8238b8e7cde6b..cd75622cedf5e 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -201,7 +201,8 @@ def __init__(self, interval): self.interval = interval def _to_java_trigger(self, sqlContext): - return sqlContext._sc._jvm.org.apache.spark.sql.ProcessingTime.create(self.interval) + return sqlContext._sc._jvm.org.apache.spark.sql.streaming.ProcessingTime.create( + self.interval) def _test(): diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 8c8768f50bfde..9ddaf78acf91d 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -71,7 +71,7 @@ def deco(*a, **kw): raise AnalysisException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.catalyst.parser.ParseException: '): raise ParseException(s.split(': ', 1)[1], stackTrace) - if s.startswith('org.apache.spark.sql.ContinuousQueryException: '): + if s.startswith('org.apache.spark.sql.streaming.ContinuousQueryException: '): raise ContinuousQueryException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.execution.QueryExecutionException: '): raise QueryExecutionException(s.split(': ', 1)[1], stackTrace) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/OutputMode.java b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/OutputMode.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java index 1936d53e5e835..41e2582921198 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/OutputMode.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql; +package org.apache.spark.sql.streaming; import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.InternalOutputModes; /** * :: Experimental :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala index 8ef5d9a653278..153f9f57faf42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/InternalOutputModes.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.spark.sql.streaming.OutputMode + /** * Internal helper class to generate objects representing various [[OutputMode]]s, */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index f4c03476096d4..8373fa336dd4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.{AnalysisException, InternalOutputModes, OutputMode} +import org.apache.spark.sql.{AnalysisException, InternalOutputModes} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.streaming.OutputMode /** * Analyzes the presence of unsupported operations in a logical plan. diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/JavaOutputModeSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java similarity index 96% rename from sql/catalyst/src/test/java/org/apache/spark/sql/JavaOutputModeSuite.java rename to sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java index 1764f3348d8fd..e0a54fe30ac7d 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/JavaOutputModeSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql; +package org.apache.spark.sql.streaming; import org.junit.Test; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index c2e3d474506de..378cca3644eab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, OutputMode} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.InternalOutputModes._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.IntegerType /** A dummy command for testing unsupported operations. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 291b8250c9051..25678e938d846 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingA import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils import org.apache.spark.sql.execution.streaming.{MemoryPlan, MemorySink, StreamExecution} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.{ContinuousQuery, OutputMode, ProcessingTime, Trigger} import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 7be49b174983f..3a6ec4595e78e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -49,6 +49,7 @@ import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython +import org.apache.spark.sql.streaming.ContinuousQuery import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils 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 0dc70c0b1c7fe..2e14c5d486d4f 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 @@ -30,13 +30,11 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.streaming.ContinuousQueryManager import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ExecutionListenerManager @@ -645,7 +643,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * Returns a [[ContinuousQueryManager]] that allows managing all the - * [[org.apache.spark.sql.ContinuousQuery ContinuousQueries]] active on `this` context. + * [[org.apache.spark.sql.streaming.ContinuousQuery ContinuousQueries]] active on `this` context. * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index dc4b72a6fb33a..52bedf9dbddae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -25,7 +25,7 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal -import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging @@ -40,8 +40,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.ui.SQLListener -import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState, SQLConf} +import org.apache.spark.sql.internal.{CatalogImpl, SessionState, SharedState} import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.streaming._ import org.apache.spark.sql.types.{DataType, LongType, StructType} import org.apache.spark.sql.util.ExecutionListenerManager import org.apache.spark.util.Utils @@ -182,7 +183,7 @@ class SparkSession private( /** * :: Experimental :: * Returns a [[ContinuousQueryManager]] that allows managing all the - * [[org.apache.spark.sql.ContinuousQuery ContinuousQueries]] active on `this`. + * [[ContinuousQuery ContinuousQueries]] active on `this`. * * @group basic * @since 2.0.0 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 e40525287a0a1..7e3e45e56e90a 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} import org.apache.spark.sql.execution.streaming.MemoryPlan import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.ContinuousQuery private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SparkPlanner => @@ -201,7 +202,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { /** * Used to plan aggregation queries that are computed incrementally as part of a - * [[org.apache.spark.sql.ContinuousQuery]]. Currently this rule is injected into the planner + * [[ContinuousQuery]]. Currently this rule is injected into the planner * on-demand, only when planning in a [[org.apache.spark.sql.execution.streaming.StreamExecution]] */ object StatefulAggregationStrategy extends Strategy { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 814880b0e0f40..93f1ad01bf9aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{CalendarIntervalType, StructType} import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala index b1d24b6cfc0bd..2a1be09693e51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerEvent} -import org.apache.spark.sql.util.ContinuousQueryListener -import org.apache.spark.sql.util.ContinuousQueryListener._ +import org.apache.spark.sql.streaming.ContinuousQueryListener import org.apache.spark.util.ListenerBus /** @@ -30,7 +29,10 @@ import org.apache.spark.util.ListenerBus * dispatch them to ContinuousQueryListener. */ class ContinuousQueryListenerBus(sparkListenerBus: LiveListenerBus) - extends SparkListener with ListenerBus[ContinuousQueryListener, ContinuousQueryListener.Event] { + extends SparkListener + with ListenerBus[ContinuousQueryListener, ContinuousQueryListener.Event] { + + import ContinuousQueryListener._ sparkListenerBus.addListener(this) @@ -74,7 +76,8 @@ class ContinuousQueryListenerBus(sparkListenerBus: LiveListenerBus) * listener bus. */ private case class WrappedContinuousQueryListenerEvent( - streamingListenerEvent: ContinuousQueryListener.Event) extends SparkListenerEvent { + streamingListenerEvent: ContinuousQueryListener.Event) + extends SparkListenerEvent { // Do not log streaming events in event log as history server does not support these events. protected[spark] override def logEvent: Boolean = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 5c86049851163..bc0e443ca7a55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.sql.{InternalOutputModes, OutputMode, SparkSession} +import org.apache.spark.sql.{InternalOutputModes, SparkSession} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} +import org.apache.spark.sql.streaming.OutputMode /** * A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ab0900d7f6ea6..16d38a2f7db56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -33,8 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.util.ContinuousQueryListener -import org.apache.spark.sql.util.ContinuousQueryListener._ +import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} /** @@ -54,6 +53,8 @@ class StreamExecution( val outputMode: OutputMode) extends ContinuousQuery with Logging { + import org.apache.spark.sql.streaming.ContinuousQueryListener._ + /** * A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala index 569907b369a54..ac510df209f0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging -import org.apache.spark.sql.ProcessingTime +import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.util.{Clock, SystemClock} trait TriggerExecutor { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index 391f1e54b7542..2ec2a3c3c4a49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, OutputMode, SQLContext} +import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} +import org.apache.spark.sql.streaming.OutputMode class ConsoleSink(options: Map[String, String]) extends Sink with Logging { // Number of rows to display, by default 20 rows diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index e4a95e7335303..4496f41615a4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType object MemoryStream { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 4c7bbf04bc72a..b2db377ec7f8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, FindDataSourceTable, PreInsertCastAndRename, ResolveDataSource} +import org.apache.spark.sql.streaming.{ContinuousQuery, ContinuousQueryManager} import org.apache.spark.sql.util.ExecutionListenerManager @@ -142,7 +143,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val listenerManager: ExecutionListenerManager = new ExecutionListenerManager /** - * Interface to start and stop [[org.apache.spark.sql.ContinuousQuery]]s. + * Interface to start and stop [[ContinuousQuery]]s. */ lazy val continuousQueryManager: ContinuousQueryManager = { new ContinuousQueryManager(sparkSession) 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 3d4edbb93d698..d2077a07f440a 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.streaming.{Sink, Source} +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/ContinuousQuery.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala index 4d5afe2eb5f4c..451cfd85e3bc8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.SparkSession /** * :: Experimental :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryException.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryException.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryException.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryException.scala index fec38629d914e..5196c5a537a71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryException.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryException.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental import org.apache.spark.sql.execution.streaming.{Offset, StreamExecution} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/ContinuousQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala similarity index 81% rename from sql/core/src/main/scala/org/apache/spark/sql/util/ContinuousQueryListener.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala index ba1facf11b7d5..6bdd513288ae1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/ContinuousQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala @@ -15,20 +15,22 @@ * limitations under the License. */ -package org.apache.spark.sql.util +package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.ContinuousQuery -import org.apache.spark.sql.util.ContinuousQueryListener._ /** * :: Experimental :: * Interface for listening to events related to [[ContinuousQuery ContinuousQueries]]. * @note The methods are not thread-safe as they may be called from different threads. + * + * @since 2.0.0 */ @Experimental abstract class ContinuousQueryListener { + import ContinuousQueryListener._ + /** * Called when a query is started. * @note This is called synchronously with @@ -36,6 +38,7 @@ abstract class ContinuousQueryListener { * that is, `onQueryStart` will be called on all listeners before * `DataFrameWriter.startStream()` returns the corresponding [[ContinuousQuery]]. Please * don't block this method as it will block your query. + * @since 2.0.0 */ def onQueryStarted(queryStarted: QueryStarted): Unit @@ -46,10 +49,14 @@ abstract class ContinuousQueryListener { * latest no matter when this method is called. Therefore, the status of [[ContinuousQuery]] * may be changed before/when you process the event. E.g., you may find [[ContinuousQuery]] * is terminated when you are processing [[QueryProgress]]. + * @since 2.0.0 */ def onQueryProgress(queryProgress: QueryProgress): Unit - /** Called when a query is stopped, with or without error */ + /** + * Called when a query is stopped, with or without error. + * @since 2.0.0 + */ def onQueryTerminated(queryTerminated: QueryTerminated): Unit } @@ -57,19 +64,32 @@ abstract class ContinuousQueryListener { /** * :: Experimental :: * Companion object of [[ContinuousQueryListener]] that defines the listener events. + * @since 2.0.0 */ @Experimental object ContinuousQueryListener { - /** Base type of [[ContinuousQueryListener]] events */ + /** + * Base type of [[ContinuousQueryListener]] events. + * @since 2.0.0 + */ trait Event - /** Event representing the start of a query */ + /** + * Event representing the start of a query. + * @since 2.0.0 + */ class QueryStarted private[sql](val query: ContinuousQuery) extends Event - /** Event representing any progress updates in a query */ + /** + * Event representing any progress updates in a query. + * @since 2.0.0 + */ class QueryProgress private[sql](val query: ContinuousQuery) extends Event - /** Event representing that termination of a query */ + /** + * Event representing that termination of a query. + * @since 2.0.0 + */ class QueryTerminated private[sql](val query: ContinuousQuery) extends Event } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala index c686400150b2e..1bfdd2da4e69e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala @@ -15,22 +15,21 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.streaming import scala.collection.mutable import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.util.ContinuousQueryListener import org.apache.spark.util.{Clock, SystemClock} /** * :: Experimental :: - * A class to manage all the [[org.apache.spark.sql.ContinuousQuery ContinuousQueries]] active - * on a [[SparkSession]]. + * A class to manage all the [[ContinuousQuery]] active on a [[SparkSession]]. * * @since 2.0.0 */ @@ -147,7 +146,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { /** * Register a [[ContinuousQueryListener]] to receive up-calls for life cycle events of - * [[org.apache.spark.sql.ContinuousQuery ContinuousQueries]]. + * [[ContinuousQuery]]. * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/SinkStatus.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index 5a9852809c0eb..79ddf01042ef6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental import org.apache.spark.sql.execution.streaming.{Offset, Sink} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/SourceStatus.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 2479e67e369ec..8fccd5b7a3e4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental import org.apache.spark.sql.execution.streaming.{Offset, Source} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Trigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/Trigger.scala similarity index 94% rename from sql/core/src/main/scala/org/apache/spark/sql/Trigger.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/Trigger.scala index 256e8a47a4665..d3fdbac576b60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Trigger.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/Trigger.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.streaming import java.util.concurrent.TimeUnit @@ -29,9 +29,11 @@ import org.apache.spark.unsafe.types.CalendarInterval /** * :: Experimental :: * Used to indicate how often results should be produced by a [[ContinuousQuery]]. + * + * @since 2.0.0 */ @Experimental -sealed trait Trigger {} +sealed trait Trigger /** * :: Experimental :: @@ -53,6 +55,8 @@ sealed trait Trigger {} * import java.util.concurrent.TimeUnit * df.write.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) * }}} + * + * @since 2.0.0 */ @Experimental case class ProcessingTime(intervalMs: Long) extends Trigger { @@ -62,6 +66,8 @@ case class ProcessingTime(intervalMs: Long) extends Trigger { /** * :: Experimental :: * Used to create [[ProcessingTime]] triggers for [[ContinuousQuery]]s. + * + * @since 2.0.0 */ @Experimental object ProcessingTime { @@ -73,6 +79,8 @@ object ProcessingTime { * {{{ * df.write.trigger(ProcessingTime("10 seconds")) * }}} + * + * @since 2.0.0 */ def apply(interval: String): ProcessingTime = { if (StringUtils.isBlank(interval)) { @@ -101,6 +109,8 @@ object ProcessingTime { * import scala.concurrent.duration._ * df.write.trigger(ProcessingTime(10.seconds)) * }}} + * + * @since 2.0.0 */ def apply(interval: Duration): ProcessingTime = { new ProcessingTime(interval.toMillis) @@ -113,6 +123,8 @@ object ProcessingTime { * {{{ * df.write.trigger(ProcessingTime.create("10 seconds")) * }}} + * + * @since 2.0.0 */ def create(interval: String): ProcessingTime = { apply(interval) @@ -126,6 +138,8 @@ object ProcessingTime { * import java.util.concurrent.TimeUnit * df.write.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) * }}} + * + * @since 2.0.0 */ def create(interval: Long, unit: TimeUnit): ProcessingTime = { new ProcessingTime(unit.toMillis(interval)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala index 0d18a645f6790..52c200796ce41 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.streaming.ProcessingTime class ProcessingTimeSuite extends SparkFunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala index 7f99d303ba08a..00d5e051de357 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming import java.util.concurrent.{CountDownLatch, TimeUnit} import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.ProcessingTime +import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.util.{Clock, ManualClock, SystemClock} class ProcessingTimeExecutorSuite extends SparkFunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala similarity index 95% rename from sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala index 8788898fc89d2..cdd97da8ae33e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.util +package org.apache.spark.sql.streaming import java.util.concurrent.ConcurrentLinkedQueue @@ -26,14 +26,13 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.util.ContinuousQueryListener.{QueryProgress, QueryStarted, QueryTerminated} -class ContinuousQueryListenerSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { + +class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ + import ContinuousQueryListener._ after { spark.streams.active.foreach(_.stop()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala index b75c3ea106e4e..c1e4970b3a877 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala @@ -28,12 +28,11 @@ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException -import org.apache.spark.sql.{ContinuousQuery, Dataset, OutputMode, StreamTest} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils -class ContinuousQueryManagerSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { +class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { import AwaitTerminationTester._ import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala index f469cde6bef80..e4ca86d9d5025 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkException -import org.apache.spark.sql.StreamTest import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset, MemoryStream, StreamExecution} -import org.apache.spark.sql.test.SharedSQLContext -class ContinuousQuerySuite extends StreamTest with SharedSQLContext { + +class ContinuousQuerySuite extends StreamTest { import AwaitTerminationTester._ import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 3d8dcaf5a5322..1c73208736f78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils -class FileStreamSinkSuite extends StreamTest with SharedSQLContext { +class FileStreamSinkSuite extends StreamTest { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 1d784f1f4ee85..f681b8878d9ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -137,7 +137,7 @@ class FileStreamSourceTest extends StreamTest with SharedSQLContext { val valueSchema = new StructType().add("value", StringType) } -class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { +class FileStreamSourceSuite extends FileStreamSourceTest { import testImplicits._ @@ -594,7 +594,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } } -class FileStreamSourceStressTestSuite extends FileStreamSourceTest with SharedSQLContext { +class FileStreamSourceStressTestSuite extends FileStreamSourceTest { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala index 4efb7cf52d4a4..1c0fb34dd0191 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala @@ -23,9 +23,7 @@ import java.util.UUID import scala.util.Random import scala.util.control.NonFatal -import org.apache.spark.sql.{ContinuousQuery, ContinuousQueryException, StreamTest} import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils /** @@ -38,7 +36,7 @@ import org.apache.spark.util.Utils * * At the end, the resulting files are loaded and the answer is checked. */ -class FileStressSuite extends StreamTest with SharedSQLContext { +class FileStressSuite extends StreamTest { import testImplicits._ testQuietly("fault tolerance stress test - unpartitioned output") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala index e5bd0b47443eb..df76499fa2801 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils -class MemorySinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { +class MemorySinkSuite extends StreamTest with BeforeAndAfter { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala index 81760d2aa8205..7f2972edea727 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala @@ -17,11 +17,9 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql.StreamTest import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.test.SharedSQLContext -class MemorySourceStressSuite extends StreamTest with SharedSQLContext { +class MemorySourceStressSuite extends StreamTest { import testImplicits._ test("memory stress test") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index c17cb1de6ce9e..9414b1ce4019b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.ManualClock -class StreamSuite extends StreamTest with SharedSQLContext { +class StreamSuite extends StreamTest { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index b033725f18b34..dd8672aa641d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.streaming import java.lang.Thread.UncaughtExceptionHandler @@ -33,10 +33,12 @@ import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.{Dataset, Encoder, QueryTest, Row} import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} /** @@ -63,7 +65,7 @@ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} * avoid hanging forever in the case of failures. However, individual suites can change this * by overriding `streamingTimeout`. */ -trait StreamTest extends QueryTest with Timeouts { +trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { /** How long to wait for an active stream to catch up when checking a result. */ val streamingTimeout = 10.seconds @@ -523,7 +525,7 @@ trait StreamTest extends QueryTest with Timeouts { case class ExpectException[E <: Exception]()(implicit val t: ClassTag[E]) extends ExpectedBehavior - private val DEFAULT_TEST_TIMEOUT = 1 second + private val DEFAULT_TEST_TIMEOUT = 1.second def test( expectedBehavior: ExpectedBehavior, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 322bbb9ea0a57..1f174aee8ce08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -20,19 +20,18 @@ package org.apache.spark.sql.streaming import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, StreamTest} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.InternalOutputModes._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext object FailureSinglton { var firstTime = true } -class StreamingAggregationSuite extends StreamTest with SharedSQLContext with BeforeAndAfterAll { +class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll { override def afterAll(): Unit = { super.afterAll() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index 38a0534ab6be4..a2aac69064f9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.streaming.{ContinuousQuery, OutputMode, ProcessingTime, StreamTest} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils @@ -101,7 +101,7 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { } } -class DataFrameReaderWriterSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { +class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { private def newMetadataDir = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath From 71e8aaeaa9c2983d6f2ab8c512e59f5b13e8844e Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 1 Jun 2016 10:28:48 -0700 Subject: [PATCH 0475/1470] [SPARK-6320][SQL] Move planLater method into GenericStrategy. ## What changes were proposed in this pull request? This PR is the minimal version of #13147 for `branch-2.0`. ## How was this patch tested? Picked `SparkPlannerSuite` from #13147. Author: Takuya UESHIN Closes #13426 from ueshin/issues/SPARK-6320_2.0. --- .../sql/catalyst/planning/QueryPlanner.scala | 15 ++--- .../spark/sql/execution/SparkPlanner.scala | 11 ++++ .../spark/sql/execution/SparkStrategies.scala | 23 +++++++ .../scala/org/apache/spark/sql/package.scala | 4 +- .../sql/execution/SparkPlannerSuite.scala | 63 +++++++++++++++++++ 5 files changed, 107 insertions(+), 9 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 8b1a34f79c42a..327a0481acf7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -27,6 +27,14 @@ import org.apache.spark.sql.catalyst.trees.TreeNode * empty list should be returned. */ abstract class GenericStrategy[PhysicalPlan <: TreeNode[PhysicalPlan]] extends Logging { + + /** + * Returns a placeholder for a physical plan that executes `plan`. This placeholder will be + * filled in automatically by the QueryPlanner using the other execution strategies that are + * available. + */ + protected def planLater(plan: LogicalPlan): PhysicalPlan + def apply(plan: LogicalPlan): Seq[PhysicalPlan] } @@ -47,13 +55,6 @@ abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] { /** A list of execution strategies that can be used by the planner */ def strategies: Seq[GenericStrategy[PhysicalPlan]] - /** - * Returns a placeholder for a physical plan that executes `plan`. This placeholder will be - * filled in automatically by the QueryPlanner using the other execution strategies that are - * available. - */ - protected def planLater(plan: LogicalPlan): PhysicalPlan = this.plan(plan).next() - def plan(plan: LogicalPlan): Iterator[PhysicalPlan] = { // Obviously a lot to do here still... val iter = strategies.view.flatMap(_(plan)).toIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index de832ec70b4df..319fff1f7c122 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileSourceStrategy} import org.apache.spark.sql.internal.SQLConf @@ -42,6 +43,16 @@ class SparkPlanner( InMemoryScans :: BasicOperators :: Nil) + override def plan(plan: LogicalPlan): Iterator[SparkPlan] = { + super.plan(plan).map { + _.transformUp { + case PlanLater(p) => + // TODO: use the first plan for now, but we will implement plan space exploaration later. + this.plan(p).next() + } + } + } + /** * Used to build table scan operators where complex projection and filtering are done using * separate physical operators. This function returns the given scan operator with Project and 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 7e3e45e56e90a..5a069f2214503 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder @@ -35,6 +37,27 @@ import org.apache.spark.sql.execution.streaming.MemoryPlan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.ContinuousQuery +/** + * Converts a logical plan into zero or more SparkPlans. This API is exposed for experimenting + * with the query planner and is not designed to be stable across spark releases. Developers + * writing libraries should instead consider using the stable APIs provided in + * [[org.apache.spark.sql.sources]] + */ +@DeveloperApi +abstract class SparkStrategy extends GenericStrategy[SparkPlan] { + + override protected def planLater(plan: LogicalPlan): SparkPlan = PlanLater(plan) +} + +private[sql] case class PlanLater(plan: LogicalPlan) extends LeafExecNode { + + override def output: Seq[Attribute] = plan.output + + protected override def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException() + } +} + private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SparkPlanner => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 97e35bb10407e..28d8bc3de68b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -18,7 +18,7 @@ package org.apache.spark import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy} /** * Allows the execution of relational queries, including those expressed in SQL using Spark. @@ -40,7 +40,7 @@ package object sql { * [[org.apache.spark.sql.sources]] */ @DeveloperApi - type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] + type Strategy = SparkStrategy type DataFrame = Dataset[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala new file mode 100644 index 0000000000000..aecfd3062147c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.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.sql.execution + +import org.apache.spark.sql.Strategy +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, ReturnAnswer, Union} +import org.apache.spark.sql.test.SharedSQLContext + +class SparkPlannerSuite extends SharedSQLContext { + import testImplicits._ + + test("Ensure to go down only the first branch, not any other possible branches") { + + case object NeverPlanned extends LeafNode { + override def output: Seq[Attribute] = Nil + } + + var planned = 0 + object TestStrategy extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ReturnAnswer(child) => + planned += 1 + planLater(child) :: planLater(NeverPlanned) :: Nil + case Union(children) => + planned += 1 + UnionExec(children.map(planLater)) :: planLater(NeverPlanned) :: Nil + case LocalRelation(output, data) => + planned += 1 + LocalTableScanExec(output, data) :: planLater(NeverPlanned) :: Nil + case NeverPlanned => + fail("QueryPlanner should not go down to this branch.") + case _ => Nil + } + } + + try { + spark.experimental.extraStrategies = TestStrategy :: Nil + + val ds = Seq("a", "b", "c").toDS().union(Seq("d", "e", "f").toDS()) + + assert(ds.collect().toSeq === Seq("a", "b", "c", "d", "e", "f")) + assert(planned === 4) + } finally { + spark.experimental.extraStrategies = Nil + } + } +} From beb4ea0b46998fc3270829a517f12adf4a94bb98 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 1 Jun 2016 10:49:51 -0700 Subject: [PATCH 0476/1470] [SPARK-15587][ML] ML 2.0 QA: Scala APIs audit for ml.feature ## What changes were proposed in this pull request? ML 2.0 QA: Scala APIs audit for ml.feature. Mainly include: * Remove seed for ```QuantileDiscretizer```, since we use ```approxQuantile``` to produce bins and ```seed``` is useless. * Scala API docs update. * Sync Scala and Python API docs for these changes. ## How was this patch tested? Exist tests. Author: Yanbo Liang Closes #13410 from yanboliang/spark-15587. (cherry picked from commit 07a98ca4ce4e715ce32b4be75010e28764da459b) Signed-off-by: Nick Pentreath --- .../apache/spark/ml/feature/Bucketizer.scala | 6 ++-- .../spark/ml/feature/CountVectorizer.scala | 10 ++----- .../ml/feature/QuantileDiscretizer.scala | 7 ++--- .../apache/spark/ml/feature/Word2Vec.scala | 3 +- python/pyspark/ml/feature.py | 29 +++++++++---------- 5 files changed, 23 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index 10e622ace6d5e..ff988cc815160 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -43,7 +43,7 @@ final class Bucketizer(override val uid: String) /** * Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets. * A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which - * also includes y. Splits should be strictly increasing. + * also includes y. Splits should be of length >= 3 and strictly increasing. * Values at -inf, inf must be explicitly provided to cover all Double values; * otherwise, values outside the splits specified will be treated as errors. * @group param @@ -51,8 +51,8 @@ final class Bucketizer(override val uid: String) val splits: DoubleArrayParam = new DoubleArrayParam(this, "splits", "Split points for mapping continuous features into buckets. With n+1 splits, there are n " + "buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last " + - "bucket, which also includes y. The splits should be strictly increasing. " + - "Values at -inf, inf must be explicitly provided to cover all Double values; " + + "bucket, which also includes y. The splits should be of length >= 3 and strictly " + + "increasing. Values at -inf, inf must be explicitly provided to cover all Double values; " + "otherwise, values outside the splits specified will be treated as errors.", Bucketizer.checkSplits) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index fc4885bf4ba84..272567d09cda7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -56,7 +56,7 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit * If this is an integer >= 1, this specifies the number of documents the term must appear in; * if this is a double in [0,1), then this specifies the fraction of documents. * - * Default: 1 + * Default: 1.0 * @group param */ val minDF: DoubleParam = new DoubleParam(this, "minDF", "Specifies the minimum number of" + @@ -86,7 +86,7 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit * Note that the parameter is only used in transform of [[CountVectorizerModel]] and does not * affect fitting. * - * Default: 1 + * Default: 1.0 * @group param */ val minTF: DoubleParam = new DoubleParam(this, "minTF", "Filter to ignore rare words in" + @@ -96,8 +96,6 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit " of the document's token count). Note that the parameter is only used in transform of" + " CountVectorizerModel and does not affect fitting.", ParamValidators.gtEq(0.0)) - setDefault(minTF -> 1) - /** @group getParam */ def getMinTF: Double = $(minTF) @@ -114,7 +112,7 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit /** @group getParam */ def getBinary: Boolean = $(binary) - setDefault(binary -> false) + setDefault(vocabSize -> (1 << 18), minDF -> 1.0, minTF -> 1.0, binary -> false) } /** @@ -145,8 +143,6 @@ class CountVectorizer(override val uid: String) /** @group setParam */ def setBinary(value: Boolean): this.type = set(binary, value) - setDefault(vocabSize -> (1 << 18), minDF -> 1) - @Since("2.0.0") override def fit(dataset: Dataset[_]): CountVectorizerModel = { transformSchema(dataset.schema, logging = true) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 61483590cde38..1fefaa1fdd8df 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -22,7 +22,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol, HasSeed} +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.{DoubleType, StructType} @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.{DoubleType, StructType} * Params for [[QuantileDiscretizer]]. */ private[feature] trait QuantileDiscretizerBase extends Params - with HasInputCol with HasOutputCol with HasSeed { + with HasInputCol with HasOutputCol { /** * Number of buckets (quantiles, or categories) into which data points are grouped. Must @@ -91,9 +91,6 @@ final class QuantileDiscretizer(override val uid: String) /** @group setParam */ def setOutputCol(value: String): this.type = set(outputCol, value) - /** @group setParam */ - def setSeed(value: Long): this.type = set(seed, value) - override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) val inputFields = schema.fields diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 1b929cdfffe36..2d89eb05a5182 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -51,7 +51,8 @@ private[feature] trait Word2VecBase extends Params def getVectorSize: Int = $(vectorSize) /** - * The window size (context words from [-window, window]) default 5. + * The window size (context words from [-window, window]). + * Default: 5 * @group expertParam */ final val windowSize = new IntParam( diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index eb555cb940931..1aff2e550f38c 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -19,8 +19,6 @@ if sys.version > '3': basestring = str -from py4j.java_collections import JavaArray - from pyspark import since, keyword_only from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.linalg import _convert_to_vector @@ -159,9 +157,9 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav "Split points for mapping continuous features into buckets. With n+1 splits, " + "there are n buckets. A bucket defined by splits x,y holds values in the " + "range [x,y) except the last bucket, which also includes y. The splits " + - "should be strictly increasing. Values at -inf, inf must be explicitly " + - "provided to cover all Double values; otherwise, values outside the splits " + - "specified will be treated as errors.", + "should be of length >= 3 and strictly increasing. Values at -inf, inf must be " + + "explicitly provided to cover all Double values; otherwise, values outside the " + + "splits specified will be treated as errors.", typeConverter=TypeConverters.toListFloat) @keyword_only @@ -1171,8 +1169,7 @@ def getDegree(self): @inherit_doc -class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, JavaMLReadable, - JavaMLWritable): +class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ .. note:: Experimental @@ -1186,9 +1183,7 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) >>> qds = QuantileDiscretizer(numBuckets=2, - ... inputCol="values", outputCol="buckets", seed=123, relativeError=0.01) - >>> qds.getSeed() - 123 + ... inputCol="values", outputCol="buckets", relativeError=0.01) >>> qds.getRelativeError() 0.01 >>> bucketizer = qds.fit(df) @@ -1220,9 +1215,9 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasSeed, Jav typeConverter=TypeConverters.toFloat) @keyword_only - def __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, relativeError=0.001): + def __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001): """ - __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, relativeError=0.001) + __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001) """ super(QuantileDiscretizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.QuantileDiscretizer", @@ -1233,11 +1228,9 @@ def __init__(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, relat @keyword_only @since("2.0.0") - def setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, - relativeError=0.001): + def setParams(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001): """ - setParams(self, numBuckets=2, inputCol=None, outputCol=None, seed=None, \ - relativeError=0.001) + setParams(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001) Set the params for the QuantileDiscretizer """ kwargs = self.setParams._input_kwargs @@ -1481,6 +1474,10 @@ class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, J Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the training set. + The "unit std" is computed using the `corrected sample standard deviation \ + `_, + which is computed as the square root of the unbiased sample variance. + >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") From 47902d4bc60561a2b0f4c7aadfdda394d4e78f75 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 1 Jun 2016 12:57:02 -0700 Subject: [PATCH 0477/1470] [SPARK-15702][DOCUMENTATION] Update document programming-guide accumulator section ## What changes were proposed in this pull request? Update document programming-guide accumulator section (scala language) java and python version, because the API haven't done, so I do not modify them. ## How was this patch tested? N/A Author: WeichenXu Closes #13441 from WeichenXu123/update_doc_accumulatorV2_clean. (cherry picked from commit 2402b91461146289a78d6cbc53ed8338543e6de7) Signed-off-by: Reynold Xin --- docs/programming-guide.md | 37 +++++++++++++++++++------------------ 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index d375926a910e6..70fd627c6fe69 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1352,41 +1352,42 @@ The code below shows an accumulator being used to add up the elements of an arra
    {% highlight scala %} -scala> val accum = sc.accumulator(0, "My Accumulator") -accum: org.apache.spark.Accumulator[Int] = 0 +scala> val accum = sc.longAccumulator("My Accumulator") +accum: org.apache.spark.util.LongAccumulator = LongAccumulator(id: 0, name: Some(My Accumulator), value: 0) -scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) +scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum.add(x)) ... 10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s scala> accum.value -res2: Int = 10 +res2: Long = 10 {% endhighlight %} -While this code used the built-in support for accumulators of type Int, programmers can also -create their own types by subclassing [AccumulatorParam](api/scala/index.html#org.apache.spark.AccumulatorParam). -The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data -type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class +While this code used the built-in support for accumulators of type Long, programmers can also +create their own types by subclassing [AccumulatorV2](api/scala/index.html#org.apache.spark.AccumulatorV2). +The AccumulatorV2 abstract class has several methods which need to override: +`reset` for resetting the accumulator to zero, and `add` for add anothor value into the accumulator, `merge` for merging another same-type accumulator into this one. Other methods need to override can refer to scala API document. For example, supposing we had a `MyVector` class representing mathematical vectors, we could write: {% highlight scala %} -object VectorAccumulatorParam extends AccumulatorParam[Vector] { - def zero(initialValue: Vector): Vector = { - Vector.zeros(initialValue.size) +object VectorAccumulatorV2 extends AccumulatorV2[MyVector, MyVector] { + val vec_ : MyVector = MyVector.createZeroVector + def reset(): MyVector = { + vec_.reset() } - def addInPlace(v1: Vector, v2: Vector): Vector = { - v1 += v2 + def add(v1: MyVector, v2: MyVector): MyVector = { + vec_.add(v2) } + ... } // Then, create an Accumulator of this type: -val vecAccum = sc.accumulator(new Vector(...))(VectorAccumulatorParam) +val myVectorAcc = new VectorAccumulatorV2 +// Then, register it into spark context: +sc.register(myVectorAcc, "MyVectorAcc1") {% endhighlight %} -In Scala, Spark also supports the more general [Accumulable](api/scala/index.html#org.apache.spark.Accumulable) -interface to accumulate data where the resulting type is not the same as the elements added (e.g. build -a list by collecting together elements), and the `SparkContext.accumulableCollection` method for accumulating -common Scala collection types. +Note that, when programmers define their own type of AccumulatorV2, the resulting type can be same or not same with the elements added.
    From 46d5f7f3862a04eedc12daa0a7c73af278251c6d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 1 Jun 2016 13:21:40 -0700 Subject: [PATCH 0478/1470] =?UTF-8?q?[SPARK-15671]=20performance=20regress?= =?UTF-8?q?ion=20CoalesceRDD.pickBin=20with=20large=20#=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit I was running a 15TB join job with 202000 partitions. It looks like the changes I made to CoalesceRDD in pickBin() are really slow with that large of partitions. The array filter with that many elements just takes to long. It took about an hour for it to pickBins for all the partitions. original change: https://github.com/apache/spark/commit/83ee92f60345f016a390d61a82f1d924f64ddf90 Just reverting the pickBin code back to get currpreflocs fixes the issue After reverting the pickBin code the coalesce takes about 10 seconds so for now it makes sense to revert those changes and we can look at further optimizations later. Tested this via RDDSuite unit test and manually testing the very large job. Author: Thomas Graves Closes #13443 from tgravescs/SPARK-15671. (cherry picked from commit 5b08ee6396aeb4e0aa6139892a27186813c90931) Signed-off-by: Davies Liu --- .../main/scala/org/apache/spark/rdd/CoalescedRDD.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index c19ed1529bbf6..2ec9846e33f5a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -169,6 +169,11 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) var noLocality = true // if true if no preferredLocations exists for parent RDD + // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) + def currPrefLocs(part: Partition, prev: RDD[_]): Seq[String] = { + prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) + } + class PartitionLocations(prev: RDD[_]) { // contains all the partitions from the previous RDD that don't have preferred locations @@ -184,7 +189,7 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) val tmpPartsWithLocs = mutable.LinkedHashMap[Partition, Seq[String]]() // first get the locations for each partition, only do this once since it can be expensive prev.partitions.foreach(p => { - val locs = prev.context.getPreferredLocs(prev, p.index).map(tl => tl.host) + val locs = currPrefLocs(p, prev) if (locs.size > 0) { tmpPartsWithLocs.put(p, locs) } else { @@ -287,9 +292,8 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) balanceSlack: Double, partitionLocs: PartitionLocations): PartitionGroup = { val slack = (balanceSlack * prev.partitions.length).toInt - val preflocs = partitionLocs.partsWithLocs.filter(_._2 == p).map(_._1).toSeq // least loaded pref locs - val pref = preflocs.map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs + val pref = currPrefLocs(p, prev).map(getLeastGroupHash(_)).sortWith(compare) val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) From 44052a70734fd0f35dc54f5553ae40ac395eef13 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 1 Jun 2016 14:26:24 -0700 Subject: [PATCH 0479/1470] [SPARK-15596][SPARK-15635][SQL] ALTER TABLE RENAME fixes ## What changes were proposed in this pull request? **SPARK-15596**: Even after we renamed a cached table, the plan would remain in the cache with the old table name. If I created a new table using the old name then the old table would return incorrect data. Note that this applies only to Hive tables. **SPARK-15635**: Renaming a datasource table would render the table not query-able. This is because we store the location of the table in a "path" property, which was not updated to reflect Hive's change in table location following a rename. ## How was this patch tested? DDLSuite Author: Andrew Or Closes #13416 from andrewor14/rename-table. (cherry picked from commit 9e2643b21d5749f2f5447b0274a6a35496054342) Signed-off-by: Yin Huai --- .../spark/sql/execution/command/tables.scala | 35 +++++++++++++++++-- .../sql/execution/command/DDLSuite.scala | 18 ++++++++++ 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 2d6a3b48603d7..1b89c6b9ce884 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal +import scala.util.Try import org.apache.hadoop.fs.Path @@ -145,8 +146,38 @@ case class AlterTableRenameCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog DDLUtils.verifyAlterTableType(catalog, oldName, isView) - catalog.invalidateTable(oldName) - catalog.renameTable(oldName, newName) + // If this is a temp view, just rename the view. + // Otherwise, if this is a real table, we also need to uncache and invalidate the table. + val isTemporary = catalog.isTemporaryTable(oldName) + if (isTemporary) { + catalog.renameTable(oldName, newName) + } else { + // If an exception is thrown here we can just assume the table is uncached; + // this can happen with Hive tables when the underlying catalog is in-memory. + val wasCached = Try(sparkSession.catalog.isCached(oldName.unquotedString)).getOrElse(false) + if (wasCached) { + try { + sparkSession.catalog.uncacheTable(oldName.unquotedString) + } catch { + case NonFatal(e) => log.warn(e.toString, e) + } + } + // For datasource tables, we also need to update the "path" serde property + val table = catalog.getTableMetadata(oldName) + if (DDLUtils.isDatasourceTable(table) && table.tableType == CatalogTableType.MANAGED) { + val newPath = catalog.defaultTablePath(newName) + val newTable = table.withNewStorage( + serdeProperties = table.storage.serdeProperties ++ Map("path" -> newPath)) + catalog.alterTable(newTable) + } + // Invalidate the table last, otherwise uncaching the table would load the logical plan + // back into the hive metastore cache + catalog.invalidateTable(oldName) + catalog.renameTable(oldName, newName) + if (wasCached) { + sparkSession.catalog.cacheTable(newName.unquotedString) + } + } Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index dd1f59880701d..741ea673e97e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -446,6 +446,24 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("alter table: rename cached table") { + import testImplicits._ + sql("CREATE TABLE students (age INT, name STRING) USING parquet") + val df = (1 to 2).map { i => (i, i.toString) }.toDF("age", "name") + df.write.insertInto("students") + spark.catalog.cacheTable("students") + assume(spark.table("students").collect().toSeq == df.collect().toSeq, "bad test: wrong data") + assume(spark.catalog.isCached("students"), "bad test: table was not cached in the first place") + sql("ALTER TABLE students RENAME TO teachers") + sql("CREATE TABLE students (age INT, name STRING) USING parquet") + // Now we have both students and teachers. + // The cached data for the old students table should not be read by the new students table. + assert(!spark.catalog.isCached("students")) + assert(spark.catalog.isCached("teachers")) + assert(spark.table("students").collect().isEmpty) + assert(spark.table("teachers").collect().toSeq == df.collect().toSeq) + } + test("rename temporary table - destination table with database name") { withTempTable("tab1") { sql( From e033fd50f0fcefb2a6cffb763ff7e026b0066c07 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 1 Jun 2016 16:02:27 -0700 Subject: [PATCH 0480/1470] [SPARK-15269][SQL] Removes unexpected empty table directories created while creating external Spark SQL data sourcet tables. This PR is an alternative to #13120 authored by xwu0226. ## What changes were proposed in this pull request? When creating an external Spark SQL data source table and persisting its metadata to Hive metastore, we don't use the standard Hive `Table.dataLocation` field because Hive only allows directory paths as data locations while Spark SQL also allows file paths. However, if we don't set `Table.dataLocation`, Hive always creates an unexpected empty table directory under database location, but doesn't remove it while dropping the table (because the table is external). This PR works around this issue by explicitly setting `Table.dataLocation` and then manullay removing the created directory after creating the external table. Please refer to [this JIRA comment][1] for more details about why we chose this approach as a workaround. [1]: https://issues.apache.org/jira/browse/SPARK-15269?focusedCommentId=15297408&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15297408 ## How was this patch tested? 1. A new test case is added in `HiveQuerySuite` for this case 2. Updated `ShowCreateTableSuite` to use the same table name in all test cases. (This is how I hit this issue at the first place.) Author: Cheng Lian Closes #13270 from liancheng/spark-15269-unpleasant-fix. (cherry picked from commit 7bb64aae27f670531699f59d3f410e38866609b7) Signed-off-by: Cheng Lian --- .../apache/spark/sql/AnalysisException.scala | 5 ++- .../sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../org/apache/spark/sql/SparkSession.scala | 6 +-- .../command/createDataSourceTables.scala | 4 +- .../apache/spark/sql/internal/HiveSerDe.scala | 1 - .../spark/sql/hive/HiveExternalCatalog.scala | 45 +++++++++++++++++-- .../spark/sql/hive/HiveSharedState.scala | 4 +- .../sql/hive/client/HiveClientImpl.scala | 22 +++++++-- .../sql/hive/HiveExternalCatalogSuite.scala | 3 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 13 ++++++ .../spark/sql/hive/ShowCreateTableSuite.scala | 36 +++++++-------- 11 files changed, 105 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index d2003fd6892e1..6911843999392 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -32,8 +32,9 @@ class AnalysisException protected[sql] ( val message: String, val line: Option[Int] = None, val startPosition: Option[Int] = None, - val plan: Option[LogicalPlan] = None) - extends Exception with Serializable { + val plan: Option[LogicalPlan] = None, + val cause: Option[Throwable] = None) + extends Exception(message, cause.orNull) with Serializable { def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = { val newException = new AnalysisException(message, line, startPosition) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index cf9286e6b97a6..371c198aa3493 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -22,7 +22,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 52bedf9dbddae..7d7fd0399deda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -557,9 +557,9 @@ class SparkSession private( } - /* ------------------------ * - | Catalog-related methods | - * ----------------- ------ */ + /* ------------------------- * + | Catalog-related methods | + * ------------------------- */ /** * Interface through which the user may create, drop, alter or query underlying diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 4b9aab612e7c3..9956c5b09236d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -118,8 +118,8 @@ case class CreateDataSourceTableCommand( /** * A command used to create a data source table using the result of a query. * - * Note: This is different from [[CreateTableAsSelect]]. Please check the syntax for difference. - * This is not intended for temporary tables. + * Note: This is different from [[CreateTableAsSelectLogicalPlan]]. Please check the syntax for + * difference. This is not intended for temporary tables. * * The syntax of using this command in SQL is: * {{{ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala index 38317d46dd82d..d554937d8b400 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -77,4 +77,3 @@ object HiveSerDe { serdeMap.get(key) } } - diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 5ffd8ef149a1e..b8bc9ab900ad1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -21,6 +21,8 @@ import java.util import scala.util.control.NonFatal +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.thrift.TException @@ -35,7 +37,9 @@ import org.apache.spark.sql.hive.client.HiveClient * A persistent implementation of the system catalog using Hive. * All public methods must be synchronized for thread-safety. */ -private[spark] class HiveExternalCatalog(client: HiveClient) extends ExternalCatalog with Logging { +private[spark] class HiveExternalCatalog(client: HiveClient, hadoopConf: Configuration) + extends ExternalCatalog with Logging { + import CatalogTypes.TablePartitionSpec // Exceptions thrown by the hive client that we would like to wrap @@ -68,7 +72,8 @@ private[spark] class HiveExternalCatalog(client: HiveClient) extends ExternalCat body } catch { case NonFatal(e) if isClientException(e) => - throw new AnalysisException(e.getClass.getCanonicalName + ": " + e.getMessage) + throw new AnalysisException( + e.getClass.getCanonicalName + ": " + e.getMessage, cause = Some(e)) } } @@ -147,7 +152,41 @@ private[spark] class HiveExternalCatalog(client: HiveClient) extends ExternalCat ignoreIfExists: Boolean): Unit = withClient { requireDbExists(db) requireDbMatches(db, tableDefinition) - client.createTable(tableDefinition, ignoreIfExists) + + if ( + // If this is an external data source table... + tableDefinition.properties.contains("spark.sql.sources.provider") && + tableDefinition.tableType == CatalogTableType.EXTERNAL && + // ... that is not persisted as Hive compatible format (external tables in Hive compatible + // format always set `locationUri` to the actual data location and should NOT be hacked as + // following.) + tableDefinition.storage.locationUri.isEmpty + ) { + // !! HACK ALERT !! + // + // Due to a restriction of Hive metastore, here we have to set `locationUri` to a temporary + // directory that doesn't exist yet but can definitely be successfully created, and then + // delete it right after creating the external data source table. This location will be + // persisted to Hive metastore as standard Hive table location URI, but Spark SQL doesn't + // really use it. Also, since we only do this workaround for external tables, deleting the + // directory after the fact doesn't do any harm. + // + // Please refer to https://issues.apache.org/jira/browse/SPARK-15269 for more details. + val tempPath = { + val dbLocation = getDatabase(tableDefinition.database).locationUri + new Path(dbLocation, tableDefinition.identifier.table + "-__PLACEHOLDER__") + } + + try { + client.createTable( + tableDefinition.withNewStorage(locationUri = Some(tempPath.toString)), + ignoreIfExists) + } finally { + FileSystem.get(tempPath.toUri, hadoopConf).delete(tempPath, true) + } + } else { + client.createTable(tableDefinition, ignoreIfExists) + } } override def dropTable( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala index f0d96403e8551..a0106ee882e76 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala @@ -51,6 +51,6 @@ private[hive] class HiveSharedState(override val sparkContext: SparkContext) /** * A catalog that interacts with the Hive metastore. */ - override lazy val externalCatalog = new HiveExternalCatalog(metadataHive) - + override lazy val externalCatalog = + new HiveExternalCatalog(metadataHive, sparkContext.hadoopConfiguration) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 71d5c9960a70c..47fa41823cd09 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.QueryExecutionException +import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.util.{CircularBuffer, Utils} /** @@ -323,7 +324,7 @@ private[hive] class HiveClientImpl( } override def listDatabases(pattern: String): Seq[String] = withHiveState { - client.getDatabasesByPattern(pattern).asScala.toSeq + client.getDatabasesByPattern(pattern).asScala } override def getTableOption( @@ -351,6 +352,8 @@ private[hive] class HiveClientImpl( unsupportedFeatures += "bucketing" } + val properties = h.getParameters.asScala.toMap + CatalogTable( identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), tableType = h.getTableType match { @@ -368,14 +371,27 @@ private[hive] class HiveClientImpl( createTime = h.getTTable.getCreateTime.toLong * 1000, lastAccessTime = h.getLastAccessTime.toLong * 1000, storage = CatalogStorageFormat( - locationUri = shim.getDataLocation(h), + locationUri = shim.getDataLocation(h).filterNot { _ => + // SPARK-15269: Persisted data source tables always store the location URI as a SerDe + // property named "path" instead of standard Hive `dataLocation`, because Hive only + // allows directory paths as location URIs while Spark SQL data source tables also + // allows file paths. So the standard Hive `dataLocation` is meaningless for Spark SQL + // data source tables. + DDLUtils.isDatasourceTable(properties) && + h.getTableType == HiveTableType.EXTERNAL_TABLE && + // Spark SQL may also save external data source in Hive compatible format when + // possible, so that these tables can be directly accessed by Hive. For these tables, + // `dataLocation` is still necessary. Here we also check for input format class + // because only these Hive compatible tables set this field. + h.getInputFormatClass == null + }, inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), serde = Option(h.getSerializationLib), compressed = h.getTTable.getSd.isCompressed, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap ), - properties = h.getParameters.asScala.toMap, + properties = properties, viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), unsupportedFeatures = unsupportedFeatures) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index bf9935ae41b30..175889b08b49f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -37,7 +37,8 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { protected override val utils: CatalogTestUtils = new CatalogTestUtils { override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" override val tableOutputFormat: String = "org.apache.hadoop.mapred.SequenceFileOutputFormat" - override def newEmptyCatalog(): ExternalCatalog = new HiveExternalCatalog(client) + override def newEmptyCatalog(): ExternalCatalog = + new HiveExternalCatalog(client, new Configuration()) } protected override def resetState(): Unit = client.reset() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 2c50cc88cc4cc..3d8123d3c06d8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -1104,4 +1104,17 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } } } + + test("SPARK-15269 external data source table creation") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(1).write.json(path) + + withTable("t") { + sql(s"CREATE TABLE t USING json OPTIONS (PATH '$path')") + sql("DROP TABLE t") + sql(s"CREATE TABLE t USING json AS SELECT 1 AS c") + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala index f789d88d5dd4a..3f3dc122093b5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -28,11 +28,11 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing import testImplicits._ test("data source table with user specified schema") { - withTable("ddl_test1") { + withTable("ddl_test") { val jsonFilePath = Utils.getSparkClassLoader.getResource("sample.json").getFile sql( - s"""CREATE TABLE ddl_test1 ( + s"""CREATE TABLE ddl_test ( | a STRING, | b STRING, | `extra col` ARRAY, @@ -45,55 +45,55 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing """.stripMargin ) - checkCreateTable("ddl_test1") + checkCreateTable("ddl_test") } } test("data source table CTAS") { - withTable("ddl_test2") { + withTable("ddl_test") { sql( - s"""CREATE TABLE ddl_test2 + s"""CREATE TABLE ddl_test |USING json |AS SELECT 1 AS a, "foo" AS b """.stripMargin ) - checkCreateTable("ddl_test2") + checkCreateTable("ddl_test") } } test("partitioned data source table") { - withTable("ddl_test3") { + withTable("ddl_test") { sql( - s"""CREATE TABLE ddl_test3 + s"""CREATE TABLE ddl_test |USING json |PARTITIONED BY (b) |AS SELECT 1 AS a, "foo" AS b """.stripMargin ) - checkCreateTable("ddl_test3") + checkCreateTable("ddl_test") } } test("bucketed data source table") { - withTable("ddl_test3") { + withTable("ddl_test") { sql( - s"""CREATE TABLE ddl_test3 + s"""CREATE TABLE ddl_test |USING json |CLUSTERED BY (a) SORTED BY (b) INTO 2 BUCKETS |AS SELECT 1 AS a, "foo" AS b """.stripMargin ) - checkCreateTable("ddl_test3") + checkCreateTable("ddl_test") } } test("partitioned bucketed data source table") { - withTable("ddl_test4") { + withTable("ddl_test") { sql( - s"""CREATE TABLE ddl_test4 + s"""CREATE TABLE ddl_test |USING json |PARTITIONED BY (c) |CLUSTERED BY (a) SORTED BY (b) INTO 2 BUCKETS @@ -101,12 +101,12 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing """.stripMargin ) - checkCreateTable("ddl_test4") + checkCreateTable("ddl_test") } } test("data source table using Dataset API") { - withTable("ddl_test5") { + withTable("ddl_test") { spark .range(3) .select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd, 'id as 'e) @@ -114,9 +114,9 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing .mode("overwrite") .partitionBy("a", "b") .bucketBy(2, "c", "d") - .saveAsTable("ddl_test5") + .saveAsTable("ddl_test") - checkCreateTable("ddl_test5") + checkCreateTable("ddl_test") } } From 8cdc0d4da6324f48e1a25496dc4bc7635d9472ef Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 1 Jun 2016 16:05:13 -0700 Subject: [PATCH 0481/1470] [SPARK-9876] [BRANCH-2.0] Revert "[SPARK-9876][SQL] Update Parquet to 1.8.1." ## What changes were proposed in this pull request? Since we are pretty late in the 2.0 release cycle, it is not clear if this upgrade can be tested thoroughly and if we can resolve the regression issue that we observed before. This PR reverts #13280 from branch 2.0. ## How was this patch tested? Existing tests This reverts commit 776d183c82b424ef7c3cae30537d8afe9b9eee83. Author: Yin Huai Closes #13450 from yhuai/revertParquet1.8.1-branch-2.0. --- dev/deps/spark-deps-hadoop-2.2 | 11 +-- dev/deps/spark-deps-hadoop-2.3 | 11 +-- dev/deps/spark-deps-hadoop-2.4 | 11 +-- dev/deps/spark-deps-hadoop-2.6 | 11 +-- dev/deps/spark-deps-hadoop-2.7 | 11 +-- pom.xml | 2 +- .../SpecificParquetRecordReaderBase.java | 20 ++--- .../parquet/CatalystReadSupport.scala | 12 +-- .../parquet/CatalystSchemaConverter.scala | 16 ---- .../datasources/parquet/ParquetFilters.scala | 83 +++++++++++++++---- .../parquet/ParquetSchemaSuite.scala | 20 ++--- 11 files changed, 117 insertions(+), 91 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index b5c38a6c056ec..96001eade028b 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -129,13 +129,14 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.3.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar +parquet-column-1.7.0.jar +parquet-common-1.7.0.jar +parquet-encoding-1.7.0.jar parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-generator-1.7.0.jar +parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 969df0495d4c9..9f3d9ad97a9f7 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -136,13 +136,14 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.3.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar +parquet-column-1.7.0.jar +parquet-common-1.7.0.jar +parquet-encoding-1.7.0.jar parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-generator-1.7.0.jar +parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index f0491ece7c2b4..77d52666ba69e 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -136,13 +136,14 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.3.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar +parquet-column-1.7.0.jar +parquet-common-1.7.0.jar +parquet-encoding-1.7.0.jar parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-generator-1.7.0.jar +parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index b3dced63b9e7f..9afe50f765d3c 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -144,13 +144,14 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.3.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar +parquet-column-1.7.0.jar +parquet-common-1.7.0.jar +parquet-encoding-1.7.0.jar parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-generator-1.7.0.jar +parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 16f60f29ffbb2..879157a6dce7e 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -145,13 +145,14 @@ opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.3.jar -parquet-column-1.8.1.jar -parquet-common-1.8.1.jar -parquet-encoding-1.8.1.jar +parquet-column-1.7.0.jar +parquet-common-1.7.0.jar +parquet-encoding-1.7.0.jar parquet-format-2.3.0-incubating.jar -parquet-hadoop-1.8.1.jar +parquet-generator-1.7.0.jar +parquet-hadoop-1.7.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.8.1.jar +parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar diff --git a/pom.xml b/pom.xml index 60c8c8dc7a727..79ee7876f466b 100644 --- a/pom.xml +++ b/pom.xml @@ -133,7 +133,7 @@ 1.2.1 10.11.1.1 - 1.8.1 + 1.7.0 1.6.0 9.2.16.v20160414 3.1.0 diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index 3f7a872ff635f..cbe8f78164ae7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -58,8 +58,6 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.hadoop.util.ConfigurationUtil; import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; import org.apache.spark.sql.types.StructType; @@ -188,19 +186,15 @@ protected void initialize(String path, List columns) throws IOException if (columns == null) { this.requestedSchema = fileSchema; } else { - if (columns.size() > 0) { - Types.MessageTypeBuilder builder = Types.buildMessage(); - for (String s: columns) { - if (!fileSchema.containsField(s)) { - throw new IOException("Can only project existing columns. Unknown field: " + s + - " File schema:\n" + fileSchema); - } - builder.addFields(fileSchema.getType(s)); + Types.MessageTypeBuilder builder = Types.buildMessage(); + for (String s: columns) { + if (!fileSchema.containsField(s)) { + throw new IOException("Can only project existing columns. Unknown field: " + s + + " File schema:\n" + fileSchema); } - this.requestedSchema = builder.named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME()); - } else { - this.requestedSchema = CatalystSchemaConverter.EMPTY_MESSAGE(); + builder.addFields(fileSchema.getType(s)); } + this.requestedSchema = builder.named("spark_schema"); } this.sparkSchema = new CatalystSchemaConverter(config).convert(requestedSchema); this.reader = new ParquetFileReader(config, file, blocks, requestedSchema.getColumns()); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index 9c885b252f01b..850e807b8677e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -109,14 +109,10 @@ private[parquet] object CatalystReadSupport { */ def clipParquetSchema(parquetSchema: MessageType, catalystSchema: StructType): MessageType = { val clippedParquetFields = clipParquetGroupFields(parquetSchema.asGroupType(), catalystSchema) - if (clippedParquetFields.isEmpty) { - CatalystSchemaConverter.EMPTY_MESSAGE - } else { - Types - .buildMessage() - .addFields(clippedParquetFields: _*) - .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) - } + Types + .buildMessage() + .addFields(clippedParquetFields: _*) + .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) } private def clipParquetType(parquetType: Type, catalystType: DataType): Type = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala index 3688c3e2b57e2..6f6340f541ada 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala @@ -538,22 +538,6 @@ private[parquet] class CatalystSchemaConverter( private[parquet] object CatalystSchemaConverter { val SPARK_PARQUET_SCHEMA_NAME = "spark_schema" - // !! HACK ALERT !! - // - // PARQUET-363 & PARQUET-278: parquet-mr 1.8.1 doesn't allow constructing empty GroupType, - // which prevents us to avoid selecting any columns for queries like `SELECT COUNT(*) FROM t`. - // This issue has been fixed in parquet-mr 1.8.2-SNAPSHOT. - // - // To workaround this problem, here we first construct a `MessageType` with a single dummy - // field, and then remove the field to obtain an empty `MessageType`. - // - // TODO Reverts this change after upgrading parquet-mr to 1.8.2+ - val EMPTY_MESSAGE = Types - .buildMessage() - .required(PrimitiveType.PrimitiveTypeName.INT32).named("dummy") - .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) - EMPTY_MESSAGE.getFields.clear() - def checkFieldName(name: String): Unit = { // ,;{}()\n\t= and space are special characters in Parquet schema checkConversionRequirement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 624081250113a..95afdc789f322 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -22,6 +22,8 @@ import java.io.Serializable import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.io.api.Binary +import org.apache.parquet.schema.OriginalType +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.spark.sql.sources import org.apache.spark.sql.types._ @@ -51,15 +53,18 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) + // See https://issues.apache.org/jira/browse/SPARK-11153 + /* // Binary.fromString and Binary.fromByteArray don't accept null values 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[String].getBytes("utf-8"))).orNull) case BinaryType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + Option(v).map(b => Binary.fromByteArray(v.asInstanceOf[Array[Byte]])).orNull) + */ } private val makeNotEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -74,14 +79,17 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.notEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) + // See https://issues.apache.org/jira/browse/SPARK-11153 + /* 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[String].getBytes("utf-8"))).orNull) case BinaryType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + Option(v).map(b => Binary.fromByteArray(v.asInstanceOf[Array[Byte]])).orNull) + */ } private val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -94,13 +102,16 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) + // See https://issues.apache.org/jira/browse/SPARK-11153 + /* case StringType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), - Binary.fromString(v.asInstanceOf[String])) + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) + */ } private val makeLtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -113,13 +124,16 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) + // See https://issues.apache.org/jira/browse/SPARK-11153 + /* case StringType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), - Binary.fromString(v.asInstanceOf[String])) + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) + */ } private val makeGt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -133,13 +147,15 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) // See https://issues.apache.org/jira/browse/SPARK-11153 + /* case StringType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), - Binary.fromString(v.asInstanceOf[String])) + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) + */ } private val makeGtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -152,13 +168,16 @@ private[sql] object ParquetFilters { case DoubleType => (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) + // See https://issues.apache.org/jira/browse/SPARK-11153 + /* case StringType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), - Binary.fromString(v.asInstanceOf[String])) + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) + */ } private val makeInSet: PartialFunction[DataType, (String, Set[Any]) => FilterPredicate] = { @@ -175,14 +194,17 @@ private[sql] object ParquetFilters { (n: String, v: Set[Any]) => FilterApi.userDefined(doubleColumn(n), SetInFilter(v.asInstanceOf[Set[java.lang.Double]])) + // See https://issues.apache.org/jira/browse/SPARK-11153 + /* case StringType => (n: String, v: Set[Any]) => FilterApi.userDefined(binaryColumn(n), - SetInFilter(v.map(s => Binary.fromString(s.asInstanceOf[String])))) + SetInFilter(v.map(s => Binary.fromByteArray(s.asInstanceOf[String].getBytes("utf-8"))))) case BinaryType => (n: String, v: Set[Any]) => FilterApi.userDefined(binaryColumn(n), - SetInFilter(v.map(e => Binary.fromReusedByteArray(e.asInstanceOf[Array[Byte]])))) + SetInFilter(v.map(e => Binary.fromByteArray(e.asInstanceOf[Array[Byte]])))) + */ } /** @@ -206,6 +228,8 @@ private[sql] object ParquetFilters { def createFilter(schema: StructType, predicate: sources.Filter): Option[FilterPredicate] = { val dataTypeOf = getFieldMap(schema).toMap + relaxParquetValidTypeMap + // NOTE: // // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, @@ -275,4 +299,35 @@ private[sql] object ParquetFilters { case _ => None } } + + // !! HACK ALERT !! + // + // This lazy val is a workaround for PARQUET-201, and should be removed once we upgrade to + // parquet-mr 1.8.1 or higher versions. + // + // In Parquet, not all types of columns can be used for filter push-down optimization. The set + // of valid column types is controlled by `ValidTypeMap`. Unfortunately, in parquet-mr 1.7.0 and + // prior versions, the limitation is too strict, and doesn't allow `BINARY (ENUM)` columns to be + // pushed down. + // + // This restriction is problematic for Spark SQL, because Spark SQL doesn't have a type that maps + // to Parquet original type `ENUM` directly, and always converts `ENUM` to `StringType`. Thus, + // a predicate involving a `ENUM` field can be pushed-down as a string column, which is perfectly + // legal except that it fails the `ValidTypeMap` check. + // + // Here we add `BINARY (ENUM)` into `ValidTypeMap` lazily via reflection to workaround this issue. + private lazy val relaxParquetValidTypeMap: Unit = { + val constructor = Class + .forName(classOf[ValidTypeMap].getCanonicalName + "$FullTypeDescriptor") + .getDeclaredConstructor(classOf[PrimitiveTypeName], classOf[OriginalType]) + + constructor.setAccessible(true) + val enumTypeDescriptor = constructor + .newInstance(PrimitiveTypeName.BINARY, OriginalType.ENUM) + .asInstanceOf[AnyRef] + + val addMethod = classOf[ValidTypeMap].getDeclaredMethods.find(_.getName == "add").get + addMethod.setAccessible(true) + addMethod.invoke(null, classOf[Binary], enumTypeDescriptor) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 0b5038cb82803..6db649228210d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import org.apache.parquet.schema.{MessageType, MessageTypeParser} +import org.apache.parquet.schema.MessageTypeParser import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.ScalaReflection @@ -1065,26 +1065,18 @@ class ParquetSchemaSuite extends ParquetSchemaTest { parquetSchema: String, catalystSchema: StructType, expectedSchema: String): Unit = { - testSchemaClipping(testName, parquetSchema, catalystSchema, - MessageTypeParser.parseMessageType(expectedSchema)) - } - - private def testSchemaClipping( - testName: String, - parquetSchema: String, - catalystSchema: StructType, - expectedSchema: MessageType): Unit = { test(s"Clipping - $testName") { + val expected = MessageTypeParser.parseMessageType(expectedSchema) val actual = CatalystReadSupport.clipParquetSchema( MessageTypeParser.parseMessageType(parquetSchema), catalystSchema) try { - expectedSchema.checkContains(actual) - actual.checkContains(expectedSchema) + expected.checkContains(actual) + actual.checkContains(expected) } catch { case cause: Throwable => fail( s"""Expected clipped schema: - |$expectedSchema + |$expected |Actual clipped schema: |$actual """.stripMargin, @@ -1437,7 +1429,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { catalystSchema = new StructType(), - expectedSchema = CatalystSchemaConverter.EMPTY_MESSAGE) + expectedSchema = "message root {}") testSchemaClipping( "disjoint field sets", From 91812226fc18d476be3dcef1351ff5f5ef6ed86f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 1 Jun 2016 16:16:54 -0700 Subject: [PATCH 0482/1470] [SPARK-15441][SQL] support null object in Dataset outer-join ## What changes were proposed in this pull request? Currently we can't encode top level null object into internal row, as Spark SQL doesn't allow row to be null, only its columns can be null. This is not a problem before, as we assume the input object is never null. However, for outer join, we do need the semantics of null object. This PR fixes this problem by making both join sides produce a single column, i.e. nest the logical plan output(by `CreateStruct`), so that we have an extra level to represent top level null obejct. ## How was this patch tested? new test in `DatasetSuite` Author: Wenchen Fan Closes #13425 from cloud-fan/outer-join2. (cherry picked from commit 8640cdb836b4964e4af891d9959af64a2e1f304e) Signed-off-by: Cheng Lian --- .../catalyst/encoders/ExpressionEncoder.scala | 3 +- .../expressions/objects/objects.scala | 1 - .../scala/org/apache/spark/sql/Dataset.scala | 67 ++++++++++++++----- .../org/apache/spark/sql/DatasetSuite.scala | 23 +++---- 4 files changed, 59 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index f21a39a2d4730..2296946cd7c5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -125,12 +125,13 @@ object ExpressionEncoder { } } else { val input = BoundReference(index, enc.schema, nullable = true) - enc.deserializer.transformUp { + val deserialized = enc.deserializer.transformUp { case UnresolvedAttribute(nameParts) => assert(nameParts.length == 1) UnresolvedExtractValue(input, Literal(nameParts.head)) case BoundReference(ordinal, dt, _) => GetStructField(input, ordinal) } + If(IsNull(input), Literal.create(null, deserialized.dataType), deserialized) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 2f2323fa3a25f..c2e3ab82ff16c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions.objects import java.lang.reflect.Modifier -import scala.annotation.tailrec import scala.language.existentials import scala.reflect.ClassTag diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 3a6ec4595e78e..369b772d322c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -747,31 +747,62 @@ class Dataset[T] private[sql]( */ @Experimental def joinWith[U](other: Dataset[U], condition: Column, joinType: String): Dataset[(T, U)] = { - val left = this.logicalPlan - val right = other.logicalPlan - - val joined = sparkSession.sessionState.executePlan(Join(left, right, joinType = - JoinType(joinType), Some(condition.expr))) - val leftOutput = joined.analyzed.output.take(left.output.length) - val rightOutput = joined.analyzed.output.takeRight(right.output.length) + // Creates a Join node and resolve it first, to get join condition resolved, self-join resolved, + // etc. + val joined = sparkSession.sessionState.executePlan( + Join( + this.logicalPlan, + other.logicalPlan, + JoinType(joinType), + Some(condition.expr))).analyzed.asInstanceOf[Join] + + // For both join side, combine all outputs into a single column and alias it with "_1" or "_2", + // to match the schema for the encoder of the join result. + // Note that we do this before joining them, to enable the join operator to return null for one + // side, in cases like outer-join. + val left = { + val combined = if (this.unresolvedTEncoder.flat) { + assert(joined.left.output.length == 1) + Alias(joined.left.output.head, "_1")() + } else { + Alias(CreateStruct(joined.left.output), "_1")() + } + Project(combined :: Nil, joined.left) + } - val leftData = this.unresolvedTEncoder match { - case e if e.flat => Alias(leftOutput.head, "_1")() - case _ => Alias(CreateStruct(leftOutput), "_1")() + val right = { + val combined = if (other.unresolvedTEncoder.flat) { + assert(joined.right.output.length == 1) + Alias(joined.right.output.head, "_2")() + } else { + Alias(CreateStruct(joined.right.output), "_2")() + } + Project(combined :: Nil, joined.right) } - val rightData = other.unresolvedTEncoder match { - case e if e.flat => Alias(rightOutput.head, "_2")() - case _ => Alias(CreateStruct(rightOutput), "_2")() + + // Rewrites the join condition to make the attribute point to correct column/field, after we + // combine the outputs of each join side. + val conditionExpr = joined.condition.get transformUp { + case a: Attribute if joined.left.outputSet.contains(a) => + if (this.unresolvedTEncoder.flat) { + left.output.head + } else { + val index = joined.left.output.indexWhere(_.exprId == a.exprId) + GetStructField(left.output.head, index) + } + case a: Attribute if joined.right.outputSet.contains(a) => + if (other.unresolvedTEncoder.flat) { + right.output.head + } else { + val index = joined.right.output.indexWhere(_.exprId == a.exprId) + GetStructField(right.output.head, index) + } } implicit val tuple2Encoder: Encoder[(T, U)] = ExpressionEncoder.tuple(this.unresolvedTEncoder, other.unresolvedTEncoder) - withTypedPlan { - Project( - leftData :: rightData :: Nil, - joined.analyzed) - } + withTypedPlan(Join(left, right, joined.joinType, Some(conditionExpr))) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 8fc4dc9f17466..0b6874e3b8ad3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -253,21 +253,6 @@ class DatasetSuite extends QueryTest with SharedSQLContext { (1, 1), (2, 2)) } - test("joinWith, expression condition, outer join") { - val nullInteger = null.asInstanceOf[Integer] - val nullString = null.asInstanceOf[String] - val ds1 = Seq(ClassNullableData("a", 1), - ClassNullableData("c", 3)).toDS() - val ds2 = Seq(("a", new Integer(1)), - ("b", new Integer(2))).toDS() - - checkDataset( - ds1.joinWith(ds2, $"_1" === $"a", "outer"), - (ClassNullableData("a", 1), ("a", new Integer(1))), - (ClassNullableData("c", 3), (nullString, nullInteger)), - (ClassNullableData(nullString, nullInteger), ("b", new Integer(2)))) - } - test("joinWith tuple with primitive, expression") { val ds1 = Seq(1, 1, 2).toDS() val ds2 = Seq(("a", 1), ("b", 2)).toDS() @@ -783,6 +768,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ds.filter(_.b > 1).collect().toSeq } } + + test("SPARK-15441: Dataset outer join") { + val left = Seq(ClassData("a", 1), ClassData("b", 2)).toDS().as("left") + val right = Seq(ClassData("x", 2), ClassData("y", 3)).toDS().as("right") + val joined = left.joinWith(right, $"left.b" === $"right.b", "left") + val result = joined.collect().toSet + assert(result == Set(ClassData("a", 1) -> null, ClassData("b", 2) -> ClassData("x", 2))) + } } case class Generic[T](id: T, value: Double) From 35195f6ce2735e71ebb0f50d7ddc2e5e09f2a52f Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 1 Jun 2016 17:03:39 -0700 Subject: [PATCH 0483/1470] [SPARK-15692][SQL] Improves the explain output of several physical plans by displaying embedded logical plan in tree style ## What changes were proposed in this pull request? Improves the explain output of several physical plans by displaying embedded logical plan in tree style Some physical plan contains a embedded logical plan, for example, `cache tableName query` maps to: ``` case class CacheTableCommand( tableName: String, plan: Option[LogicalPlan], isLazy: Boolean) extends RunnableCommand ``` It is easier to read the explain output if we can display the `plan` in tree style. **Before change:** Everything is messed in one line. ``` scala> Seq((1,2)).toDF().createOrReplaceTempView("testView") scala> spark.sql("cache table testView2 select * from testView").explain() == Physical Plan == ExecutedCommand CacheTableCommand testView2, Some('Project [*] +- 'UnresolvedRelation `testView`, None ), false ``` **After change:** ``` scala> spark.sql("cache table testView2 select * from testView").explain() == Physical Plan == ExecutedCommand : +- CacheTableCommand testView2, false : : +- 'Project [*] : : +- 'UnresolvedRelation `testView`, None ``` ## How was this patch tested? Manual test. Author: Sean Zhong Closes #13433 from clockfly/verbose_breakdown_3_2. (cherry picked from commit c8fb776d4a0134c47f90272c4bd5e4bba902aae5) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 10 +++++++--- .../sql/execution/columnar/InMemoryTableScanExec.scala | 5 +++++ .../org/apache/spark/sql/execution/command/cache.scala | 6 +++++- .../apache/spark/sql/execution/command/commands.scala | 6 +++--- .../sql/execution/command/createDataSourceTables.scala | 3 +++ .../org/apache/spark/sql/execution/command/views.scala | 3 +++ .../datasources/InsertIntoDataSourceCommand.scala | 3 +++ 8 files changed, 30 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index d4447ca32d5a0..6784c3ae1d7e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -264,7 +264,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT expressions.flatMap(_.collect {case e: SubqueryExpression => e.plan.asInstanceOf[PlanType]}) } - override def innerChildren: Seq[PlanType] = subqueries + override protected def innerChildren: Seq[QueryPlan[_]] = subqueries /** * Canonicalized copy of this query plan. 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 d87e6c76ed734..3ebd815dce32c 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 @@ -424,9 +424,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { */ protected def stringArgs: Iterator[Any] = productIterator + private lazy val allChildren: Set[TreeNode[_]] = (children ++ innerChildren).toSet[TreeNode[_]] + /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { - case tn: TreeNode[_] if containsChild(tn) => Nil + case tn: TreeNode[_] if allChildren.contains(tn) => Nil + case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil case tn: TreeNode[_] => s"${tn.simpleString}" :: Nil case seq: Seq[BaseType] if seq.toSet.subsetOf(children.toSet) => Nil case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil @@ -467,9 +470,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } /** - * All the nodes that are parts of this node, this is used by subquries. + * All the nodes that should be shown as a inner nested tree of this node. + * For example, this can be used to show sub-queries. */ - protected def innerChildren: Seq[BaseType] = Nil + protected def innerChildren: Seq[TreeNode[_]] = Seq.empty /** * Appends the string represent of this node and its children to the given StringBuilder. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index ba61940b3d5a4..7ccc9de9db233 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning @@ -70,6 +71,8 @@ private[sql] case class InMemoryRelation( private[sql] var _batchStats: ListAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) + override def producedAttributes: AttributeSet = outputSet private[sql] val batchStats: ListAccumulator[InternalRow] = @@ -222,6 +225,8 @@ private[sql] case class InMemoryTableScanExec( @transient relation: InMemoryRelation) extends LeafExecNode { + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren + private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index b1290a4759a25..3e5eed2efa76b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -19,15 +19,19 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - case class CacheTableCommand( tableName: String, plan: Option[LogicalPlan], isLazy: Boolean) extends RunnableCommand { + override protected def innerChildren: Seq[QueryPlan[_]] = { + plan.toSeq + } + override def run(sparkSession: SparkSession): Seq[Row] = { plan.foreach { logicalPlan => Dataset.ofRows(sparkSession, logicalPlan).createOrReplaceTempView(tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 642a95a99262d..38bb6e412f753 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan @@ -57,6 +58,8 @@ private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkP cmd.run(sqlContext.sparkSession).map(converter(_).asInstanceOf[InternalRow]) } + override protected def innerChildren: Seq[QueryPlan[_]] = cmd :: Nil + override def output: Seq[Attribute] = cmd.output override def children: Seq[SparkPlan] = Nil @@ -68,11 +71,8 @@ private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkP protected override def doExecute(): RDD[InternalRow] = { sqlContext.sparkContext.parallelize(sideEffectResult, 1) } - - override def argString: String = cmd.toString } - /** * An explain command for users to see how a command will be executed. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 9956c5b09236d..66753fa7f27bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.HiveSerDe @@ -138,6 +139,8 @@ case class CreateDataSourceTableAsSelectCommand( query: LogicalPlan) extends RunnableCommand { + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) + override def run(sparkSession: SparkSession): Seq[Row] = { // Since we are saving metadata to metastore, we need to check if metastore supports // the table name and database name we have for this query. MetaStoreUtils.validateName diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 20c02786ecc5c..b56c200e9e98b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} @@ -50,6 +51,8 @@ case class CreateViewCommand( isTemporary: Boolean) extends RunnableCommand { + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) + // TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is // different from Hive and may not work for some cases like create view on self join. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index 25b901f2db8d0..8549ae96e2f39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.sources.InsertableRelation @@ -32,6 +33,8 @@ private[sql] case class InsertIntoDataSourceCommand( overwrite: Boolean) extends RunnableCommand { + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) + override def run(sparkSession: SparkSession): Seq[Row] = { val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] val data = Dataset.ofRows(sparkSession, query) From 5a835b99f9852b0c2a35f9c75a51d493474994ea Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 1 Jun 2016 17:55:37 -0700 Subject: [PATCH 0484/1470] [SPARK-15646][SQL] When spark.sql.hive.convertCTAS is true, the conversion rule needs to respect TEXTFILE/SEQUENCEFILE format and the user-defined location ## What changes were proposed in this pull request? When `spark.sql.hive.convertCTAS` is true, for a CTAS statement, we will create a data source table using the default source (i.e. parquet) if the CTAS does not specify any Hive storage format. However, there are two issues with this conversion logic. 1. First, we determine if a CTAS statement defines storage format by checking the serde. However, TEXTFILE/SEQUENCEFILE does not have a default serde. When we do the check, we have not set the default serde. So, a query like `CREATE TABLE abc STORED AS TEXTFILE AS SELECT ...` actually creates a data source parquet table. 2. In the conversion logic, we are ignoring the user-specified location. This PR fixes the above two issues. Also, this PR makes the parser throws an exception when a CTAS statement has a PARTITIONED BY clause. This change is made because Hive's syntax does not allow it and our current implementation actually does not work for this case (the insert operation always throws an exception because the insertion does not pick up the partitioning info). ## How was this patch tested? I am adding new tests in SQLQuerySuite and HiveDDLCommandSuite. Author: Yin Huai Closes #13386 from yhuai/SPARK-14507. (cherry picked from commit 6dddb70c387ed1f002d2602b2b1f919ef021de91) Signed-off-by: Andrew Or --- .../spark/sql/execution/SparkSqlParser.scala | 37 ++++- .../spark/sql/execution/command/tables.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 10 ++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 57 ++------ .../spark/sql/hive/HiveSessionState.scala | 16 --- .../org/apache/spark/sql/hive/HiveUtils.scala | 6 - ...a => CreateHiveTableAsSelectCommand.scala} | 9 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 25 ++-- .../sql/hive/execution/HiveExplainSuite.scala | 6 +- .../sql/hive/execution/SQLQuerySuite.scala | 135 +++++++++++++----- 10 files changed, 176 insertions(+), 127 deletions(-) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/{CreateTableAsSelectCommand.scala => CreateHiveTableAsSelectCommand.scala} (95%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 6c19bf02dc11e..01409c6a77c1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -839,7 +839,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { /** * Create a table, returning either a [[CreateTableCommand]] or a - * [[CreateTableAsSelectLogicalPlan]]. + * [[CreateHiveTableAsSelectLogicalPlan]]. * * This is not used to create datasource tables, which is handled through * "CREATE TABLE ... USING ...". @@ -936,7 +936,40 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { comment = comment) selectQuery match { - case Some(q) => CreateTableAsSelectLogicalPlan(tableDesc, q, ifNotExists) + case Some(q) => + // Hive does not allow to use a CTAS statement to create a partitioned table. + if (tableDesc.partitionColumnNames.nonEmpty) { + val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " + + "create a partitioned table using Hive's file formats. " + + "Please use the syntax of \"CREATE TABLE tableName USING dataSource " + + "OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a " + + "CTAS statement." + throw operationNotAllowed(errorMessage, ctx) + } + + val hasStorageProperties = (ctx.createFileFormat != null) || (ctx.rowFormat != null) + if (conf.convertCTAS && !hasStorageProperties) { + val mode = if (ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists + // At here, both rowStorage.serdeProperties and fileStorage.serdeProperties + // are empty Maps. + val optionsWithPath = if (location.isDefined) { + Map("path" -> location.get) + } else { + Map.empty[String, String] + } + CreateTableUsingAsSelect( + tableIdent = tableDesc.identifier, + provider = conf.defaultDataSourceName, + temporary = false, + partitionColumns = tableDesc.partitionColumnNames.toArray, + bucketSpec = None, + mode = mode, + options = optionsWithPath, + q + ) + } else { + CreateHiveTableAsSelectLogicalPlan(tableDesc, q, ifNotExists) + } case None => CreateTableCommand(tableDesc, ifNotExists) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 1b89c6b9ce884..90db7853329f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -case class CreateTableAsSelectLogicalPlan( +case class CreateHiveTableAsSelectLogicalPlan( tableDesc: CatalogTable, child: LogicalPlan, allowExisting: Boolean) extends UnaryNode with Command { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d1db0dd800a33..437e093825f6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -310,6 +310,14 @@ object SQLConf { .stringConf .createWithDefault("parquet") + val CONVERT_CTAS = SQLConfigBuilder("spark.sql.hive.convertCTAS") + .internal() + .doc("When true, a table created by a Hive CTAS statement (no USING clause) " + + "without specifying any storage property will be converted to a data source table, " + + "using the data source set by spark.sql.sources.default.") + .booleanConf + .createWithDefault(false) + // This is used to control the when we will split a schema's JSON string to multiple pieces // in order to fit the JSON string in metastore's table property (by default, the value has // a length restriction of 4000 characters). We will split the JSON string of a schema @@ -632,6 +640,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def defaultDataSourceName: String = getConf(DEFAULT_DATA_SOURCE_NAME) + def convertCTAS: Boolean = getConf(CONVERT_CTAS) + def partitionDiscoveryEnabled(): Boolean = getConf(SQLConf.PARTITION_DISCOVERY_ENABLED) 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 ff395f39b7052..f10afa75f2bfc 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ -import org.apache.spark.sql.execution.command.CreateTableAsSelectLogicalPlan +import org.apache.spark.sql.execution.command.CreateHiveTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.orc.OrcFileFormat @@ -446,53 +446,21 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log case p: LogicalPlan if !p.childrenResolved => p case p: LogicalPlan if p.resolved => p - case p @ CreateTableAsSelectLogicalPlan(table, child, allowExisting) => - val schema = if (table.schema.nonEmpty) { - table.schema + case p @ CreateHiveTableAsSelectLogicalPlan(table, child, allowExisting) => + val desc = if (table.storage.serde.isEmpty) { + // add default serde + table.withNewStorage( + serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) } else { - child.output.map { a => - CatalogColumn(a.name, a.dataType.catalogString, a.nullable) - } + table } - val desc = table.copy(schema = schema) - - if (sessionState.convertCTAS && table.storage.serde.isEmpty) { - // Do the conversion when spark.sql.hive.convertCTAS is true and the query - // does not specify any storage format (file format and storage handler). - if (table.identifier.database.isDefined) { - throw new AnalysisException( - "Cannot specify database name in a CTAS statement " + - "when spark.sql.hive.convertCTAS is set to true.") - } + val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) - val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists - CreateTableUsingAsSelect( - TableIdentifier(desc.identifier.table), - sessionState.conf.defaultDataSourceName, - temporary = false, - Array.empty[String], - bucketSpec = None, - mode, - options = Map.empty[String, String], - child - ) - } else { - val desc = if (table.storage.serde.isEmpty) { - // add default serde - table.withNewStorage( - serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) - } else { - table - } - - val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) - - execution.CreateTableAsSelectCommand( - desc.copy(identifier = TableIdentifier(tblName, Some(dbName))), - child, - allowExisting) - } + execution.CreateHiveTableAsSelectCommand( + desc.copy(identifier = TableIdentifier(tblName, Some(dbName))), + child, + allowExisting) } } @@ -543,6 +511,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log /** * An override of the standard HDFS listing based catalog, that overrides the partition spec with * the information from the metastore. + * * @param tableBasePath The default base path of the Hive metastore table * @param partitionSpec The partition specifications from Hive metastore */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 081d85acb9084..ca8e5f8223968 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -138,22 +138,6 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) conf.getConf(HiveUtils.CONVERT_METASTORE_ORC) } - /** - * When true, a table created by a Hive CTAS statement (no USING clause) will be - * converted to a data source table, using the data source set by spark.sql.sources.default. - * The table in CTAS statement will be converted when it meets any of the following conditions: - * - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or - * a Storage Handler (STORED BY), and the value of hive.default.fileformat in hive-site.xml - * is either TextFile or SequenceFile. - * - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe - * is specified (no ROW FORMAT SERDE clause). - * - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as the file format - * and no SerDe is specified (no ROW FORMAT SERDE clause). - */ - def convertCTAS: Boolean = { - conf.getConf(HiveUtils.CONVERT_CTAS) - } - /** * When true, Hive Thrift server will execute SQL queries asynchronously using a thread pool." */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 88f4a2d2b20ba..9ed357c587c35 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -96,12 +96,6 @@ private[spark] object HiveUtils extends Logging { .booleanConf .createWithDefault(false) - val CONVERT_CTAS = SQLConfigBuilder("spark.sql.hive.convertCTAS") - .doc("When true, a table created by a Hive CTAS statement (no USING clause) will be " + - "converted to a data source table, using the data source set by spark.sql.sources.default.") - .booleanConf - .createWithDefault(false) - val CONVERT_METASTORE_ORC = SQLConfigBuilder("spark.sql.hive.convertMetastoreOrc") .doc("When set to false, Spark SQL will use the Hive SerDe for ORC tables instead of " + "the built in support.") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala similarity index 95% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index cfe614909532b..b8099385a466b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -26,16 +26,17 @@ import org.apache.spark.sql.hive.MetastoreRelation /** * Create table and insert the query result into it. + * * @param tableDesc the Table Describe, which may contains serde, storage handler etc. * @param query the query whose result will be insert into the new relation - * @param allowExisting allow continue working if it's already exists, otherwise + * @param ignoreIfExists allow continue working if it's already exists, otherwise * raise exception */ private[hive] -case class CreateTableAsSelectCommand( +case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, - allowExisting: Boolean) + ignoreIfExists: Boolean) extends RunnableCommand { private val tableIdentifier = tableDesc.identifier @@ -80,7 +81,7 @@ case class CreateTableAsSelectCommand( // add the relation into catalog, just in case of failure occurs while data // processing. if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { - if (allowExisting) { + if (ignoreIfExists) { // table already exists, will do nothing, to keep consistent with Hive } else { throw new AnalysisException(s"$tableIdentifier already exists.") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 3297a6f6c3461..ba9fe54db86ee 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -36,7 +36,7 @@ class HiveDDLCommandSuite extends PlanTest { private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { parser.parsePlan(sql).collect { case c: CreateTableCommand => (c.table, c.ifNotExists) - case c: CreateTableAsSelectLogicalPlan => (c.tableDesc, c.allowExisting) + case c: CreateHiveTableAsSelectLogicalPlan => (c.tableDesc, c.allowExisting) case c: CreateViewCommand => (c.tableDesc, c.allowExisting) }.head } @@ -58,7 +58,6 @@ class HiveDDLCommandSuite extends PlanTest { |ip STRING COMMENT 'IP Address of the User', |country STRING COMMENT 'country of origination') |COMMENT 'This is the staging page view table' - |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') |STORED AS RCFILE |LOCATION '/user/external/page_view' |TBLPROPERTIES ('p1'='v1', 'p2'='v2') @@ -76,16 +75,12 @@ class HiveDDLCommandSuite extends PlanTest { CatalogColumn("page_url", "string") :: CatalogColumn("referrer_url", "string") :: CatalogColumn("ip", "string", comment = Some("IP Address of the User")) :: - CatalogColumn("country", "string", comment = Some("country of origination")) :: - CatalogColumn("dt", "string", comment = Some("date type")) :: - CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) + CatalogColumn("country", "string", comment = Some("country of origination")) :: Nil) assert(desc.comment == Some("This is the staging page view table")) // TODO will be SQLText assert(desc.viewText.isEmpty) assert(desc.viewOriginalText.isEmpty) - assert(desc.partitionColumns == - CatalogColumn("dt", "string", comment = Some("date type")) :: - CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) + assert(desc.partitionColumns == Seq.empty[CatalogColumn]) assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) assert(desc.storage.serde == @@ -103,7 +98,6 @@ class HiveDDLCommandSuite extends PlanTest { |ip STRING COMMENT 'IP Address of the User', |country STRING COMMENT 'country of origination') |COMMENT 'This is the staging page view table' - |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') |ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe' | STORED AS | INPUTFORMAT 'parquet.hive.DeprecatedParquetInputFormat' @@ -124,16 +118,12 @@ class HiveDDLCommandSuite extends PlanTest { CatalogColumn("page_url", "string") :: CatalogColumn("referrer_url", "string") :: CatalogColumn("ip", "string", comment = Some("IP Address of the User")) :: - CatalogColumn("country", "string", comment = Some("country of origination")) :: - CatalogColumn("dt", "string", comment = Some("date type")) :: - CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) + CatalogColumn("country", "string", comment = Some("country of origination")) :: Nil) // TODO will be SQLText assert(desc.comment == Some("This is the staging page view table")) assert(desc.viewText.isEmpty) assert(desc.viewOriginalText.isEmpty) - assert(desc.partitionColumns == - CatalogColumn("dt", "string", comment = Some("date type")) :: - CatalogColumn("hour", "string", comment = Some("hour of the day")) :: Nil) + assert(desc.partitionColumns == Seq.empty[CatalogColumn]) assert(desc.storage.serdeProperties == Map()) assert(desc.storage.inputFormat == Some("parquet.hive.DeprecatedParquetInputFormat")) assert(desc.storage.outputFormat == Some("parquet.hive.DeprecatedParquetOutputFormat")) @@ -195,6 +185,11 @@ class HiveDDLCommandSuite extends PlanTest { assert(desc.properties == Map(("tbl_p1" -> "p11"), ("tbl_p2" -> "p22"))) } + test("CTAS statement with a PARTITIONED BY clause is not allowed") { + assertUnsupported(s"CREATE TABLE ctas1 PARTITIONED BY (k int)" + + " AS SELECT key, value FROM (SELECT 1 as key, 2 as value) tmp") + } + test("unsupported operations") { intercept[ParseException] { parser.parsePlan( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 0d08f7edc8ea8..a43eed9a2a4fd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -53,7 +53,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto "== Analyzed Logical Plan ==", "== Optimized Logical Plan ==", "== Physical Plan ==", - "CreateTableAsSelect", + "CreateHiveTableAsSelect", "InsertIntoHiveTable", "Limit", "src") @@ -71,7 +71,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto "== Analyzed Logical Plan ==", "== Optimized Logical Plan ==", "== Physical Plan ==", - "CreateTableAsSelect", + "CreateHiveTableAsSelect", "InsertIntoHiveTable", "Limit", "src") @@ -92,7 +92,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val shouldContain = "== Parsed Logical Plan ==" :: "== Analyzed Logical Plan ==" :: "Subquery" :: "== Optimized Logical Plan ==" :: "== Physical Plan ==" :: - "CreateTableAsSelect" :: "InsertIntoHiveTable" :: "jt" :: Nil + "CreateHiveTableAsSelect" :: "InsertIntoHiveTable" :: "jt" :: Nil for (key <- shouldContain) { assert(outputs.contains(key), s"$key doesn't exist in result") } 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 2a9b06b75efa1..b5691450ca2f0 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 @@ -24,11 +24,14 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveUtils, MetastoreRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -376,78 +379,138 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } - test("CTAS without serde") { - def checkRelation(tableName: String, isDataSourceParquet: Boolean): Unit = { - val relation = EliminateSubqueryAliases( - sessionState.catalog.lookupRelation(TableIdentifier(tableName))) - relation match { - case LogicalRelation(r: HadoopFsRelation, _, _) => - if (!isDataSourceParquet) { - fail( - s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + + def checkRelation( + tableName: String, + isDataSourceParquet: Boolean, + format: String, + userSpecifiedLocation: Option[String] = None): Unit = { + val relation = EliminateSubqueryAliases( + sessionState.catalog.lookupRelation(TableIdentifier(tableName))) + val catalogTable = + sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + relation match { + case LogicalRelation(r: HadoopFsRelation, _, _) => + if (!isDataSourceParquet) { + fail( + s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + s"${HadoopFsRelation.getClass.getCanonicalName}.") - } + } + userSpecifiedLocation match { + case Some(location) => + assert(r.options("path") === location) + case None => // OK. + } + assert( + catalogTable.properties(CreateDataSourceTableUtils.DATASOURCE_PROVIDER) === format) - case r: MetastoreRelation => - if (isDataSourceParquet) { - fail( - s"${HadoopFsRelation.getClass.getCanonicalName} is expected, but found " + + case r: MetastoreRelation => + if (isDataSourceParquet) { + fail( + s"${HadoopFsRelation.getClass.getCanonicalName} is expected, but found " + s"${classOf[MetastoreRelation].getCanonicalName}.") - } - } + } + userSpecifiedLocation match { + case Some(location) => + assert(r.catalogTable.storage.locationUri.get === location) + case None => // OK. + } + // Also make sure that the format is the desired format. + assert(catalogTable.storage.inputFormat.get.toLowerCase.contains(format)) } - val originalConf = sessionState.convertCTAS + // When a user-specified location is defined, the table type needs to be EXTERNAL. + val actualTableType = catalogTable.tableType + userSpecifiedLocation match { + case Some(location) => + assert(actualTableType === CatalogTableType.EXTERNAL) + case None => + assert(actualTableType === CatalogTableType.MANAGED) + } + } - setConf(HiveUtils.CONVERT_CTAS, true) + test("CTAS without serde without location") { + val originalConf = sessionState.conf.convertCTAS + setConf(SQLConf.CONVERT_CTAS, true) + + val defaultDataSource = sessionState.conf.defaultDataSourceName try { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") sql("CREATE TABLE IF NOT EXISTS ctas1 AS SELECT key k, value FROM src ORDER BY k, value") - var message = intercept[AnalysisException] { + val message = intercept[AnalysisException] { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") }.getMessage assert(message.contains("already exists")) - checkRelation("ctas1", true) + checkRelation("ctas1", true, defaultDataSource) sql("DROP TABLE ctas1") // Specifying database name for query can be converted to data source write path // is not allowed right now. - message = intercept[AnalysisException] { - sql("CREATE TABLE default.ctas1 AS SELECT key k, value FROM src ORDER BY k, value") - }.getMessage - assert( - message.contains("Cannot specify database name in a CTAS statement"), - "When spark.sql.hive.convertCTAS is true, we should not allow " + - "database name specified.") + sql("CREATE TABLE default.ctas1 AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true, defaultDataSource) + sql("DROP TABLE ctas1") sql("CREATE TABLE ctas1 stored as textfile" + " AS SELECT key k, value FROM src ORDER BY k, value") - checkRelation("ctas1", true) + checkRelation("ctas1", false, "text") sql("DROP TABLE ctas1") sql("CREATE TABLE ctas1 stored as sequencefile" + " AS SELECT key k, value FROM src ORDER BY k, value") - checkRelation("ctas1", true) + checkRelation("ctas1", false, "sequence") sql("DROP TABLE ctas1") sql("CREATE TABLE ctas1 stored as rcfile AS SELECT key k, value FROM src ORDER BY k, value") - checkRelation("ctas1", false) + checkRelation("ctas1", false, "rcfile") sql("DROP TABLE ctas1") sql("CREATE TABLE ctas1 stored as orc AS SELECT key k, value FROM src ORDER BY k, value") - checkRelation("ctas1", false) + checkRelation("ctas1", false, "orc") sql("DROP TABLE ctas1") sql("CREATE TABLE ctas1 stored as parquet AS SELECT key k, value FROM src ORDER BY k, value") - checkRelation("ctas1", false) + checkRelation("ctas1", false, "parquet") sql("DROP TABLE ctas1") } finally { - setConf(HiveUtils.CONVERT_CTAS, originalConf) + setConf(SQLConf.CONVERT_CTAS, originalConf) sql("DROP TABLE IF EXISTS ctas1") } } + test("CTAS without serde with location") { + withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { + withTempDir { dir => + val defaultDataSource = sessionState.conf.defaultDataSourceName + + val tempLocation = dir.getCanonicalPath + sql(s"CREATE TABLE ctas1 LOCATION 'file:$tempLocation/c1'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true, defaultDataSource, Some(s"file:$tempLocation/c1")) + sql("DROP TABLE ctas1") + + sql(s"CREATE TABLE ctas1 LOCATION 'file:$tempLocation/c2'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true, defaultDataSource, Some(s"file:$tempLocation/c2")) + sql("DROP TABLE ctas1") + + sql(s"CREATE TABLE ctas1 stored as textfile LOCATION 'file:$tempLocation/c3'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false, "text", Some(s"file:$tempLocation/c3")) + sql("DROP TABLE ctas1") + + sql(s"CREATE TABLE ctas1 stored as sequenceFile LOCATION 'file:$tempLocation/c4'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false, "sequence", Some(s"file:$tempLocation/c4")) + sql("DROP TABLE ctas1") + + sql(s"CREATE TABLE ctas1 stored as rcfile LOCATION 'file:$tempLocation/c5'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false, "rcfile", Some(s"file:$tempLocation/c5")) + sql("DROP TABLE ctas1") + } + } + } + test("CTAS with serde") { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect() sql( @@ -785,8 +848,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // generates an invalid query plan. val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) read.json(rdd).createOrReplaceTempView("data") - val originalConf = sessionState.convertCTAS - setConf(HiveUtils.CONVERT_CTAS, false) + val originalConf = sessionState.conf.convertCTAS + setConf(SQLConf.CONVERT_CTAS, false) try { sql("CREATE TABLE explodeTest (key bigInt)") @@ -805,7 +868,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql("DROP TABLE explodeTest") dropTempTable("data") } finally { - setConf(HiveUtils.CONVERT_CTAS, originalConf) + setConf(SQLConf.CONVERT_CTAS, originalConf) } } From d17db34e073311687d128a28fe95ba36a07394a1 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 1 Jun 2016 21:58:05 -0700 Subject: [PATCH 0485/1470] [SPARK-15620][SQL] Fix transformed dataset attributes revolve failure ## What changes were proposed in this pull request? Join on transformed dataset has attributes conflicts, which make query execution failure, for example: ``` val dataset = Seq(1, 2, 3).toDs val mappedDs = dataset.map(_ + 1) mappedDs.as("t1").joinWith(mappedDs.as("t2"), $"t1.value" === $"t2.value").show() ``` will throw exception: ``` org.apache.spark.sql.AnalysisException: cannot resolve '`t1.value`' given input columns: [value]; at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:62) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:59) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:287) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:287) ``` ## How was this patch tested? Unit test. Author: jerryshao Closes #13399 from jerryshao/SPARK-15620. (cherry picked from commit 8288e16a5a5a12a45207c13a1341c707c6b4b940) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 ++++ .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 10 ++++++++++ 2 files changed, 14 insertions(+) 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 bf221e0d7cfc4..eb46c0e72ecd9 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 @@ -524,6 +524,10 @@ class Analyzer( val newVersion = oldVersion.newInstance() (oldVersion, newVersion) + case oldVersion: SerializeFromObject + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + (oldVersion, oldVersion.copy(serializer = oldVersion.serializer.map(_.newInstance()))) + // Handle projects that create conflicting aliases. case oldVersion @ Project(projectList, _) if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 0b6874e3b8ad3..a3881ff920159 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -769,6 +769,16 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } } + test("mapped dataset should resolve duplicated attributes for self join") { + val ds = Seq(1, 2, 3).toDS().map(_ + 1) + val ds1 = ds.as("d1") + val ds2 = ds.as("d2") + + checkDataset(ds1.joinWith(ds2, $"d1.value" === $"d2.value"), (2, 2), (3, 3), (4, 4)) + checkDataset(ds1.intersect(ds2), 2, 3, 4) + checkDataset(ds1.except(ds1)) + } + test("SPARK-15441: Dataset outer join") { val left = Seq(ClassData("a", 1), ClassData("b", 2)).toDS().as("left") val right = Seq(ClassData("x", 2), ClassData("y", 3)).toDS().as("right") From 22eb08369a27953d064baff3f72d3d01e1d48b9a Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Wed, 1 Jun 2016 22:23:00 -0700 Subject: [PATCH 0486/1470] [SPARK-13484][SQL] Prevent illegal NULL propagation when filtering outer-join results ## What changes were proposed in this pull request? This PR add a rule at the end of analyzer to correct nullable fields of attributes in a logical plan by using nullable fields of the corresponding attributes in its children logical plans (these plans generate the input rows). This is another approach for addressing SPARK-13484 (the first approach is https://github.com/apache/spark/pull/11371). Close #113711 Author: Takeshi YAMAMURO Author: Yin Huai Closes #13290 from yhuai/SPARK-13484. (cherry picked from commit 5eea332307cbed5fc44427959f070afc16a12c02) Signed-off-by: Cheng Lian --- .../sql/catalyst/analysis/Analyzer.scala | 37 ++++++++++++++++++- .../analysis/ResolveNaturalJoinSuite.scala | 2 +- .../apache/spark/sql/DataFrameJoinSuite.scala | 21 +++++++++++ 3 files changed, 58 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 eb46c0e72ecd9..02966796afdd7 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 @@ -113,6 +113,8 @@ class Analyzer( PullOutNondeterministic), Batch("UDF", Once, HandleNullInputsForUDF), + Batch("FixNullability", Once, + FixNullability), Batch("Cleanup", fixedPoint, CleanupAliases) ) @@ -1451,6 +1453,40 @@ class Analyzer( } } + /** + * Fixes nullability of Attributes in a resolved LogicalPlan by using the nullability of + * corresponding Attributes of its children output Attributes. This step is needed because + * users can use a resolved AttributeReference in the Dataset API and outer joins + * can change the nullability of an AttribtueReference. Without the fix, a nullable column's + * nullable field can be actually set as non-nullable, which cause illegal optimization + * (e.g., NULL propagation) and wrong answers. + * See SPARK-13484 and SPARK-13801 for the concrete queries of this case. + */ + object FixNullability extends Rule[LogicalPlan] { + + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case p if !p.resolved => p // Skip unresolved nodes. + case p: LogicalPlan if p.resolved => + val childrenOutput = p.children.flatMap(c => c.output).groupBy(_.exprId).flatMap { + case (exprId, attributes) => + // If there are multiple Attributes having the same ExprId, we need to resolve + // the conflict of nullable field. We do not really expect this happen. + val nullable = attributes.exists(_.nullable) + attributes.map(attr => attr.withNullability(nullable)) + }.toSeq + // At here, we create an AttributeMap that only compare the exprId for the lookup + // operation. So, we can find the corresponding input attribute's nullability. + val attributeMap = AttributeMap[Attribute](childrenOutput.map(attr => attr -> attr)) + // For an Attribute used by the current LogicalPlan, if it is from its children, + // we fix the nullable field by using the nullability setting of the corresponding + // output Attribute from the children. + p.transformExpressions { + case attr: Attribute if attributeMap.contains(attr) => + attr.withNullability(attributeMap(attr).nullable) + } + } + } + /** * Extracts [[WindowExpression]]s from the projectList of a [[Project]] operator and * aggregateExpressions of an [[Aggregate]] operator and creates individual [[Window]] @@ -2133,4 +2169,3 @@ object TimeWindowing extends Rule[LogicalPlan] { } } } - diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala index 1423a8705af27..748579df41580 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala @@ -100,7 +100,7 @@ class ResolveNaturalJoinSuite extends AnalysisTest { val naturalPlan = r3.join(r4, NaturalJoin(FullOuter), None) val usingPlan = r3.join(r4, UsingJoin(FullOuter, Seq(UnresolvedAttribute("b"))), None) val expected = r3.join(r4, FullOuter, Some(EqualTo(bNotNull, bNotNull))).select( - Alias(Coalesce(Seq(bNotNull, bNotNull)), "b")(), a, c) + Alias(Coalesce(Seq(b, b)), "b")(), a, c) checkAnalysis(naturalPlan, expected) checkAnalysis(usingPlan, expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 031e66b57cbcb..4342c039aefc8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -204,4 +204,25 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { leftJoin2Inner, Row(1, 2, "1", 1, 3, "1") :: Nil) } + + test("process outer join results using the non-nullable columns in the join input") { + // Filter data using a non-nullable column from a right table + val df1 = Seq((0, 0), (1, 0), (2, 0), (3, 0), (4, 0)).toDF("id", "count") + val df2 = Seq(Tuple1(0), Tuple1(1)).toDF("id").groupBy("id").count + checkAnswer( + df1.join(df2, df1("id") === df2("id"), "left_outer").filter(df2("count").isNull), + Row(2, 0, null, null) :: + Row(3, 0, null, null) :: + Row(4, 0, null, null) :: Nil + ) + + // Coalesce data using non-nullable columns in input tables + val df3 = Seq((1, 1)).toDF("a", "b") + val df4 = Seq((2, 2)).toDF("a", "b") + checkAnswer( + df3.join(df4, df3("a") === df4("a"), "outer") + .select(coalesce(df3("a"), df3("b")), coalesce(df4("a"), df4("b"))), + Row(1, null) :: Row(null, 2) :: Nil + ) + } } From 8f98eef6918451ec372d527576246579f7d14b23 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Thu, 2 Jun 2016 11:07:15 -0500 Subject: [PATCH 0487/1470] [SPARK-15208][WIP][CORE][STREAMING][DOCS] Update Spark examples with AccumulatorV2 ## What changes were proposed in this pull request? The patch updates the codes & docs in the example module as well as the related doc module: - [ ] [docs] `streaming-programming-guide.md` - [x] scala code part - [ ] java code part - [ ] python code part - [x] [examples] `RecoverableNetworkWordCount.scala` - [ ] [examples] `JavaRecoverableNetworkWordCount.java` - [ ] [examples] `recoverable_network_wordcount.py` ## How was this patch tested? Ran the examples and verified results manually. Author: Liwei Lin Closes #12981 from lw-lin/accumulatorV2-examples. (cherry picked from commit a0eec8e8ffd5a43cae67aa0f5dbcf7ca19a4f3aa) Signed-off-by: Sean Owen --- docs/streaming-programming-guide.md | 12 ++++++------ .../streaming/RecoverableNetworkWordCount.scala | 3 +-- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 6550fcc0521c3..78ae6a7407467 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1395,13 +1395,13 @@ object WordBlacklist { object DroppedWordsCounter { - @volatile private var instance: Accumulator[Long] = null + @volatile private var instance: LongAccumulator = null - def getInstance(sc: SparkContext): Accumulator[Long] = { + def getInstance(sc: SparkContext): LongAccumulator = { if (instance == null) { synchronized { if (instance == null) { - instance = sc.accumulator(0L, "WordsInBlacklistCounter") + instance = sc.longAccumulator("WordsInBlacklistCounter") } } } @@ -1409,7 +1409,7 @@ object DroppedWordsCounter { } } -wordCounts.foreachRDD((rdd: RDD[(String, Int)], time: Time) => { +wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => // Get or register the blacklist Broadcast val blacklist = WordBlacklist.getInstance(rdd.sparkContext) // Get or register the droppedWordsCounter Accumulator @@ -1417,12 +1417,12 @@ wordCounts.foreachRDD((rdd: RDD[(String, Int)], time: Time) => { // Use blacklist to drop words and use droppedWordsCounter to count them val counts = rdd.filter { case (word, count) => if (blacklist.value.contains(word)) { - droppedWordsCounter += count + droppedWordsCounter.add(count) false } else { true } - }.collect() + }.collect().mkString("[", ", ", "]") val output = "Counts at time " + time + " " + counts }) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index acbcb0c4b7b78..49c0427321133 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -27,8 +27,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext, Time} -import org.apache.spark.util.IntParam -import org.apache.spark.util.LongAccumulator +import org.apache.spark.util.{IntParam, LongAccumulator} /** * Use this singleton to get or register a Broadcast variable. From b78352fc37a25db5607e08874630539117d96005 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 2 Jun 2016 11:10:13 -0500 Subject: [PATCH 0488/1470] [SPARK-15605][ML][EXAMPLES] Fix broken ML JavaDeveloperApiExample. ## What changes were proposed in this pull request? See [SPARK-15605](https://issues.apache.org/jira/browse/SPARK-15605) for the detail of this bug. This PR fix 2 major bugs in this example: * The java example class use Param ```maxIter```, it will fail when calling ```Param.shouldOwn```. We need add a public method which return the ```maxIter``` Object. Because ```Params.params``` use java reflection to list all public method whose return type is ```Param```, and invoke them to get all defined param objects in the instance. * The ```uid``` member defined in Java class will be initialized after Scala traits such as ```HasFeaturesCol```. So when ```HasFeaturesCol``` being constructed, they get ```uid``` with null, which will cause ```Param.shouldOwn``` check fail. so, here is my changes: * Add public method: ```public IntParam getMaxIterParam() {return maxIter;}``` * Use Java anonymous class overriding ```uid()``` to defined the ```uid```, and it solve the second problem described above. * To make the ```getMaxIterParam ``` can be invoked using java reflection, we must make the two class (MyJavaLogisticRegression and MyJavaLogisticRegressionModel) public. so I make them become inner public static class. ## How was this patch tested? Offline tests. Author: Yanbo Liang Closes #13353 from yanboliang/spark-15605. (cherry picked from commit 4fe7c7bd1eeb4ae954e750e521e6d72dc8ff5225) Signed-off-by: Sean Owen --- .../examples/ml/JavaDeveloperApiExample.java | 240 ------------------ 1 file changed, 240 deletions(-) delete mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java 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 deleted file mode 100644 index 8971c0ac002b3..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ /dev/null @@ -1,240 +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.examples.ml; - -import java.util.List; - -import com.google.common.collect.Lists; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.classification.Classifier; -import org.apache.spark.ml.classification.ClassificationModel; -import org.apache.spark.ml.feature.LabeledPoint; -import org.apache.spark.ml.linalg.BLAS; -import org.apache.spark.ml.linalg.Vector; -import org.apache.spark.ml.linalg.Vectors; -import org.apache.spark.ml.param.IntParam; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.util.Identifiable$; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; - - -/** - * A simple example demonstrating how to write your own learning algorithm using Estimator, - * Transformer, and other abstractions. - * This mimics {@link org.apache.spark.ml.classification.LogisticRegression}. - * - * Run with - *
    - * bin/run-example ml.JavaDeveloperApiExample
    - * 
    - */ -public class JavaDeveloperApiExample { - - public static void main(String[] args) throws Exception { - SparkSession spark = SparkSession - .builder() - .appName("JavaDeveloperApiExample") - .getOrCreate(); - - // Prepare training data. - List localTraining = Lists.newArrayList( - new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - Dataset training = spark.createDataFrame(localTraining, LabeledPoint.class); - - // Create a LogisticRegression instance. This instance is an Estimator. - MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); - // Print out the parameters, documentation, and any default values. - System.out.println("MyJavaLogisticRegression parameters:\n" + lr.explainParams() + "\n"); - - // We may set parameters using setter methods. - lr.setMaxIter(10); - - // Learn a LogisticRegression model. This uses the parameters stored in lr. - MyJavaLogisticRegressionModel model = lr.fit(training); - - // Prepare test data. - List localTest = Lists.newArrayList( - new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), - new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), - new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - Dataset test = spark.createDataFrame(localTest, LabeledPoint.class); - - // Make predictions on test documents. cvModel uses the best model found (lrModel). - Dataset results = model.transform(test); - double sumPredictions = 0; - for (Row r : results.select("features", "label", "prediction").collectAsList()) { - sumPredictions += r.getDouble(2); - } - if (sumPredictions != 0.0) { - throw new Exception("MyJavaLogisticRegression predicted something other than 0," + - " even though all coefficients are 0!"); - } - - spark.stop(); - } -} - -/** - * Example of defining a type of {@link Classifier}. - * - * Note: Some IDEs (e.g., IntelliJ) will complain that this will not compile due to - * {@link org.apache.spark.ml.param.Params#set} using incompatible return types. - * However, this should still compile and run successfully. - */ -class MyJavaLogisticRegression - extends Classifier { - - MyJavaLogisticRegression() { - init(); - } - - MyJavaLogisticRegression(String uid) { - this.uid_ = uid; - init(); - } - - private String uid_ = Identifiable$.MODULE$.randomUID("myJavaLogReg"); - - @Override - public String uid() { - return uid_; - } - - /** - * Param for max number of iterations - *

    - * NOTE: The usual way to add a parameter to a model or algorithm is to include: - * - val myParamName: ParamType - * - def getMyParamName - * - def setMyParamName - */ - IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); - - int getMaxIter() { return (Integer) getOrDefault(maxIter); } - - private void init() { - setMaxIter(100); - } - - // The parameter setter is in this class since it should return type MyJavaLogisticRegression. - MyJavaLogisticRegression setMaxIter(int value) { - return (MyJavaLogisticRegression) set(maxIter, value); - } - - // This method is used by fit(). - // In Java, we have to make it public since Java does not understand Scala's protected modifier. - public MyJavaLogisticRegressionModel train(Dataset dataset) { - // Extract columns from data using helper method. - JavaRDD oldDataset = extractLabeledPoints(dataset).toJavaRDD(); - - // Do learning to estimate the coefficients vector. - int numFeatures = oldDataset.take(1).get(0).features().size(); - Vector coefficients = Vectors.zeros(numFeatures); // Learning would happen here. - - // Create a model, and return it. - return new MyJavaLogisticRegressionModel(uid(), coefficients).setParent(this); - } - - @Override - public MyJavaLogisticRegression copy(ParamMap extra) { - return defaultCopy(extra); - } -} - -/** - * Example of defining a type of {@link ClassificationModel}. - * - * Note: Some IDEs (e.g., IntelliJ) will complain that this will not compile due to - * {@link org.apache.spark.ml.param.Params#set} using incompatible return types. - * However, this should still compile and run successfully. - */ -class MyJavaLogisticRegressionModel - extends ClassificationModel { - - private Vector coefficients_; - public Vector coefficients() { return coefficients_; } - - MyJavaLogisticRegressionModel(String uid, Vector coefficients) { - this.uid_ = uid; - this.coefficients_ = coefficients; - } - - private String uid_ = Identifiable$.MODULE$.randomUID("myJavaLogReg"); - - @Override - public String uid() { - return uid_; - } - - // This uses the default implementation of transform(), which reads column "features" and outputs - // columns "prediction" and "rawPrediction." - - // This uses the default implementation of predict(), which chooses the label corresponding to - // the maximum value returned by [[predictRaw()]]. - - /** - * Raw prediction for each possible label. - * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives - * a measure of confidence in each possible label (where larger = more confident). - * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. - * - * @return vector where element i is the raw prediction for label i. - * This raw prediction may be any real number, where a larger value indicates greater - * confidence for that label. - * - * In Java, we have to make this method public since Java does not understand Scala's protected - * modifier. - */ - public Vector predictRaw(Vector features) { - double margin = BLAS.dot(features, coefficients_); - // There are 2 classes (binary classification), so we return a length-2 vector, - // where index i corresponds to class i (i = 0, 1). - return Vectors.dense(-margin, margin); - } - - /** - * Number of classes the label can take. 2 indicates binary classification. - */ - public int numClasses() { return 2; } - - /** - * Number of features the model was trained on. - */ - public int numFeatures() { return coefficients_.size(); } - - /** - * 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 default implementation of [[transform()]]. - * - * In Java, we have to make this method public since Java does not understand Scala's protected - * modifier. - */ - @Override - public MyJavaLogisticRegressionModel copy(ParamMap extra) { - return copyValues(new MyJavaLogisticRegressionModel(uid(), coefficients_), extra) - .setParent(parent()); - } -} From a5cb7fc6d8cb954d1f8ef05d961bcca42b7476b8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 2 Jun 2016 11:12:17 -0500 Subject: [PATCH 0489/1470] [SPARK-15709][SQL] Prevent `freqItems` from raising `UnsupportedOperationException: empty.min` ## What changes were proposed in this pull request? Currently, `freqItems` raises `UnsupportedOperationException` on `empty.min` usually when its `support` argument is high. ```scala scala> spark.createDataset(Seq(1, 2, 2, 3, 3, 3)).stat.freqItems(Seq("value"), 2) 16/06/01 11:11:38 ERROR Executor: Exception in task 5.0 in stage 0.0 (TID 5) java.lang.UnsupportedOperationException: empty.min ... ``` Also, the parameter checking message is wrong. ``` require(support >= 1e-4, s"support ($support) must be greater than 1e-4.") ``` This PR changes the logic to handle the `empty` case and also improves parameter checking. ## How was this patch tested? Pass the Jenkins tests (with a new testcase). Author: Dongjoon Hyun Closes #13449 from dongjoon-hyun/SPARK-15709. (cherry picked from commit b85d18f3bdedca7ae7f2c26ff64ce38c2796bd63) Signed-off-by: Sean Owen --- .../spark/sql/execution/stat/FrequentItems.scala | 4 ++-- .../org/apache/spark/sql/DataFrameStatSuite.scala | 11 +++++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index 34bd243d58de9..b19344f04383f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -40,7 +40,7 @@ private[sql] object FrequentItems extends Logging { if (baseMap.size < size) { baseMap += key -> count } else { - val minCount = baseMap.values.min + val minCount = if (baseMap.values.isEmpty) 0 else baseMap.values.min val remainder = count - minCount if (remainder >= 0) { baseMap += key -> count // something will get kicked out, so we can add this @@ -83,7 +83,7 @@ private[sql] object FrequentItems extends Logging { df: DataFrame, cols: Seq[String], support: Double): DataFrame = { - require(support >= 1e-4, s"support ($support) must be greater than 1e-4.") + require(support >= 1e-4 && support <= 1.0, s"Support must be in [1e-4, 1], but got $support.") val numCols = cols.length // number of max items to keep counts for val sizeOfMap = (1 / support).toInt diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index ab7733b239f28..73026c749db45 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -235,6 +235,17 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { assert(items.length === 1) } + test("SPARK-15709: Prevent `UnsupportedOperationException: empty.min` in `freqItems`") { + val ds = spark.createDataset(Seq(1, 2, 2, 3, 3, 3)) + + intercept[IllegalArgumentException] { + ds.stat.freqItems(Seq("value"), 0) + } + intercept[IllegalArgumentException] { + ds.stat.freqItems(Seq("value"), 2) + } + } + test("sampleBy") { val df = spark.range(0, 100).select((col("id") % 3).as("key")) val sampled = df.stat.sampleBy("key", Map(0 -> 0.1, 1 -> 0.2), 0L) From fe639adea9892db8850afeeb3fb7290eea77fce0 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 2 Jun 2016 11:16:24 -0500 Subject: [PATCH 0490/1470] [SPARK-15322][SQL][FOLLOWUP] Use the new long accumulator for old int accumulators. ## What changes were proposed in this pull request? This PR corrects the remaining cases for using old accumulators. This does not change some old accumulator usages below: - `ImplicitSuite.scala` - Tests dedicated to old accumulator, for implicits with `AccumulatorParam` - `AccumulatorSuite.scala` - Tests dedicated to old accumulator - `JavaSparkContext.scala` - For supporting old accumulators for Java API. - `debug.package.scala` - Usage with `HashSet[String]`. Currently, it seems no implementation for this. I might be able to write an anonymous class for this but I didn't because I think it is not worth writing a lot of codes only for this. - `SQLMetricsSuite.scala` - This uses the old accumulator for checking type boxing. It seems new accumulator does not require type boxing for this case whereas the old one requires (due to the use of generic). ## How was this patch tested? Existing tests cover this. Author: hyukjinkwon Closes #13434 from HyukjinKwon/accum. (cherry picked from commit 252417fa21eb47781addfd614ff00dac793b52a9) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/DistributedSuite.scala | 5 ++--- .../test/scala/org/apache/spark/repl/ReplSuite.scala | 6 +++--- .../test/scala/org/apache/spark/repl/ReplSuite.scala | 6 +++--- .../execution/columnar/InMemoryTableScanExec.scala | 11 +++++------ .../apache/spark/sql/execution/debug/package.scala | 7 ++++--- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 6 +++--- .../spark/sql/execution/ui/SQLListenerSuite.scala | 4 ++-- 7 files changed, 22 insertions(+), 23 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 0be25e9f893b6..6e69fc4247079 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -92,8 +92,8 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("accumulators") { sc = new SparkContext(clusterUrl, "test") - val accum = sc.accumulator(0) - sc.parallelize(1 to 10, 10).foreach(x => accum += x) + val accum = sc.longAccumulator + sc.parallelize(1 to 10, 10).foreach(x => accum.add(x)) assert(accum.value === 55) } @@ -109,7 +109,6 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("repeatedly failing task") { sc = new SparkContext(clusterUrl, "test") - val accum = sc.accumulator(0) val thrown = intercept[SparkException] { // scalastyle:off println sc.parallelize(1 to 10, 10).foreach(x => println(x / 0)) diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 547da8f713ac7..19f201f606dee 100644 --- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -107,13 +107,13 @@ class ReplSuite extends SparkFunSuite { test("simple foreach with accumulator") { val output = runInterpreter("local", """ - |val accum = sc.accumulator(0) - |sc.parallelize(1 to 10).foreach(x => accum += x) + |val accum = sc.longAccumulator + |sc.parallelize(1 to 10).foreach(x => accum.add(x)) |accum.value """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) - assertContains("res1: Int = 55", output) + assertContains("res1: Long = 55", output) } test("external vars") { diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 125686030c01f..48582c19163c9 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -150,13 +150,13 @@ class ReplSuite extends SparkFunSuite { test("simple foreach with accumulator") { val output = runInterpreter("local", """ - |val accum = sc.accumulator(0) - |sc.parallelize(1 to 10).foreach(x => accum += x) + |val accum = sc.longAccumulator + |sc.parallelize(1 to 10).foreach(x => accum.add(x)) |accum.value """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) - assertContains("res1: Int = 55", output) + assertContains("res1: Long = 55", output) } test("external vars") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 7ccc9de9db233..bd55e1a8751da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -21,7 +21,6 @@ import scala.collection.JavaConverters._ import org.apache.commons.lang.StringUtils -import org.apache.spark.Accumulator import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -36,7 +35,7 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{AccumulatorContext, ListAccumulator} +import org.apache.spark.util.{AccumulatorContext, ListAccumulator, LongAccumulator} private[sql] object InMemoryRelation { @@ -294,8 +293,8 @@ private[sql] case class InMemoryTableScanExec( sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean // Accumulators used for testing purposes - lazy val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) - lazy val readBatches: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readPartitions = sparkContext.longAccumulator + lazy val readBatches = sparkContext.longAccumulator private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning @@ -339,7 +338,7 @@ private[sql] case class InMemoryTableScanExec( false } else { if (enableAccumulators) { - readBatches += 1 + readBatches.add(1) } true } @@ -361,7 +360,7 @@ private[sql] case class InMemoryTableScanExec( val columnarIterator = GenerateColumnAccessor.generate(columnTypes) columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) if (enableAccumulators && columnarIterator.hasNext) { - readPartitions += 1 + readPartitions.add(1) } columnarIterator } 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 c77c889a1b7b8..f2c558ac2de7a 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 @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.LongAccumulator /** * Contains methods for debugging query execution. @@ -122,13 +123,13 @@ package object debug { /** * A collection of metrics for each column of output. * - * @param elementTypes the actual runtime types for the output. Useful when there are bugs + * @param elementTypes the actual runtime types for the output. Useful when there are bugs * causing the wrong data to be projected. */ case class ColumnMetrics( elementTypes: Accumulator[HashSet[String]] = sparkContext.accumulator(HashSet.empty)) - val tupleCount: Accumulator[Int] = sparkContext.accumulator[Int](0) + val tupleCount: LongAccumulator = sparkContext.longAccumulator val numColumns: Int = child.output.size val columnStats: Array[ColumnMetrics] = Array.fill(child.output.size)(new ColumnMetrics()) @@ -149,7 +150,7 @@ package object debug { def next(): InternalRow = { val currentRow = iter.next() - tupleCount += 1 + tupleCount.add(1) var i = 0 while (i < numColumns) { val value = currentRow.get(i, output(i).dataType) 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 91d93022df377..49a0ba1f1149b 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 @@ -2067,9 +2067,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(df.selectExpr("a + 1", "a + (a + 1)"), Row(2, 3)) // Identity udf that tracks the number of times it is called. - val countAcc = sparkContext.accumulator(0, "CallCount") + val countAcc = sparkContext.longAccumulator("CallCount") spark.udf.register("testUdf", (x: Int) => { - countAcc.++=(1) + countAcc.add(1) x }) @@ -2092,7 +2092,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { df.selectExpr("testUdf(a + 1) + testUdf(1 + b)", "testUdf(a + 1)"), Row(4, 2), 2) val testUdf = functions.udf((x: Int) => { - countAcc.++=(1) + countAcc.add(1) x }) verifyCallCount( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index cf7e976acc65f..6788c9d65f6ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -365,9 +365,9 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { // This task has both accumulators that are SQL metrics and accumulators that are not. // The listener should only track the ones that are actually SQL metrics. val sqlMetric = SQLMetrics.createMetric(sparkContext, "beach umbrella") - val nonSqlMetric = sparkContext.accumulator[Int](0, "baseball") + val nonSqlMetric = sparkContext.longAccumulator("baseball") val sqlMetricInfo = sqlMetric.toInfo(Some(sqlMetric.value), None) - val nonSqlMetricInfo = nonSqlMetric.toInfo(Some(nonSqlMetric.localValue), None) + val nonSqlMetricInfo = nonSqlMetric.toInfo(Some(nonSqlMetric.value), None) val taskInfo = createTaskInfo(0, 0) taskInfo.accumulables ++= Seq(sqlMetricInfo, nonSqlMetricInfo) val taskEnd = SparkListenerTaskEnd(0, 0, "just-a-task", null, taskInfo, null) From 18d613a4d38ab4419ec1e5dbce7e0d73655d99d0 Mon Sep 17 00:00:00 2001 From: Pete Robbins Date: Thu, 2 Jun 2016 10:14:51 -0700 Subject: [PATCH 0491/1470] [SPARK-15606][CORE] Use non-blocking removeExecutor call to avoid deadlocks ## What changes were proposed in this pull request? Set minimum number of dispatcher threads to 3 to avoid deadlocks on machines with only 2 cores ## How was this patch tested? Spark test builds Author: Pete Robbins Closes #13355 from robbinspg/SPARK-13906. (cherry picked from commit 7c07d176f3d65235f9376898a7b10b01268c867c) Signed-off-by: Shixiong Zhu --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 2 +- .../org/apache/spark/storage/BlockManagerMaster.scala | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 0fea9c123bcfb..e84cb6346d51c 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 @@ -295,7 +295,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // manager to reregister itself. If that happens, the block manager master will know // about the executor, but the scheduler will not. Therefore, we should remove the // executor from the block manager when we hit this case. - scheduler.sc.env.blockManager.master.removeExecutor(executorId) + scheduler.sc.env.blockManager.master.removeExecutorAsync(executorId) logInfo(s"Asked to remove non-existent executor $executorId") } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 52db45bd488b9..8655cf10fc28f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -42,6 +42,14 @@ class BlockManagerMaster( logInfo("Removed " + execId + " successfully in removeExecutor") } + /** Request removal of a dead executor from the driver endpoint. + * This is only called on the driver side. Non-blocking + */ + def removeExecutorAsync(execId: String) { + driverEndpoint.ask[Boolean](RemoveExecutor(execId)) + logInfo("Removal of executor " + execId + " requested") + } + /** Register the BlockManager's id with the driver. */ def registerBlockManager( blockManagerId: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef): Unit = { From 841523cdc6194cc1ce93004a8bb8c6af7204a048 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Thu, 2 Jun 2016 10:58:00 -0700 Subject: [PATCH 0492/1470] [SPARK-14752][SQL] Explicitly implement KryoSerialization for LazilyGenerateOrdering ## What changes were proposed in this pull request? This patch fixes a number of `com.esotericsoftware.kryo.KryoException: java.lang.NullPointerException` exceptions reported in [SPARK-15604], [SPARK-14752] etc. (while executing sparkSQL queries with the kryo serializer) by explicitly implementing `KryoSerialization` for `LazilyGenerateOrdering`. ## How was this patch tested? 1. Modified `OrderingSuite` so that all tests in the suite also test kryo serialization (for both interpreted and generated ordering). 2. Manually verified TPC-DS q1. Author: Sameer Agarwal Closes #13466 from sameeragarwal/kryo. (cherry picked from commit 09b3c56c91831b3e8d909521b8f3ffbce4eb0395) Signed-off-by: Reynold Xin --- .../expressions/codegen/GenerateOrdering.scala | 14 +++++++++++++- .../sql/catalyst/expressions/OrderingSuite.scala | 14 ++++++++++---- 2 files changed, 23 insertions(+), 5 deletions(-) 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 c10829d4f14f3..f4d35d232e691 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,6 +19,9 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.io.ObjectInputStream +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -147,7 +150,8 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR /** * A lazily generated row ordering comparator. */ -class LazilyGeneratedOrdering(val ordering: Seq[SortOrder]) extends Ordering[InternalRow] { +class LazilyGeneratedOrdering(val ordering: Seq[SortOrder]) + extends Ordering[InternalRow] with KryoSerializable { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = this(ordering.map(BindReferences.bindReference(_, inputSchema))) @@ -163,6 +167,14 @@ class LazilyGeneratedOrdering(val ordering: Seq[SortOrder]) extends Ordering[Int in.defaultReadObject() generatedOrdering = GenerateOrdering.generate(ordering) } + + override def write(kryo: Kryo, out: Output): Unit = Utils.tryOrIOException { + kryo.writeObject(out, ordering.toArray) + } + + override def read(kryo: Kryo, in: Input): Unit = Utils.tryOrIOException { + generatedOrdering = GenerateOrdering.generate(kryo.readObject(in, classOf[Array[SortOrder]])) + } } object LazilyGeneratedOrdering { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala index b190d3a00dfb8..8cc2ab46c0c85 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.catalyst.expressions import scala.math._ -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering +import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateOrdering, LazilyGeneratedOrdering} import org.apache.spark.sql.types._ class OrderingSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -44,9 +45,14 @@ class OrderingSuite extends SparkFunSuite with ExpressionEvalHelper { case Ascending => signum(expected) case Descending => -1 * signum(expected) } + + val kryo = new KryoSerializer(new SparkConf).newInstance() val intOrdering = new InterpretedOrdering(sortOrder :: Nil) - val genOrdering = GenerateOrdering.generate(sortOrder :: Nil) - Seq(intOrdering, genOrdering).foreach { ordering => + val genOrdering = new LazilyGeneratedOrdering(sortOrder :: Nil) + val kryoIntOrdering = kryo.deserialize[InterpretedOrdering](kryo.serialize(intOrdering)) + val kryoGenOrdering = kryo.deserialize[LazilyGeneratedOrdering](kryo.serialize(genOrdering)) + + Seq(intOrdering, genOrdering, kryoIntOrdering, kryoGenOrdering).foreach { ordering => assert(ordering.compare(rowA, rowA) === 0) assert(ordering.compare(rowB, rowB) === 0) assert(signum(ordering.compare(rowA, rowB)) === expectedCompareResult) From cd7bf4b8ea9e8c08d8d6aaac8eb574a0cb0a0663 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 2 Jun 2016 12:34:51 -0700 Subject: [PATCH 0493/1470] [SPARK-15728][SQL] Rename aggregate operators: HashAggregate and SortAggregate ## What changes were proposed in this pull request? We currently have two physical aggregate operators: TungstenAggregate and SortBasedAggregate. These names don't make a lot of sense from an end-user point of view. This patch renames them HashAggregate and SortAggregate. ## How was this patch tested? Updated test cases. Author: Reynold Xin Closes #13465 from rxin/SPARK-15728. (cherry picked from commit 8900c8d8ff1614b5ec5a2ce213832fa13462b4d4) Signed-off-by: Reynold Xin --- .../spark/sql/execution/SparkStrategies.scala | 8 ++++---- .../sql/execution/WholeStageCodegenExec.scala | 4 ++-- .../aggregate/{utils.scala => AggUtils.scala} | 12 ++++++------ ...stenAggregate.scala => HashAggregateExec.scala} | 13 ++++++++----- ...AggregateExec.scala => SortAggregateExec.scala} | 5 ++++- .../org/apache/spark/sql/DataFrameSuite.scala | 6 +++--- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../sql/execution/WholeStageCodegenSuite.scala | 8 ++++---- .../sql/execution/metric/SQLMetricsSuite.scala | 14 +++++--------- 9 files changed, 37 insertions(+), 35 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/{utils.scala => AggUtils.scala} (98%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/{TungstenAggregate.scala => HashAggregateExec.scala} (98%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/{SortBasedAggregateExec.scala => SortAggregateExec.scala} (98%) 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 5a069f2214503..0110663fb7aa9 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 @@ -233,7 +233,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case PhysicalAggregation( namedGroupingExpressions, aggregateExpressions, rewrittenResultExpressions, child) => - aggregate.Utils.planStreamingAggregation( + aggregate.AggUtils.planStreamingAggregation( namedGroupingExpressions, aggregateExpressions, rewrittenResultExpressions, @@ -266,20 +266,20 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { sys.error("Distinct columns cannot exist in Aggregate operator containing " + "aggregate functions which don't support partial aggregation.") } else { - aggregate.Utils.planAggregateWithoutPartial( + aggregate.AggUtils.planAggregateWithoutPartial( groupingExpressions, aggregateExpressions, resultExpressions, planLater(child)) } } else if (functionsWithDistinct.isEmpty) { - aggregate.Utils.planAggregateWithoutDistinct( + aggregate.AggUtils.planAggregateWithoutDistinct( groupingExpressions, aggregateExpressions, resultExpressions, planLater(child)) } else { - aggregate.Utils.planAggregateWithOneDistinct( + aggregate.AggUtils.planAggregateWithOneDistinct( groupingExpressions, functionsWithDistinct, functionsWithoutDistinct, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index cd9ba7c75b91d..d3e8d4e8e41a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf @@ -37,7 +37,7 @@ trait CodegenSupport extends SparkPlan { /** Prefix used in the current operator's variable names. */ private def variablePrefix: String = this match { - case _: TungstenAggregate => "agg" + case _: HashAggregateExec => "agg" case _: BroadcastHashJoinExec => "bhj" case _: SortMergeJoinExec => "smj" case _: RDDScanExec => "rdd" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index d617a048130e8..a9ec0c8709db7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.streaming.{StateStoreRestoreExec, StateSto /** * Utility functions used by the query planner to convert our plan to new aggregation code path. */ -object Utils { +object AggUtils { def planAggregateWithoutPartial( groupingExpressions: Seq[NamedExpression], @@ -35,7 +35,7 @@ object Utils { val completeAggregateExpressions = aggregateExpressions.map(_.copy(mode = Complete)) val completeAggregateAttributes = completeAggregateExpressions.map(_.resultAttribute) - SortBasedAggregateExec( + SortAggregateExec( requiredChildDistributionExpressions = Some(groupingExpressions), groupingExpressions = groupingExpressions, aggregateExpressions = completeAggregateExpressions, @@ -54,10 +54,10 @@ object Utils { initialInputBufferOffset: Int = 0, resultExpressions: Seq[NamedExpression] = Nil, child: SparkPlan): SparkPlan = { - val usesTungstenAggregate = TungstenAggregate.supportsAggregate( + val useHash = HashAggregateExec.supportsAggregate( aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)) - if (usesTungstenAggregate) { - TungstenAggregate( + if (useHash) { + HashAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, groupingExpressions = groupingExpressions, aggregateExpressions = aggregateExpressions, @@ -66,7 +66,7 @@ object Utils { resultExpressions = resultExpressions, child = child) } else { - SortBasedAggregateExec( + SortAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, groupingExpressions = groupingExpressions, aggregateExpressions = aggregateExpressions, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 091177959bedb..fad81b558cfcb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -30,7 +30,10 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.{DecimalType, StringType, StructType} import org.apache.spark.unsafe.KVIterator -case class TungstenAggregate( +/** + * Hash-based aggregate operator that can also fallback to sorting when data exceeds memory size. + */ +case class HashAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], @@ -44,7 +47,7 @@ case class TungstenAggregate( aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) } - require(TungstenAggregate.supportsAggregate(aggregateBufferAttributes)) + require(HashAggregateExec.supportsAggregate(aggregateBufferAttributes)) override lazy val allAttributes: Seq[Attribute] = child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ @@ -769,15 +772,15 @@ case class TungstenAggregate( val keyString = groupingExpressions.mkString("[", ",", "]") val functionString = allAggregateExpressions.mkString("[", ",", "]") val outputString = output.mkString("[", ",", "]") - s"Aggregate(key=$keyString, functions=$functionString, output=$outputString)" + s"HashAggregate(key=$keyString, functions=$functionString, output=$outputString)" case Some(fallbackStartsAt) => - s"AggregateWithControlledFallback $groupingExpressions " + + s"HashAggregateWithControlledFallback $groupingExpressions " + s"$allAggregateExpressions $resultExpressions fallbackStartsAt=$fallbackStartsAt" } } } -object TungstenAggregate { +object HashAggregateExec { def supportsAggregate(aggregateBufferAttributes: Seq[Attribute]): Boolean = { val aggregationBufferSchema = StructType.fromAttributes(aggregateBufferAttributes) UnsafeFixedWidthAggregationMap.supportsAggregationBufferSchema(aggregationBufferSchema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index af1fb4c604c81..9e48ff8d707bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -26,7 +26,10 @@ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistrib import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics -case class SortBasedAggregateExec( +/** + * Sort-based aggregate operator. + */ +case class SortAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], 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 0e18ade09cbe5..a02e48d849ebf 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 @@ -28,7 +28,7 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -1227,7 +1227,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { private def verifyNonExchangingAgg(df: DataFrame) = { var atFirstAgg: Boolean = false df.queryExecution.executedPlan.foreach { - case agg: TungstenAggregate => + case agg: HashAggregateExec => atFirstAgg = !atFirstAgg case _ => if (atFirstAgg) { @@ -1242,7 +1242,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { private def verifyExchangingAgg(df: DataFrame) = { var atFirstAgg: Boolean = false df.queryExecution.executedPlan.foreach { - case agg: TungstenAggregate => + case agg: HashAggregateExec => if (atFirstAgg) { fail("Should not have back to back Aggregates") } 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 49a0ba1f1149b..1a7f6ebbb2956 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 @@ -246,7 +246,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df = sql(sqlText) // First, check if we have GeneratedAggregate. val hasGeneratedAgg = df.queryExecution.sparkPlan - .collect { case _: aggregate.TungstenAggregate => true } + .collect { case _: aggregate.HashAggregateExec => true } .nonEmpty if (!hasGeneratedAgg) { fail( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index f86955e5a5bc4..68f0ee864f47f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions.{avg, broadcast, col, max} @@ -39,7 +39,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val plan = df.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) assert(df.collect() === Array(Row(9, 4.5))) } @@ -48,7 +48,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val plan = df.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) } @@ -110,7 +110,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val plan = ds.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[TungstenAggregate]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 7e9160febdec7..d3639d97355b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -133,25 +133,21 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { testSparkPlanMetrics(ds.toDF(), 1, Map.empty) } - test("TungstenAggregate metrics") { + test("Aggregate metrics") { // Assume the execution plan is // ... -> TungstenAggregate(nodeId = 2) -> Exchange(nodeId = 1) // -> TungstenAggregate(nodeId = 0) val df = testData2.groupBy().count() // 2 partitions testSparkPlanMetrics(df, 1, Map( - 2L -> ("TungstenAggregate", Map( - "number of output rows" -> 2L)), - 0L -> ("TungstenAggregate", Map( - "number of output rows" -> 1L))) + 2L -> ("HashAggregate", Map("number of output rows" -> 2L)), + 0L -> ("HashAggregate", Map("number of output rows" -> 1L))) ) // 2 partitions and each partition contains 2 keys val df2 = testData2.groupBy('a).count() testSparkPlanMetrics(df2, 1, Map( - 2L -> ("TungstenAggregate", Map( - "number of output rows" -> 4L)), - 0L -> ("TungstenAggregate", Map( - "number of output rows" -> 3L))) + 2L -> ("HashAggregate", Map("number of output rows" -> 4L)), + 0L -> ("HashAggregate", Map("number of output rows" -> 3L))) ) } From 32b025e946e5ffb83dffc071c5a3c872c7bd9e85 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 2 Jun 2016 13:22:43 -0700 Subject: [PATCH 0494/1470] [SPARK-15515][SQL] Error Handling in Running SQL Directly On Files #### What changes were proposed in this pull request? This PR is to address the following issues: - **ISSUE 1:** For ORC source format, we are reporting the strange error message when we did not enable Hive support: ```SQL SQL Example: select id from `org.apache.spark.sql.hive.orc`.`file_path` Error Message: Table or view not found: `org.apache.spark.sql.hive.orc`.`file_path` ``` Instead, we should issue the error message like: ``` Expected Error Message: The ORC data source must be used with Hive support enabled ``` - **ISSUE 2:** For the Avro format, we report the strange error message like: The example query is like ```SQL SQL Example: select id from `avro`.`file_path` select id from `com.databricks.spark.avro`.`file_path` Error Message: Table or view not found: `com.databricks.spark.avro`.`file_path` ``` The desired message should be like: ``` Expected Error Message: Failed to find data source: avro. Please use Spark package http://spark-packages.org/package/databricks/spark-avro" ``` - ~~**ISSUE 3:** Unable to detect incompatibility libraries for Spark 2.0 in Data Source Resolution. We report a strange error message:~~ **Update**: The latest code changes contains - For JDBC format, we added an extra checking in the rule `ResolveRelations` of `Analyzer`. Without the PR, Spark will return the error message like: `Option 'url' not specified`. Now, we are reporting `Unsupported data source type for direct query on files: jdbc` - Make data source format name case incensitive so that error handling behaves consistent with the normal cases. - Added the test cases for all the supported formats. #### How was this patch tested? Added test cases to cover all the above issues Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13283 from gatorsmile/runSQLAgainstFile. (cherry picked from commit 9aff6f3b1915523432b1921fdd30fa015ed5d670) Signed-off-by: Shixiong Zhu --- .../execution/datasources/DataSource.scala | 34 +++++------- .../sql/execution/datasources/rules.scala | 14 ++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 53 ++++++++++++++++--- .../sql/sources/DDLSourceLoadSuite.scala | 5 +- .../sql/sources/ResolvedDataSourceSuite.scala | 14 ++++- .../sql/hive/execution/SQLQuerySuite.scala | 48 ++++++++++++++++- 6 files changed, 134 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 93f1ad01bf9aa..5f17fdf9467db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -132,28 +132,20 @@ case class DataSource( // Found the data source using fully qualified path dataSource case Failure(error) => - if (error.isInstanceOf[ClassNotFoundException]) { - val className = error.getMessage - if (spark2RemovedClasses.contains(className)) { - throw new ClassNotFoundException(s"$className is removed in Spark 2.0. " + - "Please check if your library is compatible with Spark 2.0") - } - } - if (provider.startsWith("org.apache.spark.sql.hive.orc")) { - throw new ClassNotFoundException( - "The ORC data source must be used with Hive support enabled.", error) + if (provider.toLowerCase == "orc" || + provider.startsWith("org.apache.spark.sql.hive.orc")) { + throw new AnalysisException( + "The ORC data source must be used with Hive support enabled") + } else if (provider.toLowerCase == "avro" || + provider == "com.databricks.spark.avro") { + throw new AnalysisException( + s"Failed to find data source: ${provider.toLowerCase}. Please use Spark " + + "package http://spark-packages.org/package/databricks/spark-avro") } else { - if (provider == "avro" || provider == "com.databricks.spark.avro") { - throw new ClassNotFoundException( - s"Failed to find data source: $provider. Please use Spark package " + - "http://spark-packages.org/package/databricks/spark-avro", - error) - } else { - throw new ClassNotFoundException( - s"Failed to find data source: $provider. Please find packages at " + - "http://spark-packages.org", - error) - } + throw new ClassNotFoundException( + s"Failed to find data source: $provider. Please find packages at " + + "http://spark-packages.org", + error) } } } catch { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index b622f859413a6..9afd715016d88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources +import scala.util.control.NonFatal + import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.SessionCatalog @@ -28,7 +30,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} /** - * Try to replaces [[UnresolvedRelation]]s with [[ResolvedDataSource]]. + * Try to replaces [[UnresolvedRelation]]s with [[ResolveDataSource]]. */ private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { @@ -38,6 +40,16 @@ private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[Lo sparkSession, paths = u.tableIdentifier.table :: Nil, className = u.tableIdentifier.database.get) + + val notSupportDirectQuery = try { + !classOf[FileFormat].isAssignableFrom(dataSource.providingClass) + } catch { + case NonFatal(e) => false + } + if (notSupportDirectQuery) { + throw new AnalysisException("Unsupported data source type for direct query on files: " + + s"${u.tableIdentifier.database.get}") + } val plan = LogicalRelation(dataSource.resolveRelation()) u.alias.map(a => SubqueryAlias(u.alias.get, plan)).getOrElse(plan) } catch { 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 1a7f6ebbb2956..4fcd6bc0d9ec9 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 @@ -1838,20 +1838,61 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { df) }) - val e1 = intercept[AnalysisException] { + var e = intercept[AnalysisException] { sql("select * from in_valid_table") } - assert(e1.message.contains("Table or view not found")) + assert(e.message.contains("Table or view not found")) - val e2 = intercept[AnalysisException] { + e = intercept[AnalysisException] { sql("select * from no_db.no_table").show() } - assert(e2.message.contains("Table or view not found")) + assert(e.message.contains("Table or view not found")) - val e3 = intercept[AnalysisException] { + e = intercept[AnalysisException] { sql("select * from json.invalid_file") } - assert(e3.message.contains("Path does not exist")) + assert(e.message.contains("Path does not exist")) + + e = intercept[AnalysisException] { + sql(s"select id from `org.apache.spark.sql.hive.orc`.`file_path`") + } + assert(e.message.contains("The ORC data source must be used with Hive support enabled")) + + e = intercept[AnalysisException] { + sql(s"select id from `com.databricks.spark.avro`.`file_path`") + } + assert(e.message.contains("Failed to find data source: com.databricks.spark.avro. " + + "Please use Spark package http://spark-packages.org/package/databricks/spark-avro")) + + // data source type is case insensitive + e = intercept[AnalysisException] { + sql(s"select id from Avro.`file_path`") + } + assert(e.message.contains("Failed to find data source: avro. Please use Spark package " + + "http://spark-packages.org/package/databricks/spark-avro")) + + e = intercept[AnalysisException] { + sql(s"select id from avro.`file_path`") + } + assert(e.message.contains("Failed to find data source: avro. Please use Spark package " + + "http://spark-packages.org/package/databricks/spark-avro")) + + e = intercept[AnalysisException] { + sql(s"select id from `org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`") + } + assert(e.message.contains("Table or view not found: " + + "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`")) + + e = intercept[AnalysisException] { + sql(s"select id from `Jdbc`.`file_path`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: Jdbc")) + + e = intercept[AnalysisException] { + sql(s"select id from `org.apache.spark.sql.execution.datasources.jdbc`.`file_path`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: " + + "org.apache.spark.sql.execution.datasources.jdbc")) } test("SortMergeJoin returns wrong results when using UnsafeRows") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala index f07c33042a72e..85ba33e58a787 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -42,9 +42,10 @@ class DDLSourceLoadSuite extends DataSourceTest with SharedSQLContext { } test("should fail to load ORC without Hive Support") { - intercept[ClassNotFoundException] { + val e = intercept[AnalysisException] { spark.read.format("orc").load() } + assert(e.message.contains("The ORC data source must be used with Hive support enabled")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 320aaea1e4eeb..5ea1f32433699 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.datasources.DataSource class ResolvedDataSourceSuite extends SparkFunSuite { @@ -60,13 +61,22 @@ class ResolvedDataSourceSuite extends SparkFunSuite { classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) } + test("csv") { + assert( + getProvidingClass("csv") === + classOf[org.apache.spark.sql.execution.datasources.csv.CSVFileFormat]) + assert( + getProvidingClass("com.databricks.spark.csv") === + classOf[org.apache.spark.sql.execution.datasources.csv.CSVFileFormat]) + } + test("error message for unknown data sources") { - val error1 = intercept[ClassNotFoundException] { + val error1 = intercept[AnalysisException] { getProvidingClass("avro") } assert(error1.getMessage.contains("spark-packages")) - val error2 = intercept[ClassNotFoundException] { + val error2 = intercept[AnalysisException] { getProvidingClass("com.databricks.spark.avro") } assert(error2.getMessage.contains("spark-packages")) 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 b5691450ca2f0..24de223cf899b 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 @@ -1247,11 +1247,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("run sql directly on files") { + test("run sql directly on files - parquet") { val df = spark.range(100).toDF() withTempPath(f => { df.write.parquet(f.getCanonicalPath) - checkAnswer(sql(s"select id from parquet.`${f.getCanonicalPath}`"), + // data source type is case insensitive + checkAnswer(sql(s"select id from Parquet.`${f.getCanonicalPath}`"), df) checkAnswer(sql(s"select id from `org.apache.spark.sql.parquet`.`${f.getCanonicalPath}`"), df) @@ -1260,6 +1261,49 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { }) } + test("run sql directly on files - orc") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.orc(f.getCanonicalPath) + // data source type is case insensitive + checkAnswer(sql(s"select id from ORC.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select id from `org.apache.spark.sql.hive.orc`.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select a.id from orc.`${f.getCanonicalPath}` as a"), + df) + }) + } + + test("run sql directly on files - csv") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.csv(f.getCanonicalPath) + // data source type is case insensitive + checkAnswer(sql(s"select cast(_c0 as int) id from CSV.`${f.getCanonicalPath}`"), + df) + checkAnswer( + sql(s"select cast(_c0 as int) id from `com.databricks.spark.csv`.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select cast(a._c0 as int) id from csv.`${f.getCanonicalPath}` as a"), + df) + }) + } + + test("run sql directly on files - json") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.json(f.getCanonicalPath) + // data source type is case insensitive + checkAnswer(sql(s"select id from jsoN.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select id from `org.apache.spark.sql.json`.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select a.id from json.`${f.getCanonicalPath}` as a"), + df) + }) + } + test("SPARK-8976 Wrong Result for Rollup #1") { checkAnswer(sql( "SELECT count(*) AS cnt, key % 5, grouping_id() FROM src GROUP BY key%5 WITH ROLLUP"), From a55454eb6a1d1b785982bacc282753372a8107b9 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Thu, 2 Jun 2016 14:11:01 -0700 Subject: [PATCH 0495/1470] [SPARK-15711][SQL] Ban CREATE TEMPORARY TABLE USING AS SELECT ## What changes were proposed in this pull request? This PR bans syntax like `CREATE TEMPORARY TABLE USING AS SELECT` `CREATE TEMPORARY TABLE ... USING ... AS ...` is not properly implemented, the temporary data is not cleaned up when the session exits. Before a full fix, we probably should ban this syntax. This PR only impact syntax like `CREATE TEMPORARY TABLE ... USING ... AS ...`. Other syntax like `CREATE TEMPORARY TABLE .. USING ...` and `CREATE TABLE ... USING ...` are not impacted. ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13451 from clockfly/ban_create_temp_table_using_as. (cherry picked from commit d109a1beeef5bca1e683247e0a5db4ec841bf3ba) Signed-off-by: Andrew Or --- .../apache/spark/sql/DataFrameWriter.scala | 1 - .../spark/sql/execution/SparkSqlParser.scala | 9 +- .../spark/sql/execution/SparkStrategies.scala | 10 +- .../spark/sql/execution/datasources/ddl.scala | 32 --- .../sources/CreateTableAsSelectSuite.scala | 265 +++++++++--------- .../sql/hive/execution/SQLQuerySuite.scala | 46 --- 6 files changed, 142 insertions(+), 221 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 25678e938d846..50ae9667f4c8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -561,7 +561,6 @@ final class DataFrameWriter private[sql](df: DataFrame) { CreateTableUsingAsSelect( tableIdent, source, - temporary = false, partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), getBucketSpec, mode, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 01409c6a77c1e..8ffc55668ae90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -317,17 +317,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Get the backing query. val query = plan(ctx.query) + if (temp) { + throw operationNotAllowed("CREATE TEMPORARY TABLE ... USING ... AS query", ctx) + } + // Determine the storage mode. val mode = if (ifNotExists) { SaveMode.Ignore - } else if (temp) { - SaveMode.Overwrite } else { SaveMode.ErrorIfExists } CreateTableUsingAsSelect( - table, provider, temp, partitionColumnNames, bucketSpec, mode, options, query) + table, provider, partitionColumnNames, bucketSpec, mode, options, query) } else { val struct = Option(ctx.colTypeList()).map(createStructType) CreateTableUsing( @@ -960,7 +962,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { CreateTableUsingAsSelect( tableIdent = tableDesc.identifier, provider = conf.defaultDataSourceName, - temporary = false, partitionColumns = tableDesc.partitionColumnNames.toArray, bucketSpec = None, mode = mode, 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 0110663fb7aa9..2e69027a031a6 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 @@ -420,15 +420,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { throw new AnalysisException( "allowExisting should be set to false when creating a temporary table.") - case c: CreateTableUsingAsSelect if c.temporary && c.partitionColumns.nonEmpty => - sys.error("Cannot create temporary partitioned table.") - - case c: CreateTableUsingAsSelect if c.temporary => - val cmd = CreateTempTableUsingAsSelectCommand( - c.tableIdent, c.provider, Array.empty[String], c.mode, c.options, c.child) - ExecutedCommandExec(cmd) :: Nil - - case c: CreateTableUsingAsSelect if !c.temporary => + case c: CreateTableUsingAsSelect => val cmd = CreateDataSourceTableAsSelectCommand( c.tableIdent, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index edbccde214c61..bf272e3c0659d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -56,7 +56,6 @@ case class CreateTableUsing( case class CreateTableUsingAsSelect( tableIdent: TableIdentifier, provider: String, - temporary: Boolean, partitionColumns: Array[String], bucketSpec: Option[BucketSpec], mode: SaveMode, @@ -91,37 +90,6 @@ case class CreateTempTableUsing( } } -case class CreateTempTableUsingAsSelectCommand( - tableIdent: TableIdentifier, - provider: String, - partitionColumns: Array[String], - mode: SaveMode, - options: Map[String, String], - query: LogicalPlan) extends RunnableCommand { - - if (tableIdent.database.isDefined) { - throw new AnalysisException( - s"Temporary table '$tableIdent' should not have specified a database") - } - - override def run(sparkSession: SparkSession): Seq[Row] = { - val df = Dataset.ofRows(sparkSession, query) - val dataSource = DataSource( - sparkSession, - className = provider, - partitionColumns = partitionColumns, - bucketSpec = None, - options = options) - val result = dataSource.write(mode, df) - sparkSession.sessionState.catalog.createTempView( - tableIdent.table, - Dataset.ofRows(sparkSession, LogicalRelation(result)).logicalPlan, - overrideIfExists = true) - - Seq.empty[Row] - } -} - case class RefreshTable(tableIdent: TableIdentifier) extends RunnableCommand { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index cbddb0643b26d..f9a07dbdf0be0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.sql.sources -import java.io.{File, IOException} +import java.io.File import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.AnalysisException +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter { + protected override lazy val sql = spark.sql _ private var path: File = null @@ -40,172 +44,175 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with override def afterAll(): Unit = { try { spark.catalog.dropTempView("jt") + if (path.exists()) { + Utils.deleteRecursively(path) + } } finally { super.afterAll() } } - after { - Utils.deleteRecursively(path) + before { + if (path.exists()) { + Utils.deleteRecursively(path) + } } - test("CREATE TEMPORARY TABLE AS SELECT") { - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) - - checkAnswer( - sql("SELECT a, b FROM jsonTable"), - sql("SELECT a, b FROM jt").collect()) - - spark.catalog.dropTempView("jsonTable") + test("CREATE TABLE USING AS SELECT") { + withTable("jsonTable") { + sql( + s""" + |CREATE TABLE jsonTable + |USING json + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a, b FROM jt")) + } } - test("CREATE TEMPORARY TABLE AS SELECT based on the file without write permission") { + test("CREATE TABLE USING AS SELECT based on the file without write permission") { val childPath = new File(path.toString, "child") path.mkdir() - childPath.createNewFile() path.setWritable(false) - val e = intercept[IOException] { + val e = intercept[SparkException] { sql( s""" - |CREATE TEMPORARY TABLE jsonTable + |CREATE TABLE jsonTable |USING json |OPTIONS ( - | path '${path.toString}' + | path '${childPath.toString}' |) AS |SELECT a, b FROM jt - """.stripMargin) + """.stripMargin) sql("SELECT a, b FROM jsonTable").collect() } - assert(e.getMessage().contains("Unable to clear output directory")) + assert(e.getMessage().contains("Job aborted")) path.setWritable(true) } test("create a table, drop it and create another one with the same name") { - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) - - checkAnswer( - sql("SELECT a, b FROM jsonTable"), - sql("SELECT a, b FROM jt").collect()) - - val message = intercept[ParseException]{ + withTable("jsonTable") { sql( s""" - |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a * 4 FROM jt - """.stripMargin) - }.getMessage - assert(message.toLowerCase.contains("operation not allowed")) - - // Overwrite the temporary table. - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a * 4 FROM jt - """.stripMargin) - checkAnswer( - sql("SELECT * FROM jsonTable"), - sql("SELECT a * 4 FROM jt").collect()) - - spark.catalog.dropTempView("jsonTable") - // Explicitly delete the data. - if (path.exists()) Utils.deleteRecursively(path) - - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT b FROM jt - """.stripMargin) - - checkAnswer( - sql("SELECT * FROM jsonTable"), - sql("SELECT b FROM jt").collect()) - - spark.catalog.dropTempView("jsonTable") - } + |CREATE TABLE jsonTable + |USING json + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a, b FROM jt")) + + // Creates a table of the same name with flag "if not exists", nothing happens + sql( + s""" + |CREATE TABLE IF NOT EXISTS jsonTable + |USING json + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a * 4 FROM jt + """.stripMargin) + checkAnswer( + sql("SELECT * FROM jsonTable"), + sql("SELECT a, b FROM jt")) + + // Explicitly drops the table and deletes the underlying data. + sql("DROP TABLE jsonTable") + if (path.exists()) Utils.deleteRecursively(path) - test("CREATE TEMPORARY TABLE AS SELECT with IF NOT EXISTS is not allowed") { - val message = intercept[ParseException]{ + // Creates a table of the same name again, this time we succeed. sql( s""" - |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT b FROM jt - """.stripMargin) - }.getMessage - assert(message.toLowerCase.contains("operation not allowed")) + |CREATE TABLE jsonTable + |USING json + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM jsonTable"), + sql("SELECT b FROM jt")) + } + } + + test("disallows CREATE TEMPORARY TABLE ... USING ... AS query") { + withTable("t") { + val error = intercept[ParseException] { + sql( + s""" + |CREATE TEMPORARY TABLE t USING PARQUET + |OPTIONS (PATH '${path.toString}') + |PARTITIONED BY (a) + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + }.getMessage + assert(error.contains("Operation not allowed") && + error.contains("CREATE TEMPORARY TABLE ... USING ... AS query")) + } } - test("a CTAS statement with column definitions is not allowed") { - intercept[AnalysisException]{ + test("disallows CREATE EXTERNAL TABLE ... USING ... AS query") { + withTable("t") { + val error = intercept[ParseException] { + sql( + s""" + |CREATE EXTERNAL TABLE t USING PARQUET + |OPTIONS (PATH '${path.toString}') + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + }.getMessage + + assert(error.contains("Operation not allowed") && + error.contains("CREATE EXTERNAL TABLE ... USING")) + } + } + + test("create table using as select - with partitioned by") { + val catalog = spark.sessionState.catalog + withTable("t") { sql( s""" - |CREATE TEMPORARY TABLE jsonTable (a int, b string) - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) + |CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${path.toString}') + |PARTITIONED BY (a) + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + val table = catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.getPartitionColumnsFromTableProperties(table) == Seq("a")) } } - test("it is not allowed to write to a table while querying it.") { - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) - - val message = intercept[AnalysisException] { + test("create table using as select - with bucket") { + val catalog = spark.sessionState.catalog + withTable("t") { sql( s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jsonTable - """.stripMargin) - }.getMessage - assert( - message.contains("Cannot overwrite table "), - "Writing to a table while querying it should not be allowed.") + |CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${path.toString}') + |CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + val table = catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.getBucketSpecFromTableProperties(table) == + Some(BucketSpec(5, Seq("a"), Seq("b")))) + } } } 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 24de223cf899b..499819f32b434 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 @@ -1506,52 +1506,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test( - "SPARK-14488 \"CREATE TEMPORARY TABLE ... USING ... AS SELECT ...\" " + - "shouldn't create persisted table" - ) { - withTempPath { dir => - withTempTable("t1", "t2") { - val path = dir.getCanonicalPath - val ds = spark.range(10) - ds.createOrReplaceTempView("t1") - - sql( - s"""CREATE TEMPORARY TABLE t2 - |USING PARQUET - |OPTIONS (PATH '$path') - |AS SELECT * FROM t1 - """.stripMargin) - - checkAnswer( - spark.sql("SHOW TABLES").select('isTemporary).filter('tableName === "t2"), - Row(true) - ) - - checkAnswer(table("t2"), table("t1")) - } - } - } - - test( - "SPARK-14493 \"CREATE TEMPORARY TABLE ... USING ... AS SELECT ...\" " + - "shouldn always be used together with PATH data source option" - ) { - withTempTable("t") { - spark.range(10).createOrReplaceTempView("t") - - val message = intercept[IllegalArgumentException] { - sql( - s"""CREATE TEMPORARY TABLE t1 - |USING PARQUET - |AS SELECT * FROM t - """.stripMargin) - }.getMessage - - assert(message == "'path' is not specified") - } - } - test("derived from Hive query file: drop_database_removes_partition_dirs.q") { // This test verifies that if a partition exists outside a table's current location when the // database is dropped the partition's location is dropped as well. From 847ccf793a9ff0664a01b0bb988966fa51a0cb8d Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 2 Jun 2016 15:55:14 -0700 Subject: [PATCH 0496/1470] [SPARK-15092][SPARK-15139][PYSPARK][ML] Pyspark TreeEnsemble missing methods ## What changes were proposed in this pull request? Add `toDebugString` and `totalNumNodes` to `TreeEnsembleModels` and add `toDebugString` to `DecisionTreeModel` ## How was this patch tested? Extended doc tests. Author: Holden Karau Closes #12919 from holdenk/SPARK-15139-pyspark-treeEnsemble-missing-methods. (cherry picked from commit 72353311d3a37cb523c5bdd8072ffdff99af9749) Signed-off-by: Nick Pentreath --- python/pyspark/ml/classification.py | 20 ++++++++++++ python/pyspark/ml/regression.py | 48 ++++++++++++++++++++++++++++- 2 files changed, 67 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index ea660d7808fa4..177cf9d72cfb6 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -512,6 +512,8 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred 1 >>> model.featureImportances SparseVector(1, {0: 1.0}) + >>> print(model.toDebugString) + DecisionTreeClassificationModel (uid=...) of depth 1 with 3 nodes... >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> result = model.transform(test0).head() >>> result.prediction @@ -650,6 +652,8 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 + >>> model.trees + [DecisionTreeClassificationModel (uid=...) of depth..., DecisionTreeClassificationModel...] >>> rfc_path = temp_path + "/rfc" >>> rf.save(rfc_path) >>> rf2 = RandomForestClassifier.load(rfc_path) @@ -730,6 +734,12 @@ def featureImportances(self): """ return self._call_java("featureImportances") + @property + @since("2.0.0") + def trees(self): + """Trees in this ensemble. Warning: These have null parent Estimators.""" + return [DecisionTreeClassificationModel(m) for m in list(self._call_java("trees"))] + @inherit_doc class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, @@ -772,6 +782,10 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 1.0 + >>> model.totalNumNodes + 15 + >>> print(model.toDebugString) + GBTClassificationModel (uid=...)...with 5 trees... >>> gbtc_path = temp_path + "gbtc" >>> gbt.save(gbtc_path) >>> gbt2 = GBTClassifier.load(gbtc_path) @@ -869,6 +883,12 @@ def featureImportances(self): """ return self._call_java("featureImportances") + @property + @since("2.0.0") + def trees(self): + """Trees in this ensemble. Warning: These have null parent Estimators.""" + return [DecisionTreeRegressionModel(m) for m in list(self._call_java("trees"))] + @inherit_doc class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 1b7af7ef597c2..7c79ab73c7e76 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -593,7 +593,7 @@ class RandomForestParams(TreeEnsembleParams): featureSubsetStrategy = \ Param(Params._dummy(), "featureSubsetStrategy", "The number of features to consider for splits at each tree node. Supported " + - "options: " + ", ".join(supportedFeatureSubsetStrategies), + "options: " + ", ".join(supportedFeatureSubsetStrategies) + " (0.0-1.0], [1-n].", typeConverter=TypeConverters.toString) def __init__(self): @@ -744,6 +744,12 @@ def depth(self): """Return depth of the decision tree.""" return self._call_java("depth") + @property + @since("2.0.0") + def toDebugString(self): + """Full description of model.""" + return self._call_java("toDebugString") + def __repr__(self): return self._call_java("toString") @@ -758,12 +764,36 @@ class TreeEnsembleModels(JavaModel): .. versionadded:: 1.5.0 """ + @property + @since("2.0.0") + def trees(self): + """Trees in this ensemble. Warning: These have null parent Estimators.""" + return [DecisionTreeModel(m) for m in list(self._call_java("trees"))] + + @property + @since("2.0.0") + def getNumTrees(self): + """Number of trees in ensemble.""" + return self._call_java("getNumTrees") + @property @since("1.5.0") def treeWeights(self): """Return the weights for each tree""" return list(self._call_java("javaTreeWeights")) + @property + @since("2.0.0") + def totalNumNodes(self): + """Total number of nodes, summed over all trees in the ensemble.""" + return self._call_java("totalNumNodes") + + @property + @since("2.0.0") + def toDebugString(self): + """Full description of model.""" + return self._call_java("toDebugString") + def __repr__(self): return self._call_java("toString") @@ -825,6 +855,10 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 + >>> model.trees + [DecisionTreeRegressionModel (uid=...) of depth..., DecisionTreeRegressionModel...] + >>> model.getNumTrees + 2 >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> model.transform(test1).head().prediction 0.5 @@ -896,6 +930,12 @@ class RandomForestRegressionModel(TreeEnsembleModels, JavaMLWritable, JavaMLRead .. versionadded:: 1.4.0 """ + @property + @since("2.0.0") + def trees(self): + """Trees in this ensemble. Warning: These have null parent Estimators.""" + return [DecisionTreeRegressionModel(m) for m in list(self._call_java("trees"))] + @property @since("2.0.0") def featureImportances(self): @@ -1045,6 +1085,12 @@ def featureImportances(self): """ return self._call_java("featureImportances") + @property + @since("2.0.0") + def trees(self): + """Trees in this ensemble. Warning: These have null parent Estimators.""" + return [DecisionTreeRegressionModel(m) for m in list(self._call_java("trees"))] + @inherit_doc class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, From 810472a928811f078ef42825d1cf16cf711fcefb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 2 Jun 2016 16:16:27 -0700 Subject: [PATCH 0497/1470] [SPARK-15719][SQL] Disables writing Parquet summary files by default ## What changes were proposed in this pull request? This PR disables writing Parquet summary files by default (i.e., when Hadoop configuration "parquet.enable.summary-metadata" is not set). Please refer to [SPARK-15719][1] for more details. ## How was this patch tested? New test case added in `ParquetQuerySuite` to check no summary files are written by default. [1]: https://issues.apache.org/jira/browse/SPARK-15719 Author: Cheng Lian Closes #13455 from liancheng/spark-15719-disable-parquet-summary-files. (cherry picked from commit 431542765785304edb76a19885fbc5f9b8ae7d64) Signed-off-by: Cheng Lian --- .../parquet/ParquetFileFormat.scala | 7 ++++- .../datasources/parquet/ParquetIOSuite.scala | 20 ++++++------ .../parquet/ParquetQuerySuite.scala | 31 ++++++++++++++++--- .../parquet/ParquetSchemaSuite.scala | 19 ++---------- .../ParquetHadoopFsRelationSuite.scala | 29 +++++++++-------- 5 files changed, 62 insertions(+), 44 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index ff7962df22452..ada9cd4b8e812 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -124,6 +124,11 @@ private[sql] class ParquetFileFormat // Sets compression scheme conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + // SPARK-15719: Disables writing Parquet summary files by default. + if (conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) { + conf.setBoolean(ParquetOutputFormat.ENABLE_JOB_SUMMARY, false) + } + new OutputWriterFactory { override def newInstance( path: String, @@ -786,7 +791,7 @@ private[sql] object ParquetFileFormat extends Logging { // // Parquet requires `FileStatus`es to read footers. Here we try to send cached `FileStatus`es // to executor side to avoid fetching them again. However, `FileStatus` is not `Serializable` - // but only `Writable`. What makes it worth, for some reason, `FileStatus` doesn't play well + // but only `Writable`. What makes it worse, for some reason, `FileStatus` doesn't play well // with `SerializableWritable[T]` and always causes a weird `IllegalStateException`. These // facts virtually prevents us to serialize `FileStatus`es. // diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index d0107aae5aacc..92f2db325c1aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -515,17 +515,19 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { val hadoopConf = spark.sessionState.newHadoopConfWithOptions(extraOptions) - withTempPath { dir => - val path = s"${dir.getCanonicalPath}/part-r-0.parquet" - spark.range(1 << 16).selectExpr("(id % 4) AS i") - .coalesce(1).write.options(extraOptions).mode("overwrite").parquet(path) + withSQLConf(ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part-r-0.parquet" + spark.range(1 << 16).selectExpr("(id % 4) AS i") + .coalesce(1).write.options(extraOptions).mode("overwrite").parquet(path) - val blockMetadata = readFooter(new Path(path), hadoopConf).getBlocks.asScala.head - val columnChunkMetadata = blockMetadata.getColumns.asScala.head + val blockMetadata = readFooter(new Path(path), hadoopConf).getBlocks.asScala.head + val columnChunkMetadata = blockMetadata.getColumns.asScala.head - // If the file is written with version2, this should include - // Encoding.RLE_DICTIONARY type. For version1, it is Encoding.PLAIN_DICTIONARY - assert(columnChunkMetadata.getEncodings.contains(Encoding.RLE_DICTIONARY)) + // If the file is written with version2, this should include + // Encoding.RLE_DICTIONARY type. For version1, it is Encoding.PLAIN_DICTIONARY + assert(columnChunkMetadata.getEncodings.contains(Encoding.RLE_DICTIONARY)) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 0a2fb0ef50661..78b97f6995cdb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} @@ -148,13 +149,18 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } - withSQLConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "true") { + withSQLConf( + SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", + SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "true", + ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true" + ) { testSchemaMerging(2) } - withSQLConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "false") { + withSQLConf( + SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", + SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "false" + ) { testSchemaMerging(3) } } @@ -604,6 +610,21 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } } + + test("SPARK-15719: disable writing summary files by default") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(3).write.parquet(path) + + val fs = FileSystem.get(sparkContext.hadoopConfiguration) + val files = fs.listFiles(new Path(path), true) + + while (files.hasNext) { + val file = files.next + assert(!file.getPath.getName.contains("_metadata")) + } + } + } } object TestingUDT { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 6db649228210d..1bc6f70401bb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -451,31 +451,18 @@ class ParquetSchemaSuite extends ParquetSchemaTest { } test("schema merging failure error message") { - withTempPath { dir => - val path = dir.getCanonicalPath - spark.range(3).write.parquet(s"$path/p=1") - spark.range(3).selectExpr("CAST(id AS INT) AS id").write.parquet(s"$path/p=2") - - val message = intercept[SparkException] { - spark.read.option("mergeSchema", "true").parquet(path).schema - }.getMessage + import testImplicits._ - assert(message.contains("Failed merging schema of file")) - } - - // test for second merging (after read Parquet schema in parallel done) withTempPath { dir => val path = dir.getCanonicalPath spark.range(3).write.parquet(s"$path/p=1") - spark.range(3).selectExpr("CAST(id AS INT) AS id").write.parquet(s"$path/p=2") - - spark.sparkContext.conf.set("spark.default.parallelism", "20") + spark.range(3).select('id cast IntegerType as 'id).write.parquet(s"$path/p=2") val message = intercept[SparkException] { spark.read.option("mergeSchema", "true").parquet(path).schema }.getMessage - assert(message.contains("Failed merging schema:")) + assert(message.contains("Failed merging schema")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index f9a1d16d9094d..8aa018d0a9ee5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -21,6 +21,7 @@ import java.io.File import com.google.common.io.Files import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ @@ -124,23 +125,25 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { } test("SPARK-8604: Parquet data source should write summary file while doing appending") { - withTempPath { dir => - val path = dir.getCanonicalPath - val df = spark.range(0, 5).toDF() - df.write.mode(SaveMode.Overwrite).parquet(path) + withSQLConf(ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = spark.range(0, 5).toDF() + df.write.mode(SaveMode.Overwrite).parquet(path) - val summaryPath = new Path(path, "_metadata") - val commonSummaryPath = new Path(path, "_common_metadata") + val summaryPath = new Path(path, "_metadata") + val commonSummaryPath = new Path(path, "_common_metadata") - val fs = summaryPath.getFileSystem(spark.sessionState.newHadoopConf()) - fs.delete(summaryPath, true) - fs.delete(commonSummaryPath, true) + val fs = summaryPath.getFileSystem(spark.sessionState.newHadoopConf()) + fs.delete(summaryPath, true) + fs.delete(commonSummaryPath, true) - df.write.mode(SaveMode.Append).parquet(path) - checkAnswer(spark.read.parquet(path), df.union(df)) + df.write.mode(SaveMode.Append).parquet(path) + checkAnswer(spark.read.parquet(path), df.union(df)) - assert(fs.exists(summaryPath)) - assert(fs.exists(commonSummaryPath)) + assert(fs.exists(summaryPath)) + assert(fs.exists(commonSummaryPath)) + } } } From 1153a4098956dd4ba408ab6a1c0d4263de2ef0a7 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Thu, 2 Jun 2016 16:21:33 -0700 Subject: [PATCH 0498/1470] [SPARK-15734][SQL] Avoids printing internal row in explain output ## What changes were proposed in this pull request? This PR avoids printing internal rows in explain output for some operators. **Before change:** ``` scala> (1 to 10).toSeq.map(_ => (1,2,3)).toDF().createTempView("df3") scala> spark.sql("select * from df3 where 1=2").explain(true) ... == Analyzed Logical Plan == _1: int, _2: int, _3: int Project [_1#37,_2#38,_3#39] +- Filter (1 = 2) +- SubqueryAlias df3 +- LocalRelation [_1#37,_2#38,_3#39], [[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3]] ... == Physical Plan == LocalTableScan [_1#37,_2#38,_3#39] ``` **After change:** ``` scala> spark.sql("select * from df3 where 1=2").explain(true) ... == Analyzed Logical Plan == _1: int, _2: int, _3: int Project [_1#58,_2#59,_3#60] +- Filter (1 = 2) +- SubqueryAlias df3 +- LocalRelation [_1#58,_2#59,_3#60] ... == Physical Plan == LocalTableScan , [_1#58,_2#59,_3#60] ``` ## How was this patch tested? Manual test. Author: Sean Zhong Closes #13471 from clockfly/verbose_breakdown_5. (cherry picked from commit 985d532812cf176d0e12b799c723f917282b6813) Signed-off-by: Cheng Lian --- .../spark/sql/catalyst/plans/logical/LocalRelation.scala | 8 +++++++- .../org/apache/spark/sql/execution/ExistingRDD.scala | 2 ++ .../apache/spark/sql/execution/LocalTableScanExec.scala | 8 ++++++++ 3 files changed, 17 insertions(+), 1 deletion(-) 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 5813b74c770d8..87b8647655f4a 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 @@ -57,7 +57,13 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil) LocalRelation(output.map(_.newInstance()), data).asInstanceOf[this.type] } - override protected def stringArgs = Iterator(output) + override protected def stringArgs: Iterator[Any] = { + if (data.isEmpty) { + Iterator("", output) + } else { + Iterator(output) + } + } override def sameResult(plan: LogicalPlan): Boolean = plan match { case LocalRelation(otherOutput, otherData) => 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 fef3255c739ab..b8b392608de1b 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 @@ -91,6 +91,8 @@ private[sql] case class LogicalRDD( case _ => false } + override protected def stringArgs: Iterator[Any] = Iterator(output) + override def producedAttributes: AttributeSet = outputSet @transient override lazy val statistics: Statistics = Statistics( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index c5e78b033359d..df2f238d8c2e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -48,6 +48,14 @@ private[sql] case class LocalTableScanExec( } } + override protected def stringArgs: Iterator[Any] = { + if (rows.isEmpty) { + Iterator("", output) + } else { + Iterator(output) + } + } + override def executeCollect(): Array[InternalRow] = { unsafeRows } From 698b6f67c331cf7ea38b7214d2691e29661af60b Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 2 Jun 2016 16:33:16 -0700 Subject: [PATCH 0499/1470] [MINOR] clean up style for storage param setters in ALS Clean up style for param setter methods in ALS to match standard style and the other setter in class (this is an artefact of one of my previous PRs that wasn't cleaned up). ## How was this patch tested? Existing tests - no functionality change. Author: Nick Pentreath Closes #13480 from MLnick/als-param-minor-cleanup. (cherry picked from commit ccd298eb6794cbcb220ac9889db60d745231e0fe) Signed-off-by: Nick Pentreath --- .../scala/org/apache/spark/ml/recommendation/ALS.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) 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 8dc7437d4747c..2404a69e9e8b5 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 @@ -430,15 +430,11 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] /** @group expertSetParam */ @Since("2.0.0") - def setIntermediateStorageLevel(value: String): this.type = { - set(intermediateStorageLevel, value) - } + def setIntermediateStorageLevel(value: String): this.type = set(intermediateStorageLevel, value) /** @group expertSetParam */ @Since("2.0.0") - def setFinalStorageLevel(value: String): this.type = { - set(finalStorageLevel, value) - } + def setFinalStorageLevel(value: String): this.type = set(finalStorageLevel, value) /** * Sets both numUserBlocks and numItemBlocks to the specific value. From 0802ff9f64da48491131d1524cc9416d764eae24 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 2 Jun 2016 16:37:01 -0700 Subject: [PATCH 0500/1470] [SPARK-15668][ML] ml.feature: update check schema to avoid confusion when user use MLlib.vector as input type ## What changes were proposed in this pull request? ml.feature: update check schema to avoid confusion when user use MLlib.vector as input type ## How was this patch tested? existing ut Author: Yuhao Yang Closes #13411 from hhbyyh/schemaCheck. (cherry picked from commit 5855e0057defeab8006ca4f7b0196003bbc9e899) Signed-off-by: Nick Pentreath --- .../spark/ml/feature/MaxAbsScaler.scala | 4 +-- .../spark/ml/feature/MinMaxScaler.scala | 4 +-- .../org/apache/spark/ml/feature/PCA.scala | 28 ++++++++----------- .../spark/ml/feature/StandardScaler.scala | 25 ++++++++--------- 4 files changed, 25 insertions(+), 36 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index 1b5159902eeb5..7298a18ff83b5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -39,9 +39,7 @@ private[feature] trait MaxAbsScalerParams extends Params with HasInputCol with H /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) require(!schema.fieldNames.contains($(outputCol)), s"Output column ${$(outputCol)} already exists.") val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index d15f1b8563b24..a27bed5333c56 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -63,9 +63,7 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { require($(min) < $(max), s"The specified min(${$(min)}) is larger or equal to max(${$(max)})") - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) require(!schema.fieldNames.contains($(outputCol)), s"Output column ${$(outputCol)} already exists.") val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index dbbaa5aa46f49..2f667af9d10bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -49,8 +49,16 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC /** @group getParam */ def getK: Int = $(k) -} + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + require(!schema.fieldNames.contains($(outputCol)), + s"Output column ${$(outputCol)} already exists.") + val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) + StructType(outputFields) + } +} /** * :: Experimental :: * PCA trains a model to project vectors to a lower dimensional space of the top [[PCA!.k]] @@ -86,13 +94,7 @@ class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams } override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } override def copy(extra: ParamMap): PCA = defaultCopy(extra) @@ -148,13 +150,7 @@ class PCAModel private[ml] ( } override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } override def copy(extra: ParamMap): PCAModel = { @@ -201,7 +197,7 @@ object PCAModel extends MLReadable[PCAModel] { val versionRegex = "([0-9]+)\\.([0-9]+).*".r val hasExplainedVariance = metadata.sparkVersion match { case versionRegex(major, minor) => - (major.toInt >= 2 || (major.toInt == 1 && minor.toInt > 6)) + major.toInt >= 2 || (major.toInt == 1 && minor.toInt > 6) case _ => false } 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 9d084b520c488..7cec369c23a8f 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 @@ -62,6 +62,15 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with /** @group getParam */ def getWithStd: Boolean = $(withStd) + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + require(!schema.fieldNames.contains($(outputCol)), + s"Output column ${$(outputCol)} already exists.") + val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) + StructType(outputFields) + } + setDefault(withMean -> false, withStd -> true) } @@ -105,13 +114,7 @@ class StandardScaler(override val uid: String) extends Estimator[StandardScalerM } override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } override def copy(extra: ParamMap): StandardScaler = defaultCopy(extra) @@ -159,13 +162,7 @@ class StandardScalerModel private[ml] ( } override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } override def copy(extra: ParamMap): StandardScalerModel = { From 1551a72cb7217f6d8b1c30fddcc865a9df545cff Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 2 Jun 2016 17:36:31 -0700 Subject: [PATCH 0501/1470] [SPARK-15736][CORE] Gracefully handle loss of DiskStore files If an RDD partition is cached on disk and the DiskStore file is lost, then reads of that cached partition will fail and the missing partition is supposed to be recomputed by a new task attempt. In the current BlockManager implementation, however, the missing file does not trigger any metadata updates / does not invalidate the cache, so subsequent task attempts will be scheduled on the same executor and the doomed read will be repeatedly retried, leading to repeated task failures and eventually a total job failure. In order to fix this problem, the executor with the missing file needs to properly mark the corresponding block as missing so that it stops advertising itself as a cache location for that block. This patch fixes this bug and adds an end-to-end regression test (in `FailureSuite`) and a set of unit tests (`in BlockManagerSuite`). Author: Josh Rosen Closes #13473 from JoshRosen/handle-missing-cache-files. (cherry picked from commit 229f90225748343972d7202c5567b45364cd8497) Signed-off-by: Andrew Or --- .../apache/spark/storage/BlockManager.scala | 20 +++++++--- .../scala/org/apache/spark/FailureSuite.scala | 12 ++++++ .../spark/storage/BlockManagerSuite.scala | 40 +++++++++++++++++++ 3 files changed, 66 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 2f9473aedc2dc..83a9cbd63d391 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -402,6 +402,17 @@ private[spark] class BlockManager( locations } + /** + * Cleanup code run in response to a failed local read. + * Must be called while holding a read lock on the block. + */ + private def handleLocalReadFailure(blockId: BlockId): Nothing = { + releaseLock(blockId) + // Remove the missing block so that its unavailability is reported to the driver + removeBlock(blockId) + throw new SparkException(s"Block $blockId was not found even though it's read-locked") + } + /** * Get block from local block manager as an iterator of Java objects. */ @@ -441,8 +452,7 @@ private[spark] class BlockManager( val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, releaseLock(blockId)) Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) } else { - releaseLock(blockId) - throw new SparkException(s"Block $blockId was not found even though it's read-locked") + handleLocalReadFailure(blockId) } } } @@ -489,8 +499,7 @@ private[spark] class BlockManager( // The block was not found on disk, so serialize an in-memory copy: serializerManager.dataSerialize(blockId, memoryStore.getValues(blockId).get) } else { - releaseLock(blockId) - throw new SparkException(s"Block $blockId was not found even though it's read-locked") + handleLocalReadFailure(blockId) } } else { // storage level is serialized if (level.useMemory && memoryStore.contains(blockId)) { @@ -499,8 +508,7 @@ private[spark] class BlockManager( val diskBytes = diskStore.getBytes(blockId) maybeCacheDiskBytesInMemory(info, blockId, level, diskBytes).getOrElse(diskBytes) } else { - releaseLock(blockId) - throw new SparkException(s"Block $blockId was not found even though it's read-locked") + handleLocalReadFailure(blockId) } } } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 333c23bdaf6d6..132f6361e41e6 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import java.io.{IOException, NotSerializableException, ObjectInputStream} import org.apache.spark.memory.TestMemoryConsumer +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.NonSerializable // Common state shared by FailureSuite-launched tasks. We use a global object @@ -241,6 +242,17 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { FailureSuiteState.clear() } + test("failure because cached RDD partitions are missing from DiskStore (SPARK-15736)") { + sc = new SparkContext("local[1,2]", "test") + val rdd = sc.parallelize(1 to 2, 2).persist(StorageLevel.DISK_ONLY) + rdd.count() + // Directly delete all files from the disk store, triggering failures when reading cached data: + SparkEnv.get.blockManager.diskBlockManager.getAllFiles().foreach(_.delete()) + // Each task should fail once due to missing cached data, but then should succeed on its second + // attempt because the missing cache locations will be purged and the blocks will be recomputed. + rdd.count() + } + // TODO: Need to add tests with shuffle fetch failures. } 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 a2580304c4ed2..6821582254f5b 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1139,6 +1139,46 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.getSingle("a3").isDefined, "a3 was not in store") } + private def testReadWithLossOfOnDiskFiles( + storageLevel: StorageLevel, + readMethod: BlockManager => Option[_]): Unit = { + store = makeBlockManager(12000) + assert(store.putSingle("blockId", new Array[Byte](4000), storageLevel)) + assert(store.getStatus("blockId").isDefined) + // Directly delete all files from the disk store, triggering failures when reading blocks: + store.diskBlockManager.getAllFiles().foreach(_.delete()) + // The BlockManager still thinks that these blocks exist: + assert(store.getStatus("blockId").isDefined) + // Because the BlockManager's metadata claims that the block exists (i.e. that it's present + // in at least one store), the read attempts to read it and fails when the on-disk file is + // missing. + intercept[SparkException] { + readMethod(store) + } + // Subsequent read attempts will succeed; the block isn't present but we return an expected + // "block not found" response rather than a fatal error: + assert(readMethod(store).isEmpty) + // The reason why this second read succeeded is because the metadata entry for the missing + // block was removed as a result of the read failure: + assert(store.getStatus("blockId").isEmpty) + } + + test("remove block if a read fails due to missing DiskStore files (SPARK-15736)") { + val storageLevels = Seq( + StorageLevel(useDisk = true, useMemory = false, deserialized = false), + StorageLevel(useDisk = true, useMemory = false, deserialized = true)) + val readMethods = Map[String, BlockManager => Option[_]]( + "getLocalBytes" -> ((m: BlockManager) => m.getLocalBytes("blockId")), + "getLocalValues" -> ((m: BlockManager) => m.getLocalValues("blockId")) + ) + testReadWithLossOfOnDiskFiles(StorageLevel.DISK_ONLY, _.getLocalBytes("blockId")) + for ((readMethodName, readMethod) <- readMethods; storageLevel <- storageLevels) { + withClue(s"$readMethodName $storageLevel") { + testReadWithLossOfOnDiskFiles(storageLevel, readMethod) + } + } + } + test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { val mockBlockTransferService = new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) From 1bce96db5f366099d09d9083a21e1b34d15fae19 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 2 Jun 2016 17:39:56 -0700 Subject: [PATCH 0502/1470] [SPARK-15718][SQL] better error message for writing bucketed data ## What changes were proposed in this pull request? Currently we don't support bucketing for `save` and `insertInto`. For `save`, we just write the data out into a directory users specified, and it's not a table, we don't keep its metadata. When we read it back, we have no idea if the data is bucketed or not, so it doesn't make sense to use `save` to write bucketed data, as we can't use the bucket information anyway. We can support it in the future, once we have features like bucket discovery, or we save bucket information in the data directory too, so that we don't need to rely on a metastore. For `insertInto`, it inserts data into an existing table, so it doesn't make sense to specify bucket information, as we should get the bucket information from the existing table. This PR improves the error message for the above 2 cases. ## How was this patch tested? new test in `BukctedWriteSuite` Author: Wenchen Fan Closes #13452 from cloud-fan/error-msg. (cherry picked from commit f34aadc54ca1a9fd4236a928d342324b26fb3a12) Signed-off-by: Andrew Or --- .../apache/spark/sql/DataFrameWriter.scala | 10 +++++----- .../test/DataFrameReaderWriterSuite.scala | 4 ++-- .../sql/sources/BucketedWriteSuite.scala | 19 +++++++++++++++---- 3 files changed, 22 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 50ae9667f4c8f..1dd8818dedb2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -281,7 +281,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def save(): Unit = { - assertNotBucketed() + assertNotBucketed("save") assertNotStreaming("save() can only be called on non-continuous queries") val dataSource = DataSource( df.sparkSession, @@ -330,7 +330,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { */ @Experimental def startStream(): ContinuousQuery = { - assertNotBucketed() + assertNotBucketed("startStream") assertStreaming("startStream() can only be called on continuous queries") if (source == "memory") { @@ -430,7 +430,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { } private def insertInto(tableIdent: TableIdentifier): Unit = { - assertNotBucketed() + assertNotBucketed("insertInto") assertNotStreaming("insertInto() can only be called on non-continuous queries") val partitions = normalizedParCols.map(_.map(col => col -> (None: Option[String])).toMap) val overwrite = mode == SaveMode.Overwrite @@ -500,10 +500,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { s"existing columns (${validColumnNames.mkString(", ")})")) } - private def assertNotBucketed(): Unit = { + private def assertNotBucketed(operation: String): Unit = { if (numBuckets.isDefined || sortColumnNames.isDefined) { throw new IllegalArgumentException( - "Currently we don't support writing bucketed data to this data source.") + s"'$operation' does not support bucketing right now.") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index a2aac69064f9d..431a943304f5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -456,7 +456,7 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { .stream() val w = df.write val e = intercept[IllegalArgumentException](w.bucketBy(1, "text").startStream()) - assert(e.getMessage == "Currently we don't support writing bucketed data to this data source.") + assert(e.getMessage == "'startStream' does not support bucketing right now.") } test("check sortBy() can only be called on non-continuous queries;") { @@ -465,7 +465,7 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { .stream() val w = df.write val e = intercept[IllegalArgumentException](w.sortBy("text").startStream()) - assert(e.getMessage == "Currently we don't support writing bucketed data to this data source.") + assert(e.getMessage == "'startStream' does not support bucketing right now.") } test("check save(path) can only be called on non-continuous queries") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index ff44c6f29497d..61a281db85238 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -59,11 +59,22 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle intercept[SparkException](df.write.bucketBy(3, "i").format("text").saveAsTable("tt")) } - test("write bucketed data to non-hive-table or existing hive table") { + test("write bucketed data using save()") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") - intercept[IllegalArgumentException](df.write.bucketBy(2, "i").parquet("/tmp/path")) - intercept[IllegalArgumentException](df.write.bucketBy(2, "i").json("/tmp/path")) - intercept[IllegalArgumentException](df.write.bucketBy(2, "i").insertInto("tt")) + + val e = intercept[IllegalArgumentException] { + df.write.bucketBy(2, "i").parquet("/tmp/path") + } + assert(e.getMessage == "'save' does not support bucketing right now.") + } + + test("write bucketed data using insertInto()") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + + val e = intercept[IllegalArgumentException] { + df.write.bucketBy(2, "i").insertInto("tt") + } + assert(e.getMessage == "'insertInto' does not support bucketing right now.") } private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") From 0c721eedcc8a20092e3867c4eacb1341c4cb1831 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 2 Jun 2016 17:41:31 -0700 Subject: [PATCH 0503/1470] [SPARK-15740][MLLIB] ignore big model load / save in Word2VecSuite ## What changes were proposed in this pull request? andrewor14 noticed some OOM errors caused by "test big model load / save" in Word2VecSuite, e.g., https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test/job/spark-master-test-maven-hadoop-2.2/1168/consoleFull. It doesn't show up in the test result because it was OOMed. This PR disables the test. I will leave the JIRA open for a proper fix ## How was this patch tested? No new features. Author: Xiangrui Meng Closes #13478 from mengxr/SPARK-15740. (cherry picked from commit e23370ec617c527ffa3a1f7d285ee2c4ffc51b77) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/mllib/feature/Word2VecSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index 6d699440f2f2e..c9fb9768c1b45 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -91,7 +91,7 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { } - test("big model load / save") { + ignore("big model load / save") { // create a model bigger than 32MB since 9000 * 1000 * 4 > 2^25 val word2VecMap = Map((0 to 9000).map(i => s"$i" -> Array.fill(1000)(0.1f)): _*) val model = new Word2VecModel(word2VecMap) From d02f2926bfbbeb6593cc1deccfb0360ba5b4f0f0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 2 Jun 2016 17:44:48 -0700 Subject: [PATCH 0504/1470] [SPARK-15715][SQL] Fix alter partition with storage information in Hive ## What changes were proposed in this pull request? This command didn't work for Hive tables. Now it does: ``` ALTER TABLE boxes PARTITION (width=3) SET SERDE 'com.sparkbricks.serde.ColumnarSerDe' WITH SERDEPROPERTIES ('compress'='true') ``` ## How was this patch tested? `HiveExternalCatalogSuite` Author: Andrew Or Closes #13453 from andrewor14/alter-partition-storage. (cherry picked from commit d1c1fbc345a704a2c8210960683f33f945660d5a) Signed-off-by: Andrew Or --- .../catalog/ExternalCatalogSuite.scala | 10 +++++++ .../sql/hive/client/HiveClientImpl.scala | 30 +++++++++++++------ .../spark/sql/hive/client/VersionsSuite.scala | 5 +++- .../sql/hive/execution/HiveDDLSuite.scala | 22 ++++++++++++++ 4 files changed, 57 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 377e64ba015e3..0c4d363365025 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -382,6 +382,8 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac // See HIVE-2742 for more detail. catalog.setCurrentDatabase("db2") val newLocation = newUriForDatabase() + val newSerde = "com.sparkbricks.text.EasySerde" + val newSerdeProps = Map("spark" -> "bricks", "compressed" -> "false") // alter but keep spec the same val oldPart1 = catalog.getPartition("db2", "tbl2", part1.spec) val oldPart2 = catalog.getPartition("db2", "tbl2", part2.spec) @@ -394,6 +396,14 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac assert(newPart2.storage.locationUri == Some(newLocation)) assert(oldPart1.storage.locationUri != Some(newLocation)) assert(oldPart2.storage.locationUri != Some(newLocation)) + // alter other storage information + catalog.alterPartitions("db2", "tbl2", Seq( + oldPart1.copy(storage = storageFormat.copy(serde = Some(newSerde))), + oldPart2.copy(storage = storageFormat.copy(serdeProperties = newSerdeProps)))) + val newPart1b = catalog.getPartition("db2", "tbl2", part1.spec) + val newPart2b = catalog.getPartition("db2", "tbl2", part2.spec) + assert(newPart1b.storage.serde == Some(newSerde)) + assert(newPart2b.storage.serdeProperties == newSerdeProps) // alter but change spec, should fail because new partition specs do not exist yet val badPart1 = part1.copy(spec = Map("a" -> "v1", "b" -> "v2")) val badPart2 = part2.copy(spec = Map("a" -> "v3", "b" -> "v4")) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 47fa41823cd09..1c89d8c62a3ad 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} +import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.processors._ @@ -779,18 +780,29 @@ private[hive] class HiveClientImpl( hiveTable } - private def toHiveViewTable(view: CatalogTable): HiveTable = { - val tbl = toHiveTable(view) - tbl.setTableType(HiveTableType.VIRTUAL_VIEW) - tbl.setSerializationLib(null) - tbl.clearSerDeInfo() - tbl - } - private def toHivePartition( p: CatalogTablePartition, ht: HiveTable): HivePartition = { - new HivePartition(ht, p.spec.asJava, p.storage.locationUri.map { l => new Path(l) }.orNull) + val tpart = new org.apache.hadoop.hive.metastore.api.Partition + val partValues = ht.getPartCols.asScala.map { hc => + p.spec.get(hc.getName).getOrElse { + throw new IllegalArgumentException( + s"Partition spec is missing a value for column '${hc.getName}': ${p.spec}") + } + } + val storageDesc = new StorageDescriptor + val serdeInfo = new SerDeInfo + p.storage.locationUri.foreach(storageDesc.setLocation) + p.storage.inputFormat.foreach(storageDesc.setInputFormat) + p.storage.outputFormat.foreach(storageDesc.setOutputFormat) + p.storage.serde.foreach(serdeInfo.setSerializationLib) + serdeInfo.setParameters(p.storage.serdeProperties.asJava) + storageDesc.setSerdeInfo(serdeInfo) + tpart.setDbName(ht.getDbName) + tpart.setTableName(ht.getTableName) + tpart.setValues(partValues.asJava) + tpart.setSd(storageDesc) + new HivePartition(ht, tpart) } private def fromHivePartition(hp: HivePartition): CatalogTablePartition = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 8ae4535f4ebf4..5b209acf0f212 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -354,7 +354,10 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: alterPartitions") { val spec = Map("key1" -> "1", "key2" -> "2") val newLocation = Utils.createTempDir().getPath() - val storage = storageFormat.copy(locationUri = Some(newLocation)) + val storage = storageFormat.copy( + locationUri = Some(newLocation), + // needed for 0.12 alter partitions + serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) val partition = CatalogTablePartition(spec, storage) client.alterPartitions("default", "src_part", Seq(partition)) assert(client.getPartition("default", "src_part", spec) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index a98d469880736..b2f01fcc83280 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -345,6 +345,28 @@ class HiveDDLSuite } } + test("alter table partition - storage information") { + sql("CREATE TABLE boxes (height INT, length INT) PARTITIONED BY (width INT)") + sql("INSERT OVERWRITE TABLE boxes PARTITION (width=4) SELECT 4, 4") + val catalog = spark.sessionState.catalog + val expectedSerde = "com.sparkbricks.serde.ColumnarSerDe" + val expectedSerdeProps = Map("compress" -> "true") + val expectedSerdePropsString = + expectedSerdeProps.map { case (k, v) => s"'$k'='$v'" }.mkString(", ") + val oldPart = catalog.getPartition(TableIdentifier("boxes"), Map("width" -> "4")) + assume(oldPart.storage.serde != Some(expectedSerde), "bad test: serde was already set") + assume(oldPart.storage.serdeProperties.filterKeys(expectedSerdeProps.contains) != + expectedSerdeProps, "bad test: serde properties were already set") + sql(s"""ALTER TABLE boxes PARTITION (width=4) + | SET SERDE '$expectedSerde' + | WITH SERDEPROPERTIES ($expectedSerdePropsString) + |""".stripMargin) + val newPart = catalog.getPartition(TableIdentifier("boxes"), Map("width" -> "4")) + assert(newPart.storage.serde == Some(expectedSerde)) + assume(newPart.storage.serdeProperties.filterKeys(expectedSerdeProps.contains) == + expectedSerdeProps) + } + test("drop table using drop view") { withTable("tab1") { sql("CREATE TABLE tab1(c1 int)") From 180dd53b77a042235070e27b732c8f18165894e3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 2 Jun 2016 18:13:04 -0700 Subject: [PATCH 0505/1470] [SPARK-15732][SQL] better error message when use java reserved keyword as field name ## What changes were proposed in this pull request? When users create a case class and use java reserved keyword as field name, spark sql will generate illegal java code and throw exception at runtime. This PR checks the field names when building the encoder, and if illegal field names are used, throw exception immediately with a good error message. ## How was this patch tested? new test in DatasetSuite Author: Wenchen Fan Closes #13485 from cloud-fan/java. (cherry picked from commit 6323e4bd763eafe23c619a89552c00e1244e4690) Signed-off-by: Cheng Lian --- .../apache/spark/sql/catalyst/ScalaReflection.scala | 11 +++++++++++ .../scala/org/apache/spark/sql/DatasetSuite.scala | 10 ++++++++++ 2 files changed, 21 insertions(+) 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 052cc486e8cb2..47508618178ea 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 @@ -582,6 +582,11 @@ object ScalaReflection extends ScalaReflection { case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => + if (javaKeywords.contains(fieldName)) { + throw new UnsupportedOperationException(s"`$fieldName` is a reserved keyword and " + + "cannot be used as field name\n" + walkedTypePath.mkString("\n")) + } + val fieldValue = Invoke(inputObject, fieldName, dataTypeFor(fieldType)) val clsName = getClassNameFromType(fieldType) val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath @@ -720,6 +725,12 @@ object ScalaReflection extends ScalaReflection { tpe <:< localTypeOf[Product] || tpe <:< localTypeOf[DefinedByConstructorParams] } + private val javaKeywords = Set("abstract", "assert", "boolean", "break", "byte", "case", "catch", + "char", "class", "const", "continue", "default", "do", "double", "else", "extends", "false", + "final", "finally", "float", "for", "goto", "if", "implements", "import", "instanceof", "int", + "interface", "long", "native", "new", "null", "package", "private", "protected", "public", + "return", "short", "static", "strictfp", "super", "switch", "synchronized", "this", "throw", + "throws", "transient", "true", "try", "void", "volatile", "while") } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index a3881ff920159..df8f4b0610af2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -786,6 +786,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val result = joined.collect().toSet assert(result == Set(ClassData("a", 1) -> null, ClassData("b", 2) -> ClassData("x", 2))) } + + test("better error message when use java reserved keyword as field name") { + val e = intercept[UnsupportedOperationException] { + Seq(InvalidInJava(1)).toDS() + } + assert(e.getMessage.contains( + "`abstract` is a reserved keyword and cannot be used as field name")) + } } case class Generic[T](id: T, value: Double) @@ -809,6 +817,8 @@ case class ClassNullableData(a: String, b: Integer) case class NestedStruct(f: ClassData) case class DeepNestedStruct(f: NestedStruct) +case class InvalidInJava(`abstract`: Int) + /** * A class used to test serialization using encoders. This class throws exceptions when using * Java serialization -- so the only way it can be "serialized" is through our encoders. From 8be6e8cc51c0d7bdf6575e2b07714bdea9b2ce3c Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 2 Jun 2016 19:42:05 -0700 Subject: [PATCH 0506/1470] [SPARK-15724] Add benchmarks for performance over wide schemas ## What changes were proposed in this pull request? This adds microbenchmarks for tracking performance of queries over very wide or deeply nested DataFrames. It seems performance degrades when DataFrames get thousands of columns wide or hundreds of fields deep. ## How was this patch tested? Current results included. cc rxin JoshRosen Author: Eric Liang Closes #13456 from ericl/sc-3468. (cherry picked from commit 901b2e69eaf004fedfed27818072c5e70ebfaede) Signed-off-by: Josh Rosen --- .../benchmark/WideSchemaBenchmark.scala | 376 ++++++++++++++++++ 1 file changed, 376 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala new file mode 100644 index 0000000000000..b4811fe27a513 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.io.File + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.functions._ +import org.apache.spark.util.{Benchmark, Utils} + +/** + * Benchmark for performance with very wide and nested DataFrames. + * To run this: + * build/sbt "sql/test-only *WideSchemaBenchmark" + */ +class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { + private val scaleFactor = 100000 + private val widthsToTest = Seq(1, 10, 100, 1000, 2500) + private val depthsToTest = Seq(1, 10, 100, 250) + assert(scaleFactor > widthsToTest.max) + + private lazy val sparkSession = SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .getOrCreate() + + import sparkSession.implicits._ + + private var tmpFiles: List[File] = Nil + + override def afterEach() { + try { + for (tmpFile <- tmpFiles) { + Utils.deleteRecursively(tmpFile) + } + } finally { + tmpFiles = Nil + super.afterEach() + } + } + + /** + * Writes the given DataFrame to parquet at a temporary location, and returns a DataFrame + * backed by the written parquet files. + */ + private def saveAsParquet(df: DataFrame): DataFrame = { + val tmpFile = File.createTempFile("WideSchemaBenchmark", "tmp") + tmpFiles ::= tmpFile + tmpFile.delete() + df.write.parquet(tmpFile.getAbsolutePath) + assert(tmpFile.isDirectory()) + sparkSession.read.parquet(tmpFile.getAbsolutePath) + } + + /** + * Adds standard set of cases to a benchmark given a dataframe and field to select. + */ + private def addCases( + benchmark: Benchmark, + df: DataFrame, + desc: String, + selector: String): Unit = { + benchmark.addCase(desc + " (read in-mem)") { iter => + df.selectExpr(s"sum($selector)").collect() + } + benchmark.addCase(desc + " (write in-mem)") { iter => + df.selectExpr("*", s"hash($selector) as f").selectExpr(s"sum($selector)", "sum(f)").collect() + } + val parquet = saveAsParquet(df) + benchmark.addCase(desc + " (read parquet)") { iter => + parquet.selectExpr(s"sum($selector) as f").collect() + } + benchmark.addCase(desc + " (write parquet)") { iter => + saveAsParquet(df.selectExpr(s"sum($selector) as f")) + } + } + + ignore("parsing large select expressions") { + val benchmark = new Benchmark("parsing large select", 1) + for (width <- widthsToTest) { + val selectExpr = (1 to width).map(i => s"id as a_$i") + benchmark.addCase(s"$width select expressions") { iter => + sparkSession.range(1).toDF.selectExpr(selectExpr: _*) + } + } + benchmark.run() + +/* +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 select expressions 22 / 25 0.0 22053737.0 1.0X +10 select expressions 8 / 13 0.0 8288520.0 2.7X +100 select expressions 29 / 32 0.0 29481040.0 0.7X +1000 select expressions 268 / 276 0.0 268183159.0 0.1X +2500 select expressions 683 / 691 0.0 683422241.0 0.0X +*/ + } + + ignore("many column field read and write") { + val benchmark = new Benchmark("many column field r/w", scaleFactor) + for (width <- widthsToTest) { + // normalize by width to keep constant data size + val numRows = scaleFactor / width + val selectExpr = (1 to width).map(i => s"id as a_$i") + val df = sparkSession.range(numRows).toDF.selectExpr(selectExpr: _*).cache() + df.count() // force caching + addCases(benchmark, df, s"$width cols x $numRows rows", "a_1") + } + benchmark.run() + +/* +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 cols x 100000 rows (read in-mem) 26 / 33 3.8 262.9 1.0X +1 cols x 100000 rows (write in-mem) 40 / 51 2.5 401.6 0.7X +1 cols x 100000 rows (read parquet) 37 / 57 2.7 374.3 0.7X +1 cols x 100000 rows (write parquet) 105 / 157 0.9 1054.9 0.2X +10 cols x 10000 rows (read in-mem) 26 / 39 3.8 260.5 1.0X +10 cols x 10000 rows (write in-mem) 37 / 44 2.7 367.4 0.7X +10 cols x 10000 rows (read parquet) 31 / 39 3.3 305.1 0.9X +10 cols x 10000 rows (write parquet) 86 / 137 1.2 860.2 0.3X +100 cols x 1000 rows (read in-mem) 40 / 64 2.5 401.2 0.7X +100 cols x 1000 rows (write in-mem) 112 / 139 0.9 1118.3 0.2X +100 cols x 1000 rows (read parquet) 35 / 52 2.9 349.8 0.8X +100 cols x 1000 rows (write parquet) 150 / 182 0.7 1497.1 0.2X +1000 cols x 100 rows (read in-mem) 304 / 362 0.3 3043.6 0.1X +1000 cols x 100 rows (write in-mem) 647 / 729 0.2 6467.8 0.0X +1000 cols x 100 rows (read parquet) 194 / 235 0.5 1937.7 0.1X +1000 cols x 100 rows (write parquet) 511 / 521 0.2 5105.0 0.1X +2500 cols x 40 rows (read in-mem) 915 / 924 0.1 9148.2 0.0X +2500 cols x 40 rows (write in-mem) 1856 / 1933 0.1 18558.1 0.0X +2500 cols x 40 rows (read parquet) 802 / 881 0.1 8019.3 0.0X +2500 cols x 40 rows (write parquet) 1268 / 1291 0.1 12681.6 0.0X +*/ + } + + ignore("wide struct field read and write") { + val benchmark = new Benchmark("wide struct field r/w", scaleFactor) + for (width <- widthsToTest) { + val numRows = scaleFactor / width + var datum: String = "{" + for (i <- 1 to width) { + if (i == 1) { + datum += s""""value_$i": 1""" + } else { + datum += s""", "value_$i": 1""" + } + } + datum += "}" + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum).rdd).cache() + df.count() // force caching + addCases(benchmark, df, s"$width wide x $numRows rows", "value_1") + } + benchmark.run() + +/* +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +wide struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 22 / 37 4.6 216.8 1.0X +1 wide x 100000 rows (write in-mem) 37 / 54 2.7 365.6 0.6X +1 wide x 100000 rows (read parquet) 27 / 44 3.6 274.7 0.8X +1 wide x 100000 rows (write parquet) 155 / 183 0.6 1546.3 0.1X +10 wide x 10000 rows (read in-mem) 27 / 40 3.7 272.1 0.8X +10 wide x 10000 rows (write in-mem) 32 / 44 3.2 315.7 0.7X +10 wide x 10000 rows (read parquet) 31 / 44 3.2 309.8 0.7X +10 wide x 10000 rows (write parquet) 151 / 169 0.7 1509.3 0.1X +100 wide x 1000 rows (read in-mem) 37 / 62 2.7 374.4 0.6X +100 wide x 1000 rows (write in-mem) 81 / 96 1.2 805.6 0.3X +100 wide x 1000 rows (read parquet) 31 / 44 3.3 307.3 0.7X +100 wide x 1000 rows (write parquet) 174 / 209 0.6 1745.0 0.1X +1000 wide x 100 rows (read in-mem) 308 / 339 0.3 3082.4 0.1X +1000 wide x 100 rows (write in-mem) 672 / 696 0.1 6717.7 0.0X +1000 wide x 100 rows (read parquet) 182 / 228 0.5 1821.2 0.1X +1000 wide x 100 rows (write parquet) 484 / 497 0.2 4841.2 0.0X +2500 wide x 40 rows (read in-mem) 727 / 786 0.1 7268.4 0.0X +2500 wide x 40 rows (write in-mem) 1734 / 1782 0.1 17341.5 0.0X +2500 wide x 40 rows (read parquet) 882 / 935 0.1 8816.8 0.0X +2500 wide x 40 rows (write parquet) 935 / 982 0.1 9351.9 0.0X +*/ + } + + ignore("deeply nested struct field read and write") { + val benchmark = new Benchmark("deeply nested struct field r/w", scaleFactor) + for (depth <- depthsToTest) { + val numRows = scaleFactor / depth + var datum: String = "{\"value\": 1}" + var selector: String = "value" + for (i <- 1 to depth) { + datum = "{\"value\": " + datum + "}" + selector = selector + ".value" + } + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum).rdd).cache() + df.count() // force caching + addCases(benchmark, df, s"$depth deep x $numRows rows", selector) + } + benchmark.run() + +/* +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 deep x 100000 rows (read in-mem) 24 / 39 4.2 239.0 1.0X +1 deep x 100000 rows (write in-mem) 34 / 47 3.0 335.1 0.7X +1 deep x 100000 rows (read parquet) 45 / 51 2.2 446.1 0.5X +1 deep x 100000 rows (write parquet) 86 / 108 1.2 859.4 0.3X +10 deep x 10000 rows (read in-mem) 28 / 38 3.6 275.1 0.9X +10 deep x 10000 rows (write in-mem) 43 / 64 2.3 427.1 0.6X +10 deep x 10000 rows (read parquet) 44 / 59 2.3 438.1 0.5X +10 deep x 10000 rows (write parquet) 85 / 110 1.2 853.6 0.3X +100 deep x 1000 rows (read in-mem) 79 / 100 1.3 785.5 0.3X +100 deep x 1000 rows (write in-mem) 776 / 800 0.1 7760.3 0.0X +100 deep x 1000 rows (read parquet) 3302 / 3394 0.0 33021.2 0.0X +100 deep x 1000 rows (write parquet) 226 / 243 0.4 2259.0 0.1X +250 deep x 400 rows (read in-mem) 610 / 639 0.2 6104.0 0.0X +250 deep x 400 rows (write in-mem) 8526 / 8531 0.0 85256.9 0.0X +250 deep x 400 rows (read parquet) 54968 / 55069 0.0 549681.4 0.0X +250 deep x 400 rows (write parquet) 714 / 718 0.1 7143.0 0.0X +*/ + } + + ignore("bushy struct field read and write") { + val benchmark = new Benchmark("bushy struct field r/w", scaleFactor) + for (width <- Seq(1, 10, 100, 500)) { + val numRows = scaleFactor / width + var numNodes = 1 + var datum: String = "{\"value\": 1}" + var selector: String = "value" + var depth = 1 + while (numNodes < width) { + numNodes *= 2 + datum = s"""{"left_$depth": $datum, "right_$depth": $datum}""" + selector = s"left_$depth." + selector + depth += 1 + } + // TODO(ekl) seems like the json parsing is actually the majority of the time, perhaps + // we should benchmark that too separately. + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum).rdd).cache() + df.count() // force caching + addCases(benchmark, df, s"$numNodes x $depth deep x $numRows rows", selector) + } + benchmark.run() + +/* +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 x 1 deep x 100000 rows (read in-mem) 21 / 27 4.7 212.6 1.0X +1 x 1 deep x 100000 rows (write in-mem) 27 / 38 3.8 265.8 0.8X +1 x 1 deep x 100000 rows (read parquet) 26 / 32 3.9 259.1 0.8X +1 x 1 deep x 100000 rows (write parquet) 150 / 169 0.7 1499.5 0.1X +16 x 5 deep x 10000 rows (read in-mem) 26 / 45 3.9 258.7 0.8X +16 x 5 deep x 10000 rows (write in-mem) 54 / 58 1.9 535.1 0.4X +16 x 5 deep x 10000 rows (read parquet) 60 / 84 1.7 595.8 0.4X +16 x 5 deep x 10000 rows (write parquet) 179 / 184 0.6 1787.5 0.1X +128 x 8 deep x 1000 rows (read in-mem) 26 / 40 3.8 261.4 0.8X +128 x 8 deep x 1000 rows (write in-mem) 592 / 592 0.2 5915.3 0.0X +128 x 8 deep x 1000 rows (read parquet) 203 / 251 0.5 2031.8 0.1X +128 x 8 deep x 1000 rows (write parquet) 105 / 131 1.0 1045.2 0.2X +512 x 10 deep x 200 rows (read in-mem) 101 / 125 1.0 1007.4 0.2X +512 x 10 deep x 200 rows (write in-mem) 6778 / 6943 0.0 67781.1 0.0X +512 x 10 deep x 200 rows (read parquet) 958 / 1071 0.1 9584.9 0.0X +512 x 10 deep x 200 rows (write parquet) 173 / 207 0.6 1726.1 0.1X +*/ + } + + ignore("wide array field read and write") { + val benchmark = new Benchmark("wide array field r/w", scaleFactor) + for (width <- widthsToTest) { + val numRows = scaleFactor / width + var datum: String = "{\"value\": [" + for (i <- 1 to width) { + if (i == 1) { + datum += "1" + } else { + datum += ", 1" + } + } + datum += "]}" + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum).rdd).cache() + df.count() // force caching + addCases(benchmark, df, s"$width wide x $numRows rows", "value[0]") + } + benchmark.run() + +/* +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 27 / 45 3.7 268.0 1.0X +1 wide x 100000 rows (write in-mem) 37 / 52 2.7 368.3 0.7X +1 wide x 100000 rows (read parquet) 52 / 65 1.9 524.9 0.5X +1 wide x 100000 rows (write parquet) 102 / 139 1.0 1016.7 0.3X +10 wide x 10000 rows (read in-mem) 20 / 26 5.0 201.7 1.3X +10 wide x 10000 rows (write in-mem) 26 / 35 3.8 259.8 1.0X +10 wide x 10000 rows (read parquet) 39 / 59 2.5 393.8 0.7X +10 wide x 10000 rows (write parquet) 120 / 143 0.8 1201.4 0.2X +100 wide x 1000 rows (read in-mem) 24 / 31 4.2 240.1 1.1X +100 wide x 1000 rows (write in-mem) 26 / 35 3.8 264.1 1.0X +100 wide x 1000 rows (read parquet) 30 / 47 3.4 296.8 0.9X +100 wide x 1000 rows (write parquet) 109 / 147 0.9 1094.8 0.2X +1000 wide x 100 rows (read in-mem) 20 / 38 5.0 200.6 1.3X +1000 wide x 100 rows (write in-mem) 24 / 32 4.1 242.3 1.1X +1000 wide x 100 rows (read parquet) 47 / 55 2.1 470.1 0.6X +1000 wide x 100 rows (write parquet) 146 / 164 0.7 1465.0 0.2X +2500 wide x 40 rows (read in-mem) 20 / 28 5.1 196.1 1.4X +2500 wide x 40 rows (write in-mem) 25 / 27 4.0 249.3 1.1X +2500 wide x 40 rows (read parquet) 33 / 48 3.0 332.0 0.8X +2500 wide x 40 rows (write parquet) 149 / 176 0.7 1489.3 0.2X +*/ + } + + ignore("wide map field read and write") { + val benchmark = new Benchmark("wide map field r/w", scaleFactor) + for (width <- widthsToTest) { + val numRows = scaleFactor / width + val datum = Tuple1((1 to width).map(i => ("value_" + i -> 1)).toMap) + val df = sparkSession.range(numRows).map(_ => datum).toDF.cache() + df.count() // force caching + addCases(benchmark, df, s"$width wide x $numRows rows", "_1[\"value_1\"]") + } + benchmark.run() + +/* +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +wide map field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 27 / 42 3.7 270.9 1.0X +1 wide x 100000 rows (write in-mem) 40 / 63 2.5 403.4 0.7X +1 wide x 100000 rows (read parquet) 71 / 114 1.4 705.8 0.4X +1 wide x 100000 rows (write parquet) 169 / 184 0.6 1689.7 0.2X +10 wide x 10000 rows (read in-mem) 22 / 35 4.6 216.6 1.3X +10 wide x 10000 rows (write in-mem) 29 / 34 3.5 285.6 0.9X +10 wide x 10000 rows (read parquet) 61 / 81 1.6 610.3 0.4X +10 wide x 10000 rows (write parquet) 150 / 172 0.7 1504.7 0.2X +100 wide x 1000 rows (read in-mem) 21 / 29 4.8 207.9 1.3X +100 wide x 1000 rows (write in-mem) 30 / 57 3.3 304.9 0.9X +100 wide x 1000 rows (read parquet) 36 / 61 2.8 356.7 0.8X +100 wide x 1000 rows (write parquet) 108 / 136 0.9 1075.7 0.3X +1000 wide x 100 rows (read in-mem) 22 / 31 4.5 223.0 1.2X +1000 wide x 100 rows (write in-mem) 33 / 41 3.0 332.0 0.8X +1000 wide x 100 rows (read parquet) 49 / 66 2.0 493.6 0.5X +1000 wide x 100 rows (write parquet) 127 / 139 0.8 1265.9 0.2X +2500 wide x 40 rows (read in-mem) 23 / 34 4.4 226.0 1.2X +2500 wide x 40 rows (write in-mem) 33 / 42 3.1 326.6 0.8X +2500 wide x 40 rows (read parquet) 36 / 48 2.8 359.2 0.8X +2500 wide x 40 rows (write parquet) 155 / 168 0.6 1549.2 0.2X +*/ + } +} From d5bd64a20db2377fa5efc053607de730411489cb Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Thu, 2 Jun 2016 22:45:37 -0700 Subject: [PATCH 0507/1470] [SPARK-15733][SQL] Makes the explain output less verbose by hiding some verbose output like None, null, empty List, and etc. ## What changes were proposed in this pull request? This PR makes the explain output less verbose by hiding some verbose output like `None`, `null`, empty List `[]`, empty set `{}`, and etc. **Before change**: ``` == Physical Plan == ExecutedCommand : +- ShowTablesCommand None, None ``` **After change**: ``` == Physical Plan == ExecutedCommand : +- ShowTablesCommand ``` ## How was this patch tested? Manual test. Author: Sean Zhong Closes #13470 from clockfly/verbose_breakdown_4. (cherry picked from commit 6dde27404cb3d921d75dd6afca4b383f9df5976a) Signed-off-by: Cheng Lian --- .../spark/sql/catalyst/trees/TreeNode.scala | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) 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 3ebd815dce32c..50481cd298e2a 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 @@ -427,13 +427,21 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { private lazy val allChildren: Set[TreeNode[_]] = (children ++ innerChildren).toSet[TreeNode[_]] /** Returns a string representing the arguments to this node, minus any children */ - def argString: String = productIterator.flatMap { + def argString: String = stringArgs.flatMap { case tn: TreeNode[_] if allChildren.contains(tn) => Nil case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil - case tn: TreeNode[_] => s"${tn.simpleString}" :: Nil - case seq: Seq[BaseType] if seq.toSet.subsetOf(children.toSet) => Nil - case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil - case set: Set[_] => set.mkString("{", ",", "}") :: Nil + case Some(tn: TreeNode[_]) => tn.simpleString :: Nil + case tn: TreeNode[_] => tn.simpleString :: Nil + case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil + case iter: Iterable[_] if iter.isEmpty => Nil + case seq: Seq[_] => seq.mkString("[", ", ", "]") :: Nil + case set: Set[_] => set.mkString("{", ", ", "}") :: Nil + case array: Array[_] if array.isEmpty => Nil + case array: Array[_] => array.mkString("[", ", ", "]") :: Nil + case null => Nil + case None => Nil + case Some(null) => Nil + case Some(any) => any :: Nil case other => other :: Nil }.mkString(", ") From 1e13d09c526ec37c344950b07d938751bbd6fd0a Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Thu, 2 Jun 2016 22:49:17 -0700 Subject: [PATCH 0508/1470] [SPARK-14959][SQL] handle partitioned table directories in distributed filesystem ## What changes were proposed in this pull request? ##### The root cause: When `DataSource.resolveRelation` is trying to build `ListingFileCatalog` object, `ListLeafFiles` is invoked where a list of `FileStatus` objects are retrieved from the provided path. These FileStatus objects include directories for the partitions (id=0 and id=2 in the jira). However, these directory `FileStatus` objects also try to invoke `getFileBlockLocations` where directory is not allowed for `DistributedFileSystem`, hence the exception happens. This PR is to remove the block of code that invokes `getFileBlockLocations` for every FileStatus object of the provided path. Instead, we call `HadoopFsRelation.listLeafFiles` directly because this utility method filters out the directories before calling `getFileBlockLocations` for generating `LocatedFileStatus` objects. ## How was this patch tested? Regtest is run. Manual test: ``` scala> spark.read.format("parquet").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_part").show +-----+---+ | text| id| +-----+---+ |hello| 0| |world| 0| |hello| 1| |there| 1| +-----+---+ spark.read.format("orc").load("hdfs://bdavm009.svl.ibm.com:8020/user/spark/SPARK-14959_orc").show +-----+---+ | text| id| +-----+---+ |hello| 0| |world| 0| |hello| 1| |there| 1| +-----+---+ ``` I also tried it with 2 level of partitioning. I have not found a way to add test case in the unit test bucket that can test a real hdfs file location. Any suggestions will be appreciated. Author: Xin Wu Closes #13463 from xwu0226/SPARK-14959. (cherry picked from commit 76aa45d359d034e9ccaac64b36738d47e1e42f2c) Signed-off-by: Cheng Lian --- .../datasources/ListingFileCatalog.scala | 36 ++----------------- .../datasources/fileSourceInterfaces.scala | 10 ++++++ .../datasources/FileSourceStrategySuite.scala | 1 + 3 files changed, 14 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index 644e5d65d612c..dd3c96a792357 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -83,40 +83,10 @@ class ListingFileCatalog( val statuses: Seq[FileStatus] = paths.flatMap { path => val fs = path.getFileSystem(hadoopConf) logInfo(s"Listing $path on driver") - - val statuses = { - val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) - if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats - } - - statuses.map { - case f: LocatedFileStatus => f - - // NOTE: - // - // - Although S3/S3A/S3N file system can be quite slow for remote file metadata - // operations, calling `getFileBlockLocations` does no harm here since these file system - // implementations don't actually issue RPC for this method. - // - // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should a - // a big deal since we always use to `listLeafFilesInParallel` when the number of paths - // exceeds threshold. - case f => - HadoopFsRelation.createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) - } - }.filterNot { status => - val name = status.getPath.getName - HadoopFsRelation.shouldFilterOut(name) - } - - val (dirs, files) = statuses.partition(_.isDirectory) - - // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) - if (dirs.isEmpty) { - mutable.LinkedHashSet(files: _*) - } else { - mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) + Try(HadoopFsRelation.listLeafFiles(fs, fs.getFileStatus(path), pathFilter)). + getOrElse(Array.empty) } + mutable.LinkedHashSet(statuses: _*) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index e0e569bca48b4..7f3eed3fb1f80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -381,6 +381,16 @@ private[sql] object HadoopFsRelation extends Logging { } statuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { case f: LocatedFileStatus => f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. case f => createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 52dda8c6ace84..25f1443e7080a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -490,6 +490,7 @@ class LocalityTestFileSystem extends RawLocalFileSystem { override def getFileBlockLocations( file: FileStatus, start: Long, len: Long): Array[BlockLocation] = { + require(!file.isDirectory, "The file path can not be a directory.") val count = invocations.getAndAdd(1) Array(new BlockLocation(Array(s"host$count:50010"), Array(s"host$count"), 0, len)) } From 306601282983d3efa04b274bdc72d29c056ccdb1 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 3 Jun 2016 00:13:43 -0700 Subject: [PATCH 0509/1470] [SPARK-15745][SQL] Use classloader's getResource() for reading resource files in HiveTests ## What changes were proposed in this pull request? This is a cleaner approach in general but my motivation behind this change in particular is to be able to run these tests from anywhere without relying on system properties. ## How was this patch tested? Test only change Author: Sameer Agarwal Closes #13489 from sameeragarwal/resourcepath. (cherry picked from commit f7288e166c696da15e790c28fc3ed78531fd362d) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/hive/test/TestHive.scala | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) 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 81964db5477c6..1d1d5e3f7bd60 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 @@ -179,19 +179,8 @@ private[hive] class TestHiveSparkSession( hiveFilesTemp.mkdir() ShutdownHookManager.registerShutdownDeleteDir(hiveFilesTemp) - val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { - new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) - } else { - new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + - File.separator + "resources") - } - def getHiveFile(path: String): File = { - val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar) - hiveDevHome - .map(new File(_, stripped)) - .filter(_.exists) - .getOrElse(new File(inRepoTests, stripped)) + new File(Thread.currentThread().getContextClassLoader.getResource(path).getFile) } val describedTable = "DESCRIBE (\\w+)".r From 3a9ee549c16b601d98781db83939070d67616078 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 3 Jun 2016 00:36:06 -0700 Subject: [PATCH 0510/1470] [SPARK-15744][SQL] Rename two TungstenAggregation*Suites and update codgen/error messages/comments ## What changes were proposed in this pull request? For consistency, this PR updates some remaining `TungstenAggregation/SortBasedAggregate` after SPARK-15728. - Update a comment in codegen in `VectorizedHashMapGenerator.scala`. - `TungstenAggregationQuerySuite` --> `HashAggregationQuerySuite` - `TungstenAggregationQueryWithControlledFallbackSuite` --> `HashAggregationQueryWithControlledFallbackSuite` - Update two error messages in `SQLQuerySuite.scala` and `AggregationQuerySuite.scala`. - Update several comments. ## How was this patch tested? Manual (Only comment changes and test suite renamings). Author: Dongjoon Hyun Closes #13487 from dongjoon-hyun/SPARK-15744. (cherry picked from commit b9fcfb3bd14592ac9f1a8e5c2bb31412b9603b60) Signed-off-by: Reynold Xin --- .../vectorized/AggregateHashMap.java | 2 +- .../sql/execution/aggregate/AggUtils.scala | 2 +- .../aggregate/AggregationIterator.scala | 2 +- .../aggregate/HashAggregateExec.scala | 2 +- .../VectorizedHashMapGenerator.scala | 4 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 34 +++++++++---------- .../execution/AggregationQuerySuite.scala | 12 +++---- 8 files changed, 30 insertions(+), 30 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/AggregateHashMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/AggregateHashMap.java index 69ce54390fead..25a565d32638d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/AggregateHashMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/AggregateHashMap.java @@ -30,7 +30,7 @@ * This is an illustrative implementation of an append-only single-key/single value aggregate hash * map that can act as a 'cache' for extremely fast key-value lookups while evaluating aggregates * (and fall back to the `BytesToBytesMap` if a given key isn't found). This can be potentially - * 'codegened' in TungstenAggregate to speed up aggregates w/ key. + * 'codegened' in HashAggregate to speed up aggregates w/ key. * * It is backed by a power-of-2-sized array for index lookups and a columnar batch that stores the * key-value pairs. The index lookups in the array rely on linear probing (with a small number of diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index a9ec0c8709db7..4fbb9d554c9bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -82,7 +82,7 @@ object AggUtils { aggregateExpressions: Seq[AggregateExpression], resultExpressions: Seq[NamedExpression], child: SparkPlan): Seq[SparkPlan] = { - // Check if we can use TungstenAggregate. + // Check if we can use HashAggregate. // 1. Create an Aggregate Operator for partial aggregations. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index 81aacb437ba54..34de76dd4ab4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -52,7 +52,7 @@ abstract class AggregationIterator( * - PartialMerge (for single distinct) * - Partial and PartialMerge (for single distinct) * - Final - * - Complete (for SortBasedAggregate with functions that does not support Partial) + * - Complete (for SortAggregate with functions that does not support Partial) * - Final and Complete (currently not used) * * TODO: AggregateMode should have only two modes: Update and Merge, AggregateExpression diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index fad81b558cfcb..f5bc0628b6458 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -458,7 +458,7 @@ case class HashAggregateExec( } /** - * Using the vectorized hash map in TungstenAggregate is currently supported for all primitive + * Using the vectorized hash map in HashAggregate is currently supported for all primitive * data types during partial aggregation. However, we currently only enable the hash map for a * subset of cases that've been verified to show performance improvements on our benchmarks * subject to an internal conf that sets an upper limit on the maximum length of the aggregate diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala index 61bd6eb3cde66..8a3f466ccfef3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.types._ /** * This is a helper class to generate an append-only vectorized hash map that can act as a 'cache' * for extremely fast key-value lookups while evaluating aggregates (and fall back to the - * `BytesToBytesMap` if a given key isn't found). This is 'codegened' in TungstenAggregate to speed + * `BytesToBytesMap` if a given key isn't found). This is 'codegened' in HashAggregate to speed * up aggregates w/ key. * * It is backed by a power-of-2-sized array for index lookups and a columnar batch that stores the @@ -127,7 +127,7 @@ class VectorizedHashMapGenerator( | public $generatedClassName() { | batch = org.apache.spark.sql.execution.vectorized.ColumnarBatch.allocate(schema, | org.apache.spark.memory.MemoryMode.ON_HEAP, capacity); - | // TODO: Possibly generate this projection in TungstenAggregate directly + | // TODO: Possibly generate this projection in HashAggregate directly | aggregateBufferBatch = org.apache.spark.sql.execution.vectorized.ColumnarBatch.allocate( | aggregateBufferSchema, org.apache.spark.memory.MemoryMode.ON_HEAP, capacity); | for (int i = 0 ; i < aggregateBufferBatch.numCols(); i++) { 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 4fcd6bc0d9ec9..8284e8d6d89b6 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 @@ -251,7 +251,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { if (!hasGeneratedAgg) { fail( s""" - |Codegen is enabled, but query $sqlText does not have TungstenAggregate in the plan. + |Codegen is enabled, but query $sqlText does not have HashAggregate in the plan. |${df.queryExecution.simpleString} """.stripMargin) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index d3639d97355b5..fd956bc4ef900 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.execution.metric @@ -135,8 +135,8 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { test("Aggregate metrics") { // Assume the execution plan is - // ... -> TungstenAggregate(nodeId = 2) -> Exchange(nodeId = 1) - // -> TungstenAggregate(nodeId = 0) + // ... -> HashAggregate(nodeId = 2) -> Exchange(nodeId = 1) + // -> HashAggregate(nodeId = 0) val df = testData2.groupBy().count() // 2 partitions testSparkPlanMetrics(df, 1, Map( 2L -> ("HashAggregate", Map("number of output rows" -> 2L)), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 9fc5628b28dbb..a16fe3228b1fc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -869,10 +869,10 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, new UDT.MyDenseVectorUDT()) - // Right now, we will use SortBasedAggregate to handle UDAFs. - // UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortBasedAggregate to use + // Right now, we will use SortAggregate to handle UDAFs. + // UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortAggregate to use // UnsafeRow as the aggregation buffer. While, dataTypes will trigger - // SortBasedAggregate to use a safe row as the aggregation buffer. + // SortAggregate to use a safe row as the aggregation buffer. Seq(dataTypes, UnsafeRow.mutableFieldTypes.asScala.toSeq).foreach { dataTypes => val fields = dataTypes.zipWithIndex.map { case (dataType, index) => StructField(s"col$index", dataType, nullable = true) @@ -992,10 +992,10 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te } -class TungstenAggregationQuerySuite extends AggregationQuerySuite +class HashAggregationQuerySuite extends AggregationQuerySuite -class TungstenAggregationQueryWithControlledFallbackSuite extends AggregationQuerySuite { +class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySuite { override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { Seq(0, 10).foreach { maxColumnarHashMapColumns => @@ -1013,7 +1013,7 @@ class TungstenAggregationQueryWithControlledFallbackSuite extends AggregationQue case Some(errorMessage) => val newErrorMessage = s""" - |The following aggregation query failed when using TungstenAggregate with + |The following aggregation query failed when using HashAggregate with |controlled fallback (it falls back to bytes to bytes map once it has processed |${fallbackStartsAt - 1} input rows and to sort-based aggregation once it has |processed $fallbackStartsAt input rows). The query is ${actual.queryExecution} From deda53a9e43c8be39f9e5df88371a7904c1c0e9a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 3 Jun 2016 00:43:02 -0700 Subject: [PATCH 0511/1470] [SPARK-15494][SQL] encoder code cleanup ## What changes were proposed in this pull request? Our encoder framework has been evolved a lot, this PR tries to clean up the code to make it more readable and emphasise the concept that encoder should be used as a container of serde expressions. 1. move validation logic to analyzer instead of encoder 2. only have a `resolveAndBind` method in encoder instead of `resolve` and `bind`, as we don't have the encoder life cycle concept anymore. 3. `Dataset` don't need to keep a resolved encoder, as there is no such concept anymore. bound encoder is still needed to do serialization outside of query framework. 4. Using `BoundReference` to represent an unresolved field in deserializer expression is kind of weird, this PR adds a `GetColumnByOrdinal` for this purpose. (serializer expression still use `BoundReference`, we can replace it with `GetColumnByOrdinal` in follow-ups) ## How was this patch tested? existing test Author: Wenchen Fan Author: Cheng Lian Closes #13269 from cloud-fan/clean-encoder. (cherry picked from commit 190ff274fd71662023a804cf98400c71f9f7da4f) Signed-off-by: Cheng Lian --- .../linalg/UDTSerializationBenchmark.scala | 2 +- .../scala/org/apache/spark/sql/Encoders.scala | 3 +- .../sql/catalyst/JavaTypeInference.scala | 6 +- .../spark/sql/catalyst/ScalaReflection.scala | 307 +++++++++--------- .../sql/catalyst/analysis/Analyzer.scala | 53 ++- .../sql/catalyst/analysis/unresolved.scala | 7 + .../catalyst/encoders/ExpressionEncoder.scala | 134 ++------ .../sql/catalyst/encoders/RowEncoder.scala | 8 +- .../sql/catalyst/plans/logical/object.scala | 19 +- .../encoders/EncoderResolutionSuite.scala | 42 +-- .../encoders/ExpressionEncoderSuite.scala | 11 +- .../catalyst/encoders/RowEncoderSuite.scala | 14 +- .../scala/org/apache/spark/sql/Dataset.scala | 48 +-- .../spark/sql/KeyValueGroupedDataset.scala | 26 +- .../spark/sql/RelationalGroupedDataset.scala | 2 +- .../aggregate/TypedAggregateExpression.scala | 6 +- .../org/apache/spark/sql/functions.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 8 +- .../org/apache/spark/sql/QueryTest.scala | 8 +- .../sql/execution/GroupedIteratorSuite.scala | 6 +- .../spark/sql/streaming/StreamTest.scala | 4 +- 21 files changed, 324 insertions(+), 392 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala index be7110ad6bbf0..8b439e6b7a017 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala @@ -29,7 +29,7 @@ object UDTSerializationBenchmark { val iters = 1e2.toInt val numRows = 1e3.toInt - val encoder = ExpressionEncoder[Vector].defaultBinding + val encoder = ExpressionEncoder[Vector].resolveAndBind() val vectors = (1 to numRows).map { i => Vectors.dense(Array.fill(1e5.toInt)(1.0 * i)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index fa96f8223d179..673c587b18325 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -23,6 +23,7 @@ import scala.reflect.{classTag, ClassTag} import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.objects.{DecodeUsingSerializer, EncodeUsingSerializer} import org.apache.spark.sql.catalyst.expressions.BoundReference @@ -208,7 +209,7 @@ object Encoders { BoundReference(0, ObjectType(classOf[AnyRef]), nullable = true), kryo = useKryo)), deserializer = DecodeUsingSerializer[T]( - BoundReference(0, BinaryType, nullable = true), classTag[T], kryo = useKryo), + GetColumnByOrdinal(0, BinaryType), classTag[T], kryo = useKryo), clsTag = classTag[T] ) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 1fe143494abad..b3a233ae390ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -25,7 +25,7 @@ import scala.language.existentials import com.google.common.reflect.TypeToken -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} @@ -177,8 +177,8 @@ object JavaTypeInference { .map(p => UnresolvedExtractValue(p, expressions.Literal(part))) .getOrElse(UnresolvedAttribute(part)) - /** Returns the current path or `BoundReference`. */ - def getPath: Expression = path.getOrElse(BoundReference(0, inferDataType(typeToken)._1, true)) + /** Returns the current path or `GetColumnByOrdinal`. */ + def getPath: Expression = path.getOrElse(GetColumnByOrdinal(0, inferDataType(typeToken)._1)) typeToken.getRawType match { case c if !inferExternalType(c).isInstanceOf[ObjectType] => getPath 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 47508618178ea..78c145d4fd936 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} @@ -156,17 +156,17 @@ object ScalaReflection extends ScalaReflection { walkedTypePath: Seq[String]): Expression = { val newPath = path .map(p => GetStructField(p, ordinal)) - .getOrElse(BoundReference(ordinal, dataType, false)) + .getOrElse(GetColumnByOrdinal(ordinal, dataType)) upCastToExpectedType(newPath, dataType, walkedTypePath) } - /** Returns the current path or `BoundReference`. */ + /** Returns the current path or `GetColumnByOrdinal`. */ def getPath: Expression = { val dataType = schemaFor(tpe).dataType if (path.isDefined) { path.get } else { - upCastToExpectedType(BoundReference(0, dataType, true), dataType, walkedTypePath) + upCastToExpectedType(GetColumnByOrdinal(0, dataType), dataType, walkedTypePath) } } @@ -421,7 +421,7 @@ object ScalaReflection extends ScalaReflection { def serializerFor[T : TypeTag](inputObject: Expression): CreateNamedStruct = { val tpe = localTypeOf[T] val clsName = getClassNameFromType(tpe) - val walkedTypePath = s"""- root class: "${clsName}"""" :: Nil + val walkedTypePath = s"""- root class: "$clsName"""" :: Nil serializerFor(inputObject, tpe, walkedTypePath) match { case expressions.If(_, _, s: CreateNamedStruct) if definedByConstructorParams(tpe) => s case other => CreateNamedStruct(expressions.Literal("value") :: other :: Nil) @@ -449,157 +449,156 @@ object ScalaReflection extends ScalaReflection { } } - if (!inputObject.dataType.isInstanceOf[ObjectType]) { - inputObject - } else { - tpe match { - case t if t <:< localTypeOf[Option[_]] => - val TypeRef(_, _, Seq(optType)) = t - val className = getClassNameFromType(optType) - val newPath = s"""- option value class: "$className"""" +: walkedTypePath - val unwrapped = UnwrapOption(dataTypeFor(optType), inputObject) - serializerFor(unwrapped, optType, newPath) - - // Since List[_] also belongs to localTypeOf[Product], we put this case before - // "case t if definedByConstructorParams(t)" to make sure it will match to the - // case "localTypeOf[Seq[_]]" - case t if t <:< localTypeOf[Seq[_]] => - val TypeRef(_, _, Seq(elementType)) = t - toCatalystArray(inputObject, elementType) - - case t if t <:< localTypeOf[Array[_]] => - val TypeRef(_, _, Seq(elementType)) = t - toCatalystArray(inputObject, elementType) - - case t if t <:< localTypeOf[Map[_, _]] => - val TypeRef(_, _, Seq(keyType, valueType)) = t - - val keys = - Invoke( - Invoke(inputObject, "keysIterator", - ObjectType(classOf[scala.collection.Iterator[_]])), - "toSeq", - ObjectType(classOf[scala.collection.Seq[_]])) - val convertedKeys = toCatalystArray(keys, keyType) - - val values = - Invoke( - Invoke(inputObject, "valuesIterator", - ObjectType(classOf[scala.collection.Iterator[_]])), - "toSeq", - ObjectType(classOf[scala.collection.Seq[_]])) - val convertedValues = toCatalystArray(values, valueType) - - val Schema(keyDataType, _) = schemaFor(keyType) - val Schema(valueDataType, valueNullable) = schemaFor(valueType) - NewInstance( - classOf[ArrayBasedMapData], - convertedKeys :: convertedValues :: Nil, - dataType = MapType(keyDataType, valueDataType, valueNullable)) - - case t if t <:< localTypeOf[String] => - StaticInvoke( - classOf[UTF8String], - StringType, - "fromString", - inputObject :: Nil) - - case t if t <:< localTypeOf[java.sql.Timestamp] => - StaticInvoke( - DateTimeUtils.getClass, - TimestampType, - "fromJavaTimestamp", - inputObject :: Nil) - - case t if t <:< localTypeOf[java.sql.Date] => - StaticInvoke( - DateTimeUtils.getClass, - DateType, - "fromJavaDate", - inputObject :: Nil) - - case t if t <:< localTypeOf[BigDecimal] => - StaticInvoke( - Decimal.getClass, - DecimalType.SYSTEM_DEFAULT, - "apply", - inputObject :: Nil) - - case t if t <:< localTypeOf[java.math.BigDecimal] => - StaticInvoke( - Decimal.getClass, - DecimalType.SYSTEM_DEFAULT, - "apply", - inputObject :: Nil) - - case t if t <:< localTypeOf[java.math.BigInteger] => - StaticInvoke( - Decimal.getClass, - DecimalType.BigIntDecimal, - "apply", - inputObject :: Nil) - - case t if t <:< localTypeOf[scala.math.BigInt] => - StaticInvoke( - Decimal.getClass, - DecimalType.BigIntDecimal, - "apply", - inputObject :: Nil) - - case t if t <:< localTypeOf[java.lang.Integer] => - Invoke(inputObject, "intValue", IntegerType) - case t if t <:< localTypeOf[java.lang.Long] => - Invoke(inputObject, "longValue", LongType) - case t if t <:< localTypeOf[java.lang.Double] => - Invoke(inputObject, "doubleValue", DoubleType) - case t if t <:< localTypeOf[java.lang.Float] => - Invoke(inputObject, "floatValue", FloatType) - case t if t <:< localTypeOf[java.lang.Short] => - Invoke(inputObject, "shortValue", ShortType) - case t if t <:< localTypeOf[java.lang.Byte] => - Invoke(inputObject, "byteValue", ByteType) - case t if t <:< localTypeOf[java.lang.Boolean] => - Invoke(inputObject, "booleanValue", BooleanType) - - case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t) - .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() - val obj = NewInstance( - udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), - Nil, - dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "serialize", udt, inputObject :: Nil) - - case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] - val obj = NewInstance( - udt.getClass, - Nil, - dataType = ObjectType(udt.getClass)) - Invoke(obj, "serialize", udt, inputObject :: Nil) - - case t if definedByConstructorParams(t) => - val params = getConstructorParameters(t) - val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => - if (javaKeywords.contains(fieldName)) { - throw new UnsupportedOperationException(s"`$fieldName` is a reserved keyword and " + - "cannot be used as field name\n" + walkedTypePath.mkString("\n")) - } + tpe match { + case _ if !inputObject.dataType.isInstanceOf[ObjectType] => inputObject - val fieldValue = Invoke(inputObject, fieldName, dataTypeFor(fieldType)) - val clsName = getClassNameFromType(fieldType) - val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath - expressions.Literal(fieldName) :: serializerFor(fieldValue, fieldType, newPath) :: Nil - }) - val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) - expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) - - case other => - throw new UnsupportedOperationException( - s"No Encoder found for $tpe\n" + walkedTypePath.mkString("\n")) - } + case t if t <:< localTypeOf[Option[_]] => + val TypeRef(_, _, Seq(optType)) = t + val className = getClassNameFromType(optType) + val newPath = s"""- option value class: "$className"""" +: walkedTypePath + val unwrapped = UnwrapOption(dataTypeFor(optType), inputObject) + serializerFor(unwrapped, optType, newPath) + + // Since List[_] also belongs to localTypeOf[Product], we put this case before + // "case t if definedByConstructorParams(t)" to make sure it will match to the + // case "localTypeOf[Seq[_]]" + case t if t <:< localTypeOf[Seq[_]] => + val TypeRef(_, _, Seq(elementType)) = t + toCatalystArray(inputObject, elementType) + + case t if t <:< localTypeOf[Array[_]] => + val TypeRef(_, _, Seq(elementType)) = t + toCatalystArray(inputObject, elementType) + + case t if t <:< localTypeOf[Map[_, _]] => + val TypeRef(_, _, Seq(keyType, valueType)) = t + + val keys = + Invoke( + Invoke(inputObject, "keysIterator", + ObjectType(classOf[scala.collection.Iterator[_]])), + "toSeq", + ObjectType(classOf[scala.collection.Seq[_]])) + val convertedKeys = toCatalystArray(keys, keyType) + + val values = + Invoke( + Invoke(inputObject, "valuesIterator", + ObjectType(classOf[scala.collection.Iterator[_]])), + "toSeq", + ObjectType(classOf[scala.collection.Seq[_]])) + val convertedValues = toCatalystArray(values, valueType) + + val Schema(keyDataType, _) = schemaFor(keyType) + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + NewInstance( + classOf[ArrayBasedMapData], + convertedKeys :: convertedValues :: Nil, + dataType = MapType(keyDataType, valueDataType, valueNullable)) + + case t if t <:< localTypeOf[String] => + StaticInvoke( + classOf[UTF8String], + StringType, + "fromString", + inputObject :: Nil) + + case t if t <:< localTypeOf[java.sql.Timestamp] => + StaticInvoke( + DateTimeUtils.getClass, + TimestampType, + "fromJavaTimestamp", + inputObject :: Nil) + + case t if t <:< localTypeOf[java.sql.Date] => + StaticInvoke( + DateTimeUtils.getClass, + DateType, + "fromJavaDate", + inputObject :: Nil) + + case t if t <:< localTypeOf[BigDecimal] => + StaticInvoke( + Decimal.getClass, + DecimalType.SYSTEM_DEFAULT, + "apply", + inputObject :: Nil) + + case t if t <:< localTypeOf[java.math.BigDecimal] => + StaticInvoke( + Decimal.getClass, + DecimalType.SYSTEM_DEFAULT, + "apply", + inputObject :: Nil) + + case t if t <:< localTypeOf[java.math.BigInteger] => + StaticInvoke( + Decimal.getClass, + DecimalType.BigIntDecimal, + "apply", + inputObject :: Nil) + + case t if t <:< localTypeOf[scala.math.BigInt] => + StaticInvoke( + Decimal.getClass, + DecimalType.BigIntDecimal, + "apply", + inputObject :: Nil) + + case t if t <:< localTypeOf[java.lang.Integer] => + Invoke(inputObject, "intValue", IntegerType) + case t if t <:< localTypeOf[java.lang.Long] => + Invoke(inputObject, "longValue", LongType) + case t if t <:< localTypeOf[java.lang.Double] => + Invoke(inputObject, "doubleValue", DoubleType) + case t if t <:< localTypeOf[java.lang.Float] => + Invoke(inputObject, "floatValue", FloatType) + case t if t <:< localTypeOf[java.lang.Short] => + Invoke(inputObject, "shortValue", ShortType) + case t if t <:< localTypeOf[java.lang.Byte] => + Invoke(inputObject, "byteValue", ByteType) + case t if t <:< localTypeOf[java.lang.Boolean] => + Invoke(inputObject, "booleanValue", BooleanType) + + case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => + val udt = getClassFromType(t) + .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + val obj = NewInstance( + udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), + Nil, + dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) + Invoke(obj, "serialize", udt, inputObject :: Nil) + + case t if UDTRegistration.exists(getClassNameFromType(t)) => + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() + .asInstanceOf[UserDefinedType[_]] + val obj = NewInstance( + udt.getClass, + Nil, + dataType = ObjectType(udt.getClass)) + Invoke(obj, "serialize", udt, inputObject :: Nil) + + case t if definedByConstructorParams(t) => + val params = getConstructorParameters(t) + val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => + if (javaKeywords.contains(fieldName)) { + throw new UnsupportedOperationException(s"`$fieldName` is a reserved keyword and " + + "cannot be used as field name\n" + walkedTypePath.mkString("\n")) + } + + val fieldValue = Invoke(inputObject, fieldName, dataTypeFor(fieldType)) + val clsName = getClassNameFromType(fieldType) + val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath + expressions.Literal(fieldName) :: serializerFor(fieldValue, fieldType, newPath) :: Nil + }) + val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) + expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) + + case other => + throw new UnsupportedOperationException( + s"No Encoder found for $tpe\n" + walkedTypePath.mkString("\n")) } + } /** 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 02966796afdd7..4f6b4830cd6f6 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 @@ -722,6 +722,7 @@ class Analyzer( // Else, throw exception. try { expr transformUp { + case GetColumnByOrdinal(ordinal, _) => plan.output(ordinal) case u @ UnresolvedAttribute(nameParts) => withPosition(u) { plan.resolve(nameParts, resolver).getOrElse(u) } case UnresolvedExtractValue(child, fieldName) if child.resolved => @@ -1924,10 +1925,54 @@ class Analyzer( } else { inputAttributes } - val unbound = deserializer transform { - case b: BoundReference => inputs(b.ordinal) - } - resolveExpression(unbound, LocalRelation(inputs), throws = true) + + validateTopLevelTupleFields(deserializer, inputs) + val resolved = resolveExpression( + deserializer, LocalRelation(inputs), throws = true) + validateNestedTupleFields(resolved) + resolved + } + } + + private def fail(schema: StructType, maxOrdinal: Int): Unit = { + throw new AnalysisException(s"Try to map ${schema.simpleString} to Tuple${maxOrdinal + 1}, " + + "but failed as the number of fields does not line up.") + } + + /** + * For each top-level Tuple field, we use [[GetColumnByOrdinal]] to get its corresponding column + * by position. However, the actual number of columns may be different from the number of Tuple + * fields. This method is used to check the number of columns and fields, and throw an + * exception if they do not match. + */ + private def validateTopLevelTupleFields( + deserializer: Expression, inputs: Seq[Attribute]): Unit = { + val ordinals = deserializer.collect { + case GetColumnByOrdinal(ordinal, _) => ordinal + }.distinct.sorted + + if (ordinals.nonEmpty && ordinals != inputs.indices) { + fail(inputs.toStructType, ordinals.last) + } + } + + /** + * For each nested Tuple field, we use [[GetStructField]] to get its corresponding struct field + * by position. However, the actual number of struct fields may be different from the number + * of nested Tuple fields. This method is used to check the number of struct fields and nested + * Tuple fields, and throw an exception if they do not match. + */ + private def validateNestedTupleFields(deserializer: Expression): Unit = { + val structChildToOrdinals = deserializer + .collect { case g: GetStructField => g } + .groupBy(_.child) + .mapValues(_.map(_.ordinal).distinct.sorted) + + structChildToOrdinals.foreach { case (expr, ordinals) => + val schema = expr.dataType.asInstanceOf[StructType] + if (ordinals != schema.indices) { + fail(schema, ordinals.last) + } } } } 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 e953eda7843c9..b883546135f07 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 @@ -366,3 +366,10 @@ case class UnresolvedDeserializer(deserializer: Expression, inputAttributes: Seq override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override lazy val resolved = false } + +case class GetColumnByOrdinal(ordinal: Int, dataType: DataType) extends LeafExpression + with Unevaluable with NonSQLExpression { + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 2296946cd7c5e..cc59d06fa3518 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -17,19 +17,17 @@ package org.apache.spark.sql.catalyst.encoders -import java.util.concurrent.ConcurrentMap - import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag} -import org.apache.spark.sql.{AnalysisException, Encoder} +import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.{InternalRow, JavaTypeInference, ScalaReflection} -import org.apache.spark.sql.catalyst.analysis.{SimpleAnalyzer, UnresolvedAttribute, UnresolvedDeserializer, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection} import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, NewInstance} import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LocalRelation} import org.apache.spark.sql.types.{ObjectType, StructField, StructType} import org.apache.spark.util.Utils @@ -121,15 +119,15 @@ object ExpressionEncoder { val childrenDeserializers = encoders.zipWithIndex.map { case (enc, index) => if (enc.flat) { enc.deserializer.transform { - case b: BoundReference => b.copy(ordinal = index) + case g: GetColumnByOrdinal => g.copy(ordinal = index) } } else { - val input = BoundReference(index, enc.schema, nullable = true) + val input = GetColumnByOrdinal(index, enc.schema) val deserialized = enc.deserializer.transformUp { case UnresolvedAttribute(nameParts) => assert(nameParts.length == 1) UnresolvedExtractValue(input, Literal(nameParts.head)) - case BoundReference(ordinal, dt, _) => GetStructField(input, ordinal) + case GetColumnByOrdinal(ordinal, _) => GetStructField(input, ordinal) } If(IsNull(input), Literal.create(null, deserialized.dataType), deserialized) } @@ -192,6 +190,26 @@ case class ExpressionEncoder[T]( if (flat) require(serializer.size == 1) + /** + * Returns a new copy of this encoder, where the `deserializer` is resolved and bound to the + * given schema. + * + * Note that, ideally encoder is used as a container of serde expressions, the resolution and + * binding stuff should happen inside query framework. However, in some cases we need to + * use encoder as a function to do serialization directly(e.g. Dataset.collect), then we can use + * this method to do resolution and binding outside of query framework. + */ + def resolveAndBind( + attrs: Seq[Attribute] = schema.toAttributes, + analyzer: Analyzer = SimpleAnalyzer): ExpressionEncoder[T] = { + val dummyPlan = CatalystSerde.deserialize(LocalRelation(attrs))(this) + val analyzedPlan = analyzer.execute(dummyPlan) + analyzer.checkAnalysis(analyzedPlan) + val resolved = SimplifyCasts(analyzedPlan).asInstanceOf[DeserializeToObject].deserializer + val bound = BindReferences.bindReference(resolved, attrs) + copy(deserializer = bound) + } + @transient private lazy val extractProjection = GenerateUnsafeProjection.generate(serializer) @@ -201,16 +219,6 @@ case class ExpressionEncoder[T]( @transient private lazy val constructProjection = GenerateSafeProjection.generate(deserializer :: Nil) - /** - * Returns this encoder where it has been bound to its own output (i.e. no remaping of columns - * is performed). - */ - def defaultBinding: ExpressionEncoder[T] = { - val attrs = schema.toAttributes - resolve(attrs, OuterScopes.outerScopes).bind(attrs) - } - - /** * Returns a new set (with unique ids) of [[NamedExpression]] that represent the serialized form * of this object. @@ -236,7 +244,7 @@ case class ExpressionEncoder[T]( /** * Returns an object of type `T`, extracting the required values from the provided row. Note that - * you must `resolve` and `bind` an encoder to a specific schema before you can call this + * you must `resolveAndBind` an encoder to a specific schema before you can call this * function. */ def fromRow(row: InternalRow): T = try { @@ -259,94 +267,6 @@ case class ExpressionEncoder[T]( }) } - /** - * Validates `deserializer` to make sure it can be resolved by given schema, and produce - * friendly error messages to explain why it fails to resolve if there is something wrong. - */ - def validate(schema: Seq[Attribute]): Unit = { - def fail(st: StructType, maxOrdinal: Int): Unit = { - throw new AnalysisException(s"Try to map ${st.simpleString} to Tuple${maxOrdinal + 1}, " + - "but failed as the number of fields does not line up.\n" + - " - Input schema: " + StructType.fromAttributes(schema).simpleString + "\n" + - " - Target schema: " + this.schema.simpleString) - } - - // If this is a tuple encoder or tupled encoder, which means its leaf nodes are all - // `BoundReference`, make sure their ordinals are all valid. - var maxOrdinal = -1 - deserializer.foreach { - case b: BoundReference => if (b.ordinal > maxOrdinal) maxOrdinal = b.ordinal - case _ => - } - if (maxOrdinal >= 0 && maxOrdinal != schema.length - 1) { - fail(StructType.fromAttributes(schema), maxOrdinal) - } - - // If we have nested tuple, the `fromRowExpression` will contains `GetStructField` instead of - // `UnresolvedExtractValue`, so we need to check if their ordinals are all valid. - // Note that, `BoundReference` contains the expected type, but here we need the actual type, so - // we unbound it by the given `schema` and propagate the actual type to `GetStructField`, after - // we resolve the `fromRowExpression`. - val resolved = SimpleAnalyzer.resolveExpression( - deserializer, - LocalRelation(schema), - throws = true) - - val unbound = resolved transform { - case b: BoundReference => schema(b.ordinal) - } - - val exprToMaxOrdinal = scala.collection.mutable.HashMap.empty[Expression, Int] - unbound.foreach { - case g: GetStructField => - val maxOrdinal = exprToMaxOrdinal.getOrElse(g.child, -1) - if (maxOrdinal < g.ordinal) { - exprToMaxOrdinal.update(g.child, g.ordinal) - } - case _ => - } - exprToMaxOrdinal.foreach { - case (expr, maxOrdinal) => - val schema = expr.dataType.asInstanceOf[StructType] - if (maxOrdinal != schema.length - 1) { - fail(schema, maxOrdinal) - } - } - } - - /** - * Returns a new copy of this encoder, where the `deserializer` is resolved to the given schema. - */ - def resolve( - schema: Seq[Attribute], - outerScopes: ConcurrentMap[String, AnyRef]): ExpressionEncoder[T] = { - // Make a fake plan to wrap the deserializer, so that we can go though the whole analyzer, check - // analysis, go through optimizer, etc. - val plan = Project( - Alias(UnresolvedDeserializer(deserializer, schema), "")() :: Nil, - LocalRelation(schema)) - val analyzedPlan = SimpleAnalyzer.execute(plan) - SimpleAnalyzer.checkAnalysis(analyzedPlan) - copy(deserializer = SimplifyCasts(analyzedPlan).expressions.head.children.head) - } - - /** - * Returns a copy of this encoder where the `deserializer` has been bound to the - * ordinals of the given schema. Note that you need to first call resolve before bind. - */ - def bind(schema: Seq[Attribute]): ExpressionEncoder[T] = { - copy(deserializer = BindReferences.bindReference(deserializer, schema)) - } - - /** - * Returns a new encoder with input columns shifted by `delta` ordinals - */ - def shift(delta: Int): ExpressionEncoder[T] = { - copy(deserializer = deserializer transform { - case r: BoundReference => r.copy(ordinal = r.ordinal + delta) - }) - } - protected val attrs = serializer.flatMap(_.collect { case _: UnresolvedAttribute => "" case a: Attribute => s"#${a.exprId}" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 0de9166aa29aa..3c6ae1c5ccd8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -210,12 +211,7 @@ object RowEncoder { case p: PythonUserDefinedType => p.sqlType case other => other } - val field = BoundReference(i, dt, f.nullable) - If( - IsNull(field), - Literal.create(null, externalDataTypeFor(dt)), - deserializerFor(field) - ) + deserializerFor(GetColumnByOrdinal(i, dt)) } CreateExternalRow(fields, schema) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 98ce5dd2efd91..55d8adf0408fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -30,26 +30,13 @@ object CatalystSerde { DeserializeToObject(deserializer, generateObjAttr[T], child) } - def deserialize(child: LogicalPlan, encoder: ExpressionEncoder[Row]): DeserializeToObject = { - val deserializer = UnresolvedDeserializer(encoder.deserializer) - DeserializeToObject(deserializer, generateObjAttrForRow(encoder), child) - } - def serialize[T : Encoder](child: LogicalPlan): SerializeFromObject = { SerializeFromObject(encoderFor[T].namedExpressions, child) } - def serialize(child: LogicalPlan, encoder: ExpressionEncoder[Row]): SerializeFromObject = { - SerializeFromObject(encoder.namedExpressions, child) - } - def generateObjAttr[T : Encoder]: Attribute = { AttributeReference("obj", encoderFor[T].deserializer.dataType, nullable = false)() } - - def generateObjAttrForRow(encoder: ExpressionEncoder[Row]): Attribute = { - AttributeReference("obj", encoder.deserializer.dataType, nullable = false)() - } } /** @@ -128,16 +115,16 @@ object MapPartitionsInR { schema: StructType, encoder: ExpressionEncoder[Row], child: LogicalPlan): LogicalPlan = { - val deserialized = CatalystSerde.deserialize(child, encoder) + val deserialized = CatalystSerde.deserialize(child)(encoder) val mapped = MapPartitionsInR( func, packageNames, broadcastVars, encoder.schema, schema, - CatalystSerde.generateObjAttrForRow(RowEncoder(schema)), + CatalystSerde.generateObjAttr(RowEncoder(schema)), deserialized) - CatalystSerde.serialize(mapped, RowEncoder(schema)) + CatalystSerde.serialize(mapped)(RowEncoder(schema)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index 3ad0dae767be3..7251202c7bd58 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -41,17 +41,17 @@ class EncoderResolutionSuite extends PlanTest { // int type can be up cast to long type val attrs1 = Seq('a.string, 'b.int) - encoder.resolve(attrs1, null).bind(attrs1).fromRow(InternalRow(str, 1)) + encoder.resolveAndBind(attrs1).fromRow(InternalRow(str, 1)) // int type can be up cast to string type val attrs2 = Seq('a.int, 'b.long) - encoder.resolve(attrs2, null).bind(attrs2).fromRow(InternalRow(1, 2L)) + encoder.resolveAndBind(attrs2).fromRow(InternalRow(1, 2L)) } test("real type doesn't match encoder schema but they are compatible: nested product") { val encoder = ExpressionEncoder[ComplexClass] val attrs = Seq('a.int, 'b.struct('a.int, 'b.long)) - encoder.resolve(attrs, null).bind(attrs).fromRow(InternalRow(1, InternalRow(2, 3L))) + encoder.resolveAndBind(attrs).fromRow(InternalRow(1, InternalRow(2, 3L))) } test("real type doesn't match encoder schema but they are compatible: tupled encoder") { @@ -59,7 +59,7 @@ class EncoderResolutionSuite extends PlanTest { ExpressionEncoder[StringLongClass], ExpressionEncoder[Long]) val attrs = Seq('a.struct('a.string, 'b.byte), 'b.int) - encoder.resolve(attrs, null).bind(attrs).fromRow(InternalRow(InternalRow(str, 1.toByte), 2)) + encoder.resolveAndBind(attrs).fromRow(InternalRow(InternalRow(str, 1.toByte), 2)) } test("nullability of array type element should not fail analysis") { @@ -67,7 +67,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = 'a.array(IntegerType) :: Nil // It should pass analysis - val bound = encoder.resolve(attrs, null).bind(attrs) + val bound = encoder.resolveAndBind(attrs) // If no null values appear, it should works fine bound.fromRow(InternalRow(new GenericArrayData(Array(1, 2)))) @@ -84,20 +84,16 @@ class EncoderResolutionSuite extends PlanTest { { val attrs = Seq('a.string, 'b.long, 'c.int) - assert(intercept[AnalysisException](encoder.validate(attrs)).message == + assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + - "but failed as the number of fields does not line up.\n" + - " - Input schema: struct\n" + - " - Target schema: struct<_1:string,_2:bigint>") + "but failed as the number of fields does not line up.") } { val attrs = Seq('a.string) - assert(intercept[AnalysisException](encoder.validate(attrs)).message == + assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + - "but failed as the number of fields does not line up.\n" + - " - Input schema: struct\n" + - " - Target schema: struct<_1:string,_2:bigint>") + "but failed as the number of fields does not line up.") } } @@ -106,26 +102,22 @@ class EncoderResolutionSuite extends PlanTest { { val attrs = Seq('a.string, 'b.struct('x.long, 'y.string, 'z.int)) - assert(intercept[AnalysisException](encoder.validate(attrs)).message == + assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + - "but failed as the number of fields does not line up.\n" + - " - Input schema: struct>\n" + - " - Target schema: struct<_1:string,_2:struct<_1:bigint,_2:string>>") + "but failed as the number of fields does not line up.") } { val attrs = Seq('a.string, 'b.struct('x.long)) - assert(intercept[AnalysisException](encoder.validate(attrs)).message == + assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == "Try to map struct to Tuple2, " + - "but failed as the number of fields does not line up.\n" + - " - Input schema: struct>\n" + - " - Target schema: struct<_1:string,_2:struct<_1:bigint,_2:string>>") + "but failed as the number of fields does not line up.") } } test("throw exception if real type is not compatible with encoder schema") { val msg1 = intercept[AnalysisException] { - ExpressionEncoder[StringIntClass].resolve(Seq('a.string, 'b.long), null) + ExpressionEncoder[StringIntClass].resolveAndBind(Seq('a.string, 'b.long)) }.message assert(msg1 == s""" @@ -138,7 +130,7 @@ class EncoderResolutionSuite extends PlanTest { val msg2 = intercept[AnalysisException] { val structType = new StructType().add("a", StringType).add("b", DecimalType.SYSTEM_DEFAULT) - ExpressionEncoder[ComplexClass].resolve(Seq('a.long, 'b.struct(structType)), null) + ExpressionEncoder[ComplexClass].resolveAndBind(Seq('a.long, 'b.struct(structType))) }.message assert(msg2 == s""" @@ -171,7 +163,7 @@ class EncoderResolutionSuite extends PlanTest { val to = ExpressionEncoder[U] val catalystType = from.schema.head.dataType.simpleString test(s"cast from $catalystType to ${implicitly[TypeTag[U]].tpe} should success") { - to.resolve(from.schema.toAttributes, null) + to.resolveAndBind(from.schema.toAttributes) } } @@ -180,7 +172,7 @@ class EncoderResolutionSuite extends PlanTest { val to = ExpressionEncoder[U] val catalystType = from.schema.head.dataType.simpleString test(s"cast from $catalystType to ${implicitly[TypeTag[U]].tpe} should fail") { - intercept[AnalysisException](to.resolve(from.schema.toAttributes, null)) + intercept[AnalysisException](to.resolveAndBind(from.schema.toAttributes)) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 232dcc9ee51ca..a1f9259f139ed 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -27,6 +27,7 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.Encoders import org.apache.spark.sql.catalyst.{OptionalData, PrimitiveData} import org.apache.spark.sql.catalyst.analysis.AnalysisTest +import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} @@ -334,7 +335,7 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { val encoder = implicitly[ExpressionEncoder[T]] val row = encoder.toRow(input) val schema = encoder.schema.toAttributes - val boundEncoder = encoder.defaultBinding + val boundEncoder = encoder.resolveAndBind() val convertedBack = try boundEncoder.fromRow(row) catch { case e: Exception => fail( @@ -350,12 +351,8 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { } // Test the correct resolution of serialization / deserialization. - val attr = AttributeReference("obj", ObjectType(encoder.clsTag.runtimeClass))() - val inputPlan = LocalRelation(attr) - val plan = - Project(Alias(encoder.deserializer, "obj")() :: Nil, - Project(encoder.namedExpressions, - inputPlan)) + val attr = AttributeReference("obj", encoder.deserializer.dataType)() + val plan = LocalRelation(attr).serialize[T].deserialize[T] assertAnalysisSuccess(plan) val isCorrect = (input, convertedBack) match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 39fcc7225b37e..6f1bc80c1cdda 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -135,7 +135,7 @@ class RowEncoderSuite extends SparkFunSuite { .add("string", StringType) .add("double", DoubleType)) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input: Row = Row((100, "test", 0.123)) val row = encoder.toRow(input) @@ -152,7 +152,7 @@ class RowEncoderSuite extends SparkFunSuite { .add("scala_decimal", DecimalType.SYSTEM_DEFAULT) .add("catalyst_decimal", DecimalType.SYSTEM_DEFAULT) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val javaDecimal = new java.math.BigDecimal("1234.5678") val scalaDecimal = BigDecimal("1234.5678") @@ -169,7 +169,7 @@ class RowEncoderSuite extends SparkFunSuite { test("RowEncoder should preserve decimal precision and scale") { val schema = new StructType().add("decimal", DecimalType(10, 5), false) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val decimal = Decimal("67123.45") val input = Row(decimal) val row = encoder.toRow(input) @@ -179,7 +179,7 @@ class RowEncoderSuite extends SparkFunSuite { test("RowEncoder should preserve schema nullability") { val schema = new StructType().add("int", IntegerType, nullable = false) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.serializer.length == 1) assert(encoder.serializer.head.dataType == IntegerType) assert(encoder.serializer.head.nullable == false) @@ -195,7 +195,7 @@ class RowEncoderSuite extends SparkFunSuite { new StructType().add("int", IntegerType, nullable = false), nullable = false), nullable = false) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.serializer.length == 1) assert(encoder.serializer.head.dataType == new StructType() @@ -212,7 +212,7 @@ class RowEncoderSuite extends SparkFunSuite { .add("array", ArrayType(IntegerType)) .add("nestedArray", ArrayType(ArrayType(StringType))) .add("deepNestedArray", ArrayType(ArrayType(ArrayType(LongType)))) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input = Row( Array(1, 2, null), Array(Array("abc", null), null), @@ -226,7 +226,7 @@ class RowEncoderSuite extends SparkFunSuite { private def encodeDecodeTest(schema: StructType): Unit = { test(s"encode/decode: ${schema.simpleString}") { - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val inputGenerator = RandomDataGenerator.forType(schema, nullable = false).get var input: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 369b772d322c0..96c871d034355 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -192,24 +192,24 @@ class Dataset[T] private[sql]( } /** - * An unresolved version of the internal encoder for the type of this [[Dataset]]. This one is - * marked implicit so that we can use it when constructing new [[Dataset]] objects that have the - * same object type (that will be possibly resolved to a different schema). + * Currently [[ExpressionEncoder]] is the only implementation of [[Encoder]], here we turn the + * passed in encoder to [[ExpressionEncoder]] explicitly, and mark it implicit so that we can use + * it when constructing new [[Dataset]] objects that have the same object type (that will be + * possibly resolved to a different schema). */ - private[sql] implicit val unresolvedTEncoder: ExpressionEncoder[T] = encoderFor(encoder) - unresolvedTEncoder.validate(logicalPlan.output) - - /** The encoder for this [[Dataset]] that has been resolved to its output schema. */ - private[sql] val resolvedTEncoder: ExpressionEncoder[T] = - unresolvedTEncoder.resolve(logicalPlan.output, OuterScopes.outerScopes) + private[sql] implicit val exprEnc: ExpressionEncoder[T] = encoderFor(encoder) /** - * The encoder where the expressions used to construct an object from an input row have been - * bound to the ordinals of this [[Dataset]]'s output schema. + * Encoder is used mostly as a container of serde expressions in [[Dataset]]. We build logical + * plans by these serde expressions and execute it within the query framework. However, for + * performance reasons we may want to use encoder as a function to deserialize internal rows to + * custom objects, e.g. collect. Here we resolve and bind the encoder so that we can call its + * `fromRow` method later. */ - private[sql] val boundTEncoder = resolvedTEncoder.bind(logicalPlan.output) + private val boundEnc = + exprEnc.resolveAndBind(logicalPlan.output, sparkSession.sessionState.analyzer) - private implicit def classTag = unresolvedTEncoder.clsTag + private implicit def classTag = exprEnc.clsTag // sqlContext must be val because a stable identifier is expected when you import implicits @transient lazy val sqlContext: SQLContext = sparkSession.sqlContext @@ -761,7 +761,7 @@ class Dataset[T] private[sql]( // Note that we do this before joining them, to enable the join operator to return null for one // side, in cases like outer-join. val left = { - val combined = if (this.unresolvedTEncoder.flat) { + val combined = if (this.exprEnc.flat) { assert(joined.left.output.length == 1) Alias(joined.left.output.head, "_1")() } else { @@ -771,7 +771,7 @@ class Dataset[T] private[sql]( } val right = { - val combined = if (other.unresolvedTEncoder.flat) { + val combined = if (other.exprEnc.flat) { assert(joined.right.output.length == 1) Alias(joined.right.output.head, "_2")() } else { @@ -784,14 +784,14 @@ class Dataset[T] private[sql]( // combine the outputs of each join side. val conditionExpr = joined.condition.get transformUp { case a: Attribute if joined.left.outputSet.contains(a) => - if (this.unresolvedTEncoder.flat) { + if (this.exprEnc.flat) { left.output.head } else { val index = joined.left.output.indexWhere(_.exprId == a.exprId) GetStructField(left.output.head, index) } case a: Attribute if joined.right.outputSet.contains(a) => - if (other.unresolvedTEncoder.flat) { + if (other.exprEnc.flat) { right.output.head } else { val index = joined.right.output.indexWhere(_.exprId == a.exprId) @@ -800,7 +800,7 @@ class Dataset[T] private[sql]( } implicit val tuple2Encoder: Encoder[(T, U)] = - ExpressionEncoder.tuple(this.unresolvedTEncoder, other.unresolvedTEncoder) + ExpressionEncoder.tuple(this.exprEnc, other.exprEnc) withTypedPlan(Join(left, right, joined.joinType, Some(conditionExpr))) } @@ -1024,7 +1024,7 @@ class Dataset[T] private[sql]( sparkSession, Project( c1.withInputType( - unresolvedTEncoder.deserializer, + exprEnc.deserializer, logicalPlan.output).named :: Nil, logicalPlan), implicitly[Encoder[U1]]) @@ -1038,7 +1038,7 @@ class Dataset[T] private[sql]( protected def selectUntyped(columns: TypedColumn[_, _]*): Dataset[_] = { val encoders = columns.map(_.encoder) val namedColumns = - columns.map(_.withInputType(unresolvedTEncoder.deserializer, logicalPlan.output).named) + columns.map(_.withInputType(exprEnc.deserializer, logicalPlan.output).named) val execution = new QueryExecution(sparkSession, Project(namedColumns, logicalPlan)) new Dataset(sparkSession, execution, ExpressionEncoder.tuple(encoders)) } @@ -2153,14 +2153,14 @@ class Dataset[T] private[sql]( */ def collectAsList(): java.util.List[T] = withCallback("collectAsList", toDF()) { _ => withNewExecutionId { - val values = queryExecution.executedPlan.executeCollect().map(boundTEncoder.fromRow) + val values = queryExecution.executedPlan.executeCollect().map(boundEnc.fromRow) java.util.Arrays.asList(values : _*) } } private def collect(needCallback: Boolean): Array[T] = { def execute(): Array[T] = withNewExecutionId { - queryExecution.executedPlan.executeCollect().map(boundTEncoder.fromRow) + queryExecution.executedPlan.executeCollect().map(boundEnc.fromRow) } if (needCallback) { @@ -2184,7 +2184,7 @@ class Dataset[T] private[sql]( */ def toLocalIterator(): java.util.Iterator[T] = withCallback("toLocalIterator", toDF()) { _ => withNewExecutionId { - queryExecution.executedPlan.executeToIterator().map(boundTEncoder.fromRow).asJava + queryExecution.executedPlan.executeToIterator().map(boundEnc.fromRow).asJava } } @@ -2322,7 +2322,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ lazy val rdd: RDD[T] = { - val objectType = unresolvedTEncoder.deserializer.dataType + val objectType = exprEnc.deserializer.dataType val deserialized = CatalystSerde.deserialize[T](logicalPlan) sparkSession.sessionState.executePlan(deserialized).toRdd.mapPartitions { rows => rows.map(_.get(0, objectType).asInstanceOf[T]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 53f4ea647c853..a6867a67eeade 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -42,17 +42,9 @@ class KeyValueGroupedDataset[K, V] private[sql]( private val dataAttributes: Seq[Attribute], private val groupingAttributes: Seq[Attribute]) extends Serializable { - // Similar to [[Dataset]], we use unresolved encoders for later composition and resolved encoders - // when constructing new logical plans that will operate on the output of the current - // queryexecution. - - private implicit val unresolvedKEncoder = encoderFor(kEncoder) - private implicit val unresolvedVEncoder = encoderFor(vEncoder) - - private val resolvedKEncoder = - unresolvedKEncoder.resolve(groupingAttributes, OuterScopes.outerScopes) - private val resolvedVEncoder = - unresolvedVEncoder.resolve(dataAttributes, OuterScopes.outerScopes) + // Similar to [[Dataset]], we turn the passed in encoder to `ExpressionEncoder` explicitly. + private implicit val kExprEnc = encoderFor(kEncoder) + private implicit val vExprEnc = encoderFor(vEncoder) private def logicalPlan = queryExecution.analyzed private def sparkSession = queryExecution.sparkSession @@ -67,7 +59,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( def keyAs[L : Encoder]: KeyValueGroupedDataset[L, V] = new KeyValueGroupedDataset( encoderFor[L], - unresolvedVEncoder, + vExprEnc, queryExecution, dataAttributes, groupingAttributes) @@ -187,7 +179,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( def reduceGroups(f: (V, V) => V): Dataset[(K, V)] = { val func = (key: K, it: Iterator[V]) => Iterator((key, it.reduce(f))) - implicit val resultEncoder = ExpressionEncoder.tuple(unresolvedKEncoder, unresolvedVEncoder) + implicit val resultEncoder = ExpressionEncoder.tuple(kExprEnc, vExprEnc) flatMapGroups(func) } @@ -209,8 +201,8 @@ class KeyValueGroupedDataset[K, V] private[sql]( protected def aggUntyped(columns: TypedColumn[_, _]*): Dataset[_] = { val encoders = columns.map(_.encoder) val namedColumns = - columns.map(_.withInputType(unresolvedVEncoder.deserializer, dataAttributes).named) - val keyColumn = if (resolvedKEncoder.flat) { + columns.map(_.withInputType(vExprEnc.deserializer, dataAttributes).named) + val keyColumn = if (kExprEnc.flat) { assert(groupingAttributes.length == 1) groupingAttributes.head } else { @@ -222,7 +214,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( new Dataset( sparkSession, execution, - ExpressionEncoder.tuple(unresolvedKEncoder +: encoders)) + ExpressionEncoder.tuple(kExprEnc +: encoders)) } /** @@ -287,7 +279,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( def cogroup[U, R : Encoder]( other: KeyValueGroupedDataset[K, U])( f: (K, Iterator[V], Iterator[U]) => TraversableOnce[R]): Dataset[R] = { - implicit val uEncoder = other.unresolvedVEncoder + implicit val uEncoder = other.vExprEnc Dataset[R]( sparkSession, CoGroup( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 58850a7d4b49f..49b6eab8db5b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -215,7 +215,7 @@ class RelationalGroupedDataset protected[sql]( def agg(expr: Column, exprs: Column*): DataFrame = { toDF((expr +: exprs).map { case typed: TypedColumn[_, _] => - typed.withInputType(df.unresolvedTEncoder.deserializer, df.logicalPlan.output).expr + typed.withInputType(df.exprEnc.deserializer, df.logicalPlan.output).expr case c => c.expr }) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 8f94184764c0f..ecb56e2a2848a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -33,9 +33,9 @@ object TypedAggregateExpression { aggregator: Aggregator[_, BUF, OUT]): TypedAggregateExpression = { val bufferEncoder = encoderFor[BUF] val bufferSerializer = bufferEncoder.namedExpressions - val bufferDeserializer = bufferEncoder.deserializer.transform { - case b: BoundReference => bufferSerializer(b.ordinal).toAttribute - } + val bufferDeserializer = UnresolvedDeserializer( + bufferEncoder.deserializer, + bufferSerializer.map(_.toAttribute)) val outputEncoder = encoderFor[OUT] val outputType = if (outputEncoder.flat) { 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 d89e98645b367..4dbd1665e4bbc 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 @@ -924,7 +924,7 @@ object functions { * @since 1.5.0 */ def broadcast[T](df: Dataset[T]): Dataset[T] = { - Dataset[T](df.sparkSession, BroadcastHint(df.logicalPlan))(df.unresolvedTEncoder) + Dataset[T](df.sparkSession, BroadcastHint(df.logicalPlan))(df.exprEnc) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index df8f4b0610af2..d1c232974e9ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -566,18 +566,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { }.message assert(message == "Try to map struct to Tuple3, " + - "but failed as the number of fields does not line up.\n" + - " - Input schema: struct\n" + - " - Target schema: struct<_1:string,_2:int,_3:bigint>") + "but failed as the number of fields does not line up.") val message2 = intercept[AnalysisException] { ds.as[Tuple1[String]] }.message assert(message2 == "Try to map struct to Tuple1, " + - "but failed as the number of fields does not line up.\n" + - " - Input schema: struct\n" + - " - Target schema: struct<_1:string>") + "but failed as the number of fields does not line up.") } test("SPARK-13440: Resolving option fields") { 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 a1a9b66c1feec..9c044f4e8fd6e 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 @@ -81,7 +81,7 @@ abstract class QueryTest extends PlanTest { expectedAnswer: T*): Unit = { checkAnswer( ds.toDF(), - spark.createDataset(expectedAnswer)(ds.unresolvedTEncoder).toDF().collect().toSeq) + spark.createDataset(expectedAnswer)(ds.exprEnc).toDF().collect().toSeq) checkDecoding(ds, expectedAnswer: _*) } @@ -94,8 +94,8 @@ abstract class QueryTest extends PlanTest { fail( s""" |Exception collecting dataset as objects - |${ds.resolvedTEncoder} - |${ds.resolvedTEncoder.deserializer.treeString} + |${ds.exprEnc} + |${ds.exprEnc.deserializer.treeString} |${ds.queryExecution} """.stripMargin, e) } @@ -114,7 +114,7 @@ abstract class QueryTest extends PlanTest { fail( s"""Decoded objects do not match expected objects: |$comparison - |${ds.resolvedTEncoder.deserializer.treeString} + |${ds.exprEnc.deserializer.treeString} """.stripMargin) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala index 6f10e4b80577a..80340b5552c6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala @@ -27,7 +27,7 @@ class GroupedIteratorSuite extends SparkFunSuite { test("basic") { val schema = new StructType().add("i", IntegerType).add("s", StringType) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input = Seq(Row(1, "a"), Row(1, "b"), Row(2, "c")) val grouped = GroupedIterator(input.iterator.map(encoder.toRow), Seq('i.int.at(0)), schema.toAttributes) @@ -45,7 +45,7 @@ class GroupedIteratorSuite extends SparkFunSuite { test("group by 2 columns") { val schema = new StructType().add("i", IntegerType).add("l", LongType).add("s", StringType) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input = Seq( Row(1, 2L, "a"), @@ -72,7 +72,7 @@ class GroupedIteratorSuite extends SparkFunSuite { test("do nothing to the value iterator") { val schema = new StructType().add("i", IntegerType).add("s", StringType) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input = Seq(Row(1, "a"), Row(1, "b"), Row(2, "c")) val grouped = GroupedIterator(input.iterator.map(encoder.toRow), Seq('i.int.at(0)), schema.toAttributes) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index dd8672aa641d2..194c3e7307255 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -110,7 +110,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { object CheckAnswer { def apply[A : Encoder](data: A*): CheckAnswerRows = { val encoder = encoderFor[A] - val toExternalRow = RowEncoder(encoder.schema) + val toExternalRow = RowEncoder(encoder.schema).resolveAndBind() CheckAnswerRows(data.map(d => toExternalRow.fromRow(encoder.toRow(d))), false) } @@ -124,7 +124,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { object CheckLastBatch { def apply[A : Encoder](data: A*): CheckAnswerRows = { val encoder = encoderFor[A] - val toExternalRow = RowEncoder(encoder.schema) + val toExternalRow = RowEncoder(encoder.schema).resolveAndBind() CheckAnswerRows(data.map(d => toExternalRow.fromRow(encoder.toRow(d))), true) } From da29762f1f9df028886d741c66f601d70e797ee3 Mon Sep 17 00:00:00 2001 From: bomeng Date: Fri, 3 Jun 2016 09:59:15 -0700 Subject: [PATCH 0512/1470] [SPARK-15737][CORE] fix jetty warning ## What changes were proposed in this pull request? After upgrading Jetty to 9.2, we always see "WARN org.eclipse.jetty.server.handler.AbstractHandler: No Server set for org.eclipse.jetty.server.handler.ErrorHandler" while running any test cases. This PR will fix it. ## How was this patch tested? The existing test cases will cover it. Author: bomeng Closes #13475 from bomeng/SPARK-15737. (cherry picked from commit 8fa00dd05f3b5e78e9268715b0f05f9f739eca5c) Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/deploy/rest/RestSubmissionServer.scala | 1 + core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 7e93bfc45ebc9..fa55d470842b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -89,6 +89,7 @@ private[spark] abstract class RestSubmissionServer( server.addConnector(connector) val mainHandler = new ServletContextHandler + mainHandler.setServer(server) mainHandler.setContextPath("/") contextToServlet.foreach { case (prefix, servlet) => mainHandler.addServlet(new ServletHolder(servlet), prefix) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 6854f7baf716d..50283f2b74a41 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -291,6 +291,7 @@ private[spark] object JettyUtils extends Logging { val errorHandler = new ErrorHandler() errorHandler.setShowStacks(true) + errorHandler.setServer(server) server.addBean(errorHandler) server.setHandler(collection) try { From 396be560deda9ca3a00b57baa5757937969af4e6 Mon Sep 17 00:00:00 2001 From: Ioana Delaney Date: Fri, 3 Jun 2016 12:04:27 -0700 Subject: [PATCH 0513/1470] [SPARK-15677][SQL] Query with scalar sub-query in the SELECT list throws UnsupportedOperationException ## What changes were proposed in this pull request? Queries with scalar sub-query in the SELECT list run against a local, in-memory relation throw UnsupportedOperationException exception. Problem repro: ```SQL scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") scala> sql("select (select min(c1) from t2) from t1").show() java.lang.UnsupportedOperationException: Cannot evaluate expression: scalar-subquery#62 [] at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.eval(Expression.scala:215) at org.apache.spark.sql.catalyst.expressions.ScalarSubquery.eval(subquery.scala:62) at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142) at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:45) at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:29) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.immutable.List.map(List.scala:285) at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$37.applyOrElse(Optimizer.scala:1473) ``` The problem is specific to local, in memory relations. It is caused by rule ConvertToLocalRelation, which attempts to push down a scalar-subquery expression to the local tables. The solution prevents the rule to apply if Project references scalar subqueries. ## How was this patch tested? Added regression tests to SubquerySuite.scala Author: Ioana Delaney Closes #13418 from ioana-delaney/scalarSubV2. (cherry picked from commit 9e2eb13ca59fc7ac66c6accd49469f339700b23b) Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 7 ++++- .../org/apache/spark/sql/SubquerySuite.scala | 27 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) 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 93762ad1b91c2..e455d42dac96e 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 @@ -1472,10 +1472,15 @@ object DecimalAggregates extends Rule[LogicalPlan] { */ object ConvertToLocalRelation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case Project(projectList, LocalRelation(output, data)) => + case Project(projectList, LocalRelation(output, data)) + if !projectList.exists(hasUnevaluableExpr) => val projection = new InterpretedProjection(projectList, output) LocalRelation(projectList.map(_.toAttribute), data.map(projection)) } + + private def hasUnevaluableExpr(expr: Expression): Boolean = { + expr.find(e => e.isInstanceOf[Unevaluable] && !e.isInstanceOf[AttributeReference]).isDefined + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 48196927332f1..a932125f3cc6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -123,6 +123,33 @@ class SubquerySuite extends QueryTest with SharedSQLContext { ) } + test("SPARK-15677: Queries against local relations with scalar subquery in Select list") { + withTempTable("t1", "t2") { + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") + + checkAnswer( + sql("SELECT (select 1 as col) from t1"), + Row(1) :: Row(1) :: Nil) + + checkAnswer( + sql("SELECT (select max(c1) from t2) from t1"), + Row(2) :: Row(2) :: Nil) + + checkAnswer( + sql("SELECT 1 + (select 1 as col) from t1"), + Row(2) :: Row(2) :: Nil) + + checkAnswer( + sql("SELECT c1, (select max(c1) from t2) + c2 from t1"), + Row(1, 3) :: Row(2, 4) :: Nil) + + checkAnswer( + sql("SELECT c1, (select max(c1) from t2 where t1.c2 = t2.c2) from t1"), + Row(1, 1) :: Row(2, 2) :: Nil) + } + } + test("SPARK-14791: scalar subquery inside broadcast join") { val df = sql("select a, sum(b) as s from l group by a having a > (select avg(a) from l)") val expected = Row(3, 2.0, 3, 3.0) :: Row(6, null, 6, null) :: Nil From bcf0c51b6852a237a5b7dbe0dcb6904431668074 Mon Sep 17 00:00:00 2001 From: Devaraj K Date: Fri, 3 Jun 2016 13:03:28 -0700 Subject: [PATCH 0514/1470] [SPARK-15665][CORE] spark-submit --kill and --status are not working ## What changes were proposed in this pull request? --kill and --status were not considered while handling in OptionParser and due to that it was failing. Now handling the --kill and --status options as part of OptionParser.handle. ## How was this patch tested? Added a test org.apache.spark.launcher.SparkSubmitCommandBuilderSuite.testCliKillAndStatus() and also I have verified these manually by running --kill and --status commands. Author: Devaraj K Closes #13407 from devaraj-kavali/SPARK-15665. (cherry picked from commit efd3b11a47ec553f68ab6aa714167f3aee2ecf56) Signed-off-by: Marcelo Vanzin --- .../launcher/SparkSubmitCommandBuilder.java | 26 +++++++++++-------- .../SparkSubmitCommandBuilderSuite.java | 14 ++++++++++ 2 files changed, 29 insertions(+), 11 deletions(-) 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 824500d980eee..b3ccc4805f2c5 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -89,7 +89,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { } final List sparkArgs; - private final boolean printInfo; + private final boolean isAppResourceReq; private final boolean isExample; /** @@ -101,7 +101,7 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { SparkSubmitCommandBuilder() { this.sparkArgs = new ArrayList<>(); - this.printInfo = false; + this.isAppResourceReq = true; this.isExample = false; } @@ -133,19 +133,19 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { this.isExample = isExample; OptionParser parser = new OptionParser(); parser.parse(submitArgs); - this.printInfo = parser.infoRequested; + this.isAppResourceReq = parser.isAppResourceReq; } else { this.isExample = isExample; - this.printInfo = true; + this.isAppResourceReq = false; } } @Override public List buildCommand(Map env) throws IOException, IllegalArgumentException { - if (PYSPARK_SHELL.equals(appResource) && !printInfo) { + if (PYSPARK_SHELL.equals(appResource) && isAppResourceReq) { return buildPySparkShellCommand(env); - } else if (SPARKR_SHELL.equals(appResource) && !printInfo) { + } else if (SPARKR_SHELL.equals(appResource) && isAppResourceReq) { return buildSparkRCommand(env); } else { return buildSparkSubmitCommand(env); @@ -156,7 +156,7 @@ List buildSparkSubmitArgs() { List args = new ArrayList<>(); SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); - if (!allowsMixedArguments && !printInfo) { + if (!allowsMixedArguments && isAppResourceReq) { checkArgument(appResource != null, "Missing application resource."); } @@ -208,7 +208,7 @@ List buildSparkSubmitArgs() { args.add(join(",", pyFiles)); } - if (!printInfo) { + if (isAppResourceReq) { checkArgument(!isExample || mainClass != null, "Missing example class name."); } if (mainClass != null) { @@ -388,7 +388,7 @@ private List findExamplesJars() { private class OptionParser extends SparkSubmitOptionParser { - boolean infoRequested = false; + boolean isAppResourceReq = true; @Override protected boolean handle(String opt, String value) { @@ -420,11 +420,15 @@ protected boolean handle(String opt, String value) { allowsMixedArguments = true; appResource = specialClasses.get(value); } + } else if (opt.equals(KILL_SUBMISSION) || opt.equals(STATUS)) { + isAppResourceReq = false; + sparkArgs.add(opt); + sparkArgs.add(value); } else if (opt.equals(HELP) || opt.equals(USAGE_ERROR)) { - infoRequested = true; + isAppResourceReq = false; sparkArgs.add(opt); } else if (opt.equals(VERSION)) { - infoRequested = true; + isAppResourceReq = false; sparkArgs.add(opt); } else { sparkArgs.add(opt); 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 d451651555768..16e5a22401ca8 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -72,6 +72,12 @@ public void testCliHelpAndNoArg() throws Exception { cmd.contains("org.apache.spark.deploy.SparkSubmit")); } + @Test + public void testCliKillAndStatus() throws Exception { + testCLIOpts(parser.STATUS); + testCLIOpts(parser.KILL_SUBMISSION); + } + @Test public void testCliParser() throws Exception { List sparkSubmitArgs = Arrays.asList( @@ -326,4 +332,12 @@ private List buildCommand(List args, Map env) th return newCommandBuilder(args).buildCommand(env); } + private void testCLIOpts(String opt) throws Exception { + List helpArgs = Arrays.asList(opt, "driver-20160531171222-0000"); + Map env = new HashMap<>(); + List cmd = buildCommand(helpArgs, env); + assertTrue(opt + " should be contained in the final cmd.", + cmd.contains(opt)); + } + } From 4018c48009ab479b996ba0ec0a3d0a47c97388ed Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 3 Jun 2016 13:53:02 -0700 Subject: [PATCH 0515/1470] [SPARK-15742][SQL] Reduce temp collections allocations in TreeNode transform methods In Catalyst's TreeNode transform methods we end up calling `productIterator.map(...).toArray` in a number of places, which is slightly inefficient because it needs to allocate an `ArrayBuilder` and grow a temporary array. Since we already know the size of the final output (`productArity`), we can simply allocate an array up-front and use a while loop to consume the iterator and populate the array. For most workloads, this performance difference is negligible but it does make a measurable difference in optimizer performance for queries that operate over very wide schemas (such as the benchmark queries in #13456). ### Perf results (from #13456 benchmarks) **Before** ``` Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Mac OS X 10.10.5 Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ 1 select expressions 19 / 22 0.0 19119858.0 1.0X 10 select expressions 23 / 25 0.0 23208774.0 0.8X 100 select expressions 55 / 73 0.0 54768402.0 0.3X 1000 select expressions 229 / 259 0.0 228606373.0 0.1X 2500 select expressions 530 / 554 0.0 529938178.0 0.0X ``` **After** ``` parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ 1 select expressions 15 / 21 0.0 14978203.0 1.0X 10 select expressions 22 / 27 0.0 22492262.0 0.7X 100 select expressions 48 / 64 0.0 48449834.0 0.3X 1000 select expressions 189 / 208 0.0 189346428.0 0.1X 2500 select expressions 429 / 449 0.0 428943897.0 0.0X ``` ### Author: Josh Rosen Closes #13484 from JoshRosen/treenode-productiterator-map. (cherry picked from commit e526913989d6099064886ea3ed3f6a2a0376a4f8) Signed-off-by: Josh Rosen --- .../spark/sql/catalyst/plans/QueryPlan.scala | 6 ++--- .../spark/sql/catalyst/trees/TreeNode.scala | 26 ++++++++++++++----- 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 6784c3ae1d7e7..3de15a9a3f544 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -172,7 +172,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT case null => null } - val newArgs = productIterator.map(recursiveTransform).toArray + val newArgs = mapProductIterator(recursiveTransform) if (changed) makeCopy(newArgs).asInstanceOf[this.type] else this } @@ -206,7 +206,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT case null => null } - val newArgs = productIterator.map(recursiveTransform).toArray + val newArgs = mapProductIterator(recursiveTransform) if (changed) makeCopy(newArgs).asInstanceOf[this.type] else this } @@ -318,7 +318,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT case other => other } - productIterator.map { + mapProductIterator { case s: Option[_] => s.map(cleanArg) case s: Seq[_] => s.map(cleanArg) case m: Map[_, _] => m.mapValues(cleanArg) 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 50481cd298e2a..22d82c61080c0 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 @@ -21,6 +21,7 @@ import java.util.UUID import scala.collection.Map import scala.collection.mutable.Stack +import scala.reflect.ClassTag import org.apache.commons.lang.ClassUtils import org.json4s.JsonAST._ @@ -168,12 +169,25 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } } + /** + * Efficient alternative to `productIterator.map(f).toArray`. + */ + protected def mapProductIterator[B: ClassTag](f: Any => B): Array[B] = { + val arr = Array.ofDim[B](productArity) + var i = 0 + while (i < arr.length) { + arr(i) = f(productElement(i)) + i += 1 + } + arr + } + /** * Returns a copy of this node where `f` has been applied to all the nodes children. */ def mapChildren(f: BaseType => BaseType): BaseType = { var changed = false - val newArgs = productIterator.map { + val newArgs = mapProductIterator { case arg: TreeNode[_] if containsChild(arg) => val newChild = f(arg.asInstanceOf[BaseType]) if (newChild fastEquals arg) { @@ -184,7 +198,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } case nonChild: AnyRef => nonChild case null => null - }.toArray + } if (changed) makeCopy(newArgs) else this } @@ -197,7 +211,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { var changed = false val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer - val newArgs = productIterator.map { + val newArgs = mapProductIterator { case s: StructType => s // Don't convert struct types to some other type of Seq[StructField] // Handle Seq[TreeNode] in TreeNode parameters. case s: Seq[_] => s.map { @@ -237,7 +251,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } case nonChild: AnyRef => nonChild case null => null - }.toArray + } if (changed) makeCopy(newArgs) else this } @@ -302,7 +316,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { rule: PartialFunction[BaseType, BaseType], nextOperation: (BaseType, PartialFunction[BaseType, BaseType]) => BaseType): BaseType = { var changed = false - val newArgs = productIterator.map { + val newArgs = mapProductIterator { case arg: TreeNode[_] if containsChild(arg) => val newChild = nextOperation(arg.asInstanceOf[BaseType], rule) if (!(newChild fastEquals arg)) { @@ -353,7 +367,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } case nonChild: AnyRef => nonChild case null => null - }.toArray + } if (changed) makeCopy(newArgs) else this } From 35765a0e9cddaf7efb81532570ef1f7b64ed99ab Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 3 Jun 2016 13:56:22 -0700 Subject: [PATCH 0516/1470] [SPARK-15286][SQL] Make the output readable for EXPLAIN CREATE TABLE and DESC EXTENDED #### What changes were proposed in this pull request? Before this PR, the output of EXPLAIN of following SQL is like ```SQL CREATE EXTERNAL TABLE extTable_with_partitions (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) LOCATION '/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-b39a6185-8981-403b-a4aa-36fb2f4ca8a9' ``` ``ExecutedCommand CreateTableCommand CatalogTable(`extTable_with_partitions`,CatalogTableType(EXTERNAL),CatalogStorageFormat(Some(/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-dd234718-e85d-4c5a-8353-8f1834ac0323),Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(key,int,true,None), CatalogColumn(value,string,true,None), CatalogColumn(ds,string,true,None), CatalogColumn(hr,string,true,None)),List(ds, hr),List(),List(),-1,,1463026413544,-1,Map(),None,None,None), false`` After this PR, the output is like ``` ExecutedCommand : +- CreateTableCommand CatalogTable( Table:`extTable_with_partitions` Created:Thu Jun 02 21:30:54 PDT 2016 Last Access:Wed Dec 31 15:59:59 PST 1969 Type:EXTERNAL Schema:[`key` int, `value` string, `ds` string, `hr` string] Partition Columns:[`ds`, `hr`] Storage(Location:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-a06083b8-8e88-4d07-9ff0-d6bd8d943ad3, InputFormat:org.apache.hadoop.mapred.TextInputFormat, OutputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat)), false ``` This is also applicable to `DESC EXTENDED`. However, this does not have special handling for Data Source Tables. If needed, we need to move the logics of `DDLUtil`. Let me know if we should do it in this PR. Thanks! rxin liancheng #### How was this patch tested? Manual testing Author: gatorsmile Closes #13070 from gatorsmile/betterExplainCatalogTable. (cherry picked from commit eb10b481ca51a3e010c168a1fb0baf1a94dcadfa) Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/interface.scala | 60 ++++++++++++++++++- 1 file changed, 58 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index b59195770c17e..6197acab33786 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.catalog +import java.util.Date import javax.annotation.Nullable import org.apache.spark.sql.AnalysisException @@ -48,7 +49,26 @@ case class CatalogStorageFormat( outputFormat: Option[String], serde: Option[String], compressed: Boolean, - serdeProperties: Map[String, String]) + serdeProperties: Map[String, String]) { + + override def toString: String = { + val serdePropsToString = + if (serdeProperties.nonEmpty) { + s"Properties: " + serdeProperties.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") + } else { + "" + } + val output = + Seq(locationUri.map("Location: " + _).getOrElse(""), + inputFormat.map("InputFormat: " + _).getOrElse(""), + outputFormat.map("OutputFormat: " + _).getOrElse(""), + if (compressed) "Compressed" else "", + serde.map("Serde: " + _).getOrElse(""), + serdePropsToString) + output.filter(_.nonEmpty).mkString("Storage(", ", ", ")") + } + +} object CatalogStorageFormat { /** Empty storage format for default values and copies. */ @@ -65,8 +85,18 @@ case class CatalogColumn( // as a string due to issues in converting Hive varchars to and from SparkSQL strings. @Nullable dataType: String, nullable: Boolean = true, - comment: Option[String] = None) + comment: Option[String] = None) { + + override def toString: String = { + val output = + Seq(s"`$name`", + dataType, + if (!nullable) "NOT NULL" else "", + comment.map("(" + _ + ")").getOrElse("")) + output.filter(_.nonEmpty).mkString(" ") + } +} /** * A partition (Hive style) defined in the catalog. @@ -140,6 +170,32 @@ case class CatalogTable( locationUri, inputFormat, outputFormat, serde, compressed, serdeProperties)) } + override def toString: String = { + val tableProperties = properties.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") + val partitionColumns = partitionColumnNames.map("`" + _ + "`").mkString("[", ", ", "]") + val sortColumns = sortColumnNames.map("`" + _ + "`").mkString("[", ", ", "]") + val bucketColumns = bucketColumnNames.map("`" + _ + "`").mkString("[", ", ", "]") + + val output = + Seq(s"Table: ${identifier.quotedString}", + if (owner.nonEmpty) s"Owner: $owner" else "", + s"Created: ${new Date(createTime).toString}", + s"Last Access: ${new Date(lastAccessTime).toString}", + s"Type: ${tableType.name}", + if (schema.nonEmpty) s"Schema: ${schema.mkString("[", ", ", "]")}" else "", + if (partitionColumnNames.nonEmpty) s"Partition Columns: $partitionColumns" else "", + if (numBuckets != -1) s"Num Buckets: $numBuckets" else "", + if (bucketColumnNames.nonEmpty) s"Bucket Columns: $bucketColumns" else "", + if (sortColumnNames.nonEmpty) s"Sort Columns: $sortColumns" else "", + viewOriginalText.map("Original View: " + _).getOrElse(""), + viewText.map("View: " + _).getOrElse(""), + comment.map("Comment: " + _).getOrElse(""), + if (properties.nonEmpty) s"Properties: $tableProperties" else "", + s"$storage") + + output.filter(_.nonEmpty).mkString("CatalogTable(\n\t", "\n\t", ")") + } + } From 52376e0678aab544d816850befdb69e6cbd05176 Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Fri, 3 Jun 2016 14:26:48 -0700 Subject: [PATCH 0517/1470] [SPARK-15681][CORE] allow lowercase or mixed case log level string when calling sc.setLogLevel ## What changes were proposed in this pull request? Currently `SparkContext API setLogLevel(level: String) `can not handle lower case or mixed case input string. But `org.apache.log4j.Level.toLevel` can take lowercase or mixed case. This PR is to allow case-insensitive user input for the log level. ## How was this patch tested? A unit testcase is added. Author: Xin Wu Closes #13422 from xwu0226/reset_loglevel. (cherry picked from commit 28ad0f7b0dc7bf24fac251c4f131aca74ba1c1d2) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/SparkContext.scala | 16 +++++++++------- .../org/apache/spark/SparkContextSuite.scala | 15 +++++++++++++++ 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5aba2a8c94691..33b11ed2e658f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -20,7 +20,7 @@ package org.apache.spark import java.io._ import java.lang.reflect.Constructor import java.net.URI -import java.util.{Arrays, Properties, ServiceLoader, UUID} +import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID} import java.util.concurrent.ConcurrentMap import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} @@ -356,12 +356,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN */ def setLogLevel(logLevel: String) { - val validLevels = Seq("ALL", "DEBUG", "ERROR", "FATAL", "INFO", "OFF", "TRACE", "WARN") - if (!validLevels.contains(logLevel)) { - throw new IllegalArgumentException( - s"Supplied level $logLevel did not match one of: ${validLevels.mkString(",")}") - } - Utils.setLogLevel(org.apache.log4j.Level.toLevel(logLevel)) + // let's allow lowcase or mixed case too + val upperCased = logLevel.toUpperCase(Locale.ENGLISH) + require(SparkContext.VALID_LOG_LEVELS.contains(upperCased), + s"Supplied level $logLevel did not match one of:" + + s" ${SparkContext.VALID_LOG_LEVELS.mkString(",")}") + Utils.setLogLevel(org.apache.log4j.Level.toLevel(upperCased)) } try { @@ -2179,6 +2179,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * various Spark features. */ object SparkContext extends Logging { + private val VALID_LOG_LEVELS = + Set("ALL", "DEBUG", "ERROR", "FATAL", "INFO", "OFF", "TRACE", "WARN") /** * Lock that guards access to global variables that track SparkContext construction. diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index ae665138b98d0..4fa3cab18184c 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -363,4 +363,19 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { sc.stop() assert(result == null) } + + test("log level case-insensitive and reset log level") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + val originalLevel = org.apache.log4j.Logger.getRootLogger().getLevel + try { + sc.setLogLevel("debug") + assert(org.apache.log4j.Logger.getRootLogger().getLevel === org.apache.log4j.Level.DEBUG) + sc.setLogLevel("INfo") + assert(org.apache.log4j.Logger.getRootLogger().getLevel === org.apache.log4j.Level.INFO) + } finally { + sc.setLogLevel(originalLevel.toString) + assert(org.apache.log4j.Logger.getRootLogger().getLevel === originalLevel) + sc.stop() + } + } } From 7315acf896b2474a4b7513434f5ba2faf468abd9 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 3 Jun 2016 14:28:19 -0700 Subject: [PATCH 0518/1470] [SPARK-15140][SQL] make the semantics of null input object for encoder clear ## What changes were proposed in this pull request? For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow row to be null, only its columns can be null. This PR explicitly add this constraint and throw exception if users break it. ## How was this patch tested? several new tests Author: Wenchen Fan Closes #13469 from cloud-fan/null-object. (cherry picked from commit 11c83f83d5172167cb64513d5311b4178797d40e) Signed-off-by: Cheng Lian --- .../sql/catalyst/encoders/ExpressionEncoder.scala | 13 ++++++++++--- .../spark/sql/catalyst/encoders/RowEncoder.scala | 7 +++---- .../sql/catalyst/expressions/objects/objects.scala | 4 ++-- .../sql/catalyst/encoders/RowEncoderSuite.scala | 8 ++++++++ .../scala/org/apache/spark/sql/DatasetSuite.scala | 10 ++++++++++ 5 files changed, 33 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index cc59d06fa3518..688082dcce538 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, JavaTypeInference, ScalaRefle import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection} -import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, NewInstance} +import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, Invoke, NewInstance} import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LocalRelation} import org.apache.spark.sql.types.{ObjectType, StructField, StructType} @@ -50,8 +50,15 @@ object ExpressionEncoder { val cls = mirror.runtimeClass(tpe) val flat = !ScalaReflection.definedByConstructorParams(tpe) - val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], nullable = false) - val serializer = ScalaReflection.serializerFor[T](inputObject) + val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], nullable = true) + val nullSafeInput = if (flat) { + inputObject + } else { + // For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL + // doesn't allow top-level row to be null, only its columns can be null. + AssertNotNull(inputObject, Seq("top level non-flat input object")) + } + val serializer = ScalaReflection.serializerFor[T](nullSafeInput) val deserializer = ScalaReflection.deserializerFor[T] val schema = ScalaReflection.schemaFor[T] match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 3c6ae1c5ccd8a..6cd7b34ceb88c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -57,8 +57,8 @@ import org.apache.spark.unsafe.types.UTF8String object RowEncoder { def apply(schema: StructType): ExpressionEncoder[Row] = { val cls = classOf[Row] - val inputObject = BoundReference(0, ObjectType(cls), nullable = false) - val serializer = serializerFor(inputObject, schema) + val inputObject = BoundReference(0, ObjectType(cls), nullable = true) + val serializer = serializerFor(AssertNotNull(inputObject, Seq("top level row object")), schema) val deserializer = deserializerFor(schema) new ExpressionEncoder[Row]( schema, @@ -153,8 +153,7 @@ object RowEncoder { val fieldValue = serializerFor( GetExternalRowField( inputObject, index, field.name, externalDataTypeForInput(field.dataType)), - field.dataType - ) + field.dataType) val convertedField = if (field.nullable) { If( Invoke(inputObject, "isNullAt", BooleanType, Literal(index) :: Nil), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index c2e3ab82ff16c..d4c71bffe86bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -519,7 +519,7 @@ case class CreateExternalRow(children: Seq[Expression], schema: StructType) val code = s""" $values = new Object[${children.size}]; $childrenCode - final ${classOf[Row].getName} ${ev.value} = new $rowClass($values, this.$schemaField); + final ${classOf[Row].getName} ${ev.value} = new $rowClass($values, $schemaField); """ ev.copy(code = code, isNull = "false") } @@ -675,7 +675,7 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) ${childGen.code} if (${childGen.isNull}) { - throw new RuntimeException(this.$errMsgField); + throw new RuntimeException($errMsgField); } """ ev.copy(code = code, isNull = "false", value = childGen.value) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 6f1bc80c1cdda..16abde064fc44 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -224,6 +224,14 @@ class RowEncoderSuite extends SparkFunSuite { assert(convertedBack.getSeq(2) == Seq(Seq(Seq(0L, null), null), null)) } + test("RowEncoder should throw RuntimeException if input row object is null") { + val schema = new StructType().add("int", IntegerType) + val encoder = RowEncoder(schema) + val e = intercept[RuntimeException](encoder.toRow(null)) + assert(e.getMessage.contains("Null value appeared in non-nullable field")) + assert(e.getMessage.contains("top level row object")) + } + private def encodeDecodeTest(schema: StructType): Unit = { test(s"encode/decode: ${schema.simpleString}") { val encoder = RowEncoder(schema).resolveAndBind() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index d1c232974e9ce..bf2b0a2c7c1b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -790,6 +790,16 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(e.getMessage.contains( "`abstract` is a reserved keyword and cannot be used as field name")) } + + test("Dataset should support flat input object to be null") { + checkDataset(Seq("a", null).toDS(), "a", null) + } + + test("Dataset should throw RuntimeException if non-flat input object is null") { + val e = intercept[RuntimeException](Seq(ClassData("a", 1), null).toDS()) + assert(e.getMessage.contains("Null value appeared in non-nullable field")) + assert(e.getMessage.contains("top level non-flat input object")) + } } case class Generic[T](id: T, value: Double) From 3670b2c64a76d44310f330be84d915623865fbc9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 3 Jun 2016 14:39:41 -0700 Subject: [PATCH 0519/1470] [SPARK-15722][SQL] Disallow specifying schema in CTAS statement ## What changes were proposed in this pull request? As of this patch, the following throws an exception because the schemas may not match: ``` CREATE TABLE students (age INT, name STRING) AS SELECT * FROM boxes ``` but this is OK: ``` CREATE TABLE students AS SELECT * FROM boxes ``` ## How was this patch tested? SQLQuerySuite, HiveDDLCommandSuite Author: Andrew Or Closes #13490 from andrewor14/ctas-no-column. (cherry picked from commit b1cc7da3e3c36cee59825e7b95e14b00ab1fe196) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/SparkSqlParser.scala | 7 ++++ .../sql/catalyst/LogicalPlanToSQLSuite.scala | 14 ++------ .../spark/sql/hive/HiveDDLCommandSuite.scala | 33 ++++--------------- .../sql/hive/execution/HiveUDFSuite.scala | 5 +-- .../sql/hive/execution/SQLQuerySuite.scala | 14 ++++---- .../sql/hive/execution/SQLViewSuite.scala | 10 ++---- 6 files changed, 25 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8ffc55668ae90..c68c8f80f87ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -25,6 +25,7 @@ import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ @@ -939,6 +940,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { selectQuery match { case Some(q) => + // Just use whatever is projected in the select statement as our schema + if (schema.nonEmpty) { + throw operationNotAllowed( + "Schema may not be specified in a Create Table As Select (CTAS) statement", + ctx) + } // Hive does not allow to use a CTAS statement to create a partitioned table. if (tableDesc.partitionColumnNames.nonEmpty) { val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 4315197e125f4..0827b04252bc4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -744,24 +744,14 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { test("SPARK-14933 - select parquet table") { withTable("parquet_t") { - sql( - """ - |create table parquet_t (c1 int, c2 string) - |stored as parquet select 1, 'abc' - """.stripMargin) - + sql("create table parquet_t stored as parquet as select 1 as c1, 'abc' as c2") checkHiveQl("select * from parquet_t") } } test("SPARK-14933 - select orc table") { withTable("orc_t") { - sql( - """ - |create table orc_t (c1 int, c2 string) - |stored as orc select 1, 'abc' - """.stripMargin) - + sql("create table orc_t stored as orc as select 1 as c1, 'abc' as c2") checkHiveQl("select * from orc_t") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index ba9fe54db86ee..867aadb5f5569 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -51,12 +51,6 @@ class HiveDDLCommandSuite extends PlanTest { test("Test CTAS #1") { val s1 = """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view - |(viewTime INT, - |userid BIGINT, - |page_url STRING, - |referrer_url STRING, - |ip STRING COMMENT 'IP Address of the User', - |country STRING COMMENT 'country of origination') |COMMENT 'This is the staging page view table' |STORED AS RCFILE |LOCATION '/user/external/page_view' @@ -69,13 +63,7 @@ class HiveDDLCommandSuite extends PlanTest { assert(desc.identifier.table == "page_view") assert(desc.tableType == CatalogTableType.EXTERNAL) assert(desc.storage.locationUri == Some("/user/external/page_view")) - assert(desc.schema == - CatalogColumn("viewtime", "int") :: - CatalogColumn("userid", "bigint") :: - CatalogColumn("page_url", "string") :: - CatalogColumn("referrer_url", "string") :: - CatalogColumn("ip", "string", comment = Some("IP Address of the User")) :: - CatalogColumn("country", "string", comment = Some("country of origination")) :: Nil) + assert(desc.schema.isEmpty) // will be populated later when the table is actually created assert(desc.comment == Some("This is the staging page view table")) // TODO will be SQLText assert(desc.viewText.isEmpty) @@ -91,12 +79,6 @@ class HiveDDLCommandSuite extends PlanTest { test("Test CTAS #2") { val s2 = """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view - |(viewTime INT, - |userid BIGINT, - |page_url STRING, - |referrer_url STRING, - |ip STRING COMMENT 'IP Address of the User', - |country STRING COMMENT 'country of origination') |COMMENT 'This is the staging page view table' |ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe' | STORED AS @@ -112,13 +94,7 @@ class HiveDDLCommandSuite extends PlanTest { assert(desc.identifier.table == "page_view") assert(desc.tableType == CatalogTableType.EXTERNAL) assert(desc.storage.locationUri == Some("/user/external/page_view")) - assert(desc.schema == - CatalogColumn("viewtime", "int") :: - CatalogColumn("userid", "bigint") :: - CatalogColumn("page_url", "string") :: - CatalogColumn("referrer_url", "string") :: - CatalogColumn("ip", "string", comment = Some("IP Address of the User")) :: - CatalogColumn("country", "string", comment = Some("country of origination")) :: Nil) + assert(desc.schema.isEmpty) // will be populated later when the table is actually created // TODO will be SQLText assert(desc.comment == Some("This is the staging page view table")) assert(desc.viewText.isEmpty) @@ -190,6 +166,11 @@ class HiveDDLCommandSuite extends PlanTest { " AS SELECT key, value FROM (SELECT 1 as key, 2 as value) tmp") } + test("CTAS statement with schema") { + assertUnsupported(s"CREATE TABLE ctas1 (age INT, name STRING) AS SELECT * FROM src") + assertUnsupported(s"CREATE TABLE ctas1 (age INT, name STRING) AS SELECT 1, 'hello'") + } + test("unsupported operations") { intercept[ParseException] { parser.parsePlan( 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 ffeed63695dfb..0f56b2c0d1f43 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 @@ -435,10 +435,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } // Non-External parquet pointing to /tmp/... - - sql("CREATE TABLE parquet_tmp(c1 int, c2 int) " + - " STORED AS parquet " + - " AS SELECT 1, 2") + sql("CREATE TABLE parquet_tmp STORED AS parquet AS SELECT 1, 2") val answer4 = sql("SELECT input_file_name() as file FROM parquet_tmp").head().getString(0) 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 499819f32b434..8244ff4ce0552 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 @@ -608,17 +608,15 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("specifying the column list for CTAS") { Seq((1, "111111"), (2, "222222")).toDF("key", "value").createOrReplaceTempView("mytable1") - sql("create table gen__tmp(a int, b string) as select key, value from mytable1") + sql("create table gen__tmp as select key as a, value as b from mytable1") checkAnswer( sql("SELECT a, b from gen__tmp"), sql("select key, value from mytable1").collect()) sql("DROP TABLE gen__tmp") - sql("create table gen__tmp(a double, b double) as select key, value from mytable1") - checkAnswer( - sql("SELECT a, b from gen__tmp"), - sql("select cast(key as double), cast(value as double) from mytable1").collect()) - sql("DROP TABLE gen__tmp") + intercept[AnalysisException] { + sql("create table gen__tmp(a int, b string) as select key, value from mytable1") + } sql("drop table mytable1") } @@ -1225,8 +1223,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-10741: Sort on Aggregate using parquet") { withTable("test10741") { withTempTable("src") { - Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF().createOrReplaceTempView("src") - sql("CREATE TABLE test10741(c1 STRING, c2 INT) STORED AS PARQUET AS SELECT * FROM src") + Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF("c1", "c2").createOrReplaceTempView("src") + sql("CREATE TABLE test10741 STORED AS PARQUET AS SELECT * FROM src") } checkAnswer(sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 72db3618e0870..39846f145c42c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -308,10 +308,7 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-14933 - create view from hive parquet tabale") { withTable("t_part") { withView("v_part") { - spark.sql( - """create table t_part (c1 int, c2 int) - |stored as parquet as select 1 as a, 2 as b - """.stripMargin) + spark.sql("create table t_part stored as parquet as select 1 as a, 2 as b") spark.sql("create view v_part as select * from t_part") checkAnswer( sql("select * from t_part"), @@ -323,10 +320,7 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-14933 - create view from hive orc tabale") { withTable("t_orc") { withView("v_orc") { - spark.sql( - """create table t_orc (c1 int, c2 int) - |stored as orc as select 1 as a, 2 as b - """.stripMargin) + spark.sql("create table t_orc stored as orc as select 1 as a, 2 as b") spark.sql("create view v_orc as select * from t_orc") checkAnswer( sql("select * from t_orc"), From f702e9941f27826fe78fa6dc269f7653a2153aff Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 3 Jun 2016 15:56:17 -0700 Subject: [PATCH 0520/1470] [SPARK-15168][PYSPARK][ML] Add missing params to MultilayerPerceptronClassifier ## What changes were proposed in this pull request? MultilayerPerceptronClassifier is missing step size, solver, and weights. Add these params. Also clarify the scaladoc a bit while we are updating these params. Eventually we should follow up and unify the HasSolver params (filed https://issues.apache.org/jira/browse/SPARK-15169 ) ## How was this patch tested? Doc tests Author: Holden Karau Closes #12943 from holdenk/SPARK-15168-add-missing-params-to-MultilayerPerceptronClassifier. (cherry picked from commit 67cc89ff028324ba4a7a7d9c19a268b9afea0031) Signed-off-by: Nick Pentreath --- python/pyspark/ml/classification.py | 75 +++++++++++++++++++++++++---- 1 file changed, 66 insertions(+), 9 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 177cf9d72cfb6..7710fdf2e216e 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1041,7 +1041,8 @@ def theta(self): @inherit_doc class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasMaxIter, HasTol, HasSeed, JavaMLWritable, JavaMLReadable): + HasMaxIter, HasTol, HasSeed, HasStepSize, JavaMLWritable, + JavaMLReadable): """ Classifier trainer based on the Multilayer Perceptron. Each layer has sigmoid activation function, output layer has softmax. @@ -1054,12 +1055,12 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, ... (1.0, Vectors.dense([0.0, 1.0])), ... (1.0, Vectors.dense([1.0, 0.0])), ... (0.0, Vectors.dense([1.0, 1.0]))], ["label", "features"]) - >>> mlp = MultilayerPerceptronClassifier(maxIter=100, layers=[2, 5, 2], blockSize=1, seed=123) + >>> mlp = MultilayerPerceptronClassifier(maxIter=100, layers=[2, 2, 2], blockSize=1, seed=123) >>> model = mlp.fit(df) >>> model.layers - [2, 5, 2] + [2, 2, 2] >>> model.weights.size - 27 + 12 >>> testDF = spark.createDataFrame([ ... (Vectors.dense([1.0, 0.0]),), ... (Vectors.dense([0.0, 0.0]),)], ["features"]) @@ -1083,6 +1084,12 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, True >>> model.weights == model2.weights True + >>> mlp2 = mlp2.setInitialWeights(list(range(0, 12))) + >>> model3 = mlp2.fit(df) + >>> model3.weights != model2.weights + True + >>> model3.layers == model.layers + True .. versionadded:: 1.6.0 """ @@ -1096,28 +1103,36 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, "remaining data in a partition then it is adjusted to the size of this " + "data. Recommended size is between 10 and 1000, default is 128.", typeConverter=TypeConverters.toInt) + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: l-bfgs, gd.", typeConverter=TypeConverters.toString) + initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", + typeConverter=TypeConverters.toVector) @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", - maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128): + maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128, stepSize=0.03, + solver="l-bfgs", initialWeights=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ - maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128) + maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128, stepSize=0.03, \ + solver="l-bfgs", initialWeights=None) """ super(MultilayerPerceptronClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.MultilayerPerceptronClassifier", self.uid) - self._setDefault(maxIter=100, tol=1E-4, blockSize=128) + self._setDefault(maxIter=100, tol=1E-4, blockSize=128, stepSize=0.03, solver="l-bfgs") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("1.6.0") def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", - maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128): + maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128, stepSize=0.03, + solver="l-bfgs", initialWeights=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ - maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128) + maxIter=100, tol=1e-4, seed=None, layers=None, blockSize=128, stepSize=0.03, \ + solver="l-bfgs", initialWeights=None) Sets params for MultilayerPerceptronClassifier. """ kwargs = self.setParams._input_kwargs @@ -1154,6 +1169,48 @@ def getBlockSize(self): """ return self.getOrDefault(self.blockSize) + @since("2.0.0") + def setStepSize(self, value): + """ + Sets the value of :py:attr:`stepSize`. + """ + return self._set(stepSize=value) + + @since("2.0.0") + def getStepSize(self): + """ + Gets the value of stepSize or its default value. + """ + return self.getOrDefault(self.stepSize) + + @since("2.0.0") + def setSolver(self, value): + """ + Sets the value of :py:attr:`solver`. + """ + return self._set(solver=value) + + @since("2.0.0") + def getSolver(self): + """ + Gets the value of solver or its default value. + """ + return self.getOrDefault(self.solver) + + @since("2.0.0") + def setInitialWeights(self, value): + """ + Sets the value of :py:attr:`initialWeights`. + """ + return self._set(initialWeights=value) + + @since("2.0.0") + def getInitialWeights(self): + """ + Gets the value of initialWeights or its default value. + """ + return self.getOrDefault(self.initialWeights) + class MultilayerPerceptronClassificationModel(JavaModel, JavaMLWritable, JavaMLReadable): """ From 849fe3534d4ce7af44e9327085a3ef4787ded088 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 3 Jun 2016 16:45:09 -0700 Subject: [PATCH 0521/1470] [SPARK-15391] [SQL] manage the temporary memory of timsort ## What changes were proposed in this pull request? Currently, the memory for temporary buffer used by TimSort is always allocated as on-heap without bookkeeping, it could cause OOM both in on-heap and off-heap mode. This PR will try to manage that by preallocate it together with the pointer array, same with RadixSort. It both works for on-heap and off-heap mode. This PR also change the loadFactor of BytesToBytesMap to 0.5 (it was 0.70), it enables use to radix sort also makes sure that we have enough memory for timsort. ## How was this patch tested? Existing tests. Author: Davies Liu Closes #13318 from davies/fix_timsort. (cherry picked from commit 3074f575a3c84108fddab3f5f56eb1929a4b2cff) Signed-off-by: Davies Liu --- .../spark/unsafe/memory/MemoryBlock.java | 2 +- .../shuffle/sort/ShuffleInMemorySorter.java | 38 +++++++++++----- .../shuffle/sort/ShuffleSortDataFormat.java | 13 +++--- .../spark/unsafe/map/BytesToBytesMap.java | 3 +- .../unsafe/sort/UnsafeInMemorySorter.java | 43 +++++++++++++------ .../unsafe/sort/UnsafeSortDataFormat.java | 13 +++--- .../sort/UnsafeShuffleWriterSuite.java | 35 +++++++++------ .../map/AbstractBytesToBytesMapSuite.java | 2 +- .../util/collection/ExternalSorterSuite.scala | 4 +- .../unsafe/sort/RadixSortSuite.scala | 3 +- .../sql/execution/UnsafeKVExternalSorter.java | 8 +++- .../sql/execution/joins/HashedRelation.scala | 12 +++--- .../execution/benchmark/SortBenchmark.scala | 3 +- 13 files changed, 115 insertions(+), 64 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index e3e79471154df..1bc924d424c02 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -51,6 +51,6 @@ public long size() { * Creates a memory block pointing to the memory used by the long array. */ public static MemoryBlock fromLongArray(final long[] array) { - return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8); + return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8L); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index 75a0e807d76f5..dc36809d8911f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -22,12 +22,12 @@ import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; import org.apache.spark.util.collection.unsafe.sort.RadixSort; final class ShuffleInMemorySorter { - private final Sorter sorter; private static final class SortComparator implements Comparator { @Override public int compare(PackedRecordPointer left, PackedRecordPointer right) { @@ -44,6 +44,9 @@ public int compare(PackedRecordPointer left, PackedRecordPointer right) { * An array of record pointers and partition ids that have been encoded by * {@link PackedRecordPointer}. The sort operates on this array instead of directly manipulating * records. + * + * Only part of the array will be used to store the pointers, the rest part is preserved as + * temporary buffer for sorting. */ private LongArray array; @@ -54,14 +57,14 @@ public int compare(PackedRecordPointer left, PackedRecordPointer right) { private final boolean useRadixSort; /** - * Set to 2x for radix sort to reserve extra memory for sorting, otherwise 1x. + * The position in the pointer array where new records can be inserted. */ - private final int memoryAllocationFactor; + private int pos = 0; /** - * The position in the pointer array where new records can be inserted. + * How many records could be inserted, because part of the array should be left for sorting. */ - private int pos = 0; + private int usableCapacity = 0; private int initialSize; @@ -70,9 +73,14 @@ public int compare(PackedRecordPointer left, PackedRecordPointer right) { assert (initialSize > 0); this.initialSize = initialSize; this.useRadixSort = useRadixSort; - this.memoryAllocationFactor = useRadixSort ? 2 : 1; this.array = consumer.allocateArray(initialSize); - this.sorter = new Sorter<>(ShuffleSortDataFormat.INSTANCE); + this.usableCapacity = getUsableCapacity(); + } + + private int getUsableCapacity() { + // Radix sort requires same amount of used memory as buffer, Tim sort requires + // half of the used memory as buffer. + return (int) (array.size() / (useRadixSort ? 2 : 1.5)); } public void free() { @@ -89,7 +97,8 @@ public int numRecords() { public void reset() { if (consumer != null) { consumer.freeArray(array); - this.array = consumer.allocateArray(initialSize); + array = consumer.allocateArray(initialSize); + usableCapacity = getUsableCapacity(); } pos = 0; } @@ -101,14 +110,15 @@ public void expandPointerArray(LongArray newArray) { array.getBaseOffset(), newArray.getBaseObject(), newArray.getBaseOffset(), - array.size() * (8 / memoryAllocationFactor) + pos * 8L ); consumer.freeArray(array); array = newArray; + usableCapacity = getUsableCapacity(); } public boolean hasSpaceForAnotherRecord() { - return pos < array.size() / memoryAllocationFactor; + return pos < usableCapacity; } public long getMemoryUsage() { @@ -170,6 +180,14 @@ public ShuffleSorterIterator getSortedIterator() { PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); } else { + MemoryBlock unused = new MemoryBlock( + array.getBaseObject(), + array.getBaseOffset() + pos * 8L, + (array.size() - pos) * 8L); + LongArray buffer = new LongArray(unused); + Sorter sorter = + new Sorter<>(new ShuffleSortDataFormat(buffer)); + sorter.sort(array, 0, pos, SORT_COMPARATOR); } return new ShuffleSorterIterator(pos, array, offset); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 1e924d2aec442..717bdd79d47ef 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -19,14 +19,15 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.SortDataFormat; final class ShuffleSortDataFormat extends SortDataFormat { - public static final ShuffleSortDataFormat INSTANCE = new ShuffleSortDataFormat(); + private final LongArray buffer; - private ShuffleSortDataFormat() { } + ShuffleSortDataFormat(LongArray buffer) { + this.buffer = buffer; + } @Override public PackedRecordPointer getKey(LongArray data, int pos) { @@ -70,8 +71,8 @@ public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int @Override public LongArray allocate(int length) { - // This buffer is used temporary (usually small), so it's fine to allocated from JVM heap. - return new LongArray(MemoryBlock.fromLongArray(new long[length])); + assert (length <= buffer.size()) : + "the buffer is smaller than required: " + buffer.size() + " < " + length; + return buffer; } - } diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 6c00608302c4e..dc04025692909 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -221,7 +221,8 @@ public BytesToBytesMap( SparkEnv.get() != null ? SparkEnv.get().blockManager() : null, SparkEnv.get() != null ? SparkEnv.get().serializerManager() : null, initialCapacity, - 0.70, + // In order to re-use the longArray for sorting, the load factor cannot be larger than 0.5. + 0.5, pageSizeBytes, enablePerfMetrics); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 0cce792f33d34..c7b070f519f88 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -25,6 +25,7 @@ import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; /** @@ -69,8 +70,6 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { private final MemoryConsumer consumer; private final TaskMemoryManager memoryManager; @Nullable - private final Sorter sorter; - @Nullable private final Comparator sortComparator; /** @@ -79,14 +78,12 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { @Nullable private final PrefixComparators.RadixSortSupport radixSortSupport; - /** - * Set to 2x for radix sort to reserve extra memory for sorting, otherwise 1x. - */ - private final int memoryAllocationFactor; - /** * Within this buffer, position {@code 2 * i} holds a pointer pointer to the record at * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. + * + * Only part of the array will be used to store the pointers, the rest part is preserved as + * temporary buffer for sorting. */ private LongArray array; @@ -95,6 +92,11 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { */ private int pos = 0; + /** + * How many records could be inserted, because part of the array should be left for sorting. + */ + private int usableCapacity = 0; + private long initialSize; private long totalSortTimeNanos = 0L; @@ -121,7 +123,6 @@ public UnsafeInMemorySorter( this.memoryManager = memoryManager; this.initialSize = array.size(); if (recordComparator != null) { - this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE); this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager); if (canUseRadixSort && prefixComparator instanceof PrefixComparators.RadixSortSupport) { this.radixSortSupport = (PrefixComparators.RadixSortSupport)prefixComparator; @@ -129,12 +130,17 @@ public UnsafeInMemorySorter( this.radixSortSupport = null; } } else { - this.sorter = null; this.sortComparator = null; this.radixSortSupport = null; } - this.memoryAllocationFactor = this.radixSortSupport != null ? 2 : 1; this.array = array; + this.usableCapacity = getUsableCapacity(); + } + + private int getUsableCapacity() { + // Radix sort requires same amount of used memory as buffer, Tim sort requires + // half of the used memory as buffer. + return (int) (array.size() / (radixSortSupport != null ? 2 : 1.5)); } /** @@ -150,7 +156,8 @@ public void free() { public void reset() { if (consumer != null) { consumer.freeArray(array); - this.array = consumer.allocateArray(initialSize); + array = consumer.allocateArray(initialSize); + usableCapacity = getUsableCapacity(); } pos = 0; } @@ -174,7 +181,7 @@ public long getMemoryUsage() { } public boolean hasSpaceForAnotherRecord() { - return pos + 1 < (array.size() / memoryAllocationFactor); + return pos + 1 < usableCapacity; } public void expandPointerArray(LongArray newArray) { @@ -186,9 +193,10 @@ public void expandPointerArray(LongArray newArray) { array.getBaseOffset(), newArray.getBaseObject(), newArray.getBaseOffset(), - array.size() * (8 / memoryAllocationFactor)); + pos * 8L); consumer.freeArray(array); array = newArray; + usableCapacity = getUsableCapacity(); } /** @@ -275,13 +283,20 @@ public void loadNext() { public SortedIterator getSortedIterator() { int offset = 0; long start = System.nanoTime(); - if (sorter != null) { + if (sortComparator != null) { if (this.radixSortSupport != null) { // TODO(ekl) we should handle NULL values before radix sort for efficiency, since they // force a full-width sort (and we cannot radix-sort nullable long fields at all). offset = RadixSort.sortKeyPrefixArray( array, pos / 2, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { + MemoryBlock unused = new MemoryBlock( + array.getBaseObject(), + array.getBaseOffset() + pos * 8L, + (array.size() - pos) * 8L); + LongArray buffer = new LongArray(unused); + Sorter sorter = + new Sorter<>(new UnsafeSortDataFormat(buffer)); sorter.sort(array, 0, pos / 2, sortComparator); } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java index 7bda76907f4c3..430bf677edbdf 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java @@ -19,7 +19,6 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.SortDataFormat; /** @@ -32,9 +31,11 @@ public final class UnsafeSortDataFormat extends SortDataFormat { - public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat(); + private final LongArray buffer; - private UnsafeSortDataFormat() { } + public UnsafeSortDataFormat(LongArray buffer) { + this.buffer = buffer; + } @Override public RecordPointerAndKeyPrefix getKey(LongArray data, int pos) { @@ -83,9 +84,9 @@ public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int @Override public LongArray allocate(int length) { - assert (length < Integer.MAX_VALUE / 2) : "Length " + length + " is too large"; - // This is used as temporary buffer, it's fine to allocate from JVM heap. - return new LongArray(MemoryBlock.fromLongArray(new long[length * 2])); + assert (length * 2 <= buffer.size()) : + "the buffer is smaller than required: " + buffer.size() + " < " + (length * 2); + return buffer; } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index f9dc20d8b751b..7dd61f85abefd 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -21,12 +21,15 @@ import java.nio.ByteBuffer; import java.util.*; -import scala.*; +import scala.Option; +import scala.Product2; +import scala.Tuple2; +import scala.Tuple2$; import scala.collection.Iterator; import scala.runtime.AbstractFunction1; -import com.google.common.collect.Iterators; import com.google.common.collect.HashMultiset; +import com.google.common.collect.Iterators; import com.google.common.io.ByteStreams; import org.junit.After; import org.junit.Before; @@ -35,29 +38,33 @@ import org.mockito.MockitoAnnotations; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.*; -import static org.mockito.Answers.RETURNS_SMART_NULLS; -import static org.mockito.Mockito.*; -import org.apache.spark.*; +import org.apache.spark.HashPartitioner; +import org.apache.spark.ShuffleDependency; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.executor.TaskMetrics; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.LZ4CompressionCodec; import org.apache.spark.io.LZFCompressionCodec; import org.apache.spark.io.SnappyCompressionCodec; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.network.util.LimitedInputStream; -import org.apache.spark.serializer.*; import org.apache.spark.scheduler.MapStatus; +import org.apache.spark.serializer.*; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.storage.*; -import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.util.Utils; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; +import static org.junit.Assert.*; +import static org.mockito.Answers.RETURNS_SMART_NULLS; +import static org.mockito.Mockito.*; + public class UnsafeShuffleWriterSuite { static final int NUM_PARTITITONS = 4; diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 84b82f5a4742c..fc127f07c8d69 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -589,7 +589,7 @@ public void spillInIterator() throws IOException { @Test public void multipleValuesForSameKey() { BytesToBytesMap map = - new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false); + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024, false); try { int i; for (i = 0; i < 1024; i++) { 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 699f7fa1f2727..6bcc601e13ecc 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 @@ -106,8 +106,10 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { // that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi() val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i } val buf = new LongArray(MemoryBlock.fromLongArray(ref)) + val tmp = new Array[Long](size/2) + val tmpBuf = new LongArray(MemoryBlock.fromLongArray(tmp)) - new Sorter(UnsafeSortDataFormat.INSTANCE).sort( + new Sorter(new UnsafeSortDataFormat(tmpBuf)).sort( buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] { override def compare( r1: RecordPointerAndKeyPrefix, diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index def0752b46f6a..1d26d4a8307cf 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -93,7 +93,8 @@ class RadixSortSuite extends SparkFunSuite with Logging { } private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { - new Sorter(UnsafeSortDataFormat.INSTANCE).sort( + val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) + new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { override def compare( r1: RecordPointerAndKeyPrefix, diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 38dbfef76caee..bb823cd07be5e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -73,6 +73,8 @@ public UnsafeKVExternalSorter( PrefixComparator prefixComparator = SortPrefixUtils.getPrefixComparator(keySchema); BaseOrdering ordering = GenerateOrdering.create(keySchema); KVComparator recordComparator = new KVComparator(ordering, keySchema.length()); + boolean canUseRadixSort = keySchema.length() == 1 && + SortPrefixUtils.canSortFullyWithPrefix(keySchema.apply(0)); TaskMemoryManager taskMemoryManager = taskContext.taskMemoryManager(); @@ -86,14 +88,16 @@ public UnsafeKVExternalSorter( prefixComparator, /* initialSize */ 4096, pageSizeBytes, - keySchema.length() == 1 && SortPrefixUtils.canSortFullyWithPrefix(keySchema.apply(0))); + canUseRadixSort); } else { + // The array will be used to do in-place sort, which require half of the space to be empty. + assert(map.numKeys() <= map.getArray().size() / 2); // During spilling, the array in map will not be used, so we can borrow that and use it // as the underline array for in-memory sorter (it's always large enough). // Since we will not grow the array, it's fine to pass `null` as consumer. final UnsafeInMemorySorter inMemSorter = new UnsafeInMemorySorter( null, taskMemoryManager, recordComparator, prefixComparator, map.getArray(), - false /* TODO(ekl) we can only radix sort if the BytesToBytes load factor is <= 0.5 */); + canUseRadixSort); // We cannot use the destructive iterator here because we are reusing the existing memory // pages in BytesToBytesMap to hold records during sorting. 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 cd6b97a855412..412e8c54ca308 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 @@ -540,7 +540,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap Platform.copyMemory(page, Platform.LONG_ARRAY_OFFSET, newPage, Platform.LONG_ARRAY_OFFSET, cursor - Platform.LONG_ARRAY_OFFSET) page = newPage - freeMemory(used * 8) + freeMemory(used * 8L) } // copy the bytes of UnsafeRow @@ -599,7 +599,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap i += 2 } old_array = null // release the reference to old array - freeMemory(n * 8) + freeMemory(n * 8L) } /** @@ -610,7 +610,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap // Convert to dense mode if it does not require more memory or could fit within L1 cache if (range < array.length || range < 1024) { try { - ensureAcquireMemory((range + 1) * 8) + ensureAcquireMemory((range + 1) * 8L) } catch { case e: SparkException => // there is no enough memory to convert @@ -628,7 +628,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap val old_length = array.length array = denseArray isDense = true - freeMemory(old_length * 8) + freeMemory(old_length * 8L) } } @@ -637,11 +637,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def free(): Unit = { if (page != null) { - freeMemory(page.length * 8) + freeMemory(page.length * 8L) page = null } if (array != null) { - freeMemory(array.length * 8) + freeMemory(array.length * 8L) array = null } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 0e1868dd66565..9964b7373fc20 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -36,7 +36,8 @@ import org.apache.spark.util.random.XORShiftRandom class SortBenchmark extends BenchmarkBase { private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { - new Sorter(UnsafeSortDataFormat.INSTANCE).sort( + val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) + new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { override def compare( r1: RecordPointerAndKeyPrefix, From a2540b9367b228e925ab97d7fde859e02f1e1158 Mon Sep 17 00:00:00 2001 From: Subroto Sanyal Date: Fri, 3 Jun 2016 16:50:00 -0700 Subject: [PATCH 0522/1470] [SPARK-15754][YARN] Not letting the credentials containing hdfs delegation tokens to be added in current user credential. ## What changes were proposed in this pull request? The credentials are not added to the credentials of UserGroupInformation.getCurrentUser(). Further if the client has possibility to login using keytab then the updateDelegationToken thread is not started on client. ## How was this patch tested? ran dev/run-tests Author: Subroto Sanyal Closes #13499 from subrotosanyal/SPARK-15754-save-ugi-from-changing. (cherry picked from commit 61d729abdaf9bdd54aea74a643828d570b036d87) Signed-off-by: Marcelo Vanzin --- yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 598eb17d5d4a0..9bb369549d943 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 @@ -980,7 +980,6 @@ private[spark] class Client( amContainer.setApplicationACLs( YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager).asJava) setupSecurityToken(amContainer) - UserGroupInformation.getCurrentUser().addCredentials(credentials) amContainer } @@ -1001,7 +1000,8 @@ private[spark] class Client( sparkConf.set(KEYTAB.key, keytabFileName) sparkConf.set(PRINCIPAL.key, principal) } - credentials = UserGroupInformation.getCurrentUser.getCredentials + // Defensive copy of the credentials + credentials = new Credentials(UserGroupInformation.getCurrentUser.getCredentials) } /** From cf87821166cdcba0c3c23baeb58bb8689ca6c7e6 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 3 Jun 2016 22:19:22 -0700 Subject: [PATCH 0523/1470] [SPARK-15756][SQL] Support command 'create table stored as orcfile/parquetfile/avrofile' ## What changes were proposed in this pull request? Now Spark SQL can support 'create table src stored as orc/parquet/avro' for orc/parquet/avro table. But Hive can support both commands: ' stored as orc/parquet/avro' and 'stored as orcfile/parquetfile/avrofile'. So this PR supports these keywords 'orcfile/parquetfile/avrofile' in Spark SQL. ## How was this patch tested? add unit tests Author: Lianhui Wang Closes #13500 from lianhuiwang/SPARK-15756. (cherry picked from commit 2ca563cc45d1ac1c19b8e84c5a87a950c712ab87) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/internal/HiveSerDe.scala | 3 +++ .../sql/execution/command/DDLCommandSuite.scala | 15 +++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala index d554937d8b400..ad69137f7401b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -71,6 +71,9 @@ object HiveSerDe { val key = source.toLowerCase match { case s if s.startsWith("org.apache.spark.sql.parquet") => "parquet" case s if s.startsWith("org.apache.spark.sql.orc") => "orc" + case s if s.equals("orcfile") => "orc" + case s if s.equals("parquetfile") => "parquet" + case s if s.equals("avrofile") => "avro" case s => s } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 850fca585216d..aec7e99d9d37d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -237,6 +237,21 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed4, expected4) } + test("create table - table file format") { + val allSources = Seq("parquet", "parquetfile", "orc", "orcfile", "avro", "avrofile", + "sequencefile", "rcfile", "textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab STORED AS $s" + val ct = parseAs[CreateTableCommand](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s, new SQLConf) + assert(hiveSerde.isDefined) + assert(ct.table.storage.serde == hiveSerde.get.serde) + assert(ct.table.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.table.storage.outputFormat == hiveSerde.get.outputFormat) + } + } + test("create table - row format and table file format") { val createTableStart = "CREATE TABLE my_tab ROW FORMAT" val fileFormat = s"STORED AS INPUTFORMAT 'inputfmt' OUTPUTFORMAT 'outputfmt'" From 729730159c6236cb437d215388d444f16849f405 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Sat, 4 Jun 2016 13:56:04 +0100 Subject: [PATCH 0524/1470] [SPARK-15617][ML][DOC] Clarify that fMeasure in MulticlassMetrics is "micro" f1_score ## What changes were proposed in this pull request? 1, del precision,recall in `ml.MulticlassClassificationEvaluator` 2, update user guide for `mlllib.weightedFMeasure` ## How was this patch tested? local build Author: Ruifeng Zheng Closes #13390 from zhengruifeng/clarify_f1. (cherry picked from commit 2099e05f93067937cdf6cedcf493afd66e212abe) Signed-off-by: Sean Owen --- docs/mllib-evaluation-metrics.md | 16 +++------------- .../MulticlassClassificationEvaluator.scala | 12 +++++------- .../MulticlassClassificationEvaluatorSuite.scala | 2 +- python/pyspark/ml/evaluation.py | 4 +--- 4 files changed, 10 insertions(+), 24 deletions(-) diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index a269dbf030e7c..c49bc4ff124bd 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -140,7 +140,7 @@ definitions of positive and negative labels is straightforward. #### Label based metrics Opposed to binary classification where there are only two possible labels, multiclass classification problems have many -possible labels and so the concept of label-based metrics is introduced. Overall precision measures precision across all +possible labels and so the concept of label-based metrics is introduced. Accuracy measures precision across all labels - the number of times any class was predicted correctly (true positives) normalized by the number of data points. Precision by label considers only one class, and measures the number of time a specific label was predicted correctly normalized by the number of times that label appears in the output. @@ -182,20 +182,10 @@ $$\hat{\delta}(x) = \begin{cases}1 & \text{if $x = 0$}, \\ 0 & \text{otherwise}. - Overall Precision - $PPV = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} \hat{\delta}\left(\hat{\mathbf{y}}_i - - \mathbf{y}_i\right)$ - - - Overall Recall - $TPR = \frac{TP}{TP + FN} = \frac{1}{N}\sum_{i=0}^{N-1} \hat{\delta}\left(\hat{\mathbf{y}}_i - + Accuracy + $ACC = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} \hat{\delta}\left(\hat{\mathbf{y}}_i - \mathbf{y}_i\right)$ - - Overall F1-measure - $F1 = 2 \cdot \left(\frac{PPV \cdot TPR} - {PPV + TPR}\right)$ - Precision by label $PPV(\ell) = \frac{TP}{TP + FP} = diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 0b84e0a3fa784..794b1e7d9d881 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -39,16 +39,16 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid def this() = this(Identifiable.randomUID("mcEval")) /** - * param for metric name in evaluation (supports `"f1"` (default), `"precision"`, `"recall"`, - * `"weightedPrecision"`, `"weightedRecall"`, `"accuracy"`) + * param for metric name in evaluation (supports `"f1"` (default), `"weightedPrecision"`, + * `"weightedRecall"`, `"accuracy"`) * @group param */ @Since("1.5.0") val metricName: Param[String] = { - val allowedParams = ParamValidators.inArray(Array("f1", "precision", - "recall", "weightedPrecision", "weightedRecall", "accuracy")) + val allowedParams = ParamValidators.inArray(Array("f1", "weightedPrecision", + "weightedRecall", "accuracy")) new Param(this, "metricName", "metric name in evaluation " + - "(f1|precision|recall|weightedPrecision|weightedRecall|accuracy)", allowedParams) + "(f1|weightedPrecision|weightedRecall|accuracy)", allowedParams) } /** @group getParam */ @@ -82,8 +82,6 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid val metrics = new MulticlassMetrics(predictionAndLabels) val metric = $(metricName) match { case "f1" => metrics.weightedFMeasure - case "precision" => metrics.accuracy - case "recall" => metrics.accuracy case "weightedPrecision" => metrics.weightedPrecision case "weightedRecall" => metrics.weightedRecall case "accuracy" => metrics.accuracy diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala index 522f6675d7f46..1a3a8a13a2d09 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala @@ -33,7 +33,7 @@ class MulticlassClassificationEvaluatorSuite val evaluator = new MulticlassClassificationEvaluator() .setPredictionCol("myPrediction") .setLabelCol("myLabel") - .setMetricName("recall") + .setMetricName("accuracy") testDefaultReadWrite(evaluator) } diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index b8b2b37af553d..c480525e9bd3a 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -258,9 +258,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio >>> evaluator = MulticlassClassificationEvaluator(predictionCol="prediction") >>> evaluator.evaluate(dataset) 0.66... - >>> evaluator.evaluate(dataset, {evaluator.metricName: "precision"}) - 0.66... - >>> evaluator.evaluate(dataset, {evaluator.metricName: "recall"}) + >>> evaluator.evaluate(dataset, {evaluator.metricName: "accuracy"}) 0.66... .. versionadded:: 1.5.0 From ed1e20207c1c2e503a22d5ad2cdf505ef6ecbcad Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 4 Jun 2016 21:41:27 +0100 Subject: [PATCH 0525/1470] [MINOR][BUILD] Add modernizr MIT license; specify "2014 and onwards" in license copyright ## What changes were proposed in this pull request? Per conversation on dev list, add missing modernizr license. Specify "2014 and onwards" in copyright statement. ## How was this patch tested? (none required) Author: Sean Owen Closes #13510 from srowen/ModernizrLicense. (cherry picked from commit 681387b2dc9a094cfba84188a1dd1ac9192bb99c) Signed-off-by: Sean Owen --- LICENSE | 1 + NOTICE | 2 +- licenses/LICENSE-modernizr.txt | 21 +++++++++++++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) create mode 100644 licenses/LICENSE-modernizr.txt diff --git a/LICENSE b/LICENSE index f40364046b34d..94fd46f568473 100644 --- a/LICENSE +++ b/LICENSE @@ -296,3 +296,4 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (MIT License) blockUI (http://jquery.malsup.com/block/) (MIT License) RowsGroup (http://datatables.net/license/mit) (MIT License) jsonFormatter (http://www.jqueryscript.net/other/jQuery-Plugin-For-Pretty-JSON-Formatting-jsonFormatter.html) + (MIT License) modernizr (https://github.com/Modernizr/Modernizr/blob/master/LICENSE) diff --git a/NOTICE b/NOTICE index f4b1260a22170..69b513ea3ba3c 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Spark -Copyright 2014 The Apache Software Foundation. +Copyright 2014 and onwards The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/licenses/LICENSE-modernizr.txt b/licenses/LICENSE-modernizr.txt new file mode 100644 index 0000000000000..2bf24b9b9f848 --- /dev/null +++ b/licenses/LICENSE-modernizr.txt @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file From 7e4c9dd55532b35030f6542f6640521596eb13f3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 4 Jun 2016 14:14:50 -0700 Subject: [PATCH 0526/1470] [SPARK-15762][SQL] Cache Metadata & StructType hashCodes; use singleton Metadata.empty We should cache `Metadata.hashCode` and use a singleton for `Metadata.empty` because calculating metadata hashCodes appears to be a bottleneck for certain workloads. We should also cache `StructType.hashCode`. In an optimizer stress-test benchmark run by ericl, these `hashCode` calls accounted for roughly 40% of the total CPU time and this bottleneck was completely eliminated by the caching added by this patch. Author: Josh Rosen Closes #13504 from JoshRosen/metadata-fix. (cherry picked from commit 091f81e1f7ef1581376c71e3872ce06f4c1713bd) Signed-off-by: Josh Rosen --- .../main/scala/org/apache/spark/sql/types/Metadata.scala | 7 +++++-- .../main/scala/org/apache/spark/sql/types/StructType.scala | 3 ++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 1fb2e2404cc42..657bd86ce17d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -104,7 +104,8 @@ sealed class Metadata private[types] (private[types] val map: Map[String, Any]) } } - override def hashCode: Int = Metadata.hash(this) + private lazy val _hashCode: Int = Metadata.hash(this) + override def hashCode: Int = _hashCode private def get[T](key: String): T = { map(key).asInstanceOf[T] @@ -115,8 +116,10 @@ sealed class Metadata private[types] (private[types] val map: Map[String, Any]) object Metadata { + private[this] val _empty = new Metadata(Map.empty) + /** Returns an empty Metadata. */ - def empty: Metadata = new Metadata(Map.empty) + def empty: Metadata = _empty /** Creates a Metadata instance from JSON. */ def fromJson(json: String): Metadata = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index fd2b524e22117..9a923737598c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -112,7 +112,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru } } - override def hashCode(): Int = java.util.Arrays.hashCode(fields.asInstanceOf[Array[AnyRef]]) + private lazy val _hashCode: Int = java.util.Arrays.hashCode(fields.asInstanceOf[Array[AnyRef]]) + override def hashCode(): Int = _hashCode /** * Creates a new [[StructType]] by adding a new field. From 32a64d8fc9e7ddaf993bdd7e679113dc605a69a7 Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Sat, 4 Jun 2016 22:44:03 +0100 Subject: [PATCH 0527/1470] [SPARK-15707][SQL] Make Code Neat - Use map instead of if check. ## What changes were proposed in this pull request? In forType function of object RandomDataGenerator, the code following: if (maybeSqlTypeGenerator.isDefined){ .... Some(generator) } else{ None } will be changed. Instead, maybeSqlTypeGenerator.map will be used. ## How was this patch tested? All of the current unit tests passed. Author: Weiqing Yang Closes #13448 from Sherry302/master. (cherry picked from commit 0f307db5e17e1e8a655cfa751218ac4ed88717a7) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/sql/RandomDataGenerator.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala index 711e8707116cf..850869799507f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala @@ -236,9 +236,8 @@ object RandomDataGenerator { // convert it to catalyst value to call udt's deserialize. val toCatalystType = CatalystTypeConverters.createToCatalystConverter(udt.sqlType) - if (maybeSqlTypeGenerator.isDefined) { - val sqlTypeGenerator = maybeSqlTypeGenerator.get - val generator = () => { + maybeSqlTypeGenerator.map { sqlTypeGenerator => + () => { val generatedScalaValue = sqlTypeGenerator.apply() if (generatedScalaValue == null) { null @@ -246,9 +245,6 @@ object RandomDataGenerator { udt.deserialize(toCatalystType(generatedScalaValue)) } } - Some(generator) - } else { - None } case unsupportedType => None } From 8c0ec85e62f762c11e0686d1c35d1dfec05df9de Mon Sep 17 00:00:00 2001 From: Brett Randall Date: Sun, 5 Jun 2016 15:31:56 +0100 Subject: [PATCH 0528/1470] =?UTF-8?q?[SPARK-15723]=20Fixed=20local-timezon?= =?UTF-8?q?e-brittle=20test=20where=20short-timezone=20form=20"EST"=20is?= =?UTF-8?q?=20=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Stop using the abbreviated and ambiguous timezone "EST" in a test, since it is machine-local default timezone dependent, and fails in different timezones. Fixed [SPARK-15723](https://issues.apache.org/jira/browse/SPARK-15723). ## How was this patch tested? Note that to reproduce this problem in any locale/timezone, you can modify the scalatest-maven-plugin argLine to add a timezone: -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} -Duser.timezone="Australia/Sydney" and run $ mvn test -DwildcardSuites=org.apache.spark.status.api.v1.SimpleDateParamSuite -Dtest=none. Equally this will fix it in an effected timezone: -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} -Duser.timezone="America/New_York" To test the fix, apply the above change to `pom.xml` to set test TZ to `Australia/Sydney`, and confirm the test now passes. Author: Brett Randall Closes #13462 from javabrett/SPARK-15723-SimpleDateParamSuite. (cherry picked from commit 4e767d0f9042bfea6074c2637438859699ec4dc3) Signed-off-by: Sean Owen --- .../org/apache/spark/status/api/v1/SimpleDateParamSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala index 63b0e77629dde..18baeb1cb9c71 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala @@ -26,7 +26,8 @@ class SimpleDateParamSuite extends SparkFunSuite with Matchers { test("date parsing") { new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be (1424474477190L) - new SimpleDateParam("2015-02-20T17:21:17.190EST").timestamp should be (1424470877190L) + // don't use EST, it is ambiguous, use -0500 instead, see SPARK-15723 + new SimpleDateParam("2015-02-20T17:21:17.190-0500").timestamp should be (1424470877190L) new SimpleDateParam("2015-02-20").timestamp should be (1424390400000L) // GMT intercept[WebApplicationException] { new SimpleDateParam("invalid date") From 1ece135b959d2a7f332e6ad06814a83fabc75d0f Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sun, 5 Jun 2016 11:55:25 -0700 Subject: [PATCH 0529/1470] [SPARK-15770][ML] Annotation audit for Experimental and DeveloperApi ## What changes were proposed in this pull request? 1, remove comments `:: Experimental ::` for non-experimental API 2, add comments `:: Experimental ::` for experimental API 3, add comments `:: DeveloperApi ::` for developerApi API ## How was this patch tested? manual tests Author: Zheng RuiFeng Closes #13514 from zhengruifeng/del_experimental. (cherry picked from commit 372fa61f511843f53498b9e843a84cfdd76fa2b2) Signed-off-by: Reynold Xin --- .../spark/ml/attribute/AttributeType.scala | 3 +++ .../org/apache/spark/ml/clustering/LDA.scala | 4 ++++ .../mllib/clustering/BisectingKMeans.scala | 4 ++++ .../mllib/clustering/BisectingKMeansModel.scala | 2 ++ .../spark/mllib/clustering/LDAModel.scala | 17 ++++++++++++++++- .../mllib/evaluation/MulticlassMetrics.scala | 1 - .../spark/mllib/evaluation/RankingMetrics.scala | 1 - .../spark/mllib/feature/StandardScaler.scala | 6 ++++++ .../apache/spark/mllib/random/RandomRDDs.scala | 5 +++++ .../spark/mllib/rdd/MLPairRDDFunctions.scala | 4 ++++ .../apache/spark/mllib/rdd/RDDFunctions.scala | 4 ++++ .../apache/spark/mllib/recommendation/ALS.scala | 1 + .../apache/spark/mllib/tree/model/Predict.scala | 1 + 13 files changed, 50 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala index 5c7089b491677..078fecf088282 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala @@ -27,6 +27,9 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi sealed abstract class AttributeType(val name: String) +/** + * :: DeveloperApi :: + */ @DeveloperApi object AttributeType { diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 5aec692c98e6e..609e50eb494ce 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -675,6 +675,8 @@ class DistributedLDAModel private[ml] ( private var _checkpointFiles: Array[String] = oldDistributedModel.checkpointFiles /** + * :: DeveloperApi :: + * * If using checkpointing and [[LDA.keepLastCheckpoint]] is set to true, then there may be * saved checkpoint files. This method is provided so that users can manage those files. * @@ -689,6 +691,8 @@ class DistributedLDAModel private[ml] ( def getCheckpointFiles: Array[String] = _checkpointFiles /** + * :: DeveloperApi :: + * * Remove any remaining checkpoint files from training. * * @see [[getCheckpointFiles]] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index e4bd0dc25ee54..91edcf2a7925b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -31,6 +31,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** + * :: Experimental :: + * * A bisecting k-means algorithm based on the paper "A comparison of document clustering techniques" * by Steinbach, Karypis, and Kumar, with modification to fit Spark. * The algorithm starts from a single cluster that contains all points. @@ -396,6 +398,8 @@ private object BisectingKMeans extends Serializable { } /** + * :: Experimental :: + * * Represents a node in a clustering tree. * * @param index node index, negative for internal nodes and non-negative for leaf nodes diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index b3546a1ee3677..11fd940b8b205 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -32,6 +32,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession} /** + * :: Experimental :: + * * Clustering model produced by [[BisectingKMeans]]. * The prediction is done level-by-level from the root node to a leaf node, and at each node among * its children the closest to the input point is selected. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 4f07236225cd2..d295826300419 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -425,6 +425,13 @@ class LocalLDAModel private[spark] ( } +/** + * :: Experimental :: + * + * Local (non-distributed) model fitted by [[LDA]]. + * + * This model stores the inferred topics only; it does not store info about the training dataset. + */ @Experimental @Since("1.5.0") object LocalLDAModel extends Loader[LocalLDAModel] { @@ -814,7 +821,15 @@ class DistributedLDAModel private[clustering] ( } } - +/** + * :: Experimental :: + * + * Distributed model fitted by [[LDA]]. + * This type of model is currently only produced by Expectation-Maximization (EM). + * + * This model stores the inferred topics, the full training dataset, and the topic distribution + * for each training document. + */ @Experimental @Since("1.5.0") object DistributedLDAModel extends Loader[DistributedLDAModel] { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 719695a3380df..9a6a8dbdccbf3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame /** - * ::Experimental:: * Evaluator for multiclass classification. * * @param predictionAndLabels an RDD of (prediction, label) pairs. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index 4ed4a058945c3..e29b51c3a19da 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -28,7 +28,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** - * ::Experimental:: * Evaluator for ranking algorithms. * * Java users should use [[RankingMetrics$.of]] to create a [[RankingMetrics]] instance. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index ee97045f34dc8..b7d6c6056803a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -97,6 +97,9 @@ class StandardScalerModel @Since("1.3.0") ( @Since("1.3.0") def this(std: Vector) = this(std, null) + /** + * :: DeveloperApi :: + */ @Since("1.3.0") @DeveloperApi def setWithMean(withMean: Boolean): this.type = { @@ -105,6 +108,9 @@ class StandardScalerModel @Since("1.3.0") ( this } + /** + * :: DeveloperApi :: + */ @Since("1.3.0") @DeveloperApi def setWithStd(withStd: Boolean): this.type = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index b0a716936ae6f..c2bc1f17ccd58 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -418,6 +418,7 @@ object RandomRDDs { } /** + * :: DeveloperApi :: * [[RandomRDDs#randomJavaRDD]] with the default seed. */ @DeveloperApi @@ -431,6 +432,7 @@ object RandomRDDs { } /** + * :: DeveloperApi :: * [[RandomRDDs#randomJavaRDD]] with the default seed & numPartitions */ @DeveloperApi @@ -854,6 +856,7 @@ object RandomRDDs { } /** + * :: DeveloperApi :: * Java-friendly version of [[RandomRDDs#randomVectorRDD]]. */ @DeveloperApi @@ -869,6 +872,7 @@ object RandomRDDs { } /** + * :: DeveloperApi :: * [[RandomRDDs#randomJavaVectorRDD]] with the default seed. */ @DeveloperApi @@ -883,6 +887,7 @@ object RandomRDDs { } /** + * :: DeveloperApi :: * [[RandomRDDs#randomJavaVectorRDD]] with the default number of partitions and the default seed. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala index 1b93e2d764c69..af4bc9f290c3e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -25,6 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.util.BoundedPriorityQueue /** + * :: DeveloperApi :: * Machine learning specific Pair RDD functions. */ @DeveloperApi @@ -50,6 +51,9 @@ class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Se } } +/** + * :: DeveloperApi :: + */ @DeveloperApi object MLPairRDDFunctions { /** Implicit conversion from a pair RDD to MLPairRDDFunctions. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index e8a937ffcb96f..0f7857b8d8627 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: * Machine learning specific RDD functions. */ @DeveloperApi @@ -53,6 +54,9 @@ class RDDFunctions[T: ClassTag](self: RDD[T]) extends Serializable { } +/** + * :: DeveloperApi :: + */ @DeveloperApi object RDDFunctions { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 467cb83cd1662..cc9ee15738ad6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -216,6 +216,7 @@ class ALS private ( } /** + * :: DeveloperApi :: * Set period (in iterations) between checkpoints (default = 10). Checkpointing helps with * recovery (when nodes fail) and StackOverflow exceptions caused by long lineage. It also helps * with eliminating temporary shuffle files on disk, which can be important when there are many diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 06ceff19d8633..1dbdd2d860efd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.annotation.{DeveloperApi, Since} /** + * :: DeveloperApi :: * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) From e6e1d823289a3ba18bb9b34413d7ed5a31416a23 Mon Sep 17 00:00:00 2001 From: Kai Jiang Date: Sun, 5 Jun 2016 13:03:02 -0700 Subject: [PATCH 0530/1470] [MINOR][R][DOC] Fix R documentation generation instruction. ## What changes were proposed in this pull request? changes in R/README.md - Make step of generating SparkR document more clear. - link R/DOCUMENTATION.md from R/README.md - turn on some code syntax highlight in R/README.md ## How was this patch tested? local test Author: Kai Jiang Closes #13488 from vectorijk/R-Readme. (cherry picked from commit 8a9110510c9e4cbbcb0dede62cb4b9dd1c6bc8cc) Signed-off-by: Shivaram Venkataraman --- R/DOCUMENTATION.md | 12 ++++++------ R/README.md | 30 ++++++++++++++---------------- 2 files changed, 20 insertions(+), 22 deletions(-) diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md index 931d01549b265..7314a1fcccda9 100644 --- a/R/DOCUMENTATION.md +++ b/R/DOCUMENTATION.md @@ -1,12 +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, +SparkR documentation is generated by using in-source comments and annotated by using +[`roxygen2`](https://cran.r-project.org/web/packages/roxygen2/index.html). After making changes to the documentation and generating 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")) - +```R +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 index 044f95312ae8e..932d5272d0b4f 100644 --- a/R/README.md +++ b/R/README.md @@ -7,8 +7,7 @@ SparkR is an R package that provides a light-weight frontend to use Spark from R Libraries of sparkR need to be created in `$SPARK_HOME/R/lib`. This can be done by running the script `$SPARK_HOME/R/install-dev.sh`. By default the above script uses the system wide installation of R. However, this can be changed to any user installed location of R by setting the environment variable `R_HOME` the full path of the base directory where R is installed, before running install-dev.sh script. Example: - -``` +```bash # where /home/username/R is where R is installed and /home/username/R/bin contains the files R and RScript export R_HOME=/home/username/R ./install-dev.sh @@ -20,8 +19,8 @@ export R_HOME=/home/username/R 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 +```bash +build/mvn -DskipTests -Psparkr package ``` #### Running sparkR @@ -40,9 +39,8 @@ To set other options like driver memory, executor memory etc. you can pass in th #### 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 - -``` +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 +```R # Set this to where Spark is installed Sys.setenv(SPARK_HOME="/Users/username/spark") # This line loads SparkR from the installed directory @@ -59,25 +57,25 @@ Once you have made your changes, please include unit tests for them and run exis #### 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. +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. Also, you may need to install these [prerequisites](https://github.com/apache/spark/tree/master/docs#prerequisites). See also, `R/DOCUMENTATION.md` ### Examples, Unit tests SparkR comes with several sample programs in the `examples/src/main/r` directory. To run one of them, use `./bin/spark-submit `. For example: - - ./bin/spark-submit examples/src/main/r/dataframe.R - +```bash +./bin/spark-submit examples/src/main/r/dataframe.R +``` 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 +```bash +R -e 'install.packages("testthat", repos="http://cran.us.r-project.org")' +./R/run-tests.sh +``` ### Running on YARN The `./bin/spark-submit` 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 - -``` +```bash export YARN_CONF_DIR=/etc/hadoop/conf ./bin/spark-submit --master yarn examples/src/main/r/dataframe.R ``` From 38a626a54dd0fac0ca460e1ba534048de513bc29 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 5 Jun 2016 15:59:52 -0700 Subject: [PATCH 0531/1470] [SPARK-15657][SQL] RowEncoder should validate the data type of input object ## What changes were proposed in this pull request? This PR improves the error handling of `RowEncoder`. When we create a `RowEncoder` with a given schema, we should validate the data type of input object. e.g. we should throw an exception when a field is boolean but is declared as a string column. This PR also removes the support to use `Product` as a valid external type of struct type. This support is added at https://github.com/apache/spark/pull/9712, but is incomplete, e.g. nested product, product in array are both not working. However, we never officially support this feature and I think it's ok to ban it. ## How was this patch tested? new tests in `RowEncoderSuite`. Author: Wenchen Fan Closes #13401 from cloud-fan/bug. (cherry picked from commit 30c4774f33fed63b7d400d220d710fb432f599a8) Signed-off-by: Cheng Lian --- .../main/scala/org/apache/spark/sql/Row.scala | 10 +-- .../sql/catalyst/encoders/RowEncoder.scala | 17 ++++-- .../expressions/objects/objects.scala | 61 ++++++++++++++++--- .../catalyst/encoders/RowEncoderSuite.scala | 47 +++++++++----- 4 files changed, 95 insertions(+), 40 deletions(-) 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 a257b831dd1cc..391001de26ece 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 @@ -304,15 +304,7 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ - def getStruct(i: Int): Row = { - // Product and Row both are recognized as StructType in a Row - val t = get(i) - if (t.isInstanceOf[Product]) { - Row.fromTuple(t.asInstanceOf[Product]) - } else { - t.asInstanceOf[Row] - } - } + def getStruct(i: Int): Row = getAs[Row](i) /** * Returns the value at position i. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 6cd7b34ceb88c..67fca153b551a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -51,7 +51,7 @@ import org.apache.spark.unsafe.types.UTF8String * BinaryType -> byte array * ArrayType -> scala.collection.Seq or Array * MapType -> scala.collection.Map - * StructType -> org.apache.spark.sql.Row or Product + * StructType -> org.apache.spark.sql.Row * }}} */ object RowEncoder { @@ -121,11 +121,15 @@ object RowEncoder { case t @ ArrayType(et, _) => et match { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType => + // TODO: validate input type for primitive array. NewInstance( classOf[GenericArrayData], inputObject :: Nil, dataType = t) - case _ => MapObjects(serializerFor(_, et), inputObject, externalDataTypeForInput(et)) + case _ => MapObjects( + element => serializerFor(ValidateExternalType(element, et), et), + inputObject, + ObjectType(classOf[Object])) } case t @ MapType(kt, vt, valueNullable) => @@ -151,8 +155,9 @@ object RowEncoder { case StructType(fields) => val nonNullOutput = CreateNamedStruct(fields.zipWithIndex.flatMap { case (field, index) => val fieldValue = serializerFor( - GetExternalRowField( - inputObject, index, field.name, externalDataTypeForInput(field.dataType)), + ValidateExternalType( + GetExternalRowField(inputObject, index, field.name), + field.dataType), field.dataType) val convertedField = if (field.nullable) { If( @@ -183,7 +188,7 @@ object RowEncoder { * can be `scala.math.BigDecimal`, `java.math.BigDecimal`, or * `org.apache.spark.sql.types.Decimal`. */ - private def externalDataTypeForInput(dt: DataType): DataType = dt match { + def externalDataTypeForInput(dt: DataType): DataType = dt match { // In order to support both Decimal and java/scala BigDecimal in external row, we make this // as java.lang.Object. case _: DecimalType => ObjectType(classOf[java.lang.Object]) @@ -192,7 +197,7 @@ object RowEncoder { case _ => externalDataTypeFor(dt) } - private def externalDataTypeFor(dt: DataType): DataType = dt match { + def externalDataTypeFor(dt: DataType): DataType = dt match { case _ if ScalaReflection.isNativeType(dt) => dt case TimestampType => ObjectType(classOf[java.sql.Timestamp]) case DateType => ObjectType(classOf[java.sql.Date]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index d4c71bffe86bf..87c8a2e54a2c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -26,6 +26,7 @@ import org.apache.spark.SparkConf import org.apache.spark.serializer._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.GenericArrayData @@ -692,22 +693,17 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) case class GetExternalRowField( child: Expression, index: Int, - fieldName: String, - dataType: DataType) extends UnaryExpression with NonSQLExpression { + fieldName: String) extends UnaryExpression with NonSQLExpression { override def nullable: Boolean = false + override def dataType: DataType = ObjectType(classOf[Object]) + override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val row = child.genCode(ctx) - - val getField = dataType match { - case ObjectType(x) if x == classOf[Row] => s"""${row.value}.getStruct($index)""" - case _ => s"""(${ctx.boxedType(dataType)}) ${row.value}.get($index)""" - } - val code = s""" ${row.code} @@ -720,8 +716,55 @@ case class GetExternalRowField( "cannot be null."); } - final ${ctx.javaType(dataType)} ${ev.value} = $getField; + final Object ${ev.value} = ${row.value}.get($index); """ ev.copy(code = code, isNull = "false") } } + +/** + * Validates the actual data type of input expression at runtime. If it doesn't match the + * expectation, throw an exception. + */ +case class ValidateExternalType(child: Expression, expected: DataType) + extends UnaryExpression with NonSQLExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(ObjectType(classOf[Object])) + + override def nullable: Boolean = child.nullable + + override def dataType: DataType = RowEncoder.externalDataTypeForInput(expected) + + override def eval(input: InternalRow): Any = + throw new UnsupportedOperationException("Only code-generated evaluation is supported") + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val input = child.genCode(ctx) + val obj = input.value + + val typeCheck = expected match { + case _: DecimalType => + Seq(classOf[java.math.BigDecimal], classOf[scala.math.BigDecimal], classOf[Decimal]) + .map(cls => s"$obj instanceof ${cls.getName}").mkString(" || ") + case _: ArrayType => + s"$obj instanceof ${classOf[Seq[_]].getName} || $obj.getClass().isArray()" + case _ => + s"$obj instanceof ${ctx.boxedType(dataType)}" + } + + val code = s""" + ${input.code} + ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; + if (!${input.isNull}) { + if ($typeCheck) { + ${ev.value} = (${ctx.boxedType(dataType)}) $obj; + } else { + throw new RuntimeException($obj.getClass().getName() + " is not a valid " + + "external type for schema of ${expected.simpleString}"); + } + } + + """ + ev.copy(code = code, isNull = input.isNull) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 16abde064fc44..2e513ea22c151 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -127,22 +127,6 @@ class RowEncoderSuite extends SparkFunSuite { new StructType().add("array", arrayOfString).add("map", mapOfString)) .add("structOfUDT", structOfUDT)) - test(s"encode/decode: Product") { - val schema = new StructType() - .add("structAsProduct", - new StructType() - .add("int", IntegerType) - .add("string", StringType) - .add("double", DoubleType)) - - val encoder = RowEncoder(schema).resolveAndBind() - - val input: Row = Row((100, "test", 0.123)) - val row = encoder.toRow(input) - val convertedBack = encoder.fromRow(row) - assert(input.getStruct(0) == convertedBack.getStruct(0)) - } - test("encode/decode decimal type") { val schema = new StructType() .add("int", IntegerType) @@ -232,6 +216,37 @@ class RowEncoderSuite extends SparkFunSuite { assert(e.getMessage.contains("top level row object")) } + test("RowEncoder should validate external type") { + val e1 = intercept[RuntimeException] { + val schema = new StructType().add("a", IntegerType) + val encoder = RowEncoder(schema) + encoder.toRow(Row(1.toShort)) + } + assert(e1.getMessage.contains("java.lang.Short is not a valid external type")) + + val e2 = intercept[RuntimeException] { + val schema = new StructType().add("a", StringType) + val encoder = RowEncoder(schema) + encoder.toRow(Row(1)) + } + assert(e2.getMessage.contains("java.lang.Integer is not a valid external type")) + + val e3 = intercept[RuntimeException] { + val schema = new StructType().add("a", + new StructType().add("b", IntegerType).add("c", StringType)) + val encoder = RowEncoder(schema) + encoder.toRow(Row(1 -> "a")) + } + assert(e3.getMessage.contains("scala.Tuple2 is not a valid external type")) + + val e4 = intercept[RuntimeException] { + val schema = new StructType().add("a", ArrayType(TimestampType)) + val encoder = RowEncoder(schema) + encoder.toRow(Row(Array("a"))) + } + assert(e4.getMessage.contains("java.lang.String is not a valid external type")) + } + private def encodeDecodeTest(schema: StructType): Unit = { test(s"encode/decode: ${schema.simpleString}") { val encoder = RowEncoder(schema).resolveAndBind() From d8370ef117c96ebb30e9213d8d89fd3edbd796d7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 5 Jun 2016 16:51:00 -0700 Subject: [PATCH 0532/1470] [SPARK-15748][SQL] Replace inefficient foldLeft() call with flatMap() in PartitionStatistics `PartitionStatistics` uses `foldLeft` and list concatenation (`++`) to flatten an iterator of lists, but this is extremely inefficient compared to simply doing `flatMap`/`flatten` because it performs many unnecessary object allocations. Simply replacing this `foldLeft` by a `flatMap` results in decent performance gains when constructing PartitionStatistics instances for tables with many columns. This patch fixes this and also makes two similar changes in MLlib and streaming to try to fix all known occurrences of this pattern. Author: Josh Rosen Closes #13491 from JoshRosen/foldleft-to-flatmap. (cherry picked from commit 26c1089c37149061f838129bb53330ded68ff4c9) Signed-off-by: Reynold Xin --- mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala | 2 +- .../org/apache/spark/sql/execution/columnar/ColumnStats.scala | 4 ++-- .../scala/org/apache/spark/streaming/ui/StreamingPage.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index 94d1b83ec2531..8ed40c379c123 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -422,7 +422,7 @@ private[ml] object MetaAlgorithmReadWrite { case rformModel: RFormulaModel => Array(rformModel.pipelineModel) case _: Params => Array() } - val subStageMaps = subStages.map(getUidMapImpl).foldLeft(List.empty[(String, Params)])(_ ++ _) + val subStageMaps = subStages.flatMap(getUidMapImpl) List((instance.uid, instance)) ++ subStageMaps } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index 5d4476989a369..470307bd940ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -33,9 +33,9 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl } private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { - val (forAttribute, schema) = { + val (forAttribute: AttributeMap[ColumnStatisticsSchema], schema: Seq[AttributeReference]) = { val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) - (AttributeMap(allStats), allStats.map(_._2.schema).foldLeft(Seq.empty[Attribute])(_ ++ _)) + (AttributeMap(allStats), allStats.flatMap(_._2.schema)) } } 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 b97e24f28bfc6..46cd3092e9061 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 @@ -396,11 +396,11 @@ private[ui] class StreamingPage(parent: StreamingTab) .map(_.ceil.toLong) .getOrElse(0L) - val content = listener.receivedRecordRateWithBatchTime.toList.sortBy(_._1).map { + val content: Seq[Node] = listener.receivedRecordRateWithBatchTime.toList.sortBy(_._1).flatMap { case (streamId, recordRates) => generateInputDStreamRow( jsCollector, streamId, recordRates, minX, maxX, minY, maxYCalculated) - }.foldLeft[Seq[Node]](Nil)(_ ++ _) + } // scalastyle:off From 790de600beb3f6cae1914f59a61a43c02440884f Mon Sep 17 00:00:00 2001 From: Hiroshi Inoue Date: Sun, 5 Jun 2016 20:10:33 -0700 Subject: [PATCH 0533/1470] [SPARK-15704][SQL] add a test case in DatasetAggregatorSuite for regression testing ## What changes were proposed in this pull request? This change fixes a crash in TungstenAggregate while executing "Dataset complex Aggregator" test case due to IndexOutOfBoundsException. jira entry for detail: https://issues.apache.org/jira/browse/SPARK-15704 ## How was this patch tested? Using existing unit tests (including DatasetBenchmark) Author: Hiroshi Inoue Closes #13446 from inouehrs/fix_aggregate. (cherry picked from commit 79268aa461abd237bc4f96a7d31457c98e11798c) Signed-off-by: Wenchen Fan --- .../spark/sql/DatasetAggregatorSuite.scala | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index ead7bd9642eca..f9b4cd83c3a42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StringType object ComplexResultAgg extends Aggregator[(String, Int), (Long, Long), (Long, Long)] { @@ -52,6 +53,16 @@ object ClassInputAgg extends Aggregator[AggData, Int, Int] { } +object ClassBufferAggregator extends Aggregator[AggData, AggData, Int] { + override def zero: AggData = AggData(0, "") + override def reduce(b: AggData, a: AggData): AggData = AggData(b.a + a.a, "") + override def finish(reduction: AggData): Int = reduction.a + override def merge(b1: AggData, b2: AggData): AggData = AggData(b1.a + b2.a, "") + override def bufferEncoder: Encoder[AggData] = Encoders.product[AggData] + override def outputEncoder: Encoder[Int] = Encoders.scalaInt +} + + object ComplexBufferAgg extends Aggregator[AggData, (Int, AggData), Int] { override def zero: (Int, AggData) = 0 -> AggData(0, "0") override def reduce(b: (Int, AggData), a: AggData): (Int, AggData) = (b._1 + 1, a) @@ -173,6 +184,14 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { ("one", 1)) } + test("Typed aggregation using aggregator") { + // based on Dataset complex Aggregator test of DatasetBenchmark + val ds = Seq(AggData(1, "x"), AggData(2, "y"), AggData(3, "z")).toDS() + checkDataset( + ds.select(ClassBufferAggregator.toColumn), + 6) + } + test("typed aggregation: complex input") { val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS() From 9e7e2f9164e0b3bd555e795b871626057b4fed31 Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Sun, 5 Jun 2016 23:35:04 -0700 Subject: [PATCH 0534/1470] [SPARK-15585][SQL] Fix NULL handling along with a spark-csv behaivour ## What changes were proposed in this pull request? This pr fixes the behaviour of `format("csv").option("quote", null)` along with one of spark-csv. Also, it explicitly sets default values for CSV options in python. ## How was this patch tested? Added tests in CSVSuite. Author: Takeshi YAMAMURO Closes #13372 from maropu/SPARK-15585. (cherry picked from commit b7e8d1cb3ce932ba4a784be59744af8a8ef027ce) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 81 +++++++++---------- .../datasources/csv/CSVOptions.scala | 11 ++- .../execution/datasources/csv/CSVSuite.scala | 11 +++ 3 files changed, 55 insertions(+), 48 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 9208a527d29c3..19aa8ddd0683d 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -303,10 +303,11 @@ def text(self, paths): return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(path))) @since(2.0) - def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, - comment=None, header=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, - nullValue=None, nanValue=None, positiveInf=None, negativeInf=None, dateFormat=None, - maxColumns=None, maxCharsPerColumn=None, mode=None): + def csv(self, path, schema=None, sep=u',', encoding=u'UTF-8', quote=u'\"', escape=u'\\', + comment=None, header='false', ignoreLeadingWhiteSpace='false', + ignoreTrailingWhiteSpace='false', nullValue='', nanValue='NaN', positiveInf='Inf', + negativeInf='Inf', dateFormat=None, maxColumns='20480', maxCharsPerColumn='1000000', + mode='PERMISSIVE'): """Loads a CSV file and returns the result as a [[DataFrame]]. This function goes through the input once to determine the input schema. To avoid going @@ -315,44 +316,41 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param path: string, or list of strings, for input path(s). :param schema: an optional :class:`StructType` for the input schema. :param sep: sets the single character as a separator for each field and value. - If None is set, it uses the default value, ``,``. - :param encoding: decodes the CSV files by the given encoding type. If None is set, - it uses the default value, ``UTF-8``. + The default value is ``,``. + :param encoding: decodes the CSV files by the given encoding type. + The default value is ``UTF-8``. :param quote: sets the single character used for escaping quoted values where the - separator can be part of the value. If None is set, it uses the default - value, ``"``. + separator can be part of the value. The default value is ``"``. :param escape: sets the single character used for escaping quotes inside an already - quoted value. If None is set, it uses the default value, ``\``. + quoted value. The default value is ``\``. :param comment: sets the single character used for skipping lines beginning with this character. By default (None), it is disabled. - :param header: uses the first line as names of columns. If None is set, it uses the - default value, ``false``. + :param header: uses the first line as names of columns. The default value is ``false``. :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values - being read should be skipped. If None is set, it uses - the default value, ``false``. + being read should be skipped. The default value is + ``false``. :param ignoreTrailingWhiteSpace: defines whether or not trailing whitespaces from values - being read should be skipped. If None is set, it uses - the default value, ``false``. - :param nullValue: sets the string representation of a null value. If None is set, it uses - the default value, empty string. - :param nanValue: sets the string representation of a non-number value. If None is set, it - uses the default value, ``NaN``. - :param positiveInf: sets the string representation of a positive infinity value. If None - is set, it uses the default value, ``Inf``. - :param negativeInf: sets the string representation of a negative infinity value. If None - is set, it uses the default value, ``Inf``. + being read should be skipped. The default value is + ``false``. + :param nullValue: sets the string representation of a null value. The default value is a + empty string. + :param nanValue: sets the string representation of a non-number value. The default value is + ``NaN``. + :param positiveInf: sets the string representation of a positive infinity value. The default + value is ``Inf``. + :param negativeInf: sets the string representation of a negative infinity value. The default + value is ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This applies to both date type and timestamp type. By default, it is None which means trying to parse times and date by ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. - :param maxColumns: defines a hard limit of how many columns a record can have. If None is - set, it uses the default value, ``20480``. + :param maxColumns: defines a hard limit of how many columns a record can have. The default + value is ``20480``. :param maxCharsPerColumn: defines the maximum number of characters allowed for any given - value being read. If None is set, it uses the default value, - ``1000000``. - :param mode: allows a mode for dealing with corrupt records during parsing. If None is - set, it uses the default value, ``PERMISSIVE``. + value being read. The default value is ``1000000``. + :param mode: allows a mode for dealing with corrupt records during parsing. The default + value is ``PERMISSIVE``. * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. When a schema is set by user, it sets ``null`` for extra fields. @@ -785,8 +783,8 @@ def text(self, path, compression=None): self._jwrite.text(path) @since(2.0) - def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, - header=None, nullValue=None, escapeQuotes=None): + def csv(self, path, mode='error', compression=None, sep=',', quote=u'\"', escape='\\', + header='false', nullValue='', escapeQuotes='true'): """Saves the content of the [[DataFrame]] in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -800,20 +798,19 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). - :param sep: sets the single character as a separator for each field and value. If None is - set, it uses the default value, ``,``. + :param sep: sets the single character as a separator for each field and value. The default + value is ``,``. :param quote: sets the single character used for escaping quoted values where the - separator can be part of the value. If None is set, it uses the default - value, ``"``. + separator can be part of the value. The default value is ``"``. :param escape: sets the single character used for escaping quotes inside an already - quoted value. If None is set, it uses the default value, ``\`` + quoted value. The default value is ``\`` :param escapeQuotes: A flag indicating whether values containing quotes should always be enclosed in quotes. If None is set, it uses the default value ``true``, escaping all values containing a quote character. - :param header: writes the names of columns as the first line. If None is set, it uses - the default value, ``false``. - :param nullValue: sets the string representation of a null value. If None is set, it uses - the default value, empty string. + :param header: writes the names of columns as the first line. The default value is + ``false``. + :param nullValue: sets the string representation of a null value. The default value is a + empty string. >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ @@ -831,7 +828,7 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No if nullValue is not None: self.option("nullValue", nullValue) if escapeQuotes is not None: - self.option("escapeQuotes", nullValue) + self.option("escapeQuotes", escapeQuotes) self._jwrite.csv(path) @since(1.5) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 9f4ce8358b045..044ada26073cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -30,8 +30,7 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str val paramValue = parameters.get(paramName) paramValue match { case None => default - case Some(null) => default - case Some(value) if value.length == 0 => '\u0000' + case Some(value) if value == null || value.length == 0 => '\u0000' case Some(value) if value.length == 1 => value.charAt(0) case _ => throw new RuntimeException(s"$paramName cannot be more than one character") } @@ -52,12 +51,12 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str } private def getBool(paramName: String, default: Boolean = false): Boolean = { - val param = parameters.getOrElse(paramName, default.toString) - if (param == null) { + val paramValue = parameters.getOrElse(paramName, default.toString) + if (paramValue == null) { default - } else if (param.toLowerCase == "true") { + } else if (paramValue.toLowerCase == "true") { true - } else if (param.toLowerCase == "false") { + } else if (paramValue.toLowerCase == "false") { false } else { throw new Exception(s"$paramName flag can be true or false") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index bc95446387956..b26fcea75987f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -655,4 +655,15 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(msg.contains("CSV data source does not support array data type")) } } + + test("SPARK-15585 set null at quote") { + val cars = spark.read + .format("csv") + .option("header", "true") + .option("quote", null) + .load(testFile(carsUnbalancedQuotesFile)) + + verifyCars(cars, withHeader = true, checkValues = false) + } + } From 7d10e4bdd2adbeb10904665536e4949381f19cf5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 5 Jun 2016 23:40:35 -0700 Subject: [PATCH 0535/1470] Revert "[SPARK-15585][SQL] Fix NULL handling along with a spark-csv behaivour" This reverts commit 9e7e2f9164e0b3bd555e795b871626057b4fed31. --- python/pyspark/sql/readwriter.py | 81 ++++++++++--------- .../datasources/csv/CSVOptions.scala | 11 +-- .../execution/datasources/csv/CSVSuite.scala | 11 --- 3 files changed, 48 insertions(+), 55 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 19aa8ddd0683d..9208a527d29c3 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -303,11 +303,10 @@ def text(self, paths): return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(path))) @since(2.0) - def csv(self, path, schema=None, sep=u',', encoding=u'UTF-8', quote=u'\"', escape=u'\\', - comment=None, header='false', ignoreLeadingWhiteSpace='false', - ignoreTrailingWhiteSpace='false', nullValue='', nanValue='NaN', positiveInf='Inf', - negativeInf='Inf', dateFormat=None, maxColumns='20480', maxCharsPerColumn='1000000', - mode='PERMISSIVE'): + def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, + comment=None, header=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, + nullValue=None, nanValue=None, positiveInf=None, negativeInf=None, dateFormat=None, + maxColumns=None, maxCharsPerColumn=None, mode=None): """Loads a CSV file and returns the result as a [[DataFrame]]. This function goes through the input once to determine the input schema. To avoid going @@ -316,41 +315,44 @@ def csv(self, path, schema=None, sep=u',', encoding=u'UTF-8', quote=u'\"', escap :param path: string, or list of strings, for input path(s). :param schema: an optional :class:`StructType` for the input schema. :param sep: sets the single character as a separator for each field and value. - The default value is ``,``. - :param encoding: decodes the CSV files by the given encoding type. - The default value is ``UTF-8``. + If None is set, it uses the default value, ``,``. + :param encoding: decodes the CSV files by the given encoding type. If None is set, + it uses the default value, ``UTF-8``. :param quote: sets the single character used for escaping quoted values where the - separator can be part of the value. The default value is ``"``. + separator can be part of the value. If None is set, it uses the default + value, ``"``. :param escape: sets the single character used for escaping quotes inside an already - quoted value. The default value is ``\``. + quoted value. If None is set, it uses the default value, ``\``. :param comment: sets the single character used for skipping lines beginning with this character. By default (None), it is disabled. - :param header: uses the first line as names of columns. The default value is ``false``. + :param header: uses the first line as names of columns. If None is set, it uses the + default value, ``false``. :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values - being read should be skipped. The default value is - ``false``. + being read should be skipped. If None is set, it uses + the default value, ``false``. :param ignoreTrailingWhiteSpace: defines whether or not trailing whitespaces from values - being read should be skipped. The default value is - ``false``. - :param nullValue: sets the string representation of a null value. The default value is a - empty string. - :param nanValue: sets the string representation of a non-number value. The default value is - ``NaN``. - :param positiveInf: sets the string representation of a positive infinity value. The default - value is ``Inf``. - :param negativeInf: sets the string representation of a negative infinity value. The default - value is ``Inf``. + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. + :param nanValue: sets the string representation of a non-number value. If None is set, it + uses the default value, ``NaN``. + :param positiveInf: sets the string representation of a positive infinity value. If None + is set, it uses the default value, ``Inf``. + :param negativeInf: sets the string representation of a negative infinity value. If None + is set, it uses the default value, ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This applies to both date type and timestamp type. By default, it is None which means trying to parse times and date by ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. - :param maxColumns: defines a hard limit of how many columns a record can have. The default - value is ``20480``. + :param maxColumns: defines a hard limit of how many columns a record can have. If None is + set, it uses the default value, ``20480``. :param maxCharsPerColumn: defines the maximum number of characters allowed for any given - value being read. The default value is ``1000000``. - :param mode: allows a mode for dealing with corrupt records during parsing. The default - value is ``PERMISSIVE``. + value being read. If None is set, it uses the default value, + ``1000000``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. When a schema is set by user, it sets ``null`` for extra fields. @@ -783,8 +785,8 @@ def text(self, path, compression=None): self._jwrite.text(path) @since(2.0) - def csv(self, path, mode='error', compression=None, sep=',', quote=u'\"', escape='\\', - header='false', nullValue='', escapeQuotes='true'): + def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, + header=None, nullValue=None, escapeQuotes=None): """Saves the content of the [[DataFrame]] in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -798,19 +800,20 @@ def csv(self, path, mode='error', compression=None, sep=',', quote=u'\"', escape :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). - :param sep: sets the single character as a separator for each field and value. The default - value is ``,``. + :param sep: sets the single character as a separator for each field and value. If None is + set, it uses the default value, ``,``. :param quote: sets the single character used for escaping quoted values where the - separator can be part of the value. The default value is ``"``. + separator can be part of the value. If None is set, it uses the default + value, ``"``. :param escape: sets the single character used for escaping quotes inside an already - quoted value. The default value is ``\`` + quoted value. If None is set, it uses the default value, ``\`` :param escapeQuotes: A flag indicating whether values containing quotes should always be enclosed in quotes. If None is set, it uses the default value ``true``, escaping all values containing a quote character. - :param header: writes the names of columns as the first line. The default value is - ``false``. - :param nullValue: sets the string representation of a null value. The default value is a - empty string. + :param header: writes the names of columns as the first line. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ @@ -828,7 +831,7 @@ def csv(self, path, mode='error', compression=None, sep=',', quote=u'\"', escape if nullValue is not None: self.option("nullValue", nullValue) if escapeQuotes is not None: - self.option("escapeQuotes", escapeQuotes) + self.option("escapeQuotes", nullValue) self._jwrite.csv(path) @since(1.5) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 044ada26073cd..9f4ce8358b045 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -30,7 +30,8 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str val paramValue = parameters.get(paramName) paramValue match { case None => default - case Some(value) if value == null || value.length == 0 => '\u0000' + case Some(null) => default + case Some(value) if value.length == 0 => '\u0000' case Some(value) if value.length == 1 => value.charAt(0) case _ => throw new RuntimeException(s"$paramName cannot be more than one character") } @@ -51,12 +52,12 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str } private def getBool(paramName: String, default: Boolean = false): Boolean = { - val paramValue = parameters.getOrElse(paramName, default.toString) - if (paramValue == null) { + val param = parameters.getOrElse(paramName, default.toString) + if (param == null) { default - } else if (paramValue.toLowerCase == "true") { + } else if (param.toLowerCase == "true") { true - } else if (paramValue.toLowerCase == "false") { + } else if (param.toLowerCase == "false") { false } else { throw new Exception(s"$paramName flag can be true or false") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index b26fcea75987f..bc95446387956 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -655,15 +655,4 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(msg.contains("CSV data source does not support array data type")) } } - - test("SPARK-15585 set null at quote") { - val cars = spark.read - .format("csv") - .option("header", "true") - .option("quote", null) - .load(testFile(carsUnbalancedQuotesFile)) - - verifyCars(cars, withHeader = true, checkValues = false) - } - } From 90e94b82649d9816cd4065549678b82751238552 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Mon, 6 Jun 2016 09:35:47 +0100 Subject: [PATCH 0536/1470] [MINOR] Fix Typos 'an -> a' ## What changes were proposed in this pull request? `an -> a` Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one. ## How was this patch tested? manual tests Author: Zheng RuiFeng Closes #13515 from zhengruifeng/an_a. (cherry picked from commit fd8af397132fa1415a4c19d7f5cb5a41aa6ddb27) Signed-off-by: Sean Owen --- R/pkg/R/utils.R | 2 +- .../main/scala/org/apache/spark/Accumulable.scala | 2 +- .../apache/spark/api/java/JavaSparkContext.scala | 2 +- .../org/apache/spark/api/python/PythonRDD.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 6 +++--- .../main/scala/org/apache/spark/rdd/JdbcRDD.scala | 6 +++--- .../scala/org/apache/spark/scheduler/Pool.scala | 2 +- .../apache/spark/broadcast/BroadcastSuite.scala | 2 +- .../deploy/rest/StandaloneRestSubmitSuite.scala | 2 +- .../scala/org/apache/spark/rpc/RpcEnvSuite.scala | 2 +- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 4 ++-- .../org/apache/spark/util/JsonProtocolSuite.scala | 2 +- .../spark/streaming/flume/FlumeBatchFetcher.scala | 2 +- .../spark/graphx/impl/VertexPartitionBaseOps.scala | 2 +- .../scala/org/apache/spark/ml/linalg/Vectors.scala | 2 +- .../main/scala/org/apache/spark/ml/Pipeline.scala | 2 +- .../ml/classification/LogisticRegression.scala | 4 ++-- .../apache/spark/ml/tree/impl/RandomForest.scala | 2 +- .../mllib/classification/LogisticRegression.scala | 2 +- .../apache/spark/mllib/classification/SVM.scala | 2 +- .../spark/mllib/feature/VectorTransformer.scala | 2 +- .../org/apache/spark/mllib/linalg/Vectors.scala | 2 +- .../linalg/distributed/CoordinateMatrix.scala | 2 +- .../spark/mllib/rdd/MLPairRDDFunctions.scala | 2 +- python/pyspark/ml/classification.py | 4 ++-- python/pyspark/ml/pipeline.py | 2 +- python/pyspark/mllib/classification.py | 2 +- python/pyspark/mllib/common.py | 2 +- python/pyspark/rdd.py | 4 ++-- python/pyspark/sql/session.py | 2 +- python/pyspark/sql/streaming.py | 2 +- python/pyspark/sql/types.py | 2 +- python/pyspark/streaming/dstream.py | 4 ++-- .../src/main/scala/org/apache/spark/sql/Row.scala | 2 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 4 ++-- .../sql/catalyst/analysis/FunctionRegistry.scala | 2 +- .../catalyst/analysis/MultiInstanceRelation.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 6 +++--- .../sql/catalyst/catalog/functionResources.scala | 2 +- .../catalyst/expressions/ExpectsInputTypes.scala | 2 +- .../sql/catalyst/expressions/Projection.scala | 4 ++-- .../catalyst/expressions/complexTypeCreator.scala | 2 +- .../apache/spark/sql/types/AbstractDataType.scala | 2 +- .../org/apache/spark/sql/DataFrameReader.scala | 2 +- .../scala/org/apache/spark/sql/SQLContext.scala | 14 +++++++------- .../scala/org/apache/spark/sql/SQLImplicits.scala | 2 +- .../scala/org/apache/spark/sql/SparkSession.scala | 14 +++++++------- .../org/apache/spark/sql/catalyst/SQLBuilder.scala | 2 +- .../aggregate/SortBasedAggregationIterator.scala | 2 +- .../spark/sql/execution/aggregate/udaf.scala | 2 +- .../columnar/GenerateColumnAccessor.scala | 2 +- .../execution/datasources/FileSourceStrategy.scala | 2 +- .../execution/datasources/json/JacksonParser.scala | 2 +- .../datasources/parquet/CatalystRowConverter.scala | 2 +- .../execution/exchange/ExchangeCoordinator.scala | 10 +++++----- .../sql/execution/joins/SortMergeJoinExec.scala | 2 +- .../sql/execution/r/MapPartitionsRWrapper.scala | 2 +- .../org/apache/spark/sql/expressions/udaf.scala | 2 +- .../apache/spark/sql/internal/SharedState.scala | 2 +- .../spark/sql/streaming/ContinuousQuery.scala | 2 +- .../apache/spark/sql/hive/client/HiveClient.scala | 2 +- .../spark/sql/hive/orc/OrcFileOperator.scala | 2 +- .../sql/hive/execution/HiveComparisonTest.scala | 2 +- .../scala/org/apache/spark/streaming/State.scala | 2 +- .../streaming/api/java/JavaStreamingContext.scala | 6 +++--- .../apache/spark/streaming/dstream/DStream.scala | 4 ++-- .../spark/streaming/receiver/ReceivedBlock.scala | 2 +- .../scheduler/ReceiverSchedulingPolicy.scala | 2 +- .../apache/spark/streaming/CheckpointSuite.scala | 2 +- .../scheduler/rate/PIDRateEstimatorSuite.scala | 2 +- 70 files changed, 102 insertions(+), 102 deletions(-) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index e734366131943..12e4f4f1ae8bb 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -489,7 +489,7 @@ processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { # 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). +# a new version of func that has a correct environment (closure). cleanClosure <- function(func, checkedFuncs = new.env()) { if (is.function(func)) { newEnv <- new.env(parent = .GlobalEnv) diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index 812145aaeedb7..5532931e2a794 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, LegacyAcc /** - * A data type that can be accumulated, i.e. has an commutative and associative "add" operation, + * A data type that can be accumulated, i.e. has a commutative and associative "add" operation, * but where the result type, `R`, may be different from the element type being added, `T`. * * You must define how to add data, and how to merge two of these together. For some data types, diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index bfb6a35f5bb93..485a8b4222e5a 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -789,7 +789,7 @@ class JavaSparkContext(val sc: SparkContext) def cancelAllJobs(): Unit = sc.cancelAllJobs() /** - * Returns an Java map of JavaRDDs that have marked themselves as persistent via cache() call. + * Returns a Java map of JavaRDDs that have marked themselves as persistent via cache() call. * Note that this does not necessarily mean the caching or computation was successful. */ def getPersistentRDDs: JMap[java.lang.Integer, JavaRDD[_]] = { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ab5b6c8380e8b..2822eb5d60024 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -919,7 +919,7 @@ private class PythonAccumulatorParam(@transient private val serverHost: String, } /** - * An Wrapper for Python Broadcast, which is written into disk by Python. It also will + * A Wrapper for Python Broadcast, which is written into disk by Python. It also will * write the data into disk after deserialization, then Python can read it from disks. */ // scalastyle:off no.finalize 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 78606e06fbd32..9be4cadcb430a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -408,12 +408,12 @@ object SparkSubmit { printErrorAndExit("SparkR is not supported for Mesos cluster.") } - // If we're running a R app, set the main class to our specific R runner + // If we're running an 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. + // If an 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 @@ -422,7 +422,7 @@ object SparkSubmit { } if (isYarnCluster && args.isR) { - // In yarn-cluster mode for a R app, add primary resource to files + // In yarn-cluster mode for an R app, add primary resource to files // that can be distributed with the job args.files = mergeFileLists(args.files, args.primaryResource) } diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 5426bf80bafc5..2f42916439d29 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -34,7 +34,7 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e // TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** - * An RDD that executes an SQL query on a JDBC connection and reads results. + * An RDD that executes a SQL query on a JDBC connection and reads results. * For usage example, see test case JdbcRDDSuite. * * @param getConnection a function that returns an open Connection. @@ -138,7 +138,7 @@ object JdbcRDD { } /** - * Create an RDD that executes an SQL query on a JDBC connection and reads results. + * Create an RDD that executes a SQL query on a JDBC connection and reads results. * For usage example, see test case JavaAPISuite.testJavaJdbcRDD. * * @param connectionFactory a factory that returns an open Connection. @@ -178,7 +178,7 @@ object JdbcRDD { } /** - * Create an RDD that executes an SQL query on a JDBC connection and reads results. Each row is + * Create an RDD that executes a SQL query on a JDBC connection and reads results. Each row is * converted into a `Object` array. For usage example, see test case JavaAPISuite.testJavaJdbcRDD. * * @param connectionFactory a factory that returns an open Connection. diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 5987cfea2e9b2..266afaf8d73b9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** - * An Schedulable entity that represents collection of Pools or TaskSetManagers + * A Schedulable entity that represents collection of Pools or TaskSetManagers */ private[spark] class Pool( val poolName: String, 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 6657104823e71..973676398ae54 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -138,7 +138,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { } /** - * Verify the persistence of state associated with an TorrentBroadcast in a local-cluster. + * Verify the persistence of state associated with a TorrentBroadcast in a local-cluster. * * This test creates a broadcast variable, uses it on all executors, and then unpersists it. * In between each step, this test verifies that the broadcast blocks are present only on the 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 a7bb9aa4686eb..dd50e33da30ac 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 @@ -408,7 +408,7 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { /** * Start a [[StandaloneRestServer]] that communicates with the given endpoint. - * If `faulty` is true, start an [[FaultyStandaloneRestServer]] instead. + * If `faulty` is true, start a [[FaultyStandaloneRestServer]] instead. * Return the master URL that corresponds to the address of this server. */ private def startServer( 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 505cd476ff881..acdf21df9a161 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -489,7 +489,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { /** * Setup an [[RpcEndpoint]] to collect all network events. * - * @return the [[RpcEndpointRef]] and an `ConcurrentLinkedQueue` that contains network events. + * @return the [[RpcEndpointRef]] and a `ConcurrentLinkedQueue` that contains network events. */ private def setupNetworkEndpoint( _env: RpcEnv, 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 088a476086217..5bcc8ff5252a1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1703,7 +1703,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou } test("reduce tasks should be placed locally with map output") { - // Create an shuffleMapRdd with 1 partition + // Create a shuffleMapRdd with 1 partition val shuffleMapRdd = new MyRDD(sc, 1, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) val shuffleId = shuffleDep.shuffleId @@ -1724,7 +1724,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou test("reduce task locality preferences should only include machines with largest map outputs") { val numMapTasks = 4 - // Create an shuffleMapRdd with more partitions + // Create a shuffleMapRdd with more partitions val shuffleMapRdd = new MyRDD(sc, numMapTasks, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) val shuffleId = shuffleDep.shuffleId diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 6fda7378e6cef..7a1ee03e4ce55 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -258,7 +258,7 @@ class JsonProtocolSuite extends SparkFunSuite { } test("FetchFailed backwards compatibility") { - // FetchFailed in Spark 1.1.0 does not have an "Message" property. + // FetchFailed in Spark 1.1.0 does not have a "Message" property. val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, "ignored") val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala index 5f234b1f0ccca..8af7c23431063 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala @@ -26,7 +26,7 @@ import org.apache.spark.streaming.flume.sink._ /** * This class implements the core functionality of [[FlumePollingReceiver]]. When started it * pulls data from Flume, stores it to Spark and then sends an Ack or Nack. This class should be - * run via an [[java.util.concurrent.Executor]] as this implements [[Runnable]] + * run via a [[java.util.concurrent.Executor]] as this implements [[Runnable]] * * @param receiver The receiver that owns this instance. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 31373a53cf933..43594573cf013 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -27,7 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.collection.BitSet /** - * An class containing additional operations for subclasses of VertexPartitionBase that provide + * A class containing additional operations for subclasses of VertexPartitionBase that provide * implicit evidence of membership in the `VertexPartitionBaseOpsConstructor` typeclass (for * example, [[VertexPartition.VertexPartitionOpsConstructor]]). */ diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index 909fec1c06653..c74e5d44a328d 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -548,7 +548,7 @@ object DenseVector { } /** - * A sparse vector represented by an index array and an value array. + * A sparse vector represented by an index array and a value array. * * @param size size of the vector. * @param indices index array, assume to be strictly increasing. 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 b02aea92b72f9..25e56d70c2339 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -82,7 +82,7 @@ abstract class PipelineStage extends Params with Logging { * be called on the input dataset to fit a model. Then the model, which is a transformer, will be * used to transform the dataset as the input to the next stage. If a stage is a [[Transformer]], * its [[Transformer#transform]] method will be called to produce the dataset for the next stage. - * The fitted model from a [[Pipeline]] is an [[PipelineModel]], which consists of fitted models and + * The fitted model from a [[Pipeline]] is a [[PipelineModel]], which consists of fitted models and * transformers, corresponding to the pipeline stages. If there are no stages, the pipeline acts as * an identity transformer. */ 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 cecf3c43dfad4..1ea4d90e166b2 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 @@ -854,7 +854,7 @@ class BinaryLogisticRegressionSummary private[classification] ( /** * Returns the receiver operating characteristic (ROC) curve, - * which is an Dataframe having two fields (FPR, TPR) + * which is a Dataframe having two fields (FPR, TPR) * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. * * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. @@ -874,7 +874,7 @@ class BinaryLogisticRegressionSummary private[classification] ( lazy val areaUnderROC: Double = binaryMetrics.areaUnderROC() /** - * Returns the precision-recall curve, which is an Dataframe containing + * Returns the precision-recall curve, which is a Dataframe containing * two fields recall, precision with (0.0, 1.0) prepended to it. * * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index be3792eb7732f..71c8c42ce5eba 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -491,7 +491,7 @@ private[spark] object RandomForest extends Logging { timer.start("chooseSplits") // In each partition, iterate all instances and compute aggregate stats for each node, - // yield an (nodeIndex, nodeAggregateStats) pair for each node. + // yield a (nodeIndex, nodeAggregateStats) pair for each node. // After a `reduceByKey` operation, // stats of a node will be shuffled to a particular partition and be combined together, // then best splits for nodes are found there. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index e4cc784cfe421..e4cbf5acbc11d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -86,7 +86,7 @@ class LogisticRegressionModel @Since("1.3.0") ( /** * Sets the threshold that separates positive predictions from negative predictions * in Binary Logistic Regression. An example with prediction score greater than or equal to - * this threshold is identified as an positive, and negative otherwise. The default value is 0.5. + * this threshold is identified as a positive, and negative otherwise. The default value is 0.5. * It is only used for binary classification. */ @Since("1.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 783c1c835b3b7..7c3ccbb40b812 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -44,7 +44,7 @@ class SVMModel @Since("1.1.0") ( /** * Sets the threshold that separates positive predictions from negative predictions. An example - * with prediction score greater than or equal to this threshold is identified as an positive, + * with prediction score greater than or equal to this threshold is identified as a positive, * and negative otherwise. The default value is 0.0. */ @Since("1.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala index ca7385128d79a..9db725097ae90 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -53,7 +53,7 @@ trait VectorTransformer extends Serializable { } /** - * Applies transformation on an JavaRDD[Vector]. + * Applies transformation on a JavaRDD[Vector]. * * @param data JavaRDD[Vector] to be transformed. * @return transformed JavaRDD[Vector]. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 02fd60da7d590..91f065831c804 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 @@ -731,7 +731,7 @@ object DenseVector { } /** - * A sparse vector represented by an index array and an value array. + * A sparse vector represented by an index array and a value array. * * @param size size of the vector. * @param indices index array, assume to be strictly increasing. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 97b03b340f20e..008b03d1cc334 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -24,7 +24,7 @@ import org.apache.spark.mllib.linalg.{Matrix, SparseMatrix, Vectors} import org.apache.spark.rdd.RDD /** - * Represents an entry in an distributed matrix. + * Represents an entry in a distributed matrix. * @param i row index * @param j column index * @param value value of the entry diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala index af4bc9f290c3e..e28e1af5b0a26 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -47,7 +47,7 @@ class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Se combOp = (queue1, queue2) => { queue1 ++= queue2 } - ).mapValues(_.toArray.sorted(ord.reverse)) // This is an min-heap, so we reverse the order. + ).mapValues(_.toArray.sorted(ord.reverse)) // This is a min-heap, so we reverse the order. } } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 7710fdf2e216e..77badebeb418e 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -351,7 +351,7 @@ class BinaryLogisticRegressionSummary(LogisticRegressionSummary): def roc(self): """ Returns the receiver operating characteristic (ROC) curve, - which is an Dataframe having two fields (FPR, TPR) with + which is a Dataframe having two fields (FPR, TPR) with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. .. seealso:: `Wikipedia reference \ @@ -380,7 +380,7 @@ def areaUnderROC(self): @since("2.0.0") def pr(self): """ - Returns the precision-recall curve, which is an Dataframe + Returns the precision-recall curve, which is a Dataframe containing two fields recall, precision with (0.0, 1.0) prepended to it. diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index eb6d3310d693b..0777527134ccb 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -42,7 +42,7 @@ class Pipeline(Estimator, MLReadable, MLWritable): stage. If a stage is a :py:class:`Transformer`, its :py:meth:`Transformer.transform` method will be called to produce the dataset for the next stage. The fitted model from a - :py:class:`Pipeline` is an :py:class:`PipelineModel`, which + :py:class:`Pipeline` is a :py:class:`PipelineModel`, which consists of fitted models and transformers, corresponding to the pipeline stages. If there are no stages, the pipeline acts as an identity transformer. diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index f186217031193..3734f87405e5a 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -52,7 +52,7 @@ def setThreshold(self, value): Sets the threshold that separates positive predictions from negative predictions. An example with prediction score greater - than or equal to this threshold is identified as an positive, + than or equal to this threshold is identified as a positive, and negative otherwise. It is used for binary classification only. """ diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index 6bc2b1e64651e..31afdf576b677 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -60,7 +60,7 @@ def _new_smart_decode(obj): # this will call the MLlib version of pythonToJava() def _to_java_object_rdd(rdd): - """ Return an JavaRDD of Object by unpickling + """ Return a JavaRDD of Object by unpickling It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 411e377a56f8b..6afe769662221 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1040,7 +1040,7 @@ def histogram(self, buckets): If the elements in RDD do not vary (max == min) always returns a single bucket. - It will return an tuple of buckets and histogram. + It will return a tuple of buckets and histogram. >>> rdd = sc.parallelize(range(51)) >>> rdd.histogram(2) @@ -2211,7 +2211,7 @@ def lookup(self, key): return values.collect() def _to_java_object_rdd(self): - """ Return an JavaRDD of Object by unpickling + """ Return a JavaRDD of Object by unpickling It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 8f7dcb54a7a2a..7c9f532f9412f 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -360,7 +360,7 @@ def _createFromRDD(self, rdd, schema, samplingRatio): def _createFromLocal(self, data, schema): """ - Create an RDD for DataFrame from an list or pandas.DataFrame, returns + Create an RDD for DataFrame from a list or pandas.DataFrame, returns the RDD and schema. """ # make sure data could consumed multiple times diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index cd75622cedf5e..580aba651f1b0 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -72,7 +72,7 @@ def awaitTermination(self, timeout=None): @since(2.0) def processAllAvailable(self): - """Blocks until all available data in the source has been processed an committed to the + """Blocks until all available data in the source has been processed and committed to the sink. This method is intended for testing. Note that in the case of continually arriving data, this method may block forever. Additionally, this method is only guaranteed to block until data that has been synchronously appended data to a stream source prior to invocation. diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 7d8d0230b4ab3..bb2b95404af7a 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1046,7 +1046,7 @@ def _need_converter(dataType): def _create_converter(dataType): - """Create an converter to drop the names of fields in obj """ + """Create a converter to drop the names of fields in obj """ if not _need_converter(dataType): return lambda x: x diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index fb6c66f2df997..59977dcb435a8 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -608,8 +608,8 @@ def reduceFunc(t, a, b): class TransformedDStream(DStream): """ - TransformedDStream is an DStream generated by an Python function - transforming each RDD of an DStream to another RDDs. + TransformedDStream is a DStream generated by an Python function + transforming each RDD of a DStream to another RDDs. Multiple continuous transformations of DStream can be combined into one transformation. 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 391001de26ece..d83eef7a41629 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 @@ -300,7 +300,7 @@ trait Row extends Serializable { getMap[K, V](i).asJava /** - * Returns the value at position i of struct type as an [[Row]] object. + * Returns the value at position i of struct type as a [[Row]] object. * * @throws ClassCastException when data type does not match. */ 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 4f6b4830cd6f6..44461400aa9f4 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 @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.types._ /** - * A trivial [[Analyzer]] with an dummy [[SessionCatalog]] and [[EmptyFunctionRegistry]]. + * A trivial [[Analyzer]] with a dummy [[SessionCatalog]] and [[EmptyFunctionRegistry]]. * Used for testing when all relations are already filled in and the analyzer needs only * to resolve attribute references. */ @@ -1496,7 +1496,7 @@ class Analyzer( * This rule handles three cases: * - A [[Project]] having [[WindowExpression]]s in its projectList; * - An [[Aggregate]] having [[WindowExpression]]s in its aggregateExpressions. - * - An [[Filter]]->[[Aggregate]] pattern representing GROUP BY with a HAVING + * - A [[Filter]]->[[Aggregate]] pattern representing GROUP BY with a HAVING * clause and the [[Aggregate]] has [[WindowExpression]]s in its aggregateExpressions. * Note: If there is a GROUP BY clause in the query, aggregations and corresponding * filters (expressions in the HAVING clause) should be evaluated before any 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 eca837ccf0a78..a7388c71de9e0 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 @@ -410,7 +410,7 @@ object FunctionRegistry { case Failure(e) => throw new AnalysisException(e.getMessage) } } else { - // Otherwise, find an ctor method that matches the number of arguments, and use that. + // Otherwise, find a constructor method that matches the number of arguments, and use that. val params = Seq.fill(expressions.size)(classOf[Expression]) val f = Try(tag.runtimeClass.getDeclaredConstructor(params : _*)) match { case Success(e) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index 394be47a588b7..95a3837ae1ab7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /** - * A trait that should be mixed into query operators where an single instance might appear multiple + * A trait that should be mixed into query operators where a single instance might appear multiple * times in a logical query plan. It is invalid to have multiple copies of the same attribute * produced by distinct operators in a query tree as this breaks the guarantee that expression * ids, which are used to differentiate attributes, are unique. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 371c198aa3493..943d1071e295a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -241,7 +241,7 @@ class SessionCatalog( /** * Retrieve the metadata of an existing metastore table. * If no database is specified, assume the table is in the current database. - * If the specified table is not found in the database then an [[NoSuchTableException]] is thrown. + * If the specified table is not found in the database then a [[NoSuchTableException]] is thrown. */ def getTableMetadata(name: TableIdentifier): CatalogTable = { val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase)) @@ -266,7 +266,7 @@ class SessionCatalog( /** * Load files stored in given path into an existing metastore table. * If no database is specified, assume the table is in the current database. - * If the specified table is not found in the database then an [[NoSuchTableException]] is thrown. + * If the specified table is not found in the database then a [[NoSuchTableException]] is thrown. */ def loadTable( name: TableIdentifier, @@ -283,7 +283,7 @@ class SessionCatalog( /** * Load files stored in given path into the partition of an existing metastore table. * If no database is specified, assume the table is in the current database. - * If the specified table is not found in the database then an [[NoSuchTableException]] is thrown. + * If the specified table is not found in the database then a [[NoSuchTableException]] is thrown. */ def loadPartition( name: TableIdentifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala index c66d08d2cf8fc..8e46b962ff432 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.AnalysisException -/** An trait that represents the type of a resourced needed by a function. */ +/** A trait that represents the type of a resourced needed by a function. */ abstract class FunctionResourceType(val resourceType: String) object JarResource extends FunctionResourceType("jar") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala index b3dfac806f7fe..c15a2df50855a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.types.AbstractDataType /** - * An trait that gets mixin to define the expected input types of an expression. + * A trait that gets mixin to define the expected input types of an expression. * * This trait is typically used by operator expressions (e.g. [[Add]], [[Subtract]]) to define * expected input types without any implicit casting. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 27ad8e4cf22ce..c8d18667f7c4a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -158,7 +158,7 @@ object UnsafeProjection { object FromUnsafeProjection { /** - * Returns an Projection for given StructType. + * Returns a Projection for given StructType. */ def apply(schema: StructType): Projection = { apply(schema.fields.map(_.dataType)) @@ -172,7 +172,7 @@ object FromUnsafeProjection { } /** - * Returns an Projection for given sequence of Expressions (bounded). + * Returns a Projection for given sequence of Expressions (bounded). */ private def create(exprs: Seq[Expression]): Projection = { GenerateSafeProjection.generate(exprs) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index f60d278c54873..d603d3c73ecbc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -91,7 +91,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { override def checkInputDataTypes(): TypeCheckResult = { if (children.size % 2 != 0) { - TypeCheckResult.TypeCheckFailure(s"$prettyName expects an positive even number of arguments.") + TypeCheckResult.TypeCheckFailure(s"$prettyName expects a positive even number of arguments.") } else if (keys.map(_.dataType).distinct.length > 1) { TypeCheckResult.TypeCheckFailure("The given keys of function map should all be the same " + "type, but they are " + keys.map(_.dataType.simpleString).mkString("[", ", ", "]")) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 03ea349221ec9..65eae869d40d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -148,7 +148,7 @@ abstract class NumericType extends AtomicType { // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a // type parameter and add a numeric annotation (i.e., [JvmType : Numeric]). This gets // desugared by the compiler into an argument to the objects constructor. This means there is no - // longer an no argument constructor and thus the JVM cannot serialize the object anymore. + // longer a no argument constructor and thus the JVM cannot serialize the object anymore. private[sql] val numeric: Numeric[InternalType] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 88fa5cd21d58f..b248583d79e6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -314,7 +314,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { def json(paths: String*): DataFrame = format("json").load(paths : _*) /** - * Loads an `JavaRDD[String]` storing JSON objects (one object per record) and + * Loads a `JavaRDD[String]` storing JSON objects (one object per record) and * returns the result as a [[DataFrame]]. * * Unless the schema is specified using [[schema]] function, this function goes through the 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 2e14c5d486d4f..0fb2400d1bafd 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 @@ -370,7 +370,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * :: DeveloperApi :: - * Creates a [[DataFrame]] from an [[JavaRDD]] containing [[Row]]s using the given schema. + * Creates a [[DataFrame]] from a [[JavaRDD]] containing [[Row]]s using the given schema. * It is important to make sure that the structure of every [[Row]] of the provided RDD matches * the provided schema. Otherwise, there will be runtime exception. * @@ -384,7 +384,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * :: DeveloperApi :: - * Creates a [[DataFrame]] from an [[java.util.List]] containing [[Row]]s using the given schema. + * Creates a [[DataFrame]] from a [[java.util.List]] containing [[Row]]s using the given schema. * It is important to make sure that the structure of every [[Row]] of the provided List matches * the provided schema. Otherwise, there will be runtime exception. * @@ -421,7 +421,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * Applies a schema to an List of Java Beans. + * Applies a schema to a List of Java Beans. * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. @@ -552,7 +552,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from 0 to `end` (exclusive) with step value 1. + * in a range from 0 to `end` (exclusive) with step value 1. * * @since 2.0.0 * @group dataset @@ -563,7 +563,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from `start` to `end` (exclusive) with step value 1. + * in a range from `start` to `end` (exclusive) with step value 1. * * @since 2.0.0 * @group dataset @@ -574,7 +574,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from `start` to `end` (exclusive) with an step value. + * in a range from `start` to `end` (exclusive) with a step value. * * @since 2.0.0 * @group dataset @@ -587,7 +587,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from `start` to `end` (exclusive) with an step value, with partition number + * in a range from `start` to `end` (exclusive) with a step value, with partition number * specified. * * @since 2.0.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index b7ea2a89175fe..440952572d8c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -33,7 +33,7 @@ abstract class SQLImplicits { protected def _sqlContext: SQLContext /** - * Converts $"col name" into an [[Column]]. + * Converts $"col name" into a [[Column]]. * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 7d7fd0399deda..f5b16d07ad80e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -305,7 +305,7 @@ class SparkSession private( /** * :: DeveloperApi :: - * Creates a [[DataFrame]] from an [[JavaRDD]] containing [[Row]]s using the given schema. + * Creates a [[DataFrame]] from a [[JavaRDD]] containing [[Row]]s using the given schema. * It is important to make sure that the structure of every [[Row]] of the provided RDD matches * the provided schema. Otherwise, there will be runtime exception. * @@ -319,7 +319,7 @@ class SparkSession private( /** * :: DeveloperApi :: - * Creates a [[DataFrame]] from an [[java.util.List]] containing [[Row]]s using the given schema. + * Creates a [[DataFrame]] from a [[java.util.List]] containing [[Row]]s using the given schema. * It is important to make sure that the structure of every [[Row]] of the provided List matches * the provided schema. Otherwise, there will be runtime exception. * @@ -365,7 +365,7 @@ class SparkSession private( } /** - * Applies a schema to an List of Java Beans. + * Applies a schema to a List of Java Beans. * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. @@ -475,7 +475,7 @@ class SparkSession private( /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from 0 to `end` (exclusive) with step value 1. + * in a range from 0 to `end` (exclusive) with step value 1. * * @since 2.0.0 * @group dataset @@ -486,7 +486,7 @@ class SparkSession private( /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from `start` to `end` (exclusive) with step value 1. + * in a range from `start` to `end` (exclusive) with step value 1. * * @since 2.0.0 * @group dataset @@ -499,7 +499,7 @@ class SparkSession private( /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from `start` to `end` (exclusive) with an step value. + * in a range from `start` to `end` (exclusive) with a step value. * * @since 2.0.0 * @group dataset @@ -512,7 +512,7 @@ class SparkSession private( /** * :: Experimental :: * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in an range from `start` to `end` (exclusive) with an step value, with partition number + * in a range from `start` to `end` (exclusive) with a step value, with partition number * specified. * * @since 2.0.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 9dc367920ef84..a8cc72f2e7b3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -274,7 +274,7 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { // 5. the table alias for output columns of generator. // 6. the AS keyword // 7. the column alias, can be more than one, e.g. AS key, value - // An concrete example: "tbl LATERAL VIEW EXPLODE(map_col) sub_q AS key, value", and the builder + // A concrete example: "tbl LATERAL VIEW EXPLODE(map_col) sub_q AS key, value", and the builder // will put it in FROM clause later. build( childSQL, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala index f392b135ce787..3f7f84988594a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -86,7 +86,7 @@ class SortBasedAggregationIterator( // The aggregation buffer used by the sort-based aggregation. private[this] val sortBasedAggregationBuffer: MutableRow = newBuffer - // An SafeProjection to turn UnsafeRow into GenericInternalRow, because UnsafeRow can't be + // A SafeProjection to turn UnsafeRow into GenericInternalRow, because UnsafeRow can't be // compared to MutableRow (aggregation buffer) directly. private[this] val safeProj: Projection = FromUnsafeProjection(valueAttributes.map(_.dataType)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 4ceb710f4b2b4..b047bc0641dd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -202,7 +202,7 @@ sealed trait BufferSetterGetterUtils { } /** - * A Mutable [[Row]] representing an mutable aggregation buffer. + * A Mutable [[Row]] representing a mutable aggregation buffer. */ private[sql] class MutableAggregationBufferImpl ( schema: StructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 1041bab9d5d6c..7a14879b8b9df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -58,7 +58,7 @@ class MutableUnsafeRow(val writer: UnsafeRowWriter) extends GenericMutableRow(nu } /** - * Generates bytecode for an [[ColumnarIterator]] for columnar cache. + * Generates bytecode for a [[ColumnarIterator]] for columnar cache. */ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarIterator] with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 350508c1d9f4c..7503285ee2c1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution.SparkPlan * is only done on top level columns, but formats should support pruning of nested columns as * well. * - Construct a reader function by passing filters and the schema into the FileFormat. - * - Using an partition pruning predicates, enumerate the list of files that should be read. + * - Using a partition pruning predicates, enumerate the list of files that should be read. * - Split the files into tasks and construct a FileScanRDD. * - Add any projection or filters that must be evaluated after the scan. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala index aeee2600a19ee..733fcbfea101e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala @@ -50,7 +50,7 @@ object JacksonParser extends Logging { /** * Parse the current token (and related children) according to a desired schema - * This is an wrapper for the method `convertField()` to handle a row wrapped + * This is a wrapper for the method `convertField()` to handle a row wrapped * with an array. */ def convertRootField( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala index 6bf82bee67881..85b0bc17edd32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala @@ -68,7 +68,7 @@ private[parquet] trait HasParentContainerUpdater { } /** - * A convenient converter class for Parquet group types with an [[HasParentContainerUpdater]]. + * A convenient converter class for Parquet group types with a [[HasParentContainerUpdater]]. */ private[parquet] abstract class CatalystGroupConverter(val updater: ParentContainerUpdater) extends GroupConverter with HasParentContainerUpdater diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index fb60d68f986d6..2ea6ee38a932a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -47,10 +47,10 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} * partitions. * * The workflow of this coordinator is described as follows: - * - Before the execution of a [[SparkPlan]], for an [[ShuffleExchange]] operator, + * - Before the execution of a [[SparkPlan]], for a [[ShuffleExchange]] operator, * if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator. * This happens in the `doPrepare` method. - * - Once we start to execute a physical plan, an [[ShuffleExchange]] registered to this + * - Once we start to execute a physical plan, a [[ShuffleExchange]] registered to this * coordinator will call `postShuffleRDD` to get its corresponding post-shuffle * [[ShuffledRowRDD]]. * If this coordinator has made the decision on how to shuffle data, this [[ShuffleExchange]] @@ -61,7 +61,7 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} * post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices * to a single post-shuffle partition whenever necessary. * - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered - * [[ShuffleExchange]]s. So, when an [[ShuffleExchange]] calls `postShuffleRDD`, this coordinator + * [[ShuffleExchange]]s. So, when a [[ShuffleExchange]] calls `postShuffleRDD`, this coordinator * can lookup the corresponding [[RDD]]. * * The strategy used to determine the number of post-shuffle partitions is described as follows. @@ -98,8 +98,8 @@ private[sql] class ExchangeCoordinator( @volatile private[this] var estimated: Boolean = false /** - * Registers an [[ShuffleExchange]] operator to this coordinator. This method is only allowed to - * be called in the `doPrepare` method of an [[ShuffleExchange]] operator. + * Registers a [[ShuffleExchange]] operator to this coordinator. This method is only allowed to + * be called in the `doPrepare` method of a [[ShuffleExchange]] operator. */ @GuardedBy("this") def registerExchange(exchange: ShuffleExchange): Unit = synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index f0efa52c3d862..32f0bc5bf920d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.util.collection.BitSet /** - * Performs an sort merge join of two child relations. + * Performs a sort merge join of two child relations. */ case class SortMergeJoinExec( leftKeys: Seq[Expression], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index dc6f2ef371584..6c76328c74830 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -40,7 +40,7 @@ private[sql] case class MapPartitionsRWrapper( val (newIter, deserializer, colNames) = if (!isSerializedRData) { - // Serialize each row into an byte array that can be deserialized in the R worker + // Serialize each row into a byte array that can be deserialized in the R worker (iter.asInstanceOf[Iterator[Row]].map {row => rowToRBytes(row)}, SerializationFormats.ROW, inputSchema.fieldNames) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index 48925910ac8cf..eac658c6176cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -133,7 +133,7 @@ abstract class UserDefinedAggregateFunction extends Serializable { /** * :: Experimental :: - * A [[Row]] representing an mutable aggregation buffer. + * A [[Row]] representing a mutable aggregation buffer. * * This is not meant to be extended outside of Spark. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 9f6137d6e3c7c..0d6f98416bd42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -52,7 +52,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) { org.apache.spark.util.Utils.getContextOrSparkClassLoader) /** - * Create a SQLListener then add it into SparkContext, and create an SQLTab if there is SparkUI. + * Create a SQLListener then add it into SparkContext, and create a SQLTab if there is SparkUI. */ private def createListenerAndUI(sc: SparkContext): SQLListener = { if (SparkSession.sqlListener.get() == null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala index 451cfd85e3bc8..3bbb0b8a888fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala @@ -93,7 +93,7 @@ trait ContinuousQuery { def awaitTermination(timeoutMs: Long): Boolean /** - * Blocks until all available data in the source has been processed an committed to the sink. + * Blocks until all available data in the source has been processed and committed to the sink. * This method is intended for testing. Note that in the case of continually arriving data, this * method may block forever. Additionally, this method is only guaranteed to block until data that * has been synchronously appended data to a [[org.apache.spark.sql.execution.streaming.Source]] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index befdda1621074..5f896969188d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -218,7 +218,7 @@ private[hive] trait HiveClient { /** Create a function in an existing database. */ def createFunction(db: String, func: CatalogFunction): Unit - /** Drop an existing function an the database. */ + /** Drop an existing function in the database. */ def dropFunction(db: String, name: String): Unit /** Rename an existing function in the database. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index 0589c8ece3910..f5db73b715820 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.types.StructType private[orc] object OrcFileOperator extends Logging { /** * Retrieves an ORC file reader from a given path. The path can point to either a directory or a - * single ORC file. If it points to an directory, it picks any non-empty ORC file within that + * single ORC file. If it points to a directory, it picks any non-empty ORC file within that * directory. * * The reader returned by this method is mainly used for two purposes: 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 65d53debd67cf..a7652143a4252 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 @@ -37,7 +37,7 @@ import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. * - * The "golden" results from Hive are cached in an retrieved both from the classpath and + * The "golden" results from Hive are cached in and retrieved both from the classpath and * [[answerCache]] to speed up testing. * * See the documentation of public vals in this class for information on how test execution can be diff --git a/streaming/src/main/scala/org/apache/spark/streaming/State.scala b/streaming/src/main/scala/org/apache/spark/streaming/State.scala index 42424d67d8838..3f560f889f055 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/State.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/State.scala @@ -120,7 +120,7 @@ sealed abstract class State[S] { def isTimingOut(): Boolean /** - * Get the state as an [[scala.Option]]. It will be `Some(state)` if it exists, otherwise `None`. + * Get the state as a [[scala.Option]]. It will be `Some(state)` if it exists, otherwise `None`. */ @inline final def getOption(): Option[S] = if (exists) Some(get()) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 7e78fa1d7e159..4c4376a089f59 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -349,7 +349,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { } /** - * Create an input stream from an queue of RDDs. In each batch, + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: @@ -369,7 +369,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { } /** - * Create an input stream from an queue of RDDs. In each batch, + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: @@ -393,7 +393,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { } /** - * Create an input stream from an queue of RDDs. In each batch, + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 147e8c129034b..fa15a0bf65ab9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -157,7 +157,7 @@ abstract class DStream[T: ClassTag] ( def persist(level: StorageLevel): DStream[T] = { if (this.isInitialized) { throw new UnsupportedOperationException( - "Cannot change storage level of an DStream after streaming context has started") + "Cannot change storage level of a DStream after streaming context has started") } this.storageLevel = level this @@ -176,7 +176,7 @@ abstract class DStream[T: ClassTag] ( def checkpoint(interval: Duration): DStream[T] = { if (isInitialized) { throw new UnsupportedOperationException( - "Cannot change checkpoint interval of an DStream after streaming context has started") + "Cannot change checkpoint interval of a DStream after streaming context has started") } persist() checkpointDuration = interval diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala index 47968afef2dbf..8c3a7977beae3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala @@ -31,5 +31,5 @@ private[streaming] case class ArrayBufferBlock(arrayBuffer: ArrayBuffer[_]) exte /** class representing a block received as an Iterator */ private[streaming] case class IteratorBlock(iterator: Iterator[_]) extends ReceivedBlock -/** class representing a block received as an ByteBuffer */ +/** class representing a block received as a ByteBuffer */ private[streaming] case class ByteBufferBlock(byteBuffer: ByteBuffer) extends ReceivedBlock diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index 391a461f08125..4105171a3db24 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -31,7 +31,7 @@ import org.apache.spark.streaming.receiver.Receiver * all receivers at the same time. ReceiverTracker will call `scheduleReceivers` at this phase. * It will try to schedule receivers such that they are evenly distributed. ReceiverTracker * should update its `receiverTrackingInfoMap` according to the results of `scheduleReceivers`. - * `ReceiverTrackingInfo.scheduledLocations` for each receiver should be set to an location list + * `ReceiverTrackingInfo.scheduledLocations` for each receiver should be set to a location list * that contains the scheduled locations. Then when a receiver is starting, it will send a * register request and `ReceiverTracker.registerReceiver` will be called. In * `ReceiverTracker.registerReceiver`, if a receiver's scheduled locations is set, it should diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index bdbac64b9bc79..bd8f9950bf1c7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -71,7 +71,7 @@ trait DStreamCheckpointTester { self: SparkFunSuite => /** * Tests a streaming operation under checkpointing, by restarting the operation * from checkpoint file and verifying whether the final output is correct. - * The output is assumed to have come from a reliable queue which an replay + * The output is assumed to have come from a reliable queue which a replay * data as required. * * NOTE: This takes into consideration that the last batch processed before diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala index a1af95be81c8e..1a0460cd669af 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala @@ -119,7 +119,7 @@ class PIDRateEstimatorSuite extends SparkFunSuite with Matchers { test("with no accumulated but some positive error, |I| > 0, follow the processing speed") { val p = new PIDRateEstimator(20, 1D, 1D, 0D, 10) - // prepare a series of batch updates, one every 20ms with an decreasing number of processed + // prepare a series of batch updates, one every 20ms with a decreasing number of processed // elements in each batch, but constant processing time, and no accumulated error. Even though // the integral part is non-zero, the estimated rate should follow only the proportional term, // asking for less and less elements From 86a35a22985b9e592744e6ef31453995f2322a31 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 6 Jun 2016 09:36:34 +0100 Subject: [PATCH 0537/1470] [SPARK-15771][ML][EXAMPLES] Use 'accuracy' rather than 'precision' in many ML examples ## What changes were proposed in this pull request? Since [SPARK-15617](https://issues.apache.org/jira/browse/SPARK-15617) deprecated ```precision``` in ```MulticlassClassificationEvaluator```, many ML examples broken. ```python pyspark.sql.utils.IllegalArgumentException: u'MulticlassClassificationEvaluator_4c3bb1d73d8cc0cedae6 parameter metricName given invalid value precision.' ``` We should use ```accuracy``` to replace ```precision``` in these examples. ## How was this patch tested? Offline tests. Author: Yanbo Liang Closes #13519 from yanboliang/spark-15771. (cherry picked from commit a95252823e09939b654dd425db38dadc4100bc87) Signed-off-by: Sean Owen --- .../examples/ml/JavaDecisionTreeClassificationExample.java | 2 +- .../ml/JavaGradientBoostedTreeClassifierExample.java | 2 +- .../ml/JavaMultilayerPerceptronClassifierExample.java | 6 +++--- .../org/apache/spark/examples/ml/JavaNaiveBayesExample.java | 6 +++--- .../org/apache/spark/examples/ml/JavaOneVsRestExample.java | 6 +++--- .../examples/ml/JavaRandomForestClassifierExample.java | 2 +- .../main/python/ml/decision_tree_classification_example.py | 2 +- .../python/ml/gradient_boosted_tree_classifier_example.py | 2 +- .../main/python/ml/multilayer_perceptron_classification.py | 6 +++--- examples/src/main/python/ml/naive_bayes_example.py | 6 +++--- examples/src/main/python/ml/one_vs_rest_example.py | 6 +++--- .../src/main/python/ml/random_forest_classifier_example.py | 2 +- .../examples/ml/DecisionTreeClassificationExample.scala | 2 +- .../examples/ml/GradientBoostedTreeClassifierExample.scala | 2 +- .../examples/ml/MultilayerPerceptronClassifierExample.scala | 6 +++--- .../org/apache/spark/examples/ml/NaiveBayesExample.scala | 6 +++--- .../org/apache/spark/examples/ml/OneVsRestExample.scala | 6 +++--- .../spark/examples/ml/RandomForestClassifierExample.scala | 2 +- python/pyspark/ml/evaluation.py | 2 +- 19 files changed, 37 insertions(+), 37 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java index bdb76f004fd68..a9c6e7f0bf6c5 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java @@ -90,7 +90,7 @@ public static void main(String[] args) { MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") - .setMetricName("precision"); + .setMetricName("accuracy"); double accuracy = evaluator.evaluate(predictions); System.out.println("Test Error = " + (1.0 - accuracy)); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java index 5c2e03eda90a9..3e9eb998c8e1f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -92,7 +92,7 @@ public static void main(String[] args) { MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") - .setMetricName("precision"); + .setMetricName("accuracy"); double accuracy = evaluator.evaluate(predictions); System.out.println("Test Error = " + (1.0 - accuracy)); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java index c7d03d8593a3e..0f1d9c26345bd 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java @@ -57,12 +57,12 @@ public static void main(String[] args) { .setMaxIter(100); // train the model MultilayerPerceptronClassificationModel model = trainer.fit(train); - // compute precision on the test set + // compute accuracy on the test set Dataset result = model.transform(test); Dataset predictionAndLabels = result.select("prediction", "label"); MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() - .setMetricName("precision"); - System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels)); + .setMetricName("accuracy"); + System.out.println("Accuracy = " + evaluator.evaluate(predictionAndLabels)); // $example off$ spark.stop(); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java index 50a46a5774fcc..3226d5d2fab6f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java @@ -50,12 +50,12 @@ public static void main(String[] args) { NaiveBayes nb = new NaiveBayes(); // train the model NaiveBayesModel model = nb.fit(train); - // compute precision on the test set + // compute accuracy on the test set Dataset result = model.transform(test); Dataset predictionAndLabels = result.select("prediction", "label"); MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() - .setMetricName("precision"); - System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels)); + .setMetricName("accuracy"); + System.out.println("Accuracy = " + evaluator.evaluate(predictionAndLabels)); // $example off$ spark.stop(); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java index 5bf455ebfed23..c6a083ddc984f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java @@ -71,11 +71,11 @@ public static void main(String[] args) { // obtain evaluator. MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() - .setMetricName("precision"); + .setMetricName("accuracy"); // compute the classification error on test data. - double precision = evaluator.evaluate(predictions); - System.out.println("Test Error : " + (1 - precision)); + double accuracy = evaluator.evaluate(predictions); + System.out.println("Test Error : " + (1 - accuracy)); // $example off$ spark.stop(); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java index 14af2fbbbbbe0..da2633e8860ac 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java @@ -88,7 +88,7 @@ public static void main(String[] args) { MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") - .setMetricName("precision"); + .setMetricName("accuracy"); double accuracy = evaluator.evaluate(predictions); System.out.println("Test Error = " + (1.0 - accuracy)); diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index 9b40b701ecaaf..708f1af6cc6eb 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -66,7 +66,7 @@ # Select (prediction, true label) and compute test error evaluator = MulticlassClassificationEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="precision") + labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") accuracy = evaluator.evaluate(predictions) print("Test Error = %g " % (1.0 - accuracy)) diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index 50026d7b7e3e8..6c2d7e7b810df 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -66,7 +66,7 @@ # Select (prediction, true label) and compute test error evaluator = MulticlassClassificationEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="precision") + labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") accuracy = evaluator.evaluate(predictions) print("Test Error = %g" % (1.0 - accuracy)) diff --git a/examples/src/main/python/ml/multilayer_perceptron_classification.py b/examples/src/main/python/ml/multilayer_perceptron_classification.py index 8bededc14db4e..aa33bef5a3ddd 100644 --- a/examples/src/main/python/ml/multilayer_perceptron_classification.py +++ b/examples/src/main/python/ml/multilayer_perceptron_classification.py @@ -43,11 +43,11 @@ trainer = MultilayerPerceptronClassifier(maxIter=100, layers=layers, blockSize=128, seed=1234) # train the model model = trainer.fit(train) - # compute precision on the test set + # compute accuracy on the test set result = model.transform(test) predictionAndLabels = result.select("prediction", "label") - evaluator = MulticlassClassificationEvaluator(metricName="precision") - print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) + evaluator = MulticlassClassificationEvaluator(metricName="accuracy") + print("Accuracy: " + str(evaluator.evaluate(predictionAndLabels))) # $example off$ spark.stop() diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py index 89255a2bae64c..8bc32222fe32b 100644 --- a/examples/src/main/python/ml/naive_bayes_example.py +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -43,11 +43,11 @@ # train the model model = nb.fit(train) - # compute precision on the test set + # compute accuracy on the test set result = model.transform(test) predictionAndLabels = result.select("prediction", "label") - evaluator = MulticlassClassificationEvaluator(metricName="precision") - print("Precision:" + str(evaluator.evaluate(predictionAndLabels))) + evaluator = MulticlassClassificationEvaluator(metricName="accuracy") + print("Accuracy: " + str(evaluator.evaluate(predictionAndLabels))) # $example off$ spark.stop() diff --git a/examples/src/main/python/ml/one_vs_rest_example.py b/examples/src/main/python/ml/one_vs_rest_example.py index 971156d0dd293..b82087bebae8a 100644 --- a/examples/src/main/python/ml/one_vs_rest_example.py +++ b/examples/src/main/python/ml/one_vs_rest_example.py @@ -58,11 +58,11 @@ predictions = ovrModel.transform(test) # obtain evaluator. - evaluator = MulticlassClassificationEvaluator(metricName="precision") + evaluator = MulticlassClassificationEvaluator(metricName="accuracy") # compute the classification error on test data. - precision = evaluator.evaluate(predictions) - print("Test Error : " + str(1 - precision)) + accuracy = evaluator.evaluate(predictions) + print("Test Error : " + str(1 - accuracy)) # $example off$ spark.stop() diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index c618eaf60c2e9..a7fc765318b99 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -66,7 +66,7 @@ # Select (prediction, true label) and compute test error evaluator = MulticlassClassificationEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="precision") + labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") accuracy = evaluator.evaluate(predictions) print("Test Error = %g" % (1.0 - accuracy)) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala index b3103ced91458..bc6d3275933ea 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala @@ -81,7 +81,7 @@ object DecisionTreeClassificationExample { val evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") - .setMetricName("precision") + .setMetricName("accuracy") val accuracy = evaluator.evaluate(predictions) println("Test Error = " + (1.0 - accuracy)) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala index 0d1ffbe2259c4..9a39acfbf37e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala @@ -83,7 +83,7 @@ object GradientBoostedTreeClassifierExample { val evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") - .setMetricName("precision") + .setMetricName("accuracy") val accuracy = evaluator.evaluate(predictions) println("Test Error = " + (1.0 - accuracy)) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala index 0e780fb7d342e..e8a9b32da9664 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala @@ -55,12 +55,12 @@ object MultilayerPerceptronClassifierExample { .setMaxIter(100) // train the model val model = trainer.fit(train) - // compute precision on the test set + // compute accuracy on the test set val result = model.transform(test) val predictionAndLabels = result.select("prediction", "label") val evaluator = new MulticlassClassificationEvaluator() - .setMetricName("precision") - println("Precision:" + evaluator.evaluate(predictionAndLabels)) + .setMetricName("accuracy") + println("Accuracy: " + evaluator.evaluate(predictionAndLabels)) // $example off$ spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala index 90cdebfcb08de..a59ba182fc208 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -49,9 +49,9 @@ object NaiveBayesExample { val evaluator = new MulticlassClassificationEvaluator() .setLabelCol("label") .setPredictionCol("prediction") - .setMetricName("precision") - val precision = evaluator.evaluate(predictions) - println("Precision:" + precision) + .setMetricName("accuracy") + val accuracy = evaluator.evaluate(predictions) + println("Accuracy: " + accuracy) // $example off$ spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index 0da8e3137a597..acde110683950 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -65,11 +65,11 @@ object OneVsRestExample { // obtain evaluator. val evaluator = new MulticlassClassificationEvaluator() - .setMetricName("precision") + .setMetricName("accuracy") // compute the classification error on test data. - val precision = evaluator.evaluate(predictions) - println(s"Test Error : ${1 - precision}") + val accuracy = evaluator.evaluate(predictions) + println(s"Test Error : ${1 - accuracy}") // $example off$ spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala index cccc4a6ea26b5..5eafda8ce4285 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala @@ -83,7 +83,7 @@ object RandomForestClassifierExample { val evaluator = new MulticlassClassificationEvaluator() .setLabelCol("indexedLabel") .setPredictionCol("prediction") - .setMetricName("precision") + .setMetricName("accuracy") val accuracy = evaluator.evaluate(predictions) println("Test Error = " + (1.0 - accuracy)) diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index c480525e9bd3a..cd071f1b7c7ef 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -265,7 +265,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio """ metricName = Param(Params._dummy(), "metricName", "metric name in evaluation " - "(f1|precision|recall|weightedPrecision|weightedRecall|accuracy)", + "(f1|weightedPrecision|weightedRecall|accuracy)", typeConverter=TypeConverters.toString) @keyword_only From e38ff70e6bacf1c85edc390d28f8a8d5ecc6cbc3 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Mon, 6 Jun 2016 15:19:22 +0100 Subject: [PATCH 0538/1470] [SPARK-14900][ML][PYSPARK] Add accuracy and deprecate precison,recall,f1 ## What changes were proposed in this pull request? 1, add accuracy for MulticlassMetrics 2, deprecate overall precision,recall,f1 and recommend accuracy usage ## How was this patch tested? manual tests in pyspark shell Author: Zheng RuiFeng Closes #13511 from zhengruifeng/deprecate_py_precisonrecall. (cherry picked from commit 00ad4f054cd044e17d29b7c2c62efd8616462619) Signed-off-by: Sean Owen --- python/pyspark/mllib/evaluation.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 5f32f092c7cbe..2eaac87b6774b 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -15,6 +15,8 @@ # limitations under the License. # +import warnings + from pyspark import since from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc from pyspark.sql import SQLContext @@ -181,6 +183,8 @@ class MulticlassMetrics(JavaModelWrapper): 0.66... >>> metrics.recall() 0.66... + >>> metrics.accuracy() + 0.66... >>> metrics.weightedFalsePositiveRate 0.19... >>> metrics.weightedPrecision @@ -233,6 +237,8 @@ def precision(self, label=None): Returns precision or precision for a given label (category) if specified. """ if label is None: + # note:: Deprecated in 2.0.0. Use accuracy. + warnings.warn("Deprecated in 2.0.0. Use accuracy.") return self.call("precision") else: return self.call("precision", float(label)) @@ -243,6 +249,8 @@ def recall(self, label=None): Returns recall or recall for a given label (category) if specified. """ if label is None: + # note:: Deprecated in 2.0.0. Use accuracy. + warnings.warn("Deprecated in 2.0.0. Use accuracy.") return self.call("recall") else: return self.call("recall", float(label)) @@ -254,6 +262,8 @@ def fMeasure(self, label=None, beta=None): """ if beta is None: if label is None: + # note:: Deprecated in 2.0.0. Use accuracy. + warnings.warn("Deprecated in 2.0.0. Use accuracy.") return self.call("fMeasure") else: return self.call("fMeasure", label) @@ -263,6 +273,14 @@ def fMeasure(self, label=None, beta=None): else: return self.call("fMeasure", label, beta) + @since('2.0.0') + def accuracy(self): + """ + Returns accuracy (equals to the total number of correctly classified instances + out of the total number of instances). + """ + return self.call("accuracy") + @property @since('1.4.0') def weightedTruePositiveRate(self): From d07bce49fc77aff25330920cc55b7079a3a2995c Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 6 Jun 2016 09:49:45 -0700 Subject: [PATCH 0539/1470] [SPARK-15721][ML] Make DefaultParamsReadable, DefaultParamsWritable public ## What changes were proposed in this pull request? Made DefaultParamsReadable, DefaultParamsWritable public. Also added relevant doc and annotations. Added UnaryTransformerExample to demonstrate use of UnaryTransformer and DefaultParamsReadable,Writable. ## How was this patch tested? Wrote example making use of the now-public APIs. Compiled and ran locally Author: Joseph K. Bradley Closes #13461 from jkbradley/defaultparamswritable. (cherry picked from commit 4c74ee8d8e1c3139d3d322ae68977f2ab53295df) Signed-off-by: Joseph K. Bradley --- .../examples/ml/UnaryTransformerExample.scala | 122 ++++++++++++++++++ .../org/apache/spark/ml/util/ReadWrite.scala | 44 ++++++- 2 files changed, 163 insertions(+), 3 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala new file mode 100644 index 0000000000000..13c72f88cc83b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala @@ -0,0 +1,122 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.param.DoubleParam +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} +import org.apache.spark.sql.functions.col +// $example off$ +import org.apache.spark.sql.SparkSession +// $example on$ +import org.apache.spark.sql.types.{DataType, DataTypes} +import org.apache.spark.util.Utils +// $example off$ + +/** + * An example demonstrating creating a custom [[org.apache.spark.ml.Transformer]] using + * the [[UnaryTransformer]] abstraction. + * + * Run with + * {{{ + * bin/run-example ml.UnaryTransformerExample + * }}} + */ +object UnaryTransformerExample { + + // $example on$ + /** + * Simple Transformer which adds a constant value to input Doubles. + * + * [[UnaryTransformer]] can be used to create a stage usable within Pipelines. + * It defines parameters for specifying input and output columns: + * [[UnaryTransformer.inputCol]] and [[UnaryTransformer.outputCol]]. + * It can optionally handle schema validation. + * + * [[DefaultParamsWritable]] provides a default implementation for persisting instances + * of this Transformer. + */ + class MyTransformer(override val uid: String) + extends UnaryTransformer[Double, Double, MyTransformer] with DefaultParamsWritable { + + final val shift: DoubleParam = new DoubleParam(this, "shift", "Value added to input") + + def getShift: Double = $(shift) + + def setShift(value: Double): this.type = set(shift, value) + + def this() = this(Identifiable.randomUID("myT")) + + override protected def createTransformFunc: Double => Double = (input: Double) => { + input + $(shift) + } + + override protected def outputDataType: DataType = DataTypes.DoubleType + + override protected def validateInputType(inputType: DataType): Unit = { + require(inputType == DataTypes.DoubleType, s"Bad input type: $inputType. Requires Double.") + } + } + + /** + * Companion object for our simple Transformer. + * + * [[DefaultParamsReadable]] provides a default implementation for loading instances + * of this Transformer which were persisted using [[DefaultParamsWritable]]. + */ + object MyTransformer extends DefaultParamsReadable[MyTransformer] + // $example off$ + + def main(args: Array[String]) { + val spark = SparkSession + .builder() + .appName("UnaryTransformerExample") + .getOrCreate() + + // $example on$ + val myTransformer = new MyTransformer() + .setShift(0.5) + .setInputCol("input") + .setOutputCol("output") + + // Create data, transform, and display it. + val data = spark.range(0, 5).toDF("input") + .select(col("input").cast("double").as("input")) + val result = myTransformer.transform(data) + result.show() + + // Save and load the Transformer. + val tmpDir = Utils.createTempDir() + val dirName = tmpDir.getCanonicalPath + myTransformer.write.overwrite().save(dirName) + val sameTransformer = MyTransformer.load(dirName) + + // Transform the data to show the results are identical. + val sameResult = sameTransformer.transform(data) + sameResult.show() + + Utils.deleteRecursively(tmpDir) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println + diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index 8ed40c379c123..90b8d7df7b496 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -68,6 +68,8 @@ private[util] sealed trait BaseReadWrite { } /** + * :: Experimental :: + * * Abstract class for utility classes that can save ML instances. */ @Experimental @@ -120,8 +122,11 @@ abstract class MLWriter extends BaseReadWrite with Logging { } /** + * :: Experimental :: + * * Trait for classes that provide [[MLWriter]]. */ +@Experimental @Since("1.6.0") trait MLWritable { @@ -139,12 +144,27 @@ trait MLWritable { def save(path: String): Unit = write.save(path) } -private[ml] trait DefaultParamsWritable extends MLWritable { self: Params => +/** + * :: Experimental :: + * + * Helper trait for making simple [[Params]] types writable. If a [[Params]] class stores + * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide + * a default implementation of writing saved instances of the class. + * This only handles simple [[org.apache.spark.ml.param.Param]] types; e.g., it will not handle + * [[org.apache.spark.sql.Dataset]]. + * + * @see [[DefaultParamsReadable]], the counterpart to this trait + */ +@Experimental +@Since("2.0.0") +trait DefaultParamsWritable extends MLWritable { self: Params => override def write: MLWriter = new DefaultParamsWriter(this) } /** + * :: Experimental :: + * * Abstract class for utility classes that can load ML instances. * * @tparam T ML instance type @@ -164,6 +184,8 @@ abstract class MLReader[T] extends BaseReadWrite { } /** + * :: Experimental :: + * * Trait for objects that provide [[MLReader]]. * * @tparam T ML instance type @@ -187,9 +209,25 @@ trait MLReadable[T] { def load(path: String): T = read.load(path) } -private[ml] trait DefaultParamsReadable[T] extends MLReadable[T] { - override def read: MLReader[T] = new DefaultParamsReader +/** + * :: Experimental :: + * + * Helper trait for making simple [[Params]] types readable. If a [[Params]] class stores + * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide + * a default implementation of reading saved instances of the class. + * This only handles simple [[org.apache.spark.ml.param.Param]] types; e.g., it will not handle + * [[org.apache.spark.sql.Dataset]]. + * + * @tparam T ML instance type + * + * @see [[DefaultParamsWritable]], the counterpart to this trait + */ +@Experimental +@Since("2.0.0") +trait DefaultParamsReadable[T] extends MLReadable[T] { + + override def read: MLReader[T] = new DefaultParamsReader[T] } /** From 5363783af7d93b7597181c8b39b83800fa059543 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 6 Jun 2016 11:44:51 -0700 Subject: [PATCH 0540/1470] [SPARK-15764][SQL] Replace N^2 loop in BindReferences BindReferences contains a n^2 loop which causes performance issues when operating over large schemas: to determine the ordinal of an attribute reference, we perform a linear scan over the `input` array. Because input can sometimes be a `List`, the call to `input(ordinal).nullable` can also be O(n). Instead of performing a linear scan, we can convert the input into an array and build a hash map to map from expression ids to ordinals. The greater up-front cost of the map construction is offset by the fact that an expression can contain multiple attribute references, so the cost of the map construction is amortized across a number of lookups. Perf. benchmarks to follow. /cc ericl Author: Josh Rosen Closes #13505 from JoshRosen/bind-references-improvement. (cherry picked from commit 0b8d694999b43ada4833388cad6c285c7757cbf7) Signed-off-by: Josh Rosen --- .../catalyst/expressions/AttributeMap.scala | 7 ---- .../catalyst/expressions/BoundAttribute.scala | 6 ++-- .../sql/catalyst/expressions/package.scala | 34 ++++++++++++++++++- .../spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../aggregate/HashAggregateExec.scala | 2 +- .../columnar/InMemoryTableScanExec.scala | 4 +-- 6 files changed, 40 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala index ef3cc554b79c0..96a11e352ec50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -26,13 +26,6 @@ object AttributeMap { def apply[A](kvs: Seq[(Attribute, A)]): AttributeMap[A] = { new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) } - - /** Given a schema, constructs an [[AttributeMap]] from [[Attribute]] to ordinal */ - def byIndex(schema: Seq[Attribute]): AttributeMap[Int] = apply(schema.zipWithIndex) - - /** Given a schema, constructs a map from ordinal to Attribute. */ - def toIndex(schema: Seq[Attribute]): Map[Int, Attribute] = - schema.zipWithIndex.map { case (a, i) => i -> a }.toMap } class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index a38f1ec09156d..7d16118c9d59f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -82,16 +82,16 @@ object BindReferences extends Logging { def bindReference[A <: Expression]( expression: A, - input: Seq[Attribute], + input: AttributeSeq, allowFailures: Boolean = false): A = { expression.transform { case a: AttributeReference => attachTree(a, "Binding attribute") { - val ordinal = input.indexWhere(_.exprId == a.exprId) + val ordinal = input.indexOf(a.exprId) if (ordinal == -1) { if (allowFailures) { a } else { - sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") + sys.error(s"Couldn't find $a in ${input.attrs.mkString("[", ",", "]")}") } } else { BoundReference(ordinal, a.dataType, input(ordinal).nullable) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 23baa6f7837fb..81f5bb4a65096 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst +import com.google.common.collect.Maps + import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -86,11 +88,41 @@ package object expressions { /** * Helper functions for working with `Seq[Attribute]`. */ - implicit class AttributeSeq(attrs: Seq[Attribute]) { + implicit class AttributeSeq(val attrs: Seq[Attribute]) extends Serializable { /** Creates a StructType with a schema matching this `Seq[Attribute]`. */ def toStructType: StructType = { StructType(attrs.map(a => StructField(a.name, a.dataType, a.nullable))) } + + // It's possible that `attrs` is a linked list, which can lead to bad O(n^2) loops when + // accessing attributes by their ordinals. To avoid this performance penalty, convert the input + // to an array. + @transient private lazy val attrsArray = attrs.toArray + + @transient private lazy val exprIdToOrdinal = { + val arr = attrsArray + val map = Maps.newHashMapWithExpectedSize[ExprId, Int](arr.length) + // Iterate over the array in reverse order so that the final map value is the first attribute + // with a given expression id. + var index = arr.length - 1 + while (index >= 0) { + map.put(arr(index).exprId, index) + index -= 1 + } + map + } + + /** + * Returns the attribute at the given index. + */ + def apply(ordinal: Int): Attribute = attrsArray(ordinal) + + /** + * Returns the index of first attribute with a matching expression id, or -1 if no match exists. + */ + def indexOf(exprId: ExprId): Int = { + Option(exprIdToOrdinal.get(exprId)).getOrElse(-1) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 3de15a9a3f544..19a66cff4fae4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -296,7 +296,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT /** * All the attributes that are used for this plan. */ - lazy val allAttributes: Seq[Attribute] = children.flatMap(_.output) + lazy val allAttributes: AttributeSeq = children.flatMap(_.output) private def cleanExpression(e: Expression): Expression = e match { case a: Alias => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index f5bc0628b6458..f270ca07554f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -49,7 +49,7 @@ case class HashAggregateExec( require(HashAggregateExec.supportsAggregate(aggregateBufferAttributes)) - override lazy val allAttributes: Seq[Attribute] = + override lazy val allAttributes: AttributeSeq = child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index bd55e1a8751da..a1c2f0a8fbcf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -310,7 +310,7 @@ private[sql] case class InMemoryTableScanExec( // within the map Partitions closure. val schema = relation.partitionStatistics.schema val schemaIndex = schema.zipWithIndex - val relOutput = relation.output + val relOutput: AttributeSeq = relation.output val buffers = relation.cachedColumnBuffers buffers.mapPartitionsInternal { cachedBatchIterator => @@ -321,7 +321,7 @@ private[sql] case class InMemoryTableScanExec( // Find the ordinals and data types of the requested columns. val (requestedColumnIndices, requestedColumnDataTypes) = attributes.map { a => - relOutput.indexWhere(_.exprId == a.exprId) -> a.dataType + relOutput.indexOf(a.exprId) -> a.dataType }.unzip // Do partition batch pruning if enabled From 62765cbebe0cb41b0c4fdc344828337ee15e1fd2 Mon Sep 17 00:00:00 2001 From: Subroto Sanyal Date: Mon, 6 Jun 2016 16:05:40 -0700 Subject: [PATCH 0541/1470] [SPARK-15652][LAUNCHER] Added a new State (LOST) for the listeners of SparkLauncher ## What changes were proposed in this pull request? This situation can happen when the LauncherConnection gets an exception while reading through the socket and terminating silently without notifying making the client/listener think that the job is still in previous state. The fix force sends a notification to client that the job finished with unknown status and let client handle it accordingly. ## How was this patch tested? Added a unit test. Author: Subroto Sanyal Closes #13497 from subrotosanyal/SPARK-15652-handle-spark-submit-jvm-crash. (cherry picked from commit c409e23abd128dad33557025f1e824ef47e6222f) Signed-off-by: Marcelo Vanzin --- .../apache/spark/launcher/LauncherServer.java | 4 +++ .../apache/spark/launcher/SparkAppHandle.java | 4 ++- .../spark/launcher/LauncherServerSuite.java | 31 +++++++++++++++++++ 3 files changed, 38 insertions(+), 1 deletion(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index e3413fd6652d8..28e9420b28802 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -337,6 +337,10 @@ public void close() throws IOException { } super.close(); if (handle != null) { + if (!handle.getState().isFinal()) { + LOG.log(Level.WARNING, "Lost connection to spark application."); + handle.setState(SparkAppHandle.State.LOST); + } handle.disconnect(); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java index 625d02632114a..0aa7bd197d16f 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java @@ -46,7 +46,9 @@ enum State { /** The application finished with a failed status. */ FAILED(true), /** The application was killed. */ - KILLED(true); + KILLED(true), + /** The Spark Submit JVM exited with a unknown status. */ + LOST(true); private final boolean isFinal; diff --git a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java index bfe1fcc87fe35..12f1a0ce2d1b4 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java @@ -152,6 +152,37 @@ public void testTimeout() throws Exception { } } + @Test + public void testSparkSubmitVmShutsDown() throws Exception { + ChildProcAppHandle handle = LauncherServer.newAppHandle(); + TestClient client = null; + final Semaphore semaphore = new Semaphore(0); + try { + Socket s = new Socket(InetAddress.getLoopbackAddress(), + LauncherServer.getServerInstance().getPort()); + handle.addListener(new SparkAppHandle.Listener() { + public void stateChanged(SparkAppHandle handle) { + semaphore.release(); + } + public void infoChanged(SparkAppHandle handle) { + semaphore.release(); + } + }); + client = new TestClient(s); + client.send(new Hello(handle.getSecret(), "1.4.0")); + assertTrue(semaphore.tryAcquire(30, TimeUnit.SECONDS)); + // Make sure the server matched the client to the handle. + assertNotNull(handle.getConnection()); + close(client); + assertTrue(semaphore.tryAcquire(30, TimeUnit.SECONDS)); + assertEquals(SparkAppHandle.State.LOST, handle.getState()); + } finally { + kill(handle); + close(client); + client.clientThread.join(); + } + } + private void kill(SparkAppHandle handle) { if (handle != null) { handle.kill(); From a5bec5b81d9e8ce17f1ce509731b030f0f3538e3 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Mon, 6 Jun 2016 22:40:21 -0700 Subject: [PATCH 0542/1470] [SPARK-15632][SQL] Typed Filter should NOT change the Dataset schema ## What changes were proposed in this pull request? This PR makes sure the typed Filter doesn't change the Dataset schema. **Before the change:** ``` scala> val df = spark.range(0,9) scala> df.schema res12: org.apache.spark.sql.types.StructType = StructType(StructField(id,LongType,false)) scala> val afterFilter = df.filter(_=>true) scala> afterFilter.schema // !!! schema is CHANGED!!! Column name is changed from id to value, nullable is changed from false to true. res13: org.apache.spark.sql.types.StructType = StructType(StructField(value,LongType,true)) ``` SerializeFromObject and DeserializeToObject are inserted to wrap the Filter, and these two can possibly change the schema of Dataset. **After the change:** ``` scala> afterFilter.schema // schema is NOT changed. res47: org.apache.spark.sql.types.StructType = StructType(StructField(id,LongType,false)) ``` ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13529 from clockfly/spark-15632. (cherry picked from commit 0e0904a2fce3c4447c24f1752307b6d01ffbd0ad) Signed-off-by: Cheng Lian --- .../optimizer/TypedFilterOptimizationSuite.scala | 4 +++- .../scala/org/apache/spark/sql/Dataset.scala | 16 ++++++++-------- .../org/apache/spark/sql/JavaDatasetSuite.java | 13 +++++++++++++ .../org/apache/spark/sql/DatasetSuite.scala | 6 ++++++ .../sql/execution/WholeStageCodegenSuite.scala | 2 +- 5 files changed, 31 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala index 289c16aef47aa..63d87bfb6d24d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala @@ -57,7 +57,9 @@ class TypedFilterOptimizationSuite extends PlanTest { comparePlans(optimized, expected) } - test("embed deserializer in filter condition if there is only one filter") { + // TODO: Remove this after we completely fix SPARK-15632 by adding optimization rules + // for typed filters. + ignore("embed deserializer in typed filter condition if there is only one filter") { val input = LocalRelation('_1.int, '_2.int) val f = (i: (Int, Int)) => i._1 > 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 96c871d034355..6cbc27d91c1e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1944,11 +1944,11 @@ class Dataset[T] private[sql]( */ @Experimental def filter(func: T => Boolean): Dataset[T] = { - val deserialized = CatalystSerde.deserialize[T](logicalPlan) + val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer) val function = Literal.create(func, ObjectType(classOf[T => Boolean])) - val condition = Invoke(function, "apply", BooleanType, deserialized.output) - val filter = Filter(condition, deserialized) - withTypedPlan(CatalystSerde.serialize[T](filter)) + val condition = Invoke(function, "apply", BooleanType, deserializer :: Nil) + val filter = Filter(condition, logicalPlan) + withTypedPlan(filter) } /** @@ -1961,11 +1961,11 @@ class Dataset[T] private[sql]( */ @Experimental def filter(func: FilterFunction[T]): Dataset[T] = { - val deserialized = CatalystSerde.deserialize[T](logicalPlan) + val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer) val function = Literal.create(func, ObjectType(classOf[FilterFunction[T]])) - val condition = Invoke(function, "call", BooleanType, deserialized.output) - val filter = Filter(condition, deserialized) - withTypedPlan(CatalystSerde.serialize[T](filter)) + val condition = Invoke(function, "call", BooleanType, deserializer :: Nil) + val filter = Filter(condition, logicalPlan) + withTypedPlan(filter) } /** diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 8354a5bdac68f..37577accfda21 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -92,6 +92,19 @@ public void testToLocalIterator() { Assert.assertFalse(iter.hasNext()); } + // SPARK-15632: typed filter should preserve the underlying logical schema + @Test + public void testTypedFilterPreservingSchema() { + Dataset ds = spark.range(10); + Dataset ds2 = ds.filter(new FilterFunction() { + @Override + public boolean call(Long value) throws Exception { + return value > 3; + } + }); + Assert.assertEquals(ds.schema(), ds2.schema()); + } + @Test public void testCommonOperation() { List data = Arrays.asList("hello", "world"); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index bf2b0a2c7c1b7..11b52bdead7b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -225,6 +225,12 @@ class DatasetSuite extends QueryTest with SharedSQLContext { "b") } + test("SPARK-15632: typed filter should preserve the underlying logical schema") { + val ds = spark.range(10) + val ds2 = ds.filter(_ > 3) + assert(ds.schema.equals(ds2.schema)) + } + test("foreach") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.longAccumulator diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 68f0ee864f47f..f26e5e7b6990d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -97,7 +97,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val plan = ds.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SerializeFromObjectExec]).isDefined) + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[FilterExec]).isDefined) assert(ds.collect() === Array(0, 6)) } From 57dd4efcda9158646df41ea8d70754dc110ecd6f Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Mon, 6 Jun 2016 22:59:25 -0700 Subject: [PATCH 0543/1470] [SPARK-15792][SQL] Allows operator to change the verbosity in explain output ## What changes were proposed in this pull request? This PR allows customization of verbosity in explain output. After change, `dataframe.explain()` and `dataframe.explain(true)` has different verbosity output for physical plan. Currently, this PR only enables verbosity string for operator `HashAggregateExec` and `SortAggregateExec`. We will gradually enable verbosity string for more operators in future. **Less verbose mode:** dataframe.explain(extended = false) `output=[count(a)#85L]` is **NOT** displayed for HashAggregate. ``` scala> Seq((1,2,3)).toDF("a", "b", "c").createTempView("df2") scala> spark.sql("select count(a) from df2").explain() == Physical Plan == *HashAggregate(key=[], functions=[count(1)]) +- Exchange SinglePartition +- *HashAggregate(key=[], functions=[partial_count(1)]) +- LocalTableScan ``` **Verbose mode:** dataframe.explain(extended = true) `output=[count(a)#85L]` is displayed for HashAggregate. ``` scala> spark.sql("select count(a) from df2").explain(true) // "output=[count(a)#85L]" is added ... == Physical Plan == *HashAggregate(key=[], functions=[count(1)], output=[count(a)#85L]) +- Exchange SinglePartition +- *HashAggregate(key=[], functions=[partial_count(1)], output=[count#87L]) +- LocalTableScan ``` ## How was this patch tested? Manual test. Author: Sean Zhong Closes #13535 from clockfly/verbose_breakdown_2. (cherry picked from commit 5f731d6859c4516941e5f90c99c966ef76268864) Signed-off-by: Cheng Lian --- .../sql/catalyst/expressions/Expression.scala | 4 ++++ .../spark/sql/catalyst/plans/QueryPlan.scala | 2 ++ .../spark/sql/catalyst/trees/TreeNode.scala | 23 ++++++++++++++----- .../spark/sql/execution/QueryExecution.scala | 14 ++++++----- .../sql/execution/WholeStageCodegenExec.scala | 6 +++-- .../aggregate/HashAggregateExec.scala | 12 ++++++++-- .../aggregate/SortAggregateExec.scala | 12 ++++++++-- 7 files changed, 55 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 2ec46216e1cdb..efe592d69ddc8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -190,6 +190,10 @@ abstract class Expression extends TreeNode[Expression] { case single => single :: Nil } + // Marks this as final, Expression.verboseString should never be called, and thus shouldn't be + // overridden by concrete classes. + final override def verboseString: String = simpleString + override def simpleString: String = toString override def toString: String = prettyName + flatArguments.mkString("(", ", ", ")") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 19a66cff4fae4..cf34f4b30d8d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -257,6 +257,8 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT override def simpleString: String = statePrefix + super.simpleString + override def verboseString: String = simpleString + /** * All the subqueries of current plan. */ 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 22d82c61080c0..c67366d240a30 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 @@ -462,10 +462,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { /** ONE line description of this node. */ def simpleString: String = s"$nodeName $argString".trim + /** ONE line description of this node with more information */ + def verboseString: String + override def toString: String = treeString /** Returns a string representation of the nodes in this tree */ - def treeString: String = generateTreeString(0, Nil, new StringBuilder).toString + def treeString: String = treeString(verbose = true) + + def treeString(verbose: Boolean): String = { + generateTreeString(0, Nil, new StringBuilder, verbose).toString + } /** * Returns a string representation of the nodes in this tree, where each operator is numbered. @@ -508,6 +515,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder, + verbose: Boolean, prefix: String = ""): StringBuilder = { if (depth > 0) { lastChildren.init.foreach { isLast => @@ -520,18 +528,21 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } builder.append(prefix) - builder.append(simpleString) + val headline = if (verbose) verboseString else simpleString + builder.append(headline) builder.append("\n") if (innerChildren.nonEmpty) { innerChildren.init.foreach(_.generateTreeString( - depth + 2, lastChildren :+ false :+ false, builder)) - innerChildren.last.generateTreeString(depth + 2, lastChildren :+ false :+ true, builder) + depth + 2, lastChildren :+ false :+ false, builder, verbose)) + innerChildren.last.generateTreeString( + depth + 2, lastChildren :+ false :+ true, builder, verbose) } if (children.nonEmpty) { - children.init.foreach(_.generateTreeString(depth + 1, lastChildren :+ false, builder, prefix)) - children.last.generateTreeString(depth + 1, lastChildren :+ true, builder, prefix) + children.init.foreach( + _.generateTreeString(depth + 1, lastChildren :+ false, builder, verbose, prefix)) + children.last.generateTreeString(depth + 1, lastChildren :+ true, builder, verbose, prefix) } builder diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 330459c11ea98..560214a65e6ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -201,24 +201,26 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def simpleString: String = { s"""== Physical Plan == - |${stringOrError(executedPlan)} + |${stringOrError(executedPlan.treeString(verbose = false))} """.stripMargin.trim } override def toString: String = { def output = analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}").mkString(", ") - val analyzedPlan = - Seq(stringOrError(output), stringOrError(analyzed)).filter(_.nonEmpty).mkString("\n") + val analyzedPlan = Seq( + stringOrError(output), + stringOrError(analyzed.treeString(verbose = true)) + ).filter(_.nonEmpty).mkString("\n") s"""== Parsed Logical Plan == - |${stringOrError(logical)} + |${stringOrError(logical.treeString(verbose = true))} |== Analyzed Logical Plan == |$analyzedPlan |== Optimized Logical Plan == - |${stringOrError(optimizedPlan)} + |${stringOrError(optimizedPlan.treeString(verbose = true))} |== Physical Plan == - |${stringOrError(executedPlan)} + |${stringOrError(executedPlan.treeString(verbose = true))} """.stripMargin.trim } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index d3e8d4e8e41a2..e0d8e35713291 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -250,8 +250,9 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder, + verbose: Boolean, prefix: String = ""): StringBuilder = { - child.generateTreeString(depth, lastChildren, builder, "") + child.generateTreeString(depth, lastChildren, builder, verbose, "") } } @@ -407,8 +408,9 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co depth: Int, lastChildren: Seq[Boolean], builder: StringBuilder, + verbose: Boolean, prefix: String = ""): StringBuilder = { - child.generateTreeString(depth, lastChildren, builder, "*") + child.generateTreeString(depth, lastChildren, builder, verbose, "*") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index f270ca07554f5..b617e26418c7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -764,7 +764,11 @@ case class HashAggregateExec( """ } - override def simpleString: String = { + override def verboseString: String = toString(verbose = true) + + override def simpleString: String = toString(verbose = false) + + private def toString(verbose: Boolean): String = { val allAggregateExpressions = aggregateExpressions testFallbackStartsAt match { @@ -772,7 +776,11 @@ case class HashAggregateExec( val keyString = groupingExpressions.mkString("[", ",", "]") val functionString = allAggregateExpressions.mkString("[", ",", "]") val outputString = output.mkString("[", ",", "]") - s"HashAggregate(key=$keyString, functions=$functionString, output=$outputString)" + if (verbose) { + s"HashAggregate(key=$keyString, functions=$functionString, output=$outputString)" + } else { + s"HashAggregate(key=$keyString, functions=$functionString)" + } case Some(fallbackStartsAt) => s"HashAggregateWithControlledFallback $groupingExpressions " + s"$allAggregateExpressions $resultExpressions fallbackStartsAt=$fallbackStartsAt" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 9e48ff8d707bd..41ba9f5b3feda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -103,12 +103,20 @@ case class SortAggregateExec( } } - override def simpleString: String = { + override def simpleString: String = toString(verbose = false) + + override def verboseString: String = toString(verbose = true) + + private def toString(verbose: Boolean): String = { val allAggregateExpressions = aggregateExpressions val keyString = groupingExpressions.mkString("[", ",", "]") val functionString = allAggregateExpressions.mkString("[", ",", "]") val outputString = output.mkString("[", ",", "]") - s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" + if (verbose) { + s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" + } else { + s"SortAggregate(key=$keyString, functions=$functionString)" + } } } From a7e9e60df5c10a90c06883ea3203ec895b9b1f82 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 7 Jun 2016 13:29:27 +0100 Subject: [PATCH 0544/1470] [MINOR] fix typo in documents ## What changes were proposed in this pull request? I use spell check tools checks typo in spark documents and fix them. ## How was this patch tested? N/A Author: WeichenXu Closes #13538 from WeichenXu123/fix_doc_typo. (cherry picked from commit 1e2c9311871968426e019164b129652fd6d0037f) Signed-off-by: Sean Owen --- docs/graphx-programming-guide.md | 2 +- docs/hardware-provisioning.md | 2 +- docs/streaming-programming-guide.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 9dea9b5904d2d..81cf17475fb60 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -132,7 +132,7 @@ var graph: Graph[VertexProperty, String] = null Like RDDs, property graphs are immutable, distributed, and fault-tolerant. Changes to the values or structure of the graph are accomplished by producing a new graph with the desired changes. Note -that substantial parts of the original graph (i.e., unaffected structure, attributes, and indicies) +that substantial parts of the original graph (i.e., unaffected structure, attributes, and indices) are reused in the new graph reducing the cost of this inherently functional data structure. The graph is partitioned across the executors using a range of vertex partitioning heuristics. As with RDDs, each partition of the graph can be recreated on a different machine in the event of a failure. diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md index 60ecb4f483afa..bb6f616b18a24 100644 --- a/docs/hardware-provisioning.md +++ b/docs/hardware-provisioning.md @@ -22,7 +22,7 @@ Hadoop and Spark on a common cluster manager like [Mesos](running-on-mesos.html) * If this is not possible, run Spark on different nodes in the same local-area network as HDFS. -* For low-latency data stores like HBase, it may be preferrable to run computing jobs on different +* For low-latency data stores like HBase, it may be preferable to run computing jobs on different nodes than the storage system to avoid interference. # Local Disks diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 78ae6a7407467..0a6a0397d9570 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1259,7 +1259,7 @@ dstream.foreachRDD(sendRecord) This is incorrect as this requires the connection object to be serialized and sent from the -driver to the worker. Such connection objects are rarely transferrable across machines. This +driver to the worker. Such connection objects are rarely transferable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. From ead3bbdaef428ac22ee2cecbdc76140d7700871f Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Tue, 7 Jun 2016 09:13:18 -0700 Subject: [PATCH 0545/1470] [SPARK-15684][SPARKR] Not mask startsWith and endsWith in R ## What changes were proposed in this pull request? In R 3.3.0, startsWith and endsWith are added. In this PR, I make the two work in SparkR. 1. Remove signature in generic.R 2. Add setMethod in column.R 3. Add unit tests ## How was this patch tested? Manually test it through SparkR shell for both column data and string data, which are added into the unit test file. Author: wm624@hotmail.com Closes #13476 from wangmiao1981/start. (cherry picked from commit 3ec4461c46e2959f4c640df0292cfcacfe0f727f) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/column.R | 36 ++++++++++++++++++++++- R/pkg/R/generics.R | 4 +-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 7 +++++ 3 files changed, 44 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index a3e09372bb5e9..873e8b1665a28 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -57,7 +57,7 @@ operators <- list( "^" = "pow" ) column_functions1 <- c("asc", "desc", "isNaN", "isNull", "isNotNull") -column_functions2 <- c("like", "rlike", "startsWith", "endsWith", "getField", "getItem", "contains") +column_functions2 <- c("like", "rlike", "getField", "getItem", "contains") createOperator <- function(op) { setMethod(op, @@ -151,6 +151,40 @@ setMethod("substr", signature(x = "Column"), column(jc) }) +#' startsWith +#' +#' Determines if entries of x start with string (entries of) prefix respectively, +#' where strings are recycled to common lengths. +#' +#' @rdname startsWith +#' @name startsWith +#' @family colum_func +#' +#' @param x vector of character string whose “starts” are considered +#' @param prefix character vector (often of length one) +setMethod("startsWith", signature(x = "Column"), + function(x, prefix) { + jc <- callJMethod(x@jc, "startsWith", as.vector(prefix)) + column(jc) + }) + +#' endsWith +#' +#' Determines if entries of x end with string (entries of) suffix respectively, +#' where strings are recycled to common lengths. +#' +#' @rdname endsWith +#' @name endsWith +#' @family colum_func +#' +#' @param x vector of character string whose “ends” are considered +#' @param suffix character vector (often of length one) +setMethod("endsWith", signature(x = "Column"), + function(x, suffix) { + jc <- callJMethod(x@jc, "endsWith", as.vector(suffix)) + column(jc) + }) + #' between #' #' Test if the column is between the lower bound and upper bound, inclusive. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index ed76ad6b73c8b..f0cde56b133f5 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -695,7 +695,7 @@ setGeneric("desc", function(x) { standardGeneric("desc") }) #' @rdname column #' @export -setGeneric("endsWith", function(x, ...) { standardGeneric("endsWith") }) +setGeneric("endsWith", function(x, suffix) { standardGeneric("endsWith") }) #' @rdname column #' @export @@ -727,7 +727,7 @@ setGeneric("rlike", function(x, ...) { standardGeneric("rlike") }) #' @rdname column #' @export -setGeneric("startsWith", function(x, ...) { standardGeneric("startsWith") }) +setGeneric("startsWith", function(x, prefix) { standardGeneric("startsWith") }) #' @rdname column #' @export diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 94fa363d7ef4e..375cb6f58808d 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1136,7 +1136,14 @@ test_that("string operators", { df <- read.json(jsonPath) expect_equal(count(where(df, like(df$name, "A%"))), 1) expect_equal(count(where(df, startsWith(df$name, "A"))), 1) + expect_true(first(select(df, startsWith(df$name, "M")))[[1]]) + expect_false(first(select(df, startsWith(df$name, "m")))[[1]]) + expect_true(first(select(df, endsWith(df$name, "el")))[[1]]) expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") + if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { + expect_true(startsWith("Hello World", "Hello")) + expect_false(endsWith("Hello World", "a")) + } expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30") expect_equal(collect(select(df, concat(df$name, lit(":"), df$age)))[[2, 1]], "Andy:30") expect_equal(collect(select(df, concat_ws(":", df$name)))[[2, 1]], "Andy") From f1439b90c64f9c41d3dde3950602282f54d6e96d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 7 Jun 2016 09:28:39 -0700 Subject: [PATCH 0546/1470] [SPARK-15760][DOCS] Add documentation for package-related configs. While there, also document spark.files and spark.jars. Text is the same as the spark-submit help text with some minor adjustments. Author: Marcelo Vanzin Closes #13502 from vanzin/SPARK-15760. (cherry picked from commit 200f01c8fb15680b5630fbd122d44f9b1d096e02) Signed-off-by: Marcelo Vanzin --- docs/configuration.md | 47 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index d6471a8cc7873..32c3a9266078a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -380,6 +380,53 @@ Apart from these, the following properties are also available, and may be useful from JVM to Python worker for every task. +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    spark.files + Comma-separated list of files to be placed in the working directory of each executor. +
    spark.submit.pyFiles + Comma-separated list of .zip, .egg, or .py files to place on the PYTHONPATH for Python apps. +
    spark.jars + Comma-separated list of local jars to include on the driver and executor classpaths. +
    spark.jars.packages + Comma-separated list of maven coordinates of jars to include on the driver and executor + classpaths. Will search the local maven repo, then maven central and any additional remote + repositories given by spark.jars.ivy. The format for the coordinates should be + groupId:artifactId:version. +
    spark.jars.excludes + Comma-separated list of groupId:artifactId, to exclude while resolving the dependencies + provided in spark.jars.packages to avoid dependency conflicts. +
    spark.jars.ivy + Comma-separated list of additional remote repositories to search for the coordinates given + with spark.jars.packages. +
    #### Shuffle Behavior From 7a7ec05477a80a5c92427290f0b85ab80b23a613 Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Tue, 7 Jun 2016 14:10:20 -0700 Subject: [PATCH 0547/1470] [SPARK-14279][BUILD] Pick the spark version from pom Changes are already merged in master (https://github.com/apache/spark/pull/13061) . This one's for branch-2.0 Author: Dhruve Ashar Closes #13530 from dhruve/impr/branch-2.0/SPARK-14279. --- build/spark-build-info | 38 +++++++++++++ core/pom.xml | 31 +++++++++++ .../org/apache/spark/deploy/SparkSubmit.scala | 7 ++- .../main/scala/org/apache/spark/package.scala | 55 ++++++++++++++++++- pom.xml | 6 +- project/SparkBuild.scala | 21 ++++++- 6 files changed, 150 insertions(+), 8 deletions(-) create mode 100755 build/spark-build-info diff --git a/build/spark-build-info b/build/spark-build-info new file mode 100755 index 0000000000000..ad0ec67f455cb --- /dev/null +++ b/build/spark-build-info @@ -0,0 +1,38 @@ +#!/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. +# + +# This script generates the build info for spark and places it into the spark-version-info.properties file. +# Arguments: +# build_tgt_directory - The target directory where properties file would be created. [./core/target/extra-resources] +# spark_version - The current version of spark + +RESOURCE_DIR="$1" +mkdir -p "$RESOURCE_DIR" +SPARK_BUILD_INFO="${RESOURCE_DIR}"/spark-version-info.properties + +echo_build_properties() { + echo version=$1 + echo user=$USER + echo revision=$(git rev-parse HEAD) + echo branch=$(git rev-parse --abbrev-ref HEAD) + echo date=$(date -u +%Y-%m-%dT%H:%M:%SZ) + echo url=$(git config --get remote.origin.url) +} + +echo_build_properties $2 > "$SPARK_BUILD_INFO" diff --git a/core/pom.xml b/core/pom.xml index 45f8bfcc05aa1..f5fdb40696017 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -337,7 +337,38 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + ${project.basedir}/src/main/resources + + + + ${project.build.directory}/extra-resources + true + + + + org.apache.maven.plugins + maven-antrun-plugin + + + generate-resources + + + + + + + + + + + run + + + + org.apache.maven.plugins maven-dependency-plugin 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 9be4cadcb430a..9feafc99ac07f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -40,7 +40,8 @@ import org.apache.ivy.plugins.matcher.GlobPatternMatcher import org.apache.ivy.plugins.repository.file.FileRepository import org.apache.ivy.plugins.resolver.{ChainResolver, FileSystemResolver, IBiblioResolver} -import org.apache.spark.{SPARK_VERSION, SparkException, SparkUserAppException} +import org.apache.spark.{SPARK_REVISION, SPARK_VERSION, SparkException, SparkUserAppException} +import org.apache.spark.{SPARK_BRANCH, SPARK_BUILD_DATE, SPARK_BUILD_USER, SPARK_REPO_URL} import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ import org.apache.spark.launcher.SparkLauncher @@ -103,6 +104,10 @@ object SparkSubmit { /___/ .__/\_,_/_/ /_/\_\ version %s /_/ """.format(SPARK_VERSION)) + printStream.println("Branch %s".format(SPARK_BRANCH)) + printStream.println("Compiled by user %s on %s".format(SPARK_BUILD_USER, SPARK_BUILD_DATE)) + printStream.println("Revision %s".format(SPARK_REVISION)) + printStream.println("Url %s".format(SPARK_REPO_URL)) printStream.println("Type --help for more information.") exitFn(0) } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 65a3dd2e37e16..2610d6f6e45a2 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -41,7 +41,58 @@ package org.apache * level interfaces. These are subject to changes or removal in minor releases. */ +import java.util.Properties + package object spark { - // For package docs only - val SPARK_VERSION = "2.0.0-preview" + + private object SparkBuildInfo { + + val ( + spark_version: String, + spark_branch: String, + spark_revision: String, + spark_build_user: String, + spark_repo_url: String, + spark_build_date: String) = { + + val resourceStream = Thread.currentThread().getContextClassLoader. + getResourceAsStream("spark-version-info.properties") + + try { + val unknownProp = "" + val props = new Properties() + props.load(resourceStream) + ( + props.getProperty("version", unknownProp), + props.getProperty("branch", unknownProp), + props.getProperty("revision", unknownProp), + props.getProperty("user", unknownProp), + props.getProperty("url", unknownProp), + props.getProperty("date", unknownProp) + ) + } catch { + case npe: NullPointerException => + throw new SparkException("Error while locating file spark-version-info.properties", npe) + case e: Exception => + throw new SparkException("Error loading properties from spark-version-info.properties", e) + } finally { + if (resourceStream != null) { + try { + resourceStream.close() + } catch { + case e: Exception => + throw new SparkException("Error closing spark build info resource stream", e) + } + } + } + } + } + + val SPARK_VERSION = SparkBuildInfo.spark_version + val SPARK_BRANCH = SparkBuildInfo.spark_branch + val SPARK_REVISION = SparkBuildInfo.spark_revision + val SPARK_BUILD_USER = SparkBuildInfo.spark_build_user + val SPARK_REPO_URL = SparkBuildInfo.spark_repo_url + val SPARK_BUILD_DATE = SparkBuildInfo.spark_build_date } + diff --git a/pom.xml b/pom.xml index 79ee7876f466b..92ffae2e5d31a 100644 --- a/pom.xml +++ b/pom.xml @@ -180,6 +180,8 @@ 4.5.3 1.1 2.52.0 + 2.8 + 1.8 ${java.home} @@ -2061,7 +2063,7 @@ org.apache.maven.plugins maven-antrun-plugin - 1.8 + ${maven-antrun.version} org.apache.maven.plugins @@ -2184,7 +2186,7 @@ org.apache.maven.plugins maven-antrun-plugin - [1.8,) + [${maven-antrun.version},) run diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 744f57c5177a3..304288a32cefe 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -360,6 +360,9 @@ object SparkBuild extends PomBuild { enable(MimaBuild.mimaSettings(sparkHome, x))(x) } + /* Generate and pick the spark build info from extra-resources */ + enable(Core.settings)(core) + /* Unsafe settings */ enable(Unsafe.settings)(unsafe) @@ -448,7 +451,19 @@ object SparkBuild extends PomBuild { else x.settings(Seq[Setting[_]](): _*) } ++ Seq[Project](OldDeps.project) } +} +object Core { + lazy val settings = Seq( + resourceGenerators in Compile += Def.task { + val buildScript = baseDirectory.value + "/../build/spark-build-info" + val targetDir = baseDirectory.value + "/target/extra-resources/" + val command = buildScript + " " + targetDir + " " + version.value + Process(command).!! + val propsFile = baseDirectory.value / "target" / "extra-resources" / "spark-version-info.properties" + Seq(propsFile) + }.taskValue + ) } object Unsafe { @@ -480,9 +495,9 @@ object DependencyOverrides { } /** - This excludes library dependencies in sbt, which are specified in maven but are - not needed by sbt build. - */ + * This excludes library dependencies in sbt, which are specified in maven but are + * not needed by sbt build. + */ object ExcludedDependencies { lazy val settings = Seq( libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") } From 9e16f23e7d0b0e52f66f47740be94bda6654151f Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Tue, 7 Jun 2016 15:21:55 -0700 Subject: [PATCH 0548/1470] [SPARK-15674][SQL] Deprecates "CREATE TEMPORARY TABLE USING...", uses "CREAT TEMPORARY VIEW USING..." instead ## What changes were proposed in this pull request? The current implementation of "CREATE TEMPORARY TABLE USING datasource..." is NOT creating any intermediate temporary data directory like temporary HDFS folder, instead, it only stores a SQL string in memory. Probably we should use "TEMPORARY VIEW" instead. This PR assumes a temporary table has to link with some temporary intermediate data. It follows the definition of temporary table like this (from [hortonworks doc](https://docs.hortonworks.com/HDPDocuments/HDP2/HDP-2.3.0/bk_dataintegration/content/temp-tables.html)): > A temporary table is a convenient way for an application to automatically manage intermediate data generated during a complex query **Example**: ``` scala> spark.sql("CREATE temporary view my_tab7 (c1: String, c2: String) USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')") scala> spark.sql("select c1, c2 from my_tab7").show() +----+-----+ | c1| c2| +----+-----+ |year| make| |2012|Tesla| ... ``` It NOW prints a **deprecation warning** if "CREATE TEMPORARY TABLE USING..." is used. ``` scala> spark.sql("CREATE temporary table my_tab7 (c1: String, c2: String) USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '/Users/seanzhong/csv/cars.csv')") 16/05/31 10:39:27 WARN SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE tableName USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead ``` ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13414 from clockfly/create_temp_view_using. (cherry picked from commit 890baaca5078df0b50c0054f55a2c33023f7fd67) Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +++ .../spark/sql/execution/SparkSqlParser.scala | 15 ++++++++++++- .../spark/sql/execution/SparkStrategies.scala | 9 ++++++-- .../spark/sql/execution/datasources/ddl.scala | 5 +++-- .../sql/execution/command/DDLSuite.scala | 21 ++++++++++++++++++- 5 files changed, 47 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index b0e71c7e7c7d1..2dd3cfab177cd 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -90,6 +90,9 @@ statement identifierCommentList? (COMMENT STRING)? (PARTITIONED ON identifierList)? (TBLPROPERTIES tablePropertyList)? AS query #createView + | CREATE (OR REPLACE)? TEMPORARY VIEW + tableIdentifier ('(' colTypeList ')')? tableProvider + (OPTIONS tablePropertyList)? #createTempViewUsing | ALTER VIEW tableIdentifier AS? query #alterViewQuery | CREATE TEMPORARY? FUNCTION qualifiedName AS className=STRING (USING resource (',' resource)*)? #createFunction diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index c68c8f80f87ad..dc742220512bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, ScriptInputOutputSchema} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, _} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} import org.apache.spark.sql.types.DataType @@ -346,6 +346,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } } + /** + * Creates a [[CreateTempViewUsing]] logical plan. + */ + override def visitCreateTempViewUsing( + ctx: CreateTempViewUsingContext): LogicalPlan = withOrigin(ctx) { + CreateTempViewUsing( + tableIdent = visitTableIdentifier(ctx.tableIdentifier()), + userSpecifiedSchema = Option(ctx.colTypeList()).map(createStructType), + replace = ctx.REPLACE != null, + provider = ctx.tableProvider.qualifiedName.getText, + options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) + } + /** * Create a [[LoadDataCommand]] command. * 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 2e69027a031a6..d1261dd6cae43 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 @@ -399,9 +399,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object DDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case c: CreateTableUsing if c.temporary && !c.allowExisting => + logWarning( + s"CREATE TEMPORARY TABLE ${c.tableIdent.identifier} USING... is deprecated, " + + s"please use CREATE TEMPORARY VIEW viewName USING... instead") ExecutedCommandExec( - CreateTempTableUsing( - c.tableIdent, c.userSpecifiedSchema, c.provider, c.options)) :: Nil + CreateTempViewUsing( + c.tableIdent, c.userSpecifiedSchema, replace = true, c.provider, c.options)) :: Nil case c: CreateTableUsing if !c.temporary => val cmd = @@ -432,6 +435,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { c.child) ExecutedCommandExec(cmd) :: Nil + case c: CreateTempViewUsing => + ExecutedCommandExec(c) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index bf272e3c0659d..aa42eae986be0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -64,9 +64,10 @@ case class CreateTableUsingAsSelect( override def output: Seq[Attribute] = Seq.empty[Attribute] } -case class CreateTempTableUsing( +case class CreateTempViewUsing( tableIdent: TableIdentifier, userSpecifiedSchema: Option[StructType], + replace: Boolean, provider: String, options: Map[String, String]) extends RunnableCommand { @@ -84,7 +85,7 @@ case class CreateTempTableUsing( sparkSession.sessionState.catalog.createTempView( tableIdent.table, Dataset.ofRows(sparkSession, LogicalRelation(dataSource.resolveRelation())).logicalPlan, - overrideIfExists = true) + replace) Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 741ea673e97e7..a7e6893caaeb1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, NoSuchPartitionException, NoSuchTableException} +import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog} @@ -422,6 +422,25 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("create temporary view using") { + val csvFile = Thread.currentThread().getContextClassLoader.getResource("cars.csv").toString() + withView("testview") { + sql(s"CREATE OR REPLACE TEMPORARY VIEW testview (c1: String, c2: String) USING " + + "org.apache.spark.sql.execution.datasources.csv.CSVFileFormat " + + s"OPTIONS (PATH '$csvFile')") + + checkAnswer( + sql("select c1, c2 from testview order by c1 limit 1"), + Row("1997", "Ford") :: Nil) + + // Fails if creating a new view with the same name + intercept[TempTableAlreadyExistsException] { + sql(s"CREATE TEMPORARY VIEW testview USING " + + s"org.apache.spark.sql.execution.datasources.csv.CSVFileFormat OPTIONS (PATH '$csvFile')") + } + } + } + test("alter table: rename") { val catalog = spark.sessionState.catalog val tableIdent1 = TableIdentifier("tab1", Some("dbx")) From e21a9ddefed074de84d3b3bb0f347d64b82696c6 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 7 Jun 2016 15:25:36 -0700 Subject: [PATCH 0549/1470] [SPARK-13590][ML][DOC] Document spark.ml LiR, LoR and AFTSurvivalRegression behavior difference ## What changes were proposed in this pull request? When fitting ```LinearRegressionModel```(by "l-bfgs" solver) and ```LogisticRegressionModel``` w/o intercept on dataset with constant nonzero column, spark.ml produce same model as R glmnet but different from LIBSVM. When fitting ```AFTSurvivalRegressionModel``` w/o intercept on dataset with constant nonzero column, spark.ml produce different model compared with R survival::survreg. We should output a warning message and clarify in document for this condition. ## How was this patch tested? Document change, no unit test. cc mengxr Author: Yanbo Liang Closes #12731 from yanboliang/spark-13590. (cherry picked from commit 6ecedf39b44c9acd58cdddf1a31cf11e8e24428c) Signed-off-by: Yanbo Liang --- docs/ml-classification-regression.md | 6 ++++++ .../spark/ml/classification/LogisticRegression.scala | 7 +++++++ .../spark/ml/regression/AFTSurvivalRegression.scala | 9 ++++++++- .../apache/spark/ml/regression/LinearRegression.scala | 7 +++++++ 4 files changed, 28 insertions(+), 1 deletion(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index ff8dec6d2d3a4..88457d4bb1627 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -62,6 +62,8 @@ For more background and more details about the implementation, refer to the docu > The current implementation of logistic regression in `spark.ml` only supports binary classes. Support for multiclass regression will be added in the future. + > When fitting LogisticRegressionModel without intercept on dataset with constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is the same as R glmnet but different from LIBSVM. + **Example** The following example shows how to train a logistic regression model @@ -351,6 +353,8 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat The interface for working with linear regression models and model summaries is similar to the logistic regression case. + > When fitting LinearRegressionModel without intercept on dataset with constant nonzero column by "l-bfgs" solver, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is the same as R glmnet but different from LIBSVM. + **Example** The following @@ -666,6 +670,8 @@ The optimization algorithm underlying the implementation is L-BFGS. The implementation matches the result from R's survival function [survreg](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html) + > When fitting AFTSurvivalRegressionModel without intercept on dataset with constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is different from R survival::survreg. + **Example**

    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 1ea4d90e166b2..51ede15d6c367 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 @@ -333,6 +333,13 @@ class LogisticRegression @Since("1.2.0") ( val featuresMean = summarizer.mean.toArray val featuresStd = summarizer.variance.toArray.map(math.sqrt) + if (!$(fitIntercept) && (0 until numFeatures).exists { i => + featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { + logWarning("Fitting LogisticRegressionModel without intercept on dataset with " + + "constant nonzero column, Spark MLlib outputs zero coefficients for constant " + + "nonzero columns. This behavior is the same as R glmnet but different from LIBSVM.") + } + val regParamL1 = $(elasticNetParam) * $(regParam) val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index c4400738426b5..e5f23f44bc5ee 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -209,11 +209,18 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S } val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val numFeatures = featuresStd.size + + if (!$(fitIntercept) && (0 until numFeatures).exists { i => + featuresStd(i) == 0.0 && featuresSummarizer.mean(i) != 0.0 }) { + logWarning("Fitting AFTSurvivalRegressionModel without intercept on dataset with " + + "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + + "columns. This behavior is different from R survival::survreg.") + } val costFun = new AFTCostFun(instances, $(fitIntercept), featuresStd) val optimizer = new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) - val numFeatures = featuresStd.size /* The parameters vector has three parts: the first element: Double, log(sigma), the log of scale parameter 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 6be2584785bd3..52ec40e15b213 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 @@ -267,6 +267,13 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val featuresMean = featuresSummarizer.mean.toArray val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + if (!$(fitIntercept) && (0 until numFeatures).exists { i => + featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { + logWarning("Fitting LinearRegressionModel without intercept on dataset with " + + "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + + "columns. This behavior is the same as R glmnet but different from LIBSVM.") + } + // Since we implicitly do the feature scaling when we compute the cost function // to improve the convergence, the effective regParam will be changed. val effectiveRegParam = $(regParam) / yStd From ec556fec01b1ba114d3160f59a7c40e143c60a7e Mon Sep 17 00:00:00 2001 From: zhonghaihua <793507405@qq.com> Date: Tue, 7 Jun 2016 16:30:58 -0700 Subject: [PATCH 0550/1470] [SPARK-14485][CORE] ignore task finished for executor lost and removed by driver Now, when executor is removed by driver with heartbeats timeout, driver will re-queue the task on this executor and send a kill command to cluster to kill this executor. But, in a situation, the running task of this executor is finished and return result to driver before this executor killed by kill command sent by driver. At this situation, driver will accept the task finished event and ignore speculative task and re-queued task. But, as we know, this executor has removed by driver, the result of this finished task can not save in driver because the BlockManagerId has also removed from BlockManagerMaster by driver. So, the result data of this stage is not complete, and then, it will cause fetch failure. For more details, [link to jira issues SPARK-14485](https://issues.apache.org/jira/browse/SPARK-14485) This PR introduce a mechanism to ignore this kind of task finished. N/A Author: zhonghaihua <793507405@qq.com> Closes #12258 from zhonghaihua/ignoreTaskFinishForExecutorLostAndRemovedByDriver. (cherry picked from commit 695dbc816a6d70289abeb145cb62ff4e62b3f49b) Signed-off-by: Marcelo Vanzin --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) 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 01e85ca405587..d6f58e4789cff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -346,9 +346,11 @@ private[spark] class TaskSchedulerImpl( } taskIdToTaskSetManager.get(tid) match { case Some(taskSet) => + var executorId: String = null if (TaskState.isFinished(state)) { taskIdToTaskSetManager.remove(tid) taskIdToExecutorId.remove(tid).foreach { execId => + executorId = execId if (executorIdToTaskCount.contains(execId)) { executorIdToTaskCount(execId) -= 1 } @@ -356,7 +358,17 @@ private[spark] class TaskSchedulerImpl( } if (state == TaskState.FINISHED) { taskSet.removeRunningTask(tid) - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + // In some case, executor has already been removed by driver for heartbeats timeout, + // but at sometime, before executor killed by cluster, the task of running on this + // executor is finished and return task success state to driver. However, this kinds + // of task should be ignored, because the task on this executor is already re-queued + // by driver. For more details, can check in SPARK-14485. + if (executorId != null && !executorIdToTaskCount.contains(executorId)) { + logInfo(s"Ignoring update with state $state for TID $tid because its executor " + + s"has already been removed by driver") + } else { + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + } } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { taskSet.removeRunningTask(tid) taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) From 003c447924a9da4cd3f156c2bb4adae5866b0dab Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 7 Jun 2016 16:40:03 -0700 Subject: [PATCH 0551/1470] [SPARK-15580][SQL] Add ContinuousQueryInfo to make ContinuousQueryListener events serializable ## What changes were proposed in this pull request? This PR adds ContinuousQueryInfo to make ContinuousQueryListener events serializable in order to support writing events into the event log. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13335 from zsxwing/query-info. (cherry picked from commit 0cfd6192f38932a26195a6a8dbbc637d67f5ec55) Signed-off-by: Tathagata Das --- .../ContinuousQueryListenerBus.scala | 27 +--- .../execution/streaming/StreamExecution.scala | 21 ++- .../sql/streaming/ContinuousQueryInfo.scala | 34 +++++ .../streaming/ContinuousQueryListener.scala | 34 +++-- .../spark/sql/streaming/SinkStatus.scala | 6 +- .../spark/sql/streaming/SourceStatus.scala | 8 +- .../ContinuousQueryListenerSuite.scala | 133 ++++++++++++++---- .../sql/streaming/ContinuousQuerySuite.scala | 16 +-- 8 files changed, 203 insertions(+), 76 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala index 2a1be09693e51..f50951f9bd656 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala @@ -22,15 +22,13 @@ import org.apache.spark.sql.streaming.ContinuousQueryListener import org.apache.spark.util.ListenerBus /** - * A bus to forward events to [[ContinuousQueryListener]]s. This one will wrap received - * [[ContinuousQueryListener.Event]]s as WrappedContinuousQueryListenerEvents and send them to the - * Spark listener bus. It also registers itself with Spark listener bus, so that it can receive - * WrappedContinuousQueryListenerEvents, unwrap them as ContinuousQueryListener.Events and - * dispatch them to ContinuousQueryListener. + * A bus to forward events to [[ContinuousQueryListener]]s. This one will send received + * [[ContinuousQueryListener.Event]]s to the Spark listener bus. It also registers itself with + * Spark listener bus, so that it can receive [[ContinuousQueryListener.Event]]s and dispatch them + * to ContinuousQueryListener. */ class ContinuousQueryListenerBus(sparkListenerBus: LiveListenerBus) - extends SparkListener - with ListenerBus[ContinuousQueryListener, ContinuousQueryListener.Event] { + extends SparkListener with ListenerBus[ContinuousQueryListener, ContinuousQueryListener.Event] { import ContinuousQueryListener._ @@ -45,13 +43,13 @@ class ContinuousQueryListenerBus(sparkListenerBus: LiveListenerBus) case s: QueryStarted => postToAll(s) case _ => - sparkListenerBus.post(new WrappedContinuousQueryListenerEvent(event)) + sparkListenerBus.post(event) } } override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { - case WrappedContinuousQueryListenerEvent(e) => + case e: ContinuousQueryListener.Event => postToAll(e) case _ => } @@ -71,15 +69,4 @@ class ContinuousQueryListenerBus(sparkListenerBus: LiveListenerBus) } } - /** - * Wrapper for StreamingListenerEvent as SparkListenerEvent so that it can be posted to Spark - * listener bus. - */ - private case class WrappedContinuousQueryListenerEvent( - streamingListenerEvent: ContinuousQueryListener.Event) - extends SparkListenerEvent { - - // Do not log streaming events in event log as history server does not support these events. - protected[spark] override def logEvent: Boolean = false - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 16d38a2f7db56..d9800e4afd8e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -131,12 +131,13 @@ class StreamExecution( /** Returns current status of all the sources. */ override def sourceStatuses: Array[SourceStatus] = { val localAvailableOffsets = availableOffsets - sources.map(s => new SourceStatus(s.toString, localAvailableOffsets.get(s))).toArray + sources.map(s => + new SourceStatus(s.toString, localAvailableOffsets.get(s).map(_.toString))).toArray } /** Returns current status of the sink. */ override def sinkStatus: SinkStatus = - new SinkStatus(sink.toString, committedOffsets.toCompositeOffset(sources)) + new SinkStatus(sink.toString, committedOffsets.toCompositeOffset(sources).toString) /** Returns the [[ContinuousQueryException]] if the query was terminated by an exception. */ override def exception: Option[ContinuousQueryException] = Option(streamDeathCause) @@ -167,7 +168,7 @@ class StreamExecution( // Mark ACTIVE and then post the event. QueryStarted event is synchronously sent to listeners, // so must mark this as ACTIVE first. state = ACTIVE - postEvent(new QueryStarted(this)) // Assumption: Does not throw exception. + postEvent(new QueryStarted(this.toInfo)) // Assumption: Does not throw exception. // Unblock starting thread startLatch.countDown() @@ -206,7 +207,10 @@ class StreamExecution( } finally { state = TERMINATED sparkSession.streams.notifyQueryTermination(StreamExecution.this) - postEvent(new QueryTerminated(this)) + postEvent(new QueryTerminated( + this.toInfo, + exception.map(_.getMessage), + exception.map(_.getStackTrace.toSeq).getOrElse(Nil))) terminationLatch.countDown() } } @@ -374,7 +378,7 @@ class StreamExecution( logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") // Update committed offsets. committedOffsets ++= availableOffsets - postEvent(new QueryProgress(this)) + postEvent(new QueryProgress(this.toInfo)) } private def postEvent(event: ContinuousQueryListener.Event) { @@ -484,6 +488,13 @@ class StreamExecution( """.stripMargin } + private def toInfo: ContinuousQueryInfo = { + new ContinuousQueryInfo( + this.name, + this.sourceStatuses, + this.sinkStatus) + } + trait State case object INITIALIZED extends State case object ACTIVE extends State diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala new file mode 100644 index 0000000000000..57b718b08fbbb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * A class used to report information about the progress of a [[ContinuousQuery]]. + * + * @param name The [[ContinuousQuery]] name. + * @param sourceStatuses The current statuses of the [[ContinuousQuery]]'s sources. + * @param sinkStatus The current status of the [[ContinuousQuery]]'s sink. + */ +@Experimental +class ContinuousQueryInfo private[sql]( + val name: String, + val sourceStatuses: Seq[SourceStatus], + val sinkStatus: SinkStatus) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala index 6bdd513288ae1..dd311148e053f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental +import org.apache.spark.scheduler.SparkListenerEvent /** * :: Experimental :: @@ -70,26 +71,43 @@ abstract class ContinuousQueryListener { object ContinuousQueryListener { /** - * Base type of [[ContinuousQueryListener]] events. + * :: Experimental :: + * Base type of [[ContinuousQueryListener]] events * @since 2.0.0 */ - trait Event + @Experimental + trait Event extends SparkListenerEvent /** - * Event representing the start of a query. + * :: Experimental :: + * Event representing the start of a query * @since 2.0.0 */ - class QueryStarted private[sql](val query: ContinuousQuery) extends Event + @Experimental + class QueryStarted private[sql](val queryInfo: ContinuousQueryInfo) extends Event /** - * Event representing any progress updates in a query. + * :: Experimental :: + * Event representing any progress updates in a query * @since 2.0.0 */ - class QueryProgress private[sql](val query: ContinuousQuery) extends Event + @Experimental + class QueryProgress private[sql](val queryInfo: ContinuousQueryInfo) extends Event /** - * Event representing that termination of a query. + * :: Experimental :: + * Event representing that termination of a query + * + * @param queryInfo Information about the status of the query. + * @param exception The exception message of the [[ContinuousQuery]] if the query was terminated + * with an exception. Otherwise, it will be `None`. + * @param stackTrace The stack trace of the exception if the query was terminated with an + * exception. It will be empty if there was no error. * @since 2.0.0 */ - class QueryTerminated private[sql](val query: ContinuousQuery) extends Event + @Experimental + class QueryTerminated private[sql]( + val queryInfo: ContinuousQueryInfo, + val exception: Option[String], + val stackTrace: Seq[StackTraceElement]) extends Event } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index 79ddf01042ef6..de1efe961f8bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -18,17 +18,17 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.{Offset, Sink} +import org.apache.spark.sql.execution.streaming.Sink /** * :: Experimental :: * Status and metrics of a streaming [[Sink]]. * * @param description Description of the source corresponding to this status - * @param offset Current offset up to which data has been written by the sink + * @param offsetDesc Description of the current offset up to which data has been written by the sink * @since 2.0.0 */ @Experimental class SinkStatus private[sql]( val description: String, - val offset: Offset) + val offsetDesc: String) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 8fccd5b7a3e4e..bd0c8485e4fdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -18,17 +18,17 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.{Offset, Source} +import org.apache.spark.sql.execution.streaming.Source /** * :: Experimental :: * Status and metrics of a streaming [[Source]]. * - * @param description Description of the source corresponding to this status - * @param offset Current offset of the source, if known + * @param description Description of the source corresponding to this status + * @param offsetDesc Description of the current [[Source]] offset if known * @since 2.0.0 */ @Experimental class SourceStatus private[sql] ( val description: String, - val offset: Option[Offset]) + val offsetDesc: Option[String]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala index cdd97da8ae33e..9b59ab60a688b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala @@ -26,7 +26,9 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ +import org.apache.spark.SparkException import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.util.JsonProtocol class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { @@ -51,14 +53,13 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { Assert("Incorrect query status in onQueryStarted") { val status = listener.startStatus assert(status != null) - assert(status.active == true) assert(status.sourceStatuses.size === 1) assert(status.sourceStatuses(0).description.contains("Memory")) // The source and sink offsets must be None as this must be called before the // batches have started - assert(status.sourceStatuses(0).offset === None) - assert(status.sinkStatus.offset === CompositeOffset(None :: Nil)) + assert(status.sourceStatuses(0).offsetDesc === None) + assert(status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString) // No progress events or termination events assert(listener.progressStatuses.isEmpty) @@ -73,9 +74,8 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(listener.progressStatuses.size === 1) val status = listener.progressStatuses.peek() assert(status != null) - assert(status.active == true) - assert(status.sourceStatuses(0).offset === Some(LongOffset(0))) - assert(status.sinkStatus.offset === CompositeOffset.fill(LongOffset(0))) + assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) + assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) // No termination events assert(listener.terminationStatus === null) @@ -86,10 +86,8 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { eventually(Timeout(streamingTimeout)) { val status = listener.terminationStatus assert(status != null) - - assert(status.active === false) // must be inactive by the time onQueryTerm is called - assert(status.sourceStatuses(0).offset === Some(LongOffset(0))) - assert(status.sinkStatus.offset === CompositeOffset.fill(LongOffset(0))) + assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) + assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) } listener.checkAsyncErrors() } @@ -141,6 +139,92 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { } } + test("exception should be reported in QueryTerminated") { + val listener = new QueryStatusCollector + withListenerAdded(listener) { + val input = MemoryStream[Int] + testStream(input.toDS.map(_ / 0))( + StartStream(), + AddData(input, 1), + ExpectFailure[SparkException](), + Assert { + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + assert(listener.terminationStatus !== null) + assert(listener.terminationException.isDefined) + assert(listener.terminationException.get.contains("java.lang.ArithmeticException")) + assert(listener.terminationStackTrace.nonEmpty) + } + ) + } + } + + test("QueryStarted serialization") { + val queryStartedInfo = new ContinuousQueryInfo( + "name", + Seq(new SourceStatus("source1", None), new SourceStatus("source2", None)), + new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString)) + val queryStarted = new ContinuousQueryListener.QueryStarted(queryStartedInfo) + val json = JsonProtocol.sparkEventToJson(queryStarted) + val newQueryStarted = JsonProtocol.sparkEventFromJson(json) + .asInstanceOf[ContinuousQueryListener.QueryStarted] + assertContinuousQueryInfoEquals(queryStarted.queryInfo, newQueryStarted.queryInfo) + } + + test("QueryProgress serialization") { + val queryProcessInfo = new ContinuousQueryInfo( + "name", + Seq( + new SourceStatus("source1", Some(LongOffset(0).toString)), + new SourceStatus("source2", Some(LongOffset(1).toString))), + new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) + val queryProcess = new ContinuousQueryListener.QueryProgress(queryProcessInfo) + val json = JsonProtocol.sparkEventToJson(queryProcess) + val newQueryProcess = JsonProtocol.sparkEventFromJson(json) + .asInstanceOf[ContinuousQueryListener.QueryProgress] + assertContinuousQueryInfoEquals(queryProcess.queryInfo, newQueryProcess.queryInfo) + } + + test("QueryTerminated serialization") { + val queryTerminatedInfo = new ContinuousQueryInfo( + "name", + Seq( + new SourceStatus("source1", Some(LongOffset(0).toString)), + new SourceStatus("source2", Some(LongOffset(1).toString))), + new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) + val exception = new RuntimeException("exception") + val queryQueryTerminated = new ContinuousQueryListener.QueryTerminated( + queryTerminatedInfo, + Some(exception.getMessage), + exception.getStackTrace) + val json = + JsonProtocol.sparkEventToJson(queryQueryTerminated) + val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) + .asInstanceOf[ContinuousQueryListener.QueryTerminated] + assertContinuousQueryInfoEquals(queryQueryTerminated.queryInfo, newQueryTerminated.queryInfo) + assert(queryQueryTerminated.exception === newQueryTerminated.exception) + } + + private def assertContinuousQueryInfoEquals( + expected: ContinuousQueryInfo, + actual: ContinuousQueryInfo): Unit = { + assert(expected.name === actual.name) + assert(expected.sourceStatuses.size === actual.sourceStatuses.size) + expected.sourceStatuses.zip(actual.sourceStatuses).foreach { + case (expectedSource, actualSource) => + assertSourceStatus(expectedSource, actualSource) + } + assertSinkStatus(expected.sinkStatus, actual.sinkStatus) + } + + private def assertSourceStatus(expected: SourceStatus, actual: SourceStatus): Unit = { + assert(expected.description === actual.description) + assert(expected.offsetDesc === actual.offsetDesc) + } + + private def assertSinkStatus(expected: SinkStatus, actual: SinkStatus): Unit = { + assert(expected.description === actual.description) + assert(expected.offsetDesc === actual.offsetDesc) + } private def withListenerAdded(listener: ContinuousQueryListener)(body: => Unit): Unit = { try { @@ -164,9 +248,12 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { // to catch errors in the async listener events @volatile private var asyncTestWaiter = new Waiter - @volatile var startStatus: QueryStatus = null - @volatile var terminationStatus: QueryStatus = null - val progressStatuses = new ConcurrentLinkedQueue[QueryStatus] + @volatile var startStatus: ContinuousQueryInfo = null + @volatile var terminationStatus: ContinuousQueryInfo = null + @volatile var terminationException: Option[String] = null + @volatile var terminationStackTrace: Seq[StackTraceElement] = null + + val progressStatuses = new ConcurrentLinkedQueue[ContinuousQueryInfo] def reset(): Unit = { startStatus = null @@ -182,35 +269,25 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { override def onQueryStarted(queryStarted: QueryStarted): Unit = { asyncTestWaiter { - startStatus = QueryStatus(queryStarted.query) + startStatus = queryStarted.queryInfo } } override def onQueryProgress(queryProgress: QueryProgress): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryProgress called before onQueryStarted") - progressStatuses.add(QueryStatus(queryProgress.query)) + progressStatuses.add(queryProgress.queryInfo) } } override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryTerminated called before onQueryStarted") - terminationStatus = QueryStatus(queryTerminated.query) + terminationStatus = queryTerminated.queryInfo + terminationException = queryTerminated.exception + terminationStackTrace = queryTerminated.stackTrace } asyncTestWaiter.dismiss() } } - - case class QueryStatus( - active: Boolean, - exception: Option[Exception], - sourceStatuses: Array[SourceStatus], - sinkStatus: SinkStatus) - - object QueryStatus { - def apply(query: ContinuousQuery): QueryStatus = { - QueryStatus(query.isActive, query.exception, query.sourceStatuses, query.sinkStatus) - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala index e4ca86d9d5025..55424058f5dae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala @@ -66,21 +66,21 @@ class ContinuousQuerySuite extends StreamTest { testStream(mapped)( AssertOnQuery(_.sourceStatuses.length === 1), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.sourceStatuses(0).offset === None), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), AssertOnQuery(_.sinkStatus.description.contains("Memory")), - AssertOnQuery(_.sinkStatus.offset === new CompositeOffset(None :: Nil)), + AssertOnQuery(_.sinkStatus.offsetDesc === new CompositeOffset(None :: Nil).toString), AddData(inputData, 1, 2), CheckAnswer(6, 3), - AssertOnQuery(_.sourceStatuses(0).offset === Some(LongOffset(0))), - AssertOnQuery(_.sinkStatus.offset === CompositeOffset.fill(LongOffset(0))), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), - AssertOnQuery(_.sourceStatuses(0).offset === Some(LongOffset(1))), - AssertOnQuery(_.sinkStatus.offset === CompositeOffset.fill(LongOffset(1))), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), AddData(inputData, 0), ExpectFailure[SparkException], - AssertOnQuery(_.sourceStatuses(0).offset === Some(LongOffset(2))), - AssertOnQuery(_.sinkStatus.offset === CompositeOffset.fill(LongOffset(1))) + AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString) ) } From 141e910af97a5e84082e25f1d5b3c3a9e6c231fc Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 7 Jun 2016 17:01:11 -0700 Subject: [PATCH 0552/1470] [SPARK-15789][SQL] Allow reserved keywords in most places ## What changes were proposed in this pull request? The parser currently does not allow the use of some SQL keywords as table or field names. This PR adds supports for all keywords as identifier. The exception to this are table aliases, in this case most keywords are allowed except for join keywords (```anti, full, inner, left, semi, right, natural, on, join, cross```) and set-operator keywords (```union, intersect, except```). ## How was this patch tested? I have added/move/renamed test in the catalyst `*ParserSuite`s. Author: Herman van Hovell Closes #13534 from hvanhovell/SPARK-15789. (cherry picked from commit 91fbc880b69bddcf5310afecc49df1102408e1f3) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 30 +++++++++---------- .../sql/catalyst/parser/AstBuilder.scala | 12 +++++--- .../catalyst/parser/DataTypeParserSuite.scala | 6 +++- .../catalyst/parser/ErrorParserSuite.scala | 2 -- .../sql/catalyst/parser/PlanParserSuite.scala | 1 + .../parser/TableIdentifierParserSuite.scala | 12 ++++---- 6 files changed, 35 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 2dd3cfab177cd..d10255946af90 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -109,9 +109,9 @@ statement | SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions | SHOW CREATE TABLE tableIdentifier #showCreateTable | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction + | (DESC | DESCRIBE) DATABASE EXTENDED? identifier #describeDatabase | (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)? tableIdentifier partitionSpec? describeColName? #describeTable - | (DESC | DESCRIBE) DATABASE EXTENDED? identifier #describeDatabase | REFRESH TABLE tableIdentifier #refreshTable | CACHE LAZY? TABLE identifier (AS? query)? #cacheTable | UNCACHE TABLE identifier #uncacheTable @@ -251,7 +251,7 @@ tableProperty ; tablePropertyKey - : looseIdentifier ('.' looseIdentifier)* + : identifier ('.' identifier)* | STRING ; @@ -419,9 +419,9 @@ identifierComment ; relationPrimary - : tableIdentifier sample? (AS? identifier)? #tableName - | '(' queryNoWith ')' sample? (AS? identifier)? #aliasedQuery - | '(' relation ')' sample? (AS? identifier)? #aliasedRelation + : tableIdentifier sample? (AS? strictIdentifier)? #tableName + | '(' queryNoWith ')' sample? (AS? strictIdentifier)? #aliasedQuery + | '(' relation ')' sample? (AS? strictIdentifier)? #aliasedRelation | inlineTable #inlineTableDefault2 ; @@ -456,8 +456,8 @@ expression ; booleanExpression - : predicated #booleanDefault - | NOT booleanExpression #logicalNot + : NOT booleanExpression #logicalNot + | predicated #booleanDefault | left=booleanExpression operator=AND right=booleanExpression #logicalBinary | left=booleanExpression operator=OR right=booleanExpression #logicalBinary | EXISTS '(' query ')' #exists @@ -597,16 +597,13 @@ qualifiedName : identifier ('.' identifier)* ; -// Identifier that also allows the use of a number of SQL keywords (mainly for backwards compatibility). -looseIdentifier - : identifier - | FROM - | TO - | TABLE - | WITH +identifier + : strictIdentifier + | ANTI | FULL | INNER | LEFT | SEMI | RIGHT | NATURAL | JOIN | CROSS | ON + | UNION | INTERSECT | EXCEPT ; -identifier +strictIdentifier : IDENTIFIER #unquotedIdentifier | quotedIdentifier #quotedIdentifierAlternative | nonReserved #unquotedIdentifier @@ -652,6 +649,9 @@ nonReserved | AT | NULLS | OVERWRITE | ALL | ALTER | AS | BETWEEN | BY | CREATE | DELETE | DESCRIBE | DROP | EXISTS | FALSE | FOR | GROUP | IN | INSERT | INTO | IS |LIKE | NULL | ORDER | OUTER | TABLE | TRUE | WITH | RLIKE + | AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN + | UNBOUNDED | WHEN + | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT ; SELECT: 'SELECT'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3473feec3209c..e380643f548ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -642,7 +642,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { val table = UnresolvedRelation( visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.identifier).map(_.getText)) + Option(ctx.strictIdentifier).map(_.getText)) table.optionalMap(ctx.sample)(withSample) } @@ -692,7 +692,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * hooks. */ override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) { - plan(ctx.relation).optionalMap(ctx.sample)(withSample).optionalMap(ctx.identifier)(aliasPlan) + plan(ctx.relation) + .optionalMap(ctx.sample)(withSample) + .optionalMap(ctx.strictIdentifier)(aliasPlan) } /** @@ -701,13 +703,15 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * hooks. */ override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) { - plan(ctx.queryNoWith).optionalMap(ctx.sample)(withSample).optionalMap(ctx.identifier)(aliasPlan) + plan(ctx.queryNoWith) + .optionalMap(ctx.sample)(withSample) + .optionalMap(ctx.strictIdentifier)(aliasPlan) } /** * Create an alias (SubqueryAlias) for a LogicalPlan. */ - private def aliasPlan(alias: IdentifierContext, plan: LogicalPlan): LogicalPlan = { + private def aliasPlan(alias: ParserRuleContext, plan: LogicalPlan): LogicalPlan = { SubqueryAlias(alias.getText, plan) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala index 40782978a7299..020fb16f6f3d5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser import org.apache.spark.SparkFunSuite import org.apache.spark.sql.types._ -class CatalystQlDataTypeParserSuite extends SparkFunSuite { +class DataTypeParserSuite extends SparkFunSuite { def parse(sql: String): DataType = CatalystSqlParser.parseDataType(sql) @@ -133,4 +133,8 @@ class CatalystQlDataTypeParserSuite extends SparkFunSuite { checkDataType( "struct<`x``y` int>", (new StructType).add("x`y", IntegerType)) + + // Use SQL keywords. + checkDataType("struct", + (new StructType).add("end", LongType).add("select", IntegerType).add("from", StringType)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala index 6da3eaea3d850..f67697eb86c26 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala @@ -39,8 +39,6 @@ class ErrorParserSuite extends SparkFunSuite { } test("no viable input") { - intercept("select from tbl", 1, 7, "no viable alternative at input", "-------^^^") - intercept("select\nfrom tbl", 2, 0, "no viable alternative at input", "^^^") intercept("select ((r + 1) ", 1, 16, "no viable alternative at input", "----------------^^^") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index a6fad2d8a0398..77023cfd3d60f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -107,6 +107,7 @@ class PlanParserSuite extends PlanTest { table("db", "c").select('a, 'b).where('x < 1)) assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b))) assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b)) + assertEqual("select from tbl", OneRowRelation.select('from.as("tbl"))) } test("reverse select query") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index bef7d38f1a40e..8bbf87e62d412 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -53,8 +53,9 @@ class TableIdentifierParserSuite extends SparkFunSuite { "bigint", "binary", "boolean", "current_date", "current_timestamp", "date", "double", "float", "int", "smallint", "timestamp", "at") - val hiveNonReservedRegression = Seq("left", "right", "left", "right", "full", "inner", "semi", - "union", "except", "intersect", "schema", "database") + val hiveStrictNonReservedKeyword = Seq("anti", "full", "inner", "left", "semi", "right", + "natural", "union", "intersect", "except", "database", "on", "join", "cross", "select", "from", + "where", "having", "from", "to", "table", "with", "not") test("table identifier") { // Regular names. @@ -67,11 +68,10 @@ class TableIdentifierParserSuite extends SparkFunSuite { } } - test("table identifier - keywords") { + test("table identifier - strict keywords") { // SQL Keywords. - val keywords = Seq("select", "from", "where") ++ hiveNonReservedRegression - keywords.foreach { keyword => - intercept[ParseException](parseTableIdentifier(keyword)) + hiveStrictNonReservedKeyword.foreach { keyword => + assert(TableIdentifier(keyword) === parseTableIdentifier(keyword)) assert(TableIdentifier(keyword) === parseTableIdentifier(s"`$keyword`")) assert(TableIdentifier(keyword, Option("db")) === parseTableIdentifier(s"db.`$keyword`")) } From a790ac5793e1988895341fa878f947b09b275926 Mon Sep 17 00:00:00 2001 From: yinxusen Date: Wed, 8 Jun 2016 09:18:04 +0100 Subject: [PATCH 0553/1470] [SPARK-15793][ML] Add maxSentenceLength for ml.Word2Vec ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-15793 Word2vec in ML package should have maxSentenceLength method for feature parity. ## How was this patch tested? Tested with Spark unit test. Author: yinxusen Closes #13536 from yinxusen/SPARK-15793. (cherry picked from commit 87706eb66cd1370862a1f8ea447484c80969e45f) Signed-off-by: Sean Owen --- .../apache/spark/ml/feature/Word2Vec.scala | 19 +++++++++++++++++++ .../spark/ml/feature/Word2VecSuite.scala | 1 + 2 files changed, 20 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 2d89eb05a5182..33515b22400be 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -87,6 +87,21 @@ private[feature] trait Word2VecBase extends Params /** @group getParam */ def getMinCount: Int = $(minCount) + /** + * Sets the maximum length (in words) of each sentence in the input data. + * Any sentence longer than this threshold will be divided into chunks of + * up to `maxSentenceLength` size. + * Default: 1000 + * @group param + */ + final val maxSentenceLength = new IntParam(this, "maxSentenceLength", "Maximum length " + + "(in words) of each sentence in the input data. Any sentence longer than this threshold will " + + "be divided into chunks up to the size.") + setDefault(maxSentenceLength -> 1000) + + /** @group getParam */ + def getMaxSentenceLength: Int = $(maxSentenceLength) + setDefault(stepSize -> 0.025) setDefault(maxIter -> 1) @@ -137,6 +152,9 @@ final class Word2Vec(override val uid: String) extends Estimator[Word2VecModel] /** @group setParam */ def setMinCount(value: Int): this.type = set(minCount, value) + /** @group setParam */ + def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, value) + @Since("2.0.0") override def fit(dataset: Dataset[_]): Word2VecModel = { transformSchema(dataset.schema, logging = true) @@ -149,6 +167,7 @@ final class Word2Vec(override val uid: String) extends Estimator[Word2VecModel] .setSeed($(seed)) .setVectorSize($(vectorSize)) .setWindowSize($(windowSize)) + .setMaxSentenceLength($(maxSentenceLength)) .fit(input) copyValues(new Word2VecModel(uid, wordVectors).setParent(this)) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index 280a36f56e931..16c74f6785875 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -191,6 +191,7 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul .setSeed(42L) .setStepSize(0.01) .setVectorSize(100) + .setMaxSentenceLength(500) testDefaultReadWrite(t) } From 5e9a8e715953feadaa16ecd0f8e1818272b9c952 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Wed, 8 Jun 2016 14:51:00 +0100 Subject: [PATCH 0554/1470] [MINOR] Fix Java Lint errors introduced by #13286 and #13280 ## What changes were proposed in this pull request? revived #13464 Fix Java Lint errors introduced by #13286 and #13280 Before: ``` Using `mvn` from path: /Users/pichu/Project/spark/build/apache-maven-3.3.9/bin/mvn Java HotSpot(TM) 64-Bit Server VM warning: ignoring option MaxPermSize=512M; support was removed in 8.0 Checkstyle checks failed at following occurrences: [ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[340,5] (whitespace) FileTabCharacter: Line contains a tab character. [ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[341,5] (whitespace) FileTabCharacter: Line contains a tab character. [ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[342,5] (whitespace) FileTabCharacter: Line contains a tab character. [ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[343,5] (whitespace) FileTabCharacter: Line contains a tab character. [ERROR] src/main/java/org/apache/spark/sql/streaming/OutputMode.java:[41,28] (naming) MethodName: Method name 'Append' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'. [ERROR] src/main/java/org/apache/spark/sql/streaming/OutputMode.java:[52,28] (naming) MethodName: Method name 'Complete' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'. [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[61,8] (imports) UnusedImports: Unused import - org.apache.parquet.schema.PrimitiveType. [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[62,8] (imports) UnusedImports: Unused import - org.apache.parquet.schema.Type. ``` ## How was this patch tested? ran `dev/lint-java` locally Author: Sandeep Singh Closes #13559 from techaddict/minor-3. (cherry picked from commit f958c1c3e292aba98d283637606890f353a9836c) Signed-off-by: Sean Owen --- dev/checkstyle-suppressions.xml | 2 ++ .../java/org/apache/spark/launcher/LauncherServer.java | 8 ++++---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml index bfc2e734e517d..31656ca0e5a60 100644 --- a/dev/checkstyle-suppressions.xml +++ b/dev/checkstyle-suppressions.xml @@ -42,4 +42,6 @@ files="src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java"/> + diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index 28e9420b28802..ae43f563e8b46 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -337,10 +337,10 @@ public void close() throws IOException { } super.close(); if (handle != null) { - if (!handle.getState().isFinal()) { - LOG.log(Level.WARNING, "Lost connection to spark application."); - handle.setState(SparkAppHandle.State.LOST); - } + if (!handle.getState().isFinal()) { + LOG.log(Level.WARNING, "Lost connection to spark application."); + handle.setState(SparkAppHandle.State.LOST); + } handle.disconnect(); } } From b2778c8bbdf3b3a2e650b17346f87f2568f88295 Mon Sep 17 00:00:00 2001 From: prabs Date: Wed, 8 Jun 2016 17:22:55 +0100 Subject: [PATCH 0555/1470] [DOCUMENTATION] Fixed target JAR path ## What changes were proposed in this pull request? Mentioned Scala version in the sbt configuration file is 2.11, so the path of the target JAR should be `/target/scala-2.11/simple-project_2.11-1.0.jar` ## How was this patch tested? n/a Author: prabs Author: Prabeesh K Closes #13554 from prabeesh/master. (cherry picked from commit ca70ab27cc73f6ea7fce5d179ca8f13459c8ba95) Signed-off-by: Sean Owen --- docs/quick-start.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index 72372a6bc8543..1b961fd45576b 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -289,13 +289,13 @@ $ find . # Package a jar containing your application $ sbt package ... -[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar +[info] Packaging {..}/{..}/target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar # Use spark-submit to run your application $ YOUR_SPARK_HOME/bin/spark-submit \ --class "SimpleApp" \ --master local[4] \ - target/scala-2.10/simple-project_2.10-1.0.jar + target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} From 2c39b9a5d84219dbd315a93bc6d529dc72fa0a88 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 8 Jun 2016 16:21:41 -0700 Subject: [PATCH 0556/1470] [SPARK-15735] Allow specifying min time to run in microbenchmarks ## What changes were proposed in this pull request? This makes microbenchmarks run for at least 2 seconds by default, to allow some time for jit compilation to kick in. ## How was this patch tested? Tested manually with existing microbenchmarks. This change is backwards compatible in that existing microbenchmarks which specified numIters per-case will still run exactly that number of iterations. Microbenchmarks which previously overrode defaultNumIters now override minNumIters. cc hvanhovell Author: Eric Liang Author: Eric Liang Closes #13472 from ericl/spark-15735. (cherry picked from commit 4e8ac6edd5808ca8245b39d804c6d4f5ea9d0d36) Signed-off-by: Herman van Hovell --- .../org/apache/spark/util/Benchmark.scala | 109 ++++++++++++------ 1 file changed, 72 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 0c685b1918a59..7def44bd2a2b1 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -17,10 +17,14 @@ package org.apache.spark.util +import java.io.{OutputStream, PrintStream} + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ import scala.util.Try +import org.apache.commons.io.output.TeeOutputStream import org.apache.commons.lang3.SystemUtils /** @@ -33,18 +37,37 @@ import org.apache.commons.lang3.SystemUtils * * The benchmark function takes one argument that is the iteration that's being run. * - * If outputPerIteration is true, the timing for each run will be printed to stdout. + * @param name name of this benchmark. + * @param valuesPerIteration number of values used in the test case, used to compute rows/s. + * @param minNumIters the min number of iterations that will be run per case, not counting warm-up. + * @param warmupTime amount of time to spend running dummy case iterations for JIT warm-up. + * @param minTime further iterations will be run for each case until this time is used up. + * @param outputPerIteration if true, the timing for each run will be printed to stdout. + * @param output optional output stream to write benchmark results to */ private[spark] class Benchmark( name: String, valuesPerIteration: Long, - defaultNumIters: Int = 5, - outputPerIteration: Boolean = false) { + minNumIters: Int = 2, + warmupTime: FiniteDuration = 2.seconds, + minTime: FiniteDuration = 2.seconds, + outputPerIteration: Boolean = false, + output: Option[OutputStream] = None) { + import Benchmark._ val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] + val out = if (output.isDefined) { + new PrintStream(new TeeOutputStream(System.out, output.get)) + } else { + System.out + } + /** * Adds a case to run when run() is called. The given function will be run for several * iterations to collect timing statistics. + * + * @param name of the benchmark case + * @param numIters if non-zero, forces exactly this many iterations to be run */ def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { addTimerCase(name, numIters) { timer => @@ -58,9 +81,12 @@ private[spark] class Benchmark( * Adds a case with manual timing control. When the function is run, timing does not start * until timer.startTiming() is called within the given function. The corresponding * timer.stopTiming() method must be called before the function returns. + * + * @param name of the benchmark case + * @param numIters if non-zero, forces exactly this many iterations to be run */ def addTimerCase(name: String, numIters: Int = 0)(f: Benchmark.Timer => Unit): Unit = { - benchmarks += Benchmark.Case(name, f, if (numIters == 0) defaultNumIters else numIters) + benchmarks += Benchmark.Case(name, f, numIters) } /** @@ -75,28 +101,63 @@ private[spark] class Benchmark( val results = benchmarks.map { c => println(" Running case: " + c.name) - Benchmark.measure(valuesPerIteration, c.numIters, outputPerIteration)(c.fn) + measure(valuesPerIteration, c.numIters)(c.fn) } println val firstBest = results.head.bestMs // The results are going to be processor specific so it is useful to include that. - println(Benchmark.getJVMOSInfo()) - println(Benchmark.getProcessorName()) - printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + out.println(Benchmark.getJVMOSInfo()) + out.println(Benchmark.getProcessorName()) + out.printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", "Per Row(ns)", "Relative") - println("-" * 96) + out.println("-" * 96) results.zip(benchmarks).foreach { case (result, benchmark) => - printf("%-40s %16s %12s %13s %10s\n", + out.printf("%-40s %16s %12s %13s %10s\n", benchmark.name, "%5.0f / %4.0f" format (result.bestMs, result.avgMs), "%10.1f" format result.bestRate, "%6.1f" format (1000 / result.bestRate), "%3.1fX" format (firstBest / result.bestMs)) } - println + out.println // scalastyle:on } + + /** + * Runs a single function `f` for iters, returning the average time the function took and + * the rate of the function. + */ + def measure(num: Long, overrideNumIters: Int)(f: Timer => Unit): Result = { + System.gc() // ensures garbage from previous cases don't impact this one + val warmupDeadline = warmupTime.fromNow + while (!warmupDeadline.isOverdue) { + f(new Benchmark.Timer(-1)) + } + val minIters = if (overrideNumIters != 0) overrideNumIters else minNumIters + val minDuration = if (overrideNumIters != 0) 0 else minTime.toNanos + val runTimes = ArrayBuffer[Long]() + var i = 0 + while (i < minIters || runTimes.sum < minDuration) { + val timer = new Benchmark.Timer(i) + f(timer) + val runTime = timer.totalTime() + runTimes += runTime + + if (outputPerIteration) { + // scalastyle:off + println(s"Iteration $i took ${runTime / 1000} microseconds") + // scalastyle:on + } + i += 1 + } + // scalastyle:off + println(s" Stopped after $i iterations, ${runTimes.sum / 1000000} ms") + // scalastyle:on + val best = runTimes.min + val avg = runTimes.sum / runTimes.size + Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0) + } } private[spark] object Benchmark { @@ -161,30 +222,4 @@ private[spark] object Benchmark { val osVersion = System.getProperty("os.version") s"${vmName} ${runtimeVersion} on ${osName} ${osVersion}" } - - /** - * Runs a single function `f` for iters, returning the average time the function took and - * the rate of the function. - */ - def measure(num: Long, iters: Int, outputPerIteration: Boolean)(f: Timer => Unit): Result = { - val runTimes = ArrayBuffer[Long]() - for (i <- 0 until iters + 1) { - val timer = new Benchmark.Timer(i) - f(timer) - val runTime = timer.totalTime() - if (i > 0) { - runTimes += runTime - } - - if (outputPerIteration) { - // scalastyle:off - println(s"Iteration $i took ${runTime / 1000} microseconds") - // scalastyle:on - } - } - val best = runTimes.min - val avg = runTimes.sum / iters - Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0) - } } - From 48239b5f14ed22f18d13778e43515f3bd07bca59 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 8 Jun 2016 22:47:29 -0700 Subject: [PATCH 0557/1470] [SPARK-14670] [SQL] allow updating driver side sql metrics ## What changes were proposed in this pull request? On the SparkUI right now we have this SQLTab that displays accumulator values per operator. However, it only displays metrics updated on the executors, not on the driver. It is useful to also include driver metrics, e.g. broadcast time. This is a different version from https://github.com/apache/spark/pull/12427. This PR sends driver side accumulator updates right after the updating happens, not at the end of execution, by a new event. ## How was this patch tested? new test in `SQLListenerSuite` ![qq20160606-0](https://cloud.githubusercontent.com/assets/3182036/15841418/0eb137da-2c06-11e6-9068-5694eeb78530.png) Author: Wenchen Fan Closes #13189 from cloud-fan/metrics. (cherry picked from commit afbe35cf5b272991b4986e551b42d9201c3862c3) Signed-off-by: Davies Liu --- .../exchange/BroadcastExchangeExec.scala | 9 +++ .../spark/sql/execution/ui/SQLListener.scala | 28 ++++++++-- .../sql/execution/ui/SQLListenerSuite.scala | 56 ++++++++++++++++++- 3 files changed, 85 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index d3081ba7accd2..bd0841db7e8ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning} import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates import org.apache.spark.util.ThreadUtils /** @@ -92,6 +93,14 @@ case class BroadcastExchangeExec( val broadcasted = sparkContext.broadcast(relation) longMetric("broadcastTime") += (System.nanoTime() - beforeBroadcast) / 1000000 + + // There are some cases we don't care about the metrics and call `SparkPlan.doExecute` + // directly without setting an execution id. We should be tolerant to it. + if (executionId != null) { + sparkContext.listenerBus.post(SparkListenerDriverAccumUpdates( + executionId.toLong, metrics.values.map(m => m.id -> m.value).toSeq)) + } + broadcasted } }(BroadcastExchangeExec.executionContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 03b532664a0c8..6e94791901762 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -42,6 +42,10 @@ case class SparkListenerSQLExecutionStart( case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerDriverAccumUpdates(executionId: Long, accumUpdates: Seq[(Long, Long)]) + extends SparkListenerEvent + private[sql] class SQLHistoryListenerFactory extends SparkHistoryListenerFactory { override def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener] = { @@ -251,6 +255,13 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } } } + case SparkListenerDriverAccumUpdates(executionId, accumUpdates) => synchronized { + _executionIdToData.get(executionId).foreach { executionUIData => + for ((accId, accValue) <- accumUpdates) { + executionUIData.driverAccumUpdates(accId) = accValue + } + } + } case _ => // Ignore } @@ -296,7 +307,9 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi (accumulatorUpdate._1, accumulatorUpdate._2) } }.filter { case (id, _) => executionUIData.accumulatorMetrics.contains(id) } - mergeAccumulatorUpdates(accumulatorUpdates, accumulatorId => + + val driverUpdates = executionUIData.driverAccumUpdates.toSeq + mergeAccumulatorUpdates(accumulatorUpdates ++ driverUpdates, accumulatorId => executionUIData.accumulatorMetrics(accumulatorId).metricType) case None => // This execution has been dropped @@ -368,10 +381,15 @@ private[ui] class SQLExecutionUIData( val physicalPlanDescription: String, val physicalPlanGraph: SparkPlanGraph, val accumulatorMetrics: Map[Long, SQLPlanMetric], - val submissionTime: Long, - var completionTime: Option[Long] = None, - val jobs: mutable.HashMap[Long, JobExecutionStatus] = mutable.HashMap.empty, - val stages: mutable.ArrayBuffer[Int] = mutable.ArrayBuffer()) { + val submissionTime: Long) { + + var completionTime: Option[Long] = None + + val jobs: mutable.HashMap[Long, JobExecutionStatus] = mutable.HashMap.empty + + val stages: mutable.ArrayBuffer[Int] = mutable.ArrayBuffer() + + val driverAccumUpdates: mutable.HashMap[Long, Long] = mutable.HashMap.empty /** * Return whether there are running jobs in this execution. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 6788c9d65f6ec..6e60b0e4fad15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -23,11 +23,15 @@ import org.mockito.Mockito.mock import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.rdd.RDD import org.apache.spark.scheduler._ -import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AccumulatorMetadata, LongAccumulator} @@ -386,6 +390,52 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { assert(trackedAccums.head === (sqlMetricInfo.id, sqlMetricInfo.update.get)) } + test("driver side SQL metrics") { + val listener = new SQLListener(spark.sparkContext.conf) + val expectedAccumValue = 12345 + val physicalPlan = MyPlan(sqlContext.sparkContext, expectedAccumValue) + sqlContext.sparkContext.addSparkListener(listener) + val dummyQueryExecution = new QueryExecution(spark, LocalRelation()) { + override lazy val sparkPlan = physicalPlan + override lazy val executedPlan = physicalPlan + } + SQLExecution.withNewExecutionId(spark, dummyQueryExecution) { + physicalPlan.execute().collect() + } + + def waitTillExecutionFinished(): Unit = { + while (listener.getCompletedExecutions.isEmpty) { + Thread.sleep(100) + } + } + waitTillExecutionFinished() + + val driverUpdates = listener.getCompletedExecutions.head.driverAccumUpdates + assert(driverUpdates.size == 1) + assert(driverUpdates(physicalPlan.longMetric("dummy").id) == expectedAccumValue) + } + +} + + +/** + * A dummy [[org.apache.spark.sql.execution.SparkPlan]] that updates a [[SQLMetrics]] + * on the driver. + */ +private case class MyPlan(sc: SparkContext, expectedValue: Long) extends LeafExecNode { + override def sparkContext: SparkContext = sc + override def output: Seq[Attribute] = Seq() + + override val metrics: Map[String, SQLMetric] = Map( + "dummy" -> SQLMetrics.createMetric(sc, "dummy")) + + override def doExecute(): RDD[InternalRow] = { + longMetric("dummy") += expectedValue + sc.listenerBus.post(SparkListenerDriverAccumUpdates( + sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY).toLong, + metrics.values.map(m => m.id -> m.value).toSeq)) + sc.emptyRDD + } } From 96c011d5b5a28cedef22bebd96e7c7c96b252ca5 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Wed, 8 Jun 2016 23:41:29 -0700 Subject: [PATCH 0558/1470] [MINOR][DOC] In Dataset docs, remove self link to Dataset and add link to Column ## What changes were proposed in this pull request? Documentation Fix ## How was this patch tested? Author: Sandeep Singh Closes #13567 from techaddict/minor-4. (cherry picked from commit d5807def10c21e145163dc1e34d38258dda73ebf) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/Dataset.scala | 200 +++++++++--------- 1 file changed, 100 insertions(+), 100 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 6cbc27d91c1e8..162524a9efc3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -67,7 +67,7 @@ private[sql] object Dataset { } /** - * A [[Dataset]] is a strongly typed collection of domain-specific objects that can be transformed + * A Dataset is a strongly typed collection of domain-specific objects that can be transformed * in parallel using functional or relational operations. Each Dataset also has an untyped view * called a [[DataFrame]], which is a Dataset of [[Row]]. * @@ -105,7 +105,7 @@ private[sql] object Dataset { * }}} * * Dataset operations can also be untyped, through various domain-specific-language (DSL) - * functions defined in: [[Dataset]] (this class), [[Column]], and [[functions]]. These operations + * functions defined in: Dataset (this class), [[Column]], and [[functions]]. These operations * are very similar to the operations available in the data frame abstraction in R or Python. * * To select a column from the Dataset, use `apply` method in Scala and `col` in Java. @@ -194,13 +194,13 @@ class Dataset[T] private[sql]( /** * Currently [[ExpressionEncoder]] is the only implementation of [[Encoder]], here we turn the * passed in encoder to [[ExpressionEncoder]] explicitly, and mark it implicit so that we can use - * it when constructing new [[Dataset]] objects that have the same object type (that will be + * it when constructing new Dataset objects that have the same object type (that will be * possibly resolved to a different schema). */ private[sql] implicit val exprEnc: ExpressionEncoder[T] = encoderFor(encoder) /** - * Encoder is used mostly as a container of serde expressions in [[Dataset]]. We build logical + * Encoder is used mostly as a container of serde expressions in Dataset. We build logical * plans by these serde expressions and execute it within the query framework. However, for * performance reasons we may want to use encoder as a function to deserialize internal rows to * custom objects, e.g. collect. Here we resolve and bind the encoder so that we can call its @@ -340,7 +340,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Returns a new [[Dataset]] where each record has been mapped on to the specified type. The + * Returns a new Dataset where each record has been mapped on to the specified type. The * method used to map columns depend on the type of `U`: * - When `U` is a class, fields for the class will be mapped to columns of the same name * (case sensitivity is determined by `spark.sql.caseSensitive`). @@ -349,7 +349,7 @@ class Dataset[T] private[sql]( * - When `U` is a primitive type (i.e. String, Int, etc), then the first column of the * [[DataFrame]] will be used. * - * If the schema of the [[Dataset]] does not match the desired `U` type, you can use `select` + * If the schema of the Dataset does not match the desired `U` type, you can use `select` * along with `alias` or `as` to rearrange or rename as required. * * @group basic @@ -385,7 +385,7 @@ class Dataset[T] private[sql]( } /** - * Returns the schema of this [[Dataset]]. + * Returns the schema of this Dataset. * * @group basic * @since 1.6.0 @@ -453,8 +453,8 @@ class Dataset[T] private[sql]( def isLocal: Boolean = logicalPlan.isInstanceOf[LocalRelation] /** - * Returns true if this [[Dataset]] contains one or more sources that continuously - * return data as it arrives. A [[Dataset]] that reads data from a streaming source + * Returns true if this Dataset contains one or more sources that continuously + * return data as it arrives. A Dataset that reads data from a streaming source * must be executed as a [[ContinuousQuery]] using the `startStream()` method in * [[DataFrameWriter]]. Methods that return a single answer, e.g. `count()` or * `collect()`, will throw an [[AnalysisException]] when there is a streaming @@ -467,7 +467,7 @@ class Dataset[T] private[sql]( def isStreaming: Boolean = logicalPlan.isStreaming /** - * Displays the [[Dataset]] in a tabular form. Strings more than 20 characters will be truncated, + * Displays the Dataset in a tabular form. Strings more than 20 characters will be truncated, * and all cells will be aligned right. For example: * {{{ * year month AVG('Adj Close) MAX('Adj Close) @@ -486,7 +486,7 @@ class Dataset[T] private[sql]( def show(numRows: Int): Unit = show(numRows, truncate = true) /** - * Displays the top 20 rows of [[Dataset]] in a tabular form. Strings more than 20 characters + * Displays the top 20 rows of Dataset in a tabular form. Strings more than 20 characters * will be truncated, and all cells will be aligned right. * * @group action @@ -495,7 +495,7 @@ class Dataset[T] private[sql]( def show(): Unit = show(20) /** - * Displays the top 20 rows of [[Dataset]] in a tabular form. + * Displays the top 20 rows of Dataset in a tabular form. * * @param truncate Whether truncate long strings. If true, strings more than 20 characters will * be truncated and all cells will be aligned right @@ -506,7 +506,7 @@ class Dataset[T] private[sql]( def show(truncate: Boolean): Unit = show(20, truncate) /** - * Displays the [[Dataset]] in a tabular form. For example: + * Displays the Dataset in a tabular form. For example: * {{{ * year month AVG('Adj Close) MAX('Adj Close) * 1980 12 0.503218 0.595103 @@ -727,7 +727,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Joins this [[Dataset]] returning a [[Tuple2]] for each pair where `condition` evaluates to + * Joins this Dataset returning a [[Tuple2]] for each pair where `condition` evaluates to * true. * * This is similar to the relation `join` function with one important difference in the @@ -807,7 +807,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Using inner equi-join to join this [[Dataset]] returning a [[Tuple2]] for each pair + * Using inner equi-join to join this Dataset returning a [[Tuple2]] for each pair * where `condition` evaluates to true. * * @param other Right side of the join. @@ -822,7 +822,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with each partition sorted by the given expressions. + * Returns a new Dataset with each partition sorted by the given expressions. * * This is the same operation as "SORT BY" in SQL (Hive QL). * @@ -835,7 +835,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with each partition sorted by the given expressions. + * Returns a new Dataset with each partition sorted by the given expressions. * * This is the same operation as "SORT BY" in SQL (Hive QL). * @@ -848,7 +848,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] sorted by the specified column, all in ascending order. + * Returns a new Dataset sorted by the specified column, all in ascending order. * {{{ * // The following 3 are equivalent * ds.sort("sortcol") @@ -865,7 +865,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] sorted by the given expressions. For example: + * Returns a new Dataset sorted by the given expressions. For example: * {{{ * ds.sort($"col1", $"col2".desc) * }}} @@ -879,7 +879,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] sorted by the given expressions. + * Returns a new Dataset sorted by the given expressions. * This is an alias of the `sort` function. * * @group typedrel @@ -889,7 +889,7 @@ class Dataset[T] private[sql]( def orderBy(sortCol: String, sortCols: String*): Dataset[T] = sort(sortCol, sortCols : _*) /** - * Returns a new [[Dataset]] sorted by the given expressions. + * Returns a new Dataset sorted by the given expressions. * This is an alias of the `sort` function. * * @group typedrel @@ -923,7 +923,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with an alias set. + * Returns a new Dataset with an alias set. * * @group typedrel * @since 1.6.0 @@ -933,7 +933,7 @@ class Dataset[T] private[sql]( } /** - * (Scala-specific) Returns a new [[Dataset]] with an alias set. + * (Scala-specific) Returns a new Dataset with an alias set. * * @group typedrel * @since 2.0.0 @@ -941,7 +941,7 @@ class Dataset[T] private[sql]( def as(alias: Symbol): Dataset[T] = as(alias.name) /** - * Returns a new [[Dataset]] with an alias set. Same as `as`. + * Returns a new Dataset with an alias set. Same as `as`. * * @group typedrel * @since 2.0.0 @@ -949,7 +949,7 @@ class Dataset[T] private[sql]( def alias(alias: String): Dataset[T] = as(alias) /** - * (Scala-specific) Returns a new [[Dataset]] with an alias set. Same as `as`. + * (Scala-specific) Returns a new Dataset with an alias set. Same as `as`. * * @group typedrel * @since 2.0.0 @@ -1008,7 +1008,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Returns a new [[Dataset]] by computing the given [[Column]] expression for each element. + * Returns a new Dataset by computing the given [[Column]] expression for each element. * * {{{ * val ds = Seq(1, 2, 3).toDS() @@ -1045,7 +1045,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element. + * Returns a new Dataset by computing the given [[Column]] expressions for each element. * * @group typedrel * @since 1.6.0 @@ -1056,7 +1056,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element. + * Returns a new Dataset by computing the given [[Column]] expressions for each element. * * @group typedrel * @since 1.6.0 @@ -1070,7 +1070,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element. + * Returns a new Dataset by computing the given [[Column]] expressions for each element. * * @group typedrel * @since 1.6.0 @@ -1085,7 +1085,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element. + * Returns a new Dataset by computing the given [[Column]] expressions for each element. * * @group typedrel * @since 1.6.0 @@ -1154,7 +1154,7 @@ class Dataset[T] private[sql]( } /** - * Groups the [[Dataset]] using the specified columns, so we can run aggregation on them. See + * Groups the Dataset using the specified columns, so we can run aggregation on them. See * [[RelationalGroupedDataset]] for all the available aggregate functions. * * {{{ @@ -1177,7 +1177,7 @@ class Dataset[T] private[sql]( } /** - * Create a multi-dimensional rollup for the current [[Dataset]] using the specified columns, + * Create a multi-dimensional rollup for the current Dataset using the specified columns, * so we can run aggregation on them. * See [[RelationalGroupedDataset]] for all the available aggregate functions. * @@ -1201,7 +1201,7 @@ class Dataset[T] private[sql]( } /** - * Create a multi-dimensional cube for the current [[Dataset]] using the specified columns, + * Create a multi-dimensional cube for the current Dataset using the specified columns, * so we can run aggregation on them. * See [[RelationalGroupedDataset]] for all the available aggregate functions. * @@ -1225,7 +1225,7 @@ class Dataset[T] private[sql]( } /** - * Groups the [[Dataset]] using the specified columns, so that we can run aggregation on them. + * Groups the Dataset using the specified columns, so that we can run aggregation on them. * See [[RelationalGroupedDataset]] for all the available aggregate functions. * * This is a variant of groupBy that can only group by existing columns using column names @@ -1254,7 +1254,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Scala-specific) - * Reduces the elements of this [[Dataset]] using the specified binary function. The given `func` + * Reduces the elements of this Dataset using the specified binary function. The given `func` * must be commutative and associative or the result may be non-deterministic. * * @group action @@ -1310,7 +1310,7 @@ class Dataset[T] private[sql]( groupByKey(func.call(_))(encoder) /** - * Create a multi-dimensional rollup for the current [[Dataset]] using the specified columns, + * Create a multi-dimensional rollup for the current Dataset using the specified columns, * so we can run aggregation on them. * See [[RelationalGroupedDataset]] for all the available aggregate functions. * @@ -1339,7 +1339,7 @@ class Dataset[T] private[sql]( } /** - * Create a multi-dimensional cube for the current [[Dataset]] using the specified columns, + * Create a multi-dimensional cube for the current Dataset using the specified columns, * so we can run aggregation on them. * See [[RelationalGroupedDataset]] for all the available aggregate functions. * @@ -1367,7 +1367,7 @@ class Dataset[T] private[sql]( } /** - * (Scala-specific) Aggregates on the entire [[Dataset]] without groups. + * (Scala-specific) Aggregates on the entire Dataset without groups. * {{{ * // ds.agg(...) is a shorthand for ds.groupBy().agg(...) * ds.agg("age" -> "max", "salary" -> "avg") @@ -1382,7 +1382,7 @@ class Dataset[T] private[sql]( } /** - * (Scala-specific) Aggregates on the entire [[Dataset]] without groups. + * (Scala-specific) Aggregates on the entire Dataset without groups. * {{{ * // ds.agg(...) is a shorthand for ds.groupBy().agg(...) * ds.agg(Map("age" -> "max", "salary" -> "avg")) @@ -1395,7 +1395,7 @@ class Dataset[T] private[sql]( def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs) /** - * (Java-specific) Aggregates on the entire [[Dataset]] without groups. + * (Java-specific) Aggregates on the entire Dataset without groups. * {{{ * // ds.agg(...) is a shorthand for ds.groupBy().agg(...) * ds.agg(Map("age" -> "max", "salary" -> "avg")) @@ -1408,7 +1408,7 @@ class Dataset[T] private[sql]( def agg(exprs: java.util.Map[String, String]): DataFrame = groupBy().agg(exprs) /** - * Aggregates on the entire [[Dataset]] without groups. + * Aggregates on the entire Dataset without groups. * {{{ * // ds.agg(...) is a shorthand for ds.groupBy().agg(...) * ds.agg(max($"age"), avg($"salary")) @@ -1422,9 +1422,9 @@ class Dataset[T] private[sql]( def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs : _*) /** - * Returns a new [[Dataset]] by taking the first `n` rows. The difference between this function + * Returns a new Dataset by taking the first `n` rows. The difference between this function * and `head` is that `head` is an action and returns an array (by triggering query execution) - * while `limit` returns a new [[Dataset]]. + * while `limit` returns a new Dataset. * * @group typedrel * @since 2.0.0 @@ -1434,7 +1434,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] containing union of rows in this Dataset and another Dataset. + * Returns a new Dataset containing union of rows in this Dataset and another Dataset. * This is equivalent to `UNION ALL` in SQL. * * To do a SQL-style set union (that does deduplication of elements), use this function followed @@ -1447,7 +1447,7 @@ class Dataset[T] private[sql]( def unionAll(other: Dataset[T]): Dataset[T] = union(other) /** - * Returns a new [[Dataset]] containing union of rows in this Dataset and another Dataset. + * Returns a new Dataset containing union of rows in this Dataset and another Dataset. * This is equivalent to `UNION ALL` in SQL. * * To do a SQL-style set union (that does deduplication of elements), use this function followed @@ -1463,7 +1463,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] containing rows only in both this Dataset and another Dataset. + * Returns a new Dataset containing rows only in both this Dataset and another Dataset. * This is equivalent to `INTERSECT` in SQL. * * Note that, equality checking is performed directly on the encoded representation of the data @@ -1477,7 +1477,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] containing rows in this Dataset but not in another Dataset. + * Returns a new Dataset containing rows in this Dataset but not in another Dataset. * This is equivalent to `EXCEPT` in SQL. * * Note that, equality checking is performed directly on the encoded representation of the data @@ -1491,7 +1491,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] by sampling a fraction of rows. + * Returns a new Dataset by sampling a fraction of rows. * * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. @@ -1505,7 +1505,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] by sampling a fraction of rows, using a random seed. + * Returns a new Dataset by sampling a fraction of rows, using a random seed. * * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. @@ -1518,7 +1518,7 @@ class Dataset[T] private[sql]( } /** - * Randomly splits this [[Dataset]] with the provided weights. + * Randomly splits this Dataset with the provided weights. * * @param weights weights for splits, will be normalized if they don't sum to 1. * @param seed Seed for sampling. @@ -1545,7 +1545,7 @@ class Dataset[T] private[sql]( } /** - * Returns a Java list that contains randomly split [[Dataset]] with the provided weights. + * Returns a Java list that contains randomly split Dataset with the provided weights. * * @param weights weights for splits, will be normalized if they don't sum to 1. * @param seed Seed for sampling. @@ -1559,7 +1559,7 @@ class Dataset[T] private[sql]( } /** - * Randomly splits this [[Dataset]] with the provided weights. + * Randomly splits this Dataset with the provided weights. * * @param weights weights for splits, will be normalized if they don't sum to 1. * @group typedrel @@ -1570,7 +1570,7 @@ class Dataset[T] private[sql]( } /** - * Randomly splits this [[Dataset]] with the provided weights. Provided for the Python Api. + * Randomly splits this Dataset with the provided weights. Provided for the Python Api. * * @param weights weights for splits, will be normalized if they don't sum to 1. * @param seed Seed for sampling. @@ -1580,7 +1580,7 @@ class Dataset[T] private[sql]( } /** - * (Scala-specific) Returns a new [[Dataset]] where each row has been expanded to zero or more + * (Scala-specific) Returns a new Dataset where each row has been expanded to zero or more * rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of * the input row are implicitly joined with each row that is output by the function. * @@ -1623,7 +1623,7 @@ class Dataset[T] private[sql]( } /** - * (Scala-specific) Returns a new [[Dataset]] where a single column has been expanded to zero + * (Scala-specific) Returns a new Dataset where a single column has been expanded to zero * or more rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. All * columns of the input row are implicitly joined with each value that is output by the function. * @@ -1664,7 +1664,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] by adding a column or replacing the existing column that has + * Returns a new Dataset by adding a column or replacing the existing column that has * the same name. * * @group untypedrel @@ -1689,7 +1689,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] by adding a column with metadata. + * Returns a new Dataset by adding a column with metadata. */ private[spark] def withColumn(colName: String, col: Column, metadata: Metadata): DataFrame = { val resolver = sparkSession.sessionState.analyzer.resolver @@ -1710,7 +1710,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with a column renamed. + * Returns a new Dataset with a column renamed. * This is a no-op if schema doesn't contain existingName. * * @group untypedrel @@ -1735,7 +1735,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with a column dropped. This is a no-op if schema doesn't contain + * Returns a new Dataset with a column dropped. This is a no-op if schema doesn't contain * column name. * * This method can only be used to drop top level columns. the colName string is treated @@ -1749,7 +1749,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with columns dropped. + * Returns a new Dataset with columns dropped. * This is a no-op if schema doesn't contain column name(s). * * This method can only be used to drop top level columns. the colName string is treated literally @@ -1773,8 +1773,8 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with a column dropped. - * This version of drop accepts a Column rather than a name. + * Returns a new Dataset with a column dropped. + * This version of drop accepts a [[Column]] rather than a name. * This is a no-op if the Dataset doesn't have a column * with an equivalent expression. * @@ -1796,7 +1796,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] that contains only the unique rows from this [[Dataset]]. + * Returns a new Dataset that contains only the unique rows from this Dataset. * This is an alias for `distinct`. * * @group typedrel @@ -1805,7 +1805,7 @@ class Dataset[T] private[sql]( def dropDuplicates(): Dataset[T] = dropDuplicates(this.columns) /** - * (Scala-specific) Returns a new [[Dataset]] with duplicate rows removed, considering only + * (Scala-specific) Returns a new Dataset with duplicate rows removed, considering only * the subset of columns. * * @group typedrel @@ -1825,7 +1825,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] with duplicate rows removed, considering only + * Returns a new Dataset with duplicate rows removed, considering only * the subset of columns. * * @group typedrel @@ -1838,7 +1838,7 @@ class Dataset[T] private[sql]( * If no columns are given, this function computes statistics for all numerical columns. * * This function is meant for exploratory data analysis, as we make no guarantee about the - * backward compatibility of the schema of the resulting [[Dataset]]. If you want to + * backward compatibility of the schema of the resulting Dataset. If you want to * programmatically compute summary statistics, use the `agg` function instead. * * {{{ @@ -1937,7 +1937,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Scala-specific) - * Returns a new [[Dataset]] that only contains elements where `func` returns `true`. + * Returns a new Dataset that only contains elements where `func` returns `true`. * * @group typedrel * @since 1.6.0 @@ -1954,7 +1954,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Java-specific) - * Returns a new [[Dataset]] that only contains elements where `func` returns `true`. + * Returns a new Dataset that only contains elements where `func` returns `true`. * * @group typedrel * @since 1.6.0 @@ -1971,7 +1971,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Scala-specific) - * Returns a new [[Dataset]] that contains the result of applying `func` to each element. + * Returns a new Dataset that contains the result of applying `func` to each element. * * @group typedrel * @since 1.6.0 @@ -1984,7 +1984,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Java-specific) - * Returns a new [[Dataset]] that contains the result of applying `func` to each element. + * Returns a new Dataset that contains the result of applying `func` to each element. * * @group typedrel * @since 1.6.0 @@ -1998,7 +1998,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Scala-specific) - * Returns a new [[Dataset]] that contains the result of applying `func` to each partition. + * Returns a new Dataset that contains the result of applying `func` to each partition. * * @group typedrel * @since 1.6.0 @@ -2014,7 +2014,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Java-specific) - * Returns a new [[Dataset]] that contains the result of applying `f` to each partition. + * Returns a new Dataset that contains the result of applying `f` to each partition. * * @group typedrel * @since 1.6.0 @@ -2043,7 +2043,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Scala-specific) - * Returns a new [[Dataset]] by first applying a function to all elements of this [[Dataset]], + * Returns a new Dataset by first applying a function to all elements of this Dataset, * and then flattening the results. * * @group typedrel @@ -2056,7 +2056,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: * (Java-specific) - * Returns a new [[Dataset]] by first applying a function to all elements of this [[Dataset]], + * Returns a new Dataset by first applying a function to all elements of this Dataset, * and then flattening the results. * * @group typedrel @@ -2080,7 +2080,7 @@ class Dataset[T] private[sql]( /** * (Java-specific) - * Runs `func` on each element of this [[Dataset]]. + * Runs `func` on each element of this Dataset. * * @group action * @since 1.6.0 @@ -2088,7 +2088,7 @@ class Dataset[T] private[sql]( def foreach(func: ForeachFunction[T]): Unit = foreach(func.call(_)) /** - * Applies a function `f` to each partition of this [[Dataset]]. + * Applies a function `f` to each partition of this Dataset. * * @group action * @since 1.6.0 @@ -2099,7 +2099,7 @@ class Dataset[T] private[sql]( /** * (Java-specific) - * Runs `func` on each partition of this [[Dataset]]. + * Runs `func` on each partition of this Dataset. * * @group action * @since 1.6.0 @@ -2108,7 +2108,7 @@ class Dataset[T] private[sql]( foreachPartition(it => func.call(it.asJava)) /** - * Returns the first `n` rows in the [[Dataset]]. + * Returns the first `n` rows in the Dataset. * * Running take requires moving data into the application's driver process, and doing so with * a very large `n` can crash the driver process with OutOfMemoryError. @@ -2119,7 +2119,7 @@ class Dataset[T] private[sql]( def take(n: Int): Array[T] = head(n) /** - * Returns the first `n` rows in the [[Dataset]] as a list. + * Returns the first `n` rows in the Dataset as a list. * * Running take requires moving data into the application's driver process, and doing so with * a very large `n` can crash the driver process with OutOfMemoryError. @@ -2130,7 +2130,7 @@ class Dataset[T] private[sql]( def takeAsList(n: Int): java.util.List[T] = java.util.Arrays.asList(take(n) : _*) /** - * Returns an array that contains all of [[Row]]s in this [[Dataset]]. + * Returns an array that contains all of [[Row]]s in this Dataset. * * Running collect requires moving all the data into the application's driver process, and * doing so on a very large dataset can crash the driver process with OutOfMemoryError. @@ -2143,7 +2143,7 @@ class Dataset[T] private[sql]( def collect(): Array[T] = collect(needCallback = true) /** - * Returns a Java list that contains all of [[Row]]s in this [[Dataset]]. + * Returns a Java list that contains all of [[Row]]s in this Dataset. * * Running collect requires moving all the data into the application's driver process, and * doing so on a very large dataset can crash the driver process with OutOfMemoryError. @@ -2171,9 +2171,9 @@ class Dataset[T] private[sql]( } /** - * Return an iterator that contains all of [[Row]]s in this [[Dataset]]. + * Return an iterator that contains all of [[Row]]s in this Dataset. * - * The iterator will consume as much memory as the largest partition in this [[Dataset]]. + * The iterator will consume as much memory as the largest partition in this Dataset. * * Note: this results in multiple Spark jobs, and if the input Dataset is the result * of a wide transformation (e.g. join with different partitioners), to avoid @@ -2189,7 +2189,7 @@ class Dataset[T] private[sql]( } /** - * Returns the number of rows in the [[Dataset]]. + * Returns the number of rows in the Dataset. * @group action * @since 1.6.0 */ @@ -2198,7 +2198,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] that has exactly `numPartitions` partitions. + * Returns a new Dataset that has exactly `numPartitions` partitions. * * @group typedrel * @since 1.6.0 @@ -2208,7 +2208,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] partitioned by the given partitioning expressions into + * Returns a new Dataset partitioned by the given partitioning expressions into * `numPartitions`. The resulting Dataset is hash partitioned. * * This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL). @@ -2222,7 +2222,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] partitioned by the given partitioning expressions, using + * Returns a new Dataset partitioned by the given partitioning expressions, using * `spark.sql.shuffle.partitions` as number of partitions. * The resulting Dataset is hash partitioned. * @@ -2237,7 +2237,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] that has exactly `numPartitions` partitions. + * Returns a new Dataset 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. @@ -2250,7 +2250,7 @@ class Dataset[T] private[sql]( } /** - * Returns a new [[Dataset]] that contains only the unique rows from this [[Dataset]]. + * Returns a new Dataset that contains only the unique rows from this Dataset. * This is an alias for `dropDuplicates`. * * Note that, equality checking is performed directly on the encoded representation of the data @@ -2262,7 +2262,7 @@ class Dataset[T] private[sql]( def distinct(): Dataset[T] = dropDuplicates() /** - * Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`). + * Persist this Dataset with the default storage level (`MEMORY_AND_DISK`). * * @group basic * @since 1.6.0 @@ -2273,7 +2273,7 @@ class Dataset[T] private[sql]( } /** - * Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`). + * Persist this Dataset with the default storage level (`MEMORY_AND_DISK`). * * @group basic * @since 1.6.0 @@ -2281,7 +2281,7 @@ class Dataset[T] private[sql]( def cache(): this.type = persist() /** - * Persist this [[Dataset]] with the given storage level. + * Persist this Dataset with the given storage level. * @param newLevel One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`, * `MEMORY_AND_DISK_SER`, `DISK_ONLY`, `MEMORY_ONLY_2`, * `MEMORY_AND_DISK_2`, etc. @@ -2295,7 +2295,7 @@ class Dataset[T] private[sql]( } /** - * Mark the [[Dataset]] as non-persistent, and remove all blocks for it from memory and disk. + * Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. * * @param blocking Whether to block until all blocks are deleted. * @@ -2308,7 +2308,7 @@ class Dataset[T] private[sql]( } /** - * Mark the [[Dataset]] as non-persistent, and remove all blocks for it from memory and disk. + * Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. * * @group basic * @since 1.6.0 @@ -2316,7 +2316,7 @@ class Dataset[T] private[sql]( def unpersist(): this.type = unpersist(blocking = false) /** - * Represents the content of the [[Dataset]] as an [[RDD]] of [[T]]. + * Represents the content of the Dataset as an [[RDD]] of [[T]]. * * @group basic * @since 1.6.0 @@ -2330,21 +2330,21 @@ class Dataset[T] private[sql]( } /** - * Returns the content of the [[Dataset]] as a [[JavaRDD]] of [[Row]]s. + * Returns the content of the Dataset as a [[JavaRDD]] of [[Row]]s. * @group basic * @since 1.6.0 */ def toJavaRDD: JavaRDD[T] = rdd.toJavaRDD() /** - * Returns the content of the [[Dataset]] as a [[JavaRDD]] of [[Row]]s. + * Returns the content of the Dataset as a [[JavaRDD]] of [[Row]]s. * @group basic * @since 1.6.0 */ def javaRDD: JavaRDD[T] = toJavaRDD /** - * Registers this [[Dataset]] as a temporary table using the given name. The lifetime of this + * Registers this Dataset as a temporary table using the given name. The lifetime of this * temporary table is tied to the [[SparkSession]] that was used to create this Dataset. * * @group basic @@ -2394,7 +2394,7 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Interface for saving the content of the [[Dataset]] out into external storage or streams. + * Interface for saving the content of the Dataset out into external storage or streams. * * @group basic * @since 1.6.0 @@ -2403,7 +2403,7 @@ class Dataset[T] private[sql]( def write: DataFrameWriter = new DataFrameWriter(toDF()) /** - * Returns the content of the [[Dataset]] as a Dataset of JSON strings. + * Returns the content of the Dataset as a Dataset of JSON strings. * @since 2.0.0 */ def toJSON: Dataset[String] = { From 8ee93eed931b185b887882cc77c6fe8ddc907611 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 9 Jun 2016 00:51:24 -0700 Subject: [PATCH 0559/1470] [SPARK-12712] Fix failure in ./dev/test-dependencies when run against empty .m2 cache This patch fixes a bug in `./dev/test-dependencies.sh` which caused spurious failures when the script was run on a machine with an empty `.m2` cache. The problem was that extra log output from the dependency download was conflicting with the grep / regex used to identify the classpath in the Maven output. This patch fixes this issue by adjusting the regex pattern. Tested manually with the following reproduction of the bug: ``` rm -rf ~/.m2/repository/org/apache/commons/ ./dev/test-dependencies.sh ``` Author: Josh Rosen Closes #13568 from JoshRosen/SPARK-12712. (cherry picked from commit 921fa40b14082bfd1094fa49fb3b0c46a79c1aaa) Signed-off-by: Josh Rosen --- dev/test-dependencies.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 5ea643ee48d98..28e3d4d8d4f00 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -79,7 +79,7 @@ for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do echo "Generating dependency manifest for $HADOOP_PROFILE" mkdir -p dev/pr-deps $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE dependency:build-classpath -pl assembly \ - | grep "Building Spark Project Assembly" -A 5 \ + | grep "Dependencies classpath:" -A 1 \ | tail -n 1 | tr ":" "\n" | rev | cut -d "/" -f 1 | rev | sort \ | grep -v spark > dev/pr-deps/spark-deps-$HADOOP_PROFILE done From 77c08d2240bef7d814fc6e4dd0a53fbdf1e2f795 Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Thu, 9 Jun 2016 10:34:01 +0100 Subject: [PATCH 0560/1470] [SPARK-15818][BUILD] Upgrade to Hadoop 2.7.2 ## What changes were proposed in this pull request? Updating the Hadoop version from 2.7.0 to 2.7.2 if we use the Hadoop-2.7 build profile ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Existing tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) I'd like us to use Hadoop 2.7.2 owing to the Hadoop release notes stating Hadoop 2.7.0 is not ready for production use https://hadoop.apache.org/docs/r2.7.0/ states "Apache Hadoop 2.7.0 is a minor release in the 2.x.y release line, building upon the previous stable release 2.6.0. This release is not yet ready for production use. Production users should use 2.7.1 release and beyond." Hadoop 2.7.1 release notes: "Apache Hadoop 2.7.1 is a minor release in the 2.x.y release line, building upon the previous release 2.7.0. This is the next stable release after Apache Hadoop 2.6.x." And then Hadoop 2.7.2 release notes: "Apache Hadoop 2.7.2 is a minor release in the 2.x.y release line, building upon the previous stable release 2.7.1." I've tested this is OK with Intel hardware and IBM Java 8 so let's test it with OpenJDK, ideally this will be pushed to branch-2.0 and master. Author: Adam Roberts Closes #13556 from a-roberts/patch-2. (cherry picked from commit 147c020823080c60b495f7950629d8134bf895db) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.4 | 30 +++++++++++++++--------------- dev/deps/spark-deps-hadoop-2.6 | 30 +++++++++++++++--------------- dev/deps/spark-deps-hadoop-2.7 | 30 +++++++++++++++--------------- pom.xml | 6 +++--- 4 files changed, 48 insertions(+), 48 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 77d52666ba69e..3df292ee9956e 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -53,21 +53,21 @@ eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar -hadoop-annotations-2.4.0.jar -hadoop-auth-2.4.0.jar -hadoop-client-2.4.0.jar -hadoop-common-2.4.0.jar -hadoop-hdfs-2.4.0.jar -hadoop-mapreduce-client-app-2.4.0.jar -hadoop-mapreduce-client-common-2.4.0.jar -hadoop-mapreduce-client-core-2.4.0.jar -hadoop-mapreduce-client-jobclient-2.4.0.jar -hadoop-mapreduce-client-shuffle-2.4.0.jar -hadoop-yarn-api-2.4.0.jar -hadoop-yarn-client-2.4.0.jar -hadoop-yarn-common-2.4.0.jar -hadoop-yarn-server-common-2.4.0.jar -hadoop-yarn-server-web-proxy-2.4.0.jar +hadoop-annotations-2.4.1.jar +hadoop-auth-2.4.1.jar +hadoop-client-2.4.1.jar +hadoop-common-2.4.1.jar +hadoop-hdfs-2.4.1.jar +hadoop-mapreduce-client-app-2.4.1.jar +hadoop-mapreduce-client-common-2.4.1.jar +hadoop-mapreduce-client-core-2.4.1.jar +hadoop-mapreduce-client-jobclient-2.4.1.jar +hadoop-mapreduce-client-shuffle-2.4.1.jar +hadoop-yarn-api-2.4.1.jar +hadoop-yarn-client-2.4.1.jar +hadoop-yarn-common-2.4.1.jar +hadoop-yarn-server-common-2.4.1.jar +hadoop-yarn-server-web-proxy-2.4.1.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 9afe50f765d3c..9540f5856bce0 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -58,21 +58,21 @@ gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar -hadoop-annotations-2.6.0.jar -hadoop-auth-2.6.0.jar -hadoop-client-2.6.0.jar -hadoop-common-2.6.0.jar -hadoop-hdfs-2.6.0.jar -hadoop-mapreduce-client-app-2.6.0.jar -hadoop-mapreduce-client-common-2.6.0.jar -hadoop-mapreduce-client-core-2.6.0.jar -hadoop-mapreduce-client-jobclient-2.6.0.jar -hadoop-mapreduce-client-shuffle-2.6.0.jar -hadoop-yarn-api-2.6.0.jar -hadoop-yarn-client-2.6.0.jar -hadoop-yarn-common-2.6.0.jar -hadoop-yarn-server-common-2.6.0.jar -hadoop-yarn-server-web-proxy-2.6.0.jar +hadoop-annotations-2.6.4.jar +hadoop-auth-2.6.4.jar +hadoop-client-2.6.4.jar +hadoop-common-2.6.4.jar +hadoop-hdfs-2.6.4.jar +hadoop-mapreduce-client-app-2.6.4.jar +hadoop-mapreduce-client-common-2.6.4.jar +hadoop-mapreduce-client-core-2.6.4.jar +hadoop-mapreduce-client-jobclient-2.6.4.jar +hadoop-mapreduce-client-shuffle-2.6.4.jar +hadoop-yarn-api-2.6.4.jar +hadoop-yarn-client-2.6.4.jar +hadoop-yarn-common-2.6.4.jar +hadoop-yarn-server-common-2.6.4.jar +hadoop-yarn-server-web-proxy-2.6.4.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 879157a6dce7e..b5c3de75a9c8f 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -58,21 +58,21 @@ gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar -hadoop-annotations-2.7.0.jar -hadoop-auth-2.7.0.jar -hadoop-client-2.7.0.jar -hadoop-common-2.7.0.jar -hadoop-hdfs-2.7.0.jar -hadoop-mapreduce-client-app-2.7.0.jar -hadoop-mapreduce-client-common-2.7.0.jar -hadoop-mapreduce-client-core-2.7.0.jar -hadoop-mapreduce-client-jobclient-2.7.0.jar -hadoop-mapreduce-client-shuffle-2.7.0.jar -hadoop-yarn-api-2.7.0.jar -hadoop-yarn-client-2.7.0.jar -hadoop-yarn-common-2.7.0.jar -hadoop-yarn-server-common-2.7.0.jar -hadoop-yarn-server-web-proxy-2.7.0.jar +hadoop-annotations-2.7.2.jar +hadoop-auth-2.7.2.jar +hadoop-client-2.7.2.jar +hadoop-common-2.7.2.jar +hadoop-hdfs-2.7.2.jar +hadoop-mapreduce-client-app-2.7.2.jar +hadoop-mapreduce-client-common-2.7.2.jar +hadoop-mapreduce-client-core-2.7.2.jar +hadoop-mapreduce-client-jobclient-2.7.2.jar +hadoop-mapreduce-client-shuffle-2.7.2.jar +hadoop-yarn-api-2.7.2.jar +hadoop-yarn-client-2.7.2.jar +hadoop-yarn-common-2.7.2.jar +hadoop-yarn-server-common-2.7.2.jar +hadoop-yarn-server-web-proxy-2.7.2.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar diff --git a/pom.xml b/pom.xml index 92ffae2e5d31a..0419896025413 100644 --- a/pom.xml +++ b/pom.xml @@ -2484,7 +2484,7 @@ hadoop-2.4 - 2.4.0 + 2.4.1 0.9.3 @@ -2492,7 +2492,7 @@ hadoop-2.6 - 2.6.0 + 2.6.4 0.9.3 3.4.6 2.6.0 @@ -2502,7 +2502,7 @@ hadoop-2.7 - 2.7.0 + 2.7.2 0.9.3 3.4.6 2.6.0 From eb9e8fc097384dbe0d2cb83ca5b80968e3539c78 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Thu, 9 Jun 2016 09:50:09 -0700 Subject: [PATCH 0561/1470] [SPARK-15804][SQL] Include metadata in the toStructType ## What changes were proposed in this pull request? The help function 'toStructType' in the AttributeSeq class doesn't include the metadata when it builds the StructField, so it causes this reported problem https://issues.apache.org/jira/browse/SPARK-15804?jql=project%20%3D%20SPARK when spark writes the the dataframe with the metadata to the parquet datasource. The code path is when spark writes the dataframe to the parquet datasource through the InsertIntoHadoopFsRelationCommand, spark will build the WriteRelation container, and it will call the help function 'toStructType' to create StructType which contains StructField, it should include the metadata there, otherwise, we will lost the user provide metadata. ## How was this patch tested? added test case in ParquetQuerySuite.scala (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Kevin Yu Closes #13555 from kevinyu98/spark-15804. (cherry picked from commit 99386fe3989f758844de14b2c28eccfdf8163221) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/package.scala | 2 +- .../datasources/parquet/ParquetQuerySuite.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 81f5bb4a65096..a6125c61e508a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -91,7 +91,7 @@ package object expressions { implicit class AttributeSeq(val attrs: Seq[Attribute]) extends Serializable { /** Creates a StructType with a schema matching this `Seq[Attribute]`. */ def toStructType: StructType = { - StructType(attrs.map(a => StructField(a.name, a.dataType, a.nullable))) + StructType(attrs.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) } // It's possible that `attrs` is a linked list, which can lead to bad O(n^2) loops when diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 78b97f6995cdb..ea57f71c50171 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -625,6 +625,21 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } } + + test("SPARK-15804: write out the metadata to parquet file") { + val df = Seq((1, "abc"), (2, "hello")).toDF("a", "b") + val md = new MetadataBuilder().putString("key", "value").build() + val dfWithmeta = df.select('a, 'b.as("b", md)) + + withTempPath { dir => + val path = dir.getCanonicalPath + dfWithmeta.write.parquet(path) + + readParquetFile(path) { df => + assert(df.schema.last.metadata.getString("key") == "value") + } + } + } } object TestingUDT { From 10f7599471dd0d2b5efb49c5e1664fa24dfee074 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Thu, 9 Jun 2016 09:54:38 -0700 Subject: [PATCH 0562/1470] [SPARK-15788][PYSPARK][ML] PySpark IDFModel missing "idf" property ## What changes were proposed in this pull request? add method idf to IDF in pyspark ## How was this patch tested? add unit test Author: Jeff Zhang Closes #13540 from zjffdu/SPARK-15788. (cherry picked from commit e594b492836988ef3d9487b511368c70169d1ecd) Signed-off-by: Nick Pentreath --- python/pyspark/ml/feature.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1aff2e550f38c..ebe13006ad25b 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -585,6 +585,8 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab ... (DenseVector([0.0, 1.0]),), (DenseVector([3.0, 0.2]),)], ["tf"]) >>> idf = IDF(minDocFreq=3, inputCol="tf", outputCol="idf") >>> model = idf.fit(df) + >>> model.idf + DenseVector([0.0, 0.0]) >>> model.transform(df).head().idf DenseVector([0.0, 0.0]) >>> idf.setParams(outputCol="freqs").fit(df).transform(df).collect()[1].freqs @@ -658,6 +660,14 @@ class IDFModel(JavaModel, JavaMLReadable, JavaMLWritable): .. versionadded:: 1.4.0 """ + @property + @since("2.0.0") + def idf(self): + """ + Returns the IDF vector. + """ + return self._call_java("idf") + @inherit_doc class MaxAbsScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): From 07a914c09767deb67a0088b54ee48929a8567374 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 9 Jun 2016 11:04:08 -0700 Subject: [PATCH 0563/1470] [SPARK-15827][BUILD] Publish Spark's forked sbt-pom-reader to Maven Central Spark's SBT build currently uses a fork of the sbt-pom-reader plugin but depends on that fork via a SBT subproject which is cloned from https://github.com/scrapcodes/sbt-pom-reader/tree/ignore_artifact_id. This unnecessarily slows down the initial build on fresh machines and is also risky because it risks a build breakage in case that GitHub repository ever changes or is deleted. In order to address these issues, I have published a pre-built binary of our forked sbt-pom-reader plugin to Maven Central under the `org.spark-project` namespace and have updated Spark's build to use that artifact. This published artifact was built from https://github.com/JoshRosen/sbt-pom-reader/tree/v1.0.0-spark, which contains the contents of ScrapCodes's branch plus an additional patch to configure the build for artifact publication. /cc srowen ScrapCodes for review. Author: Josh Rosen Closes #13564 from JoshRosen/use-published-fork-of-pom-reader. (cherry picked from commit f74b77713e17960dddb7459eabfdc19f08f4024b) Signed-off-by: Josh Rosen --- project/plugins.sbt | 9 +++++++++ project/project/SparkPluginBuild.scala | 28 -------------------------- 2 files changed, 9 insertions(+), 28 deletions(-) delete mode 100644 project/project/SparkPluginBuild.scala diff --git a/project/plugins.sbt b/project/plugins.sbt index 4578b56247aa9..8bebd7bcac58c 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -21,3 +21,12 @@ libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3" libraryDependencies += "org.ow2.asm" % "asm-commons" % "5.0.3" addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.7.11") + +// Spark uses a custom fork of the sbt-pom-reader plugin which contains a patch to fix issues +// related to test-jar dependencies (https://github.com/sbt/sbt-pom-reader/pull/14). The source for +// this fork is published at https://github.com/JoshRosen/sbt-pom-reader/tree/v1.0.0-spark +// and corresponds to commit b160317fcb0b9d1009635a7c5aa05d0f3be61936 in that repository. +// In the long run, we should try to merge our patch upstream and switch to an upstream version of +// the plugin; this is tracked at SPARK-14401. + +addSbtPlugin("org.spark-project" % "sbt-pom-reader" % "1.0.0-spark") diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala deleted file mode 100644 index cbb88dc7dd1dd..0000000000000 --- a/project/project/SparkPluginBuild.scala +++ /dev/null @@ -1,28 +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. - */ - -import sbt._ -import sbt.Keys._ - -/** - * This plugin project is there because we use our custom fork of sbt-pom-reader plugin. This is - * a plugin project so that this gets compiled first and is available on the classpath for SBT build. - */ -object SparkPluginDef extends Build { - lazy val root = Project("plugins", file(".")) dependsOn(sbtPomReader) - lazy val sbtPomReader = uri("https://github.com/ScrapCodes/sbt-pom-reader.git#ignore_artifact_id") -} From 0408793aaab177920363913c9e91ac3af695370a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 9 Jun 2016 12:32:29 -0700 Subject: [PATCH 0564/1470] [SPARK-15839] Fix Maven doc-jar generation when JAVA_7_HOME is set ## What changes were proposed in this pull request? It looks like the nightly Maven snapshots broke after we set `JAVA_7_HOME` in the build: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20Packaging/job/spark-master-maven-snapshots/1573/. It seems that passing `-javabootclasspath` to ScalaDoc using scala-maven-plugin ends up preventing the Scala library classes from being added to scalac's internal class path, causing compilation errors while building doc-jars. There might be a principled fix to this inside of the scala-maven-plugin itself, but for now this patch configures the build to omit the `-javabootclasspath` option during Maven doc-jar generation. ## How was this patch tested? Tested manually with `build/mvn clean install -DskipTests=true` when `JAVA_7_HOME` was set. Also manually inspected the effective POM diff to verify that the final POM changes were scoped correctly: https://gist.github.com/JoshRosen/f889d1c236fad14fa25ac4be01654653 /cc vanzin and yhuai for review. Author: Josh Rosen Closes #13573 from JoshRosen/SPARK-15839. (cherry picked from commit 6cb71f4733a920d916b91c66bb2a508a21883b16) Signed-off-by: Yin Huai --- pom.xml | 29 +++++++++++++++++++++++------ 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 0419896025413..7f9ea44b7e3a5 100644 --- a/pom.xml +++ b/pom.xml @@ -2605,12 +2605,29 @@ net.alchim31.maven scala-maven-plugin - - - -javabootclasspath - ${env.JAVA_7_HOME}/jre/lib/rt.jar - - + + + + scala-compile-first + + + -javabootclasspath + ${env.JAVA_7_HOME}/jre/lib/rt.jar + + + + + scala-test-compile-first + + + -javabootclasspath + ${env.JAVA_7_HOME}/jre/lib/rt.jar + + + + From b42e3d886f1688feb6e9ca85a60ce5e6295e8489 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 9 Jun 2016 16:37:18 -0700 Subject: [PATCH 0565/1470] [SPARK-14321][SQL] Reduce date format cost and string-to-date cost in date functions ## What changes were proposed in this pull request? The current implementations of `UnixTime` and `FromUnixTime` do not cache their parser/formatter as much as they could. This PR resolved this issue. This PR is a take over from https://github.com/apache/spark/pull/13522 and further optimizes the re-use of the parser/formatter. It also fixes the improves handling (catching the actual exception instead of `Throwable`). All credits for this work should go to rajeshbalamohan. This PR closes https://github.com/apache/spark/pull/13522 ## How was this patch tested? Current tests. Author: Herman van Hovell Author: Rajesh Balamohan Closes #13581 from hvanhovell/SPARK-14321. (cherry picked from commit b0768538e56e5bbda7aaabbe2a0197e30ba5f993) Signed-off-by: Reynold Xin --- .../expressions/datetimeExpressions.scala | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 69c32f447e867..773431dd3346b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -399,6 +399,8 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { override def nullable: Boolean = true private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] + private lazy val formatter: SimpleDateFormat = + Try(new SimpleDateFormat(constFormat.toString)).getOrElse(null) override def eval(input: InternalRow): Any = { val t = left.eval(input) @@ -411,11 +413,11 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { case TimestampType => t.asInstanceOf[Long] / 1000000L case StringType if right.foldable => - if (constFormat != null) { - Try(new SimpleDateFormat(constFormat.toString).parse( - t.asInstanceOf[UTF8String].toString).getTime / 1000L).getOrElse(null) - } else { + if (constFormat == null || formatter == null) { null + } else { + Try(formatter.parse( + t.asInstanceOf[UTF8String].toString).getTime / 1000L).getOrElse(null) } case StringType => val f = right.eval(input) @@ -434,13 +436,10 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { left.dataType match { case StringType if right.foldable => val sdf = classOf[SimpleDateFormat].getName - val fString = if (constFormat == null) null else constFormat.toString - val formatter = ctx.freshName("formatter") - if (fString == null) { - ev.copy(code = s""" - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};""") + if (formatter == null) { + ExprCode("", "true", ctx.defaultValue(dataType)) } else { + val formatterName = ctx.addReferenceObj("formatter", formatter, sdf) val eval1 = left.genCode(ctx) ev.copy(code = s""" ${eval1.code} @@ -448,10 +447,8 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; if (!${ev.isNull}) { try { - $sdf $formatter = new $sdf("$fString"); - ${ev.value} = - $formatter.parse(${eval1.value}.toString()).getTime() / 1000L; - } catch (java.lang.Throwable e) { + ${ev.value} = $formatterName.parse(${eval1.value}.toString()).getTime() / 1000L; + } catch (java.text.ParseException e) { ${ev.isNull} = true; } }""") @@ -463,7 +460,9 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { try { ${ev.value} = (new $sdf($format.toString())).parse($string.toString()).getTime() / 1000L; - } catch (java.lang.Throwable e) { + } catch (java.lang.IllegalArgumentException e) { + ${ev.isNull} = true; + } catch (java.text.ParseException e) { ${ev.isNull} = true; } """ @@ -520,6 +519,8 @@ case class FromUnixTime(sec: Expression, format: Expression) override def inputTypes: Seq[AbstractDataType] = Seq(LongType, StringType) private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] + private lazy val formatter: SimpleDateFormat = + Try(new SimpleDateFormat(constFormat.toString)).getOrElse(null) override def eval(input: InternalRow): Any = { val time = left.eval(input) @@ -527,10 +528,10 @@ case class FromUnixTime(sec: Expression, format: Expression) null } else { if (format.foldable) { - if (constFormat == null) { + if (constFormat == null || formatter == null) { null } else { - Try(UTF8String.fromString(new SimpleDateFormat(constFormat.toString).format( + Try(UTF8String.fromString(formatter.format( new java.util.Date(time.asInstanceOf[Long] * 1000L)))).getOrElse(null) } } else { @@ -549,11 +550,10 @@ case class FromUnixTime(sec: Expression, format: Expression) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val sdf = classOf[SimpleDateFormat].getName if (format.foldable) { - if (constFormat == null) { - ev.copy(code = s""" - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};""") + if (formatter == null) { + ExprCode("", "true", "(UTF8String) null") } else { + val formatterName = ctx.addReferenceObj("formatter", formatter, sdf) val t = left.genCode(ctx) ev.copy(code = s""" ${t.code} @@ -561,9 +561,9 @@ case class FromUnixTime(sec: Expression, format: Expression) ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; if (!${ev.isNull}) { try { - ${ev.value} = UTF8String.fromString(new $sdf("${constFormat.toString}").format( + ${ev.value} = UTF8String.fromString($formatterName.format( new java.util.Date(${t.value} * 1000L))); - } catch (java.lang.Throwable e) { + } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; } }""") @@ -574,7 +574,7 @@ case class FromUnixTime(sec: Expression, format: Expression) try { ${ev.value} = UTF8String.fromString((new $sdf($f.toString())).format( new java.util.Date($seconds * 1000L))); - } catch (java.lang.Throwable e) { + } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; }""".stripMargin }) From b2d076c35d801219a25b420dd373383c08859a82 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 9 Jun 2016 17:31:19 -0700 Subject: [PATCH 0566/1470] [SPARK-12447][YARN] Only update the states when executor is successfully launched The details is described in https://issues.apache.org/jira/browse/SPARK-12447. vanzin Please help to review, thanks a lot. Author: jerryshao Closes #10412 from jerryshao/SPARK-12447. (cherry picked from commit aa0364510792c18a0973b6096cd38f611fc1c1a6) Signed-off-by: Marcelo Vanzin --- .../spark/deploy/yarn/ExecutorRunnable.scala | 5 +- .../spark/deploy/yarn/YarnAllocator.scala | 72 ++++++++++++------- 2 files changed, 47 insertions(+), 30 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 fc753b7e750cb..3d0e996b18720 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 @@ -55,15 +55,14 @@ private[yarn] class ExecutorRunnable( executorCores: Int, appId: String, securityMgr: SecurityManager, - localResources: Map[String, LocalResource]) - extends Runnable with Logging { + localResources: Map[String, LocalResource]) extends Logging { var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ val yarnConf: YarnConfiguration = new YarnConfiguration(conf) lazy val env = prepareEnvironment(container) - override def run(): Unit = { + def run(): Unit = { logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() nmClient.init(yarnConf) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 066c6659543a6..b110d82fb8a9e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -24,6 +24,7 @@ import java.util.regex.Pattern import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records._ @@ -472,41 +473,58 @@ private[yarn] class YarnAllocator( */ private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = { for (container <- containersToUse) { - numExecutorsRunning += 1 - assert(numExecutorsRunning <= targetNumExecutors) + executorIdCounter += 1 val executorHostname = container.getNodeId.getHost val containerId = container.getId - executorIdCounter += 1 val executorId = executorIdCounter.toString - assert(container.getResource.getMemory >= resource.getMemory) - logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) - executorIdToContainer(executorId) = container - containerIdToExecutorId(container.getId) = executorId - - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]) - - containerSet += containerId - allocatedContainerToHostMap.put(containerId, executorHostname) - - val executorRunnable = new ExecutorRunnable( - container, - conf, - sparkConf, - driverUrl, - executorId, - executorHostname, - executorMemory, - executorCores, - appAttemptId.getApplicationId.toString, - securityMgr, - localResources) + + def updateInternalState(): Unit = synchronized { + numExecutorsRunning += 1 + assert(numExecutorsRunning <= targetNumExecutors) + executorIdToContainer(executorId) = container + containerIdToExecutorId(container.getId) = executorId + + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, + new HashSet[ContainerId]) + containerSet += containerId + allocatedContainerToHostMap.put(containerId, executorHostname) + } + if (launchContainers) { logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( driverUrl, executorHostname)) - launcherPool.execute(executorRunnable) + + launcherPool.execute(new Runnable { + override def run(): Unit = { + try { + new ExecutorRunnable( + container, + conf, + sparkConf, + driverUrl, + executorId, + executorHostname, + executorMemory, + executorCores, + appAttemptId.getApplicationId.toString, + securityMgr, + localResources + ).run() + updateInternalState() + } catch { + case NonFatal(e) => + logError(s"Failed to launch executor $executorId on container $containerId", e) + // Assigned container should be released immediately to avoid unnecessary resource + // occupation. + amClient.releaseAssignedContainer(containerId) + } + } + }) + } else { + // For test only + updateInternalState() } } } From 3119d8eef6dc2d0805c87989746cd79882b9cfea Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 9 Jun 2016 17:45:37 -0700 Subject: [PATCH 0567/1470] [SPARK-15841][Tests] REPLSuite has incorrect env set for a couple of tests. Description from JIRA. In ReplSuite, for a test that can be tested well on just local should not really have to start a local-cluster. And similarly a test is in-sufficiently run if it's actually fixing a problem related to a distributed run in environment with local run. Existing tests. Author: Prashant Sharma Closes #13574 from ScrapCodes/SPARK-15841/repl-suite-fix. (cherry picked from commit 83070cd1d459101e1189f3b07ea59e22f98e84ce) Signed-off-by: Shixiong Zhu --- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 4 ++-- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 19f201f606dee..26b8600c32c11 100644 --- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -233,7 +233,7 @@ class ReplSuite extends SparkFunSuite { } test("SPARK-1199 two instances of same class don't type check.") { - val output = runInterpreter("local-cluster[1,1,1024]", + val output = runInterpreter("local", """ |case class Sum(exp: String, exp2: String) |val a = Sum("A", "B") @@ -305,7 +305,7 @@ class ReplSuite extends SparkFunSuite { } test("SPARK-2632 importing a method from non serializable class and not using it.") { - val output = runInterpreter("local", + val output = runInterpreter("local-cluster[1,1,1024]", """ |class TestClass() { def testMethod = 3 } |val t = new TestClass diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 48582c19163c9..2444e93d9ab6e 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -276,7 +276,7 @@ class ReplSuite extends SparkFunSuite { } test("SPARK-1199 two instances of same class don't type check.") { - val output = runInterpreter("local-cluster[1,1,1024]", + val output = runInterpreter("local", """ |case class Sum(exp: String, exp2: String) |val a = Sum("A", "B") @@ -336,7 +336,7 @@ class ReplSuite extends SparkFunSuite { } test("SPARK-2632 importing a method from non serializable class and not using it.") { - val output = runInterpreter("local", + val output = runInterpreter("local-cluster[1,1,1024]", """ |class TestClass() { def testMethod = 3 } |val t = new TestClass From 00bbf787340e208cc76230ffd96026c1305f942c Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 9 Jun 2016 18:05:16 -0700 Subject: [PATCH 0568/1470] [SPARK-15794] Should truncate toString() of very wide plans ## What changes were proposed in this pull request? With very wide tables, e.g. thousands of fields, the plan output is unreadable and often causes OOMs due to inefficient string processing. This truncates all struct and operator field lists to a user configurable threshold to limit performance impact. It would also be nice to optimize string generation to avoid these sort of O(n^2) slowdowns entirely (i.e. use StringBuilder everywhere including expressions), but this is probably too large of a change for 2.0 at this point, and truncation has other benefits for usability. ## How was this patch tested? Added a microbenchmark that covers this case particularly well. I also ran the microbenchmark while varying the truncation threshold. ``` numFields = 5 wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ 2000 wide x 50 rows (write in-mem) 2336 / 2558 0.0 23364.4 0.1X numFields = 25 wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ 2000 wide x 50 rows (write in-mem) 4237 / 4465 0.0 42367.9 0.1X numFields = 100 wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ 2000 wide x 50 rows (write in-mem) 10458 / 11223 0.0 104582.0 0.0X numFields = Infinity wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ [info] java.lang.OutOfMemoryError: Java heap space ``` Author: Eric Liang Author: Eric Liang Closes #13537 from ericl/truncated-string. (cherry picked from commit b914e1930fd5c5f2808f92d4958ec6fbeddf2e30) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/util/Utils.scala | 47 ++++++++++++++++++ .../org/apache/spark/util/UtilsSuite.scala | 8 +++ .../sql/catalyst/expressions/Expression.scala | 4 +- .../spark/sql/catalyst/trees/TreeNode.scala | 6 +-- .../apache/spark/sql/types/StructType.scala | 7 +-- .../spark/sql/execution/ExistingRDD.scala | 10 ++-- .../spark/sql/execution/QueryExecution.scala | 5 +- .../aggregate/HashAggregateExec.scala | 7 +-- .../aggregate/SortAggregateExec.scala | 7 +-- .../datasources/LogicalRelation.scala | 3 +- .../apache/spark/sql/execution/limit.scala | 5 +- .../execution/streaming/StreamExecution.scala | 3 +- .../sql/execution/streaming/memory.scala | 3 +- .../benchmark/WideSchemaBenchmark.scala | 49 +++++++++++++++++++ 14 files changed, 140 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 1a9dbcae8c083..f9d05409e1c3d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,6 +26,7 @@ import java.nio.charset.StandardCharsets import java.nio.file.Files import java.util.{Locale, Properties, Random, UUID} import java.util.concurrent._ +import java.util.concurrent.atomic.AtomicBoolean import javax.net.ssl.HttpsURLConnection import scala.annotation.tailrec @@ -78,6 +79,52 @@ private[spark] object Utils extends Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overriden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. + */ + val DEFAULT_MAX_TO_STRING_FIELDS = 25 + + private def maxNumToStringFields = { + if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) + } else { + DEFAULT_MAX_TO_STRING_FIELDS + } + } + + /** Whether we have warned about plan string truncation yet. */ + private val truncationWarningPrinted = new AtomicBoolean(false) + + /** + * Format a sequence with semantics similar to calling .mkString(). Any elements beyond + * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. + * + * @return the trimmed and formatted string. + */ + def truncatedString[T]( + seq: Seq[T], + start: String, + sep: String, + end: String, + maxNumFields: Int = maxNumToStringFields): String = { + if (seq.length > maxNumFields) { + if (truncationWarningPrinted.compareAndSet(false, true)) { + logWarning( + "Truncated the string representation of a plan since it was too large. This " + + "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") + } + val numFields = math.max(0, maxNumFields - 1) + seq.take(numFields).mkString( + start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) + } else { + seq.mkString(start, sep, end) + } + } + + /** Shorthand for calling truncatedString() without start or end strings. */ + def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "") /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { 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 66987498669d4..a5363f0bfd600 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -40,6 +40,14 @@ import org.apache.spark.network.util.ByteUnit class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { + test("truncatedString") { + assert(Utils.truncatedString(Nil, "[", ", ", "]", 2) == "[]") + assert(Utils.truncatedString(Seq(1, 2), "[", ", ", "]", 2) == "[1, 2]") + assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", 2) == "[1, ... 2 more fields]") + assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", -5) == "[, ... 3 more fields]") + assert(Utils.truncatedString(Seq(1, 2, 3), ", ") == "1, 2, 3") + } + test("timeConversion") { // Test -1 assert(Utils.timeStringAsSeconds("-1") === -1) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index efe592d69ddc8..10a141254f54e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the basic expression abstract classes in Catalyst. @@ -196,7 +197,8 @@ abstract class Expression extends TreeNode[Expression] { override def simpleString: String = toString - override def toString: String = prettyName + flatArguments.mkString("(", ", ", ")") + override def toString: String = prettyName + Utils.truncatedString( + flatArguments.toSeq, "(", ", ", ")") /** * Returns SQL representation of this expression. For expressions extending [[NonSQLExpression]], 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 c67366d240a30..f924efe6e6f44 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 @@ -448,10 +448,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case tn: TreeNode[_] => tn.simpleString :: Nil case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil case iter: Iterable[_] if iter.isEmpty => Nil - case seq: Seq[_] => seq.mkString("[", ", ", "]") :: Nil - case set: Set[_] => set.mkString("{", ", ", "}") :: Nil + case seq: Seq[_] => Utils.truncatedString(seq, "[", ", ", "]") :: Nil + case set: Set[_] => Utils.truncatedString(set.toSeq, "{", ", ", "}") :: Nil case array: Array[_] if array.isEmpty => Nil - case array: Array[_] => array.mkString("[", ", ", "]") :: Nil + case array: Array[_] => Utils.truncatedString(array, "[", ", ", "]") :: Nil case null => Nil case None => Nil case Some(null) => Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 9a923737598c3..436512ff69335 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -22,11 +22,12 @@ import scala.util.Try import org.json4s.JsonDSL._ -import org.apache.spark.SparkException +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -293,8 +294,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum override def simpleString: String = { - val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") - s"struct<${fieldTypes.mkString(",")}>" + val fieldTypes = fields.view.map(field => s"${field.name}:${field.dataType.simpleString}") + Utils.truncatedString(fieldTypes, "struct<", ",", ">") } override def sql: String = { 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 b8b392608de1b..9ab98fd124a34 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 @@ -33,6 +33,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.util.Utils object RDDConversions { def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { @@ -123,7 +124,7 @@ private[sql] case class RDDScanExec( } override def simpleString: String = { - s"Scan $nodeName${output.mkString("[", ",", "]")}" + s"Scan $nodeName${Utils.truncatedString(output, "[", ",", "]")}" } } @@ -186,7 +187,8 @@ private[sql] case class RowDataSourceScanExec( key + ": " + StringUtils.abbreviate(value, 100) } - s"$nodeName${output.mkString("[", ",", "]")}${metadataEntries.mkString(" ", ", ", "")}" + s"$nodeName${Utils.truncatedString(output, "[", ",", "]")}" + + s"${Utils.truncatedString(metadataEntries, " ", ", ", "")}" } override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -239,8 +241,8 @@ private[sql] case class BatchedDataSourceScanExec( val metadataEntries = for ((key, value) <- metadata.toSeq.sorted) yield { key + ": " + StringUtils.abbreviate(value, 100) } - val metadataStr = metadataEntries.mkString(" ", ", ", "") - s"Batched$nodeName${output.mkString("[", ",", "]")}$metadataStr" + val metadataStr = Utils.truncatedString(metadataEntries, " ", ", ", "") + s"Batched$nodeName${Utils.truncatedString(output, "[", ",", "]")}$metadataStr" } override def inputRDDs(): Seq[RDD[InternalRow]] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 560214a65e6ec..a2d45026e0b85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCom import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _} +import org.apache.spark.util.Utils /** * The primary workflow for executing relational queries using Spark. Designed to allow easy @@ -206,8 +207,8 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { } override def toString: String = { - def output = - analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}").mkString(", ") + def output = Utils.truncatedString( + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ") val analyzedPlan = Seq( stringOrError(output), stringOrError(analyzed.treeString(verbose = true)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index b617e26418c7e..caeeba1793680 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.{DecimalType, StringType, StructType} import org.apache.spark.unsafe.KVIterator +import org.apache.spark.util.Utils /** * Hash-based aggregate operator that can also fallback to sorting when data exceeds memory size. @@ -773,9 +774,9 @@ case class HashAggregateExec( testFallbackStartsAt match { case None => - val keyString = groupingExpressions.mkString("[", ",", "]") - val functionString = allAggregateExpressions.mkString("[", ",", "]") - val outputString = output.mkString("[", ",", "]") + val keyString = Utils.truncatedString(groupingExpressions, "[", ",", "]") + val functionString = Utils.truncatedString(allAggregateExpressions, "[", ",", "]") + val outputString = Utils.truncatedString(output, "[", ",", "]") if (verbose) { s"HashAggregate(key=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 41ba9f5b3feda..17126519eb0c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.util.Utils /** * Sort-based aggregate operator. @@ -110,9 +111,9 @@ case class SortAggregateExec( private def toString(verbose: Boolean): String = { val allAggregateExpressions = aggregateExpressions - val keyString = groupingExpressions.mkString("[", ",", "]") - val functionString = allAggregateExpressions.mkString("[", ",", "]") - val outputString = output.mkString("[", ",", "]") + val keyString = Utils.truncatedString(groupingExpressions, "[", ",", "]") + val functionString = Utils.truncatedString(allAggregateExpressions, "[", ",", "]") + val outputString = Utils.truncatedString(output, "[", ",", "]") if (verbose) { s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 0e0748ff32df3..a418d02983eb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.util.Utils /** * Used to link a [[BaseRelation]] in to a logical query plan. @@ -82,5 +83,5 @@ case class LogicalRelation( expectedOutputAttributes, metastoreTableIdentifier).asInstanceOf[this.type] - override def simpleString: String = s"Relation[${output.mkString(",")}] $relation" + override def simpleString: String = s"Relation[${Utils.truncatedString(output, ",")}] $relation" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index b71f3335c99e5..781c016095427 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, LazilyGeneratedOrdering} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.exchange.ShuffleExchange +import org.apache.spark.util.Utils /** @@ -159,8 +160,8 @@ case class TakeOrderedAndProjectExec( override def outputOrdering: Seq[SortOrder] = sortOrder override def simpleString: String = { - val orderByString = sortOrder.mkString("[", ",", "]") - val outputString = output.mkString("[", ",", "]") + val orderByString = Utils.truncatedString(sortOrder, "[", ",", "]") + val outputString = Utils.truncatedString(output, "[", ",", "]") s"TakeOrderedAndProject(limit=$limit, orderBy=$orderByString, output=$outputString)" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index d9800e4afd8e6..954fc33ecc7ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -336,7 +336,8 @@ class StreamExecution( newData.get(source).map { data => val newPlan = data.logicalPlan assert(output.size == newPlan.output.size, - s"Invalid batch: ${output.mkString(",")} != ${newPlan.output.mkString(",")}") + s"Invalid batch: ${Utils.truncatedString(output, ",")} != " + + s"${Utils.truncatedString(newPlan.output, ",")}") replacements ++= output.zip(newPlan.output) newPlan }.getOrElse { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 4496f41615a4d..77fd043ef7219 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils object MemoryStream { protected val currentBlockId = new AtomicInteger(0) @@ -81,7 +82,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } - override def toString: String = s"MemoryStream[${output.mkString(",")}]" + override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]" override def getOffset: Option[Offset] = synchronized { if (batches.isEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index b4811fe27a513..06466e629b3f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -155,6 +155,55 @@ many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Ro */ } + ignore("wide shallowly nested struct field read and write") { + val benchmark = new Benchmark( + "wide shallowly nested struct field r/w", scaleFactor) + for (width <- widthsToTest) { + val numRows = scaleFactor / width + var datum: String = "{" + for (i <- 1 to width) { + if (i == 1) { + datum += s""""value_$i": 1""" + } else { + datum += s""", "value_$i": 1""" + } + } + datum += "}" + datum = s"""{"a": {"b": {"c": $datum, "d": $datum}, "e": $datum}}""" + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum).rdd).cache() + df.count() // force caching + addCases(benchmark, df, s"$width wide x $numRows rows", "a.b.c.value_1") + } + benchmark.run() + +/* +Java HotSpot(TM) 64-Bit Server VM 1.7.0_80-b15 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 100 / 125 1.0 997.7 1.0X +1 wide x 100000 rows (write in-mem) 130 / 147 0.8 1302.9 0.8X +1 wide x 100000 rows (read parquet) 195 / 228 0.5 1951.4 0.5X +1 wide x 100000 rows (write parquet) 248 / 259 0.4 2479.7 0.4X +10 wide x 10000 rows (read in-mem) 76 / 89 1.3 757.2 1.3X +10 wide x 10000 rows (write in-mem) 90 / 116 1.1 900.0 1.1X +10 wide x 10000 rows (read parquet) 90 / 135 1.1 903.9 1.1X +10 wide x 10000 rows (write parquet) 222 / 240 0.4 2222.8 0.4X +100 wide x 1000 rows (read in-mem) 71 / 91 1.4 710.8 1.4X +100 wide x 1000 rows (write in-mem) 252 / 324 0.4 2522.4 0.4X +100 wide x 1000 rows (read parquet) 310 / 329 0.3 3095.9 0.3X +100 wide x 1000 rows (write parquet) 253 / 267 0.4 2525.7 0.4X +1000 wide x 100 rows (read in-mem) 144 / 160 0.7 1439.5 0.7X +1000 wide x 100 rows (write in-mem) 2055 / 2326 0.0 20553.9 0.0X +1000 wide x 100 rows (read parquet) 750 / 925 0.1 7496.8 0.1X +1000 wide x 100 rows (write parquet) 235 / 317 0.4 2347.5 0.4X +2500 wide x 40 rows (read in-mem) 219 / 227 0.5 2190.9 0.5X +2500 wide x 40 rows (write in-mem) 5177 / 5423 0.0 51773.2 0.0X +2500 wide x 40 rows (read parquet) 1642 / 1714 0.1 16417.7 0.1X +2500 wide x 40 rows (write parquet) 357 / 381 0.3 3568.2 0.3X +*/ + } + ignore("wide struct field read and write") { val benchmark = new Benchmark("wide struct field r/w", scaleFactor) for (width <- widthsToTest) { From ca0801120b3c650603b98b7838e86fee45f8999f Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 9 Jun 2016 18:45:19 -0700 Subject: [PATCH 0569/1470] [SPARK-15853][SQL] HDFSMetadataLog.get should close the input stream ## What changes were proposed in this pull request? This PR closes the input stream created in `HDFSMetadataLog.get` ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13583 from zsxwing/leak. (cherry picked from commit 4d9d9cc5853c467acdb67915117127915a98d8f8) Signed-off-by: Tathagata Das --- .../spark/sql/execution/streaming/HDFSMetadataLog.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index fca3d51535a29..069e41b6cedd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -175,8 +175,12 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) val batchMetadataFile = batchIdToPath(batchId) if (fileManager.exists(batchMetadataFile)) { val input = fileManager.open(batchMetadataFile) - val bytes = IOUtils.toByteArray(input) - Some(deserialize(bytes)) + try { + val bytes = IOUtils.toByteArray(input) + Some(deserialize(bytes)) + } finally { + input.close() + } } else { logDebug(s"Unable to find batch $batchMetadataFile") None From d45aa50fc7e41f2a8a659b2f512a846342a854dd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 9 Jun 2016 18:58:24 -0700 Subject: [PATCH 0570/1470] [SPARK-15850][SQL] Remove function grouping in SparkSession ## What changes were proposed in this pull request? SparkSession does not have that many functions due to better namespacing, and as a result we probably don't need the function grouping. This patch removes the grouping and also adds missing scaladocs for createDataset functions in SQLContext. Closes #13577. ## How was this patch tested? N/A - this is a documentation change. Author: Reynold Xin Closes #13582 from rxin/SPARK-15850. (cherry picked from commit 16df133d7f5f3115cd5baa696fa73a4694f9cba9) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/SQLContext.scala | 62 ++++++++++++++++++- .../org/apache/spark/sql/SparkSession.scala | 28 --------- .../org/apache/spark/sql/functions.scala | 2 +- 3 files changed, 61 insertions(+), 31 deletions(-) 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 0fb2400d1bafd..23f2b6e8942f3 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 @@ -51,7 +51,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager * @groupname specificdata Specific Data Sources * @groupname config Configuration * @groupname dataframes Custom DataFrame Creation - * @groupname dataset Custom DataFrame Creation + * @groupname dataset Custom Dataset Creation * @groupname Ungrouped Support functions for language integrated queries * @since 1.0.0 */ @@ -346,15 +346,73 @@ class SQLContext private[sql](val sparkSession: SparkSession) sparkSession.createDataFrame(rowRDD, schema, needsConversion) } - + /** + * :: Experimental :: + * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Example == + * + * {{{ + * + * import spark.implicits._ + * case class Person(name: String, age: Long) + * val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19)) + * val ds = spark.createDataset(data) + * + * ds.show() + * // +-------+---+ + * // | name|age| + * // +-------+---+ + * // |Michael| 29| + * // | Andy| 30| + * // | Justin| 19| + * // +-------+---+ + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { sparkSession.createDataset(data) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from an RDD of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { sparkSession.createDataset(data) } + /** + * :: Experimental :: + * Creates a [[Dataset]] from a [[java.util.List]] of a given type. This method requires an + * encoder (to convert a JVM object of type `T` to and from the internal Spark SQL representation) + * that is generally created automatically through implicits from a `SparkSession`, or can be + * created explicitly by calling static methods on [[Encoders]]. + * + * == Java Example == + * + * {{{ + * List data = Arrays.asList("hello", "world"); + * Dataset ds = spark.createDataset(data, Encoders.STRING()); + * }}} + * + * @since 2.0.0 + * @group dataset + */ + @Experimental def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { sparkSession.createDataset(data) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index f5b16d07ad80e..01c2e3ad29e48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -119,7 +119,6 @@ class SparkSession private( * configurations that are relevant to Spark SQL. When getting the value of a config, * this defaults to the value set in the underlying [[SparkContext]], if any. * - * @group config * @since 2.0.0 */ @transient lazy val conf: RuntimeConfig = new RuntimeConfig(sessionState.conf) @@ -129,7 +128,6 @@ class SparkSession private( * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s * that listen for execution metrics. * - * @group basic * @since 2.0.0 */ @Experimental @@ -140,7 +138,6 @@ class SparkSession private( * A collection of methods that are considered experimental, but can be used to hook into * the query planner for advanced functionality. * - * @group basic * @since 2.0.0 */ @Experimental @@ -175,7 +172,6 @@ class SparkSession private( * DataTypes.StringType); * }}} * - * @group basic * @since 2.0.0 */ def udf: UDFRegistration = sessionState.udf @@ -185,7 +181,6 @@ class SparkSession private( * Returns a [[ContinuousQueryManager]] that allows managing all the * [[ContinuousQuery ContinuousQueries]] active on `this`. * - * @group basic * @since 2.0.0 */ @Experimental @@ -200,7 +195,6 @@ class SparkSession private( * and child sessions are set up with the same shared state. If the underlying catalog * implementation is Hive, this will initialize the metastore, which may take some time. * - * @group basic * @since 2.0.0 */ def newSession(): SparkSession = { @@ -215,7 +209,6 @@ class SparkSession private( /** * Returns a [[DataFrame]] with no rows or columns. * - * @group dataframes * @since 2.0.0 */ @transient @@ -239,7 +232,6 @@ class SparkSession private( * :: Experimental :: * Creates a [[DataFrame]] from an RDD of Product (e.g. case classes, tuples). * - * @group dataframes * @since 2.0.0 */ @Experimental @@ -255,7 +247,6 @@ class SparkSession private( * :: Experimental :: * Creates a [[DataFrame]] from a local Seq of Product. * - * @group dataframes * @since 2.0.0 */ @Experimental @@ -295,7 +286,6 @@ class SparkSession private( * sparkSession.sql("select name from people").collect.foreach(println) * }}} * - * @group dataframes * @since 2.0.0 */ @DeveloperApi @@ -309,7 +299,6 @@ class SparkSession private( * It is important to make sure that the structure of every [[Row]] of the provided RDD matches * the provided schema. Otherwise, there will be runtime exception. * - * @group dataframes * @since 2.0.0 */ @DeveloperApi @@ -323,7 +312,6 @@ class SparkSession private( * It is important to make sure that the structure of every [[Row]] of the provided List matches * the provided schema. Otherwise, there will be runtime exception. * - * @group dataframes * @since 2.0.0 */ @DeveloperApi @@ -337,7 +325,6 @@ class SparkSession private( * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. * - * @group dataframes * @since 2.0.0 */ def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame = { @@ -357,7 +344,6 @@ class SparkSession private( * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. * - * @group dataframes * @since 2.0.0 */ def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { @@ -369,7 +355,6 @@ class SparkSession private( * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. - * @group dataframes * @since 1.6.0 */ def createDataFrame(data: java.util.List[_], beanClass: Class[_]): DataFrame = { @@ -382,7 +367,6 @@ class SparkSession private( /** * Convert a [[BaseRelation]] created for external data sources into a [[DataFrame]]. * - * @group dataframes * @since 2.0.0 */ def baseRelationToDataFrame(baseRelation: BaseRelation): DataFrame = { @@ -420,7 +404,6 @@ class SparkSession private( * }}} * * @since 2.0.0 - * @group dataset */ @Experimental def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { @@ -439,7 +422,6 @@ class SparkSession private( * created explicitly by calling static methods on [[Encoders]]. * * @since 2.0.0 - * @group dataset */ @Experimental def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { @@ -465,7 +447,6 @@ class SparkSession private( * }}} * * @since 2.0.0 - * @group dataset */ @Experimental def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { @@ -478,7 +459,6 @@ class SparkSession private( * in a range from 0 to `end` (exclusive) with step value 1. * * @since 2.0.0 - * @group dataset */ @Experimental def range(end: Long): Dataset[java.lang.Long] = range(0, end) @@ -489,7 +469,6 @@ class SparkSession private( * in a range from `start` to `end` (exclusive) with step value 1. * * @since 2.0.0 - * @group dataset */ @Experimental def range(start: Long, end: Long): Dataset[java.lang.Long] = { @@ -502,7 +481,6 @@ class SparkSession private( * in a range from `start` to `end` (exclusive) with a step value. * * @since 2.0.0 - * @group dataset */ @Experimental def range(start: Long, end: Long, step: Long): Dataset[java.lang.Long] = { @@ -516,7 +494,6 @@ class SparkSession private( * specified. * * @since 2.0.0 - * @group dataset */ @Experimental def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[java.lang.Long] = { @@ -565,7 +542,6 @@ class SparkSession private( * Interface through which the user may create, drop, alter or query underlying * databases, tables, functions etc. * - * @group ddl_ops * @since 2.0.0 */ @transient lazy val catalog: Catalog = new CatalogImpl(self) @@ -573,7 +549,6 @@ class SparkSession private( /** * Returns the specified table as a [[DataFrame]]. * - * @group ddl_ops * @since 2.0.0 */ def table(tableName: String): DataFrame = { @@ -592,7 +567,6 @@ class SparkSession private( * Executes a SQL query using Spark, returning the result as a [[DataFrame]]. * The dialect that is used for SQL parsing can be configured with 'spark.sql.dialect'. * - * @group basic * @since 2.0.0 */ def sql(sqlText: String): DataFrame = { @@ -606,7 +580,6 @@ class SparkSession private( * sparkSession.read.schema(schema).json("/path/to/file.json") * }}} * - * @group genericdata * @since 2.0.0 */ def read: DataFrameReader = new DataFrameReader(self) @@ -624,7 +597,6 @@ class SparkSession private( * import sparkSession.implicits._ * }}} * - * @group basic * @since 2.0.0 */ @Experimental 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 4dbd1665e4bbc..02608b0dce042 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 @@ -2437,7 +2437,7 @@ object functions { */ def minute(e: Column): Column = withExpr { Minute(e.expr) } - /* + /** * Returns number of months between dates `date1` and `date2`. * @group datetime_funcs * @since 1.5.0 From ebbbf2136412c628421b88a2bc83091a2b473c55 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 9 Jun 2016 22:28:31 -0700 Subject: [PATCH 0571/1470] [SPARK-15791] Fix NPE in ScalarSubquery ## What changes were proposed in this pull request? The fix is pretty simple, just don't make the executedPlan transient in `ScalarSubquery` since it is referenced at execution time. ## How was this patch tested? I verified the fix manually in non-local mode. It's not clear to me why the problem did not manifest in local mode, any suggestions? cc davies Author: Eric Liang Closes #13569 from ericl/fix-scalar-npe. (cherry picked from commit 6c5fd977fbcb821a57cb4a13bc3d413a695fbc32) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/execution/subquery.scala | 2 +- .../test/scala/org/apache/spark/sql/QueryTest.scala | 10 ++++++++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 3 ++- .../scala/org/apache/spark/sql/SubquerySuite.scala | 4 ++++ 4 files changed, 15 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 4a1f12d68519a..461d3010ada7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.types.DataType * This is the physical copy of ScalarSubquery to be used inside SparkPlan. */ case class ScalarSubquery( - @transient executedPlan: SparkPlan, + executedPlan: SparkPlan, exprId: ExprId) extends SubqueryExpression { 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 9c044f4e8fd6e..acb59d46e127d 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 @@ -341,10 +341,16 @@ object QueryTest { * * @param df the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param checkToRDD whether to verify deserialization to an RDD. This runs the query twice. */ - def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { + def checkAnswer( + df: DataFrame, + expectedAnswer: Seq[Row], + checkToRDD: Boolean = true): Option[String] = { val isSorted = df.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty - + if (checkToRDD) { + df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + } val sparkAnswer = try df.collect().toSeq catch { case e: Exception => 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 8284e8d6d89b6..90465b65bdb1c 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 @@ -2118,7 +2118,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { // is correct. def verifyCallCount(df: DataFrame, expectedResult: Row, expectedCount: Int): Unit = { countAcc.setValue(0) - checkAnswer(df, expectedResult) + QueryTest.checkAnswer( + df, Seq(expectedResult), checkToRDD = false /* avoid duplicate exec */) assert(countAcc.value == expectedCount) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index a932125f3cc6c..05491a4a888d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -54,6 +54,10 @@ class SubquerySuite extends QueryTest with SharedSQLContext { t.createOrReplaceTempView("t") } + test("rdd deserialization does not crash [SPARK-15791]") { + sql("select (select 1 as b) as b").rdd.count() + } + test("simple uncorrelated scalar subquery") { checkAnswer( sql("select (select 1 as b) as b"), From 1371d5ecedb76d138dc9f431e5b40e36a58ed9ca Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 9 Jun 2016 22:46:51 -0700 Subject: [PATCH 0572/1470] [SPARK-15696][SQL] Improve `crosstab` to have a consistent column order ## What changes were proposed in this pull request? Currently, `crosstab` returns a Dataframe having **random-order** columns obtained by just `distinct`. Also, the documentation of `crosstab` shows the result in a sorted order which is different from the current implementation. This PR explicitly constructs the columns in a sorted order in order to improve user experience. Also, this implementation gives the same result with the documentation. **Before** ```scala scala> spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3))).toDF("key", "value").stat.crosstab("key", "value").show() +---------+---+---+---+ |key_value| 3| 2| 1| +---------+---+---+---+ | 2| 1| 0| 2| | 1| 0| 1| 1| | 3| 1| 1| 0| +---------+---+---+---+ scala> spark.createDataFrame(Seq((1, "a"), (1, "b"), (2, "a"), (2, "a"), (2, "c"), (3, "b"), (3, "c"))).toDF("key", "value").stat.crosstab("key", "value").show() +---------+---+---+---+ |key_value| c| a| b| +---------+---+---+---+ | 2| 1| 2| 0| | 1| 0| 1| 1| | 3| 1| 0| 1| +---------+---+---+---+ ``` **After** ```scala scala> spark.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), (3, 3))).toDF("key", "value").stat.crosstab("key", "value").show() +---------+---+---+---+ |key_value| 1| 2| 3| +---------+---+---+---+ | 2| 2| 0| 1| | 1| 1| 1| 0| | 3| 0| 1| 1| +---------+---+---+---+ scala> spark.createDataFrame(Seq((1, "a"), (1, "b"), (2, "a"), (2, "a"), (2, "c"), (3, "b"), (3, "c"))).toDF("key", "value").stat.crosstab("key", "value").show() +---------+---+---+---+ |key_value| a| b| c| +---------+---+---+---+ | 2| 2| 0| 1| | 1| 1| 1| 0| | 3| 0| 1| 1| +---------+---+---+---+ ``` ## How was this patch tested? Pass the Jenkins tests with updated testcases. Author: Dongjoon Hyun Closes #13436 from dongjoon-hyun/SPARK-15696. (cherry picked from commit 5a3533e779d8e43ce0980203dfd3cbe343cc7d0a) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/execution/stat/StatFunctions.scala | 4 ++-- .../java/test/org/apache/spark/sql/JavaDataFrameSuite.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index 9c0406168e6e3..ea58df70b3252 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -423,9 +423,9 @@ private[sql] object StatFunctions extends Logging { def cleanElement(element: Any): String = { if (element == null) "null" else element.toString } - // get the distinct values of column 2, so that we can make them the column names + // get the distinct sorted values of column 2, so that we can make them the column names val distinctCol2: Map[Any, Int] = - counts.map(e => cleanElement(e.get(1))).distinct.zipWithIndex.toMap + counts.map(e => cleanElement(e.get(1))).distinct.sorted.zipWithIndex.toMap val columnSize = distinctCol2.size require(columnSize < 1e4, s"The number of distinct values for $col2, can't " + s"exceed 1e4. Currently $columnSize") 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 1e8f1062c53ee..0152f3f85a230 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 @@ -246,8 +246,8 @@ public void testCrosstab() { Dataset crosstab = df.stat().crosstab("a", "b"); String[] columnNames = crosstab.schema().fieldNames(); Assert.assertEquals("a_b", columnNames[0]); - Assert.assertEquals("2", columnNames[1]); - Assert.assertEquals("1", columnNames[2]); + Assert.assertEquals("1", columnNames[1]); + Assert.assertEquals("2", columnNames[2]); List rows = crosstab.collectAsList(); Collections.sort(rows, crosstabRowComparator); Integer count = 1; From 02ed7b536f54f46de9b10a8a4ea79544a7a813bf Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 10 Jun 2016 00:11:46 -0700 Subject: [PATCH 0573/1470] [SPARK-15593][SQL] Add DataFrameWriter.foreach to allow the user consuming data in ContinuousQuery ## What changes were proposed in this pull request? * Add DataFrameWriter.foreach to allow the user consuming data in ContinuousQuery * ForeachWriter is the interface for the user to consume partitions of data * Add a type parameter T to DataFrameWriter Usage ```Scala val ds = spark.read....stream().as[String] ds.....write .queryName(...) .option("checkpointLocation", ...) .foreach(new ForeachWriter[Int] { def open(partitionId: Long, version: Long): Boolean = { // prepare some resources for a partition // check `version` if possible and return `false` if this is a duplicated data to skip the data processing. } override def process(value: Int): Unit = { // process data } def close(errorOrNull: Throwable): Unit = { // release resources for a partition // check `errorOrNull` and handle the error if necessary. } }) ``` ## How was this patch tested? New unit tests. Author: Shixiong Zhu Closes #13342 from zsxwing/foreach. (cherry picked from commit 00c310133df4f3893dd90d801168c2ab9841b102) Signed-off-by: Tathagata Das --- .../apache/spark/sql/DataFrameWriter.scala | 150 +++++++++++++----- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../org/apache/spark/sql/ForeachWriter.scala | 105 ++++++++++++ .../sql/execution/streaming/ForeachSink.scala | 53 +++++++ .../streaming/ForeachSinkSuite.scala | 141 ++++++++++++++++ .../spark/sql/sources/BucketedReadSuite.scala | 4 +- 6 files changed, 413 insertions(+), 42 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 1dd8818dedb2e..32e2fdc3f9707 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, DataSource, HadoopFsRelation} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils -import org.apache.spark.sql.execution.streaming.{MemoryPlan, MemorySink, StreamExecution} +import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{ContinuousQuery, OutputMode, ProcessingTime, Trigger} import org.apache.spark.util.Utils @@ -40,7 +40,9 @@ import org.apache.spark.util.Utils * * @since 1.4.0 */ -final class DataFrameWriter private[sql](df: DataFrame) { +final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { + + private val df = ds.toDF() /** * Specifies the behavior when data or table already exists. Options include: @@ -51,7 +53,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def mode(saveMode: SaveMode): DataFrameWriter = { + def mode(saveMode: SaveMode): DataFrameWriter[T] = { // mode() is used for non-continuous queries // outputMode() is used for continuous queries assertNotStreaming("mode() can only be called on non-continuous queries") @@ -68,7 +70,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def mode(saveMode: String): DataFrameWriter = { + def mode(saveMode: String): DataFrameWriter[T] = { // mode() is used for non-continuous queries // outputMode() is used for continuous queries assertNotStreaming("mode() can only be called on non-continuous queries") @@ -93,7 +95,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ @Experimental - def outputMode(outputMode: OutputMode): DataFrameWriter = { + def outputMode(outputMode: OutputMode): DataFrameWriter[T] = { assertStreaming("outputMode() can only be called on continuous queries") this.outputMode = outputMode this @@ -109,7 +111,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ @Experimental - def outputMode(outputMode: String): DataFrameWriter = { + def outputMode(outputMode: String): DataFrameWriter[T] = { assertStreaming("outputMode() can only be called on continuous queries") this.outputMode = outputMode.toLowerCase match { case "append" => @@ -147,7 +149,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ @Experimental - def trigger(trigger: Trigger): DataFrameWriter = { + def trigger(trigger: Trigger): DataFrameWriter[T] = { assertStreaming("trigger() can only be called on continuous queries") this.trigger = trigger this @@ -158,7 +160,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def format(source: String): DataFrameWriter = { + def format(source: String): DataFrameWriter[T] = { this.source = source this } @@ -168,7 +170,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def option(key: String, value: String): DataFrameWriter = { + def option(key: String, value: String): DataFrameWriter[T] = { this.extraOptions += (key -> value) this } @@ -178,28 +180,28 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 2.0.0 */ - def option(key: String, value: Boolean): DataFrameWriter = option(key, value.toString) + def option(key: String, value: Boolean): DataFrameWriter[T] = option(key, value.toString) /** * Adds an output option for the underlying data source. * * @since 2.0.0 */ - def option(key: String, value: Long): DataFrameWriter = option(key, value.toString) + def option(key: String, value: Long): DataFrameWriter[T] = option(key, value.toString) /** * Adds an output option for the underlying data source. * * @since 2.0.0 */ - def option(key: String, value: Double): DataFrameWriter = option(key, value.toString) + def option(key: String, value: Double): DataFrameWriter[T] = option(key, value.toString) /** * (Scala-specific) Adds output options for the underlying data source. * * @since 1.4.0 */ - def options(options: scala.collection.Map[String, String]): DataFrameWriter = { + def options(options: scala.collection.Map[String, String]): DataFrameWriter[T] = { this.extraOptions ++= options this } @@ -209,7 +211,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * @since 1.4.0 */ - def options(options: java.util.Map[String, String]): DataFrameWriter = { + def options(options: java.util.Map[String, String]): DataFrameWriter[T] = { this.options(options.asScala) this } @@ -232,7 +234,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ @scala.annotation.varargs - def partitionBy(colNames: String*): DataFrameWriter = { + def partitionBy(colNames: String*): DataFrameWriter[T] = { this.partitioningColumns = Option(colNames) this } @@ -246,7 +248,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0 */ @scala.annotation.varargs - def bucketBy(numBuckets: Int, colName: String, colNames: String*): DataFrameWriter = { + def bucketBy(numBuckets: Int, colName: String, colNames: String*): DataFrameWriter[T] = { this.numBuckets = Option(numBuckets) this.bucketColumnNames = Option(colName +: colNames) this @@ -260,7 +262,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0 */ @scala.annotation.varargs - def sortBy(colName: String, colNames: String*): DataFrameWriter = { + def sortBy(colName: String, colNames: String*): DataFrameWriter[T] = { this.sortColumnNames = Option(colName +: colNames) this } @@ -301,7 +303,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 2.0.0 */ @Experimental - def queryName(queryName: String): DataFrameWriter = { + def queryName(queryName: String): DataFrameWriter[T] = { assertStreaming("queryName() can only be called on continuous queries") this.extraOptions += ("queryName" -> queryName) this @@ -337,16 +339,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { val queryName = extraOptions.getOrElse( "queryName", throw new AnalysisException("queryName must be specified for memory sink")) - val checkpointLocation = extraOptions.get("checkpointLocation").map { userSpecified => - new Path(userSpecified).toUri.toString - }.orElse { - val checkpointConfig: Option[String] = - df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION) - - checkpointConfig.map { location => - new Path(location, queryName).toUri.toString - } - }.getOrElse { + val checkpointLocation = getCheckpointLocation(queryName, failIfNotSet = false).getOrElse { Utils.createTempDir(namePrefix = "memory.stream").getCanonicalPath } @@ -378,21 +371,10 @@ final class DataFrameWriter private[sql](df: DataFrame) { className = source, options = extraOptions.toMap, partitionColumns = normalizedParCols.getOrElse(Nil)) - val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) - val checkpointLocation = extraOptions.get("checkpointLocation") - .orElse { - df.sparkSession.sessionState.conf.checkpointLocation.map { l => - new Path(l, queryName).toUri.toString - } - }.getOrElse { - throw new AnalysisException("checkpointLocation must be specified either " + - "through option() or SQLConf") - } - df.sparkSession.sessionState.continuousQueryManager.startQuery( queryName, - checkpointLocation, + getCheckpointLocation(queryName, failIfNotSet = true).get, df, dataSource.createSink(outputMode), outputMode, @@ -400,6 +382,94 @@ final class DataFrameWriter private[sql](df: DataFrame) { } } + /** + * :: Experimental :: + * Starts the execution of the streaming query, which will continually send results to the given + * [[ForeachWriter]] as as new data arrives. The [[ForeachWriter]] can be used to send the data + * generated by the [[DataFrame]]/[[Dataset]] to an external system. The returned The returned + * [[ContinuousQuery]] object can be used to interact with the stream. + * + * Scala example: + * {{{ + * datasetOfString.write.foreach(new ForeachWriter[String] { + * + * def open(partitionId: Long, version: Long): Boolean = { + * // open connection + * } + * + * def process(record: String) = { + * // write string to connection + * } + * + * def close(errorOrNull: Throwable): Unit = { + * // close the connection + * } + * }) + * }}} + * + * Java example: + * {{{ + * datasetOfString.write().foreach(new ForeachWriter() { + * + * @Override + * public boolean open(long partitionId, long version) { + * // open connection + * } + * + * @Override + * public void process(String value) { + * // write string to connection + * } + * + * @Override + * public void close(Throwable errorOrNull) { + * // close the connection + * } + * }); + * }}} + * + * @since 2.0.0 + */ + @Experimental + def foreach(writer: ForeachWriter[T]): ContinuousQuery = { + assertNotBucketed("foreach") + assertStreaming( + "foreach() can only be called on streaming Datasets/DataFrames.") + + val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) + val sink = new ForeachSink[T](ds.sparkSession.sparkContext.clean(writer))(ds.exprEnc) + df.sparkSession.sessionState.continuousQueryManager.startQuery( + queryName, + getCheckpointLocation(queryName, failIfNotSet = false).getOrElse { + Utils.createTempDir(namePrefix = "foreach.stream").getCanonicalPath + }, + df, + sink, + outputMode, + trigger) + } + + /** + * Returns the checkpointLocation for a query. If `failIfNotSet` is `true` but the checkpoint + * location is not set, [[AnalysisException]] will be thrown. If `failIfNotSet` is `false`, `None` + * will be returned if the checkpoint location is not set. + */ + private def getCheckpointLocation(queryName: String, failIfNotSet: Boolean): Option[String] = { + val checkpointLocation = extraOptions.get("checkpointLocation").map { userSpecified => + new Path(userSpecified).toUri.toString + }.orElse { + df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION).map { location => + new Path(location, queryName).toUri.toString + } + } + if (failIfNotSet && checkpointLocation.isEmpty) { + throw new AnalysisException("checkpointLocation must be specified either " + + """through option("checkpointLocation", ...) or """ + + s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""") + } + checkpointLocation + } + /** * Inserts the content of the [[DataFrame]] to the specified table. It requires that * the schema of the [[DataFrame]] is the same as the schema of the table. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 162524a9efc3a..16bbf30a94370 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2400,7 +2400,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - def write: DataFrameWriter = new DataFrameWriter(toDF()) + def write: DataFrameWriter[T] = new DataFrameWriter[T](this) /** * Returns the content of the Dataset as a Dataset of JSON strings. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala new file mode 100644 index 0000000000000..09f07426a6bfa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.streaming.ContinuousQuery + +/** + * :: Experimental :: + * A class to consume data generated by a [[ContinuousQuery]]. Typically this is used to send the + * generated data to external systems. Each partition will use a new deserialized instance, so you + * usually should do all the initialization (e.g. opening a connection or initiating a transaction) + * in the `open` method. + * + * Scala example: + * {{{ + * datasetOfString.write.foreach(new ForeachWriter[String] { + * + * def open(partitionId: Long, version: Long): Boolean = { + * // open connection + * } + * + * def process(record: String) = { + * // write string to connection + * } + * + * def close(errorOrNull: Throwable): Unit = { + * // close the connection + * } + * }) + * }}} + * + * Java example: + * {{{ + * datasetOfString.write().foreach(new ForeachWriter() { + * + * @Override + * public boolean open(long partitionId, long version) { + * // open connection + * } + * + * @Override + * public void process(String value) { + * // write string to connection + * } + * + * @Override + * public void close(Throwable errorOrNull) { + * // close the connection + * } + * }); + * }}} + * @since 2.0.0 + */ +@Experimental +abstract class ForeachWriter[T] extends Serializable { + + /** + * Called when starting to process one partition of new data in the executor. The `version` is + * for data deduplication when there are failures. When recovering from a failure, some data may + * be generated multiple times but they will always have the same version. + * + * If this method finds using the `partitionId` and `version` that this partition has already been + * processed, it can return `false` to skip the further data processing. However, `close` still + * will be called for cleaning up resources. + * + * @param partitionId the partition id. + * @param version a unique id for data deduplication. + * @return `true` if the corresponding partition and version id should be processed. `false` + * indicates the partition should be skipped. + */ + def open(partitionId: Long, version: Long): Boolean + + /** + * Called to process the data in the executor side. This method will be called only when `open` + * returns `true`. + */ + def process(value: T): Unit + + /** + * Called when stopping to process one partition of new data in the executor side. This is + * guaranteed to be called either `open` returns `true` or `false`. However, + * `close` won't be called in the following cases: + * - JVM crashes without throwing a `Throwable` + * - `open` throws a `Throwable`. + * + * @param errorOrNull the error thrown during processing data or null if there was no error. + */ + def close(errorOrNull: Throwable): Unit +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala new file mode 100644 index 0000000000000..14b9b1cb09317 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import org.apache.spark.TaskContext +import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter} + +/** + * A [[Sink]] that forwards all data into [[ForeachWriter]] according to the contract defined by + * [[ForeachWriter]]. + * + * @param writer The [[ForeachWriter]] to process all data. + * @tparam T The expected type of the sink. + */ +class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Serializable { + + override def addBatch(batchId: Long, data: DataFrame): Unit = { + data.as[T].foreachPartition { iter => + if (writer.open(TaskContext.getPartitionId(), batchId)) { + var isFailed = false + try { + while (iter.hasNext) { + writer.process(iter.next()) + } + } catch { + case e: Throwable => + isFailed = true + writer.close(e) + } + if (!isFailed) { + writer.close(null) + } + } else { + writer.close(null) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala new file mode 100644 index 0000000000000..e1fb3b947837b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.mutable + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.ForeachWriter +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.test.SharedSQLContext + +class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { + + import testImplicits._ + + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("foreach") { + withTempDir { checkpointDir => + val input = MemoryStream[Int] + val query = input.toDS().repartition(2).write + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreach(new TestForeachWriter()) + input.addData(1, 2, 3, 4) + query.processAllAvailable() + + val expectedEventsForPartition0 = Seq( + ForeachSinkSuite.Open(partition = 0, version = 0), + ForeachSinkSuite.Process(value = 1), + ForeachSinkSuite.Process(value = 3), + ForeachSinkSuite.Close(None) + ) + val expectedEventsForPartition1 = Seq( + ForeachSinkSuite.Open(partition = 1, version = 0), + ForeachSinkSuite.Process(value = 2), + ForeachSinkSuite.Process(value = 4), + ForeachSinkSuite.Close(None) + ) + + val allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert { + allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || + allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) + } + query.stop() + } + } + + test("foreach with error") { + withTempDir { checkpointDir => + val input = MemoryStream[Int] + val query = input.toDS().repartition(1).write + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreach(new TestForeachWriter() { + override def process(value: Int): Unit = { + super.process(value) + throw new RuntimeException("error") + } + }) + input.addData(1, 2, 3, 4) + query.processAllAvailable() + + val allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 1) + assert(allEvents(0)(0) === ForeachSinkSuite.Open(partition = 0, version = 0)) + assert(allEvents(0)(1) === ForeachSinkSuite.Process(value = 1)) + val errorEvent = allEvents(0)(2).asInstanceOf[ForeachSinkSuite.Close] + assert(errorEvent.error.get.isInstanceOf[RuntimeException]) + assert(errorEvent.error.get.getMessage === "error") + query.stop() + } + } +} + +/** A global object to collect events in the executor */ +object ForeachSinkSuite { + + trait Event + + case class Open(partition: Long, version: Long) extends Event + + case class Process[T](value: T) extends Event + + case class Close(error: Option[Throwable]) extends Event + + private val _allEvents = new ConcurrentLinkedQueue[Seq[Event]]() + + def addEvents(events: Seq[Event]): Unit = { + _allEvents.add(events) + } + + def allEvents(): Seq[Seq[Event]] = { + _allEvents.toArray(new Array[Seq[Event]](_allEvents.size())) + } + + def clear(): Unit = { + _allEvents.clear() + } +} + +/** A [[ForeachWriter]] that writes collected events to ForeachSinkSuite */ +class TestForeachWriter extends ForeachWriter[Int] { + ForeachSinkSuite.clear() + + private val events = mutable.ArrayBuffer[ForeachSinkSuite.Event]() + + override def open(partitionId: Long, version: Long): Boolean = { + events += ForeachSinkSuite.Open(partition = partitionId, version = version) + true + } + + override def process(value: Int): Unit = { + events += ForeachSinkSuite.Process(value) + } + + override def close(errorOrNull: Throwable): Unit = { + events += ForeachSinkSuite.Close(error = Option(errorOrNull)) + ForeachSinkSuite.addEvents(events) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index bab0092c37d34..fc01ff3f5aa07 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -238,7 +238,9 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet shuffleLeft: Boolean, shuffleRight: Boolean): Unit = { withTable("bucketed_table1", "bucketed_table2") { - def withBucket(writer: DataFrameWriter, bucketSpec: Option[BucketSpec]): DataFrameWriter = { + def withBucket( + writer: DataFrameWriter[Row], + bucketSpec: Option[BucketSpec]): DataFrameWriter[Row] = { bucketSpec.map { spec => writer.bucketBy( spec.numBuckets, From 84a8421e5cd5756cffb3d796117149c413204264 Mon Sep 17 00:00:00 2001 From: Mortada Mehyar Date: Fri, 10 Jun 2016 00:23:34 -0700 Subject: [PATCH 0574/1470] [DOCUMENTATION] fixed groupby aggregation example for pyspark ## What changes were proposed in this pull request? fixing documentation for the groupby/agg example in python ## How was this patch tested? the existing example in the documentation dose not contain valid syntax (missing parenthesis) and is not using `Column` in the expression for `agg()` after the fix here's how I tested it: ``` In [1]: from pyspark.sql import Row In [2]: import pyspark.sql.functions as func In [3]: %cpaste Pasting code; enter '--' alone on the line to stop or use Ctrl-D. :records = [{'age': 19, 'department': 1, 'expense': 100}, : {'age': 20, 'department': 1, 'expense': 200}, : {'age': 21, 'department': 2, 'expense': 300}, : {'age': 22, 'department': 2, 'expense': 300}, : {'age': 23, 'department': 3, 'expense': 300}] :-- In [4]: df = sqlContext.createDataFrame([Row(**d) for d in records]) In [5]: df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")).show() +----------+----------+--------+------------+ |department|department|max(age)|sum(expense)| +----------+----------+--------+------------+ | 1| 1| 20| 300| | 2| 2| 22| 600| | 3| 3| 23| 300| +----------+----------+--------+------------+ Author: Mortada Mehyar Closes #13587 from mortada/groupby_agg_doc_fix. (cherry picked from commit 675a73715d3c8adb9d9a9dce5f76a2db5106790c) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 940c1d77045ad..efdf873c34556 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -2221,7 +2221,7 @@ import pyspark.sql.functions as func # In 1.3.x, in order for the grouping column "department" to show up, # it must be included explicitly as part of the agg function call. -df.groupBy("department").agg("department"), func.max("age"), func.sum("expense")) +df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) # In 1.4+, grouping column "department" is included automatically. df.groupBy("department").agg(func.max("age"), func.sum("expense")) From 6709ce1aea4a8d7438722f48fd7f2ed0fc7fa5be Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 10 Jun 2016 10:09:19 +0100 Subject: [PATCH 0575/1470] [SPARK-15823][PYSPARK][ML] Add @property for 'accuracy' in MulticlassMetrics ## What changes were proposed in this pull request? `accuracy` should be decorated with `property` to keep step with other methods in `pyspark.MulticlassMetrics`, like `weightedPrecision`, `weightedRecall`, etc ## How was this patch tested? manual tests Author: Zheng RuiFeng Closes #13560 from zhengruifeng/add_accuracy_property. (cherry picked from commit 16ca32eace39c423224b0ec25922038fd45c501a) Signed-off-by: Sean Owen --- python/pyspark/mllib/evaluation.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 2eaac87b6774b..fc2a0b3b5038a 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -179,11 +179,7 @@ class MulticlassMetrics(JavaModelWrapper): 1.0... >>> metrics.fMeasure(0.0, 2.0) 0.52... - >>> metrics.precision() - 0.66... - >>> metrics.recall() - 0.66... - >>> metrics.accuracy() + >>> metrics.accuracy 0.66... >>> metrics.weightedFalsePositiveRate 0.19... @@ -273,6 +269,7 @@ def fMeasure(self, label=None, beta=None): else: return self.call("fMeasure", label, beta) + @property @since('2.0.0') def accuracy(self): """ From 54b4763d295d6aeab6105d0430470343dd4ca3a3 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Fri, 10 Jun 2016 12:26:53 +0100 Subject: [PATCH 0576/1470] [SPARK-15837][ML][PYSPARK] Word2vec python add maxsentence parameter ## What changes were proposed in this pull request? Word2vec python add maxsentence parameter. ## How was this patch tested? Existing test. Author: WeichenXu Closes #13578 from WeichenXu123/word2vec_python_add_maxsentence. (cherry picked from commit cdd7f5a57a21d4a8f93456d149f65859c96190cf) Signed-off-by: Sean Owen --- python/pyspark/ml/feature.py | 29 ++++++++++++++++++++++++----- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index ebe13006ad25b..bfb2fb707137d 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2244,28 +2244,33 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, Has windowSize = Param(Params._dummy(), "windowSize", "the window size (context words from [-window, window]). Default value is 5", typeConverter=TypeConverters.toInt) + maxSentenceLength = Param(Params._dummy(), "maxSentenceLength", + "Maximum length (in words) of each sentence in the input data. " + + "Any sentence longer than this threshold will " + + "be divided into chunks up to the size.", + typeConverter=TypeConverters.toInt) @keyword_only def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=None, inputCol=None, outputCol=None, windowSize=5): + seed=None, inputCol=None, outputCol=None, windowSize=5, maxSentenceLength=1000): """ __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, \ - seed=None, inputCol=None, outputCol=None, windowSize=5) + seed=None, inputCol=None, outputCol=None, windowSize=5, maxSentenceLength=1000) """ super(Word2Vec, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid) self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=None, windowSize=5) + seed=None, windowSize=5, maxSentenceLength=1000) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("1.4.0") def setParams(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=None, inputCol=None, outputCol=None, windowSize=5): + seed=None, inputCol=None, outputCol=None, windowSize=5, maxSentenceLength=1000): """ setParams(self, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, seed=None, \ - inputCol=None, outputCol=None, windowSize=5) + inputCol=None, outputCol=None, windowSize=5, maxSentenceLength=1000) Sets params for this Word2Vec. """ kwargs = self.setParams._input_kwargs @@ -2327,6 +2332,20 @@ def getWindowSize(self): """ return self.getOrDefault(self.windowSize) + @since("2.0.0") + def setMaxSentenceLength(self, value): + """ + Sets the value of :py:attr:`maxSentenceLength`. + """ + return self._set(maxSentenceLength=value) + + @since("2.0.0") + def getMaxSentenceLength(self): + """ + Gets the value of maxSentenceLength or its default value. + """ + return self.getOrDefault(self.maxSentenceLength) + def _create_model(self, java_model): return Word2VecModel(java_model) From 47c2a265fbdb91cf5684f0d6342869ca08cb2d27 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 10 Jun 2016 10:48:28 -0700 Subject: [PATCH 0577/1470] [SPARK-15812][SQ][STREAMING] Added support for sorting after streaming aggregation with complete mode ## What changes were proposed in this pull request? When the output mode is complete, then the output of a streaming aggregation essentially will contain the complete aggregates every time. So this is not different from a batch dataset within an incremental execution. Other non-streaming operations should be supported on this dataset. In this PR, I am just adding support for sorting, as it is a common useful functionality. Support for other operations will come later. ## How was this patch tested? Additional unit tests. Author: Tathagata Das Closes #13549 from tdas/SPARK-15812. (cherry picked from commit abdb5d42c5802c8f60876aa1285c803d02881258) Signed-off-by: Tathagata Das --- .../UnsupportedOperationChecker.scala | 61 ++++++++++++------- .../analysis/UnsupportedOperationsSuite.scala | 17 +++++- .../spark/sql/streaming/StreamTest.scala | 24 +++++--- .../streaming/StreamingAggregationSuite.scala | 25 ++++++++ 4 files changed, 95 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 8373fa336dd4c..689e016a5a1d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -43,6 +43,41 @@ object UnsupportedOperationChecker { "Queries without streaming sources cannot be executed with write.startStream()")(plan) } + // Disallow multiple streaming aggregations + val aggregates = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a } + + if (aggregates.size > 1) { + throwError( + "Multiple streaming aggregations are not supported with " + + "streaming DataFrames/Datasets")(plan) + } + + // Disallow some output mode + outputMode match { + case InternalOutputModes.Append if aggregates.nonEmpty => + throwError( + s"$outputMode output mode not supported when there are streaming aggregations on " + + s"streaming DataFrames/DataSets")(plan) + + case InternalOutputModes.Complete | InternalOutputModes.Update if aggregates.isEmpty => + throwError( + s"$outputMode output mode not supported when there are no streaming aggregations on " + + s"streaming DataFrames/Datasets")(plan) + + case _ => + } + + /** + * Whether the subplan will contain complete data or incremental data in every incremental + * execution. Some operations may be allowed only when the child logical plan gives complete + * data. + */ + def containsCompleteData(subplan: LogicalPlan): Boolean = { + val aggs = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a } + // Either the subplan has no streaming source, or it has aggregation with Complete mode + !subplan.isStreaming || (aggs.nonEmpty && outputMode == InternalOutputModes.Complete) + } + plan.foreachUp { implicit subPlan => // Operations that cannot exists anywhere in a streaming plan @@ -107,8 +142,9 @@ object UnsupportedOperationChecker { case GlobalLimit(_, _) | LocalLimit(_, _) if subPlan.children.forall(_.isStreaming) => throwError("Limits are not supported on streaming DataFrames/Datasets") - case Sort(_, _, _) | SortPartitions(_, _) if subPlan.children.forall(_.isStreaming) => - throwError("Sorting is not supported on streaming DataFrames/Datasets") + case Sort(_, _, _) | SortPartitions(_, _) if !containsCompleteData(subPlan) => + throwError("Sorting is not supported on streaming DataFrames/Datasets, unless it is on" + + "aggregated DataFrame/Dataset in Complete mode") case Sample(_, _, _, _, child) if child.isStreaming => throwError("Sampling is not supported on streaming DataFrames/Datasets") @@ -123,27 +159,6 @@ object UnsupportedOperationChecker { case _ => } } - - // Checks related to aggregations - val aggregates = plan.collect { case a @ Aggregate(_, _, _) if a.isStreaming => a } - outputMode match { - case InternalOutputModes.Append if aggregates.nonEmpty => - throwError( - s"$outputMode output mode not supported when there are streaming aggregations on " + - s"streaming DataFrames/DataSets")(plan) - - case InternalOutputModes.Complete | InternalOutputModes.Update if aggregates.isEmpty => - throwError( - s"$outputMode output mode not supported when there are no streaming aggregations on " + - s"streaming DataFrames/Datasets")(plan) - - case _ => - } - if (aggregates.size > 1) { - throwError( - "Multiple streaming aggregations are not supported with " + - "streaming DataFrames/Datasets")(plan) - } } private def throwErrorIf( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 378cca3644eab..c21ad5e03a48d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -81,7 +81,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite { outputMode = Append, expectedMsgs = "commands" :: Nil) - // Multiple streaming aggregations not supported + // Aggregation: Multiple streaming aggregations not supported def aggExprs(name: String): Seq[NamedExpression] = Seq(Count("*").as(name)) assertSupportedInStreamingPlan( @@ -189,8 +189,20 @@ class UnsupportedOperationsSuite extends SparkFunSuite { _.intersect(_), streamStreamSupported = false) - // Unary operations + // Sort: supported only on batch subplans and on aggregation + complete output mode testUnaryOperatorInStreamingPlan("sort", Sort(Nil, true, _)) + assertSupportedInStreamingPlan( + "sort - sort over aggregated data in Complete output mode", + streamRelation.groupBy()(Count("*")).sortBy(), + Complete) + assertNotSupportedInStreamingPlan( + "sort - sort over aggregated data in Update output mode", + streamRelation.groupBy()(Count("*")).sortBy(), + Update, + Seq("sort", "aggregat", "complete")) // sort on aggregations is supported on Complete mode only + + + // Other unary operations testUnaryOperatorInStreamingPlan("sort partitions", SortPartitions(Nil, _), expectedMsg = "sort") testUnaryOperatorInStreamingPlan( "sample", Sample(0.1, 1, true, 1L, _)(), expectedMsg = "sampling") @@ -299,6 +311,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite { outputMode) } + /** Test output mode with and without aggregation in the streaming plan */ def testOutputMode( outputMode: OutputMode, shouldSupportAggregation: Boolean): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 194c3e7307255..7f1e5fe6135a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -111,10 +111,13 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { def apply[A : Encoder](data: A*): CheckAnswerRows = { val encoder = encoderFor[A] val toExternalRow = RowEncoder(encoder.schema).resolveAndBind() - CheckAnswerRows(data.map(d => toExternalRow.fromRow(encoder.toRow(d))), false) + CheckAnswerRows( + data.map(d => toExternalRow.fromRow(encoder.toRow(d))), + lastOnly = false, + isSorted = false) } - def apply(rows: Row*): CheckAnswerRows = CheckAnswerRows(rows, false) + def apply(rows: Row*): CheckAnswerRows = CheckAnswerRows(rows, false, false) } /** @@ -123,15 +126,22 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { */ object CheckLastBatch { def apply[A : Encoder](data: A*): CheckAnswerRows = { + apply(isSorted = false, data: _*) + } + + def apply[A: Encoder](isSorted: Boolean, data: A*): CheckAnswerRows = { val encoder = encoderFor[A] val toExternalRow = RowEncoder(encoder.schema).resolveAndBind() - CheckAnswerRows(data.map(d => toExternalRow.fromRow(encoder.toRow(d))), true) + CheckAnswerRows( + data.map(d => toExternalRow.fromRow(encoder.toRow(d))), + lastOnly = true, + isSorted = isSorted) } - def apply(rows: Row*): CheckAnswerRows = CheckAnswerRows(rows, true) + def apply(rows: Row*): CheckAnswerRows = CheckAnswerRows(rows, true, false) } - case class CheckAnswerRows(expectedAnswer: Seq[Row], lastOnly: Boolean) + case class CheckAnswerRows(expectedAnswer: Seq[Row], lastOnly: Boolean, isSorted: Boolean) extends StreamAction with StreamMustBeRunning { override def toString: String = s"$operatorName: ${expectedAnswer.mkString(",")}" private def operatorName = if (lastOnly) "CheckLastBatch" else "CheckAnswer" @@ -414,7 +424,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { failTest("Error adding data", e) } - case CheckAnswerRows(expectedAnswer, lastOnly) => + case CheckAnswerRows(expectedAnswer, lastOnly, isSorted) => verify(currentStream != null, "stream not running") // Get the map of source index to the current source objects val indexToSource = currentStream @@ -436,7 +446,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { failTest("Exception while getting data from sink", e) } - QueryTest.sameRows(expectedAnswer, sparkAnswer).foreach { + QueryTest.sameRows(expectedAnswer, sparkAnswer, isSorted).foreach { error => failTest(error) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 1f174aee8ce08..8681199817fe6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -104,6 +104,31 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll { } } + test("sort after aggregate in complete mode") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .toDF("value", "count") + .orderBy($"count".desc) + .as[(Int, Long)] + + testStream(aggregated, Complete)( + AddData(inputData, 3), + CheckLastBatch(isSorted = true, (3, 1)), + AddData(inputData, 2, 3), + CheckLastBatch(isSorted = true, (3, 2), (2, 1)), + StopStream, + StartStream(), + AddData(inputData, 3, 2, 1), + CheckLastBatch(isSorted = true, (3, 3), (2, 2), (1, 1)), + AddData(inputData, 4, 4, 4, 4), + CheckLastBatch(isSorted = true, (4, 4), (3, 3), (2, 2), (1, 1)) + ) + } + test("multiple keys") { val inputData = MemoryStream[Int] From 55a83724632aa54e49aedbab8ddd21d010eca26d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 10 Jun 2016 11:05:04 -0700 Subject: [PATCH 0578/1470] [SPARK-15753][SQL] Move Analyzer stuff to Analyzer from DataFrameWriter ## What changes were proposed in this pull request? This patch moves some codes in `DataFrameWriter.insertInto` that belongs to `Analyzer`. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13496 from viirya/move-analyzer-stuff. (cherry picked from commit 0ec279ffdf92853965e327a9f0f6956cacb7a23e) Signed-off-by: Cheng Lian --- .../spark/sql/catalyst/analysis/Analyzer.scala | 17 ++++++++++++++--- .../org/apache/spark/sql/DataFrameWriter.scala | 12 +----------- .../sql/hive/execution/HiveQuerySuite.scala | 4 ++-- 3 files changed, 17 insertions(+), 16 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 44461400aa9f4..a0813572a78f4 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 @@ -452,6 +452,17 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) if child.resolved => + // A partitioned relation's schema can be different from the input logicalPlan, since + // partition columns are all moved after data columns. We Project to adjust the ordering. + val input = if (parts.nonEmpty) { + val (inputPartCols, inputDataCols) = child.output.partition { attr => + parts.contains(attr.name) + } + Project(inputDataCols ++ inputPartCols, child) + } else { + child + } + val table = lookupTableFromCatalog(u) // adding the table's partitions or validate the query's partition info table match { @@ -467,8 +478,8 @@ class Analyzer( |Requested partitions: ${parts.keys.mkString(",")} |Table partitions: ${tablePartitionNames.mkString(",")}""".stripMargin) } - // Assume partition columns are correctly placed at the end of the child's output - i.copy(table = EliminateSubqueryAliases(table)) + // Partition columns are already correctly placed at the end of the child's output + i.copy(table = EliminateSubqueryAliases(table), child = input) } else { // Set up the table's partition scheme with all dynamic partitions by moving partition // columns to the end of the column list, in partition order. @@ -486,7 +497,7 @@ class Analyzer( child = Project(columns ++ partColumns, child)) } case _ => - i.copy(table = EliminateSubqueryAliases(table)) + i.copy(table = EliminateSubqueryAliases(table), child = input) } case u: UnresolvedRelation => val table = u.tableIdentifier diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 32e2fdc3f9707..6ce59e885acc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -505,21 +505,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val partitions = normalizedParCols.map(_.map(col => col -> (None: Option[String])).toMap) val overwrite = mode == SaveMode.Overwrite - // A partitioned relation's schema can be different from the input logicalPlan, since - // partition columns are all moved after data columns. We Project to adjust the ordering. - // TODO: this belongs to the analyzer. - val input = normalizedParCols.map { parCols => - val (inputPartCols, inputDataCols) = df.logicalPlan.output.partition { attr => - parCols.contains(attr.name) - } - Project(inputDataCols ++ inputPartCols, df.logicalPlan) - }.getOrElse(df.logicalPlan) - df.sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdent), partitions.getOrElse(Map.empty[String, Option[String]]), - input, + df.logicalPlan, overwrite, ifNotExists = false)).toRdd } 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 e0f6ccf04dd33..0a2bab4f5d1e1 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 @@ -1042,7 +1042,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .queryExecution.analyzed } - assertResult(1, "Duplicated project detected\n" + analyzedPlan) { + assertResult(2, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { case _: Project => () }.size @@ -1061,7 +1061,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .queryExecution.analyzed } - assertResult(1, "Duplicated project detected\n" + analyzedPlan) { + assertResult(2, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { case _: Project => () }.size From 935b6e0e48e258f447622033b512f7ba5d83da69 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 10 Jun 2016 11:08:39 -0700 Subject: [PATCH 0579/1470] [SPARK-15866] Rename listAccumulator collectionAccumulator ## What changes were proposed in this pull request? SparkContext.listAccumulator, by Spark's convention, makes it sound like "list" is a verb and the method should return a list of accumulators. This patch renames the method and the class collection accumulator. ## How was this patch tested? Updated test case to reflect the names. Author: Reynold Xin Closes #13594 from rxin/SPARK-15866. (cherry picked from commit 254bc8c34e70241508bdfc8ff42a65491f5280cd) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/SparkContext.scala | 16 ++++++++-------- .../org/apache/spark/util/AccumulatorV2.scala | 15 ++++++++++----- .../apache/spark/util/AccumulatorV2Suite.scala | 2 +- .../columnar/InMemoryTableScanExec.scala | 8 ++++---- 4 files changed, 23 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 33b11ed2e658f..230fabd211701 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1340,21 +1340,21 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** - * Create and register a list accumulator, which starts with empty list and accumulates inputs - * by adding them into the inner list. + * Create and register a [[CollectionAccumulator]], which starts with empty list and accumulates + * inputs by adding them into the list. */ - def listAccumulator[T]: ListAccumulator[T] = { - val acc = new ListAccumulator[T] + def collectionAccumulator[T]: CollectionAccumulator[T] = { + val acc = new CollectionAccumulator[T] register(acc) acc } /** - * Create and register a list accumulator, which starts with empty list and accumulates inputs - * by adding them into the inner list. + * Create and register a [[CollectionAccumulator]], which starts with empty list and accumulates + * inputs by adding them into the list. */ - def listAccumulator[T](name: String): ListAccumulator[T] = { - val acc = new ListAccumulator[T] + def collectionAccumulator[T](name: String): CollectionAccumulator[T] = { + val acc = new CollectionAccumulator[T] register(acc, name) acc } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 0b9a47c1124b1..044dd69cc92c7 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -415,15 +415,20 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { } -class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { +/** + * An [[AccumulatorV2 accumulator]] for collecting a list of elements. + * + * @since 2.0.0 + */ +class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { private val _list: java.util.List[T] = new ArrayList[T] override def isZero: Boolean = _list.isEmpty - override def copyAndReset(): ListAccumulator[T] = new ListAccumulator + override def copyAndReset(): CollectionAccumulator[T] = new CollectionAccumulator - override def copy(): ListAccumulator[T] = { - val newAcc = new ListAccumulator[T] + override def copy(): CollectionAccumulator[T] = { + val newAcc = new CollectionAccumulator[T] newAcc._list.addAll(_list) newAcc } @@ -433,7 +438,7 @@ class ListAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { override def add(v: T): Unit = _list.add(v) override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match { - case o: ListAccumulator[T] => _list.addAll(o.value) + case o: CollectionAccumulator[T] => _list.addAll(o.value) case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala index 439da1306f5aa..a04644d57ed88 100644 --- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -88,7 +88,7 @@ class AccumulatorV2Suite extends SparkFunSuite { } test("ListAccumulator") { - val acc = new ListAccumulator[Double] + val acc = new CollectionAccumulator[Double] assert(acc.value.isEmpty) assert(acc.isZero) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index a1c2f0a8fbcf4..ff0733102a298 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{AccumulatorContext, ListAccumulator, LongAccumulator} +import org.apache.spark.util.{AccumulatorContext, CollectionAccumulator, LongAccumulator} private[sql] object InMemoryRelation { @@ -67,16 +67,16 @@ private[sql] case class InMemoryRelation( tableName: Option[String])( @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, @transient private[sql] var _statistics: Statistics = null, - private[sql] var _batchStats: ListAccumulator[InternalRow] = null) + private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) extends logical.LeafNode with MultiInstanceRelation { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) override def producedAttributes: AttributeSet = outputSet - private[sql] val batchStats: ListAccumulator[InternalRow] = + private[sql] val batchStats: CollectionAccumulator[InternalRow] = if (_batchStats == null) { - child.sqlContext.sparkContext.listAccumulator[InternalRow] + child.sqlContext.sparkContext.collectionAccumulator[InternalRow] } else { _batchStats } From 8dd82f8de40a9ef54ef147f1acfb54a40d270c67 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 10 Jun 2016 12:43:27 -0700 Subject: [PATCH 0580/1470] [SPARK-15743][SQL] Prevent saving with all-column partitioning ## What changes were proposed in this pull request? When saving datasets on storage, `partitionBy` provides an easy way to construct the directory structure. However, if a user choose all columns as partition columns, some exceptions occurs. - **ORC with all column partitioning**: `AnalysisException` on **future read** due to schema inference failure. ```scala scala> spark.range(10).write.format("orc").mode("overwrite").partitionBy("id").save("/tmp/data") scala> spark.read.format("orc").load("/tmp/data").collect() org.apache.spark.sql.AnalysisException: Unable to infer schema for ORC at /tmp/data. It must be specified manually; ``` - **Parquet with all-column partitioning**: `InvalidSchemaException` on **write execution** due to Parquet limitation. ```scala scala> spark.range(100).write.format("parquet").mode("overwrite").partitionBy("id").save("/tmp/data") [Stage 0:> (0 + 8) / 8]16/06/02 16:51:17 ERROR Utils: Aborting task org.apache.parquet.schema.InvalidSchemaException: A group type can not be empty. Parquet does not support empty group without leaves. Empty group: spark_schema ... (lots of error messages) ``` Although some formats like JSON support all-column partitioning without any problem, it seems not a good idea to make lots of empty directories. This PR prevents saving with all-column partitioning by consistently raising `AnalysisException` before executing save operation. ## How was this patch tested? Newly added `PartitioningUtilsSuite`. Author: Dongjoon Hyun Closes #13486 from dongjoon-hyun/SPARK-15743. (cherry picked from commit 2413fce9d6812a91eeffb4435c2b5b361d23214b) Signed-off-by: Michael Armbrust --- .../execution/datasources/DataSource.scala | 32 +++++++++---------- .../datasources/PartitioningUtils.scala | 8 +++-- .../sql/execution/datasources/rules.scala | 4 +-- .../execution/streaming/FileStreamSink.scala | 2 +- .../test/DataFrameReaderWriterSuite.scala | 12 +++++++ 5 files changed, 37 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 5f17fdf9467db..d3273025b6885 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.execution.datasources @@ -432,7 +432,7 @@ case class DataSource( } val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - PartitioningUtils.validatePartitionColumnDataTypes( + PartitioningUtils.validatePartitionColumn( data.schema, partitionColumns, caseSensitive) // If we are appending to a table that already exists, make sure the partitioning matches diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 74f2993754f8f..2340ff0afed74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -339,7 +339,7 @@ private[sql] object PartitioningUtils { private val upCastingOrder: Seq[DataType] = Seq(NullType, IntegerType, LongType, FloatType, DoubleType, StringType) - def validatePartitionColumnDataTypes( + def validatePartitionColumn( schema: StructType, partitionColumns: Seq[String], caseSensitive: Boolean): Unit = { @@ -350,6 +350,10 @@ private[sql] object PartitioningUtils { case _ => throw new AnalysisException(s"Cannot use ${field.dataType} for partition column") } } + + if (partitionColumns.size == schema.fields.size) { + throw new AnalysisException(s"Cannot use all columns for partition columns") + } } def partitionColumnsSchema( @@ -359,7 +363,7 @@ private[sql] object PartitioningUtils { val equality = columnNameEquality(caseSensitive) StructType(partitionColumns.map { col => schema.find(f => equality(f.name, col)).getOrElse { - throw new RuntimeException(s"Partition column $col not found in schema $schema") + throw new AnalysisException(s"Partition column $col not found in schema $schema") } }).asNullable } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 9afd715016d88..7ac62fb191d40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -154,7 +154,7 @@ private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) // OK } - PartitioningUtils.validatePartitionColumnDataTypes( + PartitioningUtils.validatePartitionColumn( r.schema, part.keySet.toSeq, conf.caseSensitiveAnalysis) // Get all input data source relations of the query. @@ -205,7 +205,7 @@ private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) // OK } - PartitioningUtils.validatePartitionColumnDataTypes( + PartitioningUtils.validatePartitionColumn( c.child.schema, c.partitionColumns, conf.caseSensitiveAnalysis) for { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index e19101032967b..efb04912d76bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -91,7 +91,7 @@ class FileStreamSinkWriter( hadoopConf: Configuration, options: Map[String, String]) extends Serializable with Logging { - PartitioningUtils.validatePartitionColumnDataTypes( + PartitioningUtils.validatePartitionColumn( data.schema, partitionColumnNames, data.sqlContext.conf.caseSensitiveAnalysis) private val serializableConf = new SerializableConfiguration(hadoopConf) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index 431a943304f5b..bf6063a4c457c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -572,4 +572,16 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { cq.awaitTermination(2000L) } + + test("prevent all column partitioning") { + withTempDir { dir => + val path = dir.getCanonicalPath + intercept[AnalysisException] { + spark.range(10).write.format("parquet").mode("overwrite").partitionBy("id").save(path) + } + intercept[AnalysisException] { + spark.range(10).write.format("orc").mode("overwrite").partitionBy("id").save(path) + } + } + } } From f895d6d859bc3b259abe8bc39cf8367e3e72a243 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 10 Jun 2016 12:46:22 -0700 Subject: [PATCH 0581/1470] [SPARK-15766][SPARKR] R should export is.nan ## What changes were proposed in this pull request? When reviewing SPARK-15545, we found that is.nan is not exported, which should be exported. Add it to the NAMESPACE. ## How was this patch tested? Manual tests. Author: wm624@hotmail.com Closes #13508 from wangmiao1981/unused. (cherry picked from commit 2c8f40cea113b597fbaf1cdd80a5b8bdd66155fb) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 2 ++ 1 file changed, 2 insertions(+) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 239ad065d09ad..ba386da2aa178 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -185,6 +185,8 @@ exportMethods("%in%", "isNaN", "isNotNull", "isNull", + "is.nan", + "isnan", "kurtosis", "lag", "last", From c1390ccbb2968156245e267e6c5cd2a27f7d0121 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 10 Jun 2016 12:50:27 -0700 Subject: [PATCH 0582/1470] Revert [SPARK-14485][CORE] ignore task finished for executor lost This reverts commit 695dbc816a6d70289abeb145cb62ff4e62b3f49b. This change is being reverted because it hurts performance of some jobs, and only helps in a narrow set of cases. For more discussion, refer to the JIRA. Author: Kay Ousterhout Closes #13580 from kayousterhout/revert-SPARK-14485. (cherry picked from commit 5c16ad0d522e5124a6977533077afb7b38fc42a1) Signed-off-by: Kay Ousterhout --- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 14 +------------- 1 file changed, 1 insertion(+), 13 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 d6f58e4789cff..01e85ca405587 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -346,11 +346,9 @@ private[spark] class TaskSchedulerImpl( } taskIdToTaskSetManager.get(tid) match { case Some(taskSet) => - var executorId: String = null if (TaskState.isFinished(state)) { taskIdToTaskSetManager.remove(tid) taskIdToExecutorId.remove(tid).foreach { execId => - executorId = execId if (executorIdToTaskCount.contains(execId)) { executorIdToTaskCount(execId) -= 1 } @@ -358,17 +356,7 @@ private[spark] class TaskSchedulerImpl( } if (state == TaskState.FINISHED) { taskSet.removeRunningTask(tid) - // In some case, executor has already been removed by driver for heartbeats timeout, - // but at sometime, before executor killed by cluster, the task of running on this - // executor is finished and return task success state to driver. However, this kinds - // of task should be ignored, because the task on this executor is already re-queued - // by driver. For more details, can check in SPARK-14485. - if (executorId != null && !executorIdToTaskCount.contains(executorId)) { - logInfo(s"Ignoring update with state $state for TID $tid because its executor " + - s"has already been removed by driver") - } else { - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) - } + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { taskSet.removeRunningTask(tid) taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) From f15d641e297d425a8c1b4ba6c93f4f98a3f70d0f Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Fri, 10 Jun 2016 13:01:29 -0700 Subject: [PATCH 0583/1470] [SPARK-15871][SQL] Add `assertNotPartitioned` check in `DataFrameWriter` ## What changes were proposed in this pull request? It doesn't make sense to specify partitioning parameters, when we write data out from Datasets/DataFrames into `jdbc` tables or streaming `ForeachWriter`s. This patch adds `assertNotPartitioned` check in `DataFrameWriter`.
    operation should check not partitioned?
    mode
    outputMode
    trigger
    format
    option/options
    partitionBy
    bucketBy
    sortBy
    save
    queryName
    startStream
    foreach yes
    insertInto
    saveAsTable
    jdbc yes
    json
    parquet
    orc
    text
    csv
    ## How was this patch tested? New dedicated tests. Author: Liwei Lin Closes #13597 from lw-lin/add-assertNotPartitioned. (cherry picked from commit fb219029dd1b8d2783c3e202361401048296595c) Signed-off-by: Shixiong Zhu --- .../apache/spark/sql/DataFrameWriter.scala | 12 +++++- .../test/DataFrameReaderWriterSuite.scala | 42 +++++++++++++++++-- .../sql/sources/BucketedWriteSuite.scala | 8 ++-- 3 files changed, 52 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 6ce59e885acc7..78b74f948e682 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -432,6 +432,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ @Experimental def foreach(writer: ForeachWriter[T]): ContinuousQuery = { + assertNotPartitioned("foreach") assertNotBucketed("foreach") assertStreaming( "foreach() can only be called on streaming Datasets/DataFrames.") @@ -562,8 +563,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def assertNotBucketed(operation: String): Unit = { if (numBuckets.isDefined || sortColumnNames.isDefined) { - throw new IllegalArgumentException( - s"'$operation' does not support bucketing right now.") + throw new AnalysisException(s"'$operation' does not support bucketing right now") + } + } + + private def assertNotPartitioned(operation: String): Unit = { + if (partitioningColumns.isDefined) { + throw new AnalysisException( s"'$operation' does not support partitioning") } } @@ -646,6 +652,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { + assertNotPartitioned("jdbc") + assertNotBucketed("jdbc") assertNotStreaming("jdbc() can only be called on non-continuous queries") val props = new Properties() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index bf6063a4c457c..6e0d66ae7f19a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -455,8 +455,8 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write - val e = intercept[IllegalArgumentException](w.bucketBy(1, "text").startStream()) - assert(e.getMessage == "'startStream' does not support bucketing right now.") + val e = intercept[AnalysisException](w.bucketBy(1, "text").startStream()) + assert(e.getMessage == "'startStream' does not support bucketing right now;") } test("check sortBy() can only be called on non-continuous queries;") { @@ -464,8 +464,8 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write - val e = intercept[IllegalArgumentException](w.sortBy("text").startStream()) - assert(e.getMessage == "'startStream' does not support bucketing right now.") + val e = intercept[AnalysisException](w.sortBy("text").startStream()) + assert(e.getMessage == "'startStream' does not support bucketing right now;") } test("check save(path) can only be called on non-continuous queries") { @@ -558,6 +558,40 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { assert(e.getMessage == "csv() can only be called on non-continuous queries;") } + test("check foreach() does not support partitioning or bucketing") { + val df = spark.read + .format("org.apache.spark.sql.streaming.test") + .stream() + + var w = df.write.partitionBy("value") + var e = intercept[AnalysisException](w.foreach(null)) + Seq("foreach", "partitioning").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + + w = df.write.bucketBy(2, "value") + e = intercept[AnalysisException](w.foreach(null)) + Seq("foreach", "bucketing").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + + test("check jdbc() does not support partitioning or bucketing") { + val df = spark.read.text(newTextInput) + + var w = df.write.partitionBy("value") + var e = intercept[AnalysisException](w.jdbc(null, null, null)) + Seq("jdbc", "partitioning").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + + w = df.write.bucketBy(2, "value") + e = intercept[AnalysisException](w.jdbc(null, null, null)) + Seq("jdbc", "bucketing").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + test("ConsoleSink can be correctly loaded") { LastOptions.clear() val df = spark.read diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 61a281db85238..997445114ba58 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -62,19 +62,19 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle test("write bucketed data using save()") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") - val e = intercept[IllegalArgumentException] { + val e = intercept[AnalysisException] { df.write.bucketBy(2, "i").parquet("/tmp/path") } - assert(e.getMessage == "'save' does not support bucketing right now.") + assert(e.getMessage == "'save' does not support bucketing right now;") } test("write bucketed data using insertInto()") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") - val e = intercept[IllegalArgumentException] { + val e = intercept[AnalysisException] { df.write.bucketBy(2, "i").insertInto("tt") } - assert(e.getMessage == "'insertInto' does not support bucketing right now.") + assert(e.getMessage == "'insertInto' does not support bucketing right now;") } private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") From 96bb1476ce884168f232d3e63aa21b5f7dba474f Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Fri, 10 Jun 2016 13:06:51 -0700 Subject: [PATCH 0584/1470] [MINOR][X][X] Replace all occurrences of None: Option with Option.empty ## What changes were proposed in this pull request? Replace all occurrences of `None: Option[X]` with `Option.empty[X]` ## How was this patch tested? Exisiting Tests Author: Sandeep Singh Closes #13591 from techaddict/minor-7. (cherry picked from commit 865ec32dd997e63aea01a871d1c7b4947f43c111) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 4 ++-- .../scala/org/apache/spark/sql/DataFrameWriter.scala | 2 +- .../sql/execution/command/createDataSourceTables.scala | 2 +- .../spark/sql/execution/exchange/ShuffleExchange.scala | 2 +- .../org/apache/spark/sql/hive/orc/OrcQuerySuite.scala | 10 +++++----- .../streaming/receiver/ReceivedBlockHandler.scala | 2 +- 6 files changed, 11 insertions(+), 11 deletions(-) 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 f924efe6e6f44..3cc7a1a3cae83 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 @@ -105,7 +105,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { */ def find(f: BaseType => Boolean): Option[BaseType] = f(this) match { case true => Some(this) - case false => children.foldLeft(None: Option[BaseType]) { (l, r) => l.orElse(r.find(f)) } + case false => children.foldLeft(Option.empty[BaseType]) { (l, r) => l.orElse(r.find(f)) } } /** @@ -165,7 +165,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def collectFirst[B](pf: PartialFunction[BaseType, B]): Option[B] = { val lifted = pf.lift lifted(this).orElse { - children.foldLeft(None: Option[B]) { (l, r) => l.orElse(r.collectFirst(pf)) } + children.foldLeft(Option.empty[B]) { (l, r) => l.orElse(r.collectFirst(pf)) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 78b74f948e682..1c2003c18e3fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -503,7 +503,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def insertInto(tableIdent: TableIdentifier): Unit = { assertNotBucketed("insertInto") assertNotStreaming("insertInto() can only be called on non-continuous queries") - val partitions = normalizedParCols.map(_.map(col => col -> (None: Option[String])).toMap) + val partitions = normalizedParCols.map(_.map(col => col -> (Option.empty[String])).toMap) val overwrite = mode == SaveMode.Overwrite df.sparkSession.sessionState.executePlan( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 66753fa7f27bc..865e406ce27a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -169,7 +169,7 @@ case class CreateDataSourceTableAsSelectCommand( options } - var existingSchema = None: Option[StructType] + var existingSchema = Option.empty[StructType] if (sparkSession.sessionState.catalog.tableExists(tableIdent)) { // Check if we need to throw an exception or just return. mode match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala index e18b59f49b984..afe0fbea73bd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala @@ -129,7 +129,7 @@ case class ShuffleExchange( object ShuffleExchange { def apply(newPartitioning: Partitioning, child: SparkPlan): ShuffleExchange = { - ShuffleExchange(newPartitioning, child, coordinator = None: Option[ExchangeCoordinator]) + ShuffleExchange(newPartitioning, child, coordinator = Option.empty[ExchangeCoordinator]) } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 9771b2314a08d..e6c9c5d4d9cc5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -147,11 +147,11 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("save and load case class RDD with `None`s as orc") { val data = ( - None: Option[Int], - None: Option[Long], - None: Option[Float], - None: Option[Double], - None: Option[Boolean] + Option.empty[Int], + Option.empty[Long], + Option.empty[Float], + Option.empty[Double], + Option.empty[Boolean] ) :: Nil withOrcFile(data) { file => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index c4bc5cf3f6a58..80c07958b41f2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -170,7 +170,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler( */ def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { - var numRecords = None: Option[Long] + var numRecords = Option.empty[Long] // Serialize the block so that it can be inserted into both val serializedBlock = block match { case ArrayBufferBlock(arrayBuffer) => From 8b6742a37d35520eedaee5f3112529136b3a21e4 Mon Sep 17 00:00:00 2001 From: wangyang Date: Fri, 10 Jun 2016 13:10:03 -0700 Subject: [PATCH 0585/1470] [SPARK-15875] Try to use Seq.isEmpty and Seq.nonEmpty instead of Seq.length == 0 and Seq.length > 0 ## What changes were proposed in this pull request? In scala, immutable.List.length is an expensive operation so we should avoid using Seq.length == 0 or Seq.lenth > 0, and use Seq.isEmpty and Seq.nonEmpty instead. ## How was this patch tested? existing tests Author: wangyang Closes #13601 from yangw1234/isEmpty. (cherry picked from commit 026eb90644be7685971dacaabae67a293edd0133) Signed-off-by: Reynold Xin --- .../org/apache/spark/api/python/PythonWorkerFactory.scala | 2 +- .../org/apache/spark/rdd/PartitionerAwareUnionRDD.scala | 2 +- .../spark/util/collection/ExternalAppendOnlyMap.scala | 6 +++--- .../scala/org/apache/spark/mllib/clustering/KMeans.scala | 2 +- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 2 +- .../spark/sql/execution/aggregate/SortAggregateExec.scala | 4 ++-- .../org/apache/spark/sql/execution/metric/SQLMetrics.scala | 2 +- .../apache/spark/streaming/dstream/TransformedDStream.scala | 2 +- 9 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 3df87f62f2f85..6a5e6f7c5afb1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -235,7 +235,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } private def cleanupIdleWorkers() { - while (idleWorkers.length > 0) { + while (idleWorkers.nonEmpty) { val worker = idleWorkers.dequeue() try { // the worker will exit after closing the socket diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index b6366f3e68df9..d744d67592545 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -60,7 +60,7 @@ class PartitionerAwareUnionRDD[T: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[T]] ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) { - require(rdds.length > 0) + require(rdds.nonEmpty) require(rdds.forall(_.partitioner.isDefined)) require(rdds.flatMap(_.partitioner).toSet.size == 1, "Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner)) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index fc71f8365cd18..6ddc72afde270 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -375,14 +375,14 @@ class ExternalAppendOnlyMap[K, V, C]( /** * Return true if there exists an input stream that still has unvisited pairs. */ - override def hasNext: Boolean = mergeHeap.length > 0 + override def hasNext: Boolean = mergeHeap.nonEmpty /** * Select a key with the minimum hash, then combine all values with the same key from all * input streams. */ override def next(): (K, C) = { - if (mergeHeap.length == 0) { + if (mergeHeap.isEmpty) { throw new NoSuchElementException } // Select a key from the StreamBuffer that holds the lowest key hash @@ -397,7 +397,7 @@ class ExternalAppendOnlyMap[K, V, C]( // For all other streams that may have this key (i.e. have the same minimum key hash), // merge in the corresponding value (if any) from that stream val mergedBuffers = ArrayBuffer[StreamBuffer](minBuffer) - while (mergeHeap.length > 0 && mergeHeap.head.minKeyHash == minHash) { + while (mergeHeap.nonEmpty && mergeHeap.head.minKeyHash == minHash) { val newBuffer = mergeHeap.dequeue() minCombiner = mergeIfKeyExists(minKey, minCombiner, newBuffer) mergedBuffers += newBuffer diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 38728f2693c6c..871b1c7d211c8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -441,7 +441,7 @@ class KMeans private ( val rs = (0 until runs).filter { r => rand.nextDouble() < 2.0 * c(r) * k / sumCosts(r) } - if (rs.length > 0) Some((p, rs)) else None + if (rs.nonEmpty) Some((p, rs)) else None } }.collect() mergeNewCenters() 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 a0813572a78f4..07c8bf01c1f0c 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 @@ -1667,7 +1667,7 @@ class Analyzer( // We do a final check and see if we only have a single Window Spec defined in an // expressions. - if (distinctWindowSpec.length == 0 ) { + if (distinctWindowSpec.isEmpty) { failAnalysis(s"$expr does not have any WindowExpression.") } else if (distinctWindowSpec.length > 1) { // newExpressionsWithWindowFunctions only have expressions with a single 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 e455d42dac96e..aaf952f7b22ba 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 @@ -616,7 +616,7 @@ object NullPropagation extends Rule[LogicalPlan] { // For Coalesce, remove null literals. case e @ Coalesce(children) => val newChildren = children.filter(nonNullLiteral) - if (newChildren.length == 0) { + if (newChildren.isEmpty) { Literal.create(null, e.dataType) } else if (newChildren.length == 1) { newChildren.head diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 17126519eb0c0..05dbacf07a178 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -56,8 +56,8 @@ case class SortAggregateExec( override def requiredChildDistribution: List[Distribution] = { requiredChildDistributionExpressions match { - case Some(exprs) if exprs.length == 0 => AllTuples :: Nil - case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) :: Nil + case Some(exprs) if exprs.isEmpty => AllTuples :: Nil + case Some(exprs) if exprs.nonEmpty => ClusteredDistribution(exprs) :: Nil case None => UnspecifiedDistribution :: Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index e63c7c581e9b2..edfdf7cd6b7f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -113,7 +113,7 @@ private[sql] object SQLMetrics { val validValues = values.filter(_ >= 0) val Seq(sum, min, med, max) = { - val metric = if (validValues.length == 0) { + val metric = if (validValues.isEmpty) { Seq.fill(4)(0L) } else { val sorted = validValues.sorted diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 47eb9b806fa7d..0dde120927576 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -29,7 +29,7 @@ class TransformedDStream[U: ClassTag] ( transformFunc: (Seq[RDD[_]], Time) => RDD[U] ) extends DStream[U](parents.head.ssc) { - require(parents.length > 0, "List of DStreams to transform is empty") + require(parents.nonEmpty, "List of DStreams to transform is empty") require(parents.map(_.ssc).distinct.size == 1, "Some of the DStreams have different contexts") require(parents.map(_.slideDuration).distinct.size == 1, "Some of the DStreams have different slide durations") From 80b8711b342c5a569fe89d7ffbdd552653b9b6ec Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Fri, 10 Jun 2016 11:27:30 -0700 Subject: [PATCH 0586/1470] [SPARK-15738][PYSPARK][ML] Adding Pyspark ml RFormula __str__ method similar to Scala API ## What changes were proposed in this pull request? Adding __str__ to RFormula and model that will show the set formula param and resolved formula. This is currently present in the Scala API, found missing in PySpark during Spark 2.0 coverage review. ## How was this patch tested? run pyspark-ml tests locally Author: Bryan Cutler Closes #13481 from BryanCutler/pyspark-ml-rformula_str-SPARK-15738. (cherry picked from commit 7d7a0a5e0749909e97d90188707cc9220a1bb73a) Signed-off-by: Yanbo Liang --- .../org/apache/spark/ml/feature/RFormula.scala | 2 +- .../apache/spark/ml/feature/RFormulaParser.scala | 14 +++++++++++++- python/pyspark/ml/feature.py | 12 ++++++++++++ 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 2916b6d9df3be..a7ca0fe252b0d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -182,7 +182,7 @@ class RFormula(override val uid: String) override def copy(extra: ParamMap): RFormula = defaultCopy(extra) - override def toString: String = s"RFormula(${get(formula)}) (uid=$uid)" + override def toString: String = s"RFormula(${get(formula).getOrElse("")}) (uid=$uid)" } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala index 19aecff03885c..2dd565a782719 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala @@ -126,7 +126,19 @@ private[ml] case class ParsedRFormula(label: ColumnRef, terms: Seq[Term]) { * @param hasIntercept whether the formula specifies fitting with an intercept. */ private[ml] case class ResolvedRFormula( - label: String, terms: Seq[Seq[String]], hasIntercept: Boolean) + label: String, terms: Seq[Seq[String]], hasIntercept: Boolean) { + + override def toString: String = { + val ts = terms.map { + case t if t.length > 1 => + s"${t.mkString("{", ",", "}")}" + case t => + t.mkString + } + val termStr = ts.mkString("[", ",", "]") + s"ResolvedRFormula(label=$label, terms=$termStr, hasIntercept=$hasIntercept)" + } +} /** * R formula terms. See the R formula docs here for more information: diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index bfb2fb707137d..ca77ac395dcca 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2528,6 +2528,8 @@ class RFormula(JavaEstimator, HasFeaturesCol, HasLabelCol, JavaMLReadable, JavaM True >>> loadedRF.getLabelCol() == rf.getLabelCol() True + >>> str(loadedRF) + 'RFormula(y ~ x + s) (uid=...)' >>> modelPath = temp_path + "/rFormulaModel" >>> model.save(modelPath) >>> loadedModel = RFormulaModel.load(modelPath) @@ -2542,6 +2544,8 @@ class RFormula(JavaEstimator, HasFeaturesCol, HasLabelCol, JavaMLReadable, JavaM |0.0|0.0| a|[0.0,1.0]| 0.0| +---+---+---+---------+-----+ ... + >>> str(loadedModel) + 'RFormulaModel(ResolvedRFormula(label=y, terms=[x,s], hasIntercept=true)) (uid=...)' .. versionadded:: 1.5.0 """ @@ -2586,6 +2590,10 @@ def getFormula(self): def _create_model(self, java_model): return RFormulaModel(java_model) + def __str__(self): + formulaStr = self.getFormula() if self.isDefined(self.formula) else "" + return "RFormula(%s) (uid=%s)" % (formulaStr, self.uid) + class RFormulaModel(JavaModel, JavaMLReadable, JavaMLWritable): """ @@ -2597,6 +2605,10 @@ class RFormulaModel(JavaModel, JavaMLReadable, JavaMLWritable): .. versionadded:: 1.5.0 """ + def __str__(self): + resolvedFormula = self._call_java("resolvedFormula") + return "RFormulaModel(%s) (uid=%s)" % (resolvedFormula, self.uid) + @inherit_doc class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, JavaMLReadable, From f2e5d6d0f446d7f1d6d8c3208871074abd669482 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Fri, 10 Jun 2016 14:29:05 -0700 Subject: [PATCH 0587/1470] [SPARK-15825] [SQL] Fix SMJ invalid results ## What changes were proposed in this pull request? Code generated `SortMergeJoin` failed with wrong results when using structs as keys. This could (eventually) be traced back to the use of a wrong row reference when comparing structs. ## How was this patch tested? TBD Author: Herman van Hovell Closes #13589 from hvanhovell/SPARK-15822. (cherry picked from commit e05a2feebe928df691d5a8f42f22e088c6263dcf) Signed-off-by: Davies Liu --- .../expressions/codegen/CodeGenerator.scala | 1 + .../sql/execution/joins/InnerJoinSuite.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+) 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 9657f26402c01..ca202928e8823 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 @@ -490,6 +490,7 @@ class CodegenContext { addNewFunction(compareFunc, funcCode) s"this.$compareFunc($c1, $c2)" case schema: StructType => + INPUT_ROW = "i" val comparisons = GenerateOrdering.genComparisons(this, schema) val compareFunc = freshName("compareStruct") val funcCode: String = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 27f6abcd9509a..35dab63672c05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -271,4 +271,19 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { ) ) } + + { + def df: DataFrame = spark.range(3).selectExpr("struct(id, id) as key", "id as value") + lazy val left = df.selectExpr("key", "concat('L', value) as value").alias("left") + lazy val right = df.selectExpr("key", "concat('R', value) as value").alias("right") + testInnerJoin( + "SPARK-15822 - test structs as keys", + left, + right, + () => (left.col("key") === right.col("key")).expr, + Seq( + (Row(0, 0), "L0", Row(0, 0), "R0"), + (Row(1, 1), "L1", Row(1, 1), "R1"), + (Row(2, 2), "L2", Row(2, 2), "R2"))) + } } From bc53422ad54460069f0e36061c6be5ef76b4dbaa Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 10 Jun 2016 14:32:43 -0700 Subject: [PATCH 0588/1470] [SPARK-15654] [SQL] fix non-splitable files for text based file formats ## What changes were proposed in this pull request? Currently, we always split the files when it's bigger than maxSplitBytes, but Hadoop LineRecordReader does not respect the splits for compressed files correctly, we should have a API for FileFormat to check whether the file could be splitted or not. This PR is based on #13442, closes #13442 ## How was this patch tested? add regression tests. Author: Davies Liu Closes #13531 from davies/fix_split. (cherry picked from commit aec502d9114ad8e18bfbbd63f38780e076d326d1) Signed-off-by: Davies Liu --- .../ml/source/libsvm/LibSVMRelation.scala | 2 +- .../datasources/FileSourceStrategy.scala | 17 ++++++--- .../datasources/csv/CSVFileFormat.scala | 2 +- .../datasources/fileSourceInterfaces.scala | 33 ++++++++++++++++- .../datasources/json/JsonFileFormat.scala | 2 +- .../parquet/ParquetFileFormat.scala | 7 ++++ .../datasources/text/TextFileFormat.scala | 2 +- .../datasources/FileSourceStrategySuite.scala | 37 ++++++++++++++++++- .../datasources/text/TextSuite.scala | 17 +++++++++ .../spark/sql/hive/orc/OrcFileFormat.scala | 7 ++++ .../sql/sources/SimpleTextRelation.scala | 2 +- 11 files changed, 115 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 7629369ab1379..b5b2a681e9a92 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -112,7 +112,7 @@ private[libsvm] class LibSVMOutputWriter( */ // If this is moved or renamed, please update DataSource's backwardCompatibilityMap. @Since("1.6.0") -class LibSVMFileFormat extends FileFormat with DataSourceRegister { +class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { @Since("1.6.0") override def shortName(): String = "libsvm" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 7503285ee2c1f..13a86bfb3896f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -151,11 +151,18 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { val splitFiles = selectedPartitions.flatMap { partition => partition.files.flatMap { file => val blockLocations = getBlockLocations(file) - (0L until file.getLen by maxSplitBytes).map { offset => - val remaining = file.getLen - offset - val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining - val hosts = getBlockHosts(blockLocations, offset, size) - PartitionedFile(partition.values, file.getPath.toUri.toString, offset, size, hosts) + if (files.fileFormat.isSplitable(files.sparkSession, files.options, file.getPath)) { + (0L until file.getLen by maxSplitBytes).map { offset => + val remaining = file.getLen - offset + val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining + val hosts = getBlockHosts(blockLocations, offset, size) + PartitionedFile( + partition.values, file.getPath.toUri.toString, offset, size, hosts) + } + } else { + val hosts = getBlockHosts(blockLocations, 0, file.getLen) + Seq(PartitionedFile( + partition.values, file.getPath.toUri.toString, 0, file.getLen, hosts)) } } }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 4d36b760568cc..be52de8e40140 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -36,7 +36,7 @@ import org.apache.spark.util.SerializableConfiguration /** * Provides access to CSV data from pure SQL statements. */ -class CSVFileFormat extends FileFormat with DataSourceRegister { +class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { override def shortName(): String = "csv" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 7f3eed3fb1f80..890e64db59945 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -22,6 +22,7 @@ import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ +import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} @@ -29,12 +30,12 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.FileRelation import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, Filter} -import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration /** @@ -214,6 +215,16 @@ trait FileFormat { false } + /** + * Returns whether a file with `path` could be splitted or not. + */ + def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + false + } + /** * Returns a function that can be used to read a single file in as an Iterator of InternalRow. * @@ -297,6 +308,24 @@ trait FileFormat { } } +/** + * The base class file format that is based on text file. + */ +abstract class TextBasedFileFormat extends FileFormat { + private var codecFactory: CompressionCodecFactory = null + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + if (codecFactory == null) { + codecFactory = new CompressionCodecFactory( + sparkSession.sessionState.newHadoopConfWithOptions(options)) + } + val codec = codecFactory.getCodec(path) + codec == null || codec.isInstanceOf[SplittableCompressionCodec] + } +} + /** * A collection of data files from a partitioned relation, along with the partition values in the * form of an [[InternalRow]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index c7c528119660c..86aef1f7d4411 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -class JsonFileFormat extends FileFormat with DataSourceRegister { +class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { override def shortName(): String = "json" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index ada9cd4b8e812..3735c9496888b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -261,6 +261,13 @@ private[sql] class ParquetFileFormat schema.forall(_.dataType.isInstanceOf[AtomicType]) } + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + true + } + override private[sql] def buildReaderWithPartitionValues( sparkSession: SparkSession, dataSchema: StructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index 9c03ab28dd769..abb6059f75ba8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -36,7 +36,7 @@ import org.apache.spark.util.SerializableConfiguration /** * A data source for reading text files. */ -class TextFileFormat extends FileFormat with DataSourceRegister { +class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { override def shortName(): String = "text" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 25f1443e7080a..67ff257b9386a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -340,6 +340,41 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } } + test("SPARK-15654 do not split non-splittable files") { + // Check if a non-splittable file is not assigned into partitions + Seq("gz", "snappy", "lz4").map { suffix => + val table = createTable( + files = Seq(s"file1.${suffix}" -> 3, s"file2.${suffix}" -> 1, s"file3.${suffix}" -> 1) + ) + withSQLConf( + SQLConf.FILES_MAX_PARTITION_BYTES.key -> "2", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "0") { + checkScan(table.select('c1)) { partitions => + assert(partitions.size == 2) + assert(partitions(0).files.size == 1) + assert(partitions(1).files.size == 2) + } + } + } + + // Check if a splittable compressed file is assigned into multiple partitions + Seq("bz2").map { suffix => + val table = createTable( + files = Seq(s"file1.${suffix}" -> 3, s"file2.${suffix}" -> 1, s"file3.${suffix}" -> 1) + ) + withSQLConf( + SQLConf.FILES_MAX_PARTITION_BYTES.key -> "2", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "0") { + checkScan(table.select('c1)) { partitions => + assert(partitions.size == 3) + assert(partitions(0).files.size == 1) + assert(partitions(1).files.size == 2) + assert(partitions(2).files.size == 1) + } + } + } + } + // Helpers for checking the arguments passed to the FileFormat. protected val checkPartitionSchema = @@ -434,7 +469,7 @@ object LastArguments { } /** A test [[FileFormat]] that records the arguments passed to buildReader, and returns nothing. */ -class TestFileFormat extends FileFormat { +class TestFileFormat extends TextBasedFileFormat { override def toString: String = "TestFileFormat" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 7b6981f95e9dd..5695f6af7bd44 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.Utils @@ -137,6 +138,22 @@ class TextSuite extends QueryTest with SharedSQLContext { } } + test("SPARK-15654: should not split gz files") { + withTempDir { dir => + val path = dir.getCanonicalPath + val df1 = spark.range(0, 1000).selectExpr("CAST(id AS STRING) AS s") + df1.write.option("compression", "gzip").mode("overwrite").text(path) + + val expected = df1.collect() + Seq(10, 100, 1000).foreach { bytes => + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> bytes.toString) { + val df2 = spark.read.format("text").load(path) + checkAnswer(df2, expected) + } + } + } + } + private def testFile: String = { Thread.currentThread().getContextClassLoader.getResource("text-suite.txt").toString } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 0e8c37df88ee5..a2c8092e01bb9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -95,6 +95,13 @@ private[sql] class OrcFileFormat } } + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + true + } + override def buildReader( sparkSession: SparkSession, dataSchema: StructType, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 1fb777ade4b22..67a58a3859b84 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.SerializableConfiguration -class SimpleTextSource extends FileFormat with DataSourceRegister { +class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { override def shortName(): String = "test" override def inferSchema( From e6ebb547b197f906b9706847ad871b337b4a9e7f Mon Sep 17 00:00:00 2001 From: Sela Date: Fri, 10 Jun 2016 14:36:51 -0700 Subject: [PATCH 0589/1470] [SPARK-15489][SQL] Dataset kryo encoder won't load custom user settings ## What changes were proposed in this pull request? Serializer instantiation will consider existing SparkConf ## How was this patch tested? manual test with `ImmutableList` (Guava) and `kryo-serializers`'s `Immutable*Serializer` implementations. Added Test Suite. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Sela Closes #13424 from amitsela/SPARK-15489. (cherry picked from commit 127a6678d7af6b5164a115be7c64525bb80001fe) Signed-off-by: Michael Armbrust --- .../expressions/objects/objects.scala | 30 +++++--- .../DatasetSerializerRegistratorSuite.scala | 68 +++++++++++++++++++ 2 files changed, 89 insertions(+), 9 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 87c8a2e54a2c2..c597a2a709445 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -22,7 +22,7 @@ import java.lang.reflect.Modifier import scala.language.existentials import scala.reflect.ClassTag -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.serializer._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow @@ -547,11 +547,17 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) (classOf[JavaSerializer].getName, classOf[JavaSerializerInstance].getName) } } + // try conf from env, otherwise create a new one + val env = s"${classOf[SparkEnv].getName}.get()" val sparkConf = s"new ${classOf[SparkConf].getName}()" - ctx.addMutableState( - serializerInstanceClass, - serializer, - s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") + val serializerInit = s""" + if ($env == null) { + $serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance(); + } else { + $serializer = ($serializerInstanceClass) new $serializerClass($env.conf()).newInstance(); + } + """ + ctx.addMutableState(serializerInstanceClass, serializer, serializerInit) // Code to serialize. val input = child.genCode(ctx) @@ -587,11 +593,17 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B (classOf[JavaSerializer].getName, classOf[JavaSerializerInstance].getName) } } + // try conf from env, otherwise create a new one + val env = s"${classOf[SparkEnv].getName}.get()" val sparkConf = s"new ${classOf[SparkConf].getName}()" - ctx.addMutableState( - serializerInstanceClass, - serializer, - s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") + val serializerInit = s""" + if ($env == null) { + $serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance(); + } else { + $serializer = ($serializerInstanceClass) new $serializerClass($env.conf()).newInstance(); + } + """ + ctx.addMutableState(serializerInstanceClass, serializer, serializerInit) // Code to deserialize. val input = child.genCode(ctx) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala new file mode 100644 index 0000000000000..0f3d0cefe3bb5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import com.esotericsoftware.kryo.{Kryo, Serializer} +import com.esotericsoftware.kryo.io.{Input, Output} + +import org.apache.spark.serializer.KryoRegistrator +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.TestSparkSession + +/** + * Test suite to test Kryo custom registrators. + */ +class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + /** + * Initialize the [[TestSparkSession]] with a [[KryoRegistrator]]. + */ + protected override def beforeAll(): Unit = { + sparkConf.set("spark.kryo.registrator", TestRegistrator().getClass.getCanonicalName) + super.beforeAll() + } + + test("Kryo registrator") { + implicit val kryoEncoder = Encoders.kryo[KryoData] + val ds = Seq(KryoData(1), KryoData(2)).toDS() + assert(ds.collect().toSet == Set(KryoData(0), KryoData(0))) + } + +} + +/** Used to test user provided registrator. */ +class TestRegistrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo): Unit = + kryo.register(classOf[KryoData], new ZeroKryoDataSerializer()) +} + +object TestRegistrator { + def apply(): TestRegistrator = new TestRegistrator() +} + +/** A [[Serializer]] that takes a [[KryoData]] and serializes it as KryoData(0). */ +class ZeroKryoDataSerializer extends Serializer[KryoData] { + override def write(kryo: Kryo, output: Output, t: KryoData): Unit = { + output.writeInt(0) + } + + override def read(kryo: Kryo, input: Input, aClass: Class[KryoData]): KryoData = { + KryoData(input.readInt()) + } +} From f41f433b101d5eac5bdd3a033e15f69e2215d30d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 10 Jun 2016 15:40:29 -0700 Subject: [PATCH 0590/1470] [SPARK-15773][CORE][EXAMPLE] Avoid creating local variable `sc` in examples if possible ## What changes were proposed in this pull request? Instead of using local variable `sc` like the following example, this PR uses `spark.sparkContext`. This makes examples more concise, and also fixes some misleading, i.e., creating SparkContext from SparkSession. ``` - println("Creating SparkContext") - val sc = spark.sparkContext - println("Writing local file to DFS") val dfsFilename = dfsDirPath + "/dfs_read_write_test" - val fileRDD = sc.parallelize(fileContents) + val fileRDD = spark.sparkContext.parallelize(fileContents) ``` This will change 12 files (+30 lines, -52 lines). ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13520 from dongjoon-hyun/SPARK-15773. (cherry picked from commit 2022afe57dbf8cb0c9909399962c4a3649e0601c) Signed-off-by: Reynold Xin --- examples/src/main/python/pi.py | 4 +--- examples/src/main/python/transitive_closure.py | 4 +--- .../apache/spark/examples/DFSReadWriteTest.scala | 7 ++----- .../spark/examples/ExceptionHandlingTest.scala | 3 +-- .../org/apache/spark/examples/GroupByTest.scala | 14 ++++++-------- .../spark/examples/MultiBroadcastTest.scala | 8 +++----- .../spark/examples/SimpleSkewedGroupByTest.scala | 16 +++++++--------- .../spark/examples/SkewedGroupByTest.scala | 13 +++++-------- .../org/apache/spark/examples/SparkLR.scala | 4 +--- .../org/apache/spark/examples/SparkPi.scala | 3 +-- .../org/apache/spark/examples/SparkTC.scala | 3 +-- .../spark/examples/sql/hive/HiveFromSpark.scala | 3 +-- 12 files changed, 30 insertions(+), 52 deletions(-) diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index b39d710540b1b..e3f0c4aeef1b7 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -32,8 +32,6 @@ .appName("PythonPi")\ .getOrCreate() - sc = spark.sparkContext - partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * partitions @@ -42,7 +40,7 @@ def f(_): y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + count = spark.sparkContext.parallelize(range(1, n + 1), partitions).map(f).reduce(add) print("Pi is roughly %f" % (4.0 * count / n)) spark.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index d88ea94e41c27..49551d40851cc 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -46,10 +46,8 @@ def generateGraph(): .appName("PythonTransitiveClosure")\ .getOrCreate() - sc = spark.sparkContext - partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - tc = sc.parallelize(generateGraph(), partitions).cache() + tc = spark.sparkContext.parallelize(generateGraph(), partitions).cache() # Linear transitive closure: each round grows paths by one edge, # by joining the graph's edges with the already-discovered paths. diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index 4b5e36c73642d..3bff7ce736d08 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -107,16 +107,13 @@ object DFSReadWriteTest { .appName("DFS Read Write Test") .getOrCreate() - println("Creating SparkContext") - val sc = spark.sparkContext - println("Writing local file to DFS") val dfsFilename = dfsDirPath + "/dfs_read_write_test" - val fileRDD = sc.parallelize(fileContents) + val fileRDD = spark.sparkContext.parallelize(fileContents) fileRDD.saveAsTextFile(dfsFilename) println("Reading file from DFS and running Word Count") - val readFileRDD = sc.textFile(dfsFilename) + val readFileRDD = spark.sparkContext.textFile(dfsFilename) val dfsWordCount = readFileRDD .flatMap(_.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index 6a1bbed290235..45c4953a84be2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -25,9 +25,8 @@ object ExceptionHandlingTest { .builder .appName("ExceptionHandlingTest") .getOrCreate() - val sc = spark.sparkContext - sc.parallelize(0 until sc.defaultParallelism).foreach { i => + spark.sparkContext.parallelize(0 until spark.sparkContext.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 0cb61d7495fa6..2f2bbb1275438 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -32,16 +32,14 @@ object GroupByTest { .appName("GroupBy Test") .getOrCreate() - var numMappers = if (args.length > 0) args(0).toInt else 2 - var numKVPairs = if (args.length > 1) args(1).toInt else 1000 - var valSize = if (args.length > 2) args(2).toInt else 1000 - var numReducers = if (args.length > 3) args(3).toInt else numMappers + val numMappers = if (args.length > 0) args(0).toInt else 2 + val numKVPairs = if (args.length > 1) args(1).toInt else 1000 + val valSize = if (args.length > 2) args(2).toInt else 1000 + val numReducers = if (args.length > 3) args(3).toInt else numMappers - val sc = spark.sparkContext - - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => + val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random - var arr1 = new Array[(Int, Array[Byte])](numKVPairs) + val arr1 = new Array[(Int, Array[Byte])](numKVPairs) for (i <- 0 until numKVPairs) { val byteArr = new Array[Byte](valSize) ranGen.nextBytes(byteArr) diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 961ab99200d29..6495a86fcd77c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -33,8 +33,6 @@ object MultiBroadcastTest { .appName("Multi-Broadcast Test") .getOrCreate() - val sc = spark.sparkContext - val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 @@ -48,9 +46,9 @@ object MultiBroadcastTest { arr2(i) = i } - val barr1 = sc.broadcast(arr1) - val barr2 = sc.broadcast(arr2) - val observedSizes: RDD[(Int, Int)] = sc.parallelize(1 to 10, slices).map { _ => + val barr1 = spark.sparkContext.broadcast(arr1) + val barr2 = spark.sparkContext.broadcast(arr2) + val observedSizes: RDD[(Int, Int)] = spark.sparkContext.parallelize(1 to 10, slices).map { _ => (barr1.value.length, barr2.value.length) } // Collect the small RDD so we can print the observed sizes locally. diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 255c2bfcee669..8e1a574c92221 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -32,17 +32,15 @@ object SimpleSkewedGroupByTest { .appName("SimpleSkewedGroupByTest") .getOrCreate() - val sc = spark.sparkContext + val numMappers = if (args.length > 0) args(0).toInt else 2 + val numKVPairs = if (args.length > 1) args(1).toInt else 1000 + val valSize = if (args.length > 2) args(2).toInt else 1000 + val numReducers = if (args.length > 3) args(3).toInt else numMappers + val ratio = if (args.length > 4) args(4).toInt else 5.0 - var numMappers = if (args.length > 0) args(0).toInt else 2 - var numKVPairs = if (args.length > 1) args(1).toInt else 1000 - var valSize = if (args.length > 2) args(2).toInt else 1000 - var numReducers = if (args.length > 3) args(3).toInt else numMappers - var ratio = if (args.length > 4) args(4).toInt else 5.0 - - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => + val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random - var result = new Array[(Int, Array[Byte])](numKVPairs) + val result = new Array[(Int, Array[Byte])](numKVPairs) for (i <- 0 until numKVPairs) { val byteArr = new Array[Byte](valSize) ranGen.nextBytes(byteArr) diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index efd40147f7ca7..4d3c34041bc17 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -32,21 +32,18 @@ object SkewedGroupByTest { .appName("GroupBy Test") .getOrCreate() - val sc = spark.sparkContext - - var numMappers = if (args.length > 0) args(0).toInt else 2 + val numMappers = if (args.length > 0) args(0).toInt else 2 var numKVPairs = if (args.length > 1) args(1).toInt else 1000 - var valSize = if (args.length > 2) args(2).toInt else 1000 - var numReducers = if (args.length > 3) args(3).toInt else numMappers - + val valSize = if (args.length > 2) args(2).toInt else 1000 + val numReducers = if (args.length > 3) args(3).toInt else numMappers - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => + val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random // map output sizes linearly increase from the 1st to the last numKVPairs = (1.0 * (p + 1) / numMappers * numKVPairs).toInt - var arr1 = new Array[(Int, Array[Byte])](numKVPairs) + val arr1 = new Array[(Int, Array[Byte])](numKVPairs) for (i <- 0 until numKVPairs) { val byteArr = new Array[Byte](valSize) ranGen.nextBytes(byteArr) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 8ef3aab6570d6..afa8f58c96e59 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -68,10 +68,8 @@ object SparkLR { .appName("SparkLR") .getOrCreate() - val sc = spark.sparkContext - val numSlices = if (args.length > 0) args(0).toInt else 2 - val points = sc.parallelize(generateData, numSlices).cache() + val points = spark.sparkContext.parallelize(generateData, numSlices).cache() // Initialize w to a random value var w = DenseVector.fill(D) {2 * rand.nextDouble - 1} diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 5be8f3b073410..42f6cef4e189b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -29,10 +29,9 @@ object SparkPi { .builder .appName("Spark Pi") .getOrCreate() - val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow - val count = sc.parallelize(1 until n, slices).map { i => + val count = spark.sparkContext.parallelize(1 until n, slices).map { i => val x = random * 2 - 1 val y = random * 2 - 1 if (x*x + y*y < 1) 1 else 0 diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 46aa68b8b8195..558295ab928af 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -46,9 +46,8 @@ object SparkTC { .builder .appName("SparkTC") .getOrCreate() - val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 - var tc = sc.parallelize(generateGraph, slices).cache() + var tc = spark.sparkContext.parallelize(generateGraph, slices).cache() // Linear transitive closure: each round grows paths by one edge, // by joining the graph's edges with the already-discovered paths. diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 2d7a01a95d830..2343f98c8d07c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -45,7 +45,6 @@ object HiveFromSpark { .appName("HiveFromSpark") .enableHiveSupport() .getOrCreate() - val sc = spark.sparkContext import spark.implicits._ import spark.sql @@ -71,7 +70,7 @@ object HiveFromSpark { } // You can also use RDDs to create temporary views within a HiveContext. - val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) + val rdd = spark.sparkContext.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) rdd.toDF().createOrReplaceTempView("records") // Queries can then join RDD data with data stored in Hive. From 0a450cfffada67f841795a09af3bf6320343b358 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Fri, 10 Jun 2016 17:17:47 -0700 Subject: [PATCH 0591/1470] [SPARK-15884][SPARKR][SQL] Overriding stringArgs in MapPartitionsInR ## What changes were proposed in this pull request? As discussed in https://github.com/apache/spark/pull/12836 we need to override stringArgs method in MapPartitionsInR in order to avoid too large strings generated by "stringArgs" method based on the input arguments. In this case exclude some of the input arguments: serialized R objects. ## How was this patch tested? Existing test cases Author: Narine Kokhlikyan Closes #13610 from NarineK/dapply_MapPartitionsInR_stringArgs. (cherry picked from commit 54f758b5fc60ecb0da6b191939a72ef5829be38c) Signed-off-by: Cheng Lian --- .../org/apache/spark/sql/catalyst/plans/logical/object.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 55d8adf0408fd..78e8822b6405a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -141,6 +141,9 @@ case class MapPartitionsInR( outputObjAttr: Attribute, child: LogicalPlan) extends ObjectConsumer with ObjectProducer { override lazy val schema = outputSchema + + override protected def stringArgs: Iterator[Any] = Iterator(inputSchema, outputSchema, + outputObjAttr, child) } object MapElements { From 7d6bd1196410563bd1fccc10e7bff6e75b5c9f22 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 10 Jun 2016 18:23:59 -0700 Subject: [PATCH 0592/1470] [SPARK-15639][SQL] Try to push down filter at RowGroups level for parquet reader ## What changes were proposed in this pull request? The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader. ## How was this patch tested? Existing tests should be passed. Author: Liang-Chi Hsieh Closes #13371 from viirya/vectorized-reader-push-down-filter. (cherry picked from commit bba5d7999f7b3ae9d816ea552ba9378fea1615a6) Signed-off-by: Cheng Lian --- .../expressions/namedExpressions.scala | 8 +++ .../datasources/FileSourceStrategy.scala | 9 ++- .../parquet/ParquetFileFormat.scala | 61 ++----------------- 3 files changed, 21 insertions(+), 57 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 306a99d5a37bf..c06a1ea356093 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -292,6 +292,14 @@ case class AttributeReference( } } + def withMetadata(newMetadata: Metadata): AttributeReference = { + if (metadata == newMetadata) { + this + } else { + AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier, isGenerated) + } + } + override protected final def otherCopyArgs: Seq[AnyRef] = { exprId :: qualifier :: isGenerated :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 13a86bfb3896f..7fc842f83ecb7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -84,7 +84,14 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { logInfo(s"Pruning directories with: ${partitionKeyFilters.mkString(",")}") val dataColumns = - l.resolve(files.dataSchema, files.sparkSession.sessionState.analyzer.resolver) + l.resolve(files.dataSchema, files.sparkSession.sessionState.analyzer.resolver).map { c => + files.dataSchema.find(_.name == c.name).map { f => + c match { + case a: AttributeReference => a.withMetadata(f.metadata) + case _ => c + } + }.getOrElse(c) + } // Partition keys are not available in the statistics of the files. val dataFilters = normalizedFilters.filter(_.references.intersect(partitionSet).isEmpty) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 3735c9496888b..bc4a9de7a9378 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -357,6 +357,11 @@ private[sql] class ParquetFileFormat val hadoopAttemptContext = new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId) + // Try to push down filters when filter push-down is enabled. + // Notice: This push-down is RowGroups level, not individual records. + pushed.foreach { + ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, _) + } val parquetReader = if (enableVectorizedReader) { val vectorizedReader = new VectorizedParquetRecordReader() vectorizedReader.initialize(split, hadoopAttemptContext) @@ -592,62 +597,6 @@ private[sql] object ParquetFileFormat extends Logging { } } - /** This closure sets various Parquet configurations at both driver side and executor side. */ - private[parquet] def initializeLocalJobFunc( - requiredColumns: Array[String], - filters: Array[Filter], - dataSchema: StructType, - parquetBlockSize: Long, - useMetadataCache: Boolean, - parquetFilterPushDown: Boolean, - assumeBinaryIsString: Boolean, - assumeInt96IsTimestamp: Boolean)(job: Job): Unit = { - val conf = job.getConfiguration - conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[CatalystReadSupport].getName) - - // Try to push down filters when filter push-down is enabled. - if (parquetFilterPushDown) { - filters - // Collects all converted Parquet filter predicates. Notice that not all predicates can be - // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` - // is used here. - .flatMap(ParquetFilters.createFilter(dataSchema, _)) - .reduceOption(FilterApi.and) - .foreach(ParquetInputFormat.setFilterPredicate(conf, _)) - } - - conf.set(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { - val requestedSchema = StructType(requiredColumns.map(dataSchema(_))) - CatalystSchemaConverter.checkFieldNames(requestedSchema).json - }) - - conf.set( - CatalystWriteSupport.SPARK_ROW_SCHEMA, - CatalystSchemaConverter.checkFieldNames(dataSchema).json) - - // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata - conf.setBoolean(SQLConf.PARQUET_CACHE_METADATA.key, useMetadataCache) - - // Sets flags for `CatalystSchemaConverter` - conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, assumeBinaryIsString) - conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, assumeInt96IsTimestamp) - - overrideMinSplitSize(parquetBlockSize, conf) - } - - /** This closure sets input paths at the driver side. */ - private[parquet] def initializeDriverSideJobFunc( - inputFiles: Array[FileStatus], - parquetBlockSize: Long)(job: Job): Unit = { - // We side the input paths at the driver side. - logInfo(s"Reading Parquet file(s) from ${inputFiles.map(_.getPath).mkString(", ")}") - if (inputFiles.nonEmpty) { - FileInputFormat.setInputPaths(job, inputFiles.map(_.getPath): _*) - } - - overrideMinSplitSize(parquetBlockSize, job.getConfiguration) - } - private[parquet] def readSchema( footers: Seq[Footer], sparkSession: SparkSession): Option[StructType] = { From 798825c09ba55dca449bde3f00ff2aeafd6b05b7 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 10 Jun 2016 18:29:26 -0700 Subject: [PATCH 0593/1470] [SPARK-14615][ML][FOLLOWUP] Fix Python examples to use the new ML Vector and Matrix APIs in the ML pipeline based algorithms ## What changes were proposed in this pull request? This PR fixes Python examples to use the new ML Vector and Matrix APIs in the ML pipeline based algorithms. I firstly executed this shell command, `grep -r "from pyspark.mllib" .` and then executed them all. Some of tests in `ml` produced the error messages as below: ``` pyspark.sql.utils.IllegalArgumentException: u'requirement failed: Input type must be VectorUDT but got org.apache.spark.mllib.linalg.VectorUDTf71b0bce.' ``` So, I fixed them to use new ones just identically with some Python tests fixed in https://github.com/apache/spark/pull/12627 ## How was this patch tested? Manually tested for all the examples listed by `grep -r "from pyspark.mllib" .`. Author: hyukjinkwon Closes #13393 from HyukjinKwon/SPARK-14615. (cherry picked from commit 99f3c82776fe5ea4f89a9965a288c7447585dc2c) Signed-off-by: Joseph K. Bradley --- .../main/python/ml/aft_survival_regression.py | 2 +- .../main/python/ml/chisq_selector_example.py | 2 +- examples/src/main/python/ml/dct_example.py | 2 +- .../python/ml/elementwise_product_example.py | 2 +- .../ml/estimator_transformer_param_example.py | 2 +- examples/src/main/python/ml/pca_example.py | 2 +- .../python/ml/polynomial_expansion_example.py | 2 +- .../main/python/ml/simple_params_example.py | 19 +++++++++---------- .../python/ml/vector_assembler_example.py | 2 +- .../main/python/ml/vector_slicer_example.py | 2 +- 10 files changed, 18 insertions(+), 19 deletions(-) diff --git a/examples/src/main/python/ml/aft_survival_regression.py b/examples/src/main/python/ml/aft_survival_regression.py index 9879679829d4d..060f0171ffdb5 100644 --- a/examples/src/main/python/ml/aft_survival_regression.py +++ b/examples/src/main/python/ml/aft_survival_regression.py @@ -19,7 +19,7 @@ # $example on$ from pyspark.ml.regression import AFTSurvivalRegression -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors # $example off$ from pyspark.sql import SparkSession diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py index 8bafb942e0d27..5e19ef1624c7e 100644 --- a/examples/src/main/python/ml/chisq_selector_example.py +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -20,7 +20,7 @@ from pyspark.sql import SparkSession # $example on$ from pyspark.ml.feature import ChiSqSelector -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors # $example off$ if __name__ == "__main__": diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py index e36fcdeaeed28..a4f25df784886 100644 --- a/examples/src/main/python/ml/dct_example.py +++ b/examples/src/main/python/ml/dct_example.py @@ -19,7 +19,7 @@ # $example on$ from pyspark.ml.feature import DCT -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors # $example off$ from pyspark.sql import SparkSession diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index 41727edcdb09e..598deae886ee1 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -19,7 +19,7 @@ # $example on$ from pyspark.ml.feature import ElementwiseProduct -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors # $example off$ from pyspark.sql import SparkSession diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py index 0fcae0e3fc225..3bd3fd30f8e57 100644 --- a/examples/src/main/python/ml/estimator_transformer_param_example.py +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -20,7 +20,7 @@ """ # $example on$ -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors from pyspark.ml.classification import LogisticRegression # $example off$ from pyspark.sql import SparkSession diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py index f1b3cdec7bd77..414629ff88bf9 100644 --- a/examples/src/main/python/ml/pca_example.py +++ b/examples/src/main/python/ml/pca_example.py @@ -19,7 +19,7 @@ # $example on$ from pyspark.ml.feature import PCA -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors # $example off$ from pyspark.sql import SparkSession diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 08882bcb256f4..9475e33218cfd 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -19,7 +19,7 @@ # $example on$ from pyspark.ml.feature import PolynomialExpansion -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors # $example off$ from pyspark.sql import SparkSession diff --git a/examples/src/main/python/ml/simple_params_example.py b/examples/src/main/python/ml/simple_params_example.py index c57e59d01b547..54fbc2c9d05df 100644 --- a/examples/src/main/python/ml/simple_params_example.py +++ b/examples/src/main/python/ml/simple_params_example.py @@ -21,9 +21,8 @@ import sys from pyspark.ml.classification import LogisticRegression -from pyspark.mllib.linalg import DenseVector -from pyspark.mllib.regression import LabeledPoint -from pyspark.sql import SparkSession +from pyspark.ml.linalg import DenseVector +from pyspark.sql import Row, SparkSession """ A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -42,10 +41,10 @@ # A LabeledPoint is an Object with two fields named label and features # and Spark SQL identifies these fields and creates the schema appropriately. training = spark.createDataFrame([ - LabeledPoint(1.0, DenseVector([0.0, 1.1, 0.1])), - LabeledPoint(0.0, DenseVector([2.0, 1.0, -1.0])), - LabeledPoint(0.0, DenseVector([2.0, 1.3, 1.0])), - LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]) + Row(label=1.0, features=DenseVector([0.0, 1.1, 0.1])), + Row(label=0.0, features=DenseVector([2.0, 1.0, -1.0])), + Row(label=0.0, features=DenseVector([2.0, 1.3, 1.0])), + Row(label=1.0, features=DenseVector([0.0, 1.2, -0.5]))]) # Create a LogisticRegression instance with maxIter = 10. # This instance is an Estimator. @@ -77,9 +76,9 @@ # prepare test data. test = spark.createDataFrame([ - LabeledPoint(1.0, DenseVector([-1.0, 1.5, 1.3])), - LabeledPoint(0.0, DenseVector([3.0, 2.0, -0.1])), - LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]) + Row(label=1.0, features=DenseVector([-1.0, 1.5, 1.3])), + Row(label=0.0, features=DenseVector([3.0, 2.0, -0.1])), + Row(label=0.0, features=DenseVector([0.0, 2.2, -1.5]))]) # Make predictions on test data using the Transformer.transform() method. # LogisticRegressionModel.transform will only use the 'features' column. diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py index b955ff00a8195..bbfc316ff2d33 100644 --- a/examples/src/main/python/ml/vector_assembler_example.py +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -18,7 +18,7 @@ from __future__ import print_function # $example on$ -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors from pyspark.ml.feature import VectorAssembler # $example off$ from pyspark.sql import SparkSession diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index b833a894eb841..d2f46b190f9a8 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -19,7 +19,7 @@ # $example on$ from pyspark.ml.feature import VectorSlicer -from pyspark.mllib.linalg import Vectors +from pyspark.ml.linalg import Vectors from pyspark.sql.types import Row # $example off$ from pyspark.sql import SparkSession From a08715c7a79ce1953b8d64a9cf0ec1c513d56eec Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 10 Jun 2016 20:43:18 -0700 Subject: [PATCH 0594/1470] [SPARK-15678] Add support to REFRESH data source paths ## What changes were proposed in this pull request? Spark currently incorrectly continues to use cached data even if the underlying data is overwritten. Current behavior: ```scala val dir = "/tmp/test" sqlContext.range(1000).write.mode("overwrite").parquet(dir) val df = sqlContext.read.parquet(dir).cache() df.count() // outputs 1000 sqlContext.range(10).write.mode("overwrite").parquet(dir) sqlContext.read.parquet(dir).count() // outputs 1000 <---- We are still using the cached dataset ``` This patch fixes this bug by adding support for `REFRESH path` that invalidates and refreshes all the cached data (and the associated metadata) for any dataframe that contains the given data source path. Expected behavior: ```scala val dir = "/tmp/test" sqlContext.range(1000).write.mode("overwrite").parquet(dir) val df = sqlContext.read.parquet(dir).cache() df.count() // outputs 1000 sqlContext.range(10).write.mode("overwrite").parquet(dir) spark.catalog.refreshResource(dir) sqlContext.read.parquet(dir).count() // outputs 10 <---- We are not using the cached dataset ``` ## How was this patch tested? Unit tests for overwrites and appends in `ParquetQuerySuite` and `CachedTableSuite`. Author: Sameer Agarwal Closes #13566 from sameeragarwal/refresh-path-2. (cherry picked from commit 468da03e23a01e02718608f05d778386cbb8416b) Signed-off-by: Davies Liu --- .../spark/sql/catalyst/parser/SqlBase.g4 | 1 + .../apache/spark/sql/catalog/Catalog.scala | 7 +++ .../spark/sql/execution/CacheManager.scala | 51 ++++++++++++++++++- .../spark/sql/execution/SparkSqlParser.scala | 9 +++- .../spark/sql/execution/datasources/ddl.scala | 9 ++++ .../spark/sql/internal/CatalogImpl.scala | 10 ++++ .../parquet/ParquetQuerySuite.scala | 28 ++++++++++ .../spark/sql/hive/CachedTableSuite.scala | 45 ++++++++++++++++ 8 files changed, 158 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index d10255946af90..044f91038876a 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -113,6 +113,7 @@ statement | (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)? tableIdentifier partitionSpec? describeColName? #describeTable | REFRESH TABLE tableIdentifier #refreshTable + | REFRESH .*? #refreshResource | CACHE LAZY? TABLE identifier (AS? query)? #cacheTable | UNCACHE TABLE identifier #uncacheTable | CLEAR CACHE #clearCache diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 6ddb1a7a1f1a5..083a63c98c43b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -226,4 +226,11 @@ abstract class Catalog { */ def refreshTable(tableName: String): Unit + /** + * Invalidate and refresh all the cached data (and the associated metadata) for any dataframe that + * contains the given data source path. + * + * @since 2.0.0 + */ + def refreshByPath(path: String): Unit } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index c8bdb0d22c9f8..b584cf4079769 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -19,10 +19,14 @@ package org.apache.spark.sql.execution import java.util.concurrent.locks.ReentrantReadWriteLock +import org.apache.hadoop.fs.{FileSystem, Path} + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.Dataset +import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.SparkSession import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -157,4 +161,49 @@ private[sql] class CacheManager extends Logging { case _ => } } + + /** + * Invalidates the cache of any data that contains `resourcePath` in one or more + * `HadoopFsRelation` node(s) as part of its logical plan. + */ + private[sql] def invalidateCachedPath( + sparkSession: SparkSession, resourcePath: String): Unit = writeLock { + val (fs, qualifiedPath) = { + val path = new Path(resourcePath) + val fs = path.getFileSystem(sparkSession.sessionState.newHadoopConf()) + (fs, path.makeQualified(fs.getUri, fs.getWorkingDirectory)) + } + + cachedData.foreach { + case data if data.plan.find(lookupAndRefresh(_, fs, qualifiedPath)).isDefined => + val dataIndex = cachedData.indexWhere(cd => data.plan.sameResult(cd.plan)) + if (dataIndex >= 0) { + data.cachedRepresentation.cachedColumnBuffers.unpersist(blocking = true) + cachedData.remove(dataIndex) + } + sparkSession.sharedState.cacheManager.cacheQuery(Dataset.ofRows(sparkSession, data.plan)) + case _ => // Do Nothing + } + } + + /** + * Traverses a given `plan` and searches for the occurrences of `qualifiedPath` in the + * [[org.apache.spark.sql.execution.datasources.FileCatalog]] of any [[HadoopFsRelation]] nodes + * in the plan. If found, we refresh the metadata and return true. Otherwise, this method returns + * false. + */ + private def lookupAndRefresh(plan: LogicalPlan, fs: FileSystem, qualifiedPath: Path): Boolean = { + plan match { + case lr: LogicalRelation => lr.relation match { + case hr: HadoopFsRelation => + val invalidate = hr.location.paths + .map(_.makeQualified(fs.getUri, fs.getWorkingDirectory)) + .contains(qualifiedPath) + if (invalidate) hr.location.refresh() + invalidate + case _ => false + } + case _ => false + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index dc742220512bb..06d8f15dc3008 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -25,7 +25,6 @@ import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ @@ -209,6 +208,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { RefreshTable(visitTableIdentifier(ctx.tableIdentifier)) } + /** + * Create a [[RefreshTable]] logical plan. + */ + override def visitRefreshResource(ctx: RefreshResourceContext): LogicalPlan = withOrigin(ctx) { + val resourcePath = remainder(ctx.REFRESH.getSymbol).trim + RefreshResource(resourcePath) + } + /** * Create a [[CacheTableCommand]] logical plan. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index aa42eae986be0..31a2075d2ff99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -102,6 +102,15 @@ case class RefreshTable(tableIdent: TableIdentifier) } } +case class RefreshResource(path: String) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + sparkSession.catalog.refreshByPath(path) + Seq.empty[Row] + } +} + /** * Builds a map in which keys are case insensitive */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 70e17b10ac3cb..f42fd174b262b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -373,6 +373,16 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { } } + /** + * Refresh the cache entry and the associated metadata for all dataframes (if any), that contain + * the given data source path. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def refreshByPath(resourcePath: String): Unit = { + sparkSession.sharedState.cacheManager.invalidateCachedPath(sparkSession, resourcePath) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index ea57f71c50171..b4fd0ef6eda84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -68,6 +68,34 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext TableIdentifier("tmp"), ignoreIfNotExists = true) } + test("SPARK-15678: not use cache on overwrite") { + withTempDir { dir => + val path = dir.toString + spark.range(1000).write.mode("overwrite").parquet(path) + val df = spark.read.parquet(path).cache() + assert(df.count() == 1000) + spark.range(10).write.mode("overwrite").parquet(path) + assert(df.count() == 1000) + spark.catalog.refreshByPath(path) + assert(df.count() == 10) + assert(spark.read.parquet(path).count() == 10) + } + } + + test("SPARK-15678: not use cache on append") { + withTempDir { dir => + val path = dir.toString + spark.range(1000).write.mode("append").parquet(path) + val df = spark.read.parquet(path).cache() + assert(df.count() == 1000) + spark.range(10).write.mode("append").parquet(path) + assert(df.count() == 1000) + spark.catalog.refreshByPath(path) + assert(df.count() == 1010) + assert(spark.read.parquet(path).count() == 1010) + } + } + test("self-join") { // 4 rows, cells of column 1 of row 2 and row 4 are null val data = (1 to 4).map { i => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 52ba90f02c945..5121440f067bd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -206,6 +206,51 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { Utils.deleteRecursively(tempPath) } + test("SPARK-15678: REFRESH PATH") { + val tempPath: File = Utils.createTempDir() + tempPath.delete() + table("src").write.mode(SaveMode.Overwrite).parquet(tempPath.toString) + sql("DROP TABLE IF EXISTS refreshTable") + sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") + checkAnswer( + table("refreshTable"), + table("src").collect()) + // Cache the table. + sql("CACHE TABLE refreshTable") + assertCached(table("refreshTable")) + // Append new data. + table("src").write.mode(SaveMode.Append).parquet(tempPath.toString) + // We are still using the old data. + assertCached(table("refreshTable")) + checkAnswer( + table("refreshTable"), + table("src").collect()) + // Refresh the table. + sql(s"REFRESH ${tempPath.toString}") + // We are using the new data. + assertCached(table("refreshTable")) + checkAnswer( + table("refreshTable"), + table("src").union(table("src")).collect()) + + // Drop the table and create it again. + sql("DROP TABLE refreshTable") + sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + // Refresh the table. REFRESH command should not make a uncached + // table cached. + sql(s"REFRESH ${tempPath.toString}") + checkAnswer( + table("refreshTable"), + table("src").union(table("src")).collect()) + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + + sql("DROP TABLE refreshTable") + Utils.deleteRecursively(tempPath) + } + test("SPARK-11246 cache parquet table") { sql("CREATE TABLE cachedTable STORED AS PARQUET AS SELECT 1") From 91dffcabdecd4ab651024c027cf9716664084e1e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 10 Jun 2016 20:45:27 -0700 Subject: [PATCH 0595/1470] Revert "[SPARK-15639][SQL] Try to push down filter at RowGroups level for parquet reader" This reverts commit 7d6bd1196410563bd1fccc10e7bff6e75b5c9f22. --- .../expressions/namedExpressions.scala | 8 --- .../datasources/FileSourceStrategy.scala | 9 +-- .../parquet/ParquetFileFormat.scala | 61 +++++++++++++++++-- 3 files changed, 57 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index c06a1ea356093..306a99d5a37bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -292,14 +292,6 @@ case class AttributeReference( } } - def withMetadata(newMetadata: Metadata): AttributeReference = { - if (metadata == newMetadata) { - this - } else { - AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier, isGenerated) - } - } - override protected final def otherCopyArgs: Seq[AnyRef] = { exprId :: qualifier :: isGenerated :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 7fc842f83ecb7..13a86bfb3896f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -84,14 +84,7 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { logInfo(s"Pruning directories with: ${partitionKeyFilters.mkString(",")}") val dataColumns = - l.resolve(files.dataSchema, files.sparkSession.sessionState.analyzer.resolver).map { c => - files.dataSchema.find(_.name == c.name).map { f => - c match { - case a: AttributeReference => a.withMetadata(f.metadata) - case _ => c - } - }.getOrElse(c) - } + l.resolve(files.dataSchema, files.sparkSession.sessionState.analyzer.resolver) // Partition keys are not available in the statistics of the files. val dataFilters = normalizedFilters.filter(_.references.intersect(partitionSet).isEmpty) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index bc4a9de7a9378..3735c9496888b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -357,11 +357,6 @@ private[sql] class ParquetFileFormat val hadoopAttemptContext = new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId) - // Try to push down filters when filter push-down is enabled. - // Notice: This push-down is RowGroups level, not individual records. - pushed.foreach { - ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, _) - } val parquetReader = if (enableVectorizedReader) { val vectorizedReader = new VectorizedParquetRecordReader() vectorizedReader.initialize(split, hadoopAttemptContext) @@ -597,6 +592,62 @@ private[sql] object ParquetFileFormat extends Logging { } } + /** This closure sets various Parquet configurations at both driver side and executor side. */ + private[parquet] def initializeLocalJobFunc( + requiredColumns: Array[String], + filters: Array[Filter], + dataSchema: StructType, + parquetBlockSize: Long, + useMetadataCache: Boolean, + parquetFilterPushDown: Boolean, + assumeBinaryIsString: Boolean, + assumeInt96IsTimestamp: Boolean)(job: Job): Unit = { + val conf = job.getConfiguration + conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[CatalystReadSupport].getName) + + // Try to push down filters when filter push-down is enabled. + if (parquetFilterPushDown) { + filters + // Collects all converted Parquet filter predicates. Notice that not all predicates can be + // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` + // is used here. + .flatMap(ParquetFilters.createFilter(dataSchema, _)) + .reduceOption(FilterApi.and) + .foreach(ParquetInputFormat.setFilterPredicate(conf, _)) + } + + conf.set(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { + val requestedSchema = StructType(requiredColumns.map(dataSchema(_))) + CatalystSchemaConverter.checkFieldNames(requestedSchema).json + }) + + conf.set( + CatalystWriteSupport.SPARK_ROW_SCHEMA, + CatalystSchemaConverter.checkFieldNames(dataSchema).json) + + // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata + conf.setBoolean(SQLConf.PARQUET_CACHE_METADATA.key, useMetadataCache) + + // Sets flags for `CatalystSchemaConverter` + conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, assumeBinaryIsString) + conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, assumeInt96IsTimestamp) + + overrideMinSplitSize(parquetBlockSize, conf) + } + + /** This closure sets input paths at the driver side. */ + private[parquet] def initializeDriverSideJobFunc( + inputFiles: Array[FileStatus], + parquetBlockSize: Long)(job: Job): Unit = { + // We side the input paths at the driver side. + logInfo(s"Reading Parquet file(s) from ${inputFiles.map(_.getPath).mkString(", ")}") + if (inputFiles.nonEmpty) { + FileInputFormat.setInputPaths(job, inputFiles.map(_.getPath): _*) + } + + overrideMinSplitSize(parquetBlockSize, job.getConfiguration) + } + private[parquet] def readSchema( footers: Seq[Footer], sparkSession: SparkSession): Option[StructType] = { From f0fa0a8946fb4bdf0f4697a8e389f49e98422871 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 10 Jun 2016 21:12:06 -0700 Subject: [PATCH 0596/1470] [SPARK-15759] [SQL] Fallback to non-codegen when fail to compile generated code ## What changes were proposed in this pull request? In case of any bugs in whole-stage codegen, the generated code can't be compiled, we should fallback to non-codegen to make sure that query could run. The batch mode of new parquet reader depends on codegen, can't be easily switched to non-batch mode, so we still use codegen for batched scan (for parquet). Because it only support primitive types and the number of columns is less than spark.sql.codegen.maxFields (100), it should not fail. This could be configurable by `spark.sql.codegen.fallback` ## How was this patch tested? Manual test it with buggy operator, it worked well. Author: Davies Liu Closes #13501 from davies/codegen_fallback. (cherry picked from commit 7504bc73f20fe0e6546a019ed91c3fd3804287ba) Signed-off-by: Davies Liu --- .../org/apache/spark/sql/execution/ExistingRDD.scala | 5 ++++- .../spark/sql/execution/WholeStageCodegenExec.scala | 11 ++++++++++- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 11 ++++++++++- 3 files changed, 24 insertions(+), 3 deletions(-) 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 9ab98fd124a34..ee72a70cced1a 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 @@ -234,7 +234,10 @@ private[sql] case class BatchedDataSourceScanExec( "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) protected override def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException + // in the case of fallback, this batched scan should never fail because of: + // 1) only primitive types are supported + // 2) the number of columns should be smaller than spark.sql.codegen.maxFields + WholeStageCodegenExec(this).execute() } override def simpleString: String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index e0d8e35713291..ac4c3aae5f8ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoi import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * An interface for those physical operators that support codegen. @@ -339,12 +340,20 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co new CodeAndComment(CodeFormatter.stripExtraNewLines(source), ctx.getPlaceHolderToComments())) logDebug(s"\n${CodeFormatter.format(cleanedSource)}") - CodeGenerator.compile(cleanedSource) (ctx, cleanedSource) } override def doExecute(): RDD[InternalRow] = { val (ctx, cleanedSource) = doCodeGen() + // try to compile and fallback if it failed + try { + CodeGenerator.compile(cleanedSource) + } catch { + case e: Exception if !Utils.isTesting && sqlContext.conf.wholeStageFallback => + // We should already saw the error message + logWarning(s"Whole-stage codegen disabled for this plan:\n $treeString") + return child.execute() + } val references = ctx.references.toArray val durationMs = longMetric("pipelineTime") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 437e093825f6e..27b1fffe27a70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -433,7 +433,14 @@ object SQLConf { .doc("The maximum number of fields (including nested fields) that will be supported before" + " deactivating whole-stage codegen.") .intConf - .createWithDefault(200) + .createWithDefault(100) + + val WHOLESTAGE_FALLBACK = SQLConfigBuilder("spark.sql.codegen.fallback") + .internal() + .doc("When true, whole stage codegen could be temporary disabled for the part of query that" + + " fail to compile generated code") + .booleanConf + .createWithDefault(true) val MAX_CASES_BRANCHES = SQLConfigBuilder("spark.sql.codegen.maxCaseBranches") .internal() @@ -605,6 +612,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def wholeStageMaxNumFields: Int = getConf(WHOLESTAGE_MAX_NUM_FIELDS) + def wholeStageFallback: Boolean = getConf(WHOLESTAGE_FALLBACK) + def maxCaseBranchesForCodegen: Int = getConf(MAX_CASES_BRANCHES) def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED) From 4c29c55f22d57c5fbadd0b759155fbab4b07a70a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 11 Jun 2016 12:46:07 +0100 Subject: [PATCH 0597/1470] [SPARK-15879][DOCS][UI] Update logo in UI and docs to add "Apache" ## What changes were proposed in this pull request? Use new Spark logo including "Apache" (now, with crushed PNGs). Remove old unreferenced logo files. ## How was this patch tested? Manual check of generated HTML site and Spark UI. I searched for references to the deleted files to make sure they were not used. Author: Sean Owen Closes #13609 from srowen/SPARK-15879. (cherry picked from commit 3761330dd0151d7369d7fba4d4c344e9863990ef) Signed-off-by: Sean Owen --- .../spark/ui/static/spark-logo-77x50px-hd.png | Bin 3536 -> 4182 bytes .../org/apache/spark/ui/static/spark_logo.png | Bin 14233 -> 0 bytes docs/img/incubator-logo.png | Bin 11651 -> 0 bytes docs/img/spark-logo-100x40px.png | Bin 3635 -> 0 bytes docs/img/spark-logo-77x40px-hd.png | Bin 1904 -> 0 bytes docs/img/spark-logo-77x50px-hd.png | Bin 3536 -> 0 bytes docs/img/spark-logo-hd.png | Bin 13512 -> 16418 bytes 7 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 core/src/main/resources/org/apache/spark/ui/static/spark_logo.png delete mode 100644 docs/img/incubator-logo.png delete mode 100644 docs/img/spark-logo-100x40px.png delete mode 100644 docs/img/spark-logo-77x40px-hd.png delete mode 100644 docs/img/spark-logo-77x50px-hd.png diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png index 6c5f0993c43f0d9e1e1a04391d8a9157f0fdfba8..ffe255063035dc4c61d32e8989109d4cec3e0a9e 100644 GIT binary patch literal 4182 zcmZ8ic{tSH_rGK8yM}18WXkeU7)!Pgd6O8Cj6upyjHL|9mJw;}$%h(yB3ne(>}AWA zt+5l?_p+AtH{a*^{qy}k_c{06d(L^?d+s^UIXB$s=5;n^Br^bj?S?L7Oyvetl4Yc) z&SwHmv#5;LNz*_RfRb32LmN8k{+ykzG1bUV6aZo<0Q*!CaS;G-Gyu!i0I0kMfcHU4 z^(|GZg5FmDIs{JsGcxLO;;9m*2fF4209a1_hhY5^PYwa#FuMV1n%tjUv%vprLP3LHm%OHdsqEx3$;8%F2BfnP4eXpPRaJh@sj+j=AeCsXw z*6{F%hne~g^6DyP92T0i)s(pqapnl=Bq#5kI;V}l4?Q&d``+>C8_uow6%2Ry8HD`thdIacq0BHA{2QFC4c3MuxhfVe*qmrlY zeF+X$PlQOv?VQ1k)=z08%%4w=l+5DSoLPE=)N+Mv|GAaILwXtxKKHvE3PmlN%9SToT7DOZjy(3^ zefPyNWa72xTH`hSQsGFaDyJ54QY?9(zIwmj&e1l@Ch&!ZmVnmc$K81*f=W?AMUbSf z|MY6Of2-ZGJY(Q>;dsj?IMBGfEp^r2HCuW=6fuLtJBUs0*gCBd+M=2#uWcXj4WLD@ zDfcpTSy&p6EqJ7riO6vvSBg)pQ+!Q;_BkStc7_D4pSiH1+X2LcnCLZ-S-N!h5TRw9}Fnv$(d4uR6yE2P*`L62Gr* z6S{MECZ_*o*LuHv6w`0Wo>c1EmqV$0E5fx!cK;wJ-@^MWSL(8Tj^f1Desx-4L-DGz zzrvr1M>zE|7fQZ4FIsQZd1T=X11_mC*#jW;fa;mOY&nMZ-E&*_{A+tJLuaZWzx39xb`@lEc zSwId^6NEhaFsjVlbGqJO;6qnT{~{6?W1Ub$?fQ*$T5Hx)W%J9$z&h%T0Ti#+ zb0C}{MyvS?fZWkC8BsZEV07IHf=k+=*p3dnhw(BJL${cO=$anQUS>eHq4o)2FglZ`>D zUh@R;!uj7eAZSWy#XwbX=!GlNFe@_L3jx~pjw(^!ZMprjU`5e?Yy45i%zRn;${qhG zX-^^8*=5!y7qb`Djmr^>Psh`*2Ph_Y-yDvM$5h!Z-L7A3diY!uMB47+>r(V9g(bH^cklqT0F^i#y%$InO_SWQS-L&(|1MI~2araIt#zIvu!4t#W7>DxNhQ-{_Xs(tmP zbgN!wB2bUhb>bQkqH%>0$wBapl7fp*3^{3|$za2g{n#0sLJz-iT_C=qQ97rjsvGhCcg!k<7D4O9bSDcfXxatOZB^fev4BOB1ZH;d-i(;mKRGzA5*iD?7KDpH}}S+y6JNB^D~k?8<*=Ev>MP| zNAJr$;QxwqGbh){C$y&onP}~8IjxU**XBw^4Yl0k^B(xgk@yXBD8K$XvQXmv^4Z`G ze$zWiuG$u&@guQ}`r+dFga-{I%khbcx;I5d?kZgFu-@zlJTyD+^6mitY{5a$VSWbo zj8w+MobBbUsV04-X1i6YSrl}*b6#Q6g_Yw2?MGzjv;2M0kZ*`{rWdR*j5K~%BEeM0tDt(b1hXHA~DnSxq{H z6*uqs=^C{V#Q-<+vt<9X(Up58M3ALTxIWwk3Zc!x7htCE7;I^G{14B+kKX#j0<7{7`6R(3>6h z1+Ew;#lf`o?$op?h{^wgqyCmBGc2OwUP_4bjBd{*z98uG5M#r2K#dGecZlFfq*g=r8$RU$o_1Wu*UyjH;a@@S-xZ3@1unJ^9R1x}8U5H&9Hy+>o&ICjLYC~BTM#KJ`ZM2Ek+=tPNokE;@No3im*x4M0vZc)15?E{ z$KvVI$9FtpGMN3A#+w<3rdy+Vf?Aqu?)jEhq*n6u78I6qXivhpf@K_C0s{|+mb|L# z9rY<((-o4TOIJ|)kF>&AyJxprSXiF*)6IEi-=H139{QQDl=qyI&Sp&h;m5GzJ#8;P z98Lr)&fU|w5P32@Tos!W(Mi%hA3N4>5;)h`;@*Kb;a61d2_0E=&hK4dU7UlABoaFEOqMZiZx~+xJ&U5j)3v^i z*Irct3$PdK>+A8+(`6(gBVHR_ZgxrC>Wb+s8Ta_!yKr(WbLH6>Ks?8oL|#gMqv~ed zLQQfN_Gx%W%vn`w0>7LgQ>C;3m)kVlMc2GW;P8@j&o9 ztKinD%p8-Q*f@AIIQCUZ72J9jPY>MO+zLV(zZtbue^uh_n@mkjJ)jvbaFORy&-<#D zKmW*o`_WLh9vY_}yggBn1t+-+dY#;NN89>fnwcG}Bc$E2-Jj1lNYM@JKB_7%9a)2Qc$&$OcQR+`RM=0u? z6GpDX*CGOD5ZmKHk)VP60a#6Kod4+yV}*})Ozq~s-N2Jbl&QuQ$TH8=NuC9l0Mh9 z4t+Kqizgvr%e>1Be2CK((b3TjCJEQy4zNc}N2>ND@0;Hsn54dG{dHr|f>slt)I}Jp z8T{{6=jt)Y3{+w%Of1mBONsUKvad{W>Zn3k@72d?d&T)u)?o#D%Ge`OE+ytKorMW3 z1+Y7ihj9OGPz8NAi=kH<79&ZV>U8yq5;_x%TSQ0A1TqNCSj zh!%zFs-|HqeW;LpR8JDT3Hj&c3xAQO#`h&PVy!u%-QE*z0S%j3GT&qdM_uTiT#$<~ zq03(|!X9x>k+Y58tolxpTsUJ!Gx8rgy2ElsEY}wKP8BWd4|bH%v7D|oPpvYM;8E*I z?(Vu8Gs`BI&Np~R7|OzA;lZSs-|5Cnym8Ypc1rmiS3eFOJt$afQVq~o>hng;*SXwL zPq#{Iw?8J6gLpcQ-e-sFH=q zuh;A_J6}VVeMq8GxtX7g#4nK!@hA7sYG4I%6m08u;%%=&QeDel$UoV}&gMOX*WNrx zE%y{AJ}TH6bd|~dY0yYV49wh>8uG@;+Of*vypJ-B#9fD7Oer#uNp}j~a%f^XM6bl` zu$)x)<8^Tif!|3_ORLEn&cMzMnoM?d@oSIlN`}8qs=5)O^9`P<0s*323H*U$Xs z!TM#EmTt8#tC^{!7CF7khU8^Yifq<&LR^efW`XlIKtxJdrKNvA>f5#GAA`vv|tu zA|^@RIrj0mvV^K6A5eIZ@~2iWyWbG91XPcFKkOd6o$XhbG|v03UW6ZoxfoGCO0)(h zKK|w1=yZ&9h&q#r95jRee6=|EjB{HShDmacn5h0CH$E6dB>{=Bz4ES%X2xa5PK z^TzUF%cc?Q%Ll#B)wkbp1TJa`?NgVBJToVtC8`#)m-LO=mVtnp z0a>(+f+Si&QU+}zE2|=-pdv4K5iO&FMjPL}AMjrQHxGMfN8kSepf9V){XYQiQHP`g Ozzyx2P|-E(r~e1s*2RPX delta 3535 zcmV;=4KVW7AkZ6-BYy-WX+uL$Nkc;*P;zf(X>4Tx05}naRo`#hR1`jmZ&IWdKOk5~ zhl<6oRa0BJ8yc;~21%2p?MfD<>DVeH9(p*dx19w`~g7O0}n_%Aq@s% zd)fBDv`JHkDym6Hd+5XuAtvnwRpGmKVkc9?T=n|PIo~$m6N9yl%7qB(zDb{1JtYOPA!qk%@Qub75C2T;T?(y%k7cw{!aEIzj}+@(TO5Y zz-|>CeHr)+QOwk9wo4Sh2mAPGw+7tv9PraN1Olr9mwlWPc+tlvJx(D%369Vz>lLjC zyaqhH*k&)cVSir(7|4_1Rc;%)B4&}Zt&(AK&*3kRaknx54=$T^^z2=OSo@r%xnsBw ziQX&rRKAHn*R}K<@Jp~?uenKKe+BrD`qF#?cpp*psgiwk-seYOU#%402HpjG(Qu~= zP5WwfF?$>svC;J<>zK$RIM;P&60@WifbUlLwAf?lk$1u(mFRYq89Q?S8HyrsX^MVnpsf`{W6(5UXREr zZM&7ilpo6ux^|%%%N9z;(dj0Ct!8>>5!%=VyPg-?ke}Vs>~sbgdDuU^Ixpr19NS&g zQew8Fz<)>RL_{M_7A<1csY*MfP@XcB#Jxr~JS8&7goVS)VKE|4(h_Xlc{z{c$ApZs z7riZ_QKdV_uW-M~u~8PF3UX~a6)MwxDE0Ha zPjyrlI!;jX{6Kvuh*8ej?;85ekN$?5uuCiSBY#iOF*IB46Cei@XmvNzKBsGIKkN=X z$_Bgm&h5TyYjpf^^Nrn$ZO1mSZpOC4_}H(pE&Od#0y|)Oik6|u@OHupcq1J>TmjQ1 z4eM{2+Oeah>iL1tBPPK=$Ue-{x#7^N{-XY=Zm3_V7u4T7YJ?d(rX+nTeJlMa zeSar?Fa1o4bXmG0{UlwLzL72kIlJ$@pC#CPuU}yk-HKgk)oY(MCsR(!d*m18l>EH> zvOF8G6uDm>lc(gp@azfp%?$k=y%=-~vuaJXdoA$<7&H$h2Au}i##F_Gt8^Aw+nR z00009a7bBm000XU000XU0RWnu7ytkXQb|NXRA>e5S>TrFG7kt8zXBa_3XO(k23hd4eJQwp6xxdw@L` zjWZoraI3*yw|8C)=E~9l*fS3pQL?6N2WIcQfKh*LKnov~D%fv;pDMN*@qZQXY;tZu zgLzc4;IP)7+uG(PrsbTuF(pq3!dmHZu-|>01uA)rg(e??o;NK!FDb{{2c-jJZG|9M z<93{}b&y52QF<=#R&XhVY=n(cUe|? zA#=HeoVi`zG%oc;fY?+-Y^qvgAH|z@E(Rj_uG{|S!%{PfZ#`pN8$SFTecEObv1?#s z>qP&a8@dLQ;Y$Zv6vV66*&5P`k?YwQGTa48>lxYC;VR6w;;vZ>ARVX2PzR~H40 z(fs;pcjB_RthZ<29Dfp7YXa}Og2EX!40TPbz&iJJTj(?Mw0lcS_?+VWNfzGqJBKGH zi~+xy;)C|Uo?t;~Ym+sO+;d=dC6J3CUGwpd%i|#CBjLUAU3WEZN5oW1ySser@!dO1 zCoX3Wht3}G_u<+?_uieQ?UkE1r!6o&{{OwrK+Y!+U1`c>MSr9el^_0ly?XD?t&J%< zp6!D@fQohS?8^EIG65O^0a%B|*)4Pq;sGK8dCbP?wS{MC{;%0)vWnQ#V5aY^dxEvZ z-oQZqZKu@w%gOJzoJt|4t=APyL(8l"~8-1F|8TL8pAQWJ{l1_s-C@#=UYNT&6E zjkcs9fBW|rW`F-*zpG&lWv&ANZoYW+W=Gbvyu2Lb&7ktbH~%sCiqVSF55pTYN_f30 z->N$0NLbn*Z{MzJzyJ*5f!X0tCSvXR)_t#eza^TR51w{+_J8`ah;@}^Z91`~N9Ek$ znQ`XN$nPE@ZRzqY3c$99d=;pI8+BVX@o7#KYll3QxPKsyHE2!z;Wk>Q0D!d@-k;w5 z$I+X1te7IWpg%3&s-)QGYO?>Q#!i57Ur*XMg^+EIi}-v1l7N=2jhgoQzjY|mt9)Ie!aQ@NUONSgEdKLNc@T2v`YezdI z@9yoTJ+6~DqAh zRW0zF09cwggANzppRS0-P3`MUbMeFJCQ~FCi*tkS0N4%T!8)5SUmrK(HVFS|yMQkv zLw~!5Lo8E9K=sQY0U8y2x-m9gvd}3TIJnLMPuDBSA1uV?5J(*pV!M`>!~GiKs7Kkidh<9suox8TpkLeTTC;69D6AiI_%)H+{Gx7ss{~g6Oa4hygc#Br@v4 zZC2BOQ68CpD7t{(n>>U5mvqBPgLHO3 z5p#$3hVG+j9MC5{1v5e81Y879Ur$CF+3^||t@+TKeR6j6;E(!*Q5{4z!dem(;ZLhQ zz(`6(4$yl-OAU7ogdWwdrP@K1c7G_=$CFb*;}Cvvqqkf z{g`)tN~IO-+)0lBHx4v!@qNFM1lLySHCpFmX%CvucaXo}HtdqtouSay?Q||j8D0x( zT1V5oAsL5+hfgmPR+-M+MK`XKt|kDLMV_@L$O4fAtTw|3UpT_^wKLmMsDDYY&uZkj zDc+VQN-RGtE^JFD0ti61q|KU}PHz&RY-F8UMoJM%x`ClWj>m%j6|XyL#(y~``P$XQ zTJ%9+?SGFuftUuwC^;XyDaX*4n;v5pT_6Cd1=NwNm3caN4pq@PPy>Vu?@n6)xu$rY z#BD}Ptua1f%s*gE^tFK(Mt^^D*2O1{JZcxWSbRXt*D!AldDy;JA6ai1fIPuidYsPa zU2Ef^;i_TW0Ku3$L`Nw)X3|Zu{)*ajnFqt%PxQ~Kb1>Q645cy#-2gw(GH#k?4p~L> zh4y@GKC3URrJ--g=o3vx`bKr&liiVZm&uF5LlDV%Y5Y%UshP+M&3}3#9Z|w;H?>=% zgHVby$ecG>BzrS-cxFgEN1IN^Jj!PEg)9WvtI927b>?Oz<-jyQEs>5hS!K`Jw_9X!k0(#H*K&*ptRjCry)(P8c9 zj~MKuWu%9;u$gmx=-HykPOtlL)RgwLdg2*HMVhL1CEYpwJb|^8qRnfyOHIRaC(AUz zsF{AA%o||DAQLnSrWM|dN;WedDZNWYo*OjmzAOkJZcM}qkw{(UIy0tS<_-T6d%K9G z{Sxzj)?-|Oc7fx&0__U43m|ZTE?1Uz1=9>j diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png b/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png deleted file mode 100644 index 4b187347792a64a6aad514826227866ff5fb27a9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 14233 zcmb_@b95!!)^FTNI<}p3l8$Y3YCv3FDs+*1z9WRV(=6F5DN(Itx!?YXu`!Q>w}Vr$vc(A$U9u1Cfng+ z4=Td_w0u}!EC=*rhk5}OSOiBD=OKoN(g!Zyb=FEK1yC1DhyxFPBN9`bLaJ=GIO=~< z`mK5UG|aiqxzD|?y)4YDV_KJttr8etQl2(W78g^15t}X(JXJFz1wK=9C`%hPJc>fs zg?>MFyonzSJ$xE<_9lBr+VLx7Wip6FuG0z~-wpc#PGMxmyj}{e>CHQvQE z8w6MhSb|$Bit}Xu7hYLnewgqH>SOdW=vFA6)&(Q_20a~ctbv$K6)W6k5~HY^vS>ZK z@lE;&+XlvWM7k8f&54%JkLT39CWA)AptY}e%I!h?^L00P7gCiv6-^R)9*_?aPgAa7 z&V&j#MbT~!Yx`n4I%Q~UP-~FDNt=StGS6lJMj#$m)#NStapC|e$2`Cor6w2C^Zffm z?SjEURN;V<>4x4$m!w;9l1{Si_dVa$;TrX%5irt-pB)9GJD#6B`D@Es3n53RxY(Jq zF|MRk&8^Cd1eeN_)f7IzG9__qM+{I|w#0SE5)LCYJ7!@IV@UQ*9#VLe8T~RwURr=` zNyl7@%vhZ|ae%^dS>SKwqThqUTh^LU)UVI!6gv)?t_g8Ry*3gAxH;gkWKCdR|G&%gS^@iHSlHyWTPTtX@*90j=7ykg?$W+tjMDbhp zTai_vP9f&ZI=7iGHg`Um8@1clS4y`pZuGCx!47K8B(G#zaqUrbken^&Z5iYXzzQ8b z)YV`ulUef&)-Ou3L%9b!eD$_Pkda|02vy$d2<~ z&W^*aL-LL>X?V@A24d@Mf%_Hpqin#~h^yIyfY6ctv4J}0@jHTmK#rR#|8V#rBh6)C zZAqv1(^}t<&e_uDLmC8x$C>Nn)Y8yFkI325!pfe@nU~~G39gUxKgjeXM1P7nnDdhS zkdY@6vbHlMVx?oGV$B_bl?vHNMnr6?@=clD1eUJ?@r2OBPWdM76*Iwxj2Ydd3l zMovyndIlzXCMMbs30iv>D+fJiS}S|fzZv--JHm$c26m=44yM*tM1R=T)3*jV@RE@H z;png5-^OWUV(nmUZ(?opmpd!_zsL3wCH-G)dPX`1`oFw<7~%Pg#U-rgU?^l~__6=V z$Hc(MM9aWN%fznC$jZgU%*Dva^A9WkBKr5y@X-G;#D5y{pZWO1!@tr0f1wyUoBktd z{}RO?oc|2q4=R@uK;PcbK-tvNP|nWO*wjkTf{&GnhyMS@`$qpdl?*{&Kp!N^njLZxS|LpspSpVCp zb}%*izohyP&VQ!*4=UG3f(<|B>mRd{?_*N@2koEY`;(_=Xm4!+_(NR9%G81H>7S|puA%l{HU63Uzn#9H27jT=24H9LC$xVW&>LD9e#|2W`;YuF z{wvmg8gLo>0WfR3{{hF}VEiXUlnibEmHI!}u`uP+GcdIK8^As;{t@^e5`V((Uv+ut z|I^X`8azMh<6lw#3p$;M+J5=-odLFjaihCE=<$=QIJ(`r4kqjiZc<#~mI+}Xn50^^m148A1 zdelJ_m}YYo0*3+gSuU>;7eV3z3wj=cGhSjn{qyUf2@NdEVP!UeCR|6-1}7B-f?L&? zHk8NvRD}6Hk&7UT@FOoH*s{P9ZQ+-P&l^k&%gk)53DhepX>Bc6m!8FWRTz0n8Qx!= zkX@&r!Mr3cf)0Z`bFq}$?2xGwMghwD=ErBnixoqMU~?4v4sQ#$hXbg&lO51__^PmLE?$&n{{_P4Xm^0#y@j&i zW)t1XoY|GsYv`Wfo8=fz4$A5ozH$(?(s*)&lDb&(BxGs0jVZm)Np@KtL3!kvz}dH0w<`?w)zMgc>1pJ+1t>DaK$a4ROSS96M%kyzvJ-IoeroSzz@hJz50xrTN1YR*4a?B>B!Gwrbyniza zYyp53@Zw_@)kV#40GHzjwTO5MS#ItF(&H-w%a$+qd=RtT^+uf;~%yCbZ;%NjY-&pn>T&*xkPjIR$2}HL~b*E&UjQyM}UNY#2K>6ynk% z5LhrfYjrL`+20jw5U?&Ns)ppZ;zhXi;?&DKK=yXiGxjR8m!8Wj6J9&b z$fiSgpglF>91g=u`<85dLhGOHT+-@?Db`i(AQnanuNZ z{Q|KEcFOV8qi#O5u?&Qp?&FwmxLk!SwPY|pZ{8&;sVj$~eKZgKIf;J6_~3x+?fC=D zXD54@_$W|Go_(LR7Q+=>)r4t!<_ardkMN7&U4tA0BO2N#U)wzs7deb%U)DS~cIStxiVw)7!0!DFw|Zzey5veT>iS zTgC(5MTkRgV9Ew`u#Di2glbM*yc<2c&q*;xGv2qD4cv9`@(Th_{@XNHZJ9VUY1~Q0 z#wcnw%rlT;0e-L$YE?N@O9)%`S{oxQn;q6buAe*=mKn{Sb#4Twi)hhsse_AUFfw_) zHmYconFx9fs8}=V?h;8CpAlGZS(y3K0k{ANus+QpZ2Vax0WuYW>1o+-_NXg$xXy0i zU9~~Uu^_>OVV*+eA`L10-!>Yh3a_q$zW3*xUquL;5B_>U-~_$gUlGUUiv?tvt@zg} ziZm-{9fr4^3_rw@Wm7WFcqxj!cEs{j8O(D>&Hizd6}U|1h#F#_};~y2b)!aXDYgDb)sP2p}{~y z!3hU+d4mV;CBx2Gs)sbshD=ntvL%1Fg;H%DS(xTS-bCX(Vx#)z=+J=mFrCk|0~$XV zx~~|**qOzC%kpGL0!SyG!Z(hR`@KQve??`>qN<{g~C%jH1Ba z8ZbAzicpi;FjfU3bNk zVb;kloAb#m-nx~4YGNaK>E1_j+o*z2(a691+0y?}rxHXu1i=6dg0q^ce#sdaS&oZ5 zEOcppdv)hMc}aMuMi~=CjVAYvZmy7|0hAK*Iu>&&t4B?NYtRHhwii;K&Z@)K2qnv8t@V7_120Kzp{?l41EmFWpYF?%aE@9+-Ds{ZutK?&U*`?SpRi5*!6 z^8_*#v6`3?cm18NSmTLW6BJE?_2{oGYe`4T-Yddjm@%sWzdnF2+m(cSydvW79xx0&DhzL#xf?V4k&6Lkjn_JI>7AxSLCeX>o{K zLJ0>CRFlw97G+R?2(MAsOYLX63hd4I23DloGuvU5cSHl z%&N6^9IfI?UNQHQbprW1n|Rf`GvgvFVx5q>qgyaO}SrQC*cC4b_ z?2LYew~;XbkmQGYK$bta~HP6}+=V%+||fGx3GbY_i6 zbIe!|xs-E@w7l*Sn?ujL!?)?cf>G7mm3qqf)Cn5r*#n8Cigr}c;k5;SjKuSz>kF5O z%1j*h48d#e1QzJ4@`ZtXZAgCJqB^fWStc!hRu9QO%*O<#|>$wts&Zx%X&Xc z$K!cT_USfH>$*f`+B6!ZHZKBlg%C=Y0I-XT6uP&EG2{LS+M|XwExhcn?~{t6E?#YV zG?(pi%QG{c?43-0Jt;3bxOdzG(hmI#yH?;NbEpxHXMe0m;~gU-jxSS#CRsz%B=Hpn zbj%sYkCB-x+YgAkdZ`9|$~_X}_9be${6GtVMn&6aC;(2;AUQc%UQI11IvOQ5K0dpu z$_pxo@a6S&^mUA(Kx@^*gTEK+imVa{g@Y|T%kV4&=I5T%$pK5NPOd4y7SiDkA^(Gg zA9)423Qy*9yTIrPqI`pwtuh?I_7g-WKU1EjbP{`IFW?SZskXg-->pv zEXnE*^eVpU&KQw`^=o_Y*jowMOT$uhJW=cG8`x|Gp@s$@S4wPhR6U?X@$9U+AXK{G z80>ROOx*Ul;#~SfJ&YGYOAOCQe=F|4kumOYgsG${3=W39JRptOSbIJbGy4XIf}*A; zIUT=eWF@!-P4p;^HjNa=m|Vk%JL&+|&i5#T=Q1-X>UVWV=re%G9La+}KhSu6g#KB9 zd7GDoHg!a0^VgEViAH$W+<|)_@4L2eC>LHyHT(AB7s4JTIo>c+9ak^g0Q|nUcM+u@ zdsNm@psD+sW)t#X&lQ+3JT9%3dEPJ|Y?}sFyKe>GZThq<1qMtJNj6V{PPn`zA0672 z9C3bpAA(kHc-h8iz1RxD8jFO}vd%razRlt7jc{j2)&)z-+lWL`R#nMru18Q>2NLFp z;%_i4Gj&qDygx3KPix?qDU9~+A0=gEyWjOgw(XNC#^P@<7o=m5aIW=Hl!}fPi{*wr!mb->NFbla7?_lxa#NQOuLmdE{h_d zEOOlu9IF;m4ie~73P*}RseM3(3)-YQ6``S(xhDv@A!(Ki={r*Wf#+?AJppiJKd|Kc zf}x4{>9oAwg#5SqaIHJzR5iyAdWc?X^*O=7z(*-RXZO5S2K49Ld%7N%HH0=h z*c#~d)*?ne-@^=Kynqa~zsCwZV@HQ(7c#CyaO^SF#@nA0*k$>E6fgZ(^eoj+TxC@DLuJ1g>bHe#i;_(EdBlC_js5u@zP8~TVZgzfDQr>Nt zpcB54sq=lUrn=f>Vf|ClSi}1(0s<~OM#Jkhf@BG}3~O&{86cK@JJ#9!&_S-R^vwEe z*py|May;K0pt{YqUgg{}E(^Fk+)~>L7gsAJ1h$|8hglcWEfKULX3};!(m&@kH$8 zrSL#GV}O#y^`4{{k&gfw{9XyBn3f+u2Olbaln*GxW_Jx6iG&&A5A}T@r{j|mJVU0L zUTpR$QQ8+WvaRBh3Qs; znOsb6a&kXvlJ>Aojp6B^jdAqPp+icG(E};$`_(?dRE&)Q5on?k$QeixGc(GPVGK$Q zceg8&cNB80gZ-#{H0iYyV!NjKO%>I<0vK`evDtz2AcF%3ViB+Os2LDoQm!gZPY14p z(Y=jF#b1lS)uU3A!&BkS6%sQdVmX?BkwD3hqIZ_H`MklbP>q)FW6(P(;ooDq8?tFa zD?lOB&G}+ru4Hghl zQxAzFv93ha2u-#>hyT9V?12ds=t0w}*_du3Umko)lY<(~U`J#&at&*;-zPFw3Ri$P z`i6w(eNkz1bFvsCJI}uvNE)I0Mmn<7#f_M99v_Z@RUAM;@YUq=SRy0%y4y4DcRC&a z;34WIA2?rwf&gD%l4QPIWKz+M3)1l4U7rZOb92dJ%C4$fO?ATC`e`B?T$nLvGzA!F z7!^hCT@Qax(Wt#}-nU-E$+0Y#=XWSiNpx1v-^9u^B3TBUh?! zs%&b-FjuYUb59CLKGiRS{Ms1h4ZaxgSecRfsuz3GwQ@zyYT{{7y^9G6)>4w!2vQd2 z+9eLcUANu1^eAG2y+IO=+aoU#OVC!sU^?{`i_HRyuHz-RKFScAPV}IrOt#_PjI-r& zCr0k#$-H7~TaJ&_@K!un=!Y-_Nso|Bl74;S+w>KA;@d48kHbBFl3F&Sqka=-|As3{9(a5OiUJF=Xj5H6AIVD`~>js_UGHeYH zdw(BS^eBko8y;<^Wzdd!7sm~=WlVd$JQzf5UDLPMHjFYRQz%jS2u}m=%y3O` z#Wdp3!SWFrZNcuT=hbhI%7PcG)`z*48mHC%YM z3KRfS#xI#%8xob^g+E$PC_bYuZih&f9we_GU$~}~pgIRGE*@V#Y)5QY^%E3uM05vc z`A8F7`KgJ1m0ONtOr;{v&rQzDS+?!~zoy(IaJ>+O!-|}Z7kvcAxP4bVdc9xoS#?_V z+x_}oU$;X#x61OnOXm|AL#~Uuo=%h;VYlboUdH~g{+07~qk^k8 z5KVt1a6?kKfs?Rdy4SDWP3eH0C^76kHUH)P<@rg+o2}jun^RWR?PKh9eL^rvIP=U& z`t?LCO(i7}s4G8A5oMLzCurgXjT4RhWnEacr~WBE6m~H;X628K@C>UZ`TaB zJ2mN44&)@>>B{p;;==7?R~7-59+gk3s>(JKHVKRc&(w0c`@(~()5sYsp`Ynn2WGZM zn8%ssmpxwZr{&-a{ozg!`R5ecDP~nHnHUcs5-G#kqQq+i7GtPKi%LQa=ZJHxJ%q*j z6CqzeabW*C%#*-k{E3t{PMhDxUpCtJ{<`jAv#=?K1(TbusO(3d^UVhhyLp|%YON6o z;HskVdPF!;X~vg4KVnEZZia3?fXaR2ljSB>Mv@(mjnKv zmD|p!CU`kqAXT+hbUf#H((c!mc;Di(jBgfEKGu}+v>L(JW1OGA9{#}R^rmmhT-D1V zX4yuA-?-t<;a*Sep5)B9OC4EqZv&DfMhgoR78n0^e}?O+r7ze%EXe3yby^>lEpWb% zwTvjp8fi(KSNhsYp&%7JMTRJpwkyUn668NY=hc#g!2Dd-gK6Rv}2%`eYps5tHx z_*3es346@zQz)nD-|qAtr18M~`-trhOs-K@{6}#oDudq2iWML1>_r*Y3(%vNG$L?q zu88+reb%|*XwF~yBM2jg>08j`_^(T`PiJfSOOZ>%;~`phqNOgThUpDP5`Cn7Ec=&S zX$r}ftlM@mXh;kFk(E_q(FAL^gAd0Sh4W=w?{IKN#BAbY1(Ey`TUm=c9*v;Vfk!a0 z2|J`zb(mc$);!Nbi;sxBHw>p4IgE}c3v9ZJcI!Z=V$itwEC{A5?J406&aFz@#xOJ2 zhrmtIU@R43syfG0b=QZkXXm@7{UWAktRgDYVr+d005e6$B@`3_UZDKb?hGB^}TvJw4GWZHPBRV`qme*(8p93Y@(7cbPc*~QTa)A95t>XsSZ@NB}X%}D9=@lM} zg=m^4##Dgj{m@uC{e?@G%hX$XU-YKI)X)A#kYGteaLGd-W072fIS74T08Hq4=Y zxeeoo0lWpx_WU;QZd>WVoj^S+Ng{Uc6?|Nb4FYznQsC1K{i-tm-XpD>ZT$fO@Ivq` zw^06mLJ(XUUm0`PX&aus@%ZRy#DH2%c<|sXW^y`hu_OQ}{+*v)aEst~#MHX|IIk5> zFU4wnJEiSP1kX#P1UFephiOT=DxaRK`&lZdWkgXc_gUj=Zr+d`BF@D8YKvGgu)|HX zaeq9+cnW#ON)pdKnN!c4ao%X`Wlg4|MycX0DIe|MdW*72mpc%Vad*O&lB{z@*a*{B+)e3z3_&WFGZv+8=<$dk4KF zFs`tLXEqZsRWkBDT+x1~$z*5X^#UYMNm`$}7+aI_c2SyYrp4Gv`rQua>WVGJL6NFD zf;_@M%iLTzX(UjD5|-DAv-jh^HdLs3$?kHczcwz11~Op{Ds<4k%$A zr(UV+W!eEd^P4E4Sl-{;$PYBt@U@lrQ@>oIr`z44szm5lQB=^`F3I?BlKN`4W&yiV zXQ~sXIDJF{X~X4H)eG0-ya6@iobx8Id6qHCL#{xwpx;3WBm|)jQ`XxJOKUzF9Uar1 zigSuUvyr#wi0<6^HQid1tSWr4R*U3cI6apRN0yp3zHjlznb;YSdFiVC?l+sCD5*=> z?t;rzknw^{+ib==-u%{THF6bcrybSOW9dMnfs@l$>$iJ_kkjFqiV zs+K=c*xa-DSY=mK+3$X-1+WF(Z_Lwy#yh>TJ_Aq7iUy2o4GeOaL+m>@3_@45U;Q^q zDvT0~T$E737kM-_8?5m2ClRMx#Dc4^UpOQ^#@2kz%bPIN)WG3;e(R>JwMeG{bqIN% zKz`nwcyT$hS!3afn@Tpe>K%S8lSH+N*9!(^6@NtJ_B0A1_Np1DEkaSpHLzCSGciA7 zNi^$?a*?X3LzkMA>!ZtanL&M_E=cP_>7Bd2 zKFY7D!5?bc-GL)I^@YpfbHciBt&s9ROGFz(uVfxYV)+UtebOzK^t`6OJGY=l$TRu_ z-rrsYJMstnWrEPK@ch;5WA36|B#TQ*ThTz%0ZLw?HTe$@!uIP|qc0b(5-OMsttr(- zVHIXN%-eaec5Kkvd(B49n=CKZ&F&=b4|-XfX-l;QrU{@@@CrS&F|r(&!mX@^uxu|Q z_de6*HWr`gkmdazCgU?{y5@D<3=`=^UuwXW(~BWyr5#;hq|PDjtOh86WU*QMCOaWQkA&}3;;Rh} z`gw1lVNw*Aln9DuxF2Ag(!@A3On;BCOe@rOoS;m5k?NiCtYWn7`#w|G-A$sbJc}Ka zKXLN(1m)kBI|<^{t=+n8v`n;bGL{cti)ffo_bp#g!4MoXp=CQ-ioOTApCAB`YBJzK z!LPoTN$k9zR^*)Y5}%9KXrW!)bC&Ywuy~=tU$s`D%yhuJ%&d|BusBK z1#}rO_3cfhSa1UJLzgM(mVa!k9=38zcW_G{&sX#xWqh9&8Z!|GQ_1NdOb}Fz#y4mR zc4vz^qEbvsjxuGMP*c>{IjQ~mNUReJuSt`A#41c&@H}fM!tyZf{GJd=^4!WBA0MYr z`;Mf9}*a{h0#=~7?lVFw*Z053=mSHz|x;do4owbxxDBvs_D$df6 z-eJZ~!4~;m52XSY#?D!vu0cd$pzf;T6Jb9S zmk4>??=N-yv{VH5>rV76$Mr8Z=__sPX<+dOf`zOqv*d z?2=h_^GF5IU*8=?S>L+_BM5lm?WLroDxU&GYC}>dsZ^5Uu|k^c_lPrGLgr7e?~G!& z79Br&-X~zB|$D!7`qz816g$A#;*E!8dKW2AZb(-k8Rf7?r9GO z`(XON*?7tYSTtBLT|8sbP73eXVJll92&OITXB4Gu=t29KOJ((j~cdDhl)X0Hz zBou(%pKVsgGu23N|IrHLdc>s1KvWtcAzSWCekCeIWo;y@Vz(L)m1bH0JoF2uH2cT+ zw`xz7zQnqL+#hT2hMp#S#jX~wqe3rJtVMgSbi08!8ib27gF%<2EzAlN=afsnu*q8dZW58)G%+I@eG=*lWV@7*YPx>klz@^|Kjb8}gAGrGnS>ru_VfbJ}M)vk@d`Wl4 zlQIdE;wn_4Va|oMGl}iE=)Mkwxi5gV3$~w00ei!%um^>=jDphvgo|%P&osT(du&{f z^D)m`tV!w(SI)b?T#D6%a|ERzGV`ari{%u~#f1}v4BuKRjh2Br4D8*g=Hh@U^Wl|s>lXZB zlY!yd30ALioeb^9<1y|-?}sk5X@btv;kDV0hR>z-ebK}f(#gI7gzgY+&p&O?Hw0=D zkM_j~Cz@3nANO!&ein$Duo@e}9;&?D0#GZu`jzrYXnCg(EhMAE7E36*<$41*Nu_~k z3ABp)YA6pZV6V=Hu>NbUFIQ5jyZ}XN(cr#Dx+{ubD=VW3E5>8xzTr_!5ivR{~qR*ump`Sqnf7b(m ztccI~+44F+UD`2m>NDjPM#A@oE$`2C0_;z=WZh+9Mof3Ei>*CPy%MHq$1HoD;0#Zl zK5#y}oADX@&(`gX!>mczhvKKR;~;XGPcFSA?S%ShR!zI#x9O|#e|{QT5bD$K4*H_| zWK77=Zdw`SE6=IPz#f%F_yS~-<$C40A%^xg#Z$FWGg_R=OS~HdQWkLQb^FGQ+?AjY zs}f}pyg(GIR{QMElaRCeZ2*j1^YRP}vm&D*l)a&f$I_Etmpu1ZjezA{MS^JPbin6V*Wu`5i zijWo}H{pICEBD^gL&6;3xtC~0N%c`A1l^07L&yEo*TT$YN;W3wB=D!?U9>~D8i(%Q zdB~9XjiQ2#-(3NC`b!ikv=J%jwZIWIG5aA@Xx8Sq23L;g$R=F&4Zd&z>!Y-_G(jtH z8LlD_$j#ArJw}9`>MIIG4(ijhRVFoGw$?aPF7@#0d% zuG!y68~1Cv8hE3I=`y+avjy)+rMhntwNYrDCYHp#C120KwcAmWlg~Qu+{z;x*OPb& zZIo|uqi${YBN2uFu5M)w;?Xg5{ThKWB7}ue3QCq48&i;MI&^s?bLUc~_;OwTY}*6R zp42IheP$M-KSCkfI_7(c=eV$$e|m>~?&^5!65y3L7P*>xmMWH_VY>ZbJ%d{*F`t+8R%4#kFyrP4 z7i*ln@FrH6IVajOUtD>XG{+;p3U~AhW;o^sY0x|_Io@pS^JkvBtbU4V5fRT|_F`4g zv>cMrvmeOHc7w#}Q!h}&g*Zmy?Ug-2PZBZOY%QrFO`(~{WFPMs zI4a|O`GQ+|5s6W5IXE{syzYzn@%)YC5$8ITbz^vVD2y7;5G$~9N*qK)*rDh0ONK}U zpnWmD0-qc#9E+wXKPh2u2aA)8F(9}aMY21uuN-(26>v#LK07Bc+s^Aj;E@VbBw^(G zyy6!kE$qk7J&x5aU5e$dkmC; zUSCKht}sJz(@YUy#%3_5k}eGmuO8*d{}~CxbC0CML1rUbvh}Fge1sxW(pw2UtGXJm zN;Bp~k}j|yoyog%PcP%0HXJ|t2lYa@9sW4)zNSF~&&>vSBByzHQNmWW4=H|2c768P zZ?Z$tj8JWsjf!6E6=dGfV@;B>3l0BxJ8Eq;m9R=A(117EU%f!XyMBHX+%%6aKj)~L z3hRy}uU)vNRTRHV>r;6Adc?P1y>%SHxHwRV1Qu8IInGk!2g#NZWBmkN2-t1Uy43S* z>mJsy7-L$xAeq3^BBzQoj_sRi8xUgy^wdjqdrR4!J#8xPAGy`S(S0I#u0d6kD~Dnp zv7!i6Ni~WJK+hlPXwJErcRG&`*W*oxbt}2Pv?FAr@a~Job&vPReZ4Y23vWA1-(I#8 zL(2;0uvi#ABii+G+e`?E+N7?=ukm%1M5MVdhq5!XS*@4|pHGoXHO@6e9hLW0$_^t8 zhN2wwtGs{w$)9PxE-i`o@vX=Ba`^mQxzZQBegVNZQx?fw_~ZGF1ytEy)X@ES{OqE( zc;i}Hb5cthzfAF9UwIj_8gQWDPpm^SRz0|%b_08>$VNqcB%fI%GIxg)get)03qx|l zEn2b0`hMjWkTNr)x;@=&a_KZGs4i;qMGsxamH0jup0JdI4`G*xhu^w}dDa^&hR-)S z(jC^YqUa2Brm&nUs$S!>EOrnNSFF;|lUpN$+8qE2dU`d7b%!CE7{>PD6d3oL3)rV$POaHf z7E@QiP6y+OedfRh^G$X9+t#AC<2xxk4@hUTGTX-@Ep?337tu}cm77kF*Ys2lUNhM_ z9JEppPwax6xLBvyl+(6l*do6D>cdBo#LqQjF~^q~54IrqR&7Bm#pdz)-Tid9@{r4} z)-ixq=@F*&DewfKfvMpETFrZy-gd4fmNNF--nQqqGR557(Y9}qT*hmCzTA9xZ!N+N zL8F1KA;|hus+Luhvh7UbNqiM6y0PGtJU9%X{nd1O(QcuI;A=10F;Y%iEdH~Vj?TH` zBG*FBs4Y;}Dx<*7N~0g&rH{KK&25dl0-txCN_X+wGV`+b`kF%eHIJx-S(Q-WWFLsa zH`i27b=@Swr)V;h=gZ}c6O$JA=j78@@)vdf)4ivmff-SsTE1XAXqIEgq?H@0eM8LG zcFsyEvhT)fy5v#ahPCi(S+{!=Ibu#+B6X5;A|1vEC&;>L=sD0s+~=)55Nk~bN%8&M zbaeT&EWh)gd6^#~)bH`_*iWd9O!1}H^bFW1-R;Pd*sR~S@)uML0lPIo&Pp+_I$+9@ z%F@|RCxL?B8vITA)bglm>Zc(DIb??6Z@#^y2dCU9V m(vlOXdBth+x>N5Pq1TVIFSp-G694!ZMnXhZxcr;0_x}OF3;C1) diff --git a/docs/img/incubator-logo.png b/docs/img/incubator-logo.png deleted file mode 100644 index 33ca7f62272560f3da274086a7532bcdec7b76ac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 11651 zcmZX41yCJrvgpCx-QnQw?(Xg$4iMZO5`x3Q-Q8US!QC~uyL)g55N`gx_w8=go2qZ7 z=WFk&>FVwou$qc23L+sQ002Odmy^=?d!6{ZnBZamf!dW2006``8%aquc}YofHCLz4 zHujbPfE-Fqs+ShlD&EMpl5Y`Ayvj)z5;VHp2?TaAVj*Utm^FJCG)4nO6k~m1gj`cW zlm;{-c~oQsNV>sn`!CSWf@@#?MpHj1+;g}7;qNrnO(!g8bSXDU_fd z6OaMfKnMI8-a^ij8|DY?JX(TW-^R5Fe*322q?uIRJM;Y|l& zZ-Qx?RtvEKa+?t<6Gz45{&E^2moOVz5C98{^U^v>ID(z@MDaAFuD>M`45O_Myd8ui>Ao;RAwr0LEVYnCg$;MZaH!K}h9CSZH9K zKu~@{wGW_lh9C_Hlp;dz7sHN#ih_XqB~F7Lcu0;`4HcS?Zx6dJ#=Qdb7zDLL&Ih?B zj>iWz9Xzok(27C|pvQ+`?ZbZt(2&FTi(^M1+DQD4N3ek5mvA3Nx($~ofTu%M2+<&i zpbEUre^6uBg4LGv%ZEK-bB7dwG7f>w_xp*?h1eC^-Ak8_T-D!RPj3%5Z$?85Dccvd zgY8X_jfB^mwWH9A>KL}Z!}-hdnTILVw11y209de~4ogRbO>!88C^=M=s197P)xxNR z)DE{5_bB|W#;dNez_?(=g{v$*|C7fW%^h8Ua7vt2vQ8!~ziJllr^AWKz4AS}FY-Hy zWZ%iBYPKtU+|Lj+Fipgo5f!})=CS6}W{&1EY-II%)dEY%=aDG=!)EjKj1AN)Br7`h zyq*BTXp)g#=S)7vY+4rNkzue)Xe)ye=M~o#tRwUjf@=TyKFJNlvv(&TFRreacTaWK zX(w-&7G41wD@c$cx|h0|JQ?#5LjYkJo-zcXkYhIbL_Qi=im{Br2xcC18w?-xqQV@D zKaj~(RiLh+)=orCV2KZt%Nt{`P;6x74`iZn0$$U4C0!?MtJ7g?A_ilsVrcTH6Vh4H z$I<6umm{_yGQrgL6NVCo@`Ms0*5M#T(nKl^4Mny^;zY{g)X{fq0O>LqUKkoubQl-u za?}X52z5&t1nI)*tr*hico|H!{A+_W*|l$UcPp{WFSPlqIFzK-W6Ha=P(6e_$R3D2 zq;RTiOUE^oHJNpU%h&1zN=plQ44Gw^Wou+Ei!-%IN*pVvHD=UUwM7g4W#1>5HuWd< zJN1dInkU=Y&x^uZU?qdg)Qe{;T@3f5Xq4woIgQ<-K6e~ZGJj7kS^83QXdu`;ZPPWU zv-tS)$QsXvY__`&>Rj(k>ZE#MwaQJ}%1Bq0S53Fvu*6U2DJ8d^RFYM>&geRMt6W5T zTTxWI$LQnxAEZpe82O2O?Kxi?b65dbaahb?Vc2(Atgz2vvtch`HDPhl5i;&Y7DcW_ zq($uJU-#RFeTJ2XpOoQg+_AdHJd;F{;z~c4Hk1-6tt;nc*`zm2x=v+Il4fd7c1@*a zEN5&t_8RWi`wu$zk4t z)1u0e$TrJq-SKs8dWB`nG0He^)638575Vk+BO0+5aS+KZi2%u#fPsK3u^q7!zj`)g z_7a~5Usr|$cNVYT@kPh9)^Xc3QMO-pPg`qygpq-fjB$)Hma&~N+-c>#voFGX5Yh0| zh(nBF+Pak&o>#*w##4`wz0jO6p>XaDd)7c!lhbs4Spn_3TU^;f)KSVw^Sa`h!_L%W z?VW!=8_1d9A=*j_=`&$d#>?8smh6Gs#&JG|>b%;LdJ#<<~&y@W`fC?+@?L=vcA2VGI4VDdaFrG*{%&o*_IBA5-7;#Z?;JP?6(~}CZIK?PkN4jz4>>UHFm^~`-w@v_D%1+pwD?;4%m-@oW<|^XZ1l|9m@f`r zOK%h^4m}Rx44EqOQK$L4JeS<2jVX_1{Z=$q@Nyh>#I^5k8uFV6S^!mD*)Bbvu+bitgQR;!?*WlOmB(jOuRClfWyAI{S;9=-t`w{6;5lxEt#KGCY z+Iqo;$ss6^QW^uZ@lZu7ow@;CB@!DD1O@&;*{uS?y6NfE~)4H*y z9^Jm3+a*1hndi;1=Z?#Qa7W15?Ym$1CzX4$j-47Djk|@fAA5671ij#}U`xE(yz;gu zTAz;mAK$@nc4Qz@Twx7gtsA{#m(6S3o*xRv#kApXVYNcwFSno4Ip1#MsztkFv4u*6 zYP?H7s(0J=?`mfQX3tr1yYYSKzwqBw{!%>@wKL7?5qZShuVXYv8kX6L*i!0d_9=UD z{CHix_8T4;A^SG^#&*+il5|uCm3Nx=+CAv+!_)A_cx89U8|9R&jgaCYApysZ z08~x@oA?pps1Wfk9c(~O_5q2yB{SpdmynmbPb+ z(&ObTpj}jHd6mPi`7bp9AmlCZ7j&?6HzW6Uuy=G5@D>LC zi$dTp{LeNkko;dH?smdJJ!Lg=NheoJa&8uO77m~YA~`v^kgLUK0Szgc|6~7qCk(W9 zcXt+GW%cs%V)5c+adNd{W#{MTXJzAH<=|lcOTp~s0%Z$pvJ)b{00)f6M;MD)i5; zfSQfBrM;e%jf175+g}|b9PB(o|Dyl@F#pl`U!3~?;pF1`Z^r*Z{2!wb>p%AYmp}g- ztbgtPjg|ilmt9;;UR+E+=Gx2ibV1Cq zb0x4y7Y9S*NWnZ{q{ynH6P-~$=NAeENYN``1j9pMzz6=sQt(|@=2>sD{QP>A{QB*> z&5y^+%8J*pvH0zk+G~dSINR^K?O51U*vdW(eOSJ$OW4}MLEO&{J;55Er-Ov2V&2RP z$SsUSPhYX$Ibd-ERRCjPXlRz^bbj};C!%6$N$q*L7OLO$x$pXE;M9G1%Bw#2dkh?t zu<&MiMJ@WLm0jiAzC%|@M`EUt5wVzCCngJm#+hRW6243Cs$;O3XO8 zT}}{9<|C-Yo-1aE@txNbtUb|`v3x-tuc7|G=kxYPME|g3Vq*SWU+*x@H7j&#cRN0oS z#mqp1N5pz9hS-UsU9x8MNxI(B5a#IEr{U)nZ9>$Z%%#GYqdr2`Sh z4UdcsxE{f;XlExTRi}WRmQ1e}B1yrupj%*gkJ%9;h7sGZ+=TOpEgrc?cfbIdX3fV0 z8xXN4H$>g^sBfAn@>QV42@kz}QvrH(a#BiJIfiicK{=JhxKb*_=kAC?P>^UMm07d# zsqHv73QJW;-D=?PIk4GonV~JK&|F*`Vw!Isnb_~5;oYYdEAi>uSw-2eSy{3Q-%^#F zN*TQ+r$Z)eMffdd!K zk&;=N6M?VJCw0*nwI~@VZ8Cs}>j|B?Mi(Yc2_0E{dus+UxlnitqE;k&it4!;CVIXv z6RkNbxP7^F`eu0fac=t2|8ktSZQ28ioiY~|2CH@rylErzIlwUTT(vw~Idr8bFNc~S z8$bE6?cX6F*>Ih#sUj{d9ZC%+RXr;0;NT#tr}vahuPK%wjWwaJLC-X2cC1?Z)TqR)G6f5n>-e#*9utE^;jt~coTOv~f&4e!M#yol{9#T;>)9GGuc zmNha-BmVrNH)zuR%Yspqw2gkb^=6>GuGDW;^uSzC?3as`4PZ7Me-Nn`oPZ|Kc`QGh zgBneHPDC?~9|AmWGn&Z(r>QXM&p+#6zP*kjQDCBVwGo*(w>sBA4B9;tU#vyMqMIWi zz6~T%YU8)qwpk^H1P001(zD0>j+T6+rcPVK8&uC4#CXZtexp%PR9tKjsY7M@_Dn%9 z^7i+6Y7vKtmHHQe^SzIAynZe$grJGOq05r`6t~}pi@u3l^--r86$;4s`U?2pWo^@` zNU2jXGFFMAdFk;EVa2w}l70Q&{y5H97eJ#khvxqaYDj%F##-Fg-d<~?Owqmf`pdB( z!%X0xaH!!dr|t?^d4qmFJ|n+fOkyih#A6)eC>w$i;LmOxMxN~$Gyz?Zax3b@8N)A? z4zf7y=KallBD70uu{6oSmH;%5u9CTMT#)3s*0a!^W&8u>MnY7f7=+rP)Owh$_tpx; z{#N+xpHY$(-|Z6b_di(m7vrkCIYcUhMZ7uS%X5<$!JU{N%;Agr&kaAEj9HC7g}Hln z2#Lk5%>I}VMu7;Yvw?;U6f+*8fVH3w0z1yh!g?JFgk*_f4ydS@hnqze>P}AQkX(M1 zfiwoW(+lDZ(O?R)wXKz;nM9ht?cv!Ln;%p1RS&B8^}*nkYo0IHjdVS2MLDdu&|@%j zbK~%Cx+f7Xjyargl3U9u(fe}q@&;4VgZD5~j4%biR3AoV2jx>WTt%_8%U$Z~>IQ?> zIcJ25V#nnDADi+j-lwNkQZ%qFUfuHf}py>*;iGULPFvNs(N|&DMrw#|eAXFIq zwyuaz>XK3<7>hgD1j0;W7`@U(3wFg&qqxXCF-S}X*E%O1?+-#D99U#Tuo$TS0wwjiP2%?As7@`u$w7X=0b!Z_VvaXY}MT(sZGdGnHY6z zwWL=l_~$-u(|WYTive;)o+p5`pw|E(_e{=T&f;R>A~;hWLQi5r2y zp-${E>&GSEXqNT4IoDn&X;H?9KAZVc`dY>MWYaF_z79u8ke_hr@3o&R84;b(ysk+( z>6zKFMy7}@7grMPfWpCyz_4Uo?>PaTBiX+3W+|5Ppwz7Mu)3z>pmjvw3t$Yj1J6mQ3(AALxyo%z$6U-p{j zGiCN(>+~8VkkRSiLAmYZ-&P2wh3=H7D`)v-)xjGPOXOz~4GKLhc&zVpY2C%yehcO& zgXY4-ITcW9W=DdgOoNZ+>+uSz{A3mL$Gb6ttP-mA5Cx76_G7*4SRCH&i#m`g;}1Ns8RdtPQ!C1ZL2) z*jMB6jzY~)Y`#)4z)TViHC%o7Aa}Q=d6S)#Rq4Y?7x}sya1>{Uvfg-VUTkFnjz|uA zY{}P)tBl`?9}g+wvy^gYkT!A0m9LoiS;z6yUMWtPnvF9?;TTC)G9tnn`$$9+LV*uc z!qJ4CqsbLkXh9Cu|8Hrw|GNFrJ+2@&nL)|a8OP(W7P8u8RaJED|J3ox!~tbrFr-5A zVRno*FZ$b`yS)6)soCtq?{p)bNO&w-`R1OHDi@R38MUo@k^c2(?W6S8FO8Y8&6>9l z%cdiHLs8mG+EqHct^2W>VnsmJtCG9ZcDl$ISnu|Yz*I*Ae>Bf(Ux+D8+?37^l9w$l zF77Bi(SyDXXd)gDYO_p10&VAr=L2zx6$o7o5uS*vy;PM0JR}1Ypfv`PzTbDetz-*# zg3EF|EVteh2~8*Fv!DlSM@&W#h(|+s=ob}L-?fsol`6pI%Kl!VvvB)rDB;w95`%2o z?_u3V!vwYE`x3?KZeJ%CS?2Kv+NR~yXx3ROgb;8qlc;N$X_Ehfr??cS>N5#+YHcm* zMfFM{aU6C>B?BWgwkH-9(35|t|Il6D7B4@7MWR-MM2eklYWRStd<79W@vIoj>wPQF>8d|Lj|p={p0D z<3_m@oWLBHm4(ll0N47Qz>qU)q~d)-{7GDctE$1oGchzljdQ0;LTrGUtW_5w`2twv zc?U`x&4a>~S(%Hs=WktIiR@9**vLWr?d>L9iyXLa`*4Q_tnuT<^n_8H%ZUif)Pe3m zp1h+i5=xGUphO4-w(Anj&7$yPUH~hBAvuugr2Kg-AcciXya4wiK7OQ+DztKO{^flM zA9E=CX{lMsLH#RCHcxQT2F-SD5k#LHZRUDbNK*vEjyvU879(t6d3r9V`iWC70iF~3 z=7yL|yW+TX!cJnJX+ZXlYd$CqLUkO>=XeuWE)x%TQGjy3jdw0nfEZa!hl@C2tx%h| zldPUnal~}L6vZZ8Z{2$zC=DU)j(Zt@;35Sj&4)NtSI?Itkziuh3{GeFBkx$h<9c;e z4L+PO{nL2McioXmYe(pS)cLkIztZ`I-m`i$!mY8t+!mcEekJJWfW}-?8*eZiJU5A~ z<+V1XjiFH~26hh3OeKYuc4Sn-!cYncwpT@Lfqe&_g|iLW;q`t8^Cg88PFgIuhsCoJ z+av|drk+UZCjWND`%LS1V_WjCP7l=38!U%%U` zYAl;y9w3F3+3wg#wu%QM^GDJ>y*5{6D75-ee}sypo!R;96fcOV&D&0qQi!Y@!pjgP zwdWafZJkg{C^EJ&KPZX zhe{k&9#-p)$)a3YOs8Xy<5^gMjZa$*4(ca?;-46PsFL=a!PImCR#q`+!C6VMJppJ7ZD@sas)SRR4&Zm^5|1r=WJNN1?I# z!rUttaGd7{JCSF|;u`^%Hg&T*mSQ)k22eUj0WDsU?h+yJ(Mv;9V*JPy5FT|6-AS0r zOALK<#M4L$s)EmRF-6`f~*Y! zbdpwLfP$`6FdH1gieFlH!-yUOLJMa{$huo3S0<^t1Cn7K$8 zXFB(I7W9=If!|V)<%#e^1sNXIcNxC3NK20a#U#2)T~%|WgDs~wD5>h_zDvcFfGc)1 zugb0x>BRDiKggKHir|^SZ-3}p*mV5sQHhqFdd*5g;khq{^_wPs8M*NSK?#PgllDR# zRVa2pE~RwDyE*p0oEvUyq$&$!w^73eojmyLp!a}G{$Q6}8yGg4T2-W?r%q4nP*J}~ zw{4a%nY@3R&BtJo#83{D=8KdBn`8GW^_EB4z+lx&Ny`-1g|;hvXXWmb_&TKZi(BvW zv;ozhY^rm&80#KM`=e}&;;&56-X+(;Tep&-*2Vb^z|_yv+5FbG+V;6$zbX~MF~>AW znM2ovgbO^JGj$NzTJV?$&S_h(70S1Y)+*|S-_B_+=2z&EaB}l%5_1l*Nd|^w(*QrQ zdl2C$%Rh2PSR1mXe2AyXIPdL-lU>0fHqGz#v!ytF*(ai;`|4~_fD?0csRv>rjKRsH znq_Ua7)uoYK^1|2>spSrXs>STn=H?ZiBMEjvj`2rYy^$W%kYRf4Pv+@r24ARJ!kj` z=qr^l)vKz^EKm44Hlg{|LL%f_#dbt1O}5qFykh8t!Z4g(DW7Q{2)2e;EFwCqlzzJ{ z&?b&QAY`~O5$M>%hlUzr4l1jg&uaKY1FhtYu68sWUhHL3g6jy92)25Cj6kz&PR&=s zxEz)+eeTDD#d$53T^C1V^Vy4cfLN>w?!!fR_*GD3+#2Z&o0KKTkYnt0@r1|*t(^BP z{fWN2Piy*emmI{=(ufIJjD~M7f5rhFlYwvE62kuW?$z^35-Xt4-!~(usysYL@z6O# zrQFinZ4Xh>pr}{XTTjQ;ow>DB2J>My_m-(PUXU$$e=D z&NHGPZ)!seOiI&D4UWDn6aQ3-5XT;*0vC@md9NGCTInc~_V@#jUDsdH956MTc|oK{ zmMbx_t)C6C@$mLj4IR~?#B!-Va?Y>hP-v$7C3lC z?d|>6<#p_tgy%70`D500NQ5LLmoi?uWfa{6l0iV~N@*?k)dMIZjta$Ej|!nlB5RxE zMvt%)=`cQln8;h@;nXQIqVwf$t1qp-DQnPd=0F6Qi;-Ne*tInNhv2is)tq77?%Nb` z9}j}k+g;?074C8-)$c!k5MgEr+50T@`Ta6!pbDPMZ{!eHwtQWz^XEPxJWp(6I%`+@ z6y;}JZ8nTRo6UBT=`JN1fd2~rKEkoN_KEboW8^A|V5RlC`}Oz42?>FH7&EOuQjT_s zJyOPz!=T@{b79N+1b-dj{Av;OrvM79*S#%W+~J?AA_)uAhfya_Fk%W9LF(hI|n*szrk5?9T&KXs3BA23h4DN&>8BhI+!Q@I%T3&VaO=6P_rUp6ma zag#<**RcBELcT!x|?Awip6>G!~R?h2<=k=?n6t`$XB%oe$M-j5r}-C`Y4K-T1Z zgNHo*DD|j9V?*Q$0))bshM>up%{`TTWQnnh_g2{x!C+poRZ(SgkA2xl4q{zS_wBk# zgR~TW4LbR0PD_B`g2Lp_$N2vEqX$O1BmtAw1%vA~iB7{60`<2w7dv8Rj+YaO?y=SQ zDGC!WE{F)3I(4WjPJQRcmhX9hV)WqAJs2wV;@%|!x}+710DHQ*poq<;q!wS6&c_}H zC)8=g7AU^MHn##D^P5iNhmSIGG6QtjvSrf4@T69M9xXnT*WZzNz{uu0L>bU-Z;!Z$ zd8RSjs_1y`yj|opnepe_0j=n15A)N0%*H#x-o-DHzf()XPqFp;5(qeS#-Zz|QyP*m{Vi|G5Qs;#*2)jUvIJ@oF;Mtx7lZdEEPX|(~~8xxMk;DN_}B|r9o z3^widD_yFKcBbB55D_u)^LyJLF&CGc{EA86)zX#5CG+9^hH9Qejp9hzY;lDN0W&NcZ(ssBAn5N}l~llQ`T`2^~X;AY(Q0 zytl-=FYt?h)to#WK^tNWRrZJ~8RU=dyf_D4;*X328wa4(^Gb99l{X-kOCP-u1~ z6MUX{$*Qr{`#qXzVOa;oAGT>^!e{d#LmDdVSc#(zVW!Cp0X0a5C(@K7F4XL|t*O`$ zwP~9V6>dz=x+^)>fb|}RG(u+PD#pj(A;XCF(Gc96X>xuP4xEjd`TGJR2%M8eVvHrb zvvO;65|!9B>QJYz`Bjni=wR9q=^?q0_{`{mG)Gcl%Oc%qMhUI zOML!kHqIZqwsrs_6m{=!XK?BrQJ9sIlo*^K$x(R(v|387e+vR>ZwJ1G=g^tx z!PZ!CMg5Z`T~GiQ7x(3MdqP=yh4$0bDSZ9Im^gZjC^*F9}~yPX_ajl;w#Sc)`c zfJH|!5aH!#)7>vnk%k(uu9$IWn&`t#t+X}|l7x_Q!9+oqm)0ie6=xzpg$u8D9WwT2 z1_jC0WU~hBCdxshJY+$>z|;qa(v>1}H;(wh4a*J1c3)ih$`^ZERv+svRl?s9dZfdV zeNn5Sw4V}O5K*qYs9qvhXIWEb%*7d$Od-1xYO5hF2%i`^ECc044;&sRJG+j7fh`F8 zuw!j>7}24>`y{X&b0cKtX26Ke89Qv^{cecZ2V58T{rta&SX?no(!NCwz_S6vDyquO zcNu7`fuooSac)+qG=;pG3Kz~KF(jE}!JORY;Bju$)oAD;(Z)9P!@JkOF z`-{?F1KQ#7Zlb7{W`1H20*57uyAV3N76(kJ@>jTxL|Rbrbj$GZXW-d$kE59_ea7s?0rlZfYLCw(*&l!d61>|(UBadpr+Xk$g5Hm6uX5KpH}H3X4uJwX%?W1W^h?y zO0Bc=IM*-vV#caq8bi?@kB|Z%q0eSp>lZbfHdr1z;dx8PJfc|$4*pMSY|5VgMfGx< zYRP8h>rZR3C}2-bF7A|5$*Z{iOF`@_UUmKF@~W!Dyi?S2>iBF|C^rgcmPG~?MZvEe zN>g!_*x!bVMHd=5=Y#2SQp_azxr|5`Gkc6hL47Z_0#0+Aq%VcJa-N8jLVbDlqc{#%5 zj8L8vzemo>eo=LYM(Ga5T0lp&xX+dq0wk~7PS^pd~>T}}YM#?h3jK6O}4g^Vy z665WhCv&&^T2yEY5S*UtDj;0~N)1?j5%`e;CEPeIw0qs3EtL&AA{>&qRw2VBkp6y1 zT*9MvZne;;<2wT0a$+v%TZ+?yGu!o14vFI?me2``c#O27W+|2t}(axi1+1 Q^M{GNw2D-%#HXPD0lH(mhX4Qo diff --git a/docs/img/spark-logo-100x40px.png b/docs/img/spark-logo-100x40px.png deleted file mode 100644 index 54c3187bbd752c1d55ba9030fbc730b12735c9ce..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3635 zcmb_fX*`r`AD#u-8*3DzKjj=_x;t(2Un87T@Fr$*OZ*!PBQVLmXLPwUA zr9;A@jwp(pR#|%$ij(6g@93@bo)7Que0rbX^ZcIszMubf{jdM_;mP*&a8i)fk_7+& z3NFqBZ}FQfevG80003Z867!LGqsejf=lC$gIb0H(0!hwcTB4`XN*kXGp3`!$g!Tiwf zNOzV4g-UZi#-{ij^Y9Hj78Qmg!)&afmOQ+efKK6%pgejsBNorIf_>q|i`QGn2pIH> z2`9=5_8(FH?w(KwCYu69!_7>>kSG)sgMp*W(HIQM7;1_{nIe$l8)Jep$D=WLQ!MoB z1rvK?lf&`e1mag;;+++Y%Hgo^2n3hQg>%i|Om+kUg~Q>tI804V#13N(b0_HU{5q75HUFPH>?XImP2BMQMQz9VQK%x+WjLI@4%*zI83%LlNtT3#XP^s zhdTH`{b&p_lN-BLd@->&cmjn@i>Hu@Y$hH0r4D%7|8P#U#}ZK}tT_^Eio}SeTbP>K zIauHvkXQ_wXivn#zGB4*`+uYmA(n^OYR11c=eHwq$8RnF8a46euVJMy#IKkzo8@aeAvXNRk6@I|u$qz1Hfbo)8=P{^mS1Ef z$(f9u0vG|jRXnXWr=*7w7+d`IDve$j&l{g5CkJhJ0UlK<^!VjpwEiV)U$d<;!+9gB zKgSLL*&0V*>TV-;yFpDdRMT#z%0%dLYb1KBpAD5C=$XPPvli(kXvzE^bQ?E(xXp^K--{&Evkj#ECN;Eu3 z-0PQ}vj#E970w?K>4^m(Ap7k;Ym%P3bP#Rf3jzVO5!qtmLs30}%u``SkWXC0$Q=YGVYr!5DHD zh{BF0hT9s01Vsa>duc!i_0hT_U{8+ihIAG>roa)JIS4)mEz*dUIVKSROMSzRpr#5m z109l&c#4+q*KTGEABOtK`&V>IsW(5jUDs_^1>zXMVYBSfyZZM%0|0M;=L|pVsnMYw zDlS8^=C~U{gQ6@s#_KxIxwIL?Jyr;i3_mxrr>rj{b1p9KXtJ4s93(;CLOo8V_Th9RX zJw2ob$Vq4N0M)W6d(&x6SU=H0d%GLcf-j9h?L%Q^$rBx?QAzQ(mODrf`@kT^Q@#{RsZdkGt) z(Slm(>yhKWa!hlc3h6I^OPC;g+z_yo@n^rvx4CpQ?9@xS3d$^x)6WW4)pONz)ih6= z>-Lh?_b*F!^)fj?wL%<~!aOT4p3hsWz&JTbfY3^)5JA)cxl^X?5L(!Oe{-M$Bp7M!faX+0eY! z!xEJ!`W%Rn_*hjBy{fJ%G>+!iD|bHI_S0pw%I51dSVjUymQHuGl}mG)>DuTBm8tj8m@X;HW!a7@iw^+X}s_ z2J$;e*PV*CjENSkmcB^Zbt^hPvmlc`!zDU)r37{rtiyMa$86RyAituw)n~PgPiVe} zEuD`G8B4<5Q)TKq#XN;O3Ka4^^E}saugpz6PR~xjyb8z{ow~M9H?;pjcOO*B@Kz@5 zt~t11&F>&rTloGsHXiT9$$Bm`3S3NX?{Y6I#)POZgdhK4JY9LF;;Mq1I{1ELhyXXp zCA1&<Eyt>cq^{NaOs1t54rh0k#C$ z-etaeA_L{R$HoipgV~2=DYxr31}VuO(_E|beiBvS;9IlJVi1oDy*zhnhd;Jvt~9_} z_p;zTt=}NW=LDRFoT3=$GQLUFfBE+{PP_~zysD9QxeuptD^uOnn}t=~ zV5x#j9-x=QiD^ht2=M3qlFk<16?a5#f=zYdv*fZr0ll9OXk#@N9zQXcz$jyLXfrjRACO+x0q`Qk9ogtQW5ufhzO&u$n83Tvmh43O8|7M&9j&r7&hhBwYm zqC>Tn>VH8D?uyh_^BRGyxZOEC)U0!zZ3kbymOl|HartWcPFZ2$_d^hNVW3?7j?S=< zwOfk2S<+Z~Ufhp7bl#Jv8}j#x9Kex0XBMkarHY+*sMx0r@J!BsU2(bQh~c6A@0=cF z+Q#ROuu!-=hVt zQusJ(Ij~MpoH+hC!AR+MvxS}|(}V14uyTK!qvdts8vruRqq8^f2u*qR!$~GUqkCyp zp+2+qL2KUQiNUePw0a&`t(oywQ2b4C`^FTL|kzFtBzpr9jNDiBO&K}p8Z;^ zm-#Wjw|-e>Hazen03vxlaHI6qqbSbR8=GoSmG^qaxg`;nOv558^AN)x=EKFR5qhyoKINc2CPV_{OoyG`MbROtn zLm)jL$Ug|g{y=;Yi0y&c0qz+fHUKIT0MZYEf(xLE_RX-UO8)pnUoP zh%W%c%@Cu2L{E%3_#1%yKxnS{0F@7f7f?_|6VMEO0Ob}!`4^ykSWq}1=LWoak6Zu{ zfY~a#K@bL_=?3NB58OZ(Fku8I5H`RGU<26zH^2=fPS6qf@ZXN$ow-Y%O+^!o*n~9g z(f0M;-MgBZGKUFOfyv14{Xc%zX-J4UKTs{4y0_}ZLTK<90D|v%v#ez2lG~h1DoU5 z!bnyQWM7I+slsA6NSx*D00PJ664n<4@Mwb6XAl-P(v1@u`-s-)0U6o*2v`mH$Awwr zf#UIM427X(57EzwNUPGUH%4GJy!~>q$E3h15MZJTRhYhXROceJ6!I>)iEP^#Wg%^_ zwv|hs-;@ZA^QS#Od08E%D*V)T7l0TuNJFa6B(W-1G8^bHFwLKy?$%Y0!nT5nL3Bk6lwYqOdX@Z zS0*36x`iM53jEawVuiLYdYn=fGjAT#?i1!ky&TACC$1olJf7WeZ@Ltln?lSrtojeR z+(KuKp^vZYN9R;bYsSu@dv4T#=S@LZDRUx#$f()dhIh@84F7TrmoRsjD@rb)mOA+9 zi!pI<#(2VD>D~u+n=RU4j$#k6pX;5Zh9zI<_?5v5m~a*f+-Bd+eOhzI7`rBGQmw)J2h}OHR?4^k+K?lF&%=~ zD8BdE6t`t(D)L+`eNTzk9Qvbds0v=e^gRCPy^39p*_2 zxpxNod%w*i0Bn+cwUv7SveEc?M1jWbV1~}_i&X}$ec_23G06_j=?Ud^>5)X)IjLtYkN9?h5)xzbva?8A1kMnIyDoAZ6y+i^ zUoM=cq(ajc09KMm#4tTD%c}L7in)fOR7@Ws+6X{tLx>iKkn9H@Go&9E>kMqyyE_g5 z-ul#v%(a%^*-ktS^f!-$`6@`YZx(eK(v(4*{rPvAyuAMKAWkja5%FM3HPAAGdLjiK z>6j}y-mhTl-)5+@!5WK#WA$+P+(D2OXafHwO-sw+nx13T;N)`o)oj1dV^O2Q#wegC z67o~*p{aYoAP~RsBA;5x$NHhGI5#wbgxs0FN_hf|*C#W~e@-7A-e;d|fty&7riJjc z)$;mc!&I(KYdA5(9nW9G19Opc=dhMc z@m$+jtlES!P`St)iQrPi9!#+NbcCs--r~TxnPucUYzr=ocK+DprV)+5YHY+r$Z0Uw z99?^ZC~giKwtj7bdB$!pQdND=d*fJc-2iPX`?NJ_9~GzZU)nuTN=+QM0P0kMj?~Yj zmIgDo1FAV=r5c7H_UjnwR& q?zge_Y`V**jbk6@HdcUJKi&Z9m8L6l^M1?#00004KMPEP)4Tx05}naRo`#hR1`jmZ&IWdKOk5~hl<6oRa0BJ8yc;~21%2p?MfD<>DVeH z9(p*dx19w`~g7O0}n_%Aq@s%d)fBDv`JHkDym6Hd+5XuAtvnwRpGmK zVkc9?T=n|PIo~X-eVh__(Z?q}P9Z-Dj?gOW6|D%o20XmjW-qs4UjrD(li^iv8@eK9k+ZFm zVRFymFOPAzG5-%Pn|1W;U4vNroTa&AxDScmEA~{ri9gr1^c?U@uwSpaNnw8l_>cP1 zd;)kMQS_;jeRSUEM_*s96y65j1$)tOrwdK{YIQMt92l|D^(E_=$Rjw{b!QT@q!)ni zR`|5oW9X5n$Wv+HVc@|^eX5yXnsHX8PF3UX~a6)MwxDE0HaPjyrlI!;jX{6Kvuh*8ej?;85ekN$?5uuCiS zBTvvVG+XTxAO{m@bvM#Jr)z6J><&E22D|vq?Y?Vkbo_DijopiF$2PET#mZ8eu=y$(ArYkv7@Ex`GL?QCc!_*KFrd&;n1r7 zqW-CFs9&fT)ZaU5gc&=gBz-DaCw(vdOp0__x+47~U6sC(E(JNe@4cTT*n6*E zVH4eoU1-&7pEV~_PRe`a7v+@vy!^5}8?Y3)UmlaER00009a7bBm000XU000XU0RWnu7ytkXQb|NXRA>e5 zSRhOo=*hmAER0R0YUQ5&?|AZ?L`ST`WM6a>^FLsRE~`yXWtl?w+2db2a{%eVhd4JE8MirO_aSq+gq*ou-ZU=tMS$2;L~N>BV;{wvcP<7Z_^#Xj=fhGn zif=t*TpK?89DUkm5wUAvW9vl!o*TLbli^DTS`@^q*4Y};iIMBs7&6=iN$VNr&8F$Q zeF7+21l#uUaYC;V zR6w;;vZ>ARVX2PzR~H40(fs;pcjB_RthZ<291>Y;0`Izl!WlITbxo_lI`?&3=ri-Q zdrM3BoZ|dR7T)zchbJeD0l%5zgZ99lU_ohXlQoXqb6|EQkc%K)^YM<$;~?fE;l1%) zcQtND#8gYWyL{{M-8)MsE@uvh&K~gh;o3s?-kqiGm76!GEigU)|GmvX&Llr~)cVWG@3)*vA*QX@6--0RtTDtPQ6}8;?wwlz#6MCK zis=Rh+j;Tocp^xq^?r@Eq#%F$_ZMdWU%#ti4P~wa0B*i`^=3!bw7k3=En)mAD{|HE2!z z;Wk>Q0D!d@-k;w5$I+X1te7IWpg%3&s-)QGYO?>Q#!i57Ur*XMg^+EIi}-v1l7N=2 zjhUD@m+ss+Rc4$d8v;1 zYBDA=_}HfZrO9=oT>a}qG(cDXi93OfW z`SI|h^~GyPJ0$P!?WH}glXO&?@V6!}475XGPSkEJOI!pzF)l!NVEjVXz53$#xTE=` zQ)ZqPPqgXUaP?I!@S6Zwnm2r8X;!|5hdBpHiygYE#>4dKB$n=fA< zH{v!3|7p8`FC#;{hC?hY!iS>sqsqjfNxZ z_Z+_mtOaX|N(NveI;)-ljf+7Zc9NI?qY$`(&96vY$gs-J(B{`gE)!2$gKII~_!vf) zj0(p`KweK@FzI=y2Wu^jV(2(~*N4F&nF(M#q)!>;69D6AiI_%)H+{Gx7ss{~g6Oa4 zhygc#Br@v4ZC2BOQ68CpD7t{(n>>U5mOv3>l?01l!uPo9e=jh~dJ#2H~EYB%0A_<@(%v0D?Aw1w&ZxILfi zcER+6bap=xbBFeZ?xSfO&?h|wGeP47Tm(;FPevNq@fsJc`Ouqva(49KkNSjB9Yi(4 zS`rlDPpdt^NJ>Qx(0f8l4R;NM9@Va;+Ch|dDAvc5Q$gbpesQC>Ts$JLb7ahwnDG$= zbGNbD_#6G0cYaEx73|zej{r9gG;i^JzmWvjR_Qfb=VNIPn$LHTzu-3PlGdG}(AMpA zE=L(&3u{_O)4U-WhlGbuFB4Xo&fG;eu9L1N0F_0awI;{{kprwY!v|kD!t=E=+fk@V zu+M7bxGCP2CQ2+nEG}$ICjtmSwxrFPn@(>MploEFT1H9{O1goeLXO9R{uQq~YQ}## zCi&Xc#9H)0VC{d8JAs%6#3(r*yD7)emzy4A7F{3!sRh)LtCe{=cn($3IZy+H3-3-_ z0J)}kp2TfNORX_JVaz{ZO!T#Z7e;?_*2O1{JZcxWSbRXt*D!AldDy;JA6ai1fIPui zdYsPaU2Ef^;i_TW0Ku3$L`Nw)X3|Zu{)*ajnFqt%PxQ~Kb1>Q645cy#-2gw(GH#k? z4p~L>h4y@GKC3URrJ--g=o3vx`bKr&liiVZm&uF5LlDV%Y5Y%UshP+M&3YmoQNnCD zwOgZuP>M6ioHtn{doy%+W=K3on@-0(%4YS2ECkrA$}MAc=5}bLfs5rNV2uH)xDdqj zMI>J@L@Aw)q~mhhP1z5|vzOkRjF+zcR}LniFwwU-#u^j?r4#~`7G46olDPJ-7s?Dm zQW#u)A}Q_^3DB<%wD+^XlD<)Hx8mcWGc|9iy1zjxGq4>z%i7Y%4Q$WmeLReLvNq9S z?dOje?4xC*hqkbpbA9O9qR39K`*75h_OyE98AU~!s&*yaIsH6=wUnaGYqd*F!*VCf zG{C5texA%5V8kF3Gzz8_-i%5%Gaf0uOGTa=H0-`C2q11u#0rsJ<~lQ`UFHq{5_`Ld zrTr4~e%51Lfp&r8y8`VBvD@(foXg>qSyzO9lVzjpa0000< KMNUMnLSTaP)3PW4 diff --git a/docs/img/spark-logo-hd.png b/docs/img/spark-logo-hd.png index 1381e3004dca9fd2a5fada59fafa9a30843a2475..e4508e7553d4ed4c4c5a63cf7c128ef24b0b6d9a 100644 GIT binary patch literal 16418 zcmch8WmHsO-0q=MVF>9O6qF7DrIA4pkRG~QQd)Whg`q{HBt@yAq?-ZhZjc7)?ymRz z-*0!_b-&)3HJmkvbN1e|_b;C3*}oH}rXv4e}@4*dMY#|8hQLVkV# z57;&`Z)E_WG=}KL3765$M008j`0ItAOhz$Vn-~)gyQveW60syM_sdeh& z;6HH9mE>W-{eSPw#@smY48eN^JvRU#BKz-yF)|l$0{~Kw-@s(FK1}bXc_-0ZRo%*!)-~waeq+PdrZJ%C{yBEYYh1b&C_wgsQ!%%>K$Bk zLg=L}{LxAL z%&!|x3;Y-!4}X?FK95@bix;gwR^?GMWANa|V(6z6)7_hk&mosK+VMW!H%^gAKF@@e z>KNzoz zT8-d;FF-{3@masfX~DfVqA1MbG5_8n#wsN`Yn1m%?LhX-cY3|!CvzIPd-y`iS2%Un zV9|G}<*ubYCAC*ABvbpF^xe+5?;{O1-|t(%Awqu4Ftos|KY=1R6W0>z!JGWDl&Z(R z8t-u+@(XC6yA#2XdMO!v9I40H0>SUzvvCt5!l;KzsshC&?P`3rqW`zYZyPm>bvJj< z^yL~`N9UkMLbLuH^8Ym`q(o&+kC~$wAKo}mQHTgDHa;z-=_K}Az=cxWe7XHDE8O&f z6!E624|9RE<^AO<;wh=%^G8o=F|L2Vxsjat{pCwz%XgdyZ!q<>WNyO(+DH|PXGyYH zp~|C5R@Qjf&3yTsXQ?#RMb`_BQ%pFvbJ>To4&O+Em}yz&*&tpPH`lu}zSaDXEzwrv z4Hq|8$lA)@-N1O0eLl}RfsBv23^`KuS%yw#r2N=A?sL&;MqzqVz9@^Q*T0W`3X`X0_G$n4d*W)Xb8_BYpo)n&gog+U=%jdm5v8YTHZnv8BnYcRsblCAO4^JBusF z-Ts0#5w2pdi(9P5r@&xd?)=hnMe$u6(4^JNCgpipK|m%Xi1R!JzkF$$rsD^}Lvll$ zLfaUK(e{JkSan$<(|&LJH6cUs+q23*eEI+8X#ALt5lJiBPrfH%h6yX}AMZLLcMd$6 zs`5U}=g-l`jN;9uGR4HN7$>vt*R5DPw43?sI>rA}zTo8mf9j)Guj5so|04exVg_Fd zha#N>?C_ryq`W|}hp+~81@w;89;&0Bk3VF7u8ghj&mx>(Ljce^I|tgGHe5_@L2rb% zA96WnNvFwL80j=+HD9hco+oDeJBe>a^E>l9kOLVg)LS4rQdBvMPzNZG&NY}~vmEBw zqL(Zzw;X(cBL&XwV7e2qEvhwTY=8B0dLSlHQKU&rHQo@o4 z^aKO1d|hU~-%GqrkPW+zxWgkRsKy_lQpN<#eW62w$?pEaF{wPQ<; z2>Sdvp8e`d@?T8`@87O^;$fLp3jZhSum>5=$N?(&w>I)KJEM;UEdLr;80Clc81?r) z?<8)#;-eVFCDR>rkr$CKlAM`5@4P!WJk?(5KoV5F)UKEpZ8ROvc(Y?fOfKy++E=f= z4|=C1dS%!qR|PODBj!`G!PR-SR%+^qPEGNyQFzb~N7S%L>U1?|-Dr)NF!} z5NV`p_f)U5?O?D^-<15Xfq<|e+8zScHxwLYJPO{$EmU=d4O*MFvvfg;sDc5rHey*y zv7!xDqhi16$3Ddf)sTNK?1&X|3~Yt&txlWSy0sta3!n9Gzub~RC7d$fNkSIGJ@tok zgT9?l$n6dm=?%U2ZyWOJ)jQeMCIPHpwNb6`=ooTddBOW5GWGses4N^MZI47mz|x_1?OJ>8H3lxtAp=Uw$swDy+ZJdH*#dNScS2?}2B19<5y-K1 z5J-WA-?*>-;X2Y_1|4K@ek1u`{0ouiai)jF4s@J7ScA6uoJjj_gaA|@?EYm{rOEV4 zU9II37EmdpUsI5fl5X;(b*D7Z9YuaF+){ed^7EH1+qjicj@zyQm4u+_QGfg~-9VIp z|2QSL{l8`V1E!}gJBnFNsw6m4D<6tA!gXjScm^*4NdszoVRFr%nFS@$WFTS<^eV*35eG*xgyt*i0P3jU}gkhQww zZ*IIWQC2{OJ{`C-(FdpNruVSAH#(VGF!nh@F#8vc`FE|(S&3 zPV%Mh_(_mg#ox zSP=Z>3!<>LVd<%czZl{?PlLmJLz?YXx6dErXUb|gBcsknDRjk>60Y9Mf0G{VMbN4w z`0CL>NzHO8C48?O?U|nHc)s`-+Q6cYQ&jJ_=JM!>M?3XL7P~xypgvrk_E&2Jlb?lB z?5|k z^fSlDNo3kE}|3uz$HO^i@lI z@xa%c2OJAb(E%mjsiZi~H`ofoT~T48p~F^I&z#3OH-xp^HD9e7cS*`!H(b=WoV>wl zWVP;k7WN1CZ+$U^EOsX;AUE8uEv>_M;7jd)P{ULyr5%ME@VlZvV_tXgN*?i$Ydx3g z6nZMIU6DPi(y$yB6+b}}>Yg%Zn?s-TU^#ZV!H+Av00YuBtRlUP{B4w{2q6I&e*s7vm%}p402?xtKRIx}Ru8 zm4bqb-qr@ZPTA`cByGRqu}`77kdWpc<>bqvQA`MYzm-(?bT67#vMzNXNJ3|6;A^}R ziqeM}oN%+X!`^5bBj}YkgJ$c%$l~PQ+`Qw%YtI8z45g93^qc*H+`3hHZo3)L4Mn@I z5#{` zVPl5YK9iVF-vXOI=fca`+npE!`9?qgGz%W$I>*V)o~X`+i6-0Mr(kjra1eL(wtzV< z$*z7?JCi@}GIqBLwO@7SIb=#YQRQU_7@F3kdN`gAs_|B9E1d@*cxdS4n|q8=wJVOr;B8y* zO){Xxd<=#~_ImM>%4~18*1Bd!?U`8p(^t|SyV?jDe%y{k-Fdaa!=FrP!g31x*lId^ zX*Nk}WZwGCOo|JKZB_b}X6X3=rEHCLvSDw;D@s+%@+oP(x8z}MTGsmbd4c!Tu<0j3 z@}7IMml5%tI!*){DxzuWg(wqDQjEpA;_KRpi3zHpDs+DSrfsLo8rc#BoF3OU7N0Y0 zhr2hSvUqR5&Qjp=UFQ;cQRR%G&RuzBrAPJifiT!VON$E*>!9tP-CDyvUaZ*&FB%IT z?KyvDs5x>Y4m=leYgB)-e5k%mq`ORA>@yZ4@Wa$f>?0$^czu=P=0|Ggr9*3g*tlx$C|+LA+rAq%glWbXx?*H_97jw&ib%+4qnC*d$#d z_LfnE78j2v$8CUTwq7kJDQt6CE8dQiN<9(NpwMe(v7B?14++q=;82 zzXng^u|Xjv5@}yFx3LiM0O}dcA(Yq{r?|iwP7(AvYL`eobCFPSUTKk=Uv`qCFWiHO zcF)md=sg%c(B2NG*(47hZj00#dS@HHSAjR1p{Ux|)mmw;0-jyuYuQ1YbrfUNLy~`H zlQEG0Mmk~mzlAM43WL#UxzGUWa9~WU7_ittCws#NEl{FAXFWc@S`zF?>h)`xHD! z0r0Dfq_qI8BH@c);M>NMRIZj4bfSWsIZWP2-*Aj zl|VmKZEbA32^L%Rq0+FnK^jaJtqVdRYR3~&Z z5C2%m=!NS8R9K2sc-tkyY+oZAyH7WpFKvfBSOIy!|a0gJD0T(P^>F zokMWkyPPDPJ+6BjqnX+q!qsnG{xRrPs?YjWakkrXBK->$&o=s|u2Tj#J2@{`MQ9sT zc^~tjSmR9wXsrigi7QlsR7Lg-?oZ%+$Hn8kMX3u1MWQ**-?p7w2%*ZkjDk*czQE0b z={BQliz-`OC;2cvw84ndMVbV6?1joX&%^yI9oTa+7vb#Dd~NARVk$fHYinz}l@2k> zFYep>EVL+5S5i|u;q03ED-kHshl}q(k9^12Ofus!$C~`Eu<{Wwzm%(1%$ntcV1}k3 zHwNyCRkJnrao>vqc|HD@ev*kQ>`5mK1DCL83JSmy?ot zvyJWhoaR2C#WB`Vxj=(*Q&%*@Hnh-&c?re-pU^-TbH8n0M>2!`RVb>W!8tI!>i&)NNKK<+G|PvQ&pSU z8!Z-7N`C<@dq0)OWHz}P-BSFcuGV!Qkt(cz%mC%{*70i+05S}NiiURG%z7yGYUYnU zskE~Zog3>?rn-7{{~U7bakeuR2Q=)Y{fHy_{D%c=!B3*fKsV{7b>z!tYE4y`+BiGx zWIxs{&PQVx!m^ZfKe-&bT`j9*;WXNjOt5}JBpvJ#HrtY(3vT{=im^Z$(OsEscFU(G}WSNkB z{g`V@329|l-6~8G+qhQSj{c>QuLrvjK#bPOV-+nH?Gil!5XgUh?lz)n;we+LM@ykh z0ReoD4`S9ZqB98!G71iJxhGXM30DyXXS}f0{!wrgbmA|?xMHW(#~4rqX4~*V|{^@ zd}mUI*-jN_R!B5&yi50~$#jO1$8hArK!g2hwB2@L^5S5c*rda3%^@w2!I}k$C-l2d z)8adxf;x$(Io18n&aN!_UiP(nWhQyE&YKAM<|t;#7AWyR(zyuJ5Jqe&cy`NkPQhL#g6! z>vEw5L8-gk0%71H<1D5#P>pNIH|&HJ_Tq_k5}|g__qx3;5przlHz(|3idUC$felr@ z0Dr!gxnw3$)RJsbrTy0ScBnFXjH$c*a$r1B6^+qc(vxODCTH|1W*h4J46zS4uz8Cu z756e=wj+uzuuYlDok*Sg@bi%^yiNqPG~UEsxMkO3<2U0AaflbMV8H87RPgZ(k8KEG z1OTwH{RDkBawa&(QvW==(ZcL3{f9I-*CQzhVy5l!5>D8T0MrUld&2f76i^0i?bzaI z#uJ@_^#XsOm~c#!F3Yqyy8C>px3flFY;!8Fnxdnldx!Rw5&z~3Uon^2P=ImO8ymuQ z$&_IzjyQWk7N(4A8&pEHsKwzCg8a>o;Gn8w{ksjl(8SO@NQH?OzdyQ| zQG_yKY!v#0z9#;K$Bel}{L}XI&BwdR6NAaSs_S|flz)!;juz5?7)YMCMbzI)VN-(K z*dKdT#xI-i`tgOKI^rN1qTa$cg6-kwidpS zurQ+nOy^oKlgGNLAv@cunwc=#vRK zRVQ48GMnalU3&p=#S5jKlu&>`kXIspw084#c|{dIEiNVy{y=FOSq2@7)9B<8__r9$ zn$quAxY^G^IRDc~0>j$_f2z3nX zoq-{Y547$UkqaV)jHa7rTKuDE5?-o^b_r`7sKC#9E(_ef433P&YA+E@Ba`mSzqjWk zD%*z97ofOG=VWxuB==0{k(?xJR*YO!qH*24nQr?}xnaS7)y0iv1{+{*;F{!?kZv=< z;VVAx(~1unZ`~IOExFcwWA&!*8&z)>9P!@jx^p>HTT^G8_3qE-MLfo#eZga~vydS# zRw!SHX12!CYarvPm!zN=>_iTtQz*9%kyox`(rub~>FDUlkgv4`wYQ>=tt)A|vdb?i zL)u0mje(~y(&w#&VG8AyCn9JG?%Qwvag|5%=$Kp4zZI30@qiystinxFvLH-A4o-xs z(1$0Udd< zPfWH)DN5@df$j$wC&ucx2>iRM#`0sj@_;ifyv4>FhE7A-PpS}g|3D>ms2;j{dX9>S z>eWDUiHQq*K?Wf>KSeYBO$5KJ4NjPg15HZ@j1U1AOQ8#89}fThV@(u!4Nc3o z75aV-KmK<4h^=Q%vDsN{(6VMy(Cw3(07Y0##LeFjzF`GTb1ADP>Dk>k#o+y3x7 z0pzkq3nB5+*E5n%&N8Vy-K{F8IUg3m>>!oH`uI1fBT9Imx$uKx`xuKg8boR~Dgr~{ z3k^a_Qw?%m1YxaeoFNhB=gLW7xfX5`9A}ZiWxpT3wsCWFYacxVr-)L}Xqz)Vs`88* zu^uShZ1IFMJ(wwI-mN|&76V3K#Iow*Ev_)<6Ol=tMXcQAOAUw#g_nD6@+7g z%b{SBAX$yiP628;Qu=v|Pw)t*(9}Y9V;IdmYfOrfcUM$FxWq^ebb0ZYY(^iW( zPxL!sHdl~zwX*Q9CZAbE$IMc@+iX1R?5Z7@&d-I8JFN(=Y>3MJ)*K5mbYJq`-THKX zd(ffx8?$pUNS>iavlK*sahE=nz`&s@#PpHKkQ70ZPI`8&XEsHn^Cn6Z6 z)hv)X?eytX{5Nj0Wv-_;A)&;hZCU$b{G=%9PcN;d@1qce*MzxOuDEWdMXCA)Cc0pf z>XlJ(dZ#zfkwrArnmCs+Z2)N;cFD4-swn&*prFUX61_~X0{duPfr{T<$_1etu<*bX zHb@oWEFXAM+?+chuI?_^OAz@-uxHaXd&c*@^4QQ0{-_+P#d;No(=W-Jey3-0e7!+Nr0aXua64vrPKkf z)PL|No-mH%@1M~Dm15n)9Fz&!@HyGirg^FRWjH10*>@qhXqqI@oFcbk3-u91guz5# zQ>p_`;O?ZuxX??n*ilKlvAo=CH{_I#t%E~kn|WzB%BRBltM7|Ua;O+78a1ON$|pD!A{RpF|C=AD85e zZh9HmXPA_lw|~i%RD4V}nshmbXd?eYgG#cEtj;_P;CwYGQsp%(4JTw^U{K8-?UpJS zk(hE2Celjy>6fPfWG$=aT!|5sxxym#1<~PB?R|pkt;komOK3y3y&ZZN(jJ)oL#~T#jj*4? z3Qv@<1UmeFfNbH%yvt1Q@W_b3FZ;`vZfk8-&I>*X_(r%JGfD5TrRUUbf_$htkWW?G zT;P{5j(GhXl*qbk?Sg5NF#v)bZj=u}>;TS&?fs8EV_#g?IJT_AfX8W?@Y6bX(b}Z0 z0N<&y36E6+H3_%%0ihIOi9lRa9*uZSfZmSSV)30oi@tq*UESViD(8b!UPQOdHrt=x zICj;5A}Oksj*bpU$B=5+3-x)h#DI^y+30TFgE%#`F|?Jdw2U2xoBL4&rB*h1GltB- z`RxWBLnxYPvGuRW+04sWg@(YHs%TYm7goW=&p(S*RD7`?uvmXD3ZL3^-@<_08fF)_ z2D8K}C<}#1W^*tJbVP1Oe^!gKvMP~&x{Unrk%LDen`~jK%CU1&r=(c&LN@T%=w=@X zgqsvBt@rQBIHBVKoLlX>=d3z36S|GATd%9!1jjJ=t{s_nl?OnOP%$ z%mBn2ObnejsYqJiPuD1dxj(OA`<6LqkX+b@h!(-2v3aVw&chPu;(4U5Ve%f z?bE-6&l9E$MXdYsp5$wx9Cq$AEd^-U;}!i0hNSq9cjlIF<2y+l>fW7hjn}`Vk5|mb zl8|`)+g0nd_O6LDp77p+*o0z5&&*oaet&yw>&p6^A5XWe$QI2KE<{*%`jFKQUnr8T ziYSzTP+x48TGy&F zIX-?1wO#z;3QjaH->51;&qvQQeCJPZK|GhH%x0sb`C7g+fiC&7p{2v+-~{j`3qU5+cQ!q{q=kx>6crWe zCmPe2Z*=#AJ9pRD7moH)l1^hApId*T4ESDMrkuFxAE$3J9t9xFgQt>jlA#V3+gp$^OqF2j|Ob8Yt>a%N3%n^@4L{puE|C&@!HZB z=Wv_pLB)8@^OXL(U*ESu_r2#EwZY}&leiZ8E9u?CrB~*$Jy9NnJt2O3GAh6Kx3n?7 zOG#88OR;b5!m2j`+)&wAbz2+_u31KHg(z6s$hsZ(aF<*6iw;vaIWyaoru$P3v%^Ez zUg9=9)md*GdQS|=#&|4uN_|mpr+gxS?^w)o3t?t!bQ?r1T~279lX{|` zVMvyvSIEcT_dLs`XERUabCb=+{O7IhElA_+A6Z4T&1 zo&}_sfG||zR)&_2j@84XR6e-D@J3*|Y--jZu6sqj?}QyHiZZ;uY1w8fwqGdsyF7Az zBX8F;qLe#^kv_9;+LOa*Vf#nL$kT)dZ9|^%7OV|Xy+LXxLGcaR4!18&M?N7Zy(rgcg z_t}x-?-174k+RZWhEld$g6tW7N7&F=&5uLQq{D9KRYO?m%_Q0zA7u((Hb$1L1$-c2 zE#UY$W0H|~ykphN_+X`dlBwJBh^XV>k0N$kMzQ1L5jkj$*y6XSk>%?1KvoIX#o56GGj1N$RE@&Py76MGKrGC?Vj8UgqRgI z@}%?!)+(D!0Mte;jddhL_v9@Rccn%mEinuCib_hoPNTD0TfbhAdQ(VGW#^%Byx0-J z$(NCcSRZ4|9bW_e1QMmJetrJD2W=oTaqc@AwVS+dE>1yzLk<=Q_m`D;T|}TJaJDMW z)d(^pVDJ4UGZsi!>e9j3o~y-S__oain>3&6McA#SlG2^ns><7JvU9!^H;ypRU#M-Z zLl6NOz+c0iYggt69M-UlCeay#fV-b?%B~=VJ~Z}2HNRh=w7n>2Rz}aNtBB6@9SnNN zV{s8fN&FfI8je#z=GU+Pl09QkiXv~tKS`#2d~nqjU7=mfsJz&>$IT0`y*?RLWzyR` z)y!3i!nnYN;5_mHQs9So4#M}PcAppzh`*Oj-9({0F&c#2aQt(Q)z;=gLp`pqh6XPV z8VAfzO`TTReP#p_xDBe_^(I6UX`QxY_R<8_-CIMv4eoba$CwAh7Gg}aSWf_Hm^13Y zo0#KdrZAr9p7Il8p3I|ZKgoLrc;k=hQ_0^-7lVIGEoK-ra|fPnn-0YVlSp1*b-Wiq zA}*vBkME-YNCcz(rG@Kn*ikjYw)g(+O9Y^AZ%u|hj#rG*9_PhnsPNmp5-FeBk#lrx zIsbb7v$4@%O!^pRY4M%~N-b-1a?L$FETed6481D1n=$TY3}`+AOmiz;T_iK^HMx5k zRqVb7h^F)#aheCd_^>2GMelbou@7A&TR9oEiNrq0kV)fTj5=ALnEq(}uPsa@tm|v$ zt<9P`%-t_=pJ5z z?sQbuTobP!_mopnAL#z*0+-rW#kbf@2&btLmw42T>+e6##3t#A*_d*~b4~tZFJ*AB z5tPF**5y<3c4O;CAG)7x5<$sqd-nGqpCqS__L`O48-E;qXDH$@=l7$r!57|Fb@U$!DJGJZUnW{UU(^&!osb@v7~%6*q3N| zBHW8v+)KlIocerORYK^w_ty866i{>H4Z?3)$Suv*co(xfrC9Y>>u|8DstV=$BsJh_ zlNEXLd)k|w5D*mvF@Rbe<~fTU+Xe93=X8q!cEbW8cnn~Xn&lYax;njiOy&Z`#*dUH zjZG}7G(l~GS<(82ZKD>pAvpDe|D+}JC9amsXe&we!*}VTkjq9CZkeyOwQz^>hX1U- zFFwMh<(ut^ILffhVWH_(8ez9anoY=ejBS0TUgXDpMrp@!0St|d9O;+p)=9${i?lEN zEfVujPfH7~#X^y5)UCs=R+pGZ%Z(6um3VDzax$Duiy@`N;s4+chX98Bb=0u!lDzC( zt}a)V3Os}4N&r)B;kpLJr~<zkC}1-I|82Rjl&u%o;TXIsSq|N!#aonZL@PZeTgFf(S(C^H&24{{GbcWj z(PyIoaVMyIpLzJ9K?mLBjRYF>`8 zLZW8W)*4~@Tdc(dP)cG&IW9D{eIbwpKGg1R+yYfvtT#FE=u;`zf$c;s2O;NZw zG1n|-^!7}(vloI{Qy&xJ!DCn>U*E$Kdvjegob(sfILuz6g00AwkK*`aaQ#B^$>OOt zEWM;DjdFL%3)5BZ3(fZhVZPDF7JdD)3>uEke7}N70MHA$p+3*VN4vs{PjRkmUj5+9 zUUz{gKgr4z30ze>FHO-g@%NxdNcTK}&-IIg%x4Vi`IAh_snJ_G9~wffXL(lUEiMlG zo)E}e4zC{zB@e9wO0PW46~7$c)Oo_79{0K7yg2?mx2JbgbixibS<}LMoW$U|8i&wJ zqo(cIvV5STIqUpzN#U-_rVjL0zC|{*1snM=21dM6{o9P>O8_7bGQ|I;dZ(s@d_Kg5 z+bldOR3CLAXx&GRfjwicIo=P|h)0J=af*xAn5&vNI63i95`Jbvc8w7B$Caq&rqJSf z>4YXWLjN=2ulJ0ZQh8ca>r!fQUQ!{_YZ(Ve2k=UE!w0YtYIn9*zv?#`V3x!_tLEDz z@VTbytCey2kFQWzA3S#cmevyla6O_^)w=sU(fEnr-43x z^$FbY8I(aL_XhS{fBdFP)$l09vIJb$yb&R2UESAbXf)&A)rLJT(`w%`Ym^ zr3cM|m6hp8Z3p?hB>S<0Z#ZhK?!8PGICIiI|EwJv+w}w0Vptvafv>zg_+xW9H>sKb z`Mq21tUlz{^@(PHwUbsnFJ7G~lVu5EfN;uO4r=npzZKdtr4Nmk1T8u4eSNT;9Ej63 zYg{)yhFYe?2Y`+tH^6al8Kp@1qK%&RpiRFSz--52-qO65O2$$>MlH0mXNAEi{a9CY6qiDrcxgECh-NLf~}2E3dH|n+zktRr4&NJNVS6 zPdn0_J_pv2L?qYlAeYbUv#VXBr?^pe2*V9R=>^vby{M1N9V8*3<*VbHI&H7h?%1~l zXx7@B>>Hf=UH}}fo$%~a2`OKGthoZM%CFf!%sRNsZ3cM?8fGW)IWE4bun)Y+)(A|J zxc(*MA&@cXMK)a56YzMBvZy1l94}xa>>yfyp<+JOedJ9jrjOh~ANOC@vs(TQU`TKN zLS6hP^GWAR!z&XFYVZF*ssW%)B zy^2?)@Q3+*s@&stuHbGelTc=ha|ZXN1Flu34Mx@$mL8Mg~IOm z!gj!t;{k@zv*A293Bkm)hrT&gXG{~*cs_w(nMrd2%ux6=)+CxWh$ys7Rboq&+W7Y{ z#^Ke#;I4OS4tB+FE>d;4jQY(s*7$Nc!}}}ZPzleSzs?Z~N}o0F&Yfm2{;;DQuipn`doCMeO@x1NvM49N3ePi93v{UYXI;xJwD4D!6V?w*xa zkR%2wC~f+4HClEPco@fS0Bf++`>Rt!o^y?Ro6V6-AI!njWXUAOY!1Rn3DUQ3--hxX zOLQa|d6H4~{WjiX*!rGKf=Q~d^ybrp0P~4(vo$PaQ2jFMpZB*Zy>Qr;lSaHD=+|HY z8OE)mkpJJAgF?OXoL?GY+gesQ<(CxF7I@#Qt_B40=vpHhZ24OhK;+(T?gA0aAamE} zf)0`o`_FaWDDZrgeS7X82w{-_g-(<3J30IZI+Ugrc&c2MLJeKiog{u zDKAgb8Moo2dv|cE#c|l(azbzbrId#b1jAm$=Z$IRYG`vVG|EkZ1!D$Yg8a6mTH_LN zCb)92hZyb-INH8eUL+u%sRx?Mz8c1+Q!yFEf*zV*#p_EX93truc`OGNeQfLc#^lH0 zz|~bPtOgXt=1NdfwES5^H;ciJ2G{kVgc8Y~RZVV&k(o>G zBKNEUiC=6ll{mPLg>&&ZOAd7f!5;dBRkDt@SW{vMRZ-22lOk>5#Z`PLU^9^V_iWd| zI{aW(BmV(!?yXa-IfDIo|gh+FE0H8WI5vN5|KQ`TA6+_q6zp?RypgM z`EB|L6I@FQL_UzipJMCPbdhe+(LG0<!OQ$wqIe3%W1a%6cw%X*SASdv%eEQ^EH43^2HGdLjnoLKhn3SPgc7wjQR=DbRVzNLYUq)YGa z-Rd~v>Os5}V7zwr_A?+$?M>jGd7O3sG2EZl^Av@W^fp5uD#yrT^qaoJBc0-=GG&X) zCn{@SbDnAvFo(%!nlky%?J-wjN_@ya-;*VM9X)&K0eM%@hl+X4KT4AF{dLqcf4YK`#mQGA$gEP>E-LUDG6M947$GaaiIP_jG|Q_S(37#yZY<@ zwD;?i0G*M>^}M3A^40HjZcAy?$H^<}PmQI|ntIL$5f=1ZB3wv-nAt zG!+p{g#tSnIBVZk_;TAFsQ5IRz-L@*XB7c3{Ic^Mhs7$a*vxA$FK!5>9*je;`sV4a z>g0VOjpB(00XBi_-kP0@n_1^509o0pdfo6Vq4#Jw>4<#d$|>1agsRg&gYR3GU$e6;(SQUHaMlP^)S4SWUSj zvhKRVT#T=)tLvAprg2cS$d{--5^p#5Zup0{9LgKV(YLSTVT7P^GNjbWw_DtiyL+?% z0~WR!UI-`l7WrQ+?Wh7bM!(!uK}F{r`U-WCf>IPnjs%787l^MM+OL0Re;5DjOYwDZ z-L2T37dRMEBZ$cmdx-KWL0XngOa2C2NyjW zKd|Z=Nc}Yh$9VS`E#FIBTi;3qNAOn18szcl?ZT;K7c3A*Vv+Zgk6t0Sf z2VXBmAVs6K8w)qs5_=^IITng6AFbhS`KkU;yRekXrM4br{enyY?o5Dnj29^i_aw$o zYrhZK^d|C}++`x0l;8%*_UB*t%X#^9ZHf$MN4`q!F`l;s)2e_G%D;YDJAAw6bfRZkF`aX;I|>cr z4(rpyQzsmVOB!mPXmJ;ICEOV{hsXTo(v9@W(Ss+^U~0H@mjQKFj%XUDdQ8O899B>Y@q5C8Y-G;@ZM*zq5grx4VoL6 zE{f0YySWY{C-H4ldS=v5LjPB3C;RGZ(sSzk+CT)2b1(xIarIL-U@cZ-OarD+1iwV^ z+$C5)iE_gEEd!r`B-2Bx={^6N*%vql7U@+LdN3dcEd`Rd!6GHhFJ5_B1jfuGhrpIk z6ZY8l--qyqLXJT8KZFRTa2Pb^gz_i4ptnDoF={i-%6wilnw6YgWmfim*oHYG^&|0z zhIRQlLmr(_S|##8inL-$#+KkdvF5ENqkEggA(VnasX-6Ybwu-e|2YBhs*5FxcYc=XW{E% z;PVRaq2YqtrBL@@RHAzFd0$D*#306GEk016S&upNB!`|ANBqTu@|QN5juZ>9vq8{| zD_mFmlLgtUEIOluBijGT^od6+n4=t`Vi^!k)>?5WZTV6#gF^aXAKVo=4yo^FVYwFY zmFRI2nguHsf5O#;=^~^m8O9_X6}s-4>0TWK>)TwO4gdFG@wcT1EEj!#)QGYPy*mNZ zq3a43miw&+7Hp>9aaenao@bFQOk`vuBF^|N=UybZZ41VJPiW`#ihOMpx&m6krov9T zOyAHPn1eed-*-ntKHU@?g^>DT$DCYYa{R_6CYUmrTE~%MjFE>7DHUe?iUObSZv4BB z6L@oRXo1189WC~k^uOVfUpFka9=9-2aTyeRVhvDG08(41_$z9P#kyUTA0rP3OlQSp zU!Ad^dAHbDtB-q5^($l`@v)!g2LUgd?AJA0=jOls|9ZLJ1dfk2=x_R7#DfpfKXI4S zb+fGEG9D8Dc#AHOIcUokr9`v17W(b@8ywfFz~ VhJ}Bk^56}?8#xtN@hj7y{{gD*=nMb= literal 13512 zcmaib1yCLTlP6A)KyZ7w26uN2?ye6fI6Q*8ySuvum*5cG-QAr84G(z0=C}Lby}RA3 zyQ!L*o}T`8f4aX_Q`0h$DoQdRkqMEZprAg=evwrBdwu(R7Jop1f`Z-(O#TH01xIBg zA)z8GAwj0%;%I4OX8{GpixH~16pj&l6cmgt&3mTBYnA$b$fZ7F~ly{~}2K5y8}Z?N48`ta>qZNuk{5fJ5v{07g}vT(oVK$L>XZ z0o(jQ*t`Q=FL1-bRKukE=3sivjnE1yk&7V3&C*}s)O_ed5bRpDWcdL!hQ%LE-qUm- z>Y%VpY^q5Dy3Fjc#5@0Bdq-!^cDuhc4AyzcDQ!1vLyz2M_@mJiDYDXX54e`EH?C`! z1^UB-O?++X&n6-g8dnD2gv%-PpVV_ziOp)E60}l)Ni$ zR+vpCcmcn)z|E^>r->D#>NdN;-Jf#4RhhlzW9TYkeEiiZl)sDQh|{=r{1v!9TD|4Xb7ok z(x$FOdJ+`(Wi1S_SRm}QKE?RY^aqmOCqZCLNq*0Kl9=qL`B)R;s`}z{;&lKt_aW_5) zx#_E7i#O8^96l#HH9{`J%CN>zNHqvocOyT!OpSYVc=evL{F_0}QCgG-e_j;rm&o7I z%7Y{O956rfy1+(Yn)))>hqq)bR(}3ndqxH7QI7pwerGMIdt^}d?3AG&!VX%IZ1uNg z?r|LLwYsXh&qzMwZS00iGT}H=oH#Q_blmQ3-Tl$UR6W5Iai?P2FFjJ)nxU%&H0d?vrY1^O}P5%8!v zLn|pj%0u{G%R?w&O!1C8XK^Q@3+3!#N5p{j_vL})^hMVd3JT>fje-RYm6d}J1qFj- zqoD)RQB>eJbF^m$nmd|WFnicL{gsA-67=BzOWIq2fMg!_b`Gxm9zqoVF2Vno{)f#% zLH6$=AX^~{9Yqx~2}c(TGA?E|W_Ai;WHK@`K^JpNelgUw|3k@t>XEc?HFL3X0@*k^ko}_< zXzJ(&5~85^$IyQr|H5hS^k0S?T>s_OU;kJ??1O$7&%T2?Hmex8=1xdTXC{IMpm-J%KNDh)BTCHzm^h)nM*~^AF-<+k-~q%N`#dAwj?Gvvdw9wLgRB*SFnLO-GXEE+c zXSqAtk!xaU>A+n$ytr!xzCOE?epNC*c0Z>0^Iz|$^R{)CEH5(wiB(-f;L9`k$lQ5E z*?HxRifJjJJ%^!B`Vp^ zFz^|c+N8e(pCo6fFu>1XWsNrc*!ASx~J7z(4oTU#&4KM%I#ot{QMV5IpeJ-W%mB<%g1 zPG0fl$l84~I{=rF%nUk*-h-At@Fj+bz(KJ)Z&vBNvFrVbW{bu_J1KK8q@MXQEU2Zu=Q#OXf+Po@- zc1nSbq#5d->O%ltM6{fMBRuF!*tPeISd(K4lLPbnpV==Gg#|@`I4Oe@BUIrelz9<6 zm^SG~lfl6HObiox16?jloOEspuWXI;Te z)86F>db;%Osnfb5s|A;a?Q4jJ-gsI=hSY*mn-L~9ODzUcH4C!dO+fQUQ7iZ z72mbtAiFg`rYR#MGZ2b17`shlSSUyZ*EPLru%xjDSr}YVKFXqV1x+Cm#n~cCU^`i4 zn0!JAR4B;{11-#OCjwVVC=YR~`^_WRq3{Ns=#3&MC+5d5)9*gxB@;Y5Z!Zvylp|1? zDO)pY059K+*FLx{DzuR7xUG%`)}lD?HM#VqR!%g-((d+2%~JDZ@6yX0&}5>^le9W# zkdU15UAmxmA$3I9W|eUahFa}uifZ8|r31Bx`0Q=;pk~lz%P6+mqNqnzx)WY=q+xV|)kMxO9!B#&8@p zwL-OyP|bU0$}}qcnTT=%7-*a?4&Nx(_mUly8XqrZvJOG14^gx0RrNlp=(E+8%zP$0 zTZD%VgWQDA)XrjGPwu+=Yga?KX*4uMDX6#mt{kw1)vtJi8HCfFf!@u z=B+D$_-*WUap`;XxjI>p$b-Ei>aqIoU4yg}mHitMQP5o^g_H@SfiNA`GO6ViHX=Xr zOwUA*_a4v)5L1(0^h^-yZlP*xS&~3SXemGDX~!Bty=u_i`~07@zRl8q-IJ7-#V-2% zyIJ`6K>Vn{F4^sPBWjx*SBi}n0eh_T#V-xg00qk z4A`tv>hR`{sI=}Bmm8!0-t<08*21 zlF#N@sB4U2U~-&u>S~)#P7SgV8vAs?&Nic81aoyDb?-0JrMbDei8TgeRQKT9vPF%f zqyEyk`gF^ax2`+$ijf(P$<#oN%;k3L7gO0)Jv?D#YxoQan226>ujmBpUiZb8X`=#kaj^7D`&d?s-IPCEGQTBjmDYL>zA zdzk$WTHogVcnpdDHt6(`+l}XBLhvwgGVSJC-lMNlxDx;OQ`+Jg7VUPStsUM&SFMxO z$wO2dQq{KgA`07)P`7iC5sdspKq`k)68#$a>O`2Oc-wy}bT{zuWF z_=ldNyPVNl$x&w~`x|C_H|J?p=ffoe%E#Z-4qGf|dr`0mC*RuQq0Ab3XK-Y^q;4Y; zBCrF>E>=rQ8d`IxM*Hs}aPEH-DN@Ia8!r;9>2Y0X#Z(YsAgKtkBZ=+12(>3MG=0{E zLENCGv*;*fQ8vjzn}B{HF)Y85($~+fMgu+HjDx0&@7v3J7`vs4f$04(tt~jzQtGaXNvU$F|-r*$peg^UQ@J)m8Ol2_VzxIrO!+Fl-jVF?Cl3(A}1F^4QqI2$j-#y7#zta zO%U><_Q+)cX6BFTrWBQWV@ zxgX)xmCAB^muWw_wH_stKxC)NH=meuV{a5#S3{{YKtZ8u5@i7ViC2*d5%db3x& zlM>IZt-}f1k2u^i=NG1MR4cK9ojh(_?O>&mK6C3R`c&CeGcp3`g1v|rboE|^t%$L} zxFtu^o#^*D`o?7jlPjN(ofhx+!o1}pm%cM2zB>OSW^<*l(3x`WIOQ;A-kv!C^4xG( z@x&j_|3O{*^(*_Y1q*@&UW29kgnfgLupV|Y?Do>8i;z#h^i33TJK>u#V^4NYk=JA( zG+NYbWTNRQF(-bdKx=V4uARZu0A5TV&-#J1sWMM)aDsLt@bdVBlo7smI4W8^1 zGLBTgvPL_%$ve&WVA`dOuMtlaEV!}}vK_5MK=*k0u0prK5^xCfiOsnl*UsYYy?C3N zR~Hj%&9v9jNSJ;7GP$KSM)4c8jQmic^HU~|yqUSpufPGM=3psu!nMM=%+$r^jaV(si>=!>tU zchf4kTiCTZ?CBdaU9V_ReoatVjvHG6U*}5l%@Be7heudTyC5-piT1epfi<#{^^Qgs zE)XjLyQl`(ac(wIj8hlXL6Kq75pk!tH$d&7fy%U~dI}X2Tu&Wyg9MKs8lv3a4jDSx zlSasZ9y!VfsYELQN)2MRAfD}VLDVM2@^LH8rtSktTNt^oQ?XKi{uq6oWu_`~Ail;|yZ12{ zGHQJg<+sarO1N$(TSAGr_zjq0RxYf-%m%Ei<7i*Y8!YCg`vr&7^KeS?L=_qRu3|Wt zJ7JNF-hwYL|Cv_5bt4U%@YwXwWj~Fq^~#PC8L;Hc$>*k82B? zQA1xhU|~}DZcI8!Mv-V2HZseBtMa9d-Uz@Zgy{qZ?OYP#@U1if+mqZwT8?>kIV$lLJj(~XiFuliBhHzW+DCCOf-*rk z^~|z!&rhv~>IH5VS=VRIN4E}aVGKCeKoV#pS~Kd=QD6{o(1dXzGI5Uv%-8GFTjLN5 zv!G3XQSR@TB9bZhMt%THAMr-DCUoGzs;63fM*_40W`af?zgMagaD8!h>MeJx@zNrmBMN+jz>m zJRYrZrKI*|f~ui;eM!`5u68bOnMi3^kn{PRRUEQ>Pt}{RmOkcpac9u^5Lq2#$ESDr zME76h_S6eIIp_TzDCHa2ZJI3=a555|yIAS{2-QsFf%P$L(~4$((P1yv zin2s1cxJGWZ*VP7eu-4hT>ftRneoDgRC^ktx#Px^ ztE!e+f^a5(+s!>u*uoZ_PcCI3klN$dw$N==rPj@#9z+kO592)0ssrmu98g#a*gcU9 z(9TC~4r9fPV8*9W%Uu0NPbs`?5p^LnU3`h}yG!Je(SmM(ubD~scrrMsm@K93q_VWp& zGB~E%If{5Mz0_|d;|?-H1X-_Q{z@7Qwl=}`WuBiIZ+Bi_{nU?F8GSc4jnWR7e7atq z_fuPX)h=?`S?NEFtgA%+tpr4&@W$2>jHs5B3kyrILCHkUd@Eo()*N5Gch59UX zLw`Tx!{mvK;Fk`6-uPi*jWBBoFoQ{-P~72I3J+oB9HK;XymZyeqR#I(RwAe*-`$n= zJc|BvNubqX(Gf${yR_RByG9dedM;x)`EfY*;Q~>hlZOwml>)Zi?D<`@+@J9p?DZZkKRo@WO_l zq=L&;wDHn_OF9R$nxKokESezWyD_4I7u9R_c?w^#?aw*xd<=mtrMCp}Af3Oo_izxQ>u|;p%6<>P|Dzy5UlwN}QDAFm-VtxtZ1Uch zVu{mpCEykVx9s}`Rwd}s1oyzb-AxD)?)Uib?6gO2xzRAjGf#;UuBe__x8;-wpxDh^ z0rsaZh)&99k$oo>^VenViz=CdSD2(N%|`sN3^(%WR%i3vb7-4w`(?YsSu0IL;}y8P z4izm`bd!vRoE2*9ho3Uf*AxTi zwVUR13!vz&VXRz|$dNRe{x7{K=^?fUKfb-hEQWtr;kCOeVuLTLlX2mr!8<5bezGu< zSc&9p`Lp{;&_v_6#S42GJj?A3`WEBMMu(W2gQ`x=bs*7XHNyavc{LCr*1yZU63r^p zL03F2S}YV=%Myt0Ma|v8JAIrLhCy@*^1h6lh7 zxe?Y`HE^Yzn=Up(mkm$-tqJoBTV)WIacrgW%7(X=BzSBOB8@FzFUx~P;}BF<=z+wJ ze~V~xrl{(8S?6444v~vH!Kq^Yr+}o6-#cx$qubK`?J?UOOuCVXeuk0v_qF}o{W5_S zWM$~J{V!|aFoy1QbJEWF>=K~OQcAoPdKsrafFO)gpK!mrI~K)7gz3WS*rsr3H~@3) zyg9z5o8=3bPRV{9ro=7U1fH(S|s(rgXql0@_THcZvhKSs({E?jmDPkEHs{*@-3WrS;u46*7R zi#Tp(HYmRK)dS$t8o;SU`^}_mm#B_5!O-Lw4zD#Y>PpKC47Wt#HI5yY%q8$r`+S!y z9IAxJl0b;_%KRR?%CgmnQ$AJ}W947R^ThPxYv2|)3r9CBzrjYQ9}g5Cai(FSrgR!T zqe)j<-Q*ZT9EyA%^(mlTG)Z|fd{YBk1Bd7rMJ+0HR}1WyJEaG8Zzm{(M~b0;W;Ezd z5?U8|Aq||YLeric%Jll$7__6Du^H$uwGI#6s|7m>jif&SyDq1{KCc9aNt~A9I%d0I zclQTGCNmGS5^z5?q#3S2)muv_blZ@F}ac9~GGgeo5S+4C+DV@=NBp9#%^?Uq} zQ_ba?dEo*k=C5`%B2oFDg1>~>UR5TWACTj81C+1{p`2uipjr`lq}SP4mE%u`=6VJF zx@ryvo?kQ!FbdB_g|ZETMGuO>LM4THi!J({4!916!Ca}16B0)s6|bAQyqB=i2|M%N zDr1RR`!r0fbL`p;_ZDpuEKY6g#}B7IW1%r~^o}GL%}pGEgh+t;BrDR%kqE=Fsao2~ zBTm8&!pHi0+2}A~i4(m|>?#Hd2*gPUWfLnyFlmbN1Tx8)0bKEBe5;!A%;o-aU{-E9 zhFn|1U8R;`qgH~1ZW3%FF_3Zr+N>4QdZ<~RtIvu{eO35@q3;GwTD3!2v%%fxA+8lp z3rPSWYDkg01ZP+d*HJ^9E6pwi+LrUE^rwvT1r-G;y_Q9|$@{fo2N{xls*aXUi*WfF z_qk53IEP@HRyH|In8MC@b!8&kPlCo0)B3+SW9ZBz)3K+ybhE7h715?DA}@ep%n{{w zbGmUHc?sC=8BbK366hx;!mv3*GZNxaOzHTq7st_Oq=k-eu6c()@kb5G%LMEmmT{~& zbw+EQrYtU;hVOszsm%Eh`;uQ4lI&2$3@=a_PjU#LrI@{4Mvbz){2^>Aquk#)>DvtH zH8#H|#zg}$Lps`Tft4O50|&ZkDl`p&_*pFdGx@N&wtmU&w~%ULc{s!z_3qK4qhryN ztlrH_z1sD^U+?LH!C$dLNe63>VB-3XnLQDfXpluB3@6|Lp}hvTs8x#CexEwiPwc+njYfl+;lu=3;350O>rw!!9CN4_iJ=eU*6bBo4QHd1;g{m?$~G{lc0`R0Gh zRnC$&xqS&==Zn6<%`N83t4D=0A|sx9hpt?=d#BGAGlNiG1$VEn4y7uyOE}`dNkntK z^8ozr_hUn~(^dBx*wn`d(`ex5s@2zE*cXg7S^AoyeI50rB=Xin}<2(6gI8#__A_ zjR1zbWaBE}!@G4Y^2(=n*}Ax>W(D`8C*-{CCqTn+d?M-2+OH-cNfo#sfDFw)nBP zDU*6_ORs;yuG#OR+6}4v8_aMwi!cAj>5=l+q;&JRANVZ<#_D)cq%-F{M@ExBz<@8b zzZ3VDmxG0qpDNKb0L!}QeD@OSA$KMBCpgUX>HV(nIAw34ZB-)f%Of!8=!o;^-PW1+ z6H;u%dAO6)z&Yi3k(fOa-%LBFi?fT<=AwO4X(XvE%V}tfWADj)ATGM!?w9oJ51$RDU&WB8_IYTD2lC^NZ)^Fct!r63M%01t`*nj!Y zS&AuT?#sG_k9WyL`$!Px${6_d)Xu)!e(=+wJk7vI>~waH=fl`4J%-ZSm;G$hBIv0+ z)gok@T^nnR((R-ELp=Z^TX^ve@wsen);$}&9NL+6H2D(FVidz@1`o&hOWYCvkKaBe`;(33V} zf+D0}^W3FIpRJT_X7k_CPaqOXM;M74&7Dr({8x5bWyJ_ zuK|~>x@Bi^iQP2t4okCDDE7Y@(Tz~}lmq%G@C}pNm#Njbyvm!Ymq{gl_MTz7r?;`D z{w&VZoq4M$HzJ9><+0h?OClj9jEf>2Z}_mogxH9U0*dwbv8`nbf*sFroOxiGZyvrV z|DO5=mN9fQSH0<8B?+_~4&)?wA}(Q57iHNt*!F%`T<4$g$HzE>yp|*KX2eq1%MfqF$zAoX7EHhi$JVR%?S_Q#Nji6Pgzf1+^Ze{=UGz$wo6m4+# zXFBxwx)7CgDSrZ`o?Sj@ zriG;V0R*7adDB@G-QYanc@+IgT%x9ikxDOhZxc6>1zMbQ99F19M0{iCS%@WxP_+Ce zj)?TTAHYi3q}_*i#WJ;XdP7p(0lFdRa=c9y@`D7^(3dX@UwlKg2FD)2djU$zom6;` zg3IO)9#S*D_r@$n*^d!3oN%POHieKHocBe2T8Byby-YDDAT;3V&(R~g+?S-ZQxP3B z{2Ca=wL6;=3=r-SS|9cjXUpHA0{v~>e7SrH$@*KK%@xP*;Bw`hf(7Y4bQsWdCuirg z)5#cZ?m_Gaygb;#G`h@}mA@_Ot*>g%rxNos!n6R9*SqOtawpM9iP~)nbiX^5H|Z!x zj%1fc+xPegK1$KKCbRct>$&*bVLlOFe1B%N5WV7VPJt1*i++X$jg#BIsnf)_9BLj? z;q7iX{EMjW4x$9K&oji&i`*m|8F~t+J#o?LK`^zS5Ed>RF$E?G+mmCPVyN<3evj=vm~LaY=vA-9MrE#=7p_ z9gC=;q++DU$m;m0ev~_n5$E--5t%&Vo&F)2lQmbf>ag zCy&xftNXWabX^6#M!Sit&2RQXLlc~xh9ejq^6w}?1Tb+P?TS7k=eeHE>qNO2oB6 z?$O&fVR0)-Z^BxtVfgL(2@$Gy@;e3hiO|l)M?1Mvw0z8-YyLCIkdOsC6+{yA^YAR? zMyId-+hr+}E2;ikh|vc3h0`)`5BuU3O9%a*7gAD_VjUBT9LB-A&r3yLO=KL3t-b;$ z5p=Lv6m1bol_r|%CS0$(a_$D~B}o$(H=%vIV2SB#tkDC!wM5Pf^7pPF z{1CAybn|j05?b~0`QoKiRYjgZ)0zpj;<0cmntL$H>EkfXe`hfI7`W19FD9@T3e5G}Tz3I;UdsfP*JwJa8UW>AziikpaqnrZ6guJfnSMs0 z!?Esckc`={Ti+vR^xI{%l+J~88XB~|Ue6^L{FpQM{76_lDa_!n68xA;`+{k!p*K+} z$e_fAANotrZL5+amb?x=nR%;3MyAyVJMV@^&Bt80#d#kk3p?DnYcC09#k|ahZ2M=N zOcDlH6$YLXjkuextt#Z!J@I}%13udEXTzliIYlt$M6&4tQ7Pop;b>)10Cd_ zh=PS$7E+a?e0(kGcGRtpS{;O;3pX!eOSnMp?YmIKoGT}F z>jV2@I%Y2-T>!!Ykp-MCCN6+kkB_MWFs*Sgzm{5q7@ucQ2d!UWzTR{BD?D4#dQxRm zOULGFmoN9zpm;P&yd0kQ4YN+kPi(i?Mz<%!v(+EUv_M-D79&3PmfmPIDs!nU66NC8 z2im&u?=+p~0=Ip5*8v} zGfUlw?f{dtG=mb8n42}oRxuh4EsU|XB~kqg-*ze;0>yVq&lA4GfLsRQ6+F3s#1F-c zBU~;vl031H8Drhf%UJZ{{jTY!^0EA`vw0}szBi9SW7V-6aVGz@0AZz=*y02aEbhGz z(5)@OC4mQILIK`bWt*Ka9K4>y&d&_oCPHLnRo-t?S>bK8It%adI4&iN%9DK-GVW-U z{oNCi3ujl87zB|%7Gm^U$0Ryy4q0Tl<@tBNiQt{&5kt1Oy;pm}Z~_Ni_2$PBL52wDuYpe7mIO*HK_Wf0 z-~vF~B(Nyo!)FBBF)EmmET7wjbx$fMZXtbV%{wx4d%3n_pNP|2JL6 z{G2maW|Fwb?9)XTN2)XazTD6VU0k;N_tPPx+dgS3FFwBCtv}ChR`XE!uwo&}pql1S zSt&>WyEL~JI#`r1ZV`beorbjYi#Z~8AI)cvtYNncI6R#%NA0gsdq4S=cbm&)&gb=t ztC<0+#Iqst0|>CbzEUh|in6eZ{-C#n5emDP%5dI-YazF#G}M9sQhr4C_k7jbTMJe7 zZ=88Z`L^f`6e1u>qadU#$FGBQ(V@5&70$)9AG}y$h4@-OMfyDK!9SRWhwTtgw%TBv z1rdcQ-OU*~aOUyzX4Xm*F>4M4?q<+JWrxP-X{o%=3LL6IlWZ+;iDfp|rlD-M=E$22W{5#a9=d zo*`MAz%~y_WLC^=YL1)37U2>+OGxxpHiV0x1;XAb($~GlPx@yy_9?J97bq3Wq!b$( zxpZ-VbTN#d#uC$w#DxXmRRWn|Q-|1bh^`?xx`B^!3Wa$AE&=@Ja;o2TO9E|qO$f@y z36@ds-Qs7@r(k+c50=_*mT9j#C{<$P>LxU`hd+ij_S*h99psK?N3g#+N$1b&BB2MD z=__!1spco}-;7q{I#+Tu?4{l;`YSM{ss7lvXHt0^j#73LV6dsYn~1AHOIuj7Sws+E z3wPV7)9{JmT%9S2ife0hbsy6h()Hgk!y&($YF{wIj=&)47Q~HM?@dvh#T4I;-W8AmH1; zrg?j_eV?>&-~RKTBR8dD=)Q>%?m5hP*@X~~QH!t+v7=8eYYm}kL)X^TRN-z_?qdu} z)vlf!b!ZODXJU^xcX;lNkF#W5czG3UbLz^ECdlW-;?_s?I{T?3&L7Cp`_|GcYGXN(vgWkHjybbz0)ES`mQg zl4&Bxu4uykOo5MM3c0c^uxkFB-<6^cTns=*6b)bTbwCH5y}El+Y1Z#>1CI$e3eD__ zQZW&B%f$qUZG1)hvY1c;^QQN82y@ovVtXp%$5x^p8@b?k^m z`*n@?W?R~6u&OuOTK5h6)sul0;A+NKoyXK8Q4L5cj_-Fcy(=008@;^9vabc-CmeqZ iDN=@>|GT<3(z9PLOZ%x3pZ@uGKvqggvR2$A=-&XFT_Ys` From 8cf33fb8a945e8f76833f68fc99b1ad5dee13641 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 11 Jun 2016 12:55:38 +0100 Subject: [PATCH 0598/1470] [SPARK-15883][MLLIB][DOCS] Fix broken links in mllib documents ## What changes were proposed in this pull request? This issue fixes all broken links on Spark 2.0 preview MLLib documents. Also, this contains some editorial change. **Fix broken links** * mllib-data-types.md * mllib-decision-tree.md * mllib-ensembles.md * mllib-feature-extraction.md * mllib-pmml-model-export.md * mllib-statistics.md **Fix malformed section header and scala coding style** * mllib-linear-methods.md **Replace indirect forward links with direct one** * ml-classification-regression.md ## How was this patch tested? Manual tests (with `cd docs; jekyll build`.) Author: Dongjoon Hyun Closes #13608 from dongjoon-hyun/SPARK-15883. (cherry picked from commit ad102af169c7344b30d3b84aa16452fcdc22542c) Signed-off-by: Sean Owen --- docs/ml-classification-regression.md | 4 ++-- docs/mllib-data-types.md | 16 ++++++---------- docs/mllib-decision-tree.md | 6 +++--- docs/mllib-ensembles.md | 6 +++--- docs/mllib-feature-extraction.md | 2 +- docs/mllib-linear-methods.md | 10 +++++----- docs/mllib-pmml-model-export.md | 2 +- docs/mllib-statistics.md | 8 ++++---- 8 files changed, 25 insertions(+), 29 deletions(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index 88457d4bb1627..d7e5521cbcb29 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -815,7 +815,7 @@ The main differences between this API and the [original MLlib ensembles API](mll ## Random Forests [Random forests](http://en.wikipedia.org/wiki/Random_forest) -are ensembles of [decision trees](ml-decision-tree.html). +are ensembles of [decision trees](ml-classification-regression.html#decision-trees). Random forests combine many decision trees in order to reduce the risk of overfitting. The `spark.ml` implementation supports random forests for binary and multiclass classification and for regression, using both continuous and categorical features. @@ -896,7 +896,7 @@ All output columns are optional; to exclude an output column, set its correspond ## Gradient-Boosted Trees (GBTs) [Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting) -are ensembles of [decision trees](ml-decision-tree.html). +are ensembles of [decision trees](ml-classification-regression.html#decision-trees). GBTs iteratively train decision trees in order to minimize a loss function. The `spark.ml` implementation supports GBTs for binary classification and for regression, using both continuous and categorical features. diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index 2ffe0f1c2b311..ef56aebbc3608 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -33,7 +33,7 @@ implementations: [`DenseVector`](api/scala/index.html#org.apache.spark.mllib.lin using the factory methods implemented in [`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) to create local vectors. -Refer to the [`Vector` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors) for details on the API. +Refer to the [`Vector` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) for details on the API. {% highlight scala %} import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -199,7 +199,7 @@ After loading, the feature indices are converted to zero-based. [`MLUtils.loadLibSVMFile`](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) reads training examples stored in LIBSVM format. -Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils) for details on the API. +Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) for details on the API. {% highlight scala %} import org.apache.spark.mllib.regression.LabeledPoint @@ -264,7 +264,7 @@ We recommend using the factory methods implemented in [`Matrices`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices$) to create local matrices. Remember, local matrices in MLlib are stored in column-major order. -Refer to the [`Matrix` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix) and [`Matrices` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices) for details on the API. +Refer to the [`Matrix` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix) and [`Matrices` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices$) for details on the API. {% highlight scala %} import org.apache.spark.mllib.linalg.{Matrix, Matrices} @@ -331,7 +331,7 @@ sm = Matrices.sparse(3, 2, [0, 1, 3], [0, 2, 1], [9, 6, 8]) A distributed matrix has long-typed row and column indices and double-typed values, stored distributively in one or more RDDs. It is very important to choose the right format to store large and distributed matrices. Converting a distributed matrix to a different format may require a -global shuffle, which is quite expensive. Three types of distributed matrices have been implemented +global shuffle, which is quite expensive. Four types of distributed matrices have been implemented so far. The basic type is called `RowMatrix`. A `RowMatrix` is a row-oriented distributed @@ -344,6 +344,8 @@ An `IndexedRowMatrix` is similar to a `RowMatrix` but with row indices, which can be used for identifying rows and executing joins. A `CoordinateMatrix` is a distributed matrix stored in [coordinate list (COO)](https://en.wikipedia.org/wiki/Sparse_matrix#Coordinate_list_.28COO.29) format, backed by an RDD of its entries. +A `BlockMatrix` is a distributed matrix backed by an RDD of `MatrixBlock` +which is a tuple of `(Int, Int, Matrix)`. ***Note*** @@ -535,12 +537,6 @@ rowsRDD = mat.rows # Convert to a RowMatrix by dropping the row indices. rowMat = mat.toRowMatrix() - -# Convert to a CoordinateMatrix. -coordinateMat = mat.toCoordinateMatrix() - -# Convert to a BlockMatrix. -blockMat = mat.toBlockMatrix() {% endhighlight %}
    diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 9af48357b3dfc..11f5de1fc95ee 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -136,7 +136,7 @@ When tuning these parameters, be careful to validate on held-out test data to av * **`maxDepth`**: Maximum depth of a tree. Deeper trees are more expressive (potentially allowing higher accuracy), but they are also more costly to train and are more likely to overfit. -* **`minInstancesPerNode`**: For a node to be split further, each of its children must receive at least this number of training instances. This is commonly used with [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) since those are often trained deeper than individual trees. +* **`minInstancesPerNode`**: For a node to be split further, each of its children must receive at least this number of training instances. This is commonly used with [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) since those are often trained deeper than individual trees. * **`minInfoGain`**: For a node to be split further, the split must improve at least this much (in terms of information gain). @@ -152,13 +152,13 @@ These parameters may be tuned. Be careful to validate on held-out test data whe * The default value is conservatively chosen to be 256 MB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. * *Implementation details*: For faster processing, the decision tree algorithm collects statistics about groups of nodes to split (rather than 1 node at a time). The number of nodes which can be handled in one group is determined by the memory requirements (which vary per features). The `maxMemoryInMB` parameter specifies the memory limit in terms of megabytes which each worker can use for these statistics. -* **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. +* **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. * **`impurity`**: Impurity measure (discussed above) used to choose between candidate splits. This measure must match the `algo` parameter. ### Caching and checkpointing -MLlib 1.2 adds several features for scaling up to larger (deeper) trees and tree ensembles. When `maxDepth` is set to be large, it can be useful to turn on node ID caching and checkpointing. These parameters are also useful for [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) when `numTrees` is set to be large. +MLlib 1.2 adds several features for scaling up to larger (deeper) trees and tree ensembles. When `maxDepth` is set to be large, it can be useful to turn on node ID caching and checkpointing. These parameters are also useful for [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) when `numTrees` is set to be large. * **`useNodeIdCache`**: If this is set to true, the algorithm will avoid passing the current model (tree or trees) to executors on each iteration. * This can be useful with deep trees (speeding up computation on workers) and for large Random Forests (reducing communication on each iteration). diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index 2416b6fa0aeb3..5543262a8967c 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -9,7 +9,7 @@ displayTitle: Ensembles - spark.mllib An [ensemble method](http://en.wikipedia.org/wiki/Ensemble_learning) is a learning algorithm which creates a model composed of a set of other base models. -`spark.mllib` supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest). +`spark.mllib` supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$). Both use [decision trees](mllib-decision-tree.html) as their base models. ## Gradient-Boosted Trees vs. Random Forests @@ -96,7 +96,7 @@ The test error is calculated to measure the algorithm accuracy.
    -Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. +Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. {% include_example scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala %}
    @@ -127,7 +127,7 @@ The Mean Squared Error (MSE) is computed at the end to evaluate
    -Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. +Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. {% include_example scala/org/apache/spark/examples/mllib/RandomForestRegressionExample.scala %}
    diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 4c027c84ec90b..67c033e9e4003 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -333,7 +333,7 @@ Details you can read at [dimensionality reduction](mllib-dimensionality-reductio The following code demonstrates how to compute principal components on a `Vector` and use them to project the vectors into a low-dimensional space while keeping associated labels -for calculation a [Linear Regression]((mllib-linear-methods.html)) +for calculation a [Linear Regression](mllib-linear-methods.html)
    diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 63665c49bc972..17d781ac23f81 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -185,10 +185,10 @@ algorithm for 200 iterations. import org.apache.spark.mllib.optimization.L1Updater val svmAlg = new SVMWithSGD() -svmAlg.optimizer. - setNumIterations(200). - setRegParam(0.1). - setUpdater(new L1Updater) +svmAlg.optimizer + .setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater) val modelL1 = svmAlg.run(training) {% endhighlight %} @@ -395,7 +395,7 @@ section of the Spark quick-start guide. Be sure to also include *spark-mllib* to your build file as a dependency. -###Streaming linear regression +### Streaming linear regression When data arrive in a streaming fashion, it is useful to fit regression models online, updating the parameters of the model as new data arrives. `spark.mllib` currently supports diff --git a/docs/mllib-pmml-model-export.md b/docs/mllib-pmml-model-export.md index 58ed5a0e9d702..7f2347dc0b769 100644 --- a/docs/mllib-pmml-model-export.md +++ b/docs/mllib-pmml-model-export.md @@ -47,7 +47,7 @@ To export a supported `model` (see table above) to PMML, simply call `model.toPM As well as exporting the PMML model to a String (`model.toPMML` as in the example above), you can export the PMML model to other formats. -Refer to the [`KMeans` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors) for details on the API. +Refer to the [`KMeans` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) for details on the API. Here a complete example of building a KMeansModel and print it out in PMML format: {% include_example scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala %} diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index 02b81f153bf7f..329855e565b24 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -80,7 +80,7 @@ correlation methods are currently Pearson's and Spearman's correlation. calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. -Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics) for details on the API. +Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) for details on the API. {% include_example scala/org/apache/spark/examples/mllib/CorrelationsExample.scala %}
    @@ -210,7 +210,7 @@ message. run a 1-sample, 2-sided Kolmogorov-Smirnov test. The following example demonstrates how to run and interpret the hypothesis tests. -Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics) for details on the API. +Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) for details on the API. {% include_example scala/org/apache/spark/examples/mllib/HypothesisTestingKolmogorovSmirnovTestExample.scala %}
    @@ -277,12 +277,12 @@ uniform, standard normal, or Poisson.
    -[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs$) provides factory methods to generate random double RDDs or vector RDDs. The following example generates a random double RDD, whose values follows the standard normal distribution `N(0, 1)`, and then map it to `N(1, 4)`. -Refer to the [`RandomRDDs` Scala docs](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) for details on the API. +Refer to the [`RandomRDDs` Scala docs](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs$) for details on the API. {% highlight scala %} import org.apache.spark.SparkContext From 4c7b208ab6a6ae17fa137627c90256d757ad433f Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Sat, 11 Jun 2016 15:12:21 -0700 Subject: [PATCH 0599/1470] [SPARK-15585][SQL] Add doc for turning off quotations ## What changes were proposed in this pull request? This pr is to add doc for turning off quotations because this behavior is different from `com.databricks.spark.csv`. ## How was this patch tested? Check behavior to put an empty string in csv options. Author: Takeshi YAMAMURO Closes #13616 from maropu/SPARK-15585-2. (cherry picked from commit cb5d933d86ac4afd947874f1f1c31c7154cb8249) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 6 ++++-- .../scala/org/apache/spark/sql/DataFrameReader.scala | 4 +++- .../spark/sql/execution/datasources/csv/CSVSuite.scala | 10 ++++++++++ 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 9208a527d29c3..7d1f18611bd63 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -320,7 +320,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non it uses the default value, ``UTF-8``. :param quote: sets the single character used for escaping quoted values where the separator can be part of the value. If None is set, it uses the default - value, ``"``. + value, ``"``. If you would like to turn off quotations, you need to set an + empty string. :param escape: sets the single character used for escaping quotes inside an already quoted value. If None is set, it uses the default value, ``\``. :param comment: sets the single character used for skipping lines beginning with this @@ -804,7 +805,8 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No set, it uses the default value, ``,``. :param quote: sets the single character used for escaping quoted values where the separator can be part of the value. If None is set, it uses the default - value, ``"``. + value, ``"``. If you would like to turn off quotations, you need to set an + empty string. :param escape: sets the single character used for escaping quotes inside an already quoted value. If None is set, it uses the default value, ``\`` :param escapeQuotes: A flag indicating whether values containing quotes should always diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b248583d79e6b..bb5fa2b99fc0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -370,7 +370,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding * type.
  • *
  • `quote` (default `"`): sets the single character used for escaping quoted values where - * the separator can be part of the value.
  • + * the separator can be part of the value. If you would like to turn off quotations, you need to + * set not `null` but an empty string. This behaviour is different form + * `com.databricks.spark.csv`. *
  • `escape` (default `\`): sets the single character used for escaping quotes inside * an already quoted value.
  • *
  • `comment` (default empty string): sets the single character used for skipping lines diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index bc95446387956..f170065132acd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -655,4 +655,14 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(msg.contains("CSV data source does not support array data type")) } } + + test("SPARK-15585 turn off quotations") { + val cars = spark.read + .format("csv") + .option("header", "true") + .option("quote", "") + .load(testFile(carsUnbalancedQuotesFile)) + + verifyCars(cars, withHeader = true, checkValues = false) + } } From 304ec5de34a998f83db5e565b80622184d68e7f7 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sat, 11 Jun 2016 15:26:08 -0700 Subject: [PATCH 0600/1470] [SPARK-15881] Update microbenchmark results for WideSchemaBenchmark ## What changes were proposed in this pull request? These were not updated after performance improvements. To make updating them easier, I also moved the results from inline comments out into a file, which is auto-generated when the benchmark is re-run. Author: Eric Liang Closes #13607 from ericl/sc-3538. (cherry picked from commit 5bb4564cd47c8bf06409287e0de4ec45609970b2) Signed-off-by: Reynold Xin --- project/SparkBuild.scala | 2 +- .../WideSchemaBenchmark-results.txt | 93 +++++++ sql/core/src/test/resources/log4j.properties | 2 +- .../benchmark/WideSchemaBenchmark.scala | 260 ++---------------- 4 files changed, 123 insertions(+), 234 deletions(-) create mode 100644 sql/core/benchmarks/WideSchemaBenchmark-results.txt diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 304288a32cefe..2f7da31e5508c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -833,7 +833,7 @@ object TestSettings { javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", - javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", + javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=false", javaOptions in Test += "-Dderby.system.durability=test", javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark")) .map { case (k,v) => s"-D$k=$v" }.toSeq, diff --git a/sql/core/benchmarks/WideSchemaBenchmark-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-results.txt new file mode 100644 index 0000000000000..ea6a6616c23d4 --- /dev/null +++ b/sql/core/benchmarks/WideSchemaBenchmark-results.txt @@ -0,0 +1,93 @@ +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 select expressions 3 / 5 0.0 2967064.0 1.0X +100 select expressions 11 / 12 0.0 11369518.0 0.3X +2500 select expressions 243 / 250 0.0 242561004.0 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 cols x 100000 rows (read in-mem) 28 / 40 3.6 278.8 1.0X +1 cols x 100000 rows (exec in-mem) 28 / 42 3.5 284.0 1.0X +1 cols x 100000 rows (read parquet) 23 / 35 4.4 228.8 1.2X +1 cols x 100000 rows (write parquet) 163 / 182 0.6 1633.0 0.2X +100 cols x 1000 rows (read in-mem) 27 / 39 3.7 266.9 1.0X +100 cols x 1000 rows (exec in-mem) 48 / 79 2.1 481.7 0.6X +100 cols x 1000 rows (read parquet) 25 / 36 3.9 254.3 1.1X +100 cols x 1000 rows (write parquet) 182 / 196 0.5 1819.5 0.2X +2500 cols x 40 rows (read in-mem) 280 / 315 0.4 2797.1 0.1X +2500 cols x 40 rows (exec in-mem) 606 / 638 0.2 6064.3 0.0X +2500 cols x 40 rows (read parquet) 836 / 843 0.1 8356.4 0.0X +2500 cols x 40 rows (write parquet) 490 / 522 0.2 4900.6 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 22 / 35 4.6 216.0 1.0X +1 wide x 100000 rows (exec in-mem) 40 / 63 2.5 400.6 0.5X +1 wide x 100000 rows (read parquet) 93 / 134 1.1 933.9 0.2X +1 wide x 100000 rows (write parquet) 133 / 174 0.7 1334.3 0.2X +100 wide x 1000 rows (read in-mem) 22 / 44 4.5 223.3 1.0X +100 wide x 1000 rows (exec in-mem) 88 / 138 1.1 878.6 0.2X +100 wide x 1000 rows (read parquet) 117 / 186 0.9 1172.0 0.2X +100 wide x 1000 rows (write parquet) 144 / 174 0.7 1441.6 0.1X +2500 wide x 40 rows (read in-mem) 36 / 57 2.8 358.9 0.6X +2500 wide x 40 rows (exec in-mem) 1466 / 1507 0.1 14656.6 0.0X +2500 wide x 40 rows (read parquet) 690 / 802 0.1 6898.2 0.0X +2500 wide x 40 rows (write parquet) 197 / 207 0.5 1970.9 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 deep x 100000 rows (read in-mem) 22 / 35 4.5 223.9 1.0X +1 deep x 100000 rows (exec in-mem) 28 / 52 3.6 280.6 0.8X +1 deep x 100000 rows (read parquet) 41 / 65 2.4 410.5 0.5X +1 deep x 100000 rows (write parquet) 163 / 173 0.6 1634.5 0.1X +100 deep x 1000 rows (read in-mem) 43 / 63 2.3 425.9 0.5X +100 deep x 1000 rows (exec in-mem) 232 / 280 0.4 2321.7 0.1X +100 deep x 1000 rows (read parquet) 1989 / 2281 0.1 19886.6 0.0X +100 deep x 1000 rows (write parquet) 144 / 184 0.7 1442.6 0.2X +250 deep x 400 rows (read in-mem) 68 / 95 1.5 680.9 0.3X +250 deep x 400 rows (exec in-mem) 1310 / 1403 0.1 13096.4 0.0X +250 deep x 400 rows (read parquet) 41477 / 41847 0.0 414766.8 0.0X +250 deep x 400 rows (write parquet) 243 / 272 0.4 2433.1 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 x 1 deep x 100000 rows (read in-mem) 23 / 36 4.4 229.8 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 27 / 48 3.7 269.6 0.9X +1 x 1 deep x 100000 rows (read parquet) 25 / 33 4.0 247.5 0.9X +1 x 1 deep x 100000 rows (write parquet) 82 / 134 1.2 821.1 0.3X +128 x 8 deep x 1000 rows (read in-mem) 19 / 29 5.3 189.5 1.2X +128 x 8 deep x 1000 rows (exec in-mem) 144 / 165 0.7 1440.4 0.2X +128 x 8 deep x 1000 rows (read parquet) 117 / 159 0.9 1174.4 0.2X +128 x 8 deep x 1000 rows (write parquet) 135 / 162 0.7 1349.0 0.2X +1024 x 11 deep x 100 rows (read in-mem) 30 / 49 3.3 304.4 0.8X +1024 x 11 deep x 100 rows (exec in-mem) 1146 / 1183 0.1 11457.6 0.0X +1024 x 11 deep x 100 rows (read parquet) 712 / 758 0.1 7119.5 0.0X +1024 x 11 deep x 100 rows (write parquet) 104 / 143 1.0 1037.3 0.2X + +OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic +Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 18 / 31 5.6 179.3 1.0X +1 wide x 100000 rows (exec in-mem) 31 / 47 3.2 310.2 0.6X +1 wide x 100000 rows (read parquet) 45 / 73 2.2 445.1 0.4X +1 wide x 100000 rows (write parquet) 109 / 140 0.9 1085.9 0.2X +100 wide x 1000 rows (read in-mem) 17 / 25 5.8 172.7 1.0X +100 wide x 1000 rows (exec in-mem) 18 / 22 5.4 184.6 1.0X +100 wide x 1000 rows (read parquet) 26 / 42 3.8 261.8 0.7X +100 wide x 1000 rows (write parquet) 150 / 164 0.7 1499.4 0.1X +2500 wide x 40 rows (read in-mem) 19 / 31 5.1 194.7 0.9X +2500 wide x 40 rows (exec in-mem) 19 / 24 5.3 188.5 1.0X +2500 wide x 40 rows (read parquet) 33 / 47 3.0 334.4 0.5X +2500 wide x 40 rows (write parquet) 153 / 164 0.7 1528.2 0.1X + diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index e53cb1f4e681d..33b9ecf1e2826 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -16,7 +16,7 @@ # # Set everything to be logged to the file core/target/unit-tests.log -log4j.rootLogger=DEBUG, CA, FA +log4j.rootLogger=INFO, CA, FA #Console Appender log4j.appender.CA=org.apache.log4j.ConsoleAppender diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index 06466e629b3f2..d2704b3d3f371 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.File +import java.io.{File, FileOutputStream, OutputStream} import org.scalatest.BeforeAndAfterEach @@ -29,11 +29,13 @@ import org.apache.spark.util.{Benchmark, Utils} * Benchmark for performance with very wide and nested DataFrames. * To run this: * build/sbt "sql/test-only *WideSchemaBenchmark" + * + * Results will be written to "sql/core/benchmarks/WideSchemaBenchmark-results.txt". */ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { private val scaleFactor = 100000 - private val widthsToTest = Seq(1, 10, 100, 1000, 2500) - private val depthsToTest = Seq(1, 10, 100, 250) + private val widthsToTest = Seq(1, 100, 2500) + private val depthsToTest = Seq(1, 100, 250) assert(scaleFactor > widthsToTest.max) private lazy val sparkSession = SparkSession.builder @@ -44,15 +46,22 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { import sparkSession.implicits._ private var tmpFiles: List[File] = Nil + private var out: OutputStream = null + + override def beforeAll() { + super.beforeAll() + out = new FileOutputStream(new File("benchmarks/WideSchemaBenchmark-results.txt")) + } + + override def afterAll() { + super.afterAll() + out.close() + } override def afterEach() { - try { - for (tmpFile <- tmpFiles) { - Utils.deleteRecursively(tmpFile) - } - } finally { - tmpFiles = Nil - super.afterEach() + super.afterEach() + for (tmpFile <- tmpFiles) { + Utils.deleteRecursively(tmpFile) } } @@ -80,7 +89,7 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { benchmark.addCase(desc + " (read in-mem)") { iter => df.selectExpr(s"sum($selector)").collect() } - benchmark.addCase(desc + " (write in-mem)") { iter => + benchmark.addCase(desc + " (exec in-mem)") { iter => df.selectExpr("*", s"hash($selector) as f").selectExpr(s"sum($selector)", "sum(f)").collect() } val parquet = saveAsParquet(df) @@ -93,7 +102,7 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { } ignore("parsing large select expressions") { - val benchmark = new Benchmark("parsing large select", 1) + val benchmark = new Benchmark("parsing large select", 1, output = Some(out)) for (width <- widthsToTest) { val selectExpr = (1 to width).map(i => s"id as a_$i") benchmark.addCase(s"$width select expressions") { iter => @@ -101,22 +110,10 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { } } benchmark.run() - -/* -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 select expressions 22 / 25 0.0 22053737.0 1.0X -10 select expressions 8 / 13 0.0 8288520.0 2.7X -100 select expressions 29 / 32 0.0 29481040.0 0.7X -1000 select expressions 268 / 276 0.0 268183159.0 0.1X -2500 select expressions 683 / 691 0.0 683422241.0 0.0X -*/ } ignore("many column field read and write") { - val benchmark = new Benchmark("many column field r/w", scaleFactor) + val benchmark = new Benchmark("many column field r/w", scaleFactor, output = Some(out)) for (width <- widthsToTest) { // normalize by width to keep constant data size val numRows = scaleFactor / width @@ -126,38 +123,11 @@ parsing large select: Best/Avg Time(ms) Rate(M/s) Per Ro addCases(benchmark, df, s"$width cols x $numRows rows", "a_1") } benchmark.run() - -/* -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 cols x 100000 rows (read in-mem) 26 / 33 3.8 262.9 1.0X -1 cols x 100000 rows (write in-mem) 40 / 51 2.5 401.6 0.7X -1 cols x 100000 rows (read parquet) 37 / 57 2.7 374.3 0.7X -1 cols x 100000 rows (write parquet) 105 / 157 0.9 1054.9 0.2X -10 cols x 10000 rows (read in-mem) 26 / 39 3.8 260.5 1.0X -10 cols x 10000 rows (write in-mem) 37 / 44 2.7 367.4 0.7X -10 cols x 10000 rows (read parquet) 31 / 39 3.3 305.1 0.9X -10 cols x 10000 rows (write parquet) 86 / 137 1.2 860.2 0.3X -100 cols x 1000 rows (read in-mem) 40 / 64 2.5 401.2 0.7X -100 cols x 1000 rows (write in-mem) 112 / 139 0.9 1118.3 0.2X -100 cols x 1000 rows (read parquet) 35 / 52 2.9 349.8 0.8X -100 cols x 1000 rows (write parquet) 150 / 182 0.7 1497.1 0.2X -1000 cols x 100 rows (read in-mem) 304 / 362 0.3 3043.6 0.1X -1000 cols x 100 rows (write in-mem) 647 / 729 0.2 6467.8 0.0X -1000 cols x 100 rows (read parquet) 194 / 235 0.5 1937.7 0.1X -1000 cols x 100 rows (write parquet) 511 / 521 0.2 5105.0 0.1X -2500 cols x 40 rows (read in-mem) 915 / 924 0.1 9148.2 0.0X -2500 cols x 40 rows (write in-mem) 1856 / 1933 0.1 18558.1 0.0X -2500 cols x 40 rows (read parquet) 802 / 881 0.1 8019.3 0.0X -2500 cols x 40 rows (write parquet) 1268 / 1291 0.1 12681.6 0.0X -*/ } ignore("wide shallowly nested struct field read and write") { val benchmark = new Benchmark( - "wide shallowly nested struct field r/w", scaleFactor) + "wide shallowly nested struct field r/w", scaleFactor, output = Some(out)) for (width <- widthsToTest) { val numRows = scaleFactor / width var datum: String = "{" @@ -175,84 +145,10 @@ many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Ro addCases(benchmark, df, s"$width wide x $numRows rows", "a.b.c.value_1") } benchmark.run() - -/* -Java HotSpot(TM) 64-Bit Server VM 1.7.0_80-b15 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 wide x 100000 rows (read in-mem) 100 / 125 1.0 997.7 1.0X -1 wide x 100000 rows (write in-mem) 130 / 147 0.8 1302.9 0.8X -1 wide x 100000 rows (read parquet) 195 / 228 0.5 1951.4 0.5X -1 wide x 100000 rows (write parquet) 248 / 259 0.4 2479.7 0.4X -10 wide x 10000 rows (read in-mem) 76 / 89 1.3 757.2 1.3X -10 wide x 10000 rows (write in-mem) 90 / 116 1.1 900.0 1.1X -10 wide x 10000 rows (read parquet) 90 / 135 1.1 903.9 1.1X -10 wide x 10000 rows (write parquet) 222 / 240 0.4 2222.8 0.4X -100 wide x 1000 rows (read in-mem) 71 / 91 1.4 710.8 1.4X -100 wide x 1000 rows (write in-mem) 252 / 324 0.4 2522.4 0.4X -100 wide x 1000 rows (read parquet) 310 / 329 0.3 3095.9 0.3X -100 wide x 1000 rows (write parquet) 253 / 267 0.4 2525.7 0.4X -1000 wide x 100 rows (read in-mem) 144 / 160 0.7 1439.5 0.7X -1000 wide x 100 rows (write in-mem) 2055 / 2326 0.0 20553.9 0.0X -1000 wide x 100 rows (read parquet) 750 / 925 0.1 7496.8 0.1X -1000 wide x 100 rows (write parquet) 235 / 317 0.4 2347.5 0.4X -2500 wide x 40 rows (read in-mem) 219 / 227 0.5 2190.9 0.5X -2500 wide x 40 rows (write in-mem) 5177 / 5423 0.0 51773.2 0.0X -2500 wide x 40 rows (read parquet) 1642 / 1714 0.1 16417.7 0.1X -2500 wide x 40 rows (write parquet) 357 / 381 0.3 3568.2 0.3X -*/ - } - - ignore("wide struct field read and write") { - val benchmark = new Benchmark("wide struct field r/w", scaleFactor) - for (width <- widthsToTest) { - val numRows = scaleFactor / width - var datum: String = "{" - for (i <- 1 to width) { - if (i == 1) { - datum += s""""value_$i": 1""" - } else { - datum += s""", "value_$i": 1""" - } - } - datum += "}" - val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum).rdd).cache() - df.count() // force caching - addCases(benchmark, df, s"$width wide x $numRows rows", "value_1") - } - benchmark.run() - -/* -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -wide struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 wide x 100000 rows (read in-mem) 22 / 37 4.6 216.8 1.0X -1 wide x 100000 rows (write in-mem) 37 / 54 2.7 365.6 0.6X -1 wide x 100000 rows (read parquet) 27 / 44 3.6 274.7 0.8X -1 wide x 100000 rows (write parquet) 155 / 183 0.6 1546.3 0.1X -10 wide x 10000 rows (read in-mem) 27 / 40 3.7 272.1 0.8X -10 wide x 10000 rows (write in-mem) 32 / 44 3.2 315.7 0.7X -10 wide x 10000 rows (read parquet) 31 / 44 3.2 309.8 0.7X -10 wide x 10000 rows (write parquet) 151 / 169 0.7 1509.3 0.1X -100 wide x 1000 rows (read in-mem) 37 / 62 2.7 374.4 0.6X -100 wide x 1000 rows (write in-mem) 81 / 96 1.2 805.6 0.3X -100 wide x 1000 rows (read parquet) 31 / 44 3.3 307.3 0.7X -100 wide x 1000 rows (write parquet) 174 / 209 0.6 1745.0 0.1X -1000 wide x 100 rows (read in-mem) 308 / 339 0.3 3082.4 0.1X -1000 wide x 100 rows (write in-mem) 672 / 696 0.1 6717.7 0.0X -1000 wide x 100 rows (read parquet) 182 / 228 0.5 1821.2 0.1X -1000 wide x 100 rows (write parquet) 484 / 497 0.2 4841.2 0.0X -2500 wide x 40 rows (read in-mem) 727 / 786 0.1 7268.4 0.0X -2500 wide x 40 rows (write in-mem) 1734 / 1782 0.1 17341.5 0.0X -2500 wide x 40 rows (read parquet) 882 / 935 0.1 8816.8 0.0X -2500 wide x 40 rows (write parquet) 935 / 982 0.1 9351.9 0.0X -*/ } ignore("deeply nested struct field read and write") { - val benchmark = new Benchmark("deeply nested struct field r/w", scaleFactor) + val benchmark = new Benchmark("deeply nested struct field r/w", scaleFactor, output = Some(out)) for (depth <- depthsToTest) { val numRows = scaleFactor / depth var datum: String = "{\"value\": 1}" @@ -266,34 +162,11 @@ wide struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Ro addCases(benchmark, df, s"$depth deep x $numRows rows", selector) } benchmark.run() - -/* -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 deep x 100000 rows (read in-mem) 24 / 39 4.2 239.0 1.0X -1 deep x 100000 rows (write in-mem) 34 / 47 3.0 335.1 0.7X -1 deep x 100000 rows (read parquet) 45 / 51 2.2 446.1 0.5X -1 deep x 100000 rows (write parquet) 86 / 108 1.2 859.4 0.3X -10 deep x 10000 rows (read in-mem) 28 / 38 3.6 275.1 0.9X -10 deep x 10000 rows (write in-mem) 43 / 64 2.3 427.1 0.6X -10 deep x 10000 rows (read parquet) 44 / 59 2.3 438.1 0.5X -10 deep x 10000 rows (write parquet) 85 / 110 1.2 853.6 0.3X -100 deep x 1000 rows (read in-mem) 79 / 100 1.3 785.5 0.3X -100 deep x 1000 rows (write in-mem) 776 / 800 0.1 7760.3 0.0X -100 deep x 1000 rows (read parquet) 3302 / 3394 0.0 33021.2 0.0X -100 deep x 1000 rows (write parquet) 226 / 243 0.4 2259.0 0.1X -250 deep x 400 rows (read in-mem) 610 / 639 0.2 6104.0 0.0X -250 deep x 400 rows (write in-mem) 8526 / 8531 0.0 85256.9 0.0X -250 deep x 400 rows (read parquet) 54968 / 55069 0.0 549681.4 0.0X -250 deep x 400 rows (write parquet) 714 / 718 0.1 7143.0 0.0X -*/ } ignore("bushy struct field read and write") { - val benchmark = new Benchmark("bushy struct field r/w", scaleFactor) - for (width <- Seq(1, 10, 100, 500)) { + val benchmark = new Benchmark("bushy struct field r/w", scaleFactor, output = Some(out)) + for (width <- Seq(1, 100, 1000)) { val numRows = scaleFactor / width var numNodes = 1 var datum: String = "{\"value\": 1}" @@ -312,33 +185,10 @@ deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Ro addCases(benchmark, df, s"$numNodes x $depth deep x $numRows rows", selector) } benchmark.run() - -/* -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 x 1 deep x 100000 rows (read in-mem) 21 / 27 4.7 212.6 1.0X -1 x 1 deep x 100000 rows (write in-mem) 27 / 38 3.8 265.8 0.8X -1 x 1 deep x 100000 rows (read parquet) 26 / 32 3.9 259.1 0.8X -1 x 1 deep x 100000 rows (write parquet) 150 / 169 0.7 1499.5 0.1X -16 x 5 deep x 10000 rows (read in-mem) 26 / 45 3.9 258.7 0.8X -16 x 5 deep x 10000 rows (write in-mem) 54 / 58 1.9 535.1 0.4X -16 x 5 deep x 10000 rows (read parquet) 60 / 84 1.7 595.8 0.4X -16 x 5 deep x 10000 rows (write parquet) 179 / 184 0.6 1787.5 0.1X -128 x 8 deep x 1000 rows (read in-mem) 26 / 40 3.8 261.4 0.8X -128 x 8 deep x 1000 rows (write in-mem) 592 / 592 0.2 5915.3 0.0X -128 x 8 deep x 1000 rows (read parquet) 203 / 251 0.5 2031.8 0.1X -128 x 8 deep x 1000 rows (write parquet) 105 / 131 1.0 1045.2 0.2X -512 x 10 deep x 200 rows (read in-mem) 101 / 125 1.0 1007.4 0.2X -512 x 10 deep x 200 rows (write in-mem) 6778 / 6943 0.0 67781.1 0.0X -512 x 10 deep x 200 rows (read parquet) 958 / 1071 0.1 9584.9 0.0X -512 x 10 deep x 200 rows (write parquet) 173 / 207 0.6 1726.1 0.1X -*/ } ignore("wide array field read and write") { - val benchmark = new Benchmark("wide array field r/w", scaleFactor) + val benchmark = new Benchmark("wide array field r/w", scaleFactor, output = Some(out)) for (width <- widthsToTest) { val numRows = scaleFactor / width var datum: String = "{\"value\": [" @@ -355,37 +205,10 @@ bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Ro addCases(benchmark, df, s"$width wide x $numRows rows", "value[0]") } benchmark.run() - -/* -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 wide x 100000 rows (read in-mem) 27 / 45 3.7 268.0 1.0X -1 wide x 100000 rows (write in-mem) 37 / 52 2.7 368.3 0.7X -1 wide x 100000 rows (read parquet) 52 / 65 1.9 524.9 0.5X -1 wide x 100000 rows (write parquet) 102 / 139 1.0 1016.7 0.3X -10 wide x 10000 rows (read in-mem) 20 / 26 5.0 201.7 1.3X -10 wide x 10000 rows (write in-mem) 26 / 35 3.8 259.8 1.0X -10 wide x 10000 rows (read parquet) 39 / 59 2.5 393.8 0.7X -10 wide x 10000 rows (write parquet) 120 / 143 0.8 1201.4 0.2X -100 wide x 1000 rows (read in-mem) 24 / 31 4.2 240.1 1.1X -100 wide x 1000 rows (write in-mem) 26 / 35 3.8 264.1 1.0X -100 wide x 1000 rows (read parquet) 30 / 47 3.4 296.8 0.9X -100 wide x 1000 rows (write parquet) 109 / 147 0.9 1094.8 0.2X -1000 wide x 100 rows (read in-mem) 20 / 38 5.0 200.6 1.3X -1000 wide x 100 rows (write in-mem) 24 / 32 4.1 242.3 1.1X -1000 wide x 100 rows (read parquet) 47 / 55 2.1 470.1 0.6X -1000 wide x 100 rows (write parquet) 146 / 164 0.7 1465.0 0.2X -2500 wide x 40 rows (read in-mem) 20 / 28 5.1 196.1 1.4X -2500 wide x 40 rows (write in-mem) 25 / 27 4.0 249.3 1.1X -2500 wide x 40 rows (read parquet) 33 / 48 3.0 332.0 0.8X -2500 wide x 40 rows (write parquet) 149 / 176 0.7 1489.3 0.2X -*/ } ignore("wide map field read and write") { - val benchmark = new Benchmark("wide map field r/w", scaleFactor) + val benchmark = new Benchmark("wide map field r/w", scaleFactor, output = Some(out)) for (width <- widthsToTest) { val numRows = scaleFactor / width val datum = Tuple1((1 to width).map(i => ("value_" + i -> 1)).toMap) @@ -394,32 +217,5 @@ wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Ro addCases(benchmark, df, s"$width wide x $numRows rows", "_1[\"value_1\"]") } benchmark.run() - -/* -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz -wide map field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 wide x 100000 rows (read in-mem) 27 / 42 3.7 270.9 1.0X -1 wide x 100000 rows (write in-mem) 40 / 63 2.5 403.4 0.7X -1 wide x 100000 rows (read parquet) 71 / 114 1.4 705.8 0.4X -1 wide x 100000 rows (write parquet) 169 / 184 0.6 1689.7 0.2X -10 wide x 10000 rows (read in-mem) 22 / 35 4.6 216.6 1.3X -10 wide x 10000 rows (write in-mem) 29 / 34 3.5 285.6 0.9X -10 wide x 10000 rows (read parquet) 61 / 81 1.6 610.3 0.4X -10 wide x 10000 rows (write parquet) 150 / 172 0.7 1504.7 0.2X -100 wide x 1000 rows (read in-mem) 21 / 29 4.8 207.9 1.3X -100 wide x 1000 rows (write in-mem) 30 / 57 3.3 304.9 0.9X -100 wide x 1000 rows (read parquet) 36 / 61 2.8 356.7 0.8X -100 wide x 1000 rows (write parquet) 108 / 136 0.9 1075.7 0.3X -1000 wide x 100 rows (read in-mem) 22 / 31 4.5 223.0 1.2X -1000 wide x 100 rows (write in-mem) 33 / 41 3.0 332.0 0.8X -1000 wide x 100 rows (read parquet) 49 / 66 2.0 493.6 0.5X -1000 wide x 100 rows (write parquet) 127 / 139 0.8 1265.9 0.2X -2500 wide x 40 rows (read in-mem) 23 / 34 4.4 226.0 1.2X -2500 wide x 40 rows (write in-mem) 33 / 42 3.1 326.6 0.8X -2500 wide x 40 rows (read parquet) 36 / 48 2.8 359.2 0.8X -2500 wide x 40 rows (write parquet) 155 / 168 0.6 1549.2 0.2X -*/ } } From 0cf31f0c8486ac3f8efca84bcfec75c2d0dd738a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 11 Jun 2016 15:28:40 -0700 Subject: [PATCH 0601/1470] [SPARK-15856][SQL] Revert API breaking changes made in SQLContext.range ## What changes were proposed in this pull request? It's easy for users to call `range(...).as[Long]` to get typed Dataset, and don't worth an API breaking change. This PR reverts it. ## How was this patch tested? N/A Author: Wenchen Fan Closes #13605 from cloud-fan/range. (cherry picked from commit 75705e8dbb51ac91ffc7012fa67f072494c13832) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/SQLContext.scala | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) 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 23f2b6e8942f3..6fcc9bb44775a 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 @@ -609,51 +609,51 @@ class SQLContext private[sql](val sparkSession: SparkSession) /** * :: Experimental :: - * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements + * Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements * in a range from 0 to `end` (exclusive) with step value 1. * - * @since 2.0.0 - * @group dataset + * @since 1.4.1 + * @group dataframe */ @Experimental - def range(end: Long): Dataset[java.lang.Long] = sparkSession.range(end) + def range(end: Long): DataFrame = sparkSession.range(end).toDF() /** * :: Experimental :: - * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements + * Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements * in a range from `start` to `end` (exclusive) with step value 1. * - * @since 2.0.0 - * @group dataset + * @since 1.4.0 + * @group dataframe */ @Experimental - def range(start: Long, end: Long): Dataset[java.lang.Long] = sparkSession.range(start, end) + def range(start: Long, end: Long): DataFrame = sparkSession.range(start, end).toDF() /** * :: Experimental :: - * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements + * Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements * in a range from `start` to `end` (exclusive) with a step value. * * @since 2.0.0 - * @group dataset + * @group dataframe */ @Experimental - def range(start: Long, end: Long, step: Long): Dataset[java.lang.Long] = { - sparkSession.range(start, end, step) + def range(start: Long, end: Long, step: Long): DataFrame = { + sparkSession.range(start, end, step).toDF() } /** * :: Experimental :: - * Creates a [[Dataset]] with a single [[LongType]] column named `id`, containing elements - * in a range from `start` to `end` (exclusive) with a step value, with partition number + * Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements + * in an range from `start` to `end` (exclusive) with an step value, with partition number * specified. * - * @since 2.0.0 - * @group dataset + * @since 1.4.0 + * @group dataframe */ @Experimental - def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[java.lang.Long] = { - sparkSession.range(start, end, step, numPartitions) + def range(start: Long, end: Long, step: Long, numPartitions: Int): DataFrame = { + sparkSession.range(start, end, step, numPartitions).toDF() } /** From beb75300455a4f92000b69e740256102d9f2d472 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sat, 11 Jun 2016 15:42:58 -0700 Subject: [PATCH 0602/1470] [SPARK-14851][CORE] Support radix sort with nullable longs ## What changes were proposed in this pull request? This adds support for radix sort of nullable long fields. When a sort field is null and radix sort is enabled, we keep nulls in a separate region of the sort buffer so that radix sort does not need to deal with them. This also has performance benefits when sorting smaller integer types, since the current representation of nulls in two's complement (Long.MIN_VALUE) otherwise forces a full-width radix sort. This strategy for nulls does mean the sort is no longer stable. cc davies ## How was this patch tested? Existing randomized sort tests for correctness. I also tested some TPCDS queries and there does not seem to be any significant regression for non-null sorts. Some test queries (best of 5 runs each). Before change: scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id > 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect(); (System.nanoTime - start) / 1e6 start: Long = 3190437233227987 res3: Double = 4716.471091 After change: scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id > 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect(); (System.nanoTime - start) / 1e6 start: Long = 3190367870952791 res4: Double = 2981.143045 Author: Eric Liang Closes #13161 from ericl/sc-2998. (cherry picked from commit c06c58bbbb2de0c22cfc70c486d23a94c3079ba4) Signed-off-by: Reynold Xin --- .../collection/unsafe/sort/RadixSort.java | 24 +++++--- .../unsafe/sort/UnsafeExternalSorter.java | 11 ++-- .../unsafe/sort/UnsafeInMemorySorter.java | 56 +++++++++++++++---- .../sort/UnsafeExternalSorterSuite.java | 26 ++++----- .../sort/UnsafeInMemorySorterSuite.java | 2 +- .../unsafe/sort/RadixSortSuite.scala | 4 +- .../execution/UnsafeExternalRowSorter.java | 20 ++++++- .../sql/catalyst/expressions/SortOrder.scala | 40 +++++++------ .../sql/execution/UnsafeKVExternalSorter.java | 11 ++-- .../apache/spark/sql/execution/SortExec.scala | 12 +++- .../spark/sql/execution/SortPrefixUtils.scala | 32 ++++++++--- .../spark/sql/execution/WindowExec.scala | 4 +- .../joins/CartesianProductExec.scala | 2 +- .../spark/sql/execution/SortSuite.scala | 11 ++++ .../execution/benchmark/SortBenchmark.scala | 2 +- 15 files changed, 178 insertions(+), 79 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java index 4f3f0de7b8d7e..404361734a55b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java @@ -170,9 +170,13 @@ private static long[] transformCountsToOffsets( /** * Specialization of sort() for key-prefix arrays. In this type of array, each record consists * of two longs, only the second of which is sorted on. + * + * @param startIndex starting index in the array to sort from. This parameter is not supported + * in the plain sort() implementation. */ public static int sortKeyPrefixArray( LongArray array, + int startIndex, int numRecords, int startByteIndex, int endByteIndex, @@ -182,10 +186,11 @@ public static int sortKeyPrefixArray( assert endByteIndex <= 7 : "endByteIndex (" + endByteIndex + ") should <= 7"; assert endByteIndex > startByteIndex; assert numRecords * 4 <= array.size(); - int inIndex = 0; - int outIndex = numRecords * 2; + int inIndex = startIndex; + int outIndex = startIndex + numRecords * 2; if (numRecords > 0) { - long[][] counts = getKeyPrefixArrayCounts(array, numRecords, startByteIndex, endByteIndex); + long[][] counts = getKeyPrefixArrayCounts( + array, startIndex, numRecords, startByteIndex, endByteIndex); for (int i = startByteIndex; i <= endByteIndex; i++) { if (counts[i] != null) { sortKeyPrefixArrayAtByte( @@ -205,13 +210,14 @@ public static int sortKeyPrefixArray( * getCounts with some added parameters but that seems to hurt in benchmarks. */ private static long[][] getKeyPrefixArrayCounts( - LongArray array, int numRecords, int startByteIndex, int endByteIndex) { + LongArray array, int startIndex, int numRecords, int startByteIndex, int endByteIndex) { long[][] counts = new long[8][]; long bitwiseMax = 0; long bitwiseMin = -1L; - long limit = array.getBaseOffset() + numRecords * 16; + long baseOffset = array.getBaseOffset() + startIndex * 8L; + long limit = baseOffset + numRecords * 16L; Object baseObject = array.getBaseObject(); - for (long offset = array.getBaseOffset(); offset < limit; offset += 16) { + for (long offset = baseOffset; offset < limit; offset += 16) { long value = Platform.getLong(baseObject, offset + 8); bitwiseMax |= value; bitwiseMin &= value; @@ -220,7 +226,7 @@ private static long[][] getKeyPrefixArrayCounts( for (int i = startByteIndex; i <= endByteIndex; i++) { if (((bitsChanged >>> (i * 8)) & 0xff) != 0) { counts[i] = new long[256]; - for (long offset = array.getBaseOffset(); offset < limit; offset += 16) { + for (long offset = baseOffset; offset < limit; offset += 16) { counts[i][(int)((Platform.getLong(baseObject, offset + 8) >>> (i * 8)) & 0xff)]++; } } @@ -238,8 +244,8 @@ private static void sortKeyPrefixArrayAtByte( long[] offsets = transformCountsToOffsets( counts, numRecords, array.getBaseOffset() + outIndex * 8, 16, desc, signed); Object baseObject = array.getBaseObject(); - long baseOffset = array.getBaseOffset() + inIndex * 8; - long maxOffset = baseOffset + numRecords * 16; + long baseOffset = array.getBaseOffset() + inIndex * 8L; + long maxOffset = baseOffset + numRecords * 16L; for (long offset = baseOffset; offset < maxOffset; offset += 16) { long key = Platform.getLong(baseObject, offset); long prefix = Platform.getLong(baseObject, offset + 8); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index e14a23f4a6a83..ec15f0b59d3dc 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -369,7 +369,8 @@ private void acquireNewPageIfNecessary(int required) { /** * Write a record to the sorter. */ - public void insertRecord(Object recordBase, long recordOffset, int length, long prefix) + public void insertRecord( + Object recordBase, long recordOffset, int length, long prefix, boolean prefixIsNull) throws IOException { growPointerArrayIfNecessary(); @@ -384,7 +385,7 @@ public void insertRecord(Object recordBase, long recordOffset, int length, long Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length); pageCursor += length; assert(inMemSorter != null); - inMemSorter.insertRecord(recordAddress, prefix); + inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull); } /** @@ -396,7 +397,7 @@ public void insertRecord(Object recordBase, long recordOffset, int length, long * record length = key length + value length + 4 */ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, - Object valueBase, long valueOffset, int valueLen, long prefix) + Object valueBase, long valueOffset, int valueLen, long prefix, boolean prefixIsNull) throws IOException { growPointerArrayIfNecessary(); @@ -415,7 +416,7 @@ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, pageCursor += valueLen; assert(inMemSorter != null); - inMemSorter.insertRecord(recordAddress, prefix); + inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull); } /** @@ -465,7 +466,7 @@ class SpillableIterator extends UnsafeSorterIterator { private boolean loaded = false; private int numRecords = 0; - SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) { + SpillableIterator(UnsafeSorterIterator inMemIterator) { this.upstream = inMemIterator; this.numRecords = inMemIterator.getNumRecords(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index c7b070f519f88..78da38927878b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -18,6 +18,7 @@ package org.apache.spark.util.collection.unsafe.sort; import java.util.Comparator; +import java.util.LinkedList; import org.apache.avro.reflect.Nullable; @@ -93,6 +94,14 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { private int pos = 0; /** + * If sorting with radix sort, specifies the starting position in the sort buffer where records + * with non-null prefixes are kept. Positions [0..nullBoundaryPos) will contain null-prefixed + * records, and positions [nullBoundaryPos..pos) non-null prefixed records. This lets us avoid + * radix sorting over null values. + */ + private int nullBoundaryPos = 0; + + /* * How many records could be inserted, because part of the array should be left for sorting. */ private int usableCapacity = 0; @@ -160,6 +169,7 @@ public void reset() { usableCapacity = getUsableCapacity(); } pos = 0; + nullBoundaryPos = 0; } /** @@ -206,14 +216,27 @@ public void expandPointerArray(LongArray newArray) { * @param recordPointer pointer to a record in a data page, encoded by {@link TaskMemoryManager}. * @param keyPrefix a user-defined key prefix */ - public void insertRecord(long recordPointer, long keyPrefix) { + public void insertRecord(long recordPointer, long keyPrefix, boolean prefixIsNull) { if (!hasSpaceForAnotherRecord()) { throw new IllegalStateException("There is no space for new record"); } - array.set(pos, recordPointer); - pos++; - array.set(pos, keyPrefix); - pos++; + if (prefixIsNull && radixSortSupport != null) { + // Swap forward a non-null record to make room for this one at the beginning of the array. + array.set(pos, array.get(nullBoundaryPos)); + pos++; + array.set(pos, array.get(nullBoundaryPos + 1)); + pos++; + // Place this record in the vacated position. + array.set(nullBoundaryPos, recordPointer); + nullBoundaryPos++; + array.set(nullBoundaryPos, keyPrefix); + nullBoundaryPos++; + } else { + array.set(pos, recordPointer); + pos++; + array.set(pos, keyPrefix); + pos++; + } } public final class SortedIterator extends UnsafeSorterIterator implements Cloneable { @@ -280,15 +303,14 @@ public void loadNext() { * Return an iterator over record pointers in sorted order. For efficiency, all calls to * {@code next()} will return the same mutable object. */ - public SortedIterator getSortedIterator() { + public UnsafeSorterIterator getSortedIterator() { int offset = 0; long start = System.nanoTime(); if (sortComparator != null) { if (this.radixSortSupport != null) { - // TODO(ekl) we should handle NULL values before radix sort for efficiency, since they - // force a full-width sort (and we cannot radix-sort nullable long fields at all). offset = RadixSort.sortKeyPrefixArray( - array, pos / 2, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); + array, nullBoundaryPos, (pos - nullBoundaryPos) / 2, 0, 7, + radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { MemoryBlock unused = new MemoryBlock( array.getBaseObject(), @@ -301,6 +323,20 @@ public SortedIterator getSortedIterator() { } } totalSortTimeNanos += System.nanoTime() - start; - return new SortedIterator(pos / 2, offset); + if (nullBoundaryPos > 0) { + assert radixSortSupport != null : "Nulls are only stored separately with radix sort"; + LinkedList queue = new LinkedList<>(); + if (radixSortSupport.sortDescending()) { + // Nulls are smaller than non-nulls + queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); + queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); + } else { + queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); + queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); + } + return new UnsafeExternalSorter.ChainedIterator(queue); + } else { + return new SortedIterator(pos / 2, offset); + } } } diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index 2cae4beb4c777..bce958c3dc62d 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -156,14 +156,14 @@ private void assertSpillFilesWereCleanedUp() { private static void insertNumber(UnsafeExternalSorter sorter, int value) throws Exception { final int[] arr = new int[]{ value }; - sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value); + sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value, false); } private static void insertRecord( UnsafeExternalSorter sorter, int[] record, long prefix) throws IOException { - sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix); + sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix, false); } private UnsafeExternalSorter newSorter() throws IOException { @@ -206,13 +206,13 @@ public void testSortingOnlyByPrefix() throws Exception { @Test public void testSortingEmptyArrays() throws Exception { final UnsafeExternalSorter sorter = newSorter(); - sorter.insertRecord(null, 0, 0, 0); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0, false); sorter.spill(); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); sorter.spill(); - sorter.insertRecord(null, 0, 0, 0); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0, false); UnsafeSorterIterator iter = sorter.getSortedIterator(); @@ -232,7 +232,7 @@ public void testSortTimeMetric() throws Exception { long prevSortTime = sorter.getSortTimeNanos(); assertEquals(prevSortTime, 0); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); sorter.spill(); assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); prevSortTime = sorter.getSortTimeNanos(); @@ -240,7 +240,7 @@ public void testSortTimeMetric() throws Exception { sorter.spill(); // no sort needed assertEquals(sorter.getSortTimeNanos(), prevSortTime); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); UnsafeSorterIterator iter = sorter.getSortedIterator(); assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); } @@ -280,7 +280,7 @@ public void testFillingPage() throws Exception { final UnsafeExternalSorter sorter = newSorter(); byte[] record = new byte[16]; while (sorter.getNumberOfAllocatedPages() < 2) { - sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0); + sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0, false); } sorter.cleanupResources(); assertSpillFilesWereCleanedUp(); @@ -340,7 +340,7 @@ public void forcedSpillingWithReadIterator() throws Exception { int n = (int) pageSizeBytes / recordSize * 3; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); } assertTrue(sorter.getNumberOfAllocatedPages() >= 2); UnsafeExternalSorter.SpillableIterator iter = @@ -372,7 +372,7 @@ public void forcedSpillingWithNotReadIterator() throws Exception { int n = (int) pageSizeBytes / recordSize * 3; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); } assertTrue(sorter.getNumberOfAllocatedPages() >= 2); UnsafeExternalSorter.SpillableIterator iter = @@ -406,7 +406,7 @@ public void forcedSpillingWithoutComparator() throws Exception { int batch = n / 4; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); if (i % batch == batch - 1) { sorter.spill(); } diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 383c5b3b0884a..bd89085aa9a14 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -120,7 +120,7 @@ public int compare( final long address = memoryManager.encodePageNumberAndOffset(dataPage, position); final String str = getStringFromDataPage(baseObject, position + 4, recordLength); final int partitionId = hashPartitioner.getPartition(str); - sorter.insertRecord(address, partitionId); + sorter.insertRecord(address, partitionId, false); position += 4 + recordLength; } final UnsafeSorterIterator iter = sorter.getSortedIterator(); diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 1d26d4a8307cf..2c13806410192 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -152,7 +152,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { val (buf1, buf2) = generateKeyPrefixTestData(N, rand.nextLong & 0xff) referenceKeyPrefixSort(buf1, 0, N, sortType.referenceComparator) val outOffset = RadixSort.sortKeyPrefixArray( - buf2, N, sortType.startByteIdx, sortType.endByteIdx, + buf2, 0, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) @@ -177,7 +177,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { val (buf1, buf2) = generateKeyPrefixTestData(N, rand.nextLong & mask) referenceKeyPrefixSort(buf1, 0, N, sortType.referenceComparator) val outOffset = RadixSort.sortKeyPrefixArray( - buf2, N, sortType.startByteIdx, sortType.endByteIdx, + buf2, 0, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 37fbad47c1458..ad76bf5a0a81c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -51,7 +51,20 @@ public final class UnsafeExternalRowSorter { private final UnsafeExternalSorter sorter; public abstract static class PrefixComputer { - abstract long computePrefix(InternalRow row); + + public static class Prefix { + /** Key prefix value, or the null prefix value if isNull = true. **/ + long value; + + /** Whether the key is null. */ + boolean isNull; + } + + /** + * Computes prefix for the given row. For efficiency, the returned object may be reused in + * further calls to a given PrefixComputer. + */ + abstract Prefix computePrefix(InternalRow row); } public UnsafeExternalRowSorter( @@ -88,12 +101,13 @@ void setTestSpillFrequency(int frequency) { } public void insertRow(UnsafeRow row) throws IOException { - final long prefix = prefixComputer.computePrefix(row); + final PrefixComputer.Prefix prefix = prefixComputer.computePrefix(row); sorter.insertRecord( row.getBaseObject(), row.getBaseOffset(), row.getSizeInBytes(), - prefix + prefix.value, + prefix.isNull ); numRowsInserted++; if (testSpillFrequency > 0 && (numRowsInserted % testSpillFrequency) == 0) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index 42a8be6b1b1e7..de779ed3702d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -64,10 +64,21 @@ case class SortOrder(child: Expression, direction: SortDirection) } /** - * An expression to generate a 64-bit long prefix used in sorting. + * An expression to generate a 64-bit long prefix used in sorting. If the sort must operate over + * null keys as well, this.nullValue can be used in place of emitted null prefixes in the sort. */ case class SortPrefix(child: SortOrder) extends UnaryExpression { + val nullValue = child.child.dataType match { + case BooleanType | DateType | TimestampType | _: IntegralType => + Long.MinValue + case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS => + Long.MinValue + case _: DecimalType => + DoublePrefixComparator.computePrefix(Double.NegativeInfinity) + case _ => 0L + } + override def eval(input: InternalRow): Any = throw new UnsupportedOperationException override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -75,20 +86,19 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { val input = childCode.value val BinaryPrefixCmp = classOf[BinaryPrefixComparator].getName val DoublePrefixCmp = classOf[DoublePrefixComparator].getName - - val (nullValue: Long, prefixCode: String) = child.child.dataType match { + val prefixCode = child.child.dataType match { case BooleanType => - (Long.MinValue, s"$input ? 1L : 0L") + s"$input ? 1L : 0L" case _: IntegralType => - (Long.MinValue, s"(long) $input") + s"(long) $input" case DateType | TimestampType => - (Long.MinValue, s"(long) $input") + s"(long) $input" case FloatType | DoubleType => - (0L, s"$DoublePrefixCmp.computePrefix((double)$input)") - case StringType => (0L, s"$input.getPrefix()") - case BinaryType => (0L, s"$BinaryPrefixCmp.computePrefix($input)") + s"$DoublePrefixCmp.computePrefix((double)$input)" + case StringType => s"$input.getPrefix()" + case BinaryType => s"$BinaryPrefixCmp.computePrefix($input)" case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS => - val prefix = if (dt.precision <= Decimal.MAX_LONG_DIGITS) { + if (dt.precision <= Decimal.MAX_LONG_DIGITS) { s"$input.toUnscaledLong()" } else { // reduce the scale to fit in a long @@ -96,17 +106,15 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { val s = p - (dt.precision - dt.scale) s"$input.changePrecision($p, $s) ? $input.toUnscaledLong() : ${Long.MinValue}L" } - (Long.MinValue, prefix) case dt: DecimalType => - (DoublePrefixComparator.computePrefix(Double.NegativeInfinity), - s"$DoublePrefixCmp.computePrefix($input.toDouble())") - case _ => (0L, "0L") + s"$DoublePrefixCmp.computePrefix($input.toDouble())" + case _ => "0L" } ev.copy(code = childCode.code + s""" - |long ${ev.value} = ${nullValue}L; - |boolean ${ev.isNull} = false; + |long ${ev.value} = 0L; + |boolean ${ev.isNull} = ${childCode.isNull}; |if (!${childCode.isNull}) { | ${ev.value} = $prefixCode; |} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index bb823cd07be5e..99fe51db68aeb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -118,9 +118,10 @@ public UnsafeKVExternalSorter( // Compute prefix row.pointTo(baseObject, baseOffset, loc.getKeyLength()); - final long prefix = prefixComputer.computePrefix(row); + final UnsafeExternalRowSorter.PrefixComputer.Prefix prefix = + prefixComputer.computePrefix(row); - inMemSorter.insertRecord(address, prefix); + inMemSorter.insertRecord(address, prefix.value, prefix.isNull); } sorter = UnsafeExternalSorter.createWithExistingInMemorySorter( @@ -146,10 +147,12 @@ public UnsafeKVExternalSorter( * sorted runs, and then reallocates memory to hold the new record. */ public void insertKV(UnsafeRow key, UnsafeRow value) throws IOException { - final long prefix = prefixComputer.computePrefix(key); + final UnsafeExternalRowSorter.PrefixComputer.Prefix prefix = + prefixComputer.computePrefix(key); sorter.insertKVRecord( key.getBaseObject(), key.getBaseOffset(), key.getSizeInBytes(), - value.getBaseObject(), value.getBaseOffset(), value.getSizeInBytes(), prefix); + value.getBaseObject(), value.getBaseOffset(), value.getSizeInBytes(), + prefix.value, prefix.isNull); } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 66a16ac576b3a..6db7f45cfdf2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -68,10 +68,16 @@ case class SortExec( SortPrefixUtils.canSortFullyWithPrefix(boundSortExpression) // The generator for prefix - val prefixProjection = UnsafeProjection.create(Seq(SortPrefix(boundSortExpression))) + val prefixExpr = SortPrefix(boundSortExpression) + val prefixProjection = UnsafeProjection.create(Seq(prefixExpr)) val prefixComputer = new UnsafeExternalRowSorter.PrefixComputer { - override def computePrefix(row: InternalRow): Long = { - prefixProjection.apply(row).getLong(0) + private val result = new UnsafeExternalRowSorter.PrefixComputer.Prefix + override def computePrefix(row: InternalRow): + UnsafeExternalRowSorter.PrefixComputer.Prefix = { + val prefix = prefixProjection.apply(row) + result.isNull = prefix.isNullAt(0) + result.value = if (result.isNull) prefixExpr.nullValue else prefix.getLong(0) + result } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala index 1a5ff5fcece36..940467e74d597 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala @@ -33,6 +33,11 @@ object SortPrefixUtils { override def compare(prefix1: Long, prefix2: Long): Int = 0 } + /** + * Dummy sort prefix result to use for empty rows. + */ + private val emptyPrefix = new UnsafeExternalRowSorter.PrefixComputer.Prefix + def getPrefixComparator(sortOrder: SortOrder): PrefixComparator = { sortOrder.dataType match { case StringType => @@ -70,10 +75,6 @@ object SortPrefixUtils { */ def canSortFullyWithPrefix(sortOrder: SortOrder): Boolean = { sortOrder.dataType match { - // TODO(ekl) long-type is problematic because it's null prefix representation collides with - // the lowest possible long value. Handle this special case outside radix sort. - case LongType if sortOrder.nullable => - false case BooleanType | ByteType | ShortType | IntegerType | LongType | DateType | TimestampType | FloatType | DoubleType => true @@ -97,16 +98,29 @@ object SortPrefixUtils { def createPrefixGenerator(schema: StructType): UnsafeExternalRowSorter.PrefixComputer = { if (schema.nonEmpty) { val boundReference = BoundReference(0, schema.head.dataType, nullable = true) - val prefixProjection = UnsafeProjection.create( - SortPrefix(SortOrder(boundReference, Ascending))) + val prefixExpr = SortPrefix(SortOrder(boundReference, Ascending)) + val prefixProjection = UnsafeProjection.create(prefixExpr) new UnsafeExternalRowSorter.PrefixComputer { - override def computePrefix(row: InternalRow): Long = { - prefixProjection.apply(row).getLong(0) + private val result = new UnsafeExternalRowSorter.PrefixComputer.Prefix + override def computePrefix(row: InternalRow): + UnsafeExternalRowSorter.PrefixComputer.Prefix = { + val prefix = prefixProjection.apply(row) + if (prefix.isNullAt(0)) { + result.isNull = true + result.value = prefixExpr.nullValue + } else { + result.isNull = false + result.value = prefix.getLong(0) + } + result } } } else { new UnsafeExternalRowSorter.PrefixComputer { - override def computePrefix(row: InternalRow): Long = 0 + override def computePrefix(row: InternalRow): + UnsafeExternalRowSorter.PrefixComputer.Prefix = { + emptyPrefix + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala index 97bbab65af1de..1b9634cfc0eba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala @@ -347,13 +347,13 @@ case class WindowExec( SparkEnv.get.memoryManager.pageSizeBytes, false) rows.foreach { r => - sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0) + sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0, false) } rows.clear() } } else { sorter.insertRecord(nextRow.getBaseObject, nextRow.getBaseOffset, - nextRow.getSizeInBytes, 0) + nextRow.getSizeInBytes, 0, false) } fetchNextRow() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 88f78a7a73bce..d870d91edc953 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -53,7 +53,7 @@ class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numField val partition = split.asInstanceOf[CartesianPartition] for (y <- rdd2.iterator(partition.s2, context)) { - sorter.insertRecord(y.getBaseObject, y.getBaseOffset, y.getSizeInBytes, 0) + sorter.insertRecord(y.getBaseObject, y.getBaseOffset, y.getSizeInBytes, 0, false) } // Create an iterator from sorter and wrapper it as Iterator[UnsafeRow] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index c3acf29c2d36f..ba3fa3732d0df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -54,6 +54,17 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { sortAnswers = false) } + test("sorting all nulls") { + checkThatPlansAgree( + (1 to 100).map(v => Tuple1(v)).toDF().selectExpr("NULL as a"), + (child: SparkPlan) => + GlobalLimitExec(10, SortExec('a.asc :: Nil, global = true, child = child)), + (child: SparkPlan) => + GlobalLimitExec(10, ReferenceSort('a.asc :: Nil, global = true, child)), + sortAnswers = false + ) + } + test("sort followed by limit") { checkThatPlansAgree( (1 to 100).map(v => Tuple1(v)).toDF("a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 9964b7373fc20..50ae26a3ff9d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -110,7 +110,7 @@ class SortBenchmark extends BenchmarkBase { benchmark.addTimerCase("radix sort key prefix array") { timer => val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) timer.startTiming() - RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) + RadixSort.sortKeyPrefixArray(buf2, 0, size, 0, 7, false, false) timer.stopTiming() } benchmark.run() From 7e2bfff20c7278a20dca857cfd452b96d4d97c1a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 11 Jun 2016 15:47:51 -0700 Subject: [PATCH 0603/1470] [SPARK-15807][SQL] Support varargs for dropDuplicates in Dataset/DataFrame ## What changes were proposed in this pull request? This PR adds `varargs`-types `dropDuplicates` functions in `Dataset/DataFrame`. Currently, `dropDuplicates` supports only `Seq` or `Array`. **Before** ```scala scala> val ds = spark.createDataFrame(Seq(("a", 1), ("b", 2), ("a", 2))) ds: org.apache.spark.sql.DataFrame = [_1: string, _2: int] scala> ds.dropDuplicates(Seq("_1", "_2")) res0: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [_1: string, _2: int] scala> ds.dropDuplicates("_1", "_2") :26: error: overloaded method value dropDuplicates with alternatives: (colNames: Array[String])org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] (colNames: Seq[String])org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] ()org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] cannot be applied to (String, String) ds.dropDuplicates("_1", "_2") ^ ``` **After** ```scala scala> val ds = spark.createDataFrame(Seq(("a", 1), ("b", 2), ("a", 2))) ds: org.apache.spark.sql.DataFrame = [_1: string, _2: int] scala> ds.dropDuplicates("_1", "_2") res0: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [_1: string, _2: int] ``` ## How was this patch tested? Pass the Jenkins tests with new testcases. Author: Dongjoon Hyun Closes #13545 from dongjoon-hyun/SPARK-15807. (cherry picked from commit 3fd2ff4dd85633af49865456a52bf0c09c99708b) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/sql/Dataset.scala | 13 +++++++++++++ .../scala/org/apache/spark/sql/DataFrameSuite.scala | 4 ++++ .../scala/org/apache/spark/sql/DatasetSuite.scala | 13 +++++++++++++ 3 files changed, 30 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 16bbf30a94370..5a67fc79eff0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1833,6 +1833,19 @@ class Dataset[T] private[sql]( */ def dropDuplicates(colNames: Array[String]): Dataset[T] = dropDuplicates(colNames.toSeq) + /** + * Returns a new [[Dataset]] with duplicate rows removed, considering only + * the subset of columns. + * + * @group typedrel + * @since 2.0.0 + */ + @scala.annotation.varargs + def dropDuplicates(col1: String, cols: String*): Dataset[T] = { + val colNames: Seq[String] = col1 +: cols + dropDuplicates(colNames) + } + /** * Computes statistics for numeric columns, including count, mean, stddev, min, and max. * If no columns are given, this function computes statistics for all numerical columns. 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 a02e48d849ebf..6bb0ce95c4bc5 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 @@ -906,6 +906,10 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer( testData.dropDuplicates(Seq("value2")), Seq(Row(2, 1, 2), Row(1, 1, 1))) + + checkAnswer( + testData.dropDuplicates("key", "value1"), + Seq(Row(2, 1, 2), Row(1, 2, 1), Row(1, 1, 1), Row(2, 2, 2))) } test("SPARK-7150 range api") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 11b52bdead7b7..4536a7356f017 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -806,6 +806,19 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(e.getMessage.contains("Null value appeared in non-nullable field")) assert(e.getMessage.contains("top level non-flat input object")) } + + test("dropDuplicates") { + val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + checkDataset( + ds.dropDuplicates("_1"), + ("a", 1), ("b", 1)) + checkDataset( + ds.dropDuplicates("_2"), + ("a", 1), ("a", 2)) + checkDataset( + ds.dropDuplicates("_1", "_2"), + ("a", 1), ("a", 2), ("b", 1)) + } } case class Generic[T](id: T, value: Double) From 796dd15142c00e96d2d7180f7909055a3eb1dfdf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 11 Jun 2016 15:49:39 -0700 Subject: [PATCH 0604/1470] Revert "[SPARK-14851][CORE] Support radix sort with nullable longs" This reverts commit beb75300455a4f92000b69e740256102d9f2d472. --- .../collection/unsafe/sort/RadixSort.java | 24 +++----- .../unsafe/sort/UnsafeExternalSorter.java | 11 ++-- .../unsafe/sort/UnsafeInMemorySorter.java | 56 ++++--------------- .../sort/UnsafeExternalSorterSuite.java | 26 ++++----- .../sort/UnsafeInMemorySorterSuite.java | 2 +- .../unsafe/sort/RadixSortSuite.scala | 4 +- .../execution/UnsafeExternalRowSorter.java | 20 +------ .../sql/catalyst/expressions/SortOrder.scala | 40 ++++++------- .../sql/execution/UnsafeKVExternalSorter.java | 11 ++-- .../apache/spark/sql/execution/SortExec.scala | 12 +--- .../spark/sql/execution/SortPrefixUtils.scala | 32 +++-------- .../spark/sql/execution/WindowExec.scala | 4 +- .../joins/CartesianProductExec.scala | 2 +- .../spark/sql/execution/SortSuite.scala | 11 ---- .../execution/benchmark/SortBenchmark.scala | 2 +- 15 files changed, 79 insertions(+), 178 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java index 404361734a55b..4f3f0de7b8d7e 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java @@ -170,13 +170,9 @@ private static long[] transformCountsToOffsets( /** * Specialization of sort() for key-prefix arrays. In this type of array, each record consists * of two longs, only the second of which is sorted on. - * - * @param startIndex starting index in the array to sort from. This parameter is not supported - * in the plain sort() implementation. */ public static int sortKeyPrefixArray( LongArray array, - int startIndex, int numRecords, int startByteIndex, int endByteIndex, @@ -186,11 +182,10 @@ public static int sortKeyPrefixArray( assert endByteIndex <= 7 : "endByteIndex (" + endByteIndex + ") should <= 7"; assert endByteIndex > startByteIndex; assert numRecords * 4 <= array.size(); - int inIndex = startIndex; - int outIndex = startIndex + numRecords * 2; + int inIndex = 0; + int outIndex = numRecords * 2; if (numRecords > 0) { - long[][] counts = getKeyPrefixArrayCounts( - array, startIndex, numRecords, startByteIndex, endByteIndex); + long[][] counts = getKeyPrefixArrayCounts(array, numRecords, startByteIndex, endByteIndex); for (int i = startByteIndex; i <= endByteIndex; i++) { if (counts[i] != null) { sortKeyPrefixArrayAtByte( @@ -210,14 +205,13 @@ public static int sortKeyPrefixArray( * getCounts with some added parameters but that seems to hurt in benchmarks. */ private static long[][] getKeyPrefixArrayCounts( - LongArray array, int startIndex, int numRecords, int startByteIndex, int endByteIndex) { + LongArray array, int numRecords, int startByteIndex, int endByteIndex) { long[][] counts = new long[8][]; long bitwiseMax = 0; long bitwiseMin = -1L; - long baseOffset = array.getBaseOffset() + startIndex * 8L; - long limit = baseOffset + numRecords * 16L; + long limit = array.getBaseOffset() + numRecords * 16; Object baseObject = array.getBaseObject(); - for (long offset = baseOffset; offset < limit; offset += 16) { + for (long offset = array.getBaseOffset(); offset < limit; offset += 16) { long value = Platform.getLong(baseObject, offset + 8); bitwiseMax |= value; bitwiseMin &= value; @@ -226,7 +220,7 @@ private static long[][] getKeyPrefixArrayCounts( for (int i = startByteIndex; i <= endByteIndex; i++) { if (((bitsChanged >>> (i * 8)) & 0xff) != 0) { counts[i] = new long[256]; - for (long offset = baseOffset; offset < limit; offset += 16) { + for (long offset = array.getBaseOffset(); offset < limit; offset += 16) { counts[i][(int)((Platform.getLong(baseObject, offset + 8) >>> (i * 8)) & 0xff)]++; } } @@ -244,8 +238,8 @@ private static void sortKeyPrefixArrayAtByte( long[] offsets = transformCountsToOffsets( counts, numRecords, array.getBaseOffset() + outIndex * 8, 16, desc, signed); Object baseObject = array.getBaseObject(); - long baseOffset = array.getBaseOffset() + inIndex * 8L; - long maxOffset = baseOffset + numRecords * 16L; + long baseOffset = array.getBaseOffset() + inIndex * 8; + long maxOffset = baseOffset + numRecords * 16; for (long offset = baseOffset; offset < maxOffset; offset += 16) { long key = Platform.getLong(baseObject, offset); long prefix = Platform.getLong(baseObject, offset + 8); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index ec15f0b59d3dc..e14a23f4a6a83 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -369,8 +369,7 @@ private void acquireNewPageIfNecessary(int required) { /** * Write a record to the sorter. */ - public void insertRecord( - Object recordBase, long recordOffset, int length, long prefix, boolean prefixIsNull) + public void insertRecord(Object recordBase, long recordOffset, int length, long prefix) throws IOException { growPointerArrayIfNecessary(); @@ -385,7 +384,7 @@ public void insertRecord( Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length); pageCursor += length; assert(inMemSorter != null); - inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull); + inMemSorter.insertRecord(recordAddress, prefix); } /** @@ -397,7 +396,7 @@ public void insertRecord( * record length = key length + value length + 4 */ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, - Object valueBase, long valueOffset, int valueLen, long prefix, boolean prefixIsNull) + Object valueBase, long valueOffset, int valueLen, long prefix) throws IOException { growPointerArrayIfNecessary(); @@ -416,7 +415,7 @@ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, pageCursor += valueLen; assert(inMemSorter != null); - inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull); + inMemSorter.insertRecord(recordAddress, prefix); } /** @@ -466,7 +465,7 @@ class SpillableIterator extends UnsafeSorterIterator { private boolean loaded = false; private int numRecords = 0; - SpillableIterator(UnsafeSorterIterator inMemIterator) { + SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) { this.upstream = inMemIterator; this.numRecords = inMemIterator.getNumRecords(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 78da38927878b..c7b070f519f88 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -18,7 +18,6 @@ package org.apache.spark.util.collection.unsafe.sort; import java.util.Comparator; -import java.util.LinkedList; import org.apache.avro.reflect.Nullable; @@ -94,14 +93,6 @@ public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { private int pos = 0; /** - * If sorting with radix sort, specifies the starting position in the sort buffer where records - * with non-null prefixes are kept. Positions [0..nullBoundaryPos) will contain null-prefixed - * records, and positions [nullBoundaryPos..pos) non-null prefixed records. This lets us avoid - * radix sorting over null values. - */ - private int nullBoundaryPos = 0; - - /* * How many records could be inserted, because part of the array should be left for sorting. */ private int usableCapacity = 0; @@ -169,7 +160,6 @@ public void reset() { usableCapacity = getUsableCapacity(); } pos = 0; - nullBoundaryPos = 0; } /** @@ -216,27 +206,14 @@ public void expandPointerArray(LongArray newArray) { * @param recordPointer pointer to a record in a data page, encoded by {@link TaskMemoryManager}. * @param keyPrefix a user-defined key prefix */ - public void insertRecord(long recordPointer, long keyPrefix, boolean prefixIsNull) { + public void insertRecord(long recordPointer, long keyPrefix) { if (!hasSpaceForAnotherRecord()) { throw new IllegalStateException("There is no space for new record"); } - if (prefixIsNull && radixSortSupport != null) { - // Swap forward a non-null record to make room for this one at the beginning of the array. - array.set(pos, array.get(nullBoundaryPos)); - pos++; - array.set(pos, array.get(nullBoundaryPos + 1)); - pos++; - // Place this record in the vacated position. - array.set(nullBoundaryPos, recordPointer); - nullBoundaryPos++; - array.set(nullBoundaryPos, keyPrefix); - nullBoundaryPos++; - } else { - array.set(pos, recordPointer); - pos++; - array.set(pos, keyPrefix); - pos++; - } + array.set(pos, recordPointer); + pos++; + array.set(pos, keyPrefix); + pos++; } public final class SortedIterator extends UnsafeSorterIterator implements Cloneable { @@ -303,14 +280,15 @@ public void loadNext() { * Return an iterator over record pointers in sorted order. For efficiency, all calls to * {@code next()} will return the same mutable object. */ - public UnsafeSorterIterator getSortedIterator() { + public SortedIterator getSortedIterator() { int offset = 0; long start = System.nanoTime(); if (sortComparator != null) { if (this.radixSortSupport != null) { + // TODO(ekl) we should handle NULL values before radix sort for efficiency, since they + // force a full-width sort (and we cannot radix-sort nullable long fields at all). offset = RadixSort.sortKeyPrefixArray( - array, nullBoundaryPos, (pos - nullBoundaryPos) / 2, 0, 7, - radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); + array, pos / 2, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { MemoryBlock unused = new MemoryBlock( array.getBaseObject(), @@ -323,20 +301,6 @@ public UnsafeSorterIterator getSortedIterator() { } } totalSortTimeNanos += System.nanoTime() - start; - if (nullBoundaryPos > 0) { - assert radixSortSupport != null : "Nulls are only stored separately with radix sort"; - LinkedList queue = new LinkedList<>(); - if (radixSortSupport.sortDescending()) { - // Nulls are smaller than non-nulls - queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); - queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); - } else { - queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); - queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); - } - return new UnsafeExternalSorter.ChainedIterator(queue); - } else { - return new SortedIterator(pos / 2, offset); - } + return new SortedIterator(pos / 2, offset); } } diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index bce958c3dc62d..2cae4beb4c777 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -156,14 +156,14 @@ private void assertSpillFilesWereCleanedUp() { private static void insertNumber(UnsafeExternalSorter sorter, int value) throws Exception { final int[] arr = new int[]{ value }; - sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value, false); + sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value); } private static void insertRecord( UnsafeExternalSorter sorter, int[] record, long prefix) throws IOException { - sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix, false); + sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix); } private UnsafeExternalSorter newSorter() throws IOException { @@ -206,13 +206,13 @@ public void testSortingOnlyByPrefix() throws Exception { @Test public void testSortingEmptyArrays() throws Exception { final UnsafeExternalSorter sorter = newSorter(); - sorter.insertRecord(null, 0, 0, 0, false); - sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0); sorter.spill(); - sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0); sorter.spill(); - sorter.insertRecord(null, 0, 0, 0, false); - sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0); UnsafeSorterIterator iter = sorter.getSortedIterator(); @@ -232,7 +232,7 @@ public void testSortTimeMetric() throws Exception { long prevSortTime = sorter.getSortTimeNanos(); assertEquals(prevSortTime, 0); - sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0); sorter.spill(); assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); prevSortTime = sorter.getSortTimeNanos(); @@ -240,7 +240,7 @@ public void testSortTimeMetric() throws Exception { sorter.spill(); // no sort needed assertEquals(sorter.getSortTimeNanos(), prevSortTime); - sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0); UnsafeSorterIterator iter = sorter.getSortedIterator(); assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); } @@ -280,7 +280,7 @@ public void testFillingPage() throws Exception { final UnsafeExternalSorter sorter = newSorter(); byte[] record = new byte[16]; while (sorter.getNumberOfAllocatedPages() < 2) { - sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0, false); + sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0); } sorter.cleanupResources(); assertSpillFilesWereCleanedUp(); @@ -340,7 +340,7 @@ public void forcedSpillingWithReadIterator() throws Exception { int n = (int) pageSizeBytes / recordSize * 3; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); } assertTrue(sorter.getNumberOfAllocatedPages() >= 2); UnsafeExternalSorter.SpillableIterator iter = @@ -372,7 +372,7 @@ public void forcedSpillingWithNotReadIterator() throws Exception { int n = (int) pageSizeBytes / recordSize * 3; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); } assertTrue(sorter.getNumberOfAllocatedPages() >= 2); UnsafeExternalSorter.SpillableIterator iter = @@ -406,7 +406,7 @@ public void forcedSpillingWithoutComparator() throws Exception { int batch = n / 4; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); if (i % batch == batch - 1) { sorter.spill(); } diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index bd89085aa9a14..383c5b3b0884a 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -120,7 +120,7 @@ public int compare( final long address = memoryManager.encodePageNumberAndOffset(dataPage, position); final String str = getStringFromDataPage(baseObject, position + 4, recordLength); final int partitionId = hashPartitioner.getPartition(str); - sorter.insertRecord(address, partitionId, false); + sorter.insertRecord(address, partitionId); position += 4 + recordLength; } final UnsafeSorterIterator iter = sorter.getSortedIterator(); diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 2c13806410192..1d26d4a8307cf 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -152,7 +152,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { val (buf1, buf2) = generateKeyPrefixTestData(N, rand.nextLong & 0xff) referenceKeyPrefixSort(buf1, 0, N, sortType.referenceComparator) val outOffset = RadixSort.sortKeyPrefixArray( - buf2, 0, N, sortType.startByteIdx, sortType.endByteIdx, + buf2, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) @@ -177,7 +177,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { val (buf1, buf2) = generateKeyPrefixTestData(N, rand.nextLong & mask) referenceKeyPrefixSort(buf1, 0, N, sortType.referenceComparator) val outOffset = RadixSort.sortKeyPrefixArray( - buf2, 0, N, sortType.startByteIdx, sortType.endByteIdx, + buf2, N, sortType.startByteIdx, sortType.endByteIdx, sortType.descending, sortType.signed) val res1 = collectToArray(buf1, 0, N * 2) val res2 = collectToArray(buf2, outOffset, N * 2) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index ad76bf5a0a81c..37fbad47c1458 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -51,20 +51,7 @@ public final class UnsafeExternalRowSorter { private final UnsafeExternalSorter sorter; public abstract static class PrefixComputer { - - public static class Prefix { - /** Key prefix value, or the null prefix value if isNull = true. **/ - long value; - - /** Whether the key is null. */ - boolean isNull; - } - - /** - * Computes prefix for the given row. For efficiency, the returned object may be reused in - * further calls to a given PrefixComputer. - */ - abstract Prefix computePrefix(InternalRow row); + abstract long computePrefix(InternalRow row); } public UnsafeExternalRowSorter( @@ -101,13 +88,12 @@ void setTestSpillFrequency(int frequency) { } public void insertRow(UnsafeRow row) throws IOException { - final PrefixComputer.Prefix prefix = prefixComputer.computePrefix(row); + final long prefix = prefixComputer.computePrefix(row); sorter.insertRecord( row.getBaseObject(), row.getBaseOffset(), row.getSizeInBytes(), - prefix.value, - prefix.isNull + prefix ); numRowsInserted++; if (testSpillFrequency > 0 && (numRowsInserted % testSpillFrequency) == 0) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index de779ed3702d3..42a8be6b1b1e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -64,21 +64,10 @@ case class SortOrder(child: Expression, direction: SortDirection) } /** - * An expression to generate a 64-bit long prefix used in sorting. If the sort must operate over - * null keys as well, this.nullValue can be used in place of emitted null prefixes in the sort. + * An expression to generate a 64-bit long prefix used in sorting. */ case class SortPrefix(child: SortOrder) extends UnaryExpression { - val nullValue = child.child.dataType match { - case BooleanType | DateType | TimestampType | _: IntegralType => - Long.MinValue - case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS => - Long.MinValue - case _: DecimalType => - DoublePrefixComparator.computePrefix(Double.NegativeInfinity) - case _ => 0L - } - override def eval(input: InternalRow): Any = throw new UnsupportedOperationException override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -86,19 +75,20 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { val input = childCode.value val BinaryPrefixCmp = classOf[BinaryPrefixComparator].getName val DoublePrefixCmp = classOf[DoublePrefixComparator].getName - val prefixCode = child.child.dataType match { + + val (nullValue: Long, prefixCode: String) = child.child.dataType match { case BooleanType => - s"$input ? 1L : 0L" + (Long.MinValue, s"$input ? 1L : 0L") case _: IntegralType => - s"(long) $input" + (Long.MinValue, s"(long) $input") case DateType | TimestampType => - s"(long) $input" + (Long.MinValue, s"(long) $input") case FloatType | DoubleType => - s"$DoublePrefixCmp.computePrefix((double)$input)" - case StringType => s"$input.getPrefix()" - case BinaryType => s"$BinaryPrefixCmp.computePrefix($input)" + (0L, s"$DoublePrefixCmp.computePrefix((double)$input)") + case StringType => (0L, s"$input.getPrefix()") + case BinaryType => (0L, s"$BinaryPrefixCmp.computePrefix($input)") case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS => - if (dt.precision <= Decimal.MAX_LONG_DIGITS) { + val prefix = if (dt.precision <= Decimal.MAX_LONG_DIGITS) { s"$input.toUnscaledLong()" } else { // reduce the scale to fit in a long @@ -106,15 +96,17 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { val s = p - (dt.precision - dt.scale) s"$input.changePrecision($p, $s) ? $input.toUnscaledLong() : ${Long.MinValue}L" } + (Long.MinValue, prefix) case dt: DecimalType => - s"$DoublePrefixCmp.computePrefix($input.toDouble())" - case _ => "0L" + (DoublePrefixComparator.computePrefix(Double.NegativeInfinity), + s"$DoublePrefixCmp.computePrefix($input.toDouble())") + case _ => (0L, "0L") } ev.copy(code = childCode.code + s""" - |long ${ev.value} = 0L; - |boolean ${ev.isNull} = ${childCode.isNull}; + |long ${ev.value} = ${nullValue}L; + |boolean ${ev.isNull} = false; |if (!${childCode.isNull}) { | ${ev.value} = $prefixCode; |} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 99fe51db68aeb..bb823cd07be5e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -118,10 +118,9 @@ public UnsafeKVExternalSorter( // Compute prefix row.pointTo(baseObject, baseOffset, loc.getKeyLength()); - final UnsafeExternalRowSorter.PrefixComputer.Prefix prefix = - prefixComputer.computePrefix(row); + final long prefix = prefixComputer.computePrefix(row); - inMemSorter.insertRecord(address, prefix.value, prefix.isNull); + inMemSorter.insertRecord(address, prefix); } sorter = UnsafeExternalSorter.createWithExistingInMemorySorter( @@ -147,12 +146,10 @@ public UnsafeKVExternalSorter( * sorted runs, and then reallocates memory to hold the new record. */ public void insertKV(UnsafeRow key, UnsafeRow value) throws IOException { - final UnsafeExternalRowSorter.PrefixComputer.Prefix prefix = - prefixComputer.computePrefix(key); + final long prefix = prefixComputer.computePrefix(key); sorter.insertKVRecord( key.getBaseObject(), key.getBaseOffset(), key.getSizeInBytes(), - value.getBaseObject(), value.getBaseOffset(), value.getSizeInBytes(), - prefix.value, prefix.isNull); + value.getBaseObject(), value.getBaseOffset(), value.getSizeInBytes(), prefix); } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 6db7f45cfdf2c..66a16ac576b3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -68,16 +68,10 @@ case class SortExec( SortPrefixUtils.canSortFullyWithPrefix(boundSortExpression) // The generator for prefix - val prefixExpr = SortPrefix(boundSortExpression) - val prefixProjection = UnsafeProjection.create(Seq(prefixExpr)) + val prefixProjection = UnsafeProjection.create(Seq(SortPrefix(boundSortExpression))) val prefixComputer = new UnsafeExternalRowSorter.PrefixComputer { - private val result = new UnsafeExternalRowSorter.PrefixComputer.Prefix - override def computePrefix(row: InternalRow): - UnsafeExternalRowSorter.PrefixComputer.Prefix = { - val prefix = prefixProjection.apply(row) - result.isNull = prefix.isNullAt(0) - result.value = if (result.isNull) prefixExpr.nullValue else prefix.getLong(0) - result + override def computePrefix(row: InternalRow): Long = { + prefixProjection.apply(row).getLong(0) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala index 940467e74d597..1a5ff5fcece36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala @@ -33,11 +33,6 @@ object SortPrefixUtils { override def compare(prefix1: Long, prefix2: Long): Int = 0 } - /** - * Dummy sort prefix result to use for empty rows. - */ - private val emptyPrefix = new UnsafeExternalRowSorter.PrefixComputer.Prefix - def getPrefixComparator(sortOrder: SortOrder): PrefixComparator = { sortOrder.dataType match { case StringType => @@ -75,6 +70,10 @@ object SortPrefixUtils { */ def canSortFullyWithPrefix(sortOrder: SortOrder): Boolean = { sortOrder.dataType match { + // TODO(ekl) long-type is problematic because it's null prefix representation collides with + // the lowest possible long value. Handle this special case outside radix sort. + case LongType if sortOrder.nullable => + false case BooleanType | ByteType | ShortType | IntegerType | LongType | DateType | TimestampType | FloatType | DoubleType => true @@ -98,29 +97,16 @@ object SortPrefixUtils { def createPrefixGenerator(schema: StructType): UnsafeExternalRowSorter.PrefixComputer = { if (schema.nonEmpty) { val boundReference = BoundReference(0, schema.head.dataType, nullable = true) - val prefixExpr = SortPrefix(SortOrder(boundReference, Ascending)) - val prefixProjection = UnsafeProjection.create(prefixExpr) + val prefixProjection = UnsafeProjection.create( + SortPrefix(SortOrder(boundReference, Ascending))) new UnsafeExternalRowSorter.PrefixComputer { - private val result = new UnsafeExternalRowSorter.PrefixComputer.Prefix - override def computePrefix(row: InternalRow): - UnsafeExternalRowSorter.PrefixComputer.Prefix = { - val prefix = prefixProjection.apply(row) - if (prefix.isNullAt(0)) { - result.isNull = true - result.value = prefixExpr.nullValue - } else { - result.isNull = false - result.value = prefix.getLong(0) - } - result + override def computePrefix(row: InternalRow): Long = { + prefixProjection.apply(row).getLong(0) } } } else { new UnsafeExternalRowSorter.PrefixComputer { - override def computePrefix(row: InternalRow): - UnsafeExternalRowSorter.PrefixComputer.Prefix = { - emptyPrefix - } + override def computePrefix(row: InternalRow): Long = 0 } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala index 1b9634cfc0eba..97bbab65af1de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala @@ -347,13 +347,13 @@ case class WindowExec( SparkEnv.get.memoryManager.pageSizeBytes, false) rows.foreach { r => - sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0, false) + sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0) } rows.clear() } } else { sorter.insertRecord(nextRow.getBaseObject, nextRow.getBaseOffset, - nextRow.getSizeInBytes, 0, false) + nextRow.getSizeInBytes, 0) } fetchNextRow() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index d870d91edc953..88f78a7a73bce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -53,7 +53,7 @@ class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numField val partition = split.asInstanceOf[CartesianPartition] for (y <- rdd2.iterator(partition.s2, context)) { - sorter.insertRecord(y.getBaseObject, y.getBaseOffset, y.getSizeInBytes, 0, false) + sorter.insertRecord(y.getBaseObject, y.getBaseOffset, y.getSizeInBytes, 0) } // Create an iterator from sorter and wrapper it as Iterator[UnsafeRow] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index ba3fa3732d0df..c3acf29c2d36f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -54,17 +54,6 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { sortAnswers = false) } - test("sorting all nulls") { - checkThatPlansAgree( - (1 to 100).map(v => Tuple1(v)).toDF().selectExpr("NULL as a"), - (child: SparkPlan) => - GlobalLimitExec(10, SortExec('a.asc :: Nil, global = true, child = child)), - (child: SparkPlan) => - GlobalLimitExec(10, ReferenceSort('a.asc :: Nil, global = true, child)), - sortAnswers = false - ) - } - test("sort followed by limit") { checkThatPlansAgree( (1 to 100).map(v => Tuple1(v)).toDF("a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 50ae26a3ff9d9..9964b7373fc20 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -110,7 +110,7 @@ class SortBenchmark extends BenchmarkBase { benchmark.addTimerCase("radix sort key prefix array") { timer => val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) timer.startTiming() - RadixSort.sortKeyPrefixArray(buf2, 0, size, 0, 7, false, false) + RadixSort.sortKeyPrefixArray(buf2, size, 0, 7, false, false) timer.stopTiming() } benchmark.run() From ffbc6b796591d3e1f3dcb950335871b7826e6b3b Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sat, 11 Jun 2016 23:16:21 -0700 Subject: [PATCH 0605/1470] [SPARK-15860] Metrics for codegen size and perf ## What changes were proposed in this pull request? Adds codahale metrics for the codegen source text size and how long it takes to compile. The size is particularly interesting, since the JVM does have hard limits on how large methods can get. To simplify, I added the metrics under a statically-initialized source that is always registered with SparkEnv. ## How was this patch tested? Unit tests Author: Eric Liang Closes #13586 from ericl/spark-15860. (cherry picked from commit e1f986c7a3fcc3864d53ef99ef7f14fa4d262ac3) Signed-off-by: Reynold Xin --- .../apache/spark/metrics/MetricsSystem.scala | 3 +- .../spark/metrics/source/StaticSources.scala | 50 +++++++++++++++++++ .../spark/metrics/MetricsSystemSuite.scala | 8 +-- .../expressions/codegen/CodeGenerator.scala | 3 ++ .../expressions/CodeGenerationSuite.scala | 9 ++++ 5 files changed, 68 insertions(+), 5 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 0fed991049dd3..9b16c116ae5ae 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -28,7 +28,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.Logging import org.apache.spark.metrics.sink.{MetricsServlet, Sink} -import org.apache.spark.metrics.source.Source +import org.apache.spark.metrics.source.{Source, StaticSources} import org.apache.spark.util.Utils /** @@ -96,6 +96,7 @@ private[spark] class MetricsSystem private ( def start() { require(!running, "Attempting to start a MetricsSystem that is already running") running = true + StaticSources.allSources.foreach(registerSource) registerSources() registerSinks() sinks.foreach(_.start) diff --git a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala new file mode 100644 index 0000000000000..6819222e15a13 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala @@ -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. + */ + +package org.apache.spark.metrics.source + +import com.codahale.metrics.MetricRegistry + +import org.apache.spark.annotation.Experimental + +private[spark] object StaticSources { + /** + * The set of all static sources. These sources may be reported to from any class, including + * static classes, without requiring reference to a SparkEnv. + */ + val allSources = Seq(CodegenMetrics) +} + +/** + * :: Experimental :: + * Metrics for code generation. + */ +@Experimental +object CodegenMetrics extends Source { + override val sourceName: String = "CodeGenerator" + override val metricRegistry: MetricRegistry = new MetricRegistry() + + /** + * Histogram of the length of source code text compiled by CodeGenerator (in characters). + */ + val METRIC_SOURCE_CODE_SIZE = metricRegistry.histogram(MetricRegistry.name("sourceCodeSize")) + + /** + * Histogram of the time it took to compile source code text (in milliseconds). + */ + val METRIC_COMPILATION_TIME = metricRegistry.histogram(MetricRegistry.name("compilationTime")) +} diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 5d8554229dbe1..2400832f6eea7 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -24,7 +24,7 @@ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.master.MasterSource -import org.apache.spark.metrics.source.Source +import org.apache.spark.metrics.source.{Source, StaticSources} class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateMethodTester{ var filePath: String = _ @@ -43,7 +43,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sources()).length === StaticSources.allSources.length) assert(metricsSystem.invokePrivate(sinks()).length === 0) assert(metricsSystem.getServletHandlers.nonEmpty) } @@ -54,13 +54,13 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sources()).length === StaticSources.allSources.length) assert(metricsSystem.invokePrivate(sinks()).length === 1) assert(metricsSystem.getServletHandlers.nonEmpty) val source = new MasterSource(null) metricsSystem.registerSource(source) - assert(metricsSystem.invokePrivate(sources()).length === 1) + assert(metricsSystem.invokePrivate(sources()).length === StaticSources.allSources.length + 1) } test("MetricsSystem with Driver instance") { 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 ca202928e8823..ff97cd321199a 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 @@ -26,6 +26,7 @@ import scala.language.existentials import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} @@ -886,6 +887,8 @@ object CodeGenerator extends Logging { val result = doCompile(code) val endTime = System.nanoTime() def timeMs: Double = (endTime - startTime).toDouble / 1000000 + CodegenMetrics.METRIC_SOURCE_CODE_SIZE.update(code.body.length) + CodegenMetrics.METRIC_COMPILATION_TIME.update(timeMs.toLong) logInfo(s"Code generated in $timeMs ms") result } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 8ffe39084c7f0..62429a22f02fa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -49,6 +50,14 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { futures.foreach(ThreadUtils.awaitResult(_, 10.seconds)) } + test("metrics are recorded on compile") { + val startCount1 = CodegenMetrics.METRIC_COMPILATION_TIME.getCount() + val startCount2 = CodegenMetrics.METRIC_SOURCE_CODE_SIZE.getCount() + GenerateOrdering.generate(Add(Literal(123), Literal(1)).asc :: Nil) + assert(CodegenMetrics.METRIC_COMPILATION_TIME.getCount() == startCount1 + 1) + assert(CodegenMetrics.METRIC_SOURCE_CODE_SIZE.getCount() == startCount2 + 1) + } + test("SPARK-8443: split wide projections into blocks due to JVM code size limit") { val length = 5000 val expressions = List.fill(length)(EqualTo(Literal(1), Literal(1))) From d494a483aef49766edf9c148dadb5e0c7351ca0d Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sat, 11 Jun 2016 23:20:40 -0700 Subject: [PATCH 0606/1470] [SPARK-15840][SQL] Add two missing options in documentation and some option related changes ## What changes were proposed in this pull request? This PR 1. Adds the documentations for some missing options, `inferSchema` and `mergeSchema` for Python and Scala. 2. Fiixes `[[DataFrame]]` to ```:class:`DataFrame` ``` so that this can be shown - from ![2016-06-09 9 31 16](https://cloud.githubusercontent.com/assets/6477701/15929721/8b864734-2e89-11e6-83f6-207527de4ac9.png) - to (with class link) ![2016-06-09 9 31 00](https://cloud.githubusercontent.com/assets/6477701/15929717/8a03d728-2e89-11e6-8a3f-08294964db22.png) (Please refer [the latest documentation](https://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/api/python/pyspark.sql.html)) 3. Moves `mergeSchema` option to `ParquetOptions` with removing unused options, `metastoreSchema` and `metastoreTableName`. They are not used anymore. They were removed in https://github.com/apache/spark/commit/e720dda42e806229ccfd970055c7b8a93eb447bf and there are no use cases as below: ```bash grep -r -e METASTORE_SCHEMA -e \"metastoreSchema\" -e \"metastoreTableName\" -e METASTORE_TABLE_NAME . ``` ``` ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala: private[sql] val METASTORE_SCHEMA = "metastoreSchema" ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala: private[sql] val METASTORE_TABLE_NAME = "metastoreTableName" ./sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala: ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier( ``` It only sets `metastoreTableName` in the last case but does not use the table name. 4. Sets the correct default values (in the documentation) for `compression` option for ORC(`snappy`, see [OrcOptions.scala#L33-L42](https://github.com/apache/spark/blob/3ded5bc4db2badc9ff49554e73421021d854306b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala#L33-L42)) and Parquet(`the value specified in SQLConf`, see [ParquetOptions.scala#L38-L47](https://github.com/apache/spark/blob/3ded5bc4db2badc9ff49554e73421021d854306b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala#L38-L47)) and `columnNameOfCorruptRecord` for JSON(`the value specified in SQLConf`, see [JsonFileFormat.scala#L53-L55](https://github.com/apache/spark/blob/4538443e276597530a27c6922e48503677b13956/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala#L53-L55) and [JsonFileFormat.scala#L105-L106](https://github.com/apache/spark/blob/4538443e276597530a27c6922e48503677b13956/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala#L105-L106)). ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon Author: Hyukjin Kwon Closes #13576 from HyukjinKwon/SPARK-15840. (cherry picked from commit 9e204c62c6800e03759e04ef68268105d4b86bf2) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 40 +++++++++++++------ .../apache/spark/sql/DataFrameReader.scala | 18 ++++++--- .../apache/spark/sql/DataFrameWriter.scala | 11 ++--- .../parquet/ParquetFileFormat.scala | 19 ++------- .../datasources/parquet/ParquetOptions.scala | 15 ++++++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 12 ++---- 6 files changed, 65 insertions(+), 50 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 7d1f18611bd63..f3182b237ec65 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -209,7 +209,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param columnNameOfCorruptRecord: allows renaming the new field having malformed string created by ``PERMISSIVE`` mode. This overrides ``spark.sql.columnNameOfCorruptRecord``. If None is set, - it uses the default value ``_corrupt_record``. + it uses the value specified in + ``spark.sql.columnNameOfCorruptRecord``. >>> df1 = spark.read.json('python/test_support/sql/people.json') >>> df1.dtypes @@ -276,6 +277,11 @@ def table(self, tableName): def parquet(self, *paths): """Loads a Parquet file, returning the result as a :class:`DataFrame`. + You can set the following Parquet-specific option(s) for reading Parquet files: + * ``mergeSchema``: sets whether we should merge schemas collected from all \ + Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ + The default value is specified in ``spark.sql.parquet.mergeSchema``. + >>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned') >>> df.dtypes [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] @@ -285,7 +291,7 @@ def parquet(self, *paths): @ignore_unicode_prefix @since(1.6) def text(self, paths): - """Loads a text file and returns a [[DataFrame]] with a single string column named "value". + """Loads a text file and returns a :class:`DataFrame` with a single string column named "value". If the directory structure of the text files contains partitioning information, those are ignored in the resulting DataFrame. To include partitioning information as columns, use ``read.format('text').load(...)``. @@ -304,13 +310,14 @@ def text(self, paths): @since(2.0) def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, - comment=None, header=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, - nullValue=None, nanValue=None, positiveInf=None, negativeInf=None, dateFormat=None, - maxColumns=None, maxCharsPerColumn=None, mode=None): - """Loads a CSV file and returns the result as a [[DataFrame]]. + comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, + ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, + negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, mode=None): + """Loads a CSV file and returns the result as a :class:`DataFrame`. - This function goes through the input once to determine the input schema. To avoid going - through the entire data once, specify the schema explicitly using [[schema]]. + This function will go through the input once to determine the input schema if + ``inferSchema`` is enabled. To avoid going through the entire data once, disable + ``inferSchema`` option or specify the schema explicitly using ``schema``. :param path: string, or list of strings, for input path(s). :param schema: an optional :class:`StructType` for the input schema. @@ -328,6 +335,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non character. By default (None), it is disabled. :param header: uses the first line as names of columns. If None is set, it uses the default value, ``false``. + :param inferSchema: infers the input schema automatically from data. It requires one extra + pass over the data. If None is set, it uses the default value, ``false``. :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values being read should be skipped. If None is set, it uses the default value, ``false``. @@ -378,6 +387,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non self.option("comment", comment) if header is not None: self.option("header", header) + if inferSchema is not None: + self.option("inferSchema", inferSchema) if ignoreLeadingWhiteSpace is not None: self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) if ignoreTrailingWhiteSpace is not None: @@ -464,7 +475,7 @@ def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPar class DataFrameWriter(object): """ - Interface used to write a [[DataFrame]] to external storage systems + Interface used to write a :class:`DataFrame` to external storage systems (e.g. file systems, key-value stores, etc). Use :func:`DataFrame.write` to access this. @@ -701,7 +712,7 @@ def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options) In the case the table already exists, behavior of this function depends on the save mode, specified by the `mode` function (default to throwing an exception). - When `mode` is `Overwrite`, the schema of the [[DataFrame]] does not need to be + When `mode` is `Overwrite`, the schema of the :class:`DataFrame` does not need to be the same as that of the existing table. * `append`: Append contents of this :class:`DataFrame` to existing data. @@ -758,7 +769,9 @@ def parquet(self, path, mode=None, partitionBy=None, compression=None): :param partitionBy: names of partitioning columns :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, snappy, gzip, and lzo). - This will overwrite ``spark.sql.parquet.compression.codec``. + This will override ``spark.sql.parquet.compression.codec``. If None + is set, it uses the value specified in + ``spark.sql.parquet.compression.codec``. >>> df.write.parquet(os.path.join(tempfile.mkdtemp(), 'data')) """ @@ -788,7 +801,7 @@ def text(self, path, compression=None): @since(2.0) def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, header=None, nullValue=None, escapeQuotes=None): - """Saves the content of the [[DataFrame]] in CSV format at the specified path. + """Saves the content of the :class:`DataFrame` in CSV format at the specified path. :param path: the path in any Hadoop supported file system :param mode: specifies the behavior of the save operation when data already exists. @@ -852,7 +865,8 @@ def orc(self, path, mode=None, partitionBy=None, compression=None): :param partitionBy: names of partitioning columns :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, snappy, zlib, and lzo). - This will overwrite ``orc.compress``. + This will override ``orc.compress``. If None is set, it uses the + default value, ``snappy``. >>> orc_df = spark.read.orc('python/test_support/sql/orc_partitioned') >>> orc_df.write.orc(os.path.join(tempfile.mkdtemp(), 'data')) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index bb5fa2b99fc0c..078b63ee87683 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -304,9 +304,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `DROPMALFORMED` : ignores the whole corrupted records.
  • *
  • `FAILFAST` : throws an exception when it meets corrupted records.
  • * - *
  • `columnNameOfCorruptRecord` (default `_corrupt_record`): allows renaming the new field - * having malformed string created by `PERMISSIVE` mode. This overrides - * `spark.sql.columnNameOfCorruptRecord`.
  • + *
  • `columnNameOfCorruptRecord` (default is the value specified in + * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string + * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
  • * * @since 1.6.0 */ @@ -361,8 +361,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Loads a CSV file and returns the result as a [[DataFrame]]. * - * This function goes through the input once to determine the input schema. To avoid going - * through the entire data once, specify the schema explicitly using [[schema]]. + * This function will go through the input once to determine the input schema if `inferSchema` + * is enabled. To avoid going through the entire data once, disable `inferSchema` option or + * specify the schema explicitly using [[schema]]. * * You can set the following CSV-specific options to deal with CSV files: *
  • `sep` (default `,`): sets the single character as a separator for each @@ -378,6 +379,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `comment` (default empty string): sets the single character used for skipping lines * beginning with this character. By default, it is disabled.
  • *
  • `header` (default `false`): uses the first line as names of columns.
  • + *
  • `inferSchema` (default `false`): infers the input schema automatically from data. It + * requires one extra pass over the data.
  • *
  • `ignoreLeadingWhiteSpace` (default `false`): defines whether or not leading whitespaces * from values being read should be skipped.
  • *
  • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing @@ -414,6 +417,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * Loads a Parquet file, returning the result as a [[DataFrame]]. This function returns an empty * [[DataFrame]] if no paths are passed in. * + * You can set the following Parquet-specific option(s) for reading Parquet files: + *
  • `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets + * whether we should merge schemas collected from all Parquet part-files. This will override + * `spark.sql.parquet.mergeSchema`.
  • + * * @since 1.4.0 */ @scala.annotation.varargs diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 1c2003c18e3fc..8c05a7fce31ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -725,9 +725,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following Parquet-specific option(s) for writing Parquet files: - *
  • `compression` (default `null`): compression codec to use when saving to file. This can be - * one of the known case-insensitive shorten names(`none`, `snappy`, `gzip`, and `lzo`). - * This will overwrite `spark.sql.parquet.compression.codec`.
  • + *
  • `compression` (default is the value specified in `spark.sql.parquet.compression.codec`): + * compression codec to use when saving to file. This can be one of the known case-insensitive + * shorten names(none, `snappy`, `gzip`, and `lzo`). This will override + * `spark.sql.parquet.compression.codec`.
  • * * @since 1.4.0 */ @@ -744,9 +745,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following ORC-specific option(s) for writing ORC files: - *
  • `compression` (default `null`): compression codec to use when saving to file. This can be + *
  • `compression` (default `snappy`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names(`none`, `snappy`, `zlib`, and `lzo`). - * This will overwrite `orc.compress`.
  • + * This will override `orc.compress`. * * @since 1.5.0 * @note Currently, this method can only be used after enabling Hive support diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 3735c9496888b..2d4bef3f18855 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -144,12 +144,10 @@ private[sql] class ParquetFileFormat sparkSession: SparkSession, parameters: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { + val parquetOptions = new ParquetOptions(parameters, sparkSession.sessionState.conf) + // Should we merge schemas from all Parquet part-files? - val shouldMergeSchemas = - parameters - .get(ParquetFileFormat.MERGE_SCHEMA) - .map(_.toBoolean) - .getOrElse(sparkSession.conf.get(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED)) + val shouldMergeSchemas = parquetOptions.mergeSchema val mergeRespectSummaries = sparkSession.conf.get(SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES) @@ -558,17 +556,6 @@ private[sql] class ParquetOutputWriter( } private[sql] object ParquetFileFormat extends Logging { - // Whether we should merge schemas collected from all Parquet part-files. - private[sql] val MERGE_SCHEMA = "mergeSchema" - - // Hive Metastore schema, used when converting Metastore Parquet tables. This option is only used - // internally. - private[sql] val METASTORE_SCHEMA = "metastoreSchema" - - // If a ParquetRelation is converted from a Hive metastore table, this option is set to the - // original Hive table name. - private[sql] val METASTORE_TABLE_NAME = "metastoreTableName" - /** * If parquet's block size (row group size) setting is larger than the min split size, * we use parquet's block size setting as the min split size. Otherwise, we will create diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 1ff217cbf0d4e..dd2e915e7b7f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -private[parquet] class ParquetOptions( +private[sql] class ParquetOptions( @transient private val parameters: Map[String, String], @transient private val sqlConf: SQLConf) extends Serializable { @@ -44,10 +44,21 @@ private[parquet] class ParquetOptions( } shortParquetCompressionCodecNames(codecName).name() } + + /** + * Whether it merges schemas or not. When the given Parquet files have different schemas, + * the schemas can be merged. By default use the value specified in SQLConf. + */ + val mergeSchema: Boolean = parameters + .get(MERGE_SCHEMA) + .map(_.toBoolean) + .getOrElse(sqlConf.getConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED)) } -private[parquet] object ParquetOptions { +private[sql] object ParquetOptions { + private[sql] val MERGE_SCHEMA = "mergeSchema" + // The parquet compression short names private val shortParquetCompressionCodecNames = Map( "none" -> CompressionCodecName.UNCOMPRESSED, 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 f10afa75f2bfc..d24cde232119c 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 @@ -23,7 +23,7 @@ import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.command.CreateHiveTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ @@ -355,13 +355,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val fileFormatClass = classOf[ParquetFileFormat] val mergeSchema = sessionState.convertMetastoreParquetWithSchemaMerging - val options = Map( - ParquetFileFormat.MERGE_SCHEMA -> mergeSchema.toString, - ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier( - relation.tableName, - Some(relation.databaseName) - ).unquotedString - ) + val options = Map(ParquetOptions.MERGE_SCHEMA -> mergeSchema.toString) convertToLogicalRelation(relation, options, defaultSource, fileFormatClass, "parquet") } From 879e8fd09477fc78d66c9da9e0e117a513b0b046 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 12 Jun 2016 12:54:57 +0100 Subject: [PATCH 0607/1470] [SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and ReplayListenerSuite ## What changes were proposed in this pull request? These tests weren't properly using `LocalSparkContext` so weren't cleaning up correctly when tests failed. ## How was this patch tested? Jenkins. Author: Imran Rashid Closes #13602 from squito/SPARK-15878_cleanup_replaylistener. (cherry picked from commit 8cc22b0085475a188f229536b4f83988ae889a8e) Signed-off-by: Sean Owen --- .../apache/spark/scheduler/EventLoggingListenerSuite.scala | 2 +- .../org/apache/spark/scheduler/ReplayListenerSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) 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 176d8930aad19..c4c80b5b57daa 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -181,7 +181,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // into SPARK-6688. val conf = getLoggingConf(testDirPath, compressionCodec) .set("spark.hadoop.fs.defaultFS", "unsupported://example.com") - val sc = new SparkContext("local-cluster[2,2,1024]", "test", conf) + sc = new SparkContext("local-cluster[2,2,1024]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val eventLogPath = eventLogger.logPath diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 35215c15ea805..1732aca9417ea 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -23,7 +23,7 @@ import java.net.URI import org.json4s.jackson.JsonMethods._ import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} @@ -31,7 +31,7 @@ import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} /** * Test whether ReplayListenerBus replays events from logs correctly. */ -class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { +class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) private var testDir: File = _ @@ -101,7 +101,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { fileSystem.mkdirs(logDirPath) val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) - val sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf) + sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf) // Run a few jobs sc.parallelize(1 to 100, 1).count() From 8c294f4ad95e95f6c8873d7b346394d34cc40975 Mon Sep 17 00:00:00 2001 From: bomeng Date: Sun, 12 Jun 2016 12:58:34 +0100 Subject: [PATCH 0608/1470] [SPARK-15781][DOCUMENTATION] remove deprecated environment variable doc ## What changes were proposed in this pull request? Like `SPARK_JAVA_OPTS` and `SPARK_CLASSPATH`, we will remove the document for `SPARK_WORKER_INSTANCES` to discourage user not to use them. If they are actually used, SparkConf will show a warning message as before. ## How was this patch tested? Manually tested. Author: bomeng Closes #13533 from bomeng/SPARK-15781. (cherry picked from commit 3fd3ee038b89821f51f30a4ecd4452b5b3bc6568) Signed-off-by: Sean Owen --- docs/spark-standalone.md | 9 --------- 1 file changed, 9 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index fd94c34d1638d..40c72931cbac9 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -133,15 +133,6 @@ You can optionally configure the cluster further by setting environment variable SPARK_WORKER_WEBUI_PORT Port for the worker web UI (default: 8081). - - SPARK_WORKER_INSTANCES - - Number of worker instances to run on each machine (default: 1). You can make this more than 1 if - you have have very large machines and would like multiple Spark worker processes. If you do set - this, make sure to also set SPARK_WORKER_CORES explicitly to limit the cores per worker, - or else each worker will try to use all the cores. - - SPARK_WORKER_DIR Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work). From b75d1c20131b438999645d0be6ea5765a2f7da80 Mon Sep 17 00:00:00 2001 From: bomeng Date: Sun, 12 Jun 2016 14:25:48 +0100 Subject: [PATCH 0609/1470] [SPARK-15806][DOCUMENTATION] update doc for SPARK_MASTER_IP ## What changes were proposed in this pull request? SPARK_MASTER_IP is a deprecated environment variable. It is replaced by SPARK_MASTER_HOST according to MasterArguments.scala. ## How was this patch tested? Manually verified. Author: bomeng Closes #13543 from bomeng/SPARK-15806. (cherry picked from commit 50248dcfff3ba79b73323f3a804c1e19a8be6097) Signed-off-by: Sean Owen --- conf/spark-env.sh.template | 2 +- .../org/apache/spark/deploy/master/MasterArguments.scala | 8 +++++++- docs/spark-standalone.md | 4 ++-- sbin/start-master.sh | 6 +++--- sbin/start-slaves.sh | 6 +++--- 5 files changed, 16 insertions(+), 10 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 9cffdc30c20fe..c750c72d19880 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -42,7 +42,7 @@ # - SPARK_DRIVER_MEMORY, Memory for Driver (e.g. 1000M, 2G) (Default: 1G) # Options for the daemons used in the standalone deploy mode -# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname +# - SPARK_MASTER_HOST, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 585e0839d0fc1..c63793c16dcef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -20,18 +20,24 @@ package org.apache.spark.deploy.master import scala.annotation.tailrec import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.{IntParam, Utils} /** * Command-line parser for the master. */ -private[master] class MasterArguments(args: Array[String], conf: SparkConf) { +private[master] class MasterArguments(args: Array[String], conf: SparkConf) extends Logging { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 var propertiesFile: String = null // Check for settings in environment variables + if (System.getenv("SPARK_MASTER_IP") != null) { + logWarning("SPARK_MASTER_IP is deprecated, please use SPARK_MASTER_HOST") + host = System.getenv("SPARK_MASTER_IP") + } + if (System.getenv("SPARK_MASTER_HOST") != null) { host = System.getenv("SPARK_MASTER_HOST") } diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 40c72931cbac9..c864c9030835e 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -94,8 +94,8 @@ You can optionally configure the cluster further by setting environment variable - - + + diff --git a/sbin/start-master.sh b/sbin/start-master.sh index ce7f17795997e..981cb15bc0006 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -47,8 +47,8 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then SPARK_MASTER_PORT=7077 fi -if [ "$SPARK_MASTER_IP" = "" ]; then - SPARK_MASTER_IP=`hostname` +if [ "$SPARK_MASTER_HOST" = "" ]; then + SPARK_MASTER_HOST=`hostname` fi if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then @@ -56,5 +56,5 @@ if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then fi "${SPARK_HOME}/sbin"/spark-daemon.sh start $CLASS 1 \ - --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT \ + --host $SPARK_MASTER_HOST --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT \ $ORIGINAL_ARGS diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 5bf2b83b42ce4..0fa1605489704 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -31,9 +31,9 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then SPARK_MASTER_PORT=7077 fi -if [ "$SPARK_MASTER_IP" = "" ]; then - SPARK_MASTER_IP="`hostname`" +if [ "$SPARK_MASTER_HOST" = "" ]; then + SPARK_MASTER_HOST="`hostname`" fi # Launch the slaves -"${SPARK_HOME}/sbin/slaves.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin/start-slave.sh" "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" +"${SPARK_HOME}/sbin/slaves.sh" cd "${SPARK_HOME}" \; "${SPARK_HOME}/sbin/start-slave.sh" "spark://$SPARK_MASTER_HOST:$SPARK_MASTER_PORT" From f703dff0ac02fdb9eb01d3054892d1d9c65f0c9e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 12 Jun 2016 11:44:33 -0700 Subject: [PATCH 0610/1470] [SPARK-15086][CORE][STREAMING] Deprecate old Java accumulator API ## What changes were proposed in this pull request? - Deprecate old Java accumulator API; should use Scala now - Update Java tests and examples - Don't bother testing old accumulator API in Java 8 (too) - (fix a misspelling too) ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #13606 from srowen/SPARK-15086. (cherry picked from commit f51dfe616b24b4234199c98ea857a586a93a889f) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/SparkContext.scala | 6 +-- .../spark/api/java/JavaSparkContext.scala | 12 +++++- .../java/org/apache/spark/JavaAPISuite.java | 6 ++- docs/programming-guide.md | 4 +- docs/streaming-programming-guide.md | 8 ++-- .../JavaRecoverableNetworkWordCount.java | 10 ++--- .../apache/spark/java8/Java8RDDAPISuite.java | 39 ------------------- .../spark/java8/dstream/Java8APISuite.java | 28 ------------- .../apache/spark/sql/JavaDatasetSuite.java | 4 +- .../apache/spark/streaming/JavaAPISuite.java | 6 +-- 10 files changed, 33 insertions(+), 90 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 230fabd211701..8bfe7ecd8f3f4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1245,7 +1245,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values - * with `+=`. Only the driver can access the accumuable's `value`. + * with `+=`. Only the driver can access the accumulable's `value`. * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ @@ -1259,8 +1259,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the - * Spark UI. Tasks can add values to the accumuable using the `+=` operator. Only the driver can - * access the accumuable's `value`. + * Spark UI. Tasks can add values to the accumulable using the `+=` operator. Only the driver can + * access the accumulable's `value`. * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 485a8b4222e5a..131f36f5470f0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -530,6 +530,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ + @deprecated("use sc().longAccumulator()", "2.0.0") def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] = sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]] @@ -539,6 +540,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use sc().longAccumulator(String)", "2.0.0") def intAccumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = sc.accumulator(initialValue, name)(IntAccumulatorParam) .asInstanceOf[Accumulator[java.lang.Integer]] @@ -547,6 +549,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ + @deprecated("use sc().doubleAccumulator()", "2.0.0") def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] @@ -556,6 +559,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use sc().doubleAccumulator(String)", "2.0.0") def doubleAccumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = sc.accumulator(initialValue, name)(DoubleAccumulatorParam) .asInstanceOf[Accumulator[java.lang.Double]] @@ -564,6 +568,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ + @deprecated("use sc().longAccumulator()", "2.0.0") def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue) /** @@ -572,6 +577,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use sc().longAccumulator(String)", "2.0.0") def accumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = intAccumulator(initialValue, name) @@ -579,6 +585,7 @@ class JavaSparkContext(val sc: SparkContext) * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ + @deprecated("use sc().doubleAccumulator()", "2.0.0") def accumulator(initialValue: Double): Accumulator[java.lang.Double] = doubleAccumulator(initialValue) @@ -589,6 +596,7 @@ class JavaSparkContext(val sc: SparkContext) * * This version supports naming the accumulator for display in Spark's web UI. */ + @deprecated("use sc().doubleAccumulator(String)", "2.0.0") def accumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = doubleAccumulator(initialValue, name) @@ -613,7 +621,7 @@ class JavaSparkContext(val sc: SparkContext) /** * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks - * can "add" values with `add`. Only the master can access the accumuable's `value`. + * can "add" values with `add`. Only the master can access the accumulable's `value`. */ @deprecated("use AccumulatorV2", "2.0.0") def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = @@ -621,7 +629,7 @@ class JavaSparkContext(val sc: SparkContext) /** * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks - * can "add" values with `add`. Only the master can access the accumuable's `value`. + * can "add" values with `add`. Only the master can access the accumulable's `value`. * * This version supports naming the accumulator for display in Spark's web UI. */ diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 04f92d60167d8..7bac0683212b3 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -70,6 +70,7 @@ import org.apache.spark.rdd.RDD; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.util.LongAccumulator; import org.apache.spark.util.StatCounter; // The test suite itself is Serializable so that anonymous Function implementations can be @@ -287,7 +288,7 @@ public Integer call(Tuple2 t) { @Test public void foreach() { - final Accumulator accum = sc.accumulator(0); + final LongAccumulator accum = sc.sc().longAccumulator(); JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); rdd.foreach(new VoidFunction() { @Override @@ -300,7 +301,7 @@ public void call(String s) { @Test public void foreachPartition() { - final Accumulator accum = sc.accumulator(0); + final LongAccumulator accum = sc.sc().longAccumulator(); JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); rdd.foreachPartition(new VoidFunction>() { @Override @@ -1377,6 +1378,7 @@ public Iterator call(Iterator i, Iterator s) { assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); } + @SuppressWarnings("deprecation") @Test public void accumulators() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 70fd627c6fe69..3f081a0e8f06e 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1394,7 +1394,7 @@ Note that, when programmers define their own type of AccumulatorV2, the resultin
    {% highlight java %} -Accumulator accum = sc.accumulator(0); +LongAccumulator accum = sc.sc().longAccumulator(); sc.parallelize(Arrays.asList(1, 2, 3, 4)).foreach(x -> accum.add(x)); // ... @@ -1485,7 +1485,7 @@ data.map { x => accum += x; x }
    {% highlight java %} -Accumulator accum = sc.accumulator(0); +LongAccumulator accum = sc.sc().longAccumulator(); data.map(x -> { accum.add(x); return f(x); }); // Here, accum is still 0 because no actions have caused the `map` to be computed. {% endhighlight %} diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 0a6a0397d9570..4ea3b60268d12 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1452,13 +1452,13 @@ class JavaWordBlacklist { class JavaDroppedWordsCounter { - private static volatile Accumulator instance = null; + private static volatile LongAccumulator instance = null; - public static Accumulator getInstance(JavaSparkContext jsc) { + public static LongAccumulator getInstance(JavaSparkContext jsc) { if (instance == null) { synchronized (JavaDroppedWordsCounter.class) { if (instance == null) { - instance = jsc.accumulator(0, "WordsInBlacklistCounter"); + instance = jsc.sc().longAccumulator("WordsInBlacklistCounter"); } } } @@ -1472,7 +1472,7 @@ wordCounts.foreachRDD(new Function2, Time, Void>() // Get or register the blacklist Broadcast final Broadcast> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); // Get or register the droppedWordsCounter Accumulator - final Accumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); + final LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); // Use blacklist to drop words and use droppedWordsCounter to count them String counts = rdd.filter(new Function, Boolean>() { @Override diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java index 05631494484be..acbc34524328b 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java @@ -29,7 +29,6 @@ import com.google.common.io.Files; -import org.apache.spark.Accumulator; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -41,6 +40,7 @@ import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.util.LongAccumulator; /** * Use this singleton to get or register a Broadcast variable. @@ -67,13 +67,13 @@ public static Broadcast> getInstance(JavaSparkContext jsc) { */ class JavaDroppedWordsCounter { - private static volatile Accumulator instance = null; + private static volatile LongAccumulator instance = null; - public static Accumulator getInstance(JavaSparkContext jsc) { + public static LongAccumulator getInstance(JavaSparkContext jsc) { if (instance == null) { synchronized (JavaDroppedWordsCounter.class) { if (instance == null) { - instance = jsc.accumulator(0, "WordsInBlacklistCounter"); + instance = jsc.sc().longAccumulator("WordsInBlacklistCounter"); } } } @@ -158,7 +158,7 @@ public void call(JavaPairRDD rdd, Time time) throws IOException final Broadcast> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); // Get or register the droppedWordsCounter Accumulator - final Accumulator droppedWordsCounter = + final LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); // Use blacklist to drop words and use droppedWordsCounter to count them String counts = rdd.filter(new Function, Boolean>() { diff --git a/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java index 8ee0e7e4156bf..fa3a66e73ced6 100644 --- a/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java +++ b/external/java8-tests/src/test/java/test/org/apache/spark/java8/Java8RDDAPISuite.java @@ -33,8 +33,6 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.Accumulator; -import org.apache.spark.AccumulatorParam; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -302,43 +300,6 @@ public void zipPartitions() { Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); } - @Test - public void accumulators() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - - Accumulator intAccum = sc.intAccumulator(10); - rdd.foreach(intAccum::add); - Assert.assertEquals((Integer) 25, intAccum.value()); - - Accumulator doubleAccum = sc.doubleAccumulator(10.0); - rdd.foreach(x -> doubleAccum.add((double) x)); - Assert.assertEquals((Double) 25.0, doubleAccum.value()); - - // Try a custom accumulator type - AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { - @Override - public Float addInPlace(Float r, Float t) { - return r + t; - } - @Override - public Float addAccumulator(Float r, Float t) { - return r + t; - } - @Override - public Float zero(Float initialValue) { - return 0.0f; - } - }; - - Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); - rdd.foreach(x -> floatAccum.add((float) x)); - Assert.assertEquals((Float) 25.0f, floatAccum.value()); - - // Test the setValue method - floatAccum.setValue(5.0f); - Assert.assertEquals((Float) 5.0f, floatAccum.value()); - } - @Test public void keyBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); diff --git a/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java b/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java index cf5607f5e814e..338ca54ab8292 100644 --- a/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java +++ b/external/java8-tests/src/test/java/test/org/apache/spark/java8/dstream/Java8APISuite.java @@ -27,7 +27,6 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.spark.Accumulator; import org.apache.spark.HashPartitioner; import org.apache.spark.api.java.Optional; import org.apache.spark.api.java.JavaPairRDD; @@ -361,33 +360,6 @@ public void testFlatMap() { assertOrderInvariantEquals(expected, result); } - @Test - public void testForeachRDD() { - final Accumulator accumRdd = ssc.sparkContext().accumulator(0); - final Accumulator accumEle = ssc.sparkContext().accumulator(0); - List> inputData = Arrays.asList( - Arrays.asList(1,1,1), - Arrays.asList(1,1,1)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaTestUtils.attachTestOutputStream(stream.count()); // dummy output - - stream.foreachRDD(rdd -> { - accumRdd.add(1); - rdd.foreach(x -> accumEle.add(1)); - }); - - // This is a test to make sure foreachRDD(VoidFunction2) can be called from Java - stream.foreachRDD((rdd, time) -> { - return; - }); - - JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(2, accumRdd.value().intValue()); - Assert.assertEquals(6, accumEle.value().intValue()); - } - @Test public void testPairFlatMap() { List> inputData = Arrays.asList( diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 37577accfda21..a711811f418c6 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -32,7 +32,6 @@ import org.junit.*; import org.junit.rules.ExpectedException; -import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.sql.*; @@ -40,6 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericRow; import org.apache.spark.sql.test.TestSparkSession; import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.LongAccumulator; import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.expr; import static org.apache.spark.sql.types.DataTypes.*; @@ -157,7 +157,7 @@ public Iterator call(String s) { @Test public void testForeach() { - final Accumulator accum = jsc.accumulator(0); + final LongAccumulator accum = jsc.sc().longAccumulator(); List data = Arrays.asList("a", "b", "c"); Dataset ds = spark.createDataset(data, Encoders.STRING()); diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 01f0c4de9e3c9..3d54abd903b6d 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -36,7 +36,6 @@ import com.google.common.io.Files; import com.google.common.collect.Sets; -import org.apache.spark.Accumulator; import org.apache.spark.HashPartitioner; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; @@ -46,6 +45,7 @@ import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.*; +import org.apache.spark.util.LongAccumulator; import org.apache.spark.util.Utils; // The test suite itself is Serializable so that anonymous Function implementations can be @@ -794,8 +794,8 @@ public Iterator call(String x) { @SuppressWarnings("unchecked") @Test public void testForeachRDD() { - final Accumulator accumRdd = ssc.sparkContext().accumulator(0); - final Accumulator accumEle = ssc.sparkContext().accumulator(0); + final LongAccumulator accumRdd = ssc.sparkContext().sc().longAccumulator(); + final LongAccumulator accumEle = ssc.sparkContext().sc().longAccumulator(); List> inputData = Arrays.asList( Arrays.asList(1,1,1), Arrays.asList(1,1,1)); From 161d02db6d3b6148c8b561c6ccb5e54cafb7619d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 12 Jun 2016 11:46:33 -0700 Subject: [PATCH 0611/1470] [SPARK-15876][CORE] Remove support for "zk://" master URL ## What changes were proposed in this pull request? Remove deprecated support for `zk://` master (`mesos://zk//` remains supported) ## How was this patch tested? Jenkins Author: Sean Owen Closes #13625 from srowen/SPARK-15876. (cherry picked from commit 0a6f090837d8d5f6efa809fa976f09b3f0067602) Signed-off-by: Reynold Xin --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 ------ .../apache/spark/SparkContextSchedulerCreationSuite.scala | 4 ---- 2 files changed, 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8bfe7ecd8f3f4..3c5498782ce0e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2417,7 +2417,6 @@ object SparkContext extends Logging { * Create a task scheduler based on a given master URL. * Return a 2-tuple of the scheduler backend and the task scheduler. */ - @tailrec private def createTaskScheduler( sc: SparkContext, master: String, @@ -2495,11 +2494,6 @@ object SparkContext extends Logging { scheduler.initialize(backend) (backend, scheduler) - case zkUrl if zkUrl.startsWith("zk://") => - logWarning("Master URL for a multi-master Mesos cluster managed by ZooKeeper should be " + - "in the form mesos://zk://host:port. Current Master URL will stop working in Spark 2.0.") - createTaskScheduler(sc, "mesos://" + zkUrl, deployMode) - case masterUrl => val cm = getClusterManager(masterUrl) match { case Some(clusterMgr) => clusterMgr diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 11be40abca099..6e5655437aa8c 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -157,8 +157,4 @@ class SparkContextSchedulerCreationSuite classOf[MesosFineGrainedSchedulerBackend], coarse = false) } - test("mesos with zookeeper and Master URL starting with zk://") { - testMesos("zk://localhost:1234,localhost:2345", - classOf[MesosFineGrainedSchedulerBackend], coarse = false) - } } From 8e371670fe71b1745f56e801ffadcaca185b3e44 Mon Sep 17 00:00:00 2001 From: frreiss Date: Sun, 12 Jun 2016 14:21:10 -0700 Subject: [PATCH 0612/1470] [SPARK-15370][SQL] Update RewriteCorrelatedScalarSubquery rule to fix COUNT bug ## What changes were proposed in this pull request? This pull request fixes the COUNT bug in the `RewriteCorrelatedScalarSubquery` rule. After this change, the rule tests the expression at the root of the correlated subquery to determine whether the expression returns NULL on empty input. If the expression does not return NULL, the rule generates additional logic in the Project operator above the rewritten subquery. This additional logic intercepts NULL values coming from the outer join and replaces them with the value that the subquery's expression would return on empty input. ## How was this patch tested? Added regression tests to cover all branches of the updated rule (see changes to `SubquerySuite.scala`). Ran all existing automated regression tests after merging with latest trunk. Author: frreiss Closes #13155 from frreiss/master. (cherry picked from commit 9770f6ee60f6834e4e1200234109120427a5cc0d) Signed-off-by: Herman van Hovell --- .../sql/catalyst/expressions/predicates.scala | 7 +- .../sql/catalyst/optimizer/Optimizer.scala | 198 +++++++++++++++++- .../org/apache/spark/sql/SubquerySuite.scala | 81 +++++++ 3 files changed, 280 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 8a6cf53782b91..a3b098afe5728 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 @@ -69,8 +69,11 @@ trait PredicateHelper { protected def replaceAlias( condition: Expression, aliases: AttributeMap[Expression]): Expression = { - condition.transform { - case a: Attribute => aliases.getOrElse(a, a) + // Use transformUp to prevent infinite recursion when the replacement expression + // redefines the same ExprId, + condition.transformUp { + case a: Attribute => + aliases.getOrElse(a, a) } } 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 aaf952f7b22ba..39169f665764c 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 @@ -527,7 +527,8 @@ object CollapseProject extends Rule[LogicalPlan] { // Substitute any attributes that are produced by the lower projection, so that we safely // eliminate it. // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' - val rewrittenUpper = upper.map(_.transform { + // Use transformUp to prevent infinite recursion. + val rewrittenUpper = upper.map(_.transformUp { case a: Attribute => aliases.getOrElse(a, a) }) // collapse upper and lower Projects may introduce unnecessary Aliases, trim them here. @@ -1722,6 +1723,128 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { newExpression.asInstanceOf[E] } + /** + * Statically evaluate an expression containing zero or more placeholders, given a set + * of bindings for placeholder values. + */ + private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = { + val rewrittenExpr = expr transform { + case r @ AttributeReference(_, dataType, _, _) => + bindings(r.exprId) match { + case Some(v) => Literal.create(v, dataType) + case None => Literal.default(NullType) + } + } + Option(rewrittenExpr.eval()) + } + + /** + * Statically evaluate an expression containing one or more aggregates on an empty input. + */ + private def evalAggOnZeroTups(expr: Expression) : Option[Any] = { + // AggregateExpressions are Unevaluable, so we need to replace all aggregates + // in the expression with the value they would return for zero input tuples. + // Also replace attribute refs (for example, for grouping columns) with NULL. + val rewrittenExpr = expr transform { + case a @ AggregateExpression(aggFunc, _, _, resultId) => + aggFunc.defaultResult.getOrElse(Literal.default(NullType)) + + case AttributeReference(_, _, _, _) => Literal.default(NullType) + } + Option(rewrittenExpr.eval()) + } + + /** + * Statically evaluate a scalar subquery on an empty input. + * + * WARNING: This method only covers subqueries that pass the checks under + * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in + * CheckAnalysis become less restrictive, this method will need to change. + */ + private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = { + // Inputs to this method will start with a chain of zero or more SubqueryAlias + // and Project operators, followed by an optional Filter, followed by an + // Aggregate. Traverse the operators recursively. + def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = { + lp match { + case SubqueryAlias(_, child) => evalPlan(child) + case Filter(condition, child) => + val bindings = evalPlan(child) + if (bindings.isEmpty) bindings + else { + val exprResult = evalExpr(condition, bindings).getOrElse(false) + .asInstanceOf[Boolean] + if (exprResult) bindings else Map.empty + } + + case Project(projectList, child) => + val bindings = evalPlan(child) + if (bindings.isEmpty) { + bindings + } else { + projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap + } + + case Aggregate(_, aggExprs, _) => + // Some of the expressions under the Aggregate node are the join columns + // for joining with the outer query block. Fill those expressions in with + // nulls and statically evaluate the remainder. + aggExprs.map(ne => ne match { + case AttributeReference(_, _, _, _) => (ne.exprId, None) + case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None) + case _ => (ne.exprId, evalAggOnZeroTups(ne)) + }).toMap + + case _ => sys.error(s"Unexpected operator in scalar subquery: $lp") + } + } + + val resultMap = evalPlan(plan) + + // By convention, the scalar subquery result is the leftmost field. + resultMap(plan.output.head.exprId) + } + + /** + * Split the plan for a scalar subquery into the parts above the innermost query block + * (first part of returned value), the HAVING clause of the innermost query block + * (optional second part) and the parts below the HAVING CLAUSE (third part). + */ + private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = { + val topPart = ArrayBuffer.empty[LogicalPlan] + var bottomPart : LogicalPlan = plan + while (true) { + bottomPart match { + case havingPart@Filter(_, aggPart@Aggregate(_, _, _)) => + return (topPart, Option(havingPart), aggPart.asInstanceOf[Aggregate]) + + case aggPart@Aggregate(_, _, _) => + // No HAVING clause + return (topPart, None, aggPart) + + case p@Project(_, child) => + topPart += p + bottomPart = child + + case s@SubqueryAlias(_, child) => + topPart += s + bottomPart = child + + case Filter(_, op@_) => + sys.error(s"Correlated subquery has unexpected operator $op below filter") + + case op@_ => sys.error(s"Unexpected operator $op in correlated subquery") + } + } + + sys.error("This line should be unreachable") + } + + + + // Name of generated column used in rewrite below + val ALWAYS_TRUE_COLNAME = "alwaysTrue" + /** * Construct a new child plan by left joining the given subqueries to a base plan. */ @@ -1730,9 +1853,76 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = { subqueries.foldLeft(child) { case (currentChild, ScalarSubquery(query, conditions, _)) => - Project( - currentChild.output :+ query.output.head, - Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + val origOutput = query.output.head + + val resultWithZeroTups = evalSubqueryOnZeroTups(query) + if (resultWithZeroTups.isEmpty) { + // CASE 1: Subquery guaranteed not to have the COUNT bug + Project( + currentChild.output :+ origOutput, + Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + } else { + // Subquery might have the COUNT bug. Add appropriate corrections. + val (topPart, havingNode, aggNode) = splitSubquery(query) + + // The next two cases add a leading column to the outer join input to make it + // possible to distinguish between the case when no tuples join and the case + // when the tuple that joins contains null values. + // The leading column always has the value TRUE. + val alwaysTrueExprId = NamedExpression.newExprId + val alwaysTrueExpr = Alias(Literal.TrueLiteral, + ALWAYS_TRUE_COLNAME)(exprId = alwaysTrueExprId) + val alwaysTrueRef = AttributeReference(ALWAYS_TRUE_COLNAME, + BooleanType)(exprId = alwaysTrueExprId) + + val aggValRef = query.output.head + + if (!havingNode.isDefined) { + // CASE 2: Subquery with no HAVING clause + Project( + currentChild.output :+ + Alias( + If(IsNull(alwaysTrueRef), + Literal(resultWithZeroTups.get, origOutput.dataType), + aggValRef), origOutput.name)(exprId = origOutput.exprId), + Join(currentChild, + Project(query.output :+ alwaysTrueExpr, query), + LeftOuter, conditions.reduceOption(And))) + + } else { + // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join. + // Need to modify any operators below the join to pass through all columns + // referenced in the HAVING clause. + var subqueryRoot : UnaryNode = aggNode + val havingInputs : Seq[NamedExpression] = aggNode.output + + topPart.reverse.foreach( + _ match { + case Project(projList, _) => + subqueryRoot = Project(projList ++ havingInputs, subqueryRoot) + case s@SubqueryAlias(alias, _) => subqueryRoot = SubqueryAlias(alias, subqueryRoot) + case op@_ => sys.error(s"Unexpected operator $op in corelated subquery") + } + ) + + // CASE WHEN alwayTrue IS NULL THEN resultOnZeroTups + // WHEN NOT (original HAVING clause expr) THEN CAST(null AS ) + // ELSE (aggregate value) END AS (original column name) + val caseExpr = Alias(CaseWhen( + Seq[(Expression, Expression)] ( + (IsNull(alwaysTrueRef), Literal(resultWithZeroTups.get, origOutput.dataType)), + (Not(havingNode.get.condition), Literal(null, aggValRef.dataType)) + ), aggValRef + ), origOutput.name) (exprId = origOutput.exprId) + + Project( + currentChild.output :+ caseExpr, + Join(currentChild, + Project(subqueryRoot.output :+ alwaysTrueExpr, subqueryRoot), + LeftOuter, conditions.reduceOption(And))) + + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 05491a4a888d4..06ced999740eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -324,4 +324,85 @@ class SubquerySuite extends QueryTest with SharedSQLContext { """.stripMargin), Row(3) :: Nil) } + + test("SPARK-15370: COUNT bug in WHERE clause (Filter)") { + // Case 1: Canonical example of the COUNT bug + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) < l.a"), + Row(1) :: Row(1) :: Row(3) :: Row(6) :: Nil) + // Case 2: count(*) = 0; could be rewritten to NOT EXISTS but currently uses + // a rewrite that is vulnerable to the COUNT bug + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) = 0"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + // Case 3: COUNT bug without a COUNT aggregate + checkAnswer( + sql("select l.a from l where (select sum(r.d) is null from r where l.a = r.c)"), + Row(1) :: Row(1) ::Row(null) :: Row(null) :: Row(6) :: Nil) + } + + test("SPARK-15370: COUNT bug in SELECT clause (Project)") { + checkAnswer( + sql("select a, (select count(*) from r where l.a = r.c) as cnt from l"), + Row(1, 0) :: Row(1, 0) :: Row(2, 2) :: Row(2, 2) :: Row(3, 1) :: Row(null, 0) + :: Row(null, 0) :: Row(6, 1) :: Nil) + } + + test("SPARK-15370: COUNT bug in HAVING clause (Filter)") { + checkAnswer( + sql("select l.a as grp_a from l group by l.a " + + "having (select count(*) from r where grp_a = r.c) = 0 " + + "order by grp_a"), + Row(null) :: Row(1) :: Nil) + } + + test("SPARK-15370: COUNT bug in Aggregate") { + checkAnswer( + sql("select l.a as aval, sum((select count(*) from r where l.a = r.c)) as cnt " + + "from l group by l.a order by aval"), + Row(null, 0) :: Row(1, 0) :: Row(2, 4) :: Row(3, 1) :: Row(6, 1) :: Nil) + } + + test("SPARK-15370: COUNT bug negative examples") { + // Case 1: Potential COUNT bug case that was working correctly prior to the fix + checkAnswer( + sql("select l.a from l where (select sum(r.d) from r where l.a = r.c) is null"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Row(6) :: Nil) + // Case 2: COUNT aggregate but no COUNT bug due to > 0 test. + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) > 0"), + Row(2) :: Row(2) :: Row(3) :: Row(6) :: Nil) + // Case 3: COUNT inside aggregate expression but no COUNT bug. + checkAnswer( + sql("select l.a from l where (select count(*) + sum(r.d) from r where l.a = r.c) = 0"), + Nil) + } + + test("SPARK-15370: COUNT bug in subquery in subquery in subquery") { + checkAnswer( + sql("""select l.a from l + |where ( + | select cntPlusOne + 1 as cntPlusTwo from ( + | select cnt + 1 as cntPlusOne from ( + | select sum(r.c) s, count(*) cnt from r where l.a = r.c having cnt = 0 + | ) + | ) + |) = 2""".stripMargin), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + } + + test("SPARK-15370: COUNT bug with nasty predicate expr") { + checkAnswer( + sql("select l.a from l where " + + "(select case when count(*) = 1 then null else count(*) end as cnt " + + "from r where l.a = r.c) = 0"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + } + + test("SPARK-15370: COUNT bug with attribute ref in subquery input and output ") { + checkAnswer( + sql("select l.b, (select (r.c + count(*)) is null from r where l.a = r.c) from l"), + Row(1.0, false) :: Row(1.0, false) :: Row(2.0, true) :: Row(2.0, true) :: + Row(3.0, false) :: Row(5.0, true) :: Row(null, false) :: Row(null, true) :: Nil) + } } From b699a7b81acd9976c85654acab189291b1d7de9f Mon Sep 17 00:00:00 2001 From: Ioana Delaney Date: Sun, 12 Jun 2016 14:26:29 -0700 Subject: [PATCH 0613/1470] [SPARK-15832][SQL] Embedded IN/EXISTS predicate subquery throws TreeNodeException ## What changes were proposed in this pull request? Queries with embedded existential sub-query predicates throws exception when building the physical plan. Example failing query: ```SQL scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") scala> Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") scala> sql("select c1 from t1 where (case when c2 in (select c2 from t2) then 2 else 3 end) IN (select c2 from t1)").show() Binding attribute, tree: c2#239 org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: c2#239 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) ... at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:87) at org.apache.spark.sql.execution.joins.HashJoin$$anonfun$4.apply(HashJoin.scala:66) at org.apache.spark.sql.execution.joins.HashJoin$$anonfun$4.apply(HashJoin.scala:66) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.immutable.List.map(List.scala:285) at org.apache.spark.sql.execution.joins.HashJoin$class.org$apache$spark$sql$execution$joins$HashJoin$$x$8(HashJoin.scala:66) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.org$apache$spark$sql$execution$joins$HashJoin$$x$8$lzycompute(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.org$apache$spark$sql$execution$joins$HashJoin$$x$8(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.joins.HashJoin$class.buildKeys(HashJoin.scala:63) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.buildKeys$lzycompute(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.buildKeys(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.requiredChildDistribution(BroadcastHashJoinExec.scala:52) ``` **Problem description:** When the left hand side expression of an existential sub-query predicate contains another embedded sub-query predicate, the RewritePredicateSubquery optimizer rule does not resolve the embedded sub-query expressions into existential joins.For example, the above query has the following optimized plan, which fails during physical plan build. ```SQL == Optimized Logical Plan == Project [_1#224 AS c1#227] +- Join LeftSemi, (CASE WHEN predicate-subquery#255 [(_2#225 = c2#239)] THEN 2 ELSE 3 END = c2#228#262) : +- SubqueryAlias predicate-subquery#255 [(_2#225 = c2#239)] : +- LocalRelation [c2#239] :- LocalRelation [_1#224, _2#225] +- LocalRelation [c2#228#262] == Physical Plan == org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: c2#239 ``` **Solution:** In RewritePredicateSubquery, before rewriting the outermost predicate sub-query, resolve any embedded existential sub-queries. The Optimized plan for the above query after the changes looks like below. ```SQL == Optimized Logical Plan == Project [_1#224 AS c1#227] +- Join LeftSemi, (CASE WHEN exists#285 THEN 2 ELSE 3 END = c2#228#284) :- Join ExistenceJoin(exists#285), (_2#225 = c2#239) : :- LocalRelation [_1#224, _2#225] : +- LocalRelation [c2#239] +- LocalRelation [c2#228#284] == Physical Plan == *Project [_1#224 AS c1#227] +- *BroadcastHashJoin [CASE WHEN exists#285 THEN 2 ELSE 3 END], [c2#228#284], LeftSemi, BuildRight :- *BroadcastHashJoin [_2#225], [c2#239], ExistenceJoin(exists#285), BuildRight : :- LocalTableScan [_1#224, _2#225] : +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))) : +- LocalTableScan [c2#239] +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))) +- LocalTableScan [c2#228#284] +- LocalTableScan [c222#36], [[111],[222]] ``` ## How was this patch tested? Added new test cases in SubquerySuite.scala Author: Ioana Delaney Closes #13570 from ioana-delaney/fixEmbedSubPredV1. (cherry picked from commit 0ff8a68b9faefecf90c9ceef49580b2909beb19e) Signed-off-by: Herman van Hovell --- .../sql/catalyst/optimizer/Optimizer.scala | 55 ++++-- .../org/apache/spark/sql/SubquerySuite.scala | 166 ++++++++++++++++++ 2 files changed, 204 insertions(+), 17 deletions(-) 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 39169f665764c..75bd44cb31de4 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 @@ -1677,31 +1677,52 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Filter the plan by applying left semi and left anti joins. withSubquery.foldLeft(newFilter) { case (p, PredicateSubquery(sub, conditions, _, _)) => - Join(p, sub, LeftSemi, conditions.reduceOption(And)) + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions.reduceOption(And), p) + Join(outerPlan, sub, LeftSemi, joinCond) case (p, Not(PredicateSubquery(sub, conditions, false, _))) => - Join(p, sub, LeftAnti, conditions.reduceOption(And)) + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions.reduceOption(And), p) + Join(outerPlan, sub, LeftAnti, joinCond) case (p, Not(PredicateSubquery(sub, conditions, true, _))) => - // This is a NULL-aware (left) anti join (NAAJ). + // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr // Construct the condition. A NULL in one of the conditions is regarded as a positive // result; such a row will be filtered out by the Anti-Join operator. - val anyNull = conditions.map(IsNull).reduceLeft(Or) - val condition = conditions.reduceLeft(And) - // Note that will almost certainly be planned as a Broadcast Nested Loop join. Use EXISTS - // if performance matters to you. - Join(p, sub, LeftAnti, Option(Or(anyNull, condition))) + // Note that will almost certainly be planned as a Broadcast Nested Loop join. + // Use EXISTS if performance matters to you. + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions.reduceLeftOption(And), p) + val anyNull = splitConjunctivePredicates(joinCond.get).map(IsNull).reduceLeft(Or) + Join(outerPlan, sub, LeftAnti, Option(Or(anyNull, joinCond.get))) case (p, predicate) => - var joined = p - val replaced = predicate transformUp { - case PredicateSubquery(sub, conditions, nullAware, _) => - // TODO: support null-aware join - val exists = AttributeReference("exists", BooleanType, nullable = false)() - joined = Join(joined, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) - exists - } - Project(p.output, Filter(replaced, joined)) + val (newCond, inputPlan) = rewriteExistentialExpr(Option(predicate), p) + Project(p.output, Filter(newCond.get, inputPlan)) } } + + /** + * Given a predicate expression and an input plan, it rewrites + * any embedded existential sub-query into an existential join. + * It returns the rewritten expression together with the updated plan. + * Currently, it does not support null-aware joins. Embedded NOT IN predicates + * are blocked in the Analyzer. + */ + private def rewriteExistentialExpr( + expr: Option[Expression], + plan: LogicalPlan): (Option[Expression], LogicalPlan) = { + var newPlan = plan + expr match { + case Some(e) => + val newExpr = e transformUp { + case PredicateSubquery(sub, conditions, nullAware, _) => + // TODO: support null-aware join + val exists = AttributeReference("exists", BooleanType, nullable = false)() + newPlan = Join(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) + exists + } + (Option(newExpr), newPlan) + case None => + (expr, plan) + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 06ced999740eb..1d9ff21dbf5d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -266,6 +266,172 @@ class SubquerySuite extends QueryTest with SharedSQLContext { Row(null) :: Row(1) :: Row(3) :: Nil) } + test("SPARK-15832: Test embedded existential predicate sub-queries") { + withTempTable("t1", "t2", "t3", "t4", "t5") { + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") + Seq((1, 1), (2, 2), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t3") + + checkAnswer( + sql( + """ + | select c1 from t1 + | where c2 IN (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where c2 NOT IN (select c2 from t2) + | + """.stripMargin), + Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where EXISTS (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where NOT EXISTS (select c2 from t2) + | + """.stripMargin), + Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where NOT EXISTS (select c2 from t2) and + | c2 IN (select c2 from t3) + | + """.stripMargin), + Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select 1 as one) then 1 + | else 2 end) = c1 + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select 1 as one) then 1 + | else 2 end) + | IN (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select c2 from t2) then 1 + | else 2 end) + | IN (select c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select c2 from t2) then 1 + | when c2 IN (select c2 from t3) then 2 + | else 3 end) + | IN (select c2 from t1) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (c1, (case when c2 IN (select c2 from t2) then 1 + | when c2 IN (select c2 from t3) then 2 + | else 3 end)) + | IN (select c1, c2 from t1) + | + """.stripMargin), + Row(1) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t3 + | where ((case when c2 IN (select c2 from t2) then 1 else 2 end), + | (case when c2 IN (select c2 from t3) then 2 else 3 end)) + | IN (select c1, c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Row(1) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where ((case when EXISTS (select c2 from t2) then 1 else 2 end), + | (case when c2 IN (select c2 from t3) then 2 else 3 end)) + | IN (select c1, c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select c2 from t2) then 3 + | else 2 end) + | NOT IN (select c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where ((case when c2 IN (select c2 from t2) then 1 else 2 end), + | (case when NOT EXISTS (select c2 from t3) then 2 + | when EXISTS (select c2 from t2) then 3 + | else 3 end)) + | NOT IN (select c1, c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (select max(c1) from t2 where c2 IN (select c2 from t3)) + | IN (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + } + } + test("correlated scalar subquery in where") { checkAnswer( sql("select * from l where b < (select max(d) from r where a = c)"), From e92fb9669bc637cc8800331d7ffe41c53e4162f7 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 12 Jun 2016 14:26:53 -0700 Subject: [PATCH 0614/1470] [SPARK-15892][ML] Incorrectly merged AFTAggregator with zero total count ## What changes were proposed in this pull request? Currently, `AFTAggregator` is not being merged correctly. For example, if there is any single empty partition in the data, this creates an `AFTAggregator` with zero total count which causes the exception below: ``` IllegalArgumentException: u'requirement failed: The number of instances should be greater than 0.0, but got 0.' ``` Please see [AFTSurvivalRegression.scala#L573-L575](https://github.com/apache/spark/blob/6ecedf39b44c9acd58cdddf1a31cf11e8e24428c/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala#L573-L575) as well. Just to be clear, the python example `aft_survival_regression.py` seems using 5 rows. So, if there exist partitions more than 5, it throws the exception above since it contains empty partitions which results in an incorrectly merged `AFTAggregator`. Executing `bin/spark-submit examples/src/main/python/ml/aft_survival_regression.py` on a machine with CPUs more than 5 is being failed because it creates tasks with some empty partitions with defualt configurations (AFAIK, it sets the parallelism level to the number of CPU cores). ## How was this patch tested? An unit test in `AFTSurvivalRegressionSuite.scala` and manually tested by `bin/spark-submit examples/src/main/python/ml/aft_survival_regression.py`. Author: hyukjinkwon Author: Hyukjin Kwon Closes #13619 from HyukjinKwon/SPARK-15892. (cherry picked from commit e3554605b36bdce63ac180cc66dbdee5c1528ec7) Signed-off-by: Joseph K. Bradley --- .../spark/ml/regression/AFTSurvivalRegression.scala | 2 +- .../ml/regression/AFTSurvivalRegressionSuite.scala | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index e5f23f44bc5ee..7f57af19e9df9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -538,7 +538,7 @@ private class AFTAggregator( * @return This AFTAggregator object. */ def merge(other: AFTAggregator): this.type = { - if (totalCnt != 0) { + if (other.count != 0) { totalCnt += other.totalCnt lossSum += other.lossSum diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index 05aae80c660ea..1c70b702de063 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -390,6 +390,18 @@ class AFTSurvivalRegressionSuite testEstimatorAndModelReadWrite(aft, datasetMultivariate, AFTSurvivalRegressionSuite.allParamSettings, checkModelData) } + + test("SPARK-15892: Incorrectly merged AFTAggregator with zero total count") { + // This `dataset` will contain an empty partition because it has two rows but + // the parallelism is bigger than that. Because the issue was about `AFTAggregator`s + // being merged incorrectly when it has an empty partition, running the codes below + // should not throw an exception. + val dataset = spark.createDataFrame( + sc.parallelize(generateAFTInput( + 1, Array(5.5), Array(0.8), 2, 42, 1.0, 2.0, 2.0), numSlices = 3)) + val trainer = new AFTSurvivalRegression() + trainer.fit(dataset) + } } object AFTSurvivalRegressionSuite { From 3a1567a73fe457ac75dda49ea58755b08ecf7c07 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 12 Jun 2016 15:06:37 -0700 Subject: [PATCH 0615/1470] [SPARK-15370][SQL] Revert PR "Update RewriteCorrelatedSuquery rule" This reverts commit 9770f6ee60f6834e4e1200234109120427a5cc0d. Author: Herman van Hovell Closes #13626 from hvanhovell/SPARK-15370-revert. (cherry picked from commit 20b8f2c32af696c3856221c4c4fcd12c3f068af2) Signed-off-by: Herman van Hovell --- .../sql/catalyst/expressions/predicates.scala | 7 +- .../sql/catalyst/optimizer/Optimizer.scala | 198 +----------------- .../org/apache/spark/sql/SubquerySuite.scala | 81 ------- 3 files changed, 6 insertions(+), 280 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index a3b098afe5728..8a6cf53782b91 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 @@ -69,11 +69,8 @@ trait PredicateHelper { protected def replaceAlias( condition: Expression, aliases: AttributeMap[Expression]): Expression = { - // Use transformUp to prevent infinite recursion when the replacement expression - // redefines the same ExprId, - condition.transformUp { - case a: Attribute => - aliases.getOrElse(a, a) + condition.transform { + case a: Attribute => aliases.getOrElse(a, a) } } 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 75bd44cb31de4..afe4d25f299e9 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 @@ -527,8 +527,7 @@ object CollapseProject extends Rule[LogicalPlan] { // Substitute any attributes that are produced by the lower projection, so that we safely // eliminate it. // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' - // Use transformUp to prevent infinite recursion. - val rewrittenUpper = upper.map(_.transformUp { + val rewrittenUpper = upper.map(_.transform { case a: Attribute => aliases.getOrElse(a, a) }) // collapse upper and lower Projects may introduce unnecessary Aliases, trim them here. @@ -1744,128 +1743,6 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { newExpression.asInstanceOf[E] } - /** - * Statically evaluate an expression containing zero or more placeholders, given a set - * of bindings for placeholder values. - */ - private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = { - val rewrittenExpr = expr transform { - case r @ AttributeReference(_, dataType, _, _) => - bindings(r.exprId) match { - case Some(v) => Literal.create(v, dataType) - case None => Literal.default(NullType) - } - } - Option(rewrittenExpr.eval()) - } - - /** - * Statically evaluate an expression containing one or more aggregates on an empty input. - */ - private def evalAggOnZeroTups(expr: Expression) : Option[Any] = { - // AggregateExpressions are Unevaluable, so we need to replace all aggregates - // in the expression with the value they would return for zero input tuples. - // Also replace attribute refs (for example, for grouping columns) with NULL. - val rewrittenExpr = expr transform { - case a @ AggregateExpression(aggFunc, _, _, resultId) => - aggFunc.defaultResult.getOrElse(Literal.default(NullType)) - - case AttributeReference(_, _, _, _) => Literal.default(NullType) - } - Option(rewrittenExpr.eval()) - } - - /** - * Statically evaluate a scalar subquery on an empty input. - * - * WARNING: This method only covers subqueries that pass the checks under - * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in - * CheckAnalysis become less restrictive, this method will need to change. - */ - private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = { - // Inputs to this method will start with a chain of zero or more SubqueryAlias - // and Project operators, followed by an optional Filter, followed by an - // Aggregate. Traverse the operators recursively. - def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = { - lp match { - case SubqueryAlias(_, child) => evalPlan(child) - case Filter(condition, child) => - val bindings = evalPlan(child) - if (bindings.isEmpty) bindings - else { - val exprResult = evalExpr(condition, bindings).getOrElse(false) - .asInstanceOf[Boolean] - if (exprResult) bindings else Map.empty - } - - case Project(projectList, child) => - val bindings = evalPlan(child) - if (bindings.isEmpty) { - bindings - } else { - projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap - } - - case Aggregate(_, aggExprs, _) => - // Some of the expressions under the Aggregate node are the join columns - // for joining with the outer query block. Fill those expressions in with - // nulls and statically evaluate the remainder. - aggExprs.map(ne => ne match { - case AttributeReference(_, _, _, _) => (ne.exprId, None) - case Alias(AttributeReference(_, _, _, _), _) => (ne.exprId, None) - case _ => (ne.exprId, evalAggOnZeroTups(ne)) - }).toMap - - case _ => sys.error(s"Unexpected operator in scalar subquery: $lp") - } - } - - val resultMap = evalPlan(plan) - - // By convention, the scalar subquery result is the leftmost field. - resultMap(plan.output.head.exprId) - } - - /** - * Split the plan for a scalar subquery into the parts above the innermost query block - * (first part of returned value), the HAVING clause of the innermost query block - * (optional second part) and the parts below the HAVING CLAUSE (third part). - */ - private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = { - val topPart = ArrayBuffer.empty[LogicalPlan] - var bottomPart : LogicalPlan = plan - while (true) { - bottomPart match { - case havingPart@Filter(_, aggPart@Aggregate(_, _, _)) => - return (topPart, Option(havingPart), aggPart.asInstanceOf[Aggregate]) - - case aggPart@Aggregate(_, _, _) => - // No HAVING clause - return (topPart, None, aggPart) - - case p@Project(_, child) => - topPart += p - bottomPart = child - - case s@SubqueryAlias(_, child) => - topPart += s - bottomPart = child - - case Filter(_, op@_) => - sys.error(s"Correlated subquery has unexpected operator $op below filter") - - case op@_ => sys.error(s"Unexpected operator $op in correlated subquery") - } - } - - sys.error("This line should be unreachable") - } - - - - // Name of generated column used in rewrite below - val ALWAYS_TRUE_COLNAME = "alwaysTrue" - /** * Construct a new child plan by left joining the given subqueries to a base plan. */ @@ -1874,76 +1751,9 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = { subqueries.foldLeft(child) { case (currentChild, ScalarSubquery(query, conditions, _)) => - val origOutput = query.output.head - - val resultWithZeroTups = evalSubqueryOnZeroTups(query) - if (resultWithZeroTups.isEmpty) { - // CASE 1: Subquery guaranteed not to have the COUNT bug - Project( - currentChild.output :+ origOutput, - Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) - } else { - // Subquery might have the COUNT bug. Add appropriate corrections. - val (topPart, havingNode, aggNode) = splitSubquery(query) - - // The next two cases add a leading column to the outer join input to make it - // possible to distinguish between the case when no tuples join and the case - // when the tuple that joins contains null values. - // The leading column always has the value TRUE. - val alwaysTrueExprId = NamedExpression.newExprId - val alwaysTrueExpr = Alias(Literal.TrueLiteral, - ALWAYS_TRUE_COLNAME)(exprId = alwaysTrueExprId) - val alwaysTrueRef = AttributeReference(ALWAYS_TRUE_COLNAME, - BooleanType)(exprId = alwaysTrueExprId) - - val aggValRef = query.output.head - - if (!havingNode.isDefined) { - // CASE 2: Subquery with no HAVING clause - Project( - currentChild.output :+ - Alias( - If(IsNull(alwaysTrueRef), - Literal(resultWithZeroTups.get, origOutput.dataType), - aggValRef), origOutput.name)(exprId = origOutput.exprId), - Join(currentChild, - Project(query.output :+ alwaysTrueExpr, query), - LeftOuter, conditions.reduceOption(And))) - - } else { - // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join. - // Need to modify any operators below the join to pass through all columns - // referenced in the HAVING clause. - var subqueryRoot : UnaryNode = aggNode - val havingInputs : Seq[NamedExpression] = aggNode.output - - topPart.reverse.foreach( - _ match { - case Project(projList, _) => - subqueryRoot = Project(projList ++ havingInputs, subqueryRoot) - case s@SubqueryAlias(alias, _) => subqueryRoot = SubqueryAlias(alias, subqueryRoot) - case op@_ => sys.error(s"Unexpected operator $op in corelated subquery") - } - ) - - // CASE WHEN alwayTrue IS NULL THEN resultOnZeroTups - // WHEN NOT (original HAVING clause expr) THEN CAST(null AS ) - // ELSE (aggregate value) END AS (original column name) - val caseExpr = Alias(CaseWhen( - Seq[(Expression, Expression)] ( - (IsNull(alwaysTrueRef), Literal(resultWithZeroTups.get, origOutput.dataType)), - (Not(havingNode.get.condition), Literal(null, aggValRef.dataType)) - ), aggValRef - ), origOutput.name) (exprId = origOutput.exprId) - - Project( - currentChild.output :+ caseExpr, - Join(currentChild, - Project(subqueryRoot.output :+ alwaysTrueExpr, subqueryRoot), - LeftOuter, conditions.reduceOption(And))) - - } - } + Project( + currentChild.output :+ query.output.head, + Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 1d9ff21dbf5d9..1a99fb683e438 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -490,85 +490,4 @@ class SubquerySuite extends QueryTest with SharedSQLContext { """.stripMargin), Row(3) :: Nil) } - - test("SPARK-15370: COUNT bug in WHERE clause (Filter)") { - // Case 1: Canonical example of the COUNT bug - checkAnswer( - sql("select l.a from l where (select count(*) from r where l.a = r.c) < l.a"), - Row(1) :: Row(1) :: Row(3) :: Row(6) :: Nil) - // Case 2: count(*) = 0; could be rewritten to NOT EXISTS but currently uses - // a rewrite that is vulnerable to the COUNT bug - checkAnswer( - sql("select l.a from l where (select count(*) from r where l.a = r.c) = 0"), - Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) - // Case 3: COUNT bug without a COUNT aggregate - checkAnswer( - sql("select l.a from l where (select sum(r.d) is null from r where l.a = r.c)"), - Row(1) :: Row(1) ::Row(null) :: Row(null) :: Row(6) :: Nil) - } - - test("SPARK-15370: COUNT bug in SELECT clause (Project)") { - checkAnswer( - sql("select a, (select count(*) from r where l.a = r.c) as cnt from l"), - Row(1, 0) :: Row(1, 0) :: Row(2, 2) :: Row(2, 2) :: Row(3, 1) :: Row(null, 0) - :: Row(null, 0) :: Row(6, 1) :: Nil) - } - - test("SPARK-15370: COUNT bug in HAVING clause (Filter)") { - checkAnswer( - sql("select l.a as grp_a from l group by l.a " + - "having (select count(*) from r where grp_a = r.c) = 0 " + - "order by grp_a"), - Row(null) :: Row(1) :: Nil) - } - - test("SPARK-15370: COUNT bug in Aggregate") { - checkAnswer( - sql("select l.a as aval, sum((select count(*) from r where l.a = r.c)) as cnt " + - "from l group by l.a order by aval"), - Row(null, 0) :: Row(1, 0) :: Row(2, 4) :: Row(3, 1) :: Row(6, 1) :: Nil) - } - - test("SPARK-15370: COUNT bug negative examples") { - // Case 1: Potential COUNT bug case that was working correctly prior to the fix - checkAnswer( - sql("select l.a from l where (select sum(r.d) from r where l.a = r.c) is null"), - Row(1) :: Row(1) :: Row(null) :: Row(null) :: Row(6) :: Nil) - // Case 2: COUNT aggregate but no COUNT bug due to > 0 test. - checkAnswer( - sql("select l.a from l where (select count(*) from r where l.a = r.c) > 0"), - Row(2) :: Row(2) :: Row(3) :: Row(6) :: Nil) - // Case 3: COUNT inside aggregate expression but no COUNT bug. - checkAnswer( - sql("select l.a from l where (select count(*) + sum(r.d) from r where l.a = r.c) = 0"), - Nil) - } - - test("SPARK-15370: COUNT bug in subquery in subquery in subquery") { - checkAnswer( - sql("""select l.a from l - |where ( - | select cntPlusOne + 1 as cntPlusTwo from ( - | select cnt + 1 as cntPlusOne from ( - | select sum(r.c) s, count(*) cnt from r where l.a = r.c having cnt = 0 - | ) - | ) - |) = 2""".stripMargin), - Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) - } - - test("SPARK-15370: COUNT bug with nasty predicate expr") { - checkAnswer( - sql("select l.a from l where " + - "(select case when count(*) = 1 then null else count(*) end as cnt " + - "from r where l.a = r.c) = 0"), - Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) - } - - test("SPARK-15370: COUNT bug with attribute ref in subquery input and output ") { - checkAnswer( - sql("select l.b, (select (r.c + count(*)) is null from r where l.a = r.c) from l"), - Row(1.0, false) :: Row(1.0, false) :: Row(2.0, true) :: Row(2.0, true) :: - Row(3.0, false) :: Row(5.0, true) :: Row(null, false) :: Row(null, true) :: Nil) - } } From 619a11426aa6a76f88b4c0b4cc45ced8b126cbb7 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sun, 12 Jun 2016 16:37:44 -0700 Subject: [PATCH 0616/1470] [SPARK-15870][SQL] DataFrame can't execute after uncacheTable. ## What changes were proposed in this pull request? If a cached `DataFrame` executed more than once and then do `uncacheTable` like the following: ``` val selectStar = sql("SELECT * FROM testData WHERE key = 1") selectStar.createOrReplaceTempView("selectStar") spark.catalog.cacheTable("selectStar") checkAnswer( selectStar, Seq(Row(1, "1"))) spark.catalog.uncacheTable("selectStar") checkAnswer( selectStar, Seq(Row(1, "1"))) ``` , then the uncached `DataFrame` can't execute because of `Task not serializable` exception like: ``` org.apache.spark.SparkException: Task not serializable at org.apache.spark.util.ClosureCleaner$.ensureSerializable(ClosureCleaner.scala:298) at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:288) at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:108) at org.apache.spark.SparkContext.clean(SparkContext.scala:2038) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1897) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1912) at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:884) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) at org.apache.spark.rdd.RDD.withScope(RDD.scala:357) at org.apache.spark.rdd.RDD.collect(RDD.scala:883) at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:290) ... Caused by: java.lang.UnsupportedOperationException: Accumulator must be registered before send to executor at org.apache.spark.util.AccumulatorV2.writeReplace(AccumulatorV2.scala:153) at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at java.io.ObjectStreamClass.invokeWriteReplace(ObjectStreamClass.java:1118) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1136) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1548) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1509) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1432) ... ``` Notice that `DataFrame` uncached with `DataFrame.unpersist()` works, but with `spark.catalog.uncacheTable` doesn't work. This pr reverts a part of cf38fe0 not to unregister `batchStats` accumulator, which is not needed to be unregistered here because it will be done by `ContextCleaner` after it is collected by GC. ## How was this patch tested? Added a test to check if DataFrame can execute after uncacheTable and other existing tests. But I made a test to check if the accumulator was cleared as `ignore` because the test would be flaky. Author: Takuya UESHIN Closes #13596 from ueshin/issues/SPARK-15870. (cherry picked from commit caebd7f2622340fc081bb9a2ea6a0b246f1e3a3f) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/CacheManager.scala | 2 +- .../columnar/InMemoryTableScanExec.scala | 6 --- .../apache/spark/sql/CachedTableSuite.scala | 41 ++++++++++++++++++- 3 files changed, 41 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index b584cf4079769..4e95754e9bef7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -109,7 +109,7 @@ private[sql] class CacheManager 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.uncache(blocking) + cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) cachedData.remove(dataIndex) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index ff0733102a298..ce630bcbca4cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -210,12 +210,6 @@ private[sql] case class InMemoryRelation( override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) - - private[sql] def uncache(blocking: Boolean): Unit = { - AccumulatorContext.remove(batchStats.id) - cachedColumnBuffers.unpersist(blocking) - _cachedColumnBuffers = null - } } private[sql] case class InMemoryTableScanExec( 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 e08a9ab7e6914..44bafa55bc330 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 @@ -17,11 +17,13 @@ package org.apache.spark.sql +import scala.collection.mutable.HashSet import scala.concurrent.duration._ import scala.language.postfixOps import org.scalatest.concurrent.Eventually._ +import org.apache.spark.CleanerListener import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchange @@ -321,7 +323,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext assert(spark.sharedState.cacheManager.isEmpty) } - test("Clear accumulators when uncacheTable to prevent memory leaking") { + test("Ensure accumulators to be cleared after GC when uncacheTable") { sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") @@ -333,17 +335,39 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext sql("SELECT * FROM t1").count() sql("SELECT * FROM t2").count() + val toBeCleanedAccIds = new HashSet[Long] + val accId1 = spark.table("t1").queryExecution.withCachedData.collect { case i: InMemoryRelation => i.batchStats.id }.head + toBeCleanedAccIds += accId1 val accId2 = spark.table("t1").queryExecution.withCachedData.collect { case i: InMemoryRelation => i.batchStats.id }.head + toBeCleanedAccIds += accId2 + + val cleanerListener = new CleanerListener { + def rddCleaned(rddId: Int): Unit = {} + def shuffleCleaned(shuffleId: Int): Unit = {} + def broadcastCleaned(broadcastId: Long): Unit = {} + def accumCleaned(accId: Long): Unit = { + toBeCleanedAccIds.synchronized { toBeCleanedAccIds -= accId } + } + def checkpointCleaned(rddId: Long): Unit = {} + } + spark.sparkContext.cleaner.get.attachListener(cleanerListener) spark.catalog.uncacheTable("t1") spark.catalog.uncacheTable("t2") + System.gc() + + eventually(timeout(10 seconds)) { + assert(toBeCleanedAccIds.synchronized { toBeCleanedAccIds.isEmpty }, + "batchStats accumulators should be cleared after GC when uncacheTable") + } + assert(AccumulatorContext.get(accId1).isEmpty) assert(AccumulatorContext.get(accId2).isEmpty) } @@ -513,4 +537,19 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext spark.catalog.uncacheTable("t2") } } + + test("SPARK-15870 DataFrame can't execute after uncacheTable") { + val selectStar = sql("SELECT * FROM testData WHERE key = 1") + selectStar.createOrReplaceTempView("selectStar") + + spark.catalog.cacheTable("selectStar") + checkAnswer( + selectStar, + Seq(Row(1, "1"))) + + spark.catalog.uncacheTable("selectStar") + checkAnswer( + selectStar, + Seq(Row(1, "1"))) + } } From 8b6ec9b91e8f81729923ae0bcd0a9157f8894e4e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 12 Jun 2016 16:53:39 -0700 Subject: [PATCH 0617/1470] Revert "[SPARK-15753][SQL] Move Analyzer stuff to Analyzer from DataFrameWriter" This reverts commit 55a83724632aa54e49aedbab8ddd21d010eca26d. --- .../spark/sql/catalyst/analysis/Analyzer.scala | 17 +++-------------- .../org/apache/spark/sql/DataFrameWriter.scala | 12 +++++++++++- .../sql/hive/execution/HiveQuerySuite.scala | 4 ++-- 3 files changed, 16 insertions(+), 17 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 07c8bf01c1f0c..4cbedbda6da1a 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 @@ -452,17 +452,6 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) if child.resolved => - // A partitioned relation's schema can be different from the input logicalPlan, since - // partition columns are all moved after data columns. We Project to adjust the ordering. - val input = if (parts.nonEmpty) { - val (inputPartCols, inputDataCols) = child.output.partition { attr => - parts.contains(attr.name) - } - Project(inputDataCols ++ inputPartCols, child) - } else { - child - } - val table = lookupTableFromCatalog(u) // adding the table's partitions or validate the query's partition info table match { @@ -478,8 +467,8 @@ class Analyzer( |Requested partitions: ${parts.keys.mkString(",")} |Table partitions: ${tablePartitionNames.mkString(",")}""".stripMargin) } - // Partition columns are already correctly placed at the end of the child's output - i.copy(table = EliminateSubqueryAliases(table), child = input) + // Assume partition columns are correctly placed at the end of the child's output + i.copy(table = EliminateSubqueryAliases(table)) } else { // Set up the table's partition scheme with all dynamic partitions by moving partition // columns to the end of the column list, in partition order. @@ -497,7 +486,7 @@ class Analyzer( child = Project(columns ++ partColumns, child)) } case _ => - i.copy(table = EliminateSubqueryAliases(table), child = input) + i.copy(table = EliminateSubqueryAliases(table)) } case u: UnresolvedRelation => val table = u.tableIdentifier diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 8c05a7fce31ad..afae0786b73dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -506,11 +506,21 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val partitions = normalizedParCols.map(_.map(col => col -> (Option.empty[String])).toMap) val overwrite = mode == SaveMode.Overwrite + // A partitioned relation's schema can be different from the input logicalPlan, since + // partition columns are all moved after data columns. We Project to adjust the ordering. + // TODO: this belongs to the analyzer. + val input = normalizedParCols.map { parCols => + val (inputPartCols, inputDataCols) = df.logicalPlan.output.partition { attr => + parCols.contains(attr.name) + } + Project(inputDataCols ++ inputPartCols, df.logicalPlan) + }.getOrElse(df.logicalPlan) + df.sparkSession.sessionState.executePlan( InsertIntoTable( UnresolvedRelation(tableIdent), partitions.getOrElse(Map.empty[String, Option[String]]), - df.logicalPlan, + input, overwrite, ifNotExists = false)).toRdd } 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 0a2bab4f5d1e1..e0f6ccf04dd33 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 @@ -1042,7 +1042,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .queryExecution.analyzed } - assertResult(2, "Duplicated project detected\n" + analyzedPlan) { + assertResult(1, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { case _: Project => () }.size @@ -1061,7 +1061,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .queryExecution.analyzed } - assertResult(2, "Duplicated project detected\n" + analyzedPlan) { + assertResult(1, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { case _: Project => () }.size From 94482b1e49da7de8e8ba5fffbddad7c84fb1f0cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Herman=20van=20H=C3=B6vell=20tot=20Westerflier?= Date: Sun, 12 Jun 2016 21:30:32 -0700 Subject: [PATCH 0618/1470] [SPARK-15370][SQL] Fix count bug # What changes were proposed in this pull request? This pull request fixes the COUNT bug in the `RewriteCorrelatedScalarSubquery` rule. After this change, the rule tests the expression at the root of the correlated subquery to determine whether the expression returns `NULL` on empty input. If the expression does not return `NULL`, the rule generates additional logic in the `Project` operator above the rewritten subquery. This additional logic intercepts `NULL` values coming from the outer join and replaces them with the value that the subquery's expression would return on empty input. This PR takes over https://github.com/apache/spark/pull/13155. It only fixes an issue with `Literal` construction and style issues. All credits should go frreiss. # How was this patch tested? Added regression tests to cover all branches of the updated rule (see changes to `SubquerySuite`). Ran all existing automated regression tests after merging with latest trunk. Author: frreiss Author: Herman van Hovell Closes #13629 from hvanhovell/SPARK-15370-cleanup. (cherry picked from commit 1f8f2b5c2a33e63367ea4881b5918f6bc0a6f52f) Signed-off-by: Reynold Xin --- .../sql/catalyst/expressions/predicates.scala | 7 +- .../sql/catalyst/optimizer/Optimizer.scala | 221 ++++++++++++++++-- .../org/apache/spark/sql/SubquerySuite.scala | 81 +++++++ 3 files changed, 287 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 8a6cf53782b91..a3b098afe5728 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 @@ -69,8 +69,11 @@ trait PredicateHelper { protected def replaceAlias( condition: Expression, aliases: AttributeMap[Expression]): Expression = { - condition.transform { - case a: Attribute => aliases.getOrElse(a, a) + // Use transformUp to prevent infinite recursion when the replacement expression + // redefines the same ExprId, + condition.transformUp { + case a: Attribute => + aliases.getOrElse(a, a) } } 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 afe4d25f299e9..581ede5a847fb 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 @@ -527,7 +527,8 @@ object CollapseProject extends Rule[LogicalPlan] { // Substitute any attributes that are produced by the lower projection, so that we safely // eliminate it. // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' - val rewrittenUpper = upper.map(_.transform { + // Use transformUp to prevent infinite recursion. + val rewrittenUpper = upper.map(_.transformUp { case a: Attribute => aliases.getOrElse(a, a) }) // collapse upper and lower Projects may introduce unnecessary Aliases, trim them here. @@ -1676,10 +1677,10 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Filter the plan by applying left semi and left anti joins. withSubquery.foldLeft(newFilter) { case (p, PredicateSubquery(sub, conditions, _, _)) => - val (joinCond, outerPlan) = rewriteExistentialExpr(conditions.reduceOption(And), p) + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) Join(outerPlan, sub, LeftSemi, joinCond) case (p, Not(PredicateSubquery(sub, conditions, false, _))) => - val (joinCond, outerPlan) = rewriteExistentialExpr(conditions.reduceOption(And), p) + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) Join(outerPlan, sub, LeftAnti, joinCond) case (p, Not(PredicateSubquery(sub, conditions, true, _))) => // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr @@ -1688,11 +1689,11 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Note that will almost certainly be planned as a Broadcast Nested Loop join. // Use EXISTS if performance matters to you. - val (joinCond, outerPlan) = rewriteExistentialExpr(conditions.reduceLeftOption(And), p) + val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) val anyNull = splitConjunctivePredicates(joinCond.get).map(IsNull).reduceLeft(Or) Join(outerPlan, sub, LeftAnti, Option(Or(anyNull, joinCond.get))) case (p, predicate) => - val (newCond, inputPlan) = rewriteExistentialExpr(Option(predicate), p) + val (newCond, inputPlan) = rewriteExistentialExpr(Seq(predicate), p) Project(p.output, Filter(newCond.get, inputPlan)) } } @@ -1705,22 +1706,19 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { * are blocked in the Analyzer. */ private def rewriteExistentialExpr( - expr: Option[Expression], + exprs: Seq[Expression], plan: LogicalPlan): (Option[Expression], LogicalPlan) = { var newPlan = plan - expr match { - case Some(e) => - val newExpr = e transformUp { - case PredicateSubquery(sub, conditions, nullAware, _) => - // TODO: support null-aware join - val exists = AttributeReference("exists", BooleanType, nullable = false)() - newPlan = Join(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) - exists + val newExprs = exprs.map { e => + e transformUp { + case PredicateSubquery(sub, conditions, nullAware, _) => + // TODO: support null-aware join + val exists = AttributeReference("exists", BooleanType, nullable = false)() + newPlan = Join(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) + exists } - (Option(newExpr), newPlan) - case None => - (expr, plan) } + (newExprs.reduceOption(And), newPlan) } } @@ -1743,6 +1741,124 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { newExpression.asInstanceOf[E] } + /** + * Statically evaluate an expression containing zero or more placeholders, given a set + * of bindings for placeholder values. + */ + private def evalExpr(expr: Expression, bindings: Map[ExprId, Option[Any]]) : Option[Any] = { + val rewrittenExpr = expr transform { + case r: AttributeReference => + bindings(r.exprId) match { + case Some(v) => Literal.create(v, r.dataType) + case None => Literal.default(NullType) + } + } + Option(rewrittenExpr.eval()) + } + + /** + * Statically evaluate an expression containing one or more aggregates on an empty input. + */ + private def evalAggOnZeroTups(expr: Expression) : Option[Any] = { + // AggregateExpressions are Unevaluable, so we need to replace all aggregates + // in the expression with the value they would return for zero input tuples. + // Also replace attribute refs (for example, for grouping columns) with NULL. + val rewrittenExpr = expr transform { + case a @ AggregateExpression(aggFunc, _, _, resultId) => + aggFunc.defaultResult.getOrElse(Literal.default(NullType)) + + case _: AttributeReference => Literal.default(NullType) + } + Option(rewrittenExpr.eval()) + } + + /** + * Statically evaluate a scalar subquery on an empty input. + * + * WARNING: This method only covers subqueries that pass the checks under + * [[org.apache.spark.sql.catalyst.analysis.CheckAnalysis]]. If the checks in + * CheckAnalysis become less restrictive, this method will need to change. + */ + private def evalSubqueryOnZeroTups(plan: LogicalPlan) : Option[Any] = { + // Inputs to this method will start with a chain of zero or more SubqueryAlias + // and Project operators, followed by an optional Filter, followed by an + // Aggregate. Traverse the operators recursively. + def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = lp match { + case SubqueryAlias(_, child) => evalPlan(child) + case Filter(condition, child) => + val bindings = evalPlan(child) + if (bindings.isEmpty) bindings + else { + val exprResult = evalExpr(condition, bindings).getOrElse(false) + .asInstanceOf[Boolean] + if (exprResult) bindings else Map.empty + } + + case Project(projectList, child) => + val bindings = evalPlan(child) + if (bindings.isEmpty) { + bindings + } else { + projectList.map(ne => (ne.exprId, evalExpr(ne, bindings))).toMap + } + + case Aggregate(_, aggExprs, _) => + // Some of the expressions under the Aggregate node are the join columns + // for joining with the outer query block. Fill those expressions in with + // nulls and statically evaluate the remainder. + aggExprs.map { + case ref: AttributeReference => (ref.exprId, None) + case alias @ Alias(_: AttributeReference, _) => (alias.exprId, None) + case ne => (ne.exprId, evalAggOnZeroTups(ne)) + }.toMap + + case _ => sys.error(s"Unexpected operator in scalar subquery: $lp") + } + + val resultMap = evalPlan(plan) + + // By convention, the scalar subquery result is the leftmost field. + resultMap(plan.output.head.exprId) + } + + /** + * Split the plan for a scalar subquery into the parts above the innermost query block + * (first part of returned value), the HAVING clause of the innermost query block + * (optional second part) and the parts below the HAVING CLAUSE (third part). + */ + private def splitSubquery(plan: LogicalPlan) : (Seq[LogicalPlan], Option[Filter], Aggregate) = { + val topPart = ArrayBuffer.empty[LogicalPlan] + var bottomPart: LogicalPlan = plan + while (true) { + bottomPart match { + case havingPart @ Filter(_, aggPart: Aggregate) => + return (topPart, Option(havingPart), aggPart) + + case aggPart: Aggregate => + // No HAVING clause + return (topPart, None, aggPart) + + case p @ Project(_, child) => + topPart += p + bottomPart = child + + case s @ SubqueryAlias(_, child) => + topPart += s + bottomPart = child + + case Filter(_, op) => + sys.error(s"Correlated subquery has unexpected operator $op below filter") + + case op @ _ => sys.error(s"Unexpected operator $op in correlated subquery") + } + } + + sys.error("This line should be unreachable") + } + + // Name of generated column used in rewrite below + val ALWAYS_TRUE_COLNAME = "alwaysTrue" + /** * Construct a new child plan by left joining the given subqueries to a base plan. */ @@ -1751,9 +1867,74 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { subqueries: ArrayBuffer[ScalarSubquery]): LogicalPlan = { subqueries.foldLeft(child) { case (currentChild, ScalarSubquery(query, conditions, _)) => - Project( - currentChild.output :+ query.output.head, - Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + val origOutput = query.output.head + + val resultWithZeroTups = evalSubqueryOnZeroTups(query) + if (resultWithZeroTups.isEmpty) { + // CASE 1: Subquery guaranteed not to have the COUNT bug + Project( + currentChild.output :+ origOutput, + Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + } else { + // Subquery might have the COUNT bug. Add appropriate corrections. + val (topPart, havingNode, aggNode) = splitSubquery(query) + + // The next two cases add a leading column to the outer join input to make it + // possible to distinguish between the case when no tuples join and the case + // when the tuple that joins contains null values. + // The leading column always has the value TRUE. + val alwaysTrueExprId = NamedExpression.newExprId + val alwaysTrueExpr = Alias(Literal.TrueLiteral, + ALWAYS_TRUE_COLNAME)(exprId = alwaysTrueExprId) + val alwaysTrueRef = AttributeReference(ALWAYS_TRUE_COLNAME, + BooleanType)(exprId = alwaysTrueExprId) + + val aggValRef = query.output.head + + if (havingNode.isEmpty) { + // CASE 2: Subquery with no HAVING clause + Project( + currentChild.output :+ + Alias( + If(IsNull(alwaysTrueRef), + Literal.create(resultWithZeroTups.get, origOutput.dataType), + aggValRef), origOutput.name)(exprId = origOutput.exprId), + Join(currentChild, + Project(query.output :+ alwaysTrueExpr, query), + LeftOuter, conditions.reduceOption(And))) + + } else { + // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join. + // Need to modify any operators below the join to pass through all columns + // referenced in the HAVING clause. + var subqueryRoot: UnaryNode = aggNode + val havingInputs: Seq[NamedExpression] = aggNode.output + + topPart.reverse.foreach { + case Project(projList, _) => + subqueryRoot = Project(projList ++ havingInputs, subqueryRoot) + case s @ SubqueryAlias(alias, _) => + subqueryRoot = SubqueryAlias(alias, subqueryRoot) + case op => sys.error(s"Unexpected operator $op in corelated subquery") + } + + // CASE WHEN alwayTrue IS NULL THEN resultOnZeroTups + // WHEN NOT (original HAVING clause expr) THEN CAST(null AS ) + // ELSE (aggregate value) END AS (original column name) + val caseExpr = Alias(CaseWhen(Seq( + (IsNull(alwaysTrueRef), Literal.create(resultWithZeroTups.get, origOutput.dataType)), + (Not(havingNode.get.condition), Literal.create(null, aggValRef.dataType))), + aggValRef), + origOutput.name)(exprId = origOutput.exprId) + + Project( + currentChild.output :+ caseExpr, + Join(currentChild, + Project(subqueryRoot.output :+ alwaysTrueExpr, subqueryRoot), + LeftOuter, conditions.reduceOption(And))) + + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 1a99fb683e438..1d9ff21dbf5d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -490,4 +490,85 @@ class SubquerySuite extends QueryTest with SharedSQLContext { """.stripMargin), Row(3) :: Nil) } + + test("SPARK-15370: COUNT bug in WHERE clause (Filter)") { + // Case 1: Canonical example of the COUNT bug + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) < l.a"), + Row(1) :: Row(1) :: Row(3) :: Row(6) :: Nil) + // Case 2: count(*) = 0; could be rewritten to NOT EXISTS but currently uses + // a rewrite that is vulnerable to the COUNT bug + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) = 0"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + // Case 3: COUNT bug without a COUNT aggregate + checkAnswer( + sql("select l.a from l where (select sum(r.d) is null from r where l.a = r.c)"), + Row(1) :: Row(1) ::Row(null) :: Row(null) :: Row(6) :: Nil) + } + + test("SPARK-15370: COUNT bug in SELECT clause (Project)") { + checkAnswer( + sql("select a, (select count(*) from r where l.a = r.c) as cnt from l"), + Row(1, 0) :: Row(1, 0) :: Row(2, 2) :: Row(2, 2) :: Row(3, 1) :: Row(null, 0) + :: Row(null, 0) :: Row(6, 1) :: Nil) + } + + test("SPARK-15370: COUNT bug in HAVING clause (Filter)") { + checkAnswer( + sql("select l.a as grp_a from l group by l.a " + + "having (select count(*) from r where grp_a = r.c) = 0 " + + "order by grp_a"), + Row(null) :: Row(1) :: Nil) + } + + test("SPARK-15370: COUNT bug in Aggregate") { + checkAnswer( + sql("select l.a as aval, sum((select count(*) from r where l.a = r.c)) as cnt " + + "from l group by l.a order by aval"), + Row(null, 0) :: Row(1, 0) :: Row(2, 4) :: Row(3, 1) :: Row(6, 1) :: Nil) + } + + test("SPARK-15370: COUNT bug negative examples") { + // Case 1: Potential COUNT bug case that was working correctly prior to the fix + checkAnswer( + sql("select l.a from l where (select sum(r.d) from r where l.a = r.c) is null"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Row(6) :: Nil) + // Case 2: COUNT aggregate but no COUNT bug due to > 0 test. + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) > 0"), + Row(2) :: Row(2) :: Row(3) :: Row(6) :: Nil) + // Case 3: COUNT inside aggregate expression but no COUNT bug. + checkAnswer( + sql("select l.a from l where (select count(*) + sum(r.d) from r where l.a = r.c) = 0"), + Nil) + } + + test("SPARK-15370: COUNT bug in subquery in subquery in subquery") { + checkAnswer( + sql("""select l.a from l + |where ( + | select cntPlusOne + 1 as cntPlusTwo from ( + | select cnt + 1 as cntPlusOne from ( + | select sum(r.c) s, count(*) cnt from r where l.a = r.c having cnt = 0 + | ) + | ) + |) = 2""".stripMargin), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + } + + test("SPARK-15370: COUNT bug with nasty predicate expr") { + checkAnswer( + sql("select l.a from l where " + + "(select case when count(*) = 1 then null else count(*) end as cnt " + + "from r where l.a = r.c) = 0"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + } + + test("SPARK-15370: COUNT bug with attribute ref in subquery input and output ") { + checkAnswer( + sql("select l.b, (select (r.c + count(*)) is null from r where l.a = r.c) from l"), + Row(1.0, false) :: Row(1.0, false) :: Row(2.0, true) :: Row(2.0, true) :: + Row(3.0, false) :: Row(5.0, true) :: Row(null, false) :: Row(null, true) :: Nil) + } } From b96e7f6aa8c6ed592d6e0ddbece7cf8530da2194 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 12 Jun 2016 21:36:41 -0700 Subject: [PATCH 0619/1470] [SPARK-15898][SQL] DataFrameReader.text should return DataFrame ## What changes were proposed in this pull request? We want to maintain API compatibility for DataFrameReader.text, and will introduce a new API called DataFrameReader.textFile which returns Dataset[String]. affected PRs: https://github.com/apache/spark/pull/11731 https://github.com/apache/spark/pull/13104 https://github.com/apache/spark/pull/13184 ## How was this patch tested? N/A Author: Wenchen Fan Closes #13604 from cloud-fan/revert. (cherry picked from commit e2ab79d5ea00af45c083cc9a6607d2f0905f9908) Signed-off-by: Reynold Xin --- R/pkg/R/SQLContext.R | 7 +++-- .../org/apache/spark/examples/JavaHdfsLR.java | 2 +- .../apache/spark/examples/JavaPageRank.java | 2 +- .../apache/spark/examples/JavaWordCount.java | 2 +- .../spark/examples/ml/JavaALSExample.java | 2 +- .../spark/examples/sql/JavaSparkSQL.java | 2 +- .../apache/spark/examples/SparkHdfsLR.scala | 2 +- .../apache/spark/examples/SparkKMeans.scala | 2 +- .../apache/spark/examples/SparkPageRank.scala | 2 +- .../apache/spark/examples/ml/ALSExample.scala | 2 +- .../mllib/RankingMetricsExample.scala | 2 +- python/pyspark/sql/readwriter.py | 8 ++--- .../apache/spark/sql/DataFrameReader.scala | 31 ++++++++++++++----- .../apache/spark/sql/JavaDataFrameSuite.java | 4 +-- .../datasources/text/TextSuite.scala | 20 ++++++------ 15 files changed, 54 insertions(+), 36 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 584bbbf0e4c2d..e7e9e353f9e82 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -364,9 +364,10 @@ parquetFile <- function(x, ...) { #' Create a SparkDataFrame from a text file. #' -#' Loads a text file and returns a SparkDataFrame with a single string column named "value". -#' If the directory structure of the text files contains partitioning information, those are -#' ignored in the resulting DataFrame. +#' Loads text files and returns a SparkDataFrame whose schema starts with +#' a string column named "value", and followed by partitioned columns if +#' there are any. +#' #' Each line in the text file is a new row in the resulting SparkDataFrame. #' #' @param path Path of file to read. A vector of multiple paths is allowed. diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index ded442096c257..362bd4435ecb3 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -126,7 +126,7 @@ public static void main(String[] args) { .appName("JavaHdfsLR") .getOrCreate(); - JavaRDD lines = spark.read().text(args[0]).javaRDD(); + JavaRDD lines = spark.read().textFile(args[0]).javaRDD(); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[1]); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 128b5ab17c8d4..ed0bb876579ad 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -82,7 +82,7 @@ public static void main(String[] args) throws Exception { // URL neighbor URL // URL neighbor URL // ... - JavaRDD lines = spark.read().text(args[0]).javaRDD(); + JavaRDD lines = spark.read().textFile(args[0]).javaRDD(); // Loads all URLs from input file and initialize their neighbors. JavaPairRDD> links = lines.mapToPair( diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 1caee60e348d2..8f18604c0750c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -46,7 +46,7 @@ public static void main(String[] args) throws Exception { .appName("JavaWordCount") .getOrCreate(); - JavaRDD lines = spark.read().text(args[0]).javaRDD(); + JavaRDD lines = spark.read().textFile(args[0]).javaRDD(); JavaRDD words = lines.flatMap(new FlatMapFunction() { @Override diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 7f568f4e0db4e..739558e81ffb0 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -87,7 +87,7 @@ public static void main(String[] args) { // $example on$ JavaRDD ratingsRDD = spark - .read().text("data/mllib/als/sample_movielens_ratings.txt").javaRDD() + .read().textFile("data/mllib/als/sample_movielens_ratings.txt").javaRDD() .map(new Function() { public Rating call(String str) { return Rating.parseRating(str); 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 55e591d0ce166..e512979ac71b0 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 @@ -59,7 +59,7 @@ public static void main(String[] args) throws Exception { System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. String file = "examples/src/main/resources/people.txt"; - JavaRDD people = spark.read().text(file).javaRDD().map( + JavaRDD people = spark.read().textFile(file).javaRDD().map( new Function() { @Override public Person call(String line) { diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 84f133e0116a2..05ac6cbcb35bc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -72,7 +72,7 @@ object SparkHdfsLR { .getOrCreate() val inputPath = args(0) - val lines = spark.read.text(inputPath).rdd + val lines = spark.read.textFile(inputPath).rdd val points = lines.map(parsePoint).cache() val ITERATIONS = args(1).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index aa93c93c441f4..fec3160e9f37b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -71,7 +71,7 @@ object SparkKMeans { .appName("SparkKMeans") .getOrCreate() - val lines = spark.read.text(args(0)).rdd + val lines = spark.read.textFile(args(0)).rdd val data = lines.map(parseVector _).cache() val K = args(1).toInt val convergeDist = args(2).toDouble diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index b7c363c7d4fae..d0b874c48d00a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -56,7 +56,7 @@ object SparkPageRank { .getOrCreate() val iters = if (args.length > 1) args(1).toInt else 10 - val lines = spark.read.text(args(0)).rdd + val lines = spark.read.textFile(args(0)).rdd val links = lines.map{ s => val parts = s.split("\\s+") (parts(0), parts(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala index da19ea9f10ec6..bb5d163608494 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -50,7 +50,7 @@ object ALSExample { import spark.implicits._ // $example on$ - val ratings = spark.read.text("data/mllib/als/sample_movielens_ratings.txt") + val ratings = spark.read.textFile("data/mllib/als/sample_movielens_ratings.txt") .map(parseRating) .toDF() val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2)) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala index 781a934df6637..d514891da78fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -33,7 +33,7 @@ object RankingMetricsExample { import spark.implicits._ // $example on$ // Read in the ratings data - val ratings = spark.read.text("data/mllib/sample_movielens_data.txt").rdd.map { line => + val ratings = spark.read.textFile("data/mllib/sample_movielens_data.txt").rdd.map { line => val fields = line.split("::") Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble - 2.5) }.cache() diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index f3182b237ec65..0f50f672a22d1 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -291,10 +291,10 @@ def parquet(self, *paths): @ignore_unicode_prefix @since(1.6) def text(self, paths): - """Loads a text file and returns a :class:`DataFrame` with a single string column named "value". - If the directory structure of the text files contains partitioning information, - those are ignored in the resulting DataFrame. To include partitioning information as - columns, use ``read.format('text').load(...)``. + """ + Loads text files and returns a :class:`DataFrame` whose schema starts with a + string column named "value", and followed by partitioned columns if there + are any. Each line in the text file is a new row in the resulting DataFrame. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 078b63ee87683..dfe31da3f3107 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -449,30 +449,47 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName))) } + /** + * Loads text files and returns a [[DataFrame]] whose schema starts with a string column named + * "value", and followed by partitioned columns if there are any. + * + * Each line in the text files is a new row in the resulting DataFrame. For example: + * {{{ + * // Scala: + * spark.read.text("/path/to/spark/README.md") + * + * // Java: + * spark.read().text("/path/to/spark/README.md") + * }}} + * + * @param paths input path + * @since 1.6.0 + */ + @scala.annotation.varargs + def text(paths: String*): DataFrame = format("text").load(paths : _*) + /** * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset * contains a single string column named "value". * * If the directory structure of the text files contains partitioning information, those are - * ignored in the resulting Dataset. To include partitioning information as columns, use - * `read.format("text").load("...")`. + * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. * * Each line in the text files is a new element in the resulting Dataset. For example: * {{{ * // Scala: - * spark.read.text("/path/to/spark/README.md") + * spark.read.textFile("/path/to/spark/README.md") * * // Java: - * spark.read().text("/path/to/spark/README.md") + * spark.read().textFile("/path/to/spark/README.md") * }}} * * @param paths input path * @since 2.0.0 */ @scala.annotation.varargs - def text(paths: String*): Dataset[String] = { - format("text").load(paths : _*).select("value") - .as[String](sparkSession.implicits.newStringEncoder) + def textFile(paths: String*): Dataset[String] = { + text(paths : _*).select("value").as[String](sparkSession.implicits.newStringEncoder) } /////////////////////////////////////////////////////////////////////////////////////// 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 0152f3f85a230..318b53cdbbaa0 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 @@ -338,10 +338,10 @@ public void testGenericLoad() { @Test public void testTextLoad() { - Dataset ds1 = spark.read().text(getResource("text-suite.txt")); + Dataset ds1 = spark.read().textFile(getResource("text-suite.txt")); Assert.assertEquals(4L, ds1.count()); - Dataset ds2 = spark.read().text( + Dataset ds2 = spark.read().textFile( getResource("text-suite.txt"), getResource("text-suite2.txt")); Assert.assertEquals(5L, ds2.count()); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 5695f6af7bd44..4ed517cb26ae3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -36,7 +36,7 @@ class TextSuite extends QueryTest with SharedSQLContext { } test("SQLContext.read.text() API") { - verifyFrame(spark.read.text(testFile).toDF()) + verifyFrame(spark.read.text(testFile)) } test("SPARK-12562 verify write.text() can handle column name beyond `value`") { @@ -45,7 +45,7 @@ class TextSuite extends QueryTest with SharedSQLContext { val tempFile = Utils.createTempDir() tempFile.delete() df.write.text(tempFile.getCanonicalPath) - verifyFrame(spark.read.text(tempFile.getCanonicalPath).toDF()) + verifyFrame(spark.read.text(tempFile.getCanonicalPath)) Utils.deleteRecursively(tempFile) } @@ -64,20 +64,20 @@ class TextSuite extends QueryTest with SharedSQLContext { } } - test("reading partitioned data using read.text()") { + test("reading partitioned data using read.textFile()") { val partitionedData = Thread.currentThread().getContextClassLoader .getResource("text-partitioned").toString - val df = spark.read.text(partitionedData) - val data = df.collect() + val ds = spark.read.textFile(partitionedData) + val data = ds.collect() - assert(df.schema == new StructType().add("value", StringType)) + assert(ds.schema == new StructType().add("value", StringType)) assert(data.length == 2) } - test("support for partitioned reading") { + test("support for partitioned reading using read.text()") { val partitionedData = Thread.currentThread().getContextClassLoader .getResource("text-partitioned").toString - val df = spark.read.format("text").load(partitionedData) + val df = spark.read.text(partitionedData) val data = df.filter("year = '2015'").select("value").collect() assert(data(0) == Row("2015-test")) @@ -94,7 +94,7 @@ class TextSuite extends QueryTest with SharedSQLContext { testDf.write.option("compression", codecName).mode(SaveMode.Overwrite).text(tempDirPath) val compressedFiles = new File(tempDirPath).listFiles() assert(compressedFiles.exists(_.getName.endsWith(s".txt$extension"))) - verifyFrame(spark.read.text(tempDirPath).toDF()) + verifyFrame(spark.read.text(tempDirPath)) } val errMsg = intercept[IllegalArgumentException] { @@ -121,7 +121,7 @@ class TextSuite extends QueryTest with SharedSQLContext { .options(extraOptions).mode(SaveMode.Overwrite).text(tempDirPath) val compressedFiles = new File(tempDirPath).listFiles() assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) - verifyFrame(spark.read.options(extraOptions).text(tempDirPath).toDF()) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) } } From 41f309bfbcefcc9612efb7c0571a4009147e5896 Mon Sep 17 00:00:00 2001 From: Peter Ableda Date: Mon, 13 Jun 2016 09:40:17 +0100 Subject: [PATCH 0620/1470] [SPARK-15813] Improve Canceling log message to make it less ambiguous ## What changes were proposed in this pull request? Add new desired executor number to make the log message less ambiguous. ## How was this patch tested? This is a trivial change Author: Peter Ableda Closes #13552 from peterableda/patch-1. (cherry picked from commit d681742b2d37bd68cf5d8d3161e0f48846f6f9d4) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index b110d82fb8a9e..1b800716495a7 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -354,7 +354,8 @@ private[yarn] class YarnAllocator( } else if (missing < 0) { val numToCancel = math.min(numPendingAllocate, -missing) - logInfo(s"Canceling requests for $numToCancel executor containers") + logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " + + s"total $targetNumExecutors executors.") val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource) if (!matchingRequests.isEmpty) { From d64cb19945cedecb2208226199d3ca61e3d11c5b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 13 Jun 2016 09:58:48 -0700 Subject: [PATCH 0621/1470] [SPARK-15814][SQL] Aggregator can return null result ## What changes were proposed in this pull request? It's similar to the bug fixed in https://github.com/apache/spark/pull/13425, we should consider null object and wrap the `CreateStruct` with `If` to do null check. This PR also improves the test framework to test the objects of `Dataset[T]` directly, instead of calling `toDF` and compare the rows. ## How was this patch tested? new test in `DatasetAggregatorSuite` Author: Wenchen Fan Closes #13553 from cloud-fan/agg-null. (cherry picked from commit cd47e233749f42b016264569a214cbf67f45f436) Signed-off-by: Herman van Hovell --- .../aggregate/TypedAggregateExpression.scala | 7 +- .../spark/sql/DatasetAggregatorSuite.scala | 23 ++++- .../spark/sql/DatasetPrimitiveSuite.scala | 6 +- .../org/apache/spark/sql/DatasetSuite.scala | 38 ++++---- .../org/apache/spark/sql/QueryTest.scala | 95 ++++++++++++------- .../datasources/text/TextSuite.scala | 4 +- .../sql/streaming/FileStreamSinkSuite.scala | 4 +- .../spark/sql/streaming/MemorySinkSuite.scala | 4 +- 8 files changed, 117 insertions(+), 64 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index ecb56e2a2848a..8bdfa48a30c9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -127,7 +127,12 @@ case class TypedAggregateExpression( dataType match { case s: StructType => - ReferenceToExpressions(CreateStruct(outputSerializer), resultObj :: Nil) + val objRef = outputSerializer.head.find(_.isInstanceOf[BoundReference]).get + val struct = If( + IsNull(objRef), + Literal.create(null, dataType), + CreateStruct(outputSerializer)) + ReferenceToExpressions(struct, resultObj :: Nil) case _ => assert(outputSerializer.length == 1) outputSerializer.head transform { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index f9b4cd83c3a42..f955120dc5439 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -115,11 +115,23 @@ object RowAgg extends Aggregator[Row, Int, Int] { override def outputEncoder: Encoder[Int] = Encoders.scalaInt } +object NullResultAgg extends Aggregator[AggData, AggData, AggData] { + override def zero: AggData = AggData(0, "") + override def reduce(b: AggData, a: AggData): AggData = AggData(b.a + a.a, b.b + a.b) + override def finish(reduction: AggData): AggData = { + if (reduction.a % 2 == 0) null else reduction + } + override def merge(b1: AggData, b2: AggData): AggData = AggData(b1.a + b2.a, b1.b + b2.b) + override def bufferEncoder: Encoder[AggData] = Encoders.product[AggData] + override def outputEncoder: Encoder[AggData] = Encoders.product[AggData] +} -class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { +class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { import testImplicits._ + private implicit val ordering = Ordering.by((c: AggData) => c.a -> c.b) + test("typed aggregation: TypedAggregator") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() @@ -204,7 +216,7 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { ds.select(expr("avg(a)").as[Double], ComplexBufferAgg.toColumn), (1.5, 2)) - checkDataset( + checkDatasetUnorderly( ds.groupByKey(_.b).agg(ComplexBufferAgg.toColumn), ("one", 1), ("two", 1)) } @@ -271,4 +283,11 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { "RowAgg(org.apache.spark.sql.Row)") assert(df.groupBy($"j").agg(RowAgg.toColumn as "agg1").columns.last == "agg1") } + + test("SPARK-15814 Aggregator can return null result") { + val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS() + checkDatasetUnorderly( + ds.groupByKey(_.a).agg(NullResultAgg.toColumn), + 1 -> AggData(1, "one"), 2 -> null) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index a634502e2e4f0..6aa3d3fe808b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -82,7 +82,7 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { test("groupBy function, keys") { val ds = Seq(1, 2, 3, 4, 5).toDS() val grouped = ds.groupByKey(_ % 2) - checkDataset( + checkDatasetUnorderly( grouped.keys, 0, 1) } @@ -95,7 +95,7 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { (name, iter.size) } - checkDataset( + checkDatasetUnorderly( agged, ("even", 5), ("odd", 6)) } @@ -105,7 +105,7 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { val grouped = ds.groupByKey(_.length) val agged = grouped.flatMapGroups { case (g, iter) => Iterator(g.toString, iter.mkString) } - checkDataset( + checkDatasetUnorderly( agged, "1", "abc", "3", "xyz", "5", "hello") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 4536a7356f017..96d85f12e87bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -32,6 +32,8 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructT class DatasetSuite extends QueryTest with SharedSQLContext { import testImplicits._ + private implicit val ordering = Ordering.by((c: ClassData) => c.a -> c.b) + test("toDS") { val data = Seq(("a", 1), ("b", 2), ("c", 3)) checkDataset( @@ -95,12 +97,12 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } assert(ds.repartition(10).rdd.partitions.length == 10) - checkDataset( + checkDatasetUnorderly( ds.repartition(10), data: _*) assert(ds.coalesce(1).rdd.partitions.length == 1) - checkDataset( + checkDatasetUnorderly( ds.coalesce(1), data: _*) } @@ -163,7 +165,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { .map(c => ClassData(c.a, c.b + 1)) .groupByKey(p => p).count() - checkDataset( + checkDatasetUnorderly( ds, (ClassData("one", 2), 1L), (ClassData("two", 3), 1L)) } @@ -204,7 +206,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("select 2, primitive and class, fields reordered") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() - checkDecoding( + checkDataset( ds.select( expr("_1").as[String], expr("named_struct('b', _2, 'a', _1)").as[ClassData]), @@ -291,7 +293,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("groupBy function, keys") { val ds = Seq(("a", 1), ("b", 1)).toDS() val grouped = ds.groupByKey(v => (1, v._2)) - checkDataset( + checkDatasetUnorderly( grouped.keys, (1, 1)) } @@ -301,7 +303,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val grouped = ds.groupByKey(v => (v._1, "word")) val agged = grouped.mapGroups { case (g, iter) => (g._1, iter.map(_._2).sum) } - checkDataset( + checkDatasetUnorderly( agged, ("a", 30), ("b", 3), ("c", 1)) } @@ -313,7 +315,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { Iterator(g._1, iter.map(_._2).sum.toString) } - checkDataset( + checkDatasetUnorderly( agged, "a", "30", "b", "3", "c", "1") } @@ -322,7 +324,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds = Seq("abc", "xyz", "hello").toDS() val agged = ds.groupByKey(_.length).reduceGroups(_ + _) - checkDataset( + checkDatasetUnorderly( agged, 3 -> "abcxyz", 5 -> "hello") } @@ -340,7 +342,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("typed aggregation: expr") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - checkDataset( + checkDatasetUnorderly( ds.groupByKey(_._1).agg(sum("_2").as[Long]), ("a", 30L), ("b", 3L), ("c", 1L)) } @@ -348,7 +350,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("typed aggregation: expr, expr") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - checkDataset( + checkDatasetUnorderly( ds.groupByKey(_._1).agg(sum("_2").as[Long], sum($"_2" + 1).as[Long]), ("a", 30L, 32L), ("b", 3L, 5L), ("c", 1L, 2L)) } @@ -356,7 +358,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("typed aggregation: expr, expr, expr") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - checkDataset( + checkDatasetUnorderly( ds.groupByKey(_._1).agg(sum("_2").as[Long], sum($"_2" + 1).as[Long], count("*")), ("a", 30L, 32L, 2L), ("b", 3L, 5L, 2L), ("c", 1L, 2L, 1L)) } @@ -364,7 +366,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("typed aggregation: expr, expr, expr, expr") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - checkDataset( + checkDatasetUnorderly( ds.groupByKey(_._1).agg( sum("_2").as[Long], sum($"_2" + 1).as[Long], @@ -380,7 +382,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { Iterator(key -> (data1.map(_._2).mkString + "#" + data2.map(_._2).mkString)) } - checkDataset( + checkDatasetUnorderly( cogrouped, 1 -> "a#", 2 -> "#q", 3 -> "abcfoo#w", 5 -> "hello#er") } @@ -392,7 +394,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { Iterator(key -> (data1.map(_._2.a).mkString + data2.map(_._2.a).mkString)) } - checkDataset( + checkDatasetUnorderly( cogrouped, 1 -> "a", 2 -> "bc", 3 -> "d") } @@ -482,8 +484,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkDataset( ds1.joinWith(ds2, lit(true)), ((nullInt, "1"), (nullInt, "1")), - ((new java.lang.Integer(22), "2"), (nullInt, "1")), ((nullInt, "1"), (new java.lang.Integer(22), "2")), + ((new java.lang.Integer(22), "2"), (nullInt, "1")), ((new java.lang.Integer(22), "2"), (new java.lang.Integer(22), "2"))) } @@ -776,9 +778,9 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds1 = ds.as("d1") val ds2 = ds.as("d2") - checkDataset(ds1.joinWith(ds2, $"d1.value" === $"d2.value"), (2, 2), (3, 3), (4, 4)) - checkDataset(ds1.intersect(ds2), 2, 3, 4) - checkDataset(ds1.except(ds1)) + checkDatasetUnorderly(ds1.joinWith(ds2, $"d1.value" === $"d2.value"), (2, 2), (3, 3), (4, 4)) + checkDatasetUnorderly(ds1.intersect(ds2), 2, 3, 4) + checkDatasetUnorderly(ds1.except(ds1)) } test("SPARK-15441: Dataset outer join") { 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 acb59d46e127d..742f036e5503a 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 @@ -68,28 +68,62 @@ abstract class QueryTest extends PlanTest { /** * Evaluates a dataset to make sure that the result of calling collect matches the given * expected answer. - * - Special handling is done based on whether the query plan should be expected to return - * the results in sorted order. - * - This function also checks to make sure that the schema for serializing the expected answer - * matches that produced by the dataset (i.e. does manual construction of object match - * the constructed encoder for cases like joins, etc). Note that this means that it will fail - * for cases where reordering is done on fields. For such tests, user `checkDecoding` instead - * which performs a subset of the checks done by this function. */ protected def checkDataset[T]( - ds: Dataset[T], + ds: => Dataset[T], expectedAnswer: T*): Unit = { - checkAnswer( - ds.toDF(), - spark.createDataset(expectedAnswer)(ds.exprEnc).toDF().collect().toSeq) + val result = getResult(ds) - checkDecoding(ds, expectedAnswer: _*) + if (!compare(result.toSeq, expectedAnswer)) { + fail( + s""" + |Decoded objects do not match expected objects: + |expected: $expectedAnswer + |actual: ${result.toSeq} + |${ds.exprEnc.deserializer.treeString} + """.stripMargin) + } } - protected def checkDecoding[T]( + /** + * Evaluates a dataset to make sure that the result of calling collect matches the given + * expected answer, after sort. + */ + protected def checkDatasetUnorderly[T : Ordering]( ds: => Dataset[T], expectedAnswer: T*): Unit = { - val decoded = try ds.collect().toSet catch { + val result = getResult(ds) + + if (!compare(result.toSeq.sorted, expectedAnswer.sorted)) { + fail( + s""" + |Decoded objects do not match expected objects: + |expected: $expectedAnswer + |actual: ${result.toSeq} + |${ds.exprEnc.deserializer.treeString} + """.stripMargin) + } + } + + private def getResult[T](ds: => Dataset[T]): Array[T] = { + val analyzedDS = try ds catch { + case ae: AnalysisException => + if (ae.plan.isDefined) { + fail( + s""" + |Failed to analyze query: $ae + |${ae.plan.get} + | + |${stackTraceToString(ae)} + """.stripMargin) + } else { + throw ae + } + } + checkJsonFormat(analyzedDS) + assertEmptyMissingInput(analyzedDS) + + try ds.collect() catch { case e: Exception => fail( s""" @@ -99,24 +133,17 @@ abstract class QueryTest extends PlanTest { |${ds.queryExecution} """.stripMargin, e) } + } - // Handle the case where the return type is an array - val isArray = decoded.headOption.map(_.getClass.isArray).getOrElse(false) - def normalEquality = decoded == expectedAnswer.toSet - def expectedAsSeq = expectedAnswer.map(_.asInstanceOf[Array[_]].toSeq).toSet - def decodedAsSeq = decoded.map(_.asInstanceOf[Array[_]].toSeq) - - if (!((isArray && expectedAsSeq == decodedAsSeq) || normalEquality)) { - val expected = expectedAnswer.toSet.toSeq.map((a: Any) => a.toString).sorted - val actual = decoded.toSet.toSeq.map((a: Any) => a.toString).sorted - - val comparison = sideBySide("expected" +: expected, "spark" +: actual).mkString("\n") - fail( - s"""Decoded objects do not match expected objects: - |$comparison - |${ds.exprEnc.deserializer.treeString} - """.stripMargin) - } + private def compare(obj1: Any, obj2: Any): Boolean = (obj1, obj2) match { + case (null, null) => true + case (null, _) => false + case (_, null) => false + case (a: Array[_], b: Array[_]) => + a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a: Iterable[_], b: Iterable[_]) => + a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a, b) => a == b } /** @@ -143,7 +170,7 @@ abstract class QueryTest extends PlanTest { checkJsonFormat(analyzedDF) - assertEmptyMissingInput(df) + assertEmptyMissingInput(analyzedDF) QueryTest.checkAnswer(analyzedDF, expectedAnswer) match { case Some(errorMessage) => fail(errorMessage) @@ -201,10 +228,10 @@ abstract class QueryTest extends PlanTest { planWithCaching) } - private def checkJsonFormat(df: DataFrame): Unit = { + private def checkJsonFormat(ds: Dataset[_]): Unit = { // Get the analyzed plan and rewrite the PredicateSubqueries in order to make sure that // RDD and Data resolution does not break. - val logicalPlan = df.queryExecution.analyzed + val logicalPlan = ds.queryExecution.analyzed // bypass some cases that we can't handle currently. logicalPlan.transform { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 4ed517cb26ae3..71d3da915840a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -132,9 +132,9 @@ class TextSuite extends QueryTest with SharedSQLContext { ds1.write.text(s"$path/part=a") ds1.write.text(s"$path/part=b") - checkDataset( + checkAnswer( spark.read.format("text").load(path).select($"part"), - Row("a"), Row("b")) + Row("a") :: Row("b") :: Nil) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 1c73208736f78..bb3063dc34ae3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -140,7 +140,7 @@ class FileStreamSinkSuite extends StreamTest { } val outputDf = spark.read.parquet(outputDir).as[Int] - checkDataset(outputDf, 1, 2, 3) + checkDatasetUnorderly(outputDf, 1, 2, 3) } finally { if (query != null) { @@ -191,7 +191,7 @@ class FileStreamSinkSuite extends StreamTest { assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value")) // Verify the data is correctly read - checkDataset( + checkDatasetUnorderly( outputDf.as[(Int, Int)], (1000, 1), (2000, 2), (3000, 3)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala index df76499fa2801..9aada0b18dd8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala @@ -174,13 +174,13 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { input.addData(1, 2, 3) query.processAllAvailable() - checkDataset( + checkDatasetUnorderly( spark.table("memStream").as[(Int, Long)], (1, 1L), (2, 1L), (3, 1L)) input.addData(4, 5, 6) query.processAllAvailable() - checkDataset( + checkDatasetUnorderly( spark.table("memStream").as[(Int, Long)], (1, 1L), (2, 1L), (3, 1L), (4, 1L), (5, 1L), (6, 1L)) From c01dc815d01956417ece23b6220cabcfa95f42a6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 13 Jun 2016 10:30:17 -0700 Subject: [PATCH 0622/1470] [SPARK-15913][CORE] Dispatcher.stopped should be enclosed by synchronized block. ## What changes were proposed in this pull request? `Dispatcher.stopped` is guarded by `this`, but it is used without synchronization in `postMessage` function. This PR fixes this and also the exception message became more accurate. ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #13634 from dongjoon-hyun/SPARK-15913. (cherry picked from commit 938434dc78f35f77cdebd15dcce8d5e7871b396b) Signed-off-by: Marcelo Vanzin --- .../apache/spark/rpc/netty/Dispatcher.scala | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 4f8fe018b432d..d305de2e1340e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -144,25 +144,20 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging { endpointName: String, message: InboxMessage, callbackIfStopped: (Exception) => Unit): Unit = { - val shouldCallOnStop = synchronized { + val error = synchronized { val data = endpoints.get(endpointName) - if (stopped || data == null) { - true + if (stopped) { + Some(new RpcEnvStoppedException()) + } else if (data == null) { + Some(new SparkException(s"Could not find $endpointName.")) } else { data.inbox.post(message) receivers.offer(data) - false + None } } - if (shouldCallOnStop) { - // We don't need to call `onStop` in the `synchronized` block - val error = if (stopped) { - new RpcEnvStoppedException() - } else { - new SparkException(s"Could not find $endpointName or it has been stopped.") - } - callbackIfStopped(error) - } + // We don't need to call `onStop` in the `synchronized` block + error.foreach(callbackIfStopped) } def stop(): Unit = { From 413826d40df3f8c025101952c454a557cb8d4973 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 13 Jun 2016 11:13:09 -0700 Subject: [PATCH 0623/1470] [SPARK-15697][REPL] Unblock some of the useful repl commands. ## What changes were proposed in this pull request? Unblock some of the useful repl commands. like, "implicits", "javap", "power", "type", "kind". As they are useful and fully functional and part of scala/scala project, I see no harm in having them either. Verbatim paste form JIRA description. "implicits", "javap", "power", "type", "kind" commands in repl are blocked. However, they work fine in all cases I have tried. It is clear we don't support them as they are part of the scala/scala repl project. What is the harm in unblocking them, given they are useful ? In previous versions of spark we disabled these commands because it was difficult to support them without customization and the associated maintenance. Since the code base of scala repl was actually ported and maintained under spark source. Now that is not the situation and one can benefit from these commands in Spark REPL as much as in scala repl. ## How was this patch tested? Existing tests and manual, by trying out all of the above commands. P.S. Symantics of reset are to be discussed in a separate issue. Author: Prashant Sharma Closes #13437 from ScrapCodes/SPARK-15697/repl-unblock-commands. (cherry picked from commit 4134653e53adde72f53c61929473dcacae9be8b6) Signed-off-by: Shixiong Zhu --- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala index bbdb992d8a5e2..dcf3209ae7d82 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -70,7 +70,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) echo("Type :help for more information.") } - private val blockedCommands = Set("implicits", "javap", "power", "type", "kind", "reset") + private val blockedCommands = Set[String]("reset") /** Standard commands */ lazy val sparkStandardCommands: List[SparkILoop.this.LoopCommand] = From bd27dc489901e50fa1af932a948f6ff94f3c96e6 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Mon, 13 Jun 2016 11:49:15 -0700 Subject: [PATCH 0624/1470] [MINOR][SQL] Standardize 'continuous queries' to 'streaming Datasets/DataFrames' ## What changes were proposed in this pull request? This patch does some replacing (as `streaming Datasets/DataFrames` is the term we've chosen in [SPARK-15593](https://github.com/apache/spark/commit/00c310133df4f3893dd90d801168c2ab9841b102)): - `continuous queries` -> `streaming Datasets/DataFrames` - `non-continuous queries` -> `non-streaming Datasets/DataFrames` This patch also adds `test("check foreach() can only be called on streaming Datasets/DataFrames")`. ## How was this patch tested? N/A Author: Liwei Lin Closes #13595 from lw-lin/continuous-queries-to-streaming-dss-dfs. (cherry picked from commit d32e227787338a08741d8064f5dd2db1d60ddc63) Signed-off-by: Tathagata Das --- .../apache/spark/sql/DataFrameWriter.scala | 47 +++++----- .../spark/sql/streaming/StreamTest.scala | 4 +- .../test/DataFrameReaderWriterSuite.scala | 90 +++++++++++-------- 3 files changed, 77 insertions(+), 64 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index afae0786b73dd..392e3c1e4ead4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -56,7 +56,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def mode(saveMode: SaveMode): DataFrameWriter[T] = { // mode() is used for non-continuous queries // outputMode() is used for continuous queries - assertNotStreaming("mode() can only be called on non-continuous queries") + assertNotStreaming("mode() can only be called on non-streaming Datasets/DataFrames") this.mode = saveMode this } @@ -73,7 +73,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def mode(saveMode: String): DataFrameWriter[T] = { // mode() is used for non-continuous queries // outputMode() is used for continuous queries - assertNotStreaming("mode() can only be called on non-continuous queries") + assertNotStreaming("mode() can only be called on non-streaming Datasets/DataFrames") this.mode = saveMode.toLowerCase match { case "overwrite" => SaveMode.Overwrite case "append" => SaveMode.Append @@ -86,33 +86,33 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } /** - * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be + * Specifies how data of a streaming Dataset/DataFrame is written to a streaming sink. + * - `OutputMode.Append()`: only the new rows in the streaming Dataset/DataFrame will be * written to the sink - * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written + * - `OutputMode.Complete()`: all the rows in the streaming Dataset/DataFrame will be written * to the sink every time these is some updates * * @since 2.0.0 */ @Experimental def outputMode(outputMode: OutputMode): DataFrameWriter[T] = { - assertStreaming("outputMode() can only be called on continuous queries") + assertStreaming("outputMode() can only be called on streaming Datasets/DataFrames") this.outputMode = outputMode this } /** - * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to + * Specifies how data of a streaming Dataset/DataFrame is written to a streaming sink. + * - `append`: only the new rows in the streaming Dataset/DataFrame will be written to * the sink - * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink + * - `complete`: all the rows in the streaming Dataset/DataFrame will be written to the sink * every time these is some updates * * @since 2.0.0 */ @Experimental def outputMode(outputMode: String): DataFrameWriter[T] = { - assertStreaming("outputMode() can only be called on continuous queries") + assertStreaming("outputMode() can only be called on streaming Datasets/DataFrames") this.outputMode = outputMode.toLowerCase match { case "append" => OutputMode.Append @@ -150,7 +150,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ @Experimental def trigger(trigger: Trigger): DataFrameWriter[T] = { - assertStreaming("trigger() can only be called on continuous queries") + assertStreaming("trigger() can only be called on streaming Datasets/DataFrames") this.trigger = trigger this } @@ -284,7 +284,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def save(): Unit = { assertNotBucketed("save") - assertNotStreaming("save() can only be called on non-continuous queries") + assertNotStreaming("save() can only be called on non-streaming Datasets/DataFrames") val dataSource = DataSource( df.sparkSession, className = source, @@ -304,7 +304,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ @Experimental def queryName(queryName: String): DataFrameWriter[T] = { - assertStreaming("queryName() can only be called on continuous queries") + assertStreaming("queryName() can only be called on streaming Datasets/DataFrames") this.extraOptions += ("queryName" -> queryName) this } @@ -333,7 +333,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { @Experimental def startStream(): ContinuousQuery = { assertNotBucketed("startStream") - assertStreaming("startStream() can only be called on continuous queries") + assertStreaming("startStream() can only be called on streaming Datasets/DataFrames") if (source == "memory") { val queryName = @@ -434,8 +434,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def foreach(writer: ForeachWriter[T]): ContinuousQuery = { assertNotPartitioned("foreach") assertNotBucketed("foreach") - assertStreaming( - "foreach() can only be called on streaming Datasets/DataFrames.") + assertStreaming("foreach() can only be called on streaming Datasets/DataFrames.") val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) val sink = new ForeachSink[T](ds.sparkSession.sparkContext.clean(writer))(ds.exprEnc) @@ -502,7 +501,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def insertInto(tableIdent: TableIdentifier): Unit = { assertNotBucketed("insertInto") - assertNotStreaming("insertInto() can only be called on non-continuous queries") + assertNotStreaming("insertInto() can only be called on non-streaming Datasets/DataFrames") val partitions = normalizedParCols.map(_.map(col => col -> (Option.empty[String])).toMap) val overwrite = mode == SaveMode.Overwrite @@ -621,7 +620,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def saveAsTable(tableIdent: TableIdentifier): Unit = { - assertNotStreaming("saveAsTable() can only be called on non-continuous queries") + assertNotStreaming("saveAsTable() can only be called on non-streaming Datasets/DataFrames") val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) @@ -664,7 +663,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { assertNotPartitioned("jdbc") assertNotBucketed("jdbc") - assertNotStreaming("jdbc() can only be called on non-continuous queries") + assertNotStreaming("jdbc() can only be called on non-streaming Datasets/DataFrames") val props = new Properties() extraOptions.foreach { case (key, value) => @@ -723,7 +722,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def json(path: String): Unit = { - assertNotStreaming("json() can only be called on non-continuous queries") + assertNotStreaming("json() can only be called on non-streaming Datasets/DataFrames") format("json").save(path) } @@ -743,7 +742,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def parquet(path: String): Unit = { - assertNotStreaming("parquet() can only be called on non-continuous queries") + assertNotStreaming("parquet() can only be called on non-streaming Datasets/DataFrames") format("parquet").save(path) } @@ -763,7 +762,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @note Currently, this method can only be used after enabling Hive support */ def orc(path: String): Unit = { - assertNotStreaming("orc() can only be called on non-continuous queries") + assertNotStreaming("orc() can only be called on non-streaming Datasets/DataFrames") format("orc").save(path) } @@ -787,7 +786,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.6.0 */ def text(path: String): Unit = { - assertNotStreaming("text() can only be called on non-continuous queries") + assertNotStreaming("text() can only be called on non-streaming Datasets/DataFrames") format("text").save(path) } @@ -817,7 +816,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 2.0.0 */ def csv(path: String): Unit = { - assertNotStreaming("csv() can only be called on non-continuous queries") + assertNotStreaming("csv() can only be called on non-streaming Datasets/DataFrames") format("csv").save(path) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 7f1e5fe6135a7..fabb8ba6c60e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -70,7 +70,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { /** How long to wait for an active stream to catch up when checking a result. */ val streamingTimeout = 10.seconds - /** A trait for actions that can be performed while testing a streaming DataFrame. */ + /** A trait for actions that can be performed while testing a streaming DataSet/DataFrame. */ trait StreamAction /** A trait to mark actions that require the stream to be actively running. */ @@ -194,7 +194,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } /** - * Executes the specified actions on the given streaming DataFrame and provides helpful + * Executes the specified actions on the given streaming DataSet/DataFrame and provides helpful * error messages in the case of failures or incorrect answers. * * Note that if the stream is not explicitly started before an action that requires it to be diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index 6e0d66ae7f19a..51aa53287c2e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -371,66 +371,80 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { private def newTextInput = Utils.createTempDir(namePrefix = "text").getCanonicalPath - test("check trigger() can only be called on continuous queries") { + test("check trigger() can only be called on streaming Datasets/DataFrames") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.trigger(ProcessingTime("10 seconds"))) - assert(e.getMessage == "trigger() can only be called on continuous queries;") + assert(e.getMessage == "trigger() can only be called on streaming Datasets/DataFrames;") } - test("check queryName() can only be called on continuous queries") { + test("check queryName() can only be called on streaming Datasets/DataFrames") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.queryName("queryName")) - assert(e.getMessage == "queryName() can only be called on continuous queries;") + assert(e.getMessage == "queryName() can only be called on streaming Datasets/DataFrames;") } - test("check startStream() can only be called on continuous queries") { + test("check startStream() can only be called on streaming Datasets/DataFrames") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.startStream()) - assert(e.getMessage == "startStream() can only be called on continuous queries;") + assert(e.getMessage == "startStream() can only be called on streaming Datasets/DataFrames;") } - test("check startStream(path) can only be called on continuous queries") { + test("check startStream(path) can only be called on streaming Datasets/DataFrames") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.startStream("non_exist_path")) - assert(e.getMessage == "startStream() can only be called on continuous queries;") + assert(e.getMessage == "startStream() can only be called on streaming Datasets/DataFrames;") } - test("check mode(SaveMode) can only be called on non-continuous queries") { + test("check foreach() can only be called on streaming Datasets/DataFrames") { + val df = spark.read.text(newTextInput) + val w = df.write.option("checkpointLocation", newMetadataDir) + val foreachWriter = new ForeachWriter[String] { + override def open(partitionId: Long, version: Long): Boolean = false + override def process(value: String): Unit = {} + override def close(errorOrNull: Throwable): Unit = {} + } + val e = intercept[AnalysisException](w.foreach(foreachWriter)) + Seq("foreach()", "streaming Datasets/DataFrames").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + + test("check mode(SaveMode) can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.mode(SaveMode.Append)) - assert(e.getMessage == "mode() can only be called on non-continuous queries;") + assert(e.getMessage == "mode() can only be called on non-streaming Datasets/DataFrames;") } - test("check mode(string) can only be called on non-continuous queries") { + test("check mode(string) can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.mode("append")) - assert(e.getMessage == "mode() can only be called on non-continuous queries;") + assert(e.getMessage == "mode() can only be called on non-streaming Datasets/DataFrames;") } - test("check outputMode(OutputMode) can only be called on continuous queries") { + test("check outputMode(OutputMode) can only be called on streaming Datasets/DataFrames") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.outputMode(OutputMode.Append)) - Seq("outputmode", "continuous queries").foreach { s => + Seq("outputmode", "streaming Datasets/DataFrames").foreach { s => assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) } } - test("check outputMode(string) can only be called on continuous queries") { + test("check outputMode(string) can only be called on streaming Datasets/DataFrames") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.outputMode("append")) - Seq("outputmode", "continuous queries").foreach { s => + Seq("outputmode", "streaming Datasets/DataFrames").foreach { s => assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) } } @@ -450,7 +464,7 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { testError("Xyz") } - test("check bucketBy() can only be called on non-continuous queries") { + test("check bucketBy() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() @@ -459,7 +473,7 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { assert(e.getMessage == "'startStream' does not support bucketing right now;") } - test("check sortBy() can only be called on non-continuous queries;") { + test("check sortBy() can only be called on non-streaming Datasets/DataFrames;") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() @@ -468,94 +482,94 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { assert(e.getMessage == "'startStream' does not support bucketing right now;") } - test("check save(path) can only be called on non-continuous queries") { + test("check save(path) can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.save("non_exist_path")) - assert(e.getMessage == "save() can only be called on non-continuous queries;") + assert(e.getMessage == "save() can only be called on non-streaming Datasets/DataFrames;") } - test("check save() can only be called on non-continuous queries") { + test("check save() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.save()) - assert(e.getMessage == "save() can only be called on non-continuous queries;") + assert(e.getMessage == "save() can only be called on non-streaming Datasets/DataFrames;") } - test("check insertInto() can only be called on non-continuous queries") { + test("check insertInto() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.insertInto("non_exsit_table")) - assert(e.getMessage == "insertInto() can only be called on non-continuous queries;") + assert(e.getMessage == "insertInto() can only be called on non-streaming Datasets/DataFrames;") } - test("check saveAsTable() can only be called on non-continuous queries") { + test("check saveAsTable() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.saveAsTable("non_exsit_table")) - assert(e.getMessage == "saveAsTable() can only be called on non-continuous queries;") + assert(e.getMessage == "saveAsTable() can only be called on non-streaming Datasets/DataFrames;") } - test("check jdbc() can only be called on non-continuous queries") { + test("check jdbc() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.jdbc(null, null, null)) - assert(e.getMessage == "jdbc() can only be called on non-continuous queries;") + assert(e.getMessage == "jdbc() can only be called on non-streaming Datasets/DataFrames;") } - test("check json() can only be called on non-continuous queries") { + test("check json() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.json("non_exist_path")) - assert(e.getMessage == "json() can only be called on non-continuous queries;") + assert(e.getMessage == "json() can only be called on non-streaming Datasets/DataFrames;") } - test("check parquet() can only be called on non-continuous queries") { + test("check parquet() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.parquet("non_exist_path")) - assert(e.getMessage == "parquet() can only be called on non-continuous queries;") + assert(e.getMessage == "parquet() can only be called on non-streaming Datasets/DataFrames;") } - test("check orc() can only be called on non-continuous queries") { + test("check orc() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.orc("non_exist_path")) - assert(e.getMessage == "orc() can only be called on non-continuous queries;") + assert(e.getMessage == "orc() can only be called on non-streaming Datasets/DataFrames;") } - test("check text() can only be called on non-continuous queries") { + test("check text() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.text("non_exist_path")) - assert(e.getMessage == "text() can only be called on non-continuous queries;") + assert(e.getMessage == "text() can only be called on non-streaming Datasets/DataFrames;") } - test("check csv() can only be called on non-continuous queries") { + test("check csv() can only be called on non-streaming Datasets/DataFrames") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.csv("non_exist_path")) - assert(e.getMessage == "csv() can only be called on non-continuous queries;") + assert(e.getMessage == "csv() can only be called on non-streaming Datasets/DataFrames;") } test("check foreach() does not support partitioning or bucketing") { From 2a0da84dc6bd295c0aeba84562cfe448fdbca08c Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 13 Jun 2016 12:47:47 -0700 Subject: [PATCH 0625/1470] =?UTF-8?q?[HOTFIX][MINOR][SQL]=20Revert=20"=20S?= =?UTF-8?q?tandardize=20'continuous=20queries'=20to=20'streaming=20D?= =?UTF-8?q?=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit d32e227787338a08741d8064f5dd2db1d60ddc63. Broke build - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-branch-2.0-compile-maven-hadoop-2.3/326/console Author: Tathagata Das Closes #13645 from tdas/build-break. (cherry picked from commit a6a18a4573515e76d78534f1a19fcc2c3819f6c5) Signed-off-by: Tathagata Das --- .../apache/spark/sql/DataFrameWriter.scala | 47 +++++----- .../spark/sql/streaming/StreamTest.scala | 4 +- .../test/DataFrameReaderWriterSuite.scala | 90 ++++++++----------- 3 files changed, 64 insertions(+), 77 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 392e3c1e4ead4..afae0786b73dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -56,7 +56,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def mode(saveMode: SaveMode): DataFrameWriter[T] = { // mode() is used for non-continuous queries // outputMode() is used for continuous queries - assertNotStreaming("mode() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode this } @@ -73,7 +73,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def mode(saveMode: String): DataFrameWriter[T] = { // mode() is used for non-continuous queries // outputMode() is used for continuous queries - assertNotStreaming("mode() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode.toLowerCase match { case "overwrite" => SaveMode.Overwrite case "append" => SaveMode.Append @@ -86,33 +86,33 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } /** - * Specifies how data of a streaming Dataset/DataFrame is written to a streaming sink. - * - `OutputMode.Append()`: only the new rows in the streaming Dataset/DataFrame will be + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be * written to the sink - * - `OutputMode.Complete()`: all the rows in the streaming Dataset/DataFrame will be written + * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written * to the sink every time these is some updates * * @since 2.0.0 */ @Experimental def outputMode(outputMode: OutputMode): DataFrameWriter[T] = { - assertStreaming("outputMode() can only be called on streaming Datasets/DataFrames") + assertStreaming("outputMode() can only be called on continuous queries") this.outputMode = outputMode this } /** - * Specifies how data of a streaming Dataset/DataFrame is written to a streaming sink. - * - `append`: only the new rows in the streaming Dataset/DataFrame will be written to + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to * the sink - * - `complete`: all the rows in the streaming Dataset/DataFrame will be written to the sink + * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink * every time these is some updates * * @since 2.0.0 */ @Experimental def outputMode(outputMode: String): DataFrameWriter[T] = { - assertStreaming("outputMode() can only be called on streaming Datasets/DataFrames") + assertStreaming("outputMode() can only be called on continuous queries") this.outputMode = outputMode.toLowerCase match { case "append" => OutputMode.Append @@ -150,7 +150,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ @Experimental def trigger(trigger: Trigger): DataFrameWriter[T] = { - assertStreaming("trigger() can only be called on streaming Datasets/DataFrames") + assertStreaming("trigger() can only be called on continuous queries") this.trigger = trigger this } @@ -284,7 +284,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def save(): Unit = { assertNotBucketed("save") - assertNotStreaming("save() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("save() can only be called on non-continuous queries") val dataSource = DataSource( df.sparkSession, className = source, @@ -304,7 +304,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ @Experimental def queryName(queryName: String): DataFrameWriter[T] = { - assertStreaming("queryName() can only be called on streaming Datasets/DataFrames") + assertStreaming("queryName() can only be called on continuous queries") this.extraOptions += ("queryName" -> queryName) this } @@ -333,7 +333,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { @Experimental def startStream(): ContinuousQuery = { assertNotBucketed("startStream") - assertStreaming("startStream() can only be called on streaming Datasets/DataFrames") + assertStreaming("startStream() can only be called on continuous queries") if (source == "memory") { val queryName = @@ -434,7 +434,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def foreach(writer: ForeachWriter[T]): ContinuousQuery = { assertNotPartitioned("foreach") assertNotBucketed("foreach") - assertStreaming("foreach() can only be called on streaming Datasets/DataFrames.") + assertStreaming( + "foreach() can only be called on streaming Datasets/DataFrames.") val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) val sink = new ForeachSink[T](ds.sparkSession.sparkContext.clean(writer))(ds.exprEnc) @@ -501,7 +502,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def insertInto(tableIdent: TableIdentifier): Unit = { assertNotBucketed("insertInto") - assertNotStreaming("insertInto() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("insertInto() can only be called on non-continuous queries") val partitions = normalizedParCols.map(_.map(col => col -> (Option.empty[String])).toMap) val overwrite = mode == SaveMode.Overwrite @@ -620,7 +621,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def saveAsTable(tableIdent: TableIdentifier): Unit = { - assertNotStreaming("saveAsTable() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("saveAsTable() can only be called on non-continuous queries") val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) @@ -663,7 +664,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { assertNotPartitioned("jdbc") assertNotBucketed("jdbc") - assertNotStreaming("jdbc() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("jdbc() can only be called on non-continuous queries") val props = new Properties() extraOptions.foreach { case (key, value) => @@ -722,7 +723,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def json(path: String): Unit = { - assertNotStreaming("json() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("json() can only be called on non-continuous queries") format("json").save(path) } @@ -742,7 +743,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def parquet(path: String): Unit = { - assertNotStreaming("parquet() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("parquet() can only be called on non-continuous queries") format("parquet").save(path) } @@ -762,7 +763,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @note Currently, this method can only be used after enabling Hive support */ def orc(path: String): Unit = { - assertNotStreaming("orc() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("orc() can only be called on non-continuous queries") format("orc").save(path) } @@ -786,7 +787,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.6.0 */ def text(path: String): Unit = { - assertNotStreaming("text() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("text() can only be called on non-continuous queries") format("text").save(path) } @@ -816,7 +817,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 2.0.0 */ def csv(path: String): Unit = { - assertNotStreaming("csv() can only be called on non-streaming Datasets/DataFrames") + assertNotStreaming("csv() can only be called on non-continuous queries") format("csv").save(path) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index fabb8ba6c60e0..7f1e5fe6135a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -70,7 +70,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { /** How long to wait for an active stream to catch up when checking a result. */ val streamingTimeout = 10.seconds - /** A trait for actions that can be performed while testing a streaming DataSet/DataFrame. */ + /** A trait for actions that can be performed while testing a streaming DataFrame. */ trait StreamAction /** A trait to mark actions that require the stream to be actively running. */ @@ -194,7 +194,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } /** - * Executes the specified actions on the given streaming DataSet/DataFrame and provides helpful + * Executes the specified actions on the given streaming DataFrame and provides helpful * error messages in the case of failures or incorrect answers. * * Note that if the stream is not explicitly started before an action that requires it to be diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala index 51aa53287c2e8..6e0d66ae7f19a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala @@ -371,80 +371,66 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { private def newTextInput = Utils.createTempDir(namePrefix = "text").getCanonicalPath - test("check trigger() can only be called on streaming Datasets/DataFrames") { + test("check trigger() can only be called on continuous queries") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.trigger(ProcessingTime("10 seconds"))) - assert(e.getMessage == "trigger() can only be called on streaming Datasets/DataFrames;") + assert(e.getMessage == "trigger() can only be called on continuous queries;") } - test("check queryName() can only be called on streaming Datasets/DataFrames") { + test("check queryName() can only be called on continuous queries") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.queryName("queryName")) - assert(e.getMessage == "queryName() can only be called on streaming Datasets/DataFrames;") + assert(e.getMessage == "queryName() can only be called on continuous queries;") } - test("check startStream() can only be called on streaming Datasets/DataFrames") { + test("check startStream() can only be called on continuous queries") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.startStream()) - assert(e.getMessage == "startStream() can only be called on streaming Datasets/DataFrames;") + assert(e.getMessage == "startStream() can only be called on continuous queries;") } - test("check startStream(path) can only be called on streaming Datasets/DataFrames") { + test("check startStream(path) can only be called on continuous queries") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.startStream("non_exist_path")) - assert(e.getMessage == "startStream() can only be called on streaming Datasets/DataFrames;") + assert(e.getMessage == "startStream() can only be called on continuous queries;") } - test("check foreach() can only be called on streaming Datasets/DataFrames") { - val df = spark.read.text(newTextInput) - val w = df.write.option("checkpointLocation", newMetadataDir) - val foreachWriter = new ForeachWriter[String] { - override def open(partitionId: Long, version: Long): Boolean = false - override def process(value: String): Unit = {} - override def close(errorOrNull: Throwable): Unit = {} - } - val e = intercept[AnalysisException](w.foreach(foreachWriter)) - Seq("foreach()", "streaming Datasets/DataFrames").foreach { s => - assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) - } - } - - test("check mode(SaveMode) can only be called on non-streaming Datasets/DataFrames") { + test("check mode(SaveMode) can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.mode(SaveMode.Append)) - assert(e.getMessage == "mode() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "mode() can only be called on non-continuous queries;") } - test("check mode(string) can only be called on non-streaming Datasets/DataFrames") { + test("check mode(string) can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.mode("append")) - assert(e.getMessage == "mode() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "mode() can only be called on non-continuous queries;") } - test("check outputMode(OutputMode) can only be called on streaming Datasets/DataFrames") { + test("check outputMode(OutputMode) can only be called on continuous queries") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.outputMode(OutputMode.Append)) - Seq("outputmode", "streaming Datasets/DataFrames").foreach { s => + Seq("outputmode", "continuous queries").foreach { s => assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) } } - test("check outputMode(string) can only be called on streaming Datasets/DataFrames") { + test("check outputMode(string) can only be called on continuous queries") { val df = spark.read.text(newTextInput) val w = df.write.option("checkpointLocation", newMetadataDir) val e = intercept[AnalysisException](w.outputMode("append")) - Seq("outputmode", "streaming Datasets/DataFrames").foreach { s => + Seq("outputmode", "continuous queries").foreach { s => assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) } } @@ -464,7 +450,7 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { testError("Xyz") } - test("check bucketBy() can only be called on non-streaming Datasets/DataFrames") { + test("check bucketBy() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() @@ -473,7 +459,7 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { assert(e.getMessage == "'startStream' does not support bucketing right now;") } - test("check sortBy() can only be called on non-streaming Datasets/DataFrames;") { + test("check sortBy() can only be called on non-continuous queries;") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() @@ -482,94 +468,94 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { assert(e.getMessage == "'startStream' does not support bucketing right now;") } - test("check save(path) can only be called on non-streaming Datasets/DataFrames") { + test("check save(path) can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.save("non_exist_path")) - assert(e.getMessage == "save() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "save() can only be called on non-continuous queries;") } - test("check save() can only be called on non-streaming Datasets/DataFrames") { + test("check save() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.save()) - assert(e.getMessage == "save() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "save() can only be called on non-continuous queries;") } - test("check insertInto() can only be called on non-streaming Datasets/DataFrames") { + test("check insertInto() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.insertInto("non_exsit_table")) - assert(e.getMessage == "insertInto() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "insertInto() can only be called on non-continuous queries;") } - test("check saveAsTable() can only be called on non-streaming Datasets/DataFrames") { + test("check saveAsTable() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.saveAsTable("non_exsit_table")) - assert(e.getMessage == "saveAsTable() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "saveAsTable() can only be called on non-continuous queries;") } - test("check jdbc() can only be called on non-streaming Datasets/DataFrames") { + test("check jdbc() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.jdbc(null, null, null)) - assert(e.getMessage == "jdbc() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "jdbc() can only be called on non-continuous queries;") } - test("check json() can only be called on non-streaming Datasets/DataFrames") { + test("check json() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.json("non_exist_path")) - assert(e.getMessage == "json() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "json() can only be called on non-continuous queries;") } - test("check parquet() can only be called on non-streaming Datasets/DataFrames") { + test("check parquet() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.parquet("non_exist_path")) - assert(e.getMessage == "parquet() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "parquet() can only be called on non-continuous queries;") } - test("check orc() can only be called on non-streaming Datasets/DataFrames") { + test("check orc() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.orc("non_exist_path")) - assert(e.getMessage == "orc() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "orc() can only be called on non-continuous queries;") } - test("check text() can only be called on non-streaming Datasets/DataFrames") { + test("check text() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.text("non_exist_path")) - assert(e.getMessage == "text() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "text() can only be called on non-continuous queries;") } - test("check csv() can only be called on non-streaming Datasets/DataFrames") { + test("check csv() can only be called on non-continuous queries") { val df = spark.read .format("org.apache.spark.sql.streaming.test") .stream() val w = df.write val e = intercept[AnalysisException](w.csv("non_exist_path")) - assert(e.getMessage == "csv() can only be called on non-streaming Datasets/DataFrames;") + assert(e.getMessage == "csv() can only be called on non-continuous queries;") } test("check foreach() does not support partitioning or bucketing") { From 8c4050a5a46a649bd82dbe23871e929be368e509 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 13 Jun 2016 13:22:46 -0700 Subject: [PATCH 0626/1470] [SPARK-15676][SQL] Disallow Column Names as Partition Columns For Hive Tables #### What changes were proposed in this pull request? When creating a Hive Table (not data source tables), a common error users might make is to specify an existing column name as a partition column. Below is what Hive returns in this case: ``` hive> CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (data string, part string); FAILED: SemanticException [Error 10035]: Column repeated in partitioning columns ``` Currently, the error we issued is very confusing: ``` org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:For direct MetaStore DB connections, we don't support retries at the client level.); ``` This PR is to fix the above issue by capturing the usage error in `Parser`. #### How was this patch tested? Added a test case to `DDLCommandSuite` Author: gatorsmile Closes #13415 from gatorsmile/partitionColumnsInTableSchema. (cherry picked from commit 3b7fb84cf88bcae56713fd56396db537fa18f2e5) Signed-off-by: Yin Huai --- .../spark/sql/execution/SparkSqlParser.scala | 17 +++++++++++++++++ .../sql/execution/command/DDLCommandSuite.scala | 14 ++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 06d8f15dc3008..a0508ad6019bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -903,6 +903,23 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val selectQuery = Option(ctx.query).map(plan) + // Ensuring whether no duplicate name is used in table definition + val colNames = cols.map(_.name) + if (colNames.length != colNames.distinct.length) { + val duplicateColumns = colNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => "\"" + x + "\"" + } + throw operationNotAllowed(s"Duplicated column names found in table definition of $name: " + + duplicateColumns.mkString("[", ",", "]"), ctx) + } + + // For Hive tables, partition columns must not be part of the schema + val badPartCols = partitionCols.map(_.name).toSet.intersect(colNames.toSet) + if (badPartCols.nonEmpty) { + throw operationNotAllowed(s"Partition columns may not be specified in the schema: " + + badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]"), ctx) + } + // Note: Hive requires partition columns to be distinct from the schema, so we need // to include the partition columns here explicitly val schema = cols ++ partitionCols diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index aec7e99d9d37d..5bee28b4462c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -334,6 +334,20 @@ class DDLCommandSuite extends PlanTest { assert(ct.table.storage.locationUri == Some("/something/anything")) } + test("create table - column repeated in partitioning columns") { + val query = "CREATE TABLE tab1 (key INT, value STRING) PARTITIONED BY (key INT, hr STRING)" + val e = intercept[ParseException] { parser.parsePlan(query) } + assert(e.getMessage.contains( + "Operation not allowed: Partition columns may not be specified in the schema: [\"key\"]")) + } + + test("create table - duplicate column names in the table definition") { + val query = "CREATE TABLE default.tab1 (key INT, key STRING)" + val e = intercept[ParseException] { parser.parsePlan(query) } + assert(e.getMessage.contains("Operation not allowed: Duplicated column names found in " + + "table definition of `default`.`tab1`: [\"key\"]")) + } + test("create table using - with partitioned by") { val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (a)" val expected = CreateTableUsing( From d9db8a9c85163c25ae3ec84cb9ac88b5615db82e Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Mon, 13 Jun 2016 13:41:26 -0700 Subject: [PATCH 0627/1470] [SPARK-15530][SQL] Set #parallelism for file listing in listLeafFilesInParallel ## What changes were proposed in this pull request? This pr is to set the number of parallelism to prevent file listing in `listLeafFilesInParallel` from generating many tasks in case of large #defaultParallelism. ## How was this patch tested? Manually checked Author: Takeshi YAMAMURO Closes #13444 from maropu/SPARK-15530. (cherry picked from commit 5ad4e32d46599ae1b8626f08aa97345d078c28d7) Signed-off-by: Yin Huai --- .../execution/datasources/ListingFileCatalog.scala | 2 +- .../datasources/fileSourceInterfaces.scala | 13 +++++++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index dd3c96a792357..7d2854aaad394 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -75,7 +75,7 @@ class ListingFileCatalog( protected def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession.sparkContext) + HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession) } else { // Dummy jobconf to get to the pathFilter defined in configuration val jobConf = new JobConf(hadoopConf, this.getClass) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 890e64db59945..9c1898994c9a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -448,13 +448,22 @@ private[sql] object HadoopFsRelation extends Logging { def listLeafFilesInParallel( paths: Seq[Path], hadoopConf: Configuration, - sparkContext: SparkContext): mutable.LinkedHashSet[FileStatus] = { + sparkSession: SparkSession): mutable.LinkedHashSet[FileStatus] = { + assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") + val sparkContext = sparkSession.sparkContext + val sqlConf = sparkSession.sessionState.conf val serializableConfiguration = new SerializableConfiguration(hadoopConf) val serializedPaths = paths.map(_.toString) - val fakeStatuses = sparkContext.parallelize(serializedPaths).mapPartitions { paths => + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, 10000) + + val fakeStatuses = sparkContext + .parallelize(serializedPaths, numParallelism) + .mapPartitions { paths => // Dummy jobconf to get to the pathFilter defined in configuration // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) val jobConf = new JobConf(serializableConfiguration.value, this.getClass) From 97fe1d8ee10413e2dbff7b90a7e747636fcb2e6c Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 13 Jun 2016 13:44:46 -0700 Subject: [PATCH 0628/1470] [SPARK-15889][SQL][STREAMING] Add a unique id to ContinuousQuery ## What changes were proposed in this pull request? ContinuousQueries have names that are unique across all the active ones. However, when queries are rapidly restarted with same name, it causes races conditions with the listener. A listener event from a stopped query can arrive after the query has been restarted, leading to complexities in monitoring infrastructure. Along with this change, I have also consolidated all the messy code paths to start queries with different sinks. ## How was this patch tested? Added unit tests, and existing unit tests. Author: Tathagata Das Closes #13613 from tdas/SPARK-15889. (cherry picked from commit c654ae2140bc184adb407fd02072b653c5359ee5) Signed-off-by: Shixiong Zhu --- .../apache/spark/sql/DataFrameWriter.scala | 69 +++++------------ .../execution/streaming/StreamExecution.scala | 8 +- .../spark/sql/streaming/ContinuousQuery.scala | 11 ++- .../sql/streaming/ContinuousQueryInfo.scala | 5 +- .../streaming/ContinuousQueryManager.scala | 74 +++++++++++++++---- .../ContinuousQueryListenerSuite.scala | 17 ++++- .../ContinuousQueryManagerSuite.scala | 29 ++------ .../sql/streaming/ContinuousQuerySuite.scala | 43 ++++++++++- .../spark/sql/streaming/StreamTest.scala | 12 +-- 9 files changed, 167 insertions(+), 101 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index afae0786b73dd..171b1378e5f94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -336,34 +336,23 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { assertStreaming("startStream() can only be called on continuous queries") if (source == "memory") { - val queryName = - extraOptions.getOrElse( - "queryName", throw new AnalysisException("queryName must be specified for memory sink")) - val checkpointLocation = getCheckpointLocation(queryName, failIfNotSet = false).getOrElse { - Utils.createTempDir(namePrefix = "memory.stream").getCanonicalPath - } - - // If offsets have already been created, we trying to resume a query. - val checkpointPath = new Path(checkpointLocation, "offsets") - val fs = checkpointPath.getFileSystem(df.sparkSession.sessionState.newHadoopConf()) - if (fs.exists(checkpointPath)) { - throw new AnalysisException( - s"Unable to resume query written to memory sink. Delete $checkpointPath to start over.") - } else { - checkpointPath.toUri.toString + if (extraOptions.get("queryName").isEmpty) { + throw new AnalysisException("queryName must be specified for memory sink") } val sink = new MemorySink(df.schema, outputMode) val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink)) - resultDf.createOrReplaceTempView(queryName) - val continuousQuery = df.sparkSession.sessionState.continuousQueryManager.startQuery( - queryName, - checkpointLocation, + val query = df.sparkSession.sessionState.continuousQueryManager.startQuery( + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), df, sink, outputMode, - trigger) - continuousQuery + useTempCheckpointLocation = true, + recoverFromCheckpointLocation = false, + trigger = trigger) + resultDf.createOrReplaceTempView(query.name) + query } else { val dataSource = DataSource( @@ -371,14 +360,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { className = source, options = extraOptions.toMap, partitionColumns = normalizedParCols.getOrElse(Nil)) - val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) df.sparkSession.sessionState.continuousQueryManager.startQuery( - queryName, - getCheckpointLocation(queryName, failIfNotSet = true).get, + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), df, dataSource.createSink(outputMode), outputMode, - trigger) + trigger = trigger) } } @@ -437,38 +425,15 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { assertStreaming( "foreach() can only be called on streaming Datasets/DataFrames.") - val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) val sink = new ForeachSink[T](ds.sparkSession.sparkContext.clean(writer))(ds.exprEnc) df.sparkSession.sessionState.continuousQueryManager.startQuery( - queryName, - getCheckpointLocation(queryName, failIfNotSet = false).getOrElse { - Utils.createTempDir(namePrefix = "foreach.stream").getCanonicalPath - }, + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), df, sink, outputMode, - trigger) - } - - /** - * Returns the checkpointLocation for a query. If `failIfNotSet` is `true` but the checkpoint - * location is not set, [[AnalysisException]] will be thrown. If `failIfNotSet` is `false`, `None` - * will be returned if the checkpoint location is not set. - */ - private def getCheckpointLocation(queryName: String, failIfNotSet: Boolean): Option[String] = { - val checkpointLocation = extraOptions.get("checkpointLocation").map { userSpecified => - new Path(userSpecified).toUri.toString - }.orElse { - df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION).map { location => - new Path(location, queryName).toUri.toString - } - } - if (failIfNotSet && checkpointLocation.isEmpty) { - throw new AnalysisException("checkpointLocation must be specified either " + - """through option("checkpointLocation", ...) or """ + - s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""") - } - checkpointLocation + useTempCheckpointLocation = true, + trigger = trigger) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 954fc33ecc7ba..5095fe7fcaa31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.streaming import java.util.concurrent.{CountDownLatch, TimeUnit} -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.locks.ReentrantLock import scala.collection.mutable.ArrayBuffer @@ -44,6 +44,7 @@ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} */ class StreamExecution( override val sparkSession: SparkSession, + override val id: Long, override val name: String, checkpointRoot: String, private[sql] val logicalPlan: LogicalPlan, @@ -492,6 +493,7 @@ class StreamExecution( private def toInfo: ContinuousQueryInfo = { new ContinuousQueryInfo( this.name, + this.id, this.sourceStatuses, this.sinkStatus) } @@ -503,7 +505,7 @@ class StreamExecution( } private[sql] object StreamExecution { - private val nextId = new AtomicInteger() + private val _nextId = new AtomicLong(0) - def nextName: String = s"query-${nextId.getAndIncrement}" + def nextId: Long = _nextId.getAndIncrement() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala index 3bbb0b8a888fe..1e0a47deca5da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala @@ -30,11 +30,20 @@ import org.apache.spark.sql.SparkSession trait ContinuousQuery { /** - * Returns the name of the query. + * Returns the name of the query. This name is unique across all active queries. This can be + * set in the[[org.apache.spark.sql.DataFrameWriter DataFrameWriter]] as + * `dataframe.write().queryName("query").startStream()`. * @since 2.0.0 */ def name: String + /** + * Returns the unique id of this query. This id is automatically generated and is unique across + * all queries that have been started in the current process. + * @since 2.0.0 + */ + def id: Long + /** * Returns the [[SparkSession]] associated with `this`. * @since 2.0.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala index 57b718b08fbbb..19f22704ba11b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala @@ -23,12 +23,15 @@ import org.apache.spark.annotation.Experimental * :: Experimental :: * A class used to report information about the progress of a [[ContinuousQuery]]. * - * @param name The [[ContinuousQuery]] name. + * @param name The [[ContinuousQuery]] name. This name is unique across all active queries. + * @param id The [[ContinuousQuery]] id. This id is unique across + * all queries that have been started in the current process. * @param sourceStatuses The current statuses of the [[ContinuousQuery]]'s sources. * @param sinkStatus The current status of the [[ContinuousQuery]]'s sink. */ @Experimental class ContinuousQueryInfo private[sql]( val name: String, + val id: Long, val sourceStatuses: Seq[SourceStatus], val sinkStatus: SinkStatus) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala index 1bfdd2da4e69e..0f4a9c9975565 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.streaming import scala.collection.mutable +import org.apache.hadoop.fs.Path + import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.util.{Clock, SystemClock} +import org.apache.spark.util.{Clock, SystemClock, Utils} /** * :: Experimental :: @@ -39,7 +41,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { private[sql] val stateStoreCoordinator = StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) private val listenerBus = new ContinuousQueryListenerBus(sparkSession.sparkContext.listenerBus) - private val activeQueries = new mutable.HashMap[String, ContinuousQuery] + private val activeQueries = new mutable.HashMap[Long, ContinuousQuery] private val activeQueriesLock = new Object private val awaitTerminationLock = new Object @@ -55,13 +57,12 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { } /** - * Returns an active query from this SQLContext or throws exception if bad name + * Returns the query if there is an active query with the given id, or null. * * @since 2.0.0 */ - def get(name: String): ContinuousQuery = activeQueriesLock.synchronized { - activeQueries.getOrElse(name, - throw new IllegalArgumentException(s"There is no active query with name $name")) + def get(id: Long): ContinuousQuery = activeQueriesLock.synchronized { + activeQueries.get(id).orNull } /** @@ -168,20 +169,66 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { listenerBus.post(event) } - /** Start a query */ + /** + * Start a [[ContinuousQuery]]. + * @param userSpecifiedName Query name optionally specified by the user. + * @param userSpecifiedCheckpointLocation Checkpoint location optionally specified by the user. + * @param df Streaming DataFrame. + * @param sink Sink to write the streaming outputs. + * @param outputMode Output mode for the sink. + * @param useTempCheckpointLocation Whether to use a temporary checkpoint location when the user + * has not specified one. If false, then error will be thrown. + * @param recoverFromCheckpointLocation Whether to recover query from the checkpoint location. + * If false and the checkpoint location exists, then error + * will be thrown. + * @param trigger [[Trigger]] for the query. + * @param triggerClock [[Clock]] to use for the triggering. + */ private[sql] def startQuery( - name: String, - checkpointLocation: String, + userSpecifiedName: Option[String], + userSpecifiedCheckpointLocation: Option[String], df: DataFrame, sink: Sink, outputMode: OutputMode, + useTempCheckpointLocation: Boolean = false, + recoverFromCheckpointLocation: Boolean = true, trigger: Trigger = ProcessingTime(0), triggerClock: Clock = new SystemClock()): ContinuousQuery = { activeQueriesLock.synchronized { - if (activeQueries.contains(name)) { + val id = StreamExecution.nextId + val name = userSpecifiedName.getOrElse(s"query-$id") + if (activeQueries.values.exists(_.name == name)) { throw new IllegalArgumentException( s"Cannot start query with name $name as a query with that name is already active") } + val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified => + new Path(userSpecified).toUri.toString + }.orElse { + df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION).map { location => + new Path(location, name).toUri.toString + } + }.getOrElse { + if (useTempCheckpointLocation) { + Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath + } else { + throw new AnalysisException( + "checkpointLocation must be specified either " + + """through option("checkpointLocation", ...) or """ + + s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""") + } + } + + // If offsets have already been created, we trying to resume a query. + if (!recoverFromCheckpointLocation) { + val checkpointPath = new Path(checkpointLocation, "offsets") + val fs = checkpointPath.getFileSystem(df.sparkSession.sessionState.newHadoopConf()) + if (fs.exists(checkpointPath)) { + throw new AnalysisException( + s"This query does not support recovering from checkpoint location. " + + s"Delete $checkpointPath to start over.") + } + } + val analyzedPlan = df.queryExecution.analyzed df.queryExecution.assertAnalyzed() @@ -203,6 +250,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { } val query = new StreamExecution( sparkSession, + id, name, checkpointLocation, logicalPlan, @@ -211,7 +259,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { triggerClock, outputMode) query.start() - activeQueries.put(name, query) + activeQueries.put(id, query) query } } @@ -219,7 +267,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { /** Notify (by the ContinuousQuery) that the query has been terminated */ private[sql] def notifyQueryTermination(terminatedQuery: ContinuousQuery): Unit = { activeQueriesLock.synchronized { - activeQueries -= terminatedQuery.name + activeQueries -= terminatedQuery.id } awaitTerminationLock.synchronized { if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala index 9b59ab60a688b..8e1de09a96e24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala @@ -50,9 +50,11 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { withListenerAdded(listener) { testStream(input.toDS)( StartStream(), - Assert("Incorrect query status in onQueryStarted") { + AssertOnQuery("Incorrect query status in onQueryStarted") { query => val status = listener.startStatus assert(status != null) + assert(status.name === query.name) + assert(status.id === query.id) assert(status.sourceStatuses.size === 1) assert(status.sourceStatuses(0).description.contains("Memory")) @@ -67,13 +69,15 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { }, AddDataMemory(input, Seq(1, 2, 3)), CheckAnswer(1, 2, 3), - Assert("Incorrect query status in onQueryProgress") { + AssertOnQuery("Incorrect query status in onQueryProgress") { query => eventually(Timeout(streamingTimeout)) { // There should be only on progress event as batch has been processed assert(listener.progressStatuses.size === 1) val status = listener.progressStatuses.peek() assert(status != null) + assert(status.name === query.name) + assert(status.id === query.id) assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) @@ -82,12 +86,16 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { } }, StopStream, - Assert("Incorrect query status in onQueryTerminated") { + AssertOnQuery("Incorrect query status in onQueryTerminated") { query => eventually(Timeout(streamingTimeout)) { val status = listener.terminationStatus assert(status != null) + assert(status.name === query.name) + assert(status.id === query.id) assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) + assert(listener.terminationStackTrace.isEmpty) + assert(listener.terminationException === None) } listener.checkAsyncErrors() } @@ -161,6 +169,7 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { test("QueryStarted serialization") { val queryStartedInfo = new ContinuousQueryInfo( "name", + 1, Seq(new SourceStatus("source1", None), new SourceStatus("source2", None)), new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString)) val queryStarted = new ContinuousQueryListener.QueryStarted(queryStartedInfo) @@ -173,6 +182,7 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { test("QueryProgress serialization") { val queryProcessInfo = new ContinuousQueryInfo( "name", + 1, Seq( new SourceStatus("source1", Some(LongOffset(0).toString)), new SourceStatus("source2", Some(LongOffset(1).toString))), @@ -187,6 +197,7 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { test("QueryTerminated serialization") { val queryTerminatedInfo = new ContinuousQueryInfo( "name", + 1, Seq( new SourceStatus("source1", Some(LongOffset(0).toString)), new SourceStatus("source2", Some(LongOffset(1).toString))), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala index c1e4970b3a877..f81608bdb85e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala @@ -59,23 +59,15 @@ class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { assert(spark.streams.active.toSet === queries.toSet) val (q1, q2, q3) = (queries(0), queries(1), queries(2)) - assert(spark.streams.get(q1.name).eq(q1)) - assert(spark.streams.get(q2.name).eq(q2)) - assert(spark.streams.get(q3.name).eq(q3)) - intercept[IllegalArgumentException] { - spark.streams.get("non-existent-name") - } - + assert(spark.streams.get(q1.id).eq(q1)) + assert(spark.streams.get(q2.id).eq(q2)) + assert(spark.streams.get(q3.id).eq(q3)) + assert(spark.streams.get(-1) === null) // non-existent id q1.stop() assert(spark.streams.active.toSet === Set(q2, q3)) - val ex1 = withClue("no error while getting non-active query") { - intercept[IllegalArgumentException] { - spark.streams.get(q1.name) - } - } - assert(ex1.getMessage.contains(q1.name), "error does not contain name of query to be fetched") - assert(spark.streams.get(q2.name).eq(q2)) + assert(spark.streams.get(q1.id) === null) + assert(spark.streams.get(q2.id).eq(q2)) m2.addData(0) // q2 should terminate with error @@ -83,12 +75,7 @@ class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { require(!q2.isActive) require(q2.exception.isDefined) } - withClue("no error while getting non-active query") { - intercept[IllegalArgumentException] { - spark.streams.get(q2.name).eq(q2) - } - } - + assert(spark.streams.get(q2.id) === null) assert(spark.streams.active.toSet === Set(q3)) } } @@ -227,7 +214,7 @@ class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { } - /** Run a body of code by defining a query each on multiple datasets */ + /** Run a body of code by defining a query on each dataset */ private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[ContinuousQuery] => Unit): Unit = { failAfter(streamingTimeout) { val queries = withClue("Error starting queries") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala index 55424058f5dae..43a88576cf9f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala @@ -17,15 +17,56 @@ package org.apache.spark.sql.streaming +import org.scalatest.BeforeAndAfter + import org.apache.spark.SparkException import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset, MemoryStream, StreamExecution} +import org.apache.spark.util.Utils -class ContinuousQuerySuite extends StreamTest { +class ContinuousQuerySuite extends StreamTest with BeforeAndAfter { import AwaitTerminationTester._ import testImplicits._ + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("names unique across active queries, ids unique across all started queries") { + val inputData = MemoryStream[Int] + val mapped = inputData.toDS().map { 6 / _} + + def startQuery(queryName: String): ContinuousQuery = { + val metadataRoot = Utils.createTempDir(namePrefix = "streaming.checkpoint").getCanonicalPath + val writer = mapped.write + writer + .queryName(queryName) + .format("memory") + .option("checkpointLocation", metadataRoot) + .startStream() + } + + val q1 = startQuery("q1") + assert(q1.name === "q1") + + // Verify that another query with same name cannot be started + val e1 = intercept[IllegalArgumentException] { + startQuery("q1") + } + Seq("q1", "already active").foreach { s => assert(e1.getMessage.contains(s)) } + + // Verify q1 was unaffected by the above exception and stop it + assert(q1.isActive) + q1.stop() + + // Verify another query can be started with name q1, but will have different id + val q2 = startQuery("q1") + assert(q2.name === "q1") + assert(q2.id !== q1.id) + q2.stop() + } + testQuietly("lifecycle states and awaitTermination") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map { 6 / _} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 7f1e5fe6135a7..cbfa6ff07d131 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -188,8 +188,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { new AssertOnQuery(condition, message) } - def apply(message: String)(condition: StreamExecution => Boolean): AssertOnQuery = { - new AssertOnQuery(condition, message) + def apply(message: String)(condition: StreamExecution => Unit): AssertOnQuery = { + new AssertOnQuery(s => { condition; true }, message) } } @@ -305,13 +305,13 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { spark .streams .startQuery( - StreamExecution.nextName, - metadataRoot, + None, + Some(metadataRoot), stream, sink, outputMode, - trigger, - triggerClock) + trigger = trigger, + triggerClock = triggerClock) .asInstanceOf[StreamExecution] currentStream.microBatchThread.setUncaughtExceptionHandler( new UncaughtExceptionHandler { From b148b0364bcf1d0e31c320381227d13a022e6686 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 13 Jun 2016 14:57:35 -0700 Subject: [PATCH 0629/1470] [SPARK-15887][SQL] Bring back the hive-site.xml support for Spark 2.0 ## What changes were proposed in this pull request? Right now, Spark 2.0 does not load hive-site.xml. Based on users' feedback, it seems make sense to still load this conf file. This PR adds a `hadoopConf` API in `SharedState`, which is `sparkContext.hadoopConfiguration` by default. When users are under hive context, `SharedState.hadoopConf` will load hive-site.xml and append its configs to `sparkContext.hadoopConfiguration`. When we need to read hadoop config in spark sql, we should call `SessionState.newHadoopConf`, which contains `sparkContext.hadoopConfiguration`, hive-site.xml and sql configs. ## How was this patch tested? new test in `HiveDataFrameSuite` Author: Wenchen Fan Closes #13611 from cloud-fan/hive-site. (cherry picked from commit c4b1ad020962c42be804d3a1a55171d9b51b01e7) Signed-off-by: Yin Huai --- .../parquet/ParquetFileFormat.scala | 2 +- .../spark/sql/internal/SessionState.scala | 2 +- .../spark/sql/internal/SharedState.scala | 19 ++++++++++++-- sql/core/src/test/resources/hive-site.xml | 26 +++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 4 +++ .../spark/sql/hive/HiveSharedState.scala | 5 ++-- sql/hive/src/test/resources/hive-site.xml | 26 +++++++++++++++++++ .../spark/sql/hive/HiveDataFrameSuite.scala | 5 ++++ 8 files changed, 82 insertions(+), 7 deletions(-) create mode 100644 sql/core/src/test/resources/hive-site.xml create mode 100644 sql/hive/src/test/resources/hive-site.xml diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 2d4bef3f18855..71c16008be8f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -405,7 +405,7 @@ private[sql] class ParquetFileFormat new ParquetOutputWriterFactory( sqlContext.conf, dataSchema, - sqlContext.sparkContext.hadoopConfiguration, + sqlContext.sessionState.newHadoopConf(), options) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index b2db377ec7f8d..b43095041b243 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -49,7 +49,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val conf: SQLConf = new SQLConf def newHadoopConf(): Configuration = { - val hadoopConf = new Configuration(sparkSession.sparkContext.hadoopConfiguration) + val hadoopConf = new Configuration(sparkSession.sharedState.hadoopConf) conf.getAllConfs.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) } hadoopConf } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 0d6f98416bd42..c37f7f12acb55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.internal +import org.apache.hadoop.conf.Configuration + import org.apache.spark.SparkContext import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} -import org.apache.spark.util.MutableURLClassLoader +import org.apache.spark.util.{MutableURLClassLoader, Utils} /** @@ -40,10 +42,23 @@ private[sql] class SharedState(val sparkContext: SparkContext) { */ val listener: SQLListener = createListenerAndUI(sparkContext) + /** + * The base hadoop configuration which is shared among all spark sessions. It is based on the + * default hadoop configuration of Spark, with custom configurations inside `hive-site.xml`. + */ + lazy val hadoopConf: Configuration = { + val conf = new Configuration(sparkContext.hadoopConfiguration) + val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") + if (configFile != null) { + conf.addResource(configFile) + } + conf + } + /** * A catalog that interacts with external systems. */ - lazy val externalCatalog: ExternalCatalog = new InMemoryCatalog(sparkContext.hadoopConfiguration) + lazy val externalCatalog: ExternalCatalog = new InMemoryCatalog(hadoopConf) /** * A classloader used to load all user-added jar. diff --git a/sql/core/src/test/resources/hive-site.xml b/sql/core/src/test/resources/hive-site.xml new file mode 100644 index 0000000000000..17297b3e22a7e --- /dev/null +++ b/sql/core/src/test/resources/hive-site.xml @@ -0,0 +1,26 @@ + + + + + + + hive.in.test + true + Internal marker for test. + + 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 90465b65bdb1c..89f868509965e 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 @@ -2843,4 +2843,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SELECT '$literal' AS DUMMY"), Row(s"$expected") :: Nil) } + + test("SPARK-15887: hive-site.xml should be loaded") { + assert(spark.sessionState.newHadoopConf().get("hive.in.test") == "true") + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala index a0106ee882e76..78b1ecbbea742 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala @@ -45,12 +45,11 @@ private[hive] class HiveSharedState(override val sparkContext: SparkContext) */ // This needs to be a lazy val at here because TestHiveSharedState is overriding it. lazy val metadataHive: HiveClient = { - HiveUtils.newClientForMetadata(sparkContext.conf, sparkContext.hadoopConfiguration) + HiveUtils.newClientForMetadata(sparkContext.conf, hadoopConf) } /** * A catalog that interacts with the Hive metastore. */ - override lazy val externalCatalog = - new HiveExternalCatalog(metadataHive, sparkContext.hadoopConfiguration) + override lazy val externalCatalog = new HiveExternalCatalog(metadataHive, hadoopConf) } diff --git a/sql/hive/src/test/resources/hive-site.xml b/sql/hive/src/test/resources/hive-site.xml new file mode 100644 index 0000000000000..17297b3e22a7e --- /dev/null +++ b/sql/hive/src/test/resources/hive-site.xml @@ -0,0 +1,26 @@ + + + + + + + hive.in.test + true + Internal marker for test. + + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala index 1b31caa76d3a5..23798431e697f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala @@ -29,4 +29,9 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton { spark.sql("drop table usrdb.test") spark.sql("drop schema usrdb") } + + test("SPARK-15887: hive-site.xml should be loaded") { + val hiveClient = spark.sharedState.asInstanceOf[HiveSharedState].metadataHive + assert(hiveClient.getConf("hive.in.test", "") == "true") + } } From 1f3b5a5ac061c0420f30bb1a696f6445aa64b566 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 13 Jun 2016 15:46:50 -0700 Subject: [PATCH 0630/1470] [SPARK-15925][SQL][SPARKR] Replaces registerTempTable with createOrReplaceTempView ## What changes were proposed in this pull request? This PR replaces `registerTempTable` with `createOrReplaceTempView` as a follow-up task of #12945. ## How was this patch tested? Existing SparkR tests. Author: Cheng Lian Closes #13644 from liancheng/spark-15925-temp-view-for-r. (cherry picked from commit ced8d669b359d6465c3bf476af0f68cc4db04a25) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 2 +- R/pkg/R/DataFrame.R | 23 ++++++++++++----------- R/pkg/R/SQLContext.R | 10 +++++----- R/pkg/R/generics.R | 7 +++++-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 15 ++++++++------- 5 files changed, 31 insertions(+), 26 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index ba386da2aa178..a8cf53fd46c2e 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -45,6 +45,7 @@ exportMethods("arrange", "corr", "covar_samp", "covar_pop", + "createOrReplaceTempView", "crosstab", "dapply", "dapplyCollect", @@ -80,7 +81,6 @@ exportMethods("arrange", "persist", "printSchema", "rbind", - "registerTempTable", "rename", "repartition", "sample", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 30a567523fa56..0ff350d44d4b3 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -428,16 +428,17 @@ setMethod("coltypes<-", dataFrame(nx@sdf) }) -#' Register Temporary Table +#' Creates a temporary view using the given name. #' -#' Registers a SparkDataFrame as a Temporary Table in the SQLContext +#' Creates a new temporary view using a SparkDataFrame in the SQLContext. If a +#' temporary view with the same name already exists, replaces it. #' #' @param x A SparkDataFrame -#' @param tableName A character vector containing the name of the table +#' @param viewName A character vector containing the name of the table #' #' @family SparkDataFrame functions -#' @rdname registerTempTable -#' @name registerTempTable +#' @rdname createOrReplaceTempView +#' @name createOrReplaceTempView #' @export #' @examples #'\dontrun{ @@ -445,13 +446,13 @@ setMethod("coltypes<-", #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" #' df <- read.json(path) -#' registerTempTable(df, "json_df") +#' createOrReplaceTempView(df, "json_df") #' new_df <- sql("SELECT * FROM json_df") #'} -setMethod("registerTempTable", - signature(x = "SparkDataFrame", tableName = "character"), - function(x, tableName) { - invisible(callJMethod(x@sdf, "registerTempTable", tableName)) +setMethod("createOrReplaceTempView", + signature(x = "SparkDataFrame", viewName = "character"), + function(x, viewName) { + invisible(callJMethod(x@sdf, "createOrReplaceTempView", viewName)) }) #' insertInto @@ -473,7 +474,7 @@ setMethod("registerTempTable", #' sqlContext <- sparkRSQL.init(sc) #' df <- read.df(path, "parquet") #' df2 <- read.df(path2, "parquet") -#' registerTempTable(df, "table1") +#' createOrReplaceTempView(df, "table1") #' insertInto(df2, "table1", overwrite = TRUE) #'} setMethod("insertInto", diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index e7e9e353f9e82..914b02a47ad67 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -411,7 +411,7 @@ read.text <- function(x, ...) { #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" #' df <- read.json(path) -#' registerTempTable(df, "table") +#' createOrReplaceTempView(df, "table") #' new_df <- sql("SELECT * FROM table") #' } #' @name sql @@ -443,7 +443,7 @@ sql <- function(x, ...) { #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" #' df <- read.json(path) -#' registerTempTable(df, "table") +#' createOrReplaceTempView(df, "table") #' new_df <- tableToDF("table") #' } #' @note since 2.0.0 @@ -529,7 +529,7 @@ tableNames <- function(x, ...) { #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" #' df <- read.json(path) -#' registerTempTable(df, "table") +#' createOrReplaceTempView(df, "table") #' cacheTable("table") #' } #' @name cacheTable @@ -558,7 +558,7 @@ cacheTable <- function(x, ...) { #' sqlContext <- sparkRSQL.init(sc) #' path <- "path/to/file.json" #' df <- read.json(path) -#' registerTempTable(df, "table") +#' createOrReplaceTempView(df, "table") #' uncacheTable("table") #' } #' @name uncacheTable @@ -608,7 +608,7 @@ clearCache <- function() { #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #' df <- read.df(path, "parquet") -#' registerTempTable(df, "table") +#' createOrReplaceTempView(df, "table") #' dropTempTable("table") #' } #' @name dropTempTable diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index f0cde56b133f5..50fc204f998a5 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -547,9 +547,12 @@ setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) #' @export setGeneric("rename", function(x, ...) { standardGeneric("rename") }) -#' @rdname registerTempTable +#' @rdname createOrReplaceTempView #' @export -setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) +setGeneric("createOrReplaceTempView", + function(x, viewName) { + standardGeneric("createOrReplaceTempView") + }) #' @rdname sample #' @export diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 375cb6f58808d..d1ca3b726fe0b 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -445,7 +445,7 @@ test_that("jsonRDD() on a RDD with json string", { test_that("test cache, uncache and clearCache", { df <- read.json(jsonPath) - registerTempTable(df, "table1") + createOrReplaceTempView(df, "table1") cacheTable("table1") uncacheTable("table1") clearCache() @@ -454,16 +454,17 @@ test_that("test cache, uncache and clearCache", { test_that("test tableNames and tables", { df <- read.json(jsonPath) - registerTempTable(df, "table1") + createOrReplaceTempView(df, "table1") expect_equal(length(tableNames()), 1) df <- tables() expect_equal(count(df), 1) dropTempTable("table1") }) -test_that("registerTempTable() results in a queryable table and sql() results in a new DataFrame", { +test_that( + "createOrReplaceTempView() results in a queryable table and sql() results in a new DataFrame", { df <- read.json(jsonPath) - registerTempTable(df, "table1") + createOrReplaceTempView(df, "table1") newdf <- sql("SELECT * FROM table1 where name = 'Michael'") expect_is(newdf, "SparkDataFrame") expect_equal(count(newdf), 1) @@ -484,13 +485,13 @@ test_that("insertInto() on a registered table", { write.df(df2, parquetPath2, "parquet", "overwrite") dfParquet2 <- read.df(parquetPath2, "parquet") - registerTempTable(dfParquet, "table1") + createOrReplaceTempView(dfParquet, "table1") insertInto(dfParquet2, "table1") expect_equal(count(sql("select * from table1")), 5) expect_equal(first(sql("select * from table1 order by age"))$name, "Michael") dropTempTable("table1") - registerTempTable(dfParquet, "table1") + createOrReplaceTempView(dfParquet, "table1") insertInto(dfParquet2, "table1", overwrite = TRUE) expect_equal(count(sql("select * from table1")), 2) expect_equal(first(sql("select * from table1 order by age"))$name, "Bob") @@ -502,7 +503,7 @@ test_that("insertInto() on a registered table", { test_that("tableToDF() returns a new DataFrame", { df <- read.json(jsonPath) - registerTempTable(df, "table1") + createOrReplaceTempView(df, "table1") tabledf <- tableToDF("table1") expect_is(tabledf, "SparkDataFrame") expect_equal(count(tabledf), 3) From ab9a304a7c690c748d3d99f1d933116e25ea0a73 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 13 Jun 2016 17:06:22 -0700 Subject: [PATCH 0631/1470] [SPARK-15929] Fix portability of DataFrameSuite path globbing tests The DataFrameSuite regression tests for SPARK-13774 fail in my environment because they attempt to glob over all of `/mnt` and some of the subdirectories restrictive permissions which cause the test to fail. This patch rewrites those tests to remove all environment-specific assumptions; the tests now create their own unique temporary paths for use in the tests. Author: Josh Rosen Closes #13649 from JoshRosen/SPARK-15929. (cherry picked from commit a6babca1bf76e70488ce6005ec3b8b53afc7edfd) Signed-off-by: Cheng Lian --- .../org/apache/spark/sql/DataFrameSuite.scala | 45 +++++++++++++++---- 1 file changed, 36 insertions(+), 9 deletions(-) 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 6bb0ce95c4bc5..c8a0f7134d5dd 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql import java.io.File import java.nio.charset.StandardCharsets +import java.util.UUID import scala.language.postfixOps import scala.util.Random @@ -35,6 +36,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} import org.apache.spark.sql.test.SQLTestData.TestData2 import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils class DataFrameSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -1495,18 +1497,43 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-13774: Check error message for non existent path without globbed paths") { - val e = intercept[AnalysisException] (spark.read.format("csv"). - load("/xyz/file2", "/xyz/file21", "/abc/files555", "a")).getMessage() - assert(e.startsWith("Path does not exist")) + val uuid = UUID.randomUUID().toString + val baseDir = Utils.createTempDir() + try { + val e = intercept[AnalysisException] { + spark.read.format("csv").load( + new File(baseDir, "file").getAbsolutePath, + new File(baseDir, "file2").getAbsolutePath, + new File(uuid, "file3").getAbsolutePath, + uuid).rdd + } + assert(e.getMessage.startsWith("Path does not exist")) + } finally { + + } + } test("SPARK-13774: Check error message for not existent globbed paths") { - val e = intercept[AnalysisException] (spark.read.format("text"). - load( "/xyz/*")).getMessage() - assert(e.startsWith("Path does not exist")) + // Non-existent initial path component: + val nonExistentBasePath = "/" + UUID.randomUUID().toString + assert(!new File(nonExistentBasePath).exists()) + val e = intercept[AnalysisException] { + spark.read.format("text").load(s"$nonExistentBasePath/*") + } + assert(e.getMessage.startsWith("Path does not exist")) - val e1 = intercept[AnalysisException] (spark.read.json("/mnt/*/*-xyz.json").rdd). - getMessage() - assert(e1.startsWith("Path does not exist")) + // Existent initial path component, but no matching files: + val baseDir = Utils.createTempDir() + val childDir = Utils.createTempDir(baseDir.getAbsolutePath) + assert(childDir.exists()) + try { + val e1 = intercept[AnalysisException] { + spark.read.json(s"${baseDir.getAbsolutePath}/*/*-xyz.json").rdd + } + assert(e1.getMessage.startsWith("Path does not exist")) + } finally { + Utils.deleteRecursively(baseDir) + } } } From 774014250eb9e73f05387d73f48ee7c89e9a89c3 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Mon, 13 Jun 2016 17:43:55 -0700 Subject: [PATCH 0632/1470] [SPARK-15910][SQL] Check schema consistency when using Kryo encoder to convert DataFrame to Dataset ## What changes were proposed in this pull request? This PR enforces schema check when converting DataFrame to Dataset using Kryo encoder. For example. **Before the change:** Schema is NOT checked when converting DataFrame to Dataset using kryo encoder. ``` scala> case class B(b: Int) scala> implicit val encoder = Encoders.kryo[B] scala> val df = Seq((1)).toDF("b") scala> val ds = df.as[B] // Schema compatibility is NOT checked ``` **After the change:** Report AnalysisException since the schema is NOT compatible. ``` scala> val ds = Seq((1)).toDF("b").as[B] org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(`b` AS BINARY)' due to data type mismatch: cannot cast IntegerType to BinaryType; ... ``` ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13632 from clockfly/spark-15910. (cherry picked from commit 7b9071eeaa62fd9a51d9e94cfd479224b8341517) Signed-off-by: Wenchen Fan --- .../src/main/scala/org/apache/spark/sql/Encoders.scala | 6 ++++-- .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 9 +++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index 673c587b18325..e72f67c48a296 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -25,8 +25,8 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Cast} import org.apache.spark.sql.catalyst.expressions.objects.{DecodeUsingSerializer, EncodeUsingSerializer} -import org.apache.spark.sql.catalyst.expressions.BoundReference import org.apache.spark.sql.types._ /** @@ -209,7 +209,9 @@ object Encoders { BoundReference(0, ObjectType(classOf[AnyRef]), nullable = true), kryo = useKryo)), deserializer = DecodeUsingSerializer[T]( - GetColumnByOrdinal(0, BinaryType), classTag[T], kryo = useKryo), + Cast(GetColumnByOrdinal(0, BinaryType), BinaryType), + classTag[T], + kryo = useKryo), clsTag = classTag[T] ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 96d85f12e87bf..f02a3141a050a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -453,6 +453,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { (KryoData(2), KryoData(2)))) } + test("Kryo encoder: check the schema mismatch when converting DataFrame to Dataset") { + implicit val kryoEncoder = Encoders.kryo[KryoData] + val df = Seq((1)).toDF("a") + val e = intercept[AnalysisException] { + df.as[KryoData] + }.message + assert(e.contains("cannot cast IntegerType to BinaryType")) + } + test("Java encoder") { implicit val kryoEncoder = Encoders.javaSerialization[JavaData] val ds = Seq(JavaData(1), JavaData(2)).toDS() From 55c1fac212d239752d48729a604c2f4aa3784d67 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 13 Jun 2016 19:31:40 -0700 Subject: [PATCH 0633/1470] [SPARK-15808][SQL] File Format Checking When Appending Data #### What changes were proposed in this pull request? **Issue:** Got wrong results or strange errors when append data to a table with mismatched file format. _Example 1: PARQUET -> CSV_ ```Scala createDF(0, 9).write.format("parquet").saveAsTable("appendParquetToOrc") createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendParquetToOrc") ``` Error we got: ``` Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, localhost): java.lang.RuntimeException: file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-bc8fedf2-aa6a-4002-a18b-524c6ac859d4/appendorctoparquet/part-r-00000-c0e3f365-1d46-4df5-a82c-b47d7af9feb9.snappy.orc is not a Parquet file. expected magic number at tail [80, 65, 82, 49] but found [79, 82, 67, 23] ``` _Example 2: Json -> CSV_ ```Scala createDF(0, 9).write.format("json").saveAsTable("appendJsonToCSV") createDF(10, 19).write.mode(SaveMode.Append).format("parquet").saveAsTable("appendJsonToCSV") ``` No exception, but wrong results: ``` +----+----+ | c1| c2| +----+----+ |null|null| |null|null| |null|null| |null|null| | 0|str0| | 1|str1| | 2|str2| | 3|str3| | 4|str4| | 5|str5| | 6|str6| | 7|str7| | 8|str8| | 9|str9| +----+----+ ``` _Example 3: Json -> Text_ ```Scala createDF(0, 9).write.format("json").saveAsTable("appendJsonToText") createDF(10, 19).write.mode(SaveMode.Append).format("text").saveAsTable("appendJsonToText") ``` Error we got: ``` Text data source supports only a single column, and you have 2 columns. ``` This PR is to issue an exception with appropriate error messages. #### How was this patch tested? Added test cases. Author: gatorsmile Closes #13546 from gatorsmile/fileFormatCheck. (cherry picked from commit 5827b65e28da168286c771c53a38620d79f5e74f) Signed-off-by: Yin Huai --- .../command/createDataSourceTables.scala | 9 +++ .../sql/hive/MetastoreDataSourcesSuite.scala | 72 +++++++++++++++++++ 2 files changed, 81 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 865e406ce27a6..4918780873bdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -197,6 +197,15 @@ case class CreateDataSourceTableAsSelectCommand( EliminateSubqueryAliases( sessionState.catalog.lookupRelation(tableIdent)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => + // check if the file formats match + l.relation match { + case r: HadoopFsRelation if r.fileFormat.getClass != dataSource.providingClass => + throw new AnalysisException( + s"The file format of the existing table $tableIdent is " + + s"`${r.fileFormat.getClass.getName}`. It doesn't match the specified " + + s"format `$provider`") + case _ => + } if (query.schema.size != l.schema.size) { throw new AnalysisException( s"The column number of the existing schema[${l.schema}] " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 3d8123d3c06d8..b028d49aff58d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -891,6 +891,78 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } } + test("append table using different formats") { + def createDF(from: Int, to: Int): DataFrame = { + (from to to).map(i => i -> s"str$i").toDF("c1", "c2") + } + + withTable("appendOrcToParquet") { + createDF(0, 9).write.format("parquet").saveAsTable("appendOrcToParquet") + val e = intercept[AnalysisException] { + createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendOrcToParquet") + } + assert(e.getMessage.contains("The file format of the existing table `appendOrcToParquet` " + + "is `org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat`. " + + "It doesn't match the specified format `orc`")) + } + + withTable("appendParquetToJson") { + createDF(0, 9).write.format("json").saveAsTable("appendParquetToJson") + val e = intercept[AnalysisException] { + createDF(10, 19).write.mode(SaveMode.Append).format("parquet") + .saveAsTable("appendParquetToJson") + } + assert(e.getMessage.contains("The file format of the existing table `appendParquetToJson` " + + "is `org.apache.spark.sql.execution.datasources.json.JsonFileFormat`. " + + "It doesn't match the specified format `parquet`")) + } + + withTable("appendTextToJson") { + createDF(0, 9).write.format("json").saveAsTable("appendTextToJson") + val e = intercept[AnalysisException] { + createDF(10, 19).write.mode(SaveMode.Append).format("text") + .saveAsTable("appendTextToJson") + } + assert(e.getMessage.contains("The file format of the existing table `appendTextToJson` is " + + "`org.apache.spark.sql.execution.datasources.json.JsonFileFormat`. " + + "It doesn't match the specified format `text`")) + } + } + + test("append a table using the same formats but different names") { + def createDF(from: Int, to: Int): DataFrame = { + (from to to).map(i => i -> s"str$i").toDF("c1", "c2") + } + + withTable("appendParquet") { + createDF(0, 9).write.format("parquet").saveAsTable("appendParquet") + createDF(10, 19).write.mode(SaveMode.Append).format("org.apache.spark.sql.parquet") + .saveAsTable("appendParquet") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendParquet p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + + withTable("appendParquet") { + createDF(0, 9).write.format("org.apache.spark.sql.parquet").saveAsTable("appendParquet") + createDF(10, 19).write.mode(SaveMode.Append).format("parquet").saveAsTable("appendParquet") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendParquet p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + + withTable("appendParquet") { + createDF(0, 9).write.format("org.apache.spark.sql.parquet.DefaultSource") + .saveAsTable("appendParquet") + createDF(10, 19).write.mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.parquet.DefaultSource") + .saveAsTable("appendParquet") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendParquet p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + } + test("SPARK-8156:create table to specific database by 'use dbname' ") { val df = (1 to 3).map(i => (i, s"val_$i", i * 2)).toDF("a", "b", "c") From 1a57bf0f4e0fec2fdb250902089403a589ab8795 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 13 Jun 2016 19:59:53 -0700 Subject: [PATCH 0634/1470] [SPARK-15364][ML][PYSPARK] Implement PySpark picklers for ml.Vector and ml.Matrix under spark.ml.python ## What changes were proposed in this pull request? Now we have PySpark picklers for new and old vector/matrix, individually. However, they are all implemented under `PythonMLlibAPI`. To separate spark.mllib from spark.ml, we should implement the picklers of new vector/matrix under `spark.ml.python` instead. ## How was this patch tested? Existing tests. Author: Liang-Chi Hsieh Closes #13219 from viirya/pyspark-pickler-ml. (cherry picked from commit baa3e633e18c47b12e79fe3ddc01fc8ec010f096) Signed-off-by: Xiangrui Meng --- .../org/apache/spark/ml/python/MLSerDe.scala | 224 +++++++++++++ .../mllib/api/python/PythonMLLibAPI.scala | 309 ++++-------------- .../apache/spark/ml/python/MLSerDeSuite.scala | 72 ++++ python/pyspark/java_gateway.py | 1 + python/pyspark/ml/base.py | 2 +- python/pyspark/ml/classification.py | 2 +- python/pyspark/ml/clustering.py | 2 +- python/pyspark/ml/common.py | 137 ++++++++ python/pyspark/ml/evaluation.py | 2 +- python/pyspark/ml/feature.py | 2 +- python/pyspark/ml/pipeline.py | 2 +- python/pyspark/ml/recommendation.py | 2 +- python/pyspark/ml/regression.py | 2 +- python/pyspark/ml/tests.py | 10 +- python/pyspark/ml/tuning.py | 2 +- python/pyspark/ml/util.py | 2 +- python/pyspark/ml/wrapper.py | 2 +- 17 files changed, 518 insertions(+), 257 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/python/MLSerDe.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala create mode 100644 python/pyspark/ml/common.py diff --git a/mllib/src/main/scala/org/apache/spark/ml/python/MLSerDe.scala b/mllib/src/main/scala/org/apache/spark/ml/python/MLSerDe.scala new file mode 100644 index 0000000000000..1279c901c5c9e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/python/MLSerDe.scala @@ -0,0 +1,224 @@ +/* + * 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.python + +import java.io.OutputStream +import java.nio.{ByteBuffer, ByteOrder} +import java.util.{ArrayList => JArrayList} + +import scala.collection.JavaConverters._ + +import net.razorvine.pickle._ + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.ml.linalg._ +import org.apache.spark.mllib.api.python.SerDeBase +import org.apache.spark.rdd.RDD + +/** + * SerDe utility functions for pyspark.ml. + */ +private[spark] object MLSerDe extends SerDeBase with Serializable { + + override val PYSPARK_PACKAGE = "pyspark.ml" + + // Pickler for DenseVector + private[python] class DenseVectorPickler extends BasePickler[DenseVector] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val vector: DenseVector = obj.asInstanceOf[DenseVector] + val bytes = new Array[Byte](8 * vector.size) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + db.put(vector.values) + + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE1) + } + + def construct(args: Array[Object]): Object = { + require(args.length == 1) + if (args.length != 1) { + throw new PickleException("should be 1") + } + val bytes = getBytes(args(0)) + val bb = ByteBuffer.wrap(bytes, 0, bytes.length) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](bytes.length / 8) + db.get(ans) + Vectors.dense(ans) + } + } + + // Pickler for DenseMatrix + private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] + val bytes = new Array[Byte](8 * m.values.length) + val order = ByteOrder.nativeOrder() + val isTransposed = if (m.isTransposed) 1 else 0 + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 4) { + throw new PickleException("should be 4") + } + val bytes = getBytes(args(2)) + val n = bytes.length / 8 + val values = new Array[Double](n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(3).asInstanceOf[Int] == 1 + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) + } + } + + // Pickler for SparseMatrix + private[python] class SparseMatrixPickler extends BasePickler[SparseMatrix] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val s = obj.asInstanceOf[SparseMatrix] + val order = ByteOrder.nativeOrder() + + val colPtrsBytes = new Array[Byte](4 * s.colPtrs.length) + val indicesBytes = new Array[Byte](4 * s.rowIndices.length) + val valuesBytes = new Array[Byte](8 * s.values.length) + val isTransposed = if (s.isTransposed) 1 else 0 + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().put(s.colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().put(s.rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().put(s.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(colPtrsBytes.length)) + out.write(colPtrsBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indicesBytes.length)) + out.write(indicesBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valuesBytes.length)) + out.write(valuesBytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 6) { + throw new PickleException("should be 6") + } + val order = ByteOrder.nativeOrder() + val colPtrsBytes = getBytes(args(2)) + val indicesBytes = getBytes(args(3)) + val valuesBytes = getBytes(args(4)) + val colPtrs = new Array[Int](colPtrsBytes.length / 4) + val rowIndices = new Array[Int](indicesBytes.length / 4) + val values = new Array[Double](valuesBytes.length / 8) + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().get(colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().get(rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(5).asInstanceOf[Int] == 1 + new SparseMatrix( + args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], colPtrs, rowIndices, values, + isTransposed) + } + } + + // Pickler for SparseVector + private[python] class SparseVectorPickler extends BasePickler[SparseVector] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val v: SparseVector = obj.asInstanceOf[SparseVector] + val n = v.indices.length + val indiceBytes = new Array[Byte](4 * n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) + val valueBytes = new Array[Byte](8 * n) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(v.size)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) + out.write(indiceBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valueBytes.length)) + out.write(valueBytes) + out.write(Opcodes.TUPLE3) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + val size = args(0).asInstanceOf[Int] + 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) + if (n > 0) { + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) + } + new SparseVector(size, indices, values) + } + } + + var initialized = false + // This should be called before trying to serialize any above classes + // In cluster mode, this should be put in the closure + override def initialize(): Unit = { + SerDeUtil.initialize() + synchronized { + if (!initialized) { + new DenseVectorPickler().register() + new DenseMatrixPickler().register() + new SparseMatrixPickler().register() + new SparseVectorPickler().register() + initialized = true + } + } + } + // will not called in Executor automatically + initialize() +} 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 e43469bf1cf86..7df61601fb1e9 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 @@ -30,7 +30,6 @@ import net.razorvine.pickle._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil -import org.apache.spark.ml.linalg.{DenseMatrix => NewDenseMatrix, DenseVector => NewDenseVector, SparseMatrix => NewSparseMatrix, SparseVector => NewSparseVector, Vectors => NewVectors} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.evaluation.RankingMetrics @@ -1205,23 +1204,21 @@ private[python] class PythonMLLibAPI extends Serializable { } /** - * SerDe utility functions for PythonMLLibAPI. + * Basic SerDe utility class. */ -private[spark] object SerDe extends Serializable { +private[spark] abstract class SerDeBase { - val PYSPARK_PACKAGE = "pyspark.mllib" - val PYSPARK_ML_PACKAGE = "pyspark.ml" + val PYSPARK_PACKAGE: String + def initialize(): Unit /** * Base class used for pickle */ - private[python] abstract class BasePickler[T: ClassTag] + private[spark] abstract class BasePickler[T: ClassTag] extends IObjectPickler with IObjectConstructor { - protected def packageName: String = PYSPARK_PACKAGE - private val cls = implicitly[ClassTag[T]].runtimeClass - private val module = packageName + "." + cls.getName.split('.')(4) + private val module = PYSPARK_PACKAGE + "." + cls.getName.split('.')(4) private val name = cls.getSimpleName // register this to Pickler and Unpickler @@ -1268,45 +1265,73 @@ private[spark] object SerDe extends Serializable { private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } - // Pickler for (mllib) DenseVector - private[python] class DenseVectorPickler extends BasePickler[DenseVector] { + def dumps(obj: AnyRef): Array[Byte] = { + obj match { + // Pickler in Python side cannot deserialize Scala Array normally. See SPARK-12834. + case array: Array[_] => new Pickler().dumps(array.toSeq.asJava) + case _ => new Pickler().dumps(obj) + } + } - def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { - val vector: DenseVector = obj.asInstanceOf[DenseVector] - val bytes = new Array[Byte](8 * vector.size) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - val db = bb.asDoubleBuffer() - db.put(vector.values) + def loads(bytes: Array[Byte]): AnyRef = { + new Unpickler().loads(bytes) + } - out.write(Opcodes.BINSTRING) - out.write(PickleUtils.integer_to_bytes(bytes.length)) - out.write(bytes) - out.write(Opcodes.TUPLE1) + /* convert object into Tuple */ + def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { + rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) + } + + /* convert RDD[Tuple2[,]] to RDD[Array[Any]] */ + def fromTuple2RDD(rdd: RDD[(Any, Any)]): RDD[Array[Any]] = { + rdd.map(x => Array(x._1, x._2)) + } + + /** + * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + new SerDeUtil.AutoBatchedPickler(iter) } + } - def construct(args: Array[Object]): Object = { - require(args.length == 1) - if (args.length != 1) { - throw new PickleException("should be 1") + /** + * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. + */ + def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { + pyRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj match { + case list: JArrayList[_] => list.asScala + case arr: Array[_] => arr + } + } else { + Seq(obj) + } } - val bytes = getBytes(args(0)) - val bb = ByteBuffer.wrap(bytes, 0, bytes.length) - bb.order(ByteOrder.nativeOrder()) - val db = bb.asDoubleBuffer() - val ans = new Array[Double](bytes.length / 8) - db.get(ans) - Vectors.dense(ans) - } + }.toJavaRDD() } +} - // Pickler for (new) DenseVector - private[python] class NewDenseVectorPickler extends BasePickler[NewDenseVector] { +/** + * SerDe utility functions for PythonMLLibAPI. + */ +private[spark] object SerDe extends SerDeBase with Serializable { + + override val PYSPARK_PACKAGE = "pyspark.mllib" - override protected def packageName = PYSPARK_ML_PACKAGE + // Pickler for DenseVector + private[python] class DenseVectorPickler extends BasePickler[DenseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { - val vector: NewDenseVector = obj.asInstanceOf[NewDenseVector] + val vector: DenseVector = obj.asInstanceOf[DenseVector] val bytes = new Array[Byte](8 * vector.size) val bb = ByteBuffer.wrap(bytes) bb.order(ByteOrder.nativeOrder()) @@ -1330,11 +1355,11 @@ private[spark] object SerDe extends Serializable { val db = bb.asDoubleBuffer() val ans = new Array[Double](bytes.length / 8) db.get(ans) - NewVectors.dense(ans) + Vectors.dense(ans) } } - // Pickler for (mllib) DenseMatrix + // Pickler for DenseMatrix private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1371,46 +1396,7 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for (new) DenseMatrix - private[python] class NewDenseMatrixPickler extends BasePickler[NewDenseMatrix] { - - override protected def packageName = PYSPARK_ML_PACKAGE - - def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { - val m: NewDenseMatrix = obj.asInstanceOf[NewDenseMatrix] - val bytes = new Array[Byte](8 * m.values.length) - val order = ByteOrder.nativeOrder() - val isTransposed = if (m.isTransposed) 1 else 0 - ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) - - out.write(Opcodes.MARK) - out.write(Opcodes.BININT) - out.write(PickleUtils.integer_to_bytes(m.numRows)) - out.write(Opcodes.BININT) - out.write(PickleUtils.integer_to_bytes(m.numCols)) - out.write(Opcodes.BINSTRING) - out.write(PickleUtils.integer_to_bytes(bytes.length)) - out.write(bytes) - out.write(Opcodes.BININT) - out.write(PickleUtils.integer_to_bytes(isTransposed)) - out.write(Opcodes.TUPLE) - } - - def construct(args: Array[Object]): Object = { - if (args.length != 4) { - throw new PickleException("should be 4") - } - val bytes = getBytes(args(2)) - val n = bytes.length / 8 - val values = new Array[Double](n) - val order = ByteOrder.nativeOrder() - ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) - val isTransposed = args(3).asInstanceOf[Int] == 1 - new NewDenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) - } - } - - // Pickler for (mllib) SparseMatrix + // Pickler for SparseMatrix private[python] class SparseMatrixPickler extends BasePickler[SparseMatrix] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1465,64 +1451,7 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for (new) SparseMatrix - private[python] class NewSparseMatrixPickler extends BasePickler[NewSparseMatrix] { - - override protected def packageName = PYSPARK_ML_PACKAGE - - def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { - val s = obj.asInstanceOf[NewSparseMatrix] - val order = ByteOrder.nativeOrder() - - val colPtrsBytes = new Array[Byte](4 * s.colPtrs.length) - val indicesBytes = new Array[Byte](4 * s.rowIndices.length) - val valuesBytes = new Array[Byte](8 * s.values.length) - val isTransposed = if (s.isTransposed) 1 else 0 - ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().put(s.colPtrs) - ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().put(s.rowIndices) - ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().put(s.values) - - out.write(Opcodes.MARK) - out.write(Opcodes.BININT) - out.write(PickleUtils.integer_to_bytes(s.numRows)) - out.write(Opcodes.BININT) - out.write(PickleUtils.integer_to_bytes(s.numCols)) - out.write(Opcodes.BINSTRING) - out.write(PickleUtils.integer_to_bytes(colPtrsBytes.length)) - out.write(colPtrsBytes) - out.write(Opcodes.BINSTRING) - out.write(PickleUtils.integer_to_bytes(indicesBytes.length)) - out.write(indicesBytes) - out.write(Opcodes.BINSTRING) - out.write(PickleUtils.integer_to_bytes(valuesBytes.length)) - out.write(valuesBytes) - out.write(Opcodes.BININT) - out.write(PickleUtils.integer_to_bytes(isTransposed)) - out.write(Opcodes.TUPLE) - } - - def construct(args: Array[Object]): Object = { - if (args.length != 6) { - throw new PickleException("should be 6") - } - val order = ByteOrder.nativeOrder() - val colPtrsBytes = getBytes(args(2)) - val indicesBytes = getBytes(args(3)) - val valuesBytes = getBytes(args(4)) - val colPtrs = new Array[Int](colPtrsBytes.length / 4) - val rowIndices = new Array[Int](indicesBytes.length / 4) - val values = new Array[Double](valuesBytes.length / 8) - ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().get(colPtrs) - ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().get(rowIndices) - ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().get(values) - val isTransposed = args(5).asInstanceOf[Int] == 1 - new NewSparseMatrix( - args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], colPtrs, rowIndices, values, - isTransposed) - } - } - - // Pickler for (mllib) SparseVector + // Pickler for SparseVector private[python] class SparseVectorPickler extends BasePickler[SparseVector] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1564,50 +1493,6 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for (new) SparseVector - private[python] class NewSparseVectorPickler extends BasePickler[NewSparseVector] { - - override protected def packageName = PYSPARK_ML_PACKAGE - - def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { - val v: NewSparseVector = obj.asInstanceOf[NewSparseVector] - val n = v.indices.length - val indiceBytes = new Array[Byte](4 * n) - val order = ByteOrder.nativeOrder() - ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) - val valueBytes = new Array[Byte](8 * n) - ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) - - out.write(Opcodes.BININT) - out.write(PickleUtils.integer_to_bytes(v.size)) - out.write(Opcodes.BINSTRING) - out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) - out.write(indiceBytes) - out.write(Opcodes.BINSTRING) - out.write(PickleUtils.integer_to_bytes(valueBytes.length)) - out.write(valueBytes) - out.write(Opcodes.TUPLE3) - } - - def construct(args: Array[Object]): Object = { - if (args.length != 3) { - throw new PickleException("should be 3") - } - val size = args(0).asInstanceOf[Int] - 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) - if (n > 0) { - val order = ByteOrder.nativeOrder() - ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) - ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) - } - new NewSparseVector(size, indices, values) - } - } - // Pickler for MLlib LabeledPoint private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { @@ -1654,7 +1539,7 @@ private[spark] object SerDe extends Serializable { var initialized = false // This should be called before trying to serialize any above classes // In cluster mode, this should be put in the closure - def initialize(): Unit = { + override def initialize(): Unit = { SerDeUtil.initialize() synchronized { if (!initialized) { @@ -1662,10 +1547,6 @@ private[spark] object SerDe extends Serializable { new DenseMatrixPickler().register() new SparseMatrixPickler().register() new SparseVectorPickler().register() - new NewDenseVectorPickler().register() - new NewDenseMatrixPickler().register() - new NewSparseMatrixPickler().register() - new NewSparseVectorPickler().register() new LabeledPointPickler().register() new RatingPickler().register() initialized = true @@ -1674,58 +1555,4 @@ private[spark] object SerDe extends Serializable { } // will not called in Executor automatically initialize() - - def dumps(obj: AnyRef): Array[Byte] = { - obj match { - // Pickler in Python side cannot deserialize Scala Array normally. See SPARK-12834. - case array: Array[_] => new Pickler().dumps(array.toSeq.asJava) - case _ => new Pickler().dumps(obj) - } - } - - def loads(bytes: Array[Byte]): AnyRef = { - new Unpickler().loads(bytes) - } - - /* convert object into Tuple */ - def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { - rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) - } - - /* convert RDD[Tuple2[,]] to RDD[Array[Any]] */ - def fromTuple2RDD(rdd: RDD[(Any, Any)]): RDD[Array[Any]] = { - rdd.map(x => Array(x._1, x._2)) - } - - /** - * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by - * PySpark. - */ - def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { - jRDD.rdd.mapPartitions { iter => - initialize() // let it called in executor - new SerDeUtil.AutoBatchedPickler(iter) - } - } - - /** - * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. - */ - def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { - pyRDD.rdd.mapPartitions { iter => - initialize() // let it called in executor - val unpickle = new Unpickler - iter.flatMap { row => - val obj = unpickle.loads(row) - if (batched) { - obj match { - case list: JArrayList[_] => list.asScala - case arr: Array[_] => arr - } - } else { - Seq(obj) - } - } - }.toJavaRDD() - } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala new file mode 100644 index 0000000000000..5eaef9aabda50 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala @@ -0,0 +1,72 @@ +/* + * 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.python + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, SparseMatrix, Vectors} + +class MLSerDeSuite extends SparkFunSuite { + + MLSerDe.initialize() + + test("pickle vector") { + val vectors = Seq( + Vectors.dense(Array.empty[Double]), + Vectors.dense(0.0), + Vectors.dense(0.0, -2.0), + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]), + Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), + Vectors.sparse(2, Array(1), Array(-2.0))) + vectors.foreach { v => + val u = MLSerDe.loads(MLSerDe.dumps(v)) + assert(u.getClass === v.getClass) + assert(u === v) + } + } + + test("pickle double") { + for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { + val deser = MLSerDe.loads(MLSerDe.dumps(x.asInstanceOf[AnyRef])).asInstanceOf[Double] + // We use `equals` here for comparison because we cannot use `==` for NaN + assert(x.equals(deser)) + } + } + + test("pickle matrix") { + val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) + val matrix = Matrices.dense(2, 3, values) + val nm = MLSerDe.loads(MLSerDe.dumps(matrix)).asInstanceOf[DenseMatrix] + assert(matrix === nm) + + // Test conversion for empty matrix + val empty = Array[Double]() + val emptyMatrix = Matrices.dense(0, 0, empty) + val ne = MLSerDe.loads(MLSerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] + assert(emptyMatrix == ne) + + val sm = new SparseMatrix(3, 2, Array(0, 1, 3), Array(1, 0, 2), Array(0.9, 1.2, 3.4)) + val nsm = MLSerDe.loads(MLSerDe.dumps(sm)).asInstanceOf[SparseMatrix] + assert(sm.toArray === nsm.toArray) + + val smt = new SparseMatrix( + 3, 3, Array(0, 2, 3, 5), Array(0, 2, 1, 0, 2), Array(0.9, 1.2, 3.4, 5.7, 8.9), + isTransposed = true) + val nsmt = MLSerDe.loads(MLSerDe.dumps(smt)).asInstanceOf[SparseMatrix] + assert(smt.toArray === nsmt.toArray) + } +} diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index cd4c55f79f18c..527ca82d31f1b 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -116,6 +116,7 @@ def killChild(): java_import(gateway.jvm, "org.apache.spark.SparkConf") java_import(gateway.jvm, "org.apache.spark.api.java.*") java_import(gateway.jvm, "org.apache.spark.api.python.*") + java_import(gateway.jvm, "org.apache.spark.ml.python.*") java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*") # TODO(davies): move into sql java_import(gateway.jvm, "org.apache.spark.sql.*") diff --git a/python/pyspark/ml/base.py b/python/pyspark/ml/base.py index a7a58e17a43ed..339e5d6af52a7 100644 --- a/python/pyspark/ml/base.py +++ b/python/pyspark/ml/base.py @@ -19,7 +19,7 @@ from pyspark import since from pyspark.ml.param import Params -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc @inherit_doc diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 77badebeb418e..121b9262dd9de 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -26,7 +26,7 @@ from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams from pyspark.ml.wrapper import JavaWrapper -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc from pyspark.sql import DataFrame from pyspark.sql.functions import udf, when from pyspark.sql.types import ArrayType, DoubleType diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 92df19e804374..75d9a0e8cac18 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -19,7 +19,7 @@ from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel from pyspark.ml.param.shared import * -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc __all__ = ['BisectingKMeans', 'BisectingKMeansModel', 'KMeans', 'KMeansModel', diff --git a/python/pyspark/ml/common.py b/python/pyspark/ml/common.py new file mode 100644 index 0000000000000..256e91e14165e --- /dev/null +++ b/python/pyspark/ml/common.py @@ -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. +# + +import sys +if sys.version >= '3': + long = int + unicode = str + +import py4j.protocol +from py4j.protocol import Py4JJavaError +from py4j.java_gateway import JavaObject +from py4j.java_collections import ListConverter, JavaArray, JavaList + +from pyspark import RDD, SparkContext +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer +from pyspark.sql import DataFrame, SQLContext + +# Hack for support float('inf') in Py4j +_old_smart_decode = py4j.protocol.smart_decode + +_float_str_mapping = { + 'nan': 'NaN', + 'inf': 'Infinity', + '-inf': '-Infinity', +} + + +def _new_smart_decode(obj): + if isinstance(obj, float): + s = str(obj) + return _float_str_mapping.get(s, s) + return _old_smart_decode(obj) + +py4j.protocol.smart_decode = _new_smart_decode + + +_picklable_classes = [ + 'SparseVector', + 'DenseVector', + 'DenseMatrix', +] + + +# this will call the ML version of pythonToJava() +def _to_java_object_rdd(rdd): + """ Return an JavaRDD of Object by unpickling + + It will convert each Python object into Java object by Pyrolite, whenever the + RDD is serialized in batch or not. + """ + rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) + return rdd.ctx._jvm.MLSerDe.pythonToJava(rdd._jrdd, True) + + +def _py2java(sc, obj): + """ Convert Python object into Java """ + if isinstance(obj, RDD): + obj = _to_java_object_rdd(obj) + elif isinstance(obj, DataFrame): + obj = obj._jdf + elif isinstance(obj, SparkContext): + obj = obj._jsc + elif isinstance(obj, list): + 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, bytes, unicode)): + pass + else: + data = bytearray(PickleSerializer().dumps(obj)) + obj = sc._jvm.MLSerDe.loads(data) + return obj + + +def _java2py(sc, r, encoding="bytes"): + if isinstance(r, JavaObject): + clsName = r.getClass().getSimpleName() + # convert RDD into JavaRDD + if clsName != 'JavaRDD' and clsName.endswith("RDD"): + r = r.toJavaRDD() + clsName = 'JavaRDD' + + if clsName == 'JavaRDD': + jrdd = sc._jvm.MLSerDe.javaToPython(r) + return RDD(jrdd, sc) + + if clsName == 'Dataset': + return DataFrame(r, SQLContext.getOrCreate(sc)) + + if clsName in _picklable_classes: + r = sc._jvm.MLSerDe.dumps(r) + elif isinstance(r, (JavaArray, JavaList)): + try: + r = sc._jvm.MLSerDe.dumps(r) + except Py4JJavaError: + pass # not pickable + + if isinstance(r, (bytearray, bytes)): + r = PickleSerializer().loads(bytes(r), encoding=encoding) + return r + + +def callJavaFunc(sc, func, *args): + """ Call Java Function """ + args = [_py2java(sc, a) for a in args] + return _java2py(sc, func(*args)) + + +def inherit_doc(cls): + """ + A decorator that makes a class inherit documentation from its parents. + """ + for name, func in vars(cls).items(): + # only inherit docstring for public functions + if name.startswith("_"): + continue + if not func.__doc__: + for parent in cls.__bases__: + parent_func = getattr(parent, name, None) + if parent_func and getattr(parent_func, "__doc__", None): + func.__doc__ = parent_func.__doc__ + break + return cls diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index cd071f1b7c7ef..1fe8772da772a 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -21,7 +21,7 @@ from pyspark.ml.wrapper import JavaParams from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasLabelCol, HasPredictionCol, HasRawPredictionCol -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc __all__ = ['Evaluator', 'BinaryClassificationEvaluator', 'RegressionEvaluator', 'MulticlassClassificationEvaluator'] diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index ca77ac395dcca..a28764a7528dc 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -25,7 +25,7 @@ from pyspark.ml.param.shared import * from pyspark.ml.util import JavaMLReadable, JavaMLWritable from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer, _jvm -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc __all__ = ['Binarizer', 'Bucketizer', diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 0777527134ccb..a48f4bb2ad1ba 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -25,7 +25,7 @@ from pyspark.ml.param import Param, Params from pyspark.ml.util import JavaMLWriter, JavaMLReader, MLReadable, MLWritable from pyspark.ml.wrapper import JavaParams -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc @inherit_doc diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 1778bfe938b1e..0a7096794d25e 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -19,7 +19,7 @@ from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel from pyspark.ml.param.shared import * -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc __all__ = ['ALS', 'ALSModel'] diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 7c79ab73c7e76..db31993f0fb70 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -21,7 +21,7 @@ from pyspark.ml.param.shared import * from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaWrapper -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc from pyspark.sql import DataFrame diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 4358175a57e63..981ed9dda042c 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -61,7 +61,7 @@ GeneralizedLinearRegression from pyspark.ml.tuning import * from pyspark.ml.wrapper import JavaParams -from pyspark.mllib.common import _java2py +from pyspark.ml.common import _java2py from pyspark.serializers import PickleSerializer from pyspark.sql import DataFrame, Row, SparkSession from pyspark.sql.functions import rand @@ -1195,12 +1195,12 @@ class VectorTests(MLlibTestCase): 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(bytes(self.sc._jvm.SerDe.dumps(jvec))) + jvec = self.sc._jvm.MLSerDe.loads(bytearray(ser.dumps(v))) + nv = ser.loads(bytes(self.sc._jvm.MLSerDe.dumps(jvec))) self.assertEqual(v, nv) vs = [v] * 100 - jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) - nvs = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvecs))) + jvecs = self.sc._jvm.MLSerDe.loads(bytearray(ser.dumps(vs))) + nvs = ser.loads(bytes(self.sc._jvm.MLSerDe.dumps(jvecs))) self.assertEqual(vs, nvs) def test_serialize(self): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index fe87b6cdb9369..f857c5e8c86b6 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -25,7 +25,7 @@ from pyspark.ml.param.shared import HasSeed from pyspark.ml.wrapper import JavaParams from pyspark.sql.functions import rand -from pyspark.mllib.common import inherit_doc, _py2java +from pyspark.ml.common import inherit_doc, _py2java __all__ = ['ParamGridBuilder', 'CrossValidator', 'CrossValidatorModel', 'TrainValidationSplit', 'TrainValidationSplitModel'] diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 9d28823196216..4a31a298096fc 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -23,7 +23,7 @@ unicode = str from pyspark import SparkContext, since -from pyspark.mllib.common import inherit_doc +from pyspark.ml.common import inherit_doc def _jvm(): diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index fef0040faf86e..25c44b7533c77 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -22,7 +22,7 @@ from pyspark.ml import Estimator, Transformer, Model from pyspark.ml.param import Params from pyspark.ml.util import _jvm -from pyspark.mllib.common import inherit_doc, _java2py, _py2java +from pyspark.ml.common import inherit_doc, _java2py, _py2java class JavaWrapper(object): From 2841bbac46c6125d2abda743ac53bcab5037013d Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Mon, 13 Jun 2016 21:58:52 -0700 Subject: [PATCH 0635/1470] [SPARK-15663][SQL] SparkSession.catalog.listFunctions shouldn't include the list of built-in functions ## What changes were proposed in this pull request? SparkSession.catalog.listFunctions currently returns all functions, including the list of built-in functions. This makes the method not as useful because anytime it is run the result set contains over 100 built-in functions. ## How was this patch tested? CatalogSuite Author: Sandeep Singh Closes #13413 from techaddict/SPARK-15663. (cherry picked from commit 1842cdd4ee9f30b0a5f579e26ff5194e81e3634c) Signed-off-by: Yin Huai --- python/pyspark/sql/tests.py | 12 +---- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 31 +++++++++++- .../spark/sql/internal/CatalogSuite.scala | 6 +-- .../sql/hive/execution/SQLQuerySuite.scala | 48 ++++++++++++------- 6 files changed, 67 insertions(+), 35 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 0d9dd5ea2a364..e0acde678317d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1481,17 +1481,7 @@ def test_list_functions(self): spark.sql("CREATE DATABASE some_db") functions = dict((f.name, f) for f in spark.catalog.listFunctions()) functionsDefault = dict((f.name, f) for f in spark.catalog.listFunctions("default")) - self.assertTrue(len(functions) > 200) - self.assertTrue("+" in functions) - self.assertTrue("like" in functions) - self.assertTrue("month" in functions) - self.assertTrue("to_unix_timestamp" in functions) - self.assertTrue("current_database" in functions) - self.assertEquals(functions["+"], Function( - name="+", - description=None, - className="org.apache.spark.sql.catalyst.expressions.Add", - isTemporary=True)) + self.assertEquals(len(functions), 0) self.assertEquals(functions, functionsDefault) spark.catalog.registerFunction("temp_func", lambda x: str(x)) spark.sql("CREATE FUNCTION func1 AS 'org.apache.spark.data.bricks'") 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 a7388c71de9e0..42a8faa412a34 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 @@ -396,6 +396,8 @@ object FunctionRegistry { fr } + val functionSet: Set[String] = builtin.listFunction().toSet + /** See usage above. */ private def expression[T <: Expression](name: String) (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 943d1071e295a..1ec1bb1baf23b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -855,7 +855,8 @@ class SessionCatalog( .map { f => FunctionIdentifier(f, Some(dbName)) } val loadedFunctions = StringUtils.filterPattern(functionRegistry.listFunction(), pattern) .map { f => FunctionIdentifier(f) } - dbFunctions ++ loadedFunctions + (dbFunctions ++ loadedFunctions) + .filterNot(f => FunctionRegistry.functionSet.contains(f.funcName)) } 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 89f868509965e..545c1776b7e2a 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 @@ -21,8 +21,10 @@ import java.math.MathContext import java.sql.Timestamp import org.apache.spark.AccumulatorSuite +import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.expressions.SortOrder +import org.apache.spark.sql.catalyst.catalog.{CatalogTestUtils, ExternalCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.expressions.{ExpressionDescription, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.aggregate @@ -58,15 +60,40 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("show functions") { def getFunctions(pattern: String): Seq[Row] = { - StringUtils.filterPattern(spark.sessionState.functionRegistry.listFunction(), pattern) + StringUtils.filterPattern( + spark.sessionState.catalog.listFunctions("default").map(_.funcName), pattern) .map(Row(_)) } + + def createFunction(names: Seq[String]): Unit = { + names.foreach { name => + spark.udf.register(name, (arg1: Int, arg2: String) => arg2 + arg1) + } + } + + def dropFunction(names: Seq[String]): Unit = { + names.foreach { name => + spark.sessionState.catalog.dropTempFunction(name, false) + } + } + + val functions = Array("ilog", "logi", "logii", "logiii", "crc32i", "cubei", "cume_disti", + "isize", "ispace", "to_datei", "date_addi", "current_datei") + + assert(sql("SHOW functions").collect().isEmpty) + + createFunction(functions) + checkAnswer(sql("SHOW functions"), getFunctions("*")) + assert(sql("SHOW functions").collect().size === functions.size) + assert(sql("SHOW functions").collect().toSet === functions.map(Row(_)).toSet) + Seq("^c*", "*e$", "log*", "*date*").foreach { pattern => // For the pattern part, only '*' and '|' are allowed as wildcards. // For '*', we need to replace it to '.*'. checkAnswer(sql(s"SHOW FUNCTIONS '$pattern'"), getFunctions(pattern)) } + dropFunction(functions) } test("describe functions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index aec0312c4003e..df817f863dbe2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -174,8 +174,7 @@ class CatalogSuite } test("list functions") { - assert(Set("+", "current_database", "window").subsetOf( - spark.catalog.listFunctions().collect().map(_.name).toSet)) + assert(spark.catalog.listFunctions().collect().isEmpty) createFunction("my_func1") createFunction("my_func2") createTempFunction("my_temp_func") @@ -192,8 +191,7 @@ class CatalogSuite } test("list functions with database") { - assert(Set("+", "current_database", "window").subsetOf( - spark.catalog.listFunctions("default").collect().map(_.name).toSet)) + assert(spark.catalog.listFunctions("default").collect().isEmpty) createDatabase("my_db1") createDatabase("my_db2") createFunction("my_func1", Some("my_db1")) 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 8244ff4ce0552..1a0eaa66c1391 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 @@ -187,28 +187,42 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("show functions") { - val allBuiltinFunctions = FunctionRegistry.builtin.listFunction().toSet[String].toList.sorted - // The TestContext is shared by all the test cases, some functions may be registered before - // this, so we check that all the builtin functions are returned. - val allFunctions = sql("SHOW functions").collect().map(r => r(0)) - allBuiltinFunctions.foreach { f => - assert(allFunctions.contains(f)) - } withTempDatabase { db => - checkAnswer(sql("SHOW functions abs"), Row("abs")) - checkAnswer(sql("SHOW functions 'abs'"), Row("abs")) - checkAnswer(sql(s"SHOW functions $db.abs"), Row("abs")) - checkAnswer(sql(s"SHOW functions `$db`.`abs`"), Row("abs")) - checkAnswer(sql(s"SHOW functions `$db`.`abs`"), Row("abs")) - checkAnswer(sql("SHOW functions `~`"), Row("~")) + def createFunction(names: Seq[String]): Unit = { + names.foreach { name => + sql( + s""" + |CREATE TEMPORARY FUNCTION $name + |AS '${classOf[PairUDF].getName}' + """.stripMargin) + } + } + def dropFunction(names: Seq[String]): Unit = { + names.foreach { name => + sql(s"DROP TEMPORARY FUNCTION $name") + } + } + createFunction(Seq("temp_abs", "temp_weekofyear", "temp_sha", "temp_sha1", "temp_sha2")) + + checkAnswer(sql("SHOW functions temp_abs"), Row("temp_abs")) + checkAnswer(sql("SHOW functions 'temp_abs'"), Row("temp_abs")) + checkAnswer(sql(s"SHOW functions $db.temp_abs"), Row("temp_abs")) + checkAnswer(sql(s"SHOW functions `$db`.`temp_abs`"), Row("temp_abs")) + checkAnswer(sql(s"SHOW functions `$db`.`temp_abs`"), Row("temp_abs")) checkAnswer(sql("SHOW functions `a function doens't exist`"), Nil) - checkAnswer(sql("SHOW functions `weekofyea*`"), Row("weekofyear")) + checkAnswer(sql("SHOW functions `temp_weekofyea*`"), Row("temp_weekofyear")) + // this probably will failed if we add more function with `sha` prefixing. - checkAnswer(sql("SHOW functions `sha*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil) + checkAnswer( + sql("SHOW functions `temp_sha*`"), + List(Row("temp_sha"), Row("temp_sha1"), Row("temp_sha2"))) + // Test '|' for alternation. checkAnswer( - sql("SHOW functions 'sha*|weekofyea*'"), - Row("sha") :: Row("sha1") :: Row("sha2") :: Row("weekofyear") :: Nil) + sql("SHOW functions 'temp_sha*|temp_weekofyea*'"), + List(Row("temp_sha"), Row("temp_sha1"), Row("temp_sha2"), Row("temp_weekofyear"))) + + dropFunction(Seq("temp_abs", "temp_weekofyear", "temp_sha", "temp_sha1", "temp_sha2")) } } From 974be6241e7cbe5433efc9715a9e65ace2fe50e4 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 13 Jun 2016 22:02:23 -0700 Subject: [PATCH 0636/1470] [SPARK-15932][SQL][DOC] document the contract of encoder serializer expressions ## What changes were proposed in this pull request? In our encoder framework, we imply that serializer expressions should use `BoundReference` to refer to the input object, and a lot of codes depend on this contract(e.g. ExpressionEncoder.tuple). This PR adds some document and assert in `ExpressionEncoder` to make it clearer. ## How was this patch tested? existing tests Author: Wenchen Fan Closes #13648 from cloud-fan/comment. (cherry picked from commit 688b6ef9dc0943d268fab7279ef50bfac1617f04) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/encoders/ExpressionEncoder.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 688082dcce538..0023ce64aa99b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -197,6 +197,15 @@ case class ExpressionEncoder[T]( if (flat) require(serializer.size == 1) + // serializer expressions are used to encode an object to a row, while the object is usually an + // intermediate value produced inside an operator, not from the output of the child operator. This + // is quite different from normal expressions, and `AttributeReference` doesn't work here + // (intermediate value is not an attribute). We assume that all serializer expressions use a same + // `BoundReference` to refer to the object, and throw exception if they don't. + assert(serializer.forall(_.references.isEmpty), "serializer cannot reference to any attributes.") + assert(serializer.flatMap(_.collect { case b: BoundReference => b}).distinct.length <= 1, + "all serializer expressions must use the same BoundReference.") + /** * Returns a new copy of this encoder, where the `deserializer` is resolved and bound to the * given schema. From cf52375b9f3da84d6aad31134d4f2859de7d447c Mon Sep 17 00:00:00 2001 From: Mortada Mehyar Date: Tue, 14 Jun 2016 09:45:46 +0100 Subject: [PATCH 0637/1470] [DOCUMENTATION] fixed typos in python programming guide ## What changes were proposed in this pull request? minor typo ## How was this patch tested? minor typo in the doc, should be self explanatory Author: Mortada Mehyar Closes #13639 from mortada/typo. (cherry picked from commit a87a56f5c70792eccbb57046f6b26d40494c380a) Signed-off-by: Sean Owen --- docs/programming-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 3f081a0e8f06e..97bcb511b2b02 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -491,7 +491,7 @@ for examples of using Cassandra / HBase ```InputFormat``` and ```OutputFormat``` RDDs support two types of operations: *transformations*, which create a new dataset from an existing one, and *actions*, which return a value to the driver program after running a computation on the dataset. For example, `map` is a transformation that passes each dataset element through a function and returns a new RDD representing the results. On the other hand, `reduce` is an action that aggregates all the elements of the RDD using some function and returns the final result to the driver program (although there is also a parallel `reduceByKey` that returns a distributed dataset). -All transformations in Spark are lazy, in that they do not compute their results right away. Instead, they just remember the transformations applied to some base dataset (e.g. a file). The transformations are only computed when an action requires a result to be returned to the driver program. This design enables Spark to run more efficiently -- for example, we can realize that a dataset created through `map` will be used in a `reduce` and return only the result of the `reduce` to the driver, rather than the larger mapped dataset. +All transformations in Spark are lazy, in that they do not compute their results right away. Instead, they just remember the transformations applied to some base dataset (e.g. a file). The transformations are only computed when an action requires a result to be returned to the driver program. This design enables Spark to run more efficiently. For example, we can realize that a dataset created through `map` will be used in a `reduce` and return only the result of the `reduce` to the driver, rather than the larger mapped dataset. By default, each transformed RDD may be recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting RDDs on disk, or replicated across multiple nodes. @@ -618,7 +618,7 @@ class MyClass { } {% endhighlight %} -Here, if we create a `new MyClass` and call `doStuff` on it, the `map` inside there references the +Here, if we create a new `MyClass` instance and call `doStuff` on it, the `map` inside there references the `func1` method *of that `MyClass` instance*, so the whole object needs to be sent to the cluster. It is similar to writing `rdd.map(x => this.func1(x))`. @@ -1156,7 +1156,7 @@ to disk, incurring the additional overhead of disk I/O and increased garbage col Shuffle also generates a large number of intermediate files on disk. As of Spark 1.3, these files are preserved until the corresponding RDDs are no longer used and are garbage collected. This is done so the shuffle files don't need to be re-created if the lineage is re-computed. -Garbage collection may happen only after a long period time, if the application retains references +Garbage collection may happen only after a long period of time, if the application retains references to these RDDs or if GC does not kick in frequently. This means that long-running Spark jobs may consume a large amount of disk space. The temporary storage directory is specified by the `spark.local.dir` configuration parameter when configuring the Spark context. From d59859d388519b7c981b672e979192ff86823d39 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 14 Jun 2016 02:12:29 -0700 Subject: [PATCH 0638/1470] [SPARK-15935][PYSPARK] Enable test for sql/streaming.py and fix these tests ## What changes were proposed in this pull request? This PR just enables tests for sql/streaming.py and also fixes the failures. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu Closes #13655 from zsxwing/python-streaming-test. (cherry picked from commit 96c3500c666235f1e746cf17b188930280489d50) Signed-off-by: Tathagata Das --- dev/sparktestsupport/modules.py | 1 + python/pyspark/sql/context.py | 2 ++ python/pyspark/sql/session.py | 11 ++++++ python/pyspark/sql/streaming.py | 61 ++++++++++++++++++++++----------- 4 files changed, 55 insertions(+), 20 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 8e2364d2f71cb..ce5725764be6d 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -337,6 +337,7 @@ def __hash__(self): "pyspark.sql.group", "pyspark.sql.functions", "pyspark.sql.readwriter", + "pyspark.sql.streaming", "pyspark.sql.window", "pyspark.sql.tests", ] diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 486733a390a0c..60f62b219b217 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -433,6 +433,8 @@ def read(self): def streams(self): """Returns a :class:`ContinuousQueryManager` that allows managing all the :class:`ContinuousQuery` ContinuousQueries active on `this` context. + + .. note:: Experimental. """ from pyspark.sql.streaming import ContinuousQueryManager return ContinuousQueryManager(self._ssql_ctx.streams()) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 7c9f532f9412f..f0bf0923b8c75 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -549,6 +549,17 @@ def read(self): """ return DataFrameReader(self._wrapped) + @property + @since(2.0) + def streams(self): + """Returns a :class:`ContinuousQueryManager` that allows managing all the + :class:`ContinuousQuery` ContinuousQueries active on `this` context. + + .. note:: Experimental. + """ + from pyspark.sql.streaming import ContinuousQueryManager + return ContinuousQueryManager(self._jsparkSession.streams()) + @since(2.0) def stop(self): """Stop the underlying :class:`SparkContext`. diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 580aba651f1b0..bb4e62cdd6a56 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -15,6 +15,12 @@ # limitations under the License. # +import sys +if sys.version >= '3': + intlike = int +else: + intlike = (int, long) + from abc import ABCMeta, abstractmethod from pyspark import since @@ -36,10 +42,18 @@ class ContinuousQuery(object): def __init__(self, jcq): self._jcq = jcq + @property + @since(2.0) + def id(self): + """The id of the continuous query. This id is unique across all queries that have been + started in the current process. + """ + return self._jcq.id() + @property @since(2.0) def name(self): - """The name of the continuous query. + """The name of the continuous query. This name is unique across all active queries. """ return self._jcq.name() @@ -106,7 +120,7 @@ def active(self): """Returns a list of active queries associated with this SQLContext >>> cq = df.write.format('memory').queryName('this_query').startStream() - >>> cqm = sqlContext.streams + >>> cqm = spark.streams >>> # get the list of active continuous queries >>> [q.name for q in cqm.active] [u'this_query'] @@ -114,20 +128,26 @@ def active(self): """ return [ContinuousQuery(jcq) for jcq in self._jcqm.active()] + @ignore_unicode_prefix @since(2.0) - def get(self, name): + def get(self, id): """Returns an active query from this SQLContext or throws exception if an active query with this name doesn't exist. - >>> df.write.format('memory').queryName('this_query').startStream() - >>> cq = sqlContext.streams.get('this_query') + >>> cq = df.write.format('memory').queryName('this_query').startStream() + >>> cq.name + u'this_query' + >>> cq = spark.streams.get(cq.id) + >>> cq.isActive + True + >>> cq = sqlContext.streams.get(cq.id) >>> cq.isActive True >>> cq.stop() """ - if type(name) != str or len(name.strip()) == 0: - raise ValueError("The name for the query must be a non-empty string. Got: %s" % name) - return ContinuousQuery(self._jcqm.get(name)) + if not isinstance(id, intlike): + raise ValueError("The id for the query must be an integer. Got: %d" % id) + return ContinuousQuery(self._jcqm.get(id)) @since(2.0) def awaitAnyTermination(self, timeout=None): @@ -162,7 +182,7 @@ def resetTerminated(self): """Forget about past terminated queries so that :func:`awaitAnyTermination()` can be used again to wait for new terminations. - >>> sqlContext.streams.resetTerminated() + >>> spark.streams.resetTerminated() """ self._jcqm.resetTerminated() @@ -209,27 +229,28 @@ def _test(): import doctest import os import tempfile - from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext, HiveContext - import pyspark.sql.readwriter + from pyspark.sql import Row, SparkSession, SQLContext + import pyspark.sql.streaming os.chdir(os.environ["SPARK_HOME"]) - globs = pyspark.sql.readwriter.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') + globs = pyspark.sql.streaming.__dict__.copy() + try: + spark = SparkSession.builder.enableHiveSupport().getOrCreate() + except py4j.protocol.Py4JError: + spark = SparkSession(sc) globs['tempfile'] = tempfile globs['os'] = os - globs['sc'] = sc - globs['sqlContext'] = SQLContext(sc) - globs['hiveContext'] = HiveContext._createForTesting(sc) + globs['spark'] = spark + globs['sqlContext'] = SQLContext.getOrCreate(spark.sparkContext) globs['df'] = \ - globs['sqlContext'].read.format('text').stream('python/test_support/sql/streaming') + globs['spark'].read.format('text').stream('python/test_support/sql/streaming') (failure_count, test_count) = doctest.testmod( - pyspark.sql.readwriter, globs=globs, + pyspark.sql.streaming, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) - globs['sc'].stop() + globs['spark'].stop() if failure_count: exit(-1) From 0d80bc291f8c96359b22bda2df8cb7b835e31339 Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Tue, 14 Jun 2016 13:59:01 +0100 Subject: [PATCH 0639/1470] [SPARK-15821][DOCS] Include parallel build info ## What changes were proposed in this pull request? We should mention that users can build Spark using multiple threads to decrease build times; either here or in "Building Spark" ## How was this patch tested? Built on machines with between one core to 192 cores using mvn -T 1C and observed faster build times with no loss in stability In response to the question here https://issues.apache.org/jira/browse/SPARK-15821 I think we should suggest this option as we know it works for Spark and can result in faster builds Author: Adam Roberts Closes #13562 from a-roberts/patch-3. (cherry picked from commit a431e3f1f8575e2498650ac767e69fbc903e9929) Signed-off-by: Sean Owen --- README.md | 2 ++ dev/make-distribution.sh | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index d5804d1a20b43..c77c429e577cd 100644 --- a/README.md +++ b/README.md @@ -25,6 +25,8 @@ To build Spark and its example programs, run: build/mvn -DskipTests clean package (You do not need to do this if you downloaded a pre-built package.) + +You can build Spark using more than one thread by using the -T option with Maven, see ["Parallel builds in Maven 3"](https://cwiki.apache.org/confluence/display/MAVEN/Parallel+builds+in+Maven+3). More detailed documentation is available from the project site, at ["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). For developing Spark using an IDE, see [Eclipse](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-Eclipse) diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 4f7544f6ea78b..9be4fdfa51c93 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -53,7 +53,7 @@ while (( "$#" )); do --hadoop) echo "Error: '--hadoop' is no longer supported:" echo "Error: use Maven profiles and options -Dhadoop.version and -Dyarn.version instead." - echo "Error: Related profiles include hadoop-2.2, hadoop-2.3 and hadoop-2.4." + echo "Error: Related profiles include hadoop-2.2, hadoop-2.3, hadoop-2.4, hadoop-2.6 and hadoop-2.7." exit_with_usage ;; --with-yarn) @@ -150,7 +150,7 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCac # Store the command as an array because $MVN variable might have spaces in it. # Normal quoting tricks don't work. # See: http://mywiki.wooledge.org/BashFAQ/050 -BUILD_COMMAND=("$MVN" clean package -DskipTests $@) +BUILD_COMMAND=("$MVN" -T 1C clean package -DskipTests $@) # Actually build the jar echo -e "\nBuilding with..." From e90ba228787c0a8b50855bafb0bc16eddee8329b Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Tue, 14 Jun 2016 14:28:40 +0100 Subject: [PATCH 0640/1470] doc fix of HiveThriftServer ## What changes were proposed in this pull request? Just minor doc fix. \cc yhuai Author: Jeff Zhang Closes #13659 from zjffdu/doc_fix. (cherry picked from commit 53bb03084796231f724ff8369490df520e1ee33c) Signed-off-by: Sean Owen --- .../spark/sql/hive/thriftserver/ui/ThriftServerPage.scala | 2 +- .../sql/hive/thriftserver/ui/ThriftServerSessionPage.scala | 4 ++-- .../spark/sql/hive/thriftserver/ui/ThriftServerTab.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index c82fa4eaaa4e5..2e0fa1ef77f88 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -30,7 +30,7 @@ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ -/** Page for Spark Web UI that shows statistics of a thrift server */ +/** Page for Spark Web UI that shows statistics of the thrift server */ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { private val listener = parent.listener diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 008108a5ce06d..f39e9dcd3a5bb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ -/** Page for Spark Web UI that shows statistics of a streaming job */ +/** Page for Spark Web UI that shows statistics of jobs running in the thrift server */ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) extends WebUIPage("session") with Logging { @@ -60,7 +60,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) UIUtils.headerSparkPage("JDBC/ODBC Session", content, parent, Some(5000)) } - /** Generate basic stats of the streaming program */ + /** Generate basic stats of the thrift server program */ private def generateBasicStats(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime.getTime
      diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index 923ba8a30c5c5..db2066009b351 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._ import org.apache.spark.ui.{SparkUI, SparkUITab} /** - * Spark Web UI tab that shows statistics of a streaming job. + * Spark Web UI tab that shows statistics of jobs running in the thrift server. * This assumes the given SparkContext has enabled its SparkUI. */ private[thriftserver] class ThriftServerTab(sparkContext: SparkContext) From 73beb9fb392af1852a60a899488d7cc7735e0847 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Tue, 14 Jun 2016 09:10:27 -0700 Subject: [PATCH 0641/1470] [SPARK-15914][SQL] Add deprecated method back to SQLContext for backward source code compatibility ## What changes were proposed in this pull request? Revert partial changes in SPARK-12600, and add some deprecated method back to SQLContext for backward source code compatibility. ## How was this patch tested? Manual test. Author: Sean Zhong Closes #13637 from clockfly/SPARK-15914. (cherry picked from commit 6e8cdef0cf36f6e921d9e1a65c61b66196935820) Signed-off-by: Yin Huai --- project/MimaExcludes.scala | 9 + .../org/apache/spark/sql/SQLContext.scala | 284 ++++++++++++++++++ 2 files changed, 293 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 9d0d9b1be0771..a6209d78e168c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -778,6 +778,15 @@ object MimaExcludes { ) ++ Seq( ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asBreeze"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asBreeze") + ) ++ Seq( + // [SPARK-15914] Binary compatibility is broken since consolidation of Dataset and DataFrame + // in Spark 2.0. However, source level compatibility is still maintained. + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.load"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonFile"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jdbc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema") ) case v if v.startsWith("1.6") => Seq( 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 6fcc9bb44775a..58b4e6c5f604e 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 @@ -736,6 +736,290 @@ class SQLContext private[sql](val sparkSession: SparkSession) private[sql] def parseDataType(dataTypeString: String): DataType = { DataType.fromJson(dataTypeString) } + + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + // Deprecated methods + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: RDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd, beanClass) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd, beanClass) + } + + /** + * Loads a Parquet file, returning the result as a [[DataFrame]]. This function returns an empty + * [[DataFrame]] if no paths are passed in. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().parquet()`. + */ + @deprecated("Use read.parquet() instead.", "1.4.0") + @scala.annotation.varargs + def parquetFile(paths: String*): DataFrame = { + if (paths.isEmpty) { + emptyDataFrame + } else { + read.parquet(paths : _*) + } + } + + /** + * 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. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String): DataFrame = { + read.json(path) + } + + /** + * Loads a JSON file (one object per line) and applies the given schema, + * returning the result as a [[DataFrame]]. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, schema: StructType): DataFrame = { + read.schema(schema).json(path) + } + + /** + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(path) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * [[DataFrame]]. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String]): DataFrame = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * [[DataFrame]]. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String]): DataFrame = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, + * returning the result as a [[DataFrame]]. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], schema: StructType): DataFrame = { + read.schema(schema).json(json) + } + + /** + * Loads an JavaRDD storing JSON objects (one object per record) and applies the given + * schema, returning the result as a [[DataFrame]]. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], schema: StructType): DataFrame = { + read.schema(schema).json(json) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a [[DataFrame]]. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Loads a JavaRDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a [[DataFrame]]. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Returns the dataset stored at path as a DataFrame, + * using the default data source configured by spark.sql.sources.default. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().load(path)`. + */ + @deprecated("Use read.load(path) instead.", "1.4.0") + def load(path: String): DataFrame = { + read.load(path) + } + + /** + * Returns the dataset stored at path as a DataFrame, using the given data source. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).load(path)`. + */ + @deprecated("Use read.format(source).load(path) instead.", "1.4.0") + def load(path: String, source: String): DataFrame = { + read.format(source).load(path) + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: java.util.Map[String, String]): DataFrame = { + read.options(options).format(source).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: Map[String, String]): DataFrame = { + read.options(options).format(source).load() + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by + * `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load( + source: String, + schema: StructType, + options: java.util.Map[String, String]): DataFrame = { + read.format(source).schema(schema).options(options).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by + * `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load(source: String, schema: StructType, options: Map[String, String]): DataFrame = { + read.format(source).schema(schema).options(options).load() + } + + /** + * Construct a [[DataFrame]] representing the database table accessible via JDBC URL + * url named table. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String): DataFrame = { + read.jdbc(url, table, new Properties) + } + + /** + * Construct a [[DataFrame]] representing the database table accessible via JDBC URL + * url named table. Partitions of the table will be retrieved in parallel based on the parameters + * 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` 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 + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc( + url: String, + table: String, + columnName: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): DataFrame = { + read.jdbc(url, table, columnName, lowerBound, upperBound, numPartitions, new Properties) + } + + /** + * Construct a [[DataFrame]] representing the database table accessible via JDBC URL + * url named table. The theParts parameter gives a list expressions + * suitable for inclusion in WHERE clauses; each one defines one partition + * of the [[DataFrame]]. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String, theParts: Array[String]): DataFrame = { + read.jdbc(url, table, theParts, new Properties) + } } /** From d5e60748b2a871ca9a23957f4b62b4c07010c409 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 14 Jun 2016 09:58:06 -0700 Subject: [PATCH 0642/1470] [SPARK-15655][SQL] Fix Wrong Partition Column Order when Fetching Partitioned Tables #### What changes were proposed in this pull request? When fetching the partitioned table, the output contains wrong results. The order of partition key values do not match the order of partition key columns in output schema. For example, ```SQL CREATE TABLE table_with_partition(c1 string) PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) INSERT OVERWRITE TABLE table_with_partition PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') SELECT 'blarr' SELECT p1, p2, p3, p4, p5, c1 FROM table_with_partition ``` ``` +---+---+---+---+---+-----+ | p1| p2| p3| p4| p5| c1| +---+---+---+---+---+-----+ | d| e| c| b| a|blarr| +---+---+---+---+---+-----+ ``` The expected result should be ``` +---+---+---+---+---+-----+ | p1| p2| p3| p4| p5| c1| +---+---+---+---+---+-----+ | a| b| c| d| e|blarr| +---+---+---+---+---+-----+ ``` This PR is to fix this by enforcing the order matches the table partition definition. #### How was this patch tested? Added a test case into `SQLQuerySuite` Author: gatorsmile Closes #13400 from gatorsmile/partitionedTableFetch. (cherry picked from commit bc02d011294fcd1ab07b9baf1011c3f2bdf749d9) Signed-off-by: Wenchen Fan --- .../spark/sql/hive/MetastoreRelation.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 9c820144aee12..5596a4470f3f9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -160,7 +160,7 @@ private[hive] case class MetastoreRelation( val tPartition = new org.apache.hadoop.hive.metastore.api.Partition tPartition.setDbName(databaseName) tPartition.setTableName(tableName) - tPartition.setValues(p.spec.values.toList.asJava) + tPartition.setValues(partitionKeys.map(a => p.spec(a.name)).asJava) val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tPartition.setSd(sd) 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 1a0eaa66c1391..9c1f21825315b 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 @@ -1610,6 +1610,38 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(fs.exists(path), "This is an external table, so the data should not have been dropped") } + test("select partitioned table") { + val table = "table_with_partition" + withTable(table) { + sql( + s""" + |CREATE TABLE $table(c1 string) + |PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) + """.stripMargin) + sql( + s""" + |INSERT OVERWRITE TABLE $table + |PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') + |SELECT 'blarr' + """.stripMargin) + + // project list is the same order of paritioning columns in table definition + checkAnswer( + sql(s"SELECT p1, p2, p3, p4, p5, c1 FROM $table"), + Row("a", "b", "c", "d", "e", "blarr") :: Nil) + + // project list does not have the same order of paritioning columns in table definition + checkAnswer( + sql(s"SELECT p2, p3, p4, p1, p5, c1 FROM $table"), + Row("b", "c", "d", "a", "e", "blarr") :: Nil) + + // project list contains partial partition columns in table definition + checkAnswer( + sql(s"SELECT p2, p1, p5, c1 FROM $table"), + Row("b", "a", "e", "blarr") :: Nil) + } + } + test("SPARK-14981: DESC not supported for sorting columns") { withTable("t") { val cause = intercept[ParseException] { From 83aa17d44cfb19aa43593139b682adf0f2adac09 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 14 Jun 2016 10:52:13 -0700 Subject: [PATCH 0643/1470] [SPARK-15915][SQL] Logical plans should use canonicalized plan when override sameResult. ## What changes were proposed in this pull request? `DataFrame` with plan overriding `sameResult` but not using canonicalized plan to compare can't cacheTable. The example is like: ``` val localRelation = Seq(1, 2, 3).toDF() localRelation.createOrReplaceTempView("localRelation") spark.catalog.cacheTable("localRelation") assert( localRelation.queryExecution.withCachedData.collect { case i: InMemoryRelation => i }.size == 1) ``` and this will fail as: ``` ArrayBuffer() had size 0 instead of expected size 1 ``` The reason is that when do `spark.catalog.cacheTable("localRelation")`, `CacheManager` tries to cache for the plan wrapped by `SubqueryAlias` but when planning for the DataFrame `localRelation`, `CacheManager` tries to find cached table for the not-wrapped plan because the plan for DataFrame `localRelation` is not wrapped. Some plans like `LocalRelation`, `LogicalRDD`, etc. override `sameResult` method, but not use canonicalized plan to compare so the `CacheManager` can't detect the plans are the same. This pr modifies them to use canonicalized plan when override `sameResult` method. ## How was this patch tested? Added a test to check if DataFrame with plan overriding sameResult but not using canonicalized plan to compare can cacheTable. Author: Takuya UESHIN Closes #13638 from ueshin/issues/SPARK-15915. (cherry picked from commit c5b735581922c52a1b1cc6cd8c7b5878d3cf8f20) Signed-off-by: Michael Armbrust --- .../sql/catalyst/plans/logical/LocalRelation.scala | 10 ++++++---- .../org/apache/spark/sql/execution/ExistingRDD.scala | 8 +++++--- .../sql/execution/datasources/LogicalRelation.scala | 8 +++++--- .../scala/org/apache/spark/sql/CachedTableSuite.scala | 11 +++++++++++ .../org/apache/spark/sql/hive/MetastoreRelation.scala | 2 +- 5 files changed, 28 insertions(+), 11 deletions(-) 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 87b8647655f4a..9d64f35efcc6a 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 @@ -65,10 +65,12 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil) } } - override def sameResult(plan: LogicalPlan): Boolean = plan match { - case LocalRelation(otherOutput, otherData) => - otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data - case _ => false + override def sameResult(plan: LogicalPlan): Boolean = { + plan.canonicalized match { + case LocalRelation(otherOutput, otherData) => + otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data + case _ => false + } } override lazy val statistics = 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 ee72a70cced1a..e2c23a4ba8670 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 @@ -87,9 +87,11 @@ private[sql] case class LogicalRDD( override def newInstance(): LogicalRDD.this.type = LogicalRDD(output.map(_.newInstance()), rdd)(session).asInstanceOf[this.type] - override def sameResult(plan: LogicalPlan): Boolean = plan match { - case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id - case _ => false + override def sameResult(plan: LogicalPlan): Boolean = { + plan.canonicalized match { + case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id + case _ => false + } } override protected def stringArgs: Iterator[Any] = Iterator(output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index a418d02983eb9..39c8606fd14b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -60,9 +60,11 @@ case class LogicalRelation( com.google.common.base.Objects.hashCode(relation, output) } - override def sameResult(otherPlan: LogicalPlan): Boolean = otherPlan match { - case LogicalRelation(otherRelation, _, _) => relation == otherRelation - case _ => false + override def sameResult(otherPlan: LogicalPlan): Boolean = { + otherPlan.canonicalized match { + case LogicalRelation(otherRelation, _, _) => relation == otherRelation + case _ => false + } } // When comparing two LogicalRelations from within LogicalPlan.sameResult, we only need 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 44bafa55bc330..3306ac42a3650 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 @@ -552,4 +552,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext selectStar, Seq(Row(1, "1"))) } + + test("SPARK-15915 Logical plans should use canonicalized plan when override sameResult") { + val localRelation = Seq(1, 2, 3).toDF() + localRelation.createOrReplaceTempView("localRelation") + + spark.catalog.cacheTable("localRelation") + assert( + localRelation.queryExecution.withCachedData.collect { + case i: InMemoryRelation => i + }.size == 1) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 5596a4470f3f9..58bca2059cacc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -185,7 +185,7 @@ private[hive] case class MetastoreRelation( /** Only compare database and tablename, not alias. */ override def sameResult(plan: LogicalPlan): Boolean = { - plan match { + plan.canonicalized match { case mr: MetastoreRelation => mr.databaseName == databaseName && mr.tableName == tableName case _ => false From 515937046e44a19f49f7d6606ed2246c7e56da7a Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 14 Jun 2016 11:44:37 -0700 Subject: [PATCH 0644/1470] [SPARK-15864][SQL] Fix Inconsistent Behaviors when Uncaching Non-cached Tables #### What changes were proposed in this pull request? To uncache a table, we have three different ways: - _SQL interface_: `UNCACHE TABLE` - _DataSet API_: `sparkSession.catalog.uncacheTable` - _DataSet API_: `sparkSession.table(tableName).unpersist()` When the table is not cached, - _SQL interface_: `UNCACHE TABLE non-cachedTable` -> **no error message** - _Dataset API_: `sparkSession.catalog.uncacheTable("non-cachedTable")` -> **report a strange error message:** ```requirement failed: Table [a: int] is not cached``` - _Dataset API_: `sparkSession.table("non-cachedTable").unpersist()` -> **no error message** This PR will make them consistent. No operation if the table has already been uncached. In addition, this PR also removes `uncacheQuery` and renames `tryUncacheQuery` to `uncacheQuery`, and documents it that it's noop if the table has already been uncached #### How was this patch tested? Improved the existing test case for verifying the cases when the table has not been cached. Also added test cases for verifying the cases when the table does not exist Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13593 from gatorsmile/uncacheNonCachedTable. (cherry picked from commit df4ea6614d709ee66f1ceb966df6216b125b8ea1) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../spark/sql/execution/CacheManager.scala | 17 +++---------- .../spark/sql/execution/command/cache.scala | 2 +- .../spark/sql/execution/command/ddl.scala | 2 +- .../spark/sql/execution/command/tables.scala | 2 +- .../spark/sql/internal/CatalogImpl.scala | 6 ++--- .../apache/spark/sql/CachedTableSuite.scala | 6 ----- .../spark/sql/hive/CachedTableSuite.scala | 24 +++++++++++++++---- 8 files changed, 30 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 5a67fc79eff0b..53779df3d9c00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2316,7 +2316,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def unpersist(blocking: Boolean): this.type = { - sparkSession.sharedState.cacheManager.tryUncacheQuery(this, blocking) + sparkSession.sharedState.cacheManager.uncacheQuery(this, blocking) this } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 4e95754e9bef7..de2503a87ab7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -104,22 +104,11 @@ private[sql] class CacheManager extends Logging { } } - /** Removes the data for the given [[Dataset]] from the cache */ - private[sql] def uncacheQuery(query: Dataset[_], blocking: Boolean = true): Unit = writeLock { - 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.remove(dataIndex) - } - /** - * Tries to remove the data for the given [[Dataset]] from the cache - * if it's cached + * Tries to remove the data for the given [[Dataset]] from the cache. + * No operation, if it's already uncached. */ - private[sql] def tryUncacheQuery( - query: Dataset[_], - blocking: Boolean = true): Boolean = writeLock { + private[sql] def uncacheQuery(query: Dataset[_], blocking: Boolean = true): Boolean = writeLock { val planToCache = query.queryExecution.analyzed val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) val found = dataIndex >= 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 3e5eed2efa76b..5332366d242a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -53,7 +53,7 @@ case class CacheTableCommand( case class UncacheTableCommand(tableName: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.table(tableName).unpersist(blocking = false) + sparkSession.catalog.uncacheTable(tableName) Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 5fd0b83cf0a5c..fc00912bf9f59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -201,7 +201,7 @@ case class DropTableCommand( case _ => }) try { - sparkSession.sharedState.cacheManager.tryUncacheQuery( + sparkSession.sharedState.cacheManager.uncacheQuery( sparkSession.table(tableName.quotedString)) } catch { case NonFatal(e) => log.warn(e.toString, e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 90db7853329f0..58bb5cdca9910 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -374,7 +374,7 @@ case class TruncateTableCommand( spark.sessionState.invalidateTable(tableName.unquotedString) // Also try to drop the contents of the table from the columnar cache try { - spark.sharedState.cacheManager.tryUncacheQuery(spark.table(tableName.quotedString)) + spark.sharedState.cacheManager.uncacheQuery(spark.table(tableName.quotedString)) } catch { case NonFatal(e) => log.warn(s"Exception when attempting to uncache table '$tableName'", e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index f42fd174b262b..601334b97add3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -292,7 +292,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def dropTempView(viewName: String): Unit = { - sparkSession.sharedState.cacheManager.tryUncacheQuery(sparkSession.table(viewName)) + sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession.table(viewName)) sessionCatalog.dropTable(TableIdentifier(viewName), ignoreIfNotExists = true) } @@ -323,7 +323,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def uncacheTable(tableName: String): Unit = { - sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession.table(tableName)) + sparkSession.sharedState.cacheManager.uncacheQuery(query = sparkSession.table(tableName)) } /** @@ -367,7 +367,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // TODO: Use uncacheTable once it supports database name. val df = Dataset.ofRows(sparkSession, logicalPlan) // Uncache the logicalPlan. - sparkSession.sharedState.cacheManager.tryUncacheQuery(df, blocking = true) + sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true) // Cache it again. sparkSession.sharedState.cacheManager.cacheQuery(df, Some(tableIdent.table)) } 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 3306ac42a3650..d7df18ae1c42d 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 @@ -186,12 +186,6 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext assertCached(spark.table("testData"), 0) } - test("correct error on uncache of non-cached table") { - intercept[IllegalArgumentException] { - spark.catalog.uncacheTable("testData") - } - } - test("SELECT star from cached table") { sql("SELECT * FROM testData").createOrReplaceTempView("selectStar") spark.catalog.cacheTable("selectStar") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 5121440f067bd..e35a71917fbda 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,12 +20,14 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.storage.RDDBlockId import org.apache.spark.util.Utils -class CachedTableSuite extends QueryTest with TestHiveSingleton { +class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ def rddIdOf(tableName: String): Int = { @@ -95,9 +97,23 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { sql("DROP TABLE IF EXISTS nonexistantTable") } - test("correct error on uncache of non-cached table") { - intercept[IllegalArgumentException] { - spark.catalog.uncacheTable("src") + test("correct error on uncache of nonexistant tables") { + intercept[NoSuchTableException] { + spark.catalog.uncacheTable("nonexistantTable") + } + intercept[NoSuchTableException] { + sql("UNCACHE TABLE nonexistantTable") + } + } + + test("no error on uncache of non-cached table") { + val tableName = "newTable" + withTable(tableName) { + sql(s"CREATE TABLE $tableName(a INT)") + // no error will be reported in the following three ways to uncache a table. + spark.catalog.uncacheTable(tableName) + sql("UNCACHE TABLE newTable") + sparkSession.table(tableName).unpersist() } } From e03c25193b4a939edbd6e9efc4f46272defd7f0e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 14 Jun 2016 12:13:12 -0700 Subject: [PATCH 0645/1470] [SPARK-15895][SQL] Filters out metadata files while doing partition discovery ## What changes were proposed in this pull request? Take the following directory layout as an example: ``` dir/ +- p0=0/ |-_metadata +- p1=0/ |-part-00001.parquet |-part-00002.parquet |-... ``` The `_metadata` file under `p0=0` shouldn't fail partition discovery. This PR filters output all metadata files whose names start with `_` while doing partition discovery. ## How was this patch tested? New unit test added in `ParquetPartitionDiscoverySuite`. Author: Cheng Lian Closes #13623 from liancheng/spark-15895-partition-disco-no-metafiles. (cherry picked from commit bd39ffe35c6f939debe5d3c5eb4970b4e62507b0) Signed-off-by: Yin Huai --- .../datasources/ListingFileCatalog.scala | 7 +-- .../PartitioningAwareFileCatalog.scala | 15 +++++-- .../ParquetPartitionDiscoverySuite.scala | 44 +++++++++++++++++++ 3 files changed, 60 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index 7d2854aaad394..d96cf1bf0702c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable import scala.util.Try -import org.apache.hadoop.fs.{FileStatus, LocatedFileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.sql.SparkSession @@ -83,8 +83,9 @@ class ListingFileCatalog( val statuses: Seq[FileStatus] = paths.flatMap { path => val fs = path.getFileSystem(hadoopConf) logInfo(s"Listing $path on driver") - Try(HadoopFsRelation.listLeafFiles(fs, fs.getFileStatus(path), pathFilter)). - getOrElse(Array.empty) + Try { + HadoopFsRelation.listLeafFiles(fs, fs.getFileStatus(path), pathFilter) + }.getOrElse(Array.empty[FileStatus]) } mutable.LinkedHashSet(statuses: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index 406d2e8e81f49..811e96c99a96d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -50,14 +50,14 @@ abstract class PartitioningAwareFileCatalog( override def listFiles(filters: Seq[Expression]): Seq[Partition] = { val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) { - Partition(InternalRow.empty, allFiles().filterNot(_.getPath.getName startsWith "_")) :: Nil + Partition(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil } else { prunePartitions(filters, partitionSpec()).map { case PartitionDirectory(values, path) => val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match { case Some(existingDir) => // Directory has children files in it, return them - existingDir.filterNot(_.getPath.getName.startsWith("_")) + existingDir.filter(f => isDataPath(f.getPath)) case None => // Directory does not exist, or has no children files @@ -96,7 +96,11 @@ abstract class PartitioningAwareFileCatalog( protected def inferPartitioning(): PartitionSpec = { // We use leaf dirs containing data files to discover the schema. - val leafDirs = leafDirToChildrenFiles.keys.toSeq + val leafDirs = leafDirToChildrenFiles.filter { case (_, files) => + // SPARK-15895: Metadata files (e.g. Parquet summary files) and temporary files should not be + // counted as data files, so that they shouldn't participate partition discovery. + files.exists(f => isDataPath(f.getPath)) + }.keys.toSeq partitionSchema match { case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => val spec = PartitioningUtils.parsePartitions( @@ -197,4 +201,9 @@ abstract class PartitioningAwareFileCatalog( if (leafFiles.contains(qualifiedPath)) qualifiedPath.getParent else qualifiedPath }.toSet } } + + private def isDataPath(path: Path): Boolean = { + val name = path.getName + !(name.startsWith("_") || name.startsWith(".")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index e19345529e93b..133ffedf12812 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -25,11 +25,13 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.Files import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -890,4 +892,46 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } } + + test("SPARK-15895 summary files in non-leaf partition directories") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withSQLConf(ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true") { + spark.range(3).write.parquet(s"$path/p0=0/p1=0") + } + + val p0 = new File(path, "p0=0") + val p1 = new File(p0, "p1=0") + + // Builds the following directory layout by: + // + // 1. copying Parquet summary files we just wrote into `p0=0`, and + // 2. touching a dot-file `.dummy` under `p0=0`. + // + // + // +- p0=0 + // |- _metadata + // |- _common_metadata + // |- .dummy + // +- p1=0 + // |- _metadata + // |- _common_metadata + // |- part-00000.parquet + // |- part-00001.parquet + // +- ... + // + // The summary files and the dot-file under `p0=0` should not fail partition discovery. + + Files.copy(new File(p1, "_metadata"), new File(p0, "_metadata")) + Files.copy(new File(p1, "_common_metadata"), new File(p0, "_common_metadata")) + Files.touch(new File(p0, ".dummy")) + + checkAnswer(spark.read.parquet(s"$path"), Seq( + Row(0, 0, 0), + Row(1, 0, 0), + Row(2, 0, 0) + )) + } + } } From 24539223b043b621a377251bdab206833af78d0c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 14 Jun 2016 09:40:07 -0700 Subject: [PATCH 0646/1470] [MINOR] Clean up several build warnings, mostly due to internal use of old accumulators Another PR to clean up recent build warnings. This particularly cleans up several instances of the old accumulator API usage in tests that are straightforward to update. I think this qualifies as "minor". Jenkins Author: Sean Owen Closes #13642 from srowen/BuildWarnings. (cherry picked from commit 6151d2641f91c8e3ec0c324e78afb46cdb2ef111) Signed-off-by: Sean Owen --- core/pom.xml | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 12 +- .../spark/scheduler/TaskContextSuite.scala | 9 +- .../spark/sql/execution/debug/package.scala | 34 +++--- .../execution/metric/SQLMetricsSuite.scala | 105 +----------------- .../deploy/yarn/YarnAllocatorSuite.scala | 1 + 6 files changed, 31 insertions(+), 136 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index f5fdb40696017..90c8f97f2bbac 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -356,12 +356,12 @@ generate-resources - + - + - + run 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 5bcc8ff5252a1..ce4e7a237e9f9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1593,13 +1593,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou } test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { - val acc = new Accumulator[Int](0, new AccumulatorParam[Int] { - override def addAccumulator(t1: Int, t2: Int): Int = t1 + t2 - override def zero(initialValue: Int): Int = 0 - override def addInPlace(r1: Int, r2: Int): Int = { - throw new DAGSchedulerSuiteDummyException - } - }) + val acc = new LongAccumulator { + override def add(v: java.lang.Long): Unit = throw new DAGSchedulerSuiteDummyException + override def add(v: Long): Unit = throw new DAGSchedulerSuiteDummyException + } + sc.register(acc) // Run this on executors sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 368668bc7e2e4..9eda79ace18d0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -146,14 +146,13 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark test("accumulators are updated on exception failures") { // This means use 1 core and 4 max task failures sc = new SparkContext("local[1,4]", "test") - val param = AccumulatorParam.LongAccumulatorParam // Create 2 accumulators, one that counts failed values and another that doesn't - val acc1 = new Accumulator(0L, param, Some("x"), countFailedValues = true) - val acc2 = new Accumulator(0L, param, Some("y"), countFailedValues = false) + val acc1 = AccumulatorSuite.createLongAccum("x", true) + val acc2 = AccumulatorSuite.createLongAccum("y", false) // Fail first 3 attempts of every task. This means each task should be run 4 times. sc.parallelize(1 to 10, 10).map { i => - acc1 += 1 - acc2 += 1 + acc1.add(1) + acc2.add(1) if (TaskContext.get.attemptNumber() <= 2) { throw new Exception("you did something wrong") } else { 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 f2c558ac2de7a..e89f792496d6a 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution import scala.collection.mutable.HashSet -import org.apache.spark.{Accumulator, AccumulatorParam} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -28,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.util.LongAccumulator +import org.apache.spark.util.{AccumulatorV2, LongAccumulator} /** * Contains methods for debugging query execution. @@ -108,26 +107,27 @@ package object debug { private[sql] case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { def output: Seq[Attribute] = child.output - implicit object SetAccumulatorParam extends AccumulatorParam[HashSet[String]] { - def zero(initialValue: HashSet[String]): HashSet[String] = { - initialValue.clear() - initialValue - } - - def addInPlace(v1: HashSet[String], v2: HashSet[String]): HashSet[String] = { - v1 ++= v2 - v1 + class SetAccumulator[T] extends AccumulatorV2[T, HashSet[T]] { + private val _set = new HashSet[T]() + override def isZero: Boolean = _set.isEmpty + override def copy(): AccumulatorV2[T, HashSet[T]] = { + val newAcc = new SetAccumulator[T]() + newAcc._set ++= _set + newAcc } + override def reset(): Unit = _set.clear() + override def add(v: T): Unit = _set += v + override def merge(other: AccumulatorV2[T, HashSet[T]]): Unit = _set ++= other.value + override def value: HashSet[T] = _set } /** * A collection of metrics for each column of output. - * - * @param elementTypes the actual runtime types for the output. Useful when there are bugs - * causing the wrong data to be projected. */ - case class ColumnMetrics( - elementTypes: Accumulator[HashSet[String]] = sparkContext.accumulator(HashSet.empty)) + case class ColumnMetrics() { + val elementTypes = new SetAccumulator[String] + sparkContext.register(elementTypes) + } val tupleCount: LongAccumulator = sparkContext.longAccumulator @@ -155,7 +155,7 @@ package object debug { while (i < numColumns) { val value = currentRow.get(i, output(i).dataType) if (value != null) { - columnStats(i).elementTypes += HashSet(value.getClass.getName) + columnStats(i).elementTypes.add(value.getClass.getName) } i += 1 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index fd956bc4ef900..579a095ff000f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -17,13 +17,6 @@ package org.apache.spark.sql.execution.metric -import java.io.{ByteArrayInputStream, ByteArrayOutputStream} - -import scala.collection.mutable - -import org.apache.xbean.asm5._ -import org.apache.xbean.asm5.Opcodes._ - import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.execution.SparkPlanInfo @@ -31,34 +24,11 @@ import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.util.{AccumulatorContext, JsonProtocol, Utils} - +import org.apache.spark.util.{AccumulatorContext, JsonProtocol} class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ - test("SQLMetric should not box Long") { - val l = SQLMetrics.createMetric(sparkContext, "long") - val f = () => { - l += 1L - l.add(1L) - } - val cl = BoxingFinder.getClassReader(f.getClass) - val boxingFinder = new BoxingFinder() - cl.accept(boxingFinder, 0) - assert(boxingFinder.boxingInvokes.isEmpty, s"Found boxing: ${boxingFinder.boxingInvokes}") - } - - test("Normal accumulator should do boxing") { - // We need this test to make sure BoxingFinder works. - val l = sparkContext.accumulator(0L) - val f = () => { l += 1L } - val cl = BoxingFinder.getClassReader(f.getClass) - val boxingFinder = new BoxingFinder() - cl.accept(boxingFinder, 0) - assert(boxingFinder.boxingInvokes.nonEmpty, "Found find boxing in this test") - } - /** * Call `df.collect()` and verify if the collected metrics are same as "expectedMetrics". * @@ -323,76 +293,3 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } - -private case class MethodIdentifier[T](cls: Class[T], name: String, desc: String) - -/** - * If `method` is null, search all methods of this class recursively to find if they do some boxing. - * If `method` is specified, only search this method of the class to speed up the searching. - * - * This method will skip the methods in `visitedMethods` to avoid potential infinite cycles. - */ -private class BoxingFinder( - method: MethodIdentifier[_] = null, - val boxingInvokes: mutable.Set[String] = mutable.Set.empty, - visitedMethods: mutable.Set[MethodIdentifier[_]] = mutable.Set.empty) - extends ClassVisitor(ASM5) { - - private val primitiveBoxingClassName = - Set("java/lang/Long", - "java/lang/Double", - "java/lang/Integer", - "java/lang/Float", - "java/lang/Short", - "java/lang/Character", - "java/lang/Byte", - "java/lang/Boolean") - - override def visitMethod( - access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): - MethodVisitor = { - if (method != null && (method.name != name || method.desc != desc)) { - // If method is specified, skip other methods. - return new MethodVisitor(ASM5) {} - } - - new MethodVisitor(ASM5) { - override def visitMethodInsn( - op: Int, owner: String, name: String, desc: String, itf: Boolean) { - if (op == INVOKESPECIAL && name == "" || op == INVOKESTATIC && name == "valueOf") { - if (primitiveBoxingClassName.contains(owner)) { - // Find boxing methods, e.g, new java.lang.Long(l) or java.lang.Long.valueOf(l) - boxingInvokes.add(s"$owner.$name") - } - } else { - // scalastyle:off classforname - val classOfMethodOwner = Class.forName(owner.replace('/', '.'), false, - Thread.currentThread.getContextClassLoader) - // scalastyle:on classforname - val m = MethodIdentifier(classOfMethodOwner, name, desc) - if (!visitedMethods.contains(m)) { - // Keep track of visited methods to avoid potential infinite cycles - visitedMethods += m - val cl = BoxingFinder.getClassReader(classOfMethodOwner) - visitedMethods += m - cl.accept(new BoxingFinder(m, boxingInvokes, visitedMethods), 0) - } - } - } - } - } -} - -private object BoxingFinder { - - def getClassReader(cls: Class[_]): ClassReader = { - val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" - val resourceStream = cls.getResourceAsStream(className) - val baos = new ByteArrayOutputStream(128) - // Copy data over, before delegating to ClassReader - - // else we can run out of open file handles. - Utils.copyStream(resourceStream, baos, true) - new ClassReader(new ByteArrayInputStream(baos.toByteArray)) - } - -} 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 f4f8bd435d5f2..207dbf56d3606 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 @@ -111,6 +111,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter } def createContainer(host: String): Container = { + // When YARN 2.6+ is required, avoid deprecation by using version with long second arg val containerId = ContainerId.newInstance(appAttemptId, containerNum) containerNum += 1 val nodeId = NodeId.newInstance(host, 1000) From 9adba414cb69b1f2dd38f75c81ec3ab0549a353a Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Tue, 14 Jun 2016 13:05:56 -0700 Subject: [PATCH 0647/1470] [SPARK-15247][SQL] Set the default number of partitions for reading parquet schemas ## What changes were proposed in this pull request? This pr sets the default number of partitions when reading parquet schemas. SQLContext#read#parquet currently yields at least n_executors * n_cores tasks even if parquet data consist of a single small file. This issue could increase the latency for small jobs. ## How was this patch tested? Manually tested and checked. Author: Takeshi YAMAMURO Closes #13137 from maropu/SPARK-15247. (cherry picked from commit dae4d5db21368faaa46fa8d1a256c27428694c2c) Signed-off-by: Yin Huai --- .../execution/datasources/parquet/ParquetFileFormat.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 71c16008be8f9..6b25e36f7b9a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -794,11 +794,16 @@ private[sql] object ParquetFileFormat extends Logging { // side, and resemble fake `FileStatus`es there. val partialFileStatusInfo = filesToTouch.map(f => (f.getPath.toString, f.getLen)) + // Set the number of partitions to prevent following schema reads from generating many tasks + // in case of a small number of parquet files. + val numParallelism = Math.min(Math.max(partialFileStatusInfo.size, 1), + sparkSession.sparkContext.defaultParallelism) + // Issues a Spark job to read Parquet schema in parallel. val partiallyMergedSchemas = sparkSession .sparkContext - .parallelize(partialFileStatusInfo) + .parallelize(partialFileStatusInfo, numParallelism) .mapPartitions { iterator => // Resembles fake `FileStatus`es with serialized path and length information. val fakeFileStatuses = iterator.map { case (path, length) => From 96274d73ecd1ffd6069b76df070a3ecfd8409d7a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 14 Jun 2016 17:58:45 -0700 Subject: [PATCH 0648/1470] [SPARK-15933][SQL][STREAMING] Refactored DF reader-writer to use readStream and writeStream for streaming DFs ## What changes were proposed in this pull request? Currently, the DataFrameReader/Writer has method that are needed for streaming and non-streaming DFs. This is quite awkward because each method in them through runtime exception for one case or the other. So rather having half the methods throw runtime exceptions, its just better to have a different reader/writer API for streams. - [x] Python API!! ## How was this patch tested? Existing unit tests + two sets of unit tests for DataFrameReader/Writer and DataStreamReader/Writer. Author: Tathagata Das Closes #13653 from tdas/SPARK-15933. (cherry picked from commit 214adb14b8d1f1c4dce0c97dd6dc09efedbaa643) Signed-off-by: Tathagata Das --- python/pyspark/sql/context.py | 13 + python/pyspark/sql/dataframe.py | 18 +- python/pyspark/sql/readwriter.py | 640 ++++++++++++++---- python/pyspark/sql/session.py | 17 +- python/pyspark/sql/streaming.py | 8 +- python/pyspark/sql/tests.py | 42 +- .../apache/spark/sql/DataFrameReader.scala | 34 +- .../apache/spark/sql/DataFrameWriter.scala | 255 +------ .../scala/org/apache/spark/sql/Dataset.scala | 29 +- .../org/apache/spark/sql/SQLContext.scala | 20 +- .../org/apache/spark/sql/SparkSession.scala | 16 +- .../sql/streaming/DataStreamReader.scala | 294 ++++++++ .../sql/streaming/DataStreamWriter.scala | 386 +++++++++++ .../streaming/ForeachSinkSuite.scala | 7 +- .../ContinuousQueryManagerSuite.scala | 4 +- .../sql/streaming/ContinuousQuerySuite.scala | 4 +- .../sql/streaming/FileStreamSinkSuite.scala | 16 +- .../sql/streaming/FileStreamSourceSuite.scala | 12 +- .../spark/sql/streaming/FileStressSuite.scala | 10 +- .../spark/sql/streaming/MemorySinkSuite.scala | 24 +- .../spark/sql/streaming/StreamSuite.scala | 6 +- .../streaming/StreamingAggregationSuite.scala | 2 + ...cala => DataStreamReaderWriterSuite.scala} | 349 +++------- .../sql/test/DataFrameReaderWriterSuite.scala | 231 +++++++ 24 files changed, 1679 insertions(+), 758 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala rename sql/core/src/test/scala/org/apache/spark/sql/streaming/test/{DataFrameReaderWriterSuite.scala => DataStreamReaderWriterSuite.scala} (55%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 60f62b219b217..a271afe4cf9ba 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -428,6 +428,19 @@ def read(self): """ return DataFrameReader(self) + @property + @since(2.0) + def readStream(self): + """ + Returns a :class:`DataStreamReader` that can be used to read data streams + as a streaming :class:`DataFrame`. + + .. note:: Experimental. + + :return: :class:`DataStreamReader` + """ + return DataStreamReader(self._wrapped) + @property @since(2.0) def streams(self): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 4fa799ac55bdf..0126faf574829 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -33,7 +33,7 @@ from pyspark.traceback_utils import SCCallSiteSync from pyspark.sql.types import _parse_datatype_json_string from pyspark.sql.column import Column, _to_seq, _to_list, _to_java_column -from pyspark.sql.readwriter import DataFrameWriter +from pyspark.sql.readwriter import DataFrameWriter, DataStreamWriter from pyspark.sql.types import * __all__ = ["DataFrame", "DataFrameNaFunctions", "DataFrameStatFunctions"] @@ -172,12 +172,26 @@ def createOrReplaceTempView(self, name): @since(1.4) def write(self): """ - Interface for saving the content of the :class:`DataFrame` out into external storage. + Interface for saving the content of the non-streaming :class:`DataFrame` out into external + storage. :return: :class:`DataFrameWriter` """ return DataFrameWriter(self) + @property + @since(2.0) + def writeStream(self): + """ + Interface for saving the content of the streaming :class:`DataFrame` out into external + storage. + + .. note:: Experimental. + + :return: :class:`DataStreamWriter` + """ + return DataStreamWriter(self) + @property @since(1.3) def schema(self): diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 0f50f672a22d1..ad954d0ad8217 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -137,34 +137,6 @@ def load(self, path=None, format=None, schema=None, **options): else: return self._df(self._jreader.load()) - @since(2.0) - def stream(self, path=None, format=None, schema=None, **options): - """Loads a data stream from a data source and returns it as a :class`DataFrame`. - - .. note:: Experimental. - - :param path: optional string for file-system backed data sources. - :param format: optional string for format of the data source. Default to 'parquet'. - :param schema: optional :class:`StructType` for the input schema. - :param options: all other string options - - >>> df = spark.read.format('text').stream('python/test_support/sql/streaming') - >>> df.isStreaming - True - """ - if format is not None: - self.format(format) - if schema is not None: - self.schema(schema) - self.options(**options) - if path is not None: - if type(path) != str or len(path.strip()) == 0: - raise ValueError("If the path is provided for stream, it needs to be a " + - "non-empty string. List of paths are not supported.") - return self._df(self._jreader.stream(path)) - else: - return self._df(self._jreader.stream()) - @since(1.4) def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, @@ -509,26 +481,6 @@ def mode(self, saveMode): self._jwrite = self._jwrite.mode(saveMode) return self - @since(2.0) - def outputMode(self, outputMode): - """Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - - Options include: - - * `append`:Only the new rows in the streaming DataFrame/Dataset will be written to - the sink - * `complete`:All the rows in the streaming DataFrame/Dataset will be written to the sink - every time these is some updates - - .. note:: Experimental. - - >>> writer = sdf.write.outputMode('append') - """ - if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0: - raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode) - self._jwrite = self._jwrite.outputMode(outputMode) - return self - @since(1.4) def format(self, source): """Specifies the underlying output data source. @@ -571,48 +523,6 @@ def partitionBy(self, *cols): self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols)) return self - @since(2.0) - def queryName(self, queryName): - """Specifies the name of the :class:`ContinuousQuery` that can be started with - :func:`startStream`. This name must be unique among all the currently active queries - in the associated SparkSession. - - .. note:: Experimental. - - :param queryName: unique name for the query - - >>> writer = sdf.write.queryName('streaming_query') - """ - if not queryName or type(queryName) != str or len(queryName.strip()) == 0: - raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName) - self._jwrite = self._jwrite.queryName(queryName) - return self - - @keyword_only - @since(2.0) - def trigger(self, processingTime=None): - """Set the trigger for the stream query. If this is not set it will run the query as fast - as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``. - - .. note:: Experimental. - - :param processingTime: a processing time interval as a string, e.g. '5 seconds', '1 minute'. - - >>> # trigger the query for execution every 5 seconds - >>> writer = sdf.write.trigger(processingTime='5 seconds') - """ - from pyspark.sql.streaming import ProcessingTime - trigger = None - if processingTime is not None: - if type(processingTime) != str or len(processingTime.strip()) == 0: - raise ValueError('The processing time must be a non empty string. Got: %s' % - processingTime) - trigger = ProcessingTime(processingTime) - if trigger is None: - raise ValueError('A trigger was not provided. Supported triggers: processingTime.') - self._jwrite = self._jwrite.trigger(trigger._to_java_trigger(self._spark)) - return self - @since(1.4) def save(self, path=None, format=None, mode=None, partitionBy=None, **options): """Saves the contents of the :class:`DataFrame` to a data source. @@ -644,57 +554,6 @@ def save(self, path=None, format=None, mode=None, partitionBy=None, **options): else: self._jwrite.save(path) - @ignore_unicode_prefix - @since(2.0) - def startStream(self, path=None, format=None, partitionBy=None, queryName=None, **options): - """Streams the contents of the :class:`DataFrame` to a data source. - - The data source is specified by the ``format`` and a set of ``options``. - If ``format`` is not specified, the default data source configured by - ``spark.sql.sources.default`` will be used. - - .. note:: Experimental. - - :param path: the path in a Hadoop supported file system - :param format: the format used to save - - * ``append``: Append contents of this :class:`DataFrame` to existing data. - * ``overwrite``: Overwrite existing data. - * ``ignore``: Silently ignore this operation if data already exists. - * ``error`` (default case): Throw an exception if data already exists. - :param partitionBy: names of partitioning columns - :param queryName: unique name for the query - :param options: All other string options. You may want to provide a `checkpointLocation` - for most streams, however it is not required for a `memory` stream. - - >>> cq = sdf.write.format('memory').queryName('this_query').startStream() - >>> cq.isActive - True - >>> cq.name - u'this_query' - >>> cq.stop() - >>> cq.isActive - False - >>> cq = sdf.write.trigger(processingTime='5 seconds').startStream( - ... queryName='that_query', format='memory') - >>> cq.name - u'that_query' - >>> cq.isActive - True - >>> cq.stop() - """ - self.options(**options) - if partitionBy is not None: - self.partitionBy(partitionBy) - if format is not None: - self.format(format) - if queryName is not None: - self.queryName(queryName) - if path is None: - return self._cq(self._jwrite.startStream()) - else: - return self._cq(self._jwrite.startStream(path)) - @since(1.4) def insertInto(self, tableName, overwrite=False): """Inserts the content of the :class:`DataFrame` to the specified table. @@ -905,6 +764,503 @@ def jdbc(self, url, table, mode=None, properties=None): self._jwrite.mode(mode).jdbc(url, table, jprop) +class DataStreamReader(object): + """ + Interface used to load a streaming :class:`DataFrame` from external storage systems + (e.g. file systems, key-value stores, etc). Use :func:`spark.readStream` + to access this. + + .. note:: Experimental. + + .. versionadded:: 2.0 + """ + + def __init__(self, spark): + self._jreader = spark._ssql_ctx.readStream() + self._spark = spark + + def _df(self, jdf): + from pyspark.sql.dataframe import DataFrame + return DataFrame(jdf, self._spark) + + @since(2.0) + def format(self, source): + """Specifies the input data source format. + + .. note:: Experimental. + + :param source: string, name of the data source, e.g. 'json', 'parquet'. + + """ + self._jreader = self._jreader.format(source) + return self + + @since(2.0) + def schema(self, schema): + """Specifies the input schema. + + Some data sources (e.g. JSON) can infer the input schema automatically from data. + By specifying the schema here, the underlying data source can skip the schema + inference step, and thus speed up data loading. + + .. note:: Experimental. + + :param schema: a StructType object + """ + if not isinstance(schema, StructType): + raise TypeError("schema should be StructType") + jschema = self._spark._ssql_ctx.parseDataType(schema.json()) + self._jreader = self._jreader.schema(jschema) + return self + + @since(2.0) + def option(self, key, value): + """Adds an input option for the underlying data source. + + .. note:: Experimental. + """ + self._jreader = self._jreader.option(key, to_str(value)) + return self + + @since(2.0) + def options(self, **options): + """Adds input options for the underlying data source. + + .. note:: Experimental. + """ + for k in options: + self._jreader = self._jreader.option(k, to_str(options[k])) + return self + + @since(2.0) + def load(self, path=None, format=None, schema=None, **options): + """Loads a data stream from a data source and returns it as a :class`DataFrame`. + + .. note:: Experimental. + + :param path: optional string for file-system backed data sources. + :param format: optional string for format of the data source. Default to 'parquet'. + :param schema: optional :class:`StructType` for the input schema. + :param options: all other string options + + """ + if format is not None: + self.format(format) + if schema is not None: + self.schema(schema) + self.options(**options) + if path is not None: + if type(path) != str or len(path.strip()) == 0: + raise ValueError("If the path is provided for stream, it needs to be a " + + "non-empty string. List of paths are not supported.") + return self._df(self._jreader.load(path)) + else: + return self._df(self._jreader.load()) + + @since(2.0) + def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, + allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, + allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, + mode=None, columnNameOfCorruptRecord=None): + """ + Loads a JSON file stream (one object per line) and returns a :class`DataFrame`. + + If the ``schema`` parameter is not specified, this function goes + through the input once to determine the input schema. + + .. note:: Experimental. + + :param path: string represents path to the JSON dataset, + or RDD of Strings storing JSON objects. + :param schema: an optional :class:`StructType` for the input schema. + :param primitivesAsString: infers all primitive values as a string type. If None is set, + it uses the default value, ``false``. + :param prefersDecimal: infers all floating-point values as a decimal type. If the values + do not fit in decimal, then it infers them as doubles. If None is + set, it uses the default value, ``false``. + :param allowComments: ignores Java/C++ style comment in JSON records. If None is set, + it uses the default value, ``false``. + :param allowUnquotedFieldNames: allows unquoted JSON field names. If None is set, + it uses the default value, ``false``. + :param allowSingleQuotes: allows single quotes in addition to double quotes. If None is + set, it uses the default value, ``true``. + :param allowNumericLeadingZero: allows leading zeros in numbers (e.g. 00012). If None is + set, it uses the default value, ``false``. + :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character + using backslash quoting mechanism. If None is + set, it uses the default value, ``false``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. + + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted \ + record and puts the malformed string into a new field configured by \ + ``columnNameOfCorruptRecord``. When a schema is set by user, it sets \ + ``null`` for extra fields. + * ``DROPMALFORMED`` : ignores the whole corrupted records. + * ``FAILFAST`` : throws an exception when it meets corrupted records. + + :param columnNameOfCorruptRecord: allows renaming the new field having malformed string + created by ``PERMISSIVE`` mode. This overrides + ``spark.sql.columnNameOfCorruptRecord``. If None is set, + it uses the value specified in + ``spark.sql.columnNameOfCorruptRecord``. + + """ + if schema is not None: + self.schema(schema) + if primitivesAsString is not None: + self.option("primitivesAsString", primitivesAsString) + if prefersDecimal is not None: + self.option("prefersDecimal", prefersDecimal) + if allowComments is not None: + self.option("allowComments", allowComments) + if allowUnquotedFieldNames is not None: + self.option("allowUnquotedFieldNames", allowUnquotedFieldNames) + if allowSingleQuotes is not None: + self.option("allowSingleQuotes", allowSingleQuotes) + if allowNumericLeadingZero is not None: + self.option("allowNumericLeadingZero", allowNumericLeadingZero) + if allowBackslashEscapingAnyCharacter is not None: + self.option("allowBackslashEscapingAnyCharacter", allowBackslashEscapingAnyCharacter) + if mode is not None: + self.option("mode", mode) + if columnNameOfCorruptRecord is not None: + self.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + if isinstance(path, basestring): + path = [path] + return self._df(self._jreader.json(path)) + else: + raise TypeError("path can be only a single string") + + @since(2.0) + def parquet(self, path): + """Loads a Parquet file stream, returning the result as a :class:`DataFrame`. + + You can set the following Parquet-specific option(s) for reading Parquet files: + * ``mergeSchema``: sets whether we should merge schemas collected from all \ + Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ + The default value is specified in ``spark.sql.parquet.mergeSchema``. + + .. note:: Experimental. + + """ + if isinstance(path, basestring): + path = [path] + return self._df(self._jreader.parquet(self._spark._sc._jvm.PythonUtils.toSeq(path))) + else: + raise TypeError("path can be only a single string") + + @ignore_unicode_prefix + @since(2.0) + def text(self, path): + """ + Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a + string column named "value", and followed by partitioned columns if there + are any. + + Each line in the text file is a new row in the resulting DataFrame. + + .. note:: Experimental. + + :param paths: string, or list of strings, for input path(s). + + """ + if isinstance(path, basestring): + path = [path] + return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(path))) + else: + raise TypeError("path can be only a single string") + + @since(2.0) + def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, + comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, + ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, + negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, mode=None): + """Loads a CSV file stream and returns the result as a :class:`DataFrame`. + + This function will go through the input once to determine the input schema if + ``inferSchema`` is enabled. To avoid going through the entire data once, disable + ``inferSchema`` option or specify the schema explicitly using ``schema``. + + .. note:: Experimental. + + :param path: string, or list of strings, for input path(s). + :param schema: an optional :class:`StructType` for the input schema. + :param sep: sets the single character as a separator for each field and value. + If None is set, it uses the default value, ``,``. + :param encoding: decodes the CSV files by the given encoding type. If None is set, + it uses the default value, ``UTF-8``. + :param quote: sets the single character used for escaping quoted values where the + separator can be part of the value. If None is set, it uses the default + value, ``"``. If you would like to turn off quotations, you need to set an + empty string. + :param escape: sets the single character used for escaping quotes inside an already + quoted value. If None is set, it uses the default value, ``\``. + :param comment: sets the single character used for skipping lines beginning with this + character. By default (None), it is disabled. + :param header: uses the first line as names of columns. If None is set, it uses the + default value, ``false``. + :param inferSchema: infers the input schema automatically from data. It requires one extra + pass over the data. If None is set, it uses the default value, ``false``. + :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param ignoreTrailingWhiteSpace: defines whether or not trailing whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. + :param nanValue: sets the string representation of a non-number value. If None is set, it + uses the default value, ``NaN``. + :param positiveInf: sets the string representation of a positive infinity value. If None + is set, it uses the default value, ``Inf``. + :param negativeInf: sets the string representation of a negative infinity value. If None + is set, it uses the default value, ``Inf``. + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to both date type and timestamp type. By default, it is None + which means trying to parse times and date by + ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. + :param maxColumns: defines a hard limit of how many columns a record can have. If None is + set, it uses the default value, ``20480``. + :param maxCharsPerColumn: defines the maximum number of characters allowed for any given + value being read. If None is set, it uses the default value, + ``1000000``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. + + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. + When a schema is set by user, it sets ``null`` for extra fields. + * ``DROPMALFORMED`` : ignores the whole corrupted records. + * ``FAILFAST`` : throws an exception when it meets corrupted records. + + """ + if schema is not None: + self.schema(schema) + if sep is not None: + self.option("sep", sep) + if encoding is not None: + self.option("encoding", encoding) + if quote is not None: + self.option("quote", quote) + if escape is not None: + self.option("escape", escape) + if comment is not None: + self.option("comment", comment) + if header is not None: + self.option("header", header) + if inferSchema is not None: + self.option("inferSchema", inferSchema) + if ignoreLeadingWhiteSpace is not None: + self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) + if ignoreTrailingWhiteSpace is not None: + self.option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) + if nullValue is not None: + self.option("nullValue", nullValue) + if nanValue is not None: + self.option("nanValue", nanValue) + if positiveInf is not None: + self.option("positiveInf", positiveInf) + if negativeInf is not None: + self.option("negativeInf", negativeInf) + if dateFormat is not None: + self.option("dateFormat", dateFormat) + if maxColumns is not None: + self.option("maxColumns", maxColumns) + if maxCharsPerColumn is not None: + self.option("maxCharsPerColumn", maxCharsPerColumn) + if mode is not None: + self.option("mode", mode) + if isinstance(path, basestring): + path = [path] + return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path))) + else: + raise TypeError("path can be only a single string") + + +class DataStreamWriter(object): + """ + Interface used to write a streaming :class:`DataFrame` to external storage systems + (e.g. file systems, key-value stores, etc). Use :func:`DataFrame.writeStream` + to access this. + + .. note:: Experimental. + + .. versionadded:: 2.0 + """ + + def __init__(self, df): + self._df = df + self._spark = df.sql_ctx + self._jwrite = df._jdf.writeStream() + + def _cq(self, jcq): + from pyspark.sql.streaming import ContinuousQuery + return ContinuousQuery(jcq) + + @since(2.0) + def outputMode(self, outputMode): + """Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + + Options include: + + * `append`:Only the new rows in the streaming DataFrame/Dataset will be written to + the sink + * `complete`:All the rows in the streaming DataFrame/Dataset will be written to the sink + every time these is some updates + + .. note:: Experimental. + + >>> writer = sdf.writeStream.outputMode('append') + """ + if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0: + raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode) + self._jwrite = self._jwrite.outputMode(outputMode) + return self + + @since(2.0) + def format(self, source): + """Specifies the underlying output data source. + + .. note:: Experimental. + + :param source: string, name of the data source, e.g. 'json', 'parquet'. + + >>> writer = sdf.writeStream.format('json') + """ + self._jwrite = self._jwrite.format(source) + return self + + @since(2.0) + def option(self, key, value): + """Adds an output option for the underlying data source. + + .. note:: Experimental. + """ + self._jwrite = self._jwrite.option(key, to_str(value)) + return self + + @since(2.0) + def options(self, **options): + """Adds output options for the underlying data source. + + .. note:: Experimental. + """ + for k in options: + self._jwrite = self._jwrite.option(k, to_str(options[k])) + return self + + @since(2.0) + def partitionBy(self, *cols): + """Partitions the output by the given columns on the file system. + + If specified, the output is laid out on the file system similar + to Hive's partitioning scheme. + + .. note:: Experimental. + + :param cols: name of columns + + """ + if len(cols) == 1 and isinstance(cols[0], (list, tuple)): + cols = cols[0] + self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols)) + return self + + @since(2.0) + def queryName(self, queryName): + """Specifies the name of the :class:`ContinuousQuery` that can be started with + :func:`startStream`. This name must be unique among all the currently active queries + in the associated SparkSession. + + .. note:: Experimental. + + :param queryName: unique name for the query + + >>> writer = sdf.writeStream.queryName('streaming_query') + """ + if not queryName or type(queryName) != str or len(queryName.strip()) == 0: + raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName) + self._jwrite = self._jwrite.queryName(queryName) + return self + + @keyword_only + @since(2.0) + def trigger(self, processingTime=None): + """Set the trigger for the stream query. If this is not set it will run the query as fast + as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``. + + .. note:: Experimental. + + :param processingTime: a processing time interval as a string, e.g. '5 seconds', '1 minute'. + + >>> # trigger the query for execution every 5 seconds + >>> writer = sdf.writeStream.trigger(processingTime='5 seconds') + """ + from pyspark.sql.streaming import ProcessingTime + trigger = None + if processingTime is not None: + if type(processingTime) != str or len(processingTime.strip()) == 0: + raise ValueError('The processing time must be a non empty string. Got: %s' % + processingTime) + trigger = ProcessingTime(processingTime) + if trigger is None: + raise ValueError('A trigger was not provided. Supported triggers: processingTime.') + self._jwrite = self._jwrite.trigger(trigger._to_java_trigger(self._spark)) + return self + + @ignore_unicode_prefix + @since(2.0) + def start(self, path=None, format=None, partitionBy=None, queryName=None, **options): + """Streams the contents of the :class:`DataFrame` to a data source. + + The data source is specified by the ``format`` and a set of ``options``. + If ``format`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. + + .. note:: Experimental. + + :param path: the path in a Hadoop supported file system + :param format: the format used to save + + * ``append``: Append contents of this :class:`DataFrame` to existing data. + * ``overwrite``: Overwrite existing data. + * ``ignore``: Silently ignore this operation if data already exists. + * ``error`` (default case): Throw an exception if data already exists. + :param partitionBy: names of partitioning columns + :param queryName: unique name for the query + :param options: All other string options. You may want to provide a `checkpointLocation` + for most streams, however it is not required for a `memory` stream. + + >>> cq = sdf.writeStream.format('memory').queryName('this_query').start() + >>> cq.isActive + True + >>> cq.name + u'this_query' + >>> cq.stop() + >>> cq.isActive + False + >>> cq = sdf.writeStream.trigger(processingTime='5 seconds').start( + ... queryName='that_query', format='memory') + >>> cq.name + u'that_query' + >>> cq.isActive + True + >>> cq.stop() + """ + self.options(**options) + if partitionBy is not None: + self.partitionBy(partitionBy) + if format is not None: + self.format(format) + if queryName is not None: + self.queryName(queryName) + if path is None: + return self._cq(self._jwrite.start()) + else: + return self._cq(self._jwrite.start(path)) + + def _test(): import doctest import os @@ -929,7 +1285,7 @@ def _test(): globs['spark'] = spark globs['df'] = spark.read.parquet('python/test_support/sql/parquet_partitioned') globs['sdf'] = \ - spark.read.format('text').stream('python/test_support/sql/streaming') + spark.readStream.format('text').load('python/test_support/sql/streaming') (failure_count, test_count) = doctest.testmod( pyspark.sql.readwriter, globs=globs, diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index f0bf0923b8c75..11c815dd9450b 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -31,7 +31,7 @@ from pyspark.sql.catalog import Catalog from pyspark.sql.conf import RuntimeConfig from pyspark.sql.dataframe import DataFrame -from pyspark.sql.readwriter import DataFrameReader +from pyspark.sql.readwriter import DataFrameReader, DataStreamReader from pyspark.sql.types import Row, DataType, StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _parse_datatype_string from pyspark.sql.utils import install_exception_handler @@ -549,6 +549,19 @@ def read(self): """ return DataFrameReader(self._wrapped) + @property + @since(2.0) + def readStream(self): + """ + Returns a :class:`DataStreamReader` that can be used to read data streams + as a streaming :class:`DataFrame`. + + .. note:: Experimental. + + :return: :class:`DataStreamReader` + """ + return DataStreamReader(self._wrapped) + @property @since(2.0) def streams(self): @@ -556,6 +569,8 @@ def streams(self): :class:`ContinuousQuery` ContinuousQueries active on `this` context. .. note:: Experimental. + + :return: :class:`ContinuousQueryManager` """ from pyspark.sql.streaming import ContinuousQueryManager return ContinuousQueryManager(self._jsparkSession.streams()) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index bb4e62cdd6a56..0edaa51549395 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -119,7 +119,7 @@ def __init__(self, jcqm): def active(self): """Returns a list of active queries associated with this SQLContext - >>> cq = df.write.format('memory').queryName('this_query').startStream() + >>> cq = df.writeStream.format('memory').queryName('this_query').start() >>> cqm = spark.streams >>> # get the list of active continuous queries >>> [q.name for q in cqm.active] @@ -134,7 +134,7 @@ def get(self, id): """Returns an active query from this SQLContext or throws exception if an active query with this name doesn't exist. - >>> cq = df.write.format('memory').queryName('this_query').startStream() + >>> cq = df.writeStream.format('memory').queryName('this_query').start() >>> cq.name u'this_query' >>> cq = spark.streams.get(cq.id) @@ -236,7 +236,7 @@ def _test(): globs = pyspark.sql.streaming.__dict__.copy() try: - spark = SparkSession.builder.enableHiveSupport().getOrCreate() + spark = SparkSession.builder.getOrCreate() except py4j.protocol.Py4JError: spark = SparkSession(sc) @@ -245,7 +245,7 @@ def _test(): globs['spark'] = spark globs['sqlContext'] = SQLContext.getOrCreate(spark.sparkContext) globs['df'] = \ - globs['spark'].read.format('text').stream('python/test_support/sql/streaming') + globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') (failure_count, test_count) = doctest.testmod( pyspark.sql.streaming, globs=globs, diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index e0acde678317d..fee960a1a7bb4 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -892,9 +892,9 @@ def test_save_and_load_builder(self): shutil.rmtree(tmpPath) def test_stream_trigger_takes_keyword_args(self): - df = self.spark.read.format('text').stream('python/test_support/sql/streaming') + df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') try: - df.write.trigger('5 seconds') + df.writeStream.trigger('5 seconds') self.fail("Should have thrown an exception") except TypeError: # should throw error @@ -902,22 +902,25 @@ def test_stream_trigger_takes_keyword_args(self): def test_stream_read_options(self): schema = StructType([StructField("data", StringType(), False)]) - df = self.spark.read.format('text').option('path', 'python/test_support/sql/streaming')\ - .schema(schema).stream() + df = self.spark.readStream\ + .format('text')\ + .option('path', 'python/test_support/sql/streaming')\ + .schema(schema)\ + .load() self.assertTrue(df.isStreaming) self.assertEqual(df.schema.simpleString(), "struct") def test_stream_read_options_overwrite(self): bad_schema = StructType([StructField("test", IntegerType(), False)]) schema = StructType([StructField("data", StringType(), False)]) - df = self.spark.read.format('csv').option('path', 'python/test_support/sql/fake') \ - .schema(bad_schema).stream(path='python/test_support/sql/streaming', - schema=schema, format='text') + df = self.spark.readStream.format('csv').option('path', 'python/test_support/sql/fake') \ + .schema(bad_schema)\ + .load(path='python/test_support/sql/streaming', schema=schema, format='text') self.assertTrue(df.isStreaming) self.assertEqual(df.schema.simpleString(), "struct") def test_stream_save_options(self): - df = self.spark.read.format('text').stream('python/test_support/sql/streaming') + df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') for cq in self.spark._wrapped.streams.active: cq.stop() tmpPath = tempfile.mkdtemp() @@ -925,8 +928,8 @@ def test_stream_save_options(self): self.assertTrue(df.isStreaming) out = os.path.join(tmpPath, 'out') chk = os.path.join(tmpPath, 'chk') - cq = df.write.option('checkpointLocation', chk).queryName('this_query') \ - .format('parquet').outputMode('append').option('path', out).startStream() + cq = df.writeStream.option('checkpointLocation', chk).queryName('this_query') \ + .format('parquet').outputMode('append').option('path', out).start() try: self.assertEqual(cq.name, 'this_query') self.assertTrue(cq.isActive) @@ -941,7 +944,7 @@ def test_stream_save_options(self): shutil.rmtree(tmpPath) def test_stream_save_options_overwrite(self): - df = self.spark.read.format('text').stream('python/test_support/sql/streaming') + df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') for cq in self.spark._wrapped.streams.active: cq.stop() tmpPath = tempfile.mkdtemp() @@ -951,9 +954,10 @@ def test_stream_save_options_overwrite(self): chk = os.path.join(tmpPath, 'chk') fake1 = os.path.join(tmpPath, 'fake1') fake2 = os.path.join(tmpPath, 'fake2') - cq = df.write.option('checkpointLocation', fake1).format('memory').option('path', fake2) \ + cq = df.writeStream.option('checkpointLocation', fake1)\ + .format('memory').option('path', fake2) \ .queryName('fake_query').outputMode('append') \ - .startStream(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) + .start(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) try: self.assertEqual(cq.name, 'this_query') @@ -971,7 +975,7 @@ def test_stream_save_options_overwrite(self): shutil.rmtree(tmpPath) def test_stream_await_termination(self): - df = self.spark.read.format('text').stream('python/test_support/sql/streaming') + df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') for cq in self.spark._wrapped.streams.active: cq.stop() tmpPath = tempfile.mkdtemp() @@ -979,8 +983,8 @@ def test_stream_await_termination(self): self.assertTrue(df.isStreaming) out = os.path.join(tmpPath, 'out') chk = os.path.join(tmpPath, 'chk') - cq = df.write.startStream(path=out, format='parquet', queryName='this_query', - checkpointLocation=chk) + cq = df.writeStream\ + .start(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) try: self.assertTrue(cq.isActive) try: @@ -999,7 +1003,7 @@ def test_stream_await_termination(self): shutil.rmtree(tmpPath) def test_query_manager_await_termination(self): - df = self.spark.read.format('text').stream('python/test_support/sql/streaming') + df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') for cq in self.spark._wrapped.streams.active: cq.stop() tmpPath = tempfile.mkdtemp() @@ -1007,8 +1011,8 @@ def test_query_manager_await_termination(self): self.assertTrue(df.isStreaming) out = os.path.join(tmpPath, 'out') chk = os.path.join(tmpPath, 'chk') - cq = df.write.startStream(path=out, format='parquet', queryName='this_query', - checkpointLocation=chk) + cq = df.writeStream\ + .start(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) try: self.assertTrue(cq.isActive) try: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index dfe31da3f3107..2ae854d04f564 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -22,7 +22,6 @@ import java.util.Properties import scala.collection.JavaConverters._ import org.apache.spark.Partition -import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -30,12 +29,11 @@ import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.execution.datasources.json.{InferSchema, JacksonParser, JSONOptions} -import org.apache.spark.sql.execution.streaming.StreamingRelation import org.apache.spark.sql.types.StructType /** * Interface used to load a [[Dataset]] from external storage systems (e.g. file systems, - * key-value stores, etc) or data streams. Use [[SparkSession.read]] to access this. + * key-value stores, etc). Use [[SparkSession.read]] to access this. * * @since 1.4.0 */ @@ -160,36 +158,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { options = extraOptions.toMap).resolveRelation()) } } - - /** - * :: Experimental :: - * Loads input data stream in as a [[DataFrame]], for data streams that don't require a path - * (e.g. external key-value stores). - * - * @since 2.0.0 - */ - @Experimental - def stream(): DataFrame = { - val dataSource = - DataSource( - sparkSession, - userSpecifiedSchema = userSpecifiedSchema, - className = source, - options = extraOptions.toMap) - Dataset.ofRows(sparkSession, StreamingRelation(dataSource)) - } - - /** - * :: Experimental :: - * Loads input in as a [[DataFrame]], for data streams that read from some path. - * - * @since 2.0.0 - */ - @Experimental - def stream(path: String): DataFrame = { - option("path", path).stream() - } - /** * Construct a [[DataFrame]] representing the database table accessible via JDBC URL * url named table and connection properties. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 171b1378e5f94..60a9d1f020b42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -23,20 +23,15 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, DataSource, HadoopFsRelation} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils -import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.streaming.{ContinuousQuery, OutputMode, ProcessingTime, Trigger} -import org.apache.spark.util.Utils /** * Interface used to write a [[Dataset]] to external storage systems (e.g. file systems, - * key-value stores, etc) or data streams. Use [[Dataset.write]] to access this. + * key-value stores, etc). Use [[Dataset.write]] to access this. * * @since 1.4.0 */ @@ -54,9 +49,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def mode(saveMode: SaveMode): DataFrameWriter[T] = { - // mode() is used for non-continuous queries - // outputMode() is used for continuous queries - assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode this } @@ -71,9 +63,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def mode(saveMode: String): DataFrameWriter[T] = { - // mode() is used for non-continuous queries - // outputMode() is used for continuous queries - assertNotStreaming("mode() can only be called on non-continuous queries") this.mode = saveMode.toLowerCase match { case "overwrite" => SaveMode.Overwrite case "append" => SaveMode.Append @@ -85,76 +74,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { this } - /** - * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be - * written to the sink - * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written - * to the sink every time these is some updates - * - * @since 2.0.0 - */ - @Experimental - def outputMode(outputMode: OutputMode): DataFrameWriter[T] = { - assertStreaming("outputMode() can only be called on continuous queries") - this.outputMode = outputMode - this - } - - /** - * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to - * the sink - * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink - * every time these is some updates - * - * @since 2.0.0 - */ - @Experimental - def outputMode(outputMode: String): DataFrameWriter[T] = { - assertStreaming("outputMode() can only be called on continuous queries") - this.outputMode = outputMode.toLowerCase match { - case "append" => - OutputMode.Append - case "complete" => - OutputMode.Complete - case _ => - throw new IllegalArgumentException(s"Unknown output mode $outputMode. " + - "Accepted output modes are 'append' and 'complete'") - } - this - } - - /** - * :: Experimental :: - * Set the trigger for the stream query. The default value is `ProcessingTime(0)` and it will run - * the query as fast as possible. - * - * Scala Example: - * {{{ - * df.write.trigger(ProcessingTime("10 seconds")) - * - * import scala.concurrent.duration._ - * df.write.trigger(ProcessingTime(10.seconds)) - * }}} - * - * Java Example: - * {{{ - * df.write.trigger(ProcessingTime.create("10 seconds")) - * - * import java.util.concurrent.TimeUnit - * df.write.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) - * }}} - * - * @since 2.0.0 - */ - @Experimental - def trigger(trigger: Trigger): DataFrameWriter[T] = { - assertStreaming("trigger() can only be called on continuous queries") - this.trigger = trigger - this - } - /** * Specifies the underlying output data source. Built-in options include "parquet", "json", etc. * @@ -284,7 +203,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def save(): Unit = { assertNotBucketed("save") - assertNotStreaming("save() can only be called on non-continuous queries") val dataSource = DataSource( df.sparkSession, className = source, @@ -294,148 +212,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { dataSource.write(mode, df) } - - /** - * :: Experimental :: - * Specifies the name of the [[ContinuousQuery]] that can be started with `startStream()`. - * This name must be unique among all the currently active queries in the associated SQLContext. - * - * @since 2.0.0 - */ - @Experimental - def queryName(queryName: String): DataFrameWriter[T] = { - assertStreaming("queryName() can only be called on continuous queries") - this.extraOptions += ("queryName" -> queryName) - this - } - - /** - * :: Experimental :: - * Starts the execution of the streaming query, which will continually output results to the given - * path as new data arrives. The returned [[ContinuousQuery]] object can be used to interact with - * the stream. - * - * @since 2.0.0 - */ - @Experimental - def startStream(path: String): ContinuousQuery = { - option("path", path).startStream() - } - - /** - * :: Experimental :: - * Starts the execution of the streaming query, which will continually output results to the given - * path as new data arrives. The returned [[ContinuousQuery]] object can be used to interact with - * the stream. - * - * @since 2.0.0 - */ - @Experimental - def startStream(): ContinuousQuery = { - assertNotBucketed("startStream") - assertStreaming("startStream() can only be called on continuous queries") - - if (source == "memory") { - if (extraOptions.get("queryName").isEmpty) { - throw new AnalysisException("queryName must be specified for memory sink") - } - - val sink = new MemorySink(df.schema, outputMode) - val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink)) - val query = df.sparkSession.sessionState.continuousQueryManager.startQuery( - extraOptions.get("queryName"), - extraOptions.get("checkpointLocation"), - df, - sink, - outputMode, - useTempCheckpointLocation = true, - recoverFromCheckpointLocation = false, - trigger = trigger) - resultDf.createOrReplaceTempView(query.name) - query - } else { - val dataSource = - DataSource( - df.sparkSession, - className = source, - options = extraOptions.toMap, - partitionColumns = normalizedParCols.getOrElse(Nil)) - df.sparkSession.sessionState.continuousQueryManager.startQuery( - extraOptions.get("queryName"), - extraOptions.get("checkpointLocation"), - df, - dataSource.createSink(outputMode), - outputMode, - trigger = trigger) - } - } - - /** - * :: Experimental :: - * Starts the execution of the streaming query, which will continually send results to the given - * [[ForeachWriter]] as as new data arrives. The [[ForeachWriter]] can be used to send the data - * generated by the [[DataFrame]]/[[Dataset]] to an external system. The returned The returned - * [[ContinuousQuery]] object can be used to interact with the stream. - * - * Scala example: - * {{{ - * datasetOfString.write.foreach(new ForeachWriter[String] { - * - * def open(partitionId: Long, version: Long): Boolean = { - * // open connection - * } - * - * def process(record: String) = { - * // write string to connection - * } - * - * def close(errorOrNull: Throwable): Unit = { - * // close the connection - * } - * }) - * }}} - * - * Java example: - * {{{ - * datasetOfString.write().foreach(new ForeachWriter() { - * - * @Override - * public boolean open(long partitionId, long version) { - * // open connection - * } - * - * @Override - * public void process(String value) { - * // write string to connection - * } - * - * @Override - * public void close(Throwable errorOrNull) { - * // close the connection - * } - * }); - * }}} - * - * @since 2.0.0 - */ - @Experimental - def foreach(writer: ForeachWriter[T]): ContinuousQuery = { - assertNotPartitioned("foreach") - assertNotBucketed("foreach") - assertStreaming( - "foreach() can only be called on streaming Datasets/DataFrames.") - - val sink = new ForeachSink[T](ds.sparkSession.sparkContext.clean(writer))(ds.exprEnc) - df.sparkSession.sessionState.continuousQueryManager.startQuery( - extraOptions.get("queryName"), - extraOptions.get("checkpointLocation"), - df, - sink, - outputMode, - useTempCheckpointLocation = true, - trigger = trigger) - } - /** * Inserts the content of the [[DataFrame]] to the specified table. It requires that * the schema of the [[DataFrame]] is the same as the schema of the table. @@ -467,7 +243,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def insertInto(tableIdent: TableIdentifier): Unit = { assertNotBucketed("insertInto") - assertNotStreaming("insertInto() can only be called on non-continuous queries") val partitions = normalizedParCols.map(_.map(col => col -> (Option.empty[String])).toMap) val overwrite = mode == SaveMode.Overwrite @@ -586,7 +361,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def saveAsTable(tableIdent: TableIdentifier): Unit = { - assertNotStreaming("saveAsTable() can only be called on non-continuous queries") val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) @@ -629,7 +403,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { assertNotPartitioned("jdbc") assertNotBucketed("jdbc") - assertNotStreaming("jdbc() can only be called on non-continuous queries") val props = new Properties() extraOptions.foreach { case (key, value) => @@ -688,7 +461,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def json(path: String): Unit = { - assertNotStreaming("json() can only be called on non-continuous queries") format("json").save(path) } @@ -708,7 +480,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def parquet(path: String): Unit = { - assertNotStreaming("parquet() can only be called on non-continuous queries") format("parquet").save(path) } @@ -728,7 +499,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @note Currently, this method can only be used after enabling Hive support */ def orc(path: String): Unit = { - assertNotStreaming("orc() can only be called on non-continuous queries") format("orc").save(path) } @@ -752,7 +522,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.6.0 */ def text(path: String): Unit = { - assertNotStreaming("text() can only be called on non-continuous queries") format("text").save(path) } @@ -782,7 +551,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 2.0.0 */ def csv(path: String): Unit = { - assertNotStreaming("csv() can only be called on non-continuous queries") format("csv").save(path) } @@ -794,10 +562,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private var mode: SaveMode = SaveMode.ErrorIfExists - private var outputMode: OutputMode = OutputMode.Append - - private var trigger: Trigger = ProcessingTime(0L) - private var extraOptions = new scala.collection.mutable.HashMap[String, String] private var partitioningColumns: Option[Seq[String]] = None @@ -807,21 +571,4 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private var numBuckets: Option[Int] = None private var sortColumnNames: Option[Seq[String]] = None - - /////////////////////////////////////////////////////////////////////////////////////// - // Helper functions - /////////////////////////////////////////////////////////////////////////////////////// - - private def assertNotStreaming(errMsg: String): Unit = { - if (df.isStreaming) { - throw new AnalysisException(errMsg) - } - } - - private def assertStreaming(errMsg: String): Unit = { - if (!df.isStreaming) { - throw new AnalysisException(errMsg) - } - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 53779df3d9c00..f9db325ea241f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -49,7 +49,7 @@ import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython -import org.apache.spark.sql.streaming.ContinuousQuery +import org.apache.spark.sql.streaming.{ContinuousQuery, DataStreamWriter} import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -2407,13 +2407,36 @@ class Dataset[T] private[sql]( /** * :: Experimental :: - * Interface for saving the content of the Dataset out into external storage or streams. + * Interface for saving the content of the non-streaming Dataset out into external storage. * * @group basic * @since 1.6.0 */ @Experimental - def write: DataFrameWriter[T] = new DataFrameWriter[T](this) + def write: DataFrameWriter[T] = { + if (isStreaming) { + logicalPlan.failAnalysis( + "'write' can not be called on streaming Dataset/DataFrame") + } + new DataFrameWriter[T](this) + } + + /** + * :: Experimental :: + * Interface for saving the content of the streaming Dataset out into external storage. + * + * @group basic + * @since 2.0.0 + */ + @Experimental + def writeStream: DataStreamWriter[T] = { + if (!isStreaming) { + logicalPlan.failAnalysis( + "'writeStream' can be called only on streaming Dataset/DataFrame") + } + new DataStreamWriter[T](this) + } + /** * Returns the content of the Dataset as a Dataset of JSON strings. 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 58b4e6c5f604e..33f62915df694 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 @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.streaming.ContinuousQueryManager +import org.apache.spark.sql.streaming.{ContinuousQueryManager, DataStreamReader} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ExecutionListenerManager @@ -491,7 +491,8 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * Returns a [[DataFrameReader]] that can be used to read data and streams in as a [[DataFrame]]. + * Returns a [[DataFrameReader]] that can be used to read non-streaming data in as a + * [[DataFrame]]. * {{{ * sqlContext.read.parquet("/path/to/file.parquet") * sqlContext.read.schema(schema).json("/path/to/file.json") @@ -502,6 +503,21 @@ class SQLContext private[sql](val sparkSession: SparkSession) */ def read: DataFrameReader = sparkSession.read + + /** + * :: Experimental :: + * Returns a [[DataStreamReader]] that can be used to read streaming data in as a [[DataFrame]]. + * {{{ + * sparkSession.readStream.parquet("/path/to/directory/of/parquet/files") + * sparkSession.readStream.schema(schema).json("/path/to/directory/of/json/files") + * }}} + * + * @since 2.0.0 + */ + @Experimental + def readStream: DataStreamReader = sparkSession.readStream + + /** * Creates an external table from the given path and returns the corresponding DataFrame. * It will use the default data source configured by spark.sql.sources.default. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 01c2e3ad29e48..9137a735dd4da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -574,7 +574,8 @@ class SparkSession private( } /** - * Returns a [[DataFrameReader]] that can be used to read data and streams in as a [[DataFrame]]. + * Returns a [[DataFrameReader]] that can be used to read non-streaming data in as a + * [[DataFrame]]. * {{{ * sparkSession.read.parquet("/path/to/file.parquet") * sparkSession.read.schema(schema).json("/path/to/file.json") @@ -584,6 +585,19 @@ class SparkSession private( */ def read: DataFrameReader = new DataFrameReader(self) + /** + * :: Experimental :: + * Returns a [[DataStreamReader]] that can be used to read streaming data in as a [[DataFrame]]. + * {{{ + * sparkSession.readStream.parquet("/path/to/directory/of/parquet/files") + * sparkSession.readStream.schema(schema).json("/path/to/directory/of/json/files") + * }}} + * + * @since 2.0.0 + */ + @Experimental + def readStream: DataStreamReader = new DataStreamReader(self) + // scalastyle:off // Disable style checker so "implicits" object can start with lowercase i diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala new file mode 100644 index 0000000000000..248247a257d94 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -0,0 +1,294 @@ +/* + * 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.streaming + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.StreamingRelation +import org.apache.spark.sql.types.StructType + +/** + * Interface used to load a streaming [[Dataset]] from external storage systems (e.g. file systems, + * key-value stores, etc). Use [[SparkSession.readStream]] to access this. + * + * @since 2.0.0 + */ +@Experimental +final class DataStreamReader private[sql](sparkSession: SparkSession) extends Logging { + /** + * :: Experimental :: + * Specifies the input data source format. + * + * @since 2.0.0 + */ + @Experimental + def format(source: String): DataStreamReader = { + this.source = source + this + } + + /** + * :: Experimental :: + * Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema + * automatically from data. By specifying the schema here, the underlying data source can + * skip the schema inference step, and thus speed up data loading. + * + * @since 2.0.0 + */ + @Experimental + def schema(schema: StructType): DataStreamReader = { + this.userSpecifiedSchema = Option(schema) + this + } + + /** + * :: Experimental :: + * Adds an input option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: String): DataStreamReader = { + this.extraOptions += (key -> value) + this + } + + /** + * :: Experimental :: + * Adds an input option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Boolean): DataStreamReader = option(key, value.toString) + + /** + * :: Experimental :: + * Adds an input option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Long): DataStreamReader = option(key, value.toString) + + /** + * :: Experimental :: + * Adds an input option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Double): DataStreamReader = option(key, value.toString) + + /** + * :: Experimental :: + * (Scala-specific) Adds input options for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def options(options: scala.collection.Map[String, String]): DataStreamReader = { + this.extraOptions ++= options + this + } + + /** + * :: Experimental :: + * Adds input options for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def options(options: java.util.Map[String, String]): DataStreamReader = { + this.options(options.asScala) + this + } + + + /** + * :: Experimental :: + * Loads input data stream in as a [[DataFrame]], for data streams that don't require a path + * (e.g. external key-value stores). + * + * @since 2.0.0 + */ + @Experimental + def load(): DataFrame = { + val dataSource = + DataSource( + sparkSession, + userSpecifiedSchema = userSpecifiedSchema, + className = source, + options = extraOptions.toMap) + Dataset.ofRows(sparkSession, StreamingRelation(dataSource)) + } + + /** + * :: Experimental :: + * Loads input in as a [[DataFrame]], for data streams that read from some path. + * + * @since 2.0.0 + */ + @Experimental + def load(path: String): DataFrame = { + option("path", path).load() + } + + /** + * :: Experimental :: + * Loads a JSON file stream (one object per line) and returns the result as a [[DataFrame]]. + * + * This function goes through the input once to determine the input schema. If you know the + * schema in advance, use the version that specifies the schema to avoid the extra scan. + * + * You can set the following JSON-specific options to deal with non-standard JSON files: + *
    • `primitivesAsString` (default `false`): infers all primitive values as a string type
    • + *
    • `prefersDecimal` (default `false`): infers all floating-point values as a decimal + * type. If the values do not fit in decimal, then it infers them as doubles.
    • + *
    • `allowComments` (default `false`): ignores Java/C++ style comment in JSON records
    • + *
    • `allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names
    • + *
    • `allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes + *
    • + *
    • `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers + * (e.g. 00012)
    • + *
    • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all + * character using backslash quoting mechanism
    • + *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records + * during parsing.
    • + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the + * malformed string into a new field configured by `columnNameOfCorruptRecord`. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + *
    • `columnNameOfCorruptRecord` (default is the value specified in + * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string + * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • + * + * @since 2.0.0 + */ + @Experimental + def json(path: String): DataFrame = format("json").load(path) + + /** + * :: Experimental :: + * Loads a CSV file stream and returns the result as a [[DataFrame]]. + * + * This function will go through the input once to determine the input schema if `inferSchema` + * is enabled. To avoid going through the entire data once, disable `inferSchema` option or + * specify the schema explicitly using [[schema]]. + * + * You can set the following CSV-specific options to deal with CSV files: + *
    • `sep` (default `,`): sets the single character as a separator for each + * field and value.
    • + *
    • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding + * type.
    • + *
    • `quote` (default `"`): sets the single character used for escaping quoted values where + * the separator can be part of the value. If you would like to turn off quotations, you need to + * set not `null` but an empty string. This behaviour is different form + * `com.databricks.spark.csv`.
    • + *
    • `escape` (default `\`): sets the single character used for escaping quotes inside + * an already quoted value.
    • + *
    • `comment` (default empty string): sets the single character used for skipping lines + * beginning with this character. By default, it is disabled.
    • + *
    • `header` (default `false`): uses the first line as names of columns.
    • + *
    • `inferSchema` (default `false`): infers the input schema automatically from data. It + * requires one extra pass over the data.
    • + *
    • `ignoreLeadingWhiteSpace` (default `false`): defines whether or not leading whitespaces + * from values being read should be skipped.
    • + *
    • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing + * whitespaces from values being read should be skipped.
    • + *
    • `nullValue` (default empty string): sets the string representation of a null value.
    • + *
    • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
    • + *
    • `positiveInf` (default `Inf`): sets the string representation of a positive infinity + * value.
    • + *
    • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity + * value.
    • + *
    • `dateFormat` (default `null`): sets the string that indicates a date format. Custom date + * formats follow the formats at `java.text.SimpleDateFormat`. This applies to both date type + * and timestamp type. By default, it is `null` which means trying to parse times and date by + * `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()`.
    • + *
    • `maxColumns` (default `20480`): defines a hard limit of how many columns + * a record can have.
    • + *
    • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed + * for any given value being read.
    • + *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records + * during parsing.
    • + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + * + * @since 2.0.0 + */ + @Experimental + def csv(path: String): DataFrame = format("csv").load(path) + + /** + * :: Experimental :: + * Loads a Parquet file stream, returning the result as a [[DataFrame]]. + * + * You can set the following Parquet-specific option(s) for reading Parquet files: + *
    • `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets + * whether we should merge schemas collected from all Parquet part-files. This will override + * `spark.sql.parquet.mergeSchema`.
    • + * + * @since 2.0.0 + */ + @Experimental + def parquet(path: String): DataFrame = { + format("parquet").load(path) + } + + /** + * :: Experimental :: + * Loads text files and returns a [[DataFrame]] whose schema starts with a string column named + * "value", and followed by partitioned columns if there are any. + * + * Each line in the text files is a new row in the resulting DataFrame. For example: + * {{{ + * // Scala: + * spark.readStream.text("/path/to/directory/") + * + * // Java: + * spark.readStream().text("/path/to/directory/") + * }}} + * + * @since 2.0.0 + */ + @Experimental + def text(path: String): DataFrame = format("text").load(path) + + + /////////////////////////////////////////////////////////////////////////////////////// + // Builder pattern config options + /////////////////////////////////////////////////////////////////////////////////////// + + private var source: String = sparkSession.sessionState.conf.defaultDataSourceName + + private var userSpecifiedSchema: Option[StructType] = None + + private var extraOptions = new scala.collection.mutable.HashMap[String, String] +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala new file mode 100644 index 0000000000000..b035ff7938bae --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -0,0 +1,386 @@ +/* + * 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.streaming + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, ForeachWriter} +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.{ForeachSink, MemoryPlan, MemorySink} + +/** + * :: Experimental :: + * Interface used to write a streaming [[Dataset]] to external storage systems (e.g. file systems, + * key-value stores, etc). Use [[Dataset.writeStream]] to access this. + * + * @since 2.0.0 + */ +@Experimental +final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { + + private val df = ds.toDF() + + /** + * :: Experimental :: + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be + * written to the sink + * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time these is some updates + * + * @since 2.0.0 + */ + @Experimental + def outputMode(outputMode: OutputMode): DataStreamWriter[T] = { + this.outputMode = outputMode + this + } + + + /** + * :: Experimental :: + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to + * the sink + * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink + * every time these is some updates + * + * @since 2.0.0 + */ + @Experimental + def outputMode(outputMode: String): DataStreamWriter[T] = { + this.outputMode = outputMode.toLowerCase match { + case "append" => + OutputMode.Append + case "complete" => + OutputMode.Complete + case _ => + throw new IllegalArgumentException(s"Unknown output mode $outputMode. " + + "Accepted output modes are 'append' and 'complete'") + } + this + } + + /** + * :: Experimental :: + * Set the trigger for the stream query. The default value is `ProcessingTime(0)` and it will run + * the query as fast as possible. + * + * Scala Example: + * {{{ + * df.writeStream.trigger(ProcessingTime("10 seconds")) + * + * import scala.concurrent.duration._ + * df.writeStream.trigger(ProcessingTime(10.seconds)) + * }}} + * + * Java Example: + * {{{ + * df.writeStream().trigger(ProcessingTime.create("10 seconds")) + * + * import java.util.concurrent.TimeUnit + * df.writeStream().trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * }}} + * + * @since 2.0.0 + */ + @Experimental + def trigger(trigger: Trigger): DataStreamWriter[T] = { + this.trigger = trigger + this + } + + + /** + * :: Experimental :: + * Specifies the name of the [[ContinuousQuery]] that can be started with `startStream()`. + * This name must be unique among all the currently active queries in the associated SQLContext. + * + * @since 2.0.0 + */ + @Experimental + def queryName(queryName: String): DataStreamWriter[T] = { + this.extraOptions += ("queryName" -> queryName) + this + } + + /** + * :: Experimental :: + * Specifies the underlying output data source. Built-in options include "parquet", "json", etc. + * + * @since 2.0.0 + */ + @Experimental + def format(source: String): DataStreamWriter[T] = { + this.source = source + this + } + + /** + * Partitions the output by the given columns on the file system. If specified, the output is + * laid out on the file system similar to Hive's partitioning scheme. As an example, when we + * partition a dataset by year and then month, the directory layout would look like: + * + * - year=2016/month=01/ + * - year=2016/month=02/ + * + * Partitioning is one of the most widely used techniques to optimize physical data layout. + * It provides a coarse-grained index for skipping unnecessary data reads when queries have + * predicates on the partitioned columns. In order for partitioning to work well, the number + * of distinct values in each column should typically be less than tens of thousands. + * + * This was initially applicable for Parquet but in 1.5+ covers JSON, text, ORC and avro as well. + * + * @since 1.4.0 + */ + @scala.annotation.varargs + def partitionBy(colNames: String*): DataStreamWriter[T] = { + this.partitioningColumns = Option(colNames) + this + } + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: String): DataStreamWriter[T] = { + this.extraOptions += (key -> value) + this + } + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Boolean): DataStreamWriter[T] = option(key, value.toString) + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Long): DataStreamWriter[T] = option(key, value.toString) + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Double): DataStreamWriter[T] = option(key, value.toString) + + /** + * :: Experimental :: + * (Scala-specific) Adds output options for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def options(options: scala.collection.Map[String, String]): DataStreamWriter[T] = { + this.extraOptions ++= options + this + } + + /** + * :: Experimental :: + * Adds output options for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def options(options: java.util.Map[String, String]): DataStreamWriter[T] = { + this.options(options.asScala) + this + } + + /** + * :: Experimental :: + * Starts the execution of the streaming query, which will continually output results to the given + * path as new data arrives. The returned [[ContinuousQuery]] object can be used to interact with + * the stream. + * + * @since 2.0.0 + */ + @Experimental + def start(path: String): ContinuousQuery = { + option("path", path).start() + } + + /** + * :: Experimental :: + * Starts the execution of the streaming query, which will continually output results to the given + * path as new data arrives. The returned [[ContinuousQuery]] object can be used to interact with + * the stream. + * + * @since 2.0.0 + */ + @Experimental + def start(): ContinuousQuery = { + if (source == "memory") { + assertNotPartitioned("memory") + if (extraOptions.get("queryName").isEmpty) { + throw new AnalysisException("queryName must be specified for memory sink") + } + + val sink = new MemorySink(df.schema, outputMode) + val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink)) + val query = df.sparkSession.sessionState.continuousQueryManager.startQuery( + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), + df, + sink, + outputMode, + useTempCheckpointLocation = true, + recoverFromCheckpointLocation = false, + trigger = trigger) + resultDf.createOrReplaceTempView(query.name) + query + } else if (source == "foreach") { + assertNotPartitioned("foreach") + val sink = new ForeachSink[T](foreachWriter)(ds.exprEnc) + df.sparkSession.sessionState.continuousQueryManager.startQuery( + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), + df, + sink, + outputMode, + useTempCheckpointLocation = true, + trigger = trigger) + } else { + val dataSource = + DataSource( + df.sparkSession, + className = source, + options = extraOptions.toMap, + partitionColumns = normalizedParCols.getOrElse(Nil)) + df.sparkSession.sessionState.continuousQueryManager.startQuery( + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), + df, + dataSource.createSink(outputMode), + outputMode, + trigger = trigger) + } + } + + /** + * :: Experimental :: + * Starts the execution of the streaming query, which will continually send results to the given + * [[ForeachWriter]] as as new data arrives. The [[ForeachWriter]] can be used to send the data + * generated by the [[DataFrame]]/[[Dataset]] to an external system. + * + * Scala example: + * {{{ + * datasetOfString.writeStream.foreach(new ForeachWriter[String] { + * + * def open(partitionId: Long, version: Long): Boolean = { + * // open connection + * } + * + * def process(record: String) = { + * // write string to connection + * } + * + * def close(errorOrNull: Throwable): Unit = { + * // close the connection + * } + * }).start() + * }}} + * + * Java example: + * {{{ + * datasetOfString.writeStream().foreach(new ForeachWriter() { + * + * @Override + * public boolean open(long partitionId, long version) { + * // open connection + * } + * + * @Override + * public void process(String value) { + * // write string to connection + * } + * + * @Override + * public void close(Throwable errorOrNull) { + * // close the connection + * } + * }).start(); + * }}} + * + * @since 2.0.0 + */ + @Experimental + def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { + this.source = "foreach" + this.foreachWriter = if (writer != null) { + ds.sparkSession.sparkContext.clean(writer) + } else { + throw new IllegalArgumentException("foreach writer cannot be null") + } + this + } + + private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { cols => + cols.map(normalize(_, "Partition")) + } + + /** + * The given column name may not be equal to any of the existing column names if we were in + * case-insensitive context. Normalize the given column name to the real one so that we don't + * need to care about case sensitivity afterwards. + */ + private def normalize(columnName: String, columnType: String): String = { + val validColumnNames = df.logicalPlan.output.map(_.name) + validColumnNames.find(df.sparkSession.sessionState.analyzer.resolver(_, columnName)) + .getOrElse(throw new AnalysisException(s"$columnType column $columnName not found in " + + s"existing columns (${validColumnNames.mkString(", ")})")) + } + + private def assertNotPartitioned(operation: String): Unit = { + if (partitioningColumns.isDefined) { + throw new AnalysisException(s"'$operation' does not support partitioning") + } + } + + /////////////////////////////////////////////////////////////////////////////////////// + // Builder pattern config options + /////////////////////////////////////////////////////////////////////////////////////// + + private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName + + private var outputMode: OutputMode = OutputMode.Append + + private var trigger: Trigger = ProcessingTime(0L) + + private var extraOptions = new scala.collection.mutable.HashMap[String, String] + + private var foreachWriter: ForeachWriter[T] = null + + private var partitioningColumns: Option[Seq[String]] = None +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala index e1fb3b947837b..6ff597c16bb28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala @@ -38,9 +38,10 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf test("foreach") { withTempDir { checkpointDir => val input = MemoryStream[Int] - val query = input.toDS().repartition(2).write + val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) .foreach(new TestForeachWriter()) + .start() input.addData(1, 2, 3, 4) query.processAllAvailable() @@ -70,14 +71,14 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf test("foreach with error") { withTempDir { checkpointDir => val input = MemoryStream[Int] - val query = input.toDS().repartition(1).write + val query = input.toDS().repartition(1).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) .foreach(new TestForeachWriter() { override def process(value: Int): Unit = { super.process(value) throw new RuntimeException("error") } - }) + }).start() input.addData(1, 2, 3, 4) query.processAllAvailable() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala index f81608bdb85e3..ef2fcbf73e360 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala @@ -225,12 +225,12 @@ class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { val metadataRoot = Utils.createTempDir(namePrefix = "streaming.checkpoint").getCanonicalPath query = - df.write + df.writeStream .format("memory") .queryName(s"query$i") .option("checkpointLocation", metadataRoot) .outputMode("append") - .startStream() + .start() .asInstanceOf[StreamExecution] } catch { case NonFatal(e) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala index 43a88576cf9f4..ad6bc27729597 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala @@ -39,12 +39,12 @@ class ContinuousQuerySuite extends StreamTest with BeforeAndAfter { def startQuery(queryName: String): ContinuousQuery = { val metadataRoot = Utils.createTempDir(namePrefix = "streaming.checkpoint").getCanonicalPath - val writer = mapped.write + val writer = mapped.writeStream writer .queryName(queryName) .format("memory") .option("checkpointLocation", metadataRoot) - .startStream() + .start() } val q1 = startQuery("q1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index bb3063dc34ae3..a5acc970e3a78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -128,10 +128,10 @@ class FileStreamSinkSuite extends StreamTest { try { query = - df.write - .format("parquet") + df.writeStream .option("checkpointLocation", checkpointDir) - .startStream(outputDir) + .format("parquet") + .start(outputDir) inputData.addData(1, 2, 3) @@ -162,11 +162,11 @@ class FileStreamSinkSuite extends StreamTest { query = ds.map(i => (i, i * 1000)) .toDF("id", "value") - .write - .format("parquet") + .writeStream .partitionBy("id") .option("checkpointLocation", checkpointDir) - .startStream(outputDir) + .format("parquet") + .start(outputDir) inputData.addData(1, 2, 3) failAfter(streamingTimeout) { @@ -246,13 +246,13 @@ class FileStreamSinkSuite extends StreamTest { val writer = ds.map(i => (i, i * 1000)) .toDF("id", "value") - .write + .writeStream if (format.nonEmpty) { writer.format(format.get) } query = writer .option("checkpointLocation", checkpointDir) - .startStream(outputDir) + .start(outputDir) } finally { if (query != null) { query.stop() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index f681b8878d9ed..6971f93b230f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -107,11 +107,11 @@ class FileStreamSourceTest extends StreamTest with SharedSQLContext { schema: Option[StructType] = None): DataFrame = { val reader = if (schema.isDefined) { - spark.read.format(format).schema(schema.get) + spark.readStream.format(format).schema(schema.get) } else { - spark.read.format(format) + spark.readStream.format(format) } - reader.stream(path) + reader.load(path) } protected def getSourceFromFileStream(df: DataFrame): FileStreamSource = { @@ -153,14 +153,14 @@ class FileStreamSourceSuite extends FileStreamSourceTest { format: Option[String], path: Option[String], schema: Option[StructType] = None): StructType = { - val reader = spark.read + val reader = spark.readStream format.foreach(reader.format) schema.foreach(reader.schema) val df = if (path.isDefined) { - reader.stream(path.get) + reader.load(path.get) } else { - reader.stream() + reader.load() } df.queryExecution.analyzed .collect { case s @ StreamingRelation(dataSource, _, _) => s.schema }.head diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala index 1c0fb34dd0191..0e157cf7267dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala @@ -98,7 +98,7 @@ class FileStressSuite extends StreamTest { } writer.start() - val input = spark.read.format("text").stream(inputDir) + val input = spark.readStream.format("text").load(inputDir) def startStream(): ContinuousQuery = { val output = input @@ -116,17 +116,17 @@ class FileStressSuite extends StreamTest { if (partitionWrites) { output - .write + .writeStream .partitionBy("id") .format("parquet") .option("checkpointLocation", checkpoint) - .startStream(outputDir) + .start(outputDir) } else { output - .write + .writeStream .format("parquet") .option("checkpointLocation", checkpoint) - .startStream(outputDir) + .start(outputDir) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala index 9aada0b18dd8d..310d75630272b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala @@ -140,11 +140,11 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { test("registering as a table in Append output mode") { val input = MemoryStream[Int] - val query = input.toDF().write + val query = input.toDF().writeStream .format("memory") .outputMode("append") .queryName("memStream") - .startStream() + .start() input.addData(1, 2, 3) query.processAllAvailable() @@ -166,11 +166,11 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { val query = input.toDF() .groupBy("value") .count() - .write + .writeStream .format("memory") .outputMode("complete") .queryName("memStream") - .startStream() + .start() input.addData(1, 2, 3) query.processAllAvailable() @@ -191,10 +191,10 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { // Ignore the stress test as it takes several minutes to run (0 until 1000).foreach { _ => val input = MemoryStream[Int] - val query = input.toDF().write + val query = input.toDF().writeStream .format("memory") .queryName("memStream") - .startStream() + .start() input.addData(1, 2, 3) query.processAllAvailable() @@ -215,9 +215,9 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { test("error when no name is specified") { val error = intercept[AnalysisException] { val input = MemoryStream[Int] - val query = input.toDF().write + val query = input.toDF().writeStream .format("memory") - .startStream() + .start() } assert(error.message contains "queryName must be specified") @@ -227,21 +227,21 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { val location = Utils.createTempDir(namePrefix = "steaming.checkpoint").getCanonicalPath val input = MemoryStream[Int] - val query = input.toDF().write + val query = input.toDF().writeStream .format("memory") .queryName("memStream") .option("checkpointLocation", location) - .startStream() + .start() input.addData(1, 2, 3) query.processAllAvailable() query.stop() intercept[AnalysisException] { - input.toDF().write + input.toDF().writeStream .format("memory") .queryName("memStream") .option("checkpointLocation", location) - .startStream() + .start() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 9414b1ce4019b..786404a589581 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -89,9 +89,9 @@ class StreamSuite extends StreamTest { def assertDF(df: DataFrame) { withTempDir { outputDir => withTempDir { checkpointDir => - val query = df.write.format("parquet") + val query = df.writeStream.format("parquet") .option("checkpointLocation", checkpointDir.getAbsolutePath) - .startStream(outputDir.getAbsolutePath) + .start(outputDir.getAbsolutePath) try { query.processAllAvailable() val outputDf = spark.read.parquet(outputDir.getAbsolutePath).as[Long] @@ -103,7 +103,7 @@ class StreamSuite extends StreamTest { } } - val df = spark.read.format(classOf[FakeDefaultSource].getName).stream() + val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load() assertDF(df) assertDF(df) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 8681199817fe6..7f44227ec46fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -40,6 +40,8 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll { import testImplicits._ + + test("simple count, update mode") { val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala similarity index 55% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 6e0d66ae7f19a..c6d374f75467a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -101,7 +101,7 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { } } -class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { +class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { private def newMetadataDir = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath @@ -110,25 +110,38 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { spark.streams.active.foreach(_.stop()) } + test("write cannot be called on streaming datasets") { + val e = intercept[AnalysisException] { + spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + .write + .save() + } + Seq("'write'", "not", "streaming Dataset/DataFrame").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + test("resolve default source") { - spark.read + spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream() - .write + .load() + .writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) - .startStream() + .start() .stop() } test("resolve full class") { - spark.read + spark.readStream .format("org.apache.spark.sql.streaming.test.DefaultSource") - .stream() - .write + .load() + .writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) - .startStream() + .start() .stop() } @@ -136,12 +149,12 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { val map = new java.util.HashMap[String, String] map.put("opt3", "3") - val df = spark.read + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") .option("opt1", "1") .options(Map("opt2" -> "2")) .options(map) - .stream() + .load() assert(LastOptions.parameters("opt1") == "1") assert(LastOptions.parameters("opt2") == "2") @@ -149,13 +162,13 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { LastOptions.clear() - df.write + df.writeStream .format("org.apache.spark.sql.streaming.test") .option("opt1", "1") .options(Map("opt2" -> "2")) .options(map) .option("checkpointLocation", newMetadataDir) - .startStream() + .start() .stop() assert(LastOptions.parameters("opt1") == "1") @@ -164,84 +177,84 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { } test("partitioning") { - val df = spark.read + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream() + .load() - df.write + df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) - .startStream() + .start() .stop() assert(LastOptions.partitionColumns == Nil) - df.write + df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) .partitionBy("a") - .startStream() + .start() .stop() assert(LastOptions.partitionColumns == Seq("a")) withSQLConf("spark.sql.caseSensitive" -> "false") { - df.write + df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) .partitionBy("A") - .startStream() + .start() .stop() assert(LastOptions.partitionColumns == Seq("a")) } intercept[AnalysisException] { - df.write + df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) .partitionBy("b") - .startStream() + .start() .stop() } } test("stream paths") { - val df = spark.read + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) - .stream("/test") + .load("/test") assert(LastOptions.parameters("path") == "/test") LastOptions.clear() - df.write + df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) - .startStream("/test") + .start("/test") .stop() assert(LastOptions.parameters("path") == "/test") } test("test different data types for options") { - val df = spark.read + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") .option("intOpt", 56) .option("boolOpt", false) .option("doubleOpt", 6.7) - .stream("/test") + .load("/test") assert(LastOptions.parameters("intOpt") == "56") assert(LastOptions.parameters("boolOpt") == "false") assert(LastOptions.parameters("doubleOpt") == "6.7") LastOptions.clear() - df.write + df.writeStream .format("org.apache.spark.sql.streaming.test") .option("intOpt", 56) .option("boolOpt", false) .option("doubleOpt", 6.7) .option("checkpointLocation", newMetadataDir) - .startStream("/test") + .start("/test") .stop() assert(LastOptions.parameters("intOpt") == "56") @@ -253,25 +266,25 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { /** Start a query with a specific name */ def startQueryWithName(name: String = ""): ContinuousQuery = { - spark.read + spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream("/test") - .write + .load("/test") + .writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) .queryName(name) - .startStream() + .start() } /** Start a query without specifying a name */ def startQueryWithoutName(): ContinuousQuery = { - spark.read + spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream("/test") - .write + .load("/test") + .writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) - .startStream() + .start() } /** Get the names of active streams */ @@ -311,24 +324,24 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { } test("trigger") { - val df = spark.read + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream("/test") + .load("/test") - var q = df.write + var q = df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) .trigger(ProcessingTime(10.seconds)) - .startStream() + .start() q.stop() assert(q.asInstanceOf[StreamExecution].trigger == ProcessingTime(10000)) - q = df.write + q = df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) .trigger(ProcessingTime.create(100, TimeUnit.SECONDS)) - .startStream() + .start() q.stop() assert(q.asInstanceOf[StreamExecution].trigger == ProcessingTime(100000)) @@ -339,19 +352,19 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { val checkpointLocation = newMetadataDir - val df1 = spark.read + val df1 = spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream() + .load() - val df2 = spark.read + val df2 = spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream() + .load() - val q = df1.union(df2).write + val q = df1.union(df2).writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", checkpointLocation) .trigger(ProcessingTime(10.seconds)) - .startStream() + .start() q.stop() verify(LastOptions.mockStreamSourceProvider).createSource( @@ -371,76 +384,12 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { private def newTextInput = Utils.createTempDir(namePrefix = "text").getCanonicalPath - test("check trigger() can only be called on continuous queries") { - val df = spark.read.text(newTextInput) - val w = df.write.option("checkpointLocation", newMetadataDir) - val e = intercept[AnalysisException](w.trigger(ProcessingTime("10 seconds"))) - assert(e.getMessage == "trigger() can only be called on continuous queries;") - } - - test("check queryName() can only be called on continuous queries") { - val df = spark.read.text(newTextInput) - val w = df.write.option("checkpointLocation", newMetadataDir) - val e = intercept[AnalysisException](w.queryName("queryName")) - assert(e.getMessage == "queryName() can only be called on continuous queries;") - } - - test("check startStream() can only be called on continuous queries") { - val df = spark.read.text(newTextInput) - val w = df.write.option("checkpointLocation", newMetadataDir) - val e = intercept[AnalysisException](w.startStream()) - assert(e.getMessage == "startStream() can only be called on continuous queries;") - } - - test("check startStream(path) can only be called on continuous queries") { - val df = spark.read.text(newTextInput) - val w = df.write.option("checkpointLocation", newMetadataDir) - val e = intercept[AnalysisException](w.startStream("non_exist_path")) - assert(e.getMessage == "startStream() can only be called on continuous queries;") - } - - test("check mode(SaveMode) can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.mode(SaveMode.Append)) - assert(e.getMessage == "mode() can only be called on non-continuous queries;") - } - - test("check mode(string) can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.mode("append")) - assert(e.getMessage == "mode() can only be called on non-continuous queries;") - } - - test("check outputMode(OutputMode) can only be called on continuous queries") { - val df = spark.read.text(newTextInput) - val w = df.write.option("checkpointLocation", newMetadataDir) - val e = intercept[AnalysisException](w.outputMode(OutputMode.Append)) - Seq("outputmode", "continuous queries").foreach { s => - assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) - } - } - - test("check outputMode(string) can only be called on continuous queries") { - val df = spark.read.text(newTextInput) - val w = df.write.option("checkpointLocation", newMetadataDir) - val e = intercept[AnalysisException](w.outputMode("append")) - Seq("outputmode", "continuous queries").foreach { s => - assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) - } - } - test("check outputMode(string) throws exception on unsupported modes") { def testError(outputMode: String): Unit = { - val df = spark.read + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write + .load() + val w = df.writeStream val e = intercept[IllegalArgumentException](w.outputMode(outputMode)) Seq("output mode", "unknown", outputMode).foreach { s => assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) @@ -450,159 +399,46 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { testError("Xyz") } - test("check bucketBy() can only be called on non-continuous queries") { - val df = spark.read + test("check foreach() catches null writers") { + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.bucketBy(1, "text").startStream()) - assert(e.getMessage == "'startStream' does not support bucketing right now;") - } - - test("check sortBy() can only be called on non-continuous queries;") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.sortBy("text").startStream()) - assert(e.getMessage == "'startStream' does not support bucketing right now;") - } + .load() - test("check save(path) can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.save("non_exist_path")) - assert(e.getMessage == "save() can only be called on non-continuous queries;") - } - - test("check save() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.save()) - assert(e.getMessage == "save() can only be called on non-continuous queries;") - } - - test("check insertInto() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.insertInto("non_exsit_table")) - assert(e.getMessage == "insertInto() can only be called on non-continuous queries;") - } - - test("check saveAsTable() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.saveAsTable("non_exsit_table")) - assert(e.getMessage == "saveAsTable() can only be called on non-continuous queries;") - } - - test("check jdbc() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.jdbc(null, null, null)) - assert(e.getMessage == "jdbc() can only be called on non-continuous queries;") - } - - test("check json() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.json("non_exist_path")) - assert(e.getMessage == "json() can only be called on non-continuous queries;") - } - - test("check parquet() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.parquet("non_exist_path")) - assert(e.getMessage == "parquet() can only be called on non-continuous queries;") - } - - test("check orc() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.orc("non_exist_path")) - assert(e.getMessage == "orc() can only be called on non-continuous queries;") - } - - test("check text() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.text("non_exist_path")) - assert(e.getMessage == "text() can only be called on non-continuous queries;") - } - - test("check csv() can only be called on non-continuous queries") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - val w = df.write - val e = intercept[AnalysisException](w.csv("non_exist_path")) - assert(e.getMessage == "csv() can only be called on non-continuous queries;") - } - - test("check foreach() does not support partitioning or bucketing") { - val df = spark.read - .format("org.apache.spark.sql.streaming.test") - .stream() - - var w = df.write.partitionBy("value") - var e = intercept[AnalysisException](w.foreach(null)) - Seq("foreach", "partitioning").foreach { s => - assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) - } - - w = df.write.bucketBy(2, "value") - e = intercept[AnalysisException](w.foreach(null)) - Seq("foreach", "bucketing").foreach { s => + var w = df.writeStream + var e = intercept[IllegalArgumentException](w.foreach(null)) + Seq("foreach", "null").foreach { s => assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) } } - test("check jdbc() does not support partitioning or bucketing") { - val df = spark.read.text(newTextInput) - var w = df.write.partitionBy("value") - var e = intercept[AnalysisException](w.jdbc(null, null, null)) - Seq("jdbc", "partitioning").foreach { s => - assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + test("check foreach() does not support partitioning") { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + val foreachWriter = new ForeachWriter[Row] { + override def open(partitionId: Long, version: Long): Boolean = false + override def process(value: Row): Unit = {} + override def close(errorOrNull: Throwable): Unit = {} } - - w = df.write.bucketBy(2, "value") - e = intercept[AnalysisException](w.jdbc(null, null, null)) - Seq("jdbc", "bucketing").foreach { s => + var w = df.writeStream.partitionBy("value") + var e = intercept[AnalysisException](w.foreach(foreachWriter).start()) + Seq("foreach", "partitioning").foreach { s => assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) } } test("ConsoleSink can be correctly loaded") { LastOptions.clear() - val df = spark.read + val df = spark.readStream .format("org.apache.spark.sql.streaming.test") - .stream() + .load() - val cq = df.write + val cq = df.writeStream .format("console") .option("checkpointLocation", newMetadataDir) .trigger(ProcessingTime(2.seconds)) - .startStream() + .start() cq.awaitTermination(2000L) } @@ -611,10 +447,11 @@ class DataFrameReaderWriterSuite extends StreamTest with BeforeAndAfter { withTempDir { dir => val path = dir.getCanonicalPath intercept[AnalysisException] { - spark.range(10).write.format("parquet").mode("overwrite").partitionBy("id").save(path) - } - intercept[AnalysisException] { - spark.range(10).write.format("orc").mode("overwrite").partitionBy("id").save(path) + spark.range(10).writeStream + .outputMode("append") + .partitionBy("id") + .format("parquet") + .start(path) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala new file mode 100644 index 0000000000000..98e57b38044f2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -0,0 +1,231 @@ +/* + * 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.test + +import org.apache.spark.sql._ +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.util.Utils + + +object LastOptions { + + var parameters: Map[String, String] = null + var schema: Option[StructType] = null + var saveMode: SaveMode = null + + def clear(): Unit = { + parameters = null + schema = null + saveMode = null + } +} + + +/** Dummy provider. */ +class DefaultSource + extends RelationProvider + with SchemaRelationProvider + with CreatableRelationProvider { + + case class FakeRelation(sqlContext: SQLContext) extends BaseRelation { + override def schema: StructType = StructType(Seq(StructField("a", StringType))) + } + + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType + ): BaseRelation = { + LastOptions.parameters = parameters + LastOptions.schema = Some(schema) + FakeRelation(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String] + ): BaseRelation = { + LastOptions.parameters = parameters + LastOptions.schema = None + FakeRelation(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + LastOptions.parameters = parameters + LastOptions.schema = None + LastOptions.saveMode = mode + FakeRelation(sqlContext) + } +} + + +class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext { + + private def newMetadataDir = + Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + + test("writeStream cannot be called on non-streaming datasets") { + val e = intercept[AnalysisException] { + spark.read + .format("org.apache.spark.sql.test") + .load() + .writeStream + .start() + } + Seq("'writeStream'", "only", "streaming Dataset/DataFrame").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + + + test("resolve default source") { + spark.read + .format("org.apache.spark.sql.test") + .load() + .write + .format("org.apache.spark.sql.test") + .save() + } + + test("resolve full class") { + spark.read + .format("org.apache.spark.sql.test.DefaultSource") + .load() + .write + .format("org.apache.spark.sql.test") + .save() + } + + test("options") { + val map = new java.util.HashMap[String, String] + map.put("opt3", "3") + + val df = spark.read + .format("org.apache.spark.sql.test") + .option("opt1", "1") + .options(Map("opt2" -> "2")) + .options(map) + .load() + + assert(LastOptions.parameters("opt1") == "1") + assert(LastOptions.parameters("opt2") == "2") + assert(LastOptions.parameters("opt3") == "3") + + LastOptions.clear() + + df.write + .format("org.apache.spark.sql.test") + .option("opt1", "1") + .options(Map("opt2" -> "2")) + .options(map) + .save() + + assert(LastOptions.parameters("opt1") == "1") + assert(LastOptions.parameters("opt2") == "2") + assert(LastOptions.parameters("opt3") == "3") + } + + test("save mode") { + val df = spark.read + .format("org.apache.spark.sql.test") + .load() + + df.write + .format("org.apache.spark.sql.test") + .mode(SaveMode.ErrorIfExists) + .save() + assert(LastOptions.saveMode === SaveMode.ErrorIfExists) + } + + test("paths") { + val df = spark.read + .format("org.apache.spark.sql.test") + .option("checkpointLocation", newMetadataDir) + .load("/test") + + assert(LastOptions.parameters("path") == "/test") + + LastOptions.clear() + + df.write + .format("org.apache.spark.sql.test") + .option("checkpointLocation", newMetadataDir) + .save("/test") + + assert(LastOptions.parameters("path") == "/test") + } + + test("test different data types for options") { + val df = spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 56) + .option("boolOpt", false) + .option("doubleOpt", 6.7) + .load("/test") + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("boolOpt") == "false") + assert(LastOptions.parameters("doubleOpt") == "6.7") + + LastOptions.clear() + df.write + .format("org.apache.spark.sql.test") + .option("intOpt", 56) + .option("boolOpt", false) + .option("doubleOpt", 6.7) + .option("checkpointLocation", newMetadataDir) + .save("/test") + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("boolOpt") == "false") + assert(LastOptions.parameters("doubleOpt") == "6.7") + } + + test("check jdbc() does not support partitioning or bucketing") { + val df = spark.read.text(Utils.createTempDir(namePrefix = "text").getCanonicalPath) + + var w = df.write.partitionBy("value") + var e = intercept[AnalysisException](w.jdbc(null, null, null)) + Seq("jdbc", "partitioning").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + + w = df.write.bucketBy(2, "value") + e = intercept[AnalysisException](w.jdbc(null, null, null)) + Seq("jdbc", "bucketing").foreach { s => + assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + } + } + + test("prevent all column partitioning") { + withTempDir { dir => + val path = dir.getCanonicalPath + intercept[AnalysisException] { + spark.range(10).write.format("parquet").mode("overwrite").partitionBy("id").save(path) + } + intercept[AnalysisException] { + spark.range(10).write.format("orc").mode("overwrite").partitionBy("id").save(path) + } + } + } +} From 1259a6fa82a6e305ae210095a6bd0001f172356f Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 14 Jun 2016 18:24:59 -0700 Subject: [PATCH 0649/1470] [SPARK-15011][SQL] Re-enable 'analyze MetastoreRelations' in hive StatisticsSuite ## What changes were proposed in this pull request? This test re-enables the `analyze MetastoreRelations` in `org.apache.spark.sql.hive.StatisticsSuite`. The flakiness of this test was traced back to a shared configuration option, `hive.exec.compress.output`, in `TestHive`. This property was set to `true` by the `HiveCompatibilitySuite`. I have added configuration resetting logic to `HiveComparisonTest`, in order to prevent such a thing from happening again. ## How was this patch tested? Is a test. Author: Herman van Hovell Author: Herman van Hovell Closes #13498 from hvanhovell/SPARK-15011. (cherry picked from commit 0bd86c0fe4ebf05d817632a8665a02b379fa3cae) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/hive/StatisticsSuite.scala | 9 ++++----- .../spark/sql/hive/execution/HiveComparisonTest.scala | 6 ++++++ 2 files changed, 10 insertions(+), 5 deletions(-) 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 666a8da0dab85..a5975cf483c10 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 @@ -21,7 +21,7 @@ import java.io.{File, PrintWriter} import scala.reflect.ClassTag -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.joins._ @@ -115,7 +115,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils } } - ignore("analyze MetastoreRelations") { + test("analyze MetastoreRelations") { def queryTotalSize(tableName: String): BigInt = spark.sessionState.catalog.lookupRelation(TableIdentifier(tableName)).statistics.sizeInBytes @@ -155,14 +155,13 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") - // This seems to be flaky. - // assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) + assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) sql("DROP TABLE analyzeTable_part").collect() // Try to analyze a temp table sql("""SELECT * FROM src""").createOrReplaceTempView("tempTable") - intercept[UnsupportedOperationException] { + intercept[AnalysisException] { sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") } spark.sessionState.catalog.dropTable( 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 a7652143a4252..a846711b84ec9 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.execution import java.io._ import java.nio.charset.StandardCharsets +import java.util import scala.util.control.NonFatal @@ -497,6 +498,8 @@ abstract class HiveComparisonTest } } + val savedSettings = new util.HashMap[String, String] + savedSettings.putAll(TestHive.conf.settings) try { try { if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { @@ -515,6 +518,9 @@ abstract class HiveComparisonTest } } catch { case tf: org.scalatest.exceptions.TestFailedException => throw tf + } finally { + TestHive.conf.settings.clear() + TestHive.conf.settings.putAll(savedSettings) } } } From b75542603714227faa952442daa5d52d46a56d3c Mon Sep 17 00:00:00 2001 From: bomeng Date: Tue, 14 Jun 2016 18:35:29 -0700 Subject: [PATCH 0650/1470] [SPARK-15952][SQL] fix "show databases" ordering issue ## What changes were proposed in this pull request? Two issues I've found for "show databases" command: 1. The returned database name list was not sorted, it only works when "like" was used together; (HIVE will always return a sorted list) 2. When it is used as sql("show databases").show, it will output a table with column named as "result", but for sql("show tables").show, it will output the column name as "tableName", so I think we should be consistent and use "databaseName" at least. ## How was this patch tested? Updated existing test case to test its ordering as well. Author: bomeng Closes #13671 from bomeng/SPARK-15952. (cherry picked from commit 42a28caf1001244d617b9256de196129348f2fef) Signed-off-by: Reynold Xin --- .../apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala | 2 +- .../org/apache/spark/sql/execution/command/databases.scala | 4 ++-- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 60525794edc5c..14da30a36f40b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -178,7 +178,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E } override def listDatabases(): Seq[String] = synchronized { - catalog.keySet.toSeq + catalog.keySet.toSeq.sorted } override def listDatabases(pattern: String): Seq[String] = synchronized { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala index cefe0f6e629b0..597ec27ce6698 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala @@ -33,9 +33,9 @@ import org.apache.spark.sql.types.StringType */ case class ShowDatabasesCommand(databasePattern: Option[String]) extends RunnableCommand { - // The result of SHOW DATABASES has one column called 'result' + // The result of SHOW DATABASES has one column called 'databaseName' override val output: Seq[Attribute] = { - AttributeReference("result", StringType, nullable = false)() :: Nil + AttributeReference("databaseName", StringType, nullable = false)() :: Nil } override def run(sparkSession: SparkSession): Seq[Row] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index a7e6893caaeb1..e15fcf4326be2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -708,11 +708,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("show databases") { - sql("CREATE DATABASE showdb1A") sql("CREATE DATABASE showdb2B") + sql("CREATE DATABASE showdb1A") - assert( - sql("SHOW DATABASES").count() >= 2) + // check the result as well as its order + checkDataset(sql("SHOW DATABASES"), Row("default"), Row("showdb1a"), Row("showdb2b")) checkAnswer( sql("SHOW DATABASES LIKE '*db1A'"), From f277cdf787de4402cd6cdba5e15e38bb71d8c5c7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 14 Jun 2016 18:57:45 -0700 Subject: [PATCH 0651/1470] [SPARK-15945][MLLIB] Conversion between old/new vector columns in a DataFrame (Scala/Java) ## What changes were proposed in this pull request? This PR provides conversion utils between old/new vector columns in a DataFrame. So users can use it to migrate their datasets and pipelines manually. The methods are implemented under `MLUtils` and called `convertVectorColumnsToML` and `convertVectorColumnsFromML`. Both take a DataFrame and a list of vector columns to be converted. It is a no-op on vector columns that are already converted. A warning message is logged if actual conversion happens. This is the first sub-task under SPARK-15944 to make it easier to migrate existing pipelines to Spark 2.0. ## How was this patch tested? Unit tests in Scala and Java. cc: yanboliang Author: Xiangrui Meng Closes #13662 from mengxr/SPARK-15945. (cherry picked from commit 63e0aebe22ba41c636ecaddd8647721d7690a1ec) Signed-off-by: Yanbo Liang --- .../org/apache/spark/mllib/util/MLUtils.scala | 117 +++++++++++++++++- .../spark/mllib/util/JavaMLUtilsSuite.java | 49 ++++++++ .../spark/mllib/util/MLUtilsSuite.scala | 60 ++++++++- 3 files changed, 218 insertions(+), 8 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index f0346e600d239..7d5bdffc42ec8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,14 +17,19 @@ package org.apache.spark.mllib.util +import scala.annotation.varargs import scala.reflect.ClassTag import org.apache.spark.SparkContext import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.{VectorUDT => MLVectorUDT} +import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.{PartitionwiseSampledRDD, RDD} +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.BernoulliCellSampler @@ -32,7 +37,7 @@ import org.apache.spark.util.random.BernoulliCellSampler * Helper methods to load, save and pre-process data used in ML Lib. */ @Since("0.8.0") -object MLUtils { +object MLUtils extends Logging { private[mllib] lazy val EPSILON = { var eps = 1.0 @@ -50,7 +55,6 @@ object MLUtils { * where the indices are one-based and in ascending order. * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], * where the feature indices are converted to zero-based. - * * @param sc Spark context * @param path file or directory path in any Hadoop-supported file system URI * @param numFeatures number of features, which will be determined from the input data if a @@ -145,7 +149,6 @@ object MLUtils { * Save labeled data in LIBSVM format. * @param data an RDD of LabeledPoint to be saved * @param dir directory to save the data - * * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]] */ @Since("1.0.0") @@ -253,6 +256,110 @@ object MLUtils { } } + /** + * Converts vector columns in an input Dataset from the [[org.apache.spark.mllib.linalg.Vector]] + * type to the new [[org.apache.spark.ml.linalg.Vector]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of vector columns to be converted. New vector columns will be ignored. If + * unspecified, all old vector columns will be converted except nested ones. + * @return the input [[DataFrame]] with old vector columns converted to the new vector type + */ + @Since("2.0.0") + @varargs + def convertVectorColumnsToML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[VectorUDT]) { + Some(c) + } else { + // ignore new vector columns and raise an exception on other column types + require(dataType.getClass == classOf[MLVectorUDT], + s"Column $c must be old Vector type to be converted to new type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[VectorUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Vector column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + // TODO: This implementation has performance issues due to unnecessary serialization. + // TODO: It is better (but trickier) if we can cast the old vector type to new type directly. + val convertToML = udf { v: Vector => v.asML } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertToML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + + /** + * Converts vector columns in an input Dataset to the [[org.apache.spark.ml.linalg.Vector]] type + * from the new [[org.apache.spark.mllib.linalg.Vector]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of vector columns to be converted. Old vector columns will be ignored. If + * unspecified, all new vector columns will be converted except nested ones. + * @return the input [[DataFrame]] with new vector columns converted to the old vector type + */ + @Since("2.0.0") + @varargs + def convertVectorColumnsFromML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[MLVectorUDT]) { + Some(c) + } else { + // ignore old vector columns and raise an exception on other column types + require(dataType.getClass == classOf[VectorUDT], + s"Column $c must be new Vector type to be converted to old type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[MLVectorUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Vector column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + // TODO: This implementation has performance issues due to unnecessary serialization. + // TODO: It is better (but trickier) if we can cast the new vector type to old type directly. + val convertFromML = udf { Vectors.fromML _ } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertFromML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: @@ -261,7 +368,6 @@ object MLUtils { * * When both vector norms are given, this is faster than computing the squared distance directly, * especially when one of the vectors is a sparse vector. - * * @param v1 the first vector * @param norm1 the norm of the first vector, non-negative * @param v2 the second vector @@ -314,7 +420,6 @@ object MLUtils { * When `x` is positive and large, computing `math.log(1 + math.exp(x))` will lead to arithmetic * overflow. This will happen when `x > 709.78` which is not a very large number. * It can be addressed by rewriting the formula into `x + math.log1p(math.exp(-x))` when `x > 0`. - * * @param x a floating-point value as input. * @return the result of `math.log(1 + math.exp(x))`. */ diff --git a/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java new file mode 100644 index 0000000000000..2fa0bd2546594 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java @@ -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. + */ + +package org.apache.spark.mllib.util; + +import java.util.Collections; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; + +public class JavaMLUtilsSuite extends SharedSparkSession { + + @Test + public void testConvertVectorColumnsToAndFromML() { + Vector x = Vectors.dense(2.0); + Dataset dataset = spark.createDataFrame( + Collections.singletonList(new LabeledPoint(1.0, x)), LabeledPoint.class + ).select("label", "features"); + Dataset newDataset1 = MLUtils.convertVectorColumnsToML(dataset); + Row new1 = newDataset1.first(); + Assert.assertEquals(RowFactory.create(1.0, x.asML()), new1); + Row new2 = MLUtils.convertVectorColumnsToML(dataset, "features").first(); + Assert.assertEquals(new1, new2); + Row old1 = MLUtils.convertVectorColumnsFromML(newDataset1).first(); + Assert.assertEquals(RowFactory.create(1.0, x), old1); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 7b6bfee00cb1f..3801bd127a5f7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -25,12 +25,14 @@ import scala.io.Source import breeze.linalg.{squaredDistance => breezeSquaredDistance} import com.google.common.io.Files -import org.apache.spark.SparkException -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.util.Utils class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -245,4 +247,58 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(log1pExp(-13.8) ~== math.log1p(math.exp(-13.8)) absTol 1E-10) assert(log1pExp(-238423789.865) ~== math.log1p(math.exp(-238423789.865)) absTol 1E-10) } + + test("convertVectorColumnsToML") { + val x = Vectors.sparse(2, Array(1), Array(1.0)) + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Vectors.dense(2.0, 3.0) + val z = Vectors.dense(4.0) + val p = (5.0, z) + val w = Vectors.dense(6.0).asML + val df = spark.createDataFrame(Seq( + (0, x, y, p, w) + )).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertVectorColumnsToML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, x.asML, y.asML, Row(5.0, z), w)) + val new2 = convertVectorColumnsToML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertVectorColumnsToML(df, "y", "w").first() + assert(new3 === Row(0, x, y.asML, Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertVectorColumnsToML(df, "p") + } + intercept[IllegalArgumentException] { + convertVectorColumnsToML(df, "p._2") + } + } + + test("convertVectorColumnsFromML") { + val x = Vectors.sparse(2, Array(1), Array(1.0)).asML + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Vectors.dense(2.0, 3.0).asML + val z = Vectors.dense(4.0).asML + val p = (5.0, z) + val w = Vectors.dense(6.0) + val df = spark.createDataFrame(Seq( + (0, x, y, p, w) + )).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertVectorColumnsFromML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, Vectors.fromML(x), Vectors.fromML(y), Row(5.0, z), w)) + val new2 = convertVectorColumnsFromML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertVectorColumnsFromML(df, "y", "w").first() + assert(new3 === Row(0, x, Vectors.fromML(y), Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertVectorColumnsFromML(df, "p") + } + intercept[IllegalArgumentException] { + convertVectorColumnsFromML(df, "p._2") + } + } } From df9a19fe880404895e6dfb28c9c1a887913bd582 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 14 Jun 2016 19:45:11 -0700 Subject: [PATCH 0652/1470] [SPARK-15935][PYSPARK] Fix a wrong format tag in the error message ## What changes were proposed in this pull request? A follow up PR for #13655 to fix a wrong format tag. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #13665 from zsxwing/fix. (cherry picked from commit 0ee9fd9e528206a5edfb2cc4a56538250b428aaf) Signed-off-by: Shixiong Zhu --- dev/run-tests.py | 2 +- python/pyspark/sql/streaming.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 2030c4ab2308b..dcf1be9d95eb4 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -335,7 +335,7 @@ def build_spark_maven(hadoop_version): def build_spark_sbt(hadoop_version): # Enable all of the profiles for the build: build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags - sbt_goals = ["package", + sbt_goals = ["test:package", # Build test jars as some tests depend on them "streaming-kafka-0-8-assembly/assembly", "streaming-flume-assembly/assembly", "streaming-kinesis-asl-assembly/assembly"] diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 0edaa51549395..1d650946bee66 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -146,7 +146,7 @@ def get(self, id): >>> cq.stop() """ if not isinstance(id, intlike): - raise ValueError("The id for the query must be an integer. Got: %d" % id) + raise ValueError("The id for the query must be an integer. Got: %s" % id) return ContinuousQuery(self._jcqm.get(id)) @since(2.0) From 7a0ed75ea7ce7700a2c0c69ba1c7ad06571b5c22 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 15 Jun 2016 09:09:21 -0500 Subject: [PATCH 0653/1470] [SPARK-15046][YARN] Parse value of token renewal interval correctly. Use the config variable definition both to set and parse the value, avoiding issues with code expecting the value in a different format. Tested by running spark-submit with --principal / --keytab. Author: Marcelo Vanzin Closes #13669 from vanzin/SPARK-15046. (cherry picked from commit 40eeef95256b0740d759d921f0385023f0b91666) Signed-off-by: Tom Graves --- .../scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 4 ++-- .../scala/org/apache/spark/internal/config/package.scala | 7 +++++++ .../main/scala/org/apache/spark/deploy/yarn/config.scala | 5 ----- 3 files changed, 9 insertions(+), 7 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 7a5fc866bb8bf..bb1793d451dfd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -41,6 +41,7 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** @@ -288,8 +289,7 @@ class SparkHadoopUtil extends Logging { credentials: Credentials): Long = { val now = System.currentTimeMillis() - val renewalInterval = - sparkConf.getLong("spark.yarn.token.renewal.interval", (24 hours).toMillis) + val renewalInterval = sparkConf.get(TOKEN_RENEWAL_INTERVAL).get credentials.getAllTokens.asScala .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 2c1e0b71e3613..05dd68300f891 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -17,6 +17,8 @@ package org.apache.spark.internal +import java.util.concurrent.TimeUnit + import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.ByteUnit @@ -80,6 +82,11 @@ package object config { .doc("Name of the Kerberos principal.") .stringConf.createOptional + private[spark] val TOKEN_RENEWAL_INTERVAL = ConfigBuilder("spark.yarn.token.renewal.interval") + .internal() + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") .intConf .createOptional diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index c4dd3202f0a15..ad2412e025227 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -243,11 +243,6 @@ package object config { .toSequence .createWithDefault(Nil) - private[spark] val TOKEN_RENEWAL_INTERVAL = ConfigBuilder("spark.yarn.token.renewal.interval") - .internal() - .timeConf(TimeUnit.MILLISECONDS) - .createOptional - /* Private configs. */ private[spark] val CREDENTIALS_FILE_PATH = ConfigBuilder("spark.yarn.credentials.file") From 5c53442cc098dd618ba1430962727c74b2de2e68 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 15 Jun 2016 09:43:11 -0700 Subject: [PATCH 0654/1470] [SPARK-15960][SQL] Rename `spark.sql.enableFallBackToHdfsForStats` config ## What changes were proposed in this pull request? Since we are probably going to add more statistics related configurations in the future, I'd like to rename the newly added `spark.sql.enableFallBackToHdfsForStats` configuration option to `spark.sql.statistics.fallBackToHdfs`. This allows us to put all statistics related configurations in the same namespace. ## How was this patch tested? None - just a usability thing Author: Herman van Hovell Closes #13681 from hvanhovell/SPARK-15960. (cherry picked from commit de99c3d0813de8e8f83374a0a85a73f7386fdfb8) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 27b1fffe27a70..6978b506acaa0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -114,7 +114,7 @@ object SQLConf { .createWithDefault(10L * 1024 * 1024) val ENABLE_FALL_BACK_TO_HDFS_FOR_STATS = - SQLConfigBuilder("spark.sql.enableFallBackToHdfsForStats") + SQLConfigBuilder("spark.sql.statistics.fallBackToHdfs") .doc("If the table statistics are not available from table metadata enable fall back to hdfs." + " This is useful in determining if a table is small enough to use auto broadcast joins.") .booleanConf From 4c950a75767f6e47091e436b0dcc089658b937ce Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Wed, 15 Jun 2016 10:29:07 -0700 Subject: [PATCH 0655/1470] [SPARK-15637][SPARK-15931][SPARKR] Fix R masked functions checks ## What changes were proposed in this pull request? Because of the fix in SPARK-15684, this exclusion is no longer necessary. ## How was this patch tested? unit tests shivaram Author: Felix Cheung Closes #13636 from felixcheung/rendswith. (cherry picked from commit d30b7e6696e20f1014c7f26aadbc051da0fac578) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_context.R | 27 ++++++++++++++++-------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 1d56ced399a9e..126484c995fb3 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -19,21 +19,26 @@ context("test functions in sparkR.R") test_that("Check masked functions", { # Check that we are not masking any new function from base, stats, testthat unexpectedly - masked <- conflicts(detail = TRUE)$`package:SparkR` - expect_true("describe" %in% masked) # only when with testthat.. - func <- lapply(masked, function(x) { capture.output(showMethods(x))[[1]] }) - funcSparkROrEmpty <- grepl("\\(package SparkR\\)$|^$", func) - maskedBySparkR <- masked[funcSparkROrEmpty] + # NOTE: We should avoid adding entries to *namesOfMaskedCompletely* as masked functions make it + # hard for users to use base R functions. Please check when in doubt. + namesOfMaskedCompletely <- c("cov", "filter", "sample") namesOfMasked <- c("describe", "cov", "filter", "lag", "na.omit", "predict", "sd", "var", "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", "summary", "transform", "drop", "window", "as.data.frame") - namesOfMaskedCompletely <- c("cov", "filter", "sample") if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) - namesOfMaskedCompletely <- c("endsWith", "startsWith", namesOfMaskedCompletely) } + masked <- conflicts(detail = TRUE)$`package:SparkR` + expect_true("describe" %in% masked) # only when with testthat.. + func <- lapply(masked, function(x) { capture.output(showMethods(x))[[1]] }) + funcSparkROrEmpty <- grepl("\\(package SparkR\\)$|^$", func) + maskedBySparkR <- masked[funcSparkROrEmpty] expect_equal(length(maskedBySparkR), length(namesOfMasked)) - expect_equal(sort(maskedBySparkR), sort(namesOfMasked)) + # make the 2 lists the same length so expect_equal will print their content + l <- max(length(maskedBySparkR), length(namesOfMasked)) + length(maskedBySparkR) <- l + length(namesOfMasked) <- l + expect_equal(sort(maskedBySparkR, na.last = TRUE), sort(namesOfMasked, na.last = TRUE)) # above are those reported as masked when `library(SparkR)` # note that many of these methods are still callable without base:: or stats:: prefix # there should be a test for each of these, except followings, which are currently "broken" @@ -42,7 +47,11 @@ test_that("Check masked functions", { })) maskedCompletely <- masked[!funcHasAny] expect_equal(length(maskedCompletely), length(namesOfMaskedCompletely)) - expect_equal(sort(maskedCompletely), sort(namesOfMaskedCompletely)) + l <- max(length(maskedCompletely), length(namesOfMaskedCompletely)) + length(maskedCompletely) <- l + length(namesOfMaskedCompletely) <- l + expect_equal(sort(maskedCompletely, na.last = TRUE), + sort(namesOfMaskedCompletely, na.last = TRUE)) }) test_that("repeatedly starting and stopping SparkR", { From 885e74a38211d3eed20af716b2a1e3b4aa6f9080 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 15 Jun 2016 10:46:02 -0700 Subject: [PATCH 0656/1470] [SPARK-15953][WIP][STREAMING] Renamed ContinuousQuery to StreamingQuery Renamed for simplicity, so that its obvious that its related to streaming. Existing unit tests. Author: Tathagata Das Closes #13673 from tdas/SPARK-15953. (cherry picked from commit 9a5071996b968148f6b9aba12e0d3fe888d9acd8) Signed-off-by: Shixiong Zhu --- python/pyspark/sql/context.py | 8 +- python/pyspark/sql/dataframe.py | 2 +- python/pyspark/sql/readwriter.py | 40 +++++----- python/pyspark/sql/session.py | 10 +-- python/pyspark/sql/streaming.py | 79 +++++++++---------- python/pyspark/sql/tests.py | 52 ++++++------ python/pyspark/sql/utils.py | 8 +- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../org/apache/spark/sql/ForeachWriter.scala | 4 +- .../org/apache/spark/sql/SQLContext.scala | 8 +- .../org/apache/spark/sql/SparkSession.scala | 6 +- .../spark/sql/execution/SQLExecution.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../execution/streaming/StreamExecution.scala | 26 +++--- ....scala => StreamingQueryListenerBus.scala} | 28 +++---- .../org/apache/spark/sql/functions.scala | 6 +- .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../spark/sql/internal/SessionState.scala | 8 +- .../sql/streaming/DataStreamWriter.scala | 16 ++-- ...inuousQuery.scala => StreamingQuery.scala} | 10 +-- ...on.scala => StreamingQueryException.scala} | 6 +- ...eryInfo.scala => StreamingQueryInfo.scala} | 12 +-- ...ner.scala => StreamingQueryListener.scala} | 28 +++---- ...ager.scala => StreamingQueryManager.scala} | 38 ++++----- .../apache/spark/sql/streaming/Trigger.scala | 4 +- .../sql/streaming/FileStreamSinkSuite.scala | 6 +- .../spark/sql/streaming/FileStressSuite.scala | 6 +- .../spark/sql/streaming/StreamTest.scala | 4 +- .../streaming/StreamingAggregationSuite.scala | 2 - ...cala => StreamingQueryListenerSuite.scala} | 50 ++++++------ ...scala => StreamingQueryManagerSuite.scala} | 6 +- ...ySuite.scala => StreamingQuerySuite.scala} | 8 +- .../test/DataStreamReaderWriterSuite.scala | 10 +-- 33 files changed, 251 insertions(+), 254 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/{ContinuousQueryListenerBus.scala => StreamingQueryListenerBus.scala} (63%) rename sql/core/src/main/scala/org/apache/spark/sql/streaming/{ContinuousQuery.scala => StreamingQuery.scala} (91%) rename sql/core/src/main/scala/org/apache/spark/sql/streaming/{ContinuousQueryException.scala => StreamingQueryException.scala} (93%) rename sql/core/src/main/scala/org/apache/spark/sql/streaming/{ContinuousQueryInfo.scala => StreamingQueryInfo.scala} (75%) rename sql/core/src/main/scala/org/apache/spark/sql/streaming/{ContinuousQueryListener.scala => StreamingQueryListener.scala} (78%) rename sql/core/src/main/scala/org/apache/spark/sql/streaming/{ContinuousQueryManager.scala => StreamingQueryManager.scala} (87%) rename sql/core/src/test/scala/org/apache/spark/sql/streaming/{ContinuousQueryListenerSuite.scala => StreamingQueryListenerSuite.scala} (85%) rename sql/core/src/test/scala/org/apache/spark/sql/streaming/{ContinuousQueryManagerSuite.scala => StreamingQueryManagerSuite.scala} (98%) rename sql/core/src/test/scala/org/apache/spark/sql/streaming/{ContinuousQuerySuite.scala => StreamingQuerySuite.scala} (96%) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index a271afe4cf9ba..8a1a874884e2a 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -444,13 +444,13 @@ def readStream(self): @property @since(2.0) def streams(self): - """Returns a :class:`ContinuousQueryManager` that allows managing all the - :class:`ContinuousQuery` ContinuousQueries active on `this` context. + """Returns a :class:`StreamingQueryManager` that allows managing all the + :class:`StreamingQuery` StreamingQueries active on `this` context. .. note:: Experimental. """ - from pyspark.sql.streaming import ContinuousQueryManager - return ContinuousQueryManager(self._ssql_ctx.streams()) + from pyspark.sql.streaming import StreamingQueryManager + return StreamingQueryManager(self._ssql_ctx.streams()) class HiveContext(SQLContext): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 0126faf574829..acf9d08b23a27 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -257,7 +257,7 @@ def isLocal(self): def isStreaming(self): """Returns true if this :class:`Dataset` contains one or more sources that continuously return data as it arrives. A :class:`Dataset` that reads data from a streaming source - must be executed as a :class:`ContinuousQuery` using the :func:`startStream` method in + must be executed as a :class:`StreamingQuery` using the :func:`startStream` method in :class:`DataFrameWriter`. Methods that return a single answer, (e.g., :func:`count` or :func:`collect`) will throw an :class:`AnalysisException` when there is a streaming source present. diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index ad954d0ad8217..c982de6840d48 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -28,7 +28,7 @@ from pyspark.sql.types import * from pyspark.sql import utils -__all__ = ["DataFrameReader", "DataFrameWriter"] +__all__ = ["DataFrameReader", "DataFrameWriter", "DataStreamReader", "DataStreamWriter"] def to_str(value): @@ -458,9 +458,9 @@ def __init__(self, df): self._spark = df.sql_ctx self._jwrite = df._jdf.write() - def _cq(self, jcq): - from pyspark.sql.streaming import ContinuousQuery - return ContinuousQuery(jcq) + def _sq(self, jsq): + from pyspark.sql.streaming import StreamingQuery + return StreamingQuery(jsq) @since(1.4) def mode(self, saveMode): @@ -1094,9 +1094,9 @@ def __init__(self, df): self._spark = df.sql_ctx self._jwrite = df._jdf.writeStream() - def _cq(self, jcq): - from pyspark.sql.streaming import ContinuousQuery - return ContinuousQuery(jcq) + def _sq(self, jsq): + from pyspark.sql.streaming import StreamingQuery + return StreamingQuery(jsq) @since(2.0) def outputMode(self, outputMode): @@ -1169,8 +1169,8 @@ def partitionBy(self, *cols): @since(2.0) def queryName(self, queryName): - """Specifies the name of the :class:`ContinuousQuery` that can be started with - :func:`startStream`. This name must be unique among all the currently active queries + """Specifies the name of the :class:`StreamingQuery` that can be started with + :func:`start`. This name must be unique among all the currently active queries in the associated SparkSession. .. note:: Experimental. @@ -1232,21 +1232,21 @@ def start(self, path=None, format=None, partitionBy=None, queryName=None, **opti :param options: All other string options. You may want to provide a `checkpointLocation` for most streams, however it is not required for a `memory` stream. - >>> cq = sdf.writeStream.format('memory').queryName('this_query').start() - >>> cq.isActive + >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() + >>> sq.isActive True - >>> cq.name + >>> sq.name u'this_query' - >>> cq.stop() - >>> cq.isActive + >>> sq.stop() + >>> sq.isActive False - >>> cq = sdf.writeStream.trigger(processingTime='5 seconds').start( + >>> sq = sdf.writeStream.trigger(processingTime='5 seconds').start( ... queryName='that_query', format='memory') - >>> cq.name + >>> sq.name u'that_query' - >>> cq.isActive + >>> sq.isActive True - >>> cq.stop() + >>> sq.stop() """ self.options(**options) if partitionBy is not None: @@ -1256,9 +1256,9 @@ def start(self, path=None, format=None, partitionBy=None, queryName=None, **opti if queryName is not None: self.queryName(queryName) if path is None: - return self._cq(self._jwrite.start()) + return self._sq(self._jwrite.start()) else: - return self._cq(self._jwrite.start(path)) + return self._sq(self._jwrite.start(path)) def _test(): diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 11c815dd9450b..6edbd5985644c 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -565,15 +565,15 @@ def readStream(self): @property @since(2.0) def streams(self): - """Returns a :class:`ContinuousQueryManager` that allows managing all the - :class:`ContinuousQuery` ContinuousQueries active on `this` context. + """Returns a :class:`StreamingQueryManager` that allows managing all the + :class:`StreamingQuery` StreamingQueries active on `this` context. .. note:: Experimental. - :return: :class:`ContinuousQueryManager` + :return: :class:`StreamingQueryManager` """ - from pyspark.sql.streaming import ContinuousQueryManager - return ContinuousQueryManager(self._jsparkSession.streams()) + from pyspark.sql.streaming import StreamingQueryManager + return StreamingQueryManager(self._jsparkSession.streams()) @since(2.0) def stop(self): diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 1d650946bee66..ae45c99e4f352 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -26,10 +26,10 @@ from pyspark import since from pyspark.rdd import ignore_unicode_prefix -__all__ = ["ContinuousQuery"] +__all__ = ["StreamingQuery"] -class ContinuousQuery(object): +class StreamingQuery(object): """ A handle to a query that is executing continuously in the background as new data arrives. All these methods are thread-safe. @@ -39,30 +39,30 @@ class ContinuousQuery(object): .. versionadded:: 2.0 """ - def __init__(self, jcq): - self._jcq = jcq + def __init__(self, jsq): + self._jsq = jsq @property @since(2.0) def id(self): - """The id of the continuous query. This id is unique across all queries that have been + """The id of the streaming query. This id is unique across all queries that have been started in the current process. """ - return self._jcq.id() + return self._jsq.id() @property @since(2.0) def name(self): - """The name of the continuous query. This name is unique across all active queries. + """The name of the streaming query. This name is unique across all active queries. """ - return self._jcq.name() + return self._jsq.name() @property @since(2.0) def isActive(self): - """Whether this continuous query is currently active or not. + """Whether this streaming query is currently active or not. """ - return self._jcq.isActive() + return self._jsq.isActive() @since(2.0) def awaitTermination(self, timeout=None): @@ -75,14 +75,14 @@ def awaitTermination(self, timeout=None): immediately (if the query was terminated by :func:`stop()`), or throw the exception immediately (if the query has terminated with exception). - throws :class:`ContinuousQueryException`, if `this` query has terminated with an exception + throws :class:`StreamingQueryException`, if `this` query has terminated with an exception """ if timeout is not None: if not isinstance(timeout, (int, float)) or timeout < 0: raise ValueError("timeout must be a positive integer or float. Got %s" % timeout) - return self._jcq.awaitTermination(int(timeout * 1000)) + return self._jsq.awaitTermination(int(timeout * 1000)) else: - return self._jcq.awaitTermination() + return self._jsq.awaitTermination() @since(2.0) def processAllAvailable(self): @@ -92,26 +92,25 @@ def processAllAvailable(self): until data that has been synchronously appended data to a stream source prior to invocation. (i.e. `getOffset` must immediately reflect the addition). """ - return self._jcq.processAllAvailable() + return self._jsq.processAllAvailable() @since(2.0) def stop(self): - """Stop this continuous query. + """Stop this streaming query. """ - self._jcq.stop() + self._jsq.stop() -class ContinuousQueryManager(object): - """A class to manage all the :class:`ContinuousQuery` ContinuousQueries active - on a :class:`SQLContext`. +class StreamingQueryManager(object): + """A class to manage all the :class:`StreamingQuery` StreamingQueries active. .. note:: Experimental .. versionadded:: 2.0 """ - def __init__(self, jcqm): - self._jcqm = jcqm + def __init__(self, jsqm): + self._jsqm = jsqm @property @ignore_unicode_prefix @@ -119,14 +118,14 @@ def __init__(self, jcqm): def active(self): """Returns a list of active queries associated with this SQLContext - >>> cq = df.writeStream.format('memory').queryName('this_query').start() - >>> cqm = spark.streams - >>> # get the list of active continuous queries - >>> [q.name for q in cqm.active] + >>> sq = df.writeStream.format('memory').queryName('this_query').start() + >>> sqm = spark.streams + >>> # get the list of active streaming queries + >>> [q.name for q in sqm.active] [u'this_query'] - >>> cq.stop() + >>> sq.stop() """ - return [ContinuousQuery(jcq) for jcq in self._jcqm.active()] + return [StreamingQuery(jsq) for jsq in self._jsqm.active()] @ignore_unicode_prefix @since(2.0) @@ -134,20 +133,20 @@ def get(self, id): """Returns an active query from this SQLContext or throws exception if an active query with this name doesn't exist. - >>> cq = df.writeStream.format('memory').queryName('this_query').start() - >>> cq.name + >>> sq = df.writeStream.format('memory').queryName('this_query').start() + >>> sq.name u'this_query' - >>> cq = spark.streams.get(cq.id) - >>> cq.isActive + >>> sq = spark.streams.get(sq.id) + >>> sq.isActive True - >>> cq = sqlContext.streams.get(cq.id) - >>> cq.isActive + >>> sq = sqlContext.streams.get(sq.id) + >>> sq.isActive True - >>> cq.stop() + >>> sq.stop() """ if not isinstance(id, intlike): raise ValueError("The id for the query must be an integer. Got: %s" % id) - return ContinuousQuery(self._jcqm.get(id)) + return StreamingQuery(self._jsqm.get(id)) @since(2.0) def awaitAnyTermination(self, timeout=None): @@ -168,14 +167,14 @@ def awaitAnyTermination(self, timeout=None): queries, users need to stop all of them after any of them terminates with exception, and then check the `query.exception()` for each query. - throws :class:`ContinuousQueryException`, if `this` query has terminated with an exception + throws :class:`StreamingQueryException`, if `this` query has terminated with an exception """ if timeout is not None: if not isinstance(timeout, (int, float)) or timeout < 0: raise ValueError("timeout must be a positive integer or float. Got %s" % timeout) - return self._jcqm.awaitAnyTermination(int(timeout * 1000)) + return self._jsqm.awaitAnyTermination(int(timeout * 1000)) else: - return self._jcqm.awaitAnyTermination() + return self._jsqm.awaitAnyTermination() @since(2.0) def resetTerminated(self): @@ -184,11 +183,11 @@ def resetTerminated(self): >>> spark.streams.resetTerminated() """ - self._jcqm.resetTerminated() + self._jsqm.resetTerminated() class Trigger(object): - """Used to indicate how often results should be produced by a :class:`ContinuousQuery`. + """Used to indicate how often results should be produced by a :class:`StreamingQuery`. .. note:: Experimental diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index fee960a1a7bb4..1d5d69169604d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -921,32 +921,32 @@ def test_stream_read_options_overwrite(self): def test_stream_save_options(self): df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') - for cq in self.spark._wrapped.streams.active: - cq.stop() + for q in self.spark._wrapped.streams.active: + q.stop() tmpPath = tempfile.mkdtemp() shutil.rmtree(tmpPath) self.assertTrue(df.isStreaming) out = os.path.join(tmpPath, 'out') chk = os.path.join(tmpPath, 'chk') - cq = df.writeStream.option('checkpointLocation', chk).queryName('this_query') \ + q = df.writeStream.option('checkpointLocation', chk).queryName('this_query') \ .format('parquet').outputMode('append').option('path', out).start() try: - self.assertEqual(cq.name, 'this_query') - self.assertTrue(cq.isActive) - cq.processAllAvailable() + self.assertEqual(q.name, 'this_query') + self.assertTrue(q.isActive) + q.processAllAvailable() output_files = [] for _, _, files in os.walk(out): output_files.extend([f for f in files if not f.startswith('.')]) self.assertTrue(len(output_files) > 0) self.assertTrue(len(os.listdir(chk)) > 0) finally: - cq.stop() + q.stop() shutil.rmtree(tmpPath) def test_stream_save_options_overwrite(self): df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') - for cq in self.spark._wrapped.streams.active: - cq.stop() + for q in self.spark._wrapped.streams.active: + q.stop() tmpPath = tempfile.mkdtemp() shutil.rmtree(tmpPath) self.assertTrue(df.isStreaming) @@ -954,15 +954,15 @@ def test_stream_save_options_overwrite(self): chk = os.path.join(tmpPath, 'chk') fake1 = os.path.join(tmpPath, 'fake1') fake2 = os.path.join(tmpPath, 'fake2') - cq = df.writeStream.option('checkpointLocation', fake1)\ + q = df.writeStream.option('checkpointLocation', fake1)\ .format('memory').option('path', fake2) \ .queryName('fake_query').outputMode('append') \ .start(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) try: - self.assertEqual(cq.name, 'this_query') - self.assertTrue(cq.isActive) - cq.processAllAvailable() + self.assertEqual(q.name, 'this_query') + self.assertTrue(q.isActive) + q.processAllAvailable() output_files = [] for _, _, files in os.walk(out): output_files.extend([f for f in files if not f.startswith('.')]) @@ -971,50 +971,50 @@ def test_stream_save_options_overwrite(self): self.assertFalse(os.path.isdir(fake1)) # should not have been created self.assertFalse(os.path.isdir(fake2)) # should not have been created finally: - cq.stop() + q.stop() shutil.rmtree(tmpPath) def test_stream_await_termination(self): df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') - for cq in self.spark._wrapped.streams.active: - cq.stop() + for q in self.spark._wrapped.streams.active: + q.stop() tmpPath = tempfile.mkdtemp() shutil.rmtree(tmpPath) self.assertTrue(df.isStreaming) out = os.path.join(tmpPath, 'out') chk = os.path.join(tmpPath, 'chk') - cq = df.writeStream\ + q = df.writeStream\ .start(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) try: - self.assertTrue(cq.isActive) + self.assertTrue(q.isActive) try: - cq.awaitTermination("hello") + q.awaitTermination("hello") self.fail("Expected a value exception") except ValueError: pass now = time.time() # test should take at least 2 seconds - res = cq.awaitTermination(2.6) + res = q.awaitTermination(2.6) duration = time.time() - now self.assertTrue(duration >= 2) self.assertFalse(res) finally: - cq.stop() + q.stop() shutil.rmtree(tmpPath) def test_query_manager_await_termination(self): df = self.spark.readStream.format('text').load('python/test_support/sql/streaming') - for cq in self.spark._wrapped.streams.active: - cq.stop() + for q in self.spark._wrapped.streams.active: + q.stop() tmpPath = tempfile.mkdtemp() shutil.rmtree(tmpPath) self.assertTrue(df.isStreaming) out = os.path.join(tmpPath, 'out') chk = os.path.join(tmpPath, 'chk') - cq = df.writeStream\ + q = df.writeStream\ .start(path=out, format='parquet', queryName='this_query', checkpointLocation=chk) try: - self.assertTrue(cq.isActive) + self.assertTrue(q.isActive) try: self.spark._wrapped.streams.awaitAnyTermination("hello") self.fail("Expected a value exception") @@ -1027,7 +1027,7 @@ def test_query_manager_await_termination(self): self.assertTrue(duration >= 2) self.assertFalse(res) finally: - cq.stop() + q.stop() shutil.rmtree(tmpPath) def test_help_command(self): diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 9ddaf78acf91d..2a85ec01bc92a 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -45,9 +45,9 @@ class IllegalArgumentException(CapturedException): """ -class ContinuousQueryException(CapturedException): +class StreamingQueryException(CapturedException): """ - Exception that stopped a :class:`ContinuousQuery`. + Exception that stopped a :class:`StreamingQuery`. """ @@ -71,8 +71,8 @@ def deco(*a, **kw): raise AnalysisException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.catalyst.parser.ParseException: '): raise ParseException(s.split(': ', 1)[1], stackTrace) - if s.startswith('org.apache.spark.sql.streaming.ContinuousQueryException: '): - raise ContinuousQueryException(s.split(': ', 1)[1], stackTrace) + if s.startswith('org.apache.spark.sql.streaming.StreamingQueryException: '): + raise StreamingQueryException(s.split(': ', 1)[1], stackTrace) if s.startswith('org.apache.spark.sql.execution.QueryExecutionException: '): raise QueryExecutionException(s.split(': ', 1)[1], stackTrace) if s.startswith('java.lang.IllegalArgumentException: '): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index f9db325ea241f..fba4066af6bdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -49,7 +49,7 @@ import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation} import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython -import org.apache.spark.sql.streaming.{ContinuousQuery, DataStreamWriter} +import org.apache.spark.sql.streaming.{DataStreamWriter, StreamingQuery} import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -455,7 +455,7 @@ class Dataset[T] private[sql]( /** * Returns true if this Dataset contains one or more sources that continuously * return data as it arrives. A Dataset that reads data from a streaming source - * must be executed as a [[ContinuousQuery]] using the `startStream()` method in + * must be executed as a [[StreamingQuery]] using the `startStream()` method in * [[DataFrameWriter]]. Methods that return a single answer, e.g. `count()` or * `collect()`, will throw an [[AnalysisException]] when there is a streaming * source present. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala index 09f07426a6bfa..f56b25b5576f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -18,11 +18,11 @@ package org.apache.spark.sql import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.streaming.ContinuousQuery +import org.apache.spark.sql.streaming.StreamingQuery /** * :: Experimental :: - * A class to consume data generated by a [[ContinuousQuery]]. Typically this is used to send the + * A class to consume data generated by a [[StreamingQuery]]. Typically this is used to send the * generated data to external systems. Each partition will use a new deserialized instance, so you * usually should do all the initialization (e.g. opening a connection or initiating a transaction) * in the `open` method. 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 33f62915df694..e7627ac2c95ab 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 @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.streaming.{ContinuousQueryManager, DataStreamReader} +import org.apache.spark.sql.streaming.{DataStreamReader, StreamingQueryManager} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ExecutionListenerManager @@ -716,12 +716,12 @@ class SQLContext private[sql](val sparkSession: SparkSession) } /** - * Returns a [[ContinuousQueryManager]] that allows managing all the - * [[org.apache.spark.sql.streaming.ContinuousQuery ContinuousQueries]] active on `this` context. + * Returns a [[StreamingQueryManager]] that allows managing all the + * [[org.apache.spark.sql.streaming.StreamingQuery StreamingQueries]] active on `this` context. * * @since 2.0.0 */ - def streams: ContinuousQueryManager = sparkSession.streams + def streams: StreamingQueryManager = sparkSession.streams /** * Returns the names of tables in the current database as an array. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 9137a735dd4da..251f47d5fb1fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -178,13 +178,13 @@ class SparkSession private( /** * :: Experimental :: - * Returns a [[ContinuousQueryManager]] that allows managing all the - * [[ContinuousQuery ContinuousQueries]] active on `this`. + * Returns a [[StreamingQueryManager]] that allows managing all the + * [[StreamingQuery StreamingQueries]] active on `this`. * * @since 2.0.0 */ @Experimental - def streams: ContinuousQueryManager = sessionState.continuousQueryManager + def streams: StreamingQueryManager = sessionState.streamingQueryManager /** * Start a new session with isolated SQL configurations, temporary tables, registered diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 31c9f1aef2f3e..6cb1a44a2044a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -47,7 +47,7 @@ private[sql] object SQLExecution { val r = try { // sparkContext.getCallSite() would first try to pick up any call site that was previously // set, then fall back to Utils.getCallSite(); call Utils.getCallSite() directly on - // continuous queries would give us call site like "run at :0" + // streaming queries would give us call site like "run at :0" val callSite = sparkSession.sparkContext.getCallSite() sparkSession.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart( 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 d1261dd6cae43..60466e28307f7 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 @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} import org.apache.spark.sql.execution.streaming.MemoryPlan import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.streaming.ContinuousQuery +import org.apache.spark.sql.streaming.StreamingQuery /** * Converts a logical plan into zero or more SparkPlans. This API is exposed for experimenting @@ -225,7 +225,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { /** * Used to plan aggregation queries that are computed incrementally as part of a - * [[ContinuousQuery]]. Currently this rule is injected into the planner + * [[StreamingQuery]]. Currently this rule is injected into the planner * on-demand, only when planning in a [[org.apache.spark.sql.execution.streaming.StreamExecution]] */ object StatefulAggregationStrategy extends Strategy { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 5095fe7fcaa31..4aefd39b3646c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -52,9 +52,9 @@ class StreamExecution( val trigger: Trigger, private[sql] val triggerClock: Clock, val outputMode: OutputMode) - extends ContinuousQuery with Logging { + extends StreamingQuery with Logging { - import org.apache.spark.sql.streaming.ContinuousQueryListener._ + import org.apache.spark.sql.streaming.StreamingQueryListener._ /** * A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation. @@ -101,7 +101,7 @@ class StreamExecution( private[sql] var lastExecution: QueryExecution = null @volatile - private[sql] var streamDeathCause: ContinuousQueryException = null + private[sql] var streamDeathCause: StreamingQueryException = null /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() @@ -140,8 +140,8 @@ class StreamExecution( override def sinkStatus: SinkStatus = new SinkStatus(sink.toString, committedOffsets.toCompositeOffset(sources).toString) - /** Returns the [[ContinuousQueryException]] if the query was terminated by an exception. */ - override def exception: Option[ContinuousQueryException] = Option(streamDeathCause) + /** Returns the [[StreamingQueryException]] if the query was terminated by an exception. */ + override def exception: Option[StreamingQueryException] = Option(streamDeathCause) /** Returns the path of a file with `name` in the checkpoint directory. */ private def checkpointFile(name: String): String = @@ -199,7 +199,7 @@ class StreamExecution( } catch { case _: InterruptedException if state == TERMINATED => // interrupted by stop() case NonFatal(e) => - streamDeathCause = new ContinuousQueryException( + streamDeathCause = new StreamingQueryException( this, s"Query $name terminated with exception: ${e.getMessage}", e, @@ -227,7 +227,7 @@ class StreamExecution( private def populateStartOffsets(): Unit = { offsetLog.getLatest() match { case Some((batchId, nextOffsets)) => - logInfo(s"Resuming continuous query, starting with batch $batchId") + logInfo(s"Resuming streaming query, starting with batch $batchId") currentBatchId = batchId availableOffsets = nextOffsets.toStreamProgress(sources) logDebug(s"Found possibly uncommitted offsets $availableOffsets") @@ -239,7 +239,7 @@ class StreamExecution( } case None => // We are starting this stream for the first time. - logInfo(s"Starting new continuous query.") + logInfo(s"Starting new streaming query.") currentBatchId = 0 constructNextBatch() } @@ -383,7 +383,7 @@ class StreamExecution( postEvent(new QueryProgress(this.toInfo)) } - private def postEvent(event: ContinuousQueryListener.Event) { + private def postEvent(event: StreamingQueryListener.Event) { sparkSession.streams.postListenerEvent(event) } @@ -468,7 +468,7 @@ class StreamExecution( } override def toString: String = { - s"Continuous Query - $name [state = $state]" + s"Streaming Query - $name [state = $state]" } def toDebugString: String = { @@ -476,7 +476,7 @@ class StreamExecution( "Error:\n" + stackTraceToString(streamDeathCause.cause) } else "" s""" - |=== Continuous Query === + |=== Streaming Query === |Name: $name |Current Offsets: $committedOffsets | @@ -490,8 +490,8 @@ class StreamExecution( """.stripMargin } - private def toInfo: ContinuousQueryInfo = { - new ContinuousQueryInfo( + private def toInfo: StreamingQueryInfo = { + new StreamingQueryInfo( this.name, this.id, this.sourceStatuses, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala similarity index 63% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index f50951f9bd656..1e663956f980b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -18,27 +18,27 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerEvent} -import org.apache.spark.sql.streaming.ContinuousQueryListener +import org.apache.spark.sql.streaming.StreamingQueryListener import org.apache.spark.util.ListenerBus /** - * A bus to forward events to [[ContinuousQueryListener]]s. This one will send received - * [[ContinuousQueryListener.Event]]s to the Spark listener bus. It also registers itself with - * Spark listener bus, so that it can receive [[ContinuousQueryListener.Event]]s and dispatch them - * to ContinuousQueryListener. + * A bus to forward events to [[StreamingQueryListener]]s. This one will send received + * [[StreamingQueryListener.Event]]s to the Spark listener bus. It also registers itself with + * Spark listener bus, so that it can receive [[StreamingQueryListener.Event]]s and dispatch them + * to StreamingQueryListener. */ -class ContinuousQueryListenerBus(sparkListenerBus: LiveListenerBus) - extends SparkListener with ListenerBus[ContinuousQueryListener, ContinuousQueryListener.Event] { +class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) + extends SparkListener with ListenerBus[StreamingQueryListener, StreamingQueryListener.Event] { - import ContinuousQueryListener._ + import StreamingQueryListener._ sparkListenerBus.addListener(this) /** - * Post a ContinuousQueryListener event to the Spark listener bus asynchronously. This event will - * be dispatched to all ContinuousQueryListener in the thread of the Spark listener bus. + * Post a StreamingQueryListener event to the Spark listener bus asynchronously. This event will + * be dispatched to all StreamingQueryListener in the thread of the Spark listener bus. */ - def post(event: ContinuousQueryListener.Event) { + def post(event: StreamingQueryListener.Event) { event match { case s: QueryStarted => postToAll(s) @@ -49,15 +49,15 @@ class ContinuousQueryListenerBus(sparkListenerBus: LiveListenerBus) override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { - case e: ContinuousQueryListener.Event => + case e: StreamingQueryListener.Event => postToAll(e) case _ => } } override protected def doPostEvent( - listener: ContinuousQueryListener, - event: ContinuousQueryListener.Event): Unit = { + listener: StreamingQueryListener, + event: StreamingQueryListener.Event): Unit = { event match { case queryStarted: QueryStarted => listener.onQueryStarted(queryStarted) 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 02608b0dce042..e8bd489be3410 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 @@ -2587,7 +2587,7 @@ object functions { * 09:00:25-09:01:25 ... * }}} * - * For a continuous query, you may use the function `current_timestamp` to generate windows on + * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * * @param timeColumn The column or the expression to use as the timestamp for windowing by time. @@ -2641,7 +2641,7 @@ object functions { * 09:00:20-09:01:20 ... * }}} * - * For a continuous query, you may use the function `current_timestamp` to generate windows on + * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * * @param timeColumn The column or the expression to use as the timestamp for windowing by time. @@ -2683,7 +2683,7 @@ object functions { * 09:02:00-09:03:00 ... * }}} * - * For a continuous query, you may use the function `current_timestamp` to generate windows on + * For a streaming query, you may use the function `current_timestamp` to generate windows on * processing time. * * @param timeColumn The column or the expression to use as the timestamp for windowing by time. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6978b506acaa0..4b8916f59c41d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -484,14 +484,14 @@ object SQLConf { .createWithDefault(2) val CHECKPOINT_LOCATION = SQLConfigBuilder("spark.sql.streaming.checkpointLocation") - .doc("The default location for storing checkpoint data for continuously executing queries.") + .doc("The default location for storing checkpoint data for streaming queries.") .stringConf .createOptional val UNSUPPORTED_OPERATION_CHECK_ENABLED = SQLConfigBuilder("spark.sql.streaming.unsupportedOperationCheck") .internal() - .doc("When true, the logical plan for continuous query will be checked for unsupported" + + .doc("When true, the logical plan for streaming query will be checked for unsupported" + " operations.") .booleanConf .createWithDefault(true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index b43095041b243..59efa81275451 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, FindDataSourceTable, PreInsertCastAndRename, ResolveDataSource} -import org.apache.spark.sql.streaming.{ContinuousQuery, ContinuousQueryManager} +import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager} import org.apache.spark.sql.util.ExecutionListenerManager @@ -143,10 +143,10 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val listenerManager: ExecutionListenerManager = new ExecutionListenerManager /** - * Interface to start and stop [[ContinuousQuery]]s. + * Interface to start and stop [[StreamingQuery]]s. */ - lazy val continuousQueryManager: ContinuousQueryManager = { - new ContinuousQueryManager(sparkSession) + lazy val streamingQueryManager: StreamingQueryManager = { + new StreamingQueryManager(sparkSession) } private val jarClassLoader: NonClosableMutableURLClassLoader = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index b035ff7938bae..197707404edda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -109,7 +109,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * :: Experimental :: - * Specifies the name of the [[ContinuousQuery]] that can be started with `startStream()`. + * Specifies the name of the [[StreamingQuery]] that can be started with `startStream()`. * This name must be unique among all the currently active queries in the associated SQLContext. * * @since 2.0.0 @@ -221,26 +221,26 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * :: Experimental :: * Starts the execution of the streaming query, which will continually output results to the given - * path as new data arrives. The returned [[ContinuousQuery]] object can be used to interact with + * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with * the stream. * * @since 2.0.0 */ @Experimental - def start(path: String): ContinuousQuery = { + def start(path: String): StreamingQuery = { option("path", path).start() } /** * :: Experimental :: * Starts the execution of the streaming query, which will continually output results to the given - * path as new data arrives. The returned [[ContinuousQuery]] object can be used to interact with + * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with * the stream. * * @since 2.0.0 */ @Experimental - def start(): ContinuousQuery = { + def start(): StreamingQuery = { if (source == "memory") { assertNotPartitioned("memory") if (extraOptions.get("queryName").isEmpty) { @@ -249,7 +249,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { val sink = new MemorySink(df.schema, outputMode) val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink)) - val query = df.sparkSession.sessionState.continuousQueryManager.startQuery( + val query = df.sparkSession.sessionState.streamingQueryManager.startQuery( extraOptions.get("queryName"), extraOptions.get("checkpointLocation"), df, @@ -263,7 +263,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } else if (source == "foreach") { assertNotPartitioned("foreach") val sink = new ForeachSink[T](foreachWriter)(ds.exprEnc) - df.sparkSession.sessionState.continuousQueryManager.startQuery( + df.sparkSession.sessionState.streamingQueryManager.startQuery( extraOptions.get("queryName"), extraOptions.get("checkpointLocation"), df, @@ -278,7 +278,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { className = source, options = extraOptions.toMap, partitionColumns = normalizedParCols.getOrElse(Nil)) - df.sparkSession.sessionState.continuousQueryManager.startQuery( + df.sparkSession.sessionState.streamingQueryManager.startQuery( extraOptions.get("queryName"), extraOptions.get("checkpointLocation"), df, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala similarity index 91% rename from sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index 1e0a47deca5da..dc81a5b180276 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.SparkSession * @since 2.0.0 */ @Experimental -trait ContinuousQuery { +trait StreamingQuery { /** * Returns the name of the query. This name is unique across all active queries. This can be @@ -57,10 +57,10 @@ trait ContinuousQuery { def isActive: Boolean /** - * Returns the [[ContinuousQueryException]] if the query was terminated by an exception. + * Returns the [[StreamingQueryException]] if the query was terminated by an exception. * @since 2.0.0 */ - def exception: Option[ContinuousQueryException] + def exception: Option[StreamingQueryException] /** * Returns current status of all the sources. @@ -79,7 +79,7 @@ trait ContinuousQuery { * immediately (if the query was terminated by `stop()`), or throw the exception * immediately (if the query has terminated with exception). * - * @throws ContinuousQueryException, if `this` query has terminated with an exception. + * @throws StreamingQueryException, if `this` query has terminated with an exception. * * @since 2.0.0 */ @@ -95,7 +95,7 @@ trait ContinuousQuery { * `true` immediately (if the query was terminated by `stop()`), or throw the exception * immediately (if the query has terminated with exception). * - * @throws ContinuousQueryException, if `this` query has terminated with an exception + * @throws StreamingQueryException, if `this` query has terminated with an exception * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryException.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala similarity index 93% rename from sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryException.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala index 5196c5a537a71..90f95ca9d4229 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryException.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.execution.streaming.{Offset, StreamExecution} /** * :: Experimental :: - * Exception that stopped a [[ContinuousQuery]]. + * Exception that stopped a [[StreamingQuery]]. * @param query Query that caused the exception * @param message Message of this exception * @param cause Internal cause of this exception @@ -31,8 +31,8 @@ import org.apache.spark.sql.execution.streaming.{Offset, StreamExecution} * @since 2.0.0 */ @Experimental -class ContinuousQueryException private[sql]( - @transient val query: ContinuousQuery, +class StreamingQueryException private[sql]( + @transient val query: StreamingQuery, val message: String, val cause: Throwable, val startOffset: Option[Offset] = None, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala similarity index 75% rename from sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala index 19f22704ba11b..1af2668817eae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala @@ -21,16 +21,16 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * A class used to report information about the progress of a [[ContinuousQuery]]. + * A class used to report information about the progress of a [[StreamingQuery]]. * - * @param name The [[ContinuousQuery]] name. This name is unique across all active queries. - * @param id The [[ContinuousQuery]] id. This id is unique across + * @param name The [[StreamingQuery]] name. This name is unique across all active queries. + * @param id The [[StreamingQuery]] id. This id is unique across * all queries that have been started in the current process. - * @param sourceStatuses The current statuses of the [[ContinuousQuery]]'s sources. - * @param sinkStatus The current status of the [[ContinuousQuery]]'s sink. + * @param sourceStatuses The current statuses of the [[StreamingQuery]]'s sources. + * @param sinkStatus The current status of the [[StreamingQuery]]'s sink. */ @Experimental -class ContinuousQueryInfo private[sql]( +class StreamingQueryInfo private[sql]( val name: String, val id: Long, val sourceStatuses: Seq[SourceStatus], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala similarity index 78% rename from sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index dd311148e053f..c43de58faa80c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -22,22 +22,22 @@ import org.apache.spark.scheduler.SparkListenerEvent /** * :: Experimental :: - * Interface for listening to events related to [[ContinuousQuery ContinuousQueries]]. + * Interface for listening to events related to [[StreamingQuery StreamingQueries]]. * @note The methods are not thread-safe as they may be called from different threads. * * @since 2.0.0 */ @Experimental -abstract class ContinuousQueryListener { +abstract class StreamingQueryListener { - import ContinuousQueryListener._ + import StreamingQueryListener._ /** * Called when a query is started. * @note This is called synchronously with * [[org.apache.spark.sql.DataFrameWriter `DataFrameWriter.startStream()`]], * that is, `onQueryStart` will be called on all listeners before - * `DataFrameWriter.startStream()` returns the corresponding [[ContinuousQuery]]. Please + * `DataFrameWriter.startStream()` returns the corresponding [[StreamingQuery]]. Please * don't block this method as it will block your query. * @since 2.0.0 */ @@ -46,9 +46,9 @@ abstract class ContinuousQueryListener { /** * Called when there is some status update (ingestion rate updated, etc.) * - * @note This method is asynchronous. The status in [[ContinuousQuery]] will always be - * latest no matter when this method is called. Therefore, the status of [[ContinuousQuery]] - * may be changed before/when you process the event. E.g., you may find [[ContinuousQuery]] + * @note This method is asynchronous. The status in [[StreamingQuery]] will always be + * latest no matter when this method is called. Therefore, the status of [[StreamingQuery]] + * may be changed before/when you process the event. E.g., you may find [[StreamingQuery]] * is terminated when you are processing [[QueryProgress]]. * @since 2.0.0 */ @@ -64,15 +64,15 @@ abstract class ContinuousQueryListener { /** * :: Experimental :: - * Companion object of [[ContinuousQueryListener]] that defines the listener events. + * Companion object of [[StreamingQueryListener]] that defines the listener events. * @since 2.0.0 */ @Experimental -object ContinuousQueryListener { +object StreamingQueryListener { /** * :: Experimental :: - * Base type of [[ContinuousQueryListener]] events + * Base type of [[StreamingQueryListener]] events * @since 2.0.0 */ @Experimental @@ -84,7 +84,7 @@ object ContinuousQueryListener { * @since 2.0.0 */ @Experimental - class QueryStarted private[sql](val queryInfo: ContinuousQueryInfo) extends Event + class QueryStarted private[sql](val queryInfo: StreamingQueryInfo) extends Event /** * :: Experimental :: @@ -92,14 +92,14 @@ object ContinuousQueryListener { * @since 2.0.0 */ @Experimental - class QueryProgress private[sql](val queryInfo: ContinuousQueryInfo) extends Event + class QueryProgress private[sql](val queryInfo: StreamingQueryInfo) extends Event /** * :: Experimental :: * Event representing that termination of a query * * @param queryInfo Information about the status of the query. - * @param exception The exception message of the [[ContinuousQuery]] if the query was terminated + * @param exception The exception message of the [[StreamingQuery]] if the query was terminated * with an exception. Otherwise, it will be `None`. * @param stackTrace The stack trace of the exception if the query was terminated with an * exception. It will be empty if there was no error. @@ -107,7 +107,7 @@ object ContinuousQueryListener { */ @Experimental class QueryTerminated private[sql]( - val queryInfo: ContinuousQueryInfo, + val queryInfo: StreamingQueryInfo, val exception: Option[String], val stackTrace: Seq[StackTraceElement]) extends Event } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala similarity index 87% rename from sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 0f4a9c9975565..bae7f56a23f81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ContinuousQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -31,28 +31,28 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} /** * :: Experimental :: - * A class to manage all the [[ContinuousQuery]] active on a [[SparkSession]]. + * A class to manage all the [[StreamingQuery]] active on a [[SparkSession]]. * * @since 2.0.0 */ @Experimental -class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { +class StreamingQueryManager private[sql] (sparkSession: SparkSession) { private[sql] val stateStoreCoordinator = StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) - private val listenerBus = new ContinuousQueryListenerBus(sparkSession.sparkContext.listenerBus) - private val activeQueries = new mutable.HashMap[Long, ContinuousQuery] + private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus) + private val activeQueries = new mutable.HashMap[Long, StreamingQuery] private val activeQueriesLock = new Object private val awaitTerminationLock = new Object - private var lastTerminatedQuery: ContinuousQuery = null + private var lastTerminatedQuery: StreamingQuery = null /** * Returns a list of active queries associated with this SQLContext * * @since 2.0.0 */ - def active: Array[ContinuousQuery] = activeQueriesLock.synchronized { + def active: Array[StreamingQuery] = activeQueriesLock.synchronized { activeQueries.values.toArray } @@ -61,7 +61,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { * * @since 2.0.0 */ - def get(id: Long): ContinuousQuery = activeQueriesLock.synchronized { + def get(id: Long): StreamingQuery = activeQueriesLock.synchronized { activeQueries.get(id).orNull } @@ -81,7 +81,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { * users need to stop all of them after any of them terminates with exception, and then check the * `query.exception()` for each query. * - * @throws ContinuousQueryException, if any query has terminated with an exception + * @throws StreamingQueryException, if any query has terminated with an exception * * @since 2.0.0 */ @@ -113,7 +113,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { * users need to stop all of them after any of them terminates with exception, and then check the * `query.exception()` for each query. * - * @throws ContinuousQueryException, if any query has terminated with an exception + * @throws StreamingQueryException, if any query has terminated with an exception * * @since 2.0.0 */ @@ -146,31 +146,31 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { } /** - * Register a [[ContinuousQueryListener]] to receive up-calls for life cycle events of - * [[ContinuousQuery]]. + * Register a [[StreamingQueryListener]] to receive up-calls for life cycle events of + * [[StreamingQuery]]. * * @since 2.0.0 */ - def addListener(listener: ContinuousQueryListener): Unit = { + def addListener(listener: StreamingQueryListener): Unit = { listenerBus.addListener(listener) } /** - * Deregister a [[ContinuousQueryListener]]. + * Deregister a [[StreamingQueryListener]]. * * @since 2.0.0 */ - def removeListener(listener: ContinuousQueryListener): Unit = { + def removeListener(listener: StreamingQueryListener): Unit = { listenerBus.removeListener(listener) } /** Post a listener event */ - private[sql] def postListenerEvent(event: ContinuousQueryListener.Event): Unit = { + private[sql] def postListenerEvent(event: StreamingQueryListener.Event): Unit = { listenerBus.post(event) } /** - * Start a [[ContinuousQuery]]. + * Start a [[StreamingQuery]]. * @param userSpecifiedName Query name optionally specified by the user. * @param userSpecifiedCheckpointLocation Checkpoint location optionally specified by the user. * @param df Streaming DataFrame. @@ -193,7 +193,7 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { useTempCheckpointLocation: Boolean = false, recoverFromCheckpointLocation: Boolean = true, trigger: Trigger = ProcessingTime(0), - triggerClock: Clock = new SystemClock()): ContinuousQuery = { + triggerClock: Clock = new SystemClock()): StreamingQuery = { activeQueriesLock.synchronized { val id = StreamExecution.nextId val name = userSpecifiedName.getOrElse(s"query-$id") @@ -264,8 +264,8 @@ class ContinuousQueryManager private[sql] (sparkSession: SparkSession) { } } - /** Notify (by the ContinuousQuery) that the query has been terminated */ - private[sql] def notifyQueryTermination(terminatedQuery: ContinuousQuery): Unit = { + /** Notify (by the StreamingQuery) that the query has been terminated */ + private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = { activeQueriesLock.synchronized { activeQueries -= terminatedQuery.id } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/Trigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/Trigger.scala index d3fdbac576b60..55be7a711adb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/Trigger.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/Trigger.scala @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.CalendarInterval /** * :: Experimental :: - * Used to indicate how often results should be produced by a [[ContinuousQuery]]. + * Used to indicate how often results should be produced by a [[StreamingQuery]]. * * @since 2.0.0 */ @@ -65,7 +65,7 @@ case class ProcessingTime(intervalMs: Long) extends Trigger { /** * :: Experimental :: - * Used to create [[ProcessingTime]] triggers for [[ContinuousQuery]]s. + * Used to create [[ProcessingTime]] triggers for [[StreamingQuery]]s. * * @since 2.0.0 */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index a5acc970e3a78..9d0a2b3d5b462 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -124,7 +124,7 @@ class FileStreamSinkSuite extends StreamTest { val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath - var query: ContinuousQuery = null + var query: StreamingQuery = null try { query = @@ -156,7 +156,7 @@ class FileStreamSinkSuite extends StreamTest { val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath - var query: ContinuousQuery = null + var query: StreamingQuery = null try { query = @@ -240,7 +240,7 @@ class FileStreamSinkSuite extends StreamTest { val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath - var query: ContinuousQuery = null + var query: StreamingQuery = null try { val writer = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala index 0e157cf7267dc..f9e236c449634 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStressSuite.scala @@ -57,7 +57,7 @@ class FileStressSuite extends StreamTest { @volatile var continue = true @volatile - var stream: ContinuousQuery = null + var stream: StreamingQuery = null val writer = new Thread("stream writer") { override def run(): Unit = { @@ -100,7 +100,7 @@ class FileStressSuite extends StreamTest { val input = spark.readStream.format("text").load(inputDir) - def startStream(): ContinuousQuery = { + def startStream(): StreamingQuery = { val output = input .repartition(5) .as[String] @@ -139,7 +139,7 @@ class FileStressSuite extends StreamTest { try { stream.awaitTermination() } catch { - case ce: ContinuousQueryException => + case ce: StreamingQueryException => failures += 1 } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index cbfa6ff07d131..720ffaf732542 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -353,7 +353,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { case ef: ExpectFailure[_] => verify(currentStream != null, "can not expect failure when stream is not running") try failAfter(streamingTimeout) { - val thrownException = intercept[ContinuousQueryException] { + val thrownException = intercept[StreamingQueryException] { currentStream.awaitTermination() } eventually("microbatch thread not stopped after termination with failure") { @@ -563,7 +563,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { case e: ExpectException[_] => val thrownException = withClue(s"Did not throw ${e.t.runtimeClass.getSimpleName} when expected.") { - intercept[ContinuousQueryException] { + intercept[StreamingQueryException] { failAfter(testTimeout) { awaitTermFunc() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 7f44227ec46fe..8681199817fe6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -40,8 +40,6 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll { import testImplicits._ - - test("simple count, update mode") { val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala similarity index 85% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 8e1de09a96e24..7f4d28cf0598f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -31,10 +31,10 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.util.JsonProtocol -class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { +class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ - import ContinuousQueryListener._ + import StreamingQueryListener._ after { spark.streams.active.foreach(_.stop()) @@ -167,35 +167,35 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStartedInfo = new ContinuousQueryInfo( + val queryStartedInfo = new StreamingQueryInfo( "name", 1, Seq(new SourceStatus("source1", None), new SourceStatus("source2", None)), new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString)) - val queryStarted = new ContinuousQueryListener.QueryStarted(queryStartedInfo) + val queryStarted = new StreamingQueryListener.QueryStarted(queryStartedInfo) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[ContinuousQueryListener.QueryStarted] - assertContinuousQueryInfoEquals(queryStarted.queryInfo, newQueryStarted.queryInfo) + .asInstanceOf[StreamingQueryListener.QueryStarted] + assertStreamingQueryInfoEquals(queryStarted.queryInfo, newQueryStarted.queryInfo) } test("QueryProgress serialization") { - val queryProcessInfo = new ContinuousQueryInfo( + val queryProcessInfo = new StreamingQueryInfo( "name", 1, Seq( new SourceStatus("source1", Some(LongOffset(0).toString)), new SourceStatus("source2", Some(LongOffset(1).toString))), new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) - val queryProcess = new ContinuousQueryListener.QueryProgress(queryProcessInfo) + val queryProcess = new StreamingQueryListener.QueryProgress(queryProcessInfo) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[ContinuousQueryListener.QueryProgress] - assertContinuousQueryInfoEquals(queryProcess.queryInfo, newQueryProcess.queryInfo) + .asInstanceOf[StreamingQueryListener.QueryProgress] + assertStreamingQueryInfoEquals(queryProcess.queryInfo, newQueryProcess.queryInfo) } test("QueryTerminated serialization") { - val queryTerminatedInfo = new ContinuousQueryInfo( + val queryTerminatedInfo = new StreamingQueryInfo( "name", 1, Seq( @@ -203,21 +203,21 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { new SourceStatus("source2", Some(LongOffset(1).toString))), new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) val exception = new RuntimeException("exception") - val queryQueryTerminated = new ContinuousQueryListener.QueryTerminated( + val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( queryTerminatedInfo, Some(exception.getMessage), exception.getStackTrace) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[ContinuousQueryListener.QueryTerminated] - assertContinuousQueryInfoEquals(queryQueryTerminated.queryInfo, newQueryTerminated.queryInfo) + .asInstanceOf[StreamingQueryListener.QueryTerminated] + assertStreamingQueryInfoEquals(queryQueryTerminated.queryInfo, newQueryTerminated.queryInfo) assert(queryQueryTerminated.exception === newQueryTerminated.exception) } - private def assertContinuousQueryInfoEquals( - expected: ContinuousQueryInfo, - actual: ContinuousQueryInfo): Unit = { + private def assertStreamingQueryInfoEquals( + expected: StreamingQueryInfo, + actual: StreamingQueryInfo): Unit = { assert(expected.name === actual.name) assert(expected.sourceStatuses.size === actual.sourceStatuses.size) expected.sourceStatuses.zip(actual.sourceStatuses).foreach { @@ -237,7 +237,7 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(expected.offsetDesc === actual.offsetDesc) } - private def withListenerAdded(listener: ContinuousQueryListener)(body: => Unit): Unit = { + private def withListenerAdded(listener: StreamingQueryListener)(body: => Unit): Unit = { try { failAfter(1 minute) { spark.streams.addListener(listener) @@ -248,23 +248,23 @@ class ContinuousQueryListenerSuite extends StreamTest with BeforeAndAfter { } } - private def addedListeners(): Array[ContinuousQueryListener] = { + private def addedListeners(): Array[StreamingQueryListener] = { val listenerBusMethod = - PrivateMethod[ContinuousQueryListenerBus]('listenerBus) + PrivateMethod[StreamingQueryListenerBus]('listenerBus) val listenerBus = spark.streams invokePrivate listenerBusMethod() - listenerBus.listeners.toArray.map(_.asInstanceOf[ContinuousQueryListener]) + listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) } - class QueryStatusCollector extends ContinuousQueryListener { + class QueryStatusCollector extends StreamingQueryListener { // to catch errors in the async listener events @volatile private var asyncTestWaiter = new Waiter - @volatile var startStatus: ContinuousQueryInfo = null - @volatile var terminationStatus: ContinuousQueryInfo = null + @volatile var startStatus: StreamingQueryInfo = null + @volatile var terminationStatus: StreamingQueryInfo = null @volatile var terminationException: Option[String] = null @volatile var terminationStackTrace: Seq[StackTraceElement] = null - val progressStatuses = new ConcurrentLinkedQueue[ContinuousQueryInfo] + val progressStatuses = new ConcurrentLinkedQueue[StreamingQueryInfo] def reset(): Unit = { startStatus = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala index ef2fcbf73e360..41ffd56cf1290 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.Dataset import org.apache.spark.sql.execution.streaming._ import org.apache.spark.util.Utils -class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { +class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { import AwaitTerminationTester._ import testImplicits._ @@ -215,7 +215,7 @@ class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { /** Run a body of code by defining a query on each dataset */ - private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[ContinuousQuery] => Unit): Unit = { + private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[StreamingQuery] => Unit): Unit = { failAfter(streamingTimeout) { val queries = withClue("Error starting queries") { datasets.zipWithIndex.map { case (ds, i) => @@ -269,7 +269,7 @@ class ContinuousQueryManagerSuite extends StreamTest with BeforeAndAfter { } /** Stop a random active query either with `stop()` or with an error */ - private def stopRandomQueryAsync(stopAfter: Span, withError: Boolean): ContinuousQuery = { + private def stopRandomQueryAsync(stopAfter: Span, withError: Boolean): StreamingQuery = { import scala.concurrent.ExecutionContext.Implicits.global diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index ad6bc27729597..9d58315c20031 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ContinuousQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset, Me import org.apache.spark.util.Utils -class ContinuousQuerySuite extends StreamTest with BeforeAndAfter { +class StreamingQuerySuite extends StreamTest with BeforeAndAfter { import AwaitTerminationTester._ import testImplicits._ @@ -37,7 +37,7 @@ class ContinuousQuerySuite extends StreamTest with BeforeAndAfter { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map { 6 / _} - def startQuery(queryName: String): ContinuousQuery = { + def startQuery(queryName: String): StreamingQuery = { val metadataRoot = Utils.createTempDir(namePrefix = "streaming.checkpoint").getCanonicalPath val writer = mapped.writeStream writer @@ -126,7 +126,7 @@ class ContinuousQuerySuite extends StreamTest with BeforeAndAfter { } /** - * A [[StreamAction]] to test the behavior of `ContinuousQuery.awaitTermination()`. + * A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`. * * @param expectedBehavior Expected behavior (not blocked, blocked, or exception thrown) * @param timeoutMs Timeout in milliseconds @@ -151,7 +151,7 @@ class ContinuousQuerySuite extends StreamTest with BeforeAndAfter { object TestAwaitTermination { /** - * Tests the behavior of `ContinuousQuery.awaitTermination`. + * Tests the behavior of `StreamingQuery.awaitTermination`. * * @param expectedBehavior Expected behavior (not blocked, blocked, or exception thrown) * @param timeoutMs Timeout in milliseconds diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index c6d374f75467a..1aee1934c0791 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} -import org.apache.spark.sql.streaming.{ContinuousQuery, OutputMode, ProcessingTime, StreamTest} +import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, StreamingQuery, StreamTest} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils @@ -265,7 +265,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { test("unique query names") { /** Start a query with a specific name */ - def startQueryWithName(name: String = ""): ContinuousQuery = { + def startQueryWithName(name: String = ""): StreamingQuery = { spark.readStream .format("org.apache.spark.sql.streaming.test") .load("/test") @@ -277,7 +277,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { } /** Start a query without specifying a name */ - def startQueryWithoutName(): ContinuousQuery = { + def startQueryWithoutName(): StreamingQuery = { spark.readStream .format("org.apache.spark.sql.streaming.test") .load("/test") @@ -434,13 +434,13 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { .format("org.apache.spark.sql.streaming.test") .load() - val cq = df.writeStream + val sq = df.writeStream .format("console") .option("checkpointLocation", newMetadataDir) .trigger(ProcessingTime(2.seconds)) .start() - cq.awaitTermination(2000L) + sq.awaitTermination(2000L) } test("prevent all column partitioning") { From eb1d746c4556f3f712d0810ab2496e8af4444155 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 15 Jun 2016 11:50:54 -0700 Subject: [PATCH 0657/1470] [SPARK-15959][SQL] Add the support of hive.metastore.warehouse.dir back ## What changes were proposed in this pull request? This PR adds the support of conf `hive.metastore.warehouse.dir` back. With this patch, the way of setting the warehouse dir is described as follows: * If `spark.sql.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the value of `spark.sql.warehouse.dir`. * If `spark.sql.warehouse.dir` is not set but `hive.metastore.warehouse.dir` is set, `spark.sql.warehouse.dir` will be automatically set to the value of `hive.metastore.warehouse.dir`. The warehouse dir is effectively set to the value of `hive.metastore.warehouse.dir`. * If neither `spark.sql.warehouse.dir` nor `hive.metastore.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the default value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the default value of `spark.sql.warehouse.dir`. ## How was this patch tested? `set hive.metastore.warehouse.dir` in `HiveSparkSubmitSuite`. JIRA: https://issues.apache.org/jira/browse/SPARK-15959 Author: Yin Huai Closes #13679 from yhuai/hiveWarehouseDir. (cherry picked from commit e1585cc74853c497271eecdc943c0eabe1aeb4c1) Signed-off-by: Reynold Xin --- .../spark/sql/internal/SharedState.scala | 29 +++++- .../spark/sql/hive/HiveSharedState.scala | 13 +-- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 91 ++++++++++++++++--- 3 files changed, 106 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index c37f7f12acb55..bc349b4f28bf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.internal import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} import org.apache.spark.sql.execution.CacheManager @@ -30,7 +31,7 @@ import org.apache.spark.util.{MutableURLClassLoader, Utils} /** * A class that holds all state shared across sessions in a given [[SQLContext]]. */ -private[sql] class SharedState(val sparkContext: SparkContext) { +private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { /** * Class for caching query results reused in future executions. @@ -46,7 +47,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) { * The base hadoop configuration which is shared among all spark sessions. It is based on the * default hadoop configuration of Spark, with custom configurations inside `hive-site.xml`. */ - lazy val hadoopConf: Configuration = { + val hadoopConf: Configuration = { val conf = new Configuration(sparkContext.hadoopConfiguration) val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") if (configFile != null) { @@ -66,6 +67,30 @@ private[sql] class SharedState(val sparkContext: SparkContext) { val jarClassLoader = new NonClosableMutableURLClassLoader( org.apache.spark.util.Utils.getContextOrSparkClassLoader) + { + // Set the Hive metastore warehouse path to the one we use + val tempConf = new SQLConf + sparkContext.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } + val hiveWarehouseDir = hadoopConf.get("hive.metastore.warehouse.dir") + if (hiveWarehouseDir != null && !tempConf.contains(SQLConf.WAREHOUSE_PATH.key)) { + // If hive.metastore.warehouse.dir is set and spark.sql.warehouse.dir is not set, + // we will respect the value of hive.metastore.warehouse.dir. + tempConf.setConfString(SQLConf.WAREHOUSE_PATH.key, hiveWarehouseDir) + sparkContext.conf.set(SQLConf.WAREHOUSE_PATH.key, hiveWarehouseDir) + logInfo(s"${SQLConf.WAREHOUSE_PATH.key} is not set, but hive.metastore.warehouse.dir " + + s"is set. Setting ${SQLConf.WAREHOUSE_PATH.key} to the value of " + + s"hive.metastore.warehouse.dir ('$hiveWarehouseDir').") + } else { + // If spark.sql.warehouse.dir is set, we will override hive.metastore.warehouse.dir using + // the value of spark.sql.warehouse.dir. + // When neither spark.sql.warehouse.dir nor hive.metastore.warehouse.dir is set, + // we will set hive.metastore.warehouse.dir to the default value of spark.sql.warehouse.dir. + sparkContext.conf.set("hive.metastore.warehouse.dir", tempConf.warehousePath) + } + + logInfo(s"Warehouse path is '${tempConf.warehousePath}'.") + } + /** * Create a SQLListener then add it into SparkContext, and create a SQLTab if there is SparkUI. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala index 78b1ecbbea742..6b7a333f2d3b8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala @@ -18,9 +18,8 @@ package org.apache.spark.sql.hive import org.apache.spark.SparkContext -import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.client.HiveClient -import org.apache.spark.sql.internal.{SharedState, SQLConf} +import org.apache.spark.sql.internal.SharedState /** @@ -28,18 +27,10 @@ import org.apache.spark.sql.internal.{SharedState, SQLConf} * [[org.apache.spark.sql.SparkSession]] backed by Hive. */ private[hive] class HiveSharedState(override val sparkContext: SparkContext) - extends SharedState(sparkContext) with Logging { + extends SharedState(sparkContext) { // TODO: just share the IsolatedClientLoader instead of the client instance itself - { - // Set the Hive metastore warehouse path to the one we use - val tempConf = new SQLConf - sparkContext.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } - sparkContext.conf.set("hive.metastore.warehouse.dir", tempConf.warehousePath) - logInfo(s"Setting Hive metastore warehouse path to '${tempConf.warehousePath}'") - } - /** * A Hive client used to interact with the metastore. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index d56bede0cc2fd..9bca720a94736 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import java.io.File +import java.io.{BufferedWriter, File, FileWriter} import java.sql.Timestamp import java.util.Date @@ -205,7 +205,7 @@ class HiveSparkSubmitSuite val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val args = Seq( "--class", SetWarehouseLocationTest.getClass.getName.stripSuffix("$"), - "--name", "SetWarehouseLocationTest", + "--name", "SetSparkWarehouseLocationTest", "--master", "local-cluster[2,1,1024]", "--conf", "spark.ui.enabled=false", "--conf", "spark.master.rest.enabled=false", @@ -214,6 +214,45 @@ class HiveSparkSubmitSuite runSparkSubmit(args) } + test("set hive.metastore.warehouse.dir") { + // In this test, we set hive.metastore.warehouse.dir in hive-site.xml but + // not set spark.sql.warehouse.dir. So, the warehouse dir should be + // the value of hive.metastore.warehouse.dir. Also, the value of + // spark.sql.warehouse.dir should be set to the value of hive.metastore.warehouse.dir. + + val hiveWarehouseLocation = Utils.createTempDir() + hiveWarehouseLocation.delete() + val hiveSiteXmlContent = + s""" + | + | + | hive.metastore.warehouse.dir + | $hiveWarehouseLocation + | + | + """.stripMargin + + // Write a hive-site.xml containing a setting of hive.metastore.warehouse.dir. + val hiveSiteDir = Utils.createTempDir() + val file = new File(hiveSiteDir.getCanonicalPath, "hive-site.xml") + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(hiveSiteXmlContent) + bw.close() + + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SetWarehouseLocationTest.getClass.getName.stripSuffix("$"), + "--name", "SetHiveWarehouseLocationTest", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", s"spark.sql.test.expectedWarehouseDir=$hiveWarehouseLocation", + "--conf", s"spark.driver.extraClassPath=${hiveSiteDir.getCanonicalPath}", + "--driver-java-options", "-Dderby.system.durability=test", + unusedJar.toString) + runSparkSubmit(args) + } + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. // This is copied from org.apache.spark.deploy.SparkSubmitSuite private def runSparkSubmit(args: Seq[String]): Unit = { @@ -277,19 +316,43 @@ class HiveSparkSubmitSuite object SetWarehouseLocationTest extends Logging { def main(args: Array[String]): Unit = { Utils.configTestLog4j("INFO") - val warehouseLocation = Utils.createTempDir() - warehouseLocation.delete() - val hiveWarehouseLocation = Utils.createTempDir() - hiveWarehouseLocation.delete() - // We will use the value of spark.sql.warehouse.dir override the - // value of hive.metastore.warehouse.dir. - val sparkSession = SparkSession.builder() + val sparkConf = new SparkConf(loadDefaults = true) + val builder = SparkSession.builder() + .config(sparkConf) .config("spark.ui.enabled", "false") - .config("spark.sql.warehouse.dir", warehouseLocation.toString) - .config("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString) .enableHiveSupport() - .getOrCreate() + val providedExpectedWarehouseLocation = + sparkConf.getOption("spark.sql.test.expectedWarehouseDir") + + val (sparkSession, expectedWarehouseLocation) = providedExpectedWarehouseLocation match { + case Some(warehouseDir) => + // If spark.sql.test.expectedWarehouseDir is set, the warehouse dir is set + // through spark-summit. So, neither spark.sql.warehouse.dir nor + // hive.metastore.warehouse.dir is set at here. + (builder.getOrCreate(), warehouseDir) + case None => + val warehouseLocation = Utils.createTempDir() + warehouseLocation.delete() + val hiveWarehouseLocation = Utils.createTempDir() + hiveWarehouseLocation.delete() + // If spark.sql.test.expectedWarehouseDir is not set, we will set + // spark.sql.warehouse.dir and hive.metastore.warehouse.dir. + // We are expecting that the value of spark.sql.warehouse.dir will override the + // value of hive.metastore.warehouse.dir. + val session = builder + .config("spark.sql.warehouse.dir", warehouseLocation.toString) + .config("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString) + .getOrCreate() + (session, warehouseLocation.toString) + + } + + if (sparkSession.conf.get("spark.sql.warehouse.dir") != expectedWarehouseLocation) { + throw new Exception( + "spark.sql.warehouse.dir is not set to the expected warehouse location " + + s"$expectedWarehouseLocation.") + } val catalog = sparkSession.sessionState.catalog @@ -301,7 +364,7 @@ object SetWarehouseLocationTest extends Logging { val tableMetadata = catalog.getTableMetadata(TableIdentifier("testLocation", Some("default"))) val expectedLocation = - "file:" + warehouseLocation.toString + "/testlocation" + "file:" + expectedWarehouseLocation.toString + "/testlocation" val actualLocation = tableMetadata.storage.locationUri.get if (actualLocation != expectedLocation) { throw new Exception( @@ -317,7 +380,7 @@ object SetWarehouseLocationTest extends Logging { val tableMetadata = catalog.getTableMetadata(TableIdentifier("testLocation", Some("testLocationDB"))) val expectedLocation = - "file:" + warehouseLocation.toString + "/testlocationdb.db/testlocation" + "file:" + expectedWarehouseLocation.toString + "/testlocationdb.db/testlocation" val actualLocation = tableMetadata.storage.locationUri.get if (actualLocation != expectedLocation) { throw new Exception( From de56ea9bfa7dc5ac12a838ee64f435d5b146c10d Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Wed, 15 Jun 2016 11:52:36 -0700 Subject: [PATCH 0658/1470] [SPARK-15518][CORE][FOLLOW-UP] Rename LocalSchedulerBackendEndpoint -> LocalSchedulerBackend ## What changes were proposed in this pull request? This patch is a follow-up to https://github.com/apache/spark/pull/13288 completing the renaming: - LocalScheduler -> LocalSchedulerBackend~~Endpoint~~ ## How was this patch tested? Updated test cases to reflect the name change. Author: Liwei Lin Closes #13683 from lw-lin/rename-backend. (cherry picked from commit 9b234b55d1b5e4a7c80e482b3e297bfb8b583a56) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/SparkContext.scala | 8 ++++---- .../apache/spark/scheduler/TaskSchedulerImpl.scala | 4 ++-- ...dEndpoint.scala => LocalSchedulerBackend.scala} | 12 ++++++------ .../spark/SparkContextSchedulerCreationSuite.scala | 14 +++++++------- 4 files changed, 19 insertions(+), 19 deletions(-) rename core/src/main/scala/org/apache/spark/scheduler/local/{LocalSchedulerBackendEndpoint.scala => LocalSchedulerBackend.scala} (92%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3c5498782ce0e..d56946e932caf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -58,7 +58,7 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend} -import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint +import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump import org.apache.spark.ui.{ConsoleProgressBar, SparkUI} @@ -2429,7 +2429,7 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalSchedulerBackendEndpoint(sc.getConf, scheduler, 1) + val backend = new LocalSchedulerBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) (backend, scheduler) @@ -2441,7 +2441,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 LocalSchedulerBackendEndpoint(sc.getConf, scheduler, threadCount) + val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) @@ -2451,7 +2451,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 LocalSchedulerBackendEndpoint(sc.getConf, scheduler, threadCount) + val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) 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 01e85ca405587..7dd4f6e9d2d90 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -33,13 +33,13 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality -import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint +import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. - * It can also work with a local setup by using a [[LocalSchedulerBackendEndpoint]] and setting + * It can also work with a local setup by using a [[LocalSchedulerBackend]] and setting * isLocal to true. It handles common logic, like determining a scheduling order across jobs, waking * up to launch speculative tasks, etc. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala similarity index 92% rename from core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala rename to core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index ee0658837997f..e386052814039 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackendEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -39,15 +39,15 @@ private case class KillTask(taskId: Long, interruptThread: Boolean) private case class StopExecutor() /** - * Calls to [[LocalSchedulerBackendEndpoint]] are all serialized through LocalEndpoint. Using an - * RpcEndpoint makes the calls on [[LocalSchedulerBackendEndpoint]] asynchronous, which is necessary - * to prevent deadlock between [[LocalSchedulerBackendEndpoint]] and the [[TaskSchedulerImpl]]. + * Calls to [[LocalSchedulerBackend]] are all serialized through LocalEndpoint. Using an + * RpcEndpoint makes the calls on [[LocalSchedulerBackend]] asynchronous, which is necessary + * to prevent deadlock between [[LocalSchedulerBackend]] and the [[TaskSchedulerImpl]]. */ private[spark] class LocalEndpoint( override val rpcEnv: RpcEnv, userClassPath: Seq[URL], scheduler: TaskSchedulerImpl, - executorBackend: LocalSchedulerBackendEndpoint, + executorBackend: LocalSchedulerBackend, private val totalCores: Int) extends ThreadSafeRpcEndpoint with Logging { @@ -93,9 +93,9 @@ private[spark] class LocalEndpoint( /** * Used when running a local version of Spark where the executor, backend, and master all run in * the same JVM. It sits behind a [[TaskSchedulerImpl]] and handles launching tasks on a single - * Executor (created by the [[LocalSchedulerBackendEndpoint]]) running locally. + * Executor (created by the [[LocalSchedulerBackend]]) running locally. */ -private[spark] class LocalSchedulerBackendEndpoint( +private[spark] class LocalSchedulerBackend( conf: SparkConf, scheduler: TaskSchedulerImpl, val totalCores: Int) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 6e5655437aa8c..7d75a93ff6839 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.cluster.mesos.{MesosCoarseGrainedSchedulerBackend, MesosFineGrainedSchedulerBackend} -import org.apache.spark.scheduler.local.LocalSchedulerBackendEndpoint +import org.apache.spark.scheduler.local.LocalSchedulerBackend class SparkContextSchedulerCreationSuite @@ -58,7 +58,7 @@ class SparkContextSchedulerCreationSuite test("local") { val sched = createTaskScheduler("local") sched.backend match { - case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 1) + case s: LocalSchedulerBackend => assert(s.totalCores === 1) case _ => fail() } } @@ -66,7 +66,7 @@ class SparkContextSchedulerCreationSuite test("local-*") { val sched = createTaskScheduler("local[*]") sched.backend match { - case s: LocalSchedulerBackendEndpoint => + case s: LocalSchedulerBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } @@ -76,7 +76,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) sched.backend match { - case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 5) + case s: LocalSchedulerBackend => assert(s.totalCores === 5) case _ => fail() } } @@ -85,7 +85,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[* ,2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalSchedulerBackendEndpoint => + case s: LocalSchedulerBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } @@ -95,7 +95,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[4, 2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalSchedulerBackendEndpoint => assert(s.totalCores === 4) + case s: LocalSchedulerBackend => assert(s.totalCores === 4) case _ => fail() } } @@ -119,7 +119,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local", "client", conf) sched.backend match { - case s: LocalSchedulerBackendEndpoint => assert(s.defaultParallelism() === 16) + case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) case _ => fail() } } From 8ef31fbd78bd4accd3b76ce1b5770a63625aa42f Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Wed, 15 Jun 2016 12:03:00 -0700 Subject: [PATCH 0659/1470] [SPARK-15826][CORE] PipedRDD to allow configurable char encoding ## What changes were proposed in this pull request? Link to jira which describes the problem: https://issues.apache.org/jira/browse/SPARK-15826 The fix in this PR is to allow users specify encoding in the pipe() operation. For backward compatibility, keeping the default value to be system default. ## How was this patch tested? Ran existing unit tests Author: Tejas Patil Closes #13563 from tejasapatil/pipedrdd_utf8. (cherry picked from commit 279bd4aa5fddbabdb0383a3f6f0fc8d91780e092) Signed-off-by: Shixiong Zhu --- .../apache/spark/api/java/JavaRDDLike.scala | 11 ++++++++++ .../scala/org/apache/spark/rdd/PipedRDD.scala | 22 +++++-------------- .../main/scala/org/apache/spark/rdd/RDD.scala | 13 +++++++---- .../org/apache/spark/rdd/PipedRDDSuite.scala | 12 +++++++++- 4 files changed, 36 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index e4ccd9f11bd14..a37c52cbaf210 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -284,6 +284,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { rdd.pipe(command.asScala, env.asScala, null, null, separateWorkingDir, bufferSize) } + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: JList[String], + env: JMap[String, String], + separateWorkingDir: Boolean, + bufferSize: Int, + encoding: String): JavaRDD[String] = { + rdd.pipe(command.asScala, env.asScala, null, null, separateWorkingDir, bufferSize, encoding) + } + /** * 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 diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 49625b7042d94..02b28b72fb0e7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -47,22 +47,10 @@ private[spark] class PipedRDD[T: ClassTag]( printPipeContext: (String => Unit) => Unit, printRDDElement: (T, String => Unit) => Unit, separateWorkingDir: Boolean, - bufferSize: Int) + bufferSize: Int, + encoding: String) extends RDD[String](prev) { - // Similar to Runtime.exec(), if we are given a single string, split it into words - // using a standard StringTokenizer (i.e. by spaces) - def this( - prev: RDD[T], - command: String, - envVars: Map[String, String] = Map(), - printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null, - separateWorkingDir: Boolean = false) = - this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement, - separateWorkingDir, 8192) - - override def getPartitions: Array[Partition] = firstParent[T].partitions /** @@ -129,7 +117,7 @@ private[spark] class PipedRDD[T: ClassTag]( override def run(): Unit = { val err = proc.getErrorStream try { - for (line <- Source.fromInputStream(err).getLines) { + for (line <- Source.fromInputStream(err)(encoding).getLines) { // scalastyle:off println System.err.println(line) // scalastyle:on println @@ -147,7 +135,7 @@ private[spark] class PipedRDD[T: ClassTag]( override def run(): Unit = { TaskContext.setTaskContext(context) val out = new PrintWriter(new BufferedWriter( - new OutputStreamWriter(proc.getOutputStream), bufferSize)) + new OutputStreamWriter(proc.getOutputStream, encoding), bufferSize)) try { // scalastyle:off println // input the pipe context firstly @@ -171,7 +159,7 @@ private[spark] class PipedRDD[T: ClassTag]( }.start() // Return an iterator that read lines from the process's stdout - val lines = Source.fromInputStream(proc.getInputStream).getLines() + val lines = Source.fromInputStream(proc.getInputStream)(encoding).getLines new Iterator[String] { def next(): String = { if (!hasNext()) { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e251421c48fca..b7a5b222087e1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -21,6 +21,7 @@ import java.util.Random import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer +import scala.io.Codec import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} @@ -698,14 +699,14 @@ abstract class RDD[T: ClassTag]( * Return an RDD created by piping elements to a forked external process. */ def pipe(command: String): RDD[String] = withScope { - new PipedRDD(this, command) + pipe(command) } /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: String, env: Map[String, String]): RDD[String] = withScope { - new PipedRDD(this, command, env) + pipe(command, env) } /** @@ -726,6 +727,8 @@ abstract class RDD[T: ClassTag]( * for (e <- record._2) {f(e)} * @param separateWorkingDir Use separate working directories for each task. * @param bufferSize Buffer size for the stdin writer for the piped process. + * @param encoding Char encoding used for interacting (via stdin, stdout and stderr) with + * the piped process * @return the result RDD */ def pipe( @@ -734,12 +737,14 @@ abstract class RDD[T: ClassTag]( printPipeContext: (String => Unit) => Unit = null, printRDDElement: (T, String => Unit) => Unit = null, separateWorkingDir: Boolean = false, - bufferSize: Int = 8192): RDD[String] = withScope { + bufferSize: Int = 8192, + encoding: String = Codec.defaultCharsetCodec.name): RDD[String] = withScope { new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, if (printRDDElement ne null) sc.clean(printRDDElement) else null, separateWorkingDir, - bufferSize) + bufferSize, + encoding) } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index fe2058d613893..27cfdc7aced56 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.io.File import scala.collection.Map +import scala.io.Codec import scala.language.postfixOps import scala.sys.process._ import scala.util.Try @@ -207,7 +208,16 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { } } val hadoopPart1 = generateFakeHadoopPartition() - val pipedRdd = new PipedRDD(nums, "printenv " + varName) + val pipedRdd = + new PipedRDD( + nums, + PipedRDD.tokenize("printenv " + varName), + Map(), + null, + null, + false, + 4092, + Codec.defaultCharsetCodec.name) val tContext = TaskContext.empty() val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray From b8e1b7c8ac7cec71ede10bfd44b0b86d8ba80af7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Jun 2016 13:38:04 -0700 Subject: [PATCH 0660/1470] [SPARK-15888] [SQL] fix Python UDF with aggregate ## What changes were proposed in this pull request? After we move the ExtractPythonUDF rule into physical plan, Python UDF can't work on top of aggregate anymore, because they can't be evaluated before aggregate, should be evaluated after aggregate. This PR add another rule to extract these kind of Python UDF from logical aggregate, create a Project on top of Aggregate. ## How was this patch tested? Added regression tests. The plan of added test query looks like this: ``` == Parsed Logical Plan == 'Project [('k, 's) AS t#26] +- Aggregate [(key#5L)], [(key#5L) AS k#17, sum(cast((value#6) as bigint)) AS s#22L] +- LogicalRDD [key#5L, value#6] == Analyzed Logical Plan == t: int Project [(k#17, s#22L) AS t#26] +- Aggregate [(key#5L)], [(key#5L) AS k#17, sum(cast((value#6) as bigint)) AS s#22L] +- LogicalRDD [key#5L, value#6] == Optimized Logical Plan == Project [(agg#29, agg#30L) AS t#26] +- Aggregate [(key#5L)], [(key#5L) AS agg#29, sum(cast((value#6) as bigint)) AS agg#30L] +- LogicalRDD [key#5L, value#6] == Physical Plan == *Project [pythonUDF0#37 AS t#26] +- BatchEvalPython [(agg#29, agg#30L)], [agg#29, agg#30L, pythonUDF0#37] +- *HashAggregate(key=[(key#5L)#31], functions=[sum(cast((value#6) as bigint))], output=[agg#29,agg#30L]) +- Exchange hashpartitioning((key#5L)#31, 200) +- *HashAggregate(key=[pythonUDF0#34 AS (key#5L)#31], functions=[partial_sum(cast(pythonUDF1#35 as bigint))], output=[(key#5L)#31,sum#33L]) +- BatchEvalPython [(key#5L), (value#6)], [key#5L, value#6, pythonUDF0#34, pythonUDF1#35] +- Scan ExistingRDD[key#5L,value#6] ``` Author: Davies Liu Closes #13682 from davies/fix_py_udf. (cherry picked from commit 5389013acc99367729dfc6deeb2cecc9edd1e24c) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 10 ++- .../spark/sql/execution/SparkOptimizer.scala | 6 +- .../python/BatchEvalPythonExec.scala | 2 + .../execution/python/ExtractPythonUDFs.scala | 70 ++++++++++++++++--- 4 files changed, 77 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 1d5d69169604d..c631ad8a4618d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -339,13 +339,21 @@ def test_broadcast_in_udf(self): def test_udf_with_aggregate_function(self): df = self.spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"]) - from pyspark.sql.functions import udf, col + from pyspark.sql.functions import udf, col, sum from pyspark.sql.types import BooleanType my_filter = udf(lambda a: a == 1, BooleanType()) sel = df.select(col("key")).distinct().filter(my_filter(col("key"))) self.assertEqual(sel.collect(), [Row(key=1)]) + my_copy = udf(lambda x: x, IntegerType()) + my_add = udf(lambda a, b: int(a + b), IntegerType()) + my_strlen = udf(lambda x: len(x), IntegerType()) + sel = df.groupBy(my_copy(col("key")).alias("k"))\ + .agg(sum(my_strlen(col("value"))).alias("s"))\ + .select(my_add(col("k"), col("s")).alias("t")) + self.assertEqual(sel.collect(), [Row(t=4), Row(t=3)]) + def test_basic_functions(self): rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) df = self.spark.read.json(rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 08b2d7fcd4882..12a10cba20fe9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate import org.apache.spark.sql.internal.SQLConf class SparkOptimizer( @@ -28,6 +29,7 @@ class SparkOptimizer( experimentalMethods: ExperimentalMethods) extends Optimizer(catalog, conf) { - override def batches: Seq[Batch] = super.batches :+ Batch( - "User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) + override def batches: Seq[Batch] = super.batches :+ + Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ + Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index 061d7c7f79de8..d9bf4d3ccf698 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -46,6 +46,8 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi def children: Seq[SparkPlan] = child :: Nil + override def producedAttributes: AttributeSet = AttributeSet(output.drop(child.output.length)) + private def collectFunctions(udf: PythonUDF): (ChainedPythonFunctions, Seq[Expression]) = { udf.children match { case Seq(u: PythonUDF) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index ab192360e1c1f..668470ee6a29a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -18,12 +18,68 @@ package org.apache.spark.sql.execution.python import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution import org.apache.spark.sql.execution.SparkPlan + +/** + * Extracts all the Python UDFs in logical aggregate, which depends on aggregate expression or + * grouping key, evaluate them after aggregate. + */ +private[spark] object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { + + /** + * Returns whether the expression could only be evaluated within aggregate. + */ + private def belongAggregate(e: Expression, agg: Aggregate): Boolean = { + e.isInstanceOf[AggregateExpression] || + agg.groupingExpressions.exists(_.semanticEquals(e)) + } + + private def hasPythonUdfOverAggregate(expr: Expression, agg: Aggregate): Boolean = { + expr.find { + e => e.isInstanceOf[PythonUDF] && e.find(belongAggregate(_, agg)).isDefined + }.isDefined + } + + private def extract(agg: Aggregate): LogicalPlan = { + val projList = new ArrayBuffer[NamedExpression]() + val aggExpr = new ArrayBuffer[NamedExpression]() + agg.aggregateExpressions.foreach { expr => + if (hasPythonUdfOverAggregate(expr, agg)) { + // Python UDF can only be evaluated after aggregate + val newE = expr transformDown { + case e: Expression if belongAggregate(e, agg) => + val alias = e match { + case a: NamedExpression => a + case o => Alias(e, "agg")() + } + aggExpr += alias + alias.toAttribute + } + projList += newE.asInstanceOf[NamedExpression] + } else { + aggExpr += expr + projList += expr.toAttribute + } + } + // There is no Python UDF over aggregate expression + Project(projList, agg.copy(aggregateExpressions = aggExpr)) + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case agg: Aggregate if agg.aggregateExpressions.exists(hasPythonUdfOverAggregate(_, agg)) => + extract(agg) + } +} + + /** * Extracts PythonUDFs from operators, rewriting the query plan so that the UDF can be evaluated * alone in a batch. @@ -59,10 +115,12 @@ private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { } /** - * Extract all the PythonUDFs from the current operator. + * Extract all the PythonUDFs from the current operator and evaluate them before the operator. */ - def extract(plan: SparkPlan): SparkPlan = { + private def extract(plan: SparkPlan): SparkPlan = { val udfs = plan.expressions.flatMap(collectEvaluatableUDF) + // ignore the PythonUDF that come from second/third aggregate, which is not used + .filter(udf => udf.references.subsetOf(plan.inputSet)) if (udfs.isEmpty) { // If there aren't any, we are done. plan @@ -89,11 +147,7 @@ private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { // Other cases are disallowed as they are ambiguous or would require a cartesian // product. udfs.filterNot(attributeMap.contains).foreach { udf => - if (udf.references.subsetOf(plan.inputSet)) { - sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") - } else { - sys.error(s"Unable to evaluate PythonUDF $udf. Missing input attributes.") - } + sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") } val rewritten = plan.transformExpressions { From f1e9d2d92456b437803d6ebab77831e66b0eb53e Mon Sep 17 00:00:00 2001 From: Nezih Yigitbasi Date: Wed, 15 Jun 2016 14:07:36 -0700 Subject: [PATCH 0661/1470] [SPARK-15782][YARN] Set spark.jars system property in client mode ## What changes were proposed in this pull request? When `--packages` is specified with `spark-shell` the classes from those packages cannot be found, which I think is due to some of the changes in `SPARK-12343`. In particular `SPARK-12343` removes a line that sets the `spark.jars` system property in client mode, which is used by the repl main class to set the classpath. ## How was this patch tested? Tested manually. This system property is used by the repl to populate its classpath. If this is not set properly the classes for external packages cannot be found. tgravescs vanzin as you may be familiar with this part of the code. Author: Nezih Yigitbasi Closes #13527 from nezihyigitbasi/repl-fix. (cherry picked from commit 4df8df5c2e68f5a5d231c401b04d762d7a648159) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 20 +++++++++++++++++++ .../spark/deploy/SparkSubmitSuite.scala | 12 +++++++++++ .../org/apache/spark/repl/SparkILoop.scala | 11 ++++++++-- .../scala/org/apache/spark/repl/Main.scala | 4 +--- 5 files changed, 43 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d56946e932caf..d8701812ebeda 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -391,7 +391,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - _jars = _conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + _jars = Utils.getUserJars(_conf) _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.nonEmpty)) .toSeq.flatten 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 f9d05409e1c3d..aebd98b3b0254 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2352,6 +2352,26 @@ private[spark] object Utils extends Logging { log.info(s"Started daemon with process name: ${Utils.getProcessName()}") SignalUtils.registerLogger(log) } + + /** + * Unions two comma-separated lists of files and filters out empty strings. + */ + def unionFileLists(leftList: Option[String], rightList: Option[String]): Set[String] = { + var allFiles = Set[String]() + leftList.foreach { value => allFiles ++= value.split(",") } + rightList.foreach { value => allFiles ++= value.split(",") } + allFiles.filter { _.nonEmpty } + } + + def getUserJars(conf: SparkConf): Seq[String] = { + val sparkJars = conf.getOption("spark.jars") + if (conf.get("spark.master") == "yarn") { + val yarnJars = conf.getOption("spark.yarn.dist.jars") + unionFileLists(sparkJars, yarnJars).toSeq + } else { + sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + } + } } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 271897699201b..0b020592b06d3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -570,6 +570,18 @@ class SparkSubmitSuite appArgs.executorMemory should be ("2.3g") } } + + test("comma separated list of files are unioned correctly") { + val left = Option("/tmp/a.jar,/tmp/b.jar") + val right = Option("/tmp/c.jar,/tmp/a.jar") + val emptyString = Option("") + Utils.unionFileLists(left, right) should be (Set("/tmp/a.jar", "/tmp/b.jar", "/tmp/c.jar")) + Utils.unionFileLists(emptyString, emptyString) should be (Set.empty) + Utils.unionFileLists(Option("/tmp/a.jar"), emptyString) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(emptyString, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(None, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(Option("/tmp/a.jar"), None) should be (Set("/tmp/a.jar")) + } // scalastyle:on println // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index b1e95d8fdb600..66de5e462ac17 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1067,12 +1067,19 @@ object SparkILoop extends Logging { private def echo(msg: String) = Console println msg def getAddedJars: Array[String] = { + val conf = new SparkConf().setMaster(getMaster()) val envJars = sys.env.get("ADD_JARS") if (envJars.isDefined) { logWarning("ADD_JARS environment variable is deprecated, use --jar spark submit argument instead") } - val propJars = sys.props.get("spark.jars").flatMap { p => if (p == "") None else Some(p) } - val jars = propJars.orElse(envJars).getOrElse("") + val jars = { + val userJars = Utils.getUserJars(conf) + if (userJars.isEmpty) { + envJars.getOrElse("") + } else { + userJars.mkString(",") + } + } Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 771670fa559a0..28fe84d6fe9bd 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -54,9 +54,7 @@ object Main extends Logging { // Visible for testing private[repl] def doMain(args: Array[String], _interp: SparkILoop): Unit = { interp = _interp - val jars = conf.getOption("spark.jars") - .map(_.replace(",", File.pathSeparator)) - .getOrElse("") + val jars = Utils.getUserJars(conf).mkString(File.pathSeparator) val interpArguments = List( "-Yrepl-class-based", "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", From 4253ba71b6d291f0fcb3b67048ff915595c83c9e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 15 Jun 2016 14:08:55 -0700 Subject: [PATCH 0662/1470] [SPARK-15901][SQL][TEST] Verification of CONVERT_METASTORE_ORC and CONVERT_METASTORE_PARQUET #### What changes were proposed in this pull request? So far, we do not have test cases for verifying whether the external parameters `HiveUtils .CONVERT_METASTORE_ORC` and `HiveUtils.CONVERT_METASTORE_PARQUET` properly works when users use non-default values. This PR is to add such test cases for avoiding potential regression. #### How was this patch tested? N/A Author: gatorsmile Closes #13622 from gatorsmile/addTestCase4parquetOrcConversion. (cherry picked from commit 09925735b5e53db61ed12abae58864670a3a5f98) Signed-off-by: Cheng Lian --- .../spark/sql/hive/orc/OrcQuerySuite.scala | 75 +++++++++++-------- .../apache/spark/sql/hive/parquetSuites.scala | 40 ++++++++++ 2 files changed, 83 insertions(+), 32 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index e6c9c5d4d9cc5..cd41da7214a23 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive.orc -import java.io.File import java.nio.charset.StandardCharsets import org.scalatest.BeforeAndAfterAll @@ -25,7 +24,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.hive.{HiveUtils, MetastoreRelation} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.internal.SQLConf @@ -401,36 +400,48 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } } - test("SPARK-14070 Use ORC data source for SQL queries on ORC tables") { - withTempPath { dir => - withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true", - HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { - val path = dir.getCanonicalPath - - withTable("dummy_orc") { - withTempTable("single") { - spark.sql( - s"""CREATE TABLE dummy_orc(key INT, value STRING) - |STORED AS ORC - |LOCATION '$path' - """.stripMargin) - - val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1) - singleRowDF.createOrReplaceTempView("single") - - spark.sql( - s"""INSERT INTO TABLE dummy_orc - |SELECT key, value FROM single - """.stripMargin) - - val df = spark.sql("SELECT * FROM dummy_orc WHERE key=0") - checkAnswer(df, singleRowDF) - - val queryExecution = df.queryExecution - queryExecution.analyzed.collectFirst { - case _: LogicalRelation => () - }.getOrElse { - fail(s"Expecting the query plan to have LogicalRelation, but got:\n$queryExecution") + test("Verify the ORC conversion parameter: CONVERT_METASTORE_ORC") { + withTempTable("single") { + val singleRowDF = Seq((0, "foo")).toDF("key", "value") + singleRowDF.createOrReplaceTempView("single") + + Seq("true", "false").foreach { orcConversion => + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> orcConversion) { + withTable("dummy_orc") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.sql( + s""" + |CREATE TABLE dummy_orc(key INT, value STRING) + |STORED AS ORC + |LOCATION '$path' + """.stripMargin) + + spark.sql( + s""" + |INSERT INTO TABLE dummy_orc + |SELECT key, value FROM single + """.stripMargin) + + val df = spark.sql("SELECT * FROM dummy_orc WHERE key=0") + checkAnswer(df, singleRowDF) + + val queryExecution = df.queryExecution + if (orcConversion == "true") { + queryExecution.analyzed.collectFirst { + case _: LogicalRelation => () + }.getOrElse { + fail(s"Expecting the query plan to convert orc to data sources, " + + s"but got:\n$queryExecution") + } + } else { + queryExecution.analyzed.collectFirst { + case _: MetastoreRelation => () + }.getOrElse { + fail(s"Expecting no conversion from orc to data sources, " + + s"but got:\n$queryExecution") + } + } } } } 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 06b74da196854..6af9976ea0b8f 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 @@ -676,6 +676,46 @@ class ParquetSourceSuite extends ParquetPartitioningTest { } } + test("Verify the PARQUET conversion parameter: CONVERT_METASTORE_PARQUET") { + withTempTable("single") { + val singleRowDF = Seq((0, "foo")).toDF("key", "value") + singleRowDF.createOrReplaceTempView("single") + + Seq("true", "false").foreach { parquetConversion => + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> parquetConversion) { + val tableName = "test_parquet_ctas" + withTable(tableName) { + sql( + s""" + |CREATE TABLE $tableName STORED AS PARQUET + |AS SELECT tmp.key, tmp.value FROM single tmp + """.stripMargin) + + val df = spark.sql(s"SELECT * FROM $tableName WHERE key=0") + checkAnswer(df, singleRowDF) + + val queryExecution = df.queryExecution + if (parquetConversion == "true") { + queryExecution.analyzed.collectFirst { + case _: LogicalRelation => + }.getOrElse { + fail(s"Expecting the query plan to convert parquet to data sources, " + + s"but got:\n$queryExecution") + } + } else { + queryExecution.analyzed.collectFirst { + case _: MetastoreRelation => + }.getOrElse { + fail(s"Expecting no conversion from parquet to data sources, " + + s"but got:\n$queryExecution") + } + } + } + } + } + } + } + test("values in arrays and maps stored in parquet are always nullable") { val df = createDataFrame(Tuple2(Map(2 -> 3), Seq(4, 5, 6)) :: Nil).toDF("m", "a") val mapType1 = MapType(IntegerType, IntegerType, valueContainsNull = false) From fe8ac729e13a8dfe99cec6425018219bfd1e221a Mon Sep 17 00:00:00 2001 From: Egor Pakhomov Date: Wed, 15 Jun 2016 14:25:55 -0700 Subject: [PATCH 0663/1470] [SPARK-15934] [SQL] Return binary mode in ThriftServer Returning binary mode to ThriftServer for backward compatibility. Tested with Squirrel and Tableau. Author: Egor Pakhomov Closes #13667 from epahomov/SPARK-15095-2.0. --- .../hive/thriftserver/HiveThriftServer2.scala | 18 ++++---- .../HiveThriftServer2Suites.scala | 41 ++++++++++++++++--- .../hive/thriftserver/UISeleniumSuite.scala | 2 +- 3 files changed, 47 insertions(+), 14 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index de70fdc14ef34..e3258d858f1cc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.thrift.ThriftHttpCLIService +import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.HiveServer2 import org.apache.spark.SparkContext @@ -34,7 +34,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.hive.{HiveSharedState, HiveUtils} import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab import org.apache.spark.sql.internal.SQLConf @@ -271,7 +271,7 @@ object HiveThriftServer2 extends Logging { private[hive] class HiveThriftServer2(sqlContext: SQLContext) extends HiveServer2 - with ReflectedCompositeService with Logging { + with ReflectedCompositeService { // state is tracked internally so that the server only attempts to shut down if it successfully // started, and then once only. private val started = new AtomicBoolean(false) @@ -281,18 +281,20 @@ private[hive] class HiveThriftServer2(sqlContext: SQLContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) - if (isBinaryTransportMode(hiveConf)) { - logWarning("Binary mode is not supported, use HTTP mode instead") + val thriftCliService = if (isHTTPTransportMode(hiveConf)) { + new ThriftHttpCLIService(sparkSqlCliService) + } else { + new ThriftBinaryCLIService(sparkSqlCliService) } - val thriftCliService = new ThriftHttpCLIService(sparkSqlCliService) + setSuperField(this, "thriftCLIService", thriftCliService) addService(thriftCliService) initCompositeService(hiveConf) } - private def isBinaryTransportMode(hiveConf: HiveConf): Boolean = { + private def isHTTPTransportMode(hiveConf: HiveConf): Boolean = { val transportMode = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE) - transportMode.toLowerCase(Locale.ENGLISH).equals("binary") + transportMode.toLowerCase(Locale.ENGLISH).equals("http") } 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 b3f4944c9129d..e388c2a082f18 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 @@ -55,8 +55,8 @@ object TestData { val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") } -class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.http +class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { + override def mode: ServerMode.Value = ServerMode.binary private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { // Transport creation logic below mimics HiveConnection.createBinaryTransport @@ -70,8 +70,7 @@ class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { try f(client) finally transport.close() } - // TODO: update this test to work in HTTP mode - ignore("GetInfo Thrift API") { + test("GetInfo Thrift API") { withCLIServiceClient { client => val user = System.getProperty("user.name") val sessionHandle = client.openSession(user, "") @@ -567,7 +566,7 @@ class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { } class SingleSessionSuite extends HiveThriftJdbcTest { - override def mode: ServerMode.Value = ServerMode.http + override def mode: ServerMode.Value = ServerMode.binary override protected def extraConf: Seq[String] = "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil @@ -617,6 +616,38 @@ class SingleSessionSuite extends HiveThriftJdbcTest { } } +class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { + override def mode: ServerMode.Value = ServerMode.http + + test("JDBC query execution") { + withJdbcStatement { statement => + val queries = Seq( + "SET spark.sql.shuffle.partitions=3", + "DROP TABLE IF EXISTS test", + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", + "CACHE TABLE test") + + queries.foreach(statement.execute) + + assertResult(5, "Row count mismatch") { + val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") + resultSet.next() + resultSet.getInt(1) + } + } + } + + test("Checks Hive version") { + withJdbcStatement { statement => + val resultSet = statement.executeQuery("SET spark.sql.hive.version") + resultSet.next() + assert(resultSet.getString(1) === "spark.sql.hive.version") + assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) + } + } +} + object ServerMode extends Enumeration { val binary, http = Value } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index b6b9de1ba6639..bf431cd6b0260 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -36,7 +36,7 @@ class UISeleniumSuite implicit var webDriver: WebDriver = _ var server: HiveThriftServer2 = _ val uiPort = 20000 + Random.nextInt(10000) - override def mode: ServerMode.Value = ServerMode.http + override def mode: ServerMode.Value = ServerMode.binary override def beforeAll(): Unit = { webDriver = new HtmlUnitDriver { From 2c1aae44218f5b6cb2087e2d5c074438351fb250 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Wed, 15 Jun 2016 14:34:15 -0700 Subject: [PATCH 0664/1470] [SPARK-15776][SQL] Divide Expression inside Aggregation function is casted to wrong type ## What changes were proposed in this pull request? This PR fixes the problem that Divide Expression inside Aggregation function is casted to wrong type, which cause `select 1/2` and `select sum(1/2)`returning different result. **Before the change:** ``` scala> sql("select 1/2 as a").show() +---+ | a| +---+ |0.5| +---+ scala> sql("select sum(1/2) as a").show() +---+ | a| +---+ |0 | +---+ scala> sql("select sum(1 / 2) as a").schema res4: org.apache.spark.sql.types.StructType = StructType(StructField(a,LongType,true)) ``` **After the change:** ``` scala> sql("select 1/2 as a").show() +---+ | a| +---+ |0.5| +---+ scala> sql("select sum(1/2) as a").show() +---+ | a| +---+ |0.5| +---+ scala> sql("select sum(1/2) as a").schema res4: org.apache.spark.sql.types.StructType = StructType(StructField(a,DoubleType,true)) ``` ## How was this patch tested? Unit test. This PR is based on https://github.com/apache/spark/pull/13524 by Sephiroth-Lin Author: Sean Zhong Closes #13651 from clockfly/SPARK-15776. (cherry picked from commit 9bd80ad6bd43462d16ce24cda77cdfaa336c4e02) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/TypeCoercion.scala | 8 ++-- .../sql/catalyst/expressions/arithmetic.scala | 3 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 32 ++++++++++++++++ .../ExpressionTypeCheckingSuite.scala | 2 +- .../catalyst/analysis/TypeCoercionSuite.scala | 37 ++++++++++++++++++- .../ArithmeticExpressionSuite.scala | 19 +++++----- .../plans/ConstraintPropagationSuite.scala | 4 +- 7 files changed, 86 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index a5b5b91e4ab3a..16df628a5730c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -525,14 +525,16 @@ object TypeCoercion { def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who has not been resolved yet, // as this is an extra rule which should be applied at last. - case e if !e.resolved => e + case e if !e.childrenResolved => e // Decimal and Double remain the same case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d - - case Divide(left, right) => Divide(Cast(left, DoubleType), Cast(right, DoubleType)) + case Divide(left, right) if isNumeric(left) && isNumeric(right) => + Divide(Cast(left, DoubleType), Cast(right, DoubleType)) } + + private def isNumeric(ex: Expression): Boolean = ex.dataType.isInstanceOf[NumericType] } /** 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 b2df79a58884b..4db1352291e0b 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 @@ -213,7 +213,7 @@ case class Multiply(left: Expression, right: Expression) case class Divide(left: Expression, right: Expression) extends BinaryArithmetic with NullIntolerant { - override def inputType: AbstractDataType = NumericType + override def inputType: AbstractDataType = TypeCollection(DoubleType, DecimalType) override def symbol: String = "/" override def decimalMethod: String = "$div" @@ -221,7 +221,6 @@ case class Divide(left: Expression, right: Expression) private lazy val div: (Any, Any) => Any = dataType match { case ft: FractionalType => ft.fractional.asInstanceOf[Fractional[Any]].div - case it: IntegralType => it.integral.asInstanceOf[Integral[Any]].quot } override def eval(input: InternalRow): Any = { 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 77ea29ead92cc..102c78bd72111 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 @@ -345,4 +345,36 @@ class AnalysisSuite extends AnalysisTest { assertAnalysisSuccess(query) } + + private def assertExpressionType( + expression: Expression, + expectedDataType: DataType): Unit = { + val afterAnalyze = + Project(Seq(Alias(expression, "a")()), OneRowRelation).analyze.expressions.head + if (!afterAnalyze.dataType.equals(expectedDataType)) { + fail( + s""" + |data type of expression $expression doesn't match expected: + |Actual data type: + |${afterAnalyze.dataType} + | + |Expected data type: + |${expectedDataType} + """.stripMargin) + } + } + + test("SPARK-15776: test whether Divide expression's data type can be deduced correctly by " + + "analyzer") { + assertExpressionType(sum(Divide(1, 2)), DoubleType) + assertExpressionType(sum(Divide(1.0, 2)), DoubleType) + assertExpressionType(sum(Divide(1, 2.0)), DoubleType) + assertExpressionType(sum(Divide(1.0, 2.0)), DoubleType) + assertExpressionType(sum(Divide(1, 2.0f)), DoubleType) + assertExpressionType(sum(Divide(1.0f, 2)), DoubleType) + assertExpressionType(sum(Divide(1, Decimal(2))), DecimalType(31, 11)) + assertExpressionType(sum(Divide(Decimal(1), 2)), DecimalType(31, 11)) + assertExpressionType(sum(Divide(Decimal(1), 2.0)), DoubleType) + assertExpressionType(sum(Divide(1.0, Decimal(2.0))), DoubleType) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 660dc86c3e284..54436ea9a4a72 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -85,7 +85,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertError(Subtract('booleanField, 'booleanField), "requires (numeric or calendarinterval) type") assertError(Multiply('booleanField, 'booleanField), "requires numeric type") - assertError(Divide('booleanField, 'booleanField), "requires numeric type") + assertError(Divide('booleanField, 'booleanField), "requires (double or decimal) type") assertError(Remainder('booleanField, 'booleanField), "requires numeric type") assertError(BitwiseAnd('booleanField, 'booleanField), "requires integral type") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 7435399b1492a..971c99b671671 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.catalyst.analysis import java.sql.Timestamp +import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{Division, FunctionArgumentConversion} +import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -199,9 +201,20 @@ class TypeCoercionSuite extends PlanTest { } private def ruleTest(rule: Rule[LogicalPlan], initial: Expression, transformed: Expression) { + ruleTest(Seq(rule), initial, transformed) + } + + private def ruleTest( + rules: Seq[Rule[LogicalPlan]], + initial: Expression, + transformed: Expression): Unit = { val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) + val analyzer = new RuleExecutor[LogicalPlan] { + override val batches = Seq(Batch("Resolution", FixedPoint(3), rules: _*)) + } + comparePlans( - rule(Project(Seq(Alias(initial, "a")()), testRelation)), + analyzer.execute(Project(Seq(Alias(initial, "a")()), testRelation)), Project(Seq(Alias(transformed, "a")()), testRelation)) } @@ -630,6 +643,26 @@ class TypeCoercionSuite extends PlanTest { Seq(Cast(Literal(1), StringType), Cast(Literal("b"), StringType))) ) } + + test("SPARK-15776 Divide expression's dataType should be casted to Double or Decimal " + + "in aggregation function like sum") { + val rules = Seq(FunctionArgumentConversion, Division) + // Casts Integer to Double + ruleTest(rules, sum(Divide(4, 3)), sum(Divide(Cast(4, DoubleType), Cast(3, DoubleType)))) + // Left expression is Double, right expression is Int. Another rule ImplicitTypeCasts will + // cast the right expression to Double. + ruleTest(rules, sum(Divide(4.0, 3)), sum(Divide(4.0, 3))) + // Left expression is Int, right expression is Double + ruleTest(rules, sum(Divide(4, 3.0)), sum(Divide(Cast(4, DoubleType), Cast(3.0, DoubleType)))) + // Casts Float to Double + ruleTest( + rules, + sum(Divide(4.0f, 3)), + sum(Divide(Cast(4.0f, DoubleType), Cast(3, DoubleType)))) + // Left expression is Decimal, right expression is Int. Another rule DecimalPrecision will cast + // the right expression to Decimal. + ruleTest(rules, sum(Divide(Decimal(4.0), 3)), sum(Divide(Decimal(4.0), 3))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 72285c6a24199..2e37887fbc822 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -117,8 +117,13 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper } } + private def testDecimalAndDoubleType(testFunc: (Int => Any) => Unit): Unit = { + testFunc(_.toDouble) + testFunc(Decimal(_)) + } + test("/ (Divide) basic") { - testNumericDataTypes { convert => + testDecimalAndDoubleType { convert => val left = Literal(convert(2)) val right = Literal(convert(1)) val dataType = left.dataType @@ -128,12 +133,14 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Divide(left, Literal(convert(0))), null) // divide by zero } - DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => + Seq(DoubleType, DecimalType.SYSTEM_DEFAULT).foreach { tpe => checkConsistencyBetweenInterpretedAndCodegen(Divide, tpe, tpe) } } - test("/ (Divide) for integral type") { + // By fixing SPARK-15776, Divide's inputType is required to be DoubleType of DecimalType. + // TODO: in future release, we should add a IntegerDivide to support integral types. + ignore("/ (Divide) for integral type") { checkEvaluation(Divide(Literal(1.toByte), Literal(2.toByte)), 0.toByte) checkEvaluation(Divide(Literal(1.toShort), Literal(2.toShort)), 0.toShort) checkEvaluation(Divide(Literal(1), Literal(2)), 0) @@ -143,12 +150,6 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Divide(positiveLongLit, negativeLongLit), 0L) } - test("/ (Divide) for floating point") { - checkEvaluation(Divide(Literal(1.0f), Literal(2.0f)), 0.5f) - checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5) - checkEvaluation(Divide(Literal(Decimal(1.0)), Literal(Decimal(2.0))), Decimal(0.5)) - } - test("% (Remainder)") { testNumericDataTypes { convert => val left = Literal(convert(1)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index 81cc6b123cdd4..0b73b5e009b79 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -298,7 +298,7 @@ class ConstraintPropagationSuite extends SparkFunSuite { Cast(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") + Cast(100, LongType) === Cast(resolveColumn(tr, "c"), LongType), Cast(resolveColumn(tr, "d"), DoubleType) / - Cast(Cast(10, LongType), DoubleType) === + Cast(10, DoubleType) === Cast(resolveColumn(tr, "e"), DoubleType), IsNotNull(resolveColumn(tr, "a")), IsNotNull(resolveColumn(tr, "b")), @@ -312,7 +312,7 @@ class ConstraintPropagationSuite extends SparkFunSuite { Cast(resolveColumn(tr, "a"), LongType) * resolveColumn(tr, "b") - Cast(10, LongType) >= Cast(resolveColumn(tr, "c"), LongType), Cast(resolveColumn(tr, "d"), DoubleType) / - Cast(Cast(10, LongType), DoubleType) < + Cast(10, DoubleType) < Cast(resolveColumn(tr, "e"), DoubleType), IsNotNull(resolveColumn(tr, "a")), IsNotNull(resolveColumn(tr, "b")), From 73bf87f3c80a0935f67e2b20f7a9b1a6766ec227 Mon Sep 17 00:00:00 2001 From: Nirman Narang Date: Wed, 15 Jun 2016 15:36:31 -0700 Subject: [PATCH 0665/1470] [SPARK-7848][STREAMING][UPDATE SPARKSTREAMING DOCS TO INCORPORATE IMPORTANT POINTS.] Updated the SparkStreaming Doc with some important points. Author: Nirman Narang Closes #11114 from nirmannarang/SPARK-7848. (cherry picked from commit 04d7b3d2b6b9953de399fd743e596c310234042f) Signed-off-by: Reynold Xin --- docs/streaming-programming-guide.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 4ea3b60268d12..db06a65b994be 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -2181,6 +2181,25 @@ consistent batch processing times. Make sure you set the CMS GC on both the driv - Persist RDDs using the `OFF_HEAP` storage level. See more detail in the [Spark Programming Guide](programming-guide.html#rdd-persistence). - Use more executors with smaller heap sizes. This will reduce the GC pressure within each JVM heap. +*** + +##### Important points to remember: +{:.no_toc} +- A DStream is associated with a single receiver. For attaining read parallelism multiple receivers i.e. multiple DStreams need to be created. A receiver is run within an executor. It occupies one core. Ensure that there are enough cores for processing after receiver slots are booked i.e. `spark.cores.max` should take the receiver slots into account. The receivers are allocated to executors in a round robin fashion. + +- When data is received from a stream source, receiver creates blocks of data. A new block of data is generated every blockInterval milliseconds. N blocks of data are created during the batchInterval where N = batchInterval/blockInterval. These blocks are distributed by the BlockManager of the current executor to the block managers of other executors. After that, the Network Input Tracker running on the driver is informed about the block locations for further processing. + +- A RDD is created on the driver for the blocks created during the batchInterval. The blocks generated during the batchInterval are partitions of the RDD. Each partition is a task in spark. blockInterval== batchinterval would mean that a single partition is created and probably it is processed locally. + +- The map tasks on the blocks are processed in the executors (one that received the block, and another where the block was replicated) that has the blocks irrespective of block interval, unless non-local scheduling kicks in. +Having bigger blockinterval means bigger blocks. A high value of `spark.locality.wait` increases the chance of processing a block on the local node. A balance needs to be found out between these two parameters to ensure that the bigger blocks are processed locally. + +- Instead of relying on batchInterval and blockInterval, you can define the number of partitions by calling `inputDstream.repartition(n)`. This reshuffles the data in RDD randomly to create n number of partitions. Yes, for greater parallelism. Though comes at the cost of a shuffle. An RDD's processing is scheduled by driver's jobscheduler as a job. At a given point of time only one job is active. So, if one job is executing the other jobs are queued. + +- If you have two dstreams there will be two RDDs formed and there will be two jobs created which will be scheduled one after the another. To avoid this, you can union two dstreams. This will ensure that a single unionRDD is formed for the two RDDs of the dstreams. This unionRDD is then considered as a single job. However the partitioning of the RDDs is not impacted. + +- If the batch processing time is more than batchinterval then obviously the receiver's memory will start filling up and will end up in throwing exceptions (most probably BlockNotFoundException). Currently there is no way to pause the receiver. Using SparkConf configuration `spark.streaming.receiver.maxRate`, rate of receiver can be limited. + *************************************************************************************************** *************************************************************************************************** From 9acf254edadcbecb0f0bee00578dc7f228b5ec1e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Jun 2016 15:54:29 -0700 Subject: [PATCH 0666/1470] Revert "[SPARK-15782][YARN] Set spark.jars system property in client mode" This reverts commit f1e9d2d92456b437803d6ebab77831e66b0eb53e. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 20 ------------------- .../spark/deploy/SparkSubmitSuite.scala | 12 ----------- .../org/apache/spark/repl/SparkILoop.scala | 11 ++-------- .../scala/org/apache/spark/repl/Main.scala | 4 +++- 5 files changed, 6 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d8701812ebeda..d56946e932caf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -391,7 +391,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - _jars = Utils.getUserJars(_conf) + _jars = _conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.nonEmpty)) .toSeq.flatten 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 aebd98b3b0254..f9d05409e1c3d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2352,26 +2352,6 @@ private[spark] object Utils extends Logging { log.info(s"Started daemon with process name: ${Utils.getProcessName()}") SignalUtils.registerLogger(log) } - - /** - * Unions two comma-separated lists of files and filters out empty strings. - */ - def unionFileLists(leftList: Option[String], rightList: Option[String]): Set[String] = { - var allFiles = Set[String]() - leftList.foreach { value => allFiles ++= value.split(",") } - rightList.foreach { value => allFiles ++= value.split(",") } - allFiles.filter { _.nonEmpty } - } - - def getUserJars(conf: SparkConf): Seq[String] = { - val sparkJars = conf.getOption("spark.jars") - if (conf.get("spark.master") == "yarn") { - val yarnJars = conf.getOption("spark.yarn.dist.jars") - unionFileLists(sparkJars, yarnJars).toSeq - } else { - sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten - } - } } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 0b020592b06d3..271897699201b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -570,18 +570,6 @@ class SparkSubmitSuite appArgs.executorMemory should be ("2.3g") } } - - test("comma separated list of files are unioned correctly") { - val left = Option("/tmp/a.jar,/tmp/b.jar") - val right = Option("/tmp/c.jar,/tmp/a.jar") - val emptyString = Option("") - Utils.unionFileLists(left, right) should be (Set("/tmp/a.jar", "/tmp/b.jar", "/tmp/c.jar")) - Utils.unionFileLists(emptyString, emptyString) should be (Set.empty) - Utils.unionFileLists(Option("/tmp/a.jar"), emptyString) should be (Set("/tmp/a.jar")) - Utils.unionFileLists(emptyString, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) - Utils.unionFileLists(None, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) - Utils.unionFileLists(Option("/tmp/a.jar"), None) should be (Set("/tmp/a.jar")) - } // scalastyle:on println // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 66de5e462ac17..b1e95d8fdb600 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1067,19 +1067,12 @@ object SparkILoop extends Logging { private def echo(msg: String) = Console println msg def getAddedJars: Array[String] = { - val conf = new SparkConf().setMaster(getMaster()) val envJars = sys.env.get("ADD_JARS") if (envJars.isDefined) { logWarning("ADD_JARS environment variable is deprecated, use --jar spark submit argument instead") } - val jars = { - val userJars = Utils.getUserJars(conf) - if (userJars.isEmpty) { - envJars.getOrElse("") - } else { - userJars.mkString(",") - } - } + val propJars = sys.props.get("spark.jars").flatMap { p => if (p == "") None else Some(p) } + val jars = propJars.orElse(envJars).getOrElse("") Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 28fe84d6fe9bd..771670fa559a0 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -54,7 +54,9 @@ object Main extends Logging { // Visible for testing private[repl] def doMain(args: Array[String], _interp: SparkILoop): Unit = { interp = _interp - val jars = Utils.getUserJars(conf).mkString(File.pathSeparator) + val jars = conf.getOption("spark.jars") + .map(_.replace(",", File.pathSeparator)) + .getOrElse("") val interpArguments = List( "-Yrepl-class-based", "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", From 382735c419563cb51e497fd78077670520bb01db Mon Sep 17 00:00:00 2001 From: Wojciech Jurczyk Date: Wed, 15 Jun 2016 15:58:42 -0700 Subject: [PATCH 0667/1470] [DOCS] Fix Gini and Entropy scaladocs in context of multiclass classification The PR changes outdated scaladocs for Gini and Entropy classes. Since PR #886 Spark supports multiclass classification, but the docs tell only about binary classification. Author: Wojciech Jurczyk Closes #11252 from wjur/wjur/docs_multiclass. (cherry picked from commit 6e0b3d795c572a9df589e148c0e55e4eeba7946c) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/mllib/tree/impurity/Entropy.scala | 3 +-- .../main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index ff7700d2d1b7f..3a731f45d6a07 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -21,8 +21,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} /** * :: Experimental :: - * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during - * binary classification. + * Class for calculating entropy during multiclass classification. */ @Since("1.0.0") @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 58dc79b7398e2..7730c0a8c1117 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} * :: Experimental :: * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] - * during binary classification. + * during multiclass classification. */ @Since("1.0.0") @Experimental From bc83b09ee653615306e45566012d42d7917d265f Mon Sep 17 00:00:00 2001 From: KaiXinXiaoLei Date: Wed, 15 Jun 2016 16:11:46 -0700 Subject: [PATCH 0668/1470] [SPARK-12492][SQL] Add missing SQLExecution.withNewExecutionId for hiveResultString ## What changes were proposed in this pull request? Add missing SQLExecution.withNewExecutionId for hiveResultString so that queries running in `spark-sql` will be shown in Web UI. Closes #13115 ## How was this patch tested? Existing unit tests. Author: KaiXinXiaoLei Closes #13689 from zsxwing/pr13115. (cherry picked from commit 3e6d567a4688f064f2a2259c8e436b7c628a431c) Signed-off-by: Shixiong Zhu --- .../spark/sql/execution/QueryExecution.scala | 31 ++++++++++--------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index a2d45026e0b85..ba2332398f962 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -111,24 +111,27 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { */ def hiveResultString(): Seq[String] = executedPlan match { case ExecutedCommandExec(desc: DescribeTableCommand) => - // If it is a describe command for a Hive table, we want to have the output format - // be similar with Hive. - desc.run(sparkSession).map { - case Row(name: String, dataType: String, comment) => - Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse("")) - .map(s => String.format(s"%-20s", s)) - .mkString("\t") + SQLExecution.withNewExecutionId(sparkSession, this) { + // If it is a describe command for a Hive table, we want to have the output format + // be similar with Hive. + desc.run(sparkSession).map { + case Row(name: String, dataType: String, comment) => + Seq(name, dataType, + Option(comment.asInstanceOf[String]).getOrElse("")) + .map(s => String.format(s"%-20s", s)) + .mkString("\t") + } } case command: ExecutedCommandExec => command.executeCollect().map(_.getString(0)) - case other => - val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq - // We need the types so we can output struct field names - val types = analyzed.output.map(_.dataType) - // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq + SQLExecution.withNewExecutionId(sparkSession, this) { + val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq + // We need the types so we can output struct field names + val types = analyzed.output.map(_.dataType) + // Reformat to match hive tab delimited output. + result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq + } } /** Formats a datum (based on the given data type) and returns the string representation. */ From 21daee4392058365bb2a2eb658953282152fa75e Mon Sep 17 00:00:00 2001 From: Wayne Song Date: Wed, 15 Jun 2016 20:09:47 -0700 Subject: [PATCH 0669/1470] [SPARK-13498][SQL] Increment the recordsRead input metric for JDBC data source ## What changes were proposed in this pull request? This patch brings https://github.com/apache/spark/pull/11373 up-to-date and increments the record count for JDBC data source. Closes #11373. ## How was this patch tested? N/A Author: Reynold Xin Closes #13694 from rxin/SPARK-13498. (cherry picked from commit ebdd7512723851934241bd87fe7b25fd60cc58d8) Signed-off-by: Reynold Xin --- .../apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 6a5564addf48c..8d0906e574258 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -374,6 +374,7 @@ private[sql] class JDBCRDD( var nextValue: InternalRow = null context.addTaskCompletionListener{ context => close() } + val inputMetrics = context.taskMetrics().inputMetrics val part = thePart.asInstanceOf[JDBCPartition] val conn = getConnection() val dialect = JdbcDialects.get(url) @@ -398,6 +399,7 @@ private[sql] class JDBCRDD( def getNext(): InternalRow = { if (rs.next()) { + inputMetrics.incRecordsRead(1) var i = 0 while (i < conversions.length) { val pos = i + 1 From cb3bb1901dfb6dca2e190b1b0e8fe37aba76413c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 15 Jun 2016 20:11:23 -0700 Subject: [PATCH 0670/1470] [SPARK-15851][BUILD] Fix the call of the bash script to enable proper run in Windows ## What changes were proposed in this pull request? The way bash script `build/spark-build-info` is called from core/pom.xml prevents Spark building on Windows. Instead of calling the script directly we call bash and pass the script as an argument. This enables running it on Windows with bash installed which typically comes with Git. This brings https://github.com/apache/spark/pull/13612 up-to-date and also addresses comments from the code review. Closes #13612 ## How was this patch tested? I built manually (on a Mac) to verify it didn't break Mac compilation. Author: Reynold Xin Author: avulanov Closes #13691 from rxin/SPARK-15851. (cherry picked from commit 5a52ba0f952b21818ed73cb253381f6a3799dc46) Signed-off-by: Reynold Xin --- core/pom.xml | 3 ++- project/SparkBuild.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 90c8f97f2bbac..b1f0b03b4a589 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -357,7 +357,8 @@ - + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2f7da31e5508c..bce7f1d69beb7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -458,7 +458,7 @@ object Core { resourceGenerators in Compile += Def.task { val buildScript = baseDirectory.value + "/../build/spark-build-info" val targetDir = baseDirectory.value + "/target/extra-resources/" - val command = buildScript + " " + targetDir + " " + version.value + val command = Seq("bash", buildScript, targetDir, version.value) Process(command).!! val propsFile = baseDirectory.value / "target" / "extra-resources" / "spark-version-info.properties" Seq(propsFile) From 61738a38a3bc9182996455392e8ade4bed4db607 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 3 Jun 2016 14:26:24 -0700 Subject: [PATCH 0671/1470] [SPARK-15547][SQL] nested case class in encoder can have different number of fields from the real schema There are 2 kinds of `GetStructField`: 1. resolved from `UnresolvedExtractValue`, and it will have a `name` property. 2. created when we build deserializer expression for nested tuple, no `name` property. When we want to validate the ordinals of nested tuple, we should only catch `GetStructField` without the name property. new test in `EncoderResolutionSuite` Author: Wenchen Fan Closes #13474 from cloud-fan/ordinal-check. (cherry picked from commit 61b80d552aafb262b5f817f7bc9c0acd0328715b) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 7 ++++++- .../sql/catalyst/encoders/EncoderResolutionSuite.scala | 6 ++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4cbedbda6da1a..a5755616329ab 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 @@ -1964,7 +1964,12 @@ class Analyzer( */ private def validateNestedTupleFields(deserializer: Expression): Unit = { val structChildToOrdinals = deserializer - .collect { case g: GetStructField => g } + // There are 2 kinds of `GetStructField`: + // 1. resolved from `UnresolvedExtractValue`, and it will have a `name` property. + // 2. created when we build deserializer expression for nested tuple, no `name` property. + // Here we want to validate the ordinals of nested tuple, so we should only catch + // `GetStructField` without the name property. + .collect { case g: GetStructField if g.name.isEmpty => g } .groupBy(_.child) .mapValues(_.map(_.ordinal).distinct.sorted) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index 7251202c7bd58..802397d50e85c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -115,6 +115,12 @@ class EncoderResolutionSuite extends PlanTest { } } + test("nested case class can have different number of fields from the real schema") { + val encoder = ExpressionEncoder[(String, StringIntClass)] + val attrs = Seq('a.string, 'b.struct('a.string, 'b.int, 'c.int)) + encoder.resolveAndBind(attrs) + } + test("throw exception if real type is not compatible with encoder schema") { val msg1 = intercept[AnalysisException] { ExpressionEncoder[StringIntClass].resolveAndBind(Seq('a.string, 'b.long)) From f0279b05cc893fb7671bdbcbf20aded1c427d6e0 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 15 Jun 2016 21:33:26 -0700 Subject: [PATCH 0672/1470] [SPARK-15824][SQL] Execute WITH .... INSERT ... statements immediately ## What changes were proposed in this pull request? We currently immediately execute `INSERT` commands when they are issued. This is not the case as soon as we use a `WITH` to define common table expressions, for example: ```sql WITH tbl AS (SELECT * FROM x WHERE id = 10) INSERT INTO y SELECT * FROM tbl ``` This PR fixes this problem. This PR closes https://github.com/apache/spark/pull/13561 (which fixes the a instance of this problem in the ThriftSever). ## How was this patch tested? Added a test to `InsertSuite` Author: Herman van Hovell Closes #13678 from hvanhovell/SPARK-15824. (cherry picked from commit b75f454f946714b93fe561055cd53b0686187d2e) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../spark/sql/sources/InsertSuite.scala | 25 +++++++++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index fba4066af6bdb..02cc3985b0100 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -179,7 +179,7 @@ class Dataset[T] private[sql]( case _ => false } - queryExecution.logical match { + queryExecution.analyzed match { // For various commands (like DDL) and queries with side effects, we force query execution // to happen right away to let these side effects take place eagerly. case p if hasSideEffects(p) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 4780eb473d79b..bade41b1eddf9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.sources import java.io.File import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils @@ -259,4 +260,28 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { spark.catalog.dropTempView("oneToTen") } + + test("SPARK-15824 - Execute an INSERT wrapped in a WITH statement immediately") { + withTable("target", "target2") { + sql(s"CREATE TABLE target(a INT, b STRING) USING JSON") + sql("WITH tbl AS (SELECT * FROM jt) INSERT OVERWRITE TABLE target SELECT a, b FROM tbl") + checkAnswer( + sql("SELECT a, b FROM target"), + sql("SELECT a, b FROM jt") + ) + + sql(s"CREATE TABLE target2(a INT, b STRING) USING JSON") + val e = sql( + """ + |WITH tbl AS (SELECT * FROM jt) + |FROM tbl + |INSERT INTO target2 SELECT a, b WHERE a <= 5 + |INSERT INTO target2 SELECT a, b WHERE a > 5 + """.stripMargin) + checkAnswer( + sql("SELECT a, b FROM target2"), + sql("SELECT a, b FROM jt") + ) + } + } } 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 d24cde232119c..224ff3823b594 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 @@ -544,7 +544,7 @@ private[hive] case class InsertIntoHiveTable( child: LogicalPlan, overwrite: Boolean, ifNotExists: Boolean) - extends LogicalPlan { + extends LogicalPlan with Command { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty From 35c0a60a65091f8bedb34da9fce90b8f8be193cd Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Wed, 15 Jun 2016 21:42:05 -0700 Subject: [PATCH 0673/1470] [SPARK-12922][SPARKR][WIP] Implement gapply() on DataFrame in SparkR ## What changes were proposed in this pull request? gapply() applies an R function on groups grouped by one or more columns of a DataFrame, and returns a DataFrame. It is like GroupedDataSet.flatMapGroups() in the Dataset API. Please, let me know what do you think and if you have any ideas to improve it. Thank you! ## How was this patch tested? Unit tests. 1. Primitive test with different column types 2. Add a boolean column 3. Compute average by a group Author: Narine Kokhlikyan Author: NarineK Closes #12836 from NarineK/gapply2. (cherry picked from commit 7c6c6926376c93acc42dd56a399d816f4838f28c) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 82 ++++++++++- R/pkg/R/deserialize.R | 30 ++++ R/pkg/R/generics.R | 4 + R/pkg/R/group.R | 62 ++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 65 +++++++++ R/pkg/inst/worker/worker.R | 138 ++++++++++++------ .../org/apache/spark/api/r/RRunner.scala | 20 ++- .../sql/catalyst/plans/logical/object.scala | 49 +++++++ .../spark/sql/RelationalGroupedDataset.scala | 48 +++++- .../org/apache/spark/sql/api/r/SQLUtils.scala | 26 +++- .../spark/sql/execution/SparkStrategies.scala | 3 + .../apache/spark/sql/execution/objects.scala | 72 ++++++++- .../execution/r/MapPartitionsRWrapper.scala | 5 +- 14 files changed, 540 insertions(+), 65 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index a8cf53fd46c2e..8db4d5ca1ee53 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -62,6 +62,7 @@ exportMethods("arrange", "filter", "first", "freqItems", + "gapply", "group_by", "groupBy", "head", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 0ff350d44d4b3..9a9b3f7ecae16 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1181,7 +1181,7 @@ dapplyInternal <- function(x, func, schema) { #' func should have only one parameter, to which a data.frame corresponds #' to each partition will be passed. #' The output of func should be a data.frame. -#' @param schema The schema of the resulting DataFrame after the function is applied. +#' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' It must match the output of func. #' @family SparkDataFrame functions #' @rdname dapply @@ -1267,6 +1267,86 @@ setMethod("dapplyCollect", ldf }) +#' gapply +#' +#' Group the SparkDataFrame using the specified columns and apply the R function to each +#' group. +#' +#' @param x A SparkDataFrame +#' @param cols Grouping columns +#' @param func A function to be applied to each group partition specified by grouping +#' column of the SparkDataFrame. The function `func` takes as argument +#' a key - grouping columns and a data frame - a local R data.frame. +#' The output of `func` is a local R data.frame. +#' @param schema The schema of the resulting SparkDataFrame after the function is applied. +#' The schema must match to output of `func`. It has to be defined for each +#' output column with preferred output column name and corresponding data type. +#' @family SparkDataFrame functions +#' @rdname gapply +#' @name gapply +#' @export +#' @examples +#' +#' \dontrun{ +#' Computes the arithmetic mean of the second column by grouping +#' on the first and third columns. Output the grouping values and the average. +#' +#' df <- createDataFrame ( +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), +#' c("a", "b", "c", "d")) +#' +#' Here our output contains three columns, the key which is a combination of two +#' columns with data types integer and string and the mean which is a double. +#' schema <- structType(structField("a", "integer"), structField("c", "string"), +#' structField("avg", "double")) +#' df1 <- gapply( +#' df, +#' list("a", "c"), +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' }, +#' schema) +#' collect(df1) +#' +#' Result +#' ------ +#' a c avg +#' 3 3 3.0 +#' 1 1 1.5 +#' +#' Fits linear models on iris dataset by grouping on the 'Species' column and +#' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' +#' and 'Petal_Width' as training features. +#' +#' df <- createDataFrame (iris) +#' schema <- structType(structField("(Intercept)", "double"), +#' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), +#' structField("Petal_Width", "double")) +#' df1 <- gapply( +#' df, +#' list(df$"Species"), +#' function(key, x) { +#' m <- suppressWarnings(lm(Sepal_Length ~ +#' Sepal_Width + Petal_Length + Petal_Width, x)) +#' data.frame(t(coef(m))) +#' }, schema) +#' collect(df1) +#' +#'Result +#'--------- +#' Model (Intercept) Sepal_Width Petal_Length Petal_Width +#' 1 0.699883 0.3303370 0.9455356 -0.1697527 +#' 2 1.895540 0.3868576 0.9083370 -0.6792238 +#' 3 2.351890 0.6548350 0.2375602 0.2521257 +#' +#'} +setMethod("gapply", + signature(x = "SparkDataFrame"), + function(x, cols, func, schema) { + grouped <- do.call("groupBy", c(x, cols)) + gapply(grouped, func, schema) + }) + ############################## 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 # diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index ce071b1a848bb..0e99b171cabeb 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -197,6 +197,36 @@ readMultipleObjects <- function(inputCon) { data # this is a list of named lists now } +readMultipleObjectsWithKeys <- function(inputCon) { + # readMultipleObjectsWithKeys will read multiple continuous objects from + # a DataOutputStream. There is no preceding field telling the count + # of the objects, so the number of objects varies, we try to read + # all objects in a loop until the end of the stream. This function + # is for use by gapply. Each group of rows is followed by the grouping + # key for this group which is then followed by next group. + keys <- list() + data <- list() + subData <- list() + while (TRUE) { + # If reaching the end of the stream, type returned should be "". + type <- readType(inputCon) + if (type == "") { + break + } else if (type == "r") { + type <- readType(inputCon) + # A grouping boundary detected + key <- readTypedObject(inputCon, type) + index <- length(data) + 1L + data[[index]] <- subData + keys[[index]] <- key + subData <- list() + } else { + subData[[length(subData) + 1L]] <- readTypedObject(inputCon, type) + } + } + list(keys = keys, data = data) # this is a list of keys and corresponding data +} + 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 diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 50fc204f998a5..40a96d8991a5a 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -454,6 +454,10 @@ setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) #' @export setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") }) +#' @rdname gapply +#' @export +setGeneric("gapply", function(x, ...) { standardGeneric("gapply") }) + #' @rdname summary #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 08f4a490c883e..b7047769175a3 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -142,3 +142,65 @@ createMethods <- function() { } createMethods() + +#' gapply +#' +#' Applies a R function to each group in the input GroupedData +#' +#' @param x a GroupedData +#' @param func A function to be applied to each group partition specified by GroupedData. +#' The function `func` takes as argument a key - grouping columns and +#' a data frame - a local R data.frame. +#' The output of `func` is a local R data.frame. +#' @param schema The schema of the resulting SparkDataFrame after the function is applied. +#' The schema must match to output of `func`. It has to be defined for each +#' output column with preferred output column name and corresponding data type. +#' @return a SparkDataFrame +#' @rdname gapply +#' @name gapply +#' @examples +#' \dontrun{ +#' Computes the arithmetic mean of the second column by grouping +#' on the first and third columns. Output the grouping values and the average. +#' +#' df <- createDataFrame ( +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), +#' c("a", "b", "c", "d")) +#' +#' Here our output contains three columns, the key which is a combination of two +#' columns with data types integer and string and the mean which is a double. +#' schema <- structType(structField("a", "integer"), structField("c", "string"), +#' structField("avg", "double")) +#' df1 <- gapply( +#' df, +#' list("a", "c"), +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' }, +#' schema) +#' collect(df1) +#' +#' Result +#' ------ +#' a c avg +#' 3 3 3.0 +#' 1 1 1.5 +#' } +setMethod("gapply", + signature(x = "GroupedData"), + function(x, func, schema) { + try(if (is.null(schema)) stop("schema cannot be NULL")) + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "gapply", + x@sgd, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + schema$jobj) + dataFrame(sdf) + }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index d1ca3b726fe0b..c11930ada63ce 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2146,6 +2146,71 @@ test_that("repartition by columns on DataFrame", { expect_equal(nrow(df1), 2) }) +test_that("gapply() on a DataFrame", { + df <- createDataFrame ( + list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), + c("a", "b", "c", "d")) + expected <- collect(df) + df1 <- gapply(df, list("a"), function(key, x) { x }, schema(df)) + actual <- collect(df1) + expect_identical(actual, expected) + + # Computes the sum of second column by grouping on the first and third columns + # and checks if the sum is larger than 2 + schema <- structType(structField("a", "integer"), structField("e", "boolean")) + df2 <- gapply( + df, + list(df$"a", df$"c"), + function(key, x) { + y <- data.frame(key[1], sum(x$b) > 2) + }, + schema) + actual <- collect(df2)$e + expected <- c(TRUE, TRUE) + expect_identical(actual, expected) + + # Computes the arithmetic mean of the second column by grouping + # on the first and third columns. Output the groupping value and the average. + schema <- structType(structField("a", "integer"), structField("c", "string"), + structField("avg", "double")) + df3 <- gapply( + df, + list("a", "c"), + function(key, x) { + y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) + }, + schema) + actual <- collect(df3) + actual <- actual[order(actual$a), ] + rownames(actual) <- NULL + expected <- collect(select(df, "a", "b", "c")) + expected <- data.frame(aggregate(expected$b, by = list(expected$a, expected$c), FUN = mean)) + colnames(expected) <- c("a", "c", "avg") + expected <- expected[order(expected$a), ] + rownames(expected) <- NULL + expect_identical(actual, expected) + + irisDF <- suppressWarnings(createDataFrame (iris)) + schema <- structType(structField("Sepal_Length", "double"), structField("Avg", "double")) + # Groups by `Sepal_Length` and computes the average for `Sepal_Width` + df4 <- gapply( + cols = list("Sepal_Length"), + irisDF, + function(key, x) { + y <- data.frame(key, mean(x$Sepal_Width), stringsAsFactors = FALSE) + }, + schema) + actual <- collect(df4) + actual <- actual[order(actual$Sepal_Length), ] + rownames(actual) <- NULL + agg_local_df <- data.frame(aggregate(iris$Sepal.Width, by = list(iris$Sepal.Length), FUN = mean), + stringsAsFactors = FALSE) + colnames(agg_local_df) <- c("Sepal_Length", "Avg") + expected <- agg_local_df[order(agg_local_df$Sepal_Length), ] + rownames(expected) <- NULL + expect_identical(actual, expected) +}) + test_that("Window functions on a DataFrame", { setHiveContext(sc) df <- createDataFrame(list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 40cda0c5ef9c1..debf0180183a4 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -27,6 +27,54 @@ elapsedSecs <- function() { proc.time()[3] } +compute <- function(mode, partition, serializer, deserializer, key, + colNames, computeFunc, inputData) { + if (mode > 0) { + if (deserializer == "row") { + # Transform the list of rows into a data.frame + # Note that the optional argument stringsAsFactors for rbind is + # available since R 3.2.4. So we set the global option here. + oldOpt <- getOption("stringsAsFactors") + options(stringsAsFactors = FALSE) + inputData <- do.call(rbind.data.frame, inputData) + options(stringsAsFactors = oldOpt) + + names(inputData) <- colNames + } else { + # Check to see if inputData is a valid data.frame + stopifnot(deserializer == "byte") + stopifnot(class(inputData) == "data.frame") + } + + if (mode == 2) { + output <- computeFunc(key, inputData) + } else { + output <- computeFunc(inputData) + } + if (serializer == "row") { + # Transform the result data.frame back to a list of rows + output <- split(output, seq(nrow(output))) + } else { + # Serialize the ouput to a byte array + stopifnot(serializer == "byte") + } + } else { + output <- computeFunc(partition, inputData) + } + return (output) +} + +outputResult <- function(serializer, output, outputCon) { + if (serializer == "byte") { + SparkR:::writeRawSerialize(outputCon, output) + } else if (serializer == "row") { + SparkR:::writeRowSerialize(outputCon, output) + } else { + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) + } +} + # Constants specialLengths <- list(END_OF_STERAM = 0L, TIMING_DATA = -1L) @@ -79,75 +127,71 @@ if (numBroadcastVars > 0) { # Timing broadcast broadcastElap <- elapsedSecs() +# Initial input timing +inputElap <- broadcastElap # 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) -isDataFrame <- as.logical(SparkR:::readInt(inputCon)) +# 0 - RDD mode, 1 - dapply mode, 2 - gapply mode +mode <- SparkR:::readInt(inputCon) -# If isDataFrame, then read column names -if (isDataFrame) { +if (mode > 0) { colNames <- SparkR:::readObject(inputCon) } isEmpty <- SparkR:::readInt(inputCon) +computeInputElapsDiff <- 0 +outputComputeElapsDiff <- 0 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" && mode == 2) { + dataWithKeys <- SparkR:::readMultipleObjectsWithKeys(inputCon) + keys <- dataWithKeys$keys + data <- dataWithKeys$data } else if (deserializer == "row") { data <- SparkR:::readMultipleObjects(inputCon) } + # Timing reading input data for execution inputElap <- elapsedSecs() - - if (isDataFrame) { - if (deserializer == "row") { - # Transform the list of rows into a data.frame - # Note that the optional argument stringsAsFactors for rbind is - # available since R 3.2.4. So we set the global option here. - oldOpt <- getOption("stringsAsFactors") - options(stringsAsFactors = FALSE) - data <- do.call(rbind.data.frame, data) - options(stringsAsFactors = oldOpt) - - names(data) <- colNames - } else { - # Check to see if data is a valid data.frame - stopifnot(deserializer == "byte") - stopifnot(class(data) == "data.frame") - } - output <- computeFunc(data) - if (serializer == "row") { - # Transform the result data.frame back to a list of rows - output <- split(output, seq(nrow(output))) - } else { - # Serialize the ouput to a byte array - stopifnot(serializer == "byte") + if (mode > 0) { + if (mode == 1) { + output <- compute(mode, partition, serializer, deserializer, NULL, + colNames, computeFunc, data) + } else { + # gapply mode + for (i in 1:length(data)) { + # Timing reading input data for execution + inputElap <- elapsedSecs() + output <- compute(mode, partition, serializer, deserializer, keys[[i]], + colNames, computeFunc, data[[i]]) + computeElap <- elapsedSecs() + outputResult(serializer, output, outputCon) + outputElap <- elapsedSecs() + computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) + outputComputeElapsDiff <- outputComputeElapsDiff + (outputElap - computeElap) + } } } else { - output <- computeFunc(partition, data) + output <- compute(mode, partition, serializer, deserializer, NULL, + colNames, computeFunc, data) } - - # Timing computing - computeElap <- elapsedSecs() - - if (serializer == "byte") { - SparkR:::writeRawSerialize(outputCon, output) - } else if (serializer == "row") { - SparkR:::writeRowSerialize(outputCon, output) - } else { - # write lines one-by-one with flag - lapply(output, function(line) SparkR:::writeString(outputCon, line)) + if (mode != 2) { + # Not a gapply mode + computeElap <- elapsedSecs() + outputResult(serializer, output, outputCon) + outputElap <- elapsedSecs() + computeInputElapsDiff <- computeElap - inputElap + outputComputeElapsDiff <- outputElap - computeElap } - # Timing output - outputElap <- elapsedSecs() } else { if (deserializer == "byte") { # Now read as many characters as described in funcLen @@ -189,11 +233,9 @@ if (isEmpty != 0) { } # Timing output outputElap <- elapsedSecs() + computeInputElapsDiff <- computeElap - inputElap + outputComputeElapsDiff <- outputElap - computeElap } -} else { - inputElap <- broadcastElap - computeElap <- broadcastElap - outputElap <- broadcastElap } # Report timing @@ -202,8 +244,8 @@ SparkR:::writeDouble(outputCon, bootTime) SparkR:::writeDouble(outputCon, initElap - bootElap) # init SparkR:::writeDouble(outputCon, broadcastElap - initElap) # broadcast SparkR:::writeDouble(outputCon, inputElap - broadcastElap) # input -SparkR:::writeDouble(outputCon, computeElap - inputElap) # compute -SparkR:::writeDouble(outputCon, outputElap - computeElap) # output +SparkR:::writeDouble(outputCon, computeInputElapsDiff) # compute +SparkR:::writeDouble(outputCon, outputComputeElapsDiff) # output # End of output SparkR:::writeInt(outputCon, specialLengths$END_OF_STERAM) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 24ad689f8321c..496fdf851f7db 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -40,7 +40,8 @@ private[spark] class RRunner[U]( broadcastVars: Array[Broadcast[Object]], numPartitions: Int = -1, isDataFrame: Boolean = false, - colNames: Array[String] = null) + colNames: Array[String] = null, + mode: Int = RRunnerModes.RDD) extends Logging { private var bootTime: Double = _ private var dataStream: DataInputStream = _ @@ -148,8 +149,7 @@ private[spark] class RRunner[U]( } dataOut.writeInt(numPartitions) - - dataOut.writeInt(if (isDataFrame) 1 else 0) + dataOut.writeInt(mode) if (isDataFrame) { SerDe.writeObject(dataOut, colNames) @@ -180,6 +180,13 @@ private[spark] class RRunner[U]( for (elem <- iter) { elem match { + case (key, innerIter: Iterator[_]) => + for (innerElem <- innerIter) { + writeElem(innerElem) + } + // Writes key which can be used as a boundary in group-aggregate + dataOut.writeByte('r') + writeElem(key) case (key, value) => writeElem(key) writeElem(value) @@ -187,6 +194,7 @@ private[spark] class RRunner[U]( writeElem(elem) } } + stream.flush() } catch { // TODO: We should propagate this error to the task thread @@ -268,6 +276,12 @@ private object SpecialLengths { val TIMING_DATA = -1 } +private[spark] object RRunnerModes { + val RDD = 0 + val DATAFRAME_DAPPLY = 1 + val DATAFRAME_GAPPLY = 2 +} + private[r] class BufferedStreamThread( in: InputStream, name: String, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 78e8822b6405a..7beeeb4f04bf0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -246,6 +246,55 @@ case class MapGroups( outputObjAttr: Attribute, child: LogicalPlan) extends UnaryNode with ObjectProducer +/** Factory for constructing new `FlatMapGroupsInR` nodes. */ +object FlatMapGroupsInR { + def apply( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + schema: StructType, + keyDeserializer: Expression, + valueDeserializer: Expression, + inputSchema: StructType, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + child: LogicalPlan): LogicalPlan = { + val mapped = FlatMapGroupsInR( + func, + packageNames, + broadcastVars, + inputSchema, + schema, + UnresolvedDeserializer(keyDeserializer, groupingAttributes), + UnresolvedDeserializer(valueDeserializer, dataAttributes), + groupingAttributes, + dataAttributes, + CatalystSerde.generateObjAttr(RowEncoder(schema)), + child) + CatalystSerde.serialize(mapped)(RowEncoder(schema)) + } +} + +case class FlatMapGroupsInR( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + outputSchema: StructType, + keyDeserializer: Expression, + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + outputObjAttr: Attribute, + child: LogicalPlan) extends UnaryNode with ObjectProducer{ + + override lazy val schema = outputSchema + + override protected def stringArgs: Iterator[Any] = Iterator(inputSchema, outputSchema, + keyDeserializer, valueDeserializer, groupingAttributes, dataAttributes, outputObjAttr, + child) +} + /** Factory for constructing new `CoGroup` nodes. */ object CoGroup { def apply[K : Encoder, L : Encoder, R : Encoder, OUT : Encoder]( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 49b6eab8db5b0..1aa5767038d53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -20,14 +20,18 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.language.implicitConversions +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Pivot} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, FlatMapGroupsInR, Pivot} import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.NumericType +import org.apache.spark.sql.types.StructType /** * A set of methods for aggregations on a [[DataFrame]], created by [[Dataset.groupBy]]. @@ -381,6 +385,48 @@ class RelationalGroupedDataset protected[sql]( def pivot(pivotColumn: String, values: java.util.List[Any]): RelationalGroupedDataset = { pivot(pivotColumn, values.asScala) } + + /** + * Applies the given serialized R function `func` to each group of data. For each unique group, + * the function will be passed the group key and an iterator that contains all of the elements in + * the group. The function can return an iterator containing elements of an arbitrary type which + * will be returned as a new [[DataFrame]]. + * + * This function does not support partial aggregation, and as a result requires shuffling all + * the data in the [[Dataset]]. If an application intends to perform an aggregation over each + * key, it is best to use the reduce function or an + * [[org.apache.spark.sql.expressions#Aggregator Aggregator]]. + * + * Internally, the implementation will spill to disk if any given group is too large to fit into + * memory. However, users must take care to avoid materializing the whole iterator for a group + * (for example, by calling `toList`) unless they are sure that this is possible given the memory + * constraints of their cluster. + * + * @since 2.0.0 + */ + private[sql] def flatMapGroupsInR( + f: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + outputSchema: StructType): DataFrame = { + val groupingNamedExpressions = groupingExprs.map(alias) + val groupingCols = groupingNamedExpressions.map(Column(_)) + val groupingDataFrame = df.select(groupingCols : _*) + val groupingAttributes = groupingNamedExpressions.map(_.toAttribute) + Dataset.ofRows( + df.sparkSession, + FlatMapGroupsInR( + f, + packageNames, + broadcastVars, + outputSchema, + groupingDataFrame.exprEnc.deserializer, + df.exprEnc.deserializer, + df.exprEnc.schema, + groupingAttributes, + df.logicalPlan.output, + df.logicalPlan)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 486a440b6f9a8..fe426fa3c7e8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -26,7 +26,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext} +import org.apache.spark.sql.{DataFrame, RelationalGroupedDataset, Row, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types._ @@ -146,16 +146,26 @@ private[sql] object SQLUtils { packageNames: Array[Byte], broadcastVars: Array[Object], schema: StructType): DataFrame = { - val bv = broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]]) - val realSchema = - if (schema == null) { - SERIALIZED_R_DATA_SCHEMA - } else { - schema - } + val bv = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) + val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema df.mapPartitionsInR(func, packageNames, bv, realSchema) } + /** + * The helper function for gapply() on R side. + */ + def gapply( + gd: RelationalGroupedDataset, + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Object], + schema: StructType): DataFrame = { + val bv = broadcastVars.map(_.asInstanceOf[Broadcast[Object]]) + val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema + gd.flatMapGroupsInR(func, packageNames, bv, realSchema) + } + + def dfToCols(df: DataFrame): Array[Array[Any]] = { val localDF: Array[Row] = df.collect() val numCols = df.columns.length 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 60466e28307f7..8e2f2ed4f86b9 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 @@ -337,6 +337,9 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.MapPartitionsInR(f, p, b, is, os, objAttr, child) => execution.MapPartitionsExec( execution.r.MapPartitionsRWrapper(f, p, b, is, os), objAttr, planLater(child)) :: Nil + case logical.FlatMapGroupsInR(f, p, b, is, os, key, value, grouping, data, objAttr, child) => + execution.FlatMapGroupsInRExec(f, p, b, is, os, key, value, grouping, + data, objAttr, planLater(child)) :: Nil case logical.MapElements(f, objAttr, child) => execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, in, out, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 5fced940b38d1..c7e267152b5cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -20,13 +20,17 @@ package org.apache.spark.sql.execution import scala.language.existentials import org.apache.spark.api.java.function.MapFunction +import org.apache.spark.api.r._ +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.types.{DataType, ObjectType} +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataType, ObjectType, StructType} /** @@ -324,6 +328,72 @@ case class MapGroupsExec( } } +/** + * Groups the input rows together and calls the R function with each group and an iterator + * containing all elements in the group. + * The result of this function is flattened before being output. + */ +case class FlatMapGroupsInRExec( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + outputSchema: StructType, + keyDeserializer: Expression, + valueDeserializer: Expression, + groupingAttributes: Seq[Attribute], + dataAttributes: Seq[Attribute], + outputObjAttr: Attribute, + child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { + + override def output: Seq[Attribute] = outputObjAttr :: Nil + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(groupingAttributes) :: Nil + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + Seq(groupingAttributes.map(SortOrder(_, Ascending))) + + override protected def doExecute(): RDD[InternalRow] = { + val isSerializedRData = + if (outputSchema == SERIALIZED_R_DATA_SCHEMA) true else false + val serializerForR = if (!isSerializedRData) { + SerializationFormats.ROW + } else { + SerializationFormats.BYTE + } + + child.execute().mapPartitionsInternal { iter => + val grouped = GroupedIterator(iter, groupingAttributes, child.output) + val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) + val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val runner = new RRunner[Array[Byte]]( + func, SerializationFormats.ROW, serializerForR, packageNames, broadcastVars, + isDataFrame = true, colNames = inputSchema.fieldNames, + mode = RRunnerModes.DATAFRAME_GAPPLY) + + val groupedRBytes = grouped.map { case (key, rowIter) => + val deserializedIter = rowIter.map(getValue) + val newIter = + deserializedIter.asInstanceOf[Iterator[Row]].map { row => rowToRBytes(row) } + val newKey = rowToRBytes(getKey(key).asInstanceOf[Row]) + (newKey, newIter) + } + + val outputIter = runner.compute(groupedRBytes, -1) + if (!isSerializedRData) { + val result = outputIter.map { bytes => bytesToRow(bytes, outputSchema) } + result.map(outputObject) + } else { + val result = outputIter.map { bytes => Row.fromSeq(Seq(bytes)) } + result.map(outputObject) + } + } + } +} + /** * Co-groups the data from left and right children, and calls the function with each group and 2 * iterators containing all elements in the group from left and right side. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index 6c76328c74830..70539da348b0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.execution.r -import org.apache.spark.api.r.RRunner -import org.apache.spark.api.r.SerializationFormats +import org.apache.spark.api.r._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.Row @@ -55,7 +54,7 @@ private[sql] case class MapPartitionsRWrapper( val runner = new RRunner[Array[Byte]]( func, deserializer, serializer, packageNames, broadcastVars, - isDataFrame = true, colNames = colNames) + isDataFrame = true, colNames = colNames, mode = RRunnerModes.DATAFRAME_DAPPLY) // Partition index is ignored. Dataset has no support for mapPartitionsWithIndex. val outputIter = runner.compute(newIter, -1) From 52cb1ad38f669dca3f276f38a3f75d57d973e982 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 Jun 2016 00:21:08 -0700 Subject: [PATCH 0674/1470] [SPARK-15979][SQL] Rename various Parquet support classes (branch-2.0). ## What changes were proposed in this pull request? This patch renames various Parquet support classes from CatalystAbc to ParquetAbc. This new naming makes more sense for two reasons: 1. These are not optimizer related (i.e. Catalyst) classes. 2. We are in the Spark code base, and as a result it'd be more clear to call out these are Parquet support classes, rather than some Spark classes. ## How was this patch tested? Renamed test cases as well. Author: Reynold Xin Closes #13700 from rxin/parquet-rename-branch-2.0. --- .../ml/source/libsvm/LibSVMRelation.scala | 6 +- .../SpecificParquetRecordReaderBase.java | 4 +- .../parquet/VectorizedColumnReader.java | 12 +-- .../parquet/ParquetFileFormat.scala | 44 +++++------ ...Support.scala => ParquetReadSupport.scala} | 16 ++-- ....scala => ParquetRecordMaterializer.scala} | 4 +- ...verter.scala => ParquetRowConverter.scala} | 78 +++++++++---------- ...ter.scala => ParquetSchemaConverter.scala} | 28 +++---- ...upport.scala => ParquetWriteSupport.scala} | 14 ++-- .../datasources/parquet/ParquetIOSuite.scala | 4 +- .../parquet/ParquetQuerySuite.scala | 6 +- .../parquet/ParquetSchemaSuite.scala | 6 +- .../datasources/parquet/ParquetTest.scala | 4 +- 13 files changed, 111 insertions(+), 115 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{CatalystReadSupport.scala => ParquetReadSupport.scala} (95%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{CatalystRecordMaterializer.scala => ParquetRecordMaterializer.scala} (90%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{CatalystRowConverter.scala => ParquetRowConverter.scala} (90%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{CatalystSchemaConverter.scala => ParquetSchemaConverter.scala} (96%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{CatalystWriteSupport.scala => ParquetWriteSupport.scala} (96%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index b5b2a681e9a92..62e09d2e0cad9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -141,7 +141,7 @@ class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { sparkSession: SparkSession, options: Map[String, String], files: Seq[FileStatus]): Map[String, String] = { - def computeNumFeatures(): Int = { + val numFeatures = options.get("numFeatures").filter(_.toInt > 0).getOrElse { val dataFiles = files.filterNot(_.getPath.getName startsWith "_") val path = if (dataFiles.length == 1) { dataFiles.head.getPath.toUri.toString @@ -156,10 +156,6 @@ class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { MLUtils.computeNumFeatures(parsed) } - val numFeatures = options.get("numFeatures").filter(_.toInt > 0).getOrElse { - computeNumFeatures() - } - new CaseInsensitiveMap(options + ("numFeatures" -> numFeatures.toString)) } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index cbe8f78164ae7..1a25679d19f90 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -136,7 +136,7 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont ReadSupport.ReadContext readContext = readSupport.init(new InitContext( taskAttemptContext.getConfiguration(), toSetMultiMap(fileMetadata), fileSchema)); this.requestedSchema = readContext.getRequestedSchema(); - this.sparkSchema = new CatalystSchemaConverter(configuration).convert(requestedSchema); + this.sparkSchema = new ParquetSchemaConverter(configuration).convert(requestedSchema); this.reader = new ParquetFileReader(configuration, file, blocks, requestedSchema.getColumns()); for (BlockMetaData block : blocks) { this.totalRowCount += block.getRowCount(); @@ -196,7 +196,7 @@ protected void initialize(String path, List columns) throws IOException } this.requestedSchema = builder.named("spark_schema"); } - this.sparkSchema = new CatalystSchemaConverter(config).convert(requestedSchema); + this.sparkSchema = new ParquetSchemaConverter(config).convert(requestedSchema); this.reader = new ParquetFileReader(config, file, blocks, requestedSchema.getColumns()); for (BlockMetaData block : blocks) { this.totalRowCount += block.getRowCount(); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index ea37a08ab5f55..662a03d3b56ae 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -259,7 +259,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, for (int i = rowId; i < rowId + num; ++i) { // TODO: Convert dictionary of Binaries to dictionary of Longs Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putLong(i, CatalystRowConverter.binaryToSQLTimestamp(v)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); } } else { throw new NotImplementedException(); @@ -280,12 +280,12 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, if (DecimalType.is32BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putInt(i, (int) CatalystRowConverter.binaryToUnscaledLong(v)); + column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); } } else if (DecimalType.is64BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putLong(i, CatalystRowConverter.binaryToUnscaledLong(v)); + column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); } } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { @@ -375,7 +375,7 @@ private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOE if (defColumn.readInteger() == maxDefLevel) { column.putLong(rowId + i, // Read 12 bytes for INT96 - CatalystRowConverter.binaryToSQLTimestamp(data.readBinary(12))); + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12))); } else { column.putNull(rowId + i); } @@ -394,7 +394,7 @@ private void readFixedLenByteArrayBatch(int rowId, int num, for (int i = 0; i < num; i++) { if (defColumn.readInteger() == maxDefLevel) { column.putInt(rowId + i, - (int) CatalystRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); + (int) ParquetRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); } else { column.putNull(rowId + i); } @@ -403,7 +403,7 @@ private void readFixedLenByteArrayBatch(int rowId, int num, for (int i = 0; i < num; i++) { if (defColumn.readInteger() == maxDefLevel) { column.putLong(rowId + i, - CatalystRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); + ParquetRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); } else { column.putNull(rowId + i); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 6b25e36f7b9a3..f38bf81e52c0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -99,13 +99,13 @@ private[sql] class ParquetFileFormat // bundled with `ParquetOutputFormat[Row]`. job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) - ParquetOutputFormat.setWriteSupportClass(job, classOf[CatalystWriteSupport]) + ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) // We want to clear this temporary metadata from saving into Parquet file. // This metadata is only useful for detecting optional columns when pushdowning filters. val dataSchemaToWrite = StructType.removeMetadata(StructType.metadataKeyForOptionalField, dataSchema).asInstanceOf[StructType] - CatalystWriteSupport.setSchema(dataSchemaToWrite, conf) + ParquetWriteSupport.setSchema(dataSchemaToWrite, conf) // Sets flags for `CatalystSchemaConverter` (which converts Catalyst schema to Parquet schema) // and `CatalystWriteSupport` (writing actual rows to Parquet files). @@ -288,19 +288,19 @@ private[sql] class ParquetFileFormat filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[CatalystReadSupport].getName) + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set( - CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - CatalystSchemaConverter.checkFieldNames(requiredSchema).json) + ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, + ParquetSchemaConverter.checkFieldNames(requiredSchema).json) hadoopConf.set( - CatalystWriteSupport.SPARK_ROW_SCHEMA, - CatalystSchemaConverter.checkFieldNames(requiredSchema).json) + ParquetWriteSupport.SPARK_ROW_SCHEMA, + ParquetSchemaConverter.checkFieldNames(requiredSchema).json) // We want to clear this temporary metadata from saving into Parquet file. // This metadata is only useful for detecting optional columns when pushdowning filters. val dataSchemaToWrite = StructType.removeMetadata(StructType.metadataKeyForOptionalField, requiredSchema).asInstanceOf[StructType] - CatalystWriteSupport.setSchema(dataSchemaToWrite, hadoopConf) + ParquetWriteSupport.setSchema(dataSchemaToWrite, hadoopConf) // Sets flags for `CatalystSchemaConverter` hadoopConf.setBoolean( @@ -369,10 +369,10 @@ private[sql] class ParquetFileFormat val reader = pushed match { case Some(filter) => new ParquetRecordReader[InternalRow]( - new CatalystReadSupport, + new ParquetReadSupport, FilterCompat.get(filter, null)) case _ => - new ParquetRecordReader[InternalRow](new CatalystReadSupport) + new ParquetRecordReader[InternalRow](new ParquetReadSupport) } reader.initialize(split, hadoopAttemptContext) reader @@ -433,14 +433,14 @@ private[sql] class ParquetOutputWriterFactory( // bundled with `ParquetOutputFormat[Row]`. job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) - ParquetOutputFormat.setWriteSupportClass(job, classOf[CatalystWriteSupport]) + ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) // We want to clear this temporary metadata from saving into Parquet file. // This metadata is only useful for detecting optional columns when pushdowning filters. val dataSchemaToWrite = StructType.removeMetadata( StructType.metadataKeyForOptionalField, dataSchema).asInstanceOf[StructType] - CatalystWriteSupport.setSchema(dataSchemaToWrite, conf) + ParquetWriteSupport.setSchema(dataSchemaToWrite, conf) // Sets flags for `CatalystSchemaConverter` (which converts Catalyst schema to Parquet schema) // and `CatalystWriteSupport` (writing actual rows to Parquet files). @@ -590,7 +590,7 @@ private[sql] object ParquetFileFormat extends Logging { assumeBinaryIsString: Boolean, assumeInt96IsTimestamp: Boolean)(job: Job): Unit = { val conf = job.getConfiguration - conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[CatalystReadSupport].getName) + conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) // Try to push down filters when filter push-down is enabled. if (parquetFilterPushDown) { @@ -603,14 +603,14 @@ private[sql] object ParquetFileFormat extends Logging { .foreach(ParquetInputFormat.setFilterPredicate(conf, _)) } - conf.set(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { + conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { val requestedSchema = StructType(requiredColumns.map(dataSchema(_))) - CatalystSchemaConverter.checkFieldNames(requestedSchema).json + ParquetSchemaConverter.checkFieldNames(requestedSchema).json }) conf.set( - CatalystWriteSupport.SPARK_ROW_SCHEMA, - CatalystSchemaConverter.checkFieldNames(dataSchema).json) + ParquetWriteSupport.SPARK_ROW_SCHEMA, + ParquetSchemaConverter.checkFieldNames(dataSchema).json) // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata conf.setBoolean(SQLConf.PARQUET_CACHE_METADATA.key, useMetadataCache) @@ -639,7 +639,7 @@ private[sql] object ParquetFileFormat extends Logging { footers: Seq[Footer], sparkSession: SparkSession): Option[StructType] = { def parseParquetSchema(schema: MessageType): StructType = { - val converter = new CatalystSchemaConverter( + val converter = new ParquetSchemaConverter( sparkSession.sessionState.conf.isParquetBinaryAsString, sparkSession.sessionState.conf.isParquetBinaryAsString, sparkSession.sessionState.conf.writeLegacyParquetFormat) @@ -653,7 +653,7 @@ private[sql] object ParquetFileFormat extends Logging { val serializedSchema = metadata .getKeyValueMetaData .asScala.toMap - .get(CatalystReadSupport.SPARK_METADATA_KEY) + .get(ParquetReadSupport.SPARK_METADATA_KEY) if (serializedSchema.isEmpty) { // Falls back to Parquet schema if no Spark SQL schema found. Some(parseParquetSchema(metadata.getSchema)) @@ -820,7 +820,7 @@ private[sql] object ParquetFileFormat extends Logging { // Converter used to convert Parquet `MessageType` to Spark SQL `StructType` val converter = - new CatalystSchemaConverter( + new ParquetSchemaConverter( assumeBinaryIsString = assumeBinaryIsString, assumeInt96IsTimestamp = assumeInt96IsTimestamp, writeLegacyParquetFormat = writeLegacyParquetFormat) @@ -864,12 +864,12 @@ private[sql] object ParquetFileFormat extends Logging { * a [[StructType]] converted from the [[MessageType]] stored in this footer. */ def readSchemaFromFooter( - footer: Footer, converter: CatalystSchemaConverter): StructType = { + footer: Footer, converter: ParquetSchemaConverter): StructType = { val fileMetaData = footer.getParquetMetadata.getFileMetaData fileMetaData .getKeyValueMetaData .asScala.toMap - .get(CatalystReadSupport.SPARK_METADATA_KEY) + .get(ParquetReadSupport.SPARK_METADATA_KEY) .flatMap(deserializeSchemaString) .getOrElse(converter.convert(fileMetaData.getSchema)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 850e807b8677e..12f497421f4b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -48,7 +48,7 @@ import org.apache.spark.sql.types._ * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]] * to [[prepareForRead()]], but use a private `var` for simplicity. */ -private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with Logging { +private[parquet] class ParquetReadSupport extends ReadSupport[InternalRow] with Logging { private var catalystRequestedSchema: StructType = _ /** @@ -58,13 +58,13 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with override def init(context: InitContext): ReadContext = { catalystRequestedSchema = { val conf = context.getConfiguration - val schemaString = conf.get(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA) + val schemaString = conf.get(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA) assert(schemaString != null, "Parquet requested schema not set.") StructType.fromString(schemaString) } val parquetRequestedSchema = - CatalystReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema) + ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema) new ReadContext(parquetRequestedSchema, Map.empty[String, String].asJava) } @@ -92,13 +92,13 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with """.stripMargin } - new CatalystRecordMaterializer( + new ParquetRecordMaterializer( parquetRequestedSchema, - CatalystReadSupport.expandUDT(catalystRequestedSchema)) + ParquetReadSupport.expandUDT(catalystRequestedSchema)) } } -private[parquet] object CatalystReadSupport { +private[parquet] object ParquetReadSupport { val SPARK_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema" val SPARK_METADATA_KEY = "org.apache.spark.sql.parquet.row.metadata" @@ -112,7 +112,7 @@ private[parquet] object CatalystReadSupport { Types .buildMessage() .addFields(clippedParquetFields: _*) - .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + .named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) } private def clipParquetType(parquetType: Type, catalystType: DataType): Type = { @@ -265,7 +265,7 @@ private[parquet] object CatalystReadSupport { private def clipParquetGroupFields( parquetRecord: GroupType, structType: StructType): Seq[Type] = { val parquetFieldMap = parquetRecord.getFields.asScala.map(f => f.getName -> f).toMap - val toParquet = new CatalystSchemaConverter(writeLegacyParquetFormat = false) + val toParquet = new ParquetSchemaConverter(writeLegacyParquetFormat = false) structType.map { f => parquetFieldMap .get(f.name) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRecordMaterializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala similarity index 90% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRecordMaterializer.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala index eeead9f5d88a2..0818d802b077a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRecordMaterializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala @@ -29,11 +29,11 @@ import org.apache.spark.sql.types.StructType * @param parquetSchema Parquet schema of the records to be read * @param catalystSchema Catalyst schema of the rows to be constructed */ -private[parquet] class CatalystRecordMaterializer( +private[parquet] class ParquetRecordMaterializer( parquetSchema: MessageType, catalystSchema: StructType) extends RecordMaterializer[InternalRow] { - private val rootConverter = new CatalystRowConverter(parquetSchema, catalystSchema, NoopUpdater) + private val rootConverter = new ParquetRowConverter(parquetSchema, catalystSchema, NoopUpdater) override def getCurrentRecord: InternalRow = rootConverter.currentRecord diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala similarity index 90% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 85b0bc17edd32..9dad59647e0db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -70,7 +70,7 @@ private[parquet] trait HasParentContainerUpdater { /** * A convenient converter class for Parquet group types with a [[HasParentContainerUpdater]]. */ -private[parquet] abstract class CatalystGroupConverter(val updater: ParentContainerUpdater) +private[parquet] abstract class ParquetGroupConverter(val updater: ParentContainerUpdater) extends GroupConverter with HasParentContainerUpdater /** @@ -78,7 +78,7 @@ private[parquet] abstract class CatalystGroupConverter(val updater: ParentContai * are handled by this converter. Parquet primitive types are only a subset of those of Spark * SQL. For example, BYTE, SHORT, and INT in Spark SQL are all covered by INT32 in Parquet. */ -private[parquet] class CatalystPrimitiveConverter(val updater: ParentContainerUpdater) +private[parquet] class ParquetPrimitiveConverter(val updater: ParentContainerUpdater) extends PrimitiveConverter with HasParentContainerUpdater { override def addBoolean(value: Boolean): Unit = updater.setBoolean(value) @@ -90,7 +90,7 @@ private[parquet] class CatalystPrimitiveConverter(val updater: ParentContainerUp } /** - * A [[CatalystRowConverter]] is used to convert Parquet records into Catalyst [[InternalRow]]s. + * A [[ParquetRowConverter]] is used to convert Parquet records into Catalyst [[InternalRow]]s. * Since Catalyst `StructType` is also a Parquet record, this converter can be used as root * converter. Take the following Parquet type as an example: * {{{ @@ -104,11 +104,11 @@ private[parquet] class CatalystPrimitiveConverter(val updater: ParentContainerUp * }}} * 5 converters will be created: * - * - a root [[CatalystRowConverter]] for [[MessageType]] `root`, which contains: - * - a [[CatalystPrimitiveConverter]] for required [[INT_32]] field `f1`, and - * - a nested [[CatalystRowConverter]] for optional [[GroupType]] `f2`, which contains: - * - a [[CatalystPrimitiveConverter]] for required [[DOUBLE]] field `f21`, and - * - a [[CatalystStringConverter]] for optional [[UTF8]] string field `f22` + * - a root [[ParquetRowConverter]] for [[MessageType]] `root`, which contains: + * - a [[ParquetPrimitiveConverter]] for required [[INT_32]] field `f1`, and + * - a nested [[ParquetRowConverter]] for optional [[GroupType]] `f2`, which contains: + * - a [[ParquetPrimitiveConverter]] for required [[DOUBLE]] field `f21`, and + * - a [[ParquetStringConverter]] for optional [[UTF8]] string field `f22` * * When used as a root converter, [[NoopUpdater]] should be used since root converters don't have * any "parent" container. @@ -118,11 +118,11 @@ private[parquet] class CatalystPrimitiveConverter(val updater: ParentContainerUp * types should have been expanded. * @param updater An updater which propagates converted field values to the parent container */ -private[parquet] class CatalystRowConverter( +private[parquet] class ParquetRowConverter( parquetType: GroupType, catalystType: StructType, updater: ParentContainerUpdater) - extends CatalystGroupConverter(updater) with Logging { + extends ParquetGroupConverter(updater) with Logging { assert( parquetType.getFieldCount == catalystType.length, @@ -150,7 +150,7 @@ private[parquet] class CatalystRowConverter( """.stripMargin) /** - * Updater used together with field converters within a [[CatalystRowConverter]]. It propagates + * Updater used together with field converters within a [[ParquetRowConverter]]. It propagates * converted filed values to the `ordinal`-th cell in `currentRow`. */ private final class RowUpdater(row: MutableRow, ordinal: Int) extends ParentContainerUpdater { @@ -213,33 +213,33 @@ private[parquet] class CatalystRowConverter( catalystType match { case BooleanType | IntegerType | LongType | FloatType | DoubleType | BinaryType => - new CatalystPrimitiveConverter(updater) + new ParquetPrimitiveConverter(updater) case ByteType => - new CatalystPrimitiveConverter(updater) { + new ParquetPrimitiveConverter(updater) { override def addInt(value: Int): Unit = updater.setByte(value.asInstanceOf[ByteType#InternalType]) } case ShortType => - new CatalystPrimitiveConverter(updater) { + new ParquetPrimitiveConverter(updater) { override def addInt(value: Int): Unit = updater.setShort(value.asInstanceOf[ShortType#InternalType]) } // For INT32 backed decimals case t: DecimalType if parquetType.asPrimitiveType().getPrimitiveTypeName == INT32 => - new CatalystIntDictionaryAwareDecimalConverter(t.precision, t.scale, updater) + new ParquetIntDictionaryAwareDecimalConverter(t.precision, t.scale, updater) // For INT64 backed decimals case t: DecimalType if parquetType.asPrimitiveType().getPrimitiveTypeName == INT64 => - new CatalystLongDictionaryAwareDecimalConverter(t.precision, t.scale, updater) + new ParquetLongDictionaryAwareDecimalConverter(t.precision, t.scale, updater) // For BINARY and FIXED_LEN_BYTE_ARRAY backed decimals case t: DecimalType if parquetType.asPrimitiveType().getPrimitiveTypeName == FIXED_LEN_BYTE_ARRAY || parquetType.asPrimitiveType().getPrimitiveTypeName == BINARY => - new CatalystBinaryDictionaryAwareDecimalConverter(t.precision, t.scale, updater) + new ParquetBinaryDictionaryAwareDecimalConverter(t.precision, t.scale, updater) case t: DecimalType => throw new RuntimeException( @@ -248,11 +248,11 @@ private[parquet] class CatalystRowConverter( "FIXED_LEN_BYTE_ARRAY, or BINARY.") case StringType => - new CatalystStringConverter(updater) + new ParquetStringConverter(updater) case TimestampType => // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. - new CatalystPrimitiveConverter(updater) { + new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { assert( @@ -268,7 +268,7 @@ private[parquet] class CatalystRowConverter( } case DateType => - new CatalystPrimitiveConverter(updater) { + new ParquetPrimitiveConverter(updater) { override def addInt(value: Int): Unit = { // DateType is not specialized in `SpecificMutableRow`, have to box it here. updater.set(value.asInstanceOf[DateType#InternalType]) @@ -286,13 +286,13 @@ private[parquet] class CatalystRowConverter( } case t: ArrayType => - new CatalystArrayConverter(parquetType.asGroupType(), t, updater) + new ParquetArrayConverter(parquetType.asGroupType(), t, updater) case t: MapType => - new CatalystMapConverter(parquetType.asGroupType(), t, updater) + new ParquetMapConverter(parquetType.asGroupType(), t, updater) case t: StructType => - new CatalystRowConverter(parquetType.asGroupType(), t, new ParentContainerUpdater { + new ParquetRowConverter(parquetType.asGroupType(), t, new ParentContainerUpdater { override def set(value: Any): Unit = updater.set(value.asInstanceOf[InternalRow].copy()) }) @@ -306,8 +306,8 @@ private[parquet] class CatalystRowConverter( /** * Parquet converter for strings. A dictionary is used to minimize string decoding cost. */ - private final class CatalystStringConverter(updater: ParentContainerUpdater) - extends CatalystPrimitiveConverter(updater) { + private final class ParquetStringConverter(updater: ParentContainerUpdater) + extends ParquetPrimitiveConverter(updater) { private var expandedDictionary: Array[UTF8String] = null @@ -337,9 +337,9 @@ private[parquet] class CatalystRowConverter( /** * Parquet converter for fixed-precision decimals. */ - private abstract class CatalystDecimalConverter( + private abstract class ParquetDecimalConverter( precision: Int, scale: Int, updater: ParentContainerUpdater) - extends CatalystPrimitiveConverter(updater) { + extends ParquetPrimitiveConverter(updater) { protected var expandedDictionary: Array[Decimal] = _ @@ -371,7 +371,7 @@ private[parquet] class CatalystRowConverter( protected def decimalFromBinary(value: Binary): Decimal = { if (precision <= Decimal.MAX_LONG_DIGITS) { // Constructs a `Decimal` with an unscaled `Long` value if possible. - val unscaled = CatalystRowConverter.binaryToUnscaledLong(value) + val unscaled = ParquetRowConverter.binaryToUnscaledLong(value) Decimal(unscaled, precision, scale) } else { // Otherwise, resorts to an unscaled `BigInteger` instead. @@ -380,9 +380,9 @@ private[parquet] class CatalystRowConverter( } } - private class CatalystIntDictionaryAwareDecimalConverter( + private class ParquetIntDictionaryAwareDecimalConverter( precision: Int, scale: Int, updater: ParentContainerUpdater) - extends CatalystDecimalConverter(precision, scale, updater) { + extends ParquetDecimalConverter(precision, scale, updater) { override def setDictionary(dictionary: Dictionary): Unit = { this.expandedDictionary = Array.tabulate(dictionary.getMaxId + 1) { id => @@ -391,9 +391,9 @@ private[parquet] class CatalystRowConverter( } } - private class CatalystLongDictionaryAwareDecimalConverter( + private class ParquetLongDictionaryAwareDecimalConverter( precision: Int, scale: Int, updater: ParentContainerUpdater) - extends CatalystDecimalConverter(precision, scale, updater) { + extends ParquetDecimalConverter(precision, scale, updater) { override def setDictionary(dictionary: Dictionary): Unit = { this.expandedDictionary = Array.tabulate(dictionary.getMaxId + 1) { id => @@ -402,9 +402,9 @@ private[parquet] class CatalystRowConverter( } } - private class CatalystBinaryDictionaryAwareDecimalConverter( + private class ParquetBinaryDictionaryAwareDecimalConverter( precision: Int, scale: Int, updater: ParentContainerUpdater) - extends CatalystDecimalConverter(precision, scale, updater) { + extends ParquetDecimalConverter(precision, scale, updater) { override def setDictionary(dictionary: Dictionary): Unit = { this.expandedDictionary = Array.tabulate(dictionary.getMaxId + 1) { id => @@ -431,11 +431,11 @@ private[parquet] class CatalystRowConverter( * * @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists */ - private final class CatalystArrayConverter( + private final class ParquetArrayConverter( parquetSchema: GroupType, catalystSchema: ArrayType, updater: ParentContainerUpdater) - extends CatalystGroupConverter(updater) { + extends ParquetGroupConverter(updater) { private var currentArray: ArrayBuffer[Any] = _ @@ -512,11 +512,11 @@ private[parquet] class CatalystRowConverter( } /** Parquet converter for maps */ - private final class CatalystMapConverter( + private final class ParquetMapConverter( parquetType: GroupType, catalystType: MapType, updater: ParentContainerUpdater) - extends CatalystGroupConverter(updater) { + extends ParquetGroupConverter(updater) { private var currentKeys: ArrayBuffer[Any] = _ private var currentValues: ArrayBuffer[Any] = _ @@ -638,7 +638,7 @@ private[parquet] class CatalystRowConverter( } } -private[parquet] object CatalystRowConverter { +private[parquet] object ParquetRowConverter { def binaryToUnscaledLong(binary: Binary): Long = { // The underlying `ByteBuffer` implementation is guaranteed to be `HeapByteBuffer`, so here // we are using `Binary.toByteBuffer.array()` to steal the underlying byte array without diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 6f6340f541ada..1ac083f48a8c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -26,7 +26,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition._ import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.execution.datasources.parquet.CatalystSchemaConverter.maxPrecisionForBytes +import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter.maxPrecisionForBytes import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -52,7 +52,7 @@ import org.apache.spark.sql.types._ * When set to false, use standard format defined in parquet-format spec. This argument only * affects Parquet write path. */ -private[parquet] class CatalystSchemaConverter( +private[parquet] class ParquetSchemaConverter( assumeBinaryIsString: Boolean = SQLConf.PARQUET_BINARY_AS_STRING.defaultValue.get, assumeInt96IsTimestamp: Boolean = SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get, writeLegacyParquetFormat: Boolean = SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get) { @@ -125,7 +125,7 @@ private[parquet] class CatalystSchemaConverter( val precision = field.getDecimalMetadata.getPrecision val scale = field.getDecimalMetadata.getScale - CatalystSchemaConverter.checkConversionRequirement( + ParquetSchemaConverter.checkConversionRequirement( maxPrecision == -1 || 1 <= precision && precision <= maxPrecision, s"Invalid decimal precision: $typeName cannot store $precision digits (max $maxPrecision)") @@ -163,7 +163,7 @@ private[parquet] class CatalystSchemaConverter( } case INT96 => - CatalystSchemaConverter.checkConversionRequirement( + ParquetSchemaConverter.checkConversionRequirement( assumeInt96IsTimestamp, "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") @@ -206,11 +206,11 @@ private[parquet] class CatalystSchemaConverter( // // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists case LIST => - CatalystSchemaConverter.checkConversionRequirement( + ParquetSchemaConverter.checkConversionRequirement( field.getFieldCount == 1, s"Invalid list type $field") val repeatedType = field.getType(0) - CatalystSchemaConverter.checkConversionRequirement( + ParquetSchemaConverter.checkConversionRequirement( repeatedType.isRepetition(REPEATED), s"Invalid list type $field") if (isElementType(repeatedType, field.getName)) { @@ -226,17 +226,17 @@ private[parquet] class CatalystSchemaConverter( // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules-1 // scalastyle:on case MAP | MAP_KEY_VALUE => - CatalystSchemaConverter.checkConversionRequirement( + ParquetSchemaConverter.checkConversionRequirement( field.getFieldCount == 1 && !field.getType(0).isPrimitive, s"Invalid map type: $field") val keyValueType = field.getType(0).asGroupType() - CatalystSchemaConverter.checkConversionRequirement( + ParquetSchemaConverter.checkConversionRequirement( keyValueType.isRepetition(REPEATED) && keyValueType.getFieldCount == 2, s"Invalid map type: $field") val keyType = keyValueType.getType(0) - CatalystSchemaConverter.checkConversionRequirement( + ParquetSchemaConverter.checkConversionRequirement( keyType.isPrimitive, s"Map key type is expected to be a primitive type, but found: $keyType") @@ -311,7 +311,7 @@ private[parquet] class CatalystSchemaConverter( Types .buildMessage() .addFields(catalystSchema.map(convertField): _*) - .named(CatalystSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) + .named(ParquetSchemaConverter.SPARK_PARQUET_SCHEMA_NAME) } /** @@ -322,7 +322,7 @@ private[parquet] class CatalystSchemaConverter( } private def convertField(field: StructField, repetition: Type.Repetition): Type = { - CatalystSchemaConverter.checkFieldName(field.name) + ParquetSchemaConverter.checkFieldName(field.name) field.dataType match { // =================== @@ -394,7 +394,7 @@ private[parquet] class CatalystSchemaConverter( .as(DECIMAL) .precision(precision) .scale(scale) - .length(CatalystSchemaConverter.minBytesForPrecision(precision)) + .length(ParquetSchemaConverter.minBytesForPrecision(precision)) .named(field.name) // ======================== @@ -428,7 +428,7 @@ private[parquet] class CatalystSchemaConverter( .as(DECIMAL) .precision(precision) .scale(scale) - .length(CatalystSchemaConverter.minBytesForPrecision(precision)) + .length(ParquetSchemaConverter.minBytesForPrecision(precision)) .named(field.name) // =================================== @@ -535,7 +535,7 @@ private[parquet] class CatalystSchemaConverter( } } -private[parquet] object CatalystSchemaConverter { +private[parquet] object ParquetSchemaConverter { val SPARK_PARQUET_SCHEMA_NAME = "spark_schema" def checkFieldName(name: String): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystWriteSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 67bfd39697ed7..307c64d6e895b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -33,7 +33,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.parquet.CatalystSchemaConverter.minBytesForPrecision +import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter.minBytesForPrecision import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -48,7 +48,7 @@ import org.apache.spark.sql.types._ * of this option is propagated to this class by the `init()` method and its Hadoop configuration * argument. */ -private[parquet] class CatalystWriteSupport extends WriteSupport[InternalRow] with Logging { +private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { // A `ValueWriter` is responsible for writing a field of an `InternalRow` to the record consumer. // Here we are using `SpecializedGetters` rather than `InternalRow` so that we can directly access // data in `ArrayData` without the help of `SpecificMutableRow`. @@ -73,7 +73,7 @@ private[parquet] class CatalystWriteSupport extends WriteSupport[InternalRow] wi private val decimalBuffer = new Array[Byte](minBytesForPrecision(DecimalType.MAX_PRECISION)) override def init(configuration: Configuration): WriteContext = { - val schemaString = configuration.get(CatalystWriteSupport.SPARK_ROW_SCHEMA) + val schemaString = configuration.get(ParquetWriteSupport.SPARK_ROW_SCHEMA) this.schema = StructType.fromString(schemaString) this.writeLegacyParquetFormat = { // `SQLConf.PARQUET_WRITE_LEGACY_FORMAT` should always be explicitly set in ParquetRelation @@ -82,8 +82,8 @@ private[parquet] class CatalystWriteSupport extends WriteSupport[InternalRow] wi } this.rootFieldWriters = schema.map(_.dataType).map(makeWriter) - val messageType = new CatalystSchemaConverter(configuration).convert(schema) - val metadata = Map(CatalystReadSupport.SPARK_METADATA_KEY -> schemaString).asJava + val messageType = new ParquetSchemaConverter(configuration).convert(schema) + val metadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> schemaString).asJava logInfo( s"""Initialized Parquet WriteSupport with Catalyst schema: @@ -423,11 +423,11 @@ private[parquet] class CatalystWriteSupport extends WriteSupport[InternalRow] wi } } -private[parquet] object CatalystWriteSupport { +private[parquet] object ParquetWriteSupport { val SPARK_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.attributes" def setSchema(schema: StructType, configuration: Configuration): Unit = { - schema.map(_.name).foreach(CatalystSchemaConverter.checkFieldName) + schema.map(_.name).foreach(ParquetSchemaConverter.checkFieldName) configuration.set(SPARK_ROW_SCHEMA, schema.json) configuration.setIfUnset( ParquetOutputFormat.WRITER_VERSION, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 92f2db325c1aa..fc9ce6bb3041b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -362,7 +362,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE))) assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) - val expectedSchema = new CatalystSchemaConverter().convert(schema) + val expectedSchema = new ParquetSchemaConverter().convert(schema) val actualSchema = readFooter(path, hadoopConf).getFileMetaData.getSchema actualSchema.checkContains(expectedSchema) @@ -432,7 +432,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { """.stripMargin) withTempPath { location => - val extraMetadata = Map(CatalystReadSupport.SPARK_METADATA_KEY -> sparkSchema.toString) + val extraMetadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> sparkSchema.toString) val path = new Path(location.getCanonicalPath) val conf = spark.sessionState.newHadoopConf() writeMetadata(parquetSchema, path, conf, extraMetadata) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index b4fd0ef6eda84..83d10010f9dcb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -574,7 +574,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("expand UDT in StructType") { val schema = new StructType().add("n", new NestedStructUDT, nullable = true) val expected = new StructType().add("n", new NestedStructUDT().sqlType, nullable = true) - assert(CatalystReadSupport.expandUDT(schema) === expected) + assert(ParquetReadSupport.expandUDT(schema) === expected) } test("expand UDT in ArrayType") { @@ -592,7 +592,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext containsNull = false), nullable = true) - assert(CatalystReadSupport.expandUDT(schema) === expected) + assert(ParquetReadSupport.expandUDT(schema) === expected) } test("expand UDT in MapType") { @@ -612,7 +612,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext valueContainsNull = false), nullable = true) - assert(CatalystReadSupport.expandUDT(schema) === expected) + assert(ParquetReadSupport.expandUDT(schema) === expected) } test("returning batch for wide table") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 1bc6f70401bb5..51bb236fe8441 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -54,7 +54,7 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { binaryAsString: Boolean, int96AsTimestamp: Boolean, writeLegacyParquetFormat: Boolean): Unit = { - val converter = new CatalystSchemaConverter( + val converter = new ParquetSchemaConverter( assumeBinaryIsString = binaryAsString, assumeInt96IsTimestamp = int96AsTimestamp, writeLegacyParquetFormat = writeLegacyParquetFormat) @@ -78,7 +78,7 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { binaryAsString: Boolean, int96AsTimestamp: Boolean, writeLegacyParquetFormat: Boolean): Unit = { - val converter = new CatalystSchemaConverter( + val converter = new ParquetSchemaConverter( assumeBinaryIsString = binaryAsString, assumeInt96IsTimestamp = int96AsTimestamp, writeLegacyParquetFormat = writeLegacyParquetFormat) @@ -1054,7 +1054,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { expectedSchema: String): Unit = { test(s"Clipping - $testName") { val expected = MessageTypeParser.parseMessageType(expectedSchema) - val actual = CatalystReadSupport.clipParquetSchema( + val actual = ParquetReadSupport.clipParquetSchema( MessageTypeParser.parseMessageType(parquetSchema), catalystSchema) try { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index 1953d6fa5a7af..9fb34e03cb201 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -124,8 +124,8 @@ private[sql] trait ParquetTest extends SQLTestUtils { protected def writeMetadata( schema: StructType, path: Path, configuration: Configuration): Unit = { - val parquetSchema = new CatalystSchemaConverter().convert(schema) - val extraMetadata = Map(CatalystReadSupport.SPARK_METADATA_KEY -> schema.json).asJava + val parquetSchema = new ParquetSchemaConverter().convert(schema) + val extraMetadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> schema.json).asJava val createdBy = s"Apache Spark ${org.apache.spark.SPARK_VERSION}" val fileMetadata = new FileMetaData(parquetSchema, extraMetadata, createdBy) val parquetMetadata = new ParquetMetadata(fileMetadata, Seq.empty[BlockMetaData].asJava) From 26359d27c47ae3ec53e442de3884ec9245d15cee Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 16 Jun 2016 10:01:59 -0700 Subject: [PATCH 0675/1470] [SPARK-15862][SQL] Better Error Message When Having Database Name in CACHE TABLE AS SELECT #### What changes were proposed in this pull request? ~~If the temp table already exists, we should not silently replace it when doing `CACHE TABLE AS SELECT`. This is inconsistent with the behavior of `CREAT VIEW` or `CREATE TABLE`. This PR is to fix this silent drop.~~ ~~Maybe, we also can introduce new syntax for replacing the existing one. For example, in Hive, to replace a view, the syntax should be like `ALTER VIEW AS SELECT` or `CREATE OR REPLACE VIEW AS SELECT`~~ The table name in `CACHE TABLE AS SELECT` should NOT contain database prefix like "database.table". Thus, this PR captures this in Parser and outputs a better error message, instead of reporting the view already exists. In addition, refactoring the `Parser` to generate table identifiers instead of returning the table name string. #### How was this patch tested? - Added a test case for caching and uncaching qualified table names - Fixed a few test cases that do not drop temp table at the end - Added the related test case for the issue resolved in this PR Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13572 from gatorsmile/cacheTableAsSelect. (cherry picked from commit 6451cf9270b55465d8ecea4c4031329a1058561a) Signed-off-by: Cheng Lian --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 10 ++- .../spark/sql/execution/command/cache.scala | 20 ++--- .../spark/sql/execution/command/views.scala | 2 +- .../apache/spark/sql/CachedTableSuite.scala | 68 +++++++++------- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../spark/sql/hive/CachedTableSuite.scala | 79 ++++++++++++++----- 7 files changed, 121 insertions(+), 64 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 044f91038876a..b60319668c5b6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -114,8 +114,8 @@ statement tableIdentifier partitionSpec? describeColName? #describeTable | REFRESH TABLE tableIdentifier #refreshTable | REFRESH .*? #refreshResource - | CACHE LAZY? TABLE identifier (AS? query)? #cacheTable - | UNCACHE TABLE identifier #uncacheTable + | CACHE LAZY? TABLE tableIdentifier (AS? query)? #cacheTable + | UNCACHE TABLE tableIdentifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index a0508ad6019bd..154c25adfac43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -221,14 +221,20 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) { val query = Option(ctx.query).map(plan) - CacheTableCommand(ctx.identifier.getText, query, ctx.LAZY != null) + val tableIdent = visitTableIdentifier(ctx.tableIdentifier) + if (query.isDefined && tableIdent.database.isDefined) { + val database = tableIdent.database.get + throw new ParseException(s"It is not allowed to add database prefix `$database` to " + + s"the table name in CACHE TABLE AS SELECT", ctx) + } + CacheTableCommand(tableIdent, query, ctx.LAZY != null) } /** * Create an [[UncacheTableCommand]] logical plan. */ override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { - UncacheTableCommand(ctx.identifier.getText) + UncacheTableCommand(visitTableIdentifier(ctx.tableIdentifier)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 5332366d242a8..697e2ff21159b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -18,15 +18,17 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan case class CacheTableCommand( - tableName: String, - plan: Option[LogicalPlan], - isLazy: Boolean) - extends RunnableCommand { + tableIdent: TableIdentifier, + plan: Option[LogicalPlan], + isLazy: Boolean) extends RunnableCommand { + require(plan.isEmpty || tableIdent.database.isEmpty, + "Database name is not allowed in CACHE TABLE AS SELECT") override protected def innerChildren: Seq[QueryPlan[_]] = { plan.toSeq @@ -34,13 +36,13 @@ case class CacheTableCommand( override def run(sparkSession: SparkSession): Seq[Row] = { plan.foreach { logicalPlan => - Dataset.ofRows(sparkSession, logicalPlan).createOrReplaceTempView(tableName) + Dataset.ofRows(sparkSession, logicalPlan).createTempView(tableIdent.quotedString) } - sparkSession.catalog.cacheTable(tableName) + sparkSession.catalog.cacheTable(tableIdent.quotedString) if (!isLazy) { // Performs eager caching - sparkSession.table(tableName).count() + sparkSession.table(tableIdent).count() } Seq.empty[Row] @@ -50,10 +52,10 @@ case class CacheTableCommand( } -case class UncacheTableCommand(tableName: String) extends RunnableCommand { +case class UncacheTableCommand(tableIdent: TableIdentifier) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.catalog.uncacheTable(tableName) + sparkSession.catalog.uncacheTable(tableIdent.quotedString) Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index b56c200e9e98b..088f684365db3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -79,7 +79,7 @@ case class CreateViewCommand( if (isTemporary && tableDesc.identifier.database.isDefined) { val database = tableDesc.identifier.database.get throw new AnalysisException( - s"It is not allowed to add database prefix ${database} for the TEMPORARY view name.") + s"It is not allowed to add database prefix `$database` for the TEMPORARY view name.") } override def run(sparkSession: SparkSession): Seq[Row] = { 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 d7df18ae1c42d..6f6abfa93c1d8 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 @@ -73,11 +73,13 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("cache temp table") { - testData.select('key).createOrReplaceTempView("tempTable") - assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) - spark.catalog.cacheTable("tempTable") - assertCached(sql("SELECT COUNT(*) FROM tempTable")) - spark.catalog.uncacheTable("tempTable") + withTempTable("tempTable") { + testData.select('key).createOrReplaceTempView("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) + spark.catalog.cacheTable("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + spark.catalog.uncacheTable("tempTable") + } } test("unpersist an uncached table will not raise exception") { @@ -95,9 +97,11 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("cache table as select") { - sql("CACHE TABLE tempTable AS SELECT key FROM testData") - assertCached(sql("SELECT COUNT(*) FROM tempTable")) - spark.catalog.uncacheTable("tempTable") + withTempTable("tempTable") { + sql("CACHE TABLE tempTable AS SELECT key FROM testData") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + spark.catalog.uncacheTable("tempTable") + } } test("uncaching temp table") { @@ -223,32 +227,36 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { - sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") - assertCached(spark.table("testCacheTable")) - - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") - - spark.catalog.uncacheTable("testCacheTable") - eventually(timeout(10 seconds)) { - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + withTempTable("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + assertCached(spark.table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + spark.catalog.uncacheTable("testCacheTable") + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } } test("CACHE TABLE tableName AS SELECT ...") { - sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10") - assertCached(spark.table("testCacheTable")) - - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") - - spark.catalog.uncacheTable("testCacheTable") - eventually(timeout(10 seconds)) { - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + withTempTable("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10") + assertCached(spark.table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + spark.catalog.uncacheTable("testCacheTable") + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } } 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 1d1d5e3f7bd60..b45be0251d953 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 @@ -449,7 +449,7 @@ private[hive] class TestHiveQueryExecution( override lazy val analyzed: LogicalPlan = { val describedTables = logical match { - case CacheTableCommand(tbl, _, _) => tbl :: Nil + case CacheTableCommand(tbl, _, _) => tbl.table :: Nil case _ => Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index e35a71917fbda..f7c3e347b61e1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -21,6 +21,7 @@ import java.io.File import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -128,29 +129,33 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { - sql("CACHE TABLE testCacheTable AS SELECT * FROM src") - assertCached(table("testCacheTable")) + withTempTable("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT * FROM src") + assertCached(table("testCacheTable")) - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") - uncacheTable("testCacheTable") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("CACHE TABLE tableName AS SELECT ...") { - sql("CACHE TABLE testCacheTable AS SELECT key FROM src LIMIT 10") - assertCached(table("testCacheTable")) + withTempTable("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM src LIMIT 10") + assertCached(table("testCacheTable")) - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") - uncacheTable("testCacheTable") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("CACHE LAZY TABLE tableName") { @@ -172,9 +177,11 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("CACHE TABLE with Hive UDF") { - sql("CACHE TABLE udfTest AS SELECT * FROM src WHERE floor(key) = 1") - assertCached(table("udfTest")) - uncacheTable("udfTest") + withTempTable("udfTest") { + sql("CACHE TABLE udfTest AS SELECT * FROM src WHERE floor(key) = 1") + assertCached(table("udfTest")) + uncacheTable("udfTest") + } } test("REFRESH TABLE also needs to recache the data (data source tables)") { @@ -267,6 +274,40 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto Utils.deleteRecursively(tempPath) } + test("Cache/Uncache Qualified Tables") { + withTempDatabase { db => + withTempTable("cachedTable") { + sql(s"CREATE TABLE $db.cachedTable STORED AS PARQUET AS SELECT 1") + sql(s"CACHE TABLE $db.cachedTable") + assertCached(spark.table(s"$db.cachedTable")) + + activateDatabase(db) { + assertCached(spark.table("cachedTable")) + sql("UNCACHE TABLE cachedTable") + assert(!spark.catalog.isCached("cachedTable"), "Table 'cachedTable' should not be cached") + sql(s"CACHE TABLE cachedTable") + assert(spark.catalog.isCached("cachedTable"), "Table 'cachedTable' should be cached") + } + + sql(s"UNCACHE TABLE $db.cachedTable") + assert(!spark.catalog.isCached(s"$db.cachedTable"), + "Table 'cachedTable' should not be cached") + } + } + } + + test("Cache Table As Select - having database name") { + withTempDatabase { db => + withTempTable("cachedTable") { + val e = intercept[ParseException] { + sql(s"CACHE TABLE $db.cachedTable AS SELECT 1") + }.getMessage + assert(e.contains("It is not allowed to add database prefix ") && + e.contains("to the table name in CACHE TABLE AS SELECT")) + } + } + } + test("SPARK-11246 cache parquet table") { sql("CREATE TABLE cachedTable STORED AS PARQUET AS SELECT 1") From c53eda03a282fb0569dd7e0dae3785999d022c8f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 16 Jun 2016 10:24:29 -0700 Subject: [PATCH 0676/1470] [SPARK-15983][SQL] Removes FileFormat.prepareRead ## What changes were proposed in this pull request? Interface method `FileFormat.prepareRead()` was added in #12088 to handle a special case in the LibSVM data source. However, the semantics of this interface method isn't intuitive: it returns a modified version of the data source options map. Considering that the LibSVM case can be easily handled using schema metadata inside `inferSchema`, we can remove this interface method to keep the `FileFormat` interface clean. ## How was this patch tested? Existing tests. Author: Cheng Lian Closes #13698 from liancheng/remove-prepare-read. (cherry picked from commit 9ea0d5e326e08b914aa46f1eec8795688a61bf74) Signed-off-by: Wenchen Fan --- .../ml/source/libsvm/LibSVMRelation.scala | 33 ++++++++++--------- .../execution/datasources/DataSource.scala | 5 +-- .../datasources/fileSourceInterfaces.scala | 11 ------- 3 files changed, 18 insertions(+), 31 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 62e09d2e0cad9..4988dd66f8274 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -120,9 +120,12 @@ class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { override def toString: String = "LibSVM" private def verifySchema(dataSchema: StructType): Unit = { - if (dataSchema.size != 2 || - (!dataSchema(0).dataType.sameType(DataTypes.DoubleType) - || !dataSchema(1).dataType.sameType(new VectorUDT()))) { + if ( + dataSchema.size != 2 || + !dataSchema(0).dataType.sameType(DataTypes.DoubleType) || + !dataSchema(1).dataType.sameType(new VectorUDT()) || + !(dataSchema(1).metadata.getLong("numFeatures").toInt > 0) + ) { throw new IOException(s"Illegal schema for libsvm data, schema=$dataSchema") } } @@ -131,17 +134,8 @@ class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { sparkSession: SparkSession, options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { - Some( - StructType( - StructField("label", DoubleType, nullable = false) :: - StructField("features", new VectorUDT(), nullable = false) :: Nil)) - } - - override def prepareRead( - sparkSession: SparkSession, - options: Map[String, String], - files: Seq[FileStatus]): Map[String, String] = { - val numFeatures = options.get("numFeatures").filter(_.toInt > 0).getOrElse { + val numFeatures: Int = options.get("numFeatures").map(_.toInt).filter(_ > 0).getOrElse { + // Infers number of features if the user doesn't specify (a valid) one. val dataFiles = files.filterNot(_.getPath.getName startsWith "_") val path = if (dataFiles.length == 1) { dataFiles.head.getPath.toUri.toString @@ -156,7 +150,14 @@ class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { MLUtils.computeNumFeatures(parsed) } - new CaseInsensitiveMap(options + ("numFeatures" -> numFeatures.toString)) + val featuresMetadata = new MetadataBuilder() + .putLong("numFeatures", numFeatures) + .build() + + Some( + StructType( + StructField("label", DoubleType, nullable = false) :: + StructField("features", new VectorUDT(), nullable = false, featuresMetadata) :: Nil)) } override def prepareWrite( @@ -185,7 +186,7 @@ class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { options: Map[String, String], hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { verifySchema(dataSchema) - val numFeatures = options("numFeatures").toInt + val numFeatures = dataSchema("features").metadata.getLong("numFeatures").toInt assert(numFeatures > 0) val sparse = options.getOrElse("vectorType", "sparse") == "sparse" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index d3273025b6885..7f3683fc98197 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -386,9 +386,6 @@ case class DataSource( "It must be specified manually") } - val enrichedOptions = - format.prepareRead(sparkSession, caseInsensitiveOptions, fileCatalog.allFiles()) - HadoopFsRelation( sparkSession, fileCatalog, @@ -396,7 +393,7 @@ case class DataSource( dataSchema = dataSchema.asNullable, bucketSpec = bucketSpec, format, - enrichedOptions) + caseInsensitiveOptions) case _ => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 9c1898994c9a9..bfb34c098f9ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompres import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} -import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.sql._ @@ -186,15 +185,6 @@ trait FileFormat { options: Map[String, String], files: Seq[FileStatus]): Option[StructType] - /** - * Prepares a read job and returns a potentially updated data source option [[Map]]. This method - * can be useful for collecting necessary global information for scanning input data. - */ - def prepareRead( - sparkSession: SparkSession, - options: Map[String, String], - files: Seq[FileStatus]): Map[String, String] = options - /** * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can * be put here. For example, user defined output committer can be configured here @@ -453,7 +443,6 @@ private[sql] object HadoopFsRelation extends Logging { logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") val sparkContext = sparkSession.sparkContext - val sqlConf = sparkSession.sessionState.conf val serializableConfiguration = new SerializableConfiguration(hadoopConf) val serializedPaths = paths.map(_.toString) From 0a2291cd15751018f1680e92aa8f63be4546e7a7 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 16 Jun 2016 12:46:25 -0700 Subject: [PATCH 0677/1470] [SPARK-15996][R] Fix R examples by removing deprecated functions ## What changes were proposed in this pull request? Currently, R examples(`dataframe.R` and `data-manipulation.R`) fail like the following. We had better update them before releasing 2.0 RC. This PR updates them to use up-to-date APIs. ```bash $ bin/spark-submit examples/src/main/r/dataframe.R ... Warning message: 'createDataFrame(sqlContext...)' is deprecated. Use 'createDataFrame(data, schema = NULL, samplingRatio = 1.0)' instead. See help("Deprecated") ... Warning message: 'read.json(sqlContext...)' is deprecated. Use 'read.json(path)' instead. See help("Deprecated") ... Error: could not find function "registerTempTable" Execution halted ``` ## How was this patch tested? Manual. ``` curl -LO http://s3-us-west-2.amazonaws.com/sparkr-data/flights.csv bin/spark-submit examples/src/main/r/dataframe.R bin/spark-submit examples/src/main/r/data-manipulation.R flights.csv ``` Author: Dongjoon Hyun Closes #13714 from dongjoon-hyun/SPARK-15996. (cherry picked from commit a865f6e05297f6121bb2fde717860f9edeed263e) Signed-off-by: Shivaram Venkataraman --- examples/src/main/r/data-manipulation.R | 8 ++++---- examples/src/main/r/dataframe.R | 11 +++++++---- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/examples/src/main/r/data-manipulation.R b/examples/src/main/r/data-manipulation.R index 58a30135aa923..badb98bc78fd1 100644 --- a/examples/src/main/r/data-manipulation.R +++ b/examples/src/main/r/data-manipulation.R @@ -49,10 +49,10 @@ flights_df$date <- as.Date(flights_df$date) SFO_df <- flights_df[flights_df$dest == "SFO", ] # Convert the local data frame into a SparkDataFrame -SFO_DF <- createDataFrame(sqlContext, SFO_df) +SFO_DF <- createDataFrame(SFO_df) # Directly create a SparkDataFrame from the source data -flightsDF <- read.df(sqlContext, flightsCsvPath, source = "csv", header = "true") +flightsDF <- read.df(flightsCsvPath, source = "csv", header = "true") # Print the schema of this SparkDataFrame printSchema(flightsDF) @@ -75,8 +75,8 @@ destDF <- select(flightsDF, "dest", "cancelled") # Using SQL to select columns of data # First, register the flights SparkDataFrame as a table -registerTempTable(flightsDF, "flightsTable") -destDF <- sql(sqlContext, "SELECT dest, cancelled FROM flightsTable") +createOrReplaceTempView(flightsDF, "flightsTable") +destDF <- sql("SELECT dest, cancelled FROM flightsTable") # Use collect to create a local R data frame local_df <- collect(destDF) diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R index 436bac6aaf455..04347057178b1 100644 --- a/examples/src/main/r/dataframe.R +++ b/examples/src/main/r/dataframe.R @@ -25,7 +25,7 @@ sqlContext <- sparkRSQL.init(sc) localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) # Convert local data frame to a SparkDataFrame -df <- createDataFrame(sqlContext, localDF) +df <- createDataFrame(localDF) # Print its schema printSchema(df) @@ -35,14 +35,17 @@ printSchema(df) # Create a DataFrame from a JSON file path <- file.path(Sys.getenv("SPARK_HOME"), "examples/src/main/resources/people.json") -peopleDF <- read.json(sqlContext, path) +peopleDF <- read.json(path) printSchema(peopleDF) +# root +# |-- age: long (nullable = true) +# |-- name: string (nullable = true) # Register this DataFrame as a table. -registerTempTable(peopleDF, "people") +createOrReplaceTempView(peopleDF, "people") # SQL statements can be run by using the sql methods provided by sqlContext -teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Call collect to get a local data.frame teenagersLocalDF <- collect(teenagers) From e11c279188b34d410f6ecf17cb1773c95f24a19e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 16 Jun 2016 13:17:41 -0700 Subject: [PATCH 0678/1470] [SPARK-15981][SQL][STREAMING] Fixed bug and added tests in DataStreamReader Python API ## What changes were proposed in this pull request? - Fixed bug in Python API of DataStreamReader. Because a single path was being converted to a array before calling Java DataStreamReader method (which takes a string only), it gave the following error. ``` File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 947, in pyspark.sql.readwriter.DataStreamReader.json Failed example: json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema) Exception raised: Traceback (most recent call last): File "/System/Library/Frameworks/Python.framework/Versions/2.6/lib/python2.6/doctest.py", line 1253, in __run compileflags, 1) in test.globs File "", line 1, in json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema) File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 963, in json return self._df(self._jreader.json(path)) File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ answer, self.gateway_client, self.target_id, self.name) File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 316, in get_return_value format(target_id, ".", name, value)) Py4JError: An error occurred while calling o121.json. Trace: py4j.Py4JException: Method json([class java.util.ArrayList]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326) at py4j.Gateway.invoke(Gateway.java:272) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:211) at java.lang.Thread.run(Thread.java:744) ``` - Reduced code duplication between DataStreamReader and DataFrameWriter - Added missing Python doctests ## How was this patch tested? New tests Author: Tathagata Das Closes #13703 from tdas/SPARK-15981. (cherry picked from commit 084dca770f5c26f906e7555707c7894cf05fb86b) Signed-off-by: Shixiong Zhu --- python/pyspark/sql/readwriter.py | 258 ++++++++++++++++--------------- 1 file changed, 136 insertions(+), 122 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index c982de6840d48..72fd184d580cc 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -44,7 +44,82 @@ def to_str(value): return str(value) -class DataFrameReader(object): +class ReaderUtils(object): + + def _set_json_opts(self, schema, primitivesAsString, prefersDecimal, + allowComments, allowUnquotedFieldNames, allowSingleQuotes, + allowNumericLeadingZero, allowBackslashEscapingAnyCharacter, + mode, columnNameOfCorruptRecord): + """ + Set options based on the Json optional parameters + """ + if schema is not None: + self.schema(schema) + if primitivesAsString is not None: + self.option("primitivesAsString", primitivesAsString) + if prefersDecimal is not None: + self.option("prefersDecimal", prefersDecimal) + if allowComments is not None: + self.option("allowComments", allowComments) + if allowUnquotedFieldNames is not None: + self.option("allowUnquotedFieldNames", allowUnquotedFieldNames) + if allowSingleQuotes is not None: + self.option("allowSingleQuotes", allowSingleQuotes) + if allowNumericLeadingZero is not None: + self.option("allowNumericLeadingZero", allowNumericLeadingZero) + if allowBackslashEscapingAnyCharacter is not None: + self.option("allowBackslashEscapingAnyCharacter", allowBackslashEscapingAnyCharacter) + if mode is not None: + self.option("mode", mode) + if columnNameOfCorruptRecord is not None: + self.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + + def _set_csv_opts(self, schema, sep, encoding, quote, escape, + comment, header, inferSchema, ignoreLeadingWhiteSpace, + ignoreTrailingWhiteSpace, nullValue, nanValue, positiveInf, negativeInf, + dateFormat, maxColumns, maxCharsPerColumn, mode): + """ + Set options based on the CSV optional parameters + """ + if schema is not None: + self.schema(schema) + if sep is not None: + self.option("sep", sep) + if encoding is not None: + self.option("encoding", encoding) + if quote is not None: + self.option("quote", quote) + if escape is not None: + self.option("escape", escape) + if comment is not None: + self.option("comment", comment) + if header is not None: + self.option("header", header) + if inferSchema is not None: + self.option("inferSchema", inferSchema) + if ignoreLeadingWhiteSpace is not None: + self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) + if ignoreTrailingWhiteSpace is not None: + self.option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) + if nullValue is not None: + self.option("nullValue", nullValue) + if nanValue is not None: + self.option("nanValue", nanValue) + if positiveInf is not None: + self.option("positiveInf", positiveInf) + if negativeInf is not None: + self.option("negativeInf", negativeInf) + if dateFormat is not None: + self.option("dateFormat", dateFormat) + if maxColumns is not None: + self.option("maxColumns", maxColumns) + if maxCharsPerColumn is not None: + self.option("maxCharsPerColumn", maxCharsPerColumn) + if mode is not None: + self.option("mode", mode) + + +class DataFrameReader(ReaderUtils): """ Interface used to load a :class:`DataFrame` from external storage systems (e.g. file systems, key-value stores, etc). Use :func:`spark.read` @@ -193,26 +268,10 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, [('age', 'bigint'), ('name', 'string')] """ - if schema is not None: - self.schema(schema) - if primitivesAsString is not None: - self.option("primitivesAsString", primitivesAsString) - if prefersDecimal is not None: - self.option("prefersDecimal", prefersDecimal) - if allowComments is not None: - self.option("allowComments", allowComments) - if allowUnquotedFieldNames is not None: - self.option("allowUnquotedFieldNames", allowUnquotedFieldNames) - if allowSingleQuotes is not None: - self.option("allowSingleQuotes", allowSingleQuotes) - if allowNumericLeadingZero is not None: - self.option("allowNumericLeadingZero", allowNumericLeadingZero) - if allowBackslashEscapingAnyCharacter is not None: - self.option("allowBackslashEscapingAnyCharacter", allowBackslashEscapingAnyCharacter) - if mode is not None: - self.option("mode", mode) - if columnNameOfCorruptRecord is not None: - self.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + self._set_json_opts(schema, primitivesAsString, prefersDecimal, + allowComments, allowUnquotedFieldNames, allowSingleQuotes, + allowNumericLeadingZero, allowBackslashEscapingAnyCharacter, + mode, columnNameOfCorruptRecord) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -345,42 +404,11 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non >>> df.dtypes [('_c0', 'string'), ('_c1', 'string')] """ - if schema is not None: - self.schema(schema) - if sep is not None: - self.option("sep", sep) - if encoding is not None: - self.option("encoding", encoding) - if quote is not None: - self.option("quote", quote) - if escape is not None: - self.option("escape", escape) - if comment is not None: - self.option("comment", comment) - if header is not None: - self.option("header", header) - if inferSchema is not None: - self.option("inferSchema", inferSchema) - if ignoreLeadingWhiteSpace is not None: - self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) - if ignoreTrailingWhiteSpace is not None: - self.option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) - if nullValue is not None: - self.option("nullValue", nullValue) - if nanValue is not None: - self.option("nanValue", nanValue) - if positiveInf is not None: - self.option("positiveInf", positiveInf) - if negativeInf is not None: - self.option("negativeInf", negativeInf) - if dateFormat is not None: - self.option("dateFormat", dateFormat) - if maxColumns is not None: - self.option("maxColumns", maxColumns) - if maxCharsPerColumn is not None: - self.option("maxCharsPerColumn", maxCharsPerColumn) - if mode is not None: - self.option("mode", mode) + + self._set_csv_opts(schema, sep, encoding, quote, escape, + comment, header, inferSchema, ignoreLeadingWhiteSpace, + ignoreTrailingWhiteSpace, nullValue, nanValue, positiveInf, negativeInf, + dateFormat, maxColumns, maxCharsPerColumn, mode) if isinstance(path, basestring): path = [path] return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path))) @@ -764,7 +792,7 @@ def jdbc(self, url, table, mode=None, properties=None): self._jwrite.mode(mode).jdbc(url, table, jprop) -class DataStreamReader(object): +class DataStreamReader(ReaderUtils): """ Interface used to load a streaming :class:`DataFrame` from external storage systems (e.g. file systems, key-value stores, etc). Use :func:`spark.readStream` @@ -791,6 +819,7 @@ def format(self, source): :param source: string, name of the data source, e.g. 'json', 'parquet'. + >>> s = spark.readStream.format("text") """ self._jreader = self._jreader.format(source) return self @@ -806,6 +835,8 @@ def schema(self, schema): .. note:: Experimental. :param schema: a StructType object + + >>> s = spark.readStream.schema(sdf_schema) """ if not isinstance(schema, StructType): raise TypeError("schema should be StructType") @@ -818,6 +849,8 @@ def option(self, key, value): """Adds an input option for the underlying data source. .. note:: Experimental. + + >>> s = spark.readStream.option("x", 1) """ self._jreader = self._jreader.option(key, to_str(value)) return self @@ -827,6 +860,8 @@ def options(self, **options): """Adds input options for the underlying data source. .. note:: Experimental. + + >>> s = spark.readStream.options(x="1", y=2) """ for k in options: self._jreader = self._jreader.option(k, to_str(options[k])) @@ -843,6 +878,13 @@ def load(self, path=None, format=None, schema=None, **options): :param schema: optional :class:`StructType` for the input schema. :param options: all other string options + >>> json_sdf = spark.readStream.format("json")\ + .schema(sdf_schema)\ + .load(os.path.join(tempfile.mkdtemp(),'data')) + >>> json_sdf.isStreaming + True + >>> json_sdf.schema == sdf_schema + True """ if format is not None: self.format(format) @@ -905,29 +947,18 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. + >>> json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), \ + schema = sdf_schema) + >>> json_sdf.isStreaming + True + >>> json_sdf.schema == sdf_schema + True """ - if schema is not None: - self.schema(schema) - if primitivesAsString is not None: - self.option("primitivesAsString", primitivesAsString) - if prefersDecimal is not None: - self.option("prefersDecimal", prefersDecimal) - if allowComments is not None: - self.option("allowComments", allowComments) - if allowUnquotedFieldNames is not None: - self.option("allowUnquotedFieldNames", allowUnquotedFieldNames) - if allowSingleQuotes is not None: - self.option("allowSingleQuotes", allowSingleQuotes) - if allowNumericLeadingZero is not None: - self.option("allowNumericLeadingZero", allowNumericLeadingZero) - if allowBackslashEscapingAnyCharacter is not None: - self.option("allowBackslashEscapingAnyCharacter", allowBackslashEscapingAnyCharacter) - if mode is not None: - self.option("mode", mode) - if columnNameOfCorruptRecord is not None: - self.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + self._set_json_opts(schema, primitivesAsString, prefersDecimal, + allowComments, allowUnquotedFieldNames, allowSingleQuotes, + allowNumericLeadingZero, allowBackslashEscapingAnyCharacter, + mode, columnNameOfCorruptRecord) if isinstance(path, basestring): - path = [path] return self._df(self._jreader.json(path)) else: raise TypeError("path can be only a single string") @@ -943,10 +974,15 @@ def parquet(self, path): .. note:: Experimental. + >>> parquet_sdf = spark.readStream.schema(sdf_schema)\ + .parquet(os.path.join(tempfile.mkdtemp())) + >>> parquet_sdf.isStreaming + True + >>> parquet_sdf.schema == sdf_schema + True """ if isinstance(path, basestring): - path = [path] - return self._df(self._jreader.parquet(self._spark._sc._jvm.PythonUtils.toSeq(path))) + return self._df(self._jreader.parquet(path)) else: raise TypeError("path can be only a single string") @@ -964,10 +1000,14 @@ def text(self, path): :param paths: string, or list of strings, for input path(s). + >>> text_sdf = spark.readStream.text(os.path.join(tempfile.mkdtemp(), 'data')) + >>> text_sdf.isStreaming + True + >>> "value" in str(text_sdf.schema) + True """ if isinstance(path, basestring): - path = [path] - return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(path))) + return self._df(self._jreader.text(path)) else: raise TypeError("path can be only a single string") @@ -1034,46 +1074,20 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non * ``DROPMALFORMED`` : ignores the whole corrupted records. * ``FAILFAST`` : throws an exception when it meets corrupted records. + >>> csv_sdf = spark.readStream.csv(os.path.join(tempfile.mkdtemp(), 'data'), \ + schema = sdf_schema) + >>> csv_sdf.isStreaming + True + >>> csv_sdf.schema == sdf_schema + True """ - if schema is not None: - self.schema(schema) - if sep is not None: - self.option("sep", sep) - if encoding is not None: - self.option("encoding", encoding) - if quote is not None: - self.option("quote", quote) - if escape is not None: - self.option("escape", escape) - if comment is not None: - self.option("comment", comment) - if header is not None: - self.option("header", header) - if inferSchema is not None: - self.option("inferSchema", inferSchema) - if ignoreLeadingWhiteSpace is not None: - self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) - if ignoreTrailingWhiteSpace is not None: - self.option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) - if nullValue is not None: - self.option("nullValue", nullValue) - if nanValue is not None: - self.option("nanValue", nanValue) - if positiveInf is not None: - self.option("positiveInf", positiveInf) - if negativeInf is not None: - self.option("negativeInf", negativeInf) - if dateFormat is not None: - self.option("dateFormat", dateFormat) - if maxColumns is not None: - self.option("maxColumns", maxColumns) - if maxCharsPerColumn is not None: - self.option("maxCharsPerColumn", maxCharsPerColumn) - if mode is not None: - self.option("mode", mode) + + self._set_csv_opts(schema, sep, encoding, quote, escape, + comment, header, inferSchema, ignoreLeadingWhiteSpace, + ignoreTrailingWhiteSpace, nullValue, nanValue, positiveInf, negativeInf, + dateFormat, maxColumns, maxCharsPerColumn, mode) if isinstance(path, basestring): - path = [path] - return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path))) + return self._df(self._jreader.csv(path)) else: raise TypeError("path can be only a single string") @@ -1286,7 +1300,7 @@ def _test(): globs['df'] = spark.read.parquet('python/test_support/sql/parquet_partitioned') globs['sdf'] = \ spark.readStream.format('text').load('python/test_support/sql/streaming') - + globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) (failure_count, test_count) = doctest.testmod( pyspark.sql.readwriter, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) From 5b003c9bcf43709408ed8f68d17b249675f50fbc Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 16 Jun 2016 13:47:36 -0700 Subject: [PATCH 0679/1470] [SPARK-15977][SQL] Fix TRUNCATE TABLE for Spark specific datasource tables ## What changes were proposed in this pull request? `TRUNCATE TABLE` is currently broken for Spark specific datasource tables (json, csv, ...). This PR correctly sets the location for these datasources which allows them to be truncated. ## How was this patch tested? Extended the datasources `TRUNCATE TABLE` tests in `DDLSuite`. Author: Herman van Hovell Closes #13697 from hvanhovell/SPARK-15977. (cherry picked from commit f9bf15d9bde4df2178f7a8f932c883bb77c46149) Signed-off-by: Herman van Hovell --- .../spark/sql/execution/command/tables.scala | 4 ++- .../sql/execution/command/DDLSuite.scala | 28 ++++++++++++------- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 58bb5cdca9910..3eb93a2922708 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -348,7 +348,9 @@ case class TruncateTableCommand( s"for tables that are not partitioned: '$tableName'") } val locations = - if (isDatasourceTable || table.partitionColumnNames.isEmpty) { + if (isDatasourceTable) { + Seq(table.storage.serdeProperties.get("path")) + } else if (table.partitionColumnNames.isEmpty) { Seq(table.storage.locationUri) } else { catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e15fcf4326be2..7eb2fff91d6e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1280,17 +1280,25 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("truncate table - datasource table") { import testImplicits._ val data = (1 to 10).map { i => (i, i) }.toDF("width", "length") - data.write.saveAsTable("rectangles") - spark.catalog.cacheTable("rectangles") - assume(spark.table("rectangles").collect().nonEmpty, "bad test; table was empty to begin with") - assume(spark.catalog.isCached("rectangles"), "bad test; table was not cached to begin with") - sql("TRUNCATE TABLE rectangles") - assert(spark.table("rectangles").collect().isEmpty) - assert(!spark.catalog.isCached("rectangles")) + + // Test both a Hive compatible and incompatible code path. + Seq("json", "parquet").foreach { format => + withTable("rectangles") { + data.write.format(format).saveAsTable("rectangles") + assume(spark.table("rectangles").collect().nonEmpty, + "bad test; table was empty to begin with") + sql("TRUNCATE TABLE rectangles") + assert(spark.table("rectangles").collect().isEmpty) + } + } + // truncating partitioned data source tables is not supported - data.write.partitionBy("length").saveAsTable("rectangles2") - assertUnsupported("TRUNCATE TABLE rectangles PARTITION (width=1)") - assertUnsupported("TRUNCATE TABLE rectangles2 PARTITION (width=1)") + withTable("rectangles", "rectangles2") { + data.write.saveAsTable("rectangles") + data.write.partitionBy("length").saveAsTable("rectangles2") + assertUnsupported("TRUNCATE TABLE rectangles PARTITION (width=1)") + assertUnsupported("TRUNCATE TABLE rectangles2 PARTITION (width=1)") + } } test("truncate table - external table, temporary table, view (not allowed)") { From 5792684268b273562e694855eb671c21c4044280 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 16 Jun 2016 23:02:46 +0200 Subject: [PATCH 0680/1470] [SPARK-15922][MLLIB] `toIndexedRowMatrix` should consider the case `cols < offset+colsPerBlock` ## What changes were proposed in this pull request? SPARK-15922 reports the following scenario throwing an exception due to the mismatched vector sizes. This PR handles the exceptional case, `cols < (offset + colsPerBlock)`. **Before** ```scala scala> import org.apache.spark.mllib.linalg.distributed._ scala> import org.apache.spark.mllib.linalg._ scala> val rows = IndexedRow(0L, new DenseVector(Array(1,2,3))) :: IndexedRow(1L, new DenseVector(Array(1,2,3))):: IndexedRow(2L, new DenseVector(Array(1,2,3))):: Nil scala> val rdd = sc.parallelize(rows) scala> val matrix = new IndexedRowMatrix(rdd, 3, 3) scala> val bmat = matrix.toBlockMatrix scala> val imat = bmat.toIndexedRowMatrix scala> imat.rows.collect ... // java.lang.IllegalArgumentException: requirement failed: Vectors must be the same length! ``` **After** ```scala ... scala> imat.rows.collect res0: Array[org.apache.spark.mllib.linalg.distributed.IndexedRow] = Array(IndexedRow(0,[1.0,2.0,3.0]), IndexedRow(1,[1.0,2.0,3.0]), IndexedRow(2,[1.0,2.0,3.0])) ``` ## How was this patch tested? Pass the Jenkins tests (including the above case) Author: Dongjoon Hyun Closes #13643 from dongjoon-hyun/SPARK-15922. (cherry picked from commit 36110a8306608186696c536028d2776e022d305a) Signed-off-by: Sean Owen --- .../apache/spark/mllib/linalg/distributed/BlockMatrix.scala | 2 +- .../spark/mllib/linalg/distributed/BlockMatrixSuite.scala | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 7a24617781ece..639295c695255 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -288,7 +288,7 @@ class BlockMatrix @Since("1.3.0") ( vectors.foreach { case (blockColIdx: Int, vec: BV[Double]) => val offset = colsPerBlock * blockColIdx - wholeVector(offset until offset + colsPerBlock) := vec + wholeVector(offset until Math.min(cols, offset + colsPerBlock)) := vec } new IndexedRow(rowIdx, Vectors.fromBreeze(wholeVector)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index e5a2cbbb588df..61266f3c78dbc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -135,6 +135,11 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(rowMat.numCols() === n) assert(rowMat.toBreeze() === gridBasedMat.toBreeze()) + // SPARK-15922: BlockMatrix to IndexedRowMatrix throws an error" + val bmat = rowMat.toBlockMatrix + val imat = bmat.toIndexedRowMatrix + imat.rows.collect + val rows = 1 val cols = 10 From 095ddb4c9e7ab9193c15c69eb057a9bb2dbdaed1 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 16 Jun 2016 23:04:10 +0200 Subject: [PATCH 0681/1470] [SPARK-15796][CORE] Reduce spark.memory.fraction default to avoid overrunning old gen in JVM default config ## What changes were proposed in this pull request? Reduce `spark.memory.fraction` default to 0.6 in order to make it fit within default JVM old generation size (2/3 heap). See JIRA discussion. This means a full cache doesn't spill into the new gen. CC andrewor14 ## How was this patch tested? Jenkins tests. Author: Sean Owen Closes #13618 from srowen/SPARK-15796. (cherry picked from commit 457126e420e66228cc68def4bc3d87e7a282069a) Signed-off-by: Sean Owen --- .../spark/memory/UnifiedMemoryManager.scala | 8 ++++---- .../org/apache/spark/DistributedSuite.scala | 2 +- docs/configuration.md | 7 ++++--- docs/tuning.md | 18 +++++++++++++++++- 4 files changed, 26 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index ae747c1d163e8..c7b36be6027a5 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -25,9 +25,9 @@ import org.apache.spark.storage.BlockId * either side can borrow memory from the other. * * The region shared between execution and storage is a fraction of (the total heap space - 300MB) - * configurable through `spark.memory.fraction` (default 0.75). The position of the boundary + * configurable through `spark.memory.fraction` (default 0.6). The position of the boundary * within this space is further determined by `spark.memory.storageFraction` (default 0.5). - * This means the size of the storage region is 0.75 * 0.5 = 0.375 of the heap space by default. + * This means the size of the storage region is 0.6 * 0.5 = 0.3 of the heap space by default. * * Storage can borrow as much execution memory as is free until execution reclaims its space. * When this happens, cached blocks will be evicted from memory until sufficient borrowed @@ -187,7 +187,7 @@ object UnifiedMemoryManager { // Set aside a fixed amount of memory for non-storage, non-execution purposes. // This serves a function similar to `spark.memory.fraction`, but guarantees that we reserve // sufficient memory for the system even for small heaps. E.g. if we have a 1GB JVM, then - // the memory used for execution and storage will be (1024 - 300) * 0.75 = 543MB by default. + // the memory used for execution and storage will be (1024 - 300) * 0.6 = 434MB by default. private val RESERVED_SYSTEM_MEMORY_BYTES = 300 * 1024 * 1024 def apply(conf: SparkConf, numCores: Int): UnifiedMemoryManager = { @@ -223,7 +223,7 @@ object UnifiedMemoryManager { } } val usableMemory = systemMemory - reservedMemory - val memoryFraction = conf.getDouble("spark.memory.fraction", 0.75) + val memoryFraction = conf.getDouble("spark.memory.fraction", 0.6) (usableMemory * memoryFraction).toLong } } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 6e69fc4247079..0515e6e3a6319 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -223,7 +223,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("compute when only some partitions fit in memory") { val size = 10000 - val numPartitions = 10 + val numPartitions = 20 val conf = new SparkConf() .set("spark.storage.unrollMemoryThreshold", "1024") .set("spark.testing.memory", size.toString) diff --git a/docs/configuration.md b/docs/configuration.md index 32c3a9266078a..fbda91c109626 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -778,14 +778,15 @@ Apart from these, the following properties are also available, and may be useful
    - + diff --git a/docs/tuning.md b/docs/tuning.md index e73ed69ffbbf8..1ed14091c0546 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -115,12 +115,28 @@ Although there are two relevant configurations, the typical user should not need as the default values are applicable to most workloads: * `spark.memory.fraction` expresses the size of `M` as a fraction of the (JVM heap space - 300MB) -(default 0.75). The rest of the space (25%) is reserved for user data structures, internal +(default 0.6). The rest of the space (25%) is reserved for user data structures, internal metadata in Spark, and safeguarding against OOM errors in the case of sparse and unusually large records. * `spark.memory.storageFraction` expresses the size of `R` as a fraction of `M` (default 0.5). `R` is the storage space within `M` where cached blocks immune to being evicted by execution. +The value of `spark.memory.fraction` should be set in order to fit this amount of heap space +comfortably within the JVM's old or "tenured" generation. Otherwise, when much of this space is +used for caching and execution, the tenured generation will be full, which causes the JVM to +significantly increase time spent in garbage collection. See +Java GC sizing documentation +for more information. + +The tenured generation size is controlled by the JVM's `NewRatio` parameter, which defaults to 2, +meaning that the tenured generation is 2 times the size of the new generation (the rest of the heap). +So, by default, the tenured generation occupies 2/3 or about 0.66 of the heap. A value of +0.6 for `spark.memory.fraction` keeps storage and execution memory within the old generation with +room to spare. If `spark.memory.fraction` is increased to, say, 0.8, then `NewRatio` may have to +increase to 6 or more. + +`NewRatio` is set as a JVM flag for executors, which means adding +`spark.executor.extraJavaOptions=-XX:NewRatio=x` to a Spark job's configuration. ## Determining Memory Consumption From d9dd46edd3635ed79134a1521403c4478a34d3b3 Mon Sep 17 00:00:00 2001 From: bomeng Date: Thu, 16 Jun 2016 14:18:02 -0700 Subject: [PATCH 0682/1470] [SPARK-15978][SQL] improve 'show tables' command related codes ## What changes were proposed in this pull request? I've found some minor issues in "show tables" command: 1. In the `SessionCatalog.scala`, `listTables(db: String)` method will call `listTables(formatDatabaseName(db), "*")` to list all the tables for certain db, but in the method `listTables(db: String, pattern: String)`, this db name is formatted once more. So I think we should remove `formatDatabaseName()` in the caller. 2. I suggest to add sort to listTables(db: String) in InMemoryCatalog.scala, just like listDatabases(). ## How was this patch tested? The existing test cases should cover it. Author: bomeng Closes #13695 from bomeng/SPARK-15978. (cherry picked from commit bbad4cb48df2ac3ed7edb4c02db79540bd4085d8) Signed-off-by: Andrew Or --- .../org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala | 2 +- .../org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 14da30a36f40b..fb3e1b3637f21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -286,7 +286,7 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E override def listTables(db: String): Seq[String] = synchronized { requireDbExists(db) - catalog(db).tables.keySet.toSeq + catalog(db).tables.keySet.toSeq.sorted } override def listTables(db: String, pattern: String): Seq[String] = synchronized { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 1ec1bb1baf23b..7ab10d1c38237 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -445,7 +445,7 @@ class SessionCatalog( /** * List all tables in the specified database, including temporary tables. */ - def listTables(db: String): Seq[TableIdentifier] = listTables(formatDatabaseName(db), "*") + def listTables(db: String): Seq[TableIdentifier] = listTables(db, "*") /** * List all matching tables in the specified database, including temporary tables. From 9389887571705e03d18e695301f0cb0aa5bd9e21 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 16 Jun 2016 14:18:58 -0700 Subject: [PATCH 0683/1470] [SPARK-15975] Fix improper Popen retcode code handling in dev/run-tests In the `dev/run-tests.py` script we check a `Popen.retcode` for success using `retcode > 0`, but this is subtlety wrong because Popen's return code will be negative if the child process was terminated by a signal: https://docs.python.org/2/library/subprocess.html#subprocess.Popen.returncode In order to properly handle signals, we should change this to check `retcode != 0` instead. Author: Josh Rosen Closes #13692 from JoshRosen/dev-run-tests-return-code-handling. (cherry picked from commit acef843f67e770f0a2709fb3fbd1a53c200b2bc5) Signed-off-by: Andrew Or --- dev/run-tests.py | 2 +- dev/sparktestsupport/shellutils.py | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index dcf1be9d95eb4..930d7f8bd9459 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -294,7 +294,7 @@ def exec_sbt(sbt_args=()): print(line, end='') retcode = sbt_proc.wait() - if retcode > 0: + if retcode != 0: exit_from_command_with_retcode(sbt_cmd, retcode) diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index d280e797077d1..05af87189b18d 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -53,7 +53,10 @@ def subprocess_check_call(*popenargs, **kwargs): def exit_from_command_with_retcode(cmd, retcode): - print("[error] running", ' '.join(cmd), "; received return code", retcode) + if retcode < 0: + print("[error] running", ' '.join(cmd), "; process was terminated by signal", -retcode) + else: + print("[error] running", ' '.join(cmd), "; received return code", retcode) sys.exit(int(os.environ.get("CURRENT_BLOCK", 255))) From 1230516d9314f55183bfa542eb7cdfac9d8dfec5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 16 Jun 2016 14:20:44 -0700 Subject: [PATCH 0684/1470] [SQL] Minor HashAggregateExec string output fixes ## What changes were proposed in this pull request? This PR fixes some minor `.toString` format issues for `HashAggregateExec`. Before: ``` *HashAggregate(key=[a#234L,b#235L], functions=[count(1),max(c#236L)], output=[a#234L,b#235L,count(c)#247L,max(c)#248L]) ``` After: ``` *HashAggregate(keys=[a#234L, b#235L], functions=[count(1), max(c#236L)], output=[a#234L, b#235L, count(c)#247L, max(c)#248L]) ``` ## How was this patch tested? Manually tested. Author: Cheng Lian Closes #13710 from liancheng/minor-agg-string-fix. (cherry picked from commit 7a89f2adbbc82a23f06638806ffc8596a7efe7f3) Signed-off-by: Andrew Or --- .../sql/execution/aggregate/HashAggregateExec.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index caeeba1793680..54d7340d8acd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -774,13 +774,13 @@ case class HashAggregateExec( testFallbackStartsAt match { case None => - val keyString = Utils.truncatedString(groupingExpressions, "[", ",", "]") - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ",", "]") - val outputString = Utils.truncatedString(output, "[", ",", "]") + val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]") + val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]") + val outputString = Utils.truncatedString(output, "[", ", ", "]") if (verbose) { - s"HashAggregate(key=$keyString, functions=$functionString, output=$outputString)" + s"HashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { - s"HashAggregate(key=$keyString, functions=$functionString)" + s"HashAggregate(keys=$keyString, functions=$functionString)" } case Some(fallbackStartsAt) => s"HashAggregateWithControlledFallback $groupingExpressions " + From 7d8cddfb495d406b9f2fb5216edd14dea442ec73 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 16 Jun 2016 14:23:17 -0700 Subject: [PATCH 0685/1470] [SPARK-15998][SQL] Verification of SQLConf HIVE_METASTORE_PARTITION_PRUNING #### What changes were proposed in this pull request? `HIVE_METASTORE_PARTITION_PRUNING` is a public `SQLConf`. When `true`, some predicates will be pushed down into the Hive metastore so that unmatching partitions can be eliminated earlier. The current default value is `false`. For performance improvement, users might turn this parameter on. So far, the code base does not have such a test case to verify whether this `SQLConf` properly works. This PR is to improve the test case coverage for avoiding future regression. #### How was this patch tested? N/A Author: gatorsmile Closes #13716 from gatorsmile/addTestMetastorePartitionPruning. --- .../hive/execution/HiveTableScanSuite.scala | 60 ++++++++++++++++++- 1 file changed, 57 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 60f8be5e0e825..76d3f3dbab01f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.Row -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.util.Utils -class HiveTableScanSuite extends HiveComparisonTest { +class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestHiveSingleton { createQueryTest("partition_based_table_scan_with_different_serde", """ @@ -89,4 +90,57 @@ class HiveTableScanSuite extends HiveComparisonTest { assert(sql("select CaseSensitiveColName from spark_4959_2").head() === Row("hi")) assert(sql("select casesensitivecolname from spark_4959_2").head() === Row("hi")) } + + private def checkNumScannedPartitions(stmt: String, expectedNumParts: Int): Unit = { + val plan = sql(stmt).queryExecution.sparkPlan + val numPartitions = plan.collectFirst { + case p: HiveTableScanExec => + p.relation.getHiveQlPartitions(p.partitionPruningPred).length + }.getOrElse(0) + assert(numPartitions == expectedNumParts) + } + + test("Verify SQLConf HIVE_METASTORE_PARTITION_PRUNING") { + val view = "src" + withTempTable(view) { + spark.range(1, 5).createOrReplaceTempView(view) + val table = "table_with_partition" + withTable(table) { + sql( + s""" + |CREATE TABLE $table(id string) + |PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) + """.stripMargin) + sql( + s""" + |FROM $view v + |INSERT INTO TABLE $table + |PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') + |SELECT v.id + |INSERT INTO TABLE $table + |PARTITION (p1='a',p2='c',p3='c',p4='d',p5='e') + |SELECT v.id + """.stripMargin) + + Seq("true", "false").foreach { hivePruning => + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING.key -> hivePruning) { + // If the pruning predicate is used, getHiveQlPartitions should only return the + // qualified partition; Otherwise, it return all the partitions. + val expectedNumPartitions = if (hivePruning == "true") 1 else 2 + checkNumScannedPartitions( + stmt = s"SELECT id, p2 FROM $table WHERE p2 <= 'b'", expectedNumPartitions) + } + } + + Seq("true", "false").foreach { hivePruning => + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING.key -> hivePruning) { + // If the pruning predicate does not exist, getHiveQlPartitions should always + // return all the partitions. + checkNumScannedPartitions( + stmt = s"SELECT id, p2 FROM $table WHERE id <= 3", expectedNumParts = 2) + } + } + } + } + } } From fb0fab63cb005d9efc624aeb0ac85476a9ddc4f4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 16 Jun 2016 14:27:09 -0700 Subject: [PATCH 0686/1470] [MINOR][DOCS][SQL] Fix some comments about types(TypeCoercion,Partition) and exceptions. ## What changes were proposed in this pull request? This PR contains a few changes on code comments. - `HiveTypeCoercion` is renamed into `TypeCoercion`. - `NoSuchDatabaseException` is only used for the absence of database. - For partition type inference, only `DoubleType` is considered. ## How was this patch tested? N/A Author: Dongjoon Hyun Closes #13674 from dongjoon-hyun/minor_doc_types. (cherry picked from commit 2d27eb1e753daefbd311136fc7de1a3e8fb9dc63) Signed-off-by: Andrew Or --- .../org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala | 4 ++-- .../apache/spark/sql/catalyst/catalog/ExternalCatalog.scala | 2 +- .../src/main/scala/org/apache/spark/sql/types/Decimal.scala | 2 +- .../spark/sql/execution/datasources/PartitioningUtils.scala | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 16df628a5730c..baec6d14a212a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -73,7 +73,7 @@ object TypeCoercion { DoubleType) /** - * Case 1 type widening (see the classdoc comment above for HiveTypeCoercion). + * Case 1 type widening (see the classdoc comment above for TypeCoercion). * * Find the tightest common type of two types that might be used in a binary expression. * This handles all numeric types except fixed-precision decimals interacting with each other or @@ -132,7 +132,7 @@ object TypeCoercion { } /** - * Case 2 type widening (see the classdoc comment above for HiveTypeCoercion). + * Case 2 type widening (see the classdoc comment above for TypeCoercion). * * i.e. the main difference with [[findTightestCommonTypeOfTwo]] is that here we allow some * loss of precision when widening decimal and double. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 81974b282b367..6714846e8cbda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException * can be accessed in multiple threads. This is an external catalog because it is expected to * interact with external systems. * - * Implementations should throw [[NoSuchDatabaseException]] when table or database don't exist. + * Implementations should throw [[NoSuchDatabaseException]] when databases don't exist. */ abstract class ExternalCatalog { import CatalogTypes.TablePartitionSpec diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 52e021070e940..cc8175c0a366d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -322,7 +322,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } - // HiveTypeCoercion will take care of the precision, scale of result + // TypeCoercion will take care of the precision, scale of result def * (that: Decimal): Decimal = Decimal(toJavaBigDecimal.multiply(that.toJavaBigDecimal, MATH_CONTEXT)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 2340ff0afed74..388df7002dc36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -159,7 +159,7 @@ private[sql] object PartitioningUtils { * Seq( * Literal.create(42, IntegerType), * Literal.create("hello", StringType), - * Literal.create(3.14, FloatType))) + * Literal.create(3.14, DoubleType))) * }}} * and the path when we stop the discovery is: * {{{ From 27e274c3e8cad29fc684a1611cef19d60acdfbc0 Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Thu, 16 Jun 2016 14:29:11 -0700 Subject: [PATCH 0687/1470] [SPARK-15868][WEB UI] Executors table in Executors tab should sort Executor IDs in numerical order ## What changes were proposed in this pull request? Currently the Executors table sorts by id using a string sort (since that's what it is stored as). Since the id is a number (other than the driver) we should be sorting numerically. I have changed both the initial sort on page load as well as the table sort to sort on id numerically, treating non-numeric strings (like the driver) as "-1" ## How was this patch tested? Manually tested and dev/run-tests ![pageload](https://cloud.githubusercontent.com/assets/13952758/16027882/d32edd0a-318e-11e6-9faf-fc972b7c36ab.png) ![sorted](https://cloud.githubusercontent.com/assets/13952758/16027883/d34541c6-318e-11e6-9ed7-6bfc0cd4152e.png) Author: Alex Bozarth Closes #13654 from ajbozarth/spark15868. (cherry picked from commit e849285df03b1233d5f647f1b6c5a6dad0665855) Signed-off-by: Andrew Or --- .../scala/org/apache/spark/ui/exec/ExecutorsPage.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 791dbe5c272b5..67deb7b14bcb9 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -20,6 +20,7 @@ package org.apache.spark.ui.exec import java.net.URLEncoder import javax.servlet.http.HttpServletRequest +import scala.util.Try import scala.xml.Node import org.apache.spark.status.api.v1.ExecutorSummary @@ -53,6 +54,9 @@ private[ui] class ExecutorsPage( // When GCTimePercent is edited change ToolTips.TASK_TIME to match private val GCTimePercent = 0.1 + // a safe String to Int for sorting ids (converts non-numeric Strings to -1) + private def idStrToInt(str: String) : Int = Try(str.toInt).getOrElse(-1) + def render(request: HttpServletRequest): Seq[Node] = { val (activeExecutorInfo, deadExecutorInfo) = listener.synchronized { // The follow codes should be protected by `listener` to make sure no executors will be @@ -69,13 +73,14 @@ private[ui] class ExecutorsPage( } val execInfo = activeExecutorInfo ++ deadExecutorInfo + implicit val idOrder = Ordering[Int].on((s: String) => idStrToInt(s)).reverse val execInfoSorted = execInfo.sortBy(_.id) val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty val execTable = {
    Environment VariableMeaning
    SPARK_MASTER_IPBind the master to a specific IP address, for example a public one.SPARK_MASTER_HOSTBind the master to a specific hostname or IP address, for example a public one.
    SPARK_MASTER_PORT
    Property NameDefaultMeaning
    spark.memory.fraction0.750.6 Fraction of (heap space - 300MB) used for execution and storage. The lower this is, the more frequently spills and cached data eviction occur. The purpose of this config is to set aside memory for internal metadata, user data structures, and imprecise size estimation in the case of sparse, unusually large records. Leaving this at the default value is - recommended. For more detail, see - this description. + recommended. For more detail, including important information about correctly tuning JVM + garbage collection when increasing this value, see + this description.
    - + @@ -136,7 +141,7 @@ private[ui] class ExecutorsPage( } - + From b3678eb7e4ac6bb08ba8579867944ba42da99b81 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 16 Jun 2016 17:06:24 -0700 Subject: [PATCH 0690/1470] [SPARK-15991] SparkContext.hadoopConfiguration should be always the base of hadoop conf created by SessionState ## What changes were proposed in this pull request? Before this patch, after a SparkSession has been created, hadoop conf set directly to SparkContext.hadoopConfiguration will not affect the hadoop conf created by SessionState. This patch makes the change to always use SparkContext.hadoopConfiguration as the base. This patch also changes the behavior of hive-site.xml support added in https://github.com/apache/spark/pull/12689/. With this patch, we will load hive-site.xml to SparkContext.hadoopConfiguration. ## How was this patch tested? New test in SparkSessionBuilderSuite. Author: Yin Huai Closes #13711 from yhuai/SPARK-15991. (cherry picked from commit d9c6628c47de547dc537310e3c775c7f3e0e4a12) Signed-off-by: Shixiong Zhu --- .../spark/sql/internal/SessionState.scala | 2 +- .../spark/sql/internal/SharedState.scala | 14 ++++--------- .../org/apache/spark/sql/SQLQuerySuite.scala | 4 ---- .../spark/sql/SparkSessionBuilderSuite.scala | 20 +++++++++++++++++++ .../spark/sql/hive/HiveSharedState.scala | 5 +++-- 5 files changed, 28 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 59efa81275451..dc95123d0088b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -49,7 +49,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val conf: SQLConf = new SQLConf def newHadoopConf(): Configuration = { - val hadoopConf = new Configuration(sparkSession.sharedState.hadoopConf) + val hadoopConf = new Configuration(sparkSession.sparkContext.hadoopConfiguration) conf.getAllConfs.foreach { case (k, v) => if (v ne null) hadoopConf.set(k, v) } hadoopConf } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index bc349b4f28bf4..6c43fe3177d65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -43,23 +43,17 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { */ val listener: SQLListener = createListenerAndUI(sparkContext) - /** - * The base hadoop configuration which is shared among all spark sessions. It is based on the - * default hadoop configuration of Spark, with custom configurations inside `hive-site.xml`. - */ - val hadoopConf: Configuration = { - val conf = new Configuration(sparkContext.hadoopConfiguration) + { val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") if (configFile != null) { - conf.addResource(configFile) + sparkContext.hadoopConfiguration.addResource(configFile) } - conf } /** * A catalog that interacts with external systems. */ - lazy val externalCatalog: ExternalCatalog = new InMemoryCatalog(hadoopConf) + lazy val externalCatalog: ExternalCatalog = new InMemoryCatalog(sparkContext.hadoopConfiguration) /** * A classloader used to load all user-added jar. @@ -71,7 +65,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { // Set the Hive metastore warehouse path to the one we use val tempConf = new SQLConf sparkContext.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } - val hiveWarehouseDir = hadoopConf.get("hive.metastore.warehouse.dir") + val hiveWarehouseDir = sparkContext.hadoopConfiguration.get("hive.metastore.warehouse.dir") if (hiveWarehouseDir != null && !tempConf.contains(SQLConf.WAREHOUSE_PATH.key)) { // If hive.metastore.warehouse.dir is set and spark.sql.warehouse.dir is not set, // we will respect the value of hive.metastore.warehouse.dir. 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 545c1776b7e2a..bbe821b76805c 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 @@ -2870,8 +2870,4 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SELECT '$literal' AS DUMMY"), Row(s"$expected") :: Nil) } - - test("SPARK-15887: hive-site.xml should be loaded") { - assert(spark.sessionState.newHadoopConf().get("hive.in.test") == "true") - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 786956df8a555..418345b9ee8f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -102,4 +102,24 @@ class SparkSessionBuilderSuite extends SparkFunSuite { assert(session.sparkContext.conf.get("key2") == "value2") session.stop() } + + test("SPARK-15887: hive-site.xml should be loaded") { + val session = SparkSession.builder().master("local").getOrCreate() + assert(session.sessionState.newHadoopConf().get("hive.in.test") == "true") + assert(session.sparkContext.hadoopConfiguration.get("hive.in.test") == "true") + session.stop() + } + + test("SPARK-15991: Set global Hadoop conf") { + val session = SparkSession.builder().master("local").getOrCreate() + val mySpecialKey = "my.special.key.15991" + val mySpecialValue = "msv" + try { + session.sparkContext.hadoopConfiguration.set(mySpecialKey, mySpecialValue) + assert(session.sessionState.newHadoopConf().get(mySpecialKey) == mySpecialValue) + } finally { + session.sparkContext.hadoopConfiguration.unset(mySpecialKey) + session.stop() + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala index 6b7a333f2d3b8..12b4962fba178 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala @@ -36,11 +36,12 @@ private[hive] class HiveSharedState(override val sparkContext: SparkContext) */ // This needs to be a lazy val at here because TestHiveSharedState is overriding it. lazy val metadataHive: HiveClient = { - HiveUtils.newClientForMetadata(sparkContext.conf, hadoopConf) + HiveUtils.newClientForMetadata(sparkContext.conf, sparkContext.hadoopConfiguration) } /** * A catalog that interacts with the Hive metastore. */ - override lazy val externalCatalog = new HiveExternalCatalog(metadataHive, hadoopConf) + override lazy val externalCatalog = + new HiveExternalCatalog(metadataHive, sparkContext.hadoopConfiguration) } From 68e7a25cc06cbfe357be8d224c117abaa7ba94f4 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Thu, 16 Jun 2016 17:35:40 -0700 Subject: [PATCH 0691/1470] [SPARK-15608][ML][EXAMPLES][DOC] add examples and documents of ml.isotonic regression ## What changes were proposed in this pull request? add ml doc for ml isotonic regression add scala example for ml isotonic regression add java example for ml isotonic regression add python example for ml isotonic regression modify scala example for mllib isotonic regression modify java example for mllib isotonic regression modify python example for mllib isotonic regression add data/mllib/sample_isotonic_regression_libsvm_data.txt delete data/mllib/sample_isotonic_regression_data.txt ## How was this patch tested? N/A Author: WeichenXu Closes #13381 from WeichenXu123/add_isotonic_regression_doc. (cherry picked from commit 9040d83bc2cdce06dab0e1bdee4f796da9a9a55c) Signed-off-by: Yanbo Liang --- .../mllib/sample_isotonic_regression_data.txt | 100 ------------------ ...sample_isotonic_regression_libsvm_data.txt | 100 ++++++++++++++++++ docs/ml-classification-regression.md | 70 ++++++++++++ .../ml/JavaIsotonicRegressionExample.java | 62 +++++++++++ .../mllib/JavaIsotonicRegressionExample.java | 19 ++-- .../python/ml/isotonic_regression_example.py | 54 ++++++++++ .../mllib/isotonic_regression_example.py | 11 +- .../ml/IsotonicRegressionExample.scala | 62 +++++++++++ .../mllib/IsotonicRegressionExample.scala | 9 +- 9 files changed, 373 insertions(+), 114 deletions(-) delete mode 100644 data/mllib/sample_isotonic_regression_data.txt create mode 100644 data/mllib/sample_isotonic_regression_libsvm_data.txt create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java create mode 100644 examples/src/main/python/ml/isotonic_regression_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala diff --git a/data/mllib/sample_isotonic_regression_data.txt b/data/mllib/sample_isotonic_regression_data.txt deleted file mode 100644 index d257b509d4d37..0000000000000 --- a/data/mllib/sample_isotonic_regression_data.txt +++ /dev/null @@ -1,100 +0,0 @@ -0.24579296,0.01 -0.28505864,0.02 -0.31208567,0.03 -0.35900051,0.04 -0.35747068,0.05 -0.16675166,0.06 -0.17491076,0.07 -0.04181540,0.08 -0.04793473,0.09 -0.03926568,0.10 -0.12952575,0.11 -0.00000000,0.12 -0.01376849,0.13 -0.13105558,0.14 -0.08873024,0.15 -0.12595614,0.16 -0.15247323,0.17 -0.25956145,0.18 -0.20040796,0.19 -0.19581846,0.20 -0.15757267,0.21 -0.13717491,0.22 -0.19020908,0.23 -0.19581846,0.24 -0.20091790,0.25 -0.16879143,0.26 -0.18510964,0.27 -0.20040796,0.28 -0.29576747,0.29 -0.43396226,0.30 -0.53391127,0.31 -0.52116267,0.32 -0.48546660,0.33 -0.49209587,0.34 -0.54156043,0.35 -0.59765426,0.36 -0.56144824,0.37 -0.58592555,0.38 -0.52983172,0.39 -0.50178480,0.40 -0.52626211,0.41 -0.58286588,0.42 -0.64660887,0.43 -0.68077511,0.44 -0.74298827,0.45 -0.64864865,0.46 -0.67261601,0.47 -0.65782764,0.48 -0.69811321,0.49 -0.63029067,0.50 -0.61601224,0.51 -0.63233044,0.52 -0.65323814,0.53 -0.65323814,0.54 -0.67363590,0.55 -0.67006629,0.56 -0.51555329,0.57 -0.50892402,0.58 -0.33299337,0.59 -0.36206017,0.60 -0.43090260,0.61 -0.45996940,0.62 -0.56348802,0.63 -0.54920959,0.64 -0.48393677,0.65 -0.48495665,0.66 -0.46965834,0.67 -0.45181030,0.68 -0.45843957,0.69 -0.47118817,0.70 -0.51555329,0.71 -0.58031617,0.72 -0.55481897,0.73 -0.56297807,0.74 -0.56603774,0.75 -0.57929628,0.76 -0.64762876,0.77 -0.66241713,0.78 -0.69301377,0.79 -0.65119837,0.80 -0.68332483,0.81 -0.66598674,0.82 -0.73890872,0.83 -0.73992861,0.84 -0.84242733,0.85 -0.91330954,0.86 -0.88016318,0.87 -0.90719021,0.88 -0.93115757,0.89 -0.93115757,0.90 -0.91942886,0.91 -0.92911780,0.92 -0.95665477,0.93 -0.95002550,0.94 -0.96940337,0.95 -1.00000000,0.96 -0.89801122,0.97 -0.90311066,0.98 -0.90362060,0.99 -0.83477817,1.0 \ No newline at end of file diff --git a/data/mllib/sample_isotonic_regression_libsvm_data.txt b/data/mllib/sample_isotonic_regression_libsvm_data.txt new file mode 100644 index 0000000000000..f39fe0269c2fe --- /dev/null +++ b/data/mllib/sample_isotonic_regression_libsvm_data.txt @@ -0,0 +1,100 @@ +0.24579296 1:0.01 +0.28505864 1:0.02 +0.31208567 1:0.03 +0.35900051 1:0.04 +0.35747068 1:0.05 +0.16675166 1:0.06 +0.17491076 1:0.07 +0.04181540 1:0.08 +0.04793473 1:0.09 +0.03926568 1:0.10 +0.12952575 1:0.11 +0.00000000 1:0.12 +0.01376849 1:0.13 +0.13105558 1:0.14 +0.08873024 1:0.15 +0.12595614 1:0.16 +0.15247323 1:0.17 +0.25956145 1:0.18 +0.20040796 1:0.19 +0.19581846 1:0.20 +0.15757267 1:0.21 +0.13717491 1:0.22 +0.19020908 1:0.23 +0.19581846 1:0.24 +0.20091790 1:0.25 +0.16879143 1:0.26 +0.18510964 1:0.27 +0.20040796 1:0.28 +0.29576747 1:0.29 +0.43396226 1:0.30 +0.53391127 1:0.31 +0.52116267 1:0.32 +0.48546660 1:0.33 +0.49209587 1:0.34 +0.54156043 1:0.35 +0.59765426 1:0.36 +0.56144824 1:0.37 +0.58592555 1:0.38 +0.52983172 1:0.39 +0.50178480 1:0.40 +0.52626211 1:0.41 +0.58286588 1:0.42 +0.64660887 1:0.43 +0.68077511 1:0.44 +0.74298827 1:0.45 +0.64864865 1:0.46 +0.67261601 1:0.47 +0.65782764 1:0.48 +0.69811321 1:0.49 +0.63029067 1:0.50 +0.61601224 1:0.51 +0.63233044 1:0.52 +0.65323814 1:0.53 +0.65323814 1:0.54 +0.67363590 1:0.55 +0.67006629 1:0.56 +0.51555329 1:0.57 +0.50892402 1:0.58 +0.33299337 1:0.59 +0.36206017 1:0.60 +0.43090260 1:0.61 +0.45996940 1:0.62 +0.56348802 1:0.63 +0.54920959 1:0.64 +0.48393677 1:0.65 +0.48495665 1:0.66 +0.46965834 1:0.67 +0.45181030 1:0.68 +0.45843957 1:0.69 +0.47118817 1:0.70 +0.51555329 1:0.71 +0.58031617 1:0.72 +0.55481897 1:0.73 +0.56297807 1:0.74 +0.56603774 1:0.75 +0.57929628 1:0.76 +0.64762876 1:0.77 +0.66241713 1:0.78 +0.69301377 1:0.79 +0.65119837 1:0.80 +0.68332483 1:0.81 +0.66598674 1:0.82 +0.73890872 1:0.83 +0.73992861 1:0.84 +0.84242733 1:0.85 +0.91330954 1:0.86 +0.88016318 1:0.87 +0.90719021 1:0.88 +0.93115757 1:0.89 +0.93115757 1:0.90 +0.91942886 1:0.91 +0.92911780 1:0.92 +0.95665477 1:0.93 +0.95002550 1:0.94 +0.96940337 1:0.95 +1.00000000 1:0.96 +0.89801122 1:0.97 +0.90311066 1:0.98 +0.90362060 1:0.99 +0.83477817 1:1.0 \ No newline at end of file diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index d7e5521cbcb29..3d6106b532ff9 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -691,6 +691,76 @@ The implementation matches the result from R's survival function +## Isotonic regression +[Isotonic regression](http://en.wikipedia.org/wiki/Isotonic_regression) +belongs to the family of regression algorithms. Formally isotonic regression is a problem where +given a finite set of real numbers `$Y = {y_1, y_2, ..., y_n}$` representing observed responses +and `$X = {x_1, x_2, ..., x_n}$` the unknown response values to be fitted +finding a function that minimises + +`\begin{equation} + f(x) = \sum_{i=1}^n w_i (y_i - x_i)^2 +\end{equation}` + +with respect to complete order subject to +`$x_1\le x_2\le ...\le x_n$` where `$w_i$` are positive weights. +The resulting function is called isotonic regression and it is unique. +It can be viewed as least squares problem under order restriction. +Essentially isotonic regression is a +[monotonic function](http://en.wikipedia.org/wiki/Monotonic_function) +best fitting the original data points. + +We implement a +[pool adjacent violators algorithm](http://doi.org/10.1198/TECH.2010.10111) +which uses an approach to +[parallelizing isotonic regression](http://doi.org/10.1007/978-3-642-99789-1_10). +The training input is a DataFrame which contains three columns +label, features and weight. Additionally IsotonicRegression algorithm has one +optional parameter called $isotonic$ defaulting to true. +This argument specifies if the isotonic regression is +isotonic (monotonically increasing) or antitonic (monotonically decreasing). + +Training returns an IsotonicRegressionModel that can be used to predict +labels for both known and unknown features. The result of isotonic regression +is treated as piecewise linear function. The rules for prediction therefore are: + +* If the prediction input exactly matches a training feature + then associated prediction is returned. In case there are multiple predictions with the same + feature then one of them is returned. Which one is undefined + (same as java.util.Arrays.binarySearch). +* If the prediction input is lower or higher than all training features + then prediction with lowest or highest feature is returned respectively. + In case there are multiple predictions with the same feature + then the lowest or highest is returned respectively. +* If the prediction input falls between two training features then prediction is treated + as piecewise linear function and interpolated value is calculated from the + predictions of the two closest features. In case there are multiple values + with the same feature then the same rules as in previous point are used. + +### Examples + +
    +
    + +Refer to the [`IsotonicRegression` Scala docs](api/scala/index.html#org.apache.spark.ml.regression.IsotonicRegression) for details on the API. + +{% include_example scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala %} +
    +
    + +Refer to the [`IsotonicRegression` Java docs](api/java/org/apache/spark/ml/regression/IsotonicRegression.html) for details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java %} +
    +
    + +Refer to the [`IsotonicRegression` Python docs](api/python/pyspark.ml.html#pyspark.ml.regression.IsotonicRegression) for more details on the API. + +{% include_example python/ml/isotonic_regression_example.py %} +
    +
    + + # Decision trees diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java new file mode 100644 index 0000000000000..0ec17b0471553 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.ml; + +// $example on$ + +import org.apache.spark.ml.regression.IsotonicRegression; +import org.apache.spark.ml.regression.IsotonicRegressionModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * An example demonstrating IsotonicRegression. + * Run with + *
    + * bin/run-example ml.JavaIsotonicRegressionExample
    + * 
    + */ +public class JavaIsotonicRegressionExample { + + public static void main(String[] args) { + // Create a SparkSession. + SparkSession spark = SparkSession + .builder() + .appName("JavaIsotonicRegressionExample") + .getOrCreate(); + + // $example on$ + // Loads data. + Dataset dataset = spark.read().format("libsvm") + .load("data/mllib/sample_isotonic_regression_libsvm_data.txt"); + + // Trains an isotonic regression model. + IsotonicRegression ir = new IsotonicRegression(); + IsotonicRegressionModel model = ir.fit(dataset); + + System.out.println("Boundaries in increasing order: " + model.boundaries()); + System.out.println("Predictions associated with the boundaries: " + model.predictions()); + + // Makes predictions. + model.transform(dataset).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java index c6361a3729988..a30b5f1f73eaf 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java @@ -17,6 +17,7 @@ package org.apache.spark.examples.mllib; // $example on$ + import scala.Tuple2; import scala.Tuple3; import org.apache.spark.api.java.function.Function; @@ -27,6 +28,8 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.regression.IsotonicRegression; import org.apache.spark.mllib.regression.IsotonicRegressionModel; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; // $example off$ import org.apache.spark.SparkConf; @@ -35,27 +38,29 @@ public static void main(String[] args) { SparkConf sparkConf = new SparkConf().setAppName("JavaIsotonicRegressionExample"); JavaSparkContext jsc = new JavaSparkContext(sparkConf); // $example on$ - JavaRDD data = jsc.textFile("data/mllib/sample_isotonic_regression_data.txt"); + JavaRDD data = MLUtils.loadLibSVMFile( + jsc.sc(), "data/mllib/sample_isotonic_regression_libsvm_data.txt").toJavaRDD(); // Create label, feature, weight tuples from input data with weight set to default value 1.0. JavaRDD> parsedData = data.map( - new Function>() { - public Tuple3 call(String line) { - String[] parts = line.split(","); - return new Tuple3<>(new Double(parts[0]), new Double(parts[1]), 1.0); + new Function>() { + public Tuple3 call(LabeledPoint point) { + return new Tuple3<>(new Double(point.label()), + new Double(point.features().apply(0)), 1.0); } } ); // Split data into training (60%) and test (40%) sets. JavaRDD>[] splits = - parsedData.randomSplit(new double[]{0.6, 0.4}, 11L); + parsedData.randomSplit(new double[]{0.6, 0.4}, 11L); JavaRDD> training = splits[0]; JavaRDD> test = splits[1]; // Create isotonic regression model from training data. // Isotonic parameter defaults to true so it is only shown for demonstration - final IsotonicRegressionModel model = new IsotonicRegression().setIsotonic(true).run(training); + final IsotonicRegressionModel model = + new IsotonicRegression().setIsotonic(true).run(training); // Create tuples of predicted and real labels. JavaPairRDD predictionAndLabel = test.mapToPair( diff --git a/examples/src/main/python/ml/isotonic_regression_example.py b/examples/src/main/python/ml/isotonic_regression_example.py new file mode 100644 index 0000000000000..1e61bd8eff143 --- /dev/null +++ b/examples/src/main/python/ml/isotonic_regression_example.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Isotonic Regression Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml.regression import IsotonicRegression, IsotonicRegressionModel +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating isotonic regression. +Run with: + bin/spark-submit examples/src/main/python/ml/isotonic_regression_example.py +""" +if __name__ == "__main__": + + spark = SparkSession\ + .builder\ + .appName("PythonIsotonicRegressionExample")\ + .getOrCreate() + + # $example on$ + # Loads data. + dataset = spark.read.format("libsvm")\ + .load("data/mllib/sample_isotonic_regression_libsvm_data.txt") + + # Trains an isotonic regression model. + model = IsotonicRegression().fit(dataset) + print("Boundaries in increasing order: " + str(model.boundaries)) + print("Predictions associated with the boundaries: " + str(model.predictions)) + + # Makes predictions. + model.transform(dataset).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/mllib/isotonic_regression_example.py b/examples/src/main/python/mllib/isotonic_regression_example.py index 89dc9f4b6611a..33d618ab48ea9 100644 --- a/examples/src/main/python/mllib/isotonic_regression_example.py +++ b/examples/src/main/python/mllib/isotonic_regression_example.py @@ -23,7 +23,8 @@ from pyspark import SparkContext # $example on$ import math -from pyspark.mllib.regression import IsotonicRegression, IsotonicRegressionModel +from pyspark.mllib.regression import LabeledPoint, IsotonicRegression, IsotonicRegressionModel +from pyspark.mllib.util import MLUtils # $example off$ if __name__ == "__main__": @@ -31,10 +32,14 @@ sc = SparkContext(appName="PythonIsotonicRegressionExample") # $example on$ - data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt") + # Load and parse the data + def parsePoint(labeledData): + return (labeledData.label, labeledData.features[0], 1.0) + + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_isotonic_regression_libsvm_data.txt") # Create label, feature, weight tuples from input data with weight set to default value 1.0. - parsedData = data.map(lambda line: tuple([float(x) for x in line.split(',')]) + (1.0,)) + parsedData = data.map(parsePoint) # Split data into training (60%) and test (40%) sets. training, test = parsedData.randomSplit([0.6, 0.4], 11) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala new file mode 100644 index 0000000000000..7c5d3f23411f0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.regression.IsotonicRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating Isotonic Regression. + * Run with + * {{{ + * bin/run-example ml.IsotonicRegressionExample + * }}} + */ +object IsotonicRegressionExample { + + def main(args: Array[String]): Unit = { + + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + + // $example on$ + // Loads data. + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_isotonic_regression_libsvm_data.txt") + + // Trains an isotonic regression model. + val ir = new IsotonicRegression() + val model = ir.fit(dataset) + + println(s"Boundaries in increasing order: ${model.boundaries}") + println(s"Predictions associated with the boundaries: ${model.predictions}") + + // Makes predictions. + model.transform(dataset).show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala index c4336639d7c0b..e5dea129c113d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala @@ -21,6 +21,7 @@ package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.regression.{IsotonicRegression, IsotonicRegressionModel} +import org.apache.spark.mllib.util.MLUtils // $example off$ object IsotonicRegressionExample { @@ -30,12 +31,12 @@ object IsotonicRegressionExample { val conf = new SparkConf().setAppName("IsotonicRegressionExample") val sc = new SparkContext(conf) // $example on$ - val data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt") + val data = MLUtils.loadLibSVMFile(sc, + "data/mllib/sample_isotonic_regression_libsvm_data.txt").cache() // Create label, feature, weight tuples from input data with weight set to default value 1.0. - val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - (parts(0), parts(1), 1.0) + val parsedData = data.map { labeledPoint => + (labeledPoint.label, labeledPoint.features(0), 1.0) } // Split data into training (60%) and test (40%) sets. From feaba979b30b27f661ae44ae3f12eabc3a6e55b3 Mon Sep 17 00:00:00 2001 From: Nezih Yigitbasi Date: Thu, 16 Jun 2016 18:19:29 -0700 Subject: [PATCH 0692/1470] [SPARK-15782][YARN] Fix spark.jars and spark.yarn.dist.jars handling When `--packages` is specified with spark-shell the classes from those packages cannot be found, which I think is due to some of the changes in SPARK-12343. Tested manually with both scala 2.10 and 2.11 repls. vanzin davies can you guys please review? Author: Marcelo Vanzin Author: Nezih Yigitbasi Closes #13709 from nezihyigitbasi/SPARK-15782. (cherry picked from commit 63470afc997fb9d6b6f8a911c25964743556c9cc) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 25 +++++++++++++++ .../spark/deploy/SparkSubmitSuite.scala | 12 +++++++ .../org/apache/spark/repl/SparkILoop.scala | 32 ++++++++++++------- .../scala/org/apache/spark/repl/Main.scala | 4 +-- 5 files changed, 59 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d56946e932caf..d8701812ebeda 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -391,7 +391,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - _jars = _conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + _jars = Utils.getUserJars(_conf) _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.nonEmpty)) .toSeq.flatten 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 f9d05409e1c3d..17d193b773fd6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2352,6 +2352,31 @@ private[spark] object Utils extends Logging { log.info(s"Started daemon with process name: ${Utils.getProcessName()}") SignalUtils.registerLogger(log) } + + /** + * Unions two comma-separated lists of files and filters out empty strings. + */ + def unionFileLists(leftList: Option[String], rightList: Option[String]): Set[String] = { + var allFiles = Set[String]() + leftList.foreach { value => allFiles ++= value.split(",") } + rightList.foreach { value => allFiles ++= value.split(",") } + allFiles.filter { _.nonEmpty } + } + + /** + * In YARN mode this method returns a union of the jar files pointed by "spark.jars" and the + * "spark.yarn.dist.jars" properties, while in other modes it returns the jar files pointed by + * only the "spark.jars" property. + */ + def getUserJars(conf: SparkConf): Seq[String] = { + val sparkJars = conf.getOption("spark.jars") + if (conf.get("spark.master") == "yarn") { + val yarnJars = conf.getOption("spark.yarn.dist.jars") + unionFileLists(sparkJars, yarnJars).toSeq + } else { + sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + } + } } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 271897699201b..0b020592b06d3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -570,6 +570,18 @@ class SparkSubmitSuite appArgs.executorMemory should be ("2.3g") } } + + test("comma separated list of files are unioned correctly") { + val left = Option("/tmp/a.jar,/tmp/b.jar") + val right = Option("/tmp/c.jar,/tmp/a.jar") + val emptyString = Option("") + Utils.unionFileLists(left, right) should be (Set("/tmp/a.jar", "/tmp/b.jar", "/tmp/c.jar")) + Utils.unionFileLists(emptyString, emptyString) should be (Set.empty) + Utils.unionFileLists(Option("/tmp/a.jar"), emptyString) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(emptyString, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(None, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(Option("/tmp/a.jar"), None) should be (Set("/tmp/a.jar")) + } // scalastyle:on println // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index b1e95d8fdb600..8fcab386ec8f9 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -201,10 +201,10 @@ class SparkILoop( if (Utils.isWindows) { // Strip any URI scheme prefix so we can add the correct path to the classpath // e.g. file:/C:/my/path.jar -> C:/my/path.jar - SparkILoop.getAddedJars.map { jar => new URI(jar).getPath.stripPrefix("/") } + getAddedJars().map { jar => new URI(jar).getPath.stripPrefix("/") } } else { // We need new URI(jar).getPath here for the case that `jar` includes encoded white space (%20). - SparkILoop.getAddedJars.map { jar => new URI(jar).getPath } + getAddedJars().map { jar => new URI(jar).getPath } } // work around for Scala bug val totalClassPath = addedJars.foldLeft( @@ -1005,7 +1005,7 @@ class SparkILoop( @DeveloperApi def createSparkSession(): SparkSession = { val execUri = System.getenv("SPARK_EXECUTOR_URI") - val jars = SparkILoop.getAddedJars + val jars = getAddedJars() val conf = new SparkConf() .setMaster(getMaster()) .setJars(jars) @@ -1060,22 +1060,30 @@ class SparkILoop( @deprecated("Use `process` instead", "2.9.0") private def main(settings: Settings): Unit = process(settings) -} -object SparkILoop extends Logging { - implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp - private def echo(msg: String) = Console println msg - - def getAddedJars: Array[String] = { + private[repl] def getAddedJars(): Array[String] = { + val conf = new SparkConf().setMaster(getMaster()) val envJars = sys.env.get("ADD_JARS") if (envJars.isDefined) { logWarning("ADD_JARS environment variable is deprecated, use --jar spark submit argument instead") } - val propJars = sys.props.get("spark.jars").flatMap { p => if (p == "") None else Some(p) } - val jars = propJars.orElse(envJars).getOrElse("") + val jars = { + val userJars = Utils.getUserJars(conf) + if (userJars.isEmpty) { + envJars.getOrElse("") + } else { + userJars.mkString(",") + } + } Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) } +} + +object SparkILoop extends Logging { + implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp + private def echo(msg: String) = Console println msg + // Designed primarily for use by test code: take a String with a // bunch of code, and prints out a transcript of what it would look // like if you'd just typed it into the repl. @@ -1109,7 +1117,7 @@ object SparkILoop extends Logging { if (settings.classpath.isDefault) settings.classpath.value = sys.props("java.class.path") - getAddedJars.map(jar => new URI(jar).getPath).foreach(settings.classpath.append(_)) + repl.getAddedJars().map(jar => new URI(jar).getPath).foreach(settings.classpath.append(_)) repl process settings } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 771670fa559a0..28fe84d6fe9bd 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -54,9 +54,7 @@ object Main extends Logging { // Visible for testing private[repl] def doMain(args: Array[String], _interp: SparkILoop): Unit = { interp = _interp - val jars = conf.getOption("spark.jars") - .map(_.replace(",", File.pathSeparator)) - .getOrElse("") + val jars = Utils.getUserJars(conf).mkString(File.pathSeparator) val interpArguments = List( "-Yrepl-class-based", "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", From 2127f99f2c8cf6d3f85e6408ce47b82e0c3cad4d Mon Sep 17 00:00:00 2001 From: Kai Jiang Date: Thu, 16 Jun 2016 19:39:33 -0700 Subject: [PATCH 0693/1470] [SPARK-15490][R][DOC] SparkR 2.0 QA: New R APIs and API docs for non-MLib changes ## What changes were proposed in this pull request? R Docs changes include typos, format, layout. ## How was this patch tested? Test locally. Author: Kai Jiang Closes #13394 from vectorijk/spark-15490. (cherry picked from commit 5fd20b66ffe18c05cf257af7f30d32464d2fe8e7) Signed-off-by: Joseph K. Bradley --- R/pkg/R/DataFrame.R | 91 +++++++++++++++++++++++++------------------- R/pkg/R/RDD.R | 14 ++++--- R/pkg/R/WindowSpec.R | 7 ++-- R/pkg/R/broadcast.R | 8 ++-- R/pkg/R/column.R | 6 ++- R/pkg/R/context.R | 41 ++++++++++---------- R/pkg/R/functions.R | 2 +- R/pkg/R/group.R | 6 ++- R/pkg/R/mllib.R | 34 +++++++++++------ R/pkg/R/utils.R | 2 + 10 files changed, 123 insertions(+), 88 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 9a9b3f7ecae16..d72cbbd79e817 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -23,9 +23,11 @@ NULL setOldClass("jobj") setOldClass("structType") -#' @title S4 class that represents a SparkDataFrame -#' @description DataFrames can be created using functions like \link{createDataFrame}, -#' \link{read.json}, \link{table} etc. +#' S4 class that represents a SparkDataFrame +#' +#' DataFrames can be created using functions like \link{createDataFrame}, +#' \link{read.json}, \link{table} etc. +#' #' @family SparkDataFrame functions #' @rdname SparkDataFrame #' @docType class @@ -629,8 +631,6 @@ setMethod("repartition", #' #' @param x A SparkDataFrame #' @return A StringRRDD of JSON objects -#' @family SparkDataFrame functions -#' @rdname tojson #' @noRd #' @examples #'\dontrun{ @@ -648,7 +648,7 @@ setMethod("toJSON", RDD(jrdd, serializedMode = "string") }) -#' write.json +#' Save the contents of SparkDataFrame as a JSON file #' #' Save the contents of a SparkDataFrame as a JSON file (one object per line). Files written out #' with this method can be read back in as a SparkDataFrame using read.json(). @@ -675,7 +675,7 @@ setMethod("write.json", invisible(callJMethod(write, "json", path)) }) -#' write.parquet +#' Save the contents of SparkDataFrame as a Parquet file, preserving the schema. #' #' Save the contents of a SparkDataFrame as a Parquet file, preserving the schema. Files written out #' with this method can be read back in as a SparkDataFrame using read.parquet(). @@ -713,9 +713,9 @@ setMethod("saveAsParquetFile", write.parquet(x, path) }) -#' write.text +#' Save the content of SparkDataFrame in a text file at the specified path. #' -#' Saves the content of the SparkDataFrame in a text file at the specified path. +#' Save the content of the SparkDataFrame in a text file at the specified path. #' The SparkDataFrame must have only one column of string type with the name "value". #' Each row becomes a new line in the output file. #' @@ -820,8 +820,6 @@ setMethod("sample_frac", sample(x, withReplacement, fraction, seed) }) -#' nrow -#' #' Returns the number of rows in a SparkDataFrame #' #' @param x A SparkDataFrame @@ -874,6 +872,8 @@ setMethod("ncol", length(columns(x)) }) +#' Returns the dimensions of SparkDataFrame +#' #' Returns the dimensions (number of rows and columns) of a SparkDataFrame #' @param x a SparkDataFrame #' @@ -2012,8 +2012,9 @@ setMethod("join", dataFrame(sdf) }) +#' Merges two data frames +#' #' @name merge -#' @title Merges two data frames #' @param x the first data frame to be joined #' @param y the second data frame to be joined #' @param by a character vector specifying the join columns. If by is not @@ -2127,7 +2128,6 @@ setMethod("merge", joinRes }) -#' #' Creates a list of columns by replacing the intersected ones with aliases. #' The name of the alias column is formed by concatanating the original column name and a suffix. #' @@ -2182,8 +2182,9 @@ setMethod("unionAll", dataFrame(unioned) }) -#' @title Union two or more SparkDataFrames -#' @description Returns a new SparkDataFrame containing rows of all parameters. +#' Union two or more SparkDataFrames +#' +#' Returns a new SparkDataFrame containing rows of all parameters. #' #' @rdname rbind #' @name rbind @@ -2254,20 +2255,22 @@ setMethod("except", dataFrame(excepted) }) -#' Save the contents of the SparkDataFrame to a data source +#' Save the contents of SparkDataFrame 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: \cr -#' append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr -#' overwrite: Existing data is expected to be overwritten by the contents of this -#' SparkDataFrame. \cr -#' error: An exception is expected to be thrown. \cr -#' ignore: The save operation is expected to not save the contents of the SparkDataFrame -#' and to not change the existing data. \cr +#' Additionally, mode is used to specify the behavior of the save operation when data already +#' exists in the data source. There are four modes: +#' \itemize{ +#' \item append: Contents of this SparkDataFrame are expected to be appended to existing data. +#' \item overwrite: Existing data is expected to be overwritten by the contents of this +#' SparkDataFrame. +#' \item error: An exception is expected to be thrown. +#' \item ignore: The save operation is expected to not save the contents of the SparkDataFrame +#' and to not change the existing data. +#' } #' #' @param df A SparkDataFrame #' @param path A name for the table @@ -2315,8 +2318,6 @@ setMethod("saveDF", write.df(df, path, source, mode, ...) }) -#' saveAsTable -#' #' Save the contents of the SparkDataFrame to a data source as a table #' #' The data source is specified by the `source` and a set of options (...). @@ -2543,11 +2544,12 @@ setMethod("fillna", dataFrame(sdf) }) +#' Download data from a SparkDataFrame into a data.frame +#' #' This function downloads the contents of a SparkDataFrame into an R's data.frame. #' Since data.frames are held in memory, ensure that you have enough memory #' in your system to accommodate the contents. #' -#' @title Download data from a SparkDataFrame into a data.frame #' @param x a SparkDataFrame #' @return a data.frame #' @family SparkDataFrame functions @@ -2563,13 +2565,14 @@ setMethod("as.data.frame", as.data.frame(collect(x), row.names, optional, ...) }) +#' Attach SparkDataFrame to R search path +#' #' The specified SparkDataFrame is attached to the R search path. This means that #' the SparkDataFrame is searched by R when evaluating a variable, so columns in #' the SparkDataFrame can be accessed by simply giving their names. #' #' @family SparkDataFrame functions #' @rdname attach -#' @title Attach SparkDataFrame to R search path #' @param what (SparkDataFrame) The SparkDataFrame to attach #' @param pos (integer) Specify position in search() where to attach. #' @param name (character) Name to use for the attached SparkDataFrame. Names @@ -2589,6 +2592,8 @@ setMethod("attach", attach(newEnv, pos = pos, name = name, warn.conflicts = warn.conflicts) }) +#' Evaluate a R expression in an environment constructed from a SparkDataFrame +#' #' Evaluate a R expression in an environment constructed from a SparkDataFrame #' with() allows access to columns of a SparkDataFrame by simply referring to #' their name. It appends every column of a SparkDataFrame into a new @@ -2596,7 +2601,7 @@ setMethod("attach", #' environment. #' #' @rdname with -#' @title Evaluate a R expression in an environment constructed from a SparkDataFrame +#' @family SparkDataFrame functions #' @param data (SparkDataFrame) SparkDataFrame to use for constructing an environment. #' @param expr (expression) Expression to evaluate. #' @param ... arguments to be passed to future methods. @@ -2612,10 +2617,12 @@ setMethod("with", eval(substitute(expr), envir = newEnv, enclos = newEnv) }) +#' Compactly display the structure of a dataset +#' #' Display the structure of a SparkDataFrame, including column names, column types, as well as a #' a small sample of rows. +#' #' @name str -#' @title Compactly display the structure of a dataset #' @rdname str #' @family SparkDataFrame functions #' @param object a SparkDataFrame @@ -2728,10 +2735,11 @@ setMethod("drop", base::drop(x) }) +#' Compute histogram statistics for given column +#' #' This function computes a histogram for a given SparkR Column. #' #' @name histogram -#' @title Histogram #' @param nbins the number of bins (optional). Default value is 10. #' @param df the SparkDataFrame containing the Column to build the histogram from. #' @param colname the name of the column to build the histogram from. @@ -2847,18 +2855,21 @@ setMethod("histogram", return(histStats) }) -#' Saves the content of the SparkDataFrame to an external database table via JDBC +#' Save the content of SparkDataFrame to an external database table via JDBC. #' -#' Additional JDBC database connection properties can be set (...) +#' Save the content of the SparkDataFrame to an external database table via JDBC. Additional JDBC +#' database connection properties can be set (...) #' #' Also, mode is used to specify the behavior of the save operation when -#' data already exists in the data source. There are four modes: \cr -#' append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr -#' overwrite: Existing data is expected to be overwritten by the contents of this -#' SparkDataFrame. \cr -#' error: An exception is expected to be thrown. \cr -#' ignore: The save operation is expected to not save the contents of the SparkDataFrame -#' and to not change the existing data. \cr +#' data already exists in the data source. There are four modes: +#' \itemize{ +#' \item append: Contents of this SparkDataFrame are expected to be appended to existing data. +#' \item overwrite: Existing data is expected to be overwritten by the contents of this +#' SparkDataFrame. +#' \item error: An exception is expected to be thrown. +#' \item ignore: The save operation is expected to not save the contents of the SparkDataFrame +#' and to not change the existing data. +#' } #' #' @param x A SparkDataFrame #' @param url JDBC database url of the form `jdbc:subprotocol:subname` diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index f1badf4364da0..72a805256523e 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -19,9 +19,11 @@ setOldClass("jobj") -#' @title S4 class that represents an RDD -#' @description RDD can be created using functions like +#' S4 class that represents an RDD +#' +#' 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 @@ -497,9 +499,9 @@ setMethod("map", lapply(X, FUN) }) -#' Flatten results after apply a function to all elements +#' Flatten results after applying a function to all elements #' -#' This function return a new RDD by first applying a function to all +#' This function returns 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. @@ -713,7 +715,7 @@ setMethod("sumRDD", reduce(x, "+") }) -#' Applies a function to all elements in an RDD, and force evaluation. +#' Applies a function to all elements in an RDD, and forces evaluation. #' #' @param x The RDD to apply the function #' @param func The function to be applied. @@ -737,7 +739,7 @@ setMethod("foreach", invisible(collect(mapPartitions(x, partition.func))) }) -#' Applies a function to each partition in an RDD, and force evaluation. +#' Applies a function to each partition in an RDD, and forces evaluation. #' #' @examples #'\dontrun{ diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index 581176a6c0918..d8405420d0a49 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -20,9 +20,10 @@ #' @include generics.R jobj.R column.R NULL -#' @title S4 class that represents a WindowSpec -#' @description WindowSpec can be created by using window.partitionBy() -#' or window.orderBy() +#' S4 class that represents a WindowSpec +#' +#' WindowSpec can be created by using window.partitionBy() or window.orderBy() +#' #' @rdname WindowSpec #' @seealso \link{window.partitionBy}, \link{window.orderBy} #' diff --git a/R/pkg/R/broadcast.R b/R/pkg/R/broadcast.R index 38f0eed95e065..398dffc4ab1b4 100644 --- a/R/pkg/R/broadcast.R +++ b/R/pkg/R/broadcast.R @@ -23,9 +23,11 @@ .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}. +# S4 class that represents a Broadcast variable +# +# Broadcast variables can be created using the broadcast +# function from a \code{SparkContext}. +# # @rdname broadcast-class # @seealso broadcast # diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 873e8b1665a28..cc2876ed94b7f 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -22,8 +22,10 @@ NULL setOldClass("jobj") -#' @title S4 class that represents a SparkDataFrame column -#' @description The column class supports unary, binary operations on SparkDataFrame columns +#' S4 class that represents a SparkDataFrame column +#' +#' The column class supports unary, binary operations on SparkDataFrame columns +#' #' @rdname column #' #' @slot jc reference to JVM SparkDataFrame column diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 44bca877fd45a..5c886030ff5c5 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -173,9 +173,8 @@ includePackage <- function(sc, pkg) { .sparkREnv$.packages <- packages } -#' @title Broadcast a variable to all workers +#' 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. #' @@ -207,7 +206,7 @@ broadcast <- function(sc, object) { Broadcast(id, object, jBroadcast, objName) } -#' @title Set the checkpoint directory +#' 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. @@ -226,30 +225,31 @@ setCheckpointDir <- function(sc, dirName) { invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) } -#' @title Run a function over a list of elements, distributing the computations with Spark. +#' Run a function over a list of elements, distributing the computations with Spark. #' -#' @description #' Applies a function in a manner that is similar to doParallel or lapply to elements of a list. #' The computations are distributed using Spark. It is conceptually the same as the following code: #' lapply(list, func) #' #' Known limitations: -#' - variable scoping and capture: compared to R's rich support for variable resolutions, the -# distributed nature of SparkR limits how variables are resolved at runtime. All the variables -# that are available through lexical scoping are embedded in the closure of the function and -# available as read-only variables within the function. The environment variables should be -# stored into temporary variables outside the function, and not directly accessed within the -# function. +#' \itemize{ +#' \item variable scoping and capture: compared to R's rich support for variable resolutions, +#' the distributed nature of SparkR limits how variables are resolved at runtime. All the +#' variables that are available through lexical scoping are embedded in the closure of the +#' function and available as read-only variables within the function. The environment variables +#' should be stored into temporary variables outside the function, and not directly accessed +#' within the function. #' -#' - loading external packages: In order to use a package, you need to load it inside the -#' closure. For example, if you rely on the MASS module, here is how you would use it: -#'\dontrun{ -#' train <- function(hyperparam) { -#' library(MASS) -#' lm.ridge(“y ~ x+z”, data, lambda=hyperparam) -#' model +#' \item loading external packages: In order to use a package, you need to load it inside the +#' closure. For example, if you rely on the MASS module, here is how you would use it: +#' \preformatted{ +#' train <- function(hyperparam) { +#' library(MASS) +#' lm.ridge(“y ~ x+z”, data, lambda=hyperparam) +#' model +#' } +#' } #' } -#'} #' #' @rdname spark.lapply #' @param sc Spark Context to use @@ -259,7 +259,8 @@ setCheckpointDir <- function(sc, dirName) { #' @export #' @examples #'\dontrun{ -#' doubled <- spark.lapply(1:10, function(x){2 * x}) +#' sc <- sparkR.init() +#' doubled <- spark.lapply(sc, 1:10, function(x){2 * x}) #'} spark.lapply <- function(sc, list, func) { rdd <- parallelize(sc, list, length(list)) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 2665d1d477802..a779127b379a0 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2185,7 +2185,7 @@ setMethod("from_unixtime", signature(x = "Column"), #' # 09:01:15-09:02:15... #' window(df$time, "1 minute", startTime = "15 seconds") #' -#' # Thirty second windows every 10 seconds, e.g. 09:00:00-09:00:30, 09:00:10-09:00:40, ... +#' # Thirty-second windows every 10 seconds, e.g. 09:00:00-09:00:30, 09:00:10-09:00:40, ... #' window(df$time, "30 seconds", "10 seconds") #'} setMethod("window", signature(x = "Column"), diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index b7047769175a3..eba083fe4b124 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -22,8 +22,10 @@ NULL setOldClass("jobj") -#' @title S4 class that represents a GroupedData -#' @description GroupedDatas can be created using groupBy() on a SparkDataFrame +#' S4 class that represents a GroupedData +#' +#' GroupedDatas can be created using groupBy() on a SparkDataFrame +#' #' @rdname GroupedData #' @seealso groupBy #' diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index d4152b43b6f5f..ba2eee2fca76a 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -25,22 +25,26 @@ # - a set of methods that reflect the arguments of the other languages supported by Spark. These # methods are prefixed with the `spark.` prefix: spark.glm, spark.kmeans, etc. -#' @title S4 class that represents a generalized linear model +#' S4 class that represents a generalized linear model +#' #' @param jobj a Java object reference to the backing Scala GeneralizedLinearRegressionWrapper #' @export setClass("GeneralizedLinearRegressionModel", representation(jobj = "jobj")) -#' @title S4 class that represents a NaiveBayesModel +#' S4 class that represents a NaiveBayesModel +#' #' @param jobj a Java object reference to the backing Scala NaiveBayesWrapper #' @export setClass("NaiveBayesModel", representation(jobj = "jobj")) -#' @title S4 class that represents a AFTSurvivalRegressionModel +#' S4 class that represents a AFTSurvivalRegressionModel +#' #' @param jobj a Java object reference to the backing Scala AFTSurvivalRegressionWrapper #' @export setClass("AFTSurvivalRegressionModel", representation(jobj = "jobj")) -#' @title S4 class that represents a KMeansModel +#' S4 class that represents a KMeansModel +#' #' @param jobj a Java object reference to the backing Scala KMeansModel #' @export setClass("KMeansModel", representation(jobj = "jobj")) @@ -197,7 +201,7 @@ print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { invisible(x) } -#' Make predictions from a generalized linear model +#' Predicted values based on model #' #' Makes predictions from a generalized linear model produced by glm() or spark.glm(), #' similarly to R's predict(). @@ -218,9 +222,9 @@ setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) }) -#' Make predictions from a naive Bayes model +#' Predicted values based on model #' -#' Makes predictions from a model produced by spark.naiveBayes(), +#' Makes predictions from a naive Bayes model or a model produced by spark.naiveBayes(), #' similarly to R package e1071's predict. #' #' @param object A fitted naive Bayes model @@ -357,9 +361,9 @@ setMethod("summary", signature(object = "KMeansModel"), cluster = cluster, is.loaded = is.loaded)) }) -#' Make predictions from a k-means model +#' Predicted values based on model #' -#' Make predictions from a model produced by spark.kmeans(). +#' Makes predictions from a k-means model or a model produced by spark.kmeans(). #' #' @param object A fitted k-means model #' @param newData SparkDataFrame for testing @@ -402,6 +406,8 @@ setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "form return(new("NaiveBayesModel", jobj = jobj)) }) +#' Save fitted MLlib model to the input path +#' #' Save the Bernoulli naive Bayes model to the input path. #' #' @param object A fitted Bernoulli naive Bayes model @@ -428,6 +434,8 @@ setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), invisible(callJMethod(writer, "save", path)) }) +#' Save fitted MLlib model to the input path +#' #' Save the AFT survival regression model to the input path. #' #' @param object A fitted AFT survival regression model @@ -453,6 +461,8 @@ setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "c invisible(callJMethod(writer, "save", path)) }) +#' Save fitted MLlib model to the input path +#' #' Save the generalized linear model to the input path. #' #' @param object A fitted generalized linear model @@ -478,6 +488,8 @@ setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", pat invisible(callJMethod(writer, "save", path)) }) +#' Save fitted MLlib model to the input path +#' #' Save the k-means model to the input path. #' #' @param object A fitted k-means model @@ -582,9 +594,9 @@ setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), return(list(coefficients = coefficients)) }) -#' Make predictions from an AFT survival regression model +#' Predicted values based on model #' -#' Make predictions from a model produced by spark.survreg(), +#' Makes predictions from an AFT survival regression model or a model produced by spark.survreg(), #' similarly to R package survival's predict. #' #' @param object A fitted AFT survival regression model diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 12e4f4f1ae8bb..b1b8adaa66a25 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -110,9 +110,11 @@ isRDD <- function(name, env) { #' @return the hash code as an integer #' @export #' @examples +#'\dontrun{ #' hashCode(1L) # 1 #' hashCode(1.0) # 1072693248 #' hashCode("1") # 49 +#'} hashCode <- function(key) { if (class(key) == "integer") { as.integer(key[[1]]) From f530331e6f8160f3fb2613722fae01ea589f0e99 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 16 Jun 2016 20:35:17 -0700 Subject: [PATCH 0694/1470] [SPARK-15908][R] Add varargs-type dropDuplicates() function in SparkR ## What changes were proposed in this pull request? This PR adds varargs-type `dropDuplicates` function to SparkR for API parity. Refer to https://issues.apache.org/jira/browse/SPARK-15807, too. ## How was this patch tested? Pass the Jenkins tests with new testcases. Author: Dongjoon Hyun Closes #13684 from dongjoon-hyun/SPARK-15908. (cherry picked from commit 513a03e41e27d9c5f70911faccc5d3aecd8bdde9) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 25 +++++++++++++++++------ R/pkg/R/generics.R | 7 ++----- R/pkg/inst/tests/testthat/test_sparkSQL.R | 8 ++++++++ 3 files changed, 29 insertions(+), 11 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index d72cbbd79e817..c710bffa2c407 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1936,10 +1936,11 @@ setMethod("where", #' the subset of columns. #' #' @param x A SparkDataFrame. -#' @param colnames A character vector of column names. +#' @param ... A character vector of column names or string column names. +#' If the first argument contains a character vector, the followings are ignored. #' @return A SparkDataFrame with duplicate rows removed. #' @family SparkDataFrame functions -#' @rdname dropduplicates +#' @rdname dropDuplicates #' @name dropDuplicates #' @export #' @examples @@ -1949,14 +1950,26 @@ setMethod("where", #' path <- "path/to/file.json" #' df <- read.json(path) #' dropDuplicates(df) +#' dropDuplicates(df, "col1", "col2") #' dropDuplicates(df, c("col1", "col2")) #' } setMethod("dropDuplicates", signature(x = "SparkDataFrame"), - function(x, colNames = columns(x)) { - stopifnot(class(colNames) == "character") - - sdf <- callJMethod(x@sdf, "dropDuplicates", as.list(colNames)) + function(x, ...) { + cols <- list(...) + if (length(cols) == 0) { + sdf <- callJMethod(x@sdf, "dropDuplicates", as.list(columns(x))) + } else { + if (!all(sapply(cols, function(c) { is.character(c) }))) { + stop("all columns names should be characters") + } + col <- cols[[1]] + if (length(col) > 1) { + sdf <- callJMethod(x@sdf, "dropDuplicates", as.list(col)) + } else { + sdf <- callJMethod(x@sdf, "dropDuplicates", cols) + } + } dataFrame(sdf) }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 40a96d8991a5a..8164e7731a15f 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -466,12 +466,9 @@ setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) #' @export setGeneric("drop", function(x, ...) { standardGeneric("drop") }) -#' @rdname dropduplicates +#' @rdname dropDuplicates #' @export -setGeneric("dropDuplicates", - function(x, colNames = columns(x)) { - standardGeneric("dropDuplicates") - }) +setGeneric("dropDuplicates", function(x, ...) { standardGeneric("dropDuplicates") }) #' @rdname nafunctions #' @export diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index c11930ada63ce..11d69366df320 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -796,6 +796,14 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { result[order(result$key, result$value1, result$value2), ], expected) + result <- collect(dropDuplicates(df, "key", "value1")) + expected <- rbind.data.frame( + c(1, 1, 1), c(1, 2, 1), c(2, 1, 2), c(2, 2, 2)) + names(expected) <- c("key", "value1", "value2") + expect_equivalent( + result[order(result$key, result$value1, result$value2), ], + expected) + result <- collect(dropDuplicates(df, "key")) expected <- rbind.data.frame( c(1, 1, 1), c(2, 1, 2)) From 3994372f48eefe080ce7a80750ccf960e3a7968b Mon Sep 17 00:00:00 2001 From: Pete Robbins Date: Thu, 16 Jun 2016 22:27:32 -0700 Subject: [PATCH 0695/1470] [SPARK-15822] [SQL] Prevent byte array backed classes from referencing freed memory ## What changes were proposed in this pull request? `UTF8String` and all `Unsafe*` classes are backed by either on-heap or off-heap byte arrays. The code generated version `SortMergeJoin` buffers the left hand side join keys during iteration. This was actually problematic in off-heap mode when one of the keys is a `UTF8String` (or any other 'Unsafe*` object) and the left hand side iterator was exhausted (and released its memory); the buffered keys would reference freed memory. This causes Seg-faults and all kinds of other undefined behavior when we would use one these buffered keys. This PR fixes this problem by creating copies of the buffered variables. I have added a general method to the `CodeGenerator` for this. I have checked all places in which this could happen, and only `SortMergeJoin` had this problem. This PR is largely based on the work of robbinspg and he should be credited for this. closes https://github.com/apache/spark/pull/13707 ## How was this patch tested? Manually tested on problematic workloads. Author: Pete Robbins Author: Herman van Hovell Closes #13723 from hvanhovell/SPARK-15822-2. (cherry picked from commit 5ada606144c7bf38a797764619d7d1ff677802b3) Signed-off-by: Davies Liu --- .../expressions/codegen/CodeGenerator.scala | 16 ++++++++++++++++ .../sql/execution/joins/SortMergeJoinExec.scala | 8 +------- 2 files changed, 17 insertions(+), 7 deletions(-) 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 ff97cd321199a..6392ff42d7099 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 @@ -130,6 +130,22 @@ class CodegenContext { mutableStates += ((javaType, variableName, initCode)) } + /** + * Add buffer variable which stores data coming from an [[InternalRow]]. This methods guarantees + * that the variable is safely stored, which is important for (potentially) byte array backed + * data types like: UTF8String, ArrayData, MapData & InternalRow. + */ + def addBufferedState(dataType: DataType, variableName: String, initCode: String): ExprCode = { + val value = freshName(variableName) + addMutableState(javaType(dataType), value, "") + val code = dataType match { + case StringType => s"$value = $initCode.clone();" + case _: StructType | _: ArrayType | _: MapType => s"$value = $initCode.copy();" + case _ => s"$value = $initCode;" + } + ExprCode(code, "false", value) + } + def declareMutableStates(): String = { // It's possible that we add same mutable state twice, e.g. the `mergeExpressions` in // `TypedAggregateExpression`, we should call `distinct` here to remove the duplicated ones. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 32f0bc5bf920d..fac6b8de8ed5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -336,13 +336,7 @@ case class SortMergeJoinExec( private def copyKeys(ctx: CodegenContext, vars: Seq[ExprCode]): Seq[ExprCode] = { vars.zipWithIndex.map { case (ev, i) => - val value = ctx.freshName("value") - ctx.addMutableState(ctx.javaType(leftKeys(i).dataType), value, "") - val code = - s""" - |$value = ${ev.value}; - """.stripMargin - ExprCode(code, "false", value) + ctx.addBufferedState(leftKeys(i).dataType, "value", ev.value) } } From b82abde060d97bd95f4fba547545a830602a35fa Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 16 Jun 2016 22:54:02 -0700 Subject: [PATCH 0696/1470] [SPARK-15706][SQL] Fix Wrong Answer when using IF NOT EXISTS in INSERT OVERWRITE for DYNAMIC PARTITION #### What changes were proposed in this pull request? `IF NOT EXISTS` in `INSERT OVERWRITE` should not support dynamic partitions. If we specify `IF NOT EXISTS`, the inserted statement is not shown in the table. This PR is to issue an exception in this case, just like what Hive does. Also issue an exception if users specify `IF NOT EXISTS` if users do not specify any `PARTITION` specification. #### How was this patch tested? Added test cases into `PlanParserSuite` and `InsertIntoHiveTableSuite` Author: gatorsmile Closes #13447 from gatorsmile/insertIfNotExist. (cherry picked from commit e5d703bca85c65ce329b1e202283cfa35d109146) Signed-off-by: Yin Huai --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 6 ++ .../plans/logical/basicLogicalOperators.scala | 1 + .../sql/catalyst/parser/PlanParserSuite.scala | 13 ++-- .../sql/hive/InsertIntoHiveTableSuite.scala | 68 +++++++++++++++++++ 5 files changed, 85 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index b60319668c5b6..23e925e4de63c 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -203,7 +203,7 @@ query ; insertInto - : INSERT OVERWRITE TABLE tableIdentifier partitionSpec? (IF NOT EXISTS)? + : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)? | INSERT INTO TABLE? tableIdentifier partitionSpec? ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index e380643f548ba..c7420a1c5965d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -171,6 +171,12 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val tableIdent = visitTableIdentifier(ctx.tableIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) + val dynamicPartitionKeys = partitionKeys.filter(_._2.isEmpty) + if (ctx.EXISTS != null && dynamicPartitionKeys.nonEmpty) { + throw new ParseException(s"Dynamic partitions do not support IF NOT EXISTS. Specified " + + "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx) + } + InsertIntoTable( UnresolvedRelation(tableIdent, None), partitionKeys, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 898784dab1d98..6c3eb3a5a28ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -377,6 +377,7 @@ case class InsertIntoTable( } assert(overwrite || !ifNotExists) + assert(partition.values.forall(_.nonEmpty) || !ifNotExists) override lazy val resolved: Boolean = childrenResolved && table.resolved && expectedColumns.forall { expected => child.output.size == expected.size && child.output.zip(expected).forall { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 77023cfd3d60f..456948d6455c9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -183,14 +183,12 @@ class PlanParserSuite extends PlanTest { // Single inserts assertEqual(s"insert overwrite table s $sql", insert(Map.empty, overwrite = true)) - assertEqual(s"insert overwrite table s if not exists $sql", - insert(Map.empty, overwrite = true, ifNotExists = true)) + assertEqual(s"insert overwrite table s partition (e = 1) if not exists $sql", + insert(Map("e" -> Option("1")), overwrite = true, ifNotExists = true)) assertEqual(s"insert into s $sql", insert(Map.empty)) assertEqual(s"insert into table s partition (c = 'd', e = 1) $sql", insert(Map("c" -> Option("d"), "e" -> Option("1")))) - assertEqual(s"insert overwrite table s partition (c = 'd', x) if not exists $sql", - insert(Map("c" -> Option("d"), "x" -> None), overwrite = true, ifNotExists = true)) // Multi insert val plan2 = table("t").where('x > 5).select(star()) @@ -201,6 +199,13 @@ class PlanParserSuite extends PlanTest { table("u"), Map.empty, plan2, overwrite = false, ifNotExists = false))) } + test ("insert with if not exists") { + val sql = "select * from t" + intercept(s"insert overwrite table s partition (e = 1, x) if not exists $sql", + "Dynamic partitions do not support IF NOT EXISTS. Specified partitions with value: [x]") + intercept[ParseException](parsePlan(s"insert overwrite table s if not exists $sql")) + } + test("aggregation") { val sql = "select a, b, sum(c) as c from d group by a, b" 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 fae59001b98e1..3bf45ced75e0a 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 @@ -166,6 +166,74 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef sql("DROP TABLE tmp_table") } + test("INSERT OVERWRITE - partition IF NOT EXISTS") { + withTempDir { tmpDir => + val table = "table_with_partition" + withTable(table) { + val selQuery = s"select c1, p1, p2 from $table" + sql( + s""" + |CREATE TABLE $table(c1 string) + |PARTITIONED by (p1 string,p2 string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + s""" + |INSERT OVERWRITE TABLE $table + |partition (p1='a',p2='b') + |SELECT 'blarr' + """.stripMargin) + checkAnswer( + sql(selQuery), + Row("blarr", "a", "b")) + + sql( + s""" + |INSERT OVERWRITE TABLE $table + |partition (p1='a',p2='b') + |SELECT 'blarr2' + """.stripMargin) + checkAnswer( + sql(selQuery), + Row("blarr2", "a", "b")) + + var e = intercept[AnalysisException] { + sql( + s""" + |INSERT OVERWRITE TABLE $table + |partition (p1='a',p2) IF NOT EXISTS + |SELECT 'blarr3', 'newPartition' + """.stripMargin) + } + assert(e.getMessage.contains( + "Dynamic partitions do not support IF NOT EXISTS. Specified partitions with value: [p2]")) + + e = intercept[AnalysisException] { + sql( + s""" + |INSERT OVERWRITE TABLE $table + |partition (p1='a',p2) IF NOT EXISTS + |SELECT 'blarr3', 'b' + """.stripMargin) + } + assert(e.getMessage.contains( + "Dynamic partitions do not support IF NOT EXISTS. Specified partitions with value: [p2]")) + + // If the partition already exists, the insert will overwrite the data + // unless users specify IF NOT EXISTS + sql( + s""" + |INSERT OVERWRITE TABLE $table + |partition (p1='a',p2='b') IF NOT EXISTS + |SELECT 'blarr3' + """.stripMargin) + checkAnswer( + sql(selQuery), + Row("blarr2", "a", "b")) + } + } + } + test("Insert ArrayType.containsNull == false") { val schema = StructType(Seq( StructField("a", ArrayType(StringType, containsNull = false)))) From 3c3865d0b42413435f8cc9044da842cfb4458afc Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 17 Jun 2016 00:34:33 -0700 Subject: [PATCH 0697/1470] [SPARK-15811][SQL] fix the Python UDF in Scala 2.10 ## What changes were proposed in this pull request? Iterator can't be serialized in Scala 2.10, we should force it into a array to make sure that . ## How was this patch tested? Build with Scala 2.10 and ran all the Python unit tests manually (will be covered by a jenkins build). Author: Davies Liu Closes #13717 from davies/fix_udf_210. (cherry picked from commit ef43b4ed87894982678fcc6f2c61cf1487ee9e14) Signed-off-by: Reynold Xin --- .../apache/spark/sql/execution/python/ExtractPythonUDFs.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index 668470ee6a29a..87583c82347eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -132,7 +132,7 @@ private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { val validUdfs = udfs.filter { case udf => // Check to make sure that the UDF can be evaluated with only the input of this child. udf.references.subsetOf(child.outputSet) - } + }.toArray // Turn it into an array since iterators cannot be serialized in Scala 2.10 if (validUdfs.nonEmpty) { val resultAttrs = udfs.zipWithIndex.map { case (u, i) => AttributeReference(s"pythonUDF$i", u.dataType)() From de964e4194cac1ad9ed4ca7f16dbb809fc7c9471 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 17 Jun 2016 09:47:41 -0700 Subject: [PATCH 0698/1470] Remove non-obvious conf settings from TPCDS benchmark ## What changes were proposed in this pull request? My fault -- these 2 conf entries are mysteriously hidden inside the benchmark code and makes it non-obvious to disable whole stage codegen and/or the vectorized parquet reader. PS: Didn't attach a JIRA as this change should otherwise be a no-op (both these conf are enabled by default in Spark) ## How was this patch tested? N/A Author: Sameer Agarwal Closes #13726 from sameeragarwal/tpcds-conf. (cherry picked from commit 34d6c4cd113729fcc1d0bc1df8916d06b8854922) Signed-off-by: Herman van Hovell --- .../spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala index 398d8d98be9b3..957a1d6426e87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -64,8 +64,6 @@ object TPCDSQueryBenchmark { require(dataLocation.nonEmpty, "please modify the value of dataLocation to point to your local TPCDS data") val tableSizes = setupTables(dataLocation) - spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") queries.foreach { name => val queryString = fileToString(new File(Thread.currentThread().getContextClassLoader .getResource(s"tpcds/$name.sql").getFile)) From 269b715e478fea7cb5ec6237d6439c6642ebc5e6 Mon Sep 17 00:00:00 2001 From: sethah Date: Fri, 17 Jun 2016 09:58:49 -0700 Subject: [PATCH 0699/1470] [SPARK-16008][ML] Remove unnecessary serialization in logistic regression JIRA: [SPARK-16008](https://issues.apache.org/jira/browse/SPARK-16008) ## What changes were proposed in this pull request? `LogisticAggregator` stores references to two arrays of dimension `numFeatures` which are serialized before the combine op, unnecessarily. This results in the shuffle write being ~3x (for multiclass logistic regression, this number will go up) larger than it should be (in MLlib, for instance, it is 3x smaller). This patch modifies `LogisticAggregator.add` to accept the two arrays as method parameters which avoids the serialization. ## How was this patch tested? I tested this locally and verified the serialization reduction. ![image](https://cloud.githubusercontent.com/assets/7275795/16140387/d2974bac-3404-11e6-94f9-268860c931a2.png) Additionally, I ran some tests of a 4 node cluster (4x48 cores, 4x128 GB RAM). Data set size of 2M rows and 10k features showed >2x iteration speedup. Author: sethah Closes #13729 from sethah/lr_improvement. (cherry picked from commit 1f0a46958ef51a01560ada23665dccde89696e12) Signed-off-by: Xiangrui Meng --- .../classification/LogisticRegression.scala | 57 ++++++++++--------- 1 file changed, 29 insertions(+), 28 deletions(-) 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 51ede15d6c367..9469acf62e13d 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 @@ -937,50 +937,47 @@ class BinaryLogisticRegressionSummary private[classification] ( * Two LogisticAggregator can be merged together to have a summary of loss and gradient of * the corresponding joint dataset. * - * @param coefficients The coefficients corresponding to the features. * @param numClasses the number of possible outcomes for k classes classification problem in * Multinomial Logistic Regression. * @param fitIntercept Whether to fit an intercept term. - * @param featuresStd The standard deviation values of the features. - * @param featuresMean The mean values of the features. */ private class LogisticAggregator( - coefficients: Vector, + private val numFeatures: Int, numClasses: Int, - fitIntercept: Boolean, - featuresStd: Array[Double], - featuresMean: Array[Double]) extends Serializable { + fitIntercept: Boolean) extends Serializable { private var weightSum = 0.0 private var lossSum = 0.0 - private val coefficientsArray = coefficients match { - case dv: DenseVector => dv.values - case _ => - throw new IllegalArgumentException( - s"coefficients only supports dense vector but got type ${coefficients.getClass}.") - } - - private val dim = if (fitIntercept) coefficientsArray.length - 1 else coefficientsArray.length - - private val gradientSumArray = Array.ofDim[Double](coefficientsArray.length) + private val gradientSumArray = + Array.ofDim[Double](if (fitIntercept) numFeatures + 1 else numFeatures) /** * Add a new training instance to this LogisticAggregator, and update the loss and gradient * of the objective function. * * @param instance The instance of data point to be added. + * @param coefficients The coefficients corresponding to the features. + * @param featuresStd The standard deviation values of the features. * @return This LogisticAggregator object. */ - def add(instance: Instance): this.type = { + def add( + instance: Instance, + coefficients: Vector, + featuresStd: Array[Double]): this.type = { instance match { case Instance(label, weight, features) => - require(dim == features.size, s"Dimensions mismatch when adding new instance." + - s" Expecting $dim but got ${features.size}.") + require(numFeatures == features.size, s"Dimensions mismatch when adding new instance." + + s" Expecting $numFeatures but got ${features.size}.") require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") if (weight == 0.0) return this - val localCoefficientsArray = coefficientsArray + val coefficientsArray = coefficients match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"coefficients only supports dense vector but got type ${coefficients.getClass}.") + } val localGradientSumArray = gradientSumArray numClasses match { @@ -990,11 +987,11 @@ private class LogisticAggregator( var sum = 0.0 features.foreachActive { (index, value) => if (featuresStd(index) != 0.0 && value != 0.0) { - sum += localCoefficientsArray(index) * (value / featuresStd(index)) + sum += coefficientsArray(index) * (value / featuresStd(index)) } } sum + { - if (fitIntercept) localCoefficientsArray(dim) else 0.0 + if (fitIntercept) coefficientsArray(numFeatures) else 0.0 } } @@ -1007,7 +1004,7 @@ private class LogisticAggregator( } if (fitIntercept) { - localGradientSumArray(dim) += multiplier + localGradientSumArray(numFeatures) += multiplier } if (label > 0) { @@ -1034,8 +1031,8 @@ private class LogisticAggregator( * @return This LogisticAggregator object. */ def merge(other: LogisticAggregator): this.type = { - require(dim == other.dim, s"Dimensions mismatch when merging with another " + - s"LeastSquaresAggregator. Expecting $dim but got ${other.dim}.") + require(numFeatures == other.numFeatures, s"Dimensions mismatch when merging with another " + + s"LeastSquaresAggregator. Expecting $numFeatures but got ${other.numFeatures}.") if (other.weightSum != 0.0) { weightSum += other.weightSum @@ -1086,13 +1083,17 @@ private class LogisticCostFun( override def calculate(coefficients: BDV[Double]): (Double, BDV[Double]) = { val numFeatures = featuresStd.length val coeffs = Vectors.fromBreeze(coefficients) + val n = coeffs.size + val localFeaturesStd = featuresStd + val logisticAggregator = { - val seqOp = (c: LogisticAggregator, instance: Instance) => c.add(instance) + val seqOp = (c: LogisticAggregator, instance: Instance) => + c.add(instance, coeffs, localFeaturesStd) val combOp = (c1: LogisticAggregator, c2: LogisticAggregator) => c1.merge(c2) instances.treeAggregate( - new LogisticAggregator(coeffs, numClasses, fitIntercept, featuresStd, featuresMean) + new LogisticAggregator(numFeatures, numClasses, fitIntercept) )(seqOp, combOp) } From 3457497e0257cd2ce8d87cccf258a5abe2339eb9 Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Fri, 17 Jun 2016 15:44:33 -0500 Subject: [PATCH 0700/1470] [SPARK-16018][SHUFFLE] Shade netty to load shuffle jar in Nodemanger ## What changes were proposed in this pull request? Shade the netty.io namespace so that we can use it in shuffle independent of the dependencies being pulled by hadoop jars. ## How was this patch tested? Ran a decent job involving shuffle write/read and tested the new spark-x-yarn-shuffle jar. After shading netty.io namespace, the nodemanager loads and shuffle job completes successfully. Author: Dhruve Ashar Closes #13739 from dhruve/bug/SPARK-16018. (cherry picked from commit 298c4ae81520b6b39230a6b0bf733c2b7caea627) Signed-off-by: Tom Graves --- common/network-yarn/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 1fd3af2e6e622..3b7ffe827705b 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -96,6 +96,13 @@ com.fasterxml.jackson.** + + io.netty + ${spark.shade.packageName}.io.netty + + io.netty.** + + From 0701b8d95caa79f220425b2a7376e88d69864663 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 17 Jun 2016 15:48:17 -0700 Subject: [PATCH 0701/1470] [SPARK-16017][CORE] Send hostname from CoarseGrainedExecutorBackend to driver ## What changes were proposed in this pull request? [SPARK-15395](https://issues.apache.org/jira/browse/SPARK-15395) changes the behavior that how the driver gets the executor host and the driver will get the executor IP address instead of the host name. This PR just sends the hostname from executors to driver so that driver can pass it to TaskScheduler. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu Closes #13741 from zsxwing/SPARK-16017. (cherry picked from commit 62d8fe2089659e8212753a622708517e0f4a77bc) Signed-off-by: Shixiong Zhu --- .../spark/executor/CoarseGrainedExecutorBackend.scala | 7 ++++--- .../scheduler/cluster/CoarseGrainedClusterMessage.scala | 4 ++-- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 6 +++--- .../scala/org/apache/spark/HeartbeatReceiverSuite.scala | 4 ++-- .../spark/deploy/StandaloneDynamicAllocationSuite.scala | 2 +- 5 files changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e08729510926b..ccc6c36e9c79a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -39,6 +39,7 @@ private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, driverUrl: String, executorId: String, + hostname: String, cores: Int, userClassPath: Seq[URL], env: SparkEnv) @@ -57,7 +58,7 @@ private[spark] class CoarseGrainedExecutorBackend( rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) - ref.ask[Boolean](RegisterExecutor(executorId, self, cores, extractLogUrls)) + ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls)) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" case Success(msg) => @@ -75,7 +76,7 @@ private[spark] class CoarseGrainedExecutorBackend( } override def receive: PartialFunction[Any, Unit] = { - case RegisteredExecutor(hostname) => + case RegisteredExecutor => logInfo("Successfully registered with driver") executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) @@ -201,7 +202,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverConf, executorId, hostname, port, cores, isLocal = false) env.rpcEnv.setupEndpoint("Executor", new CoarseGrainedExecutorBackend( - env.rpcEnv, driverUrl, executorId, cores, userClassPath, env)) + env.rpcEnv, driverUrl, executorId, hostname, cores, userClassPath, env)) workerUrl.foreach { url => env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 46a829114ec86..edc8aac5d1515 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -40,8 +40,7 @@ private[spark] object CoarseGrainedClusterMessages { sealed trait RegisterExecutorResponse - case class RegisteredExecutor(hostname: String) extends CoarseGrainedClusterMessage - with RegisterExecutorResponse + case object RegisteredExecutor extends CoarseGrainedClusterMessage with RegisterExecutorResponse case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage with RegisterExecutorResponse @@ -50,6 +49,7 @@ private[spark] object CoarseGrainedClusterMessages { case class RegisterExecutor( executorId: String, executorRef: RpcEndpointRef, + hostname: String, cores: Int, logUrls: Map[String, String]) extends CoarseGrainedClusterMessage 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 e84cb6346d51c..967c4d5325d63 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 @@ -148,7 +148,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterExecutor(executorId, executorRef, cores, logUrls) => + case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls) => if (executorDataMap.contains(executorId)) { executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) context.reply(true) @@ -164,7 +164,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) - val data = new ExecutorData(executorRef, executorRef.address, executorAddress.host, + val data = new ExecutorData(executorRef, executorRef.address, hostname, cores, cores, logUrls) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -178,7 +178,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") } } - executorRef.send(RegisteredExecutor(executorAddress.host)) + executorRef.send(RegisteredExecutor) // Note: some tests expect the reply to come after we put the executor in the map context.reply(true) listenerBus.post( diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 81b94b57219db..5e2ba311ee773 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -174,9 +174,9 @@ class HeartbeatReceiverSuite val dummyExecutorEndpointRef1 = rpcEnv.setupEndpoint("fake-executor-1", dummyExecutorEndpoint1) val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2) fakeSchedulerBackend.driverEndpoint.askWithRetry[Boolean]( - RegisterExecutor(executorId1, dummyExecutorEndpointRef1, 0, Map.empty)) + RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty)) fakeSchedulerBackend.driverEndpoint.askWithRetry[Boolean]( - RegisterExecutor(executorId2, dummyExecutorEndpointRef2, 0, Map.empty)) + RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty)) heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 3d39bd4a748cf..814027076d6fe 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -559,7 +559,7 @@ class StandaloneDynamicAllocationSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor(id, endpointRef, 10, Map.empty) + val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askWithRetry[Boolean](message) } From 2e5211ff9b141c7cc39a7fe05c278364d934fc52 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 17 Jun 2016 15:51:20 -0700 Subject: [PATCH 0702/1470] [SPARK-16014][SQL] Rename optimizer rules to be more consistent ## What changes were proposed in this pull request? This small patch renames a few optimizer rules to make the naming more consistent, e.g. class name start with a verb. The main important "fix" is probably SamplePushDown -> PushProjectThroughSample. SamplePushDown is actually the wrong name, since the rule is not about pushing Sample down. ## How was this patch tested? Updated test cases. Author: Reynold Xin Closes #13732 from rxin/SPARK-16014. (cherry picked from commit 1a65e62a7ff961ba140b9189e5335736334edbd6) Signed-off-by: Reynold Xin --- ....scala => RewriteDistinctAggregates.scala} | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 31 +++++++++---------- .../BinaryComparisonSimplificationSuite.scala | 2 +- .../optimizer/FilterPushdownSuite.scala | 2 +- .../optimizer/OuterJoinEliminationSuite.scala | 2 +- .../optimizer/SetOperationSuite.scala | 2 +- 6 files changed, 19 insertions(+), 22 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/{DistinctAggregationRewriter.scala => RewriteDistinctAggregates.scala} (99%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDistinctAggregates.scala similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDistinctAggregates.scala index 063eff4f984e1..8afd28dbba5c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDistinctAggregates.scala @@ -99,7 +99,7 @@ import org.apache.spark.sql.types.IntegerType * we could improve this in the current rule by applying more advanced expression canonicalization * techniques. */ -object DistinctAggregationRewriter extends Rule[LogicalPlan] { +object RewriteDistinctAggregates extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case a: Aggregate => rewrite(a) 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 581ede5a847fb..6190f7ab1c2e3 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 @@ -52,7 +52,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) ReplaceExpressions, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), - DistinctAggregationRewriter) :: + RewriteDistinctAggregates) :: ////////////////////////////////////////////////////////////////////////////////////////// // Optimizer rules start here ////////////////////////////////////////////////////////////////////////////////////////// @@ -74,10 +74,10 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) RemoveRepetitionFromGroupExpressions) :: Batch("Operator Optimizations", fixedPoint, // Operator push down - SetOperationPushDown, - SamplePushDown, + PushThroughSetOperations, + PushProjectThroughSample, ReorderJoin, - OuterJoinElimination, + EliminateOuterJoin, PushPredicateThroughJoin, PushDownPredicate, LimitPushDown, @@ -98,7 +98,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) BooleanSimplification, SimplifyConditionals, RemoveDispensableExpressions, - BinaryComparisonSimplification, + SimplifyBinaryComparison, PruneFilters, EliminateSorts, SimplifyCasts, @@ -147,10 +147,9 @@ class SimpleTestOptimizer extends Optimizer( new SimpleCatalystConf(caseSensitiveAnalysis = true)) /** - * Pushes operations down into a Sample. + * Pushes projects down beneath Sample to enable column pruning with sampling. */ -object SamplePushDown extends Rule[LogicalPlan] { - +object PushProjectThroughSample extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Push down projection into sample case Project(projectList, Sample(lb, up, replace, seed, child)) => @@ -170,7 +169,7 @@ object RemoveAliasOnlyProject extends Rule[LogicalPlan] { projectList: Seq[NamedExpression], childOutput: Seq[Attribute]): Boolean = { if (!projectList.forall(_.isInstanceOf[Alias]) || projectList.length != childOutput.length) { - return false + false } else { projectList.map(_.asInstanceOf[Alias]).zip(childOutput).forall { case (a, o) => a.child match { @@ -182,11 +181,9 @@ object RemoveAliasOnlyProject extends Rule[LogicalPlan] { } def apply(plan: LogicalPlan): LogicalPlan = { - val aliasOnlyProject = plan.find { p => - p match { - case Project(pList, child) if isAliasOnly(pList, child.output) => true - case _ => false - } + val aliasOnlyProject = plan.find { + case Project(pList, child) if isAliasOnly(pList, child.output) => true + case _ => false } aliasOnlyProject.map { case p: Project => @@ -291,7 +288,7 @@ object LimitPushDown extends Rule[LogicalPlan] { * safe to pushdown Filters and Projections through it. Once we add UNION DISTINCT, * we will not be able to pushdown Projections. */ -object SetOperationPushDown extends Rule[LogicalPlan] with PredicateHelper { +object PushThroughSetOperations extends Rule[LogicalPlan] with PredicateHelper { /** * Maps Attributes from the left side to the corresponding Attribute on the right side. @@ -879,7 +876,7 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { * 2) Replace '=', '<=', and '>=' with 'true' literal if both operands are non-nullable. * 3) Replace '<' and '>' with 'false' literal if both operands are non-nullable. */ -object BinaryComparisonSimplification extends Rule[LogicalPlan] with PredicateHelper { +object SimplifyBinaryComparison extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { // True with equality @@ -1217,7 +1214,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { * * This rule should be executed before pushing down the Filter */ -object OuterJoinElimination extends Rule[LogicalPlan] with PredicateHelper { +object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { /** * Returns whether the expression returns null or false when all inputs are nulls. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index 7cd038570bbdf..a313681eeb8f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -36,7 +36,7 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper NullPropagation, ConstantFolding, BooleanSimplification, - BinaryComparisonSimplification, + SimplifyBinaryComparison, PruneFilters) :: Nil } 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 fcc14a803beaa..b8f28e83e736d 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 @@ -34,7 +34,7 @@ class FilterPushdownSuite extends PlanTest { Batch("Subqueries", Once, EliminateSubqueryAliases) :: Batch("Filter Pushdown", FixedPoint(10), - SamplePushDown, + PushProjectThroughSample, CombineFilters, PushDownPredicate, BooleanSimplification, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala index 5e6e54dc741f3..41754adef4216 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala @@ -30,7 +30,7 @@ class OuterJoinEliminationSuite extends PlanTest { Batch("Subqueries", Once, EliminateSubqueryAliases) :: Batch("Outer Join Elimination", Once, - OuterJoinElimination, + EliminateOuterJoin, PushPredicateThroughJoin) :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala index 83ca9d5ec9f80..dab45a6b166be 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala @@ -31,7 +31,7 @@ class SetOperationSuite extends PlanTest { EliminateSubqueryAliases) :: Batch("Union Pushdown", Once, CombineUnions, - SetOperationPushDown, + PushThroughSetOperations, PruneFilters) :: Nil } From d4bb9a3ff16bc1f51bcf7156abff783e901d19d2 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 17 Jun 2016 15:56:03 -0700 Subject: [PATCH 0703/1470] [SPARK-15925][SPARKR] R DataFrame add back registerTempTable, add tests ## What changes were proposed in this pull request? Add registerTempTable to DataFrame with Deprecate ## How was this patch tested? unit tests shivaram liancheng Author: Felix Cheung Closes #13722 from felixcheung/rregistertemptable. (cherry picked from commit ef3cc4fc096e831823d62af4fd2a12ae88d434b4) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 30 +++++++++++++++++++++-- R/pkg/R/generics.R | 14 +++++++---- R/pkg/inst/tests/testthat/test_sparkSQL.R | 30 ++++++++++++++--------- 4 files changed, 57 insertions(+), 18 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 8db4d5ca1ee53..5db43ae6498db 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -82,6 +82,7 @@ exportMethods("arrange", "persist", "printSchema", "rbind", + "registerTempTable", "rename", "repartition", "sample", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index c710bffa2c407..231e4f0f4e9ce 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -457,6 +457,32 @@ setMethod("createOrReplaceTempView", invisible(callJMethod(x@sdf, "createOrReplaceTempView", viewName)) }) +#' (Deprecated) Register Temporary Table +#' Registers a SparkDataFrame as a Temporary Table in the SQLContext +#' @param x A SparkDataFrame +#' @param tableName A character vector containing the name of the table +#' +#' @family SparkDataFrame functions +#' @seealso \link{createOrReplaceTempView} +#' @rdname registerTempTable-deprecated +#' @name registerTempTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' registerTempTable(df, "json_df") +#' new_df <- sql("SELECT * FROM json_df") +#'} +setMethod("registerTempTable", + signature(x = "SparkDataFrame", tableName = "character"), + function(x, tableName) { + .Deprecated("createOrReplaceTempView") + invisible(callJMethod(x@sdf, "createOrReplaceTempView", tableName)) + }) + #' insertInto #' #' Insert the contents of a SparkDataFrame into a table registered in the current SQL Context. @@ -1286,7 +1312,7 @@ setMethod("dapplyCollect", #' @name gapply #' @export #' @examples -#' +#' #' \dontrun{ #' Computes the arithmetic mean of the second column by grouping #' on the first and third columns. Output the grouping values and the average. @@ -1317,7 +1343,7 @@ setMethod("dapplyCollect", #' Fits linear models on iris dataset by grouping on the 'Species' column and #' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' #' and 'Petal_Width' as training features. -#' +#' #' df <- createDataFrame (iris) #' schema <- structType(structField("(Intercept)", "double"), #' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 8164e7731a15f..594bf2eadc5a2 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -446,6 +446,13 @@ setGeneric("covar_samp", function(col1, col2) {standardGeneric("covar_samp") }) #' @export setGeneric("covar_pop", function(col1, col2) {standardGeneric("covar_pop") }) +#' @rdname createOrReplaceTempView +#' @export +setGeneric("createOrReplaceTempView", + function(x, viewName) { + standardGeneric("createOrReplaceTempView") + }) + #' @rdname dapply #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) @@ -548,12 +555,9 @@ setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) #' @export setGeneric("rename", function(x, ...) { standardGeneric("rename") }) -#' @rdname createOrReplaceTempView +#' @rdname registerTempTable-deprecated #' @export -setGeneric("createOrReplaceTempView", - function(x, viewName) { - standardGeneric("createOrReplaceTempView") - }) +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) #' @rdname sample #' @export diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 11d69366df320..7aa03a9048a80 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -443,22 +443,21 @@ test_that("jsonRDD() on a RDD with json string", { expect_equal(count(df), 6) }) -test_that("test cache, uncache and clearCache", { - df <- read.json(jsonPath) - createOrReplaceTempView(df, "table1") - cacheTable("table1") - uncacheTable("table1") - clearCache() - dropTempTable("table1") -}) - test_that("test tableNames and tables", { df <- read.json(jsonPath) createOrReplaceTempView(df, "table1") expect_equal(length(tableNames()), 1) - df <- tables() - expect_equal(count(df), 1) + tables <- tables() + expect_equal(count(tables), 1) + + suppressWarnings(registerTempTable(df, "table2")) + tables <- tables() + expect_equal(count(tables), 2) dropTempTable("table1") + dropTempTable("table2") + + tables <- tables() + expect_equal(count(tables), 0) }) test_that( @@ -471,6 +470,15 @@ test_that( dropTempTable("table1") }) +test_that("test cache, uncache and clearCache", { + df <- read.json(jsonPath) + createOrReplaceTempView(df, "table1") + cacheTable("table1") + uncacheTable("table1") + clearCache() + dropTempTable("table1") +}) + test_that("insertInto() on a registered table", { df <- read.df(jsonPath, "json") write.df(df, parquetPath, "parquet", "overwrite") From ca0802fd55f42fdcdd98533ee515d40d9f04a4b3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 17 Jun 2016 16:07:33 -0700 Subject: [PATCH 0704/1470] [SPARK-16005][R] Add `randomSplit` to SparkR ## What changes were proposed in this pull request? This PR adds `randomSplit` to SparkR for API parity. ## How was this patch tested? Pass the Jenkins tests (with new testcase.) Author: Dongjoon Hyun Closes #13721 from dongjoon-hyun/SPARK-16005. (cherry picked from commit 7d65a0db4a231882200513836f2720f59b35f364) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 37 +++++++++++++++++++++++ R/pkg/R/generics.R | 4 +++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 18 +++++++++++ 4 files changed, 60 insertions(+) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 5db43ae6498db..9412ec3f9e09b 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -81,6 +81,7 @@ exportMethods("arrange", "orderBy", "persist", "printSchema", + "randomSplit", "rbind", "registerTempTable", "rename", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 231e4f0f4e9ce..4e044565f4954 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2934,3 +2934,40 @@ setMethod("write.jdbc", write <- callJMethod(write, "mode", jmode) invisible(callJMethod(write, "jdbc", url, tableName, jprops)) }) + +#' randomSplit +#' +#' Return a list of randomly split dataframes with the provided weights. +#' +#' @param x A SparkDataFrame +#' @param weights A vector of weights for splits, will be normalized if they don't sum to 1 +#' @param seed A seed to use for random split +#' +#' @family SparkDataFrame functions +#' @rdname randomSplit +#' @name randomSplit +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#' df <- createDataFrame(data.frame(id = 1:1000)) +#' df_list <- randomSplit(df, c(2, 3, 5), 0) +#' # df_list contains 3 SparkDataFrames with each having about 200, 300 and 500 rows respectively +#' sapply(df_list, count) +#' } +#' @note since 2.0.0 +setMethod("randomSplit", + signature(x = "SparkDataFrame", weights = "numeric"), + function(x, weights, seed) { + if (!all(sapply(weights, function(c) { c >= 0 }))) { + stop("all weight values should not be negative") + } + normalized_list <- as.list(weights / sum(weights)) + if (!missing(seed)) { + sdfs <- callJMethod(x@sdf, "randomSplit", normalized_list, as.integer(seed)) + } else { + sdfs <- callJMethod(x@sdf, "randomSplit", normalized_list) + } + sapply(sdfs, dataFrame) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 594bf2eadc5a2..6e754afab6c6d 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -679,6 +679,10 @@ setGeneric("withColumnRenamed", #' @export setGeneric("write.df", function(df, path, ...) { standardGeneric("write.df") }) +#' @rdname randomSplit +#' @export +setGeneric("randomSplit", function(x, weights, seed) { standardGeneric("randomSplit") }) + ###################### Column Methods ########################## #' @rdname column diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 7aa03a9048a80..607bd9c12fa05 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2280,6 +2280,24 @@ test_that("createDataFrame sqlContext parameter backward compatibility", { expect_equal(collect(before), collect(after)) }) +test_that("randomSplit", { + num <- 4000 + df <- createDataFrame(data.frame(id = 1:num)) + + weights <- c(2, 3, 5) + df_list <- randomSplit(df, weights) + expect_equal(length(weights), length(df_list)) + counts <- sapply(df_list, count) + expect_equal(num, sum(counts)) + expect_true(all(sapply(abs(counts / num - weights / sum(weights)), function(e) { e < 0.05 }))) + + df_list <- randomSplit(df, weights, 0) + expect_equal(length(weights), length(df_list)) + counts <- sapply(df_list, count) + expect_equal(num, sum(counts)) + expect_true(all(sapply(abs(counts / num - weights / sum(weights)), function(e) { e < 0.05 }))) +}) + unlink(parquetPath) unlink(jsonPath) unlink(jsonPathNa) From b22b20db640e9fac20c5d54cc83964dc74393821 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 17 Jun 2016 17:11:38 -0700 Subject: [PATCH 0705/1470] [SPARK-15916][SQL] JDBC filter push down should respect operator precedence ## What changes were proposed in this pull request? This PR fixes the problem that the precedence order is messed when pushing where-clause expression to JDBC layer. **Case 1:** For sql `select * from table where (a or b) and c`, the where-clause is wrongly converted to JDBC where-clause `a or (b and c)` after filter push down. The consequence is that JDBC may returns less or more rows than expected. **Case 2:** For sql `select * from table where always_false_condition`, the result table may not be empty if the JDBC RDD is partitioned using where-clause: ``` spark.read.jdbc(url, table, predicates = Array("partition 1 where clause", "partition 2 where clause"...) ``` ## How was this patch tested? Unit test. This PR also close #13640 Author: hyukjinkwon Author: Sean Zhong Closes #13743 from clockfly/SPARK-15916. (cherry picked from commit ebb9a3b6fd834e2c856a192b4455aab83e9c4dc8) Signed-off-by: Cheng Lian --- .../execution/datasources/jdbc/JDBCRDD.scala | 4 +-- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 8d0906e574258..44cfbb9fbd811 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -305,14 +305,14 @@ private[sql] class JDBCRDD( * `filters`, but as a WHERE clause suitable for injection into a SQL query. */ private val filterWhereClause: String = - filters.flatMap(JDBCRDD.compileFilter).mkString(" AND ") + filters.flatMap(JDBCRDD.compileFilter).map(p => s"($p)").mkString(" AND ") /** * A WHERE clause representing both `filters`, if any, and the current partition. */ private def getWhereClause(part: JDBCPartition): String = { if (part.whereClause != null && filterWhereClause.length > 0) { - "WHERE " + filterWhereClause + " AND " + part.whereClause + "WHERE " + s"($filterWhereClause)" + " AND " + s"(${part.whereClause})" } else if (part.whereClause != null) { "WHERE " + part.whereClause } else if (filterWhereClause.length > 0) { 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 abb7918ae607b..d6ec40c18be2b 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 @@ -661,4 +661,30 @@ class JDBCSuite extends SparkFunSuite assert(oracleDialect.getJDBCType(StringType). map(_.databaseTypeDefinition).get == "VARCHAR2(255)") } + + private def assertEmptyQuery(sqlString: String): Unit = { + assert(sql(sqlString).collect().isEmpty) + } + + test("SPARK-15916: JDBC filter operator push down should respect operator precedence") { + val TRUE = "NAME != 'non_exists'" + val FALSE1 = "THEID > 1000000000" + val FALSE2 = "THEID < -1000000000" + + assertEmptyQuery(s"SELECT * FROM foobar WHERE ($TRUE OR $FALSE1) AND $FALSE2") + assertEmptyQuery(s"SELECT * FROM foobar WHERE $FALSE1 AND ($FALSE2 OR $TRUE)") + + // Tests JDBCPartition whereClause clause push down. + withTempTable("tempFrame") { + val jdbcPartitionWhereClause = s"$FALSE1 OR $TRUE" + val df = spark.read.jdbc( + urlWithUserAndPass, + "TEST.PEOPLE", + predicates = Array[String](jdbcPartitionWhereClause), + new Properties) + + df.createOrReplaceTempView("tempFrame") + assertEmptyQuery(s"SELECT * FROM tempFrame where $FALSE2") + } + } } From 57feaa572db62059555b573e145449d0063b08c8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 17 Jun 2016 20:13:04 -0700 Subject: [PATCH 0706/1470] [SPARK-16033][SQL] insertInto() can't be used together with partitionBy() ## What changes were proposed in this pull request? When inserting into an existing partitioned table, partitioning columns should always be determined by catalog metadata of the existing table to be inserted. Extra `partitionBy()` calls don't make sense, and mess up existing data because newly inserted data may have wrong partitioning directory layout. ## How was this patch tested? New test case added in `InsertIntoHiveTableSuite`. Author: Cheng Lian Closes #13747 from liancheng/spark-16033-insert-into-without-partition-by. (cherry picked from commit 10b671447bc04af250cbd8a7ea86f2769147a78a) Signed-off-by: Yin Huai --- .../apache/spark/sql/DataFrameWriter.scala | 12 ++++-- .../sql/hive/InsertIntoHiveTableSuite.scala | 37 +++++++++++++++++++ 2 files changed, 46 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 60a9d1f020b42..e6fc9749c7267 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -21,8 +21,6 @@ import java.util.Properties import scala.collection.JavaConverters._ -import org.apache.hadoop.fs.Path - import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} @@ -243,7 +241,15 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def insertInto(tableIdent: TableIdentifier): Unit = { assertNotBucketed("insertInto") - val partitions = normalizedParCols.map(_.map(col => col -> (Option.empty[String])).toMap) + + if (partitioningColumns.isDefined) { + throw new AnalysisException( + "insertInto() can't be used together with partitionBy(). " + + "Partition columns are defined by the table into which is being inserted." + ) + } + + val partitions = normalizedParCols.map(_.map(col => col -> Option.empty[String]).toMap) val overwrite = mode == SaveMode.Overwrite // A partitioned relation's schema can be different from the input logicalPlan, since 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 3bf45ced75e0a..b890b4bffdcfe 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 @@ -346,6 +346,43 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef } } + private def testPartitionedHiveSerDeTable(testName: String)(f: String => Unit): Unit = { + test(s"Hive SerDe table - $testName") { + val hiveTable = "hive_table" + + withTable(hiveTable) { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql(s"CREATE TABLE $hiveTable (a INT) PARTITIONED BY (b INT, c INT) STORED AS TEXTFILE") + f(hiveTable) + } + } + } + } + + private def testPartitionedDataSourceTable(testName: String)(f: String => Unit): Unit = { + test(s"Data source table - $testName") { + val dsTable = "ds_table" + + withTable(dsTable) { + sql(s"CREATE TABLE $dsTable (a INT, b INT, c INT) USING PARQUET PARTITIONED BY (b, c)") + f(dsTable) + } + } + } + + private def testPartitionedTable(testName: String)(f: String => Unit): Unit = { + testPartitionedHiveSerDeTable(testName)(f) + testPartitionedDataSourceTable(testName)(f) + } + + testPartitionedTable("partitionBy() can't be used together with insertInto()") { tableName => + val cause = intercept[AnalysisException] { + Seq((1, 2, 3)).toDF("a", "b", "c").write.partitionBy("b", "c").insertInto(tableName) + } + + assert(cause.getMessage.contains("insertInto() can't be used together with partitionBy().")) + } + test("InsertIntoTable#resolved should include dynamic partitions") { withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") From f0de45cb1591c5e43a8a7c74566a9f43b6af7dcb Mon Sep 17 00:00:00 2001 From: GayathriMurali Date: Fri, 17 Jun 2016 21:10:29 -0700 Subject: [PATCH 0707/1470] [SPARK-15129][R][DOC] R API changes in ML ## What changes were proposed in this pull request? Make user guide changes to SparkR documentation for all changes that happened in 2.0 to Machine Learning APIs Author: GayathriMurali Closes #13285 from GayathriMurali/SPARK-15129. (cherry picked from commit af2a4b0826b2358c0fe75c3e4d7fd8f7bccdd8e5) Signed-off-by: Xiangrui Meng --- docs/sparkr.md | 77 ++++++++++------------------------------ examples/src/main/r/ml.R | 4 +-- 2 files changed, 21 insertions(+), 60 deletions(-) diff --git a/docs/sparkr.md b/docs/sparkr.md index 59e486d1929f0..961bd323fabcb 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -285,71 +285,32 @@ head(teenagers) # Machine Learning -SparkR allows the fitting of generalized linear models over DataFrames using the [glm()](api/R/glm.html) function. Under the hood, SparkR uses MLlib to train a model of the specified family. Currently the gaussian and binomial families are supported. We support a subset of the available R formula operators for model fitting, including '~', '.', ':', '+', and '-'. +SparkR supports the following Machine Learning algorithms. -The [summary()](api/R/summary.html) function gives the summary of a model produced by [glm()](api/R/glm.html). +* Generalized Linear Regression Model [spark.glm()](api/R/spark.glm.html) +* Naive Bayes [spark.naiveBayes()](api/R/spark.naiveBayes.html) +* KMeans [spark.kmeans()](api/R/spark.kmeans.html) +* AFT Survival Regression [spark.survreg()](api/R/spark.survreg.html) -* For gaussian GLM model, it returns a list with 'devianceResiduals' and 'coefficients' components. The 'devianceResiduals' gives the min/max deviance residuals of the estimation; the 'coefficients' gives the estimated coefficients and their estimated standard errors, t values and p-values. (It only available when model fitted by normal solver.) -* For binomial GLM model, it returns a list with 'coefficients' component which gives the estimated coefficients. +[Generalized Linear Regression](api/R/spark.glm.html) can be used to train a model from a specified family. Currently the Gaussian, Binomial, Poisson and Gamma families are supported. We support a subset of the available R formula operators for model fitting, including '~', '.', ':', '+', and '-'. -The examples below show the use of building gaussian GLM model and binomial GLM model using SparkR. +The [summary()](api/R/summary.html) function gives the summary of a model produced by different algorithms listed above. +It produces the similar result compared with R summary function. -## Gaussian GLM model +## Model persistence -
    -{% highlight r %} -# Create the DataFrame -df <- createDataFrame(sqlContext, iris) - -# Fit a gaussian GLM model over the dataset. -model <- glm(Sepal_Length ~ Sepal_Width + Species, data = df, family = "gaussian") - -# Model summary are returned in a similar format to R's native glm(). -summary(model) -##$devianceResiduals -## Min Max -## -1.307112 1.412532 -## -##$coefficients -## Estimate Std. Error t value Pr(>|t|) -##(Intercept) 2.251393 0.3697543 6.08889 9.568102e-09 -##Sepal_Width 0.8035609 0.106339 7.556598 4.187317e-12 -##Species_versicolor 1.458743 0.1121079 13.01195 0 -##Species_virginica 1.946817 0.100015 19.46525 0 - -# Make predictions based on the model. -predictions <- predict(model, newData = df) -head(select(predictions, "Sepal_Length", "prediction")) -## Sepal_Length prediction -##1 5.1 5.063856 -##2 4.9 4.662076 -##3 4.7 4.822788 -##4 4.6 4.742432 -##5 5.0 5.144212 -##6 5.4 5.385281 -{% endhighlight %} -
    +* [write.ml](api/R/write.ml.html) allows users to save a fitted model in a given input path +* [read.ml](api/R/read.ml.html) allows users to read/load the model which was saved using write.ml in a given path -## Binomial GLM model +Model persistence is supported for all Machine Learning algorithms for all families. -
    -{% highlight r %} -# Create the DataFrame -df <- createDataFrame(sqlContext, iris) -training <- filter(df, df$Species != "setosa") - -# Fit a binomial GLM model over the dataset. -model <- glm(Species ~ Sepal_Length + Sepal_Width, data = training, family = "binomial") - -# Model coefficients are returned in a similar format to R's native glm(). -summary(model) -##$coefficients -## Estimate -##(Intercept) -13.046005 -##Sepal_Length 1.902373 -##Sepal_Width 0.404655 -{% endhighlight %} -
    +The examples below show how to build several models: +* GLM using the Gaussian and Binomial model families +* AFT survival regression model +* Naive Bayes model +* K-Means model + +{% include_example r/ml.R %} # R Function Name Conflicts diff --git a/examples/src/main/r/ml.R b/examples/src/main/r/ml.R index fd35936635334..495f392c26542 100644 --- a/examples/src/main/r/ml.R +++ b/examples/src/main/r/ml.R @@ -25,6 +25,7 @@ library(SparkR) sc <- sparkR.init(appName="SparkR-ML-example") sqlContext <- sparkRSQL.init(sc) +# $example on$ ############################ spark.glm and glm ############################################## irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) @@ -57,7 +58,6 @@ binomialPredictions <- predict(binomialGLM, binomialTestDF) showDF(binomialPredictions) ############################ spark.survreg ############################################## - # Use the ovarian dataset available in R survival package library(survival) @@ -121,7 +121,7 @@ gaussianGLM <- spark.glm(gaussianDF, Sepal_Length ~ Sepal_Width + Species, famil modelPath <- tempfile(pattern = "ml", fileext = ".tmp") write.ml(gaussianGLM, modelPath) gaussianGLM2 <- read.ml(modelPath) - +# $example off$ # Check model summary summary(gaussianGLM2) From 0a8fd2eb8966afaff3030adef5fc6fd73171607c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 17 Jun 2016 21:22:29 -0700 Subject: [PATCH 0708/1470] [SPARK-15946][MLLIB] Conversion between old/new vector columns in a DataFrame (Python) ## What changes were proposed in this pull request? This PR implements python wrappers for #13662 to convert old/new vector columns in a DataFrame. ## How was this patch tested? doctest in Python cc: yanboliang Author: Xiangrui Meng Closes #13731 from mengxr/SPARK-15946. (cherry picked from commit edb23f9e47eecfe60992dde0e037ec1985c77e1d) Signed-off-by: Yanbo Liang --- .../mllib/api/python/PythonMLLibAPI.scala | 14 ++++ python/pyspark/mllib/util.py | 82 +++++++++++++++++++ 2 files changed, 96 insertions(+) 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 7df61601fb1e9..f2c70baf472a7 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 @@ -1201,6 +1201,20 @@ private[python] class PythonMLLibAPI extends Serializable { val spark = SparkSession.builder().sparkContext(sc).getOrCreate() spark.createDataFrame(blockMatrix.blocks) } + + /** + * Python-friendly version of [[MLUtils.convertVectorColumnsToML()]]. + */ + def convertVectorColumnsToML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertVectorColumnsToML(dataset, cols.asScala: _*) + } + + /** + * Python-friendly version of [[MLUtils.convertVectorColumnsFromML()]] + */ + def convertVectorColumnsFromML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertVectorColumnsFromML(dataset, cols.asScala: _*) + } } /** diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index a316ee1ad45ff..a7e6bcc754dc7 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -26,6 +26,7 @@ from pyspark import SparkContext, since from pyspark.mllib.common import callMLlibFunc, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector +from pyspark.sql import DataFrame class MLUtils(object): @@ -200,6 +201,86 @@ def loadVectors(sc, path): """ return callMLlibFunc("loadVectors", sc, path) + @staticmethod + @since("2.0.0") + def convertVectorColumnsToML(dataset, *cols): + """ + Converts vector columns in an input DataFrame from the + :py:class:`pyspark.mllib.linalg.Vector` type to the new + :py:class:`pyspark.ml.linalg.Vector` type under the `spark.ml` + package. + + :param dataset: + input dataset + :param cols: + a list of vector columns to be converted. + New vector columns will be ignored. If unspecified, all old + vector columns will be converted excepted nested ones. + :return: + the input dataset with old vector columns converted to the + new vector type + + >>> import pyspark + >>> from pyspark.mllib.linalg import Vectors + >>> from pyspark.mllib.util import MLUtils + >>> df = spark.createDataFrame( + ... [(0, Vectors.sparse(2, [1], [1.0]), Vectors.dense(2.0, 3.0))], + ... ["id", "x", "y"]) + >>> r1 = MLUtils.convertVectorColumnsToML(df).first() + >>> isinstance(r1.x, pyspark.ml.linalg.SparseVector) + True + >>> isinstance(r1.y, pyspark.ml.linalg.DenseVector) + True + >>> r2 = MLUtils.convertVectorColumnsToML(df, "x").first() + >>> isinstance(r2.x, pyspark.ml.linalg.SparseVector) + True + >>> isinstance(r2.y, pyspark.mllib.linalg.DenseVector) + True + """ + if not isinstance(dataset, DataFrame): + raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset))) + return callMLlibFunc("convertVectorColumnsToML", dataset, list(cols)) + + @staticmethod + @since("2.0.0") + def convertVectorColumnsFromML(dataset, *cols): + """ + Converts vector columns in an input DataFrame to the + :py:class:`pyspark.mllib.linalg.Vector` type from the new + :py:class:`pyspark.ml.linalg.Vector` type under the `spark.ml` + package. + + :param dataset: + input dataset + :param cols: + a list of vector columns to be converted. + Old vector columns will be ignored. If unspecified, all new + vector columns will be converted except nested ones. + :return: + the input dataset with new vector columns converted to the + old vector type + + >>> import pyspark + >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.mllib.util import MLUtils + >>> df = spark.createDataFrame( + ... [(0, Vectors.sparse(2, [1], [1.0]), Vectors.dense(2.0, 3.0))], + ... ["id", "x", "y"]) + >>> r1 = MLUtils.convertVectorColumnsFromML(df).first() + >>> isinstance(r1.x, pyspark.mllib.linalg.SparseVector) + True + >>> isinstance(r1.y, pyspark.mllib.linalg.DenseVector) + True + >>> r2 = MLUtils.convertVectorColumnsFromML(df, "x").first() + >>> isinstance(r2.x, pyspark.mllib.linalg.SparseVector) + True + >>> isinstance(r2.y, pyspark.ml.linalg.DenseVector) + True + """ + if not isinstance(dataset, DataFrame): + raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset))) + return callMLlibFunc("convertVectorColumnsFromML", dataset, list(cols)) + class Saveable(object): """ @@ -355,6 +436,7 @@ def _test(): .master("local[2]")\ .appName("mllib.util tests")\ .getOrCreate() + globs['spark'] = spark globs['sc'] = spark.sparkContext (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) spark.stop() From 8b7e561210a29d66317ce66f598d4bd2ad2c8087 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 17 Jun 2016 21:36:01 -0700 Subject: [PATCH 0709/1470] [SPARK-15159][SPARKR] SparkR SparkSession API ## What changes were proposed in this pull request? This PR introduces the new SparkSession API for SparkR. `sparkR.session.getOrCreate()` and `sparkR.session.stop()` "getOrCreate" is a bit unusual in R but it's important to name this clearly. SparkR implementation should - SparkSession is the main entrypoint (vs SparkContext; due to limited functionality supported with SparkContext in SparkR) - SparkSession replaces SQLContext and HiveContext (both a wrapper around SparkSession, and because of API changes, supporting all 3 would be a lot more work) - Changes to SparkSession is mostly transparent to users due to SPARK-10903 - Full backward compatibility is expected - users should be able to initialize everything just in Spark 1.6.1 (`sparkR.init()`), but with deprecation warning - Mostly cosmetic changes to parameter list - users should be able to move to `sparkR.session.getOrCreate()` easily - An advanced syntax with named parameters (aka varargs aka "...") is supported; that should be closer to the Builder syntax that is in Scala/Python (which unfortunately does not work in R because it will look like this: `enableHiveSupport(config(config(master(appName(builder(), "foo"), "local"), "first", "value"), "next, "value"))` - Updating config on an existing SparkSession is supported, the behavior is the same as Python, in which config is applied to both SparkContext and SparkSession - Some SparkSession changes are not matched in SparkR, mostly because it would be breaking API change: `catalog` object, `createOrReplaceTempView` - Other SQLContext workarounds are replicated in SparkR, eg. `tables`, `tableNames` - `sparkR` shell is updated to use the SparkSession entrypoint (`sqlContext` is removed, just like with Scale/Python) - All tests are updated to use the SparkSession entrypoint - A bug in `read.jdbc` is fixed TODO - [x] Add more tests - [ ] Separate PR - update all roxygen2 doc coding example - [ ] Separate PR - update SparkR programming guide ## How was this patch tested? unit tests, manual tests shivaram sun-rui rxin Author: Felix Cheung Author: felixcheung Closes #13635 from felixcheung/rsparksession. (cherry picked from commit 8c198e246d64b5779dc3a2625d06ec958553a20b) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 8 +- R/pkg/R/DataFrame.R | 8 +- R/pkg/R/SQLContext.R | 109 ++++++----- R/pkg/R/backend.R | 2 +- R/pkg/R/sparkR.R | 183 +++++++++++++----- R/pkg/R/utils.R | 9 + R/pkg/inst/profile/shell.R | 12 +- R/pkg/inst/tests/testthat/jarTest.R | 4 +- R/pkg/inst/tests/testthat/packageInAJarTest.R | 4 +- R/pkg/inst/tests/testthat/test_Serde.R | 2 +- R/pkg/inst/tests/testthat/test_binaryFile.R | 3 +- .../tests/testthat/test_binary_function.R | 3 +- R/pkg/inst/tests/testthat/test_broadcast.R | 3 +- R/pkg/inst/tests/testthat/test_context.R | 41 ++-- .../inst/tests/testthat/test_includePackage.R | 3 +- R/pkg/inst/tests/testthat/test_mllib.R | 5 +- .../tests/testthat/test_parallelize_collect.R | 3 +- R/pkg/inst/tests/testthat/test_rdd.R | 3 +- R/pkg/inst/tests/testthat/test_shuffle.R | 3 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 86 ++++++-- R/pkg/inst/tests/testthat/test_take.R | 17 +- R/pkg/inst/tests/testthat/test_textFile.R | 3 +- R/pkg/inst/tests/testthat/test_utils.R | 16 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 76 ++++++-- 24 files changed, 420 insertions(+), 186 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 9412ec3f9e09b..82e56ca437299 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -6,10 +6,15 @@ importFrom(methods, setGeneric, setMethod, setOldClass) #useDynLib(SparkR, stringHashCode) # S3 methods exported +export("sparkR.session") export("sparkR.init") export("sparkR.stop") +export("sparkR.session.stop") export("print.jobj") +export("sparkRSQL.init", + "sparkRHive.init") + # MLlib integration exportMethods("glm", "spark.glm", @@ -287,9 +292,6 @@ exportMethods("%in%", exportClasses("GroupedData") exportMethods("agg") -export("sparkRSQL.init", - "sparkRHive.init") - export("as.DataFrame", "cacheTable", "clearCache", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 4e044565f4954..ea091c81016d4 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2333,9 +2333,7 @@ setMethod("write.df", signature(df = "SparkDataFrame", path = "character"), function(df, path, source = NULL, mode = "error", ...){ if (is.null(source)) { - sqlContext <- getSqlContext() - source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", - "org.apache.spark.sql.parquet") + source <- getDefaultSqlSource() } jmode <- convertToJSaveMode(mode) options <- varargsToEnv(...) @@ -2393,9 +2391,7 @@ setMethod("saveAsTable", signature(df = "SparkDataFrame", tableName = "character"), function(df, tableName, source = NULL, mode="error", ...){ if (is.null(source)) { - sqlContext <- getSqlContext() - source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", - "org.apache.spark.sql.parquet") + source <- getDefaultSqlSource() } jmode <- convertToJSaveMode(mode) options <- varargsToEnv(...) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 914b02a47ad67..3232241f8af55 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -53,7 +53,8 @@ dispatchFunc <- function(newFuncSig, x, ...) { # Strip sqlContext from list of parameters and then pass the rest along. contextNames <- c("org.apache.spark.sql.SQLContext", "org.apache.spark.sql.hive.HiveContext", - "org.apache.spark.sql.hive.test.TestHiveContext") + "org.apache.spark.sql.hive.test.TestHiveContext", + "org.apache.spark.sql.SparkSession") if (missing(x) && length(list(...)) == 0) { f() } else if (class(x) == "jobj" && @@ -65,14 +66,12 @@ dispatchFunc <- function(newFuncSig, x, ...) { } } -#' return the SQL Context -getSqlContext <- function() { - if (exists(".sparkRHivesc", envir = .sparkREnv)) { - get(".sparkRHivesc", envir = .sparkREnv) - } else if (exists(".sparkRSQLsc", envir = .sparkREnv)) { - get(".sparkRSQLsc", envir = .sparkREnv) +#' return the SparkSession +getSparkSession <- function() { + if (exists(".sparkRsession", envir = .sparkREnv)) { + get(".sparkRsession", envir = .sparkREnv) } else { - stop("SQL context not initialized") + stop("SparkSession not initialized") } } @@ -109,6 +108,13 @@ infer_type <- function(x) { } } +getDefaultSqlSource <- function() { + sparkSession <- getSparkSession() + conf <- callJMethod(sparkSession, "conf") + source <- callJMethod(conf, "get", "spark.sql.sources.default", "org.apache.spark.sql.parquet") + source +} + #' Create a SparkDataFrame #' #' Converts R data.frame or list into SparkDataFrame. @@ -131,7 +137,7 @@ infer_type <- function(x) { # TODO(davies): support sampling and infer type from NA createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { - sqlContext <- getSqlContext() + sparkSession <- getSparkSession() if (is.data.frame(data)) { # get the names of columns, they will be put into RDD if (is.null(schema)) { @@ -158,7 +164,7 @@ createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { data <- do.call(mapply, append(args, data)) } if (is.list(data)) { - sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sqlContext) + sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) rdd <- parallelize(sc, data) } else if (inherits(data, "RDD")) { rdd <- data @@ -201,7 +207,7 @@ createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { jrdd <- getJRDD(lapply(rdd, function(x) x), "row") srdd <- callJMethod(jrdd, "rdd") sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", - srdd, schema$jobj, sqlContext) + srdd, schema$jobj, sparkSession) dataFrame(sdf) } @@ -265,10 +271,10 @@ setMethod("toDF", signature(x = "RDD"), #' @method read.json default read.json.default <- function(path) { - sqlContext <- getSqlContext() + sparkSession <- getSparkSession() # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) - read <- callJMethod(sqlContext, "read") + read <- callJMethod(sparkSession, "read") sdf <- callJMethod(read, "json", paths) dataFrame(sdf) } @@ -336,10 +342,10 @@ jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { #' @method read.parquet default read.parquet.default <- function(path) { - sqlContext <- getSqlContext() + sparkSession <- getSparkSession() # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) - read <- callJMethod(sqlContext, "read") + read <- callJMethod(sparkSession, "read") sdf <- callJMethod(read, "parquet", paths) dataFrame(sdf) } @@ -385,10 +391,10 @@ parquetFile <- function(x, ...) { #' @method read.text default read.text.default <- function(path) { - sqlContext <- getSqlContext() + sparkSession <- getSparkSession() # Allow the user to have a more flexible definiton of the text file path paths <- as.list(suppressWarnings(normalizePath(path))) - read <- callJMethod(sqlContext, "read") + read <- callJMethod(sparkSession, "read") sdf <- callJMethod(read, "text", paths) dataFrame(sdf) } @@ -418,8 +424,8 @@ read.text <- function(x, ...) { #' @method sql default sql.default <- function(sqlQuery) { - sqlContext <- getSqlContext() - sdf <- callJMethod(sqlContext, "sql", sqlQuery) + sparkSession <- getSparkSession() + sdf <- callJMethod(sparkSession, "sql", sqlQuery) dataFrame(sdf) } @@ -449,8 +455,8 @@ sql <- function(x, ...) { #' @note since 2.0.0 tableToDF <- function(tableName) { - sqlContext <- getSqlContext() - sdf <- callJMethod(sqlContext, "table", tableName) + sparkSession <- getSparkSession() + sdf <- callJMethod(sparkSession, "table", tableName) dataFrame(sdf) } @@ -472,12 +478,8 @@ tableToDF <- function(tableName) { #' @method tables default tables.default <- function(databaseName = NULL) { - sqlContext <- getSqlContext() - jdf <- if (is.null(databaseName)) { - callJMethod(sqlContext, "tables") - } else { - callJMethod(sqlContext, "tables", databaseName) - } + sparkSession <- getSparkSession() + jdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getTables", sparkSession, databaseName) dataFrame(jdf) } @@ -503,12 +505,11 @@ tables <- function(x, ...) { #' @method tableNames default tableNames.default <- function(databaseName = NULL) { - sqlContext <- getSqlContext() - if (is.null(databaseName)) { - callJMethod(sqlContext, "tableNames") - } else { - callJMethod(sqlContext, "tableNames", databaseName) - } + sparkSession <- getSparkSession() + callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "getTableNames", + sparkSession, + databaseName) } tableNames <- function(x, ...) { @@ -536,8 +537,9 @@ tableNames <- function(x, ...) { #' @method cacheTable default cacheTable.default <- function(tableName) { - sqlContext <- getSqlContext() - callJMethod(sqlContext, "cacheTable", tableName) + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + callJMethod(catalog, "cacheTable", tableName) } cacheTable <- function(x, ...) { @@ -565,8 +567,9 @@ cacheTable <- function(x, ...) { #' @method uncacheTable default uncacheTable.default <- function(tableName) { - sqlContext <- getSqlContext() - callJMethod(sqlContext, "uncacheTable", tableName) + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + callJMethod(catalog, "uncacheTable", tableName) } uncacheTable <- function(x, ...) { @@ -587,8 +590,9 @@ uncacheTable <- function(x, ...) { #' @method clearCache default clearCache.default <- function() { - sqlContext <- getSqlContext() - callJMethod(sqlContext, "clearCache") + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + callJMethod(catalog, "clearCache") } clearCache <- function() { @@ -615,11 +619,12 @@ clearCache <- function() { #' @method dropTempTable default dropTempTable.default <- function(tableName) { - sqlContext <- getSqlContext() + sparkSession <- getSparkSession() if (class(tableName) != "character") { stop("tableName must be a string.") } - callJMethod(sqlContext, "dropTempTable", tableName) + catalog <- callJMethod(sparkSession, "catalog") + callJMethod(catalog, "dropTempView", tableName) } dropTempTable <- function(x, ...) { @@ -655,21 +660,21 @@ dropTempTable <- function(x, ...) { #' @method read.df default read.df.default <- function(path = NULL, source = NULL, schema = NULL, ...) { - sqlContext <- getSqlContext() + sparkSession <- getSparkSession() options <- varargsToEnv(...) if (!is.null(path)) { options[["path"]] <- path } if (is.null(source)) { - source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", - "org.apache.spark.sql.parquet") + source <- getDefaultSqlSource() } if (!is.null(schema)) { stopifnot(class(schema) == "structType") - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "loadDF", sqlContext, source, + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "loadDF", sparkSession, source, schema$jobj, options) } else { - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "loadDF", sqlContext, source, options) + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "loadDF", sparkSession, source, options) } dataFrame(sdf) } @@ -715,12 +720,13 @@ loadDF <- function(x, ...) { #' @method createExternalTable default createExternalTable.default <- function(tableName, path = NULL, source = NULL, ...) { - sqlContext <- getSqlContext() + sparkSession <- getSparkSession() options <- varargsToEnv(...) if (!is.null(path)) { options[["path"]] <- path } - sdf <- callJMethod(sqlContext, "createExternalTable", tableName, source, options) + catalog <- callJMethod(sparkSession, "catalog") + sdf <- callJMethod(catalog, "createExternalTable", tableName, source, options) dataFrame(sdf) } @@ -767,12 +773,11 @@ read.jdbc <- function(url, tableName, partitionColumn = NULL, lowerBound = NULL, upperBound = NULL, numPartitions = 0L, predicates = list(), ...) { jprops <- varargsToJProperties(...) - - read <- callJMethod(sqlContext, "read") + sparkSession <- getSparkSession() + read <- callJMethod(sparkSession, "read") if (!is.null(partitionColumn)) { if (is.null(numPartitions) || numPartitions == 0) { - sqlContext <- getSqlContext() - sc <- callJMethod(sqlContext, "sparkContext") + sc <- callJMethod(sparkSession, "sparkContext") numPartitions <- callJMethod(sc, "defaultParallelism") } else { numPartitions <- numToInt(numPartitions) diff --git a/R/pkg/R/backend.R b/R/pkg/R/backend.R index 6c81492f8b675..03e70bb2cb82e 100644 --- a/R/pkg/R/backend.R +++ b/R/pkg/R/backend.R @@ -68,7 +68,7 @@ isRemoveMethod <- function(isStatic, objId, methodName) { # 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") + stop("No connection to backend found. Please re-run sparkR.session()") } # If this isn't a removeJObject call diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 04a8b1e1f3952..0dfd7b753033e 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -28,10 +28,21 @@ connExists <- function(env) { }) } -#' Stop the Spark context. -#' -#' Also terminates the backend this R session is connected to +#' @rdname sparkR.session.stop +#' @name sparkR.stop +#' @export sparkR.stop <- function() { + sparkR.session.stop() +} + +#' Stop the Spark Session and Spark Context. +#' +#' Also terminates the backend this R session is connected to. +#' @rdname sparkR.session.stop +#' @name sparkR.session.stop +#' @export +#' @note since 2.0.0 +sparkR.session.stop <- function() { env <- .sparkREnv if (exists(".sparkRCon", envir = env)) { if (exists(".sparkRjsc", envir = env)) { @@ -39,12 +50,8 @@ sparkR.stop <- function() { callJMethod(sc, "stop") rm(".sparkRjsc", envir = env) - if (exists(".sparkRSQLsc", envir = env)) { - rm(".sparkRSQLsc", envir = env) - } - - if (exists(".sparkRHivesc", envir = env)) { - rm(".sparkRHivesc", envir = env) + if (exists(".sparkRsession", envir = env)) { + rm(".sparkRsession", envir = env) } } @@ -80,7 +87,7 @@ sparkR.stop <- function() { clearJobjs() } -#' Initialize a new Spark Context. +#' (Deprecated) Initialize a new Spark Context. #' #' This function initializes a new SparkContext. For details on how to initialize #' and use SparkR, refer to SparkR programming guide at @@ -93,6 +100,8 @@ sparkR.stop <- function() { #' @param sparkExecutorEnv Named list of environment variables to be used when launching executors #' @param sparkJars Character vector of jar files to pass to the worker nodes #' @param sparkPackages Character vector of packages from spark-packages.org +#' @seealso \link{sparkR.session} +#' @rdname sparkR.init-deprecated #' @export #' @examples #'\dontrun{ @@ -114,18 +123,35 @@ sparkR.init <- function( sparkExecutorEnv = list(), sparkJars = "", sparkPackages = "") { + .Deprecated("sparkR.session") + sparkR.sparkContext(master, + appName, + sparkHome, + convertNamedListToEnv(sparkEnvir), + convertNamedListToEnv(sparkExecutorEnv), + sparkJars, + sparkPackages) +} + +# Internal function to handle creating the SparkContext. +sparkR.sparkContext <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkEnvirMap = new.env(), + sparkExecutorEnvMap = new.env(), + sparkJars = "", + sparkPackages = "") { if (exists(".sparkRjsc", envir = .sparkREnv)) { cat(paste("Re-using existing Spark Context.", - "Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n")) + "Call sparkR.session.stop() or restart R to create a new Spark Context\n")) return(get(".sparkRjsc", envir = .sparkREnv)) } jars <- processSparkJars(sparkJars) packages <- processSparkPackages(sparkPackages) - sparkEnvirMap <- convertNamedListToEnv(sparkEnvir) - existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") if (existingPort != "") { backendPort <- existingPort @@ -183,7 +209,6 @@ sparkR.init <- function( sparkHome <- suppressWarnings(normalizePath(sparkHome)) } - sparkExecutorEnvMap <- convertNamedListToEnv(sparkExecutorEnv) if (is.null(sparkExecutorEnvMap$LD_LIBRARY_PATH)) { sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- paste0("$LD_LIBRARY_PATH:", Sys.getenv("LD_LIBRARY_PATH")) @@ -225,12 +250,17 @@ sparkR.init <- function( sc } -#' Initialize a new SQLContext. +#' (Deprecated) Initialize a new SQLContext. #' #' This function creates a SparkContext from an existing JavaSparkContext and #' then uses it to initialize a new SQLContext #' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' #' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRSQL.init-deprecated #' @export #' @examples #'\dontrun{ @@ -239,29 +269,26 @@ sparkR.init <- function( #'} sparkRSQL.init <- function(jsc = NULL) { - if (exists(".sparkRSQLsc", envir = .sparkREnv)) { - return(get(".sparkRSQLsc", envir = .sparkREnv)) - } + .Deprecated("sparkR.session") - # If jsc is NULL, create a Spark Context - sc <- if (is.null(jsc)) { - sparkR.init() - } else { - jsc + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) } - sqlContext <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", - "createSQLContext", - sc) - assign(".sparkRSQLsc", sqlContext, envir = .sparkREnv) - sqlContext + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = FALSE) } -#' Initialize a new HiveContext. +#' (Deprecated) Initialize a new HiveContext. #' #' This function creates a HiveContext from an existing JavaSparkContext #' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' #' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRHive.init-deprecated #' @export #' @examples #'\dontrun{ @@ -270,27 +297,93 @@ sparkRSQL.init <- function(jsc = NULL) { #'} sparkRHive.init <- function(jsc = NULL) { - if (exists(".sparkRHivesc", envir = .sparkREnv)) { - return(get(".sparkRHivesc", envir = .sparkREnv)) + .Deprecated("sparkR.session") + + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) } - # If jsc is NULL, create a Spark Context - sc <- if (is.null(jsc)) { - sparkR.init() - } else { - jsc + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = TRUE) +} + +#' Get the existing SparkSession or initialize a new SparkSession. +#' +#' Additional Spark properties can be set (...), and these named parameters take priority over +#' over values in master, appName, named lists of sparkConfig. +#' +#' @param master The Spark master URL +#' @param appName Application name to register with cluster manager +#' @param sparkHome Spark Home directory +#' @param sparkConfig Named list of Spark configuration to set on worker nodes +#' @param sparkJars Character vector of jar files to pass to the worker nodes +#' @param sparkPackages Character vector of packages from spark-packages.org +#' @param enableHiveSupport Enable support for Hive, fallback if not built with Hive support; once +#' set, this cannot be turned off on an existing session +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- read.json(path) +#' +#' sparkR.session("local[2]", "SparkR", "/home/spark") +#' sparkR.session("yarn-client", "SparkR", "/home/spark", +#' list(spark.executor.memory="4g"), +#' c("one.jar", "two.jar", "three.jar"), +#' c("com.databricks:spark-avro_2.10:2.0.1")) +#' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g") +#'} +#' @note since 2.0.0 + +sparkR.session <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkConfig = list(), + sparkJars = "", + sparkPackages = "", + enableHiveSupport = TRUE, + ...) { + + sparkConfigMap <- convertNamedListToEnv(sparkConfig) + namedParams <- list(...) + if (length(namedParams) > 0) { + paramMap <- convertNamedListToEnv(namedParams) + # Override for certain named parameters + if (exists("spark.master", envir = paramMap)) { + master <- paramMap[["spark.master"]] + } + if (exists("spark.app.name", envir = paramMap)) { + appName <- paramMap[["spark.app.name"]] + } + overrideEnvs(sparkConfigMap, paramMap) } - ssc <- callJMethod(sc, "sc") - hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.HiveContext", ssc) - }, - error = function(err) { - stop("Spark SQL is not built with Hive support") - }) + if (!exists(".sparkRjsc", envir = .sparkREnv)) { + sparkExecutorEnvMap <- new.env() + sparkR.sparkContext(master, appName, sparkHome, sparkConfigMap, sparkExecutorEnvMap, + sparkJars, sparkPackages) + stopifnot(exists(".sparkRjsc", envir = .sparkREnv)) + } - assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv) - hiveCtx + if (exists(".sparkRsession", envir = .sparkREnv)) { + sparkSession <- get(".sparkRsession", envir = .sparkREnv) + # Apply config to Spark Context and Spark Session if already there + # Cannot change enableHiveSupport + callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "setSparkContextSessionConf", + sparkSession, + sparkConfigMap) + } else { + jsc <- get(".sparkRjsc", envir = .sparkREnv) + sparkSession <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "getOrCreateSparkSession", + jsc, + sparkConfigMap, + enableHiveSupport) + assign(".sparkRsession", sparkSession, envir = .sparkREnv) + } + sparkSession } #' Assigns a group ID to all the jobs started by this thread until the group ID is set to a diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index b1b8adaa66a25..aafb34472feb1 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -317,6 +317,15 @@ convertEnvsToList <- function(keys, vals) { }) } +# Utility function to merge 2 environments with the second overriding values in the first +# env1 is changed in place +overrideEnvs <- function(env1, env2) { + lapply(ls(env2), + function(name) { + env1[[name]] <- env2[[name]] + }) +} + # Utility function to capture the varargs into environment object varargsToEnv <- function(...) { # Based on http://stackoverflow.com/a/3057419/4577954 diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R index 90a3761e41f82..8a8111a8c5419 100644 --- a/R/pkg/inst/profile/shell.R +++ b/R/pkg/inst/profile/shell.R @@ -18,17 +18,17 @@ .First <- function() { home <- Sys.getenv("SPARK_HOME") .libPaths(c(file.path(home, "R", "lib"), .libPaths())) - Sys.setenv(NOAWT=1) + Sys.setenv(NOAWT = 1) # Make sure SparkR package is the last loaded one old <- getOption("defaultPackages") options(defaultPackages = c(old, "SparkR")) - sc <- SparkR::sparkR.init() - assign("sc", sc, envir=.GlobalEnv) - sqlContext <- SparkR::sparkRSQL.init(sc) + spark <- SparkR::sparkR.session() + assign("spark", spark, envir = .GlobalEnv) + sc <- SparkR:::callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", spark) + assign("sc", sc, envir = .GlobalEnv) sparkVer <- SparkR:::callJMethod(sc, "version") - assign("sqlContext", sqlContext, envir=.GlobalEnv) cat("\n Welcome to") cat("\n") cat(" ____ __", "\n") @@ -43,5 +43,5 @@ cat(" /_/", "\n") cat("\n") - cat("\n Spark context is available as sc, SQL context is available as sqlContext\n") + cat("\n SparkSession available as 'spark'.\n") } diff --git a/R/pkg/inst/tests/testthat/jarTest.R b/R/pkg/inst/tests/testthat/jarTest.R index d68bb20950b00..84e4845f180b3 100644 --- a/R/pkg/inst/tests/testthat/jarTest.R +++ b/R/pkg/inst/tests/testthat/jarTest.R @@ -16,7 +16,7 @@ # library(SparkR) -sc <- sparkR.init() +sparkSession <- sparkR.session() helloTest <- SparkR:::callJStatic("sparkR.test.hello", "helloWorld", @@ -27,6 +27,6 @@ basicFunction <- SparkR:::callJStatic("sparkR.test.basicFunction", 2L, 2L) -sparkR.stop() +sparkR.session.stop() output <- c(helloTest, basicFunction) writeLines(output) diff --git a/R/pkg/inst/tests/testthat/packageInAJarTest.R b/R/pkg/inst/tests/testthat/packageInAJarTest.R index c26b28b78dee8..940c91f376cd5 100644 --- a/R/pkg/inst/tests/testthat/packageInAJarTest.R +++ b/R/pkg/inst/tests/testthat/packageInAJarTest.R @@ -17,13 +17,13 @@ library(SparkR) library(sparkPackageTest) -sc <- sparkR.init() +sparkSession <- sparkR.session() run1 <- myfunc(5L) run2 <- myfunc(-4L) -sparkR.stop() +sparkR.session.stop() if (run1 != 6) quit(save = "no", status = 1) diff --git a/R/pkg/inst/tests/testthat/test_Serde.R b/R/pkg/inst/tests/testthat/test_Serde.R index dddce54d70443..96fb6dda26450 100644 --- a/R/pkg/inst/tests/testthat/test_Serde.R +++ b/R/pkg/inst/tests/testthat/test_Serde.R @@ -17,7 +17,7 @@ context("SerDe functionality") -sc <- sparkR.init() +sparkSession <- sparkR.session() test_that("SerDe of primitive types", { x <- callJStatic("SparkRHandler", "echo", 1L) diff --git a/R/pkg/inst/tests/testthat/test_binaryFile.R b/R/pkg/inst/tests/testthat/test_binaryFile.R index 976a7558a816d..b69f017de81d1 100644 --- a/R/pkg/inst/tests/testthat/test_binaryFile.R +++ b/R/pkg/inst/tests/testthat/test_binaryFile.R @@ -18,7 +18,8 @@ context("functions on binary files") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) mockFile <- c("Spark is pretty.", "Spark is awesome.") diff --git a/R/pkg/inst/tests/testthat/test_binary_function.R b/R/pkg/inst/tests/testthat/test_binary_function.R index 7bad4d2a7e106..6f51d20687277 100644 --- a/R/pkg/inst/tests/testthat/test_binary_function.R +++ b/R/pkg/inst/tests/testthat/test_binary_function.R @@ -18,7 +18,8 @@ context("binary functions") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Data nums <- 1:10 diff --git a/R/pkg/inst/tests/testthat/test_broadcast.R b/R/pkg/inst/tests/testthat/test_broadcast.R index 8be6efc3dbed3..cf1d43277105e 100644 --- a/R/pkg/inst/tests/testthat/test_broadcast.R +++ b/R/pkg/inst/tests/testthat/test_broadcast.R @@ -18,7 +18,8 @@ context("broadcast variables") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Partitioned data nums <- 1:2 diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 126484c995fb3..f123187adf3ef 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -56,31 +56,33 @@ test_that("Check masked functions", { test_that("repeatedly starting and stopping SparkR", { for (i in 1:4) { - sc <- sparkR.init() + sc <- suppressWarnings(sparkR.init()) rdd <- parallelize(sc, 1:20, 2L) expect_equal(count(rdd), 20) - sparkR.stop() + suppressWarnings(sparkR.stop()) } }) -test_that("repeatedly starting and stopping SparkR SQL", { - for (i in 1:4) { - sc <- sparkR.init() - sqlContext <- sparkRSQL.init(sc) - df <- createDataFrame(data.frame(a = 1:20)) - expect_equal(count(df), 20) - sparkR.stop() - } -}) +# Does not work consistently even with Hive off +# nolint start +# test_that("repeatedly starting and stopping SparkR", { +# for (i in 1:4) { +# sparkR.session(enableHiveSupport = FALSE) +# df <- createDataFrame(data.frame(dummy=1:i)) +# expect_equal(count(df), i) +# sparkR.session.stop() +# Sys.sleep(5) # Need more time to shutdown Hive metastore +# } +# }) +# nolint end test_that("rdd GC across sparkR.stop", { - sparkR.stop() - sc <- sparkR.init() # sc should get id 0 + sc <- sparkR.sparkContext() # 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() + sparkR.session.stop() - sc <- sparkR.init() # sc should get id 0 again + sc <- sparkR.sparkContext() # sc should get id 0 again # GC rdd1 before creating rdd3 and rdd2 after rm(rdd1) @@ -97,15 +99,17 @@ test_that("rdd GC across sparkR.stop", { }) test_that("job group functions can be called", { - sc <- sparkR.init() + sc <- sparkR.sparkContext() setJobGroup(sc, "groupId", "job description", TRUE) cancelJobGroup(sc, "groupId") clearJobGroup(sc) + sparkR.session.stop() }) test_that("utility function can be called", { - sc <- sparkR.init() + sc <- sparkR.sparkContext() setLogLevel(sc, "ERROR") + sparkR.session.stop() }) test_that("getClientModeSparkSubmitOpts() returns spark-submit args from whitelist", { @@ -156,7 +160,8 @@ test_that("sparkJars sparkPackages as comma-separated strings", { }) test_that("spark.lapply should perform simple transforms", { - sc <- sparkR.init() + sc <- sparkR.sparkContext() doubled <- spark.lapply(sc, 1:10, function(x) { 2 * x }) expect_equal(doubled, as.list(2 * 1:10)) + sparkR.session.stop() }) diff --git a/R/pkg/inst/tests/testthat/test_includePackage.R b/R/pkg/inst/tests/testthat/test_includePackage.R index 8152b448d0870..d6a3766539c02 100644 --- a/R/pkg/inst/tests/testthat/test_includePackage.R +++ b/R/pkg/inst/tests/testthat/test_includePackage.R @@ -18,7 +18,8 @@ context("include R packages") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Partitioned data nums <- 1:2 diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index 59ef15c1e9fd5..c8c5ef2476b32 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -20,10 +20,7 @@ library(testthat) context("MLlib functions") # Tests for MLlib functions in SparkR - -sc <- sparkR.init() - -sqlContext <- sparkRSQL.init(sc) +sparkSession <- sparkR.session() test_that("formula of spark.glm", { training <- suppressWarnings(createDataFrame(iris)) diff --git a/R/pkg/inst/tests/testthat/test_parallelize_collect.R b/R/pkg/inst/tests/testthat/test_parallelize_collect.R index 2552127cc547f..f79a8a70aafb1 100644 --- a/R/pkg/inst/tests/testthat/test_parallelize_collect.R +++ b/R/pkg/inst/tests/testthat/test_parallelize_collect.R @@ -33,7 +33,8 @@ 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() +sparkSession <- sparkR.session() +jsc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Tests diff --git a/R/pkg/inst/tests/testthat/test_rdd.R b/R/pkg/inst/tests/testthat/test_rdd.R index b6c8e1dc6c1b7..429311d2924f0 100644 --- a/R/pkg/inst/tests/testthat/test_rdd.R +++ b/R/pkg/inst/tests/testthat/test_rdd.R @@ -18,7 +18,8 @@ context("basic RDD functions") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Data nums <- 1:10 diff --git a/R/pkg/inst/tests/testthat/test_shuffle.R b/R/pkg/inst/tests/testthat/test_shuffle.R index d3d0f8a24d01c..7d4f342016441 100644 --- a/R/pkg/inst/tests/testthat/test_shuffle.R +++ b/R/pkg/inst/tests/testthat/test_shuffle.R @@ -18,7 +18,8 @@ context("partitionBy, groupByKey, reduceByKey etc.") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Data intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 607bd9c12fa05..fcc2ab3ed6a2b 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -33,26 +33,35 @@ markUtf8 <- function(s) { } setHiveContext <- function(sc) { - ssc <- callJMethod(sc, "sc") - hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) - }, - error = function(err) { - skip("Hive is not build with SparkSQL, skipped") - }) - assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv) - hiveCtx + if (exists(".testHiveSession", envir = .sparkREnv)) { + hiveSession <- get(".testHiveSession", envir = .sparkREnv) + } else { + # initialize once and reuse + ssc <- callJMethod(sc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, + error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + hiveSession <- callJMethod(hiveCtx, "sparkSession") + } + previousSession <- get(".sparkRsession", envir = .sparkREnv) + assign(".sparkRsession", hiveSession, envir = .sparkREnv) + assign(".prevSparkRsession", previousSession, envir = .sparkREnv) + hiveSession } unsetHiveContext <- function() { - remove(".sparkRHivesc", envir = .sparkREnv) + previousSession <- get(".prevSparkRsession", envir = .sparkREnv) + assign(".sparkRsession", previousSession, envir = .sparkREnv) + remove(".prevSparkRsession", envir = .sparkREnv) } # Tests for SparkSQL functions in SparkR -sc <- sparkR.init() - -sqlContext <- sparkRSQL.init(sc) +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) mockLines <- c("{\"name\":\"Michael\"}", "{\"name\":\"Andy\", \"age\":30}", @@ -79,7 +88,16 @@ complexTypeJsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLinesComplexType, complexTypeJsonPath) test_that("calling sparkRSQL.init returns existing SQL context", { - expect_equal(sparkRSQL.init(sc), sqlContext) + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sqlContext) +}) + +test_that("calling sparkRSQL.init returns existing SparkSession", { + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sparkSession) +}) + +test_that("calling sparkR.session returns existing SparkSession", { + expect_equal(sparkR.session(), sparkSession) }) test_that("infer types and check types", { @@ -431,6 +449,7 @@ test_that("read/write json files", { }) test_that("jsonRDD() on a RDD with json string", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) rdd <- parallelize(sc, mockLines) expect_equal(count(rdd), 3) df <- suppressWarnings(jsonRDD(sqlContext, rdd)) @@ -2228,7 +2247,6 @@ test_that("gapply() on a DataFrame", { }) test_that("Window functions on a DataFrame", { - setHiveContext(sc) df <- createDataFrame(list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), schema = c("key", "value")) ws <- orderBy(window.partitionBy("key"), "value") @@ -2253,10 +2271,10 @@ test_that("Window functions on a DataFrame", { result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) names(result) <- c("key", "value") expect_equal(result, expected) - unsetHiveContext() }) test_that("createDataFrame sqlContext parameter backward compatibility", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) a <- 1:3 b <- c("a", "b", "c") ldf <- data.frame(a, b) @@ -2283,7 +2301,6 @@ test_that("createDataFrame sqlContext parameter backward compatibility", { test_that("randomSplit", { num <- 4000 df <- createDataFrame(data.frame(id = 1:num)) - weights <- c(2, 3, 5) df_list <- randomSplit(df, weights) expect_equal(length(weights), length(df_list)) @@ -2298,6 +2315,41 @@ test_that("randomSplit", { expect_true(all(sapply(abs(counts / num - weights / sum(weights)), function(e) { e < 0.05 }))) }) +test_that("Change config on SparkSession", { + # first, set it to a random but known value + conf <- callJMethod(sparkSession, "conf") + property <- paste0("spark.testing.", as.character(runif(1))) + value1 <- as.character(runif(1)) + callJMethod(conf, "set", property, value1) + + # next, change the same property to the new value + value2 <- as.character(runif(1)) + l <- list(value2) + names(l) <- property + sparkR.session(sparkConfig = l) + + conf <- callJMethod(sparkSession, "conf") + newValue <- callJMethod(conf, "get", property, "") + expect_equal(value2, newValue) + + value <- as.character(runif(1)) + sparkR.session(spark.app.name = "sparkSession test", spark.testing.r.session.r = value) + conf <- callJMethod(sparkSession, "conf") + appNameValue <- callJMethod(conf, "get", "spark.app.name", "") + testValue <- callJMethod(conf, "get", "spark.testing.r.session.r", "") + expect_equal(appNameValue, "sparkSession test") + expect_equal(testValue, value) +}) + +test_that("enableHiveSupport on SparkSession", { + setHiveContext(sc) + unsetHiveContext() + # if we are still here, it must be built with hive + conf <- callJMethod(sparkSession, "conf") + value <- callJMethod(conf, "get", "spark.sql.catalogImplementation", "") + expect_equal(value, "hive") +}) + unlink(parquetPath) unlink(jsonPath) unlink(jsonPathNa) diff --git a/R/pkg/inst/tests/testthat/test_take.R b/R/pkg/inst/tests/testthat/test_take.R index c2c724cdc762f..daf5e41abe13f 100644 --- a/R/pkg/inst/tests/testthat/test_take.R +++ b/R/pkg/inst/tests/testthat/test_take.R @@ -30,10 +30,11 @@ strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", "raising me. But they're both dead now. I didn't kill them. Honest.") # JavaSparkContext handle -jsc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) test_that("take() gives back the original elements in correct count and order", { - numVectorRDD <- parallelize(jsc, numVector, 10) + numVectorRDD <- parallelize(sc, 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 @@ -42,20 +43,20 @@ test_that("take() gives back the original elements in correct count and order", 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) + numListRDD <- parallelize(sc, numList, 1) + numListRDD2 <- parallelize(sc, 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) + strVectorRDD <- parallelize(sc, strVector, 2) + strVectorRDD2 <- parallelize(sc, 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) + strListRDD <- parallelize(sc, strList, 4) + strListRDD2 <- parallelize(sc, strList, 1) expect_equal(take(strListRDD, 3), as.list(head(strList, n = 3))) expect_equal(take(strListRDD2, 1), as.list(head(strList, n = 1))) diff --git a/R/pkg/inst/tests/testthat/test_textFile.R b/R/pkg/inst/tests/testthat/test_textFile.R index e64ef1bb31a3a..7b2cc74753fe2 100644 --- a/R/pkg/inst/tests/testthat/test_textFile.R +++ b/R/pkg/inst/tests/testthat/test_textFile.R @@ -18,7 +18,8 @@ context("the textFile() function") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) mockFile <- c("Spark is pretty.", "Spark is awesome.") diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 54d2eca50eaf5..21a119a06b937 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -18,7 +18,8 @@ context("functions in utils.R") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session() +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) test_that("convertJListToRList() gives back (deserializes) the original JLists of strings and integers", { @@ -168,3 +169,16 @@ test_that("convertToJSaveMode", { test_that("hashCode", { expect_error(hashCode("bc53d3605e8a5b7de1e8e271c2317645"), NA) }) + +test_that("overrideEnvs", { + config <- new.env() + config[["spark.master"]] <- "foo" + config[["config_only"]] <- "ok" + param <- new.env() + param[["spark.master"]] <- "local" + param[["param_only"]] <- "blah" + overrideEnvs(config, param) + expect_equal(config[["spark.master"]], "local") + expect_equal(config[["param_only"]], "blah") + expect_equal(config[["config_only"]], "ok") +}) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index fe426fa3c7e8a..0a995d2e9d180 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -18,27 +18,61 @@ package org.apache.spark.sql.api.r import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} +import java.util.{Map => JMap} import scala.collection.JavaConverters._ import scala.util.matching.Regex +import org.apache.spark.internal.Logging +import org.apache.spark.SparkContext import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, RelationalGroupedDataset, Row, SaveMode, SQLContext} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.types._ -private[sql] object SQLUtils { +private[sql] object SQLUtils extends Logging { SerDe.registerSqlSerDe((readSqlObject, writeSqlObject)) - def createSQLContext(jsc: JavaSparkContext): SQLContext = { - SQLContext.getOrCreate(jsc.sc) + private[this] def withHiveExternalCatalog(sc: SparkContext): SparkContext = { + sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive") + sc } - def getJavaSparkContext(sqlCtx: SQLContext): JavaSparkContext = { - new JavaSparkContext(sqlCtx.sparkContext) + def getOrCreateSparkSession( + jsc: JavaSparkContext, + sparkConfigMap: JMap[Object, Object], + enableHiveSupport: Boolean): SparkSession = { + val spark = if (SparkSession.hiveClassesArePresent && enableHiveSupport) { + SparkSession.builder().sparkContext(withHiveExternalCatalog(jsc.sc)).getOrCreate() + } else { + if (enableHiveSupport) { + logWarning("SparkR: enableHiveSupport is requested for SparkSession but " + + "Spark is not built with Hive; falling back to without Hive support.") + } + SparkSession.builder().sparkContext(jsc.sc).getOrCreate() + } + setSparkContextSessionConf(spark, sparkConfigMap) + spark + } + + def setSparkContextSessionConf( + spark: SparkSession, + sparkConfigMap: JMap[Object, Object]): Unit = { + for ((name, value) <- sparkConfigMap.asScala) { + spark.conf.set(name.toString, value.toString) + } + for ((name, value) <- sparkConfigMap.asScala) { + spark.sparkContext.conf.set(name.toString, value.toString) + } + } + + def getJavaSparkContext(spark: SparkSession): JavaSparkContext = { + new JavaSparkContext(spark.sparkContext) } def createStructType(fields : Seq[StructField]): StructType = { @@ -95,10 +129,10 @@ private[sql] object SQLUtils { StructField(name, dtObj, nullable) } - def createDF(rdd: RDD[Array[Byte]], schema: StructType, sqlContext: SQLContext): DataFrame = { + def createDF(rdd: RDD[Array[Byte]], schema: StructType, sparkSession: SparkSession): DataFrame = { val num = schema.fields.length val rowRDD = rdd.map(bytesToRow(_, schema)) - sqlContext.createDataFrame(rowRDD, schema) + sparkSession.createDataFrame(rowRDD, schema) } def dfToRowRDD(df: DataFrame): JavaRDD[Array[Byte]] = { @@ -191,18 +225,18 @@ private[sql] object SQLUtils { } def loadDF( - sqlContext: SQLContext, + sparkSession: SparkSession, source: String, options: java.util.Map[String, String]): DataFrame = { - sqlContext.read.format(source).options(options).load() + sparkSession.read.format(source).options(options).load() } def loadDF( - sqlContext: SQLContext, + sparkSession: SparkSession, source: String, schema: StructType, options: java.util.Map[String, String]): DataFrame = { - sqlContext.read.format(source).schema(schema).options(options).load() + sparkSession.read.format(source).schema(schema).options(options).load() } def readSqlObject(dis: DataInputStream, dataType: Char): Object = { @@ -227,4 +261,22 @@ private[sql] object SQLUtils { false } } + + def getTables(sparkSession: SparkSession, databaseName: String): DataFrame = { + databaseName match { + case n: String if n != null && n.trim.nonEmpty => + Dataset.ofRows(sparkSession, ShowTablesCommand(Some(n), None)) + case _ => + Dataset.ofRows(sparkSession, ShowTablesCommand(None, None)) + } + } + + def getTableNames(sparkSession: SparkSession, databaseName: String): Array[String] = { + databaseName match { + case n: String if n != null && n.trim.nonEmpty => + sparkSession.catalog.listTables(n).collect().map(_.name) + case _ => + sparkSession.catalog.listTables().collect().map(_.name) + } + } } From 2859ea3ecc41d12e03231cc50aac63bf22e91047 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 17 Jun 2016 21:58:10 -0700 Subject: [PATCH 0710/1470] [SPARK-16020][SQL] Fix complete mode aggregation with console sink ## What changes were proposed in this pull request? We cannot use `limit` on DataFrame in ConsoleSink because it will use a wrong planner. This PR just collects `DataFrame` and calls `show` on a batch DataFrame based on the result. This is fine since ConsoleSink is only for debugging. ## How was this patch tested? Manually confirmed ConsoleSink now works with complete mode aggregation. Author: Shixiong Zhu Closes #13740 from zsxwing/complete-console. (cherry picked from commit d0ac0e6f433bfccf4ced3743a2526f67fdb5c38e) Signed-off-by: Shixiong Zhu --- .../spark/sql/execution/streaming/Sink.scala | 3 + .../sql/execution/streaming/console.scala | 4 +- .../streaming/ConsoleSinkSuite.scala | 99 +++++++++++++++++++ 3 files changed, 105 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala index e641e09b56adf..2571b59be54f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala @@ -30,6 +30,9 @@ trait Sink { * Adds a batch of data to this sink. The data for a given `batchId` is deterministic and if * this method is called more than once with the same batchId (which will happen in the case of * failures), then `data` should only be added once. + * + * Note: You cannot apply any operators on `data` except consuming it (e.g., `collect/foreach`). + * Otherwise, you may get a wrong result. */ def addBatch(batchId: Long, data: DataFrame): Unit } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index 2ec2a3c3c4a49..e8b9712d19cd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -45,7 +45,9 @@ class ConsoleSink(options: Map[String, String]) extends Sink with Logging { println(batchIdStr) println("-------------------------------------------") // scalastyle:off println - data.show(numRowsToShow, isTruncated) + data.sparkSession.createDataFrame( + data.sparkSession.sparkContext.parallelize(data.collect()), data.schema) + .show(numRowsToShow, isTruncated) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala new file mode 100644 index 0000000000000..e853d8c465419 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.{ByteArrayOutputStream, PrintStream} +import java.nio.charset.StandardCharsets.UTF_8 + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.streaming.StreamTest + +class ConsoleSinkSuite extends StreamTest with BeforeAndAfter { + + import testImplicits._ + + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("SPARK-16020 Complete mode aggregation with console sink") { + withTempDir { checkpointLocation => + val origOut = System.out + val stdout = new ByteArrayOutputStream() + try { + // Hook Java System.out.println + System.setOut(new PrintStream(stdout)) + // Hook Scala println + Console.withOut(stdout) { + val input = MemoryStream[String] + val df = input.toDF().groupBy("value").count() + val query = df.writeStream + .format("console") + .outputMode("complete") + .option("checkpointLocation", checkpointLocation.getAbsolutePath) + .start() + input.addData("a") + query.processAllAvailable() + input.addData("a", "b") + query.processAllAvailable() + input.addData("a", "b", "c") + query.processAllAvailable() + query.stop() + } + System.out.flush() + } finally { + System.setOut(origOut) + } + + val expected = """------------------------------------------- + |Batch: 0 + |------------------------------------------- + |+-----+-----+ + ||value|count| + |+-----+-----+ + || a| 1| + |+-----+-----+ + | + |------------------------------------------- + |Batch: 1 + |------------------------------------------- + |+-----+-----+ + ||value|count| + |+-----+-----+ + || a| 2| + || b| 1| + |+-----+-----+ + | + |------------------------------------------- + |Batch: 2 + |------------------------------------------- + |+-----+-----+ + ||value|count| + |+-----+-----+ + || a| 3| + || b| 2| + || c| 1| + |+-----+-----+ + | + |""".stripMargin + assert(expected === new String(stdout.toByteArray, UTF_8)) + } + } + +} From 2066258eff18b1c98ac30d59b224c40a9af1c482 Mon Sep 17 00:00:00 2001 From: andreapasqua Date: Fri, 17 Jun 2016 22:41:05 -0700 Subject: [PATCH 0711/1470] [SPARK-16035][PYSPARK] Fix SparseVector parser assertion for end parenthesis ## What changes were proposed in this pull request? The check on the end parenthesis of the expression to parse was using the wrong variable. I corrected that. ## How was this patch tested? Manual test Author: andreapasqua Closes #13750 from andreapasqua/sparse-vector-parser-assertion-fix. (cherry picked from commit 4c64e88d5ba4c36cbdbc903376492f0f43401e4e) Signed-off-by: Xiangrui Meng --- python/pyspark/mllib/linalg/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index 70509a6d9bece..3a345b2b5638c 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -569,7 +569,7 @@ def parse(s): if start == -1: raise ValueError("Tuple should start with '('") end = s.find(')') - if start == -1: + if end == -1: raise ValueError("Tuple should end with ')'") s = s[start + 1: end].strip() From feeef497d8af0cc3bac962e622fb000bd42b63a3 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Fri, 17 Jun 2016 22:57:38 -0700 Subject: [PATCH 0712/1470] [SPARK-15803] [PYSPARK] Support with statement syntax for SparkSession ## What changes were proposed in this pull request? Support with statement syntax for SparkSession in pyspark ## How was this patch tested? Manually verify it. Although I can add unit test for it, it would affect other unit test because the SparkContext is stopped after the with statement. Author: Jeff Zhang Closes #13541 from zjffdu/SPARK-15803. (cherry picked from commit 898cb652558ce31a9883448334a45ff0265e2241) Signed-off-by: Davies Liu --- python/pyspark/sql/session.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 6edbd5985644c..0c8024e4a8b49 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -581,6 +581,22 @@ def stop(self): """ self._sc.stop() + @since(2.0) + def __enter__(self): + """ + Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax. + """ + return self + + @since(2.0) + def __exit__(self, exc_type, exc_val, exc_tb): + """ + Enable 'with SparkSession.builder.(...).getOrCreate() as session: app' syntax. + + Specifically stop the SparkSession on exit of the with block. + """ + self.stop() + def _test(): import os From 329dfe706572400b77ff03883bde60bcf9d47d3b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 17 Jun 2016 23:41:09 -0700 Subject: [PATCH 0713/1470] [SPARK-16023][SQL] Move InMemoryRelation to its own file ## What changes were proposed in this pull request? Improve readability of `InMemoryTableScanExec.scala`, which has too much stuff in it. ## How was this patch tested? Jenkins Author: Andrew Or Closes #13742 from andrewor14/move-inmemory-relation. (cherry picked from commit 35a2f3c01201ee1e9f6247b2d919a82d933d3888) Signed-off-by: Reynold Xin --- .../execution/columnar/InMemoryRelation.scala | 210 ++++++++++++++++++ .../columnar/InMemoryTableScanExec.scala | 186 +--------------- 2 files changed, 211 insertions(+), 185 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala new file mode 100644 index 0000000000000..c546d4bc979c1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -0,0 +1,210 @@ +/* + * 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.columnar + +import scala.collection.JavaConverters._ + +import org.apache.commons.lang.StringUtils + +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.CollectionAccumulator + + +private[sql] object InMemoryRelation { + def apply( + useCompression: Boolean, + batchSize: Int, + storageLevel: StorageLevel, + child: SparkPlan, + tableName: Option[String]): InMemoryRelation = + new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)() +} + + +/** + * CachedBatch is a cached batch of rows. + * + * @param numRows The total number of rows in this batch + * @param buffers The buffers for serialized columns + * @param stats The stat of columns + */ +private[columnar] +case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) + +private[sql] case class InMemoryRelation( + output: Seq[Attribute], + useCompression: Boolean, + batchSize: Int, + storageLevel: StorageLevel, + @transient child: SparkPlan, + tableName: Option[String])( + @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, + @transient private[sql] var _statistics: Statistics = null, + private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) + extends logical.LeafNode with MultiInstanceRelation { + + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) + + override def producedAttributes: AttributeSet = outputSet + + private[sql] val batchStats: CollectionAccumulator[InternalRow] = + if (_batchStats == null) { + child.sqlContext.sparkContext.collectionAccumulator[InternalRow] + } else { + _batchStats + } + + @transient val partitionStatistics = new PartitionStatistics(output) + + private def computeSizeInBytes = { + val sizeOfRow: Expression = + BindReferences.bindReference( + output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), + partitionStatistics.schema) + + batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum + } + + // Statistics propagation contracts: + // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data + // 2. Only propagate statistics when `_statistics` is non-null + private def statisticsToBePropagated = if (_statistics == null) { + val updatedStats = statistics + if (_statistics == null) null else updatedStats + } else { + _statistics + } + + override def statistics: Statistics = { + if (_statistics == null) { + if (batchStats.value.isEmpty) { + // Underlying columnar RDD hasn't been materialized, no useful statistics information + // available, return the default statistics. + Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) + } else { + // Underlying columnar RDD has been materialized, required information has also been + // collected via the `batchStats` accumulator, compute the final statistics, + // and update `_statistics`. + _statistics = Statistics(sizeInBytes = computeSizeInBytes) + _statistics + } + } else { + // Pre-computed statistics + _statistics + } + } + + // If the cached column buffers were not passed in, we calculate them in the constructor. + // As in Spark, the actual work of caching is lazy. + if (_cachedColumnBuffers == null) { + buildBuffers() + } + + def recache(): Unit = { + _cachedColumnBuffers.unpersist() + _cachedColumnBuffers = null + buildBuffers() + } + + private def buildBuffers(): Unit = { + val output = child.output + val cached = child.execute().mapPartitionsInternal { rowIterator => + new Iterator[CachedBatch] { + def next(): CachedBatch = { + val columnBuilders = output.map { attribute => + ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) + }.toArray + + var rowCount = 0 + var totalSize = 0L + while (rowIterator.hasNext && rowCount < batchSize + && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { + val row = rowIterator.next() + + // Added for SPARK-6082. This assertion can be useful for scenarios when something + // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM + // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat + // hard to decipher. + assert( + row.numFields == columnBuilders.length, + s"Row column number mismatch, expected ${output.size} columns, " + + s"but got ${row.numFields}." + + s"\nRow content: $row") + + var i = 0 + totalSize = 0 + while (i < row.numFields) { + columnBuilders(i).appendFrom(row, i) + totalSize += columnBuilders(i).columnStats.sizeInBytes + i += 1 + } + rowCount += 1 + } + + val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) + .flatMap(_.values)) + + batchStats.add(stats) + CachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, stats) + } + + def hasNext: Boolean = rowIterator.hasNext + } + }.persist(storageLevel) + + cached.setName( + tableName.map(n => s"In-memory table $n") + .getOrElse(StringUtils.abbreviate(child.toString, 1024))) + _cachedColumnBuffers = cached + } + + def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { + InMemoryRelation( + newOutput, useCompression, batchSize, storageLevel, child, tableName)( + _cachedColumnBuffers, statisticsToBePropagated, batchStats) + } + + override def newInstance(): this.type = { + new InMemoryRelation( + output.map(_.newInstance()), + useCompression, + batchSize, + storageLevel, + child, + tableName)( + _cachedColumnBuffers, + statisticsToBePropagated, + batchStats).asInstanceOf[this.type] + } + + def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers + + override protected def otherCopyArgs: Seq[AnyRef] = + Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index ce630bcbca4cd..2695f356cd3e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,200 +17,16 @@ package org.apache.spark.sql.execution.columnar -import scala.collection.JavaConverters._ - -import org.apache.commons.lang.StringUtils - -import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{AccumulatorContext, CollectionAccumulator, LongAccumulator} - - -private[sql] object InMemoryRelation { - def apply( - useCompression: Boolean, - batchSize: Int, - storageLevel: StorageLevel, - child: SparkPlan, - tableName: Option[String]): InMemoryRelation = - new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)() -} - -/** - * CachedBatch is a cached batch of rows. - * - * @param numRows The total number of rows in this batch - * @param buffers The buffers for serialized columns - * @param stats The stat of columns - */ -private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) - -private[sql] case class InMemoryRelation( - output: Seq[Attribute], - useCompression: Boolean, - batchSize: Int, - storageLevel: StorageLevel, - @transient child: SparkPlan, - tableName: Option[String])( - @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - @transient private[sql] var _statistics: Statistics = null, - private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) - extends logical.LeafNode with MultiInstanceRelation { - - override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) - - override def producedAttributes: AttributeSet = outputSet - - private[sql] val batchStats: CollectionAccumulator[InternalRow] = - if (_batchStats == null) { - child.sqlContext.sparkContext.collectionAccumulator[InternalRow] - } else { - _batchStats - } - - @transient val partitionStatistics = new PartitionStatistics(output) - - private def computeSizeInBytes = { - val sizeOfRow: Expression = - BindReferences.bindReference( - output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), - partitionStatistics.schema) - - batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum - } - - // Statistics propagation contracts: - // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data - // 2. Only propagate statistics when `_statistics` is non-null - private def statisticsToBePropagated = if (_statistics == null) { - val updatedStats = statistics - if (_statistics == null) null else updatedStats - } else { - _statistics - } - - override def statistics: Statistics = { - if (_statistics == null) { - if (batchStats.value.isEmpty) { - // Underlying columnar RDD hasn't been materialized, no useful statistics information - // available, return the default statistics. - Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) - } else { - // Underlying columnar RDD has been materialized, required information has also been - // collected via the `batchStats` accumulator, compute the final statistics, - // and update `_statistics`. - _statistics = Statistics(sizeInBytes = computeSizeInBytes) - _statistics - } - } else { - // Pre-computed statistics - _statistics - } - } - - // If the cached column buffers were not passed in, we calculate them in the constructor. - // As in Spark, the actual work of caching is lazy. - if (_cachedColumnBuffers == null) { - buildBuffers() - } - def recache(): Unit = { - _cachedColumnBuffers.unpersist() - _cachedColumnBuffers = null - buildBuffers() - } - - private def buildBuffers(): Unit = { - val output = child.output - val cached = child.execute().mapPartitionsInternal { rowIterator => - new Iterator[CachedBatch] { - def next(): CachedBatch = { - val columnBuilders = output.map { attribute => - ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) - }.toArray - - var rowCount = 0 - var totalSize = 0L - while (rowIterator.hasNext && rowCount < batchSize - && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) { - val row = rowIterator.next() - - // Added for SPARK-6082. This assertion can be useful for scenarios when something - // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM - // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat - // hard to decipher. - assert( - row.numFields == columnBuilders.length, - s"Row column number mismatch, expected ${output.size} columns, " + - s"but got ${row.numFields}." + - s"\nRow content: $row") - - var i = 0 - totalSize = 0 - while (i < row.numFields) { - columnBuilders(i).appendFrom(row, i) - totalSize += columnBuilders(i).columnStats.sizeInBytes - i += 1 - } - rowCount += 1 - } - - val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) - .flatMap(_.values)) - - batchStats.add(stats) - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - } - - def hasNext: Boolean = rowIterator.hasNext - } - }.persist(storageLevel) - - cached.setName( - tableName.map(n => s"In-memory table $n") - .getOrElse(StringUtils.abbreviate(child.toString, 1024))) - _cachedColumnBuffers = cached - } - - def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { - InMemoryRelation( - newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, statisticsToBePropagated, batchStats) - } - - override def newInstance(): this.type = { - new InMemoryRelation( - output.map(_.newInstance()), - useCompression, - batchSize, - storageLevel, - child, - tableName)( - _cachedColumnBuffers, - statisticsToBePropagated, - batchStats).asInstanceOf[this.type] - } - - def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers - - override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) -} private[sql] case class InMemoryTableScanExec( attributes: Seq[Attribute], From f159eb52171ee595a10c0a130ecabcfcf0a191af Mon Sep 17 00:00:00 2001 From: Josh Howes Date: Fri, 17 Jun 2016 23:43:31 -0700 Subject: [PATCH 0714/1470] [SPARK-15973][PYSPARK] Fix GroupedData Documentation *This contribution is my original work and that I license the work to the project under the project's open source license.* ## What changes were proposed in this pull request? Documentation updates to PySpark's GroupedData ## How was this patch tested? Manual Tests Author: Josh Howes Author: Josh Howes Closes #13724 from josh-howes/bugfix/SPARK-15973. (cherry picked from commit e574c9973ddbef023c066ccd6f771ab01cbf2d88) Signed-off-by: Reynold Xin --- python/pyspark/sql/group.py | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 6987af69cfb50..a4232065540ea 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -27,7 +27,7 @@ def dfapi(f): def _api(self): name = f.__name__ - jdf = getattr(self._jdf, name)() + jdf = getattr(self._jgd, name)() return DataFrame(jdf, self.sql_ctx) _api.__name__ = f.__name__ _api.__doc__ = f.__doc__ @@ -35,9 +35,9 @@ def _api(self): def df_varargs_api(f): - def _api(self, *args): + def _api(self, *cols): name = f.__name__ - jdf = getattr(self._jdf, name)(_to_seq(self.sql_ctx._sc, args)) + jdf = getattr(self._jgd, name)(_to_seq(self.sql_ctx._sc, cols)) return DataFrame(jdf, self.sql_ctx) _api.__name__ = f.__name__ _api.__doc__ = f.__doc__ @@ -54,8 +54,8 @@ class GroupedData(object): .. versionadded:: 1.3 """ - def __init__(self, jdf, sql_ctx): - self._jdf = jdf + def __init__(self, jgd, sql_ctx): + self._jgd = jgd self.sql_ctx = sql_ctx @ignore_unicode_prefix @@ -83,11 +83,11 @@ def agg(self, *exprs): """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): - jdf = self._jdf.agg(exprs[0]) + jdf = self._jgd.agg(exprs[0]) else: # Columns assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" - jdf = self._jdf.agg(exprs[0]._jc, + jdf = self._jgd.agg(exprs[0]._jc, _to_seq(self.sql_ctx._sc, [c._jc for c in exprs[1:]])) return DataFrame(jdf, self.sql_ctx) @@ -178,18 +178,18 @@ def pivot(self, pivot_col, values=None): :param pivot_col: Name of the column to pivot. :param values: List of values that will be translated to columns in the output DataFrame. - // Compute the sum of earnings for each year by course with each course as a separate column + # Compute the sum of earnings for each year by course with each course as a separate column >>> df4.groupBy("year").pivot("course", ["dotNET", "Java"]).sum("earnings").collect() [Row(year=2012, dotNET=15000, Java=20000), Row(year=2013, dotNET=48000, Java=30000)] - // Or without specifying column values (less efficient) + # Or without specifying column values (less efficient) >>> df4.groupBy("year").pivot("course").sum("earnings").collect() [Row(year=2012, Java=20000, dotNET=15000), Row(year=2013, Java=30000, dotNET=48000)] """ if values is None: - jgd = self._jdf.pivot(pivot_col) + jgd = self._jgd.pivot(pivot_col) else: - jgd = self._jdf.pivot(pivot_col, values) + jgd = self._jgd.pivot(pivot_col, values) return GroupedData(jgd, self.sql_ctx) From 8d2fc010b36a8ecbbecdfd0ea331b9a5ffaf716d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 18 Jun 2016 10:32:27 -0700 Subject: [PATCH 0715/1470] [SPARK-16036][SPARK-16037][SQL] fix various table insertion problems ## What changes were proposed in this pull request? The current table insertion has some weird behaviours: 1. inserting into a partitioned table with mismatch columns has confusing error message for hive table, and wrong result for datasource table 2. inserting into a partitioned table without partition list has wrong result for hive table. This PR fixes these 2 problems. ## How was this patch tested? new test in hive `SQLQuerySuite` Author: Wenchen Fan Closes #13754 from cloud-fan/insert2. (cherry picked from commit 3d010c837582c23b5ddf65602213e3772b418f08) Signed-off-by: Yin Huai --- .../sql/catalyst/analysis/Analyzer.scala | 37 +------ .../plans/logical/basicLogicalOperators.scala | 6 +- .../sql/execution/datasources/rules.scala | 100 +++++++++++------- .../spark/sql/internal/SessionState.scala | 4 +- .../spark/sql/sources/InsertSuite.scala | 6 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 43 -------- .../spark/sql/hive/HiveSessionCatalog.scala | 1 - .../spark/sql/hive/HiveSessionState.scala | 3 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 21 ---- .../hive/execution/HiveComparisonTest.scala | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 35 ------ .../sql/hive/execution/SQLQuerySuite.scala | 32 ++++++ 12 files changed, 104 insertions(+), 185 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 a5755616329ab..96f2e38946f1c 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 @@ -452,42 +452,7 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) if child.resolved => - val table = lookupTableFromCatalog(u) - // adding the table's partitions or validate the query's partition info - table match { - case relation: CatalogRelation if relation.catalogTable.partitionColumns.nonEmpty => - val tablePartitionNames = relation.catalogTable.partitionColumns.map(_.name) - if (parts.keys.nonEmpty) { - // the query's partitioning must match the table's partitioning - // this is set for queries like: insert into ... partition (one = "a", two = ) - // TODO: add better checking to pre-inserts to avoid needing this here - if (tablePartitionNames.size != parts.keySet.size) { - throw new AnalysisException( - s"""Requested partitioning does not match the ${u.tableIdentifier} table: - |Requested partitions: ${parts.keys.mkString(",")} - |Table partitions: ${tablePartitionNames.mkString(",")}""".stripMargin) - } - // Assume partition columns are correctly placed at the end of the child's output - i.copy(table = EliminateSubqueryAliases(table)) - } else { - // Set up the table's partition scheme with all dynamic partitions by moving partition - // columns to the end of the column list, in partition order. - val (inputPartCols, columns) = child.output.partition { attr => - tablePartitionNames.contains(attr.name) - } - // All partition columns are dynamic because this InsertIntoTable had no partitioning - val partColumns = tablePartitionNames.map { name => - inputPartCols.find(_.name == name).getOrElse( - throw new AnalysisException(s"Cannot find partition column $name")) - } - i.copy( - table = EliminateSubqueryAliases(table), - partition = tablePartitionNames.map(_ -> None).toMap, - child = Project(columns ++ partColumns, child)) - } - case _ => - i.copy(table = EliminateSubqueryAliases(table)) - } + i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u))) case u: UnresolvedRelation => val table = u.tableIdentifier if (table.database.isDefined && conf.runSQLonFile && diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 6c3eb3a5a28ab..69b8b059fde1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -369,10 +369,8 @@ case class InsertIntoTable( if (table.output.isEmpty) { None } else { - val numDynamicPartitions = partition.values.count(_.isEmpty) - val (partitionColumns, dataColumns) = table.output - .partition(a => partition.keySet.contains(a.name)) - Some(dataColumns ++ partitionColumns.takeRight(numDynamicPartitions)) + val staticPartCols = partition.filter(_._2.isDefined).keySet + Some(table.output.filterNot(a => staticPartCols.contains(a.name))) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 543389efd5b95..5963c53a1b1cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -21,7 +21,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.catalog.SessionCatalog +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ @@ -62,53 +62,79 @@ private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[Lo } /** - * A rule to do pre-insert data type casting and field renaming. Before we insert into - * an [[InsertableRelation]], we will use this rule to make sure that - * the columns to be inserted have the correct data type and fields have the correct names. + * Preprocess the [[InsertIntoTable]] plan. Throws exception if the number of columns mismatch, or + * specified partition columns are different from the existing partition columns in the target + * table. It also does data type casting and field renaming, to make sure that the columns to be + * inserted have the correct data type and fields have the correct names. */ -private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - // Wait until children are resolved. - case p: LogicalPlan if !p.childrenResolved => p - - // We are inserting into an InsertableRelation or HadoopFsRelation. - case i @ InsertIntoTable( - l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _), _, child, _, _) => - // 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) { - sys.error( - s"$l requires that the data to be inserted have the same number of columns as the " + - s"target table: target table has ${l.output.size} column(s) but " + - s"the inserted data has ${child.output.size} column(s).") - } - castAndRenameChildOutput(i, l.output, child) +private[sql] object PreprocessTableInsertion extends Rule[LogicalPlan] { + private def preprocess( + insert: InsertIntoTable, + tblName: String, + partColNames: Seq[String]): InsertIntoTable = { + + val expectedColumns = insert.expectedColumns + if (expectedColumns.isDefined && expectedColumns.get.length != insert.child.schema.length) { + throw new AnalysisException( + s"Cannot insert into table $tblName because the number of columns are different: " + + s"need ${expectedColumns.get.length} columns, " + + s"but query has ${insert.child.schema.length} columns.") + } + + if (insert.partition.nonEmpty) { + // the query's partitioning must match the table's partitioning + // this is set for queries like: insert into ... partition (one = "a", two = ) + if (insert.partition.keySet != partColNames.toSet) { + throw new AnalysisException( + s""" + |Requested partitioning does not match the table $tblName: + |Requested partitions: ${insert.partition.keys.mkString(",")} + |Table partitions: ${partColNames.mkString(",")} + """.stripMargin) + } + expectedColumns.map(castAndRenameChildOutput(insert, _)).getOrElse(insert) + } else { + // All partition columns are dynamic because this InsertIntoTable had no partitioning + expectedColumns.map(castAndRenameChildOutput(insert, _)).getOrElse(insert) + .copy(partition = partColNames.map(_ -> None).toMap) + } } - /** If necessary, cast data types and rename fields to the expected types and names. */ + // TODO: do we really need to rename? def castAndRenameChildOutput( - insertInto: InsertIntoTable, - expectedOutput: Seq[Attribute], - child: LogicalPlan): InsertIntoTable = { - val newChildOutput = expectedOutput.zip(child.output).map { + insert: InsertIntoTable, + expectedOutput: Seq[Attribute]): InsertIntoTable = { + val newChildOutput = expectedOutput.zip(insert.child.output).map { case (expected, actual) => - val needCast = !expected.dataType.sameType(actual.dataType) - // We want to make sure the filed names in the data to be inserted exactly match - // names in the schema. - val needRename = expected.name != actual.name - (needCast, needRename) match { - case (true, _) => Alias(Cast(actual, expected.dataType), expected.name)() - case (false, true) => Alias(actual, expected.name)() - case (_, _) => actual + if (expected.dataType.sameType(actual.dataType) && expected.name == actual.name) { + actual + } else { + Alias(Cast(actual, expected.dataType), expected.name)() } } - if (newChildOutput == child.output) { - insertInto + if (newChildOutput == insert.child.output) { + insert } else { - insertInto.copy(child = Project(newChildOutput, child)) + insert.copy(child = Project(newChildOutput, insert.child)) } } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case i @ InsertIntoTable(table, partition, child, _, _) if table.resolved && child.resolved => + table match { + case relation: CatalogRelation => + val metadata = relation.catalogTable + preprocess(i, metadata.identifier.quotedString, metadata.partitionColumnNames) + case LogicalRelation(h: HadoopFsRelation, _, identifier) => + val tblName = identifier.map(_.quotedString).getOrElse("unknown") + preprocess(i, tblName, h.partitionSchema.map(_.name)) + case LogicalRelation(_: InsertableRelation, _, identifier) => + val tblName = identifier.map(_.quotedString).getOrElse("unknown") + preprocess(i, tblName, Nil) + case other => i + } + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index dc95123d0088b..b033e19ddf06c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.AnalyzeTableCommand -import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, FindDataSourceTable, PreInsertCastAndRename, ResolveDataSource} +import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, FindDataSourceTable, PreprocessTableInsertion, ResolveDataSource} import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager} import org.apache.spark.sql.util.ExecutionListenerManager @@ -111,7 +111,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val analyzer: Analyzer = { new Analyzer(catalog, conf) { override val extendedResolutionRules = - PreInsertCastAndRename :: + PreprocessTableInsertion :: new FindDataSourceTable(sparkSession) :: DataSourceAnalysis :: (if (conf.runSQLonFile) new ResolveDataSource(sparkSession) :: Nil else Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index d7179551d62df..6454d716ec0db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -88,15 +88,13 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { } test("SELECT clause generating a different number of columns is not allowed.") { - val message = intercept[RuntimeException] { + val message = intercept[AnalysisException] { sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a FROM jt """.stripMargin) }.getMessage - assert( - message.contains("requires that the data to be inserted have the same number of columns"), - "SELECT clause generating a different number of columns should not be not allowed." + assert(message.contains("the number of columns are different") ) } 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 224ff3823b594..2e0b5d59b5783 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 @@ -457,49 +457,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log allowExisting) } } - - /** - * Casts input data to correct data types according to table definition before inserting into - * that table. - */ - object PreInsertionCasts extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.transform { - // Wait until children are resolved. - case p: LogicalPlan if !p.childrenResolved => p - - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _, _) => - castChildOutput(p, table, child) - } - - def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) - : LogicalPlan = { - val childOutputDataTypes = child.output.map(_.dataType) - val numDynamicPartitions = p.partition.values.count(_.isEmpty) - val tableOutputDataTypes = - (table.attributes ++ table.partitionKeys.takeRight(numDynamicPartitions)) - .take(child.output.length).map(_.dataType) - - if (childOutputDataTypes == tableOutputDataTypes) { - InsertIntoHiveTable(table, p.partition, p.child, p.overwrite, p.ifNotExists) - } else if (childOutputDataTypes.size == tableOutputDataTypes.size && - childOutputDataTypes.zip(tableOutputDataTypes) - .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(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 { - case (input, output) if input.dataType != output.dataType => - Alias(Cast(input, output.dataType), input.name)() - case (input, _) => input - } - - p.copy(child = logical.Project(castedChildOutput, child)) - } - } - } - } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 4f8aac8c2fcdd..2f6a2207855ec 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -87,7 +87,6 @@ private[sql] class HiveSessionCatalog( val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions val CreateTables: Rule[LogicalPlan] = metastoreCatalog.CreateTables - val PreInsertionCasts: Rule[LogicalPlan] = metastoreCatalog.PreInsertionCasts override def refreshTable(name: TableIdentifier): Unit = { metastoreCatalog.refreshTable(name) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index ca8e5f8223968..2d286715b57b6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -65,8 +65,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) catalog.ParquetConversions :: catalog.OrcConversions :: catalog.CreateTables :: - catalog.PreInsertionCasts :: - PreInsertCastAndRename :: + PreprocessTableInsertion :: DataSourceAnalysis :: (if (conf.runSQLonFile) new ResolveDataSource(sparkSession) :: Nil else Nil) 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 b890b4bffdcfe..c48735142dd00 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 @@ -325,27 +325,6 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef } } - test("Detect table partitioning with correct partition order") { - withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { - sql("CREATE TABLE source (id bigint, part2 string, part1 string, data string)") - val data = (1 to 10).map(i => (i, if ((i % 2) == 0) "even" else "odd", "p", s"data-$i")) - .toDF("id", "part2", "part1", "data") - - data.write.insertInto("source") - checkAnswer(sql("SELECT * FROM source"), data.collect().toSeq) - - // the original data with part1 and part2 at the end - val expected = data.select("id", "data", "part1", "part2") - - sql( - """CREATE TABLE partitioned (id bigint, data string) - |PARTITIONED BY (part1 string, part2 string)""".stripMargin) - spark.table("source").write.insertInto("partitioned") - - checkAnswer(sql("SELECT * FROM partitioned"), expected.collect().toSeq) - } - } - private def testPartitionedHiveSerDeTable(testName: String)(f: String => Unit): Unit = { test(s"Hive SerDe table - $testName") { val hiveTable = "hive_table" 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 a846711b84ec9..f5d2f02d512be 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 @@ -348,6 +348,7 @@ abstract class HiveComparisonTest queryString.replace("../../data", testDataPath)) val containsCommands = originalQuery.analyzed.collectFirst { case _: Command => () + case _: InsertIntoTable => () case _: LogicalInsertIntoHiveTable => () }.nonEmpty 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 e0f6ccf04dd33..a16b5b2e23c3d 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 @@ -1033,41 +1033,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SELECT * FROM boom").queryExecution.analyzed } - test("SPARK-3810: PreInsertionCasts static partitioning support") { - val analyzedPlan = { - loadTestTable("srcpart") - sql("DROP TABLE IF EXISTS withparts") - sql("CREATE TABLE withparts LIKE srcpart") - sql("INSERT INTO TABLE withparts PARTITION(ds='1', hr='2') SELECT key, value FROM src") - .queryExecution.analyzed - } - - assertResult(1, "Duplicated project detected\n" + analyzedPlan) { - analyzedPlan.collect { - case _: Project => () - }.size - } - } - - test("SPARK-3810: PreInsertionCasts dynamic partitioning support") { - val analyzedPlan = { - loadTestTable("srcpart") - sql("DROP TABLE IF EXISTS withparts") - sql("CREATE TABLE withparts LIKE srcpart") - sql("SET hive.exec.dynamic.partition.mode=nonstrict") - - sql("CREATE TABLE IF NOT EXISTS withparts LIKE srcpart") - sql("INSERT INTO TABLE withparts PARTITION(ds, hr) SELECT key, value FROM src") - .queryExecution.analyzed - } - - assertResult(1, "Duplicated project detected\n" + analyzedPlan) { - analyzedPlan.collect { - case _: Project => () - }.size - } - } - test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" 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 9c1f21825315b..46a77dd917fb3 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 @@ -1684,4 +1684,36 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } } + + test("SPARK-16036: better error message when insert into a table with mismatch schema") { + withTable("hive_table", "datasource_table") { + sql("CREATE TABLE hive_table(a INT) PARTITIONED BY (b INT, c INT)") + sql("CREATE TABLE datasource_table(a INT, b INT, c INT) USING parquet PARTITIONED BY (b, c)") + val e1 = intercept[AnalysisException] { + sql("INSERT INTO TABLE hive_table PARTITION(b=1, c=2) SELECT 1, 2, 3") + } + assert(e1.message.contains("the number of columns are different")) + val e2 = intercept[AnalysisException] { + sql("INSERT INTO TABLE datasource_table PARTITION(b=1, c=2) SELECT 1, 2, 3") + } + assert(e2.message.contains("the number of columns are different")) + } + } + + test("SPARK-16037: INSERT statement should match columns by position") { + withTable("hive_table", "datasource_table") { + sql("CREATE TABLE hive_table(a INT) PARTITIONED BY (b INT, c INT)") + sql("CREATE TABLE datasource_table(a INT, b INT, c INT) USING parquet PARTITIONED BY (b, c)") + + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql("INSERT INTO TABLE hive_table SELECT 1, 2 AS c, 3 AS b") + checkAnswer(sql("SELECT a, b, c FROM hive_table"), Row(1, 2, 3)) + sql("INSERT OVERWRITE TABLE hive_table SELECT 1, 2, 3") + checkAnswer(sql("SELECT a, b, c FROM hive_table"), Row(1, 2, 3)) + } + + sql("INSERT INTO TABLE datasource_table SELECT 1, 2 AS c, 3 AS b") + checkAnswer(sql("SELECT a, b, c FROM datasource_table"), Row(1, 2, 3)) + } + } } From ee6eea644fe0197a183385ef5879911ae8ab9ccb Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Sat, 18 Jun 2016 10:41:33 -0700 Subject: [PATCH 0716/1470] [SPARK-16034][SQL] Checks the partition columns when calling dataFrame.write.mode("append").saveAsTable ## What changes were proposed in this pull request? `DataFrameWriter` can be used to append data to existing data source tables. It becomes tricky when partition columns used in `DataFrameWriter.partitionBy(columns)` don't match the actual partition columns of the underlying table. This pull request enforces the check so that the partition columns of these two always match. ## How was this patch tested? Unit test. Author: Sean Zhong Closes #13749 from clockfly/SPARK-16034. (cherry picked from commit ce3b98bae28af72299722f56e4e4ef831f471ec0) Signed-off-by: Yin Huai --- .../command/createDataSourceTables.scala | 9 ++++- .../execution/datasources/DataSource.scala | 39 +++++++++---------- .../sql/execution/command/DDLSuite.scala | 24 ++++++++++++ 3 files changed, 50 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 4918780873bdf..c38eca5156e5a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -242,8 +242,13 @@ case class CreateDataSourceTableAsSelectCommand( bucketSpec = bucketSpec, options = optionsWithPath) - val result = dataSource.write(mode, df) - + val result = try { + dataSource.write(mode, df) + } catch { + case ex: AnalysisException => + logError(s"Failed to write to table ${tableIdent.identifier} in $mode mode", ex) + throw ex + } if (createMetastoreTable) { // We will use the schema of resolved.relation as the schema of the table (instead of // the schema of df). It is important since the nullability may be changed by the relation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 7f3683fc98197..f274fc77daef0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -435,26 +435,25 @@ case class DataSource( // If we are appending to a table that already exists, make sure the partitioning matches // up. If we fail to load the table for whatever reason, ignore the check. if (mode == SaveMode.Append) { - val existingPartitionColumnSet = try { - Some( - resolveRelation() - .asInstanceOf[HadoopFsRelation] - .location - .partitionSpec() - .partitionColumns - .fieldNames - .toSet) - } catch { - case e: Exception => - None - } - - existingPartitionColumnSet.foreach { ex => - if (ex.map(_.toLowerCase) != partitionColumns.map(_.toLowerCase()).toSet) { - throw new AnalysisException( - s"Requested partitioning does not equal existing partitioning: " + - s"$ex != ${partitionColumns.toSet}.") - } + val existingColumns = Try { + resolveRelation() + .asInstanceOf[HadoopFsRelation] + .location + .partitionSpec() + .partitionColumns + .fieldNames + .toSeq + }.getOrElse(Seq.empty[String]) + val sameColumns = + existingColumns.map(_.toLowerCase) == partitionColumns.map(_.toLowerCase) + if (existingColumns.size > 0 && !sameColumns) { + throw new AnalysisException( + s"""Requested partitioning does not match existing partitioning. + |Existing partitioning columns: + | ${existingColumns.mkString(", ")} + |Requested partitioning columns: + | ${partitionColumns.mkString(", ")} + |""".stripMargin) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 7eb2fff91d6e1..8827649d0a0d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1317,4 +1317,28 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assertUnsupported("TRUNCATE TABLE my_tab PARTITION (age=10)") } + test("SPARK-16034 Partition columns should match when appending to existing data source tables") { + import testImplicits._ + val df = Seq((1, 2, 3)).toDF("a", "b", "c") + withTable("partitionedTable") { + df.write.mode("overwrite").partitionBy("a", "b").saveAsTable("partitionedTable") + // Misses some partition columns + intercept[AnalysisException] { + df.write.mode("append").partitionBy("a").saveAsTable("partitionedTable") + } + // Wrong order + intercept[AnalysisException] { + df.write.mode("append").partitionBy("b", "a").saveAsTable("partitionedTable") + } + // Partition columns not specified + intercept[AnalysisException] { + df.write.mode("append").saveAsTable("partitionedTable") + } + assert(sql("select * from partitionedTable").collect().size == 1) + // Inserts new data successfully when partition columns are correctly specified in + // partitionBy(...). + df.write.mode("append").partitionBy("a", "b").saveAsTable("partitionedTable") + assert(sql("select * from partitionedTable").collect().size == 2) + } + } } From dc85bd0a08d7a511c667df2cbf27d6c2085c19dc Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 19 Jun 2016 00:34:52 -0700 Subject: [PATCH 0717/1470] [SPARK-15613] [SQL] Fix incorrect days to millis conversion due to Daylight Saving Time ## What changes were proposed in this pull request? Internally, we use Int to represent a date (the days since 1970-01-01), when we convert that into unix timestamp (milli-seconds since epoch in UTC), we get the offset of a timezone using local millis (the milli-seconds since 1970-01-01 in a timezone), but TimeZone.getOffset() expect unix timestamp, the result could be off by one hour (in Daylight Saving Time (DST) or not). This PR change to use best effort approximate of posix timestamp to lookup the offset. In the event of changing of DST, Some time is not defined (for example, 2016-03-13 02:00:00 PST), or could lead to multiple valid result in UTC (for example, 2016-11-06 01:00:00), this best effort approximate should be enough in practice. ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #13652 from davies/fix_timezone. (cherry picked from commit 001a58960311b07fe80e2f01e473f4987948d06e) Signed-off-by: Davies Liu --- .../sql/catalyst/util/DateTimeUtils.scala | 51 +++++++++++++++++-- .../org/apache/spark/sql/types/DateType.scala | 2 +- .../sql/catalyst/util/DateTimeTestUtils.scala | 40 +++++++++++++++ .../catalyst/util/DateTimeUtilsSuite.scala | 40 +++++++++++++++ 4 files changed, 129 insertions(+), 4 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index e08328a32079e..56bf9a7863e61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -100,8 +100,8 @@ object DateTimeUtils { // reverse of millisToDays def daysToMillis(days: SQLDate): Long = { - val millisUtc = days.toLong * MILLIS_PER_DAY - millisUtc - threadLocalLocalTimeZone.get().getOffset(millisUtc) + val millisLocal = days.toLong * MILLIS_PER_DAY + millisLocal - getOffsetFromLocalMillis(millisLocal, threadLocalLocalTimeZone.get()) } def dateToString(days: SQLDate): String = @@ -850,6 +850,41 @@ object DateTimeUtils { } } + /** + * Lookup the offset for given millis seconds since 1970-01-01 00:00:00 in given timezone. + */ + private def getOffsetFromLocalMillis(millisLocal: Long, tz: TimeZone): Long = { + var guess = tz.getRawOffset + // the actual offset should be calculated based on milliseconds in UTC + val offset = tz.getOffset(millisLocal - guess) + if (offset != guess) { + guess = tz.getOffset(millisLocal - offset) + if (guess != offset) { + // fallback to do the reverse lookup using java.sql.Timestamp + // this should only happen near the start or end of DST + val days = Math.floor(millisLocal.toDouble / MILLIS_PER_DAY).toInt + val year = getYear(days) + val month = getMonth(days) + val day = getDayOfMonth(days) + + var millisOfDay = (millisLocal % MILLIS_PER_DAY).toInt + if (millisOfDay < 0) { + millisOfDay += MILLIS_PER_DAY.toInt + } + val seconds = (millisOfDay / 1000L).toInt + val hh = seconds / 3600 + val mm = seconds / 60 % 60 + val ss = seconds % 60 + val nano = millisOfDay % 1000 * 1000000 + + // create a Timestamp to get the unix timestamp (in UTC) + val timestamp = new Timestamp(year - 1900, month - 1, day, hh, mm, ss, nano) + guess = (millisLocal - timestamp.getTime).toInt + } + } + guess + } + /** * Returns a timestamp of given timezone from utc timestamp, with the same string * representation in their timezone. @@ -866,7 +901,17 @@ object DateTimeUtils { */ def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { val tz = TimeZone.getTimeZone(timeZone) - val offset = tz.getOffset(time / 1000L) + val offset = getOffsetFromLocalMillis(time / 1000L, tz) time - offset * 1000L } + + /** + * Re-initialize the current thread's thread locals. Exposed for testing. + */ + private[util] def resetThreadLocals(): Unit = { + threadLocalGmtCalendar.remove() + threadLocalLocalTimeZone.remove() + threadLocalTimestampFormat.remove() + threadLocalDateFormat.remove() + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala index 1d73e40ffcd36..2c966230e447e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.ScalaReflectionLock * * Please use the singleton [[DataTypes.DateType]]. * - * Internally, this is represented as the number of days from epoch (1970-01-01 00:00:00 UTC). + * Internally, this is represented as the number of days from 1970-01-01. */ @DeveloperApi class DateType private() extends AtomicType { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala new file mode 100644 index 0000000000000..0c1feb3aa0882 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import java.util.TimeZone + +/** + * Helper functions for testing date and time functionality. + */ +object DateTimeTestUtils { + + val ALL_TIMEZONES: Seq[TimeZone] = TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone) + + def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = { + val originalDefaultTimeZone = TimeZone.getDefault + try { + DateTimeUtils.resetThreadLocals() + TimeZone.setDefault(newDefaultTimeZone) + block + } finally { + TimeZone.setDefault(originalDefaultTimeZone) + DateTimeUtils.resetThreadLocals() + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 28e30c2219e3f..f9cb97629fcf3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -492,6 +492,13 @@ class DateTimeUtilsSuite extends SparkFunSuite { test("2011-12-25 09:00:00.123456", "JST", "2011-12-25 18:00:00.123456") test("2011-12-25 09:00:00.123456", "PST", "2011-12-25 01:00:00.123456") test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456") + + // Daylight Saving Time + test("2016-03-13 09:59:59.0", "PST", "2016-03-13 01:59:59.0") + test("2016-03-13 10:00:00.0", "PST", "2016-03-13 03:00:00.0") + test("2016-11-06 08:59:59.0", "PST", "2016-11-06 01:59:59.0") + test("2016-11-06 09:00:00.0", "PST", "2016-11-06 01:00:00.0") + test("2016-11-06 10:00:00.0", "PST", "2016-11-06 02:00:00.0") } test("to UTC timestamp") { @@ -503,5 +510,38 @@ class DateTimeUtilsSuite extends SparkFunSuite { test("2011-12-25 18:00:00.123456", "JST", "2011-12-25 09:00:00.123456") test("2011-12-25 01:00:00.123456", "PST", "2011-12-25 09:00:00.123456") test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456") + + // Daylight Saving Time + test("2016-03-13 01:59:59", "PST", "2016-03-13 09:59:59.0") + // 2016-03-13 02:00:00 PST does not exists + test("2016-03-13 02:00:00", "PST", "2016-03-13 10:00:00.0") + test("2016-03-13 03:00:00", "PST", "2016-03-13 10:00:00.0") + test("2016-11-06 00:59:59", "PST", "2016-11-06 07:59:59.0") + // 2016-11-06 01:00:00 PST could be 2016-11-06 08:00:00 UTC or 2016-11-06 09:00:00 UTC + test("2016-11-06 01:00:00", "PST", "2016-11-06 09:00:00.0") + test("2016-11-06 01:59:59", "PST", "2016-11-06 09:59:59.0") + test("2016-11-06 02:00:00", "PST", "2016-11-06 10:00:00.0") + } + + test("daysToMillis and millisToDays") { + // There are some days are skipped entirely in some timezone, skip them here. + val skipped_days = Map[String, Int]( + "Kwajalein" -> 8632, + "Pacific/Apia" -> 15338, + "Pacific/Enderbury" -> 9131, + "Pacific/Fakaofo" -> 15338, + "Pacific/Kiritimati" -> 9131, + "Pacific/Kwajalein" -> 8632, + "MIT" -> 15338) + for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { + DateTimeTestUtils.withDefaultTimeZone(tz) { + val skipped = skipped_days.getOrElse(tz.getID, Int.MinValue) + (-20000 to 20000).foreach { d => + if (d != skipped) { + assert(millisToDays(daysToMillis(d)) === d) + } + } + } + } } } From 2c1c337ba5984b9e495b4d02bf865e56fd83ab03 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 19 Jun 2016 20:12:00 +0100 Subject: [PATCH 0718/1470] [SPARK-15942][REPL] Unblock `:reset` command in REPL. ## What changes were proposed in this pull (Paste from JIRA issue.) As a follow up for SPARK-15697, I have following semantics for `:reset` command. On `:reset` we forget all that user has done but not the initialization of spark. To avoid confusion or make it more clear, we show the message `spark` and `sc` are not erased, infact they are in same state as they were left by previous operations done by the user. While doing above, somewhere I felt that this is not usually what reset means. But an accidental shutdown of a cluster can be very costly, so may be in that sense this is less surprising and still useful. ## How was this patch tested? Manually, by calling `:reset` command, by both altering the state of SparkContext and creating some local variables. Author: Prashant Sharma Author: Prashant Sharma Closes #13661 from ScrapCodes/repl-reset-command. (cherry picked from commit 1b3a9b966a7813e2406dfb020e83605af22f9ef3) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/repl/SparkILoop.scala | 16 ++++++++++++++-- .../scala/org/apache/spark/repl/ReplSuite.scala | 3 ++- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala index dcf3209ae7d82..2707b0847aefc 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -36,7 +36,11 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) def initializeSpark() { intp.beQuietDuring { processLine(""" - @transient val spark = org.apache.spark.repl.Main.createSparkSession() + @transient val spark = if (org.apache.spark.repl.Main.sparkSession != null) { + org.apache.spark.repl.Main.sparkSession + } else { + org.apache.spark.repl.Main.createSparkSession() + } @transient val sc = { val _sc = spark.sparkContext _sc.uiWebUrl.foreach(webUrl => println(s"Spark context Web UI available at ${webUrl}")) @@ -50,6 +54,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) processLine("import spark.implicits._") processLine("import spark.sql") processLine("import org.apache.spark.sql.functions._") + replayCommandStack = Nil // remove above commands from session history. } } @@ -70,7 +75,8 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) echo("Type :help for more information.") } - private val blockedCommands = Set[String]("reset") + /** Add repl commands that needs to be blocked. e.g. reset */ + private val blockedCommands = Set[String]() /** Standard commands */ lazy val sparkStandardCommands: List[SparkILoop.this.LoopCommand] = @@ -88,6 +94,12 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) initializeSpark() super.loadFiles(settings) } + + override def resetCommand(line: String): Unit = { + super.resetCommand(line) + initializeSpark() + echo("Note that after :reset, state of SparkSession and SparkContext is unchanged.") + } } object SparkILoop { diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 2444e93d9ab6e..c10db947bcb44 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -49,7 +49,8 @@ class ReplSuite extends SparkFunSuite { val oldExecutorClasspath = System.getProperty(CONF_EXECUTOR_CLASSPATH) System.setProperty(CONF_EXECUTOR_CLASSPATH, classpath) - + Main.sparkContext = null + Main.sparkSession = null // causes recreation of SparkContext for each test. Main.conf.set("spark.master", master) Main.doMain(Array("-classpath", classpath), new SparkILoop(in, new PrintWriter(out))) From 80c6d4e3a49fad4dac46738fe5458641f21b96a1 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Sun, 19 Jun 2016 20:19:40 +0100 Subject: [PATCH 0719/1470] [SPARK-16040][MLLIB][DOC] spark.mllib PIC document extra line of refernece ## What changes were proposed in this pull request? In the 2.0 document, Line "A full example that produces the experiment described in the PIC paper can be found under examples/." is redundant. There is already "Find full example code at "examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala" in the Spark repo.". We should remove the first line, which is consistent with other documents. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manual test Author: wm624@hotmail.com Closes #13755 from wangmiao1981/doc. (cherry picked from commit 5930d7a2e95b2fe4d470cf39546e5a12306553fe) Signed-off-by: Sean Owen --- docs/mllib-clustering.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 6897ba4a5d57d..073927c30bc63 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -170,10 +170,6 @@ which contains the computed clustering assignments. Refer to the [`PowerIterationClustering` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClustering) and [`PowerIterationClusteringModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClusteringModel) for details on the API. {% include_example scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala %} - -A full example that produces the experiment described in the PIC paper can be found under -[`examples/`](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala). -
    From d11f533ded502c6cc4a129e201362bca6e302028 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 19 Jun 2016 21:27:04 -0700 Subject: [PATCH 0720/1470] [SPARK-16031] Add debug-only socket source in Structured Streaming ## What changes were proposed in this pull request? This patch adds a text-based socket source similar to the one in Spark Streaming for debugging and tutorials. The source is clearly marked as debug-only so that users don't try to run it in production applications, because this type of source cannot provide HA without storing a lot of state in Spark. ## How was this patch tested? Unit tests and manual tests in spark-shell. Author: Matei Zaharia Closes #13748 from mateiz/socket-source. (cherry picked from commit 4f17fddcd57adeae0d7e31bd14423283d4b625e9) Signed-off-by: Reynold Xin --- ...pache.spark.sql.sources.DataSourceRegister | 1 + .../streaming/FileStreamSource.scala | 2 + .../sql/execution/streaming/Source.scala | 3 + .../execution/streaming/StreamExecution.scala | 1 + .../sql/execution/streaming/memory.scala | 2 + .../sql/execution/streaming/socket.scala | 144 ++++++++++++++++++ .../streaming/TextSocketStreamSuite.scala | 136 +++++++++++++++++ .../spark/sql/streaming/StreamSuite.scala | 2 + .../test/DataStreamReaderWriterSuite.scala | 2 + 9 files changed, 293 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 9f8bb5d38fd6c..27d32b5dca431 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -4,3 +4,4 @@ org.apache.spark.sql.execution.datasources.json.JsonFileFormat org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat org.apache.spark.sql.execution.datasources.text.TextFileFormat org.apache.spark.sql.execution.streaming.ConsoleSinkProvider +org.apache.spark.sql.execution.streaming.TextSocketSourceProvider diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index bef56160f6bb8..9886ad0b41089 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -128,4 +128,6 @@ class FileStreamSource( override def getOffset: Option[Offset] = Some(fetchMaxOffset()).filterNot(_.offset == -1) override def toString: String = s"FileStreamSource[$qualifiedBasePath]" + + override def stop() {} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index 14450c2e2fd14..971147840d2fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -39,4 +39,7 @@ trait Source { * same data for a particular `start` and `end` pair. */ def getBatch(start: Option[Offset], end: Offset): DataFrame + + /** Stop this source and free any resources it has allocated. */ + def stop(): Unit } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 4aefd39b3646c..bb42a11759b72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -399,6 +399,7 @@ class StreamExecution( microBatchThread.interrupt() microBatchThread.join() } + uniqueSources.foreach(_.stop()) logInfo(s"Query $name was stopped") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 77fd043ef7219..e37f0c77795c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -110,6 +110,8 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) sys.error("No data selected!") } } + + override def stop() {} } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala new file mode 100644 index 0000000000000..d07d88dcdcc44 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.{BufferedReader, InputStreamReader, IOException} +import java.net.Socket +import javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +object TextSocketSource { + val SCHEMA = StructType(StructField("value", StringType) :: Nil) +} + +/** + * A source that reads text lines through a TCP socket, designed only for tutorials and debugging. + * This source will *not* work in production applications due to multiple reasons, including no + * support for fault recovery and keeping all of the text read in memory forever. + */ +class TextSocketSource(host: String, port: Int, sqlContext: SQLContext) + extends Source with Logging +{ + @GuardedBy("this") + private var socket: Socket = null + + @GuardedBy("this") + private var readThread: Thread = null + + @GuardedBy("this") + private var lines = new ArrayBuffer[String] + + initialize() + + private def initialize(): Unit = synchronized { + socket = new Socket(host, port) + val reader = new BufferedReader(new InputStreamReader(socket.getInputStream)) + readThread = new Thread(s"TextSocketSource($host, $port)") { + setDaemon(true) + + override def run(): Unit = { + try { + while (true) { + val line = reader.readLine() + if (line == null) { + // End of file reached + logWarning(s"Stream closed by $host:$port") + return + } + TextSocketSource.this.synchronized { + lines += line + } + } + } catch { + case e: IOException => + } + } + } + readThread.start() + } + + /** Returns the schema of the data from this source */ + override def schema: StructType = TextSocketSource.SCHEMA + + /** Returns the maximum available offset for this source. */ + override def getOffset: Option[Offset] = synchronized { + if (lines.isEmpty) None else Some(LongOffset(lines.size - 1)) + } + + /** Returns the data that is between the offsets (`start`, `end`]. */ + override def getBatch(start: Option[Offset], end: Offset): DataFrame = synchronized { + val startIdx = start.map(_.asInstanceOf[LongOffset].offset.toInt + 1).getOrElse(0) + val endIdx = end.asInstanceOf[LongOffset].offset.toInt + 1 + val data = synchronized { lines.slice(startIdx, endIdx) } + import sqlContext.implicits._ + data.toDF("value") + } + + /** Stop this source. */ + override def stop(): Unit = synchronized { + if (socket != null) { + try { + // Unfortunately, BufferedReader.readLine() cannot be interrupted, so the only way to + // stop the readThread is to close the socket. + socket.close() + } catch { + case e: IOException => + } + socket = null + } + } +} + +class TextSocketSourceProvider extends StreamSourceProvider with DataSourceRegister with Logging { + /** Returns the name and schema of the source that can be used to continually read data. */ + override def sourceSchema( + sqlContext: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + logWarning("The socket source should not be used for production applications! " + + "It does not support recovery and stores state indefinitely.") + if (!parameters.contains("host")) { + throw new AnalysisException("Set a host to read from with option(\"host\", ...).") + } + if (!parameters.contains("port")) { + throw new AnalysisException("Set a port to read from with option(\"port\", ...).") + } + ("textSocket", TextSocketSource.SCHEMA) + } + + override def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + val host = parameters("host") + val port = parameters("port").toInt + new TextSocketSource(host, port, sqlContext) + } + + /** String that represents the format that this data source provider uses. */ + override def shortName(): String = "socket" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala new file mode 100644 index 0000000000000..ca577631854ef --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.{IOException, OutputStreamWriter} +import java.net.ServerSocket +import java.util.concurrent.LinkedBlockingQueue + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class TextSocketStreamSuite extends StreamTest with SharedSQLContext with BeforeAndAfterEach { + import testImplicits._ + + override def afterEach() { + sqlContext.streams.active.foreach(_.stop()) + if (serverThread != null) { + serverThread.interrupt() + serverThread.join() + serverThread = null + } + if (source != null) { + source.stop() + source = null + } + } + + private var serverThread: ServerThread = null + private var source: Source = null + + test("basic usage") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString) + val schema = provider.sourceSchema(sqlContext, None, "", parameters)._2 + assert(schema === StructType(StructField("value", StringType) :: Nil)) + + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val offset1 = source.getOffset.get + val batch1 = source.getBatch(None, offset1) + assert(batch1.as[String].collect().toSeq === Seq("hello")) + + serverThread.enqueue("world") + while (source.getOffset.get === offset1) { + Thread.sleep(10) + } + val offset2 = source.getOffset.get + val batch2 = source.getBatch(Some(offset1), offset2) + assert(batch2.as[String].collect().toSeq === Seq("world")) + + val both = source.getBatch(None, offset2) + assert(both.as[String].collect().sorted.toSeq === Seq("hello", "world")) + + // Try stopping the source to make sure this does not block forever. + source.stop() + source = null + } + } + + test("params not given") { + val provider = new TextSocketSourceProvider + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map()) + } + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map("host" -> "localhost")) + } + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map("port" -> "1234")) + } + } + + test("no server up") { + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> "0") + intercept[IOException] { + source = provider.createSource(sqlContext, "", None, "", parameters) + } + } + + private class ServerThread extends Thread with Logging { + private val serverSocket = new ServerSocket(0) + private val messageQueue = new LinkedBlockingQueue[String]() + + val port = serverSocket.getLocalPort + + override def run(): Unit = { + try { + val clientSocket = serverSocket.accept() + clientSocket.setTcpNoDelay(true) + val out = new OutputStreamWriter(clientSocket.getOutputStream) + while (true) { + val line = messageQueue.take() + out.write(line + "\n") + out.flush() + } + } catch { + case e: InterruptedException => + } finally { + serverSocket.close() + } + } + + def enqueue(line: String): Unit = { + messageQueue.put(line) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 786404a589581..b8e40e71bfce5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -282,6 +282,8 @@ class FakeDefaultSource extends StreamSourceProvider { val startOffset = start.map(_.asInstanceOf[LongOffset].offset).getOrElse(-1L) + 1 spark.range(startOffset, end.asInstanceOf[LongOffset].offset + 1).toDF("a") } + + override def stop() {} } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 1aee1934c0791..943e7b761e6e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -84,6 +84,8 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { Seq[Int]().toDS().toDF() } + + override def stop() {} } } From 19397caab62b550269961a123bd11a34afc3a09b Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 19 Jun 2016 21:45:53 -0700 Subject: [PATCH 0721/1470] [SPARK-16036][SPARK-16037][SPARK-16034][SQL] Follow up code clean up and improvement ## What changes were proposed in this pull request? This PR is the follow-up PR for https://github.com/apache/spark/pull/13754/files and https://github.com/apache/spark/pull/13749. I will comment inline to explain my changes. ## How was this patch tested? Existing tests. Author: Yin Huai Closes #13766 from yhuai/caseSensitivity. (cherry picked from commit 6d0f921aedfdd3b7e8472b6776d0c7d8299190bd) Signed-off-by: Yin Huai --- .../plans/logical/basicLogicalOperators.scala | 2 ++ .../apache/spark/sql/DataFrameWriter.scala | 24 ++++--------- .../execution/datasources/DataSource.scala | 9 ++--- .../sql/execution/datasources/rules.scala | 13 +++++-- .../spark/sql/internal/SessionState.scala | 2 +- .../sql/execution/command/DDLSuite.scala | 20 +++++++++-- .../spark/sql/hive/HiveSessionState.scala | 2 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 20 +++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 35 +++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 32 ----------------- 10 files changed, 98 insertions(+), 61 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 69b8b059fde1c..ff3dcbc957ac1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -369,6 +369,8 @@ case class InsertIntoTable( if (table.output.isEmpty) { None } else { + // Note: The parser (visitPartitionSpec in AstBuilder) already turns + // keys in partition to their lowercase forms. val staticPartCols = partition.filter(_._2.isDefined).keySet Some(table.output.filterNot(a => staticPartCols.contains(a.name))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index e6fc9749c7267..ca3972d62dfb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -245,29 +245,17 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { if (partitioningColumns.isDefined) { throw new AnalysisException( "insertInto() can't be used together with partitionBy(). " + - "Partition columns are defined by the table into which is being inserted." + "Partition columns have already be defined for the table. " + + "It is not necessary to use partitionBy()." ) } - val partitions = normalizedParCols.map(_.map(col => col -> Option.empty[String]).toMap) - val overwrite = mode == SaveMode.Overwrite - - // A partitioned relation's schema can be different from the input logicalPlan, since - // partition columns are all moved after data columns. We Project to adjust the ordering. - // TODO: this belongs to the analyzer. - val input = normalizedParCols.map { parCols => - val (inputPartCols, inputDataCols) = df.logicalPlan.output.partition { attr => - parCols.contains(attr.name) - } - Project(inputDataCols ++ inputPartCols, df.logicalPlan) - }.getOrElse(df.logicalPlan) - df.sparkSession.sessionState.executePlan( InsertIntoTable( - UnresolvedRelation(tableIdent), - partitions.getOrElse(Map.empty[String, Option[String]]), - input, - overwrite, + table = UnresolvedRelation(tableIdent), + partition = Map.empty[String, Option[String]], + child = df.logicalPlan, + overwrite = mode == SaveMode.Overwrite, ifNotExists = false)).toRdd } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index f274fc77daef0..557445c2bc91f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -435,7 +435,7 @@ case class DataSource( // If we are appending to a table that already exists, make sure the partitioning matches // up. If we fail to load the table for whatever reason, ignore the check. if (mode == SaveMode.Append) { - val existingColumns = Try { + val existingPartitionColumns = Try { resolveRelation() .asInstanceOf[HadoopFsRelation] .location @@ -444,13 +444,14 @@ case class DataSource( .fieldNames .toSeq }.getOrElse(Seq.empty[String]) + // TODO: Case sensitivity. val sameColumns = - existingColumns.map(_.toLowerCase) == partitionColumns.map(_.toLowerCase) - if (existingColumns.size > 0 && !sameColumns) { + existingPartitionColumns.map(_.toLowerCase()) == partitionColumns.map(_.toLowerCase()) + if (existingPartitionColumns.size > 0 && !sameColumns) { throw new AnalysisException( s"""Requested partitioning does not match existing partitioning. |Existing partitioning columns: - | ${existingColumns.mkString(", ")} + | ${existingPartitionColumns.mkString(", ")} |Requested partitioning columns: | ${partitionColumns.mkString(", ")} |""".stripMargin) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 5963c53a1b1cb..10425af3e1f18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -67,7 +67,7 @@ private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[Lo * table. It also does data type casting and field renaming, to make sure that the columns to be * inserted have the correct data type and fields have the correct names. */ -private[sql] object PreprocessTableInsertion extends Rule[LogicalPlan] { +private[sql] case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { private def preprocess( insert: InsertIntoTable, tblName: String, @@ -84,7 +84,13 @@ private[sql] object PreprocessTableInsertion extends Rule[LogicalPlan] { if (insert.partition.nonEmpty) { // the query's partitioning must match the table's partitioning // this is set for queries like: insert into ... partition (one = "a", two = ) - if (insert.partition.keySet != partColNames.toSet) { + val samePartitionColumns = + if (conf.caseSensitiveAnalysis) { + insert.partition.keySet == partColNames.toSet + } else { + insert.partition.keySet.map(_.toLowerCase) == partColNames.map(_.toLowerCase).toSet + } + if (!samePartitionColumns) { throw new AnalysisException( s""" |Requested partitioning does not match the table $tblName: @@ -94,7 +100,8 @@ private[sql] object PreprocessTableInsertion extends Rule[LogicalPlan] { } expectedColumns.map(castAndRenameChildOutput(insert, _)).getOrElse(insert) } else { - // All partition columns are dynamic because this InsertIntoTable had no partitioning + // All partition columns are dynamic because because the InsertIntoTable command does + // not explicitly specify partitioning columns. expectedColumns.map(castAndRenameChildOutput(insert, _)).getOrElse(insert) .copy(partition = partColNames.map(_ -> None).toMap) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index b033e19ddf06c..5300cfa8a7b06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -111,7 +111,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val analyzer: Analyzer = { new Analyzer(catalog, conf) { override val extendedResolutionRules = - PreprocessTableInsertion :: + PreprocessTableInsertion(conf) :: new FindDataSourceTable(sparkSession) :: DataSourceAnalysis :: (if (conf.runSQLonFile) new ResolveDataSource(sparkSession) :: Nil else Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 8827649d0a0d9..f40ddcc95affd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1337,8 +1337,24 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(sql("select * from partitionedTable").collect().size == 1) // Inserts new data successfully when partition columns are correctly specified in // partitionBy(...). - df.write.mode("append").partitionBy("a", "b").saveAsTable("partitionedTable") - assert(sql("select * from partitionedTable").collect().size == 2) + // TODO: Right now, partition columns are always treated in a case-insensitive way. + // See the write method in DataSource.scala. + Seq((4, 5, 6)).toDF("a", "B", "c") + .write + .mode("append") + .partitionBy("a", "B") + .saveAsTable("partitionedTable") + + Seq((7, 8, 9)).toDF("a", "b", "c") + .write + .mode("append") + .partitionBy("a", "b") + .saveAsTable("partitionedTable") + + checkAnswer( + sql("select a, b, c from partitionedTable"), + Row(1, 2, 3) :: Row(4, 5, 6) :: Row(7, 8, 9) :: Nil + ) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 2d286715b57b6..f6675f09041d3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -65,7 +65,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) catalog.ParquetConversions :: catalog.OrcConversions :: catalog.CreateTables :: - PreprocessTableInsertion :: + PreprocessTableInsertion(conf) :: DataSourceAnalysis :: (if (conf.runSQLonFile) new ResolveDataSource(sparkSession) :: Nil else Nil) 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 c48735142dd00..d4ebd051d2eed 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 @@ -372,4 +372,24 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef assert(!logical.resolved, "Should not resolve: missing partition data") } } + + testPartitionedTable( + "SPARK-16036: better error message when insert into a table with mismatch schema") { + tableName => + val e = intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION(b=1, c=2) SELECT 1, 2, 3") + } + assert(e.message.contains("the number of columns are different")) + } + + testPartitionedTable( + "SPARK-16037: INSERT statement should match columns by position") { + tableName => + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql(s"INSERT INTO TABLE $tableName SELECT 1, 2 AS c, 3 AS b") + checkAnswer(sql(s"SELECT a, b, c FROM $tableName"), Row(1, 2, 3)) + sql(s"INSERT OVERWRITE TABLE $tableName SELECT 1, 2, 3") + checkAnswer(sql(s"SELECT a, b, c FROM $tableName"), Row(1, 2, 3)) + } + } } 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 a16b5b2e23c3d..85b159e2a5ded 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 @@ -1033,6 +1033,41 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SELECT * FROM boom").queryExecution.analyzed } + test("SPARK-3810: PreprocessTableInsertion static partitioning support") { + val analyzedPlan = { + loadTestTable("srcpart") + sql("DROP TABLE IF EXISTS withparts") + sql("CREATE TABLE withparts LIKE srcpart") + sql("INSERT INTO TABLE withparts PARTITION(ds='1', hr='2') SELECT key, value FROM src") + .queryExecution.analyzed + } + + assertResult(1, "Duplicated project detected\n" + analyzedPlan) { + analyzedPlan.collect { + case _: Project => () + }.size + } + } + + test("SPARK-3810: PreprocessTableInsertion dynamic partitioning support") { + val analyzedPlan = { + loadTestTable("srcpart") + sql("DROP TABLE IF EXISTS withparts") + sql("CREATE TABLE withparts LIKE srcpart") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + sql("CREATE TABLE IF NOT EXISTS withparts LIKE srcpart") + sql("INSERT INTO TABLE withparts PARTITION(ds, hr) SELECT key, value, '1', '2' FROM src") + .queryExecution.analyzed + } + + assertResult(2, "Duplicated project detected\n" + analyzedPlan) { + analyzedPlan.collect { + case _: Project => () + }.size + } + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" 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 46a77dd917fb3..9c1f21825315b 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 @@ -1684,36 +1684,4 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } } - - test("SPARK-16036: better error message when insert into a table with mismatch schema") { - withTable("hive_table", "datasource_table") { - sql("CREATE TABLE hive_table(a INT) PARTITIONED BY (b INT, c INT)") - sql("CREATE TABLE datasource_table(a INT, b INT, c INT) USING parquet PARTITIONED BY (b, c)") - val e1 = intercept[AnalysisException] { - sql("INSERT INTO TABLE hive_table PARTITION(b=1, c=2) SELECT 1, 2, 3") - } - assert(e1.message.contains("the number of columns are different")) - val e2 = intercept[AnalysisException] { - sql("INSERT INTO TABLE datasource_table PARTITION(b=1, c=2) SELECT 1, 2, 3") - } - assert(e2.message.contains("the number of columns are different")) - } - } - - test("SPARK-16037: INSERT statement should match columns by position") { - withTable("hive_table", "datasource_table") { - sql("CREATE TABLE hive_table(a INT) PARTITIONED BY (b INT, c INT)") - sql("CREATE TABLE datasource_table(a INT, b INT, c INT) USING parquet PARTITIONED BY (b, c)") - - withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { - sql("INSERT INTO TABLE hive_table SELECT 1, 2 AS c, 3 AS b") - checkAnswer(sql("SELECT a, b, c FROM hive_table"), Row(1, 2, 3)) - sql("INSERT OVERWRITE TABLE hive_table SELECT 1, 2, 3") - checkAnswer(sql("SELECT a, b, c FROM hive_table"), Row(1, 2, 3)) - } - - sql("INSERT INTO TABLE datasource_table SELECT 1, 2 AS c, 3 AS b") - checkAnswer(sql("SELECT a, b, c FROM datasource_table"), Row(1, 2, 3)) - } - } } From 0b0b5fe549086171d851d7c4458d48be9409380f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 20 Jun 2016 20:17:47 +0800 Subject: [PATCH 0722/1470] [SPARK-16030][SQL] Allow specifying static partitions when inserting to data source tables ## What changes were proposed in this pull request? This PR adds the static partition support to INSERT statement when the target table is a data source table. ## How was this patch tested? New tests in InsertIntoHiveTableSuite and DataSourceAnalysisSuite. **Note: This PR is based on https://github.com/apache/spark/pull/13766. The last commit is the actual change.** Author: Yin Huai Closes #13769 from yhuai/SPARK-16030-1. (cherry picked from commit 905f774b71f4b814d5a2412c7c35bd023c3dfdf8) Signed-off-by: Cheng Lian --- .../sql/catalyst/analysis/CheckAnalysis.scala | 19 ++ .../datasources/DataSourceStrategy.scala | 127 ++++++++++- .../sql/execution/datasources/rules.scala | 7 - .../spark/sql/internal/SessionState.scala | 2 +- .../sql/sources/DataSourceAnalysisSuite.scala | 202 ++++++++++++++++++ .../spark/sql/hive/HiveSessionState.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 3 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 97 ++++++++- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 9 files changed, 436 insertions(+), 25 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala 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 7b451baaa02b9..899227674f2ac 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 @@ -313,6 +313,8 @@ trait CheckAnalysis extends PredicateHelper { |${s.catalogTable.identifier} """.stripMargin) + // TODO: We need to consolidate this kind of checks for InsertIntoTable + // with the rule of PreWriteCheck defined in extendedCheckRules. case InsertIntoTable(s: SimpleCatalogRelation, _, _, _, _) => failAnalysis( s""" @@ -320,6 +322,23 @@ trait CheckAnalysis extends PredicateHelper { |${s.catalogTable.identifier} """.stripMargin) + case InsertIntoTable(t, _, _, _, _) + if !t.isInstanceOf[LeafNode] || + t == OneRowRelation || + t.isInstanceOf[LocalRelation] => + failAnalysis(s"Inserting into an RDD-based table is not allowed.") + + case i @ InsertIntoTable(table, partitions, query, _, _) => + val numStaticPartitions = partitions.values.count(_.isDefined) + if (table.output.size != (query.output.size + numStaticPartitions)) { + failAnalysis( + s"$table requires that the data to be inserted have the same number of " + + s"columns as the target table: target table has ${table.output.size} " + + s"column(s) but the inserted data has " + + s"${query.output.size + numStaticPartitions} column(s), including " + + s"$numStaticPartitions partition column(s) having constant value(s).") + } + case o if !o.resolved => failAnalysis( s"unresolved operator ${operator.simpleString}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 2b4786542c72f..27133f0a43f2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -22,15 +22,15 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SimpleCatalogRelation} import org.apache.spark.sql.catalyst.expressions 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.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.DataSourceScanExec.PUSHED_FILTERS @@ -43,8 +43,127 @@ import org.apache.spark.unsafe.types.UTF8String * Replaces generic operations with specific variants that are designed to work with Spark * SQL Data Sources. */ -private[sql] object DataSourceAnalysis extends Rule[LogicalPlan] { +private[sql] case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { + + def resolver: Resolver = { + if (conf.caseSensitiveAnalysis) { + caseSensitiveResolution + } else { + caseInsensitiveResolution + } + } + + // The access modifier is used to expose this method to tests. + private[sql] def convertStaticPartitions( + sourceAttributes: Seq[Attribute], + providedPartitions: Map[String, Option[String]], + targetAttributes: Seq[Attribute], + targetPartitionSchema: StructType): Seq[NamedExpression] = { + + assert(providedPartitions.exists(_._2.isDefined)) + + val staticPartitions = providedPartitions.flatMap { + case (partKey, Some(partValue)) => (partKey, partValue) :: Nil + case (_, None) => Nil + } + + // The sum of the number of static partition columns and columns provided in the SELECT + // clause needs to match the number of columns of the target table. + if (staticPartitions.size + sourceAttributes.size != targetAttributes.size) { + throw new AnalysisException( + s"The data to be inserted needs to have the same number of " + + s"columns as the target table: target table has ${targetAttributes.size} " + + s"column(s) but the inserted data has ${sourceAttributes.size + staticPartitions.size} " + + s"column(s), which contain ${staticPartitions.size} partition column(s) having " + + s"assigned constant values.") + } + + if (providedPartitions.size != targetPartitionSchema.fields.size) { + throw new AnalysisException( + s"The data to be inserted needs to have the same number of " + + s"partition columns as the target table: target table " + + s"has ${targetPartitionSchema.fields.size} partition column(s) but the inserted " + + s"data has ${providedPartitions.size} partition columns specified.") + } + + staticPartitions.foreach { + case (partKey, partValue) => + if (!targetPartitionSchema.fields.exists(field => resolver(field.name, partKey))) { + throw new AnalysisException( + s"$partKey is not a partition column. Partition columns are " + + s"${targetPartitionSchema.fields.map(_.name).mkString("[", ",", "]")}") + } + } + + val partitionList = targetPartitionSchema.fields.map { field => + val potentialSpecs = staticPartitions.filter { + case (partKey, partValue) => resolver(field.name, partKey) + } + if (potentialSpecs.size == 0) { + None + } else if (potentialSpecs.size == 1) { + val partValue = potentialSpecs.head._2 + Some(Alias(Cast(Literal(partValue), field.dataType), "_staticPart")()) + } else { + throw new AnalysisException( + s"Partition column ${field.name} have multiple values specified, " + + s"${potentialSpecs.mkString("[", ", ", "]")}. Please only specify a single value.") + } + } + + // We first drop all leading static partitions using dropWhile and check if there is + // any static partition appear after dynamic partitions. + partitionList.dropWhile(_.isDefined).collectFirst { + case Some(_) => + throw new AnalysisException( + s"The ordering of partition columns is " + + s"${targetPartitionSchema.fields.map(_.name).mkString("[", ",", "]")}. " + + "All partition columns having constant values need to appear before other " + + "partition columns that do not have an assigned constant value.") + } + + assert(partitionList.take(staticPartitions.size).forall(_.isDefined)) + val projectList = + sourceAttributes.take(targetAttributes.size - targetPartitionSchema.fields.size) ++ + partitionList.take(staticPartitions.size).map(_.get) ++ + sourceAttributes.takeRight(targetPartitionSchema.fields.size - staticPartitions.size) + + projectList + } + override def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // If the InsertIntoTable command is for a partitioned HadoopFsRelation and + // the user has specified static partitions, we add a Project operator on top of the query + // to include those constant column values in the query result. + // + // Example: + // Let's say that we have a table "t", which is created by + // CREATE TABLE t (a INT, b INT, c INT) USING parquet PARTITIONED BY (b, c) + // The statement of "INSERT INTO TABLE t PARTITION (b=2, c) SELECT 1, 3" + // will be converted to "INSERT INTO TABLE t PARTITION (b, c) SELECT 1, 2, 3". + // + // Basically, we will put those partition columns having a assigned value back + // to the SELECT clause. The output of the SELECT clause is organized as + // normal_columns static_partitioning_columns dynamic_partitioning_columns. + // static_partitioning_columns are partitioning columns having assigned + // values in the PARTITION clause (e.g. b in the above example). + // dynamic_partitioning_columns are partitioning columns that do not assigned + // values in the PARTITION clause (e.g. c in the above example). + case insert @ logical.InsertIntoTable( + relation @ LogicalRelation(t: HadoopFsRelation, _, _), parts, query, overwrite, false) + if query.resolved && parts.exists(_._2.isDefined) => + + val projectList = convertStaticPartitions( + sourceAttributes = query.output, + providedPartitions = parts, + targetAttributes = relation.output, + targetPartitionSchema = t.partitionSchema) + + // We will remove all assigned values to static partitions because they have been + // moved to the projectList. + insert.copy(partition = parts.map(p => (p._1, None)), child = Project(projectList, query)) + + case i @ logical.InsertIntoTable( l @ LogicalRelation(t: HadoopFsRelation, _, _), part, query, overwrite, false) if query.resolved && t.schema.asNullable == query.schema.asNullable => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 10425af3e1f18..15b9d14bd73fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -206,13 +206,6 @@ private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") - case logical.InsertIntoTable(t, _, _, _, _) => - if (!t.isInstanceOf[LeafNode] || t == OneRowRelation || t.isInstanceOf[LocalRelation]) { - failAnalysis(s"Inserting into an RDD-based table is not allowed.") - } else { - // OK - } - case c: CreateTableUsingAsSelect => // When the SaveMode is Overwrite, we need to check if the table is an input table of // the query. If so, we will throw an AnalysisException to let users know it is not allowed. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 5300cfa8a7b06..5f5cf5c6d30c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -113,7 +113,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { override val extendedResolutionRules = PreprocessTableInsertion(conf) :: new FindDataSourceTable(sparkSession) :: - DataSourceAnalysis :: + DataSourceAnalysis(conf) :: (if (conf.runSQLonFile) new ResolveDataSource(sparkSession) :: Nil else Nil) override val extendedCheckRules = Seq(datasources.PreWriteCheck(conf, catalog)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala new file mode 100644 index 0000000000000..448adcf11d656 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.SimpleCatalystConf +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, Literal} +import org.apache.spark.sql.execution.datasources.DataSourceAnalysis +import org.apache.spark.sql.types.{IntegerType, StructType} + +class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll { + + private var targetAttributes: Seq[Attribute] = _ + private var targetPartitionSchema: StructType = _ + + override def beforeAll(): Unit = { + targetAttributes = Seq('a.int, 'd.int, 'b.int, 'c.int) + targetPartitionSchema = new StructType() + .add("b", IntegerType) + .add("c", IntegerType) + } + + private def checkProjectList(actual: Seq[Expression], expected: Seq[Expression]): Unit = { + // Remove aliases since we have no control on their exprId. + val withoutAliases = actual.map { + case alias: Alias => alias.child + case other => other + } + assert(withoutAliases === expected) + } + + Seq(true, false).foreach { caseSensitive => + val rule = DataSourceAnalysis(SimpleCatalystConf(caseSensitive)) + test( + s"convertStaticPartitions only handle INSERT having at least static partitions " + + s"(caseSensitive: $caseSensitive)") { + intercept[AssertionError] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> None, "c" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"Missing columns (caseSensitive: $caseSensitive)") { + // Missing columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int), + providedPartitions = Map("b" -> Some("1"), "c" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"Missing partitioning columns (caseSensitive: $caseSensitive)") { + // Missing partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + // Missing partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int, 'g.int), + providedPartitions = Map("b" -> Some("1")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1"), "d" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"Wrong partitioning columns (caseSensitive: $caseSensitive)") { + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1"), "d" -> Some("2")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int), + providedPartitions = Map("b" -> Some("1"), "c" -> Some("3"), "d" -> Some("2")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + if (caseSensitive) { + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1"), "C" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + } + + test( + s"Static partitions need to appear before dynamic partitions" + + s" (caseSensitive: $caseSensitive)") { + // Static partitions need to appear before dynamic partitions. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> None, "c" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"All static partitions (caseSensitive: $caseSensitive)") { + if (!caseSensitive) { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val expected = nonPartitionedAttributes ++ + Seq(Cast(Literal("1"), IntegerType), Cast(Literal("3"), IntegerType)) + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes, + providedPartitions = Map("b" -> Some("1"), "C" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + checkProjectList(actual, expected) + } + + { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val expected = nonPartitionedAttributes ++ + Seq(Cast(Literal("1"), IntegerType), Cast(Literal("3"), IntegerType)) + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes, + providedPartitions = Map("b" -> Some("1"), "c" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + checkProjectList(actual, expected) + } + + // Test the case having a single static partition column. + { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val expected = nonPartitionedAttributes ++ Seq(Cast(Literal("1"), IntegerType)) + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes, + providedPartitions = Map("b" -> Some("1")), + targetAttributes = Seq('a.int, 'd.int, 'b.int), + targetPartitionSchema = new StructType().add("b", IntegerType)) + checkProjectList(actual, expected) + } + } + + test(s"Static partition and dynamic partition (caseSensitive: $caseSensitive)") { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val dynamicPartitionAttributes = Seq('g.int) + val expected = + nonPartitionedAttributes ++ + Seq(Cast(Literal("1"), IntegerType)) ++ + dynamicPartitionAttributes + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes ++ dynamicPartitionAttributes, + providedPartitions = Map("b" -> Some("1"), "c" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + checkProjectList(actual, expected) + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index f6675f09041d3..8773993d362c4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -66,7 +66,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) catalog.OrcConversions :: catalog.CreateTables :: PreprocessTableInsertion(conf) :: - DataSourceAnalysis :: + DataSourceAnalysis(conf) :: (if (conf.runSQLonFile) new ResolveDataSource(sparkSession) :: Nil else Nil) override val extendedCheckRules = Seq(PreWriteCheck(conf, catalog)) 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 b3896484da178..97cd29f541ed9 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 @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.mapred.{FileOutputFormat, JobConf} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} @@ -196,7 +197,7 @@ case class InsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - throw new SparkException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } 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 d4ebd051d2eed..46432512bac8d 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 @@ -19,11 +19,11 @@ package org.apache.spark.sql.hive import java.io.File -import org.apache.hadoop.hive.conf.HiveConf import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -331,7 +331,11 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef withTable(hiveTable) { withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { - sql(s"CREATE TABLE $hiveTable (a INT) PARTITIONED BY (b INT, c INT) STORED AS TEXTFILE") + sql( + s""" + |CREATE TABLE $hiveTable (a INT, d INT) + |PARTITIONED BY (b INT, c INT) STORED AS TEXTFILE + """.stripMargin) f(hiveTable) } } @@ -343,7 +347,11 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef val dsTable = "ds_table" withTable(dsTable) { - sql(s"CREATE TABLE $dsTable (a INT, b INT, c INT) USING PARQUET PARTITIONED BY (b, c)") + sql( + s""" + |CREATE TABLE $dsTable (a INT, b INT, c INT, d INT) + |USING PARQUET PARTITIONED BY (b, c) + """.stripMargin) f(dsTable) } } @@ -356,7 +364,7 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef testPartitionedTable("partitionBy() can't be used together with insertInto()") { tableName => val cause = intercept[AnalysisException] { - Seq((1, 2, 3)).toDF("a", "b", "c").write.partitionBy("b", "c").insertInto(tableName) + Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d").write.partitionBy("b", "c").insertInto(tableName) } assert(cause.getMessage.contains("insertInto() can't be used together with partitionBy().")) @@ -382,14 +390,83 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef assert(e.message.contains("the number of columns are different")) } - testPartitionedTable( - "SPARK-16037: INSERT statement should match columns by position") { + testPartitionedTable("SPARK-16037: INSERT statement should match columns by position") { tableName => withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { - sql(s"INSERT INTO TABLE $tableName SELECT 1, 2 AS c, 3 AS b") - checkAnswer(sql(s"SELECT a, b, c FROM $tableName"), Row(1, 2, 3)) - sql(s"INSERT OVERWRITE TABLE $tableName SELECT 1, 2, 3") - checkAnswer(sql(s"SELECT a, b, c FROM $tableName"), Row(1, 2, 3)) + sql(s"INSERT INTO TABLE $tableName SELECT 1, 4, 2 AS c, 3 AS b") + checkAnswer(sql(s"SELECT a, b, c, d FROM $tableName"), Row(1, 2, 3, 4)) + sql(s"INSERT OVERWRITE TABLE $tableName SELECT 1, 4, 2, 3") + checkAnswer(sql(s"SELECT a, b, c, 4 FROM $tableName"), Row(1, 2, 3, 4)) + } + } + + testPartitionedTable("INSERT INTO a partitioned table (semantic and error handling)") { + tableName => + withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=2, c=3) SELECT 1, 4") + + sql(s"INSERT INTO TABLE $tableName PARTITION (b=6, c=7) SELECT 5, 8") + + sql(s"INSERT INTO TABLE $tableName PARTITION (c=11, b=10) SELECT 9, 12") + + // c is defined twice. Parser will complain. + intercept[ParseException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c=15, c=16) SELECT 13") + } + + // d is not a partitioning column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c=15, d=16) SELECT 13, 14") + } + + // d is not a partitioning column. The total number of columns is correct. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c=15, d=16) SELECT 13") + } + + // The data is missing a column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (c=15, b=16) SELECT 13") + } + + // d is not a partitioning column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=15, d=15) SELECT 13, 14") + } + + // The statement is missing a column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=15) SELECT 13, 14") + } + + // The statement is missing a column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=15) SELECT 13, 14, 16") + } + + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c) SELECT 13, 16, 15") + + // Dynamic partitioning columns need to be after static partitioning columns. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b, c=19) SELECT 17, 20, 18") + } + + sql(s"INSERT INTO TABLE $tableName PARTITION (b, c) SELECT 17, 20, 18, 19") + + sql(s"INSERT INTO TABLE $tableName PARTITION (c, b) SELECT 21, 24, 22, 23") + + sql(s"INSERT INTO TABLE $tableName SELECT 25, 28, 26, 27") + + checkAnswer( + sql(s"SELECT a, b, c, d FROM $tableName"), + Row(1, 2, 3, 4) :: + Row(5, 6, 7, 8) :: + Row(9, 10, 11, 12) :: + Row(13, 14, 15, 16) :: + Row(17, 18, 19, 20) :: + Row(21, 22, 23, 24) :: + Row(25, 26, 27, 28) :: Nil + ) } } } 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 85b159e2a5ded..f8c55ec45650e 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 @@ -1006,7 +1006,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SET hive.exec.dynamic.partition.mode=nonstrict") // Should throw when a static partition appears after a dynamic partition - intercept[SparkException] { + intercept[AnalysisException] { sql( """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1) |SELECT key, value, key % 5 FROM src From 363db9f8be53773238854ab16c3459ba46a6961b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 20 Jun 2016 10:35:37 -0700 Subject: [PATCH 0723/1470] [SPARK-16050][TESTS] Remove the flaky test: ConsoleSinkSuite ## What changes were proposed in this pull request? ConsoleSinkSuite just collects content from stdout and compare them with the expected string. However, because Spark may not stop some background threads at once, there is a race condition that other threads are outputting logs to **stdout** while ConsoleSinkSuite is running. Then it will make ConsoleSinkSuite fail. Therefore, I just deleted `ConsoleSinkSuite`. If we want to test ConsoleSinkSuite in future, we should refactoring ConsoleSink to make it testable instead of depending on stdout. Therefore, this test is useless and I just delete it. ## How was this patch tested? Just removed a flaky test. Author: Shixiong Zhu Closes #13776 from zsxwing/SPARK-16050. (cherry picked from commit 5cfabec8724714b897d6e23e670c39e58f554ea2) Signed-off-by: Michael Armbrust --- .../streaming/ConsoleSinkSuite.scala | 99 ------------------- 1 file changed, 99 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala deleted file mode 100644 index e853d8c465419..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ConsoleSinkSuite.scala +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming - -import java.io.{ByteArrayOutputStream, PrintStream} -import java.nio.charset.StandardCharsets.UTF_8 - -import org.scalatest.BeforeAndAfter - -import org.apache.spark.sql.streaming.StreamTest - -class ConsoleSinkSuite extends StreamTest with BeforeAndAfter { - - import testImplicits._ - - after { - sqlContext.streams.active.foreach(_.stop()) - } - - test("SPARK-16020 Complete mode aggregation with console sink") { - withTempDir { checkpointLocation => - val origOut = System.out - val stdout = new ByteArrayOutputStream() - try { - // Hook Java System.out.println - System.setOut(new PrintStream(stdout)) - // Hook Scala println - Console.withOut(stdout) { - val input = MemoryStream[String] - val df = input.toDF().groupBy("value").count() - val query = df.writeStream - .format("console") - .outputMode("complete") - .option("checkpointLocation", checkpointLocation.getAbsolutePath) - .start() - input.addData("a") - query.processAllAvailable() - input.addData("a", "b") - query.processAllAvailable() - input.addData("a", "b", "c") - query.processAllAvailable() - query.stop() - } - System.out.flush() - } finally { - System.setOut(origOut) - } - - val expected = """------------------------------------------- - |Batch: 0 - |------------------------------------------- - |+-----+-----+ - ||value|count| - |+-----+-----+ - || a| 1| - |+-----+-----+ - | - |------------------------------------------- - |Batch: 1 - |------------------------------------------- - |+-----+-----+ - ||value|count| - |+-----+-----+ - || a| 2| - || b| 1| - |+-----+-----+ - | - |------------------------------------------- - |Batch: 2 - |------------------------------------------- - |+-----+-----+ - ||value|count| - |+-----+-----+ - || a| 3| - || b| 2| - || c| 1| - |+-----+-----+ - | - |""".stripMargin - assert(expected === new String(stdout.toByteArray, UTF_8)) - } - } - -} From bb80d1c24a633ceb4ad63b1fa8c02c66d79b2540 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Jun 2016 11:12:41 -0700 Subject: [PATCH 0724/1470] [SPARK-16059][R] Add `monotonically_increasing_id` function in SparkR ## What changes were proposed in this pull request? This PR adds `monotonically_increasing_id` column function in SparkR for API parity. After this PR, SparkR supports the followings. ```r > df <- read.json("examples/src/main/resources/people.json") > collect(select(df, monotonically_increasing_id(), df$name, df$age)) monotonically_increasing_id() name age 1 0 Michael NA 2 1 Andy 30 3 2 Justin 19 ``` ## How was this patch tested? Pass the Jenkins tests (with added testcase). Author: Dongjoon Hyun Closes #13774 from dongjoon-hyun/SPARK-16059. (cherry picked from commit 9613424898fd2a586156bc4eb48e255749774f20) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 27 +++++++++++++++++++++++ R/pkg/R/generics.R | 5 +++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 4 files changed, 34 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 82e56ca437299..0cfe1902794ab 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -218,6 +218,7 @@ exportMethods("%in%", "mean", "min", "minute", + "monotonically_increasing_id", "month", "months_between", "n", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index a779127b379a0..0fb38bc2891ad 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -911,6 +911,33 @@ setMethod("minute", column(jc) }) +#' monotonically_increasing_id +#' +#' Return a column that generates monotonically increasing 64-bit integers. +#' +#' The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. +#' The current implementation puts the partition ID in the upper 31 bits, and the record number +#' within each partition in the lower 33 bits. The assumption is that the SparkDataFrame has +#' less than 1 billion partitions, and each partition has less than 8 billion records. +#' +#' As an example, consider a SparkDataFrame with two partitions, each with 3 records. +#' This expression would return the following IDs: +#' 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. +#' +#' This is equivalent to the MONOTONICALLY_INCREASING_ID function in SQL. +#' +#' @rdname monotonically_increasing_id +#' @name monotonically_increasing_id +#' @family misc_funcs +#' @export +#' @examples \dontrun{select(df, monotonically_increasing_id())} +setMethod("monotonically_increasing_id", + signature(x = "missing"), + function() { + jc <- callJStatic("org.apache.spark.sql.functions", "monotonically_increasing_id") + column(jc) + }) + #' month #' #' Extracts the month as an integer from a given date/timestamp/string. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 6e754afab6c6d..37d05560c3e00 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -993,6 +993,11 @@ setGeneric("md5", function(x) { standardGeneric("md5") }) #' @export setGeneric("minute", function(x) { standardGeneric("minute") }) +#' @rdname monotonically_increasing_id +#' @export +setGeneric("monotonically_increasing_id", + function(x) { standardGeneric("monotonically_increasing_id") }) + #' @rdname month #' @export setGeneric("month", function(x) { standardGeneric("month") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index fcc2ab3ed6a2b..c5c5a069a817f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1047,7 +1047,7 @@ test_that("column functions", { c5 <- hour(c) + initcap(c) + last(c) + last_day(c) + length(c) c6 <- log(c) + (c) + log1p(c) + log2(c) + lower(c) + ltrim(c) + max(c) + md5(c) c7 <- mean(c) + min(c) + month(c) + negate(c) + quarter(c) - c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + sha1(c) + c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + sha1(c) + monotonically_increasing_id() c9 <- signum(c) + sin(c) + sinh(c) + size(c) + stddev(c) + soundex(c) + sqrt(c) + sum(c) c10 <- sumDistinct(c) + tan(c) + tanh(c) + toDegrees(c) + toRadians(c) c11 <- to_date(c) + trim(c) + unbase64(c) + unhex(c) + upper(c) From 5b22e34e96f7795a0e8d547eba2229b60f999fa5 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 20 Jun 2016 11:24:41 -0700 Subject: [PATCH 0725/1470] [SPARK-16029][SPARKR] SparkR add dropTempView and deprecate dropTempTable ## What changes were proposed in this pull request? Add dropTempView and deprecate dropTempTable ## How was this patch tested? unit tests shivaram liancheng Author: Felix Cheung Closes #13753 from felixcheung/rdroptempview. (cherry picked from commit 36e812d4b695566437c6bac991ef06a0f81fb1c5) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/SQLContext.R | 39 +++++++++++++++++++---- R/pkg/inst/tests/testthat/test_sparkSQL.R | 14 ++++---- 3 files changed, 41 insertions(+), 13 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 0cfe1902794ab..cc129a73fed0f 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -299,6 +299,7 @@ export("as.DataFrame", "createDataFrame", "createExternalTable", "dropTempTable", + "dropTempView", "jsonFile", "loadDF", "parquetFile", diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 3232241f8af55..b0ccc42ff829a 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -599,13 +599,14 @@ clearCache <- function() { dispatchFunc("clearCache()") } -#' Drop Temporary Table +#' (Deprecated) 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 tableName The name of the SparkSQL table to be dropped. -#' @rdname dropTempTable +#' @seealso \link{dropTempView} +#' @rdname dropTempTable-deprecated #' @export #' @examples #' \dontrun{ @@ -619,16 +620,42 @@ clearCache <- function() { #' @method dropTempTable default dropTempTable.default <- function(tableName) { - sparkSession <- getSparkSession() if (class(tableName) != "character") { stop("tableName must be a string.") } - catalog <- callJMethod(sparkSession, "catalog") - callJMethod(catalog, "dropTempView", tableName) + dropTempView(tableName) } dropTempTable <- function(x, ...) { - dispatchFunc("dropTempTable(tableName)", x, ...) + .Deprecated("dropTempView") + dispatchFunc("dropTempView(viewName)", x, ...) +} + +#' Drops the temporary view with the given view name in the catalog. +#' +#' Drops the temporary view with the given view name in the catalog. +#' If the view has been cached before, then it will also be uncached. +#' +#' @param viewName the name of the view to be dropped. +#' @rdname dropTempView +#' @name dropTempView +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' df <- read.df(path, "parquet") +#' createOrReplaceTempView(df, "table") +#' dropTempView("table") +#' } +#' @note since 2.0.0 + +dropTempView <- function(viewName) { + sparkSession <- getSparkSession() + if (class(viewName) != "character") { + stop("viewName must be a string.") + } + catalog <- callJMethod(sparkSession, "catalog") + callJMethod(catalog, "dropTempView", viewName) } #' Load a SparkDataFrame diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index c5c5a069a817f..ceba0d138e27d 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -472,8 +472,8 @@ test_that("test tableNames and tables", { suppressWarnings(registerTempTable(df, "table2")) tables <- tables() expect_equal(count(tables), 2) - dropTempTable("table1") - dropTempTable("table2") + suppressWarnings(dropTempTable("table1")) + dropTempView("table2") tables <- tables() expect_equal(count(tables), 0) @@ -486,7 +486,7 @@ test_that( newdf <- sql("SELECT * FROM table1 where name = 'Michael'") expect_is(newdf, "SparkDataFrame") expect_equal(count(newdf), 1) - dropTempTable("table1") + dropTempView("table1") }) test_that("test cache, uncache and clearCache", { @@ -495,7 +495,7 @@ test_that("test cache, uncache and clearCache", { cacheTable("table1") uncacheTable("table1") clearCache() - dropTempTable("table1") + dropTempView("table1") }) test_that("insertInto() on a registered table", { @@ -516,13 +516,13 @@ test_that("insertInto() on a registered table", { insertInto(dfParquet2, "table1") expect_equal(count(sql("select * from table1")), 5) expect_equal(first(sql("select * from table1 order by age"))$name, "Michael") - dropTempTable("table1") + dropTempView("table1") createOrReplaceTempView(dfParquet, "table1") insertInto(dfParquet2, "table1", overwrite = TRUE) expect_equal(count(sql("select * from table1")), 2) expect_equal(first(sql("select * from table1 order by age"))$name, "Bob") - dropTempTable("table1") + dropTempView("table1") unlink(jsonPath2) unlink(parquetPath2) @@ -536,7 +536,7 @@ test_that("tableToDF() returns a new DataFrame", { expect_equal(count(tabledf), 3) tabledf2 <- tableToDF("table1") expect_equal(count(tabledf2), 3) - dropTempTable("table1") + dropTempView("table1") }) test_that("toRDD() returns an RRDD", { From ead872e4996ad0c0b02debd1ab829ff67b79abfb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Jun 2016 11:30:26 -0700 Subject: [PATCH 0726/1470] [SPARK-16051][R] Add `read.orc/write.orc` to SparkR ## What changes were proposed in this pull request? This issue adds `read.orc/write.orc` to SparkR for API parity. ## How was this patch tested? Pass the Jenkins tests (with new testcases). Author: Dongjoon Hyun Closes #13763 from dongjoon-hyun/SPARK-16051. (cherry picked from commit c44bf137c7ca649e0c504229eb3e6ff7955e9a53) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 2 ++ R/pkg/R/DataFrame.R | 27 +++++++++++++++++++++++ R/pkg/R/SQLContext.R | 21 +++++++++++++++++- R/pkg/R/generics.R | 4 ++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 21 ++++++++++++++++++ 5 files changed, 74 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index cc129a73fed0f..aaeab665a4690 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -117,6 +117,7 @@ exportMethods("arrange", "write.df", "write.jdbc", "write.json", + "write.orc", "write.parquet", "write.text", "write.ml") @@ -306,6 +307,7 @@ export("as.DataFrame", "read.df", "read.jdbc", "read.json", + "read.orc", "read.parquet", "read.text", "spark.lapply", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ea091c81016d4..f3a3eff46dd93 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -701,6 +701,33 @@ setMethod("write.json", invisible(callJMethod(write, "json", path)) }) +#' Save the contents of SparkDataFrame as an ORC file, preserving the schema. +#' +#' Save the contents of a SparkDataFrame as an ORC file, preserving the schema. Files written out +#' with this method can be read back in as a SparkDataFrame using read.orc(). +#' +#' @param x A SparkDataFrame +#' @param path The directory where the file is saved +#' +#' @family SparkDataFrame functions +#' @rdname write.orc +#' @name write.orc +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' write.orc(df, "/tmp/sparkr-tmp1/") +#' } +#' @note write.orc since 2.0.0 +setMethod("write.orc", + signature(x = "SparkDataFrame", path = "character"), + function(x, path) { + write <- callJMethod(x@sdf, "write") + invisible(callJMethod(write, "orc", path)) + }) + #' Save the contents of SparkDataFrame as a Parquet file, preserving the schema. #' #' Save the contents of a SparkDataFrame as a Parquet file, preserving the schema. Files written out diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index b0ccc42ff829a..b7e1c062c714e 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -330,6 +330,25 @@ jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { } } +#' Create a SparkDataFrame from an ORC file. +#' +#' Loads an ORC file, returning the result as a SparkDataFrame. +#' +#' @param path Path of file to read. +#' @return SparkDataFrame +#' @rdname read.orc +#' @export +#' @name read.orc +#' @note read.orc since 2.0.0 +read.orc <- function(path) { + sparkSession <- getSparkSession() + # Allow the user to have a more flexible definiton of the ORC file path + path <- suppressWarnings(normalizePath(path)) + read <- callJMethod(sparkSession, "read") + sdf <- callJMethod(read, "orc", path) + dataFrame(sdf) +} + #' Create a SparkDataFrame from a Parquet file. #' #' Loads a Parquet file, returning the result as a SparkDataFrame. @@ -343,7 +362,7 @@ jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { read.parquet.default <- function(path) { sparkSession <- getSparkSession() - # Allow the user to have a more flexible definiton of the text file path + # Allow the user to have a more flexible definiton of the Parquet file path paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sparkSession, "read") sdf <- callJMethod(read, "parquet", paths) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 37d05560c3e00..dcc1cf241f42f 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -610,6 +610,10 @@ setGeneric("write.jdbc", function(x, url, tableName, mode = "error", ...) { #' @export setGeneric("write.json", function(x, path) { standardGeneric("write.json") }) +#' @rdname write.orc +#' @export +setGeneric("write.orc", function(x, path) { standardGeneric("write.orc") }) + #' @rdname write.parquet #' @export setGeneric("write.parquet", function(x, path) { standardGeneric("write.parquet") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index ceba0d138e27d..114fec6e36d52 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -68,6 +68,7 @@ mockLines <- c("{\"name\":\"Michael\"}", "{\"name\":\"Justin\", \"age\":19}") jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") parquetPath <- tempfile(pattern = "sparkr-test", fileext = ".parquet") +orcPath <- tempfile(pattern = "sparkr-test", fileext = ".orc") writeLines(mockLines, jsonPath) # For test nafunctions, like dropna(), fillna(),... @@ -1667,6 +1668,25 @@ test_that("mutate(), transform(), rename() and names()", { detach(airquality) }) +test_that("read/write ORC files", { + df <- read.df(jsonPath, "json") + + # Test write.df and read.df + write.df(df, orcPath, "orc", mode = "overwrite") + df2 <- read.df(orcPath, "orc") + expect_is(df2, "SparkDataFrame") + expect_equal(count(df), count(df2)) + + # Test write.orc and read.orc + orcPath2 <- tempfile(pattern = "orcPath2", fileext = ".orc") + write.orc(df, orcPath2) + orcDF <- read.orc(orcPath2) + expect_is(orcDF, "SparkDataFrame") + expect_equal(count(orcDF), count(df)) + + unlink(orcPath2) +}) + test_that("read/write Parquet files", { df <- read.df(jsonPath, "json") # Test write.df and read.df @@ -2351,5 +2371,6 @@ test_that("enableHiveSupport on SparkSession", { }) unlink(parquetPath) +unlink(orcPath) unlink(jsonPath) unlink(jsonPathNa) From d2c94e6a45090cf545fe1e243f3dfde5ed87b4d0 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 20 Jun 2016 12:08:42 -0700 Subject: [PATCH 0727/1470] [SPARK-16028][SPARKR] spark.lapply can work with active context ## What changes were proposed in this pull request? spark.lapply and setLogLevel ## How was this patch tested? unit test shivaram thunterdb Author: Felix Cheung Closes #13752 from felixcheung/rlapply. (cherry picked from commit 46d98e0a1f40a4c6ae92253c5c498a3a924497fc) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/context.R | 20 +++++++++++++------- R/pkg/inst/tests/testthat/test_context.R | 6 +++--- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 5c886030ff5c5..968a9d2251b18 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -252,17 +252,20 @@ setCheckpointDir <- function(sc, dirName) { #' } #' #' @rdname spark.lapply -#' @param sc Spark Context to use #' @param list the list of elements #' @param func a function that takes one argument. #' @return a list of results (the exact type being determined by the function) #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' doubled <- spark.lapply(sc, 1:10, function(x){2 * x}) +#' sparkR.session() +#' doubled <- spark.lapply(1:10, function(x){2 * x}) #'} -spark.lapply <- function(sc, list, func) { +spark.lapply <- function(list, func) { + if (!exists(".sparkRjsc", envir = .sparkREnv)) { + stop("SparkR has not been initialized. Please call sparkR.session()") + } + sc <- get(".sparkRjsc", envir = .sparkREnv) rdd <- parallelize(sc, list, length(list)) results <- map(rdd, func) local <- collect(results) @@ -274,14 +277,17 @@ spark.lapply <- function(sc, list, func) { #' Set new log level: "ALL", "DEBUG", "ERROR", "FATAL", "INFO", "OFF", "TRACE", "WARN" #' #' @rdname setLogLevel -#' @param sc Spark Context to use #' @param level New log level #' @export #' @examples #'\dontrun{ -#' setLogLevel(sc, "ERROR") +#' setLogLevel("ERROR") #'} -setLogLevel <- function(sc, level) { +setLogLevel <- function(level) { + if (!exists(".sparkRjsc", envir = .sparkREnv)) { + stop("SparkR has not been initialized. Please call sparkR.session()") + } + sc <- get(".sparkRjsc", envir = .sparkREnv) callJMethod(sc, "setLogLevel", level) } diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index f123187adf3ef..b149818ff46f6 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -107,8 +107,8 @@ test_that("job group functions can be called", { }) test_that("utility function can be called", { - sc <- sparkR.sparkContext() - setLogLevel(sc, "ERROR") + sparkR.sparkContext() + setLogLevel("ERROR") sparkR.session.stop() }) @@ -161,7 +161,7 @@ test_that("sparkJars sparkPackages as comma-separated strings", { test_that("spark.lapply should perform simple transforms", { sc <- sparkR.sparkContext() - doubled <- spark.lapply(sc, 1:10, function(x) { 2 * x }) + doubled <- spark.lapply(1:10, function(x) { 2 * x }) expect_equal(doubled, as.list(2 * 1:10)) sparkR.session.stop() }) From dfa920204e3407c38df9012ca42b7b56c416a5b3 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 20 Jun 2016 12:31:00 -0700 Subject: [PATCH 0728/1470] [SPARKR] fix R roxygen2 doc for count on GroupedData ## What changes were proposed in this pull request? fix code doc ## How was this patch tested? manual shivaram Author: Felix Cheung Closes #13782 from felixcheung/rcountdoc. (cherry picked from commit aee1420eca64dfc145f31b8c653388fafc5ccd8f) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/group.R | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index eba083fe4b124..65b9e841be0d7 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -58,7 +58,7 @@ setMethod("show", "GroupedData", #' #' @param x a GroupedData #' @return a SparkDataFrame -#' @rdname agg +#' @rdname count #' @export #' @examples #' \dontrun{ @@ -83,6 +83,7 @@ setMethod("count", #' @rdname summarize #' @name agg #' @family agg_funcs +#' @export #' @examples #' \dontrun{ #' df2 <- agg(df, age = "sum") # new column name will be created as 'SUM(age#0)' @@ -160,6 +161,7 @@ createMethods() #' @return a SparkDataFrame #' @rdname gapply #' @name gapply +#' @export #' @examples #' \dontrun{ #' Computes the arithmetic mean of the second column by grouping From 45c41aa33b39bfc38b8615fde044356a590edcfb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Jun 2016 13:41:03 -0700 Subject: [PATCH 0729/1470] [SPARK-16053][R] Add `spark_partition_id` in SparkR ## What changes were proposed in this pull request? This PR adds `spark_partition_id` virtual column function in SparkR for API parity. The following is just an example to illustrate a SparkR usage on a partitioned parquet table created by `spark.range(10).write.mode("overwrite").parquet("/tmp/t1")`. ```r > collect(select(read.parquet('/tmp/t1'), c('id', spark_partition_id()))) id SPARK_PARTITION_ID() 1 3 0 2 4 0 3 8 1 4 9 1 5 0 2 6 1 3 7 2 4 8 5 5 9 6 6 10 7 7 ``` ## How was this patch tested? Pass the Jenkins tests (including new testcase). Author: Dongjoon Hyun Closes #13768 from dongjoon-hyun/SPARK-16053. (cherry picked from commit b0f2fb5b9729b38744bf784f2072f5ee52314f87) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 21 +++++++++++++++++++++ R/pkg/R/generics.R | 4 ++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 1 + 4 files changed, 27 insertions(+) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index aaeab665a4690..45663f4c2c70a 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -260,6 +260,7 @@ exportMethods("%in%", "skewness", "sort_array", "soundex", + "spark_partition_id", "stddev", "stddev_pop", "stddev_samp", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 0fb38bc2891ad..c26f9632589ff 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1206,6 +1206,27 @@ setMethod("soundex", column(jc) }) +#' Return the partition ID as a column +#' +#' Return the partition ID of the Spark task as a SparkDataFrame column. +#' Note that this is nondeterministic because it depends on data partitioning and +#' task scheduling. +#' +#' This is equivalent to the SPARK_PARTITION_ID function in SQL. +#' +#' @rdname spark_partition_id +#' @name spark_partition_id +#' @export +#' @examples +#' \dontrun{select(df, spark_partition_id())} +#' @note spark_partition_id since 2.0.0 +setMethod("spark_partition_id", + signature(x = "missing"), + function() { + jc <- callJStatic("org.apache.spark.sql.functions", "spark_partition_id") + column(jc) + }) + #' @rdname sd #' @name stddev setMethod("stddev", diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index dcc1cf241f42f..f6b9276d86f39 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1135,6 +1135,10 @@ setGeneric("sort_array", function(x, asc = TRUE) { standardGeneric("sort_array") #' @export setGeneric("soundex", function(x) { standardGeneric("soundex") }) +#' @rdname spark_partition_id +#' @export +setGeneric("spark_partition_id", function(x) { standardGeneric("spark_partition_id") }) + #' @rdname sd #' @export setGeneric("stddev", function(x) { standardGeneric("stddev") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 114fec6e36d52..d53c40d42382b 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1059,6 +1059,7 @@ test_that("column functions", { c16 <- is.nan(c) + isnan(c) + isNaN(c) c17 <- cov(c, c1) + cov("c", "c1") + covar_samp(c, c1) + covar_samp("c", "c1") c18 <- covar_pop(c, c1) + covar_pop("c", "c1") + c19 <- spark_partition_id() # Test if base::is.nan() is exposed expect_equal(is.nan(c("a", "b")), c(FALSE, FALSE)) From f90b2ea1d96bba4650b8d1ce37a60c81c89bca96 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 20 Jun 2016 13:46:24 -0700 Subject: [PATCH 0730/1470] [SPARK-15159][SPARKR] SparkSession roxygen2 doc, programming guide, example updates ## What changes were proposed in this pull request? roxygen2 doc, programming guide, example updates ## How was this patch tested? manual checks shivaram Author: Felix Cheung Closes #13751 from felixcheung/rsparksessiondoc. (cherry picked from commit 359c2e827d5682249c009e83379a5ee8e5aa4e89) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 169 ++++++++---------------- R/pkg/R/SQLContext.R | 47 +++---- R/pkg/R/mllib.R | 6 +- R/pkg/R/schema.R | 24 ++-- R/pkg/R/sparkR.R | 7 +- docs/sparkr.md | 99 +++++++------- examples/src/main/r/data-manipulation.R | 15 +-- examples/src/main/r/dataframe.R | 13 +- examples/src/main/r/ml.R | 21 ++- 9 files changed, 162 insertions(+), 239 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index f3a3eff46dd93..583d3aefd4e6b 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -35,12 +35,11 @@ setOldClass("structType") #' @slot env An R environment that stores bookkeeping states of the SparkDataFrame #' @slot sdf A Java object reference to the backing Scala DataFrame #' @seealso \link{createDataFrame}, \link{read.json}, \link{table} -#' @seealso \url{https://spark.apache.org/docs/latest/sparkr.html#sparkr-dataframes} +#' @seealso \url{https://spark.apache.org/docs/latest/sparkr.html#sparkdataframe} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df <- createDataFrame(faithful) #'} setClass("SparkDataFrame", @@ -77,8 +76,7 @@ dataFrame <- function(sdf, isCached = FALSE) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' printSchema(df) @@ -102,8 +100,7 @@ setMethod("printSchema", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' dfSchema <- schema(df) @@ -126,8 +123,7 @@ setMethod("schema", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' explain(df, TRUE) @@ -157,8 +153,7 @@ setMethod("explain", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' isLocal(df) @@ -182,8 +177,7 @@ setMethod("isLocal", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' showDF(df) @@ -207,8 +201,7 @@ setMethod("showDF", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' df @@ -234,8 +227,7 @@ setMethod("show", "SparkDataFrame", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' dtypes(df) @@ -261,8 +253,7 @@ setMethod("dtypes", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' columns(df) @@ -396,8 +387,7 @@ setMethod("coltypes", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' coltypes(df) <- c("character", "integer") @@ -432,7 +422,7 @@ setMethod("coltypes<-", #' Creates a temporary view using the given name. #' -#' Creates a new temporary view using a SparkDataFrame in the SQLContext. If a +#' Creates a new temporary view using a SparkDataFrame in the Spark Session. If a #' temporary view with the same name already exists, replaces it. #' #' @param x A SparkDataFrame @@ -444,8 +434,7 @@ setMethod("coltypes<-", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' createOrReplaceTempView(df, "json_df") @@ -469,8 +458,7 @@ setMethod("createOrReplaceTempView", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' registerTempTable(df, "json_df") @@ -498,8 +486,7 @@ setMethod("registerTempTable", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df <- read.df(path, "parquet") #' df2 <- read.df(path2, "parquet") #' createOrReplaceTempView(df, "table1") @@ -526,8 +513,7 @@ setMethod("insertInto", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' cache(df) @@ -554,8 +540,7 @@ setMethod("cache", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' persist(df, "MEMORY_AND_DISK") @@ -582,8 +567,7 @@ setMethod("persist", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' persist(df, "MEMORY_AND_DISK") @@ -617,8 +601,7 @@ setMethod("unpersist", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' newDF <- repartition(df, 2L) @@ -660,8 +643,7 @@ setMethod("repartition", #' @noRd #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' newRDD <- toJSON(df) @@ -688,8 +670,7 @@ setMethod("toJSON", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' write.json(df, "/tmp/sparkr-tmp/") @@ -742,8 +723,7 @@ setMethod("write.orc", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' write.parquet(df, "/tmp/sparkr-tmp1/") @@ -781,8 +761,7 @@ setMethod("saveAsParquetFile", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.txt" #' df <- read.text(path) #' write.text(df, "/tmp/sparkr-tmp/") @@ -806,8 +785,7 @@ setMethod("write.text", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' distinctDF <- distinct(df) @@ -842,8 +820,7 @@ setMethod("unique", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' collect(sample(df, FALSE, 0.5)) @@ -883,8 +860,7 @@ setMethod("sample_frac", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' count(df) @@ -913,8 +889,7 @@ setMethod("nrow", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' ncol(df) @@ -936,8 +911,7 @@ setMethod("ncol", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' dim(df) @@ -960,8 +934,7 @@ setMethod("dim", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' collected <- collect(df) @@ -1032,8 +1005,7 @@ setMethod("collect", #' @export #' @examples #' \dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' limitedDF <- limit(df, 10) @@ -1053,8 +1025,7 @@ setMethod("limit", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' take(df, 2) @@ -1082,8 +1053,7 @@ setMethod("take", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' head(df) @@ -1105,8 +1075,7 @@ setMethod("head", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' first(df) @@ -1126,8 +1095,7 @@ setMethod("first", #' @noRd #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' rdd <- toRDD(df) @@ -1662,8 +1630,7 @@ setMethod("select", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' selectExpr(df, "col1", "(col2 * 5) as newCol") @@ -1692,8 +1659,7 @@ setMethod("selectExpr", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' newDF <- withColumn(df, "newCol", df$col1 * 5) @@ -1721,8 +1687,7 @@ setMethod("withColumn", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' newDF <- mutate(df, newCol = df$col1 * 5, newCol2 = df$col1 * 2) @@ -1812,8 +1777,7 @@ setMethod("transform", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' newDF <- withColumnRenamed(df, "col1", "newCol1") @@ -1837,8 +1801,7 @@ setMethod("withColumnRenamed", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' newDF <- rename(df, col1 = df$newCol1) @@ -1881,8 +1844,7 @@ setClassUnion("characterOrColumn", c("character", "Column")) #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' arrange(df, df$col1) @@ -1957,8 +1919,7 @@ setMethod("orderBy", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' filter(df, "col1 > 0") @@ -1998,8 +1959,7 @@ setMethod("where", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' dropDuplicates(df) @@ -2045,8 +2005,7 @@ setMethod("dropDuplicates", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) #' join(df1, df2) # Performs a Cartesian @@ -2103,8 +2062,7 @@ setMethod("join", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) #' merge(df1, df2) # Performs a Cartesian @@ -2235,8 +2193,7 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) #' unioned <- unionAll(df, df2) @@ -2279,8 +2236,7 @@ setMethod("rbind", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) #' intersectDF <- intersect(df, df2) @@ -2306,8 +2262,7 @@ setMethod("intersect", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) #' exceptDF <- except(df, df2) @@ -2349,8 +2304,7 @@ setMethod("except", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' write.df(df, "myfile", "parquet", "overwrite") @@ -2408,8 +2362,7 @@ setMethod("saveDF", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' saveAsTable(df, "myfile") @@ -2445,8 +2398,7 @@ setMethod("saveAsTable", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' describe(df) @@ -2501,10 +2453,9 @@ setMethod("summary", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- read.json(sqlCtx, path) +#' df <- read.json(path) #' dropna(df) #' } setMethod("dropna", @@ -2555,10 +2506,9 @@ setMethod("na.omit", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- read.json(sqlCtx, path) +#' df <- read.json(path) #' fillna(df, 1) #' fillna(df, list("age" = 20, "name" = "unknown")) #' } @@ -2769,10 +2719,9 @@ setMethod("str", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- read.json(sqlCtx, path) +#' df <- read.json(path) #' drop(df, "col1") #' drop(df, c("col1", "col2")) #' drop(df, df$col1) @@ -2943,8 +2892,7 @@ setMethod("histogram", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" #' write.jdbc(df, jdbcUrl, "table", user = "username", password = "password") #' } @@ -2972,8 +2920,7 @@ setMethod("write.jdbc", #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df <- createDataFrame(data.frame(id = 1:1000)) #' df_list <- randomSplit(df, c(2, 3, 5), 0) #' # df_list contains 3 SparkDataFrames with each having about 200, 300 and 500 rows respectively diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index b7e1c062c714e..b846b3dec246c 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -126,8 +126,7 @@ getDefaultSqlSource <- function() { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df1 <- as.DataFrame(iris) #' df2 <- as.DataFrame(list(3,4,5,6)) #' df3 <- createDataFrame(iris) @@ -238,8 +237,7 @@ as.DataFrame <- function(x, ...) { #' @noRd #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) #' df <- toDF(rdd) #'} @@ -261,8 +259,7 @@ setMethod("toDF", signature(x = "RDD"), #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' df <- jsonFile(path) @@ -309,8 +306,7 @@ jsonFile <- function(x, ...) { #' @noRd #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' rdd <- texFile(sc, "path/to/json") #' df <- jsonRDD(sqlContext, rdd) #'} @@ -401,8 +397,7 @@ parquetFile <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.txt" #' df <- read.text(path) #' } @@ -432,8 +427,7 @@ read.text <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' createOrReplaceTempView(df, "table") @@ -464,8 +458,7 @@ sql <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' createOrReplaceTempView(df, "table") @@ -489,8 +482,7 @@ tableToDF <- function(tableName) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' tables("hive") #' } #' @name tables @@ -516,8 +508,7 @@ tables <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' tableNames("hive") #' } #' @name tableNames @@ -545,8 +536,7 @@ tableNames <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' createOrReplaceTempView(df, "table") @@ -575,8 +565,7 @@ cacheTable <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) #' createOrReplaceTempView(df, "table") @@ -629,8 +618,7 @@ clearCache <- function() { #' @export #' @examples #' \dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df <- read.df(path, "parquet") #' createOrReplaceTempView(df, "table") #' dropTempTable("table") @@ -694,8 +682,7 @@ dropTempView <- function(viewName) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' df1 <- read.df("path/to/file.json", source = "json") #' schema <- structType(structField("name", "string"), #' structField("info", "map")) @@ -758,9 +745,8 @@ loadDF <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df <- sparkRSQL.createExternalTable("myjson", path="path/to/json", source="json") +#' sparkR.session() +#' df <- createExternalTable("myjson", path="path/to/json", source="json") #' } #' @name createExternalTable #' @method createExternalTable default @@ -806,8 +792,7 @@ createExternalTable <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" #' df <- read.jdbc(jdbcUrl, "table", predicates = list("field<=123"), user = "username") #' df2 <- read.jdbc(jdbcUrl, "table2", partitionColumn = "index", lowerBound = 0, diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index ba2eee2fca76a..2127daee18b0a 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -67,8 +67,7 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' @export #' @examples #' \dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' data(iris) #' df <- createDataFrame(iris) #' model <- spark.glm(df, Sepal_Length ~ Sepal_Width, family="gaussian") @@ -115,8 +114,7 @@ setMethod( #' @export #' @examples #' \dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' data(iris) #' df <- createDataFrame(iris) #' model <- glm(Sepal_Length ~ Sepal_Width, df, family="gaussian") diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R index 039aa008b3edd..89a2cfac3cb0d 100644 --- a/R/pkg/R/schema.R +++ b/R/pkg/R/schema.R @@ -29,11 +29,11 @@ #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' schema <- structType(structField("a", "integer"), structField("b", "string")) -#' df <- createDataFrame(sqlCtx, rdd, schema) +#' schema <- structType(structField("a", "integer"), structField("c", "string"), +#' structField("avg", "double")) +#' df1 <- gapply(df, list("a", "c"), +#' function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, +#' schema) #' } structType <- function(x, ...) { UseMethod("structType", x) @@ -90,13 +90,13 @@ print.structType <- function(x, ...) { #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' field1 <- structField("a", "integer", TRUE) -#' field2 <- structField("b", "string", TRUE) -#' schema <- structType(field1, field2) -#' df <- createDataFrame(sqlCtx, rdd, schema) +#' field1 <- structField("a", "integer") +#' field2 <- structField("c", "string") +#' field3 <- structField("avg", "double") +#' schema <- structType(field1, field2, field3) +#' df1 <- gapply(df, list("a", "c"), +#' function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, +#' schema) #' } structField <- function(x, ...) { diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 0dfd7b753033e..d05660ccf6c22 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -89,9 +89,7 @@ sparkR.session.stop <- function() { #' (Deprecated) Initialize a new Spark Context. #' -#' This function initializes a new SparkContext. For details on how to initialize -#' and use SparkR, refer to SparkR programming guide at -#' \url{http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparkcontext-sqlcontext}. +#' This function initializes a new SparkContext. #' #' @param master The Spark master URL #' @param appName Application name to register with cluster manager @@ -312,6 +310,9 @@ sparkRHive.init <- function(jsc = NULL) { #' Additional Spark properties can be set (...), and these named parameters take priority over #' over values in master, appName, named lists of sparkConfig. #' +#' For details on how to initialize and use SparkR, refer to SparkR programming guide at +#' \url{http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparksession}. +#' #' @param master The Spark master URL #' @param appName Application name to register with cluster manager #' @param sparkHome Spark Home directory diff --git a/docs/sparkr.md b/docs/sparkr.md index 961bd323fabcb..023bbcd39c7e3 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -14,29 +14,24 @@ supports operations like selection, filtering, aggregation etc. (similar to R da [dplyr](https://github.com/hadley/dplyr)) but on large datasets. SparkR also supports distributed machine learning using MLlib. -# SparkR DataFrames +# SparkDataFrame -A DataFrame is a distributed collection of data organized into named columns. It is conceptually +A SparkDataFrame is a distributed collection of data organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R, but with richer -optimizations under the hood. DataFrames can be constructed from a wide array of sources such as: +optimizations under the hood. SparkDataFrames can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing local R data frames. All of the examples on this page use sample data included in R or the Spark distribution and can be run using the `./bin/sparkR` shell. -## Starting Up: SparkContext, SQLContext +## Starting Up: SparkSession
    -The entry point into SparkR is the `SparkContext` which connects your R program to a Spark cluster. -You can create a `SparkContext` using `sparkR.init` and pass in options such as the application name -, any spark packages depended on, etc. Further, to work with DataFrames we will need a `SQLContext`, -which can be created from the SparkContext. If you are working from the `sparkR` shell, the -`SQLContext` and `SparkContext` should already be created for you, and you would not need to call -`sparkR.init`. +The entry point into SparkR is the `SparkSession` which connects your R program to a Spark cluster. +You can create a `SparkSession` using `sparkR.session` and pass in options such as the application name, any spark packages depended on, etc. Further, you can also work with SparkDataFrames via `SparkSession`. If you are working from the `sparkR` shell, the `SparkSession` should already be created for you, and you would not need to call `sparkR.session`.
    {% highlight r %} -sc <- sparkR.init() -sqlContext <- sparkRSQL.init(sc) +sparkR.session() {% endhighlight %}
    @@ -45,13 +40,13 @@ sqlContext <- sparkRSQL.init(sc) You can also start SparkR from RStudio. You can connect your R program to a Spark cluster from RStudio, R shell, Rscript or other R IDEs. To start, make sure SPARK_HOME is set in environment (you can check [Sys.getenv](https://stat.ethz.ch/R-manual/R-devel/library/base/html/Sys.getenv.html)), -load the SparkR package, and call `sparkR.init` as below. In addition to calling `sparkR.init`, you -could also specify certain Spark driver properties. Normally these +load the SparkR package, and call `sparkR.session` as below. In addition to calling `sparkR.session`, + you could also specify certain Spark driver properties. Normally these [Application properties](configuration.html#application-properties) and [Runtime Environment](configuration.html#runtime-environment) cannot be set programmatically, as the driver JVM process would have been started, in this case SparkR takes care of this for you. To set -them, pass them as you would other configuration properties in the `sparkEnvir` argument to -`sparkR.init()`. +them, pass them as you would other configuration properties in the `sparkConfig` argument to +`sparkR.session()`.
    {% highlight r %} @@ -59,11 +54,11 @@ if (nchar(Sys.getenv("SPARK_HOME")) < 1) { Sys.setenv(SPARK_HOME = "/home/spark") } library(SparkR, lib.loc = c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"))) -sc <- sparkR.init(master = "local[*]", sparkEnvir = list(spark.driver.memory="2g")) +sc <- sparkR.session(master = "local[*]", sparkConfig = list(spark.driver.memory="2g")) {% endhighlight %}
    -The following options can be set in `sparkEnvir` with `sparkR.init` from RStudio: +The following Spark driver properties can be set in `sparkConfig` with `sparkR.session` from RStudio:
    Executor IDExecutor ID Address Status RDD Blocks
    {info.id}{info.id} {info.hostPort} {executorStatus} From 2280ad8a3ddfff0b7cc10de6eadb2cc93423bbcf Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 16 Jun 2016 14:37:10 -0700 Subject: [PATCH 0688/1470] [SPARK-15749][SQL] make the error message more meaningful ## What changes were proposed in this pull request? For table test1 (C1 varchar (10), C2 varchar (10)), when I insert a row using ``` sqlContext.sql("insert into test1 values ('abc', 'def', 1)") ``` I got error message ``` Exception in thread "main" java.lang.RuntimeException: RelationC1#0,C2#1 JDBCRelation(test1) requires that the query in the SELECT clause of the INSERT INTO/OVERWRITE statement generates the same number of columns as its schema. ``` The error message is a little confusing. In my simple insert statement, it doesn't have a SELECT clause. I will change the error message to a more general one ``` Exception in thread "main" java.lang.RuntimeException: RelationC1#0,C2#1 JDBCRelation(test1) requires that the data to be inserted have the same number of columns as the target table. ``` ## How was this patch tested? I tested the patch using my simple unit test, but it's a very trivial change and I don't think I need to check in any test. Author: Huaxin Gao Closes #13492 from huaxingao/spark-15749. (cherry picked from commit 62d2fa5e996d428caaea005041b17ec115473762) Signed-off-by: Andrew Or --- .../org/apache/spark/sql/execution/datasources/rules.scala | 5 +++-- .../scala/org/apache/spark/sql/sources/InsertSuite.scala | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 7ac62fb191d40..543389efd5b95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -78,8 +78,9 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // schema of the relation. if (l.output.size != child.output.size) { sys.error( - s"$l requires that the query in the SELECT clause of the INSERT INTO/OVERWRITE " + - s"statement generates the same number of columns as its schema.") + s"$l requires that the data to be inserted have the same number of columns as the " + + s"target table: target table has ${l.output.size} column(s) but " + + s"the inserted data has ${child.output.size} column(s).") } castAndRenameChildOutput(i, l.output, child) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index bade41b1eddf9..d7179551d62df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -95,7 +95,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { """.stripMargin) }.getMessage assert( - message.contains("generates the same number of columns as its schema"), + message.contains("requires that the data to be inserted have the same number of columns"), "SELECT clause generating a different number of columns should not be not allowed." ) } From 8f713885963a410571370faa6b147cd0ada3832b Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Thu, 16 Jun 2016 16:44:54 -0700 Subject: [PATCH 0689/1470] [SPARK-15966][DOC] Add closing tag to fix rendering issue for Spark monitoring ## What changes were proposed in this pull request? Adds the missing closing tag for spark.ui.view.acls.groups ## How was this patch tested? I built the docs locally and verified the changed in browser. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) **Before:** ![image](https://cloud.githubusercontent.com/assets/7732317/16135005/49fc0724-33e6-11e6-9390-98711593fa5b.png) **After:** ![image](https://cloud.githubusercontent.com/assets/7732317/16135021/62b5c4a8-33e6-11e6-8118-b22fda5c66eb.png) Author: Dhruve Ashar Closes #13719 from dhruve/doc/SPARK-15966. --- docs/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index 78a3470abf40f..fa6c899a40b68 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -157,7 +157,7 @@ The history server can be configured as follows: If enabled, access control checks are made regardless of what the individual application had set for spark.ui.acls.enable when the application was run. The application owner will always have authorization to view their own application and any users specified via - spark.ui.view.acls and groups specified via spark.ui.view.acls.groups + spark.ui.view.acls and groups specified via spark.ui.view.acls.groups when the application was run will also have authorization to view that application. If disabled, no access control checks are made.
    @@ -91,17 +86,17 @@ The following options can be set in `sparkEnvir` with `sparkR.init` from RStudio -## Creating DataFrames -With a `SQLContext`, applications can create `DataFrame`s from a local R data frame, from a [Hive table](sql-programming-guide.html#hive-tables), or from other [data sources](sql-programming-guide.html#data-sources). +## Creating SparkDataFrames +With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-programming-guide.html#hive-tables), or from other [data sources](sql-programming-guide.html#data-sources). ### From local data frames -The simplest way to create a data frame is to convert a local R data frame into a SparkR DataFrame. Specifically we can use `createDataFrame` and pass in the local R data frame to create a SparkR DataFrame. As an example, the following creates a `DataFrame` based using the `faithful` dataset from R. +The simplest way to create a data frame is to convert a local R data frame into a SparkDataFrame. Specifically we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a SparkDataFrame. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R.
    {% highlight r %} -df <- createDataFrame(sqlContext, faithful) +df <- as.DataFrame(faithful) -# Displays the content of the DataFrame to stdout +# Displays the first part of the SparkDataFrame head(df) ## eruptions waiting ##1 3.600 79 @@ -113,16 +108,15 @@ head(df) ### From Data Sources -SparkR supports operating on a variety of data sources through the `DataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. -The general method for creating DataFrames from data sources is `read.df`. This method takes in the `SQLContext`, the path for the file to load and the type of data source. SparkR supports reading JSON, CSV and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by +The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. SparkR supports reading JSON, CSV and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by specifying `--packages` with `spark-submit` or `sparkR` commands, or if creating context through `init` you can specify the packages with the `packages` argument.
    {% highlight r %} -sc <- sparkR.init(sparkPackages="com.databricks:spark-avro_2.11:2.0.1") -sqlContext <- sparkRSQL.init(sc) +sc <- sparkR.session(sparkPackages="com.databricks:spark-avro_2.11:3.0.0") {% endhighlight %}
    @@ -131,7 +125,7 @@ We can see how to use data sources using an example JSON input file. Note that t
    {% highlight r %} -people <- read.df(sqlContext, "./examples/src/main/resources/people.json", "json") +people <- read.df("./examples/src/main/resources/people.json", "json") head(people) ## age name ##1 NA Michael @@ -147,8 +141,8 @@ printSchema(people) {% endhighlight %}
    -The data sources API can also be used to save out DataFrames into multiple file formats. For example we can save the DataFrame from the previous example -to a Parquet file using `write.df` (Until Spark 1.6, the default mode for writes was `append`. It was changed in Spark 1.7 to `error` to match the Scala API) +The data sources API can also be used to save out SparkDataFrames into multiple file formats. For example we can save the SparkDataFrame from the previous example +to a Parquet file using `write.df`.
    {% highlight r %} @@ -158,20 +152,19 @@ write.df(people, path="people.parquet", source="parquet", mode="overwrite") ### From Hive tables -You can also create SparkR DataFrames from Hive tables. To do this we will need to create a HiveContext which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details on the difference between SQLContext and HiveContext can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sqlcontext). +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sqlcontext). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`).
    {% highlight r %} -# sc is an existing SparkContext. -hiveContext <- sparkRHive.init(sc) +sparkR.session() -sql(hiveContext, "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sql(hiveContext, "LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results <- sql(hiveContext, "FROM src SELECT key, value") +results <- sql("FROM src SELECT key, value") -# results is now a DataFrame +# results is now a SparkDataFrame head(results) ## key value ## 1 238 val_238 @@ -181,19 +174,19 @@ head(results) {% endhighlight %}
    -## DataFrame Operations +## SparkDataFrame Operations -SparkR DataFrames support a number of functions to do structured data processing. +SparkDataFrames support a number of functions to do structured data processing. Here we include some basic examples and a complete list can be found in the [API](api/R/index.html) docs: ### Selecting rows, columns
    {% highlight r %} -# Create the DataFrame -df <- createDataFrame(sqlContext, faithful) +# Create the SparkDataFrame +df <- as.DataFrame(faithful) -# Get basic information about the DataFrame +# Get basic information about the SparkDataFrame df ## SparkDataFrame[eruptions:double, waiting:double] @@ -207,7 +200,7 @@ head(select(df, df$eruptions)) # You can also pass in column name as strings head(select(df, "eruptions")) -# Filter the DataFrame to only retain rows with wait times shorter than 50 mins +# Filter the SparkDataFrame to only retain rows with wait times shorter than 50 mins head(filter(df, df$waiting < 50)) ## eruptions waiting ##1 1.750 47 @@ -251,7 +244,7 @@ SparkR also provides a number of functions that can directly applied to columns {% highlight r %} # Convert waiting time from hours to seconds. -# Note that we can assign this to a new column in the same DataFrame +# Note that we can assign this to a new column in the same SparkDataFrame df$waiting_secs <- df$waiting * 60 head(df) ## eruptions waiting waiting_secs @@ -263,19 +256,19 @@ head(df)
    ## Running SQL Queries from SparkR -A SparkR DataFrame can also be registered as a temporary table in Spark SQL and registering a DataFrame as a table allows you to run SQL queries over its data. -The `sql` function enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. +A SparkDataFrame can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`.
    {% highlight r %} # Load a JSON file -people <- read.df(sqlContext, "./examples/src/main/resources/people.json", "json") +people <- read.df("./examples/src/main/resources/people.json", "json") -# Register this DataFrame as a table. -registerTempTable(people, "people") +# Register this SparkDataFrame as a temporary view. +createOrReplaceTempView(people, "people") # SQL statements can be run by using the sql method -teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") head(teenagers) ## name ##1 Justin @@ -353,4 +346,8 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma - The method `table` has been removed and replaced by `tableToDF`. - The class `DataFrame` has been renamed to `SparkDataFrame` to avoid name conflicts. - - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable` + - Spark's `SQLContext` and `HiveContext` have been deprecated to be replaced by `SparkSession`. Instead of `sparkR.init()`, call `sparkR.session()` in its place to instantiate the SparkSession. Once that is done, that currently active SparkSession will be used for SparkDataFrame operations. + - The parameter `sparkExecutorEnv` is not supported by `sparkR.session`. To set environment for the executors, set Spark config properties with the prefix "spark.executorEnv.VAR_NAME", for example, "spark.executorEnv.PATH" + - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable`. + - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. + - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. diff --git a/examples/src/main/r/data-manipulation.R b/examples/src/main/r/data-manipulation.R index badb98bc78fd1..371335a62e92d 100644 --- a/examples/src/main/r/data-manipulation.R +++ b/examples/src/main/r/data-manipulation.R @@ -17,7 +17,7 @@ # For this example, we shall use the "flights" dataset # The dataset consists of every flight departing Houston in 2011. -# The data set is made up of 227,496 rows x 14 columns. +# The data set is made up of 227,496 rows x 14 columns. # To run this example use # ./bin/spark-submit examples/src/main/r/data-manipulation.R @@ -33,11 +33,8 @@ if (length(args) != 1) { q("no") } -## Initialize SparkContext -sc <- sparkR.init(appName = "SparkR-data-manipulation-example") - -## Initialize SQLContext -sqlContext <- sparkRSQL.init(sc) +## Initialize SparkSession +sparkR.session(appName = "SparkR-data-manipulation-example") flightsCsvPath <- args[[1]] @@ -46,7 +43,7 @@ flights_df <- read.csv(flightsCsvPath, header = TRUE) flights_df$date <- as.Date(flights_df$date) ## Filter flights whose destination is San Francisco and write to a local data frame -SFO_df <- flights_df[flights_df$dest == "SFO", ] +SFO_df <- flights_df[flights_df$dest == "SFO", ] # Convert the local data frame into a SparkDataFrame SFO_DF <- createDataFrame(SFO_df) @@ -102,5 +99,5 @@ if("magrittr" %in% rownames(installed.packages())) { head(dailyDelayDF) } -# Stop the SparkContext now -sparkR.stop() +# Stop the SparkSession now +sparkR.session.stop() diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R index 04347057178b1..a377d6e864d2b 100644 --- a/examples/src/main/r/dataframe.R +++ b/examples/src/main/r/dataframe.R @@ -17,9 +17,8 @@ library(SparkR) -# Initialize SparkContext and SQLContext -sc <- sparkR.init(appName="SparkR-DataFrame-example") -sqlContext <- sparkRSQL.init(sc) +# Initialize SparkSession +sc <- sparkR.session(appName="SparkR-DataFrame-example") # Create a simple local data.frame localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) @@ -44,14 +43,14 @@ printSchema(peopleDF) # Register this DataFrame as a table. createOrReplaceTempView(peopleDF, "people") -# SQL statements can be run by using the sql methods provided by sqlContext +# SQL statements can be run by using the sql methods teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Call collect to get a local data.frame teenagersLocalDF <- collect(teenagers) -# Print the teenagers in our dataset +# Print the teenagers in our dataset print(teenagersLocalDF) -# Stop the SparkContext now -sparkR.stop() +# Stop the SparkSession now +sparkR.session.stop() diff --git a/examples/src/main/r/ml.R b/examples/src/main/r/ml.R index 495f392c26542..940c98dcb97a1 100644 --- a/examples/src/main/r/ml.R +++ b/examples/src/main/r/ml.R @@ -21,14 +21,13 @@ # Load SparkR library into your R session library(SparkR) -# Initialize SparkContext and SQLContext -sc <- sparkR.init(appName="SparkR-ML-example") -sqlContext <- sparkRSQL.init(sc) +# Initialize SparkSession +sparkR.session(appName="SparkR-ML-example") # $example on$ ############################ spark.glm and glm ############################################## -irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +irisDF <- suppressWarnings(createDataFrame(iris)) # Fit a generalized linear model of family "gaussian" with spark.glm gaussianDF <- irisDF gaussianTestDF <- irisDF @@ -62,7 +61,7 @@ showDF(binomialPredictions) library(survival) # Fit an accelerated failure time (AFT) survival regression model with spark.survreg -ovarianDF <- suppressWarnings(createDataFrame(sqlContext, ovarian)) +ovarianDF <- suppressWarnings(createDataFrame(ovarian)) aftDF <- ovarianDF aftTestDF <- ovarianDF aftModel <- spark.survreg(aftDF, Surv(futime, fustat) ~ ecog_ps + rx) @@ -78,7 +77,7 @@ showDF(aftPredictions) # Fit a Bernoulli naive Bayes model with spark.naiveBayes titanic <- as.data.frame(Titanic) -titanicDF <- suppressWarnings(createDataFrame(sqlContext, titanic[titanic$Freq > 0, -5])) +titanicDF <- createDataFrame(titanic[titanic$Freq > 0, -5]) nbDF <- titanicDF nbTestDF <- titanicDF nbModel <- spark.naiveBayes(nbDF, Survived ~ Class + Sex + Age) @@ -93,7 +92,7 @@ showDF(nbPredictions) ############################ spark.kmeans ############################################## # Fit a k-means model with spark.kmeans -irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +irisDF <- suppressWarnings(createDataFrame(iris)) kmeansDF <- irisDF kmeansTestDF <- irisDF kmeansModel <- spark.kmeans(kmeansDF, ~ Sepal_Length + Sepal_Width + Petal_Length + Petal_Width, @@ -111,7 +110,7 @@ showDF(kmeansPredictions) ############################ model read/write ############################################## -irisDF <- suppressWarnings(createDataFrame(sqlContext, iris)) +irisDF <- suppressWarnings(createDataFrame(iris)) # Fit a generalized linear model of family "gaussian" with spark.glm gaussianDF <- irisDF gaussianTestDF <- irisDF @@ -139,11 +138,11 @@ train <- function(family) { model <- glm(Sepal.Length ~ Sepal.Width + Species, iris, family = family) summary(model) } -model.summaries <- spark.lapply(sc, families, train) +model.summaries <- spark.lapply(families, train) # Print the summary of each model print(model.summaries) -# Stop the SparkContext now -sparkR.stop() +# Stop the SparkSession now +sparkR.session.stop() From 54aef1c1414589b5143ec3cbbf3b1e17648b7067 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Jun 2016 14:24:41 -0700 Subject: [PATCH 0731/1470] [SPARK-14995][R] Add `since` tag in Roxygen documentation for SparkR API methods ## What changes were proposed in this pull request? This PR adds `since` tags to Roxygen documentation according to the previous documentation archive. https://home.apache.org/~dongjoon/spark-2.0.0-docs/api/R/ ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #13734 from dongjoon-hyun/SPARK-14995. (cherry picked from commit d0eddb80eca04e4f5f8af3b5143096cf67200277) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 93 +++++++++++++++++++++++++- R/pkg/R/SQLContext.R | 42 ++++++------ R/pkg/R/WindowSpec.R | 8 +++ R/pkg/R/column.R | 10 +++ R/pkg/R/context.R | 3 +- R/pkg/R/functions.R | 153 +++++++++++++++++++++++++++++++++++++++++++ R/pkg/R/group.R | 6 ++ R/pkg/R/jobj.R | 1 + R/pkg/R/mllib.R | 24 +++++++ R/pkg/R/schema.R | 5 +- R/pkg/R/sparkR.R | 18 ++--- R/pkg/R/stats.R | 6 ++ R/pkg/R/utils.R | 1 + R/pkg/R/window.R | 4 ++ 14 files changed, 340 insertions(+), 34 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 583d3aefd4e6b..ecdcd6e29f190 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -25,7 +25,7 @@ setOldClass("structType") #' S4 class that represents a SparkDataFrame #' -#' DataFrames can be created using functions like \link{createDataFrame}, +#' SparkDataFrames can be created using functions like \link{createDataFrame}, #' \link{read.json}, \link{table} etc. #' #' @family SparkDataFrame functions @@ -42,6 +42,7 @@ setOldClass("structType") #' sparkR.session() #' df <- createDataFrame(faithful) #'} +#' @note SparkDataFrame since 2.0.0 setClass("SparkDataFrame", slots = list(env = "environment", sdf = "jobj")) @@ -81,6 +82,7 @@ dataFrame <- function(sdf, isCached = FALSE) { #' df <- read.json(path) #' printSchema(df) #'} +#' @note printSchema since 1.4.0 setMethod("printSchema", signature(x = "SparkDataFrame"), function(x) { @@ -105,6 +107,7 @@ setMethod("printSchema", #' df <- read.json(path) #' dfSchema <- schema(df) #'} +#' @note schema since 1.4.0 setMethod("schema", signature(x = "SparkDataFrame"), function(x) { @@ -128,6 +131,7 @@ setMethod("schema", #' df <- read.json(path) #' explain(df, TRUE) #'} +#' @note explain since 1.4.0 setMethod("explain", signature(x = "SparkDataFrame"), function(x, extended = FALSE) { @@ -158,6 +162,7 @@ setMethod("explain", #' df <- read.json(path) #' isLocal(df) #'} +#' @note isLocal since 1.4.0 setMethod("isLocal", signature(x = "SparkDataFrame"), function(x) { @@ -182,6 +187,7 @@ setMethod("isLocal", #' df <- read.json(path) #' showDF(df) #'} +#' @note showDF since 1.4.0 setMethod("showDF", signature(x = "SparkDataFrame"), function(x, numRows = 20, truncate = TRUE) { @@ -206,6 +212,7 @@ setMethod("showDF", #' df <- read.json(path) #' df #'} +#' @note show(SparkDataFrame) since 1.4.0 setMethod("show", "SparkDataFrame", function(object) { cols <- lapply(dtypes(object), function(l) { @@ -232,6 +239,7 @@ setMethod("show", "SparkDataFrame", #' df <- read.json(path) #' dtypes(df) #'} +#' @note dtypes since 1.4.0 setMethod("dtypes", signature(x = "SparkDataFrame"), function(x) { @@ -259,6 +267,7 @@ setMethod("dtypes", #' columns(df) #' colnames(df) #'} +#' @note columns since 1.4.0 setMethod("columns", signature(x = "SparkDataFrame"), function(x) { @@ -269,6 +278,7 @@ setMethod("columns", #' @rdname columns #' @name names +#' @note names since 1.5.0 setMethod("names", signature(x = "SparkDataFrame"), function(x) { @@ -277,6 +287,7 @@ setMethod("names", #' @rdname columns #' @name names<- +#' @note names<- since 1.5.0 setMethod("names<-", signature(x = "SparkDataFrame"), function(x, value) { @@ -288,6 +299,7 @@ setMethod("names<-", #' @rdname columns #' @name colnames +#' @note colnames since 1.6.0 setMethod("colnames", signature(x = "SparkDataFrame"), function(x) { @@ -296,6 +308,7 @@ setMethod("colnames", #' @rdname columns #' @name colnames<- +#' @note colnames<- since 1.6.0 setMethod("colnames<-", signature(x = "SparkDataFrame"), function(x, value) { @@ -338,6 +351,7 @@ setMethod("colnames<-", #' irisDF <- createDataFrame(iris) #' coltypes(irisDF) #'} +#' @note coltypes since 1.6.0 setMethod("coltypes", signature(x = "SparkDataFrame"), function(x) { @@ -393,6 +407,7 @@ setMethod("coltypes", #' coltypes(df) <- c("character", "integer") #' coltypes(df) <- c(NA, "numeric") #'} +#' @note coltypes<- since 1.6.0 setMethod("coltypes<-", signature(x = "SparkDataFrame", value = "character"), function(x, value) { @@ -440,6 +455,7 @@ setMethod("coltypes<-", #' createOrReplaceTempView(df, "json_df") #' new_df <- sql("SELECT * FROM json_df") #'} +#' @note createOrReplaceTempView since 2.0.0 setMethod("createOrReplaceTempView", signature(x = "SparkDataFrame", viewName = "character"), function(x, viewName) { @@ -464,6 +480,7 @@ setMethod("createOrReplaceTempView", #' registerTempTable(df, "json_df") #' new_df <- sql("SELECT * FROM json_df") #'} +#' @note registerTempTable since 1.4.0 setMethod("registerTempTable", signature(x = "SparkDataFrame", tableName = "character"), function(x, tableName) { @@ -492,6 +509,7 @@ setMethod("registerTempTable", #' createOrReplaceTempView(df, "table1") #' insertInto(df2, "table1", overwrite = TRUE) #'} +#' @note insertInto since 1.4.0 setMethod("insertInto", signature(x = "SparkDataFrame", tableName = "character"), function(x, tableName, overwrite = FALSE) { @@ -518,6 +536,7 @@ setMethod("insertInto", #' df <- read.json(path) #' cache(df) #'} +#' @note cache since 1.4.0 setMethod("cache", signature(x = "SparkDataFrame"), function(x) { @@ -545,6 +564,7 @@ setMethod("cache", #' df <- read.json(path) #' persist(df, "MEMORY_AND_DISK") #'} +#' @note persist since 1.4.0 setMethod("persist", signature(x = "SparkDataFrame", newLevel = "character"), function(x, newLevel) { @@ -573,6 +593,7 @@ setMethod("persist", #' persist(df, "MEMORY_AND_DISK") #' unpersist(df) #'} +#' @note unpersist since 1.4.0 setMethod("unpersist", signature(x = "SparkDataFrame"), function(x, blocking = TRUE) { @@ -609,6 +630,7 @@ setMethod("unpersist", #' newDF <- repartition(df, col = df$"col1", df$"col2") #' newDF <- repartition(df, 3L, col = df$"col1", df$"col2") #'} +#' @note repartition since 1.4.0 setMethod("repartition", signature(x = "SparkDataFrame"), function(x, numPartitions = NULL, col = NULL, ...) { @@ -675,6 +697,7 @@ setMethod("toJSON", #' df <- read.json(path) #' write.json(df, "/tmp/sparkr-tmp/") #'} +#' @note write.json since 1.6.0 setMethod("write.json", signature(x = "SparkDataFrame", path = "character"), function(x, path) { @@ -729,6 +752,7 @@ setMethod("write.orc", #' write.parquet(df, "/tmp/sparkr-tmp1/") #' saveAsParquetFile(df, "/tmp/sparkr-tmp2/") #'} +#' @note write.parquet since 1.6.0 setMethod("write.parquet", signature(x = "SparkDataFrame", path = "character"), function(x, path) { @@ -739,6 +763,7 @@ setMethod("write.parquet", #' @rdname write.parquet #' @name saveAsParquetFile #' @export +#' @note saveAsParquetFile since 1.4.0 setMethod("saveAsParquetFile", signature(x = "SparkDataFrame", path = "character"), function(x, path) { @@ -766,6 +791,7 @@ setMethod("saveAsParquetFile", #' df <- read.text(path) #' write.text(df, "/tmp/sparkr-tmp/") #'} +#' @note write.text since 2.0.0 setMethod("write.text", signature(x = "SparkDataFrame", path = "character"), function(x, path) { @@ -790,6 +816,7 @@ setMethod("write.text", #' df <- read.json(path) #' distinctDF <- distinct(df) #'} +#' @note distinct since 1.4.0 setMethod("distinct", signature(x = "SparkDataFrame"), function(x) { @@ -799,6 +826,7 @@ setMethod("distinct", #' @rdname distinct #' @name unique +#' @note unique since 1.5.0 setMethod("unique", signature(x = "SparkDataFrame"), function(x) { @@ -826,6 +854,7 @@ setMethod("unique", #' collect(sample(df, FALSE, 0.5)) #' collect(sample(df, TRUE, 0.5)) #'} +#' @note sample since 1.4.0 setMethod("sample", signature(x = "SparkDataFrame", withReplacement = "logical", fraction = "numeric"), @@ -843,6 +872,7 @@ setMethod("sample", #' @rdname sample #' @name sample_frac +#' @note sample_frac since 1.4.0 setMethod("sample_frac", signature(x = "SparkDataFrame", withReplacement = "logical", fraction = "numeric"), @@ -865,6 +895,7 @@ setMethod("sample_frac", #' df <- read.json(path) #' count(df) #' } +#' @note count since 1.4.0 setMethod("count", signature(x = "SparkDataFrame"), function(x) { @@ -873,6 +904,7 @@ setMethod("count", #' @name nrow #' @rdname nrow +#' @note nrow since 1.5.0 setMethod("nrow", signature(x = "SparkDataFrame"), function(x) { @@ -894,6 +926,7 @@ setMethod("nrow", #' df <- read.json(path) #' ncol(df) #' } +#' @note ncol since 1.5.0 setMethod("ncol", signature(x = "SparkDataFrame"), function(x) { @@ -916,6 +949,7 @@ setMethod("ncol", #' df <- read.json(path) #' dim(df) #' } +#' @note dim since 1.5.0 setMethod("dim", signature(x = "SparkDataFrame"), function(x) { @@ -940,6 +974,7 @@ setMethod("dim", #' collected <- collect(df) #' firstName <- collected[[1]]$name #' } +#' @note collect since 1.4.0 setMethod("collect", signature(x = "SparkDataFrame"), function(x, stringsAsFactors = FALSE) { @@ -1010,6 +1045,7 @@ setMethod("collect", #' df <- read.json(path) #' limitedDF <- limit(df, 10) #' } +#' @note limit since 1.4.0 setMethod("limit", signature(x = "SparkDataFrame", num = "numeric"), function(x, num) { @@ -1030,6 +1066,7 @@ setMethod("limit", #' df <- read.json(path) #' take(df, 2) #' } +#' @note take since 1.4.0 setMethod("take", signature(x = "SparkDataFrame", num = "numeric"), function(x, num) { @@ -1058,6 +1095,7 @@ setMethod("take", #' df <- read.json(path) #' head(df) #' } +#' @note head since 1.4.0 setMethod("head", signature(x = "SparkDataFrame"), function(x, num = 6L) { @@ -1080,6 +1118,7 @@ setMethod("head", #' df <- read.json(path) #' first(df) #' } +#' @note first(SparkDataFrame) since 1.4.0 setMethod("first", signature(x = "SparkDataFrame"), function(x) { @@ -1131,6 +1170,7 @@ setMethod("toRDD", #' # Compute the max age and average salary, grouped by department and gender. #' agg(groupBy(df, "department", "gender"), salary="avg", "age" -> "max") #' } +#' @note groupBy since 1.4.0 setMethod("groupBy", signature(x = "SparkDataFrame"), function(x, ...) { @@ -1146,6 +1186,7 @@ setMethod("groupBy", #' @rdname groupBy #' @name group_by +#' @note group_by since 1.4.0 setMethod("group_by", signature(x = "SparkDataFrame"), function(x, ...) { @@ -1161,6 +1202,7 @@ setMethod("group_by", #' @rdname agg #' @name agg #' @export +#' @note agg since 1.4.0 setMethod("agg", signature(x = "SparkDataFrame"), function(x, ...) { @@ -1169,6 +1211,7 @@ setMethod("agg", #' @rdname agg #' @name summarize +#' @note summarize since 1.4.0 setMethod("summarize", signature(x = "SparkDataFrame"), function(x, ...) { @@ -1233,6 +1276,7 @@ dapplyInternal <- function(x, func, schema) { #' # 1 2 2 2 3 #' # 2 3 3 3 4 #' } +#' @note dapply since 2.0.0 setMethod("dapply", signature(x = "SparkDataFrame", func = "function", schema = "structType"), function(x, func, schema) { @@ -1273,6 +1317,7 @@ setMethod("dapply", #' # 2 2 2 3 #' # 3 3 3 4 #' } +#' @note dapplyCollect since 2.0.0 setMethod("dapplyCollect", signature(x = "SparkDataFrame", func = "function"), function(x, func) { @@ -1361,6 +1406,7 @@ setMethod("dapplyCollect", #' 3 2.351890 0.6548350 0.2375602 0.2521257 #' #'} +#' @note gapply(SparkDataFrame) since 2.0.0 setMethod("gapply", signature(x = "SparkDataFrame"), function(x, cols, func, schema) { @@ -1444,6 +1490,7 @@ getColumn <- function(x, c) { #' @rdname select #' @name $ +#' @note $ since 1.4.0 setMethod("$", signature(x = "SparkDataFrame"), function(x, name) { getColumn(x, name) @@ -1451,6 +1498,7 @@ setMethod("$", signature(x = "SparkDataFrame"), #' @rdname select #' @name $<- +#' @note $<- since 1.4.0 setMethod("$<-", signature(x = "SparkDataFrame"), function(x, name, value) { stopifnot(class(value) == "Column" || is.null(value)) @@ -1468,6 +1516,7 @@ setClassUnion("numericOrcharacter", c("numeric", "character")) #' @rdname subset #' @name [[ +#' @note [[ since 1.4.0 setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"), function(x, i) { if (is.numeric(i)) { @@ -1479,6 +1528,7 @@ setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"), #' @rdname subset #' @name [ +#' @note [ since 1.4.0 setMethod("[", signature(x = "SparkDataFrame"), function(x, i, j, ..., drop = F) { # Perform filtering first if needed @@ -1543,6 +1593,7 @@ setMethod("[", signature(x = "SparkDataFrame"), #' subset(df, df$age %in% c(19), select = c(1,2)) #' subset(df, select = c(1,2)) #' } +#' @note subset since 1.5.0 setMethod("subset", signature(x = "SparkDataFrame"), function(x, subset, select, drop = F, ...) { if (missing(subset)) { @@ -1573,6 +1624,7 @@ setMethod("subset", signature(x = "SparkDataFrame"), #' # Similar to R data frames columns can also be selected using `$` #' df[,df$age] #' } +#' @note select(SparkDataFrame, character) since 1.4.0 setMethod("select", signature(x = "SparkDataFrame", col = "character"), function(x, col, ...) { if (length(col) > 1) { @@ -1590,6 +1642,7 @@ setMethod("select", signature(x = "SparkDataFrame", col = "character"), #' @family SparkDataFrame functions #' @rdname select #' @export +#' @note select(SparkDataFrame, Column) since 1.4.0 setMethod("select", signature(x = "SparkDataFrame", col = "Column"), function(x, col, ...) { jcols <- lapply(list(col, ...), function(c) { @@ -1602,6 +1655,7 @@ setMethod("select", signature(x = "SparkDataFrame", col = "Column"), #' @family SparkDataFrame functions #' @rdname select #' @export +#' @note select(SparkDataFrame, list) since 1.4.0 setMethod("select", signature(x = "SparkDataFrame", col = "list"), function(x, col) { @@ -1635,6 +1689,7 @@ setMethod("select", #' df <- read.json(path) #' selectExpr(df, "col1", "(col2 * 5) as newCol") #' } +#' @note selectExpr since 1.4.0 setMethod("selectExpr", signature(x = "SparkDataFrame", expr = "character"), function(x, expr, ...) { @@ -1666,6 +1721,7 @@ setMethod("selectExpr", #' # Replace an existing column #' newDF2 <- withColumn(newDF, "newCol", newDF$col1) #' } +#' @note withColumn since 1.4.0 setMethod("withColumn", signature(x = "SparkDataFrame", colName = "character", col = "Column"), function(x, colName, col) { @@ -1698,6 +1754,7 @@ setMethod("withColumn", #' # Replace the "age" column #' df1 <- mutate(df, age = df$age + 1L) #' } +#' @note mutate since 1.4.0 setMethod("mutate", signature(.data = "SparkDataFrame"), function(.data, ...) { @@ -1756,6 +1813,7 @@ setMethod("mutate", #' @export #' @rdname mutate #' @name transform +#' @note transform since 1.5.0 setMethod("transform", signature(`_data` = "SparkDataFrame"), function(`_data`, ...) { @@ -1782,6 +1840,7 @@ setMethod("transform", #' df <- read.json(path) #' newDF <- withColumnRenamed(df, "col1", "newCol1") #' } +#' @note withColumnRenamed since 1.4.0 setMethod("withColumnRenamed", signature(x = "SparkDataFrame", existingCol = "character", newCol = "character"), function(x, existingCol, newCol) { @@ -1806,6 +1865,7 @@ setMethod("withColumnRenamed", #' df <- read.json(path) #' newDF <- rename(df, col1 = df$newCol1) #' } +#' @note rename since 1.4.0 setMethod("rename", signature(x = "SparkDataFrame"), function(x, ...) { @@ -1852,6 +1912,7 @@ setClassUnion("characterOrColumn", c("character", "Column")) #' arrange(df, "col1", decreasing = TRUE) #' arrange(df, "col1", "col2", decreasing = c(TRUE, FALSE)) #' } +#' @note arrange(SparkDataFrame, Column) since 1.4.0 setMethod("arrange", signature(x = "SparkDataFrame", col = "Column"), function(x, col, ...) { @@ -1866,6 +1927,7 @@ setMethod("arrange", #' @rdname arrange #' @name arrange #' @export +#' @note arrange(SparkDataFrame, character) since 1.4.0 setMethod("arrange", signature(x = "SparkDataFrame", col = "character"), function(x, col, ..., decreasing = FALSE) { @@ -1898,6 +1960,7 @@ setMethod("arrange", #' @rdname arrange #' @name orderBy #' @export +#' @note orderBy(SparkDataFrame, characterOrColumn) since 1.4.0 setMethod("orderBy", signature(x = "SparkDataFrame", col = "characterOrColumn"), function(x, col, ...) { @@ -1925,6 +1988,7 @@ setMethod("orderBy", #' filter(df, "col1 > 0") #' filter(df, df$col2 != "abcdefg") #' } +#' @note filter since 1.4.0 setMethod("filter", signature(x = "SparkDataFrame", condition = "characterOrColumn"), function(x, condition) { @@ -1938,6 +2002,7 @@ setMethod("filter", #' @family SparkDataFrame functions #' @rdname filter #' @name where +#' @note where since 1.4.0 setMethod("where", signature(x = "SparkDataFrame", condition = "characterOrColumn"), function(x, condition) { @@ -1966,6 +2031,7 @@ setMethod("where", #' dropDuplicates(df, "col1", "col2") #' dropDuplicates(df, c("col1", "col2")) #' } +#' @note dropDuplicates since 2.0.0 setMethod("dropDuplicates", signature(x = "SparkDataFrame"), function(x, ...) { @@ -2012,6 +2078,7 @@ setMethod("dropDuplicates", #' join(df1, df2, df1$col1 == df2$col2) # Performs an inner join based on expression #' join(df1, df2, df1$col1 == df2$col2, "right_outer") #' } +#' @note join since 1.4.0 setMethod("join", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y, joinExpr = NULL, joinType = NULL) { @@ -2073,6 +2140,7 @@ setMethod("join", #' merge(df1, df2, by.x = "col1", by.y = "col2", all = TRUE, sort = FALSE) #' merge(df1, df2, by = "col1", all = TRUE, suffixes = c("-X", "-Y")) #' } +#' @note merge since 1.5.0 setMethod("merge", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y, by = intersect(names(x), names(y)), by.x = by, by.y = by, @@ -2160,6 +2228,7 @@ setMethod("merge", #' @param suffix a suffix for the column name #' @return list of columns #' +#' @note generateAliasesForIntersectedCols since 1.6.0 generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { allColNames <- names(x) # sets alias for making colnames unique in dataframe 'x' @@ -2198,6 +2267,7 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { #' df2 <- read.json(path2) #' unioned <- unionAll(df, df2) #' } +#' @note unionAll since 1.4.0 setMethod("unionAll", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { @@ -2212,6 +2282,7 @@ setMethod("unionAll", #' @rdname rbind #' @name rbind #' @export +#' @note rbind since 1.5.0 setMethod("rbind", signature(... = "SparkDataFrame"), function(x, ..., deparse.level = 1) { @@ -2241,6 +2312,7 @@ setMethod("rbind", #' df2 <- read.json(path2) #' intersectDF <- intersect(df, df2) #' } +#' @note intersect since 1.4.0 setMethod("intersect", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { @@ -2269,6 +2341,7 @@ setMethod("intersect", #' } #' @rdname except #' @export +#' @note except since 1.4.0 setMethod("except", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { @@ -2310,6 +2383,7 @@ setMethod("except", #' write.df(df, "myfile", "parquet", "overwrite") #' saveDF(df, parquetPath2, "parquet", mode = saveMode, mergeSchema = mergeSchema) #' } +#' @note write.df since 1.4.0 setMethod("write.df", signature(df = "SparkDataFrame", path = "character"), function(df, path, source = NULL, mode = "error", ...){ @@ -2330,6 +2404,7 @@ setMethod("write.df", #' @rdname write.df #' @name saveDF #' @export +#' @note saveDF since 1.4.0 setMethod("saveDF", signature(df = "SparkDataFrame", path = "character"), function(df, path, source = NULL, mode = "error", ...){ @@ -2367,6 +2442,7 @@ setMethod("saveDF", #' df <- read.json(path) #' saveAsTable(df, "myfile") #' } +#' @note saveAsTable since 1.4.0 setMethod("saveAsTable", signature(df = "SparkDataFrame", tableName = "character"), function(df, tableName, source = NULL, mode="error", ...){ @@ -2405,6 +2481,7 @@ setMethod("saveAsTable", #' describe(df, "col1") #' describe(df, "col1", "col2") #' } +#' @note describe(SparkDataFrame, character) since 1.4.0 setMethod("describe", signature(x = "SparkDataFrame", col = "character"), function(x, col, ...) { @@ -2415,6 +2492,7 @@ setMethod("describe", #' @rdname summary #' @name describe +#' @note describe(SparkDataFrame) since 1.4.0 setMethod("describe", signature(x = "SparkDataFrame"), function(x) { @@ -2425,6 +2503,7 @@ setMethod("describe", #' @rdname summary #' @name summary +#' @note summary(SparkDataFrame) since 1.5.0 setMethod("summary", signature(object = "SparkDataFrame"), function(object, ...) { @@ -2458,6 +2537,7 @@ setMethod("summary", #' df <- read.json(path) #' dropna(df) #' } +#' @note dropna since 1.4.0 setMethod("dropna", signature(x = "SparkDataFrame"), function(x, how = c("any", "all"), minNonNulls = NULL, cols = NULL) { @@ -2478,6 +2558,7 @@ setMethod("dropna", #' @rdname nafunctions #' @name na.omit #' @export +#' @note na.omit since 1.5.0 setMethod("na.omit", signature(object = "SparkDataFrame"), function(object, how = c("any", "all"), minNonNulls = NULL, cols = NULL) { @@ -2512,6 +2593,7 @@ setMethod("na.omit", #' fillna(df, 1) #' fillna(df, list("age" = 20, "name" = "unknown")) #' } +#' @note fillna since 1.4.0 setMethod("fillna", signature(x = "SparkDataFrame"), function(x, value, cols = NULL) { @@ -2571,6 +2653,7 @@ setMethod("fillna", #' irisDF <- createDataFrame(iris) #' df <- as.data.frame(irisDF[irisDF$Species == "setosa", ]) #' } +#' @note as.data.frame since 1.6.0 setMethod("as.data.frame", signature(x = "SparkDataFrame"), function(x, row.names = NULL, optional = FALSE, ...) { @@ -2597,6 +2680,7 @@ setMethod("as.data.frame", #' summary(Sepal_Width) #' } #' @seealso \link{detach} +#' @note attach since 1.6.0 setMethod("attach", signature(what = "SparkDataFrame"), function(what, pos = 2, name = deparse(substitute(what)), warn.conflicts = TRUE) { @@ -2622,6 +2706,7 @@ setMethod("attach", #' with(irisDf, nrow(Sepal_Width)) #' } #' @seealso \link{attach} +#' @note with since 1.6.0 setMethod("with", signature(data = "SparkDataFrame"), function(data, expr, ...) { @@ -2645,6 +2730,7 @@ setMethod("with", #' # Show the structure of the SparkDataFrame #' str(irisDF) #' } +#' @note str since 1.6.1 setMethod("str", signature(object = "SparkDataFrame"), function(object) { @@ -2726,6 +2812,7 @@ setMethod("str", #' drop(df, c("col1", "col2")) #' drop(df, df$col1) #' } +#' @note drop since 2.0.0 setMethod("drop", signature(x = "SparkDataFrame"), function(x, col) { @@ -2775,6 +2862,7 @@ setMethod("drop", #' geom_bar(stat = "identity") + #' xlab("Sepal_Length") + ylab("Frequency") #' } +#' @note histogram since 2.0.0 setMethod("histogram", signature(df = "SparkDataFrame", col = "characterOrColumn"), function(df, col, nbins = 10) { @@ -2896,6 +2984,7 @@ setMethod("histogram", #' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" #' write.jdbc(df, jdbcUrl, "table", user = "username", password = "password") #' } +#' @note write.jdbc since 2.0.0 setMethod("write.jdbc", signature(x = "SparkDataFrame", url = "character", tableName = "character"), function(x, url, tableName, mode = "error", ...){ @@ -2926,7 +3015,7 @@ setMethod("write.jdbc", #' # df_list contains 3 SparkDataFrames with each having about 200, 300 and 500 rows respectively #' sapply(df_list, count) #' } -#' @note since 2.0.0 +#' @note randomSplit since 2.0.0 setMethod("randomSplit", signature(x = "SparkDataFrame", weights = "numeric"), function(x, weights, seed) { diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index b846b3dec246c..8d2c4ac7cee91 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -67,6 +67,7 @@ dispatchFunc <- function(newFuncSig, x, ...) { } #' return the SparkSession +#' @note getSparkSession since 2.0.0 getSparkSession <- function() { if (exists(".sparkRsession", envir = .sparkREnv)) { get(".sparkRsession", envir = .sparkREnv) @@ -76,6 +77,7 @@ getSparkSession <- function() { } #' infer the SQL type +#' @note infer_type since 1.4.0 infer_type <- function(x) { if (is.null(x)) { stop("can not infer type from NULL") @@ -133,7 +135,7 @@ getDefaultSqlSource <- function() { #' } #' @name createDataFrame #' @method createDataFrame default - +#' @note createDataFrame since 1.4.0 # TODO(davies): support sampling and infer type from NA createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { sparkSession <- getSparkSession() @@ -218,7 +220,7 @@ createDataFrame <- function(x, ...) { #' @aliases createDataFrame #' @export #' @method as.DataFrame default - +#' @note as.DataFrame since 1.6.0 as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { createDataFrame(data, schema, samplingRatio) } @@ -266,7 +268,7 @@ setMethod("toDF", signature(x = "RDD"), #' } #' @name read.json #' @method read.json default - +#' @note read.json since 1.6.0 read.json.default <- function(path) { sparkSession <- getSparkSession() # Allow the user to have a more flexible definiton of the text file path @@ -284,7 +286,7 @@ read.json <- function(x, ...) { #' @name jsonFile #' @export #' @method jsonFile default - +#' @note jsonFile since 1.4.0 jsonFile.default <- function(path) { .Deprecated("read.json") read.json(path) @@ -355,7 +357,7 @@ read.orc <- function(path) { #' @export #' @name read.parquet #' @method read.parquet default - +#' @note read.parquet since 1.6.0 read.parquet.default <- function(path) { sparkSession <- getSparkSession() # Allow the user to have a more flexible definiton of the Parquet file path @@ -373,7 +375,7 @@ read.parquet <- function(x, ...) { #' @name parquetFile #' @export #' @method parquetFile default - +#' @note parquetFile since 1.4.0 parquetFile.default <- function(...) { .Deprecated("read.parquet") read.parquet(unlist(list(...))) @@ -403,7 +405,7 @@ parquetFile <- function(x, ...) { #' } #' @name read.text #' @method read.text default - +#' @note read.text since 1.6.1 read.text.default <- function(path) { sparkSession <- getSparkSession() # Allow the user to have a more flexible definiton of the text file path @@ -435,7 +437,7 @@ read.text <- function(x, ...) { #' } #' @name sql #' @method sql default - +#' @note sql since 1.4.0 sql.default <- function(sqlQuery) { sparkSession <- getSparkSession() sdf <- callJMethod(sparkSession, "sql", sqlQuery) @@ -464,8 +466,7 @@ sql <- function(x, ...) { #' createOrReplaceTempView(df, "table") #' new_df <- tableToDF("table") #' } -#' @note since 2.0.0 - +#' @note tableToDF since 2.0.0 tableToDF <- function(tableName) { sparkSession <- getSparkSession() sdf <- callJMethod(sparkSession, "table", tableName) @@ -487,7 +488,7 @@ tableToDF <- function(tableName) { #' } #' @name tables #' @method tables default - +#' @note tables since 1.4.0 tables.default <- function(databaseName = NULL) { sparkSession <- getSparkSession() jdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getTables", sparkSession, databaseName) @@ -513,7 +514,7 @@ tables <- function(x, ...) { #' } #' @name tableNames #' @method tableNames default - +#' @note tableNames since 1.4.0 tableNames.default <- function(databaseName = NULL) { sparkSession <- getSparkSession() callJStatic("org.apache.spark.sql.api.r.SQLUtils", @@ -544,7 +545,7 @@ tableNames <- function(x, ...) { #' } #' @name cacheTable #' @method cacheTable default - +#' @note cacheTable since 1.4.0 cacheTable.default <- function(tableName) { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") @@ -573,7 +574,7 @@ cacheTable <- function(x, ...) { #' } #' @name uncacheTable #' @method uncacheTable default - +#' @note uncacheTable since 1.4.0 uncacheTable.default <- function(tableName) { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") @@ -596,7 +597,7 @@ uncacheTable <- function(x, ...) { #' } #' @name clearCache #' @method clearCache default - +#' @note clearCache since 1.4.0 clearCache.default <- function() { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") @@ -625,7 +626,7 @@ clearCache <- function() { #' } #' @name dropTempTable #' @method dropTempTable default - +#' @note dropTempTable since 1.4.0 dropTempTable.default <- function(tableName) { if (class(tableName) != "character") { stop("tableName must be a string.") @@ -691,7 +692,7 @@ dropTempView <- function(viewName) { #' } #' @name read.df #' @method read.df default - +#' @note read.df since 1.4.0 read.df.default <- function(path = NULL, source = NULL, schema = NULL, ...) { sparkSession <- getSparkSession() options <- varargsToEnv(...) @@ -719,7 +720,7 @@ read.df <- function(x, ...) { #' @rdname read.df #' @name loadDF #' @method loadDF default - +#' @note loadDF since 1.6.0 loadDF.default <- function(path = NULL, source = NULL, schema = NULL, ...) { read.df(path, source, schema, ...) } @@ -750,7 +751,7 @@ loadDF <- function(x, ...) { #' } #' @name createExternalTable #' @method createExternalTable default - +#' @note createExternalTable since 1.4.0 createExternalTable.default <- function(tableName, path = NULL, source = NULL, ...) { sparkSession <- getSparkSession() options <- varargsToEnv(...) @@ -798,8 +799,7 @@ createExternalTable <- function(x, ...) { #' df2 <- read.jdbc(jdbcUrl, "table2", partitionColumn = "index", lowerBound = 0, #' upperBound = 10000, user = "username", password = "password") #' } -#' @note since 2.0.0 - +#' @note read.jdbc since 2.0.0 read.jdbc <- function(url, tableName, partitionColumn = NULL, lowerBound = NULL, upperBound = NULL, numPartitions = 0L, predicates = list(), ...) { diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index d8405420d0a49..9f3b1e4be5609 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -29,6 +29,7 @@ NULL #' #' @param sws A Java object reference to the backing Scala WindowSpec #' @export +#' @note WindowSpec since 2.0.0 setClass("WindowSpec", slots = list(sws = "jobj")) @@ -43,6 +44,7 @@ windowSpec <- function(sws) { } #' @rdname show +#' @note show(WindowSpec) since 2.0.0 setMethod("show", "WindowSpec", function(object) { cat("WindowSpec", callJMethod(object@sws, "toString"), "\n") @@ -63,6 +65,7 @@ setMethod("show", "WindowSpec", #' partitionBy(ws, "col1", "col2") #' partitionBy(ws, df$col1, df$col2) #' } +#' @note partitionBy(WindowSpec) since 2.0.0 setMethod("partitionBy", signature(x = "WindowSpec"), function(x, col, ...) { @@ -93,6 +96,7 @@ setMethod("partitionBy", #' orderBy(ws, "col1", "col2") #' orderBy(ws, df$col1, df$col2) #' } +#' @note orderBy(WindowSpec, character) since 2.0.0 setMethod("orderBy", signature(x = "WindowSpec", col = "character"), function(x, col, ...) { @@ -102,6 +106,7 @@ setMethod("orderBy", #' @rdname arrange #' @name orderBy #' @export +#' @note orderBy(WindowSpec, Column) since 2.0.0 setMethod("orderBy", signature(x = "WindowSpec", col = "Column"), function(x, col, ...) { @@ -133,6 +138,7 @@ setMethod("orderBy", #' \dontrun{ #' rowsBetween(ws, 0, 3) #' } +#' @note rowsBetween since 2.0.0 setMethod("rowsBetween", signature(x = "WindowSpec", start = "numeric", end = "numeric"), function(x, start, end) { @@ -163,6 +169,7 @@ setMethod("rowsBetween", #' \dontrun{ #' rangeBetween(ws, 0, 3) #' } +#' @note rangeBetween since 2.0.0 setMethod("rangeBetween", signature(x = "WindowSpec", start = "numeric", end = "numeric"), function(x, start, end) { @@ -182,6 +189,7 @@ setMethod("rangeBetween", #' @name over #' @family colum_func #' @export +#' @note over since 2.0.0 setMethod("over", signature(x = "Column", window = "WindowSpec"), function(x, window) { diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index cc2876ed94b7f..1af65d5d6e155 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -30,6 +30,7 @@ setOldClass("jobj") #' #' @slot jc reference to JVM SparkDataFrame column #' @export +#' @note Column since 1.4.0 setClass("Column", slots = list(jc = "jobj")) @@ -46,6 +47,7 @@ setMethod("column", #' @rdname show #' @name show +#' @note show(Column) since 1.4.0 setMethod("show", "Column", function(object) { cat("Column", callJMethod(object@jc, "toString"), "\n") @@ -127,6 +129,7 @@ createMethods() #' @name alias #' @family colum_func #' @export +#' @note alias since 1.4.0 setMethod("alias", signature(object = "Column"), function(object, data) { @@ -147,6 +150,7 @@ setMethod("alias", #' #' @param start starting position #' @param stop ending position +#' @note substr since 1.4.0 setMethod("substr", signature(x = "Column"), function(x, start, stop) { jc <- callJMethod(x@jc, "substr", as.integer(start - 1), as.integer(stop - start + 1)) @@ -164,6 +168,7 @@ setMethod("substr", signature(x = "Column"), #' #' @param x vector of character string whose “starts” are considered #' @param prefix character vector (often of length one) +#' @note startsWith since 1.4.0 setMethod("startsWith", signature(x = "Column"), function(x, prefix) { jc <- callJMethod(x@jc, "startsWith", as.vector(prefix)) @@ -181,6 +186,7 @@ setMethod("startsWith", signature(x = "Column"), #' #' @param x vector of character string whose “ends” are considered #' @param suffix character vector (often of length one) +#' @note endsWith since 1.4.0 setMethod("endsWith", signature(x = "Column"), function(x, suffix) { jc <- callJMethod(x@jc, "endsWith", as.vector(suffix)) @@ -196,6 +202,7 @@ setMethod("endsWith", signature(x = "Column"), #' @family colum_func #' #' @param bounds lower and upper bounds +#' @note between since 1.5.0 setMethod("between", signature(x = "Column"), function(x, bounds) { if (is.vector(bounds) && length(bounds) == 2) { @@ -216,6 +223,7 @@ setMethod("between", signature(x = "Column"), #' cast(df$age, "string") #' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE)) #' } +#' @note cast since 1.4.0 setMethod("cast", signature(x = "Column"), function(x, dataType) { @@ -242,6 +250,7 @@ setMethod("cast", #' filter(df, "age in (10, 30)") #' where(df, df$age %in% c(10, 30)) #' } +#' @note \%in\% since 1.5.0 setMethod("%in%", signature(x = "Column"), function(x, table) { @@ -258,6 +267,7 @@ setMethod("%in%", #' @name otherwise #' @family colum_func #' @export +#' @note otherwise since 1.5.0 setMethod("otherwise", signature(x = "Column", value = "ANY"), function(x, value) { diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 968a9d2251b18..42f89c806bfd9 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -261,6 +261,7 @@ setCheckpointDir <- function(sc, dirName) { #' sparkR.session() #' doubled <- spark.lapply(1:10, function(x){2 * x}) #'} +#' @note spark.lapply since 2.0.0 spark.lapply <- function(list, func) { if (!exists(".sparkRjsc", envir = .sparkREnv)) { stop("SparkR has not been initialized. Please call sparkR.session()") @@ -283,7 +284,7 @@ spark.lapply <- function(list, func) { #'\dontrun{ #' setLogLevel("ERROR") #'} - +#' @note setLogLevel since 2.0.0 setLogLevel <- function(level) { if (!exists(".sparkRjsc", envir = .sparkREnv)) { stop("SparkR has not been initialized. Please call sparkR.session()") diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index c26f9632589ff..ce2386998c1bc 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -33,6 +33,7 @@ NULL #' select(df, lit("x")) #' select(df, lit("2015-01-01")) #'} +#' @note lit since 1.5.0 setMethod("lit", signature("ANY"), function(x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -50,6 +51,7 @@ setMethod("lit", signature("ANY"), #' @family normal_funcs #' @export #' @examples \dontrun{abs(df$c)} +#' @note abs since 1.5.0 setMethod("abs", signature(x = "Column"), function(x) { @@ -67,6 +69,7 @@ setMethod("abs", #' @family math_funcs #' @export #' @examples \dontrun{acos(df$c)} +#' @note acos since 1.5.0 setMethod("acos", signature(x = "Column"), function(x) { @@ -83,6 +86,7 @@ setMethod("acos", #' @family agg_funcs #' @export #' @examples \dontrun{approxCountDistinct(df$c)} +#' @note approxCountDistinct(Column) since 1.4.0 setMethod("approxCountDistinct", signature(x = "Column"), function(x) { @@ -100,6 +104,7 @@ setMethod("approxCountDistinct", #' @family string_funcs #' @export #' @examples \dontrun{\dontrun{ascii(df$c)}} +#' @note ascii since 1.5.0 setMethod("ascii", signature(x = "Column"), function(x) { @@ -117,6 +122,7 @@ setMethod("ascii", #' @family math_funcs #' @export #' @examples \dontrun{asin(df$c)} +#' @note asin since 1.5.0 setMethod("asin", signature(x = "Column"), function(x) { @@ -133,6 +139,7 @@ setMethod("asin", #' @family math_funcs #' @export #' @examples \dontrun{atan(df$c)} +#' @note atan since 1.5.0 setMethod("atan", signature(x = "Column"), function(x) { @@ -149,6 +156,7 @@ setMethod("atan", #' @family agg_funcs #' @export #' @examples \dontrun{avg(df$c)} +#' @note avg since 1.4.0 setMethod("avg", signature(x = "Column"), function(x) { @@ -166,6 +174,7 @@ setMethod("avg", #' @family string_funcs #' @export #' @examples \dontrun{base64(df$c)} +#' @note base64 since 1.5.0 setMethod("base64", signature(x = "Column"), function(x) { @@ -183,6 +192,7 @@ setMethod("base64", #' @family math_funcs #' @export #' @examples \dontrun{bin(df$c)} +#' @note bin since 1.5.0 setMethod("bin", signature(x = "Column"), function(x) { @@ -199,6 +209,7 @@ setMethod("bin", #' @family normal_funcs #' @export #' @examples \dontrun{bitwiseNOT(df$c)} +#' @note bitwiseNOT since 1.5.0 setMethod("bitwiseNOT", signature(x = "Column"), function(x) { @@ -215,6 +226,7 @@ setMethod("bitwiseNOT", #' @family math_funcs #' @export #' @examples \dontrun{cbrt(df$c)} +#' @note cbrt since 1.4.0 setMethod("cbrt", signature(x = "Column"), function(x) { @@ -231,6 +243,7 @@ setMethod("cbrt", #' @family math_funcs #' @export #' @examples \dontrun{ceil(df$c)} +#' @note ceil since 1.5.0 setMethod("ceil", signature(x = "Column"), function(x) { @@ -254,6 +267,7 @@ col <- function(x) { #' @family normal_funcs #' @export #' @examples \dontrun{column(df)} +#' @note column since 1.6.0 setMethod("column", signature(x = "character"), function(x) { @@ -268,6 +282,7 @@ setMethod("column", #' @family math_funcs #' @export #' @examples \dontrun{corr(df$c, df$d)} +#' @note corr since 1.6.0 setMethod("corr", signature(x = "Column"), function(x, col2) { stopifnot(class(col2) == "Column") @@ -290,6 +305,7 @@ setMethod("corr", signature(x = "Column"), #' covar_samp(df$c, df$d) #' covar_samp("c", "d") #' } +#' @note cov since 1.6.0 setMethod("cov", signature(x = "characterOrColumn"), function(x, col2) { stopifnot(is(class(col2), "characterOrColumn")) @@ -298,6 +314,7 @@ setMethod("cov", signature(x = "characterOrColumn"), #' @rdname cov #' @name covar_samp +#' @note covar_samp since 2.0.0 setMethod("covar_samp", signature(col1 = "characterOrColumn", col2 = "characterOrColumn"), function(col1, col2) { stopifnot(class(col1) == class(col2)) @@ -322,6 +339,7 @@ setMethod("covar_samp", signature(col1 = "characterOrColumn", col2 = "characterO #' covar_pop(df$c, df$d) #' covar_pop("c", "d") #' } +#' @note covar_pop since 2.0.0 setMethod("covar_pop", signature(col1 = "characterOrColumn", col2 = "characterOrColumn"), function(col1, col2) { stopifnot(class(col1) == class(col2)) @@ -342,6 +360,7 @@ setMethod("covar_pop", signature(col1 = "characterOrColumn", col2 = "characterOr #' @family math_funcs #' @export #' @examples \dontrun{cos(df$c)} +#' @note cos since 1.5.0 setMethod("cos", signature(x = "Column"), function(x) { @@ -358,6 +377,7 @@ setMethod("cos", #' @family math_funcs #' @export #' @examples \dontrun{cosh(df$c)} +#' @note cosh since 1.5.0 setMethod("cosh", signature(x = "Column"), function(x) { @@ -374,6 +394,7 @@ setMethod("cosh", #' @family agg_funcs #' @export #' @examples \dontrun{count(df$c)} +#' @note count since 1.4.0 setMethod("count", signature(x = "Column"), function(x) { @@ -391,6 +412,7 @@ setMethod("count", #' @family misc_funcs #' @export #' @examples \dontrun{crc32(df$c)} +#' @note crc32 since 1.5.0 setMethod("crc32", signature(x = "Column"), function(x) { @@ -407,6 +429,7 @@ setMethod("crc32", #' @family misc_funcs #' @export #' @examples \dontrun{hash(df$c)} +#' @note hash since 2.0.0 setMethod("hash", signature(x = "Column"), function(x, ...) { @@ -427,6 +450,7 @@ setMethod("hash", #' @family datetime_funcs #' @export #' @examples \dontrun{dayofmonth(df$c)} +#' @note dayofmonth since 1.5.0 setMethod("dayofmonth", signature(x = "Column"), function(x) { @@ -443,6 +467,7 @@ setMethod("dayofmonth", #' @family datetime_funcs #' @export #' @examples \dontrun{dayofyear(df$c)} +#' @note dayofyear since 1.5.0 setMethod("dayofyear", signature(x = "Column"), function(x) { @@ -460,6 +485,7 @@ setMethod("dayofyear", #' @family string_funcs #' @export #' @examples \dontrun{decode(df$c, "UTF-8")} +#' @note decode since 1.6.0 setMethod("decode", signature(x = "Column", charset = "character"), function(x, charset) { @@ -477,6 +503,7 @@ setMethod("decode", #' @family string_funcs #' @export #' @examples \dontrun{encode(df$c, "UTF-8")} +#' @note encode since 1.6.0 setMethod("encode", signature(x = "Column", charset = "character"), function(x, charset) { @@ -493,6 +520,7 @@ setMethod("encode", #' @family math_funcs #' @export #' @examples \dontrun{exp(df$c)} +#' @note exp since 1.5.0 setMethod("exp", signature(x = "Column"), function(x) { @@ -509,6 +537,7 @@ setMethod("exp", #' @family math_funcs #' @export #' @examples \dontrun{expm1(df$c)} +#' @note expm1 since 1.5.0 setMethod("expm1", signature(x = "Column"), function(x) { @@ -525,6 +554,7 @@ setMethod("expm1", #' @family math_funcs #' @export #' @examples \dontrun{factorial(df$c)} +#' @note factorial since 1.5.0 setMethod("factorial", signature(x = "Column"), function(x) { @@ -548,6 +578,7 @@ setMethod("factorial", #' first(df$c) #' first(df$c, TRUE) #' } +#' @note first(characterOrColumn) since 1.4.0 setMethod("first", signature(x = "characterOrColumn"), function(x, na.rm = FALSE) { @@ -569,6 +600,7 @@ setMethod("first", #' @family math_funcs #' @export #' @examples \dontrun{floor(df$c)} +#' @note floor since 1.5.0 setMethod("floor", signature(x = "Column"), function(x) { @@ -585,6 +617,7 @@ setMethod("floor", #' @family math_funcs #' @export #' @examples \dontrun{hex(df$c)} +#' @note hex since 1.5.0 setMethod("hex", signature(x = "Column"), function(x) { @@ -601,6 +634,7 @@ setMethod("hex", #' @family datetime_funcs #' @export #' @examples \dontrun{hour(df$c)} +#' @note hour since 1.5.0 setMethod("hour", signature(x = "Column"), function(x) { @@ -620,6 +654,7 @@ setMethod("hour", #' @family string_funcs #' @export #' @examples \dontrun{initcap(df$c)} +#' @note initcap since 1.5.0 setMethod("initcap", signature(x = "Column"), function(x) { @@ -640,6 +675,7 @@ setMethod("initcap", #' is.nan(df$c) #' isnan(df$c) #' } +#' @note is.nan since 2.0.0 setMethod("is.nan", signature(x = "Column"), function(x) { @@ -648,6 +684,7 @@ setMethod("is.nan", #' @rdname is.nan #' @name isnan +#' @note isnan since 2.0.0 setMethod("isnan", signature(x = "Column"), function(x) { @@ -664,6 +701,7 @@ setMethod("isnan", #' @family agg_funcs #' @export #' @examples \dontrun{kurtosis(df$c)} +#' @note kurtosis since 1.6.0 setMethod("kurtosis", signature(x = "Column"), function(x) { @@ -687,6 +725,7 @@ setMethod("kurtosis", #' last(df$c) #' last(df$c, TRUE) #' } +#' @note last since 1.4.0 setMethod("last", signature(x = "characterOrColumn"), function(x, na.rm = FALSE) { @@ -710,6 +749,7 @@ setMethod("last", #' @family datetime_funcs #' @export #' @examples \dontrun{last_day(df$c)} +#' @note last_day since 1.5.0 setMethod("last_day", signature(x = "Column"), function(x) { @@ -726,6 +766,7 @@ setMethod("last_day", #' @family string_funcs #' @export #' @examples \dontrun{length(df$c)} +#' @note length since 1.5.0 setMethod("length", signature(x = "Column"), function(x) { @@ -742,6 +783,7 @@ setMethod("length", #' @family math_funcs #' @export #' @examples \dontrun{log(df$c)} +#' @note log since 1.5.0 setMethod("log", signature(x = "Column"), function(x) { @@ -758,6 +800,7 @@ setMethod("log", #' @family math_funcs #' @export #' @examples \dontrun{log10(df$c)} +#' @note log10 since 1.5.0 setMethod("log10", signature(x = "Column"), function(x) { @@ -774,6 +817,7 @@ setMethod("log10", #' @family math_funcs #' @export #' @examples \dontrun{log1p(df$c)} +#' @note log1p since 1.5.0 setMethod("log1p", signature(x = "Column"), function(x) { @@ -790,6 +834,7 @@ setMethod("log1p", #' @family math_funcs #' @export #' @examples \dontrun{log2(df$c)} +#' @note log2 since 1.5.0 setMethod("log2", signature(x = "Column"), function(x) { @@ -806,6 +851,7 @@ setMethod("log2", #' @family string_funcs #' @export #' @examples \dontrun{lower(df$c)} +#' @note lower since 1.4.0 setMethod("lower", signature(x = "Column"), function(x) { @@ -822,6 +868,7 @@ setMethod("lower", #' @family string_funcs #' @export #' @examples \dontrun{ltrim(df$c)} +#' @note ltrim since 1.5.0 setMethod("ltrim", signature(x = "Column"), function(x) { @@ -838,6 +885,7 @@ setMethod("ltrim", #' @family agg_funcs #' @export #' @examples \dontrun{max(df$c)} +#' @note max since 1.5.0 setMethod("max", signature(x = "Column"), function(x) { @@ -855,6 +903,7 @@ setMethod("max", #' @family misc_funcs #' @export #' @examples \dontrun{md5(df$c)} +#' @note md5 since 1.5.0 setMethod("md5", signature(x = "Column"), function(x) { @@ -872,6 +921,7 @@ setMethod("md5", #' @family agg_funcs #' @export #' @examples \dontrun{mean(df$c)} +#' @note mean since 1.5.0 setMethod("mean", signature(x = "Column"), function(x) { @@ -888,6 +938,7 @@ setMethod("mean", #' @family agg_funcs #' @export #' @examples \dontrun{min(df$c)} +#' @note min since 1.5.0 setMethod("min", signature(x = "Column"), function(x) { @@ -904,6 +955,7 @@ setMethod("min", #' @family datetime_funcs #' @export #' @examples \dontrun{minute(df$c)} +#' @note minute since 1.5.0 setMethod("minute", signature(x = "Column"), function(x) { @@ -947,6 +999,7 @@ setMethod("monotonically_increasing_id", #' @family datetime_funcs #' @export #' @examples \dontrun{month(df$c)} +#' @note month since 1.5.0 setMethod("month", signature(x = "Column"), function(x) { @@ -963,6 +1016,7 @@ setMethod("month", #' @family normal_funcs #' @export #' @examples \dontrun{negate(df$c)} +#' @note negate since 1.5.0 setMethod("negate", signature(x = "Column"), function(x) { @@ -979,6 +1033,7 @@ setMethod("negate", #' @family datetime_funcs #' @export #' @examples \dontrun{quarter(df$c)} +#' @note quarter since 1.5.0 setMethod("quarter", signature(x = "Column"), function(x) { @@ -995,6 +1050,7 @@ setMethod("quarter", #' @family string_funcs #' @export #' @examples \dontrun{reverse(df$c)} +#' @note reverse since 1.5.0 setMethod("reverse", signature(x = "Column"), function(x) { @@ -1012,6 +1068,7 @@ setMethod("reverse", #' @family math_funcs #' @export #' @examples \dontrun{rint(df$c)} +#' @note rint since 1.5.0 setMethod("rint", signature(x = "Column"), function(x) { @@ -1028,6 +1085,7 @@ setMethod("rint", #' @family math_funcs #' @export #' @examples \dontrun{round(df$c)} +#' @note round since 1.5.0 setMethod("round", signature(x = "Column"), function(x) { @@ -1047,6 +1105,7 @@ setMethod("round", #' @family math_funcs #' @export #' @examples \dontrun{bround(df$c, 0)} +#' @note bround since 2.0.0 setMethod("bround", signature(x = "Column"), function(x, scale = 0) { @@ -1064,6 +1123,7 @@ setMethod("bround", #' @family string_funcs #' @export #' @examples \dontrun{rtrim(df$c)} +#' @note rtrim since 1.5.0 setMethod("rtrim", signature(x = "Column"), function(x) { @@ -1086,6 +1146,7 @@ setMethod("rtrim", #'select(df, stddev(df$age)) #'agg(df, sd(df$age)) #'} +#' @note sd since 1.6.0 setMethod("sd", signature(x = "Column"), function(x) { @@ -1102,6 +1163,7 @@ setMethod("sd", #' @family datetime_funcs #' @export #' @examples \dontrun{second(df$c)} +#' @note second since 1.5.0 setMethod("second", signature(x = "Column"), function(x) { @@ -1119,6 +1181,7 @@ setMethod("second", #' @family misc_funcs #' @export #' @examples \dontrun{sha1(df$c)} +#' @note sha1 since 1.5.0 setMethod("sha1", signature(x = "Column"), function(x) { @@ -1135,6 +1198,7 @@ setMethod("sha1", #' @family math_funcs #' @export #' @examples \dontrun{signum(df$c)} +#' @note signum since 1.5.0 setMethod("signum", signature(x = "Column"), function(x) { @@ -1151,6 +1215,7 @@ setMethod("signum", #' @family math_funcs #' @export #' @examples \dontrun{sin(df$c)} +#' @note sin since 1.5.0 setMethod("sin", signature(x = "Column"), function(x) { @@ -1167,6 +1232,7 @@ setMethod("sin", #' @family math_funcs #' @export #' @examples \dontrun{sinh(df$c)} +#' @note sinh since 1.5.0 setMethod("sinh", signature(x = "Column"), function(x) { @@ -1183,6 +1249,7 @@ setMethod("sinh", #' @family agg_funcs #' @export #' @examples \dontrun{skewness(df$c)} +#' @note skewness since 1.6.0 setMethod("skewness", signature(x = "Column"), function(x) { @@ -1199,6 +1266,7 @@ setMethod("skewness", #' @family string_funcs #' @export #' @examples \dontrun{soundex(df$c)} +#' @note soundex since 1.5.0 setMethod("soundex", signature(x = "Column"), function(x) { @@ -1229,6 +1297,7 @@ setMethod("spark_partition_id", #' @rdname sd #' @name stddev +#' @note stddev since 1.6.0 setMethod("stddev", signature(x = "Column"), function(x) { @@ -1246,6 +1315,7 @@ setMethod("stddev", #' @seealso \link{sd}, \link{stddev_samp} #' @export #' @examples \dontrun{stddev_pop(df$c)} +#' @note stddev_pop since 1.6.0 setMethod("stddev_pop", signature(x = "Column"), function(x) { @@ -1263,6 +1333,7 @@ setMethod("stddev_pop", #' @seealso \link{stddev_pop}, \link{sd} #' @export #' @examples \dontrun{stddev_samp(df$c)} +#' @note stddev_samp since 1.6.0 setMethod("stddev_samp", signature(x = "Column"), function(x) { @@ -1283,6 +1354,7 @@ setMethod("stddev_samp", #' struct(df$c, df$d) #' struct("col1", "col2") #' } +#' @note struct since 1.6.0 setMethod("struct", signature(x = "characterOrColumn"), function(x, ...) { @@ -1304,6 +1376,7 @@ setMethod("struct", #' @family math_funcs #' @export #' @examples \dontrun{sqrt(df$c)} +#' @note sqrt since 1.5.0 setMethod("sqrt", signature(x = "Column"), function(x) { @@ -1320,6 +1393,7 @@ setMethod("sqrt", #' @family agg_funcs #' @export #' @examples \dontrun{sum(df$c)} +#' @note sum since 1.5.0 setMethod("sum", signature(x = "Column"), function(x) { @@ -1336,6 +1410,7 @@ setMethod("sum", #' @family agg_funcs #' @export #' @examples \dontrun{sumDistinct(df$c)} +#' @note sumDistinct since 1.4.0 setMethod("sumDistinct", signature(x = "Column"), function(x) { @@ -1352,6 +1427,7 @@ setMethod("sumDistinct", #' @family math_funcs #' @export #' @examples \dontrun{tan(df$c)} +#' @note tan since 1.5.0 setMethod("tan", signature(x = "Column"), function(x) { @@ -1368,6 +1444,7 @@ setMethod("tan", #' @family math_funcs #' @export #' @examples \dontrun{tanh(df$c)} +#' @note tanh since 1.5.0 setMethod("tanh", signature(x = "Column"), function(x) { @@ -1384,6 +1461,7 @@ setMethod("tanh", #' @family math_funcs #' @export #' @examples \dontrun{toDegrees(df$c)} +#' @note toDegrees since 1.4.0 setMethod("toDegrees", signature(x = "Column"), function(x) { @@ -1400,6 +1478,7 @@ setMethod("toDegrees", #' @family math_funcs #' @export #' @examples \dontrun{toRadians(df$c)} +#' @note toRadians since 1.4.0 setMethod("toRadians", signature(x = "Column"), function(x) { @@ -1416,6 +1495,7 @@ setMethod("toRadians", #' @family datetime_funcs #' @export #' @examples \dontrun{to_date(df$c)} +#' @note to_date since 1.5.0 setMethod("to_date", signature(x = "Column"), function(x) { @@ -1432,6 +1512,7 @@ setMethod("to_date", #' @family string_funcs #' @export #' @examples \dontrun{trim(df$c)} +#' @note trim since 1.5.0 setMethod("trim", signature(x = "Column"), function(x) { @@ -1449,6 +1530,7 @@ setMethod("trim", #' @family string_funcs #' @export #' @examples \dontrun{unbase64(df$c)} +#' @note unbase64 since 1.5.0 setMethod("unbase64", signature(x = "Column"), function(x) { @@ -1466,6 +1548,7 @@ setMethod("unbase64", #' @family math_funcs #' @export #' @examples \dontrun{unhex(df$c)} +#' @note unhex since 1.5.0 setMethod("unhex", signature(x = "Column"), function(x) { @@ -1482,6 +1565,7 @@ setMethod("unhex", #' @family string_funcs #' @export #' @examples \dontrun{upper(df$c)} +#' @note upper since 1.4.0 setMethod("upper", signature(x = "Column"), function(x) { @@ -1504,6 +1588,7 @@ setMethod("upper", #'select(df, var_pop(df$age)) #'agg(df, var(df$age)) #'} +#' @note var since 1.6.0 setMethod("var", signature(x = "Column"), function(x) { @@ -1513,6 +1598,7 @@ setMethod("var", #' @rdname var #' @name variance +#' @note variance since 1.6.0 setMethod("variance", signature(x = "Column"), function(x) { @@ -1530,6 +1616,7 @@ setMethod("variance", #' @seealso \link{var}, \link{var_samp} #' @export #' @examples \dontrun{var_pop(df$c)} +#' @note var_pop since 1.5.0 setMethod("var_pop", signature(x = "Column"), function(x) { @@ -1547,6 +1634,7 @@ setMethod("var_pop", #' @seealso \link{var_pop}, \link{var} #' @export #' @examples \dontrun{var_samp(df$c)} +#' @note var_samp since 1.6.0 setMethod("var_samp", signature(x = "Column"), function(x) { @@ -1563,6 +1651,7 @@ setMethod("var_samp", #' @family datetime_funcs #' @export #' @examples \dontrun{weekofyear(df$c)} +#' @note weekofyear since 1.5.0 setMethod("weekofyear", signature(x = "Column"), function(x) { @@ -1579,6 +1668,7 @@ setMethod("weekofyear", #' @family datetime_funcs #' @export #' @examples \dontrun{year(df$c)} +#' @note year since 1.5.0 setMethod("year", signature(x = "Column"), function(x) { @@ -1596,6 +1686,7 @@ setMethod("year", #' @family math_funcs #' @export #' @examples \dontrun{atan2(df$c, x)} +#' @note atan2 since 1.5.0 setMethod("atan2", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1614,6 +1705,7 @@ setMethod("atan2", signature(y = "Column"), #' @family datetime_funcs #' @export #' @examples \dontrun{datediff(df$c, x)} +#' @note datediff since 1.5.0 setMethod("datediff", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1632,6 +1724,7 @@ setMethod("datediff", signature(y = "Column"), #' @family math_funcs #' @export #' @examples \dontrun{hypot(df$c, x)} +#' @note hypot since 1.4.0 setMethod("hypot", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1650,6 +1743,7 @@ setMethod("hypot", signature(y = "Column"), #' @family string_funcs #' @export #' @examples \dontrun{levenshtein(df$c, x)} +#' @note levenshtein since 1.5.0 setMethod("levenshtein", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1668,6 +1762,7 @@ setMethod("levenshtein", signature(y = "Column"), #' @family datetime_funcs #' @export #' @examples \dontrun{months_between(df$c, x)} +#' @note months_between since 1.5.0 setMethod("months_between", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1687,6 +1782,7 @@ setMethod("months_between", signature(y = "Column"), #' @family normal_funcs #' @export #' @examples \dontrun{nanvl(df$c, x)} +#' @note nanvl since 1.5.0 setMethod("nanvl", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1706,6 +1802,7 @@ setMethod("nanvl", signature(y = "Column"), #' @family math_funcs #' @export #' @examples \dontrun{pmod(df$c, x)} +#' @note pmod since 1.5.0 setMethod("pmod", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1724,6 +1821,7 @@ setMethod("pmod", signature(y = "Column"), #' @return the approximate number of distinct items in a group. #' @export #' @examples \dontrun{approxCountDistinct(df$c, 0.02)} +#' @note approxCountDistinct(Column, numeric) since 1.4.0 setMethod("approxCountDistinct", signature(x = "Column"), function(x, rsd = 0.05) { @@ -1739,6 +1837,7 @@ setMethod("approxCountDistinct", #' @return the number of distinct items in a group. #' @export #' @examples \dontrun{countDistinct(df$c)} +#' @note countDistinct since 1.4.0 setMethod("countDistinct", signature(x = "Column"), function(x, ...) { @@ -1761,6 +1860,7 @@ setMethod("countDistinct", #' @name concat #' @export #' @examples \dontrun{concat(df$strings, df$strings2)} +#' @note concat since 1.5.0 setMethod("concat", signature(x = "Column"), function(x, ...) { @@ -1782,6 +1882,7 @@ setMethod("concat", #' @name greatest #' @export #' @examples \dontrun{greatest(df$c, df$d)} +#' @note greatest since 1.5.0 setMethod("greatest", signature(x = "Column"), function(x, ...) { @@ -1804,6 +1905,7 @@ setMethod("greatest", #' @name least #' @export #' @examples \dontrun{least(df$c, df$d)} +#' @note least since 1.5.0 setMethod("least", signature(x = "Column"), function(x, ...) { @@ -1824,6 +1926,7 @@ setMethod("least", #' @name ceiling #' @export #' @examples \dontrun{ceiling(df$c)} +#' @note ceiling since 1.5.0 setMethod("ceiling", signature(x = "Column"), function(x) { @@ -1838,6 +1941,7 @@ setMethod("ceiling", #' @name sign #' @export #' @examples \dontrun{sign(df$c)} +#' @note sign since 1.5.0 setMethod("sign", signature(x = "Column"), function(x) { signum(x) @@ -1851,6 +1955,7 @@ setMethod("sign", signature(x = "Column"), #' @name n_distinct #' @export #' @examples \dontrun{n_distinct(df$c)} +#' @note n_distinct since 1.4.0 setMethod("n_distinct", signature(x = "Column"), function(x, ...) { countDistinct(x, ...) @@ -1864,6 +1969,7 @@ setMethod("n_distinct", signature(x = "Column"), #' @name n #' @export #' @examples \dontrun{n(df$c)} +#' @note n since 1.4.0 setMethod("n", signature(x = "Column"), function(x) { count(x) @@ -1885,6 +1991,7 @@ setMethod("n", signature(x = "Column"), #' @name date_format #' @export #' @examples \dontrun{date_format(df$t, 'MM/dd/yyy')} +#' @note date_format since 1.5.0 setMethod("date_format", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_format", y@jc, x) @@ -1900,6 +2007,7 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' @name from_utc_timestamp #' @export #' @examples \dontrun{from_utc_timestamp(df$t, 'PST')} +#' @note from_utc_timestamp since 1.5.0 setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x) @@ -1919,6 +2027,7 @@ setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), #' @name instr #' @export #' @examples \dontrun{instr(df$c, 'b')} +#' @note instr since 1.5.0 setMethod("instr", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "instr", y@jc, x) @@ -1945,6 +2054,7 @@ setMethod("instr", signature(y = "Column", x = "character"), #'next_day(df$d, 'Sun') #'next_day(df$d, 'Sunday') #'} +#' @note next_day since 1.5.0 setMethod("next_day", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "next_day", y@jc, x) @@ -1960,6 +2070,7 @@ setMethod("next_day", signature(y = "Column", x = "character"), #' @name to_utc_timestamp #' @export #' @examples \dontrun{to_utc_timestamp(df$t, 'PST')} +#' @note to_utc_timestamp since 1.5.0 setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x) @@ -1975,6 +2086,7 @@ setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), #' @rdname add_months #' @export #' @examples \dontrun{add_months(df$d, 1)} +#' @note add_months since 1.5.0 setMethod("add_months", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "add_months", y@jc, as.integer(x)) @@ -1990,6 +2102,7 @@ setMethod("add_months", signature(y = "Column", x = "numeric"), #' @name date_add #' @export #' @examples \dontrun{date_add(df$d, 1)} +#' @note date_add since 1.5.0 setMethod("date_add", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_add", y@jc, as.integer(x)) @@ -2005,6 +2118,7 @@ setMethod("date_add", signature(y = "Column", x = "numeric"), #' @name date_sub #' @export #' @examples \dontrun{date_sub(df$d, 1)} +#' @note date_sub since 1.5.0 setMethod("date_sub", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_sub", y@jc, as.integer(x)) @@ -2026,6 +2140,7 @@ setMethod("date_sub", signature(y = "Column", x = "numeric"), #' @name format_number #' @export #' @examples \dontrun{format_number(df$n, 4)} +#' @note format_number since 1.5.0 setMethod("format_number", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2046,6 +2161,7 @@ setMethod("format_number", signature(y = "Column", x = "numeric"), #' @name sha2 #' @export #' @examples \dontrun{sha2(df$c, 256)} +#' @note sha2 since 1.5.0 setMethod("sha2", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "sha2", y@jc, as.integer(x)) @@ -2062,6 +2178,7 @@ setMethod("sha2", signature(y = "Column", x = "numeric"), #' @name shiftLeft #' @export #' @examples \dontrun{shiftLeft(df$c, 1)} +#' @note shiftLeft since 1.5.0 setMethod("shiftLeft", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2080,6 +2197,7 @@ setMethod("shiftLeft", signature(y = "Column", x = "numeric"), #' @name shiftRight #' @export #' @examples \dontrun{shiftRight(df$c, 1)} +#' @note shiftRight since 1.5.0 setMethod("shiftRight", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2098,6 +2216,7 @@ setMethod("shiftRight", signature(y = "Column", x = "numeric"), #' @name shiftRightUnsigned #' @export #' @examples \dontrun{shiftRightUnsigned(df$c, 1)} +#' @note shiftRightUnsigned since 1.5.0 setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2116,6 +2235,7 @@ setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), #' @name concat_ws #' @export #' @examples \dontrun{concat_ws('-', df$s, df$d)} +#' @note concat_ws since 1.5.0 setMethod("concat_ws", signature(sep = "character", x = "Column"), function(sep, x, ...) { jcols <- lapply(list(x, ...), function(x) { x@jc }) @@ -2132,6 +2252,7 @@ setMethod("concat_ws", signature(sep = "character", x = "Column"), #' @name conv #' @export #' @examples \dontrun{conv(df$n, 2, 16)} +#' @note conv since 1.5.0 setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeric"), function(x, fromBase, toBase) { fromBase <- as.integer(fromBase) @@ -2152,6 +2273,7 @@ setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeri #' @name expr #' @export #' @examples \dontrun{expr('length(name)')} +#' @note expr since 1.5.0 setMethod("expr", signature(x = "character"), function(x) { jc <- callJStatic("org.apache.spark.sql.functions", "expr", x) @@ -2167,6 +2289,7 @@ setMethod("expr", signature(x = "character"), #' @name format_string #' @export #' @examples \dontrun{format_string('%d %s', df$a, df$b)} +#' @note format_string since 1.5.0 setMethod("format_string", signature(format = "character", x = "Column"), function(format, x, ...) { jcols <- lapply(list(x, ...), function(arg) { arg@jc }) @@ -2191,6 +2314,7 @@ setMethod("format_string", signature(format = "character", x = "Column"), #'from_unixtime(df$t) #'from_unixtime(df$t, 'yyyy/MM/dd HH') #'} +#' @note from_unixtime since 1.5.0 setMethod("from_unixtime", signature(x = "Column"), function(x, format = "yyyy-MM-dd HH:mm:ss") { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2236,6 +2360,7 @@ setMethod("from_unixtime", signature(x = "Column"), #' # Thirty-second windows every 10 seconds, e.g. 09:00:00-09:00:30, 09:00:10-09:00:40, ... #' window(df$time, "30 seconds", "10 seconds") #'} +#' @note window since 2.0.0 setMethod("window", signature(x = "Column"), function(x, windowDuration, slideDuration = NULL, startTime = NULL) { stopifnot(is.character(windowDuration)) @@ -2273,6 +2398,7 @@ setMethod("window", signature(x = "Column"), #' @name locate #' @export #' @examples \dontrun{locate('b', df$c, 1)} +#' @note locate since 1.5.0 setMethod("locate", signature(substr = "character", str = "Column"), function(substr, str, pos = 1) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2290,6 +2416,7 @@ setMethod("locate", signature(substr = "character", str = "Column"), #' @name lpad #' @export #' @examples \dontrun{lpad(df$c, 6, '#')} +#' @note lpad since 1.5.0 setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), function(x, len, pad) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2307,6 +2434,7 @@ setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), #' @name rand #' @export #' @examples \dontrun{rand()} +#' @note rand since 1.5.0 setMethod("rand", signature(seed = "missing"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "rand") @@ -2316,6 +2444,7 @@ setMethod("rand", signature(seed = "missing"), #' @rdname rand #' @name rand #' @export +#' @note rand(numeric) since 1.5.0 setMethod("rand", signature(seed = "numeric"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "rand", as.integer(seed)) @@ -2331,6 +2460,7 @@ setMethod("rand", signature(seed = "numeric"), #' @name randn #' @export #' @examples \dontrun{randn()} +#' @note randn since 1.5.0 setMethod("randn", signature(seed = "missing"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "randn") @@ -2340,6 +2470,7 @@ setMethod("randn", signature(seed = "missing"), #' @rdname randn #' @name randn #' @export +#' @note randn(numeric) since 1.5.0 setMethod("randn", signature(seed = "numeric"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "randn", as.integer(seed)) @@ -2355,6 +2486,7 @@ setMethod("randn", signature(seed = "numeric"), #' @name regexp_extract #' @export #' @examples \dontrun{regexp_extract(df$c, '(\d+)-(\d+)', 1)} +#' @note regexp_extract since 1.5.0 setMethod("regexp_extract", signature(x = "Column", pattern = "character", idx = "numeric"), function(x, pattern, idx) { @@ -2373,6 +2505,7 @@ setMethod("regexp_extract", #' @name regexp_replace #' @export #' @examples \dontrun{regexp_replace(df$c, '(\\d+)', '--')} +#' @note regexp_replace since 1.5.0 setMethod("regexp_replace", signature(x = "Column", pattern = "character", replacement = "character"), function(x, pattern, replacement) { @@ -2391,6 +2524,7 @@ setMethod("regexp_replace", #' @name rpad #' @export #' @examples \dontrun{rpad(df$c, 6, '#')} +#' @note rpad since 1.5.0 setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), function(x, len, pad) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -2415,6 +2549,7 @@ setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), #'substring_index(df$c, '.', 2) #'substring_index(df$c, '.', -1) #'} +#' @note substring_index since 1.5.0 setMethod("substring_index", signature(x = "Column", delim = "character", count = "numeric"), function(x, delim, count) { @@ -2436,6 +2571,7 @@ setMethod("substring_index", #' @name translate #' @export #' @examples \dontrun{translate(df$c, 'rnlt', '123')} +#' @note translate since 1.5.0 setMethod("translate", signature(x = "Column", matchingString = "character", replaceString = "character"), function(x, matchingString, replaceString) { @@ -2458,6 +2594,7 @@ setMethod("translate", #'unix_timestamp(df$t) #'unix_timestamp(df$t, 'yyyy-MM-dd HH') #'} +#' @note unix_timestamp since 1.5.0 setMethod("unix_timestamp", signature(x = "missing", format = "missing"), function(x, format) { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp") @@ -2467,6 +2604,7 @@ setMethod("unix_timestamp", signature(x = "missing", format = "missing"), #' @rdname unix_timestamp #' @name unix_timestamp #' @export +#' @note unix_timestamp(Column) since 1.5.0 setMethod("unix_timestamp", signature(x = "Column", format = "missing"), function(x, format) { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc) @@ -2476,6 +2614,7 @@ setMethod("unix_timestamp", signature(x = "Column", format = "missing"), #' @rdname unix_timestamp #' @name unix_timestamp #' @export +#' @note unix_timestamp(Column, character) since 1.5.0 setMethod("unix_timestamp", signature(x = "Column", format = "character"), function(x, format = "yyyy-MM-dd HH:mm:ss") { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc, format) @@ -2492,6 +2631,7 @@ setMethod("unix_timestamp", signature(x = "Column", format = "character"), #' @seealso \link{ifelse} #' @export #' @examples \dontrun{when(df$age == 2, df$age + 1)} +#' @note when since 1.5.0 setMethod("when", signature(condition = "Column", value = "ANY"), function(condition, value) { condition <- condition@jc @@ -2514,6 +2654,7 @@ setMethod("when", signature(condition = "Column", value = "ANY"), #' ifelse(df$a > 1 & df$b > 2, 0, 1) #' ifelse(df$a > 1, df$a, 1) #' } +#' @note ifelse since 1.5.0 setMethod("ifelse", signature(test = "Column", yes = "ANY", no = "ANY"), function(test, yes, no) { @@ -2544,6 +2685,7 @@ setMethod("ifelse", #' @family window_funcs #' @export #' @examples \dontrun{cume_dist()} +#' @note cume_dist since 1.6.0 setMethod("cume_dist", signature(x = "missing"), function() { @@ -2566,6 +2708,7 @@ setMethod("cume_dist", #' @family window_funcs #' @export #' @examples \dontrun{dense_rank()} +#' @note dense_rank since 1.6.0 setMethod("dense_rank", signature(x = "missing"), function() { @@ -2586,6 +2729,7 @@ setMethod("dense_rank", #' @family window_funcs #' @export #' @examples \dontrun{lag(df$c)} +#' @note lag since 1.6.0 setMethod("lag", signature(x = "characterOrColumn"), function(x, offset, defaultValue = NULL) { @@ -2613,6 +2757,7 @@ setMethod("lag", #' @family window_funcs #' @export #' @examples \dontrun{lead(df$c)} +#' @note lead since 1.6.0 setMethod("lead", signature(x = "characterOrColumn", offset = "numeric", defaultValue = "ANY"), function(x, offset, defaultValue = NULL) { @@ -2640,6 +2785,7 @@ setMethod("lead", #' @family window_funcs #' @export #' @examples \dontrun{ntile(1)} +#' @note ntile since 1.6.0 setMethod("ntile", signature(x = "numeric"), function(x) { @@ -2662,6 +2808,7 @@ setMethod("ntile", #' @family window_funcs #' @export #' @examples \dontrun{percent_rank()} +#' @note percent_rank since 1.6.0 setMethod("percent_rank", signature(x = "missing"), function() { @@ -2685,6 +2832,7 @@ setMethod("percent_rank", #' @family window_funcs #' @export #' @examples \dontrun{rank()} +#' @note rank since 1.6.0 setMethod("rank", signature(x = "missing"), function() { @@ -2710,6 +2858,7 @@ setMethod("rank", #' @family window_funcs #' @export #' @examples \dontrun{row_number()} +#' @note row_number since 1.6.0 setMethod("row_number", signature(x = "missing"), function() { @@ -2730,6 +2879,7 @@ setMethod("row_number", #' @family collection_funcs #' @export #' @examples \dontrun{array_contains(df$c, 1)} +#' @note array_contains since 1.6.0 setMethod("array_contains", signature(x = "Column", value = "ANY"), function(x, value) { @@ -2746,6 +2896,7 @@ setMethod("array_contains", #' @family collection_funcs #' @export #' @examples \dontrun{explode(df$c)} +#' @note explode since 1.5.0 setMethod("explode", signature(x = "Column"), function(x) { @@ -2762,6 +2913,7 @@ setMethod("explode", #' @family collection_funcs #' @export #' @examples \dontrun{size(df$c)} +#' @note size since 1.5.0 setMethod("size", signature(x = "Column"), function(x) { @@ -2787,6 +2939,7 @@ setMethod("size", #' sort_array(df$c) #' sort_array(df$c, FALSE) #' } +#' @note sort_array since 1.6.0 setMethod("sort_array", signature(x = "Column"), function(x, asc = TRUE) { diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 65b9e841be0d7..51e151623cf0e 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -31,6 +31,7 @@ setOldClass("jobj") #' #' @param sgd A Java object reference to the backing Scala GroupedData #' @export +#' @note GroupedData since 1.4.0 setClass("GroupedData", slots = list(sgd = "jobj")) @@ -46,6 +47,7 @@ groupedData <- function(sgd) { #' @rdname show +#' @note show(GroupedData) since 1.4.0 setMethod("show", "GroupedData", function(object) { cat("GroupedData\n") @@ -64,6 +66,7 @@ setMethod("show", "GroupedData", #' \dontrun{ #' count(groupBy(df, "name")) #' } +#' @note count since 1.4.0 setMethod("count", signature(x = "GroupedData"), function(x) { @@ -90,6 +93,7 @@ setMethod("count", #' df3 <- agg(df, ageSum = sum(df$age)) # Creates a new column named ageSum #' df4 <- summarize(df, ageSum = max(df$age)) #' } +#' @note agg since 1.4.0 setMethod("agg", signature(x = "GroupedData"), function(x, ...) { @@ -117,6 +121,7 @@ setMethod("agg", #' @rdname summarize #' @name summarize +#' @note summarize since 1.4.0 setMethod("summarize", signature(x = "GroupedData"), function(x, ...) { @@ -190,6 +195,7 @@ createMethods() #' 3 3 3.0 #' 1 1 1.5 #' } +#' @note gapply(GroupedData) since 2.0.0 setMethod("gapply", signature(x = "GroupedData"), function(x, func, schema) { diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R index 5b9142bbd338e..4905e1fe5c61f 100644 --- a/R/pkg/R/jobj.R +++ b/R/pkg/R/jobj.R @@ -71,6 +71,7 @@ jobj <- function(objId) { #' #' @param x The JVM object reference #' @param ... further arguments passed to or from other methods +#' @note print.jobj since 1.4.0 print.jobj <- function(x, ...) { name <- getClassName.jobj(x) cat("Java ref type", name, "id", x$id, "\n", sep = " ") diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 2127daee18b0a..d6ff2aa22df2f 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -29,24 +29,28 @@ #' #' @param jobj a Java object reference to the backing Scala GeneralizedLinearRegressionWrapper #' @export +#' @note GeneralizedLinearRegressionModel since 2.0.0 setClass("GeneralizedLinearRegressionModel", representation(jobj = "jobj")) #' S4 class that represents a NaiveBayesModel #' #' @param jobj a Java object reference to the backing Scala NaiveBayesWrapper #' @export +#' @note NaiveBayesModel since 2.0.0 setClass("NaiveBayesModel", representation(jobj = "jobj")) #' S4 class that represents a AFTSurvivalRegressionModel #' #' @param jobj a Java object reference to the backing Scala AFTSurvivalRegressionWrapper #' @export +#' @note AFTSurvivalRegressionModel since 2.0.0 setClass("AFTSurvivalRegressionModel", representation(jobj = "jobj")) #' S4 class that represents a KMeansModel #' #' @param jobj a Java object reference to the backing Scala KMeansModel #' @export +#' @note KMeansModel since 2.0.0 setClass("KMeansModel", representation(jobj = "jobj")) #' Fits a generalized linear model @@ -73,6 +77,7 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' model <- spark.glm(df, Sepal_Length ~ Sepal_Width, family="gaussian") #' summary(model) #' } +#' @note spark.glm since 2.0.0 setMethod( "spark.glm", signature(data = "SparkDataFrame", formula = "formula"), @@ -120,6 +125,7 @@ setMethod( #' model <- glm(Sepal_Length ~ Sepal_Width, df, family="gaussian") #' summary(model) #' } +#' @note glm since 1.5.0 setMethod("glm", signature(formula = "formula", family = "ANY", data = "SparkDataFrame"), function(formula, family = gaussian, data, epsilon = 1e-06, maxit = 25) { spark.glm(data, formula, family, epsilon, maxit) @@ -138,6 +144,7 @@ setMethod("glm", signature(formula = "formula", family = "ANY", data = "SparkDat #' model <- glm(y ~ x, trainingData) #' summary(model) #' } +#' @note summary(GeneralizedLinearRegressionModel) since 2.0.0 setMethod("summary", signature(object = "GeneralizedLinearRegressionModel"), function(object, ...) { jobj <- object@jobj @@ -173,6 +180,7 @@ setMethod("summary", signature(object = "GeneralizedLinearRegressionModel"), #' @rdname print #' @name print.summary.GeneralizedLinearRegressionModel #' @export +#' @note print.summary.GeneralizedLinearRegressionModel since 2.0.0 print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { if (x$is.loaded) { cat("\nSaved-loaded model does not support output 'Deviance Residuals'.\n") @@ -215,6 +223,7 @@ print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { #' predicted <- predict(model, testData) #' showDF(predicted) #' } +#' @note predict(GeneralizedLinearRegressionModel) since 1.5.0 setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), function(object, newData) { return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) @@ -236,6 +245,7 @@ setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), #' predicted <- predict(model, testData) #' showDF(predicted) #'} +#' @note predict(NaiveBayesModel) since 2.0.0 setMethod("predict", signature(object = "NaiveBayesModel"), function(object, newData) { return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) @@ -256,6 +266,7 @@ setMethod("predict", signature(object = "NaiveBayesModel"), #' model <- spark.naiveBayes(trainingData, y ~ x) #' summary(model) #'} +#' @note summary(NaiveBayesModel) since 2.0.0 setMethod("summary", signature(object = "NaiveBayesModel"), function(object, ...) { jobj <- object@jobj @@ -289,6 +300,7 @@ setMethod("summary", signature(object = "NaiveBayesModel"), #' \dontrun{ #' model <- spark.kmeans(data, ~ ., k=2, initMode="random") #' } +#' @note spark.kmeans since 2.0.0 setMethod("spark.kmeans", signature(data = "SparkDataFrame", formula = "formula"), function(data, formula, k, maxIter = 10, initMode = c("random", "k-means||")) { formula <- paste(deparse(formula), collapse = "") @@ -313,6 +325,7 @@ setMethod("spark.kmeans", signature(data = "SparkDataFrame", formula = "formula" #' fitted.model <- fitted(model) #' showDF(fitted.model) #'} +#' @note fitted since 2.0.0 setMethod("fitted", signature(object = "KMeansModel"), function(object, method = c("centers", "classes"), ...) { method <- match.arg(method) @@ -339,6 +352,7 @@ setMethod("fitted", signature(object = "KMeansModel"), #' model <- spark.kmeans(trainingData, ~ ., 2) #' summary(model) #' } +#' @note summary(KMeansModel) since 2.0.0 setMethod("summary", signature(object = "KMeansModel"), function(object, ...) { jobj <- object@jobj @@ -374,6 +388,7 @@ setMethod("summary", signature(object = "KMeansModel"), #' predicted <- predict(model, testData) #' showDF(predicted) #' } +#' @note predict(KMeansModel) since 2.0.0 setMethod("predict", signature(object = "KMeansModel"), function(object, newData) { return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) @@ -396,6 +411,7 @@ setMethod("predict", signature(object = "KMeansModel"), #' df <- createDataFrame(infert) #' model <- spark.naiveBayes(df, education ~ ., laplace = 0) #'} +#' @note spark.naiveBayes since 2.0.0 setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "formula"), function(data, formula, laplace = 0, ...) { formula <- paste(deparse(formula), collapse = "") @@ -423,6 +439,7 @@ setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "form #' path <- "path/to/model" #' write.ml(model, path) #' } +#' @note write.ml(NaiveBayesModel, character) since 2.0.0 setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), function(object, path, overwrite = FALSE) { writer <- callJMethod(object@jobj, "write") @@ -450,6 +467,7 @@ setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), #' path <- "path/to/model" #' write.ml(model, path) #' } +#' @note write.ml(AFTSurvivalRegressionModel, character) since 2.0.0 setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "character"), function(object, path, overwrite = FALSE) { writer <- callJMethod(object@jobj, "write") @@ -477,6 +495,7 @@ setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "c #' path <- "path/to/model" #' write.ml(model, path) #' } +#' @note write.ml(GeneralizedLinearRegressionModel, character) since 2.0.0 setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", path = "character"), function(object, path, overwrite = FALSE) { writer <- callJMethod(object@jobj, "write") @@ -504,6 +523,7 @@ setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", pat #' path <- "path/to/model" #' write.ml(model, path) #' } +#' @note write.ml(KMeansModel, character) since 2.0.0 setMethod("write.ml", signature(object = "KMeansModel", path = "character"), function(object, path, overwrite = FALSE) { writer <- callJMethod(object@jobj, "write") @@ -525,6 +545,7 @@ setMethod("write.ml", signature(object = "KMeansModel", path = "character"), #' path <- "path/to/model" #' model <- read.ml(path) #' } +#' @note read.ml since 2.0.0 read.ml <- function(path) { path <- suppressWarnings(normalizePath(path)) jobj <- callJStatic("org.apache.spark.ml.r.RWrappers", "load", path) @@ -558,6 +579,7 @@ read.ml <- function(path) { #' df <- createDataFrame(ovarian) #' model <- spark.survreg(df, Surv(futime, fustat) ~ ecog_ps + rx) #' } +#' @note spark.survreg since 2.0.0 setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula"), function(data, formula, ...) { formula <- paste(deparse(formula), collapse = "") @@ -581,6 +603,7 @@ setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula #' model <- spark.survreg(trainingData, Surv(futime, fustat) ~ ecog_ps + rx) #' summary(model) #' } +#' @note summary(AFTSurvivalRegressionModel) since 2.0.0 setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), function(object, ...) { jobj <- object@jobj @@ -608,6 +631,7 @@ setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), #' predicted <- predict(model, testData) #' showDF(predicted) #' } +#' @note predict(AFTSurvivalRegressionModel) since 2.0.0 setMethod("predict", signature(object = "AFTSurvivalRegressionModel"), function(object, newData) { return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R index 89a2cfac3cb0d..fb23c780ad860 100644 --- a/R/pkg/R/schema.R +++ b/R/pkg/R/schema.R @@ -35,6 +35,7 @@ #' function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, #' schema) #' } +#' @note structType since 1.4.0 structType <- function(x, ...) { UseMethod("structType", x) } @@ -67,6 +68,7 @@ structType.structField <- function(x, ...) { #' #' @param x A StructType object #' @param ... further arguments passed to or from other methods +#' @note print.structType since 1.4.0 print.structType <- function(x, ...) { cat("StructType\n", sapply(x$fields(), @@ -98,7 +100,7 @@ print.structType <- function(x, ...) { #' function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, #' schema) #' } - +#' @note structField since 1.4.0 structField <- function(x, ...) { UseMethod("structField", x) } @@ -202,6 +204,7 @@ structField.character <- function(x, type, nullable = TRUE) { #' #' @param x A StructField object #' @param ... further arguments passed to or from other methods +#' @note print.structField since 1.4.0 print.structField <- function(x, ...) { cat("StructField(name = \"", x$name(), "\", type = \"", x$dataType.toString(), diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index d05660ccf6c22..94d0e63c8a46a 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -31,6 +31,7 @@ connExists <- function(env) { #' @rdname sparkR.session.stop #' @name sparkR.stop #' @export +#' @note sparkR.stop since 1.4.0 sparkR.stop <- function() { sparkR.session.stop() } @@ -41,7 +42,7 @@ sparkR.stop <- function() { #' @rdname sparkR.session.stop #' @name sparkR.session.stop #' @export -#' @note since 2.0.0 +#' @note sparkR.session.stop since 2.0.0 sparkR.session.stop <- function() { env <- .sparkREnv if (exists(".sparkRCon", envir = env)) { @@ -112,7 +113,7 @@ sparkR.session.stop <- function() { #' c("one.jar", "two.jar", "three.jar"), #' c("com.databricks:spark-avro_2.10:2.0.1")) #'} - +#' @note sparkR.init since 1.4.0 sparkR.init <- function( master = "", appName = "SparkR", @@ -265,7 +266,7 @@ sparkR.sparkContext <- function( #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #'} - +#' @note sparkRSQL.init since 1.4.0 sparkRSQL.init <- function(jsc = NULL) { .Deprecated("sparkR.session") @@ -293,7 +294,7 @@ sparkRSQL.init <- function(jsc = NULL) { #' sc <- sparkR.init() #' sqlContext <- sparkRHive.init(sc) #'} - +#' @note sparkRHive.init since 1.4.0 sparkRHive.init <- function(jsc = NULL) { .Deprecated("sparkR.session") @@ -334,8 +335,7 @@ sparkRHive.init <- function(jsc = NULL) { #' c("com.databricks:spark-avro_2.10:2.0.1")) #' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g") #'} -#' @note since 2.0.0 - +#' @note sparkR.session since 2.0.0 sparkR.session <- function( master = "", appName = "SparkR", @@ -399,7 +399,7 @@ sparkR.session <- function( #' sc <- sparkR.init() #' setJobGroup(sc, "myJobGroup", "My job group description", TRUE) #'} - +#' @note setJobGroup since 1.5.0 setJobGroup <- function(sc, groupId, description, interruptOnCancel) { callJMethod(sc, "setJobGroup", groupId, description, interruptOnCancel) } @@ -412,7 +412,7 @@ setJobGroup <- function(sc, groupId, description, interruptOnCancel) { #' sc <- sparkR.init() #' clearJobGroup(sc) #'} - +#' @note clearJobGroup since 1.5.0 clearJobGroup <- function(sc) { callJMethod(sc, "clearJobGroup") } @@ -426,7 +426,7 @@ clearJobGroup <- function(sc) { #' sc <- sparkR.init() #' cancelJobGroup(sc, "myJobGroup") #'} - +#' @note cancelJobGroup since 1.5.0 cancelJobGroup <- function(sc, groupId) { callJMethod(sc, "cancelJobGroup", groupId) } diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index 6b53517873a72..e92b9e3d84f1e 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -40,6 +40,7 @@ setOldClass("jobj") #' df <- read.json("/path/to/file.json") #' ct <- crosstab(df, "title", "gender") #' } +#' @note crosstab since 1.5.0 setMethod("crosstab", signature(x = "SparkDataFrame", col1 = "character", col2 = "character"), function(x, col1, col2) { @@ -65,6 +66,7 @@ setMethod("crosstab", #' df <- read.json("/path/to/file.json") #' cov <- cov(df, "title", "gender") #' } +#' @note cov since 1.6.0 setMethod("cov", signature(x = "SparkDataFrame"), function(x, col1, col2) { @@ -95,6 +97,7 @@ setMethod("cov", #' corr <- corr(df, "title", "gender") #' corr <- corr(df, "title", "gender", method = "pearson") #' } +#' @note corr since 1.6.0 setMethod("corr", signature(x = "SparkDataFrame"), function(x, col1, col2, method = "pearson") { @@ -123,6 +126,7 @@ setMethod("corr", #' df <- read.json("/path/to/file.json") #' fi = freqItems(df, c("title", "gender")) #' } +#' @note freqItems since 1.6.0 setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), function(x, cols, support = 0.01) { statFunctions <- callJMethod(x@sdf, "stat") @@ -160,6 +164,7 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), #' df <- read.json("/path/to/file.json") #' quantiles <- approxQuantile(df, "key", c(0.5, 0.8), 0.0) #' } +#' @note approxQuantile since 2.0.0 setMethod("approxQuantile", signature(x = "SparkDataFrame", col = "character", probabilities = "numeric", relativeError = "numeric"), @@ -188,6 +193,7 @@ setMethod("approxQuantile", #' df <- read.json("/path/to/file.json") #' sample <- sampleBy(df, "key", fractions, 36) #' } +#' @note sampleBy since 1.6.0 setMethod("sampleBy", signature(x = "SparkDataFrame", col = "character", fractions = "list", seed = "numeric"), diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index aafb34472feb1..d5c062d3bcc7d 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -115,6 +115,7 @@ isRDD <- function(name, env) { #' hashCode(1.0) # 1072693248 #' hashCode("1") # 49 #'} +#' @note hashCode since 1.4.0 hashCode <- function(key) { if (class(key) == "integer") { as.integer(key[[1]]) diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R index 7ecf70abc6b50..e4bc933b9aaba 100644 --- a/R/pkg/R/window.R +++ b/R/pkg/R/window.R @@ -32,6 +32,7 @@ #' ws <- window.partitionBy(df$key1, df$key2) #' df1 <- select(df, over(lead("value", 1), ws)) #' } +#' @note window.partitionBy(character) since 2.0.0 setMethod("window.partitionBy", signature(col = "character"), function(col, ...) { @@ -45,6 +46,7 @@ setMethod("window.partitionBy", #' @rdname window.partitionBy #' @name window.partitionBy #' @export +#' @note window.partitionBy(Column) since 2.0.0 setMethod("window.partitionBy", signature(col = "Column"), function(col, ...) { @@ -72,6 +74,7 @@ setMethod("window.partitionBy", #' ws <- window.orderBy(df$key1, df$key2) #' df1 <- select(df, over(lead("value", 1), ws)) #' } +#' @note window.orderBy(character) since 2.0.0 setMethod("window.orderBy", signature(col = "character"), function(col, ...) { @@ -85,6 +88,7 @@ setMethod("window.orderBy", #' @rdname window.orderBy #' @name window.orderBy #' @export +#' @note window.orderBy(Column) since 2.0.0 setMethod("window.orderBy", signature(col = "Column"), function(col, ...) { From 8159da20ee9c170324772792f2b242a85cbb7d34 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 20 Jun 2016 14:50:28 -0700 Subject: [PATCH 0732/1470] [SPARK-15863][SQL][DOC] Initial SQL programming guide update for Spark 2.0 ## What changes were proposed in this pull request? Initial SQL programming guide update for Spark 2.0. Contents like 1.6 to 2.0 migration guide are still incomplete. We may also want to add more examples for Scala/Java Dataset typed transformations. ## How was this patch tested? N/A Author: Cheng Lian Closes #13592 from liancheng/sql-programming-guide-2.0. (cherry picked from commit 6df8e3886063a9d8c2e8499456ea9166245d5640) Signed-off-by: Yin Huai --- docs/sql-programming-guide.md | 605 ++++++++++++++++++---------------- 1 file changed, 317 insertions(+), 288 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index efdf873c34556..d93f30b1e5cc3 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -12,130 +12,129 @@ title: Spark SQL and DataFrames Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, Spark SQL uses this extra information to perform extra optimizations. There are several ways to -interact with Spark SQL including SQL, the DataFrames API and the Datasets API. When computing a result +interact with Spark SQL including SQL and the Dataset API. When computing a result the same execution engine is used, independent of which API/language you are using to express the -computation. This unification means that developers can easily switch back and forth between the -various APIs based on which provides the most natural way to express a given transformation. +computation. This unification means that developers can easily switch back and forth between +different APIs based on which provides the most natural way to express a given transformation. All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`, `pyspark` shell, or `sparkR` shell. ## SQL -One use of Spark SQL is to execute SQL queries written using either a basic SQL syntax or HiveQL. +One use of Spark SQL is to execute SQL queries. Spark SQL can also be used to read data from an existing Hive installation. For more on how to configure this feature, please refer to the [Hive Tables](#hive-tables) section. When running -SQL from within another programming language the results will be returned as a [DataFrame](#DataFrames). +SQL from within another programming language the results will be returned as a [DataFrame](#datasets-and-dataframes). You can also interact with the SQL interface using the [command-line](#running-the-spark-sql-cli) or over [JDBC/ODBC](#running-the-thrift-jdbcodbc-server). -## DataFrames +## Datasets and DataFrames -A DataFrame is a distributed collection of data organized into named columns. It is conceptually -equivalent to a table in a relational database or a data frame in R/Python, but with richer -optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such -as: structured data files, tables in Hive, external databases, or existing RDDs. +A Dataset is a new interface added in Spark 1.6 that tries to provide the benefits of RDDs (strong +typing, ability to use powerful lambda functions) with the benefits of Spark SQL's optimized +execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then +manipulated using functional transformations (`map`, `flatMap`, `filter`, etc.). -The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), -[Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), -[Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html). +The Dataset API is the successor of the DataFrame API, which was introduced in Spark 1.3. In Spark +2.0, Datasets and DataFrames are unified, and DataFrames are now equivalent to Datasets of `Row`s. +In fact, `DataFrame` is simply a type alias of `Dataset[Row]` in [the Scala API][scala-datasets]. +However, [Java API][java-datasets] users must use `Dataset` instead. -## Datasets +[scala-datasets]: api/scala/index.html#org.apache.spark.sql.Dataset +[java-datasets]: api/java/index.html?org/apache/spark/sql/Dataset.html -A Dataset is a new experimental interface added in Spark 1.6 that tries to provide the benefits of -RDDs (strong typing, ability to use powerful lambda functions) with the benefits of Spark SQL's -optimized execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then manipulated -using functional transformations (map, flatMap, filter, etc.). +Python does not have support for the Dataset API, but due to its dynamic nature many of the +benefits are already available (i.e. you can access the field of a row by name naturally +`row.columnName`). The case for R is similar. -The unified Dataset API can be used both in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset) and -[Java](api/java/index.html?org/apache/spark/sql/Dataset.html). Python does not yet have support for -the Dataset API, but due to its dynamic nature many of the benefits are already available (i.e. you can -access the field of a row by name naturally `row.columnName`). Full python support will be added -in a future release. +Throughout this document, we will often refer to Scala/Java Datasets of `Row`s as DataFrames. # Getting Started -## Starting Point: SQLContext +## Starting Point: SparkSession
    -The entry point into all functionality in Spark SQL is the -[`SQLContext`](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic `SQLContext`, all you need is a SparkContext. +The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.build()`: {% highlight scala %} -val sc: SparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +import org.apache.spark.sql.SparkSession + +val spark = SparkSession.build() + .master("local") + .appName("Word Count") + .config("spark.some.config.option", "some-value") + .getOrCreate() // this is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ +import spark.implicits._ {% endhighlight %}
    -The entry point into all functionality in Spark SQL is the -[`SQLContext`](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic `SQLContext`, all you need is a SparkContext. +The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.build()`: {% highlight java %} -JavaSparkContext sc = ...; // An existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); -{% endhighlight %} +import org.apache.spark.sql.SparkSession +SparkSession spark = SparkSession.build() + .master("local") + .appName("Word Count") + .config("spark.some.config.option", "some-value") + .getOrCreate(); +{% endhighlight %}
    -The entry point into all relational functionality in Spark is the -[`SQLContext`](api/python/pyspark.sql.html#pyspark.sql.SQLContext) class, or one -of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. +The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.build`: {% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) +from pyspark.sql import SparkSession + +spark = SparkSession.build \ + .master("local") \ + .appName("Word Count") \ + .config("spark.some.config.option", "some-value") \ + .getOrCreate() {% endhighlight %}
    -The entry point into all relational functionality in Spark is the -`SQLContext` class, or one of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. +Unlike Scala, Java, and Python API, we haven't finished migrating `SQLContext` to `SparkSession` for SparkR yet, so +the entry point into all relational functionality in SparkR is still the +`SQLContext` class in Spark 2.0. To create a basic `SQLContext`, all you need is a `SparkContext`. {% highlight r %} -sqlContext <- sparkRSQL.init(sc) +spark <- sparkRSQL.init(sc) {% endhighlight %} +Note that when invoked for the first time, `sparkRSQL.init()` initializes a global `SQLContext` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SQLContext` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SQLContext` instance around.
    -In addition to the basic `SQLContext`, you can also create a `HiveContext`, which provides a -superset of the functionality provided by the basic `SQLContext`. Additional features include -the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the -ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an -existing Hive setup, and all of the data sources available to a `SQLContext` are still available. -`HiveContext` is only packaged separately to avoid including all of Hive's dependencies in the default -Spark build. If these dependencies are not a problem for your application then using `HiveContext` -is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up -to feature parity with a `HiveContext`. - +`SparkSession` (or `SQLContext` for SparkR) in Spark 2.0 provides builtin support for Hive features including the ability to +write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. +To use these features, you do not need to have an existing Hive setup. ## Creating DataFrames -With a `SQLContext`, applications can create `DataFrame`s from an existing `RDD`, from a Hive table, or from data sources. - -As an example, the following creates a `DataFrame` based on the content of a JSON file: -
    -{% highlight scala %} -val sc: SparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). + +As an example, the following creates a DataFrame based on the content of a JSON file: -val df = sqlContext.read.json("examples/src/main/resources/people.json") +{% highlight scala %} +val spark: SparkSession // An existing SparkSession. +val df = spark.read.json("examples/src/main/resources/people.json") // Displays the content of the DataFrame to stdout df.show() @@ -144,11 +143,14 @@ df.show()
    -{% highlight java %} -JavaSparkContext sc = ...; // An existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). + +As an example, the following creates a DataFrame based on the content of a JSON file: -DataFrame df = sqlContext.read().json("examples/src/main/resources/people.json"); +{% highlight java %} +SparkSession spark = ...; // An existing SparkSession. +Dataset df = spark.read().json("examples/src/main/resources/people.json"); // Displays the content of the DataFrame to stdout df.show(); @@ -157,11 +159,14 @@ df.show();
    -{% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). -df = sqlContext.read.json("examples/src/main/resources/people.json") +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% highlight python %} +# spark is an existing SparkSession +df = spark.read.json("examples/src/main/resources/people.json") # Displays the content of the DataFrame to stdout df.show() @@ -170,34 +175,37 @@ df.show()
    -{% highlight r %} -sqlContext <- SQLContext(sc) +With a `SQLContext`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). + +As an example, the following creates a DataFrame based on the content of a JSON file: -df <- read.json(sqlContext, "examples/src/main/resources/people.json") +{% highlight r %} +df <- read.json("examples/src/main/resources/people.json") # Displays the content of the DataFrame to stdout showDF(df) {% endhighlight %}
    -
    -## DataFrame Operations +## Untyped Dataset Operations (aka DataFrame Operations) + +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/DataFrame.html). -DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/DataFrame.html). +As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. -Here we include some basic examples of structured data processing using DataFrames: +Here we include some basic examples of structured data processing using Datasets:
    {% highlight scala %} -val sc: SparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +val spark: SparkSession // An existing SparkSession // Create the DataFrame -val df = sqlContext.read.json("examples/src/main/resources/people.json") +val df = spark.read.json("examples/src/main/resources/people.json") // Show the content of the DataFrame df.show() @@ -239,20 +247,19 @@ df.groupBy("age").count().show() // 30 1 {% endhighlight %} -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.DataFrame). +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$).
    {% highlight java %} -JavaSparkContext sc // An existing SparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc) +SparkSession spark = ...; // An existing SparkSession // Create the DataFrame -DataFrame df = sqlContext.read().json("examples/src/main/resources/people.json"); +Dataset df = spark.read().json("examples/src/main/resources/people.json"); // Show the content of the DataFrame df.show(); @@ -294,9 +301,9 @@ df.groupBy("age").count().show(); // 30 1 {% endhighlight %} -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/java/org/apache/spark/sql/DataFrame.html). +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html).
    @@ -308,11 +315,10 @@ latter form, which is future proof and won't break with column names that are also attributes on the DataFrame class. {% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) +# spark is an existing SparkSession # Create the DataFrame -df = sqlContext.read.json("examples/src/main/resources/people.json") +df = spark.read.json("examples/src/main/resources/people.json") # Show the content of the DataFrame df.show() @@ -363,10 +369,8 @@ In addition to simple column references and expressions, DataFrames also have a
    {% highlight r %} -sqlContext <- sparkRSQL.init(sc) - # Create the DataFrame -df <- read.json(sqlContext, "examples/src/main/resources/people.json") +df <- read.json("examples/src/main/resources/people.json") # Show the content of the DataFrame showDF(df) @@ -419,35 +423,39 @@ In addition to simple column references and expressions, DataFrames also have a ## Running SQL Queries Programmatically -The `sql` function on a `SQLContext` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. -
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + {% highlight scala %} -val sqlContext = ... // An existing SQLContext -val df = sqlContext.sql("SELECT * FROM table") +val spark = ... // An existing SparkSession +val df = spark.sql("SELECT * FROM table") {% endhighlight %}
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. + {% highlight java %} -SQLContext sqlContext = ... // An existing SQLContext -DataFrame df = sqlContext.sql("SELECT * FROM table") +SparkSession spark = ... // An existing SparkSession +Dataset df = spark.sql("SELECT * FROM table") {% endhighlight %}
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + {% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) -df = sqlContext.sql("SELECT * FROM table") +# spark is an existing SparkSession +df = spark.sql("SELECT * FROM table") {% endhighlight %}
    +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + {% highlight r %} -sqlContext <- sparkRSQL.init(sc) -df <- sql(sqlContext, "SELECT * FROM table") +df <- sql("SELECT * FROM table") {% endhighlight %}
    @@ -456,7 +464,7 @@ df <- sql(sqlContext, "SELECT * FROM table") ## Creating Datasets -Datasets are similar to RDDs, however, instead of using Java Serialization or Kryo they use +Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects for processing or transmitting over the network. While both encoders and standard serialization are responsible for turning an object into bytes, encoders are code generated dynamically and use a format @@ -467,7 +475,7 @@ the bytes back into an object.
    {% highlight scala %} -// Encoders for most common types are automatically provided by importing sqlContext.implicits._ +// Encoders for most common types are automatically provided by importing spark.implicits._ val ds = Seq(1, 2, 3).toDS() ds.map(_ + 1).collect() // Returns: Array(2, 3, 4) @@ -477,7 +485,7 @@ val ds = Seq(Person("Andy", 32)).toDS() // DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name. val path = "examples/src/main/resources/people.json" -val people = sqlContext.read.json(path).as[Person] +val people = spark.read.json(path).as[Person] {% endhighlight %} @@ -486,8 +494,30 @@ val people = sqlContext.read.json(path).as[Person]
    {% highlight java %} -JavaSparkContext sc = ...; // An existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); +SparkSession spark = ... // An existing SparkSession + +// Encoders for most common types are provided in class Encoders. +Dataset ds = spark.createDataset(Arrays.asList(1, 2, 3), Encoders.INT()); +ds.map(new MapFunction() { + @Override + public Integer call(Integer value) throws Exception { + return value + 1; + } +}, Encoders.INT()); // Returns: [2, 3, 4] + +Person person = new Person(); +person.setName("Andy"); +person.setAge(32); + +// Encoders are also created for Java beans. +Dataset ds = spark.createDataset( + Collections.singletonList(person), + Encoders.bean(Person.class) +); + +// DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name. +String path = "examples/src/main/resources/people.json"; +Dataset people = spark.read().json(path).as(Encoders.bean(Person.class)); {% endhighlight %}
    @@ -495,14 +525,14 @@ SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); ## Interoperating with RDDs -Spark SQL supports two different methods for converting existing RDDs into DataFrames. The first +Spark SQL supports two different methods for converting existing RDDs into Datasets. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This reflection based approach leads to more concise code and works well when you already know the schema while writing your Spark application. -The second method for creating DataFrames is through a programmatic interface that allows you to +The second method for creating Datasets is through a programmatic interface that allows you to construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows -you to construct DataFrames when the columns and their types are not known until runtime. +you to construct Datasets when the columns and their types are not known until runtime. ### Inferring the Schema Using Reflection
    @@ -513,28 +543,30 @@ The Scala interface for Spark SQL supports automatically converting an RDD conta to a DataFrame. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex -types such as Sequences or Arrays. This RDD can be implicitly converted to a DataFrame and then be +types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +val spark: SparkSession // An existing SparkSession // this is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ +import spark.implicits._ // Define the schema using a case class. // Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) -// Create an RDD of Person objects and register it as a table. -val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF() +// Create an RDD of Person objects and register it as a temporary view. +val people = sc + .textFile("examples/src/main/resources/people.txt") + .map(_.split(",")) + .map(p => Person(p(0), p(1).trim.toInt)) + .toDF() people.createOrReplaceTempView("people") -// SQL statements can be run by using the sql methods provided by sqlContext. -val teenagers = sqlContext.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19") +// SQL statements can be run by using the sql methods provided by spark. +val teenagers = spark.sql("SELECT name, age 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 field index: teenagers.map(t => "Name: " + t(0)).collect().foreach(println) @@ -550,11 +582,12 @@ teenagers.map(_.getValuesMap[Any](List("name", "age"))).collect().foreach(printl
    -Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) -into a DataFrame. The BeanInfo, obtained using reflection, defines the schema of the table. -Currently, Spark SQL does not support JavaBeans that contain -nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a -class that implements Serializable and has getters and setters for all of its fields. +Spark SQL supports automatically converting an RDD of +[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. +The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL +does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` +fields are supported though. You can create a JavaBean by creating a class that implements +Serializable and has getters and setters for all of its fields. {% highlight java %} @@ -586,11 +619,10 @@ A schema can be applied to an existing RDD by calling `createDataFrame` and prov for the JavaBean. {% highlight java %} -// sc is an existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); +SparkSession spark = ...; // An existing SparkSession // Load a text file and convert each line to a JavaBean. -JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").map( +JavaRDD people = spark.sparkContext.textFile("examples/src/main/resources/people.txt").map( new Function() { public Person call(String line) throws Exception { String[] parts = line.split(","); @@ -604,19 +636,18 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m }); // Apply a schema to an RDD of JavaBeans and register it as a table. -DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); +Dataset schemaPeople = spark.createDataFrame(people, Person.class); schemaPeople.createOrReplaceTempView("people"); // SQL can be run over RDDs that have been registered as tables. -DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +Dataset teenagers = spark.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. -List teenagerNames = teenagers.javaRDD().map(new Function() { +List teenagerNames = teenagers.map(new MapFunction() { public String call(Row row) { return "Name: " + row.getString(0); } -}).collect(); +}).collectAsList(); {% endhighlight %} @@ -626,15 +657,12 @@ List teenagerNames = teenagers.javaRDD().map(new Function() Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, -and the types are inferred by looking at the first row. Since we currently only look at the first -row, it is important that there is no missing data in the first row of the RDD. In future versions we -plan to more completely infer the schema by looking at more data, similar to the inference that is -performed on JSON files. +and the types are inferred by sampling the whole datase, similar to the inference that is performed on JSON files. {% highlight python %} -# sc is an existing SparkContext. -from pyspark.sql import SQLContext, Row -sqlContext = SQLContext(sc) +# spark is an existing SparkSession. +from pyspark.sql import Row +sc = spark.sparkContext # Load a text file and convert each line to a Row. lines = sc.textFile("examples/src/main/resources/people.txt") @@ -642,11 +670,11 @@ parts = lines.map(lambda l: l.split(",")) people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) # Infer the schema, and register the DataFrame as a table. -schemaPeople = sqlContext.createDataFrame(people) +schemaPeople = spark.createDataFrame(people) schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. -teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # The results of SQL queries are RDDs and support all the normal RDD operations. teenNames = teenagers.map(lambda p: "Name: " + p.name) @@ -673,12 +701,11 @@ a `DataFrame` can be created programmatically with three steps. 2. Create the schema represented by a `StructType` matching the structure of `Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SQLContext`. +by `SparkSession`. For example: {% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +val spark: SparkSession // An existing SparkSession // Create an RDD val people = sc.textFile("examples/src/main/resources/people.txt") @@ -690,26 +717,25 @@ val schemaString = "name age" import org.apache.spark.sql.Row; // Import Spark SQL data types -import org.apache.spark.sql.types.{StructType,StructField,StringType}; +import org.apache.spark.sql.types.{StructType, StructField, StringType}; // Generate the schema based on the string of schema -val schema = - StructType( - schemaString.split(" ").map(fieldName => StructField(fieldName, StringType, true))) +val schema = StructType(schemaString.split(" ").map { fieldName => + StructField(fieldName, StringType, true) +}) // Convert records of the RDD (people) to Rows. val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) // Apply the schema to the RDD. -val peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema) +val peopleDataFrame = spark.createDataFrame(rowRDD, schema) // Creates a temporary view using the DataFrame. peopleDataFrame.createOrReplaceTempView("people") -// SQL statements can be run by using the sql methods provided by sqlContext. -val results = sqlContext.sql("SELECT name FROM people") +// SQL statements can be run by using the sql methods provided by spark. +val results = spark.sql("SELECT name FROM people") -// 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 field index or by field name. results.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -722,13 +748,13 @@ results.map(t => "Name: " + t(0)).collect().foreach(println) When JavaBean classes cannot be defined ahead of time (for example, the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), -a `DataFrame` can be created programmatically with three steps. +a `Dataset` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of `Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SQLContext`. +by `SparkSession`. For example: {% highlight java %} @@ -743,8 +769,8 @@ import org.apache.spark.sql.Row; // Import RowFactory. import org.apache.spark.sql.RowFactory; -// sc is an existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); +SparkSession spark = ...; // An existing SparkSession. +JavaSparkContext sc = spark.sparkContext // Load a text file and convert each line to a JavaBean. JavaRDD people = sc.textFile("examples/src/main/resources/people.txt"); @@ -769,13 +795,13 @@ JavaRDD rowRDD = people.map( }); // Apply the schema to the RDD. -DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema); +Dataset peopleDataFrame = spark.createDataFrame(rowRDD, schema); // Creates a temporary view using the DataFrame. peopleDataFrame.createOrReplaceTempView("people"); // SQL can be run over a temporary view created using DataFrames. -DataFrame results = sqlContext.sql("SELECT name FROM people"); +Dataset results = spark.sql("SELECT name FROM people"); // 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. @@ -799,16 +825,15 @@ a `DataFrame` can be created programmatically with three steps. 1. Create an RDD of tuples or lists from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of tuples or lists in the RDD created in the step 1. -3. Apply the schema to the RDD via `createDataFrame` method provided by `SQLContext`. +3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. For example: {% highlight python %} -# Import SQLContext and data types -from pyspark.sql import SQLContext +# Import SparkSession and data types from pyspark.sql.types import * -# sc is an existing SparkContext. -sqlContext = SQLContext(sc) +# spark is an existing SparkSession. +sc = spark.sparkContext # Load a text file and convert each line to a tuple. lines = sc.textFile("examples/src/main/resources/people.txt") @@ -822,13 +847,13 @@ fields = [StructField(field_name, StringType(), True) for field_name in schemaSt schema = StructType(fields) # Apply the schema to the RDD. -schemaPeople = sqlContext.createDataFrame(people, schema) +schemaPeople = spark.createDataFrame(people, schema) # Creates a temporary view using the DataFrame schemaPeople.createOrReplaceTempView("people") # SQL can be run over DataFrames that have been registered as a table. -results = sqlContext.sql("SELECT name FROM people") +results = spark.sql("SELECT name FROM people") # The results of SQL queries are RDDs and support all the normal RDD operations. names = results.map(lambda p: "Name: " + p.name) @@ -843,9 +868,9 @@ for name in names.collect(): # Data Sources -Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. -A DataFrame can be operated on as normal RDDs and can also be used to create a temporary view. -Registering a DataFrame as a table allows you to run SQL queries over its data. This section +Spark SQL supports operating on a variety of data sources through the DataFrame interface. +A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. +Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section describes the general methods for loading and saving data using the Spark Data Sources and then goes into specific options that are available for the built-in data sources. @@ -858,7 +883,7 @@ In the simplest form, the default data source (`parquet` unless otherwise config
    {% highlight scala %} -val df = sqlContext.read.load("examples/src/main/resources/users.parquet") +val df = spark.read.load("examples/src/main/resources/users.parquet") df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") {% endhighlight %} @@ -868,7 +893,7 @@ df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") {% highlight java %} -DataFrame df = sqlContext.read().load("examples/src/main/resources/users.parquet"); +Dataset df = spark.read().load("examples/src/main/resources/users.parquet"); df.select("name", "favorite_color").write().save("namesAndFavColors.parquet"); {% endhighlight %} @@ -879,7 +904,7 @@ df.select("name", "favorite_color").write().save("namesAndFavColors.parquet"); {% highlight python %} -df = sqlContext.read.load("examples/src/main/resources/users.parquet") +df = spark.read.load("examples/src/main/resources/users.parquet") df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") {% endhighlight %} @@ -889,7 +914,7 @@ df.select("name", "favorite_color").write.save("namesAndFavColors.parquet")
    {% highlight r %} -df <- read.df(sqlContext, "examples/src/main/resources/users.parquet") +df <- read.df("examples/src/main/resources/users.parquet") write.df(select(df, "name", "favorite_color"), "namesAndFavColors.parquet") {% endhighlight %} @@ -901,14 +926,14 @@ write.df(select(df, "name", "favorite_color"), "namesAndFavColors.parquet") You can also manually specify the data source that will be used along with any extra options that you would like to pass to the data source. Data sources are specified by their fully qualified name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short -names (`json`, `parquet`, `jdbc`). DataFrames of any type can be converted into other types +names (`json`, `parquet`, `jdbc`). DataFrames loaded from any data source type can be converted into other types using this syntax.
    {% highlight scala %} -val df = sqlContext.read.format("json").load("examples/src/main/resources/people.json") +val df = spark.read.format("json").load("examples/src/main/resources/people.json") df.select("name", "age").write.format("parquet").save("namesAndAges.parquet") {% endhighlight %} @@ -918,7 +943,7 @@ df.select("name", "age").write.format("parquet").save("namesAndAges.parquet") {% highlight java %} -DataFrame df = sqlContext.read().format("json").load("examples/src/main/resources/people.json"); +Dataset df = spark.read().format("json").load("examples/src/main/resources/people.json"); df.select("name", "age").write().format("parquet").save("namesAndAges.parquet"); {% endhighlight %} @@ -929,7 +954,7 @@ df.select("name", "age").write().format("parquet").save("namesAndAges.parquet"); {% highlight python %} -df = sqlContext.read.load("examples/src/main/resources/people.json", format="json") +df = spark.read.load("examples/src/main/resources/people.json", format="json") df.select("name", "age").write.save("namesAndAges.parquet", format="parquet") {% endhighlight %} @@ -939,7 +964,7 @@ df.select("name", "age").write.save("namesAndAges.parquet", format="parquet") {% highlight r %} -df <- read.df(sqlContext, "examples/src/main/resources/people.json", "json") +df <- read.df("examples/src/main/resources/people.json", "json") write.df(select(df, "name", "age"), "namesAndAges.parquet", "parquet") {% endhighlight %} @@ -956,7 +981,7 @@ file directly with SQL.
    {% highlight scala %} -val df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") +val df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") {% endhighlight %}
    @@ -964,14 +989,14 @@ val df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/user
    {% highlight java %} -DataFrame df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); +Dataset df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); {% endhighlight %}
    {% highlight python %} -df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") +df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") {% endhighlight %}
    @@ -979,7 +1004,7 @@ df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.pa
    {% highlight r %} -df <- sql(sqlContext, "SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") +df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") {% endhighlight %}
    @@ -989,7 +1014,7 @@ df <- sql(sqlContext, "SELECT * FROM parquet.`examples/src/main/resources/users. Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if present. It is important to realize that these save modes do not utilize any locking and are not -atomic. Additionally, when performing a `Overwrite`, the data will be deleted before writing out the +atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the new data.
    Property NameProperty groupspark-submit equivalent
    @@ -1032,12 +1057,13 @@ new data. ### Saving to Persistent Tables -When working with a `HiveContext`, `DataFrames` can also be saved as persistent tables using the -`saveAsTable` command. Unlike the `registerTempTable` command, `saveAsTable` will materialize the -contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables -will still exist even after your Spark program has restarted, as long as you maintain your connection -to the same metastore. A DataFrame for a persistent table can be created by calling the `table` -method on a `SQLContext` with the name of the table. +`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` +command. Notice existing Hive deployment is not necessary to use this feature. Spark will create a +default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, +`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the +Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as +long as you maintain your connection to the same metastore. A DataFrame for a persistent table can +be created by calling the `table` method on a `SparkSession` with the name of the table. By default `saveAsTable` will create a "managed table", meaning that the location of the data will be controlled by the metastore. Managed tables will also have their data deleted automatically @@ -1047,7 +1073,7 @@ when a table is dropped. [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. When writing Parquet files, all columns are automatically converted to be nullable for +of the original data. When writing Parquet files, all columns are automatically converted to be nullable for compatibility reasons. ### Loading Data Programmatically @@ -1059,9 +1085,9 @@ Using the data from the above example:
    {% highlight scala %} -// sqlContext from the previous example is used in this example. +// spark from the previous example is used in this example. // This is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ +import spark.implicits._ val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. @@ -1070,11 +1096,11 @@ people.write.parquet("people.parquet") // 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. -val parquetFile = sqlContext.read.parquet("people.parquet") +val parquetFile = spark.read.parquet("people.parquet") // Parquet files can also be used to create a temporary view and then used in SQL statements. parquetFile.createOrReplaceTempView("parquetFile") -val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +val teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -1083,20 +1109,20 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
    {% highlight java %} -// sqlContext from the previous example is used in this example. +// spark from the previous example is used in this example. -DataFrame schemaPeople = ... // The DataFrame from the previous example. +Dataset schemaPeople = ... // The DataFrame from the previous example. // DataFrames can be saved as Parquet files, maintaining the schema information. schemaPeople.write().parquet("people.parquet"); // 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 = sqlContext.read().parquet("people.parquet"); +Dataset parquetFile = spark.read().parquet("people.parquet"); // Parquet files can also be used to create a temporary view and then used in SQL statements. parquetFile.createOrReplaceTempView("parquetFile"); -DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); +Dataset teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); @@ -1109,7 +1135,7 @@ List teenagerNames = teenagers.javaRDD().map(new Function()
    {% highlight python %} -# sqlContext from the previous example is used in this example. +# spark from the previous example is used in this example. schemaPeople # The DataFrame from the previous example. @@ -1118,11 +1144,11 @@ schemaPeople.write.parquet("people.parquet") # 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. -parquetFile = sqlContext.read.parquet("people.parquet") +parquetFile = spark.read.parquet("people.parquet") # Parquet files can also be used to create a temporary view and then used in SQL statements. parquetFile.createOrReplaceTempView("parquetFile"); -teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) for teenName in teenNames.collect(): print(teenName) @@ -1133,7 +1159,7 @@ for teenName in teenNames.collect():
    {% highlight r %} -# sqlContext from the previous example is used in this example. +# spark from the previous example is used in this example. schemaPeople # The DataFrame from the previous example. @@ -1142,11 +1168,11 @@ write.parquet(schemaPeople, "people.parquet") # 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. -parquetFile <- read.parquet(sqlContext, "people.parquet") +parquetFile <- read.parquet("people.parquet") # Parquet files can also be used to create a temporary view and then used in SQL statements. -registerTempTable(parquetFile, "parquetFile") -teenagers <- sql(sqlContext, "SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +createOrReplaceTempView(parquetFile, "parquetFile") +teenagers <- sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") schema <- structType(structField("name", "string")) teenNames <- dapply(df, function(p) { cbind(paste("Name:", p$name)) }, schema) for (teenName in collect(teenNames)$name) { @@ -1160,7 +1186,7 @@ for (teenName in collect(teenNames)$name) { {% highlight sql %} -CREATE TEMPORARY TABLE parquetTable +CREATE TEMPORARY VIEW parquetTable USING org.apache.spark.sql.parquet OPTIONS ( path "examples/src/main/resources/people.parquet" @@ -1207,7 +1233,7 @@ path {% endhighlight %} -By passing `path/to/table` to either `SQLContext.read.parquet` or `SQLContext.read.load`, Spark SQL +By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL will automatically extract the partitioning information from the paths. Now the schema of the returned DataFrame becomes: @@ -1228,8 +1254,8 @@ can be configured by `spark.sql.sources.partitionColumnTypeInference.enabled`, w `true`. When type inference is disabled, string type will be used for the partitioning columns. Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths -by default. For the above example, if users pass `path/to/table/gender=male` to either -`SQLContext.read.parquet` or `SQLContext.read.load`, `gender` will not be considered as a +by default. For the above example, if users pass `path/to/table/gender=male` to either +`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a partitioning column. If users need to specify the base path that partition discovery should start with, they can set `basePath` in the data source options. For example, when `path/to/table/gender=male` is the path of the data and @@ -1254,9 +1280,9 @@ turned it off by default starting from 1.5.0. You may enable it by
    {% highlight scala %} -// sqlContext from the previous example is used in this example. +// spark from the previous example is used in this example. // This is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ +import spark.implicits._ // Create a simple DataFrame, stored into a partition directory val df1 = sc.makeRDD(1 to 5).map(i => (i, i * 2)).toDF("single", "double") @@ -1268,7 +1294,7 @@ val df2 = sc.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple") df2.write.parquet("data/test_table/key=2") // Read the partitioned table -val df3 = sqlContext.read.option("mergeSchema", "true").parquet("data/test_table") +val df3 = spark.read.option("mergeSchema", "true").parquet("data/test_table") df3.printSchema() // The final schema consists of all 3 columns in the Parquet files together @@ -1285,21 +1311,21 @@ df3.printSchema()
    {% highlight python %} -# sqlContext from the previous example is used in this example. +# spark from the previous example is used in this example. # Create a simple DataFrame, stored into a partition directory -df1 = sqlContext.createDataFrame(sc.parallelize(range(1, 6))\ +df1 = spark.createDataFrame(sc.parallelize(range(1, 6))\ .map(lambda i: Row(single=i, double=i * 2))) df1.write.parquet("data/test_table/key=1") # Create another DataFrame in a new partition directory, # adding a new column and dropping an existing column -df2 = sqlContext.createDataFrame(sc.parallelize(range(6, 11)) +df2 = spark.createDataFrame(sc.parallelize(range(6, 11)) .map(lambda i: Row(single=i, triple=i * 3))) df2.write.parquet("data/test_table/key=2") # Read the partitioned table -df3 = sqlContext.read.option("mergeSchema", "true").parquet("data/test_table") +df3 = spark.read.option("mergeSchema", "true").parquet("data/test_table") df3.printSchema() # The final schema consists of all 3 columns in the Parquet files together @@ -1316,7 +1342,7 @@ df3.printSchema()
    {% highlight r %} -# sqlContext from the previous example is used in this example. +# spark from the previous example is used in this example. # Create a simple DataFrame, stored into a partition directory write.df(df1, "data/test_table/key=1", "parquet", "overwrite") @@ -1326,7 +1352,7 @@ write.df(df1, "data/test_table/key=1", "parquet", "overwrite") write.df(df2, "data/test_table/key=2", "parquet", "overwrite") # Read the partitioned table -df3 <- read.df(sqlContext, "data/test_table", "parquet", mergeSchema="true") +df3 <- read.df("data/test_table", "parquet", mergeSchema="true") printSchema(df3) # The final schema consists of all 3 columns in the Parquet files together @@ -1381,8 +1407,8 @@ metadata.
    {% highlight scala %} -// sqlContext is an existing HiveContext -sqlContext.refreshTable("my_table") +// spark is an existing HiveContext +spark.refreshTable("my_table") {% endhighlight %}
    @@ -1390,8 +1416,8 @@ sqlContext.refreshTable("my_table")
    {% highlight java %} -// sqlContext is an existing HiveContext -sqlContext.refreshTable("my_table") +// spark is an existing HiveContext +spark.refreshTable("my_table") {% endhighlight %}
    @@ -1399,8 +1425,8 @@ sqlContext.refreshTable("my_table")
    {% highlight python %} -# sqlContext is an existing HiveContext -sqlContext.refreshTable("my_table") +# spark is an existing HiveContext +spark.refreshTable("my_table") {% endhighlight %}
    @@ -1417,7 +1443,7 @@ REFRESH TABLE my_table; ### Configuration -Configuration of Parquet can be done using the `setConf` method on `SQLContext` or by running +Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running `SET key=value` commands using SQL.
    @@ -1469,7 +1495,7 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext` - + - + - +
    spark.sql.parquet.mergeSchemafalsefalse

    When true, the Parquet data source merges schemas collected from all data files, otherwise the @@ -1483,8 +1509,8 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`

    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SQLContext.read.json()` on either an RDD of String, +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. +This conversion can be done using `SparkSession.read.json()` on either an RDD of String, or a JSON file. Note that the file that is offered as _a json file_ is not a typical JSON file. Each @@ -1492,13 +1518,12 @@ line must contain a separate, self-contained valid JSON object. As a consequence a regular multi-line JSON file will most often fail. {% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +val spark: SparkSession // An existing SparkSession // A JSON dataset is pointed to by path. // The path can be either a single text file or a directory storing text files. val path = "examples/src/main/resources/people.json" -val people = sqlContext.read.json(path) +val people = spark.read.json(path) // The inferred schema can be visualized using the printSchema() method. people.printSchema() @@ -1509,21 +1534,21 @@ people.printSchema() // Creates a temporary view using the DataFrame people.createOrReplaceTempView("people") -// SQL statements can be run by using the sql methods provided by sqlContext. -val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +// SQL statements can be run by using the sql methods provided by spark. +val teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") // Alternatively, a DataFrame can be created for a JSON dataset represented by // an RDD[String] storing one JSON object per string. val anotherPeopleRDD = sc.parallelize( """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) -val anotherPeople = sqlContext.read.json(anotherPeopleRDD) +val anotherPeople = spark.read.json(anotherPeopleRDD) {% endhighlight %}
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SQLContext.read().json()` on either an RDD of String, +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. +This conversion can be done using `SparkSession.read().json()` on either an RDD of String, or a JSON file. Note that the file that is offered as _a json file_ is not a typical JSON file. Each @@ -1532,11 +1557,11 @@ a regular multi-line JSON file will most often fail. {% highlight java %} // sc is an existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); +SparkSession spark = new org.apache.spark.sql.SparkSession(sc); // A JSON dataset is pointed to by path. // The path can be either a single text file or a directory storing text files. -DataFrame people = sqlContext.read().json("examples/src/main/resources/people.json"); +Dataset people = spark.read().json("examples/src/main/resources/people.json"); // The inferred schema can be visualized using the printSchema() method. people.printSchema(); @@ -1547,34 +1572,32 @@ people.printSchema(); // Creates a temporary view using the DataFrame people.createOrReplaceTempView("people"); -// SQL statements can be run by using the sql methods provided by sqlContext. -DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); +// SQL statements can be run by using the sql methods provided by spark. +Dataset teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // Alternatively, a DataFrame can be created for a JSON dataset represented by // an RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = sc.parallelize(jsonData); -DataFrame anotherPeople = sqlContext.read().json(anotherPeopleRDD); +Dataset anotherPeople = spark.read().json(anotherPeopleRDD); {% endhighlight %}
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SQLContext.read.json` on a JSON file. +This conversion can be done using `SparkSession.read.json` on a JSON file. Note that the file that is offered as _a json file_ is not a typical JSON file. Each line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. {% highlight python %} -# sc is an existing SparkContext. -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) +# spark is an existing SparkSession. # A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. -people = sqlContext.read.json("examples/src/main/resources/people.json") +people = spark.read.json("examples/src/main/resources/people.json") # The inferred schema can be visualized using the printSchema() method. people.printSchema() @@ -1585,14 +1608,14 @@ people.printSchema() # Creates a temporary view using the DataFrame. people.createOrReplaceTempView("people") -# SQL statements can be run by using the sql methods provided by `sqlContext`. -teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +# SQL statements can be run by using the sql methods provided by `spark`. +teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Alternatively, a DataFrame can be created for a JSON dataset represented by # an RDD[String] storing one JSON object per string. anotherPeopleRDD = sc.parallelize([ '{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}']) -anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) +anotherPeople = spark.jsonRDD(anotherPeopleRDD) {% endhighlight %}
    @@ -1606,14 +1629,11 @@ line must contain a separate, self-contained valid JSON object. As a consequence a regular multi-line JSON file will most often fail. {% highlight r %} -# sc is an existing SparkContext. -sqlContext <- sparkRSQL.init(sc) - # A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. path <- "examples/src/main/resources/people.json" # Create a DataFrame from the file(s) pointed to by path -people <- read.json(sqlContext, path) +people <- read.json(path) # The inferred schema can be visualized using the printSchema() method. printSchema(people) @@ -1622,10 +1642,10 @@ printSchema(people) # |-- name: string (nullable = true) # Register this DataFrame as a table. -registerTempTable(people, "people") +createOrReplaceTempView(people, "people") -# SQL statements can be run by using the sql methods provided by `sqlContext`. -teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") +# SQL statements can be run by using the sql methods provided by `spark`. +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") {% endhighlight %}
    @@ -1633,7 +1653,7 @@ teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= {% highlight sql %} -CREATE TEMPORARY TABLE jsonTable +CREATE TEMPORARY VIEW jsonTable USING org.apache.spark.sql.json OPTIONS ( path "examples/src/main/resources/people.json" @@ -1650,14 +1670,14 @@ SELECT * FROM jsonTable ## Hive Tables Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). -However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -Hive support is enabled by adding the `-Phive` and `-Phive-thriftserver` flags to Spark's build. -This command builds a new assembly directory that includes Hive. Note that this Hive assembly directory must also be present -on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries -(SerDes) in order to access data stored in Hive. +However, since Hive has a large number of dependencies, these dependencies are not included in the +default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them +automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as +they will need access to the Hive serialization and deserialization libraries (SerDes) in order to +access data stored in Hive. Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), -`hdfs-site.xml` (for HDFS configuration) file in `conf/`. +and `hdfs-site.xml` (for HDFS configuration) file in `conf/`.
    @@ -1668,7 +1688,7 @@ connectivity to a persistent Hive metastore, support for Hive serdes, and Hive u Users who do not have an existing Hive deployment can still enable Hive support. When not configured by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that +`spark-warehouse` in the current directory that the spark application is started. Note that the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. You may need to grant write privilege to the user who starts the spark application. @@ -1694,7 +1714,7 @@ connectivity to a persistent Hive metastore, support for Hive serdes, and Hive u Users who do not have an existing Hive deployment can still enable Hive support. When not configured by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that +`spark-warehouse` in the current directory that the spark application is started. Note that the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. You may need to grant write privilege to the user who starts the spark application. @@ -1706,7 +1726,7 @@ spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = spark.sql("FROM src SELECT key, value").collect(); +List results = spark.sql("FROM src SELECT key, value").collectAsList(); {% endhighlight %} @@ -1719,14 +1739,13 @@ connectivity to a persistent Hive metastore, support for Hive serdes, and Hive u Users who do not have an existing Hive deployment can still enable Hive support. When not configured by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that +`spark-warehouse` in the current directory that the spark application is started. Note that the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. You may need to grant write privilege to the user who starts the spark application. {% highlight python %} -from pyspark.sql import SparkSession -spark = SparkSession.builder.enableHiveSupport().getOrCreate() +# spark is an existing SparkSession spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") @@ -1740,17 +1759,14 @@ results = spark.sql("FROM src SELECT key, value").collect()
    -When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and +When working with Hive one must construct a `HiveContext`, which inherits from `SparkSession`, and adds support for finding tables in the MetaStore and writing queries using HiveQL. {% highlight r %} -# sc is an existing SparkContext. -sqlContext <- sparkRHive.init(sc) - -sql(sqlContext, "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sql(sqlContext, "LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results <- collect(sql(sqlContext, "FROM src SELECT key, value")) +results <- collect(sql("FROM src SELECT key, value")) {% endhighlight %} @@ -1870,7 +1886,7 @@ the Data Sources API. The following options are supported: The class name of the JDBC driver to use to connect to this URL.
    partitionColumn, lowerBound, upperBound, numPartitions @@ -1882,7 +1898,7 @@ the Data Sources API. The following options are supported: partitioned and returned.
    fetchSize @@ -1896,7 +1912,7 @@ the Data Sources API. The following options are supported:
    {% highlight scala %} -val jdbcDF = sqlContext.read.format("jdbc").options( +val jdbcDF = spark.read.format("jdbc").options( Map("url" -> "jdbc:postgresql:dbserver", "dbtable" -> "schema.tablename")).load() {% endhighlight %} @@ -1911,7 +1927,7 @@ Map options = new HashMap<>(); options.put("url", "jdbc:postgresql:dbserver"); options.put("dbtable", "schema.tablename"); -DataFrame jdbcDF = sqlContext.read().format("jdbc"). options(options).load(); +Dataset jdbcDF = spark.read().format("jdbc"). options(options).load(); {% endhighlight %} @@ -1921,7 +1937,7 @@ DataFrame jdbcDF = sqlContext.read().format("jdbc"). options(options).load(); {% highlight python %} -df = sqlContext.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbtable='schema.tablename').load() +df = spark.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbtable='schema.tablename').load() {% endhighlight %} @@ -1931,7 +1947,7 @@ df = sqlContext.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbta {% highlight r %} -df <- loadDF(sqlContext, source="jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") +df <- loadDF(spark, source="jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") {% endhighlight %} @@ -1941,7 +1957,7 @@ df <- loadDF(sqlContext, source="jdbc", url="jdbc:postgresql:dbserver", dbtable= {% highlight sql %} -CREATE TEMPORARY TABLE jdbcTable +CREATE TEMPORARY VIEW jdbcTable USING org.apache.spark.sql.jdbc OPTIONS ( url "jdbc:postgresql:dbserver", @@ -1966,11 +1982,11 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `dataFrame.cache()`. +Spark SQL can cache tables using an in-memory columnar format by calling `spark.cacheTable("tableName")` or `dataFrame.cache()`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. +memory usage and GC pressure. You can call `spark.uncacheTable("tableName")` to remove the table from memory. -Configuration of in-memory caching can be done using the `setConf` method on `SQLContext` or by running +Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running `SET key=value` commands using SQL. @@ -2109,6 +2125,19 @@ options. # Migration Guide +## Upgrading From Spark SQL 1.6 to 2.0 + + - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. + + - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for + `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed + transformations (e.g. `map`, `filter`, and `groupByKey`) and untyped transformations (e.g. + `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + APIs. Instead, `DataFrame` remains the primary programing abstraction, which is analogous to the + single-node data frame notion in these languages. + ## Upgrading From Spark SQL 1.5 to 1.6 - From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC @@ -2139,7 +2168,7 @@ options. `spark.sql.parquet.mergeSchema` to `true`. - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or access nested values. For example `df['table.column.nestedField']`. However, this means that if - your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). - In-memory columnar storage partition pruning is on by default. It can be disabled by setting `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum From 54001cb129674be9f2459368fb608367f52371c2 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 20 Jun 2016 14:52:28 -0700 Subject: [PATCH 0733/1470] [SPARK-15982][SPARK-16009][SPARK-16007][SQL] Harmonize the behavior of DataFrameReader.text/csv/json/parquet/orc ## What changes were proposed in this pull request? Issues with current reader behavior. - `text()` without args returns an empty DF with no columns -> inconsistent, its expected that text will always return a DF with `value` string field, - `textFile()` without args fails with exception because of the above reason, it expected the DF returned by `text()` to have a `value` field. - `orc()` does not have var args, inconsistent with others - `json(single-arg)` was removed, but that caused source compatibility issues - [SPARK-16009](https://issues.apache.org/jira/browse/SPARK-16009) - user specified schema was not respected when `text/csv/...` were used with no args - [SPARK-16007](https://issues.apache.org/jira/browse/SPARK-16007) The solution I am implementing is to do the following. - For each format, there will be a single argument method, and a vararg method. For json, parquet, csv, text, this means adding json(string), etc.. For orc, this means adding orc(varargs). - Remove the special handling of text(), csv(), etc. that returns empty dataframe with no fields. Rather pass on the empty sequence of paths to the datasource, and let each datasource handle it right. For e.g, text data source, should return empty DF with schema (value: string) - Deduped docs and fixed their formatting. ## How was this patch tested? Added new unit tests for Scala and Java tests Author: Tathagata Das Closes #13727 from tdas/SPARK-15982. (cherry picked from commit b99129cc452defc266f6d357f5baab5f4ff37a36) Signed-off-by: Shixiong Zhu --- .../apache/spark/sql/DataFrameReader.scala | 132 +++++++++---- .../sql/JavaDataFrameReaderWriterSuite.java | 158 +++++++++++++++ .../sql/test/DataFrameReaderWriterSuite.scala | 186 +++++++++++++++--- 3 files changed, 420 insertions(+), 56 deletions(-) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 2ae854d04f564..841503b260c35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -119,13 +119,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def load(): DataFrame = { - val dataSource = - DataSource( - sparkSession, - userSpecifiedSchema = userSpecifiedSchema, - className = source, - options = extraOptions.toMap) - Dataset.ofRows(sparkSession, LogicalRelation(dataSource.resolveRelation())) + load(Seq.empty: _*) // force invocation of `load(...varargs...)` } /** @@ -135,7 +129,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def load(path: String): DataFrame = { - option("path", path).load() + load(Seq(path): _*) // force invocation of `load(...varargs...)` } /** @@ -146,18 +140,15 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { */ @scala.annotation.varargs def load(paths: String*): DataFrame = { - if (paths.isEmpty) { - sparkSession.emptyDataFrame - } else { - sparkSession.baseRelationToDataFrame( - DataSource.apply( - sparkSession, - paths = paths, - userSpecifiedSchema = userSpecifiedSchema, - className = source, - options = extraOptions.toMap).resolveRelation()) - } + sparkSession.baseRelationToDataFrame( + DataSource.apply( + sparkSession, + paths = paths, + userSpecifiedSchema = userSpecifiedSchema, + className = source, + options = extraOptions.toMap).resolveRelation()) } + /** * Construct a [[DataFrame]] representing the database table accessible via JDBC URL * url named table and connection properties. @@ -245,6 +236,17 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { sparkSession.baseRelationToDataFrame(relation) } + /** + * Loads a JSON file (one object per line) and returns the result as a [[DataFrame]]. + * See the documentation on the overloaded `json()` method with varargs for more details. + * + * @since 1.4.0 + */ + def json(path: String): DataFrame = { + // This method ensures that calls that explicit need single argument works, see SPARK-16009 + json(Seq(path): _*) + } + /** * Loads a JSON file (one object per line) and returns the result as a [[DataFrame]]. * @@ -252,6 +254,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * schema in advance, use the version that specifies the schema to avoid the extra scan. * * You can set the following JSON-specific options to deal with non-standard JSON files: + *
      *
    • `primitivesAsString` (default `false`): infers all primitive values as a string type
    • *
    • `prefersDecimal` (default `false`): infers all floating-point values as a decimal * type. If the values do not fit in decimal, then it infers them as doubles.
    • @@ -266,17 +269,17 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records * during parsing.
    • *
        - *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the - * malformed string into a new field configured by `columnNameOfCorruptRecord`. When + *
      • - `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts + * the malformed string into a new field configured by `columnNameOfCorruptRecord`. When * a schema is set by user, it sets `null` for extra fields.
      • - *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      • - `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • - `FAILFAST` : throws an exception when it meets corrupted records.
      • *
      *
    • `columnNameOfCorruptRecord` (default is the value specified in * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • - * - * @since 1.6.0 + *
    + * @since 2.0.0 */ @scala.annotation.varargs def json(paths: String*): DataFrame = format("json").load(paths : _*) @@ -326,6 +329,17 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { parsedOptions))(sparkSession)) } + /** + * Loads a CSV file and returns the result as a [[DataFrame]]. See the documentation on the + * other overloaded `csv()` method for more details. + * + * @since 2.0.0 + */ + def csv(path: String): DataFrame = { + // This method ensures that calls that explicit need single argument works, see SPARK-16009 + csv(Seq(path): _*) + } + /** * Loads a CSV file and returns the result as a [[DataFrame]]. * @@ -334,6 +348,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * specify the schema explicitly using [[schema]]. * * You can set the following CSV-specific options to deal with CSV files: + *
      *
    • `sep` (default `,`): sets the single character as a separator for each * field and value.
    • *
    • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding @@ -370,26 +385,37 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records * during parsing.
    • *
        - *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + *
      • - `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When * a schema is set by user, it sets `null` for extra fields.
      • - *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      • - `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • - `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      *
    - * * @since 2.0.0 */ @scala.annotation.varargs def csv(paths: String*): DataFrame = format("csv").load(paths : _*) /** - * Loads a Parquet file, returning the result as a [[DataFrame]]. This function returns an empty - * [[DataFrame]] if no paths are passed in. + * Loads a Parquet file, returning the result as a [[DataFrame]]. See the documentation + * on the other overloaded `parquet()` method for more details. + * + * @since 2.0.0 + */ + def parquet(path: String): DataFrame = { + // This method ensures that calls that explicit need single argument works, see SPARK-16009 + parquet(Seq(path): _*) + } + + /** + * Loads a Parquet file, returning the result as a [[DataFrame]]. * * You can set the following Parquet-specific option(s) for reading Parquet files: + *
      *
    • `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets * whether we should merge schemas collected from all Parquet part-files. This will override * `spark.sql.parquet.mergeSchema`.
    • - * + *
    * @since 1.4.0 */ @scala.annotation.varargs @@ -404,7 +430,20 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.5.0 * @note Currently, this method can only be used after enabling Hive support. */ - def orc(path: String): DataFrame = format("orc").load(path) + def orc(path: String): DataFrame = { + // This method ensures that calls that explicit need single argument works, see SPARK-16009 + orc(Seq(path): _*) + } + + /** + * Loads an ORC file and returns the result as a [[DataFrame]]. + * + * @param paths input paths + * @since 2.0.0 + * @note Currently, this method can only be used after enabling Hive support. + */ + @scala.annotation.varargs + def orc(paths: String*): DataFrame = format("orc").load(paths: _*) /** * Returns the specified table as a [[DataFrame]]. @@ -417,6 +456,18 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName))) } + /** + * Loads text files and returns a [[DataFrame]] whose schema starts with a string column named + * "value", and followed by partitioned columns if there are any. See the documentation on + * the other overloaded `text()` method for more details. + * + * @since 2.0.0 + */ + def text(path: String): DataFrame = { + // This method ensures that calls that explicit need single argument works, see SPARK-16009 + text(Seq(path): _*) + } + /** * Loads text files and returns a [[DataFrame]] whose schema starts with a string column named * "value", and followed by partitioned columns if there are any. @@ -430,12 +481,22 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * spark.read().text("/path/to/spark/README.md") * }}} * - * @param paths input path + * @param paths input paths * @since 1.6.0 */ @scala.annotation.varargs def text(paths: String*): DataFrame = format("text").load(paths : _*) + /** + * Loads text files and returns a [[Dataset]] of String. See the documentation on the + * other overloaded `textFile()` method for more details. + * @since 2.0.0 + */ + def textFile(path: String): Dataset[String] = { + // This method ensures that calls that explicit need single argument works, see SPARK-16009 + textFile(Seq(path): _*) + } + /** * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset * contains a single string column named "value". @@ -457,6 +518,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { */ @scala.annotation.varargs def textFile(paths: String*): Dataset[String] = { + if (userSpecifiedSchema.nonEmpty) { + throw new AnalysisException("User specified schema not supported with `textFile`") + } text(paths : _*).select("value").as[String](sparkSession.implicits.newStringEncoder) } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java new file mode 100644 index 0000000000000..7babf7573c075 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java @@ -0,0 +1,158 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package test.org.apache.spark.sql; + +import java.io.File; +import java.util.HashMap; + +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.Utils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class JavaDataFrameReaderWriterSuite { + private SparkSession spark = new TestSparkSession(); + private StructType schema = new StructType().add("s", "string"); + private transient String input; + private transient String output; + + @Before + public void setUp() { + input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "input").toString(); + File f = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "output"); + f.delete(); + output = f.toString(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @Test + public void testFormatAPI() { + spark + .read() + .format("org.apache.spark.sql.test") + .load() + .write() + .format("org.apache.spark.sql.test") + .save(); + } + + @Test + public void testOptionsAPI() { + HashMap map = new HashMap(); + map.put("e", "1"); + spark + .read() + .option("a", "1") + .option("b", 1) + .option("c", 1.0) + .option("d", true) + .options(map) + .text() + .write() + .option("a", "1") + .option("b", 1) + .option("c", 1.0) + .option("d", true) + .options(map) + .format("org.apache.spark.sql.test") + .save(); + } + + @Test + public void testSaveModeAPI() { + spark + .range(10) + .write() + .format("org.apache.spark.sql.test") + .mode(SaveMode.ErrorIfExists) + .save(); + } + + @Test + public void testLoadAPI() { + spark.read().format("org.apache.spark.sql.test").load(); + spark.read().format("org.apache.spark.sql.test").load(input); + spark.read().format("org.apache.spark.sql.test").load(input, input, input); + spark.read().format("org.apache.spark.sql.test").load(new String[]{input, input}); + } + + @Test + public void testTextAPI() { + spark.read().text(); + spark.read().text(input); + spark.read().text(input, input, input); + spark.read().text(new String[]{input, input}) + .write().text(output); + } + + @Test + public void testTextFileAPI() { + spark.read().textFile(); + spark.read().textFile(input); + spark.read().textFile(input, input, input); + spark.read().textFile(new String[]{input, input}); + } + + @Test + public void testCsvAPI() { + spark.read().schema(schema).csv(); + spark.read().schema(schema).csv(input); + spark.read().schema(schema).csv(input, input, input); + spark.read().schema(schema).csv(new String[]{input, input}) + .write().csv(output); + } + + @Test + public void testJsonAPI() { + spark.read().schema(schema).json(); + spark.read().schema(schema).json(input); + spark.read().schema(schema).json(input, input, input); + spark.read().schema(schema).json(new String[]{input, input}) + .write().json(output); + } + + @Test + public void testParquetAPI() { + spark.read().schema(schema).parquet(); + spark.read().schema(schema).parquet(input); + spark.read().schema(schema).parquet(input, input, input); + spark.read().schema(schema).parquet(new String[] { input, input }) + .write().parquet(output); + } + + /** + * This only tests whether API compiles, but does not run it as orc() + * cannot be run without Hive classes. + */ + public void testOrcAPI() { + spark.read().schema(schema).orc(); + spark.read().schema(schema).orc(input); + spark.read().schema(schema).orc(input, input, input); + spark.read().schema(schema).orc(new String[]{input, input}) + .write().orc(output); + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 98e57b38044f2..3fa3864bc9690 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql.test +import java.io.File + +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -79,10 +83,19 @@ class DefaultSource } -class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext { +class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { + - private def newMetadataDir = - Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + private val userSchema = new StructType().add("s", StringType) + private val textSchema = new StructType().add("value", StringType) + private val data = Seq("1", "2", "3") + private val dir = Utils.createTempDir(namePrefix = "input").getCanonicalPath + private implicit var enc: Encoder[String] = _ + + before { + enc = spark.implicits.newStringEncoder + Utils.deleteRecursively(new File(dir)) + } test("writeStream cannot be called on non-streaming datasets") { val e = intercept[AnalysisException] { @@ -157,24 +170,6 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext { assert(LastOptions.saveMode === SaveMode.ErrorIfExists) } - test("paths") { - val df = spark.read - .format("org.apache.spark.sql.test") - .option("checkpointLocation", newMetadataDir) - .load("/test") - - assert(LastOptions.parameters("path") == "/test") - - LastOptions.clear() - - df.write - .format("org.apache.spark.sql.test") - .option("checkpointLocation", newMetadataDir) - .save("/test") - - assert(LastOptions.parameters("path") == "/test") - } - test("test different data types for options") { val df = spark.read .format("org.apache.spark.sql.test") @@ -193,7 +188,6 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext { .option("intOpt", 56) .option("boolOpt", false) .option("doubleOpt", 6.7) - .option("checkpointLocation", newMetadataDir) .save("/test") assert(LastOptions.parameters("intOpt") == "56") @@ -228,4 +222,152 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext { } } } + + test("load API") { + spark.read.format("org.apache.spark.sql.test").load() + spark.read.format("org.apache.spark.sql.test").load(dir) + spark.read.format("org.apache.spark.sql.test").load(dir, dir, dir) + spark.read.format("org.apache.spark.sql.test").load(Seq(dir, dir): _*) + Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) + } + + test("text - API and behavior regarding schema") { + // Writer + spark.createDataset(data).write.mode(SaveMode.Overwrite).text(dir) + testRead(spark.read.text(dir), data, textSchema) + + // Reader, without user specified schema + testRead(spark.read.text(), Seq.empty, textSchema) + testRead(spark.read.text(dir, dir, dir), data ++ data ++ data, textSchema) + testRead(spark.read.text(Seq(dir, dir): _*), data ++ data, textSchema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.text).get, data, textSchema) + + // Reader, with user specified schema, should just apply user schema on the file data + testRead(spark.read.schema(userSchema).text(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).text(dir), data, userSchema) + testRead(spark.read.schema(userSchema).text(dir, dir), data ++ data, userSchema) + testRead(spark.read.schema(userSchema).text(Seq(dir, dir): _*), data ++ data, userSchema) + } + + test("textFile - API and behavior regarding schema") { + spark.createDataset(data).write.mode(SaveMode.Overwrite).text(dir) + + // Reader, without user specified schema + testRead(spark.read.textFile().toDF(), Seq.empty, textSchema) + testRead(spark.read.textFile(dir).toDF(), data, textSchema) + testRead(spark.read.textFile(dir, dir).toDF(), data ++ data, textSchema) + testRead(spark.read.textFile(Seq(dir, dir): _*).toDF(), data ++ data, textSchema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.text).get, data, textSchema) + + // Reader, with user specified schema, should just apply user schema on the file data + val e = intercept[AnalysisException] { spark.read.schema(userSchema).textFile() } + assert(e.getMessage.toLowerCase.contains("user specified schema not supported")) + intercept[AnalysisException] { spark.read.schema(userSchema).textFile(dir) } + intercept[AnalysisException] { spark.read.schema(userSchema).textFile(dir, dir) } + intercept[AnalysisException] { spark.read.schema(userSchema).textFile(Seq(dir, dir): _*) } + } + + test("csv - API and behavior regarding schema") { + // Writer + spark.createDataset(data).toDF("str").write.mode(SaveMode.Overwrite).csv(dir) + val df = spark.read.csv(dir) + checkAnswer(df, spark.createDataset(data).toDF()) + val schema = df.schema + + // Reader, without user specified schema + intercept[IllegalArgumentException] { + testRead(spark.read.csv(), Seq.empty, schema) + } + testRead(spark.read.csv(dir), data, schema) + testRead(spark.read.csv(dir, dir), data ++ data, schema) + testRead(spark.read.csv(Seq(dir, dir): _*), data ++ data, schema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.csv).get, data, schema) + + // Reader, with user specified schema, should just apply user schema on the file data + testRead(spark.read.schema(userSchema).csv(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).csv(dir), data, userSchema) + testRead(spark.read.schema(userSchema).csv(dir, dir), data ++ data, userSchema) + testRead(spark.read.schema(userSchema).csv(Seq(dir, dir): _*), data ++ data, userSchema) + } + + test("json - API and behavior regarding schema") { + // Writer + spark.createDataset(data).toDF("str").write.mode(SaveMode.Overwrite).json(dir) + val df = spark.read.json(dir) + checkAnswer(df, spark.createDataset(data).toDF()) + val schema = df.schema + + // Reader, without user specified schema + intercept[AnalysisException] { + testRead(spark.read.json(), Seq.empty, schema) + } + testRead(spark.read.json(dir), data, schema) + testRead(spark.read.json(dir, dir), data ++ data, schema) + testRead(spark.read.json(Seq(dir, dir): _*), data ++ data, schema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.json).get, data, schema) + + // Reader, with user specified schema, data should be nulls as schema in file different + // from user schema + val expData = Seq[String](null, null, null) + testRead(spark.read.schema(userSchema).json(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).json(dir), expData, userSchema) + testRead(spark.read.schema(userSchema).json(dir, dir), expData ++ expData, userSchema) + testRead(spark.read.schema(userSchema).json(Seq(dir, dir): _*), expData ++ expData, userSchema) + } + + test("parquet - API and behavior regarding schema") { + // Writer + spark.createDataset(data).toDF("str").write.mode(SaveMode.Overwrite).parquet(dir) + val df = spark.read.parquet(dir) + checkAnswer(df, spark.createDataset(data).toDF()) + val schema = df.schema + + // Reader, without user specified schema + intercept[AnalysisException] { + testRead(spark.read.parquet(), Seq.empty, schema) + } + testRead(spark.read.parquet(dir), data, schema) + testRead(spark.read.parquet(dir, dir), data ++ data, schema) + testRead(spark.read.parquet(Seq(dir, dir): _*), data ++ data, schema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.parquet).get, data, schema) + + // Reader, with user specified schema, data should be nulls as schema in file different + // from user schema + val expData = Seq[String](null, null, null) + testRead(spark.read.schema(userSchema).parquet(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).parquet(dir), expData, userSchema) + testRead(spark.read.schema(userSchema).parquet(dir, dir), expData ++ expData, userSchema) + testRead( + spark.read.schema(userSchema).parquet(Seq(dir, dir): _*), expData ++ expData, userSchema) + } + + /** + * This only tests whether API compiles, but does not run it as orc() + * cannot be run without Hive classes. + */ + ignore("orc - API") { + // Reader, with user specified schema + // Refer to csv-specific test suites for behavior without user specified schema + spark.read.schema(userSchema).orc() + spark.read.schema(userSchema).orc(dir) + spark.read.schema(userSchema).orc(dir, dir, dir) + spark.read.schema(userSchema).orc(Seq(dir, dir): _*) + Option(dir).map(spark.read.schema(userSchema).orc) + + // Writer + spark.range(10).write.orc(dir) + } + + private def testRead( + df: => DataFrame, + expectedResult: Seq[String], + expectedSchema: StructType): Unit = { + checkAnswer(df, spark.createDataset(expectedResult).toDF()) + assert(df.schema === expectedSchema) + } } From b4066354141b933cdfdfdf266c6d4ff21338dcdf Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 20 Jun 2016 15:12:40 -0700 Subject: [PATCH 0734/1470] [SPARK-16061][SQL][MINOR] The property "spark.streaming.stateStore.maintenanceInterval" should be renamed to "spark.sql.streaming.stateStore.maintenanceInterval" ## What changes were proposed in this pull request? The property spark.streaming.stateStore.maintenanceInterval should be renamed and harmonized with other properties related to Structured Streaming like spark.sql.streaming.stateStore.minDeltasForSnapshot. ## How was this patch tested? Existing unit tests. Author: Kousuke Saruta Closes #13777 from sarutak/SPARK-16061. (cherry picked from commit 6daa8cf1a642a669cd3a0305036c4390e4336a73) Signed-off-by: Reynold Xin --- .../apache/spark/sql/execution/streaming/state/StateStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 9948292470ff8..066765324ac94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -115,7 +115,7 @@ case class KeyRemoved(key: UnsafeRow) extends StoreUpdate */ private[sql] object StateStore extends Logging { - val MAINTENANCE_INTERVAL_CONFIG = "spark.streaming.stateStore.maintenanceInterval" + val MAINTENANCE_INTERVAL_CONFIG = "spark.sql.streaming.stateStore.maintenanceInterval" val MAINTENANCE_INTERVAL_DEFAULT_SECS = 60 private val loadedProviders = new mutable.HashMap[StateStoreId, StateStoreProvider]() From c7006538a88bee85e0292bc9564ae8bfdf734ed6 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Mon, 20 Jun 2016 16:28:11 -0700 Subject: [PATCH 0735/1470] [SPARK-16079][PYSPARK][ML] Added missing import for DecisionTreeRegressionModel used in GBTClassificationModel ## What changes were proposed in this pull request? Fixed missing import for DecisionTreeRegressionModel used in GBTClassificationModel trees method. ## How was this patch tested? Local tests Author: Bryan Cutler Closes #13787 from BryanCutler/pyspark-GBTClassificationModel-import-SPARK-16079. (cherry picked from commit a42bf555326b75c8251be77db68105c29e8c95c4) Signed-off-by: Xiangrui Meng --- python/pyspark/ml/classification.py | 6 ++++-- python/pyspark/ml/regression.py | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 121b9262dd9de..a3cd91790c42e 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -21,8 +21,8 @@ from pyspark import since, keyword_only from pyspark.ml import Estimator, Model from pyspark.ml.param.shared import * -from pyspark.ml.regression import ( - RandomForestParams, TreeEnsembleParams, DecisionTreeModel, TreeEnsembleModels) +from pyspark.ml.regression import DecisionTreeModel, DecisionTreeRegressionModel, \ + RandomForestParams, TreeEnsembleModels, TreeEnsembleParams from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams from pyspark.ml.wrapper import JavaWrapper @@ -798,6 +798,8 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol True >>> model.treeWeights == model2.treeWeights True + >>> model.trees + [DecisionTreeRegressionModel (uid=...) of depth..., DecisionTreeRegressionModel...] .. versionadded:: 1.4.0 """ diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index db31993f0fb70..8d2378d51fb7e 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -994,6 +994,8 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, True >>> model.treeWeights == model2.treeWeights True + >>> model.trees + [DecisionTreeRegressionModel (uid=...) of depth..., DecisionTreeRegressionModel...] .. versionadded:: 1.4.0 """ From f573176902ebff0fd6a2f572c94a2cca3e057b72 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Mon, 20 Jun 2016 19:36:51 -0700 Subject: [PATCH 0736/1470] remove duplicated docs in dapply ## What changes were proposed in this pull request? Removed unnecessary duplicated documentation in dapply and dapplyCollect. In this pull request I created separate R docs for dapply and dapplyCollect - kept dapply's documentation separate from dapplyCollect's and referred from one to another via a link. ## How was this patch tested? Existing test cases. Author: Narine Kokhlikyan Closes #13790 from NarineK/dapply-docs-fix. (cherry picked from commit e2b7eba87cdf67fa737c32f5f6ca075445ff28cb) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 4 +++- R/pkg/R/generics.R | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ecdcd6e29f190..b3f2dd82ff549 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1250,6 +1250,7 @@ dapplyInternal <- function(x, func, schema) { #' @family SparkDataFrame functions #' @rdname dapply #' @name dapply +#' @seealso \link{dapplyCollect} #' @export #' @examples #' \dontrun{ @@ -1294,8 +1295,9 @@ setMethod("dapply", #' to each partition will be passed. #' The output of func should be a data.frame. #' @family SparkDataFrame functions -#' @rdname dapply +#' @rdname dapplyCollect #' @name dapplyCollect +#' @seealso \link{dapply} #' @export #' @examples #' \dontrun{ diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index f6b9276d86f39..3fb6370497701 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -457,7 +457,7 @@ setGeneric("createOrReplaceTempView", #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) -#' @rdname dapply +#' @rdname dapplyCollect #' @export setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") }) From 087bd2799366f4914d248e9b1f0fb921adbbdb43 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 20 Jun 2016 20:50:30 -0700 Subject: [PATCH 0737/1470] [SPARK-16086] [SQL] fix Python UDF without arguments (for 1.6) Fix the bug for Python UDF that does not have any arguments. Added regression tests. Author: Davies Liu Closes #13793 from davies/fix_no_arguments. (cherry picked from commit abe36c53d126bb580e408a45245fd8e81806869c) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 5 +++++ python/pyspark/sql/types.py | 9 +++------ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index c631ad8a4618d..ecd1a0563e959 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -318,6 +318,11 @@ def test_multiple_udfs(self): [row] = self.spark.sql("SELECT double(add(1, 2)), add(double(2), 1)").collect() self.assertEqual(tuple(row), (6, 5)) + def test_udf_without_arguments(self): + self.sqlCtx.registerFunction("foo", lambda: "bar") + [row] = self.sqlCtx.sql("SELECT foo()").collect() + self.assertEqual(row[0], "bar") + def test_udf_with_array_type(self): d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index bb2b95404af7a..f0b56be8da438 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1401,11 +1401,7 @@ def __new__(self, *args, **kwargs): if args and kwargs: raise ValueError("Can not use both args " "and kwargs to create Row") - if args: - # create row class or objects - return tuple.__new__(self, args) - - elif kwargs: + if kwargs: # create row objects names = sorted(kwargs.keys()) row = tuple.__new__(self, [kwargs[n] for n in names]) @@ -1413,7 +1409,8 @@ def __new__(self, *args, **kwargs): return row else: - raise ValueError("No args or kwargs") + # create row class or objects + return tuple.__new__(self, args) def asDict(self, recursive=False): """ From 10c476fc8f4780e487d8ada626f6924866f5711f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Jun 2016 21:09:39 -0700 Subject: [PATCH 0738/1470] [SPARK-15294][R] Add `pivot` to SparkR ## What changes were proposed in this pull request? This PR adds `pivot` function to SparkR for API parity. Since this PR is based on https://github.com/apache/spark/pull/13295 , mhnatiuk should be credited for the work he did. ## How was this patch tested? Pass the Jenkins tests (including new testcase.) Author: Dongjoon Hyun Closes #13786 from dongjoon-hyun/SPARK-15294. (cherry picked from commit 217db56ba11fcdf9e3a81946667d1d99ad7344ee) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/generics.R | 4 +++ R/pkg/R/group.R | 43 +++++++++++++++++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 25 +++++++++++++ 4 files changed, 73 insertions(+) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 45663f4c2c70a..ea42888eaebfa 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -294,6 +294,7 @@ exportMethods("%in%", exportClasses("GroupedData") exportMethods("agg") +exportMethods("pivot") export("as.DataFrame", "cacheTable", diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 3fb6370497701..c307de7c07633 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -160,6 +160,10 @@ setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) # @export setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")}) +# @rdname pivot +# @export +setGeneric("pivot", function(x, colname, values = list()) { standardGeneric("pivot") }) + # @rdname reduce # @export setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 51e151623cf0e..0687f14adf7b9 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -134,6 +134,49 @@ methods <- c("avg", "max", "mean", "min", "sum") # These are not exposed on GroupedData: "kurtosis", "skewness", "stddev", "stddev_samp", "stddev_pop", # "variance", "var_samp", "var_pop" +#' Pivot a column of the GroupedData and perform the specified aggregation. +#' +#' Pivot a column of the GroupedData and perform the specified aggregation. +#' There are two versions of pivot function: one that requires the caller to specify the list +#' of distinct values to pivot on, and one that does not. The latter is more concise but less +#' efficient, because Spark needs to first compute the list of distinct values internally. +#' +#' @param x a GroupedData object +#' @param colname A column name +#' @param values A value or a list/vector of distinct values for the output columns. +#' @return GroupedData object +#' @rdname pivot +#' @name pivot +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(data.frame( +#' earnings = c(10000, 10000, 11000, 15000, 12000, 20000, 21000, 22000), +#' course = c("R", "Python", "R", "Python", "R", "Python", "R", "Python"), +#' period = c("1H", "1H", "2H", "2H", "1H", "1H", "2H", "2H"), +#' year = c(2015, 2015, 2015, 2015, 2016, 2016, 2016, 2016) +#' )) +#' group_sum <- sum(pivot(groupBy(df, "year"), "course"), "earnings") +#' group_min <- min(pivot(groupBy(df, "year"), "course", "R"), "earnings") +#' group_max <- max(pivot(groupBy(df, "year"), "course", c("Python", "R")), "earnings") +#' group_mean <- mean(pivot(groupBy(df, "year"), "course", list("Python", "R")), "earnings") +#' } +#' @note pivot since 2.0.0 +setMethod("pivot", + signature(x = "GroupedData", colname = "character"), + function(x, colname, values = list()){ + stopifnot(length(colname) == 1) + if (length(values) == 0) { + result <- callJMethod(x@sgd, "pivot", colname) + } else { + if (length(values) > length(unique(values))) { + stop("Values are not unique") + } + result <- callJMethod(x@sgd, "pivot", colname, as.list(values)) + } + groupedData(result) + }) + createMethod <- function(name) { setMethod(name, signature(x = "GroupedData"), diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index d53c40d42382b..7c192fb5a0afa 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1398,6 +1398,31 @@ test_that("group by, agg functions", { unlink(jsonPath3) }) +test_that("pivot GroupedData column", { + df <- createDataFrame(data.frame( + earnings = c(10000, 10000, 11000, 15000, 12000, 20000, 21000, 22000), + course = c("R", "Python", "R", "Python", "R", "Python", "R", "Python"), + year = c(2013, 2013, 2014, 2014, 2015, 2015, 2016, 2016) + )) + sum1 <- collect(sum(pivot(groupBy(df, "year"), "course"), "earnings")) + sum2 <- collect(sum(pivot(groupBy(df, "year"), "course", c("Python", "R")), "earnings")) + sum3 <- collect(sum(pivot(groupBy(df, "year"), "course", list("Python", "R")), "earnings")) + sum4 <- collect(sum(pivot(groupBy(df, "year"), "course", "R"), "earnings")) + + correct_answer <- data.frame( + year = c(2013, 2014, 2015, 2016), + Python = c(10000, 15000, 20000, 22000), + R = c(10000, 11000, 12000, 21000) + ) + expect_equal(sum1, correct_answer) + expect_equal(sum2, correct_answer) + expect_equal(sum3, correct_answer) + expect_equal(sum4, correct_answer[, c("year", "R")]) + + expect_error(collect(sum(pivot(groupBy(df, "year"), "course", c("R", "R")), "earnings"))) + expect_error(collect(sum(pivot(groupBy(df, "year"), "course", list("R", "R")), "earnings"))) +}) + test_that("arrange() and orderBy() on a DataFrame", { df <- read.json(jsonPath) sorted <- arrange(df, df$age) From 603424c161e9be670ee8461053225364cc700515 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 20 Jun 2016 21:46:12 -0700 Subject: [PATCH 0739/1470] [SPARK-13792][SQL] Limit logging of bad records in CSV data source ## What changes were proposed in this pull request? This pull request adds a new option (maxMalformedLogPerPartition) in CSV reader to limit the maximum of logging message Spark generates per partition for malformed records. The error log looks something like ``` 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: More than 10 malformed records have been found on this partition. Malformed records from now on will not be logged. ``` Closes #12173 ## How was this patch tested? Manually tested. Author: Reynold Xin Closes #13795 from rxin/SPARK-13792. (cherry picked from commit c775bf09e0c3540f76de3f15d3fd35112a4912c1) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 4 ++ .../apache/spark/sql/DataFrameReader.scala | 2 + .../datasources/csv/CSVFileFormat.scala | 9 +++- .../datasources/csv/CSVOptions.scala | 2 + .../datasources/csv/CSVRelation.scala | 42 ++++++++++++------- 5 files changed, 44 insertions(+), 15 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 72fd184d580cc..89506ca02f273 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -392,6 +392,10 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param maxCharsPerColumn: defines the maximum number of characters allowed for any given value being read. If None is set, it uses the default value, ``1000000``. + :param maxMalformedLogPerPartition: sets the maximum number of malformed rows Spark will + log for each partition. Malformed records beyond this + number will be ignored. If None is set, it + uses the default value, ``10``. :param mode: allows a mode for dealing with corrupt records during parsing. If None is set, it uses the default value, ``PERMISSIVE``. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 841503b260c35..35ba9c50790e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -382,6 +382,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * a record can have. *
  • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed * for any given value being read.
  • + *
  • `maxMalformedLogPerPartition` (default `10`): sets the maximum number of malformed rows + * Spark will log for each partition. Malformed records beyond this number will be ignored.
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records * during parsing.
  • *
      diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index be52de8e40140..12e19f955caa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -120,7 +120,14 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { val tokenizedIterator = new BulkCsvReader(lineIterator, csvOptions, headers) val parser = CSVRelation.csvParser(dataSchema, requiredSchema.fieldNames, csvOptions) - tokenizedIterator.flatMap(parser(_).toSeq) + var numMalformedRecords = 0 + tokenizedIterator.flatMap { recordTokens => + val row = parser(recordTokens, numMalformedRecords) + if (row.isEmpty) { + numMalformedRecords += 1 + } + row + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 9f4ce8358b045..581eda7e09a3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -113,6 +113,8 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str val escapeQuotes = getBool("escapeQuotes", true) + val maxMalformedLogPerPartition = getInt("maxMalformedLogPerPartition", 10) + val inputBufferSize = 128 val isCommentSet = this.comment != '\u0000' diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index d72c8b9ac2e7c..083ac3350ef02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -50,10 +50,19 @@ object CSVRelation extends Logging { } } + /** + * Returns a function that parses a single CSV record (in the form of an array of strings in which + * each element represents a column) and turns it into either one resulting row or no row (if the + * the record is malformed). + * + * The 2nd argument in the returned function represents the total number of malformed rows + * observed so far. + */ + // This is pretty convoluted and we should probably rewrite the entire CSV parsing soon. def csvParser( schema: StructType, requiredColumns: Array[String], - params: CSVOptions): Array[String] => Option[InternalRow] = { + params: CSVOptions): (Array[String], Int) => Option[InternalRow] = { val schemaFields = schema.fields val requiredFields = StructType(requiredColumns.map(schema(_))).fields val safeRequiredFields = if (params.dropMalformed) { @@ -72,9 +81,16 @@ object CSVRelation extends Logging { val requiredSize = requiredFields.length val row = new GenericMutableRow(requiredSize) - (tokens: Array[String]) => { + (tokens: Array[String], numMalformedRows) => { if (params.dropMalformed && schemaFields.length != tokens.length) { - logWarning(s"Dropping malformed line: ${tokens.mkString(params.delimiter.toString)}") + if (numMalformedRows < params.maxMalformedLogPerPartition) { + logWarning(s"Dropping malformed line: ${tokens.mkString(params.delimiter.toString)}") + } + if (numMalformedRows == params.maxMalformedLogPerPartition - 1) { + logWarning( + s"More than ${params.maxMalformedLogPerPartition} malformed records have been " + + "found on this partition. Malformed records from now on will not be logged.") + } None } else if (params.failFast && schemaFields.length != tokens.length) { throw new RuntimeException(s"Malformed line in FAILFAST mode: " + @@ -109,23 +125,21 @@ object CSVRelation extends Logging { Some(row) } catch { case NonFatal(e) if params.dropMalformed => - logWarning("Parse exception. " + - s"Dropping malformed line: ${tokens.mkString(params.delimiter.toString)}") + if (numMalformedRows < params.maxMalformedLogPerPartition) { + logWarning("Parse exception. " + + s"Dropping malformed line: ${tokens.mkString(params.delimiter.toString)}") + } + if (numMalformedRows == params.maxMalformedLogPerPartition - 1) { + logWarning( + s"More than ${params.maxMalformedLogPerPartition} malformed records have been " + + "found on this partition. Malformed records from now on will not be logged.") + } None } } } } - def parseCsv( - tokenizedRDD: RDD[Array[String]], - schema: StructType, - requiredColumns: Array[String], - options: CSVOptions): RDD[InternalRow] = { - val parser = csvParser(schema, requiredColumns, options) - tokenizedRDD.flatMap(parser(_).toSeq) - } - // Skips the header line of each file if the `header` option is set to true. def dropHeaderLine( file: PartitionedFile, lines: Iterator[String], csvOptions: CSVOptions): Unit = { From b998c33c0d38f8f724d8846bc8e919ec8b92012e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 20 Jun 2016 21:49:33 -0700 Subject: [PATCH 0740/1470] [SPARK-16056][SPARK-16057][SPARK-16058][SQL] Fix Multiple Bugs in Column Partitioning in JDBC Source #### What changes were proposed in this pull request? This PR is to fix the following bugs: **Issue 1: Wrong Results when lowerBound is larger than upperBound in Column Partitioning** ```scala spark.read.jdbc( url = urlWithUserAndPass, table = "TEST.seq", columnName = "id", lowerBound = 4, upperBound = 0, numPartitions = 3, connectionProperties = new Properties) ``` **Before code changes:** The returned results are wrong and the generated partitions are wrong: ``` Part 0 id < 3 or id is null Part 1 id >= 3 AND id < 2 Part 2 id >= 2 ``` **After code changes:** Issue an `IllegalArgumentException` exception: ``` Operation not allowed: the lower bound of partitioning column is larger than the upper bound. lowerBound: 5; higherBound: 1 ``` **Issue 2: numPartitions is more than the number of key values between upper and lower bounds** ```scala spark.read.jdbc( url = urlWithUserAndPass, table = "TEST.seq", columnName = "id", lowerBound = 1, upperBound = 5, numPartitions = 10, connectionProperties = new Properties) ``` **Before code changes:** Returned correct results but the generated partitions are very inefficient, like: ``` Partition 0: id < 1 or id is null Partition 1: id >= 1 AND id < 1 Partition 2: id >= 1 AND id < 1 Partition 3: id >= 1 AND id < 1 Partition 4: id >= 1 AND id < 1 Partition 5: id >= 1 AND id < 1 Partition 6: id >= 1 AND id < 1 Partition 7: id >= 1 AND id < 1 Partition 8: id >= 1 AND id < 1 Partition 9: id >= 1 ``` **After code changes:** Adjust `numPartitions` and can return the correct answers: ``` Partition 0: id < 2 or id is null Partition 1: id >= 2 AND id < 3 Partition 2: id >= 3 AND id < 4 Partition 3: id >= 4 ``` **Issue 3: java.lang.ArithmeticException when numPartitions is zero** ```Scala spark.read.jdbc( url = urlWithUserAndPass, table = "TEST.seq", columnName = "id", lowerBound = 0, upperBound = 4, numPartitions = 0, connectionProperties = new Properties) ``` **Before code changes:** Got the following exception: ``` java.lang.ArithmeticException: / by zero ``` **After code changes:** Able to return a correct answer by disabling column partitioning when numPartitions is equal to or less than zero #### How was this patch tested? Added test cases to verify the results Author: gatorsmile Closes #13773 from gatorsmile/jdbcPartitioning. (cherry picked from commit d9a3a2a0bec504d17d3b94104d449ee3bd850120) Signed-off-by: Reynold Xin --- .../datasources/jdbc/JDBCRelation.scala | 48 +++++++++----- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 65 +++++++++++++++++++ 2 files changed, 98 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 233b7891d664c..11613dd912eca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -21,6 +21,7 @@ import java.util.Properties import scala.collection.mutable.ArrayBuffer +import org.apache.spark.internal.Logging import org.apache.spark.Partition import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession, SQLContext} @@ -36,7 +37,7 @@ private[sql] case class JDBCPartitioningInfo( upperBound: Long, numPartitions: Int) -private[sql] object JDBCRelation { +private[sql] object JDBCRelation extends Logging { /** * Given a partitioning schematic (a column of integral type, a number of * partitions, and upper and lower bounds on the column's value), generate @@ -52,29 +53,46 @@ private[sql] object JDBCRelation { * @return an array of partitions with where clause for each partition */ def columnPartition(partitioning: JDBCPartitioningInfo): Array[Partition] = { - if (partitioning == null) return Array[Partition](JDBCPartition(null, 0)) + if (partitioning == null || partitioning.numPartitions <= 1 || + partitioning.lowerBound == partitioning.upperBound) { + return Array[Partition](JDBCPartition(null, 0)) + } - val numPartitions = partitioning.numPartitions - val column = partitioning.column - if (numPartitions == 1) return Array[Partition](JDBCPartition(null, 0)) + val lowerBound = partitioning.lowerBound + val upperBound = partitioning.upperBound + require (lowerBound <= upperBound, + "Operation not allowed: the lower bound of partitioning column is larger than the upper " + + s"bound. Lower bound: $lowerBound; Upper bound: $upperBound") + + val numPartitions = + if ((upperBound - lowerBound) >= partitioning.numPartitions) { + partitioning.numPartitions + } else { + logWarning("The number of partitions is reduced because the specified number of " + + "partitions is less than the difference between upper bound and lower bound. " + + s"Updated number of partitions: ${upperBound - lowerBound}; Input number of " + + s"partitions: ${partitioning.numPartitions}; Lower bound: $lowerBound; " + + s"Upper bound: $upperBound.") + upperBound - lowerBound + } // Overflow and silliness can happen if you subtract then divide. // Here we get a little roundoff, but that's (hopefully) OK. - val stride: Long = (partitioning.upperBound / numPartitions - - partitioning.lowerBound / numPartitions) + val stride: Long = upperBound / numPartitions - lowerBound / numPartitions + val column = partitioning.column var i: Int = 0 - var currentValue: Long = partitioning.lowerBound + var currentValue: Long = lowerBound var ans = new ArrayBuffer[Partition]() while (i < numPartitions) { - val lowerBound = if (i != 0) s"$column >= $currentValue" else null + val lBound = if (i != 0) s"$column >= $currentValue" else null currentValue += stride - val upperBound = if (i != numPartitions - 1) s"$column < $currentValue" else null + val uBound = if (i != numPartitions - 1) s"$column < $currentValue" else null val whereClause = - if (upperBound == null) { - lowerBound - } else if (lowerBound == null) { - s"$upperBound or $column is null" + if (uBound == null) { + lBound + } else if (lBound == null) { + s"$uBound or $column is null" } else { - s"$lowerBound AND $upperBound" + s"$lBound AND $uBound" } ans += JDBCPartition(whereClause, i) i = i + 1 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 d6ec40c18be2b..fd6671a39b6ef 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 @@ -184,6 +184,16 @@ class JDBCSuite extends SparkFunSuite "insert into test.emp values ('kathy', null, null)").executeUpdate() conn.commit() + conn.prepareStatement( + "create table test.seq(id INTEGER)").executeUpdate() + (0 to 6).foreach { value => + conn.prepareStatement( + s"insert into test.seq values ($value)").executeUpdate() + } + conn.prepareStatement( + "insert into test.seq values (null)").executeUpdate() + conn.commit() + sql( s""" |CREATE TEMPORARY TABLE nullparts @@ -373,6 +383,61 @@ class JDBCSuite extends SparkFunSuite .collect().length === 4) } + test("Partitioning on column where numPartitions is zero") { + val res = spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 0, + upperBound = 4, + numPartitions = 0, + connectionProperties = new Properties + ) + assert(res.count() === 8) + } + + test("Partitioning on column where numPartitions are more than the number of total rows") { + val res = spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 1, + upperBound = 5, + numPartitions = 10, + connectionProperties = new Properties + ) + assert(res.count() === 8) + } + + test("Partitioning on column where lowerBound is equal to upperBound") { + val res = spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 5, + upperBound = 5, + numPartitions = 4, + connectionProperties = new Properties + ) + assert(res.count() === 8) + } + + test("Partitioning on column where lowerBound is larger than upperBound") { + val e = intercept[IllegalArgumentException] { + spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 5, + upperBound = 1, + numPartitions = 3, + connectionProperties = new Properties + ) + }.getMessage + assert(e.contains("Operation not allowed: the lower bound of partitioning column " + + "is larger than the upper bound. Lower bound: 5; Upper bound: 1")) + } + test("SELECT * on partitioned table with a nullable partition column") { assert(sql("SELECT * FROM nullparts").collect().size == 4) } From 9d513b8d22065cccc7cd6c4ab6f182f446b4107d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 20 Jun 2016 21:51:02 -0700 Subject: [PATCH 0741/1470] [SPARK-16074][MLLIB] expose VectorUDT/MatrixUDT in a public API ## What changes were proposed in this pull request? Both VectorUDT and MatrixUDT are private APIs, because UserDefinedType itself is private in Spark. However, in order to let developers implement their own transformers and estimators, we should expose both types in a public API to simply the implementation of transformSchema, transform, etc. Otherwise, they need to get the data types using reflection. ## How was this patch tested? Unit tests in Scala and Java. Author: Xiangrui Meng Closes #13789 from mengxr/SPARK-16074. (cherry picked from commit 18a8a9b1f4114211cd108efda5672f2bd2c6e5cd) Signed-off-by: Reynold Xin --- .../apache/spark/ml/linalg/dataTypes.scala | 35 +++++++++++++++++++ .../ml/linalg/JavaSQLDataTypesSuite.java | 31 ++++++++++++++++ .../spark/ml/linalg/SQLDataTypesSuite.scala | 27 ++++++++++++++ 3 files changed, 93 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/linalg/dataTypes.scala create mode 100644 mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/dataTypes.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/dataTypes.scala new file mode 100644 index 0000000000000..52a6fd25e2fa7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/dataTypes.scala @@ -0,0 +1,35 @@ +/* + * 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.linalg + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.types.DataType + +/** + * :: DeveloperApi :: + * SQL data types for vectors and matrices. + */ +@DeveloperApi +object sqlDataTypes { + + /** Data type for [[Vector]]. */ + val VectorType: DataType = new VectorUDT + + /** Data type for [[Matrix]]. */ + val MatrixType: DataType = new MatrixUDT +} diff --git a/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java b/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java new file mode 100644 index 0000000000000..b09e13112f124 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java @@ -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. + */ + +package org.apache.spark.ml.linalg; + +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.ml.linalg.sqlDataTypes.*; + +public class JavaSQLDataTypesSuite { + @Test + public void testSQLDataTypes() { + Assert.assertEquals(new VectorUDT(), VectorType()); + Assert.assertEquals(new MatrixUDT(), MatrixType()); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala new file mode 100644 index 0000000000000..13bf3d3015f64 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.SparkFunSuite + +class SQLDataTypesSuite extends SparkFunSuite { + test("sqlDataTypes") { + assert(sqlDataTypes.VectorType === new VectorUDT) + assert(sqlDataTypes.MatrixType === new MatrixUDT) + } +} From 12f00b6edde9b6f97d2450e2cd99edd5e31b9169 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 20 Jun 2016 21:55:34 -0700 Subject: [PATCH 0742/1470] [SPARK-16044][SQL] input_file_name() returns empty strings in data sources based on NewHadoopRDD ## What changes were proposed in this pull request? This PR makes `input_file_name()` function return the file paths not empty strings for external data sources based on `NewHadoopRDD`, such as [spark-redshift](https://github.com/databricks/spark-redshift/blob/cba5eee1ab79ae8f0fa9e668373a54d2b5babf6b/src/main/scala/com/databricks/spark/redshift/RedshiftRelation.scala#L149) and [spark-xml](https://github.com/databricks/spark-xml/blob/master/src/main/scala/com/databricks/spark/xml/util/XmlFile.scala#L39-L47). The codes with the external data sources below: ```scala df.select(input_file_name).show() ``` will produce - **Before** ``` +-----------------+ |input_file_name()| +-----------------+ | | +-----------------+ ``` - **After** ``` +--------------------+ | input_file_name()| +--------------------+ |file:/private/var...| +--------------------+ ``` ## How was this patch tested? Unit tests in `ColumnExpressionSuite`. Author: hyukjinkwon Closes #13759 from HyukjinKwon/SPARK-16044. (cherry picked from commit 4f7f1c436205630ab77d3758d7210cc1a2f0d04a) Signed-off-by: Reynold Xin --- .../spark/rdd/InputFileNameHolder.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 7 ++++ .../spark/sql/ColumnExpressionSuite.scala | 34 +++++++++++++++++-- 3 files changed, 40 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala index 108e9d2558190..f40d4c8e0a4d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala +++ b/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala @@ -21,7 +21,7 @@ import org.apache.spark.unsafe.types.UTF8String /** * This holds file names of the current Spark task. This is used in HadoopRDD, - * FileScanRDD and InputFileName function in Spark SQL. + * FileScanRDD, NewHadoopRDD and InputFileName function in Spark SQL. */ private[spark] object InputFileNameHolder { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 189dc7b331337..b086baa084080 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -135,6 +135,12 @@ class NewHadoopRDD[K, V]( val inputMetrics = context.taskMetrics().inputMetrics val existingBytesRead = inputMetrics.bytesRead + // Sets the thread local variable for the file's name + split.serializableHadoopSplit.value match { + case fs: FileSplit => InputFileNameHolder.setInputFileName(fs.getPath.toString) + case _ => InputFileNameHolder.unsetInputFileName() + } + // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match { @@ -201,6 +207,7 @@ class NewHadoopRDD[K, V]( private def close() { if (reader != null) { + InputFileNameHolder.unsetInputFileName() // Close the reader and release it. Note: it's very important that we don't close the // reader more than once, since that exposes us to MAPREDUCE-5918 when running against // Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index e89fa32b15143..a66c83dea00b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.scalatest.Matchers._ import org.apache.spark.sql.catalyst.expressions.NamedExpression @@ -592,7 +594,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { ) } - test("input_file_name") { + test("input_file_name - FileScanRDD") { withTempPath { dir => val data = sparkContext.parallelize(0 to 10).toDF("id") data.write.parquet(dir.getCanonicalPath) @@ -604,6 +606,35 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { } } + test("input_file_name - HadoopRDD") { + withTempPath { dir => + val data = sparkContext.parallelize((0 to 10).map(_.toString)).toDF() + data.write.text(dir.getCanonicalPath) + val df = spark.sparkContext.textFile(dir.getCanonicalPath).toDF() + val answer = df.select(input_file_name()).head.getString(0) + assert(answer.contains(dir.getCanonicalPath)) + + checkAnswer(data.select(input_file_name()).limit(1), Row("")) + } + } + + test("input_file_name - NewHadoopRDD") { + withTempPath { dir => + val data = sparkContext.parallelize((0 to 10).map(_.toString)).toDF() + data.write.text(dir.getCanonicalPath) + val rdd = spark.sparkContext.newAPIHadoopFile( + dir.getCanonicalPath, + classOf[NewTextInputFormat], + classOf[LongWritable], + classOf[Text]) + val df = rdd.map(pair => pair._2.toString).toDF() + val answer = df.select(input_file_name()).head.getString(0) + assert(answer.contains(dir.getCanonicalPath)) + + checkAnswer(data.select(input_file_name()).limit(1), Row("")) + } + } + test("columns can be compared") { assert('key.desc == 'key.desc) assert('key.desc != 'key.asc) @@ -707,5 +738,4 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { testData2.select($"a".bitwiseXOR($"b").bitwiseXOR(39)), testData2.collect().toSeq.map(r => Row(r.getInt(0) ^ r.getInt(1) ^ 39))) } - } From 4fc4eb9434676d6c7be1b0dd8ff1dc67d7d2b308 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 20 Jun 2016 21:56:44 -0700 Subject: [PATCH 0743/1470] [SPARK-16025][CORE] Document OFF_HEAP storage level in 2.0 This has changed from 1.6, and now stores memory off-heap using spark's off-heap support instead of in tachyon. Author: Eric Liang Closes #13744 from ericl/spark-16025. (cherry picked from commit 07367533de68817e1e6cf9cf2b056a04dd160c8a) Signed-off-by: Reynold Xin --- docs/programming-guide.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 97bcb511b2b02..3872aecff25d7 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1220,6 +1220,11 @@ storage levels is:
    + + + +
    MEMORY_ONLY_2, MEMORY_AND_DISK_2, etc. Same as the levels above, but replicate each partition on two cluster nodes.
    OFF_HEAP (experimental) Similar to MEMORY_ONLY_SER, but store the data in + off-heap memory. This requires off-heap memory to be enabled.
    **Note:** *In Python, stored objects will always be serialized with the [Pickle](https://docs.python.org/2/library/pickle.html) library, From dbf7f48b6e73f3500b0abe9055ac204a3f756418 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 21 Jun 2016 13:56:37 +0800 Subject: [PATCH 0744/1470] [SPARK-15863][SQL][DOC][SPARKR] sql programming guide updates to include sparkSession in R ## What changes were proposed in this pull request? Update doc as per discussion in PR #13592 ## How was this patch tested? manual shivaram liancheng Author: Felix Cheung Closes #13799 from felixcheung/rsqlprogrammingguide. (cherry picked from commit 58f6e27dd70f476f99ac8204e6b405bced4d6de1) Signed-off-by: Cheng Lian --- docs/sparkr.md | 2 +- docs/sql-programming-guide.md | 34 ++++++++++++++++------------------ 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/docs/sparkr.md b/docs/sparkr.md index 023bbcd39c7e3..f0189012f36ed 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -152,7 +152,7 @@ write.df(people, path="people.parquet", source="parquet", mode="overwrite") ### From Hive tables -You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sqlcontext). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`). +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`).
    {% highlight r %} diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d93f30b1e5cc3..4206f730de076 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -107,19 +107,17 @@ spark = SparkSession.build \
    -Unlike Scala, Java, and Python API, we haven't finished migrating `SQLContext` to `SparkSession` for SparkR yet, so -the entry point into all relational functionality in SparkR is still the -`SQLContext` class in Spark 2.0. To create a basic `SQLContext`, all you need is a `SparkContext`. +The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: {% highlight r %} -spark <- sparkRSQL.init(sc) +sparkR.session() {% endhighlight %} -Note that when invoked for the first time, `sparkRSQL.init()` initializes a global `SQLContext` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SQLContext` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SQLContext` instance around. +Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around.
    -`SparkSession` (or `SQLContext` for SparkR) in Spark 2.0 provides builtin support for Hive features including the ability to +`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. To use these features, you do not need to have an existing Hive setup. @@ -175,7 +173,7 @@ df.show()
    -With a `SQLContext`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +With a `SparkSession`, applications can create DataFrames from a local R data.frame, from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: @@ -183,7 +181,7 @@ As an example, the following creates a DataFrame based on the content of a JSON {% highlight r %} df <- read.json("examples/src/main/resources/people.json") -# Displays the content of the DataFrame to stdout +# Displays the content of the DataFrame showDF(df) {% endhighlight %} @@ -415,7 +413,7 @@ showDF(count(groupBy(df, "age"))) For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/index.html). +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html).
    @@ -452,7 +450,7 @@ df = spark.sql("SELECT * FROM table")
    -The `sql` function enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. {% highlight r %} df <- sql("SELECT * FROM table") @@ -1159,11 +1157,10 @@ for teenName in teenNames.collect():
    {% highlight r %} -# spark from the previous example is used in this example. -schemaPeople # The DataFrame from the previous example. +schemaPeople # The SparkDataFrame from the previous example. -# DataFrames can be saved as Parquet files, maintaining the schema information. +# SparkDataFrame can be saved as Parquet files, maintaining the schema information. write.parquet(schemaPeople, "people.parquet") # Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. @@ -1342,7 +1339,6 @@ df3.printSchema()
    {% highlight r %} -# spark from the previous example is used in this example. # Create a simple DataFrame, stored into a partition directory write.df(df1, "data/test_table/key=1", "parquet", "overwrite") @@ -1621,7 +1617,7 @@ anotherPeople = spark.jsonRDD(anotherPeopleRDD)
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using -the `jsonFile` function, which loads data from a directory of JSON files where each line of the +the `read.json()` function, which loads data from a directory of JSON files where each line of the files is a JSON object. Note that the file that is offered as _a json file_ is not a typical JSON file. Each @@ -1644,7 +1640,7 @@ printSchema(people) # Register this DataFrame as a table. createOrReplaceTempView(people, "people") -# SQL statements can be run by using the sql methods provided by `spark`. +# SQL statements can be run by using the sql methods. teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") {% endhighlight %}
    @@ -1759,9 +1755,11 @@ results = spark.sql("FROM src SELECT key, value").collect()
    -When working with Hive one must construct a `HiveContext`, which inherits from `SparkSession`, and +When working with Hive one must instantiate `SparkSession` with Hive support. This adds support for finding tables in the MetaStore and writing queries using HiveQL. {% highlight r %} +# enableHiveSupport defaults to TRUE +sparkR.session(enableHiveSupport = TRUE) sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") @@ -1947,7 +1945,7 @@ df = spark.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbtable=' {% highlight r %} -df <- loadDF(spark, source="jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") +df <- read.jdbc("jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password") {% endhighlight %} From 4e193d3daf5bdfb38d7df6da5b7abdd53888ec99 Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Tue, 21 Jun 2016 14:27:16 +0800 Subject: [PATCH 0745/1470] [SPARK-15894][SQL][DOC] Update docs for controlling #partitions ## What changes were proposed in this pull request? Update docs for two parameters `spark.sql.files.maxPartitionBytes` and `spark.sql.files.openCostInBytes ` in Other Configuration Options. ## How was this patch tested? N/A Author: Takeshi YAMAMURO Closes #13797 from maropu/SPARK-15894-2. (cherry picked from commit 41e0ffb19f678e9b1e87f747a5e4e3d44964e39a) Signed-off-by: Cheng Lian --- docs/sql-programming-guide.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4206f730de076..ddf8f701ca036 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -2015,6 +2015,23 @@ that these options will be deprecated in future release as more optimizations ar + + + + + + + + + + From 38f3b76bd6b4a3e4d20048beeb92275ebf93c8d8 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 20 Jun 2016 23:51:08 -0700 Subject: [PATCH 0746/1470] [SPARKR][DOCS] R code doc cleanup ## What changes were proposed in this pull request? I ran a full pass from A to Z and fixed the obvious duplications, improper grouping etc. There are still more doc issues to be cleaned up. ## How was this patch tested? manual tests Author: Felix Cheung Closes #13798 from felixcheung/rdocseealso. (cherry picked from commit 09f4ceaeb0a99874f774e09d868fdf907ecf256f) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 39 ++++++++++++++++++--------------------- R/pkg/R/SQLContext.R | 6 +++--- R/pkg/R/column.R | 6 ++++++ R/pkg/R/context.R | 5 +++-- R/pkg/R/functions.R | 40 +++++++++++++--------------------------- R/pkg/R/generics.R | 44 ++++++++++++++++++++++---------------------- R/pkg/R/mllib.R | 6 ------ R/pkg/R/sparkR.R | 8 +++++--- 8 files changed, 70 insertions(+), 84 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index b3f2dd82ff549..a8ade1ac9ab72 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -463,6 +463,7 @@ setMethod("createOrReplaceTempView", }) #' (Deprecated) Register Temporary Table +#' #' Registers a SparkDataFrame as a Temporary Table in the SQLContext #' @param x A SparkDataFrame #' @param tableName A character vector containing the name of the table @@ -606,10 +607,10 @@ setMethod("unpersist", #' #' The following options for repartition are possible: #' \itemize{ -#' \item{"Option 1"} {Return a new SparkDataFrame partitioned by +#' \item{1.} {Return a new SparkDataFrame partitioned by #' the given columns into `numPartitions`.} -#' \item{"Option 2"} {Return a new SparkDataFrame that has exactly `numPartitions`.} -#' \item{"Option 3"} {Return a new SparkDataFrame partitioned by the given column(s), +#' \item{2.} {Return a new SparkDataFrame that has exactly `numPartitions`.} +#' \item{3.} {Return a new SparkDataFrame partitioned by the given column(s), #' using `spark.sql.shuffle.partitions` as number of partitions.} #'} #' @param x A SparkDataFrame @@ -1053,7 +1054,7 @@ setMethod("limit", dataFrame(res) }) -#' Take the first NUM rows of a SparkDataFrame and return a the results as a data.frame +#' Take the first NUM rows of a SparkDataFrame and return a the results as a R data.frame #' #' @family SparkDataFrame functions #' @rdname take @@ -1076,7 +1077,7 @@ setMethod("take", #' Head #' -#' Return the first NUM rows of a SparkDataFrame as a data.frame. If NUM is NULL, +#' Return the first NUM rows of a SparkDataFrame as a R data.frame. If NUM is NULL, #' then head() returns the first 6 rows in keeping with the current data.frame #' convention in R. #' @@ -1157,7 +1158,6 @@ setMethod("toRDD", #' #' @param x a SparkDataFrame #' @return a GroupedData -#' @seealso GroupedData #' @family SparkDataFrame functions #' @rdname groupBy #' @name groupBy @@ -1242,9 +1242,9 @@ dapplyInternal <- function(x, func, schema) { #' #' @param x A SparkDataFrame #' @param func A function to be applied to each partition of the SparkDataFrame. -#' func should have only one parameter, to which a data.frame corresponds +#' func should have only one parameter, to which a R data.frame corresponds #' to each partition will be passed. -#' The output of func should be a data.frame. +#' The output of func should be a R data.frame. #' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' It must match the output of func. #' @family SparkDataFrame functions @@ -1291,9 +1291,9 @@ setMethod("dapply", #' #' @param x A SparkDataFrame #' @param func A function to be applied to each partition of the SparkDataFrame. -#' func should have only one parameter, to which a data.frame corresponds +#' func should have only one parameter, to which a R data.frame corresponds #' to each partition will be passed. -#' The output of func should be a data.frame. +#' The output of func should be a R data.frame. #' @family SparkDataFrame functions #' @rdname dapplyCollect #' @name dapplyCollect @@ -1641,7 +1641,6 @@ setMethod("select", signature(x = "SparkDataFrame", col = "character"), } }) -#' @family SparkDataFrame functions #' @rdname select #' @export #' @note select(SparkDataFrame, Column) since 1.4.0 @@ -1654,7 +1653,6 @@ setMethod("select", signature(x = "SparkDataFrame", col = "Column"), dataFrame(sdf) }) -#' @family SparkDataFrame functions #' @rdname select #' @export #' @note select(SparkDataFrame, list) since 1.4.0 @@ -2001,7 +1999,6 @@ setMethod("filter", dataFrame(sdf) }) -#' @family SparkDataFrame functions #' @rdname filter #' @name where #' @note where since 1.4.0 @@ -2222,11 +2219,13 @@ setMethod("merge", joinRes }) +#' Creates a list of columns by replacing the intersected ones with aliases +#' #' Creates a list of columns by replacing the intersected ones with aliases. #' The name of the alias column is formed by concatanating the original column name and a suffix. #' -#' @param x a SparkDataFrame on which the -#' @param intersectedColNames a list of intersected column names +#' @param x a SparkDataFrame +#' @param intersectedColNames a list of intersected column names of the SparkDataFrame #' @param suffix a suffix for the column name #' @return list of columns #' @@ -2513,9 +2512,9 @@ setMethod("summary", }) -#' dropna +#' A set of SparkDataFrame functions working with NA values #' -#' Returns a new SparkDataFrame omitting rows with null values. +#' dropna, na.omit - Returns a new SparkDataFrame omitting rows with null values. #' #' @param x A SparkDataFrame. #' @param how "any" or "all". @@ -2567,9 +2566,7 @@ setMethod("na.omit", dropna(object, how, minNonNulls, cols) }) -#' fillna -#' -#' Replace null values. +#' fillna - Replace null values. #' #' @param x A SparkDataFrame. #' @param value Value to replace null values with. @@ -2640,7 +2637,7 @@ setMethod("fillna", dataFrame(sdf) }) -#' Download data from a SparkDataFrame into a data.frame +#' Download data from a SparkDataFrame into a R data.frame #' #' This function downloads the contents of a SparkDataFrame into an R's data.frame. #' Since data.frames are held in memory, ensure that you have enough memory diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 8d2c4ac7cee91..ee3a41cacbee6 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -67,7 +67,7 @@ dispatchFunc <- function(newFuncSig, x, ...) { } #' return the SparkSession -#' @note getSparkSession since 2.0.0 +#' @noRd getSparkSession <- function() { if (exists(".sparkRsession", envir = .sparkREnv)) { get(".sparkRsession", envir = .sparkREnv) @@ -77,7 +77,7 @@ getSparkSession <- function() { } #' infer the SQL type -#' @note infer_type since 1.4.0 +#' @noRd infer_type <- function(x) { if (is.null(x)) { stop("can not infer type from NULL") @@ -451,7 +451,7 @@ sql <- function(x, ...) { #' Create a SparkDataFrame from a SparkSQL Table #' #' Returns the specified Table as a SparkDataFrame. The Table must have already been registered -#' in the SQLContext. +#' in the SparkSession. #' #' @param tableName The SparkSQL Table to convert to a SparkDataFrame. #' @return SparkDataFrame diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 1af65d5d6e155..1a65912d3aed1 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -34,6 +34,11 @@ setOldClass("jobj") setClass("Column", slots = list(jc = "jobj")) +#' A set of operations working with SparkDataFrame columns +#' @rdname columnfunctions +#' @name columnfunctions +NULL + setMethod("initialize", "Column", function(.Object, jc) { .Object@jc <- jc .Object @@ -47,6 +52,7 @@ setMethod("column", #' @rdname show #' @name show +#' @export #' @note show(Column) since 1.4.0 setMethod("show", "Column", function(object) { diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 42f89c806bfd9..96ef9438ad5dc 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -225,9 +225,10 @@ setCheckpointDir <- function(sc, dirName) { invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) } -#' Run a function over a list of elements, distributing the computations with Spark. +#' Run a function over a list of elements, distributing the computations with Spark #' -#' Applies a function in a manner that is similar to doParallel or lapply to elements of a list. +#' Run a function over a list of elements, distributing the computations with Spark. Applies a +#' function in a manner that is similar to doParallel or lapply to elements of a list. #' The computations are distributed using Spark. It is conceptually the same as the following code: #' lapply(list, func) #' diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index ce2386998c1bc..6e0009f7c90af 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -77,13 +77,14 @@ setMethod("acos", column(jc) }) -#' approxCountDistinct +#' Returns the approximate number of distinct items in a group #' -#' Aggregate function: returns the approximate number of distinct items in a group. +#' Returns the approximate number of distinct items in a group. This is a column +#' aggregate function. #' #' @rdname approxCountDistinct #' @name approxCountDistinct -#' @family agg_funcs +#' @return the approximate number of distinct items in a group. #' @export #' @examples \dontrun{approxCountDistinct(df$c)} #' @note approxCountDistinct(Column) since 1.4.0 @@ -234,7 +235,7 @@ setMethod("cbrt", column(jc) }) -#' ceil +#' Computes the ceiling of the given value #' #' Computes the ceiling of the given value. #' @@ -254,15 +255,16 @@ setMethod("ceil", #' Though scala functions has "col" function, we don't expose it in SparkR #' because we don't want to conflict with the "col" function in the R base #' package and we also have "column" function exported which is an alias of "col". +#' @noRd col <- function(x) { column(callJStatic("org.apache.spark.sql.functions", "col", x)) } -#' column +#' Returns a Column based on the given column name #' #' Returns a Column based on the given column name. #' -#' @rdname col +#' @rdname column #' @name column #' @family normal_funcs #' @export @@ -385,9 +387,9 @@ setMethod("cosh", column(jc) }) -#' count +#' Returns the number of items in a group #' -#' Aggregate function: returns the number of items in a group. +#' Returns the number of items in a group. This is a column aggregate function. #' #' @rdname count #' @name count @@ -1193,7 +1195,7 @@ setMethod("sha1", #' #' Computes the signum of the given value. #' -#' @rdname signum +#' @rdname sign #' @name signum #' @family math_funcs #' @export @@ -1717,7 +1719,7 @@ setMethod("datediff", signature(y = "Column"), #' hypot #' -#' Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. +#' Computes "sqrt(a^2 + b^2)" without intermediate overflow or underflow. #' #' @rdname hypot #' @name hypot @@ -1813,12 +1815,8 @@ setMethod("pmod", signature(y = "Column"), }) -#' Approx Count Distinct -#' -#' @family agg_funcs #' @rdname approxCountDistinct #' @name approxCountDistinct -#' @return the approximate number of distinct items in a group. #' @export #' @examples \dontrun{approxCountDistinct(df$c, 0.02)} #' @note approxCountDistinct(Column, numeric) since 1.4.0 @@ -1918,10 +1916,6 @@ setMethod("least", column(jc) }) -#' ceiling -#' -#' Computes the ceiling of the given value. -#' #' @rdname ceil #' @name ceiling #' @export @@ -1933,11 +1927,7 @@ setMethod("ceiling", ceil(x) }) -#' sign -#' -#' Computes the signum of the given value. -#' -#' @rdname signum +#' @rdname sign #' @name sign #' @export #' @examples \dontrun{sign(df$c)} @@ -1961,10 +1951,6 @@ setMethod("n_distinct", signature(x = "Column"), countDistinct(x, ...) }) -#' n -#' -#' Aggregate function: returns the number of items in a group. -#' #' @rdname count #' @name n #' @export diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index c307de7c07633..ead403be98002 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -430,7 +430,7 @@ setGeneric("coltypes", function(x) { standardGeneric("coltypes") }) #' @export setGeneric("coltypes<-", function(x, value) { standardGeneric("coltypes<-") }) -#' @rdname schema +#' @rdname columns #' @export setGeneric("columns", function(x) {standardGeneric("columns") }) @@ -495,7 +495,7 @@ setGeneric("na.omit", standardGeneric("na.omit") }) -#' @rdname schema +#' @rdname dtypes #' @export setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) @@ -551,7 +551,7 @@ setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) #' @export setGeneric("orderBy", function(x, col, ...) { standardGeneric("orderBy") }) -#' @rdname schema +#' @rdname printSchema #' @export setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) @@ -638,7 +638,7 @@ setGeneric("schema", function(x) { standardGeneric("schema") }) #' @export setGeneric("select", function(x, col, ...) { standardGeneric("select") } ) -#' @rdname select +#' @rdname selectExpr #' @export setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") }) @@ -693,67 +693,67 @@ setGeneric("randomSplit", function(x, weights, seed) { standardGeneric("randomSp ###################### Column Methods ########################## -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("asc", function(x) { standardGeneric("asc") }) -#' @rdname column +#' @rdname between #' @export setGeneric("between", function(x, bounds) { standardGeneric("between") }) -#' @rdname column +#' @rdname cast #' @export setGeneric("cast", function(x, dataType) { standardGeneric("cast") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("contains", function(x, ...) { standardGeneric("contains") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("desc", function(x) { standardGeneric("desc") }) -#' @rdname column +#' @rdname endsWith #' @export setGeneric("endsWith", function(x, suffix) { standardGeneric("endsWith") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("getField", function(x, ...) { standardGeneric("getField") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("getItem", function(x, ...) { standardGeneric("getItem") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("isNaN", function(x) { standardGeneric("isNaN") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("isNull", function(x) { standardGeneric("isNull") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("isNotNull", function(x) { standardGeneric("isNotNull") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("like", function(x, ...) { standardGeneric("like") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("rlike", function(x, ...) { standardGeneric("rlike") }) -#' @rdname column +#' @rdname startsWith #' @export setGeneric("startsWith", function(x, prefix) { standardGeneric("startsWith") }) -#' @rdname column +#' @rdname when #' @export setGeneric("when", function(condition, value) { standardGeneric("when") }) -#' @rdname column +#' @rdname otherwise #' @export setGeneric("otherwise", function(x, value) { standardGeneric("otherwise") }) @@ -825,7 +825,7 @@ setGeneric("cbrt", function(x) { standardGeneric("cbrt") }) #' @export setGeneric("ceil", function(x) { standardGeneric("ceil") }) -#' @rdname col +#' @rdname column #' @export setGeneric("column", function(x) { standardGeneric("column") }) @@ -1119,7 +1119,7 @@ setGeneric("shiftRight", function(y, x) { standardGeneric("shiftRight") }) #' @export setGeneric("shiftRightUnsigned", function(y, x) { standardGeneric("shiftRightUnsigned") }) -#' @rdname signum +#' @rdname sign #' @export setGeneric("signum", function(x) { standardGeneric("signum") }) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index d6ff2aa22df2f..74dba8fe966fb 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -235,8 +235,6 @@ setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), #' similarly to R package e1071's predict. #' #' @param object A fitted naive Bayes model -#' @param newData SparkDataFrame for testing -#' @return SparkDataFrame containing predicted labels in a column named "prediction" #' @rdname predict #' @export #' @examples @@ -378,8 +376,6 @@ setMethod("summary", signature(object = "KMeansModel"), #' Makes predictions from a k-means model or a model produced by spark.kmeans(). #' #' @param object A fitted k-means model -#' @param newData SparkDataFrame for testing -#' @return SparkDataFrame containing predicted labels in a column named "prediction" #' @rdname predict #' @export #' @examples @@ -621,8 +617,6 @@ setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), #' similarly to R package survival's predict. #' #' @param object A fitted AFT survival regression model -#' @param newData SparkDataFrame for testing -#' @return SparkDataFrame containing predicted labels in a column named "prediction" #' @rdname predict #' @export #' @examples diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 94d0e63c8a46a..2b6e124151397 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -36,6 +36,8 @@ sparkR.stop <- function() { sparkR.session.stop() } +#' Stop the Spark Session and Spark Context +#' #' Stop the Spark Session and Spark Context. #' #' Also terminates the backend this R session is connected to. @@ -88,7 +90,7 @@ sparkR.session.stop <- function() { clearJobjs() } -#' (Deprecated) Initialize a new Spark Context. +#' (Deprecated) Initialize a new Spark Context #' #' This function initializes a new SparkContext. #' @@ -249,7 +251,7 @@ sparkR.sparkContext <- function( sc } -#' (Deprecated) Initialize a new SQLContext. +#' (Deprecated) Initialize a new SQLContext #' #' This function creates a SparkContext from an existing JavaSparkContext and #' then uses it to initialize a new SQLContext @@ -278,7 +280,7 @@ sparkRSQL.init <- function(jsc = NULL) { sparkR.session(enableHiveSupport = FALSE) } -#' (Deprecated) Initialize a new HiveContext. +#' (Deprecated) Initialize a new HiveContext #' #' This function creates a HiveContext from an existing JavaSparkContext #' From 34feea336886b241135e6c60677000c2ca6b52b4 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 21 Jun 2016 00:19:09 -0700 Subject: [PATCH 0747/1470] [SPARK-15319][SPARKR][DOCS] Fix SparkR doc layout for corr and other DataFrame stats functions ## What changes were proposed in this pull request? Doc only changes. Please see screenshots. Before: http://spark.apache.org/docs/latest/api/R/statfunctions.html ![image](https://cloud.githubusercontent.com/assets/8969467/15264110/cd458826-1924-11e6-85bd-8ee2e2e1a85f.png) After ![image](https://cloud.githubusercontent.com/assets/8969467/16218452/b9e89f08-3732-11e6-969d-a3a1796e7ad0.png) (please ignore the style differences - this is due to not having the css in my local copy) This is still a bit weird. As discussed in SPARK-15237, I think the better approach is to separate out the DataFrame stats function instead of putting everything on one page. At least now it is clearer which description is on which function. ## How was this patch tested? Build doc Author: Felix Cheung Author: felixcheung Closes #13109 from felixcheung/rstatdoc. (cherry picked from commit 843a1eba8ec9d5a7beac0c74b54d24cb3c41b45a) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/generics.R | 8 ++++---- R/pkg/R/stats.R | 32 +++++++++++++------------------- 2 files changed, 17 insertions(+), 23 deletions(-) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index ead403be98002..43395aaa1da96 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -434,19 +434,19 @@ setGeneric("coltypes<-", function(x, value) { standardGeneric("coltypes<-") }) #' @export setGeneric("columns", function(x) {standardGeneric("columns") }) -#' @rdname statfunctions +#' @rdname cov #' @export setGeneric("cov", function(x, ...) {standardGeneric("cov") }) -#' @rdname statfunctions +#' @rdname corr #' @export setGeneric("corr", function(x, ...) {standardGeneric("corr") }) -#' @rdname statfunctions +#' @rdname cov #' @export setGeneric("covar_samp", function(col1, col2) {standardGeneric("covar_samp") }) -#' @rdname statfunctions +#' @rdname covar_pop #' @export setGeneric("covar_pop", function(col1, col2) {standardGeneric("covar_pop") }) diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index e92b9e3d84f1e..e40b1773d70e4 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -19,9 +19,10 @@ setOldClass("jobj") -#' crosstab -#' -#' Computes a pair-wise frequency table of the given columns. Also known as a contingency +#' @title SparkDataFrame statistic functions + +#' @description +#' crosstab - Computes a pair-wise frequency table of the given columns. Also known as a contingency #' table. The number of distinct values for each column should be less than 1e4. At most 1e6 #' non-zero pair frequencies will be returned. #' @@ -49,8 +50,6 @@ setMethod("crosstab", collect(dataFrame(sct)) }) -#' cov -#' #' Calculate the sample covariance of two numerical columns of a SparkDataFrame. #' #' @param x A SparkDataFrame @@ -58,7 +57,7 @@ setMethod("crosstab", #' @param col2 the name of the second column #' @return the covariance of the two columns. #' -#' @rdname statfunctions +#' @rdname cov #' @name cov #' @export #' @examples @@ -75,8 +74,6 @@ setMethod("cov", callJMethod(statFunctions, "cov", col1, col2) }) -#' corr -#' #' Calculates the correlation of two columns of a SparkDataFrame. #' Currently only supports the Pearson Correlation Coefficient. #' For Spearman Correlation, consider using RDD methods found in MLlib's Statistics. @@ -88,7 +85,7 @@ setMethod("cov", #' only "pearson" is allowed now. #' @return The Pearson Correlation Coefficient as a Double. #' -#' @rdname statfunctions +#' @rdname corr #' @name corr #' @export #' @examples @@ -106,9 +103,8 @@ setMethod("corr", callJMethod(statFunctions, "corr", col1, col2, method) }) -#' freqItems -#' -#' Finding frequent items for columns, possibly with false positives. +#' @description +#' freqItems - Finding frequent items for columns, possibly with false positives. #' Using the frequent element count algorithm described in #' \url{http://dx.doi.org/10.1145/762471.762473}, proposed by Karp, Schenker, and Papadimitriou. #' @@ -134,10 +130,8 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), collect(dataFrame(sct)) }) -#' approxQuantile -#' -#' Calculates the approximate quantiles of a numerical column of a SparkDataFrame. -#' +#' @description +#' approxQuantile - Calculates the approximate quantiles of a numerical column of a SparkDataFrame. #' The result of this algorithm has the following deterministic bound: #' If the SparkDataFrame has N elements and if we request the quantile at probability `p` up to #' error `err`, then the algorithm will return a sample `x` from the SparkDataFrame so that the @@ -174,9 +168,9 @@ setMethod("approxQuantile", as.list(probabilities), relativeError) }) -#' sampleBy -#' -#' Returns a stratified sample without replacement based on the fraction given on each stratum. +#' @description +#' sampleBy - Returns a stratified sample without replacement based on the fraction given on each +#' stratum. #' #' @param x A SparkDataFrame #' @param col column that defines strata From 37d05ec9e96c0da786ee26b5c25216bf98f239c0 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 21 Jun 2016 00:33:38 -0700 Subject: [PATCH 0748/1470] Revert "[SPARK-16086] [SQL] fix Python UDF without arguments (for 1.6)" This reverts commit 087bd2799366f4914d248e9b1f0fb921adbbdb43. --- python/pyspark/sql/tests.py | 5 ----- python/pyspark/sql/types.py | 9 ++++++--- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index ecd1a0563e959..c631ad8a4618d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -318,11 +318,6 @@ def test_multiple_udfs(self): [row] = self.spark.sql("SELECT double(add(1, 2)), add(double(2), 1)").collect() self.assertEqual(tuple(row), (6, 5)) - def test_udf_without_arguments(self): - self.sqlCtx.registerFunction("foo", lambda: "bar") - [row] = self.sqlCtx.sql("SELECT foo()").collect() - self.assertEqual(row[0], "bar") - def test_udf_with_array_type(self): d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index f0b56be8da438..bb2b95404af7a 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1401,7 +1401,11 @@ def __new__(self, *args, **kwargs): if args and kwargs: raise ValueError("Can not use both args " "and kwargs to create Row") - if kwargs: + if args: + # create row class or objects + return tuple.__new__(self, args) + + elif kwargs: # create row objects names = sorted(kwargs.keys()) row = tuple.__new__(self, [kwargs[n] for n in names]) @@ -1409,8 +1413,7 @@ def __new__(self, *args, **kwargs): return row else: - # create row class or objects - return tuple.__new__(self, args) + raise ValueError("No args or kwargs") def asDict(self, recursive=False): """ From 14e5decc5f8977e253cde0135d57204a7c0ebb7f Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Tue, 21 Jun 2016 00:39:47 -0700 Subject: [PATCH 0749/1470] [SPARK-10258][DOC][ML] Add @Since annotations to ml.feature This PR adds missing `Since` annotations to `ml.feature` package. Closes #8505. ## How was this patch tested? Existing tests. Author: Nick Pentreath Closes #13641 from MLnick/add-since-annotations. (cherry picked from commit 37494a18e8d6e22113338523d6498e00ac9725ea) Signed-off-by: Xiangrui Meng --- .../apache/spark/ml/feature/Binarizer.scala | 11 ++++++- .../apache/spark/ml/feature/Bucketizer.scala | 12 ++++++- .../spark/ml/feature/ChiSqSelector.scala | 19 +++++++++-- .../spark/ml/feature/CountVectorizer.scala | 24 ++++++++++++-- .../org/apache/spark/ml/feature/DCT.scala | 7 +++- .../spark/ml/feature/ElementwiseProduct.scala | 7 +++- .../apache/spark/ml/feature/HashingTF.scala | 14 +++++++- .../org/apache/spark/ml/feature/IDF.scala | 20 ++++++++--- .../apache/spark/ml/feature/Interaction.scala | 4 +-- .../spark/ml/feature/MaxAbsScaler.scala | 26 ++++++++++----- .../spark/ml/feature/MinMaxScaler.scala | 23 ++++++++++--- .../org/apache/spark/ml/feature/NGram.scala | 7 +++- .../apache/spark/ml/feature/Normalizer.scala | 7 +++- .../spark/ml/feature/OneHotEncoder.scala | 10 +++++- .../org/apache/spark/ml/feature/PCA.scala | 23 ++++++++++--- .../ml/feature/PolynomialExpansion.scala | 8 ++++- .../ml/feature/QuantileDiscretizer.scala | 10 +++++- .../apache/spark/ml/feature/RFormula.scala | 18 ++++++++-- .../spark/ml/feature/SQLTransformer.scala | 2 +- .../spark/ml/feature/StandardScaler.scala | 24 +++++++++++--- .../spark/ml/feature/StopWordsRemover.scala | 14 +++++++- .../spark/ml/feature/StringIndexer.scala | 33 ++++++++++++++++--- .../apache/spark/ml/feature/Tokenizer.scala | 22 +++++++++++-- .../spark/ml/feature/VectorAssembler.scala | 8 ++++- .../spark/ml/feature/VectorIndexer.scala | 24 +++++++++++--- .../spark/ml/feature/VectorSlicer.scala | 14 +++++++- .../apache/spark/ml/feature/Word2Vec.scala | 29 ++++++++++++++-- python/pyspark/ml/feature.py | 10 +++--- 28 files changed, 362 insertions(+), 68 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 318c8b8b2f7d6..fa9634fdfa7e9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -35,9 +35,11 @@ import org.apache.spark.sql.types._ * Binarize a column of continuous features given a threshold. */ @Experimental -final class Binarizer(override val uid: String) +@Since("1.4.0") +final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("binarizer")) /** @@ -47,21 +49,26 @@ final class Binarizer(override val uid: String) * Default: 0.0 * @group param */ + @Since("1.4.0") val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold used to binarize continuous features") /** @group getParam */ + @Since("1.4.0") def getThreshold: Double = $(threshold) /** @group setParam */ + @Since("1.4.0") def setThreshold(value: Double): this.type = set(threshold, value) setDefault(threshold -> 0.0) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -96,6 +103,7 @@ final class Binarizer(override val uid: String) } } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val inputType = schema($(inputCol)).dataType val outputColName = $(outputCol) @@ -115,6 +123,7 @@ final class Binarizer(override val uid: String) StructType(schema.fields :+ outCol) } + @Since("1.4.1") override def copy(extra: ParamMap): Binarizer = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index ff988cc815160..caffc39e2be14 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -35,9 +35,11 @@ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} * `Bucketizer` maps a column of continuous features to a column of feature buckets. */ @Experimental -final class Bucketizer(override val uid: String) +@Since("1.4.0") +final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Model[Bucketizer] with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("bucketizer")) /** @@ -48,6 +50,7 @@ final class Bucketizer(override val uid: String) * otherwise, values outside the splits specified will be treated as errors. * @group param */ + @Since("1.4.0") val splits: DoubleArrayParam = new DoubleArrayParam(this, "splits", "Split points for mapping continuous features into buckets. With n+1 splits, there are n " + "buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last " + @@ -57,15 +60,19 @@ final class Bucketizer(override val uid: String) Bucketizer.checkSplits) /** @group getParam */ + @Since("1.4.0") def getSplits: Array[Double] = $(splits) /** @group setParam */ + @Since("1.4.0") def setSplits(value: Array[Double]): this.type = set(splits, value) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -86,16 +93,19 @@ final class Bucketizer(override val uid: String) attr.toStructField() } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) SchemaUtils.appendColumn(schema, prepOutputField(schema)) } + @Since("1.4.1") override def copy(extra: ParamMap): Bucketizer = { defaultCopy[Bucketizer](extra).setParent(parent) } } +@Since("1.6.0") object Bucketizer extends DefaultParamsReadable[Bucketizer] { /** We require splits to be of length >= 3 and to be in strictly increasing order. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index e73a8f5d66087..1c329267d70d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -62,21 +62,27 @@ private[feature] trait ChiSqSelectorParams extends Params * categorical label. */ @Experimental -final class ChiSqSelector(override val uid: String) +@Since("1.6.0") +final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Estimator[ChiSqSelectorModel] with ChiSqSelectorParams with DefaultParamsWritable { + @Since("1.6.0") def this() = this(Identifiable.randomUID("chiSqSelector")) /** @group setParam */ + @Since("1.6.0") def setNumTopFeatures(value: Int): this.type = set(numTopFeatures, value) /** @group setParam */ + @Since("1.6.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.6.0") def setLabelCol(value: String): this.type = set(labelCol, value) @Since("2.0.0") @@ -91,12 +97,14 @@ final class ChiSqSelector(override val uid: String) copyValues(new ChiSqSelectorModel(uid, chiSqSelector).setParent(this)) } + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) SchemaUtils.checkNumericType(schema, $(labelCol)) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } + @Since("1.6.0") override def copy(extra: ParamMap): ChiSqSelector = defaultCopy(extra) } @@ -112,23 +120,28 @@ object ChiSqSelector extends DefaultParamsReadable[ChiSqSelector] { * Model fitted by [[ChiSqSelector]]. */ @Experimental +@Since("1.6.0") final class ChiSqSelectorModel private[ml] ( - override val uid: String, + @Since("1.6.0") override val uid: String, private val chiSqSelector: feature.ChiSqSelectorModel) extends Model[ChiSqSelectorModel] with ChiSqSelectorParams with MLWritable { import ChiSqSelectorModel._ /** list of indices to select (filter). Must be ordered asc */ + @Since("1.6.0") val selectedFeatures: Array[Int] = chiSqSelector.selectedFeatures /** @group setParam */ + @Since("1.6.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.6.0") def setLabelCol(value: String): this.type = set(labelCol, value) @Since("2.0.0") @@ -143,6 +156,7 @@ final class ChiSqSelectorModel private[ml] ( dataset.withColumn($(outputCol), selector(col($(featuresCol))), newField.metadata) } + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) val newField = prepOutputField(schema) @@ -165,6 +179,7 @@ final class ChiSqSelectorModel private[ml] ( newAttributeGroup.toStructField() } + @Since("1.6.0") override def copy(extra: ParamMap): ChiSqSelectorModel = { val copied = new ChiSqSelectorModel(uid, chiSqSelector) copyValues(copied, extra).setParent(parent) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 272567d09cda7..3250fe55980d0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -120,27 +120,35 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit * Extracts a vocabulary from document collections and generates a [[CountVectorizerModel]]. */ @Experimental -class CountVectorizer(override val uid: String) +@Since("1.5.0") +class CountVectorizer @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Estimator[CountVectorizerModel] with CountVectorizerParams with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("cntVec")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setVocabSize(value: Int): this.type = set(vocabSize, value) /** @group setParam */ + @Since("1.5.0") def setMinDF(value: Double): this.type = set(minDF, value) /** @group setParam */ + @Since("1.5.0") def setMinTF(value: Double): this.type = set(minTF, value) /** @group setParam */ + @Since("2.0.0") def setBinary(value: Boolean): this.type = set(binary, value) @Since("2.0.0") @@ -176,10 +184,12 @@ class CountVectorizer(override val uid: String) copyValues(new CountVectorizerModel(uid, vocab).setParent(this)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): CountVectorizer = defaultCopy(extra) } @@ -196,26 +206,34 @@ object CountVectorizer extends DefaultParamsReadable[CountVectorizer] { * @param vocabulary An Array over terms. Only the terms in the vocabulary will be counted. */ @Experimental -class CountVectorizerModel(override val uid: String, val vocabulary: Array[String]) +@Since("1.5.0") +class CountVectorizerModel( + @Since("1.5.0") override val uid: String, + @Since("1.5.0") val vocabulary: Array[String]) extends Model[CountVectorizerModel] with CountVectorizerParams with MLWritable { import CountVectorizerModel._ + @Since("1.5.0") def this(vocabulary: Array[String]) = { this(Identifiable.randomUID("cntVecModel"), vocabulary) set(vocabSize, vocabulary.length) } /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setMinTF(value: Double): this.type = set(minTF, value) /** @group setParam */ + @Since("2.0.0") def setBinary(value: Boolean): this.type = set(binary, value) /** Dictionary created from [[vocabulary]] and its indices, broadcast once for [[transform()]] */ @@ -252,10 +270,12 @@ class CountVectorizerModel(override val uid: String, val vocabulary: Array[Strin dataset.withColumn($(outputCol), vectorizer(col($(inputCol)))) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): CountVectorizerModel = { val copied = new CountVectorizerModel(uid, vocabulary).setParent(parent) copyValues(copied, extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 301358ef1226c..9605145e12c27 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -36,9 +36,11 @@ import org.apache.spark.sql.types.DataType * More information on [[https://en.wikipedia.org/wiki/Discrete_cosine_transform#DCT-II Wikipedia]]. */ @Experimental -class DCT(override val uid: String) +@Since("1.5.0") +class DCT @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends UnaryTransformer[Vector, Vector, DCT] with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("dct")) /** @@ -46,13 +48,16 @@ class DCT(override val uid: String) * Default: false * @group param */ + @Since("1.5.0") def inverse: BooleanParam = new BooleanParam( this, "inverse", "Set transformer to perform inverse DCT") /** @group setParam */ + @Since("1.5.0") def setInverse(value: Boolean): this.type = set(inverse, value) /** @group getParam */ + @Since("1.5.0") def getInverse: Boolean = $(inverse) setDefault(inverse -> false) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 9d2e60fa3f1e4..92fefb1e6c0f3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -33,21 +33,26 @@ import org.apache.spark.sql.types.DataType * multiplier. */ @Experimental -class ElementwiseProduct(override val uid: String) +@Since("2.0.0") +class ElementwiseProduct @Since("2.0.0") (@Since("2.0.0") override val uid: String) extends UnaryTransformer[Vector, Vector, ElementwiseProduct] with DefaultParamsWritable { + @Since("2.0.0") def this() = this(Identifiable.randomUID("elemProd")) /** * the vector to multiply with input vectors * @group param */ + @Since("2.0.0") val scalingVec: Param[Vector] = new Param(this, "scalingVec", "vector for hadamard product") /** @group setParam */ + @Since("2.0.0") def setScalingVec(value: Vector): this.type = set(scalingVec, value) /** @group getParam */ + @Since("2.0.0") def getScalingVec: Vector = getOrDefault(scalingVec) override protected def createTransformFunc: Vector => Vector = { 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 94e1825ba61e4..6ca7336cd048e 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 @@ -38,15 +38,19 @@ import org.apache.spark.sql.types.{ArrayType, StructType} * otherwise the features will not be mapped evenly to the columns. */ @Experimental -class HashingTF(override val uid: String) +@Since("1.2.0") +class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.2.0") def this() = this(Identifiable.randomUID("hashingTF")) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @@ -54,6 +58,7 @@ class HashingTF(override val uid: String) * (default = 2^18^) * @group param */ + @Since("1.2.0") val numFeatures = new IntParam(this, "numFeatures", "number of features (> 0)", ParamValidators.gt(0)) @@ -64,6 +69,7 @@ class HashingTF(override val uid: String) * (default = false) * @group param */ + @Since("2.0.0") val binary = new BooleanParam(this, "binary", "If true, all non zero counts are set to 1. " + "This is useful for discrete probabilistic models that model binary events rather " + "than integer counts") @@ -71,15 +77,19 @@ class HashingTF(override val uid: String) setDefault(numFeatures -> (1 << 18), binary -> false) /** @group getParam */ + @Since("1.2.0") def getNumFeatures: Int = $(numFeatures) /** @group setParam */ + @Since("1.2.0") def setNumFeatures(value: Int): this.type = set(numFeatures, value) /** @group getParam */ + @Since("2.0.0") def getBinary: Boolean = $(binary) /** @group setParam */ + @Since("2.0.0") def setBinary(value: Boolean): this.type = set(binary, value) @Since("2.0.0") @@ -92,6 +102,7 @@ class HashingTF(override val uid: String) dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val inputType = schema($(inputCol)).dataType require(inputType.isInstanceOf[ArrayType], @@ -100,6 +111,7 @@ class HashingTF(override val uid: String) SchemaUtils.appendColumn(schema, attrGroup.toStructField()) } + @Since("1.4.1") override def copy(extra: ParamMap): HashingTF = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 08beda6d7515d..cf03a2845ced5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -64,18 +64,23 @@ private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol * Compute the Inverse Document Frequency (IDF) given a collection of documents. */ @Experimental -final class IDF(override val uid: String) extends Estimator[IDFModel] with IDFBase - with DefaultParamsWritable { +@Since("1.4.0") +final class IDF @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends Estimator[IDFModel] with IDFBase with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("idf")) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.4.0") def setMinDocFreq(value: Int): this.type = set(minDocFreq, value) @Since("2.0.0") @@ -88,10 +93,12 @@ final class IDF(override val uid: String) extends Estimator[IDFModel] with IDFBa copyValues(new IDFModel(uid, idf).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): IDF = defaultCopy(extra) } @@ -107,17 +114,20 @@ object IDF extends DefaultParamsReadable[IDF] { * Model fitted by [[IDF]]. */ @Experimental +@Since("1.4.0") class IDFModel private[ml] ( - override val uid: String, + @Since("1.4.0") override val uid: String, idfModel: feature.IDFModel) extends Model[IDFModel] with IDFBase with MLWritable { import IDFModel._ /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -128,17 +138,19 @@ class IDFModel private[ml] ( dataset.withColumn($(outputCol), idf(col($(inputCol)))) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): IDFModel = { val copied = new IDFModel(uid, idfModel) copyValues(copied, extra).setParent(parent) } /** Returns the IDF vector. */ - @Since("1.6.0") + @Since("2.0.0") def idf: Vector = idfModel.idf.asML @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index fa65ff987917e..dca28b5c5d34f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -42,9 +42,9 @@ import org.apache.spark.sql.types._ * `Vector(6, 8)` if all input features were numeric. If the first feature was instead nominal * with four categories, the output would then be `Vector(0, 0, 0, 0, 3, 4, 0, 0)`. */ -@Since("1.6.0") @Experimental -class Interaction @Since("1.6.0") (override val uid: String) extends Transformer +@Since("1.6.0") +class Interaction @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Transformer with HasInputCols with HasOutputCol with DefaultParamsWritable { @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index 7298a18ff83b5..31a58152671cd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -54,16 +54,19 @@ private[feature] trait MaxAbsScalerParams extends Params with HasInputCol with H * any sparsity. */ @Experimental -class MaxAbsScaler @Since("2.0.0") (override val uid: String) +@Since("2.0.0") +class MaxAbsScaler @Since("2.0.0") (@Since("2.0.0") override val uid: String) extends Estimator[MaxAbsScalerModel] with MaxAbsScalerParams with DefaultParamsWritable { @Since("2.0.0") def this() = this(Identifiable.randomUID("maxAbsScal")) /** @group setParam */ + @Since("2.0.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("2.0.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -81,17 +84,19 @@ class MaxAbsScaler @Since("2.0.0") (override val uid: String) copyValues(new MaxAbsScalerModel(uid, Vectors.dense(maxAbs)).setParent(this)) } + @Since("2.0.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("2.0.0") override def copy(extra: ParamMap): MaxAbsScaler = defaultCopy(extra) } -@Since("1.6.0") +@Since("2.0.0") object MaxAbsScaler extends DefaultParamsReadable[MaxAbsScaler] { - @Since("1.6.0") + @Since("2.0.0") override def load(path: String): MaxAbsScaler = super.load(path) } @@ -101,17 +106,20 @@ object MaxAbsScaler extends DefaultParamsReadable[MaxAbsScaler] { * */ @Experimental +@Since("2.0.0") class MaxAbsScalerModel private[ml] ( - override val uid: String, - val maxAbs: Vector) + @Since("2.0.0") override val uid: String, + @Since("2.0.0") val maxAbs: Vector) extends Model[MaxAbsScalerModel] with MaxAbsScalerParams with MLWritable { import MaxAbsScalerModel._ /** @group setParam */ + @Since("2.0.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("2.0.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -126,10 +134,12 @@ class MaxAbsScalerModel private[ml] ( dataset.withColumn($(outputCol), reScale(col($(inputCol)))) } + @Since("2.0.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("2.0.0") override def copy(extra: ParamMap): MaxAbsScalerModel = { val copied = new MaxAbsScalerModel(uid, maxAbs) copyValues(copied, extra).setParent(parent) @@ -139,7 +149,7 @@ class MaxAbsScalerModel private[ml] ( override def write: MLWriter = new MaxAbsScalerModelWriter(this) } -@Since("1.6.0") +@Since("2.0.0") object MaxAbsScalerModel extends MLReadable[MaxAbsScalerModel] { private[MaxAbsScalerModel] @@ -171,9 +181,9 @@ object MaxAbsScalerModel extends MLReadable[MaxAbsScalerModel] { } } - @Since("1.6.0") + @Since("2.0.0") override def read: MLReader[MaxAbsScalerModel] = new MaxAbsScalerModelReader - @Since("1.6.0") + @Since("2.0.0") override def load(path: String): MaxAbsScalerModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index a27bed5333c56..dd5a1f9b41fc8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -85,23 +85,29 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H * transformer will be DenseVector even for sparse input. */ @Experimental -class MinMaxScaler(override val uid: String) +@Since("1.5.0") +class MinMaxScaler @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Estimator[MinMaxScalerModel] with MinMaxScalerParams with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("minMaxScal")) setDefault(min -> 0.0, max -> 1.0) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setMin(value: Double): this.type = set(min, value) /** @group setParam */ + @Since("1.5.0") def setMax(value: Double): this.type = set(max, value) @Since("2.0.0") @@ -114,10 +120,12 @@ class MinMaxScaler(override val uid: String) copyValues(new MinMaxScalerModel(uid, summary.min, summary.max).setParent(this)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): MinMaxScaler = defaultCopy(extra) } @@ -138,24 +146,29 @@ object MinMaxScaler extends DefaultParamsReadable[MinMaxScaler] { * TODO: The transformer does not yet set the metadata in the output column (SPARK-8529). */ @Experimental +@Since("1.5.0") class MinMaxScalerModel private[ml] ( - override val uid: String, - val originalMin: Vector, - val originalMax: Vector) + @Since("1.5.0") override val uid: String, + @Since("2.0.0") val originalMin: Vector, + @Since("2.0.0") val originalMax: Vector) extends Model[MinMaxScalerModel] with MinMaxScalerParams with MLWritable { import MinMaxScalerModel._ /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setMin(value: Double): this.type = set(min, value) /** @group setParam */ + @Since("1.5.0") def setMax(value: Double): this.type = set(max, value) @Since("2.0.0") @@ -181,10 +194,12 @@ class MinMaxScalerModel private[ml] ( dataset.withColumn($(outputCol), reScale(col($(inputCol)))) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): MinMaxScalerModel = { val copied = new MinMaxScalerModel(uid, originalMin, originalMax) copyValues(copied, extra).setParent(parent) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index f8bc7e3f0c031..9c1f1ad443bba 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -35,9 +35,11 @@ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} * returned. */ @Experimental -class NGram(override val uid: String) +@Since("1.5.0") +class NGram @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends UnaryTransformer[Seq[String], Seq[String], NGram] with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("ngram")) /** @@ -45,13 +47,16 @@ class NGram(override val uid: String) * Default: 2, bigram features * @group param */ + @Since("1.5.0") val n: IntParam = new IntParam(this, "n", "number elements per n-gram (>=1)", ParamValidators.gtEq(1)) /** @group setParam */ + @Since("1.5.0") def setN(value: Int): this.type = set(n, value) /** @group getParam */ + @Since("1.5.0") def getN: Int = $(n) setDefault(n -> 2) 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 942ac7ebdb3bd..9a4e682890f12 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 @@ -31,9 +31,11 @@ import org.apache.spark.sql.types.DataType * Normalize a vector to have unit norm using the given p-norm. */ @Experimental -class Normalizer(override val uid: String) +@Since("2.0.0") +class Normalizer @Since("2.0.0") (@Since("2.0.0") override val uid: String) extends UnaryTransformer[Vector, Vector, Normalizer] with DefaultParamsWritable { + @Since("2.0.0") def this() = this(Identifiable.randomUID("normalizer")) /** @@ -41,14 +43,17 @@ class Normalizer(override val uid: String) * (default: p = 2) * @group param */ + @Since("2.0.0") val p = new DoubleParam(this, "p", "the p norm value", ParamValidators.gtEq(1)) setDefault(p -> 2.0) /** @group getParam */ + @Since("2.0.0") def getP: Double = $(p) /** @group setParam */ + @Since("2.0.0") def setP(value: Double): this.type = set(p, value) override protected def createTransformFunc: Vector => Vector = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 3d1e6dd818829..4fafc1e349c22 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -43,28 +43,35 @@ import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} * @see [[StringIndexer]] for converting categorical values into category indices */ @Experimental -class OneHotEncoder(override val uid: String) extends Transformer +@Since("1.4.0") +class OneHotEncoder @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("oneHot")) /** * Whether to drop the last category in the encoded vector (default: true) * @group param */ + @Since("1.4.0") final val dropLast: BooleanParam = new BooleanParam(this, "dropLast", "whether to drop the last category") setDefault(dropLast -> true) /** @group setParam */ + @Since("1.4.0") def setDropLast(value: Boolean): this.type = set(dropLast, value) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val inputColName = $(inputCol) val outputColName = $(outputCol) @@ -168,6 +175,7 @@ class OneHotEncoder(override val uid: String) extends Transformer dataset.select(col("*"), encode(col(inputColName).cast(DoubleType)).as(outputColName, metadata)) } + @Since("1.4.1") override def copy(extra: ParamMap): OneHotEncoder = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 2f667af9d10bd..b89c85991f39a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -65,18 +65,24 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC * principal components. */ @Experimental -class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams - with DefaultParamsWritable { +@Since("1.5.0") +class PCA @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends Estimator[PCAModel] with PCAParams with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("pca")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setK(value: Int): this.type = set(k, value) /** @@ -93,10 +99,12 @@ class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams copyValues(new PCAModel(uid, pcaModel.pc, pcaModel.explainedVariance).setParent(this)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): PCA = defaultCopy(extra) } @@ -116,18 +124,21 @@ object PCA extends DefaultParamsReadable[PCA] { * each principal component. */ @Experimental +@Since("1.5.0") class PCAModel private[ml] ( - override val uid: String, - val pc: DenseMatrix, - val explainedVariance: DenseVector) + @Since("1.5.0") override val uid: String, + @Since("2.0.0") val pc: DenseMatrix, + @Since("2.0.0") val explainedVariance: DenseVector) extends Model[PCAModel] with PCAParams with MLWritable { import PCAModel._ /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @@ -149,10 +160,12 @@ class PCAModel private[ml] ( dataset.withColumn($(outputCol), pcaOp(col($(inputCol)))) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): PCAModel = { val copied = new PCAModel(uid, pc, explainedVariance) copyValues(copied, extra).setParent(parent) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index a01867701bd8b..026014c7d64e3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -35,9 +35,11 @@ import org.apache.spark.sql.types.DataType * `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. */ @Experimental -class PolynomialExpansion(override val uid: String) +@Since("2.0.0") +class PolynomialExpansion @Since("2.0.0") (@Since("2.0.0") override val uid: String) extends UnaryTransformer[Vector, Vector, PolynomialExpansion] with DefaultParamsWritable { + @Since("2.0.0") def this() = this(Identifiable.randomUID("poly")) /** @@ -45,15 +47,18 @@ class PolynomialExpansion(override val uid: String) * Default: 2 * @group param */ + @Since("2.0.0") val degree = new IntParam(this, "degree", "the polynomial degree to expand (>= 1)", ParamValidators.gtEq(1)) setDefault(degree -> 2) /** @group getParam */ + @Since("2.0.0") def getDegree: Int = $(degree) /** @group setParam */ + @Since("2.0.0") def setDegree(value: Int): this.type = set(degree, value) override protected def createTransformFunc: Vector => Vector = { v => @@ -62,6 +67,7 @@ class PolynomialExpansion(override val uid: String) override protected def outputDataType: DataType = new VectorUDT() + @Since("1.4.1") override def copy(extra: ParamMap): PolynomialExpansion = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 1fefaa1fdd8df..96b8e7d9f7faf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -74,23 +74,30 @@ private[feature] trait QuantileDiscretizerBase extends Params * covering all real values. */ @Experimental -final class QuantileDiscretizer(override val uid: String) +@Since("1.6.0") +final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Estimator[Bucketizer] with QuantileDiscretizerBase with DefaultParamsWritable { + @Since("1.6.0") def this() = this(Identifiable.randomUID("quantileDiscretizer")) /** @group setParam */ + @Since("2.0.0") def setRelativeError(value: Double): this.type = set(relativeError, value) /** @group setParam */ + @Since("1.6.0") def setNumBuckets(value: Int): this.type = set(numBuckets, value) /** @group setParam */ + @Since("1.6.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) val inputFields = schema.fields @@ -112,6 +119,7 @@ final class QuantileDiscretizer(override val uid: String) copyValues(bucketizer.setParent(this)) } + @Since("1.6.0") override def copy(extra: ParamMap): QuantileDiscretizer = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index a7ca0fe252b0d..546dc7e8c08ff 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -70,15 +70,18 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol { * will be created from the specified response variable in the formula. */ @Experimental -class RFormula(override val uid: String) +@Since("1.5.0") +class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Estimator[RFormulaModel] with RFormulaBase with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("rFormula")) /** * R formula parameter. The formula is provided in string form. * @group param */ + @Since("1.5.0") val formula: Param[String] = new Param(this, "formula", "R model formula") /** @@ -86,15 +89,19 @@ class RFormula(override val uid: String) * @group setParam * @param value an R formula in string form (e.g. "y ~ x + z") */ + @Since("1.5.0") def setFormula(value: String): this.type = set(formula, value) /** @group getParam */ + @Since("1.5.0") def getFormula: String = $(formula) /** @group setParam */ + @Since("1.5.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.5.0") def setLabelCol(value: String): this.type = set(labelCol, value) /** Whether the formula specifies fitting an intercept. */ @@ -170,6 +177,7 @@ class RFormula(override val uid: String) copyValues(new RFormulaModel(uid, resolvedFormula, pipelineModel).setParent(this)) } + @Since("1.5.0") // optimistic schema; does not contain any ML attributes override def transformSchema(schema: StructType): StructType = { if (hasLabelCol(schema)) { @@ -180,8 +188,10 @@ class RFormula(override val uid: String) } } + @Since("1.5.0") override def copy(extra: ParamMap): RFormula = defaultCopy(extra) + @Since("2.0.0") override def toString: String = s"RFormula(${get(formula).getOrElse("")}) (uid=$uid)" } @@ -201,8 +211,9 @@ object RFormula extends DefaultParamsReadable[RFormula] { * @param pipelineModel the fitted feature model, including factor to index mappings. */ @Experimental +@Since("1.5.0") class RFormulaModel private[feature]( - override val uid: String, + @Since("1.5.0") override val uid: String, private[ml] val resolvedFormula: ResolvedRFormula, private[ml] val pipelineModel: PipelineModel) extends Model[RFormulaModel] with RFormulaBase with MLWritable { @@ -213,6 +224,7 @@ class RFormulaModel private[feature]( transformLabel(pipelineModel.transform(dataset)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { checkCanTransform(schema) val withFeatures = pipelineModel.transformSchema(schema) @@ -231,9 +243,11 @@ class RFormulaModel private[feature]( } } + @Since("1.5.0") override def copy(extra: ParamMap): RFormulaModel = copyValues( new RFormulaModel(uid, resolvedFormula, pipelineModel)) + @Since("2.0.0") override def toString: String = s"RFormulaModel($resolvedFormula) (uid=$uid)" private def transformLabel(dataset: Dataset[_]): DataFrame = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index bd8f9494fb193..b8715746fee5b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.StructType */ @Experimental @Since("1.6.0") -class SQLTransformer @Since("1.6.0") (override val uid: String) extends Transformer +class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Transformer with DefaultParamsWritable { @Since("1.6.0") 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 7cec369c23a8f..5e1bacf876caf 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 @@ -85,21 +85,28 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with * which is computed as the square root of the unbiased sample variance. */ @Experimental -class StandardScaler(override val uid: String) extends Estimator[StandardScalerModel] - with StandardScalerParams with DefaultParamsWritable { +@Since("1.2.0") +class StandardScaler @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends Estimator[StandardScalerModel] with StandardScalerParams with DefaultParamsWritable { + @Since("1.2.0") def this() = this(Identifiable.randomUID("stdScal")) /** @group setParam */ + @Since("1.2.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.2.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.4.0") def setWithMean(value: Boolean): this.type = set(withMean, value) /** @group setParam */ + @Since("1.4.0") def setWithStd(value: Boolean): this.type = set(withStd, value) @Since("2.0.0") @@ -113,10 +120,12 @@ class StandardScaler(override val uid: String) extends Estimator[StandardScalerM copyValues(new StandardScalerModel(uid, scalerModel.std, scalerModel.mean).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): StandardScaler = defaultCopy(extra) } @@ -135,18 +144,21 @@ object StandardScaler extends DefaultParamsReadable[StandardScaler] { * @param mean Mean of the StandardScalerModel */ @Experimental +@Since("1.2.0") class StandardScalerModel private[ml] ( - override val uid: String, - val std: Vector, - val mean: Vector) + @Since("1.4.0") override val uid: String, + @Since("2.0.0") val std: Vector, + @Since("2.0.0") val mean: Vector) extends Model[StandardScalerModel] with StandardScalerParams with MLWritable { import StandardScalerModel._ /** @group setParam */ + @Since("1.2.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.2.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -161,10 +173,12 @@ class StandardScalerModel private[ml] ( dataset.withColumn($(outputCol), scale(col($(inputCol)))) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): StandardScalerModel = { val copied = new StandardScalerModel(uid, std, mean) copyValues(copied, extra).setParent(parent) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala index 11864cb8f439a..1a6f42f773cd7 100755 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -33,15 +33,19 @@ import org.apache.spark.sql.types.{ArrayType, StringType, StructType} * @see [[http://en.wikipedia.org/wiki/Stop_words]] */ @Experimental -class StopWordsRemover(override val uid: String) +@Since("1.5.0") +class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("stopWords")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @@ -50,13 +54,16 @@ class StopWordsRemover(override val uid: String) * @see [[StopWordsRemover.loadDefaultStopWords()]] * @group param */ + @Since("1.5.0") val stopWords: StringArrayParam = new StringArrayParam(this, "stopWords", "the words to be filtered out") /** @group setParam */ + @Since("1.5.0") def setStopWords(value: Array[String]): this.type = set(stopWords, value) /** @group getParam */ + @Since("1.5.0") def getStopWords: Array[String] = $(stopWords) /** @@ -64,13 +71,16 @@ class StopWordsRemover(override val uid: String) * Default: false * @group param */ + @Since("1.5.0") val caseSensitive: BooleanParam = new BooleanParam(this, "caseSensitive", "whether to do a case-sensitive comparison over the stop words") /** @group setParam */ + @Since("1.5.0") def setCaseSensitive(value: Boolean): this.type = set(caseSensitive, value) /** @group getParam */ + @Since("1.5.0") def getCaseSensitive: Boolean = $(caseSensitive) setDefault(stopWords -> StopWordsRemover.loadDefaultStopWords("english"), caseSensitive -> false) @@ -95,6 +105,7 @@ class StopWordsRemover(override val uid: String) dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { val inputType = schema($(inputCol)).dataType require(inputType.sameType(ArrayType(StringType)), @@ -102,6 +113,7 @@ class StopWordsRemover(override val uid: String) SchemaUtils.appendColumn(schema, $(outputCol), inputType, schema($(inputCol)).nullable) } + @Since("1.5.0") override def copy(extra: ParamMap): StopWordsRemover = defaultCopy(extra) } 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 cc0571fd7e39b..0f7337ce6b554 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 @@ -64,22 +64,27 @@ private[feature] trait StringIndexerBase extends Params with HasInputCol with Ha * @see [[IndexToString]] for the inverse transformation */ @Experimental -class StringIndexer(override val uid: String) extends Estimator[StringIndexerModel] +@Since("1.4.0") +class StringIndexer @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) extends Estimator[StringIndexerModel] with StringIndexerBase with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("strIdx")) /** @group setParam */ + @Since("1.6.0") def setHandleInvalid(value: String): this.type = set(handleInvalid, value) setDefault(handleInvalid, "error") /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - @Since("2.0.0") override def fit(dataset: Dataset[_]): StringIndexerModel = { val counts = dataset.select(col($(inputCol)).cast(StringType)) @@ -90,10 +95,12 @@ class StringIndexer(override val uid: String) extends Estimator[StringIndexerMod copyValues(new StringIndexerModel(uid, labels).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): StringIndexer = defaultCopy(extra) } @@ -115,13 +122,15 @@ object StringIndexer extends DefaultParamsReadable[StringIndexer] { * @param labels Ordered list of labels, corresponding to indices to be assigned. */ @Experimental +@Since("1.4.0") class StringIndexerModel ( - override val uid: String, - val labels: Array[String]) + @Since("1.4.0") override val uid: String, + @Since("1.5.0") val labels: Array[String]) extends Model[StringIndexerModel] with StringIndexerBase with MLWritable { import StringIndexerModel._ + @Since("1.5.0") def this(labels: Array[String]) = this(Identifiable.randomUID("strIdx"), labels) private val labelToIndex: OpenHashMap[String, Double] = { @@ -136,13 +145,16 @@ class StringIndexerModel ( } /** @group setParam */ + @Since("1.6.0") def setHandleInvalid(value: String): this.type = set(handleInvalid, value) setDefault(handleInvalid, "error") /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -177,6 +189,7 @@ class StringIndexerModel ( indexer(dataset($(inputCol)).cast(StringType)).as($(outputCol), metadata)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { if (schema.fieldNames.contains($(inputCol))) { validateAndTransformSchema(schema) @@ -186,6 +199,7 @@ class StringIndexerModel ( } } + @Since("1.4.1") override def copy(extra: ParamMap): StringIndexerModel = { val copied = new StringIndexerModel(uid, labels) copyValues(copied, extra).setParent(parent) @@ -245,19 +259,24 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { * @see [[StringIndexer]] for converting strings into indices */ @Experimental -class IndexToString private[ml] (override val uid: String) +@Since("1.5.0") +class IndexToString private[ml] (@Since("1.5.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("idxToStr")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setLabels(value: Array[String]): this.type = set(labels, value) /** @@ -266,13 +285,16 @@ class IndexToString private[ml] (override val uid: String) * Default: Not specified, in which case [[inputCol]] metadata is used for labels. * @group param */ + @Since("1.5.0") final val labels: StringArrayParam = new StringArrayParam(this, "labels", "Optional array of labels specifying index-string mapping." + " If not provided or if empty, then metadata from inputCol is used instead.") /** @group getParam */ + @Since("1.5.0") final def getLabels: Array[String] = $(labels) + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { val inputColName = $(inputCol) val inputDataType = schema(inputColName).dataType @@ -310,6 +332,7 @@ class IndexToString private[ml] (override val uid: String) indexer(dataset($(inputCol)).cast(DoubleType)).as(outputColName)) } + @Since("1.5.0") override def copy(extra: ParamMap): IndexToString = { defaultCopy(extra) } 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 8456a0e915804..010c948749f3b 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 @@ -30,9 +30,11 @@ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} * @see [[RegexTokenizer]] */ @Experimental -class Tokenizer(override val uid: String) +@Since("1.2.0") +class Tokenizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[String, Seq[String], Tokenizer] with DefaultParamsWritable { + @Since("1.2.0") def this() = this(Identifiable.randomUID("tok")) override protected def createTransformFunc: String => Seq[String] = { @@ -45,6 +47,7 @@ class Tokenizer(override val uid: String) override protected def outputDataType: DataType = new ArrayType(StringType, true) + @Since("1.4.1") override def copy(extra: ParamMap): Tokenizer = defaultCopy(extra) } @@ -63,9 +66,11 @@ object Tokenizer extends DefaultParamsReadable[Tokenizer] { * It returns an array of strings that can be empty. */ @Experimental -class RegexTokenizer(override val uid: String) +@Since("1.4.0") +class RegexTokenizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[String, Seq[String], RegexTokenizer] with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("regexTok")) /** @@ -73,13 +78,16 @@ class RegexTokenizer(override val uid: String) * Default: 1, to avoid returning empty strings * @group param */ + @Since("1.4.0") val minTokenLength: IntParam = new IntParam(this, "minTokenLength", "minimum token length (>= 0)", ParamValidators.gtEq(0)) /** @group setParam */ + @Since("1.4.0") def setMinTokenLength(value: Int): this.type = set(minTokenLength, value) /** @group getParam */ + @Since("1.4.0") def getMinTokenLength: Int = $(minTokenLength) /** @@ -87,12 +95,15 @@ class RegexTokenizer(override val uid: String) * Default: true * @group param */ + @Since("1.4.0") val gaps: BooleanParam = new BooleanParam(this, "gaps", "Set regex to match gaps or tokens") /** @group setParam */ + @Since("1.4.0") def setGaps(value: Boolean): this.type = set(gaps, value) /** @group getParam */ + @Since("1.4.0") def getGaps: Boolean = $(gaps) /** @@ -100,12 +111,15 @@ class RegexTokenizer(override val uid: String) * Default: `"\\s+"` * @group param */ + @Since("1.4.0") val pattern: Param[String] = new Param(this, "pattern", "regex pattern used for tokenizing") /** @group setParam */ + @Since("1.4.0") def setPattern(value: String): this.type = set(pattern, value) /** @group getParam */ + @Since("1.4.0") def getPattern: String = $(pattern) /** @@ -113,13 +127,16 @@ class RegexTokenizer(override val uid: String) * Default: true * @group param */ + @Since("1.6.0") final val toLowercase: BooleanParam = new BooleanParam(this, "toLowercase", "whether to convert all characters to lowercase before tokenizing.") /** @group setParam */ + @Since("1.6.0") def setToLowercase(value: Boolean): this.type = set(toLowercase, value) /** @group getParam */ + @Since("1.6.0") def getToLowercase: Boolean = $(toLowercase) setDefault(minTokenLength -> 1, gaps -> true, pattern -> "\\s+", toLowercase -> true) @@ -138,6 +155,7 @@ class RegexTokenizer(override val uid: String) override protected def outputDataType: DataType = new ArrayType(StringType, true) + @Since("1.4.1") override def copy(extra: ParamMap): RegexTokenizer = defaultCopy(extra) } 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 1bc24202b7615..4939dabd987ec 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 @@ -36,15 +36,19 @@ import org.apache.spark.sql.types._ * A feature transformer that merges multiple columns into a vector column. */ @Experimental -class VectorAssembler(override val uid: String) +@Since("1.4.0") +class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCols with HasOutputCol with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("vecAssembler")) /** @group setParam */ + @Since("1.4.0") def setInputCols(value: Array[String]): this.type = set(inputCols, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -106,6 +110,7 @@ class VectorAssembler(override val uid: String) dataset.select(col("*"), assembleFunc(struct(args: _*)).as($(outputCol), metadata)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val inputColNames = $(inputCols) val outputColName = $(outputCol) @@ -122,6 +127,7 @@ class VectorAssembler(override val uid: String) StructType(schema.fields :+ new StructField(outputColName, new VectorUDT, true)) } + @Since("1.4.1") override def copy(extra: ParamMap): VectorAssembler = defaultCopy(extra) } 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 d814528ec48d1..52db996c841b3 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 @@ -94,18 +94,24 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu * - Add option for allowing unknown categories. */ @Experimental -class VectorIndexer(override val uid: String) extends Estimator[VectorIndexerModel] - with VectorIndexerParams with DefaultParamsWritable { +@Since("1.4.0") +class VectorIndexer @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends Estimator[VectorIndexerModel] with VectorIndexerParams with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("vecIdx")) /** @group setParam */ + @Since("1.4.0") def setMaxCategories(value: Int): this.type = set(maxCategories, value) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -126,6 +132,7 @@ class VectorIndexer(override val uid: String) extends Estimator[VectorIndexerMod copyValues(model) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { // We do not transfer feature metadata since we do not know what types of features we will // produce in transform(). @@ -136,6 +143,7 @@ class VectorIndexer(override val uid: String) extends Estimator[VectorIndexerMod SchemaUtils.appendColumn(schema, $(outputCol), dataType) } + @Since("1.4.1") override def copy(extra: ParamMap): VectorIndexer = defaultCopy(extra) } @@ -256,15 +264,17 @@ object VectorIndexer extends DefaultParamsReadable[VectorIndexer] { * If a feature is not in this map, it is treated as continuous. */ @Experimental +@Since("1.4.0") class VectorIndexerModel private[ml] ( - override val uid: String, - val numFeatures: Int, - val categoryMaps: Map[Int, Map[Double, Int]]) + @Since("1.4.0") override val uid: String, + @Since("1.4.0") val numFeatures: Int, + @Since("1.4.0") val categoryMaps: Map[Int, Map[Double, Int]]) extends Model[VectorIndexerModel] with VectorIndexerParams with MLWritable { import VectorIndexerModel._ /** Java-friendly version of [[categoryMaps]] */ + @Since("1.4.0") def javaCategoryMaps: JMap[JInt, JMap[JDouble, JInt]] = { categoryMaps.mapValues(_.asJava).asJava.asInstanceOf[JMap[JInt, JMap[JDouble, JInt]]] } @@ -342,9 +352,11 @@ class VectorIndexerModel private[ml] ( } /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -356,6 +368,7 @@ class VectorIndexerModel private[ml] ( dataset.withColumn($(outputCol), newCol, newField.metadata) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val dataType = new VectorUDT require(isDefined(inputCol), @@ -415,6 +428,7 @@ class VectorIndexerModel private[ml] ( newAttributeGroup.toStructField() } + @Since("1.4.1") override def copy(extra: ParamMap): VectorIndexerModel = { val copied = new VectorIndexerModel(uid, numFeatures, categoryMaps) copyValues(copied, extra).setParent(parent) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index 103738cd91c09..6769e490c51c7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -41,9 +41,11 @@ import org.apache.spark.sql.types.StructType * followed by the selected names (in the order given). */ @Experimental -final class VectorSlicer(override val uid: String) +@Since("1.5.0") +final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("vectorSlicer")) /** @@ -52,6 +54,7 @@ final class VectorSlicer(override val uid: String) * Default: Empty array * @group param */ + @Since("1.5.0") val indices = new IntArrayParam(this, "indices", "An array of indices to select features from a vector column." + " There can be no overlap with names.", VectorSlicer.validIndices) @@ -59,9 +62,11 @@ final class VectorSlicer(override val uid: String) setDefault(indices -> Array.empty[Int]) /** @group getParam */ + @Since("1.5.0") def getIndices: Array[Int] = $(indices) /** @group setParam */ + @Since("1.5.0") def setIndices(value: Array[Int]): this.type = set(indices, value) /** @@ -71,6 +76,7 @@ final class VectorSlicer(override val uid: String) * Default: Empty Array * @group param */ + @Since("1.5.0") val names = new StringArrayParam(this, "names", "An array of feature names to select features from a vector column." + " There can be no overlap with indices.", VectorSlicer.validNames) @@ -78,15 +84,19 @@ final class VectorSlicer(override val uid: String) setDefault(names -> Array.empty[String]) /** @group getParam */ + @Since("1.5.0") def getNames: Array[String] = $(names) /** @group setParam */ + @Since("1.5.0") def setNames(value: Array[String]): this.type = set(names, value) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) @Since("2.0.0") @@ -134,6 +144,7 @@ final class VectorSlicer(override val uid: String) indFeatures ++ nameFeatures } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { require($(indices).length > 0 || $(names).length > 0, s"VectorSlicer requires that at least one feature be selected.") @@ -148,6 +159,7 @@ final class VectorSlicer(override val uid: String) StructType(outputFields) } + @Since("1.5.0") override def copy(extra: ParamMap): VectorSlicer = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 33515b22400be..05c4f2f1a7891 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -120,39 +120,52 @@ private[feature] trait Word2VecBase extends Params * natural language processing or machine learning process. */ @Experimental -final class Word2Vec(override val uid: String) extends Estimator[Word2VecModel] with Word2VecBase - with DefaultParamsWritable { +@Since("1.4.0") +final class Word2Vec @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends Estimator[Word2VecModel] with Word2VecBase with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("w2v")) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.4.0") def setVectorSize(value: Int): this.type = set(vectorSize, value) /** @group expertSetParam */ + @Since("1.6.0") def setWindowSize(value: Int): this.type = set(windowSize, value) /** @group setParam */ + @Since("1.4.0") def setStepSize(value: Double): this.type = set(stepSize, value) /** @group setParam */ + @Since("1.4.0") def setNumPartitions(value: Int): this.type = set(numPartitions, value) /** @group setParam */ + @Since("1.4.0") def setMaxIter(value: Int): this.type = set(maxIter, value) /** @group setParam */ + @Since("1.4.0") def setSeed(value: Long): this.type = set(seed, value) /** @group setParam */ + @Since("1.4.0") def setMinCount(value: Int): this.type = set(minCount, value) /** @group setParam */ + @Since("2.0.0") def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, value) @Since("2.0.0") @@ -172,10 +185,12 @@ final class Word2Vec(override val uid: String) extends Estimator[Word2VecModel] copyValues(new Word2VecModel(uid, wordVectors).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra) } @@ -191,8 +206,9 @@ object Word2Vec extends DefaultParamsReadable[Word2Vec] { * Model fitted by [[Word2Vec]]. */ @Experimental +@Since("1.4.0") class Word2VecModel private[ml] ( - override val uid: String, + @Since("1.4.0") override val uid: String, @transient private val wordVectors: feature.Word2VecModel) extends Model[Word2VecModel] with Word2VecBase with MLWritable { @@ -202,6 +218,7 @@ class Word2VecModel private[ml] ( * Returns a dataframe with two fields, "word" and "vector", with "word" being a String and * and the vector the DenseVector that it is mapped to. */ + @Since("1.5.0") @transient lazy val getVectors: DataFrame = { val spark = SparkSession.builder().getOrCreate() val wordVec = wordVectors.getVectors.mapValues(vec => Vectors.dense(vec.map(_.toDouble))) @@ -213,6 +230,7 @@ class Word2VecModel private[ml] ( * Returns a dataframe with the words and the cosine similarities between the * synonyms and the given word. */ + @Since("1.5.0") def findSynonyms(word: String, num: Int): DataFrame = { findSynonyms(wordVectors.transform(word), num) } @@ -222,15 +240,18 @@ class Word2VecModel private[ml] ( * of the word. Returns a dataframe with the words and the cosine similarities between the * synonyms and the given word vector. */ + @Since("1.5.0") def findSynonyms(word: Vector, num: Int): DataFrame = { val spark = SparkSession.builder().getOrCreate() spark.createDataFrame(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") } /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @@ -262,10 +283,12 @@ class Word2VecModel private[ml] ( dataset.withColumn($(outputCol), word2Vec(col($(inputCol)))) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): Word2VecModel = { val copied = new Word2VecModel(uid, wordVectors) copyValues(copied, extra).setParent(parent) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index a28764a7528dc..1e9ec0fbb411c 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -149,7 +149,7 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav >>> loadedBucketizer.getSplits() == bucketizer.getSplits() True - .. versionadded:: 1.3.0 + .. versionadded:: 1.4.0 """ splits = \ @@ -486,14 +486,14 @@ def setParams(self, scalingVec=None, inputCol=None, outputCol=None): kwargs = self.setParams._input_kwargs return self._set(**kwargs) - @since("1.5.0") + @since("2.0.0") def setScalingVec(self, value): """ Sets the value of :py:attr:`scalingVec`. """ return self._set(scalingVec=value) - @since("1.5.0") + @since("2.0.0") def getScalingVec(self): """ Gets the value of scalingVec or its default value. @@ -1584,7 +1584,7 @@ class StandardScalerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ @property - @since("1.5.0") + @since("2.0.0") def std(self): """ Standard deviation of the StandardScalerModel. @@ -1592,7 +1592,7 @@ def std(self): return self._call_java("std") @property - @since("1.5.0") + @since("2.0.0") def mean(self): """ Mean of the StandardScalerModel. From 0499ed961838686acccefc08a42efa523f1648dd Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Tue, 21 Jun 2016 00:47:36 -0700 Subject: [PATCH 0750/1470] [SPARK-16045][ML][DOC] Spark 2.0 ML.feature: doc update for stopwords and binarizer ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-16045 2.0 Audit: Update document for StopWordsRemover and Binarizer. ## How was this patch tested? manual review for doc Author: Yuhao Yang Author: Yuhao Yang Closes #13375 from hhbyyh/stopdoc. (cherry picked from commit a58f40239444d42adbc480ddde02cbb02a79bbe4) Signed-off-by: Xiangrui Meng --- docs/ml-features.md | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index 3db24a3840599..3cb26443b9516 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -251,11 +251,12 @@ frequently and don't carry as much meaning. `StopWordsRemover` takes as input a sequence of strings (e.g. the output of a [Tokenizer](ml-features.html#tokenizer)) and drops all the stop words from the input sequences. The list of stopwords is specified by -the `stopWords` parameter. We provide [a list of stop -words](http://ir.dcs.gla.ac.uk/resources/linguistic_utils/stop_words) by -default, accessible by calling `getStopWords` on a newly instantiated -`StopWordsRemover` instance. A boolean parameter `caseSensitive` indicates -if the matches should be case sensitive (false by default). +the `stopWords` parameter. Default stop words for some languages are accessible +by calling `StopWordsRemover.loadDefaultStopWords(language)`, for which available +options are "danish", "dutch", "english", "finnish", "french", "german", "hungarian", +"italian", "norwegian", "portuguese", "russian", "spanish", "swedish" and "turkish". +A boolean parameter `caseSensitive` indicates if the matches should be case sensitive +(false by default). **Examples** @@ -346,7 +347,10 @@ for more details on the API. Binarization is the process of thresholding numerical features to binary (0/1) features. -`Binarizer` takes the common parameters `inputCol` and `outputCol`, as well as the `threshold` for binarization. Feature values greater than the threshold are binarized to 1.0; values equal to or less than the threshold are binarized to 0.0. +`Binarizer` takes the common parameters `inputCol` and `outputCol`, as well as the `threshold` +for binarization. Feature values greater than the threshold are binarized to 1.0; values equal +to or less than the threshold are binarized to 0.0. Both Vector and Double types are supported +for `inputCol`.
    From 34a8e23c739532cd2cb059d9d4e785368d6d0a98 Mon Sep 17 00:00:00 2001 From: bomeng Date: Tue, 21 Jun 2016 08:51:43 +0100 Subject: [PATCH 0751/1470] [SPARK-16084][SQL] Minor comments update for "DESCRIBE" table ## What changes were proposed in this pull request? 1. FORMATTED is actually supported, but partition is not supported; 2. Remove parenthesis as it is not necessary just like anywhere else. ## How was this patch tested? Minor issue. I do not think it needs a test case! Author: bomeng Closes #13791 from bomeng/SPARK-16084. (cherry picked from commit f3a768b7b96f00f33d2fe4e6c0bf4acf373ad4f4) Signed-off-by: Sean Owen --- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 154c25adfac43..2ae8380644acd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -279,15 +279,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * Create a [[DescribeTableCommand]] logical plan. */ override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { - // FORMATTED and columns are not supported. Return null and let the parser decide what to do - // with this (create an exception or pass it on to a different system). + // Describe partition and column are not supported yet. Return null and let the parser decide + // what to do with this (create an exception or pass it on to a different system). if (ctx.describeColName != null || ctx.partitionSpec != null) { null } else { DescribeTableCommand( visitTableIdentifier(ctx.tableIdentifier), ctx.EXTENDED != null, - ctx.FORMATTED() != null) + ctx.FORMATTED != null) } } From 282be71dd69da87e7f3885b803fc9a4795bbc3fb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 21 Jun 2016 08:31:15 -0700 Subject: [PATCH 0752/1470] [SPARK-15177][.1][R] make SparkR model params and default values consistent with MLlib ## What changes were proposed in this pull request? This PR is a subset of #13023 by yanboliang to make SparkR model param names and default values consistent with MLlib. I tried to avoid other changes from #13023 to keep this PR minimal. I will send a follow-up PR to improve the documentation. Main changes: * `spark.glm`: epsilon -> tol, maxit -> maxIter * `spark.kmeans`: default k -> 2, default maxIter -> 20, default initMode -> "k-means||" * `spark.naiveBayes`: laplace -> smoothing, default 1.0 ## How was this patch tested? Existing unit tests. Author: Xiangrui Meng Closes #13801 from mengxr/SPARK-15177.1. (cherry picked from commit 4f83ca1059a3b580fca3f006974ff5ac4d5212a1) Signed-off-by: Xiangrui Meng --- R/pkg/R/mllib.R | 74 +++++++++---------- R/pkg/inst/tests/testthat/test_mllib.R | 4 +- .../GeneralizedLinearRegressionWrapper.scala | 8 +- .../apache/spark/ml/r/NaiveBayesWrapper.scala | 4 +- 4 files changed, 44 insertions(+), 46 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 74dba8fe966fb..b83b3b3d3f6d4 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -64,8 +64,8 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' This can be a character string naming a family function, a family function or #' the result of a call to a family function. Refer R family at #' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. -#' @param epsilon Positive convergence tolerance of iterations. -#' @param maxit Integer giving the maximal number of IRLS iterations. +#' @param tol Positive convergence tolerance of iterations. +#' @param maxIter Integer giving the maximal number of IRLS iterations. #' @return a fitted generalized linear model #' @rdname spark.glm #' @export @@ -74,32 +74,30 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' sparkR.session() #' data(iris) #' df <- createDataFrame(iris) -#' model <- spark.glm(df, Sepal_Length ~ Sepal_Width, family="gaussian") +#' model <- spark.glm(df, Sepal_Length ~ Sepal_Width, family = "gaussian") #' summary(model) #' } #' @note spark.glm since 2.0.0 -setMethod( - "spark.glm", - signature(data = "SparkDataFrame", formula = "formula"), - function(data, formula, family = gaussian, epsilon = 1e-06, maxit = 25) { - if (is.character(family)) { - family <- get(family, mode = "function", envir = parent.frame()) - } - if (is.function(family)) { - family <- family() - } - if (is.null(family$family)) { - print(family) - stop("'family' not recognized") - } +setMethod("spark.glm", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, family = gaussian, tol = 1e-6, maxIter = 25) { + if (is.character(family)) { + family <- get(family, mode = "function", envir = parent.frame()) + } + if (is.function(family)) { + family <- family() + } + if (is.null(family$family)) { + print(family) + stop("'family' not recognized") + } - formula <- paste(deparse(formula), collapse = "") + formula <- paste(deparse(formula), collapse = "") - jobj <- callJStatic("org.apache.spark.ml.r.GeneralizedLinearRegressionWrapper", - "fit", formula, data@sdf, family$family, family$link, - epsilon, as.integer(maxit)) - return(new("GeneralizedLinearRegressionModel", jobj = jobj)) -}) + jobj <- callJStatic("org.apache.spark.ml.r.GeneralizedLinearRegressionWrapper", + "fit", formula, data@sdf, family$family, family$link, + tol, as.integer(maxIter)) + return(new("GeneralizedLinearRegressionModel", jobj = jobj)) + }) #' Fits a generalized linear model (R-compliant). #' @@ -122,13 +120,13 @@ setMethod( #' sparkR.session() #' data(iris) #' df <- createDataFrame(iris) -#' model <- glm(Sepal_Length ~ Sepal_Width, df, family="gaussian") +#' model <- glm(Sepal_Length ~ Sepal_Width, df, family = "gaussian") #' summary(model) #' } #' @note glm since 1.5.0 setMethod("glm", signature(formula = "formula", family = "ANY", data = "SparkDataFrame"), - function(formula, family = gaussian, data, epsilon = 1e-06, maxit = 25) { - spark.glm(data, formula, family, epsilon, maxit) + function(formula, family = gaussian, data, epsilon = 1e-6, maxit = 25) { + spark.glm(data, formula, family, tol = epsilon, maxIter = maxit) }) #' Get the summary of a generalized linear model @@ -296,17 +294,17 @@ setMethod("summary", signature(object = "NaiveBayesModel"), #' @export #' @examples #' \dontrun{ -#' model <- spark.kmeans(data, ~ ., k=2, initMode="random") +#' model <- spark.kmeans(data, ~ ., k = 4, initMode = "random") #' } #' @note spark.kmeans since 2.0.0 setMethod("spark.kmeans", signature(data = "SparkDataFrame", formula = "formula"), - function(data, formula, k, maxIter = 10, initMode = c("random", "k-means||")) { + function(data, formula, k = 2, maxIter = 20, initMode = c("k-means||", "random")) { formula <- paste(deparse(formula), collapse = "") initMode <- match.arg(initMode) jobj <- callJStatic("org.apache.spark.ml.r.KMeansWrapper", "fit", data@sdf, formula, as.integer(k), as.integer(maxIter), initMode) return(new("KMeansModel", jobj = jobj)) - }) + }) #' Get fitted result from a k-means model #' @@ -397,7 +395,7 @@ setMethod("predict", signature(object = "KMeansModel"), #' @param data SparkDataFrame for training #' @param formula A symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', '.', ':', '+', and '-'. -#' @param laplace Smoothing parameter +#' @param smoothing Smoothing parameter #' @return a fitted naive Bayes model #' @rdname spark.naiveBayes #' @seealso e1071: \url{https://cran.r-project.org/web/packages/e1071/} @@ -405,16 +403,16 @@ setMethod("predict", signature(object = "KMeansModel"), #' @examples #' \dontrun{ #' df <- createDataFrame(infert) -#' model <- spark.naiveBayes(df, education ~ ., laplace = 0) +#' model <- spark.naiveBayes(df, education ~ ., smoothing = 0) #'} #' @note spark.naiveBayes since 2.0.0 setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "formula"), - function(data, formula, laplace = 0, ...) { - formula <- paste(deparse(formula), collapse = "") - jobj <- callJStatic("org.apache.spark.ml.r.NaiveBayesWrapper", "fit", - formula, data@sdf, laplace) - return(new("NaiveBayesModel", jobj = jobj)) - }) + function(data, formula, smoothing = 1.0, ...) { + formula <- paste(deparse(formula), collapse = "") + jobj <- callJStatic("org.apache.spark.ml.r.NaiveBayesWrapper", "fit", + formula, data@sdf, smoothing) + return(new("NaiveBayesModel", jobj = jobj)) + }) #' Save fitted MLlib model to the input path #' @@ -431,7 +429,7 @@ setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "form #' @examples #' \dontrun{ #' df <- createDataFrame(infert) -#' model <- spark.naiveBayes(df, education ~ ., laplace = 0) +#' model <- spark.naiveBayes(df, education ~ ., smoothing = 0) #' path <- "path/to/model" #' write.ml(model, path) #' } diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index c8c5ef2476b32..753da81760971 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -288,7 +288,7 @@ test_that("spark.kmeans", { take(training, 1) - model <- spark.kmeans(data = training, ~ ., k = 2) + model <- spark.kmeans(data = training, ~ ., k = 2, maxIter = 10, initMode = "random") sample <- take(select(predict(model, training), "prediction"), 1) expect_equal(typeof(sample$prediction), "integer") expect_equal(sample$prediction, 1) @@ -363,7 +363,7 @@ test_that("spark.naiveBayes", { t <- as.data.frame(Titanic) t1 <- t[t$Freq > 0, -5] df <- suppressWarnings(createDataFrame(t1)) - m <- spark.naiveBayes(df, Survived ~ .) + m <- spark.naiveBayes(df, Survived ~ ., smoothing = 0.0) s <- summary(m) expect_equal(as.double(s$apriori[1, "Yes"]), 0.5833333, tolerance = 1e-6) expect_equal(sum(s$apriori), 1) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala index 9618a3423e9a6..5642abc6450f1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala @@ -67,8 +67,8 @@ private[r] object GeneralizedLinearRegressionWrapper data: DataFrame, family: String, link: String, - epsilon: Double, - maxit: Int): GeneralizedLinearRegressionWrapper = { + tol: Double, + maxIter: Int): GeneralizedLinearRegressionWrapper = { val rFormula = new RFormula() .setFormula(formula) val rFormulaModel = rFormula.fit(data) @@ -82,8 +82,8 @@ private[r] object GeneralizedLinearRegressionWrapper .setFamily(family) .setLink(link) .setFitIntercept(rFormula.hasIntercept) - .setTol(epsilon) - .setMaxIter(maxit) + .setTol(tol) + .setMaxIter(maxIter) val pipeline = new Pipeline() .setStages(Array(rFormulaModel, glr)) .fit(data) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala index 28925c79da66e..1dac246b03329 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala @@ -56,7 +56,7 @@ private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" val PREDICTED_LABEL_COL = "prediction" - def fit(formula: String, data: DataFrame, laplace: Double): NaiveBayesWrapper = { + def fit(formula: String, data: DataFrame, smoothing: Double): NaiveBayesWrapper = { val rFormula = new RFormula() .setFormula(formula) .fit(data) @@ -70,7 +70,7 @@ private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { val features = featureAttrs.map(_.name.get) // assemble and fit the pipeline val naiveBayes = new NaiveBayes() - .setSmoothing(laplace) + .setSmoothing(smoothing) .setModelType("bernoulli") .setPredictionCol(PREDICTED_LABEL_INDEX_COL) val idxToStr = new IndexToString() From 943239bf43c5e59c71db218627ee4dc4308a0680 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 21 Jun 2016 10:47:51 -0700 Subject: [PATCH 0753/1470] [SPARK-13792][SQL] Addendum: Fix Python API ## What changes were proposed in this pull request? This is a follow-up to https://github.com/apache/spark/pull/13795 to properly set CSV options in Python API. As part of this, I also make the Python option setting for both CSV and JSON more robust against positional errors. ## How was this patch tested? N/A Author: Reynold Xin Closes #13800 from rxin/SPARK-13792-2. (cherry picked from commit 93338807aafdb2db9fb036ceadee1467cd367cdd) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 54 +++++++++++++++++++------------- 1 file changed, 33 insertions(+), 21 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 89506ca02f273..ccbf895c2d888 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -77,7 +77,7 @@ def _set_json_opts(self, schema, primitivesAsString, prefersDecimal, def _set_csv_opts(self, schema, sep, encoding, quote, escape, comment, header, inferSchema, ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace, nullValue, nanValue, positiveInf, negativeInf, - dateFormat, maxColumns, maxCharsPerColumn, mode): + dateFormat, maxColumns, maxCharsPerColumn, maxMalformedLogPerPartition, mode): """ Set options based on the CSV optional parameters """ @@ -115,6 +115,8 @@ def _set_csv_opts(self, schema, sep, encoding, quote, escape, self.option("maxColumns", maxColumns) if maxCharsPerColumn is not None: self.option("maxCharsPerColumn", maxCharsPerColumn) + if maxMalformedLogPerPartition is not None: + self.option("maxMalformedLogPerPartition", maxMalformedLogPerPartition) if mode is not None: self.option("mode", mode) @@ -268,10 +270,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, [('age', 'bigint'), ('name', 'string')] """ - self._set_json_opts(schema, primitivesAsString, prefersDecimal, - allowComments, allowUnquotedFieldNames, allowSingleQuotes, - allowNumericLeadingZero, allowBackslashEscapingAnyCharacter, - mode, columnNameOfCorruptRecord) + self._set_json_opts( + schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, + allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, + allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, + allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, + mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -343,7 +347,8 @@ def text(self, paths): def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, - negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, mode=None): + negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, + maxMalformedLogPerPartition=None, mode=None): """Loads a CSV file and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -408,11 +413,13 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non >>> df.dtypes [('_c0', 'string'), ('_c1', 'string')] """ - - self._set_csv_opts(schema, sep, encoding, quote, escape, - comment, header, inferSchema, ignoreLeadingWhiteSpace, - ignoreTrailingWhiteSpace, nullValue, nanValue, positiveInf, negativeInf, - dateFormat, maxColumns, maxCharsPerColumn, mode) + self._set_csv_opts( + schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, + header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, + ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, + nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, + dateFormat=dateFormat, maxColumns=maxColumns, maxCharsPerColumn=maxCharsPerColumn, + maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode) if isinstance(path, basestring): path = [path] return self._df(self._jreader.csv(self._spark._sc._jvm.PythonUtils.toSeq(path))) @@ -958,10 +965,12 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, >>> json_sdf.schema == sdf_schema True """ - self._set_json_opts(schema, primitivesAsString, prefersDecimal, - allowComments, allowUnquotedFieldNames, allowSingleQuotes, - allowNumericLeadingZero, allowBackslashEscapingAnyCharacter, - mode, columnNameOfCorruptRecord) + self._set_json_opts( + schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, + allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, + allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, + allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, + mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord) if isinstance(path, basestring): return self._df(self._jreader.json(path)) else: @@ -1019,7 +1028,8 @@ def text(self, path): def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, - negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, mode=None): + negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, + maxMalformedLogPerPartition=None, mode=None): """Loads a CSV file stream and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -1085,11 +1095,13 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non >>> csv_sdf.schema == sdf_schema True """ - - self._set_csv_opts(schema, sep, encoding, quote, escape, - comment, header, inferSchema, ignoreLeadingWhiteSpace, - ignoreTrailingWhiteSpace, nullValue, nanValue, positiveInf, negativeInf, - dateFormat, maxColumns, maxCharsPerColumn, mode) + self._set_csv_opts( + schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, + header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, + ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, + nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, + dateFormat=dateFormat, maxColumns=maxColumns, maxCharsPerColumn=maxCharsPerColumn, + maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode) if isinstance(path, basestring): return self._df(self._jreader.csv(path)) else: From 052779a0cf8676683b3b83203e440d2266eef790 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 21 Jun 2016 12:48:06 -0500 Subject: [PATCH 0754/1470] [SPARK-16080][YARN] Set correct link name for conf archive in executors. This makes sure the files are in the executor's classpath as they're expected to be. Also update the unit test to make sure the files are there as expected. Author: Marcelo Vanzin Closes #13792 from vanzin/SPARK-16080. (cherry picked from commit bcb0258ae62f23f71a067c1304232f272d7374aa) Signed-off-by: Tom Graves --- .../spark/deploy/yarn/ApplicationMaster.scala | 14 ++++++++++---- .../spark/deploy/yarn/YarnClusterSuite.scala | 8 ++++++++ 2 files changed, 18 insertions(+), 4 deletions(-) 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 4df90d7b6b0b8..847d1de50f890 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 @@ -160,11 +160,17 @@ private[spark] class ApplicationMaster( } // Distribute the conf archive to executors. - sparkConf.get(CACHED_CONF_ARCHIVE).foreach { uri => - val fs = FileSystem.get(new URI(uri), yarnConf) + sparkConf.get(CACHED_CONF_ARCHIVE).foreach { path => + val uri = new URI(path) + val fs = FileSystem.get(uri, yarnConf) val status = fs.getFileStatus(new Path(uri)) - setupDistributedCache(uri, LocalResourceType.ARCHIVE, status.getModificationTime().toString, - status.getLen.toString, LocalResourceVisibility.PRIVATE.name()) + // SPARK-16080: Make sure to use the correct name for the destination when distributing the + // conf archive to executors. + val destUri = new URI(uri.getScheme(), uri.getRawSchemeSpecificPart(), + Client.LOCALIZED_CONF_DIR) + setupDistributedCache(destUri.toString(), LocalResourceType.ARCHIVE, + status.getModificationTime().toString, status.getLen.toString, + LocalResourceVisibility.PRIVATE.name()) } // Clean up the configuration so it doesn't show up in the Web UI (since it's really noisy). 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 c4656048455a8..4ce33e0e8565d 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 @@ -292,6 +292,14 @@ private object YarnClusterDriver extends Logging with Matchers { sc.stop() } + // Verify that the config archive is correctly placed in the classpath of all containers. + val confFile = "/" + Client.SPARK_CONF_FILE + assert(getClass().getResource(confFile) != null) + val configFromExecutors = sc.parallelize(1 to 4, 4) + .map { _ => Option(getClass().getResource(confFile)).map(_.toString).orNull } + .collect() + assert(configFromExecutors.find(_ == null) === None) + // verify log urls are present val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] assert(listeners.size === 1) From 703a526e79aa30f5a5bab5477a5ed02b8ecc29ea Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 21 Jun 2016 10:53:33 -0700 Subject: [PATCH 0755/1470] [SPARK-16086] [SQL] [PYSPARK] create Row without any fields ## What changes were proposed in this pull request? This PR allows us to create a Row without any fields. ## How was this patch tested? Added a test for empty row and udf without arguments. Author: Davies Liu Closes #13812 from davies/no_argus. (cherry picked from commit 2d6919bea9fc213b5af530afab7793b63c6c8b51) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 9 +++++++++ python/pyspark/sql/types.py | 9 +++------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index c631ad8a4618d..388ac919221c7 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -177,6 +177,10 @@ def test_datetype_equal_zero(self): dt = DateType() self.assertEqual(dt.fromInternal(0), datetime.date(1970, 1, 1)) + def test_empty_row(self): + row = Row() + self.assertEqual(len(row), 0) + class SQLTests(ReusedPySparkTestCase): @@ -318,6 +322,11 @@ def test_multiple_udfs(self): [row] = self.spark.sql("SELECT double(add(1, 2)), add(double(2), 1)").collect() self.assertEqual(tuple(row), (6, 5)) + def test_udf_without_arguments(self): + self.spark.catalog.registerFunction("foo", lambda: "bar") + [row] = self.spark.sql("SELECT foo()").collect() + self.assertEqual(row[0], "bar") + def test_udf_with_array_type(self): d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index bb2b95404af7a..f0b56be8da438 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1401,11 +1401,7 @@ def __new__(self, *args, **kwargs): if args and kwargs: raise ValueError("Can not use both args " "and kwargs to create Row") - if args: - # create row class or objects - return tuple.__new__(self, args) - - elif kwargs: + if kwargs: # create row objects names = sorted(kwargs.keys()) row = tuple.__new__(self, [kwargs[n] for n in names]) @@ -1413,7 +1409,8 @@ def __new__(self, *args, **kwargs): return row else: - raise ValueError("No args or kwargs") + # create row class or objects + return tuple.__new__(self, args) def asDict(self, recursive=False): """ From 867baaadad48a378b36933df0635a09cddc4c8de Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 21 Jun 2016 11:01:42 -0700 Subject: [PATCH 0756/1470] [SPARK-16109][SPARKR][DOC] R more doc fixes ## What changes were proposed in this pull request? Found these issues while reviewing for SPARK-16090 ## How was this patch tested? roxygen2 doc gen, checked output html Author: Felix Cheung Closes #13803 from felixcheung/rdocrd. (cherry picked from commit 57746295e6fb705f8393a00ab1cc570ddb7da44e) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 7 +++++-- R/pkg/R/functions.R | 4 +++- R/pkg/R/generics.R | 8 ++++---- R/pkg/R/schema.R | 7 +++++-- R/pkg/R/stats.R | 37 +++++++++++++++++++++++-------------- 5 files changed, 40 insertions(+), 23 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index a8ade1ac9ab72..ed0bb85f43d09 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -59,6 +59,7 @@ setMethod("initialize", "SparkDataFrame", function(.Object, sdf, isCached) { #' @export #' @param sdf A Java object reference to the backing Scala DataFrame #' @param isCached TRUE if the SparkDataFrame is cached +#' @noRd dataFrame <- function(sdf, isCached = FALSE) { new("SparkDataFrame", sdf, isCached) } @@ -119,7 +120,7 @@ setMethod("schema", #' Print the logical and physical Catalyst plans to the console for debugging. #' #' @param x A SparkDataFrame -#' @param extended Logical. If extended is False, explain() only prints the physical plan. +#' @param extended Logical. If extended is FALSE, explain() only prints the physical plan. #' @family SparkDataFrame functions #' @rdname explain #' @name explain @@ -175,6 +176,8 @@ setMethod("isLocal", #' #' @param x A SparkDataFrame #' @param numRows The number of rows to print. Defaults to 20. +#' @param truncate Whether truncate long strings. If true, strings more than 20 characters will be +#' truncated and all cells will be aligned right #' #' @family SparkDataFrame functions #' @rdname showDF @@ -1854,7 +1857,7 @@ setMethod("withColumnRenamed", select(x, cols) }) -#' @param newColPair A named pair of the form new_column_name = existing_column +#' @param ... A named pair of the form new_column_name = existing_column #' @rdname rename #' @name rename #' @export diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 6e0009f7c90af..09e5afa97060e 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1777,7 +1777,7 @@ setMethod("months_between", signature(y = "Column"), #' nanvl #' #' Returns col1 if it is not NaN, or col2 if col1 is NaN. -#' hhBoth inputs should be floating point columns (DoubleType or FloatType). +#' Both inputs should be floating point columns (DoubleType or FloatType). #' #' @rdname nanvl #' @name nanvl @@ -2008,6 +2008,8 @@ setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), #' NOTE: The position is not zero based, but 1 based index, returns 0 if substr #' could not be found in str. #' +#' @param y column to check +#' @param x substring to check #' @family string_funcs #' @rdname instr #' @name instr diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 43395aaa1da96..7b08a8ee66fd4 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -59,15 +59,15 @@ setGeneric("count", function(x) { standardGeneric("count") }) # @export setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) -# @rdname statfunctions +# @rdname crosstab # @export setGeneric("crosstab", function(x, col1, col2) { standardGeneric("crosstab") }) -# @rdname statfunctions +# @rdname freqItems # @export setGeneric("freqItems", function(x, cols, support = 0.01) { standardGeneric("freqItems") }) -# @rdname statfunctions +# @rdname approxQuantile # @export setGeneric("approxQuantile", function(x, col, probabilities, relativeError) { @@ -575,7 +575,7 @@ setGeneric("sample", setGeneric("sample_frac", function(x, withReplacement, fraction, seed) { standardGeneric("sample_frac") }) -#' @rdname statfunctions +#' @rdname sampleBy #' @export setGeneric("sampleBy", function(x, col, fractions, seed) { standardGeneric("sampleBy") }) diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R index fb23c780ad860..a91e9980df937 100644 --- a/R/pkg/R/schema.R +++ b/R/pkg/R/schema.R @@ -86,9 +86,8 @@ print.structType <- function(x, ...) { #' Create a structField object that contains the metadata for a single field in a schema. #' #' @param x The name of the field -#' @param type The data type of the field -#' @param nullable A logical vector indicating whether or not the field is nullable #' @return a structField object +#' @rdname structField #' @export #' @examples #'\dontrun{ @@ -176,6 +175,10 @@ checkType <- function(type) { stop(paste("Unsupported type for SparkDataframe:", type)) } +#' @param type The data type of the field +#' @param nullable A logical vector indicating whether or not the field is nullable +#' @rdname structField +#' @export structField.character <- function(x, type, nullable = TRUE) { if (class(x) != "character") { stop("Field name must be a string.") diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index e40b1773d70e4..c92352e1b063d 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -19,10 +19,9 @@ setOldClass("jobj") -#' @title SparkDataFrame statistic functions - -#' @description -#' crosstab - Computes a pair-wise frequency table of the given columns. Also known as a contingency +#' Computes a pair-wise frequency table of the given columns +#' +#' Computes a pair-wise frequency table of the given columns. Also known as a contingency #' table. The number of distinct values for each column should be less than 1e4. At most 1e6 #' non-zero pair frequencies will be returned. #' @@ -33,8 +32,9 @@ setOldClass("jobj") #' of `col2`. The name of the first column will be `$col1_$col2`. Pairs that have no #' occurrences will have zero as their counts. #' -#' @rdname statfunctions +#' @rdname crosstab #' @name crosstab +#' @family stat functions #' @export #' @examples #' \dontrun{ @@ -59,6 +59,7 @@ setMethod("crosstab", #' #' @rdname cov #' @name cov +#' @family stat functions #' @export #' @examples #'\dontrun{ @@ -87,6 +88,7 @@ setMethod("cov", #' #' @rdname corr #' @name corr +#' @family stat functions #' @export #' @examples #'\dontrun{ @@ -103,8 +105,10 @@ setMethod("corr", callJMethod(statFunctions, "corr", col1, col2, method) }) -#' @description -#' freqItems - Finding frequent items for columns, possibly with false positives. + +#' Finding frequent items for columns, possibly with false positives +#' +#' Finding frequent items for columns, possibly with false positives. #' Using the frequent element count algorithm described in #' \url{http://dx.doi.org/10.1145/762471.762473}, proposed by Karp, Schenker, and Papadimitriou. #' @@ -114,8 +118,9 @@ setMethod("corr", #' Should be greater than 1e-4. Default support = 0.01. #' @return a local R data.frame with the frequent items in each column #' -#' @rdname statfunctions +#' @rdname freqItems #' @name freqItems +#' @family stat functions #' @export #' @examples #' \dontrun{ @@ -130,8 +135,9 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), collect(dataFrame(sct)) }) -#' @description -#' approxQuantile - Calculates the approximate quantiles of a numerical column of a SparkDataFrame. +#' Calculates the approximate quantiles of a numerical column of a SparkDataFrame +#' +#' Calculates the approximate quantiles of a numerical column of a SparkDataFrame. #' The result of this algorithm has the following deterministic bound: #' If the SparkDataFrame has N elements and if we request the quantile at probability `p` up to #' error `err`, then the algorithm will return a sample `x` from the SparkDataFrame so that the @@ -150,8 +156,9 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), #' Note that values greater than 1 are accepted but give the same result as 1. #' @return The approximate quantiles at the given probabilities. #' -#' @rdname statfunctions +#' @rdname approxQuantile #' @name approxQuantile +#' @family stat functions #' @export #' @examples #' \dontrun{ @@ -168,8 +175,9 @@ setMethod("approxQuantile", as.list(probabilities), relativeError) }) -#' @description -#' sampleBy - Returns a stratified sample without replacement based on the fraction given on each +#' Returns a stratified sample without replacement +#' +#' Returns a stratified sample without replacement based on the fraction given on each #' stratum. #' #' @param x A SparkDataFrame @@ -179,8 +187,9 @@ setMethod("approxQuantile", #' @param seed random seed #' @return A new SparkDataFrame that represents the stratified sample #' -#' @rdname statfunctions +#' @rdname sampleBy #' @name sampleBy +#' @family stat functions #' @export #' @examples #'\dontrun{ From f805b989b380981f5515334f9554648f6bf632af Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Tue, 21 Jun 2016 11:43:25 -0700 Subject: [PATCH 0757/1470] [SPARK-15741][PYSPARK][ML] Pyspark cleanup of set default seed to None ## What changes were proposed in this pull request? Several places set the seed Param default value to None which will translate to a zero value on the Scala side. This is unnecessary because a default fixed value already exists and if a test depends on a zero valued seed, then it should explicitly set it to zero instead of relying on this translation. These cases can be safely removed except for the ALS doc test, which has been changed to set the seed value to zero. ## How was this patch tested? Ran PySpark tests locally Author: Bryan Cutler Closes #13672 from BryanCutler/pyspark-cleanup-setDefault-seed-SPARK-15741. (cherry picked from commit b76e3553760b3c68bebc2c71b0851598718e6f87) Signed-off-by: Xiangrui Meng --- python/pyspark/ml/classification.py | 4 ++-- python/pyspark/ml/feature.py | 2 +- python/pyspark/ml/recommendation.py | 4 ++-- python/pyspark/ml/regression.py | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index a3cd91790c42e..e86c27ecaf3df 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -685,7 +685,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.RandomForestClassifier", self.uid) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, - maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, + maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", numTrees=20, featureSubsetStrategy="auto") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -825,7 +825,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred "org.apache.spark.ml.classification.GBTClassifier", self.uid) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - lossType="logistic", maxIter=20, stepSize=0.1, seed=None) + lossType="logistic", maxIter=20, stepSize=0.1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1e9ec0fbb411c..bbbb94f9a0a04 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2260,7 +2260,7 @@ def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, super(Word2Vec, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid) self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=None, windowSize=5, maxSentenceLength=1000) + windowSize=5, maxSentenceLength=1000) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 0a7096794d25e..e28d38bd19f80 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -68,7 +68,7 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha >>> df = spark.createDataFrame( ... [(0, 0, 4.0), (0, 1, 2.0), (1, 1, 3.0), (1, 2, 4.0), (2, 1, 1.0), (2, 2, 5.0)], ... ["user", "item", "rating"]) - >>> als = ALS(rank=10, maxIter=5) + >>> als = ALS(rank=10, maxIter=5, seed=0) >>> model = als.fit(df) >>> model.rank 10 @@ -142,7 +142,7 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB super(ALS, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid) self._setDefault(rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, - implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, + implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", finalStorageLevel="MEMORY_AND_DISK") diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 8d2378d51fb7e..29efd6a852e84 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -894,7 +894,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred "org.apache.spark.ml.regression.RandomForestRegressor", self.uid) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, + impurity="variance", subsamplingRate=1.0, numTrees=20, featureSubsetStrategy="auto") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -1023,7 +1023,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, - seed=None, impurity="variance") + impurity="variance") kwargs = self.__init__._input_kwargs self.setParams(**kwargs) From 0d7e1d11d2ea2b7005208951518fdf882fc36ec2 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 21 Jun 2016 11:58:33 -0700 Subject: [PATCH 0758/1470] [SPARK-16037][SQL] Follow-up: add DataFrameWriter.insertInto() test cases for by position resolution ## What changes were proposed in this pull request? This PR migrates some test cases introduced in #12313 as a follow-up of #13754 and #13766. These test cases cover `DataFrameWriter.insertInto()`, while the former two only cover SQL `INSERT` statements. Note that the `testPartitionedTable` utility method tests both Hive SerDe tables and data source tables. ## How was this patch tested? N/A Author: Cheng Lian Closes #13810 from liancheng/spark-16037-follow-up-tests. (cherry picked from commit f4a3d45e38f18278bbdb7cc32486ded50f76d54b) Signed-off-by: Yin Huai --- .../sql/hive/InsertIntoHiveTableSuite.scala | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) 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 46432512bac8d..d9ce1c3dc18ff 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 @@ -469,4 +469,52 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef ) } } + + testPartitionedTable("insertInto() should match columns by position and ignore column names") { + tableName => + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + // Columns `df.c` and `df.d` are resolved by position, and thus mapped to partition columns + // `b` and `c` of the target table. + val df = Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d") + df.write.insertInto(tableName) + + checkAnswer( + sql(s"SELECT a, b, c, d FROM $tableName"), + Row(1, 3, 4, 2) + ) + } + } + + testPartitionedTable("insertInto() should match unnamed columns by position") { + tableName => + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + // Columns `c + 1` and `d + 1` are resolved by position, and thus mapped to partition + // columns `b` and `c` of the target table. + val df = Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d") + df.select('a + 1, 'b + 1, 'c + 1, 'd + 1).write.insertInto(tableName) + + checkAnswer( + sql(s"SELECT a, b, c, d FROM $tableName"), + Row(2, 4, 5, 3) + ) + } + } + + testPartitionedTable("insertInto() should reject missing columns") { + tableName => + sql("CREATE TABLE t (a INT, b INT)") + + intercept[AnalysisException] { + spark.table("t").write.insertInto(tableName) + } + } + + testPartitionedTable("insertInto() should reject extra columns") { + tableName => + sql("CREATE TABLE t (a INT, b INT, c INT, d INT, e INT)") + + intercept[AnalysisException] { + spark.table("t").write.insertInto(tableName) + } + } } From afa14b71b28d788c53816bd2616ccff0c3967f40 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 21 Jun 2016 12:42:49 -0700 Subject: [PATCH 0759/1470] [SPARK-16002][SQL] Sleep when no new data arrives to avoid 100% CPU usage ## What changes were proposed in this pull request? Add a configuration to allow people to set a minimum polling delay when no new data arrives (default is 10ms). This PR also cleans up some INFO logs. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu Closes #13718 from zsxwing/SPARK-16002. (cherry picked from commit c399c7f0e485dcfc6cbc343bc246b8adc3f0648c) Signed-off-by: Yin Huai --- .../org/apache/spark/util/ManualClock.scala | 18 +++++++++++++++--- .../datasources/ListingFileCatalog.scala | 2 +- .../datasources/fileSourceInterfaces.scala | 2 +- .../execution/streaming/FileStreamSource.scala | 8 +++++++- .../execution/streaming/StreamExecution.scala | 5 +++++ .../apache/spark/sql/internal/SQLConf.scala | 9 ++++++++- .../spark/sql/streaming/StreamTest.scala | 5 +++++ 7 files changed, 42 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala index e7a65d74a440e..91a95871014f0 100644 --- a/core/src/main/scala/org/apache/spark/util/ManualClock.scala +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -26,6 +26,8 @@ package org.apache.spark.util */ private[spark] class ManualClock(private var time: Long) extends Clock { + private var _isWaiting = false + /** * @return `ManualClock` with initial time 0 */ @@ -57,9 +59,19 @@ private[spark] class ManualClock(private var time: Long) extends Clock { * @return current time reported by the clock when waiting finishes */ def waitTillTime(targetTime: Long): Long = synchronized { - while (time < targetTime) { - wait(10) + _isWaiting = true + try { + while (time < targetTime) { + wait(10) + } + getTimeMillis() + } finally { + _isWaiting = false } - getTimeMillis() } + + /** + * Returns whether there is any thread being blocked in `waitTillTime`. + */ + def isWaiting: Boolean = synchronized { _isWaiting } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index d96cf1bf0702c..f713fdec4ecf3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -82,7 +82,7 @@ class ListingFileCatalog( val pathFilter = FileInputFormat.getInputPathFilter(jobConf) val statuses: Seq[FileStatus] = paths.flatMap { path => val fs = path.getFileSystem(hadoopConf) - logInfo(s"Listing $path on driver") + logTrace(s"Listing $path on driver") Try { HadoopFsRelation.listLeafFiles(fs, fs.getFileStatus(path), pathFilter) }.getOrElse(Array.empty[FileStatus]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index bfb34c098f9ad..5689cf688bfbe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -388,7 +388,7 @@ private[sql] object HadoopFsRelation extends Logging { // tasks/jobs may leave partial/corrupted data files there. Files and directories whose name // start with "." are also ignored. def listLeafFiles(fs: FileSystem, status: FileStatus, filter: PathFilter): Array[FileStatus] = { - logInfo(s"Listing ${status.getPath}") + logTrace(s"Listing ${status.getPath}") val name = status.getPath.getName.toLowerCase if (shouldFilterOut(name)) { Array.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 9886ad0b41089..11bf3c0bd2e00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -120,7 +120,13 @@ class FileStreamSource( val catalog = new ListingFileCatalog(sparkSession, globbedPaths, options, Some(new StructType)) val files = catalog.allFiles().map(_.getPath.toUri.toString) val endTime = System.nanoTime - logInfo(s"Listed ${files.size} in ${(endTime.toDouble - startTime) / 1000000}ms") + val listingTimeMs = (endTime.toDouble - startTime) / 1000000 + if (listingTimeMs > 2000) { + // Output a warning when listing files uses more than 2 seconds. + logWarning(s"Listed ${files.size} file(s) in $listingTimeMs ms") + } else { + logTrace(s"Listed ${files.size} file(s) in $listingTimeMs ms") + } logTrace(s"Files are:\n\t" + files.mkString("\n\t")) files } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index bb42a11759b72..1428b971490d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} @@ -56,6 +57,8 @@ class StreamExecution( import org.apache.spark.sql.streaming.StreamingQueryListener._ + private val pollingDelayMs = sparkSession.conf.get(SQLConf.STREAMING_POLLING_DELAY) + /** * A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation. */ @@ -190,6 +193,8 @@ class StreamExecution( runBatch() // We'll increase currentBatchId after we complete processing current batch's data currentBatchId += 1 + } else { + Thread.sleep(pollingDelayMs) } true } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4b8916f59c41d..1a9bb6a0b54e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -534,7 +534,7 @@ object SQLConf { val FILE_SINK_LOG_CLEANUP_DELAY = SQLConfigBuilder("spark.sql.streaming.fileSink.log.cleanupDelay") .internal() - .doc("How long in milliseconds a file is guaranteed to be visible for all readers.") + .doc("How long that a file is guaranteed to be visible for all readers.") .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(60 * 1000L) // 10 minutes @@ -545,6 +545,13 @@ object SQLConf { .booleanConf .createWithDefault(false) + val STREAMING_POLLING_DELAY = + SQLConfigBuilder("spark.sql.streaming.pollingDelay") + .internal() + .doc("How long to delay polling new data when no data is available") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(10L) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 720ffaf732542..f9496520f3836 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -326,6 +326,11 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { "can not advance manual clock when a stream is not running") verify(currentStream.triggerClock.isInstanceOf[ManualClock], s"can not advance clock of type ${currentStream.triggerClock.getClass}") + val clock = currentStream.triggerClock.asInstanceOf[ManualClock] + // Make sure we don't advance ManualClock too early. See SPARK-16002. + eventually("ManualClock has not yet entered the waiting state") { + assert(clock.isWaiting) + } currentStream.triggerClock.asInstanceOf[ManualClock].advance(timeToAdd) case StopStream => From 591bf79093933429d4a5d0b0797961f9eb9566eb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 21 Jun 2016 13:35:06 -0700 Subject: [PATCH 0760/1470] [MINOR][MLLIB] move setCheckpointInterval to non-expert setters ## What changes were proposed in this pull request? The `checkpointInterval` is a non-expert param. This PR moves its setter to non-expert group. Author: Xiangrui Meng Closes #13813 from mengxr/checkpoint-non-expert. (cherry picked from commit 918c91954fb46400ce2c5ab066d2ec0ae48dda4a) Signed-off-by: Xiangrui Meng --- mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index d7559f8950c3d..57c7e44e97607 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -151,7 +151,7 @@ private[ml] trait DecisionTreeParams extends PredictorParams * [[org.apache.spark.SparkContext]]. * Must be >= 1. * (default = 10) - * @group expertSetParam + * @group setParam */ def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) From aeda9a153c117921e95cf204daab0df3202f1d95 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 21 Jun 2016 13:36:50 -0700 Subject: [PATCH 0761/1470] [SPARK-16096][SPARKR] add union and deprecate unionAll ## What changes were proposed in this pull request? add union and deprecate unionAll, separate roxygen2 doc for rbind (since their usage and parameter lists are quite different) `explode` is also deprecated - but seems like replacement is a combination of calls; not sure if we should deprecate it in SparkR, yet. ## How was this patch tested? unit tests, manual checks for r doc Author: Felix Cheung Closes #13805 from felixcheung/runion. (cherry picked from commit dbfdae4e41a900de01b48639d6554d32edbb2e0b) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 43 ++++++++++++++++++----- R/pkg/R/generics.R | 6 +++- R/pkg/inst/tests/testthat/test_context.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 8 +++-- 5 files changed, 47 insertions(+), 13 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index ea42888eaebfa..2272d8bdd52c2 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -107,6 +107,7 @@ exportMethods("arrange", "summary", "take", "transform", + "union", "unionAll", "unique", "unpersist", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ed0bb85f43d09..725cbf24f236d 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2251,7 +2251,7 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { cols } -#' rbind +#' Return a new SparkDataFrame containing the union of rows #' #' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame #' and another SparkDataFrame. This is equivalent to `UNION ALL` in SQL. @@ -2261,39 +2261,64 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { #' @param y A SparkDataFrame #' @return A SparkDataFrame containing the result of the union. #' @family SparkDataFrame functions -#' @rdname rbind -#' @name unionAll +#' @rdname union +#' @name union +#' @seealso \link{rbind} #' @export #' @examples #'\dontrun{ #' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) -#' unioned <- unionAll(df, df2) +#' unioned <- union(df, df2) +#' unions <- rbind(df, df2, df3, df4) #' } +#' @note union since 2.0.0 +setMethod("union", + signature(x = "SparkDataFrame", y = "SparkDataFrame"), + function(x, y) { + unioned <- callJMethod(x@sdf, "union", y@sdf) + dataFrame(unioned) + }) + +#' unionAll is deprecated - use union instead +#' @rdname union +#' @name unionAll +#' @export #' @note unionAll since 1.4.0 setMethod("unionAll", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { - unioned <- callJMethod(x@sdf, "unionAll", y@sdf) - dataFrame(unioned) + .Deprecated("union") + union(x, y) }) #' Union two or more SparkDataFrames #' -#' Returns a new SparkDataFrame containing rows of all parameters. +#' Union two or more SparkDataFrames. This is equivalent to `UNION ALL` in SQL. +#' Note that this does not remove duplicate rows across the two SparkDataFrames. #' +#' @param x A SparkDataFrame +#' @param ... Additional SparkDataFrame +#' @return A SparkDataFrame containing the result of the union. +#' @family SparkDataFrame functions #' @rdname rbind #' @name rbind +#' @seealso \link{union} #' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' unions <- rbind(df, df2, df3, df4) +#' } #' @note rbind since 1.5.0 setMethod("rbind", signature(... = "SparkDataFrame"), function(x, ..., deparse.level = 1) { if (nargs() == 3) { - unionAll(x, ...) + union(x, ...) } else { - unionAll(x, Recall(..., deparse.level = 1)) + union(x, Recall(..., deparse.level = 1)) } }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 7b08a8ee66fd4..27dfd67ffc932 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -662,7 +662,11 @@ setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) -#' @rdname rbind +#' @rdname union +#' @export +setGeneric("union", function(x, y) { standardGeneric("union") }) + +#' @rdname union #' @export setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index b149818ff46f6..3d232df566a83 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -24,7 +24,7 @@ test_that("Check masked functions", { namesOfMaskedCompletely <- c("cov", "filter", "sample") namesOfMasked <- c("describe", "cov", "filter", "lag", "na.omit", "predict", "sd", "var", "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", - "summary", "transform", "drop", "window", "as.data.frame") + "summary", "transform", "drop", "window", "as.data.frame", "union") if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) } diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 7c192fb5a0afa..9378c7afac8bd 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1590,7 +1590,7 @@ test_that("isLocal()", { expect_false(isLocal(df)) }) -test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", { +test_that("union(), rbind(), except(), and intersect() on a DataFrame", { df <- read.json(jsonPath) lines <- c("{\"name\":\"Bob\", \"age\":24}", @@ -1600,10 +1600,11 @@ test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", { writeLines(lines, jsonPath2) df2 <- read.df(jsonPath2, "json") - unioned <- arrange(unionAll(df, df2), df$age) + unioned <- arrange(union(df, df2), df$age) expect_is(unioned, "SparkDataFrame") expect_equal(count(unioned), 6) expect_equal(first(unioned)$name, "Michael") + expect_equal(count(arrange(suppressWarnings(unionAll(df, df2)), df$age)), 6) unioned2 <- arrange(rbind(unioned, df, df2), df$age) expect_is(unioned2, "SparkDataFrame") @@ -1620,6 +1621,9 @@ test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", { expect_equal(count(intersected), 1) expect_equal(first(intersected)$name, "Andy") + # Test base::union is working + expect_equal(union(c(1:3), c(3:5)), c(1:5)) + # Test base::rbind is working expect_equal(length(rbind(1:4, c = 2, a = 10, 10, deparse.level = 0)), 16) From 8d5a62d5f8ee9a2f4643075efdbee9822e85d141 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 21 Jun 2016 15:46:14 -0700 Subject: [PATCH 0762/1470] [SPARK-16117][MLLIB] hide LibSVMFileFormat and move its doc to LibSVMDataSource ## What changes were proposed in this pull request? LibSVMFileFormat implements data source for LIBSVM format. However, users do not really need to call its APIs to use it. So we should hide it in the public API docs. The main issue is that we still need to put the documentation and example code somewhere. The proposal it to have a dummy class to hold the documentation, as a workaround to https://issues.scala-lang.org/browse/SI-8124. ## How was this patch tested? Manually checked the generated API doc and tested loading LIBSVM data. Author: Xiangrui Meng Closes #13819 from mengxr/SPARK-16117. (cherry picked from commit f4e8c31adf45af05751e0d77aefb5cacc58375ee) Signed-off-by: Reynold Xin --- .../ml/source/libsvm/LibSVMDataSource.scala | 56 +++++++++++++++++++ .../ml/source/libsvm/LibSVMRelation.scala | 41 +------------- 2 files changed, 59 insertions(+), 38 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMDataSource.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMDataSource.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMDataSource.scala new file mode 100644 index 0000000000000..73d813064decb --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMDataSource.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.source.libsvm + +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.sql.{DataFrame, DataFrameReader} + +/** + * `libsvm` package implements Spark SQL data source API for loading LIBSVM data as [[DataFrame]]. + * The loaded [[DataFrame]] has two columns: `label` containing labels stored as doubles and + * `features` containing feature vectors stored as [[Vector]]s. + * + * To use LIBSVM data source, you need to set "libsvm" as the format in [[DataFrameReader]] and + * optionally specify options, for example: + * {{{ + * // Scala + * val df = spark.read.format("libsvm") + * .option("numFeatures", "780") + * .load("data/mllib/sample_libsvm_data.txt") + * + * // Java + * Dataset df = spark.read().format("libsvm") + * .option("numFeatures, "780") + * .load("data/mllib/sample_libsvm_data.txt"); + * }}} + * + * LIBSVM data source supports the following options: + * - "numFeatures": number of features. + * If unspecified or nonpositive, the number of features will be determined automatically at the + * cost of one additional pass. + * This is also useful when the dataset is already split into multiple files and you want to load + * them separately, because some features may not present in certain files, which leads to + * inconsistent feature dimensions. + * - "vectorType": feature vector type, "sparse" (default) or "dense". + * + * Note that this class is public for documentation purpose. Please don't use this class directly. + * Rather, use the data source API as illustrated above. + * + * @see [[https://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/ LIBSVM datasets]] + */ +class LibSVMDataSource private() {} diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 4988dd66f8274..034223e115389 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -25,11 +25,10 @@ import org.apache.hadoop.io.{NullWritable, Text} import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat -import org.apache.spark.annotation.Since import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{DataFrame, DataFrameReader, Row, SparkSession} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.AttributeReference @@ -77,44 +76,10 @@ private[libsvm] class LibSVMOutputWriter( } } -/** - * `libsvm` package implements Spark SQL data source API for loading LIBSVM data as [[DataFrame]]. - * The loaded [[DataFrame]] has two columns: `label` containing labels stored as doubles and - * `features` containing feature vectors stored as [[Vector]]s. - * - * To use LIBSVM data source, you need to set "libsvm" as the format in [[DataFrameReader]] and - * optionally specify options, for example: - * {{{ - * // Scala - * val df = spark.read.format("libsvm") - * .option("numFeatures", "780") - * .load("data/mllib/sample_libsvm_data.txt") - * - * // Java - * Dataset df = spark.read().format("libsvm") - * .option("numFeatures, "780") - * .load("data/mllib/sample_libsvm_data.txt"); - * }}} - * - * LIBSVM data source supports the following options: - * - "numFeatures": number of features. - * If unspecified or nonpositive, the number of features will be determined automatically at the - * cost of one additional pass. - * This is also useful when the dataset is already split into multiple files and you want to load - * them separately, because some features may not present in certain files, which leads to - * inconsistent feature dimensions. - * - "vectorType": feature vector type, "sparse" (default) or "dense". - * - * @see [[https://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/ LIBSVM datasets]] - * - * Note that this class is public for documentation purpose. Please don't use this class directly. - * Rather, use the data source API as illustrated above. - */ +/** @see [[LibSVMDataSource]] for public documentation. */ // If this is moved or renamed, please update DataSource's backwardCompatibilityMap. -@Since("1.6.0") -class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { +private[libsvm] class LibSVMFileFormat extends TextBasedFileFormat with DataSourceRegister { - @Since("1.6.0") override def shortName(): String = "libsvm" override def toString: String = "LibSVM" From f2413736d915e8b39dea6e5935a4956b669b30ce Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 21 Jun 2016 15:52:31 -0700 Subject: [PATCH 0763/1470] [SPARK-16118][MLLIB] add getDropLast to OneHotEncoder ## What changes were proposed in this pull request? We forgot the getter of `dropLast` in `OneHotEncoder` ## How was this patch tested? unit test Author: Xiangrui Meng Closes #13821 from mengxr/SPARK-16118. (cherry picked from commit 9493b079a0050f0a6f4936c17622b96fb185b67f) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/ml/feature/OneHotEncoder.scala | 4 ++++ .../org/apache/spark/ml/feature/OneHotEncoderSuite.scala | 4 +++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 4fafc1e349c22..01828ede6bc69 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -59,6 +59,10 @@ class OneHotEncoder @Since("1.4.0") (@Since("1.4.0") override val uid: String) e new BooleanParam(this, "dropLast", "whether to drop the last category") setDefault(dropLast -> true) + /** @group getParam */ + @Since("2.0.0") + def getDropLast: Boolean = $(dropLast) + /** @group setParam */ @Since("1.4.0") def setDropLast(value: Boolean): this.type = set(dropLast, value) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 7841b4fbc77a9..d41eeec1329c5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -49,7 +49,9 @@ class OneHotEncoderSuite val encoder = new OneHotEncoder() .setInputCol("labelIndex") .setOutputCol("labelVec") - .setDropLast(false) + assert(encoder.getDropLast === true) + encoder.setDropLast(false) + assert(encoder.getDropLast === false) val encoded = encoder.transform(transformed) val output = encoded.select("id", "labelVec").rdd.map { r => From fe41f68fdcf914b9de79b3cdd372e7dc891b9a52 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 21 Jun 2016 15:53:18 -0700 Subject: [PATCH 0764/1470] Update branch-2.0 for 2.0.0 release. --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index c0a3be7f8208e..3951cadb0ffd5 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,7 +14,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.0-preview +SPARK_VERSION: 2.0.0 SPARK_VERSION_SHORT: 2.0.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" From 0c66ca41afade6db73c9aeddd5aed6e5dcea90df Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 21 Jun 2016 15:59:31 -0700 Subject: [PATCH 0765/1470] Preparing Spark release v2.0.0-rc1 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 32 files changed, 32 insertions(+), 32 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 75ac9262cbae5..5f546bb9ee697 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5444ae6d70a92..2eaa8100a61d4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index e736436aec4cf..f068d9d4ac428 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3b7ffe827705b..fd221883b1fba 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index bbbb0bd5aa050..a17aba5abe548 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 14e94eca93b22..0bd8846f452c5 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index d0d1da69ea802..f495b0dddee54 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index b1f0b03b4a589..093ca69abb866 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 771da5b9a6e6e..fe9f85ae5a085 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 21d40863b77f5..ca5e4980eb8c7 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index ac15b93c048da..b4cdbda6b613b 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 016af934bb897..4e7e17de6a417 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f51d334de0876..cc5a13721ee3f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 74a3ee1ce11e2..636b233a0de04 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 3cc288abeaa22..1438e2ba59e6a 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 4a20b78917efa..dc704375dcc44 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 6fb88ebae5b32..af0b81a0cff82 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b5f5ff2854cfb..919778bac01e9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index bfb92791de3d8..f7588d1917ba5 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc6c700dd1ec8..f8dd7f7fd751c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e7303853e6565..e015b95cafd15 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1c6ab2b62d8f0..86edf650b5c7b 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a59fcdff7aae..407cb6da8281c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 7f9ea44b7e3a5..9c2c16efe25ef 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index c12d121c61156..9ca84e986a3dc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 1923199f4b861..259e56e4afef9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index b833b9369ec64..0b06c29302d6e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 809d36dc69b99..ce66c71ccae6e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c8b20f0afc4ea..5ad0c51ad05c8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 3f6774593644d..21d8d062bd380 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 9bb20e1381067..a2124104f4fb9 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index e07b93ab95450..1ca840d8c0968 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.0 ../pom.xml From 5a4fce456f8d94f27e847e00840b5a640f11486d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 21 Jun 2016 15:59:37 -0700 Subject: [PATCH 0766/1470] Preparing development version 2.0.1-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 32 files changed, 32 insertions(+), 32 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5f546bb9ee697..507ddc778383d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2eaa8100a61d4..bc3b0fe73f6eb 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f068d9d4ac428..2fb5835305a25 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index fd221883b1fba..07d9f1c58f7a3 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index a17aba5abe548..5e02efdc45e67 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 0bd8846f452c5..e7fc6a2a02417 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index f495b0dddee54..24f0e75f2f045 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 093ca69abb866..bb27ec916c65a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index fe9f85ae5a085..d2227944d92dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ca5e4980eb8c7..18e14c7981d80 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index b4cdbda6b613b..3847b39e17315 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 4e7e17de6a417..7fc8a4b00dc20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index cc5a13721ee3f..dcdbcc55d8785 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 636b233a0de04..1eb349ff4c3b9 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 1438e2ba59e6a..a346d76210932 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dc704375dcc44..2aa7d82e8e064 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index af0b81a0cff82..58c57c1c539df 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 919778bac01e9..b524001d04719 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index f7588d1917ba5..93ac8b6e664d7 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f8dd7f7fd751c..4f8af77792b0b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e015b95cafd15..b6591598ee127 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 86edf650b5c7b..1d8f7f4d9bbed 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 407cb6da8281c..40fde1bab7ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 9c2c16efe25ef..e2730ee1c74c0 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 9ca84e986a3dc..0b5ec1a08c821 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 259e56e4afef9..0bfdb13cec941 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0b06c29302d6e..02a18b33b087a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index ce66c71ccae6e..672425c86ecbe 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5ad0c51ad05c8..add4375364b1a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 21d8d062bd380..e7614fe66110d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2124104f4fb9..3f4cce1ca354e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 1ca840d8c0968..7dba1a829fb94 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml From 77d8226dfc93fc5f7cde3cc601984fc1a1a54be5 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Wed, 22 Jun 2016 10:37:13 +0800 Subject: [PATCH 0767/1470] [SQL][DOC] SQL programming guide add deprecated methods in 2.0.0 ## What changes were proposed in this pull request? Doc changes ## How was this patch tested? manual liancheng Author: Felix Cheung Closes #13827 from felixcheung/sqldocdeprecate. (cherry picked from commit 79aa1d82ca56eb847cbf4ff81de0564b339988f6) Signed-off-by: Cheng Lian --- docs/sql-programming-guide.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index ddf8f701ca036..4b52c942e5449 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -2143,7 +2143,7 @@ options. ## Upgrading From Spark SQL 1.6 to 2.0 - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and - `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed @@ -2153,6 +2153,10 @@ options. APIs. Instead, `DataFrame` remains the primary programing abstraction, which is analogous to the single-node data frame notion in these languages. + - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` + ## Upgrading From Spark SQL 1.5 to 1.6 - From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC From f3a2ebe0bcac6d6d3b370ce66b5458e55b7e884d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 21 Jun 2016 20:53:38 -0700 Subject: [PATCH 0768/1470] [MINOR][MLLIB] deprecate setLabelCol in ChiSqSelectorModel ## What changes were proposed in this pull request? Deprecate `labelCol`, which is not used by ChiSqSelectorModel. Author: Xiangrui Meng Closes #13823 from mengxr/deprecate-setLabelCol-in-ChiSqSelectorModel. (cherry picked from commit d77c4e6e2eef24f4276c38b3add8c29bb885f4db) Signed-off-by: Joseph K. Bradley --- .../main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 1c329267d70d8..33723287b0558 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -142,6 +142,7 @@ final class ChiSqSelectorModel private[ml] ( /** @group setParam */ @Since("1.6.0") + @deprecated("labelCol is not used by ChiSqSelectorModel.", "2.0.0") def setLabelCol(value: String): this.type = set(labelCol, value) @Since("2.0.0") From e7a489c7fef895fd2ca651f1c67b5495862b3e3e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 21 Jun 2016 23:12:08 -0700 Subject: [PATCH 0769/1470] [SPARK-15644][MLLIB][SQL] Replace SQLContext with SparkSession in MLlib #### What changes were proposed in this pull request? This PR is to use the latest `SparkSession` to replace the existing `SQLContext` in `MLlib`. `SQLContext` is removed from `MLlib`. Also fix a test case issue in `BroadcastJoinSuite`. BTW, `SQLContext` is not being used in the `MLlib` test suites. #### How was this patch tested? Existing test cases. Author: gatorsmile Author: xiaoli Author: Xiao Li Closes #13380 from gatorsmile/sqlContextML. (cherry picked from commit 0e3ce75332dd536c0db8467d456ad46e4bf228f4) Signed-off-by: Joseph K. Bradley --- .../DecisionTreeClassifier.scala | 4 +- .../ml/classification/GBTClassifier.scala | 4 +- .../classification/LogisticRegression.scala | 4 +- .../MultilayerPerceptronClassifier.scala | 4 +- .../spark/ml/classification/NaiveBayes.scala | 4 +- .../RandomForestClassifier.scala | 4 +- .../spark/ml/clustering/GaussianMixture.scala | 4 +- .../apache/spark/ml/clustering/KMeans.scala | 10 ++--- .../spark/ml/feature/ChiSqSelector.scala | 4 +- .../spark/ml/feature/CountVectorizer.scala | 4 +- .../org/apache/spark/ml/feature/IDF.scala | 4 +- .../spark/ml/feature/MaxAbsScaler.scala | 4 +- .../spark/ml/feature/MinMaxScaler.scala | 4 +- .../org/apache/spark/ml/feature/PCA.scala | 6 +-- .../apache/spark/ml/feature/RFormula.scala | 12 +++--- .../spark/ml/feature/StandardScaler.scala | 4 +- .../spark/ml/feature/StringIndexer.scala | 4 +- .../spark/ml/feature/VectorIndexer.scala | 4 +- .../apache/spark/ml/feature/Word2Vec.scala | 4 +- .../apache/spark/ml/recommendation/ALS.scala | 4 +- .../ml/regression/AFTSurvivalRegression.scala | 4 +- .../ml/regression/DecisionTreeRegressor.scala | 4 +- .../spark/ml/regression/GBTRegressor.scala | 4 +- .../GeneralizedLinearRegression.scala | 4 +- .../ml/regression/IsotonicRegression.scala | 4 +- .../ml/regression/LinearRegression.scala | 4 +- .../ml/regression/RandomForestRegressor.scala | 4 +- .../org/apache/spark/ml/tree/treeModels.scala | 12 +++--- .../org/apache/spark/ml/util/ReadWrite.scala | 41 ++++++++++++++----- .../ml/util/JavaDefaultReadWriteSuite.java | 2 +- .../org/apache/spark/sql/SparkSession.scala | 2 +- 31 files changed, 100 insertions(+), 81 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 881dcefb79be3..c65d3d5b54423 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -243,7 +243,7 @@ object DecisionTreeClassificationModel extends MLReadable[DecisionTreeClassifica DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) val (nodeData, _) = NodeData.build(instance.rootNode, 0) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(nodeData).write.parquet(dataPath) + sparkSession.createDataFrame(nodeData).write.parquet(dataPath) } } @@ -258,7 +258,7 @@ object DecisionTreeClassificationModel extends MLReadable[DecisionTreeClassifica val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numClasses = (metadata.metadata \ "numClasses").extract[Int] - val root = loadTreeNodes(path, metadata, sqlContext) + val root = loadTreeNodes(path, metadata, sparkSession) val model = new DecisionTreeClassificationModel(metadata.uid, root, numFeatures, numClasses) DefaultParamsReader.getAndSetParams(model, metadata) model diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index f843df449c614..4e534baddc633 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -270,7 +270,7 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { val extraMetadata: JObject = Map( "numFeatures" -> instance.numFeatures, "numTrees" -> instance.getNumTrees) - EnsembleModelReadWrite.saveImpl(instance, path, sqlContext, extraMetadata) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) } } @@ -283,7 +283,7 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { override def load(path: String): GBTClassificationModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = - EnsembleModelReadWrite.loadImpl(path, sqlContext, className, treeClassName) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] 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 9469acf62e13d..a7ba39e432f83 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 @@ -660,7 +660,7 @@ object LogisticRegressionModel extends MLReadable[LogisticRegressionModel] { val data = Data(instance.numClasses, instance.numFeatures, instance.intercept, instance.coefficients) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -674,7 +674,7 @@ object LogisticRegressionModel extends MLReadable[LogisticRegressionModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.format("parquet").load(dataPath) + val data = sparkSession.read.format("parquet").load(dataPath) .select("numClasses", "numFeatures", "intercept", "coefficients").head() // We will need numClasses, numFeatures in the future for multinomial logreg support. // val numClasses = data.getInt(0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index c4e882240ffd2..700542117ee77 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -356,7 +356,7 @@ object MultilayerPerceptronClassificationModel // Save model data: layers, weights val data = Data(instance.layers, instance.weights) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -370,7 +370,7 @@ object MultilayerPerceptronClassificationModel val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath).select("layers", "weights").head() + val data = sparkSession.read.parquet(dataPath).select("layers", "weights").head() val layers = data.getAs[Seq[Int]](0).toArray val weights = data.getAs[Vector](1) val model = new MultilayerPerceptronClassificationModel(metadata.uid, layers, weights) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index a98bdeca6b723..a9d493032b28a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -262,7 +262,7 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { // Save model data: pi, theta val data = Data(instance.pi, instance.theta) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -275,7 +275,7 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath).select("pi", "theta").head() + val data = sparkSession.read.parquet(dataPath).select("pi", "theta").head() val pi = data.getAs[Vector](0) val theta = data.getAs[Matrix](1) val model = new NaiveBayesModel(metadata.uid, pi, theta) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index b3c074f839250..9a26a5c5b1431 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -282,7 +282,7 @@ object RandomForestClassificationModel extends MLReadable[RandomForestClassifica "numFeatures" -> instance.numFeatures, "numClasses" -> instance.numClasses, "numTrees" -> instance.getNumTrees) - EnsembleModelReadWrite.saveImpl(instance, path, sqlContext, extraMetadata) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) } } @@ -296,7 +296,7 @@ object RandomForestClassificationModel extends MLReadable[RandomForestClassifica override def load(path: String): RandomForestClassificationModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], _) = - EnsembleModelReadWrite.loadImpl(path, sqlContext, className, treeClassName) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numClasses = (metadata.metadata \ "numClasses").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 563a3b14e9a6f..81749055c7613 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -195,7 +195,7 @@ object GaussianMixtureModel extends MLReadable[GaussianMixtureModel] { val sigmas = gaussians.map(c => OldMatrices.fromML(c.cov)) val data = Data(weights, mus, sigmas) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -208,7 +208,7 @@ object GaussianMixtureModel extends MLReadable[GaussianMixtureModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val row = sqlContext.read.parquet(dataPath).select("weights", "mus", "sigmas").head() + val row = sparkSession.read.parquet(dataPath).select("weights", "mus", "sigmas").head() val weights = row.getSeq[Double](0).toArray val mus = row.getSeq[OldVector](1).toArray val sigmas = row.getSeq[OldMatrix](2).toArray diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 790ef1fe8dc94..6f63d0481896e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -211,7 +211,7 @@ object KMeansModel extends MLReadable[KMeansModel] { Data(idx, center) } val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(data).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(data).repartition(1).write.parquet(dataPath) } } @@ -222,8 +222,8 @@ object KMeansModel extends MLReadable[KMeansModel] { override def load(path: String): KMeansModel = { // Import implicits for Dataset Encoder - val sqlContext = super.sqlContext - import sqlContext.implicits._ + val sparkSession = super.sparkSession + import sparkSession.implicits._ val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString @@ -232,11 +232,11 @@ object KMeansModel extends MLReadable[KMeansModel] { val versionRegex(major, _) = metadata.sparkVersion val clusterCenters = if (major.toInt >= 2) { - val data: Dataset[Data] = sqlContext.read.parquet(dataPath).as[Data] + val data: Dataset[Data] = sparkSession.read.parquet(dataPath).as[Data] data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) } else { // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. - sqlContext.read.parquet(dataPath).as[OldData].head().clusterCenters + sparkSession.read.parquet(dataPath).as[OldData].head().clusterCenters } val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) DefaultParamsReader.getAndSetParams(model, metadata) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 33723287b0558..38b4db99bc298 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -202,7 +202,7 @@ object ChiSqSelectorModel extends MLReadable[ChiSqSelectorModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.selectedFeatures.toSeq) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -213,7 +213,7 @@ object ChiSqSelectorModel extends MLReadable[ChiSqSelectorModel] { override def load(path: String): ChiSqSelectorModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath).select("selectedFeatures").head() + val data = sparkSession.read.parquet(dataPath).select("selectedFeatures").head() val selectedFeatures = data.getAs[Seq[Int]](0).toArray val oldModel = new feature.ChiSqSelectorModel(selectedFeatures) val model = new ChiSqSelectorModel(metadata.uid, oldModel) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 3250fe55980d0..96e6f1c512e90 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -297,7 +297,7 @@ object CountVectorizerModel extends MLReadable[CountVectorizerModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.vocabulary) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -308,7 +308,7 @@ object CountVectorizerModel extends MLReadable[CountVectorizerModel] { override def load(path: String): CountVectorizerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("vocabulary") .head() val vocabulary = data.getAs[Seq[String]](0).toArray diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index cf03a2845ced5..02d4e6a9f7555 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -168,7 +168,7 @@ object IDFModel extends MLReadable[IDFModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.idf) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -179,7 +179,7 @@ object IDFModel extends MLReadable[IDFModel] { override def load(path: String): IDFModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("idf") .head() val idf = data.getAs[Vector](0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index 31a58152671cd..acabf0b892660 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -161,7 +161,7 @@ object MaxAbsScalerModel extends MLReadable[MaxAbsScalerModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = new Data(instance.maxAbs) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -172,7 +172,7 @@ object MaxAbsScalerModel extends MLReadable[MaxAbsScalerModel] { override def load(path: String): MaxAbsScalerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val Row(maxAbs: Vector) = sqlContext.read.parquet(dataPath) + val Row(maxAbs: Vector) = sparkSession.read.parquet(dataPath) .select("maxAbs") .head() val model = new MaxAbsScalerModel(metadata.uid, maxAbs) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index dd5a1f9b41fc8..562b3f38e4ec6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -221,7 +221,7 @@ object MinMaxScalerModel extends MLReadable[MinMaxScalerModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = new Data(instance.originalMin, instance.originalMax) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -232,7 +232,7 @@ object MinMaxScalerModel extends MLReadable[MinMaxScalerModel] { override def load(path: String): MinMaxScalerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val Row(originalMin: Vector, originalMax: Vector) = sqlContext.read.parquet(dataPath) + val Row(originalMin: Vector, originalMax: Vector) = sparkSession.read.parquet(dataPath) .select("originalMin", "originalMax") .head() val model = new MinMaxScalerModel(metadata.uid, originalMin, originalMax) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index b89c85991f39a..72167b50e384c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -186,7 +186,7 @@ object PCAModel extends MLReadable[PCAModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.pc, instance.explainedVariance) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -217,12 +217,12 @@ object PCAModel extends MLReadable[PCAModel] { val dataPath = new Path(path, "data").toString val model = if (hasExplainedVariance) { val Row(pc: DenseMatrix, explainedVariance: DenseVector) = - sqlContext.read.parquet(dataPath) + sparkSession.read.parquet(dataPath) .select("pc", "explainedVariance") .head() new PCAModel(metadata.uid, pc, explainedVariance) } else { - val Row(pc: DenseMatrix) = sqlContext.read.parquet(dataPath).select("pc").head() + val Row(pc: DenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() new PCAModel(metadata.uid, pc, Vectors.dense(Array.empty[Double]).asInstanceOf[DenseVector]) } DefaultParamsReader.getAndSetParams(model, metadata) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 546dc7e8c08ff..c95dacfce8cfa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -297,7 +297,7 @@ object RFormulaModel extends MLReadable[RFormulaModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) // Save model data: resolvedFormula val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(instance.resolvedFormula)) + sparkSession.createDataFrame(Seq(instance.resolvedFormula)) .repartition(1).write.parquet(dataPath) // Save pipeline model val pmPath = new Path(path, "pipelineModel").toString @@ -314,7 +314,7 @@ object RFormulaModel extends MLReadable[RFormulaModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath).select("label", "terms", "hasIntercept").head() + val data = sparkSession.read.parquet(dataPath).select("label", "terms", "hasIntercept").head() val label = data.getString(0) val terms = data.getAs[Seq[Seq[String]]](1) val hasIntercept = data.getBoolean(2) @@ -372,7 +372,7 @@ private object ColumnPruner extends MLReadable[ColumnPruner] { // Save model data: columnsToPrune val data = Data(instance.columnsToPrune.toSeq) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -385,7 +385,7 @@ private object ColumnPruner extends MLReadable[ColumnPruner] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath).select("columnsToPrune").head() + val data = sparkSession.read.parquet(dataPath).select("columnsToPrune").head() val columnsToPrune = data.getAs[Seq[String]](0).toSet val pruner = new ColumnPruner(metadata.uid, columnsToPrune) @@ -463,7 +463,7 @@ private object VectorAttributeRewriter extends MLReadable[VectorAttributeRewrite // Save model data: vectorCol, prefixesToRewrite val data = Data(instance.vectorCol, instance.prefixesToRewrite) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -476,7 +476,7 @@ private object VectorAttributeRewriter extends MLReadable[VectorAttributeRewrite val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath).select("vectorCol", "prefixesToRewrite").head() + val data = sparkSession.read.parquet(dataPath).select("vectorCol", "prefixesToRewrite").head() val vectorCol = data.getString(0) val prefixesToRewrite = data.getAs[Map[String, String]](1) val rewriter = new VectorAttributeRewriter(metadata.uid, vectorCol, prefixesToRewrite) 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 5e1bacf876caf..be58dc27e0602 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 @@ -200,7 +200,7 @@ object StandardScalerModel extends MLReadable[StandardScalerModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.std, instance.mean) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -211,7 +211,7 @@ object StandardScalerModel extends MLReadable[StandardScalerModel] { override def load(path: String): StandardScalerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val Row(std: Vector, mean: Vector) = sqlContext.read.parquet(dataPath) + val Row(std: Vector, mean: Vector) = sparkSession.read.parquet(dataPath) .select("std", "mean") .head() val model = new StandardScalerModel(metadata.uid, std, mean) 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 0f7337ce6b554..028e540fe5356 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 @@ -221,7 +221,7 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.labels) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -232,7 +232,7 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { override def load(path: String): StringIndexerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("labels") .head() val labels = data.getAs[Seq[String]](0).toArray 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 52db996c841b3..5656a9f979fc1 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 @@ -450,7 +450,7 @@ object VectorIndexerModel extends MLReadable[VectorIndexerModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.numFeatures, instance.categoryMaps) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -461,7 +461,7 @@ object VectorIndexerModel extends MLReadable[VectorIndexerModel] { override def load(path: String): VectorIndexerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("numFeatures", "categoryMaps") .head() val numFeatures = data.getAs[Int](0) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 05c4f2f1a7891..a74d31ff9d611 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -310,7 +310,7 @@ object Word2VecModel extends MLReadable[Word2VecModel] { DefaultParamsWriter.saveMetadata(instance, path, sc) val data = Data(instance.wordVectors.wordIndex, instance.wordVectors.wordVectors.toSeq) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -321,7 +321,7 @@ object Word2VecModel extends MLReadable[Word2VecModel] { override def load(path: String): Word2VecModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("wordIndex", "wordVectors") .head() val wordIndex = data.getAs[Map[String, Int]](0) 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 2404a69e9e8b5..5dc2433e55c39 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 @@ -320,9 +320,9 @@ object ALSModel extends MLReadable[ALSModel] { implicit val format = DefaultFormats val rank = (metadata.metadata \ "rank").extract[Int] val userPath = new Path(path, "userFactors").toString - val userFactors = sqlContext.read.format("parquet").load(userPath) + val userFactors = sparkSession.read.format("parquet").load(userPath) val itemPath = new Path(path, "itemFactors").toString - val itemFactors = sqlContext.read.format("parquet").load(itemPath) + val itemFactors = sparkSession.read.format("parquet").load(itemPath) val model = new ALSModel(metadata.uid, rank, userFactors, itemFactors) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 7f57af19e9df9..fe65e3e810f0a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -375,7 +375,7 @@ object AFTSurvivalRegressionModel extends MLReadable[AFTSurvivalRegressionModel] // Save model data: coefficients, intercept, scale val data = Data(instance.coefficients, instance.intercept, instance.scale) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -388,7 +388,7 @@ object AFTSurvivalRegressionModel extends MLReadable[AFTSurvivalRegressionModel] val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("coefficients", "intercept", "scale").head() val coefficients = data.getAs[Vector](0) val intercept = data.getDouble(1) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index c4df9d11127f4..7ff6d0afd55c2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -249,7 +249,7 @@ object DecisionTreeRegressionModel extends MLReadable[DecisionTreeRegressionMode DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) val (nodeData, _) = NodeData.build(instance.rootNode, 0) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(nodeData).write.parquet(dataPath) + sparkSession.createDataFrame(nodeData).write.parquet(dataPath) } } @@ -263,7 +263,7 @@ object DecisionTreeRegressionModel extends MLReadable[DecisionTreeRegressionMode implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] - val root = loadTreeNodes(path, metadata, sqlContext) + val root = loadTreeNodes(path, metadata, sparkSession) val model = new DecisionTreeRegressionModel(metadata.uid, root, numFeatures) DefaultParamsReader.getAndSetParams(model, metadata) model diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 81f2139f0b421..6223555504d71 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -252,7 +252,7 @@ object GBTRegressionModel extends MLReadable[GBTRegressionModel] { val extraMetadata: JObject = Map( "numFeatures" -> instance.numFeatures, "numTrees" -> instance.getNumTrees) - EnsembleModelReadWrite.saveImpl(instance, path, sqlContext, extraMetadata) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) } } @@ -265,7 +265,7 @@ object GBTRegressionModel extends MLReadable[GBTRegressionModel] { override def load(path: String): GBTRegressionModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = - EnsembleModelReadWrite.loadImpl(path, sqlContext, className, treeClassName) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index adbdd345e92ec..a23e90d9e1259 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -813,7 +813,7 @@ object GeneralizedLinearRegressionModel extends MLReadable[GeneralizedLinearRegr // Save model data: intercept, coefficients val data = Data(instance.intercept, instance.coefficients) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -827,7 +827,7 @@ object GeneralizedLinearRegressionModel extends MLReadable[GeneralizedLinearRegr val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("intercept", "coefficients").head() val intercept = data.getDouble(0) val coefficients = data.getAs[Vector](1) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index d16e8e3f6b259..f05b47eda7b63 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -284,7 +284,7 @@ object IsotonicRegressionModel extends MLReadable[IsotonicRegressionModel] { val data = Data( instance.oldModel.boundaries, instance.oldModel.predictions, instance.oldModel.isotonic) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -297,7 +297,7 @@ object IsotonicRegressionModel extends MLReadable[IsotonicRegressionModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) .select("boundaries", "predictions", "isotonic").head() val boundaries = data.getAs[Seq[Double]](0).toArray val predictions = data.getAs[Seq[Double]](1).toArray 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 52ec40e15b213..5e8ef1b375001 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 @@ -486,7 +486,7 @@ object LinearRegressionModel extends MLReadable[LinearRegressionModel] { // Save model data: intercept, coefficients val data = Data(instance.intercept, instance.coefficients) val dataPath = new Path(path, "data").toString - sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } @@ -499,7 +499,7 @@ object LinearRegressionModel extends MLReadable[LinearRegressionModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sqlContext.read.format("parquet").load(dataPath) + val data = sparkSession.read.format("parquet").load(dataPath) .select("intercept", "coefficients").head() val intercept = data.getDouble(0) val coefficients = data.getAs[Vector](1) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index a6dbf21d55e2b..4f4d3d27841da 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -244,7 +244,7 @@ object RandomForestRegressionModel extends MLReadable[RandomForestRegressionMode val extraMetadata: JObject = Map( "numFeatures" -> instance.numFeatures, "numTrees" -> instance.getNumTrees) - EnsembleModelReadWrite.saveImpl(instance, path, sqlContext, extraMetadata) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) } } @@ -257,7 +257,7 @@ object RandomForestRegressionModel extends MLReadable[RandomForestRegressionMode override def load(path: String): RandomForestRegressionModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = - EnsembleModelReadWrite.loadImpl(path, sqlContext, className, treeClassName) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index 56c85c9b53e17..5b6fcc53c2dd5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -31,7 +31,7 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, SQLContext} +import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.util.collection.OpenHashMap /** @@ -332,8 +332,8 @@ private[ml] object DecisionTreeModelReadWrite { def loadTreeNodes( path: String, metadata: DefaultParamsReader.Metadata, - sqlContext: SQLContext): Node = { - import sqlContext.implicits._ + sparkSession: SparkSession): Node = { + import sparkSession.implicits._ implicit val format = DefaultFormats // Get impurity to construct ImpurityCalculator for each node @@ -343,7 +343,7 @@ private[ml] object DecisionTreeModelReadWrite { } val dataPath = new Path(path, "data").toString - val data = sqlContext.read.parquet(dataPath).as[NodeData] + val data = sparkSession.read.parquet(dataPath).as[NodeData] buildTreeFromNodes(data.collect(), impurityType) } @@ -393,7 +393,7 @@ private[ml] object EnsembleModelReadWrite { def saveImpl[M <: Params with TreeEnsembleModel[_ <: DecisionTreeModel]]( instance: M, path: String, - sql: SQLContext, + sql: SparkSession, extraMetadata: JObject): Unit = { DefaultParamsWriter.saveMetadata(instance, path, sql.sparkContext, Some(extraMetadata)) val treesMetadataWeights: Array[(Int, String, Double)] = instance.trees.zipWithIndex.map { @@ -424,7 +424,7 @@ private[ml] object EnsembleModelReadWrite { */ def loadImpl( path: String, - sql: SQLContext, + sql: SparkSession, className: String, treeClassName: String): (Metadata, Array[(Metadata, Node)], Array[Double]) = { import sql.implicits._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index 90b8d7df7b496..1582a73ea047b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -40,28 +40,41 @@ import org.apache.spark.util.Utils * Trait for [[MLWriter]] and [[MLReader]]. */ private[util] sealed trait BaseReadWrite { - private var optionSQLContext: Option[SQLContext] = None + private var optionSparkSession: Option[SparkSession] = None /** - * Sets the SQL context to use for saving/loading. + * Sets the Spark SQLContext to use for saving/loading. */ @Since("1.6.0") + @deprecated("Use session instead", "2.0.0") def context(sqlContext: SQLContext): this.type = { - optionSQLContext = Option(sqlContext) + optionSparkSession = Option(sqlContext.sparkSession) this } /** - * Returns the user-specified SQL context or the default. + * Sets the Spark Session to use for saving/loading. */ - protected final def sqlContext: SQLContext = { - if (optionSQLContext.isEmpty) { - optionSQLContext = Some(SQLContext.getOrCreate(SparkContext.getOrCreate())) + @Since("2.0.0") + def session(sparkSession: SparkSession): this.type = { + optionSparkSession = Option(sparkSession) + this + } + + /** + * Returns the user-specified Spark Session or the default. + */ + protected final def sparkSession: SparkSession = { + if (optionSparkSession.isEmpty) { + optionSparkSession = Some(SparkSession.builder().getOrCreate()) } - optionSQLContext.get + optionSparkSession.get } - protected final def sparkSession: SparkSession = sqlContext.sparkSession + /** + * Returns the user-specified SQL context or the default. + */ + protected final def sqlContext: SQLContext = sparkSession.sqlContext /** Returns the underlying [[SparkContext]]. */ protected final def sc: SparkContext = sparkSession.sparkContext @@ -118,7 +131,10 @@ abstract class MLWriter extends BaseReadWrite with Logging { } // override for Java compatibility - override def context(sqlContext: SQLContext): this.type = super.context(sqlContext) + override def session(sparkSession: SparkSession): this.type = super.session(sparkSession) + + // override for Java compatibility + override def context(sqlContext: SQLContext): this.type = super.session(sqlContext.sparkSession) } /** @@ -180,7 +196,10 @@ abstract class MLReader[T] extends BaseReadWrite { def load(path: String): T // override for Java compatibility - override def context(sqlContext: SQLContext): this.type = super.context(sqlContext) + override def session(sparkSession: SparkSession): this.type = super.session(sparkSession) + + // override for Java compatibility + override def context(sqlContext: SQLContext): this.type = super.session(sqlContext.sparkSession) } /** diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java index 7bda219243bf5..e4f678fef1d13 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -56,7 +56,7 @@ public void testDefaultReadWrite() throws IOException { } catch (IOException e) { // expected } - instance.write().context(spark.sqlContext()).overwrite().save(outputPath); + instance.write().session(spark).overwrite().save(outputPath); MyParams newInstance = MyParams.load(outputPath); Assert.assertEquals("UID should match.", instance.uid(), newInstance.uid()); Assert.assertEquals("Params should be preserved.", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 251f47d5fb1fd..a3fd39d42eeb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -110,7 +110,7 @@ class SparkSession private( * A wrapped version of this session in the form of a [[SQLContext]], for backward compatibility. */ @transient - private[sql] val sqlContext: SQLContext = new SQLContext(this) + private[spark] val sqlContext: SQLContext = new SQLContext(this) /** * Runtime configuration interface for Spark. From 838143a2a02192a9ebc955b8060a6520b62d9644 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 22 Jun 2016 11:54:49 +0200 Subject: [PATCH 0770/1470] [SPARK-15162][SPARK-15164][PYSPARK][DOCS][ML] update some pydocs ## What changes were proposed in this pull request? Mark ml.classification algorithms as experimental to match Scala algorithms, update PyDoc for for thresholds on `LogisticRegression` to have same level of info as Scala, and enable mathjax for PyDoc. ## How was this patch tested? Built docs locally & PySpark SQL tests Author: Holden Karau Closes #12938 from holdenk/SPARK-15162-SPARK-15164-update-some-pydocs. (cherry picked from commit d281b0bafe6aa23085d4d2b68f0ce321f1978b50) Signed-off-by: Nick Pentreath --- .../classification/LogisticRegression.scala | 5 +-- python/docs/conf.py | 1 + python/pyspark/ml/classification.py | 38 ++++++++++++++++++- 3 files changed, 39 insertions(+), 5 deletions(-) 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 a7ba39e432f83..2fa8fbcc76a25 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 @@ -72,10 +72,9 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas /** * Get threshold for binary classification. * - * If [[threshold]] is set, returns that value. - * Otherwise, if [[thresholds]] is set with length 2 (i.e., binary classification), + * If [[thresholds]] is set with length 2 (i.e., binary classification), * this returns the equivalent threshold: {{{1 / (1 + thresholds(0) / thresholds(1))}}}. - * Otherwise, returns [[threshold]] default value. + * Otherwise, returns [[threshold]] if set, or its default value if unset. * * @group getParam * @throws IllegalArgumentException if [[thresholds]] is set to an array of length other than 2. diff --git a/python/docs/conf.py b/python/docs/conf.py index d35bf73c30510..50fb3175a7dc4 100644 --- a/python/docs/conf.py +++ b/python/docs/conf.py @@ -32,6 +32,7 @@ 'sphinx.ext.autodoc', 'sphinx.ext.viewcode', 'epytext', + 'sphinx.ext.mathjax', ] # Add any paths that contain templates here, relative to this directory. diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index e86c27ecaf3df..d6d713ca53035 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -49,6 +49,8 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti HasElasticNetParam, HasFitIntercept, HasStandardization, HasThresholds, HasWeightCol, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Logistic regression. Currently, this class only supports binary classification. @@ -96,7 +98,8 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti threshold = Param(Params._dummy(), "threshold", "Threshold in binary classification prediction, in range [0, 1]." + - " If threshold and thresholds are both set, they must match.", + " If threshold and thresholds are both set, they must match." + + "e.g. if threshold is p, then thresholds must be equal to [1-p, p].", typeConverter=TypeConverters.toFloat) @keyword_only @@ -154,7 +157,12 @@ def setThreshold(self, value): @since("1.4.0") def getThreshold(self): """ - Gets the value of threshold or its default value. + Get threshold for binary classification. + + If :py:attr:`thresholds` is set with length 2 (i.e., binary classification), + this returns the equivalent threshold: + :math:`\\frac{1}{1 + \\frac{thresholds(0)}{thresholds(1)}}`. + Otherwise, returns :py:attr:`threshold` if set or its default value if unset. """ self._checkThresholdConsistency() if self.isSet(self.thresholds): @@ -208,6 +216,8 @@ def _checkThresholdConsistency(self): class LogisticRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Model fitted by LogisticRegression. .. versionadded:: 1.3.0 @@ -491,6 +501,8 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred TreeClassifierParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + `Decision tree `_ learning algorithm for classification. It supports both binary and multiclass labels, as well as both continuous and categorical @@ -587,6 +599,8 @@ def _create_model(self, java_model): @inherit_doc class DecisionTreeClassificationModel(DecisionTreeModel, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Model fitted by DecisionTreeClassifier. .. versionadded:: 1.4.0 @@ -620,6 +634,8 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred RandomForestParams, TreeClassifierParams, HasCheckpointInterval, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + `Random Forest `_ learning algorithm for classification. It supports both binary and multiclass labels, as well as both continuous and categorical @@ -714,6 +730,8 @@ def _create_model(self, java_model): class RandomForestClassificationModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Model fitted by RandomForestClassifier. .. versionadded:: 1.4.0 @@ -746,6 +764,8 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol GBTParams, HasCheckpointInterval, HasStepSize, HasSeed, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + `Gradient-Boosted Trees (GBTs) `_ learning algorithm for classification. It supports binary labels, as well as both continuous and categorical features. @@ -865,6 +885,8 @@ def getLossType(self): class GBTClassificationModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Model fitted by GBTClassifier. .. versionadded:: 1.4.0 @@ -896,6 +918,8 @@ def trees(self): class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, HasRawPredictionCol, HasThresholds, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Naive Bayes Classifiers. It supports both Multinomial and Bernoulli NB. `Multinomial NB `_ @@ -1019,6 +1043,8 @@ def getModelType(self): class NaiveBayesModel(JavaModel, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Model fitted by NaiveBayes. .. versionadded:: 1.5.0 @@ -1046,6 +1072,8 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasMaxIter, HasTol, HasSeed, HasStepSize, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Classifier trainer based on the Multilayer Perceptron. Each layer has sigmoid activation function, output layer has softmax. Number of inputs has to be equal to the size of feature vectors. @@ -1216,6 +1244,8 @@ def getInitialWeights(self): class MultilayerPerceptronClassificationModel(JavaModel, JavaMLWritable, JavaMLReadable): """ + .. note:: Experimental + Model fitted by MultilayerPerceptronClassifier. .. versionadded:: 1.6.0 @@ -1265,6 +1295,8 @@ def getClassifier(self): @inherit_doc class OneVsRest(Estimator, OneVsRestParams, MLReadable, MLWritable): """ + .. note:: Experimental + Reduction of Multiclass Classification to Binary Classification. Performs reduction using one against all strategy. For a multiclass classification with k classes, train k models (one per class). @@ -1419,6 +1451,8 @@ def _to_java(self): class OneVsRestModel(Model, OneVsRestParams, MLReadable, MLWritable): """ + .. note:: Experimental + Model fitted by OneVsRest. This stores the models resulting from training k binary classifiers: one for each class. Each example is scored against all k models, and the model with the highest score From 60bd704b541c4d1991922ffd3dd5b47de9bd5821 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 22 Jun 2016 18:07:07 +0800 Subject: [PATCH 0771/1470] [SPARK-16121] ListingFileCatalog does not list in parallel anymore ## What changes were proposed in this pull request? Seems the fix of SPARK-14959 breaks the parallel partitioning discovery. This PR fixes the problem ## How was this patch tested? Tested manually. (This PR also adds a proper test for SPARK-14959) Author: Yin Huai Closes #13830 from yhuai/SPARK-16121. (cherry picked from commit 39ad53f7ffddae5ba0ff0a76089ba671b14c44c8) Signed-off-by: Cheng Lian --- .../datasources/ListingFileCatalog.scala | 58 +++++++++++++++++-- .../datasources/fileSourceInterfaces.scala | 7 ++- .../datasources/FileSourceStrategySuite.scala | 45 +++++++++++++- 3 files changed, 101 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index f713fdec4ecf3..675e755cb2d02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable import scala.util.Try -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, LocatedFileStatus, Path} import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.sql.SparkSession @@ -73,21 +73,67 @@ class ListingFileCatalog( cachedPartitionSpec = null } - protected def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + /** + * List leaf files of given paths. This method will submit a Spark job to do parallel + * listing whenever there is a path having more files than the parallel partition discovery + * discovery threshold. + */ + protected[spark] def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession) } else { + // Right now, the number of paths is less than the value of + // parallelPartitionDiscoveryThreshold. So, we will list file statues at the driver. + // If there is any child that has more files than the threshold, we will use parallel + // listing. + // Dummy jobconf to get to the pathFilter defined in configuration val jobConf = new JobConf(hadoopConf, this.getClass) val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + val statuses: Seq[FileStatus] = paths.flatMap { path => val fs = path.getFileSystem(hadoopConf) logTrace(s"Listing $path on driver") - Try { - HadoopFsRelation.listLeafFiles(fs, fs.getFileStatus(path), pathFilter) - }.getOrElse(Array.empty[FileStatus]) + + val childStatuses = { + // TODO: We need to avoid of using Try at here. + val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) + if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats + } + + childStatuses.map { + case f: LocatedFileStatus => f + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // paths exceeds threshold. + case f => + if (f.isDirectory ) { + // If f is a directory, we do not need to call getFileBlockLocations (SPARK-14959). + f + } else { + HadoopFsRelation.createLocatedFileStatus(f, fs.getFileBlockLocations(f, 0, f.getLen)) + } + } + }.filterNot { status => + val name = status.getPath.getName + HadoopFsRelation.shouldFilterOut(name) + } + + val (dirs, files) = statuses.partition(_.isDirectory) + + // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) + if (dirs.isEmpty) { + mutable.LinkedHashSet(files: _*) + } else { + mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) } - mutable.LinkedHashSet(statuses: _*) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 5689cf688bfbe..20399e190f430 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -391,13 +391,14 @@ private[sql] object HadoopFsRelation extends Logging { logTrace(s"Listing ${status.getPath}") val name = status.getPath.getName.toLowerCase if (shouldFilterOut(name)) { - Array.empty + Array.empty[FileStatus] } else { val statuses = { val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) val stats = files ++ dirs.flatMap(dir => listLeafFiles(fs, dir, filter)) if (filter != null) stats.filter(f => filter.accept(f.getPath)) else stats } + // statuses do not have any dirs. statuses.filterNot(status => shouldFilterOut(status.getPath.getName)).map { case f: LocatedFileStatus => f @@ -459,7 +460,9 @@ private[sql] object HadoopFsRelation extends Logging { val pathFilter = FileInputFormat.getInputPathFilter(jobConf) paths.map(new Path(_)).flatMap { path => val fs = path.getFileSystem(serializableConfiguration.value) - Try(listLeafFiles(fs, fs.getFileStatus(path), pathFilter)).getOrElse(Array.empty) + // TODO: We need to avoid of using Try at here. + Try(listLeafFiles(fs, fs.getFileStatus(path), pathFilter)) + .getOrElse(Array.empty[FileStatus]) } }.map { status => val blockLocations = status match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 67ff257b9386a..8d8a18fa9332b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -21,7 +21,7 @@ import java.io.File import java.util.concurrent.atomic.AtomicInteger import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{BlockLocation, FileStatus, RawLocalFileSystem} +import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path, RawLocalFileSystem} import org.apache.hadoop.mapreduce.Job import org.apache.spark.SparkConf @@ -375,6 +375,38 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } } + test("SPARK-14959: Do not call getFileBlockLocations on directories") { + // Setting PARALLEL_PARTITION_DISCOVERY_THRESHOLD to 2. So we will first + // list file statues at driver side and then for the level of p2, we will list + // file statues in parallel. + withSQLConf( + "fs.file.impl" -> classOf[MockDistributedFileSystem].getName, + SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "2") { + withTempPath { path => + val tempDir = path.getCanonicalPath + + Seq("p1=1/p2=2/p3=3/file1", "p1=1/p2=3/p3=3/file1").foreach { fileName => + val file = new File(tempDir, fileName) + assert(file.getParentFile.exists() || file.getParentFile.mkdirs()) + util.stringToFile(file, fileName) + } + + val fileCatalog = new ListingFileCatalog( + sparkSession = spark, + paths = Seq(new Path(tempDir)), + parameters = Map.empty[String, String], + partitionSchema = None) + // This should not fail. + fileCatalog.listLeafFiles(Seq(new Path(tempDir))) + + // Also have an integration test. + checkAnswer( + spark.read.text(tempDir).select("p1", "p2", "p3", "value"), + Row(1, 2, 3, "p1=1/p2=2/p3=3/file1") :: Row(1, 3, 3, "p1=1/p2=3/p3=3/file1") :: Nil) + } + } + } + // Helpers for checking the arguments passed to the FileFormat. protected val checkPartitionSchema = @@ -530,3 +562,14 @@ class LocalityTestFileSystem extends RawLocalFileSystem { Array(new BlockLocation(Array(s"host$count:50010"), Array(s"host$count"), 0, len)) } } + +// This file system is for SPARK-14959 (DistributedFileSystem will throw an exception +// if we call getFileBlockLocations on a dir). +class MockDistributedFileSystem extends RawLocalFileSystem { + + override def getFileBlockLocations( + file: FileStatus, start: Long, len: Long): Array[BlockLocation] = { + require(!file.isDirectory, "The file path can not be a directory.") + super.getFileBlockLocations(file, start, len) + } +} From 503eb882c14eac9681981199ccf8f699cab23bf0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 22 Jun 2016 18:32:14 +0800 Subject: [PATCH 0772/1470] [SPARK-16097][SQL] Encoders.tuple should handle null object correctly ## What changes were proposed in this pull request? Although the top level input object can not be null, but when we use `Encoders.tuple` to combine 2 encoders, their input objects are not top level anymore and can be null. We should handle this case. ## How was this patch tested? new test in DatasetSuite Author: Wenchen Fan Closes #13807 from cloud-fan/bug. (cherry picked from commit 01277d4b259dcf9cad25eece1377162b7a8c946d) Signed-off-by: Cheng Lian --- .../catalyst/encoders/ExpressionEncoder.scala | 48 ++++++++++++++----- .../org/apache/spark/sql/DatasetSuite.scala | 7 +++ 2 files changed, 42 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 0023ce64aa99b..1fac26c4388a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, Invoke, NewInstance} import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LocalRelation} -import org.apache.spark.sql.types.{ObjectType, StructField, StructType} +import org.apache.spark.sql.types.{BooleanType, ObjectType, StructField, StructType} import org.apache.spark.util.Utils /** @@ -110,16 +110,34 @@ object ExpressionEncoder { val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}") - val serializer = encoders.map { - case e if e.flat => e.serializer.head - case other => CreateStruct(other.serializer) - }.zipWithIndex.map { case (expr, index) => - expr.transformUp { - case BoundReference(0, t, _) => - Invoke( - BoundReference(0, ObjectType(cls), nullable = true), - s"_${index + 1}", - t) + val serializer = encoders.zipWithIndex.map { case (enc, index) => + val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head + val newInputObject = Invoke( + BoundReference(0, ObjectType(cls), nullable = true), + s"_${index + 1}", + originalInputObject.dataType) + + val newSerializer = enc.serializer.map(_.transformUp { + case b: BoundReference if b == originalInputObject => newInputObject + }) + + if (enc.flat) { + newSerializer.head + } else { + // For non-flat encoder, the input object is not top level anymore after being combined to + // a tuple encoder, thus it can be null and we should wrap the `CreateStruct` with `If` and + // null check to handle null case correctly. + // e.g. for Encoder[(Int, String)], the serializer expressions will create 2 columns, and is + // not able to handle the case when the input tuple is null. This is not a problem as there + // is a check to make sure the input object won't be null. However, if this encoder is used + // to create a bigger tuple encoder, the original input object becomes a filed of the new + // input tuple and can be null. So instead of creating a struct directly here, we should add + // a null/None check and return a null struct if the null/None check fails. + val struct = CreateStruct(newSerializer) + val nullCheck = Or( + IsNull(newInputObject), + Invoke(Literal.fromObject(None), "equals", BooleanType, newInputObject :: Nil)) + If(nullCheck, Literal.create(null, struct.dataType), struct) } } @@ -203,8 +221,12 @@ case class ExpressionEncoder[T]( // (intermediate value is not an attribute). We assume that all serializer expressions use a same // `BoundReference` to refer to the object, and throw exception if they don't. assert(serializer.forall(_.references.isEmpty), "serializer cannot reference to any attributes.") - assert(serializer.flatMap(_.collect { case b: BoundReference => b}).distinct.length <= 1, - "all serializer expressions must use the same BoundReference.") + assert(serializer.flatMap { ser => + val boundRefs = ser.collect { case b: BoundReference => b } + assert(boundRefs.nonEmpty, + "each serializer expression should contains at least one `BoundReference`") + boundRefs + }.distinct.length <= 1, "all serializer expressions must use the same BoundReference.") /** * Returns a new copy of this encoder, where the `deserializer` is resolved and bound to the diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index f02a3141a050a..bd8479b2d380d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -830,6 +830,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ds.dropDuplicates("_1", "_2"), ("a", 1), ("a", 2), ("b", 1)) } + + test("SPARK-16097: Encoders.tuple should handle null object correctly") { + val enc = Encoders.tuple(Encoders.tuple(Encoders.STRING, Encoders.STRING), Encoders.STRING) + val data = Seq((("a", "b"), "c"), (null, "d")) + val ds = spark.createDataset(data)(enc) + checkDataset(ds, (("a", "b"), "c"), (null, "d")) + } } case class Generic[T](id: T, value: Double) From 1cfdd25fdb87012187b1e01f9c5ac4b6218dc840 Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Wed, 22 Jun 2016 09:13:08 -0700 Subject: [PATCH 0773/1470] [SPARK-16107][R] group glm methods in documentation ## What changes were proposed in this pull request? This groups GLM methods (spark.glm, summary, print, predict and write.ml) in the documentation. The example code was updated. ## How was this patch tested? N/A (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) ![screen shot 2016-06-21 at 2 31 37 pm](https://cloud.githubusercontent.com/assets/15318264/16247077/f6eafc04-37bc-11e6-89a8-7898ff3e4078.png) ![screen shot 2016-06-21 at 2 31 45 pm](https://cloud.githubusercontent.com/assets/15318264/16247078/f6eb1c16-37bc-11e6-940a-2b595b10617c.png) Author: Junyang Qian Author: Junyang Qian Closes #13820 from junyangq/SPARK-16107. (cherry picked from commit ea3a12b0147821960f8dabdc58d726f07f1f0e52) Signed-off-by: Xiangrui Meng --- R/pkg/R/mllib.R | 80 ++++++++++++++++++++++--------------------------- 1 file changed, 36 insertions(+), 44 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index b83b3b3d3f6d4..dbff1b900d38a 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -53,9 +53,10 @@ setClass("AFTSurvivalRegressionModel", representation(jobj = "jobj")) #' @note KMeansModel since 2.0.0 setClass("KMeansModel", representation(jobj = "jobj")) -#' Fits a generalized linear model +#' Generalized Linear Models #' -#' Fits a generalized linear model against a Spark DataFrame. +#' Fits generalized linear model against a Spark DataFrame. Users can print, make predictions on the +#' produced model and save the model to the input path. #' #' @param data SparkDataFrame for training. #' @param formula A symbolic description of the model to be fitted. Currently only a few formula @@ -66,8 +67,9 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. #' @param tol Positive convergence tolerance of iterations. #' @param maxIter Integer giving the maximal number of IRLS iterations. -#' @return a fitted generalized linear model +#' @return \code{spark.glm} returns a fitted generalized linear model #' @rdname spark.glm +#' @name spark.glm #' @export #' @examples #' \dontrun{ @@ -76,8 +78,21 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' df <- createDataFrame(iris) #' model <- spark.glm(df, Sepal_Length ~ Sepal_Width, family = "gaussian") #' summary(model) +#' +#' # fitted values on training data +#' fitted <- predict(model, df) +#' head(select(fitted, "Sepal_Length", "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) #' } #' @note spark.glm since 2.0.0 +#' @seealso \link{glm}, \link{read.ml} setMethod("spark.glm", signature(data = "SparkDataFrame", formula = "formula"), function(data, formula, family = gaussian, tol = 1e-6, maxIter = 25) { if (is.character(family)) { @@ -99,10 +114,9 @@ setMethod("spark.glm", signature(data = "SparkDataFrame", formula = "formula"), return(new("GeneralizedLinearRegressionModel", jobj = jobj)) }) -#' Fits a generalized linear model (R-compliant). +#' Generalized Linear Models (R-compliant) #' #' Fits a generalized linear model, similarly to R's glm(). -#' #' @param formula A symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', '.', ':', '+', and '-'. #' @param data SparkDataFrame for training. @@ -112,7 +126,7 @@ setMethod("spark.glm", signature(data = "SparkDataFrame", formula = "formula"), #' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. #' @param epsilon Positive convergence tolerance of iterations. #' @param maxit Integer giving the maximal number of IRLS iterations. -#' @return a fitted generalized linear model +#' @return \code{glm} returns a fitted generalized linear model. #' @rdname glm #' @export #' @examples @@ -124,24 +138,21 @@ setMethod("spark.glm", signature(data = "SparkDataFrame", formula = "formula"), #' summary(model) #' } #' @note glm since 1.5.0 +#' @seealso \link{spark.glm} setMethod("glm", signature(formula = "formula", family = "ANY", data = "SparkDataFrame"), function(formula, family = gaussian, data, epsilon = 1e-6, maxit = 25) { spark.glm(data, formula, family, tol = epsilon, maxIter = maxit) }) -#' Get the summary of a generalized linear model -#' -#' Returns the summary of a model produced by glm() or spark.glm(), similarly to R's summary(). +# Returns the summary of a model produced by glm() or spark.glm(), similarly to R's summary(). #' #' @param object A fitted generalized linear model -#' @return coefficients the model's coefficients, intercept -#' @rdname summary +#' @return \code{summary} returns a summary object of the fitted model, a list of components +#' including at least the coefficients, null/residual deviance, null/residual degrees +#' of freedom, AIC and number of iterations IRLS takes. +#' +#' @rdname spark.glm #' @export -#' @examples -#' \dontrun{ -#' model <- glm(y ~ x, trainingData) -#' summary(model) -#' } #' @note summary(GeneralizedLinearRegressionModel) since 2.0.0 setMethod("summary", signature(object = "GeneralizedLinearRegressionModel"), function(object, ...) { @@ -173,10 +184,10 @@ setMethod("summary", signature(object = "GeneralizedLinearRegressionModel"), return(ans) }) -#' Print the summary of GeneralizedLinearRegressionModel +# Prints the summary of GeneralizedLinearRegressionModel #' -#' @rdname print -#' @name print.summary.GeneralizedLinearRegressionModel +#' @rdname spark.glm +#' @param x Summary object of fitted generalized linear model returned by \code{summary} function #' @export #' @note print.summary.GeneralizedLinearRegressionModel since 2.0.0 print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { @@ -205,22 +216,13 @@ print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { invisible(x) } -#' Predicted values based on model +# Makes predictions from a generalized linear model produced by glm() or spark.glm(), +# similarly to R's predict(). #' -#' Makes predictions from a generalized linear model produced by glm() or spark.glm(), -#' similarly to R's predict(). -#' -#' @param object A fitted generalized linear model #' @param newData SparkDataFrame for testing -#' @return SparkDataFrame containing predicted labels in a column named "prediction" -#' @rdname predict +#' @return \code{predict} returns a SparkDataFrame containing predicted labels in a column named "prediction" +#' @rdname spark.glm #' @export -#' @examples -#' \dontrun{ -#' model <- glm(y ~ x, trainingData) -#' predicted <- predict(model, testData) -#' showDF(predicted) -#' } #' @note predict(GeneralizedLinearRegressionModel) since 1.5.0 setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), function(object, newData) { @@ -471,24 +473,14 @@ setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "c invisible(callJMethod(writer, "save", path)) }) -#' Save fitted MLlib model to the input path -#' -#' Save the generalized linear model to the input path. +# Saves the generalized linear model to the input path. #' -#' @param object A fitted generalized linear model #' @param path The directory where the model is saved #' @param overwrite Overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. #' -#' @rdname write.ml -#' @name write.ml +#' @rdname spark.glm #' @export -#' @examples -#' \dontrun{ -#' model <- glm(y ~ x, trainingData) -#' path <- "path/to/model" -#' write.ml(model, path) -#' } #' @note write.ml(GeneralizedLinearRegressionModel, character) since 2.0.0 setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", path = "character"), function(object, path, overwrite = FALSE) { From 0cde3ad6d8ac822b73f42b8158ba09f7be00a2c9 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Wed, 22 Jun 2016 10:05:25 -0700 Subject: [PATCH 0774/1470] [SPARK-16127][ML][PYPSARK] Audit @Since annotations related to ml.linalg [SPARK-14615](https://issues.apache.org/jira/browse/SPARK-14615) and #12627 changed `spark.ml` pipelines to use the new `ml.linalg` classes for `Vector`/`Matrix`. Some `Since` annotations for public methods/vals have not been updated accordingly to be `2.0.0`. This PR updates them. ## How was this patch tested? Existing unit tests. Author: Nick Pentreath Closes #13840 from MLnick/SPARK-16127-ml-linalg-since. (cherry picked from commit 18faa588ca11190890d2eb569d7497fbb25eee5c) Signed-off-by: Xiangrui Meng --- .../spark/ml/classification/LogisticRegression.scala | 2 +- .../MultilayerPerceptronClassifier.scala | 2 +- .../apache/spark/ml/classification/NaiveBayes.scala | 4 ++-- .../org/apache/spark/ml/clustering/KMeans.scala | 2 +- .../scala/org/apache/spark/ml/clustering/LDA.scala | 4 ++-- .../apache/spark/ml/feature/ElementwiseProduct.scala | 6 +++--- .../org/apache/spark/ml/feature/Normalizer.scala | 12 ++++++------ .../spark/ml/feature/PolynomialExpansion.scala | 12 ++++++------ .../scala/org/apache/spark/ml/feature/Word2Vec.scala | 2 +- .../spark/ml/regression/AFTSurvivalRegression.scala | 6 +++--- .../spark/ml/regression/IsotonicRegression.scala | 4 ++-- .../spark/ml/regression/LinearRegression.scala | 6 +++--- python/pyspark/ml/classification.py | 8 ++++---- python/pyspark/ml/regression.py | 8 ++++++-- 14 files changed, 41 insertions(+), 37 deletions(-) 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 2fa8fbcc76a25..be69d46eebef2 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 @@ -482,7 +482,7 @@ object LogisticRegression extends DefaultParamsReadable[LogisticRegression] { @Experimental class LogisticRegressionModel private[spark] ( @Since("1.4.0") override val uid: String, - @Since("1.6.0") val coefficients: Vector, + @Since("2.0.0") val coefficients: Vector, @Since("1.3.0") val intercept: Double) extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with LogisticRegressionParams with MLWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 700542117ee77..76ef32aa3dc1d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -296,7 +296,7 @@ object MultilayerPerceptronClassifier class MultilayerPerceptronClassificationModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("1.5.0") val layers: Array[Int], - @Since("1.5.0") val weights: Vector) + @Since("2.0.0") val weights: Vector) extends PredictionModel[Vector, MultilayerPerceptronClassificationModel] with Serializable with MLWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index a9d493032b28a..7c340312df3e1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -130,8 +130,8 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { @Experimental class NaiveBayesModel private[ml] ( @Since("1.5.0") override val uid: String, - @Since("1.5.0") val pi: Vector, - @Since("1.5.0") val theta: Matrix) + @Since("2.0.0") val pi: Vector, + @Since("2.0.0") val theta: Matrix) extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams with MLWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 6f63d0481896e..9fb7d6a9a21ae 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -131,7 +131,7 @@ class KMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) - @Since("1.5.0") + @Since("2.0.0") def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 609e50eb494ce..b333d59258238 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -432,7 +432,7 @@ sealed abstract class LDAModel private[ml] ( * If Online LDA was used and [[optimizeDocConcentration]] was set to false, * then this returns the fixed (given) value for the [[docConcentration]] parameter. */ - @Since("1.6.0") + @Since("2.0.0") def estimatedDocConcentration: Vector = getModel.docConcentration /** @@ -444,7 +444,7 @@ sealed abstract class LDAModel private[ml] ( * the Expectation-Maximization ("em") [[optimizer]], then this method could involve * collecting a large amount of data to the driver (on the order of vocabSize x k). */ - @Since("1.6.0") + @Since("2.0.0") def topicsMatrix: Matrix = oldLocalModel.topicsMatrix.asML /** Indicates whether this instance is of type [[DistributedLDAModel]] */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 92fefb1e6c0f3..d07833e5805df 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -33,11 +33,11 @@ import org.apache.spark.sql.types.DataType * multiplier. */ @Experimental -@Since("2.0.0") -class ElementwiseProduct @Since("2.0.0") (@Since("2.0.0") override val uid: String) +@Since("1.4.0") +class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[Vector, Vector, ElementwiseProduct] with DefaultParamsWritable { - @Since("2.0.0") + @Since("1.4.0") def this() = this(Identifiable.randomUID("elemProd")) /** 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 9a4e682890f12..f9cbad90c9f3f 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 @@ -31,11 +31,11 @@ import org.apache.spark.sql.types.DataType * Normalize a vector to have unit norm using the given p-norm. */ @Experimental -@Since("2.0.0") -class Normalizer @Since("2.0.0") (@Since("2.0.0") override val uid: String) +@Since("1.4.0") +class Normalizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[Vector, Vector, Normalizer] with DefaultParamsWritable { - @Since("2.0.0") + @Since("1.4.0") def this() = this(Identifiable.randomUID("normalizer")) /** @@ -43,17 +43,17 @@ class Normalizer @Since("2.0.0") (@Since("2.0.0") override val uid: String) * (default: p = 2) * @group param */ - @Since("2.0.0") + @Since("1.4.0") val p = new DoubleParam(this, "p", "the p norm value", ParamValidators.gtEq(1)) setDefault(p -> 2.0) /** @group getParam */ - @Since("2.0.0") + @Since("1.4.0") def getP: Double = $(p) /** @group setParam */ - @Since("2.0.0") + @Since("1.4.0") def setP(value: Double): this.type = set(p, value) override protected def createTransformFunc: Vector => Vector = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 026014c7d64e3..7b35fdeaf40c6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -35,11 +35,11 @@ import org.apache.spark.sql.types.DataType * `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. */ @Experimental -@Since("2.0.0") -class PolynomialExpansion @Since("2.0.0") (@Since("2.0.0") override val uid: String) +@Since("1.4.0") +class PolynomialExpansion @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[Vector, Vector, PolynomialExpansion] with DefaultParamsWritable { - @Since("2.0.0") + @Since("1.4.0") def this() = this(Identifiable.randomUID("poly")) /** @@ -47,18 +47,18 @@ class PolynomialExpansion @Since("2.0.0") (@Since("2.0.0") override val uid: Str * Default: 2 * @group param */ - @Since("2.0.0") + @Since("1.4.0") val degree = new IntParam(this, "degree", "the polynomial degree to expand (>= 1)", ParamValidators.gtEq(1)) setDefault(degree -> 2) /** @group getParam */ - @Since("2.0.0") + @Since("1.4.0") def getDegree: Int = $(degree) /** @group setParam */ - @Since("2.0.0") + @Since("1.4.0") def setDegree(value: Int): this.type = set(degree, value) override protected def createTransformFunc: Vector => Vector = { v => diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index a74d31ff9d611..0cac3fa2d7e57 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -240,7 +240,7 @@ class Word2VecModel private[ml] ( * of the word. Returns a dataframe with the words and the cosine similarities between the * synonyms and the given word vector. */ - @Since("1.5.0") + @Since("2.0.0") def findSynonyms(word: Vector, num: Int): DataFrame = { val spark = SparkSession.builder().getOrCreate() spark.createDataFrame(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index fe65e3e810f0a..2dbac49ccf25b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -286,7 +286,7 @@ object AFTSurvivalRegression extends DefaultParamsReadable[AFTSurvivalRegression @Since("1.6.0") class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") override val uid: String, - @Since("1.6.0") val coefficients: Vector, + @Since("2.0.0") val coefficients: Vector, @Since("1.6.0") val intercept: Double, @Since("1.6.0") val scale: Double) extends Model[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams with MLWritable { @@ -307,7 +307,7 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") def setQuantilesCol(value: String): this.type = set(quantilesCol, value) - @Since("1.6.0") + @Since("2.0.0") def predictQuantiles(features: Vector): Vector = { // scale parameter for the Weibull distribution of lifetime val lambda = math.exp(BLAS.dot(coefficients, features) + intercept) @@ -319,7 +319,7 @@ class AFTSurvivalRegressionModel private[ml] ( Vectors.dense(quantiles) } - @Since("1.6.0") + @Since("2.0.0") def predict(features: Vector): Double = { math.exp(BLAS.dot(coefficients, features) + intercept) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index f05b47eda7b63..9b9429a328d08 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -221,14 +221,14 @@ class IsotonicRegressionModel private[ml] ( def setFeatureIndex(value: Int): this.type = set(featureIndex, value) /** Boundaries in increasing order for which predictions are known. */ - @Since("1.5.0") + @Since("2.0.0") def boundaries: Vector = Vectors.dense(oldModel.boundaries) /** * Predictions associated with the boundaries at the same index, monotone because of isotonic * regression. */ - @Since("1.5.0") + @Since("2.0.0") def predictions: Vector = Vectors.dense(oldModel.predictions) @Since("1.5.0") 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 5e8ef1b375001..2723f74724927 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 @@ -387,9 +387,9 @@ object LinearRegression extends DefaultParamsReadable[LinearRegression] { @Since("1.3.0") @Experimental class LinearRegressionModel private[ml] ( - override val uid: String, - val coefficients: Vector, - val intercept: Double) + @Since("1.4.0") override val uid: String, + @Since("2.0.0") val coefficients: Vector, + @Since("1.3.0") val intercept: Double) extends RegressionModel[Vector, LinearRegressionModel] with LinearRegressionParams with MLWritable { diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d6d713ca53035..c035942f73863 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -224,7 +224,7 @@ class LogisticRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ @property - @since("1.6.0") + @since("2.0.0") def coefficients(self): """ Model coefficients. @@ -1051,7 +1051,7 @@ class NaiveBayesModel(JavaModel, JavaMLWritable, JavaMLReadable): """ @property - @since("1.5.0") + @since("2.0.0") def pi(self): """ log of class priors. @@ -1059,7 +1059,7 @@ def pi(self): return self._call_java("pi") @property - @since("1.5.0") + @since("2.0.0") def theta(self): """ log of class conditional probabilities. @@ -1260,7 +1260,7 @@ def layers(self): return self._call_java("javaLayers") @property - @since("1.6.0") + @since("2.0.0") def weights(self): """ vector of initial weights for the model that consists of the weights of layers. diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 29efd6a852e84..8de9ad85311fa 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -138,7 +138,7 @@ class LinearRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ @property - @since("1.6.0") + @since("2.0.0") def coefficients(self): """ Model coefficients. @@ -511,6 +511,7 @@ class IsotonicRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ @property + @since("2.0.0") def boundaries(self): """ Boundaries in increasing order for which predictions are known. @@ -518,6 +519,7 @@ def boundaries(self): return self._call_java("boundaries") @property + @since("2.0.0") def predictions(self): """ Predictions associated with the boundaries at the same index, monotone because of isotonic @@ -1248,7 +1250,7 @@ class AFTSurvivalRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ @property - @since("1.6.0") + @since("2.0.0") def coefficients(self): """ Model coefficients. @@ -1271,12 +1273,14 @@ def scale(self): """ return self._call_java("scale") + @since("2.0.0") def predictQuantiles(self, features): """ Predicted Quantiles """ return self._call_java("predictQuantiles", features) + @since("2.0.0") def predict(self, features): """ Predicted value From 76d0ef34e4a5b91b883141f839adc493205fa429 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 22 Jun 2016 10:06:43 -0700 Subject: [PATCH 0775/1470] [MINOR][MLLIB] DefaultParamsReadable/Writable should be DeveloperApi ## What changes were proposed in this pull request? `DefaultParamsReadable/Writable` are not user-facing. Only developers who implement `Transformer/Estimator` would use it. So this PR changes the annotation to `DeveloperApi`. Author: Xiangrui Meng Closes #13828 from mengxr/default-readable-should-be-developer-api. (cherry picked from commit 6a6010f0015542dc2753b2cb12fdd1204db63ea6) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/ml/util/ReadWrite.scala | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index 1582a73ea047b..4413fefdea3ca 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -26,7 +26,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.classification.{OneVsRest, OneVsRestModel} @@ -161,7 +161,7 @@ trait MLWritable { } /** - * :: Experimental :: + * :: DeveloperApi :: * * Helper trait for making simple [[Params]] types writable. If a [[Params]] class stores * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide @@ -171,8 +171,7 @@ trait MLWritable { * * @see [[DefaultParamsReadable]], the counterpart to this trait */ -@Experimental -@Since("2.0.0") +@DeveloperApi trait DefaultParamsWritable extends MLWritable { self: Params => override def write: MLWriter = new DefaultParamsWriter(this) @@ -230,7 +229,7 @@ trait MLReadable[T] { /** - * :: Experimental :: + * :: DeveloperApi :: * * Helper trait for making simple [[Params]] types readable. If a [[Params]] class stores * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide @@ -239,11 +238,9 @@ trait MLReadable[T] { * [[org.apache.spark.sql.Dataset]]. * * @tparam T ML instance type - * * @see [[DefaultParamsWritable]], the counterpart to this trait */ -@Experimental -@Since("2.0.0") +@DeveloperApi trait DefaultParamsReadable[T] extends MLReadable[T] { override def read: MLReader[T] = new DefaultParamsReader[T] From 520828c90d25acf733ffa70fe269dcfe93b56a31 Mon Sep 17 00:00:00 2001 From: Ahmed Mahran Date: Wed, 22 Jun 2016 10:39:24 -0700 Subject: [PATCH 0776/1470] [SPARK-16120][STREAMING] getCurrentLogFiles in ReceiverSuite WAL generating and cleaning case uses external variable instead of the passed parameter ## What changes were proposed in this pull request? In `ReceiverSuite.scala`, in the test case "write ahead log - generating and cleaning", the inner method `getCurrentLogFiles` uses external variable `logDirectory1` instead of the passed parameter `logDirectory`. This PR fixes this by using the passed method argument instead of variable from the outer scope. ## How was this patch tested? The unit test was re-run and the output logs were checked for the correct paths used. tdas Author: Ahmed Mahran Closes #13825 from ahmed-mahran/b-receiver-suite-wal-gen-cln. (cherry picked from commit c2cebdb7ddff3d041d548fe1cd8de4efb31b294f) Signed-off-by: Shixiong Zhu --- .../test/scala/org/apache/spark/streaming/ReceiverSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 917232c9cdd63..1b1e21f6e5bab 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -215,7 +215,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { def getCurrentLogFiles(logDirectory: File): Seq[String] = { try { if (logDirectory.exists()) { - logDirectory1.listFiles().filter { _.getName.startsWith("log") }.map { _.toString } + logDirectory.listFiles().filter { _.getName.startsWith("log") }.map { _.toString } } else { Seq.empty } From e043c02d039809be149622a4d7562f332cfa25aa Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 22 Jun 2016 12:12:34 -0700 Subject: [PATCH 0777/1470] [SPARK-16003] SerializationDebugger runs into infinite loop ## What changes were proposed in this pull request? This fixes SerializationDebugger to not recurse forever when `writeReplace` returns an object of the same class, which is the case for at least the `SQLMetrics` class. See also the OpenJDK unit tests on the behavior of recursive `writeReplace()`: https://github.com/openjdk-mirror/jdk7u-jdk/blob/f4d80957e89a19a29bb9f9807d2a28351ed7f7df/test/java/io/Serializable/nestedReplace/NestedReplace.java cc davies cloud-fan ## How was this patch tested? Unit tests for SerializationDebugger. Author: Eric Liang Closes #13814 from ericl/spark-16003. (cherry picked from commit 6f915c9ec24003877d1ef675a59145699780a2ff) Signed-off-by: Davies Liu --- .../spark/serializer/SerializationDebugger.scala | 9 ++++----- .../serializer/SerializationDebuggerSuite.scala | 13 ++++++++++++- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala index c04b483831704..5e7a98c8aa89c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala @@ -155,7 +155,7 @@ private[spark] object SerializationDebugger extends Logging { // If the object has been replaced using writeReplace(), // then call visit() on it again to test its type again. - if (!finalObj.eq(o)) { + if (finalObj.getClass != o.getClass) { return visit(finalObj, s"writeReplace data (class: ${finalObj.getClass.getName})" :: stack) } @@ -265,11 +265,10 @@ private[spark] object SerializationDebugger extends Logging { if (!desc.hasWriteReplaceMethod) { (o, desc) } else { - // write place val replaced = desc.invokeWriteReplace(o) - // `writeReplace` may return the same object. - if (replaced eq o) { - (o, desc) + // `writeReplace` recursion stops when the returned object has the same class. + if (replaced.getClass == o.getClass) { + (replaced, desc) } else { findObjectAndDescriptor(replaced) } diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala index f019b1e25900b..912a516dff0f4 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala @@ -126,7 +126,11 @@ class SerializationDebuggerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(find(new SerializableClassWithWriteReplace(new SerializableClass1)).isEmpty) } - test("object containing writeObject() and not serializable field") { + test("no infinite loop with writeReplace() which returns class of its own type") { + assert(find(new SerializableClassWithRecursiveWriteReplace).isEmpty) + } + + test("object containing writeObject() and not serializable field") { val s = find(new SerializableClassWithWriteObject(new NotSerializable)) assert(s.size === 3) assert(s(0).contains("NotSerializable")) @@ -229,6 +233,13 @@ class SerializableClassWithWriteReplace(@(transient @param) replacementFieldObje } +class SerializableClassWithRecursiveWriteReplace extends Serializable { + private def writeReplace(): Object = { + new SerializableClassWithRecursiveWriteReplace + } +} + + class ExternalizableClass(objectField: Object) extends java.io.Externalizable { val serializableObjectField = new SerializableClass1 From 299f427b70f8dedbc0b554f83c4fde408caf4d15 Mon Sep 17 00:00:00 2001 From: Kai Jiang Date: Wed, 22 Jun 2016 12:50:36 -0700 Subject: [PATCH 0778/1470] [SPARK-15672][R][DOC] R programming guide update ## What changes were proposed in this pull request? Guide for - UDFs with dapply, dapplyCollect - spark.lapply for running parallel R functions ## How was this patch tested? build locally screen shot 2016-06-14 at 03 12 56 Author: Kai Jiang Closes #13660 from vectorijk/spark-15672-R-guide-update. (cherry picked from commit 43b04b7ecb313a2cee6121dd575de1f7dc785c11) Signed-off-by: Joseph K. Bradley --- R/pkg/R/context.R | 2 +- docs/sparkr.md | 77 +++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 96ef9438ad5dc..dd0ceaeb08a5d 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -246,7 +246,7 @@ setCheckpointDir <- function(sc, dirName) { #' \preformatted{ #' train <- function(hyperparam) { #' library(MASS) -#' lm.ridge(“y ~ x+z”, data, lambda=hyperparam) +#' lm.ridge("y ~ x+z", data, lambda=hyperparam) #' model #' } #' } diff --git a/docs/sparkr.md b/docs/sparkr.md index f0189012f36ed..9e74e4a96acdc 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -255,6 +255,83 @@ head(df) {% endhighlight %}
    +### Applying User-Defined Function +In SparkR, we support several kinds of User-Defined Functions: + +#### Run a given function on a large dataset using `dapply` or `dapplyCollect` + +##### dapply +Apply a function to each partition of a `SparkDataFrame`. The function to be applied to each partition of the `SparkDataFrame` +and should have only one parameter, to which a `data.frame` corresponds to each partition will be passed. The output of function +should be a `data.frame`. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match the R function's output. +
    +{% highlight r %} + +# Convert waiting time from hours to seconds. +# Note that we can apply UDF to DataFrame. +schema <- structType(structField("eruptions", "double"), structField("waiting", "double"), + structField("waiting_secs", "double")) +df1 <- dapply(df, function(x) {x <- cbind(x, x$waiting * 60)}, schema) +head(collect(df1)) +## eruptions waiting waiting_secs +##1 3.600 79 4740 +##2 1.800 54 3240 +##3 3.333 74 4440 +##4 2.283 62 3720 +##5 4.533 85 5100 +##6 2.883 55 3300 +{% endhighlight %} +
    + +##### dapplyCollect +Like `dapply`, apply a function to each partition of a `SparkDataFrame` and collect the result back. The output of function +should be a `data.frame`. But, Schema is not required to be passed. Note that `dapplyCollect` only can be used if the +output of UDF run on all the partitions can fit in driver memory. +
    +{% highlight r %} + +# Convert waiting time from hours to seconds. +# Note that we can apply UDF to DataFrame and return a R's data.frame +ldf <- dapplyCollect( + df, + function(x) { + x <- cbind(x, "waiting_secs" = x$waiting * 60) + }) +head(ldf, 3) +## eruptions waiting waiting_secs +##1 3.600 79 4740 +##2 1.800 54 3240 +##3 3.333 74 4440 + +{% endhighlight %} +
    + +#### Run local R functions distributed using `spark.lapply` + +##### spark.lapply +Similar to `lapply` in native R, `spark.lapply` runs a function over a list of elements and distributes the computations with Spark. +Applies a function in a manner that is similar to `doParallel` or `lapply` to elements of a list. The results of all the computations +should fit in a single machine. If that is not the case they can do something like `df <- createDataFrame(list)` and then use +`dapply` +
    +{% highlight r %} + +# Perform distributed training of multiple models with spark.lapply. Here, we pass +# a read-only list of arguments which specifies family the generalized linear model should be. +families <- c("gaussian", "poisson") +train <- function(family) { + model <- glm(Sepal.Length ~ Sepal.Width + Species, iris, family = family) + summary(model) +} +# Return a list of model's summaries +model.summaries <- spark.lapply(families, train) + +# Print the summary of each model +print(model.summaries) + +{% endhighlight %} +
    + ## Running SQL Queries from SparkR A SparkDataFrame can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. From 282a3cd02389464d6adbf02921281c963da29b00 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 22 Jun 2016 13:40:24 -0700 Subject: [PATCH 0779/1470] [SPARK-16078][SQL] from_utc_timestamp/to_utc_timestamp should not depends on local timezone ## What changes were proposed in this pull request? Currently, we use local timezone to parse or format a timestamp (TimestampType), then use Long as the microseconds since epoch UTC. In from_utc_timestamp() and to_utc_timestamp(), we did not consider the local timezone, they could return different results with different local timezone. This PR will do the conversion based on human time (in local timezone), it should return same result in whatever timezone. But because the mapping from absolute timestamp to human time is not exactly one-to-one mapping, it will still return wrong result in some timezone (also in the begging or ending of DST). This PR is kind of the best effort fix. In long term, we should make the TimestampType be timezone aware to fix this totally. ## How was this patch tested? Tested these function in all timezone. Author: Davies Liu Closes #13784 from davies/convert_tz. (cherry picked from commit 20d411bc5d05dd099f6d5234a24e10a519a39bdf) Signed-off-by: Herman van Hovell --- .../expressions/datetimeExpressions.scala | 10 +-- .../sql/catalyst/util/DateTimeUtils.scala | 34 ++++++++-- .../catalyst/util/DateTimeUtilsSuite.scala | 65 +++++++++++-------- 3 files changed, 73 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 773431dd3346b..04c17bdaf2989 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -730,16 +730,17 @@ case class FromUTCTimestamp(left: Expression, right: Expression) """.stripMargin) } else { val tzTerm = ctx.freshName("tz") + val utcTerm = ctx.freshName("utc") val tzClass = classOf[TimeZone].getName ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""") + ctx.addMutableState(tzClass, utcTerm, s"""$utcTerm = $tzClass.getTimeZone("UTC");""") val eval = left.genCode(ctx) ev.copy(code = s""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; |long ${ev.value} = 0; |if (!${ev.isNull}) { - | ${ev.value} = ${eval.value} + - | ${tzTerm}.getOffset(${eval.value} / 1000) * 1000L; + | ${ev.value} = $dtu.convertTz(${eval.value}, $utcTerm, $tzTerm); |} """.stripMargin) } @@ -869,16 +870,17 @@ case class ToUTCTimestamp(left: Expression, right: Expression) """.stripMargin) } else { val tzTerm = ctx.freshName("tz") + val utcTerm = ctx.freshName("utc") val tzClass = classOf[TimeZone].getName ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""") + ctx.addMutableState(tzClass, utcTerm, s"""$utcTerm = $tzClass.getTimeZone("UTC");""") val eval = left.genCode(ctx) ev.copy(code = s""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; |long ${ev.value} = 0; |if (!${ev.isNull}) { - | ${ev.value} = ${eval.value} - - | ${tzTerm}.getOffset(${eval.value} / 1000) * 1000L; + | ${ev.value} = $dtu.convertTz(${eval.value}, $tzTerm, $utcTerm); |} """.stripMargin) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 56bf9a7863e61..df480a1d65bc9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -885,14 +885,38 @@ object DateTimeUtils { guess } + /** + * Convert the timestamp `ts` from one timezone to another. + * + * TODO: Because of DST, the conversion between UTC and human time is not exactly one-to-one + * mapping, the conversion here may return wrong result, we should make the timestamp + * timezone-aware. + */ + def convertTz(ts: SQLTimestamp, fromZone: TimeZone, toZone: TimeZone): SQLTimestamp = { + // We always use local timezone to parse or format a timestamp + val localZone = threadLocalLocalTimeZone.get() + val utcTs = if (fromZone.getID == localZone.getID) { + ts + } else { + // get the human time using local time zone, that actually is in fromZone. + val localTs = ts + localZone.getOffset(ts / 1000L) * 1000L // in fromZone + localTs - getOffsetFromLocalMillis(localTs / 1000L, fromZone) * 1000L + } + if (toZone.getID == localZone.getID) { + utcTs + } else { + val localTs2 = utcTs + toZone.getOffset(utcTs / 1000L) * 1000L // in toZone + // treat it as local timezone, convert to UTC (we could get the expected human time back) + localTs2 - getOffsetFromLocalMillis(localTs2 / 1000L, localZone) * 1000L + } + } + /** * Returns a timestamp of given timezone from utc timestamp, with the same string * representation in their timezone. */ def fromUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { - val tz = TimeZone.getTimeZone(timeZone) - val offset = tz.getOffset(time / 1000L) - time + offset * 1000L + convertTz(time, TimeZoneGMT, TimeZone.getTimeZone(timeZone)) } /** @@ -900,9 +924,7 @@ object DateTimeUtils { * string representation in their timezone. */ def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { - val tz = TimeZone.getTimeZone(timeZone) - val offset = getOffsetFromLocalMillis(time / 1000L, tz) - time - offset * 1000L + convertTz(time, TimeZone.getTimeZone(timeZone), TimeZoneGMT) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index f9cb97629fcf3..059a5b7d07cde 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -488,17 +488,23 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(toJavaTimestamp(fromUTCTime(fromJavaTimestamp(Timestamp.valueOf(utc)), tz)).toString === expected) } - test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456") - test("2011-12-25 09:00:00.123456", "JST", "2011-12-25 18:00:00.123456") - test("2011-12-25 09:00:00.123456", "PST", "2011-12-25 01:00:00.123456") - test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456") - - // Daylight Saving Time - test("2016-03-13 09:59:59.0", "PST", "2016-03-13 01:59:59.0") - test("2016-03-13 10:00:00.0", "PST", "2016-03-13 03:00:00.0") - test("2016-11-06 08:59:59.0", "PST", "2016-11-06 01:59:59.0") - test("2016-11-06 09:00:00.0", "PST", "2016-11-06 01:00:00.0") - test("2016-11-06 10:00:00.0", "PST", "2016-11-06 02:00:00.0") + for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { + DateTimeTestUtils.withDefaultTimeZone(tz) { + test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456") + test("2011-12-25 09:00:00.123456", "JST", "2011-12-25 18:00:00.123456") + test("2011-12-25 09:00:00.123456", "PST", "2011-12-25 01:00:00.123456") + test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456") + } + } + + DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("PST")) { + // Daylight Saving Time + test("2016-03-13 09:59:59.0", "PST", "2016-03-13 01:59:59.0") + test("2016-03-13 10:00:00.0", "PST", "2016-03-13 03:00:00.0") + test("2016-11-06 08:59:59.0", "PST", "2016-11-06 01:59:59.0") + test("2016-11-06 09:00:00.0", "PST", "2016-11-06 01:00:00.0") + test("2016-11-06 10:00:00.0", "PST", "2016-11-06 02:00:00.0") + } } test("to UTC timestamp") { @@ -506,21 +512,28 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(toJavaTimestamp(toUTCTime(fromJavaTimestamp(Timestamp.valueOf(utc)), tz)).toString === expected) } - test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456") - test("2011-12-25 18:00:00.123456", "JST", "2011-12-25 09:00:00.123456") - test("2011-12-25 01:00:00.123456", "PST", "2011-12-25 09:00:00.123456") - test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456") - - // Daylight Saving Time - test("2016-03-13 01:59:59", "PST", "2016-03-13 09:59:59.0") - // 2016-03-13 02:00:00 PST does not exists - test("2016-03-13 02:00:00", "PST", "2016-03-13 10:00:00.0") - test("2016-03-13 03:00:00", "PST", "2016-03-13 10:00:00.0") - test("2016-11-06 00:59:59", "PST", "2016-11-06 07:59:59.0") - // 2016-11-06 01:00:00 PST could be 2016-11-06 08:00:00 UTC or 2016-11-06 09:00:00 UTC - test("2016-11-06 01:00:00", "PST", "2016-11-06 09:00:00.0") - test("2016-11-06 01:59:59", "PST", "2016-11-06 09:59:59.0") - test("2016-11-06 02:00:00", "PST", "2016-11-06 10:00:00.0") + + for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { + DateTimeTestUtils.withDefaultTimeZone(tz) { + test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456") + test("2011-12-25 18:00:00.123456", "JST", "2011-12-25 09:00:00.123456") + test("2011-12-25 01:00:00.123456", "PST", "2011-12-25 09:00:00.123456") + test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456") + } + } + + DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("PST")) { + // Daylight Saving Time + test("2016-03-13 01:59:59", "PST", "2016-03-13 09:59:59.0") + // 2016-03-13 02:00:00 PST does not exists + test("2016-03-13 02:00:00", "PST", "2016-03-13 10:00:00.0") + test("2016-03-13 03:00:00", "PST", "2016-03-13 10:00:00.0") + test("2016-11-06 00:59:59", "PST", "2016-11-06 07:59:59.0") + // 2016-11-06 01:00:00 PST could be 2016-11-06 08:00:00 UTC or 2016-11-06 09:00:00 UTC + test("2016-11-06 01:00:00", "PST", "2016-11-06 09:00:00.0") + test("2016-11-06 01:59:59", "PST", "2016-11-06 09:59:59.0") + test("2016-11-06 02:00:00", "PST", "2016-11-06 10:00:00.0") + } } test("daysToMillis and millisToDays") { From 02435acf3bf84f77bb3c70a2fd548af8bad4c28e Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 22 Jun 2016 15:50:21 -0700 Subject: [PATCH 0780/1470] [SPARK-16153][MLLIB] switch to multi-line doc to avoid a genjavadoc bug ## What changes were proposed in this pull request? We recently deprecated setLabelCol in ChiSqSelectorModel (#13823): ~~~scala /** group setParam */ Since("1.6.0") deprecated("labelCol is not used by ChiSqSelectorModel.", "2.0.0") def setLabelCol(value: String): this.type = set(labelCol, value) ~~~ This unfortunately hit a genjavadoc bug and broken doc generation. This is the generated Java code: ~~~java /** group setParam */ public org.apache.spark.ml.feature.ChiSqSelectorModel setOutputCol (java.lang.String value) { throw new RuntimeException(); } * * deprecated labelCol is not used by ChiSqSelectorModel. Since 2.0.0. */ public org.apache.spark.ml.feature.ChiSqSelectorModel setLabelCol (java.lang.String value) { throw new RuntimeException(); } ~~~ Switching to multiline is a workaround. Author: Xiangrui Meng Closes #13855 from mengxr/SPARK-16153. (cherry picked from commit 00cc5cca4522297b63b1522a2b8643b1a098e2b3) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/ml/feature/ChiSqSelector.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 38b4db99bc298..712634dffbf17 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -140,7 +140,9 @@ final class ChiSqSelectorModel private[ml] ( @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) - /** @group setParam */ + /** + * @group setParam + */ @Since("1.6.0") @deprecated("labelCol is not used by ChiSqSelectorModel.", "2.0.0") def setLabelCol(value: String): this.type = set(labelCol, value) From 1d3c56e778b28ad4587d07765896814bfc1201f4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 22 Jun 2016 15:52:37 -0700 Subject: [PATCH 0781/1470] [SPARK-16155][DOC] remove package grouping in Java docs ## What changes were proposed in this pull request? In 1.4 and earlier releases, we have package grouping in the generated Java API docs. See http://spark.apache.org/docs/1.4.0/api/java/index.html. However, this disappeared in 1.5.0: http://spark.apache.org/docs/1.5.0/api/java/index.html. Rather than fixing it, I'd suggest removing grouping. Because it might take some time to fix and it is a manual process to update the grouping in `SparkBuild.scala`. I didn't find anyone complaining about missing groups since 1.5.0 on Google. Manually checked the generated Java API docs and confirmed that they are the same as in master. Author: Xiangrui Meng Closes #13856 from mengxr/SPARK-16155. (cherry picked from commit 857ecff1d8268b28bb287e47cda370c87afe9d41) Signed-off-by: Xiangrui Meng --- project/SparkBuild.scala | 20 -------------------- 1 file changed, 20 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bce7f1d69beb7..4b44469576126 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -684,11 +684,6 @@ object Unidoc { import sbtunidoc.Plugin._ import UnidocKeys._ - // for easier specification of JavaDoc package groups - private def packageList(names: String*): String = { - names.map(s => "org.apache.spark." + s).mkString(":") - } - private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = { packages .map(_.filterNot(_.getName.contains("$"))) @@ -731,21 +726,6 @@ object Unidoc { javacOptions in doc := Seq( "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", "-public", - "-group", "Core Java API", packageList("api.java", "api.java.function"), - "-group", "Spark Streaming", packageList( - "streaming.api.java", "streaming.flume", "streaming.kafka", "streaming.kinesis" - ), - "-group", "MLlib", packageList( - "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", - "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", - "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", - "mllib.tree.impurity", "mllib.tree.model", "mllib.util", - "mllib.evaluation", "mllib.feature", "mllib.random", "mllib.stat.correlation", - "mllib.stat.test", "mllib.tree.impl", "mllib.tree.loss", - "ml", "ml.attribute", "ml.classification", "ml.clustering", "ml.evaluation", "ml.feature", - "ml.param", "ml.recommendation", "ml.regression", "ml.tuning" - ), - "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"), "-noqualifier", "java.lang" ), From e2eb8e002acb19fd266d2237baec31f74aa02ef8 Mon Sep 17 00:00:00 2001 From: Prajwal Tuladhar Date: Wed, 22 Jun 2016 16:30:10 -0700 Subject: [PATCH 0782/1470] [SPARK-16131] initialize internal logger lazily in Scala preferred way ## What changes were proposed in this pull request? Initialize logger instance lazily in Scala preferred way ## How was this patch tested? By running `./build/mvn clean test` locally Author: Prajwal Tuladhar Closes #13842 from infynyxx/spark_internal_logger. (cherry picked from commit 044971eca0ff3c2ce62afa665dbd3072d52cbbec) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/internal/Logging.scala | 14 ++++---------- .../cluster/CoarseGrainedSchedulerBackend.scala | 2 -- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index 66a0cfec6296d..c51050c13d3a1 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -32,7 +32,10 @@ private[spark] trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine - @transient private var log_ : Logger = null + @transient lazy val log: Logger = { + initializeLogIfNecessary(false) + LoggerFactory.getLogger(logName) + } // Method to get the logger name for this object protected def logName = { @@ -40,15 +43,6 @@ private[spark] trait Logging { this.getClass.getName.stripSuffix("$") } - // Method to get or create the logger for this object - protected def log: Logger = { - if (log_ == null) { - initializeLogIfNecessary(false) - log_ = LoggerFactory.getLogger(logName) - } - log_ - } - // Log methods that take only a String protected def logInfo(msg: => String) { if (log.isInfoEnabled) log.info(msg) 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 967c4d5325d63..8259923ce31c3 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 @@ -100,8 +100,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // instance across threads private val ser = SparkEnv.get.closureSerializer.newInstance() - override protected def log = CoarseGrainedSchedulerBackend.this.log - protected val addressToExecutorId = new HashMap[RpcAddress, String] private val reviveThread = From e0a43235d9d59736ceb0d703c653ef1350e143ec Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 23 Jun 2016 09:12:20 +0800 Subject: [PATCH 0783/1470] [SPARK-16024][SQL][TEST] Verify Column Comment for Data Source Tables #### What changes were proposed in this pull request? This PR is to improve test coverage. It verifies whether `Comment` of `Column` can be appropriate handled. The test cases verify the related parts in Parser, both SQL and DataFrameWriter interface, and both Hive Metastore catalog and In-memory catalog. #### How was this patch tested? N/A Author: gatorsmile Closes #13764 from gatorsmile/dataSourceComment. (cherry picked from commit 9f990fa3f9e0b798d8018cf4132b93a3468f33bb) Signed-off-by: Wenchen Fan --- .../sql/execution/command/DDLCommandSuite.scala | 10 +++++++--- .../spark/sql/execution/command/DDLSuite.scala | 13 +++++++++++++ .../spark/sql/hive/execution/HiveDDLSuite.scala | 14 ++++++++++++++ 3 files changed, 34 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 5bee28b4462c6..7b96f4c99ab5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsing} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.sql.types.{IntegerType, MetadataBuilder, StringType, StructType} // TODO: merge this with DDLSuite (SPARK-14441) @@ -349,10 +349,14 @@ class DDLCommandSuite extends PlanTest { } test("create table using - with partitioned by") { - val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (a)" + val query = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + + "USING parquet PARTITIONED BY (a)" val expected = CreateTableUsing( TableIdentifier("my_tab"), - Some(new StructType().add("a", IntegerType).add("b", StringType)), + Some(new StructType() + .add("a", IntegerType, nullable = true, + new MetadataBuilder().putString("comment", s"test").build()) + .add("b", StringType)), "parquet", false, Map.empty, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index f40ddcc95affd..47d8a28f49927 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -252,6 +252,19 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("desc table for parquet data source table using in-memory catalog") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + val tabName = "tab1" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(a int comment 'test') USING parquet ") + + checkAnswer( + sql(s"DESC $tabName").select("col_name", "data_type", "comment"), + Row("a", "int", "test") + ) + } + } + test("Alter/Describe Database") { withTempDir { tmpDir => val path = tmpDir.toString diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index b2f01fcc83280..89f69c8e4d7f1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach +import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTableType} import org.apache.spark.sql.catalyst.TableIdentifier @@ -407,6 +408,19 @@ class HiveDDLSuite } } + test("desc table for data source table using Hive Metastore") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive") + val tabName = "tab1" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(a int comment 'test') USING parquet ") + + checkAnswer( + sql(s"DESC $tabName").select("col_name", "data_type", "comment"), + Row("a", "int", "test") + ) + } + } + private def createDatabaseWithLocation(tmpDir: File, dirExists: Boolean): Unit = { val catalog = spark.sessionState.catalog val dbName = "db1" From 5b4a9a4c37822cd7528c6bb933da3454fd3bcd37 Mon Sep 17 00:00:00 2001 From: bomeng Date: Thu, 23 Jun 2016 11:06:19 +0800 Subject: [PATCH 0784/1470] [SPARK-15230][SQL] distinct() does not handle column name with dot properly ## What changes were proposed in this pull request? When table is created with column name containing dot, distinct() will fail to run. For example, ```scala val rowRDD = sparkContext.parallelize(Seq(Row(1), Row(1), Row(2))) val schema = StructType(Array(StructField("column.with.dot", IntegerType, nullable = false))) val df = spark.createDataFrame(rowRDD, schema) ``` running the following will have no problem: ```scala df.select(new Column("`column.with.dot`")) ``` but running the query with additional distinct() will cause exception: ```scala df.select(new Column("`column.with.dot`")).distinct() ``` The issue is that distinct() will try to resolve the column name, but the column name in the schema does not have backtick with it. So the solution is to add the backtick before passing the column name to resolve(). ## How was this patch tested? Added a new test case. Author: bomeng Closes #13140 from bomeng/SPARK-15230. (cherry picked from commit 925884a612dd88beaddf555c74d90856ab040ec7) Signed-off-by: Wenchen Fan --- .../src/main/scala/org/apache/spark/sql/Dataset.scala | 8 +++++++- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 5 +++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 02cc3985b0100..f1d33c3e5ce92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1812,7 +1812,13 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ def dropDuplicates(colNames: Seq[String]): Dataset[T] = withTypedPlan { - val groupCols = colNames.map(resolve) + val resolver = sparkSession.sessionState.analyzer.resolver + val allColumns = queryExecution.analyzed.output + val groupCols = colNames.map { colName => + allColumns.find(col => resolver(col.name, colName)).getOrElse( + throw new AnalysisException( + s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""")) + } val groupColExprIds = groupCols.map(_.exprId) val aggCols = logicalPlan.output.map { attr => if (groupColExprIds.contains(attr.exprId)) { 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 c8a0f7134d5dd..1afee9f02104e 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 @@ -1536,4 +1536,9 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Utils.deleteRecursively(baseDir) } } + + test("SPARK-15230: distinct() does not handle column name with dot properly") { + val df = Seq(1, 1, 2).toDF("column.with.dot") + checkAnswer(df.distinct(), Row(1) :: Row(2) :: Nil) + } } From 4ad731ed6a963131f05c387c2f9536b56d228090 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 22 Jun 2016 22:28:54 -0700 Subject: [PATCH 0785/1470] [SQL][MINOR] Fix minor formatting issues in SHOW CREATE TABLE output ## What changes were proposed in this pull request? This PR fixes two minor formatting issues appearing in `SHOW CREATE TABLE` output. Before: ``` CREATE EXTERNAL TABLE ... ... WITH SERDEPROPERTIES ('serialization.format' = '1' ) ... TBLPROPERTIES ('avro.schema.url' = '/tmp/avro/test.avsc', 'transient_lastDdlTime' = '1466638180') ``` After: ``` CREATE EXTERNAL TABLE ... ... WITH SERDEPROPERTIES ( 'serialization.format' = '1' ) ... TBLPROPERTIES ( 'avro.schema.url' = '/tmp/avro/test.avsc', 'transient_lastDdlTime' = '1466638180' ) ``` ## How was this patch tested? Manually tested. Author: Cheng Lian Closes #13864 from liancheng/show-create-table-format-fix. (cherry picked from commit f34b5c62b2da3fe0ea989acea46fff949d349afc) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/execution/command/tables.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 3eb93a2922708..30dc7e81e9eeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -830,7 +830,7 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" } - builder ++= serdeProps.mkString("WITH SERDEPROPERTIES (", ",\n ", "\n)\n") + builder ++= serdeProps.mkString("WITH SERDEPROPERTIES (\n ", ",\n ", "\n)\n") } if (storage.inputFormat.isDefined || storage.outputFormat.isDefined) { @@ -864,7 +864,7 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman } if (props.nonEmpty) { - builder ++= props.mkString("TBLPROPERTIES (", ",\n ", ")\n") + builder ++= props.mkString("TBLPROPERTIES (\n ", ",\n ", "\n)\n") } } } From 567093596057eb77d940d53c88b82da128acfd9b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 23 Jun 2016 08:26:17 -0700 Subject: [PATCH 0786/1470] [SPARK-16154][MLLIB] Update spark.ml and spark.mllib package docs ## What changes were proposed in this pull request? Since we decided to switch spark.mllib package into maintenance mode in 2.0, it would be nice to update the package docs to reflect this change. ## How was this patch tested? Manually checked generated APIs. Author: Xiangrui Meng Closes #13859 from mengxr/SPARK-16154. (cherry picked from commit 65d1f0f716f50dd14b5dfe1e7fac772f1b4d2be0) Signed-off-by: Xiangrui Meng --- .../org/apache/spark/ml/package-info.java | 7 ++--- .../scala/org/apache/spark/ml/package.scala | 4 +-- .../org/apache/spark/mllib/JavaPackage.java | 31 +++++++++++++++++++ .../org/apache/spark/mllib/package-info.java | 22 ++++++++++++- .../org/apache/spark/mllib/package.scala | 17 +++++++++- 5 files changed, 72 insertions(+), 9 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/JavaPackage.java diff --git a/mllib/src/main/scala/org/apache/spark/ml/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/package-info.java index 9a40f5dd69136..cb97382207b00 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/ml/package-info.java @@ -16,10 +16,7 @@ */ /** - * Spark ML is a component that adds a new set of machine learning APIs to let users quickly - * assemble and configure practical machine learning pipelines. + * DataFrame-based machine learning APIs to let users quickly assemble and configure practical + * machine learning pipelines. */ -@Experimental package org.apache.spark.ml; - -import org.apache.spark.annotation.Experimental; diff --git a/mllib/src/main/scala/org/apache/spark/ml/package.scala b/mllib/src/main/scala/org/apache/spark/ml/package.scala index 5cc328b4a9a13..a445c675e41e6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/package.scala @@ -18,8 +18,8 @@ package org.apache.spark /** - * Spark ML is a component that adds a new set of machine learning APIs to let users quickly - * assemble and configure practical machine learning pipelines. + * DataFrame-based machine learning APIs to let users quickly assemble and configure practical + * machine learning pipelines. * * @groupname param Parameters * @groupdesc param A list of (hyper-)parameter keys this algorithm can take. Users can set and get diff --git a/mllib/src/main/scala/org/apache/spark/mllib/JavaPackage.java b/mllib/src/main/scala/org/apache/spark/mllib/JavaPackage.java new file mode 100644 index 0000000000000..22e34524aa592 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/JavaPackage.java @@ -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. + */ + +package org.apache.spark.mllib; + +import org.apache.spark.annotation.AlphaComponent; + +/** + * A dummy class as a workaround to show the package doc of spark.mllib in generated + * Java API docs. + * @see + * JDK-4492654 + */ +@AlphaComponent +public class JavaPackage { + private JavaPackage() {} +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package-info.java b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java index 5962efa96baf3..72b71b7cd9b14 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java @@ -16,6 +16,26 @@ */ /** - * Spark's machine learning library. + * RDD-based machine learning APIs (in maintenance mode). + * + * The spark.mllib package is in maintenance mode as of the Spark 2.0.0 release to + * encourage migration to the DataFrame-based APIs under the spark.ml package. + * While in maintenance mode, + *
      + *
    • + * no new features in the RDD-based spark.mllib package will be accepted, unless + * they block implementing new features in the DataFrame-based spark.ml package; + *
    • + *
    • + * bug fixes in the RDD-based APIs will still be accepted. + *
    • + *
    + * + * The developers will continue adding more features to the DataFrame-based APIs in the 2.x series + * to reach feature parity with the RDD-based APIs. + * And once we reach feature parity, this package will be deprecated. + * + * @see SPARK-4591 to + * track the progress of feature parity */ package org.apache.spark.mllib; diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/package.scala index 5c2b2160c030e..9810b6f668064 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/package.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/package.scala @@ -18,6 +18,21 @@ package org.apache.spark /** - * Spark's machine learning library. + * RDD-based machine learning APIs (in maintenance mode). + * + * The `spark.mllib` package is in maintenance mode as of the Spark 2.0.0 release to encourage + * migration to the DataFrame-based APIs under the [[org.apache.spark.ml]] package. + * While in maintenance mode, + * + * - no new features in the RDD-based `spark.mllib` package will be accepted, unless they block + * implementing new features in the DataFrame-based `spark.ml` package; + * - bug fixes in the RDD-based APIs will still be accepted. + * + * The developers will continue adding more features to the DataFrame-based APIs in the 2.x series + * to reach feature parity with the RDD-based APIs. + * And once we reach feature parity, this package will be deprecated. + * + * @see [[https://issues.apache.org/jira/browse/SPARK-4591 SPARK-4591]] to track the progress of + * feature parity */ package object mllib From 9f18c8f386af558ed72b88ad372835f25e807e79 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 23 Jun 2016 09:45:01 -0700 Subject: [PATCH 0787/1470] [SPARK-16088][SPARKR] update setJobGroup, cancelJobGroup, clearJobGroup ## What changes were proposed in this pull request? Updated setJobGroup, cancelJobGroup, clearJobGroup to not require sc/SparkContext as parameter. Also updated roxygen2 doc and R programming guide on deprecations. ## How was this patch tested? unit tests Author: Felix Cheung Closes #13838 from felixcheung/rjobgroup. (cherry picked from commit b5a997667f4c0e514217da6df5af37b8b849dfdf) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 1 - R/pkg/R/context.R | 10 +--- R/pkg/R/sparkR.R | 68 +++++++++++++++++++----- R/pkg/R/utils.R | 8 +++ R/pkg/inst/tests/testthat/test_context.R | 10 ++-- docs/sparkr.md | 2 + 6 files changed, 75 insertions(+), 24 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 725cbf24f236d..f856979c2a814 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -55,7 +55,6 @@ setMethod("initialize", "SparkDataFrame", function(.Object, sdf, isCached) { .Object }) -#' @rdname SparkDataFrame #' @export #' @param sdf A Java object reference to the backing Scala DataFrame #' @param isCached TRUE if the SparkDataFrame is cached diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index dd0ceaeb08a5d..2538bb25073e1 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -264,10 +264,7 @@ setCheckpointDir <- function(sc, dirName) { #'} #' @note spark.lapply since 2.0.0 spark.lapply <- function(list, func) { - if (!exists(".sparkRjsc", envir = .sparkREnv)) { - stop("SparkR has not been initialized. Please call sparkR.session()") - } - sc <- get(".sparkRjsc", envir = .sparkREnv) + sc <- getSparkContext() rdd <- parallelize(sc, list, length(list)) results <- map(rdd, func) local <- collect(results) @@ -287,9 +284,6 @@ spark.lapply <- function(list, func) { #'} #' @note setLogLevel since 2.0.0 setLogLevel <- function(level) { - if (!exists(".sparkRjsc", envir = .sparkREnv)) { - stop("SparkR has not been initialized. Please call sparkR.session()") - } - sc <- get(".sparkRjsc", envir = .sparkREnv) + sc <- getSparkContext() callJMethod(sc, "setLogLevel", level) } diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 2b6e124151397..62659b0c0ce5f 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -392,47 +392,91 @@ sparkR.session <- function( #' Assigns a group ID to all the jobs started by this thread until the group ID is set to a #' different value or cleared. #' -#' @param sc existing spark context #' @param groupid the ID to be assigned to job groups #' @param description description for the job group ID #' @param interruptOnCancel flag to indicate if the job is interrupted on job cancellation +#' @rdname setJobGroup +#' @name setJobGroup #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' setJobGroup(sc, "myJobGroup", "My job group description", TRUE) +#' sparkR.session() +#' setJobGroup("myJobGroup", "My job group description", TRUE) #'} #' @note setJobGroup since 1.5.0 -setJobGroup <- function(sc, groupId, description, interruptOnCancel) { +#' @method setJobGroup default +setJobGroup.default <- function(groupId, description, interruptOnCancel) { + sc <- getSparkContext() callJMethod(sc, "setJobGroup", groupId, description, interruptOnCancel) } +setJobGroup <- function(sc, groupId, description, interruptOnCancel) { + if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("setJobGroup(groupId, description, interruptOnCancel)", + old = "setJobGroup(sc, groupId, description, interruptOnCancel)") + setJobGroup.default(groupId, description, interruptOnCancel) + } else { + # Parameter order is shifted + groupIdToUse <- sc + descriptionToUse <- groupId + interruptOnCancelToUse <- description + setJobGroup.default(groupIdToUse, descriptionToUse, interruptOnCancelToUse) + } +} + #' Clear current job group ID and its description #' -#' @param sc existing spark context +#' @rdname clearJobGroup +#' @name clearJobGroup #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' clearJobGroup(sc) +#' sparkR.session() +#' clearJobGroup() #'} #' @note clearJobGroup since 1.5.0 -clearJobGroup <- function(sc) { +#' @method clearJobGroup default +clearJobGroup.default <- function() { + sc <- getSparkContext() callJMethod(sc, "clearJobGroup") } +clearJobGroup <- function(sc) { + if (!missing(sc) && + class(sc) == "jobj" && + any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("clearJobGroup()", old = "clearJobGroup(sc)") + } + clearJobGroup.default() +} + + #' Cancel active jobs for the specified group #' -#' @param sc existing spark context #' @param groupId the ID of job group to be cancelled +#' @rdname cancelJobGroup +#' @name cancelJobGroup #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' cancelJobGroup(sc, "myJobGroup") +#' sparkR.session() +#' cancelJobGroup("myJobGroup") #'} #' @note cancelJobGroup since 1.5.0 -cancelJobGroup <- function(sc, groupId) { +#' @method cancelJobGroup default +cancelJobGroup.default <- function(groupId) { + sc <- getSparkContext() callJMethod(sc, "cancelJobGroup", groupId) } +cancelJobGroup <- function(sc, groupId) { + if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("cancelJobGroup(groupId)", old = "cancelJobGroup(sc, groupId)") + cancelJobGroup.default(groupId) + } else { + # Parameter order is shifted + groupIdToUse <- sc + cancelJobGroup.default(groupIdToUse) + } +} + sparkConfToSubmitOps <- new.env() sparkConfToSubmitOps[["spark.driver.memory"]] <- "--driver-memory" sparkConfToSubmitOps[["spark.driver.extraClassPath"]] <- "--driver-class-path" diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index d5c062d3bcc7d..e75bfbf037fbb 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -685,3 +685,11 @@ launchScript <- function(script, combinedArgs, capture = FALSE) { system2(script, combinedArgs, wait = capture, stdout = capture) } } + +getSparkContext <- function() { + if (!exists(".sparkRjsc", envir = .sparkREnv)) { + stop("SparkR has not been initialized. Please call sparkR.session()") + } + sc <- get(".sparkRjsc", envir = .sparkREnv) + sc +} diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 3d232df566a83..2a1bd61b11118 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -100,9 +100,13 @@ test_that("rdd GC across sparkR.stop", { test_that("job group functions can be called", { sc <- sparkR.sparkContext() - setJobGroup(sc, "groupId", "job description", TRUE) - cancelJobGroup(sc, "groupId") - clearJobGroup(sc) + setJobGroup("groupId", "job description", TRUE) + cancelJobGroup("groupId") + clearJobGroup() + + suppressWarnings(setJobGroup(sc, "groupId", "job description", TRUE)) + suppressWarnings(cancelJobGroup(sc, "groupId")) + suppressWarnings(clearJobGroup(sc)) sparkR.session.stop() }) diff --git a/docs/sparkr.md b/docs/sparkr.md index 9e74e4a96acdc..32ef815eb11c4 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -428,3 +428,5 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable`. - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. + - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` + From 63fd3301c5f341d616b314a0a44bfd7d2cf4fbae Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 23 Jun 2016 10:46:20 -0700 Subject: [PATCH 0788/1470] [SPARK-16116][SQL] ConsoleSink should not require checkpointLocation ## What changes were proposed in this pull request? When the user uses `ConsoleSink`, we should use a temp location if `checkpointLocation` is not specified. ## How was this patch tested? The added unit test. Author: Shixiong Zhu Closes #13817 from zsxwing/console-checkpoint. (cherry picked from commit d85bb10ce49926b8b661bd2cb97392205742fc14) Signed-off-by: Shixiong Zhu --- .../apache/spark/sql/streaming/DataStreamWriter.scala | 8 ++++++++ .../streaming/test/DataStreamReaderWriterSuite.scala | 10 ++++++++++ 2 files changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 197707404edda..d4b0a3cca2402 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -272,6 +272,12 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { useTempCheckpointLocation = true, trigger = trigger) } else { + val (useTempCheckpointLocation, recoverFromCheckpointLocation) = + if (source == "console") { + (true, false) + } else { + (false, true) + } val dataSource = DataSource( df.sparkSession, @@ -284,6 +290,8 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { df, dataSource.createSink(outputMode), outputMode, + useTempCheckpointLocation = useTempCheckpointLocation, + recoverFromCheckpointLocation = recoverFromCheckpointLocation, trigger = trigger) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 943e7b761e6e9..f0994395813e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -457,4 +457,14 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { } } } + + test("ConsoleSink should not require checkpointLocation") { + LastOptions.clear() + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + + val sq = df.writeStream.format("console").start() + sq.stop() + } } From dff3d75db4c2848a43ed8a3084c75f38c93138af Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 23 Jun 2016 11:00:00 -0700 Subject: [PATCH 0789/1470] [SPARK-16130][ML] model loading backward compatibility for ml.classfication.LogisticRegression ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-16130 model loading backward compatibility for ml.classfication.LogisticRegression ## How was this patch tested? existing ut and manual test for loading old models. Author: Yuhao Yang Closes #13841 from hhbyyh/lrcomp. (cherry picked from commit 60398dabc50d402bbab4190fbe94ebed6d3a48dc) Signed-off-by: Xiangrui Meng --- .../spark/ml/classification/LogisticRegression.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 be69d46eebef2..9c9f5ced4e35c 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 @@ -674,12 +674,12 @@ object LogisticRegressionModel extends MLReadable[LogisticRegressionModel] { val dataPath = new Path(path, "data").toString val data = sparkSession.read.format("parquet").load(dataPath) - .select("numClasses", "numFeatures", "intercept", "coefficients").head() + // We will need numClasses, numFeatures in the future for multinomial logreg support. - // val numClasses = data.getInt(0) - // val numFeatures = data.getInt(1) - val intercept = data.getDouble(2) - val coefficients = data.getAs[Vector](3) + val Row(numClasses: Int, numFeatures: Int, intercept: Double, coefficients: Vector) = + MLUtils.convertVectorColumnsToML(data, "coefficients") + .select("numClasses", "numFeatures", "intercept", "coefficients") + .head() val model = new LogisticRegressionModel(metadata.uid, coefficients, intercept) DefaultParamsReader.getAndSetParams(model, metadata) From 214676d29d3d66c3e37ab6ff9fae70adb056b8b2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 23 Jun 2016 11:48:48 -0700 Subject: [PATCH 0790/1470] [SPARK-16163] [SQL] Cache the statistics for logical plans ## What changes were proposed in this pull request? This calculation of statistics is not trivial anymore, it could be very slow on large query (for example, TPC-DS Q64 took several minutes to plan). During the planning of a query, the statistics of any logical plan should not change (even InMemoryRelation), so we should use `lazy val` to cache the statistics. For InMemoryRelation, the statistics could be updated after materialization, it's only useful when used in another query (before planning), because once we finished the planning, the statistics will not be used anymore. ## How was this patch tested? Testsed with TPC-DS Q64, it could be planned in a second after the patch. Author: Davies Liu Closes #13871 from davies/fix_statistics. (cherry picked from commit 10396d9505c752cc18b6424f415d4ff0f460ad65) Signed-off-by: Davies Liu --- .../plans/logical/basicLogicalOperators.scala | 20 +++--- .../execution/columnar/InMemoryRelation.scala | 65 ++++++------------- .../org/apache/spark/sql/QueryTest.scala | 3 +- 3 files changed, 30 insertions(+), 58 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index ff3dcbc957ac1..79f9a210a30b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -159,7 +159,7 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation } } - override def statistics: Statistics = { + override lazy val statistics: Statistics = { val leftSize = left.statistics.sizeInBytes val rightSize = right.statistics.sizeInBytes val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize @@ -184,7 +184,7 @@ case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(le left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } && duplicateResolved - override def statistics: Statistics = { + override lazy val statistics: Statistics = { left.statistics.copy() } } @@ -224,7 +224,7 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { children.length > 1 && childrenResolved && allChildrenCompatible } - override def statistics: Statistics = { + override lazy val statistics: Statistics = { val sizeInBytes = children.map(_.statistics.sizeInBytes).sum Statistics(sizeInBytes = sizeInBytes) } @@ -333,7 +333,7 @@ case class Join( case _ => resolvedExceptNatural } - override def statistics: Statistics = joinType match { + override lazy val statistics: Statistics = joinType match { case LeftAnti | LeftSemi => // LeftSemi and LeftAnti won't ever be bigger than left left.statistics.copy() @@ -351,7 +351,7 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output // set isBroadcastable to true so the child will be broadcasted - override def statistics: Statistics = super.statistics.copy(isBroadcastable = true) + override lazy val statistics: Statistics = super.statistics.copy(isBroadcastable = true) } case class InsertIntoTable( @@ -451,7 +451,7 @@ case class Range( override def newInstance(): Range = copy(output = output.map(_.newInstance())) - override def statistics: Statistics = { + override lazy val statistics: Statistics = { val sizeInBytes = LongType.defaultSize * numElements Statistics( sizeInBytes = sizeInBytes ) } @@ -486,7 +486,7 @@ case class Aggregate( override def validConstraints: Set[Expression] = child.constraints.union(getAliasedConstraints(aggregateExpressions)) - override def statistics: Statistics = { + override lazy val statistics: Statistics = { if (groupingExpressions.isEmpty) { super.statistics.copy(sizeInBytes = 1) } else { @@ -586,7 +586,7 @@ case class Expand( override def references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) - override def statistics: Statistics = { + override lazy val statistics: Statistics = { val sizeInBytes = super.statistics.sizeInBytes * projections.length Statistics(sizeInBytes = sizeInBytes) } @@ -706,7 +706,7 @@ case class Sample( override def output: Seq[Attribute] = child.output - override def statistics: Statistics = { + override lazy val statistics: Statistics = { val ratio = upperBound - lowerBound // BigInt can't multiply with Double var sizeInBytes = child.statistics.sizeInBytes * (ratio * 100).toInt / 100 @@ -753,5 +753,5 @@ case object OneRowRelation extends LeafNode { * * [[LeafNode]]s must override this. */ - override def statistics: Statistics = Statistics(sizeInBytes = 1) + override lazy val statistics: Statistics = Statistics(sizeInBytes = 1) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index c546d4bc979c1..02866c76cb7aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -63,58 +63,32 @@ private[sql] case class InMemoryRelation( @transient child: SparkPlan, tableName: Option[String])( @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - @transient private[sql] var _statistics: Statistics = null, - private[sql] var _batchStats: CollectionAccumulator[InternalRow] = null) + private[sql] val batchStats: CollectionAccumulator[InternalRow] = + child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) extends logical.LeafNode with MultiInstanceRelation { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) override def producedAttributes: AttributeSet = outputSet - private[sql] val batchStats: CollectionAccumulator[InternalRow] = - if (_batchStats == null) { - child.sqlContext.sparkContext.collectionAccumulator[InternalRow] - } else { - _batchStats - } - @transient val partitionStatistics = new PartitionStatistics(output) - private def computeSizeInBytes = { - val sizeOfRow: Expression = - BindReferences.bindReference( - output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), - partitionStatistics.schema) - - batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum - } - - // Statistics propagation contracts: - // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data - // 2. Only propagate statistics when `_statistics` is non-null - private def statisticsToBePropagated = if (_statistics == null) { - val updatedStats = statistics - if (_statistics == null) null else updatedStats - } else { - _statistics - } - - override def statistics: Statistics = { - if (_statistics == null) { - if (batchStats.value.isEmpty) { - // Underlying columnar RDD hasn't been materialized, no useful statistics information - // available, return the default statistics. - Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) - } else { - // Underlying columnar RDD has been materialized, required information has also been - // collected via the `batchStats` accumulator, compute the final statistics, - // and update `_statistics`. - _statistics = Statistics(sizeInBytes = computeSizeInBytes) - _statistics - } + override lazy val statistics: Statistics = { + if (batchStats.value.isEmpty) { + // Underlying columnar RDD hasn't been materialized, no useful statistics information + // available, return the default statistics. + Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) } else { - // Pre-computed statistics - _statistics + // Underlying columnar RDD has been materialized, required information has also been + // collected via the `batchStats` accumulator. + val sizeOfRow: Expression = + BindReferences.bindReference( + output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), + partitionStatistics.schema) + + val sizeInBytes = + batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum + Statistics(sizeInBytes = sizeInBytes) } } @@ -187,7 +161,7 @@ private[sql] case class InMemoryRelation( def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, statisticsToBePropagated, batchStats) + _cachedColumnBuffers, batchStats) } override def newInstance(): this.type = { @@ -199,12 +173,11 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, - statisticsToBePropagated, batchStats).asInstanceOf[this.type] } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) + Seq(_cachedColumnBuffers, batchStats) } 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 742f036e5503a..b15f38c2a71e2 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 @@ -323,8 +323,7 @@ abstract class QueryTest extends PlanTest { origin.child, l.tableName)( origin.cachedColumnBuffers, - l._statistics, - origin._batchStats) + origin.batchStats) case p => p.transformExpressions { case s: SubqueryExpression => From b8818d892a4f62ee4e8a1c16146b5ee4c7de9eab Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 23 Jun 2016 13:54:37 -0500 Subject: [PATCH 0791/1470] [SPARK-15725][YARN] Ensure ApplicationMaster sleeps for the min interval. ## What changes were proposed in this pull request? Update `ApplicationMaster` to sleep for at least the minimum allocation interval before calling `allocateResources`. This prevents overloading the `YarnAllocator` that is happening because the thread is triggered when an executor is killed and its connections die. In YARN, this prevents the app from overloading the allocator and becoming unstable. ## How was this patch tested? Tested that this allows the an app to recover instead of hanging. It is still possible for the YarnAllocator to be overwhelmed by requests, but this prevents the issue for the most common cause. Author: Ryan Blue Closes #13482 from rdblue/SPARK-15725-am-sleep-work-around. (cherry picked from commit a410814c87b120cb5cfbf095b1bd94b1de862844) Signed-off-by: Tom Graves --- .../spark/deploy/yarn/ApplicationMaster.scala | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) 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 847d1de50f890..b6f45dd63473b 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 @@ -457,8 +457,10 @@ private[spark] class ApplicationMaster( } try { val numPendingAllocate = allocator.getPendingAllocate.size + var sleepStart = 0L + var sleepInterval = 200L // ms allocatorLock.synchronized { - val sleepInterval = + sleepInterval = if (numPendingAllocate > 0 || allocator.getNumPendingLossReasonRequests > 0) { val currentAllocationInterval = math.min(heartbeatInterval, nextAllocationInterval) @@ -468,10 +470,27 @@ private[spark] class ApplicationMaster( nextAllocationInterval = initialAllocationInterval heartbeatInterval } - logDebug(s"Number of pending allocations is $numPendingAllocate. " + - s"Sleeping for $sleepInterval.") + sleepStart = System.currentTimeMillis() allocatorLock.wait(sleepInterval) } + val sleepDuration = System.currentTimeMillis() - sleepStart + if (sleepDuration < sleepInterval) { + // log when sleep is interrupted + logDebug(s"Number of pending allocations is $numPendingAllocate. " + + s"Slept for $sleepDuration/$sleepInterval ms.") + // if sleep was less than the minimum interval, sleep for the rest of it + val toSleep = math.max(0, initialAllocationInterval - sleepDuration) + if (toSleep > 0) { + logDebug(s"Going back to sleep for $toSleep ms") + // use Thread.sleep instead of allocatorLock.wait. there is no need to be woken up + // by the methods that signal allocatorLock because this is just finishing the min + // sleep interval, which should happen even if this is signalled again. + Thread.sleep(toSleep) + } + } else { + logDebug(s"Number of pending allocations is $numPendingAllocate. " + + s"Slept for $sleepDuration/$sleepInterval.") + } } catch { case e: InterruptedException => } From 2ce240cfe0cbcb944d225b2455a9cb2e806699f0 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 23 Jun 2016 14:03:46 -0500 Subject: [PATCH 0792/1470] [SPARK-13723][YARN] Change behavior of --num-executors with dynamic allocation. ## What changes were proposed in this pull request? This changes the behavior of --num-executors and spark.executor.instances when using dynamic allocation. Instead of turning dynamic allocation off, it uses the value for the initial number of executors. This changes was discussed on [SPARK-13723](https://issues.apache.org/jira/browse/SPARK-13723). I highly recommend using it while we can change the behavior for 2.0.0. In practice, the 1.x behavior causes unexpected behavior for users (it is not clear that it disables dynamic allocation) and wastes cluster resources because users rarely notice the log message. ## How was this patch tested? This patch updates tests and adds a test for Utils.getDynamicAllocationInitialExecutors. Author: Ryan Blue Closes #13338 from rdblue/SPARK-13723-num-executors-with-dynamic-allocation. (cherry picked from commit 738f134bf4bf07bafb17e7066cf1a36e315872c2) Signed-off-by: Tom Graves --- .../spark/ExecutorAllocationManager.scala | 11 +++++----- .../spark/deploy/SparkSubmitArguments.scala | 2 ++ .../scala/org/apache/spark/util/Utils.scala | 22 +++++++++++-------- .../org/apache/spark/util/UtilsSuite.scala | 18 ++++++++++++++- docs/configuration.md | 3 +++ docs/running-on-yarn.md | 2 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- 7 files changed, 42 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 0926d05414ba7..932ba16812bbb 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -25,9 +25,10 @@ import scala.util.control.ControlThrowable import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{DYN_ALLOCATION_MAX_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS} import org.apache.spark.metrics.source.Source import org.apache.spark.scheduler._ -import org.apache.spark.util.{Clock, SystemClock, ThreadUtils} +import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} /** * An agent that dynamically allocates and removes executors based on the workload. @@ -87,11 +88,9 @@ private[spark] class ExecutorAllocationManager( import ExecutorAllocationManager._ // Lower and upper bounds on the number of executors. - private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) - private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", - Integer.MAX_VALUE) - private val initialNumExecutors = conf.getInt("spark.dynamicAllocation.initialExecutors", - minNumExecutors) + private val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS) + private val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS) + private val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) // How long there must be backlogged tasks for before an addition is triggered (seconds) private val schedulerBacklogTimeoutS = conf.getTimeAsSeconds( 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 206c130c76373..f1761e7c1ec92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -550,6 +550,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | (Default: 1). | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). + | If dynamic allocation is enabled, the initial number of + | executors will be at least NUM. | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. | --principal PRINCIPAL Principal to be used to login to KDC, while running on 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 17d193b773fd6..f77cc2f9b7aa0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -52,6 +52,7 @@ import org.slf4j.Logger import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{DYN_ALLOCATION_INITIAL_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS, EXECUTOR_INSTANCES} import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} @@ -2309,21 +2310,24 @@ private[spark] object Utils extends Logging { } /** - * Return whether dynamic allocation is enabled in the given conf - * Dynamic allocation and explicitly setting the number of executors are inherently - * incompatible. In environments where dynamic allocation is turned on by default, - * the latter should override the former (SPARK-9092). + * Return whether dynamic allocation is enabled in the given conf. */ def isDynamicAllocationEnabled(conf: SparkConf): Boolean = { - val numExecutor = conf.getInt("spark.executor.instances", 0) val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) - if (numExecutor != 0 && dynamicAllocationEnabled) { - logWarning("Dynamic Allocation and num executors both set, thus dynamic allocation disabled.") - } - numExecutor == 0 && dynamicAllocationEnabled && + dynamicAllocationEnabled && (!isLocalMaster(conf) || conf.getBoolean("spark.dynamicAllocation.testing", false)) } + /** + * Return the initial number of executors for dynamic allocation. + */ + def getDynamicAllocationInitialExecutors(conf: SparkConf): Int = { + Seq( + conf.get(DYN_ALLOCATION_MIN_EXECUTORS), + conf.get(DYN_ALLOCATION_INITIAL_EXECUTORS), + conf.get(EXECUTOR_INSTANCES).getOrElse(0)).max + } + def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = { val resource = createResource try f.apply(resource) finally resource.close() 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 a5363f0bfd600..e3a8e83f3eaa5 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -761,13 +761,29 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.isDynamicAllocationEnabled( conf.set("spark.dynamicAllocation.enabled", "true")) === true) assert(Utils.isDynamicAllocationEnabled( - conf.set("spark.executor.instances", "1")) === false) + conf.set("spark.executor.instances", "1")) === true) assert(Utils.isDynamicAllocationEnabled( conf.set("spark.executor.instances", "0")) === true) assert(Utils.isDynamicAllocationEnabled(conf.set("spark.master", "local")) === false) assert(Utils.isDynamicAllocationEnabled(conf.set("spark.dynamicAllocation.testing", "true"))) } + test("getDynamicAllocationInitialExecutors") { + val conf = new SparkConf() + assert(Utils.getDynamicAllocationInitialExecutors(conf) === 0) + assert(Utils.getDynamicAllocationInitialExecutors( + conf.set("spark.dynamicAllocation.minExecutors", "3")) === 3) + assert(Utils.getDynamicAllocationInitialExecutors( // should use minExecutors + conf.set("spark.executor.instances", "2")) === 3) + assert(Utils.getDynamicAllocationInitialExecutors( // should use executor.instances + conf.set("spark.executor.instances", "4")) === 4) + assert(Utils.getDynamicAllocationInitialExecutors( // should use executor.instances + conf.set("spark.dynamicAllocation.initialExecutors", "3")) === 4) + assert(Utils.getDynamicAllocationInitialExecutors( // should use initialExecutors + conf.set("spark.dynamicAllocation.initialExecutors", "5")) === 5) + } + + test("encodeFileNameToURIRawPath") { assert(Utils.encodeFileNameToURIRawPath("abc") === "abc") assert(Utils.encodeFileNameToURIRawPath("abc xyz") === "abc%20xyz") diff --git a/docs/configuration.md b/docs/configuration.md index fbda91c109626..cee59cf2aa05f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1236,6 +1236,9 @@ Apart from these, the following properties are also available, and may be useful
    diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9833806716aab..dbd46cc48c14b 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -244,7 +244,7 @@ To use a custom metrics.properties for the application master and executors, upd diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index de6cd946137cb..156a7a30eaa93 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -520,7 +520,7 @@ object YarnSparkHadoopUtil { numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = { if (Utils.isDynamicAllocationEnabled(conf)) { val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS) - val initialNumExecutors = conf.get(DYN_ALLOCATION_INITIAL_EXECUTORS) + val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS) require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, s"initial executor number $initialNumExecutors must between min executor number " + From 6cb24de99e011ce97fb7d3513a2760b0d1a85a45 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 23 Jun 2016 15:27:43 -0700 Subject: [PATCH 0793/1470] [SPARK-16164][SQL] Update `CombineFilters` to try to construct predicates with child predicate first ## What changes were proposed in this pull request? This PR changes `CombineFilters` to compose the final predicate condition by using (`child predicate` AND `parent predicate`) instead of (`parent predicate` AND `child predicate`). This is a best effort approach. Some other optimization rules may destroy this order by reorganizing conjunctive predicates. **Reported Error Scenario** Chris McCubbin reported a bug when he used StringIndexer in an ML pipeline with additional filters. It seems that during filter pushdown, we changed the ordering in the logical plan. ```scala import org.apache.spark.ml.feature._ val df1 = (0 until 3).map(_.toString).toDF val indexer = new StringIndexer() .setInputCol("value") .setOutputCol("idx") .setHandleInvalid("skip") .fit(df1) val df2 = (0 until 5).map(_.toString).toDF val predictions = indexer.transform(df2) predictions.show() // this is okay predictions.where('idx > 2).show() // this will throw an exception ``` Please see the notebook at https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/1233855/2159162931615821/588180/latest.html for error messages. ## How was this patch tested? Pass the Jenkins tests (including a new testcase). Author: Dongjoon Hyun Closes #13872 from dongjoon-hyun/SPARK-16164. (cherry picked from commit 91b1ef28d134313d7b6faaffa1c390f3ca4455d0) Signed-off-by: Xiangrui Meng --- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../optimizer/FilterPushdownSuite.scala | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) 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 6190f7ab1c2e3..6b10484928046 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 @@ -963,7 +963,7 @@ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { (ExpressionSet(splitConjunctivePredicates(fc)) -- ExpressionSet(splitConjunctivePredicates(nc))).reduceOption(And) match { case Some(ac) => - Filter(And(ac, nc), grandChild) + Filter(And(nc, ac), grandChild) case None => nf } 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 b8f28e83e736d..9cb49e74ad34f 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 @@ -94,6 +94,24 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("SPARK-16164: Filter pushdown should keep the ordering in the logical plan") { + val originalQuery = + testRelation + .where('a === 1) + .select('a, 'b) + .where('b === 1) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + testRelation + .where('a === 1 && 'b === 1) + .select('a, 'b) + .analyze + + // We can not use comparePlans here because it normalized the plan. + assert(optimized == correctAnswer) + } + test("can't push without rewrite") { val originalQuery = testRelation From 05677bb5a1fed91711a0e1be466dbc86d15bbf8e Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 23 Jun 2016 16:04:16 -0700 Subject: [PATCH 0794/1470] [SPARK-15443][SQL] Fix 'explain' for streaming Dataset ## What changes were proposed in this pull request? - Fix the `explain` command for streaming Dataset/DataFrame. E.g., ``` == Parsed Logical Plan == 'SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- 'MapElements , obj#6: java.lang.String +- 'DeserializeToObject unresolveddeserializer(createexternalrow(getcolumnbyordinal(0, StringType).toString, StructField(value,StringType,true))), obj#5: org.apache.spark.sql.Row +- Filter .apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] == Analyzed Logical Plan == value: string SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements , obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter .apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] == Optimized Logical Plan == SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements , obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter .apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] == Physical Plan == *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- *MapElements , obj#6: java.lang.String +- *DeserializeToObject createexternalrow(value#0.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- *Filter .apply +- StreamingRelation FileSource[/Users/zsx/stream], [value#0] ``` - Add `StreamingQuery.explain` to display the last execution plan. E.g., ``` == Parsed Logical Plan == SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements , obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter .apply +- Relation[value#12] text == Analyzed Logical Plan == value: string SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements , obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter .apply +- Relation[value#12] text == Optimized Logical Plan == SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- MapElements , obj#6: java.lang.String +- DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- Filter .apply +- Relation[value#12] text == Physical Plan == *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#7] +- *MapElements , obj#6: java.lang.String +- *DeserializeToObject createexternalrow(value#12.toString, StructField(value,StringType,true)), obj#5: org.apache.spark.sql.Row +- *Filter .apply +- *Scan text [value#12] Format: org.apache.spark.sql.execution.datasources.text.TextFileFormat1836ab91, InputPaths: file:/Users/zsx/stream/a.txt, file:/Users/zsx/stream/b.txt, file:/Users/zsx/stream/c.txt, PushedFilters: [], ReadSchema: struct ``` ## How was this patch tested? The added unit tests. Author: Shixiong Zhu Closes #13815 from zsxwing/sdf-explain. (cherry picked from commit 0e4bdebece892edb126fa443f67c846e44e7367e) Signed-off-by: Yin Huai --- .../spark/sql/execution/SparkStrategies.scala | 18 +++++++++- .../sql/execution/command/commands.scala | 11 +++++- .../streaming/IncrementalExecution.scala | 1 + .../execution/streaming/StreamExecution.scala | 20 +++++++++++ .../streaming/StreamingRelation.scala | 14 ++++++++ .../spark/sql/streaming/StreamingQuery.scala | 15 ++++++++ .../sql/streaming/FileStreamSourceSuite.scala | 36 +++++++++++++++++++ .../spark/sql/streaming/StreamSuite.scala | 29 +++++++++++++++ 8 files changed, 142 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 8e2f2ed4f86b9..b619d4edc30de 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} -import org.apache.spark.sql.execution.streaming.MemoryPlan +import org.apache.spark.sql.execution.streaming.{MemoryPlan, StreamingExecutionRelation, StreamingRelation, StreamingRelationExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery @@ -307,6 +307,22 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } + /** + * This strategy is just for explaining `Dataset/DataFrame` created by `spark.readStream`. + * It won't affect the execution, because `StreamingRelation` will be replaced with + * `StreamingExecutionRelation` in `StreamingQueryManager` and `StreamingExecutionRelation` will + * be replaced with the real relation using the `Source` in `StreamExecution`. + */ + object StreamingRelationStrategy extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case s: StreamingRelation => + StreamingRelationExec(s.sourceName, s.output) :: Nil + case s: StreamingExecutionRelation => + StreamingRelationExec(s.toString, s.output) :: Nil + case _ => Nil + } + } + // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { def numPartitions: Int = self.numPartitions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 38bb6e412f753..7eaad81a81615 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -27,6 +27,8 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ +import org.apache.spark.sql.execution.streaming.IncrementalExecution +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types._ /** @@ -98,7 +100,14 @@ case class ExplainCommand( // Run through the optimizer to generate the physical plan. override def run(sparkSession: SparkSession): Seq[Row] = try { - val queryExecution = sparkSession.sessionState.executePlan(logicalPlan) + val queryExecution = + if (logicalPlan.isStreaming) { + // This is used only by explaining `Dataset/DataFrame` created by `spark.readStream`, so the + // output mode does not matter since there is no `Sink`. + new IncrementalExecution(sparkSession, logicalPlan, OutputMode.Append(), "", 0) + } else { + sparkSession.sessionState.executePlan(logicalPlan) + } val outputString = if (codegen) { codegenString(queryExecution.executedPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index bc0e443ca7a55..0ce00552bf6cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -37,6 +37,7 @@ class IncrementalExecution private[sql]( // TODO: make this always part of planning. val stateStrategy = sparkSession.sessionState.planner.StatefulAggregationStrategy +: + sparkSession.sessionState.planner.StreamingRelationStrategy +: sparkSession.sessionState.experimentalMethods.extraStrategies // Modified planner with stateful operations. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 1428b971490d7..f1af79e738faf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} @@ -473,6 +474,25 @@ class StreamExecution( } } + /** Expose for tests */ + def explainInternal(extended: Boolean): String = { + if (lastExecution == null) { + "N/A" + } else { + val explain = ExplainCommand(lastExecution.logical, extended = extended) + sparkSession.sessionState.executePlan(explain).executedPlan.executeCollect() + .map(_.getString(0)).mkString("\n") + } + } + + override def explain(extended: Boolean): Unit = { + // scalastyle:off println + println(explainInternal(extended)) + // scalastyle:on println + } + + override def explain(): Unit = explain(extended = false) + override def toString: String = { s"Streaming Query - $name [state = $state]" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala index 4d65d2f4f57fc..e8b00094add3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.execution.streaming +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.DataSource object StreamingRelation { @@ -50,6 +53,17 @@ case class StreamingExecutionRelation(source: Source, output: Seq[Attribute]) ex override def toString: String = source.toString } +/** + * A dummy physical plan for [[StreamingRelation]] to support + * [[org.apache.spark.sql.Dataset.explain]] + */ +case class StreamingRelationExec(sourceName: String, output: Seq[Attribute]) extends LeafExecNode { + override def toString: String = sourceName + override protected def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException("StreamingRelationExec cannot be executed") + } +} + object StreamingExecutionRelation { def apply(source: Source): StreamingExecutionRelation = { StreamingExecutionRelation(source, source.schema.toAttributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index dc81a5b180276..19d1ecf740d0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -107,6 +107,7 @@ trait StreamingQuery { * method may block forever. Additionally, this method is only guaranteed to block until data that * has been synchronously appended data to a [[org.apache.spark.sql.execution.streaming.Source]] * prior to invocation. (i.e. `getOffset` must immediately reflect the addition). + * @since 2.0.0 */ def processAllAvailable(): Unit @@ -116,4 +117,18 @@ trait StreamingQuery { * @since 2.0.0 */ def stop(): Unit + + /** + * Prints the physical plan to the console for debugging purposes. + * @since 2.0.0 + */ + def explain(): Unit + + /** + * Prints the physical plan to the console for debugging purposes. + * + * @param extended whether to do extended explain or not + * @since 2.0.0 + */ + def explain(extended: Boolean): Unit } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 6971f93b230f1..0eade71d1ebc3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -592,6 +592,42 @@ class FileStreamSourceSuite extends FileStreamSourceTest { ) } } + + test("explain") { + withTempDirs { case (src, tmp) => + src.mkdirs() + + val df = spark.readStream.format("text").load(src.getCanonicalPath).map(_ + "-x") + // Test `explain` not throwing errors + df.explain() + + val q = df.writeStream.queryName("file_explain").format("memory").start() + .asInstanceOf[StreamExecution] + try { + assert("N/A" === q.explainInternal(false)) + assert("N/A" === q.explainInternal(true)) + + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + require(stringToFile(tempFile, "foo").renameTo(finalFile)) + + q.processAllAvailable() + + val explainWithoutExtended = q.explainInternal(false) + // `extended = false` only displays the physical plan. + assert("Relation.*text".r.findAllMatchIn(explainWithoutExtended).size === 0) + assert("TextFileFormat".r.findAllMatchIn(explainWithoutExtended).size === 1) + + val explainWithExtended = q.explainInternal(true) + // `extended = true` displays 3 logical plans (Parsed/Optimized/Optimized) and 1 physical + // plan. + assert("Relation.*text".r.findAllMatchIn(explainWithExtended).size === 3) + assert("TextFileFormat".r.findAllMatchIn(explainWithExtended).size === 1) + } finally { + q.stop() + } + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index b8e40e71bfce5..c4a894b6816ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -242,6 +242,35 @@ class StreamSuite extends StreamTest { val o2 = OutputMode.Complete assert(o2 === InternalOutputModes.Complete) } + + test("explain") { + val inputData = MemoryStream[String] + val df = inputData.toDS().map(_ + "foo") + // Test `explain` not throwing errors + df.explain() + val q = df.writeStream.queryName("memory_explain").format("memory").start() + .asInstanceOf[StreamExecution] + try { + assert("N/A" === q.explainInternal(false)) + assert("N/A" === q.explainInternal(true)) + + inputData.addData("abc") + q.processAllAvailable() + + val explainWithoutExtended = q.explainInternal(false) + // `extended = false` only displays the physical plan. + assert("LocalRelation".r.findAllMatchIn(explainWithoutExtended).size === 0) + assert("LocalTableScan".r.findAllMatchIn(explainWithoutExtended).size === 1) + + val explainWithExtended = q.explainInternal(true) + // `extended = true` displays 3 logical plans (Parsed/Optimized/Optimized) and 1 physical + // plan. + assert("LocalRelation".r.findAllMatchIn(explainWithExtended).size === 3) + assert("LocalTableScan".r.findAllMatchIn(explainWithExtended).size === 1) + } finally { + q.stop() + } + } } /** From e8d1bf60a45c6662c955b5a3618ff7299713b6d8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 24 Jun 2016 07:19:20 +0800 Subject: [PATCH 0795/1470] [SPARK-16165][SQL] Fix the update logic for InMemoryTableScanExec.readBatches ## What changes were proposed in this pull request? Currently, `readBatches` accumulator of `InMemoryTableScanExec` is updated only when `spark.sql.inMemoryColumnarStorage.partitionPruning` is true. Although this metric is used for only testing purpose, we had better have correct metric without considering SQL options. ## How was this patch tested? Pass the Jenkins tests (including a new testcase). Author: Dongjoon Hyun Closes #13870 from dongjoon-hyun/SPARK-16165. (cherry picked from commit 264bc63623b20529abcf84abcb333e7c16ad1ef9) Signed-off-by: Cheng Lian --- .../columnar/InMemoryTableScanExec.scala | 6 +++--- .../columnar/PartitionBatchPruningSuite.scala | 15 +++++++++++++++ 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 2695f356cd3e6..183e4947b6d72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -147,9 +147,6 @@ private[sql] case class InMemoryTableScanExec( logInfo(s"Skipping partition based on stats $statsString") false } else { - if (enableAccumulators) { - readBatches.add(1) - } true } } @@ -159,6 +156,9 @@ private[sql] case class InMemoryTableScanExec( // update SQL metrics val withMetrics = cachedBatchesToScan.map { batch => + if (enableAccumulators) { + readBatches.add(1) + } numOutputRows += batch.numRows batch } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index a118cec0bbb0f..7ca8e047f081d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -119,6 +119,21 @@ class PartitionBatchPruningSuite } } + // With disable IN_MEMORY_PARTITION_PRUNING option + test("disable IN_MEMORY_PARTITION_PRUNING") { + spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, false) + + val df = sql("SELECT key FROM pruningData WHERE key = 1") + val result = df.collect().map(_(0)).toArray + assert(result.length === 1) + + val (readPartitions, readBatches) = df.queryExecution.sparkPlan.collect { + case in: InMemoryTableScanExec => (in.readPartitions.value, in.readBatches.value) + }.head + assert(readPartitions === 5) + assert(readBatches === 10) + } + def checkBatchPruning( query: String, expectedReadPartitions: Int, From edd5dd24a56f1316d14ca23261cea5b49b16adf2 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Thu, 23 Jun 2016 18:21:41 -0700 Subject: [PATCH 0796/1470] [SPARK-16123] Avoid NegativeArraySizeException while reserving additional capacity in VectorizedColumnReader ## What changes were proposed in this pull request? This patch fixes an overflow bug in vectorized parquet reader where both off-heap and on-heap variants of `ColumnVector.reserve()` can unfortunately overflow while reserving additional capacity during reads. ## How was this patch tested? Manual Tests Author: Sameer Agarwal Closes #13832 from sameeragarwal/negative-array. (cherry picked from commit cc71d4fa372f6eb187c68dbd8358de4003ace3fe) Signed-off-by: Herman van Hovell --- .../execution/vectorized/ColumnVector.java | 26 +++++++++++++++++-- .../vectorized/OffHeapColumnVector.java | 8 ++---- .../vectorized/OnHeapColumnVector.java | 8 ++---- .../vectorized/ColumnarBatchSuite.scala | 19 ++++++++++++++ 4 files changed, 47 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index 3f94255256699..80c84b13365f0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -19,6 +19,7 @@ import java.math.BigDecimal; import java.math.BigInteger; +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang.NotImplementedException; import org.apache.parquet.column.Dictionary; import org.apache.parquet.io.api.Binary; @@ -27,6 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -277,11 +279,25 @@ public void reset() { */ public abstract void close(); - /* + public void reserve(int requiredCapacity) { + if (requiredCapacity > capacity) { + int newCapacity = (int) Math.min(MAX_CAPACITY, requiredCapacity * 2L); + if (requiredCapacity <= newCapacity) { + reserveInternal(newCapacity); + } else { + throw new RuntimeException("Cannot reserve more than " + newCapacity + + " bytes in the vectorized reader (requested = " + requiredCapacity + " bytes). As a " + + "workaround, you can disable the vectorized reader by setting " + + SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key() + " to false."); + } + } + } + + /** * Ensures that there is enough storage to store capcity elements. That is, the put() APIs * must work for all rowIds < capcity. */ - public abstract void reserve(int capacity); + protected abstract void reserveInternal(int capacity); /** * Returns the number of nulls in this column. @@ -846,6 +862,12 @@ public final int appendStruct(boolean isNull) { */ protected int capacity; + /** + * Upper limit for the maximum capacity for this column. + */ + @VisibleForTesting + protected int MAX_CAPACITY = Integer.MAX_VALUE; + /** * Data type for this column. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 70b4a683311ca..913a05a0aa0ec 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -422,13 +422,9 @@ public void loadBytes(ColumnVector.Array array) { array.byteArrayOffset = 0; } - @Override - public void reserve(int requiredCapacity) { - if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); - } - // Split out the slow path. - private void reserveInternal(int newCapacity) { + @Override + protected void reserveInternal(int newCapacity) { if (this.resultArray != null) { this.lengthData = Platform.reallocateMemory(lengthData, elementsAppended * 4, newCapacity * 4); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index 7fb7617050f2d..85067df4ebf91 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -392,13 +392,9 @@ public int putByteArray(int rowId, byte[] value, int offset, int length) { return result; } - @Override - public void reserve(int requiredCapacity) { - if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); - } - // Spilt this function out since it is the slow path. - private void reserveInternal(int newCapacity) { + @Override + protected void reserveInternal(int newCapacity) { if (this.resultArray != null || DecimalType.isByteArrayDecimalType(type)) { int[] newLengths = new int[newCapacity]; int[] newOffsets = new int[newCapacity]; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 7e576a8657998..100cc4daca875 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -787,4 +787,23 @@ class ColumnarBatchSuite extends SparkFunSuite { } } } + + test("exceeding maximum capacity should throw an error") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => + val column = ColumnVector.allocate(1, ByteType, memMode) + column.MAX_CAPACITY = 15 + column.appendBytes(5, 0.toByte) + // Successfully allocate twice the requested capacity + assert(column.capacity == 10) + column.appendBytes(10, 0.toByte) + // Allocated capacity doesn't exceed MAX_CAPACITY + assert(column.capacity == 15) + val ex = intercept[RuntimeException] { + // Over-allocating beyond MAX_CAPACITY throws an exception + column.appendBytes(10, 0.toByte) + } + assert(ex.getMessage.contains(s"Cannot reserve more than ${column.MAX_CAPACITY} bytes in " + + s"the vectorized reader")) + } + } } From a6edec2c5c1d217a9fc16ee6696a558a6c88a34e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 23 Jun 2016 20:20:55 -0700 Subject: [PATCH 0797/1470] [SQL][MINOR] ParserUtils.operationNotAllowed should throw exception directly ## What changes were proposed in this pull request? It's weird that `ParserUtils.operationNotAllowed` returns an exception and the caller throw it. ## How was this patch tested? N/A Author: Wenchen Fan Closes #13874 from cloud-fan/style. (cherry picked from commit 6a3c6276f5cef26b0a4fef44c8ad99bbecfe006d) Signed-off-by: Herman van Hovell --- .../sql/catalyst/parser/ParserUtils.scala | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 62 +++++++++---------- 2 files changed, 33 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 9619884edeafe..b04ce58e233aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -39,8 +39,8 @@ object ParserUtils { stream.getText(Interval.of(0, stream.size())) } - def operationNotAllowed(message: String, ctx: ParserRuleContext): ParseException = { - new ParseException(s"Operation not allowed: $message", ctx) + def operationNotAllowed(message: String, ctx: ParserRuleContext): Nothing = { + throw new ParseException(s"Operation not allowed: $message", ctx) } /** Check if duplicate keys exist in a set of key-value pairs. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2ae8380644acd..066ff57721a3d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -168,7 +168,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val lookupTable = Option(ctx.db) match { case None => table case Some(db) if table.database.exists(_ != db) => - throw operationNotAllowed( + operationNotAllowed( s"SHOW COLUMNS with conflicting databases: '$db' != '${table.database.get}'", ctx) case Some(db) => TableIdentifier(table.identifier, Some(db.getText)) @@ -253,10 +253,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitExplain(ctx: ExplainContext): LogicalPlan = withOrigin(ctx) { if (ctx.FORMATTED != null) { - throw operationNotAllowed("EXPLAIN FORMATTED", ctx) + operationNotAllowed("EXPLAIN FORMATTED", ctx) } if (ctx.LOGICAL != null) { - throw operationNotAllowed("EXPLAIN LOGICAL", ctx) + operationNotAllowed("EXPLAIN LOGICAL", ctx) } val statement = plan(ctx.statement) @@ -304,7 +304,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val temporary = ctx.TEMPORARY != null val ifNotExists = ctx.EXISTS != null if (temporary && ifNotExists) { - throw operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) + operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) } (visitTableIdentifier(ctx.tableIdentifier), temporary, ifNotExists, ctx.EXTERNAL != null) } @@ -317,7 +317,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitCreateTableUsing(ctx: CreateTableUsingContext): LogicalPlan = withOrigin(ctx) { val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) if (external) { - throw operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) + operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } val options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val provider = ctx.tableProvider.qualifiedName.getText @@ -332,7 +332,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val query = plan(ctx.query) if (temp) { - throw operationNotAllowed("CREATE TEMPORARY TABLE ... USING ... AS query", ctx) + operationNotAllowed("CREATE TEMPORARY TABLE ... USING ... AS query", ctx) } // Determine the storage mode. @@ -428,7 +428,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val props = visitTablePropertyList(ctx) val badKeys = props.filter { case (_, v) => v == null }.keys if (badKeys.nonEmpty) { - throw operationNotAllowed( + operationNotAllowed( s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) } props @@ -441,7 +441,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val props = visitTablePropertyList(ctx) val badKeys = props.filter { case (_, v) => v != null }.keys if (badKeys.nonEmpty) { - throw operationNotAllowed( + operationNotAllowed( s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) } props.keys.toSeq @@ -564,7 +564,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case "jar" | "file" | "archive" => FunctionResource(FunctionResourceType.fromString(resourceType), string(resource.STRING)) case other => - throw operationNotAllowed(s"CREATE FUNCTION with resource type '$resourceType'", ctx) + operationNotAllowed(s"CREATE FUNCTION with resource type '$resourceType'", ctx) } } @@ -600,7 +600,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { if (ctx.PURGE != null) { - throw operationNotAllowed("DROP TABLE ... PURGE", ctx) + operationNotAllowed("DROP TABLE ... PURGE", ctx) } DropTableCommand( visitTableIdentifier(ctx.tableIdentifier), @@ -692,7 +692,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitAddTablePartition( ctx: AddTablePartitionContext): LogicalPlan = withOrigin(ctx) { if (ctx.VIEW != null) { - throw operationNotAllowed("ALTER VIEW ... ADD PARTITION", ctx) + operationNotAllowed("ALTER VIEW ... ADD PARTITION", ctx) } // Create partition spec to location mapping. val specsAndLocs = if (ctx.partitionSpec.isEmpty) { @@ -743,10 +743,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { override def visitDropTablePartitions( ctx: DropTablePartitionsContext): LogicalPlan = withOrigin(ctx) { if (ctx.VIEW != null) { - throw operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx) + operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx) } if (ctx.PURGE != null) { - throw operationNotAllowed("ALTER TABLE ... DROP PARTITION ... PURGE", ctx) + operationNotAllowed("ALTER TABLE ... DROP PARTITION ... PURGE", ctx) } AlterTableDropPartitionCommand( visitTableIdentifier(ctx.tableIdentifier), @@ -789,7 +789,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { .map { orderedIdCtx => Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => if (dir.toLowerCase != "asc") { - throw operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) + operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) } } @@ -825,7 +825,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // SET ROLE is the exception to the rule, because we handle this before other SET commands. "SET ROLE" } - throw operationNotAllowed(keywords, ctx) + operationNotAllowed(keywords, ctx) } /** @@ -844,7 +844,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.identifier.getText.toLowerCase match { case "file" => AddFileCommand(mayebePaths) case "jar" => AddJarCommand(mayebePaths) - case other => throw operationNotAllowed(s"ADD with resource type '$other'", ctx) + case other => operationNotAllowed(s"ADD with resource type '$other'", ctx) } case SqlBaseParser.LIST => ctx.identifier.getText.toLowerCase match { @@ -860,9 +860,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } else { ListJarsCommand() } - case other => throw operationNotAllowed(s"LIST with resource type '$other'", ctx) + case other => operationNotAllowed(s"LIST with resource type '$other'", ctx) } - case _ => throw operationNotAllowed(s"Other types of operation on resources", ctx) + case _ => operationNotAllowed(s"Other types of operation on resources", ctx) } } @@ -898,10 +898,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { "Please use CREATE TEMPORARY VIEW as an alternative.", ctx) } if (ctx.skewSpec != null) { - throw operationNotAllowed("CREATE TABLE ... SKEWED BY", ctx) + operationNotAllowed("CREATE TABLE ... SKEWED BY", ctx) } if (ctx.bucketSpec != null) { - throw operationNotAllowed("CREATE TABLE ... CLUSTERED BY", ctx) + operationNotAllowed("CREATE TABLE ... CLUSTERED BY", ctx) } val comment = Option(ctx.STRING).map(string) val partitionCols = Option(ctx.partitionColumns).toSeq.flatMap(visitCatalogColumns) @@ -915,14 +915,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val duplicateColumns = colNames.groupBy(identity).collect { case (x, ys) if ys.length > 1 => "\"" + x + "\"" } - throw operationNotAllowed(s"Duplicated column names found in table definition of $name: " + + operationNotAllowed(s"Duplicated column names found in table definition of $name: " + duplicateColumns.mkString("[", ",", "]"), ctx) } // For Hive tables, partition columns must not be part of the schema val badPartCols = partitionCols.map(_.name).toSet.intersect(colNames.toSet) if (badPartCols.nonEmpty) { - throw operationNotAllowed(s"Partition columns may not be specified in the schema: " + + operationNotAllowed(s"Partition columns may not be specified in the schema: " + badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]"), ctx) } @@ -954,7 +954,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val location = Option(ctx.locationSpec).map(visitLocationSpec) // If we are creating an EXTERNAL table, then the LOCATION field is required if (external && location.isEmpty) { - throw operationNotAllowed("CREATE EXTERNAL TABLE must be accompanied by LOCATION", ctx) + operationNotAllowed("CREATE EXTERNAL TABLE must be accompanied by LOCATION", ctx) } val storage = CatalogStorageFormat( locationUri = location, @@ -985,7 +985,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case Some(q) => // Just use whatever is projected in the select statement as our schema if (schema.nonEmpty) { - throw operationNotAllowed( + operationNotAllowed( "Schema may not be specified in a Create Table As Select (CTAS) statement", ctx) } @@ -996,7 +996,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { "Please use the syntax of \"CREATE TABLE tableName USING dataSource " + "OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a " + "CTAS statement." - throw operationNotAllowed(errorMessage, ctx) + operationNotAllowed(errorMessage, ctx) } val hasStorageProperties = (ctx.createFileFormat != null) || (ctx.rowFormat != null) @@ -1055,7 +1055,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { case (c: GenericFileFormatContext, null) => visitGenericFileFormat(c) case (null, storageHandler) => - throw operationNotAllowed("STORED BY", ctx) + operationNotAllowed("STORED BY", ctx) case _ => throw new ParseException("Expected either STORED AS or STORED BY, not both", ctx) } @@ -1084,7 +1084,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { outputFormat = s.outputFormat, serde = s.serde) case None => - throw operationNotAllowed(s"STORED AS with file format '$source'", ctx) + operationNotAllowed(s"STORED AS with file format '$source'", ctx) } } @@ -1174,14 +1174,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ffGeneric.identifier.getText.toLowerCase match { case ("sequencefile" | "textfile" | "rcfile") => // OK case fmt => - throw operationNotAllowed( + operationNotAllowed( s"ROW FORMAT SERDE is incompatible with format '$fmt', which also specifies a serde", parentCtx) } case (rfDelimited: RowFormatDelimitedContext, ffGeneric: GenericFileFormatContext) => ffGeneric.identifier.getText.toLowerCase match { case "textfile" => // OK - case fmt => throw operationNotAllowed( + case fmt => operationNotAllowed( s"ROW FORMAT DELIMITED is only compatible with 'textfile', not '$fmt'", parentCtx) } case _ => @@ -1189,7 +1189,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { def str(ctx: ParserRuleContext): String = { (0 until ctx.getChildCount).map { i => ctx.getChild(i).getText }.mkString(" ") } - throw operationNotAllowed( + operationNotAllowed( s"Unexpected combination of ${str(rowFormatCtx)} and ${str(createFileFormatCtx)}", parentCtx) } @@ -1209,7 +1209,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) { if (ctx.identifierList != null) { - throw operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx) + operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx) } else { val identifiers = Option(ctx.identifierCommentList).toSeq.flatMap(_.identifierComment.asScala) val schema = identifiers.map { ic => From ea0cf93d3969845e9df8305c0ce54326cdfb2bbd Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 23 Jun 2016 20:43:19 -0700 Subject: [PATCH 0798/1470] [SPARK-16177][ML] model loading backward compatibility for ml.regression ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-16177 model loading backward compatibility for ml.regression ## How was this patch tested? existing ut and manual test for loading 1.6 models. Author: Yuhao Yang Closes #13879 from hhbyyh/regreComp. (cherry picked from commit 14bc5a7f36bed19cd714a4c725a83feaccac3468) Signed-off-by: Xiangrui Meng --- .../spark/ml/regression/AFTSurvivalRegression.scala | 9 +++++---- .../apache/spark/ml/regression/LinearRegression.scala | 8 +++++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 2dbac49ccf25b..7c51845a25815 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -33,6 +33,7 @@ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ @@ -389,10 +390,10 @@ object AFTSurvivalRegressionModel extends MLReadable[AFTSurvivalRegressionModel] val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) - .select("coefficients", "intercept", "scale").head() - val coefficients = data.getAs[Vector](0) - val intercept = data.getDouble(1) - val scale = data.getDouble(2) + val Row(coefficients: Vector, intercept: Double, scale: Double) = + MLUtils.convertVectorColumnsToML(data, "coefficients") + .select("coefficients", "intercept", "scale") + .head() val model = new AFTSurvivalRegressionModel(metadata.uid, coefficients, intercept, scale) DefaultParamsReader.getAndSetParams(model, metadata) 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 2723f74724927..0a4d98cab64aa 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 @@ -39,6 +39,7 @@ import org.apache.spark.mllib.evaluation.RegressionMetrics import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ @@ -500,9 +501,10 @@ object LinearRegressionModel extends MLReadable[LinearRegressionModel] { val dataPath = new Path(path, "data").toString val data = sparkSession.read.format("parquet").load(dataPath) - .select("intercept", "coefficients").head() - val intercept = data.getDouble(0) - val coefficients = data.getAs[Vector](1) + val Row(intercept: Double, coefficients: Vector) = + MLUtils.convertVectorColumnsToML(data, "coefficients") + .select("intercept", "coefficients") + .head() val model = new LinearRegressionModel(metadata.uid, coefficients, intercept) DefaultParamsReader.getAndSetParams(model, metadata) From 557eee5b6d07f8a17257cd9aae5d7830b4de4690 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 23 Jun 2016 21:43:13 -0700 Subject: [PATCH 0799/1470] [SPARK-16142][R] group naiveBayes method docs in a single Rd ## What changes were proposed in this pull request? This PR groups `spark.naiveBayes`, `summary(NB)`, `predict(NB)`, and `write.ml(NB)` into a single Rd. ## How was this patch tested? Manually checked generated HTML doc. See attached screenshots. ![screen shot 2016-06-23 at 2 11 00 pm](https://cloud.githubusercontent.com/assets/829644/16320452/a5885e92-394c-11e6-994f-2ab5cddad86f.png) ![screen shot 2016-06-23 at 2 11 15 pm](https://cloud.githubusercontent.com/assets/829644/16320455/aad1f6d8-394c-11e6-8ef4-13bee989f52f.png) Author: Xiangrui Meng Closes #13877 from mengxr/SPARK-16142. (cherry picked from commit 4a40d43bb29704734b8128bf2a3f27802ae34e17) Signed-off-by: Xiangrui Meng --- R/pkg/R/mllib.R | 90 +++++++++++++++++++++++-------------------------- 1 file changed, 42 insertions(+), 48 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index dbff1b900d38a..853cfce74ae56 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -218,9 +218,10 @@ print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { # Makes predictions from a generalized linear model produced by glm() or spark.glm(), # similarly to R's predict(). -#' + #' @param newData SparkDataFrame for testing -#' @return \code{predict} returns a SparkDataFrame containing predicted labels in a column named "prediction" +#' @return \code{predict} returns a SparkDataFrame containing predicted labels in a column named +#' "prediction" #' @rdname spark.glm #' @export #' @note predict(GeneralizedLinearRegressionModel) since 1.5.0 @@ -229,41 +230,26 @@ setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) }) -#' Predicted values based on model -#' -#' Makes predictions from a naive Bayes model or a model produced by spark.naiveBayes(), -#' similarly to R package e1071's predict. -#' -#' @param object A fitted naive Bayes model -#' @rdname predict +# Makes predictions from a naive Bayes model or a model produced by spark.naiveBayes(), +# similarly to R package e1071's predict. + +#' @rdname spark.naiveBayes +#' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named +#' "prediction" #' @export -#' @examples -#' \dontrun{ -#' model <- spark.naiveBayes(trainingData, y ~ x) -#' predicted <- predict(model, testData) -#' showDF(predicted) -#'} #' @note predict(NaiveBayesModel) since 2.0.0 setMethod("predict", signature(object = "NaiveBayesModel"), function(object, newData) { return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) }) -#' Get the summary of a naive Bayes model -#' -#' Returns the summary of a naive Bayes model produced by spark.naiveBayes(), -#' similarly to R's summary(). -#' -#' @param object A fitted MLlib model -#' @return a list containing 'apriori', the label distribution, and 'tables', conditional -# probabilities given the target label -#' @rdname summary +# Returns the summary of a naive Bayes model produced by \code{spark.naiveBayes} + +#' @param object A naive Bayes model fitted by \code{spark.naiveBayes} +#' @return \code{summary} returns a list containing \code{apriori}, the label distribution, and +#' \code{tables}, conditional probabilities given the target label +#' @rdname spark.naiveBayes #' @export -#' @examples -#' \dontrun{ -#' model <- spark.naiveBayes(trainingData, y ~ x) -#' summary(model) -#'} #' @note summary(NaiveBayesModel) since 2.0.0 setMethod("summary", signature(object = "NaiveBayesModel"), function(object, ...) { @@ -390,23 +376,41 @@ setMethod("predict", signature(object = "KMeansModel"), return(dataFrame(callJMethod(object@jobj, "transform", newData@sdf))) }) -#' Fit a Bernoulli naive Bayes model +#' Naive Bayes Models #' -#' Fit a Bernoulli naive Bayes model on a Spark DataFrame (only categorical data is supported). +#' \code{spark.naiveBayes} fits a Bernoulli naive Bayes model against a SparkDataFrame. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' Only categorical data is supported. #' -#' @param data SparkDataFrame for training +#' @param data A \code{SparkDataFrame} of observations and labels for model fitting #' @param formula A symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', '.', ':', '+', and '-'. #' @param smoothing Smoothing parameter -#' @return a fitted naive Bayes model +#' @return \code{spark.naiveBayes} returns a fitted naive Bayes model #' @rdname spark.naiveBayes +#' @name spark.naiveBayes #' @seealso e1071: \url{https://cran.r-project.org/web/packages/e1071/} #' @export #' @examples #' \dontrun{ #' df <- createDataFrame(infert) +#' +#' # fit a Bernoulli naive Bayes model #' model <- spark.naiveBayes(df, education ~ ., smoothing = 0) -#'} +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } #' @note spark.naiveBayes since 2.0.0 setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "formula"), function(data, formula, smoothing = 1.0, ...) { @@ -416,25 +420,15 @@ setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "form return(new("NaiveBayesModel", jobj = jobj)) }) -#' Save fitted MLlib model to the input path -#' -#' Save the Bernoulli naive Bayes model to the input path. -#' -#' @param object A fitted Bernoulli naive Bayes model +# Saves the Bernoulli naive Bayes model to the input path. + #' @param path The directory where the model is saved #' @param overwrite Overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. #' -#' @rdname write.ml -#' @name write.ml +#' @rdname spark.naiveBayes #' @export -#' @examples -#' \dontrun{ -#' df <- createDataFrame(infert) -#' model <- spark.naiveBayes(df, education ~ ., smoothing = 0) -#' path <- "path/to/model" -#' write.ml(model, path) -#' } +#' @seealso \link{read.ml} #' @note write.ml(NaiveBayesModel, character) since 2.0.0 setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), function(object, path, overwrite = FALSE) { From 3d8d956448fd3b7ae8d380e655bfa245b11c4ea0 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 23 Jun 2016 21:50:25 -0700 Subject: [PATCH 0800/1470] [SPARK-16133][ML] model loading backward compatibility for ml.feature ## What changes were proposed in this pull request? model loading backward compatibility for ml.feature, ## How was this patch tested? existing ut and manual test for loading 1.6 models. Author: Yuhao Yang Author: Yuhao Yang Closes #13844 from hhbyyh/featureComp. (cherry picked from commit cc6778ee0bf4fa7a78abd30542c4a6f80ea371c5) Signed-off-by: Xiangrui Meng --- .../src/main/scala/org/apache/spark/ml/feature/IDF.scala | 3 ++- .../scala/org/apache/spark/ml/feature/MinMaxScaler.scala | 9 ++++++--- .../org/apache/spark/ml/feature/StandardScaler.scala | 4 +++- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 02d4e6a9f7555..5d6287f0e3f15 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -27,6 +27,7 @@ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ @@ -180,9 +181,9 @@ object IDFModel extends MLReadable[IDFModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) + val Row(idf: Vector) = MLUtils.convertVectorColumnsToML(data, "idf") .select("idf") .head() - val idf = data.getAs[Vector](0) val model = new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf))) DefaultParamsReader.getAndSetParams(model, metadata) model diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 562b3f38e4ec6..d5ad5abced469 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ @@ -232,9 +233,11 @@ object MinMaxScalerModel extends MLReadable[MinMaxScalerModel] { override def load(path: String): MinMaxScalerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val Row(originalMin: Vector, originalMax: Vector) = sparkSession.read.parquet(dataPath) - .select("originalMin", "originalMax") - .head() + val data = sparkSession.read.parquet(dataPath) + val Row(originalMin: Vector, originalMax: Vector) = + MLUtils.convertVectorColumnsToML(data, "originalMin", "originalMax") + .select("originalMin", "originalMax") + .head() val model = new MinMaxScalerModel(metadata.uid, originalMin, originalMax) DefaultParamsReader.getAndSetParams(model, metadata) model 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 be58dc27e0602..b4be95494fd10 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 @@ -28,6 +28,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ @@ -211,7 +212,8 @@ object StandardScalerModel extends MLReadable[StandardScalerModel] { override def load(path: String): StandardScalerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val Row(std: Vector, mean: Vector) = sparkSession.read.parquet(dataPath) + val data = sparkSession.read.parquet(dataPath) + val Row(std: Vector, mean: Vector) = MLUtils.convertVectorColumnsToML(data, "std", "mean") .select("std", "mean") .head() val model = new StandardScalerModel(metadata.uid, std, mean) From 3ccdd6b9c6e63c7498771dcd8673914c46f6794a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 23 Jun 2016 23:11:46 -0700 Subject: [PATCH 0801/1470] [SPARK-13709][SQL] Initialize deserializer with both table and partition properties when reading partitioned tables ## What changes were proposed in this pull request? When reading partitions of a partitioned Hive SerDe table, we only initializes the deserializer using partition properties. However, for SerDes like `AvroSerDe`, essential properties (e.g. Avro schema information) may be defined in table properties. We should merge both table properties and partition properties before initializing the deserializer. Note that an individual partition may have different properties than the one defined in the table properties (e.g. partitions within a table can have different SerDes). Thus, for any property key defined in both partition and table properties, the value set in partition properties wins. ## How was this patch tested? New test case added in `QueryPartitionSuite`. Author: Cheng Lian Closes #13865 from liancheng/spark-13709-partitioned-avro-table. (cherry picked from commit 2d2f607bfae97f2681df24f48bb8b1b483c6b309) Signed-off-by: Yin Huai --- .../apache/spark/sql/hive/TableReader.scala | 17 +++- .../spark/sql/hive/QueryPartitionSuite.scala | 81 +++++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) 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 d044811052219..e4cb33b28520f 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 @@ -17,6 +17,10 @@ package org.apache.spark.sql.hive +import java.util.Properties + +import scala.collection.JavaConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ @@ -230,10 +234,21 @@ class HadoopTableReader( // Fill all partition keys to the given MutableRow object fillPartitionKeys(partValues, mutableRow) + val tableProperties = relation.tableDesc.getProperties + createHadoopRdd(tableDesc, inputPathStr, ifc).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = localDeserializer.newInstance() - deserializer.initialize(hconf, partProps) + // SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema + // information) may be defined in table properties. Here we should merge table properties + // and partition properties before initializing the deserializer. Note that partition + // properties take a higher priority here. For example, a partition may have a different + // SerDe as the one defined in table properties. + val props = new Properties(tableProperties) + partProps.asScala.foreach { + case (key, value) => props.setProperty(key, value) + } + deserializer.initialize(hconf, props) // get the table deserializer val tableSerDe = tableDesc.getDeserializerClass.newInstance() tableSerDe.initialize(hconf, tableDesc.getProperties) 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 f7650e001a8b9..feeaade561441 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,7 +17,10 @@ package org.apache.spark.sql.hive +import java.io.File + import com.google.common.io.Files +import org.apache.hadoop.fs.FileSystem import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -65,4 +68,82 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl sql("DROP TABLE IF EXISTS createAndInsertTest") } } + + test("SPARK-13709: reading partitioned Avro table with nested schema") { + withTempDir { dir => + val path = dir.getCanonicalPath + val tableName = "spark_13709" + val tempTableName = "spark_13709_temp" + + new File(path, tableName).mkdir() + new File(path, tempTableName).mkdir() + + val avroSchema = + """{ + | "name": "test_record", + | "type": "record", + | "fields": [ { + | "name": "f0", + | "type": "int" + | }, { + | "name": "f1", + | "type": { + | "type": "record", + | "name": "inner", + | "fields": [ { + | "name": "f10", + | "type": "int" + | }, { + | "name": "f11", + | "type": "double" + | } ] + | } + | } ] + |} + """.stripMargin + + withTable(tableName, tempTableName) { + // Creates the external partitioned Avro table to be tested. + sql( + s"""CREATE EXTERNAL TABLE $tableName + |PARTITIONED BY (ds STRING) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |LOCATION '$path/$tableName' + |TBLPROPERTIES ('avro.schema.literal' = '$avroSchema') + """.stripMargin + ) + + // Creates an temporary Avro table used to prepare testing Avro file. + sql( + s"""CREATE EXTERNAL TABLE $tempTableName + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |LOCATION '$path/$tempTableName' + |TBLPROPERTIES ('avro.schema.literal' = '$avroSchema') + """.stripMargin + ) + + // Generates Avro data. + sql(s"INSERT OVERWRITE TABLE $tempTableName SELECT 1, STRUCT(2, 2.5)") + + // Adds generated Avro data as a new partition to the testing table. + sql(s"ALTER TABLE $tableName ADD PARTITION (ds = 'foo') LOCATION '$path/$tempTableName'") + + // The following query fails before SPARK-13709 is fixed. This is because when reading data + // from table partitions, Avro deserializer needs the Avro schema, which is defined in + // table property "avro.schema.literal". However, we only initializes the deserializer using + // partition properties, which doesn't include the wanted property entry. Merging two sets + // of properties solves the problem. + checkAnswer( + sql(s"SELECT * FROM $tableName"), + Row(1, Row(2, 2.5D), "foo") + ) + } + } + } } From b6420db9ebc59c453a6a523aba68addf5762bb2c Mon Sep 17 00:00:00 2001 From: "peng.zhang" Date: Fri, 24 Jun 2016 08:28:32 +0100 Subject: [PATCH 0802/1470] [SPARK-16125][YARN] Fix not test yarn cluster mode correctly in YarnClusterSuite ## What changes were proposed in this pull request? Since SPARK-13220(Deprecate "yarn-client" and "yarn-cluster"), YarnClusterSuite doesn't test "yarn cluster" mode correctly. This pull request fixes it. ## How was this patch tested? Unit test (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: peng.zhang Closes #13836 from renozhang/SPARK-16125-test-yarn-cluster-mode. (cherry picked from commit f4fd7432fb9cf7b197ccada1378c4f2a6d427522) Signed-off-by: Sean Owen --- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 3 ++- python/pyspark/context.py | 4 ---- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 -- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 4 files changed, 3 insertions(+), 8 deletions(-) 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 e3a8e83f3eaa5..df279b5a37c7d 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -754,7 +754,8 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("isDynamicAllocationEnabled") { val conf = new SparkConf() - conf.set("spark.master", "yarn-client") + conf.set("spark.master", "yarn") + conf.set("spark.submit.deployMode", "client") assert(Utils.isDynamicAllocationEnabled(conf) === false) assert(Utils.isDynamicAllocationEnabled( conf.set("spark.dynamicAllocation.enabled", "false")) === false) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index aec0215b40945..7217a9907a482 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -155,10 +155,6 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self.appName = self._conf.get("spark.app.name") self.sparkHome = self._conf.get("spark.home", None) - # Let YARN know it's a pyspark app, so it distributes needed libraries. - if self.master == "yarn-client": - self._conf.set("spark.yarn.isPython", "true") - for (k, v) in self._conf.getAll(): if k.startswith("spark.executorEnv."): varName = k[len("spark.executorEnv."):] diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 8fcab386ec8f9..e871004173704 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -943,8 +943,6 @@ class SparkILoop( }) private def process(settings: Settings): Boolean = savingContextLoader { - if (getMaster() == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") - this.settings = settings createInterpreter() 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 4ce33e0e8565d..6b20dea5908a8 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 @@ -312,7 +312,7 @@ private object YarnClusterDriver extends Logging with Matchers { // If we are running in yarn-cluster mode, verify that driver logs links and present and are // in the expected format. - if (conf.get("spark.master") == "yarn-cluster") { + if (conf.get("spark.submit.deployMode") == "cluster") { assert(listener.driverLogs.nonEmpty) val driverLogs = listener.driverLogs.get assert(driverLogs.size === 2) From 201d5e8db3fd29898a6cd69e015ca491e5721b08 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 24 Jun 2016 10:35:54 +0100 Subject: [PATCH 0803/1470] [SPARK-16129][CORE][SQL] Eliminate direct use of commons-lang classes in favor of commons-lang3 ## What changes were proposed in this pull request? Replace use of `commons-lang` in favor of `commons-lang3` and forbid the former via scalastyle; remove `NotImplementedException` from `comons-lang` in favor of JDK `UnsupportedOperationException` ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #13843 from srowen/SPARK-16129. (cherry picked from commit 158af162eac7348464c6751c8acd48fc6c117688) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/SparkContext.scala | 5 ++-- scalastyle-config.xml | 6 +++++ .../sql/catalyst/expressions/TimeWindow.scala | 2 +- .../spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../parquet/VectorizedColumnReader.java | 25 +++++++++---------- .../execution/vectorized/ColumnVector.java | 17 ++++++------- .../vectorized/ColumnVectorUtils.java | 6 ++--- .../execution/vectorized/ColumnarBatch.java | 12 ++++----- .../spark/sql/execution/ExistingRDD.scala | 2 +- .../execution/columnar/InMemoryRelation.scala | 2 +- .../service/cli/session/HiveSessionImpl.java | 2 +- .../spark/streaming/StreamingContext.scala | 5 ++-- .../streaming/scheduler/JobScheduler.scala | 6 ++--- 13 files changed, 44 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d8701812ebeda..fe15052b62478 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -24,7 +24,6 @@ import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID} import java.util.concurrent.ConcurrentMap import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} -import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.generic.Growable @@ -34,7 +33,7 @@ import scala.reflect.{classTag, ClassTag} import scala.util.control.NonFatal import com.google.common.collect.MapMaker -import org.apache.commons.lang.SerializationUtils +import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, @@ -334,7 +333,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli override protected def childValue(parent: Properties): Properties = { // Note: make a clone such that changes in the parent properties aren't reflected in // the those of the children threads, which has confusing semantics (SPARK-10563). - SerializationUtils.clone(parent).asInstanceOf[Properties] + SerializationUtils.clone(parent) } override protected def initialValue(): Properties = new Properties() } diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 270104f85b838..9a35183c63733 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -210,6 +210,12 @@ This file is divided into 3 sections: scala.collection.JavaConverters._ and use .asScala / .asJava methods + + org\.apache\.commons\.lang\. + Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead + of Commons Lang 2 (package org.apache.commons.lang.*) + + java,scala,3rdParty,spark diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 83fa447cf8c85..66c4bf29ea4b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.commons.lang.StringUtils +import org.apache.commons.lang3.StringUtils import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult 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 3cc7a1a3cae83..072445af4f41f 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 @@ -23,7 +23,7 @@ import scala.collection.Map import scala.collection.mutable.Stack import scala.reflect.ClassTag -import org.apache.commons.lang.ClassUtils +import org.apache.commons.lang3.ClassUtils import org.json4s.JsonAST._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 662a03d3b56ae..a18b881c78a09 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -19,7 +19,6 @@ import java.io.IOException; -import org.apache.commons.lang.NotImplementedException; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; @@ -228,7 +227,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getInt(i))); } } else { - throw new NotImplementedException("Unimplemented type: " + column.dataType()); + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); } break; @@ -239,7 +238,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, column.putLong(i, dictionary.decodeToLong(dictionaryIds.getInt(i))); } } else { - throw new NotImplementedException("Unimplemented type: " + column.dataType()); + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); } break; @@ -262,7 +261,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); } } else { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } break; case BINARY: @@ -293,12 +292,12 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, column.putByteArray(i, v.getBytes()); } } else { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } break; default: - throw new NotImplementedException("Unsupported type: " + descriptor.getType()); + throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType()); } } @@ -327,7 +326,7 @@ private void readIntBatch(int rowId, int num, ColumnVector column) throws IOExce defColumn.readShorts( num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); } else { - throw new NotImplementedException("Unimplemented type: " + column.dataType()); + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); } } @@ -360,7 +359,7 @@ private void readDoubleBatch(int rowId, int num, ColumnVector column) throws IOE defColumn.readDoubles( num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); } else { - throw new NotImplementedException("Unimplemented type: " + column.dataType()); + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); } } @@ -381,7 +380,7 @@ private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOE } } } else { - throw new NotImplementedException("Unimplemented type: " + column.dataType()); + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); } } @@ -417,7 +416,7 @@ private void readFixedLenByteArrayBatch(int rowId, int num, } } } else { - throw new NotImplementedException("Unimplemented type: " + column.dataType()); + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); } } @@ -459,13 +458,13 @@ private void initDataReader(Encoding dataEncoding, byte[] bytes, int offset) thr @SuppressWarnings("deprecation") Encoding plainDict = Encoding.PLAIN_DICTIONARY; // var to allow warning suppression if (dataEncoding != plainDict && dataEncoding != Encoding.RLE_DICTIONARY) { - throw new NotImplementedException("Unsupported encoding: " + dataEncoding); + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); } this.dataColumn = new VectorizedRleValuesReader(); this.useDictionary = true; } else { if (dataEncoding != Encoding.PLAIN) { - throw new NotImplementedException("Unsupported encoding: " + dataEncoding); + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); } this.dataColumn = new VectorizedPlainValuesReader(); this.useDictionary = false; @@ -485,7 +484,7 @@ private void readPageV1(DataPageV1 page) throws IOException { // Initialize the decoders. if (page.getDlEncoding() != Encoding.RLE && descriptor.getMaxDefinitionLevel() != 0) { - throw new NotImplementedException("Unsupported encoding: " + page.getDlEncoding()); + throw new UnsupportedOperationException("Unsupported encoding: " + page.getDlEncoding()); } int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); this.defColumn = new VectorizedRleValuesReader(bitWidth); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index 80c84b13365f0..bbbb796aca0de 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -20,7 +20,6 @@ import java.math.BigInteger; import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.lang.NotImplementedException; import org.apache.parquet.column.Dictionary; import org.apache.parquet.io.api.Binary; @@ -100,7 +99,7 @@ protected Array(ColumnVector data) { @Override public ArrayData copy() { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } // TODO: this is extremely expensive. @@ -171,7 +170,7 @@ public Object[] array() { } } } else { - throw new NotImplementedException("Type " + dt); + throw new UnsupportedOperationException("Type " + dt); } return list; } @@ -181,7 +180,7 @@ public Object[] array() { @Override public boolean getBoolean(int ordinal) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } @Override @@ -189,7 +188,7 @@ public boolean getBoolean(int ordinal) { @Override public short getShort(int ordinal) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } @Override @@ -200,7 +199,7 @@ public short getShort(int ordinal) { @Override public float getFloat(int ordinal) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } @Override @@ -240,12 +239,12 @@ public ArrayData getArray(int ordinal) { @Override public MapData getMap(int ordinal) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } @Override public Object get(int ordinal, DataType dataType) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } } @@ -562,7 +561,7 @@ private Array getByteArray(int rowId) { * Returns the value for rowId. */ public MapData getMap(int ordinal) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java index f50c35fc64a75..2fa476b9cfb71 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java @@ -23,8 +23,6 @@ import java.util.Iterator; import java.util.List; -import org.apache.commons.lang.NotImplementedException; - import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; @@ -112,7 +110,7 @@ public static Object toPrimitiveJavaArray(ColumnVector.Array array) { } return result; } else { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } } @@ -161,7 +159,7 @@ private static void appendValue(ColumnVector dst, DataType t, Object o) { } else if (t instanceof DateType) { dst.appendInt(DateTimeUtils.fromJavaDate((Date)o)); } else { - throw new NotImplementedException("Type " + t); + throw new UnsupportedOperationException("Type " + t); } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index 8cece73faa4b9..f3afa8f938f86 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -19,8 +19,6 @@ import java.math.BigDecimal; import java.util.*; -import org.apache.commons.lang.NotImplementedException; - import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericMutableRow; @@ -166,7 +164,7 @@ public InternalRow copy() { @Override public boolean anyNull() { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } @Override @@ -227,12 +225,12 @@ public ArrayData getArray(int ordinal) { @Override public MapData getMap(int ordinal) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } @Override public Object get(int ordinal, DataType dataType) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } @Override @@ -258,7 +256,7 @@ public void update(int ordinal, Object value) { setDecimal(ordinal, Decimal.apply((BigDecimal) value, t.precision(), t.scale()), t.precision()); } else { - throw new NotImplementedException("Datatype not supported " + dt); + throw new UnsupportedOperationException("Datatype not supported " + dt); } } } @@ -430,7 +428,7 @@ public int numValidRows() { */ public void setColumn(int ordinal, ColumnVector column) { if (column instanceof OffHeapColumnVector) { - throw new NotImplementedException("Need to ref count columns."); + throw new UnsupportedOperationException("Need to ref count columns."); } columns[ordinal] = column; } 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 e2c23a4ba8670..09203e69983da 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.commons.lang.StringUtils +import org.apache.commons.lang3.StringUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 02866c76cb7aa..079e122a5a85a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar import scala.collection.JavaConverters._ -import org.apache.commons.lang.StringUtils +import org.apache.commons.lang3.StringUtils import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java index cc3e807e7a840..47bfaa86021d6 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -29,7 +29,7 @@ import java.util.Set; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.common.cli.HiveFileProcessor; diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index b524af9578b72..6046426fdf8cb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.Queue import scala.reflect.ClassTag import scala.util.control.NonFatal -import org.apache.commons.lang.SerializationUtils +import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} @@ -579,8 +579,7 @@ class StreamingContext private[streaming] ( sparkContext.setCallSite(startSite.get) sparkContext.clearJobGroup() sparkContext.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false") - savedProperties.set(SerializationUtils.clone( - sparkContext.localProperties.get()).asInstanceOf[Properties]) + savedProperties.set(SerializationUtils.clone(sparkContext.localProperties.get())) scheduler.start() } state = StreamingContextState.ACTIVE 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 ac18f73ea86aa..79d6254eb372b 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 @@ -17,13 +17,12 @@ package org.apache.spark.streaming.scheduler -import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ import scala.util.Failure -import org.apache.commons.lang.SerializationUtils +import org.apache.commons.lang3.SerializationUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.{PairRDDFunctions, RDD} @@ -219,8 +218,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { def run() { val oldProps = ssc.sparkContext.getLocalProperties try { - ssc.sparkContext.setLocalProperties( - SerializationUtils.clone(ssc.savedProperties.get()).asInstanceOf[Properties]) + ssc.sparkContext.setLocalProperties(SerializationUtils.clone(ssc.savedProperties.get())) val formattedTime = UIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" From 76741b570e20eb7957ada28ad3c5babc0abb738f Mon Sep 17 00:00:00 2001 From: GayathriMurali Date: Fri, 24 Jun 2016 13:25:40 +0200 Subject: [PATCH 0804/1470] [SPARK-15997][DOC][ML] Update user guide for HashingTF, QuantileVectorizer and CountVectorizer ## What changes were proposed in this pull request? Made changes to HashingTF,QuantileVectorizer and CountVectorizer Author: GayathriMurali Closes #13745 from GayathriMurali/SPARK-15997. (cherry picked from commit be88383e15a86d094963de5f7e8792510bc990de) Signed-off-by: Nick Pentreath --- docs/ml-features.md | 29 +++++++++++-------- .../ml/JavaQuantileDiscretizerExample.java | 7 ++++- .../python/ml/quantile_discretizer_example.py | 11 +++++-- .../ml/QuantileDiscretizerExample.scala | 9 ++++-- 4 files changed, 38 insertions(+), 18 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index 3cb26443b9516..88fd291b4be50 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -46,14 +46,18 @@ In MLlib, we separate TF and IDF to make them flexible. `HashingTF` is a `Transformer` which takes sets of terms and converts those sets into fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. `HashingTF` utilizes the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). -A raw feature is mapped into an index (term) by applying a hash function. Then term frequencies +A raw feature is mapped into an index (term) by applying a hash function. The hash function +used here is [MurmurHash 3](https://en.wikipedia.org/wiki/MurmurHash). Then term frequencies are calculated based on the mapped indices. This approach avoids the need to compute a global term-to-index map, which can be expensive for a large corpus, but it suffers from potential hash collisions, where different raw features may become the same term after hashing. To reduce the chance of collision, we can increase the target feature dimension, i.e. the number of buckets of the hash table. Since a simple modulo is used to transform the hash function to a column index, it is advisable to use a power of two as the feature dimension, otherwise the features will -not be mapped evenly to the columns. The default feature dimension is `$2^{18} = 262,144$`. +not be mapped evenly to the columns. The default feature dimension is `$2^{18} = 262,144$`. +An optional binary toggle parameter controls term frequency counts. When set to true all nonzero +frequency counts are set to 1. This is especially useful for discrete probabilistic models that +model binary, rather than integer, counts. `CountVectorizer` converts text documents to vectors of term counts. Refer to [CountVectorizer ](ml-features.html#countvectorizer) for more details. @@ -145,9 +149,11 @@ for more details on the API. passed to other algorithms like LDA. During the fitting process, `CountVectorizer` will select the top `vocabSize` words ordered by - term frequency across the corpus. An optional parameter "minDF" also affects the fitting process + term frequency across the corpus. An optional parameter `minDF` also affects the fitting process by specifying the minimum number (or fraction if < 1.0) of documents a term must appear in to be - included in the vocabulary. + included in the vocabulary. Another optional binary toggle parameter controls the output vector. + If set to true all nonzero counts are set to 1. This is especially useful for discrete probabilistic + models that model binary, rather than integer, counts. **Examples** @@ -1096,14 +1102,13 @@ for more details on the API. ## QuantileDiscretizer `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned -categorical features. -The bin ranges are chosen by taking a sample of the data and dividing it into roughly equal parts. -The lower and upper bin bounds will be `-Infinity` and `+Infinity`, covering all real values. -This attempts to find `numBuckets` partitions based on a sample of the given input data, but it may -find fewer depending on the data sample values. - -Note that the result may be different every time you run it, since the sample strategy behind it is -non-deterministic. +categorical features. The number of bins is set by the `numBuckets` parameter. +The bin ranges are chosen using an approximate algorithm (see the documentation for +[approxQuantile](api/scala/index.html#org.apache.spark.sql.DataFrameStatFunctions) for a +detailed description). The precision of the approximation can be controlled with the +`relativeError` parameter. When set to zero, exact quantiles are calculated +(**Note:** Computing exact quantiles is an expensive operation). The lower and upper bin bounds +will be `-Infinity` and `+Infinity` covering all real values. **Examples** diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java index 16f58a852d8a2..dd20cac621102 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java @@ -54,7 +54,12 @@ public static void main(String[] args) { }); Dataset df = spark.createDataFrame(data, schema); - + // $example off$ + // Output of QuantileDiscretizer for such small datasets can depend on the number of + // partitions. Here we force a single partition to ensure consistent results. + // Note this is not necessary for normal use cases + df = df.repartition(1); + // $example on$ QuantileDiscretizer discretizer = new QuantileDiscretizer() .setInputCol("hour") .setOutputCol("result") diff --git a/examples/src/main/python/ml/quantile_discretizer_example.py b/examples/src/main/python/ml/quantile_discretizer_example.py index 6ae7bb18f8c67..5444cacd957f3 100644 --- a/examples/src/main/python/ml/quantile_discretizer_example.py +++ b/examples/src/main/python/ml/quantile_discretizer_example.py @@ -28,11 +28,16 @@ # $example on$ data = [(0, 18.0,), (1, 19.0,), (2, 8.0,), (3, 5.0,), (4, 2.2,)] - dataFrame = spark.createDataFrame(data, ["id", "hour"]) - + df = spark.createDataFrame(data, ["id", "hour"]) + # $example off$ + # Output of QuantileDiscretizer for such small datasets can depend on the number of + # partitions. Here we force a single partition to ensure consistent results. + # Note this is not necessary for normal use cases + df = df.repartition(1) + # $example on$ discretizer = QuantileDiscretizer(numBuckets=3, inputCol="hour", outputCol="result") - result = discretizer.fit(dataFrame).transform(dataFrame) + result = discretizer.fit(df).transform(df) result.show() # $example off$ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala index 1a16515594161..2f7e217b8fe2d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -32,8 +32,13 @@ object QuantileDiscretizerExample { // $example on$ val data = Array((0, 18.0), (1, 19.0), (2, 8.0), (3, 5.0), (4, 2.2)) - val df = spark.createDataFrame(data).toDF("id", "hour") - + var df = spark.createDataFrame(data).toDF("id", "hour") + // $example off$ + // Output of QuantileDiscretizer for such small datasets can depend on the number of + // partitions. Here we force a single partition to ensure consistent results. + // Note this is not necessary for normal use cases + .repartition(1) + // $example on$ val discretizer = new QuantileDiscretizer() .setInputCol("hour") .setOutputCol("result") From 4bb8cca441b7c6e584374429fca1499659582dc6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 24 Jun 2016 14:35:34 -0700 Subject: [PATCH 0805/1470] [SPARK-16077] [PYSPARK] catch the exception from pickle.whichmodule() ## What changes were proposed in this pull request? In the case that we don't know which module a object came from, will call pickle.whichmodule() to go throught all the loaded modules to find the object, which could fail because some modules, for example, six, see https://bitbucket.org/gutworth/six/issues/63/importing-six-breaks-pickling We should ignore the exception here, use `__main__` as the module name (it means we can't find the module). ## How was this patch tested? Manual tested. Can't have a unit test for this. Author: Davies Liu Closes #13788 from davies/whichmodule. (cherry picked from commit d48935400ca47275f677b527c636976af09332c8) Signed-off-by: Davies Liu --- python/pyspark/cloudpickle.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index e56e22a9b920e..822ae46e45111 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -169,7 +169,12 @@ def save_function(self, obj, name=None): if name is None: name = obj.__name__ - modname = pickle.whichmodule(obj, name) + try: + # whichmodule() could fail, see + # https://bitbucket.org/gutworth/six/issues/63/importing-six-breaks-pickling + modname = pickle.whichmodule(obj, name) + except Exception: + modname = None # print('which gives %s %s %s' % (modname, obj, name)) try: themodule = sys.modules[modname] @@ -326,7 +331,12 @@ def save_global(self, obj, name=None, pack=struct.pack): modname = getattr(obj, "__module__", None) if modname is None: - modname = pickle.whichmodule(obj, name) + try: + # whichmodule() could fail, see + # https://bitbucket.org/gutworth/six/issues/63/importing-six-breaks-pickling + modname = pickle.whichmodule(obj, name) + except Exception: + modname = '__main__' if modname == '__main__': themodule = None From df137e3e0a7b24675be8702b408a75f14e9995c2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 24 Jun 2016 15:20:39 -0700 Subject: [PATCH 0806/1470] [SPARK-16179][PYSPARK] fix bugs for Python udf in generate ## What changes were proposed in this pull request? This PR fix the bug when Python UDF is used in explode (generator), GenerateExec requires that all the attributes in expressions should be resolvable from children when creating, we should replace the children first, then replace it's expressions. ``` >>> df.select(explode(f(*df))).show() Traceback (most recent call last): File "", line 1, in File "/home/vlad/dev/spark/python/pyspark/sql/dataframe.py", line 286, in show print(self._jdf.showString(n, truncate)) File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/home/vlad/dev/spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o52.showString. : org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree: Generate explode((_1#0L)), false, false, [col#15L] +- Scan ExistingRDD[_1#0L] at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:387) at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:69) at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:45) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDown(QueryPlan.scala:177) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressions(QueryPlan.scala:144) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.org$apache$spark$sql$execution$python$ExtractPythonUDFs$$extract(ExtractPythonUDFs.scala:153) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:114) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:113) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:300) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:298) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:113) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:93) at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95) at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124) at scala.collection.immutable.List.foldLeft(List.scala:84) at org.apache.spark.sql.execution.QueryExecution.prepareForExecution(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:85) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:85) at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2557) at org.apache.spark.sql.Dataset.head(Dataset.scala:1923) at org.apache.spark.sql.Dataset.take(Dataset.scala:2138) at org.apache.spark.sql.Dataset.showString(Dataset.scala:239) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) at py4j.Gateway.invoke(Gateway.java:280) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:211) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.reflect.InvocationTargetException at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:412) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:387) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) ... 42 more Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#20 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:278) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:268) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:87) at org.apache.spark.sql.execution.GenerateExec.(GenerateExec.scala:63) ... 52 more Caused by: java.lang.RuntimeException: Couldn't find pythonUDF0#20 in [_1#0L] at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:94) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) ... 67 more ``` ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #13883 from davies/udf_in_generate. (cherry picked from commit 4435de1bd36e2c30b764725fae05a08733f4aad4) Signed-off-by: Reynold Xin --- python/pyspark/sql/tests.py | 7 +++++++ .../spark/sql/execution/python/ExtractPythonUDFs.scala | 4 ++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 388ac919221c7..3dc4083704526 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -363,6 +363,13 @@ def test_udf_with_aggregate_function(self): .select(my_add(col("k"), col("s")).alias("t")) self.assertEqual(sel.collect(), [Row(t=4), Row(t=3)]) + def test_udf_in_generate(self): + from pyspark.sql.functions import udf, explode + df = self.spark.range(5) + f = udf(lambda x: list(range(x)), ArrayType(LongType())) + row = df.select(explode(f(*df))).groupBy().sum().first() + self.assertEqual(row[0], 10) + def test_basic_functions(self): rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) df = self.spark.read.json(rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index 87583c82347eb..829bcae6f95d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -150,10 +150,10 @@ private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") } - val rewritten = plan.transformExpressions { + val rewritten = plan.withNewChildren(newChildren).transformExpressions { case p: PythonUDF if attributeMap.contains(p) => attributeMap(p) - }.withNewChildren(newChildren) + } // extract remaining python UDFs recursively val newPlan = extract(rewritten) From 9de095513c1776f556839a15ba0923baebcec3b8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 24 Jun 2016 17:26:39 -0700 Subject: [PATCH 0807/1470] [SPARK-16173] [SQL] Can't join describe() of DataFrame in Scala 2.10 ## What changes were proposed in this pull request? This PR fixes `DataFrame.describe()` by forcing materialization to make the `Seq` serializable. Currently, `describe()` of DataFrame throws `Task not serializable` Spark exceptions when joining in Scala 2.10. ## How was this patch tested? Manual. (After building with Scala 2.10, test on `bin/spark-shell` and `bin/pyspark`.) Author: Dongjoon Hyun Closes #13900 from dongjoon-hyun/SPARK-16173. (cherry picked from commit e5d0928e2473d1838ff5420c6a8964557c33135e) Signed-off-by: Davies Liu --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index f1d33c3e5ce92..85d060639c7ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1908,7 +1908,8 @@ class Dataset[T] private[sql]( // All columns are string type val schema = StructType( StructField("summary", StringType) :: outputCols.map(StructField(_, StringType))).toAttributes - LocalRelation.fromExternalRows(schema, ret) + // `toArray` forces materialization to make the seq serializable + LocalRelation.fromExternalRows(schema, ret.toArray.toSeq) } /** From 9e2384845f0900229144d557eac82b03a2ba06d1 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Fri, 24 Jun 2016 17:27:33 -0700 Subject: [PATCH 0808/1470] [SPARK-16195][SQL] Allow users to specify empty over clause in window expressions through dataset API ## What changes were proposed in this pull request? Allow to specify empty over clause in window expressions through dataset API In SQL, its allowed to specify an empty OVER clause in the window expression. ```SQL select area, sum(product) over () as c from windowData where product > 3 group by area, product having avg(month) > 0 order by avg(month), product ``` In this case the analytic function sum is presented based on all the rows of the result set Currently its not allowed through dataset API and is handled in this PR. ## How was this patch tested? Added a new test in DataframeWindowSuite Author: Dilip Biswal Closes #13897 from dilipbiswal/spark-empty-over. (cherry picked from commit 9053054c7f5ec2b9e3d8efbe6bfbfa68a6d1f0d0) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/Column.scala | 17 +++++++++++++++++ .../apache/spark/sql/expressions/Window.scala | 2 +- .../apache/spark/sql/DataFrameWindowSuite.scala | 12 ++++++++++++ 3 files changed, 30 insertions(+), 1 deletion(-) 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 713f7941beeb2..9f35107e5bb6d 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 @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression +import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions.lit import org.apache.spark.sql.types._ @@ -1094,6 +1095,22 @@ class Column(protected[sql] val expr: Expression) extends Logging { */ def over(window: expressions.WindowSpec): Column = window.withAggregate(this) + /** + * Define a empty analytic clause. In this case the analytic function is applied + * and presented for all rows in the result set. + * + * {{{ + * df.select( + * sum("price").over(), + * avg("price").over() + * ) + * }}} + * + * @group expr_ops + * @since 2.0.0 + */ + def over(): Column = over(Window.spec) + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index 350c2836461e2..c29ec6f426789 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -74,7 +74,7 @@ object Window { spec.orderBy(cols : _*) } - private def spec: WindowSpec = { + private[sql] def spec: WindowSpec = { new WindowSpec(Seq.empty, Seq.empty, UnspecifiedFrame) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala index 9a1aa46947c91..c6f8c3ad3fc93 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowSuite.scala @@ -245,6 +245,18 @@ class DataFrameWindowSuite extends QueryTest with SharedSQLContext { Seq(Row("a", 6, 9), Row("b", 9, 6))) } + test("SPARK-16195 empty over spec") { + val df = Seq(("a", 1), ("a", 1), ("a", 2), ("b", 2)). + toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select($"key", $"value", sum($"value").over(), avg($"value").over()), + Seq(Row("a", 1, 6, 1.5), Row("a", 1, 6, 1.5), Row("a", 2, 6, 1.5), Row("b", 2, 6, 1.5))) + checkAnswer( + sql("select key, value, sum(value) over(), avg(value) over() from window_table"), + Seq(Row("a", 1, 6, 1.5), Row("a", 1, 6, 1.5), Row("a", 2, 6, 1.5), Row("b", 2, 6, 1.5))) + } + test("window function with udaf") { val udaf = new UserDefinedAggregateFunction { def inputSchema: StructType = new StructType() From d079b5de70e0987050a3ef23c6a35a61854f6d6f Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Fri, 24 Jun 2016 21:07:03 -0700 Subject: [PATCH 0809/1470] [SPARK-16192][SQL] Add type checks in CollectSet ## What changes were proposed in this pull request? `CollectSet` cannot have map-typed data because MapTypeData does not implement `equals`. So, this pr is to add type checks in `CheckAnalysis`. ## How was this patch tested? Added tests to check failures when we found map-typed data in `CollectSet`. Author: Takeshi YAMAMURO Closes #13892 from maropu/SPARK-16192. (cherry picked from commit d2e44d7db82ff3c3326af7bf7ea69c803803698e) Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- .../sql/catalyst/expressions/aggregate/collect.scala | 9 +++++++++ .../org/apache/spark/sql/DataFrameAggregateSuite.scala | 10 ++++++++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 899227674f2ac..ac9693e079f51 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 @@ -73,9 +73,9 @@ trait CheckAnalysis extends PredicateHelper { s"invalid cast from ${c.child.dataType.simpleString} to ${c.dataType.simpleString}") case g: Grouping => - failAnalysis(s"grouping() can only be used with GroupingSets/Cube/Rollup") + failAnalysis("grouping() can only be used with GroupingSets/Cube/Rollup") case g: GroupingID => - failAnalysis(s"grouping_id() can only be used with GroupingSets/Cube/Rollup") + failAnalysis("grouping_id() can only be used with GroupingSets/Cube/Rollup") case w @ WindowExpression(AggregateExpression(_, _, true, _), _) => failAnalysis(s"Distinct window functions are not supported: $w") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 1f4ff9c4b184e..ac2cefaddcf59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import scala.collection.generic.Growable import scala.collection.mutable +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.GenericArrayData import org.apache.spark.sql.catalyst.InternalRow @@ -107,6 +108,14 @@ case class CollectSet( def this(child: Expression) = this(child, 0, 0) + override def checkInputDataTypes(): TypeCheckResult = { + if (!child.dataType.existsRecursively(_.isInstanceOf[MapType])) { + TypeCheckResult.TypeCheckSuccess + } else { + TypeCheckResult.TypeCheckFailure("collect_set() cannot have map type data") + } + } + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = copy(mutableAggBufferOffset = newMutableAggBufferOffset) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 69a990789bcfd..92aa7b95434dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -457,6 +457,16 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { ) } + test("collect_set functions cannot have maps") { + val df = Seq((1, 3, 0), (2, 3, 0), (3, 4, 1)) + .toDF("a", "x", "y") + .select($"a", map($"x", $"y").as("b")) + val error = intercept[AnalysisException] { + df.select(collect_set($"a"), collect_set($"b")) + } + assert(error.message.contains("collect_set() cannot have map type data")) + } + test("SPARK-14664: Decimal sum/avg over window should work.") { checkAnswer( spark.sql("select sum(a) over () from values 1.0, 2.0, 3.0 T(a)"), From cbfcdcfb60d41126e17cddda52922d6058f1a401 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Antonio?= Date: Sat, 25 Jun 2016 09:11:25 +0100 Subject: [PATCH 0810/1470] [MLLIB] org.apache.spark.mllib.util.SVMDataGenerator generates ArrayIndexOutOfBoundsException. I have found the bug and tested the solution. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Just adjust the size of an array in line 58 so it does not cause an ArrayOutOfBoundsException in line 66. ## How was this patch tested? Manual tests. I have recompiled the entire project with the fix, it has been built successfully and I have run the code, also with good results. line 66: val yD = blas.ddot(trueWeights.length, x, 1, trueWeights, 1) + rnd.nextGaussian() * 0.1 crashes because trueWeights has length "nfeatures + 1" while "x" has length "features", and they should have the same length. To fix this just make trueWeights be the same length as x. I have recompiled the project with the change and it is working now: [spark-1.6.1]$ spark-submit --master local[*] --class org.apache.spark.mllib.util.SVMDataGenerator mllib/target/spark-mllib_2.11-1.6.1.jar local /home/user/test And it generates the data successfully now in the specified folder. Author: José Antonio Closes #13895 from j4munoz/patch-2. (cherry picked from commit a3c7b4187bad00dad87df7e3b5929a44d29568ed) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/mllib/util/SVMDataGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index cde5979396178..c9468606544db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -55,7 +55,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) + val trueWeights = Array.fill[Double](nfeatures)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) From b03b0976fac878bf7e5d1721441179a4d4d9c317 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 25 Jun 2016 12:14:14 +0100 Subject: [PATCH 0811/1470] [SPARK-16193][TESTS] Address flaky ExternalAppendOnlyMapSuite spilling tests ## What changes were proposed in this pull request? Make spill tests wait until job has completed before returning the number of stages that spilled ## How was this patch tested? Existing Jenkins tests. Author: Sean Owen Closes #13896 from srowen/SPARK-16193. (cherry picked from commit e87741589a24821b5fe73e5d9ee2164247998580) Signed-off-by: Sean Owen --- .../src/main/scala/org/apache/spark/TestUtils.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 43c89b258f2fa..871b9d1ad575b 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -22,6 +22,7 @@ import java.net.{URI, URL} import java.nio.charset.StandardCharsets import java.nio.file.Paths import java.util.Arrays +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConverters._ @@ -190,8 +191,14 @@ private[spark] object TestUtils { private class SpillListener extends SparkListener { private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]] private val spilledStageIds = new mutable.HashSet[Int] + private val stagesDone = new CountDownLatch(1) - def numSpilledStages: Int = spilledStageIds.size + def numSpilledStages: Int = { + // Long timeout, just in case somehow the job end isn't notified. + // Fails if a timeout occurs + assert(stagesDone.await(10, TimeUnit.SECONDS)) + spilledStageIds.size + } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { stageIdToTaskMetrics.getOrElseUpdate( @@ -206,4 +213,8 @@ private class SpillListener extends SparkListener { spilledStageIds += stageId } } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + stagesDone.countDown() + } } From e01776395d14bb3ff1b9d6f9317938871457ac2c Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sun, 26 Jun 2016 13:10:43 -0700 Subject: [PATCH 0812/1470] [SPARK-16184][SPARKR] conf API for SparkSession ## What changes were proposed in this pull request? Add `conf` method to get Runtime Config from SparkSession ## How was this patch tested? unit tests, manual tests This is how it works in sparkR shell: ``` SparkSession available as 'spark'. > conf() $hive.metastore.warehouse.dir [1] "file:/opt/spark-2.0.0-bin-hadoop2.6/R/spark-warehouse" $spark.app.id [1] "local-1466749575523" $spark.app.name [1] "SparkR" $spark.driver.host [1] "10.0.2.1" $spark.driver.port [1] "45629" $spark.executorEnv.LD_LIBRARY_PATH [1] "$LD_LIBRARY_PATH:/usr/lib/R/lib:/usr/lib/x86_64-linux-gnu:/usr/lib/jvm/default-java/jre/lib/amd64/server" $spark.executor.id [1] "driver" $spark.home [1] "/opt/spark-2.0.0-bin-hadoop2.6" $spark.master [1] "local[*]" $spark.sql.catalogImplementation [1] "hive" $spark.submit.deployMode [1] "client" > conf("spark.master") $spark.master [1] "local[*]" ``` Author: Felix Cheung Closes #13885 from felixcheung/rconf. (cherry picked from commit 30b182bcc088aef161585211c517f473b9ee6632) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/SQLContext.R | 50 +++++++++++++++++-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 ++--- .../org/apache/spark/sql/api/r/SQLUtils.scala | 4 ++ 4 files changed, 57 insertions(+), 10 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 2272d8bdd52c2..e0ffde922dacf 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -10,6 +10,7 @@ export("sparkR.session") export("sparkR.init") export("sparkR.stop") export("sparkR.session.stop") +export("sparkR.conf") export("print.jobj") export("sparkRSQL.init", diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index ee3a41cacbee6..8df73db36e956 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -110,11 +110,53 @@ infer_type <- function(x) { } } -getDefaultSqlSource <- function() { +#' Get Runtime Config from the current active SparkSession +#' +#' Get Runtime Config from the current active SparkSession. +#' To change SparkSession Runtime Config, please see `sparkR.session()`. +#' +#' @param key (optional) The key of the config to get, if omitted, all config is returned +#' @param defaultValue (optional) The default value of the config to return if they config is not +#' set, if omitted, the call fails if the config key is not set +#' @return a list of config values with keys as their names +#' @rdname sparkR.conf +#' @name sparkR.conf +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' allConfigs <- sparkR.conf() +#' masterValue <- unlist(sparkR.conf("spark.master")) +#' namedConfig <- sparkR.conf("spark.executor.memory", "0g") +#' } +#' @note sparkR.conf since 2.0.0 +sparkR.conf <- function(key, defaultValue) { sparkSession <- getSparkSession() - conf <- callJMethod(sparkSession, "conf") - source <- callJMethod(conf, "get", "spark.sql.sources.default", "org.apache.spark.sql.parquet") - source + if (missing(key)) { + m <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getSessionConf", sparkSession) + as.list(m, all.names = TRUE, sorted = TRUE) + } else { + conf <- callJMethod(sparkSession, "conf") + value <- if (missing(defaultValue)) { + tryCatch(callJMethod(conf, "get", key), + error = function(e) { + if (any(grep("java.util.NoSuchElementException", as.character(e)))) { + stop(paste0("Config '", key, "' is not set")) + } else { + stop(paste0("Unknown error: ", as.character(e))) + } + }) + } else { + callJMethod(conf, "get", key, defaultValue) + } + l <- setNames(list(value), key) + l + } +} + +getDefaultSqlSource <- function() { + l <- sparkR.conf("spark.sql.sources.default", "org.apache.spark.sql.parquet") + l[["spark.sql.sources.default"]] } #' Create a SparkDataFrame diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 9378c7afac8bd..74def5ce4245d 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2365,7 +2365,7 @@ test_that("randomSplit", { expect_true(all(sapply(abs(counts / num - weights / sum(weights)), function(e) { e < 0.05 }))) }) -test_that("Change config on SparkSession", { +test_that("Setting and getting config on SparkSession", { # first, set it to a random but known value conf <- callJMethod(sparkSession, "conf") property <- paste0("spark.testing.", as.character(runif(1))) @@ -2378,17 +2378,17 @@ test_that("Change config on SparkSession", { names(l) <- property sparkR.session(sparkConfig = l) - conf <- callJMethod(sparkSession, "conf") - newValue <- callJMethod(conf, "get", property, "") + newValue <- unlist(sparkR.conf(property, ""), use.names = FALSE) expect_equal(value2, newValue) value <- as.character(runif(1)) sparkR.session(spark.app.name = "sparkSession test", spark.testing.r.session.r = value) - conf <- callJMethod(sparkSession, "conf") - appNameValue <- callJMethod(conf, "get", "spark.app.name", "") - testValue <- callJMethod(conf, "get", "spark.testing.r.session.r", "") + allconf <- sparkR.conf() + appNameValue <- allconf[["spark.app.name"]] + testValue <- allconf[["spark.testing.r.session.r"]] expect_equal(appNameValue, "sparkSession test") expect_equal(testValue, value) + expect_error(sparkR.conf("completely.dummy"), "Config 'completely.dummy' is not set") }) test_that("enableHiveSupport on SparkSession", { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 0a995d2e9d180..7d8ea03a27910 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -71,6 +71,10 @@ private[sql] object SQLUtils extends Logging { } } + def getSessionConf(spark: SparkSession): JMap[String, String] = { + spark.conf.getAll.asJava + } + def getJavaSparkContext(spark: SparkSession): JavaSparkContext = { new JavaSparkContext(spark.sparkContext) } From efce6e17c3a7c2c63b9d40bd02fe4f4fec4085bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=A8=E6=B5=A9?= Date: Mon, 27 Jun 2016 08:31:52 +0100 Subject: [PATCH 0813/1470] [SPARK-16214][EXAMPLES] fix the denominator of SparkPi MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? reduce the denominator of SparkPi by 1 ## How was this patch tested? integration tests Author: 杨浩 Closes #13910 from yanghaogn/patch-1. (cherry picked from commit b452026324da20f76f7d8b78e5ba1c007712e585) Signed-off-by: Sean Owen --- examples/src/main/scala/org/apache/spark/examples/SparkPi.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 42f6cef4e189b..272c1a4fc2f47 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -36,7 +36,7 @@ object SparkPi { val y = random * 2 - 1 if (x*x + y*y < 1) 1 else 0 }.reduce(_ + _) - println("Pi is roughly " + 4.0 * count / n) + println("Pi is roughly " + 4.0 * count / (n - 1)) spark.stop() } } From ea8d419c106ad90f8f5b48e6bf897b0ff3f49f1f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 27 Jun 2016 09:23:58 +0100 Subject: [PATCH 0814/1470] [MINOR][CORE] Fix display wrong free memory size in the log ## What changes were proposed in this pull request? Free memory size displayed in the log is wrong (used memory), fix to make it correct. ## How was this patch tested? N/A Author: jerryshao Closes #13804 from jerryshao/memory-log-fix. (cherry picked from commit 52d4fe057909e8d431ae36f538dc4cafb351cdb5) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/storage/memory/MemoryStore.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 99be4de0658cc..0349da0d8aa00 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -377,7 +377,8 @@ private[spark] class MemoryStore( entries.put(blockId, entry) } logInfo("Block %s stored as bytes in memory (estimated size %s, free %s)".format( - blockId, Utils.bytesToString(entry.size), Utils.bytesToString(blocksMemoryUsed))) + blockId, Utils.bytesToString(entry.size), + Utils.bytesToString(maxMemory - blocksMemoryUsed))) Right(entry.size) } else { // We ran out of space while unrolling the values for this block From 664426e003bb83e020800798013cf5d8a68051f2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 27 Jun 2016 19:04:50 +0800 Subject: [PATCH 0815/1470] [SPARK-10591][SQL][TEST] Add a testcase to ensure if `checkAnswer` handles map correctly ## What changes were proposed in this pull request? This PR adds a testcase to ensure if `checkAnswer` handles Map type correctly. ## How was this patch tested? Pass the jenkins tests. Author: Dongjoon Hyun Closes #13913 from dongjoon-hyun/SPARK-10591. (cherry picked from commit 11f420b4bbcd607346204fb6fd7db7efe948cdac) Signed-off-by: Cheng Lian --- .../src/test/scala/org/apache/spark/sql/DatasetSuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index bd8479b2d380d..43cbc03b7aa0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -34,6 +34,13 @@ class DatasetSuite extends QueryTest with SharedSQLContext { private implicit val ordering = Ordering.by((c: ClassData) => c.a -> c.b) + test("checkAnswer should compare map correctly") { + val data = Seq((1, "2", Map(1 -> 2, 2 -> 1))) + checkAnswer( + data.toDF(), + Seq(Row(1, "2", Map(2 -> 1, 1 -> 2)))) + } + test("toDS") { val data = Seq(("a", 1), ("b", 2), ("c", 3)) checkDataset( From 22fe336c8d379ecb723c58264f8c4b98c244008a Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Mon, 27 Jun 2016 21:45:22 +0800 Subject: [PATCH 0816/1470] [SPARK-16135][SQL] Remove hashCode and euqals in ArrayBasedMapData ## What changes were proposed in this pull request? This pr is to remove `hashCode` and `equals` in `ArrayBasedMapData` because the type cannot be used as join keys, grouping keys, or in equality tests. ## How was this patch tested? Add a new test suite `MapDataSuite` for comparison tests. Author: Takeshi YAMAMURO Closes #13847 from maropu/UnsafeMapTest. (cherry picked from commit 3e4e868c850e6b6da2c0d005167316e1abdc7460) Signed-off-by: Wenchen Fan --- .../catalyst/expressions/UnsafeArrayData.java | 4 ++ .../sql/catalyst/util/ArrayBasedMapData.scala | 17 ------ .../spark/sql/catalyst/util/MapData.scala | 5 ++ .../expressions/CodeGenerationSuite.scala | 8 +-- .../expressions/ExpressionEvalHelper.scala | 8 ++- .../catalyst/expressions/MapDataSuite.scala | 57 +++++++++++++++++++ 6 files changed, 76 insertions(+), 23 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 02a863b2bb498..6302660548ec1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -298,6 +298,10 @@ public UnsafeMapData getMap(int ordinal) { return map; } + // This `hashCode` computation could consume much processor time for large data. + // If the computation becomes a bottleneck, we can use a light-weight logic; the first fixed bytes + // are used to compute `hashCode` (See `Vector.hashCode`). + // The same issue exists in `UnsafeRow.hashCode`. @Override public int hashCode() { return Murmur3_x86_32.hashUnsafeBytes(baseObject, baseOffset, sizeInBytes, 42); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala index d46f03ad8fbb3..4449da13c083c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala @@ -24,23 +24,6 @@ class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) exte override def copy(): MapData = new ArrayBasedMapData(keyArray.copy(), valueArray.copy()) - override def equals(o: Any): Boolean = { - if (!o.isInstanceOf[ArrayBasedMapData]) { - return false - } - - val other = o.asInstanceOf[ArrayBasedMapData] - if (other eq null) { - return false - } - - this.keyArray == other.keyArray && this.valueArray == other.valueArray - } - - override def hashCode: Int = { - keyArray.hashCode() * 37 + valueArray.hashCode() - } - override def toString: String = { s"keys: $keyArray, values: $valueArray" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala index 40db6067adf71..94e8824cd18cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala @@ -19,6 +19,11 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.sql.types.DataType +/** + * This is an internal data representation for map type in Spark SQL. This should not implement + * `equals` and `hashCode` because the type cannot be used as join keys, grouping keys, or + * in equality tests. See SPARK-9415 and PR#13847 for the discussions. + */ abstract class MapData extends Serializable { def numElements(): Int diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 62429a22f02fa..60dd03f5d0c13 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -110,10 +110,10 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { case (expr, i) => Seq(Literal(i), expr) })) val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) - val expected = Seq(new ArrayBasedMapData( - new GenericArrayData(0 until length), - new GenericArrayData(Seq.fill(length)(true)))) + val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)).map { + case m: ArrayBasedMapData => ArrayBasedMapData.toScalaMap(m) + } + val expected = (0 until length).map((_, true)).toMap :: Nil if (!checkResult(actual, expected)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 8a9617cfbf5df..e58a0df317fe9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.optimizer.SimpleTestOptimizer import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} +import org.apache.spark.sql.catalyst.util.MapData import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils @@ -52,7 +53,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { /** * Check the equality between result of expression and expected value, it will handle - * Array[Byte] and Spread[Double]. + * Array[Byte], Spread[Double], and MapData. */ protected def checkResult(result: Any, expected: Any): Boolean = { (result, expected) match { @@ -60,7 +61,10 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { java.util.Arrays.equals(result, expected) case (result: Double, expected: Spread[Double @unchecked]) => expected.asInstanceOf[Spread[Double]].isWithin(result) - case _ => result == expected + case (result: MapData, expected: MapData) => + result.keyArray() == expected.keyArray() && result.valueArray() == expected.valueArray() + case _ => + result == expected } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala new file mode 100644 index 0000000000000..0f1264c7c3269 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MapDataSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import scala.collection._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData +import org.apache.spark.sql.types.{DataType, IntegerType, MapType, StringType} +import org.apache.spark.unsafe.types.UTF8String + +class MapDataSuite extends SparkFunSuite { + + test("inequality tests") { + def u(str: String): UTF8String = UTF8String.fromString(str) + + // test data + val testMap1 = Map(u("key1") -> 1) + val testMap2 = Map(u("key1") -> 1, u("key2") -> 2) + val testMap3 = Map(u("key1") -> 1) + val testMap4 = Map(u("key1") -> 1, u("key2") -> 2) + + // ArrayBasedMapData + val testArrayMap1 = ArrayBasedMapData(testMap1.toMap) + val testArrayMap2 = ArrayBasedMapData(testMap2.toMap) + val testArrayMap3 = ArrayBasedMapData(testMap3.toMap) + val testArrayMap4 = ArrayBasedMapData(testMap4.toMap) + assert(testArrayMap1 !== testArrayMap3) + assert(testArrayMap2 !== testArrayMap4) + + // UnsafeMapData + val unsafeConverter = UnsafeProjection.create(Array[DataType](MapType(StringType, IntegerType))) + val row = new GenericMutableRow(1) + def toUnsafeMap(map: ArrayBasedMapData): UnsafeMapData = { + row.update(0, map) + val unsafeRow = unsafeConverter.apply(row) + unsafeRow.getMap(0).copy + } + assert(toUnsafeMap(testArrayMap1) !== toUnsafeMap(testArrayMap3)) + assert(toUnsafeMap(testArrayMap2) !== toUnsafeMap(testArrayMap4)) + } +} From f2017c59b1646811279e3104bae952c641e753f2 Mon Sep 17 00:00:00 2001 From: Bill Chambers Date: Mon, 27 Jun 2016 11:50:34 -0700 Subject: [PATCH 0817/1470] [SPARK-16220][SQL] Revert Change to Bring Back SHOW FUNCTIONS Functionality ## What changes were proposed in this pull request? - Fix tests regarding show functions functionality - Revert `catalog.ListFunctions` and `SHOW FUNCTIONS` to return to `Spark 1.X` functionality. Cherry picked changes from this PR: https://github.com/apache/spark/pull/13413/files ## How was this patch tested? Unit tests. Author: Bill Chambers Author: Bill Chambers Closes #13916 from anabranch/master. (cherry picked from commit c48c8ebc0aad433aab7af9e2ddf544d253ab9fd7) Signed-off-by: Herman van Hovell --- python/pyspark/sql/tests.py | 12 +++++++++++- .../spark/sql/catalyst/catalog/SessionCatalog.scala | 3 +-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 +---- .../org/apache/spark/sql/internal/CatalogSuite.scala | 6 ++++-- .../spark/sql/hive/execution/SQLQuerySuite.scala | 5 +++++ 5 files changed, 22 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 3dc4083704526..3f564110ed8bc 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1509,7 +1509,17 @@ def test_list_functions(self): spark.sql("CREATE DATABASE some_db") functions = dict((f.name, f) for f in spark.catalog.listFunctions()) functionsDefault = dict((f.name, f) for f in spark.catalog.listFunctions("default")) - self.assertEquals(len(functions), 0) + self.assertTrue(len(functions) > 200) + self.assertTrue("+" in functions) + self.assertTrue("like" in functions) + self.assertTrue("month" in functions) + self.assertTrue("to_unix_timestamp" in functions) + self.assertTrue("current_database" in functions) + self.assertEquals(functions["+"], Function( + name="+", + description=None, + className="org.apache.spark.sql.catalyst.expressions.Add", + isTemporary=True)) self.assertEquals(functions, functionsDefault) spark.catalog.registerFunction("temp_func", lambda x: str(x)) spark.sql("CREATE FUNCTION func1 AS 'org.apache.spark.data.bricks'") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 7ab10d1c38237..2880087b581c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -855,8 +855,7 @@ class SessionCatalog( .map { f => FunctionIdentifier(f, Some(dbName)) } val loadedFunctions = StringUtils.filterPattern(functionRegistry.listFunction(), pattern) .map { f => FunctionIdentifier(f) } - (dbFunctions ++ loadedFunctions) - .filterNot(f => FunctionRegistry.functionSet.contains(f.funcName)) + dbFunctions ++ loadedFunctions } 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 bbe821b76805c..59b25cb8d0030 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 @@ -80,13 +80,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val functions = Array("ilog", "logi", "logii", "logiii", "crc32i", "cubei", "cume_disti", "isize", "ispace", "to_datei", "date_addi", "current_datei") - assert(sql("SHOW functions").collect().isEmpty) - createFunction(functions) checkAnswer(sql("SHOW functions"), getFunctions("*")) - assert(sql("SHOW functions").collect().size === functions.size) - assert(sql("SHOW functions").collect().toSet === functions.map(Row(_)).toSet) + assert(sql("SHOW functions").collect().size > 200) Seq("^c*", "*e$", "log*", "*date*").foreach { pattern => // For the pattern part, only '*' and '|' are allowed as wildcards. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index df817f863dbe2..d862e4cfa943a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -174,7 +174,8 @@ class CatalogSuite } test("list functions") { - assert(spark.catalog.listFunctions().collect().isEmpty) + assert(Set("+", "current_database", "window").subsetOf( + spark.catalog.listFunctions().collect().map(_.name).toSet)) createFunction("my_func1") createFunction("my_func2") createTempFunction("my_temp_func") @@ -191,7 +192,8 @@ class CatalogSuite } test("list functions with database") { - assert(spark.catalog.listFunctions("default").collect().isEmpty) + assert(Set("+", "current_database", "window").subsetOf( + spark.catalog.listFunctions().collect().map(_.name).toSet)) createDatabase("my_db1") createDatabase("my_db2") createFunction("my_func1", Some("my_db1")) 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 9c1f21825315b..e8af4fbe876e1 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 @@ -187,6 +187,11 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("show functions") { + val allBuiltinFunctions = FunctionRegistry.builtin.listFunction().toSet[String].toList.sorted + val allFunctions = sql("SHOW functions").collect().map(r => r(0)) + allBuiltinFunctions.foreach { f => + assert(allFunctions.contains(f)) + } withTempDatabase { db => def createFunction(names: Seq[String]): Unit = { names.foreach { name => From e4bb31fb3afeaf6b6ddc1af4c9c07f1f7001b7cc Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Mon, 27 Jun 2016 12:27:39 -0700 Subject: [PATCH 0818/1470] [SPARK-16187][ML] Implement util method for ML Matrix conversion in scala/java ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-16187 This is to provide conversion utils between old/new vector columns in a DataFrame. So users can use it to migrate their datasets and pipelines manually. ## How was this patch tested? java and scala ut Author: Yuhao Yang Closes #13888 from hhbyyh/matComp. (cherry picked from commit c17b1abff8f8c6d24cb0cf4ff4f8c14a780c64b0) Signed-off-by: Xiangrui Meng --- .../apache/spark/ml/linalg/MatrixUDT.scala | 2 +- .../org/apache/spark/mllib/util/MLUtils.scala | 107 +++++++++++++++++- .../spark/mllib/util/JavaMLUtilsSuite.java | 29 ++++- .../spark/mllib/util/MLUtilsSuite.scala | 56 ++++++++- 4 files changed, 187 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala index 521a216c6785d..a1e53662f02a8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types._ * User-defined type for [[Matrix]] in [[mllib-local]] which allows easy interaction with SQL * via [[org.apache.spark.sql.Dataset]]. */ -private[ml] class MatrixUDT extends UserDefinedType[Matrix] { +private[spark] class MatrixUDT extends UserDefinedType[Matrix] { override def sqlType: StructType = { // type: 0 = sparse, 1 = dense diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 7d5bdffc42ec8..e96c2bc6edfc3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.linalg.{VectorUDT => MLVectorUDT} +import org.apache.spark.ml.linalg.{MatrixUDT => MLMatrixUDT, VectorUDT => MLVectorUDT} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.regression.LabeledPoint @@ -309,8 +309,8 @@ object MLUtils extends Logging { } /** - * Converts vector columns in an input Dataset to the [[org.apache.spark.ml.linalg.Vector]] type - * from the new [[org.apache.spark.mllib.linalg.Vector]] type under the `spark.ml` package. + * Converts vector columns in an input Dataset to the [[org.apache.spark.mllib.linalg.Vector]] + * type from the new [[org.apache.spark.ml.linalg.Vector]] type under the `spark.ml` package. * @param dataset input dataset * @param cols a list of vector columns to be converted. Old vector columns will be ignored. If * unspecified, all new vector columns will be converted except nested ones. @@ -360,6 +360,107 @@ object MLUtils extends Logging { dataset.select(exprs: _*) } + /** + * Converts Matrix columns in an input Dataset from the [[org.apache.spark.mllib.linalg.Matrix]] + * type to the new [[org.apache.spark.ml.linalg.Matrix]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of matrix columns to be converted. New matrix columns will be ignored. If + * unspecified, all old matrix columns will be converted except nested ones. + * @return the input [[DataFrame]] with old matrix columns converted to the new matrix type + */ + @Since("2.0.0") + @varargs + def convertMatrixColumnsToML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[MatrixUDT]) { + Some(c) + } else { + // ignore new matrix columns and raise an exception on other column types + require(dataType.getClass == classOf[MLMatrixUDT], + s"Column $c must be old Matrix type to be converted to new type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[MatrixUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Matrix column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + val convertToML = udf { v: Matrix => v.asML } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertToML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + + /** + * Converts matrix columns in an input Dataset to the [[org.apache.spark.mllib.linalg.Matrix]] + * type from the new [[org.apache.spark.ml.linalg.Matrix]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of matrix columns to be converted. Old matrix columns will be ignored. If + * unspecified, all new matrix columns will be converted except nested ones. + * @return the input [[DataFrame]] with new matrix columns converted to the old matrix type + */ + @Since("2.0.0") + @varargs + def convertMatrixColumnsFromML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[MLMatrixUDT]) { + Some(c) + } else { + // ignore old matrix columns and raise an exception on other column types + require(dataType.getClass == classOf[MatrixUDT], + s"Column $c must be new Matrix type to be converted to old type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[MLMatrixUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Matrix column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + val convertFromML = udf { Matrices.fromML _ } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertFromML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + + /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: diff --git a/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java index 2fa0bd2546594..e271a0a77c782 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java @@ -17,18 +17,22 @@ package org.apache.spark.mllib.util; +import java.util.Arrays; import java.util.Collections; import org.junit.Assert; import org.junit.Test; import org.apache.spark.SharedSparkSession; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.linalg.*; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; public class JavaMLUtilsSuite extends SharedSparkSession { @@ -46,4 +50,25 @@ public void testConvertVectorColumnsToAndFromML() { Row old1 = MLUtils.convertVectorColumnsFromML(newDataset1).first(); Assert.assertEquals(RowFactory.create(1.0, x), old1); } + + @Test + public void testConvertMatrixColumnsToAndFromML() { + Matrix x = Matrices.dense(2, 1, new double[]{1.0, 2.0}); + StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("features", new MatrixUDT(), false, Metadata.empty()) + }); + Dataset dataset = spark.createDataFrame( + Arrays.asList( + RowFactory.create(1.0, x)), + schema); + + Dataset newDataset1 = MLUtils.convertMatrixColumnsToML(dataset); + Row new1 = newDataset1.first(); + Assert.assertEquals(RowFactory.create(1.0, x.asML()), new1); + Row new2 = MLUtils.convertMatrixColumnsToML(dataset, "features").first(); + Assert.assertEquals(new1, new2); + Row old1 = MLUtils.convertMatrixColumnsFromML(newDataset1).first(); + Assert.assertEquals(RowFactory.create(1.0, x), old1); + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 3801bd127a5f7..6aa93c9076007 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -26,7 +26,7 @@ import breeze.linalg.{squaredDistance => breezeSquaredDistance} import com.google.common.io.Files import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, Matrices, SparseVector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ import org.apache.spark.mllib.util.TestingUtils._ @@ -301,4 +301,58 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { convertVectorColumnsFromML(df, "p._2") } } + + test("convertMatrixColumnsToML") { + val x = Matrices.sparse(3, 2, Array(0, 2, 3), Array(0, 2, 1), Array(0.0, -1.2, 0.0)) + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Matrices.dense(2, 1, Array(0.2, 1.3)) + val z = Matrices.ones(1, 1) + val p = (5.0, z) + val w = Matrices.dense(1, 1, Array(4.5)).asML + val df = spark.createDataFrame(Seq( + (0, x, y, p, w) + )).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertMatrixColumnsToML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, x.asML, y.asML, Row(5.0, z), w)) + val new2 = convertMatrixColumnsToML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertMatrixColumnsToML(df, "y", "w").first() + assert(new3 === Row(0, x, y.asML, Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertMatrixColumnsToML(df, "p") + } + intercept[IllegalArgumentException] { + convertMatrixColumnsToML(df, "p._2") + } + } + + test("convertMatrixColumnsFromML") { + val x = Matrices.sparse(3, 2, Array(0, 2, 3), Array(0, 2, 1), Array(0.0, -1.2, 0.0)).asML + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Matrices.dense(2, 1, Array(0.2, 1.3)).asML + val z = Matrices.ones(1, 1).asML + val p = (5.0, z) + val w = Matrices.dense(1, 1, Array(4.5)) + val df = spark.createDataFrame(Seq( + (0, x, y, p, w) + )).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertMatrixColumnsFromML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, Matrices.fromML(x), Matrices.fromML(y), Row(5.0, z), w)) + val new2 = convertMatrixColumnsFromML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertMatrixColumnsFromML(df, "y", "w").first() + assert(new3 === Row(0, x, Matrices.fromML(y), Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertMatrixColumnsFromML(df, "p") + } + intercept[IllegalArgumentException] { + convertMatrixColumnsFromML(df, "p._2") + } + } } From 27f3462d0e11b4768140e452f02ab043438b8e86 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Mon, 27 Jun 2016 12:58:39 -0700 Subject: [PATCH 0819/1470] [SPARK-16231][PYSPARK][ML][EXAMPLES] dataframe_example.py fails to convert ML style vectors ## What changes were proposed in this pull request? Need to convert ML Vectors to the old MLlib style before doing Statistics.colStats operations on the DataFrame ## How was this patch tested? Ran example, local tests Author: Bryan Cutler Closes #13928 from BryanCutler/pyspark-ml-example-vector-conv-SPARK-16231. (cherry picked from commit 1aa191e58e905f470f73663fc1c35f36e05e929a) Signed-off-by: Xiangrui Meng --- examples/src/main/python/ml/dataframe_example.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py index a7d8b9056dffe..c1818d72fe467 100644 --- a/examples/src/main/python/ml/dataframe_example.py +++ b/examples/src/main/python/ml/dataframe_example.py @@ -28,6 +28,7 @@ from pyspark.sql import SparkSession from pyspark.mllib.stat import Statistics +from pyspark.mllib.util import MLUtils if __name__ == "__main__": if len(sys.argv) > 2: @@ -55,7 +56,8 @@ labelSummary.show() # Convert features column to an RDD of vectors. - features = df.select("features").rdd.map(lambda r: r.features) + features = MLUtils.convertVectorColumnsFromML(df, "features") \ + .select("features").rdd.map(lambda r: r.features) summary = Statistics.colStats(features) print("Selected features column with average values:\n" + str(summary.mean())) From c7704099da6c2b949a17abb3a75b7b4e01ff869a Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 27 Jun 2016 16:57:34 -0700 Subject: [PATCH 0820/1470] [SPARK-16220][SQL] Add scope to show functions ## What changes were proposed in this pull request? Spark currently shows all functions when issue a `SHOW FUNCTIONS` command. This PR refines the `SHOW FUNCTIONS` command by allowing users to select all functions, user defined function or system functions. The following syntax can be used: **ALL** (default) ```SHOW FUNCTIONS``` ```SHOW ALL FUNCTIONS``` **SYSTEM** ```SHOW SYSTEM FUNCTIONS``` **USER** ```SHOW USER FUNCTIONS``` ## How was this patch tested? Updated tests and added tests to the DDLSuite Author: Herman van Hovell Closes #13929 from hvanhovell/SPARK-16220. (cherry picked from commit 02a029df43392c5d73697203bf6ff51b8d6efb83) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 ++- .../sql/catalyst/catalog/SessionCatalog.scala | 20 +++++++++++++------ .../catalog/SessionCatalogSuite.scala | 6 +++--- .../spark/sql/execution/SparkSqlParser.scala | 17 ++++++++++++---- .../sql/execution/command/functions.scala | 12 ++++++++--- .../spark/sql/internal/CatalogImpl.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../sql/execution/SparkSqlParserSuite.scala | 15 ++++++++++---- .../sql/execution/command/DDLSuite.scala | 17 +++++++++++++++- 9 files changed, 70 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 23e925e4de63c..4c15f9cec6573 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -106,7 +106,8 @@ statement | SHOW COLUMNS (FROM | IN) tableIdentifier ((FROM | IN) db=identifier)? #showColumns | SHOW PARTITIONS tableIdentifier partitionSpec? #showPartitions - | SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions + | SHOW identifier? FUNCTIONS + (LIKE? (qualifiedName | pattern=STRING))? #showFunctions | SHOW CREATE TABLE tableIdentifier #showCreateTable | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction | (DESC | DESCRIBE) DATABASE EXTENDED? identifier #describeDatabase diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 2880087b581c7..8c620d36e5679 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -841,21 +841,29 @@ class SessionCatalog( } /** - * List all functions in the specified database, including temporary functions. + * List all functions in the specified database, including temporary functions. This + * returns the function identifier and the scope in which it was defined (system or user + * defined). */ - def listFunctions(db: String): Seq[FunctionIdentifier] = listFunctions(db, "*") + def listFunctions(db: String): Seq[(FunctionIdentifier, String)] = listFunctions(db, "*") /** - * List all matching functions in the specified database, including temporary functions. + * List all matching functions in the specified database, including temporary functions. This + * returns the function identifier and the scope in which it was defined (system or user + * defined). */ - def listFunctions(db: String, pattern: String): Seq[FunctionIdentifier] = { + def listFunctions(db: String, pattern: String): Seq[(FunctionIdentifier, String)] = { val dbName = formatDatabaseName(db) requireDbExists(dbName) val dbFunctions = externalCatalog.listFunctions(dbName, pattern) .map { f => FunctionIdentifier(f, Some(dbName)) } val loadedFunctions = StringUtils.filterPattern(functionRegistry.listFunction(), pattern) .map { f => FunctionIdentifier(f) } - dbFunctions ++ loadedFunctions + val functions = dbFunctions ++ loadedFunctions + functions.map { + case f if FunctionRegistry.functionSet.contains(f.funcName) => (f, "SYSTEM") + case f => (f, "USER") + } } @@ -877,7 +885,7 @@ class SessionCatalog( listTables(default).foreach { table => dropTable(table, ignoreIfNotExists = false) } - listFunctions(default).foreach { func => + listFunctions(default).map(_._1).foreach { func => if (func.database.isDefined) { dropFunction(func, ignoreIfNotExists = false) } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index a4dc03cd8b260..c8e7c5103b6eb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -950,16 +950,16 @@ class SessionCatalogSuite extends SparkFunSuite { catalog.createFunction(newFunc("not_me", Some("db2")), ignoreIfExists = false) catalog.createTempFunction("func1", info1, tempFunc1, ignoreIfExists = false) catalog.createTempFunction("yes_me", info2, tempFunc2, ignoreIfExists = false) - assert(catalog.listFunctions("db1", "*").toSet == + assert(catalog.listFunctions("db1", "*").map(_._1).toSet == Set(FunctionIdentifier("func1"), FunctionIdentifier("yes_me"))) - assert(catalog.listFunctions("db2", "*").toSet == + assert(catalog.listFunctions("db2", "*").map(_._1).toSet == Set(FunctionIdentifier("func1"), FunctionIdentifier("yes_me"), FunctionIdentifier("func1", Some("db2")), FunctionIdentifier("func2", Some("db2")), FunctionIdentifier("not_me", Some("db2")))) - assert(catalog.listFunctions("db2", "func*").toSet == + assert(catalog.listFunctions("db2", "func*").map(_._1).toSet == Set(FunctionIdentifier("func1"), FunctionIdentifier("func1", Some("db2")), FunctionIdentifier("func2", Some("db2")))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 066ff57721a3d..42ec210baa2d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -538,14 +538,23 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { */ override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { import ctx._ - if (qualifiedName != null) { + val (user, system) = Option(ctx.identifier).map(_.getText.toLowerCase) match { + case None | Some("all") => (true, true) + case Some("system") => (false, true) + case Some("user") => (true, false) + case Some(x) => throw new ParseException(s"SHOW $x FUNCTIONS not supported", ctx) + } + + val (db, pat) = if (qualifiedName != null) { val name = visitFunctionName(qualifiedName) - ShowFunctionsCommand(name.database, Some(name.funcName)) + (name.database, Some(name.funcName)) } else if (pattern != null) { - ShowFunctionsCommand(None, Some(string(pattern))) + (None, Some(string(pattern))) } else { - ShowFunctionsCommand(None, None) + (None, None) } + + ShowFunctionsCommand(db, pat, user, system) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index d2d8e3ddeae26..26593d2918a6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -181,8 +181,11 @@ case class DropFunctionCommand( * '|' is for alternation. * For example, "show functions like 'yea*|windo*'" will return "window" and "year". */ -case class ShowFunctionsCommand(db: Option[String], pattern: Option[String]) - extends RunnableCommand { +case class ShowFunctionsCommand( + db: Option[String], + pattern: Option[String], + showUserFunctions: Boolean, + showSystemFunctions: Boolean) extends RunnableCommand { override val output: Seq[Attribute] = { val schema = StructType(StructField("function", StringType, nullable = false) :: Nil) @@ -196,7 +199,10 @@ case class ShowFunctionsCommand(db: Option[String], pattern: Option[String]) val functionNames = sparkSession.sessionState.catalog .listFunctions(dbName, pattern.getOrElse("*")) - .map(_.unquotedString) + .collect { + case (f, "USER") if showUserFunctions => f.unquotedString + case (f, "SYSTEM") if showSystemFunctions => f.unquotedString + } // The session catalog caches some persistent functions in the FunctionRegistry // so there can be duplicates. functionNames.distinct.sorted.map(Row(_)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 601334b97add3..44babcc93a1de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -121,7 +121,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { @throws[AnalysisException]("database does not exist") override def listFunctions(dbName: String): Dataset[Function] = { requireDatabaseExists(dbName) - val functions = sessionCatalog.listFunctions(dbName).map { funcIdent => + val functions = sessionCatalog.listFunctions(dbName).map { case (funcIdent, _) => val metadata = sessionCatalog.lookupFunctionInfo(funcIdent) new Function( name = funcIdent.identifier, 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 59b25cb8d0030..b1dbf21d4b80f 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 @@ -61,7 +61,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("show functions") { def getFunctions(pattern: String): Seq[Row] = { StringUtils.filterPattern( - spark.sessionState.catalog.listFunctions("default").map(_.funcName), pattern) + spark.sessionState.catalog.listFunctions("default").map(_._1.funcName), pattern) .map(Row(_)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index e2858bb475401..8161c08b2cb48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -46,10 +46,17 @@ class SparkSqlParserSuite extends PlanTest { } test("show functions") { - assertEqual("show functions", ShowFunctionsCommand(None, None)) - assertEqual("show functions foo", ShowFunctionsCommand(None, Some("foo"))) - assertEqual("show functions foo.bar", ShowFunctionsCommand(Some("foo"), Some("bar"))) - assertEqual("show functions 'foo\\\\.*'", ShowFunctionsCommand(None, Some("foo\\.*"))) + assertEqual("show functions", ShowFunctionsCommand(None, None, true, true)) + assertEqual("show all functions", ShowFunctionsCommand(None, None, true, true)) + assertEqual("show user functions", ShowFunctionsCommand(None, None, true, false)) + assertEqual("show system functions", ShowFunctionsCommand(None, None, false, true)) + intercept("show special functions", "SHOW special FUNCTIONS") + assertEqual("show functions foo", + ShowFunctionsCommand(None, Some("foo"), true, true)) + assertEqual("show functions foo.bar", + ShowFunctionsCommand(Some("foo"), Some("bar"), true, true)) + assertEqual("show functions 'foo\\\\.*'", + ShowFunctionsCommand(None, Some("foo\\.*"), true, true)) intercept("show functions foo.bar.baz", "Unsupported function name") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 47d8a28f49927..0ee8d179d79eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{DatabaseAlreadyExistsException, FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, SessionCatalog} @@ -1370,4 +1370,19 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { ) } } + + test("show functions") { + withUserDefinedFunction("add_one" -> true) { + val numFunctions = FunctionRegistry.functionSet.size.toLong + assert(sql("show functions").count() === numFunctions) + assert(sql("show system functions").count() === numFunctions) + assert(sql("show all functions").count() === numFunctions) + assert(sql("show user functions").count() === 0L) + spark.udf.register("add_one", (x: Long) => x + 1) + assert(sql("show functions").count() === numFunctions + 1L) + assert(sql("show system functions").count() === numFunctions) + assert(sql("show all functions").count() === numFunctions + 1L) + assert(sql("show user functions").count() === 1L) + } + } } From 4c4f7775cbf5dd69e688350ee59a9319bcaa56fe Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Mon, 27 Jun 2016 20:32:27 -0700 Subject: [PATCH 0821/1470] [SPARK-16143][R] group AFT survival regression methods docs in a single Rd ## What changes were proposed in this pull request? This PR groups `spark.survreg`, `summary(AFT)`, `predict(AFT)`, `write.ml(AFT)` for survival regression into a single Rd. ## How was this patch tested? Manually checked generated HTML doc. See attached screenshots. ![screen shot 2016-06-27 at 10 28 20 am](https://cloud.githubusercontent.com/assets/15318264/16392008/a14cf472-3c5e-11e6-9ce5-490ed1a52249.png) ![screen shot 2016-06-27 at 10 28 35 am](https://cloud.githubusercontent.com/assets/15318264/16392009/a14e333c-3c5e-11e6-8bd7-c2e9ba71f8e2.png) Author: Junyang Qian Closes #13927 from junyangq/SPARK-16143. (cherry picked from commit 1b7fc5817203db5a56489b289fb1a0dd44b2e26b) Signed-off-by: Xiangrui Meng --- R/pkg/R/mllib.R | 88 +++++++++++++++++++++++-------------------------- 1 file changed, 42 insertions(+), 46 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 853cfce74ae56..8e6c2ddf93cfc 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -233,9 +233,10 @@ setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), # Makes predictions from a naive Bayes model or a model produced by spark.naiveBayes(), # similarly to R package e1071's predict. -#' @rdname spark.naiveBayes +#' @param newData A SparkDataFrame for testing #' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named #' "prediction" +#' @rdname spark.naiveBayes #' @export #' @note predict(NaiveBayesModel) since 2.0.0 setMethod("predict", signature(object = "NaiveBayesModel"), @@ -439,25 +440,16 @@ setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), invisible(callJMethod(writer, "save", path)) }) -#' Save fitted MLlib model to the input path -#' -#' Save the AFT survival regression model to the input path. -#' -#' @param object A fitted AFT survival regression model -#' @param path The directory where the model is saved -#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +# Saves the AFT survival regression model to the input path. + +#' @param path The directory where the model is savedist containing the model's coefficien #' which means throw exception if the output path exists. #' -#' @rdname write.ml +#' @rdname spark.survreg #' @name write.ml #' @export -#' @examples -#' \dontrun{ -#' model <- spark.survreg(trainingData, Surv(futime, fustat) ~ ecog_ps + rx) -#' path <- "path/to/model" -#' write.ml(model, path) -#' } #' @note write.ml(AFTSurvivalRegressionModel, character) since 2.0.0 +#' @seealso \link{read.ml} setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "character"), function(object, path, overwrite = FALSE) { writer <- callJMethod(object@jobj, "write") @@ -542,15 +534,18 @@ read.ml <- function(path) { } } -#' Fit an accelerated failure time (AFT) survival regression model. +#' Accelerated Failure Time (AFT) Survival Regression Model #' -#' Fit an accelerated failure time (AFT) survival regression model on a Spark DataFrame. +#' \code{spark.survreg} fits an accelerated failure time (AFT) survival regression model on +#' a SparkDataFrame. Users can call \code{summary} to get a summary of the fitted AFT model, +#' \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} to +#' save/load fitted models. #' -#' @param data SparkDataFrame for training. +#' @param data A SparkDataFrame for training #' @param formula A symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', ':', '+', and '-'. -#' Note that operator '.' is not supported currently. -#' @return a fitted AFT survival regression model +#' Note that operator '.' is not supported currently +#' @return \code{spark.survreg} returns a fitted AFT survival regression model #' @rdname spark.survreg #' @seealso survival: \url{https://cran.r-project.org/web/packages/survival/} #' @export @@ -558,6 +553,19 @@ read.ml <- function(path) { #' \dontrun{ #' df <- createDataFrame(ovarian) #' model <- spark.survreg(df, Surv(futime, fustat) ~ ecog_ps + rx) +#' +#' # get a summary of the model +#' summary(model) +#' +#' # make predictions +#' predicted <- predict(model, df) +#' showDF(predicted) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) #' } #' @note spark.survreg since 2.0.0 setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula"), @@ -569,20 +577,14 @@ setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula }) -#' Get the summary of an AFT survival regression model -#' -#' Returns the summary of an AFT survival regression model produced by spark.survreg(), -#' similarly to R's summary(). -#' -#' @param object a fitted AFT survival regression model -#' @return coefficients the model's coefficients, intercept and log(scale). -#' @rdname summary +# Returns a summary of the AFT survival regression model produced by spark.survreg, +# similarly to R's summary(). + +#' @param object A fitted AFT survival regression model +#' @return \code{summary} returns a list containing the model's coefficients, +#' intercept and log(scale) +#' @rdname spark.survreg #' @export -#' @examples -#' \dontrun{ -#' model <- spark.survreg(trainingData, Surv(futime, fustat) ~ ecog_ps + rx) -#' summary(model) -#' } #' @note summary(AFTSurvivalRegressionModel) since 2.0.0 setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), function(object, ...) { @@ -595,20 +597,14 @@ setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), return(list(coefficients = coefficients)) }) -#' Predicted values based on model -#' -#' Makes predictions from an AFT survival regression model or a model produced by spark.survreg(), -#' similarly to R package survival's predict. -#' -#' @param object A fitted AFT survival regression model -#' @rdname predict +# Makes predictions from an AFT survival regression model or a model produced by +# spark.survreg, similarly to R package survival's predict. + +#' @param newData A SparkDataFrame for testing +#' @return \code{predict} returns a SparkDataFrame containing predicted values +#' on the original scale of the data (mean predicted value at scale = 1.0) +#' @rdname spark.survreg #' @export -#' @examples -#' \dontrun{ -#' model <- spark.survreg(trainingData, Surv(futime, fustat) ~ ecog_ps + rx) -#' predicted <- predict(model, testData) -#' showDF(predicted) -#' } #' @note predict(AFTSurvivalRegressionModel) since 2.0.0 setMethod("predict", signature(object = "AFTSurvivalRegressionModel"), function(object, newData) { From 7177e184368393b1f5ecd246635168ffc10bf91e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 27 Jun 2016 21:58:16 -0700 Subject: [PATCH 0822/1470] [SPARK-16111][SQL][DOC] Hide SparkOrcNewRecordReader in API docs ## What changes were proposed in this pull request? Currently, Spark Scala/Java API documents shows **org.apache.hadoop.hive.ql.io.orc** package at the top. http://spark.apache.org/docs/2.0.0-preview/api/scala/index.html#org.apache.spark.package http://spark.apache.org/docs/2.0.0-preview/api/java/index.html This PR hides `SparkOrcNewRecordReader` from API docs. ## How was this patch tested? Manual. (`build/sbt unidoc`). The following is the screenshot after this PR. **Scala API doc** ![Scala API doc](https://app.box.com/representation/file_version_75673952621/image_2048/1.png?shared_name=2mdqydygs8le6q9x00356898662zjwz6) **Java API doc** ![Java API doc](https://app.box.com/representation/file_version_75673951725/image_2048/1.png?shared_name=iv23eeqy3avvkqz203v9ygfaqeyml85j) Author: Dongjoon Hyun Closes #13914 from dongjoon-hyun/SPARK-16111. (cherry picked from commit 50fdd866b55cb9b51427095e56b2aafea12a7c23) Signed-off-by: Reynold Xin --- project/SparkBuild.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4b44469576126..4c01ad3c33712 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -720,6 +720,7 @@ object Unidoc { // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { ignoreUndocumentedPackages((unidocAllSources in (JavaUnidoc, unidoc)).value) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hadoop"))) }, // Javadoc options: create a window title, and group key packages on index page @@ -733,7 +734,8 @@ object Unidoc { unidocSourceBase := s"https://github.com/apache/spark/tree/v${version.value}", scalacOptions in (ScalaUnidoc, unidoc) ++= Seq( - "-groups" // Group similar methods together based on the @group annotation. + "-groups", // Group similar methods together based on the @group annotation. + "-skip-packages", "org.apache.hadoop" ) ++ ( // Add links to sources when generating Scaladoc for a non-snapshot release if (!isSnapshot.value) { From af70ad02859900e8c890e38b6fec0d12d42461f2 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 27 Jun 2016 22:44:08 -0700 Subject: [PATCH 0823/1470] [SPARK-15863][SQL][DOC][FOLLOW-UP] Update SQL programming guide. ## What changes were proposed in this pull request? This PR makes several updates to SQL programming guide. Author: Yin Huai Closes #13938 from yhuai/doc. (cherry picked from commit dd6b7dbe7043f3fa3d2e3993d2e13f87231a59ca) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 34 ++++++++++++++++------------------ 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4b52c942e5449..6c6bc8db6a1ff 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -25,29 +25,35 @@ the `spark-shell`, `pyspark` shell, or `sparkR` shell. One use of Spark SQL is to execute SQL queries. Spark SQL can also be used to read data from an existing Hive installation. For more on how to configure this feature, please refer to the [Hive Tables](#hive-tables) section. When running -SQL from within another programming language the results will be returned as a [DataFrame](#datasets-and-dataframes). +SQL from within another programming language the results will be returned as a [Dataset/DataFrame](#datasets-and-dataframes). You can also interact with the SQL interface using the [command-line](#running-the-spark-sql-cli) or over [JDBC/ODBC](#running-the-thrift-jdbcodbc-server). ## Datasets and DataFrames -A Dataset is a new interface added in Spark 1.6 that tries to provide the benefits of RDDs (strong +A Dataset is a distributed collection of data. +Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong typing, ability to use powerful lambda functions) with the benefits of Spark SQL's optimized execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then manipulated using functional transformations (`map`, `flatMap`, `filter`, etc.). +The Dataset API is available in [Scala][scala-datasets] and +[Java][java-datasets]. Python does not have the support for the Dataset API. But due to Python's dynamic nature, +many of the benefits of the Dataset API are already available (i.e. you can access the field of a row by name naturally +`row.columnName`). The case for R is similar. -The Dataset API is the successor of the DataFrame API, which was introduced in Spark 1.3. In Spark -2.0, Datasets and DataFrames are unified, and DataFrames are now equivalent to Datasets of `Row`s. -In fact, `DataFrame` is simply a type alias of `Dataset[Row]` in [the Scala API][scala-datasets]. -However, [Java API][java-datasets] users must use `Dataset` instead. +A DataFrame is a *Dataset* organized into named columns. It is conceptually +equivalent to a table in a relational database or a data frame in R/Python, but with richer +optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such +as: structured data files, tables in Hive, external databases, or existing RDDs. +The DataFrame API is available in Scala, +Java, [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html). +In Scala and Java, a DataFrame is represented by a Dataset of `Row`s. +In [the Scala API][scala-datasets], `DataFrame` is simply a type alias of `Dataset[Row]`. +While, in [Java API][java-datasets], users need to use `Dataset` to represent a `DataFrame`. [scala-datasets]: api/scala/index.html#org.apache.spark.sql.Dataset [java-datasets]: api/java/index.html?org/apache/spark/sql/Dataset.html -Python does not have support for the Dataset API, but due to its dynamic nature many of the -benefits are already available (i.e. you can access the field of a row by name naturally -`row.columnName`). The case for R is similar. - Throughout this document, we will often refer to Scala/Java Datasets of `Row`s as DataFrames. # Getting Started @@ -2042,14 +2048,6 @@ that these options will be deprecated in future release as more optimizations ar ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. - - - - - From b349237e4b4cf60fccf9bfcf76deca78f1224bf1 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 28 Jun 2016 06:28:22 -0700 Subject: [PATCH 0824/1470] [SPARK-16242][MLLIB][PYSPARK] Conversion between old/new matrix columns in a DataFrame (Python) ## What changes were proposed in this pull request? This PR implements python wrappers for #13888 to convert old/new matrix columns in a DataFrame. ## How was this patch tested? Doctest in python. Author: Yanbo Liang Closes #13935 from yanboliang/spark-16242. (cherry picked from commit e158478a9fff5e63ae0336a54b3f360d0cd38921) Signed-off-by: Yanbo Liang --- .../mllib/api/python/PythonMLLibAPI.scala | 14 ++++ python/pyspark/mllib/util.py | 80 +++++++++++++++++++ 2 files changed, 94 insertions(+) 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 f2c70baf472a7..f4819f77ebdb8 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 @@ -1215,6 +1215,20 @@ private[python] class PythonMLLibAPI extends Serializable { def convertVectorColumnsFromML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { MLUtils.convertVectorColumnsFromML(dataset, cols.asScala: _*) } + + /** + * Python-friendly version of [[MLUtils.convertMatrixColumnsToML()]]. + */ + def convertMatrixColumnsToML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertMatrixColumnsToML(dataset, cols.asScala: _*) + } + + /** + * Python-friendly version of [[MLUtils.convertMatrixColumnsFromML()]] + */ + def convertMatrixColumnsFromML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertMatrixColumnsFromML(dataset, cols.asScala: _*) + } } /** diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index a7e6bcc754dc7..48867a08dbfad 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -281,6 +281,86 @@ def convertVectorColumnsFromML(dataset, *cols): raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset))) return callMLlibFunc("convertVectorColumnsFromML", dataset, list(cols)) + @staticmethod + @since("2.0.0") + def convertMatrixColumnsToML(dataset, *cols): + """ + Converts matrix columns in an input DataFrame from the + :py:class:`pyspark.mllib.linalg.Matrix` type to the new + :py:class:`pyspark.ml.linalg.Matrix` type under the `spark.ml` + package. + + :param dataset: + input dataset + :param cols: + a list of matrix columns to be converted. + New matrix columns will be ignored. If unspecified, all old + matrix columns will be converted excepted nested ones. + :return: + the input dataset with old matrix columns converted to the + new matrix type + + >>> import pyspark + >>> from pyspark.mllib.linalg import Matrices + >>> from pyspark.mllib.util import MLUtils + >>> df = spark.createDataFrame( + ... [(0, Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]), + ... Matrices.dense(2, 2, range(4)))], ["id", "x", "y"]) + >>> r1 = MLUtils.convertMatrixColumnsToML(df).first() + >>> isinstance(r1.x, pyspark.ml.linalg.SparseMatrix) + True + >>> isinstance(r1.y, pyspark.ml.linalg.DenseMatrix) + True + >>> r2 = MLUtils.convertMatrixColumnsToML(df, "x").first() + >>> isinstance(r2.x, pyspark.ml.linalg.SparseMatrix) + True + >>> isinstance(r2.y, pyspark.mllib.linalg.DenseMatrix) + True + """ + if not isinstance(dataset, DataFrame): + raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset))) + return callMLlibFunc("convertMatrixColumnsToML", dataset, list(cols)) + + @staticmethod + @since("2.0.0") + def convertMatrixColumnsFromML(dataset, *cols): + """ + Converts matrix columns in an input DataFrame to the + :py:class:`pyspark.mllib.linalg.Matrix` type from the new + :py:class:`pyspark.ml.linalg.Matrix` type under the `spark.ml` + package. + + :param dataset: + input dataset + :param cols: + a list of matrix columns to be converted. + Old matrix columns will be ignored. If unspecified, all new + matrix columns will be converted except nested ones. + :return: + the input dataset with new matrix columns converted to the + old matrix type + + >>> import pyspark + >>> from pyspark.ml.linalg import Matrices + >>> from pyspark.mllib.util import MLUtils + >>> df = spark.createDataFrame( + ... [(0, Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]), + ... Matrices.dense(2, 2, range(4)))], ["id", "x", "y"]) + >>> r1 = MLUtils.convertMatrixColumnsFromML(df).first() + >>> isinstance(r1.x, pyspark.mllib.linalg.SparseMatrix) + True + >>> isinstance(r1.y, pyspark.mllib.linalg.DenseMatrix) + True + >>> r2 = MLUtils.convertMatrixColumnsFromML(df, "x").first() + >>> isinstance(r2.x, pyspark.mllib.linalg.SparseMatrix) + True + >>> isinstance(r2.y, pyspark.ml.linalg.DenseMatrix) + True + """ + if not isinstance(dataset, DataFrame): + raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset))) + return callMLlibFunc("convertMatrixColumnsFromML", dataset, list(cols)) + class Saveable(object): """ From 4c5e16f58043b3103bbd59c5fa8fec4c411e5e11 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 28 Jun 2016 07:54:44 -0700 Subject: [PATCH 0825/1470] [SPARK-16224] [SQL] [PYSPARK] SparkSession builder's configs need to be set to the existing Scala SparkContext's SparkConf ## What changes were proposed in this pull request? When we create a SparkSession at the Python side, it is possible that a SparkContext has been created. For this case, we need to set configs of the SparkSession builder to the Scala SparkContext's SparkConf (we need to do so because conf changes on a active Python SparkContext will not be propagated to the JVM side). Otherwise, we may create a wrong SparkSession (e.g. Hive support is not enabled even if enableHiveSupport is called). ## How was this patch tested? New tests and manual tests. Author: Yin Huai Closes #13931 from yhuai/SPARK-16224. (cherry picked from commit 0923c4f5676691e28e70ecb05890e123540b91f0) Signed-off-by: Davies Liu --- python/pyspark/context.py | 2 ++ python/pyspark/sql/session.py | 7 ++++++ python/pyspark/sql/tests.py | 43 ++++++++++++++++++++++++++++++++++- python/pyspark/tests.py | 8 +++++++ 4 files changed, 59 insertions(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 7217a9907a482..6e9f24ef1026b 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -166,6 +166,8 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, # Create the Java SparkContext through Py4J self._jsc = jsc or self._initialize_context(self._conf._jconf) + # Reset the SparkConf to the one actually used by the SparkContext in JVM. + self._conf = SparkConf(_jconf=self._jsc.sc().conf()) # Create a single Accumulator in Java that we'll send all our updates through; # they will be passed back to us through a TCP server diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 0c8024e4a8b49..b4152a34ad97e 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -165,6 +165,13 @@ def getOrCreate(self): for key, value in self._options.items(): sparkConf.set(key, value) sc = SparkContext.getOrCreate(sparkConf) + # This SparkContext may be an existing one. + for key, value in self._options.items(): + # we need to propagate the confs + # before we create the SparkSession. Otherwise, confs like + # warehouse path and metastore url will not be set correctly ( + # these confs cannot be changed once the SparkSession is created). + sc._conf.set(key, value) session = SparkSession(sc) for key, value in self._options.items(): session.conf.set(key, value) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 3f564110ed8bc..f863485e6c0d8 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -22,6 +22,7 @@ """ import os import sys +import subprocess import pydoc import shutil import tempfile @@ -48,7 +49,7 @@ from pyspark.sql import SparkSession, HiveContext, Column, Row from pyspark.sql.types import * from pyspark.sql.types import UserDefinedType, _infer_type -from pyspark.tests import ReusedPySparkTestCase +from pyspark.tests import ReusedPySparkTestCase, SparkSubmitTests from pyspark.sql.functions import UserDefinedFunction, sha2 from pyspark.sql.window import Window from pyspark.sql.utils import AnalysisException, ParseException, IllegalArgumentException @@ -1619,6 +1620,46 @@ def test_cache(self): lambda: spark.catalog.uncacheTable("does_not_exist")) +class HiveSparkSubmitTests(SparkSubmitTests): + + def test_hivecontext(self): + # This test checks that HiveContext is using Hive metastore (SPARK-16224). + # It sets a metastore url and checks if there is a derby dir created by + # Hive metastore. If this derby dir exists, HiveContext is using + # Hive metastore. + metastore_path = os.path.join(tempfile.mkdtemp(), "spark16224_metastore_db") + metastore_URL = "jdbc:derby:;databaseName=" + metastore_path + ";create=true" + hive_site_dir = os.path.join(self.programDir, "conf") + hive_site_file = self.createTempFile("hive-site.xml", (""" + | + | + | javax.jdo.option.ConnectionURL + | %s + | + | + """ % metastore_URL).lstrip(), "conf") + script = self.createTempFile("test.py", """ + |import os + | + |from pyspark.conf import SparkConf + |from pyspark.context import SparkContext + |from pyspark.sql import HiveContext + | + |conf = SparkConf() + |sc = SparkContext(conf=conf) + |hive_context = HiveContext(sc) + |print(hive_context.sql("show databases").collect()) + """) + proc = subprocess.Popen( + [self.sparkSubmit, "--master", "local-cluster[1,1,1024]", + "--driver-class-path", hive_site_dir, script], + stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("default", out.decode('utf-8')) + self.assertTrue(os.path.exists(metastore_path)) + + class HiveContextSQLTests(ReusedPySparkTestCase): @classmethod diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 97ea39dde05fa..b1e92e13a1fa3 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1921,6 +1921,14 @@ def test_parallelize_eager_cleanup(self): post_parallalize_temp_files = os.listdir(sc._temp_dir) self.assertEqual(temp_files, post_parallalize_temp_files) + def test_set_conf(self): + # This is for an internal use case. When there is an existing SparkContext, + # SparkSession's builder needs to set configs into SparkContext's conf. + sc = SparkContext() + sc._conf.set("spark.test.SPARK16224", "SPARK16224") + self.assertEqual(sc._jsc.sc().conf().get("spark.test.SPARK16224"), "SPARK16224") + sc.stop() + def test_stop(self): sc = SparkContext() self.assertNotEqual(SparkContext._active_spark_context, None) From e68872f2ef89e85ab7a856bee82d16c938de6db0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 28 Jun 2016 10:26:01 -0700 Subject: [PATCH 0826/1470] [SPARK-16181][SQL] outer join with isNull filter may return wrong result ## What changes were proposed in this pull request? The root cause is: the output attributes of outer join are derived from its children, while they are actually different attributes(outer join can return null). We have already added some special logic to handle it, e.g. `PushPredicateThroughJoin` won't push down predicates through outer join side, `FixNullability`. This PR adds one more special logic in `FoldablePropagation`. ## How was this patch tested? new test in `DataFrameSuite` Author: Wenchen Fan Closes #13884 from cloud-fan/bug. (cherry picked from commit 1f2776df6e87a84991537ac20e4b8829472d3462) Signed-off-by: Yin Huai --- .../apache/spark/sql/catalyst/optimizer/Optimizer.scala | 8 ++++++++ .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 +++++++++ 2 files changed, 17 insertions(+) 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 6b10484928046..f24f8b78d476f 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 @@ -687,6 +687,14 @@ object FoldablePropagation extends Rule[LogicalPlan] { case c: Command => stop = true c + // For outer join, although its output attributes are derived from its children, they are + // actually different attributes: the output of outer join is not always picked from its + // children, but can also be null. + // TODO(cloud-fan): It seems more reasonable to use new attributes as the output attributes + // of outer join. + case j @ Join(_, _, LeftOuter | RightOuter | FullOuter, _) => + stop = true + j case p: LogicalPlan if !stop => p.transformExpressions { case a: AttributeReference if foldableMap.contains(a) => foldableMap(a) 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 1afee9f02104e..5151532ed2e1f 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 @@ -1541,4 +1541,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val df = Seq(1, 1, 2).toDF("column.with.dot") checkAnswer(df.distinct(), Row(1) :: Row(2) :: Nil) } + + test("SPARK-16181: outer join with isNull filter") { + val left = Seq("x").toDF("col") + val right = Seq("y").toDF("col").withColumn("new", lit(true)) + val joined = left.join(right, left("col") === right("col"), "left_outer") + + checkAnswer(joined, Row("x", null, null)) + checkAnswer(joined.filter($"new".isNull), Row("x", null, null)) + } } From a1d04cc03edac729488e9eb2b9e34e6c951ddbae Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 28 Jun 2016 11:54:25 -0700 Subject: [PATCH 0827/1470] [SPARK-15643][DOC][ML] Update spark.ml and spark.mllib migration guide from 1.6 to 2.0 ## What changes were proposed in this pull request? Update ```spark.ml``` and ```spark.mllib``` migration guide from 1.6 to 2.0. ## How was this patch tested? Docs update, no tests. Author: Yanbo Liang Closes #13378 from yanboliang/spark-13448. (cherry picked from commit 26252f7064ba852e1bce6d8233a95aeb395f826a) Signed-off-by: Joseph K. Bradley --- docs/mllib-guide.md | 60 +++++++++++++++++++++++----------- docs/mllib-migration-guides.md | 27 +++++++++++++++ 2 files changed, 68 insertions(+), 19 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index fa5e90603505d..c28d13732eed8 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -102,32 +102,54 @@ MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, and the migration guide below will explain all changes between releases. -## From 1.5 to 1.6 +## From 1.6 to 2.0 -There are no breaking API changes in the `spark.mllib` or `spark.ml` packages, but there are -deprecations and changes of behavior. +The deprecations and changes of behavior in the `spark.mllib` or `spark.ml` packages include: Deprecations: -* [SPARK-11358](https://issues.apache.org/jira/browse/SPARK-11358): - In `spark.mllib.clustering.KMeans`, the `runs` parameter has been deprecated. -* [SPARK-10592](https://issues.apache.org/jira/browse/SPARK-10592): - In `spark.ml.classification.LogisticRegressionModel` and - `spark.ml.regression.LinearRegressionModel`, the `weights` field has been deprecated in favor of - the new name `coefficients`. This helps disambiguate from instance (row) "weights" given to - algorithms. +* [SPARK-14984](https://issues.apache.org/jira/browse/SPARK-14984): + In `spark.ml.regression.LinearRegressionSummary`, the `model` field has been deprecated. +* [SPARK-13784](https://issues.apache.org/jira/browse/SPARK-13784): + In `spark.ml.regression.RandomForestRegressionModel` and `spark.ml.classification.RandomForestClassificationModel`, + the `numTrees` parameter has been deprecated in favor of `getNumTrees` method. +* [SPARK-13761](https://issues.apache.org/jira/browse/SPARK-13761): + In `spark.ml.param.Params`, the `validateParams` method has been deprecated. + We move all functionality in overridden methods to the corresponding `transformSchema`. +* [SPARK-14829](https://issues.apache.org/jira/browse/SPARK-14829): + In `spark.mllib` package, `LinearRegressionWithSGD`, `LassoWithSGD`, `RidgeRegressionWithSGD` and `LogisticRegressionWithSGD` have been deprecated. + We encourage users to use `spark.ml.regression.LinearRegresson` and `spark.ml.classification.LogisticRegresson`. +* [SPARK-14900](https://issues.apache.org/jira/browse/SPARK-14900): + In `spark.mllib.evaluation.MulticlassMetrics`, the parameters `precision`, `recall` and `fMeasure` have been deprecated in favor of `accuracy`. +* [SPARK-15644](https://issues.apache.org/jira/browse/SPARK-15644): + In `spark.ml.util.MLReader` and `spark.ml.util.MLWriter`, the `context` method has been deprecated in favor of `session`. +* In `spark.ml.feature.ChiSqSelectorModel`, the `setLabelCol` method has been deprecated since it was not used by `ChiSqSelectorModel`. Changes of behavior: -* [SPARK-7770](https://issues.apache.org/jira/browse/SPARK-7770): - `spark.mllib.tree.GradientBoostedTrees`: `validationTol` has changed semantics in 1.6. - Previously, it was a threshold for absolute change in error. Now, it resembles the behavior of - `GradientDescent`'s `convergenceTol`: For large errors, it uses relative error (relative to the - previous error); for small errors (`< 0.01`), it uses absolute error. -* [SPARK-11069](https://issues.apache.org/jira/browse/SPARK-11069): - `spark.ml.feature.RegexTokenizer`: Previously, it did not convert strings to lowercase before - tokenizing. Now, it converts to lowercase by default, with an option not to. This matches the - behavior of the simpler `Tokenizer` transformer. +* [SPARK-7780](https://issues.apache.org/jira/browse/SPARK-7780): + `spark.mllib.classification.LogisticRegressionWithLBFGS` directly calls `spark.ml.classification.LogisticRegresson` for binary classification now. + This will introduce the following behavior changes for `spark.mllib.classification.LogisticRegressionWithLBFGS`: + * The intercept will not be regularized when training binary classification model with L1/L2 Updater. + * If users set without regularization, training with or without feature scaling will return the same solution by the same convergence rate. +* [SPARK-13429](https://issues.apache.org/jira/browse/SPARK-13429): + In order to provide better and consistent result with `spark.ml.classification.LogisticRegresson`, + the default value of `spark.mllib.classification.LogisticRegressionWithLBFGS`: `convergenceTol` has been changed from 1E-4 to 1E-6. +* [SPARK-12363](https://issues.apache.org/jira/browse/SPARK-12363): + Fix a bug of `PowerIterationClustering` which will likely change its result. +* [SPARK-13048](https://issues.apache.org/jira/browse/SPARK-13048): + `LDA` using the `EM` optimizer will keep the last checkpoint by default, if checkpointing is being used. +* [SPARK-12153](https://issues.apache.org/jira/browse/SPARK-12153): + `Word2Vec` now respects sentence boundaries. Previously, it did not handle them correctly. +* [SPARK-10574](https://issues.apache.org/jira/browse/SPARK-10574): + `HashingTF` uses `MurmurHash3` as default hash algorithm in both `spark.ml` and `spark.mllib`. +* [SPARK-14768](https://issues.apache.org/jira/browse/SPARK-14768): + The `expectedType` argument for PySpark `Param` was removed. +* [SPARK-14931](https://issues.apache.org/jira/browse/SPARK-14931): + Some default `Param` values, which were mismatched between pipelines in Scala and Python, have been changed. +* [SPARK-13600](https://issues.apache.org/jira/browse/SPARK-13600): + `QuantileDiscretizer` now uses `spark.sql.DataFrameStatFunctions.approxQuantile` to find splits (previously used custom sampling logic). + The output buckets will differ for same input data and params. ## Previous Spark versions diff --git a/docs/mllib-migration-guides.md b/docs/mllib-migration-guides.md index f3daef2dbadbe..970c6697f433e 100644 --- a/docs/mllib-migration-guides.md +++ b/docs/mllib-migration-guides.md @@ -7,6 +7,33 @@ description: MLlib migration guides from before Spark SPARK_VERSION_SHORT The migration guide for the current Spark version is kept on the [MLlib Programming Guide main page](mllib-guide.html#migration-guide). +## From 1.5 to 1.6 + +There are no breaking API changes in the `spark.mllib` or `spark.ml` packages, but there are +deprecations and changes of behavior. + +Deprecations: + +* [SPARK-11358](https://issues.apache.org/jira/browse/SPARK-11358): + In `spark.mllib.clustering.KMeans`, the `runs` parameter has been deprecated. +* [SPARK-10592](https://issues.apache.org/jira/browse/SPARK-10592): + In `spark.ml.classification.LogisticRegressionModel` and + `spark.ml.regression.LinearRegressionModel`, the `weights` field has been deprecated in favor of + the new name `coefficients`. This helps disambiguate from instance (row) "weights" given to + algorithms. + +Changes of behavior: + +* [SPARK-7770](https://issues.apache.org/jira/browse/SPARK-7770): + `spark.mllib.tree.GradientBoostedTrees`: `validationTol` has changed semantics in 1.6. + Previously, it was a threshold for absolute change in error. Now, it resembles the behavior of + `GradientDescent`'s `convergenceTol`: For large errors, it uses relative error (relative to the + previous error); for small errors (`< 0.01`), it uses absolute error. +* [SPARK-11069](https://issues.apache.org/jira/browse/SPARK-11069): + `spark.ml.feature.RegexTokenizer`: Previously, it did not convert strings to lowercase before + tokenizing. Now, it converts to lowercase by default, with an option not to. This matches the + behavior of the simpler `Tokenizer` transformer. + ## From 1.4 to 1.5 In the `spark.mllib` package, there are no breaking API changes but several behavior changes: From c86d29b2e6bfda05124c20ba3c6db9275c24faa8 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 28 Jun 2016 12:12:20 -0700 Subject: [PATCH 0828/1470] [MINOR][SPARKR] update sparkR DataFrame.R comment ## What changes were proposed in this pull request? update sparkR DataFrame.R comment SQLContext ==> SparkSession ## How was this patch tested? N/A Author: WeichenXu Closes #13946 from WeichenXu123/sparkR_comment_update_sparkSession. (cherry picked from commit d59ba8e30751bbf91d49f5530b8242a12bbfb569) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index f856979c2a814..567758d2e2f28 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -466,7 +466,7 @@ setMethod("createOrReplaceTempView", #' (Deprecated) Register Temporary Table #' -#' Registers a SparkDataFrame as a Temporary Table in the SQLContext +#' Registers a SparkDataFrame as a Temporary Table in the SparkSession #' @param x A SparkDataFrame #' @param tableName A character vector containing the name of the table #' @@ -493,7 +493,7 @@ setMethod("registerTempTable", #' insertInto #' -#' Insert the contents of a SparkDataFrame into a table registered in the current SQL Context. +#' Insert the contents of a SparkDataFrame into a table registered in the current SparkSession. #' #' @param x A SparkDataFrame #' @param tableName A character vector containing the name of the table From 5c9555e1115ce52954db2a1b18f78cd77ec8c15f Mon Sep 17 00:00:00 2001 From: Tom Magrino Date: Tue, 28 Jun 2016 13:36:41 -0700 Subject: [PATCH 0829/1470] [SPARK-16148][SCHEDULER] Allow for underscores in TaskLocation in the Executor ID ## What changes were proposed in this pull request? Previously, the TaskLocation implementation would not allow for executor ids which include underscores. This tweaks the string split used to get the hostname and executor id, allowing for underscores in the executor id. This addresses the JIRA found here: https://issues.apache.org/jira/browse/SPARK-16148 This is moved over from a previous PR against branch-1.6: https://github.com/apache/spark/pull/13857 ## How was this patch tested? Ran existing unit tests for core and streaming. Manually ran a simple streaming job with an executor whose id contained underscores and confirmed that the job ran successfully. This is my original work and I license the work to the project under the project's open source license. Author: Tom Magrino Closes #13858 from tmagrino/fixtasklocation. (cherry picked from commit ae14f362355b131fcb3e3633da7bb14bdd2b6893) Signed-off-by: Shixiong Zhu --- .../org/apache/spark/scheduler/TaskLocation.scala | 14 +++++++------- .../spark/scheduler/TaskSetManagerSuite.scala | 2 ++ 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala index 1eb6c1614fc0b..06b52935c696c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala @@ -64,18 +64,18 @@ private[spark] object TaskLocation { /** * Create a TaskLocation from a string returned by getPreferredLocations. - * These strings have the form [hostname] or hdfs_cache_[hostname], depending on whether the - * location is cached. + * These strings have the form executor_[hostname]_[executorid], [hostname], or + * hdfs_cache_[hostname], depending on whether the location is cached. */ def apply(str: String): TaskLocation = { val hstr = str.stripPrefix(inMemoryLocationTag) if (hstr.equals(str)) { if (str.startsWith(executorLocationTag)) { - val splits = str.split("_") - if (splits.length != 3) { - throw new IllegalArgumentException("Illegal executor location format: " + str) - } - new ExecutorCacheTaskLocation(splits(1), splits(2)) + val hostAndExecutorId = str.stripPrefix(executorLocationTag) + val splits = hostAndExecutorId.split("_", 2) + require(splits.length == 2, "Illegal executor location format: " + str) + val Array(host, executorId) = splits + new ExecutorCacheTaskLocation(host, executorId) } else { new HostTaskLocation(str) } 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 9b7b945bf3677..862313326c93e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -787,6 +787,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(TaskLocation("host1") === HostTaskLocation("host1")) assert(TaskLocation("hdfs_cache_host1") === HDFSCacheTaskLocation("host1")) assert(TaskLocation("executor_host1_3") === ExecutorCacheTaskLocation("host1", "3")) + assert(TaskLocation("executor_some.host1_executor_task_3") === + ExecutorCacheTaskLocation("some.host1", "executor_task_3")) } private def createTaskResult( From 43bd612f35490c11a76d5379d723ba65f7afbefd Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 28 Jun 2016 14:09:38 -0700 Subject: [PATCH 0830/1470] [SPARK-16175] [PYSPARK] handle None for UDT ## What changes were proposed in this pull request? Scala UDT will bypass all the null and will not pass them into serialize() and deserialize() of UDT, this PR update the Python UDT to do this as well. ## How was this patch tested? Added tests. Author: Davies Liu Closes #13878 from davies/udt_null. (cherry picked from commit 35438fb0ad3bcda5c5a3a0ccde1a620699d012db) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 11 +++++++++++ python/pyspark/sql/types.py | 7 +++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index f863485e6c0d8..a8ca386e1ce31 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -575,6 +575,17 @@ def check_datatype(datatype): _verify_type(PythonOnlyPoint(1.0, 2.0), PythonOnlyUDT()) self.assertRaises(ValueError, lambda: _verify_type([1.0, 2.0], PythonOnlyUDT())) + def test_udt_with_none(self): + df = self.spark.range(0, 10, 1, 1) + + def myudf(x): + if x > 0: + return PythonOnlyPoint(float(x), float(x)) + + self.spark.catalog.registerFunction("udf", myudf, PythonOnlyUDT()) + rows = [r[0] for r in df.selectExpr("udf(id)").take(2)] + self.assertEqual(rows, [None, PythonOnlyPoint(1, 1)]) + def test_infer_schema_with_udt(self): from pyspark.sql.tests import ExamplePoint, ExamplePointUDT row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index f0b56be8da438..a3679873e1d8d 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -648,10 +648,13 @@ def _cachedSqlType(cls): return cls._cached_sql_type def toInternal(self, obj): - return self._cachedSqlType().toInternal(self.serialize(obj)) + if obj is not None: + return self._cachedSqlType().toInternal(self.serialize(obj)) def fromInternal(self, obj): - return self.deserialize(self._cachedSqlType().fromInternal(obj)) + v = self._cachedSqlType().fromInternal(obj) + if v is not None: + return self.deserialize(v) def serialize(self, obj): """ From 5626a0af598168a15d68a8817d1dec2a0e3dec7e Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 28 Jun 2016 15:32:45 -0700 Subject: [PATCH 0831/1470] [SPARK-16236][SQL] Add Path Option back to Load API in DataFrameReader #### What changes were proposed in this pull request? koertkuipers identified the PR https://github.com/apache/spark/pull/13727/ changed the behavior of `load` API. After the change, the `load` API does not add the value of `path` into the `options`. Thank you! This PR is to add the option `path` back to `load()` API in `DataFrameReader`, if and only if users specify one and only one `path` in the `load` API. For example, users can see the `path` option after the following API call, ```Scala spark.read .format("parquet") .load("/test") ``` #### How was this patch tested? Added test cases. Author: gatorsmile Closes #13933 from gatorsmile/optionPath. (cherry picked from commit 25520e976275e0d1e3bf9c73128ef4dec4618568) Signed-off-by: Reynold Xin --- .../apache/spark/sql/DataFrameReader.scala | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 28 +++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 35ba9c50790e4..35ba522786337 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -129,7 +129,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def load(path: String): DataFrame = { - load(Seq(path): _*) // force invocation of `load(...varargs...)` + option("path", path).load(Seq.empty: _*) // force invocation of `load(...varargs...)` } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 3fa3864bc9690..ebbcc1d7ffbb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -170,6 +170,34 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be assert(LastOptions.saveMode === SaveMode.ErrorIfExists) } + test("test path option in load") { + spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 56) + .load("/test") + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("path") == "/test") + + LastOptions.clear() + spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 55) + .load() + + assert(LastOptions.parameters("intOpt") == "55") + assert(!LastOptions.parameters.contains("path")) + + LastOptions.clear() + spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 54) + .load("/test", "/test1", "/test2") + + assert(LastOptions.parameters("intOpt") == "54") + assert(!LastOptions.parameters.contains("path")) + } + test("test different data types for options") { val df = spark.read .format("org.apache.spark.sql.test") From d73c38ed0e129bdcb634000153516fca4b31b9d0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 29 Jun 2016 06:39:28 +0800 Subject: [PATCH 0832/1470] [SPARK-16100][SQL] fix bug when use Map as the buffer type of Aggregator ## What changes were proposed in this pull request? The root cause is in `MapObjects`. Its parameter `loopVar` is not declared as child, but sometimes can be same with `lambdaFunction`(e.g. the function that takes `loopVar` and produces `lambdaFunction` may be `identity`), which is a child. This brings trouble when call `withNewChildren`, it may mistakenly treat `loopVar` as a child and cause `IndexOutOfBoundsException: 0` later. This PR fixes this bug by simply pulling out the paremters from `LambdaVariable` and pass them to `MapObjects` directly. ## How was this patch tested? new test in `DatasetAggregatorSuite` Author: Wenchen Fan Closes #13835 from cloud-fan/map-objects. (cherry picked from commit 8a977b065418f07d2bf4fe1607a5534c32d04c47) Signed-off-by: Cheng Lian --- .../expressions/objects/objects.scala | 28 +++++++++++-------- .../spark/sql/DatasetAggregatorSuite.scala | 15 ++++++++++ 2 files changed, 32 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index c597a2a709445..ea4dee174e74e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -353,7 +353,7 @@ object MapObjects { val loopValue = "MapObjects_loopValue" + curId.getAndIncrement() val loopIsNull = "MapObjects_loopIsNull" + curId.getAndIncrement() val loopVar = LambdaVariable(loopValue, loopIsNull, elementType) - MapObjects(loopVar, function(loopVar), inputData) + MapObjects(loopValue, loopIsNull, elementType, function(loopVar), inputData) } } @@ -365,14 +365,20 @@ object MapObjects { * The following collection ObjectTypes are currently supported: * Seq, Array, ArrayData, java.util.List * - * @param loopVar A place holder that used as the loop variable when iterate the collection, and - * used as input for the `lambdaFunction`. It also carries the element type info. + * @param loopValue the name of the loop variable that used when iterate the collection, and used + * as input for the `lambdaFunction` + * @param loopIsNull the nullity of the loop variable that used when iterate the collection, and + * used as input for the `lambdaFunction` + * @param loopVarDataType the data type of the loop variable that used when iterate the collection, + * and used as input for the `lambdaFunction` * @param lambdaFunction A function that take the `loopVar` as input, and used as lambda function * to handle collection elements. * @param inputData An expression that when evaluated returns a collection object. */ case class MapObjects private( - loopVar: LambdaVariable, + loopValue: String, + loopIsNull: String, + loopVarDataType: DataType, lambdaFunction: Expression, inputData: Expression) extends Expression with NonSQLExpression { @@ -386,9 +392,9 @@ case class MapObjects private( override def dataType: DataType = ArrayType(lambdaFunction.dataType) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val elementJavaType = ctx.javaType(loopVar.dataType) - ctx.addMutableState("boolean", loopVar.isNull, "") - ctx.addMutableState(elementJavaType, loopVar.value, "") + val elementJavaType = ctx.javaType(loopVarDataType) + ctx.addMutableState("boolean", loopIsNull, "") + ctx.addMutableState(elementJavaType, loopValue, "") val genInputData = inputData.genCode(ctx) val genFunction = lambdaFunction.genCode(ctx) val dataLength = ctx.freshName("dataLength") @@ -443,11 +449,11 @@ case class MapObjects private( } val loopNullCheck = inputData.dataType match { - case _: ArrayType => s"${loopVar.isNull} = ${genInputData.value}.isNullAt($loopIndex);" + case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" // The element of primitive array will never be null. case ObjectType(cls) if cls.isArray && cls.getComponentType.isPrimitive => - s"${loopVar.isNull} = false" - case _ => s"${loopVar.isNull} = ${loopVar.value} == null;" + s"$loopIsNull = false" + case _ => s"$loopIsNull = $loopValue == null;" } val code = s""" @@ -462,7 +468,7 @@ case class MapObjects private( int $loopIndex = 0; while ($loopIndex < $dataLength) { - ${loopVar.value} = ($elementJavaType) ($getLoopVar); + $loopValue = ($elementJavaType) ($getLoopVar); $loopNullCheck ${genFunction.code} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index f955120dc5439..32fcf84b02f92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -74,6 +74,16 @@ object ComplexBufferAgg extends Aggregator[AggData, (Int, AggData), Int] { } +object MapTypeBufferAgg extends Aggregator[Int, Map[Int, Int], Int] { + override def zero: Map[Int, Int] = Map.empty + override def reduce(b: Map[Int, Int], a: Int): Map[Int, Int] = b + override def finish(reduction: Map[Int, Int]): Int = 1 + override def merge(b1: Map[Int, Int], b2: Map[Int, Int]): Map[Int, Int] = b1 + override def bufferEncoder: Encoder[Map[Int, Int]] = ExpressionEncoder() + override def outputEncoder: Encoder[Int] = ExpressionEncoder() +} + + object NameAgg extends Aggregator[AggData, String, String] { def zero: String = "" def reduce(b: String, a: AggData): String = a.b + b @@ -290,4 +300,9 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { ds.groupByKey(_.a).agg(NullResultAgg.toColumn), 1 -> AggData(1, "one"), 2 -> null) } + + test("SPARK-16100: use Map as the buffer type of Aggregator") { + val ds = Seq(1, 2, 3).toDS() + checkDataset(ds.select(MapTypeBufferAgg.toColumn), 1) + } } From 5fb7804e55e50ba61c3a780b771d9b20b0bf2424 Mon Sep 17 00:00:00 2001 From: James Thomas Date: Tue, 28 Jun 2016 16:12:48 -0700 Subject: [PATCH 0833/1470] [SPARK-16114][SQL] structured streaming network word count examples ## What changes were proposed in this pull request? Network word count example for structured streaming ## How was this patch tested? Run locally Author: James Thomas Author: James Thomas Closes #13816 from jjthomas/master. (cherry picked from commit 3554713a163c58ca176ffde87d2c6e4a91bacb50) Signed-off-by: Tathagata Das --- .../JavaStructuredNetworkWordCount.java | 82 +++++++++++++++++++ .../streaming/structured_network_wordcount.py | 76 +++++++++++++++++ .../StructuredNetworkWordCount.scala | 76 +++++++++++++++++ 3 files changed, 234 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java create mode 100644 examples/src/main/python/sql/streaming/structured_network_wordcount.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java new file mode 100644 index 0000000000000..a2cf9389543e8 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql.streaming; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.util.Arrays; +import java.util.Iterator; + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * + * Usage: JavaStructuredNetworkWordCount + * and describe the TCP server that Structured Streaming + * would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example sql.streaming.JavaStructuredNetworkWordCount + * localhost 9999` + */ +public final class JavaStructuredNetworkWordCount { + + public static void main(String[] args) throws Exception { + if (args.length < 2) { + System.err.println("Usage: JavaNetworkWordCount "); + System.exit(1); + } + + String host = args[0]; + int port = Integer.parseInt(args[1]); + + SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCount") + .getOrCreate(); + + // Create DataFrame representing the stream of input lines from connection to host:port + Dataset lines = spark + .readStream() + .format("socket") + .option("host", host) + .option("port", port) + .load().as(Encoders.STRING()); + + // Split the lines into words + Dataset words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterator call(String x) { + return Arrays.asList(x.split(" ")).iterator(); + } + }, Encoders.STRING()); + + // Generate running word count + Dataset wordCounts = words.groupBy("value").count(); + + // Start running the query that prints the running counts to the console + StreamingQuery query = wordCounts.writeStream() + .outputMode("complete") + .format("console") + .start(); + + query.awaitTermination(); + } +} diff --git a/examples/src/main/python/sql/streaming/structured_network_wordcount.py b/examples/src/main/python/sql/streaming/structured_network_wordcount.py new file mode 100644 index 0000000000000..32d63c52c9191 --- /dev/null +++ b/examples/src/main/python/sql/streaming/structured_network_wordcount.py @@ -0,0 +1,76 @@ +# +# 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. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Usage: structured_network_wordcount.py + and describe the TCP server that Structured Streaming + would connect to receive data. + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py + localhost 9999` +""" +from __future__ import print_function + +import sys + +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: structured_network_wordcount.py ", file=sys.stderr) + exit(-1) + + host = sys.argv[1] + port = int(sys.argv[2]) + + spark = SparkSession\ + .builder\ + .appName("StructuredNetworkWordCount")\ + .getOrCreate() + + # Create DataFrame representing the stream of input lines from connection to host:port + lines = spark\ + .readStream\ + .format('socket')\ + .option('host', host)\ + .option('port', port)\ + .load() + + # Split the lines into words + words = lines.select( + explode( + split(lines.value, ' ') + ).alias('word') + ) + + # Generate running word count + wordCounts = words.groupBy('word').count() + + # Start running the query that prints the running counts to the console + query = wordCounts\ + .writeStream\ + .outputMode('complete')\ + .format('console')\ + .start() + + query.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala new file mode 100644 index 0000000000000..433f7a181bbf8 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala @@ -0,0 +1,76 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.sql.streaming + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.SparkSession + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * + * Usage: StructuredNetworkWordCount + * and describe the TCP server that Structured Streaming + * would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example sql.streaming.StructuredNetworkWordCount + * localhost 9999` + */ +object StructuredNetworkWordCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println("Usage: StructuredNetworkWordCount ") + System.exit(1) + } + + val host = args(0) + val port = args(1).toInt + + val spark = SparkSession + .builder + .appName("StructuredNetworkWordCount") + .getOrCreate() + + import spark.implicits._ + + // Create DataFrame representing the stream of input lines from connection to host:port + val lines = spark.readStream + .format("socket") + .option("host", host) + .option("port", port) + .load().as[String] + + // Split the lines into words + val words = lines.flatMap(_.split(" ")) + + // Generate running word count + val wordCounts = words.groupBy("value").count() + + // Start running the query that prints the running counts to the console + val query = wordCounts.writeStream + .outputMode("complete") + .format("console") + .start() + + query.awaitTermination() + } +} +// scalastyle:on println From 52c9d69f7da05c45cb191fef8f7ce54c8f40b1bb Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 28 Jun 2016 17:02:16 -0700 Subject: [PATCH 0834/1470] [MINOR][DOCS][STRUCTURED STREAMING] Minor doc fixes around `DataFrameWriter` and `DataStreamWriter` ## What changes were proposed in this pull request? Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start Author: Burak Yavuz Closes #13952 from brkyvz/minor-doc-fix. (cherry picked from commit 5545b791096756b07b3207fb3de13b68b9a37b00) Signed-off-by: Shixiong Zhu --- python/pyspark/sql/dataframe.py | 4 ++-- .../sql/catalyst/analysis/UnsupportedOperationChecker.scala | 6 +++--- .../sql/catalyst/analysis/UnsupportedOperationsSuite.scala | 6 +++--- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 ++-- .../org/apache/spark/sql/streaming/DataStreamWriter.scala | 2 +- .../org/apache/spark/sql/streaming/StreamingQuery.scala | 4 ++-- .../apache/spark/sql/streaming/StreamingQueryListener.scala | 4 ++-- .../scala/org/apache/spark/sql/streaming/StreamSuite.scala | 4 ++-- 8 files changed, 17 insertions(+), 17 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index acf9d08b23a27..c8c8e7d790356 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -257,8 +257,8 @@ def isLocal(self): def isStreaming(self): """Returns true if this :class:`Dataset` contains one or more sources that continuously return data as it arrives. A :class:`Dataset` that reads data from a streaming source - must be executed as a :class:`StreamingQuery` using the :func:`startStream` method in - :class:`DataFrameWriter`. Methods that return a single answer, (e.g., :func:`count` or + must be executed as a :class:`StreamingQuery` using the :func:`start` method in + :class:`DataStreamWriter`. Methods that return a single answer, (e.g., :func:`count` or :func:`collect`) will throw an :class:`AnalysisException` when there is a streaming source present. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 689e016a5a1d9..f6e32e29ebca8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -30,7 +30,7 @@ object UnsupportedOperationChecker { def checkForBatch(plan: LogicalPlan): Unit = { plan.foreachUp { case p if p.isStreaming => - throwError("Queries with streaming sources must be executed with write.startStream()")(p) + throwError("Queries with streaming sources must be executed with writeStream.start()")(p) case _ => } @@ -40,7 +40,7 @@ object UnsupportedOperationChecker { if (!plan.isStreaming) { throwError( - "Queries without streaming sources cannot be executed with write.startStream()")(plan) + "Queries without streaming sources cannot be executed with writeStream.start()")(plan) } // Disallow multiple streaming aggregations @@ -154,7 +154,7 @@ object UnsupportedOperationChecker { case ReturnAnswer(child) if child.isStreaming => throwError("Cannot return immediate result on streaming DataFrames/Dataset. Queries " + - "with streaming DataFrames/Datasets must be executed with write.startStream().") + "with streaming DataFrames/Datasets must be executed with writeStream.start().") case _ => } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index c21ad5e03a48d..6df47acaba85b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -53,12 +53,12 @@ class UnsupportedOperationsSuite extends SparkFunSuite { assertNotSupportedInBatchPlan( "streaming source", streamRelation, - Seq("with streaming source", "startStream")) + Seq("with streaming source", "start")) assertNotSupportedInBatchPlan( "select on streaming source", streamRelation.select($"count(*)"), - Seq("with streaming source", "startStream")) + Seq("with streaming source", "start")) /* @@ -70,7 +70,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite { // Batch plan in streaming query testError( "streaming plan - no streaming source", - Seq("without streaming source", "startStream")) { + Seq("without streaming source", "start")) { UnsupportedOperationChecker.checkForStreaming(batchRelation.select($"count(*)"), Append) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 85d060639c7ff..153af742892a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -455,8 +455,8 @@ class Dataset[T] private[sql]( /** * Returns true if this Dataset contains one or more sources that continuously * return data as it arrives. A Dataset that reads data from a streaming source - * must be executed as a [[StreamingQuery]] using the `startStream()` method in - * [[DataFrameWriter]]. Methods that return a single answer, e.g. `count()` or + * must be executed as a [[StreamingQuery]] using the `start()` method in + * [[DataStreamWriter]]. Methods that return a single answer, e.g. `count()` or * `collect()`, will throw an [[AnalysisException]] when there is a streaming * source present. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index d4b0a3cca2402..d38e3e58125d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -109,7 +109,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * :: Experimental :: - * Specifies the name of the [[StreamingQuery]] that can be started with `startStream()`. + * Specifies the name of the [[StreamingQuery]] that can be started with `start()`. * This name must be unique among all the currently active queries in the associated SQLContext. * * @since 2.0.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index 19d1ecf740d0a..91f0a1e3446a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -31,8 +31,8 @@ trait StreamingQuery { /** * Returns the name of the query. This name is unique across all active queries. This can be - * set in the[[org.apache.spark.sql.DataFrameWriter DataFrameWriter]] as - * `dataframe.write().queryName("query").startStream()`. + * set in the [[org.apache.spark.sql.DataStreamWriter DataStreamWriter]] as + * `dataframe.writeStream.queryName("query").start()`. * @since 2.0.0 */ def name: String diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index c43de58faa80c..3b3cead3a66de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -35,9 +35,9 @@ abstract class StreamingQueryListener { /** * Called when a query is started. * @note This is called synchronously with - * [[org.apache.spark.sql.DataFrameWriter `DataFrameWriter.startStream()`]], + * [[org.apache.spark.sql.DataStreamWriter `DataStreamWriter.start()`]], * that is, `onQueryStart` will be called on all listeners before - * `DataFrameWriter.startStream()` returns the corresponding [[StreamingQuery]]. Please + * `DataStreamWriter.start()` returns the corresponding [[StreamingQuery]]. Please * don't block this method as it will block your query. * @since 2.0.0 */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index c4a894b6816ac..28170f30646ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -120,12 +120,12 @@ class StreamSuite extends StreamTest { } // Running streaming plan as a batch query - assertError("startStream" :: Nil) { + assertError("start" :: Nil) { streamInput.toDS.map { i => i }.count() } // Running non-streaming plan with as a streaming query - assertError("without streaming sources" :: "startStream" :: Nil) { + assertError("without streaming sources" :: "start" :: Nil) { val ds = batchInput.map { i => i } testStream(ds)() } From d7a59f1f450aae06baac96867a26042bd1ccd1d5 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 28 Jun 2016 17:08:28 -0700 Subject: [PATCH 0835/1470] [SPARKR] add csv tests ## What changes were proposed in this pull request? Add unit tests for csv data for SPARKR ## How was this patch tested? unit tests Author: Felix Cheung Closes #13904 from felixcheung/rcsv. (cherry picked from commit 823518c2b5259c8a954431467639198c808c9198) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 74def5ce4245d..deda1b6f43588 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -208,6 +208,24 @@ test_that("create DataFrame from RDD", { unsetHiveContext() }) +test_that("read csv as DataFrame", { + csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") + mockLinesCsv <- c("year,make,model,comment,blank", + "\"2012\",\"Tesla\",\"S\",\"No comment\",", + "1997,Ford,E350,\"Go get one now they are going fast\",", + "2015,Chevy,Volt") + writeLines(mockLinesCsv, csvPath) + + # default "header" is false + df <- read.df(csvPath, "csv", header = "true") + expect_equal(count(df), 3) + expect_equal(columns(df), c("year", "make", "model", "comment", "blank")) + expect_equal(sort(unlist(collect(where(df, df$year == "2015")))), + sort(unlist(list(year = "2015", make = "Chevy", model = "Volt")))) + + unlink(csvPath) +}) + test_that("convert NAs to null type in DataFrames", { rdd <- parallelize(sc, list(list(1L, 2L), list(NA, 4L))) df <- createDataFrame(rdd, list("a", "b")) From 835c5a3bd549811178f5b455dc127401c5551866 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 28 Jun 2016 18:33:37 -0700 Subject: [PATCH 0836/1470] [SPARK-16268][PYSPARK] SQLContext should import DataStreamReader ## What changes were proposed in this pull request? Fixed the following error: ``` >>> sqlContext.readStream Traceback (most recent call last): File "", line 1, in File "...", line 442, in readStream return DataStreamReader(self._wrapped) NameError: global name 'DataStreamReader' is not defined ``` ## How was this patch tested? The added test. Author: Shixiong Zhu Closes #13958 from zsxwing/fix-import. (cherry picked from commit 5bf8881b34a18f25acc10aeb28a06af4c44a6ac8) Signed-off-by: Tathagata Das --- python/pyspark/sql/context.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 8a1a874884e2a..b5dde13ed7cc2 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -26,7 +26,7 @@ from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.session import _monkey_patch_RDD, SparkSession from pyspark.sql.dataframe import DataFrame -from pyspark.sql.readwriter import DataFrameReader +from pyspark.sql.readwriter import DataFrameReader, DataStreamReader from pyspark.sql.types import Row, StringType from pyspark.sql.utils import install_exception_handler @@ -438,8 +438,12 @@ def readStream(self): .. note:: Experimental. :return: :class:`DataStreamReader` + + >>> text_sdf = sqlContext.readStream.text(os.path.join(tempfile.mkdtemp(), 'data')) + >>> text_sdf.isStreaming + True """ - return DataStreamReader(self._wrapped) + return DataStreamReader(self) @property @since(2.0) @@ -515,6 +519,7 @@ def register(self, name, f, returnType=StringType()): def _test(): import os import doctest + import tempfile from pyspark.context import SparkContext from pyspark.sql import Row, SQLContext import pyspark.sql.context @@ -523,6 +528,8 @@ def _test(): globs = pyspark.sql.context.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') + globs['tempfile'] = tempfile + globs['os'] = os globs['sc'] = sc globs['sqlContext'] = SQLContext(sc) globs['rdd'] = rdd = sc.parallelize( From dd70a115cd562223e97f0b5e6172a9ea758be95d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 28 Jun 2016 19:36:53 -0700 Subject: [PATCH 0837/1470] [SPARK-16248][SQL] Whitelist the list of Hive fallback functions ## What changes were proposed in this pull request? This patch removes the blind fallback into Hive for functions. Instead, it creates a whitelist and adds only a small number of functions to the whitelist, i.e. the ones we intend to support in the long run in Spark. ## How was this patch tested? Updated tests to reflect the change. Author: Reynold Xin Closes #13939 from rxin/hive-whitelist. (cherry picked from commit 363bcedeea40fe3f1a92271b96af2acba63e058c) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../execution/HiveCompatibilitySuite.scala | 22 +++++----- .../HiveWindowFunctionQuerySuite.scala | 25 ----------- .../spark/sql/hive/HiveSessionCatalog.scala | 42 ++++++++++++------- 4 files changed, 40 insertions(+), 50 deletions(-) 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 42a8faa412a34..0bde48ce57c86 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 @@ -248,6 +248,7 @@ object FunctionRegistry { expression[Average]("mean"), expression[Min]("min"), expression[Skewness]("skewness"), + expression[StddevSamp]("std"), expression[StddevSamp]("stddev"), expression[StddevPop]("stddev_pop"), expression[StddevSamp]("stddev_samp"), 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 2d5a970c12006..13d18fdec0e9d 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 @@ -517,6 +517,18 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // This test uses CREATE EXTERNAL TABLE without specifying LOCATION "alter2", + // [SPARK-16248][SQL] Whitelist the list of Hive fallback functions + "udf_field", + "udf_reflect2", + "udf_xpath", + "udf_xpath_boolean", + "udf_xpath_double", + "udf_xpath_float", + "udf_xpath_int", + "udf_xpath_long", + "udf_xpath_short", + "udf_xpath_string", + // These tests DROP TABLE that don't exist (but do not specify IF EXISTS) "alter_rename_partition1", "date_1", @@ -1004,7 +1016,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_elt", "udf_equal", "udf_exp", - "udf_field", "udf_find_in_set", "udf_float", "udf_floor", @@ -1049,7 +1060,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_power", "udf_radians", "udf_rand", - "udf_reflect2", "udf_regexp", "udf_regexp_extract", "udf_regexp_replace", @@ -1090,14 +1100,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_variance", "udf_weekofyear", "udf_when", - "udf_xpath", - "udf_xpath_boolean", - "udf_xpath_double", - "udf_xpath_float", - "udf_xpath_int", - "udf_xpath_long", - "udf_xpath_short", - "udf_xpath_string", "union10", "union11", "union13", diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index 6c3978154d4b6..7ba5790c2979d 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -534,31 +534,6 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte | rows between 2 preceding and 2 following); """.stripMargin, reset = false) - // collect_set() output array in an arbitrary order, hence causes different result - // when running this test suite under Java 7 and 8. - // We change the original sql query a little bit for making the test suite passed - // under different JDK - /* Disabled because: - - Spark uses a different default stddev. - - Tiny numerical differences in stddev results. - createQueryTest("windowing.q -- 20. testSTATs", - """ - |select p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp - |from ( - |select p_mfgr,p_name, p_size, - |stddev(p_retailprice) over w1 as sdev, - |stddev_pop(p_retailprice) over w1 as sdev_pop, - |collect_set(p_size) over w1 as uniq_size, - |variance(p_retailprice) over w1 as var, - |corr(p_size, p_retailprice) over w1 as cor, - |covar_pop(p_size, p_retailprice) over w1 as covarp - |from part - |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name - | rows between 2 preceding and 2 following) - |) t lateral view explode(uniq_size) d as uniq_data - |order by p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp - """.stripMargin, reset = false) - */ createQueryTest("windowing.q -- 21. testDISTs", """ |select p_mfgr,p_name, p_size, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 2f6a2207855ec..8a47dcf908030 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -162,17 +162,6 @@ private[sql] class HiveSessionCatalog( } } - // We have a list of Hive built-in functions that we do not support. So, we will check - // Hive's function registry and lazily load needed functions into our own function registry. - // Those Hive built-in functions are - // assert_true, collect_list, collect_set, compute_stats, context_ngrams, create_union, - // current_user ,elt, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field, - // histogram_numeric, in_file, index, inline, java_method, map_keys, map_values, - // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming, - // parse_url, parse_url_tuple, percentile, percentile_approx, posexplode, reflect, reflect2, - // regexp, sentences, stack, std, str_to_map, windowingtablefunction, xpath, xpath_boolean, - // xpath_double, xpath_float, xpath_int, xpath_long, xpath_number, - // xpath_short, and xpath_string. override def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { // TODO: Once lookupFunction accepts a FunctionIdentifier, we should refactor this method to // if (super.functionExists(name)) { @@ -196,10 +185,12 @@ private[sql] class HiveSessionCatalog( // built-in function. // Hive is case insensitive. val functionName = funcName.unquotedString.toLowerCase - // TODO: This may not really work for current_user because current_user is not evaluated - // with session info. - // We do not need to use executionHive at here because we only load - // Hive's builtin functions, which do not need current db. + if (!hiveFunctions.contains(functionName)) { + failFunctionLookup(funcName.unquotedString) + } + + // TODO: Remove this fallback path once we implement the list of fallback functions + // defined below in hiveFunctions. val functionInfo = { try { Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse( @@ -221,4 +212,25 @@ private[sql] class HiveSessionCatalog( } } } + + /** List of functions we pass over to Hive. Note that over time this list should go to 0. */ + // We have a list of Hive built-in functions that we do not support. So, we will check + // Hive's function registry and lazily load needed functions into our own function registry. + // List of functions we are explicitly not supporting are: + // compute_stats, context_ngrams, create_union, + // current_user, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field, + // in_file, index, java_method, + // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming, + // parse_url_tuple, posexplode, reflect2, + // str_to_map, windowingtablefunction. + private val hiveFunctions = Seq( + "elt", "hash", "java_method", "histogram_numeric", + "map_keys", "map_values", + "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "stack", "str_to_map", + "xpath", "xpath_boolean", "xpath_double", "xpath_float", "xpath_int", "xpath_long", + "xpath_number", "xpath_short", "xpath_string", + + // table generating function + "inline", "posexplode" + ) } From 22b4072e704f9a68a605e9a4cebf54d2122fe448 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 28 Jun 2016 19:53:07 -0700 Subject: [PATCH 0838/1470] [SPARK-16245][ML] model loading backward compatibility for ml.feature.PCA ## What changes were proposed in this pull request? model loading backward compatibility for ml.feature.PCA. ## How was this patch tested? existing ut and manual test for loading models saved by Spark 1.6. Author: Yanbo Liang Closes #13937 from yanboliang/spark-16245. (cherry picked from commit 0df5ce1bc1387a58b33cd185008f4022bd3dcc69) Signed-off-by: Xiangrui Meng --- .../org/apache/spark/ml/feature/PCA.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 72167b50e384c..ef8b08545db2a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -206,24 +206,22 @@ object PCAModel extends MLReadable[PCAModel] { override def load(path: String): PCAModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - // explainedVariance field is not present in Spark <= 1.6 - val versionRegex = "([0-9]+)\\.([0-9]+).*".r - val hasExplainedVariance = metadata.sparkVersion match { - case versionRegex(major, minor) => - major.toInt >= 2 || (major.toInt == 1 && minor.toInt > 6) - case _ => false - } + val versionRegex = "([0-9]+)\\.(.+)".r + val versionRegex(major, _) = metadata.sparkVersion val dataPath = new Path(path, "data").toString - val model = if (hasExplainedVariance) { + val model = if (major.toInt >= 2) { val Row(pc: DenseMatrix, explainedVariance: DenseVector) = sparkSession.read.parquet(dataPath) .select("pc", "explainedVariance") .head() new PCAModel(metadata.uid, pc, explainedVariance) } else { - val Row(pc: DenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() - new PCAModel(metadata.uid, pc, Vectors.dense(Array.empty[Double]).asInstanceOf[DenseVector]) + // pc field is the old matrix format in Spark <= 1.6 + // explainedVariance field is not present in Spark <= 1.6 + val Row(pc: OldDenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() + new PCAModel(metadata.uid, pc.asML, + Vectors.dense(Array.empty[Double]).asInstanceOf[DenseVector]) } DefaultParamsReader.getAndSetParams(model, metadata) model From 345212b9fc91638f6cda8519ddbfec6a780854c1 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 28 Jun 2016 13:43:59 -0700 Subject: [PATCH 0839/1470] [SPARK-16259][PYSPARK] cleanup options in DataFrame read/write API ## What changes were proposed in this pull request? There are some duplicated code for options in DataFrame reader/writer API, this PR clean them up, it also fix a bug for `escapeQuotes` of csv(). ## How was this patch tested? Existing tests. Author: Davies Liu Closes #13948 from davies/csv_options. --- python/pyspark/sql/readwriter.py | 119 ++++++------------------------- 1 file changed, 20 insertions(+), 99 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index ccbf895c2d888..3f28d7ad50512 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -44,84 +44,20 @@ def to_str(value): return str(value) -class ReaderUtils(object): +class OptionUtils(object): - def _set_json_opts(self, schema, primitivesAsString, prefersDecimal, - allowComments, allowUnquotedFieldNames, allowSingleQuotes, - allowNumericLeadingZero, allowBackslashEscapingAnyCharacter, - mode, columnNameOfCorruptRecord): + def _set_opts(self, schema=None, **options): """ - Set options based on the Json optional parameters + Set named options (filter out those the value is None) """ if schema is not None: self.schema(schema) - if primitivesAsString is not None: - self.option("primitivesAsString", primitivesAsString) - if prefersDecimal is not None: - self.option("prefersDecimal", prefersDecimal) - if allowComments is not None: - self.option("allowComments", allowComments) - if allowUnquotedFieldNames is not None: - self.option("allowUnquotedFieldNames", allowUnquotedFieldNames) - if allowSingleQuotes is not None: - self.option("allowSingleQuotes", allowSingleQuotes) - if allowNumericLeadingZero is not None: - self.option("allowNumericLeadingZero", allowNumericLeadingZero) - if allowBackslashEscapingAnyCharacter is not None: - self.option("allowBackslashEscapingAnyCharacter", allowBackslashEscapingAnyCharacter) - if mode is not None: - self.option("mode", mode) - if columnNameOfCorruptRecord is not None: - self.option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) - - def _set_csv_opts(self, schema, sep, encoding, quote, escape, - comment, header, inferSchema, ignoreLeadingWhiteSpace, - ignoreTrailingWhiteSpace, nullValue, nanValue, positiveInf, negativeInf, - dateFormat, maxColumns, maxCharsPerColumn, maxMalformedLogPerPartition, mode): - """ - Set options based on the CSV optional parameters - """ - if schema is not None: - self.schema(schema) - if sep is not None: - self.option("sep", sep) - if encoding is not None: - self.option("encoding", encoding) - if quote is not None: - self.option("quote", quote) - if escape is not None: - self.option("escape", escape) - if comment is not None: - self.option("comment", comment) - if header is not None: - self.option("header", header) - if inferSchema is not None: - self.option("inferSchema", inferSchema) - if ignoreLeadingWhiteSpace is not None: - self.option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) - if ignoreTrailingWhiteSpace is not None: - self.option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) - if nullValue is not None: - self.option("nullValue", nullValue) - if nanValue is not None: - self.option("nanValue", nanValue) - if positiveInf is not None: - self.option("positiveInf", positiveInf) - if negativeInf is not None: - self.option("negativeInf", negativeInf) - if dateFormat is not None: - self.option("dateFormat", dateFormat) - if maxColumns is not None: - self.option("maxColumns", maxColumns) - if maxCharsPerColumn is not None: - self.option("maxCharsPerColumn", maxCharsPerColumn) - if maxMalformedLogPerPartition is not None: - self.option("maxMalformedLogPerPartition", maxMalformedLogPerPartition) - if mode is not None: - self.option("mode", mode) - - -class DataFrameReader(ReaderUtils): + for k, v in options.items(): + if v is not None: + self.option(k, v) + + +class DataFrameReader(OptionUtils): """ Interface used to load a :class:`DataFrame` from external storage systems (e.g. file systems, key-value stores, etc). Use :func:`spark.read` @@ -270,7 +206,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, [('age', 'bigint'), ('name', 'string')] """ - self._set_json_opts( + self._set_opts( schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, @@ -413,7 +349,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non >>> df.dtypes [('_c0', 'string'), ('_c1', 'string')] """ - self._set_csv_opts( + self._set_opts( schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, @@ -484,7 +420,7 @@ def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPar return self._df(self._jreader.jdbc(url, table, jprop)) -class DataFrameWriter(object): +class DataFrameWriter(OptionUtils): """ Interface used to write a :class:`DataFrame` to external storage systems (e.g. file systems, key-value stores, etc). Use :func:`DataFrame.write` @@ -649,8 +585,7 @@ def json(self, path, mode=None, compression=None): >>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) - if compression is not None: - self.option("compression", compression) + self._set_opts(compression=compression) self._jwrite.json(path) @since(1.4) @@ -676,8 +611,7 @@ def parquet(self, path, mode=None, partitionBy=None, compression=None): self.mode(mode) if partitionBy is not None: self.partitionBy(partitionBy) - if compression is not None: - self.option("compression", compression) + self._set_opts(compression=compression) self._jwrite.parquet(path) @since(1.6) @@ -692,8 +626,7 @@ def text(self, path, compression=None): The DataFrame must have only one column that is of string type. Each row becomes a new line in the output file. """ - if compression is not None: - self.option("compression", compression) + self._set_opts(compression=compression) self._jwrite.text(path) @since(2.0) @@ -731,20 +664,8 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) - if compression is not None: - self.option("compression", compression) - if sep is not None: - self.option("sep", sep) - if quote is not None: - self.option("quote", quote) - if escape is not None: - self.option("escape", escape) - if header is not None: - self.option("header", header) - if nullValue is not None: - self.option("nullValue", nullValue) - if escapeQuotes is not None: - self.option("escapeQuotes", nullValue) + self._set_opts(compression=compression, sep=sep, quote=quote, escape=escape, header=header, + nullValue=nullValue, escapeQuotes=escapeQuotes) self._jwrite.csv(path) @since(1.5) @@ -803,7 +724,7 @@ def jdbc(self, url, table, mode=None, properties=None): self._jwrite.mode(mode).jdbc(url, table, jprop) -class DataStreamReader(ReaderUtils): +class DataStreamReader(OptionUtils): """ Interface used to load a streaming :class:`DataFrame` from external storage systems (e.g. file systems, key-value stores, etc). Use :func:`spark.readStream` @@ -965,7 +886,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, >>> json_sdf.schema == sdf_schema True """ - self._set_json_opts( + self._set_opts( schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, @@ -1095,7 +1016,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non >>> csv_sdf.schema == sdf_schema True """ - self._set_csv_opts( + self._set_opts( schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, From 6650c0533e5c60f8653d2e0a608a42d5838fa553 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 28 Jun 2016 22:07:11 -0700 Subject: [PATCH 0840/1470] [SPARK-16266][SQL][STREAING] Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming ## What changes were proposed in this pull request? - Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming to make them consistent with scala packaging - Exposed the necessary classes in sql.streaming package so that they appear in the docs - Added pyspark.sql.streaming module to the docs ## How was this patch tested? - updated unit tests. - generated docs for testing visibility of pyspark.sql.streaming classes. Author: Tathagata Das Closes #13955 from tdas/SPARK-16266. --- python/docs/pyspark.sql.rst | 6 + python/pyspark/sql/context.py | 3 +- python/pyspark/sql/dataframe.py | 3 +- python/pyspark/sql/readwriter.py | 493 +----------------------------- python/pyspark/sql/session.py | 3 +- python/pyspark/sql/streaming.py | 502 ++++++++++++++++++++++++++++++- 6 files changed, 511 insertions(+), 499 deletions(-) diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst index 6259379ed05b7..3be9533c126d2 100644 --- a/python/docs/pyspark.sql.rst +++ b/python/docs/pyspark.sql.rst @@ -21,3 +21,9 @@ pyspark.sql.functions module .. automodule:: pyspark.sql.functions :members: :undoc-members: + +pyspark.sql.streaming module +---------------------------- +.. automodule:: pyspark.sql.streaming + :members: + :undoc-members: diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index b5dde13ed7cc2..3503fb90c3f8d 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -26,7 +26,8 @@ from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.session import _monkey_patch_RDD, SparkSession from pyspark.sql.dataframe import DataFrame -from pyspark.sql.readwriter import DataFrameReader, DataStreamReader +from pyspark.sql.readwriter import DataFrameReader +from pyspark.sql.streaming import DataStreamReader from pyspark.sql.types import Row, StringType from pyspark.sql.utils import install_exception_handler diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c8c8e7d790356..e6e70291da5b3 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -33,7 +33,8 @@ from pyspark.traceback_utils import SCCallSiteSync from pyspark.sql.types import _parse_datatype_json_string from pyspark.sql.column import Column, _to_seq, _to_list, _to_java_column -from pyspark.sql.readwriter import DataFrameWriter, DataStreamWriter +from pyspark.sql.readwriter import DataFrameWriter +from pyspark.sql.streaming import DataStreamWriter from pyspark.sql.types import * __all__ = ["DataFrame", "DataFrameNaFunctions", "DataFrameStatFunctions"] diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 3f28d7ad50512..10f307b987dad 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -28,7 +28,7 @@ from pyspark.sql.types import * from pyspark.sql import utils -__all__ = ["DataFrameReader", "DataFrameWriter", "DataStreamReader", "DataStreamWriter"] +__all__ = ["DataFrameReader", "DataFrameWriter"] def to_str(value): @@ -724,494 +724,6 @@ def jdbc(self, url, table, mode=None, properties=None): self._jwrite.mode(mode).jdbc(url, table, jprop) -class DataStreamReader(OptionUtils): - """ - Interface used to load a streaming :class:`DataFrame` from external storage systems - (e.g. file systems, key-value stores, etc). Use :func:`spark.readStream` - to access this. - - .. note:: Experimental. - - .. versionadded:: 2.0 - """ - - def __init__(self, spark): - self._jreader = spark._ssql_ctx.readStream() - self._spark = spark - - def _df(self, jdf): - from pyspark.sql.dataframe import DataFrame - return DataFrame(jdf, self._spark) - - @since(2.0) - def format(self, source): - """Specifies the input data source format. - - .. note:: Experimental. - - :param source: string, name of the data source, e.g. 'json', 'parquet'. - - >>> s = spark.readStream.format("text") - """ - self._jreader = self._jreader.format(source) - return self - - @since(2.0) - def schema(self, schema): - """Specifies the input schema. - - Some data sources (e.g. JSON) can infer the input schema automatically from data. - By specifying the schema here, the underlying data source can skip the schema - inference step, and thus speed up data loading. - - .. note:: Experimental. - - :param schema: a StructType object - - >>> s = spark.readStream.schema(sdf_schema) - """ - if not isinstance(schema, StructType): - raise TypeError("schema should be StructType") - jschema = self._spark._ssql_ctx.parseDataType(schema.json()) - self._jreader = self._jreader.schema(jschema) - return self - - @since(2.0) - def option(self, key, value): - """Adds an input option for the underlying data source. - - .. note:: Experimental. - - >>> s = spark.readStream.option("x", 1) - """ - self._jreader = self._jreader.option(key, to_str(value)) - return self - - @since(2.0) - def options(self, **options): - """Adds input options for the underlying data source. - - .. note:: Experimental. - - >>> s = spark.readStream.options(x="1", y=2) - """ - for k in options: - self._jreader = self._jreader.option(k, to_str(options[k])) - return self - - @since(2.0) - def load(self, path=None, format=None, schema=None, **options): - """Loads a data stream from a data source and returns it as a :class`DataFrame`. - - .. note:: Experimental. - - :param path: optional string for file-system backed data sources. - :param format: optional string for format of the data source. Default to 'parquet'. - :param schema: optional :class:`StructType` for the input schema. - :param options: all other string options - - >>> json_sdf = spark.readStream.format("json")\ - .schema(sdf_schema)\ - .load(os.path.join(tempfile.mkdtemp(),'data')) - >>> json_sdf.isStreaming - True - >>> json_sdf.schema == sdf_schema - True - """ - if format is not None: - self.format(format) - if schema is not None: - self.schema(schema) - self.options(**options) - if path is not None: - if type(path) != str or len(path.strip()) == 0: - raise ValueError("If the path is provided for stream, it needs to be a " + - "non-empty string. List of paths are not supported.") - return self._df(self._jreader.load(path)) - else: - return self._df(self._jreader.load()) - - @since(2.0) - def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, - allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, - allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, - mode=None, columnNameOfCorruptRecord=None): - """ - Loads a JSON file stream (one object per line) and returns a :class`DataFrame`. - - If the ``schema`` parameter is not specified, this function goes - through the input once to determine the input schema. - - .. note:: Experimental. - - :param path: string represents path to the JSON dataset, - or RDD of Strings storing JSON objects. - :param schema: an optional :class:`StructType` for the input schema. - :param primitivesAsString: infers all primitive values as a string type. If None is set, - it uses the default value, ``false``. - :param prefersDecimal: infers all floating-point values as a decimal type. If the values - do not fit in decimal, then it infers them as doubles. If None is - set, it uses the default value, ``false``. - :param allowComments: ignores Java/C++ style comment in JSON records. If None is set, - it uses the default value, ``false``. - :param allowUnquotedFieldNames: allows unquoted JSON field names. If None is set, - it uses the default value, ``false``. - :param allowSingleQuotes: allows single quotes in addition to double quotes. If None is - set, it uses the default value, ``true``. - :param allowNumericLeadingZero: allows leading zeros in numbers (e.g. 00012). If None is - set, it uses the default value, ``false``. - :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character - using backslash quoting mechanism. If None is - set, it uses the default value, ``false``. - :param mode: allows a mode for dealing with corrupt records during parsing. If None is - set, it uses the default value, ``PERMISSIVE``. - - * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted \ - record and puts the malformed string into a new field configured by \ - ``columnNameOfCorruptRecord``. When a schema is set by user, it sets \ - ``null`` for extra fields. - * ``DROPMALFORMED`` : ignores the whole corrupted records. - * ``FAILFAST`` : throws an exception when it meets corrupted records. - - :param columnNameOfCorruptRecord: allows renaming the new field having malformed string - created by ``PERMISSIVE`` mode. This overrides - ``spark.sql.columnNameOfCorruptRecord``. If None is set, - it uses the value specified in - ``spark.sql.columnNameOfCorruptRecord``. - - >>> json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), \ - schema = sdf_schema) - >>> json_sdf.isStreaming - True - >>> json_sdf.schema == sdf_schema - True - """ - self._set_opts( - schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, - allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, - allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, - allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, - mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord) - if isinstance(path, basestring): - return self._df(self._jreader.json(path)) - else: - raise TypeError("path can be only a single string") - - @since(2.0) - def parquet(self, path): - """Loads a Parquet file stream, returning the result as a :class:`DataFrame`. - - You can set the following Parquet-specific option(s) for reading Parquet files: - * ``mergeSchema``: sets whether we should merge schemas collected from all \ - Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ - The default value is specified in ``spark.sql.parquet.mergeSchema``. - - .. note:: Experimental. - - >>> parquet_sdf = spark.readStream.schema(sdf_schema)\ - .parquet(os.path.join(tempfile.mkdtemp())) - >>> parquet_sdf.isStreaming - True - >>> parquet_sdf.schema == sdf_schema - True - """ - if isinstance(path, basestring): - return self._df(self._jreader.parquet(path)) - else: - raise TypeError("path can be only a single string") - - @ignore_unicode_prefix - @since(2.0) - def text(self, path): - """ - Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a - string column named "value", and followed by partitioned columns if there - are any. - - Each line in the text file is a new row in the resulting DataFrame. - - .. note:: Experimental. - - :param paths: string, or list of strings, for input path(s). - - >>> text_sdf = spark.readStream.text(os.path.join(tempfile.mkdtemp(), 'data')) - >>> text_sdf.isStreaming - True - >>> "value" in str(text_sdf.schema) - True - """ - if isinstance(path, basestring): - return self._df(self._jreader.text(path)) - else: - raise TypeError("path can be only a single string") - - @since(2.0) - def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, - comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, - ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, - negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, - maxMalformedLogPerPartition=None, mode=None): - """Loads a CSV file stream and returns the result as a :class:`DataFrame`. - - This function will go through the input once to determine the input schema if - ``inferSchema`` is enabled. To avoid going through the entire data once, disable - ``inferSchema`` option or specify the schema explicitly using ``schema``. - - .. note:: Experimental. - - :param path: string, or list of strings, for input path(s). - :param schema: an optional :class:`StructType` for the input schema. - :param sep: sets the single character as a separator for each field and value. - If None is set, it uses the default value, ``,``. - :param encoding: decodes the CSV files by the given encoding type. If None is set, - it uses the default value, ``UTF-8``. - :param quote: sets the single character used for escaping quoted values where the - separator can be part of the value. If None is set, it uses the default - value, ``"``. If you would like to turn off quotations, you need to set an - empty string. - :param escape: sets the single character used for escaping quotes inside an already - quoted value. If None is set, it uses the default value, ``\``. - :param comment: sets the single character used for skipping lines beginning with this - character. By default (None), it is disabled. - :param header: uses the first line as names of columns. If None is set, it uses the - default value, ``false``. - :param inferSchema: infers the input schema automatically from data. It requires one extra - pass over the data. If None is set, it uses the default value, ``false``. - :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values - being read should be skipped. If None is set, it uses - the default value, ``false``. - :param ignoreTrailingWhiteSpace: defines whether or not trailing whitespaces from values - being read should be skipped. If None is set, it uses - the default value, ``false``. - :param nullValue: sets the string representation of a null value. If None is set, it uses - the default value, empty string. - :param nanValue: sets the string representation of a non-number value. If None is set, it - uses the default value, ``NaN``. - :param positiveInf: sets the string representation of a positive infinity value. If None - is set, it uses the default value, ``Inf``. - :param negativeInf: sets the string representation of a negative infinity value. If None - is set, it uses the default value, ``Inf``. - :param dateFormat: sets the string that indicates a date format. Custom date formats - follow the formats at ``java.text.SimpleDateFormat``. This - applies to both date type and timestamp type. By default, it is None - which means trying to parse times and date by - ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. - :param maxColumns: defines a hard limit of how many columns a record can have. If None is - set, it uses the default value, ``20480``. - :param maxCharsPerColumn: defines the maximum number of characters allowed for any given - value being read. If None is set, it uses the default value, - ``1000000``. - :param mode: allows a mode for dealing with corrupt records during parsing. If None is - set, it uses the default value, ``PERMISSIVE``. - - * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. - When a schema is set by user, it sets ``null`` for extra fields. - * ``DROPMALFORMED`` : ignores the whole corrupted records. - * ``FAILFAST`` : throws an exception when it meets corrupted records. - - >>> csv_sdf = spark.readStream.csv(os.path.join(tempfile.mkdtemp(), 'data'), \ - schema = sdf_schema) - >>> csv_sdf.isStreaming - True - >>> csv_sdf.schema == sdf_schema - True - """ - self._set_opts( - schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, - header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, - ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, - nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, - dateFormat=dateFormat, maxColumns=maxColumns, maxCharsPerColumn=maxCharsPerColumn, - maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode) - if isinstance(path, basestring): - return self._df(self._jreader.csv(path)) - else: - raise TypeError("path can be only a single string") - - -class DataStreamWriter(object): - """ - Interface used to write a streaming :class:`DataFrame` to external storage systems - (e.g. file systems, key-value stores, etc). Use :func:`DataFrame.writeStream` - to access this. - - .. note:: Experimental. - - .. versionadded:: 2.0 - """ - - def __init__(self, df): - self._df = df - self._spark = df.sql_ctx - self._jwrite = df._jdf.writeStream() - - def _sq(self, jsq): - from pyspark.sql.streaming import StreamingQuery - return StreamingQuery(jsq) - - @since(2.0) - def outputMode(self, outputMode): - """Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - - Options include: - - * `append`:Only the new rows in the streaming DataFrame/Dataset will be written to - the sink - * `complete`:All the rows in the streaming DataFrame/Dataset will be written to the sink - every time these is some updates - - .. note:: Experimental. - - >>> writer = sdf.writeStream.outputMode('append') - """ - if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0: - raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode) - self._jwrite = self._jwrite.outputMode(outputMode) - return self - - @since(2.0) - def format(self, source): - """Specifies the underlying output data source. - - .. note:: Experimental. - - :param source: string, name of the data source, e.g. 'json', 'parquet'. - - >>> writer = sdf.writeStream.format('json') - """ - self._jwrite = self._jwrite.format(source) - return self - - @since(2.0) - def option(self, key, value): - """Adds an output option for the underlying data source. - - .. note:: Experimental. - """ - self._jwrite = self._jwrite.option(key, to_str(value)) - return self - - @since(2.0) - def options(self, **options): - """Adds output options for the underlying data source. - - .. note:: Experimental. - """ - for k in options: - self._jwrite = self._jwrite.option(k, to_str(options[k])) - return self - - @since(2.0) - def partitionBy(self, *cols): - """Partitions the output by the given columns on the file system. - - If specified, the output is laid out on the file system similar - to Hive's partitioning scheme. - - .. note:: Experimental. - - :param cols: name of columns - - """ - if len(cols) == 1 and isinstance(cols[0], (list, tuple)): - cols = cols[0] - self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols)) - return self - - @since(2.0) - def queryName(self, queryName): - """Specifies the name of the :class:`StreamingQuery` that can be started with - :func:`start`. This name must be unique among all the currently active queries - in the associated SparkSession. - - .. note:: Experimental. - - :param queryName: unique name for the query - - >>> writer = sdf.writeStream.queryName('streaming_query') - """ - if not queryName or type(queryName) != str or len(queryName.strip()) == 0: - raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName) - self._jwrite = self._jwrite.queryName(queryName) - return self - - @keyword_only - @since(2.0) - def trigger(self, processingTime=None): - """Set the trigger for the stream query. If this is not set it will run the query as fast - as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``. - - .. note:: Experimental. - - :param processingTime: a processing time interval as a string, e.g. '5 seconds', '1 minute'. - - >>> # trigger the query for execution every 5 seconds - >>> writer = sdf.writeStream.trigger(processingTime='5 seconds') - """ - from pyspark.sql.streaming import ProcessingTime - trigger = None - if processingTime is not None: - if type(processingTime) != str or len(processingTime.strip()) == 0: - raise ValueError('The processing time must be a non empty string. Got: %s' % - processingTime) - trigger = ProcessingTime(processingTime) - if trigger is None: - raise ValueError('A trigger was not provided. Supported triggers: processingTime.') - self._jwrite = self._jwrite.trigger(trigger._to_java_trigger(self._spark)) - return self - - @ignore_unicode_prefix - @since(2.0) - def start(self, path=None, format=None, partitionBy=None, queryName=None, **options): - """Streams the contents of the :class:`DataFrame` to a data source. - - The data source is specified by the ``format`` and a set of ``options``. - If ``format`` is not specified, the default data source configured by - ``spark.sql.sources.default`` will be used. - - .. note:: Experimental. - - :param path: the path in a Hadoop supported file system - :param format: the format used to save - - * ``append``: Append contents of this :class:`DataFrame` to existing data. - * ``overwrite``: Overwrite existing data. - * ``ignore``: Silently ignore this operation if data already exists. - * ``error`` (default case): Throw an exception if data already exists. - :param partitionBy: names of partitioning columns - :param queryName: unique name for the query - :param options: All other string options. You may want to provide a `checkpointLocation` - for most streams, however it is not required for a `memory` stream. - - >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() - >>> sq.isActive - True - >>> sq.name - u'this_query' - >>> sq.stop() - >>> sq.isActive - False - >>> sq = sdf.writeStream.trigger(processingTime='5 seconds').start( - ... queryName='that_query', format='memory') - >>> sq.name - u'that_query' - >>> sq.isActive - True - >>> sq.stop() - """ - self.options(**options) - if partitionBy is not None: - self.partitionBy(partitionBy) - if format is not None: - self.format(format) - if queryName is not None: - self.queryName(queryName) - if path is None: - return self._sq(self._jwrite.start()) - else: - return self._sq(self._jwrite.start(path)) - - def _test(): import doctest import os @@ -1235,9 +747,6 @@ def _test(): globs['sc'] = sc globs['spark'] = spark globs['df'] = spark.read.parquet('python/test_support/sql/parquet_partitioned') - globs['sdf'] = \ - spark.readStream.format('text').load('python/test_support/sql/streaming') - globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) (failure_count, test_count) = doctest.testmod( pyspark.sql.readwriter, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index b4152a34ad97e..55f86a16f50a2 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -31,7 +31,8 @@ from pyspark.sql.catalog import Catalog from pyspark.sql.conf import RuntimeConfig from pyspark.sql.dataframe import DataFrame -from pyspark.sql.readwriter import DataFrameReader, DataStreamReader +from pyspark.sql.readwriter import DataFrameReader +from pyspark.sql.streaming import DataStreamReader from pyspark.sql.types import Row, DataType, StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _parse_datatype_string from pyspark.sql.utils import install_exception_handler diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index ae45c99e4f352..8cf70983a4514 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -18,15 +18,18 @@ import sys if sys.version >= '3': intlike = int + basestring = unicode = str else: intlike = (int, long) from abc import ABCMeta, abstractmethod -from pyspark import since +from pyspark import since, keyword_only from pyspark.rdd import ignore_unicode_prefix +from pyspark.sql.readwriter import OptionUtils, to_str +from pyspark.sql.types import * -__all__ = ["StreamingQuery"] +__all__ = ["StreamingQuery", "StreamingQueryManager", "DataStreamReader", "DataStreamWriter"] class StreamingQuery(object): @@ -118,7 +121,7 @@ def __init__(self, jsqm): def active(self): """Returns a list of active queries associated with this SQLContext - >>> sq = df.writeStream.format('memory').queryName('this_query').start() + >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() >>> sqm = spark.streams >>> # get the list of active streaming queries >>> [q.name for q in sqm.active] @@ -133,7 +136,7 @@ def get(self, id): """Returns an active query from this SQLContext or throws exception if an active query with this name doesn't exist. - >>> sq = df.writeStream.format('memory').queryName('this_query').start() + >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() >>> sq.name u'this_query' >>> sq = spark.streams.get(sq.id) @@ -224,6 +227,494 @@ def _to_java_trigger(self, sqlContext): self.interval) +class DataStreamReader(OptionUtils): + """ + Interface used to load a streaming :class:`DataFrame` from external storage systems + (e.g. file systems, key-value stores, etc). Use :func:`spark.readStream` + to access this. + + .. note:: Experimental. + + .. versionadded:: 2.0 + """ + + def __init__(self, spark): + self._jreader = spark._ssql_ctx.readStream() + self._spark = spark + + def _df(self, jdf): + from pyspark.sql.dataframe import DataFrame + return DataFrame(jdf, self._spark) + + @since(2.0) + def format(self, source): + """Specifies the input data source format. + + .. note:: Experimental. + + :param source: string, name of the data source, e.g. 'json', 'parquet'. + + >>> s = spark.readStream.format("text") + """ + self._jreader = self._jreader.format(source) + return self + + @since(2.0) + def schema(self, schema): + """Specifies the input schema. + + Some data sources (e.g. JSON) can infer the input schema automatically from data. + By specifying the schema here, the underlying data source can skip the schema + inference step, and thus speed up data loading. + + .. note:: Experimental. + + :param schema: a StructType object + + >>> s = spark.readStream.schema(sdf_schema) + """ + if not isinstance(schema, StructType): + raise TypeError("schema should be StructType") + jschema = self._spark._ssql_ctx.parseDataType(schema.json()) + self._jreader = self._jreader.schema(jschema) + return self + + @since(2.0) + def option(self, key, value): + """Adds an input option for the underlying data source. + + .. note:: Experimental. + + >>> s = spark.readStream.option("x", 1) + """ + self._jreader = self._jreader.option(key, to_str(value)) + return self + + @since(2.0) + def options(self, **options): + """Adds input options for the underlying data source. + + .. note:: Experimental. + + >>> s = spark.readStream.options(x="1", y=2) + """ + for k in options: + self._jreader = self._jreader.option(k, to_str(options[k])) + return self + + @since(2.0) + def load(self, path=None, format=None, schema=None, **options): + """Loads a data stream from a data source and returns it as a :class`DataFrame`. + + .. note:: Experimental. + + :param path: optional string for file-system backed data sources. + :param format: optional string for format of the data source. Default to 'parquet'. + :param schema: optional :class:`StructType` for the input schema. + :param options: all other string options + + >>> json_sdf = spark.readStream.format("json")\ + .schema(sdf_schema)\ + .load(os.path.join(tempfile.mkdtemp(),'data')) + >>> json_sdf.isStreaming + True + >>> json_sdf.schema == sdf_schema + True + """ + if format is not None: + self.format(format) + if schema is not None: + self.schema(schema) + self.options(**options) + if path is not None: + if type(path) != str or len(path.strip()) == 0: + raise ValueError("If the path is provided for stream, it needs to be a " + + "non-empty string. List of paths are not supported.") + return self._df(self._jreader.load(path)) + else: + return self._df(self._jreader.load()) + + @since(2.0) + def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, + allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, + allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, + mode=None, columnNameOfCorruptRecord=None): + """ + Loads a JSON file stream (one object per line) and returns a :class`DataFrame`. + + If the ``schema`` parameter is not specified, this function goes + through the input once to determine the input schema. + + .. note:: Experimental. + + :param path: string represents path to the JSON dataset, + or RDD of Strings storing JSON objects. + :param schema: an optional :class:`StructType` for the input schema. + :param primitivesAsString: infers all primitive values as a string type. If None is set, + it uses the default value, ``false``. + :param prefersDecimal: infers all floating-point values as a decimal type. If the values + do not fit in decimal, then it infers them as doubles. If None is + set, it uses the default value, ``false``. + :param allowComments: ignores Java/C++ style comment in JSON records. If None is set, + it uses the default value, ``false``. + :param allowUnquotedFieldNames: allows unquoted JSON field names. If None is set, + it uses the default value, ``false``. + :param allowSingleQuotes: allows single quotes in addition to double quotes. If None is + set, it uses the default value, ``true``. + :param allowNumericLeadingZero: allows leading zeros in numbers (e.g. 00012). If None is + set, it uses the default value, ``false``. + :param allowBackslashEscapingAnyCharacter: allows accepting quoting of all character + using backslash quoting mechanism. If None is + set, it uses the default value, ``false``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. + + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted \ + record and puts the malformed string into a new field configured by \ + ``columnNameOfCorruptRecord``. When a schema is set by user, it sets \ + ``null`` for extra fields. + * ``DROPMALFORMED`` : ignores the whole corrupted records. + * ``FAILFAST`` : throws an exception when it meets corrupted records. + + :param columnNameOfCorruptRecord: allows renaming the new field having malformed string + created by ``PERMISSIVE`` mode. This overrides + ``spark.sql.columnNameOfCorruptRecord``. If None is set, + it uses the value specified in + ``spark.sql.columnNameOfCorruptRecord``. + + >>> json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), \ + schema = sdf_schema) + >>> json_sdf.isStreaming + True + >>> json_sdf.schema == sdf_schema + True + """ + self._set_opts( + schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal, + allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, + allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, + allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, + mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord) + if isinstance(path, basestring): + return self._df(self._jreader.json(path)) + else: + raise TypeError("path can be only a single string") + + @since(2.0) + def parquet(self, path): + """Loads a Parquet file stream, returning the result as a :class:`DataFrame`. + + You can set the following Parquet-specific option(s) for reading Parquet files: + * ``mergeSchema``: sets whether we should merge schemas collected from all \ + Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ + The default value is specified in ``spark.sql.parquet.mergeSchema``. + + .. note:: Experimental. + + >>> parquet_sdf = spark.readStream.schema(sdf_schema)\ + .parquet(os.path.join(tempfile.mkdtemp())) + >>> parquet_sdf.isStreaming + True + >>> parquet_sdf.schema == sdf_schema + True + """ + if isinstance(path, basestring): + return self._df(self._jreader.parquet(path)) + else: + raise TypeError("path can be only a single string") + + @ignore_unicode_prefix + @since(2.0) + def text(self, path): + """ + Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a + string column named "value", and followed by partitioned columns if there + are any. + + Each line in the text file is a new row in the resulting DataFrame. + + .. note:: Experimental. + + :param paths: string, or list of strings, for input path(s). + + >>> text_sdf = spark.readStream.text(os.path.join(tempfile.mkdtemp(), 'data')) + >>> text_sdf.isStreaming + True + >>> "value" in str(text_sdf.schema) + True + """ + if isinstance(path, basestring): + return self._df(self._jreader.text(path)) + else: + raise TypeError("path can be only a single string") + + @since(2.0) + def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, + comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, + ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, + negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, + maxMalformedLogPerPartition=None, mode=None): + """Loads a CSV file stream and returns the result as a :class:`DataFrame`. + + This function will go through the input once to determine the input schema if + ``inferSchema`` is enabled. To avoid going through the entire data once, disable + ``inferSchema`` option or specify the schema explicitly using ``schema``. + + .. note:: Experimental. + + :param path: string, or list of strings, for input path(s). + :param schema: an optional :class:`StructType` for the input schema. + :param sep: sets the single character as a separator for each field and value. + If None is set, it uses the default value, ``,``. + :param encoding: decodes the CSV files by the given encoding type. If None is set, + it uses the default value, ``UTF-8``. + :param quote: sets the single character used for escaping quoted values where the + separator can be part of the value. If None is set, it uses the default + value, ``"``. If you would like to turn off quotations, you need to set an + empty string. + :param escape: sets the single character used for escaping quotes inside an already + quoted value. If None is set, it uses the default value, ``\``. + :param comment: sets the single character used for skipping lines beginning with this + character. By default (None), it is disabled. + :param header: uses the first line as names of columns. If None is set, it uses the + default value, ``false``. + :param inferSchema: infers the input schema automatically from data. It requires one extra + pass over the data. If None is set, it uses the default value, ``false``. + :param ignoreLeadingWhiteSpace: defines whether or not leading whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param ignoreTrailingWhiteSpace: defines whether or not trailing whitespaces from values + being read should be skipped. If None is set, it uses + the default value, ``false``. + :param nullValue: sets the string representation of a null value. If None is set, it uses + the default value, empty string. + :param nanValue: sets the string representation of a non-number value. If None is set, it + uses the default value, ``NaN``. + :param positiveInf: sets the string representation of a positive infinity value. If None + is set, it uses the default value, ``Inf``. + :param negativeInf: sets the string representation of a negative infinity value. If None + is set, it uses the default value, ``Inf``. + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to both date type and timestamp type. By default, it is None + which means trying to parse times and date by + ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. + :param maxColumns: defines a hard limit of how many columns a record can have. If None is + set, it uses the default value, ``20480``. + :param maxCharsPerColumn: defines the maximum number of characters allowed for any given + value being read. If None is set, it uses the default value, + ``1000000``. + :param mode: allows a mode for dealing with corrupt records during parsing. If None is + set, it uses the default value, ``PERMISSIVE``. + + * ``PERMISSIVE`` : sets other fields to ``null`` when it meets a corrupted record. + When a schema is set by user, it sets ``null`` for extra fields. + * ``DROPMALFORMED`` : ignores the whole corrupted records. + * ``FAILFAST`` : throws an exception when it meets corrupted records. + + >>> csv_sdf = spark.readStream.csv(os.path.join(tempfile.mkdtemp(), 'data'), \ + schema = sdf_schema) + >>> csv_sdf.isStreaming + True + >>> csv_sdf.schema == sdf_schema + True + """ + self._set_opts( + schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment, + header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, + ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, + nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, + dateFormat=dateFormat, maxColumns=maxColumns, maxCharsPerColumn=maxCharsPerColumn, + maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode) + if isinstance(path, basestring): + return self._df(self._jreader.csv(path)) + else: + raise TypeError("path can be only a single string") + + +class DataStreamWriter(object): + """ + Interface used to write a streaming :class:`DataFrame` to external storage systems + (e.g. file systems, key-value stores, etc). Use :func:`DataFrame.writeStream` + to access this. + + .. note:: Experimental. + + .. versionadded:: 2.0 + """ + + def __init__(self, df): + self._df = df + self._spark = df.sql_ctx + self._jwrite = df._jdf.writeStream() + + def _sq(self, jsq): + from pyspark.sql.streaming import StreamingQuery + return StreamingQuery(jsq) + + @since(2.0) + def outputMode(self, outputMode): + """Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + + Options include: + + * `append`:Only the new rows in the streaming DataFrame/Dataset will be written to + the sink + * `complete`:All the rows in the streaming DataFrame/Dataset will be written to the sink + every time these is some updates + + .. note:: Experimental. + + >>> writer = sdf.writeStream.outputMode('append') + """ + if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0: + raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode) + self._jwrite = self._jwrite.outputMode(outputMode) + return self + + @since(2.0) + def format(self, source): + """Specifies the underlying output data source. + + .. note:: Experimental. + + :param source: string, name of the data source, e.g. 'json', 'parquet'. + + >>> writer = sdf.writeStream.format('json') + """ + self._jwrite = self._jwrite.format(source) + return self + + @since(2.0) + def option(self, key, value): + """Adds an output option for the underlying data source. + + .. note:: Experimental. + """ + self._jwrite = self._jwrite.option(key, to_str(value)) + return self + + @since(2.0) + def options(self, **options): + """Adds output options for the underlying data source. + + .. note:: Experimental. + """ + for k in options: + self._jwrite = self._jwrite.option(k, to_str(options[k])) + return self + + @since(2.0) + def partitionBy(self, *cols): + """Partitions the output by the given columns on the file system. + + If specified, the output is laid out on the file system similar + to Hive's partitioning scheme. + + .. note:: Experimental. + + :param cols: name of columns + + """ + if len(cols) == 1 and isinstance(cols[0], (list, tuple)): + cols = cols[0] + self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols)) + return self + + @since(2.0) + def queryName(self, queryName): + """Specifies the name of the :class:`StreamingQuery` that can be started with + :func:`start`. This name must be unique among all the currently active queries + in the associated SparkSession. + + .. note:: Experimental. + + :param queryName: unique name for the query + + >>> writer = sdf.writeStream.queryName('streaming_query') + """ + if not queryName or type(queryName) != str or len(queryName.strip()) == 0: + raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName) + self._jwrite = self._jwrite.queryName(queryName) + return self + + @keyword_only + @since(2.0) + def trigger(self, processingTime=None): + """Set the trigger for the stream query. If this is not set it will run the query as fast + as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``. + + .. note:: Experimental. + + :param processingTime: a processing time interval as a string, e.g. '5 seconds', '1 minute'. + + >>> # trigger the query for execution every 5 seconds + >>> writer = sdf.writeStream.trigger(processingTime='5 seconds') + """ + from pyspark.sql.streaming import ProcessingTime + trigger = None + if processingTime is not None: + if type(processingTime) != str or len(processingTime.strip()) == 0: + raise ValueError('The processing time must be a non empty string. Got: %s' % + processingTime) + trigger = ProcessingTime(processingTime) + if trigger is None: + raise ValueError('A trigger was not provided. Supported triggers: processingTime.') + self._jwrite = self._jwrite.trigger(trigger._to_java_trigger(self._spark)) + return self + + @ignore_unicode_prefix + @since(2.0) + def start(self, path=None, format=None, partitionBy=None, queryName=None, **options): + """Streams the contents of the :class:`DataFrame` to a data source. + + The data source is specified by the ``format`` and a set of ``options``. + If ``format`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. + + .. note:: Experimental. + + :param path: the path in a Hadoop supported file system + :param format: the format used to save + + * ``append``: Append contents of this :class:`DataFrame` to existing data. + * ``overwrite``: Overwrite existing data. + * ``ignore``: Silently ignore this operation if data already exists. + * ``error`` (default case): Throw an exception if data already exists. + :param partitionBy: names of partitioning columns + :param queryName: unique name for the query + :param options: All other string options. You may want to provide a `checkpointLocation` + for most streams, however it is not required for a `memory` stream. + + >>> sq = sdf.writeStream.format('memory').queryName('this_query').start() + >>> sq.isActive + True + >>> sq.name + u'this_query' + >>> sq.stop() + >>> sq.isActive + False + >>> sq = sdf.writeStream.trigger(processingTime='5 seconds').start( + ... queryName='that_query', format='memory') + >>> sq.name + u'that_query' + >>> sq.isActive + True + >>> sq.stop() + """ + self.options(**options) + if partitionBy is not None: + self.partitionBy(partitionBy) + if format is not None: + self.format(format) + if queryName is not None: + self.queryName(queryName) + if path is None: + return self._sq(self._jwrite.start()) + else: + return self._sq(self._jwrite.start(path)) + + def _test(): import doctest import os @@ -243,6 +734,9 @@ def _test(): globs['os'] = os globs['spark'] = spark globs['sqlContext'] = SQLContext.getOrCreate(spark.sparkContext) + globs['sdf'] = \ + spark.readStream.format('text').load('python/test_support/sql/streaming') + globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) globs['df'] = \ globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') From 904122335d94681be2afbaf4f41a50d468e707b9 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 29 Jun 2016 01:52:20 -0700 Subject: [PATCH 0841/1470] =?UTF-8?q?[TRIVIAL][DOCS][STREAMING][SQL]=20The?= =?UTF-8?q?=20return=20type=20mentioned=20in=20the=20Javadoc=20is=20incorr?= =?UTF-8?q?ect=20for=20toJavaRDD,=20=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Change the return type mentioned in the JavaDoc for `toJavaRDD` / `javaRDD` to match the actual return type & be consistent with the scala rdd return type. ## How was this patch tested? Docs only change. Author: Holden Karau Closes #13954 from holdenk/trivial-streaming-tojavardd-doc-fix. (cherry picked from commit 757dc2c09d23400dacac22e51f52062bbe471136) Signed-off-by: Tathagata Das --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 153af742892a7..067cbec4bf618 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2350,14 +2350,14 @@ class Dataset[T] private[sql]( } /** - * Returns the content of the Dataset as a [[JavaRDD]] of [[Row]]s. + * Returns the content of the Dataset as a [[JavaRDD]] of [[T]]s. * @group basic * @since 1.6.0 */ def toJavaRDD: JavaRDD[T] = rdd.toJavaRDD() /** - * Returns the content of the Dataset as a [[JavaRDD]] of [[Row]]s. + * Returns the content of the Dataset as a [[JavaRDD]] of [[T]]s. * @group basic * @since 1.6.0 */ From 1b4d63f6f1e9f5aa819a149e1f5e45bba7d865bb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 29 Jun 2016 19:08:36 +0800 Subject: [PATCH 0842/1470] [SPARK-16291][SQL] CheckAnalysis should capture nested aggregate functions that reference no input attributes ## What changes were proposed in this pull request? `MAX(COUNT(*))` is invalid since aggregate expression can't be nested within another aggregate expression. This case should be captured at analysis phase, but somehow sneaks off to runtime. The reason is that when checking aggregate expressions in `CheckAnalysis`, a checking branch treats all expressions that reference no input attributes as valid ones. However, `MAX(COUNT(*))` is translated into `MAX(COUNT(1))` at analysis phase and also references no input attribute. This PR fixes this issue by removing the aforementioned branch. ## How was this patch tested? New test case added in `AnalysisErrorSuite`. Author: Cheng Lian Closes #13968 from liancheng/spark-16291-nested-agg-functions. (cherry picked from commit d1e8108854deba3de8e2d87eb4389d11fb17ee57) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 1 - .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 12 +++++++++++- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 4 +--- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index ac9693e079f51..7b30fcc6c5314 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 @@ -206,7 +206,6 @@ trait CheckAnalysis extends PredicateHelper { "Add to group by or wrap in first() (or first_value) if you don't care " + "which value you get.") case e if groupingExprs.exists(_.semanticEquals(e)) => // OK - case e if e.references.isEmpty => // OK case e => e.children.foreach(checkValidAggregateExpression) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index a41383fbf6562..a9cde1e19efc8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count, Max} import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} @@ -162,6 +162,16 @@ class AnalysisErrorSuite extends AnalysisTest { UnspecifiedFrame)).as('window)), "Distinct window functions are not supported" :: Nil) + errorTest( + "nested aggregate functions", + testRelation.groupBy('a)( + AggregateExpression( + Max(AggregateExpression(Count(Literal(1)), Complete, isDistinct = false)), + Complete, + isDistinct = false)), + "not allowed to use an aggregate function in the argument of another aggregate function." :: Nil + ) + errorTest( "offset window function", testRelation2.select( 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 b1dbf21d4b80f..084ba9b78ec50 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 @@ -21,10 +21,8 @@ import java.math.MathContext import java.sql.Timestamp import org.apache.spark.AccumulatorSuite -import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.catalog.{CatalogTestUtils, ExternalCatalog, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{ExpressionDescription, SortOrder} +import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.aggregate From ba71cf451efceaa6b454baa51c7a6b7e184d3cb7 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Wed, 29 Jun 2016 14:06:38 +0200 Subject: [PATCH 0843/1470] [SPARK-16261][EXAMPLES][ML] Fixed incorrect appNames in ML Examples ## What changes were proposed in this pull request? Some appNames in ML examples are incorrect, mostly in PySpark but one in Scala. This corrects the names. ## How was this patch tested? Style, local tests Author: Bryan Cutler Closes #13949 from BryanCutler/pyspark-example-appNames-fix-SPARK-16261. (cherry picked from commit 21385d02a987bcee1198103e447c019f7a769d68) Signed-off-by: Nick Pentreath --- examples/src/main/python/ml/decision_tree_regression_example.py | 2 +- examples/src/main/python/ml/lda_example.py | 2 +- examples/src/main/python/ml/simple_params_example.py | 2 +- .../org/apache/spark/examples/ml/CountVectorizerExample.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index b734d4974a4f6..58d7ad921d8e0 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -31,7 +31,7 @@ if __name__ == "__main__": spark = SparkSession\ .builder\ - .appName("decision_tree_classification_example")\ + .appName("DecisionTreeRegressionExample")\ .getOrCreate() # $example on$ diff --git a/examples/src/main/python/ml/lda_example.py b/examples/src/main/python/ml/lda_example.py index 6ca56adf3cb12..5ce810fccc6fb 100644 --- a/examples/src/main/python/ml/lda_example.py +++ b/examples/src/main/python/ml/lda_example.py @@ -35,7 +35,7 @@ # Creates a SparkSession spark = SparkSession \ .builder \ - .appName("PythonKMeansExample") \ + .appName("LDAExample") \ .getOrCreate() # $example on$ diff --git a/examples/src/main/python/ml/simple_params_example.py b/examples/src/main/python/ml/simple_params_example.py index 54fbc2c9d05df..2f1eaa6f947f0 100644 --- a/examples/src/main/python/ml/simple_params_example.py +++ b/examples/src/main/python/ml/simple_params_example.py @@ -33,7 +33,7 @@ if __name__ == "__main__": spark = SparkSession \ .builder \ - .appName("SimpleTextClassificationPipeline") \ + .appName("SimpleParamsExample") \ .getOrCreate() # prepare training data. diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index 51aa5179fa4a8..988d8941a4ce7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -27,7 +27,7 @@ object CountVectorizerExample { def main(args: Array[String]) { val spark = SparkSession .builder - .appName("CounterVectorizerExample") + .appName("CountVectorizerExample") .getOrCreate() // $example on$ From d96e8c2dd0a9949751d3074b6ab61eee12f5d622 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 29 Jun 2016 11:20:35 -0700 Subject: [PATCH 0844/1470] [MINOR][SPARKR] Fix arguments of survreg in SparkR ## What changes were proposed in this pull request? Fix wrong arguments description of ```survreg``` in SparkR. ## How was this patch tested? ```Arguments``` section of ```survreg``` doc before this PR (with wrong description for ```path``` and missing ```overwrite```): ![image](https://cloud.githubusercontent.com/assets/1962026/16447548/fe7a5ed4-3da1-11e6-8b96-b5bf2083b07e.png) After this PR: ![image](https://cloud.githubusercontent.com/assets/1962026/16447617/368e0b18-3da2-11e6-8277-45640fb11859.png) Author: Yanbo Liang Closes #13970 from yanboliang/spark-16143-followup. (cherry picked from commit c6a220d756f23ee89a0d1366b20259890c9d67c9) Signed-off-by: Xiangrui Meng --- R/pkg/R/mllib.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 8e6c2ddf93cfc..897a376a94abb 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -442,11 +442,11 @@ setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), # Saves the AFT survival regression model to the input path. -#' @param path The directory where the model is savedist containing the model's coefficien +#' @param path The directory where the model is saved +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. #' #' @rdname spark.survreg -#' @name write.ml #' @export #' @note write.ml(AFTSurvivalRegressionModel, character) since 2.0.0 #' @seealso \link{read.ml} From 1cde325e29286a8c6631b0b32351994aad7db567 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Wed, 29 Jun 2016 11:25:00 -0700 Subject: [PATCH 0845/1470] [SPARK-16140][MLLIB][SPARKR][DOCS] Group k-means method in generated R doc https://issues.apache.org/jira/browse/SPARK-16140 ## What changes were proposed in this pull request? Group the R doc of spark.kmeans, predict(KM), summary(KM), read/write.ml(KM) under Rd spark.kmeans. The example code was updated. ## How was this patch tested? Tested on my local machine And on my laptop `jekyll build` is failing to build API docs, so here I can only show you the html I manually generated from Rd files, with no CSS applied, but the doc content should be there. ![screenshotkmeans](https://cloud.githubusercontent.com/assets/3925641/16403203/c2c9ca1e-3ca7-11e6-9e29-f2164aee75fc.png) Author: Xin Ren Closes #13921 from keypointt/SPARK-16140. (cherry picked from commit 8c9cd0a7a719ce4286f77f35bb787e2b626a472e) Signed-off-by: Xiangrui Meng --- R/pkg/R/generics.R | 2 ++ R/pkg/R/mllib.R | 72 +++++++++++++++++++++------------------------- 2 files changed, 35 insertions(+), 39 deletions(-) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 27dfd67ffc932..0e4350f861e4a 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1247,6 +1247,7 @@ setGeneric("spark.glm", function(data, formula, ...) { standardGeneric("spark.gl #' @export setGeneric("glm") +#' predict #' @rdname predict #' @export setGeneric("predict", function(object, ...) { standardGeneric("predict") }) @@ -1271,6 +1272,7 @@ setGeneric("spark.naiveBayes", function(data, formula, ...) { standardGeneric("s #' @export setGeneric("spark.survreg", function(data, formula, ...) { standardGeneric("spark.survreg") }) +#' write.ml #' @rdname write.ml #' @export setGeneric("write.ml", function(object, path, ...) { standardGeneric("write.ml") }) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 897a376a94abb..4fe73671f80df 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -267,9 +267,10 @@ setMethod("summary", signature(object = "NaiveBayesModel"), return(list(apriori = apriori, tables = tables)) }) -#' Fit a k-means model +#' K-Means Clustering Model #' -#' Fit a k-means model, similarly to R's kmeans(). +#' Fits a k-means clustering model against a Spark DataFrame, similarly to R's kmeans(). +#' Users can print, make predictions on the produced model and save the model to the input path. #' #' @param data SparkDataFrame for training #' @param formula A symbolic description of the model to be fitted. Currently only a few formula @@ -278,14 +279,32 @@ setMethod("summary", signature(object = "NaiveBayesModel"), #' @param k Number of centers #' @param maxIter Maximum iteration number #' @param initMode The initialization algorithm choosen to fit the model -#' @return A fitted k-means model +#' @return \code{spark.kmeans} returns a fitted k-means model #' @rdname spark.kmeans +#' @name spark.kmeans #' @export #' @examples #' \dontrun{ -#' model <- spark.kmeans(data, ~ ., k = 4, initMode = "random") +#' sparkR.session() +#' data(iris) +#' df <- createDataFrame(iris) +#' model <- spark.kmeans(df, Sepal_Length ~ Sepal_Width, k = 4, initMode = "random") +#' summary(model) +#' +#' # fitted values on training data +#' fitted <- predict(model, df) +#' head(select(fitted, "Sepal_Length", "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) #' } #' @note spark.kmeans since 2.0.0 +#' @seealso \link{predict}, \link{read.ml}, \link{write.ml} setMethod("spark.kmeans", signature(data = "SparkDataFrame", formula = "formula"), function(data, formula, k = 2, maxIter = 20, initMode = c("k-means||", "random")) { formula <- paste(deparse(formula), collapse = "") @@ -301,7 +320,7 @@ setMethod("spark.kmeans", signature(data = "SparkDataFrame", formula = "formula" #' Note: A saved-loaded model does not support this method. #' #' @param object A fitted k-means model -#' @return SparkDataFrame containing fitted values +#' @return \code{fitted} returns a SparkDataFrame containing fitted values #' @rdname fitted #' @export #' @examples @@ -323,20 +342,12 @@ setMethod("fitted", signature(object = "KMeansModel"), } }) -#' Get the summary of a k-means model -#' -#' Returns the summary of a k-means model produced by spark.kmeans(), -#' similarly to R's summary(). +# Get the summary of a k-means model #' -#' @param object a fitted k-means model -#' @return the model's coefficients, size and cluster -#' @rdname summary +#' @param object A fitted k-means model +#' @return \code{summary} returns the model's coefficients, size and cluster +#' @rdname spark.kmeans #' @export -#' @examples -#' \dontrun{ -#' model <- spark.kmeans(trainingData, ~ ., 2) -#' summary(model) -#' } #' @note summary(KMeansModel) since 2.0.0 setMethod("summary", signature(object = "KMeansModel"), function(object, ...) { @@ -358,19 +369,11 @@ setMethod("summary", signature(object = "KMeansModel"), cluster = cluster, is.loaded = is.loaded)) }) -#' Predicted values based on model -#' -#' Makes predictions from a k-means model or a model produced by spark.kmeans(). +# Predicted values based on a k-means model #' -#' @param object A fitted k-means model -#' @rdname predict +#' @return \code{predict} returns the predicted values based on a k-means model +#' @rdname spark.kmeans #' @export -#' @examples -#' \dontrun{ -#' model <- spark.kmeans(trainingData, ~ ., 2) -#' predicted <- predict(model, testData) -#' showDF(predicted) -#' } #' @note predict(KMeansModel) since 2.0.0 setMethod("predict", signature(object = "KMeansModel"), function(object, newData) { @@ -477,24 +480,15 @@ setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", pat invisible(callJMethod(writer, "save", path)) }) -#' Save fitted MLlib model to the input path -#' -#' Save the k-means model to the input path. +# Save fitted MLlib model to the input path #' -#' @param object A fitted k-means model #' @param path The directory where the model is saved #' @param overwrite Overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. #' -#' @rdname write.ml +#' @rdname spark.kmeans #' @name write.ml #' @export -#' @examples -#' \dontrun{ -#' model <- spark.kmeans(trainingData, ~ ., k = 2) -#' path <- "path/to/model" -#' write.ml(model, path) -#' } #' @note write.ml(KMeansModel, character) since 2.0.0 setMethod("write.ml", signature(object = "KMeansModel", path = "character"), function(object, path, overwrite = FALSE) { From edd1905c0fde69025cb6d8d8f15d13d6a6da0e3b Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 29 Jun 2016 11:30:49 -0700 Subject: [PATCH 0846/1470] [SPARK-16236][SQL][FOLLOWUP] Add Path Option back to Load API in DataFrameReader #### What changes were proposed in this pull request? In Python API, we have the same issue. Thanks for identifying this issue, zsxwing ! Below is an example: ```Python spark.read.format('json').load('python/test_support/sql/people.json') ``` #### How was this patch tested? Existing test cases cover the changes by this PR Author: gatorsmile Closes #13965 from gatorsmile/optionPaths. (cherry picked from commit 39f2eb1da34f26bf68c535c8e6b796d71a37a651) Signed-off-by: Shixiong Zhu --- python/pyspark/sql/readwriter.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 10f307b987dad..44bf74476b099 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -143,7 +143,9 @@ def load(self, path=None, format=None, schema=None, **options): if schema is not None: self.schema(schema) self.options(**options) - if path is not None: + if isinstance(path, basestring): + return self._df(self._jreader.load(path)) + elif path is not None: if type(path) != list: path = [path] return self._df(self._jreader.load(self._spark._sc._jvm.PythonUtils.toSeq(path))) From 3cc258efb14ee9a35163daa3fa8f4724507ac4af Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 29 Jun 2016 11:45:57 -0700 Subject: [PATCH 0847/1470] [SPARK-16256][SQL][STREAMING] Added Structured Streaming Programming Guide Title defines all. Author: Tathagata Das Closes #13945 from tdas/SPARK-16256. (cherry picked from commit 64132a14fb7a7255feeb5847a54f541fe551bf23) Signed-off-by: Tathagata Das --- docs/_layouts/global.html | 1 + .../structured-streaming-example-model.png | Bin 0 -> 125504 bytes docs/img/structured-streaming-late-data.png | Bin 0 -> 138931 bytes docs/img/structured-streaming-model.png | Bin 0 -> 66098 bytes ...structured-streaming-stream-as-a-table.png | Bin 0 -> 82251 bytes docs/img/structured-streaming-window.png | Bin 0 -> 128930 bytes docs/img/structured-streaming.pptx | Bin 0 -> 1105315 bytes .../structured-streaming-programming-guide.md | 1156 +++++++++++++++++ 8 files changed, 1157 insertions(+) create mode 100644 docs/img/structured-streaming-example-model.png create mode 100644 docs/img/structured-streaming-late-data.png create mode 100644 docs/img/structured-streaming-model.png create mode 100644 docs/img/structured-streaming-stream-as-a-table.png create mode 100644 docs/img/structured-streaming-window.png create mode 100644 docs/img/structured-streaming.pptx create mode 100644 docs/structured-streaming-programming-guide.md diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index d493f62f0e578..2d0c3fd71293d 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -73,6 +73,7 @@
  • Spark Streaming
  • DataFrames, Datasets and SQL
  • +
  • Structured Streaming
  • MLlib (Machine Learning)
  • GraphX (Graph Processing)
  • SparkR (R on Spark)
  • diff --git a/docs/img/structured-streaming-example-model.png b/docs/img/structured-streaming-example-model.png new file mode 100644 index 0000000000000000000000000000000000000000..af98765fd83391b444e4f00e9c8c76ca743206f3 GIT binary patch literal 125504 zcmeFZXE>a1*EOC@q!5V~EuushHKGI&Le$Yhq6DKyCsBhS6VZDQf`}fyjXFqlqIbiH z7G;dl#$XulCBOT5kN0?v-1o=-hyQVZ$OL1q>zva*_g;IgwI@_dL+J*|ZITNYF5FO2 zeyV-p!j=0ME?gSFb_Mte*R5_{;J=Hm+DdX4N(b(&1K$w8RyJ_GaDkTc{NF|E%n{%v zvKLgIK6(E3;^y4d&(FuPtve48v^lCT8bI2&E^*sEe_BCB9bsec{N>p*9*>JYPoCYs z9m-L$?5r6v{4}J3mz(@jFRMbs`x`lJm5qBct7zm78j&)cV&yqKeF`63z3*yLk88qg zB>DVC^8N_A0J_HV{=X0Z-Gl$e!GDjze^bGKGvR+T;eRvX|EHO7%nPcOP7B~7qKyx- zKEB=3?q}_|a)V>S^Et9Hc;c1`>ah~81=VbfvxMFB-nrO-8=ga!4bROS2X7g=TgwA4 zB$xd+4_`@UzbftdT1QRdirmRigpw^@Fw)Q~X)E1L(-NUB=&kM9IC#+CFgOV>$Z1kF zi5^bB@gFWn9&*oAg%m8jyy2I=B__EDnM15?!Saq@ec19y=U^Rii>}3v$A`PsqL_#0 zdg&Vw6Z*J#H&5cos!bM#{}fi_w_dA6@OYaSZi!)aiqUta%|b)(RtO9&iGkV_ogIcx z)#fZ5vDo&9bLhnXxIk{UUvYGM`w=CP6wU~+=8T(GS!aEBTd7#bLdd>B#P!DYH-;ul*XR1uT<$4S-%d4jL79}iK!<0ZEDch`7znagp_J$3g2lk+S*N2SL_o;Mwte+ljD= z`j>_|_y2m~jRN4plE;Co6%?{xet?qNr_^z})TK}cioXUf%Uxj2#P{D&03kz8QO+LZ zcq!cEkoNC^`AzjbyW)!;Z9&krF;CDi&^ znjXg=P!7=f4A&^ zq7NzzO0v0zLT3L=S-Z-2_Zj)=^TKSgRP>d(meIKl944*iBy|h@?{Wf^QkN z&2?ZgTKSh})<$s?@cOMPw=+4M+<(6H-#7p7d;Hg<^8ddJKxeqfZ_D_-OT|>ECZa!| zz3aipG&X;*@@zarO$6l52$VVCjxmB>nXerL;rrSRH47g?-?T{a6JWjJ{BMz4*m;xP z$0Hs&6v)3~ju;S#o`;<6E)b|;*Av`wtk+QJN3{u;&in zD?9SsyCOvC(!`weAa`i4|A(yZZrmD2A`$j0^2!j7o{Ceg0Mk1$TZRsqy&} zsyC^eyMA8E5`+FERmON!uN%y3Xx!}5JySVAh}(lqQ&Q_rG>mLE!Cy|7?sesL^|fzA ztCtB<7eP$!i8W4?Rx~(X?|fxtpM6-I5H@bU+x5HGyJVG)U$>|%N5Gy}r!4g0O$M=_ zy#wEkJ9+Np0m7K?}7qLU1(W^k?4!?2=sm6gIn@P90d3;RNz#41tq|ZAr)5=_7 zDmnqPSes&Rqi zDc7jR=pO7#c6g}XU-Rrg$5HDdnDQ_l^gRk^s?9RBQ#zTfO1Xx$%8RL#1eb%=`*OOP zDl^JngW6_XO}b5szfXN&6Dlqq+{$YVOLM58JMBp{H9!bFm`j`=pV{fU-7);&_27^H zw1LJfmg(zZ@2bkX zYb-Ghpd{rXcz4E^9C;bn!|esbVh2VvuU@GdvxLjCF0Y}sDG>iK@Ox`ju zFsZC=l8+9~PQ<>3Z6UxLB#B0t#L>}F`M|(H-4@z#WM;E zgP5xFvODb@r)wI}LzCF)>FIgG;ZpPi!eN(8DU7JI(yGs;GlJg2$;`dSX0E|KNix;& z;o9x~XJx!2_@*hW^xw}G&s|wcHjZXBqEOk5>qJsBHS*F$W^F-lsOZ$w#BM~>-8aXt z4QESzYYl858y+B(9;fCJUnEnA2O*0*P7FtWAkgny942QilsN|8hra4LQPL&>cwnF*yX9DW1q%w%8eMAE0+zviJA}t z7Wh0`_Rvv6Vp>c|+9z7#15n$8Rz8jnKe$Kv8M#c25A}?Y`-*Fl%<@z{QKbyl)Bty;- zpZNLnX9T8cZ=jyWTsV{^+@LHvBCJCAeCg@3htdXxH5dbTeyu$LnukhKDj9Pf_lF++ zDb;bH7btoDJo^>o#B}tuY*i9Chb#ur7*6()xm(&3 zI}qe8ZAm${9O8)UHEm}C=sIW98=b26*yC=M`#44?m6ux(a-KB7HfP?h22>E0;W9ld z*l(vH?RE6hn;j9bCweBn*#5}dqQ?OmHPpM&t=BX~l(<*wub>44)~|H0`?Q2)4dVN~_7Q?|WmE^3NhUh<}cF3&hMKSxZXGa_$wr zO1^%wUDw@Nc(8X=Wz2N_;-9+nJQM(D_snv&iO64rQmd~`Cur$vSP}(6lB&!*!h&l? z34H$fEpwS_(;UPn#3WKQWe$$*s!3DuodGVYv~e8>-C}h$hE7GOAAi{DXl2?IfRpVm%LyAYOo3Bx(Av7#jl&es<6Px7G0NX- zCyyK1dq~tAHBCH7LJ~C|R;orJWgU`YM>$ z;KyDmH+rCNf5WV&woZL5i|rUXB_?$r9PhjC-hL%ph&1YNjRs3D(TMiFmmd@^Kof42 zcjP;g6u#AAp1!4xnk=iOpo4(gn8qT*l*LrE(uqm!iyGqj2NXiDY=lH~tq!E4IJiY( zpV?73GeEN@^+9a&kFbyC_ilXn>y504)`nD51gW8{F{z-5H_hJIjiK&XHh31C!~Q`+ z%CXIKr4_B6+%G;b$<%Mjjd&H)DfpImQClwzpC41XL~yCzD|M8zHvDQe32Q`}z#2ZR zm?FMc)}W(BUm758SFUFGKp6R<6=rQ${6TD8TYqmPSrUmpY1~U}e%T%-OG09^>aE{) zpa^PLHCk1fWyxlKubB{*uob}5t{2+vScP+KpdA;1x>CINBKEwe|= zlx-3zs`%C$);cjdJEj!*69sVy?Qh>yH@!9G_4lSpTLlQMd%iUx)X-bhLpS$}YwRbJ zQswTkv1KbO-E?4<_I6WJ2Yq&mrs3@&7gUK+)^07`;d;1OV>QX?(<~g%-vdLJPS#zG zrho9tANX4%k=?Y(t#psy)s>`Bj3_jZqP_pkjTPRn=V-6+Dtx|{Q}EuSAiu_GPO_MB z9kEGK{dpAbgJZ3os`h9$$srDkV5g!x@*ebt*T!xc5PSe!U`D}bNJ>EQR&8p+^c!J~ zM8UlHcpjj-udrKyM;ivUCx<;ugrHel(_UU)5SO-rsy_$qe`C)W5>Q0Da$p(deb$&C zpcyYGVp0a;UmQXXBs|HjL$Md@ftn|8r2Rea0_>M&KFmf`|*@(=|>#q%hI_xrcVK+U?0<$f$`vWt1b8z)UXDJwF})f+{HX0MU* zui7nk=Ovh{lLlD*KQX`(c$si?(B-mCyqdpFMK1Xdi0dJaX;&TyFQSlT@w(`&PKuTt zd}D8_PYv^E+g~3|v*q&FwyU%%~=N-cMh+%RZ|)5Qu_1M{UOnp8Z~*Xf6M!SjxXq z5}GYnlze_#r>H>_!{&?v4*x_HF3#C6-Ju#7wxwiQdYQ?MD*H_j8w-5N@{=f06Dv|zF+Sx6j9;jZf zrkeZxdbd*&ahtB8&Mk|nN>tc^C`g@)dc7_>nSJOcp_87T-m%8QFA(X>Ce%dK`RTp| z%*NPEIw^}ds?lxvN$iAI{KU!)i`SVa>K@PKw~t-S_u^=_3^pN%Q^%`)L0dA+C7;#g z+G_X3eCg%x)M(n5LLbRPrTyp5$)c}8sXG(_aj5j z^j2iwti4KjxFV9Xs0IJ*V9r|5p!{w9&(yudvA+JK+0ToaWoKS1&(_DY!6`lxT6m%# zMX$u-=PVP{-Ys>;41CbSXT)&HU4TBn#S$7DW+G~?$P!58w)}J^nAuNXOPG>nnUc=C z$rMN}#2i9K`7ItJ!TYCEf!Aqg$ioc*KoLh2RzD6X}ePsJu;^r=!q)Il9&&};gG zX}*8uO1iqKz7K)o_JNh#V3Gshchsc{FCFE2p4?6Qp>ne}2X#~S0}8(D5kQ`@YijL% zF>e>#hCU|llNj-7@m+3Yp6H!%XIPFx`E$``c)76qp*H;wduUfeTTYC{9FeSZlAZ}o zm&0?-0*-f&A5i7Gav7>BEzd>2>YYxGPOOn}j5#@B9QiW35G(okbTTv7XHp~9rCOY~ z*??^|icavsnYiZh*1Wo|KR#e{>KO|eQcP9&f%#c)z2&sH3*kkrcPN=pRBoSMIEOY< zv#Z)qo)S)(J-vFzMXE~wL4ml~3;|}fGb29Yk>^r&4pjr`Ehh^HHfAoQe_|h+KwfA0 zGhG;as<{%-B2(IP42OS0J6{Dm7`gNtADew#BR<>Px31t|d8Og2d$6H(0=q#Dg=6(@%l%KUhm}=2^B?%%d6$XQNiYgQ^Ix+gQkjZlCMTY$-uRb zu(kO?Y-xPaRqRa@-=EHMkv9zttuxZE0@&|1FF9VY_cntt^zM7vd&D+Zy_e$|fO0eC zxpv=>t&Nfr>D*SzHa}0{oWB2E<1DVS`D3Roh`q#X?>Ci**+OJ@IZ;q)a_oJVWo3rg zyOJ9$t}DHXwJ1mdR%xR9wWEPCtiH0vQ7S`uIx~hlwBIRG62$fvS?cO|)qxk2AKGqAK~7wr9MH*_m6AZMBTx?_fy?_ zZHp`p_1ygJ!?Z_v=j^2ADmb#aju189*RvWG*FC*L#lIJm`A#LTNnD+< zvw4YF-k|aHDp_*)prk@x^C4dOQFwAm!u8Sy^;ai?f_BDY<`%6n;N_XV!9-8UX#fsB zDRAO8k?gV&IQh0G=}mTUVgs&TkH8b3@%Bj`{3xM%t}4-Us}VEVtFjuEaKA;$P=nF! zc_E7DA0!fE2Vz#kmz}JP*IE*|%6gefQk8zxwonN)nih!6!$W8B-49}yw)$H{*LQv+ zn0vmim#Jh2HP*L5C>LWJ%Mwcvu)a@*OmVABaDJC9;KsRFQGvwP^7z$^$I68JSox#T z>t`_wtE8bugK3iNyn29VTOfJ#{aHMu1x_eG4vCN3*O zpuyPgTefraQC_#?vg4lTPlVtqtP3DkYLcLJ21=ItAC`5+L2b#c-6NOVM@0;Fz;2l^$NNS4Ht{cuXxQhgKL~RLr(-7b^P6zBJtL^@`R<|ks3a$2NyPyZ z|1P%OS+-h5N;;3Pff~8k_EYhm`}rmslx(aWX}#n_S($fJSdHNgBr9FL1@V%0?`2ah z*YseqmOZec3kEMQv9b~osS-!PJu8j8cIa~{=?K?1IzGwmB;g5bzCx{;t=sdSeV@!m zj(*)0hM3IVN_b$ss&W>9KNJja!uY>-P7!mxsS**Q%=nIq#6505!ge9^aU;vvVJj`_ z%Z2EO3n0;_01UrzBu9P)RAV43HRSq%m=t9L(4Uc!aomX_ArtG&Y*|^s;YyYTbJ83- z%G1{(_!xa-A#ttbzWoz>wT`oO6|(WH;>ql?tSTSFIEDDO*v+xcO&)N^7eGK*5A)AY zz=u{f_PB(KX8~|cZf{OU!2l}c-yR5lupI7X(l?*kER%Kh&AiVC+I*CV+Ev`N-fzK| zFdaZBx-ga(<9)Q#7@+>v%3g*`s_18Zt<2$Ank&9!m07@b;HLg+Z@v809!9dq)JN@W zwex4(FlHCoGGW~QSfLmUR^VJKDkIi0;nYd7uE0Opr{gW zLl+cDrsA)}Uz?bwwkHZo-^-lz0V@68?|udg_j;nBsP1wKx}?@_+nMS@egYg#z+;%p zMoVUMiHQ)K)q2Tr&lOX?o!;;Il>CjbO>YDxe^+?N#+%iCjZ7d*jaLqmtT=PvzWN^&@znM#=1wr<7ceZR~&SWj8P5SQ7c`)x0LU&R53rpWcDx#bt-c z!pMf87n5)V7xtP*W(m(q994zWR;8X#dg&XL2aggDhx6W|``m-P3|+dUec7u!Q6Qrqj?fyP}W zx_i6pcl0dS8rgY|R+~zBs`*e}< zEMoKILs_zlnOW9UckGqM@nnc}L$@Kuk5yjdQRNt1%>U3SmE3kR?`?X7HWv?z)-$Nv zG+lj7U)OOVv@$!4b=50g5|m~gR$TYopFUopkV+(Wf*N`f_taPSl}orSE%oV%_=`tH zBJ=b59&>M$m^Mp9-E%c=c%2;oJpO{Hc^uT-0>UR>B>}^MlT0( zas0gG=!unlWu;?jHoyLaA3g@@Ga5$Cd$S5v9r-yldm1#f81ex{dA>GPgq?ozkZWZgXj9K`cA)zo07*uUV8b#o4)%7F73Y-DsfK& zR#3UTE9hkA#Gh)9@wdM(s#nM=ODcP>#cv;{v;-0X*Dnyb=y&@?vIv?*EBLZVVGKG= zt`}^huhi)GZ6XakDmj0u@_x5hkzNh7CY@UJ=@7$x-d_bp5z0d$@-thaGZ*QO4xP#a z_#o4Z-;^B)*5Nch+Xm)|RH5i!kA27(UF7DG7G;w66I!yl`FX1>7S_xj^z-I04~)o! z$VGIP3xXIxFZj#1;~=9luZZ<+|p3eC=87cE`J8Hq~=Q7d!EmN>6sGwSmqduF?5h zrvh9`=J@s7Qz!%LlbyT+l3T@^$%0Ye;DdUTnS9pK2MgFT%|aBZ`^Hg=&nvQ{SK5(& zmf+iNHi(yx2@bF*eYJNv*{A6vMLx7C$AtV5c_kx7gf7rJHJ@zQGmcM5xOneWChp?% zA#u}~lT1t0ZQVY-(%1?9wA@#_Upt1XexS-E*mqwy6@HTC{@hyLrd8@kvm&^FeQ?~H zz+s`!m2EyY8+ZW}di~^t)@%{?ngeK4V#?n~Ug_@ZtLn%L7fE>rK#OPDiVBJn`hnEy zlxrmo%N?f{BG84?%3O2;ke?~(Bv}_;s+-Q*s@EUznf;2p@(I02c%YdSnI?uhMdT{%-kolo~qRr`G6@3OTX&|8ePl*yha z{HiK=7R~4$``PzMzka)xOTk#meij~YwY{U1pcw3J`Y zl@|+4i0mF>4@e#wvH<$-kYsit^~s9gaTmP1!KOCp5X}td8xh)ULSH$APOwMuM7o~k zMvxz@Z;>Xo;i7at!d8tF#>^4@CNPidZZ{DT`-?BPhCu)AD$cz_l@UT=Q;)dnxs z=!pV2TO;)T_Vy+CdNGstNuV%p>LT89O=qyBbQF?qvT-to%LBI{rJF2Zii$N{LaGY# z7yn5vy|i`vvpR70vqD-UHp%zWBdy{^E%EUK^gLC2G*6?B=I==TGL>*@z3VciLhB== zG$)aE{H8sUYPr8wCJx_evU1a>%=ZArdFH)pVLXecGQf>qsjzpfUFV%9SJpCpKbCB; zkooFBW7}Kcy!6J=03ELgy*N-dFB7rqC?~RuHUW%*Ce;0>A(y`pF*%=MS}D~$$L0N@ z(}vCtQ70nBuK=do$4%0zf$CsXk^RQb?hR8PDmY(8(HIu%m+()*!pGxi2tMy`3{D zVUhjJ$Qqnu{x3a0W)UPNw^LP{F)=)*Fw)dDyxaGLR&|h&@1yfH!*wQfcL<$r@pxgB zEP67QjtiWbJ$SIS?%yk9JK^|+5Vf8D(+hVQ6F^Q#!|!i!%4klNVBgv= zJ0yHGba*+hl$S7aJM4sUN7sUNB`yh0q&S+V4j;FyfjE=!`1TLikA|D8-=$X5Z@efheD6Lw@=qw_Ewm`!CoG`ihxy?zS*L;+xmge?&1*Qt6x~$`LM`LM6PTL4LIK9 zeUSHvf7QgRX!d9*GY98qyaKQeJ<;});p!Z3!ZfJO40RY^>?A8WCtojXRxhmw(Er+u z%5Q$7drEgkl#ssI`za@zCUDfr+2(jwqh*J2|fk)W4XI8mL zGAW-B5!*4Xfs?s@?G{LBsl~AF?r*_ZLhybP83do5{EORNUgdjULaHd%KmyZixLm5@ zpJcL$c#$Mv!k<>_kf`Dr5hZzJqTR48>GaCyyJttQgHGz4d~;?TjE!a$3qBtdWm}%C zncdGX2%BwX_=m)_D}R4V>VKLNiX01d-K9}!lr~f~G~P!zKESS8SNHp^GQZiOcR31_ zw<3JCs&qQNlAc;oG!=aFtK$JQV0@my()WtcmZsf1F}(WdK!NmXnZ7pM22p%ai#C(>a`V@lx2CMVRj;Ye3?Me!Tq2gx&0(z|#xRuxt7w za-L}I9aYo|oS}(k}NkEt0Nc)uvGiyj~C-`^x&c|1^{AU7E3Zu{m3z5a33-P|IqZNl`fOySlNsHAc=o z%Px!$S=4WQS1c0~z8aw!`72a7ul08dE3W``&ons4=SXL-qd1tVkZZdQ#oFgQj#&r} ztM+BVE|x8apjSQ8PBqEFAyekaVKIiCI=vR1Wxw>Gdnux@uj4PQx+Z3`TRh>cBi{Nu z5UU%{-YkL^a8`R0qjDu=XODM$>L`JH8^YhJ95E&K`N;*SD)~@*U`g=Z??gfThtj^s z`Jy?=sDr>;c+nv$v&yO0rOt9b=Hr#S0vR-;RyWmt##D)u;rldg$0 z@XT9^`5o-7mDK70S6un)$;K&T#6*g9+rTLR(s*)}O{bDS-hjU387f$Ca+!M45`uRS z;}2A)O8ogM`?iDTt1662x>U{Hf#p_}J%hL%OEB{t|9dN|kfUTb%*ZgX2x-aP0Z3Dc ze4})`2Se=ccD$RaBrW70PhJQ|x!cjb=~9(2ePYr^!3PiIWkj^rA}b?akabaT^OA1} zVqM(RBQ%3Q-o}moybtttEak4C4yrbc5#wwmz%;j0ha|P?!yHl~tp{ouN=Ey{CPOj3 zr4~p%zY#^#r)hHE0KY;YfhT7l`BBkSe-37d@E&wIb=w{K{P_-^+qDgNT6E}UW#`PR zz8ef;BIPb^K{wJYt*mOM>fU7QP2kk`Q!~6Vn3|gm9F$mt4}&yCp8-|EXw#3jG11Z0 zFZJp&#~ZXw6tch>6%k#E>)F@fn)uyCInTmaRh|7%aMIqs&f|K+Pd zQ~c~=8dERf-O;4KTaG~lx>Qv}G(6rhc&}NLCw^rDd3|{i|&|Xm9|$@mS#86#Dhr z$;nO)H+8BMJ_W#z%3MA?}YM#lKA(&yZiz66aFs#8}(-Hyj-e zr~lFU3i*a=^*p_HFgjLK(*fBUFKYBBG3BCig#DqZ=YDGDN?&G(HgT8I?gar&&qm(+ z)6<<1W=v96TWS0Q4GG13i$_K?)lONc*AB4&JG@e14N0WwPwh3yOl_T~bl;PSReft` z%q1`9KXSPR-~D5#TWh}gWT=qn#W&e60_S8}xQTlPJ=E^jP}Et>7NVv{o0yc>KEGkI zC_@(1CW)EEZcgi1$Gl_+6Aw%Pq{4B-{1(*Vx%~4M5*^#R9VZCTtJp`CqGlD-hkVfC z9F=%d##6IZh4=>sK=n$?aWr-s%uQpAsy2qyf1U#N(TAp}&!Z)+<&K)^Et;9`8)+Va z>_Iq%h4V6H00j?7zo6YGsNHlW`9kz@vYUh>_q1Zli!zWI|^S2p<&~PE1T9h=+aFz@1Bg`3&P1FgZZeKU)66Z z+llImC_4wsvyCJEdW9j~rNb)Gn29#YtQ9F`4H2L}0Eo~sJ7}N4Y7ve5> z3;SoK|+cn3jyM$;Tw)WKvc*_#m0P zP=`5~`oTn0(rJTdlHk10vyI6?b#yShH_=8xYYrDA!6{uTuDu}J#GlD$l0?LDL>Zvg zf0y3)FoF8eY8IDvAFO0WN-Qe*tEyqU^${CzSTX?PNpDpSCy+YtlJi5&R4%HUw;NNj z#SoRF%7!^hKRfxN~U`>7`>vIJ=sUZ=W3Fy2viyyJwo;lm;u=PuR+j$7sgf9bjfCA9-Z$O}CUanr6XUH{GIw2`qH`#=k?{>7Bu!8 zsuJD8vc##DB<*fCH|Or1JlOI{Pzi_OiszbT>GyEHLyQmk-nNpkgztON=EorM8+qp} zjbe92&!jJGaNaGLwJgc)A*wRXYW=7G(-G#8+escqI3ztVSdNEp#^kZ~GSV zjfziSc4Ssb9n}$bXR~V+N|$bc5?RRw40IkAU;1&~uOQ)GVa}Cnw-VXkh3h|PxbjYv zlI$*fvTE|S-vZ__-o2m$dVu@m{3F)^*l)Gl8bAzMU9(yIVMxXHskJ<(_2g)e3Id#w zTLon+aK7d+c4Bjm%he4#l2+rx*-FuVo`7htaGi#)d(tvA*}k$KliU9w4!0SkHYvh^LcT^wInxlB=-2OI2bd@(n0&6f`B+r2PzPIPuJ#N9( z>vw4&#^B6P<7F)$WZ!#eIJh^#Ge23y+yMcbW4LP;zB@4fR59AO$t0OIL}xxtpo`f{ zIp_0c8n|D6+dU&wxfK{+9V}>r)~)Q?RV^s zEP1MUcTiyK8Y1?29$}&;aP@e4hv}+Fn^vU zOAzl!bpFI@E@g-1o=lQy2NnTqu(wj`4j2 zhP-wB#&@_}HuAL|SAPTJVHl0dD-z^;@nY|lx+PLcF!>1H7WZHh;#c~JFl!&nu2{$b zaKz&vjur&1c-!X8i+*EYtWRdz=x;i2i0|*jnGq(+-nS-`J#})EJ%<*#6KYmK60tuY z7D*N4Ap0O2Nqw!vpo-I?J%r+5XB^&M~XZA|@{K9Uu zK-^qu7(@249hLrBhTqaS5ATTomSrgnW?P_peb!~389w0a4{AGWt&O1POebSvUBbF! zq8zO<2N-kQ!(Z(#u&r{~an%;fgAl4P-H^XA`(~w<> z(c5%cU)Z@5iT`BedL#iTU^`p2^HN_OgBhz^&e?)tjW?vwZ?pyT;N`BCaFN^GBS+KU zoTF*JDVa`VS?A9Ta-Lu|Y@>DmB<^pCRg zz9&3W6o3a45DD<{poxqYqgkj4q2P*MGQLeJrBWv8*C|2&LxsU~DX!Ma>g-xL;3^4q z_Z5OZ)6&YprYY?Y5EX?3(C(fAixL$rqM63lFG@MwmABsI8^Q|Ydc5)UgJdsfRfoWJ zXpnvW`x4%rDfkRAt%ZG~HShXxO?)X&MJw2R$|@j=Cquh8v;EUIHp$Y)#YeVNUz){5 z6ZuIYWsoR6l>g(;cZ2uQ99Yz9rws?eEPQ7{s;j+)yVC%lZgtcvD(p!xKU%9g>%qyC zhQHwm{?lx1d}+ORN{1|ykHo7N1q=?bvAvi5pdcv%d|JRip9KJl$W$W_>jUTvbC8pI zgegcq|5NXXjLm~n#+?Z{g}$9QqNEEzL;E$6cLp|2EqL&@V%nFm7utTigg<#Z1UOx` zc>oCqpmpF9zln}*V`m+=i{_T256Gd|`c9^Nrdiy8M@rF*kPL@#!HQ^IKJC;k`5 z(IBsX%vEP*juysuMooqi!GG>!P4pw)%4{t}Yp=No^^n4;BQY}3P7XF*URhD-9hO`~OHM9qG3T28srS0IXabRfmsR6njWNXG`(9}0D=Y%O z(xHt!vZixeIf}9B()DJ@@6(%B_O{P0?Ey!Tc!R}`aJmLR)N#?mKUVV$pZjh< znrm4+AR2IsZxU@SQl4Wes3dE-ao7*qz}rl(RSKNv8Qqst&PCKI72xej5qG13ciQ|u zs=d3m=-k1G`c{Da?MoJxD5A{CV_I`mR;C=+x)DMh4s2#qi(_MHlZzK|*zQ1AILD_} zZUI^@_097F`zI(1BtQobTJ&%N<|u>qng7uq0E3ZhA@~ZTZ$Z=PcWZ5Z{Q`T_pL=;R zF0@miUmov5IhhD0?wPNjcjU!Y9`ki(;5T#cYj}=#`TFjmqn8(}u7CsPHoD*g{v&?l zE-k@F(R+UkWM6cN-?5pkGuypujj!1#d`kyZc^BXx@6+fPX+wZU3%Kay$L)~WvN5`!nklK$NZg*nRnP~327 z5w7{F$2xN`5sBYo0}OwJQDk>St=k-_;aKBW9hG_#=cWx%Jt|Aa$KImOq5@LuG$%m6 zzWz+J1;6vNpv%*FMBkyB_zzG7Ci@PxU^M&0q8kiS_9phs=~n^%Gacx&Dylxr0zSAE zlBusqfO`NG+U0(pFX^YisoRaavJ(gLC_8({s&euAROYV`-Ka$N5Sk)mR4J~@9DCdc zvIKS4HT`3zyQuAmHG3I-XRp|oMm`9e-J|~anwb>tM3qU&U_b=;Mb(Ydcrs$Kn zQD8($Wa3)OD>b;R9LxXjQMpx~Im#yRSYDvzb-_=QYPE@yc@{iw?Y8LAFaHmKrz#XZ z3_F$_F-PoXDv1-7Rfx45uTiH+NDboQ?S7e|wbMuUlhfI<1*E|=8)KR>#l~g#Jt*wE zTCA+Cdw4aGY<;@L6R{lg%F(6iIe=If3lE-1c z+JMM8CUFt7bmm0DBM!0vyblMHPN^*x^Oh%8KA2uWzFg$EyczALmyG&ZR_xL=wV7gs zn8<+FN<)gD1Y7pPID^1|&++@mpz)6L#$5B)uZvv0GWkvE!emKyJ9(sh&u&QPCPY#@8}4*I`plS!mf1XoZ+?iGXVn%S0 zgH?bBk9t{s&YoiROY$XtM{@$vffs;Xp11}OYN95pOhnLBo-f1^)#nqR0mKnm^1;;a_C%RSiYCFy0!(>^fC;ozS;*1wfXiYF72>k0r%vyMdo{ zdfBJQ3+$jDBH`7cNiT+f9%C6S4`^L;eH+CZYO$&1*w?)XzhhhY&m&>Y8Q8Rrq=$_j za^?8cvhioSxCsdfiPbz4UfIdXsUG*Vk!5@_@_^JTirOi)^ufL(x9jIiHJV=vgp8Rt z3TAMok7BvVkqytzOfrHL87a9>Y{`6QG71mc?>@kx19Y*Otn$tJnxsr3Dm#zn5d6E^uUOyn9n2o^NEgcx-2IdP2c!Q z3xDeB7M~~%Fp9~&B6z~~(Q{zkvLg+T&8fR&NS`#zSFbu9?0QC4TnHR`UZK%QJcbvX zToTiNOgs>bY1GkR*exo+sr&8H*Q2+tTArPHhiF{lW`6g`7;rRK+freUEv_=%_4nEs zF2FSe6w;K1OJz+4wluPe$iCuqEjE$66e$3rsn+cZ@0d>2i)5JAOMiHdtUK z0s(l>>pWF%Rw2VGBV_9`SV3fypAUQ^NnMhz~|xOGY?+>%Uo{EzwN9p&?K?qXdq&pDq=S#M5&>x zoAhY6)wt3frAzO+RI)o8AN_Xq3ZrLr1l;k1c&f0Jl#~nHmu_JwfGk0m->}>8%#TZz za!I=0>}O&u$}2$D2wKjU_;vf2R{W*xGth0^D~TGGC&tJ zP4&Buzr848P2V8oynlffY-E3;u4bxVOp_@Xs+HSA(-LlvfEa@mOdiZ#BW1X(1I#qd z0xFU7I8Nao6LDT^$bIqNdh45sjZW~2u}f|nQxN3oWKksss+1tPMUoud^h7bTt{rS% zXK^rtd5_#h8u}+ft}=4S2Y+UtJ6NZb@I9@Qs(G-Civ?_;Hp5?29_@5LmH3@AN~LO^ z^Rg*6c=ZP83DoGws}#mBc^S-oP>}ztB*DJC6?R#m?hYYtT7o4La{y-m4}8DdHvI z4@Q%1)1f$#9=tux zCpNbW%(T!jL-0u}`#B>oqbGf81`=Ko>uE#so<*8XutP(nl+ zMFa`yk`j0dM+F3F1e7l6j-f+Zx;vB<7+Qvg{~EZx?{m&`{?F^*i{}j= z7-sLi_O-6~Ub(!US+*eRC}?Y}g{G&rUQ=KZc`@77Azlxk9%~&h7*Zd1Rw0`tE^Jj?rW6RX+!6P@8S>~Z?l|@S*XL3eON8Og zwb?YFY(rTl@qK&NR}}8l-D3aE;3H>PuWG2RZXG?<#!9u#yAYvC5S{>0m*b>^MLx2q zi6X-rU-pk4*conn>|1|%5q?p7ka9k~#kjvt98>oWWa~MhlvVN>ELCzBS!MJekv0x$ ziOe(QPjmVS&)^sJ?o~OhZ1$_$+4x1Eiq}VD0c(hhY)<0-2`Z&QpKmTLsQHuThW*(8 z-g2K6xIM2n#SP2{n`8_)t;gB4=cZywfUyoULMatY%2yWt{V`7>-28BK$6@@@t=5fC zMMs5pBtB;8ouIs?Zp#KXDsG>EeKKp=J~S8@*&H5+VyDDBD||?ROUE3>a^%r?c6u5c za8G70-I|`d>i)^Y>O$EGGm333Rs$;;&{`-aq2c`DBu|$QGPwGPxFefJed3c+nk@+{ zzDXXWWvv*8pVMx(n~~gEnB4s$&7%5do1)&tX!FyQMK3ABTEHI^OxnK-;aDQUIBqGw zF{A;?5#9%Vgjs@CS}&62FwOm|ySdnJ=TL%{;X*!;rm1%u(niB4`sP?t z+;$(^X#WuZ!K7RV4O{mt4W|ZvbNcvQN!jb`pMOo77N)gwuX@;BQ-tGS1K`iPn*i;MU&Gbd12H@I3ueVG;M#k;O-BD-w=^f$u76rfkW&_Ay^ z4w$36S!&l$w42ZUiUej~`CwHx$bblrKN&nYR;O-R+wPlPob=i_R6;bT2#0OF&)o;l zc#*J~Y6YkpdZ@iyz10sqH`>y#5BsPy z_^2Ou>#Stc`yQmBww*C9Be&};HOU%*2>me>0LC1N5 z&V74!%1D)vHxUQ{3Q!}LIFn1VQIIn%E{^JBz21Euh7-0I2VT|U1;t_wx9&XE0WGIf zOok5A12<|95TSJy#xQ1$!*ZUc!O=kjk!0YbWvu1(dr2953jYb-5{?4;gyAU8v7*TZ z80P*SumUJZHGS=SSc2;3QdX#0n$gd0`Y9O+r3^GAJndzJL?cj^2_B39z!XVcWHqzDgGUn3@@fp#7$aI&+cp2P zob=WHVV`zWlhD^sUf9UcBp2va;>oO-L5@tNr|j3{Cr$J^H{Cjskt%tT(s3{r5*iTs z5LqF*M$4Pl=f|n4w*4$%eErPx@-L)nXqycU- z{;SIps>{NLI1YD2h^e~?McOEC{@~t3U_3cYU-&AsGB_lsWz^I=;eXF#t3f{-hf#y9 zHI03C2XxjZ9ay6uqP+k`p3l3+jzV{kVXpBJQ217!+kjFQD$GnwcaKovF|uv^_?IS2 zKp2S3W#Mi7#(9jP)2^R)0>T?s?fRfqy(Xgsr-S=otWd!4@(E72j|xNUnQ^qCHyi#02bz_IA|Cw0AAlP}NqyZ0QM ze=*7=RLH+ZtWQnxzp^r+j%+N71Da0>NB*x8&HL2$@W$(o@tgi+P9@`gHWf=NE9)P= zlY&&w?5G7rdsTDz7yuR@HgJRxCfV8P(`fBvUg?mnGI_JkH_bQ>{#1pqo6)j3NIw#( z)nsvjmRGLFYk>-H5Ycg=GzSy(ORyC69*IVvw&62g$ITxSa;p=b7AzRGKQP)OtX)9)(gA1hCl~#(p zY4Kh^e-5qP-nDXE`$8-Y!i^FkH9QD`pW(MQBRt97L?1spG+#UkNl0Q}cUtv<8{FfX zAAF*$@}S?$r)ncHK+o(ufTb;b($@0MGUaagUMvMy9UKPg=+Dj$sjIf*=cJQFLcT0w1VW9WJ^_CZv!!Hiv(@Zdi$R>%qB9@oI*3HN#02*|fEZ zgaBEq`+QjG+f2I4s18Mu3>ao$8B;Vwhs?1=5>3EvireSk+;}r(xL2et5$8i7;%+*W zYmV2rhM3!uM;YI7Gp(thEIvx=nR8`}-t7!%$=I8+RozN8Kv^gol9SmQNb{Go=!_*n za2beDjDV=9J&e<0TBEo2$CK>B1nh)^O*;2IBFT&9JJC29wad)aWE)=)bA_4R$L2Zl zJg3rC>-AE*AN2%GS4#*>+`EXraQ?h=&tpx>)HcrTfVo7!fyyN?Aat?4e4TQo)Fq`~ zK&#@@(9H|!wz;Q#%LJ;fA~~(3?vj7in<_M;!tLJUHq=P6G%TVN55%l{Yi-G*-D5ub zf#Wreoz{vl*FHPmH<_ADl5o9>8J&uJIhvc0k~h1T_wmVHZPt9f+__-ODgU)vMsY*;^&l;dH_Ep=^APGpf55RzvAIqo<=1?G^S z*&vNIF9M8>;F~n-jXz3o37c5n`VguLTPSCT8ur=tEhRF4c`x2CyQRCD^r92*vJ?8g z;OnOtw@+|BU_%Lri!N69w`RUiMS6W|3nq=4y0dra?< z^EbexzpWz#8q$7#M)C4Ih#=NK^X&GmC1%xp43YaxO~q12_lj)9opNM8UygM0-^Vgq zegt3zbvT-Ko)<*A#4*fQ++#PVxU%f+^Zg>|zs?TRf34Q)AEmbg;woC^h@9;*s=I|p zFwn{J{AU4VY1arN5#mx~CIvFd+#K6rz|PS+%y{jPryN!;ajGooODmrkUj?SAI44Y6 z#a*%{dJA6VVyct#^DUgEAJ^Y4LYjN`#7#M^i{h9xVLG$!7sl@U&%%2~^%t+NMQy=#1CJL8oBi{%!x7R*Wk>1k^qUPkbojz;_!-yEm zBL!y)uw1*>BDsmnZL?Sw^SkE@c%R z?)7?BMNg7w5eW3Bn)(36xD%tR)TpoSLd$mAw;al>mgh}OoI86UUizwMIf`kXOv->0 zIV(!S03?yVO->?xgg3wk2H=$vmf4An+y*i_TuJC3hU0qv=4imGNgYaZH){rrx=AFY zg8s~(35$wysd4T0Bm=W0e#$eD$RqQ-faA-}T&8Tdz$y~3qBq>cjEk%1XUJLAh~pY; z^sT3UANJ5-+_p>q2%w*hsU_gXC!i$kT3Y&Q@xs`Dsa0d z6Wh-dsI#JQ5*j)M)QpjCG}2$2qyd`P)l0zrKB{H>o5u{bTylI~P&uc>aO>(;o2 z=4q9$DQtWens`V#kq&Jbn%fLjzw`JygA$Be$1%NDW1+RAwAgaSZk1CBr!t%cWSNDa z#g0pz&S!jXR8R5*06{=~Pf^(cpVVkQONosk7pElt2&KZdvIB>qg(maH0Q#l2T*7>B zqF>n5lk}14X`SPG227XrZiHTqAUfq$at9OXho?v<06$zhIUbZe?Uk2s4L4%0TKCPhd(8K8acb>2t%rPcL7j)sB}{uU0R&qK)Z7W^OKkKs~Z!GM|dg znmP!~ni=DaQ7O?tHqT(9y%_Aw^%|n3gq4o4Kshb6Y94;bME38j1vTFHQaYJ&E8Pqa z-5UFnS(uM=Kbq+HXSn#WTdvhQ)vgYb0dTQbg?tiC7%XgbXH<7{uZQ!x|5Zjd2K7h(bD7SN9yI2@q zC|*lAYnG1+{ls4k%4>fQC)+jUDig=Ql*%ZkyL>NfhZ$`C%wd8O;L&Z8y-az70U8CB zWQ6$#sFKbPz&JLsTARhj<(@bW9WWdBPM&VfXxvvqS0ig&b9tY*SSXCteYvZHW>0a?pk+{2yP-D#D$VZ4aJqZ{Ux`Y*QfkdFCCGwT9 zZ~GW^Oj1A%G5<#j0BN4GAOCQ{%+Ts@0n>7`X@1BUxG76lm6M*5ZGZ{ENsY)kn1{Xm zTk#wY^DU1wH8pQp(w@JIZPdu9aas1f#**?Apg9bm2^zHN@`491@eaGx{4b6Zn6-5a z1bx7msD=kYJx%>yzJe#z!FCsZ*ra!C2U1&`MDaT-rW`0t^IxmoNyv9fy%jh}5 zDjm15iiec@MPJYK&sR@4zEBUXKu(|aZmdjiExn4^n|^z#oxzXAple$#9bZJmL~uJa zp_DY#_p=rRPSLYFAUyB&1j*`eG~IsAwLkW$VdsyDh+MWDB>aQh!FpM%z!7eKE~hVBKhs2NL^I zZI71zNTFbRjz)mnN%b1oHUgY*<)=CM(uapm&Vla_pzC@?3z6Hz2jUC)0!#^im?u`)?Tb+CVp99?j5j^3nP;= zK#gS760|L8jV?Dml=$#xum2r_U1|hH)Vy~><2w&G3jAX6NL*y`L}f5hBT->93K5M~ zRWxjtQ)=3@WmAfPM?VxD9p^>p1Uv{G9aWfW!U!_ZX68+?dB0^6;2aD>$X+R~yU_pT zQKJq8+x$mqg3e#6YaYCS=)B+R_yCC!DN7dyO~Ev1meK~}PtFy~O>fu) z$2JymkMC;r${2`-OgYEd4ZX7&`fUj@5Pi({)@EdP+AfIQ#bQxB^H}>QwHMD1}M6W3ovNels6yzCK>S2A7V&l2BCF!bt z2GfkoV?M9+YBP@;bL}Y{gmvsljw!sLg6Ay7q3WJ_E&gvz#DEsa6#m0lqupz}{5d0dUOAloKr$zG zTmuCD!-RH}y*VqE`j-!r3s6WcTHg_x(ObKDwwhsr5ig6;ZxU(*2=WOc3Fqj%l@ve> z?*xro4ZlLwS)p6;b>X4X^MAR)9}|tKmRyXO=6lLWX;kx;(6Al1Ll!D9VlAWbiT)Mc zV|q1u__ViVamHH!R3_+)vSasCntIjw?PwX6ecOaaS9+08{!e-;grT%fu9MwxNc79c z2Ypy~>ChKHG-g|_NVnDn%0an#2DcRhj{>s;GmuxC#577-_TsfN2HqfGCiGs-h8_@h z5l$WAAN^ey^ADK7)HLwfcqSF};+(j#ekHci!m?<_%P{oqM^)~j+cniE#~N5{WYJ)0bl;eRzTt>ts&bQ@EuEqmGb?}SYuL1J@A>rk3m@)>%t_MA+u17vv840O2C|xDU zTgW#?;Bygr&ILy?GYB0DUD2uvl;KhFgp%g}`#BSXkG8e5B3yY!I9?VvGm%4x|9LWnKPpT~Tb<6W!21pvXKbLbq=CNIdG_H$+^erSY9>?1 zkjhW}$Jq-?3`M1|b6+DP`NT3fv>?imY>hT^9TPyER#iy~c{@EvFLq z#Ld!7t|QV+udH7d(*&8>65&QY#Ede#l41@?s^lWt53QiTLG3?`u~-4xzfJ|~;kt&H z&r)X+3(b{3*YxR8g>j%9WPcqZMEv6M{CVIRI73KqweZ~+Z`!4nS`AHT|MhjRLDMne z8TtF_@fjLn!_1|iBTEw2h%=d7SJAc^zm``7nYgWRn(@Pn`$h6U5C8kO>40wy*HL#R zlB--zBj%mN9M0iHo^OSpqjUD_Jo+ za$l4C(IkxsjvKBagve3l2IaI2%gy8>8!fCO=qLhIq(Uss)SfNRmz)U%>>MxM5K%tGMm21c^&S@Lhe<*isP5SvD#!m<{hyx(y z396#}TPxx+0F17`yr9AZ7=mR%HdNX_oO`b#?YQN%RoUff4w zY?B6IWv=3SA^EQI;3eD|OLeZTTsH-FUZaO;KvfApMq^UkF?@?nQn&<@vDYL5fy>_* z))_f@dqa4{?S6vVze0eTz`nBAfbqv2CFD z_=+dPn4s zx?U+<;X9ba2c173Oq<4n;c4j9x<9OKicRD5^)lQMANR-{qKk9y``=oglHtl^tfmvE zgELV6`!?RC?-#?bW*Ai+M^l?)sI=1#Z{d%tGPPw-$k6D?agkwMIS;q|KAr2t3azCv!~hy1^kDZ;KFQ? zHaf(>j=0B#SAxVOVPq(XO1ytxz2f$jzhAZ5{*Jc4tA+7@_SGZEay=oK6*iQrr=^O?Tobfw8m2|uGjJN zc9=D)9MMH?+C6rr9M2d9`r;1Ej z6i~Z1?|&&MIuc?yN5`HzOb8SY6qUAO{vF>m%BcRj@bmrqFpE%nI1b9cQzwiC8ao?c z=#$%mHTxyJfJ#i@9~$2ymUO{u_TI70s8lRgFzj=&$u2U2Xum|?3eoC(<*v_pc5i%c939!cF%f6E7!d~+jUp4xtr#LlE_fd~?Djp{q?I|2uRt5URx$r0Jym#m z5p|n);26O`;Ebw6L(^vZ_SW2?YsCN&x~!*2QXhkyI_lOjde8DdZ-G>( z9UjnS%aj=lL1l-?)C45_0W5bDX@Z3_Fp;_o8>Kxr@H_;~V&zF15PM@YM|PAT?qD;nt9z@QTo}e0 zv7SHFvyuCap>J7IXEGf@j?|PK*JTU{h}HE-IXr%wt1BAH5B#x)!qsT-wVhdty3q>n3~ zef+C%9Z3>U9*Fjq7>|IE(82II^O>ba{Gu4Pa5wVZlt4tAO0~oqd6ZL!AaqcR^SeIx zsPU~0*eZRXtYvYJP&xc&xrU%R5m}y}mcm^+jqAEQRnq)hXL{~SQR>_ep1BoWaHJc^ zjys?_s{RO|3s#(iOp)UyfkF1FAPmm4S_9s6N-cP74 zxl8uqq_zuP|e%@1~9HH~WcUz0qWc(4tE>6bG|MSj=YMy4H z5y$j}2N4OtfW1e33o$CQo`|>;`F>af9hwKKzrxG?{rny&lnQUl1%YU-BdvIk$u}BF zJyb3du+Q3RANjeRyA04-XPTuFtEABE0zou`l>C9%1{6s6!n`z;u>J@!u?L5sUASsnX z9i+Qkk1uj`8wl$AOa|15M|W%A<$(YHJ`$CnamStU%|51h^>&x7=kErd~@EI*bO5}&BpQt!UdAo+Pg9`r& zMabQjd37pNoI8)vni&`n7!ue+a6&Jq)~z0-nx}&F{X$XSs(bC?4Lsv3f=j1>p&WF> zRDIqnAB7KFXKV4BkG_A5bFp{Vgw&s6nF)c&ECt%UE|+!~1)TFztn*Qc%vS{i&jWJ_ zA@r0VX;^JlrZXs@xY^B5QQRpE)V&zOe{M*DBLFJ#sc=H*YZUqc=VKmkm+n=u+CDn)wlAk*#ighe$hdh!<7)(-%`%Qfo3H|7G6g$k3@!lkzQ=66wi zS(2T2<6HW(dp_1-ospxSBsrmctD)LYWQZW)g`irnUDARRuwZI7+dp>=EAsXSE zs7;{-boBHo&D1w;y>AQF*o|E&GjDmZih19OLq^0 z38VtLyqH24#lu7I|2ks2Te}v?&%S#kS!#zy3DmLh9nP9BNq-6d?uYi%XWei@B)8Zj*=aMZ zwE1t6Iidj_$7$aG=1%0|ZV4A46tFKd^TYnacxZqL13p`0BHxb!*?w-)lal=C?<;LF zk>sj!q>&1meil@A|GGZK=N5D!?YX4yNG)F{d|11L8>;D(XHIds?zbp37%c5Zf364I zD@xd}np9yS&ghvp463$?#C1OaJR8P{@(pXI)A8#-83+S2p#4sG9qKnZ0V!c&`s|N| zywx?7BMPue@({|CtBnWuGEqp@Z}5(8L2|Y?I^b-es>$0w)zMKvjY9=oHLh>4)C68i z3g!ZzANa1O(DF)=Y+IKO5R&i?wC9ZV=x{_5Y3s>#&qq;fTXga}Ya zL^y%kEQ=;?eTY7QUAmc?y=0-k9|`V|esMo!2P1H1gI{yW28bET2~XoDvXbG5P9Z*& zzg6%b43sV}tnR6HSWZYx>{<8t{Ato(YYtO`IyP|EkT#S5m1%A11779T zk~>hY>u%hWGx`)272eYE*7yZJXgGWG5C>ArC&;7lq9m`Z>^Hl>L)%xn6|EkC;k>Wc zM-JJ^1z$cQ8|1meJ&dnTAVE@KS4)^IaVe!;tui74ui&_QJ*}sg@Qtt!4IAS2#>%Z@ z>Hx)Sw-R)_$)P_Wj$l~spM_#w8CxK+2 z))WgpUDROv$F~5TmeIyS9jWh6d_DuhcHF!EkT#0LKhW>8%7dB#FuNjs^i^FS@-813 zl5+){LnEbus;1^km41Q?9mce-{OMQo44jX4GNK^!G$3VZ{xSPQPk6Dh$H69nbGkg1;cn0u<~ z&;N4R9dG|YMpvNPb_!;be`CM>>69T6h#jgm+QwHae)rdodQV~)&`LMryOq~pGJ;1y zB_1^Xa%IO%%dX2}C_~_Zsn=sLU?33n=53Pg^4>X{s5frC5di}^1E<@*D4-riF_P?9 zZY#}2sGo|0JQ#+Z@mI{m21W6rqZ=kZ`0y`w1dQK}e*3{w`)s}^b=_A#&1Rf2!@TKoGIs}HS$vNwuV)H`Rqu78X08{ zHZ416oiv_^(2Wq=@uHu!v9r2D#07(d(_eAdp8)+vPt;$=lmUq14pJq{bvq{X@o3oa zu|mwJD|g%kxNWC|H%1~zv@%6kb~m)n8v>WLd`co7$)k=M-UXL{%H862ITh73f|;m} zL(zaBDyp12wCv6WekZTty}`CK)xjgYud|s8-4O{d|B86gqBiN9^A2d;4?ybkOA8R@ z3dLfnH25;Nn%x;{JyX3;8~SWl64LM}mRRm(mPo6N(R)6DEj6?6@3eaL?(<1O41*A3 zKY@99LYs`*vXJAYmW4*PAAfcLrqF=Ps^N&eCr_+^v~HrDe>%VRjw1)Yg@~Eg?;VyrCewDtwJxDy+po;g$_+w zT3(56@+J~c*tW<1wQ2yY(2)g5*EhgM4nJ5w+nk^=3K0+7dq|hgR_Y|bRjY@Y#N?g%~-v>y<_2%-ck2j zFve*!wrtDfuXqSA@a(9}?hs=CZT_Cn+Bad)GO2Hj-O}opPKYxe=5>tXd)X8krsVT| z6ybK?B^v%v1HoT+?Xz6pHS~ArCRzsIm>a6$o2=}s6=Ew}4e!VjV!aDfJNY?=k9Z}A zBqfi{UsHxXx;_)^P@Z$AzTy?D7r9S?lN(io)O{h@1y;-2DDWm*47f+$2Jz3YIJ5*A zHy$OYybMcT?!SqD<0pD$_<(}YO)cegxxVLX6cYnU89!?X5>6dXhIL8Cc7J_IMsG$p z(fbka)T^0tr#GeCA|U$WWkOQl*ekMV;DimM-zH%*2!vUO`eDWNAK1zJylp#BR9QV3 zaJT-B^M@of3QJu~nObtj{}{YMhGfweX;n8}X2yin8n(o|B|aw5IAC%2aHC>wQ{8`g zT1&-6j5*p=5#YvRVY)|#q5d*i=0`Xu`|AgCFR)7QUU9J|H6ZuLJ2z$j&Zv~=8Vsb0 z;bh~RxTbClmrVPH(-Cz&3VU_M>%9Zoc-d-QUdg?EV|bd;rU0Y~pzzU}{hEFdZf>I$ zUBiagYQ&2q88i@}_JP9||1P#GY zsEWX;g%?@NXj~%&d*{2c(S{(H zy$4p$fl^Is4C@{E2_9=&iIw)4dik=v*CZd9p*#W3e`4Gna@Xrf(^S+=_3;Z`q8C5& zxmZRKhqP%Yx&v#@i)5zE-U;7gtz`FFEGnWqL%0ADB&3by-eWZ465A6}FIia9*!u!D z2?(Mv4Cx3Us6LlxdqT4sKT zAT8ZT*_Lp$3N>d&@nueKS+L=Q#E8MD`a)4KyU`&SSb*ry$V!;%o zf(LHgynRUdG}sWJ3_0jR2>&K=Lg__-wXVB+Ro!6?r6b?M)zTLuz|wX2w92k#>;urz zL3!QNqJr;bL5TTasK@YdGwu)xkedg070gz8_%dU(mG?CZYS;?8p|6HSQSYK-{IU&- zbKbnUl@*&VAuE9O7n@hHD12Jy7yjGry41n#(u6>Vo{^Rt3@I@74inBCm=uT&2MOC^ zsMM|^R6dgaYT&hvV(fBWpF6bLPF`AK4z&2xRGyh4+Bxl!(~xIQejwL2 z`Mec<@pB`-{4#EtVXH#xU!$H0g8B^6@M*In^Ae9Us-`gbfTsLji`S zLyLxnhJ$NoC0jWY(cgL#!Qpom$rqNx+U|1v6H7%3pD$gqTwiY4T;2Zr^`wz~UIEhE zuUMy!I+n4$PYP_(EFzywCSs0>JhsZlhGZ|4PL_V-nB-=Vpa^`^YgTiTI!?&5Cz#5l zlpM(r&v=j+s9fMw^&CoBG{sAiT$6pro@UduFq`m8LtN*iqN0k5b%kM8vjnY@xJ-C` zBr`^p@QTA!5bS;DrSfvrN~X@#mS6az$)HUHlD0jAP&fGICj3_1l*v5M>6%1qc zgb*P^Za{kZfIMe$kTD{$^9U+j@m4S)BjMMPO&U$Udpe|zcmK>0AX-lfWG3{)BC})VYpH|C*ul$(p@?>WHigQaRQle zp%6h?$(r>s$+FEs+GZtsVKXTPmkDl^Rs0`1a!FiVfBRA8K=UgyvJhQ{#k10c_xK9h zE&qZaVTMq#k1>^{EcYUQswp5(egiP-an8=yvN37j%kPW;r(cE25am)&VuC-?uSLmH zhHapX&FzH6J#$fm@o>aVN|;G;lLbGs>58D~ny#4Z%Nm)Dii)a9^8TWo>w@{(uy+aN z;#OF2O&`QDAw*gx-$fx|)P&}eBI(G}6d7nqFY(cbNBvA~&X2(ykr4Eti(PGXMMcHj z(cXjuf9+yN)BI8TZQoTU`fZbU9vbLTD@m(5WB>yvs-eopJIlW=nZEngtvUkFG% z`iu+m@|YrJ>E2g^#l269|LLVu;;fmkt)QZ!vJPG!?dtW{W>)xlnQFuRV9KZ$w$i_2 zq6)P7&WS8?KB!uHve1_FXt>aL9^ah&!Hs#M9f?#Qi?wfD1Qjnia*)zo?XLG{Tho3lfrLR4bAkTpTsuQ za#a$0@R(Vb1*U~Q%M|DV0S<%Q4y~VH!+_a(Bj=R*M}s3(s&!OaHjwk2pcih>gy`gz zn#i_}1rp8@tXP7k7nU;HJiWzg=jh?G9t95NbHh1O8v=YnMrA#1s5;xv_3xV{q?-=F zCn~6{98YjK4c`nqgY2nD%gD?W`Abw)%wBZoYZNPWsZ@H}R*$=^Pkt8=Kwv}qKA27S zHS{{^d-0Ey+fB7$H*5r2#=1sT*vv^Uv;|ME-U!Kpb!Xx~u6i!Axxd+8!pFjw+b{(= z$QgS)EpTda&{A4hDIR>^17hl8UJ-3IS{6X+zU49ki&@5q?<)Sr8)H0W(i4wzA;&TZ zJyfT=zqV)QdsgEmyvFn+q&CdlJNWJ&Ahc(c2lQo;0nRHn1PJ7QC4kX z-;GmTugBFrjSTF!l1Ad?2}F#T*oB8S>UfqF1ta8>LQ$w?NzIeRjpV%T7!pIs^AkZBas6nK_u(ePR?%T25}J5>iam$U!& ziP$N3D^W^c%)Uj#wJFwlD@Hi6Zmw}&6O3$~*tkbpYeHGOWb5Agwx_P)cl?u9J`E-d z5ilm56J4WyIZ9*`LN5RL*Zt^Em2zbKm2tN~+mpFPmE_7}*~CrT>J!C-c73dwCNY`m z*&5rGn`d+Mvopl)iM$_89wdYs*%hOO1=}95$JAfz|A^H?>;ygwXkXZ|&k^v##^=cKw2p`^-9w zatyhed zxU%SZoE0&2d2?9=*8Ga)pZVqQSIKP!b3`Nt3uc~cV}QXCeSXD-Ax;Yo$D_fdP7PM$ zMJ+Q&KO$#gEYB7y8K$Hc&{v+2C0U%V5H&E=M=qSSe6jL57@yle`i5? zs>rTad>n%rjM~Y5zh2eQ$0woSP|DC0ztsv;MrC|XU#(ujx?o(+(5Ky5<*@a&csSk@^&-oc17)lszp@)pgUqj)g zHTa@=>1+reqqSvq4iO-_mc=lIpefI{HUoTCha&FG5c<+o&RLq-@2;)NxPqrsb=%`MdYKEqHJzqKU0!jU_cyMePFpV&*mi%2 zt2+t@r*fCt_`H)No8eE}a$*&5@aLCK_*z(h*1%BXbjtfqx38zVibouKZ~ljutAj-1 zYX`S`H94ohpfS~?to&Vdh~b${Q=9hL=ACUa^;Zl~!vq>ux?V*MvjCbeo^+E8ZS-(6 z&D!3G70x-2m<_W{56pGRaUF-Sd2wI;)82??5&XQ;mQboQ-`^GqGy%38MOxk8Oq`?h za`Uy#q|AJ=Baoa_L(gO{HHx+Ggdy}!#$qW z(blt_l^3KhvveGW>TJh@i9IoHYk$ML8^)9z7Id0SG;7(Nw7!|D|BRWY&_RfEn zT<=#ldtptf9DHHM*YJCv^6|c7U;Ux0n|JK3r!iKWhl@IL3qsQMk?TP=-*26Gwil$k zsVf!~8DXH;uvrs}<$1Hnc#qGiA>MV$8<_XYs*TT?X41vd=Z()P`Y;epdqNPwB9sr);B(@CKnbN^?$J%DDD`s6)yR`q+dqDS%h7H1+u%nmPr%0R&_lIUa7uAffuh@X=vARfv} zb6RKS!NAgS`1aj4p*-KNzT|=Tw%+DDc_v~_rz4|z(EL(7jty_{IGM5?X(lU~EBG08 z788cxpKshQdv`F?YEz;}d@x8F?XMc`vK(rg&z(QL9A~>2Jf-@>bF(%dGE&L&=~0u+C4r zWI|9^T+HW2Yp|?Vl{{a3ObpGgDIFcuz3*)?U75nDocct>*NK<=*@~le+^6oJ{8hbq zh#O0mtuBE{#@43Q8STfGJ`ZwOGt07=^)f5D;Cg_dSgu z0Z$`Q44?}fFzUqZ?Pkz8Yl?totx5LD_E2HG!0vY+xUwu`$Vn2+N8@y(q40f2xc|8C z8`Y1pr3lU9_EA)R0UNEjQ7bnLJW@HQlu~NJrwKGWUP@064$yq6xwI?t9HyF+NY*`8 zckDugJtm?)aoX#S>W}d^SZWPSdFOJ?v%KmWH=a!;^n5wfP0rv`SA1anjQx0I+XxX7 z@^FR+Jv7tdjT5gR_lo1|r1ZWeK%R8hTjEZwYufc|wb_=8(dL0OC9fdVG`ZLn5Ux!xE)JZPVpdQJ{dMX{x zEbD~}ueuB1zo%8m;*(Z=n_OuirLPdfH&(>!wxd0NM0Xk4J#}#!9MO!(YhOd-mx(Cv zIsT<(_Wdr9@KePk?BY#J=xDxHwb;jvrn&JY*onC17a9+%*_VD*$ic_P^T*IlK8Iuy<8ag z-1iYlyaa<##`p}$#ismNm_&ZW<1AuaDNSLwR27_y2B`rPz|x3kI*QCYDR@#+Sh|8+YIr;YS79V?n2tsioAr1|zody%Ce{G-YLVX&HHW-4nc| z>$d2guZL^;W{BksL~Se6(_5m}bvXtC>n@By`@HQ$$x?yEP^YNTqT}dIMxXQY4$T)} z3Xxi#vy=Ci^~v)5+Kw?W%H3nBN)KhE_bixa?Ln#RIk(d9?^QZN8of)$N2=O!TZ@*| z0dl``(9CG6?Sq!!(f#Y7>?xFPQ{i>mp)R|OVoXG!&FEJ$u6*OrPbEg*?_c62aPSH5 zn9+S?*&RW&aJFj&-S--}kjSWB6KH&bqTDT7S!wrdXsx{P+CX-ItQO;qp3K zV(YMsx_L$4i`b@oyDU+ly6Qma{K>OTobj@J?yb$>NtxQY*IUXwt96;W`gNZV_cQ&f zhC*@b$0qd|nUlV+F&$~-d+8}=6!k_V5FnoVx7a!jyv1$hlZmM7`M?nIMA}SBi6qyq z11zia~TiFmiM$Ht0n z!$rgSyqTC& z=&`fYoctb$BUgkdZTHCAT>aUoUs|~_{95n&82sqq#5gjqf|NB-(h6vR60_2_)GCxQ zhh}pzcOb)YAZ7%zwxpq$Hc0LQRG z*Rg`K#EcNR^_LdYS>Y$Ic7L-qcShw-_guI?duDq>fM!RSlz^~OP3#ILt@?JsHHHiupO~I;*zld@U17D~q;Id}xbTYPpKtl-OgwySJK4>l0};V4q%9ml{tc##J}pnQpcz zgD&k1sy&Tamzbd`;{#jvpxFvkEoEeFG``GNNcvc0J9ccw5V4D^e`!H+4wYcpzj~`Tf-1Af!rh@1# z^;SE7SO@bX#w_?;+Cps)ZPY6t%ii~#vYbe*T2JmBPxO4az?VPv&GUEnO^@0UxhY%C zg^$h6!BVr=5@_~I(@WF6y&c8`w}$IC^YBmHwjx(+XZZ?>BZ$2x-NQAn_|_9R zTsOs{&HUyT^R2dWiqZ{>z>K(l0sHl(Sx+u+{cZKbJNb34g*wmLQwMScOh@LRLq0WrRTgC_1r>^%YJ|zNS^D zl5%cR*D9)OVqrUE8;E+*Ws}oaPKXGJnf#=h;=6GxUI<>OKW1~hE^$(Z`mUTqt@y0a z>RuM%T+VuD9h67&tjx?#h(lsVe!Dr2X*W{X9Vt)PnV;?`mi)mj>vMnBI}i>R zRX2ZEn`YK!#L84A&ErI$-)cONS(J*LvSAmM7c?66=dL+1EpFE1BYK&c0(xuhN#0KC zbH~XCrx!Oq2}>ELl%|)IW)&;~w_KIw+dh1TD4ODfDm*?^m8n>9Xy$ITO4$c>qw;T? zIVDbzh2Uzp+$l$90#A5ta18?Xvv(5__|l^#H#2eXX$)`7`jI@{QSM%X4z2RZRBJ&& zrOFPDm@j3?sSJMVzS9I5y4t7`)3#sEd0~UN`h`)MN8-$P(^l_CT4y?fb`cQuci1DY zabTX{#f6G2tVRreoIr`C& z_&pinm(iyEyW5c@#opIp2lgs@M7p8U8X)#uqTHxg4URrR)wW<=-Nz^Swt8MvC;3zh zv@F~8)ox9?*JWYc_PKXbCLt&Gn9cgQW1z6w_&3XK#%PV=5c&0AtB{uKn?tOR-G_z_ z8w|_>Tpo)u1~^BHYO{v-Ms)1IC1VTUvwk+poyNQ_U)qc3*{e`?w#P4XYDy(^2EMS$ zUGLw-Cv%>kOpq-6ak*!D_=`|=;bU2ht+N+1DJ}=wnc&ndo7i4dzmdKwor#@$f<(AC z;=6DuRv!SYH9Dzo9!v(R{}~D^v!+FPyn>YAdD{)Fj4LYV7iM0s3qTdI>k{?W#kohj zB|Njy*~dQQCj6GBozGjrIA6bY5PHbS!4}M+ca_t-_t{wewNBPkc|(j4jaEDYea1T~ zhhARM{j0fdiVk0z>JR7v6@}$64~51ZOLi`u0^wvi6n_imni5(yyanQ(QvXRFWCznV zXc>lec8t#ymaqD+Vh-=(8jel3A5vot2j9-qoxKM&@wDT5BgVPA!y9G(KepaGp349K zAAcpI6cr9dM#(%zlsy~B-h1!8(d_eaDOCfRRI1kzZo>A60|zcL`JpU5-p#HZ)3!RXB&YntcE(OSv! zQ$Tw^$*#4j&}u}oq`uLh?Ig@3W>&sgittFN6$yPBpx*JF5}jT=-(*QBtMpF2Hhn~q zOaGSks$*t9D2tix(ndz((AJqkZYOVrSCQ@rF$|OZHKV~*()A3sE^RW^ajfT=vY$M}8{*AZz)ewh#%~t|XOzsx!Yu+1G7ccw5 zelyNp+=zc@y0PzFW(m0wP|Be*!tuEw*1$7>qFB07Gsv!6-w1s@nwF zXFf<`pjU^x#c2J41H%b=Rr=0}COa96lvl~nBl|@FQ!p~GTIQYz-%Y#y*#n>j2u&>? z&p9=d1iDqdRMN7@;us&W;R>D7{uLL)FKcYCugE0{gu1#cCLlChwYD*FWy%yd|3kB& zTz=U+2ruIm-s+~ark;~aC6EGc6(s>OSoVZo3go^_N3Eur`0i@u%G+wksaF{3YHWpd z;xj|W!Mv*ZdHX82vLU zHVr*W%O_`qv}o}%;ua8qfO&de z#(b+ZR@l9YrG@cdS_mS78D^)hD!eeNdfGXHq9O-6h4=cUxHPiSyryecRU7;gzP0R_?^76r8lT7gF$Rr)36h{mx^-H!)ySih)LA4s>4NWDF0@^;ZnJqshP zZ(Ic`iCoXqt$=C@z-hjdpO|}|(2rs=ir6at)(C-hlDWq|7iy+~fyq)>SGiOum>rb9 zfv@rOV;X+u7B4mLs%|UUrSd=D%xxsO?Vc6HAiT;En$UsLN&P&_1AMH43`G{hZ2kE> z)pM@%{`*5o{*}<5qEwr-fAHW68Z^wjst{^Hh#X<>F$5nnug2$w02=(GevU`k?B-jL zXy3jAAv8Qc`Z+KW@S}E%tg;S5=F8IGV1@@5I=T-uLIZ~EZ_)SS)f*r*dBbuWTZma< ze<-P299!1{*0jl#NXyVnyJjOGEBGmgU(+}uY)pqs)fl3BoRMu`(yBT-mtLtfl62cW zIEdld{i(W=>P!X|Oy5&Q)=Cw-?0scp&a5)`i>CMAhX`E2hI0d<5wce`J?0q6gB4IV zU@|JwoZ!tO_NJt7G7s{uN;r}4u)D`LAAQbVue9@l(gL6du#1orf%7uv!PW?WqtHh3quSLz?I^v`4rF_b_xp*0#onR)L>{-+ zC|7|yl^KN!z<{t7QsJ93`{}6P~iqU z*9->&Cdodo;7B+?*~n$1W;$`aprbkoL(%%kO#dNJ+W6--}35IpFEU*~pQe za;E2OPVt%Y&9+PagqkZT?ZI8phZwC$KX@n0LX7 z)B2qMoVT(5U%tFx)Aw#T?)~%^rM&a&|KA^Ng4hfVl$25BVqKON<=+`Mpb?soO{aV$$?YqHHL6T*AbyBCof~US zF38Wva;#hM8~ndDHK0z$?Vmj6GW^j#i6~1w6$q_QwzrBr4J~yb4-sJ7+(qgyRg%ut zvrZD%@EC&e3TeH96OT^@$}vwESLD#d7JlYK6!47M>&-hbQZh92#S{=1P+t4?%J?^g z9B_>yocN!$ZO@aW`F-Wf!(yX~lb-WM<{nlmU*=Hv&fHhQrUAw8T}@w^fSJd^4tJH! z#hCHIY|bSyvr{l>mL%awgZ+zh_Ptfye#bGZOf$(-mDCDgp*$p%91r>O0h__;zV=`C3BX zTZ;8|uK|N98Fi8I*^jOY#pk+$HTt9UrNJ0c@!e;wQ=02#UA>z^#TB89`%9i~ zbV);Aw7t-gKz$!JYu7w?D|$V1a~Kpe?ZNeXnD8r5w`1P-!};f5)%ESB6^*G^E@M`1 zJsO9IhWfU}!&clD#r+@py!(s^R;)fYz_e_7WMRez_?Noh%ZiP)WV!MMXceO{8l&LzD1KAQrBQQ~6U6!d^YLNOqu9)%(abvEtKlhk$JmzRts z>yEl}h^Yj;K_vl-{)<_)xV3c-;y9NI#WcrMWg(wfO_TT%0@f_d{b3^n=cMuuIGRN} z)-I!XS66}z_}X|JC)sKzR%v}-S|`wV}acdLumwRbome%utwrZ`s zZi%em>9{`0$mtO!@$Su0p+yc2z1un#$)X6>RanOdP#G`_marnEMfFp;UlM1F2|wsg z7pXtEu)3WHv4#)i_Yr1S-*$2c4ktN{dCrKG3q8ct%`pOhCoSlA-S=~FtBVj}`1H+V zWwZqIx{SaJVm!i~^{KCS&Btjy8|Zu;u(p>m$yuK0h;{WW={io)l%6+LZ71j7EZ;)) z2stb{)V54^8hB>)E1wY5s45Ec>J4sQ?sX$rgTtxPz6D0d8u;{6kFJ~Tb;LzZdH@|Z zLS7Bl7=MJLbE0EqNTLv=Ug&V&A2S1W)Z{iLa*Uro<5spTK7Bw$(NSHSTLsBu2fBN@o&8h%4hHS(24kG}Ouo|k)yVT=G|ca;D=W(<)&Q#zS0a_W zYr*hg8`HDrU|FDKtFNtHEvB>Zh}2EPTKq)HMi72X>2jhsX(03~9{I@IBCCPIQ2ymL z3U-&viYv4Z)0Mi83&a9e$c-wI`feJ$;q{;Njs3);Mznj2LhGLG*Qt_ueKK`rAv)SP zO)yk^#gc;%f^U*$DSXxKg@*_gpdHmI-`7Sy<&RxMOqOalMYXoX0IcMHe5)ATdZAZ$ zD6I13N1`8voa*VTb{1luWc!^|;z2h*xh1IPUK`n5E@aLT1!t&ot2@d-U|~z);y!|$ zyrUlUMNQv!c@2W4(K-q}4PX<=91%ra57(&b$Z*56#EfO1(<1Wx$_2wvc8WJ}Br z%EGC|DR|gpJoy~WS19qm;cZq@z?ScskH2o28@h@eZmv^cj3iX!Od4AMY0-1|J+Wln z0O7gbv=QrtM(x!}jqT5!S~9L|qZMW?<(Fa%)j3i+V_ua2HS6A~JECQ(MiWP8u2R&_ zRue6CXERKgwx&~M`#XudlLXPhBLG^E-1D(U`yoBmt5MU#stb$&GgY)OSw5{#>GY%M z4~0dL?L7#7SAf;22Mq5qJ)e$$m^xxlYY=BQbcw%t3|5`T)Oxo@+jZ4)<5isOL+tEk zc9yu05NERAGLI!NyjIqf&fzxnA*V1;m-|qr`3|A1({u2%$D3q<43C2(bzfOud2qit zD2g;RmMsKfg&r!HNNfHc8_#zJ8Lmg^Odc}rinZ9+>=f2KJ)(lUAGtgHfJ`B>XzGf_ zDjxxNE17Qjv@oS}DRh~$d1Tv3Symq}v-i`2`-tc%QU4>0Zg&L6&QC*)U0<+h-Jj0z=lBc_|4fiaBrk6#eg{D={;s&BlF2FwC z0y8r7goGdd+Ia7q7ZTIeS2CuvL(Xzam36w1!1uHy_%#-1!;Q3f&yY>Ori)#-vWsW% z%jdLcd?HGZ4`;_)oR(LKkoj6Ajok;!6vKu3x{g0g&wN^-(ir3nbYWg4oxN$bH%Ad?#uHc*NPUVpiC2_WMlIDXPh)Z}eU{xZL_&y@Ue^46>94yS z931YJDeh_GHmyFj2;NnP+_?ym4!7}a+j$!fK*DUFA2QC5Rog{WCFN-46blJ*DveO5 zPN*~Bo=0RX?|+&q-XAsvrnG(&O4o^}^4-&UIeHj@T=z|-3EAu|cZ(ZPvkW2*sDqfm z?jLsIYVg^V;`8Z?blk*oBjwLjl^5_VnmLCdj6?@!7VJwuf#L{ftYM?{_W}mMy-%rCdj(I zx7ESE!R?ohFzV*Bu7)D)frCzf~(ads$F!Z|yEs>`nWb&rCv{`I9yNi-E>Pl%len5~zPhA5-@AmX zLBv`5^qFa+DbI~>N6Fcq_AzZT{y>9IH9-(+mV9}k4#~OSH2UaQ!D>Y-hx!peMnCz;1xS&*Vs?%77U5_tMd(U z13@~Bfh#$?7hj9d4>!NR|GohW3(IB{S!Cx3XaZnA0q;OK`*4~>$havyX`JiuHBr5V%Kf2kz)fG(H%y9j1K^+`5kB7V*tg}avZJ0pJ4EoG9PKe2zj`|4 z2JYB7QU<%O>4i&}M&z!N*fRD5aiRZH?COqesbm07#@`NPl^-;k5iP_@qbZzUrDA-e zuj2VPA{Yfoglq~7bm=S$X*nk}o;Fl~cPyC4z`$&@u5l73=1Pc$!uM|ovY~jt*Nwg* zmGkWmFeup(E8EP?wOxxNaDMcCgA#*So-{V4sS{C+@mj;p^4XWo4+8C7sRH9i%RLtO zyUS0?-UJJr6c%@)fc7H(5w0qUTiSqh%6BYxG8Iy0>J#{-BR-6E>AKG}N?^W6 zewnEBL*L=?zXb`n+ST|vfSsW zQaSd{!L}7dE~Tl~lZf*1hUl8YJ6|d48PUV}G8^gZx^$#&UPQ8|XTKKo!kJ7dUQAFr zURuP)#(~UAdT)igMX}WqZ4RSeS05E)zf^;~V%#7QT1!lQGilaaqP(<{5mQ)oBL_h{Iiw35ZNRmm8Kq#6M%{MDZ8owy9%aZjj~ zrZ+ygl9&_s;! zGW-rZ5w3Xi&HlTEWITCMJ#*-ztNvrxYPVPCFbA;)u1wd3hpC}I;xXB$mw}wPE`qAC zK7W@=)+r2N!SCfBOB#irI-xcx$?BgnJw#2;ZnM+*v-7RbI3SNM!GQsb4TEvi9mvGi zEA$jx(L;xu08mZuc9(ut+aba4)YUBBe_5aYo-iS|sk}WZhi&sE)J#oxf+UGxr}0WP z8U&{y>c=;~1S;Ad-s4(iA9^XP^szNgOrd`YKo9T$o5Tz8oQ51 zqI~d-^!JX)0sIk;n2ivPuExTsdd4w1`7&^loo~LBWL=u4yJc%U3=HxGPVm$nKovk^ zPY}?*tCW0|iJv~|6S>7rxP|V_F(CnC#SG<=>R(+7Wq8yYyr{d>_@l3w!R>~|v7#~c zjVadr5$+*+@50#^fyi3?xtN5NP=w37czK?>2M>FLs91FHGqsO8aOSQrmb?=85HUe}Hv&aX zVjz!~E=wT$EW7Z~55I66p(JupGa&Y=mpC#*a*lxL;9v{p)u*rJ$&lbx!Kbas#Z(?C|14$hS<0rUT|BZ`mri0U4H#;2KoHh&m_I5o(-a>p zpDL?(=l1I$qkiXbd;?2M!p=1 z=6=pGPY^2!GABt&df^k|5V~+v+^9nQ{u24pFB{IFn{68YK>f?_Rw3SbvXbtS|8t>n zk`b~{?Cs&P8)^1vm3(XHom4tLbNjKnH^&)OT5bEL@@}on2&StTe3O(VulzlG3Q2Hw{@1Mo@QT+`oBbAMu#3nxD6_AwPUQxZjQBz+y4{0kqZNHQ`X zq+{WM)0ts%D{q{HydQ_^Oryyb%=We9;X{!zBZz$Y!WR&fMPIN}PTc|`{aAB^`c4IV zCG8r>eW(1)r0#j8@{|{8P5>}Kc9HV@8!!Ch{~8PWy>W&MED;C5pGLlX&II!+7(v;A z)opQ58=THelSupW?{^3hUm7{50`XVl*J<}jO}%#qwymV!NhD0Nwg~1V*c};XB+kk@ zX1kx2ofK1i8W;7^3spz!IPsUTV5p{p%pdrvYGuBXdKFY znpLHBi!;Rd>P@Kh>Q`7Bw-o8B`Ai8+rQi2&m2(f(4n~)wG^I&|>r7e4CbdKVJG>^w{JMq3m5$PF((5w%z2Y z;QLJYE%rzz=G74q>Npz5hj(^b#znVozU5S!seukVa31zmYoAg0jXq68Y*YC~tSzzQ z`Rm=xR-$)9`qx{8t}cqhn3R+;dVza+Q3pyl+bsM!4CXpcog9t@M<;sZD(&;+G!#(h zL{@VG(ehrzJt#o%D7}&9j_&JO-)2TaS+f!PFCAoSASssorW4iX%P=*vJlp(Pf#wQA zC?p`719lJQLng%qCCszTNDG@42XNYgLF3@G=#T`D&$G_WQ)F<~ zMV9=J%)}~jp=3F>d6IVxMKCuYLN{2iAqN9G9t+Q~YhlJQWdiioBkzUnnP zHTkl_DZ`=s)8`_Ct{Km(|p~Nr6CzYMd?A2S@vv(y*LmAk1m9DD`e<|jcE%7(0O^XFGatko6 z1>){vgp?hUQ=>~m3xNO+gw8TNJ^mIXSr_%r>%@LDMBE-Pc}Ou5w6b6Bm*QS;UhWOw zPyL_~1E1)l{9e+&bfeP#vAtvqrP>UyD#~8J8ZxZak^s!c(NW@~ZvDXmz$&ZzO1V>1 zvgJk1>`CaAGgckCBZGbi4$7qo$Ki36-Mw2E)fix~ifqtRUgI1;G+Vl%5NA^B^U|{_ z7D5C#O)av}B!mFrHokHuSE8hx2w)5%SyqWjkD5%Q??;+I9A2uApqOKx$EOyXDG16u zQ!lMfOKOZF2_>swFQ|CU0NuWOZM}{mvOUylTvoE~du8WiT$iUt^KFriIiI20rn0iK zdzdNY*|@-WC(00U@+)e!U)EU>q+jwJ(&iQ34N95qB;5`T-F>-2ok{mT2}dxP@4LWp z64)W5({Qq@N75o!{n}KYPgktY+B(^q@&WW|X2R3D3;t#|8oqWB`l`@xM6gnGz#qr?6_A76ce>6{NE7*gFa+AX5~a` z7Xzh#D_ieVaNH}{{!_~GPrm{nt;R}QZ3S}@w*S9XtaIJW zd>e=p@m#Y<&($VYPt)<>e6^I0j*WF?lC+5%Wd+cAJhqo(Cv?~S5`GAd@c-3Qz}7k} z#193}(BJXmbAA8o#dHLcyrx+;Cc`Wu#tJdzD@M;SZt>*`U9nE;nYLyjjqyc#-d`9y zL@Y({LH~chpj3W3IK|hEUJ99D-<>9MWj?7PLhLM|)CXwKf)}Q}PGncb@4_N?VK=S- z<5-$|9vb%RhT;e*%c($gA; zh$Q&c^{;pV4sO_nbRn}dRAYZ(_0AJ)FD9yndMq%jnKEu;#|$z`w8vk!*8<>1N!%$;uy*&%yQH$ zD)1eCzE{|tQ1GeFHLW-2OvB}K+gcrZ*P)36T@8$}Tw~&5+uZ1!m_lt8)ln!2k7Q0B zIMpI%Uw^QDne^DRM(F6rXNT>kXA@&*(+MWbm~79Lq9!1g<0uBkePVAPo6PRfNWcrxh|d8%ZosBgx0u^^_lu}< z9~`gNamVWMb%ptC$2>%FBD*0sgik+FZTn@t0Zy{m-aug9cW?7eNTIQG2zsC-0O7e& z?GP5jUaP(;j9kV@CDqYd&ZIFMYL_O#^$|z2BaaBG6_IiC#6qV}*YAq0517`b@RY2W zV0(1{9o0@g?X1#HZVvrxcQ0N0`CUc$A>vhp7$TX*{|O&z2}A7?65CKrSTD1))1t++>;O!DoK!n0dnBOl7$?PrkRdM5O@5B`046@_r9ZS_$}dU1 z(L||_i`ERE8Z2eXPo(Qo?6sy`YudU4I4LVQttpR2b|&%@p{4^5If*ANDIoLh6wqri z%qkp(H1C}J62fGHd)zsN-4m(!7Jd1AhVJrhBCHi6Lbgew6744j!|phj z@l8c#oY!dq;0!F-anjg8;J*q4cr~DE^fkrLGvxKOg`aru-1&mlC|Bc<5V0GdH9gVSn^kcrEEh3&IDI=lxF4 z!{p;7=-r9Zg-%pXD>Y5BZF_40eByzy1H>GU^nNgHZS2wX2hQ`;J=bL)fXpLzj^c&e z0X%4Nnkzpyxn^3WmrHm6ASQy=kNRx}{y?P>s4^-a?8mlgj3t2{`R%tS+Fd0?$c%T) zTbH3SQG6RTH%zY*f#*aM-YbzbtpL>*S;r)gGpf^XTEnst37&QXorUS?{(B0B+j(syFgUC8aT8I^%)QbfAA6U3fn=doUUc;3JA3IHSnI+y-QVg+R0|IBio zV|s#lODtT>7Uhn172XW*@1)(&^i zPBIdgA19i9Ks^wW7F9GvT5unBLAeuAOYuvdH($I+4H(jqm;r?|mV>~*5onrlk_0Ou zK$R`TE9jABFA>ov`)iO<1-w)ndt=bmf&uQPA47@uH(5WGTk8(%I6jVFN)B@`PjDhnBGqD@PzZPTvZbB~;ZJ*$ zn00y{$yN!=cTP7Z-L137bBKnuQMx_27X-7ji!(^d8eR&=0{1=UF!JGlFm%gp+lL2o z^b;ySHPKmtCL$XmZ2IM3zvid*n_vUcq2b0YGU~a4Z``;p^5UkRQ1(ow!M4(V>zWU_Lom_HDal>BZEL~ej&Gva1{YkR}pdTz#FNKP{s9LLQ<0;9u6 z3_V7F#GD8&?C3N*Sq-Ky+3KcVj#2V~p=lZFu+MZf(Oavm?aTS+YBia!rJQ2U34xI{K&@MSP(9(<3<9 z<1_TQdia^X`Y28QwVX#B^#bK~-{Lick}0PQJ+EgGuZ`#MNF`DE{%Xd|yB2L#UgKN> z`FoQVKdBYFBf>b4cHbLA%B1M3Sig}eblf1)SoU^vTj4TwR-2DzO;lD_UvoI#`!b{ctOyXU zkrQrAFKBTSsQF_n<}Avb6S#4KW!Ebsco38}Z1&kP#S3#Ug7JHS>K_w`wsmDZnD@2^_Wy=TI4j}U%&v;ef`J2?3ansi`7O6Hhq2pp=l zK;#+8uicn-7UEJ+IrKqZTdR}0RaPasrfqbN1|E&OM?(Z(D9g)B9cE}whZis(w;Y|o zJD8a<}S6 zRljaWxkMp@=yF) zz-h2$neN`J25$dT@Qw?osCQW+m#c!+KWp{5dT(PyJr~X>yL?Z@_;}+i9~t=0lk+Lg z={}cr_p3kLw|7{~uP#h0bqOTb`79GI#Bn36LRD22UE`*|3qFWREL>Xa$Q1M1Jnhq> znJFx&4g5oh1(uNOT^N9nBHXdX!BLmeg9iWf-D-*L#f2gT#vF`=rsS4h&0(!aHT`?LC+p%#hvcy5G_rcU zE{iVUyf>3d-z^XFKgx{?*yX-GW!Cx?T#QWP)p+0%-o7q!NPd||NO|}tv~6O-L2zJ^ zkm{T8D8?gyP*G6y@|b$3w$N|0|O z211%%JdOT?v-G%hW>w+Oam%4~kJl7=4=2@OR|(PF%YHC1kSYkuYLc;Xh=)e+Ff`w7 zPK34ZIENb~e#kG_&w_cvEJ2lqopZjh%m?EsVYVgW#s}rDNtwm2aKgAYvuJ8DR<;{4 zk}Qqk&z`x?o(-!F??dH%mqo|Syx(=(X;5~mP_`GQhH;<-@arDdmk@$>EYPeTerO3F|>8syK8a`*Yvwd-yMGy1fuDQGyb&#~cDPee=GyXsGvPSKSYrN^2W&x^&alTytgdy> zi+|KpB(1nsL{-eb)rL*`$q_YSUV6z%Q2$f;+O*o5zc@+a7mz&1|zlvY?*XfSX{JIerv!2fUw)EtF zcUjEU(?;7f8`_^?6Qc; zNMhg)Yz9_vd@X?sb$8g}Zl!dh&J1RV&<`)jPf8ITrmpf_cKPu7b0PAr|A~MvY=rm~ zrI1^1JH;8X2IGB$$H|n8vA&!H048K~i&G>waGZtYWOPy*jmg@Ny>Czoi>;U|P(tSC z89>za2HR?eOy=pEizSC-2WLGBa|9*Rj&cTIq_g?L%AJQTTFK$P;_seuQTeCirK#65 z){*l5Hv~WT-&IoiUGUe><6CY+`K; zSlJ|KAdSW97*#?_EKUyBhmkE_CFJDhCFF;T6;je0-VY0NkL5=lliZBOJu6-#jB2`&9qrw$m&wD zQ;Y6saD3j9@)wQrUEMX6lNw?0P_a|B*RQZoni)JK#2LvmR}C)m4py*#kT-cH!tAj; zl6m!0QFo7$UD1p5P9XhL3ga!7c{4>au_0A3{LC1{vJ)Z1SY^IFXy>;}00X6SzO9Gx zLEdKCov5Y1L}cENh#!-x?fu7;+-!x_=`L{SbOQcT>K!lz@T4UhRl4P1lU?1%$j+D&}K9(QK{>Jg+FN_1Ubiu*Tqtx$@?B7DO zh{~D;NDdw3_#V`Fon%M4t;OZ!etQA8rOzVCZY~kMd@Z^&+`oe*{yd9ymblj4RH?Ne6C(%rnOl4mic%%pKTO*yHu;6tBA%)-@ zmxUJdyJ+IRoCjUNCZFirfY%^Ei+j_G#b1ZIO6FEx>WB_ge&)VRWVl(Yh_rF^gOJQ+ z9q07M8j~@h?_sm~mUCR*x%`^9ib$FW#~>Q3IaUrba9h`{*!HzKAACj@dfsVhzF87o zenA|Gd0>m^xH7M?M*UTw6Kw#koSU6(WnG^nwiGH}cqMx_Ew#8$##Eg~SE-@y6PO}q z>qU|H6Dv7>5z6#?^Z0s+lcwh5lS{R?3%?alNnopH*dRpPXw)%Ve)1F1Id6F5XuL&+ ztk?PdsL@RLQVe@%)s8=RinO}6vQ#mI9^#78<$E&BvUc&FZi-luOTl?&z(li#Oz?_!O_M20isk0sI*A1X4DqIt#nt872}J^i9}W& zspbxI?DX&cdNX@Ju;OX)Yg1VfP%BtewYK1G^s2ST_}OLHllo|Un?si7udpwm1z6i- z^+);tO#C8^ukU|{1k|CYCCQ(=wb-~!dxcTC5&Sx*) zQ~KvsB#r`Z4%NDG@+^r2zqa{TT^JwPBQi0tulj0ku?G1tbDH{7EBEa8o;-C!%yvRx zvy^PWd4Rnq0dmsts@H!&^^(|Qot<|+MRA!gwjZUcqaFtO z5c{f*xL4BJo|p~~iz0SC!~W3c$(&l4;o&88qev+06BW)-lG?spRcsSU?Z4AU<>jwL z%w^9-`9;_(^?&SgwDe10_x0|iwkzfKoi%}$z(Xop_e#SemGiD-m2SLQ-?<~Nt0nlv zIl;sy(5-tru#qrxE`yc*Zt94JaWvugDA3rrs1Qs3({u%6#id@g#|(8vVL{CKubWxf zyIKe*^FW(X@t%0hn;P5fwz;;#u^ye?!440Xks6WL=GdR~s@as;WjYE`)ZD7T0aqrK&fj*~Qy+77=Zh4Vz8k(l8y&d$#?zo_PABJ-pbGzg0 zhaF%Ruy_T(qYEgGn#uem&%!%ELz&~{KW!vDV>e-d5>BS&s&%S^A3Mnh4qt0;u-=QY}>XZUV<*a?X`_~+|*)* z{zQQ*xRS40pImljX}f|WPqCqCV#QgWYa_^@dmr#^O%;Uui6}>=>h{K5G0{sjZUnSd z5zvzYh$erFGSer_kgg>=(1WQMY|@KzcOf7a8q&=wEbO6nnGeC#fH_$ zYkzp>x86FzN^FX(nmzaFQC<$=a9MJ`FPsjk?d8-*He382ZxJ>)OP2~+~*>`L;?q;A%$X6@u zwS_Z;*REB^p76`j_dUg#^F5;7 zOB91_?M|}iDFb|mJkfJK{`XyH<6K{*9sob9|E+#6Sl0hK;Xe4mU{#- z{2VAT%@)eh>a_Rmr9q+cM_noNxm}0shQC1JectMFHOM57ce;MAVbPz-Dc5Voeo*rq z8}WGzMpN0fUcb=ouNlQ<-=*r``W5h3M_^^B{1xCSu)pfLcgqUEe$Ner>K^FZ=d)J+5J4lbW^0bFi}I)<{4`*u-NI|1KI}nSLu6} zzlH>gtM+i#?TM{^qIK00joPiLFKXPmf1M4yGt3W`b#MC%~<~&nkt6M@2>TO;1t_xo-@a|KW*i zfKKTvx_R&~B0ctax#<9cuhY3+!HpsV)*`sVr#zy^*WZ5uprGtm7kdq-$KZ}jK^SBN zoC(dk@)_y~$x3$fM~`R0!7ME9rhZ2#5vSqqLO1l#=N-F50}*J|&S_}?8mVHKbtLk5 zsD(VyewJAD;c3Inq<`GS^Gpo{B5SLA9>nLG8GF3*uvf_GN18n$TjF%Ic!1I|HJP7d zdgV%-+YpoJxFYJi#IT3JU&aiUA&y2bwR!7hSUzlcU; zf<6T`V`FQZ^^yGUK&PsR? zCcX5CBz1MM#8M5I$vdTKP)aC$g+T^O(*CFGvh7;`P#2$=|q>*6&aN^0V*c zX&yNUB;a;-+a`o-b^7we(ii0J`fKZp2aLl6WRIsguHL!uhySQro#zcyQ<73yBwr*r zE)hro2+-)lx=h4Q_>rEuc?RSk+a%|4{k>{>VycyH)8CJeju3c!H;V{8|9(}eYy)^d zok7xFpWh|)l_5aVF{K>+dtIDh>RMz=2eY$fGBgLw$q8UH^*WzC=50xW>!I_(tbbDZ zY{vUr3-f_Ss+hQd`QO)@oY8-9) zX=JQ*NF+~e*hNPh)-nrl0teS)Xrh2cQN7?A?BQAIHK1i>@^YUB5Vp6pYyG>XjP5cZ zS!VylKf5sNz&BL0O>^#dEo{wC+-$pispdmRKxh}sU*Bew5%~T}eXd`BSrAtX8=z*v zFMIp6QR0MM@$jSlk)%ei!gg-&&TpYDPTaXA{G0!OZV3B#9PqTGdIn*8O@qb9clQ80 z;AO14*)d&`MzDVV&IC5#S|652{v{>VpaK0lDCV{|U|=Vu>Cb2l^-(BG zyC)+a;Y4cRjLnkZo_UHKjX+{}g)gsp%!Ox$tJq-6akn-SfLGJKnL~Sx+XdOwqlX zDXxyC;*oi@Vv^MQWDuKQ4lXWDX_YhmxkQS;^M~T9xIYzgS#0e7SI5rBz0#!0*-8`2 zxr?a!^p}SB&Dck~*-VfALxcED++)0yCv7)8&!ge}b#ovKY0;+f=l}oUzwNzD>j^M^InGX^+pxkvXTaP&PLiNJWPBMFPwkp zM;Gl!`V02!!v#bl;Z+OczxNBg5-+?8STps{KyL%KNJNW!AD2-fo}4Qfe(^S3Rp5F? z1{5y;TRb+?X+SH;5M+|j_aENHE(yC37P?%(UMXwG{b$+2n^&3}+Pb>3UoA0JZ5oTEYlHbh6w+a!b{{E zVs+Hhc2PX%qHPOE5s{8M52l$C>lkJCbGTwlDs?+uNuNhtvkE5sTa&2-eVv_468Sw7 z5|Kucu_=52O6}mlP}S8BF%i(sQzpZjNfE$FeE8c*th_>IDJzLxZSQIshwY!Ck2gB`A-|Lfsv(lTm~hdT zDSfqjqLgbK{3lk9w0~*=?0{Id#egM5FY-J8YB+JabzI5t!xrkQy>OzQ@kO8gQO(Z> z2!}I@y-6v;GhcxM#7F)6Y=kqd7_I%z9l06PkGb6ES{A| zGPrWeI8VOT?T<5=h%I7f=1KfctpO^;h|q%}gST$H_&iqB&9OlO)}8j}Y`kP5@uZ)e9Lcl}_bP zKWV+qfj@EKMdD|XsW2JwuGnrq^Vjk zQP=wDbyS8)ufMxhCgk=_$WhmLgUXB{)!#n{1pop66P!G!TxpF%*uNooKqkw0@fCP=b4yo z4y{Y!9A%OT|5pd7+yh(%RncP`IQ;p% zf5X+EA@<21i6_+WJ<(03u!uTeuT(-LT z;KFJ;Gxa@>R@`Sy8IA`*h)9DXBAqjIr=)b3ibzTbNOyM(U4nFX z4jsbK$k6be!F#>G|N53|v0O`e-}jsyXYc1cdvA^jQ~5Eav+V5bSV#2wM44UJ^O2mu zQnwNl8D-@a%)Bi-Fuu{4&-LhcQkbGfg;UMma5neB%tvB`Ig)$aJcieIsViEMQ@7cl z5;fPZG3|uPg}{dmb>2}P20C5p@-LtALgm-FrScz#jT?Hwl)7Sh%49~;!d$P8BTPnS zoKnV_+2W_&&t2QR$=&WB@rHl`}>BQZZE-&3%GEoBVZnH65)^1-%R<%06vrV?JUdknUMCr^~e}v8%iUI7Zw6l zg?;5e`t>`vbR25_@|bHmW?->&!W^XrgFWxYo5al6Ad}|Zetj*rqaW=02n z^nO$bu9Ab>4v^HO~7{;BW3n{KspA1J7XwSs5$Y zzVUZqcTcn(gJq@15M}RRcO~STx+tSdQ+k9G)!_VV1&vGDaym;z6%1z9Wi0_QGc+b!hNLWfKn{i)&s5Eqp7ZKojfT*Ppe49-Bw5L`9apsD5)|CNsl!7w7O z>GN9EGV|xp47*Pn1z3Vtn)Aw_RZ#Zt2NKyz{T1jy+Zw9%xko&AsRYpO0}^kS$9_XC z8?~X~x;Iqh5OyC*$}%`Gk?Kuq)!MLS_%mM7q>mL$qSHX9oC=(6e;xBCbNWTem#S%o zl8WIh{MGeh+SGcZV%bg}+r`l@^7ao&GV={ z0574R)GSgLVa*r!@E2F_6=BU`MnJ|vXYQ$V_$4JJ$t2b&t68;CtM7esJ0DWimxhQ3(5|xK?INckumwkD7Qg?p3xqI~%sgg;HOq0;8euVn) zVP4h7YW49()&3!HM=_ob5t`@sti;^#Gyv4&<$ef{&pg~jk-7Bh;C3`_m zdvEToGQ+n=+%`rQ*)|S9XlYq9Za(g@Zl^~>SwZf7VBnBa?f#VWe(jtuL*>fKsj=CB zmgd>IdG&U4;AYLfMr7q&q#lHFRv-SJB(7A^*IY-_{hz{EZ zFA6aUg4Xf|FiQO+Ct8%@;e`A7sk7>$KlBDo79w)Xk@XP^LS)vC=AlL7v+1 zWt-fHwtPNM)tes2ohx72E2RkG-&F@Jv=;lN_+rLaOBmwNB2^8M&!P&V2TkRERnn82 zRATL$7~8dKTmc!U$NZw_Z6Tbm-(Fq!ZKEY9J`u!H^`0+=t92B|JqcBo?MfMsvtk^%t%WS|vrtJ&j+@uix5a;myrUtjvH37V>SADSO`0 zXwk@peTK>`txV!ufD?-OOutPc-u>)QJ%@qX6ug6V%GCX5R6;!pd)}32+|kMkcir)ToW_zNbnnqr#$^;(EF(C4_MGVNR?qLLcH2 z+|qu$zYd!%X_F=lwWU*9_Z$d8(gg2;!|#X(F*UV^;?LP=+Yr;^`KN4xq7`eei|k{J z@D64&^`C9N{r>XiirH24t%ES%P(GCLJ5+9&#?Ib|W?AXWfaKgP_bs8FIQmTyybMDf~%_PhG8Po6FI%A(ZWb+vDeAw#Hv@kJb2HNo%G0@%&dX z<4}Wg|BhQeTr!p8O~B4bN{_jNK()aeL8h$g`00nz`d;xhk!_bn{(v1|p7B(K>Z*Mg z*}s7%CYPaz##Uf0RMOYLae7!SS>mF)FT=s{Q;Z>3!9?0aweHB+tdC>768_%eiqj@= zuM9R2JyaHsNaQ0C>G=Go-7iZ{KP_iM8eB+ztA^3YekInB#;KN7W&IQ}kzLvQ&Sd_o zQFQ>6 zGiwPw6(;Xj@gj)8w;8V_=SvkQqbPa5A@p*#J8phzWUTPmD*MYWY%0M!Q-d*FR8c0KSY*U-=!AzXpo?^N!4H_{d$V zZezxFEYG8RH^epFPxmr%8KzwKk-ZN0XXX1mCd7X~u_qxRr(#p~^6o0$h^LG=1(9lE z)D^_Z+wyYHD9CVp+U21rRgi_TJOO8%gH3Jt9&{GG&MKL zEhfy$BoADu2);=bx9aMz)&$sUtNW%NJ7}LvsBh}^EQ8QbA@+54>cf4i#2)X{Rs@{! zp%IS(^&854&*sw1g5j5|{noh>hF+(!f~IW1ipI<_6#Hr z+|`|F&nYbXT7#)ny_dZ-%}j4XQij2mOx9NQRyL$nHQN`&=`=$_K8O=K78Vv#0XLW8 zxT&)FiO{v|yh>w9PU#gF#48(E!=MU&cp6!HYFAX$ro)6Xv3ew(ZrAUO~#aOE@5 z$VXuhBq@laIX{-Wct`=;;Tn~aQ|f2ZS6o=WKh=KKMq*KY2eB6hWmZ-7%M!Z9^(ikU z#M5ldw#F*Xe^QIdIgj*W@GylmcGO;nd)$4?zvDfDhgbl&-4aFxC}vE=?gM47ehEwn zKOkCm**yw*^-5X0hoXjKs%h1F>24fleol@J$E5?1&yXW#22|v4knxtkTzsS3I!^PA zoM8>-^(K^oB8Tk_QXBRHVUpgw^PplfFdZ*6rDvorUgGlW+3^I?Z)BMJnvM0qsmRgr z^gEUY>Me3W6h_oCNrnFh3Npg5Z&VJj-aGtF`zCOHr=*WKdqM20<&0qJ;JMf@{{oSxVC9$Q?X1d9b0p(+}0;o$6S_U@UxMv)o!zzB~rc~vP# zYfYtDR8(i_ICJDsa^3`JM#gPRRrc#}Io{%h{l;WNcPx*y*VM4&q#sM|s1&l9sdO)C z!z0p@aH}>b?3s!V_Qp=oW6WwiY49AnZbX;!MgQ;Zh2X90riD%S!jP4=*^a=US2L$7 z%+3tzE07?njb!K(9sTNn+V@93GjwuSRT^Q3zt8t%SoN-hYPg>ja_;?f${4f* z*_RnS`>OK?*PVs7E|$xIVSQf?hw+l>mr>J_G@UuG*jywapOz%N^wRR}N1yg7R+SE? z4tQL?VjSwj?|Q^u;mqbf#a_Sff{25j^thi5bPG&x%#Rft5|i7FYTx*^?tT5B~k-?lPwPgYX9#B62fy>nngX%1M{ z%#Tsy8;rnnDSX+{uix;04L+POh&F?cDCq{YgQ!2_qx+b=G~ZC3*+i)=Ci~VocS-xm zln_5VwVWYEKUzU;|7S^0Xl7h1s7pbW$=?N16eM}nq3~=4LU^d-8R85yYUG=K>Y#45 zLs`~-YGN{vv(sb6{EDThn@2N>FF1#sD65aaAx!|&0oiNv7RaXQVCP3Sp}Y-W#-%f; zy=;{saU-KB3@lLshV=|O>7qDF)#kN7H4iwPpYBKO2MmqA=~6QH8Z50KO}-Tu)0#J+ zo#r`YveXl=NtBl(_Jn+_T#7r-9=blPVv?J(>8+sYeTV8zC;2l22@~NAuaGcT)R}2E z^XxsuyMfQ~sLZR<^JgbatX7%Au!n7<&oSZym979C#Ub`#{0nsh35#AvU2Q*FaR&g1 zkk`g=kxt7adw*s6LQF{B$9Z2L{ks-sp!9egw z0+HluGO$P%^Y_Zi$pBE-@vB@`o1U`LycG6?7wV$VRbR~WY)!K$2F~{OSJ#-&XFd|No96gg)SV8iz<;jE836yInUojlTX!($ z2Y)D@9DwK?)%ti-Cwy4dUS>Jge##UQ7h!aJ`whcgo0AX`GRKtpW)cfEx0CG_7E|Ub zs2;R*%E<)9mMxJhDMm6e4NPKq33y@)_V_~dGo#*wr~zpK*SyLeXaz8Qo@SLEf$6I- zbh^#=n`FG4gGEbiwDG1W0r)}*f`V)NC!zD3Zs~nzlgqtez^w{m<6TJzwT4E&WsUs1 zk!;2MO5l|o&_8_$#q(;9R=c;msnt6av`c>HyPu*OXyY7`YG5VGigFJ-t7ue+tybEv z97{%e)Wvvjq{ub52VOX36}kHJ#o>^{nbH5OoQpz6UcOO2p6mUib)vz5&bj>BqpUrU z1y2kgohwqmam%>%g$Jz2Tmsa&K`2QrR#TA&3-%so4&} z)TR0M2J?o7ZaSIo?(720>>hv-8yRjitTUN5{{-Tw*6NPoY64ST9jYQUoHAqiKl=Il zMm0_OPZqp;hFwh_a#Yc7Z}p5~jOZc?<;0S8m%S;4ZoU4$ezWA02i{ZSYzl;+SDkte z&Zo*MALaW7(v~K%cnJ6}PVI(IcTs^^g}m8S&~)aEw2TirZD6-XVEL|tHS*VBF(pSJ zO7_hOLjOwsx?vu6V;B;uaz$cTYnD~jp53mxb&*>mwX!?X;EHN?NK!uD`zjSbY`&YC zDfj47;e|!BpGorc)HO0MxqpVLF1_BMirmk_R?fnUYl}83Rc-?AYuYwDw{+FI@#YJg z6AXNj4gkL2)8}VpYn{(}){;J~omo_GzfnacF<6~vaulP|cH<03Bu4V50VDzkF=X|4 z-R*yS0SNFMRxQW7*V=$^1eM#cw_z2Twek7h%Cn$tF@Jd}F8x|45uO6;u!wpXqL5V) zV>@??RSUE{6_rTdK@R!*P-bDctHbX~q5g9kn{Rns(G#u!ZKa0)*aE3a1ht8xlF~2W zlWOQ!Kb!Rdn;cVSN`x0#lX6+a;Oyc@SAfQ9ni02~v=NAnXPHfqVtIW*a@H-`n^NP<;s1*)#6=1DwMzR=?Z> zg4M8_bKalq^k^@;)HnXy`u9ir5`xohUbFX1oFor>y8L5tul-?@)YHzKQxCm}7X3F; z1bFgY!rp-ic?FsNiD9_FnJfFH?ZKa=5~nKA>IczBHAO8F*Zs)_YL&vSrtU6vF(~+X ziIFH$$s@;pea8V%) zF*S$CD~QCuoB8(G>+f3*exE>*&7EFv*3+I;${* zfGp4ZuE|Ow{G~O!oA~4~olnda)X#gp`%5#U;?cFQk)C|1x!&VszhhkrYD44AkETZN zUMQod7H;ods{13JYX8M%6hZq~!*V~8s3~%s6V&aIhz3ge7IMz_TDg=<9xFZzR4{|# zhe97?M>6+Be-*5rWeMjA<%abKvDp1#3kZhdRI`-PYMWLYpnk zzW;2KPbs0j{3I`+^VFp40VW)9qynkql$x>Bg_p+MPP~owP}B-3r7F<9`0SGFgF{0V z15!M>(ajGsxt6}`-b-Bg8 zJqdr)N~?tn#s@_lxH{anQTh)O`V*B323`jsCEq#*{^ohtpE3u$K;>zMZ1Q-F%TAj$ zxB!VN`Y~Yb&uAjmGJRL*%HT7aH^XXNY!|h0x1~W84dU!XeS*JXTu(EjbgDoA07JxHndzB}dy z?FYfnI2ai0s||tKJ8#(R^5aHtf^*JB<(c; zJgS5KUW|en5=1hOef9@wuQ(gd;ymfV9mEH`9b3^x!bul4FxCF@CQsxsCl3on#l>l0 zL~e?6qQFslU&O`yU7#zp%-9v?i&@p3*4!o(WVqKl9HfkMzPi$V=& z=fW<6jKP=6F}0ul`*crU!Y$(OfRq@g?jrRf9YYllyU%6&jw_P)u8jDN{21>-vz=cJ z)>)W@C=lO4)11pKE{5!X!!+;@q?g@aus1a|`CAaS_zL4tRPx17<|moUShR2g(^&+JU=NbP=CiE~LP&2)xA-UJ+)B`~SR)1dGURso8dcaYqU4 z25;>_45pt-P$RhBS=ee?9Q;=T7dqzo8s`MrO9W_mSx{NdVIi>E%C9w6WR;xg)r#G zZSi_dt-B_VmcT$mIIe@P&fxH{+(9HH!e4?CAOI>l4vW0cpFg){U|xLmXD`O+fgtn^ z(QOwAHRc%}fU-7TY6kdgLCm<0FDbY#Z*B@zhKk(^MBKsk^*H~5)m(bL9rl>~&ypcq zZlY!zfusLUarkG`zzJ3ze*Xq;%+N@1=W^jCKs3g1+kg-BB*hS7y;+#k|DFhXZvn#= ztn-2!-pYYd+jZYy&HuB}fEuik4OP)ex%Rz= zGU)QI-<>$YFl z%3_g+Vt3;xM}ZBQ^j=R2JfIl< zK6ie02*q7IAPSgJ#dCe#(!V^+KXProdD%T13Ru1YChf`#sU9PDKeDp!>>OG5V-!aU zLskoo2n+qooo0j1K)!#Ah}ivY_hyCbK2|2-R@qMwR}5AZ(PFdF*Sp4ungw3?pdfl6 z7&huI%YPK;LjNz;2nhjfb8kpa~W%lWd_;e zZx(t=9Qy1+Y)Sy)zZ#GISLZwpt{=o7nGOez-UnPTnyp?Tg52o(?wUFR(93PUG|Oor zSOhMq<(>mtlP78$J+dWBqlP` zCqj-~DC%JLQw^Noyq8xPexCrq1`k)-=$-$lAO!-;vhGCAnGXHecqxL+u+*#U_!Oko zEWZ1?OoUDVeDYDSuWKOzh(@Kf4z^~=P5f#f790yfKIIYVn&gu_gvqfxe3*a==1{(# z&;Zn+Lw8p64^!~pLp`7mv#%16oqX7sy=8f87TK~5r2;Y6cJZK`t~ckg_MJa&3xoxC zsh&oB@c(O)KM0jdjQVz=3l7RSG>y88-RFbL4PN-kr9WkUER#WrPTxv<)e@Nq05{-S zg0Gf%Y)gn7NVb)>c;Lu>UZbY2Uei#M|}jF0avHAL~dQ#A8`$2fCf2?-dR7hLr1{QG^l z*so?j5lVf}_n%Du5G11T%X0VSwMPoy^qTQXkiz55P?k9UomMJZ+6umG@J3(k&o@cX ziFSLnN$Gemrw2kuX%FnzfxiH^${N(4p}cm3PMXFfCSlJV0gkhORHh5^6gFBJMMcA~ z;OvNR-(2!GNBELRxeUbp&rg;GvVVDgkVc$7R(SI!Socqt87&MAc`fy^?q8#MiVZBp zTwZD^yLY~`?rbmLB5S5tm6FSO#`9VZ!P#;NGEAi|tGfE6@^z{1BOoqNXD6g??mXi9 zZhl}g7tgoKRd>k2E5345DC~aabdN0c5{XKkiECaS*LukS6qW7>dOX~~&v@RfnC0Q1 z?7g1hAiBSGc*c0ZKn3)^ru}X9mP}KN5wZhi6e|j{v)Z1zNB`E6CtVT!1#7XDdXcds4k3IOPYw?UKTg$qm)NH~n)6uM97j|%1o96v_dZbRnbxc13CfqsC1R^yh zjHPM*3d%pe#RFQEqvLsjrC)j8lkGpaZ*k3)wl2YWvN8cL?A&PT6G->j?tLcsdVxcD z&VXyzJ0~W9qx~oQ1a=GS77kJcDnohPpv=Y+%FWu zcxV+LS+(A<`~0*vZU1^*nIf`7QTD+KvGzr#O+kX;4S&vl{r46efC7_A4PSB2axy#? zHt|rZ0^itOJ!z`NZsuwxrq{mnlHF{?$9j3Ex8{NJrH5b|RXqH@rI*qdK?9syJZlBg zwm-{GM274O0+eB^(IY)C%tGB6HL4E;?(91JsY(%?!C1lxHRZ?uu>@d#9Ex1qD!NZX z7$6ywx%>y|G)2rPVB1UBgAeB=VG=Ikt#=Spn?x=&lug{YdTJDz?Qi)M-Y;IKjgg7n zRJm z*{uiS-iMeNS4Gz)CAltI>sB*{Wko<+&Yb-Rfw)Zv&ddK(TXoy?vPnQ9R+l?+1tvE2|n)Kllftwi<3a)Qo8kgZVUG(p^S+?!SC1; z?P}W+m$mb={W_8P+{S-q#eKlk<^<*6a{QaHW6`}|D{SeOEuAZEmho(>d*^kh-cKa_ z_Ltk>If^!QZVy;_e#3X&ooBs>$$U!v9YJ(U3YgC09rKHC_IDsXyt@)+)wch@3W7lx z5$+5$@wfQF`!40Yp79r!RN=TRqysun|DxVLGCewXT z4EhXdVHpQ(Cl&p+eO>AXW4M9GPbK5vCn$rHgAn$wR?jPZx~jvd%kk&vQpRwP;c7H& ze``jtr4uSEk7yQ+g9Qwsua%SF4@(-hlAi{E>#fm*x--+&(?uJBd0-zP;=4i4e{EJ4hZC>tLfz&xg_ps$^Bh^HdEY~$j)cQ> z)P48fN5Vf5IKAhG7;X}&KdrkiK71a-S|7E2NItKIXg^5DlQ|3CzVTiV(~`^Iuxz@h zG@-maYf_Tlw5FsqRa$>HYqrd_?hh-WqXN$ECB|12^H?ncHS#SS!d{`_kIHA#v0zksyoB6Z5m7H= z;kih=)`PiVT|-Df^n?Q$o^M2Dst}%<=T6n{^yU$ZhKFFjZz_&?kTuS(InPW$Z%-i2 znwR8-dh94AaDMHHmH$(Liir!~dU`;gnhz4jivY9f!=u89TcfPQ4zCOe78b|H8!x%l z?C1{LNnyxRtyBzV@QqJAnj0$$VSm2yf`CEJmf9|b{n3*%`XVAvJ?t!X{cJYH;XJo! zNd>L=&Lw(Pd#P=5^QC@%bNKAR51*x%ko$WrETioD54JboR^t4=s z7Bnm-(OY3)_ov~@AMH@a6~*Q7TFYZk3nZz45Th`t<`3sPUBmOm7ylOg{-sXG5`A#; z$RNxMF0NS{!Hu{Y{Dtq4)GLYuEF63kR4rZqaPr-dc3wfkqOWSs&DI&LA?+tY&@AFA zg7brXV?2DM)JFnoEx!fDNJaieSN@gfs5xKO;*ezklwkJxddXxN^yFV72L&YelvLL2 zWM|`EWfsf?(QMKQlXSDhWHmHnlZV68gbtbCT6pGK={AW1tn=5y;^^PF88Q|=D$fu+ zTIR)5qWvTd{2)WdNl}C&#@F$CTW^P-kZN%B;6nUPLk7WE!uA1ee2HK zkAk+M6O~)$$F?d+rtBH~p6eT^q;|)|P6!cE<>lCsx8iS9i6gxBRmtN1sdfXh19vZP z{A~Z4qz5izZg#YWjM75873B&8Y6YJ}JF>dcgX@O;SBB#q4Vy?J1=vAa0Xpu!XGy;us{Rkc zEog`TWOPZON;+inZ;=Jaw`j4BfAN0p2(?~hztgIJd1Fznwi5KU_#=@g)WEmNspb6F z_~Ade<=?+W-UXri|MKa%(2rxLv^BKc>%tz?)U2yrIDQvYru zFfOAkL1(ja5&0JeJRrICzqD_q#Vq`uZRhkm`J@UeZ-re0uJUYeN|n64Ow+j~j3x7Z zY^9A=dYo$#svjY)^0lq|&%>UAQ}lrJzbKs|=orh)`kaWXQ4c$uk=|aOM$$6-mNIo@ zSBQrFMoh&<;xUuAjRT~KF{$uSc(jLxZ$y$_*TkiG(3+4wS&-2PT zU4?lGrgjUW1?+39wz77?{67*y2yOZ!=WyqRifaQJj(7Qc7~PQyhnqPLR2@AZ*DxE+ zLM{2*Z)P{lwF)z4M;$Emk)DueIYn1veqNIyv+4ND+%t z_>JE^61z6H@%XQ;kiL+6qVvt*mYsm@j>3LBets%MXUSiABSxO(ebq)qXw`N%YEodc zdbs9rbzM8cx zN(Z1JrSUxj_|r!6zJO&!ygP#Ag`Laz*JYl1>ZsaRvMCON+SKgQbj&gbp)ifn>M=YL zi#hEkmnI}0s*In_Ea5>sLoIAlNq|o%Kb@YEJRTA6eJ=A;OvLzy2&PFD7`Y|t+q1gN zG;G4DksnXInVNa{v@sh3*S5@@vu`@~jUd7&l4clZc+2!8$mdlLvpP=|%Dk#Qw-CLW zSjKiSd2g!Mvy+8zwQ@a9>XnMZCnO4~)8&Eu6B%~G<& z*PXI;$J!qWZ!9EBS&NWzHXv;~_!9LI2*k!_ddza|@zS;ZvO9*K!`;z+bQV<#Qoq-b z$v-xRmEFx=*rO+}?neGoKAD9>d9Eb3cLv}tu#@#guL}A(Z;Q_{>SEH}+niwTr>ZvX z=^?FVx&4$wkL&TCJ@m;~SH+J9$ps(w1AR>(_z-mnx_JEjQV{HYb;hr>Q3+PKcH5Ld zmVtt`~E{v9R_#{ib=}9wLLX>J%x9w?6nM2T19SQj(4L zwzjr;1qHp2tbWRropw)rw$t)TlIH#W?RYJ@M;!(W8&+Oru2^z;q1Ah7|47`Tl9tSE zyz4v1!0FC>ex7(4^F|g4i}*?X^*m(-g@sDnvI#>+73mUYx=3@v+BbHpv%(Z0WQoQ}uiD^S&;D7l+fzPzdNiQ) zLNDM)B3w%3kxkC=$d$4b1W#tKC)L^U-g4naLGl{c6S-&DYgLnO*G7vU9BavA;;3be zA|Hx(UI)m@C2TaDY>v-LDRWVt2=4G)#nM|R<^u+C#g?NWlzucye55LJ++x}0Jp53Gr*x2z3j|(yy};4Uxu`o&qzurO6srK z>hzn^Xm&ZwMUCfXSIwWB-`rCaI4he*)|sE!XVhDl)!6qHF>izh$5#xT7v}U1PCV@* z9L|Cw=ik4M`GC}%?6^ZGTDzzrb(b-6D3`5&V>+?%))vQvQSr^1chQ_HH;!d1c-O0s zZ*63=)Pk}G#`x&5%uM;Timdjm-ky+7GOp#oVGY9)k?vRsEoLuLlt7F@^sxx|VnK>P z515|I{lZDNTdEUl8hFlwt2dh7`7RTINf;dj{k~Y~sP!jae1)%H0d^toDB-yMq)pi| zI@?0sQnV0-OpoAMo-0M-gN3^`d(Mh1cp$b>UT&Y+YIAoawdGOrK@^T zrAY}16obz@HA{=USFD-GW}=syIkRs?S%lqjhaE6HZOEReo`AGK*On`&8xis9i)b3@ z{wnUCRy_Z$%!&PI+baEO=NI0FJ5lBJxyGKBD+ppIrbl9`5tSba1X?Fv>OKC|o_z}s zIWUFC#J)Y<65sK=ermF#t8N(lc;R2+-DhlDIKf}_Q*A@Dz+pQ?29+7_b|B1@ne(C{ zT)a%pxr%2oCsc*`N`>`8X@xa321A|+;=jzmOLcL`>b;q{@@T2_%`HS+xMmJ#fm_XD z>YV`7?;?KE0~IRzx5Mr`+VpV^4dUsUltD6KMm84cg;GS;mvD}L36ohZff3zQR+aZL z7Ks4@mLD7N)^k5JPG<>O?6TFQ6>|_W=*2}xV0Cf6LwQeChKN2Ne)s@8JD~)o`$hjkrB7&cg^@HESq7R&NSADfXt{g z;>gGt()7BC+*cxnLP6(7zmJ`)OS(}f9U%=Vrmch#1&A1~+%>?Qz)VmrFO6#q8nQ~l zpKTR&q_aZ8@v&!#PZuk+@jMv`TZhdVR6Mves6X1~vPl)fVqBEzw=YfaOtaV2&1p2X z`9>2F&_jt1qjE@u#ZMs$I-lt79$QRJI_3$9pgfwZguV|N9*O5$;Jj8eB_~_##~;-A)euW zL24Uedinmeui{rSYx^z@tz1D3rwy3>e&e|@x;Codqj*U`n67$W!RA0C(eRVS>rIZn z@1eav4}I$xmgvLja;zF^?bo$Gv~Et_Wbb(+Ud9@|cI+C|&=2#iB03|FK~oITdBGY) z#6^tL_5M${+lIjUFhQ6K7@x0VF>&X0h`vZ%t&upXCT~Mc6gqZNv z>v`6NQk`uU^DTbJ6$rsT7(hAc;ym%D>y}kZEEm$oc~T5H3~#{Un8k{5;W?*?w>ZAF z9&5`$$iincoorrM`LXJ>Hd;tmXSC9y4sBj7*N-6#HPOq1l~FY(-ydqdz(MTmtA|h4oMVE-wcqxV-bi2v{d$! zyuD;~mV=CCVZ+(I;*!KUHZRz{PYkQ<53(*5V4W+qwU4Q^sQijPN2KM_chKNn9C(Ke zk`23*nz@uo^oLJ{c+T^Q$Qg2q7Kpv5j~^{sHW}g-tEPZf#+R@>F}>MuM|fTofZfEV zF%IN|=}F@R!-?orx0ygM5{d_&rfa2?k-HtemFQJxhe=N4mgYx27^^y+jOKm;rLy76eToCKF;6AyaXTR{P6;RiN(e+-X^c=fPD&u z>`WZ3h@p=?5O=SaITWl&P@Bj)`OkTMxiY@V!2_*@k>-~xzrh5b${ zT8_m_g8|f_58;WVJo_SnnloGG$A-9sBHCOhXlHXgLQFaGc3 z9a~wv9kef6JNH@6?H_n}Xt__JrxL~Sg-o&ikzXHkbD@Ovjm6{jILO>b`@Abv^XnSb zZH62*gq!wt-QJtz6@O-6WVAsbY|ZE5LnrIQBKC|qma;978D9lJt0q^SUpC$ua>F9jAi^SSYENY;GNR|94KD7xuGa<;s zmUA=Z2d7J_7E2m#XE6J|v53mRSRO7Cg5~-~w0#f%c8vX(NA?pxVt31Sk;4-fE5phb zW@RW19_J3Z=azlbEVhuv#f><3HL@_O?mEko&XNW#KS;hHjbBO_~cAavS49=GcO+ zWBf^0y z;~g(f$5j`y%W2ouyXjj{_NzNtAfNvB`A)u`duq9r2qtA&_Q8$xruJ3Lmmf-1nKU%S z=X^*hbH55e|6Uo2=*nE9@5l9>gM8win-b)F zu9#|td^dNWy7P*Fkgj%T6j#R{c|pj;JA3DPJ7wdM+Su&eY}@5HJ5M~xiHcnoftIju zqb*dnZOt|a^+R6Lk*c8gEm?*eEwQ(0{6=$1&$^^ns3B7 z=AGBOO_gTG6y&X7g-n4WYKs)3Hn)sHI{q)E1#VOKYmUyz9Mw0?T=0*hTcDgLx!k`w zSO!eEO`a7JOiWf2&7=TW@;yd;8zC2^928HqPOHW>DYo4%Pb+oA!$&o_(&^$9I%SoL zu-vo1l)uqNndN>t(VupscZK!AEk89tdBgeS5T?h--)Bxu#lFvpAim*Z6R3K0tiGe0 z{C3ftm6rguJl*ni;5+OcF$&enT-*cU$lQ~$&^Yw9T1W$n9D6vt4{u&4rzBuWI#9_V ztRG!mBY5X*j8tP$OHwW6fS#AibL7DuN{OMe?|KfoMk?1~f$!q5uhBI5#Je+Q(5gFy z29MNo?Wz5!?6qtxT`QxU>)F+~v?!rz$`aGmsl?$2v--0mszdoTh~ZzuH74y0PuK1b zLl>eg?p#bbZ1m<9)4AlpF&M@clT$9b9Yi^9A7QX8jCU^8uW2Yw{YFF0O6umB~v!7sw$BbGj$oM|q(>&Slcoku~)>Bu(su^e* z3&v@mgF1LsVH@C6PN%XS`mFJ^9bDNM^ zU!R-oW~Xu@XZ~(VyLob>Zp`vZMe&{@bT{guoKmpYG})cZeleN*p+t4iTXLV=-Nv27 ziMHkBceRhzRJU`%fL;HcSsTwZRFo~h%#k)he0hk_eCA=X zAg0JY;8~jRv91MauG-~es*B7D0R5Mp=Z2#e&LzQET(~gsDKluFw6f<_zPdOx7>XCsuW?L}!H+ z#}dIux8oY1&g;wDm%I>ZRnm9dLJHpuL_?Vssxys}3mRNa)C~kWSl}n|zzwlAKuO7D zmGqm4yda&`2?g_c`Cr1P`Z}|T2%v%J+yaFGwEC82(-|H^LP0-D!xt)?GdSondbY;% z>y4Ld1YhETaABHGUPE_RWj4j*-hQmfj&1j}(Y;v>^A&0O7HI{oBsgh)!DfYCsH>uY z3OPC`!u7?(w4N%x8*D-Tp^|%%?(Rwomm)y7N(C(oibXbV3(lO6@epv*`<4)lY+#AM zxg&cUVtvc7+mN46y2r*r=aR6nbdyVZx|_}|!=)Sdxdt-7Tk~80WHi8m;Jg$K374Ec zf0PzArEOpb{@6P$Ii8-bZ>;)CTd+RcvxB`%V8mt2gflaYO_bA$anwJ97rpRY>##}6 z&G8^ikqx;S^W2aa`r+oiM-=L$zLT)!C3e=anR{0+<9>T^IXw66VSUeA$MltetLo!* z*-K`H?$PT^-TA4OAxQjaE-&e?9i+r2;sW9v1s1o0^b9_poIX({YYf+S65ol(&tI7| zn4=!|#w(~C-GTX_KVXH24u@r>vdVE^`)uCA7~ynKVzH|CEVE2+aa^!>|L#@KVG2P_ zbFs+U$#2&bQ$^}EXj)6A(@jq2D-XZvvD{K)J{8?Dh>u1`P7cFcopQ38yMky+ZK>$V zv2J^K(0ULyhR)z##uZH1!Zr;;efXraHa+Wwuf%eHw2+g-tAgfnUfrZTZTOM_d2i?T z6=d0EHj+WSXf8ohpRYA3*BV}MdSAk(xa66G`<-fZ3f}oUpedec^Y<;Ke~LYJ^^8<}8nvW_2J zm*K~zkdeYR-C7Fxbk%-fZ<;##sQTl#woBh@f=zA(+NVf#x&=+B=KEMIYJjb+mduS_|7Y7&BsCZJk|Z3g~2n8*5l7S!(> z{N#&y3!N)gPlnGj^)K>D4%&vI5T@c_pWMv1rKByZA+BRxuLmF&IP|9ICVSd<)i#LV zh~MrZLpUG*`Lut&^7eBq#f*j}ho|24SFeX@z8QJ(7S+D^Yuxu65tS|Y$fa{Uhi0XY+H3j z{$1LFq2l1mq#JDw6Cagis75${xaqyq$bKfd(as0jx4d;zG@l&v?N#X2_2o<|d_3<# zO_hcWF_;Eaa)J>TMJ4FtZc>s*DK_CIe2hJYsUK5|pLDdKCyPoopD0f-hdsr}S6H~7 zH~0R9h)b9};}uRELtNkSN*1wnteDoCj}x!g3m9vQbJ_x1f0`u*BI*@gCGTz@?;kI1+q z$AuAt<6|w0@zIA_(r-&QR^w9?45a*XQ>atvGbs^IC#QaLy)Mcej1XOJ$+yMroZ={Y z;dLQAj1F1R-ack@-92tmKh838$r;y5FH>w5n8||foEI0B);1I0xk*=BO4<=Fs4J;2 zqKHU1>~VD!S>R+h`p__-83;A1VSwsboD!GZVEf~;`kw^-KdP=WAgZlvDTt4}VIXjmo#B{cb;lEam+1zB1#Rfbc4Y}DzrT-SD#Fbu2wUM(em^ZuMgwUv_(!IV z<&s<4J;OYBX6KZ@CWgd)+!eqZT5?C#M9>naU_<1>NSiY)v#(s$m5|72Wd391A)XK? zxcOxCXUp-%TqA`;fE4&xN@uwIXgkl|Ox~jX4Bv|I`zrthgFYiHq0$ zN;TuVprz{t6fj`Op~k&JfcoKA@NE7BM6SywaQCrpnyi3>Xn?J_C~Rrj#~) z>*f`-A6li<(;lfzwfj{%qTw|UxRpQd8pqK<)4j1`YmQ$Ln5uD7BO=@5KN(x}DO;3YP)W9`;Okq>>s!N(TN;Qc z;;(s!=M~HLKkK+SNC`z64v`#Plq&|B^-)r2sN#*$wnE zYxc!Lj3+;y7KlBpp#Q?hsqh0l7$}fYNefD9H_@6v4@}5trGdN@WS)fLn<&b7h2PL( zgoEpmAehl>GI?@RGHbrhGBs~x%VJQA*_vB3oM__64jQ;Lv_bsZQ?E;6hzDi550q>S ztlI}2hiv1y1gd@=t<2ZVD7$Kx^bZx(3uTB%r=@EFcn3#yNodsooeB)!=CY>fE; zqAtipJEis*Jrq3F!3RTT90ugIKk}uikCTCBd{Me?Mtyqe5-cSk2Dh0MH%cbQx}+%W0QgCWm={~uM}kU&Ous8^uIO( zkdJ@N2k2ZDL>(+y?IX45C2Gkz`eN?Ln`B*WuK{IS@X2v zsYaoTCVw1@o%I)@2~Wa`bi8}i)Q_vFsN5%WZF#61fb$BW)H7~%K@P-S6eW>!!`kZYZh>qXf!t?cK)h7naGD<_>L%f0L%fWpL%(D?SeeRO2P zO{m?nsO~7xZ6!r2s1`bcd2fTiz-_QgI2RoA+^@x0v1#t?>W|zs-4isN-!^ow~%tXV^Q*W{L3`)aGU1ow`TyNIjkeqfJ{d z53{~&0*<1?HUfZw=By9r^#iLPSGdo5KOtYBlV=_vGHCDnRs(mPz&(S2Vy$$9sQtS} z;I?p@y_wR%+GmL#&ARTB?ZCi2)}7W<5&O&A)d#05rQd<#WX-eU70t0p%r}MUYV|ZE zX7MDwl(n}XKzfeVxV0~sF?MfrrWn7b}1$GJFUeeo9c`#+Nf8v9p8_wm_B$r2i6+%lBVx`l=-_V^{B)?noIBF zKrE9r&j$mNbCZwcro=y-hiUgFvy!S+70%XrpvslMmKiO#SI+c~84LjbBi3J$-Q76dpA#Ze$bQK0aGlUcyzqd^70BG zQdpNDu|*w+^USp*i#-gvXTi9qUZzPzy|bj@ z{$zH(0?vEQV&vUwdbWEeaPi z-^M(xeQvw`&c*3sFov@;r0QUPXH38fCH}=v*}X((%6Qfw%+H-`mkC0mR{U_tH2njOwZqMu&J$P| zw7~L%Pl{2xWKI^B&V|yH>P1pGK+?g?_?akHA78h%y=+C~<&rjJ9p}0B*c5x^KJWMP zoKJjNhVAo_#OrnKZ`yZYOt<(D1I_6ApB}5PG-@b`&yEy={hvEekU1Rvi1iz$jND2V zds0*HG+#-ffIjWCo=IZmXtb)sr}E*(9P4lpe`84Zcpv{V&1RMdkPF3hnK=g z2~E~!W_ku<5a{_&-5&ne!lr(B*Oz!4;7#l1ErmYeZ-owE-$bHyO zfsGx_Wex=a_geUD8q@#El85Bw*T{x}I&huK9}?8% zN1N4e6e|I(H3P7&o1R>ILFvwv)%i{DWh$C&U%Itcwen8y+NVkWIV1}-0(OL=#*><9 zh&4&Af+aBRAanY9Fo>VZF$%JjL!W53)|Q=DfI%85oYnqJUkCH)t@4^U$kE-YuePwE zaL%%#I;6QW#zD2)3)~Jn)SWZQt*u9Ve|s|XV+b>j&Xi4QB0sqE>5eT5JYwELnZV=z z8#CVJ6xvYARze1W83ME?9a)sJryZu0*i%n=q)7Qk5^Sepc1$d>br4O`8cf^q9SwZ* z90OQ$X(46?*j*&v@S(Wv1e-s&MS_eh7U@fY5sB5ijhDiX7Sr3FCn&=0b>q%+6nc^~d8uETH`k3+pN=a+khIC{K4}?7=>0vC6xrn3m!uE;HYy zhDUZv{UXX&42Y>}95&ZX22rg?OUx>ST#sx@O(xY*f~MFSul~5QOfoqdEr>muB_OfR zao84dlsnObdF)R<+v!lZ!DB6;T?9vVJxlUh9MhhrV=9l4sAgkh`dG2mH>YA5hm*cQ zfK%LXG2OS~F=`5y6TiOLRP!PtZS|Nuso`_~{obmUrP7I8e0eoW&-N1QDi36yf$ww1 zB6RKo8?*HaHG)+MP|C^}NLD;%gM7Df6Ij z2q%v1-h|AGB**a1d=XVg1m6Z#L)JS+0Le->&&$mwDR8?K3yEzKm9JTPq zZM}e6;??Cueu>EygGFd)RgaSTR^a?((I4#~7;OMIE?WP19tGYYVopU=4-Z>x#S*rk z-Y&N6A;_;E(B6?}VJvS|nC1yMT-&s+DIfpI5)bYTmOO<8u~lo#d$xOAkB(bS?02qt zi0^)Qb=Ydnyu)ND2OKeCsLj1_FtkuiB*ltO zO>?o=i^L4;kn7iitZrwbZ_S6k-CjngK2fa;AnKqchU5DZDge*sKj#VYyGL)L$vP-+ zHbMr~2F;4OMTE9@vntDRjUJEgUaC2u`+-P7M~x~U2NrQ;$5uwh5NoPsq%VT5?FGh5 zVC#Lesg%X14Q{nR#^)BkiELD9CV(G6+TPt-sWhf|3A7|Q4V>>lds#6_aW(WDBLo(b z{XAdAq5SM6ce1S*3CqDgfpQbvoTVwkkrtV;$f7^WPj7XSCQGBBkNOSTGp=&L)xyu~P&-$%^Xq3)^}JA?%*y^hzw_lA$8TpZtj1>@-!*VI%-dbopUeV7oY-c?F0t&cBXuPm zVpr(Q9yqp@Yw&>(vO-t_JthS;E@inx1@c19mEF-$CcqtNtkq3qb+-nU`*bALz*M%H zLfzVqlkfXlAd%{|3!EKt`lPjsqit#_34zmXp81-*!fKcUFx`xxz@oCS&}nyW=M)64 zG`I~8M6J9!=&cUnnVr}!{~j_P=^eXhsQqy65Anzxc&(%3WBSM z815%QLc(s-S8$}@HSuA+!ZSW87a?ZL6YN7mP)Td@h}o%v<`FFNxi=;me-%@i`!sXC zi)EZMTUN7dX@OUxLs$nU-_lU@FT=XXfKu8;X59&^>V5UoGtTcd5$bx~qvPYXE{7jy z{CB+;6a;r@PKS)JeA6m^Uwx&|s^R&5kpJ<`pz4m#*HTFO5D5a(jsU~C=b6(xmf zJo*I@T-`>W=)qheObYwi!UjwGgG<2z`EtawH&A(B2s){qYl-sKXn@=x(M6J^TDiFR56N= z*6y*7jLu?AJLkqKDixDFF8C__xtV88n}q$E!lF$r1(WN)-OO~lDAIEigC7NAT{QWo zWe7;uyrL8w&0wR}a>;>V>#ZSfZCj}F14F|TKw1icZ{jjmO@XmSSrfv9j)O}M+fxIU z)yI)LC0@FVz*M)!ufM3-zk`e5)1<&*E7YDf4fq)i;BP?!G$5njqwb$5zb{cLrNrP~ zYy6%E`+>uDjU8`)KZ$4IG+jGw(`vXqu+sO?n`*!hxHPBfO?%VAtt7Bj+2r-B%rtQ) z1X>X>H5IU5`sbxWESn`&0>k9nFdmwFu<;ZgSv+W=g_Gtf%=*(Kav_C5Q&@k@yM@R9MC~##PhWkXTHr1 zUb>XG5U3tS0^1}>SSgT#;}Ptqw3S#2!5iP`k^A9mfV`rR48(RnY0ZVto%6Y$=rKhO zTXT9rPOnb&04Ja~M@CpZUe?a|>$o8MXm$HdGq(t@ifelNo37*)$cgIi$v%chn6vzuPw!k&&XHMz#{SWw44q@6DrIr~b=;bn|NB$AAx%-HxZK0|vF zcwI!v6A*#$dCshMz;|xq{LV?;LNn(!*(_4%p%V+Ny2iIs&4rZT$WB0ysw3kpYJ_E- zEA0!7t=~CB1&aXj7!CHOkT@bbESl0%Kwxc|7oaJM&ZQ!9FTiL1L|ote(>vYge(#M*8!I0?eQ ziNG!XGeMYXkaznnkM)iq&+&Lpp><=!&QdQcYNRnUE)7$5LdCc|#Kbl->P5yH>GPa} zl6m5j1GUVGB&tlNUg7UWVVSbj8P90qa=h=O^LWt`Hqj<;xIOp2!md)EoOm_{-eQu_ zdliS+7J~C~;2mrg%J+He$tM(mq6cr2-_D(Z@hZ~NFVpL3<1By1mz5d&_`QO0173s7 zJeEsK8|11nrF8MSXC#)1?U#GYK6nArX(M`d<*PJ7?r>v#M+Ae?1cjSUnT)xmwF~v1 zr>g=-pK4@JK6kt`e?Hb)Q)wD|IrG96!E4UqI~~dAC})W3t3kk(WY!R&>PxPR$Zo`= zru>#s+@AmD51o^FZgxxX!SIv;HVVUp>Bi7dV8SRJ92?XlKa?X)JQ(J;k!czoZ81hsCGH{{~QNo8wlGu=!?O-*~fnQ-@BEKg{0wAv7je%aE9T>YWayL3+Nw|3u zw|Eu5<3my~9`GMo5qo}zC|){U9GehMHm;NN;L`t^jtP%4*J}fKLGZLI-Fi6c*})Pz zLc44-w;ubnQ%}-mO+!`~CQJ|JBm8pFYy%T&C6BD=u+b~6u9wTjg25xzx60D*djtoM zvLT-e9)7(OgTFB7pMVV>rbMGmz(RllXhJ~7v*|JHTqUY4Zh!MVf2{aYAl@mi&##Cm z!pdrkyfBvQ3ia5p6@%)gIn~b?V8+cXa3_@^1LQ8vg*CJRSRYRJa5{Ljq1c+)p)WZA z1I0EHc8mnAPYG8n&A`mpJMp2=ugK?^lI}@QIDZl5{{M52OSCV=-j%LK-aO4s#kg>* zRim%SUnYUY3M)wk84ZM#W~4wZ(-Q(-Y3R|pobgsPj`hNrVAwffD+RCV{iUfhjA*%! z2@v5XP&68dQD0&VP{Pxx^xl^53bHcvGb+>9Gv?EWp_V7nJ>ZK{J_3cP4%vzspa57= z#wN=oU9BDQG?89DQaCg*f?0n$cp%cTryU&73dxXxjvf=4XBt~m6|m8k;^LnbaEVSy z#XcPu;5Ym?eRG!r+rTzcQ~d_TjB0>xIeq0LzLL|f&}ip|QA4kv7rRyTjo6#JnTu~Q zGOAh>*1bL4-)H^Roe1S)-f|l8hw|xU%AUk+i<*bumH0l1B0MB{c^`ErJEKfY^IpnE zNuRWjHT<4JN4rGutTfgi+D3EX0Ev5_ta{Yh#~n=-?aa1G*o`;r2 zm-!o?Pw>Om@T&D(^-Hwy*6h1ST$xhKKbUVQZG7k+E7Fta*LIa7rX%`Ze68_pP-b1` zB*rn<*yQrt-+Z-qD!tO;ix>G&&NX@4ke}brQj1FRtc$;2KjyX^FxHrD2y5({wp|k> z`2`s`>;`ohBYGp?{2tn`{J37zY1g=&02wW1LQ_-mEG@7uOL}s_F|V|4<+7Q7W%7K` z((_s{$r?y8eO^YBt&J}$q*=2H#=F_E#^4t(>BtB4aqBdIkHhZgT2C*x;$=cpnsHzqy{|ugEjmHSu&TXS9vQhSr-$iqQVz2~7^E*Qqg_3(nmK$kT2$ z6S^E(&8x666>i5As%iA^^63lD0tPx|*6`R(1Nd+iegGAH*aY&|@(;2M!_WB1>9ho- zq)#$#IlHIdBt&Nd^f~!)&!h*4RN~1$Q-78s{h3YJ)g_JL_h)kL74RcEI{H9J?lpje z(Ec{Lxx6%{hdqvKQ^1Qj#V9TAEY2f=Egra+;qpz-jJS!-)z5Z=a zFIBCk+N}WHw0#@AN~shH_in33S;K&A`yx-oN#ty)&Yp=*Y00j&5f<*fdGh76p?cnJ z67^)%Ez6P=S!_85o>6Go4?IVMGhc4G6~PT*^C~XT#{q5)U!M%J3;N1<~CDQMF!T({*_w(pwr!x`pvkIP9?8tbp9`d~; ze%Bo67(qsLxK>fmtOL@=U7l(v(JuckPuI<~-qC@iG5&Ji#E_629YI1dIko1!9(bVi zW6iIgC%}0LGxW;7f`nB)VxFmYe;N}s%8Sm=E6pKZj@H+Ez8|h2HxPQMyVO6!6NKS2 zH5Pw*F%q~MuShO{XhC!muo}EY;WLZ`cj|45SEqDgibexpqS6)JMWM@wsHGUy8sQu+ zm*&Y!^^Elt8Kn@1L%wcqhV>^({3zsa*TcPZ1F&$1`O}Bauam`P@9v2yXS6SK}eqJ8) zN_AFz78>TbQ9<)2qs*Egx+5F49e<9Zsw`r@ zJNYHLGuY8#%qE7P{+8n|D)olJgVeG_`U1c?%O*7~eO#KdEyze?**FRe&x^q|K4*Rk zidk!uHa+>6$w#CP<*+r(PjTKbiT~}`=S~`AeQt!!V1a;9xWC=g5jbxlZ4j&XPGMaW zjISU!TJ+O>{0lJNy)5&#C}O#DF#m#mqk-2otHwGlPg4f-OSd-NTF`4Xg%A1392f7n zxEnOTUUlQ%rySrpt9wVG{T(oPX}Uj3sY2Zlan zi<3aIcu`n?U2nl@UbE>rXtH!R9l0)kL)CBlWk)`e`6dV`+?CM1G)yv;Liv=#ur(m? zWHz<3+v#V|k(K7^l=mZ~Ic+@rw5R{ir~(RrZ4Q(osjknm1p#G1LcGWe8I(SeJamBa zaklinh*nMW0SHz4zM1r(&#t&+C*pZOMGeclh=56*%eBacgYyZ!=flx)yB2*o2&>|v zVN3+kkwIoh>!fhW`n$!yzK8`5CTNeO(cqr37!mv)Je}9#*i7r`0BvmPbTjd+89bqUpxFi_Xape1n(XdGepa~ zY)&eoqe^X!jFgO@jlq#p0~27*Vq*EQJ)_K5(3d?joi{&Vk9Rf){CLQmDwV{eS`Mo<9@_SCpzq#y--w_ zJxN{5NPx~AWs?PwU*e_-CUO=4&@5HPAU@Z!SLj_vb^{B0xpkl$L7ktm+)LWhS9s2! z?IXkc6A9TE8u&PFSSCG*&tFa-wsM~|sK~yX7jm%X*)Y3XVd#Ti8T>`@6j|)9(55~j z&RL*?m?JpBqbz_}jm)8Sd*>q0s!y1m5rBoM6gHb((?B%qxuPJl%jI-_9930+dRa>1 zI1%v8vN&*QUkL@y{QuW!5Zz>y>qc+xTL!Kpkdlc!h3h-ECnF-CiKRX@LzTaiDmz3lU56(u zDSbENCrr&e+*nT3k~)I~qd11RH61NI24)IF-!L(+D*LVm0GUpF=fLnjqW!QIp2+8s ziJ{E9P8mtGd52J-<%-*6ds>uZy(bUG)$NHR{{u-zKyC~Z09Oop2+ejXc4k}z)~|Ai z5GjVH5N})t=4Xl4lg6gnRh}aTCW=~G7StYHG0lWte-GY>_tINf>x&`DbJEpK5LjMW zl%Lu=j+(DB!W3$dnFvqHs0_^AFtz+@0cuZq9wmQvhVlaGGptQZ&}R$abicl zx6kd=#bZ^fJK}_sJv1>4pQX-f29}SAGLk*8R_CmPD)6$zya!Rj^Yw+!#{uuf!)qma zbt`_0A^Et)_r0Ky2s;M@26KbRPB$n^L`1+NYUwT@3pM13*wQNlRQ^mZcHd6Fi-uBS zDi>0r%llU(R`<%idgLf@<;A|BypWG#5-jmP>Okzvu+n zUd9FdyiI=iXV?0Y!qmehsDKBp5VYBLQWK-kInOejXKH~j!*|Dir4Be6+HcL;_}5KP ze244P_m|zM2sy}Cm$1zlt!Ifl1fV)8=D-3)ubm9jY75pzVaNI%q|GWt;R`P5n@Rel z+7Ry9rfcryN(%5ndUv7X70V7qC@x_4-=Wl9o?Ke3&pPU)_a!(@cZaxGcrP3F&RgR^ zR5a5sE?hOVloz>YR-ATb&pgnQxc1j4wwMI0|pPq~%U_R{0(O{9YSK*aYrA-*sgko9&Ob z0;%&^W?p}r)|8WDDb>+Gy*?u+@U0i_uxvQa{0X@MIuOe+yiw*AZ9Fbm>!lZg()K2u z`>7GL`|&Fc2YoMdriza}rSV}F=A7Gwvo$r7rE6_1o{7AZ$M3GjD*SidMkX5tHgBx9 z$5>+z?oG;D6u^J|dZJkRuXhv`Mk$V06E=rFLxkc+3tPH^A8=0U#%b*@pNh0wSvHJj z?vNFPS)Xk4EYHa1<^SfMwo^Tgj7(rBIb0)TQT4&4M)J88OAbM19=1T?vHWu+A)CbS z5W7jFpDM$KT%{+cU1z@2KR%}}GDkz;#tEIfUq;A3(EE{T^Xnm`q~)M{N+`rgFjjB2)Lbm<|)QyVn5a(Inxm&0LQsc zFPKe^c1`l$ytMU}oaLwmAeLaiURxp6nQl=PH%c z_A@~!r`BD8Nz@Ku8~OzIzW454t$=3k;-60HRbSD)XOKw+R~|U6wo}iSqrX>lZLdIbI^`e<+Wb^^%%h-s-mkyoDJEAtEAT2ch4L_cl04 zXE(#mVoXiobhc`DwmK?tHCI6WNkplZs*%V=9|mciKK7UxW(aU!NiC3%W=B4xkKEm4 zYfSNMsK)B)-46~Mv^y-@9q7X0X(!DJDP)h_h1N#}U%Q1zz3%$kpcDETAUU1^@|XW1 zc|3p_J`4m#W1FFPlR3@K$=|pH3LGpL0lfu9iwzf7OqCVfmt-~z4e}+kudS{1-P(_L z!Z{VuTX?2F;yR5a8_nXhVxqNd8Lz(6u;bA`C9e#bL$@AhZ4|*FuUdb93LHkS?)V~U zsdN=0u%F%7*>lr~rr;FBGM6SJRP{-;8s~RjJ-D#?_{Hfao2ss^WdaEBa_PuBu4f`a zzTc}h0wr3Q7p9aoYUfREB9HXs5Q-Z14G~7oU21Inn`~l9o3JIFp>qF*5edGRpck@K1Cr8vsyL#Wl9< z@xMVq85v-}J_?+K>i}8*jyJdXQG=k~exlF+n_x;NL;<9ST^Rq*fBhF!k`&QjQe1hs z%w`GUX`26d1phPi34rkr9{gbbM^^yS0XedCK_5U*;t8K4-N%Lh^D8qPz_zlqNE!Zz z;elK>P^x;1ena=~Q~th%LLcxND?v2paQnd^f}oMpy&j=oY7P)JG+5M`kFmeUfKHM_ z`k#IO`+Lzb#EA(33uBtQH+%CwoPKD}&jtTy!iOLQSppwrEv*1t9}%?wR&gIxbfE!` zdF3%RaIW+3X1z=3v6<=H-v*c5f*$<;-9PZI1Bz@vc*2eU(#!0%fue5x1dT5ma`;Pc z1L7}JUvblX$+_hJVxoT@3V2sdh1>{lRnj}@2*>b#{?UBB#GD0##SMX({@sZe!UOoq zmBtXK#s|jaQgw~c8|!C;$t`O7j2heTngz`@Eekwy4f5V)&? z_rHP7aGu?7Ae7IWHAae-4pH_qpi>cTM(H?~qRCS|?#q`_x|cl@<|JyAwNxatEtb}L z&z-!)kSr`n)-Giw9t9S$be7N#O%#-*SzjIM&u=JFL%=o|J26F8OlBUhEJ~fk&?q2x zuVl~y#6b3B7zV)PoF+ssF$%W%J{>XvR@6JZ7E7c zNx@Cz?hHC^MOi%&K^Rx8hgBh!LlkVkq-~Oy{er>M%RHV-riB8V4sd=c~tig=gMwcN!21OuvhOA(h<5Qu$Nkc{bRG+v^ANet&UAwh`}pd9r(| z1zUvjJu{Kz_VNfm;@Nn_7*U@~(wY&KPLeJL<)cXq1(QFq=6-VXw)>B{!qdy)_aEvg z$mm^6zbKS)T)sMX+91p+;(Aw-5j7X40L}Ldh_rCq4yhtfpUvRGTx9V0A6?8^~P zb~6Uc=ke@Z*4+XZJD?3K2Bj9LA>7HNY|qZj$B14^FTv3G}ug-J%-l&wBPI{4C zSIFV|Q(305!jE+th#@l*M(A89cwr9dw3NN~RLmS>gRmQ>7#10Y`6EBRKOKw4n#Hn+ z&t>Ye4@%mfV`vNYpd+S-l+92SydY!KO0e8FfJn|T#*{9KcE|CtTm9J50yAQS*YzLL z^II_yEr-$erm6p-v1GfJyHIh}280wJ0ly&?ytOT3<12cISU9yF^CvR_f4 znEWKaQ~l%V9m!e{6NlR6p%})uZ0_RgCypug%~g%|rN%y9Au1CUqkb5!jl`HE=d@4^ zM{8!k`Gecn<7;(Ub8KJi+o z+vJr3v^2r_tQKJ>wB^woh=Zk0QaCYC&ip}%t*4Nil0vyZU)ojX6Zks052X!Ba2eg% zqPFjr(9E+8yUkhWuc-)3d_Zn2ar-r!0?2{cBAlpuaL{+KqIyY|RFdk4Zt)NI1?<0hy&$UF z$fy--?^tLx)#sg|@}@>%kyCYsB;N*7Jd?EKS=d~m)8Pg)(lQP$Ix$gRp;A`mBW6zJ ze+>A$ z9*uO3B0aJBllT85&y=Z}o@j+95;y?K%v9+nq^)h1>o=yK`MALZUnWacQqzq1{0iuV z82v0UbSJjHt@U;OJnII3ENOC{h~qnWo>0X0~6_I4e?SEsGVx(`Yp(74YoZtS>l~b*~#WiGU z9?quDxRkJNX#3z8I46g#L_S>1Jx{LUyh75AGuu@x)xAiECv*pTLp!OW&F#}cs!0g! zm_k!Q)+g!Q@w4QQCk|gk@le#xW%Ed;!O8%{`O4tE2YwM57Wi8{ed^BKV`Dc3;3O~H zouLy$o)Fbrf~u?~JL?^|F;6U%o4Wx-yv)X0W-o82 z+o-HY*oUbxGwaBAJr?$HIFQ7$I%7<9knko`cRX>kD0BcIln~qyUr4=qOgY5-kdO8A zy#M0EArsuE4*}rAkFfX*wZ?Iq_-H#dj{*PlJqj@v{e)G5PJsi*c%o=1(YUSgPj6UO zb?vQ6*;-7-J!Kd#D<@A%rDw2Om%LsS7}ED^KAWv1A}0_5ar(c!6~XJU++)!QG^=D={Vzmr;4It3__d6c?_L;VIIGTPvVLtfDO$%ERd>Cji#uo6|MNpW#gcH`Wb#UiU!9oBA^aus z8$Z#PqOhT%A;Jh*RYhb`14{3`M8ZFv&8`<{Mm9h!_{Q&3JXDMv=r!du5>a-s8M)7h zL*aIXkpSVYAQ92Tyn2#aR$unR*vjwylqRx!sCvAIbyss%@5fSK-+zj6wK2))|gHD># z59m8y6U4)DI0MZ?kV;QHp@1CsCe<*wX_}OXo1_)#A}JI)$%*PNU87P$n+BasM4gf6 z=b!9HZ?RH|Ps)I#FZBQMwU0ba!flDIbyvgb;|nX?5prmBVn}l@JEdN&bYd9X+c2rA z;wKI5JPhdRjaTUgp4{4!=kck%|NbDMGvCr`7bp?EocmWMf6(&k|EfRAf9R z?>#$}mDQ~P>xO!@+J8t$q6-Ok4FAkCYM;pFrb09cj-^aN zzYY#9z;|r4uPkd=;rl%@S)Z4VM&m{R==p038`qL}HMQqN$G%Neh*wUiwaQ-yJ0wtP z;M6AcPR81V@Xb`BsZ$XBHQuNafKt8wL8}knnmo#c$s}%ic}7&+9BN@-!256C25CVE zj1eA1Bx`S-bzR@Z(s?k#^Q_#{tw>EE!;0Wv>d@`MeIMfw3Xx#Rfbv$PDUryMS8-Yf zJ}0}<+=s?rj$({0tE4&nB@AalQ+lB1y(Zt{>96TL=(U6@!W}~!BJ!V~SIG1X^^`ka z(}aBQHHj5uDxjRS(_oWTFLI(avM9BYlwOwwq)Z%;GCYOgEgp10bSSyp<8wEJ?ViL358we%*>msmSjL|CXU%ba{cQT<20?@EEi)4lBb_?e3qsPHgi z6~Z_0CPcL0#g(2nljXM`S!haMMz33|IQkoCw-a(+>kXDA3wB(-a}i3S6Cv5U<}c(L zu6hhq!m)YZuMPoZIr<)i`r1GlA%cTES8UMW!HB;tQ0AbLpg3bTQ4E$A;&}ltB0WHm zK}OV937ertzIRObnIIw@JKeEY2y9+aoQS;aXvFsQ)H4gPo(s8a!zr3P0_Tie!TYi&|7`>PZe+iGGM@i z?`1w1>c8?alnE~hm~UGG5yb@;11SO@u7P53{O}KKgmwvvt*__;3_*F3BS`lY3?%|m zGBLJP_sQ=&9*GhIQFjCqW+eqS83LT0pqb$u;a31!6omVOsk|36l^%Eij)Mz>cNaF` zVmZDC1_Ims`_&RHfJv7;xJS?tVx9u;ek+56h42Fy14Ag3Jl=Q<_k9P z#;d<#MPUS&Wx(M8?`P+xdY$Zy?E8ejRc;th5hDkvc7!r|+mK_PZomD7`?sR1^cF}G z;r;rR%=+I$QsJeARIG0w#VOv1UPlYDzZT^66Aee{vc>6Z1ByiqA~Jsfm;&ro@&P@P z*OPyL3HN8o%}@h6BoenuLV6<6WO-!Y`7zAK_F1}!6eU3ZBoIgZeYAcoHq55JmHDgV zX4C)+rjP~O|04h}U5%F=sI7eV4tr`r-R~`$gA0h%QX9jul>DV^f~YgymkY=qk?x=E zOpm*_E5+W))TAK1C+n}GPz(_xwmhQQ3K%ACV|;{gPJgGPY9g4Csoh>OD?q}h3;}Rr zfMnpCsUqGfIbOKv9&LGYMkwFi$9rzd;0`^I*^4v<&WpzcD8m%vIwD>^JZWQC^q2e9 z^9@KBmAI{N0XqLlbOYY~Rx=nG`EOp(dyMAJ)C|W?Frp!xQ)7FAK&2d?9}Znbh$;ZT zu)sk+U9_3EDmY;LtC69K@T8zjSlyjo!$m?AkO96N4v#&qT0ngpv05J`XN*ZtJ@U4r zpy-rX_?jOcAs%N=eYfIpRzN#tPbXm<|54#1T$~+#4|%VqYljFv7NhV&T&V)c$=!7j z#mNUO)l!t=cPn7=9$nzUPw@c53TUz2W%?}&|DM~sr?3@VL1z+qw^=0MLPmU)&(@9v zqeBiygCqWU!qtNjb9%Ty!{F}5K8c0pzr0<6BplS~>XLD?-!GvE@gAesMT(jLJIuol zU8Xu?F3inEy>QIKQ=q&TDd6HWKsN?OyYJ7Y!YL9?{V*f~k&_)*3+XL_8H5oVxv`*t z_Q)s0Og6=`>>BiWq){hR|Z7VnI)eE*12 zJVA){z_}6*=j0%K&`-$(1z2JEmO?NNm0prW1U>N1d3Y1xA~&R0@X~Yqw-;;%5Zq4y z9<9O)iA!(^5c`C!0Gxe)nPbW(AT^A1x7K7|pi7z|#a~ACpQZm#@fW)b7jUER&scT@ zSLL|@mM}dHRpp45r_C}Rss#N+1w!FFiRvN&1nl4oH2A+~oIiz|e_h@Fj=9y#EYaRH z`A=9dfGz02ZiA@-e`&BB6kjhBy@bQ@*>p*oO*Uvwg-FCPXu^ZUHEq@e>`Lpf@G zu=LjkC(EO7or;ZRMx@#Ns|9!YqUJ%549|^^PZI-%f%{MPnqujG7(S!oo z?BoSRlfgtG$XKgijmSxDG^KW7T0FxC&Y&%K7_!obpNZCpN$}0y)3zC;xK(lc+6D zw20S@9PtAe#+x;V+@8nQ#^YmSbYv5CtC66{?a{pPJmM?Tq!+mGd{+xLA;0}^^yB&` z0PNF~S(P5S1{hWnQa|b+H3QrSoT!X}HPVPiji#z<2S|M^^-Jfyg`a=Yd{*77+DL=C zm;A{9yG!;h&kIdm$9=zjUP&)01G>JoG0~b{zO3RWtrVo_q7RkIj2DB%YPc zjb(c3ithwCjN+!dH(H}YSFMl_ihxi5xuU4`UI z&9ZPjE{^)u*(9Y7Cq^V-EP#6Ah%w!lrKIG_#-w-dB*uJ9@!x<=Cm$oD6LbC6h;S^3 zF!M>u%P;9MqL%u$g_A3`cd2yh^H2%H@p<8<2A;Ln@$v4f1s7EYj4XifB-p$QpB~M< z^K#deu(rmOgmBTRQU7iYe0F3RlvdT5rzleJE3w8YMfjJN?2CubqwWo;4q~i(D89yM zHpc&mz)x&m;8t$0G11Orb#gw=Z$k^~Xee=rPZZEuIB_9{W;SJ|CRQ^UXHiUA0vTSe z3r*hEcye)o6!rZ0WMZU~*}&U5RoGB%QE4?hH}*DXbZNYKSG+&_!It3 z%tl#AA(G%a$o;-NiGhJ}V))5pjn(3tq{qthHH-9ynyYRuBvmm&*KEYqYlxedP#DL{4o`HiRC`DXkx>Zuw z?Q2o)j#r)*VcOLzp!k(+i9%{WBu1W#blPe0{#yshYI;?3JieU`q5?V}%M0s<^R>uKB};8g$>Y6EZ|Oo)(|e6bG8UQ++1%K#^dafL4U2J(R0Zgeow); z52jDQ+MsxI(GE8!lj)_+>fZN(H!P!7B(LX+f9|5@9x%N#2Dwe1E=Ho$SV&;tfPP+E;Wdb?X@&34@T2h1YMDhE@!NWbvyGC(9za`*31|tJOwtj z&F>Wdo8+)tb)Ya2B zP6WJqA2_{@K||Gp`J7X8_A`1QtXHvq3kygn+;aVLD=gBmS-o~5Te=!bCusY{h17v) z=-0VDT)>6EWf%Q<(B>CQ2z_pE9GG{MFMOrRG9;VPE}qd1L#b|M-0$7fcP|^GdR(O* z#ZC$2TOloH$9u&xBP_}?a^?J=G{`i7YR;dFvW%(9`5Dyxr6td0cgXq{GpZ=m=M=J{ z#`%xd=Zb}d07B4)OC-77qUjX`5><(vq>LI=G_<(+R0GmAg{P7w9>wPf6I_u`$={DSdk_c4y<2n{k|B(&{hq7y!g2j*3|1&$ut) z3IaPvmK-HuDb5yu^cmRch?wX0mXX?)(r$oBf>K#-d?W?j%j@8F;JV1i2TP6#i~bw& z0<6gupMNrNAeTamLA}F=ABq2<-V-eFYdhm$@WUd=&t>QR+qeo28 za+V&CkjBcdVr?)l2BM$m!BuDv@Yq032B?LDL=M*)#L%MD^L}ceargokF;{u*E`mej znI;h==04K^?sy$z44qgR>T@``Q;5rnTh95LkAbBmPr9Uw$7te*@=%Z2gX)aX?7x_! zt$Ita@1s9h3YCo#Tf z_6ptL%xp3Kf^M*Y=W<^Q*gAwocsoBposQk0>T8nz$6fu<>vfMv*e_%6&+GU`0M67k zg1$WJvYB6{E3xF5GCl&@@V`;YjVzY@)-DlYWoCBIRnl1%kHls1VjOv-pFp1IIbFVf zH12Yz*bC+xxDg4Lg6fnQ35R~>Uob%rJWE>!O3Gi-0ho#vg3&jW*@`h$??{9ZSz|Xy zk}QM$fE(dY99DpczSZSy=VE03(fnRf=*THp*CRvDjoxe zDIww3b&QC+aw}^=l3l7hwBo=GX&HM)CXB#uH<&7S zTGJEm%BBTu)?RT$zF;|H#O}W%D-(UVmwia0#3Gxn!O|vlA+t;YfWrNZ?Ef@vfR<}j z;Jy*#=k7j)ZL2Cp(TX@PcyEl;5BBwy9IPHdR?`Q-q?T=BGML%w0d4hv9}Qs_P8z!1fLysadL6V#Nx~=!OD$kV2dsMC90}n4xt^V zgL#_GI!?>|-3)Y94je!;F(z3V-tbN(W}pE{h9=%0^!}j+)YOY*GvE57r9h;%3iR;R z^h1gD7A^Ht;I7fVTgTkOgd^f1#-ydo(rGd#88XPuXKjz(*}o0jJ=f|o+ZX^Y`dN9Z zt6rx5IVhO!mKHW32CUn_nFfF)EDzqIN%=)A-2wMB((Gg0FZUS0c?u4!t(F?(R6q!8 z&a*3_K2~qrTa$gHbFP*zx zRR|26LG|$aVpJED2Z->wfmK}O1G0b^MD7#BKAUm8A*1uR3D3G^Rn8CG^Jrj0M5xa> ziem#$;8=pkl~|9~&h(=CZ_f6j1d{n*J5p)XGuc`j9i;&ibse|0u4|4KzS4ZV2`|*i z!h)wKz6j{BsS=kFE0i4C+0L3~CWwrE`r z31V)DZ(%InMNcrA-n3uoX*=e@H>RO#S_0mkG90#I1Z|WQHhESGlSTr6gk#(C9IJ{L zAK1ZLzz)7sRF(sn6EI8Szh?=Aa{kv@CWjRWwh|I1Loyp#qVzR{Rora!f*4C!SA*Cw|#|TTHK$cW-Vp{F&!pm$0rc8=h2F15HYHHDu67^VJ210|s7O zEZd4=ylE#u7ZqX)pl{0}K*;F1@mld;=^vk>-*P(@@Lv~(-Ozy<0bG8U37(bcMajh@koPwKSYVq)Vnq(_WzFer~6$6oA1|J;cO zNR2Q)+fLbOyYbm5m=x4CL=7Xw)@B2M;)Fa0O#*a!k)8w^5&yFNMF4mCa`OhkKasL8 zIjuCRu?H;RwW_xfVvZPjx=N3YrXfz|dJHqm&5tWM*-D6ise+#m27AFXdKD+vJ9 z01T)BMcQqr=PeI_;k@D&{S^lM@c@vIS}o?kye>)#RlGg=`!1|ElI&_$qjmfKdc4_ztBkedp7!5JyC!K z0@b5a68J?Q05x17EO|#IGS+=98Ej1y8BWe7yxL%q{CGN>B~0ZYaX49X4Z5-1epX6i z-6Y8+w-M3PvVHpKW0MzJS`q(YDxe;#nz1Kx861Fm=rWl7B2}0(+sxEYbN*(@@=CHH z#}Hv0xz1yqPM|T(aBlJ#iH~^2;DEMBiaVSmqyb0H!O1w^Z?fK}*_0@xoq}(X1z&(HI_YS{ zy|pSrdClu4KG?+fH1RaP>$PbedU1Zr42Sx{r0% z;}s5#jW}|%*B80_?u=LK85q33%UtrZqSTdfF@|cn7gd^TUHam?a59+msiWju9Na9; z=fQNS$4b|jNbUf}a}YS>`E%=IG0d~pjhXTfFkwtu(VHtsrz@dBorT|bHCj8SZg3yi zsw&Xe0j*Uu2bykH5d&<=`WyDfJL}{?;x~_|yp>o&Uu_+GfGTPL5h*9wvoo%5z-d3c z)PISTHXuNhjv4d%V?`w#CZNe&{tyR^$dr;eOePlSfKiKs=>d0J5WoRjJ^WudWP3luoXRcyqVWG6`B7L_8uM6WwPJ2SE?;i`z`(%C z0M>x^K=6R|K!}d%s2iu}3)`+Q2@#M-$tnwDebH@X_;fTkXP_q=)zbxSA?M+DSyxwA zqrWrIAQ22SUlU_hQh>upWp}=hj5PNF^C58SMh9%u@-g>&ZruYN>j0Pweh(rItkffpNt3#g-DgT)>ejEEd{65A@xh zPVYMA$6O)4fq+^zAHNgjZxc=Mej39dWuW_3%)ZB%77-%1&|&}lXg7?A-o$4vob7%5 z;T17KPkj656;BKloSZg3Nd+7RYLCH$1(L`=Xm_cM+5Z6QEBzG|XE<3Fm?dS7HlC{3VXn?DtWOzZ*=FPC2V1 zN+PbFB3i0gMMUWF;)xhgK>#$5`+PVPyO(umy`|63OI^LEo&z2;d#FJPB9sZyv!<8u zUeE9qzzeOU#8Y^CbLqdd+h#yi;LEF<4j92@VY7nXNaq_;La0;)>fJfaxjikY5PrGvFo0c5dGmGH4>)$$K}Df8iKxof#=!{uAo+A9(uC<)qRtlN*H#IzoD z86W%7-6F9Abemm9YLy5qk8#KB9^hX98Bi^7ZSml7a>UUE%tf|bx;({f&W7SKerew} zHHh-c^~&jeAD-NcsKV`X8n|`bT@kg%2`s6APDBhoc+^M2!|E;0{pfId{jJ9hasUbC zL7Nc~czAwSG7q9fNrC92x3*S9Rm!5t0ES)q!GM1qFqLTp!ev4MvJhtDkOk zYBMW#io<;p29P5H?01cWsTO;ZW-%6Woo0L#K}SbZ95-Pjieh=u$a*?dd!7ZRT$**`(5wV||P&|gdk)4Z|6Qsjpp*c&RTdgxejE6}6 znyeb118o!WY>NDDzRKFjJ@zd2T5idh4o~>PsJmg};epn}v|jjLAR`j8EwR?7_WPB#1hn)YR@B?s)_ev!Os30uEg6mYXUp{&{SfFIb#cf zy&WI`8)URxgTT^=qBTB{a=9KDx!OE@5uEX_s}Ua`hZe;lo2c zvBdw9JVTyem%S)3?{mjC{amh;QX8xO?Rvynl`WAyf}$a|DphdWXk5$!X!o%$USvLp zBqJ7H`#gRc>X~-$!MKWGXV+B#N1NV=2{0t0c`HLJbJUBqW#G?cTM0%NGpE3rttkJV zz-1DKGbE)FwLP-o*f+mzpr;5w?_h^)?_k86uoF2u%by3`+d zxdUFg$+y^_$HHmA>T@(rPPd}ncJ`&4sqB*kkMZhkx}<_=Vs=!|#@ias8hsP)gc*9- z(Z;#3k989!t825pLvc^?z000=TzT|0F_N>QvO@jUC&ptrARoByq={jHg!v{o=)Lxk za|5g`8hpc!uwxrhi>9_fzu7ag_BP~MtxV{eEp0$DPJ3HLsZZ+oO$p?&+e@HkqAw=< zZ2zMKE)}b6xi(N0rJd0ai*NOyMg>stN!6RawRBuY^hJ239vOlQP>K46Pa;us{%A-Oi>YbD;AJi@n5x_faDruXbL6o zZH#xUq@0nrIeL?OUbo}RdT9Dr>to7tb11y0H{(TBLt;T$kETBxsoi-iSS1m z&9X5aF!Swoy>0oNYuBdtme6lGEkV0sQFlxYpfMo)mrYH0(5I@YdGA0UG z8nWGK)cf;ar0wmvp&*>ko%G|gRA3A08Lx@8$FXSUlA24<6_}JS=B`%Yre;d4I|2nC zORobTN^)BEpY&f*{lrZi3+~x`X0l6DE#5wLB!iTH z&6RU%REdx1m!p@m^JPbMt}+vK&NYnAUEGcnhQ4MWqDm|$zwHnQbOwJ;VLXfuac|DM z>`RZ_U&-FX=f2hA=ubdKb#d9YmdSJF$N}4D&w(MaT$Lo_xgJjv4!O!rM?FNaK{=YU zRwzk%;9C4?i-Z1QM(5>yhu+KaY?{GiYnP7B1i!`oi_t#NnBLH;#;Kaz zMP!}N(Kyf?7*AtR4dZ0=U+Pxd?NeB95ltk(N>Jy;g~(dP0V^R-F>xQbh>%1!qwG>V z4P<4M1o&X2HWRw6T^>(kecCtmNMfZQ&n7EaW;h#_hrr)YkbT?omfyPVuglk{8tB9s z({b7Biq>-!Z0cuAUG@$w?JZB#BmGXqIY3?pN2DHTb=uCw;NEN7;+x01U$B@;-eT5Y z)E6^ZVErAi(CvorBd-G&w}T&oo8?uf-_%r%w#5}xl?y5!2b8M&XAK{HO!7fKbXj4+ zmg3I1kID_Y9LmP;Nwc>E9G9N%FnWfa*=>9Oos;w=^D)F=hEaTXEuk@?7=lPB zTOe_oDWnR{=vEu^O-l?Y%T7{x`?-cIcd*e{jU%k>5w18_2KdcoYwUJQE+O|HUuO4~ zdKr)W*^ov04`$P$#RKF*@R#}H2x_Cgj(aMUR`RCqk=BXAEKd(u9&^Ys-VlP6!=P_Cx-@7C%* zSDV-TfY|ML%?8&*xg?i8(|FsmUAijG*iFzfao7NiC{5-S-6ibT1l#W|B17;l7Dj!#g-2bFA0m^Y zcnbG6>xV`i6Vf>~^X^SJb|$#fa4Tf&VV~^u$&=HdP4a;#G*JwaQw{&6=})CC?`lKO zZ3yvRsBkNa*bGd1$MIhHELR;n{{Aaqcuk^Fbhy~BaedYKvSx@d;&TZba{2A&k7}Kv z7+K1C4(%{9sZ|-fRoe098u4(JN5T>W%4xjN^khNW7*gTvvv8IoZXu=WXFwp+v|%R? zKe^25_vnZ1!sUgU$dgHTwHxNPl-P~8GFs1W&34N5imw&Lc>oNS(VyWCL0fTB*y(K5 zm|fKA)tF=_k$t|2Rmc3rPy#~D;fLw&QCrc#z7}CtQOe7c4*U?<*Kls`hZp;r04P^( z&BsVU>V90E$hetqFsv15r!ZyPZm74!79k_Q4mDS)H@RPU1ulQGC$^vqe~P1;cs4I* zvM;fqwE$x^~`yfZD7;|1oE_PC-qvas-I+$351y)i=&!cf8_X$ZlQp z)tV-!0m9trh12cZhP6Y|#-w-1+nmgJegQ()zOIZ>ljqx_{+YhNc&bJp&4|vUW~8wj zxdAOn>SYL0v{D8Jull|O7Xu8Jz=GK5Dch?*)wK~@q_8$DK?*|5S$FrG{RX}q1^_Ts}C6LWMJ_O=$wPA@NaNqcqfHsnpsxhgIdd-966JBnUr{( z0qj0pO}A~$7cSYA2`UmSsMnPj&Jjcsb4?{dYqkxCk0M@=aHKsHGaWP9*$Pa_w1sFe z>KaTNg~@6XsJ*|pvcM0{WNfhT2WkrDg_O5jw?oEfh1KX@nljn~s5hACSBnTA*%M4^ zuoK!mpCi{0o#ka*yF&Yn|2#hZj3u7tnkE+L5De^>v}iaTHnspTWdXv?+ZbSA($a3R z^n36V7@tHvXT#Rek-T5}X26JH=tVk59#`5WcHO|rPLTxC>$E8Sn8wZFC&%=4jTXmG z_C+@q*^6BRb{IPketbbB$Z%yItz~LxSrn`3YiJW0Bf+Vd+|u%`j6&<6qNJZJ7Go#04TayB?j62Tjzt zjYQqJYoFF^pwbn#91-bfEme)bUk1#}LONid9?W_fP#1?bGR#h5;li)-hie-|o?z6) z4;(lCxjCjZP0-}s2Q_VU=Dod@Sx6-+5xy7!{jmMHHC1J34*GmtVNA*I;c>E{d3e$h zi^6CUn{=Sp*Q$t^F2k%NE@(l52AwV>xa>N%ITv6RUS_7X^ktw^_yT9ILEnqGOig;3 zW6g#{Yf9t3{0YlLmE28GAlvCuIq8j9;b&{6Yck+0TA~66V+<4V<2W1H8A@O;phm95 z7vjf>t@cPy+lCgHfz|d^7WGGj7T2Tcz`W$!F<}KjE(x5;oqf7z-~D#y;~jxv!gzw& zVRK@D1-!HSv_>}VJ^RKm5jOJbbXK^@>8suO4Zs5yNB6|=gJ(5sHBvz%uoOo zx&=qP!d>cJzjAfWtpR{bJ6*LdIMn>zl9}=je_+x379*0wULI@|6QE*@-)?t4Z{fmj z_+VK&!*Bs(`nUj$WpxN&U@ovZ)K{Ox+TK61cJb;x|3)haLZ59ddk2qw#b&e^y))r& z%x-+~qP2~70Nx*sVbk{#1Fop>`WNPy6ncr_J|1V%|7KP#EDEf`!k8;;7?_6PUO?FnNHMpJ&kziHR16m*@$^y=A<6E{{CXbbUCuf|u`Q$f!|<`eMr&E4&L@ zFYWv!`S@FkV@05(4dz}V0`HaHhk1HO>Bd{0TxwzzV5 z7ee3x0TU2T5=~l#T({5~(`{_J>!Q=DI4s&v8ZG>64WR|t?7H!6xAXDNNoIveU&VRc zjvrs+@$T^7QG440p_zT@ir51t9)PBBzl6_y2cU5RL<~RK1e2EXb1nhS(o#}jOZlH% zt|7DOOYQ;l|ktQR0EaMaBr?8-hO` zzvuQ94?cJq!!8UiS-#X>=1L1S94RZgTb(Cpe!TMH@-=TW5bA)Kj~`4b{NL%Q3O4%T zrOUsh30QJs3|S^;(=f{*c%8Y!#V-kV`zE$Tm@Mv9fTy#_#SAfqyZ1_;t{bV_7Tpaz z+FsbNzv1BpdMRiaPlxuIi}mBQ`0(|Wn2GTBuKBLe{3m6zc6PxZm+Zz3UQ<0OPYMH$ zNEUP>1)<2MhG)xtnR^o-5W_#0SB3HZyAfUQn-dkp=fi>kQyT4ai5 za69n5tiX4HLa$ltvX*$f_w2r6P0xT_l?RG>+#37VSxHqD{n7R4Z7=az`TebvKev%( zMl)bF;`MCZ8Y9x8qdIuKHGZ@#ci;CVh|siJPV!c!7tou|;oX{V?VNH9*uT#0wEfK? zq@vp^+i4;%cfV#I-56CJb%Lk?RcqeQ;KKzZ2j=jDiNuHnkX+KsK7S+I@87_=fe@z^ zg2)Qx%RIfwEk-I{RO*QpPqDMV;9gx~085hrD6PmAFLz%S05HFs7oz{uX#W0jUpD^g zY(dy9^nPBFVppKV=dY%V8(IFUr<{Z%G2=IB{vznodu(NC(&*XVBpEj_jH-{Jz&(*? zYN#4ViO+dW@bxl9XCnZX=YzPBDf4?5e!2_)M?dxpTUthN_Sah7%zi%1oYl3~f1Yu% zv9i!#NK|}}o_>)QQh23v);;0hUe?Eljb^a9;67`(!+L`VE*pqqf-;=Yjz7^&d|Cfw zrp0hbfV)sKznjBXvaoPU$?m-%bVk_Az|=HK%Jgwa#jAfGbcLMNOc*_)lo{Y3*!YBk z;B3zZVlIXXY0KlcyR@(vO>N(E$_FutjQP`PvEoHm=V66Y=KaF7)Dxjk|80oCvXfbn zMCD;&cnh>8;;D0%dx86VK?`H`F`XTK{8O$lc952V7JCoDdT|e%(8$BwVm8M@oNLat zd8<;z-0Vi_|M8)5fEq|r0%8F^!A&&KJp;!SlU5`LqXeqiZB@i^W3Abu4;V+y)jXnw3x;rkJ6g)j?$b-u44iBBh&d+Px9R5Y$<^J*h zKY)k%?ny|C^mrIv#OuIDEd!rWhXsxZyvE5XPrGRM)Uv2I`se~k06W^G&<66D0%SMT z1=j^UDXroD)Gms14Y`jVxtlTC0}%*N;avDDK29I@yAW!~7(l4kj)J`40xBjq3Mlh9 zH966%Jkuu;VU|FF%g(No)V85l?UWg;liEFMQzTK)NyHg(bX0A!?FEZ}c$Y$xCWq3p z*sVs;V8?G%O_{SN#ix2{UP9T=>>cz>aYwB7!Pvv&eD_ap3|9A*5qMp|ru@sC{(KZ? zacwwUyMJI0zd3$%W#PUZAM4pRve5IK8x}I4$nF4($Hn8K<~T20|K#gB-b}lrz944D z12eRrC=RsPnWM2FiR6Io#2y+Bm9GMoNY66hDD!K0?y2 zMfbm;hUh2kr~vM8`O;*ki1JQOZvcdUwmfYUWU%wHBLwq5lhWgqf7_`4Wuh6$xM#{p z8IiHfy-Ji5e3fIv7rWiiE+^>ybUg0egu9fQYWJu#wa)Z+Y`ch}xDI?3VK^HuYDP=) z&eI{gVW;;W>`>lB|2VHCSZnx#$WwphE7*qG%Tcn~F=VvJOX;t90I0RIiV=gQUgd@_ z?apc1=aSN{WJo6@2l`mjqzi~%IFii=7P~AqoXa?!yV#|?(QDWWNT^VbtP6T`(jxdD z#Q*13fr+mA#!iNO)$fWP@3Y+aQOH&XXDL2wPPm%Jauf^~=aW|1VS!Nv$LcVAcPDOQ zqpUtZ&Y>)|lP21-$r~+m9YL_EKYCWUP@Z5I7C}(mBnY!KuMBLjYfz4~A9o%d$_m?2 zCe*Z$ENHOnA2-~bD5^hIARfFU@Lp;Qs(p$+VYdXw>EC&3H^8F66lt$p4DS}!6i2v zRgb-N<}W8!j_1ize|*uBLAt%Q{7hOY+B+qB&{FKJL5G9<@JYo6oZF>S8onP^2aHb| z>^81nIBhfTb8nsCmzt0@qp_(lTLa$3CU%JTg1b~bxdxK|K20s9(oDXZ6O^lWl6Vco zBI(#TqHfsc!uexJ-Sph0m?pVO(E6Cl#M`ND_b6-3fBR*bcAz!*wD_6CI(n(DJZ26H zt|cip8-5unQFt$4N{VUqE1xAwC%2+KPo;VE3OJZz`!ofrBvHgj9BuQ5$p5tBCanBL z`qu|BK_vb!xPe~nchu?1@v=dNR;NpPOUkeVHV!+LIPm6euiZm2b;{wcLJHF@8)Hkq zfM>=7la!UNo&yn)c9@YIdQa}K41+kxvtFfr7L}|^_nJowc}z(W?Bo^E|0r?{)6}3g zioz`3kS*`5_iXm?nP18KM&)-C?-wPeR$Eu^%P?AYf+a@58mS14jp~r^>o zdi7s2>(|94S$4+Ke6&9HkQ$Xe>SsAb@jks!;?s07EdF|L;_?LkP-=#mi4`qKU;o_z zylNg*BTIY+(UFecpdi*c;VRoz0d(FF)R0geA9=8``*j(+kc8$Qwo6|ekl>|7xPG=A zly3UCzs91_%xFoQcH6xE^~G=;HF4?xzyG^ns^Yma8f2 zXmA*v8hJYx(7;OXvZT2}kAej66TKBi1XCXY*sF5TDIrdMJy70ht@+yOktfpR>0?uv z6BP&g6c$qEMFo;igHlbX+|Yj9PC5Sc@CfcN?A?!WDX3UI*UtW4m0Boy9w;=E2zSk; zRWV-UT=I@2`p)?B$zr8}U2|(S{o>?q3VVQbNSsTunC1q*H=`+P{7{~ zqp!&rYqkjbBpt-gupON=z1nICsW|^@tJrG~QqhW*sNAGsEnY96OndefsFHuU21*!s zjyQ+B8E)>W*u1eouA=jGb(*gd+-#q_%w+!LoMh5C!&co^&q-oEduMoQrMu&m|9X>5 z7F}wIb3@|TltL72Ltko3I%#UjPh~t~fvwUx8nDTFtB1`gj*C2WwT2t`e+z& z>f`e!Vmbh-kIt>xoR5GWUai$fb|nT?wlY$fz6|aj4%+J6a&%a@W)gQGq3O34Kh6^n z7EUzZU|tmtN^tEC?iuR^EnU}-NGR*g@3Svo z%uFppXh^fYmr9?lc`v5fb5o=VEth`Nr7_*0sfrfaY4ck!e;VuzF5I&fMPhv4mL*%e z$9BfB^w9|*r1m);&PQ?H$zdT`>sIYvKNo)0WxlLn?!2g1KU#Mn0`IX2z!v^G+$q_W zhIKB?o#$+3OJ<*$DAP(Fl@E`xmLTFozYM$bmvud9K8S#lwFh>SAA>ZXJS#NIn8+|INGW8207m<; zRS>sMyCuYom;=F{R#aoS6Y!lj;d%lJV#yi2aU#XO(b%&wk$vQ~gbYiDAlXeh>p3!i zTrcoLp(4&1#7zD4j-$)+%EIT6=1}C?WRNp=@YY$)t~@LV?YBSLmC_E$vUAkme}13} zKZcV=AZz2?W9Q}%F6|aVa&&$eEwMViEiGE#c_lr%ho@rP+Xw0K>I5~PY&FHOCv&3f zAM-8r!mmY4hZRztBW4U>8!1`qnMK((LFpg~^Q@O3^5xB}-J6qp&9rGcfm2RDnsp_Q z_ibRkA?Y012^DiyP-W8_ronZa0W!+DslDdS-_RM7*>t%_m8B7LHs@u6a_>G;aM-2m zI+qjeH8gphi1n`@@9zkCq^2tKO_^tJ1$)59E*t@-x7!D-IO_H zeBy*-nMnsbvErjlLDd~$20c@f$tLA$<6Qc9n$C|n^;`_DJ8s9iG149UZ!_xiVTf;f z-<0wNZkr$j^?D67*{5RIf`oP7z!fX|yci4_A~&I#2;cm1?}jMX^XfVedta?>VMnrd z^?M#uji|#2yfR6v1+XyjJ$3!20^dtp_gJ0C0ZU;Ov zF;?Z8n3fylq%hiSmD;j-rZQPsIT4Rd{hJ@Dq_Ad^sXKIVm!SX@Gd3j@C_>|v_pXq`V{5@-x zquo%4GTC_^h|f;fZsmoqd&*BUX^k0c^1+a5H*84Qzt>94! zeR|k%T3eR)MtC-UWY!t^bxmfCl%09zJ}&` zY8~54C_*;6JVlwoaF&&Gm!CfysIp58l7~oPCF~15myZ^5d@fMPC?5kQ>b?7&6eR!S z=?(hbX|Z08JlpTerUyL5C9@q4l!A!?b50-T2y%;dIg0o9R--r|tM5NQ8Sy}#ibsd( zMCY!~W%cIQb(f!vUGEJSw~)~9)iv+%+qz#I?DRO*W*CzY`-knQ(8_Ql%YId#<$y8U z3)p@kW;;ZIvc!@~$@gYSubfHKOf$^}PoW#83`2iFwh5NJbU3N#W7WqRw&zmy2k)4& zKGXhTgb)btk6bdMPhz8qRuXOMkbmp|K9A>B z4(G5C@CRPYCUHxj*mmsM+uIa`CaA_-QWqV}Hxrj0(r?M@f74CvF#b90yj06X=8Iv= zk^|KHjw8?s?jrMe0j^uPyfM*VEXx!00_8j9mAO=S_i|({-cji*C4c^cX!>-}^q@6& z5M?y_`iD&N&V;i1Ndyh??h>V@mG7mxW&LpHUPA|eCbUGbR++D>9e(ne&|`y6aakxKQMtHg3J{fmj{!9@q=B`?aopIY10T z9lPvTfD`yucKX^;{4^i?S{_|K-kD@vfArk+CpTl~S#<<@pOU(1{DB$Gj}<&C2^I{S{0O&xPh|7TgA zWjV%4D}#`NGrI*ESiE8wd((K%P%iyq<6+7d@pND*B_dIh4hN3)L%s6jia!_4N(;!T z2(W8P;6?or2rV6}f>po=On6rCxByz1#*jCWZ3dj$3&U=!St5o&)uTD|nx0=KQ7BKLM4=)JncMn73ZVvKaL-dhr>oId|L3 z?gg2&Uva%^)E;AyS>jNsy0*0891SD4wvX6Qs&Q6uLjm+;Tc1w8tLS4^Oui*tNSp-3 z6AL8~H#@G_y|Ew(YYv+!%v94&bA>a>h)y2Y$ zE8}BJr~Xb{79<}7CHrwLSH%;BC_y~ER_qKl$j>kkmwd2f9iw{Sr)q8VXjVwQ;s%3W zsS|Rej|>qCja%(p+S+hGN^p;ugTCE;*zp05NZ|L&u2)a;ev91m$ok^JpBncQ6WH0lMyUj0$GHZ98aRR_6vv<>CCLdPc{bR(_` zXc=l1&@z+rRiiX7j0D?k2b&y^e`*gG{BZ6Cl4eDF zSy>NudUfK|`N85%i+R0L5U;;06DMq$WtGbxn&$bLjMOcN#SJ%$C$XjGdDEQIV0LCO zyz1A8n1FgBhwz=( ziQK_U0#atGU+d;<9gAfGcX~{AKSq13EumFa-Z6IamQ9QI3~v8#4HvlNZ9A_Elm&R! zy{jfm;B{(9e1ZkL$B;6y+G#`tCu`u%z?g(6)Hxo=sdr0vxZ2+hSWw_7cF^VX(GnS< zCijj|$0z0xF3ad)??O&$b}n{hn9SjNcytFeaCb|_H@V2kvhd(Bham@fLXP_@Dk31& ztF7Zko5apws{F48|V62vAFM0!7hNv zZtx$L0v6P~<24XF_*Yx;^CQdE!27BwhmAr?5ZUP<{7yvI`-o=Ub1b`pvO*wA7Tx@m z6Z&JQ%am~(`QxPwtijYM)-@m^@mM7SMMNy6_g));K04r|Kf21h(q|mHib=h0@};jbbI!ST@Y)b z*$0RRMe~)3(ATGKS%T_*@$PST69AW`%o}06!DEe`n>V1(ovJg@(WU%!=^gq@d!@gqVr z*{`h)XNy7WuozE;PhT=KPJeFWo-Xu>1j2!BbK3ca#2#BFsIVrD?XEpXnyc??ogCk? zcJdo(DWxHJp_9Y$@T3I((pjwiAV|j?c;PXnW$PT)SjU^{rqbZ~`P-d_c}q7}TcDH@ zSMchO(6>LJm>@3u!XsI`mGyn`KHHthuxEff9%VDia_=m$p1i4k=;rJbrfa`rDL> zQ`L9x)-`HZn{x1O>5mJ9$N4x*iMf5;<%4yuFCEOo17W=fP=Gv_u>uzy9Io=mR3~{h z$NwZ3J!yINQLmeRaxPD2s8Sv*iG*3mUt0U_NIYZ>je!M`E@Ud-B4`rh$zgkexle{CsS$X^Pn)iE}~~kKf<}N z0LM-WwX(N?h58DoS7B4>_gJ%;GP#j=W?HMdgR3>9~C_^Zr~41n;2WXvv5av z8(Jz$NE(Uh`9aG<1y%h_9Q0RM(+zGA9(wx`5gtC4_2}%~%M+8SQuzB>e|}rO4aWPP zRO__a8#+GS1>Zt6OqbOoENIoVCq|Cp-OEjEfuZchoQ-pX1@n26xruXPE#(hlN#1wfKKQxh(3J7bi^A!j56@7<4J}Nk1}bm(*qd&*`!xa=eadEjXD-QlKlboO zw&&Wpi?`PFu=c;Ca)1zw(bp5bix>^&F~0IMfs^#P-C9K21ymrR0fPPx!{tLoecDI$ zpGc2l{pU3Ao<9_09xkM{LDrh8w~$kA=^PqPeeiW>Pw=brcrAFpR_k4XXI-E?{ZZLm zc%c*E7NHEjjbI%&mZ*qX&`#>x$yHsPrTDWXzX&Dx8TkB3Wi#iOF@gH0i{s^4!V}+o z?mBu;t2AkF9;c)f!lcvdS)k4;Lj0D8D@0R$L7J=0!twK<7dJNomulJueh@E!^T#uk6 z9u0yX*}opzhm>lvS93UrDHGoMkzhe%bq#04afwbfZYZqqO*$=FCF^!BbzlUYq7D5y ze;o(}qe^0TAP;kNm~?}`j##rOh}09;pDLoC&NTZ}H1RA$5Cev z8~uHl${dVN?KfOH|FZt`1jvPo$%b9t96_U2wvb#vsNI9usXEj>?!gKp7I99!WbO8c ze3>Mdr)%g@Uq<>4J_=10x|{~nS_@@P-OBBIc3caVa0$pXXKLN;d6kXRA-Dt(CJW}3 zmicFq&{Im_xaz5%t~CH6YStM8%!=_fZwcP-f3&4UBZFa=aEeF#`qrGYr08S;N_ZjzvviwRj>>`^NXc$+iA`wmB?3*!mtphHkJIt? zSUMJ89In}d=b(QT=zQu|0O=-bN9PY*Li~9$@DAA}+=4(Ck0t-EwBJ=e;7xKG2CuK= zq%MY2*%YWcT98n*496z)&A58L$#8!tRKLpbtkU3I5B=!~d1;YTzw144foB9r%b#_D z5#9s0WmU7#a{`%Hmd29>r;G(P$=9k9IITG&RPQD>y|{J7W}6 zPb!_aJ`J=!N!IlWc(r?Lp_wv0tfMS~swn?R=vZwYp3oWGC|0E=E;7}rqRP?))pi16rV-A%5e@0D_Kp))&lL&m#3siy-4JQMo|bnwM1X>)EocC{ zw0-^bCpDdj6huYv(c7s{4Zu71q+weKv_$6eo{2tJl0h?@-X;z0Kf-^%Elh}Z3yu-G zDTtcb%Fk+_I9MyAC{vt)U*c}Tsqo~9qFXbq6tDaI{Tk(nYSjoleVL*J9)^3(I?7K6 zyCg-Iz#>Ve3Ab@TIFN%dyc%@Y%c`ZdCUS=9kEinqMP-==b?B{fPKa4`+N!3`1DTsH zuZ_}@N`s!bWUW$gFmw|leoXpb61R90I8|_!p%Ux4y)gWyAD zPP;>~2iMICYbJ~4mBuxV7LgBJnmvvBK5uAin+8i!a!6ZFRrku9?69-~^;~#nA{-)) zwbQ4&DkV`{Tl`M@6>q+8=9)|$-JCS)`lOaGBi~hRxT~_+`tn z**0Ab6IbNn-~x*AI(_5@s(7@l~r#EY8(@Mc>N{vt;oVXVTCoI>8qE@{8~%$_&OnM zOPiq&8899OiGtWsG#j1@?a;zdAiYv^1A?k)Emq{v#;MRf?J@Nm#SJRQ8FPmdo0KXg zV~R+rcXJ*`q;@!VLa;4Ha*C1k;%B+GkiM95y|1U&$hDJJ(e)IX38zOs22B|+!~55~ zrk#yhE6hE~vdy;$jDzgy{{eZw!B6(ZbG%K42%c# zQE35=rSV)HABS;ziiPT$l+{9=2nT~}7w6V@7252h#tSHul~%cGb6Yaz3RKBz zfEG`&FHgmf#ZWFyFmQdP*jE3py zvy^?=K$BV}uCw4u8&qI{&R10i!&E)Hg6+oBcbKg3Lz z0s`bjeE<5VmHD;geMPeKETMShjE2r|2j7O_0H&2AqelMN6p`@R)-J__S~yYZ#^7f3 z_p6}yh*Js4;x~!)Q#cW8jhY-ffplrlTrJBY5KWdlvdPt7xfr!CBj+m;BJ8e=>b$2h zn&4Cw*~F4(lw)HPXgJS@7eX4`i#co1Q#UBrq+@Q>-!$@yKzpVKvTD=ALcTcn#=9uz z4g%&^kg4GcQf1Yutt^wEtKs7pM_ZAhJx8Pl3}>Pzp^i=9|7-8s|Cw(8f1kVDl1g{Z z$*5>%O{EWVRw^Ne896KMj?5J1(6B_@mgqLqdCq4Vawx}gNalPNW(+wFlf#(V_kG{r z)aU*@?$3X4?}zu}dhGq)^SU2j9?efU57CGm z4GHq()aC;`E)!%!=uJ4pE!5koTcZ^B%p%>tUB&a@*x8;tgTn^DGd^6vm-X)q$T^NS z^SKAR)>SW7OzqU14mXTy=!rQ>yfnXXgwWOz31Bp5bYdXoHbrbs*`&^^Dpx~qKlJFT zn1W#_?-c+Tt>mbkb1NCIy!6qHKUjAW*MG_Ajr^7CFO$Nn98aZq`ObKc=lr_4RcNB` zAVj98XyuLiM`gnuHP|qC^So z`Dwn@O43Y!9Z{0py?f#UjTJmBDc0@78O@}@Zx%g4y(oFcBj3h9BoCf6Sc$ppZq}5u zT-&maT?}Jvp_AoEM?L%)+DEb|C%!wBvZrnwID9PP`|2t}jEFIto!Y#dsiviUUP6Z* zo)+r37w|7{3XD5JV^GK|SJ#-X&(`QyweF0DwCF-gRbHc)3cyE(Rue>>_E{7Si6fVt zZJ|avC+s@o?O4&4hSN(uyW9yMX zVXNSyyhs)|S-2PcGY`kJe$uRRz?vdT5IVv=^sA*+*|SfmsE)!i3_0?U5l-5Y*47;> z6}~YWKlJ)C7QTmjB-^zqJ56{&0Lh{HZfl6KpK9+`@O)&pmjeJq6YS3FB1wzz644ch zGxncMz8SBs={k=OBDx=T>FHju}@B2{sO&6w@hZKq?C;rp%Lk~ z&VDI_H7zH-m{IyPeFfY)w$nU8_0zOnM^Dl#yf`iK(MkZv+cNQ&+z`()b~b zR0ckr{KdVJh3iYAt`NZbToudU%`l4+_R}Dh=P+Ny<_f+^QR$2*FTeERY>y39<4K-V z2_?orDy$FtndzX59`smXBBLL8eCtnQ(;xWFCE*Ke+h2hBKvoo9a)lGiI`fY{+%0n5_%5W18wl@9H_TiKk0M&Wtx?Z*k`|! zz0-98dx`s`x_7B6-2z{U?(RmzT6WQ`Dt@MYfW;+1K^5dw|NASaChXJ133SS9tKsCK zthgXu$t(|@WHz^7`0N|e(n;tSj71<{l!q^}qJnPBR-_caINQGX5{6T%QLtHKnPh_%G?OCk-#nX+i3grV^%s(@iX81#$W38+21obNwb-k^kgZvm;fC9dw zaxJ{^aXZGOrMI(L&>T8ieAPI`{8$PW7s)8$>i6qxsX?3&n~uw2oy>Zopr_TZ3S=MJ zMb?EozcU10J|tN~x2P3DimSW7lxa>Gz3qer=2?o~GpztyPJ_4OFFl#vK=Ftl{JZ{k zLg2Lsn{;<#;pG~btQ~pU{eOHdb26`!m zPN8b>uf>UU|WT2bUevn>k(e$LQ?FSqlPK= zwN+rxdo@xUG+5~H94c^EJKOJ^;ZHm>VZqU6LKL+pewNsyo5?C>K)LC-TTdm&33$(TI8Gey+ zA8&P=S6>r30K5FgVJYn`8QK@pLT1ej)sQ+FcRL#EXC{s-8!BE*?@V}g7oOQ|v(0!7 z85WzW({0DuMGb#13)b@~nIYlMY~gke6hX>~%a-3;;pvbR0xN#RV~P*_rgmEXt{KWp zZrHSwn_uk_Yvp^}X3CltFRhgtKd<3m%+U(gH&8K9w`a*Y_bkx|+KESQLPp(h9~#_% z7rhT1W-47fSWD^BFh2XMlW6Z%oh-hp*in80#dqrCk}Z}hd{r0mI$FcO1jEZdoHi!` zGcL113Ijz zH3L6@J31y#{VDC%I|r#&vsEjq=~;GHlM-mt{RE~DrHiT$!L4vdOA?U7Ibm>KZ{OXO zcw3ZBrNOjaxXgRzcr{HE3z89UTBq)zX2dC?UaBt1F$4*^&l0RakF25$a~|ghz6(Pk z#c0`&epTcsNFm*1Q7`O@HlM~}$rW12&t~eU(@eqQxO|#513*i@DTuDfLu<6JzcK5;An#s#aIoMP)0682i+db zvWW|0aTSFnibLuCQg{brE0!!nSF}%CHs|^W?AG|(!Hu8~8*V+esA(=O?QM)P=Nz#< z^c`k)>ORwwB`M7uD~k&14g#I=Ew0NxX}~SFE_GKbP!0M}n#`Q<&5Jpm!(Ly^ygpkI z8O20SO4d95GPz;XHk&-@VpIGzks%;eEh7S%H|UfN>Ik|kLylr`8dsL`37BGaHQFk73fTypJ2lTQOP zT1e0f!*yhC*x+Ttg8i2~24AjL3@luvTLLjS5F8D(;$oQNxbs%L{{pD~zpHvcK;&7S z0s-lBu$IZpZZ_9i^iPgeHLEJ_H8Iz6UD1@MT(RWqTgfOT{>)dq%9)9#?0nf}V*CEtghpa?Z}&(vVB#M*k7Tq~e>H+yI|C+Hu!mMFXQ;>s0H}&M%M%=uFOW= zBp`#)7y*Y?n>kv3fqgLs-(rK2Us=3IK?YQ`AZHw@c86-|7#rp4cs~g>UN(!mGr95%zr?6DX>!q4n0(xq9Y? z1Xm*4OhO9MOqjc@=@)<`Ef! zXXZJTZ7Xn-%rTb$Wn26M3_S~-%U)+hx;XuQ3c-^5w4XpX9_m#$vW2+M`YHkWB7a>4 zK>JGf@i3QY?J+r)lTFlzR0JVGCIS-dgrj)HdnGVd?fs%+t;4fw&}maRG?*^D6#Zc3 zZ}dW<*y4MdzKVdq9?Y{n`;K*!dk8;5dvVHRjFmwMN(Mg6yYkz_TtnquadaYGdD8^)}PMHY;dmBELk65sb5dUKy#6>02lTLv*T=)8_shHUda^W!vdB;LF}jsWK2EVZK}VAm|9b%Oe*} z$xUZ>4FaqtXXi5je}rbr%82I*I5}zGd1GO1W3X?%s|G_4TB&O+ruxWJ{7#_Pb|yHg+}#pwsD1ZZ5ibt*7`XW&QPh#^jWX z(Hm6mp?-_cS6HxudLy{exKtYC}{>9L%bJqS+pIIo|`&Hd*kGuYX}@Hl&l|IvK$ zrtq2Nodj^u8bSIV7{3|FqVvNQ)D(pjQO0K$3IIPw0TY((XxJva8}DXoT0#CI#(KKD+59E= zRDds^VxP+{J+Zm7b!&98hg3}J65RvrKr~_)l}fLhns7;8_{$G}7)JmBo||-RS?&RT zYOq?eT>{VSc71SP@1)>u`Aa7gv^S!^J({))*bPHOoU~o*_~VZQg~ULWkH@M0sAKg+ zaX5r$fkABcAVhZEMFLxQ++Atz@ch4P|pLf3hf3A!BGXMYp literal 0 HcmV?d00001 diff --git a/docs/img/structured-streaming-late-data.png b/docs/img/structured-streaming-late-data.png new file mode 100644 index 0000000000000000000000000000000000000000..5276b4786826236eb9ce043c9373870350aea7ff GIT binary patch literal 138931 zcmeFZcR1T`-#^@{R#B81t=XkE9YPy|+Iz*GEw%TE5kyr>QPi$Y39*veTdJzs*du}% zMa_u4WBh!t>wDezb6@TK`|mlP<9Lqi4{`VqpYuH5>-BoQ-=9QiYbw!Du~MBocaBCy zSzh9 zkvXR#|5VTK{K^DHAfv)*$A%>XREh8QZ3TyqZzD&N&hh)SqiF*ChdL&inBFz-VXWOo z0+jBS=e;D0R2CsqrEtjQK3#M6oc0`T9G_d&70&j5GC8rjveFf>7t`6*{qDy3^W;}$ z&i&`*KWFeCKlo1={3jLsClmfB6aJkE-|FAQujIwqUa$rU>koF$uCP~%gR5cX!%?I= z1`h`AXg?v}7c61N$JdI4F=&R{nNS}$cGU5f6uicdh|JM@cpFH zf84ema%8CwuQCdQ)0h4Ccb+FNe*KxnR%qyPW%s2@G)W-Y zz<#g9O@oiw>$;;sWvFoq#MX;W+?3ht5}bX#z*zCoT((oqx^|_)(;woOKq5O1{{CN~ zSp!s@osj+=0Ut^mLqi{qDvt@A?W2mWO;riY^%ElkgPL(@FSx(gy38)8DLV~|w619m zG_*_WrgDzmdLV>%mM}*gwKUjb)Jn_W-W|oy3tY4t z4#sw0$25cN(}SK3e<~j(jSXv)ux?gRhrw5ayxA1cbPz{VH|P{osugCOoVBZ0TC!*o zX2&OW96f4o{6K8AaW=F~x$=T&5Jm&z=9`oof-hXCWSTNBB1U&1`*x`VO04wYfG z&CwJ5Uv)0-H3{Olc@|H;i{VudCX4*y-#M~mSx0kbr#bsf_}earZ_$+x!v$@2q$^W< zn%}HWtqqN?8*xpbrU&LeOrS_Vwc(9=nG9`r@&n|Sm~GO`8Po0!LoacOSo-!(@!*fQ z5%{4-@f5<2a?%ns$-JxlO>B4C@;$Vc5=Y_U#2VDBXtQ_k$C5C1slAVA6Qorvg0b!ZLB+mv`;MTBeNu3}5&WatOdt^l&F-VHQ&v+G#k)mJgg_daP866Uib~mF`d` zLSB$bb&~mS7&TvPqDx=akRkpS2IoRN>D|%fXcZbb7<+$IX@eVJ#)<3@i%gAlE85ts zVP8{PNbk(kq{gkE)XZ!}2J?5Twky5LV|X_j_!`$1pH=2nR5_A=3edMJ+&NGPt$o;lqpv7 z)eptmXh4EblFi*wmuUHRHmECi(`J|H=K9tAPo}6Kfyf6=9|I;K&==8^(kcE7HkCif#3qBu>5jNeo%Qpc@AS!D+1_F6ENTbtOz&Ol!UT?4V~)TV22> zhWyCxeo^bEjgcVi$>!!;f@C`taqBh@hVapQ^<-biqkbkr$~d*gj?a8DW%)H_r8KgE zcP%#(J}C!Vpmhe{aMR}`Y+t`VM?W$LDqd-{KXq*HWGW0`3pyzzZSfL&PgMuK-mDr- z+8_&Kj=?&oZ-TOk*Pt!B7LBy8GuSIBm+$HZD{bWTd^jAF*4XHu2@pdMwA4)|Jh`~n zb9oLtAJCC>kA9B#C?gHo@hJwVm5Rz1u+vQ3b0Awzc;EZb(3b4q>&icq?_KSig^9E& z1Dfz}ZI^`U+8d7t97)>kZ=>fh)&lSY!&qbHTZr;)X1FoZA-Lx4`_ivSnG+z)C?fL~ zTjK6y#$^uSTR+sAhBiC`62A}G$vG@+Jjov^$lygrNysdXSDPbfQO)B{=$s1CPQFD! zwwYRVl?xoJ;7_Oj&W}#PBLO1Z)apF$+U?f#MGcs%W zssmEe-0o*8e2$(7n;?+tBq0+I+c@$z*@**-9wmLEuIan&J6UIIn39O#58oU`EGkkb zJz3nZo5F7l^SC!5-3FvPF<06{^fhlbW0F3Q-T5CC_8%v~$DcDiu3tPB8`(lL)URhM zXXo03tI~3!9;ToSP_}5@R*J6TRGRt}5$MCaX39udmXr(hVZrB%0yReEZcBcA65yX% z_(?E!SD@}&ivrk~xzo2`Wo-W*uZ^|;(413cw*EVXND2~@(wwxB$NtlcgA3ir8&VTb zRSTu!N95|D!&t^Zj8}qH)QH!Pv@8vxkslpR=!UIp<#Il(bQ_m~dx2e;s2h?apjzJ` zYj;1f4|;E8^BMfOZuz42`RCr0$#6twwRdmHxfbg$!LYD&?<^ z$unt&tSp2>T|Q5H>ExX<03Hi#3qHsM@8>lgw3(f7m9%=B%LGyL9UK5fbu6IhXw(6O$hJu7lowYGWlpmaa$ zgGCa;u0YX+j{@i5Z(L)X^1+ij!N_LZ#KxR;2?VDS1~-@iYJApz6x7zVA7|;E9HNG< zE=?|#<;@C*Z~77aAdX2U@dMWq>tJv$T|ou6W`3`;jgD{39T*o3((9V%C>9&OKfLw3 zJ(KirHa_boTHLUm(EG-LOu-vc4mrvS!w3bhhLl7?8FCfxLSeI zdVMEBvUNdycnhPQLbI&gulJmdNbPWZl(rijNLCc-Iv(&tMDm}Xt7Xy2j4)9~>2ih_AoRF#D5|Wf91f;W+ zy)L0h((2pYaq9kdX-#W4;NiYmT$Z8y6AbkDHulaZk;%imvz3C{Vw@$lFQuq2|ArmU zTwv~2j^fXGM{*zV1DC>YQ-**v>$MhJGQ-R=|@=F|Lj_`6n%lc+d#;`~`@sAPxjl zMVSLpZS8=sgp-v#Ke?Oci`Gp$dP+TL%+|Nqu2uVIJ{eh-1$z@t&?e(Ib?R%4y|TSG zRw2?FXz_s+;fYgU$Ge@rP|&T%WxK>&T&RAwskcRKjfRf$Za~GNJY14*>!(v1(Ba zNidR7soBA^U0GVSS%E&hd}i<lNM)wIRf*b z7gL4Tu8Cr9>MZnsdYO7?Q*Wwlh4pwg~wjDtpKaYDC`5T>!?X&;MaU|L6>64TeQ|b%DK| zTJK7-zu2wa^q~%|jTYb9Tz9t-L~VT~(06Ynl?g9zzf^RR!f3I+KEaT(f)=Mgh6=+(G4rPoQcB9net$b#lpuoy8 z1AJlTzIJh{U&z(&_VUu^1~q?FI_;W6J8cNTb=){)q|W0}eU_RjCUR?S(vJ!&>qz>@eZj5Ud~PdjWE7Gcg& zJ^9ATPkPKPc)G@OqpzO9BkMAD-6siZuG)DMa~y$C``CFHfoZ`d+p;b`cR~wX@18N_ z@Ri=~#2uH_X$o`(&!!;UO;ZV}E2)H9WX8#BCn*H$^L zY3A0m-W4m)8O1!bMCU|y4Q;&$9&XO@x;s-Th!~M9TO~WpxcNUdhQG0(@=vYm`9mMs zUj*Dxq-z|}7khoYJ+n8O>bgTs?2ORZNx}?^s#kDb1!8*7g=kr3IWJu_OfV46+;l9A zNivs6Q+b^?qkApRPtc6$~THWIs(by{3~Qp8|1g zo!KCGi%b*{q$CBp5OF(Wg^pr6tiD=-kfv0zgYd_MQL}|Qizl%^3_2pMQUp<`ujz!+ zW(&H(7WL?u#~r6D+_jF~@KpXm5@rw1UYU!Tm7Q3)a6t#9)fez7w(%5F6@L(x-^>%# z%M7y~=)6_p>F>s3P~NrkL%q+=*w1lSPwD)bele?G*z;Qrde3+sAPUQQ*Nfr1Zlk-x z9@KSj!NJ1??f_>VRdLKWAOGT<%4*l5KdwaDU32jXrA;$k`JBT-kyh5X!DA2Z1=?sjx~J^i6h6H(&b5;7*HO^PjkzFXN-t-D~Bj}vam+YdKsq9EsDA#TC=D>2hR z|9kw|DU7GOlLA=!?eHu*U31r7Q3(}?`PZG`IRTbf`3xR24l;aRvvA0^O^~|0IfMD| zR+rc!kj^mE2MfXF@SNsoM!iuLH`wcNRi*aZ+mbFYYH20uQ^rcL$z4xl2uk^dUIZpv*RhOVr~B`5=F*O zn3}%t zA1;Vm%|-VG!BBJnlHrnFNk2plee5iEE@X%~r;WQBB~mT-@3`?F!ePimDtrF&-Lx1H zxlG>9k3VDwW2?0fdsNe}X+Q^VzLar1gPrjvxP-AkuU+O9`H=rZi%g-CD&avsxXseA zpbkZMl&Z8{-e@=+ZYTX`8v4(FrtLyr(cRPgpwqBvN(i%R=;6|<*RYmoyL2W@9uIt? zIZv)@DU(_Crwa7XwY!rot3Q#WDGIGdN6*-|*GG0T;T`0o(q)|mh+Jt-K8C@S=$*8) z!^3Ky>7b5&{Y4M|yq>T3-rFR@UK(=&5I)`Djwnt7f4IZ4dulPxnv^Q^R!B+r4;1pZ zbnw4g>HojK;1hoDvfHm*8Nz3wQOpwlay72ft84Pz9co&Ib0LgZm9;{E+82K-w0}vM z@%P@_4jY!s4QR@I6Z=v+NMTD;!vAJ9R+B$qt=I&Hi9SabL63<0;QamK9~ADKNUHpx z=fh3OqXX)d(fz~I0+kkr+-tO|Q;(-swA!agY28t~`vU_->MnW(EttJ2fbD2_x{v_z zgE9%N_eA`|99QwtZMe1N0rjE<@OIlu+c#S;QThC~4cl}Knla)~8x^uI;lj%8HF3uL!3iVUE(Y7Mp2L_l2T zCX=iY?~A@|C_&T>;hAN12IUDrOGo>$vIXoDPaDb1J8j*-;=H+_x%yC`>HM-J4eJMh zu77oBOsA+RyyL?w%=l9qPSk!JZRr3wl>1QXE#*U%2#EP|T`8p5smM^HmVraqCRg)U zvyQPeSB9E#C-eR2y?q%e$=g49eoB4*W717B%PA-j&|PQ~+F;%SSB^bljriD9z%P}* z2(gn%BCw z#+J2p8lr(6I=~Jq-*=qi%g^&SrF7BkCyEDCC>%>eP(OLn+$FeEp`^KzW)fYolI&_I z=2Cx$v$Kr=SYJTv_G&cd=skDP&orcl zoCO2;ZkIlh=aNF$P=)`XnOxg5^WW7qz9i5)yS>i^Dv(wF9`v5->QL@$nejbE1McHM zcR+NW^XhtC?B2ua;LC}(bGR;D{ZpqFV=6OfSM8p8B|E)3a(;YS@NMG*K-$)8wLIDw z{mADMreqyO8OTS4{xzwWQslXYZh1!PilkfJV?NDknlb>EmAQtK3P+msTXq1@4h`;I zB!)Qrnr~3lE`%N8iqp|&G&}jSh=jU=tIFB45bP@TbXLt=LB(nJqKGyFt#+2{1@?m% zxe`7t$aSBQn}kd$F(;E`^{F97$a4Nl!BSCLI6L~mFUYuvb@RO+4dNe*-cDMp90$HF zxRU=g=i%l1U*HeTE@FO-U)xrv8rjpg_^t@ zh1G7RMpowJ?)qvwXSj&*eVPLC|J@5IyPaKPXcba2c(1Z-N82ht5(+li(+{fF!j7`* z#U@luw<$aM)zOO~h&xsk^-X!Ab_h=C1Goo%D92Jw=wLQ!)d0>mU&Bq1 zdb3r?=t(<>ziXAJb}PHF|`RX8e@(*lUX%tJps z&ybvkD!Q?!KPPKm7eklfaEcJcc zEqbO?_2@B>UN%Hcf3k1jMY?&xv6q)DP;K1)0@gtQek!(oJsUU0cZ!t4M~M}A9Pt96 znL@LJhv`MU79DzK#AiUemd$31Iqytvr=-0n%X*$pZ7YQ?bahHDTxFt-^;>T)5m9B{ z=H3}jC3I}82!wE^RsZF8cgHDg8n{BK%Qhjy)y_qStGPr}39L=87PM2B5(f3-P!J9h zT=HH?Ze8f}+DS07bMfVHrk9%rc^5E5Q)zy%NzN+U%fcMOH&Z|Ql;O$A#$r2E1e(sS(k`X}lJR4!a*YCkQg`=s zGEmzyi5@vqQ%E3OIrZEx_#QsHEQSpbDP7jIx>nN90&~DrX!uW-&NA~)g&uwg;!5mr z2Ayi+h#Y`>bcUmo;2XdfX)ae(ui?ORNsAL8l(2hZAWmj0T|tMcrAUr^ zmg2o}b`9%7uR3~xz1ORBuriC%*=?W(Q-F{k>AbCBeo>yf&0|fQKymvi1`t;vK};4$ zbb{BLq1TfhGIOA2y)J89mc6%^EtP<`{O2w?@3XTVg#~KRZJ8 z+tI;Qjh3?vQE_>3EAsd3aMpNsJ`Z?jf!u%9Gs|kAMh@7Q%89N&gshBm`xwO0xOP)A zLAdnq<5Az2L+Z@&Q-ddJp?idCT*$|;Vue!l~Pjv~%*Ulolx_do*{GB|-DUH%i@6G!T<}rSSnCoYHu(DNy>$R#=opu3far{U7 z@T|Joo%=?a0MyR#N$E_c(3t+GtBpPW=q#yum$$RQ7#_Iv9n9guRSLfF+uCOWfZ*-l zyy$EqK(Fd0tEpkp71!2-7$)=NiY!ZT!9rZ@nUSZ`&5*Oo zd-rNW>F*UT|!%oGcaw2CIkZhDD>88#m504n?J;tn)nT(USBo& zV6~ol)MQVd#(b|Roh|Jy4E5Xh5(+It6>Q!imBiR!a`GA=``sBNXBSm+Ub(qzw@Ts$_n_xslf-?d4(K zGi1|K?g#R^L3_ewQZFvL^F89A_`{@2Ii)>k**jZX3PQ@39pb^BGN=bT%M0j>HPqx< zI(PzgW-~=Hr325fNYw|70KbAj5<11Agp02xXL$c2GES9tCW1Xtu`U#5 z=MQ~Js(tsz#PZ_&@l1=qxrR7g@h5TUEvXpFy}MdMf2*}nziquAIw9s~Y7^Xo9yRBm zoi5fom9oizs@$VC2Vs8_fnZf?B-qx&;CV+(*DiJg_kEk9ouCvZ-HoHmms>HDto`f~uKhaCzqz=vKA6Mu zM@pKpiUd6$E8E_Wh=FAC;#pj`0^DiV4}d8t-^{7e@{}XsPx)UhOGtnGJsE9^y^?tPHoO zjS$e#Z@LH9e=PAdn+~2CetcQ)-dPY8-B0`jb5vG14;XJyoTe?ti!D1J@GvbS;%-xv zgtnF8H{}aPXE%maI#9Z^E7=$=$>}W&zx}hdLQ#dNJc1E!e--eV@WA=|1Cj;qbF|}) z>tZ@&J_y>h>8^qoZ`8{Q0ItK0E7<94*Zrq043rU(dwH(!(4JJl9Zlvm-)6e|ggmD) z*~~kr*Rq_ByQtM7|FN}U#mFb@`9;&-fk>nw{sWzG+j;$sH2GrN+tOUOGsA-}>xrEY zvi&1;kLSy|^a1AhmaF93$wS{7#PnlJol(6&Z8>i@>=G#HS2KHM)HR*k`pzGgeRDQl zuL?_RXjP>QRCj^L+#UEm8^iXi3^>dR`w@Mfr4Q0A>$;=AQ?iM!C{73K< z_e|Q|nMvOGs96;s1}Ux?UfQz)gd~HCT>CvapJ@PmIbqth$Z*?h)Lda{4o|55(MBk} z^ixL8>-CEt6Yy_CAV^rOULlId>G)mg`#5$RP$Q5vtO%Co-)-9QO`MG=ts`-Aa__~o z_taOWPdYR%xT1IIFT9)ouw_)Xm98$7UCpy=;dE08dpOYQA!GCf#ff3zM)(0^T>R`(N@SCr1c7^ zaz?P0xT7;259wpwRz(%r~S)IyEUrGpo-_9$My=UGk9qhHx|{Fl->B5ICOhAchmk z+zl%5OPhL2&0#6muhBnj;ox~3BQh=qj@_7~h^XVb!}#DyV~n_|0Nci>N`=&yOq<+M z(sWNj{lY!_DQne$=GA8FFK{oslKrp5-V>H8moQdsuD;v%i9x&RNaM=+2Dj=pd-U-G zXSJ^Ym9Ne8a&b{S8uJTs8;>@qLNEf_d7I49@qg&l7oBx&XO#4>;clKHS?znfhoIb! z03>`eWukhb?2Ts;Cl?q3#yja%FX5;(tzW)ko3 zZR?O=+If4gKKlU7rjU}FxLt5n+1Iz{up9E&2TCL<-AThD-1FBx`dSM>*$%1V*)W^5 z3Fhx8NiN9h0Y52ybXj!t#qQfCMZ!@%v?VGlyEG`JxU`qBfpni6I0;DfdC9t1nbdor zx<|S#MEHm!5$a60p~wPG38O7b4Z0Mi+p8C_+^eJqW(V=j<~G4kH`p=@b7xGE*9sE7 zCK200zg8y$Vwa%JRYT5AF{UV7cNHpSg+d?}7}-d^kmOL~@p`hhfoG~;LrQfC)pvrb z4nh_H2*r|gL*<72v0RMdiwE4T9UYrzGJcT8OtZKiT9tHgRcmw{@^)|E2+8g6S36zg zk802l#zkwf0|7u)$>2{907g@H_6&wMK)B7i4CIbZR3BO3%{!vp9``x7D_36g9xrjc zk(HgN(3+ZD4`wNq8?k zU64>OitH(zO|Hw^Mu@G&w_=}k)J5{%k?&FtPak-Tyb2?l}$A&su0(ZyZ-wrAH z@j;%4x^siU$tAcQy_{e@+#rO;GE%Ri#a7zRDe)a`pw1FZ-)u_uj;Z$%YkPRh@M5-L z^rTMO-1Mf(*PtBFZ$Z*w0+toEo$KKluV{-v*j9H9hlNhj!DoVO2y3Z$(nNy=%a3tL z%~1&s8T!C5wr%*$(sx4PUdq5~v8YHrV*AV@s=elRd5x>$KemGJ06bOj75Z|K0Cnw_oqv*CSLpTi(BIB>@;lnxQn4L%YwMU2Dfe zwAiM`tHGFU?tupAx^*nB;a2%C9UX@W1n17kw!R2P>$uIkfgDr7m$6*p5g6QTk=t;R zQPh7aB#7yFKMrU-bfYRAgO$ZSdj~vk+o5~!$q*Ki>4BV<Pf)#h>YNw~_jS*>p@g zQausBZ|6_6;y>4Z!%qJ>lK(f4J@uM`ct^t^J=+ zhqqA%7Bi8XN09`gh>_1>r7W$y1&e#)sjx6qM7ZEFM7XJh@#f2}^6}zHCA(=uh*=1Rmejs)-sJn*W zi8=Wojq^eiJT7etiiBt^KTc1(@i=`C=l1hr?B9JcZA$Wo&Kf#=H5`t~ir0+dlok^p zkGgF+-I6C?uAStHCC%+5!Vag0sl+3QQ%P{;#0%%ik1h@zi&y?%A8C-eX>!(&|Cn@} zDb$ihpc+{>`mFGM%+XP51`>08;+DxA&ZU(Vg-_o9%Dl?*B1||Nr%p zLVi)MKo(PXm)@==_>y`4_okKkLih|Po?FuavB^OaK=`tP%~hG3)n{8$DI=)3{L@9w z{^&;K!kEalld5VbuTC=*z{%5P8+A=c+}B+1tYT#7WFpdd#y)xE&CSi}dtEF(^OAp@ zPtom9&zC$+5AfMFDpR0Y%7a5#6?8xpUuR__Ihy~g+^L|qM*k51Y&J3&TmjQNlQeYY zaugl)LQVe8hcY(IBZzN14`z8rJKE6IcHu$xDcRzE$ofUmsoN+^bgOFkDiPromV8U!qW6U0Ml#Mc=-?O?(Jz(<{fm!;d?F(-3)U!b!5p@-U~R)97LJXu z-B;MP@m%@CBaL+DUAf-Tnb?*JY_O)t*i_YC*6_xvQ<*E?C9)=I_v;*h_>JV!Dv>=} z5NwzVhZ~1x0!>+he9NXFqygI6rCLd=ElWSVTJ4@&wyEMV6wWC}_LnP#e5Jnsg?ppm z*S{XkxkV<@d=O#&;MY^OgNf_-I)^-$0j@;Y^Nskq;k7zy=W^Js{RySskr5HgaxN+V z9^2@HzWL+I@0#exook;M@Mb5czCG0z9hfGMY~!8)IhR#b_E<62X^8UC$mZ5oRE)~d zlcL2nl&)OkEN$wuy)Fik=?Smaf*6 zl}%JHn0C(6R%3(Jt35ih+)6}exOI&-w*1mbzXU9dn>Q6zyQKbc7Wq_+r~Y=jS_leR#-g3L#`RQ3dgGAMul+OgW9i!S{mCBufrT+9XIaJ>J3;$+vW4W3L9AU6p4^qsd{lL%Xd7DX zWdGlDQCOLrACry7nJ%@vj)!{!Yxn~Fk|3vM&vALvljG}7UeiUxC*Ps~H@UmjaI0xs zE43jhV-=|23S(bso6Ob5$-Vub(P~O3_ED2ut~3EF&m;)pvNs?EzecvbvC;7c?nk0f zILRrNoJ)5*Fri>FGjOTfz$@xoa(_tN%8uiC^U#OWN$5a!Yv$CJvshlR?5@Wb?kX&3b4@>7L!vi>k1z?AO_xfKuf-Qok z@M}OPvP>aYxe{sCk{jX!=ZcG7tz( zwResFU?P-mY#tP>NqvYk$v3!9&B}pR$lUfgVr}_9E^>*!(sXYH2hI7 z(PuU|5qAE2_%=k9SG&;1h1%sP8*xw(eEm^t&yxfDwUjrm4WH;*_Qx(cs8XQaw{JBQ z5re3zT%Dt_K8`hCO?03@TTT5I%g7VMaa-wkS@GS6J4tQRu!)Vy z)mk_JoL--@0>S7m$*l_0CQa6}XFj}6c9En!Uo#FLj?`R{W~W-I^>;fX z7y$RYQep6$MI7z0*)mGX-_(7>FXmrd;4Amlu~CNa&eTC`?zem=Uwyu}=8@WB+U>t+ z>;(H2QnTZ5p~)p(bdG3M=CwF%=1{S=+CI?8(tXJF%qts;H3b)RUo2$Cqh<4I7C>7K zfWAGrK_R<~7pH&A$Vu_ORUYiI=o|*`(obQ@EO26Qh+Gq|HoX}~5cRkdGG67}nmcUJ zT~;~VIy&7dY4^p_i zbxmfQ^U}pd;8zh>&OpzXQ#jkfz&|(wb^#V0eHwVn>J_*jSbD{@j9WcPi}H zlHmx}@x$ZY^_KR5EwX<#9+V3{rxjH;t+2fo^pPe_=FS6E#=l3GX-a-!^DkdrdsA?c zw#=f?r{&HA*2Rvy!oN;-YWJ@1PZa>_cMA&`)Jlxo8YJUw91eRDQC#2k=H3@_&A*e_ zl%5;|R0F>F6CR!PF4y3{892UfBq)zo4Fp4L0}rBcdp${qFk@d*P0h|yN16nzJr+IE zv@lsQrCImAK@t{KExG&NO0r}p2wOHftH@oy<7;A%W=j&?D|%jIkY{dE1TKXw5KAri zq#yDpJ3##7Y$=Rk-Ten@N*oC_?OE4~)vAkl$okJFc^IM@9R3Lc5VmCLV%3h%3_2h3 z>of-+6Scg+VB&DIQ@QT3Ves4Pfxc4?>cZPzP8Fg9N7#^R(@5@N>RL|qu#=nSIhPJx zf(mRdYjUk-v^wy2Zc$TA5Fs_={%s2Fk-rMmNwOTAt<$Tm=?gdLCSiEUY4w5ysn2xQ zb_xU~UyD3A;1-{>E946P@FuxK!!-GL6tpSb*97w zc#0Mt^aN>E%aNC=-Ai4Q!ZVp*VTxAG?OS5IqejE(Ms@PJAYm6v_u2#1iW?1hmGYQ^ieLSU;X4~2> zD42JspFR7eyl^F>q(U<0$fxR9%%LG@edbt)iSHA_$$sX_ zaf2Ib!l&ZLpslGShwziB*qs9U0K!tEtk!h1&jM=pQG@+gap}>-8(u40IL;kVPnnX} z;YK)~NC&b*NqUr=wMLX!_9mPCLc{R&c`*G3P4spK{K;oLlQ4{s{(y}hasB+>^ZCPS zb3vnVgcT;P*v_RivR+yFVo0Ds3$H2vPy{eB>Mxp?1nxGt68yv99jlw^_C$wCc2J;h zZ?i$3|AYzu(qT_ScRdf21kOj?d^y^=sMfr7yUexF)a&aTR*hpc`V!D;_zYdDEu7Sj zN00X{TAb-}(J(|j|MQ+ZB2aw?6xaESKvOb^*Bt}LTGu7~^g~kX0#!`o$*Ye&$)7#C zMKj$Kj-jKQZoxHZz{N7Q9}JfTI9gV_neEW!JaTakmmUD}-O%_AU8`x{5sQ$q)0q+!>DFYEdKHGWOvkU9S6led@YjSij;pZNRVNVR1X%JbPM zqLz(M#&<4+1=&ZX2-kMXp4jgQyi9+^uf-u6Cu?D0(XXdJJ<#si;J0tDO zIA3YaJ-YD4lnC6^fqZ`t^bFo*+Pk`ZNK4bb^SDm)Q7!1ckny@y)*bC>vIDVB0l6^N zkum3%LV+eT{pNE~;f~Hxmt;k;A92yK#mm*XN0n!zuQ(Yvjlm`laq;HQJL^T4k`Uqn zz8cC6r{c##;ON$5bU$}s#m#DFSuNY}DAMYKxnc<+twDTE<8ENdM6p7I?l4_yR`Fu^ z{pl7oGpq~(4y*rJz>r>Ay=oS&`Ha7ufv4yd+b}^1%qBWl)>3EuC|W&2tS$x--D&WF z!02(u=t#CwF5nE8mE6-1sA(*Ky{`6YP_I5wv;RI=Q^k72;>tva6IaM~RSEIuZsV_0 z(qM!`^sdoN!Yhr-dTb#x!AS<`j$oa?iNyM~5Bs*s%m5Wbr=)^VWTy2JVEmmR6B{gb zz0f=WpSQbILKmKSPq6N_5FlyjWBatk6*IdiQ{_oGHH#Icabyh|o5>DfYkos@0{JwC3ZNtpvcywD<>olLD_qWvuyITDj zW9g!XwQO;N+Jha$(xp36y(g`gULSd!M8l3pWirL%6dxXBhPqrni;#x0y`F>u=^xA9 zf+=8wsDu}cw2no&4)*S4hbfX87e0?ms{n@PWMw`p$#4jE?>veId8e^N1MA}$qeZRiy5AHkBSMAxa=l7;rfCOT3W&8)7C)EFGnI3y2q z3kmI^(yGx;*26+IR~Ry&SW0xq;=w6#*BXunyoY zJ9r80lli>i^p$v$->PCsU0D22&0LZwsH2F5rrgEU%?1>(@{{9{N^n)SSG&t@UzuU@ z_fVRDaYMjG#Shhba>FMkM~>((w8@#~8n=T25P3Db_edPB(R*VWPF~wocwR2l_am06 zax`a9zoQFwya5!f)z2yfpJQ88rt~g-T|npkb;z+E?lJ@o+sY0aRC_p^taL`Q?mI5X zoqZaD6l|DvZ)G(lf73N;!&f?lHtr}|3grBz`pOD*K8;?>nUTjnYAKG5^H|gz?GT#z z7KZH|Cr0S7}kEel<~jaAu9pW5@qTA>C|J28)Us3bI~ zQQ3jctz6Ol8MO`xio>5^Ag?@bAL1nZ>#Z}+@Vv~#J}AwE+pv(2@I);mKSSJmv?cTS zkn{`Dvs^HOVRvr<`R_q|j&vv#A?l?Oz(vE?;A&>I(&rRHx`$h+Yj-&3fveS#L&EjbQut5(>Ed4v5Km{2Y$6rbbZcx6(63h{7fG?3hC?ys{x@&g-AgDI z{Ir}BCblzPY<2+?BXdCM9Wa-A#AGgZ`VM{6)0N6^^o;oH1|<32e)-ZS7i_|6t{3f6 z51VFio78c;9A!B=Ft|x@#yje_O*9BOHQ;Ck(p*m0H31U?yqRv=DD};>&&wo)w9%lNk+k6QHT(i>RQKX4by7qrh4ZYNga?It~V{}pA;S%t*M4Pc)OhuOiFf> z8ILoa20EBpExB(D5X)A4H5wHLOIcIasV{0YZKqJ0&>k4)HKu97lg~-B1wE zlzU20=O3={#QqQx{@GGqS64klrXU`%%-bI6GOB<6GCldg%FoLIqe-{P2!-|1^P6Sg zhFt`kmu57XVFt*2oL||zy41{2rwfmU2ZWSEZ3WCuSN-`xE&v~pZAy_}aw-!N{N$>A; zcnT57sLx>UY%W62+4YUXu|A82!te`NTMXynSS)KSr2l6T!+9xJMJIoLyw zP_^4U$#V&1{c`7`I92Y`y`EveZ1d8M$;Pv1Xk8rutix^yYu2%Q8qpFb)5_sIFQ($@ zr|?`0W`$uV-h|!p=HXB7RW4PjcJ_yZLUdA}nVs_6E7lb8$w3oq*<#_`ZD@TvowVw{ z?=Mnn27_>JlQ%Y3DMp<}q0sG?{K(Y+qws;A=PrFi~E!-s?mJVOlPk(kNzx877{FGt2y;hmxQnn<(>4C-waF=5BoB&nl5SB(wFY1q(D?T99 zYHbF;nTXrOOxmwnhi{NZngOL7 zX%&zT=|;MS?nY$jM!G?|r6h-L7?1`5=^nbfd*&Ixzvq8l`_;UebJmXaiM7{hBJ(V| z^Stsk1t81ns3ehlJ8RS`opedQ-( z?bJrATOZtgSWc_>buaMylZjVhF0_f3*dDk|^k1q_R($)9N=cPuwc5>f#&m#R{qo3f z!tXBNt>bfLN@wJNr6_*~A@k%zRi6wr@F`^AZxes>?b_8FX&U ze7q?<(iC_I?rB=15^(_Mw^oiKK(b}ouM;QYVRH{7bM!oyT0X)ZyD?eHH+mm+Ah$wi zheF5M$?`~MY;5yNf}LnXysY1RFP4i_ifsj_wQb)rNp6OQt9h*u7~^33Fv%^fB0 zi}qz^9n9;B$l!W1;wPQsJBRB$@0ZThf-@dY2hfV?9I#M1wc-MPHw~Uyq9*ScPY8W< z+L`-0JS}&!{-xPm=;-nh5UR1#h6_k5B`^4(q5)~*e-t+; zq;7xaK7LhEaMpZ+ZcLGFIU?%(t;e=$y-L-wk)#>Elds`8Dv4V|_V^+8XhHvG*`lJt z)^h$)$zWMstV#b_zKLTFHt%<*YG+PQ-OP}O$VtW`0;Bs11eK?e2Tt^V04u#!ogTsqN& z8l6f4YH1k{MFiD$7bNTylUn zykbpRR>yc_%Km{RPxreoaIc&X{S)fvBg$-_-0w%-2a5TYa}Jf3ElbBg*^jR^d5`J) zNKKpHD~H~8bEFyE{UJK3aF$(8vSf>@tE!TD&3C!iLjR~gr?scwVmlIryZ--7Z3ZQi zt&!O#*W;x2afMKyTP|!Yws-RCTCEDr>_X1n)iPxs3$(0bGd)rZ3TC?t9bi|^UO56m z-$F~i+V34dLp8KZZ?p^AJt>pL^(hgdNtz2^9rM@o($crG+Qth>zhwG=oT4bs?Mj6I zs|i`QV)u;2s3iS4(y)T0K5K7>yMOarQ=bt!AK4DA^xeH2ad@*%rc(bNU6HcpB3*W{ zVFB_3JkIpP{+A!bUY)iFW}}p1OFRtShqp`fQ%7@LzZNek#@5_Y^j1%;5H9yjBM5tS zVrm9SDC|LUMw$N8M!c<2siNPQk70pn++3j^{aKYWx4ovT|AAD>7{?p2jrcMn>yRV% z3v8#c%(a`N&wS;5AUlhD_V;AoE6)KWi<&RxHTHwIYO2dSGQV3AKm5}vR@*!pb!Bo| z68WpC9C^4ro_>dFN{(($AM^ikGzstKnB%q%}gxpT= zXV^>PaXs1daXo9^sisQAQ{PaPw&4JBXaAn5H+dMhV_*{8KWWzk zs55yXU#@6#6BF2ni;z)Eu0M<#*fxduqidHsqLO^qkfpErtdR^8dh~F9N!by5M#wl~ zZZDT~RQeTKSHCCOP$}7O+T9}3MN4v^WSpJYxCfO171MToZk~07;Hmh)Ll7%;gV48H zI5u6c?2HT4WNX=Y)0^mud1RG2CTz53&GWl(I>J@J(w2Gg*I9kVwMb2_e}hIvD6A^x zqo+O6LHnL}6?|I!h+@u$*nh0yZL_G^Rj$vpJK1f-?Cd)ykC?D%pXs(%wNC*e4^e!@ z%i7G82cRV>z369bA`uj62)(no0L9ODurW8U?E(*5r1fYC=sN= z+^YOS&nY149Ww|6Y3d-O7wN1@Jj%i~>HvDwT)O{@&~*bc#U z`!lU>>z1kvQoWVR8Ex_>I}(G*9|+$_6R{YUpM9fbR91KpJ+fZ!@|l~yQ$f5Ru!{M9 zev4~)PM@zte&cRz9k+TQGi)KnG)d;q9_+@WsB(d{UVI5j8(cjHdaxDOPZq%Pz%sOT4Mt zkf2$I>FTzRU(Q%L&$rSsp*+@!KhWKY$kD{%`aB)`bKYxjXMs+Huh8o>1bKRX| z=S`R3V`i_P?Qn zE8*9lyt%aRuqo?#acGa*R4p!dHB;*dI8r#&TfqHAmR9dnbnh4L?Vx4Vz1e z;ab<&r=_x-ozdig05ysVew7bP>-v4-jfwXNpMc}A6}+*UdP-3PdTkhDoYs}yP#OR7;0|+BtPM210VN=aQ6qVS0|a# zDpw-@oh22m7uC!sIq|ak$pbi*?p6AS?=wtap;I_`*qiRk;H>I2Pf1pjbc^o%roFw1 zH*chxaW^U*?diOmdOBgM(EE0kmsw!%6*~G*hOl=tGMhrbw4z!ar9quu8nzm``Dj6? z9kv-I<8%8d#vQ!6sg`gD@)b8l0*OrM9gh3qHmI5h>O*Q@x8_7p`YkE}fDpFY#0ztU z^xFIx-m4yU&P-%zce^P1-Mr#?#tPsKL^%nY5vvsWnk^Ziq~6UHCB z`LZ(IrS&Ir$;(S$HLCCijjaW1R9VfS2lZiO7dxP_w5gDg;;Q+oOoAHNHQnHEvVZLd z+sAm|EOW0!zwg=c2>I!A);c#fN?cD&&e|peFOJImvy9D#ofiPS7hLHjTH2TsT@(Zc z=bxN{_ne9I)SgxUy5G(4_MyX9hGu8`*VhH@^iiGP?X({0@{3Qt>>LZCjF64!f7b3?gka=TynRqIv-cIzyiI47C6BwGCkje=N%uo3wY{<5 zQ~e2OA;Z1r?cidZzGkCZ@8c(pqsN#Tvx+m{k5i|Vu8}vZ6E7;kf#Qc)9oM_ew|^?A zJpn5%^7OPkjzz}0mAoWq*`P*8aO)xDY5%SLY+XhgqFqXw3WQ+r#ocu0hexp?-S~kz+lm_(sb<=z}RITA}V-_;o6pwD6-%Y2X_tuO1<|H~m z2%U?yH1H_f=b%b;-=!=&t@Kavh)(HAdF1p=>&I#n;!(ar$1T@8X50IO>H~T5)2YV% z5b5X@J~ifO`)($hF8~xfr^!Bd-vfW=ks20+d#?d*UpC-oR)wcl8Q17?{C6<7`Ri3x zNSAC|!*>R(qGCgf9EZJU)WSGL`_{c+hTO z5=hSnp8O_>s{XvfYZOqjp!GrZ190eXTql59bEWM3LPG>rv@fI9687i^SQS_?9^*Xo zAx@@T8G)Q~_cV6k>OAXSSoN>tAAxalXM0HBApm(h;%KV-?m4n6I121E(iZ+Q3DWJT zZjqS;`B!SZ#ud5QRF!Gr*|SS)1IG`*Q69>@TUYMSNWuH>~c0J{%CsSz*|o+WFY$ zICm1bb<&M&`YILY%STzN1CdOQ?ayVJplq_dHi+b0fzh} zO9iZFF`qr8H`0xr7u@YE;;fX?Qsvh_X(H77gHwJkVBbRV0RUa`n!uf)$C*3g+_Jq?HPgM7s+|+Yh6<@T~ZXP;G}V;rqQw{zX0_E`*v^#9_a@FkJoES{ z@yD#mEkk+fwzC9u8uQ^`OAT2EYUwT%y#OK9`73^xVZSmtIX}jP#7RNe|(N zt}Ql=cJbjBvsdYIH20X-C!jVPD>@{?bCrA%?I2tCw9PHEBAq*xI;*WzLe#W7u_uLU zO83vT=^U&sMv;8MYw+U$QA|!Eqv!b7$KPcB!k(0s6cTT;n&p#;$Q8Ji{-T;G$cvzn z2c(7F9O;@h@R27cX;fYCh!%A5ID028mlFe5%rNhB7Fh-vEN`srZ=$f+<;AK1+=|qU z!lf?HR%R3X7X2oZ;&+p}V?V!6BA)UGhRy4=N76%~+{}#fba8W_1>v7Qz}=gR)wOan zg9Q=H9~u9lrhP2j3Yv}S9)1H>nLE4r=ma1-mFyyEW$2Mc#Q@N`nXPFog$9!+1W3#c ze{x7NKsY&(jb8aPa~fS5gPm}wATwQsr<$fiPxbRPj_8<7 z7xshnw#X3bw7c(Hb(iq034w)Dv&wfmAIRA2*$G=qC?jK?- zw*In#WdZoD$VH<`J^q|x2|^4C839{UHh<*^yAq`Ul8{$OJ|sP|NLta<4v}ztsmqMPO|!S@dq$)H z^RCD99f=Bk8CCCo!tqaBFEi!y@MEVs>l2;J5rg`3A6-u4j#8?m4CGL8CLk@N9@6Q& ze$Z*W{{jWRLmzIt#sP1=M3wz#r zMp)7Xt<#G4FnW#yII8rK4@BL;0VvNRK`)$3nn6E>Tx+m6-y7c2R^(ssT&@n?(Hxb> zJtc}sXeXMGyGj=gi}DD*0lCSDOVb!9VVOR*e<>)b9PSGm>FL3+mGpFysYwty^x*N1|S^1Li==);5(J7xAEIw!4 z`bItf$m*0f$%JX0-KC7T6Y_!LPjAu5Y;<=f4kWVehhV41hR5gAwgXpz0#lKLguN)G zt?oV<*_}Zr76tjOo`u>>b1|ks)!X+$S*fX<&H>l0&-c>jFR`s2*H<4etB=%CRic-1z+Bbz| zG}zgw(67SRO%WFpkXo$651v_LqM7#Xk+MMO6Z)>#%(p_8_Bx8tZFikUIPZ888QNys zGtZxhqX&)IrQcBbZVzN)ps#g*Cc`>5l3~@lGx{ZD4|MEIJD7H%DV5V}<3@LRHPq&6 zLtIx)*ox)#`m%nARqCF5XN{? zd@FltnO*G<1Rb@zyO1xT)VVtlaf>KBsC^)GSD#Ko&q}!?8rJUFxk);zmpQ;F@(tgY zA5tp#-2DeL_NxnVib8ir28WdjeQmgp7y8(QG16wrnbnZ{; zSmsb{FITEO7bJF;1+hcnRl5C`1MT0~o2ourIynWX+n7K32+tw)75t;2gbN597w#-{c<>`dCC!u51~R_fWi&aG zBCpW5bYih9YRm1yB#ne-Ol~K0^!HF(&gd>W+vRl=s+g_>#f`DXkb_4z;P`n%iC;HBr_PoDRAl!>~^`E}_0$u>_R{ zoXBk-9~Mg1Xsc=dH%~eWpEl1t6`L}78PGnT@S;Umn?xSXN*^!iR3)peAL`?s-CvwF z?&V2-qH>J)N0M=Q{Guw7$&E@8B-wJNNibP5jjhBWjvS;E7_3N>N4X?}+=}8h8&g8$ z%13)=eA9bLdr7wqP6W#if}*7Z}7pBacH#(q1J{sgdn>~J2e(H=&b0uKRz zhg$`Z@%4XOYMtTXwT0e=?&)`do~nKMOmAw) zej(bPT&_+TTkK1}swChNUrD76dDP3B2w!(SBbpffK6u*Db++muertLoq1y6}O>yb_ zJx8&jz>oAFba{W^GFJZ2p+!`x$#E_sb4Wrx$sKN?*QnjLsQn$8~Qf+e)gPbnM9sI)f_gmH(C1rkUUJxC+GD@fsl}P^9cRbk!=mWp^_R9 zpS`@gA^p0om`H)ax#7h%fLfjdYsTiH-2*!H-RYjfOBU%cHD4pac&p!3JEWjI%jc^Z znftsf+WglPs?)_WZ&p!584p|x#y$mZlUf)?s%LKNv#4LEx7V5r)?u+ddg0dHl`?6X z&>;m74ME!tvWFu*^*E-YRyWUX7peXaSsY+mz+hRRFS>nPX7xTKoC<5M2l0Q0Ulg4^ zbzwKxaspcox1zlH5>YB6rKQ$#v1F)BJXdkrwbDA>JiqpgTps)e^IRD|$hW!k&z8cF z%aWqxWiYXqK$h`Q#HgWB(c>bcu0*?pz|`j!g%XjM2;PQiq2?6Bh{3OK*kTUnBGa5} z+ICjDM6mkZSa4pkQN?go=v|Q+LcFqoZYLoTLfl`hp6!9OwN^*BtHlSIAU}qLWN%=S zj`KKdLOp{U0-@#d(We{mrO%^Pg~r5EBmBo2@Oxa>N9uxv1WgAevwqM z`7*y{m$Ab(uUP6{wGzb}YJk<@e4@{nY&_?1X_p>FT$P!}Zi)&!HS%?GZap4Iocl^1 z@M4&m<7?Js@DX5xb}Jy+753NlCgJL8vpIXG5zjf4jL2$ifCE~-on~oisR%Fd7=h&Z z0K<&konZd{=UhRRy^%>BuJkA*Ef~G5soN>{IG;PNQt8NM>$R(YwfsEo`MK0eh22Tn z|M7ou1R(E{*NwGI?9Z*XlTotv?(j=Q{by|?ap>myo>rk7WL?F^xWZ^?s`fl9;lUeX zo)^V=7*)Z0*`~mwhf0c~mGC%}j>___Jd{390*2)E#ufJ?5vC?)(4VS%y*-QQP9w?h zX11D)pK3Jz{T)rvZ`>Ag8r&i{@Z@D%Vl#R4Nw*$zd4?Vn zu%Cukz$EPe;O>jFeAf}XM-}X0-rv^MOwvOd2?EB}-s!l14FCCT=N%DW>oV+mFlFRy z372Pmzq8G6`7|z)TI1Ri$PH)dV}9u}nf%ylckWfUAD(h~*$4bu@`aHaVq-iFd#W7- z$f0JbT~sh>v!uv4^c-+Cc$b>^JZCG?5OLn|+#={E**0t;Y=zXMTdWH_QY`G}oOb!rzhKnJOMc=!~*tpbYBQko{{2!kV zqSD)`eY9oP#3bxyA7yzE9GfZ6|03tl;{-QV*7H-hDFV3g`~5vq__3=@J_>tvkX8oA zFcu(Aci@6)joLi_-yJ3!qn|8h{KtDfoue5mFE(A8n(Grzf*t-nRL2eL|E*CoWS@e4 zywHV9Mf~`nlH@=@U&b2*_d`zWwtQd3hM;DWL#I>FjvQu__EE!72y@-(qXtzC=r?@@k1QIuNnMphE{b_wRKdULfI#p~+pE$CM>h5qKODOmi&3%R@A%QF;GWOKFQSdv?ySB#bjzW!(n) z+`TziEsV}yRCY#PEqy_cmmAf{Y~gx+-pQ)c`FLFT(PY8Uv$TgPe#OtP%cK#xdLRt? z{41v8O3*ew|4KT!4e)e^epRW(Nx9s_u33p39mSww>|@L+JlH8zYt!;|bm-k;eN@{! zL~)?(&NJItz4p(Twmt7jzsy?`&?IFc(`eN^?f`2sY^qaHNFJff;O(*3tYjSA^Qera zt+A&}CL1+sLx?{T8@G8lI>&xCzi4ns^`YW5Qr7$4{?ie?*Mri2smU;@f#P3k)Vqwf zfSnwuad=1_&heI;v$5Sq0YY|;oe0D7@BRT3#CSkSRnWf$$n@rV+9yP!1ZRosx@4&y z$t4-=sp z3<#-Y6LFHrjXzfOCQi}h_nwrr3|QDjd_EyIy$#t1@L*~&0k-tEqxFCs`DRCfN4hoC zzaAJ_0Mg^U%KKgzzK|ei1UL{8T^Klgs3n|hIX5V?w2dG9N6i1j{E;Vq3tcrUp(HJ| z`thCDE(XIHk_MUVAb~!bbgjJH*FT(CTJH^2PCx*IGkjHkf@%Tx&1aPp;IVt9T`nbj z!(tbSQf!7B-As$U?QQC#$MeO2ycAQ?JafOYWJqII{rwCHQv z=TvNtSaMgVJgUoax6aU;Y`4(Sj8#GJYW#J^Lv4rI`LkhVyqGj=8X3@uXZi6;r)nIf zYO-xI)y00U{eMi0;J}6Aulg&+*pGQE$uM5YRQTHRX_TeO8F%HkrTg!XMutrXJtC@S z@7PGX=;^7_X_6;*ZF5voUb%b&-ODA#P*K#4(;)!w&=wj@ zRN8$o!u`07pc$AC8h?<`-UV@)%)4#Z4zV~nI7Im6(R;{f*IH87l@^_XuQ3gn&(IRZ@D!7LA4M&ZNIgt{5(lhOQE{FuW-Qh!zc}>?f;+k`v}> zj?zVnFPC=93|GOo&|u-WdN06Vl#4^B>fFc2Vb8XSzXA8BQeCeeIJSAVf67Z2za4r> zh^MJ|-!Jv=6i!ZJSoCq%ca+XvDp=}pJS6pW`2IgNDk3mai!r0U$)HNM!GT@|YWFGt z<1;(rgF8iXs;M!8%b)#o zxta@6eXu$v#@4l!1`8@A=bWKd{Y8_0(4+xhfvzj`c&%I+cFM8bq};9CQV)E!j;YY{nIEI>>f~l zrKXo!gQX@rZB9~~egp85$asvBykWC6A;AhXs^_wo^{Yhsb?R~LB;j&Z2q1*9T1O{R zq%e7*<}mty@|#>YITAM|)%DrO&GGbuDw0t-ngxt=T$+XPHzzm%WG-_WK3e0TXXovm zA@YP}`a799X7ZdL^C={(+MbHT?`8yXEpmfWdT zG&(LRe|$a1ijYe1)|k|s79V3V{O5063(7UgAwHUbunr6_zPhD(jB`hZK3V zy3%dDW#PsuiQ(2E){Yp7xo;Opb&q&jXph8Id;7u`BC_ELpc@sYI<}MF;gLgI36SCC z>fhSd)S7*V;g#mkfL83fTyqgV1Tu6&KHQ}HHY!t*bIvr@M@6J_`e{-8oRGgBF8y)$ zB7FPz)r~9FJ(t*H$lc6|{o6yVaow39a*H7?zR_`GkauX= zZ;bTaJTpN8aB+B-=BN}htuzgLkxXw-g7`W9-&vftC<*3W5J0+Ten!dHI9{0I!P>%j z%2RVHg-|1Ljmxzg?;eWo0TC9|`ti%IGlD*A(@4 zT?!}l&{^w9o0O5Yq-iS41+5?q`2V5o_MjD_2n`ef>%q1WV1Sw`^DXFR_|?#(LVg!n z^D8~$LOBUeM|IP65eoqS5-l@*g-bnVtI6S)^XRk*xWwY%kYR!W4LtIhzHZ4dv#3I= zWM?P2b~XdekmYB6u0f(!_XBn-&#=z2f)5O9>7296W^Peigl{adnVxMT=~!!W-*#Ik zQB`)1xlVJNV~8zL*+NIXN4dUKGxGzx3_asZiQx+co3!ElhZVHtD!5^IK*<%nS zs3-l^k!lB;tx>CYj_(7+dArI(^hfw zC>>rt^S5YKvvnUCt~1>NPnAguWx#$j*sdyYh5QdQeDOxs3s`qYPv*NtV~(I?1OD?t zSkuJOke8U@b)wFwqh#`i;O1?pMyv*z0kWd7c1_=%kfa@t?e@!msbLgBjVqf?Sw&s>4K$#xuJ40w`D-%Q*h<(1d1Q1n#|%2;e@=ruO7@^S!_!%( zC*|*i2&*cE@^M!LoE5{DTf1ro_(-pYFv417rkYN!p{y{!Il;!aD5(gh^FOusMGzWq zYCA;q>)1yJ_0axbC?E2SBXIFA^p#pyBaUE35x5DxbV0rdIrmcS{MG*yi2pEAn1hZ( z0u;w-D+dP3QzNt;U%BG*t+_PX%(;z4vMtB)giqyDK->zf9$8d}er zE7kZ!&K$w*==b{h6(+eDc7_GKM5`wgb^4cuM-UoU_+W_=z8l-K4P|H*Zu|I@hx z`&j=E0yE(MLQsgqk(JvDOCzc1IWx)UR1=kxR2duQQ;mJ8UA?f^%BNRH!3&$Lh*rW2@ltI~r z;f#A2BqK{Fgp-;g$sZFD5I8L`5>Pq#$$0{V7xso-mPlLflUW@?o+L{|NHJB2MWa*u z>#rFYuyUse9T=?}VO$6M%unf49wSalN^tezV%?30K$OosXU>vhl|E|kdTJBz9M}Rm zdMEugpUQCwR5JY^^vhpY=TPBBm;Xl(b{I?^XB&Y6u^Stxm=;cp3j=@Tob95cR9=hK z4o`&Ap#Um%mnIe?aa`o)r>L|iZ+Q1#YRw0+LG`v*Bzm`3SgkiD_8dV{YwkJl9f`M{ z_b`bYpfQ^9M>)>+OC~S#N8))-@Un={-LSyR_^p0vITn1Se%`gd$c;e^F*mTo z5g>cW~dEXnn^COM2ihEpkt-ThBnqyi26#!l@{9B_o-Y;ItI|*TQTBJ{E|}Jp{Cf#l3@)XKGI>21XXtumh&LRi zWxu*q%x3zCN5&K{;tOVfJmgkuCeC!L?8O(ezH&COSRslh7QB6l7frT>`zxk$!9I!& zj3KoIw7cEod_3%Fw_I3{#-8=KV_Y}}@u9^(b%`DknzvYZvs-^|q)!kGcy6R?y6WeF z3wD_K%oC}$ch#uBrrV`EVE^_OGnu@vVJf%9=iE{F3@@K^X1a|84)(sk1m#)J*=+JucsSu*Mc8k!V(B~S`QAO^^%EY&kL0Xu z%?4f9^=gX=C}dkC;`LRRF3mSl-(|;6Gj#3CxF=Z=7Cs{(U~a~_ML}*BpFL|doX$1v0*~Q~0&

    z&UH@ho0`QDy0v7kska}ksR8|0tp71 zXFNG{0#HM`l&SBBhN~H+euC4(ycqtNdXX$i9gjNlkG7o*FC9in^Cq|-(BVuSrGBi{ z`6E*f4ikXI^-e)W`FuN!d+#%Sqod#*Xk)|HGn&K@oa0&B-xvbi+%&twi;sv%&@uLw znK}=i{l(0@PIaSIalDB=o>EIxO6*gnpktZmxBGVLm+33L^D#QU;88*^vzb>9Cs&dJ znCW#I*}M3Kw{LyRin8B%d7l59qmI3})a3t38JgzEq*CtQZ+dmojD?c!gqR=6rB%j< zzlNPe@RIJIHH}W?wqa@%nEctoBfVPmeG5dlac2amACkIotuLJ|k%iHMj^^CiT4M5q zwQ~JWEr-MQr*MONFQttY94ftEFe@7vOf!Cw*Q%~P#T)dx#qh55T|Ky~1v|X>wX>(+ zB`$C2H`!Q4@+X%(v%xn&a99XQg^@@*sZhzQ%@&Jw4oQ%_yTbI!!iRoVQ`wruBd6CZ zvdCS8He^l^JWy8e=t>FdBKg{#XSTAwzNLVyR1?m%ZY-)LgUdY!!I%<~G|g6x+$B?D z5opL$iveUXygSQ}1bY3jdvARC;d|=YI<=k7uxu?F<~HHbxE-zO#xfhFtnI1cs5!7^ zr9IU$W-HGNi3?*@L(YL^Vf?;SM!rI&vo9!R(Hn1hcSd_oQCBhGSq}|&KiODIr!(6_ zMa*^~zf1UO)9vj;x8Z}OY&_FQm@)*BP`vlfw!&QzQigNCi;?BbX;6g-LI^iTepCSv zA&NbA2t3IRZ=|7WUWRqd2+jBly^GszvhoGGZ+0BEbHnP%#mRx|YhiA`zm~Qvg$!(s zdII_vYX#1br$mb2iW>B@)~Y_4S?N@?x$D&PIj=6Zs^FW`c0}51+Q-g#np~c`JLb0R z4ox#p8HHnMV@6C=%vJVgd5c_bmEPO^Xd^mWttgd}_QK^U=L9t5vdxxjO-IEhr=;vG zr9ftfQC6Mn589t|H4ro4#lTLBbcPc5$D4he$D(1*HFI}pqgQO~!XHIZo~8k_sQ2Ix z%vq|bi>7W9Eio@F(t(Pt&Z7yv7BxV{ju#-Xp`O^J8ksGC&i4KyPCWk3;_$!3W|pS! zw`cupg^FUZ{q>kFKKaMmikDTl$di;^Xk*_%hbBg z-?9ewYO|5>K5(>3w@0!I<`nt*c+Fs3u)e)gzuqX0-l>4iui2vP`fzdVsI#_$vzZa! zWkgf2Web@;)l3nyNEe8M;S#FI%RfwECQh(`YR|)s05=KZ8JORf>5cn))Y6Y(@11^` zf6gGkbjfdH`zA>KnIfz?D}oi|-*y zVAr{$-4KTsIKT4Zme`Z_Itg+|oxuHM{N3dGX{UPGiB-RIq{|uRSTXp97aB9QrQhsu zS{0?$1zM&~wdZYL<^13>G1qGD(_*5_F3HvY>u)Wl7<=0OuU~|Q)u>sWp=iTg3}3L2 z2?*3wHubL}g+Le7k&s8h z$=Ztou1(ysk0u@g+nZfN*fWEJ4mRw%XLO4FoQl1L7V>v2m+|qPOZTJzoK;8`vCa#2 zbFVYQmxyk&aC-J-o&l3d@Ir4h32VhZS8=BYN4!Xn>zV%22Ux2$m_o>%f=x%|!rSIZ zdyF8!r9ynPF{uqe{_tJ01mlMxgQKjD-#plO;t@*3_pr7F1O4?%a8Bg$9}V@&tD<~X zJ-|#yMjh?wk2o98*DEd^q?h=CD9OVYsih$KKmXE(P%eWHrl2GQM5$$*VMAV z#5m%?&VX8g20#pd)QKtxP!e7}ThMnUC}lLmliLq=0C6HC^Q z`=u7uIae{H108Gzz&ie7JmIX8$2z~_>miQGnW3viNI+%Ot-dB&zEE^J(7|X(Tg0Zr z(>^sbY`*hoa28TCrQ4P>*T3|_M&;z(XywWn8Q%YJb8kH*v#H&^(T@xP znelfXps3R=23IvY|EKX3X@ALs)&jls!D{*`DCZPDWpb!wP*#TPE5 z<>ImhBRWC7@FHuDw_Yv&rgTA<1pClge>{Dgg1dUt4f~OKU6we`(N24oOs-nHd}w>B zjb|cOwU4&(sR4Gi!`bqa(&%he7duFzFl^2)?F#*N)Y(wS^+-_)6*VpVteqZ`O|2l` z&Ev7oQPm@plj}k_EQQRlPAA^eFCA>~FNgn2lTxxE6U(;XJZ z2oBjr*v&xFJZIms%E|_}lSZ8r$g5i=syjz+p!&n%^VO7GA12l2mYom?S@ZZmKkQQ! z3>Gq%=`=U1c&Ah$Za+yCLsAq8JAVa@#io~d^{gpeCsKdC1HUO#ME#N+TV*gN_S4>{ zT8(h`2umW4JF!b*U<9qy2X%GVI$@y=`M00$HwjNL5b>tKXG;hqN8T`PvUWMC+#tSs zxgVuqTh-j)B*d^o#@D6GFL7(!iDE<4!u1E50YiR!c*ryU$V*HUq^+hP)Wa2FoHmp` zrAG=N;OB==zFA=tD}xM2kYQicVfy=}B#d~U9Xd}VUm%jRxp3>Syn4MA&V%36xY_4Z z?55(1I&@>j2EbTw`m7ph~?~Bn`8S9iH$}O~Mz<1xo z;CU@Xz+J!A|Eg;_uUk9&iWq9mXJrjr$gqFRUo=&`4!ZmQ&UFY3p9^%@EOswu$1wO8j^K`vcUiN3kX;#i_D2!mM0+i z854r$>RG})C@|ibXF!Nyg?t`K8mdFoX1=N4p8|a|y11aPC@T`dCb$wZB({-`0?;Onb(zWwvWWQPXt{T?Fg@1-n)nq#Jbkdqwvm2cE zU|!o`zktz!F-U+qc6McOKEo< z)BleChP$l_Ik~;$Io#^cf3A(dr?-tAOWlOy`r3Tp=GF)Ug6jJQ`i2bP{bu|V-`(C>~Kp$CJadkCZR+EkDp%@Hhm*i{!u1g_oetrD*YqEr zh9x0C&(ZKegrq0r*0!F~xBTPd?O><9?eXgY1f!3c(DFFm58ufkEm3r>d$LaYs;Zhu zn}(Y-SEb*ATie9?lU-ids_Jkq4UAR2G>6qTV%ftNz1_~Jcx4z3unRQqAbhWK>^>B6 z0^{!|0jKNgjv?HgvR$$NaP$7h5^2scZT_let=^WX>}G^rEvAnSI+ee#P;Ql9#*JPG z6G$6TU0huJ`24ucNY5=P>U9@uh9`H*?oEPc=Gd0YL6+5@GLs_i(=tK!buYvvD3pbU z!S78MNv*DT-mK)zY%trbRCXGhbBm-=CM@h1WH0z2B;I=T-r6W30DiO>++OiqA0rMPYld6g8jUgHfBwj3sxwpEr-?d^Ffq@T~gTYRdHxwfsC$!yXOSh&kp!P=$~u-cwFFmBS}!7+CK|55do zQBi-<+AHbI5?=L7VF$Gb*^-f#XT%jSxj%2xOa-$CI z^-qE4pzfRAUSIC0R%v8rWlF4Lz#yP-QB9;3@}S2-4=|{Mgd@EeNjml&Vx?)1nK|+g zZ@wwq*hNN^uD z#Lya7WBGndT~%)-5zBc1W8h1jW%S|edL=|DzGYb=8N715 z!n-dAEv&~&#y3phF)6&j31WFmggPxM8?^q?Uw-1 z+t^uw&+$#O`J7tB>(7$=HW~Adl}gF4nNL}ktyMMg zeGUR3q3L3xby_97USR+1(wZT-TUXF&ziz^EOzryokQ!z4i^{&sL8V!eRjm2?ebT9C z`EUC9@@FOEcE{ccFD*mverv_dwuj1kf#qI^w&zGlz}CATepIC)wl!I<@rx~QmM3sH zwidSAL#-M56Qf;m@?Ig-V_*=CXRh}!OQqTWeO{*$-CW|N8a-RTr?m64CB$4kqqqS} ztJSu01D}z;zfr5KtLp^=;pb1gwtT=r@&Kd|TK|{JDE>MOVPQ=Ko+SuD$i|AaOO{gC=F6S~JQpzdM4nl2 zOCp2HgU>6*!P_zy`r$d-miNq_DWS=A={m9t*B{ z-N=F+#kyYHPBJ`-tMz?j1fu)wpsn%7B-o*C?v1+F%^hn=%YbMAQ#Upk{{vei@;o@2 zw(ZTYu~W%Vyuu%3FEGIXRLDR4t{na-<3%9H=eLj~ZJ1yzF#e?@b;&SqhS?7{44(?U z6=g4f;7#((Qv+qJCjhj2-;>&bAFb`V9K`YNkNL?;K;4r~tUd+zY)XGuJKw)=6>26r zwdTytF~}Qxv`GcMKMYg zn;T!GbKgN+S*L`3@7#wk3rTBZj6mca+wpc`m_o+KJ(CO_DqXn~NA@fgDXjB$3_|j^ z38VJZhR2Z;w2Ok&CpF)cI*#G2X%G|D; z%;lOj1$3j>5K&VZ_>0(=QT7@K@QJC{LKFwT1?~G=B2Ykq$~#uccqDG#x@EYyu|mXY zKeF##E^N$(9?%Z7kZ^DzQqhTCI$5@#Jryu~_Y$Sa(4|Y#n%~xTJ(X$W0R`l$G00^% zA~RyXRx#v5k)Aqp;@l+@Yu=c7&ju`F75TQk6=Y4UmC}#`y%K$LNM>E8Z`R(tzM921 z?$c(NDD&Xhm>pocI^W~Ijh~OU=N%`i;V^-#vHS$|1e6}T75AZRz`N&XmzlJ<_zh|qCTW$JRdW0LZvb+qOh zvPz6ERO9nT4W)CWC?Zzbg1}q|(p0#@DN=(oYDUyR#>1E_90r|b(}o&9LTA3r&|iG$ zb$W40|}Z z%JUb>-`Z*^(VoYijo~rU?T3sID-UkUYL7mHlu3<`$N$1Bhb+P;F zf4ugy?+-z_)h>SvuS&|WP_NuWeJcm-!ltJZ%}GA!N0|x9^w_P_*!`&w4{HSb%Zmy& zEqY=wY2af1VOR3A#XwcgJzc9->m!{^K8?3!vm}h#X0)+{GgMrPqAw&)5{OTO6lBsF z!T7<+;PPyqyQxO~vMPoQGL@L+V~P*$nV^9+6p4!%8`NM~NH)@oA-jq8R;q#KtY7h2 z+e>WON<0YCFgpHsL-&Ek7s_Oa$)tMjq;Nmx^{V?~Tb(I9N-}$s1x-24|ygX#wZ{1d`rT+76lo4oF}UY4Q<;rjw(Fl&XM&CEttDm3QGOU}A`Fy`FgD z;gqDzLt?b2s{P6<-Ip_hnjo0eQ7b_}Ha!u$Ome1YRT%&9a`0XfgY5D(S9hDFtF!MJKy zfDA^zjDOzSA$poK*GY4LyjlM9B8Y0mi-A?5$~w&7a(XxBZo60*j2nj3A;*ubUeM;< zg!?5NXN7VFPqI-qB)Wriz9R$>OQ6-* zkBDf%78Fa~oUir|{jfE_RwVw9h8P%8M#{z%k2J!Z?^ivJRfq4u6NM&HPW#ZU>rCd= z_}}=`=%qraM`pbl!0{Zv(5q@0cc!AFoq5TcWzFn>j{^#jgW3i0%hsV1F(GYODvCGB==>%d9BvYX#H-|#w| z-*&Jp-H!yBerNC#h%r+h4m z7wDH=nUUiSdS#4^21?M!2!Yz*rXmV1%r@vadG95r+&EEmeUixCmC!Ah5d=q(H+aKq zeP|wgx{h(#^W^*jNF|maSOB9WQKtBq0L$+3ZTQBEJR||Gf5aB`LV-SSpdFcTx@h{1 zOpXFI&QAh4m!psDdEbk%K2$GOIs+Qvt!*{OmCTu zkLxh#^OMAe$W#6rzEc37_f9CiSDPwVLdM%A$Ybo1_bNVqN#t65sQ()U8fu!u{pSw3 zD^#21650`bFjwyQ80cGX8`6- z3~fn%+Zf)g8yC4&Es~>i<{)($qhH%^|M{Q=UoZCX5hD*{MUhc3+c>Q$qB=9r;B*_YU?zpV{PxYlKDH+JRtr|?mF0)JyHyn(4DWkq3kYuKE;e5%^QMb+ey}{9{Kq?8aV=C zHSP7*H@V%Y#fPNz6H7TRjMXjLsBZjRh+T`mHAS<$Ue_@{KXI(@@D{9+zn|fvqk` zbqDsFRXK~(si*F_*R_r##W{l%%NTFv?FUH?QyYf6ns9*d0kn@^ z0wBNU+^0APwskt5L_&t7_+qT^Byb4Y(DuuZ?nSz=j>hAnmylR@kL{I4pXe1mo<{PI zW9pi!G#Q;L{V@M>*mfug>Gtm-)f>MdLUYH}Xz9kqliY?D{l@_$1nNZv91vYmL_&M~ zARiqZDPmv6hFZQ9%WePbqTj=mQ(+$Qa|nSo!mtA`X}tWuzoe6cwIY1C%uAt#IXbjy zbnl0cpt4T}q&l?%9NrM<{dt#X)p4zlGCF84Muzv!Qw<95h%<)Sx*`_9oEoN!1T@D+ zH~~SyuBdxIiC33*l-R_32MhM^W+@(6e}eP}$icy-*&1R03i4a&Hr9@PYtgd+uk915 z{jzeA>r%5LCbHyvQ`AqWlHB%o$ww}wLvLXSVSWpE-)Y`Hp#1Az)zxx*y8(~NM{KB& z9eA<;snvS19Jw1?LzfA^N`@(J5>q*dT49E~XLw_VSrxjG{v&9WlCPWI4gsK06jSE! zaiCwWnV?}aYjjl9-X6-^b#H0M_OF{ZA(a5lWnxg+lwQ&g8AoJp=z01 zP98F2*>@g0rC-O`jJGIX?BfdRyf@^Ko=Qw*2|KX?&;s}=T%}q_p6%;U?(Sq9fr%4w z))@to$71+93(OZT7wmsM5(<0rYij#?`+~2~eEdrE;CX0QKj+hOEh*D4ifkq-#;yCo zFb=!}y_gho+jO25YU$()43^WzTVQZ$5l}A4`O2^aP&a3UZ_~$)vr26vKr<^vZCQ`w^`TegaQ$hxyMX(%F$DCOn9& zw+v_2{L*&m^Q8M1uXN2a;0~Y1 z<_DMEVJar>#tQ~8>>@BCJQPzp$OL-wW!{no6|8{%dx7|>T(WLDkuT}~S8<(~)cnysIOZM5qZJS}Ov4yy7|UPHE7#`r=&^8H)~ z9E%!V{MHtJ)(&S%>4j>T9uNT#EkS-}X_4->tf?VHyS$ilSsUAriWB<^$V-_oiWKk* z7_udblmBs^8y({t6RT#G=VW4X9R2)^KR?M<)<};{L?x5c%nSuXi-MwUY4CyZg zaAT|T3-JNi(&Bdv=tTW`X;1ShQ61gB{Orf8%i&3XPU**Lc*PnBwYRUM4<3aUE{X+u znfBW2-qj#GK6gA}y(z1;GvY(G+}AJFvldiUIOAr6`NzXEa1RoG4vm+(c*A-FRyh2N z&0$Eot{t;VI*eVQsh-`~qYE|7q#T(2Q7vb>i`W&jL0`5K51#litCT%vgv0#WfNX;)vDaFNq-G$m|vAXcP(znum!I1d5GBh;7^y$aB`g zmT)Gwz}l-sO}@H?97rLkLxNEh_kATQA?BncEy=_lX0MoMK>p<=7BwX*C^1){lKx6S zLxJc)jZN}$3W+! zMz5Mzqb}6GqU}e);sq$tYlxeCdY2GvoJznklTpYRzkuo0_O{heh6PU}Jv{%19r5?o z!YyC7#&8^E*{2J%sF<^B_1y9>T?#;{D#?R6Uauo*M|JOTN~2NSy=+2x08L@mbquVv zVGC{r#Xg2gFcI`=C20u6PC>15wSTXrVVWpoC0Y$l~Qhh_9Pb-yFM7@{Re#4_jrd_jE*=Q&R91d;5-9W10tNozK@cyPPp(9Xk3V_?aPVeG=7v*&w_Dumirsj7 zhEJ6(al#dcKXZAU^~&+=NCacJ6q#g(_QAZ^;k4qf1huDq+Z~7YG!`rP!YzwU=UZ?d zqD6Fm-5A9c?g0)^&VA838#+a!Q@52ojf7JAFRE=S!lDx$v@Jq2G|ILeD!|5OuaM#1 zi|tY#|B!VctT4g`PsIxV5tb}Ews7LZ3g-y#XUoXZ>8hlP*30?J*gSJE8sni0`@XN+ z`XHOtX_V{^$)YK^9=a`m7bV;N(@Ke&+Oxm9{58Ho+Av2;?i|pGcLCOr% zhd)_x-N}g|t5s;zc%wbo7I4nT#rK1{-?UB8mFI1Ed13!DrPkw66aww11<=di36;{1 z5f+p7c$y^iy;G;l5OMCf$M<2I-ox*Nzx#Qt{Amfr*K1`$Wu5g^ZP8&u6V=6G-Dj6% z;_P;5xhw0t0nN|-)~wZ#Nc-C$N@oik!ACg(7qT{&=<@mT!;qp$2Knb*jC-_^CB3ld z3YQT4G}EIa9rrvsjWxF)%Z)m2S_zW|0`!0PGS%}@&C09*VuFsTgB+mEO!0t252 zz_G-~my;hp_^w(*OZ53r=IIQrcj$JR#J}e7a>MnOY|};_96X7AqzuuQ$O*3v4J@3j z<{n^>Co*S9@*iP%Fv8#+FCLaTUtm^%Lx&T75?MbHNpFv^n3d}`By|6X*ZF~du6#BVFNVV=OwtSYDrqn6RBPfMVcx2#Xmk70#1otKD z6rl(iMrp{5Kal8@wpl9B&rf0=uYm~@9cv0C(03dV)%FP8B$ap1{q1I~^pCKWFRP;=Esevpv6MlhzU!M-9oHm1oVy6kC+)1L-I-$HaG`P$cj#8^>%J@0V zeNV{5HjwvYGXlEHvFFp<478 zxJcJFkfGtgD>)6>)9JvsuA-Z|Om{i*j?!J*XyHNH5KNc4N;c4RsbbxirF!ej_65QK zs4k)9f0oY<;9zg1Yy`hO5ak6%ejE_$bEK&u=qTp7TKlYqPIzy`{GH|eMU&#efo33O zbUE}`T!bS{pw&7=z z3ulD%rwKPrRhLhtio6vD#)zh4{(%7+mVT?s?lwpK4U>PzIY8aqbN}HVM$~{`7X5O0 zyhY~;5-tRj2#b%bJoF&Pd3cp=-W<|#INus#RKLFwSKhMWjJ4)U=P z#A+_NTDZt>l9UIWcfb+fvH}X!C@YVUK!-BYAVK4m_{7GttCNqW(aV)JH-HDOmOR$l- z9-IE>)gR1%=0~;Q@hxuF+$}B9U~Qa$U&Y6=YA}8P$G~-Pc4P5i3XD6<6`=8PnD7aV zJq~eKTU`6y^T;&-P$>b}#c`T=DDX|NFGEH$WqgwKvQ&e^F!xI0Ath$;RT%hXJ-cp| zvt+d!OlUBAY^K`A!uW9J#mWewe9ac)J02D635@uo=Jq^;Gypn$d}T3%ox`oW_tI=- z8>lMU)HOg0!NuFO*tjRgf@lFZFcWW+Y|A$yDJ|@@eSOEiSz%kJ@V`0I6qX~2w<5A0 ztuCJu#MK#+2Tb_E7{LrW8e2ubMbpmLL!Rpso_taoRPODe@940C{ZGgWBB9f%>kkqq zM8eW8bVrL~nWtEUG|xWCDTTCyUP*rNJ(mbQrYzYmT>O$X9Oeod7_*n0S`%n_oN1b3 z$s}%=B9(7&pC6bS53FZ!@rU|7ilzYvu=~X?aHJxSN=-m|F`Q!4phtMQ+o>PB)3ms1 z(-z`6Ni%Nz?I)7ZxY=<2d-)lGha8Cx{pE_|CT>wrSkm>jdq<^K-}7~_9Yq4 zo*-!eA@$h*VR{Dqwh@HYdT`3PaHaT5dUJ?`@$#&0L%Z|e89D0*0ENsKQ-Ov zEV7!7Nla#6VhM?@j6kINEj4&Kd;IKD5K%O6!y?rGfhZZwhvlxi_xsSj^A>|XS(Z2> z<>esGUOPM>cxq%8YP^sA8H#hx4bdu8ACqWlRsN-m)LZsYFo>mk>5bsE0de@#-}cZ} zG0!$q$fK^R0U#h}F2xT0v9L&Q0y|J6kz@Cg#M_EP)*DG+9gyq%+4u>jwr&qzqm%MH(q#(R6!zR!TozEH z0hXQ*y!?!PMZXjq>bLz{c439>qpB!DjupF6%TsG@5~gm>%cdqpR`nl`-upB6O;qEn z40xJL&ky0%wz&e7!IB-2SO>o{eI#IHGa;!#R=oF0do*+XhX@r5(Ef-2s92@c_|ftB z=PRMGWj@r)k*H#J@t(=n{VQDO^Sb{K3GWo3CDIk~sd`+)NPmm8o#g7G(FE3)f9s|= zCmu6bA2*CT&Mjv0>^0l6EdZX}8^8hsbocej@lB|PHcpy1A(LV!DVN8eJdNmpm6w!U zWjoIzBtQ6^_iR-@uaPiMpoPIEZ)!-j%;5))C>ahh`^F4l%@FV-NAt6|@<6f6$_x$8 zW1zo!_&(MHYz}MAU^2CVjbf6wLew7dc@;Rgl%`IfwW@`Zl(0z(Zr6`^@o$O-T9-e+ zK#uZ?+mp!P7*EzL0y#hYG1m)BB1;l9q#PZ!{u2uI50?s?lo3dEio3fpsWVc$T;uOL zxEw|Lzo$nU%$hQ#MpFU|*E3vZT(HmIDEs{UPS7|SGRiMfJ-!uSD9!N25sl>!m51AH|4$S!@ zZ_&6j2>+?oUCKx|s-p!e?mYIE@Fd6@0K2>mb`m!8p!x&{55xpfmMLdzc8RP7>D<4M z`5A}P(f@2hGf1`-EYbMUF5^c#HgKo>gH?D@AWm=|C{c;3F*Ol#GFZj0(~5BR(LCCd z3?v=aOKgveuRneOg0}6XSYn7JQ>xrJ3iPZXMMmY>^Eh z0uc-JOxw4*8ygRpPkRzh2QKv(&!CzV$A(y9IU1;uYu@d-fRQ!aN@OBy4S-oFEDQog z|LQu(F(N=qlf78*KUxCXM(ac0S#T5O(QNRhICBx* zy?vh^zh%Q~;WuNY?5@zsN2R=n#^G5ABlP+<*|yS$2gv;8cw_(JLIA2?HlLSnmbBxbXp-EH>^*4wND?R% z@OCc&YR-U%Tn7n9^qsdT)pL|*8jr?U^XHXAZqP2s6`9d|)*&elJXR)TTxMXF<1{=B zW1tNK+=3lg^n48Mj`&8D^9^*v0+DufDN(9fVnQ>t7|+Luq$^ecd{S*wuGsu{4NspC zM4=yLP?|Bb7D6=u`w!`qNV<*#PpZDasRhs`nQe~4CcyuJ38KJ0Ds}kqGfSY%f@y;e z^zxKe`|DNXtM~~4`eD*KJd`3s9t$eQU{rgW8rG?o9;A`?ZK~hV* z_-wCzKrT6YCen|&4Jr!QG9(#(?lI?L+JIN&7|C%wSPJxPd|2t7^4CQnL}s%lKXe7V z`8lZ9Q$#C5YG6^vE4U3*_Y{SIPhP!DH|<&7k)L@X*{s$z&R6;8WiKFia-g-++uh=d zSj)tX055Avv_aI*3s;7a`Y()qP7jAQMDu~tMBtA9+YaYK-l1&2pd6gwgd%J2qnoY7 z061i+i^k*3UHJo&@2thh>gye;Tr=hnGWNp*(RECkp2@OOKePDdiDrvPMa9R0CkDrB zxf|!iE5oW?n@xkiBOm(=Z#H)=Xp=O88dJk5&I$6m^WZWg?w!rh((#(xY4aHQ$n~Qz z?w*%f&V7#1_$u^9tCK4QRQIK<0`D>ML0~!y0Nw`!HZNepBn9=Ik~X8VS#@f;CaiH{ z5<`v`9ErFMe8*UmH^22~bm%;o0WY*06hSp2M5u!Ip?mDROR<>y1m27kp1QKgbMap{ zr;`2E-pcB?uRJ_K)0TgTIHdVV(GPqcN1Ex#h@2L0@8PsVlh*iK?{cb#FALkiPLJKD zC`c7H^^RFH+4`qQDcuc{T7MG1gV!@gE3Zyw<1-B89E{%08ptGUnIfDvtiL|&5)a&5 z=s!&R3q({EEl>{+rRDLH)u7ffj2DWGdRKDzB{_wvy02>V%XH!8BSnz>NNQ-iS$EU_ zO4|DLGM$;sTRcF|GQeGivX!Kc$8UeGy8*qMKdA0#TjK#`l7aZbB0)w}-@}Rl>}rv= z0<qK5eT#S?ftMJMVuLpx7j ziPz3-aOGqyjR41sti-}%oBlCo>H&p_C*XjXH z*%(kl3}Pldqr}VGNygHc8YKo2h%U`>jEo@~xS9oX<<8<8Z4VcXLMEO(rkX=_za5_X zN-u+6(hrv7&Un=#efR^!;HuXhT#e7P1Dh#C2Fn|~8}uq4>7&M!Nj_KMV8m!8xvu3H za1d4#bHsnlX5pegVCk(QFZStU=|sl_*NAn+A6fvNVd60GN#ZCR@Yy6A_ zqO*R|x*xA+!8hSq*zCg;TZ0iBtv%ZxnQ`W#oz8vj8Z^{x&hIrw@n#ttwwUZFdVlUF z0gp%NBu|j^a$5JRyj18oU+({-O&GIGT0+=o&ldc><(+$(K{1AZ8S+CO0_uPlKppVh z^*wNI958+ebPQW2MPQX*@C7>OYd#Bd=VwJ)#GRbd5H&c37@{09sy%=w6>AWR#I8ip z830;OW8!Hj`p@VKVXQRH`!={M=borz+pW-8r2jCk#Vgb`i66QURI8nVL@JdEb(L2l zE9k~ON(>UtyF7~ag~89jQQplj;T2qREAk@^pI!s@UoHHXRPZrbV*h-SmOJF~hL&_S zc5}7PoYWr_q0nD6dj7mx+t+{Af-m(>2Qnv<1o$P${|Pu2r%Zce9|V$@)bpgV-RxGW{3#IbfHMt>v{ko^gJP7mW2rUVYVfUGE}z+nnYI zLq))h0MOnfrUs*35YP_fkCCWVsmV7XGVUT|E!JVsRvuNvl<`%gp z4huHDSbql}i(cg`02l51~>p*99l0f*Yn!P4bcySjIu)7%5RpCO?;Rm8BbF%wB! z5s$Nr`i1#rbWd;jBMrcsE#QeSQMkw-13e}-n0v?$QkHjshT6lL!v4}J-@rT_ozeEt zcbqL^4hbg*q4se3#wdMGwh_=ByCJifH(>jx%URCFA?19$B(UmhIMvVBQ z-$CHgfWqp82#?`Wh`z+oD5`@jHl*2SC0(A2DH~M__|ni+o4aD0vw=t-_q14Lq=IL?7 z6u1O$S_*^wQm-u3#|ah*M9Xmg+$>6GLTY%6Yzc=)n?%qqIOHDC1uyWeBrs)dm8l?m zbb)lraKYYD;;6_hJx-neD1kW)XYhXZuNt#U$&SH7E^MEOjbtp9bTdXET`K__2@+p} z&s3oB(=Qycrf{a|x|%K%(K8Rj=*xg~Ee2Zn$~Up68<3l=8Ok5<yZ5MS(?FYRnO2%ST`( z-v;D2boY!uuxmH=z6YJWv8qFWQ5pf5t+%3DV@|!oDa>MVE@F{>Cm0RD9ioVBEV>7D z5tWeGvtb;KhSf?(>(JtPUpMy%rb}JE8Fbv){Tls%!z#(S9#N;z-eY_T3^IRA?tg(^ z^%dm2la_dm{2Gf5RrN|*^}hiK%`^1$?)c9%bj2LFKR09o3y(*J@@*%?{;0gkdbEL69@U#aa+e8#L&LuXxQxuzA%j2Hoe`_%oBIDfBWwNMFHTvZW zf3>J#>3F`$M_kFJXsZ#Ons_%tT>I0M+~LkgD#}0AKQ4iI5jDxo*oV_9=Z$nhBn6jn z@riv%pLfB!XCJoY&=;*`cPHMKe7nut11(CD7*14-T+5-B_fIMh_S`kr;-pxxV_)Xl zcT4;|E?H{-Eq$Ba%~PFSv@g{h*`VNG;#y^YpYf3(O>}f(hULJ*(8uttPLwpw0dTyi zi8A~h)My00wu14-;e%Q62NL$qQ|#X?;j&uM0|pZTD*PK<12*4?v3}vffd!dF{e$y( zFaQ|?qOj3|{4Du}F0?*7iGLG<{@LHH1^@4!+V?MwP7Rw442@DgobuG3j!1dB!LJr_YhPzIe8lS~R+U(y%;9N?#r48vdAAAx2d#Uuq zF~m5(>Bp(3+wKhmr%v{$Lz{LU+nen<*3RS!Prf&q`8$%cNWAZ}vn$b&auLREzIO`~ zzeWgN5}MZ6D6kjdN?~w3Kk*TljIFH$b992J)#}8@#%gX#NRbN$ zqsKn0IrLXI1BeetgWIZ0cn;3<8#=X`QLzS>pb3MqWI3pN#>ws*aU;Po5nu-lfE}3d zpuvl$Z^Mt(5Ak{%D>C0{X4ND242WM!^qcDp1x~*r9w!3*U{sQ!WVKf*3n42bXQh=8 z*ALsCIJ%~YVvM|;9q9QsBmV_>^htbs(9`m~cJl@Q^IFiy-Q7%q%`?)*g$R8tH?szH zuEfGC^sztCo*7ngTGGrD+YnkaLAiB#;&CMRJ+ZJ)$weo;A-M+B*v7jg@4~EzODF$C z^+=Gp6y?t+rAfIs8zf7H;!Z7C-uZX|?AtHNepvrzje7rq#Kp~ms zcb}-=M|rU+kBhOB&akSkU4!`w8^@vFNlAISK_l?D6T`&23jHLrDpu9g5x0Q3n1HUG ze=bB6@{qo-jZL#zr{L3KV+g|gkxrv#FNn?Q(&Ij4vl&R2U!zk>DQ zKy2L3vuTOBR%*L8aKwrLrS;yP+~;bophV-Ap7O-awOUEHq()ue@CZ>>LezL}fd%wX z3RgX;gz~bF;HHJ)9u9}MOFw@9CnZ{t8{^-t>-DjS$}M5fgsVl#<4~`t8oKG;{FCV9 zXr2ne-h3=19W2qdQ_#Zd+SO|bI4nA1C{A^5m?>j!hWlU-*ZHsmA|whU10LFM+irP= zywhrqJ&!(Xa$t8MUZ^E=7^qoEfm$=p(17sX%}lT}&{aQ*m#K^W!wSnwW=`qx=E4)v zkj)p(JDg6~{aod)iUUc+Uo7?7@%WW);JQ2rY9>wj0}aFt>-=2HIgqbT9h0}{v65{OqJO*Ppd^E0Zh3!xRwFFdEWCO4;v&+EVqS4w zJ-RnvXU>)m1%#N*{PT}E`?;h{AP`avB_UDs-`vdtZ+RUs#N`Zud*ovaE~=yBO-!P9 z`bH7xuQC3XX(D4Nnl4k>B&VA0r>8JesGMOaUg~1KjBErMlTc*X;)ICs4mPCaPTzru zqB`toLUwHUWY<+~$E5wt>9*11kSNW~7cZx{+Xa6uq2d6q3{JH~;ZxX&bUKuo&K+}T zI3(AlDY%I_fjvn++s=zZSQmbk2yLia3b2|Uu24e414U^LENJ#Zbf$(b`1wCBQH85Q ziyKP6qzb>adH30Az`1lvfjnk1CZc{C-1O6lI4YA@>U^#zPY@9jp8AB0d8fV{7l@uL z$0^2U`Y(#|tPcY`WPbC%+iESejqfpI%AT}0T1AjeqG)^-;`YY=eY+q^`!C0DJ>Plx z;$3OY_ls=1M6CWo2a4|jJrP83<)$LJwby6;%E_9bq@ zXa`4Q7O-f zKhbwkE5SXc<$p0L@TOx#_s#yXvNe`Nv9-m|7R7|jQ`+4;gMMmG)#Fw#bQ5S zwLwP`4ba6-5M443IOpJ9zr7c1#Nv`nuLEz}yev6*ZHYyv0@Yr^KbTU+wAy%=KR*%U zrYhv7dC~!`5WTv@D zT}C9yN423MD6*pB9T%0*xM|4F)a8_%Iy*HE3rf{^$3eDjP|sIe7|RG zJf~C?Z(X>^I5v^3apHQjdDac2o$KP8h87{4pQZ|k*JG4uw^+c{6q4>LPCq$*=h{<~ zR&y>Y1fJP1q?@VlG%1}qW-gtx7pVREE1sm)^&vKp57TtfDL)imGvhBKi{mzd2Q7lR zA>LP(a5I6u*zo`O$3|u(bq?8i!Rfp|@2FPyR8a4|F>bb(`kbQAUZcT;_X1Tu;1Jm_ zME&LE6kY%?V1jT0l6^R6N!Gu9cF?c!Kh%>6botG!sKsyfCnMHqM||J_ukhffE3w;*==x+Yq?&kS=?bjR8Ml-jqvc9MARnFAnQQ4 zuz#1fkm+xBTJ^zjMG~`b=*F{+TuZs-{BBYuOAK!{3CJo&QL&8qH5{5ZT{Z9`{DfZd z(O27VJWb(|bq}Rs#=dgVg`y?JrZ2T5H90uga17K8PkB$zR@>zq(ThTx>0#WfH7{6{ zF4E&%LBf^OtE=i?nbeH7W+*xHbN|uo@ExR6fh(;_>6t$5&Hvhfqs#(SmdrqXVb2jO z`ULJC#n4Wxs=j|9B*T57zxQ5?3WW<%#A3GM5W~cBpltZBw}^=zmu6qZr~TunXB8gy zk?zL+2L8>!#U$Ycap1X6dJ0?J6vRhz%4NhUcU$07{++WaVyYB6q)4s=QrV>Z&hFbf zZnBa4QB%c|cmeAM^97yt6XMcoUXAr9As@{LKmA0Aa8s2J*yDUaH{we~!7me$N91f}g&u(eL@!^Z1 zYwi4=ObbwP<3YPe!nSkb?vwL<_@^c&C$t~Qg>M>jlKQ3q)YzeKsNmOuiW@TkC^M4~v+vdmh=)$|ll`pFP~vN7CNYWd2OToI zp4poWkQ4h5o0xw?XW91G;?&7u%Ri9~5isW@6D#?OM494nzbxO1u!^u?-qUDmF*%0J z*}Q|8mOT*vyn}`u@;k+Ist+uJ#7M@Y@7&CB!n^3epr)Cj=SeSMbY3Nzdr+hmW8c{9 zRH*&SM^qaCy{`Ppl@P?9-?S(`Elnfh!NNh&Y-6xA>Eb==&5!Ut4g&Jv3miZK37Dr? zqUk@u?9EJb4$&S*PpHTE3}S=H8+wa9NO5y`Bh^Ak%A0k_J{JhrQmdHw4SA-F zcIlpq$%%ZPSU*EbXfLbaQ2WRhui*^@Zg1bB_3tpj@yuz~e!?}gU;G640l^jIrvgNf zvs43NZ_!VFe=>t4ky=+Q1I4A_;G4$QEg@HhK2Q%mfy)Cq)bZj`34sv*XhO#4GZ%@G zJBmGSo0uxzUg)wCmND%;7t+-FPHS*oumee%>|=6s^`c>H{KQ1}FEn zUoMcm&V4T609{s`VRC1#i@EEV?ZkJVjhqbbtqT%2*1=&Fw~t2{PAZfh^4Z})%U$o1 z^$(UU{7e*@3cLgQ?Hvh?RblB-(&Z~MWD5}?*{l?r&Ec3I92y-XL2Run>+r7|<+eV5 zFJ0X#HQgFSxCvJ2mLI6*MonYN`r_Yl=^UMrTZun%^ zF}62ba^f_bhhH&())&8Rq5XFs7G?5Jv$mHtarI#zr*!qqA)9Ld^q+yae?si#(rgL z3X!6;#hxKYPvnq|zUJldIx_lC$E3o`)=3w@G~)tER!N2oMM9dnJX2qEBB!(_^~lPO zb*B_JOB_@oxH})1&R!BkjGx!An=#qFTj)Z=sW#|#IBd!*-9(R-<>#Iun3Ih%Gf|Q+ zG)xTx&pGeAB;Y$+E{X{6{cdg2w{R9}U)i7YV$O^*URAu`GEAQ3XTE+elRb2&5}dF}7M z=KS6%(~?l;@~8KlDR^!%XVo9Vnpb(k@Yu|3(Jh-(gfqpaMpqqj)y^oc^~T_3R-~aq z)jwHGlZ#K3LpGT)YQ5`0=b(^HSNcAF?a&EMBjO=;Ss&q*mMo`|{T$%>Wr_HDpKb@8 zT6PpGi@QLav;A-N-)$mW6vvEcE(-3;2V|3Y)w+cXb_)vKO{K4l=3Jx|5Ag($&PNm= zNe33j)LNAeQ*(PamJ}Uk#tm^ek*X9$P4^uT1E->M z0|XY1;CO&y+g3b65L2|R5R_-y-oy`UzUr!`tM-)H#00ds$0+>MTD3W@gySlP%(&s^ zi7oiQS)nl_dr#2Op%dNKqDb#_(BRkF^dDhNxyCdgG>}sx_p1XsmTAQ}TfDzM zM{qvCrNy&WCLDC{qa;6=mnQw{M*y?qYVh;dCESsTNMlHyzX`}-U>^+a{!6tIS2G^C zZJg+W5wb~{9UcWQ8w`V~sDZ*tEAp4m6Oazhk15MhD>V33HDMk55i?i(FpBahCpGQ- zr{oipQ(uRPd5<-h(cV?xeB_by# zu`suE(xs`#1gFSAI>oc9=iW^F)7Ul{Bgt<2itut!a%;&Ce11U<0*>ady5?K`&Tg~j zd6$w?lc`DTXXjV=V;m^J&A9_TVGD1M%=yGyiL0F~x4+i~#s`Rg0@Z&qdB^bjw~ATy zbYCSkJZJ|6+z;Z}=@xMt`g+dN+$f2Gf+z_A7(QiR#NRXJB!gDV)&$=bA!95T`d7)a zh{;9bRl-uu8GPi2@_5f-Mp%3OO9_%xE15*mM-$J`7xgvP@6eVZ&CrN$=Bizn^=E$# zIiP6j=kqO%<6bKSo<&7?rtX>OdGB&*$v*CJuk`SF7Ckjmq$|rA9u-C#?M?mhhWNF# zIe$h7^V#{Q?gy-{w`36?&j$Nht_RML z{&qJ@PxGcQ&DFmxJmL6of!-`pc7#Gna*lQ7lKsOiu{Xt`ri=ZQPZe&U#(E0JJ!j76 zQl{Isy`X?unjfqWwwoR9DdIl}ds#i5*jUf|wv`qPu3#v0P_V&hog1ghmMm`DniXce}hEXD)02`Gz(j#{2NX!kn@UXtsrE zt~dj-(4QAxlake--C(fdabPhk`GxeZA=56w`U~m%%+5fwvTouHM6a${eBKZ0Wln=~ zgHN|!&&aNGtvZ|EY35oVeK>sspqI6r7eD{R_gHcs=8oPHS`?3t#nUxqax4WWx>mJ_ zRIQ%EUw{8gI@Dp2!Y-3lKi`G=(W&)fEu>M~@Nc;Af)0vj*(nj<|rYJpPIziX^7J%;#nzDq(vbaZ53=(7h_TGpSjw9 z$8cq=BH^%S{e-`r8a95v^y9)}N@wp|IdS!Di~(0_j;(kI!KYglDmZ8nGFVRjPESqO z|5vEhUvK047cN?m@tXWRHI}&E{Djy`6{b@1)S#6R{~H?Zk$R6s#Wq`SLe)1h>3as!)|j!k#owNcM`-}8Up``i!r zez{+Gc-Ee4tvSXV`5R--718cWLer6xrQ&b+TAY#+2?|q&;iA}Q+Sc7MjDLpCOW$HO zT&^sM|L#B>k9W@|wZLplOv40AWe!Dw97m|ge6^*X)3ZutE^V4^h10pT% zeL)58D-to=()Y%OW44;IdT{k24IEBOjj5d!q24QwUQvQk1Vgsey_xxbgaRALx5J!XF~`XG%1)-~P^1G~ z|5*5(eEA`F+BYUtOc$-@J%=@hN2n9`V=^b!6%UEg8|HXD{Olh`9G#gIy|P~_tI=Ad z4F1rICL7SRO+o?qoC6ucUX<}Vl0Kjg!PQ=eszXnIH;WUT&@RWFl*gebWO9A_Dt)bf zQYL+ZBJDkKZHG>{*-$LS@TVeq2PO(#aa(jjO5f`5H8J+g99}y?f`ysEqWIkDje980=y8?hTPDfuRe}!vFcNK0LaJ?AyA6MFF`&wH>{Jh#r;R(B ztK67tGs#twvpYtPO~J-SI-RIBR{YOju`D)xs-;q-M7_hPMoId`{25-grb?B=VLvzD z?8Qk_ouAJ2$5_)XBX(!C7#t#dRg!P9&1-Q}V`9f#X#wam}m@s^m{I$ z;$7q5WeIt{(^Z2u{HS2EiLtVbVwR&aRW-AZ`WAZL5jNXODp?emB49@*%jjm$J zRcVP#7#3WKOy}!y&ZB1X1C<0O8=nwTB=qY~@y|cX4z`|gfHX20g{(eG;YZx^eT^pM zzEhxvS90(t^Ia#ihlwn?IpK5xsLKW<`1WK%Ll5F4MSyAye8Nsm%(UkK+IU;=>u_Xw z*$cc*$H^`Zb1H=32)ZxogMrx@G$2L3A5)wx7Z6CS)+UuR(8>?uTG;BS(;Ao3Yl}L< zE-WdP;f|4K8BV>AwG3J1r>1z+c%?f$OIZ22M#aRuWo6^w@GQr*)Vgn)X}O~d!ok^_ z_&nOogU7@zE!1i$g5mmItd&2XnMJgBHg2r`EJSR@d6j=N*{e`z@rkwI#);!;shINt zk{L&6diZNa4UF#b{SWXR<(5P65!N7%3xYKr4l)+_H-n-mI?6dL0K{&AmKQX8!ruxV zma~aX9%KDk6F7^n?f-((N2C-h#&Dg->GTf5W{tc|?2_pWhm*+j{!LX9h)K{s&)&6#irJ{e-26Q0wb%uVyIs5JmO-jUMaZJ$pfhI(Jkzj2?ftH-zNmviveYR;VUr zH#WGs$S=n!H7s8+*d!XjzNn7()MP~_PuMRgyzs2Bx(ISYyOB;nKG2`LacJ~R;> zj;W(>8XDW#c>_e5qGz&ECRnjPBIJOzkhZA1yozFJXuDm&6b06Xa92W#tUR(&z9p|K zY)SD>d(Y;t3R`-R2X;O|27YL--&~!{)w(6T-D)4hvEgt&$N$ZAaUclR0nUm7Qk+}9 zo0y-;Jw2qPJ{^W>m^`CBhwLr9%TnpBN_9->>(YwG6Wo2)_~XMI>a5y_ciGJMMgW^3 zTY@-5lQX(KN)dOL1BO15^jo zGZY)(J@W>Ey2v6IUdh5tt-5L_n!c$=z30o`_o7&sF3Kr(Q=JPH zZ$ekSCQCC>*Y5>R#-H;2Iis9kVPq=Mz&?FHhm2_@)sK%u4(=71sl1NqzH=V}zN-xZ{?x9%~F_)JPj@wpHfMI z$HOGDGn=o$Q&)F-p6nb$zt22Xr`w}z+=7>-v^!*kP1U-g*By*tKa*>+Rk6exjR|#rmCptjgurPqHl4eua91E0YQ{x>AOvY9W=3t)VC z#l~{HF6GjFgNzXetsO5BsyK5d+L;vK#cB5u8+-C`2`;fJKXSd+vgeY#Ic&1D_-U{m)E`D}W1*(J;`&#kaWt0|R!y4NeoZh&bwDVgW^xM>rptuEuREBNP? z(y&GPc$Qyvb0=o!e$WI^DpB#-w}vWGvT+w!Kvlhp=a7tP-n7I_%4x)G3&)bN7YGFC z)=|bN5;x{ox zQ8YW|0o}2xruLW4VIIZ%ve_N;Bvqp>VeL?^c5EhxA7Uf{ATBb-<)TQJd96e<`H)8` z97?7^np<~+4Y{u57RX4v0gY{b;7hBZH&K(}xZ!-DRo9nblGAfAo7Zu+U=ng0XZ88B z>h86@33+{PuU3}Yke+l_Mp@&?4fKNMeboj3S>+k$@yJ|e#gCwwkvg#ho^1R|D|d@2 zWqx|N1$~>H=cTCrEY2qZ$2TF?Qe!iSyWOy<27@gFDVS)Ec;xnBvdv0h2tKb;kAO_} z{d)`yzGZt~uz^Z8S46JYhF>C@Bagdss|1=O!1B{sx_S z>3MZAUqm?qy1>r3vIDPx!59<6Opm#isE@bKJ@7Fqsp}FmY)d5 zr_c?*+XR2&)BcWikX%JN6T`3b1Kr;4+yF>h6H+)dtEKQ)+otC6Q@VzpQuuyFmxfol zKTGV7YJadAGi>3AbCJDUE-}MQ%KziFpnFat3+PWK8*xu^5bO4ETZKyc3P1IN%;EeF zyv8oIeTJl`l8^7g<;0QRv*9(rq0pHB#sfsm#A9t$PhBH>-HtV_q>cke6ARCxsG@VM zVABo0ct^-MwIhk}SbqxtVP$gr*0}|SD+QteHkQ3+sl@$4^!BQvfoZiW$7NdbmYN%nup} z2?e5=rFblA30k>?$D)b>sKfp7WL(Ajvx6&IYa5SPdK`2+Xlhp}J*PGFuataU zak?K!O|pD@YNGLU*mfU;B>d1L?IO>1c%1kqrORI&s)!EO9C(5i!w(#S=hW8Qhmgo` zt(6=b=NPVBkDMW+y;n4*8oRu7xI}NZ@=5jjUYnYHcV16SYI{4#v~SQ2L#i8h5_TB5 zZr5=5B*OF1QNPcWP1ky*v-Z#)Xs(FbXvKA$d5JIr>MNd8*^M-{np$*}ip;2L3>AqG zl_IsVC*_c_JEqlfjIqd`D|A*xl7Y9DEKK`$ts1P)vY6@?O_JdPD`2ci1&X}z6;Cf7z`;7F41n@V z#UzizQtc_43NGRAtvKo1_6gs8OYvGM_*V3wxvV#II-h&}^y>Sa;0%myPU>pCq| zG6O7EacB5G!xP^y-TqJ)8 z(s1~?%H7+Z88q4`c3M>1=!{p$Ni@thDGwQTD@6RLg$=)LgCGWkPYQ!v-JURc@&pny z)mzZ{4arA@z}@YNmTlj6tr&*~La5#2`X3~krcD6F<3KEa4K9R^9Pgk*$;cnk~6lDx55lBy{d;we?+bE~PL{GtJsEZdE5tH`*kLZ`dJ zb&zp&1=^~MR8X*AJ;U5g7aJQa$GOKaEJn(Iv zrw0b1iKaUB>PBk2<}?A>6#KdfZpB<<`h+ZW~|l6w1*-6+i-jK#D@ zt0CP#gd~#~$Vic!09eQPMkSHV4v;YWNl zXkIfoVc^jFhX!7vd5Pl;8&IZZT6S!F1^7NAI(FA(o!r9wh#@@Z#;$yejm041M{!0F zRSgnA+vO&L?qNS;70Kb}#~{ei-#=TQ>QVBmprmDle4_6e#V-5WlgBiOhc z%?cSdc;hTZ$eu{OMz5PzHD3oh4D#=Zs?+x;5k%3kb-T%{92df}QkA=c-KsxL%*U@$}3af1=>I&&F;DLmk8BSc(?DnXPc2?xI6@#+r8$>GAU& zspx&NcXxGI7iIglQG%C!d6{(jCxa$-#b80&nJ3)`bJs$4g)K`-w%bG$l-x%Z&9)t6 zV;VwyIwXuP3mm@I-cHvKPXaWt1=VB%7kE)ZR4yTH9;!1$l@+4F`X_lK1dtr`Jz;Vg zrF|7wG&{PTilVNBwgx@J`4IK5obqamuYoL2_7a>l+=iER4eF;pwUxgDs+RJ}d6A=7 zJ!=l_f$JovdmR(aKr# zVE#u>->cO7k}`AG4vhAy&A}wrqvJzi?*tva(pHo_XDd0Fj8%w$F4w8IZoaMaIHP`Na=eHQH@=N+G~pe9@e6j|XbwsW14-!yY z5ay1HNOJ}`xT8__eNikx-AE$SQ6jpX$OtbCgZQ#q_`RTaqIg&!&5s*Rv^nLyItDQo zi~&L1LRNfnF5fTPK1L@!U5sOwHSytvlRl&?1hu$_``O6AStU!BL!A@<-cE)O=shjLm0Yya>A}fAuY~)f5o8A_$VpUPaz1@qjFSWRvT!5{sF=IWhU0S|(>AJ+GS7OcY4WMjy*zmG^*ZxG_uHjGv&kE^WI+X^j1x9nk@t{teOurc)S)l_@vJkqG-ufG?iRBKk!-D z1YC^?CQF}rWN7X!MkdNkR(ZdY@DXYqJOjHn)aGrS6w#w|k`sO(bld;x@~%_PZO1kf zJHN@$J6uM$vx8St;BZ6*4t-4_|7;#DWS;tAy_@u#36*b^8Nm+WU+^p(ji(a~K)^SK z?=#54QQ`gTl_-ZTTk^?54Us|}nLJ2L*!K|@0J(F#4$Und?qI6mGCGI@%4rJ_S{nXG z?$bNm_^tsWCa90HO_tSf$rKkmer(vGQ>v7ekWpM9`UDsRlqPJOfS}nb$|VU(PL%zN z_hZ~~D|Pj$;r)c-lt%g~KgKuoa&q%0<=H&FURpb8j(#VJ-XLDa$Y8CgiRRaqJ?-1a zv^vk*+6e>eXC7(|y-56Cevr4FdA1qd%)WjtV!5RrgJF^Q4U z?xfmtuNY>0InG=d#5ud1HkJTr_MyZ^of`at{<`cSOT=7V#HD=m>utaRi!^tj>SCaI zkNrrl0oo^{E1xh4Yw%OJNj0zM6&JQbyhb_6BchA+4&XpbfzDdA`FdnYY6?zQwxe50 zf$Ax-+u+Q~~q!&de4mlh^$x+jXUu&-(MZqoP-s;nxsLZq7 z8vlpf9ZlAx>kf6W=9jU;S9>`5E0BCnDF2Ri;S`fAMT6K9INbAqXVR2=458Ae$*+~Q zKF)fHOL!_YYK&|zK~5|W)y?{c`U-PBiA9#f|q@xhT#2SKr2v&suWa4ipi^ z9vD;}h>z~BMD`iV@b%v8=K#=UU#pH8t+f|C=#gzG)WffWHjApNb&5qmTyk^E&0G0+ny$3w|4*|lPeK|n<)eAgZ<+iy-O|NPv z3UWB*80wjbt>>orWiSiy-P}2xXdUZYup2hbsrzbb*?4*}0s}w$;3Ui6M0fx7Pv=V@ zDv>?VApAwf4J|-xDka*%R#ZDx8zGxsFBvO(scJ47a@sf0Z3(YxOle$Gyj0W~r*(xIB5yFG*a*NlBcaFAk2#gmT=EB=1= zy`3x{W6faaYM1g4KXvft=jII{xpMKuRucNxvB)?Wdz^%t2{fE_*ZKc8fv`et`~N|p z;<_~Jzt9(^ibA$#;-aQ5vW%3VaBMSv_j0@nm>HW^MNVz!^Bf_7HDP79I&v9R!tebX zp?@vXzYzcb^|cfc^a9jEqS3z={I>L57k%=bS5F*%e4^)R43)iULKOjg2-uWu_z~Qq zA}=q`O7MKMT-_*&Mh0mN+o+mt&224B&4I3pT&%BIQJ-J`7N}dOrvxR;78)aG@0GS) zZ)9@roe!iu+Ja|VRt@;H(OGumz`OxFCUT+hfT%m-76&Pi&PY=#zOKwH`};9J|7*Qp z!u^siIx6ZZ$xF2N>VJtHfK090NW}(z_~#8jW&M}I_5jl4A9d`=+R4AHY05`52g!tX zex1Dh{F;``j}SrCzcg|R4+w);q;8V!f4OI;T*NQ{TP*O@l~dX2_roZ?0c}`>CUO7r z5C5)izoy`eIWVt`tkM4K#DI|*Y;P<+GWU_O-fV>Vt$Yc<=pQeGD*izgI-8=3N(-mX zZ7(Jj(ciCc_5`S#;-!`Nec{(blCdwXf13;eq*0LIL2LULMj&tK%>PN* zk|TeK-R~zgR{@lrI1MY={J!#gg8#~sKp^MUo51_}$G@|e(i#=w-`OoAE$!kj^MA5t zeoyXQ-ieNjBLOfCM==$k8T{uQDT9Dn7~PL9_?wD5(C2DCfY~jzsQ6nIxyRCE0?_B-^*yk060p2ALfwX7k-)1d#FyLBk!dD zUe852z=+ommK*$@_%CJ^V*(2?KCcWcU6_I11pD8N+6f0(Cj{3Z_=jZv8<*=DffeJe z!p{6lQ-4z5nd(zWRmPVzj~71o_x;VEfJe$#NdDyoe$6%q;32_!RHoiU)wP}v-W0rxuq;u{R}0dT zzD~ZrFinhF=EMai$3Ots2W(>*W!!(8Xo@6SlFN@z!$}@{UHH>EMO_jK%s5UoYfMx0+}>lrXrnIRPRp4cVFiFhJrjh)wgt|fPn6VE}VPxja&^%a?OK`^+k zf5V`#jJ6VV+-|_OB{@Gj6JO($^fjg1=`oS1x0t(R$!lTAg$jF5t$Y*BBPD4tni%DzN z+T6n89EXI<%p}i|H<4>;Y3X{Xf46D1#LyYZl<16F<~8K10LehY`Z0^-DG*ji1H0_L z$beyfza{zcy@7m;zR{s_F7g!meIfVF(heu-u!=6Di2e-T9#BH2bJaz@6sph-Z$6+O zutuv_He5&wZz^hOoy8K9Fh7V>N`D0|NMzXtnhWHb%CfSSVXvcT?jjk)0OTly-BHLG zBMcibQFf`le81(y_c5xN|D9YMzZbl*MsCX1G@ftv&zD`9Sy)V#;~nTc#cE0AgjiTw z1b_uki9xSuG1d^ftTUTD4Vr<$AR0nIibF)r{^Y{9qwc0q>+Nmdq zhfTCRWXXWx@in+N_pI_E^&F8coFcJ9P41Gn#N zd-YMB0{h+5EQ2KV5`dm)-5W5isZRy3opVS*Mxn_-#FXa*sVAD8NZZd-FVt$AX5ldN zczMhB2OdCLHH5cP9O$M*t8Sqcg?urW+!)j#&u?kbiYh4W^u<1m2- zz5&h*E$?b26f(o;^K^6WL1%gV7HNqU36LVvW|H*=?G6q~SW=m-WQ`aYRFB}3W>IzY{WKo%-MY#=EC7-EMJ zjX!@Iw$8NXFe94kq=;Hun0B{5b}Sg= znwtKC_Gtp3PzOoj*ijY)Fa)wdGmNQT2C@uu!y7y5uZB`pxA5%*km4l%05_UGFN3H2;CRJ_{oTvw!3W0~>RI@Q5}Xs9^l`zPC_tF&|eW zA0SWyW&qMR0{><#`h=Y#95NoNTAnl4N~h|nU-EhjRSvMf;Pr3xNZNon=eyU>5D8Ed z7Y)i8Bu9SC-+eU*{2Gl4FKQR+F&;?~eeZQ>Y|OxgR8AUTrR~xB%z**ws8PkzGH-uQ zgF)FhsQuAiqMB#<>iVQXU*J{m`S%AU2PCnH=6@E}30e165_O^_1Y;Ax(Pru25sMACyd>y{sTTzPHji39OkP zFS{W2?RK?i0?A!f@1Y^RVy`%E?WTAkGvzdkxt#`ZvFWI~n|w=YUdtW3m?_8~S};cg z*h!b}A9}O=CL|R18TfpM((*YT{FJ!n08|Lr5%vdMZBTs7a5p$zc)u|W{l(M#Mbe3C zC*1wK4(rt54ey{#%@u!rySRXfY#w<|A__K>*m7Sw%STsZ``27wA*l%{JzI;X)aNBW z5Ek8ojJH7Fcz&su)%}-^uAqFgfvKUQ z-9WV*c_Wr#WWL0i__VE(k`v!(0o%f$fs93*hLt%0J0k|DBiDci+Koc{CSYrn2{9&J8I7e+mNI!r8Tat)kq#>(F>q$Nl z_XLTPWL})?+9cKK(; z-7vY4C?<`2oU4}bvZ{{#jAr}3q~@D%dzw$3x*Obn6zc?ng|#XSiawtnv4T|K{X~w} z;4Us#$FrOmu!T1M^p={vUrw z4hmEQOYVCFB#@O(p zK$gr1ms4o_%V7V{$2GGc^T;ocDSrNRTXC*R-z$2X&YLY?tw_4giAlZaYnQ$!(`bI4 zN_PwB?CaCM7u39#34}dtJn|djz%Hl4zD>=w(bz%Bi)%+MlXJYQgdKmI9w}s zug`g}KOTSS)U>@Tg;V~^FjSD+D%HYVtw->>T4BcJlx0nEP_(- zYT5KEG~|Jg4wo4X;<^4fRoa_kQs_}xporiV^3YuFOQcuV_0XMKGN6Zzj&_Ng>eFQw zN*jMSE$%Ud3Q^+=dCCD<3Gvp4QgB@twfr!z=Cc;u8VE<5oMR##SP0yEwojIHay&fm z$u_$rS>Z)L`BaHqmW|o>^)VKFa`T$ysk>Ut;*EE0C67d0-Ti<<}mPbyYEKNHjwMt8(JH zq4WztQ6^Zuj&7U}!5?-lTWNHEtYbcZAzrqP;vTU5VhxtAZCD$U_+&Awt|%IMU=?ZI zm@xHBa4`OO_EH2mQY3#W#ku@z6i~ms(Y!fF$QDimojHQmklG;&AuTrP+El-%D8?-UE|(HSS9(0Frr9tMUNH%tdq8D8!Bs9#@URYKcLgHqnrXlZV`8i4 z5g19-+`8hYu>7)B|BcAp%hRa8;Q3Rt=C!hoD0lyb2VzB5UqWy(3GzgqfCGnlzJuI~ z;aLVZHYJvS1^4+zhQ0B%E>3uzbpzW}(pKHD@6+YAQn@@G6%Tp6{k z(0(9@9_7O(vkev;m`*bhT!%k2xOctg9O{spezgc0z^**@zfG+#H>$q(X>fwa8I}dz zYMr*kOpzFt2g9UgRitHI3s0HRE}mwkd8B3iTD#nkxR7OiSGz$wArJlgUUozIVTrw6 zxCOw%o=u|od38NDipJy8i}NPjF11NZ;-V~K==uElRUs73MYO+m?(LJ`N&0m=2k_11p$JGej( zW|;I(8Jfu-$sNCuAz6%0N(9&QD>Q_N>j60^r@NGA*KQMz?Da z$z^8W3{IOmwj=KHXf=f|^v7=v%EtvQwP8U6vJs6dMQ5KY$CkR3pdSLzpPru4woc++ zC_8n>3i`U7qj%&T&H%5Qe(Gh$FPHdCO}E^50fAw$yoZB=HU$)V+U|EkwLf^C++N4j{0+JUo$h<6J#pNE;GbZ-JlGxFH0#tKGzsx#5rIMeG z0e+>1wDskimQwS}tlnj>u%Z4J)c;JpeI3vm<1ZSfycVYCZRdKrwP^dk-(jkBdHVx@ zC5zK!?MvoRrS_POo;*;%Zw`W9E9gDkS1#8q6)WJ937ba)5-ar;AU<;BlM*&P($NI0 zG|yH|ctqP`JR5z__#oSdWTKL*KR3FTwZTQsovXS^a1=2l4we96l%K@RURwZT!za(0 zbaySwp7%M(T9<(oB=R*N70P<9yWGUt2t?omCL?UH1@G8=M>kV*Z2}Ma7X>VLowOad9(0(oti-7g?qq~^boJyjbaMWg;j_4o`E4# z-Yf5Ojl@XMZYxtJ^0a|S=eaxu%;1yU(7eEvh(b_S!`-hLFj(AE*>kVj3-AEC-S}X@ zYDsd7aYP@?0d{dQTs-l_L~UPgh5ySaYIqGj+Ry}B0>lqkiub9AUj0dszXtQN;iy9+ z&F<0LEC38yM`Alx;?h6E-azR$r+W|Vw&tkpHhq9Ez+FjX=o|w9NQUmY(_T7`Zdu=v zHm~+p$C_}L*7Vqy&grK>EY+h(Ur_a}cK)HY9#2-L#X<$M)%Y{r?1+Pnq9*y13Bjhz zvpUP#Q#vUk&x6$K58GC*2Ad2Z{TU&U8y>RFb?x`Mo(|8@H4SqnX=e?)SwU=1Qpa)^ z%iRvmi3N6KiQRyWwZLH#QM+z&4ox(A#Y9WMCvc}Pzk{?_xjgH&2A!6(5zI#bX-$8V zma9)4;g*zf(`*2w7RlHJS~b%B{Em_?C*jz|&t{dty{bkSWKKTyr|Xlp(e4|SSav(} z7+n4KagDa$1RA;k3lQF5W!PPH8WrD#fiEnZYmPq~_qgM-vMIfvupFle$1?)&_NPES z%7|a9)yOMP_frX3s%CVAt`~cDrqjFDj{qn1VVj)GxhjLCehRj1CoDM{JACPXfJsmb6g2!A{cs{y9OJwQZuu_u z>Ur;rkagz|^#WI_nl*OD5#oLOLVE6N!8;eL#DMW*i|;uG)`O*Z4BX#1=iqwM=pB%} z^rgTm-yx>v%g_n0hWEv{Y+Iu3k(zWse%NeRIPL?udG+8=GY^d}B}LL>9N%y}D^q*C zN~dsfzLqh1yQ-3QcM{$fMA&nXE6mlC-dK29IdR~j(AnV?w#`?5wTSvV-=QxScTz@X zJVts|jO~V_9-A2Q$D~6vik-*=dk>ar^)py}Mvzlj7%6k*R5S8=tP^oOOPz@&A0lCT zz9W3iA_45HielD-KRZ=8-(-ykm*(o&cd>Jb#mBtCxnOIWGhb^s?=Lgl)(Zypk!G*O zGbg0uA|t_~F)+@viqOr9o)A0brf+*az1;L0FH*xkyC0!HUpnC=*;eFmJl*zw#XF%+ z$g1&J{Z7p*J=gisYA*iGiU!%Z9j^s}EhkXMTrd=U%3LJ@avw_=VAi!t+@-(UAsw;w2oy*-kOKR z4{Ob=8;pCG&0UI@H|lpy3X1jF)ve%j{c?M|-6#y!mCx45dOW-9Jr7!TWulPv;=QKP6wj|d;u{rO=^7&(-8x6Mvpzldb#6hlkSc5?axTgOZYZ{FTOOl3r0(F0u}rj| z!k=+#Cadwfkn!>fb7;+uEHZY@-2T=fBi<3X^K@ax1)^E&A!}31vaj?C7OVYl8K}ap zoBO223YpQtbn~u)q%A}t0vjK01lzNxKh#ywrfSyR!f zX}4$x{azMg`cR>+GiMODXO%w5;;3!hX5sGQCY5PtGmXNXs_Cr#rFT93_6K*WOkz!* zQ6+9m`sW+%3Sd1uEy^m_w<&--w9xT~IF|{^_Ul>m%Mk~yX>ccQf8?MB+`Bt!l)Gj) zaG2W-I%gy^_njjkWK0;|@p8p`*@{=fyLZxkpS^vesV}Op3eIdr+{y(&vW=44vK>?tOMwrVF9zb>NUA=Yp_Q%;{QYY-6&$krWt z6SJd1Me{*yAx&5g7N_Qt(c&Qg;@cF#d8 z7KWzoTl>~BSzY2W5a;*A8EuOdSt%*0dLy~4zIcANsSxY3Q@StG=Tk=W{Uz*>o`u;stkgyY6i+^2hZK)wx?Wo>z|*n?DW5wMF;;{M<-Gq~Og#uh%3_ zcqP1;RFJWFJE9WzC)QHAC#9CuOV{&#Z)Xp^LR{t^M@?C=G<%g4#$)1RMWvu&`Uc3;@oR#_sZu3i;2t^d>k3 z{$wCKsrk&Ak^72nMEF1bd5JYJkLUMI`+s6-f5UZ@WWd{olnjagw}YgCpo`(o%&*}1 zKfiMgsFi#PA?f)~Gx_V965vo?zTNppv)bR&LH;Kxk;0Ye_g8hyj_J)O%LjGs(T&~= zPI$@pC8gdt>N-A~?uIWZTb?U{bLe!A*J~pibI$YN5BeavV04HzPdFAUrUrkj$sX); zcE;mog)9vsA$}r_GIKAN)ZBgH@Y!g2=Oo|Is1n){4Y+#Nr8e^=v?mo(bhZ6Lme{?` zW7!ZDh3Rp_<7MR;Db5bb{4X%9s5ehhbFs$_crXX)^`)=kS`d+dc^@K{x{b-#EAGJ9F4SPhIz3_SW{&eX0ss zD-E!m0IyW+?ex5;OdZ|(;fxy)W(&qhq*>Zv5^n}TNr=sq~ea3qV`@_Qb|K4YF%yz{IS zA7nn`EdiGGFu{rK{Gv^OOehZNSsFsBJZ`?t++3a8soHYdke5uWdy-y~h&g$>phX<& z%wvK3l!I~AxNsS~LT3SHvv3~INy(R6vZGNX$^&oOOyr#x4b@hFt<(~-D?cairHs2_ zoDsq~W>Xzk1R|*`M_G$CVf8KI&tA#Ng}(M-hW>465)#?KkM{*me)!M&4$R`8VB&uq`u}oDepEZ{8`j-# zG02r3My3)XrxbuLt8D?-`Prjuj#|zJH1JPtzsYrTRyM~DL-uL^qbz1{|HO3}x_78P z+u0q-G+eX?B&Ul#(5A0Ir%&eB0Kicg$N9ZP2Ix@~mOxxO0xNRT;-1Y;=5!D7IP1y= zFdWp`F~8f363xNCQVcJnms2IST28{_F%VcqO&gw>X^^o@wi*;J9%QsM7D1cQ5gUu9 z-6WRZc&@h|#Vp4N1jH)cTtT0v?Ci8&7W1iIvx}*%S;s2|PELXvq<}NsF_By8#&Lf- z4hB?Y<>7R57`oUV&f5ivx28Xyan%2=H8wILxl=n|CR04Uc%ae`AeD8}kU?YCx)Zxx z9kp1SnsMT+_r^P2*s2E78{9+37pW#(fozv&(3i|PM&LC{QfJ+W5sPMP~|IKaM}* zIrk3gm5*9KVh4e&gWox$A_|-#;yf~Y8&dBx_u8W4i>Uh(wH;-myR6?szb;uf&=m5( z6>Aeb{m~Adm9Ga^eINE(x6iRWT60&17itgJ|L}O~tbb9|kq#VcAHl;6)T7|s`LHsYbja@qZf zi&q%i5i<==kx=1t9k=bFx~_|HTUeH-Px~@wEP$pUYT_#Dsr%!ngv$N#Bn6bN{68^O7c%CX#%~@^!B8wo( z!BEh31it*4#l@W%trFB+uO^Un`E^z2rctgHZbZ}(jKPI|px@0@+o=38;&?`;n+MKN zAJ`UpDe)pw)mFV~FNn>!t@Su_!*4|@bbgHc3W4xCT-xig^^RP!ZerRmY7&PvoI|R6 zBoqqV_7P|OQHjo9liUVhrhq4RHX-htboK|t6yA5ps7Qldvuy9*2*$1)Ah5f?1vZ3G zYy@4OC5tH~mWqtT4L0j9DHY9%m%|V}%IVFR0#|x~_K&VgN;Xt&a6@FwLpF!a5hbTd zuzGC4f-(Jujq6quw9Ci$kMHW&5-X#Pp6+Xt*BTuA>HIngS)u)$>f6(0K!Uq(!&fu9(C6u;ls)ZDF(oaT_0a!C*jGTsl`UOQ2th&ufdC0ka3?r4?ry=I;Eh}3 zt|7tQ-Q8V6aDq3k3EpUMhyOB@dGp?9|5|tTYP;R%+*7A&SJl2%Ue$A^htf%eh$4tU zh2OOM>n+>h@5g9-*WTwNPx(G)zg)BNAv;}$FPd<-nNSvaqA0SLO$^8Wx$qL%Z?=+l z7E7d3KP!alvwGoYxaTA~Zc1(_0h5QW9CK?2_RdXh+V<{ETN@XZEi5%PEDuX5i^DA} zYxo(|imN<`~fQmAA2B5_sBflBhHF$KN6q_R6~F&{5pCO~zU zW!QG_m$_qdDK@1$PR?7_VRrI(yHC%$97#ZJ(Uc2cP$T%-tiLu>ao~<7<LybE-=zYZ*F2*oDQNbnlGYF!ZiE8sGqDhTa+ibvElYF za_PB>vRDnW8s61!Ww`3*_ z7;=DD_*nX9d+9YOxZQOr#g8mg@-5U-ubq6?skJ4ZL(FXuxa&omomujAKuPx+Rmsay zib&3Wodf1a``y$yE20;tL}Xf3y~}*HTJF$`3p7s!vYwk*%^lWph-dwzIaP~IqwF~v z&O4Aw+NQ&Gfv9d&s^(?PEws9APejY}4-4Icc9_#ufePYPfh%_~D&L9U%t;Lk32M!A z{2u4STUlI5WKK+_s$PmPpgGbFvdcqI{N2nZ)?YoDfX8p4UyI{PjK_HWwWWBSPS=iE zmf3jQ=hedrd1*|K{PV^M(xVe;&uXDw|#^2iWe3NOxh0nKmrysx8(eEnx7zk#kif#W|xK$ z;Y`>m4^J%86L%57xqPG?XHBBkF!!28_lK+S_R)NAhM1K!x|26vYl+n({X-NFuv+(y zhxQM;$-lB?rMe9QNS}DmW0qI}!soQH35cF?11X+o*3TX5wp^Qz1*6RbJ&ATl#P1^n z+%FL?GQ3NV)MX(;7tK4GI3SNtLsxF~+mqVn^Xz9W`*L%w;F?g6(~k)brXGC4%kHMt z9v596eaXtfs07gq)~z}RL{1r}t(!Cnr&)aZ+;}_O`W3qw3$0UB3YKjryX9tIJC7Lq zaU3}_Ixl8c8RUj&(7EJglqP%6t+jHtsX4(PMf=}k)_<52F6U$%myma zOiwJ>8Ow8BTp$btM|qMT?l3Y<6-(Xf?XGXHqmgZ_Q%9&s_pnqwmD_QtEn&KRsbv3^ z07IuLyPN(-G+Zv3q~+JDe2=7!)DhlwV=7UElN*{Y!so#RH7R~hna?Y;;*PmfQ5?%r z@S#Htc3T7(S_Sp%6Q+8_`s@9;LwDui)O-5r+YI{ z8t$OE+m^+G@roY1>URrq_71uS6%$V;CDqbhqc1u@r<7=%iW!> z$#0L2I&)#GVe?_XWF`n-unl;&B;r{gzrxg8HgKX8jn7XV@*(6uYJTMw* zLvi+qaD1y;XTx6J%@1kCQ^kV`D??GJR-mSU6vT%vy!m-+R8Rd+I7XHb&{)%o2Xz`_ zdw1e;4K^rM78>>0Cg-lq<)bKz#GwAs5wcAy$&6ua76Cqkr71x}5VtkFx z&J(;JI8>iIH_05w(f#DS`$b!UlOs0Rahe*7Kf8jYW|}VN+fb*W#f2Cw^Gs#~>$!A_ zlx0C|SF-oQ4v*BeZ1;08P-~A&g}m7&gT`9~CCYmXnj3nnpizMx#*a zbCPAKS5P}X(vjWs8T2jHnXogv;iNs4a-m`07QW)jp-g+RJ#B5)wP2}d--vv3qu#&m z*WQ;8RWjt@>B_*NbT0ADp%mbDMo6XbxO&igGX1>Y>&fAvI;&3KvNTJl*Q`B#+ayNl zV%FGKyuC7Uo45&G!hUFdyQ^cB(0S1uF2)8`=_pKIbou`Bcu@NF5HXj_ugs%{Miq;h z5p!9E23;MlV2*5OL;3&(+=V%XiH3{1*Y8(%`%@03FsJ=#A&2@2-{QU9QngPPFpAEX zw8a}?5(B%F;Db-ya_x@dG84Y6!C$6MM_lK?D>RgV7D6cyu4uK!Y}+%80HxBK=wvgk zif#;`w*4}nca4Z};?x>)6s_1{2n0FD8p(LO@A;z*>l?ab%i-1uB{)robvbHs==6!s zL`ouevg+Eo2uc-T(9Y(^#p2pW|-^=wa`70%nY(kEBxZD0$ zyYkd4=g%QcLodIuH8Ul{xM6kVgT0k(jLEItNGe^3|&xQ+m!sm82i4abe2$HD|r+{f*!?g+H^aOdS+(f zQ%WW#rY^3X^@(2!4Gos2EQ`1ZK0bUIH%!A|n*3^d{{&Drt#+iZw3o~lW*i@IyFrNQ z(tAc}SF`k>Jo%nrwBWi^a1rl)hTkDW^K?jyBdBDr)(6;93a_{{+G>;dkfFpB!e*5s z;`x^rA#cxOJ@yhsHbx`R=@Q&{WYZYTk9Ij4ySM1$*gw#m?cpTY@{VnN$ly)bViYd^ zfw-+~aC3kMBmrpK)7azGe^q?L2RcK0p#O+9Xy1Mx{`o^WFl8zD(g6OJ z$`nVfB^Hj)w}-!8T`>}Gc&V5Jn*~?5&flj^X2HbKi^C>_N!}sO>NF65Ninawr>q3jJOmQ=8Xi+RKh7qL}r`Y1ShEs3I^7dI+n+ov#vn5xZ8N@%!q-S#n9 zNl`=gkXoK)uYcF(yP!vfT?0)Gf5Nu}h!Yy>yd80BsV+|<@US4`jk=$q=@=)~+e~ia zVzS_X$BrXuP^`?JI+{KqZrJ^Kz9}q4CmY3Z{rlXC>Y$yBzZRj=Oj16n48~ELhIrz7 zZP!MJoxJpFxXy=zqCRp)FPkF&(T_0Ho>*u>SSHWgq;iXvF#zNE0TkARAIC=4e|2Cb zi!4FHpY;~7ZYin=^}Yv#gIp``n9@#oQPXyh-tuJ-*?ewF0coAewoDjH;VG&@Teb@N z%8syYpV3(+#b6`hm299O;FZjr#__#-x4*x&ooJ>vz}#IFBfFBWdc}O#Spb%8d0~Si z_stP0hY*Kep6?{!=PxL#P+IyPXL?Rb)gR>qW%!#Y-o~ra%r0Mf9UEtl5aw!Sd~2yy z7m!L|yjceBDd!E}U#I8?f!!_Y%BFTx_RA`&-=a{i9agT%dXagp!tCr1GjG;~LfI`R zN<-|fQmNWHyo_(a_cG;|++UlKHl$z5T(A{d!#k*_pN3x-UGcq=^pQM zNBQMK^SgEMx(Qst5&L8r$1(-SvWNd^Q&q%@3cYT2BPNY@zsvV}yO>(WlM%|@g@$=W zgYb9&Uvn`!bTBVVC~d?(8|*c|^6Tu~Ve`!EsQe&mm^+}`TBSxB!%>)aGR~2Mf=NZp zU4Qts_9G-UNBjINZ1+I@bw6v`xBOi!9pjx?ust}K;lpz%``%oQ8c_b*U|NlZ9GBl{ z-<h2B=oB%*2VdxH%SxM6jTJ<$f zO|QB1!edT4VxFS$n#&|9yI$A6uz+0BL@{AJ9ldXoO(A~*+l@)>%w>>knOAa9m zmxHL@PN%#GR^V%;5(j$UBJug2tW^4Ca5((g)AbQLA}PU{9*JtF(KoA?%WWC;IARK; z$&}<;sJqxVdfc2fOmu+`wi0&(cpevPOZV8!^+&;nnTqYJa*q~ zJ9#n|`rk#yvGQ&WVQcr8MY&wqWskkAI?$sIIw;H&t z=07mluDv60c59cMZp!ShC&65J5o}DB%jkVh(;L`gL!Hzz)Xf*?EunHpgGAoM&nCo; zu26{4^80D$@1ApAO>tl}7$sqLCX01(ca*W!FAt5M_MWX?Pai| zG#QC{ZEK)>b&NvoAxS9-a#1+OWIVX|M4K?=XPbGE1t+HAUZb?a5J`6HITpp?dgO=y zi09prY_r?peLDuSMGq?TUB?;K)Llp@I8RJ%9mzOJ4*OLC(Cw8*WXKHRgE%ijzz@Jpl^k@a|SSmuJn4RqEJ%Sr7yzh(yHEv%l zFm4US1#OSfsaNU;aUw+eND{xb`aH${&D^b~l&kF}pl6NiU#AfAy7$kqm>MrHR`9B> zWt2U9MQ&ph)~0m4R4LLFU3Pc>5XAXXLHztQf-eq{5PuV+0}1tL6Q&f5&G2Csdjz|_ zhN0mTBC+jtpJ?^g^nEW?z~`8i5yz#NM&pkDk@T5?Dlk#YFIv>T?D;HHy;w{cUyJ!F zH2$c7eeS43^6a#Wd<|KfbRfx62>=Z5LN3^+_~@rMcs=)r1E%ByRC); zu&%XR?kq@;3(#rNDfU|5zaYDhU{lA-w+|X8HH~NBnAj9-DT=SKoQver3WfW{4{Q{z zS3i*6Xe}aO`~AihdxFHs4sXyJ4Kq`_@Vki&J|k?w#9HHQNxWKt1cB-C{x_S@z%DkO z0%?%cJ_us8-PX0fFKL65wm$F5ep&9GYt_O&F=17%Ujl23Bj@h&@98zi+3(GN@bNnC zTt~}Yv2tErZl@0XMUj?V&VMxaD|o{(75XhzOlICY!Kq0^Sxv{aPNqwHSEWRSjpVe+ z<+jSM`jrjIeWlAs&DGNfJPt5j1TRR{OZzfr99{htl)Vr}GFq&VTrSUAxgyYHGwW%; z51Em=I_3IVC&O}-pHt4l;(josfE_hc&*iIM_m`m)_GMDHERA21vz*kp=@=(9=A2fb z)^4oScDl+FgTxN2>5asw>3U45?=825gFb9sC1fO6xSzAn58MPT@4DEnj!t8{I1v>YFinQmSq|4hoiC( zv9V6mbPmw$Xz8`^=XXR7X>TBfn4;;FcKd0(Zqik|<(CZh7bth`H~{mTHZ)F?6iAaf-$-iP?Su>yP5GCPhT4wY26u zJzu?0iXv3Q9gR?=;`9dG!H`{*7@;(fOKLuy@J}28II>8K$Bep#H94)&?H+oSTo1Q`AEs9HtyzGqDT(d(>75_5=*tHlK`F6HcuOzGcBf%S;+{LAQkIQyL zxeNX=-<*?9B9LSMPuubE7>7VMWc>Md1FkO9zbKgAIFSvsA+Zaw^RojTH2G zlBI9Bk*rByLiZG?K?Vki4qx!oN8TLb&_If3Sj6m7y9?t)9DPD@xuwh}TP1>)I2H}P zY}#*vWWFs};dtF1B1>2V(k%JERPx#zU zLmL{CbyT?Q=2U2Ww?5j7KMfL2>{l~S^3pb1h&tG&xm^r~VtTjnAS6~x)l-Et1uXf} z&`nu~Y^v6vFeLRWkD9jO`wmgngJ3=}Ckq>!L+=*Mu1QvBwAQ}2^aZ=+mYvP7_vag8 zTvtW{Sxf@@3iQNh?vN1AU)4fi>n54i#<^FrcP2HmFy9B7=Bgdm7DcDZdzs(HOcd#u zYE+vQjV~wL?bs?J4o$VW4fcx@or7ePlK2te7z5_LV$;U($qPNhDAv`GvyYP925VJs9joB?>c7>&!ysMC=sQ<&mei-e8rR8 z^x;zJ{w||LSawvY=6KdQhSX4Zav;KLz0ILy)S_cpX@abhXPRr#4p1gl+J@U9T9OP zRpdry!LFef11w1%kXT7?1oSWe!3jhWG+9vVDJE zAyIXkm*y|joS~6o;6*XFY;wo0VVpFFwN;iu=1rhZlQ7vRIIq%1Q?t%+1kzbM-{)T6 zjv`gZE2~n+fn@K;+`laUg#H9Hl}HKQ+S%me_&$a%ym4HvBceAn=A~%GJ+Q~N1fDbYRc`-Y2DF!v-_Yu_{Zj`?B2;; z4udbxFA4qa((;I{s6xAusHd9zutiDxmfw3IUzm0JY@tqP#P1f_LFb3r5Q5$-Hv)$5(+7f^TzpTP&&o+>Mcv93CaI*HdIiN$ zMRxIJmxaXq@=Gg@E-*`{1)oWG4gYyU>?G~s<@MjS0F2gdn!My2YI?|1W09o~oaqhB zcidlJd+^V)MT(QXa)syoMt*$2Wg+uhvk8(byd^c-SuJC-$#>{%u!at5IR@F5z*vm* zb9{yXAXB_F3$ID&`K48LUz4w-g5Rkx+-ot>M5Z84k%b!vQUaX-_pRMlbi_?1(sb(q zRb{>)ywxamVG1`vjwFz;5hJc)@7Xb(Fn6=a8=jj?s;bh&<*phBe`62|&KfKR-%K8? z*7M1yTwT2=QYonaX*nJ+0u52^O@3C~mAE4*P8BpbI}-HS<5I8T4>6A`<}{Pi%mw9! zPMJemM(!Tt-T`xsGW)vZ4u05_;s+!p94$602gC8@Vh@egWlwvf)s)_5rof2>4buXh ztokx?n<|59=dVlAjiwb9rEkRnUH=_veZctaBjooM{1f@vKk;2suVj5<* z@#J$&m#ZNQlh8+yj|W7f0*1>B1b?H;$KSC>+Wv&)+4T$H^tqLt#zgQNx5kz2Vm&(J zzyU`(mJ5Dex__QWrf*IzU+>T-=X$mg!n}RZ!XwHO`uG)+XaQeUda~y^l@W6H0^y#l zONnAb@KGLnjrbn-#$2D+SRhuJ(U$zOtc%eCBCmk347y#4hUD*n_2aM@;XwPdH&BWJ zA8v%fI#0I&EH&MCSTwrC1@am3!zuOslhn{$8Q28#{u~R+5ah8@g)%?Mts5uT7bsTD-vjx;mUE6~~%i=XwrU6W1@BA61C3;~%fl zo`gVw>FqWYT_ZTml%83liTMS~)eCXj59Ey0trz6s&iC-_@3Th;@gC}u8lNs@go zkkh%o+fqzd_JR1sVcbF){hrg1@IS)^1mMpD?~!waz~L+U_Z$33lg{!ikd4;|f;<#Y zq~S-%(xW9j!&hTgzttZox@IS9Hjm5vtwa79Y$WNQwS$IQk$%S|fb02q_@p6=h)UvJ z9Lwr5i5-jBVVgKJ@YKuw(6(>2o2Vvm*l5sM(Htd~8Few7ds@%vDrIgT#GK7*mhvw0 zNjh?_6!(SVY(}XUww0;wC=5pm?g@sA#g*`rbR52dr2X++tem4N`;_IcCQ}l!vIFYb zRNyqG$ajz0Zre7b)qMmv=C5J={n&^uAshGBvAQ`iJRlorS&vLck?~q3H!(c-h5VP# z$fNdK9!2BUVl8@Xe6UdWR}qrL-{HSC@K))b)r1t3Wf(=U%sS1l@l|5JqX5{TpKShM zl>g5ad*ueFs901YlAX+%kgHYp?LWfrulElPtCFcy0I(-}x{`POA_nFJ!21&K$ zq@*MxoA8!dKQENr-$irK2z$4d726*#Drz)BlBJV?0#Rzs@V~y&`cf&XDkgAJHtuC> z<+=D)j|eVO`uZT{=28r#o>ky&>FONnhK#(%Q{ek)KT+C2nORsg(RMCC{mSZ){ARN1 zdN|)%XKZyNZi?Xv53w&2(D+X}VA(}A)?UueQo`QM?h8Q98q3qAH{6AanW3(PJ4D*9osK!Jbnct zzV^|K5p4tvu8aJaU`|evuF*ac(d>~%`BrsS?;`Do)SHmtfoch2F7kmWVPwuDgX2wS z{H@+Js#qCW^~j zMfdLv&R=`-pU1Cya0iSem+pknsK15f_u2o`%UqxhabN`8f6Ds*^hKk`dR`jv{m_4O z|DVd>pFjTRCF&c#r~mAz|3wdwMT!FE#0Z^Op3)bn%kl=Y_Yy0_r`bfcXHP$hz|}+* zt@ppk{J&2FJ_#6(Aj06D1o0b;Mr26xGnJ7*NR4LRpC@aUy@CE8C;w|MUC9EvvF({? zfsl=FmA{^OETZUL3~(`cznxAIrTDE_{!%Xs=x;%`0Ylf0;roLC^^gZ(F9OkD8E30362=~ODIXg#FX+_ji)Sj|Y;#zel}k#!X}KjjxI=G1sL z|4s4CiR+J+w^e^!5)0uk(Gz@HQ)4Xh07ya!#? z2}*JbNqec!aY~8t@b;+!@8=OqGMr0b4){{nK&z=<^SLe{#O8LD!uQO|#K0iCV;~7g zV*p6KKS6r8$2`gh3+{S;8l&O$X^kQ_4H^s=uPZBS%(9RuH|>E{4X(?!@y!IR53Xv< zHo5|0AVzx6$~rE-`Ed2g5c3~7G2-^k%_Y-#m_g7u?ZQa9n@h02(PgOc9ug7~=b? zN`vIhh}8xxAtMriD;MiQ!o5=0FL2=BzIx8CT3$3&J;*G8$ukW1q~u~uXj;ac6i)Sk zWnpCRp~YasYum899x6tZf10h{7JOdvL|O#y=VLP9SDQ1P>n|iaG1Yd^(V0|O7Lx)I zua8bf{m%zE_{n#9;UxkD3>DlP8{ms4@PvN7?w^g`{xi#2->+&c9q~yp=zDTD?D z>OKPkZa0}AR1bw>c#Y;rE`(@Mmw7Cao(Tq3Vhr#*A3q_cFrm6>7cT%GmQ35Wn0W|c}6W|)9l2&3T zG^1<&Q3Tzv{QJI$~m^@Tm%z5OT^5q($Q--$XNz!Yrt zK(Zt?wZ}izB&X->A~~5o*7Ee4brimIQFxT=Ju5!arF+JllgYyhmZ$D*8&Pt+Rl$eK z42`DXb6VcVwRtQk_^aKU&T|A~Z00p$zFxJ(h%Sh(Tr5~xeUSni02P1fldq`ojyn~; z7fY}zD$y@kQ53^?Ul`g`l7>@w^~#&;TLA77L{hU-{|U?qqL##QgO>n6e;v7zEPs6f zs4v>;I~;dYci}WRU+|So8M-|(9Q>yE)bCogWQm^7b{aeqiA-~Li!B$Lb?&w6#2mwL za39y-hyx(aR;Ro_8MT5x&lEz;%5MUIZ=rlA@j5z#MsAR_poZMvD8%;{*(*FeTGNq6 zrx&n!I+py=_{quI3X+%!)C0T5d{Q0EdWWi*4xLgMzGr|txX+!Jwe731cc5~JZxyk4HJKj=e; zA}^QMUSoG(pY+t1;^tfyv|`l?b4MCjG6QHhaq85B&uQVhtJgeff5-$HG(7PTrgVC2Ye`7{M2|h6LBMheqp^G-5F2Cdb&E~r{$IVz8iS5HVY-l_=^`)eihWT zaFo7HXZ5tq#`EUpY9`$ddFu&s03N_j`bz=t4-OFlJQ!#w^#Tn?FryFiHmC>i7t#q&mDk-ms(&zeLN_jcoU8#G23(yAWtLhxJ9?(|q>l z(eQ+Ce_9tjBA`VKv3%h-886iJ4$|JZpoLV3pkf>G%eHjHk9^FiKb2uvn7KmGn^X*- zA9Cy!rpl&qz2QU*(AOgnL=zsYK+0Wo7TzI-Ar{F7A^=YYhbDh!Uz|MIMwTd0;zlL_ zNE=Z~up2!?JUWyO2dq-id!m2s2N@(Ft6m_V9`6>m)12E{ba?h5-x6G;(GrASZh`Zd z(5rI#B5i=b?e>12%lq;ddXd5r9FWoD<>$t~`QB@;u|Kq#)R!LP!u=C zZ~~UD+olRPI5(-N;6;mIhSW*M+XvzaSG@E1@e;!7NTimg9g zv=?ymA0>d~*sly9r?K4<^j*)?2{^kQq$ap#w25&d5J?t=GKL#`GtLD1E8Hz>@>lPP zCiG?-gm*B(&voFzsp(qJlAmn^3xN}UtkSC z4>K+UGJtOBP@aWS_pR5QdVg?pCXd!=>s)7IX1~m$#!IDGCElX^kovH#{7^nmqu`XI zZrE@sozv@sTpABK+wNqfIa~4P@Nhf!{nR~|qV`=;Qb1Me=Xq9MyEY^*Tf;vxRh(L? z$BW(^HaBE^sp)C5zTkC?sy&qRH;pbF_jE`AmMhPyk3ER=#zRK4 zr5OU{pUC&L*wX(?YaIo#GXZ-0QvS!$Y3`2tFQXm&ak+CU3GRMK?SNu`EkHhcvV#B? zf1v7L5+luI&K-%5ASQTcn4JEAlRJ{g5Luzf{%OQX%b&Jz&CKdL4z+;DB9WSffmbrz zhW+Pfe!-WS@5ZG1lZwLmc2GQ@!4{tnva@LU!0+stqw*r$!I2A^KM3x;ut2o|(UQ$}Q+VmR?znX?O@jlOEdYRaO>a;Zo+s2Z&fN&N7Txy7npfOvU1E14u z18}8X_XhzzML*H1!a}WH=`%wEb7W5*Vdb5s?wRZ~nE!0tk)@=t!u>5pXq#se){|=0 z#6*uuz%U}yNTds2T8pO2f2ayw2usC!R7Ps-DdQCNhoy(2ZuKZMu?e_!4AH}QqYY_k z47*g{-y8JI)mm9DWYw`+jEk`AyD_JEjn`s*_l}{U$Gx%47_MCeA^;-Tz8+C3I@9s+ za1@!Sv7M1lv%9}Wq<3`z)a>7+2tl9SE^`5n^bRj@bX|hSALm2#_0Pr4PHqg(w{62%Deiqg*6aKX|A^~XbrTSooD7r1>71e{dCXoT^-!??J8 zw-NB)!hEKFs}=d%S*QFvg5z$A*^h|l4*(QMg_JI=(tBHRH(1RvOz_f+ZZQz72Nvi? z1rxagr6D~G3=Ns0l__b^OTSD>sFtaRv1v`DLOZ^4bw&9i;NAoC{~-q4YQ;g!L-vFc z&bU$!+5CB!&yL`3R!SyW|B^Yezyq^mHwo_5x|$VMg5gY5!nhNO7cA)u zd+#zUs`34HuA|z3hHzY3Lv&tU;o)%2=x&%&com@pD7{;QaC8*ELsGuBgEBXy3F}O6uRV}LdiYuY1)m213!jk8W|_5K^2 zvEkYLpX9Ln+>XLc(cWz=J+QO&3Vn0Dbi2paO2Oo_%!H5oPuO$}GY#@Shl6cuV-ySe z$9vMJ8oj>w`aT4{^A%Y53xV)v87<)KBTLr@=l~XVAR^(F)8XP8i?b z44(9hHNOm|Z$CeXrvmpIycVvF0W{ecOEq@L&P+7F>OJ+~w-i|R2=K}&q+F*VK9{WE zEkStiHQhFqBm2I{sP~2*I3LlKD&t{rtIW zjsF8-Y+1^KAZY)9%Omb_APNYJsM5U%lK^6W~5@7OUBUs!nDb|7~xE zO2Dn;Ov|@O_AQij2^U~mz+V}jMVg@~BTk4;+>P7iI(U;C(0*eGb`q)(od3v=JyAVY zEtg4L-S~3irljTee`;E8WQfCvr{dcb7( zpq}mqJCZ5jKT^&sDc@%}+49<|j;zT4!c9Pl(ymK|duXW&<<3ZnxdDW!uGPsXN6Ax1 z{YvP{?6VhS*v#E&A4}(z``F!6W)Y{>Qh%U6g_I%u_ntIu0)~B1JzGA;LO@{Ncf%ty zog6?AtsZCH^>wEmkgs*{ypmIx*2Kz#TrSP$_cj^XfA%G8 z3S%KEti9-NmL*~^wxK`>syz)JNA2Xkei7;AZv)uVCB)$`j=i-8(q5W0JF!J74f!aj zkjmH|+-FKFX`5&CR%zvO8S=QuvmpYNzdZ%vPP#w&5aE`SOxZgNH*z^IS$A{2iTWZu zbsu3Vp_fd;4@ezzVvth7>g~G!XfE%xr5y*s%fhX4ox6lu^OEJ}pk z_avrOj2&@09i9h$ED9NPyur1tX!c_L`{_*op@NNG2v0Y;D-x9WrCOg)@ypy5glhpc z0cm2-Fj?`AF=k3z#^_d5s{luDc1P?%WZDd`D9>+_RA7OVZ#7{i+iKvBdJeEYy_XU>=%;+Qx5*n&xXODY-g*?9BI5lxg-- z3w6nS!U#_g0{p__EX?bKdMH?OAxt{6p;sA$h3JOH6Dfz%OV<#imE}A8^C1nKvvpE? zE zF5b6%@4BrC_KCY0s%N=*C|7SHx)zzO+xKw#revGy0TlY}&=`zo{L+!qBl^Hs?7#gz zP){Nxnw*DkEbTq$SDD&M9C}WW@h6)=C&=ZG;9RU}W?F?zfgl$GCZ$XPucn*xcK~*o ze};}*;kg5sx+m{1Ux{y^qZTr}w-!c_wMN zgebYtPgcu~Q?|{Rbct1@{dZ&~u%*!ksGSKw^BXmg)VafJ`radyn1- z+kxNfJab6BAO<&;qtxnTv72vY_mDx80zPPD=h}rZ1a~WSZ~j_1&+*+u2F{j4jUj){ zU$E?YuTZJ-cnF{t?82wW+Dk>tXq+V~hH@uI#3Z&}m0=I4I~(`ar`aEOORgQ}P>P3a z$atpOp6_XBbU695T1-vaid{)0zcW7`RD3&-NYcQHLmp`G+2m7$($~3e=Uys7=cZAj7Sjz=971jLG5R*H7`Gzp%r!Yq>2q{%?orj z{1Abd6~5j!-cC9-w8W1OXjAV~G&;l-IXhdJW8MZ);S7RV+(jH6Wr2KaS^BBg47z9zfs;D~7gzhR ztc8%hB7sY)`^z&3E}Zw$VP|3YGG2}C54UuL9}#6M;f`Z=cb>|MiP?8NpAKpQ0zvRX z#PYM9aX9VU;QA`A*Ex>}`DI)76p0oYww)@$v5|xCNTx)U zN;FhXxV!_pf4q7Bg5n7W9NG`{g6!l!T=961saGA{7zmVL z^e?mhOe~^eXE{h_OnX-0udC{Ge$^3@_1q5Nq5R}&c5?x#$JS++(q zB{N({&aZH@8%@tEFM0mTb^nu?_&bsWWIp|YSbdaThQ{~O*R$btx_%6PVr~O+1?W&% zNwFYlYwi){Y{o!hPNbrzggE~;rol)(+g)aZLuT-uG&=i~unV!^PACX94sCtilBdr| zkvqnMYV@7LuFOQzc-$}nkrsrH_eok1AU_Eju{!A`LxTe@f3M+xBo-du{A31zE|JrH z(+zqI7i^fbaI;HxINg!aA@MIXGoDkpb-rtOK~$x_p|kLuT7vykHAde zVf;@Av??gN6I?UVas8#cUbiRm5^|4V`<}L2E{NP?jz*;3fAC5ww#$yzt_{=d{kqj@ zSUCIycuhCJF(8_s9WBK1W!g7g3|fCt1w)C9z^9QMC&3j2>9jT*rkwvo!v8(_^5=6P zr~R$(F&S)&AW0NZyQVv%aAY*!RAJFL7oQGQ;G48CjQqIeZoR*%xe|BTHCTSyaeqfjnBCDS>x z;~l-;>urmEBIr$y9KO!jz2Y`T;fqNe+^2V_Fl7VKgd_o^eTb;t$)((5%Gd5UkpCM$ z{PXhF5xhs6C()5Dk`Fx3tHhAX3}T3Cp`j@E5pcT}oVAa2aC(4l@Z$b^xOD$XrsyA- zMLAl{f-^!91t@rLzH~*&>Ex5rwbH{sz^~TgGYS#`>iY#EK^7a#H;GF-o^+{W41U{0 zB8@NIk^wGQ*ljo?4Z{qFLy(^{aAUn3r0qCg7ZvG_<3ub&KxQLc6uF!*By&{~BT67M zD!fK)T9C9^gFRvT)iGaxrsV4T4C-jyjMV$Txuw6s_P>cgN!p{--@}VI-F1kZ<-iGK z=8)Z1p+!lk;F=?-1lI%J%6dVc> z9C7Dgl=)2x8YGw3ty;9j^`TdMSU_Atc__bljjT^DqBj zl$eHQt+sF>ygcsq>Uod~ue!QB1}A#Px$bCIV_WrAq?gW6TYPHvAt-MyzN>`XMp^e{ zEPjCn`I=>5DSNOb_}46pZ{g(Ru^+KvD|s2i5A+!yQ^c7 zl;9@&l;q@T-m3k6&5+@Qm1&5K#B})!=PwhmDKHrz2igxbVmzpDfAb4|@8(Y;a9t)& zcPunCQhK<5YP+u$ugkSEca;{HVj$1`ikKPYe^nvBN9Ei9n9es^@dl9Ke_#Ipc!wAX{`lED zG?Tw+82^&_pK?EJl)oCkOdy0HN^~Y>|I}rSUqOIZJC z$X$m|I=M*cedfl|s&_o^5w#n0ByQZf3APsaFw^wx*BHu^Qt-XAGNGMD*T%7CgVYh6 zaa(GqJ`g4qkTS0xy54153($Re5Vzf0b)SJBL~UB0MpYC|qBey`^(-4@p&8*Q#MEdx z$a(2|<1(*LAK9yrM?yq*%LwSd8h@tbA~&t$pM+W%)6+aJY0#WS&aH}1MP1Ep&Nz^Nm0nc zB+-8qV1DUe7?Juf!~1XQ6B#}R*o&RH1FiqDD*v`w|Mfobqd*Te*98sm|9AQIADfpX zvL?OxKNR<$9|?#&ev`z8n^yKec_05>{9efb)(Jj_;^a3p{BP##UpwNH#G@Tsw=&K9 z|B?j!B!T>&{)Cb4zr^nMlUINErg1wB&a-(B%+;7a<>cOvWicJ;&nS6_y}Z2iIjLL= z0o>a@w<;On1P#hpq36C{O* z&QP}gWa|hBfXCt3L7_VP1#YmZ(7e7`lpowiC%gL9yChp?ue2(-Ck9Ot-d5S(0Dr*LB%f(~0 zfoE=R_35S?geDe(iLUh+mG-|gqqG;K(F<~S3PYD3LAMtxotAeV@X<|i?r$aTd4hq! z6}PX)a?V@f|2h>i{lKqV8D8&*LeZJ`VcNil5>dMfi?4Q@G6yebH%YhNbjtJJ(=9YuC5U$1S^^a~_}JN1my>Se zYi*VkRd26TYOM$j&Ik{{j7d4=ln-QbCl?lk0$4Nu*Lm91V= ziqKl#f1B5rf6c4G;s(4yXwGW}h*~G5&34oxX5_ zIN1NQiKq z=0vkj2#40eN()8M3?O*QNZQ@ga}|&YZm^!C>3FE0Fx9Nj;A3Kr1{j#X^$X0BwXbPa#(rh&zH^M@g>FYmOhcR^$!RW7&`kxl~y^j)7n$ ze&*?q4V+|`Fl)P?E7f6{mDog7rt|vgkRMx14TU-12-p{W@?1XaPS*h@6_f{J8 zthYULSPP*Nz5HCdkGE(}mW&JV-M_cO<*tbal`W=Jw2@mSH^h9=a*j+$_tq$HJ~jt0 ztQyP(p)2OoYgNnOPJ`)4#d%2h>24K^RSP%AOk`NZNPTTdf5_i_tv|WdfjqR>^YuQo z*_v_Ju@lMOsGF;lJeHr3On2XO1%K#BT6A0*w%31nk}!DRX3aZ4y^tq&=4L;dd1Ew` zW0>bu*L=L{FfVGmY~5B>f!^*sstcRj#}!{FvMBtMKAG~U?e}g*=MfXW-t}Aao+c~R zgTNQ)1#63yT7SfC8)cDz(lq@7PqXASk)DmI38Dzg5<^nQ_hfx8hL$ga^ZpFE0%_gf zl{r94`>uH0*LGJ|eWDYMcBJ6N+A4frH|3FZURpNuY0iv(aD`YHZsN|yr>gufaxD2y zn5YTNa-`a)w{Xu+cXcik`7Pm!!j|R=pNgbWG+gY`LOU)+jyI%lxGD(p4aLIZXj}B& zAC^+mezk|bsf(Q!9LV!svw2~1zhMo$D=O44;qW9--PcQFXh)UUoBl6S>DlQ zcpO!c(mP8K<0D@}`6~B$yiJ!L%2OkbyhjhL3Z~aPf1-Vl7X4y8(mFSh`O!1Sz8PUY zAc1nw6w3eq*n97wrq(TPSSbP`pdz4jL_kG~ARxUdAV?9A-lTVs5~PcONbexhyGU=L z6QxS;1QLq$8W2KFNb+rt_nzyy=Y03i_m6kp8OPa^+3fx7^*n2>-|ERkjI@qH3=%#! z?Xt)cVxuJQ1E&#B9u>a4h$dgD`fbg7+$RXP4pUEj%m^|(;UNa#|gZO|+PYNd1%>e9xG zk_Cyip?r4@@+!;_)X5!>hYwBmmwQ-0?X)S<(2l@H-Yj;IG$4;IR=M@?J$tTjHQld7 zb}fnzG-IZoZAy4-Z|}a%lPY!D!9$Rh=`3@>{#GD**g=ey z{LMmEM6RhPay5}ie)GZ4HRbe4NSR(nHl{?Y<;xWYiH@6WW+DcInVsV2^2V7B5Jv`{ zKr$^Jn;IMD3AFNahU-}3i}HPXD(dX~Q6*t&IJMupwa)ma4~s?x{O0DBS;FAzX@qWH z+5U2YH}&8UUEi+;L01XzL?#Asj3_3N0LFxOgrtqjQ>F4ko)`p_-f70R@YTlXsy2Ue zURMC9KKX&}sB`_tN~-&sF=q17J<4@+ckCu{ETTsa$)}`{SnG(S;i5vHQsk@EYhOyK z1OpOA2JMr$jqcMyoe~yU4={=S1M$^QhepY}s7v*}OX}l##3&1x19*Y%1HLA3Pi$8s z&QO?X373%$S1AovVG=Z-!f+2#8`mNy{lGTR_YQ?T&Jh4pIiJ_r5nbMikxC-Y8yI1} zzAMiyf_d>cwV`)YQPb4@v37_2wCZig%IFj4&?CnHldNg`hbnC^X7`_rGri1$j@n|iWB$Wv&xTc)^Jv@4mwAH z2U>yex&a%spDab|*Ho7)jNA^;9MJjcxwTqab!{vfF;}Y}ML+0J%}5j!%&~GCPYXr_ zPXE@j2XYjxE%3*+7-jhGO~1Wg8YO0!Ye8Ayc7}k?1ETy%EJv)=@-+apofkjQd*Mw_ zZ=_My@-uKRJW|$Cajskv*gh(mE)@ecTm4D{kC7sv)By@fJYBXmx*F+#CJZTYcQwFQ zMEu>@HHQkw6U@4!LJN@Hj9$wuHnSy>mD~|^?uM2?U!2yH2f4CczKWdwWmasFF9o&s zk2p#t?az8Rh99L0IVOFowbc>9bnH>?uWa+IjHyZp(Q_YeLa6|TxSJ*j3sqnZn34Hv z3kYHUwH=?&@Ihbi3uA+1^ChQ7)7e zZ9H~|OTy{VGb{nl99I>7ePbg7OeOvHb9sSNGnA0)IBV`ovIV4fnCLxFrZ2H^g8q_= zZ|t+f=3n)S77%H~9Or4vk{3!^1K+BUfNqUn`jWu-Wz3@VomL)`H*9_D9f}m}=attg zIKtvBrVVs zKKZ0te()1g;Y&1XlfuW3r11uc8i*w}5L|}R*k0bS^AtHfaI@wBs?J&sDGaaNLqoDx zHrl!hH)=U$dygWL6Eb=W%X5r~YTxzB!kH~ZX+(%l`3!~RycPMpFFm3tO`U`IXP3rQ z2cvXdxx;se^cYore8sD=GyH2+8<>_Gws;$^wbXh>fe}TLhd;*ossCa#+W@X3b(@Pt!W=xQB9RSyzr%u|>)jXvYvPoDF zBjW+k;|af>oT}jbOk~s%mjA{40Z{r**vme(f(%ivDBnoS6nQ?0$v1w+FP>XUZ*Kpp*Zgo6Y zho|p3-#vO+;2V~%&y3jSalB*-=Q320mRma|9K5K4_Szv*U)jKR0`h9kRutC@-qej+j3b7>DJzIC?l!B+ez7s{uUoE)x%Gl)-S9y zZ@W?E9TB;S*BDYXPw5kvU$SZ8<1Zb$Zq4^-nCtcDOo6&Zg0pQ-W6q%x$>ZmJ1?BI_ zLoyWSp=j?bn-@3M=HfB$zX~$mewUJLX-wbSGt?jwHXi_!-PD=EqYADO*VMm!kJH!B z=9j@W{p1JGMB#mPK6e4732s6d#;zQLm{{mJfw>!Z| zYhv7Be}<+mzfVZnN3}o)`B0xa5i0mU(!_R}yyU?J(jxbz)430s-8vi~*I| za=5as(=SJF2#jaPP*O*R-w4VGGz!jm=k`d(T_K&^FpG6qDP}q~i=l%w`!r@!DSvI< z*N}`XpiAl-FIT3>+wq6Ibn0L)abG{~poG#SWqAr8(v>Hvb|F?|FWq&tdMQh+CSTm# ze*W$tW4&e6N_Tg_Vgp@gT0BVOKtn5gIF_Huqe|W1toH*?_*57=6yztjDN`ql^0fUU z*KD%k@*~n-ihq^!R0T-M$h@0*Z6|q8J@3PFAJ=!*g^@3}TxFS3#9hH@!nv(TWP$7C z5N4lq!?oQ#>T9OoiS7ZHu~UIY{^vk;V+l;}O+BDZ3a3DYXlh@h!TlAUyA*}ttxijD zptF~EONe|FeJoIvY56_tNze7RqAos@S8tU`2yy|9ek(22?ij3LC%l-wz_+&oji@!? zE3Pd+a-AMsVN|t3Z(JkAf5x0cvixMt_zAw{ii5HVrQ$K=_=QoGlf;-P#)l4^Ae4-; zbPEgOqni-oqamS1G_~6e zKEz{?(c4C8AhU71R>6Jh^HKr+BSJp^6sB*jkOCC~Dl1d7Mg=4ziLkN8?nL{r zUCdmjks2t~Fhig^BOs|;O!DSXNlLPWJSICi`&-QEgICasjS~oGo=V78HrWO&ptOmQ zzr5|cSfS)v?(2cxVZ^X}j4RxOzB^MIwpygY&zSQorb3V+rLG=#<-B!(XfktrVe49L z;LK?irLm_n?2!{kchn~OqRr$+T$Fl~3d1e0UEYzRyzv@-7ofwT0vzO!XTj-P$1!lU zJI^!Av&&N5`jM?MtI?+MQok9Fj`h5eTZZ?m=3Det4i!^6MQ>-P*HQRGEe9pd`=iDC zFrd+lqGDLC7y1~HoQj^gJ>QS+BDRB-gr+@y-Bv%e`k6`@H+#0b4D{#|6FK@MXm?)} zw4H!}5Beh>4!IkA5WQo9|I0=#n0>D5uU#@@*;JRV2aFkSd{!*TmETPzZU}qqKqHxo zZ7>*XNj3}0l%Qd1mYi7G-OiQ`Si70a=Sa)jPT;H8sNx!*pPx`_un2qnln$hltP3IAEguZVciMwV+X06Et2e99|NtIL66dRMNr7|5& z!*>-jDV+fJd^a%1Wa;qYA*N*TSW7x+Ywaog*{^u}+e`up>`%%fHIz)qYClRn!xtso z{d(X_ErYOf%2~k3#n#stxgwxcKY>SAy7vZWvNUql5Ak>{dcvPJ_i7EY`w`-Uc8Id6 zY22fdSsm0tq6HHPpW(+88Y=Ht_3Q8D$vj zE7}nEvD%08SpwCPpB*V1urRtjo50omiDcw*pOa-@;%s@pbC2yraqou^rWo#Yp5(2v z2BS|xwlDT;M$B4GHh+BaHn$4!-5%%EGL-04z8f!n!9>1kmuCha9Br~<+VEOg+3(yQ5^ts7`|+K~Vr7>z zvc^lL5R^=?H+D@&OKazqiS`d&KxwrSuj?bvy)^R#2d?`}2OMs}$$5xttv6`Nl@H?N z0w+APJe7k{vaKbTZbN0qvinDEC6rmx;{Z8c5Kr9nUOWI-8+byr?!tQKCvJRNoj5G; zVBifRYtqGbmR1HTE5Z;D8aE;;4aR#|-9PZUYO*g`{KNO;tZT!?t=DgtvDV!5A{f2K zpV8k8!hd2rE0&T{KzLi*Tk!`qn6ex8ZMK(dMr?eq2Z;GwzKt#w!HbfP&kise4c^p6 zs?#eCwXVL}KRiwm81rtjI*lomXc95qy!1WUFz`DMi~AhcK@9Il&)`khvdlixESDJM zIyJ?F#np3U^W;*%EB+>xjszo=OlbR#6E5sVpE2w;{U(DUv)9nhY|d888@_Zodkl)~ z+@`;$sQ@YStP3QfpdAa(&&yL3U)Ba{dv-(Hnc4XTwZ6nK*suPz{JmH4wOE0;=VIRL z^1fsqo=qU4+1S-tK$aR2)$u-(P7&}einF#bvAeU{nc*+iToAcgPm%r4Yp_IJurl$*^9WQS|)N2k~kB+Cp^?mxxud%ud^HD8lo z-te+`d@taw zz3Iwjid5^|V%Q1l%lfxSc|BCz(WPTl+#>{ZTPYt7|J;Ieba{dq@AH&F9EvXA(=J>2H8ok<4{$&$7#& zzWOT$&U}S<{WF!*AT^gC;QY;ZPu239$g`iW%IW1wx<$u%c!fw1sICzxh(Nw9$0NL~ z05r+)W$U?GxYhC5`c6zLvH2w6^*Pw?s6$-|l7$!L%GiyUTs$o^%hF@tPVcDPR-^cuBp z?Y-|NSC@C}A0n081~rD^F_U(5`*{EE1t2XyXd!0 zt}BA{vq<8uKH=XWIor_X8>8Qgv*AVdk{@Aty(!(t+{Ut5)E+(gPl$ldcP_ zvna@q`ChQWoO_D@@gFsyv`nwvj9*j4D^)8F7$%GHWA_7s>aW>qu8rRN?1o}6q`S?b zENbd*9$BYi>|`;$z;!#Med4s5K%GPDwL-Mb(VeS1L!TryCl(H`0Zms8FKw*N#8W-_ zfj?|yU0O&$OloPAZH}Ya%t62|C^U#&6er(!WV;!Z@^2LQXSC_hi?cgGdPZi}#|O~s z^)v$B4C>usJ6n2}tXM=pmU-`s=8=7$qz0&PvMIg&ScWxrTe!8UDZE z=%4TZ^MVh^6H#J+do#V5BtjaI5wt2JB)u1tZjXOCcPtc9t&#m8sq=xiXZYGf&%x{8 z$p9*JsIbP0P`WMmW)2fr-^|ezvd2%=4-RHX2?A9xdj*8mq?eK*Zq(v$aHC)O)KeaW&3$-S14Kaz-F?2w@_$ zHkmQ#i&XFX|G#&~oX_OoyV$XApp=DgON~De5a*z$H1_OugL2=RY9%e}$Spva*tC+Y z>q)b*_;S0#&fiPH?_RNkJwO?&qRowY!MY18O* zxfHCj6m+qsZwiUc6uHF5-z09Q)f=xnix&wY<=Fgi-H`4X7nQoaHuL!oV1=O@&_sq9#(JJ+W^ z8EwF(xDPH$K}0qi$xE{%{FQyu*4Y#bxyOw4u_ZP+kR5kewNUo8;cB(J2o?i_^o!9_ zDelep>yDW*f{$o(EL6VwuzRy^6idVzht(HJEDqfLgD7T`EK~ZJE|hT$ADNa>HY!b< zfM~2|jV#nJYD-q+GDr1LbDh}DKisdcWG|!>M9!kFSPafP1uLKG4*z7Xu*BVd@ifdA zyqaH8VymgC8DH_>&E3lkY`1g<<{z=$U|SGy6%OE;+`Mr1y<&Q6j<_#O?&-7KiU*HL zmGPv{69i%c_?jwMDz39lyr2opKF+R=X)1DE!2isjXD2~m_oeXV_hL#-^4=2UF+<+1 z3y~p;Qn}1d44D&Gp?IGzT7(EZDX5d>%BIW4kKy2q_!>IQ7URdNcV&c+gO}kMkO%Ya z_?p{Im%3JBHKMdKh8*(J{(i@yPl@~HT6I#3$v^I3#5O)e?&9}+-AFT8O$0ncET6lP z!#WD--fPBm?E&#*?S542hK0O4e>9&cV|%Oe?(uOiRX7)$=l-($uaj{#*;)O1+!C9A zYnwW=UUGbYOM#O$Q#8~=@k!Vk59f*fG%OBK`OCtXhBfn|TX6hBogEQMH7&>EdIftD zc(lDs8Qw8MNs7paK3dZHM#UQ}pS9`?qad1EJS&btG*t^~da9(9@mcKRN~{J)eqO=% z(32~()VErdRx~}|06hgL0nXfZ6VgdayHi0*<9*qSe)Jl5qaoRbrg3`K;PBzFoWlIn z$TfD6ede15<@enJsHNIj(kDyqxDh_05+O?|P|5CNLB#mp+`Qn6w~?UWmC>4C=9Fbe zv~L)LtxY9mlllhs`|ix2mtK>aHIzL)3{rqljla{A>HQ*-w?a=ga>nX>y{880O`k_D z%`khwkvI7D(y*_Zs6u8tQx;9u?=XBI~P5W~UM%N|3KlOZe!cQcyOTseKzNc+(Ueb-c9xBdtJ7Q zDDKOAxV7I^g~+&<>v)h}VVh)H-+Z_1%l~@ZXIZ!2`xu)CF3K1*Z@v!p?LJ9zJbO7k zTamN*!DoJj%bhoB-7AOKGm@flKCs$|qJ96P9ekB9o7+n#)d%;fD`=ibUG${d$+F#$du!Oxmu$ag3H&yrd5B94x@Vi0i5`0I8q5s|d+++P`Vr|9*Qf-5N~_@qv;iO9s@9*3-7dU~x&X5O_$88`s$ApX z+Y}R3;AIJxMM|#Zdy^eEHUY3kc`MXm@8W{*4gbP7WsBCG9X}ArLRO!ztykl@j50+} zn30$;iPP%x#a$}O_HI~ZU=D>XWfkRP`5uf9x^^!!sAR~_Tfei8_Gn1qafUs=JyZ{j zcs5z+b9DNoSe2W;Z0hA}$xiaHqPB7o=awAfT|Z3sd*;4(eqWWBEefFYWnDPDAto6XMqBexckIKTut}yJ49*(eGPL%Od zA!*U2mt!l>-1~jWwZ-&a<9ANn&{j+PvL1hNp!VrAzRS}!4RUOjn}7%y$REhtc|O_! z8YimbCy~fLa_T(10ex}6@nG)70g|3>GICa-+GyL3OIhP~2%m#0xHTp3xMza(24*8_ zFN{$H%0r9W^5>gS8o{-l-sz;I!qA_XTiFv=l}w$}!C#Ha+d%d;YiHsOt9z)9-8)Zl zrC0xHf*#)u>*QA=aV*1egnn9UXGL)x97>mG*LGh@bE^iNefCkMX?%SvK~mMm$mYYX zI~>ob@vsNIu4E6x?eE3EN9gs{ir*9c_$?qkVfFe`<0$Qj_YEV7@AI9|;tR5!-8UrF z#GX6^wXuD#(sUbD>z_Yevkj2{W6acl&Bu~r#Bbd^+Qy`VqPx#UK9+FZ4`Nv=$zUx1oNG(vasn!8ocAaL6TbBYKaO%-0{q6Wj zT?pW#F=@;1S7jcMKR;gI6--LhPnUcAZsnKxM^51qKwlPxNj%>@;LNrb{P*Da-xtvP z5;%Y^;;uDx=f5)31@I@HkahZV8fDdgGOem80c8}PfK7>>`uk2DhgGtCuQKhkVEFw3 z|Ju*2i1RO4ikBbz^##-)ixNcdV&(UqJoR^3VR-5vwxZR^aYIt`peNCcs$6}$@bS}! z4+(U&OsDPtpx5~bvh1smSC2WSFdpJJ!@$GDz=E>q3D{eNrsb~OdP%OF9|^Q7k-Erk z02qHGf6P+LzJBteRQSnArsLkTL@VHwm1Um!D`)^-O*16btO(n-5uqgXdc|3{lb2LFQeFON~H5RL)5 zrfM*FGD>v+&K9h+c%m6w!|niszdI$}4$8$|PYG%`gDI8d9NGfx8z>9rgwjw2U{M*1pVEQAba}Cq6VIN{ZVJGv-Ojpiu2=8KY{W ze20n*DwMAw2-v3t7~^d47MI*_ayKC6@^HEEj5(Yf4)enm$}Wu_ z=y=_0TJJ?Vpi4nj9&$$9IR`lV1hp672yBr;LK;wJ~PIl;H?UA zM5Ok7jk4f;{lip>jsu-W`4JBN=8K)+o&ptBeW2JgGsb{VB~@hB-=s$ea}+iC%4 zxM#;chFq{|@Mx5IBcle1{dw>qiCN7L&!_$GOXj2bsqN3E0)Em|qZz#eUSOyD=aeWcS_6-?*T?kURoSNYDQ8B%}C6=$x z2wJ?)=^db=(&Nm zvG0aE+@&RZ6+a^MTjodF8kig?vn<;SIKV6X4w<)HAp(|8n}p1;tX|N z?jUP<=S6X@ed)2@i+lc$r?Gx}*0VzLQ@ejN}DPIZ{C zXK3Pkp&A=?lkYxAdmS2&k&~IXru`t5LIQ|#OOA-u-v_5k_0hZm9$=3lgGO$x=dT$$ zPWTHCNxbn;H^)U}T1^4X4@!>pT9!ti3V9uGFx=AIcWQhRNWsTNhYad#Zb0UQ^jfBb zDs!ik6b~#@6im(K9@RN~0aww?r<>HB9m{!d26u07>X+4o51pnOx^^B$&d$E6CS#4~ z+HJx3Ewus%bICZOxoanwgy*S;05_(*6;lprA;-g8c@Wcs3GiNv<4-%;?TuqiDGxrJ z^MvhLJ;f?4WOVP=$al>-)y*HPdM~8!O}d_DfYBiIRLyRo0I164L~yly{rMF@2G;RN zphuH*K{m*qqK==7)jWx_zWmcpcPfnm5iNCoIN2m6UH+8Te@Nhf%O}MhmRn1}h-69L zvKfqDQ{CaH)C)*0TJ8w#4Qm2z!A=lK^&POEyWeVT#=88@UN@FD_bSG=^ZQ|+GCj_) z=w|Y)YS~{b1_Sf4w?=Ej$)>$382mRY!HbHVcc>}9Af7w8*tqF9a6eu$6+QVdKfNH& za2?8pYaMbpp0W3in1O<(Eb>Yx%A;$6Q6pg5#{5g<#=Fv0I#F*9EWMWes?~OEDvwoHpL!ubo8SxiU z6CKNz^%h0G({O0T>MT@|VjYvumfF5oX0w88N0N=Lutb-!3^mqoFo{C_997;&n>Is{ z{@>2r4J*Ix*2t@Bk+|pD(BoO${YuGg@n+PT?{@%E^ZcU!#K0-739h|T zUI)ktT@n#~UQXQfBg~bKDA4CN+w9i2`}gnLZ&-Lu9oUS~_3e>RG10?)GYuX3IaOiB z@x~RRj3{mMEI;TgZ&-4s|HMZ@d(P>p<n+a>_im&Gmj9E_^i{Nn#h7z#0f=!bg}SSv=;meMZ^Mt{yBH|X)84Gw=2bu3bo^W zi&+H+7CYTz3k`D6XU{%(XV#DTsI>xOZ#AB;xU5yt{B}3wR2>6F=020H>+fPu?g|;P z+$Um!u?Gr8r4?)X?nB>tHd_C5@X!A!*>i)8M-r#Fe2;y8PBAb9Tcc$EhGk!mp54Gy zS%-%wzS?R)dsjt8^b>uT31jT+XEySTv9 zGlFw&vMqy1d?b`duDy?^*_6()Q5uGJedKU6Y5GjAZP*C*R!W%kr(&gxgm=wf5R_iT zTmRIRebVx(EhmV`id~zjXj58i=RWe_gaX&ac--^R*z(M@?@*5$!wT*7#1cU1By;>SHdoj3l{BP zR%nt_PNHu)ZDBu*3fu;oc-JC8^o%`QWe^VQy;)n?)GhYNTE zmw6XPA6z5(i-i!-cKIahIFnp_Qbm02hOf8TY{Y&2vNe}jX*S$vr*0L7)aAV%YX4vb zw~Eit-UTrfmsWmpR~OchJng?!Cd#~EjaltVb8+boj5(rf*TS$40`{bMU5Trd3{1pXimx> z()C~e4f7HDVceq_+`m5r!!w$uxV;~p6**ioL{uKA$5PCpHLh-z2vs~I$Mwx-^d0xF z+uR!3j^}4QYu_qryI)*t&lpbSQcKe4`|Cv1X+dhYXY1ya^bf2u{a#_YlBbZXOm8Jn z!%MeM&*<=rOTdatc1}JqBgU~yOx|-g#|)9Xb$Kf*CLai0TK@RhXTZn0=D*YmJKOt8 zlj%FX%z9iYe!OdTTKp&%2oN6~9?P}J*&nxfL^gXpJUYzK*LJk=nDNrS)3A2fG1+%D;pMxIy!RroBe#dFl_kNV`qQ<^;>)NCN8_PQw}At+r3(Gs4F0 zpvf@obqJJkYMvU(Ey&~?;gk6kKe+N6#No&M>%DKo+6SAJZu24^2D(M?US0dXhW*-7 z{l=>D#<7Yd#IC#=v^x(N0+tLXU!mt{~|`WnbZ6B&eb`X=AhFyjsMuYw^9wnf7m&<`lrdaTE-v@u45B*Q`@@L@t}iE&a&vUZ!(KQ zRp49T+EdP(n4UKGNrs!p%ULwbstsc0PNpqqZJd$i)Jl5Np&pm^=?tk1@)yvY z$J$MAaHKf^(|#kij{}l4us)w0s!qGk5%8k|3>LnN_&Q5b)y?SB24LIb*ARdF)x}N7 z^j3NE$Vqy1S0I}XNiTb9LC8cGP8}UVs#2?8)}*{=99iy*Y(0JZL)x3cl8EWFDX)B> zXxy$dgW)*sH19bs0JHem=beS8Z|)B;9X4^lyr#(walJuei4Vxt2_<~j2yj4&WNZ)C zuxIr~$l=@uB`c$UQ+IZ(w@HmEc6TY*%FvXuOTvmH^R&o`Xp$e$6sRNi{PxpzHQO7U1cy= z9fK(i7BllPuK#EThr2!R)u0g1S;**7aJW%txtzNZKc8u;zY!V&=-}$gnfi!}_6cq5 zsUk`?WAwXHE!^9496MEGDv$dd8&@+0orf==QN>R7J82EEBvlNcTPZak2I^+|qj2p# zg0^)Gag4{`zSGI$#&&=v-Or+(Di2pdk!7<~T6z-u%NnCk2Z3Bt@-EG#r+%=l3dYkWA`DV74$Z+O~*g}n2Dm_yP zdh@k>hIwG!)VPSp&dhSXYu&Q#w??^r#@y zF6%E4er<#4$)n(yfLyYln9vRt)%p+_UFee&+BOK@|H5fyaMlJp1;IP2SvM%A55(-C z1-@OLO0mveGfdla9_1g3_2$71Ctq4JBm^8e=$;WqHTW-2JCx*z9R0wNiFA2^2F^Z( z4(NY4Ts_A*8n~vofz!JkE!CjC+f6Hn{I>@lTgTgO4TCiVmBaE5cgvn*mifh}h57e) z&8kA;6OXy6y=GAs>6A0#SI<`kwkIfF54 z1IVSV(&DDQUSK_ZejI>5SK?xZvyJas8sqQAgw(CeZWEMzMeO>`qJ~ZGwUI(_-rS7C z?bBvEKq6bCAMG|s#>R?AeB(KDQ_xY5u>!mA~GV<#X;7n|GD(`ycOPz5}TIpyemSSN_89e_hpGW5Dgz zdptAZ@;`1O4wY((sR^|FtCq>Xp~F+T|4DTR40eEV>{!wTlo0x5pndNQCD@ct3-L?# zi1-Uze6H?{9Pr(2hnKXqVLURzsd=pJo!s(sa>{xz%w7=|%~DSdXHi-&uGJw=hZw8K z9YnhjX7~XH1e+yg58?bw7}0~{_^z(idUb7We;oaBkKrWv_KYkTaH9!qM#HoHRcYO-s-4ve3W(F>^pzW3bopNnCO#n!DO_&pSqj zSw7&hU3qwMX<6aZ#CrTtx3Z;Cl|z$STWO6ehbu5$!PhEpP}-)l`11 zDcqgF<@!&GYGbjTVTHLRa0^TXyp^UQyb`yOQyDU~*r7!BVKFDwK3uXUM0ZS9kvxwf2GhLv{0kQ%T&hn&-raJq>6etj=)5%2I;+B z7gGhD=9+URQk0%bPnqk>HB2opraa4N>6vnCY`py7ohXuL|Td%{}Yl4_+c zzg2#J@*Cz<5%F@>3i^GxtV6DJ>XXfRlOJ+NA@fWY0RDRZKx7zU54#nl%x#g-%dWOg zt)(LfRVLjMnU&63#zQF1;nDtelYgCwVOv71H&py6(|`8b1}^7mt(Lxi>bNQDU<33s zAY|&3?_^DPhS#{3!Rug!##@p@6_S{Rq1ZR*^g6bI)O!9g5%J8rwc&Y2%h^VcYLCKZ z2L=(hic{noe1VARmH&8NA5i0DO+`f&`jVh3-9|L9q9;E>9Y%mRBSYGA&}frATc}T* zp!t@=K(SYK;T#SL;Q~Rk-n&&!dNfgs!my;Oca4iEr03jG}Xt`a(!C+1I`>a z$ZYR&FE>Fj_^0DN3rnxrngr}91wq@LuR%&52QNAN&cOd>Wrh2^COY15 zQP7FK;rG^N;-(OpIi5$`5BVO|sHBT@Qq9)dX~TV^XZxd*MF?KSE8Tikr(3f7#pm&g ztoLCev>)eG#V{g(A5iEyh| zk=liw?`P!@j!p-HCtS4*JnFv;nfk;c@H9n*8S2QRxpJ@G&3%wD4( zhfsEyT&g`iG`d=#};1!2;>cpd2k9nbtvRKiG8rl{? z?uY%ULPG_9A`?maj=IRurVd`enKyK;vp+j#&wZVo=RvhZ;|p&8K)Jgf0UihU&)Eic ziLQ(ZnkY*yI&~y!VoX;t__iLC;&|er48&|KN1W>JtRi>2&%K^vhB~rW1$%mliz8Ko z#E|1Rz0Zo|9R?&?=v2cq+m(1AKuYFK%L%KDv3mbya`<%k5X3w*-jb)~XO)A5IZ^@j z+N?{UAi8Wkg29OuOh-Ly9tud6CqS5HLFTtT{Zl2})xYRpJ7KJfDo%gX8#)QV(0ZUn z^H`K1s?q<6gbR}gb30k4=K8d2LqZWJ;8~ouNG3xBDP0vT0+u*1&)wP|C5#;cYewhC zWz<2^Kw;vj;oPGQ4h`0B^x>`*XG#6Lmb1ma*Bh9kRF z(o=c&SxW?WfdP~mibaZ^B3iocRCK(~4oJ5VUKcdtQPFjrUL~#gJ^mV7eL6AoiWbZy z{PjGM7pTn@JiVH#3oSFueOhtuRk)3ZaQ^Mp|5;8JJ>lGBo$jFcmRJ+IRaB$(2xz}{ zsMngTjN9u@czR;qHzbL(&)|Mpe+)T>iF?e$mw&cU%#E2F$B4eGc&06PRZ#i}+J{0F zmm{FYYwIsf?FzWON?Yxw6M8i4lV?Fuc}Qqjy1Q3xx~reBAWxxLVU$;6k(GIOL8_xE zPwUS(BNi>(0JXov^oLj<2pbn4(KYLQKyTEHQmWSd9W(~GP@f#7Iw6G6m`U#DWgqyP zu}IRIQ(C1PWkdA*OyK|fj{mA=RG-A6Ew@}S&r z4%{vMqm#Az+Tvt;uSR8jahEwCWFBpNntri`l z?^d}6(g|{&YY^}RP?y)APW>JP{I@hDF8vCK6d=BW54~kwnCY%34c|~z+^#|1^r>mLEs=tHPt0WN^ z(sOVqZJpkaa%h}pkL*U5)D}!|m(>nQqG#A6Qon9>&|2t7|L-;Be+8<1x3lmmz&o=| z@Rv}7>Fwg&ck3NS=xqy%nPaIL`LULn>1s=OJB7HZH=^3=>cp}ka1N)FuF0jwV_o75 z-A^U$C;N4uDf2Y^REOInznIjXe;rE3Di>8jli~ku?&Hb(Qph*2$uh&U4d1Q^^m9V4@l>QbHC6V!eN)wCizt?gohN_ZPm_><90+kOpdzL6E zH81Q6-HUwfw=bs$MYnAMOjHs(J2YKXm}WU7b{GLLjr$UIJq$wY=-HX-bYeBL)sfTf z+t!IZ9}WdR6x4s}A;gWDa3!T4H)e_y)dgs)#R2HNV$l`1oB!VC|8PEW$X9xiGx?rWb`7>X z+*3yGx<5}9EPZV(=PEr-;S@I=;8Y3n_-xyD*IucFE?ZryIy zI)7bxJ^le@mn5Ahm2NV(VNMfWvfV~#%AlbL9P7<6Twi{JbJ{>C-s2M(>ERw3Wy6xUZ9j zPBHEXn81hIN?LML4z@HrRpH$Xw0puu`5*P4MFdK*0q3VQZ<5hw27o;U4J=Pspc zYI`gtdW-v-pQ(VaTKnSd@vz$aVhX$ej?gaPacO?dlp(RL>I^+ss;LyWW^UK$1x88+ zjQJz5PeqL-va46L^ly|(+?Tl!@RI}xu>4G?)C>9t(EV?KYQYYikd$=ZPvQULrUf8- zfX!zuh??c!fA;sq{d1W3zaA=e0%&QHdExA*e}BNAx2jp5yZxt0avT1~O;;rXKnr%) zxn%xi$lp$OlT`3@Oo!myVORbnn5Qz0?ahJTRDF#s44FjtPMXJ2k)6LP>|AJ!?$e*KI=n zeePU?%fOcMtc;GIfV6bt(b7J6z-}_&Eo5*zFs!{b`bX99i<ud~NmS%;pX|f4Eb{f|Ab-x-*5kq zVE;!B2=mPVx_%g-=fyvc-UR@e-aDz(q+t@&Z|w7@a5?|&5r>)?XzES1_a~*~os*l@ z68Ft!V3yV19|zP@c5UZD8~e^B6?oEV&^B<)sD8Lq)8yz2)VYoE->j+TbCK-HF>YcT zKHG0e1p(PUVK&N=f#x1*cI^)MDB+{i6HwD4hC|^4ASlW$v7r`RXl|r9GN z!&(I~20aBJMnxJ4Ea30Yui*`-v}~#o@P49t(I;E}GHTmQ3*OeK;GviKRho9j_$lQB z{E$>-$8~IRX=uQ_yPXL9KtTVZ(Y>V7n;ZWixcF|ok;CsmZJh;%PrE6DPsT13eA^(G zOnpfH^7&cJ_{*@Zvag2o!Kg%U zOF$9Tcjii}B^~#Ub6{t}E?F?$4vdYpB#poGN}H7CZl}Apf;#5CmiAdejC)Yawjz+6 zA^EhXJP5FeN`g)_Bme&KHFi#f zWS3LJhHBTMSH2){**s;X`w$HU}#dAGz%mMGtR50Cp zFweA+$H>td+^Dv;)Q=T^H3h?5ZZqOiYjC65DB-1(lGJf@L|I>s+x-wlmj=J<#Uz`} zm)@vcPyKid0>+|v3T5Cu$5~$gyHE~D;>-%z{N~l!O}M#WNlUyjGCbz=NcN$0;I~3@ zA+*@b-R_SsH_BfQy=aU~6xvTl>EZNVr(JSUrKggNq0^4oFUuwQgibs%Y4KZ2JU2^d zPqv2pwjJ_CRX27bgm37UKJ?b;RvE3Vq)js}%AF1tser(8Clo&B;*2Cax69?(eK=NL zRqt&6logpXv^W?UMdK<>&D4aoH9yVXn9W8BPM|DckjNwlAISdT@X_Ox^a9Z5Fa3eN z-Wr}xC6Ai8b-5+W_FXrPtM*-l=jSdRBAytMF4eRK!n`AUK|H(YqL-F5-IMtu?>A<& z>@xM;kIWR0Qx=tn?mM`Lk?B-vN(#Xd;U3K8fu|f4ipR$Jsah5wH~Y&@sr2DTP2`)+t1c>G z*0k#_LPtK8Mq(Fd{fYnkKr_FU>ZAM#F=2E96;(xhEYMinzg}Uhq&+)Nt>^DeYS*^RY>O^Yc^iru|Xbz9UnnYfsX?+xXk%3uM+H z-Kx5cPjW$=V(E)~%F%Lii@qj)Y!kH&mO4gbsLP5TalOZ__PK9z!Kym&ZOPD6f%R?(W_%7%Yu6Ty048&*;B{oGwwq&b5>pKZylm%X{*U{H~4kG zl+`BsH)z^m>P=NJ;avQgNQ8qmr?!w*ibrva$7VCxhzsehM=I}L+Tkf;B=v1FM=B|( zJREpo3R}4=u3q_F&53e_wKgA)UJkxgM5n3uYl(Yr6(wSOA$4F;8=QZp&BvScPDk=9 z4YJ)vnvII-#fP=N&7))!wWnV;N2a{?)fm+DXY+=k>Br3TM}o_evJz%1wK{!fHQO9Q z5Tl{wt*aPDiDA^$PJ_0k`$%ewGxfe9Dl>n1BS*7g&mJ^(A-E_CV9Bnq+N!{eHVw!c zXSuOzkxcL?{pUb&cdp%}`y-T=O7~_N?-%#nDXvrQzKm@uD}zKKCCUXn)8b7(Zf0CP zX%U+6w2=I}?g12;E~)ifM8_U#3B<+o#qo_gEBGoIJ7~PJT3~p>d{_-Vq&BC=r5ZLs zX?QFqA}D9`|FQSp@oc_t-}tAcI(4ZIiVmZwYSpMsRn?4Iu|ivWrbeg{s#=PcR%{|_ z$4F`gLDHpa&j?YpwnXe0iNr7c-p}WIfA3HGJpccm*Yn@y^^)s4kNrN6^Ei(4w5#F; zta<=abM3W%W7mG7!=@N2L|Fyiwrx6bV|=he(Jqx;CC*GN=w-PV*pw;dg+s@M)8of; zwzOd!HsRrRKQg#m_|@g9eG- z$^xaP=X68_S47#rjHZ}+u4a($jU1}{UGFn+B3*!UFi?Se|PHM%@-v-v%R66(t#;C0)ItEi`d$e zVQqi&L+f#qDjCyBQ*Cmullu}zS9Q;sdp3lqXZC?(wHYd2s`yw%Q&gaN8&0WpX>`-v zP3G?WPOZuNge%(UcB)+nQ#}hI>=j7PSAlZH-txJ#?3&^qA|Zm3GgD zcbd5Jq*=9TuZfxmFqW8K z@GWC=@h^Jn+IGC#4v2AQM)A?c7qo@@nr^s^L|EXC#b;BA@lt?Cvk-FJ9fek6l^1zl zgC~ZTt~FJrSj!$vCYdf{=h--bXW6-1WGZO1l*4G1P7l#N>q7 zMcxu4?2{!xY$`**gJQ`?{#f4lYrB+_#koE^Bp~-;P-i95a|D!^@W$hfkKuG?maZLl zl9UNHA;|@9(0Arm=T%z0el}zQy~CR78mX+ax-q}$wpwtm;u#A}+u2+Itm)i>G0y6o@Y=z103u7Ud1$@a(?hFCw_D#cv#*>Du0VNkzOr$B zAmB!Pkj?stl__Lj(C)n;P~n-Yr>p*+1wet3exg5+XL-ab=2QA$z*V{(Po&URSixFM zsvby(b*S=d4!7!G4t_J*Y9jr04aX&Q(TJQbk2kaV9||AhHcwA9)(**x)C^_!-uU;q z_1~LC*W|r?SJs!_R1cm}6w@olwP<^K3D13sFD&S<2KdHD2BP~r6cbW|3J%<5gP>`id|9vBKA_Bt=2y)HwGQtS+ zIS$xr33qf~T2dkKcBX1$9*YwGq8QXHywag@XL>t_crb6(e=nF;= zn&NmmMh7O=yA}fPqS>6AVJLm}j}VnC!UVUhd7)PW!GC25TyZ@5;>A3x0zND~NxPFm zn&P=y!E=q(goNddmiAG~+&kSyO zxwd3B5w=~YsQ+1h`7gmlyy6Taz@##+EaQ6?Q}0Km-3bnLFkxI>E6HZH$%q%Govy&@ zGoKUi1L@(fZ;t44|$>TfB+G4dnh;m0&vDal@e*E}J zslqnT+`*!G^LuI+Cm%lVOJ0~yp^I8Rm^FtUT*ylLWHu|-u{jpi-+W0TXPo~}R9WrK z_;O1p(zCP}sN*tnbGqCjn|Iy1QoX3X^p2P8d*63`jip%Gqe4P_*^Oy=vTlf2V^!jp z*{ef?3GOZlp?bs|%*@*kQs(@zrq4X9Y-8ED4aq8Cq018Dgb;4oCIyGBY%-e`SG~|( z?XI8^j;|T>E~!d$IwjExmEo&WkQl>+Vrc>?bCa3p*2;!-Vlq=V4WLUoCnadHQsEiw zDow|Qek_n`>@aAUlIg(+kBNzGDwpT<{Ux-5>&e6KZK$GYKjJ#H?EPC8$Q zGFER~=cUwTEXtQvw57})ARknomiq1qybpzxr6*~B{A@<(0Rf8E^(|l|-!~KIV(D7! z*Jf?-Hzy2?FC`XU>FxSuc7}uY7<^te@bJ&-FCL;5M?gSH;YG=rfN!LX4Xj`4U!;)C z*ljZoIB;&|wOmujmmCV#1x;^_8s$mYS1&NdyqpuDit@!+b{QB})prxsI}^)X5iif( zFKRv4yxrfCZJ1L0jp1zy4MGt2X2CQJAzmrerrK{rEi1E@=lWJzqI|#Sjua1^FH%9f zE>GU)b_JDbE_d#r8*9dNDI>0m*mA4a@!(|3jZ0N{_Xp6dSpjfBM4xwYqZch6ULZT9 zW-{M0kSTtC4fz2 zC!Rc?M3Jq;CbH90>lWei-o;g^B)^P7dm?N6%F*KZQ1;zWM|0(BphfZKZqMeIqt+kV zOtGzF?%O$tbHW@pNJB6$kC-$|3;-Z<`d38+02`+?PZKy3Wlwo^{=mis$j@$AxP3CE zoiC%*>gYi8?#%wsXf|*9;nKlHZakmoz#^3iQP%TKsz2S9BBW?6vYo_I9-Yw<3WCkH z|ClGA0u)L0+#u9e_tkX|LHTf+ym$D0rSc>`tnp1Lia&L$9}n?UZI+Tjuy(or(6HRI z$|-BP>EOh4{HjYH{YmYxBYbHw;J(r|K^R-75H`|M#JA$Qwu z?^XH%HGwlq>I`WQGr?ayY^doaEZcc=vvg2lYf^+YJf8^^H!3k}a6cvYl8M9+@3VfA zP1_ENN*9@FQoFZ9T}aM^8zE`Kd-TGessyZg=*{}2yef1XnPkQaxh@AM7__s!Xvv%E zntIAL(r)O8@}~tw5JD|z3)l6L8f#B(?C9g1+;6RJj;9?qYhmfFX;*jMmWsrNbN1pc zcb+ckR!gs}IFi9WLbq<&a8|!EddJEu7rGm_C{2@LM~W%MgCsMelht^LKJ@kT@-Sx9 zA7jOt9(ybrwZ~WJ4ZIv55Z1Z>hYYob%R9v5x-Sp6B`g2qdQ66Jdb*n(ps2`N$)nQc zP1_v$W13K9xq2oxL!a?HyRWz9ynK)SPgLPXXqk?O=%Pbmeaco(kIQBT+8>7KpE7{N z;^Py#&&8MjD?Faf14hq>V(*LX!4+9m0bo|n)^->Bld9I8P4BXO4dMy!(2%fm91R z+mMe@Rq9%JdL2_f=l;VdOT z0;2Rwc3=iKlCEQ{z4%c9O((yyOG^za=g>-$aUf~hMdPkt=a_55dDK7T5}GucO`V&R z(wJsWm@B4970!_~It-G8i{|NDTg4ab3iq zFgun5nWMTZuH5dM1cUYm<@mv@2)kJ`!k}mM>Jc+iH}9;(260MXxyfkW!Z1E7jXHJD z6As9v_1|gNBGtxok?4`CX6!DJw{Z(5#JRC zyu{ zE5RoVJ}q2smgI#_h~(uQOuj7pqcr;L!uAVi4+}DbFE+9;9YXl96(-?9L^-3$pSj`5 zqLiLx|K~IQo>6Th#d?5x*Crsl6y;qd5g2L-+sn#8=3gA&(LK!V>XJ6P=9am3stq(7 z0%->CRodrdZxGUyt1md40@c|&_t)t?7fkNV{W`WD6@RDs2T2C+X-aT=m+d^P@1viP zF?34}8c)y5nA?ivibR>xc}l8r@ZmGPp^P-C9^5W@$e2nQ?O*7atME(_kb9TA9%t6& zEwLokT{?ZK$;dqm_p_o@KH4k|w8((H2wfACyjcu=7|>ngbwBfVJw<6Qr|yC^z{9@n z_F6jUnGeq>IU>Ir1y6QMK>xr#DPu3pljpd<0qOpoO6!jwADE0{;B~y%|>Wnck@QEnH*4oM4 zx`!^HBpd(8eNL^;Y_9x<$*z7qb7AkE!_|lDRAT>B)N{pePZOQOz>u}dhuf@WgG{tJ zI!fnwAF@D4UkgbeL95tQDufvV>KoCT%i?+=PvgIz36cF}m&udCsKxD# zn&XQ~Et@5vCf@zQjOx*?Qz@vo{SX0_9}I^`(YqzGmO=6H;4*n_QPHF&53htwmw<^Y zRc;mYJjCP;!*khlXmmgQsKg{+!Tcmg4oR)1ZDHbzO1&pzDpu{beaTmOH!iKdzChnGfujm3@V3aN(2lB z0ve?e$XA)!=HdPtpf0Ib{S=OoGH)|8 zA)R;O>4%ehxR4tWdwPA)Fl`ZpIz)2oa{T=BgxmyjnCaS4vVyzo%66!L? zg;Z0(ELX+p#*Svyan|V`rVr>?Z86l|eY<`_{n5n!3hJO7!D!`@;j34@>|0jZ#}OH6 zJ(IkV$%``F54;J(3n0Veor|)Mwr#Z$E(so9XZMmg7hId=Do>TprQSW0*1N zRkWte1hbjz%_HWlE^AbwWx*3P=`ricP1QG*!dK1J_sqlMY9)*;GOTW#K5?eg7>5(q zaf~W3d`&h9&%R$TB(Yecwm#5jKdfLaS_I5cwaSZ0912Y@17-#dyA*Rfm=8XZO6aXb z&@11qBG8r5#hqdCQTgI%%hQSkSmEgT+c`OV)jsO~5ozSy*N8;hxJ`CK6T=rmJpjeB zHuBCrv4T35A8X}Y8CnZ0M@kzbk_&QqKCfl1Q%I^PE499Ec4Yi^JurQA#;f$kh!(Si zumN!+(Sh7_M6Q4MeQf)qK0^ZC!=Zy!9uvJw&ImG_E=S86itfz|@@LXav1>Ko-O`2+ z_cHfL5$@PnuejFyw6~@Q$c7dquS|A?nJ@|NYtxry?@#Pc4#+|O|it;4QJn=#lrTG*Om}=-P1fkA?1AHMnrZQj;#%aRR zbT+y;8k89I$ZEI`C<0~pO8~iF56}W>u|i!q$COstT;F%cqq$6 zvY(&yAcA$aR{^RJS&&zvc(&uEiGwgn#vf81iM61iTvja%aI&-+?IW z^AmQ2B^(2odF`$8#!=e|Uf`eVo_J}lHRYa{`{YV&t(z7>mb+y45EV}BT6kZmb92Nr zPpp}b?sNYW`vrSsf^@jgmOq(*Ik(FTx;i(vTO{6;d+P@*ShZ*YVD9)b)MzWdUbX_m zZzt6COdlUcV=ZaYW6S!M@~oy0Fru z0C6b5`3cs)qQz{nJ{F$5R5rwH9ZbG!J|$SP_x-p0jYkBSuFbtjcEgG-$3K_;tL)*# z9%xR8P5F+|ivMfZ-6ZZfE*IMH{w8G!n&eBqI8#2Mq>bvgD!6QLm~@`4mKxg!)(qqR_HbZ&#Wo7}Mzt*{bG;Jo=0Y_%R=XrE8AinTGa5fI<^#RV&~(<-zLMb}yU zTBPh*TO_qbR(Hi`fwdPk>+#mDKh_*jNWBP>sXyt+5Aq0aIAz@$ko3#Lm3t_n|xJ9*wcdpa(hy)=sDlm<{Rk#}>n+$e;Uyk2t;$Q)b_<03n>QN1><1+(>DwJv`0=$wV>ZF2nRI ztPit#<6>dJ)pEWdg?uJ>_0%@id^^HA-`%i+tZS~$IC1Y+6{eCv{2!|4$~AIGWv|r= zSA`%D5$^+qTR3y5)pLd-qv5*&648%N>osmXz$S9qHTVbO~qPZCVgd z&+isCyeRdEB5k2s0jZ4FxVD$Ibm0$)ER7uA!*i;S&Xh8gJyM+Z8){6gmt5pNx%=Ze z-wv%cY33QAZO2mJW(5q)GUIiByseqcw5XgU?b=l`n@MM$vni1JMS;mEoWHLC3bD9t zwdQY>w|(*`ZWObUYzqxt|MrJ+$E5nFT+zq+f2_3f zeOI-HNjdmYOT5ZQX)&qPskzOPaB-+jO7M!Zy| zOHDQ=8hjN>1Co9vgkh_ZMr-*fs*y9#x($+Wq?e1n*paG9ya9kxOzh`g<)A@900NI} z;RZSz+z|Khr`J^r)$QOnLJ`GJ7m!odQnsrjR<_A812vMENN2|%`nbf_EzSO$e#+_ z%kSlUEi>WJI#$wtEM9@(cN|1?Y=`spj2bxw>YI;YoNca*9*RqrzhQ{m=4n4X8Ni;C zO?43;HQ(gN9t{j1DEs*_exRxM4#7wdUyZ7$13oF}IYU3?c=n3>Vf?y4p?dsul`Xn4|LZ6fF0*hBxD>a}G5JPgkuy(ru3M9UnxfCn39%7|eL-u-v{LJmlN~tzCEqX6RUaCf6(;Vy%B5-fAWgh zi$6GspQn2?C|})w?#j;5<8fI{v+SZOHd0(gn>YkuCJJM=*H@MSvKsXKHk@7yLsSs_ zNeWbU4;v40RvsEsP=SXWF; zN--Zo_>6tT>Pj!JA z*^a%Y(jdsgqrDKnjE!TxkU(E2=9f76%%~M-zogCE1V~taUOM&uITp53L9+9xe#g7< zZj$@wJK6|grQ0-vLD&TWSw>}gpEo){uymTBG^p7w;2+xbMTRd6Kgdd@6?cGNta`n+ z0yFQX&{~UJgQ;ud)2}7^@%(-@q8Z-1tqW9W$LTps0F}QiAPcz;H|xIJ8=8JqPPgiH zSl*^&Ieu(`Q%bD(6?y;@;YHKaAOz)ZFrbrAf{#JL@TzsfVYa5_X4gekMQB9qTOYaf z&oahes6Hv~DVH;`H)oEgl~nq%R}xZo!?kA};315q-39;6Of~%c~#(|y9s9{wE zM%zhmei2eC&_GCIY7#r@V~gCLhsSShQ|2Z7D-6uOH|p^m?cF%-jtmyGU=k*24afl= zxeD5mo0+*{MjJiPzsTv`)TP2v1=m*YmGuSTzFW$EraSg!?LKA0|X_KC`$-6acfuvN~xhMCpJ}VY;+dmYe5y5G~bUtJz(EO)@ z?xd=xyV`hX&nRix5bba{!oN?1q~26nT*?&-J-(eP?~Jx^+LkB|UDFxqJ()863(%W1 zcr4>1A;yCDFNId^f8fk|43YaAJ9!!w=#vKsP7dT z5>h1%D=@49+)UG-zmdkh-e76y6k2B05yP0%%`uyoi>ZIc<;#RTg(uB5oeP$-#I7UT zbiYeeyd}VSQ30VT)TL_6)0zXYp+L;ru|+y_+JO3DPlQdn zyreGmCu%?)RU7dL*j$}38{NP!;sz2W^5R4k<;-G#$HxDqkj4ih0A|H@+xqt|_2@9x z4-rL#3JemXK&|^^>QT#v-Ys^2f_*BS>jJ`m`<49TL$(Z{u1}fBqPG{*D`xhM+HXEp zBZi&K5kH|qh`RejzGf<{Iku2u$1`T4#>X4W?0HXp?c;(kNY#s=&T ztXQNW=)jcddzl=Ug!Q=QIP8UR%r`2!4tcQS??+dFE7WJ|48c7{*CG@ zg&GZyEbfBX4N5Q#VwUe)arj?_^ZvI^^PVkDhUD>=KNu#_M-CZ@gML58espO0x&DnB z?{m;Z&WS^Mo{r1vdzs)gc4-n`p0jR^J zc_LDj6pd|e@20q0-|p)%1GP~kWDm%4{^1ijTfgQVZOse}1#>DDmwrB_U}kIB8x96v zF)|Ca@p<>Cq{m`ER{UB)orzZNn01$@Wdg{c_v^%vfbJlWfRo~_b-gu1ePZg9_~;^6 zG0?bP>y!d;;oTfG%y5OH*%dlZM?30jSsJXl>GZ`r)m7b|#{m6_W3l5`aHU&~6TsNv zs0;L7yJS?sEkmDR1CD3jrDs6J6p6xNpVD3&v!u!Xt3b^7%Rkk%=oi?T*%+6reRw&7x`K9g`9a0cP#!&@NjD`K<$p0 zNi|8B-BaAUDl$TB(p?l$4$Yk&POA*9{OnMj1*{&kxl#NB%2#?;O5JBWCf zRGL3|y8i;Hw})AMSjaN4LF|#q>jAnBH-iV{;udh0uJpXSV+sy zQ6xw+mg8wN3Ml7Hlra~Gkc*eR8{Xhoy$Dl|A!<%+jOxJ*59o(ix!rX8vo~Om<*U91VMUJf2PZ6XbPpwdWeL%Q&2X5|$ zJ>B$B>!&N!o%=p-C#GXRpMv^+-4CucX*BLlV3yUv9^pQUapHjMnBegQv-8Dg%ezPD z&vJ!!YbPAFYxT~2dc^0b+(%OgzUuRXAuO=$C#TtqzKFPPb@xWFU>?Uxw5vVV_+{9g zkzgyWW-*U*BZFh4u7YyzW~+;+QrFDL?8oBIKl(n~vg{34*fqBW_9EE#cE?WGpPCh? zuedjVcF5hGsX91}J(Y_eu1>A~&0hbN@H&1e;)0khw%8saiOZcs_v`UW`2Rq}s%Zp+ zO77fXEP33cpij2RE|fi2n_qh|A@3&hx~D>j7ux4EFFT*1e5$;AKMUX{x`IgbjZ)VmTvBW1*r`5}zW@}jTg@n({4GGI1x_iU?${qDwFIvDD@~oLn)3Fy= zRIb<2;7<)2VQotQ!$s1__?6Mu{CgPzftYJA=D;K_-{XHUGWJ2pC&}Ye z&*L_IA=(IIw`MM!* z%XGE->oOZ%@Bad)0#;=eeoi#M_*iXBUG{~&nEOzh8CF{7U3upqBfB3{G_LwEe8RaW z`-Qv5Mf891{vbV+I`r5Gp{Ej~i%DR!32=6ruM%<)i?7FIhW^fMjVqj_=efMHdm~aV zFTcP}M84_Vhj6X4??v1yHajAdRs3ntLW4X3|4zunp3%30dI!tM`{jZz+f)<6y1q`SF}|{}*JP9xthN2;qpSQFNsIG7%Y6 zR~`0t1D^*O5VHz2%9I$2)ZA1U8obZ&KP4m~f&lbTBAIts# z+NQ6t^9qXDa-{DjoIApU$0zi%l(qT*SFDnDM5@@B;JTC;vKiR z3c2#!?HW`iULg`i|afYx6k(H0{eRn_8sE(LE0EiJy=i|8`+ZA%ADH&YH_D z4lWF1oo+B3i2glrb4v7F>*>3~jvjpnx0bweCP=X?ZkQCW>X!Z5e%>hgQ&Zz>-$b`G z%J+x!TFF0zM}1+XE*2bUzTTN{|Ik%uo!4vI7SK%A@$-_RL6Tb)yA=%mxa@EMld`fo zn5(^{{299BN0nxDhQ8&q$@KMF&dyxD-@>8BgXQXzjioZ^XC3@!E&bp zJmOzCD;wLCMXZ=)d^L$G5UB8Kb5dPkUKxZW$ANCFZ7ppbi*=DyL(Pr5>GF2Bc zs_3t9H`m9q42!jR6HK#EPu@)I)&3j#{igsl4*o)Tdz8#EmJy``HxZsl3l*J;Qy+If zjYg_BFF$cCf4{q3{cekJ*v#;(L6gTrHyvEZbYH@4d*HUx>6>`GsOP)f@HMxa%`Uu5 z!XsA6r)#l-i}%^r2V{W%II>^wGTRW{z9Va<4_+<0MlGA;$YUMKG1(XXsrhphE$-Pa z5pid^2OoA~F8uQ&zLbEeI2$A8l_^Lv+Tq3;1Ll5A5>-@J&W0mli20%+i(Y%1IY2`w z>fWDn|&uD-a3^NjcUgj zSFZ6|9bQy_ctgrj`B}d^F5j1|!*qgZU`C~ zFHj{rpC?0hteR&;vpzkt{uUss#XI-Ym!k6yRjP6SusV)_NY?&JOZ>=ZS-&e_5K45%G^Gm~{OJ`23?0p%L^KNdNxtaaZN719)YAL&B zyq%?n=uVLHm`Qh;*+yAWn9C|2gGLr6?At;JMI)j`BAZ5q$$6FUL?dyC(1+Fq&P?kY zpqZeKi)SHg!IYLtRO#&|>{~buca}H&#p|Y`VYHg*QM-ry0hcwN{=YiuF{(cy` zyI0vOZ}QfWh?;B5gZlu>e#qU07-)R!8bBBiTR=35A{!2oXPY(*!^DvXqy1nEF}a_{ zyBn>It4Oor2qMCWT=c>&d0+*cta58RGoP!dp@1co9+`+Y%FZ~S-FQ>+GV)-8jqrZq z&3l5G3mKsbZeoFzb;d#d?)F&&ce-0!M_|W9Q(wB9x_niJPGd^reuct+%m@a|iG{iZ;C zcBm8*N^YXT?_B(w$bVDIe)u5IQ>-~CRCb}rNbxgA6$@laVwemPg*$052}tztW@5(E zG@y6Jp{0AG-8GA+g1)|jdsI&D)8)PnShVS;prSKgB4gg%(Bpn%pSCVQDJu)VGkc-M zrJgHxHo4{zwrp*VvO8vF6(vUV%Bu9Pbw;=mof}4mn3gl_OC%d#QOGr~fr`66AvrNE z@1XqF=#iZsBBg71esyZCRZ+Leg)wG73K_o#7ZZz1dB;3!jSks)$D}Sgr2j*37v7^d zQ4OfB!Pb+O>U-siL4j{P^{(lX`PU&WI)-n`ss|V}qt;PaQ{o!?9zwLRR+}Kw-`F>G z%0}1l)c{NGmU2M7Az!15NlS^cJ=tc{McBr4Fm_(1%fZ9e0=pEv(2_H>5bcxPQTAupz2CkqxarvIUg|MAbi^mL^BLsj2ehHOyBAx`f!D0%h}W!(S% z|LOlVM&Tyvnort!4-7nS4@?B0AJ{kix32&F`ZD93uu0JW(vHCtjnPq)UGM&Q$y?>0 zE4#Z|8-VZ8p#S|>`0xHW)(6Y*5Bl^>m842j&kR-AM(*1*hyPPR|JJ_u?>ILAVAMU7 z!O?R=RZXT#>-nPYVXFTpN}4`IUNYW2-A-sd*!r*)#b$Jh*d|~sZ2%+%6afHE{SO=d zo34LR8gchWjtWi@shf)REAX}QweWRYYaAqe>5znti2g2|4-y9k)W4PfFD7}|{0LBN z!cID&9lP^ckbK|Kb0Z+JBS%fB(aS$wTMgphg`g1HhdduNWCnH~FcU z`Z4*b_7cQHdmZ*hvqjF#bW#$X`?E+!}W2gMH@!;{H_j_61-b1MtE&tHi2-=lp9mV-t z(LkH!sf3YOCQFpDc+00zy=zTKm?#_wAHt*{=ras*qf)y+-T}M5F!i~WpbJX%LVa7s zizXNa=q6FM9;6dTxNmRO>Ozl>yEA?aw7=yXq>OjI#iGysQw2Yec((epOt?kWp%JEx z7-s}&ss8Xn;MxE)Y;`3YItGVOb2fNFFld5#`}vokQmEbG*_tDp*vRQ_7(J#Adm5h_ zD!d&%C7oC+;vx#+CT?_#MT@q5J!AQmHdw;+Jx%Jyje7%wg|BNoXkv|??~ybA2g!T( zQcQmoX9aL?z~6tES4R!cuBza`5lq3pg-vWWG)#%22G2_SUgpBE2Gw*#4AqckYaZJ< zG>j|4=_V#iFAl@T4ah}b=z4~?aKgx#RFA}Qy`-4UHUB-@`3Gv5ROeOTt0K5>weFXk^SS@)ycE_CG;|{L2q3^E%wCDxMnQM_tKk zA_*b#?b1T;?ajA?eOEqUDFviTu17@uT_8oJ#W&ng$(YERFD0C40iF5R*WggF#Q?}e zS)N}lwWV9OT@a%lp{|>1SmzmzZV=@8+nCq}^WgKLE!{M3Mjv_Td<4CgqvXi81@)&* ziQVj!${;?-N3##rJ~7}M?OE^SOB&##_ zOtc#v*0s_zm7njmD^wMlI<**6$vik95cm(*H*}p1n9{Q64n#C_8hMo*gFi+po0Gr~ z@#^gfM&#LV?#${wj4D;=1n!d;{JZuuC)n9Zy8q42o*;9WfNyuNKZf{AfN*qKE%g#MaUmWFm z;+s7{^VeII9_7A#4bWh>l@OM@;J%#k~^dRs*JZpwhi5ReRkW6H^3uaJIJg7!Fv18&h~q)4}g-VthSGD8cp zA%;PZ4vxPH7I1+UE~QZw^t=6h0@&B2^geFv8+s&BB1{X_tWBw;Xuem< zif$trp;yOSM5^?-t%+fBRyXsrQuWMt5z%Hw@Mr~^yo!B*O<0nm9{fFe$f-IG95 zD}@Tm<#~^^IAki(&Y83n$G|ZOFebCqb41Z9zigA@Rf(2cSr^hJzFX)5kV$3J+HEML zEuxC1c6Br3dQ+X1w|z9Og~y}_{fb~00xerty*@)xvp`s*l^uW9<=eHWjaUcXH9>gw zw~19fB;jBbuuAVe59OsL=fK(<>ZYVg;_MxpudI;o7|Pq<91Mzc3|>|K5zchUqH$y> z;fIk`;HpJ#I0PD3z z$Dn&b_atgKn#G*!>A|!4T@9oi_*A5>Rg*X}-0$JsbA9-@LLzB%YpSAEO_w{Dz}Z#N znhGP?Mdm2z)7#dnD#<&WxYsC*&ojd>mGMUVLfd?$(6fo?FIE5LUx!}OSm36;qUF|Eo zg4YsN=y-w)!9Xt}nPZ9P(-zXqhA#(Mtitm#LYe1Bb(;dk+S90Bogoi8h|mr-0?ig} z3f&+&K4O+Xv8@DG>YD^63M4m5iLV`o`OZlubhXhNOtznx&~~eX#!z4(G=etTN9tlU z3@@6_uD{NY_X(}+!)p#h{M-`oW%w4VkIKpa@RR@Y&x7y}LesB2ju&!0C`_Jp(IV7O z-gzNCv(2s@zyi>zVsb;f6^m^#(*Zx6$3EP*L3728W!b*nA>FYVEjrc>Q<|-U31Iny zQ-o3=a!)K^vU1l|N>wiLT8WiFMtsKgDS_U+mV2pIem}iBoM;ds-W1W@vqarKlH5scbiuGF$M7dXtjj#jr$hK%j`^ z^N9wMayK@|@=0)7de7pxoe*U^*n`ohCYbqxyxUzPsg$tkjcLMFd5=vT>{Q$Jy4&2~ zJW`vuDMEGZs3tpLh;~A8xcy(lMq3v8xjBGD#(XQlP^*JZ5e45P*Mxy&&^sVDp^UE5 zAy2RufX_Jm973GBlDe|_L?hMgw;g90C?a}hz&)pE(L2q8oJo=UCTm67QvelHknLVd zZ*gbTm|}?JzgbRV==naNsuT?b0htKIeekTOauIT~{U#ZrK=@*6DB6+vcvudC8yMi1 zd1EoVipuZWPIIgrxL`+XiVuOczUJu$bgbP>v=Np|UypUPsMnGo^zp#sOlB>{^LGPe z6jug;y9e2p(Nqj}7?kRVwp~;}tI6+{uLY0HxU@3>Ax(oI0iYO^V*#ck#9#*m`;F)N zL4!E~+;_VrX;?(cfSOK6apQedZR@o#HA(kR0ju#ZFk|&rb;8J2a?rGk$-DErL3Yia z;nt>`={L#oqhoqw91i$JhfUv)l&`=o%_HO>;aKqG7p%bAPRSIMhtdFO@dk@5MO4^tRxk!=e3hZL5(E#{VLu6J zk55ikY&G);xlfmy8J|=%kpE8@^k3#3{h`P=Lk?M~-#ssw4v~8>u(<}R?V@8tCdzf~ zx7Dqz7>oUEhe$m7osAgNf#{H8t=0Va-7}-f@aasadQr}jV}VO$LY23`p9)0J;a~#- z7ekJ?y*HiZwnD9iX}W+KEI9&es5R8s?3VoskfUChn#j7^CFBUoeg6)bmUgwlNm~(~ zx*%Kc^L1_3$p%iDdb%t#>XOwpW7wiFyPhA!Tk9JV0wOX>y8SD+oj}r%EGIzS)z$2> zwfeBtntar2W@2mQTU4$0IO*f=WvrxFO1MmS`*bG{g^`_|m2AG50_6`GgujK*)KIOd zIA33j_~EhKe=&?aIRq_{KO>%?KP|(gDk|3?@SAW`^yqg|SMZY5+D)f#Dg)4kWg*~hOyz30;a`6VZ0X+7G~<&k zZ_U^*3SG)$I5LT_j4aF7tAv@Heo+(h^)}!&ic6LADl4OApOSX3%M` zw^~=xR`(;fj={h81lf(aKcEdbgQ%uXBn>e4hr#@Bc(GYgSOF%vzkzKP&aR3LNXtmS zQFee@^r5GW9L(EY7`aukSQx}QO)bMhEt?ww5X>G3@wO0eC{2uWF(-R69;K9aswz{kaSpXa zj4B%xY*6f4>@5%^TxUpQ8S;HgW32uVZcxp1hN)_|H$wE_e{N16)Ns(%V28$#UNm1M2$Nv{vh8WDJSt1&Ttadz`1m zq5LjorF+IC7@k3O8+i7Hb>o8RX_CjpQBzDHaQr$TJ{!iwDz9IyMQe5!3Rw|wgAVr6 z>b`H})~DJ#>lX-qM*Q=^9u1oxuCig}!J!~n3RoA8-3U`lH+TQ5fy^z?f%d3b#5P=K zgRoknpu=3qg9Tz!Jibh=2AUcAU0iZ8W5;B5zb-V)EPX@4wS91UC$Jn^SdC~ZeOPh+ zVfp`vwjazQ=t7P!K5Sfl?B2iKdN9?-g&EQ6;rydfY$QxUQDysHksbGhy~4J!%P7Wf zhn2;F;n+km zo);6Naj()FGtV!VV@k5Br+>DvMvlf3r~J9kACrPG&4#PD@#;GOkE$b^LV^Y1NaojV z0lLEOHxhbk_?mj1T z&B|yAB96@)j*@1#oa>i9W28I*DOZ0!V|GD@G28wp4mRc_6vGWEQ3>tmwZ)p-3Gj;)K5-0?!403B9|K#kGsrtP@v5%d<-Db&C>N!sQCsg@R zayad=6)p9{!LmvXZc?}^G9u4gu`b92Ceh!ulw7^w>C!a!K9F6V%h(INKcZm-Eb?*^mYY9$fxCo~dS7tLqtNa$^ zS&F50KNv;LK4t4u=^{1jJYjteozZgf7!B8`-FK7(STtQ~5fkYK7#WaJb#Bv&!;hP$ z5QXO7=oKTGKE-2(>}Uvi<3vN+^@|D9oFbu#^sKGgDL~>eF@(`Iy`zTiO!p)z^`?{D zD*@R5K-`4~(jKSpb)W@^Ja!MZQGZw!VzS$%k(}`u2`&xVz@f$~i^)8*?QYn$$VQ?cv0D!tmnN-}8ZmKhy!FbLT(u3|&;_UeGI zcH_7%Dh21t`1RhnMSMA>Hc*{5nqI;7Uqa0=AWMS3wUJmaaj*Rb;V|lGOnfUD{Vg^b1M-3+ z2JZB5!*Fb5U$$|K>RGm@GUNLbhzlsNqAO@s4ylWApVb|W!Tj|*bI$yJbKaTfo#%PpcV?)6)$v%W3Az}rSSpDBE>S)R z)WZk6ALUH5=c|s=(=w9?==wKNNtac%%?lhD&;U7gwa5b`x@p~|78p{C^0br~U>1xI z!uy-~qsn5)*`B*nmmMY13rk<<(?ck(6h}Slpj{CM*;k=Y1e}B+u-XKIu(i|pFy?C@~gu=)k@Lp7{$P)9d#`kw-D}0i-EI{9{`RuCqSpZLU$hrjLQMD}jFU^<9>s;L zG5g=<5~%SFLWHboiBZbGXbex7@GYJo;D>_jhg`?11sgP9fC_n@&)h@{3B){D*g6?5 zyDFh%a0M4r3L6$Un%mLAiSyK`-oG_$U4>rXz%^@N@CP!PD#AA=jHV&9KbC9ulma?| zY!nYu3KO(?ve6$CIy(q6jE9>}in0hVXS8RURIvVddtgyjr zG*wnK*vdZ%@he7Fda*7c3ACuVrK>Gx!%S~;a0|laGw|FaZbvilVGkj(O6Tm9`Ruue zkeISk!GWGwS97b;@kkdxH)9TRlB|`mr_u;Y3 zCf2(c)Bj;!2{n1EZdjP{gbJcq>6ulwEQ~Xg6Z`7^&_~M6pHgRwx>I<*$gTN-s5-PT zf0+X@m6r(=&-d}Jz=AZ>C4vNP7danNFO!I~al2Z$SaZw9RT^;3qCTXhk~>%HVIEsl z1H8_(A3K7s)k<}1lJsg~8oJy;ce}xKNtpZL=t@HBrAb}uHDchE6o_+w1i@^1uh`>$ zL{+fRGcox2L!h~LdWeNIC=Y#d7H!ee_8CfkBDTDg++Cj=S~F6OZ>LOCY(r(m-RY=# zO~-p8JxneCiXOT>rXKP#xj+pRv&nXxzEq0@+1t@uVT!tPNSfwe5)M(J?@!bF+rZ^2 zbJd}P?KCRJ*cBqq7*M6(7X_HU5H+nZ4=P4DOV4|~@|BX|nrK!&E-WNzWCiT2{M-5c z$Q0x21;BoL&Yn3IV0c#0?JZ&%S`SKgB0ce0YF(c`DWxwBbobz7m`A^ ze{0;oD+=@N%uezWe5sXp(Oru%_K%+hJ;LT2=qMfBMX*=vOIn)Lj=~BvOdbLAC;DJY z{^homkp#d8M3!&J_|)l|&Gnhmp6rGmCnn{vE7g37e9rm!yhp%BI(AA7GqpEr3gVIq zRJC3KRt4EMy6J*8eiV{Mb9TJIDW7SmrkPjI$?Ud98{|*HWOEtBk?MWiBHhcoT9BaW zcUoZ7EoC7lCZjFC*3CG1hkfmK{MNq`$6s*e(tONxL3Hf9oN6v4s;p#-9bl@#Zv$Fw zjh!79*K)(~-*Gj6B4GJ+2&tkl>#Q$T>&L5zM>QdB1g?B6ZTAyeIKKNuam@SNQ1*Pc zhJNnrP@HkOwOfn=m@w2`787a8=iBtXdVw*bVA6eQ8L5VN6#e*%d`LqI(keQ-=$+TX zn-zy0)!yk^{v)zI6SJ6;ZK>}*$q<2xM5ws}=G&CFZCNYMi4m5uuu1L^&I;?JcmPfU zYokDgnv%)Wvfqj%VI^%t>Fn2YH>V>}hn0yBaK^iy5_lbEdT~S*C%c@ijpcke<+j0U1}kt)`T_5w z;g=gZFa_pVxuNN?{_wE)hP&Pi`?k#&)Vo6}OZ?P^J~{YNDT1Li%Jh#RhrU7k=4{u+ z@x;8=&6LyPY%NciQ3InWpw3foA~a=%3hQ)BUubm|b?X6IAd1E!IK`jGBog`6kin|M zMDK>l{FKHNxbR{eH4gj@Hnx9uNW7vPe;FSIVvQ9S%rK7C{JbK-BD{Y}C&$C-qIcuv zi@LL++rP;{TZ^y_Y2Ve|R6Vw6WO@u#0JDx_xXCl1`KSxsi(edr)kgx#QX1{em+ua_ zzU*#?RV8!#7X2p+fm|-N$P7;GNqmfEPDj!#6u?*KjtzzDV^5lC#BEbmPbFApz8@q; zS=yB??a~Z-Pb_!0LYgWIJ?lqjNLUi*KsZr$xd&O2>3rW?mF5gOOnymQdoAOL6}G9% zlLbMItZkNSM9jqvdFp_-rQdTlU1#_!B~Nb@7)!bvQYP-?d6-!=?gFds!w*5*uLBq< z=j*%2dwD5PXI-js9L9SG!!dung(Ph!@7r!qXX_+JT2Gvs`1RbQ@jt;DV!P-bLCQv8 z=IQdOy6agvS#6u*4+pFeoXTVFos|14vv2RJ^R47l>z419FB$^p>c&6g<-!o2jt6^| zbkwT(1Rg8ZkQ52VdMubh%!bDl3v@%5eS>@q$#d1bKCtwwO!s!9i+)xLJEm%glsGWx z#cyKFx`|zFsN_-?iFOfzU~InEN%jScStFu_bOeKDaCf5TdfD8Ek9f7Y?kFn}IA@=q zqwoe`z!-%TVVwXKqJ-QaCsvI}g`<7{WGR|v9v$sI?1lq5!SR| zP04iGX=w%+uT*OBmfB`3|6J>)=+Kz~$^mU)Gy0U|sdw1tp1D+gG7OjzWIp-JKAjktf&->?g7J(5`?Nuc42@ zQ<0hv{kcw!t<%K8tW~OjUY&M0rO3=!`C+^oCJEL)U7F{Q zd-3jo9ByA1)1BjwD$aI%xA?eanVaNcO`n(UyB`iu<>DpLFM!ngJ#TR^PKY|ybeomX zE$@@BfUALxrGjZrf@O$*+u*c>jm%H4-Ba>vwV-fFHJh0goO#NSJCYBao!t?F8_7)X zV-WbwJ>)*9;s791HEZoe@z@}rg+fP(WJKE$$<*J=v8gv97Ymg2hR`8z4?1t$CY>u% zytuQOQc@xFAo1{6AN4L6r{)crZ7}AJ@h4aiJNpnKADv&ia2$pRx5E6j3Sy1G=G-f4%3a@qCs zd;|mgh%w8wu7oO}aRs+7aBBr<+H~YhLmGKMDbpz2Kt$F{>%z*W%aTrFUca9zR~mfg z7R&8ctqG48V<%&^=tVjNW#;cRVWJ8OQXHC46bk2vA8a4prk#?SPu;u9o5Ed<3xWR+f?A66F_sst73G(@84x@F~i<*C3z60~SR;JtgyPuXl z)6k7|<(hMD59?=!)#=5khUd42E5kd#KeI*$e9-2RWfzH^=nVFJl+1g_FU>6XBbvl0 zrsy3^s+(28I5(vzj#?NcyPgwe7tm$>uO{0ES2xf4cc8legXUs|Re`%4#t2p>7j}}6 zpsLDVS^#6o!)b8?Io`9f3Px&xbo}HFYKR$RSj)e;gAEa z&QwhP{-b1yiz@jxm*h!a;zZf|^ONUzNA2f!-;$!t+LdVVo)Mn<@4Y`CPIi6p0ZHqF y4|rG~tm9%G4(ljbN5MJ@)=}{P2?f}Yw?>on8Td2ig-XZPs$PKfLzO9+V}dy}v2qtr}xgR+i2`##erG+AS; zV~{lllV$9~c}J1@zRq=>>zs3)KhE!uu3j_m^W6LL-1q&=vuir4%=?b*qoSf>R#&^C zPeny{o{DNu2R$9|3Hb0wBj7)34}H~3R5{HjCV)Q}?y8x1P*JfRr2M0HNN59=C{d|j zQNDhkdJ4aH-D})m@!R;bkNppg4;&D9cj&ay!((r}o~IsG$vW4eZ+7@(Mv-Q=Vu!v& z=2gegLbr-ncL!#uog-vZ(dt(p7)FQz>oJG zzj%g*em{@W11f6J{~rFwg#T&6|E%DDR`5S7prLOxwI+C46HUA?t?f1m<0H`LnfXbJ z@NF%zo}l51mG}Ml(hp)&>^HI>7s!X3ik0K{nct7Y{e>>2E!KS$%jfXC!iIEUhp$n4 zp-tm&4z2!-JJ$42gZw7RiaXxpjRU8BG$TigrbX>Ci=M}H-JHy0hU(E4tJ9WfbG?!i zT_KLdKfCnb5sQ)MnzwqK95iqyO2!RQn&bm2%{ZYpn490Zg5eWd`KV-O`fMBX2WZf~ z(nGhXS!09`ZDZD`mB`gIUmIPNa}#v@VkMY<69<~Di*SVhJ(d_3CB+^bfvB)2yQ zX0FJwIIuB%H$8AcAZf9{ij0p4>zyg|FDC^ZocGpDk<@&=(!<9(P@yv>H6pU3?;Z3Gpc% z*TtrYl94{?SS|Y{GJ3;Sf6p=er$iP~?O26ZrY+}uS9?oCO1}uN2uI$=al5LyepJmW z!LbA~AE>Yi54Y+pR}PzSwlyIU6S)XX`RQ79vj}#GRstd}{(_=7slY72-$s7}|0yqC zJ3zwP-B{nHqfCE#lYd2juJp>upGvTcq52G2-us6qde(^iq(^veHK)5@0@W?(6-6C2 zvb}H4r#2A2LqW(|z^#KU6dP?Nw9dF6xB31^aWiwfK@nA}j&XRBHZkFY}=w9h?##avl%iEeys9HJ6DasBFS z*6O1A3!TaFxknfaSSH_+nq@e4Wsw_`=Y7!5;L(oa?S7c&`!?n|J_7MX3hN}imQzG3 zhq|(9tufp5N#i=f7*-vb@YU-pCYX4{QTu*CP_(nXL|xtr-s#bM(X*)|3%2!(k9-uL z)-V@X&a0E_z0z;JG^tti*eoh0cBQ?N2~!lbwb~niR+V5tV2DHX7m4#=nbW~Hh=VN8tK?&2EKaRdBE zEEQJ57XOy5%%u*Uyo~2)v%CtqotsN0r<46v&t;MQt^I19i6x&1+-|ZuzG-aRpQtVf0u4Bkpl{Wp(2+Jy0Fm-is*_h@)dNH>(IOTu**UXVebjpL1M-^9-&_<;$_x z%Q}YGG%VHCa87uj_ltF}+VdOdUj$amlUE{bmqt`w0y5t}UtL)-!L7R&;*9LNO57S@ zC=C`3g}&m>%X8H;bt_u9brV#8p3Jp@(OK9?wsY?2dTKIVeQ!g0d{ID$>?ib1zl2Kg zHnzr>sDs%x?_Zlf_(Sg2?`bDN-z=Tiz9B0uWs$mfL@r6oA+kcSq^EK9U^aejq;4@v z0jWEODvj=0l=-x1mX>n}hH}j;=)xWH6N+oicVC#_Ghr6+q~<+F(`?n}krwxB?2>+A zB!IA`L&=iFx5bH(7TBV4=&`7^5?d~je(vZ%Jb#({J+-;y5u|!Ry;=abTC}9y^%fU> zy)E=yPY6sEjr~9qKY4LM=qh%itof) zxz&H3@%5i{SXv*^=vX!_)3YQv+(Oycv0I{E$|#6E_LVVoA8pK1e%l+?!fVE-%>fws zYV+-L^E>LPlo*S)#~+Q--PTtn>rWs_Ex+75j`Im(Nqun@ z8-PdXg-K-)4=ii7WryaIOAIDlF~azB3GkKS$#Go1;;0Wig6w-cmEe%tj*uO8DoH7e zhjV{y6NgV47;wFQHVn(XAU?KEa+nwphpF=K<;}KARmd3k)*sNpoTF8RtTX3CzAfB7Q*CDU4Wr(QA2u~!HM=C6rJbEr|CQR67y=yg3ozSFzu z%0Hbln z_7_8YTT<~3c0(VMh4Z^SnVlBSB^6g#Cafw%9y3PPt9G<_lKrhDIUJ(vc&?I`{u*I+ z;pFa^A4cZVsTR9rcA5RmfKZ3NNg)UHqPFv^{ZX%!LMojGt-HlE6wPq! zSNi=G%FgsuL6jpW^m;izq*e9owY^FX?Rs$4Vu3rQ@6>T_Y{aekjRu!_Cxifh@auB3 z5k|}krU53)*MyLhP$Si2&Z^@jK06LNA8xt6f+I4WO>hG*>Bb~<$%~QmM*J`g7==G5 zBa-1xcm$A}*xXMMu6sdvCsuIm!$;ve%U#LBtDQ=h3f!i1QJ8#hK3VXz9dGJ2yTyrW^EOMmGBqo!59Z|D@=2SOGj`p9#l=2%`>$Im^vD#nEaXkI z47K+!jEWDBjJ!=QQyQplP9!S6I^t}#7bCwn`vRe3@IwOMfR?b9)62;;byC?2(ihC4 zdWl$;gMk%uvcC+n*m#VX(^c`G{4f)ZUOPbICUldvOnO z{a%Ip^+Kciy(6x?0ugO(*DE7T}erP*z?4Kz8nJ){V7(xpD{W^xiw2P6kKQsk6!p z20t6P-thQI&o0-?Bl;IQz9*IIP&<=NrQcJTF?V>7TH1y;CugQ^Y~=f*&SD^?f?ROv zIJ-|y&ZWa!VZrxA`{9bDvpYOAST?@_iHU`j52V_D5w&J*%Y?!Ry%M+VhJ`ycIg$2m(O902 z3fSu0;$GfABPr2VvUT zu@S)eT+eL0Hdy3XvBIHvv3+l2o=6P;4wiX`L5Pjv!o#xm;2kQ)u3FD8jEG}gx_qw> z>~Sv`VRI%%&W<|uAXhI0GWTN_95L<~Z}_psbVhs?aKu{5z~y5Yu$#F#h&V)?Oqo4a zYbSpF@-TexzWO;Q1w^27s+)Mu-yN5K`F6L`up1>-A0HY@=J=2)b-A6uPvStZohwch0xuZ@h8|+PzZyZ4Z4W%N)OD}L{*n2ZV?ut z|1WZOmLEE>Z0>}AquWGE!E5dKg6L)Q)ca{t&7%*pE!1<9{)s zk86$&vT5skG^-fhFQ@l^LdTK&UQTXwXS@ilZF@~465=QDvRSu|{-^7s>|#9yBI-PE z;;Bn0?ATxvYH&i7J74}!zqBO`EtNr>V+zNI`_7snk}V5vrV*so=Fo^Ht%JJ-|BKwN zvUQ6Y76#s2V(rT4JDCeO*7who#Hwn0#YBE*(GHf-XN-Lpgf8qk(nNrW5iNw~TRtuP zu}}l-5*354Y10?K|G6@-*rh)>wevXh&(!{R%sOc{gmaq)Rnod7U;f5>_`z;}{Ue!? z+XPsjrE$gH5)p3BXM}h0n}KmzRh^!MHLJ0J?D^*8&KrN0!)eC2*T-b=(DNTF+Y6?} z;nP=V9_{<{>q4LkTi=)O|6YUyzM77Q-x<)4RR8IPf8!Inn}$VAdwS2|s{Vv%QbNL% zR54fj(`}spW#X;MgX+;;&aH1vyo@^iZsr?2k!k#m?p**vEe4+>-*KdWUGUY3;EurF zN;~{#M6fi~FC%1_E#qQWo`5TKvcbpxym1!8oknN8<7b(t?w0*1xiXu#e>N`rQ+Klq zdjZdJu02;RM-Co;`}Lp6J9U?+wJGBRxfdR>!oL%!{>5wYBc(p3s9r6~~ z+h&(IV5IWP%J({=m#*xk-_Ncx7}0Z|Xj?xbi#d-`2l7C)|GqonuAEU*Wt~;TQ4#?& z;Y_UhH|#6c1-6?Ja(wMGALt)mvR0;4Ym_d2A&`t`61g2Dvk6k4j<;QI)XV%{jltT@r+kRHfA5cLJ z52x#T-atCc3eeCCK2}2Hf<><0Ai=g4XjuX^=8@+`yEPbU!q9ht87P`IXUQa5fM`1T z^87k;h>gob<;D48TVB}MhEj3xzdLD~bi~AvAIt#O z)g0tadifokDge$iu4s3-hdMHvpJndksY^@4=PlCjH;zPmnXCbIzVk(9_qs+R|4Zon zcS|aBML4TeaQ;B@X)FgVNoz>ss}Qv8B1NIw7_r2jTt3K)P9pwT;eDNagKb|T1e*D9 z`@N|8E++`(J5YLgaab4b07e7^TOHZHvkEIr9o~sp4muQY^ zCejHhyu{+!{Dh3UND)*}`aU_Ns%BL%-7dy{S4(s%i0=GFiStj;rx+>)5mh{8^QR!D zRt)<~L(wOFZVbTn_8j{ck`I(Z0F5}x1gANwHK;e?X0hx)^pIrc4XnySbNQH!&m9t$ z#mWBLcWe+_i}1&h&K6B|b{C2W3@3-Mbz5UOfDKYKmt=r&9!?cwgT0sW}#fwi0} zl(wa}3lND?H%msVfEYJ+d9ZV$E{I>1pH*8}2k=v4x`T5QZF^ur#}l2G%+1CH_GCF9 z+1_fKXXv(lSS1yC55JcuiVF96g`$V{VIw!JWW!QkzvQD9yln(iom4Xpyg$Y-DfV;B3w>6gFg^5rv(f2kWXW=f*GwhEl zMME9m%zQ|n|7M7J0ylu4s{pfV#3#hX^@&M~|B{cj>RA&Svvn`LfaxUUWr*PYeeRKW zIulF+-$Z)uzN|<$?JBF!d*1Vc3i^)0TtcEsT3Qx4KVO+VQ+Yep>gB|$;?rL&C>{j1 zb0h?SQfsoI@(0rrwb5*TCD8l8 zt56AOsU*3z6ZRVxzfxr$h;;uf5=f?e)3B_{-W5f+#8z0mM$w=oga}hU?Quw z3(f*#Y#bzP9d*6S(Vty`>a4r+gwbINWs#4;}e*+VJ0;&lF*D>ac zNYKC24GYMCG`pE>q-!O>u{{5-888=XTOY2j(b-$1W{v z0h(M~A+_Rqudmm$$xk)kW!qo<8<+>b=mb!7J=4wa<9N?&ZozgfSp)CrK0Y#%OvgSG zeceYxf6wt;^_=(%qjb^B8v2mxyy%``2ud*XJ*?bdgaIS0Zs2q`X;pvExqAKz{y!6> z3TkKnAa!cZEbL*qP9!Ih^tRcKm@Xl%TxmpS^!6kB|MR2@&9WWE)7#&h|0CsJ0hS!w?j)~@BnhTP$4|%o{LXF*j1oK za$^PC`wzY%tMJCyrrlieNV_^}4wjW?I0q&h6@HJG6gBy&QKEo@eXLi&3`jZk^tV{* z7->5fgmPy#u{HhPb4{y@EA!RS23G6j2<-^Zcy~|xuE@}?!6(01q8`np@O&eNCGI<hQGV)R zj$03}Q#g6l5CHd+Vl#|46iCXglD^Ab$L;>4}2((U@eGREl z^#w4b2G#7t^3>(1mxyd|65b??zkk4DT1Xnr?bZJ{iaIdKUC?c65wG)(8_Hp zz#1*RL1ssS9_A*d8B+jbgzKcx$u4U4w>y4$*erX29Jujt(V}FRCUob$;b>gGC(5F& zK({5mqh@FXdq}|gRxlktS}Jv<)7u7%{?HC<;R}k#2GJDjENaZaD}`5d=04em?1Yx@ zE|cLxFh@0&fBcKTtAP)!>47Zna$?V40Fj!)-$JGaf?-d*qQP{ai z{x#n5Up?QLULr^1eUhA)P69>hi70g4My7Evg=a?Q)?HJZO>lYmFx-t#Q9I(@i5SG3 z6w*#Ee{p9X!rG8l&dA9ytP;=(YeyP?a9!I84OzgBpwAfMB4QNJzw0&JU9y7CGO|~) zqmajTF@_QW8fk>p8?@~^8JM@mDxgP>+z#B^MaYO~FBr~dsnJqX)LP>@-g$A2LQWPq^UQh`6`Ibc@sr^N!>&;CD!M6`Q z?jHz?Q8=_*1w7R}2AT($Xymzg{L>a*|aBcd+eaI^#G)} zrT6m}o^-PamjQ@iSo82^zSQG)HI7c^ zFYu9cU%>vFL6xP1SGZCt@^KKk#C==>p_# zfP~{0B=hN@K1c7ew9J40ED0jh#?_#>KZxG{yg6Qb>?b=ZkgP{(%;ikVF1tR7)Iv*F z+U~{9WW{YUG;A!LyYQZBKMO^mN>_6z=$q@`u}?TtUcupR{*OxAiB71gg2+}T4kfkMK3 z>DMDWP=9?e_5qAqVDmK9PrQF6vM~*Cdn?qpsCGbJ1Bsx2{L!d| zCVq+j>`ZafG-DRl!1 z5^Y?jvR8lod}nLG?Rnb>F3z_5?dHqt2jdfK(txV`O5wJF@2viR=K)OxI==(M)NkY8q@(Z(eFi4 z=vn$@c{y5JZ3+=+n|*hRqwt;1yNTXSopleOE`nZ8s%$9x*qT&@fVGHFi;aLZ4&aE z0U7H-T?z)$cw464ZARLfz+E#t7QtY{1WKik;lj9E6`=Huk-*w{h}^>C-xfz*hNh+I zLfq%8FkYOiT?EkWY%5%cOc4I2SpiV=s#41JxW8D{z6V6J&b281FXR1s@>&8&UINL8 zsORBjwooO2Y z{3t9rlw^4R7mERMZYEH60w|ji!N%8OihH-nm?(jYketUqdPg_<2+c$@*+;6nDE^>} z7kZwNY1QxktzMGf{l?{tpEE_@DLr~Qkz|nAM5PiiP}=~%++`U{MBR?)%KT2;8zqmK z&XqEYCxM0Wxzi!e;tL>w)pF9F(?!x&W?i#>Qc}Mk;qul& z(+MCK;g{GY%RV8;>dm*&R=G3(7;hvn`blkqvd3K!X`}#&>=aU*3&M&ZF<#_RW>|Lt z+CyI$-;h0?Jy261&1JefncedB*Am;)l*~ku3w75P#!qlvu`pGKc6xLz^=6HPJwR9< z1pX)^d%LDB1+t1@pe4tDbwdx()*!qBZSTZC&vac`?VW@`?fY*ldI#h-UBDJ8TLUPM z-%A@nHOZl#)Dv^B{^stiscGoQKoK)Y82uhw8-DPs=ZtUsh6b6wyNB<zum`e+wU}xDjxT86!rB(d zLFJCo@cm!CXNoOpvK`0gG}eu~F|>!jdJD^XvwcgP{9X|K=>SC)pq2?B(nQOwbmlW) zD%lrNF};VqdDNYd83w@gMKm`ZyCZV}1*`P`-n=6=)~h6Er=qO@Gp$zpxFB1%)s_1syZ@ni2l2Tm;mocLz7% zxJWZ;kT{?*X?Rk}+MUK!@jaf&cA0l+=!Bv?Un>wN`_%6LuBh~$2TJuLSL_e*hz@|7 zz@@RuObH|L#NiZgnsUHB_G#Do|E67A$-_~jM3O6$_f5g6o7g`!fPm9{6P_I~rRHJh z++Q)>;%?)W!Fk6bADYlEO(6wI_~|!#AOhRn4X@3$2A|4}1hAQw6;^q&+CTug<7C6K4ddrm-#-!EOG!lsdA{ys=)b>_hXhhh zZSBTi+8MjI8&d-Ez7r>CEPmS0k9Z*X4qe?-;wLz}(IY_-SQO>3&M=e_vkk*~hK4JW z%ffxeH{~)=pHM~Dwvuj}M56ua`{eQW|MB-!5uq^F`ZEoXA@v?;x7rpvRFB3I{ybM& zgSMDY$$AVV{J432|JR*ueF36U9kMA0{9lxj3i_FrEBqV_SJg2I$`|NU^z0vu|4K>@ z+j(*ROrD_(>6On^l)?d?JYj0K%U!j0b|89E49-sS%ONKMS04kHVsGHN@h6*uF_0ztHoB8aVB52m)BYPVW5Obnos61r8 z6JuDXAFBcEl6MNA)@Vctr~PA7^uB#l;Q`eu`F0J}tiA{ATJCsc+7cpMaNsTv>Xc&6 zzD?LqwlKyS9lgqUj>ulHi(MP%dO1 zyIEDqg)P>RCnvAo2tdjxyjmCZ@tCVgzB7Z^x=7fnYF(hvAheyS0;H&cj<@%uffaSD zVU_a8kh4VXoQSvtPXQ9`iST_p@&w0VK)cZrdG4ghCEXB)>)b{b?aQuy4uzO z?pLd_AA$emVM(S*#7%wQzCQX3qWlId)VH4lU4c?N9&k&|85rrj=>x>RHRvPG|N>Hyaa9=FtX>ZYXT07Qv`%_tTIxE;bYTehRykBx& ztn6kLr1K(NP&Q>Q7EIK263y2jk)LDW@EVv73MIZZ2M0YU`t-~A&N3}H8%eE8lOhU3F;kk`0Pyzt4wXX0L87?q8_Z?4aFZSs_Mc==s@HqBIOsq7YEV zpeZMVS$=v^sLH|a^9V9-q!y!F;kmxt=tg(3_lC5Hc#QJ1MY~M|2ZZedq?c{MNAa1= z+Jzp-Y+C+Euth{CkFNaC&{>O!SZ(VVz-cXZDqIM$V48NZomilqf$K?4m3J|AqP_8 zN%Akmp2?_Abk*n2Ho$z3`jjvxCt=;pK*9S;*=5qPsf03iN^ELB&@J0kY0sqHVF6vz zX3?5__ql$2%hfIUG3SMBb<008FGg1e4yr+Fb)O) zi({1`Z<1Dee90gZZO08Y9rMjp6wZdA#ThRrg(Lz!wfBLB_sr*7ZKGsq!FmqmVz2bW z-ev_ke0Liv=;7pYbd>hCD&<`R`K6YCUvv+zx;_=*=hhI`m;`-tfjRSn!8NbdvC>+K zt1j*SpVLFmkdp-ylF_BYkE>-6>o@OhCUdpVMI|>6JMh=ME@zjSk$Eyv&w~RndRK?#2UKx>%?q?(;`Byv#&F;S*)G>#MZTq`oz$$*)yk0`T%FG3jN^cS>k%-{DX-5{}v05h(G{kWwm}d{qL}cegwtboAgdYD&k#ymWeh1tj>6 z$7phOpRasM+2FkumgCH34o@-yB{PNJURSILcV)X#o;@J5`Oaq^)5(#_(5Qy4y~cLg zVIe84=Ku{$OEGErgPAw-R!d`U#Ua`7V@kO|uRlD~b}Yfm^ra7E@0fPE;PK*S;SQH& zpQCZnSp2E!6t3VUE6n0FqiKH^r#?ah-veCTNPw?;hi-L zjJn~abu|hh)wGlFdTy=waBLc%Y#lIW#_FO|LP2lz0j6(F4*9}ZmiNeeVc7k zw`f#6;~1#hv6;jL!&)4u2ZsHF?8o-NdUMLI8dipUddK7%n`H)t17_YE^9kY=!oG4SJ9l6@ zrwd~MvqWXOMvV@rnTi~AiC==eN{?UKoEiac{fRmQiI@cwg4B|@_~eijZ)}w)a-amI z_CmvnYUqGEK4q4uo~FNtMMSMX61!@p75KVOR+`7hr44yqmPDUQ_-xdPjPUx#$ z`ELz9Ht66a$ddfzGVlRfjvlM>sGR* z!U~J!$A1cPc;5iOmrl-NJmUv;;58fv`hVz3`V+7h*p}NC$(1}Sf%mFLg5W^z7No!D z?sN+2pV2blW*F0w#-za@K>LXykC(=8mvd(s4+-vze|oU3(R+(8y&f! z9)1Ysz2W85u4oA}*?ir1Mp^_pP{YP&IJtQo5s2y-U?#0)HmArq1OX`oIhD!4p}-}r zYgKmiy*jx+o4jUp?_S&MiVKH0Eu9nMBiCK-!EGCo7iYV^PYLY-Sp0Qsn(eFb-2uT> zrMcH5hWHz17`}RkT*X+qdwfg%I+TN|1^vKXOpg_Z)pJJk_Bzf*XJ5rhl!3Rp3gk_q zLX%056|;UuQvfWJUlp7N+>e;fZ+i|eKclxdpLA*{50V;_W0xt24D2LC0zEv-Q5Aq~ z<{P(NQp)|3R@bVM{a=5fjYDxx)!pZ^67J|~ab2rzWy!X*H$rS8gdbblr?9w{TQFrC zKFKb)ERKrpn6~*!G`bz#E!PhDT)aq#O75>tDPH}} zKxxl94n$(h_}+GYe4yJa>iHdDavOS@B1;aw+O;%@2xgN;Z7HRbUjO9Dp7t-RA!8es`VgJOqKN?hDozIY6Jp zx|y~(dVXW8Df{b-z^KsuIP61iB|KQkT9_s$+R|LeG2r#bqdrzvIrkfEOOm;?zdavO zgbnakX{4=A<~+%8LyhdiNWzm~zKg3S>L~pdE!TKNzZ-rPf6Xe&N8_G3?o&~MEN=zB z#Qwn1kwtZ9q)0)wL*Xxl1!F^~Z}P|zJaBW<%#eEx$ONEJAd>tzS$Q5=E)evw*Wbft z$fD67gB?_n_pYq|YxbmicrG8SR^TXWp(H13W_s7YlMwc!6YFTYvdq<8yUxV}uhU z0St9~nN-Ki&(4jEVd`OlL7SGsgg*Y2OLK2WBrwykj5NB_P7g-HV8HZ7g=~DhR#M;j z=Y6Nj2&T6r?&}lHJeG>=EVm@9Kx6sJRnAeN{4E;|J=57%b0!y*rq{>GJKHT8;}gL7 zQ?U76n*q@Ju^J4X7$U)Bc}eMMo%-Ioaze-oiH%cbZu-HTRiDNSzHPz+;J0u_>n8mu z5oEo3z$t0DmHYYx-TV_(Ml0c+mbwkjB<2683y69_0~2Vt{DRL>=sw6e6(3OH(x&j> zb9qB2|m~NN77Qr>Nb5VOcPuZntKs{GE^x5Z(C@ScY z!}o75t|Tk01^7p7oG@zrP01Pi(gDIdg>N((!6@~Suk``e!*T{ z_b2uE{?3WD3fQs2(pz(ckq09mN3_++_&58FFGAfDRLTBz`2M4g-;MwUgfxEA@_;-{ z#(`j7ef2x+P#OUC>HAF-U)Tr&ZYR96n9L@A>CSHuD*#L?PY{H2P@Oqni-^AN=iQ=U zu84$AtkRTRvhJX9jptN={4Fcryg2zeo?AmlD<>*0@wu4Mw$u3o>Fpd79k82ThhmN3ub zAEcxkW?o{!q0PFKFeI3SE;i^#suwK{MUd|YH596YGEa#jJu+hlq;M5H&nqQYDn^Ei z>4Kt`bk!B&PDHGW>Cp>@pru-pl(hgJ9BR8QA5%iE%$7xR#m2$e^=Je!w$33LT(7+5{^Br$La_D>eRmGR&g(+4B#Qm(Lhjm5mvu1oX z8y3a=i(|Fwiu8>YXy4{+Q}4~>nS1QTS!q>)E?878i*B#TUl(x!GKY@C%W)bf^Rz$& z>9Dcs%Ix85VXr2=`a2Of71f>gU#6HZl!G4ViPL?Jp_uCfP=!O48_A?ZxUz~zS`s;) z<}Fa4EmRt0NifO9e?sIBy5Nr%H|FZ{BBG}{#aeg)6R_vI8**VCYFl5#QW#ji;t|~? zBW1=tv=W=l#i}->8hB>!ltytc?PatAr^QLmXrapJu zec!g%GGPRNR}E%RrF5Cy9C^eE;|62n&()q}=?F}*@~cmKD6o)e>*8ZEvfe{Q=sc|u z6n+5+^m)$==^KlbQ1iDrnsE)|^ybpY3f$IoWE2!|9UO_ylr!s|ed`XLe%f=DBXKM^ ztYWg_>pos2b*SZLdd{YK<%N~`5l-ON-bJp6m0ZmYL7!V~x&C|tCthAxq+8MTL#+x@ zCb;rIRxOW76CaP6FJIAT6OYwrV*>u2REWUxxLfScM%7PVa}{VIxz? zbZ6WWuqcn2xIU2y$(NsVt5%mf5_p4O2Q_Ns9<#1JZi|?P<+9}E7}5(q1jtVBd7`4= z#vP835?Zk-5Vzb4O68O)C?T6tdd*TD3^!RhfY}4@0$Zt7M^C}AL2cX!?C_iT7;Mmx zU-)WEK(@zieKy*7I98?9iceB2KB|wuEa={qx%^}+q^|Ua^4x54JkdM=$5&=HSDZ|; zx!h0bnny(`F<-~xZ#F$})Al~6`>9bLMuv3)cU+c&USVnfmgFNX-3fSGVaAbMMS$SC zfx>_Rni5@8qJJ$GB^1&;^Kr^SkQ7n%;ao@1=!n(V<((`^Z#*bOhaEGftjg?Cn`w;DbNX!S$?KYPAgH6HZO4= z;6sMEOKMMYcM;B#p4AF~C9L%l&d5lW-P8|0sS4Ic`h+?5FWeW3%mjPuW+W`>FxLljixsj$lkd9Ai^2p%pc1~rjsi-EV+CX3)a~y#M=CK zO|iu7e%UP6%Z>Szki~7woMuy0i6OCKLOjH(w=l zxszHfI?u-f1EhjOKDa=+$(azDC!1pDpMK66mIpj=Di4sY@B0`-$qlx0lBhCtNqaDX zr8Q*rgJ-s^2JlNUL7XWd+(DjJ!^Gwm8C}Ir$a77UU)Ib&>?HFXRlVnPQW-A!0nE2M zmr~_V1A#sReF{;6f1xK2%b&q!5xeHT1iNaCQF4V#pjhp6%MOa1e*h~+(>gO-R$s2M zYN!-~twD@C8#AW4BCsC0d(T$&9=G)aiao6+iOtzERf)a&>PrTYK`1Qy6!%}U3nfz_sDGb zG0*Js@$VRUMM}NgBRy~VQchC3q-IX1z_+N&ksLTbNqn`+66*UJTLn|*Ru4Bk0%8FXYcyJ8DA|P! zO@9o8RZYjZ{-P13Aq&;yp8dA0B*!l;EG>hCK0Zd1u0LSt{v=sgAL}vWY$9);A)M29 z7Ri{n8O~%n`RX7b=fe0-QidCeFI{`mppT^TwrS0`%K7bnV|!?M$QJ{3`SQ^ab-X%h zB>60{=?e2bIbP&^N%Ob?tcBN2eI~;EHIwOTk>qXpqW<^l%*iv+adFsN*%_OP#FxkI z;x7UHf>ng-ynlb^W`!#pJ0P8)9T?@WfMmVuuejo_zZtk}_#>^QnLH%4Gg1L0NDMC& zNiP*7D=Hf%n;0dL3k*cEsMbmMQP&Dmd|4C{NEVx3TmA7Wwvr?;Gl;is-1$YB9qat@ zV1Et}cIx`EKfX}T!2aPY|1E_d*lp|bPo*{hAc&84N&V@CR&C%)lq3gmndW!c-yI?} znld@L>rNs-*Dr5ET>_6rHX+8k>wHP2@bQPocx!+*_Kvnz*a9%SA1P=OtcmYo-62s5$x@Yq%zW_-pheppi z|Htix@807pf1XbX?N1D}bghAL0PWoKLcflzLTL}&1=l@1aR&dx@kAI621R(aJRW%0 z!iqu){UQeIXyCy0PD{0&GBi%1Eac4H3h9XjJ|S{2K?bu{0F`7R`?08uZry$k)} z^>;I1>#eA0il~2#+Wr@oU-^LF(u#ax_XiRv_)@Gs-vVH+3Do(Nb|Ea=GmF9n{@ISBcqhR+xRrtT73U?F= zBDURoRx+9|(tK_>yl+-mw#t3D&e_w%_Sb>;{lKjV@Bi=bRRB@x(>-u4A zIy+W3>6vm8UC))1=adgd(p+)BdB9fhXb;DIH*1n-PD zTzAIXxTXqz{!;MuYu1Sm1KNHJBw8P2WMXE<<0on=*5?}15&{-`3Q+S1V&gNKx74y)oLW zme!VczfMUhj^R_V~L4w_$!TY5K!4o>mbMjGcThmD53h zmiPRz+Zr|(l-_OqCQbP{o@bv8f273FjYD6(4kqz)k)!U-*0QxMk4d7W$NdJ@`lnXr z$92taJ(n=58ak4Df9mP!7smGnp?if97hO!VeQ{iT=XK+yKP`Wc)H<6WY0D#N4llRsjkfFNG;n5p8=F+3}iW{?y9?jAYi;vhWOJ&}?>-QyO zH%4>#T>i#ZIY`^^H_2=Tpd&nHQ6(`K*kyHfL3Y`FpPJ2u2k;NKAJ`nAq(7d90N~8+ z6A~GGt(N!dtYDUDkrRpOxm(a0&ZAjzbG>J2AZW9&=_2kUC6k zw{ayw(?;uh>mty7V^p(Q*5eWNq6W~nhp{tXqYP0{L303@p5Z@^afzEW8Sx*|5wGkT zBD68l97LZMzs`Vf@AoE@=3M7JA5k^!;^w!Ip7g@B?e8@QH5=>)`GY*7%D|Qkpx#Om ziWuU&cC#nds|d1lGC9Y87Z2%T<_cP$CEUr!@VA6ssr8L@EuZ)#5;z;e7`TQT=wIq- z3G`C#2MqlOxpP%F z5vLw1ORV(hMUio+rf1P?&V7(Bi{JwLCQ}F?zVWP(+O?^Imi%{due5AGTs+?AKi?EQ zKi;xAY+Rh(5&j`v{XVaf^+Uj*4$&0)SW)y)YI?Vll!CABfTQ=kWIfTAV$du%-`GPD zU00Z&$({9Qy8bqZ(z8pgXVc>H2LYEs9yW166j^66gibAC{}1-QGpfm@`}YVaA}R)u zBGp7drGxYmREjhaP{{P~hA1 z>Zz;>4@?L%iD*dSm9>uNU$RYuL{?0smF|C}c@N5qeq}-qAH04N1L7BQ`}O)R*Aqfl zQH3kLX80qvipp3PE8zL?PQTjsikH{OrX|uE$3sl>@E9%OC;Rdxhgy&ua~>nJ%dgB`ejCP zClL8N*Wr|psBwRM-dciVBalYvOy_@ww_ESt9PN|1-`xa=o%7F76GL`$b$jHhOeU|3 zv4k(w!EpV^*&QB)OMB+i(EmP$BslH$PAW9}<*x5E=-C*SyWiYPKXDtrz0`B-R{~)L z%peIEq0H`e?LOh95#;6bPZ|UH(a37LtzrBwA6eOex#m8GKBa))mvhdf49MFl`5A0~ z0PyS7XO$09UgYZyxvy#$w}@_W^OC5W^pV&WPf%x#@ z!I#83Dw(~Nzub?SaokfG)UBONf_(F21?}< zYU1iaqbz_2%*)rmF#~u3re1B|K_(}N*b`F#MFa>G^xHNLTNDiNvg!b!b>;Qeq-sTUK|Nzk zhUvpf*+{k*Tz20+Nv8kdOXy_*%ZNeD`~XAeCdw;Ioflu1Z_ili8gI@N46P-H-qslJ z8`@ck1zI?$9k9OFJx9Pb!@lR4FE}iG;%v~ioe(nvT32K0fz|%;dV-f(i1z(%^F5UZO{_#j z_ptT4Y;Aj9WckvoAa=EPdc7j6{Uh$}PJdY(>CDMUKOBT181C6`dzYGDITooaD&oSw z6zEuLfN^pyZoe(W+G3Ga^ewtS@@Q(x!#e<~cy&kb*W?N^TQ7f6ZRs{mf2-2`Ldemi zl+_B$#ddv~mPLlvrJL`LQiVelPz6)e z*opK%8esNb27E55RkH7uDrSBMr<-C z3oWA99J$}5qra{gvB8!@{=NhBD%-n}K%q?ji3BfNp|Agtbd~|#Q0u1Pw_3hhHl=0% zn;#BbWE9yJ&^vP;&QklQ|&~VErW4DTK$J)qZ{Z>#A zd}l4}mR80xKreqcsg_ONlMNJE>1Kd^DfqqT#L06eK|q|kYR!cIm~$$!vcJgAyBr`7 z5emL_TK?fjxV-#Z^HuRSZ4%^7b>b54MY}tjDLb1LyPMN9`F)lnLGIHAQ^Ko7E!sPI zS?(6s-9@sFs@dj~{+o=EFTTg^cXUP6Gjd$J_Rx_p@O!WnE#=z0H@vhH%Z+(lFF6fr z{9nE;7<8#V!`IK@9;xl1?odnPD5v>O??di%pWswR&Lq(OZfk2JLAt{Brmhu>oE#kQ z)cJ%jwa({rlMF!yE#FPQ%#6D!#RE}@JjX2i51WYSf26R- zc;d~Y-DkgI^#K|Ej%2G+Ow9uf1o^@;=SziKp^Z^I)1!et}>IGx?W=~`MV(fSZ zKTwI0=DBT>G)2J$mn3IC{cqsX@0hyy1u8YV@RH=0X5`Mf33b5eaU%ADgA(B3#M-YY zD|dC{q_)VlL&SVUmmb3@V*z~HXsj@FgzN`QSb2%;8R|KTsq5wMuK9z%|Ao*2Uzixn zNtrt$M-KQHYYW~+8`zob%aeG*e(S0;UQF?B3mR|r__vqZ$#eDLE7y-Oei->P)Mcwz zj6ViyHNf#)tD<`yXH!FI=3mg4W89*X*DN@7S|a1ZoG7+*>)Xv(={Ut|<=oh>tRS-dlCNUDDJURo*;ez+>C@&a znMQqPXU=O-j7Q^-{g9R^VPso1YuI6!ItyQT8x@ z>Bp&*!$I8Ex&7w6)*i0*TxyfT$Lbk;jqHjI_rln_Wymf-FXaUOsZT@BBjUO0Lj>Sc zFXdTVrJoiZQ1?I4<2`cz*}@UpGGN&o>)18%uvMuqhrar~w*$v_yi1&n16S@e}-bVgDgr4omTl#W=ixZdkiOPHO zQw*Qgq%_lu+~%e4?cNSOhUC)zycB&5zj4z!3TYrZFVRfx|E!!ut@z>hus+KYi{5&s z_|k8*KxcmiYLI&kYqEUB3c-!DRmYAXV>ZIm4n^)w2cPybT7Q2P+3YkhrBk5Ob0UIO zllPtV*N;r)quHGrW^Y|~fdw%SFj{%F;p$6lv#6h23{njQ!BOv2@7>4DcQ_I1IYwtJ zuo|(!ftihp#}F`?MiCbJ^jnt9s8HK?D9UzfJ8E-tG~m2Ba$Gc6I9k!$JqHR@PkisR zNM*DdGvl8)!JA%pkvx=V5T-JkJaGcSd6J^api(Kk^4*VPAVYr~ft8G93%{I!dDIwj zaWCP&iygd4z#o%hr9Tr2_M!0s7JPG&-bXQDF#)9c9(?rsgV+neFEq#_Tl`ZFi1FZ^ zNN+n+cHqX$l^O;vAL_!T&$7llSX}6_yS=L53Xb7xHtphvdO1zJnnhn!G~ad`#aO7+ zI`ml-*aq-)ULK?`33SX@%rF))tH%0zs^ytAj>;^#TY7poH2e1Y&Vbo+ zNFr|i)h|*bUvqD}cq)@>s}f)I9oBurFXX^6=^BUKG8*U6;2@8AFFLn}eysUw0vbud zmED&~E=FqE)^D{<2IhL`>4E-nosQPpoCxkWs`#6)uHfJ`5KvT&#dbjNkgADTJ|)vu z*Xmqt!S5U_GaB9&lm7_Aa?8Vem7cc_k@N%{;lRWRSRwxb-A1c-=T2uqIgjy|H}`Nk z`}EhbpSv}(7H9=?Ghs1N_#P~9oh>rf=%6XxUg^W11Fnt-K%e$}+MF%1>cyUxvqIDF zs_Ay-z3F-YwPF~#R{)+}Hi)Rrr2<-T$0ru}wR|ReK)cY#V5_i-l>zI|5tIJa>0qmL zI7Rwa83aIt7Au;A3lT+)k?eHcHnlX7)pGt)gKIT>qP+H#?N+$?jcg?u$o$C8-XK{y z`Sp#i3eoYL`0Z&8<5-)TXsn}Ty72TD-0b*_uGG`wLIDw)M+30Aw}!uawiTGvIvjwl ziR4Ow@mTJu=LUgMLRbi4*COJ$v)rW%p93yDljoHt9a&M_#5|^I#%NWiX#%; zSsTG+cmFQiRWX%Wa-m$eA&Q!_BQ`ye zO`j>O7W0q5JiJrD)Hr?`C@d`JBL+#4F*aFcK(M66Z*-)Pm2Z5>qIJYca$1TyxxSPF zFXNbX=G+cbzO9~9ILoZN-f)3gC4-{ds$yH?vx?2o5$paKj>0M7G0i6mY%2#L;Z|CN z%22m!&wE2^U?oDaJmXl#vGxCIz<@CT`)zE#g74?f*$f1bvV=s65 zRz(CCUY@Ta(&cJ%=jq+S`VqH=y@+m?e2X@FJp`;PWw0XE%-u>{BDw7eDfq^`I0;2S zZ_dHw4}^}BlHPoMO56d&aZwmv>9dNDC>sPmugHR=RUqGMC%5AsewH@do;HTZShlbBgY`+Ge(4~T0 z$LVWq;rG>6EPC|@l73aBH!y$t@Gil9VdnG(7$eFerTdnUiyZx&e6V&L+|xsIunYAn zS!He&VCR_-O<*;aW32<$$hVW64g*J;PS8M18tFD^z86Uz2-1u1*1yGM#a)5$^;k9< za%ENQY!# z$XBex+aK$^w;P;)%EzJu-0Gh`nlNO#AMDbPWgv)ga2Y0356nlCsLIyX)&bVOp7}s@ zh=LbzkWNVgtDkbCn%#rb3hO@QY~#%YAAi8Q&8g0wK7Cr-V|{E9YZmAC^9!$3^oVcv zCeHXtUzmev{$t9ACwN)!wBtEV?E{Bm{#jTwGPtS+RlS;d| z#n|&$VM`_gY#?P_jIU~jXP364{Deqz6W@^V%oy<2ZZt-)#w={ld3t)35L3M6ci22o zSo>-(R9)|e=dgkiMeV*ETZ*XkVTxO;D**%PS$f}s}Ah`+4OIYxT7>K>zV2ia?b1<$`slOG?AfK^IfpM zFkf{_vs#ANMshK{78CYat^>e_x%wV!VU26ap_e50G$9EoFkt8tnl^#`oAlv9iT$okx-x}(jJPkdyS29v53;oLgCf3o)V@Gro0t6G@*+Ao*cqg80 zrlR<++veo?KXWBp1%S|MRyX_2|1<(AWZ*5nrtgO_*Xm|0XxZ-9Q2Tb}D4F+o|0+8N zsYr&zm{7VG-o~9_IQlsqvOB9jyQ{`Zl=o`Uz!q}x!rNSbkQPAb&-=7w@H)*O66c|E zr8m<+M;9;XUFG0=H65xVUYc@!Ik?8Uvt`D1vp?v@BQfLS`7CYyw+sqB7Fj3BoemqL z9O$nXz7v8=*Y8=qFrsH_h(o~4*Z$i!{eDY>0a-SXV zflb$fI?`UIJ23%@R9UK0%9^M7R5c@l)NP$Jvsn2EVfRRJ1|SsvMvH^I#*(C=LDhpD zHKrD)$-BlD``G|ZM|bwBihd;r$Uxv13jFpJsmd$xrcG419xQ)Lr2wpd70)+^)3*Pc zJKKT>c{Qh2$*zP=lsd(Lkg*2iGaM2Xsq0e{hR!SyxzR|3DH7cwH3?aI_CBq&>cH+}v>NHaal;WVWSabu6m(&qoz zyWo})uI1?=en9tPW-vq&uVjo!^pm!M#aH$?Jcy(bX^U8+16<2eg3i#RTYbWW*lPxiBoiCTXyo=u9;}X8%y==sSLZj{~Q=1 zhx__`31=C+&ntNAwS#D9;!d7AMd(iL$73rAVo^O5V1~m!U&f@_TTOT+XLg5-z?|ha z=m_zTM5IoV9-lNe0C$}x=L=7PP62ln^m4JkzB?ED9fBa`7ff97G>@uHvV|QDU1QUl z0*t^<&sDSdd#tBf$Ch2@_-uYUhrg7R1XqdS>$GVDdb-()q) z-KF$*1Q13!oS#0X_NRTmh@$}E7u4a|#JLW*NBIym96Y!)-7(^oZ2nm10;*KLoHx0K zCD~*gy9n$K$z^Wrf1)Gyf+&Yn0(gi0xMU0%Oj378jemlcG&LUkAEG#M?O9pC9;tpg zY?JZhE`Khs00-7o!$Fv2<;gn-HZP_-k((-d$$J!@33`J2oAVf)3tyZj zBk)DVy9=>m%UI@K0QVUsIN}8cW_Gqz)^!zWVGG`VJ3vciVo7`N+YUPZh?cajheE(% z@yD+&?9ITjIX>=Z&Df8%(VY9S5d2o2>j`IYtW2v!XF`BvBBZ8~2U8t2`(r_O2oSPK zV#^L_1yWYm*f| z7_`@(z0MOo7M&BgSE8rItq_B-m2JR)9Q{$8<=}c zdpPzV4{>btKA49jvIm@T z|LYhjAX7(p07ZcG z877#Ifdy}_!oBhEtV|GzJAb}z)0;-Et^7NpA0c4D7yr_Ln?A8DT-3rF$;haA3iqe0oK# z*`V*LegTq8QeF3A>nU*Z&imF_nUT(b9J)Ac?opBQYf!FoS4+=uX|DULXn8K$eMWNi z&AhiOU!Z0T@VwuqH@+}Fw#DU~t$<>Mj7x1n^_!6TJ#0E#!stO{iuaef&(^5Ot%@pq}$~VvfOe9m&Z&2$T3|_LoFO zxWO`sVd-`%rBMjuDkcdjJewuiPif9`BQ1ce*#WmyV^(Y?r))9@CX6+XzaDsx>=k|%jv2K9MnHjDiC@7#i% zJQPq_Pb0t7*of}vIrZEk8kv#0-~rW_L`g#mq_!@hA0s6oLxF0{H!ilC9vfMeRs{cm z^A@lh8)C~qZm(dt^VF!f(7Lzv4kAPNo?f%EF1DdRBLC#*Tr>?#lpM2xj1_rz9R-8r zEqr!b@Ix*_;~>Fw9GT?7K99Go=L{j$A&o=S*{-Ow5NCo9{j<%N)q&~M zP`XcnDucx5^ZnMOQN}P{Ruucl`Yl7~cHNHWMJOEHIq{g!$LY4X2ks6Xv^jvE5lr#kbVmUt2`OrN z3c(jIy#$Rs-}uRiv#c;-)|pY{0)x!t8~GwZvilWhX>}B-8wI=^bGZgRlD`tdGJWf8 zVr!$Gt_yocTF?gjg!8~iao&=0c*J*H^N7!#$70M8;yOO9y(-RJo#EHrh8{P0cT?Tn z!UnA9rsCn$%xKQtx-EN@6|^BNQu38H%og4VT?iWOuN{Ll$#|?qxR+$8Kl!+@RRP^} z)J4q^Ha3bj!9OqF%vy#m;O1>vC+zf9^~`a(`k{Epfk|XnOJHcFbb6wKEH?LS~ss`$$sNX45X+!fxmj3)7S~ z%dUVY>${DQKQ>HPsX`Z4*O_3=z7#XNMX8DBC<-dm^&92*NkfgQFr4 zcR7`f0jdk8LmEurv!%yM0x=fPfV5?dQuW45Y;mBgAT54Q(x?i_`!|MDAnRY1lNMg7 zJ-3LwvffnM-!&AT)}PS4oIULbDGFj?52_olPTfrsoVo(k#&e|Dj!qf_$K>kbpxQG! z^G+rX)8z%n1tW-nP?E~RD#nSpC~w9UgMqbu*9YO1(<#+{ACOgkgd$9>ClGR1cicm_ zGie&RDKRTZ@OKrFNcW5A-3~I8L3vgY6EyVMwi25!uEmOunN5Ze4|>hxgLN#;c@7Vz zW{a)c{jIcM26#BiOdmM;@qi||D^JgE|BTLK+u@(U8QQ5#Unh0cZa|&-DS=S^K4cMO zy>^2Nmg!$-zs#^pzm|oB2k~=3cj;TK=q+i2r)>L5?~3W1xMZyLFlU-7U>qM2l~BL1 zjdauY$F;@w2ttsZ8uj-z8e_~zRThQ!ZjLFT_WND#18+lDV-Y?7Na^M&pYH9|$aMRV32>(XdZeZPN<~ps?#a0<0 zPs5`FRt>j!9YB|8gft@ue|=!>SL|M%!eKh&2!&@@-R1iFjPB`J0j^^ZSXA|U4xL!y zU;xpGW{4z9nG)4*BgxmV_X1A(bQ!eeg_mx&AdMwQ@#P0tDkv@8s=n54(W4tI9d7jj zmHwpi4J6RkTP?D}YnAfl?VZ{F_wROSFk5C2J}EaLbUUO=V#%(E@2rkeO1vkIpxrbq zV2mb-ut3uPTEybElt*@wilq+|zLXmS)Y#gM7fl5j@Tefwp3tbzj%BE|%jr=~XI}HJ zq`<1L#{o&j7@@P*VgNL91^(7D$L1vv*P1+aTD)uU0;UI_*X@bBGl(;^P`?B44NHlJLnfw^W}NOJ)*5$YGe z!b0*k!4@ivjklT2~fFlgiKH*z80 zF?{4T1A@N}wZ&_$X^+t+@n%x|Oy4EZLDlmN)q+9_IMuXX(hhHz2 zdL%Y>oEJ%JNtP`uzH{uw(CoH__>N*JG_l;Agm93?MnhcZsX=ZCpTEd8D4c(@_QqIj zQ?PWX|I3fIdof78`v@uhu6)Q$KFod1@=+SxAKhl}St{}E(UPQi(x>W?{ZQdlGp$n#So26A` zZbaATA}u}tP94~;+dS2;CS($XS4`jPn;kn)&Y?etjQu-2N7s*`Rb*i2?0YtV9j}u-ITgU@EOzuT!qcU+d{WGEjdNu{UuvuisjDzQ1m}Y zIm>Iz>o3(aD-zY86pslsqBD_iry_P`OPlc;dVuWeSvvUxpb=;he;K6aypy?s%=4f$ zTO@w$sh%26hxe_VkA;OulwzVzwv5n*VxK4Dw4+ZUNXdjD*iMd; zOB+d+79Y3B@^>GNl=C$ZaEuu-Npky~e2r>c{*3})bi<6f0gZtm_%d2yx15J|r=YOX z4u|F(&}ZVbQ**&g1zYW7{sBXY$Z^oSLIwbKX=e=@(qlLl=hhUb7f0-ahbo&ceKdjy zNrYVy5`Pr~i=)Z5t5>5~&iB0>F;wffwK7Ko(M1+Api#(azdfiAZl9)3=kwur7&TSM zctP0`z3tINM7!X)QKrxl$2Ep>E*hyiKx1pRTlx(eB&9K_#_;)i=njuvawPtEAk0fixVqeF&~&vsJ?>OrB% zvU`DDlsLO-{-fJanWOklpeSo%oA)lWf%D4Swq2%Os>XP_><8iz)5bYO463RWJI2D> zsTQ6Z!u3&}Q(ZvecsyUwg=k8;aR4x6foPGy=fYSUqB8AY)MFOHk>h9B>HKRKVII`M zbG5eAUu2qDHl~oCZ&)cjKHb;e>5spw7FXX`5x_5s-X)75j~69nzYs9ys_*RalEJg3 zLng^3&pxjfvjQtE=Y_I_&>cYk8;I!G!D14bENA->(VRTCk#rlVPN$;U#9s>H_$P7Cj)DDDP zQj0iS{)ZW)K)%+Zmg|ZthXCvqxb|g!akbij@r>zM4CWGi%hdo?ill-qIGTG{7aO9k zLu8@8W1$!z7t~h^AgdBvb|@xjIa#npXQZ6kE(f&F_4+`A-H5$T@s-uc?N~ZYCbF%; z2^)Y7*QhTcJz#&*zwB@K|N(b8ang_IU`Fg$+!IlQ34uMVXb_=RP? z{6P!SjeR8vc^m={sFVG;t$^}|mD4J+METeKQzZno?KkA2ul>7xwZS3AeJ#G?CIBzn zS0i3J*=V7%Aw@G*&{?175KbjZE4XbuJdmFYos8^nF@!iz$NSb9Y{~(6Lt-ZHFg!R%iQG68G-f093ygsR@e?kFWMM zmNIY6xIq|UMZUQvhH8U2N=~20cq8@KolCo~>W88~GPZODjIDdms=^kCW+lgxvgc9A zaq&@a`v4gaGfub~KP$Qk0x+g0aUD_a>;7x(o&lHpte)!z>S&)ZRS=vnRLKn<`q(UP zH*>SasOj6>*7T?#Wc|9)_FR|RQrLnq=Bnr1_*@Ma-NtyN|4QJRJIWoh%)tI6#{%^N z5~X%ov*vTqhwcX+@4{-G-NnlX1@w(j_aH%1mpzF&JgBxVJ$yiItgx_J!wYHg1rlTh zc@F^Oa-xP+w)LG$p$}o(<}JR<>!R#*+yw8S`ne&SEu1lPyOWL4*?K8RYE@N9VPeJC z{&{qCK`%sw(9gcU7~jk`U;TzGc^bLA=*8JTJni`4G*yKHoV;ePU7iJKM}^}HL_hAS zWMq6s=VBN#QUT>^!}ubN7~s@>0M1bmT}!|AVQgLMdkk!Wq^(8LO@GZ^SXgm^Xl^S> zea|w2yF0D5hn!m%+U;VPGRkv(nl#P0jxb9i5;p5IPIODY@BCJ0dqizhv0rqzg`$y@ zykI$2jOq9S?svJrjKZ!&{X@+PQSVb=Bj5hwA={=MtNCooC7quWu) zhdzGB^jiU9T=J92yXL+8n*(3VzH1`<98h1V2d$vhB%@Dd$q4^KpG|y_32)=qm~1xo zrZ8o%m@hu}Sed&}arwy9c<0Fa0G5`m_A2BqEeE=Hfq5~yL)~ci`-ci}-jBQ*PZH!4 zQ}ono0Kw=lF`S>e6@@Ha!3^>AaIjxoeu;bw9c)j&9aS9@ks8(&-ekr%J{?p_dMmqw zEyvp0=5Md@&Fc5<7g*?866NMu{JWimsrrsy{REKaF33RHn2QwxanxRBYuo9aOk`=c zL-Qk(ySD?LI2&{*NX8N?!eyakB8pr+HWMUSJ4>T4Fwdls6#>szQY6mox~sA@iMpkK z=*=61x<=*mbJ`d$CwFO73M>=3 zb_>M}wS8Ju4DqDI1R%}fQ-+Dj<)g1&0LwcA*EdHwW zLvv%~dO)4IC;+$FkMgWsX}x5t-TXA^65#X#rC|K*f%Az-xxeneN7q7@$6l(`)<-go7(g3H91t2 z{du;4E_VO5rul`o5DIvZ%c_K%C^`TVb?d?1=Ab9FQukxX$*H3t3Jw`N-@oRPi0D@jQ2Go3F*{N?p{gxNE!D;ZkUuS%nJr1IFww={M zaZWt>OpcPv{+Ng-*?;SNzG?tvF!pmR89h52j zfj5&J!}CWJ?cgY01L)Mm(%_(<@)0*Uux+q~25e}v!)tDie!~QKecSqLE?~jU!ApOA zNbtk)(l*y@|K-pAZ0~O0?O{NtK++|%GEV?!Bgu1`ZD)Dl_>qNqkOM!L6c#MKVST2% z<9-_@3h^YvY$EBzw>WR?iPN(LAW8kANz>sl59NlRn|z-S%;Pnb5$e@s-tZI$n19x% zkY8UC=*)OLqv{Vj-5+o*rCiy4c81Plf{{G1PCqA4lof0x;Lb@o6@-I-N^nMR+n}zv zS5)qr4r&(C2-Bglu|t*oz-<%H9z{{mJnsB?!BH)qLqNng`;5FR%HVfKZM(QngT69h z)_lK?)1F)k6=PgHJ3V}?ptx)#agHvCp9NiOUyCWgY`~#$G;Q@ms`dl(A!2=a8X|F$ zQzq$sn~f&(6|g}{8zb!zcKRd->_P*c?ONgxt9$U$%%I~dH=-ZfIf ztHm!>LGM1?;{xUjGS(*zOG`0)ASwkPWdKqCV=obbr~Jpf{4+MUkgSZYRoS_G2qSZu zggLw#!TkKQFH5&%2fg%#4Kro}I&1h6Kjy*iHjLKXhGi7^im?*{RDI84>+JEJ7qHkz z>H6A(%s=|Go;?YQ4)CAx95QVy1d)>z9x47Qpw3KXR~umBS5Y1+u8<&Sg2*khu{cP9 zxcP-(zW!}*q-S6q5xXRFQl6vPG<}w*?(5O9`?KuIiZnRx^zWpoM;u|Dl%k#0PYjxE zv`RWO*qYhDZM%!>#9e8`1%RQ>N@h16pz*yI+?m$%ZgWsl^VI&jeoKKaAUw~2IxWr~ z&qZ1<7jsR$#%gf1uc^%SJCm|5C)@yaEUF z*b3E#2sx(=7Da*3)b$dXiyTpSl2i(L1~X%rm!F0BwwAt^jx0}_xXP0uLNBcclPDigZK-OP=w}J-xteK*28Ro8tl2KvrDDju4 zueA;wl6uN#b}hp0-k`N(BXR)dYftOj(TIsx>ZHH7K>)7|21=F`4Lu}PV{S5OY=;AHB=N4E@?7=4j0y;>_ zjZ6PFfcs)R$KxOaf)Dt83HhUeCk#Sq@-Aaes~OVfx$*2)`@qZ4#W)cbVx{(^WS4-=znT_^$|pMc+_2&83y zkek>+ew7RicnccKe1V@sOQ%24Ks-fdMA88Z#fb%uZHaNRALmo5$LPGuj*Wb{yN#7Es z&EOJhA9pWt@YcPB@(a#6U%AHMiAsa!AfA~JTdAV>c3N!^I(ZxL-`BQ`v8{n%?!tx$ zKJjWJ3-^ni78ID?_d?1+AmTLf1rw8^k9C#`lVZk@eo))jPxm&*0%XaVz%sQD3RzCx zmC;jN0$B_4uuPg)#_B1b@RMOPjD2;DJz-b)!qSq#twIVXN7wo{eN>y`N7c&t64jMn zu16?j*7v9nxh{;QwmgmQa8hM~U(ydkO2u?!KNo|aTtctCg|iR&9*+uhvqAMNGw)qS zi|7QM1adpu2M_hi8`k-fiNO`fuE-6(MUjOZ==)jmhV$7rV6I|Q$8Fs#RrL!52AKpS zxP5F?eDWpVgmANqLV|O0W~tK>?eR9m5xq!t*L$? zV0=AWxYn-OQVo` zID(nMx)N6*d1IJ|4VI8dseV#S7R0de)=c<-RQ6!5Q?|-QQ@U2N$T57TQ;2#tYXxdO zoy@V;j|ei)6*x(wo!l;$zIU-;RDaz+qnfY?($0XThq)_~z4>j$TOwgTenjH2B-xgz zjWhZ)YOl{dH^)f|NVys`6dSPKj*P9gmqSe>HR@MMtpe9H4MfXStVxx;v04@N`dk~l~Y%~GW zS9m+9TN5hrB-OpqPUr8Q>bwS80y%v|>Gc&GNUh|SD4Gwl{$au&$s!BWp#ae;>e?bU zcID6!mkOku+zx%BIpy23_nT>7gt+4}C^6rWh7dw1t;xBRTCOLR_em%$gzvd;dGOw@ zGlO#d9lX1%LlR8T?hU<-TuVo`gFqh8wiA!UjGmo8)Vs3;qjc-ameG#dt1fq5MidOq z7v&9t*yChpc8&#X*5AaTX>wVAU=g}=&PEya28oH_nh9_m%1?j&2-7!Qn(IVuOc-^t z`wFn0!vV7Gh_lQ64dVA z0~xpU<^M#8{};Lh00xNiANvJh9s1eGbj6ubU8!Zi>s zH(@jo_)oLLj@Nn*5b6IhBj_dcocg(qf>4f*U=^1v$egn4T=$mma*XV$r@^~JD_nqPF<+<@hgW0QL;8<-vq^xbS$VkK1 z25)ImRDtIo)(BzU8%GK)yMw)?J$KoI`^#oEE&t!ew-RnwQe@&{La;gNC3`vQ*bNqB z2DYDZS>DldwL_|g0lj;f;MIkHB__5GyxGOLE(oHPGbHHIoUzCsoIPS%A z=f93{2U|jMDa`agu*x2dK$<476@E12Kb-cj4++VDVWcmAIiTp#7a^Ll&7(Rs&5r)z zV5(x+9Q%+vZ+@=j*MnjbZ=%(VOOQn#G6K-E|CI;;2>w?h{Ff2- zEQ0?s!vB|yz#)yNLs84wnrKkJU78@V`JU>R9uPdo3<^w*#XYeQCczkBewS}|zhBg-eFE;$@H*Dr$LgF&I+S~Mfav10~~I8MAr$KQ}jSg~l;_=tw1Tg_JMQ&}5p zSwMXVd1TC!V6Ib$Sh_4eH(POYz@NoIJG=QtdsDt~xN4zVM6xI>b<%WJ}T)pWHam4S6qcic9I!}KvN z!9&$birLJHP6B_`%5Ivh*qL8{gtaHj_kPIg+&10D;$OZAv8g-{>=9;<#}9+B6|4mZ z6|BD)gNQ`2Br7joSv^q35VjZ#I-dA{HI2nDqvucol*!PaJZgKoGv8V1O8_M&GeJn) ztU|CB3`w#pUNj(tydKNXqOg67lk|9l2USNIhp#CJBm)aa>xJD*CcGM`-gKaZtoH9z z69Y|WB{z9ZGF3r3M<)30fL)R?o+L4#(&y}>eZ%NESl;Oh7n47w!^bIO2FSBf`NxAj&Sv$LWDaLP zVba+KS&*;}Na&*f(690`PR;CPF_ohf!865K2E5h<+;?R#Fp=gWaB(DHu!7XDe;BO^ zK9wHXrTkPdU67NvgthpD#$^-;NRRsuBt8XsPruDY>=hncz#u6JejNNeCP(JQl@}LV zslchS{5BWt25w84paFw8HisXn7eSO0Kg)IEdeByvnB_r1ZLV+f0I!xq& zkVOVBU3$RIE-M>%1k=nrYdtV(YI&vDa!IQ9{sa)x?9b?k<% z{Qfam92s%2g7(+*2sT2BbOWJdH4ELP9ax@0bP~#0Y_rYmH`P3{BiLzi*Z`}|hpKnK ziReCi{X(zS6W|NKhO&+br2l{3fcU^W-OdIiVk;4jS?%>%l5BMpHz*MR*+!p{W7sU4 zBa8+pb1Jej+o==125vdK5 zvOyugfo-gm)b&SVo5w{d<=w98+n^euXF+B?xvE4z1e1%z zKvE>kA#LB^OQujwi@Anko(opKJ`7eKkV|Y17~M-i%zzG@f-sJAO>R*A2PohV4!o?q zV5=-8{t4tth^I8(->UTFqE$}uWXnC$BnuqbD}YJk1hoZaABmCUoeki1^jUL3l^K}t zK2G(tN5w#Q-2D4)=WfLM%qBeUQ>Za`L!ZO;ha2Gyj6>~3<^;Cl=PDBitIHA*2rBQ` z#FjO>K8U5lV30<9y@LXsO}>RJAXTZNV383G64c5w7hYprW{}G;F6lBw@xi1dwTNoJ z4Tl9=)uI{pQ|IkEZ>~75BA20=euYqU1JWBN$vhzXx^2T)0l zWq!9B9WNdWvAK_Ol3^sV^zUYbh8`n?PByj}d{BYJbYz*#y9}sBY=Zb-w*8cyb0%Hd z{e=;bfFXJzO95GjC%v+|h71zVIG=qP{SwIvdoN6EiD$Umj6ZooV*x?EkAe(Gp0N5q{j71^!m^y)YB&`8uME{aL>2 znn!O$5eAs?=@efj)1}R}5FaEJo(pfwd~Y?)@0kUMp`~^`T^(`Yy7#)yY#md_6NjK^ z)=P#1Q^hw-t`>!T!%;VV^7{VzT_e+|2xhh@mDg#@-J!3qm_|oOxU!`t;6T{uuhJvv zp~#abed$}q3GECijN)BAE24(x+;y^uxb5bO#^V;_T)o$axVbBm884VVPOR7}+`T@u zz0oNhuo_-JSk_1MEx*R&fzr_ATG8m2`puAyo8z8Q54FiCmAl;A|GafdtQLonQ}9Hm z@G%@_*7=;WK~IAZim{*j%6q45iAM;_w)@{Xy56>&8jpX?>Yain58aqpB*3M55nelY z`%PIiy|Xj8)O^xWos32~(!Kr2T@pYHgZJ;ebF8h526;xWj$8ImxpNb6&gn(Z#H@U? zNST`1VG}Bp;^IfNRSYRFCagbrYrK+WeYE8d_C0-Ip1ovWKodGDX}zw~uRZuy!Oa49 z7v%#@t)d#IJ{QTpp3vJy-F3f-=B#IZotpI@#1x_kPfq_+wGj*lkJqR2Hs$z}dLnXa ztL+Tsa8s+KFg|1YWyfnPeV1{$&i*0d5Mv}9KG|+9KC}?}cfvSB3aSpbGfVU$Ee8Ru z`Ysm+6Kfsx0vJmk6mugnND@w}Im{(E&^Kx~j)hXlt|dv)v^3H=X5e$*y9ZW-4iOJZ za$H{&)pOa)=>C*l2l|aS;*qp}z_Qx~kWVvn%B{SsKQWX7JL=4et-D5vja7khM$7C$ z6V+eTrylpo=cw+}E$=x&OPQ=nNZA=Wek#Q)=z zgWI^DqtwdB#e+_sQ=>%~d=776tO%RtHZC~Z?6CayNf#4&fXT>bmaO`~dsmw-)eyi0 z%X|$kaFY7{A=vRcWh*b$xYYNt9|IQy&-d#{?V1Ok)n)G6lD4&32;uCDkFqae>@8?#OQ!mfhx(r04lgX9IC#H8C9%*!Pr5gwH>2w zojy7jTAT`hbIns#KN=pgJ**h8#0sx)^z<$_v=|hj4;WxwA0Lf{oXP5aE_u4@M;DZX z?Zr|Ku}lg$C2nIy{!!(dlN|+|`7)xx{RR!U;TM;8E*8-{t8-<>c5ANJ*@VBJ-KmdZ z>`40pmur@dhL>wMzP?iMJW)yBzm%hh!+GWTMkCj2PVL;ohAfBc(CY!*z_Q7m40>L& zcvX(K`=ngrlbej83}vL!2@|xEo1DAR(ZUw6k$*!)TX8UA*8{%pHF`porRbX{ZKTAU zya)682Od>j1LqySo}F zrNw)`srS-kkYd7TaFnx*5*A*!R@%&}4~?cywNjV6nkE0lA~fjICQ}K6D+7O7bV%%M)W3QlyDi>cRIkhAeb)S;6YUV+`VFh~YoQc# zvI8j$Uv%7EldnXMHLHeRsMSle3c5R_8{45EG4uS|*h2rh(!#`Oozd5S`V1-B|6$Yk zGnMV#Cmc;yyYVG#G(?u=o%@E0C!Imi{jSlz`YZLE(%SR!1=AsCR&P(nP%K+KjppYU zjVkC+5F6krvR`N#MJ6-KZpg~>3d4)+K2Mv)^k>Bf6uT;oys<)dDCjgPcN)koF17RG zbt(`0b_K|`q(+{51?m1~w8`sOW^8e(pDq27ol_n62sQ0v{Wcdue${cO6S0kU2|fQ{ z)v(6-RKL4>GV&AKe3JRX_a45O!fNLs$Q`MsOq57x2|V64L{c~uAO zgRuYQO{+|!bd)$uQTQo5-`&M2_sPDi;N5(%(mw=q01(TPACH>N|3~_;=081tgze+FV_2=rmcM3=l zeaT<482BcW?E{Xs6Ts_K2QL+bA9Db?4T$sSy9Ks02`3shPMOCK4Dt3DxtIL?MyzQbEV4+ZI?N?)v!%4U*tBsiyb3SrUiSHhra zG|`l&ylNaGV$^N)ZZb~YI&1W`7qd&yRFJT0#q9G>i{5clROcKOB=h3$*@Z_xU+Wot zs%$PYb*YBMLLqx|_sc6xVy#%tw*1Bk+XqA2(vE%k(Yf%cKhzo_AsGAl9bCMV=1@eR zaYpH zzC@Smmqkxg zB^3pEHd)4S-VDZ>s-w=4M`BD^4EgsL;P0clGi*cI?#X12ajF>Is9z4zRhW&Zjy@eR zRR0(j3T_=gS7xTvC&HPw5_rop>@@T8Ti=b&n(YuGra2kQxart_p(175TS{3YS#q>5 zghhK8(mcwb>nWq;p?|;X&8iQED~N`;38z`v^wXz_im(e@S|7h3r!*oYyne@aB)LY( z-Sf7Rif8P#S8NS%MrS9h4>@+5kIFlaZ|!i&_HFl_8Jz1aTq&sIyweBgO@ljl*eVT$ zq!~;UOFM7NEbQFZdeCUvFPib2>j8W8A2PnXXUuAbZK_KpHLOC}fT_P&-x&HqiM+~D zw){&$BIC*OSYd{BPxr={l4bmishD%RGLttJ^_=gLbyV+ghv0^ordM=~ifdQ?ulBw> zs>!VD_tmkCW1AUA0YR}qGNOR=4vuvKgc53`iS!yNp@$4(VU(r`y+tLI&}%{u>IjB9 zNJ0w`loCw{(O`%qhTI2qzHjDz@3-zhcddK>7}jDf0wm8l`|PvN-oO3ZXRmw0kgV%# z#Sk=sayLM6|B~Byn;VC$p?8KL#ue_#%sod~l~0e0yXF-n5#mASJNlFxW=}()@7g18 zcpWuSSR2DTld+n`!mA!pzA%QSjQwIztfP~QtZK}jbW6QX&FJC#TLKHbv|g>eo?nC< z*(olDR;))%b1%>G{8wLUq`ocT9}Gg~<#FgArea4gKjqYaqnns$s4@8J_>YIGNWX+3 zQ+>Kx41s=E8%E@zT|(=>K6m_$N^tN&0;i>0YT0Vh8Odl@>tspiZ8d-nRbE(Ns2m;0 z3!~J+BrjUs0Ds;ZX_A|d%g>U_r#_cF;jNzenDgW|DowRtyi?&B=xq69DMq;V*oSsg zrk8K+2C4*RXnbY$hZ9C5S zY^8^f)^9XHtySB`V&YhNTlkSO)gR|ae|R>jjAZ3~pAoWvg-1t)T5%(4gQ2N$VK>~l zA^c+FXf*FPep53D4E)a+#Q*8_TI!{Dz0IM%_MOKG5H18+QQ(643L8tisaD)MVQo^U zW%l+lr#e~6Xh+0ebG^}_ThKY2R@K|$5F0+Qb3oYo&D}Z=0$< zVjgPGjP{+IeR(|t!k^|A^$O;#IC~6s~!h}b}wO~XWB#$<KmVyeFvP4 zwOy++bVZHSn$DK9O9 zQzezVdke*h&K^(B{Cl{iZV-k7i=3I$a|^#ZrJXstZmK<;?48 z1diBtoQY(Ww)vBv^_~dYA5hchm2~tgJ_$DXbD^dW7j5DuhoayJ(QNpm1 zpdpJ0XgQS~?1HaB_}dy-LPjfW-DV!NtF_p5s-q_RS;_m80$}b~oC|YfB!0Mo4t_D%x z3u-;v@TSpn>2WG+zUt+b7+RUypDlm>%b^PlO=$h4c;jW4XF~4%C|};Kc>{=e;MWGM zsshqIo;x24K}W-nemW2QA%C6?p%HLS<-12B%`Bj!70&5L7%FSQq0TVIuOGcbLqm(^ zmnmS3dok$vprysn!5aAUsJ?k7MyTBjhln4sTFcgN&f8IBF+ABb=)&zZ+z6&dfvGrT zDeP+x5|HZb1yXuDR3Ko1E&Z#GtU)vH4cYFvR67 z&NjmQwn&km@=I=@v|R+{UhcPRkkB~wK;DTY^wsK&SdwLMfhJbo#6|Ch&nR)Hw%bTz zlH6DPPg8mXL{qw8JEhVqqtg2b{c;&PdhcuCe46;$!!KwLeJYSqqRfTM?jrRm&B`6+ z^#h00L&wIMt%EeZ!YxdmoUisK%guiKr<=8Q7Ab&EUGv(u1FU9WGN*5P?dCVlW$N>H zk;9%}-@EjG{Dgeliu}X>aq9oM$dli{gC-LdngAC%fwu88EvEBB#N2M@_mn?UO!`}{ zyfI==g=;&X+`@bpy{z!`kSq-yZhu=Z$W2XF?*hcS8o-EwE+UDx&{2tnNf=J+7!uZH7q{(OSP z_Tc`nO{9Xd4Bi&wTW%6>HCaS33IE}lq~Fk)lB+N zqk7IJ0pT~)5fflpV>0P|VyH+MEn7d`zcsDz5Pu+52ZW}P_zzG5ou|_~J-@ZIWR=GH zKP_2<(07MP<{;4dPXVFu$KN_4xAch0O>P#6$Z!TC`B2^8?x;Ajo2u}9fwKP&KEAMc zl+T(GWBz$lX2(PlH+xS~#pR z@#}RyXa>K_=_YDMUsQItPQTM1^Zxglx!fOP5uwg6xIA+mA6mE?G0(KSbr7xJU;!Zy$Qe0A(T6Rn?iuN1h#5MTLBX}lpv%Axa<+1 ztM981_unhDX1>e9=&QOJFxOd5Ek25F=H*AQ(m!QevGNXh(YhJG`>Oml9sbnVA%u?{ zd0V6C#Ya$P0PIRC-Yv_vgR7Wk(VYSI@}gDY!Y7t)KYxAYoD)_$Rqz~_WasPN(8+MT z&AcrCmjRAH+&TwRyR$c%edZGfuX2<|QfEEIp8uswxQnXysSqC@^JdAj5Vs5UGqEB} z2=4$sEg{=#GXHm7W08_1y0C56w6lQ5hbA4SGb|>sU)j{8zV*Na^#t0OnAvf1e{bf2 zTOE`297`N&qY8KCP~QZ8X;HUj>&iFO?O!fnFh6N!vK?l!h4&iYS^Rz!*bLWD+pDqD z^0`yT)}HINhFTF@&JqyYehg6kqg{#|$cdbOdz6Ide}xg=p~{0t(p$fd%bNd=lGYPg zJCcpe3p$vz5Y%ccqzqEtTo`Z@Oj%yqydU;E9wJ)4Y8{codwnluDYs7}<~T-Ad7 z()9rVn%<}4;}@$y3f`xXTN;hUu2N}<;>xKrK?JjhZV%UFm9_NjdlnlPl5H!KPpeG1 z6+e5el*DgmM!8v{L|4@q(YgA)~TmX|YPy`SI2 z#JE~OSo<%@XU3W2*k)GAoBW(e)+-q)5;`%?m*K9Ldw7_^hN6%7B!ZF2{Z|he4SG+w zr4V`_K?>Z+`R$51C5GA=gQlzzo(Q~>sWM1o^`&6+joHyHt}y0{VTuE0@#&l<)2!UB zXv)xRIYSfousZDB`1~!rpFyO-DP+5g4nd8^{wc#ifsaK&TaHBz`W{Vg4tj>7>Z!5W z{A*H{9OdS@*?W5OKn|X^&81};Kh~W&H5%~Y+S=7aADjj?y4j^E`s=JuA6?a<1c{{` zi|^q`zT8w7Yo8t|mMm*Spr%K&+Ery%_1k~D`pqx-DwvY)7ZiKm4puT;dh3(|^P z7>i-nHDR(G)R1=^d00;NZ;Y?nlC4@#tQh5GyZsw7$%8wZIM~EMN2wbD(H@kB z2x(2o)QR?8-hVpQ^OZZq2Jw!lvd_pkr;OWB)K9kksMi*Wt`)EMNHF?huLpJ0 z>U+f(U{b|G!7y6#rZPa~kA>wIFF(w!p|s)+1Va!ZJz!aKC6hblKw4hTh%@wKeP9$~ z7PO)@31^&4jDZU;o(Y|>QecpH4zWCTd)^KAhqr`9Ol?@#?wPI$`^fJ2Mk~QiEWW{g zkI$1WIS>68HQIXN2_!|N#lP13@_EbEw(78J)H*oIP%qjFwlMCQc{jQBtl%v^I#sn# zY`XIC#S{aRe0m^6^KFzYty{MK@dTrcjy}>9lF5A_mzg{nkvaCl`^=D8terQiqcgN3 zV4VcQm80FviJgnAoUFK0PRWP4v%Du`thnU zm(;LqiTR0q=}hb4F~e?|NlPGPz%ac;sc5nf+oA^|&a>grs)B@Cr&L=Y#RhMM@R z`?2wp0ED?42k;C)B(tM=wz0J3L`)S%Db3tpeIbBy0NeV974BI4HQpeeCHb+%3?Ldf znUr*tSq0m%wz?n5!iEPd0&{^t`3_QrHsjIo zgH?XLs>Lt$ATk{%TF|f>WhkAWKi0}s+z-8+vDl0YZ%>)*9BepzrC5eNBFv=mSnS%D z#8{2YXxExVehu8dMhsTu8;UPA+(@9`tPk>7y**VEAt^z?8>)jzX2L#0k5A7utwd3w z=nDsyhk(Vo--z^viOM8b)?Mo@0f&l=^-fb4?!a#-@;Qqoor*q(pK8&zV9aY}=5- z5w;Tsr&Y{Z*CRBZs8uc-deGJO7Uat1OocfXb$Ia&+m91E*BLYQ);ds=nappvkWMUB zVW<}73I@@FU8dFs@P%y-U^?iTsf%2t5Ft;jM@-c7JtmXBK@3RuF8?}ThnkaVk3DxN z{lZ$+LWuTwnspjmuG*vG6mpL(ntgr&&s($AeN+R}7#;hdEQ=&ngz?=c++E(4&#mMX zMa(#@XHOMoQKQ)BC-5&DQCF2_e9RYeujzu995e231{1S z1JSJxm{NS_s4%9Zf?3k-P~$fSnf#G!dXlziR1z3is1|{&BeyzB-K&sjx!WHg6q$-8 ze0i+?;AJt|&;IkGq=?)h1uLbBNrhQ{uS+?|_lIGew9^%F?PJZ8aVpx+M1_zV$$(y+ z^$a4}V72P6Am_Q#tt%GS4NrBH?>^g;{Kf^?mqJRC8748UY>eEjG8(5r*zG#voLz2d zG+OFVAy30uFC2$kBaPL|&og*Jg>c%DNNc~sI+{xG1M{#4^`L`;5dYVuo!^};*A$nn z7h;z5%C95VxQj&AD7c9IbOMVFS%`tdvHEtAX=(?vs^mvpZGTM4{jF^!yH2aZ7;9iB z>GFPZpH56`INm1R>U-n+B9-v?(|Z2CoQgv;pvVC8p$`AygXkaHGoiV^zhOC7g$d{J z!gpn!xv;r1${h$+nig?HBn567!1I2$EsJjtzxe#~ei{GdpJe%>XmSicmRLQNSJ%&r zK{ahLk9x<3yJ2r@0nN}$6$})T(j&d*{1Lel2WtGHjE94isHR|DvnLwrZEnMAKWgXQ z^v2(LMVf?ijaWOZ-iyAmR(_NlWFoatKg3qN5a$ zN7WyA-mpH%$=YdR&N?9~L*M61o^h1LvS}@DQoAcuYh-PFZqPUl4^`@M1<9?mlHSUS z2>JWwO2|iV9?#o&C=F^5)du#u@b|;7qShB41=Ki?e<)IML(*<>q+Cm%S(%glXimD%AcOb#1sn3cb@EB zX=rCrSH{x63N8I2T5-e7I)f4J+e>dfX6X0*+(ZX4Kz<}W*W6HxaJPN>PT(JM0&FVj zc%G}ra3faQrA5bsz>J?9V+ZWH{`6P6m)IS`DGIgVF)2`NbZ@29GILX&ZuhrjJuzU) ze7qm@cVXU&pMPv;I75$Dx%@iWvF;u>7Syq>!~FenG+FI6q2bSdwwrP0^zMH3yLFMR zD|hh56lbYx`%D3|C?A}KKR=DLWu0(`Cfm|=@@%88q4)&h3MOG;;Mud#(oycrnd}>} z;iF@>`*Qz~wg-|9wb~%%*f=Fy98SID`bZi?^4l`vu}ZIf+lXsB+O=j~@)qGayI#)s zC|sW*G2f+q<-tWqIo4P>^}x~ex#clE=b->}i_=l+O5D$qrwJmrzPXb-GNBrGhiNTW zP3X4FX-f=hhfwS7D?B9GW#n8F%Cvp}-z^-~Ibie*bdIT)D$MPioc05$wT9NDYI@|X z@`e5dxc5)QF!I_Cmm@_@Ll>}ZpoVm~>eP@%w~ET@82f@4UEOWOVZT$zf-AR5JMcs7 z=ZHgnfK>(=?JOFt>3u)58m89kF~#fg@#+8t;sr(ndTM>5Mz46X+VWb%=LAsY-LBDq zdMSg=9E#VWGRxmbOXEZ!7~gZ&+M@=Aix~jmEmp+diw)xW!-_ z$>NA@;UkNY<;FZsK7F~TQ6!0|dELw`s+bF$G+rGu(B7YFxsT;G9!Va|po2#8pP}+x zLI5k<1X5wOi+o`AHAMDCv@w%F3zXMVzJ$J&aaRohreuK?ns?dPb_`txuL797&X)g zs#uNpm<%C&rrlrtE_>mCU)7S;$&DO9ZTQxr`0K^S2VLB5nkW-YOvNR5m2DPI@rGW{ z4L-0?UcEvwjdfrlOA#t7X0zhOH{X>|1~3{JNU_TDDCxWf&MIU>IE(jae93E;go6== zFLz*+_>`C0eUUzal?1Z2NKLt?rQ_b5T&wB8<5J|owoooNB(KI#%-Pq5c0P18 z-_Ri1pC_W6#3q8!_uGx%KCWDlPUMq~y&vhhh-e7etZ1+;mD>vhFRtKaRkdlsLYPDzeD%$52g@;j7|#QTL> zg?hR%YcfRSOsb_&6KjjA0MBq(&Owt^5DKwRIc9Hg0Ud{X^j?t+ftVX{{|G zAI`}l`sWx>*ZXnD2#BC$rJ;d)L0E~jo%@pox{nA9LZ?$p+tBH8>IBPv`3!2Q7CTQ> zVCLUaufG*x?vS;hd%9@^V+l@!u%ZG$3#&uoR$a-cXW>g)%jClL7t^s{RWW@FUyctn z1Ub>e7iR1whk{DuO&2LZ9vKZS#I>ugh1NeX4LhT_9V})9iE#yG1&ZJC+{U(^(^#ec+C^-k)r2Xn2^aUMa*}Ps!_iigGSl9Lsovi`f zLv-OMpwF1P0_ng73M3asHX!gB#gs|j6{%Di$;K7 z{EWvaXR?wJr4`>moDLjo)L^d$*h!`*hAS|oU{#BK=GrhnS8m=M>SRjLXk8=9RlCHc z{Z79=(T}a#x1r6$$qZ59<;T->FCd6q{`$-HCYJ$byYv|uT2fZLHecc}A}Mznos8s2!pNz|fCu+){<^BVE-uFUrk@kP5lFSElUPZ%yD z?h*_OkeFcgj&Sz`zVYCWw=)^rYoIJ}vH$lT^!W#rSd9KE^OQ?LMJZmx1h%-gxU&hPY^m_dY}y$Ic8Y9<7Hll^;h$z8?G9ZTHG-~UEQOGYiA zB|>}rEPM}b8&H?qaH6y#xUnTh zTNf)xfGRG^-mdzriphW~dT+BS*npp|O8Kh%!O^(V7wePP`hwc1HhvM+yMlty+7O{J zyjr}<37TbP+;IZJD?c~fI>M}|iJW_Jjja;#aG>~GNA6tU59gtdpZr75@n+}y=MPZY zi#Khl4%0a_W^F8xA@VrUmtE&>8SDbo4Mzrta!i9-*PP=34+I7FOE(?Tz(S; z@@_VbKSEU|b;Zd06zy;^IcX=QjMgFSp5^PP$743vqGDPp&0<}bFm{t@+Jth2VfBVA zEllmJ!WTM|_vx6HvYe#BAZF^D+2W1`7mHBpt;h^nw`dI-IuEg$QvT_7W%G~Go-heQ zR}bFU??3#!c~Ry9o$1x&P@FV&|FqNcBDwEIJ4x&u#TXf=d)e3uptQN`EuS&=f?eTj zi{u82O@8Dy?ewlSnE`=KTW*D(P{2>ov!gNW`lXIAi(I<;FhNflNF9 zz>!RClQjd5G0n;RT1Q%gbw>@E3p2@xzyb6UC+sCt2PP}zEp;b`HHYro zl+9ZLz-ha6{m4z^na&&Mr~Mp;Hs)Ev2|#yz_Me0zqowt7`3+s15$rKAopd4s)0~uj zl=)c`zXq)R$|H0ci?G2Ty$d)>g93mq}QrpPPof%>`-`wBdXZ~hLW~heR1!wfQ?1V+fjJM z2g{hvy)j389F|cZ(I$D%>EKA2&X~nwz%usW!N`463TtsNso*OaZUC1ky$si_?`K=w zxqv*n?g~3_4*_N6Infn?K=}8*0BmAv*$oI{ty%kWpuXUB^%QgYkKgvH4)GEdaFAte z{(ZyK6N)l+EXK(+GvoZFbVDNW@;)6@YMIA)2EkRR!nHzTyv2l zY%bqc#o~79s*>0#*eyL-C>Y-dcfjgj0iiSZoNl-pu2t=h%4@9Ttkn07-M99b_^~H8 zZ|G7J|NZZxLhCC7*Zca2z3QEe=6av?&n?dNzaK9tiqQJ}!u#MyTkU2a9&!HuygLOA z!a)a3{asK>tKUIf(yNk8u<`M%k5mHXLF+NNRBmTRJ^=r|RW%s|pzbEzd4!*78k|eK3UWwc&+$Ug4@p}r;yHET)SelD@;;T5kfU)@g>!eBM!8gqZ9+oWKXSn8UPy~ z9wkHi{QChL+*ETDQ|)@8Tu1e_yNd zeeE7?VTXk`)C1iZ{X8EqUD1wc!5XPYK8Uc*DV671dNfH|eRZLFXEVyz1G;Tpo8tRN z7%s8DmGBUC01z#Th5qYns%ry?3_04L)`x5N1U)GKeEcj!Ws6^tX|mpOp_8q zHk|~lYxE=xdt4Rbf8u&AC&syEzr3mcV^8NbQDqgk*jv4TmfNw&m4j}1*HXiH^;JZVtQ$GentS>rZsju ztVPwRHSwJTR|Q5Z`3L$w#e0mu50=@m72i7AB*537G?pA`(Aao64)h?ElyzW)jVl{T zlT@gRd5JQ~;`xvQp}a;m^fp-$ zXFTiL$!|6s_{gU zUZBdBo{7bn&Xml}0ahmdB9LW23dU^?<4dv48^>A85g;(;-qlzxUGIt%50A=&1^FRP zG#%V8bAc;H%**DR|2lfLOO(=dh-9)klrPkR%7iXXecEn}yn>)06iqViIEl^u6m|ab zx7iA_x_JtoUy96Y%YY{*eKT43MNpgXHdDGG`M^#DDpturZJ~jp*lWzN*@4%M??i=I z^FCZWezpo2z|~Xy&mvBU-rmm0b{LHK>*yUr6WU7A2s8f^MV;=}Yp$W^dDAmL_-{#xt8nALT!ZZ7`oF#e%${v(aQs`fw9`2S8Cm@B&s z|MjmerI3q1+5WS6znJeo$@G6c#QvZEDZzcKL|)FxIr#OP-mrfGUyw@{7pu=-`}KbT D)j8rz literal 0 HcmV?d00001 diff --git a/docs/img/structured-streaming-stream-as-a-table.png b/docs/img/structured-streaming-stream-as-a-table.png new file mode 100644 index 0000000000000000000000000000000000000000..81812161178fa9ce89e94d9dbf326de8da385be8 GIT binary patch literal 82251 zcmeFZXIN8d7d31dyMP^~>Zl+BA}Sq%Wh`{*gc6#72q7Z9B&aATRYek{1(hPbcSvv$ zA_N2}2{kAI0tqdF5E4ju4>->AJg;+ozu)V-=7-0SoRjQx@4eSvYwtUGep~-0-)^DZ zn>KCY)4KJC(WXt?8a8d(dUWS@;F)RuKN5leHhUS}yt=92i^vr4$FBRgz+Rg+2_9Jg zx7h{V1zd7@lhz+sO#C-b5AUjzxoN{%1Nn8-D+O(_*Xv`2>pj}pf6HR*BW~yKXG&U5 zs=qt%V)*HA2@yZzL=has=^UZ*^Hl2Wwmv)F;-m#;33k&=d(>R6Z_m(!Cy zl{)pN`(n4(mn=wOVor8J0jIZCO~s44lwx)0)NlJPZ`%Ct;omj*HxB+ygMXpmUnKk& z3I9dHf06KCB>Wc%{~tlZG^;IsrT4`g+21^#mgl^$cDtmR9-2G-z@7X+Z&VEZDS=!N zyYSDLm3HHs69IFVEvA5T^|1Y>D{M5ux)(BeIJR%Rui0kk)GS>qN@bR#Ebueo6CxVI zvfKVV)qTVQcF?0BVew)4u*p~bgIjhUf4B)aq;)(^l9{Wh3;82v8!hC;c)fhAzv}S* zgqnx5=EIfJTEP{N=VFxKH_inP#s@5vG6E8uYqntRx@Yn}D>YZ>-whP6vLz z>HMLSos@9Ns4JdX{YG7{#5yJd zwJrF{fyEoS;!ySxq(ND5;<&jLB63`_Xmu;TTi>Pc3Ca?IE0av^49kMC7>8F> zZSl~u8NRSgy}fKeG0J=}Ff%~zyx`w!wC<^&F38_qw3&34Vl9w4x0z`Ea#FlAc>a+2 zJ3UtKYqP;Xv+iQW0O`8Cz)bVqyOeEVR?xGcWqh zjl*Ec@^sx)IvBY!n(1?QLmHjkRC^hrG9waVyvOnA%GEc^n4GG31u+Nq89e7&LXs8e zaNke>!_;Ib;qxaA>7JF(kw z#XK*#ew-$2e!3}>*SaDj4PuA%KjjTkDW~Er3HJanZMQPx&$BzHy+$<%?G z)-((Cy4>bz^*$$5bXB{B$tKO&m&aO}xvR<21VP*jv&B>?_6k_PyNVX@hHn%*L_?93 zED7oZZJ1WZIXtIH${-xvSSyoTnCi0ukB96D9hiF5ZNAG*^ zE|Mwvg$NQ&9UfM7iNJ&iw-3yVyuOsm5pxMh8K-~=E(gk8OIJoMh(U1YfU@z-sr$6$ zLZnXtqZiptU-U8;CO5v~-Dcq$Rh!5hc0u(Qu}&=X8d^JP1y(sti$@bC(XxwdiU$(y z28Pc%1z6l8FGNfv{Ndi{K%`5{&;%xMT2RE7S1eEDU=5Co6^2iUA_r>18YjLh|5rks z+>`9#GTF#Db`lLf)DcVJX}4Fe4G%DNUMgN%Yb`;rt%|0n^vwcto!ZygDo&AZHE)JB=7RvnRBTX)&Hj&_M6rD(sS}J#vw;xjt#`Qt ztB540_7!0y*UcoREHHKKeY6uaMibB2kJONA7PKTxEGWr{J5`@8pPF>L-4=2WEW9`3 zyB%KJaxfcN{_(BtJB7+FFPRk$Z(3(p&{sJkx>5$EIi*04SWoctrysexKcubA?EoW+ z$Ax9F0}nV}moifPeXvIcn9V^ICpm7O*r0wjOMX42=@|O7VAbH}*{=%ZkwP z!G7ul;d?lc5{tTczsfp0@YE4xNP9_Y7onqA5G?juqIfeY^Q{9GJw9his0wjq^N>U< zvo4p&a4TM(6Zb9mDY6&Vx7}YDZSJ1<{8Of9Qo3|rH!;z>jWb#LMO#yP@ji#YSrNK;HYVp1HE>mMpN*sjSn;#=0<<#E z_M2Qqo^3Pj70>Ppc5H1Vksqqc5dt}QO`$%i^|ob~yKc1>-aN3_87OXFeYRy8hb%0f z7M?QY-o3VN*In~(Q;Nw5evoIBJ+t~Uh4=BH7uThEM}Eg?~Bl{0IagJ=kxjIVHQ1=LIutc-DA|1*n5j zw5Yv~$S6v0pECzz0IUFJdb)7GmEMs5&W$G-v$As5%j)RtXw&8^&Im{p6U+ir7w> z(z>KEywtoM-g_PUZR8FZ`zb2vhG&m$iWu@-q-f$|Z~gn8JLLDoIP-N9^eA9|XGfNV zmZs7GPlWC)>{UOWQ)nCDt-gdaQ`8G|uonK#QPF7c344oy+(66nK9yh`-beQkDDhgK z2+f=U5l6sJ@7vTS#b6iofy8dHUcB;?=w$w0WPi5we2e<7p+6t@ibt*9Z%Y#>UMQQ& zk^Ed~l{(?4Tw|SWlNvCHggqBp(3_bJ(r3yZF3NI zV4{5wQ1ffpiFw@}Oblo1t_|AQFmR}I+r1X4(mcf*IYe{NmS6D7$bC2AUfmhn^lDzQL{-uZ=lBP);xpkqxrAH(2a}f&1(@J5XOZd*bhV=-2|! zynB(W%ol*R$)W>pvFDJy=^(nK4SSprl=w`0u zl(L;A{|0sOHn9tdaw8w5Pvu^E9UMifovo)nzk8Y+U8z1L@CKWzr08F$dD*<{;26|H zBK(Tv@;iU58~bvkR~a+F=|FLLRojhg?Nl#9T2P(VH>gdcgxA*{e`k0EKy!`lQ**UE z1mBC6ZRW|)-iJg(goksm-Hd2t*2_%!%2Xt0FhTrI29MriTd}$mFK9uSs6yB$J9lZ$ zGF9+_kMF)tFE`0>RW@ta5niwRm;JUaDuuu1ryV|7&ru0{T!WuVIBMOi`DZq|9P?7M zRcKWtMbp6b>tma{D&HzVg}T32AW^FKtH6jaLf)|gdsW5U|9Odrbl=%NPTlvOTbTG^ z?eoA|A0tR6cTNwP74Iz1a&h;JtqgX-%N4Jh;Khqqy^ta90hU?@=~FFbIktT_qwSBx zyl=i0vXPEcd#pvXL?|P+RbX`@E&;LF?p)_Mz0@8t%K-40l;sjsf% z+nDv#=#<^OHmJuEJ}JXJ`9j+i!26pubZSy4$I3mRt78C)gvE;=Wk7_#1v0oL5eQC_ zMUwPDZD%@mCacg~t**&#;D^Q5&NH`dmVa2R{tp)O&G&QsTzxgTTqFDr8Cpwxj`w2) zkCN4M_?28$-6AN`D+|xTF5NJ&mriw+2>E@@UOKrt4PbQmQ2!%dyfA zk2;S5U0bair^(Ppx+Wi}4nOWDy7fRGN-r5A2&5s-$Y6;gU+ycO-s_Vxv;|}0tJD&+ zE~bZc`Hwg)Z@xnI{=Ock!MCRodHHUWWW&FO?L|Smt-%whw2PbTKL1{g8V@P6X6Ynr z?qBLfzQcfGyl#im2U2T4zAm@nb;nFf&GUg{4GD26s*5v&dEu$&f~v>k)wKwxac?6t zx30%%L(evn+r@heD@`ZcR{x*{iR0Jo!67e5y^E1Y5-_7<;xe}2{bOq|Cqynp`!##p>2#?KgefXm`DIT9D@Uxj7 zw9=9t1cJsaneXgiO<3r}x1;Ne{1?VcADPE;{F@)XBn%cC%Z3ax-i?b}6679)zDf-w z!N3L&X@V^|>5|DHJ8;rR%?fMglXwM`7S?~JI1hVO{fwT&tTzbSH&?P6j4#R78QAv% z#kUWCS>m#1W7Xj==`On zQx3V2)j8bVZawwqwN(kKbCp8`7bh%fgPpN7cE|u3p=`JNM@{&#wnY~)~x4dtsV2gs&$0ep@Xz2@!cVZB1wKj&VjOIrMP1em?wA-080v4;- z%!?%4B*Xg|Ey%;T6q&V{@fkpYBI{kfX1(3p**s;QV7@w@@m^Nk5$x6@XTN3##_kJg zdoV1)K0~A1-b>^iaO)16+uxg6Bm#nh^#zLZG+$J`2V|TQAozJN$Muj4-bjVAaljAy z&lgHIr`ql(t)s^90gIyh5PL9sf7u(>cDzAlUi*EnI?ztSR^*`-1`>ku$7|4$z8`eg z-J!*)dq=0eip|0{RWvdLWU!wYQpnhf;F598l2tg~67MCeq{Zo9+l=Rc*(dNFWx)pc z%8EQqYl4eZQwCeVL|rDuRrC^}35mYTo<+i3*n%={Vo(Iy89Zf#?;hGjOpee(D1i~t zPH<$^)A1c(B%1faF>74X3h~M*->GwZm8i$~LPRq_W`ruWgaGkGPIUs`Ko`|@b+iI< z>u}ttS|dXd?=!Vit*EOb*~hVa1HylVe?!DJ{mT7|9~C<|rh7rpEwTE@*7 z6m1xCpL-gwhZ$|GiFf^YucJ7)dfZ#Rm#NfziTE9vlsQr!6W}gHBO{F)3;S0}Wr-OT zd~h-N5q6zZKnAUM#Y>h3xgA*MBr0C9xd5YWP!arm+*`L$V5QC>>*=NJzTnjwBl=We zRN??Hxpzqe({yC`^9qp#<&L6W;W7dQH;Ni~jpaWaTJ10??6)VhpC;0g_ zprrN2g$$TP8xd6 zxiqiWqE+beK`YGX65f3>4w;nhs;u}t^arIIKGHezrADVMVsTL3EwsqZwRD^JT!LH$ z<0_sAhG^B~r>tH?3Y4#I1L7?=eD@8#@cvx`$^_>Uu~W+tPSL#?>4olJDdJ5ZXt~(K zEDgN0o;15|)XU(kqBwTuff2fNpbDqd?*Ih-j@yen^z@8`0Xnb#A3Fa}-N=8R`$<^; z9TWZRssEIJ|Hj7utFe)Jyy*N>m4q!n?X&RO5=l8SCq+0MTk>?SESQ_P)q=YR^e3{a zpP_(+S#t3VFsegI<@)lvc|D-~~ z!v@Z1r-OVjLAB?GxkSGHqxa^N5)0EqRfiAcnte+%t-WyMr{Kl|YSp&+qwvo#N<00i zX>^ZbmsZnK_cXXMtL)O5M(-QnO3eGbMn;Oi@a$KWG2{ z^euD^?dWx*QB8XkLxx&qjgyhtzke64G<-e-!tcmjBrqUPVqb|F4>^PlCfp@c}s z^fOMZ=+C#i={_9)TX6ok3iy<$_4dww_3rgoav_&(AC=sd+e0^;KZ)>iD`tDJq${8S zSdCp6*WN<(wqGxA0~QF)K7D)>kZ^YKd+r`oKQ*{hbM#onE=1OjBT6Hykd<(Gr94J_ zsrt{9{2SR5ih=ITMa(AMSm56C0yk@HDoB4Y%z3BGsB}NL$}b>AfFde2c6spq!=_TZ z3+ox!9B;UhJ&zx1a7D#7#b!|V4E(kxI{2IB(w(3KtF}iv&Mt#L4L|y(cS1g*y%W8PgFw|iz$BW=reP#X>W`)uBeOGyFV-VtJO~ctK)7v>8`)>=u8?P z?6h1Q=P`WSvSGz83_andpun(q40B#iXP3+Tub;d#&jj#P{O$*!^+*J!&%9WNum|tq zqOD$)hiM+Xgwa(!*iq>5z`wcSPgiZ)61pCZ(~X83pIp3c`i%$l0R>EEOH@&5m4}+X zb_|;3X>#b1g6@cv#9RNl?&qV+!dsSD9a;+#Ma#vb51Z6m3U^yrD+erwye^3J^pTes zhS@~Bk$6_eNh5S^sApU@n4-JYL8_M+#ck z27UjeAtz*iqvjVAHf@0evB7%T2EK+VJmmYbEUbvugH0&OtWOz3z&z zDfE&J5jO1z+XCCN9YxFgv}C^2k(_<#<*&y)I~EKHwVSYN(3k z%qv`+0LURYfQ9+K5Vraxn5Tv9Ah%F*qhW4UFmH?Og0^tmko&8uugx`F=&LlCrLyY? zK4|Ow-<`Yvl!+*b2F=GRiRONRl(i4ZSOHaN6bCLcdXO6_4>>&r9 z?a05|Q}fy+liB$J<&No`g*s)Wzv6#Ia1_*32-EPcx-$k*5PrxMj(qc)l}@TL6o-Ud z3w+)c*x%i38S)JJ7<=EU*kdJ9P_A$OPsYB(zhvHJyWaqKyxBlCF^38PU46F1YnX0! zELCr%(EFtx<&CqoO_bIbo9NsYjkLaaDxp)&axHNri#i20bz#fLyAH2pLS2Vr3ok*$ zarpQcRC59WzFMcdlI}Bys>txs#B}+?MdZv?AwrIpEPB0l$Opit&zOTwgF1js%hs^W zKmNw9zu8IR`hEzL95?P_QAWut6rS{n|9e9Hr%HUy6D3(}&zFN+id!7Q=LwL}!RO7U zYBzT}D9XEjw29J5dTD>{g4DG%u_A4qsZ1x!IZ;ubDs_8Kh(hd|-nRB{(6?fRF}O+tuU^Az2rwhMQkwLk(O%O`&9AQ6-(0xz()1W zm&(EIHq|Pq9#hj=r2u&LiPS8F_V*_iNXe%s$@R-e!N)R-1!1Z8tU<13)O_4a0_yR! zsN9LGS$z|Z@jGxIex9*(6eZ-a7v!XTq9R_B{qd{zh#ld44yqA+-3PWase^EPDb@$N z-5k0z3;MR8SRu_ETy;N3D5Cb}xUeMR@3uA>$WM3foTCf}l!MrnCv6{j<^KV=L< zpwstCEOH8*<*o?^N$KQ1aV*xQVP3we$Y9oxsd3IhgOOVC5?UXno*oZ+N~J@qkD#yZ zFk$gSSY%ZLA5G0q`7A=3NKDg({Meu&3bQMd+LWaomwq|k#F8Hd+}tyHWKM1<%5|7x z&|wHaLL2eFV<^JBx^h!NX>ELzO+&x?p!LgrJZw1sh_6xgejRy>{X)BMwuxEBVvCum zK$5lA8+0KLnjSn9+y?#xAm> z2R*SjSN)&KexWcgk(5m*Q)7v4)`9t|AyapKps(pVN&c@2W;)2f`=J+M7~kLI?oY=_k;)PZ5$LD?`p-=V=S%(#m0 z`JP@W?;_OVZ?HW(;v9>yXe+f;w7D_IvU}}Zoow>x9w)A;NWG#`i63H$dM?XWQqJeG zZR3N-%AdoM`iXDVVQ!wiZlo&zk9t9Z`ZA>bHS>=z3vN3Sp%$(wP)px){jIKL$`&Hm z%J58_Syn@QNU18ZDd&{kTIbH6`?3g+0S>Sb7%wRB>2tYZdf*V4q4!=wSWD>RO2Dz3 zqP3F_02E2}`7H{C^#waB^?$N%(HNx!zWHJpP|B`@gdIE^v?NrIMg@exSQj)?`DQ!= zkJV&RoVnrkOwT$I)zpr0CuREv8Bzzn4qF-YM!X|zNx6RSaJN?}uaU%2hVYAaP#71I zi!R}B1btZgJW{GRzqT;Z<7c<2~cSqf}+}}rk z^RCaY+-7j)P*bK=#f4L$-nYJ41~IHc9{YTjddd|(w0@qMOwruHyFR)UpBFg-=de!g z)^RUk4}iXu(F=ULyOf^QnT(WyCapyy4NN)PIkzbMLW7R;M(}P7xv~`IYQw;^Hok!j*%hT)XF6QAI*TEZq!4*g%%P6TuMxcmv zO5;iZxHqJ{Afz!ra&+DWCadVqK{dr560q8m@l(A17aT0E({h&DN2Pibv+?=+MWV)l zt%&jFaKASM&IRn0poPd*~4tYLs75v7#G_(2EaiTDCKCOk zKqcQTul3OO(vyeDQ@N*OmQ(VZn)T)?^yZDv8f5*~Zj--y286tgJen$Z?YZN4))u`~ z``RQd&(p;GSZPdbQt;jxa(XE%wE97)J;|#KgWt z63VUPsMOpsN|45-fyghWXL0j2tz{mttLUItL;N5r*-LsrlQ~Y{B+&nIS@eji9I6(gDNINm$%yxy5z1m9S!8Wc|1H=w<&0dWRADD2)qGxw? z*U{j-GT9j#CD&F+$C4$Vhy*#S8mMAp<0>-X++d~-P^du7XEI`wzQi|wbIgFc+G^Tz zy8zA=D}V1zhihej*A0(k18AeeJUJlSze`yGX_Sy<&d0Q{@%-);AR^Q`n6ur5t8rN6 zi8~B`VZdjG1H-HfjI{yhq;GQoRFHGeNkjiD*nZT|qNVkiBf8BB8%Nms7+|s~!tj7> z%suIjSYd0-LPE^AV~i1!+^n)X1YYOvV_i*E>Jsq7F+lvy#V)O!vL=pUr2TSxkweBH zQc>5O7C^Z=ba1*fgt~@gK0ts?R_`s0`OM)slm!l~Iug(nj(I$3uIPHrfUE7L7pVM;J zG%!+1JwEDFFJiO2>hzfE>}pUdC#RsRW5I|r!R4yUb%SzNs(r}gxbj2g*^5pA*>ncG zTd{Y6-US?9#|lnV>@v-CWCsL<*jA#b6u5~2?iNio^L0gbL3!X5ZOMoWpZ275A9ZcT z?eW4fVHh_K-0g@v{!1J`M67rj>T2NU z8l8}*WE4s!3zI%ss%zFQ$TKsk91b^&-6M%aB%)Nyrc)VPAz}Rek>UVi^$xG3Gy8~h zNx{QBQ1XRu!J@W`Aez9b@1S|wY=+AARtwc!6keuxa!1{Cw6!}McHMI_PT-s(bhQeI z*I`HLZNJ9EdXtO7eC$?uW?Ha6GZuw8hC(9p3b2Z?J(}*n2f(+T@mWZh&jz(u0XXqY z?HFog&-X2-uLL%nm7OYjRd|JEaCc$B=eCalBo@a_U0As~eAP;a%SD{SpP)%OGMB>_ z1>DLlOv7qYDB~r?mX;kKAmcT{1uoz=)tiy@3o>nj16H>O#%N0q`Yq~bby!!UTzZ&M z10zK2IB8FlJ=V7fFkMoxLw0)4pT82kqV?VonkZescftuSSb+K-3o7d?qF;=p z9LBL-U;heL9H7r&>qE4ty{&yVzFcc8gBnng*_rv^RrW@0v(l5w}T%+6N00nRxv7+C(*R-wYkMqM>iMI%7tll{zz3JcCr*K7nyGCv4i zCYz7KK}YXn;&)$zw;ILw5ptt>ImPj(}Qm)=?4OC$aoK_DqF4XL>X|!Yc6MKVUQD2R8Rkym-}G zWcd2E{hE<*7&T20|29OX9?M^u@~67u2W>P?Cl-fyZCbAvq^DJkqg_=d9mW{@Y%(b` zaKyi*K8q&GX*TjxnhN zU(Q3U9WDNd`YM_;+B>gmu&%muvgE@MzMc3SdY2dQ_Mp_5AS;sj-1;5O(I`24nMcub zXObPEXXAm0y&X&GfBRrKCuq0hfN=a&wJ8~@8E~zb`|_q>5>VpnMvsa=KJas>@wd)5 z%i5qSHgLzA^xWANpr#G))dr4o_~Fyy_dj*vx?OXE7&YPsfZPP0H9vrg&9E?GjrnM% zHoqir{UHUXB$~dF1L~zcGw8476O*X+?H-dE3V0nYrw@bjaz6T#LipR22ne-K{b_Yx z@zg4uORfeLbYxuUnQ6jbc+uIZ_8z9rBzksGk&8fiuMDW>4T{yg?eiUIkMsfPGiG!W zGGF)K?XXmIZPm};0@%p!hI@yA83L*B-pGv<+0TQCdHFDML5mh2$A2%t`9h7ap|n25 z1LnFMi<{5~ZlpBEt$&rh?dwG0z-ZY0XKE1I?Mg=olKYB>AE+DFiOvSnPBjzgAcZ$? zmjfYN%ShjPI!~t<`$T`w=mm##{K1bHu9rzUQ0N3D%{uimzsjGEch$J0=&yx~V$KMR zSh%aP6C7414CD~$$|WCLJua|-Q?(|HOhRu_$ zo2qvY>B)4Ug~Dvy2UfLKR9`o~I%)D>yU%|Fie3CpAe~u)7e*$(u3U0^M9OM$D5n!| zFjUriW)t%uy$QFI=g@LMH?8CMOz$t3kY%Y$Z(f>yt4uKAWXzXn-3kyN$y?;pjm`kF z6h52DU0{b}P*|fnetXRRQCf=iS}P|=(WZ(wxQ3e;P1_-6o(B***5U9T+eo zRYmR|c5>BYHz#re0mP1$Oc9R3(N|p^nWEA@>j-~psGvEcuY}J|As(93Oh$-DTb}D< ztKM8sucN4l6u>$%AWnnp9^^-yjLvWdiDvQH1qE)`rECh8&ui$4``JXyE!S<#B>-P#S9D>;(wwjE!flY3KgIs zBC>GZ>rDJYGe>4!n0bsjRVmsO8C^ztAG=n8Dk+O7c#ji;y=S!CPeD~Yb@PyOIwp|G zvOz#Eg;i#Q9vBsApO`#q5T&MX+uTR1T2)Z1_Y-$hR{KU;v4)1EXs!oBvr?(Gs}E!gu-NQ;*NE@&MCB+q z$(7|3=Yr_VT?yUQ_1Fp1;W#4KKTuy487s~KWSdt~Ocp|@(v@+WHX$&xG>Yp^=q_L8 zTq?w+09f?*dBGxb-E@A$D$=1_W6n^dWc+<3Opd?n$A04f>9Dm`ZonV6tmS>4=qWi< zfh5)9y-6UjMzRCciBY4Cx6fmIn~eX_)E`_I2E%GOD;nwkjrAE38PgS0%yM5%TQ~rS zKzEJpui7yf997na6FyR?2^CHKR1tSg1TxU@9d5)g+~H`PowrbLV$*^zn_WnU=1|BP zz@Y`VL=S7ibAlc>7Obgm)mSv2(VDL%)havV!>0kEOj)e^m3Olb0O)}!AR3ho%o`ET zC85S+lICJ$eQb01`d#-p$}+{O2~F!n(XRHW`7z_D4%jAahKZR|O4br17^~@nJq?&^ zze9P+RKvZm`9gk>W;B@`R--rTB=_Rz-7dKq@x+!&WyO^%elB<9s)<?=oJrx4yT4TN}u`gk81(p#uG_LgvP=DIO(lk9=Phl#q6B z43g9_LmzWe9>(PfN${P`H)obttkZ(5YnjZ%?_jKLZXyk9rzs==Z@to#sit03#hwVs zbsGDhAlbklZ;0JB%E?m=LYqg%ySB$rF>mAiSdKp2QZj~)6zE=t5392`_1B$1z>IAR zadfjo<|6N5G2rGgH=0YtAZai@V9D>!X>tVar!|Gg>dh!PcOPPyL zaU2v6eLT4hUs#T!z9KLYtLZJjPP@)$t*f1>r&gHnGf8(Xbuoq$xVj$5m(6%g7I~(4 z(V0)dj6wH)03m)OAyIKrcnZ}>*R=S9o)-Pf=T;>?cwqdAalU{k0OlmP-Lz3kw3Rdc z4ov6E1C4(5Yb#1%16AiZUY_P*E3iiG!~VgSg;<_=05{3+aIGB2n%+H(Eh_0#7XF1u zI7zV*@Q!?W9C3dyFP&5E|twAkY%OOffBlh#OA zD|ILEOQv`rJclFM9dBIW0j;7H0cx~ES)bGX^I=&br#q=%S8%D4v+ARirrWz z1rZs2aefqyUsP9H?0WipX>vr&k_(ev)h{^-sNZv+9JckZvb zR{9(;*Xpeb-eLG?%x4=QoMR<5&9=|%2+U9lI;L1UqsRUfkQW}*t4~Kj?!7J}3Hwa- z#s<7$i7VPx7kI-1Y(lf3I^A;IoAKLUh_IaH!kft4x;A3mUeTQOmgrg$yVWOyhNlUs zLv=#u9bGrIc*2$*Sg5a;B#oBNw^ZQl6@jD<;$cPXh_7QYCCOIm+z9vl;3_G_peg7V zWtVGd3eWgEU0Q&qBSP$?euA1 z@d51KW`i>~sis#&eA~!|Zk8QuwO`2HMWhKzU=j(ngPaDcK;>y3Xae>f60zJ&hYoWr zWT=W_GPsJ7zs>k{pzI|J#M2N2Q{{PZ)PRb4XK^n-j ztI6Jw!WDV6<{Vm_7(g;-+H)7ks*-x_W&2^@1GG7*Dbmu~jt{3(moP;VQ%)Z?Dv9qA=|Gff$rod@$9z zc(2AC5#2Lt8~g4;9Wk<(yNcn}H;&U8BBp0wb&QjKxfDDM^A_Wzkhpjum)E@hsd4eg@vH>C&m6rL5nmo| zb~LlY#%OwE;lKeeRU4+;fO@@R{%eS*1| z#i!QW-ClQOf6Y;ILB|6Dg5;c9y;Y=CtXnCOlyIHR%%C#F=r99aW@Doxaz;L0FD@5Fb0q@_bbJ7^h}JEC!%+90kU+`+bZ;l25h zTIj;c_a>+MJ#Erk4j!GI+*yx-bG?Q4t2Om&sI7jDaG*6pxA}d z^9lY}w+OD~@VOi89+_ZXFaG{{m)@U67x(wU4GmY9n-V!$7vG?@{?cf6RDcc`q}1i_ zLd|@NKL6p%!tnK6XISIg{WoZH0~;K0O!cUFO?DDtVb4tE3+;?m5F;Z1EK;OT%`Wjy0Ov7F)JvlgiOZ_8}n(;-8W*;?nTCE-i zLK6Yb=YKVc&KzM*ahtvfSlbMhu^#P-J%dQLHq68<#dsU z4s3$&rWf4BhfhKUwk+as(7ndT(7ld*%Ee*Yb>j6q6X(VMN)=smpk+4| zXkf26ViF~+2VX%{?8=C~3IAXoaUHy5DLMb+?j~#ZnU6#}dIlur6hk}?&@kFDZpU01 zw!d>R#v`4XUNW-O8)boKuIcy-yK%2iLr9eXO+U#ms=rhn%!4@k^T|SK^Vu z`EN}^%Y%|g6H;xkXi$il98D>Cm+hJjx*UN0s!Zj5irA0@+ut|74e>v^se!sK=RFGD zYHCq(zpET!(U6L}$rFD597W9z(#m$5rnk;|Ekkh0Q@Bha^9wK+2czRQQPR;;*|fl^ zd|Rcb=**(GZhYfm_XBUw(CblwwRwf&&z$}w=_(5PvBRIKavpy8!)ql#$Uytl$cL9XI(0q_J zRVFoQfDbFbBOI_^eM2S-V5%ZUd)VHS(*q zExr5rn~mlgqYvxUiz68ey(1Kc*8+<~O53|8YN4w~ym&7J&`uug*nV&4jvIR3$E8P`SmLXk9yaqE_D#&k-prBkZ%}!jA1*nD?Iwn4VfD9r@BhAmb93 z=JuQsP~LMvGcUkmZLDA|71lq&(j2)x-S1ZdMT98S3e{czWdNjbYTXpuX?x*NcC*!p z-!Y&K6i`w*wViyvaziwa^<`HDJ6b%-@ToV56JIdGbp;L(sR-ME?4Wb*TWF+iPqlB* z@9?`es&dfC@a=+PIyP_KQ@EFEDoxwRaCIVGBgC1ELWR{z7fK0lsbFn(4)B$gZ(rO< z*1!Y{l~W-04--G%^~#Q=_Vy;cqM$rIE4#h28tZl)QmkLE(dU60r^v*dueEK1AV^U;9y{u$z}%ti4=us zKTT!sOjy~rICh#f7}I()1U3WIj*9!KfI%5S z3czRA=R$5hAFF~-67j2MyMA}}7jx_Xy6%DlV(eJ;41iJR{ z02pGg-edLE8(4B`>UCqY#hN2CNI{dN5#j;uIoJ;P7sK^*Z+e~CTgR(MEh%DIvl_#S zSWV=)B-LNI;d&Esh_sbAmUrZ~DN8XPqb67@0b5CJWAdr;shKeXd>)t%4`v=ib@XQ> zP#v@eHD;wO*-Qb}v-aEzPtt4ZrDAb`)gRqb{!{@*c|a@9ZU*5oIXY9^Q+yv0Jm(cp zcERC;a_vLJt(;7$Nh7KONu2PSlsuzsIZ!z`CLDFvsj;hC6+3-2cE!4!Y^7;?t(Og_ z-{Eqt<4v<+2-+BwDo*u%01YNcw!QtLjq#A{eciJ_2BBETD#SEH=3=MSuxkkOV%VJA z-~=nYv8V2r9`2?s&l_se^?k&)1vlPu`XHGBI}o2C+o{xG5>~@x8F5#awPiOLePJcKf?ZEB7u1&}*?Jnf{^u90fUVBXTl~sBVvB=smCi0w(;9|lBy1W( z)s70EIC0(!0Dc$5u4cZ=wn_FX?pvC>lzQs&1}xUM^7T(kD}iv)T9l0t?lF`eR{Nf+ zt^G6+h_>;J+4qK}&5J?uQR0%FaoDMDtPhzp1ng&F84PySha5c8}FT-y;)vPhzRV_{1TrZ}y>=BUV<%i?`XLDeb)V#mioG3C@bRk$j zslMT$%x^Z0gI_?0e!rZyJ`DGpON=`5684TQVOXPrBG6*oDympVB=l+Tub0&MZGeIi z?T$4@CAaH=zpzekWBp+rtV>}V{mnCbVOWxGhbN8MSLz7G&apnI**H?hWku7Zp2`<5c54K2<;O+rJ z@C1TuaCdhC!8HVTcXxO95E$Iu-3Ixa`|a-C-TN<3Kb^xcneKB=$y;w#*H~1!K7Srg zTwEcYOTsk&;}?U*t?Hxg9KEBaPJpv0FPdqC>>^Y9qdW6R{&g3C@RR%mf%ld@_L7Yc zh)tCeUMvG}0rzu~sO001Y(Zgyxz4Di5GzR+e2%z-u(ULL)G7PqdzI*G0n4~GfCH#J z(A5Ab5NOvQoqE!7N{Q$_3*dS9Ex2RN2~)mG9a@9Jc7AB-8bKfIKchbN+jC1 zCZts;r^X1m^V=ky!a3eDa45@Bil{NU7buO(N+mJ8iarc0n|ii@nyOEaMF%9!_f48T zYfH;&&jp0#C|ZBRIX*Fn=2eqVQ)iAt*A?!G&z;+f$)QpK(-%^JT2{Vmkffqz+EvIn z5BBooh=~F#Pi3dDSMHiqmR=-0$(t-Sw?@*FGm&W;j-l%rX zo@Y^*r{LGv;@ohM&ZB--;la1KUI=+86DV{Wgg{)h3e&C;w{IGNq!(9xvqXSHX&m6+ zczX5$$qRpz^ZWnQDT7opvn{Eq05duqI4-|^IQSNQ)y6IP@vlnS?v9lF<%?h~MaHtR zjRwBn?H6%fc|1*)y;sN?q)d%3e!ZF6mMI7Q@@&0Y7IX^+@0d(7dZCx=`W)p^3f=WW7~5y$J! z1?Ru=q_rXpu4)frm73*dW7*0dXSJR@z!w~C=m(yUodfVoDq7^UfAumMyd+u2 z0##^BvDZ^>X?;W0C53Gfo0E`|?bXYkB=qm-k;KNbvB$yT_HSkx zK|QycNd3CIiJ$tM5#W8c07d7l+f$NzDz&uua64HU{1KudsdZP;_!z@$)I!$lT9VWU zta;*dYJC+bbQwf0L*7oxCUbT>7~;Pdg$sc zo*F_`D}y+@aGT$2VM~>{c!h0S>}@*4;0%BV?ZEWmXPixOvk>6sd4{y_hM^{IcyP$asF+vN%-keL%^-<-k}~x@zG%ysuR#qDJ!CYiW7x(st$+h&;Zm?>eUz zj>^oxKTJ#IrS}X=xdNyy$44K+gMnkt1NzraEw-!K9$vBHU4LU%jb+-Of?dpDIPi?k zdj zsm?K-ofus%`;Ey32^j%8dOVzC4C?P3ScMl`#~S53|3Ub;+3B{l=1}IAa!4!lHVgyo zKytHnTIVrUq~)-fok>8ETO@E(0>6A4Q|7*@7#%FJ9!>z3(smupIfmSwqPt!eX?yH; zuiYtC6x8QfsC|EBEV8n)h8xl%=iGCujsscN01uK0p!hy=0bF0O8d%p#CB zBD#9^9&Sjj?K{Wr=DGclT7<64b03qHxpGgRO(PY7nMxv0o!bvXay7;F;hT-jZH9dt z8xmj{Zkf2*B=v06iq@YEbp9IC*=~C|vBxI2ZL!3vuB-(ROB4T0m7q^N;TphVixR*) z!19m*DoOpetJ`;`!eYMmkYh-cvt1l;Zfk1}gdFDM=o>*6-Vqp+iu>R)!Oqd79k`o( z8`UQ%PRj*nvlzYP`JEr%jQr7q?$STJ)9HDBMD|XMlZ&I0J4nha8+a|@Rx=Hq!&^e8WDkDK^nZNh%q zXz5#xR`Lx!QT=qzrvR|xLqtIJ-AhOFrDjzxo+3esL;1!fjt=x1Z_k;G^wQe+3*mKM z;$VpkjmGkFplur_^f_btanW^|@Pz5-Wq>;Yx0o1JJT-DMCO2O3+T{UyD2ZNh{%kjP*`$GX*QerIWjcC%ZsoCYlTFjiO3MOh7kUOY z)Mj;iP4D*eeJMvf$rl%(Cr0@G{bj}Rc}cNU-mH&9Y>kVc=}%YeXwN#2!c8wgS4+eR z$4P4Fv#RLPrIxGFwt?o0dv$a?`%XL5s!N8q%(0j7Xv^ivxr}|BoFO@nOhQRJUGju> z=t+7%){kpPEf$;T7>xpNKR6WFH)FN8ni=xMFBrWf%xDwpU zF__tTsWkTU+<)IJ9I+BqUT>job;;$d{WxlSRCqGqcwE@4Ws-Eo3NnA5YOy_Wa)eM3 z;FjwKXGb~?g0=|hD@QH7)uqAn;IJLUlealV;1cybLCRyb#{RWNaqw4i5UfAaOxNP- zH7es}Qvg_`0o;1EF~S=HquiFoWpm8xA1)66m$)?+)^o8?P|a0cb#iMLoqjp@%=pv7 z#9YlM_wdbgV9Uy?(bVY$nRkul?ET7EUD9J5+Sd1iG*a?VEevvm3Ag56JpI_*|7GL1 zv45MIV&~X(UTw3ebXSdP|IL6OO5l*Kq+!1Pn95-Cy}2iPik@&dWVt(gy00D~loAL@ zRm|<2^ySQhJ`MFEp9h6b7-BN3-ajp7z3uR-;%aXR3UWWI_==vWduRe*SqG zYN=)eI3?5%@0ud#+R1sh-ET6s&9o+NibJlC1v6MxV804s`JDWybGVS-HfD970I@VY zGF+uB3qAe?arAfyqMnkgDwa9t+ugOwv1xsR9JMHfoa=FyyhE>B96MX5-?vH!yUb@yPttFL zs^f*P7t_ykZOlKKTRdcc%0}Q>_ z9HCwTsx?1lDRu9&Dy2d{KIT)0&v+GGf+x2kD(|ZR;{naGWT1+!epJm@z&6w%t~!|x zvHW2}HR1HpyfI+z#!45{hLs9h@0lxpJ@1~PpOniv^WJWoqn#ml|In>uzm>#-s{hLG z(rfPeyPySCGOvutBgsRXL_VGO;B$5-=(9Ai+8w7{-+MAT zZ7tc2-HzHF`E9fqwIY`p6*#`EjCn~*zTydeMQ8mW9|CT97&l0M%Fq7MggR`422hoD-&boskzU_SVx5hrfeO8K(UAb+XV9gb(tYhd8$p@3dR z`~^3e0$Z1K+6W$5dX@jU7uNoDP+>B>(5=wpMxjO-R#fP}2c}4Ru;o_fVPb(ur=JmA zeauIs=(MGWYT5b?XdoM7U;$T?w07 zj{IfCCL{4zmr3^pL6iv|eR_PBmXiZxeo1`=GxCAnO5Sus>DcS`%KP%LlfSmnR5jTNDUWE|Y3H#FhEyb_k{4&8IOHdr`j1zt#G1|Gqs{ z@yljkK>z4|ulVjgcIso8kI-fCxP&27VW2lgd<6Z&0Qf5R+a~d7-4p5XuT*^VA@?}F z@%RAVh2oLMNNS6T%DKRn>4$h;T6d<-Q0TI0vPL*cKHeMUj`VBuDd3td~R8v!Dr!@Br#+4hH4h6_8rrWybFUScIc{ek;!*dT&ECL9I=CR5e1wLZFq2L1x>e(&RUn(wG^>|nRj zPf+NAs~ZwK{S`?+d%sfXG9jOFSwj zCCUxOdnp!3_|wk9tZ#H0qEqEH5ewW1yfX zxkuN_+Nx`0IInQ9@bf(SNQsV-Yro*)y`aEmG5&?GBH z2Z03fhJ@?PH9}~1`btemGpdoVRd2RW^^f+x4mk78%S8r@&~u;>3T6LmMb%L&byUmT z3*BGk^g4Yk_T(%u?sWQXY#SLWL)#r#=mrqy;xIO@V}zxoj4`k-rm3eo9-x4Cay@Go zM&vm7ro(S3NMRNXurMR6>0crpHjchX<8j!NRAL2hY;RvDvKS1WIfwGca;zHAwIl0t zb7E{`zkc1y?0J9V)MUSVrhM1LgW*w#juDt3bW1dR{0$izM^XHX=RW#Kyzri=9HueO zP4?R(S#db%mON)$1f)g`Xk@RxIX;w=68O`e%Spn5e38`O%I`2LXDY3$u79H^0eeLK zjQ*gBWzRjveLwKxX@Z^kO{6|neZUK(as7)Irn}@6QkuGSZ4vJcz2}3EdRuAr`((wg zZk&1Za9p8#)DF;=tnDdn2Vc}-XDA8Wn@p~Gf#KAR3B{yjuZhLtO7njby_bbQxG?J6 zz4?`jz?S6wprd%o4K=)Q{E?9pEjnhmRQ=^>7Zq+t;d%b2tj4COgr3ieiUxGhXU?R# z{Hy9>unQ78lY7%X!7rtt1(h!4c*#Btt;1o_~KY1rDdYR82=8hM@?(B+bA+&w=Fy~Dl6a41?%!%5g5YuX-xV)Bfr3t zE?{R7J6I9!oq^)K*1UkG<6=L!}o7<7l$Z)%VAz!eOR% zVD;ggrIftvTs84=7t{o7rN}S~Hmuu+&vIWEHH)Q;?66?(9?PlQ8&!-6O>BU?l3cGi zozQ=&1c@()U2f;ZAh;gEOjf3MMa&c&9XUAJ_jqPaU z=2X}7t>GSlnf+2qM!qYR&E#hr)1KhLOvQSQ1gdCCh!7sMiTKM>8D8BH8vYX&6rw{+ zD2!Ks9M*(TtWFH@H|A{6s(YAH+%B(Jy1=i0r1)D3rIA7i>9NFUBkbD4CY5wcke48Ak8#-#{ZmNsSG>LV{=h0^@`&YzpHD>mnVlKQg_y8a=t zFAs&;v4+0ku6REG9Q-V*ckbZj=YAo%4553)%SXjCTNG&0d#V#%^RTSX_d1X^!iZJr&ZAhfj$XeT zgBz>3Iuh=q;B|zpwDF2aq(xWwg2xhD`)+Ae)vr_xKI^V$w{^ena0@*01v|oq>oy@! z+}%_GHj^sy_5V!`lz8*v{cZ~uwQTZXu1}(=;7xv0G@YpF2#r+BT9|3H%8B7`hd7fIsQq1CmE19YgY2-d)Miy};_ zzv|XyToDZKX{gIKog8m=$>QBwvYy}YdTp|bAJ(ozV*0AG8VZU^O$?SCbEh}zVLW_C z;er{Efsh;u;p33IjF=tG3;f`mndKu0g}P21h(XjUOATA90-FYso-!U6_@n`2*z8XS zn2=I*&LcClEa%?iyfINe?g9&3%yA@ZTvf&|q)cw}4cBBjJMhyO7@`Pm~+h!}XN#q>v&^2z5 zpa<33fP_U;q&Ld%_olD)t|?Y-$l+!5Y>dgr4~%C_6~vBHipsN^vdXwhSIeVB+R>@0 zkLI49t&7A;7=LKaANTc#F8TbRCWYY>d+U=9b<~Kn;-5AnU^!J7v-x1F_~z}~x76v} zv$M0Y+as6XVphBYHrp&HZ~CJsG~?uZFJIuK|HVZ91!V?$A0za?xL*9>Px7iapH3ki zHp?R*W*U{m7Q>SiacM||=eRTZN#tu&SOc%p*ZD5tRI0K&ED?Vc6B0WSo=^Wpd;ib~ zcOfCTdUPu0L^8l>dHN14yrHJf(`38EQq9ZET*F7X(?dc+(s1H$JsYHx!wA3hK2&;hZnrym(Z&YkawT1wNMA+H;9Re zT`@&jsaSaEe4i0Ls4XrohGV_W1^#IcjL~rRj(>Wma$I9FxrM`zH~9@KJMtg5{^zfl z?*JqJj;SdNOUm~v^Bxb(!p^HW{>xifr^q>$;2JZq7JL({ip-+h5XF&4pXuF5wg`SL zpiJs6Oj~Apec_AXmI+#xKX&hbb~O+kz&xj`d(%pBuj1)Yt(Ck4#Y!!d305d6DT`J( zLKIgh7I`Y7*48@W!JSt~O&v^@3-!_DQLL0|5`Mg zE_O+{xw1DK-8(xwGh>PxBTMP8m3WCGvK>t==lKfj{IDSTtI>3d)@|Ilu2i$BmDz9@PuY{O zosbPD=ig)p_gU&Y>AA7>K7E?tQSxCtT55qjqOtE9vlf5!OuW%B^Oj3sF?b2nO2NzP z=9dK{r=wR~tL7Rmw~P=uTq4p?=l* zb_OkPpIyNM7y>o=g)l*X8lf=RA1Law$zFt)y%b?DY zFydMKS#Wm5<~^3Wjg8H1cMuk1_7Mu`9y*U!g)jKfIxL~7;2(Da}2Jt zT6P*heKhOyE+G3?W%z&ztd0b9+J>cO5`1z1o6>jP4bbprNc3?2#L?3=x+&+_t>!PPXrc*|+3VP42Z9mm5XBa6LZ=jrkP)agHi zby7?y?F7dtxPk|D%yymBvDDJr+gtuxzO_`R_=_jgslqABqaYgh-^!a&VVW_sZWve1 zoZT1;v{K(PGA{dmqBon&9f2=)Bw=;GIjLO&#bo}Qet*D$W@%S2$gLwXVzAQg(h)Ep zT0C+Qca3Udj+G z<}})7y*CASbGMM{zo_|l?TEyKRvsTw4cet8Yq_!DL-fg&1?KH3Aq02Bn2_BKQlf7Q zPS~AeqnXT=Q6Hf_p=M!a^*X*nB*Ph`?&e|#!pld1l{9SRUs73LCrglG?i?#9D7b~; zq%Q4-^+KI|N>`bmbor+$l0t`L-Vm0gtQm(4SPWaOOT3l2d<*?X8rZvFSpm5EqI2*0 z>R0P+Olq2Ga6P(|+K!0+4`2_-6+`W{=UcqhG9AC9QxD|Z4ic0t_~p?tz3KnOm@u$& zE~LF%emixiY(@u#bkO80q=4|DbzaCTn1M?@LYyI+8yodkg4kzrf&m51I4*hDSJ3l= zgM%SK=lV+T(J$ULGlW*Wo_xAOCFHosNMW3-wSWk$w@#0^Y^rlwK%(Wfs z!EV=`b_WXQ1$iS>{{TpcSj>x^+h^0S?{Jv}V2-{PXD*|o=|hRvko<|i(#!o zs39)ywKT^}`rvNyovHMh{6L{S%nwqF8D<;tG4qj)Yh9Z|ao5T0W=b{cLFJh@0>gI= z*$ee^CCnV5A@RrLTVdVzeg1v&tNiGF3#s=jS@`?M^#isdVq!h)weJLQdP|vu0c0o? z2ta)u=(4_-wNxTX$8Rzg=kC6>cP_t#-oHt*?Ms68DZQ$z2`)ou_J#ixB3^~NB)-Mb zN}x7~o=e1N)*<2lMRm^GJ3)q)d|U(Ckp2-XVZyivW$Jibwx2IHZo*(k)E;#X*&~n~ zhqYKdy(1r{@|*V1xRSA%eh#*3k|BtbzG!x$VRaPlxNXy;3=VQ&Aq7F3+<4vOxSf(P z8jyu*jKSjRc-|gdLEyhs!`T6>=fQPpRKK(_>sKq`xOAER$JaGU*oIWfnRh)ii~h3W zhD?m_9Vy7z(w??1)1pSCve&T13A<>1;2-SB*MiAI7E2K!=mWh*k6~d`rDY#?$3ncw z^dLB<5$5JfhM7dzM|TzL0CVdGeU+gk@d+Bhm&;5Sa;{egF4ajGCrrZw-A*`kh4b~? zqjwcCadGpEi~--1bS-(B`TZ+Szv35YnSQVYbuG57( z7?QMqWX;%-@(4I@RIIqo<9<&#MGe97;54s#bYo50PH*jhe9Ix?sIA{yNz98rE0YiZ z@uBTfj~IRA7k(e0p+T}DhNMoSj;tL@Q=DvV7LVov7E5o%7eA6MY2ZjM{GH>zo4PHw)NUWhh^e*pXmu@J7MQ{oE zao98&4#o#=Xp=Xbx~R^`&=lg~_iaE3!?`_{);0nV*_;+O<6>54=jYd5(w^~LPUNF2 z+P}R;VtPk|cipx*&QPo?_|BhkxnXc%HW9w;5tZr+Sr4C+-OyTLy#(Q=TK!q}_+-+r zdW7&FjlJa1-*x;a()U9Am5qupLyU%l5g6-T($AiCXKSyAO)Zl>s`4uy3M9trpoq4) ztQ?h4Zg}EPH^Ls;RfA!v&E^rnJGQone~IA46znlVEZ5Ih%lpf*s$F`&SJGG@BKIe9 znBvCFL;qv#@=K3zx^E_|QNlVb<5e`4Ie$(Ng{SyUr|V73)5yPB@m|DcioVy%CWn3_ z4tUQ&8Q|+fLnR;_a29BA7PE{IjYjA;0{&m_@ceg2j)DqzKLkSLid{!3iTg7Y@?KDDERd?YYbajI=^-UUyjw_)MuznL0 zd1F>wa$|aGsC10G#$#6P-`u)8TpjF%@R=#g;)B6B?Z<>^!L>q- zr%+`In_zGJhT661;jzWYgkl*SRwCp`EVz0X&e}#dP^CTbs-zw2y0wMhJg&_0>Zn^< zN=qdEO3f-QTSZyJI$}cK*v}&ynQG$1v8Dni8!U&zlhz$gHcT1uiXX;Y{=Mab#4284 zF-2U)MF+sQQ|3sT6cY_};6^RBzcDS&3=i_wM$Cpd=nyDZA@H@sAdjtuRvM9zs6Y@EqlvsKi8;Q{4H9fV%E_1N5+M)Fxdjn+}XZ{ZIhaQm~4lr;i^{ast;ao`< zGz$F~THQprWm+wFO9`w-O1T7BVL(Es-3#`oC*e)vrvxA?&@TN7=LvKC{?5tLLSOsF z&6z-x2q9S_DZv;sxcA;pi1DWdrRFrC@cemH85UHT5s)~BMq1EIm1rG0n6pV?`~QNRznyF zY3Opv8iXTb^)Tp)l+T5ReWM8_CR@%=x`VnC1+7=wr)9v7%BW{?PYE-em&eObBuBx+ z#1q!hEBN-h1_qm7;#XC7fvm2cpWrYITLkqx<>h1>cwsTJzXP*Z)X)#cV5Jv~VG588 z!zvdW0hUQ;<~xiQU&#p$np9Buq=))_Uk2m&auk2y1W10@U$LJfEhI9hYL-`WR^uLt zT`_qJub?gQ1AONJ`c&BVC04FFK-ylv8t*wyZTMjN8(uiN&ZrszwsKr&XQy)D`B19| z=l4Zmetse72` zwkC{n^gfZYUXh41X2xzzZC%7m;uwvH&9f`K_NxDm5WadhC4lkDhX&DkR4CL|qi|Xp zdgIGuq=G46z8Tu$i-tA^qSK6UZqV^ow;OrJ5jr$IE~l2UAVHyIBos~`{Y>XHBVK^F z_kll0m$jlB-WZj;1rj1yWetgk6V2vmDYjt$lTXP*beHwzJ)IX87V6=3iAtYzXc399=u&AvNQRyiWQ$d3 zE4|O?s;sOO=t5`%+kSI9Rfif_$rlj&GvxfwAW>|E)l^*#lNxtfe{u=$0;#gCi=ods zQ<4Qz%dGZz7&4*QZ%L1xDf*hV-6F?BiUcXaJUmE0w^g)#@|liuy-P*~DclMcm)0~@ zoZgwn=|NSI-zXB$0pU=}L+8qXE;>mMEQxmlFA@wmHMj(m5-Rj)1aI|fe0aG`wQ5T_ z`AT&W-0cVwi;H_S?i3cDuCwbsDaGufZX>QO(ZYG`w7UXTt5+s^S_feaBI9!8}5pIq<-a) z#!yfXgs&Q-PZXy|KfgPEcX^BMeqWrKnJLJu;e%-D)TWpyu`*{?_5iRExrkU)L@EF$ z#KLJa_r<>rZj8k~O1X`4TpFeDAK}_7L1=Wiq}HDuZ{O{T3)<(8)O0w`y`M;(^bnsg z=%zik>sIo~wUPqIDiNMWpcnU7k+Vm@?Pn; zc^D!MZ6Ja>WuWuS({+4YyzJn>T)EhtjEll*7To;^7mY^Q7FEde3)bh2Law_XzUSuI z=Fs66DsV>CO>DbzF;W{A>wPv93>@2iPGQh%2bI48Uf^6_EK=Wo10i>7TR*me0Qrbj{2Sp7aRVx0<)+B|NCAI1D!zbCmJb8^M?JRY-Dg z`c80VVGH3njggrS*C4|nqPQ-l0K)T6&qN}7;jjF9-Jakk{M}W+Z!9e#Q+VEP#*N$` zg*CV&;;w~Z>wWpM3I@s+cg98n1a+om13$|_vM87U($qBcBw)N?b92+9{?i#1Ctfd+ z%SP04`!$s%+l=zmK^(*X*o#2__IjNhZgWDZ*{H5lqmM4-p4lVnL^GjpMWU1}uGeO; zjNvMAhGb&u3Uo}=T-iP`gFL{;KT+IR1H#1i@8AiVTfbh{-a%U7AR<;-#B%wbcD0-e zx0-A(aWZ;eOC@{~c*RbCx;0ZS*a8qNfl+{p;&t3!vW0e`;(c?<$;)$F>j*V4FkmlE zt$osvfqOEK>e*7w=@r_mw`QPpKiX^nvtEt<5dBb?1Q@1%;Y{D=2tcw1J85C+M9t*RumjFdvD$1K`z0FZGW+|X@2c9*Cb;ZZe^Nn1LoWY;H zU-A|Z29poXYPEGI={>YwsQpK%+9jZ~-iOVh>RI2ebod?$$^!AlFl5464t8ENvMT`n zcu-~O{S}G18rGBp*L!GJAsgJJ>LzI=w&t}2eKdEd&RJ6l*{d@s6BLO}WxL?aDEa&d1Zn1q4(x@?sX4gZwH-mH zJ59p4KCCmRph6O&p zwzKfkiH?pAeWT;MHrIlCXA+RR-2V|1b4f$&)nq%o1m_6N!XNsz0%==C8V#GqV{4Ctc3C}xqIocx`lpr`F+PP# z;umi!+u+-TCw_=phf<+U0a-2t5dh!g(aiNY)%1q(B2%1LWryt?0?cH!r>59z=Uj)^?lIu8#P%stAC^3O%zjaQtk z_a$78CAt6bG=FaG8jzTBCQ84{7y#^2_Vw|y*PGtuQ!Uk(X8tzdQcc0m`g=M$FvE5N zBT~^RHB;m{bBC8NV3Y?HF-5(TXdy}XvQCD9bzQH#`iD=ZOVry!Ra?1*zvL7C6FmHp z(7=1nt7B-;Cil{wK=;m~vfh?y;UC8{>&f;e)2lW)9tr^Y9>Nw(Cj}ebZmqyuHPM#y z_z2v@!m{;Itd+SvGefR(9se809So`HN}Z0c+W?2Jx#=EZ`ftg%^c=u7*8ND27^7x9 ztKATL)Y%-2;azZE4~q>L#`=ac>GVbV(fiXqryMMiL1g^O1GDzAo;ihTj5fihN#Sew z|BE$;hoz;Wbm_XT_071?N#+5Mg~e((e8YjJSt#+ZQw5SL5m0KhDPI%rU-T|b#t+mv z8!UABqwX&N+jc#$d7n2BO)YpBQ=b(St=R9O^6j3>f1iJYhoq0YLI9gG@#r%(611|W zgJPZI;ap00a&o89CYP-0SJ3vITXT8&39*)0c;-qz?f^paHwAl2cv4vR~zVheIzVWC+N;f8{;$V2T zEI+Z_BbO@iWH1lK!Tef2wIV2%vroIJZYY%nf+~QI6~2Vrap# zS!(K2zYLK8Lm>3Iv_R=2mpa)2pqjZ%O>c38*n{~cmpOiy{n3PT#O&6=47s}5&>tNz!{(tAa4_w&k0lR@hNQoeV{QDxa%9xY5 z=!CfN0Y&};VQ6m_G7=JpxL2-|sqV$~V1Iu-E=7&|Vxz-0gyFcs$&kGE1B27emjGf% zCjWYV(R1f58z+=gW!2dy6#tt^g0*#GaP5wba+mz@9UQ-o)KB-vq;)A8dQ z!{luxA3fK)&JXS&1sgP%;gC#FbsueFkn2X2xGdR075=_Z?_F%4f4a3VERcTX0o*fj z7^0BuVngtrM7<}Fr9hrqnuNT`_&im-94x&3u!@?@ss&LUpn9`SS2jWz=A&1s9%NS2 z(BIj?!D03}W~={SNB>6=k|h~Xn#RDo>cjP}sJ*DL@J=p;=MoFuuURZ`V^AQCjmD)& zsf4Hf_%?=F+6l+USqs{SMPQ`YOQ3LftL}jK5~CO=S?gSpKu?lfotRME(EAACP_n zgh_67Ai`irOZz2gxa6KHcZCgQ^`{D+^C6LZriSP+>sP(Et2Ln?%Vcy}*K2;f$-r4l zcIDa|zHb}7Qmid2(Qfm6i!Ql1Gi<1+`8W9Rr-9%DNUZwpT>4umz5FeVo~7c6Umi84f6%#!eg&?!8ip5z0Amqg+^W?Or-EDcgieP~t(M>0i#2;;w zfC~l7U9OiU4ea301@10-W}=-P+=Kr^=KXoaKL_L(00IsQy%PUTr>MQjVZRm@=9WgD zjy_Ojq6ePIPp${0xivR;s~rIE#nv|Y;op=#ejGOi(qi24soZ$!-0@FNT+{UgXV_3u z5mkq^^B^7ndi-@*FB-K7t#!KxTJV4w@ZB_z)>j9!Nqp=~FM)*AAM*3p0V=SsRxf~} z9tz+#nB}j@aBo~DQdV2lHdsjlZk-l@m?0wdIcotS50w)whFhWoa&XJlt@!}?!vZqC z6LEKyFVZ4!=cI6?jJWyYUW&d67nHCzf9eV4-A;ZrKCI*#3}o*p0I`92Qb~n>TNtxQoRE4o9v8dsA@LaCv?v|wVN&rR($sg z7Z1oRUl%2fXMB2*C>XfEK`MO$`%%tFCN$J3E&ZJVriiyOoHX+vDE6W+wcuzGT zjy)-ZwP8Ak4tjznP(ghUN6Ju+D*~v<-1)}~3%s@wD^hFNG;N6bF6vhOpjVgFE=G-! z;k9pjgb56EUp5;es@CtCBkud?%Tit-8zIb*2gxvy4WQ8>TIoX+hUmcuYD%cNqboIdlbg?aalveG_fz z)0WxNFT4>R=fik);05cupAQuBe&nxsedcgs4DgH|^sI!9{!encql8f5oZl18D^LEp zk8ht%Atl!86cZdeD0#6+x$I?e>LZNdG@TZ4K#z>T$%MIQ6@bg2s*J{D2Vw;N19vL) z`pAa|qnT))IDnGufM&kD;6--WpYN@mLYt)`z?!xiST^)U83?X@b{AU- zn8zPAoW4mTMyR&jF3QNc*C}j6hn*HM#Z%Jrc9L-cLI0tWZ=QiKlC*7eKtJtCl`1!% z>Erz^yz@eRWq|IeKbSUjx#M(eYwKzN)syS}CcUo4=TDzb!2l~!wm?*-drswZe}50U zB7(xA$Y<@Dwe%HkeWiCVQcF^ycHafiKoRz+u6E%H{Cjhi?XY;E$o3#Sgo|Mr2ck#9 zCzjN-mQ@Q+QyDZ^p464L)114&*B}CfUrO-My!@{b^e;X*5Ihn*y=UQk+tS}#jRNBi z|IEPuSqO*loCx5eodcUgulGfGK~?Q}hOQ?idQ^tU;<4y|fLJazw!Ht~#1f1D7Kvds z0|lr)sj^ybb-|1B>heqWBlGkS7KW0__$*55345SU$?tkGlLDfTy8)SB>oif%bL>z= zC#>yIfEMneY(qpudujjx-Q;MRj3^CP&q}m77mQ+EH!3}}{vB$`*56e4cX8!=4_Nj- zs^OB=%I|g&p`oGMDh0C<@VbvH(_C3WAyZ@X-`j3*P#&PAPv*0@`^Fr;q9eAv1RZ{G zV2vl*J-&5mCP?B%v?jLfk%{L4p6Re*?P;KLGE$fACOMr`3Z3tA#hVdhYEG%OnXt(UbbE=4GJ7^`r4w|ue<j)oE-oz%x3%ylpNEN# z#ig3yrh*U2v5IB1?PD5|)#EF3 z^Wc;19;012N=Qfmk58{!<9Ch#4G?n4p^qq-fU1(x!}+>n-_w?&pnV72Og1}rqw3dF z3G0TzNkA^YIuPF4Gqw=D{ab(}(Q^y}^O=+V4a|Z})v7x^OM8mZUhHcF?L9X*8`Joubx~@V+-JSjOA%nTwmfKl)rmaQCe6Etf+sn z&SZ49F%a_dJuKflQU*>=Cw0q>U-;{W;aE;7&Mo|gH{j;wPNMDeHsiaCUG2)YX$rm1 z@xL1nj4~!*u|kS@=VG9a!d(Kzv7ZAKMT9l)ah@f%n zi?`mMBHJZ1JBUfuFC7}nEf(6SbC`p@d!}Ui5+RR+WmISdN=Ocbu z+?PMH*;46l1L&Y+tEcj^a!IJ$#FL6)>?&L+k-o=Z4?NL)t&qo0ukpi~OW_^Fd z^fU9vLuKq$p!X9(B%v2wsfRjyD}~^7tY& zP^2m)eHQjek%ckv;`){y@BzzM^e_u;=XAiyt|$tqg?WY-|DjANyeE!gG1^fMqoSft zKPf6seJPHwb#H8?iC?MH)xC7x&54s`prb1~?MWvB!MaS2We(uuxu1UsVbPtdHmRjm z5`&eGZI2`qnLGDhS-O`lX6=FcREL_d(Y%aSBHqR{aO+YPIyMoP=Jj}M3mR_-;-LkR z19v16jU$!M(yo`Wm0}05;YOv2gas%kTbK$S7{+gWzOY%_OA#X4J?dYLpjK%&J>$X^ zTtFY6po1K_#9mO?bHRi)b!o>eq+!*JNuqQB#Mw6;mRoozjEP1)Gib zyS9|)UX%e(5(I&70MNPL`FZLsKms7B&}INR>80#f!TR5wvgYdY-?jvoP;uHNkEtv= zKHOc_25S^3VcN4ty{2>O3C00)URsdiuOKEy>N`-t@TqQKO_rgYbX&{5XJl!Q7S{C0 znS-YtS_AvX~V3ghI>v*}Aw;Y1k%(SX~$o~}79=GcPm@mg;qOm0qA zEqEOdZrDPXdlyKK;HE9Ze5{}^bf@;Q*LYK_Y4n({C1XeyN~&_|7o@$3Ht;o@92aB9 zK8i*YiSYR{yZgRr{SKycc}Qv5s=7WmJNsTHx?GWY@22!axxr=FGCa%*#;4x#CG+dI zD&;G&ALs!);_{j}PzJ6Z=9T7xA5h{gCyOi+4Yl;nn@1CPE`(IyoA(xVPdYOaYU!Y0 z9FaV-5m@rPqXlaH4In~?1AxO^joI~yLijxJw4DQ0P6w-yyBL-8WZci4i7a$d@w2I4 z2mU5({!{0*!CGQK0#+Ehp`n48so(8Xb$@ibm{-uP?)~C!=jUsl`sJm0qlvR{Fd!*f zCWZ|yoLzhub0~IhEyfd>F;UW5qY?$WoD{B?dza@UFf$A<+hIdbcy+495($6>ZDZ5- zhPZt-@P;^fKA$YvS?Vyk0@|-4M67eFxdqV<|HJfS< z{NuSeJDc7a$!>>1`Xr+jUxf(1vqopv8tD@%mAb$(iZe&}Vu1;e^?Hvm6V`lnI1k3( zGz_f+Y6(Yawz21xujj7dwZZsXhW>W|c>wsEwVtiU zK+0sh(f<~&ha?hd6B=c#@-^FLqr!m0Vw470P8X>q0#U&4g@@5qHtxV=& z*)T#MlPfFQL;f?z1&`>5a-ROC_5aaz6<|@eOB;~x?nb1gySuwX+NDdnK?J0`L6Gil zq+7bATe_wDfAM^AzH`p+<+az!;_kll&NI)vq5RjqccECcHUtyXai2Tvao=1j?=f!^IojCndLwq@C=`A+xyjs zcq&1CyB>{bFF0)clNs2ihWidAfpzN*jn)1K;s8jZ5KloAQ*c$Nu;mY}>-Q(^t6}Q? z2PPKXWjl}~(cC@PaBpr5>sdN;l1tZKiNh#6^LZqxA(-ULrTkLHEGT}_kaEK2o}E|m ze{~rc9zOJ=v8K`Cv7ah=we7O(DR+kKvQ+Q#t%~7tl7Mwm6q@&PpJ1PWyCCSPc;3#P z>S&Q_2I>0jyx_R;h^nu7Ga(^i2KS5|pf5S!w>@4h;>$X8!Jn4%!EC@RSQdY~S`>() zU-3h#47Y62Xu2=2c4n$f(y~fz=#os75h? zy7txRIco(Zw=tAmbdtyO2OdSz?{`c1t{8@2LeR4`d(0TCwd~7Sz%pyH!e)Vd+LiVN zLt#D;cxh29VdzM_PCK0?n(+2PxspbOi?mLxqetLh_sGs_-sC(9c#`m7FOb-hIO|lc z{_@h*RZbvk@xtdih)xx1Au10oZliVo42U#PemE=Fa>x(RL zv`S4@9=ILxPhgJgV_xhJMQ@5dkC{*;hK}cJb|@g9kWfTh8rQqW`8cy6Vfai3qOWvq z>Shv3r)$0TujB3vS?IcwsJIZFsb2Qi2sEk9uWhaFhQ#i-{gZUaMGI>DO#?V z%G)*kB*BLr)d8ol{ho?d3HZYLFSYl0YhO~|{#9Nppqg)qQCNyXW|JwYsnteS#2Lse z`)a+NHy=Gy$a#IIYtgRuZ@w{K>1zC|} z=nJy&nwR5}@#TvNL&)bhZoH|;Q@zD`UOnQt@$(fJuX%L+`1@=BVUT|n6VYP88jFJ! z#A^A%=l<(F{J#CC;$sU*zX7n9JM2y76)@M2Ca+!3X`xa7M!zwG+@4su zn0iA&VIm-Lvw92N>2h|cY4bum5@LhAnK#{2@lj@KV_3bBi#+KWtCYQcwmoB{wfIf5 z{#PlVOYC2Tcq<=#$#X>X#eF-ga0BUK3F)I@DPSn*1k%OP*G%k+6j zV!ZS|2CTd#0BHYt3fMaCU6!3^9kq#U=^)fog|h%m*{b+C0&6X>6uP|_m}YVkC;22i zoM(Y6o5_(wqmK2`SMP^CWylO_=Pk=SBLFINB^5or(yf)q7!A?CtRg^GiLl89#@z~5 zQ=)idjtjg1N4j=_k_Gvx^zAo=4wXPdV`O7O&Uel;>dW5*zSy6QrE}MO04gvuhf*xi zv-RXx)1Z<9Z+Q$-lpA>4_@97|;fIgIT~P2!t3R604xom-WB0DN{B4FB$bG!E)pOF( zDj<2E_s3ezG4018W{LNA^}D;=0cR0PQaRQYvTS-84a(ou8X%gXfHqdv-cqZx70roMNo#e z*m+x(kzWJEXq6Zt9a>}^6Biip$xo&+Z+^bh7+*I~Xr3if<|Yvr78a%(jzxFEq+3;l z&%d&7%fYmYNkf=l(!0SZVRpp5gVB?WYl@`DN&hXDZ~XY1tLpz}x#=t}2j3+OG4#|$ zk+zBqS)W1(>%`aD88j152TRY#hFWR1aD1Uf8vS+0p~YAs9p0GY%lu?mY2AAOis0p% zDP8!h5}=(j5T_F%RzlBlJ{ zpg(oML7SFHB}Y%PI3>gI9(H!(T`8=u4}iW(Qn4UkQvPpfjC@TWSV_ygoHS^7bS|r% zjo&DFdGASlNSI6*`cB6-56;I-K3-qnyJ4^k$g&5Z`3%W2w${=~xY2l3^uzk%^zGEh z@KGo`6q5XD^1p2V$CK-qe7TKVy6JNwqk#b4<_BHGp-vx?Z+RX@r8~xxD^b#(k*ob_ z17|OhzT+Bs*k0F25Wf#6hJ4GBD+*u0tMLo!mw`bggTGw;A5YfMU%)07v?4<2BKl%& zJa1yUPAZd+CkL+BxV4OGnwlitTxLuwbdLfnufr>_`xK*Hwz+IM(#4)zoOZ{J25-(b z?QmaO0n(o(-~Cw){U`eVIt17hU;Aq|=%ObB@kZoIZ?t&%8Igda^gL1W@!Ix#JPnDu zni}DEH^#nXhANo%cXsiEBxrP9f<1`sx%v6|jG!A7psr-BA%8&o^$A7F$0_um&ECHt zQ!&yNNYm!tw9193R4|DxYq{@|fWX3umnu)(J6rO%G&T^+&IVvl#wJ_&Qg`AO|9yP2+ zc0is)GN%wkT`m9E1}GM@!o2ZeFubbmQ z=Asf(R{ngQS^@|8bW$!v>TM;NrePxeMyZMk^=7}qU&v;yq&5d)k%}AN`S*_ca^s`H zfCYj9rv0u*;&;D#dvi1W5uDEH+}Nr&%`67NYX?x{J{xxc!^cm*Ccf!Sqd0S!2J-zy zfFNqQ&ONBf`GdsEX_sX}A7Ls4^6AB7A zDx&!-5+uDO?vo+PT7qaVOH2dMU7SRxZrJ^__IaJqh=|9M>wSH^Y(wcb2B<;O*-iSw z?NluuZ1E5Lz+X+G0AP=*ZiDS3j@&7KNLjqqFYNNcv1QBxW5CDkaRA>vPDK%?cnNEN z;`@KxEr6{6)D_MkZ??B8UmsF|qcGGoG>BOMlxEC}t=2rqc#o^E65Y#QUVcP7Ffh9B?$ zpRq7*V&4xX66pYc&S~rR!?_#K5X16<+aT1c^r*Hq9)_~w;$}Ml53*W9%{2bUqWmjR`>Rm`aErRx zO28)ac8?E&qXRymb0a1c2@@p-s1N->Ca?k(fDDuuuL=}0nyY&wih+y-z&IEy-5g_(Qd0jQrx&&(3 zHGLpj5E%ENYlHeaVKpXAro(x*)8+om^+-CmqfPSu!vDK}q5__X?61ZeLg{zwiN}v@ zCVoWrBJ9eTTUnmc1z;fJ+*!A10tm)Zpzha_mL8P27Aug#N3X=B0SEy6%#E}UT9ISx z5~D{5IB&gdUs_rCKW07uT15YR;-3iEZiU5%hc^V$jd~lgSz@(8+FhY0CUWBTXOla9M%}yvcz}98e`#$J6I*Z`c7};Nv8BdSKY#WN-tS)4$qn|>bU|HluaKfb^cYq|Fp1EtwELjlkRJxpHV2{jmx zZ1q}wA7d6^C@uxUrvtIDumTEzeDF=8i<81H*tr0GRu-0FSWeL;ABj~0J-qUS$H6r0 zOZxwn;H)VF!H=KE0sh_>av)&&Rre{9T-QYMn(On4lw}+KAH<&WK3@JJpbyEXfi@;D zkG?{}UUGNckg%S(5H#^HaJ2$V*zg?md``KREI|9;3svX=z)zzDO)i0GiyPQtf+-NS z?ui{#v8AMN+hut=W%PlSwEPXF472W5!qu^QOdq=sH7EP132eJb%k;4QWbV?~9V09*AZRp=;^`UA{OqF`n>NdS69D~)~^+fn{*QT=7x1c0!X_3CnfZk!EJ z>`(LvN+H5RJ39Pd>0a+m?GyqXzBD`^9*G->dK93X=Fd%7S0+=V{HK5YD=7G50^lBB z;2U17pzpNvg4O%(Jpg2cLi5sggEz6Y3eV1M`f>_L; zgJw#~?-NUejR74pz~;(C%#o&tlM4}=BJ`4@t7H>)5iS)Z=j>l< zRQN9<^1m)H>SZsQ2vMDjK$ju_TKbREg~?1tT~HA}AkbcS>E#}~5|959!=SAEZhE@#TfDW{l=bM25Q(?FA&7J2q+N_Cv z)U>zP^7I|BSY(*RIDbq}>#1mIHL__}6|VN6Roa0wog^|Y4WCjlbsI=J7MGOl?5RBT68SC?kz!Bnu#M`G0@Y6cC#FI{2m$mZI(CYxe*bA4+Z1lI35bk zXsz?x1UK$iZZ*6S%Tr&m!s2Ct*U@Qdi~b4g{bzXGa2OA-TkyJ=c9O~30BRu4OL^?Y zWsM~LUzeD*1=%+x2L7VdTT)vhOiSa+lLLAws-R+3*R13t$EQu~hbw1@=x_)c;y> z{$d#RHG_xSbLjkv`!5tSXS`fQaJf@argLJD_j_OA>7(Ccv>dw?{_wk;uo7KP1krh#YK_LF)?-|(|C~khF zs@IHT+A;@-Xq=&vFppA(ON|cgMtTj#3q-+H2lX)QjqvU-?GqqfZ0O!UPfbl70>H)U zE_HC)!2YQ24%wD5cAgk5-LS3>p0OM`|xR9-bMNiX_|CPT9slYbQo4Pn!OQ)2o zov1;(6&rxl@cDdrheDot2C2b8B8|M6M-xt7{59^?9oS_51A^%J0U(tg`i(KFQ5CDm zVEk?_1Nj&6U>ycak81~N1L4Ei*}P|@9hO)8q#v8xfBMqt8~>_L{AfpkOIJr8oc-G) z^H*-uz!oUr58wVCQVY(9n*p)^GcN>%t^1J9`SX0|(poojr{f@!{;xr*Ow5|phVRc5 zv#=TDhPT}{05flUTQ!4Bq2(xsU+G*t{7myEsMAY-mi!;Yb63TnH?s>Deeu6@t;j_CI zmo0e|O!?)zoxn|#O>QDQ&dm?3)ydsh2iF~0Jk@no?Gf8wrF^X@PV{HYi{Tj z(rTIt^C-ih>B}}c&F>9O$cQ-@XA>%}6!DMx2=m7mu5B38`1y;BDHKuVyED^%NVv_U^*02!52Xw;?j7 zQTK`z>IPazf2Q0G7|B9@ml1PuZta9A;Gi+sJKCFtl{9UOJ~g&H)&yb^mlaSgDg$y` zy|*hlD!Pnv89uT3X&d878PQY~Qe>c~Ro@C{VG{yWgyO)3;?8B+z#SXw>c*1a z@SKfSbLFNWHRdL*zOVjCYZa3TpNceF@G{)b_oiow6s*DwK}rvdK$G!ID|Fe7TEZo+ zV?j5{+9>>;!PhIhO=neKf;l~0S2`DOy$}(b)cHql_&T$DMPc~_w_3b?)*oX+yIx@o z#F_FfXpVyLRV*%-f@vJHSGV}Yr8QKG`7*No_?CjiRe=kD3=K%fALPNNY|w)iTX0)4 z9{5$vBsl>U$COb6rb}M4{~GSu-2bsPV zLc*ba%<%p(R-tUtJKU78>uI`eY0|3!Z6^ZK=WO_NlYHxqQvLK+czz(XZn=_hG?FnU z!hvr;crrRcfA@M<*K`|146lhk_!R-1UYw&IL5W&JAKJQYRNUpkaT=Pv;34QcsB%He zjGXoHprIY%J=qx1s{&F_MSR)4m zzo><%e?hpuM(9M({asT(W?5OCND*i2fo#NnM)N_dLo0WMipr7Zn`h zS=^A8LlE^WLr>BAoten05s3mi@UA!f$;^RF-#=!OND^ohpZY*6iE5S4v3fG;ukjw>S_^RQ} zyzi$xfZUS_sF>*Q7g9oDd%}e9>3Uo@Q+Xj&A5eL?VK4g-eKU{r>K%cZ96P_X>jen` za@1EyO9RAsqx4Hf^Md_<@rp=wAlPx&@Fp}A&OFHCj=;FSVL*j37M>GTuMXj3vnmTp zfVlLRP7RZ<@2gK8F1$>h|pA<>=*ci{kJC5%5r;3;&1FrstfN z%pfPdIf#(}qse5sZ1jYPGC70+UiMw;#6i!&q3UtZn_TbDSpdKC(TTt}T@jXuEoaM@ zXMm3M9j@btT|lBq5_aq~gYFN)DBQZ!>Cn#CcB#*kx0uS0ctm$Y&A$;67-E&mn1p6s z`Y=7R{LTGbuWgI@dcUNiK0%b098l-gIUAH5k(iIQrj!@-SCG`D6GG74#HilV{1|fQ z%eKk<*^{NeDQT!JU}FalS#OlKLgVvTxYdJE*t4VO(5mew_@>Qq>{<(L3$`s!{{+_g zNU%rF$?DT4l52ZE^1&=(R4;FGU0pKUDy>_{`8Ip##=PmunRPu} z23pl@u6xY=T8Vl?>IGxEKF62{*AD@4=)LPucddY2yX(vrRYPwj^{kC>UXNS*IIGGk zI`0b3%&&h^)5M1760eXX2_tia)nZ&AEl!DT+x78@qSeK+^z1W(%J9O8Dv&>W^Hkw$ z48a<1Y#V<}GBY!4XY;t&HF=ryWUC!k=0c>a2eVou~ zh~f#3Fd-)In8#S#V!A{+K7a{Ksu2n>V7b2ma1D(8yYg+rWmotJX;C}$<80$>7_UZc-RTgZeSnt2SXMlE_ovM6gJ&_Y?2OD1# zJ%e~!+Z?=k3(|T^Fd(xrFDN3{Qqzt;-}T8nH08?`XO99B?bv!NQB6X}I->UCGZ%b9 zfx81Z%XtdQHTKrcS_@VE+~Nc01bsS7&GPp&L7VJ5*H?AJ!-8P*wB`h^g|9=qk(|ui zhnFHGveIHy-_Xdj>+pP-w)LB1A18JeQb?ri>?2>U4auwMTX5QEMU~Zy@aACXjZDCK z5t&w~nnXxiMjwh7fgwkk34tRNlGbPyfLltH8=kQdizHR|wLJ;3CuWw~V_ z{5a@6j7##wLxTs*7x*^8&qoKBSBtW&9ql3@mj0Hx8yhqU$Mz8O;X4Xei==ENY0ThW zC-c2`$CAF1+`Af11b67%+SL}r8V_H>k;On>iY6p+sJu7QYSwphm|B_82u!ZQya) z8*baS9My9csa*9!t_CM;>}2yxW&U~6i3Q?+8_e6T-99U%ykdF*?gNB7_*r+d&PUTjvC0%pGFj=2=`p0AN z#x;iL?97$kga#~=&b@jUS}ueUedY_DLL&fEmrK|)=qqAIabEmI6jZ&~07bw2HRX|! zG^}&bxFoym;?81opgp2)@26QujA1~;T-|%RSx(-$!r$J5jI1i}{(@gK4#yhH(fUir zy;%-7()|KZ)c8O*(%*JG<7h`Hg>>L`)g1(A-qS<%)-e%WOmD6PDVlzr`6lLWUE~BLE z-%7ELM99PIvZeuz8;*jx?=wM^rVqqp{z1@$@jTq8-@VK#^%73rj4YFhJz2%|~^4b@QLm zl|7(&9axSmfar9KXCJG`5HT;~Nfq|n2nvy^Htv3a5m4k}MdSp)HRx7Dj9S2J4rmaU zR0rW~+iQz@gh&UDm2BUnBPecKqEHU2hNj>e?#GLvRwiuJZ}hS8PLr*|&WENM-y3?R zG5cW7{<2qjJRH8XZa-ZNebb_Rh1PU}6D_hOtG~!-^KZ(u=zYlTR~;I{yha>tqP(l* ztOg@|b;iW~%;w>(7D``!dNc2bYCuJ4>qJ>zSf6@-5ipGwnH7@ z9xx^dgeKc`qrQU{*X>hrAosw{2%}XZ9~$i(6>zAW0%FH6)-}Tc|1k?|Nh*R|hJrr) zs5wI7;#5ao02sH;BRpy(da~vlBg4K;o^ZQVfL0r0PxY? zSf%t?+x)h(-{kYEj*`-v>2Vb<#k2)ED!;$sU7UNm(Or{axNGjk=8l9x`H>IqS^)Q4 z1;Di751{Ki!V$d>^b|l@)~j?o+Ys+`q8DuVit{%RvxWgd0@H6qS8?KED3|!OiGE>q z!BRc521=Z7xW>o6_qM|0K}!K|!>XQ{bEj7B^*O0A-ej!9QI02`o+oHslzPW$UzvuW znN5KoQnOB0k_L<-7-VW^_%ao>xe`Yjo`D7>M#q-iw^tBaY{5xKDctReN{e)n{vqO; z<+3>mqEge)%nxuKrgA>&y=X$JW;AqW13Ubx7^W$j+O{Nk4LM2?$fl1{i<46PH^+ko0?}?~%EoE41%(7oaryRtD;;jWt ztPJ?{>ja6EsJY`^8;9Y+N0%i&S=kT0F|O)G$37|99sOMABeRaKh&+*NU6lskLa~V# zack2J_n0voJUccQ3ER8Cq2DgCg|6p;*2^@RfWo(b1V^ zp}C_yA%m_=C8MlzFENFG%JW2&`2@J@9I)3t}Ohn z>F4&g%Ua)}!kh4u4Cc;jin8xQ`X{C|>Nllc(p!|E=$bOt0f6CaTGB6wf`2U7koRv3 zDEc)7=hR7_IsE>|VXtu0pw z@?>SZ%?6kc^`ddg43=7vM*ZFc8bt@_VGIH^y1F6)+U;6E7RrV3FP71RDioAga9g4jt*Wm6wZAu2P~lLCz)lI#M%p2tGc4eJH%Ah3 z#cDw{e3mhSYu18s|AaD6f>{6E0`LAPX@*f-_=&Q=y@&KU_JWpfZg!ePLrwaq3u(;J zs6I9G`pU6=LPb}jq1aq6>t5Wy<0=lntR#T=b>*(!kMP#Mf`dDutb-zALO&7P_>ONG zw;@_su!o%xjl=p88Z*LCXw<%$KaIEJpcC_u3Rw^MIQ()y@%rl@ouuxt*enj(*I&>uzh&Kv*EU$r_KjE&2T{EB7qS?%< znYKhL-Y3p&yBl_G6-yg;fAPS=L@&n3w7QyO4Yq(rX~SfA}SBCJC)?` z7`?S1J-I_Jg>EqTtl;l>7K%kzZkBflz2m78M!qHp!KYTQ?(AIm$lS!AnR9fDXcHx^ z-_Sd}{N=HAem3NY6ZHAKc@!4~?9=+RXg7LCT$DHl%>KY{$)AVyth9!2+R##jE&A+D zdJ$DvjYr1mElbJ!C9;bO(QN|1E6d6_-hR4c5!{A1ZP&CWTEJVfHN7Xr57mIV7Ox4R z3TB1vcp_fpl_Tx5hiBQGZ{#Sb&97!25GTSnjheo}1@5qi~EvN;<0Bx4fhv&B~hL zc!*M~3IXCPzP^~OyX(jJJ|Kkw;+kj=fh>Vbj+Ei>$5Kz-WXU*9q*KK`Y0Ws zFe2u|o9>L|Hk_r&w>>ER{j9rbdBHoQqd!Yem-JCfm>5qdz|}#9}NM%6_tr$W*LNFm)mw zeadxPwpQqfgrM|<@)=1W01#_W7k_D(6xHMro_JTKe3sgUev$||mgn>%Kv9h_e=b+6 z*iFMoTC;&tTVyV_HL3E4+^w@#$%WiC2lY zTqEDw0~o;&pBg8+>=9mwM`dm4=R$!aZ&oEMTD>oMmS*c^{7w!IYs!q%D0}PPe)GsvjU0Hzcco?i|g)ggbwD ze8~lcTJiEt2cTqv=BREkGa=()1>eCn4py(|CgGMjyPr&PV1}?|N^vnj_@m+i6=DG} zokH8#OEX&!$x0^v;Bq0JewKOBW#+MdK?mKd^~&2?_sD8t!F##Bz0hL8tRchuPQfn< zpdhzA{m`|myCgbSxp)%Loqcww4lA;`T5+t1y)&uR8@77g5l16#P6Brdiv)@-j|Nlx zh#vFqbq57Yw9_m9TnL}nC68QYlYKQ9;E)d(q~OPy3}$aXfKy7pMeQ6(J-cZQ!Jrb* z`bpU4YOu1iM;>=5QcvhVOmg;^tp^4U$%j_i{$&Ek!DV`o=ipVD0x;R=imEarUeR{J z(qM|iAv1P7#Mw9zbJjsN+iqgBet|=M_r|tA{l30#{IY((!mZw_^#Yo*hT!6hN`>|b z62K0#8E0XxhYT&_Pbohf@8oTB-LU&XGQGY`az>fFS195Y4(aw8W=woa+OI_j-U17K zjf{vZHNuY$TrhUVCw)rYtnd>U#0p$A*!5^ED_Lc*ABZZmRO~xR0$)mQ%9`VKVF&UU zRIS8Kpp1^Z68b@RP*6~ePSCdW7-JV4nh>cHgr=T@cnP3OsHoAuK1zVK+x&z-FLBaY zW|M~N!guBqx|qOYkg~sk!+lltTNcpSxxc z!x>Ge}U*d*Vk+Y?uYJzG!SJNVhHos_;3>*q3^x_v3i zc9L4{n`++|6+I$~!Wi)XvKpbjvpTg)obc}Kz8X_ac+gM$`ZgHt-%i&Ayu=F|@GDlP z0?7_Ypf!7Uw$U3lHdET!3^cj~C7y>K@=lY9qV0Pfl@vPi~ZUStDa{9sZ zOcMkIteCRU$!7iQ@5JKLC&6{LH+wQ(nPrTd-qum@hyVO2RO-(_82MP~IAM&LEa>Y~ zhE9=Ge8PUmQs)btm-A3*>_jb=Iie3&l$J!V{UNv9x5hXzc5RIMFib`pz8E#2iyaNN z$PazF1%0jAX3NK0c{{`->bAJeajhuYB@9II=P8)Vs5fOi6OvVv-J>@S3MjzYVW>hK z5kmQbo6e5N`=ow;p8gV)oh6rB$#lS|3v5fzuORB@p~vo`lrTT^btI36jL=n@+edjD z$tYl8U^co|TmoS`l@DK%?5P$c>gPtk5wKZLiGX_7MbF@sZi5K7q&?z4Pbfsi_HUS0 zLSPId*}16pPDcQT)PrSuwFHH(j(Vyi1Q%yO=kOu0Oup>(&3paK%=7n{=F`?A_gGXG zgh7Q@&%m&ldv?m+_EkW+5E>qp511FT%x{I2xU{T{Uxozall0SdLI7!(>(7sSx#Uk$ zeu9rE6NVGhyKv?Vrz-%n`?|y|ie9VI5FlO3dtZ3qMH24?d}d|OsR3r(mfh?DeH&kX zKub`zZq8pO2Ifx+&-SCIujHMHv8MrX7!eoQXZc+DCfIL2()9JxB`X&X?({#l2%6GE z`k|HA+{@HvwFtt9&YkEr=J`J}^?*I#&fUMy02oAB#+;1dS}U+m?O@O3mp zXg9aM&_oF(Lx8s()?l;f658w{1yC7@30O=A1kesa)Xljm=?Lr;4?{>uNI1ovX08BL zJTP`cRDB^h`xP5H_`jWN`NuKphA80ZYo5`Sulyh2<$!4r7>_iq@(wq9xn zLUfdY>$3AQ{1ke5Dm9bi_OP=K{`;<2N?8xf=XZC}#>E1;ZsNelz08W^se1drO+W^i z0C!V}4s}l65FoP>Bv?N(9z9qAbWUt_O-)}P;MgRZ(y%EE>8D2l_OUBD4Fdo|ggv$0JCG5Mtj1leR0awPZd zsNZo8NI~jT7ST{i!HNBaa}i>&%xm%bWvZVl0fHTPng!wfm!Mh`{LFr=)$wX}GxKyF zpq=1Rg}&YpLCF2FBm+>q}b zM%9kYe5C*@20N{pyPUyNci!R1R$2~5_{)^9Ojj7Rf8W^i=JF_hZzyLfEYLVsRu(*i~J!U=ucUWAR6!hy%VcoM7&9k2mV}`-*@{T z4?>NQ-@9k@bi#zr;DlHI8F=w_z|0ss7J3DrX?$OyG_d8*z`nQg64c3q3pW1lDJ9u z==N^Wp!I6O>UuuO5}O?{Lz1)$A_Bpf-NW@5WBu(UJK#d3!w{E1TzTE;O47O2eE+HI zsa*yxzr&5)DtrSsP~dCMPhrE-a%P1yyxyTu(geK)jl$U$k})H}QM_D-djyElo|=CJ>fYC>doTL~7Y{;)2F9aMnnbS-L|XXC2@7 znO4v1HszWwyBXl*XSVz+5>$9#!;{WpzKUs9EvRf?(7W8EgvHmJ zn-{_zZTuJBGcL1n4_qpoPa1lmV!F@W-fARcQ%urY8<+7)R_jO6# zw~?AVp-ntpId+1OeJ_b1|_{&~mluBYZ}?m&LUnnw+%ipA~T zgXNOhtjsYZWixvH2<1KmW$>5V)#2%nK0-`f+OubAb0jDn9}SO)&sJF1clsfaA?nu^ zAQJC4*JykU4*dA{4&bK=L2@9L_)`{#B)3V+jHrck{p{)yhYsVIamq$Ed`E|#Sx~e} z*vO{tWLB$Gi>Tf1-d0)b19#FdMgAM_B&yw950~Y%sp$sB9a*_7AMxvvDPl7lUrwLJ zW8II_=wJ3iGsgC>)aGEY1F%3w_w@6vEWk@?GqYH>VVWj*PNMzY0)k8a4kJA4Z~ zDPP^#CGg&M8~xrRR-xjl`GL*xeW-yB`OG!K2XF%@iAcg*yofDEOcGkWC&^x@$NTGw z)6491!hhdZA~mTLRnki;bC)wWuWd3h`uAL_KLok7$QO2VSHdvwut+>qB>^~gRE{!s~`D(9BE{uCX^Es z!wrnu1d=2l7+J4=(1YG$e_|QqN{I)#zlbu#+G3_mT8*AZ(JPy_W6O97aU`e&d8%+CRM59?Wo|5=TDg~=RrP;I8hqZOYv33 z@o&M8)6cOvsZ=vCHg*Vpl(sn%GcNu(L~l6;oqMcalHzbKeS>cGLDsEd$(X7daSqo* zRRVU!y&sA2mT49^5U{2+Ou%fbJQDAViJi+zK+~W;W|d#LoG&J3*!2nS`K2zN!QXtu zCm`^Br=s#f8c5Jq)I}b!y>BKomusItEB%oSzNCexB5MacAq*%+tR~CYY45FTI+llV zT|Gy)FX^Q%446`^K{n^jw{3Pq=8ZHZYJ&$?&zYkZn+evp30aZvS?CxMZQSwFxCMoc zh6db}54ml29wLU~TC+TU;K+V|gDtp_==S8bbalXaQhylqSl&9Y zwRza|T;KJjBBVE}p7P7;*rHE6^b~*a^2JaI^9M(UeMmI`&=P_P0+FpH`GEqSr46EQ z^?=dlc8frX)f`XMt83@0%#*LCp-WT0dbymK z^JS6N-}lJr`+we}Xc+&#gNpVi9AGlFuhzlG)okrf9833jI(E^eHm3k|VgK9n9nL~s z6y4nlum!6n#&L9-M;_PX2S>_ot6JYDVBYFeE{YV#J_poan;)3JDMa0Zg%z=xZhHUR zW(~vaF1RCR;%@!oHBCJ&bqZ8-Z^^f$TuZV(7HQUB^(D%hzdV9k@dSXKabg}bt>}x^ zgSqa;ts^ipY}T-Or!vjQon6AVTs*@MJ{k;Vw_}_h@)u>+dnS!t9dCAohyGNkCp$`! zei~*0&HFMk%*+IM7;p^i*3x)0hy7t zxAewd_Dy;0T9gIIO;IaUv?Eb)BAB=#ibU|~0hlayJ#O%7$!;tFKxhF(u*>_-{%5RJ zc~jq0Y1F@GS(I@Lv{z%XUFd7|E;q&W#Iq94k~Ck`^m0#mZ*6Kmlwc~;D*PG;jd3#i zgujnQviX(ts6W-jErxtOQdwerLO5!Ka+zbp%~7%mI$G-Y*iuz>?tkXV;6*X!Hiq z1pdv}f5CKUW+`7QDcAd7x8ul-PH?{I?B{BW6ZZX3P)MlWlSfBXecrwyN5!>QJ9JtD z_mpF!`=jY~D}IjY4#+?XK!t-%v&`Q>++vR>8NF5JRu4Ia(jj;dOuhAd-?rFs`H=!u zF=#dW(~V}G+GwYTK3^${RNHKsZo~EEQlsYJ=@0-dS^M5e{vA(YGyNDCs`V#E#HVcF z;v7Q1gj!*xgqB5F`)UJ6!yl#4a`Lv(`#cYT?$cK57&!>DZuHJyX|K4JyBY=cdkgp5 zjVSexb;C4qHF>Ww=^L%?v9gGJ3-+-L1Sx)Bw}U4TtD{ z77$;H7BWT-mhluT3?1RA_F1U$}Mx^Z^5B%5EfgR*QU=K++Ix0DDJ?VTp$5)LI=~C-MH6 z0V|-43*Z+qCT^%Ekz;5mn#r)Vx|~Be$ep65C075UF?uwf$uFx!BAd3H_|h81JVkX( zsaJ`OQhjuM6_VwaRL7+k4XW zC~L72rb@B-w%wSdwwV*eyNCx5^1nXbw@lvhV8&L72EHl9NG>w0663(J%j{LhG1Q^)yXR*TthHp z`eVr8p~4;q;9aI2bcy|9R_i%3bU&YQ^@7m9+#4A834eg@ar>qokGIRUKKbpX=Mgn)DrJ|3v`TK2(== zkM$$cQ@~E+aE*jl&L?-UcydGQkKRwSCG8*lHH-~9wg>x`j>L`WeYAFJ8aFLu19bIS z|Lh%W z^p1`|T~vy|(dR{|4f`t8s($BdupOn#tQqUN#2HDqA;7NdkL@DrxiDE#Pnbh2-E#CG0(x9q#o}Cc$ zr_lU?34hFjVpQzVHV$VP|+vbuAYIx*)UPMp$cW*iXW`KteXcREdmHc?>A6V?MR}*ku1f-*l z0N$BGWXZFvEo(Gb#^hi-)^IQgY6S*(U#OB+)lP=w6!8*UMb^~RpyU(oe675gnnd{fYzK(k0Qk3* zv(_vH-dnOu!amzqK}d&Eve{I21whI`>)A8PYh7M+yK6ABNC_L7FxKdOE$R+4|7%K=nn2n#%wuJiA8&{J)18Y{tbCr2CpmR4nr-*wsX{w2mDLKX zMBxBLcZ9q{Wzy;1sf$85uv~2r84BLm*icjeUB+BjEkCLLV>+;EaSI5DASpnEIq`HF z(!~J>{wDKP&*vt&=8wmo$Dek2o=!0{gml=M%}Mo(CZDum?x1M2g1LGvvG=$c+u1FO z`I42VGda!N{RJR7etk)-h&=rxvG^JpKCZ9KHyItoA1#S?W@I>@fG|JSAJ} z3*MbNeM^7+yokwD=u)Ioj&+lx0N@ok1IH)#>&`WR^7Fj9Vanh3SX4UgvbK z?XwNdRaYSqM?GE}^5sp2-iwt-ptBGPA<{locAo4Lo*;KA=$X`V67UhFh}4shQ@*vi z$PR+y`A<)StfIXdl|O||0gXqzgJd02vcI0Y{9)m^XRnx!YB(OKYqT57S7+4~sAIlb zEU_J0bC4Z96}BE2fiA~tvlA!~7z+CwB5ZL{3D2j_IwI6{jw;Oc%BvmR9-{a+Lq`|F z>^wn2#KDBEZ=%kzGP%$EqEJz6t7xQ8Gg7i#Qw z)PenobJ!^va7R`6uvt)p(=p+fxxNYT&r;|<5i7R~0Vk_3+QB&+cgQUB0(?uqS>+_V zh4+;p^HLb+J*{;UAq`rstCE8pT{@Ft;N*3Lgj|m0AVXF^R$2EigXnnh)B`-hnkhN( zc$&dV#JBYQ$zsqOuuZ+4+A-=$@#~M$_1!2{7I%FQols|a$%3EC#6if0^nG{ zNMKhOXFa{0t*r7OP3(N5edH5>c5(+QAG!+q3HU3;h)DKWfydF?f~U|K{YvOLo=_1I z@G{Is{Si1=OC{u00U@~_ts;A<6R5i0E_C@1#-Vdue znMnyT5J}y5@?3#IiROgj^7tITWr>%7v`k|nfYJCpv02!#jIH&uOZW?6Je`7TAFprh zABOMF-$2{d#xD)U+cgw=$ZQz`qQ+VqINe51^9+aa;&tMclE|bsw--3>#FSmuH{OWz zJ94~|$2+DY9B+?LnPMljZkrZM4HBuQ+6BTHoF7UE;i&}5iF`~>WWt7s)Zd-7|? z;2nqg&fzKZ^5y7YBjWE5V~ED=Wt`-m+90+IU>J}#JKP15-bVpZd&4h}CunqP($xiP zOsEUjp5u`Ad3X#^1NYI|nd8K~iD5;S;+@2GK|{k5Af)xAs?qRab`JrawGQ1G*c2FaSNz5eS|3KkQ}B_jcB-z zVa(2>S93IC4Bzoi5uRJ<0ru?r-G{@`6Y?@n^*l*mw$I7IT3$@ShwOT_5q<}|W+1^n zK<1ew==KcXgRbjr{WVXgveVUc2qt@mO#Edu96uF z?2te5cE)6C>ody5omSVSbVx&xccsqcbJJL(9~7r zqoZW7tGf&D&SY$TYfse7*^@K`EeZEZ;u0@skPcnUddGm9qQHpP0<47*S@)wrd=KiR zXY|)_>AvG;?wp95DVE~k%|#c8gNJ+ZlTNF8GQaCCJ!Ja{v**^!XtjsOHsFnXwiOZn z{WpCWq-m7_G>!t4a*JN9@l8XFP|UB_cu8%Il@=|bGB7Uz>#e~fr>xA`wFfh$c_iD5 z`D0_`1O3Ltn;A8F#B-dnP>KP6nE;UTOymbD_J6fpAn3>FxyaXIv%zWPo=^?NrS9;R zc}DqH>jqWclJn=8k^&%pu?vgE7in@au`ei~pQ5K^i3El^YK{US@6?y=5y~)Ba`InP2v@%$hjQfYq*Dbl0Ej% zaiRvFcF~^*$Q^)ybxKPR;FA%RzX6300<04}YQ*^kF+sKRhIseP0C+s#ILtU0DWXKc z#^*Bw&Ywu*%&ELQg%y*fb6xZBS;XdI&P?VWz0Hx19pO;jXh|%mZ#CUI3!zjAU1!w^ULX@M#1z7p+Y>_ z=y+PUeq;Tf!}`i|JHoHz8;}O^-t(^2$i?y^modbkg)1-Q<_A~*h{&SeK9hR#Z&zGX zA^;WVO6Ay`{cxFCTeJI^uX{I^@1fpOeg|vO0nn$D`YmXv=ZT>;Vn3#JTE1Fiqq&KH zqZ`!1cMArIzor00jt|BzBbMaj#d%_y&+$EGo`nB#!qIwG0IZH4fu|rKFUf*4e`};X zJ$LSL?-t0awD*IP$2u+AUsZCeC8w1VQN)Y1fF7{B@wo9L*_XVK3J zq05_I`bcEaNYq$gwo?WKZC7w0PwZ@wIuf-!=Bl^HW@P|x>5umt{J4~@lwPk8x``M( zO4~GJN%UH*|5ANA(rdg>LmwrIC-_3+Qq&6^(y82P2QFgAEe+!W89YJT>-W7+Qr_>i zfr0J_N&-$&BhxLwipy9J_Px-v*&WmKssf5AhVW@UsE%xpq$5~X;RmSP+(fM+Ztk0P_AOxtI(&dt?S&?8lrow-5DMM5(#i^jW5udywWOqBW}ZU`Ha$ zJ9w|jF9EU%F2JS`oI4M6*Zs=&drqZl`*ggHN--S$dd)l_r< z7d*=G6S(;JQPspT-!>iTz^+Raq-z8MyH0)ogAK#p!ME{NZquYO6<(g8i{BW+74~j~ zOD75k&yUEGm?H6fO4*Fi7j<-&TpSzU_qGt|cOTE$$QW0?zmbw3B8(ClG@RT=32I`u zqCc;B)~jdFWpw~ zAmyIZ=Ad; z<>EQoVu?^z;9TBv>>)TDx6Wo=C1Q9+6$qBOxEM4B5M4gY`Q_`MdiX!{1lUgujNqnx ztE)!%WK0)v*(1s%eCr>Ct~_r0@Zkx<24kNGh*Yz{aj^bl#v45;0zJ><7I>MQu{-qQ3sfZJpCq%57Kc?gDaK;uAnvkv5{1 z5&vt>etn-;K*HA~u&b`akoJ{SSm~ONBZ$tQ`0$))P5QAuU8$V{~I`Y%~#2tT% z&-h~;Vn=!X>DtFp@UKcYwxhL<3W zc+AI9L4u1f_J~%NodKLryz$!YKLweYgu|B(b@CXqvein|{kg1KD73Z<8fj7$q43!-cdU$*j3 z?9iVs{+V_DYkJX;H)mERsU5t)Pzfkrg5rrQHTwY|^$?L?Sh(NNN1O_m*!I?;Gu!OS zl3Pv%JYI3!#Fng|+@QaP6ZoF^)+s?}aoG?ce=~TXOXHp?NEGw38Psjxf}k8eeP@Es zjnru3aQq8m>U?+7yL=ChGo^3r7L{SUo5z<`xt>@ zc#Ro3u4QY0gHm|nQp~d9Np^lyq=abPs|BCW>T|<^9qZbY7 zuedKPtThvqYI(Ly&iluB>+OwpjB+>YNL44UpZy`K!jw!q>o<|d!y#dJ1bxI^yQwEI zxwHqB2HXB1&23?Gb`?cgqpwGgi(f`%sYg~2%H6zcdL)hYtbkS$2JS{Tl)?n{Lgapo zNF)&J1A5?8ft`=x*#mHb3$8i{%?=}@$4|M|>B(gcR{_ju=y?0n_63t6vVu=-W3pxf zDGiIkH!SEw=g|BCPp-&CU~Qy->xxW) z4$~){>)x5oZjZ^w36U!_g2$``pmbsY>=2ZAEU5~j#d`!5Jlg|tL>Su@)yjJPwv2t9 zwhT6Ag|l8*v;WV9AgZDQT=7xZEAYLDC;_n3u8!K;1Ef7qS!2$C(%5!zzDr%a_5=a# zMUImw>epM~0-`zH_}kZ3*sJdZf&=E`Qk|MF8*;LJXQv9w14S4j)CC759pIPXLLycP8>W$lc(VX8cPCf0~^@ z8eny%16ss^qz0xDtmf*5?Lu0;d#^=sqNOqmxc8%r!n#D87((@+o?3KG>!W}ca zT`6aZ|1o*kSl1|mun0T*c-KDxmT#Fsy;t%>xy)Q;Lrp=m))n;v4WV10rz3ly71P=PS||Rck961r+!5>mguNFahIgUR*uFSA@76(1V235^lh_?po9Co*LOPZ!{{KK8>iu~^Fb zbPUl^+&RPCT=n6cpeo2aJ9CeWKmW(B`$Zi**`T`m5-hj$cNzmb0#%v0hm#=T4A`6hX+1=-)b1eVw6j?K4}C!){-&krOFXv4lc`o=8~hCF6rWd$h(*{Adc2@|9RM_Ud9 zWnW(dzr~j0=IPzPz__39ad;Y39*wmUH-U$iz8-kjg zR}ZGg|4}n?JOdPPpb(R^-d`8YomD{vev4C&#e&tk{ZN`*a$jP#$8RP;ur8U9^@_mHS@5!bTh@BlBEjyp_K^9xIl2Vk(j;pA z%^mz}NFwW4gw;QM_9|Au*YN;{svn#77(Q7U!N5XIv3(~<6BRbK6AJ3IQHJZ%IRBJ! zA_lS(+*GXwc<*3~*GQ9S%c9Bos);B2L>rsj|aDLlsrQGxb++eS=goDiVaE zn2sm#?S))Tyt->1?dg&~CUm?KNCsSzj~*)-z;VO63%TaOTcr?C(z%>5#Iv?WtShcJ zM1T#R3fZ~vuk(JdcFkQ~?eEc&vDGSU3og~#O8w$?hmKY-Fcu0!BLu0!7NHy$=S@Mi z&vSB*4O`b7j*9}54O)dJsnV`RGXmvipkV1c%U<4rLF+*2DcI;KoRhhQiKNvL?MhSq zh~{NPxEkW;@h_q(FMxWMtrF8J@(G0AhcrN)bL+BCq&+kh7k`&tX@e04RXKI+8u2Pk zcn}@xce2`hT7`3I*PcAJ#K_qZ5X8cY!zvbnmcx3?>o3HY}KD(BRkJOzuh_h@-yX2&@o_dZbX0e z=?3s{zCXPlxgBmEnwox3eRZn5gTFuuR9+eMu&qkMENlxBzBr(t8 zh)gBGIN*`u#2@FQpQap_`}7uo;`g4nfGU-(50Y;RDtGox(f-W1!dM`HcG5NFvzsz- zqPSi#DjufI;_g4j4*l=V2m6T+XnJSDvkNERKGQZ@8m>-ME4WD~Px_ZR=F^^knhGd3 zy`UHZrJ*w17hx~^bjS^4k~^^n|Gvm3Tp|7awnU~7u(7dnu$%x)KZQ1pzgf2dS< zs1%UA5_51*0Y=tl4Rv)|c;+@Wq}8zRKr@EP0oGffi7MtCEVS_=JNoC#f_E2D!K#FI zf9UZ@M2zt(IJRZ=2PW-p0{bp-3VeTmom5q2|I$e1OpfWchwtZaqN|*PAkS-a#Jbzq zuDK3LVr>DLLGi7BrAaC3!33xKli(cB51A=$5#wmuCVNB+eG=l_rCVwFz?9a5%wnXn z*f9B&rgpTR%u2CrIOpes?53(El%B5cp6q@Ye{x7U))n|=#Lf<(}x8=ncUutb6A!v1h2{$-=ez@38Rb>Xn*;Iz1bA zp_f;FTRI|%2Ve(o`XgrPc$O%T0p?x~F>g>)rKcZ)CXf2q`e5SKhmej#?tDy;9ga9joP_YjDpz_hG{Q?j3o zDh-W?O=si~VwxUeJh3Ed<+fa!{K|Kvy&d8)`-g>yqt4A_*FNo$z4h6(l%?f38sv?g z-HrKpn~S?o^ch*AujqImFz$XD*YXv)r7>SQL`e8su00S3yZfw+f4X}zi~AQ}0&W^( zO|#2t|DrR`P&GhZgkor*5?sxs>dwQ_Us91ngZ@v+-wxR!s@iEVhm^4F%U^iBeZbL} z1=sjhYhriz^)KrcR`&UqgWMZ&A-rLz`-#9mW>X{$ED@pgJ4+0jpc|&8?7O?sf`Iny&czp`LdO7-+5?UU^Q9Oz@4oX-3Au-jL#L~{ZBD*+zWDO? z!)EH_rp7)R2Lss{hB4jwNOxe+YojMSHi<@-GVq8%hU2LlS3N-ua@8AaTwi~0TBdGI0!|hY7nOU+4F<(Pi_|UiSEId2P@|T zK4(ZN_EFEG!qMMu6toEcV^(ki1+l>Yb)NVietV!0!6bMWvAs&6D8CvDK#nVly)h=& zR{Th|;x_k+-J*VgxDT zS2S;GB+ra^RD4N-fP9=KeBcX6IWPtmYk!k(bxJN#X`Ls5e6Wi{7OjFhTel%|zk(FI z+Un{OI+t;-<&?T*1G~Ovg~68g>$$&~=J61)TSL|O7=Nr0zH%BiOMa#2R+zg^mk9g2 z?^lG1HEUW}WDZ|Pc2;lp@Y(SrcTFwX+-z&+Xnxx}R+Ix!A2!a%tDdYZs13xoz3K6a zt{y%x^%!Y8&^5I?52uHJ*+SfPK6gay1UvWB(rquSC~s@cHkn(>?*P>#+&Mx+yN+ zq*NWs`N*cK0fSe$yd*fai)uEhbny0d>(I0eY<`l=uH|jRrqS2%sc2%?fTX9`xudV6 z`t8LZ@0!kTgo;!CIV8c0f1!9DJ~|4=dE4;_5nq`xtzh1qNwyzS0QPEgEP=rC$`1hu z(o7THG%aL9!1-4i$R}7EhtBJysHNR-3*U5Csmor~kFPi<@Qzi9TtGeo`Sn`r=k2~H z$mje6#Pozm{D2e6E^Qq-uq_4@InkWG1~-0X-p;MsOO`*xKf%0@Uz= zUy6V#>fXN-lVHX~J`>ZV)A6#(PNqU$W3E&F-rn?f@1V<2Id@m~TQ0%25-G*{4}9A%FvlOKx}>ka=}f+R zkLDys04;k!bpDR1Tk65<2BIa$D;xcT6s@cFo7mi{Tu-@B)aF! zV3X&A6Ae`s+qh9__Y9oU8vTlebGD-z`>5cmnJi}fMuvR@qMwBf@^TPYCXp9-5qfQr zO|qhh5Gtr9QNxX zplt~?=<5iDQ60Ds_C`;?SnIb%8dM1MR>kJto@TbaoRtFGqU2;v0nKM@s^dEN-!{N{ zVNv=ig#~lkTDsBo@P4cPCB?NZk=jvBN=D`dZYx8vI_cHIMmv<6Ym~{@SEFe`ZDU^q z2ASO5oaT9@g4K$+z{$j7fzM=_a8qz0wtkhk#_oCTbmOBX{3ScY$gcNwMi6XPU)I zI6p2=B6myYz|>eKWC^AQDxO;D;%yE0C>U4MX_*z0A8nZp+O!JhL=0W9A8McuhS(>@ z>+O0yRWP5XqlH?r+Bqoq6XGcd*zR#lmYJZ2p|2h1bjaT~Rv%T^Z>d!=8+ZvEP`_}T zpr6^~5AH@ZEv8nYcGkun7roge4`K%$T17a+;t!B+mFqh(cByr??qyEpTVZCfrd4Nq zhn_L zK<4U?lxgy+)FympR!Zy>t7A`7ZsLAaL2psU<(4MEr9GXj7+SY;tv)IxyZCoIi_mN6`DL9ACVV2y}}p^@Foj}-P!pIa;4 z`bbDI&AO@Fzuj-+k=rPvaVUqZWgezjJCrtVaA)CA?XO(mwqGWmqF82L*j}bgrkIuO z4^M9B-&?j0-stwbsb4gccOuiQvaLBC7X?SJ&#_ zMPY*0yj&7-w!HLcU4B$f{B8I>$+nd+C5GSSbUcdao9U0%D54xP4jhGakK(&MTmZtCCe;h+mO`m8nryK!+OmQT;QalT_cP7sMNo~Cw zTb}*2@BQcYXBy25mzt8c_`IPdq~u+(4zuH4!P-4p1(%@TlJ@LIIyZ!R6lRk;qY%xB zNzN|tNG;Uq9dIlfCdkE}m)OyQrj8mDGxIy%wIaoU!QYPWW!r=Bjc`6EoUKa4uZ7&i zFthDE@nMm1@>+`7y@b)mT!<~+axbOl@}-%c+4xnM<=1|QUiq=|gX*boNujP~o<9sAXs=XH zhmkmSQr*r#W*OKef{biF!PK4^;>!K0BlbF{F)a1D;;f*?x&CVAepEH$ojINkpWk#g z!;wy7vpimD*xUf;jsuHdzN6~GFiKe~dLT^LeCpuOVyn*D2k67-@y8xnQ1g5m=u&th z71tYmgR(rG>_dUY?~VM4Zf5z3c_X1wNGF%w=T*=FJGpy0d`8nZpB?S?((7KkndppS zfh(^o(>(tYxQW}V2>M-Ih^A4joad&klK-rJ`<^-^b<5joSjMhMqra1N$V9Ya%gOKe~+ulC4a}?}yGrSmP0o+&mr_eW8b;k zyX|dp)o5mIKc1`~i8O8C5-|As<)iv(gX=F=ovdHicM9cue4gs_I-r}g&=p$p3R9fM zc!zK^b%XI_{5BpYQVCYeAKk0ol9eQs#qIm4eolLj=aldO3@h$!2JBG`&3J<%0E5o0FFo?3ro_9KF zQ5Gmd+)wK8m!rQxF>^V@k*m8ue~`^7QQjjk9X^R4yk_R0`^Y=F5k^(s>Yl>9gW9nS zVZUNzV@j(2Q8iqBn4A=8kiX>Z6m7mK+B~4GXE)|ClEOjvf^^xm-SPo~tG&T?K8LT@ zKBR8O|7wb&L4`LjQM|ibRUuWJwcpnC!M&h09fe0u(RpyjrAMAa=3Nb0C8+O~@b_-a zBtkbFATLU%jAVS_VN*-J-nprE8#T*$9h$jY=W>K*4K64oKhTA@aCdL~h?@SHwFG2V zuVy&vnz+D%;tC25+RE@e^r)S5LQ|l*V44)0r)}ti7q<$&)Y3leeQwBlpSj_}n1+pR z#&-3ju;lBu(bYeOyBeQlCibQ{EMIhciwyA`Ir0%o1$&Ldo1`ikBfDW^)D60h1v`t@ z^c(73OJz;fkT+2(Y_hss{e%Y(SiFxV>VPgQt zJkbq&i8PBkTF{G3s-R<;vW01(vy)R@Fs)?Z^A20Eb&J5L?=(^>vHO zW|13CK-*0_b$FI}Sf=FeY9#AA1IPn6+SbxKZGe!kqLO2zg_fT!RrviV({ z-)&O4pdO3URL3jQXMHScT2**(L`9V@UT3^y z`!yz7&B|V1$eaP;B2IpR~ zM{!EcT2=Zd=zKCymbd&p*E@E=+&;ie{UDVEl9yDD{KZWvaiLwFA~5DsIf};na(@w= zhtR_Ogo&Cmm`T(sq(fYP4$+{uR1HR=>)efj*^R23)%ZV%d_=_}xAz(Ot9C1nna~WEdF2jq8 zW=HB4vS@k4>lAKhfB>vBCq5tE78L_zAu*CnSl>kib+viWJ*~ z#!stW9ZGJkfV4ovq3;cJl{q%wCaewWzsk($nBS|q(Q;>zUxRYNoqn38eLkes!>n$V zeTN~<6O}mT?vvzX=_fd}=ZD}PLR96^S8PQ-fD`ljmhl@z6%5U*kRIo!0!2XhOxkqo z6VY)1jCWcuZ9X-vgj2d!{dplxe)6Wvnicw+DW!A&w9jgKVrrWo;(7?jg4c}rs;wE?&+oHtu_QB0-go+L%K7E7GA|G z14qj4i6*km{gE|CX)W!JhEiUkXb-DzushNDhX;pRh^66ex%@ox6z z!%`RT9js{|+2l`56%h4at`bXru=UE(W*o#L^&H1!X@Cp_1tc0)ocgTv1GEWH#N%mO z#H_kR_3`w^XYd(1YsX0n40rVD(LyaOsHQi|X*oESYd$=o;4YuFfvIMW)GJF#G28YH zS5$D1(4GlH){dGQvN9bgZurRdW~5aJsPC1H92xB6+Q)D5$!+*2ElQH;jTw}KO@pFU zQKbT%LO4aSDJ&>Dfk;Y+QL&7w^nbtDD*H0rP?_*Bz2uAgk!i9_DLi9y=#KgwHj|NmqkZdzz3&{nX!B0r%#xOUjps{F zagGD&C5G2nDHA@#=9j1&Pd6}&n7Hxc#jCwoga@_j-TaVOpa=rcLdv`c{XR!4P$KCbqofyoD^1FbX$&Z~06XV=iH3+EU?q_)Ok= zwN2GvD$31Ex*u5eOEFQJ%dg#Np+m2V={rho{g$>5MkiWq8a?aHcKLwb@i*B1Gr^7j zooW0g?x@V-4#Fg`ksLQpVkfVW{8qw;RQ0!XTmtfvd(oBNlsqr??Sp$eV{o862A$Ct z?4#%XD#hc+GJF)h3rRQS_IA8!MtoZoA7SQ%)1|*A*cpx-nxMSpOiDYgLVS&k7ps+@ zceonT9#v1;z)aDkV9pB2h#l8f%8m@6snHSf<(+MA0@JpwRi4jrkn^M({cAhRB#AqT zh5JaiK|+T$%6R5NzpEF<-45(v&tYnEWfgs;-V_?{u_peK*EOzC zkp+|tvLnt-QwIxe8VCC_f17}f!cJ4T$;;IYE=cS_TsVV!+hcGvW3vtQik4o$!YEUf zlQLdr-&20BwQnmm>AK*SU=jeDfa>hgY3gO%lXoha^gu&kEfjA1v}j@2Beb84`w%AX z22HXJeb$xEX1Liy2SbBhcJz7BCsiF*4U;Fa>12~En+$Q6+f~oSuZlm|n%ar6{b-vS zzvx!{IofHB1$WdSLaVrBdX5}z6kIR0uB=z&m@<#*StJSeO?&8y%8*vypNn!C9h+_3 z#Eq$6*1_8}2LXPn`aTrhNzq2pr{Ep6lA81LO`OES}1M>-Hk6D}S zg1Om;33H?8@%>CwH^lw#Z18NleZTFQqNk_;@%=FCGpl_^TZ$bszTBSaH={WqZN2hU zl%0)8om$l~`uWk@ecVn^JNuS{&AS1iuHy8V&=K7mA^|wKc=M23^J4^< zf><4K?63HJifMi{7=~T*??QRJ=M5RTjYg0?V_&_v=53wt4#*nZFnpnH7Vh>jzn~8j z_3gCGSw~uhxrl9Qqn((ZA&a&+VPAc>v2tPe8&~FvK2wM9CHDj<2yF&S% z_at_PM7ztJ^~1Q1A{Qj5Y8d0~My3!iyk_=zF6WvQ*JImR!0_E3SWG z11FCz!0smJ3(eLtv-eXqw#mX$$KxX+V>li+{(m39YX3J0|4qVw>hPZy{HF!~X~F*+ vS|C)Nk%EJRQz0WEs`me-&FCW0KEcJVSC!e-UrWIIBcyafWezO1n zT`kJkIj5nkQ2v5dxwUAfPg6}A)O!Y>g*kQDi(;)eKp>4lf_6#k3jm_sF6 z&%x!l1N7R)cE$l{6kmh=UcRgn$;6oReL)+7Mo-Ub9P*pr_()f5uFsavss1M8Y(6k| z*m{CP()a3#`-$3a`A(LJ<*3Dz^zeDi9o8fFZZSntCOh7h_qGI9mJoP_F1dO9_lt~Z!Y-iuyB_WFsC`Y{1UUSUN{{2>7{dybc zu8i8!RPE=tCv-(+zbakIcfI&imeqG(p8Z?Dp|V1){(N-kvMO?g`CYp%hP`~z`1&cg zXY7`YPke`ovi7H1B}F{?-c2Vo=e5OzORlHW+3v6G%@Kh*fZGk>P4d4_fCXjdvgpKh zpV7qD4i`Ogstj33Rk0QnoyH0``Ptuv0)EYfDSanhLJHQ>OBE8New6vbcjG;<&`M;a zS3mF(IX#PB?jinDRQz;N1L(J?Fhi0P(t2ChGUr5|i&?nM1@J36k>DvZ{Vj|)b4!z}jY2f=?i;2QLPfuVUyvsnM-aRjg7YyA=fPxwD)_un)4 z|HjtRC{gy6DrgIz{>$Sikt7-j{5uT)&$IZ~;1n+8W82Ku2_DG#QT|^q#Fgf!BRJp; z{0ql>;=~Q`oHhE$z_9sSkOC~f`UT1^fd{Iq_|vaPG28wXoNv1NXsc(rcF`C znz6gL*US-xjr8BE%-;NTHlFuQc6^|niKhLtn!i;MQ1Lw^(Rl{*H?%%?d+!|RxNc!I=a)1~t)opWs;dd5~Os|yj1KH5(@dEi|x?)XTmOMK1S z_sl8M5aNB;-^-xX%GAnl&e#15`1IAZ1VmjHP~Fn*uuoE^Q9rDi9N_YQ0C@}&p~HdY zFn8YRw5?rLhX(3jHwFieef3Wj#ilT+WF0>No|E29lVI%ejP)%Zvfu)=m5 zWs%c&ikhRNjKVLEd;>1>!s&`f*&AMp>Y|O)6T4Bv2G4z`8L;%Lm$Z)=+|x^~NN%4h-Ol1)w2JmQmKx9o7#D3JjvVLs@Ca znHz@5?{))0j{pH5X4SLDL_Sotf9e|9ml>q25+v=$BAu&OMQF_pdNiC@)fQ^{W-Spi zA4}f?G|JQ_zGZ$fF*h7ejfmF~9^hlCT4@)TiyqoGmu$RS5uUGoB3Np9$e>Cwo41*C zjoeZcNKbR$oQcDU1z{#KzDX{q^1 zB1w3=gpRUrY53Zt5U6pBks?ztfqC#}s<6}1U1lpa{i!iWF zQb=SuoQRXo=h^x9)`GVapU%+U^_~fCji#i42}fkjZj#%*2ZMk!PWc0iqzTYpbLfTk z9bU0170={T8%2hRrt?v%1@rhCLBIvoH;fNK@>!nPB+4c}*9<%JvOGQ5$Hpg@Ksy!| zle+`o;R1RelbC}PI{d;;cbzRpqn!jvX0t|T=!o4>yK~gz9{n8JXOh&Lh-l<#&%L>7wQxldMsEh_nm_I)Wd+p z#)%(5-DBLt2QjbrN6+{eFnLdfgeY!bl`F)v`>^}jW-JP=FzSr-yx2KfX`8gQI{cPeZ%}}} z^H9$VjW<~_-v=G^*zECIqY-nvFRzjV`=JeI2=DkCI1e;#h#MP#e0(&*{m$-ou@Hjf zYf0Z$?>zB|#f*5Uzb~F*bSFxzGd77p)@3VrqVH2{p>%V;$HE5YztWBzO{PbiDb(q3 ziZwZg=@<5@%;4Tf3%vHj^k^pbzjuyu->hPrp{`%hHEvdUe_|6=jlQ2ynH38kb97aeII zSX}$1&l^29S+}0rL)-W9Ex<-bCa5piqDYVTfO!~~?NH14H7M@J?*R)Uws1UxsC ze_)SLwvdiaN2v(Osd?s7ERGp71VdJ}tNp89+Vi#r)thyd?->#?6rieW(~A$cn-W_A z4x1_0S2sakZNT7yPQWxfxs&0I%a&O*X^T z1-9FqdnUZdGNN5_!_H&($=|2=9HHJEzENbkIR5j?oQ$sra}Ql5;6O=C#mG(u3P2^$0h0F1{04v&N7T-8YMiP?4 z^MkPQx339BDQUcA+eb69HS;h&rY3l{?oewRn;db@kFRK7LPw>a^0Ak9OP5GkATJA5 zsQRZ4{CHsY`l=CHgOJj0?U%+{a1h3NARZa?4@wC`7ZG}530c`dj|>3m!J@B<051@HDPis|1D+Mqte+u$sJD2n-hOD z>lrwr(;#K#`;Z~3`s$f*cN}=VSUZPRx5+U;DaJnfq{4)t5?65Kz0*6Yxeb#{*YP@j zaI67<&&rM4{W}9lHOy&yGX$Pqy)6{YL|4{Y6QDJ7@zh2Mg z&~N@0CAE|k!^HcsDLucOGDff2)JgZ?;2y`6Nx`FLpIX~#Ke1c;soBqTDVJN2mOs}y zhAG+k6a?*iRLQqBLxW{F%F=D_6rD}}u*KYThCW$o)bkLp8iYpMOVY={T0^R$226^0 zU=o<~Z}`-vT0MKh*Aj9q-O&BR-UxM4<-i>D{;qL;X=s6Em|jc*%e5aOJ!Q+&S!lRN z`}UZ%sv=1EQL_vlR#>$$3BS^(NiKqH{#YyBi zb+kB(#4}FamXj&;o2dNX;^0Uh9^NL@IaUkDr`(mY_jby^%q{UtE@#*VZ6Z5FC$vxO zYP~Syo6A92tpCRQ8Riu*!9<)|`7NnLmh<*xR$)<*F6hv>JSe9&{w3cX!;hAMDd%T) z+FvOX*Hvw*ozv|*Al?%mMB+^mGFIqTk-n*5=)I-cEw&+f*dWc<3E#uUheoZ3SI}y7mFwQ15XF1ZS(Zq3In&-UJn4X;^Vzg>Q@HbacwXupeHWtkG0oOuiF&Lh^<}=ZR&rMc;)gRuA8?ba1>FgRl zpMIwgylY+E*OaB`o{cwgXBKZ<{tapHKer|9#Xj{Ca8am&4FnzB?G8sUg%P6-VxktR z3`-n%s63q8)2NPU(O}wqHF?#CfrMHeWf~Np9sVy?PQ<0%TVs_IEk31Jg)zB~dYjhE z?_(Y=dI}w^5X$Mx6Mw2LzSDH#k<67qB8T)^aHs(1DEPmeZhlC$m+a@pOrzmXpz8fdDIKH5D?h~riPI-H!%;Oq{S=heXOMt3K&yu;-_TlPD!FZ983S@r5_MIi?FcV>>9 z?ofp{pPMAJxN*SrF~c~hds0iE*w-KOunOpvox5>xSeNIAGcPh|m2miZD9O8|-+GtjkzCnzB(Pnupm)Tcy1e#% z#?!^Zrc!ML&=xyP6MA~C<@|Kof5}Vg6ppsFabtl2L{{<`F{Y8+M%k3cFTgQl(R5NH zG%Ru0Mm%2{`Y_ol3<|xhbjg8qqfKmDah5HG)5kM?0?VgRx-R&)e;x)bi} zSB9~s&PO?bXsJ2YLOph>x7jwTJ0|mVm1BCV`v>sup47k|`Q_SCvh-gAxMF{vZOrXB z`htsD>Cyc$aE|XDXq?U6NwhJjs?kMs1@!4JN5Y}&tMTo}&ZB5?UR97mOK0;@7+a|rHEt0E&4G-YmKA*LL8g`)j5FYFNS z*GvJiHYc<^;f4D=yPePqi=O0EO1rGP<7OsN{gGlTg718tp-L#bEnHAyu--1saj)J1}0x^bo zmAd~91Ae2Wr6Yq%ppK&ey>yTGrqmg=h=_<|)uvJLkg$#V&tz=($xMcp=W^sF> z9WXQO<682zIh)V>mK4EV$Bh}Wu7gxgmjdciTbl9!$eMpmA z(E8zus@9I;bmSsBKGTRD5Cb27-$Rl>AlO8j_=X2sg_l37#!9MxLV;rOw5F^^r31Td zm+G#nF}WhoK}~3W;#b*XC|y?#oh&w1y;#awQvHTRVv+2Z@ygx>5rb~)cgFkq7k?wbA3|hXhfE>Z?w?_Y zpGc?%HGhcD+fXds@Lmu~wrN&FJ6U=ex}Irxa_qi7+uj>~OF~jqU$Pjce{$t=_+diI zu90b^$n~d_4`mnmTl%Pk80NvDbndTUEf8qq!wF`rc%<;o&^LzuY1l|oTCUmmygX0} zb;!fO1uw>-0P15cq>L>boKh1!po-ab4bCv5J!c#`2EZXNQ>NTvvuOQx7@*|MRdBR%=-XX7 zN=eeUDQIleq&)dp+*(Wd;@ur&UwYxP(d-F0p!gr61{D?}CYBvn#|g?gWmNW8QHw~j z9^~mcwO3eLAr)5CDKHU~nIPgAm~}TDdiXXlb2PBQ2)5*mCc5(3lv2zvh>-{ygHY%P zd1Aej$Fx+>YvPmf%g=?SCVL-j7MxSunBLO?=h1(_D8a^HS=$hs6z2&Ngn)J z!2|ikPnLKw8Z4V48VOfoXFu!&<~Nh^Po385KZ=dRvFn62))EM?meupl<=#&W!;Nmk0AELYD6;+xQes|(Du9B&mA6&)3iw#j=u=8g-w z-Xi~2L$J_pelS<4bb1xl>uItWXC4vQ?67MOmGMC%v~ zUr6kwh;HLgUlb!Zg33icyt?}Wi-oG=$XYh@A&i4hMn)UvH%&Hgxwc=JXsV^#ZJu-= z4D8;QI7k@1wr%v(E#K zf_K;amMZN*%RX)~ZxG|~vA@+NZiCA{^>zs;vfGB;+0z;P62Kz=N$Hp)L^gru1dFfO ziV_F3jnts+P8?f%B}W4&xvB`b@_>4<)m^@AzzlbyVC702j z4b=7WYKbY6pF!bt7f80hWM+%&<2=Y0gP?B6XSA0BiGZ4hg?$E^p~#Go;=!Sle^n+%KFH zNEr!uB+RpwRGepjkCeZTp}OFAxkd8kTa)%^6@7Nd|8aixgHp}=2yjWmBA3BT@r&zr z^1+3;>@x$JZJd|b6iu@MX5J&hDjqp!wJbSWPLV7$!ZWWoAEIclnh90WQ)kqWKFJ2J zcLmdz15c~GkZiQmNP7FZ{AXWW(e8CxJl=2gv#aZ!o|j0`b24NQ#`~7YvrCzaa6i^- znQT5t%aullQN)znv+=brwc}>ht9`r_Ja5 znBF}rhy&5*Z5%4O+ZQZ@(-X=1TeMvv7M?vIhi3~fBzU7^#kr5Ly`^)C?3ChFD{N(7MgStxCCwMor}^0obW3FaA?Ek7h_;|( z&J>k|buVN>d9iZ2I!DH>hR&0~95ivJ2oJypw-%_Sn|wbUS9^MRmiq?)De1G&Fu!>A z*AY4!XBNFJldA3xEf#*`Fe>d_I@SX52d&d>>a*D2ZJK|CVrn8rdF%lNgXy=sOcVyP z(FUCZv5dum?05Zj+*M{dGkeJ-hc{R497^3)RI$dPzc=zhAM@dK8A#6Wy~U(MU84A> z=yH_`nWe9k+ElCWwx$yU>M#qrkQrYGx4DNKz5zEoZ!~K04XChj!HTGQt9osQberdm z(moc;(X~b-EFa=t-;Sw7gTIO)`-qA+n$;#GVEBBVe-QvfBq#i;L4v1M?Z8RQsg#gJE`8_z4kgPOFh#Hrm^T5_Dpl&b+R>E~X zE^9eB{&mAOA^>o&K~uIGmCka(?co@^$u*)F+@(>ex)erCI!GR6foxc=Pwp(jRRjm2 zp3&<``Ag=7AXEUCM=}jBlTDSP6tp1lL8jcQs!9XZ-w7|OF)|{ftuQPh1F8%Li$jL0 zfgswf+%zGK(EwchNJX~^IRfz!b42zlso*9zu@q|u7S#y8!CP+CDyp1thsOjYsIWi? zn2sS)-&UP~`S8$o(UV7x@ZHQYo<_1EVqEdd$f_Ue55JF_=>(p;1j4yJ+%PL3#VAJc z$G|(T9KX3cbAY|xm*?V0<0C zRSH}|lprKT@wYDvm7?Rzma@b4Z3-V9KWc8(5S~5~1*sVkBBzQ8QQHc*{l@U-@D{)= zT?{exrc4XL#3y{AzhzIaT0-f(&l&VQGEEf4EyF)2f8rwv%?gwPHNN~PJl_!Uz#K?g zA#qW6ewlf~xAeJmncTP>ag@S&xM$S&RN)J)3yb~6G{05{s4`3FXFQT+y8QdS^A91d zz0a7Aor}!!EMbZ@xlHo@MxTV_g@RsgM9kJW-1)R_Hre&aq>1AW#wlzOOzK9=)V*1C z*X`RhbKZ$|-^&sIAW%`J;gE`fZ$EhaE)!0h{P@vk>5Itu)s4x#H&(7Mv{_tzaSjpo zS!TS=N}6^_a4N{7ijZ^fye6qgq{4jRaHM<=WLkWgQ#nHQKGPL{zU}J$l+6P8dFw*B zXLBsYinYMoA$g|5S=T_d$Lau6d4kWfM13#gXE3i(67l47 zRr}}u*L^QT_`G8$S-95r4?pQWviD7Kfv`Y+U<@cdspmB{m|- z*1MN~7(2#Gg5=rdpQfhx{Xg&SgWr5hT!NlrTB~vnjs`Q4uNos&y4sz1h005AuKj`P zJevDssYBFc!u+K-7rr%jD@Ia|CQDoVaE-BOQRcEjBWUDMOH4q3wUY`BIaNE;E)ok! zp4aQ=AV%c*|Iz<4?`^p{KhG50lBa49TC!NliZR`WK#T_Ljo}VuzNmgUO_DZR_alm4 zp!lpWm7*~Cbuc=*sz+U6PL4zzRSCq};iN}h0DOZ$+>#xxkC`cbrw)bJ@t7fWx9#>9 zu{7#f^j+9gLuowF-|gS9mc2Yne4d@U@bTVQAyAka3w;ybMXAki zTs-;~gIe?+Qz`~eNPA7NmM9L+H~YHCjnFr@&3|TTXWxWVZIAz^?2lLHu5#265cp<# zlpv_A#VNcDeT)D&%G@lrZM*dt6vl3Rt7;7l3nxDt1Z;5i8(2ioT_rnCuF*moVgSMd zD*@k+WFOv7KHrTP3(a{4PToN4eG{<{kPP+OTH@LR^=*5`3*X8R9dQmbD5k0lwGD~s zH|MgA#0V!0AdlkUhu& z+0l3-D&40$YsO66>U;L_7E@PAuitCe-#5I6JAY+3QVM`aSS?Bd zZKFkGKP{n{>PZ`Bk&Am`Nxsaj$*r+n=MNHwkQlc43CDLk1UfoxT_;moyas1*^EgU; z-e3iL={@6ii&ei(GeYP^$1)OqqejVn3*Q@tg=JV&?tC*q$!G1_Pxy^yPspVDmFqFT z&l4_~6AX|SaJA4U1J_)jW68euyzyr0cSsQy@RgjDNa{U71s8z^U1;T~U*S5XPk<|7 z$qX!wn=yhWf7$$G2>uC3dgz7C7u7pryjaG4~84r>hgftT(yjPn9}^M<3Dcc zgD0#;srN->`dCcC=+0SXzD)vKYJc2h&@SCyrbx>h(PSq<3PDJ-#Vz#z}!@pGl?GA6(qKNiRXA>5MGP)Gg zj3~j4Pvtu9DIS0qT=|ZD;YaG+FfIsJG;gLZcPH1hW7xVxBs5W8DJDmBy?>gfEx{z& zJg#8`N_r~t8QALzc#JN1)Mhaa}hy)9B;m|y8+~IIY}e5bB~NeHiRSi z0GnI`_ONw=;S)UYxk9vakd(F_!LJX28|o2S0EkmjA9*{c3HKw+jq`-&>Afb*RpEAO zboClygeESBb)&{08EXoNatRE7Ni&gkr*1NdBmELG1Yw7Dpb8TOv~*?4?zv#ah;VieFKgR>p;LUwHm z+$jf1Pjx;xQm&{ZJq_wl=76LJ4jB?YMkoXOm2yN`ESGXg_}d!N>Ygr$b@^L@rcaN9 zNae4;QV-vJT1Zs4QqO{?2Y4JQ)_SU3P}*l(_7Q4r)G|1ZVk=N?MT4{!d|_{EZRz!i zN?Equ)8pu6JrrsOHh22-4r;6}ka*f%QD&S4nDp!8NCZ5J`FF09Rwgv%ifuGbahC)d z{p-rF;}28kC6a$7nF@pRMK3MFncJNnKmW2_;EK;V>l|hkBksVVWHo zDw1y?TM+>la9|OGUU=<4Q?-}2_X500LWqV8kP@=}oDEL~Y@xs^c^h|`BOQn(t8Q?7 z1R=tP+>j!Btj8T0qQ%7u#G|-($U{!B;qg^^AWyW$5%Rm5QaR zHZl?ylz8=hO;9otkGiGjsX&ix%eM1Fr^~5+@9Ft zNQ#I6nQlSE2TuzpkKu%TE_mSt*Bs%Gh4p$9Cd)*U)+!dKcz?(>eJs-_TU*;nxK1-) zb@!#pV4mZEOGwb3C6ikBg%aS2_h4_Yn14Avpf@g^8s(JGvrW9Yr~{ugB2Ds-fxo)} zb38JMUW0M4+-+?l`LKiT#id56TEE?!?j`yUm;ujDTC|!ip;K_kG zc$7CgzsC>2;@hL9dMn+)*E3=!2XWoGm{UsoM zokaJjubR=K!dEfa!Cd^?8Ot<;Iwx9=93C)J_6jo>;d%LX6Vx|+%S%1No-pJ6Ydoe) zr&ILJD@gEIBNEK@cJq0=H(2;lF*;@2QAeEdtPvW?+eh+DpNVZ-Rov(`1FM|>EouG(AoAcG(HEFPvsUp-zU{N04KR<+ZslYQ> zL=M4uSrj%4zs8fhB6A3P6|sb=Qt=;ueu&Ot>Q(x44uVnJWcK=AI-c~4*;WwvP-sSy zQGZx_R-!`NDjg#Zm%Y~65IjpzauM`F|oZi0Fpm{|~EWOu&1lyw?l{5B=E*LR6 zy*=r=Q;cWS^T6F3D<+pT4*@EjZjEHQr~EE}JJgGfRiMtz%JIO#j1o@eDJhd8ucY<< z%ld7Ug-bworWL2fM8+A0S~Z(|&0@3XUq0^notOJaFiTwn{@~Z{lDFE!a`NsP4w?B|>;WVe4Jj3}mC^JF1O5ji)V47Fv@$`t_uPGm3Akkt}^Ok!K!F zlUSVs8f30ojLFm8oZxKq^!4aI#nlG|vM*PE#MlS%K(zVqKNobcmHY9Ewp1fh0HUz) zBnnp+=kmueTkqhRRG3?U5hEC|@_Nv*{9F%yCq8fkB|m&OA_oJDzj8yXes2Xxvv#V` z4KoOCU4Jv80XMRyaqjexe}6BTnwtw)O(Sts`SE4j)t$(wA8+_@;YQR7tCBt5dfd7> zKwkzHWlRwggZ@hpJ122rCQi*Gw!(NW3e%PwM`iK!^ovtjstzr23V!F!eBzT^REPTa z7OC8g?+)}S=zYr7kqK0Q_zbA1zh5oVhx!bK-)Em-S<9k_(}$_g7g+u;KiShMDHQmZ zpNx?c=QQ$W>44MvDM8O}T*x(AxJAJEkd?hqHykc4g-@#oy}NU|^*Xqnmfd3j6-gYC z4|3HKAU-rp%7d?&|#cCs}l3Yao$&`|$fiiN@N2rq5U)*bnPEsU+(M6qI+ zptB*s!5ik}VY@FZTSih}Ket75TXudck-OpoW_^e=desXCKue5X_GSPez=ok{Jde+Li)dcl_0tfP7pvmoLwGk2F^2thj(T+Qz?V36zn*w= z{aEEL^QyfZ%GEsT8g93KloF?Z!>K!C6jQN|H@W-5aE26}S`Zf~t@jAckbeA~$yMry zv7IT~@Fn;S_yO>-aQ$R$lXDiSjQjdnO=L~{EWUzU}q2W;Jtx+1E+#eTu6=g zk-^?#-FN7^ET5fp!E|`+3={Vn>-B^toBKJ!=0ln%_RjKyxC$6dD=oD?Z!{kLq!(v-%yEi;yC<}5q5EKjFe?e05#DvFy0HA2oRJ9TRgL2*QjneP3L0pVuCzt}RmMbt@Ex2-ePiOs zs0$E+p4mQY`KGt@Y=%iATXrE_Nu z((j8n=9Pl}>-4ov+WS@OHYMPvH5t*itidQlK~PKo32rZ21RCJTKG(IVj|SH3QP_e@ z6K~fYABGzHt$u%KJSc}6477kMxohr)2DVD@g#%`bK!nram3!T zK4pz}YI7lc=S+`t^(ojdaN+oqwM=^iU!*Dg9D0}%MPynn^b-*OpLMhyW zI>L6y6e5#F&JGx1=;CbH>X)~5N6+o&wpaKoBlJU#?AG%05#H5jZuf|o?db#St};pr z3huen52{-RIRd1-eZ`c6vJ-MP@5c8qc`*;`O*fbi#x#RC8l0~LaOq-K#^{^%)#5xi zfAczp{2JAmg9CKc3x<+DIj&|*p5){9f0i3r~~y$`&X#CdAr7c=`d z)?4AJR!1E;VxDBXhs^yfc5+^tG1hC5`W06ZhbPDHkoFq5LU>$R$>%!T8A&$jqs5CH zCF)kypTS2dRLDY_`!AJmrU98~FNLbQmDsJ#)c3~!0y<7})JnSGTFt=VMYnV4tJTE?LPhn}oj62snXlQkIgq%M?hRVu~9SvED`PJPH zVmE)2JRb?Fdu()s(chFfY8WHq357PXi5x+0pVCs8BM3$Vl7{qS6wB~s5gs$l| zJ-^|jPHR$U*98vtpNMlvdp#&JNlk3wY9*jImb2a{;u8zMqH$tbg|Cj~5APY<#e8_$ zaAU3E7O1^ar|CUkc@zu{$TNWiN`(eF$7cPKI|;lJMcXd$qTXtkbXf(W1KpJih4hl| zCaQo|koNFZgSEv)otTPMDtNYijPNi#rsj`>4JK_)a{90=?6-Xuk>3{shBG+dT97j( z%M*eliwZL99~F-gx1o#hVMc}9av-1>u3u{WnRmI&5$^sN?PscV8Q4pn*1QJo`evpA zBph4dO-mWjMJxn@fpw**Z8>SBVcS$Ij`zdl0idUj*B$;y{!XtI&ba2rKcRM|xg8&4 zGVAR=WGFYW;n4hHjH?Q8jrWd{hsz;RoKUNm-KZ_3yQfF#>Z(`a?FxbJWfxnkK}DGV zNa7uOT*XRUIv?hgEyed2?T&ecqdz2pT}qPCB66djHi|wS6svFw@vsmvUy%wp%m#*@ zAKtk1&?a9T7$y&A@yZXrCb29v=*2ew`bxqTak&0NlFG6| zv|U34Cqi9TJdiOaH}u{HN(%0{zO2)tM8&@mi{gxXG1PhI+fx~tgART)%4k7AL9$?a z<$b_fubxcJqTR{zcFlJ?^=k=;62hsA8dE<4VU?r#@L)RAOX zRLK;a7gPpt^|4b`TRHTQlwPpf^dw(@JL#~r^xVEbMi@j3zN703AznBoDqCra{8eW$ zpxHJt0 zPvJ8GKm$p*HIo9|*MfNe`&j@6vYPk9z$TwFX4O^H4d+l9kM_OoIbc_Cnt-opQvI*q zUayLQB`cpX{xIP-=8moqN#0!7zQ5ve7RE_M>jwv-YcrckfG!jI(L73n(mpet0x>E` zI&*w%RbI&W;{`h1`5Y)O0vaF&p^}td*s4%Ej;?jOTvy#}%7VM_d=y0`Rdd&C4-nh( zmJB`*1k9bKZ_N75lDGr05>v0tpUeAw!=ik3w?Z7!)Hga*GzCkhA$3k%3~Ik1gO3SjLx(AG?wNkKlbs%Ri!f5?}tGoL2BqeD#% zy09dhGsK|9VT^3^`IgO`ss>?#mdsZxIRmv>JGKvh%knwwVc=FMOw6Suog65a*qtOM zV5+o9)EVD{{ol=M!N;+*L)YD2{gBBlL`jkCGv=g9%-u9_iK0QA;IQXqP6(3(} z1c3Kbbo43BFRw<>6$vqyQ!I%9Tl_YMFNWvr=b)SKkqKB3IJqrri^vMmnhy}-!#*L% z*>%?#SlsmIpCoYp6Se%JCA5_b5s{X$?^Gv^WX0{TwiZ1vHRaIOX-=M&Z=2RE*Nv%X zG>Y%zQyb?SC6v-aD|A&Baq|1)|5~`?0f>+YsowVnN7oNnf+0wvPx21OxZ2%UoA>14B3KH-nC6(wKjt#C{Z^va8F5^Td zqwa8ZjMeK@VF+9P8=0aqBGP5^oAepL_Ny~ZfD#K_kN>!YQOegjXlx3_<_f!b1IdMo zMa%)@x#1+o@{cW-9>#qlUE#In%zm+-vNxgWjgXXkv$|$`Un$N~xxRj-prcDL52l|-_dVclr+j=*S*}~gO;fH*eAAvSqvy&# zoM$weQiL|?8`V!W=0)>ovh=&44%=`!O56IppD}JS_(c42;Q@GvXR?@alDl&LI+i3$8Q_Z6f5Y14<=>N>>T` zD-(q0bwY(9G*FYwXJ9ue-A#~8vP4tIV7|S|`s~1XEaeL@P;(h}oemGUaj9);jKX1y z3c85Cz-aaJ?r5uJBt9KkDNtTfdC>SXM!l-9L6J!kye>yM8s!CEl58p@{2nsH^Uek0 zkr`ucHv2oXk4)!B5f@Gm_pc!ZR~}cShUWFggRpBji*j|%;Z2YqE0-a{39d3g2VGi zQH@Sn$+!?cslL3oe~otb;s!^)EC}9sK`Q`5E#l+rFY&Me1E|L#@??L;_T=Kzac0Wn zy}r)&JR-Zb;nm&}t6utL759W29aQ)fzDBmD`Jx}$K3}fO^#=L3?y6aN=6QA$(HS&_ z5*?i1r~7jP07>M^)imGQ2I@Tp8e#PwN!Bg(9_epzB{lE?`#ps1W==Z|0FEtbJVbrO zN|J0EJaj)@mg`u-=a+80Cj~8XT8Ae{nr%^C$GWo#p0mZaERlszQh@G9;)s6085bm8 zyyeStFt8~*rW|&4+Bksni#8%xt)s=74-8C%RmU4;36ZHHRboHoG>G5(y}GhIp;abV z?UgD?yK6hcla$yUH_IHKSoA#(p0BZ%G7|I3t(P51# z;o%{h^E$zNgUO3Bt$P34b4V)PU+X@Il3mefg5#RB*@*c^PimvA`!g8*1)jyuB7f_? zmB>mOH$cPSe)A$KG=VQ>E0qeq9irn<(E9uox9@gpld5Htq@^m2Y6JqIA~Fh+91C3X zo$&o-o{8$u-XdpW*jk#LNMOn_QGS@~ZSweSarP<7h!(!6QO#h-k9rjk|@* zEew;&pR!fIlrDaHcz#sA9xL=FT!aMc##+izXY-Xj=prt6Tm@?vRXHGs4cJ2T^+wV4jmOWZ^CR;Ia(b}`TP1Q&Abyl zCXo%oXI8Yf2B=;#$S*I`HIJ{GU$B>bt$p>L@zvG&hi?A0ZF(Ku{+vV|vCO}wHDwUp z8))|KivrhPgEqNH}E8+)|l0kdIl6d*(KVUX8t!kbmi;B7r(w75f;w%;1Sb znb$kEbR^I|SK3S#S7CSh_@<7wt~{lZr(d)K&xm1|(RW}!4=fGUV>WfE?wYbpc_f)N zct%_=HJ&gMuP!c%2RiA0x2{*%N8akBXPJyk1c`L5h))*I?QVkzxt~zz9F1*{=x6(4 z6KDAxU8;B#_SqX)&DiBv@`88ZmfOV1#q{J_xT{4+%UrT3weNBpC^DJDV+zJWSbI8s zz&r5X3SPjJ-UwekdS(tOo=ug~@uem=k6WVXw@Z3I$54i_O_iwS?{nIYhaTG}bDF(# z-oJ0^c)(-Skx@4FD)Q=g-bi8eyd?c=HZ)FouFa%cObVMKfR~2ZspdA>GW4E9IIaV?T;y%hc#pS85@GU;V zG%1Q_t`Hf|wNhmwQO7-7nAT=|!A~Y^!6$V3xDuoL5`0q1tCr;a%sGyQZitn7N*d#d z(52TwGNf)K?J0(}-Ei$Q3ax9o&On#?a0**43dl?`osad_{d2TC0dJj^XxHq#E{D(I{nD5oinjqKgvv>IP+KAY_Bi&CgMOC9iJ3^Mnk12je1FY?#8IN z!OIV!H9w+pP|4QTMM$ZW|9m0?SsAih>Xe#=vo(No6N%jF*|Wt-W^jYPvbC3lxv=V` z7PMrTTC_EW=hQ)c0rrK1udk!4*L)|_$3qyNOwrXAT@L-B`xhRba%r5 zL)gnTniLk0_`6$%d zMTb1k{)qC?$>rr>i}enaV7+zVSkCE@?pa_+fZgmRHKm|E5oG#3fY4zF?Q%uj&^CzLk2D`jhz%6oRj#9XWOvNoDxrk`u=&e05TPSP8tQDK*U z5Vyq{Xqr|YQ&^|Yn0#guK!6j^wYxGoITAnv&2Z{&C|DfZC^tNw%`lnY@hDI7X0upL zl3mVfzsDA@-!NI>dZSH`W@Nx~YGJRi$en40Tsd;>2M>V}cG{@)cURRZ&kse(*`r?s zSTv0HTOYy=x`{cze*NmctoLHMJ1u<4o5#lKoC66}-y~O5Q)~KySXJ9aX-33-m3Xyu zvEk!$5Ti^^GdQWCDznG?2YqpQM5qonp~gx{UsJD#>yK7_!i zZ=h$A_m5>axzDF>S?v-TqKn_dS`5kVHqWNulmC{9V~y92?-hB`jHs|(bsRz_&i?j& z0ZTQny`G~d<2aBT9wViokg3_7%AFPH{;{&FSY_M&^aGL+`7({--ebW?Z`7932y@hk zuv<5AhxKY_oJO72v!>-_{tqDma?gVAsStA}TO6nd+aRlL$TSN_V(X!f%49tWV=x+F z7s|`#Jt~I^DXpd6gb@Gp69U_ztNY=^oRL~a4vYIBkQRO_m}V`DY0UibBDa{~rA-zg z=o=$AR^AR!fAP7o zm+?4zI%jWPaOqi;1_RW}tE*HT zr+C_6#p>ducb)Vnp-#jT{-iQh3kIibkJ99kk1UnnL^T+jDvV0O;rL5~GAZz#7tziL zsf@_juQRIlPc!p2Ny~P=_vrTSjTrF>;ynOip)omU^K07C|5kArt7OLIUppJpOZwxy zZlHbLF(+N9^N2#5r=e##)RpT6Sx7ko_DUg<0lzcqak_oq$^#MiK+NS*#iB? zy~r4ZQ~Nw$>(QL7T2cOhj^osnkX`T;(t9%{lm(y$hoIOvRQb|1}-0PBt`hPtA z1d5^{mdz9h2}}-iku(>38_PrPS>>y5!lj%L;A#7tHFd|kATsAca~d{ss`E9VZN~SU!{y0M}j%2>Edg&*EO?R2z}LU)8(! zlnmemfFaMiS?{X<^6IW{c2#1AHI@0% z`npE+TN(Hc(L=r%)AWRQ9jf**-Q;PU(aW#!R(9Sm@Sje0 z-)gtPU-cs3wLrs~G|H3$yY0;S;iopzbm=R9R_3%=jTpdv?@S;}MB;=A3VZZ9p*NQ# z=k!54u=H8iaNko)RLoY|ZGJ>O-zcjJ*Zo;J|44Um=SNq$08)m81$UWnMw_xifdYI@6YqlX-aTxpM9oe>k*CDF@U1yT?SI8>E}16<${K zS0nv>X>u$+u5tpnCW9=fv#egK*Q~MfX}bf7%W_;cDSVG}z}94-$7lClwCQLmKIF4I z31{2ubMXcmt`bs};tyvmrIAJYkW_y$RW^$sUQ7VN&iL&blzFz!`Q!6oZQ3(|>A^1~ zz(SV?Ty&CS1;3Z&ZE$DtX_HF3s?at=&*O^g2?-hBIMQ^DEbNif9-bXi9C=Sb*V1UF z3szH-SI7Hy)48o4^u&+JcukgiYhIfR|5kU#5&Zg;#8FReWBh?iF!wj+CG$0q&vW3W zSx&EZnXUm)T5tWD&&hKV!ZxUrwy34}l7j5*eS0lO=Ea#(!n5>k&=a>@XSp$JFn#Dt z_hNy3mxtx%JWy&N>FjS|-3IMAa*muiKsu zFfr?ieGMyQ@t1=~l91Hj`Kzf|THM!Gol~)2eIVe~-ri=%&CfMcfBU=hW~^aD-k!TO zw6e=iJ135}BHQ|__HQII9H!rtZXe?_ZSY;&qsybTI(+H8S$A>bUGT%r>BIYsb=krU zF2ySmHZv%Kxa|*|Mw-DLadk#2YIK=-9B*Gb6}pZscCTS3bPa+MX1_xc7I=Zas}_j= zw3YaV+bSVK$TL+_|DvSV#?h|Jlv$^|GACz|jMn zD!!*r-y80ZhqIzJGY4VUOeozn3Tw52&W zeh}5<=2?636Y*JTs+3nqsdOiyX{3F2tGao7v=_Y19a)xiSTwqScWR1jtlDxm)-fg7 zZXz~)qJAQBs!$a-iwxB*nmM zAYNS~pFD~zRY&fSmuE0Y_UDAt@Hn+kfnx8+tPY;d{SgorgnYoN$Z9y@+P4V*o)u9B3piOy~*1i*Ak`C0OwyoteK)4Eg? zo%KYRSx&!BvDv-)Ag%nvcUeeDTqB!Z_lq)*llv-<9z9-;vr=-2{$bQlZhicE3H~}6 zrQzX{Zaa;Nw=d7VuasrAS$4~CX6cfajz^S(H%b$@8mey>vGtlOQP{s=Vn1s z7)%l;24Yqa8B6%Yn;k5*a34h@^{)1@-f2C#4A?t2~Y={DV?8qrK=9Gn;85 zut9^Ky0T28M|GZGvICgUPN7tQvT7|;|ATFOFYj8ka_7+N6!=i8O}B7dj|d;}EB8bO z?U;D2{*>ov8ld%0*`md@Zrii5DEaD|4%0;oiUsa1#i$8x{dcMR9di(+rh)W+#6izR z?rBdB^M;KLWHR55S$mPIK=Ukb-b}lOgD)k(c)m>jH@B6l13TJ|BO*gY9W4g^ia*KJ ze?3p3*DIRzCaFZ=hm@3T+6 z8ad?Kvnu|U%XJ3+mnJX|VXvV^8%d$sP1?z(-CJZF+8^XJi%RXe%2s_ZsV^%Jphdbl z$unXndwBt~hDRecM$W|RlDRXANRQm77R4jU$5b|hE4)1~?gekKG(jS_krF`Xgt0u~ z@_T<7)FTI)b)`F8MjmlxANEQiDeKTC zUQo}E_qbCTH}T=z<2+3fD8b-Nb{MG$BAwqaF#Bc{YiGOsm53=;(>d<4 zux3X(O|*tT)}p7%I6X`5jysEXl8@RGfVhYG)R?zd7xS*Vj{+FyJJoU}kTBzue|OBY z8b`ATq2eX*TzM$W9X%i_E|9yMgL^BG7r{^3|)KemL{op^UQh$x%###bSj*LY+Q(%xf_ z;VwRy+kz7sc=|a5H#LP9{V~mVU~H*P^bPApnz=mMn~fyau*D^u^(>BG=jCJRts}%? z&ktiUvAhr*p!uiCloN0zlG+o;VR0m%`jbSrVo!~qX2PCUwR`uoE2PStuoAhq+CdU_ zX9P%n3#54iV!9X6HX^1;taR~b#hK~Xs+*8*bPDLFDRhNSqepF!DE zA$J#WXnxZxs(tQZ)V$_!?eXJF`W}lWetX@K^SMecQ*j|^!sy|fw^EK8oOM}Mxek#u zvX$J|yWcGBl2`tkc|a{u+*~vsmBNQ2Evus4?j8Y$A|tQ!PhS;J!75{D4=TMBFGx*- zA{ir$hH%@F$ICNzz1kg6q*q>p*sp1NcWk2P#zB$sdEcm!jVL5F$gRRw3_nOR%Dum5zCBqz{wzw&#BVEyc6`*GzE11%<#aI< z?BQkeu|Jxq*qAu?%?|&D1RXnw(FE_K0Fn4DENmiiWBH(%#KW|RZu09$dnRzYjR$0w z{JQr%+V*&Id^^lMpCbovmCbUr$GMxFq`YqR_s_e2&kN_6l~Rd%hS;CsIOX9XAql9X zxUb_>8=@n#%D(nXKHW2CBR(aBTg_mjijL8j&NN%l-Kq~3Jr&DrJPvJyA-1OB6WQdu zx+XF6jGfm0}tcb%Hnzgl@sust>juC$Go zpJy)yPYouShbPSp9gGhWxD84RLlqfMo~JkD4Y#z{Gp+CYr$w#UzQju#8u;o)MzBp6 zD1L1B&T;RK}xbD@wG-ZU5^<051!yCF#L%xP4Guh`oCxbgM@?1xS*Qt7$ z%L4kYW%oE#o&k@EP70L0r;K#jAUxhZa_K2M+DJWcX9TZUY1uumfy>+H@6zZsJM^1* zcd6_=doKa#B21c&VR0d>GbQqA;4RE=h;7~YM4@=#hrUx;0UH&rso3ecp2=h=q*McU z!!MHj9CWYVkHXEjB`5@(ul-6I$ZK?0?&L~xTzjpPZHC!7!3QonCqlf1SIeY0W>Rkt z8PkDQ9+m#Iph)iFf%N0RHXT!~xS}Tuu>m{yt)JP1NGmb#-BqZNT8&BDuSI?8`K18Y zNuJqfm`<>K&E2#p%(hr#4wJbL$PhT+cdl04grn`h#272O6;y&3tMR2pE6!;fe08bG zdt75xO0U^if*6}-+HO0dhkR%m2gmlq3xvHv`M#G7o2UTaR-v`%oDy}bH2dwdpy^Up z#gUU?wlckd+Uvpbg>fUDiGku8F~8qkl0$U`9aQ){@89XtX}yFQ=74yInvOT|#_qSs z)`9pic3km1)4Z2csm`s;0nxD0lQyw!xceKgt5(^vyt`3zn!m}c&vApG6~$%Z-PO2_ zPbrPD{WS`@`Cnxqj7{7E(UYv{Ech$X`)Sff`}iW$)3*0IV>_C&ZCp%~EwigP1fWuI zWX_c}UTl24UMY0iar(Mda(n@9r1u)V>{J5ToB%g8T3Olciq1NTUZT*Y`-@$l^3+ws z`FWhNW2#zZIx_@y$K0tPus^Td|$G3wbI~4CC8MTTd)stDr1MJhbsdX?k$9X!q_J-teqsyFiP2Ta<&^C#gyWh`m8PBqSWJmIc^ zm!d&^#mQo;2z#0D>+Q|iE9`@oLgn^gnYt@w|BIY>7FAtf5>Z1xbfiMgba_P#Yx(@? zp@-Wbn*v6SKXBdQeEzEHpD*9-Cz_3NO`LcdP1X-=da{&&J2+CSFctP+~*FE-eDD+?zh85QpqHla3) zT63esR=7C^m>#ENL=VO8lr)lQcJMZ>>#8UlJ`X+nny-nG3KljWh4IJU6&hd}GDfZ9GuV^6 zNe#sgXbKe0oKOCi_X@`};z6g!R??-zl*{MM6UzqC4H@8b+3YF{;0ast%sY3^`IVAi z?#4YwefkT)&VaY934Hq{qOc(^&SQY6vD4UFQg=T|2^8CQ~Z~O8=*f2V}NsUfv7nY znKxORB#)(LTwgbPEb5QyMw+Xwe}7z$@A`hyx3k}eaO>!g>br!dlhm-8ZHt@&1@nV@#!%(~9B!2W5m7MZ2; zIvEJ}t(37vWrLXw;G2=MK8b}UJ{-FQBz99W*~9@%E^sc#^UQO**^tCvYL7soS|u-pB)Eb}1gtWe7nf&}>sOX*3=LpIpV=|SbcL-|Q+GFB zXJ8PrBG6xRVdd@95XGKn^B^sz`Op%K5^41BxraJ7LA}*~W=uTmT}n*Ry9Qc$Pdsob z1MS(RBs+6RAAcxr%oEi9S~RD$3;j<&bkU4hU5|^s;M=7~S^KECvHJv-qkWcRFg@RO zB{}#L)3fpV`&8po)rgdpyMyNuoS-SqK_Lwqey(!tr#}*GlNe?Lx}jclylS%k*uIRq z_M3CfoygMTGrad&ZuG`^M(*}VUWfJY{%}{9fKvUVa6Z5al0nq+l=;H(P48AC7ky{X zTkeb6MJDM(AwE1l-@HeuIoygN574sIQQG!vx=85HVchDHQ6JP%)2=Sz+LZk52HjxN zy^`{Sv&RwhI7rJ$;&<<})mZFcFSpZ?bXlzw%Yv~aF^z=o!{ehqy z%%SW`r0Bb+Th>vy|4l##l7j(_ih1d@UxF($jDws$m$A34X8DXGd)JOx;y$OOL`5Lr zW$VAG&@mZULLHrR+Ly1v=;e;Knnk(_l|R9G!;LY9zc_udQDOTR%>tskAW>I{ z`(G&KKD|mHh-t{#6EVX-35NvN=Ld0IrS>GB_Y(Xle&(^hH!Ny}fM5D!HJ)xEou>E} zdKyFj;Z!s*aqpChl!vCZ_o z-CoUxst>@2-zv67MgWby^vuO&E>>Ggf7qOf8?m_~j}FmI94`-j6i&8IOA zUGZNoKN(IN;5)&CLOIE#J)dv>QA#vhti?uqUx|(R0iwq_)fSb)k6fUT} zNdp#ALeG|!zyerp_er<_@qjant8KU#ANdy=Ze^-+3~m|*$SgUZBUI)g9QEc@vo(V| z4aU!DW(E4q3Z61N!n`{c_7&UQ1&U?x=4-%NC_vPuibf-A@ycuXO*KB3r&4v++(XiJ zM*C8tv}JDxFG;LwuVottFnM1fumoz7g5OU?U2NbFHF*5Ey{YrMxs4kGy=c&OX++2w z{5X^No*Z0<@O&R~m&~rB!y3KVsSh)A{Y0!R9|rlTd=ju}jdk<>&#{O(-^i*FVt(;0 zkSwk{pOJ_r>UM3tljf|jomxbLP-s60EQNARvSB@B4|ro7r}hw|OJk5yD5SLB+|IrC zSlzO=j6w&s1_0zooy^Ky8?8QgK$aVvl@aV;aTI5fQCQS;kVX`eI`-#0Z+=@i;Mm8n z*V4PI#8zDp^IUNvkOB+&$ET2+w2v;jL>%!q#r@NadFj<=&qi=V%ryJwjq3YvF#O~d3?%ywMBcmWuPnA zG_S^6(kQ>rsTAs#CVk${ll!++bIdKB!w}Cq^2RyI09mLa)Z1@dhR7=j#u$+?7P7Y3 zN6OnII-S2#FIi4}U0iPrrq}5oQL1-Tgdy`qF=F2W_4jvm+IKZ)b1BV2^Hb+MK8ji{ zNZtxsBWSE@rLG)X`b)jhT`r_NHK#kXRkRVDO&c+VSn-!wg~`}fnItTGDGtq!9AP|D zJo6FYsibX&dH*oYi{b_CcfYPPSRP5G2+}rD2YM!7cdrLxAd?hNqL^aHH4?f!oDK&L zAp{CZ*gk_qNC11#K+QM)PoDNWloT*sR^z;s>Yh}-v}ZKm&6FFqeYlG-emr)C{kUek z3pQ3Qp|@4cYBz21W?a$W#t?ALC(j>+1vKa{hYtyDBBdS)MZNt@uJy9oBUm$Y&e&S`jr4pH+anythCP zN!P4X&|=iiJ7cGdC~Naxws^AF+1Z_!R@e>1p1%z^B8v{sRIub~ebB{e={ zl%v7}QW`pn{ol%kIcUHq`Q%TW*Q#Fgu2<7OBBa*Q2`1_C#468b1hti&k^Xezq$=bV zXo#pFm0Z}!Xvd+L3w5QXJJRY0%PvQLCJy?I@6Ks+{{4>AMJ^Y@^emue9ENtb;O!=_ zl+3X4==JSto)SUXUsr5fHrVyM-K0){DCq#I0^cVT_yZ5XYi6?>gBed8&WX!LYf5(& z66eKIHwQ`#J%sA;c+L3Kc6g$*=Ad3!>?$emTl%iW7cP_TJ<`&dfB%&<|o`OJN_Fo54Z1_JOe(4 zOIir5#eO=y?@o2&#{8f(j)?pkY1;nq7+o; z%I*OP#^VB)1tNsBw3E5S*zy|!ByQ@ncyX)VDZzNNYn@dK3%Tc0?w5-*_oSUKw0$fN zKau#(Hn%q}cwL=AAkeOBH0OWNh1+rq<&1YN_Eh4=Og`hY+j-=#w;ho&`6@0$I%7$u zB7NOEfNYq`Y%fWPomM!I@1Z!|C}@k&(eRmp5jAu~K)t$hk;B;!??h~8#Tbr*zxM$6M zY=`Yx$5AB0Bv0$+-PsAEox5MX82hK5d~qN7#d#pPDxN7@Uq4amO?baUb*R~q6;(Q( zZ7|rFD{SQQhZE2Rku=cVQM`+Q8u18HDr^|d-U1=W=4dz($GILir+P~2 z>DDaFX^EU&->hFX;;S3*V?3<{2^11t5SP@D$>PP`ZDf!l3f9-*LDJJX2cyh_(URre3qeb}|k3nozAzb}iZk zE4#%RH=lsEp9YHS;0S>!t=8oKeBf;wtT@(My>Alntrn0-GAOOoEh@CbG~mUm>Bi48 zNb1?&*vC7YLtBa`Ih^HYOjwSaogw_nMJ`4I{^3;^Hvxi+2Y{#MU2%Fh=Lpj<9*fEI zd(tb`J9V#m;>2=nY`Qe%#j2I+6K=)c^UUkEi?`y%n2!nI0A(35W9O>Fmw~bjq08g} zXK6VNKwnZ=(~`R)5#!e5vVV09PG4Qm>@Y_y2ykDd+dHNCd~6gm+Y$WRU^yMvR@NPXMu9%H9|LF~(%+ouEw=syM2;HG_X)g!yiz2|Z-`oAJAz3i*yj+Qg|O{c{zNSh z9FpAygmDXGLMq^F1M!*bjG)|bl@YPMPsSM+}(;N*RI?_2{20$zfGYUHm9EjCbMv*@epm% zNrHpNcO5VuWGgZ-hfQdEjMC4XzKUN66qE4 zmQZXn$^pb8eSxWmsgWLv#X`^~&k1g)^#KR$s^EWy@_z+G`kafcMT*}DHvBn;@$3pz z*3@wsZHj2~oxAQ@oN1D(6<&JZZ+W=yE_T3FqK72wseGt{Nao!L{{yy>_aKr|xt2AX{ zDgfYAOK&oB9Cn84Da0dS@jF{PJ7U~~^!DT5&4ju83F@QzqLNsxxGDVs6+ESQ1YYSQ z`{~K}eYdd4gO=6xL4DLM&}00n8eW>hGw`)J;>1IU@WQOoZd*nc&JZ$L{(D-^?u4-! zGGI${xdb=o_~*C3H~_z$LWiu~c(DYh@ea=1>39wL*U_as9Ec&(Vv_zgLBi780p0D! zU6;uIX9O6d4zz%9CZYW!uGA$acgNC7xH@iv%G=D!>iw$3-KzEM6#R%kwTolvrzTT- zqF$;#sUz0XA)AR8;XUj3QH(8&cK@1^M2=W_qy(rddZ8Nwi6;2MQOz~s0QUG3($Qjc z{V@SW<)@TpJ+a2x8_Ja4qla3LK72?693h$;OL)a7N5fqqK_#G}Xxxv}>i!535Gi8= zH|qxk(E<~!L#~nQiNv)LalH2(-#}jVrr9`oKvgT21Nx>S{mUS#@hLgp|D@o`3P>S; zis!~ORNjG#!O0K)glv!WhjiR;=9i~!g+z^Ba5kF_j`n9B&(9h~&%U#!1|l&o&{47* z>58)#lv59d=%!ock(pr$#RIl{5d(Y6|G`4bZTfF9!%q)maGC#L_JV8Eq3?7nxze($#AE800WM{Sr5 zYMA1-D{EoIobgO;Cy7ez;lDK^fIRS_r};-~tTfsNwUi z8$JDEQgMu%Eh;2pjdicHlQ>TGX(KYSNRkc?k{qGvM2M8*S-ALuG- zrD})GrgPU_qn4SyqwTZEkP{#=(gRUDC>o8;%^R%?ym~Y8dAbQ8bF3|<<9c((!kypx zWNgM;SIv|84gJ+>8(p7p4}SIvxF7;ZSe#A=<2K{pCU4vpU%yp9DW5RhNTnjz&(C)n zVX_X}7UqU5=UVpd2Q1aYDB^2B-v&PSJE2t=TCXX@WSD1OwT!q)@9b2bVq7{pHgFm- zJohnhE#3FOV{`?|Cti zZRyE7ZT6J#1t{#jO^k=KAm)adCg!;IXp$^%Mo`~qkHVQnZ+7GfW5ypG*-|( zTTZ@XT|&I{I!$HqisOJf`!=8S7BRIXsyA5Zi^?(m5 ztm{AGKC;P(WjNQ6vUB4+?^@s76#jswq{(0KpM4P=aG;!C(1Ak@EK1Qwzrjm8(_e@hywSc9sq3xP{N-# z$NKnYF3>O4?dXjQt)I%pVA0N)TN{!3)8IsdN&fndgUf~*m|FvJ*&(JDBrOi44mYqP zUTWT+6)Imjpa>re@_!a6?h3+Gj~=Gw*Od--VFUU8Gh;JXp+Bbbx$0;>LQ4TCxn^U+ z+JX2a1oNvsmHg*60)Vmp)K$Bwio~%1p1@^xTR`i>dh*KtRp$c;Ka4=0-(Uh_-z(VN zzKEn53H@ZzMRXU4%!-+nrTav@S6+<2g)ew#8XIvW+JN)}FDP<6PoBL?x4^g;GIe?a ziSik{=@l@%j}nhgA)aeSQSWd+3Zj<&d?SY-46`Br=!PN_&s4#Yqh1yE8=*TL|Mg>< z=QstelQATU@3H5%+0kQ*3X+(p$Sc>eBf-~HHdHh6x~)4@)7ky+0`u*=?%X`>|9U3k z;U5iC63Ex1h&+$k!-29QNPiZIgh zHFH;An^jB_+pYeWat4ZX;&`w|q-B}+HbDCPH-ecjX;&*8CV>>)8n+5VKd75t>w6&_ zc&Il=sJt7WF;CQ$Nk2PZdwX`#Zxw4tdZJI_GF9;|P{wNFAaQ3u7@YFLHI}x7EFGDwyb?Rx}(j z4>Gu{4RJaf~ zoo=Z13a@&b^Gzo}x2;W?x32VlGj!g0b6@D6ZF~Z#x7!7qRH{8nKIPfyrN&f?^e5jh3<${5qih(b9^nB)p-eikdnuOv3(6tJ+!wu(uq2s zOQT^iou66b=fis#m^1cDX_(fTJ^N&VMBB6<;LoD+v^5(DmpL+@+6ZWA`R7J}N9yxrj)M zI@aP%FQjq@ap``@X12S)hStbsWDmehW3^mP-XE9&YpMbnplW@<&4==7Yz}4N>gnNV zQ5l7jx{Ho%oNm=FZGbV`n5r81!nASHARD;)JI(Uup}$z>K60~X|Hnx}K`VDMn$2d7 z^+v0BRyC+sVZRMFyK6=fzWW-vdVQmhn#L!OUn}{`k7GhJzI%I8roex zB0JqGIt>XkbkXtRXkyf%zwFk}EA!slhEY(W?)a+b>NC$*fHtZ$P^G($#BO+%bwy8z=rn}~hhMx;`_T$F6@Nh9B8jP@5?0cT7=9hS>+2Nwy zn085SgNf3(89?cx)CVu)`-*7xJNqaEwK&aYcnYL#x#CuKi(82#=udaBv^RVmhl)jK ztOWR$ED7DK`264OWW360_wgRrk4w?A!7$x7J`H%0C+hy!n}E9{o>|#{`7v2gHy4R& zRCoPjKB;&Lhg1+tT9Q5(3C%729SlS>NRZnJTv?;Yf;QsB=eTqtK0tWmlHMOUz`NER zTZcxyN`e!`yPG~&%W=HN3N~>MZ4RjT`Dv*D690-Je3qwfXX9k4#uC$N!-!QK2>B*E z+TNcql8OG<%H0oh+*EbEZn?KJn zAFv8aAg?uh(v{Tk^4@jKO%rX9enVQ zpVVXfX$xU}onL5^+9pp!!A0?1B*?f;90Ji${E(*3=s#n3$pJ^pEE$4yyb> z8%;aYEwrI}Ti`WN;I*w7D#fq%|M|onfa}!$i2^Y!JU~@u9*&&gMm8iKj{HJ)9brl& zaXL{T6Irh-2MBEdX4-Lpq^f}h;aXmw6@E~H@ENTWHQ63MyDmHUkCra{P270U=2jq&HWZI{g@^gn}f9oO{v)g~*jOjQkB$k$VjiYb1Wa zzmEd5R#VD|-13mBxEPM{S@-KELhbPZdPEGRS*INoLzFStSM`jeK4Ei!Jis@A zi8v`$KgP+Zp?8$7`vJ(b9IvW|FU=e|;KbAE5aVJklN6)75YV`*wEGx!Ysok@%D!rD9o7RA7}~N%zCZz*x{w$#gaAKeb3px1lq~<(>v|mBcBP{dwO7hnkai>o$LJ694<*7 z?mn0&6Fr4!| z_sYw%!fC^n3GR`@iX|huZ)U2TgQ_HI>+N>%`6s;ezi(K@oA3to)gG?druoQdBlTM< zOV?{mLT$ba2wi|-<)cgTT)m&E+qJq!n6`-by zLxUKSG{LF*GW085Jff)G7t3P7SE_>n(&wh`sPWhVA;ZQnuDs&1ImEi;hy997XL{H@ zF!F<;Hh82Ss>d}=KXU$8%pIgEJYY%bWO~=!yHROW{ZG^&PlE+e8NcRRl@JQq)LdAm z!kM1Ob%#|}7nWX69SJ|gt%;WGJJN~WbQ@JzTz*m=9gwn41;SwhCdfd@EO$&ZMBLaD zNIyE2?Y{pGJs0vanA#0(zw3cKQRXvq5g7!(@E!DImIrke&+Ha|k*)MU?AROBn8`PV z-Qq<7f9pt=eWhn*x2R7Yp}edUr$A0pc6QLlK+><_6^g!0WTW8HILnhu=y?)1AJ1~> zG`hJzGp|Ltciw|F=oKceg>4GAkY=0|@|fkx^`5cQwfQJ_$Vk+&XwP)b$&mU%z>s-^ z^>5c00ar398&uD}_v(8dY~#SU#S!RSmf^$O9g=z)rsC!eI3M?;Go)HWh>CHv%Q==Q; z8z<~7Hm3c{T$Wi$;>BA!Q|>>+MtAa9+-ENOgY%!ge9Ej6c%glgDDfyUG39%P)4Nyf zZe%lMntfxw8LD3IEcOoG8=j{2P_r@@Jjp#Eh;t`*Yiu-FW{ES?P%vdIZ~V@jN2{*M zseTsKdX%nv1&iq;HTPpXe2rbf_-AI^^Lw+sXgjJD%6C*!#|qw7ahk!-c2DJf zm(-pj{$Rs1t)PsKRvX~`Nx}tD8@sdmReRE9R@8!p@Sk6ZExL089YOZQbCd*jM(Qq8 zWKOFuV@;ie0K0|_s`fYcFwC?!uKjwV^6I$&xK;Q7vv9 zT^M$jZCzynE;S}*i24Fwz}@8<^$}$ij{Gqo(1fZ8JNqL%qho6JpX#Q%6EAWW*jy3YhBym{R@8sT^z>QMwA6aj48u8TUJ9)!(# zKW=)ku74hWiC*f@g{E5N<(T3{iL-WP>u2OV$c>e24BZx#QGB-tRK-~oe{k?`c<<-f zk=mq|MQL|6`ZNT-X{zb=jfxT$jTR&p`oL9RYVZ0=bliJAquDVks%-I9E-yTk|0N{K z#)>&kRAVN`H1GZ^byT0`V+XvV^r1Fuob=Qc(njN-}jFb)ih0Mi5pcW(<|0nzSc0LHVLox9-Jbu zHX?Itq_mY)nD}8|!0&ZGNFP77GD>s=j=thF^v)G_%$qfPW1K0@8C7{3*}vo!-L~K$T#;C&wjB(F7oV&g>=}2vhaF6uhj6d z#Fiy#RGQ^$=SaQJEfGg}K_D`KX~-0gccdqJgXPvyopfMn86e2~Ni6YW*Q`cg+_dFJ zx}Ao?f#ux6d;b^esHsBNFFmTP>zp6*-+Q`>OL48&t#=W}^=c>t1cTofS=F9<`Q-$< za+IINJ0z@ETwa$Jm0rt}8h1J^KFQ&tWQ%k6D!??M@n4ddNrNnD6N(eRI~0Y>@?w2H;JY)(!_gt|M4ct}bNO@}k6CZ@h`Zsr;dH}tM$3+u z?VI$rtg|WnkMneI`Wr<-Yqm(qfq?Q!`l*`4RcRt{-d z_TaK@_UK@f)iNcE{LsXs8sEr~Ft^Q4kR(xu525e5&q<9gBM?680;> zfjeUex~Pda9kD2H3_+(z>zT6nSDUSE!w$E@D{f>pRZkmQb5;|Ov0=z=g*-6D8kY{i0+b0Ae8dOY`4`2!rcD+S{mqS5qz9hBxfs%V@m)YJv6b1z)cn6)hjSMKJiH@H#H-c&6%R9tGd7&yUEgCc+tk_LlCy$}#|cuv0fb;V>0ycn z#1Jaz4E!G6N?SQfcYmo5^JHZDnG za+4(`x_ezpBj5p5x0*prWA7EI@kjaKKzm>a<}0bElIp6@22Eb<@1JFnuO0go>w)c( zNw*taz1yY*tzShmnOayd7eDnIvF1w0kfP$OxiK#YzNkB+3lrP7ge9@-(SZ{lRC^-( zqf?c(UJD@%G>WE{z5$OL>yv1GjDK~EI>d0_EoN957*{6;rm9uM;D&B*v{(iY!h0xD zQM&e_Md31ZBa8MbYHw)DK7_G29XF16uRpLL8~DJ4$PVJ@UGTq9X8<3-g@f(?Zar%< zk={bPG;m>wEqOQU`~aBYf++n(E%BbNdN-3P^!=0MHmK)EP!HqZ%!%|d6C{7#scyl6CLJAjFFLy<= zyQqS7x_pBY`qtvx^Fo)tARPBhVKiH6FO6W0B^4UW{&9CT6{wgVyA_7X>Q&Ma)dKyD zhVX4tJo?cwM4K&(_=5VStrm0~H^?|W{9JWQJxt0Q^$;k5 zzOzJk|5(e+dyiX`&Vz^VQ`wxtfjohpd#{7hf?BZw8cUA)w)`>97K$yZkiW%;9oL*SdfkO@ zfBR?oGy` z!qyxNWc5lWO4Q&hf1|zFBtuI?dyQV4aW*n#)*>9saRSIwB6dHl{hSEGEc__r_Y}2n zHFru}E5hn%HH&&jukQDiGa(pXIwZ+v-!ts{9UoOt%j#5gPv1X}7N%9F)1j$bP|*@5 ziwod$xwwzrNC>1F{nFBW!5cZ^l8O@~j{Qn>;12OeR=jgS$@+q&`IQzskCVB_Qd0zz z4~|c)X8rxfgDRbn(Ia%$>@2|T7)a?rKK0;Xi)i`IK|;*r=Qw;>I`*!Ck1VSNczFLq z-djgS)qM}+FoF)T^hhIv=7j+WJ?Gb@I8P?Wn8rPJx8|f)sD_t z0uu;36F+H1C#MDE-w#xN|9rG$EVs$YdB5%yAJOh@cX+2{nld)EC5y5HGO1eVc45eF zOUU*hkkxp*<5hFl7l>>2l5G$2pf79ksuMh+3GMQgk-!(KyH2-W@R6H{e0af; zWod@HL&JV2vO>Hg`s*6tig&oU^S}t@UZ}CW0X(6C*Z8~y}K_x7X_2U-S*LN(23gqrvdq( z{jVMkTpo@JLN4(5ao7s`w_>W|C2jGRT<;@k0p*CRtX=J!TSqUaH6ZU6WjQq%qq!0p z^fe1J#0sqQOrpO%0B5rf%y?P%gimT6Xv}+tspT0BC%Twg2e(T3(?wKDBrb0N)Ol_( zT3IxiqFEibcb`!`V(jI+*i?isEVJH6MK`uwMG>uUvP0mhuej%iYNXbjhw}-mfJKbd z=ezh{pPNH@KW61ZzMcYhqFP{6wmvNY$Ws!6-}Z+OMXIjcFIoy=pVXw+D|uM#+3YGc z(p{#4Z{OmCr{vexKL<`FI|qTwm=@JlqXGOto$q&chsWVb*;pFZYX6K<>do7=$VL1 z?mEC|pe98?=!8Gg1|IXF3$Q0-gCuZ5a4jLh>FXHGu0EeBh$tN^ZS;to4u14Qp_ID2 zL{CnO1#|j(nv1>9N~qUu-o7oCp?Ok@zedjisSCGrNTe&R+*&&AA zwjAXF$ruf5`*S&K;KDo5;2gY z5VLCkD7K+@MjdPI{>_N;N(Kmc)*m7%GciQF7o&KYOZZ~Ca7UUYm+v7VGrQ* z0R3V9r9Wg?gR&w7V4nDxXaWxe^9$&JUG-+>1f!s-ct^(jnQLe?s{!C>Svz@m_d>JR zNifbX?WH9$cF+6UPdoQ~Np3BKSn;dI_qR<6+jg@diP^8j31c-ZQt%2AY&Z>izDZOH zSMK;<(MQ&|9%B&o4U<(2dv$x%R^<8*dKVB?v@e4u1^aC0WKYGGHr!Ht+AORm2ZxS% zSj^ysQC1qVu$gKTjq_oRj=BQ*oMfp&>*#`CCYK#{&V%Ef_nQLzx*XPCTELgomH>73bnG8n7j+iXf`3Vew)0cr5U}y*L}2!ZoK$ z0}Z{acaG6+A?h#hroZrqc1_5RAECNYM-=}_+RfnnIEp{8Q&;?WpowtpG;@xnwRl;nR{ zwn&T@&$XSD)g;g~l)im)r0uKig5G0faTyBw;yW{a@uS9(R$`e|uz8j*B>^#AV{S;v z^G3NfRJcwnM{cPS5MyX{!bM$|SVumSNkE%byBUOX1SgZLAo-3F`|1iOLUgI!H&KIupON!jlkZK&r6-Qmx{% zY!1LcK>pS^AwH_!Sb*{kY%)~mOG)j~v8uG^)8=bgUX6U)z0kr2p@OEj)Q5jh0V+6F;LAk0pWJc+O ztV;Kr?f|6#fcXLjnyhm@sm*@{?D~VqzAcx%+`BXRpi}>;=+KCW#Aqc)(%thGz}R14 z2hYU|;^;qW;c<9-_LBIvwfW}Gsl8q=|Nj@moS^i8uo%C82m2Q{6Q&HSyFBdpB zpw2ON2f_!PDgFFfybA=&rQlSZuFyQ5M5i{8i%)h8{Ng*kXkePUsm?$0-p7Jrv z3|2Zx0mQt}MfPZfx`RXRA+5w!=oKOlPMHNQpu#UKrZ@i#y6k6&j9H<3`9>2 zc-0zBPZCVx1tqa~=1$5ro3b;_G?8?sdMl_XQ4-hHEi~#Js@*<6f@h4|a@ZZR3vKKi zL8kN1WtMJ0mdy_CJyR($Da5Whm$eh-pYLT~J$ZH^mX^^WhUdnDy|Ezag+W2MxePy4?Oy0_l%81ZQNHqu3$5FSAyh2jC(|vWbCJ# z9`Dfo2hjyd`KXK7kD%4dNzNjMi>rENQvR5axA>DS8sE((tnCP@D9zCd=_ZSo`1a_= zS|?iPx)@9^Qdc{tG;kyQ3B+YVicdFnEBz5VeDcK(y@>mAwdkB8q+3PXOS@D&!){8N zjUTh#p1DT<3dF--Z5VRZ>GlNd%l^EOfN*}O@KqZov?cTjmfGc>sdS(9o1i2jX($Gg%>I`}dMK=9vMM)Px{>6y19HD)0o z!r&?3FyM9zQ7<1c9IZvwgC$Q$WpsYmN59!p6Hz8FMbmti)`Q*rkf*uT&_=DL z7qJ=Z94%sImULVS1D_zvN5GVGw>h*TlMi}=)-p=&jdyz^ZmC2R6y}q9>x`R}gpY;A zk!fj;(-MvL71F|0#rj5K1Z&ha`vth57j)Yn!JEaJJtXgzFIk$NRvi-*@A<^4#*(RD zW=Hq9=oZ^?bbR(!cTwiZaxzrq=nyRNA_2*rBIEeEtUKzYbsB5NojY6yM}L|TlVX9M z;maKGQ>ynQA`iv%YaW8rNo>Pl1IL9(sV8pm`b&%}Dush}Mgvrqx03Usz$ba^jaDpD zLYkFUM)KA^PKAb>O3-Qe>(dGW2192&q%QhH-;cTdm7-$69sF8Zj!>tAz8nv@*YVXD zxV%|pRfH@0Q)tmc3^GR&4mnGZ(eQdZx3iLDhnItUbQfW)Q!a|^h?;r?GL4VN`AKyh z^M=Q?5EnEB&Y#&XI@bobI>XAG!u#F_B&qi$MQ`qwX>v2pnl|yCTlqe%!G%3C=JJBA zG=DSEq*X%~9_qodx80#O1DnwDRY$oYjUBf9ELv9xS9{G`q~%#_e}S-w5sML@@eMB3 zW%z|y|KQ@%jXBEmkx)YpTE#>1{kOLh9mBh30}N@mJCH`O%h&F+^3D&(N{{nwIa``U zSJDgtcbb*DP<(Bd-uPQl6^ZBPEt2!S0AkxIl`3skrezbZ&Z0URn{XXw)ujv= zr?EEx!yG9ZH_mz+;eD945giqDecY}2XbDIcN;A)3hW7aT3FOf%YBi?6=^mSkubzr$ z6?hp36o&&mtN2Wqs8Y4W?Y6le1PE7^h0J7G;zrO9mApUn!fI8-5((Q(Y`>Z34nCEi z@NY35FK0etRS8O$`fyG=M~GE zUC6_)g|)3No5isxS6rQO!P}2NUnj+bi-E(!z%~U4P(p<((j(6MRtqa%qMN8XMB5KC;5=3|`k>k7qGGHM~m&Zzz&CY(NLeo01 zdz8Cdz8fKvyxtsVX<{{xLbvmlvA&WNQwamU@mBjbvuY^`lRkQM#h~@Yd!vfW}!%ER=O{LC$rR@OSke(G#TQ=m{L%a#*2K?J0?VHmN zreEz3{;YrhT>k@c$foLcsDO(1yS0!G7<6%lBx(!n4u3wWH=YR~mH3l3wLbxx%aG@C^W7_D z9ibi%vlQ2Y1mdl6gNK@03h>UBMenma zKRBQuDwRGMbSX-_=OWhUWTG+~)CPpy#*I!29+3-p8M((|(7Ug89XIuyHQQBBGnQ-6 z7k7ICEWAX=YGWAFuYsx z5o2|a0ElMACe$uTgKKgeEz6j4T$PO!CTykNHhWar*A6kgK=>*|@D+q=xNDdS0a_sr zWEt)FW^CK;Q<0O5E>Cqstper5e3Ljm^**&V4zbXR1RK(P^}+D`y!$vb)%}WOf$x<8rc56TpT^l1US6AWmdj-e} z2;UNs(H|Cqafy3)j@yy@na|HGr+34P^Ifjk*TS>`LkbXX4#eo2t!Gd0ao4~LV5SI8 zb#&S6#{9jg{-W%V4chU-dMyM`qt4`k*ie<9-J0g zJ5Lx8%@>8%yYju}{p9#tTtge1V&^0f@>^gGd! z^X02(L+6p|PKr+C{^{sFV#3k|36I~MnDFid7A8(@96^Gh2}^_|Ttn>)=JLDk`hhVr zu+@SC_E@F-#++C6vN>J$A-4&Jb4{92c;At6&2U9o52{&Jzkm0HwybDYU4HrjV1z2m zJQ;EgD}|mmG@~V?SXbn*gQp>$Kz#gA>kZ7Z&tb!M`v#qR?ixbY746&w%r3YTRvMRu zS=HtDTJiHSkrW8M4jTu8JlQW^gkUHUg-rJ00XaYqhuq~f&m@+>#87dq!m~u}cNgMA zouoIOO$Q0|)%vdb$R|N3MD^+K#(6qyC`Yp{T(NR=4DKB|PW7y=y@+O|bgj`?h!49+ z`*{=&22P0ILDDRjzWE7=4DMn^EKoZ=5bfr6$RmOi-#Ui5q6RTz=u{4Qnw?sNSG{kc-z&ww|#(0J1W}obaD)@23oGYv=G#xbzzqgVw$eA ze@<;N-U`Hg`H3r{S$AJ~DC4YQo|mF)kR8w|Z-Rhm#-}~yzDz+ow{<4Qh>+>%o%y)D zeeYUvbNXWIgV!RfC@Z%2SV0f6W^~!J=*n~_5)bwZl4Br9PE0Ke8_($L5GjZ)gCBH-1n`DbJRh8 zh45%VegnhOv%R9)q@H>BRGvDwM1JF$Cvw;70XY+n@Vr4*1TNI)gP)z9i@-P-Gsfe+ zE;31>bc!M)n)&r5+lHX70asTlV;czRW>}o?K04`PlI^JIUvIcFmZ~m8R&I)n2GAW- zt1a-SMnIi5T3pV+U|YRf(4-k&lPw{iFYWThCw8lKvki^dBV3<=Gy)kQYceJI5Ej5& z^*sD`Im!F>@JaZH#uO1J&exG=LJMnJxfe)CnS@cN_YSe^wOkULv<942X+iBw#zLC* zS^W=I>P@&qRnz3CHb9K6Tvwvf&Tg>v>kOAy?Bd@UkCv+v=>2=H{QFeWxdX%rF!yr@ zR@&;n0EP&0XOX;$hAz>kaih9+=79X+B-Nhu${-}XH0LZ~Qt`K=g#kqxIFmW)RkUtK zlx(cq^`$0CkR;n#PTyZgH%RY#4aEJtQEIK+nvZB`<`M7`VZ4i$$ynCdJL(;~iFcEv zT56W2s=P1VGCP$Hz2^d(hp^yYzMu^p>T|iwrP_i}Qbtt6bLdT;P$fzfp?c6IvhJ=ra^|q&9GnlEfHuEt$Ohd4_;%*r`DdA}D zsoPv?T*)CUFQFZfENb*oXC1khwT&{Br#QD9kOT#AsP2#Vr z~$6Hy_D;jO3vHL4d-7w zZAj~KZF!mYoUG->IWG2r(9gP%7T+L4JSPFD&nJ+^w-eCKbKDm{h3uzTZCWsZV)C&m zmnm^+do{QFo)2Hmc=ZQJAP|#$Y(5DW)O{DdKF?Yn|8SkJMv0JPFa-2V=s6h8wc5ew zJF?P{jN*KnZm<0PL#bDVm876gn4v7}I55dkO^WU-o*P6gpoUue4?rZ6nbX9ZPe7D6 zIJQ-doweFy(V>um6pNt*rwItLBID=5K>Hy3Ubpq!;AYeQ*QWHs>}EeSs{6-|^BhZF zJj4Ur85BTY#a0GG`IAK1tgb9T<9jUW=c_?qo(v}R4@H;%y!0HR1|*2-Q<4BmX6ulK z0kbW~>3U(qSVXgG-1w?Ckr%eeDJUAc}t zxsC`Mq<(IAp%JU`=hyeG&)t?|Yi^WsjbdI^I+;R~L+dZQ<+Q@;aaoA-P-tI-IJN{8 zkMH@O*c)TSC+l5N!L|vMqOhnOdpMHOGGpvdI`-_emr3Q`y%@|@%sy!wIcN*9%c=W2gAs_aXPwv=Zmp+ei0RK22@Y zY?PD1<(7*w%&5iMiN)@$Q~KlnZ6e(bQOJ#8a$+|%tJcsU?u zR6e(+PpR##PF&QAW!LH3oQd4}lzPJEtbpv+LBr9u#RoQ=FNpEeR7216 zgo;+FurOBkv!_RI=M>DItObxtGH5`6HuC10q0ZFe?stjzpYOV!w+SY2>V9bUC9eKb+dChN@((~c#t zs!7y#7?nAD7<&XueRx@K!j;{kBLKrWuMX)c{{Rnc^&xX1KgNn&a3o~C4XSXzLW*@9Ugt&_+u?N|f#+`<{CNr`UL(;Rug z%`T1xOWrC8^PIcVmSxE{$beOgAdZVSX66yN$!5wQEH~1A+IadTCEHEO8%28oE(LFo zVJu-}){f+ZhtBn9X23@

    ?DMrPNil?mG+b@}n(hk-4DUzS`HxkKVY!NgZF_kggTp z>)~3)++t$&i5R^gxlqUzS%FT%#v^D;6(cVBs2LMMp`%m*tfdGp1F5FrlNG+R^{#wU zUgXJ~laOvOnt+xvNCgP@CO^XDxDzSj9}At5C-_McYolSSa#Hvn`0SZ`tR~4`^X<&$ zAlV2^ERa8je7?nTY_q+P8cFV@L^GmW5K;6#=`H*gsj@f;tG6kTZ88wBrkQ6$>r>q! zBs9AHF224c5}Or*c%Q{@&OkhLgAFORV&QNBwL ziXuw@TtGbCA-=MCI?V9MM=RzmG1ezNKqBqZw@1WJZ$Yfq?YKI|of&{ zfbrZ(eB0x6i(;Cyvcjb=`o&m(hhQO@A$P%x^W!v2(wY98k!HIKSObgCCQv3LG%rx5 zz(6znV{7JUx5PUikr+zj+F&um!>MbEBO6ubGSS*>m2>(h_3vsU82qj(VXHg znP7;%$v!KHS64G0LaV7FvhpyRHNZV{&(YM`Tdi^YG~dVjic@z)cg5cje)X_3ZZO*o zC?8fq*BTu6AYIjzAN{T;;cZ35U-=?38pg<)C$ilT(L%h7GwpDZClNVub}djl;Z#d_ z*)E^Z$ZRd!sNh~x^O~BDMfgeyYPh*g%ks|q)8WMpr-)Ba1y7u~Pfu~+l{A%GZhEHG zXZ=xD^9E=`MDxNtA~m2X%WML+(3i~V)CyYBL^8JDEM7U6lLGV%2mxj+`I3fu1IpQ9 zBh@+4P|E$IHIH_)_tlMCue)&MI@)_La6gl?b6jVO0Qvp*UR3ZbSX}Ln>DiI=w7(PT zcj|rlz2i(SGenp)y|wLjNn&P@g$JfR9Ed=^%Osx3_*y6|GJpWk{X^-if91kFENVq2 zE#scuv-|bHPka^ao>Hp*L%<-oKnUdO1AOrgR^&_OA5C)bXrMGiG%*p&ZvcMBr-O-W zTXPB6(C3uufzl+%4w0Z`xDp|&#Cuh(Y=HAW{uK}P`vt$6;DhQjYHdBs9gO8nO&RI+ zUyJA9UR(1_@ZBDzj9kb8$r7z#{)hkW^yhv5JbQ5uaGrCC!LF^Czmg-aO|K}ZVYZV% zH7Te@A^CzT%2^EC-#&Guz#Oa`rhX-xB3obg1E=^Usy-kXIo?@(TFdaz?K&O!`ad$r ztk@y-qU)MRz#?6*)PMiM21+!Tz9chz@^`@NzD#WX3iSG4!>PN$@!zM8_DTP~4p|_F zE>s<<>;DA}NKIk_rKF?)0dl71LwLU-6Q}#D3;e4k0LTJynT_W#INn}L1}K8^2eN0# z%<4mt$6o(Dy1vpO`&rFEahuBwn_i!gT=-XI2V^G(WOWG(h~_^t{s)nNuigS!kP(h&VFWe#@RzT7dZ7C|JP-2lefKS-c z7dJKTGHLB-mFQ0z{Gf6Ic0N9U>AjKc0;ILF-vn)u&Hl=h18|2ai{wtW znU6YXea~}O-i`mF&TsPK9g}#V#_eyGB*h-D<@|+dIN(j;1iNEL+`e_uQQ1^ z0+!<^jPt}AcGpD%|4`aBSv>+D>k_iZ$HY9Od<;TH1OA}^-?N3W4Pv5}eQzxK>y`BvB#z!mP23NHNf zuQy1BAmf7e?7txWq48^CyY;j$ss4Z%Tm%69ZCen8tYZ6bsQ+~fM63W>YB8?bOdrNcPtM^t}C5q|^NB_6*tCxHBW;D6%;c;g#n;w1kO7Eo^CKi>aCxph4H zWw_`68{Oms^pEM>Kg9lb5u}mlJ~V~j#fJXtrN3^KYmvQN;X_=1di`21oOhx%7+XW{~#54_5SgE zpAh1F{c|8z1YTxCE{%6NKUZ*lK5~OG5u-B7h!#HIs*~84KJZfPn3N014-rI;R)%Gt zO7Tl>;qwBX(>FE|_`geFf{9aFyJA_$EFsB-u@CANCZ0!x<=ilaaZOuI7it!1moudZ zUe}>!QuilP65eqQU!*F0+?jc@w-O;&lQwnkHCpM`o5TYLzO4#F%Kc1+R(H#oL(*~} z|C<~wA)Xi8)1Uk|eWtIXI~~khoDw3YUmpi_kwpk?*OGK*u40LEY4jNYw*HT~HZ31E zVMRz?hd9Jfs?JN4G~`0|!n>lTrsg#JgQ5rXWOFiu!jmWMcrjF0(Ye$OJI9L=V)rL) zbnpNi1lI$W2^Dxi8-VCP{%~XgiEAv9ONndYB^Bntl?Bv|mPK~6XfIy2Jky-rG_6e4 zfm`S8zLlyuH?AZ(zOEROt|XqN?^C?E^yTy!B04%+x@W|=bKrq?+5D_5?0k|zuj;Zs zH8s^LdNSPkzgEm&A7Brift)sfNh}Ne(S12_Hg(0oLUTCYLul=7uP^V>^V=3&d(Ari zMevq6lHAlN*aEV=W_kj8jJ?Gw;_3ypkZ%sbYqGE(4gmd7>1t{o29M2-7v*i)B?~w% z5B+IBfX1)HS|uXu3vkwCfP5?-{-j#;F zM4`58LG#GTk}J-?AyPapj=ty1f!H9o6vZ9Va$<&JA7SWz13=^u)SYXXUzs5ly)(ntLnElJUra$0ZQNzw0c|e zhw+|bg6QViQG4un;;DB1QNVt`e&Q%#w?>iGe2MAtCvo_Tw|v4cbg3+M-%YZXaNA>p z=ukkS;Gc$USd(#VN6H1=D^?SIhcC>#(^C6L73CCVWj}9{;=SZYN5u!TgKCvHTiAT{=8s!$(2={R5MuvfJT#6i39@y1$O^Jeep25sIUr0lyh=SxOVf-Mc1H zdf5>~E(4*9ldMXo$`XV~7~i0y&g8vqf0*?F0Hux&7`DzYs0g_J{lf=?4CiR}HAfe^ zPv?dR@UEZFOZcUXw3HJqDMh@r%j>kTu&{HNdn|lC9uac9;g?yZzyO!{eqY*YEx+CP zGa3pAmEqybf%F%t)w{V-kjF~e#M&lXReX=(Lx(Vf+tzsIf?Vp0FY>jjD^-6KI(yfrq)8$c)3c9{Uq5^F~fnf-x0>VmO|5@`lr2bTMG;N!40~Cr?q-f%;uhHsV z5zTLs88gXLqd$DWVMXM}-*!L_6q$xGFUlhp!FGBRxS<5u*Zz1*v-@`>zyOAE?BR)D zsRshnJP80+i_%|fU$DpJuzRk_lXgT75Ru3=6hh+sjKuE}XyQf>;D?Vh_|_l7KtaXA z|G9feXI7(aY(0R6kK%tWG5_fnt$vqW5Ua};PFh~Y7tLri33QQs@5h?08}jw{q{I^l-7GyiY-SEDOJ-|6Y?W;YgzvGVQnt_!Kg`*P9Xym& zDS*AH!r>Ukn`Hiiz@6Sad|9EQBcvi2wUq!z4>bGAL^;{Lb0m#IQ|&=^Y$g)-&2XwZ z<+jW83axYU)f-?Qj!J%8n|AtLI5dpZi&lXx_rVL|=`QQ6OYhBMSiT*OHZyySE&K?^ zvM@VH-m=AjH3UvkYQ)vVHGGktNYs>Xhb=1sq;PZO6+EE_d zj#IRoN(~nEy=D!wtfPt@TfBNuJu1arR@6#_nt_C=XjPR~PiQ6e0>TWOKs-*vL^Hib zVw0F6>TzPIQSK+KUNcQ!Tdd_aQfuZkj$SPM-5wM4Oqvd|=@E(xtR0ePh=U2srfnFH z-||jXuX(W>b{Vul&674Yhbtq^i7vX%$BG%l^U z%)DsENbpKaD&|}nbqd>KN>&toS9`Aasi|&i*SpuU5X;N+X*?0tYw-!f)@YMwLi?-y z*KfgX6La0$XhU7An8~x!>Cdzc+p43YxW3XJwyipm2Q8!APqBc$G+|TPDZx>3w(e z1TaJ;|6{YNXqZB`%wlueJY)Oa{1@PT5k`?jHik48N*vFd22LTZ?*!1SVb=N~WK42V z1X0T|5vre)THAkA#(YebLC*?N&RG3?k`$OSz9N*8P$#0@`FiPH{CcI6@9>H1!h2Q2G0*fJzRt(WZ$exf#TQ)G!jKu}Im! z9ntby!F^x+#92#hyPA0$aFo(%hBrfO4+1@tM8S=Jc>ARj5fZK9bU!exmuS?Tx7ue9 zZZ@QTCr863;&)$r*gGQ_M|)0^Yu^yAc%Le3O;1YAKVS0B;8B3N>p)|Qkw<8;H*j`T z?4>iVxoTH>JIm{_rC!NL65bvsU^b6)#2L4pA}gd+lIO8hl6RA2YT1gaV3R4e%ID2k zhtP(o?a3{KIZ?#$NH1+|#JiNS2I=`8O4L->hSYIr%Pdcm_vozxCw=sBCaM5GBeb?V zr|tD*)s3zrw`Dk@(9a?F)j1XPg+ezszpd(5vbVPrS36Y_sj(07Q5w2hW%Y)8s`JtD zW9WeDqwim= z4~flx|5_ga+&bY2;SzGw{(t`7umVK)*=2Rm;eUqiUyxP;4A+G8Y0SR_=Kl;bJ`8Q% z9%DK~y5kqu{twXp4JHa&&s{(u=rE|@{s!~MBhalQ+Dx0|tBmY6h8EBGganynKHH2R z*)JZM=C4yixKU)y%2aGL^UGAW$o7HpgL}%Dp}dkBV%m!@Zc>iIf-osZeF7_d9+m_s zFO#s@@GaIWUo;o4-?DnpW1Ka0sQfHzs#JLl@2fL@SL1*S{=)$AnA?Hr*ng*QMI<99 zmsRX*m&F)znSLAIpGo6Qn=2Y|Fk<5*D+}@7-{#llKjMR^R)O2w+6J~$JXgoqQdTFQ z1r*pveXc(drf>hWd!D{f&C41YQ{f+yU@cG2 z9|HtF0wyf8UegM>KD4j&5_I5EGq4*cHfb;7c4Tp|TMc}?NOA4_Nh<99b;tr$2*?~|Jrzapn6gs!|&ErgoX#LzO8JEC1WvDN~R}&K-6;@uATenWv~)Rs-@7( zfD8(E)$mSB(U9-4;vEvsa(-|7a8l;t6Ru{17SDzdp>;_UI z!T#vNfV<-zzmhJ@c7bQSfGDrP`Al=_K@8_o7yrR~5_Va;aG#X19qZT@K5Lh+o-aM8{lMu)0m^B`SG+;GCd*@iQtsWRIvrP+^N4kKYo_+TE8 z*(9+BWd@WMyTq(q#m1AV%wd0Rs9kP-gqo!byAeHs#euUCpW{{0VrW3b>5C!^af0%|3b*6N?xW zM{{oL1Lvd^0~Z*lh*gIgTRS)`Aa(VExca@F5<9k!ir%{7ZrIgNF21G?Pn_l5slmZ7 z(go7vq9aviatf)m+&v0-_vs>yFv!cVnF}0h9R_e6&-^k0mwmDItDnzDU{9qf_WJmH zQ(?Al^a@wDxq9(x(Q2#RO81A0ZR!W8`CM{#vc&SX4LpZ!Y!D{*dwue@*Ga4j42Hw^ zr+%tbS?|2cD<(;1=?>KqST(q%Y0Ewp$A`9Gf;CfUR>AjK$~i`|-9@&=zI6RUP?_3w z70|$r@8K^A-7fft=|(;=9)8hMQc9i zOXORXOVu_OpKBJYt>i*l#ek3HpoLCc(~ORX#E@7d8?t4rS_w$9tfID+cX7rIiv z$oUQibT(|HT$CU%{Vf#5$?+p|+SpQCWSrN>T{gBZW)T^jrgCZrNuyP6I(+;k_8%P) zq&a|XgKPV2K zWxC!N$>TvS1p<1hG=krXJuAF=-q?vbgm}74s7+cIGhtKg^7kG;i-YYyUo^v+-F-9k zu4aQS=F|_Cq|`d??C+^D6oyNydq-i)k8R_Z2 zk0&>Ss~l-@1Td-nu8?!M3| z;L2+h<0muksjFPcS5H*c)q$RR;3RM@lAT(d_D@C&$7^@yMs|ZWc%h?-pQNf4NiMRa z!v}Lh;`3FPfbTsI!YO%&duV-!4zX%?q3_k3G@VoLe5uTwKn3cLFUX6J=H^dG`nzG1x)xu;eO-pKL{hW$Y!TnOeZsRL zqZJ={&JJn7>RdN;OT^$&e!Sp7z1MzXPh3PczFsc)QmG6@wErS&<{{ zBVRE9b@(F*_ir7St_ZAOA0r-dK=Ihsr76X>rJ-`6nT*9S2D64-9X^NGVd3TNZ ziXp@R#w}phoxg6*5=jp+)Q})*suFKI>6WTy1ytK&wI?l~W3^s-9#IL~2SYVWqo0pG zP4g}8wLLLgjwjaQXOs43@)NJc|2h6^Ek!QNY6SZCM;7CzYr94v7RiQ`%=dYCVm)T3 zBMfPH^c`!-JD)pK`?k<)BFLuVg+Q0TSFlv8Eag*q%7RbnI>PlafnC$XsGB6wyPT z9WVo|S;gq2%o%jNNQ&iB+alk}_Z${lFJfO4UHN1kJep&NE#a>^LG&%`7k~If^AXvn zVRUk&$RK5`bTYk{rnkT@dF-mVmnLSCRVpCX#>UX_mXx~$`!U)pM}vGHxW++%!hJq# z^=VsY-l1muCavD?4e^6|e^KxcSEQZlh(PM<_o+iXN*vJIan_GQVH^l6MA$%FsOp2) zNR_BJMI?fdh-kK=%ZGN)*Dyb);a>iF!*Xx;b1pzsjjvnZ6rK#a7kVZ+-OZTS%(%np z7^QMbRHRj^U6|bD_Rgfr4&gJ){rI(bKGDN3*K2WM%)uKFW;tElldCbkQD9xix!+3$ zUz%X;9g|w0p5n^uNAkal*eDAtycZ#OE{}mjit*9ygBAJK&<)9hvl3{UBReST8E}q? z(@9a_pDVn%D5%_In~n#V4FOg5#NBC-@ApAPnISSRPO4v6@Qo^N2e)(uoOiiJ z#9s1WoDK^ca34?_%5{^nyXHQFg%F|RN`Cay;e9eHmA`)LMx_pNjZAqfu3AG%Tg21C zQe=oae4m;;r(T#Cyn0E87_V3z?@7!#_`Wd|xJS0SlIWOnsI~8FRQ85uZA|Ov31+9G z{gnit5RJ9xM!)mC@$3F`xz@A|&=GX5^p*(kT=jukk+o1x`+`g9dcW&q-xolt5Yp0p zF>ZLyigWV!0O*XDdsq{u=Aq*HYfc(A>q-2tZcO%~0J62$M}2sxdBCS%8dx$?k8?^%NA7$?1;C z+|`Q7+M0HW6mIogyol-x^1N~DAgih2GG`3OF8ij!_qFr7y{Rwa@8Wapl15%87Ek#H zoT*#|(Ppvn%|3b(LNdDBqsz~eZ20i5dIu-D&EphzIq{_=zb2e>yQnU=Z?PuMo|N!> z(Tn5mnyGH=w1^8as9Gs$im3D3lnL{-!MxMqu$ro2Hn$U2AM;HnkF`3j)eDuZYBIXE zL*Awm4DXXgw9}nri^*nkq~3B9dtL5m?ljV@q5Wj{hhqfG)^B+) z5CpE_i&S3g@vIe(b|8y)81-gdWGTO_8IGnQf*%4tTl|L+e!Y#E%GaiSqyi?M4#rOP z*|WYJZ2nSx?$bN>N(5)B-1@8Ak9h*9VPTRj<%`?p)K{BBAK3yiNk&E;!qi7yd#$P$ zKO*<%1K#`W)VYaz9jf|HDjsDE4@b@1M&tSNP2;)~ht0jikD?w)ug?2_nR?U;NE{E= z{wTW2ti#QjtfoPON=2MUT%}a*B4SFv4NRZ%KkwUTk%@d@b@lnViJMfjCs*S#$;Ejy zyv}^V+%}58lgIzluweK?_1vvG(tYuBEOmZZt(CfRgk!}o`$K+8f5z^|i64<7dVsq3 z#{`wDH7AJ!sp{2CvwYKIJlWY&Z0eTC;SjfquG8W>9zL5i+`ubjewUv9U}WYyaH=ku z+<}Z1MiQKD#+~dwwe;Xr&=r#u+8F)8^Zq@!&ur4eW;jGB+3jPl?PV+LJM*JPp4{oz z8325Pj1OOWbPy1{@NDPVDX9<0N1Cwm(`i;-&|V)XCAPe9V^`Yl1#M17_nLi3jgm4d z>TWdh%-&%kCeoqPOMwt*wrHi)x{$Yo#z^4tl1n*oJkh9+Td99nK(>~QQ`#_YRpTpK zKm8hYJBt5?|4*csXMt~dmE*VRCP9`i-29uHBTVLrL*Qb*%J9v%8|tHc`u6z5jsSaM zD6EX)qhir2Zu{O~PEt}Nepl*|ygYa37vQ^NU{{d9ud}6{>7@X0wgiL$F|+Ptz06^) z7EIy?;^VyeD{_o_x|6%6Pk~DByF~sUHv`s>3^d?f50Hbq6jpv}g02TkB9%m1!1VTC zj-7vwOn=Xq*B5Mny5ivP|LCvr=YPHU83gd(OXWv2|7(lCmNNT0fK3Yvo9Z7=$^T}7 z1606y`<)<)-QUNN|E(SH+z$(|)9D+}3s*?2>Rdf^Uw2nJbxx%&O3O2%!%T2gk=PyyG1)IzM@B=p9R3=cO#Qhi`E!(4Ke|{ z?GUGGRrCG4`)?n^KMWC8gcZz^I@rKu9w&&mYQifc;Vw!U-@xacKRVG@*3tUzr3`S| zh?T#^O_?q*6bCPrslBbNZrMJ5ZGD~=`$*Sg8Sz@0a#5tuVLi}~HQ;;aqQ;!*m15rN zH*IzSd>lX%o9NtDPi=7|)FX-r=}k-$!`(cTKdXCrR6}eLQstC4#5Gh|({J`g#L@*4 zjPVF`ym1Hn7PSG=9R&E@L_jV=B-QT1rTiYx>{gx8SYgEe*q(xc8_vXC-VsPo#kyn{ zUl>8j_S?hboEs;(uB*X3V0{@;g}V!c;ReTi3=Ni*lg8#@;reK%&z(6J$_8f!ZlHca z3pbcB*XA1`i7=X>%`|8gu?)tgO0wYu7E=D^d!RV;^-|K&p;=cYce5gN4-F1z97WnW zz~uMrd3VrQ=eS}qAGfRI=|+(E#nTe^Gp6LiU450xqtFW+w+!PDL%w9V1Me@y8fu0W zuWA7!HWkQa+0trWcdVDEUM(ZxZix#2P1F`}k=VWvHGf0OdW@9>PdE2&VW@IvOk9+# zBTQi+$aU(v1%tQX(MTozmUi2m*pM9HgbY zyHT1$_aUVlq#J(w827o)d*Azh|Nf0}hU3Ul&fa^?wbq=Um}?|cXQ^R%y>6$jLG*hk z15DFGYxr~JeeGwnDlG}2c#Qjm9}F>ngb6(!zWm&Bro?n;?R>51^yZV^dZ1*khL*rG z55G&}hr?miWADaz5!T^vX-(cSo;drQ^I45JUo}2s`*l!BBa$rqe(pT55Z&MsrB=~i zXV@pTkNpidFU-*swtDeV>UOs-yF`*YsOVXFDX&S0RA`esK9Jg^R(Hq8Eor_3YoGMx zXib7Svkee_jEo>4qu)`MZ%&vDE-RMyPQG~OdApYjVxnT|+qNsvDJ5WGGTukb1~!Yn z4ZG#dRLGvN&F8h<=Gwj5D#>lj0TVVHR1=s%O-TupO1eT5efB({BME?({xT8#5E<>cL!Yt4!(f$p- zb1HNYQ1C$~=S$f~7lqv#!T@2^E(m1ympxA!{!AQ3q`ezgQtR1EE-Gs3>xHGjK=|2m zA2=#1Dgt^Dl_mKM7KJ+nMj5X(p~UW>SzqFIbN~k%J#n_$(m5U3qC5mPCL8uB>71f zw7u`@*2`UvVd3XbUG&|Av(Rp%TX46;^YEN#n%HQpbsE;W7&}<-{DuH`y)7)Xe)Hrf zq8@(~l@Zu-R{tfy3x$tp4W_qaN2Rr>JheR4-OW76NTodW1&@`PlK{ZSNC^;1J0bjz z(Ppg6RuII+ax-=$4pyg zH;C;%UC^(LzQ#I5?wyOiSo9XW>#C~rS=sd*44P!Oo{FrtpL=XEQVz4QbOyugd(iA- z(z#&?kO52Ob^Z@bF{p!EZXwKjCGzF6eyKG>>T9A>0G$4g?;YRCiU-;{uRfoM`{~H} zEtG=1WsTYTL)2c9{gA&v~DUbV**02qL_ceZd3T9j(3 zqeqxJGg=w?sgL+9Q!9GIK=^m1-(0@AY;>LB_dB<{1RZZ>>|cF=mW9TYqZq&Ql9&EM zC9^eVKiC4Cr07+f@i{D~;y7(*i<1usdG>?NC(ag2mA?qVOq+#1Kx;KiV@_5j^Ua9VSqJ9`Zyz8pb==ErcBi?TD%O>%oT`hq+uQm}v(<{{Ij>u9 zh)3GiL}VMtic_n$ky6&A!D^|NUyG1js9m_7@fQH|e18xuR|U)&NWYa)_k8F&!}fRo zfFT=E{PMcZp|tVi{iV94#NlaS2Zzd9*Ezjghro`~n=uS0!ab`m$icc-CJ@BQ!=^%? zm%T|{Vm14!COD-Rh2^P*YJZ&yw`)VuTrf_~@1<9$l^RovjB^g|c>lpFuZ-GzUz=L` zzWnMe!QE+hKh;^oskH>>j@7i3O-5xYant}2fTTHr3v2!SgG!|-_wkuqIZexXKheY& zSqj42OC_hHZ=&f7x3}ZS$}NgAI}2GXE%*Jqv-d*$qz9svZ$fl25vA&wFOFMjFU+SC zw?K5N5v9y}A1#y%bNWDoT4$AyISzW4DU|O+i!}^{t6e zMujsN<9RM~-Ga>7F8dePMK3k2{&MpG6L#N4*wBd3<5?N|@_9%DDo5c`;G*{~|BlBd zzjB@?e-i%*O_@8OZPS=*+u!F}%*9wD6`7t2o!%?ym_pyW$)9fi`qnS?$z$i;Y4c8j zpO8t3WFLU|*J>p`IZ!zh$Ej6HM>P;`mxf(D92qD^F#gy6wT242J5`!@A9?P&3kG+H zkE|?#OGG?gT$-h%6hNtci}NlicS92hn7wBALL8$()v z_#X4jT6&_0DsSyWrr-Gs6Dx9H{|1c~BMKJw`OyMY=ff*Nh@m`Tj0kS^1+t8TC%)Db z`?=g1P4g~zPds)clCDnBV%Ci*L#%hM>}+Q%yb+%eTnpAUEBStwO!wuD-e5}F)uuk` z{D^bc}T+%(98Hj$QpYtx4!LK??HODXz3IX*$z9_t=K%OfYMyMdu z=LGhM0qW4>KYLd?_GNMcc9Y%xt1hnm!>JBkZx6)=hxL5bR87t1;u4+ux9`d@z8MSR zyI)dxsOdWAoEKyssy2wqegZg$QhKS$SQ`=VRsTfbE6eJdT<9D#E5u|EY-#)|}jEbm>gkOX4 zlk8;~M75|^;4BQUCI_JP#z=CjJU)yRe<58=K*!@QOyK-`0~QLwiGrXWKiY)67*@=9 z{bAl?e0xfk{X90Z8bTjK=Ia)}5xlne{?%s|jmqrY&B)d|hk$2gK)i$(GOBDTQt1X`ZZhbV0NOT$;cCH8zfCnd3!S)`QMbH1*HnGE-qMD z84VYP6LF4>5A)~DJ-cFJY6p9H1OdC`SlWW8dAs33mBMoml|t3r88r%B zYT3dNB813Tfc!Ehp;`g&cR3eG>u=kZ_Bu~4_5Au#PmLCoNa;8rjNe!1lyPOW&n!$Y zDJK&LG#A!|&mdN_r(DiEUSSq$X1k!0iB=K5va7X{9nBg`3lH=h@>fcdpl9Q#l}{5_ z3CE;{!Sz0H*JEYP&5$+Qwro&IX;za>Tk^(zN{yU?ZY_fI2LYI8zd0~5r4P`V(`_=| zEi;iR>PW9p8K|h%IaO;0Nm*H0Y2Mhi8xOCZG(IePjer`>ZKtTDRJ9(BSc6S!<%!Cd zY;A_QEG*RZYq8@`!aY?2c%1MWXt0=slUteMq7nna0u_C^OAtB#*)x!^qxiXWy{|Z~ zeP*AsR_ci*RvK?j^u_9W(hq^yOoyVjB;+Tj8PuoJ5^_>rbxb4*dWa=@l=h3&H~_ie z)V7o!r){g#5w`nWdXJVa(^3xnVqH!p*E6P8i<-jOMc9ZOOU>gwkV08R<}wQg_`KY9 z`Ms!<>ol_f7+mzmos51g3Yc_Srq_pUs}luqz9hi;nAhh6MJ8X;yoCPnHT>D-b%|{{ z9##XN<|@4NDof(8snTZ@^@PVR_7+7d2c(URvTM=I7A$wi4#RNAtzGALKq{aOx`81g z4OsTiFfN>*PfPT;tnx1YQZ!y|p}c#_=A(8hbz2zh5(k$>%E4{2XUrib*Sp8?v@-xj+M15T}xp&#F_=`<;4>-Uf{`&d(l$uuORR3Xl2*oF_yJsH7535d8!<<$l`Tn>q zjyo-`Sf@elE*_+?zB6ttGJ;*7mY&_k+=X?u6x!O9r?y^^ zJ^N3Y5*sX@J9UT11;6wKza5KnQ(5xz;xiV7loga7b6ZF^SAIHEdb2kJ*G&Y3weT8( zvKCTQ)}Z-MqGvqeK3+{_!X$}!C6Rg3!7X66hS;60QUB}v)5AqycSyNrsf10>aksPS zB2F)sQDSHWmut3!o^DTrPkl&3CYzZIjF!0dM>7V|Qrs^3FF3~8WzN@)sNx$lxQZ3N@ zkeOLYf;ds46&(7M?zKoTTHZvp^yc*k$Ia~=HC^(aVv+O@k6&Q9a3YN`sZs&)1ap)S zet~_aG;pai$^vOqpD)@Ncv8z|DmL=Se;&KHv7_HwqW70D8$jHmY>_CbsV#yt94B5({<;WfQAa-fY3usZfwSap82KjTmvO;WLpBW50p zY&_cm@{t5L=Xzp3k59F>bF!7i;S|t9Q0Dgb!E8IU>nRbDRPYQ$vDAx4icgc+TG00( zWg{gfhZt~(;`z`2(jy!eEx1R6hyd+N6(JF1UKiBwv~V);Hhrh!$(_+W4dq_vXbB$f zL+$6C$+Arl`|St38t++>10W@Dwu{D&#gi%R_lg}pBvO&`^BQ`lZJPz zO>27vs)hlclY5411aNcP67ZX3`U3t)Ct+_s{tHpdkQU;k@LkJDn@uh?6d*}mWtI2v z-gmL(O8U$CF%>0cJr!6a1ybdcX>;M?9Hutu3Qh^8flA*>CudG~=E6ufybJl^PxrzY z8mt1 z22;fu)QZRU!`=)f@g$c5sTF{*9~TC||En6-503$u*Et(1&bmb(1}bXGu@s2lu8ZMC z>B3&1KnhcdOcSS*OpfK&i5#_&R4kIck=)fapT{-itIf95%!|d#rDrc>& zJx$`?cC1?$Yj(C6KqOTN>t@;>*k+TB=iJv+8q};Y8!-%^K$G(EI6Iv4AZUiyq z)3Ma8{iC+w;B2`*UAuF~F4*j7aW6}g8agoB4P2T|t+j&ed~)PBTgrI;aoz>|d=^en zJV_f52>&dAm5SMW@ttsmsB_#F)pvSrFR_ZUk-Rcesytv%ZlcUo@Oo`kZhNNUVLpis zD6nP)*baSuL}5+FLxeTe@{~mk0ywoULj{)UK59qsh*EQK=!~#9CMS>}Ojwn3 zi@geHw!S~Vu^pfJYt?N2bL7IgIRrp67DgLT`Onv~ASOhdW)o4+ihoc5Ok#!ZxvSx$ zo7OS3LA5_p0A~F4XfYluK#MIPsrYuM`>a;!3?)ba>9gD;bi9+q%s0PF#PX;9wyoD7 z`nc9HI)IYQQkPL&cUPrDqcjEeR_oR4i7pYF1m8KA@~VaJOE$?k(NfL z$=AaZjy&-adguHTnL5RyjgkUziR)-f^U`pBU5jn$?UCu~Rt5>iSh%&#*zx@6wrrt7 z1MYJkuQ*Vt_UrXB+Cx=quMW4ylTyB|kN$FsC56uU28l)N(YuPJb4`a^V&DWOS8%Kv zjrCkzj(gjkB*VS7#FHKNZ0e)I>fKjqk%AZKC2B1^W(nwa(ox`hb*Hq z;_#fyQo|tRKNlgKIaI)&bC}-Op=))4yFN?MGao9eaxdsEroz{0gJo?xsLMe$|Jsa@ zR1)W9kVRsUcUffnDSI>ciM`5ngdWiu{Vq#(#J){NQNPY=b)A7L(X_23g`SU!RxYDf z_am|Pp~sk|U7@O1zlwRMtDqYIN-~vILmCq%`vGs|Uk1zZdjo!iv<7PbzRfkfLAuYU z4JKGzW|$ravdi2QL=58ZIq4R=wZb5*xkhKAzQ$vFd;3P-(rBovuf!zvR0Y?uF9H7CY?8+_09sNME!dJ=AdvvTSHeXBWIs}mga{Ejv{gKlM3i96b zP)mdKjWUHR^67dpgfslwalZSM8Monrbc5P`>EX9obv}nBD(BjB!?c`|zP9vtQ&F`v z@iXo64Qk#eb-Hoyk}g<(eRS~_sAMj&LQvY6 zvwHruNtj?qX)r9NFeyedx{o!BvUR8w?hr9^BZWdqvY5%4%Q+JFux7DALV+e0417Hj zf%mvaTCjP$8e9O-emH8vQ!6bYI1(nK6OnQ2&`oO zb`*Y|lXgV*r+Dz6Z~!M4H-p)~6K-o*FdF32i#xj2Ke);`T6tGE_DifFWq_OWC&(-a z`-G75iw2vCf}~`q4b08`f;efbV9+(jUQu$y+n=&fm4~}qM;)`Y3B7-+v%q#a)RRwW#M0@ zthF|kV7G9gk1QXjy`_+sZ4J0}fZ&s=|HOWc{P*>|5c{ICt6FVwhjMMxf59GqfzREa z%#}c|qNp{sG3dJZrCWwDSUqw?8+vc$BrqCjN4tmAPO!@b4 z_;q3Gen5qTix1EKD+(ULqhaRn7Op4aUz8O;$Cltrpz{)lbA zTpzwlWOJ(I*&7X1>A137uW0t~YJ68`YGfN7o@5&N1`R2Y6hM}~*$#{v{@&ex@g|R8 zFaTDw>xYPH9T{5&koii(7Hs!XVlxYyCowQC`ab@va%1BE6fD*WCNkUt* zIA^6axw^{3M&ZQesZ;of)#Savy&@k!`r`4=-|zn&ryR1k8)EfDTWhR9jFkSV(gX)e z57~D-f9K$h@%yT|eBFlK0+U1&pLi_#M1T)AJFktb*>EYifft>vrqjZeLjeDTo`b3!-%`zW+@Ad(AKuF}#eJ5B(a&&KsOP z&X46jzF()M|2B)BfHchFLyayy`qE9qD2nN;<1%mpw&eQF?mQVu{(6j%hgecoAogBo z*hj|iuZANLIcvrDei#QE6NFY>qLO-4*|}xDv@-Q+emP2G3?$366e5p*k4mp6tl@#~ zOUgq_(eQu7>c8n7#E{)uL-g56PyZ3L{~o~q_MRD7QxOt=&EWseJ_6HfEe4oV4e%j* z|Ltr4?dQR$r9WA{KQa2xkoxmcFBOzfL&k&%lc6C^YzQn4DpI0dCn@2^<-k8A1 z{y$jJ&(c7$8|$CR4fGX*KiCwIF>T-_J(sDG^4G@-VO6<7@c!Sw6naVi+R%tecofZ_ zjr_)87xgMC>OdQbv-L}4#SaeIQgejY*@xF!7VGzTOHLTJ(y(yU&Q7W|IZKN|8pq%i)Yj$*it;S({)?W>yyeBuDV4P@h`Z z+fPW3iQ;*4vO!N0L7V^e4FWZlBC)w7h7vK;3hKj z00^$UcXDlSL^Qf(<0Ph9MgBOY7>LEgrkV8fWY}{k^G4Vr`OA3TEs{QPLaaDVUld*s zeWE5_;~+Ax9OV^dm1dRw(+hy>vsJbayf-YpVZI)$fr!7~sD02r+U;SeHj7w~h|@d1 z5AV<3_v34pota@F1#-eL(&5go22X81_m|P0^kzKK=-g(FLin00lyXBi^!Fk_c8fIc z?TZIkyDF)fwCaAKe4qF@KGEpSV2vBv0U_@PGR7P(@iy6Ax7vO{UhX+~`1apTQJoO- z7u1|Viv)b3C$<@FQ&x8Q7U};xv77%X!v{I!2?DHG4?!*4k2VH}7X0TJqaXI4qD-OE z8=AFDWr>nu&O&#b{Uedx6OdnX+RULt{PUCAov8dL=m}By)a=-QHeFyKjJC^sA}^+_ zB}WZJ^MzO4Vg_9uQ5;PMa^?~%7L-w+(Z5YS*m~<|a=U_C_P5$~US8kvejZ_v zz;&zr-Q?+=-Vqwqm0dIv-Q56;n{&7W3C7kvca5z_)ye#s zKL0TJ{zzt(<mt_Bit1p;n~{R{rv;(T= z-fC%|!0)K*-Ze*yZoJcTZonppbY?7KPA9s0>#Pv7VAWK~ZbH{ZZs}WW9L+jh0Z$IU z+KBQ-P@ekFQtXEZo6LDDvvBij3v@kXd3Z9$#84pHqN!Zx^BmAYGXjzjg`C3p`BP$= z_(J6|68gS0=E{N3g{@h{Mc#zKBR65Fg0xyAo;B8yp1Bv}d0%yGKE;Z@TFmB_e|+H! zaygtAi4X7L(S*S2)O^`y=X z$UpM2UjE?(jZJ=Xf&(Tq<1FJ-gL7LwdbuN$&3Z|JadFBXo@<^HK$~Lo*?GF5yYbmY zw}4u9&B#`ah8J%Ej|AKey5h%}E(u+aD2+Fz@484Cwu4-zo5YrXrpAD8j^h!m4`zO+ zyKLp9L+#VeQx=WJ@?M6JCiOLzEO8B^=`maNwX#epMvn|;ai9LNP%3|8nlNt8>#Dcv z-N#SExbNlQWpV-NZYFCNY8J;&!{$Ul_6zDHQN z9!X0;@>~s2tAP@z5pgj&Yv=NUGgh4Rx|#TVnzH67h1b_Pk6Ba!{*Ixf-(u?b ztV^lMgQuoPolCULBKC^oq?;NoZ)_|^3mYuie=cJ}o+3o}KZgO|UPFZ#)k8Ri;y7xd zn;fIpxVFz6SDYJ+cv$Xa5RG`(q5i7FE}}k3G=)j8suztwJnD9-DugW;F04$0Gllp^ zd9)>F&uA`~BemI!*&f|KTSv9;BywMS^^oyQJ4vj`}IJY8r7&>#}w$sC!0z1L$uhS8M5BZ6$R zGW#>PI`O;7^rEYbFU_uXc?!-TsLIQCU+44~NB=U+vMcQRQtWMX_dU5+VD? z3-k2njhFC08`&rOH8YWn8Gl`?Y64)d??NswbkxpL%s|F6+C}1v-^R%2b~9 z)}u1ZMsCBQRDSWC{)2`B+m#ZE7P z-9Nstcg^JxpjuBDKcAjD#Tb7B@Q$yZw-Y%~L!(Tp%>-+|BKLEfXhDUU<*^N3kJX&I zt-ei?@u3I6`zA{wC~*{CIyrQNffPsxqw))=pI$|rOT6OC?&;pE2Ml})uU}_6F5%j) zhwaZ-IkG5hN{81zxB23#d*gS1u`;e17>7ed1ZI;k&cI&ruGkrGRM~YO21#$8qPPs{ zn|uj+`~ValZqSBCa7ePk?H_~)1X@FLUVzycoz-hA=Z9muR|pF218-U73^WE}l-y#U z9lVQ1JUT94!ZP9v;C6eXZEC5zTgrf-F8?P&<*CRpT1bDH`@Uf~8yVOPWhiee&Wcb$Cb(*atZLB!dN{|37$Z#i ztm;`WR(KL;HsE_1{|7l7zBLCi7i(ruDZ|8iw`#v3NW7K~&KL|Yc}wz`mzWvw`DMK; z6DZ4>D$xs$Am){Q&rMNSTwz~pl-%RlTxSUYJAXx;a}Km!d}>6 zujE~ytQ2FJ%E?x-YM*@PzJOn2naE;P+i0b&!){%vdJD?6spyuo(;+ZLkogm;60ix( zR3nBhk4IB=Z1d(0TWDll`)VM{Cj!5XC&lBTq6#rwuV>sHd7yim9 zj>^HP$`pB~O|-+$_jQzC+qp2*bXFFOUCXA4Atkg@n(v12?0W(x`X1j_l^29m3iQk| z@bl*_>A7BTh(uHlo=yUlrUsLJ!no+P41T+b zbJJzS52eB+QsP`&Q0t;(!b!ZcbRe_>^>W1&DiSE;<20T&EWXh8vgG=A5C1(-OX z9e!TI2Ho9y}r_t@>#zS)y_AHasG@S=;M2`_rTq^G!~B&tW=o+)$lZfP zb@D*hf&~+FN#O_$<|0s;3o}!$77RI%I(gJ9rx5K{A|A0pjGdQ!FxRTi|o${wcc z4X}rpJ7m3QH`GboULm;OK+GYVBABObIvC;g5Swc`!)R7*w|L0c(A;r14@hF{k|23^Qp(n z^jfc}PyGv06ziW;#jicLsVnA_a2! z7mMn4B*S@NzmwU3uF<-ovJjt#M2q*QVX~PLQF)d7Xya2<~?-(@g#Ht|<}P z_+rRnd~D2oF$IA=Z7DmR45M7lO1JR2@55wdwcA&0sARdueBK|MnaV=yv)M=Hhd=g9 zCNYw7aN|*)Y}(4;P?))WC226X)&0eqyv6jIiM4rwZ>?1xs6=ZdA4AWXj_yQWMNkXI zM-2X_|FT5Cq7T{JT9?^sXa3M^hK*%BF!CYNV!A{WP~T>B2EHLY-#Kvt?>~1AGXt7S z@bmxhowyp(avEQMx3+2c5ScrWR-EjkQ?zmVmtNC)De!d07xIg6w%$(uh34cb@1u{D z|B21Y^t-Q`a<@n+Qqk-XfR}Vf#2#*?4PnRwrnM{AD(=3+Ebb5!rrT&Bd&uwox0-G z0QpT+BDrF4wFp&6mUJ5DfJs9-)yFO^^>4_EKISYaV(M-t-cb~-#suxh4W{Jjo65N= zr3|UL6ez#pnPZG!en|*?fVSr^zoe>7LNS(6z!n8!BsA-HBBBI`>|dHDY&g*=)7}J- z4NC(?hpQ5&G=Pe%O&T=WUM#3SS@W>)SZA8}@>*+hgZpeay-A4gp-=nqUVo8sjEjE9 z!E?_U1@$O;ZSys`uz}j_T$=IhGm7e1T^Zf1{E)47PS%8+beUJON?hQ~#~S_@weC=2 z_t+KhH9GubnL}>!_f}=~g$pXKnDG&8K7;iePde>oe{GATAp(giCbB^LfxnVvJj4jS zD-J|U2qM{iJ{XXr=5tESxvnqq-)upBx`$b5S#KKTNeQ43?QX91M81Z|Ai=r_ht^nU zO@|w`;bZ<8eSDx-*dWLzC!}*C>~!$4yEueuW*?+*)Wd++ z^WF`7H?cVkQoK^Sh&?!1i8`xt<^9g+<`6FQ0*@POT9y1BKEg!nz9=j|aY8N}`Naz7gX#aqH|M8|7 zn(dvm*d{x%D3(J9Fw!4( zShn`q!faNqz!f}YIvL6Q`qAOc35;y--(wxDNfK2`oA7L#!U11?1@Zc`c>N39{?{A- zA7lUn>-LGwBheGqvmxX{y|5DPNHK+xf*P!*V&dK%77=Ym5%$+y{g*~Y)*E3ErsI<) z=Jfe}!UD(nv{mZu&Kn%6q5LYw^}9ae_E+e*9FF-SZUTK{uMqjgk0&GGNYhum2>TqO zjXz_wq{IN1vpymF-&qeJbO7F>IG8k5jq(TeE~$R>Ada!gN8QUfiU$LNu4DAD%{`n5 zhsX8<^Q*7GucCtQS18>gs94z~FP6R%Ggw~=(>SB_q#C-v@0!~am^Pxm{xa zu^`BpaplsioBYZM&RN4`#}Ks7SouZ+EsSDdH6RC|CG=7d%F5@v71MIKrBFeAjEca? ztV%s>y~nzwmCEm;SxF2EIusMvU2qFTb`l6)uB1dK(zSlu0e|E=Hci3kvQNPsWKGXu zlA;6cFpl}!Mj*?gn9U65v~?>+8ovlT73L0&UH1XWY1Ed#y&){W9RLu{%McuK&T7*B zv4|WrDJ@Wf`c!$Zq5ULh-nOd`ym^MXc$eO_P0W6KyQ6Jc|@46`dWsstS~XZUX?bOoK1VE>UNO)tSQK& zl!muQf|zE1>sw4=>;WN1kX);&dAnl#I$0R^(@u@3Ir;8k*v{n>UT_a%xJF@xwh`oO zWrQd(WdxaL$>prAF??RwTke75mr|slOYkv^Z+~&7BcoUViXI)?%yj&yk1)pXWd-Q} zItcRCnBdRjQh>IYoWx+CN`fDvGH#uATp4>YYya8?@5J`)aFlB*YsXORF5MXJHh0aM z>Fdzj#5Dq|b$&y;Qn_}Mx&#`|FQxAu5=)%uh3~Zi_-Y=jw;XJMk@#-9uZ@E>Ot;`^3$Mb1lF2S-2DPr9^9|{bwcls*qe~qnJWljk3N4hz%xqL zktRF-aQ_xOlCua11_&%)IU-re&ansW#R=C|{rm}%6m(?P%d^J+$H@HWUrhi8ze_ra_TL(C|GKVQvhu?v z?`rear8of35I-1T1e#n`zq@-3w8%SdLSryt4+wvD^MMnvg{-Cd5uC%+vR$nDVgLHi zl~4k|3s(GpeqR&=EYM!O+Gd}Z@N70&xz2;15J3EabFby)*%|VXdd%97PJy zH{B1mUpOt_B9vXd!s+}vR^(}Gp zT>HvI^k)(C)yNRn&=<_(_ML1n?JBkn0+%Fbj0QK0`IJN+mVOm&j^J|XS^N=_=^UM2 zXn~+~&ci()$;e@_;D@458G`E+K<5TOS6GF`*eKZl80{~wzXu=sWZP2v78y$b-Ejbv zb73z}HZ?O1<1DjZg?@baWK;g?c%8ka%5-RK6)t^Zt%xqx>7-Fi@~WljBIBDV%J|0f zWLj$KAfSbKf$i}9DJ`E4wbvC7l+gyAYz|qn=|`mTN=HXW4_!8M_X5^fnm^XquV{01 zhhIpqW?KMt?|TT^LZeEzeBGlrLGUnHzi+ES|5rQE{kVJI`|US{3R#ci8Prr|nQ)Z5 zM_9wfM1;uFfpm~69gVW_9RcjL0@+kv87j%RUPsfgiL_}e<(*%4<^Teql!dSI zNOvY`vGKxv{|1cnl6R;))&D^!I;;z@EZcC;$!@o)?C164OA100-T?|fs;&XUk1;}D zxGB`T$xl4jSyu5urs_fE(1QK%mtQQ@>RZX9MGO^nW|<5ojcv#zKUdHd%)5IT-ncjy zkMflEdoB(hp196IYb=2Br0_Tgtjr(0( z-LP7zx@hGz0G4^G19b^4=1Z_ZyGs(+Y=#k^@fBM|@-geT9H|7a*O&`Be6FKOla}5) z-+Zn+L*8KiJyT)Ge-J0kr5L}IdJaNoq*5m_5JqBx<#KdfGAwweIf%YHWiar~<8ss; zIySw&@CK-zF`FVOzm2;k`yJ%`pkU#)d)7#smX<1Jzo9zkOudZ(XQ^1O=#m3Fd&}~n zk64_%5Qk4F2%O^Rq#2%vt&uyEg>qTC&qXnomQr_=i^P|_;u}7tX2n((&-ajq=i!EH zjhzkJez{omNw;ZQM{g|cJF5;heZ%uD-S<`kaL?ww7=9%ehI=m-Pp|CKkFINi2AsJV z?uT{rXfBbXFs+S2gRVRB^;Z{3+t#$(CzJWwVF!mUf-Ps=a}!@|C>q;#;hau;73zj; zAkFdCmKcR9Iq3Ye5B47BTVUlX2WyWDCigc{&qx}}ns9x$zQ1T}8Bdj+qw~xM4ut`L z80ZSWxEi^bEcG3k*@giu&fL|Xxsazlbkgxo_}~rEhlpMW3p$kRud3mzk9&*0e9#~) z4_QU~It%~OWQv5e8vSQd(~j}mz1uHYqp`fcOks9pD@7~ND$vFCDGx9PDH#}$s}$)n zN%gJg1ODCS2{4_t9G-@W^fz;t`Z)W5oEEM%W)#879oUN#^ZEPR>|iU zl$pu;kdlCy%ksoC(PrC=EuJ}*$8&pddMPuOMt1CCJXb!aKR$(9*K_^s!SwLe_Y~UM zm-i&=5sS%W?@t$-*kPg#U_^^)kwTyC`53?;U@;i5goK8oz^cY&0-sU0NQleBY!hqW zrMWf)QYNf%Hpo;qykcn!-jkh6eAqi!7)(x#_Q8Lv-MptpQEYuvTj?@xLGW;ZSYNl% zZyGD;oVyN#{DuOC?GUk*Z}_Zub2#4duZ_sP`-2J4E4QVNO8rHz#QO#I0=2f%QSJq} z{mF!$KD^l0xEs>OVI)4cA7tl3O)@0zd+)BkuU>>x@p2ddI$j6SNQUtAx|Xs76l`>z@BxzB+jWQ`yVXE+wEXYfq z$4&_4_@I3;XoTBqgUOC`8hfMsoh%zYfj_!bnzPKd&XY;`tWUxlG{|h~f=vd4P3E5> zr2hnj&F8cCncZrly@Utu@p(^fvvH1UYE`VAGfTkh52!pBHuh4Va27DAfm6{kw};By zTCqyt!o?#=;H8!kGQp8GyED3JghzpKW%7$v_*$vS#yHYB`$HSih!3XR zT4`?t8a*}>~Fw75qI%*W!9{Jn=6;fETzW$6%SpKj|jx)YUBO`O9$aTrC{8n?^rZ(F&$5} zfKlY2Y5FtOs$G2wxQhaMmv%JmR9*A#KR&^~r;?|m$hy&1KFrx3%1>~Q3=JKLA*k^} zNgtHnYRfm9Y>>=dE`KqBMF=li1=-g{Rq=LEOcy=5>u!vFgdFtW8A-Te{0K7m%@n_WG(0XTt$CHR4laeBcm z^|rz%XYbOoY2fo+z*?6TvW`dpk^pH0EBP4#QnLPznJFhgRmX+za`k&-Y+a~Ta@r2z z4(7mp8q{Qu6|B2YbbcW%46E>lEb8au;hp;Pczj$f7@V%kJ%bllH>{4PdG!v5RjoCT z7Pc{nSKs=|^Cz3vU1QTAV-_$TRGW5{MEQY$PoIVSz*=#v6`H=8N^tW*u46GNAj2XP83}(C~zy zN}Cv_rL<8|xm2~lfvO`q%iI|fU$7)?A6`3XPiFPLpgI*&tkWdxR9*(K1Yg+)XMVWv zZj@AFfg|@9YqZdx511E`Ab$9e=z!@jw20y5syTP$0^F0-bcq(raLc8(e7LH3+QDcg zvJ@|~PG#KT&=*=(9OO*e&g|<}X?gF> zm=@zehb~G84M+X1htPeaAiBGi=;-K3BZgKdm;JG?6P5B|E_z%?Fh*D|n;{1ObyT(0 zSpN)O_jD_>Qfmr%Y+LVM)RFdB|G*in1P*PoP(Nqv6dDoRI()LpN%x>)i&%|cB`Lpy za%UloI;PlA7owQJa zwKxTuReW;IJFs>Z&OyzVO`TOlxk*1-Aixx(ruP3#B@`?*;KENtA`E+ADe|VATv&Iu z!Uo|vvoI{0A!pNqrAv_yqyD^QV=xGdt;(3a zYBjmZ{p9+dO`&bO2qggl+`V(1_|EW*LkFxB+&9Z{T_iaLN=adCVQ!Mt^;-WR4el{c zO{RNG=79u%^K-PkiML%%yHM!~Z_@T>8>B2IildD?mi|BDzA`Mzb^BXsq+38_Xpod< zXi!ob>6GpqY6t-Z6(ps*LApBx$)P)?yQCZ52i@nK|Mu+j`Ms|BFwCVh>sj}@*ZQsY zUbfLJ^~}0vJK|?htx=5LjC+d!D3S)8pEn5=)uZTCzKN0|WaL=l)#C5dA_r}AsYl6` zuTV(PH$^gY!Rs;EFe6<)v~H_sQV=N* zPuvD{{Ok+)G)7clg&-BkW`;4W4ZL6gr;0d@wrMA`m8%3L^FTRN_h~#;Q4};6Tp(+^!&WK3c231B4at*o&9vIqDT#HG~L`7LZ2}PFEW(060FZGUc?(^%`v=gT+S6;OTpRETPJ#87|^-uQLORC8=)qMw^OW=S273m?Sdk)y-a4QhKhG@39o0-uIXc`>_FI@M2L^TU-PePSvrwGz}!L>IG> zT3I>{a9>r1`zqnk|vGl1HM~!;de&m+42bLfk9xR zX>lS^60=H%J6LLVSk(z{W;EOlI%F6xwn~?bTtPHOd^!hQ-)?vrlQr%n?2M)QxU98F zf&yGG1@lI5Y|K6#Ya6r*J5^w%A2p4E_6_wLY%h1&wF_?3%kF199E1%TbUCmt4wgut zH#26+2U?|K5KsFmD3q5Kh|vqo4qPnPtRW8e4bp{e*YBhBH@`hj%8~CI`RnLf-oP<1 zFtGoOo>}RQqb9x^f(jvb>9X?gMNMSB9^rO87ryas(cHFofaYoDN2AwT#zdR`+!Vs01v0uNPfO2FcbH~y9i;p55T;Z z+I0ZBWY*$}NsiImI}I5*Te|lm7@^0G{Xa*pmfEm{9IyDG3oQ?8kE-fTE=M=@hB(`s ztei~mmV%e-NA4jXb~(a^{l}>yvo=u!(os_7oR6%sFTWoVhl52(ZAXaNSJSj9$gWRJ z%5k0()Zq=s5j5?*d@=jL z@2W28U(FoijUru=XTq;>P}sRV^4%A0Pfh-Ti-42uKTaeHyIy%67=&p)zd`RYZROct%@Sf3yM1R%4_9LY(R&c{RLI`gSl_tS-v z+|`OSOnSuzzmW%SiZp<&BO+BYz54=&bJX>h%N5i74mqb*7iT{So1){)FdD^WCi#6KiyIGz&yg zZ;|r77hQ;7Y{4YL@H@ql3)`EkGSfS;+8Ql=3zds9k*~{KyQLYO!A&gcMTxoP-xD^U z5+!h3vm@bd^AxbwI<22Z_cHJ7-}u0)Y-UO)L?H~ibrsr)Q<0fpMh=%6WImo^ zu7&UYvY_GQG9uYFY+RJ4-D%E!~<`SuF4$; z_5?cyy5cLX;ysAAdWYpmd1S#o#X%|GzOnRO%nra)GYh^Bs34_#kN3&!YjYBuJS|zI zI??R1Ue@vPncgmkRS6U{AO$)_*+|9tEm35_vsg*{!f~smjf&~FkVD_@h&z$Gkuc1K z%d(*+V8`=>?+3L(6m#+wOK@yq|I?dgV}xVkQ3M%3ze)~T_AMq-E)I)I*0E>FH(jT9 zJ+r|UWUOqFY;YE04#k>d=!|=sg|c$=52sV4Zy#}-m{8defkO7w^Lazv%XPWFl<8L z^Nh+jE}TX(UR>x=Wm$n@2~XROT!q|-oZsI2L$Tjj^e+eT7cTw%-q#v?0X@dL#c2GA zWaF zZUqqJ!n+MQ@h;1FdDUndu=$kw5{vzaOF@z(iEFJ7V7&59b-H1UNMzbXY9J^{?u;UH zH{<@U{nHD;Uzjlue4o3AP!GZt)?sX+ZE_rNWWtBSQa^_QmFTejjUU*HoMa!y;<%cS zyu8Lt0*gPe1?C@_J_9a}s>&4igl0*?94OcAl&ge!=&c9|cx z0t7|uOr$bj>XD)_#N?zchOuO1%f}M`1t5MoIQJm%->(*GAinU*?@ieg*#(x09f%Ye zqelUhm|Dz*ZSHKvRf>$zH`WMA667ux<)R|-tn4?ICm1cP9=Jy`xKfPo? zM9;%Sf^q5Jx@_}|`n)d+*jcj{je3FnMpo2i&RlmpHfpn_aQd>G{CXbiVUk7I0-ewo zCB8*H4%oUHwJLHD3#l7FC%m#`TlwJAT*Tui#vKLpu#lbV)PuUSVShy$sj0#Lu>Ifs z|MjK}1cX2c-#$(Ks}gN4bD1vGVC0DCy!&2k_|qJNge%lCrY*Wq+%J{}x6OLWG)3 zjo-hyaVQFGT=0IkduK$f-eb4)DTS&J#T=iL;IKLcA>GDoOff@k@DY8VOJ%wKduIdf zT>m)(B^l%rB^W_kv&wYMhVXr9C-bDVXHRu5zOYZQ|!pG8AFL9ejvSg8H4(w~+XA z(Z_))?*|*jA$7&#vh+s^c*4Y9FE)owE+?Yn1m%v4p}MC#avKjf_~;`}u~~}x-%6$x zAn8TTK*Mgik9l9f;@>VOH$0N=yp;Zd>Og++(e=fBU^P=Hgj?e#uetQrNGVM%iXd;j z0gjh9(X%OR!%2r|Sd&*%@TjTBfcIWkyuf&fj>~!cnB&Xl_K#2Tm*we0q6?v3M=uB% zPus~^Uz(^aFD;S0G-;<&+T`v2ZrPEI8lH<<9M^{WO0N*_r%_vWp%>LNQW7FMjP$al z0!~zoS8p+if_%$raw$sCxyCe!{L`>fUv!O(*8T)yzr%v$1zKAcd_7waqTZL0Tzer~ zAc9g)XV|Ib$rG&Y^~;T20CkbSPv! zEe8bIOS+nVEU#ltAHpcQU-#}H>|P%^G65V&Jhs*udKVGFsBSx8kQ^z}f^ctdGe<`v zSQRL`Q`D;?duy}j2ZbE9npG&3ne>Ujm77?}%E^WsnH>)Z63Gr+3)z2MO=Em$vO3fW znCuyE(?}~kH*0-3@ZxgUrqAa`gfMlIeZ0_wzNXh$Urht?2C6E}o0KCMS=1Fx-Z83Z zdBi^vOtP4y9NAZ1{EAg94(CEC5*U=^;SWKj^VO}Hlrrbd5~zO5hE9+0|n-(Y~V^E88RyQAeze*yU~W z3_>2bV`mEQHu*+?@(YC@&7yCDc4;?c%}FZAhUXtyiu6&Kyha@M5W*bl#o-Mw2A>PU zDF>$Z+iD@^ag;oD<{3pAzOu(WhWirrTh=rynPbm~x-thMg`lz&jVUL^R@Wr^oU1oH z6l9HeTtfubM86v4oE66{$acGrNr zsk^T?gVjL(?ZNt&%1gmgr)ZYVODIksg6}b}W(dVkm`sebFYR<27bdW^VjRGpj?nM89vrUS>y*d5oiBp(R1{Wg`K@#icaNrNTk38@! zAHDW>cKPVN?ZvkOZULjOKhr;Dr2N_bbc7p|q4_O|=<77WK}soPU9YJZFLGr>X< z%P_RCHmeVA;)+FI`?!0yGo*+D7EwHxQk8J}VIBu4zYg?~0WGuNNhw|P795uNT8m?X zJF=rP4P0Z>F`9OYs!IM9eiJ>2E9|s9f6Phn!0E{zv`^u6Fn#$6>X+iU<1L*=vf(MR zBT(hFnQ-FWwtBXV_$9`AU@9`jE33?RI`80mm2DV#q0P7UkB@gbHl<{)Xrmhvyzi9@d6ShD@B|QMCV#dGKry$^5~}mVId% z6aMa4>qX+e7>4R!2k_&=H_?C10{Abw0MOVK^|_%}<}ai$B({L0>deDQ_4X0D$TW?j ziIiCrZ*Nc86O+Tp{{yA|uT2N_pbjqLu99_a{Z6XIQVx>kIgcx*X2WbYlj*=2FhjNI zRr<)3f5aldhf4AQ9HQSBQA_#t3oZB$=d`RcMrnJAR+a|N0K|`@~Ge>62}_ z-(hJSz44nFhH$eX5g~NL3 z*6+v7MtvgQV>_5L=pP24H~3AaXsk5iK|-k{M)5QUMTj}aiUq(OP3{5dfAh435Fudp z^zEw~&pPI7&p8rw)`i!myauRY+12dMAD9+0t6yo06x{(sgl7eJOA4~D0h zw^ZXW$O4~<-ieWm^N2;I<)l)Du>$v0f8w|K1rldCljT;$)*AIn@^s0Vbz>htek?l; zg*dJq$ECtbj3gK*Crfo*idAGBKV@cTO*O_B=m+DKECC*pS@8S|LT+J<{oa)oV>H=b zZzVMWs;Oc`S&;7}Wf)ymJZu~w?VjMbPagR5FfSRHXcE5u*#6r?co6~YRQpt*UoN`V zrwJNqI@hv&>KuvQkl#G<4Cu9oS+O=zl$%{M0rMZD0gvm#3C6uSm0vHwYn_Y#OsC6A zXg~x~A^=`-&^(=al&4ny^}5zQp<;JZOQP|*Vx=!(g4x}}qmDCwNCFt~JKIV|7mDd4 zRn)wy)tQ%-5^jLk#YNTKp*+Y}P$F{47RV)-9gim^d^AUVnZoFMj69!o3RNxikl2}E zAOC=m?hmkcXh@-e=I`!}2tM}tY5x9dey~X_P_Ok|95iNCh;`?x0i0N>>Q@~lp`_Ea zFD%;AQm)5c&xu{`A`c%QU!Irh9n~QL^HX}f{K+Hy-aqP@T`gaz=w_5Qi&*ikf6A4} z#@c++@V&jAuN-I)6YEdp1GbMcMb6eb@wc@(VB2y})0R(+R)ia8Oys!JVM~#Fu==JA ziL_Y%MZZUL>@JoZ%r!^3BvkVxcrxc@0V?VXa=_n7FCNe^`a|p!aBH^e^wO1~=gFO( z>f#Z&BF2ZycO8LOY<)nPsJ>e3_8a*0#UpltKfbVJuFbHvs@+Au52X{oZI?U4pNHt4+D zK;NJ_g+W!y#UA?2070W!?`dH6YP|}>WM=`u^byMOcL?dB56n z-?H4SZdyBV=1C2;4(D2k5B0g0m+fcwg~2?|!rpnJ+DfD0cd^bDZ#WX`+3TI5l z4uxNwZFg^)j&Vec@RfQ|W6J;+`dtJI?9UtT3qql$rtZp9*@G9avPnFG5wiml7CM)L{xL4x?N=W!U6%4)PVDw1Li@m?kD^h%I1RqJp_)m3d$*nRd= zN7574nC97I;%H9U9Uq;KVT$2C;s1ci<`Lf?`&GNN)bl;GD`C!K&LHk|mFGwLV+lAV z@fKk2)PBFy+~$STs!s3)6@N3oa1vX$8{H3|&NgQ-l(;_{l0U)U_5``PU)!3=sM`D%S2FBKrTL_A{&w7IPzn@}8tX3nbI!deP|&9+ zG3dFE@r3Jo$yEw_QzqL*p!DJHfQreII(b(**HI?07j<|$qRDX~1-@UxWoK62`J=o7 zztd&#&W~D2Abmr-m3Uauju!o54mvW!u{6b!4A?W{dy(vZD!lgR=|7Z3-B5l*S5oVh zgj6}#(^xdmrAz|ikRDNG(p*tmT_5Jat&s87$M$tptzTq1$p?Cqk zqAia?=lX_x)k3|Ohb!GwL=ba--!_`K(>tX=*vNn|1AI}VE}#SFP7gG1XJqn}cg2=( z9qUmPq(GlgV!eJbOyyq;>vYWlivj(dir$we+6+3GCFRc#4%}C}&B!A;&dXY52STFO z^+|2MLpFx>^n5OChFIx0gv>Fuo1C+9sp^5g?K1lxltZg^I&$-s)${y=TwUXxP02Xg z>hV!jzVy^B#4Jja76t5o3KMmz>|Ai`gDWy|yO`^Xqk66~yl52YX_Q%_zv!0KLA8K} zj!q)b)2xa~#=F8KN%joPJD$H!z;3pwaCnvd@^ET`oD}{<-MLN+7`3zg%BtF5WD-He zmA_20>?hAutyj!3sdL5z?d!UqaJ7f=u+_$iPX1+nw~Or4A*xPvNo}B$Y^BvQ>SYX( z6Bk*c^?Y8T6?UjJ=qn1sA&^Hu(wXqM)!`#Dy6pdK z;te>m+pzhjZZx(%bDhZWr=nSX%`~Nv<`b}vPTWu4y5EmGGTl_UT<^s3Xh7PsByeOx@j&dEL#HNQGihoouY7ciJ7#!>&4- ziw?Ge`8-x~zcA?nY+FA_I-6q=oYrTkE0%4#zG%q4vYo~{+#TKLXm7lmnu%*IKu7uV zIcj3Othc1}GG6NkpuDpjqh!2;#?N(Q5tlV+3xEwg%GIK}fXMDUN}u&h=;=Pnk*${R zICx2>1uItw35Sx`!Fcb??wTEc&ifCSNP9I;7*eB7Mv0`}=Wp94BE#KW_vX*46TwmR zM5;v_)oGV|vsEVbQXR=>Wc3x1L=sJuSi%m?;R>)Jj5XOWOooX(=CdNvcU5rEsziu9 zYk(82mMId_W$-y-rxf6BG66_Af`*dzPw&8Y5O+{?zuu1QS*YL*%IS|>v#mEYvQeB! zQO+v!fOU&I%q@QzgYAG$Xd<(KqgXn85rMV3Gg+l#Ig!7L13j&fKI$->U#EG@t;d-1 zS%ky7OsQd@VADR{kR)Qj?Q>TYBjyd0gg!s6%f_slGuz;zy=IkhEjn~fiK@=lA6y1Z zU14H&MUd?$h-}xWDuC3h*+T1hdlu-^9Mi)1Zhh!! zx33ST)Mi`kAxl$bdc+?G(TI;ucD3Ds?fkOYJ(;rD&=_ZzT!Jhs0Rjc zuybikaO68l$nof%nGigSCVE}Nq`|E=F#r@e73ZzK28J4p-<7`9v2s(y?ieS*H-AM_dtDjC(wQH}1||1Vb3b41>goam z?iHHd`p%AeYhxGfU5FWqQ3c>24;TpM(Z}V+b`7QsjUYEv?b&K`BxR0M&y3=D^lh{_ zeM*5j1tZLLz+{4m=y+Q2g89yXpRbtuNF#s`n~d}t@N0TJBPy}2ZA z(hL4xuX0swniStrKcFsqcXOpbRd=C~nZFpFNpd zzw^tu&u3+ zn);g;JTHt}sf+uj0Ka8JqN{qIgH6Trs(;sbbIGIwJvs==4VQ0fcQ-w+@8`HKn zRF{MJ86o}szbY|o>YMl>`>|*9u^_sTl%LasJbLb(=dFJ0Pc?VJl?1weN@j%PdFnGZ zYpH~pZ(Gz-+5gf8cDXvGA4Rm=D)aNA-pEicdA3Q1iJhX&pw=S-!M zfc}cRojXa}9Y~O;uvY-ErC^r02-*@%e2~thUpHJk5Xiv}hpbkLnz`j-k}*H(y)@Nn z+&V)pO3okSHBzshbo#t`zK6HDd*Ma>9INcT>mWeQ~L9bxlk`} z*?N(0a2+?B(cyH|*oOZqQ3(H4qR{T*<@(j=#n}E=?a!{M8FH6L4#z43u2?9dXIJzi=pf|vx?GQGxj1JX|y%skH%i^Hn?3M z5jrd)kjUgXCPTv~}tn2*uCmr~v7#+!E=59(P49D#y(YZtvyl#v_WQNyF3jiKG` zeJotj(bIstTwyUUBN6J=kwqPzU%YR#b8`43kuh}cEK@blJFV2>GW#T9m9p0@mfa|# zT&FsiF!9Cjl7NJQpW_{lhAO6>ENxHSWPbP@!RP5#M_*UErcfu#a6U1fW7*%T{19yu z54&*{U=l3aS>w_}rkaP|z{ef;b?Pl^SKBbyE&X7I>Jb~QMNYh+z(mi)64xOhBo~A8 zJ?5BCO1lH{wmy>=a~HGuRG$Tb$_IzWFAVA(=k7Venj3{8kX^3O25rL-z+X@Ml>0Ddut>EZeSo$IV;aRbM}$*Y4!6qGe$RLMp+4BQVi za1N!T8dXt$-pvM_UwQFZIkxXydzS%skz%OrI{uU%1qt(eGvx|jC%&##KCKI$T>Gqz zkt6F17l-Vp!$Brsj{*zYZTC3doD|NEFD;0(@~j{;Yo`r6edpJAy{lpj`$jN)eQi6O z>)8q9#iP~CXl4KSaTN$i9SUN^Vs=kur+@XZf@9Iic}fsl4J}Z-9+XjWpH;~ zlx}%{ z*&P-eOTVpd7e>oW66DQ5iA~yU{n(T}yBp(kfHV+<^)<$TPJds)JKX%Lv^OEA;Ils_ zRE|`Sd>D-ICiH<>&3C<&^7cO>SxQo=Ak!{jjGnpEeO;$qJP-R=8L&P2)3Y26HK4jE zC+U7Ku||BQC5e<%SyYu)-`A<0fnI6@WH`rui3_ZX|No_7owO$gTz_U2K@kXBP(9DRDpqROqK4#e0(8D_QJ=_PcXpLuw6IUPT5Gpl zWH5-=W%A|oJHaGz=}vEx&1AG9%9}#`Yif>P?ZZnx5uQW+o|VHBR>ND}qVMX>d{B!w z+v+9;_YGSr#@VcDN?W+}=IXoV6bK7gw$&u;Gj&?K5>kE^W@D)(ldYCsy=C&q_yOv7 z&_k$jpxwc7by5=4O{bcE*?k+i3nvc7bfGx(?1ae>8&dl;g$|dPT892=w!RD z`$pE6>oTE%9xA{MT?rjnOOLPnkk5xhC`q&5?sEYN!GJO2C@Ihi!p_a>$S|H1qW^VR z|M}c^2QRga<_cXC?(go`zi`*@xZJM*V=&%A#GB{d@Ff?2$O#BLCONpWbAXw!Sd=nn zPF(5}f*&W}U!W2;V? z&1}4<(3*`Ocn(huT?)Y4_+$vm!~H@b?r#hfxFoe9cf1XO);61)n-TGQx6A1JBdWih zM>ZUQ_*bd!#D56=74Bo#lO4j-LNcpgRo17}VNZ#wvEkd2bgd?u+wB?DYa)I*bM$a^ z&#z+jZMhDAOx$8fcSQ>4I;PW%I0<8gRUE6CVA1U}tOKbz+`wf|F^oxHk3iuDb*ctaIXDGz1 z{124#Cz)>%3a9`L0%6j`{onHSRt~i{nM<+`vS&ClU8)8A44*H+F}^gSDWrd<`NMNK ze%GBZV?G7pz@pTW%IncQl`_lrXF$V)^JEcm@B0D+Jagiq7FuRzIc4Ptf6*R+d*Tf~ z0biTV_8jMb&>}*Ql9`#=b!Y-JIqq7|v_0`Syv(;dZ+HcKN3h$Wg~fQud-MjAuch0p~j^$LmRllnVhWxZ3Qs*T}H9-qkY8b#g#)V!@ooUd=9xv^?hSf*0td!cUIP zeK-3uy}7oO4iF$;f=o%+B^; zUK>mkAHM5&=x7*)wt5veFJ0?s^4S?^Rt&G-ubCM9E=}&g3HfTB?Ga!iISb6{&OHr7Ujb>ff{;`VMrE zMppmu5(nyOnHZysf5y#oweE0n4X_FiMD5O0kxsle3(^u3^K0O^W!ACFx~?+L!y){1 z*E>G)i2K_Y9BJ;fg(Q_8PU!r~m(wybY6Chz?kDqjG>HCLGC&WD5iG}wXvE1MFL4zC zvF2f5-=Ai^N8!Zz4ubc|s=6a&KA%Rv$3~@>3+_8NsYai)EcL$|B=i5 zKmdn&fy<5&W;b0UP3wL9abp3>mVM!Uy-2s|?v7PF1=ShX^**-^nBe6{)cV#@j?P~m z_wzPTxlkFS_9Ebx-SK<*aIQQavB#O0)Ao({@!k%yBj(%f4bI*rCy|ZzFObo z;0Zxp=eXzf@dVZauoA`uq@$C?Hvu{tTvvw*5inpEoX9xE^YPXa5lT5syZ^{Dw zm~L^;uaJ$J$;}Y30P5fP-NUUHJmum4QBD7jnE`LYoKOqCDZl9qC5r^QXll3|TUu@% z=P)K@3c;1Lt$oQGO|tj_XEuPwD$JTx=Ny5fAha9p`Y1YrQS)mi zFbWB8<3{bk@6+K$PD*{&I_j^IQ@~(xU&a;w;qm{aihZ9*igBH6KU9Do@e#hg zmC;%9-0DfSKAOkrM=*hmq2gT3rik}PTpH_b`%=Hp#q2v>YZqoIJf?h6{KbhAEHHgg z^Tbb}-HKX#uUQ?aXBIT7JL8|ad$5n1LuyHl5z%WtcSLi>x+7CV1qRFc z=sJz!X|Ve z8xGjWGyBpI8rgsKwf%X$QK%lyIW5QKNc0ft%dtoMlwA&I!M;R`BV%1MsU+dLTF2T=Dlk->%7@7FO-HeH6z0+DxEin9dZ2+fDOmdo)?EEbpc>)=k7uP{F{7o zLa4AFQ|?jEUEsxlOPwtzUdlu>Q9w_3qSlxbV54@;5e@F}PS?CWt8@U#&i631?kcpB?w?7Hr3Ab( zb6#!Bi98#ob^M-=?SjDnYO4|y*L?rBr^@g16#Gtx1lbcJ)vxr;cc)MhOd=b}IS+{O z9_x`OSm*Y~UA6UeiS*WZ@%Bz=63~`$ekK%l5wtD?*rA>Ga**M7}O(9-AkKqvxv#xIxfURg{YVHyyZg3 z`S-YPFNRp924K({K1ae{ZBXU*D0?DvytG-5y;_jECpxunEKrms`wGHCD*MkidPo{X zL-P{phhst_UnKRiu!jPW67pD#b|`M3EsU=UZ@VMItQ#U@v1R458jGNC#fVy2DZhG` zYxL%4Dt{IvK@Hdtc9aUMdX_<8?Z?n&YmJ1y6MEx2FhSX}P#z5-DoYy|JRiuzZjbnS0`LvUta<$@EdvaTkDtag11f@|0hJMI?y(z>3 zSWe%``^3l2si5|M!wdrjxR;8|b8lPjF#Asn_^n>TJ;XV8UNTc|8oiL0OS!BR;IRxo zYVcT%??0pszEkv$KZvx}X;#`A&t+Ye`4Thy5@tH=I`6sPTkhEYY^ae3^9k<)I@(Hn zy!6?YBC>*`|dBBDAW1v#D_ADK5)G)naAF=ls z{i-bKp);&9U1iPM=y4j|G`Vkfe3ap762&v`C2rerfwVchh)XpKZ2aKVMUg&E+j@>1Hr=jBi zM71X@Pd9Kqq6Vi(j%3m?^9!R+Hy3?+XWhkB@?3rCv5JlCRvPR>p?vHZ7>|N*gl52584CGRxB42O1dC?Nruh$^8$2?2ii_eW;U&o2he2MaITc|W5xQJW@{n4G@omy3^j;3c*ufv;X zfCWv!9i(Z!hc5j)DPH40;})s1Bnsn|n?(ix^GgA;{o5*4AO7ZWI#flZYsUI?cYCr%Yx6QF)af2ixS;~zR7<6AvT+jw;xAzQEe zY?U#Q4(MtQ>r1HWlEf2=9?p{K?o9DE=QVwUgiZ=nl3|a=mwo@;0R4XeTR0{596!F~ zC#@abm^0~}6FzMKm59I@(hOF4XuIss}#dBMc zj9>nI1N_@l5Rgtj-9R4q``ZH*P=*m9ZcY;frX>MwsdIW<1Hyw(If$9$RUWOr4!nI| zCB7NP%E2F2WHZbT;SRP1CnLNchO_Xu67<_-?NcEFqu!vpFJ~ok_Q^}tBE5+UtAVS) zn?`omf`i)0HV2L?D`UQkT^UM)xVbZ-Zc z2s@{15!AUpGGzMOO?ZLktChRKI_O~jw1W7Olmu-xut8HCfyFOGv>c?{BXtz)ckRWu z&G22p|9Xn?L*fF>Hv$wVrm@#gJ4YzUdJ|FejzmK1#f#MN=t$EV6(4RmZM+R)X)A4$ zSy&+3K*&VPAL!=2eVWqKRbU}FcCp;&XW?(g=X)HZr*I{?`$me7*}o#R5=&QZAVx$o zH7^X4b3edC^ha+cGgp|w!^|_^)9|&VhqYi<+s@kPGsX(c;cfJ{Wo0{~jQD3CGn#zI zVc`m7U(hJLT)vF4NbL`#DL%vhuqyrr^2o9|kNN$RGgp18?>EAQPHY1U$3d^abHK zhU$;Xe*kZ61a1|-b8e~T%S^B%dS4U>7EKXLb|HmxHc&_2gV-x0pK{KNNWn%P&XtAn zsN@7j?+-ZJSAK{=Kt;(%sSI##@jx7XJX~?fd|FHa3E6m%-R8`8rL}Pw{MPV%XNQtm zk{R<7*Al@Zp+`i^6e5*`zb|Ggx2-uWxnwZCprx96lQl6yOt*#Yj#h`|c>K-s3I8kD zYss!qMLsD$qJXD?$4}j`4>={Z!Js9&4aN;msUuy4vG)=pm(|M}Qe+RRMcP4)$a(>z zr7jFBFhKScXX!#bkW^Uvpm>>>0XAV^%Hye_EA{=Me2PXN_)==>CQL`(qA$hEw`yh~ z`6lbNTu|~_Jv$D0dC_EfMM)AHA6u!E>Ygy3kp}HFJeF{+7)3$-sofNV$ANy4QTrT3 z#{&O$T2N>n-aM55qlguV0KZl|IT=LRaPJy<0-tcDL+QNW&Fsv2oGe?jtjO~u1-{VN zS#f%#10+Jm52oLNRp^j?*jo-?AW7U_xSJ1q3cWJD2EV&|H21JJ9&;yKL@h~-eL@%gn3$RRIzYq{9jw6ve;=DS+#7aa@Iw{g;F5DrES{(R1tLyq5Fcxv2sw$`?Jfe=Sh$bIfu}B7 zE8p3P0lQd8n*=By5&_EWuS@Ncm89AZcWGR8l6MHqk7gO~hRG{8m5XOz;-+Z(c$F8^ zF-hX>z7~#r1+g)N*Rw& zEm75L+n4;}a%)zadK#CZg#li;}=)7mhNnL zn1H{VJlqs*iO0O3bqo$OV!;(usX}$EFxibfas972t^1pbIcx5}>CP1dxH<$4g$;Ho zWct2XmgNlijAuUFMB&Xq>#E63#qcQV3T!HUV#xRIkq35a9$wyqQKC!AmIhy1tZa;L z$1xdt9*qtmc%cJtc=r)24^YkoZipY&9Y^L^OaWVU$W|JX7XB}8hbL(KE^T^0gO3aTu|ah z*+7qZeRW(Tyc_|nq`iec$cxuWx#)3tZJ)C#r%Y9$sZ<)>_sO=BpzZ{ zzs$C?R``o8OW+4y!wyRanndd#71g}H!czze9LDqAum-&daogok50uT_Uj6Jq_?j!- zzc6NbO7qPcC1H+(4u8JTIP4N?dow55^tIqDIt1=(H?Ypq(zSP7$M~+{s#ME;EvJTb zq{-hrd=WR)jR#GbF61Z+W%28Z5={kVo5RxJHk$YyV%1ai4MTHyPQ2kpO6OGG*>YTa zX&$uf=GjL-9nx37_Ho>v-K2Q?H^1FCPkFv6WYjNl{s{}xSe6?-uwk&B6+!}ewsWHC zxItObAMR-jsb%uY)+r>FKfX{Nfr^B3^^Pq%qV@-hEtc$@h7#OA@cNl1@8y??e}#;4 zfdISgD1?~t7gqXrAWR+(l~=|wIgWpj#>d$m#E~rvZ*ghPq3iSFHPCMz50`gjf_==7 z&j3)`JIeL+V~E&Uf|Td<3zGbZl?snll%1<~p~4N>J}& z{2z%H$?_403*D-8`eXWSX%TdKW;!5DwD@M9nI`9Vxyj;fvgxjE&;lJs@NJhE*J^{N z^tgnteB$zQPL+tIX7XVqQ|*pgjV=!g`f;D3Y`%)Ik1wlOs>P`42j`n{Wy(iagCN-} z0pp^W37`EHK!xkVit6S#U&f$F(oChlcq!i9ZM)AQ3;z86uPm2{JXIKD4%dfWU`o+{ zk|Vy4DO2IkEiwANV=sM-B%X1&lTa37ig`>sAmE>%R~*!OQ#JEs%CgU4fAc^Z(Kr2? z@0IB@W8pK^4Uku5)>8QNVHcq3rV6WAVj$8I6kBc=-5DZYO===r=1L<2A4r((bfP&p4?7p>}@j{6sW~A>>BeeuERzx*^1JiD{12 z_qNq!YL`6!aiD)B*$=bDEp@%XzuD^zo(@s4{l+fomKC-hrX_}ZJZ6PA`s=gck_{c?mkC$&`DCq-ZaTYcpDRwk zacB^_k;Xm!9>Cm@!MTZ7OKjO6RU5D8=%%j?wzB+ZGtzDqnY$8RzgR%9!y$ZUa#q`C zO8}&4=I-(coN+Q`Ke)$(bw-Itkih7L{;eiz$M;ui2_Iv>-2Q|a=x_UdeU!w3G+gfH zGqo>&b0QjWHG&n%5XR%^6~wYA0>{38dtuI^I*apR;PoKhpt1R5RXF95W~|Y;G!BD` zL@0$__63p1zUXTX+?6&n1u32`wVaM^Mfx35d=EZV+KitFo|ZQd1!9{Kqz%)qD1UI+ zkGDNycB0`G2A8l`UK5l<;y_b~Q<&A&(SlBrBN$uf>7$&NU-g>y@FZva3n$$0Q_q@D zmrWD}GPBPXK8FeMm!~RfxTMVNj5A;$=E9#^NRcr}f)qxeujZ*P^$ykwPF~raxVv0+ zKI%}j{}g=|H+Iu`_%BJ@Lfv5fvvC|~eoGBOg18+2@{o{Ig+XN2#aLtkIyUI?bz(Pw zt4k;~!t~CWt@e2Z+Mx-!#uNP#ZyKJ=k5+_SWI*f(k`Yd#%}CjaRy_rzXakb$H=JfW zm@#NHI3G5yPfQ=_p-3bZ6ZxqE5x6y50+=7C#lTSMnY@G#`9_P>$ImjoGR|JsFyo#a zLQ-m~1YADA6!q(n8=E8_UOw4;MQE4ZkCb$gz8_8e z1NR+o@27j!or9`BpbC7OSL}2Ld5(gYQhXzp* zX&6#q7(j+@hE74np&hzGfnn(G_5TL1v)@Xr?Dm9yJO){)^)TWO_vi+fv!W=v~42@8D_qik7}FX{a*>?F*AIZ-Dx= z;k>Cb@-15Q9ntFnBThdFJUeqETXLp6I-4)|2+qC{e6m1V`tj^kBWl-z5+8}4E947* zF#YlqgilX+5i%#~sL7MyIV)q~I)k@)fEaeVPn<<`IQFhAJgP2^?AV;*YlRQ;`iBu$ zckUpn>5=dCi#MXwiP~!~1}wRE7uKI)j%O9yn0OPD-L%TjpP%kKZwlt9<<&6GC|o`! zdQ z2_?Ii{EEb1DX!GL;+d0El%z)%-kbIP^7pY0mE(VJ85aqB)it+P@u9qtJon?5L7{=; z+1^jyi0O3VEYy$Z3r|(|d8v<#r#P_BrUAd{!vgK&bdwIoYtD%d1MM9?KVJIsY^%3s za6cwvySh^yrs>n-W5jXL${qR<(1b3uSs593-}GR?ksTMyg|i&hUz4C^YH);CKG&F-QPY+)5Jp>3nHxq z^6n|eS5OOA@zDehg(NCCz-WroU?T)DoUtY$^GtJL-Kp}}KIX9L}BtT>y33oDF`Iw)0YD*D7R1Dj%wXWlaRX3V;;Z*)m*><*~*4|Y?6)2qbSm{@D!plfNI&;ghsg6FC?)EGeML8Z6 zohF>Mu%$SB88cNE`SlQ$_wf|Qv|MME^x7v4;KYhPt@{N+Wxr$E@UU&PvvQEdYEp9V z^NP2GOVV`lKDo__|E(PPTS)>iExINq9vFBLRr*J(PFQ6KzIojI_BUHOi_y?_%5d;OMKlR>?4||K6 z!O=OBk5?Lhno)|)xk4waLau#*yVSTO=8Zdb(tQv<;TxpGtxx&7DP+yXi*_5`)Hb!y zh}V`d`~{^wB6t#b)t=x_Uc`U+2KeBg{oh6zG2=fq%5aS>VWL`OXSVRY>eW_|B0YAU z9Gh!Xiz(bNL5?B!H~^wT@txWYSa_? z?HO&wj2W-uxSTTG?sF!p1ctmJk^-K#3n=uNy?>SFFJAed`06R;d7}}lJ`(ep@U%*W zlcW7{tH~=0z7mIRd|#P&?QS!x^hK$szi>q60FwhUbl20J=3hxrb88o`YrQiXt`pcX zqIA*mJL_#3^sW7K1BZ946$KdD;jOoR2(H`;nFGZME*mH;?iBD`un00313E=sEM`if zv_|9Xrp13L3QrV97C-uY{LG1qg`WO=SrYgz?)p8U z$XQ)j{`eQ$u+1+HUFcPpogst$Gbs4I5fFlXwwo-Q{ZBCSmk+nf2~UPki^hBwRXvvQ zvKfzmdeTE>G3rcV=e@@tNzj`-$+SLeh8T0Y)?o>L^$KVcQ#k9H3L(}@_yc0SAQyN+ zG2TOCpRapZq*GrYx^ZVI`oih`NMI>qd%8!+Z2FA?gbcd&?qOAj(UMCQr~X>fn3I>3 zrTE#!$~8THr4TG4xAw0M&+M0z#XZ0)_jsNb)*U!iALWKZiRrUFfE7p?Ky`JgEr*4) z`$Ghi8u6ju{JW)w{3l;Fuyw9wiW{+#wbRjHal#Xh5AK|owRW?37tnk9obhB1& zB1YXP+k-_2%1>SPd5oSZF6#&zj=<`G4VXU#FB`l_ww^Go{_%9zH|G6sLY}6Q*>uF2$qMRcCAwOP zQnO4POiebP#ug-Vq8%Qfz$74`HDKNy_fBa+>?enjyMIZ1_r$m#=6Q$R zw9|8CesLhIS_9_0gj~9n@|Te2mn?rK^|yMtY{6Q$c>J04xnLYnJQf)fb7oE-5tDH< zGML;;nZ>GU8}9$7OCYguHbvY~-NIMOfPdnK1yRzYcA$7eY_-;C!q5D5vaoNnb!FXp zo7g)NrCmqIa@bHb1X!Yc*2}D(Wo(OyZuMcg*vc6{NzUcU%{7;ZY(3+&R3L7h2Ie>Y zh3kMXZ?zM;Xk0u_EIa(B&q-o7D&LfOwhM8_UyL>?gHL{A!=T&p-mP8j<=x3-Itj5Y zH~?{ah6s{nDky2?scF&pnN~5fuuysQJzW1*-;x1sXFlsw)tG(D8}$oV4Y%ogn7-iF zUr=f}Xv_jzF8i(`0owF4W;<5|q;REVDuhBO`Z)&VLIgX- zKZc=jaL4O_QG8@WG6jk$89%j=_)E@fk0Dj8Yhe&J`UG&z8`*>3U+XOjo;Z(Zm0_D# zc`3qAE_#E=ygW&We*ceI08u%hJ|ghjxkpIp6ADX;D}uDaH-d)!QG%R3o-b)?OZr5I zI|*j30xg+n>27orWE7_Hi-quO%Z~b^D%c`DhhkCvJ|yssOkYW(XOfa5dN81X?PzP9 z)8lH{tv;92m@n2is#X)lWKL(x&gk?n!QjA5qnSR;Q}cZm2JCEO$!Y_uk=K00^@kQ5 z^C?t;H)!BMk-|MQe?twv`HPDSQyzO0tusF?;=u(q0uZ3=4x8!2K`F6A`LRRwh(hjn zujzR-*(%PM`bsMX=cE*|h*GTPoj-j344Lz0esMS&1+;(}hs^g>BhHL~hv`s{;yTv| zfBfM^DMqehS{1;Q@O@bf{Qxu|VyJ~CIaU?DB{aON2ned}o3&Vgu=jb;s}q91hYxh; zbqER^ov8XKYv>r#?@OLygm-ovnFv^( zzEaAjMpJEAe45+T)`qn#HS;ETz8qct%v|RS2 zV2&>FhJ_T}nt}ao`pack+f5hDgyQJuX;x)!{1HIh8?9Zr#imtUXRZ1)!gX`k$7@F( zXl~@p3FKy|@c60Sf(Hhh&pmci;J%ajWT}f;fJ=s%)2}(zQK9!XkbL$@^49}Aud`;d zaB2yQtq(w%yeZRx3gW+I@>JoXr#a*1tsp99JRa&K{n1RFr{TRTLt5!Dx-x59 z&8PaEr{}lmW_To!1fBk9@oxVgrkiQmkqOkgRxJbtfpaBP3I);QoT`N|Pi?S8Q{ShD z-8j}e$;(aZqtq`Y_#?)3tq?CM35lP-3k9HIu2R5{!DdvS>;?IT5DAL@+C68s$mu%I zZ{5#tGx#$AQ2I-Ymvi z{%@hzA3Z?Wt`*Dva7muLw*1y*pu;jUk0`0T{2E8Z895|>kh5gr?^NDfmfn7Xc)K)Z z^WC@n0`Qn=9!p=<7lF+Kzq^v}&-okPu3`qs|5ZChmR;t2?k<8bcrg}GyEJ0 zQi#4YUNtcuE&Aw4AOkASHRPQ#`*cB^l=F?PLbjgvvo9l$M5CEid(Yb$9ptc+QarLS zipZpNNl{24jsK}i`!3z@%`yulyW0Y-&#HT|{lSRd#ZkMt|B$jq7uo(ZIqSb#a?Nk2 z>%v%guFWXp+ledn?FSAD<;qMikdjH=&ey%=`}AoE9cyrQ6N^@p3M%i>%QX3@ zi7U78_0pI3kt`nqqA?vQrIs+UT`!~V4jyd}EbIBo=+B+M{3!Jb0Vua;Z3N-I z@B6)xqU-0JTRA5Sg6gK?(xE)1lop>GXPs4p?ronEgLjbr8v0A*l_hkA65V9z)=a4BQHr9>0Ye!&8LT}gB7vE!)~!#r8A|5ICp$I_GwOhe#c$~ zpRyz^as?GG)Dy?cIhy0kT5J<|)8iL0=Ln?Vox2fF&)4TdN^tAdIYPQK63-DBF-)=C zI(o91bcf4fuOF)m{i!>K9ofwbUkM3da|AHPh?So;n4FS9p z8&rX9N)dV!(-m!+a_LZ>O8$~OGX94};9G=wtLp;>pCPIB;07Ru>Vwd zB&~O^Yr|A)GvKJcp`f|c3R&|ByJ`9iFdW z;{{%{*PigR3iYw#Ml{!_rL_ZE{9S0>mTb;sJ~2R$@HbB9Dn)p-ofR$qZIHdv?v9xK z+zi7#f_*~fhD9%q%%M>dfvhD@TGJ$cDEQL&Amzk*tJ&(v%(@)ENN~L=5-%PR_I)wY+^<|q% z9$mh)Dn%t|v#}^~SOhFcClDZ6qcFVoXVJU$>d`o%cl+ikxIto@is`B!81IAgO$pN^ z$hxjAVDb$)mh$mj;{s>S1^5*zNxMKue`bFt??ia4%Nhel1(nR=^VEqJ1Yh76b6jFw z)3{e}vm9z4{d1PFgS@z!Fw;}C4y#D?ELaxiA4L0HV850V^U$d+O|65oYG&Y*(Fz*U@r80Y4}pt84K z+;X4H?@W1)utv`GDaB%@)o)Dt`+J)5T=7;PKS)G}Z8-#ROXi3-lmTyr3O!PIHmyvR zZ`BYfRekKmn|if%8BjBbz4J?g$l|v`c_mXFq@X{(SQKVTkZltz;{Kt;I5VQs_crR1OM){uL(mS(gW~G-!=y%Ql z55Dc%+|J5{3}skniq4(M`kzj5_R$}Dc`w#f|3ZS@C+PFc>qFqj_Hw?#X+~l)M0-N? zr5%lV8}%{Ya`)APb0xA3IHRfAM#4q)4*G3UWJj5)NW^GRK3RH_RKdyz zDpYopHRi59!zd#;4IbU@mYeJb_c*wJ7tt4Le^I$H^8F@fw9k*b|9f@6G06Nae-Ev0 z^@l!khsL4CLgaIMdy7HTIF?(!ES)T2@VX`hGic`PGB!z^XiTmudTlb4pw&8L-S_6C1dP{IYSq_vDS*1e!%Jd5Z-bf;ES) zR_(q%eypCN)fum2_hHZ2Ob>#esgNiW{P zK6@)H#FWWsDbXMu!d{jeVS^Z~UqfOax;lFXL8e;B|Lq)ipA#OvZ4njm+3Dq{lDC$J zVM)t#_k-gmt?|Ld`6F`x-wGIyD!^cSLBYx&s4u5TkAj_DaHo#a{mbFD82!zx`mxxj_<|H0`moo3vqXRSzkrloC)o>VlS+S1O{>{pnIV*M z1xw4AZ%nWps#4FD#(%j_(bT{lp++D+&HMCHikPK z{q}2|?8x>U#shXtJ(5i%gjxBaJeSwrwws1B)c>O(BItMwSlH9T8(5me@rU#o7XYV| zj6m2hSC!%BKV9oL=HkfofmjQSGOtp{4`MypM32OTxeG@ZV`4M5nNC&Rfhoa}4aLMG zs=N`I1&XwA+Zq9256 zE6O{UH&+BT@AmIJ#B4Xv?*}e)N5aG7%}IAsw}NMUtCvza(6p5Z7YF1FZ>#)DQuQ90 z*^0B8vzQbzstQc|Lnf9QG+0^w0*8@@w+f7bs%UOuc`dGOZ3mDfA_$aPdRU+sFUFr@zF|gK$oZK#aPs zsv(O%;)Q{_sNx2J9oE0}RdZqVS9vkHGKC7V409$Bjh`~O>;u{SmpvuygO9~B>HK+_ z8PytQNVqRYSQIRJE~ZYd1_hY)T+&K4QC%a^$2ANiyCt(kE2c@-t>N!4WURdiL=Fw9 z)hLW8vs>BW9#aP8ZPAlzLFx=AX|P48(^&Fo8)xNV#gj#+q8Phq!N`u@iWrxmtzv$4 zaANxj+NEYo2xC4~cS0S51cT(SD&)!U%9+HAS1zSea*cBsaX|FMmaVK(stUA5XT8lC zk@hfNLfL^IBd579aP7i5p)Fs$CE`z+jqWZVy34THEhkrUXMGb79>g8{Fp3RYY&{M} z6cvx6qBVZppRo?wbofineb5=D-)S53l>shmg=iUZo)M(Hf!NFv+UE&4Ny_(0%O(fD7`ky}W$CS~ zZIH)f_gVun$me?X&A;hS{oA@KQ6x+tD{=u)q}O=?k5&)f)4N1R43ybFMelR_sx!1N zz4cJo&|s5qcdpvI68a|k13L{4I4Az>oSWlumZ&`vt{XQ6^~LPLdGB7u6i|FIiMLSr zT)kuhbb|z%4ER&C=Y76Fa`@B5+w?(4;BrMCvH4(L^k zP?rKpWp?km98+Ir+H;`d_fM|PBQe4q^|>Ub#(}JCAH^&6a-Mq17eV^=TwDp!h2Sk8 zF5Z11os<}I+bCv^wnn|OEPiY4*Y{fTw$j?%1DL(eAIv&L4*9CL>d3h`cb=-(F6TB2qig#stZF3-o~*`o*=o5|o-lNjeX99|ssB%Q^HXT-XxARPy~<7*QI|q?_YQpP!uk8$$2?&4 z?fDtjrT6A)bK4V4NOp<{D4!&v2ZPDJO=!a@V#JOFpM7|JL+ytLQ{{Nb!nB(q zdel*25qK@)>wc0;88*jT7%jHxbvh}5tr)bTcyW_&o~JY{OP;ETd33g>_imtI`G~e{ zeCw(xM&-z5WI@zEB-FCh9yVV{&z11Lm}Cylxk5Lv`cCN*rJdT6H$Fp zby0ayH7CQ$PR{mlohZRtLy%pg6w7*YaHVu^D48So3ly}}2yzgqXi#nh>)Q(ow@Vlu zE7;tz=s(Uthq>gw*d%!wqA+;)2v*_}%3r$$e504EQo)xXmtN40zwNi=~jRDD%FmpX&SqHJJt6eZ0W&=SALov z)o|*nJkylnm@c;xF`vZi3RjoXiiP^)NHQiByyROIR(em=EA!jj$)I_y-xh6zNae>q z@xK}179%}kR1@mp3`t(}z}fa!j81P*VQ}@-S5NBWLQqxvLq%j|Es!d7IXbV8&Dza- z!E}r?ONgjsu+S25PiKMcaT5#8KkHM~uqKJN3QtVF8%Mog1bz3Dd!Q#@J5~DXxH`#h zE=~JP`uhcMF+okB7C8pmkh)-Wzd2ev>MgZ0-^*oBxudvMo)%Rjz_(z{TG&Ch%}?Uj zx0!`CxaCPL0z;@DR+KY|oiAE{9GE}8p7X}BH`x>KIgn

    m&FU(VZ ztn>W>j7|FO_7;c1Gdkk4VbG}31;yS1*lcHX?sAQx11$1rMklLae6wPF=fF+xtDFHi z#qvZczBQxDDD8p?#Svq8k-}-@65V`ExtOFY&kAYQ!`CCri+WjKV?DvQnx@qW$mRdZ zC4cn#m0wv)Na9I}RU??Sb53iDpycxDe5IdvtyYQ=Y2(K&8MaP4jj2NDZQ{{KqJj*C z<5Mh}uPB^*levAJn8u8d+TAaSlp|MELY7o@W7pZS==PXz%5P1;&g7V$H2!g5gO|sT zLLJ+|)tFs2yaU@}tf!x?L0ZTsQX9c)_QcUGvrlrO0r~0s zQguW5*g;}(W=LJ)$be`&l-H6s&?>M`hT1+&T{XO+R_d z*4vWc4e^SAf9TM&GE-&p(ko*|C*8F-H&iEThbH6c$#;i{4|<;uSNF|WhQ6z36;Lf( z38*|d%|o|4x8k~5Q5vW(j7bO(Y+huE1k0ojr~{8tyQD6i^GRUZNbG%X?#by|1uB`t z5tojIyGYTygP(Ft3N5Lr4B7uB8hVsTvB)gU;xl2KHsA30IUCqqB#m z=QGbIqTs~GfVx#5wYL=Dw?%`0e0aEuvtsyXA3)5%UD4QMzHMMF~e3% zgIejZuUXVz(|lH1tkuQN%x?g!gxY~%=j8bx?$lp1Fx$7*p0BoHAHXTZkue(`9@^1M zXa^cl4~0NuLz{A}2P23lJX+Mu_l~6!MrCe{wVw4$r?qms9E@jZ+NtQprKd%DXWV?GWv>oWkK(9TM~=EED)4r6NulWja`~q$|!f9fR!S~^Yg~n@nm0*og^0L zbTx$ji8KE(+WKem|L?oML5=?(y@xF-f0OJ_vReipp=W5GKcoL`%(?J?QmDUqz%G$Z z4{{g3+!&D6Iy=!l`d2Ne+xeu#{}dWn(H9qk_cL6Iw&IiT1ah26V_P$3GX^unP<$B* zD0abm-s-vnR6lAF6@t>X9hP4Q$^e!|gOmH%A75}Bx1g-i%%rYw_)7|&)BV7Vy17xO}4j+UEXh%x}>`##6W7M?4$eo4}#lKR) zbG-_jVh@cWi`mWPLCz?!*w|n$PFGJ^1`GJ|xMe(}i(@OXL$%6hu#4ivwH)kQ3 z2ad5t27QZz568KU9(1O_?Q8?;+4kjQc`anS`LNg^#l^+F>-e&uHHST+(&4D zdVhPO?5I~rRp_k4hsWzC@Vd@YB>Rtw{BJR9dSpAnGc26gC2Qh35Z4nkbEekabk{eK zdx3SZx8n63yy%i}TdhEa(iQPSrDP6_;GS?1wQ_Hl$lQ^VfidlZ#jvtReOD>IB>jly zfoV6>zdS}GA#*mqRer)^cs5v9js3u@!Ix>M7Q3&Wh{$PzFi9=L(fg!pn!Lp-;Yp^$ z%7aNS@Pk4H^aUK9hNV#{vK;a>e(@S#<5Yiq$A*?Du72K08e2xK+v4t9!_~f*E}l_w z+*QizjNLvD`2RjUa{T|`|DV5%mY}Ke41Tb3; zOH)!@ET%4|68Pr)SR7>d#%pF>yTXh6+s@tcg9;}FsXB~O5cm#Qlt{&`93AHN;t+hI zg!EG|&xi+@duB|%$7_z3PfF3#&}jnLS+V&5>Vdb-(aG2NIoc&Mz>&W(q9)XyzwwTs zU3}p4?qz^f3Eo@$7YKZwnv#{f7D!sHrt*`myy1l(f4@0r;1^0#@LzNeaykFwXXANa z`Y1e!=uR)l5BwHX@H2_T4AWvjTCDh>=jtMC;@Yp$0>bp>K&4{w$K^;4BW7F1Vfi2H zBf9zq8}?r_KJ@Mh3How)blqp&$5V;Dtbhx~ml}Hs75!N18#WPrR7!%;*FMN{DL{Yd z;~2UI9}1h?iv1ScD_t$se1qyr)Z4y*UA ztr9m>$Z;Zkz(4+OEv}LK@8*aNqFnG{teiH|aH8@}Md5Elq7y|K!ATT@f$|`EFL~q7 zCXYN-Yve^$5*fG4Vj+6vMZf{n!uI2~ywlZ$=G|HF%9x_NwQArIN8+`-nTog{Md!*k zzlO*3qTZm^)yKlI{-=$)D)(53ycETxVo%ujp}EE-%3=2X&iOjRWKaMxD{G@7<_uY5 zR=RoX?!*P#)GN2{wh<2K;3%lu&dqyu-(Hh?)}R0-!kDZ)Jd@@6r?0v@jqhC_`3*5W ze%hqMLw%e*G`FK?sFT@hbAEni^F0iK_1I^dZ+-V2vPIQ$>-P# zqF2$|&8&N(a2+ga@Puj=+f{(Apr5C_&8=;Y0-OKU2{o{P*8BXn;@iH+ry*Aku}b4|21DmpIJ(AI8K zbg}Or%`eNMg4By!%AMcOfEca8l?rzWYaW~GY1L$*LuRV)`g!8Ds7+W9oCuD3HnUqV z1DiP8jvC>o>nE@IzZE5_?iSy>vR19fpcwqiZS$v>opRsa^Tw}IN;C@HUP&rC^5L>J zlVo(OJ5>oE{V2NU?^5F8D*ogUa64-D{6fBC4b&Tfy*kTVMO2*2_uFe8ZZ(i2N=W7~ zF?C4op>cR?9qsS3aP$87*BU*GSRqo~o%V!Y<=aiu&HQ(R3FlO^++vTutX8}}N3J0{ z`NeE=oJMAI`1{5%tIybQdMaiWyggv@Tqx3^D&gXQJi&?j^GoTgqMJB>TsCgk9Z!jI z)8JyWWwPa>X7`XZp*Opg4Q?u88VCv-AC}&H-=)h7~pBe)j$89&(35nuDmfA zt(|xl=5VMj*>Gtn?H}Km%pj{AbX^;Fe`Is|czKm=oXJP#AgxbDI{+w&SR_;^GT6)1 zY*)FJ&tk-+RYDO1U3vO7)^#aq?_gTM32}cC)>T6IBSkzv36i&Q5dl%T!4uZTJmshR z@bWbQg2AWMJJ#~Si4@7UKn>n>b@qOJR!fGp2Jc=7YBJ0-%>@P)e|+mD{MG63LFJYd zRs)XUzAC>cS1Sk0Ut%xd;-Wv+)z_|3+|amkmoWFvUIOnymvH0r=EI>LrYS1QqX@U} z-R6=)fh&e-xL=Z}ig-oQ?Z>nX3X2X-MB4c2B}%cpLE5dKl0a;d?;x3M_33&A!Oj}* zEVv^ZyBM|f28Q3g(ECZM8Uqn88s6u+A1`-PK`L^|Vf~)W29$-U90L2%XJlb;z3tl= ze%%Aj~{_ebJ*d-t2Jv!1C%^|qu2dt#75}v@f zjZnAUva_*?-L;cB;bHjnHgt{ob-ZfYaHdV=wRf- z!~1>32pLG7nzV=#`S*hLg2VfZ0cR(aY0pc7L~n+bUP=aP%g)o9*2IIS90S6mpUGd9 zQ&I4+2yBnV3c!(Cz(nV;7v2oOwDL|szyu!Z@?$q_0T&f3G!hBFyDEbu`s}B8nX>tdh?Woihc@oS5PW0!7`8a(-U=vD&cx+ z9+{3=DUNXy#)nY4p7R8-Qcjj}x>(1Gx0w>T)Q$S*-T>f=Z$QFo#-;#}{@BbTaxX}r zyQbFiC(P*4PeKdr>wZyn2`9I67CO~^iGN&BkE^CRwO?|9*a|?Z%55^aGm1hW(Mq5e zY*BKe^ug=CE0!UL$C2ndH_x6Li3X(v!6fBdE9g70(4KbUtD$w79FU?wC~;Bkj0>$l^5FbAjzWRL-8)e)&(GBngkTk{R}c zUN56yc5$Lc`|;7*u~-`_q5~m?DFy(}*4TcJrQ~Yq)Eo^EhW-j1ViQm;|hE@SOkc>oM|Cx8^Sqn16gm?ph^Yu;up5 z<0zxu7)tPhhd&S96&paFz4WS?B1Fso!Ml}xEu~et1VSmCKg^jqv;&X>y zx1s%KcyVw$xLwQ&yqN&n53)FMDi22&@*`c!W7E-OW=?~shM!#}`N0d*9N zjTNlj7Mib7s6i&Nz73m)Oha-bJmC3;J++_VPxbCccKjpH6`EJeK@28*L!^c4T zN9*Ds{YJR5`LXCEoO*pcwgu&Ee%o*0WDr%K;n&*SzH~6QEnuyL8HAXe9;qN&c{;@l z^&FKa?L720KQ4)=;;Je~vlwMf_$p&M#s&?#eM~MLf0%Ymrh^P`1t%CKYUm4t`N5;; z&CbE2J?d(I+XP`P4^Fw@fsva6@eBEcp~)2aofe!d+jh zd=LeGw7d?m%kRW~X|1jM=S`YSqUtU$1RQa00ujG|S1{9%%HhfDwePyRBP-n>T?{g0 z*gLM8a^spq_io+C`hW;u)*@UQXO85nO+0Kh;5i*3#_?cGY4iG<;8Er_$B+t=^y3Fq zps!(3PxLSa`@Uc+h%p)_DP|o!)B>K_iYJxN=RbLl+iIQ5*d>^cRnyPfrO<3Y9N5OP z=41J&r(E3Kk6X&4RsqR7-mnl--Wc2RrA$%^@R(5k>aIrmUsWn<5T3BiEiX6h8`qK>(ycz-hY zr#)k{cU9h2+>FT5YHfuX_9bR>M-;P-WQ{FhH$-HuoQ021q~ZsZKR9#Fuy$BhC=JF|LOqeSdaNGxprYVTK){cy=+%Zo#_DnI&31Sr zPi==3cvp@jOM>>&7Vpub+5gXM$>6_b=a}(AxT?QXMRL8;kr#Ip8k~C75oU2*sb*JwGzPRZOerjYA1Fys!A(|%rm;2<)ZWAA+%$#l; z#~(?}2xGP*Vzr}1A-~GtI39(@hku$Asx>|Gxmx|KE-g-?^@qpeND=ld?AI`!>bc4(Z7}o%;GDZo zKuF-Zbkh^m^kO(I*dY-@a)svmdeu|iqaG9Tx*I>Q0rENsOTd2TO1|OOjqax5qS3Cqb4h8D3_Kb5#F!Hy9*$QVNHp z)iGCD_E*P0=x8}*6@V(_ISg(p;AOGa*stmU_GR+KDCMeM2=0m6rqsKxUaPKmk_Vzw z`#zQEoEQkK!-%_S6dmp5o zM)_&gJR$B%$e(eIqnMK&c1~S!EGkneBTmL0ZeuU%-6E4zJgA>pKPr#0#IgGK79!($ zf^XV@I3pW^tijohLES$Ij2>M1-RF8paB^+eZKvAFN>^@uRK8)l=>5Lf5Ic4UeyWqG za?gdMt&|gZimr@{dtOx(RTQkF34*<#>?0s)QLd*fR4OFUkOqQumqZrPRu(U{s(tJ{ zbH|-7?_HRT6~Pb@5p_IP>-y?uxx&tPpGzMGa^Zj;wX)rt1#rO<~o#@-@H_ExZ;UZe8wCO z>RAtiH+GDEa0%^ei&M{7bZy{w9`$B9rc%eQ$Hn=Y5Y>oJBBCwML>KgyI+MC`v70_M z%{HOPH@C&D3B^^+l2_=}@OC2!R+W%NQC@!+VM(*Q$cS5)O4bu(1%$NmkkmAAITZ7L zevjT38M~C<#Wxvc86H2wq$QOqd4Afpz(q~S|# zRpE`X)#xAoLB}7{`Nf!;dvEyBpvKv`cy9~VhnvI~$^r1%B6m@nh$=8!b39A9=)oZi zYgFrzv`A2rQx(6f(iQW=ssk+?&hZ9w`ak7oANk#?GU`NYK;~lRRK$-ZKcyeF!ID@zEz}HkKO6Bf*Pr)m(V2J7*;Dsk zRH%*Pk;Iqk0u0eKb9Lp*u9@uKbU`0UkXh2YnekyA*gTQ5A!q-GXCkaBMucTNg%Xv; z*c~>D%D9g84%y}6Jh7(*x`T>x%$-FfZl}P6pZP0``RTs^4}}qLdf4C;62sI?5wo!p zmlkN;c1u9d#988 z5QaBt%MCxNHc}&)mq*t~)7q7N8pFAjz*@ihUS&jzbV}xE8GgIOyw$_Z)rNFVydpoq z7}PYv=2Wy58N=WDp;wAY%15JbCE|{!Be$KBQ^a6&1?~N3_9z+V<~i(okcg+r1C@;) z71E4{N}1MH4UCKU0P}8>M$ztyv6xOxze~`Dfbji;c^0pfXm1+Ga6xdD%81^E$Y5K? ziYxwucF=Rjh?kQPZknqhs{=;HthMuJd{ldqc|dv)SLs6!E`Dj;eDR0Zaw2@!W9%G3 z@PE^ZzoqB=?QY8%u&!tuWK$4~)7>JU>s=`v3`y1 z(CnIedL}WkxzVoohg>*~=$=Yn9FzJ8zuYF5i~ZGQPR`-;NjS8)0#tk#7N!4zq8L#) z|GWX0i>#pnq4dUQ14M&UF1SY2F52g+q}gMZFB9c619j@0`v+$22iFqsjCM>K*Ji%8 zwinXnmrhXsyHL?ZzdC*Vm5wLoN@$oU66h+=UeiLEl)rM$`O)yemg|@!t}A2@NY$8| zAtro;vYLz6>@UfVleg`RFF-m~&JF3AWwqVMa;T4S_k5~=Q?uUu%z$o8KnHIn=~f}< z?lSD(a57niUv1!ShfT+H^82-+PfvDLOnV7;PY2))6f43e>=K^2 zJuEK#o0{~e5_`EYzqQGhid}+e<~fDYFzeJ;tsi+?#)5TjH(xYs2=Mjc9HGp!ofxZ? z1TTtb$s9tHs7PPR?vAKCn4(b5L;2WXIC4fI<&-$l#xYs|>L*XLfQ=#d_d z1;_;-#msZUJ5^{U#60ZVEX~lPC7o?*O7%`k9+l=2Sr7gWa?FbYIcy_Zf_ps>0%NqF z{zkLfm|)P2B~45z2kEp}f>3-o{B;4zvP5CVxqBvbr-FpVu%6hsrKTNua!9#7a1WR z-m6PiAr9HWh)dE+fvDMy==O7}I1dXSSVVTUgHzXP+O5^JtTe}eu7DF4qFY@|FhW$E z4Ev&v)mzf|i})LOM<^d=db~Idri8y}zVJVibPa&?d7EyDXIu`|vOgwHiaLYhf#Qo* zFf{{Y*gCjMc|l%d#vKOUdCNj z*@2xA*7WQaro7~umKNbaN05QzMC~8o zy}jfbyX5cz$4EJoXkR)Xg+xXsTWcb3qZ$oSc1lXI!rD6B?*Xty8aJJhr$}wJYD;h) zxDreof5KkRz6}qseq1FOV{7p1?;yAS$ocjKFWQ#q2g*U1!%b#$~#7R#iL3EnFB zWViZAwb4LS((_imTHoLnk1}IRvJEmN3apn{(M;*GBDidlqi3B~Ii%x?m+HEstSs;T zL*i`xiuQXf!?*eKVk~4idS>m(L$(M$AgCHc0UvyPZYE&}Gv_2ptCBEu)2E~bb;BC& zWqjE2?P{$>t-J9L^7Tp8!EoaGD2p_F#;wxRFg*!WOA|toJWev+e%A) zLgStDZ;WNCe7*4Txas+=xujZ`ikXIyAoV_N%_q2Oc9u7YG3B4=c!eanGWiyiRt6GG zg33ZY*dEj~9^O~mt;%bUF6fdBQxCF$PQYM9xDuF#t#g3r@|B#D)9d0a;K~!fH~K4N zW>oHij!JCmex(;Ykgb@l`+u*lend;M)=YTOKTVlVm}wyFQAf`50v1KowAW=jIv0Ja z>;blj*$J6&QhTI(?wsJJ{L{ypoz#%Uy}7jPuT1x{_Y1Q-W;tRc$~J~5d6Yekz-5~s zo0gwP8RsX-)-aNjPi|{)pM?1nJ@+4dc1ZE18D87KH!(*vCYmq5;<3E>W0MF@0lwL# zF!pJW85cs~ZCk&kSGg3!tmLcCG2&y-CkNzNz9SeL^?C41-bF2JV!NzMlW^pH8&@A< zl`4rmmj;*#TRr1&j(gyYjtAwVZCDJ$T2`>IdG_c>pS)o?K zQps4uv#^Z+@_&${-?_*$?+QIMNeL0f z1^h7P0lf;WcCiP)Aqa_NW|GfIwd<-h>Q@(2?dI(N!4}kw{kl2*5gU<7sI)z(+|mec zFt%Y2i(eYkQ{E^))W%!n^j8qGbcUwhvf)l}DY`-orxMU#*IsMRxhvlQyU;K? z1a@rcHLwx+=0rM}&CoA4w6pNyx>CEtOg|d?#b@`zo~6iN;dLANe@*br5AM?x2M$zE zRg>{|_a0ck-*He>*x4>!4aO8=;xQ1_<^VDikO6wcj^-7UrDT{Zv2-bc0 zm$k*yjU-z3KHtWU@vE7R4t7*4S^M=O5&Xxdl4;H6+O%6?Y*DN`L=5{wJIhCgA?)mub8(Cg9 zvp!a|)|sUA(Lr3?cIfsJPU(pR24BG*iPQ7Op}gT#hIK%Fbf@vfL$sy4FUy{L+bk8g z-+&kvaB;Nn-2geI<~DS&(a!d~;Pdktn}T?wM{5SLb{LVt&Zg9EbiP0{C&rfNIqRiz zo4jjI7Bwc$RtW=wKGY!Ee2eUd5BrDW{~L7s-c?uO;BGdNiC{2a>f+*51%6!iS?%0g zlg~jvZX)ErXO~=Eu5~G#Wg*Lsm>ef?(s?LRQauxSqA6&tFh@sCXZBS5T(*~w3bbl)nV_qSgEo3BXfm*4ssguFE?F4njrfD z(T&#{DHHKBEsP)6wp1JfK_#72$_4??dBrr7NUz%?g4MH(O*+%dZ?8yvuH4!@+!_}1 zWr}J{jJ!4D#1Id9vPbrk)tzh+q@j%Q z`G%r7qV0nvuXc+K73@`pMT&nNGsms}OpkvSnjDw~tV5b~00v2;d&ieO=?a=d7EkB3 zposz?>Qw9V%R|0tKmr3qMlC0orOCbq{lXG;#ya>*4ZrSBSgf2WS;Py}oECswdl(Fua#OjSm57Cu_;d zDP;6}i&d>Igk;Z>Ez`PQJ`}I_?$0_hZn&?<*td+I9j*)XY9FrdqX-eFv<#>x)pK|a z92xS4sq`WdiIE$Ok~@UX7w6yAN7k+{-5qimzPgpo^f;iIijWh*sJ3ZsOX$|BAACz`9T2 zWSGTtEGz%62WkTb^Okzn{ExNWA^$y8F#tckD>0)#e+>M_wAa)k9zp9AXE;!;juR^- zq!)RlV3%#)$iY-vA5if|cJJ{*0#sRBQ9Y5aBN|$|;g+m3?`xNP^hx2C$AQEuc`FCF zAnFF?xYXD5&0WoeMo_?=cyI}@h7;2V-$!IPa_`tDY)CEMi3zQ}wqG@g1Wdcr$r(qd z5hbizKK{imze{%PDbzYCqVm8K+EU|u+;-$Qfw8 zRoS@GY5mego-n|wpC*NoN|H_yWRyulqnHGh+D=GPP)@Y#QWaen^yB$BykbVf_`vdSl@ z_EAOC2IEXUY4>fJ3UnSIsPG1e7^so1C870KSr=dN{r1gY9s+zkEdqc5h09qI7zLy+ zQ0gNlXo|6y&0UwknZhTr{XjLMHf!YaqLo^h<)6W%x##vZJM8~Ay@sLV%BFIjjsj)C zM~^4YVUn1^rd2a|FqxfxcvlB=Zgscy-Ev6x`?}aBcc)RTdYf8oWi`2 zyQeyUvSUSFV~ys+$obP(r62rhKS2EiF&>d4{j8n;9N8xmCc7)CE7vNIpH1DBK%#G} zE2Y&gQ?KiXz(P2-`&6D;|GgLdt1bU+k*?$4)SNQiv+Ss>pzjH*nK&q9Qu6CmaMQGe zww7FL$uf1)GsGtvc#vFL-|rePzfaZC699QZXp~9c@0GG2$PXOl;28*2mGY&{yPl

    {o3jSl7c=al7Mh&c|LXW3E1 zcq6>edhTcDvB-$ZbUzIBCpPy}Uj{{EMsRXU2Eplt^Nqg1ESBnF_x%rV`M(-0`}_R- zZy1Hj@n?jB0LcSDyHG3URR4Vy%n*j$IDW?Aj~HRFEB9K2B84an&ehiZDl=$4YZl!t z=y3GEL3M@B=`g3+x`-R|vYVslSedtyTkhRly&rXJlfy5t%t2xtT>gX;OptDc$77r9Reo6KmxFTrJb zpz2s`0z@zCdAk0rSubqv-bCIbj=7V-FC@Jv_^m#*svR@I7-tFpBZeSSZqes}zQea_ z)6t;<_ZQO!HRx@@>R(%7zuFtBfj7YEM!W#OG6X;(-t*A&^hKRgdIqp!hE0#p#f1&A z!REO9<|%Sa62sh*`MweH#|!OO$P z{xrcHG5189qdHfPDqIK4B=@>p6ZPK@g&^ib_$C2jB2cU>t9#6gO|?3&P*C0`R;Vr{ z_FU*tP>N2noo;YdWM7VH?#SS$PtvnyLT_q#5APTzhUf9}8D#xBWWaA%nf5gg90KA( zcuLEyrlHrJw)7BNo=^U{_2+ zbi(RbM`!P69Bg(s>^O+lX;ycNXe&l*91bs-9FSU->AnBUm-^bz;1w5?y3! zp1k+7Aazpp9(!8y&K(J;w)l!*<_13|>aDvQ zQ56(EqLp+APnr-j)rZKrzqfCTmqViifjmT9ouhcFBl6OSFiD3ZVczz*)wmH1Q}1Fork z<-a^b!}49_vBnB`_te&7vM_vN&0F%aD6>yd+%y^bK=ME;zC5gITB1IJFq=Bm4(>wd!46R}36M>t9`gONbT%V`s*60G zCf>D^k;a{#rX|;?7oE8g73 z;;0adO+$D3!dGXNqK&*0EGvRD>J3+(ml=i;>_p5iLcj6!?41$|Rh3pFE?$U|x!@rie^ni}mOgYSD(-s_#}6HSS;i5fWBxO@-`6V2j{ zq@58>;WUWm8H7JZW&2EgLG+@L4)$to#HB6Fbllvh+1l&d?0jd#6bZ&YwMQpLrNh~0 zv51Z8Wx7>OiAfG$)vFZkba9XXc@XQJ$2iAJs_~XKv8mFV0!g738uS?KjgZq~VlX9^ z3%dy(S7N4m^uwCRL}2~!R>YC1a3i1x&|^$$M2-lo_TPwpUE?ViVW?hu%K*FKoj7v) zb{yWOl3CyfUDH|{J<>$5nJ^Rh56=Mv6oi@LrBiv25UhmO{sszJ8_TT%c&NpaJ?m1? zcaxU~bjKj=v>BsXgP9#!9-c{jjzxlF6=14?8;!xnyO*~Tv1hU6I@Pu7S>@8~ z>4L9>1+tt$${!j2&&N|ga+;m4V_b=Ohs_tOQg0>6Ju%8imeR%P9!V{V#|Dbwze9^^?7rZ^ZDn6Ae zT^^48RFnrhuMs4|*_e90ij3}Q^AT;^vuH5P6yjDyXA!_-VP+}#mo<+)3ZL^|h&SF# zKC_}fUk@};tz%4k6a54BoHwROS|lX0=Lub`YP({3p|GczA1nSkN%el1_3D{UFz?pT z`yHu~0r67av;A)liauf3Bg+8XRd3D9RTGK{@khKd`VkboWLLqDqSzA$@o12I8yR$g z_emdN3i1>g*cC_>rZ^J=agUv`@1Hj#yghCfI{y%PS0{jN$-p4QHrl$AO)wV7GPWvk zE!e9u&-#O@-F5Ov`PBl+C5Xk>Na^Vkq}ccGJN)wea$#q>(^S(Rw37q*2h}G26&-&` zS2?a3wuVT)4_y}R(hvHgQ3dafOWos79fsn$>3P#Xo_~^dvlcx$g9C>up+u(u5Fn0p`)E1a!^T7Q< z%$H=DFzA2)#(F+ZP;OqF8`+wii{$PF}xo>=C z>;5#$XZ667o4)IOSh}b}-I=r1l~^tc3kMloRUT^Y@nZQ5_qWP+4X5ljhNDfp(s=d4 zYU8xnYmSvIXZWdjM6X5<6{zm@p6Q;7o0I1G@ZR&HtXziGM>6*c?wApQ_NBYdRyAFu z>DwF*QEPN^I9W|iSEKd?=hWyiGT7>>*NLfvVpU)DqyeFXWZrPv8Tl%yE_aIt5pvzg zWaHRa8Uup~_Q__q)EgDoENt`)t`_dh&m+4+8pO$FLp1}DO`**PMm!;wuAh!QCPg*r z48IojqWCQHA$uw=j@TU@C7k6v%s92wL)GM*C}C%uR!m=egt`)3EyKnCVNI0A(N9T3N&cTC3y>E$>N z(--$P=}v=datHZ8|1*2|S7siwerU&Zh;b}J#_%4mCH~oQ9&uk1!G*i89nUczP{lI= zM5wR2ISVi6Z-+UBA!Lo}lbXb8H_Qe5ZUIVTYQ;x=pX5mDcd>OrMx!t{9$aZSCq7%_ zAtJx@iERdQC{VLGElo&l5hclBm+}{u{QGLnIYWkMyJTLUZ2Weqn;d^xx~@7!yk{Z+ zL#kUG=aTIu;#=rRtB+Ceq#*ZfKvrV(i5%8Ezd4ww7A>G}-rBa>U3OyUCHS(C@>k(U}Qslh$nvTd5nwWJ73O`g!u zU3vT5+{-%=TJ{XpUOBc^wCHap)%h1iX70eg@nq--yj^^4rGPcZX6p{u_A!{+R!=zmgQ(e+Rp-}F+GA+NM)7oIh)IUkEJ5S946x)l10g|w$74=y&p{LB=Qn= zEz94Y6F~4@brH3fXsdVWPS?ojEV;ayG)L!~jVoA`^Aq4-!;aI6rRWWc zrf$Av8Kmc?ENi-a$VgbguIPevfD^fXVk@@lt=aYI4Bqh-)Ii!|gp?0W`W`>u6eb@l z;uTL`5mRW9AkPw6t0E)X3ueYYCpa5d=D!`>t?M5v4IT2`d-3D$V+Lp5_sH{;Q*Wvk zT1@w%!Eh0qigQt3;pMoO%FWX&;0~jxo1w=jQj{tr**P=t+8b==%#r>cG6YXZJ$>=S zuT4AGmjH|9e4jz~?g6H~Pu(LVxUi7t!i8~uP1S@zfsu79)!v;o-S@VnCX!z7`*p9D z6pJeS4HDaITt>Q>LO^;&Ix1A0xKg-s|B^)|@@Z5Bl0Famx_sf-Bpuo2arDi`^%DM4 zNbc)iTRXSsWl@aoXb?}tRaDd%Dq`$pVYI>0W9ZZsePN*s(sSI?V%s)(Y zcl2tXZE7xq_L#=#W8}u!a^T+KEPx=XI{dC^$uhjPG13g9dB0=`g!u6&EVkyXR-O_M(|epbxlwBJ|c2O~nU3hc7KW88>*b|nSTys~~x z25ot#bHJ>DtJ)1TXQzD0Sy}HWrClq=N;UGRNp{ggjSpd;y3;R;yS*7@Pil}uMY_br zGCZF1-eq7~&PdS#6Gb%*O^}>Oepr9fhL|Q!(-T-sp)T)$259u;nsNmiuu?Z)UIHVt z{9=~PVOv0)0PpHDkoNHWrse{V7#B4$f!d}fVR%sR34m9u?fdbTB#6cNgi1Dd!^JYq z=RUD>XeO^ilZU(SFJX(ZhK`tGrFmm+c&PQa?qHs5E8F2~)$Nw5k;Roo@43~-!3V(|}0kEiSn8=3w$C8j2;=kShA)`1sTWB|nQhj{TU|YToAz)YV{v-py9@LOPsyg-|Dqs(#**MQZkb+}J~1yA}cH2()%6H0&jucrITe1Vbf5dEWv|^~jwlV@K0N|C_VL#uO^mQEKtbU+b)z8z^i4SKs~kPaUg1D%7km0aF@=*K ze9$!os`5uJQkoDjuo*_-%meMT_&jiMK1YzFf$Zzju+I*rj zt=9aKSU>J>K$DC5YHbunmFYl|F6k?&8d+9G1`UE2j7=OBaBm`S1?(z!l^fC3BtM&B zK9i1NKDhw6aL0wY1yT;l!m5=^yIk*0O+Lb#Vp-~h_c8P+p=l1pPye37z zq;ol?DAel`mL=#u>9xY{bzc5 zE*DTFIWpHmDtXH+Zw^wZd%wb(18IYZvUYPX5xgN6zsGu!CMvOS4U8D88j32lWZ4TQ zv7A_={d&VsYmEd*)@RS5x_QSLzK#Ly1q00?`dB%;3hVKiMV6^M=cyrgvLRNDrfI5a zc%kTe5AP0bY%s?3TJ6ZE=^YAlvq*3R_|Z;6ksm1C_$R(&*O$Vwjs~S0DHx%VU(<@N zU2s|KtIv-W=y|ys&Am*!QHBcDbn{AIL=h_shEr`^ioa)ZL6@ZJ$Nc$T?T;=5NM%kJ z^Y(8G(0(9CzVTU=haU>s-m*1QkYxEa814rFQ+=0Rc1-%J5I(BS4984Mk&ZG+@Na_PmG7eRzC6*nEP{~WH{i-Dw+O4X|K zvhs;3kaW`P^bZssmD^yVnOdi?Gk~U&v^LKlNK{o;k3Jn|lXv^c6f;pfmrxpnT}1+w z>P6rdn4lyNG9qW6%~H3d`=XL^T_5EElQfNW8j z??)h4c8_?G5i9Mh{Eg6`a4FnTQ=Y;h1FFdUS*;dn&O@Er)_M1zu2WBh895}jm}>xF z1E1nat_-Bow%_f#*U34eN?t0hn^l?s%A=gOjcBG9dO0czx`Z$QYw{elJR3C?lV7ziNWyzG>BcO-A>Qgz9;%?0^wPolPm^T3P1a_X0e#P&)Ee<54` zfo6r`3I0PX)_j@S+Wajj$5vC&1mDr5l!&o)l`m|&@VYb73^Z`L)*&xwTiY^H+h+k6 zX#J&_Wau8Zl`SZobAk;#0;_a}uFb^N3l}+ z7E0Gy4c5Mrp8370^enT*+~H$R1NqG5Z_Ui|pGtwkkrENXxd4K{^~m5=!GD3ab)JLz z5P5b#=LGcaj(^V*rQuwxT~fbqzpRF=0_U9i1;y`gpSi&9shH*3^`tl4iHc_pa#^}H zRCZ+oKDyO!U$W?_FCb2DphdoVSyFhv5%k2U`6IEfV`_RgWF*s9FNIb(vgi~Yy5t@z zT|rbG%Ah$z4^RwM>{c_|t9n~aTUzrfH%RM;DJ!4-2MUOA&;AVFC|#86%8l)Pu&$-m zxw`oeo}19r^Rqr=H;V@i&4Q`WHuI|^t7|aH^Vpu>l9!+4LpN*DGpefa6-wlWCMM(3i?JGAY+5yyAuVRa{^satY(cYdq{(ouzM3s?A0;3c`izJ2 zuqa)p?770)dH@JYuo_?IhN40OZXIuRk)jVj^=^}TlS2|9m4-dy*0!Y9yJ?zJ)k*Cd zil*=S?kX5hP+Dx4{72yyv3yD=w!#~%4V-LmkHJ0`6@o$mQ-JQwchW+wpGBL!o znJg5IDBbM$3?3#*chct0BFSiUd-=V$Xt}Tfk4EaeDWcgXW)OS_d{%V4?t3Rm30hD@n=QZy)mmq258+*Y}ampU%_Fx5kTuFP4 zrbc9Iiqidd`S8M#7|11;%sGaMbYDobf-F5XcmZTzlO=)|xPux;-E0T!t;_N^ElS^2PJ?Ve@x*gd0z#*m$3>?fB7 zylRU9*_0B1sM*SvwPD0&G-4Eu#ODseXu8EKc_zc77A^C`yNah?xhrX25*xitNQuVb zo=i7RG`UXu|A?KFCoKbYt9LsnwgG!4_<{9dI*DuF_-B|zF7KT-BY5bS%RUD?k08ml zysZN%_)lTEL8Z!|_9_{a>wY5njT_07q>ZTn_~%oqNG-{sePMMT2i+}v!m~{+^t@6= zOPh_|F;!xk*3X~LB6mcGy|HrGA0!E^u_Xzf$kf@_sbD7l@j_|UQL`d;7x%?$BiE>F z>*}xgeFAoIy`^y4*H=sg$*>;`5XGNjK@pSk%h24)S z5sOY5l^hf4k|+@=5es|XC0-weK3LL8`%IfV2SrU+?$90kGH=>_-b$f#TekFIUBF79 zLh#bCn6#@NWdqqZ7Qx{6c)a>-n)ZHiF!>^xH?1^nJsFZxev^3~Pq6XXSO04k^7lGF z_u#ls7cLk1F$_;E7CV%x2n2j1cL$`STNy~3Ml_1?(a%-p?z^+n1 zY~|+hzmE7(!#%cVAG3uIWn2Z(6{HtApmlwvHWNG_JceRZSMq{p1Dp`6`Xd&~&iEpY z59$y|S&l-E>!kksAC|*A1jh-%q_&B%A!2FGS3l|}*?zck?&K7mnHjvFet8$s}uH5{* z&JaQl_Zn4z!68GhoRZit?c8MwYYA11qa9wyN@cYUler)0L5+`bILJs z))T{|c4STr!Dr9l6+5jSfXV!g!tCJw%(n8t^iD=W<$?Wa7DKydsB58jh=F~J5lu-W zhxi|(42m|^-o`p&V?nD+h>!S|+sr~kkVk;mIP3f0b$V7imDap5pX>l4pB{3%2 zC7i0LJvQ08iRkbUfkBy?&DjIj6HlM~Ejs+(Z91aJ6vgxw@m`a?B5K1ir1?~0uY|O6 zgzZe2VR(@)q>}6Gx55O0bYF$EZ0p}<;dkW2IV0vYJ8yyHoj&G)5$2tsgy?4XUDhr@ zr_O|u#2QAleB2@c@@kvmMTkjo|4*;O8gsNS89>XdmG|Wa-MqW)pS(58!ibYz^^cca z0R*9uV(7bQ--=$U=c~nquT-!E%m47SRpprI;@}O?&etnu$9nH1YF@houUp)P7**sX zgW&1Pmn&0_7Nc8)rAh9z>McaUmEX3W-*b_;~VTQTOu`d}wc{8ud1zhZ&T6V5M|OLwfo9W`iswIT76tPdP}^~=3YH_3I+8VXJdc1CbOUX_TFwRulnaFF1;c1STcot z9hvrO`DQ&9dLG6+YhPtryqYq9u9&_00g*#U=k6c-n3xa!=ZC#3-(=8%Df6GOgvGL` zUM*+A!8Hf|PFa%S`6cEW`!#Y@C(VAo#?g=Sw{PP`%(XAO{`+Zok?$F5{rn13%AhqD zxx%1OT*s7H#yZoz*Nd3I?S9o0|F_@&^Bev3pZ_{hf0^Ykv;5^me|f=QUhtO}{N)Az fKfE9&gk^c3(<`OAuCf0IMQWe@ literal 0 HcmV?d00001 diff --git a/docs/img/structured-streaming.pptx b/docs/img/structured-streaming.pptx new file mode 100644 index 0000000000000000000000000000000000000000..c278323554da8ca3cfca0e3aa7eef3f6225997a6 GIT binary patch literal 1105315 zcmeFYQ&3iCblRl#%X&}B{lEmfR z)f8UZ$D4E#P*fBx0+BJY1m149X_9FTyqr!`sx6BZF`xSZ@LHulgLCulx`=4ZrAf_0;5AD#f2*IA8O6v^lbkono@Kd5(&7{EK% zFE1X}EP=y#$9UE(JV7cls9-0xEV>DCS)ipy?L4Z3!Oc>`K$MdY6t#=InizhF<23R* zAQh}|NqX_tC5iN?PiRZQMKsH=zngq7yr_~av?NtQB%FkEd&fWmwfAndjGO$mBR9Ej zNhlM7D|aM8OC*&sg$?&D#1ib+Xq0n94akao>36v?YR&JZ+>5RncrZTM*AiZ0)6iI^ z>_0>GJ6r;@mmyuX$ycpH2v#<^jC_#4Eetwi^vU*$5d+~5TtFoL;*XpC&8@G~%v9u| zY8&_y=vV7G_AcIWqck~%g6>s^!?F-Bb<@Mdx=?t|$hy${PciaOv3AaQ>4lI!hm8|N4k|ugXHiqTOwU4;n$NSECYt|RrLP|YuVl&_ zH=q+D(e2zGhvUdu@=I*-W;Ib~=_y;L2Xj^?&Cm*^sQM4!93JuCSXx|0EaN0D60Jd% z66B4BNOQ)gK`5Eonc^(@-xYC!)6iRAx)jdYOOF_j+wmuy^jr)c!pEIR!|v1O9+HI? z`{HC`DU6DAL(1=2v*!(V!vou09!&?6OWrRPx!nMHmp(8t8l~F#L{_4pfATHmbHoC`@N_gT$@v zsvQN#6ZSQ&rj%DS6U@cw=Gdh5&f(AviPo~uWTnhd?j9CY8;`Fc>LLpq`f^NH0oQrS z7Es_uT2ld6-N+T^iv|=$3crqJuvKwgs*6&!$BU16^zb`{JlL;OitRx#Jh$wbl-| z`JgPkcZotKvbX-4rfq$3Pwoqa_(WiNt_9?oCpf0D$s){{?5SPoY@K!ob`5O$Oh*#$0 z*ud{M8Nb7GbDqFE_l}u4$smUl~QBrY%&mUSY$kN(4lVz>RZKobMfFxrPd>3lt+bJg2595HjXG?u? z9g$FaTIq0GCQejiIx?ZF`1FXg5!qNWL^FWWj6@WiXzptNBGrSp5;Q)TvZ{9}kU?hJ zKB7fxiFqXcEk#9R`!<7NmQ6=RLr}ewGo%+Ep}Qc4JzYdDL^@Q@1MG zgw|)j65?H>d(ye#AXOxahb;p5Z+uVy)yYQ)3OYwLrxX6`lPjY*1}t~4^$#69u$S5+ zcLMO0duM_$bw?J=QN`RhhM6Zj>K$O(nQS0g`%!qxhy;c?T;wR{3O?XuuCdaJ~#+2HFk?I78Zp+a9tv=Y0zG0KMC zW6Yi!KR4$70u)v<5dMZ@cXoPhJIZBO!S#NXIy-gPiR5K)O^NVBGOlTE#CPt%5NusX zEi_HZ^0QN2iRr`P>%)yWbm?S0HXbf7+Q!c=r2Ixie~ ztT3njxq>@S7D0@QU~xV$sfm%BH)jMfJ}jvT^3%6A-%By#ycj8H3LF5b&Ou{-pm`N7 z0Y&)HhlJ5jP%(jw9}6Bp1h5}Zq5O|Q5?*2f6aEZ28Ew`XfCB2^E{;!;k$vTY+LSlQ z)hjh5WhtCV)UVsjZEgO+#|!B#9oKzgR&KY{Odg+!AA0L3oBPj4Wh4OmBKt1BtR@1) z{1Tz9yXtQh6S}rTY?s7`3R;V-3_k?Lhj|s&8n-Ip7&q(DVy}$LtcQ22JNW;Edti!- zc#A(4dTKxb)c=a{|00+FR{v6qOI>Zd1C9n?z1>gnzL^o5JF_#8gSOsA=x+?vCqF+}eslqTlxo5;NR83l$HsxnC!Cez`fS9{c6%qAqCJZo@9T zycbUu?pbmL=gOS2&`%q6Nr`1@VUCN^#*bw+Tw`9VFG1TR;}J2H^e^RlaABHRzf$h3 z6VRHw!et)H)Z~9fH(D&|*tE|ql`vH=n#f!gi@!@Y({xWEcU70TG08sty1Uhsvn$;x zdVAA@-ziEZt?1$Ug($1i>g;(9u0Dx_25(y0s^qhHDP|@w%sHpM#CCJmS;+O>@wKyX zGGDqLc{S$lz=rqTkZ;Y?y{^!{vYEHiUT&N0)An=>Qe`nC%Z9bG9g=iZh(7Ca!Nz9e zuoWHM59EMl|Ih~U5vbU*+3W8({@KgNfyQ@m9QsfiJ6+OH!C!hbxWIjaTaG`?qb>f8 z#p_*d*~Kiixgb9FT^^4q=Lk9(M)3kP9CuOfirJ|G2F_#u;-Y>@dtFtjW|Bj?qM4Hk z3|s9SCoQk4rXFb&LS~smo-6B1hwSxhZ`{pIt@LR&eZjz~eWg8t*K&u1zEhkx#j}Uv zNmHAYUS?$jPJQFuD`)+l{Y)YHFj4H7S$<7NGs=cxX0_`R~jMN_Z9O$`;$ z`f0$11B)zg@W3A^`=ga6feWq;6u@WUV{OsAXUZe$2>-%3h3cSX*3}HSGE2-eI*cUI z{96?%XkOw!Er#08C;bbfC~%l7>=BC-hGoMlS8^p4d#a2L5IPAiob9J zx=UbP%e%JfkI=le@6$RPUfBucL=YUX)d}#B!bw#W9(YIrSF;HM2!U>=?oav~;!VY$ z%SGRc=Yp(0R$z9XjFn5b9agJCURFTLN;Ri|H`}Kvlb`ZTn%HfgOGS8x7Ah=QAJvsm z79aUvkbBIzXUxypX4AHc0hyV;ToHGe8|J#kQSYAhqab`{Gy()nH!8+)@I*#uGAy`e zcg-sEa|*>n>8gxS*o^Iw*U1ofXbxI6D73n-R+!&f9-`&uNi9s(;WlL^qzZP)D44|C zmmbXJ@82)d5%df_mu?@aATAdCIL^Y30qrY&OG~?0%2EvP!ps69Gh`~5>DoyiYug_)!YuV`G+&5rgCb|)CBL;k4T7t5r?O_z z-Q3=28V6elrIXfbZAd~DXE9Q%7dFkpGi@7E%`KPbv8fZ|Ub_wbkAu&WR9XM<`>j?1 zRqnt5KFiPh_Ignb;AY>-Eg53dxKyq4{616<5k^8<;&QQH;A>OXhXye`&fCG^(9k*s z1b_$B$62Dj0{KO5s`{KkcFDoKZp|%cK;CfBF8V~xdq++ z8FYWwGL0U{ZwSH=7m+Wun!5!S8yiQ$pA;f&({2~0d`{0k^UJ@j8tk~y;B6q&drK|g zx`7a>SJH9$5NLtF-T6oo#u*8`lwznq8F%&bPeYqLX-D(kXyIU>5t zB=~%*eQETgtln)O*gi;ib1u`9BPZBmw_u&Hn)xBOlH=7FpF=)gK)hJ67b>2UnZ`p^ za$SFoBIQ7pnkU5yRn#VA{o>VhwWXRM-gtA=L<0@!qK4-7kHZzHL<&_kIrMhGwVqZJ z{0v`vCdkfgy)29h`Gd<@^!Sg1Gw2c{i6L400~?nh*#*n#dsTq)`83LUP!`qz9r{b> zFaJ)aBAcJu&@Pxaw62$sSG2%8qursJ#n;F4g*J|k$apc-u|n(hkVG2DVK!cD0MmK< zEfJW|AbiTwj|ue^x4{Kg(si6i(!mOE!UkRVzBtbIkX=T=jH3F)H&UQvO*+}r$R)-R zUALFJU^$728dV3upOx_yEtyo}dzQ%MiF&33e{kSwthM~Em+*0f++U1G|^a1BckFJ2^ z@YJ=+?`(e|Uny+dJn*dmP$MltK7VvR@3-IO9u8eC@-+QI#9f(gUHpo&>KOc0S-P}6 zH0$l;Db4wE@mD7Lb1U&I?qluB?qSPnDe5gi`?rk_n`^YEbBX7{a81Szc`~fVt$trF z2VHMgn7?#5K+xn4f<5Fu81#z3D#7mk0jLfTE-2SK@fn2^jx}d0Qyn72d|=r>1pzGr zyFM|2H9`@npjNM#e>4C#;0}b$a-bau49RV@e;Xh+ARAD3yjxBb& zIONHq@5=qs7==5+d-s>hi|-f(kk4&Ef};Ll*;~VmBNG%Ei01bp3rV3rf{ca;sAB5w z3CIZaD`(%#&wK5IjZx^E0)A%2MFIVf5&mf;C<-t^Nr(ZCQS@$y8lxcCf#136_X#sW zF`N+YA`$fx>>>ft32q8({xr<7Ky-wdpz;dqKA{7f2rNM+iT~S~UNHg*=mfG@-i@|8b=+0TF>spLq9Q!=X(*;-NNMVCunwf3&|H2$_{Z0OYHfz-BKyP(Ro| zH~h~JV2JxM?1Nc@tU|nsZR-?Vze=XRun9i@P=(^-VOh)T4J|ROj(@otJ4GZ>^o5e= zD#Jy&5!ZG)_TnHrzDc?##>NXt}CS?x5SOjv~(d^u08!F5&mUgtdtwG{c# z1bSKSv0KZL&4o=R>nGOe&7f}`_6M1lJofzYZE+fzuLMvqZ3$yvTIfU?+^528zoC{r zzdqQ5zeVq3P8yjxc}aB+4_*l+Q5Gk!Y>+}7s*DldJ%FZ+Q$Ge1k#Qh?%Gu%$}>K4dmi)H{q@nBvuv0OC+O)nJ)++waMwXiQjI4+e&1-Tbo zo@jZ1VV^2Z_9{FQT8%c?$R|ro!eWl8Hev@Rt5@$;t(a_g+XI(niPfUkOd`A#w<_Kb z#3#|Rb)g?^_j7IuWOT5zvl2I_qHWt~uHMAs$U=i>A@-#;(s=g|Mxcp>hsaDMGSUVU zEN6kS!dSZNEeKtLH3M$V2~4HaV>le&lPv_kYcP~naS{dsl>jb$q@y_pSqSP`XuU$j z`(w9J(Lh=DI4w5>Ya$>;n;Getf3+Ho?|W<%u-v6}Hou}fg74~V;W)5sod zYt+g4>8D+^mFRA5j4eiI+3k#Mt0OaITYpS8l{%o6Ur8-jW;uqSn%9dXHhv>4(v|5d zbdwmUE|Ly>wxLEV#XsP1FrCaCQG)ki76dL5lN-t@Q7r3f~(a#d-I-%%7l&UNhh9fA|qsF zOyY0Kn=GQ}IaSy9pV8U}yO6J!+Ls+Ddc3h`EO!frm3 zr6UfEsyz+1+XrI;Y+r6qFB-g&c+~VP@B_~l9B(##9eam{eH$ZB;4$3zs;Po}zXv9S zQaWq2KZ^831uwX^L0TmbA?M34yc_iCE^0Mj#m|bpA*<Pe=qb<;#nMvWBLYR{cJAJdaToZROppRj+l6 z&r8zJ|A1wTpj@b)e`XQ$e_M4X*q5+3=hrba&TMt^z*7mZD31o$FK&0ryRWcC{viJN zREv44ycBdv_buCQujwYAOPxF62joRhP5OlsAIa+e7P_Wmg#SDez0_etn}kk9xxDwJp4?W^CZlUrQ5$Z#=SnA zMYd195l=gcIx3#Q9!E`BWQuVW*ndeWWtD2*bHon_l7Nr7GEofQVd*MYJndT*6)?S4&NZ-s>fU4^!Y~>w zXN`g&lhM|RgeC3|*B`|cn`UqfjaLSFhV-5locXtVKO6AVki+v8kuZh-4P2ZU7Q^8t zGWRZmc!MaqZp{x<5DwYzo!^#SjUcK*0FZa*EfDp(c=IO3Q80=q8bje2;ZGca^O$9D z7*0eE4-e?QDmXJHERMqIaoT(GI1np>lKjWO49yznPI3xLx4hk>R5YH~y>}&t{(UNB zL*hJZd^D7~G(&_Yr6`usZ(uV1su@)cS&5_A%E0(0ZKpN(9 zL57!AGC_sC%fkMx(m2}Vn+j{9+q=pmhqSoLDc-KKb(raZjL@4e-$(Cd&mAXZiA>K% z>6G5gBGVnKRI{EUXbzwC^XjAIxn%hb8SxTTm1z=*mNSqg%UD$WyoNL*6`~Y3na>8= zV$ymV?ZH$!-5~PM*Q!C?kdFN*M?KK%(ML6hKY}4G3~1?U>!mp@H0Ex%ifuE+Ak3KL zUSa%@us0Y$eyMo)2fFxcg2Nhp>fve9`(x4vPPT4Dk0{zk1-(JY5zW5z+h=;1y@ss!Y2dIPbp zDYcpGTOZlIj;z9dUt058azQ)~$F!+<<(!IOG9tF(^0+*m*WOXweu^qwbu4~73WNYQ zB(R{my~C7x6-ABa&m@m#X>!)!m^~*oCLVc;m_unTDQF~pijKVVZIrFdjceJd5S6E+ zIw~#KGiNoEgVmG;Z<9Z564$g{^Wm_f?ThKj9`2^ExE@c{R8t~zx4Kh)c4j@M<>Q*o zQE!{beMVuK_Z8~~Bcn$$lTc;sRno9e%aS)H3aL)4`Vp5)3MT?YJAzw+KHSpWe$eO9 z?>fS|z>d%gsS8MCv(^VL3T_HJ=g+xE3h?*~R*}#7?cPA{i@=?G9#7P=Kx<{mos3pa zmNv6BGc00;OX)s5v$+2BWmB=g@5t_p0@;_sG7poNjnMSV_85~Tg$6}@9yz0w6LdZ#ZH1aj*y z<5W-ymJO#%-5d(KT7{+|Mb;_iR=cNP>w!gsO06rt=TZntG~VkEMrK*k8%t7BQ2?wG zJm!*bQv3C+%<)XjZsSeX#wPV&6$9S1ZmHtog|@8{z=1X-m(+wF8-i*O+1muFjwi%H4ypXM|ff59p15MQY$y_CFd@K%-! zjSDhps2=0unVq>yjVQMu+xh7twZ>F!FgFttARu)mtSa`^%^(=Ylsf-F3-P;j#tfU| z_U2s#KkiXX^xu!uH{LGQXL*w9^o&&4gu%!a;H8|#h8eAAA3u;qIgChJFS=irtMtys zgh}raY7AuENh1ZVHUQMbYZ)$ayk^Us$luMfpI;Fa>nX(0dr@!Qob$8kuhWBWWZlen z*}wf1$iA@pLZ3GW{nYHN!au8xiW zMZxJEpZw^+=Lg4I*UJ&SdfSLh_6=xh%NQw&&*x!HsW}#mxLl`8;MHG_$*TCat>C*# zj2ISmfUe*Z7O`)ZuTgyZ*mF**(5t=VQr#(k&#ZX2lda`+*(l%maHm)fsN86hPuyP) zamjDrqLA58ZEZd;x>tK3B)OIREPP_97*@C?EOUiuP(yp1B~44TzjIZ9cDf{a=u z4ox&kc?JkgWO#&v2=of4abw9=mQD9yI`9bzM-PRxUs;IK^p|4ETEr8ZjPMCm<<8jI zSx#!rNyB~GTBDRXh$I)}Ss%3hxDU#22Ivl53!e(_Y+ms7fu_h#aIx?umK&(3F?!Qb6TWffrmD&qYM93f028_}ff zlon^S8Yr2vUl+d~$Yxc=Z7?Dd|4UFV!=-*;E1d}cwpSi}0^T`C0vUa%0@$WAs!JUl zgjh#t4s6f;&4UQzfJ*=`IE$bnv$&BFqI_09rOQwRWB!rg5m5Vy^xdD^{0&x4$4luO znBRUX`|riH(0ANOHK#t8zVB#Z84iQqUDM;#qU(|7n(BsVN2$lnl54JGKO~!hZcFbz zWOMF-Eld!Q;rzzog6ak+V!>OT@<#Tli)F`8<$Ad23N8aKQ`*9FfNW>C zZmjFn1bxLOxyjctWxl1T8M4v1#gC9`iE%Zu`l6^gaam5wj-Xn1Fbg$M=8E1syNp}{ zL9lHNOTgP@)5oue*AS-9bD}HK^FEXt9(MEW8mieh>a^MFPlr%-_<|IEih?YWS4?$+ zok2>9W`l1y(v^F67?Uk&YXL+yUMtYL@-Sn?8tdH-n(gt`2TJ;W*y5JU-v!NKP@BI4|8WHW?$G+UTIErGvO$c$O2g*QG( z)x5Lbm9Y~6L=n|(+`Ka+c$43HhDt?!SrFi#ps5{(&YfvXFAIK&fjygZJ=PTLciAy2 zpQx5nFn2c|JDv!}pwYBS2nN#@;+7x{g$RFtmt9i0C6yP`!ZsD)u}%-v=!-ZN=@a-y zx$SiRMV5LFmDuUJ;a?`hb_eH!!*`uDcHX$IhWj_hPC&#D2_E?p}MM( z-U1(=b%}BvWw2|8T90SYdOIvDJSbFF8IUt7ui%JJmx>(zTo={10D~`r&s+sG*Y?Md zox6*eTzz|?h55Oo#|A`5wg)ha*RsN~ZbxcmrEgFH>07;Def&ZMA%P%ZQat`mD z2Ef}Rcm|Z4tE=enG=U&0%>DWL|s#D&G9@dNyxH6P5s%aV$LMsoBl^&@EeQe_qSL*}|x1=x)|G3#d-Y2K0&l9z=jhkNjozvs{h5&r8z_F;3>83-6 zhPNwQPnfvw&Eb(O(-VQZFgY5$uTKa3-*?h3zfQ`@cj(R^UJV&g3%Ky%E{w-b9scH5 zX%OZ{OY=pBbRH%ej33-IENb$>c=O{-?^j{4+&uVf(~lg`7^s^t<<<+O32{sq5%zTE zn5uSBRIisdMKo%m74BSn&x@}$9J0wT+39XM469VQzecOKF2SzEetWm={YsJYgzY5C zk!*kooeCn$?54qUHZwv|1|jFhvYBQfy1^Ai@Jc{BILn2nN`yx7(>Q4(L{BAOsw1(zu(`hrBultJ)AA>P5{QB@NR$J_ESeNa>`)*QadHzSAgxqzJ70F;SdZ9mHBOgE zm~D=k3O~vK3*@3DoL%LeQZ!ML=Fc(6jLnO|0XHHosS?x-uuiSqnRHit)@5-yhG`;SiXp8MqqMRX*SFpDdP zi2fqr3!wnyKrgsX++A&}_u~gVIub1EzEy9qdNj?AGHdCl#t^iQ~@nekRNh^rAkD z;Q&g3l zm{LEWI2B6w#8DOC`|n#=`jE4t2dlwSU>Ye;VkF2sit1PCyG5ZH@1 z4B(k%FbY*3iWs$s1aTQ`fO0sya(JQ=a9!%8UEIE1L!a^Wbs zfYqExP-vA0sS%B(RM7ID%8`<7NREgtHnp0yI$sOYuE> z1g{)!^|=#yb$ettm>uFOhBI)3kEmi;W>rhzgWQ?7asBH0X7V;Zq)g5D?LnTe<;BM! zVuueacn6kVCYE*uS=vcncPzjdpfw8?DJ-IWWN4q_2f!i>8bf~rka{Q9Ut>KS9~UhS zjOOGq`%BD>Ub+{8+SrXHx7UUo*&_ttYcG!Jjw#@;eV(0uanyyS1$GO`+b}}{1R^$Z9SUR^qXZ7ig!<5*NRS`ptgV0yivIjWxudgSRY{Tf<_i{%vvwt&ED77GtvjA;zPpj6#vfX%yE z6&9$EIn5Ti6ugxgH|b`)Y1ti+W7`-ou7hX0KtK+pX4?oFj@&b(5Y)jALyloMLQeh? z6qf15!(j@zAU2YT23nxFUqNZZH6j`fi=DWF$j3;+o4T!`)(+SZ#gi zl$6ws8snS+=bF)ru)N-v)^1wITVyoI4uYjCK~&f}KY_LN_uedHB1K6TBr&tk?(@k7 zb5oBczqJ9?P2U>u;vu}v$Le9_&0#1q?Hzq`|5ZNHm^k5@hF=YK8*|ujzEzgSM+owf z`3J{fbx?Y9^0)fgGMMWwri-`(Tw{Z2Nnr*TptlvR@Wyz1Ir8*)yX{qD#UC^WE;&}n zK_t1U_R167T~kXk0S#C0CmFvJK<+JPyj!uI(Ht?G0)Fv#F%V>bblfFf#&2akNOvH- zy1>br$R?~z{)CB98N5&gWem;8T(k>bVv)E9f4v#t&}SG?;j4p5Ykf5V33v{IlGOGg z;O#!$p9fQa@bT)mu~Aoyj16EYA)5#Vi_B7VVXrzQhJ9dcym6u1+MBhS6_d3%zi6G3 zw_y0!-Q^Zn!!Gu3$mXa*P>9;9E>r)*8Ip(N>qVO*urlV{MHq)gM#$T(qz6n7FhqRz zdos*ic;BPM)u$UBbCx6zYPX)wK>Z*~x+7G%KL2*yt>CqDBl$rF&r`Amxn|y_<|)uF zI=o)ouI%05#fy4HDAT`gZ0pJA*UDD|Bn|Aa_v>-ndy{-+{B4jiR@}4Grw?}isvXb^ z>+uprTVV@6qiw)=XUH3t?B^>$GOiva5@gV_et__e9!E1Z`@!5l zeakh_!PM!x3jVyL2bB*$>*x@|-1*Ph3z%Bw&YRI&S9ufr&{iY&6}E@d3j{(T2AF>P zusxgSImgfy%x!e+U&J?l%&6ISPQy3OaF!0kwNCyyyzphMitPwMNOa3s?TTFNQYT_( zzdVwzOLrSC%B=w}nt`p}fvkiinpW2)1AaQ-%Oio!PF$@bIw>D>JCfeyfs~u((PjorpCs1d0 zibO7)$aprhHNW=hexZa~ma0}uD(Zj0+*5)m&R!}bD!Qc1Y; zk;k%#2x3Ip4o)N2Vs6Xp5`N{f_SSK^T4B~VqPiY_Q_W~Q$N{lWWx}cPwyjBg(1V6Uf0E$-yXoLkSJP>O1LW~pj&)u^F-yt|cS;-JPP&Jx3F?~{W`FU~T_jORZ zCYnAJQTgRM=AycKhNQUj*0SGhF*M7*e*W>w75 z^k!xF*iL?CQ@2`>^7Bpu|4)^v{WAkTg_|}X*>HO!3+fq*sK3KnaVh2@l zk5;ozCSpjw%$|BmuSsW;k<&Bh_H9|-Y=qa$9UJqVYkdW3pA=ZB7XCc#F=)k}u;I#6 zok)f6j~NtUZVl(tu5WQr#1iK+=((YL(3J+$!N(!yDx{{zcx@JU`|2-z+!~UNpZh4u z;JP@jAyzoG+u=T0y}s60Esk-tbip5dkEdhGGDA*|FgRfpk1AE@4r@~#Fy^kEtfP5F zl2=xk#24Q&OfmW7=Ox21?TwB%G*cv)6SecNdA~~|W_}>4*Z4GaR+q@_z@2Y&sYxGK z{Eee=%URT^c<5AclTD^sDPjjn9XWSONxN?ylbAki6go!t)NCb^u2dpkvsa$m(49+J zapJ%tRdg9u&cu2j6YVbJk$n)dwj}rhUXo`!nmU~-Geci;>Ivts^n|~s$fv2WRxr*$ z@8`)tQR);)u4`0KbE_!mPCN5!5KD`GP|GKY6R6Fd$BF_=%qB=a((S#w%iAs)~nd7br9~*?uz6+zT zd2bnqTVKbO;d#J{0Vm_pqr9niLE1?@UnYrxcjAcWFM94j-i59zb`1_AV+lIBr;e&= zrjd@x5~NU#(I)D}g0-xFHbDOhO-rwzwPe>K-={GjC-y?T0Vs>yM#Q}UPx3N1r#VD* z;WGVlwf=}|3Esha;fG``&uHO7SS9 z3DVKWG@1@>l*ED!IgD9X(c08BDawYC{a!Jf){%Jsc9HF*p6R)E`&hwnk>tm42KEkV7j|`G#_$Z*hyPnmBOpRo zBKn!%Vt4eEA|JcC z9MtYPJ08a~is=XO@Y>>D0Sl}vB^?KUH{%B@floMI)6QXU*KT|Y?)WRT$!kc*6}1iU z8vbo3V`P2l5q7D~XWT1(*c8TsrrEccND;nXtJQkzSvSI4mTW;pw-ISt#4>o*r5Lcx z&b**XYV7Y+*c4LTr}9S6wPXeSc20vmIO2Ahigi$N#9>-#{wsw;I53SkIlb)$`U{eE z6mbRp`%_3raR45`Edqd;K{HT~FvKlFiew7a&as8U!0$*#`-8bQMO+w@AMNc@EhJ2D z@XBB;P+n3b0d`M_HOqMuUklCe=Vn+zTI`50%$QN#7M!;=jhNBW(Ibep+eP(_7(Q`4 z$f{b`!mCrqnWs_B^_XC$@f92KP|7=&!|mX!HePSWRjlq+!YQ~!qcF)jT}xojoJrr0 zBu?z1{a^uam1bZ97_p6)C@23pFtH7IVEwx!z|ZZHzGa#i7O+gs05?3eOr{Ku!a*9( z56j4Cuo`h)!&C)B`3yU~@Mn*-x6oCDt0428R{Mi@f&@Y0*8r@5M!Y*!s@ zW&2{+2z6I150>4{R_<_!JDrNulddEWwUn)!)xjU#!dh5o=cT;}4`o}$bBMVfPZRz5 z6`raOSJ|K;aJ1phMCuX zZY~A-Qsmw1jiM~Gc1Pmu}YN#R7mIVI1rES zXZ1)_S6o?W)$s7;D~ROINxNw`a14YvYKy60Gi*W$RAZ;!$V0zBoHH($Q`z9&X;g|$ zu`^998?1G3ewW7|0<{!SNy?GafvZBMz3Z2J#myXpSE=(Q8{Cw3XZ`FYpo$M(=o(_< zRIK;WuN&2aM6qX`tvp>G!Ew~s}k?pI-xC|@&~@xE@ae5WXu;k#k2y%xUAfXD1O zo6qFCNg%jw2gH8vSPA+H& z_3*Vg;$cUuf<}r!%R9a;fj~`03}uyIw+ym9pgg{qMS?~1ajs4w1y z#{HwpJ-7@qbc;F!Bv6YXQuE*P-*RohzT^bqK1bj_#|asC+nO9v-*1cz-Yvz;Ev9=0 zk_j}rIx<=%PrJd)6}3zR=FJ75Rj1s&N{MmxPcaB~c}Inq4KxD8f!@{d2cQ*+A`pN! z1O;m9wGFGoCZPf!0Rk~66oCG(H}K_#LcxR35o#fWI1K3ad4GRqUlfMj!p@-fKdL7Q ze3P&j<|LOBd)XV<%Js}_+Rm_GXXTcBWxR>xe`N1jG7GPDF1G)L>c{|<{%oe{OcaoP zd?5e8Q*r((nX)tGT)O0YTGrq}wDFd)LHfMTO*>N~_CBzpJ-()gf9!aClglcIckJfJ zRw~RHb=-eFuuO>ED9qa#x7fz&b^Ib}42;ENeoGz$t#D${MM?t@)VWi)jkF5;Tn*Ux zv%dbIZ3lTwkygk*k5jx1rafH*vkXBqAE-)PCjpIS0oW113N#58*`yzjra`P9?avBS zk7&OvWDSBs5*`ikk98ShP;A?s4r)esOf(-EIBO)V7+8*@zk3Yx?rDs|8?CM8UEv>H zZurSM{a%(h6BN8+LC-~FR#c#4%nCmT?2iN&lFYgwP&F0dB)AuPkdts4@)GRwN1EH) z2;eZCC?{4gWArmwQt+6AGo(gb=`RngJ~Dv$fTk=%Dqg;}tZtCQX;_hG-0ug!K9@-{ z`kU;?tVbaSG5`Vuxd4{4>$?yR`Vd?Lm$Ua)`5zV6ZNsLd_TMY%vupf<5+wL3ehfIE zPAF_0eF-cY#K6+?0)jxUz^=enAXSJO31B~FEWm%-;nDt{AoV{LKt~@KW}O%T+s|0k ziS=T6q_YO<7)85JTrBir1+0#$^+1&c%pcTa(&%r&w| z%kGZ`aOp~Go-I*Q-%ue^;kncVQ{-d`#-B*4p!VN63rXpgTL>XlC7`!MCKoeCt4a-v z?vF1wsw|5;`ISQ#+lZ4|Ig4p>eYn|JZotH$AcrEZU=kpf&^g^Xf=LKrLZKiS2LM=k z=#Ofg_^B+s@==z=aK%5EgRe$S^3Zv+95*2GwFClYW{@7fU6)jsW#%1X<}>GhDGeU; zRc-;lKifI#yt7%@=~ioNiSIGlLusl4lVfXdjwWKUWpsuX+?Q>asC|(ZdRn%)ZcJv- zR!c=6K4eC?-s)ADPh6B^Mw6Jpmv1i4%G2G89$93dr$nY=m*r%VXWpAWnl3{5vL3n_ z=3WhVZ&PoFHg$VbmegbGhw=njzX>?dE|8>%G?=NVo|t4xO6QKT}Rswq6R8El6p@E3HCnPD56}2UxI? zNkyBzFei(S>q?e~e{e|QFeX%GH*`a`->`Qno-?K82&5nfi=mzJ?+E_C*n7tyTbC|d zxQt!4ZQHhO+qSiVnwXy$+doEWX?Hqj5$Xx zD<*;sJh_Q05Og%-IiD^}L`)BR_rYtrDy)uXV~`FREX#aG!Mj{7!db6JEa< zh2K?JLOpnoyB7ns`5-LWt|GRmc13hcEtodfZ^9|Q*x*F+mvO%JpzPz`r%6}QN906V zg3gCJQ-P)h%BeYTYLVOqosBF&iO&L&?!>Pv5r&-6{Y7``__}l7I6VAP)e{HUpTVrS zHNW;G(P*nv$g6R0A>Hcg8h|7@^jzj&0gqJe1|yBEyFDLRAC{O#rdpy*WGO7W11wxU z=*exl(G%>G+08fg>ab4>Sm)IIB4}bV@N@0_%K7?T?cr+&{Xja#G@0@I?z9CXby|1$ zmTPfuARnSnWo&it$9Y>P`;K&YV?*k6xHcaBX1cI(#t}}eE2o~I1!~xZRI^A;l7ci? zR)_5zLz$#KvS5EAuEUb}3X?MsE()F6g&+!cN67%B>%ycb%AMddm!+PfvX{j;;=;%R z<^ifuqKX$U|17#lH@I`vA`iyIrE@bU6h&b5B!EPRY`j>zHJkH)N%iLQD`sKsD|m@= zmbjqeoT>0AE}T2^W`U$Srl5u^TGddn%sdE#NUBjol+vI&kaujFOl~0D{N4GAshLG# z6)ad71G6zB5Ha3&<5S8fIe4tL2-~SH_=eaONLXlCqWNb4_oj)>E6>&blHtW{5jGqm zpG$eV>tj>&%95-1DZDD9^(&dY<K%qB19Xc*ToOq4lgTOVrL_=41BqJqDLS>THlSd|_)|>5LvG{zp zup=f(m&Q$Es7Bn8%CuA5dPcCtR$WFJ@N&uuY~hRDir`}v@97q0(HCt53!cediv8ny zfhp=sEhPf5k5%%Kk<}qhcxm0Z=ht`1??2!j#uqr>%D#=+nw0;s?Dn^#%(?cNZNUiQ zr(fY6?8p~SF&#gWjVTJ(O=gnmbaZ?g_z6HWM+DYF?0WU%rku+>%*qf`+vNF>Xk334 zoywwKQbo9>@9T?i7Dr1M^}Kdw^bD8V0Wsm&zB09(m>%!+u=PPA3L3+-V_ar5ZoK3D z-J9RnS;=1EgdwgS7Loam?r?q(&D!eZbE@ji;YZuMM#V*yGGMpdi?IC7euywflR!UxpkLRA2oWg`5{F*qTXUqY{^i=TTv~J{Qf=AoN*Qyem#`Y5no;oCH&IHP8%RSfCyyUEL=eF`p zCW z6r1$MIL|RVzT_lMjiF$7+%LE#le#H@2JCtTjr-_pY4 z8|vE?aUMkSz69X1X+%mC2>0Z)Z>8Mh9ZXyi89p6N%*pT1T@|~a^@eMg_0iEocIyXb z``vkeBHRf@=+gmk+fk6`GwXx(CiZR@-(CQrLs;cmtzgGK9>pGA5l(s`%*mfH{K^Kj z0G?xwMLj2!`O4+NvjBt*{A;8+ZpbwbwG-MRJ7UJa(MV)X@zyX&)Cn@T-a*;Rj4acU z=AHM>%0C+xMOwXSHNr0Pogv1T4ku11wy01?sG&qHO4|kG-hxaohl7og2j{~>_67rX zs$~BUhky~MNdp6snK}0p9j&R8G)3PD9SRe!Yj@uu=Qta+Cu5>xGD(Slfd@q zZr=FJoWo;}bD|)f=!Tqe>DANP*m&=Bz9QQqrH^;@et2PR-ovo8rX2=K@27__gS~Un zYw+{Ga#s4RU3eOE>62ukj39vg@nU(A znQ@ss6oc?E=f~gs6)}D}XqHQh-(5pj{Mdi3F#lSP+=If|(__4tH?-{^2T0^cCkkfV z3Bw7V!LSl3W?1+rbYHih=29aw<~RT3n@<IWGkqfx6%=KvH9}INrvU$Sz?e?NE?dAxR3n3B^W6Ew}L}l-9Bp zrGqs?xKBRgV)0aQ-AC4H$Dm}^`lcs|*b$2JQ=ev=RE;f)ZOkP9TDT`Dr=looCtY~umdDJ*TPldsF%)mI+r zI1NY5FRVa`kTMNW2waOofK*DO%4ohU5At+n<@F?zA1W%BEnJs1<(|UFX38omPkLqL ziYxUHU!)5eitRe3EhH{o@=s4fS?>^~`~HN%g1{~+Z`|NVGoyuR;}uF2b)j0=JIeGSm>f5|`5xz64rLEos!cWef8og6XrN-2Lg5Ak z)!brXkW8VV72Qc3O^G%2p{(dRXppLO;HC0)$tm+TO$#&SvYd9wz9c3|KWl;qJJ}3%JPF8*_~IBu(pw_BhvX8$*l$%gdm}8rM1}Sd z&Aglm7s;D?yA6+MNY1{%`X^Epico$l*j`1?Vn)~rtX+4 z#mL3-tadM#^bXezVuH=W<8=P%DZ>hJHrtb6|UGm$p%uyb+#56*f2 zGQ{?;=0@MvPT!3M-~M?z*Eszbq*Z~T%q|FheIoaO(Fbc_>%<57C+iurKLhuDK0dV@ z{A3E;6%ElkgEdw0M>6!Yu2?pzhZj*n%}h3mZ>33;&>^Zm9LfF+wgNS4USrhlW7IEb`yOVg;YBQ!}f8L z-oA(h?agX%5}8T<8eC9X!asELYgQgd1T42%INy8({znLJ@&ETMu>bEYd@qmxVDt4q zb}g{}?^^g?3mpGD3*WQwAKXR%$L@&#;4=C@b{7842JpX6revOMn?n7@3d7$|n>hXlyznnW|F$Cr7B?+_<0FIrWw$9X^W%t6N=I-G)*v0dln%RX|fBO z+p`jGwdTzp0kvp$Xcfs7&#NHqx(NPSQyDs{N?}rwYMrIn96icPVUm$rDNR9TRBt1> z%1Tl4nuaz9%Ngz?}FEE@tDNgynjLCchX_rsT;o5m=0N+ z&8}G+<>6@xU4t^R-sjDhp3(w-baeO-u&Ub=@U&PwRI${0d4UaC+2)XJck2EtafVQn`{UbLnCQKAS4eqK5VJ0*VdJ zDtP7*(KsJd>FGmT%%zI)C&DITJ}r~tA)db>09q1|U97S?fK}uaAdJO|!gtqUn1w?K zh-XX4M*T?DY}YScx!sm9XfMJJb09a6!&c+M8Bq|r0+r9Q2Ee$Y0yoVoQoq1;V1?wM zFj-hEUSGrk`p_IO?Q=3-W+Z8R5rgv(gL*t!n)}&u2O%q==G#tr5u-8`1zv?Z6-z%U zsToMEDTgU@KnPoU4xp`a51{w_%7L4c-LMm;X2f!COkPfuZcSWaKDSTduVlXDgoeYI z#jC>ifKU48IDS6FldnvEha7&T!s~?dI@F$fdAh{Ab8O2RY%`97lvh#H&S$88XZXV3 zyZK5w<-x^ z+55`q*NjFoB>d8=F#N)&Q!yzMrW(hiBEE<1V%%;c%*>zgz`%HnoRCn_zjjsGzPbUL z?u($cWxf}%?7n;Yk`p-ljSsN{(LQg?X&*@ZW8>ZuzU>dZtU+vHfd%^-SQT@&lFepa z-RsNzc8~Yx{WON*BCAh8fm{$)Sv#(Jm?T$xF%i14#A&fFvs1wz^Ap`k{qw0k`tW+@ zwz2-Rm}2?c0tedKs!m_3*YwGlY!w@qBq-8VMV*7n%Q3!~1~F>(3Tx04xc2lafun=4 z5Qd;xa`PBDE3Iu!tEg+I{T|ye_rPZxqrk%EV{OxrR$7G)^i>&#?kAF~!+)$kAXdx~ zS%G|?&=LN54EP&yOsHRd#{l>*Ui~Y4^Bk101-}AWgw-2Xe3_q|J^-TmMtO;)agA1X zBU=yATNfWvzs?h9bC8KJsurpsZGV-HXOL?Cj$*Mn#Y~wXB~AP zyQW9P%_JygQ2p*Q!;i_5Zx?^2`MSPu;Xpw%9#fV4BO z>Y-^Z4V@EmI7a>HJ=aWiWH=qM5uVtz`6oi?{>4z?$a(BeMZ{_m6Cn|y=w{qxb!1D^ zDYQvczHAcx;w^TedDfa_(yySfmWGIY$@9l*NxB!W*)Np@Nx2CN*i%#pMT73KBT@Mu zU7jBm=0}rF^pYuORt8C+=zkJzTHa>D(hRy-ah?KRb+$Z$^pf9ate zorr39aHYXDyxg85ta^KVhBmI#?Of9QsMNIIp6lbu%1~sm%)_a`t+A$c|5VUqS@!n# zm9iuIE^%(waIIF;FHf-IOP>NbQjjC&mF!O@4DVuoOn8$O&<*V$Ia}Ey9;n0PJJ?vW z6}3#bF@!p8Xu z1_Y4(Em=!AieNo~xibg0p$Ceo^)5Xwla^y%!fw|{7(3JY>=(ZDG^95?sRjIt94PLu z?2!0C;IA%oEffSa)2l?6ylI3J%&NsAUh1eNUB;bWD?X{IF+){GQ!snZSxBU%bo#7E zEOvco_;Jl9CI@!nS=m+`7}pKoqPI5Fc6WZvAOeB1$zL)b8{4Kq`LA0Bo=V{x|J z;q0)V!R=bm2BGk|I!FD{dU3nAbrI?O&##NU+zaS^v)*9%d3FB)cA>5$-J7ZUlG_zu zk9#G3eJMZX)?6y(pLCttO8#-G@XE7B3L!Cw=pJi{{;*ABtKij$nCyLa$-CO`-K!xV zj`JICZAnfRCDTsu5Ti#1!y@wg z&M;C@OZ zXUe?aO<#+O_K~A4aCiFAy^?YL==Nw-`^)_p$|e4;-0PzQ=1W-Qx6~atz->EO2S<$( zAOCB9%*3TVOk+2TKEb6^ZjBb{R@{-LjbkAr`00Q&i;2&2(Fi`!!W$pA>_5JN0@U*d zxYsvYNrC$3k?QXgVx{W3?K(TWk6!K<|97O)KpRI}pdjSXrH*A4$npgsQb5oOKZtL% zY9XK5Ui--Ol?eHUD;B8s@Li#3FmT7281Z+nlhp?1Qi;Vs={m2i4+#396cl}Xu6Z}A z-^)T8`#J}t?dz`}g}v;i^geDhr&y?Ju63=UfuO68-FFvk`x3o2Jzd5;6;vAWr4+0&`(>c=A%26sFS8`IizSrgN{BG` zwl2cyFnU2+4{nOVaS%2AGC0WszA~JY9B=?eYFVKF$&Zr=WeYHhVeU}@QODE$AvN06 zsbIU=aSAUTAUU)Qk_dM#CMty5$SZCFTl-a$)R{ zNppHeW*zZ*3wf5ijsI+-G+yp<>BgxbSCT};U3>bmcq@sV!@LLdto};V6(};k+FXB` z$c9?pVCC07HJc_&-7qfP){?Oya7KEQY0<#yCAz`C7ULuo&;z|bK)Uf&8$4huJq2+D zE->LSWqt#R5g;xB;~CLJ@Por_2OXeLWtcq*n|SvSM)moF=s;EEPv#{Lv?t`3-bgE2 z{Wz)fZ+ijDK1q8*&$PU^{8Bdx^%yh^&J&C8>z_;`N0hy7L+^5Y6NMMyNAeS^N>&c`Q^JSJH0TIQyi13#XU%t$@VRB+nBunffv7 zJZOSHXqI1?Tip#UrX$ii_;{Emy3Z~{c2fw2^t{wErG@p}bO=YIWLMX@)lO`&!IjcM z0SRiKLL`PX-bGhB?e}fd|DQW68Y+ZAXumOr`uBZv|LmOc_c^OEW!5o+9Xa^Q&2&4! zpaxRhA3!ckwst=&OCmu!i`O5zeo`47NS8<*ZW5c+XMg1 zJDA<7q-MB67$EMN6vbJo}V2)D77nZ5b2~TpI~3Wrxmm zjK=iQ9U+lXa+RdwC|Ez_pyDcBvIq^QrVn)0kuZ6rhL3st#?qLUhPb5lxb(gi3&6S5 zp$BVGo64>t4EG7fQX(n`LyYi8FXU{OfPw z10Ic@vkv#lgh!bc3WOC?W0J1c$Jr@EeVpnNwCRU=bElW7ZF8ruY4f_x&ZSLxmt~~a zDeU=%1rVUCuFGTiYp@C4FBOKv7}p*LJGO!wjh2V{C{NP1ye6fLrv(SDNIrnN6MDO)X(&v znsRv&V=H!NvzZgy6#_RN=0sgq4GLT>f36rTKa4u}ED;7@0piDA=vBaL6Nx14n~>ym zqs`-d8i^akb;ZwcP~+{#3oAIQ!%P3noqq%Au`7n*wM-vJBL$oH%Xbz6cMazrB%k(l z7O#G%6I#^8k^x_9)C9xBhUmnaSqk)#Sk!}gFcdacd! zJqrLYjyxNmOr&BcFH3k)`Smx=dx3Tf-H1moXQT+V0)ft$rA}0wd-EMJ{+vp+hPV9{ zAT8I{;b?6@{Iz$ixt980%Gs>7|{Vxm*A5|8T7u z70f7)&aYrTQ&DE0{Ftes^Au3KA5up=%ifEm#+Xb(I3jVfycsXD*=rtf-a_dm_1~`Y zH(f>>Pkb@z=T_6BUW@R9m*hzmooSN<-iiDM3W)H({sIaRKNak8MD*IPaO>IeP;2-kZn9N7JXg@}Z|E$ysGgY)F80XFX0rhN37-$bvG0D)K@Xdopz9HkAZi|| zpGj)Ow@OkqldU$^Zshc^zU=q){P@trbWm>54(0!P(uz=jp+TifE;h$3V2@lP`{QRA zQe?tNTYBzHT97Y%fQ86{+6iCK+L}b#%#YojE zS-+KPDiq@N8r3AG5{Xq*&2T}S8tpcH zl3<-7*EQIpiMI$O;QA+Iq&Gs4}DyooYm+q ze*~lO#fvKL`onJBkM?FsX*_jolIF~VTIi~<9L}&+4g3^;rZn9YjNKM^`so5ZE!7UC z=(yZThkNVGl9kNCQ+9j!)3GcGVl4E#8`ndFO;zgy-li!RrnIl6MVj;H%i(dQ{Pd`_ z-=}nmtcG?#3imWP$e#jJPQ!i;1Xg=8rtJ&du1m~Xi?WoAS3&u5P49=U(&;m%0HRK< zEXWy1N}Z4d&}TJ@URdDYHQv9XE~A#snk8P|=mYfg+Yq8!C+NB{|zrkFYFw zzy82l^ZGe~w_AaqG=r%$%nfa3%j%anDM*TDF^M{I)~Ey4{iGGFZ0ahLt9Y+>a0hi# zs-98OA-aEGY{=%iZ3k;oEMsBc42y~~aPuUAV$w%1!6L208(9Cm=~{_F0+mO4`i&o* zU~lzWs23sXD_8NW025ekM%$1$cF}@}WkGO;)J9@6MeTQatLg|Ocb`flpap=QaEDp_BGGE8p*Yem#|cJj8?rf62mqh%n58CW#C>de1k0_q zE>rkj^&)sUDEdI?nEM!4Po~fi-9e~A|FQ!?jA-Qq9gYS^BYBY#BA$o;D*)vpKVpbb zQX-m}aBPm9IV-K;wt7x(#$`;2J&s&>r7Jt=G=l7yK=tTFiEvU5sDq=Q`rkZC^OwQH ziFi56B9%AVFS$tSLtf}gQ>uFB`weVF4X=UNW9Az7UHbO?bHG!E5!}2_L*4KYt6|JL zh>Lwoii7vQ(}Ll0x)L`_zst;MYuOz5wl3jZq3rbeUqP!~A=>!XmX25!?=@{%LA96z zY{?7wQGy_)3tN>Gbmo1%e^!i#9>ZTKT_G|AlsfS36+^mBR zlD6Rv>cfA;5ew0w7YiJ_qrYS+p#RFs z&Kbg?i~g;!6O+$o@mgYK1Vkv448=)nCMDpfX-SD2k1WA9((YR?5n8r}#p@FpCm5Jfn(&X26)5(yw zlFALPReERJmbwutP5f9E2N@zE@1tgKxWiX%N!IOVXa9LM{@5LS)NcJI0S_WU_b1bs zOqXmVKbML7F@K6)Mm8QF>ru9UtKFK7@|InASg7J*7|ze{?&de&w0zZ#!Sp3OZ*R8+ zhm=CUaPV(k#K?LBvlFp>S?4(^uHk*+%gBBDaM9oA?#SIQN;+iE3VdkDc9>hJkJ0@x zwwsF7W7^6xEJ-!DmIl>a?Jf^oC@v~{Q6X1pR!D{9VO`AP^U7lM>83g6m1A5^adAvH zL}gi{6CuBJUU#;r%u!Ni(kFmOpc1L&8O`IkOSFOdnJbBF<5@@><7XrZslqCWo8wm` z5vc+YnaoIfK=evO&ZMax15sC!X&K~+Wk+3_678Ubgr)N7aNo(J>gmHgV>x|D5bU6w z4r`&XDwrG8ow7H=9<J*?dyn(PnPXdD>{ya>><`{ zFQ5>z>E&6$KI9!VG>}Xeor+DqcTPRLCg&U$3qN+sgjoXzp90xqwO1FOZH?aQ3_f7A zt5i0{r6`wTt%ji57PLq8Qy9Qx^;`1T_vpN@8O%}tIuHdXIRdL~n~h@=eaaH?U4LVt z^=c_qhj;ja4DLCu?pS)Xvk;AP4yRN^Qy(0b=h-dBSkf|&2BI&yZ^5- z2R%|d_B)o2+dAFF?aSCM2|Z?!8B!KjIF(;+oM4WM~597IS-}p_jpvhI=rq# zpV5@j48rm23iv%Pd)K#ai$@qcA2d5nG#pn_&YA4OsXs~z zCJ8z()f{C9U&UeVGA&#u&u#UBZ1LBRpQiiorS5Jv#f&>E`JT;=H5NE`cXDvdy6IJh zy8{DVajM^`M`m8XpE3>YX?WSqpB(x=kG$+>qJ210y~yXDZ2S}0d655HWSM$7h4o~w z>mA<5xV}zjihcb<=x*TXhIIZ7-I+1|uWb5%3V8mVMDQ;S(rKHH1+2*aS6uUM`X?-a zknlSiW@IZ4B}}WtN?v&UfE>k8h$!_&Ee%X>niu=$3Cf5&J?tF=kLNN@Mv$?*S$J6m zNQawmCUIC${S}e3-H^1{^Z3jL7NU4>N5i8dKV%9JQ;XTn67MHb$|jyt_`dF|&s97; z1J$f)p+|F7;GQPa;7z8RD~v^dXzSF*%fHx+xd;66K8SH0yGbxcYlFc7Uj>~iU7a}A zIcoUBEHx5Ka`#hSL5O{{F7{Tfp|Vm_bM=?1qoPD)b{AE=dXl2n%>e26C{l;l^Ka&R z{qESB01t`>xgifrdF>H2VAaG_Gn9f?DHU0Ef^spmo?A!lEhnB`SBL2+Y88Ledgqh6V-5$&Fi4& zRU%2MP4?4^0acs{(gyT1?F|S= zxg>&YKlP6+Y%{gz34ZvtX6MKyK=x5X~I$5`6W!%QaTlF~o%wJs1h#;F)kD>6?) z6hF)vS{sF1?> zAZ5N0de2pym4OtE1yvsE>pHFDGOc2B>$HQQDYaY?$TjwsA}Zsd;Mh1Vvwrc%doo{E!Xn8p>WzJZ=!7=4Ms?wf`hhgoJ8fA!t`HUKFsvK;C=DIs zu=*$!eWF_G2jFCF(siews?sNs#i((>OJ!#R&?WuqWUZEDzi|=7S1p3?EE0xJk)4b&1H1RR`nGl(5{9&qJ-e}e)>iL`c$#_&n~$iHu@i!37e+>Z z9^hQBeb)5KnPObY+R3I)-6Lc;F`Pv{=|)T!ga0|M26{)U0HC??CD6e3e~K* zs-<(SiuV58j&E*c9tye&iSHcUj{lv2YE^UtUyJ_?^3PScZd}ZBFKWI=$)ESK`!A`S z16u8PR-**;wqS3&fx|<0p5Z@lY+V#7s>GofC-O{jv}HhLQFXY3uF^LGuDJ{Uu=mXg zs*)3pEF<4|l(dJfkp|^ct_--+-sv1-I&KnQ&@|38qhPQ-9{ioHU zeH_zYz)HgpFNjQFg$|1Y6^z@sM>X8A2LOWry>EUZdM2YqF^vaK z-WR>uS$OHdb(*0)iX_{s)|Iu2tIuw{uakHYb{eO3zk|y<>GbrT9^3?u#BB6NVX{fA zuBl2@VDqWIj z%kj@xj=yWVDpjRy*XiLm(N1t6T&hN1)j#oN;YS@5@c9--0nx*vf5{U|%|&k#@>eh7 zl9Bz1MZ92^2%v66pl~#O-pa@m4H?cz&u57%-?QHP1Iyqn_)5tUd9I#m{fc1S|VLCot46OzOTQeH$E@qGNqhflQl ztsnXoR5GI^1zK<#N^QY|Gh9HAgPF;pyO^leQ7HAMrea}M5}|2z4{8)gdy<+V5f4ei zDi%PE(YiCXji!%DRTsqWhrY+f?wuMVYmvF2Ytr1}t4rL#4Wg`=D3gYDdNj!?S1YLF z^KO;EZv5AK^1sUgpOR${PMA4^xKv?l5Y0MU)j}P1j853@T~T4on8FXGcC|W8=Vc%D zHS6w4_Yaq_9ia+z28_z@we2YC==!;-p&{KVSY2dN{g^`9t6gLvO}PZ_6=W;+a4n+6 zH7uN?lSmOY=_|ow*NUEd3S-&v`IPV7CvF2g6LdVZm0Jr-Z>A-84cp2n)#8{S-a5yQ$&GN^5c*F z+FgMG;>)T8-A#i4r%ZtXYx+=u&Mf8cOu+V-rPI6lE4pbpKh9DWuGUW=yf>zzbqvkX zAECypy7mg_DSY5l? zCT`);Dr21l@C(8w>Rble?P=QK{#&N}&kP~~3);xVH$XA}KG^@Wz4dnmaiz+??ky0P zsuQbKMnmvq95m3mfzpKULIo43Jn`g0oemLy?Ft@Q**O9dPK$UT4R<2Ng0$VvC(+;Y z4LU-ZG^q8a&IiF{0cZM}we`gnab8Y5PhizVDETCWk$Z6+!*e`Xe(&~UA@PZ5=F_ZW z0ocm0vm-iGgCX5?5iGW`+y>Z6&#DZw_@Fh0hbcfG{u?ki1^D70V2cYEdxKtk8Y+vR z*>#5V`sHiHKuCqTzHQZQjjPXI^$r0fg$%P_$%T*D-xQ_rmG*FJ7!A6h8}vv72?b6# zCjG&X5Ee-fD*74DrZ1}ZC|dust}R-BnH0|YV5{#_o)c4zvlZ)S#C6hU{%$p5lx_f! z8JnX(S(oqYTCm#~g{IMsG0l>i-81Uh9b2c5m;cwfd{jBjB}?W=e;UEBEKmVZT_#1t z-5_xPX6I2lbq6w>F@SW_d=bxU^WC$g8m?0*(5NRQVc|l@U4C*lYpA9v(>rCe;exfp z%8fH2I|CL`W@1H}#cch%{)Yk}P?m5#i*#^>t(Z=U?4T`XSwlLgHy*l^hdm2q*@Qda zudPQRNO_gb9gnUx2=69^>M0U`1ashL1NsNsWO=uOVh|$FTatqa9Yw}(Q zv2jqull2JXD*O%q#!&Byp)Z!ODpIE*WbzyBq!aXVWy!{R?`?cQGIbN{bA&_=TB0zM zZ`~b{D<$LT5tQzR!973&qHMcx=-T(77-pV4e2)6TG*DwlB#`*_RwCBjgDO_}u@p-t3qh)J5+{ZF{3a@|8DH&AB`caJ zx-*e)L}+peUWh~{s0JC*f~lHP6{>I9&2Py8wSk)EVu>Akcn?cyY}>|av21N-S1+OH ze+=1Qa=llkzw=NvGynkMKdI_r_!qieZ8Lg}9l2Z2V!Lll*G%vem{JJ=y*#NnYgX`S zOJ+iUF-&J&-dv@uX3-~;cNCWk(iwdX6o!i6oPGG)@UPp+`gm|$W^g85T*QON`Mn}L zP+aFaehE(K_`>_t6jl*)nZ&e{RLD#!@$UBZXo!T$U!ejJms6^0u=h^6 zju=H%O*9@*HdyRsCkfFHv@VrUVcJ7%u>6rcUxZ) zY9h8ttQS2!A*$I_a)N}QSgl4+f&uAiZpyczJ0&z3tV+b8^HIdg6z{C*!X11bubmdx z8nvF0rZ$jO2wZq0-ETTF0IAd*EaJnwu9M49i`p()-D?wQ3oatP6!SPc7|D4TJ9h7? z{|sAgFB$C$rJMuA)ZKh4scD15@}z7s)eq zz!#qHh*JFLL^m7uK-!g=f$NAcFb$Yz!kwd)X-^<3Iz287&QFgI$Ar2rF2_%Tx&T_yzM+dGF5 zzg+w`T4^h{vwB#A^v=+E7ia06r|rKpS<;TAJ){JT19*7pPO9l7bMke7P6N z)5Fp0*|uwE2jq`A5>yuw*1mM^a%!m&C#s4dNyVHU@&W7FMdgN)ug22z@{qQ#1Z%zC z8?*f9Z>@@MWWE>YFzzZkZ)Br8XUO)OQF7RcDac^{n9C|B2`aY`DZV!#8QmPijxQ;T zcNaK%8P!Fw-?>d6$Rbsml0o?_4WBGHF8@xNafiQuNU7&39q~1HI3>1G6G;q@M9#+N zvQ?aJ0@AtW_|VeX7l51;EUZi(_e-3S3BVJ73}hK6WMdgC>>L&79voRP20-y|X&ev_b|a%6idD8rI@E#pcj zt>cRNO^$iIYL30-&A_I>hlw1&a(0IeK&EyWjv7zZJ);I%v&Pjz_AEvSTJpGd z@w66kiEncR2WV^#AAK;O}D3Z{0llyTZqZF4hk}(X6U6;t*;NM)kIxympk< zvg9gKxGGeh@Ee(C;q&`ed}Mlm>*nTWbb#m8dIJ%`A!bw1qaJsz>-3ETiV9m(o!5pt z1aqjJg%=+aZ?&4wMsO5S>xfKS@o6nYV_se_r!8!Tt^--hf$ZY+DT6D_++xNPneN2z-qdx))|$+{@qNqX##w80s4@4) z+7vg$)|59e6F9N_tPE<%YJRuRP;+TbP95w9Sf5-5W(}v~H?5Y*l|{b0MK!DzhNbJR zSbDe<6(CKQyk*+CktV|{Xhn9DNYS;xC3?=E8dF3N zK=+&Ux5SyBx?nIHrx{3#m%zQrU-J1kN|j*o4EYPCGTP$;%#{{YiK50g9^xcFPm7IY zh4;t1LIacS*8GahiiLChvB{tvo8Ev}DtrvTEPhBpOM(59cAaO%&r97qeD4So5cwy} zAK9UjS16*H;EP|r2HcU9tk|al+?-GI(RTSnLp$z%x0&35E0!FkJJ;u!Fzw)&H>8{_E*qdE9|F=!w? zdu(}m=Suy+VZZ9_=@~ z#I3b=((v-O^dKtfyda_$?59GzAft44Vmu5Dv0CRG8mQfCIy3w`Q`}D_t>oaq?${0e zyIbKvcW&(Hghn_dd#5TBE1(|cPoePxE^1pgRte0vsY6#u{>_c!k-mZF%hT)s$BDCS zTv|Bso17y@{O73n@BW$2)U&_pcZYl6R=6P6%tQ!X*wT^XaNvOFh)d_e^@%xA%h$=* z>e!G^tZZWD$^HV8tWkbE74Z2;mcof@^SIpP5FVChGSv7`^6Vu>M zKgc~f5zREEClC^7Xv{eIh*^Dn4A%5?R*O2Y;LP`E)6S<(jk#yd|73#~W$N$y@=Q{7 zk*bK!Q;A&ikLwDdz0m=kRkiT$S!A&oV!a0Vg9h)>If7V)&%xa)Wz=n|>?QdknGr7; zTPUNm$s{RKvGP7uF5)t5yqqzKGe0Msi!8{th&Lv^g^IOG?|1UK!a8c)Vcga#DB5_@ z2K#G0OizisK&yIi@^`BjW{oY}%1}iG@RBK)|00Q9VbPw5|eGFNjW>o=Lo zkp^m$*_c)WHKgvSF=H}MzFrhU5|K_ygM41*iOM6W?(tGnSra7Cvv<}`Bk5`JW7 z+TmDnO=7nogk9O34R_^H{dIhk4L-X1P1LrmM>mYHKjjmM_CPTTj_MAa&`lSWF?5{~ zKv-pCApuwVZt5OzFY1Xxs#}oW>sHj!W$#jlOOE~3hjS7UAcexrCN&Q*Cq?HEW7e@U z8+i+vJmDdMymI`=)0gf9j+q4>{f^T}@PkgAxb;4DmeC-ZKDoK%x$D4S?C=hm0LaLo z_ni1c0b~=h06)TkUU_n_M#i%K8iwY~&Bf761eBLYrauSdS>~%F|LLfnFtiA3_Vp9~ ztd0)*5xt)vxs3L=R-APiv#{h`86BK;Tel`0T_?{u00CO%ml{l5gq<*qKp@MPD~tee z6qbO{%nBwWNOUfN&<@1Ef9S3x0D=R1gPFj3;xNP*vytUlInK-%v#k(g>JP!fFoWii zd}d-1lo`@ZB;q)g6A{`P>7?C-5*EH`ITk^U_O{j*g=H*mHo_vf)%oqH;}t%;(02Pl zckv_*ly}Qqa(9p5YS&^#OXV%i!Hon?dfQ$Z2)y4z)Y(jWSC9N~sU~8(xTTpyVkV>U zsf`3i)R^R9((s~3$z&_~&Po(PEmP{+Foowk62T^_yQ76W1Fm#8#&usP5y2fCI7+04MRvIIJIGKnrEjy#q(V_*t{=Yu;XZ{=YwzzIFdFHITjeq7ylRv#4!y^ zam#FKqC@+lp?!4P_`7n9Y^sw)-2C{O5zded7V>z;{S%`!9sFznf$y)c?{O z(7Udrw*&H)L5M$jGR{G`E}cm;j8D&j$??J4ffNY5E?iwr^(uTaec~m7Ug%^aFSPH6 zymtzSe757F`SEyZB@HJStbS#8+v@-DN ze!S0zc+SM0qtg6|#QIra33f6!!BGR=yjjRWtKcMHr5*NFoR}0Eq4j8O2JM5gK~k5u z58@gYK6bv=zS+mexcwQrnYAc>z8Xj%m?l#|+S!*+-%Cr@Mx%}ZV_PHK)b*_XD z7p*u?-cx(3F|Z3~3@F(OKt<)fd&X;Wu^whYJ|SM3Z^qV-*(2&2m#w56 zsdCum&$6En`j4uFj^7HXa(Fi051AILy&>8l+m)jqK*N9FGcZ~v9{lML(`ii3$Q)48 z@+l|FQcLA%$Sj_N=pa>ZGW~7&F|t2tdnfhsTUg$H;O|ETH@z!dq?!xIYJ0TS^U4j3 zQN6_KD_0HdsR=0->OsrGWTI?52Vq91<^3R8D$`Fq*M7(&cSz#F4?eJGO^3iLmg&^3-=DePI}4b`3*#SzN`|FCHTh8HXgh+mHoWHU$b zIp7sCLg`OYN4GI#;SajRt!xQf1VHfgs+}TL1r|QR>$miW2ozF>fj4l6r~o94gf;m7 z+YNy=)JjXRCCBKoag+L#wvr=Q8Ofq`RhjGNxoa8eC4!Euy=27%zI(I;oM%&Wbgth7 z)KbnSPrQ$`nY6W=&z~-%KM{X)AD>*A?3zp@%Gj!;Q4TgXr94B+NbRFX7OY`pAw5oA z!EpG_xp$L3CZYhYL0_ce0_=yISqx6f zvd+E-?XDn5wEPsd)&Q6dc0$O)Z@*SZqVI_G-ew^jIuFIh1g0MfB75wKW>n8rxD>m( zEig|-7hNd3!;De=zym2Tq{HvFiG1*?i2$};qePGG z@St`=Bt_$u){G=ME*HqH0$P|+kcX`~>=hxo^hSJtcrXY3sQfsizL7vx#|w8=M6dz? zQ3-Cx{l!8XWo$bFdUC;aT=M~q4ipwk+RKJ%k)|SXnBnn|1aTqXrBq9~y*2=A@ zgSk!&@HXaK^)XDsiRa0dcLEu5FIFLRMM2CqqmMaIC)zNnNgU(a3`LK)#cEd13d={h zh76Po2yo`*jB)fqML94qyqND>Qc-YmjphHx-gm&uQdH@`xp!`whmW93&;) zyZTo5RCS%6sh;WS{_6Lu+cVWQr>oBS&iSgUyX&6*kUzfk0gpL!<(d<&xaQ+;zR8x` zedq1}aWFOlzQcXKyWN}4y4%yw`U~E>8V-T>J9@cyAk8h;$2|RrW1eAeqmaCtwU3mx z=iRh79{b6E?k}!e_sSctTX%zW6>H;BM;v>^t&_jWy45cqx%N8R#@{^ZV>kO-f79TQ zk6rkW_d8_2qt7_(+dKT@2j6w?7o7Uqo$qt}T|f7{-bJUM)ceGRUwrc5m~VaOalhE@ z@i#c)f>R%O>z`e*ZsB1!I{m;a?)>82zqQBRAM)LcHlKg<-ia@H+5GRI~@&48LqB_~=bv!odwz4P?VtJUU7vg11&@B#O?P?3 z=0SUY;YaT|?DDrC^P<0E-tmARp7gJ8yw@J@yVYs;-R7|Sedgv{@9^j|9{n$SeDm*L z{kenBdd_|)UAo=zFaP|OXB>K`-LL=W*R343-w7wYb*Ix_^SHO4e(op#_0T`PcIkxU z|M|ebyy};GzvJOM{p?Y9eBM3w`N=Ck{p0uk*Fksx^dXPA(e@ww^20uS(tE!3)h8dZ z-8WA9>Z`tZ-bdel`JWHleEJ_gugZG|$+8+Dff44_mde@6j z+W6*M{%P@{=O6j-qu=(@=RD&dUwG?(IO)LW-G84O+;RT#AHT=GJKz7%bt@;|Zkq=^ z?JZ}&^o-~2a@mJ2IQFdTpK-J2zWf94zW*)uymI-T-#F`-!Ji*=(C3z}eD$R-{Mnr! zbN>&#YVp}8zu=mKZ+gc=uRiNZSFJnpgHOKd*C+nwy6b*^qbtvU%H2P+`PxtI`mIfW zT|VNf^LIY-Z{I!soa_DYx+{PBUl(8e$!o8=@b1rj&auaDyYpMFxN`kl_S~@BJ8pU9 z3GaIPQ{TAZoY#NqZC||4+y|e0+p{0PVc$o5?-A!5zRTTrz4oaqD<8S!MQ6X_C;RTT z<@1Z5eA{Pkv-~#vv%&Ap-Q|7{nSa&+H{azUNA7yw{mwmcw-?^!<-dCChY$VoU5~%= z+uwcW>2J9FX`g=eGj_Z1(8ZP89d)<8Zg<&j4%lUfQ_j5a<;#!!`pJhM^M$LodGRZr zzW?WA@qpwx562?mwM-*TcSao15+b+Z(*>3kUr6xUbynCYKy~_5;5Bk_X@BRoh+u zPbb~?n4{0R>uYv6{9mrw<&p2&>$NL)Ib)B1*<;I>=HC6g8z2Ak&wq87e|zbxu6_Qu zF1x`CFS+gCJ#2^X9Juiy9~-~7cPZ$9})zdYdhi(mWifBV>$Yj?cQwO2mj|=hu?IkukC&5bvOON-|lgIp> z^}Y`{`{R$;VdGcd@Udsy>61t8d83EkXP5WyeuEufdaKjU+V`oqeZhOT+4=7t^SIZ% zW$&l%@rk3~v)ki8{q$X*ar!BT{(8fEFSz26U+wek|2W{CANuII?>YH_TY5J*>EA#8 ziGzRj^jq(6#FIYp;b$)&@`;Ck^bvQx)x-OP3;(#s+czEg+M^zM;KuiU=|%^>=QX>Y zaG!5|{vC&$dGXio_2`E^@-@HM_Om~F%JnY3`?tR7E<1ho@n^mIx@W!pq*s68kr%Ig@fPR& z?LLp(@f9!M{wMcaIp~b9UiI!r+<0;ML6;qR{(1NK(DlwZ;KK*K;;Nh6?EGik{nKY} zefh~x*zcsbKlqnNUvThcpZfMoum1JY^Ityr#?5~IyPKXhchGA;y!V3_zxtc~UbgOl z|9<(V{ciI0k9_4*Fa6OQ4}Jd|-*MI3E;#a~_xjL*mpt*l=YH{%C%o^sZ+_K3ZTRBG z%?BU(oZJ8R(TCpj=P!8XMJK)TWA}c>w-z7r%Qye~wP$|nfrlOb(Q}{kf`{Jxq6;5& z{`)@ie3rM|A1r9d(%DN|JY-&xA<@Fc**hi_{Ed&diy{AYWo-e@OmHnw_`te z&x0?&|3fc)_MPte?py!l>$`vD$4`7v|JHB&+XpV&_g)Xa_Wg(6;`k@N|2vmG>UnRu z^wYol_SH*=o%8z_y!pbbzkK@-T=wygo_6=6{%`Tq7eDb8*S_e^H~go^eC3pv-1OK# z9dYsX|9qGK_r~`<^VaWu&gIws^P_hC@NZ7o@BIgD+Hl$}55M}lE$9B=Z`Un6c9%Du z_^Y3udj1vHyYQojJ?S=EpZ(6qo$=lE-~Zq5zh~Et2fXfIFL~zX*FAj0nFl}iDd!A6 z`G+$PdD0!8f67lkdeQU$^Nff8;#uGL>1Thj%~^*(<;o}i-IMP0+bbUR!iQhA|8>^G zzxbbJN!nxGMK|7j@BO#iZk=TFKOBxd;t5AT?$$>i`?!BQ;L=V->^=q zlu&g{=Zo1>RaIP+CEry)^i-le>dylWXEE_Yyf}g3DI{Kq7Y{U? z#ZAYHv(fHCbfwovUa&HrSQ>fmPkGQM_2&kgqm+0dUL3IS77;JRix(V@6J5iLLj~R< z(apl#+}xtc@*TBDUH7Lv>dy-{#|h$vcyYkOTSUANFJ5psPIL_~4i$Kddf@yeKh#b7~mBwmOY4Ngp*uHgm!MwgRk_k;YM>PC2f8!NAua-+X0N6Jg|Nc|~6 zL%a|#N~k2&h!^68cq!EYTqJpRU*6p-<#6Lm{eFLg$ub5A{myswRNgj5mK-V1r>)^~ z>W_FCMuoqGcp+Z==omgjytErHusBSdy|WkM{~owm*8B}$Z!3H)7#T)RxI})J^1jG1 znexJ_OKMPm31)~F;>D=}`Um~P2@FqR*YE;0$ny>I?EWJ9F7NHj-a5Yf|H-ou;QnOY zKjm%4UJsUv#a+V|`^(r{^ntSP#0&9KhL7So;$?iiq|e&R^ZPdc7ga6a%{XDj@V~49 zhz(qoLuItx__pJ^3D=!MerC?o-{@}!3z{SG(tf;vzX#xbd2i3~zw9;VN`OPMw@>n( zU&{YuJ*!6UB6qof<|<<>YJV(ZJYYO<`M#@+@nHOTU^v|HzwkSB?OgZAxxTCqq~ipc z|Bv;K8hxFNB$qQv|V)D zao#^1BD^KUAMxh}hvNkCLcBO&;Vqg)yvY53X}7mi=?STWT(%!TsfbzBjoiQ1=*HD? z_Ls54g$rL9@j|@#u;D5vUWgYLE_`JX@Z!BU;JcjZ3-Lm{1R#<-O}r2< zIY0!y#KueDa=Fv7uT6Qxehy%PFT@M+5`akVH1R^bffr->KhB?pMW7vhC@2|y%wIyPP)93qqb{#1Jd$v#5ZFURbc z%YEr`)+zrI+jf-4*inwS{8{3Kc*!H8{2TGoZoI(yFmbPsEc^YI^1qyy{euht>y7*7 zmAg=OR`gfU9mh_YcETU1zjntW+L!hXM}f5(#LIYifhM8C-a?u~{NDrj57N2+hIIZ< z?q7pr=_g`!JgicG)L*HC6;2W_#7hAZrC+;-7bv5L{{g&&yokQb7@%ywf2q4HbAOBf z$^E5>T07Y_Y_Y$Ly=&+Xev8lq`kNLjQjd6PKVHD!191O}#sBhueq9N0=q;r64F3oJf3QoC zlo4C)#}>yX9Gm$3zgW(Az<5x^MA>)ZWqiD(bNKa!|ApV7bLV@1IM+wLMT`FjVuLap z`OeXH_}qU<~I(tfi-Q*l?7tMCu zZy^LW$9dw9_;bL*TSUANFJ5psPRtly4F3<%6(9E}oQ-}Z{Ewse{(wUrxuO}e=HBTAy} zqT6myjH90by>5YlE<>&q;~3j&k8QL^2_}g@;?EPaevR_ek^c>OH{>LdKce%;Sofff zT{K79E}HFb1I|Al`D<8XzD{B&^j|qM_A>TX!42_3yi5@<{|EfN1TpU}$lD?Fp0S2> ztvb+O)O#kD=-|iL*Y2RaUGZ$jalrkzkkfngyuS?Ksm49POL9McO?QmtTHQ%K>W_G7 z4qoPe06%^?=05ug51?I@QwRBj-0wk2gRvm@^B;ireF#{;mfXd44&Wg;-T_`X*5R|4 zL7W&Y%{X2@kA6HFc?DUO8XWJF_BXwKDkZ0tn%FvuJs`$|KzBI6XH-=hYjr==guDK@nIe3pl zu%OAnGslV?E2^;4;`k!(61)@n*2uZkz<57X?MoZ0dhd_AuJo**Jm30eWWg@yHGIB7 z9m9zi^DGSK!3GPOV@Jk{AAm0QLFPN>W9~J#s={Qyzu@y0!u_(35bjr-XEeup$DqYy ziJhU>*CGEIIg9xn@lEzO8|RR_Z2X5l(?5up5Jv3tzzg;pn%|dvG}^P0&?QvZ>m$p4 zzoq;y=Vkw3=-S@fkNf3AiB^*RuH^oHvd)hqxsY6#VZg>i=rjC{YyMpGS7WEcc)13g z^hV^}g_}Y=2ETJA`CWnAT0OUg642$I?AuOaM4_$d2XIpaHhld;@~4OcnwLcF+e;VUCvhVXJWbag9aRf7IH z#r@J>;qx@x{O{72RF-kVis6451BeV)lywJ{{LW?ASr8^u5XAgorUa;^XM;QIoJHX_$XqBcp-NcF;VuNcv0a+{`Zh`Eudw5 zgl?86>2ErRUvKyyWefO~bKM{3`lz>P@&7<%lwZYtR({u_XDj4B|1Xifu^s&-_mlg* z_$XqBcu~c2uRf_i>aXZi*7rZ*^RdVo&$Wz?(6#a;{grk5G#mbxwUJCGwgl9d=K*Ez zZ}@*%WY~4IpXGNg;eOHAiOAkKBvKd=$Ffp+$Uz z?u3)o-whW3FXNX^?w4{ToBzGzgwWNkYtQ*z1JpWI)H5gS9q%gC`j z(J}Q${n`E$`V74M5OY!aKZCQ*N9bxep}+Rr+wlJYzw~i`a-SZ*OZXp0@BM+$)eF}) z`kux22JV5rUV&U1*U?{cKe@jYBQ}PJ7jl=4htOx@B^qA-2>w4R9RtiF9|0=iYdmg7E2V**Y zvsT{>>c40s+Jp8eVxsIj@j|=+A74P;9yyp=!f(NTj#8l|x{WWO84}^`D9!`-<+s#ax7LDj}m{x zpNU19Bk`ib%lXiy?Bza`TEcHb{fbf065R;*e-4>kQ_ulphbne>aYOx4f10>r>JTqe z#f$v^w!cTtQcL(P>rc!NEzymv`F{f08^g^x`sP%9bI3ll2kl{eQ*$IO1UEf7}G!wBa2Hc^7gEaxlg<9IM=OnD2iFF>K?UcqUidcnE#&C|?%&1Y|0?g$ zL8WV{>3o^CRQ29p)B94PeJ@16gK@K?*thmoVXV~P#MB{Ph!=we&5?L%G+sUo{@($) z&D;+NCj0%V_6CxDgsd^qtgeYXfxbCB_F*h)bS&aNcZ%`4?bpOJ@oeKE^x1_M@ZIZB z9^^*I9!l~%89OwUFVSE$vDZhI{eDaPDCcGW;6i_9aa?pS^!u;KrE`Dum;3s0U%yg} z*cc*S$Tc<|LZ6A3nZnCauxF(td|BYiM$MJc88orCkmeBo_rU#wbnb7)GqcG3yP@B| zKrW5z&_TZD_$uI9V^Zi>%B$bOE$M)t;bw2$0R?)T!Oh#lgkHL=`jbJ~~oE&9Cm zU7j7k19Fz8%x_scd>^EJvTmPd!~aXD@8f=X9#H1~hX0pEhEIN3`IL1dZT=*()6b32n_mxkmi7qVVDY~^3+Lqi)h;FP0~-GKjuWK+ zWogRXFHJid*&EyGo2~TC%-*yw?OW#m#dE|9@lwP@$@gSmfGMtPqiN--z`ut72l$nb z`;+kuewWPuarE9FD8F(e#znI3|6AmY5A4tMb^5xEhtOx@g?I^J#6C~Fv>q?lf&1@+ zJS+bnK=?oS{2p*Shb;Ur&jotN2{`9XB4f(qB5C)Bk;(nM$ASAht&c^#hsAqX&RFM) zfr!t53yy(bPUkwcj9I3UwGL|+UW_YgIu;BK&c4%{gk<1+UI@VAMGRd<6uAK zRdOrv#@Or80sW2s(Q^MdLHJI1Pl%^#Y6xH2%^qpaF9I6UmlZVSV&tKi%vi)&dR)dkKE67j%1xjVzLVlp)&bh zAL}F-+go`)d0!;-Ti1EY>moN0FD_qjmCd++oQghGO6V`S-=ncHbaO+9{5^6m=Fd2` za(yZG&GnuACD*uc;VUCvh!-C=T;;@z6EAK0{{S$%E&&To8poW7R_FMVSao7Gr=z0I z?C;bc^=IQD^qF`eUP2hL&$kFKpF_Wb_s-GZCh;YjJbG$aU-m#xy zSNzQWGWHfRQTCm9AzsSxQ9MVy1o3ij=**j-KXSiEI}vnqGDK%QNvsC3n(4~coB2%r zQGW&tnj`T-yl8M@>a-LuUq=6&YXImkxgW}Lr3kvY3FOMY%n7Hp6kA@tW`7xb4fZuh z;)Qt8;KbA+UMk_`e$ZPkL4UKr{myYqt|R!;(7wDLO(XYb&#LpeV}pm#k+H2(Y#SPb zQh(H6$j9vS9l*<{(cdgJlrOXP;RBkO?DrSkzd-sT`v~Eks3z@sczyKYYGnQ|AMSHE z#7}|?Uk0A}jvx1e^5Ub28{&myogyYY-}gm%Q|HRkG?M`7ULRTZ`z_^vIWPMM%U(wo z|2JXZqvU;mw2!fzv7E8oGv1+pkh>7{=^q{)MD~w&3ttwvQs&G};7gnP7tl`0A^z`y z`v>XV-|&AE`Xc|I=%IT+}zv} z@*-@L@IUbDyZ_%qy=9sETm0Vy?njf?J-CC|;yMS{In?75`Ul4*9GgsKY;uCd|4re` ztQh{6F#uvp>AixY5c6GrCu<;I@b7~{xOmN?e#9` zxpESojXYOQ-rJM4dyD^<&@dnO%kzLT_c#2%+%WE!dkEiIjm`b3ybTurFQY+D?w4{ToBzGzgixny?91)R{a#F7 z2D^M4`RmB%BVUD_j16E9&VxA*R*e${GsFwI-^32hk$7?9~Yb6;M(%Kx1A!w%;m|0nWYkj1veB(~le_BaU7OPm6|{f1*F z@7T%9jnE&*P8>UV@c|#pog-eHT$s~QQD^q|7U}Q3=vOeIzje*~h5HTv2cO@Qb|4G? z%X5L=aY9*#CeW92<+*=cM~x@6H>)eZYr%U1A4Z?{FSCO=SFp!Uz{8V~FYUPgGCbJW zaD8qGUf5qN2H19{eaY1}9zvfh;zjNOEFyaok*of`<}pt;dg_hn)T2Bzz8X1mv}0>=f*$(FOZUZZKHT9Elg=#etK1q0E^| z;&YIzT>>M>8;SR&EfKDr>ylgE{Lv^^$8QMC>dNnCeBS>d^k=)Al1jd~^*8iBs;nVg zj-2Vq_qhf73-;0OJNwI6ZsQ^JnRp>yJQ#U7%7>CNJ`erQR}fpC?>esX*w-$R_dZ;| zQXH51kT>hx{}%Maw^zfENlNtR8dvTC0T+s%e7Obs^Y+cvXZDw|+=UBY8Sz5A_^^TU z?;z(=#rT~2t?Nr{d9G{t%3@!;M4sH2xB|IrvHV2z*~QyVla%PMOva5M?E5a=Lf)*8 zow|k>_O~QP5YOalSDf>eMZ(KYke{Rx%g`7A^2q&s)>Q3t#0$94~6X5@`f%@QV{#v3VDx1 zbGn4B)6rj9^H3#){xov3F#zO|`^o)X@;7x~;Ye+`r^x=6@JZzO_sU4|Ig8=31F6?9kxO)Peq{ffqIZvwg&~!K&s+yhOpvUl2!cfSm2eBXJh`ubkv}kP~ezkv2f} z-e2r%SCA)rDEtMvG|oX@8)EPKwVGhFizu(d}$$9zTfd#&vWM2+NZ@h6B?Q0A7e=TS=R1kX< zY*FV}EPE!f%m((k7TLp#IreuV|3g)!f7=%FnfT*atcZ!S?-RmH@(du_Q8~qq_h6{;{P7Fe;|7ZTbwgV?iYQ%2-z76(1*5gzw~dDbha#^zmw#C)07+HgK8b1W%h7l z4!X$7*Q#(j3h4`1GLy4#xA)d`%7+s zL*+10_MLbkUMzg^TeoxwgEc`Fe1$xH`LmjXRYUs&LpsyV^?K z-;j?2*39nvvC-CV6RZgs2%#{i6Nt&D9Mx6!_|FTzHKhtTH^;pI&d zbZh2I!~Y}mhqO_fCeWAv=-9Jmhk^e`w%r8mp&3_t#{eDj^(6kWzvM#1QgWe32OaZ| zQyLPVwe((EZ!&$E-z*rZ;zOqHEMg1y0KS8EFP19ab8<5FP>)T9#sITukB*H!ljkwS zPR56-9BE&Ub%+=LSP%ZOMdo(t`tI2CbjXW(&)q{S&f?hD_Q}J=<$vN;jz^F;%g>e_ z>r{G4p-+EOUh*^VsYvp&F~E%5qigtMe*-axc;;9K!Bg!gDPFdc`-dqfz%?7I=vnHv zz&HPZe%xYMQ|^+0{@gz*DUB>o_6fY8L+^050e|Ya1{krw$-Bv&cs>bpD&c){zu;b* zEk3gOzcKfCOQ>MzRAg?@JPT2Y|1Dk5I#w@)@^>6EK%rm?_9(QV{OfA_l6?ZPM-XEN zw55(4Qny8TR`r|xCAU`bkr~5_;s2&_zv{ie)mT39`xk2dpD`?G#?jw^FKmzOOjELl zvn{T|Li_f{8UXhRtib;*#B=Ewfc+)+BaW0}#6CZRcoF_@68A%e$$o#qd&H=h{2w8- z-#oXVc@Eq~3rU&C6J03esrVoAz^AR;C-Cb~*w7fDg8sZVuun`z-u%sQol1KIp^ZsA zXt8~r@s9l^7aHtqj>OAM;AJc96y#bvE}PHtJ0)M5pM;JO+3O?Ae!rz{lJl~EaN+us z8pj0rAMI-^_Zx1=jRC|S1zSvNEEd94az4<$T~Mjg7@!08HFoA$hxdqbSb(ntztcZv zg8SuO-WF8=EmR7`(?V_4SBUH_q&dX@J#ha3c^iv!CdK{InNt2oAKK3S#{ZKs0Qx)2 z+;70C9s_j9zT|##zwveTk^V6=+%Gmg)dWE8F?PJib$fGjb4$pJkS$|?vi<(0?y}7N zE&iV*_lugm{NF+Be;xNYcEdPq()VCgaIJ{d;25BT_U+18mSY``b<|@V1fRCxr4s*F zjHkgEE%JoV(`@s<>$hf{uwwXM#sDJ2RXJ2PMqix#4|!dUmEQvY@1BkUCOsDOnt4R~ zddC1Awl8okbDEi&*K{$KSL!qIOgtlas`-D!cv>2t!QTUL|BA)`@_&9^3779Jr1cE{ z2mgPtOOTX}*_R>yhrF(Gzu^Xq0p@oTz4~-sRz9XZhD47kZ)gla`!ZGnWNLmTUWk_& z!HbIj$Bd`W_?*t+*Bk!FH5O3O`Q9JS^-*up;{SoPX-Kf5AVObq{116u=l-Ps&_nK- zK!quiDiQ3FyHq53G6wh_GT(RSykE`x#EbAgxnFRg%~R%&Apb{-r%=ag5_z(2pJv1V zOE~M}et8~H=KhBNmqmt=SLGO~FB$(s9=SjN|Kcl+0V3HWgFs|?yFlLeTKW#BGd>Z| zlg1+9R=hJ@V#n2LAs(?|wI|ZZQ7rJkq|cu&xrk?{yK*L2Z$Fw z?oY-GXh*{TIC}37loG#O0P6*a;hMdDLZamSM#v-gySd-QiqbJaWP8N6Z&9A?6NoVY z_frSgV)BiZ)@!9WaK6L1SZb|!2aI2=sJh?ajeL(lNvL`OKZ5*Y4e8JcZmNd5l?09U#{ch zY*Q|EkjwT1Fjbw$dipZId2KzIScT%9I8V!SV}N?>5p7+kIPW)m8klz)vu6SWea`fO z{*wF2{TUuYxFKE`PeT~7&qu&Zj{h4NpGTVWLr0-lE>$WeOE<@@@xN*w?DYQ#w#&(F zu)JgpVCm2BsvHfsRaxfoLRop@;_UNZilQaqJV9zE6E zmz4j>{V<#=e!w2zz&(>T?GspfFMzSKm9a9zI_*pQYGa?&YcO6C{zv;_J|7o-ZdPT0 zFoC{+|H=JY{9D*R2lM^ik!NwI)SUq=OzK&++%tjDk19cbO&q6Y>j1PbxdA{{&k2YZ;-vw2 z*_y7$nrE^a^E{S{6w*o)=7wzlKdt@f>qEX?X$&x7`&NOT)Rlb#&*wX#yeGzZ;2!@l z{td-H^*B*5RiplF{wMcC*$S?y#Q!zgZr1NF%NSq>m$p4zol)G^Rj=ij9o1LpVT-e$p2^`^gC~@x6(Jo=Sp&J3?TL`*mhE5 zJCz3n+Lt{OSkm8|E{GTUhZZYRk9Z+oBH?8V02rPqYvc%(bm9|F#z;8%iM3UzcW6l)&YnY#&TdKHy$kePP`B=W%wwb3*aT& z{|~=YrP%Qq7wyf>%`G7>Lbi+n%J%!0y2~>6xA=dO+%G!v@_#_rY9GfnlDZv~hkXLG zJ%Y@Aok`nQZ0t<1N0qTja178v`*LhN+Ss_O@ql=#7cZ6gzheAz#!$OZp7O~tRY2kileS7Al!wG1yo{*M&@vY0CJWZgc^hX0pP!N>jbJfO_|4gW8T zj7(sK50Uzk@jv8|`|II;Hdy?>4Ba}pU&@he{`Za(P`5Ni>`TJ`kVo#HOf3H<4BM7{0&j}@8r&za zLd=EjllV-3BUc+NXpY2-3NK#%kI0uWS3>0Wv(twE2XZQ@;P@kLKu-7{NALXs$4_%b zk^8a{^2q&@;eInVFC7C!wy!--HSZwA+<#!N)iD6~351gx`<5Q*Z>m`C)gkpq{ds)N z$^R3Gr^5fi=l7%?$in}9wBI{U$dP^FOYFWt{3h=0GZpS~jsarZcQW$kZ;I=D7g+Mf zY~MNjj$_Zsjy+3lM0=21eZJ}{Zx~*N_#f@F=jmqt6ZJhTS^JmkKtH8P>L8cx2N3){ z=2~Wtph$h0-wc_E!6r|I3!O0ux@tQ&jj)$l}>d(YB&9R|)$@m}bYxGat7*yJkj5MC1gM}Nuvp6Kq-4fvnj@A4a0S&hDQ zF8IGq`vf}g1=OfN-osLkUu|5|9vn;9cnE!NI9^Ug`^fz;qAR&|Grw7Eiu#pv!?7@n z@3+brpiTP(=EeZT3%RCBoFQI_mx_4Vg7_Kv`M7xUT%7DH1NJ8}j=}XeRnf7jOW&-` zH*?Pf`tVUKpA=r?I>l?3e9!I$5HGdirSiBP_L{}9adMAGa3t-onu)(o;BRYF@1d5& z$e~jbV<$L{$@c%#`V@Vsc3-MA2AD8jhVU9FkbMHb%Xdt9kB#xgJ#Jw1v~eiO?_}%{i+^ze zi0t)|WxwCjHpzL}Ke%xHNsVKI{Ezk}&)%SYp*3s9j`6sQ+&`ZFh(GF&&o(A;x|8?V zwhR%wqCoh!z`bMbzmjtQ0^~{#@qZ88Kaf3ytu{=G`$apY{7>$m*f^iuPwpajb(gzT zV^_rMTs_9;Tzt;<{}W=&LcEbN?hwuJ>5A~L1?r-t`B)MNy2 zD~GL|-$8G|?-le%?sw?3g8qWG5PMPH#PCde5YN-_w@Unvco3EQMV|0^nr;3Uo-HP+ zEaQX~!~Ze{5E;dan(t%u#mWDW$2I@S?2Ab5XRMq?tW{vHUQTm0X5R$hHJU?Ht%_&@mngFUjw{og=C{68hzspCEllztEOBS&(jwbhu-#}@VGySbJmIw7o{ZW4f zzboZ`;3fLJKYeC@gT?>LXp@usr5wrTfA2T}bxTvkz9jq)dE|a_e<|kdldJe6_lw^^ zFO@U>joeiQH^hqyFJAs%jaPB!{r3BMDgO^(Qy=#y_vz7&g#U5$-XE~ztm|{+zHEd% zazDA>1vO{cD*nj*;x|xG_^=)Clfh?aTaTtYaVsn>-ON%;F7n;r)#EwS^nV{SG^~$i7|- z+xTXG$*ndXLZ6A3%6Q4~KlE4E{r*w+=pTvqfs=9H@e>ibWa;KOa=(8J0;M@)dj;Pl z_d9f0L4QvD7wtlMm1AX2uhbv)SM;f}?+wLE#{Xzv)3_h%94*z{mz4iG?}v4AG12$E z0`|%M4xLueUu=6&9__*R=i9sALc;$w^QD>lkGxNBl~KS1`U3tZ_xrfMSYCm<$o=FR za!q@=CK*dshzA^d)*lZ}od92s`@ezI^go%t%x@M97kN(eo!IxT8Mcl4zsY+W})f{VDVrc-ewD743icLSEE+2xz}L#lE&r9vZX>S>=y*J;-azvw5n% zSLiRf-+`wJ`ipH3%A-BF4%3Zwn62P{vpzr8Jl)5aQm!WP5i7>Nc0gXX|DVn8WuGCB z-0!|;LhdKmkZZcZHP$;*oTr!ZMe$sX{9*I|$oJ<1e2MPVmttRAA+Hku*JwM-pJwuT z1ad#QpWILGSFJaJL6Ulmr%}gFhX0$!{i^r=6kJ_t;GvYK=6_<P@=;(kzOvfp3uo`G<+>?1UV`xh$yC-;;4jUQ`{#ETa%DvqH3s6UOLnL5Nv zf|sq(Ly&8sv!Nuvi}~yzE&!3eKCgBOazD90hHt860Fzl=;*Z>5<015!ce!s=hz)-^1#ETa%*?7k2%5|jvs6Q7jd}YMT5MCxk`r_n&$m6`9^L`a?0iIm*=NhgW zE5wWN))045e~vkYV5I_H)cijNU!pm~N#udQ2g2tT|M!P$CMBreTS)5}{ty2DV2`YE z|2NPO|D$c>esaG$9uY5$<%mthi$@3akBRw*ivMAc==1(`4!_>;Kd!L=pK!kShjV?@ zTeSFpAZ_yTds%sezU25H^2q(G8XS|8ex*A4o1x@~3cZ!~dhcH?R@%$o=Gg z#!3%g&_7zi4PJcHzOAvZlm8?0rR#Y+;s44u5lEManDS#Vf;>PkD?lHXcHsiI+-v8RCEF zFZ#27h^1zqo@DMVYyWZ`*s?T99ptk80D`|SUR%CvuI&;mzXSJgh761UXYzmV#E!k> zeg`K4FWRwFY+9_4P*T|RVe*3uV?l!%bey+JMDgTrE$^GtiaK^Sq z#I`JE(jK&jaHckQ2rmi$*UXoO|3_YTFD3iJ$qDoY{7>$O+Bxrc#FP%jjwGJ4zvNae zuSz}QrBQe}bpm|ZmU(|Vj+x&q7_NeOQ@2IfC-;;48OyomUnvf@DE_JXJp=vSBKP-^ z%^ZC!u10^OK065BRHxY2_Q^woHX*C*Z`YdvENGuqBlkP(-y-{}ux<7CkBA>vn~d_` z1NW7){LaX*j}4G_HTrX{)n7jQ>%hMq>s+6yzedgb&Deb_e2v@>z}g8|EgQ#V`~TrL zRb%C>$Bv88=Ur`GxjqN<=lV%$Sv~UR`Ho+;b*$@Pq|YKh8}_&t?t$G5v111-evEv7 zuO9nGo!6k7^LygY{gI!G`~`f!4((R?KXjI>C%=<>h>&At-cu~<$@iW5i#C?9`M)v# z3%je+6k94@+9g$1Z zlFxxGtvf_~ni(WmI0+&__crNOpY1Gm;- zqRp!LdE|r^Y!N#4)7h(U(Y-#7b|@~A1bd$ z9=Ts|31x?}%xNEL8>Jq5Oq{FjE|q>Bp}(T9-y=J9D|#pQH<}9*`zZZm$i_2j->ri4 zQb+Kdl;6qNAr}AQG7#D8Bg=ljrEQY)vVU;l`jZ;R1oLwy|f!zVTww82uH!T~XAr=$+h8?l0oP_kDtR*@8AyPGk6o z?^+PqTS#+=|9jy6K|1$0%m)zf;{UEM{N%yhsJsb9I z1os=YiT*^7$vCTp@t}14$o|gMJ%nukKLg&-F?Kw_^?Gx2b4$pJkS$|?vi<(0?y}7N zE&iV*_ltVG{NFP6E#AAuO|1C`vX3g`KI{k6!4-V{jWAQ0w#%@C&Xs;1qkZ?! zDK7morv8RR)+3MHFLn=MBRzj9j_ZwK>ajP@&7>DWa3qoV}!os_#g7x zjQ#D3x{aW}oXC3Qk^2RgP_#B5X^c{jJ-WvIS%05S?pI?l!K&bvV~L5an*{m4j(8yJ z_Gvc!FVC0zxL=+Jl)1m*|7B^DiC0yQk@}MHKjgI;t3NI3Hj@4_vGvF!_X{qeWNrQx z>Mn~>>aj=0h2}loNk6Y6U2~=h^mk|Q|CJT(k=adtM=qQg7iM|~Se&0*Xpd656M_5IaZo zj|4W9m+VU-W1Tm9^zO1nR@Q;|lJ!;e!~S~L=yQ7VVQEDDdHEmm>U^)u@c#h)@o|3= zlhBTY|8ex*AApR~6uB=OA+PP+FPQtBQEFuU)xj)31YLkw{ zmkg)u?^L*7^zY>V3B&{8|KRg`(hg+d|32F49Vg(NH^uG?F0`p5YjR%_rN-7@J@Ux? z&}$Y?VK((={4Uw632P>1eZGNvH_a^fiwkXE(H>oxTV(hY+;$|MCVg&Le~0)V?W=R% zpTrnh`qvqSU>G)H%?k|eWuHy{h1#v-m znq2L&qpz&N_D%S|X1+A@$C3F1dR3? z`>kg-)4r|Zeu-UY`fOFO2f4K(rm4TN#$u;Je~rG!E6$@$lj+O+X2BfVR5{NeR=*5> zAonY6J%jcY{%@FlTgUwpzMT9&gLp~Cjvb1BZSU_E@&9xVF!nqhxQIHI(DJ_2i+ydM zJX~rMau>MYVES){KPj&&OX{!6qW}`st$oBt}xT?uB?(6Ru)=+s#K!4>wcAQ<6 zNA4FKK|Pgo3I0CnPwrTJ zSW*zdH*?Yz@+$GarR#KzI&)Z+{|kHjoRUhuDD+qI>)}(Y^2q&yBWR~`4&d)&!x}Fy zsmH#p=6+G=S;&d)0``b)U&;&kOvmkE_`hk~uX^uqO0h`I|1HASd5B*t63EI@J@z2? z3yz@o%2|QGY>&q>yP9+L*thlEFY5bWWX{uhkE4?1UV`xh$ypMdR>`-gHJi7i$02<k0r=dfsD^F;+1BwxiI1zSYeD~Q}LII?uF{#*fne}R1YzZ#{f9{VD=GUHz&pY zqLEVmM<3eC{RV5#6D630{*wD=dq0Rf7+boI_1L#F+%GD<3%Q@%4;+~IS$do>e`Ndr zxqRPM7CRo`8ojx>xh3R9$d)lc*?#|0cUk8C7XMF@`$Z*Q{-3bzN^yjC4xNwKz88w! zkm}(6J+YSX&9$`?#txS)U1cNM_dg-and-4`cesC5fAcR}!!p;b0komGD04wa437bpLB5L>5UUb7=&o7p!a#+v_r zkokkPcuDT}#l~X!82FQC0EV9H7{foDuGdj{@QL{!63=?hH@F@H&rbfQydmxvJ*oM> z4*vyz55WB^7XSC1!6Ustkk&K&AN>Ep9$Dkw-y7op4rA-Ph;8>s#{lE3!z@6?Q<49G zoIW!_?l-Y8bc}$%18iTZ!`ER~bFa>f7stU%4eL&|wC`ZzncP1C|ILkgD#Qa7|JM-@ z(mDKk!~eL3Jcl$i50SWpbA8lXwD^A@ZIW|UQ-r?c_#g7xe!oA2t?Mwy+0yH+yG2!7 zpuYTn=LZ7|=d8s^=QuUj2K>(Zo%R?>e_w=76W@#AA2}FwSf2dv%x|pXnq%2ApS91? ze@F7)EFM3pB_e&{6gW6kHt_&W^pW@TP&)(|4x zbE;+DFYP*p+&@)3bH9b~eha1iUt>JjVDbMlVx5mG(1hfDK*RsuaROwNrigt>_#g7f z{pHWS{}u-M7V@W%WuMv4Azz4m4KjT*?3<-yfg134CB|a^2+o=XZZ&Oe>OF+pK##wZ zv8~ui@Lo07U|+^7+M|d=FaHC7b` zCHhj)m%QI+2ru6OZ|$0n0h-FKP3L~mAD)XL_ah*CeZoFBL;hi{Rkp&uPX3R~m#*jS zg#Uxj?@2q5h5!4&i+7yhC4Tu)wt^*G7|0MiP*8Vl|Md~1z?FZlsLrHl(eVN}3A8*?@kX*=E zlwvv=+noak+N0FD94G7Xx7zN{$6iO+2XW#5$o=GgxW=r+(;WXd(0@mo^Gj%|u`kE1 z@jvImKCBnZVUMoQtCRkV9*Vlre4m2;u0*>ILst1S;C$SycbrkYd`~c&0rK}&d4Fp5aQYwpgZ6-9IKeg%p zyBcM5e<#zI`OV^E4qXPz=$rJ-D!6GdUjBrBoQL*(3LN-hWB26*hSUkzs`Gut zYz-0(+JroW?NAx9Pwo$4#6C~F5HB_!LZ68j;w6L;`+SO*t>AydZB1HdLsRt$eVGXN z%XQGIZ2v!eR*eJboAgaBR-_*BLc9Jl%+ zi@}2CNW2g)8l0Fq9=vQtd5~)%V<^e*Vy?U60ub5jBg;EBrEQY);QsZ6>rZMN6Xbui zkMm5B6dZA@Na>8$9i@!<^JS<{6Kby|9jy6K|1$0 z?=DP=`$apY{7Z% z67nKs%NU?+zyBWUEz8{B;{Qoe;EUajG>x}f-(By zbJbWOUS!eTl!~eL(0&MDh?+@qt zsJCeG|3KR0{HP{g8UzWFLQJ(@?_mU z&4&M%P|3&r@;so-{SE&wi;SF5<(Ei($@m}g$o=Gg6>kBeT=VA|t{N-E3)lPw16A?} zd{oN+k@yl_SWT9^4Ho||5I9B9XQH2%34e_GpE~ieZze@TW;{Vuu>5?aF|56UTmnNx$T(%!T z@b@wIDO&Bz{N~knQAE#`bUlPu$Rjt98-{QJP&>^t#7yp-Xic#e1( z2`?G{*T|P<98s0~VdBwJ&3#GvpWF}Ykoz4vU@RIb7G+qX{-{4~Y>|4z%gA_1_`hbp zH2gp6^LCX60Tbv8_@CUb^-ucd$iA8IA@xW7X?=}cC|Dt0D&XbR3GiiG=Kbk7W`488 zN1woH>===cp+Yh7ylX~{bOwZ*n;s=lkWEe?xQ7OA^IMLF5RGYn~*b3*M25e z$Hpq3mE^jU+VALZ^fzs6mU@;ezj#NuCE03v&RWZCbxv`une_75&xe^TR^ApfI%3D={2 zT(hn?2L=whz9z?YlZu$3zmfYTZfG;{lHp|wbXGYD{}#A+tY-(&0Ep}@q&dX@J#haZ zo%@^d$E3JlbXCg#=mWW*>l}ArRpDB;NuEmPfBVLFX;>90Z z=^sP>QHlSdzo^_V@`TURZ1X>0Jd~uej1yK2|H~LaWDM0*6pYaqC;vkp=lz`btHyPJ zC%KyMgsQPZya;a%aToPhGLKa9k(&R<;7cbPi#+i6K=|C^|Gu+wdVe6TXZSz(|ARfU z#{IuUL;SDQwHmLKM_-qiug$~@@uJ0w)FWPqmlQ85{s&&7&->Fk{CdOxxW)q7bALLeq z;!=#*=ZTl8;Kj@T5&1G<y;Dy{z?hoR`V2OAk_Zuu|juG&Z zVT3%O7mTZk9I z%1C(0_}}6;bANoSb@JHqU}JTPeeHt0l>f>7Fo-IiLx1Fca=!{If+^x67ck$C9@Ubd3^;SGi4 z<9;be)olMid{)I@`cfx+DSZY<`pW)leM9QeKZqCN#m^-*`iIT`yx$KH3daZ5L@M!r zjkdG+hR-8t;|J|QuF+ye>Jcx*ON;Sh_`hk~uX^uK!PPLGJ}Kp?`Jb_au>Nl+LG)TBMw+^`2kl|7 zpg9sR#ES+erp}DwWh>ep*Pbf2^2tNua7MI{g$>#&ddJ6h3ijh924Y! zv@hW*w2y1_73ROdN5bvYA7ihm^#EuhLkSjUF|2=U3 zAf5Z0cNZqb{i3T<{wKCFY;m5h82b<}#0z4I+E<7d;w2(pvi<*r*fa_Fi_ZOXb8}0` zi;$i0Kic8D|KCHsWtsb1{69(V7ZrK=pT0@o1U?wcJvyL&&_C!O{`fEj|ER?OXkS$B z7kR?xX}0-ac($0NvWyc}4FAg*Kx7mvYQB%r7bpKi9@qT&K0djh+@D~8@ql9;@KDp^ zftvrv;7cbRi#+i6K=|C^|Gu+w@>*VRA+2ZlKluNHJ+d_K`1yV+L>IU<2p6KhE`0Z_(oafwT$rE2jv3$?-qrao*2) zKe?6MnqYwOfMW@e(yYXTApb|;%N$3GJXyC-v*G_GRPu4ZJP#;yf5ZRFA|oeM`6W_c zGX94=azD9$R(w6!PU4x|g-{yAiK#=p%sgI7`9Bh0X6$Uq+hFnkGAcN^U&@he{`Za( zvWs{=Sl1r+e;p+J4|(K%azA6GhcDRaKf57*9Q(m~X;r~(J8`ubWVx~Zw-5K>0~WlT zV9o)594m6H7{m$jB4Y@X`TtA|9zw@4QRB8nGul8kr z^BPV$?uBg_+dMh|Uda9Ae)|fi_2^u$Ff>1OFiO+cxeG%GXAfTFU>eYj-%dlhpP3d=Dwu-Pwt0BavbXW z®<>BEMroOmH#T)6O+5icX-CE@>?`O@%z8@ONk0{$oW`+TujPTw5aH=DXcua zzO--A=gYnmFO9;>spNj3)sx)1ncplJ#!o6|V!u(?=NfMEycNf?T3@FB(tovBk$N+L zmo2g;k8I}XW3ADnzfs>CfDY9u_O*TT(4bAo zm((ftwH5Lz@jtQOSnONl9j<}H;Z$>B;+c5XVnyl^FO9~F;s2&_zv{g|1y`%b5mHep zPtE_textFU@;|v>;V>u;zffKQ%8Z9O8gIbQMq5_37@Cg=6`fyC`n}*C#)F$mob3I7^gBI z*Zpy>k9vz1{|}^1K7uYQkI8<_R63;os6X)|Z6;ny@e<^J z=r8)3zpUG*+3^1o?CIlvc^**a{)YdTrA??`IYsJA#{ZBjP^UZU&@he{`Za(P`5Ni>`TJ`kVoz(_t)v0 zrLu`XasvWhDMsw`#0&9a<01X*<^PC$iAyC!ZjU7l{}1F;Qo->@+JKz!KaSq}1CF2O ziX!)ABjl0$$^DGI9{!?>Hex z_JuF8`vUQs`rtn9?ZbWZfQ4X;1O7Nx- zNB>Hb)Ipv{rABc7YG39zll#g2z+^C{125!$a(@sf21~>Xx!+(xbButO9RIKKTi@)d zw=H;1@}&3ILD#c167S2eYwXK$Yy8i5aG+{ce2ak9+;u3Au~LgYsYkpJFT~4Gtj_qq zM!q!Th_S{Ju$?;9+?SO9$^9@$j+1@g8G9Lfeb{i76EDPz3m3jJ;$>vKB>Z19UmE@& zc^v_ESEmW|1^iF$he62w4jnKSjU0;-JW+qtpEkBgJ>q37yqr1#zHH09KOM)+Zx+l| z!KkTAtd50M`g*eW8^@l;pF&6a8~rVW5&L`x@v=qk@2B_q$DXG{fA%|mW6OhW)G7A0 zee%$tO~{q`zk}G~vu|7{Qu_)0gIrxDt`aX%@UoTMKRm?nale$KcDDbY7O%u=6s+={ zORhVq#uM~6`kOX3OT7-`#peH!@6QV_##~2$P1Gs&wH5Lz@qdT0W$`uFtO00pmqQ2i z4|28EH>4i%5*06o|C`4Bs`vgBT&*5QNJXVQHUAT?1UV`xh$yr!RHRmy+il#O@}ZRyh(c#0&A_AJ@`9YVnV) z&})!ufy<#Jzl*%)50Sk-vh4R;+9o+K`v(`UKdEs{kpI!XgbUCLx+>*= zVt;0^pN%(QQ}t)sm-Z!I{Qg4!m?{6r_Wu)N(LUzLcXov6qe-HJR zW$thB|0KCzROIDP!JImG~d+i^}~XPxw5| zHvdBR?p}**B{<3bLX2btWu&0mv<#|Av`y2jW zmNub&*3BD~jBgjgUv~C-*b;VC@_%H$bUkk;{2zRNPuhVj{ND$jyyJu%*%!XV?hC|k>Vx~Zw-5KZ0~UfY4*26( zkz>UmP7Ica7ml3_7Bt5(@iN5!vH8*^PuBjW9Q`XzQU`e+l^VhQt9_Z@OztQ51Czm+ z4!n^2$^Ai`7%UMl1*4`eu{su3>FdefZyb9Xe+nJxZ}hhi zM(pz)#LE`Bzn|XcAA6n-{n_vMjV%whQK#6~_Q^woHX&E${|;h{&%SY;NbM)|4{~*t zxJtZ4!OK>1|L_pQ$Nf@{+S&epTD%ggQLxH)F1hZc8c)#Q=x^HCEcH5!7n}b_zCSO# z7;_x~Hc_Y8*H*}@#Qz<}mc`dxvj(8aT@D@4KgiWu-;jF5OH{lV{%;!htKR!laJ70I zAr+PK)cjAZM#XA6p5RzQ;-DrIf5e{#C#DYZ(mA{c|2K*I0l{RyKh@qqvX9Ud?q8_* zpT5*NUrL^L5WAarTIEQ*5HG}we_TudsKr0FLa#xt1uln@{4VmEKScKW$g>pgX{-nk+LH++x!oe3?->7#%FJQR`GvL+%IeR>ka?o z8Vj(ebKM{3`lz>P@&7>D^@(6v&@jv8o-p_fzDh^4!B3`_)NToySkNOio(q`hN z6fZ&khyJ3k`OCU}nhpOi!JaiS=xm9l~bg?Wc&|#43*yAo838Y){1182xj%hpZ-d4E%V@uo`=uPo=6~-v0d-4L#J(i_4|(K% za(|t^St^_OBR3%6m14v`PrMK>HXhQ?UjC2Bm$+0y){{z2e}3TnEv6>L4*9m$^Vi0()GNZ@PF|6J!uEB@P8k8 z@{SX7WMB9ayDt#GsSobs-ag#t4p<1rIN*$L33y zJX!mfa`dk>Ngd>QRB8nGul8krGr6DK4@?GQI`Bg7C-(<&Vz5NKkoyf5G{*>d$?^Xx zzxBk!0o$ol&3#GvpWF|Fz5ei($@@6Z8b(a5nV!4vgI{b^&1)FWQT!po@>;LEnm`_pmE z{AR&i6^xp?#OhdBrLQM@zj5qo{3&##ztP`97_rZH5HDNg{(gF&f9!cW^k={0H?}<3 zMxA0`+b0hV+Js!0|2v2+KKsUXBDJ5;KgiWp;wteH1ut94{lh~HANNZ+YG?cZY4J*| zM!_oIx#YT&YCJ)IqrYimv()P_UTpp!`To4{V$5{}*hHOTUt1xs690D?TNYn)%^H9v zcR6%G{~%XueM9OIFH!Mg_`hk~uX^uK!PV+=mfJ#haZo%@@27beC1 zqN`H=C-!Fs``LH{HdTM7eQ96f#qTflkD2n1Z2vzYHcf(kqjUe<+}slKB4j80k9PR( z|MyUDS?2y0|4)+pMMYlzC-!Fw`)baI&6B#cFYRlvpg9sR#ES+erp^@bQi=c3zNp+U z@`TURZ1X=b-)-a=Z>@PF|C2YX~`pY!+qe2D*v)rnx0v68Wp z>oD0fMSKqi0a_Ik7@xiIS;hY~alfqLuQ&XUYb?N?&UJsB>!aSH#s33olaHXw$|Lk8 z$N!MWc|YgI@;~H7=l=Aa zy$u%sFQffV?w4{ToBzGz1k^1}5&M$xKje}7$^CWuW~prAkKBNOSBeq)Jn=%j*my`k zd-*>iU*b{;k=tVl!~X+0l~i#2kv1SF{Ewse{($4BxuVE@*$8>$esVu!uZMr=ALJSY zVETtg2MzKMC;vz0OV{&u!vDeN_oN-j!vB5X$vaNSk$vGy?7l$!rarijd;4&oJ76Ig z2Kyc;VQ|U_o;n6E8#jADb^-@?`B_%F(~lBz2JIQK=E!zuK4i&E$S^ zKQI}L>A(xQpWGkBiNO-_Lhd(M&>SP+CCC4({MI*n>TL_2lRWADbl*uV z+#3J$9UQ1y72hIYHFq7#W31F-PwEjb#0&8<6st4-uaPg!IAW}^1Z<~HHTNaue{w$z zlH+9Gcg9}EULQ7G<-`l|;=+ZmjCdItFA4wG%$J7$M_xyO-PLIVeF6WI`(Y4rze5L% zMI*AU@u$#{ z{ziWbVZ=V)LA-2{`}^sA{;}uj(4YN|-`MhC8+D3(ZJ#_eXcKZ}{_h~R`0N|kiPU~V z{~%XaiL1m*6ufLD_YV&-eB3YPsGaTqr^PF=8U?F-=aTDAs__K#{exVs^$n>Bo1ma@kjh=aAN8ZFP+1S@PCuI9}rCT`%~==B>MrZMN^9(4XG{&QS2^XM!Tmuj2%B4%{uNGfV)=pje4wca!v_}Xd_IcuE zX7KV(*sXGcee2-<1;~{g;{P7Fe~`}o&ASVe;(pQ9vB+NjC-!Fs`(7K;zH9g^0?;D_uYqZ;lhfP5ijHhA2wX�&A_!iBF)iI-R5_o@VW z(YYV|JrF*(_`mOI3@+bWNb4E?5B~pPk1XwT{=T0-5+`zTUa9NcZ;CSP@03ShxA74A zOuP^;A&l7Pi5KD}#lwqm3?|^E4(^vV{Pl+aag7Bu#JTQ|bA8lXwD^A@Z3;HrQ6WNK z?hCOQmqQ-s{haq_c(AeI`b@kq_PTK4D4qId(6_i1@9^Eyc?N(LZO3$d`%yf;_|j1Dy16e{!E5zf1TZNALXs zX9ujp!nF|pH3VgoQy`k8nk_lGcIpC?|37aI?u&tu?af5;w6kQaT; zzqnVAto_S%hAwUu$a!Spe|av@J5I2EGW1bo6s|q$yg!kh%DZZHKErtplv2ul#q-b~ z$4(qOsWC&mkPC}3r0hHKQVB1+LYKLe@|)RLXWY56^I3aY`gK-T)6O+5ii7x4;!v>;-xZPegHjYsb;<$dmSNbsN%x} z`trs;=W#0Dhq*Y9mFE5(!xq>4xrR&sD2Wk_2OOJJ z;gI-vS+_s#GmZ0XUXC^ce3`2o`%CO=E9G6JwO@SO@i}@vf1w|L5`W|xZLE@d#7ige zvJ3d%JeN1t{dpx{#-{y5UhHdEkoP`~Ptlh;;Y;Z=ThdqdSK}9^4*i38F<8(XiI?&5 zg7@#g-*E0%z4upvs{yTWswnTNkXX9LNvw{KRhj3na;nIpyb$K>^TZ$VV&ft7nRw|8 zUfyBx|ET-mMt{d1_yh!#{r*&Y1Ia!@RzD`ahH?8cf1)pS#+R(|3HO62^9lPL{e%8t z<015!c!`RaC#JmFH16+f_#buVWdC6F!G3;W(tVNtM{uRt{%95kk0)L|4*wgvWG(M z9st@7zT%p-=%J_^&3EdrbFnD#7vd!!M|{3Q|Db>Pu;D6q;N^0e`=h*!VT ze{ODW33(B+WeiZZ-+vGFmSyg5@&B~AUljBV7)x~L2K2mPaniL&p+ONN(+K~I$v^cQvBFY>_O1L1Rv|K4W}}#okOOwthmNsXyv3gc19E310q+ zdfOpaN|09v_sbgodc*&?hCEN_eD4qE`lz>P@&7>DRH+kbCHk7b_1=Kt{{b4~EUy9dfE`6Sw$)R*6p`^o(V51}LR!dM={h<%=TAzo5EJQl}b z0$yDF7WEx_S^Jmkgc>7tkjwT1C>1e@zQ|Z0ZR0s;Orzk5&gp-ALt+Sj~e~s zM93~pQ{c|>pV(Jht#EZd#=19B{F9n?34&S|LN_^>U z&p0Qh+?O|@?WO!r-=uGbFk+u4UWgYP524S*3-MCG$d~Zlo2JZ{-gb?0q5*x`8}0V; zKYf$FS%#0|IpT$QDPp4RJMluiIPmeWD08L;@TId|qZBr%FCRqvhsFT(P5Pz_7rrv$ zg?RB{!&Oea5HAi~T!=Dxetsx5h%blQR<)pEeYq>zpW}b}CVjJriL&p+3-MBhkK#Gv zg?K4qBK>|8jzg(od^yyqKV0Vf5H6$q%X3MP`aOAUHkftv+vQzxvwAh^#ia=Ip3TIp1Btk_kvPkMKD9W zko!&Sa2zi}ne3OJCE-iQzP01yD4N*oBg=ljrTj1FW&hwpe;U&lX&<=%ZfGB3 zy2<21*jN?cpg(dUxln}_!4&aAF4V@Bj^O1`^fODVe3bm(Ju^?PV-Nh1`4||r2>@)E~E-b@G@f`8eO1#_wc1{xP8+DDo4tc`oX}0;_biqEFal(q>e;ETvo9s&3 z&vo<#TyNGRvT+vL^K9hG{EzmL`#BF* zllzGmkI&FQ$c3R8YM<{2UVe=}?}R*Qev?a9O2U`v9Dcpwf0W7diq7}`aITMfix&S6 zL`J2Hrj_M)6Xbr=#5>``b;!Y(j`orJ$^Ai`7%UMl zG6qQbU*^a@?w98QW$thIe;MCriTlMYl3s^g8rSKYt@X{s?zAuM8;Sw;dE$k5vGGvy z`BOLtxJ+eki%aX0)xHfD|1aa$PVSd-B%A-eURsNn-^1oNX#l^W8U9pgpW*)je&OT(WGw@~OZXp0@BIP) zRaegxu5I{#8t)D4j~4wEITz>AK5{>~KZl7TM(ppzOKW3!f&to>_O<;z@cD7T%QWt3 zqglG-XrJ(Z@cBLY9kTGhJQwI4CzQ3qaqUUZ`-{zcE()?(kM@!K$^97~Y;3qb6EEZj z7cP8d#7nF3@?O|?ehSaC0dBg4_6?0qWbI$dLlUHp^bhsg!u`@^8OwYTnR}P;{{yIb zpLmft-s)J+ctHDBi3dLdH(r0%xL?|<;eNzBNh9%o@j zGUqi)4m4cJ*w*UUM(*M~SUt|^;#dcLK9bz;9tq3__hLi+a~^ECJ#-}Zllwy$ zvCk7P#EXrG(B~Pz%Uj9)u%aW)828I_h}+>Fz$t<`bfMC)t zgLwHI_~HiO4f77$w4Y_qu`uirb=^HDJn$vzcOq~I|oDc5bjoc5b7SaT{I{3~)p=rriMf*0O;lc!OTMNG^|2jo^H$=-WLN1M)jFpU)A&l7Pi5KF<#zW{c@zOZF zTm|mGYsa`>_1@nUxf*U1P9wDM#*q4BWak)wzS%h6)=#02_Ft|8ZuYdAJeFu-`w`mw~HFzhseyZ#%lxu*LjJ)pl)=gN@Uo12?kLSBUIg#V>1chR*k zQg>PA{uckYk^7S#Lm#`MpWj=X(~t|Hm}&YZxq;k3{CRw^1}^NHzx|?1&-%$&Lijw* zHvg|N{@N!qPFOMgFJl0au~tR@Nqz@jHR}gvKZPcaC3Z(U|95o^z%@-+SQUR6i#S%) zVnyl^FT~4~@p1uT{H>7<7qxocFZ>Js9)SB-EdH1Ojps_J-djlP8U7Fc|6rFODU)kM z&$Gb&1`9i4T=*WeTg83oVy>S24!=hhA2*p}iK3o-->E-xgAW_7a^i(}apA&O2E54p zPUd+g-gklfWetD5;eY600s3*S`{P_6^%gDuAE2!5^Zp_h)d$Ig%eXoypX#L7BolVWo*2hg1K2|_Vg9rmUa6y8~&F)aDCh_&jZTb-|+vk$Uyzd zNq&bh)-28YUE{#}q2s?n4#sr!m)uY858}jNiFg@1meb$pZ(4u*3;gcr!kpg~?@>*c zW{BT5Sp2_?-Z{Bn%8_jT_l^@n8k!;96zBb}vCf|8?>Wd`tf9Y*MT|vWd=#-mypa2g zm?-;Byo`mHpY(d`_HV&l6cEv+0{_of} zFMfZae~jcGR{>K;BFj2zSUZ zKkBcF-w`jZ#LIht0r{WqUAPA=-OLj39CfUd308cFeeD^RcT1@24CIWLG@luZS{aKn z4A8!`uO<$dI>bw5ynF+?y9;uXW|%*M_vcL??PKiz+Ok8RV_&-^^6n44or|1tfbTQ< z%V!7q>|hWl20z40Wp2pnmilXz{(c0%csz2(n=RoFbV-+D_g8n-)mpKy-7m7ZllzA_061}$5ie?Pp#FfNmg40S_{UR_SCDh5#r%=$r}+}Q zzecX+82j2Cktgp}+#C5!^q1UE?sxJO@G(?Qypa1{xbT&=5HE6n{}IT}|5eu!{y_Iv zQ|$g4u||H`G4{2)DQ|v1yf5&2D03#MBl`$}v#RWw>y#yhA`h}={eJ^_`&T^S z=&$^b;m4v}-Wj+CSsmA;E`CQYBp0eNL%hr^7piRB68nAuvF#}2ZIMe8`rCE>(Ck@X zrbDQ(*GCTWzvx=_4=(WatncUV)s6JG(ATkat&n$pH0E&Rvyg*h0mviwllxV?06cN5 z$i2DLSeXJ|zJhB$9eGz|RZ_%Ll^j#4Yx2-X<`Dn)!2N@C?r-Mn6xA>pjUy7);lM%2bcDWhyi-0>>V3!?k?)MWy~IW6w{nu-;rC_^ zYVJdmkCd7e{Ry9^+2((j1T#)pG5jxM0NT!_s^PL4u`2rwAAtNcWEm5D4p+Yf`C4Sg zHjfUVCvrczpZ-ywf5?0JUqqJY_MQrVyEo=4dm=^|p04{IDlV%}HOA-0_80s;0Qaw0 z{O^3H0$PN6dJAbi!~enmAMDe97j+HXPkNlX0R8WYd=KPxp>X(l-MC&W(F@GKrDDS@+rt~Mt&{wbCG5L9eF43 z-pG3+=ibTx|LvUzJXhcU`0snzJ6mREl%1%AWN(tyvc2sc8cJKTskB5zi^wPxQqd-) zP#Gag$e!>2xkh=*o4%iKzsLXk{dR91@3Zdfp7T1-b6)41_kG_B+80@;gYh5M1JKRC zeEeVNZ+Z<+HDmw5HDG%&@4i1s!-9OIv+O_Re^OokKiJmaZo~hJwnOUspVgO;_CVU> zcLxMyJeXYn|3}%vzpekm`}Wgr#{R?4P9$dg;rjvMwf~I$Cr51m@AY=1uOof^_ehz_ zh46y#g789I14#e)WBr46mi;FO8lTtp!|~vlZU2eR6NtCkR0`5|f2{5PW4}-O+vd?o zeUWwlV&mRSIx_wt;0GOl}|` z_kR3`-TQ&8Tak6^LVb32+JE8LhTK;^y8##RBklY5&F$d5f4JQjLF32zALK)9|6~H(*jB{0{stt28 zLpJcNA$z!nK0ydb{x)8w)$v=u`}^B;gVYxpI~Ex( z2$B(A5MBs?SR{b(g7C5kA_U2Qnin7(h@5=h-)}t|2!4(bc%H?~^LLTwZV|N3!ht{C zc8EM=>{ysM!c>G8gcm|077ii2{BQFDS|2nq3JV5E zq{8}}qdR2o^g}y~dl3EoZ;wSt`y%bT_$cr*2g1w0#|zLTP$7zB+AU`PQNZ?7POtrG zr`P|8?f;46f|k>e5nd2p<{+_P@NeSmmit7(LScj#gqMZ*n45<1^6&Ek+w*BR+y2j0 zH5{IqCs54Te|Qc6x50dw7mGvM?%!`a@;?Q^IYqP#Y0TL)z}&Z#%@cBDR%S|C$?* z^f#ox&1GVtFv83KJ}>aT{j{60|73s(VzwW?9}r&q&)9!*n307TB9?}*^}o*+GB-iy zCdB&x+<0U>K*oc)Oe_>ec==qYz#YUWmdmPr`p0FEjR^5>$nl?Vq&Spk9;qAAA!1eqbJT5Jdf# z>1M{hBEM@OXhq@yqFVc3nV&E zAl`Dwx*zO^BhU6(2#`5xNMA>InS;cFL4+5CmjwZtlZfzwrGxkWVLi-I%`ZVX4{!@T z58#(`W=9~pM*8M#9v1T>ydbnh+gcpRD**q-fM|k;{dHF|cez+YFUH{9vCcqcb zAEG}3AQlNAydb z7`Fd2?%#SwFpz`D`}g)u5d9(gTd?2GBqIHHa?C=;s~I2``9yg6-{xhJ=AWB9`A*>J z_Xxo65}JIU5XkgTe>;H41Mg%+^fwnXqG5y=gcqW4%#(odg77jA7DS`|HeO&w{I2bW zbwdW+%sYhP&wu=kVEDR-JVbv)d*ZJN2ro!~`;`jfF$gaRFU0BiHRZ441?=s>&-?qG z`}IJ!AMXW%-wi|xhk*)zckVD(+lV|we{-2wD2(ue@UjpebJGxB5MJgou~7JL{2vZSW5MBt0SU806g7C61A%v;_ZM=ZeFSLgb zsx$ds0QlVi@b?4@0}{#a8>vqpZ*6I zb|DAnL<5Hfj&%AQO7KYy94&CD>3=}BwWp^mnjV7O+l05YNBU*Aw>l0Y8&BC?Eup!oM6~a-8yW!hV}6zvN_|O~$Gq&qkmIYHMq6 zM+l;Y<@5U6Is!Sgp1?2ZZijOR{zTwkZ|h`j2mB4dzrxMU1NiBvfnUUCmK?`fers!+ zA2O}29e?H^LL7v6~T@pQEgoDBTQKW?smKwp!`GVqt$$y?JHd@=#OrMh_Nz=}QGNAn(aOzsXPY?WA zK~9^@fFG0-{m>t`Ss(amfxq0x#RM(`EayXzlPVlPnaA7D7%sB_i0^aoR)^~d@?hB9 ze1LCKK1STudh%LqAYR$mX_F?12j^nU?S0JQ@=TV&&VHjB@Pjg7?Cjl5r^|)`bZvlT z!sj0Lbe&`!akH!oB>O`xS(yQq&=9$QyEm?4gy@F*9eFO{Tbmza~%+)CswR zJU&0p+c24G5BY=m@gK2M=dOetCR6<<&v1cS!4=%F9N6{PwOC#7R~a~ItUOj><~&}y znR1eOHz=MCZJiPHgT_Mmp&;II2k9e&CqX(g#3X(Fj6X&`wA zZH7KgN|~=v?;CF1C6;u?K63>LZH;Eup6+7p!}fhSQab?v=S=| zYNCQw0(DyjVrR4mm)h?~*?#JBM(;EE;8OF>s^fG#ttjoP=~CFt)C#oZj3(wv`I&-) z>_9G^xj8XMF}E?Lm}1NmOaUer;>8qV?qHr{?!o@)GOJH&$MMJ2j3+hf0_uR9$s>7P zvR!g)CiJ7O)*uEh0o(?lFZkLA_yYbQRS(ZVZ=9o(FRuz1k?nc4+-=2I@JdTc$w3f2 zzQcJSsF!7O+(xlH{Tb&~1VIX+U~GVgR$e$41X=S#(A`!DVjTDx$NwF4uSf{GS8nU) z?LU08SD@@#1DypI!Qxv&{{|tQiHT1185Uu0c`_XcY@qNsdqxVq5aTd z=r|M!#X$+s1t=Lxg>FKbP%cye6+@4qXHYp*4b=fQTA(jbH#7i^0z*!QqCqjD*ik$v zL6jIu8YPcfk5WVFqKr`%s2wOrlsn1~wHtK+bsQCqNl@fC|Hc;lr%N$YWG71{h0>1I8P(7ZZw!$0T9WFnOS@D=`h2ub5#hDV7n-i(Lu$ z)Wn)%ZLwb1z1T2p0yY(!gMEmt#5Q8Pu@fXTB%CB-Bx^{tNX$tbNp_MPCW$AxLXu7L z2=tv6k^xduQWjES($%1ET97)E?jj8%Jx7{OT0~kw+DzI{Mn=X?CPt=6W% z>@?X8vO==gWG!Sv_J;iC>UY$=G&D3qG|DuVG`=+9G*@X#Xg<&k(K6DC(`wP$ z(FW6=q0OOvP5X_Gf=-Z5g>D;N5M4aoZMs)7_MV@TU!LEAKbHRif42aS zfS$l^f$IXbf)s-Cf=+@b1xp17g@lF7gpLU13Vjx46V?(A622~6FG3@tB;qBKBvQ4U zbh-R;m*wY{my4oBSBW}_o)LW|h8B|(!-<^}t5`v@LScpbilh}aE2&qitn^=bePyFK zi@2_Mhv`AjSf9AQL5WYv zS}94XNts{SPWg&*o62$(XO$Z&-!@2Y@ZONUVNi9A>K@gijhKxZ8$&m~R9m8Es+OSk zL4BFJqxyCAZjDtMyEKY4Ni=mdqcm%@xU_7wu4;8_uh!n9{ZNNm$3!PVr%6{-*IPGF z53Q%I7o%6NFR1UXpKAaaXc@#9yf+j!^fJshA~iBJI%CvoEM>ggxYUHvWV^{VlYvbe zHbrczGZiuQGkvg`Zu6GSS2quuZ8VEEYcQ8E-(&vFg2Mu5amSL(aPquMvbKQ1tJKc8c?YDQ3>@eGLZO53kfpwB~zm1m7S(|QKRoj!c zU+k3aV(r@P744(#+Z+@fq8&avDmunEwmT^~#XEh)sp1lFJ2Bn1JKXNNFL8Hqf9S#O5#UkbDdu_Dv%zbPSDaV3w~qG}AC%8lpF6%xV5;-nPsHz# zU!(te|1KZFt}x(#=g}3l>1%wzX*{Gi4N&IV0IwyAlJct2b&LR9J+p(_OSQinj>qETsVq7 ziaYx9n9Q-$$0kDULZ2R&JRX01EX+3S=?SS5r%vF*9l~Em$VHrwB#m^Bd=sS%#F1BZ;`gw-+@o<|GLv#V3;{`zNvi#+%S2(U5zcQZU zp3;zNoci$U>Z@0;abAnKj=t`9z5T|P8|61Q-n^5xGVNkIOZxF!&@I1P9T__^-el@z zKDaG^J1t8v>ufe-c4!VNCorcw*D1H*j>VmdyV`djniTQR(f4rxv}zbl~&cWYW?bpH>Pji z)@-S1cx(6e^E=mfJ+=O|qjmf0N$Nx2)4z}Z!1*ERp0=x>r9T(6tGAbTSaq~~@%%FS^>8Oc=h<%}-?F-tyPkKOcenI- z_u#*W^|JS-^vU);=r`>DFyJ;YIv6^{K9o8vKU_LuHqth_bBt{4)cEr8ya}C&db}GR z?`iFAJvkS^fWK%52MGF}0zotuVD3;3L6o#p>#xZi@CtP5fIo@&r`Bkb*}%*tMHzxj z_d$?u5d@tFP7wI1!6!WZ-lPnnWn_Lh5Cb1R4l>e{9u&1Vn3@dX@x5FSM3x6Zt(oST#$f)d4HUwO+n_=Dgu)WKjWr(^h_f*~>|+a(YLJ5~^D z=V5E;?cwPo;p^n*X5(&+a}{^?w0Fe6#Wz6o+Ui>B;1)m#(ggn?d?U0^EdU3Uq^Ado zf^)%g9mN4AP*dtyk6QY}uZ|L#-XK6j&=K6^CIYN7c!2~T@L~rJPkmw%r?)ZSxJ2|! zZWNS=nel_^7C7*oki4Ojhp&f^lZPj-jJPBuzd=h6EL|p#Nx$Hvm%fTC>|F$uiEZCU zaq8>PhQ0VAh=Cl61lK_=gU}2p3?*m{@(OEIH>#;?XliMjY%<+!2Ii%aAPB>>DUqAny0f9jW4jwvu z-qjCnP0bx_l)i_3E|E+gaH;xp(g7Jt%qj=y7S;lc%pMtE%7BynR>O z*wp-~rM2yIdq>ar-oE~U!J*+1xL$BQXO5|w&8Zgys23WG#bC+cdZExe;fgb0N%*8l zm#7$#S$i=qlip9xv?1zZMll7yjBz)!jdukli-2sSU=Lii$(sFM#X|m3&3;tuXT9pc z$^rbG1soXgp9DOKiiDH|9Hiu6%1KH}PDws_P);AzlLsw)(ET{@KnVQXA@B%nGVq_C zih_#%Uq0~f0haXmT8J8h0x)41ASGygEGbk3`ZpW~U;aY|=7IrpK;S;15&36YEg~^e zw-tDyi~9tYlwdu5-dqk|G8803dH=Hey6!7VAH><@1vM4A4n%D6+H|RV`=J)E74eiW zsp#v@@aAse?5T`UDJr_k`ha_>D#{9rht$)xxkBe~Njvtq-Npqj9*%*w4z4 zhi&}3&-qhrwYe@Q4-{`>&RJnyX=D7EzD>Ag+ku$Zd7+TVB>`%?=5@vuo|Gb~;_m#} z`@ad5ZDf~cUnP?e>K&7*8fN9M=JMW5zfzNRkO~sD68DrlY$|YmpZ&hJH&?BOE|lcz z8q;XMKPC4(FqFd8G3Gm{7@@*Nfr@FGdVv(ZqjnlGS zn0WC4GHK2|HQ#*pA1HeDT&m)}^{Z7yy{^*LRT>6WF2Ve3EDB5;ukSQ(sp>3|Va7w2 zDcI$Nm=UAOxzBnh>vzPR-)=;-=n zt6}zBx2AO3txs=Rbx7vlj>QDDlrYySYVq&?nA(&_+55$9b+KHGs$$NCOdWrkrLH!H zVa@IakX=t@%p<>ngqxSYJ|^GkL21=$ERUu8VyP;9Q7D=HdZzusYjO1!m(`fea4(sR z)S&rXFur06mMhwtC(0W3;rhZZ#$NrbF6xk$plWqqW<2|9u#Es7ayc1zS1dp9<+@v5 zXZv2@J~-#L2c|8nFJnP{hf1kNA9DJXl4jE#>dSt8B10N``19@E_51G#r|6wpb#e9K zzG07kY}7d`qcG04a5k1wuA?S>tAgDFQOqoWmw3| zGW_l}HJbh5-_&|nR-d5`6ntwpVR{-5QLv=rA*D<_^wIP)=E2CSj$#+T)>6eA8x+rU zYlrA-@JK%CP~vtr7*fuXuqf;*oXGZSk~dL^O=rJuu_yO!d-B-^CO44@s=KX~ECxCap)}qObt@6b={mV? zpQ2{_*0{@;@14lAtW69Tc?|Ki-Q2ZKEYcxhx7F2S7xLZw3Rl~=xOIKEx_vso z)Ysv;oi4#8@h&nJ05dT_s0F!<-YbPeX1RHc&I?E%SUkHmC2aI_w_zh zW11t$;oq~mhR2l>f)A;zv)s%7s9kBzph5Tb3-5A+d#jbYVkTB$NBqja_loZ6pJ1F= z)h!>&9LVzxXlFum zh>DXRK3PtAas13qJfvtcF|5>VX{mVQ&B#&dyd0a|I~c}vI$ZK8)RV50iCApi*)<{L z-IT(rzd5dNB9F_S$xQgBN;6pjUDDP%RcC+FB^Mny$DXgT8)dBgm@rz|5}p~_>$v^s zYB~ROn*QTkTD9n+AF6+TirpS)9!Vh_<{0i3W%91Lc_6%LY5o_PWryB=dC8g27)L3I zj)WA}*`3#ef|!}xYBuxGQ{PmKzwEAgltjbmR>nqZPbSW9)aqJ1a%;S{enMMgZ5j^8 zihNKmS{bg%+2F8GaL(9^!N2}Fp00}H&kwL?Kis#oy?~kl521}lIF6Y-M?Yu@eZl&I z>A-bV(T+VD>OAFd!ny??stcEqVHv6z%J}O!lB9el4(<8=@`7|`&skqz#fa3-@=x5# z>?@)od8?C+4t#$$h8;OjO;c`4KOcEyal-SYaf^uUPR$2Ohui}ZNN zf-_^>)^;RyS8*Mbh`Ce;?LulAujX1+}0o(%u_4XMI0`}7sBt4X-F;32bh9(_B4Z*HzKF^H}j z&sG&umm0iX=dSST^{q#8SM^_6DGt=7M_pd(bA3=sDckqL2-$_vCZ(P$Iq8?gSL52E zp<(`xV2$oT8?2pMO3cODs-p_zN^%VhnjU=+{nD$wFd1vTSgj2M~oCU z=foTV6)X!mA<=TOzgJ+Q`gQPU?%=MYc<6DKY~2^C<1x(H>5-!pFRR(1u&}G^lWyOC zk%Wi5gI|Cg>FrUtsP~M{nH1VQh51XpWop9jq!sXA@3aln2$q;oW3ypiuVF9orPZUF zOUK}VX~r3YyDRul?TC9E)OLq=oSe6hH&oBSDEGDNrNX<^6^}R3AHRFdffC{|7K_ap zI??EIb@wTU*!&e~i4U5+A5ckc2@lv@T0UY{!_B_!V`IvZoAwq-(GAiC1{WlKKE;XL zt&2={YOo!)R1`OBHb2R zC(L&BZn;Ajw>;YAkh$uiRfa&s-S7q#Y=hS;{c~a&>558>>!jU2v@)&>y^9&t-G9oF zlB&V3p;>^IcGu>^Cz?+z-FE+Qss57NPb2tN96}%D4Q8Qu@>!Wb+%q7N_Gzd{B6^mi z!&m<3VNk00%BH(|(X>ZIiyC7`4!o`EGbG(v>yS0yEzB@kpDprV9_%6_%AUd>!Vx($$WKAcy?b$1a}7TvTCxsK)c$LMn3fqyvRt^U~YJ z{h#*Tyue%Gc~eG7^TXFHdIX-gG1d7K+1d)vu>*^bM!otaWk$h=#MibQGO@C*2{ zy$IUFglhH7eQ04~|32=LhHZQ;iwb`@tN+$O`LZIYE&5LJ!7aOW8J~HvoOU|k!?fYa z;9ggB@e_gWhhAkoauk@V)I{>8UR_NGNGr>os?uMfG+RrJ2{QWWQ245!JkMB@)XRt2 zYg{XC9H`jG-+$HltmUDDUF!_%AMw|5YG3QnbxdQ!eDCX0rr~e{L(lci5#yzHJkgP2 zN`@6QpS+HywS_(6jTDLO+->K!`%|1kdk2YIRz?!LMUk@1ov5%qxOd#y%($zMiof=W z0&5XQ_x0H;b37z7!OWSU*W5XrywrTPNnLkZ#Impm%ihbemC?67K+Ej&yRk~;qU6}-N`bw;Z@=BVydpz8 z+%Qskho-2f$_O_Njyfj+_x`hPMvU|G^P%Z9?{B_b|4Ldb)@-RcPgHOAov)sryP9s8 zC3=%E87a@_c<|M~Dm#6B)dXidN7i?}KtJ!zT9z9^4aM0{Ki%^@(jvlGpLUH|Vudod zmqf`*4i}0QLV0(h1TS<_8=S0p_I=>cyIbNP$m@)@>|RkWEN#=;!6$PmoT17tCFQ9n z#i!=`d-Q_rnNY=A-Ce3KY8fF~H#;Adb_=^@!0B$Z+2PH-#S`7 zGU}+DsxnDxQK+L-gNMCSGT50vZ!#o9rH(07WLu=A+`DrpDs6Qq_96oXXFisM&RA37 z<)+Sd)51mz9LE1y-6dMNTk0ovN?_9|DtUJVlMkCXpEbK=q|A+zcu+|DWOahz9W`nT zBlb-qHWY@Ys%~t=Yup71>&t<+hpV!w5 zQe;B4D!tb`@`efAljHQ|1jv4Ca%XA$dRMyw70x9!laXQ4IdguWFxR{Wm`` z_?kZ;;~l&&#WZ=d2L$m!CsIzy-T=EWfnX<&0c=@#diqLudfWR>Q-h6!@uLqpX!jT`0OyT!>tKs6I zm4CbrN1b*k!}x_@(NZj0%BT9-uwe*~8CQKs(Hgly*G;!YiqG9@u)z;k>a*68i#e2@ zJ-IERh%UjcSliQApViuF*hu%edo0WCE}Eyh&xda+9K2_}dSV}rnqmV^UP|GBsmF&T zvl8CBPRvSr+zsm<9`)kO(74*!EIS^L!;b_U&KeXNcDjGMa&IZ63$4h>>Guf*0oI{s^rO` zc7;5=o?H4}dpnFsFs>!9)v1v#TEYZMYE~9e5G1t zxnHr~=ds)aW^FB!+qzVSjYnDfyRISf$WNNJ=FOk2gwaH|A(oonbrm?cHeCaU!_g7u{Y|aj9cx*xeS!_W< z^Ih>8?g_lrKrm=HJRSoPHf=?m!DnW$d3H|{Y^zQ;t*^C>tG$VDpsT$P+_+y)TX~$3 zWGekA+_{$bgQELmDbsc=HcRV0caD2keRQJNu_CLrdZXVks%K=WVg(Lo$Vp&1M#o+$ z4D@u#Zn=L?$L5>HDORKSj)Fe*Pj@BzZ}V!OT!U|Uj5^gu*>q+9Qb$~BuTn`!@UH%m z>dU)S3Wj8c*^ktaG-`|Cyc#>kLL>VR@iI+_>ee^u<zH#eN( zF%k==sK`4;PkmJGln-yeCiqPBAnB=ha%jW-(voUNQJY`fy{zE##5nBZSarOAyH(&G zp1$*242C`n*{|Svz&mo@Gn?-;=4|7rwYK~*?fSG#`GOuB(YBBE71q@y%XFTkcowS_ zpL;26l+fZkwnpM8b6Y*#>NkNB{^K46o_@>Mg{40(&pJq#9O7Xt(;5Z!=pSZW*sIf0f+gtbCfR zSHOxc_B#Xh;wF08g zj!tCX=y`oly=(6S?taVt&T?nA2W$!17D%ckOOoHhcea&p-J_BdX4U8FTR)z+=r><; zA+7aOQqndfgSxRgR?~Yj@4t&Y=e}_EY)Y?%_C33u``ib-+fYHC;)B(tA9UT+@{Wq) zw{NVfySSNsDE7U+P)A{R(vD+_+rkw&>1xgOEO9w0@l9LvGGBhbH(tzm?}>ZATIJ@P ztW%*BdNgO2DJNN@M69GGIaNungx(x>ZDotTx!zqW{d#yxyX0k)wa*{CI=PJ*m)2{a zcCxR-v?hfmzMiYXX>*P2^-MDZw=QNYb!q3py!cqJXEmzVE_b?UsaUnWQ7E`PT+6NG zpa1Fcm$+w%$!E+9AIjcHvE=D!tSlaIT^?llU>7*SU zN}af)g0%d!^^TWZ%xsz)Nm(*%)>tZKl{kj`%X8PRy?iR^p2@M5!1Ffwo{2k@HCVou zqtg$k$KILGtxwbWb$o5|_-B55w3_6y+S4OIs-SEQ=Rw!R+AE~IkEM&sq>AfB@quP; zkBnX4(sh15oy>LO$bkOxD^6XTY>orZfKylOQm9k0FI;`sR!bB0(X=Bbs%v-QUpgs` zXr5MP2s`S3Wp8fco-d7gyk_%{B9BN{u!REykvYpi_~Yz%{xA@+CVB=XlGOM3dw7!S z`QDSgPkp1efm}RBxwe7ZIGsv6M&yZl?)HefZ`X~Z9;wCnBIl-zWy9L z@Qq(vnJPx>TuPtqu=naK56RMG(!vhMek-bc*Eir`6La(6g`FCa%A*HUe7QpO*m`Yf zNCpQVhp4;SZz;wc7I!?|tfaX`@@+iXOWAs1}=CGL3xXXv4)v`yQ5@@3QY9hN`(BHLJ3v-)U5k4-j( zmx{ohkoJSZ_XVoen#r_p-DXvblfJT(3+LZ)`P5;%efwxs8$@(zpql_ z!$Vv2#6$JX`Nysy(IegkO5?PoTm9?ax#ym!u`Slx?^ou+SR`?gx<lbgEc}j^ZD-3_Ov~2pLHqYjcfV3~6J z_VuGfyt+bJt0a&4Hzd1Ej4!oXW3``(a-9lAv#=4vhE-Rpo}slH$;{=b6_~vQb65YfSd2XWsiXGBrFR$rKIy0Bnb>}9Xj`Iti;CpYHT592Zp!}-ne&L>B z=Hw%%6fI8%?u|MtQ@g?6#MDxYp?t}G!`Buby~->bL>X^Ll)aun=Vq}<)wU%GH)I?U zN?5h>yGGpple9&WK}pZ!9~?WuXxca|$<5bBe>*M>|6GE7kJ{HGG((IswP-mr3)0r? z(Dvk0D_Kl#Ub}MZh^yeC!^?w8d@d>``kvh3L*HM2a)W}Mdr1J5w7EvX$cY#pwNF|B zsa)qaS~bQ>@;T$&%6DuXZWwx=(D0;I^|8}Q*Buw?N;{JFzGB&}>6&dk#@Rqs`-C-X z>8R$l5K??s=W3N^{wCWabsF_;^1Yr9x&_+Tevqvgc|Rb-Y^kVF-PP@OqW-ELgTD0U zfp=QGV%F=5O^2!5Fxg9YTkOg7{^%+kxH_GdEB;vDU~_OQ_ldC=ktsPh4!*O#sJip% zSl<`X?A&EVdsjxpxSEoav2)*bcgx3~X&!9!eQ|;1fh%vF&j;3y%7Jq3UFWyZHU-GL z>c3rY_Azywxpn@r@zs9gBMx9}%mQ7EVRrYj_3*Yw`q$5~BrLkZJw%e}aM#JNT$)`O zB#dnwMRqdVUOcp0VdY;F%P)5zKFcx5qc9{iFEHa-EB<9^`}XK}X=x!V%~QhkwPbAs zpp|+TeNRW{?q<-}YB2n~VUxP5_CQ0~iu^(yyOPTpw=%Dwj#F-F(5v-&cggO+TTktC z^~x*j%NT2qxY)YvpK#KtB_9^lPL#4`w8VsNNK8B@=}OA4AWEema>m1|KV*Y(Y9i^0 zaP{P)=Pj^S`DZYDyj9*%4${_?$jf?F!#8~Ne2-QBbc z*rx!`5Zb8)z|Z!8eO9U-?!NZ!zS}nidfNMJg>g01Arz4OHADyv1T+D@sR+)-fOW-o zZ+llC332%E+!JMJxv@pygw^0=UV;ho806;Wx7Nui2wL6Hi7X@t`>#H7g(mA3zl*Q!jw!4q&UxSxI2kq+>J%Qm-)bDh{?*#6=5pP zE%h8Ujrp;;B3Pw=6)}m?ToKhPGV;|xDU3nG6IO&p@mCR(4K!E8v#K?tR{^});Aa4Y zBEt246*1X0=8CAUi0&x>YJ~gP3IcUHZ=NMW*E-O{&zGR5O+~FH+SJ4)dxBqM{Aehm zlI8z_WPkJDkxU$R;$_|q?kZcn%yR#r%pu@6F^fx9_y>}YTK$e>;v*(;ZG><69m&K; zPvVkexBiY~;scEAi@Psb0L!JoLJ;ztux*xPJ!>Cddv5}sr=osO8!HKV%EZdT(7ELLYkn7CwO50& zmQWluxU=vVjAx0PcNirJq9oe|N_P^>3kb#KI4>-2-XZn&^^I6@P&#v9O9;hzx-Tql z-my{g0k0h$;K%`NeG-a$?>$T03`>)hokh{yq1M`aC*xO8I$8F);>3bxiJJjvmhR>q z{13J(eB=VU%L4|0uymWjI%A=xm>i|%mX2=~Ygs3-%ub-fgyJ$o78W;Y0_KVvPTc+W z6)0UAm~t$eMI8ivIJB@hsd?L?Y@!Z(2FQB}NFkKh2~6-pwVmpBbIW)2neN6|&=?}% zPFlk1mYrBw-sDI;SDa_oCz6Li+}&Tqorzvp+@zVCD~|a1c>d%(;%3c%XZ78AXTkAq z1v*%;k8^@!?gZ@8sVT|6PWEp0;D%!GcUDV)xz6Ol2fgRrwQ&tSJp?<#Nk{GZDG6@& zb~tMZoSU^Hkl>4RaG0vzPZ4u(WB$p9MmjyQo0{k9Xzq7TKZSfOrk^B&;2Ws9r_cRf zDxT!YlWa!?LSvr{BVw-z5;hw?lvpHYP bI(udZ!BB?;v^fNcgMU>t0Ml1s;tBmf_-QWM literal 0 HcmV?d00001 diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md new file mode 100644 index 0000000000000..9ed06be62b1a5 --- /dev/null +++ b/docs/structured-streaming-programming-guide.md @@ -0,0 +1,1156 @@ +--- +layout: global +displayTitle: Structured Streaming Programming Guide [Alpha] +title: Structured Streaming Programming Guide +--- + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Overview +Structured Streaming is a scalable and fault-tolerant stream processing engine built on the Spark SQL engine. You can express your streaming computation the same way you would express a batch computation on static data.The Spark SQL engine will take care of running it incrementally and continuously and updating the final result as streaming data continues to arrive. You can use the [Dataset/DataFrame API](sql-programming-guide.html) in Scala, Java or Python to express streaming aggregations, event-time windows, stream-to-batch joins, etc. The computation is executed on the same optimized Spark SQL engine. Finally, the system ensures end-to-end exactly-once fault-tolerance guarantees through checkpointing and Write Ahead Logs. In short, *Structured Streaming provides fast, scalable, fault-tolerant, end-to-end exactly-once stream processing without the user having to reason about streaming.* + +**Spark 2.0 is the ALPHA RELEASE of Structured Streaming** and the APIs are still experimental. In this guide, we are going to walk you through the programming model and the APIs. First, let's start with a simple example - a streaming word count. + +# Quick Example +Let’s say you want to maintain a running word count of text data received from a data server listening on a TCP socket. Let’s see how you can express this using Structured Streaming. You can see the full code in +[Scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/ +[Java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/ +[Python]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/sql/streaming/structured_network_wordcount.py). And if you +[download Spark](http://spark.apache.org/downloads.html), you can directly run the example. In any case, let’s walk through the example step-by-step and understand how it works. First, we have to import the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark. + +

    + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + +
    +
    + +{% highlight scala %} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.SparkSession + +val spark = SparkSession + .builder + .appName("StructuredNetworkWordCount") + .getOrCreate() +{% endhighlight %} + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + +{% highlight scala %} +// Create DataFrame representing the stream of input lines from connection to localhost:9999 +val lines = spark.readStream + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load() + +// Split the lines into words +val words = lines.as[String].flatMap(_.split(" ")) + +// Generate running word count +val wordCounts = words.groupBy("value").count() +{% endhighlight %} + +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. + +
    +
    + +{% highlight java %} +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.util.Arrays; +import java.util.Iterator; + +SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCount") + .getOrCreate(); + +import spark.implicits._ +{% endhighlight %} + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + +{% highlight java %} +// Create DataFrame representing the stream of input lines from connection to localhost:9999 +Dataset lines = spark + .readStream() + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load(); + +// Split the lines into words +Dataset words = lines + .as(Encoders.STRING()) + .flatMap( + new FlatMapFunction() { + @Override + public Iterator call(String x) { + return Arrays.asList(x.split(" ")).iterator(); + } + }, Encoders.STRING()); + +// Generate running word count +Dataset wordCounts = words.groupBy("value").count(); +{% endhighlight %} + +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. + +
    +
    + +{% highlight python %} +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +spark = SparkSession\ + .builder()\ + .appName("StructuredNetworkWordCount")\ + .getOrCreate() +{% endhighlight %} + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + +{% highlight python %} +# Create DataFrame representing the stream of input lines from connection to localhost:9999 +lines = spark\ + .readStream\ + .format('socket')\ + .option('host', 'localhost')\ + .option('port', 9999)\ + .load() + +# Split the lines into words +words = lines.select( + explode( + split(lines.value, ' ') + ).alias('word') +) + +# Generate running word count +wordCounts = words.groupBy('word').count() +{% endhighlight %} + +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used two built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function `alias` to name the new column as “word”. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. + +
    +
    + +We have now set up the query on the streaming data. All that is left is to actually start receiving data and computing the counts. To do this, we set it up to print the complete set of counts (specified by `outputMode(“complete”)`) to the console every time they are updated. And then start the streaming computation using `start()`. + +
    +
    + +{% highlight scala %} +// Start running the query that prints the running counts to the console +val query = wordCounts.writeStream + .outputMode("complete") + .format("console") + .start() + +query.awaitTermination() +{% endhighlight %} + +
    +
    + +{% highlight java %} +// Start running the query that prints the running counts to the console +StreamingQuery query = wordCounts.writeStream() + .outputMode("complete") + .format("console") + .start(); + +query.awaitTermination(); +{% endhighlight %} + +
    +
    + +{% highlight python %} + # Start running the query that prints the running counts to the console +query = wordCounts\ + .writeStream\ + .outputMode('complete')\ + .format('console')\ + .start() + +query.awaitTermination() +{% endhighlight %} + +
    +
    + +After this code is executed, the streaming computation will have started in the background. The `query` object is a handle to that active streaming query, and we have decided to wait for the termination of the query using `query.awaitTermination()` to prevent the process from exiting while the query is active. + +To actually execute this example code, you can either compile the code in your own +[Spark application](quick-start.html#self-contained-applications), or simply +[run the example](index.html#running-the-examples-and-shell) once you have downloaded Spark. We are showing the latter. You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using + + + $ nc -lk 9999 + +Then, in a different terminal, you can start the example by using + +
    +
    +{% highlight bash %} +$ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999 +{% endhighlight %} +
    +
    +{% highlight bash %} +$ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999 +{% endhighlight %} +
    +
    + {% highlight bash %} +$ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999 +{% endhighlight %} +
    +
    + +Then, any lines typed in the terminal running the netcat server will be counted and printed on screen every second. It will look something like the following. + +
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) + The maximum number of bytes to pack into a single partition when reading files. +
    spark.sql.files.openCostInBytes4194304 (4 MB) + The estimated cost to open a file, measured by the number of bytes could be scanned in the same + time. This is used when putting multiple files into a partition. It is better to over estimated, + then the partitions with small files will be faster than partitions with bigger files (which is + scheduled first). +
    spark.sql.autoBroadcastJoinThreshold 10485760 (10 MB)spark.dynamicAllocation.minExecutors Initial number of executors to run if dynamic allocation is enabled. +

    + If `--num-executors` (or `spark.executor.instances`) is set and larger than this value, it will + be used as the initial number of executors.
    spark.executor.instances 2 - The number of executors. Note that this property is incompatible with spark.dynamicAllocation.enabled. If both spark.dynamicAllocation.enabled and spark.executor.instances are specified, dynamic allocation is turned off and the specified number of spark.executor.instances is used. + The number of executors for static allocation. With spark.dynamicAllocation.enabled, the initial set of executors will be at least this large.
    spark.sql.tungsten.enabledtrue - When true, use the optimized Tungsten physical execution backend which explicitly manages memory - and dynamically generates bytecode for expression evaluation. -
    spark.sql.shuffle.partitions 200
    + + + +
    +{% highlight bash %} +# TERMINAL 1: +# Running Netcat + +$ nc -lk 9999 +apache spark +apache hadoop + + + + + + + + + + + + + + + + + + + +... +{% endhighlight %} + +
    + +
    +{% highlight bash %} +# TERMINAL 2: RUNNING StructuredNetworkWordCount + +$ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    + +
    +{% highlight bash %} +# TERMINAL 2: RUNNING JavaStructuredNetworkWordCount + +$ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    +
    +{% highlight bash %} +# TERMINAL 2: RUNNING structured_network_wordcount.py + +$ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    +
    +
    + + +# Programming Model + +The key idea in Structured Streaming is to treat a live data stream as a +table that is being continuously appended. This leads to a new stream +processing model that is very similar to a batch processing model. You will +express your streaming computation as standard batch-like query as on a static +table, and Spark runs it as an *incremental* query on the *unbounded* input +table. Let’s understand this model in more detail. + +## Basic Concepts +Consider the input data stream as the “Input Table”. Every data item that is +arriving on the stream is like a new row being appended to the Input Table. + +![Stream as a Table](img/structured-streaming-stream-as-a-table.png "Stream as a Table") + +A query on the input will generate the “Result Table”. Every trigger interval (say, every 1 second), new rows get appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want to write the changed result rows to an external sink. + +![Model](img/structured-streaming-model.png) + +The “Output” is defined as what gets written out to the external storage. The output can be defined in different modes + + - *Complete Mode* - The entire updated Result Table will be written to the external storage. It is up to the storage connector to decide how to handle writing of the entire table. + + - *Append Mode* - Only the new rows appended in the Result Table since the last trigger will be written to the external storage. This is applicable only on the queries where existing rows in the Result Table are not expected to change. + + - *Update Mode* - Only the rows that were updated in the Result Table since the last trigger will be written to the external storage (not available yet in Spark 2.0). Note that this is different from the Complete Mode in that this mode does not output the rows that are not changed. + +Note that each mode is applicable on certain types of queries. This is discussed in detail [later](#output-modes). + +To illustrate the use of this model, let’s understand the model in context of +the Quick Example above. The first `lines` DataFrame is the input table, and +the final `wordCounts` DataFrame is the result table. Note that the query on +streaming `lines` DataFrame to generate `wordCounts` is *exactly the same* as +it would be a static DataFrame. However, when this query is started, Spark +will continuously check for new data from the socket connection. If there is +new data, Spark will run an “incremental” query that combines the previous +running counts with the new data to compute updated counts, as shown below. + +![Model](img/structured-streaming-example-model.png) + +This model is significantly different from many other stream processing +engines. Many streaming systems require the user to maintain running +aggregations themselves, thus having to reason about fault-tolerance, and +data consistency (at-least-once, or at-most-once, or exactly-once). In this +model, Spark is responsible for updating the Result Table when there is new +data, thus relieving the users from reasoning about it. As an example, let’s +see how this model handles event-time based processing and late arriving data. + +## Handling Event-time and Late Data +Event-time is the time embedded in the data itself. For many applications, you may want to operate on this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time is a column value in the row. This allows window-based aggregations (e.g. number of event every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on a data stream, making the life of the user much easier. + +Furthermore this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0, event-time watermarking will be used to manage this data. These are explained later in more details in the [Window Operations](#window-operations-on-event-time) section. + +## Fault Tolerance Semantics +Delivering end-to-end exactly-once semantics was one of key goals behind the design of Structured Streaming. To achieve that, we have designed the Structured Streaming sources, the sinks and the execution engine to reliably track the exact progress of the processing so that it can handle any kind of failure by restarting and/or reprocessing. Every streaming source is assumed to have offsets (similar to Kafka offsets, or Kinesis sequence numbers) +to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, using replayable sources and idempotant sinks, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. + +# API using Datasets and DataFrames +Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` ( +[Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/ +[Java](api/java/org/apache/spark/sql/SparkSession.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.SparkSession) docs) to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the +[DataFrame/Dataset Programming Guide](sql-programming-guide.html). + +## Creating streaming DataFrames and streaming Datasets +Streaming DataFrames can be created through the `DataStreamReader` interface +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/ +[Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source - data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. + + - **File sources** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. + + - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. + +Here are some examples. + +
    +
    + +{% highlight scala %} +val spark: SparkSession = … + +// Read text from socket +val socketDF = spark + .readStream + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load() + +socketDF.isStreaming // Returns True for DataFrames that have streaming sources + +socketDF.printSchema + +// Read all the csv files written atomically in a directory +val userSchema = new StructType().add("name", "string").add("age", "integer") +val csvDF = spark + .readStream + .option("sep", ";") + .schema(userSchema) // Specify schema of the parquet files + .csv("/path/to/directory") // Equivalent to format("cv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +{% highlight java %} +SparkSession spark = ... + +// Read text from socket +Dataset[Row] socketDF = spark + .readStream() + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load(); + +socketDF.isStreaming(); // Returns True for DataFrames that have streaming sources + +socketDF.printSchema(); + +// Read all the csv files written atomically in a directory +StructType userSchema = new StructType().add("name", "string").add("age", "integer"); +Dataset[Row] csvDF = spark + .readStream() + .option("sep", ";") + .schema(userSchema) // Specify schema of the parquet files + .csv("/path/to/directory"); // Equivalent to format("cv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +{% highlight python %} +spark = SparkSession. …. + +# Read text from socket +socketDF = spark \ + .readStream() \ + .format("socket") \ + .option("host", "localhost") \ + .option("port", 9999) \ + .load() + +socketDF.isStreaming() # Returns True for DataFrames that have streaming sources + +socketDF.printSchema() + +# Read all the csv files written atomically in a directory +userSchema = StructType().add("name", "string").add("age", "integer") +csvDF = spark \ + .readStream() \ + .option("sep", ";") \ + .schema(userSchema) \ + .csv("/path/to/directory") # Equivalent to format("cv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. + +## Operations on streaming DataFrames/Datasets +You can apply all kinds of operations on streaming DataFrames/Datasets - ranging from untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the SQL programming guide for more details. Let’s take a look at a few example operations that you can use. + +### Basic Operations - Selection, Projection, Aggregation +Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are discussed later in this section. + +
    +
    + +{% highlight scala %} +case class DeviceData(device: String, type: String, signal: Double, time: DateTime) + +val df: DataFrame = ... // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: string } +val ds: Dataset[DeviceData] = df.as[DeviceData] // streaming Dataset with IOT device data + +// Select the devices which have signal more than 10 +df.select("device").where("signal > 10") // using untyped APIs +ds.filter(_.signal > 10).map(_.device) // using typed APIs + +// Running count of the number of updates for each device type +df.groupBy("type").count() // using untyped API + +// Running average signal for each device type +Import org.apache.spark.sql.expressions.scalalang.typed._ +ds.groupByKey(_.type).agg(typed.avg(_.signal)) // using typed API +{% endhighlight %} + +
    +
    + +{% highlight java %} +import org.apache.spark.api.java.function.*; +import org.apache.spark.sql.*; +import org.apache.spark.sql.expressions.javalang.typed; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; + +public class DeviceData { + private String device; + private String type; + private Double signal; + private java.sql.Date time; + ... + // Getter and setter methods for each field +} + +Dataset df = ...; // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType } +Dataset ds = df.as(ExpressionEncoder.javaBean(DeviceData.class)); // streaming Dataset with IOT device data + +// Select the devices which have signal more than 10 +df.select("device").where("signal > 10"); // using untyped APIs +ds.filter(new FilterFunction() { // using typed APIs + @Override + public boolean call(DeviceData value) throws Exception { + return value.getSignal() > 10; + } +}).map(new MapFunction() { + @Override + public String call(DeviceData value) throws Exception { + return value.getDevice(); + } +}, Encoders.STRING()); + +// Running count of the number of updates for each device type +df.groupBy("type").count(); // using untyped API + +// Running average signal for each device type +ds.groupByKey(new MapFunction() { // using typed API + @Override + public String call(DeviceData value) throws Exception { + return value.getType(); + } +}, Encoders.STRING()).agg(typed.avg(new MapFunction() { + @Override + public Double call(DeviceData value) throws Exception { + return value.getSignal(); + } +})); +{% endhighlight %} + + +
    +
    + +{% highlight python %} + +df = ... # streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType } + +# Select the devices which have signal more than 11 +df.select("device").where("signal > 10") + +# Running count of the number of updates for each device type +df.groupBy("type").count() +{% endhighlight %} +
    +
    + +### Window Operations on Event Time +Aggregations over a sliding event-time window are straightforward with Structured Streaming. The key idea to understand about window-based aggregations are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of, window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration. + +Imagine the quick example is modified and the stream contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). + +The result tables would look something like the following. + +![Window Operations](img/structured-streaming-window.png) + +Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window()` operations to express windowed aggregations. + +
    +
    + +{% highlight scala %} +// Number of events in every 1 minute time windows +df.groupBy(window(df.col("time"), "1 minute")) + .count() + + +// Average number of events for each device type in every 1 minute time windows +df.groupBy( + df.col("type"), + window(df.col("time"), "1 minute")) + .avg("signal") +{% endhighlight %} + +
    +
    + +{% highlight java %} +import static org.apache.spark.sql.functions.window; + +// Number of events in every 1 minute time windows +df.groupBy(window(df.col("time"), "1 minute")) + .count(); + +// Average number of events for each device type in every 1 minute time windows +df.groupBy( + df.col("type"), + window(df.col("time"), "1 minute")) + .avg("signal"); + +{% endhighlight %} + +
    +
    +{% highlight python %} +from pyspark.sql.functions import window + +# Number of events in every 1 minute time windows +df.groupBy(window("time", "1 minute")).count() + +# Average number of events for each device type in every 1 minute time windows +df.groupBy("type", window("time", "1 minute")).avg("signal") +{% endhighlight %} + +
    +
    + + +Now consider what happens if one of the events arrives late to the application. +For example, a word that was generated at 12:04 but it was received at 12:11. +Since this windowing is based on the time in the data, the time 12:04 should considered for windowing. This occurs naturally in our window-based grouping --the late data is automatically placed in the proper windows and the correct aggregates updated as illustrated below. + +![Handling Late Data](img/structured-streaming-late-data.png) + +### Join Operations +Streaming DataFrames can be joined with static DataFrames to create new streaming DataFrames. Here are a few examples. + +
    +
    + +{% highlight scala %} +val staticDf = spark.read. ... +val streamingDf = spark.readStream. ... + +streamingDf.join(staticDf, “type”) // inner equi-join with a static DF +streamingDf.join(staticDf, “type”, “right_join”) // right outer join with a static DF + +{% endhighlight %} + +
    +
    + +{% highlight java %} +Dataset staticDf = spark.read. ...; +Dataset streamingDf = spark.readStream. ...; +streamingDf.join(staticDf, "type"); // inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join"); // right outer join with a static DF +{% endhighlight %} + + +
    +
    + +{% highlight python %} +staticDf = spark.read. … +streamingDf = spark.readStream. … +streamingDf.join(staticDf, "type") # inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join") # right outer join with a static DF +{% endhighlight %} + +
    +
    + +### Unsupported Operations +However, note that all of the operations applicable on static DataFrames/Datasets are not supported in streaming DataFrames/Datasets yet. While some of these unsupported operations will be supported in future releases of Spark, there are others which are fundamentally hard to implement on streaming data efficiently. As of Spark 2.0, some of the unsupported operations are as follows + +- Multiple aggregations (i.e. a chain of aggregations on a streaming DF) are not yet supported + +- Limit and take first N rows are not supported + +- Distinct and sorting operations are not supported + +- Stream-batch outer joins are conditionally supported + + + Full outer join not allowed + + + Left outer join with a streaming DF on the left is not supported + + + Right outer join with a streaming DF on the right is not supported + +- Stream-stream joins are not yet supported + +In addition, there are some Dataset methods that will not work on streaming Datasets. They are actions that will immediately run queries and return results, which does not makes sense on a streaming Dataset. Rather those functionalities can be done by explicitly starting a streaming query (see the next section regarding that). + +- `count()` - Cannot return a single count from a streaming Dataset. Instead, use `ds.groupBy.count()` which returns a streaming Dataset containing a running count. + +- `foreach()` - Instead use `ds.writeStream.foreach(...)` (see next section). + +- `show()` - Instead use the console sink (see next section). + +If you try any of these operations, you will see an AnalysisException like "operation XYZ is not supported with streaming DataFrames/Datasets". + +## Starting Streaming Queries +Once you have defined the final result DataFrame/Dataset, all that is left is for you start the StreamingQuery. To do that, you have to use the +`DataStreamWriter` ( +[Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/ +[Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) returned through `Dataset.writeSteram()`. You will have to specify one or more of the following in this interface. + +- *Details of the output sink:* Data format, location, etc. + +- *Output mode:* Specify what gets written to the output sink. + +- *Query name:* Optionally, specify a unique name of the query for identification. + +- *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will attempt to trigger at the next trigger point, not immediately after the processing has completed. + +- *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in a HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. + +#### Output Modes +There are two types of output mode currently implemented. + +- **Append mode (default)** - This is the default mode, where only the new rows added to the result table since the last trigger will be outputted to the sink. This is only applicable to queries that *do not have any aggregations* (e.g. queries with only select, where, map, flatMap, filter, join, etc.). + +- **Complete mode** - The whole result table will be outputted to the sink.This is only applicable to queries that *have aggregations*. + +#### Output Sinks +There are a few types of built-in output sinks. + +- **File sink** - Stores the output to a directory. As of Spark 2.0, this only supports Parquet file format, and Append output mode. + +- **Foreach sink** - Runs arbitrary computation on the records in the output. See later in the section for more details. + +- **Console sink (for debugging)** - Prints the output to the console/stdout every time there is a trigger. Both, Append and Complete output modes, are supported. This should be used for debugging purposes on low data volumes as the entire output is collected and stored in the driver's memory after every trigger. + +- **Memory sink (for debugging)** - The output is stored in memory as an in-memory table. Both, Append and Complete output modes, are supported. This should be used for debugging purposes on low data volumes as the entire output is collected and stored in the driver's memory after every trigger. + +Here is a table of all the sinks, and the corresponding settings. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    SinkSupported Output ModesUsageFault-tolerantNotes
    File Sink
    (only parquet in Spark 2.0)
    Append
    writeStream
    .format(“parquet”)
    .start()
    YesSupports writes to partitioned tables. Partitioning by time may be useful.
    Foreach SinkAll modes
    writeStream
    .foreach(...)
    .start()
    Depends on ForeachWriter implementationMore details in the next section
    Console SinkAppend, Complete
    writeStream
    .format(“console”)
    .start()
    No
    Memory SinkAppend, Complete
    writeStream
    .format(“memory”)
    .queryName(“table”)
    .start()
    NoSaves the output data as a table, for interactive querying. Table name is the query name.
    + +Finally, you have to call `start()` to actually to start the execution of the query. This returns a StreamingQuery object which is a handle to the continuously running execution. You can use this object to manage the query, which we will discuss in the next subsection. For now, let’s understand all this with a few examples. + + +
    +
    + +{% highlight scala %} +// ========== DF with no aggregations ========== +val noAggDF = deviceDataDf.select("device").where("signal > 10") + +// Print new data to console +noAggDF + .writeStream + .format("console") + .start() + +// Write new data to Parquet files +noAggDF + .writeStream + .parquet("path/to/destination/directory") + .start() + +// ========== DF with aggregation ========== +val aggDF = df.groupBy(“device”).count() + +// Print updated aggregations to console +aggDF + .writeStream + .outputMode("complete") + .format("console") + .start() + +// Have all the aggregates in an in memory table +aggDF + .writeStream + .queryName("aggregates") // this query name will be the table name + .outputMode("complete") + .format("memory") + .start() + +spark.sql("select * from aggregates).show() // interactively query in-memory table +{% endhighlight %} + +
    +
    + +{% highlight java %} +// ========== DF with no aggregations ========== +Dataset noAggDF = deviceDataDf.select("device").where("signal > 10") + +// Print new data to console +noAggDF + .writeStream() + .format("console") + .start(); + +// Write new data to Parquet files +noAggDF + .writeStream() + .parquet("path/to/destination/directory") + .start(); + +// ========== DF with aggregation ========== +Dataset aggDF = df.groupBy(“device”).count(); + +// Print updated aggregations to console +aggDF + .writeStream() + .outputMode("complete") + .format("console") + .start(); + +// Have all the aggregates in an in memory table +aggDF + .writeStream() + .queryName("aggregates") // this query name will be the table name + .outputMode("complete") + .format("memory") + .start(); + +spark.sql("select * from aggregates).show(); // interactively query in-memory table +{% endhighlight %} + +
    +
    + +{% highlight python %} +# ========== DF with no aggregations ========== +noAggDF = deviceDataDf.select("device").where("signal > 10") + +# Print new data to console +noAggDF\ + .writeStream()\ + .format("console")\ + .start() + +# Write new data to Parquet files +noAggDF\ + .writeStream()\ + .parquet("path/to/destination/directory")\ + .start() + +# ========== DF with aggregation ========== +aggDF = df.groupBy(“device”).count() + +# Print updated aggregations to console +aggDF\ + .writeStream()\ + .outputMode("complete")\ + .format("console")\ + .start() + +# Have all the aggregates in an in memory table. The query name will be the table name +aggDF\ + .writeStream()\ + .queryName("aggregates")\ + .outputMode("complete")\ + .format("memory")\ + .start() + +spark.sql("select * from aggregates).show() # interactively query in-memory table +{% endhighlight %} + +
    +
    + +#### Using Foreach +The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.0, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/ +[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), which has methods that gets called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. + +- The writer must be serializable, as it will be serialized and sent to the executors for execution. + +- All the three methods, `open`, `process` and `close` will be called on the executors. + +- The writer must do all the initialization (e.g. opening connections, starting a transaction, etc.) only when the `open` method is called. Be aware that, if there is any initialization in the class as soon as the object is created, then that initialization will happen in the driver (because that is where the instance is being created), which may not be what you intend. + +- `version` and `partition` are two parameters in `open` that uniquely represent a set of rows that needs to be pushed out. `version` is a monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. + +- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If `false` is returned, then `process` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. + +- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. + +## Managing Streaming Queries +The `StreamingQuery` object created when a query is started can be used to monitor and manage the query. + +
    +
    + +{% highlight scala %} +val query = df.writeStream.format("console").start() // get the query object + +query.id // get the unique identifier of the running query + +query.name // get the name of the auto-generated or user-specified name + +query.explain() // print detailed explanations of the query + +query.stop() // stop the query + +query.awaitTermination() // block until query is terminated, with stop() or with error + +query.exception() // the exception if the query has been terminated with error + +query.souceStatus() // progress information about data has been read from the input sources + +query.sinkStatus() // progress information about data written to the output sink +{% endhighlight %} + + +
    +
    + +{% highlight java %} +StreamingQuery query = df.writeStream().format("console").start(); // get the query object + +query.id(); // get the unique identifier of the running query + +query.name(); // get the name of the auto-generated or user-specified name + +query.explain(); // print detailed explanations of the query + +query.stop(); // stop the query + +query.awaitTermination(); // block until query is terminated, with stop() or with error + +query.exception(); // the exception if the query has been terminated with error + +query.souceStatus(); // progress information about data has been read from the input sources + +query.sinkStatus(); // progress information about data written to the output sink + +{% endhighlight %} + +
    +
    + +{% highlight python %} +query = df.writeStream().format("console").start() # get the query object + +query.id() # get the unique identifier of the running query + +query.name() # get the name of the auto-generated or user-specified name + +query.explain() # print detailed explanations of the query + +query.stop() # stop the query + +query.awaitTermination() # block until query is terminated, with stop() or with error + +query.exception() # the exception if the query has been terminated with error + +query.souceStatus() # progress information about data has been read from the input sources + +query.sinkStatus() # progress information about data written to the output sink + +{% endhighlight %} + +
    +
    + +You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the `StreamingQueryManager` ( +[Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryManager)/ +[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryManager.html)/ +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.StreamingQueryManager) docs) that can be used to manage the currently active queries. + +
    +
    + +{% highlight scala %} +val spark: SparkSession = … + +spark.streams.active // get the list of currently active streaming queries + +spark.streams.get(id) // get a query object by its unique id + +spark.streams.awaitAnyTermination() // block until any one of them terminates +{% endhighlight %} + +
    +
    + +{% highlight java %} +SparkSession spark = ... + +spark.streams().active() // get the list of currently active streaming queries + +spark.streams().get(id) // get a query object by its unique id + +spark.streams().awaitAnyTermination() // block until any one of them terminates +{% endhighlight %} + +
    +
    + +{% highlight python %} +spark = ... # spark session + +spark.streams().active # get the list of currently active streaming queries + +spark.streams().get(id) # get a query object by its unique id + +spark.streams().awaitAnyTermination() # block until any one of them terminates +{% endhighlight %} + +
    +
    + +Finally, for asynchronous monitoring of streaming queries, you can create and attach a `StreamingQueryListener` ( +[Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryListener)/ +[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html) docs), which will give you regular callback-based updates when queries are started and terminated. + +## Recovering from Failures with Checkpointing +In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure a query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger), and the running aggregates (e.g. word counts in the quick example) will be saved the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in a HDFS compatible file system, and can be set as an option in the DataStreamWriter when [starting a query](#starting-streaming-queries). + +
    +
    + +{% highlight scala %} +aggDF + .writeStream + .outputMode("complete") + .option(“checkpointLocation”, “path/to/HDFS/dir”) + .format("memory") + .start() +{% endhighlight %} + +
    +
    + +{% highlight java %} +aggDF + .writeStream() + .outputMode("complete") + .option(“checkpointLocation”, “path/to/HDFS/dir”) + .format("memory") + .start(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +aggDF\ + .writeStream()\ + .outputMode("complete")\ + .option(“checkpointLocation”, “path/to/HDFS/dir”)\ + .format("memory")\ + .start() +{% endhighlight %} + +
    +
    + +# Where to go from here +- Examples: See and run the +[Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/sql/streaming) +examples. +- Spark Summit 2016 Talk - [A Deep Dive into Structured Streaming(https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/) + + + + + + + + + From 809af6d9d7df17f5889ebd8640c189e8d1e143a8 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 29 Jun 2016 13:32:03 -0700 Subject: [PATCH 0848/1470] [TRIVIAL] [PYSPARK] Clean up orc compression option as well ## What changes were proposed in this pull request? This PR corrects ORC compression option for PySpark as well. I think this was missed mistakenly in https://github.com/apache/spark/pull/13948. ## How was this patch tested? N/A Author: hyukjinkwon Closes #13963 from HyukjinKwon/minor-orc-compress. (cherry picked from commit d8a87a3ed211dd08f06eeb9560661b8f11ce82fa) Signed-off-by: Davies Liu --- python/pyspark/sql/readwriter.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 44bf74476b099..78d992e415489 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -695,8 +695,7 @@ def orc(self, path, mode=None, partitionBy=None, compression=None): self.mode(mode) if partitionBy is not None: self.partitionBy(partitionBy) - if compression is not None: - self.option("compression", compression) + self._set_opts(compression=compression) self._jwrite.orc(path) @since(1.4) From a7f66ef62b94cdcf65c3043406fd5fd8d6a584c1 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 29 Jun 2016 14:42:58 -0700 Subject: [PATCH 0849/1470] [SPARK-16301] [SQL] The analyzer rule for resolving using joins should respect the case sensitivity setting. ## What changes were proposed in this pull request? The analyzer rule for resolving using joins should respect the case sensitivity setting. ## How was this patch tested? New tests in ResolveNaturalJoinSuite Author: Yin Huai Closes #13977 from yhuai/SPARK-16301. (cherry picked from commit 8b5a8b25b9d29b7d0949d5663c7394b26154a836) Signed-off-by: Davies Liu --- .../sql/catalyst/analysis/Analyzer.scala | 26 +++++++++++----- .../analysis/ResolveNaturalJoinSuite.scala | 30 +++++++++++++++++++ 2 files changed, 49 insertions(+), 7 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 96f2e38946f1c..d1d2c59caed9a 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 @@ -1836,13 +1836,25 @@ class Analyzer( } private def commonNaturalJoinProcessing( - left: LogicalPlan, - right: LogicalPlan, - joinType: JoinType, - joinNames: Seq[String], - condition: Option[Expression]) = { - val leftKeys = joinNames.map(keyName => left.output.find(_.name == keyName).get) - val rightKeys = joinNames.map(keyName => right.output.find(_.name == keyName).get) + left: LogicalPlan, + right: LogicalPlan, + joinType: JoinType, + joinNames: Seq[String], + condition: Option[Expression]) = { + val leftKeys = joinNames.map { keyName => + val joinColumn = left.output.find(attr => resolver(attr.name, keyName)) + assert( + joinColumn.isDefined, + s"$keyName should exist in ${left.output.map(_.name).mkString(",")}") + joinColumn.get + } + val rightKeys = joinNames.map { keyName => + val joinColumn = right.output.find(attr => resolver(attr.name, keyName)) + assert( + joinColumn.isDefined, + s"$keyName should exist in ${right.output.map(_.name).mkString(",")}") + joinColumn.get + } val joinPairs = leftKeys.zip(rightKeys) val newCondition = (condition ++ joinPairs.map(EqualTo.tupled)).reduceOption(And) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala index 748579df41580..100ec4d53fb81 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveNaturalJoinSuite.scala @@ -113,4 +113,34 @@ class ResolveNaturalJoinSuite extends AnalysisTest { assert(error.message.contains( "using columns ['d] can not be resolved given input columns: [b, a, c]")) } + + test("using join with a case sensitive analyzer") { + val expected = r1.join(r2, Inner, Some(EqualTo(a, a))).select(a, b, c) + + { + val usingPlan = r1.join(r2, UsingJoin(Inner, Seq(UnresolvedAttribute("a"))), None) + checkAnalysis(usingPlan, expected, caseSensitive = true) + } + + { + val usingPlan = r1.join(r2, UsingJoin(Inner, Seq(UnresolvedAttribute("A"))), None) + assertAnalysisError( + usingPlan, + Seq("using columns ['A] can not be resolved given input columns: [b, a, c, a]")) + } + } + + test("using join with a case insensitive analyzer") { + val expected = r1.join(r2, Inner, Some(EqualTo(a, a))).select(a, b, c) + + { + val usingPlan = r1.join(r2, UsingJoin(Inner, Seq(UnresolvedAttribute("a"))), None) + checkAnalysis(usingPlan, expected, caseSensitive = false) + } + + { + val usingPlan = r1.join(r2, UsingJoin(Inner, Seq(UnresolvedAttribute("A"))), None) + checkAnalysis(usingPlan, expected, caseSensitive = false) + } + } } From ef0253ff6d7fb9bf89ef023f2d5864c70d9d792d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 29 Jun 2016 15:00:41 -0700 Subject: [PATCH 0850/1470] [SPARK-16006][SQL] Attemping to write empty DataFrame with no fields throws non-intuitive exception ## What changes were proposed in this pull request? This PR allows `emptyDataFrame.write` since the user didn't specify any partition columns. **Before** ```scala scala> spark.emptyDataFrame.write.parquet("/tmp/t1") org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns; scala> spark.emptyDataFrame.write.csv("/tmp/t1") org.apache.spark.sql.AnalysisException: Cannot use all columns for partition columns; ``` After this PR, there occurs no exceptions and the created directory has only one file, `_SUCCESS`, as expected. ## How was this patch tested? Pass the Jenkins tests including updated test cases. Author: Dongjoon Hyun Closes #13730 from dongjoon-hyun/SPARK-16006. (cherry picked from commit 9b1b3ae771babf127f64898d5dc110721597a760) Signed-off-by: Reynold Xin --- .../spark/sql/execution/datasources/PartitioningUtils.scala | 2 +- .../org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 388df7002dc36..c3561099d6842 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -351,7 +351,7 @@ private[sql] object PartitioningUtils { } } - if (partitionColumns.size == schema.fields.size) { + if (partitionColumns.nonEmpty && partitionColumns.size == schema.fields.length) { throw new AnalysisException(s"Cannot use all columns for partition columns") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index ebbcc1d7ffbb5..7308f85a883d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -246,8 +246,9 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be spark.range(10).write.format("parquet").mode("overwrite").partitionBy("id").save(path) } intercept[AnalysisException] { - spark.range(10).write.format("orc").mode("overwrite").partitionBy("id").save(path) + spark.range(10).write.format("csv").mode("overwrite").partitionBy("id").save(path) } + spark.emptyDataFrame.write.format("parquet").mode("overwrite").save(path) } } From c4cebd5725e6d8ade8c0a02652e251d04903da72 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 29 Jun 2016 15:07:32 -0700 Subject: [PATCH 0851/1470] [SPARK-16238] Metrics for generated method and class bytecode size ## What changes were proposed in this pull request? This extends SPARK-15860 to include metrics for the actual bytecode size of janino-generated methods. They can be accessed in the same way as any other codahale metric, e.g. ``` scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getSnapshot().getValues() res7: Array[Long] = Array(532, 532, 532, 542, 1479, 2670, 3585, 3585) scala> org.apache.spark.metrics.source.CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getSnapshot().getValues() res8: Array[Long] = Array(5, 5, 5, 5, 10, 10, 10, 10, 15, 15, 15, 38, 63, 79, 88, 94, 94, 94, 132, 132, 165, 165, 220, 220) ``` ## How was this patch tested? Small unit test, also verified manually that the performance impact is minimal (<10%). hvanhovell Author: Eric Liang Closes #13934 from ericl/spark-16238. (cherry picked from commit 23c58653f900bfb71ef2b3186a95ad2562c33969) Signed-off-by: Reynold Xin --- .../spark/metrics/source/StaticSources.scala | 12 ++++++ .../expressions/codegen/CodeGenerator.scala | 40 ++++++++++++++++++- .../expressions/CodeGenerationSuite.scala | 4 ++ 3 files changed, 55 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala index 6819222e15a13..6bba259acc391 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala @@ -47,4 +47,16 @@ object CodegenMetrics extends Source { * Histogram of the time it took to compile source code text (in milliseconds). */ val METRIC_COMPILATION_TIME = metricRegistry.histogram(MetricRegistry.name("compilationTime")) + + /** + * Histogram of the bytecode size of each class generated by CodeGenerator. + */ + val METRIC_GENERATED_CLASS_BYTECODE_SIZE = + metricRegistry.histogram(MetricRegistry.name("generatedClassSize")) + + /** + * Histogram of the bytecode size of each method in classes generated by CodeGenerator. + */ + val METRIC_GENERATED_METHOD_BYTECODE_SIZE = + metricRegistry.histogram(MetricRegistry.name("generatedMethodSize")) } 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 6392ff42d7099..16fb1f683710f 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 @@ -17,11 +17,16 @@ package org.apache.spark.sql.catalyst.expressions.codegen +import java.io.ByteArrayInputStream +import java.util.{Map => JavaMap} + +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import com.google.common.cache.{CacheBuilder, CacheLoader} -import org.codehaus.janino.ClassBodyEvaluator +import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, SimpleCompiler} +import org.codehaus.janino.util.ClassFile import scala.language.existentials import org.apache.spark.SparkEnv @@ -876,6 +881,7 @@ object CodeGenerator extends Logging { try { evaluator.cook("generated.java", code.body) + recordCompilationStats(evaluator) } catch { case e: Exception => val msg = s"failed to compile: $e\n$formatted" @@ -885,6 +891,38 @@ object CodeGenerator extends Logging { evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass] } + /** + * Records the generated class and method bytecode sizes by inspecting janino private fields. + */ + private def recordCompilationStats(evaluator: ClassBodyEvaluator): Unit = { + // First retrieve the generated classes. + val classes = { + val resultField = classOf[SimpleCompiler].getDeclaredField("result") + resultField.setAccessible(true) + val loader = resultField.get(evaluator).asInstanceOf[ByteArrayClassLoader] + val classesField = loader.getClass.getDeclaredField("classes") + classesField.setAccessible(true) + classesField.get(loader).asInstanceOf[JavaMap[String, Array[Byte]]].asScala + } + + // Then walk the classes to get at the method bytecode. + val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") + val codeAttrField = codeAttr.getDeclaredField("code") + codeAttrField.setAccessible(true) + classes.foreach { case (_, classBytes) => + CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length) + val cf = new ClassFile(new ByteArrayInputStream(classBytes)) + cf.methodInfos.asScala.foreach { method => + method.getAttributes().foreach { a => + if (a.getClass.getName == codeAttr.getName) { + CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update( + codeAttrField.get(a).asInstanceOf[Array[Byte]].length) + } + } + } + } + } + /** * A cache of generated classes. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 60dd03f5d0c13..8ea8f61150844 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -53,9 +53,13 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { test("metrics are recorded on compile") { val startCount1 = CodegenMetrics.METRIC_COMPILATION_TIME.getCount() val startCount2 = CodegenMetrics.METRIC_SOURCE_CODE_SIZE.getCount() + val startCount3 = CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getCount() + val startCount4 = CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getCount() GenerateOrdering.generate(Add(Literal(123), Literal(1)).asc :: Nil) assert(CodegenMetrics.METRIC_COMPILATION_TIME.getCount() == startCount1 + 1) assert(CodegenMetrics.METRIC_SOURCE_CODE_SIZE.getCount() == startCount2 + 1) + assert(CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getCount() > startCount1) + assert(CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getCount() > startCount1) } test("SPARK-8443: split wide projections into blocks due to JVM code size limit") { From 011befd2098bf78979cc8e00de1576bf339583b2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 29 Jun 2016 16:08:10 -0700 Subject: [PATCH 0852/1470] [SPARK-16228][SQL] HiveSessionCatalog should return `double`-param functions for decimal param lookups ## What changes were proposed in this pull request? This PR supports a fallback lookup by casting `DecimalType` into `DoubleType` for the external functions with `double`-type parameter. **Reported Error Scenarios** ```scala scala> sql("select percentile(value, 0.5) from values 1,2,3 T(value)") org.apache.spark.sql.AnalysisException: ... No matching method for class org.apache.hadoop.hive.ql.udf.UDAFPercentile with (int, decimal(38,18)). Possible choices: _FUNC_(bigint, array) _FUNC_(bigint, double) ; line 1 pos 7 scala> sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)") org.apache.spark.sql.AnalysisException: ... Only a float/double or float/double array argument is accepted as parameter 2, but decimal(38,18) was passed instead.; line 1 pos 7 ``` ## How was this patch tested? Pass the Jenkins tests (including a new testcase). Author: Dongjoon Hyun Closes #13930 from dongjoon-hyun/SPARK-16228. (cherry picked from commit 2eaabfa4142d4050be2b45fd277ff5c7fa430581) Signed-off-by: Reynold Xin --- .../spark/sql/hive/HiveSessionCatalog.scala | 16 +++++++++++++++- .../spark/sql/hive/execution/HiveUDFSuite.scala | 7 +++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 8a47dcf908030..2589b9d4a0284 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -30,12 +30,13 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DecimalType, DoubleType} import org.apache.spark.util.Utils @@ -163,6 +164,19 @@ private[sql] class HiveSessionCatalog( } override def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { + try { + lookupFunction0(name, children) + } catch { + case NonFatal(_) => + // SPARK-16228 ExternalCatalog may recognize `double`-type only. + val newChildren = children.map { child => + if (child.dataType.isInstanceOf[DecimalType]) Cast(child, DoubleType) else child + } + lookupFunction0(name, newChildren) + } + } + + private def lookupFunction0(name: FunctionIdentifier, children: Seq[Expression]): Expression = { // TODO: Once lookupFunction accepts a FunctionIdentifier, we should refactor this method to // if (super.functionExists(name)) { // super.lookupFunction(name, children) 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 0f56b2c0d1f43..def4601cf6156 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 @@ -142,6 +142,13 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { sql("SELECT array(max(key), max(key)) FROM src").collect().toSeq) } + test("SPARK-16228 Percentile needs explicit cast to double") { + sql("select percentile(value, cast(0.5 as double)) from values 1,2,3 T(value)") + sql("select percentile_approx(value, cast(0.5 as double)) from values 1.0,2.0,3.0 T(value)") + sql("select percentile(value, 0.5) from values 1,2,3 T(value)") + sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)") + } + test("Generic UDAF aggregates") { checkAnswer(sql("SELECT ceiling(percentile_approx(key, 0.99999D)) FROM src LIMIT 1"), sql("SELECT max(key) FROM src LIMIT 1").collect().toSeq) From 8da4314735ed55f259642e2977d8d7bf2212474f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 30 Jun 2016 08:15:08 +0800 Subject: [PATCH 0853/1470] [SPARK-16134][SQL] optimizer rules for typed filter ## What changes were proposed in this pull request? This PR adds 3 optimizer rules for typed filter: 1. push typed filter down through `SerializeFromObject` and eliminate the deserialization in filter condition. 2. pull typed filter up through `SerializeFromObject` and eliminate the deserialization in filter condition. 3. combine adjacent typed filters and share the deserialized object among all the condition expressions. This PR also adds `TypedFilter` logical plan, to separate it from normal filter, so that the concept is more clear and it's easier to write optimizer rules. ## How was this patch tested? `TypedFilterOptimizationSuite` Author: Wenchen Fan Closes #13846 from cloud-fan/filter. (cherry picked from commit d063898bebaaf4ec2aad24c3ac70aabdbf97a190) Signed-off-by: Cheng Lian --- .../spark/sql/catalyst/dsl/package.scala | 6 +- .../expressions/ReferenceToExpressions.scala | 1 + .../sql/catalyst/optimizer/Optimizer.scala | 98 +++++++++---------- .../sql/catalyst/plans/logical/object.scala | 47 ++++++++- .../TypedFilterOptimizationSuite.scala | 86 ++++++++++++---- .../scala/org/apache/spark/sql/Dataset.scala | 12 +-- .../spark/sql/execution/SparkStrategies.scala | 2 + .../org/apache/spark/sql/QueryTest.scala | 1 + 8 files changed, 162 insertions(+), 91 deletions(-) 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 2ca990d19a2cb..84c9cc8c8e7f7 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,11 +293,7 @@ package object dsl { def where(condition: Expression): LogicalPlan = Filter(condition, logicalPlan) - def filter[T : Encoder](func: T => Boolean): LogicalPlan = { - val deserialized = logicalPlan.deserialize[T] - val condition = expressions.callFunction(func, BooleanType, deserialized.output.head) - Filter(condition, deserialized).serialize[T] - } + def filter[T : Encoder](func: T => Boolean): LogicalPlan = TypedFilter(func, logicalPlan) def serialize[T : Encoder]: LogicalPlan = CatalystSerde.serialize[T](logicalPlan) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala index 502d791c6e85c..127797c0974bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala @@ -45,6 +45,7 @@ case class ReferenceToExpressions(result: Expression, children: Seq[Expression]) var maxOrdinal = -1 result foreach { case b: BoundReference if b.ordinal > maxOrdinal => maxOrdinal = b.ordinal + case _ => } if (maxOrdinal > children.length) { return TypeCheckFailure(s"The result expression need $maxOrdinal input expressions, but " + 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 f24f8b78d476f..aa907356b4e1d 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 @@ -21,6 +21,7 @@ import scala.annotation.tailrec import scala.collection.immutable.HashSet import scala.collection.mutable.ArrayBuffer +import org.apache.spark.api.java.function.FilterFunction import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} @@ -109,8 +110,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: Batch("Typed Filter Optimization", fixedPoint, - EmbedSerializerInFilter, - RemoveAliasOnlyProject) :: + CombineTypedFilters) :: Batch("LocalRelation", fixedPoint, ConvertToLocalRelation) :: Batch("OptimizeCodegen", Once, @@ -205,15 +205,33 @@ object RemoveAliasOnlyProject extends Rule[LogicalPlan] { object EliminateSerialization extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case d @ DeserializeToObject(_, _, s: SerializeFromObject) - if d.outputObjectType == s.inputObjectType => + if d.outputObjAttr.dataType == s.inputObjAttr.dataType => // Adds an extra Project here, to preserve the output expr id of `DeserializeToObject`. // We will remove it later in RemoveAliasOnlyProject rule. - val objAttr = - Alias(s.child.output.head, s.child.output.head.name)(exprId = d.output.head.exprId) + val objAttr = Alias(s.inputObjAttr, s.inputObjAttr.name)(exprId = d.outputObjAttr.exprId) Project(objAttr :: Nil, s.child) + case a @ AppendColumns(_, _, _, s: SerializeFromObject) - if a.deserializer.dataType == s.inputObjectType => + if a.deserializer.dataType == s.inputObjAttr.dataType => AppendColumnsWithObject(a.func, s.serializer, a.serializer, s.child) + + // If there is a `SerializeFromObject` under typed filter and its input object type is same with + // the typed filter's deserializer, we can convert typed filter to normal filter without + // deserialization in condition, and push it down through `SerializeFromObject`. + // e.g. `ds.map(...).filter(...)` can be optimized by this rule to save extra deserialization, + // but `ds.map(...).as[AnotherType].filter(...)` can not be optimized. + case f @ TypedFilter(_, _, s: SerializeFromObject) + if f.deserializer.dataType == s.inputObjAttr.dataType => + s.copy(child = f.withObjectProducerChild(s.child)) + + // If there is a `DeserializeToObject` upon typed filter and its output object type is same with + // the typed filter's deserializer, we can convert typed filter to normal filter without + // deserialization in condition, and pull it up through `DeserializeToObject`. + // e.g. `ds.filter(...).map(...)` can be optimized by this rule to save extra deserialization, + // but `ds.filter(...).as[AnotherType].map(...)` can not be optimized. + case d @ DeserializeToObject(_, _, f: TypedFilter) + if d.outputObjAttr.dataType == f.deserializer.dataType => + f.withObjectProducerChild(d.copy(child = f.child)) } } @@ -1606,54 +1624,30 @@ case class GetCurrentDatabase(sessionCatalog: SessionCatalog) extends Rule[Logic } /** - * Typed [[Filter]] is by default surrounded by a [[DeserializeToObject]] beneath it and a - * [[SerializeFromObject]] above it. If these serializations can't be eliminated, we should embed - * the deserializer in filter condition to save the extra serialization at last. + * Combines two adjacent [[TypedFilter]]s, which operate on same type object in condition, into one, + * mering the filter functions into one conjunctive function. */ -object EmbedSerializerInFilter extends Rule[LogicalPlan] { +object CombineTypedFilters extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) - // SPARK-15632: Conceptually, filter operator should never introduce schema change. This - // optimization rule also relies on this assumption. However, Dataset typed filter operator - // does introduce schema changes in some cases. Thus, we only enable this optimization when - // - // 1. either input and output schemata are exactly the same, or - // 2. both input and output schemata are single-field schema and share the same type. - // - // The 2nd case is included because encoders for primitive types always have only a single - // field with hard-coded field name "value". - // TODO Cleans this up after fixing SPARK-15632. - if s.schema == d.child.schema || samePrimitiveType(s.schema, d.child.schema) => - - val numObjects = condition.collect { - case a: Attribute if a == d.output.head => a - }.length - - if (numObjects > 1) { - // If the filter condition references the object more than one times, we should not embed - // deserializer in it as the deserialization will happen many times and slow down the - // execution. - // TODO: we can still embed it if we can make sure subexpression elimination works here. - s - } else { - val newCondition = condition transform { - case a: Attribute if a == d.output.head => d.deserializer - } - val filter = Filter(newCondition, d.child) - - // Adds an extra Project here, to preserve the output expr id of `SerializeFromObject`. - // We will remove it later in RemoveAliasOnlyProject rule. - val objAttrs = filter.output.zip(s.output).map { case (fout, sout) => - Alias(fout, fout.name)(exprId = sout.exprId) - } - Project(objAttrs, filter) - } - } - - def samePrimitiveType(lhs: StructType, rhs: StructType): Boolean = { - (lhs, rhs) match { - case (StructType(Array(f1)), StructType(Array(f2))) => f1.dataType == f2.dataType - case _ => false + case t1 @ TypedFilter(_, _, t2 @ TypedFilter(_, _, child)) + if t1.deserializer.dataType == t2.deserializer.dataType => + TypedFilter(combineFilterFunction(t2.func, t1.func), t1.deserializer, child) + } + + private def combineFilterFunction(func1: AnyRef, func2: AnyRef): Any => Boolean = { + (func1, func2) match { + case (f1: FilterFunction[_], f2: FilterFunction[_]) => + input => f1.asInstanceOf[FilterFunction[Any]].call(input) && + f2.asInstanceOf[FilterFunction[Any]].call(input) + case (f1: FilterFunction[_], f2) => + input => f1.asInstanceOf[FilterFunction[Any]].call(input) && + f2.asInstanceOf[Any => Boolean](input) + case (f1, f2: FilterFunction[_]) => + input => f1.asInstanceOf[Any => Boolean].apply(input) && + f2.asInstanceOf[FilterFunction[Any]].call(input) + case (f1, f2) => + input => f1.asInstanceOf[Any => Boolean].apply(input) && + f2.asInstanceOf[Any => Boolean].apply(input) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 7beeeb4f04bf0..e1890edcbb110 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.catalyst.plans.logical +import scala.language.existentials + +import org.apache.spark.api.java.function.FilterFunction import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.{Encoder, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.types._ object CatalystSerde { @@ -45,13 +49,11 @@ object CatalystSerde { */ trait ObjectProducer extends LogicalPlan { // The attribute that reference to the single object field this operator outputs. - protected def outputObjAttr: Attribute + def outputObjAttr: Attribute override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) - - def outputObjectType: DataType = outputObjAttr.dataType } /** @@ -64,7 +66,7 @@ trait ObjectConsumer extends UnaryNode { // This operator always need all columns of its child, even it doesn't reference to. override def references: AttributeSet = child.outputSet - def inputObjectType: DataType = child.output.head.dataType + def inputObjAttr: Attribute = child.output.head } /** @@ -167,6 +169,43 @@ case class MapElements( outputObjAttr: Attribute, child: LogicalPlan) extends ObjectConsumer with ObjectProducer +object TypedFilter { + def apply[T : Encoder](func: AnyRef, child: LogicalPlan): TypedFilter = { + TypedFilter(func, UnresolvedDeserializer(encoderFor[T].deserializer), child) + } +} + +/** + * A relation produced by applying `func` to each element of the `child` and filter them by the + * resulting boolean value. + * + * This is logically equal to a normal [[Filter]] operator whose condition expression is decoding + * the input row to object and apply the given function with decoded object. However we need the + * encapsulation of [[TypedFilter]] to make the concept more clear and make it easier to write + * optimizer rules. + */ +case class TypedFilter( + func: AnyRef, + deserializer: Expression, + child: LogicalPlan) extends UnaryNode { + + override def output: Seq[Attribute] = child.output + + def withObjectProducerChild(obj: LogicalPlan): Filter = { + assert(obj.output.length == 1) + Filter(typedCondition(obj.output.head), obj) + } + + def typedCondition(input: Expression): Expression = { + val (funcClass, methodName) = func match { + case m: FilterFunction[_] => classOf[FilterFunction[_]] -> "call" + case _ => classOf[Any => Boolean] -> "apply" + } + val funcObj = Literal.create(func, ObjectType(funcClass)) + Invoke(funcObj, methodName, BooleanType, input :: Nil) + } +} + /** Factory for constructing new `AppendColumn` nodes. */ object AppendColumns { def apply[T : Encoder, U : Encoder]( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala index 63d87bfb6d24d..56f096f3ecf8c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala @@ -19,12 +19,11 @@ package org.apache.spark.sql.catalyst.optimizer import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, TypedFilter} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.types.BooleanType @@ -33,44 +32,91 @@ class TypedFilterOptimizationSuite extends PlanTest { val batches = Batch("EliminateSerialization", FixedPoint(50), EliminateSerialization) :: - Batch("EmbedSerializerInFilter", FixedPoint(50), - EmbedSerializerInFilter) :: Nil + Batch("CombineTypedFilters", FixedPoint(50), + CombineTypedFilters) :: Nil } implicit private def productEncoder[T <: Product : TypeTag] = ExpressionEncoder[T]() - test("back to back filter") { + test("filter after serialize with the same object type") { val input = LocalRelation('_1.int, '_2.int) - val f1 = (i: (Int, Int)) => i._1 > 0 - val f2 = (i: (Int, Int)) => i._2 > 0 + val f = (i: (Int, Int)) => i._1 > 0 - val query = input.filter(f1).filter(f2).analyze + val query = input + .deserialize[(Int, Int)] + .serialize[(Int, Int)] + .filter(f).analyze val optimized = Optimize.execute(query) - val expected = input.deserialize[(Int, Int)] - .where(callFunction(f1, BooleanType, 'obj)) - .select('obj.as("obj")) - .where(callFunction(f2, BooleanType, 'obj)) + val expected = input + .deserialize[(Int, Int)] + .where(callFunction(f, BooleanType, 'obj)) .serialize[(Int, Int)].analyze comparePlans(optimized, expected) } - // TODO: Remove this after we completely fix SPARK-15632 by adding optimization rules - // for typed filters. - ignore("embed deserializer in typed filter condition if there is only one filter") { + test("filter after serialize with different object types") { + val input = LocalRelation('_1.int, '_2.int) + val f = (i: OtherTuple) => i._1 > 0 + + val query = input + .deserialize[(Int, Int)] + .serialize[(Int, Int)] + .filter(f).analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } + + test("filter before deserialize with the same object type") { val input = LocalRelation('_1.int, '_2.int) val f = (i: (Int, Int)) => i._1 > 0 - val query = input.filter(f).analyze + val query = input + .filter(f) + .deserialize[(Int, Int)] + .serialize[(Int, Int)].analyze val optimized = Optimize.execute(query) - val deserializer = UnresolvedDeserializer(encoderFor[(Int, Int)].deserializer) - val condition = callFunction(f, BooleanType, deserializer) - val expected = input.where(condition).select('_1.as("_1"), '_2.as("_2")).analyze + val expected = input + .deserialize[(Int, Int)] + .where(callFunction(f, BooleanType, 'obj)) + .serialize[(Int, Int)].analyze comparePlans(optimized, expected) } + + test("filter before deserialize with different object types") { + val input = LocalRelation('_1.int, '_2.int) + val f = (i: OtherTuple) => i._1 > 0 + + val query = input + .filter(f) + .deserialize[(Int, Int)] + .serialize[(Int, Int)].analyze + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } + + test("back to back filter with the same object type") { + val input = LocalRelation('_1.int, '_2.int) + val f1 = (i: (Int, Int)) => i._1 > 0 + val f2 = (i: (Int, Int)) => i._2 > 0 + + val query = input.filter(f1).filter(f2).analyze + val optimized = Optimize.execute(query) + assert(optimized.collect { case t: TypedFilter => t }.length == 1) + } + + test("back to back filter with different object types") { + val input = LocalRelation('_1.int, '_2.int) + val f1 = (i: (Int, Int)) => i._1 > 0 + val f2 = (i: OtherTuple) => i._2 > 0 + + val query = input.filter(f1).filter(f2).analyze + val optimized = Optimize.execute(query) + assert(optimized.collect { case t: TypedFilter => t }.length == 2) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 067cbec4bf618..8e914fcb9d728 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1964,11 +1964,7 @@ class Dataset[T] private[sql]( */ @Experimental def filter(func: T => Boolean): Dataset[T] = { - val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer) - val function = Literal.create(func, ObjectType(classOf[T => Boolean])) - val condition = Invoke(function, "apply", BooleanType, deserializer :: Nil) - val filter = Filter(condition, logicalPlan) - withTypedPlan(filter) + withTypedPlan(TypedFilter(func, logicalPlan)) } /** @@ -1981,11 +1977,7 @@ class Dataset[T] private[sql]( */ @Experimental def filter(func: FilterFunction[T]): Dataset[T] = { - val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer) - val function = Literal.create(func, ObjectType(classOf[FilterFunction[T]])) - val condition = Invoke(function, "call", BooleanType, deserializer :: Nil) - val filter = Filter(condition, logicalPlan) - withTypedPlan(filter) + withTypedPlan(TypedFilter(func, logicalPlan)) } /** 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 b619d4edc30de..5e643ea75a16b 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 @@ -385,6 +385,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ProjectExec(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.FilterExec(condition, planLater(child)) :: Nil + case f: logical.TypedFilter => + execution.FilterExec(f.typedCondition(f.deserializer), planLater(f.child)) :: Nil case e @ logical.Expand(_, _, child) => execution.ExpandExec(e.projections, e.output, planLater(child)) :: Nil case logical.Window(windowExprs, partitionSpec, orderSpec, child) => 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 b15f38c2a71e2..ab505139a860a 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 @@ -238,6 +238,7 @@ abstract class QueryTest extends PlanTest { case _: ObjectConsumer => return case _: ObjectProducer => return case _: AppendColumns => return + case _: TypedFilter => return case _: LogicalRelation => return case p if p.getClass.getSimpleName == "MetastoreRelation" => return case _: MemoryPlan => return From e1bdf1e02483bf513b6e012e8921d440a5efbc11 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 30 Jun 2016 08:17:43 +0800 Subject: [PATCH 0854/1470] Revert "[SPARK-16134][SQL] optimizer rules for typed filter" This reverts commit 8da4314735ed55f259642e2977d8d7bf2212474f. --- .../spark/sql/catalyst/dsl/package.scala | 6 +- .../expressions/ReferenceToExpressions.scala | 1 - .../sql/catalyst/optimizer/Optimizer.scala | 98 ++++++++++--------- .../sql/catalyst/plans/logical/object.scala | 47 +-------- .../TypedFilterOptimizationSuite.scala | 86 ++++------------ .../scala/org/apache/spark/sql/Dataset.scala | 12 ++- .../spark/sql/execution/SparkStrategies.scala | 2 - .../org/apache/spark/sql/QueryTest.scala | 1 - 8 files changed, 91 insertions(+), 162 deletions(-) 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 84c9cc8c8e7f7..2ca990d19a2cb 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,11 @@ package object dsl { def where(condition: Expression): LogicalPlan = Filter(condition, logicalPlan) - def filter[T : Encoder](func: T => Boolean): LogicalPlan = TypedFilter(func, logicalPlan) + def filter[T : Encoder](func: T => Boolean): LogicalPlan = { + val deserialized = logicalPlan.deserialize[T] + val condition = expressions.callFunction(func, BooleanType, deserialized.output.head) + Filter(condition, deserialized).serialize[T] + } def serialize[T : Encoder]: LogicalPlan = CatalystSerde.serialize[T](logicalPlan) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala index 127797c0974bb..502d791c6e85c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala @@ -45,7 +45,6 @@ case class ReferenceToExpressions(result: Expression, children: Seq[Expression]) var maxOrdinal = -1 result foreach { case b: BoundReference if b.ordinal > maxOrdinal => maxOrdinal = b.ordinal - case _ => } if (maxOrdinal > children.length) { return TypeCheckFailure(s"The result expression need $maxOrdinal input expressions, but " + 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 aa907356b4e1d..f24f8b78d476f 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 @@ -21,7 +21,6 @@ import scala.annotation.tailrec import scala.collection.immutable.HashSet import scala.collection.mutable.ArrayBuffer -import org.apache.spark.api.java.function.FilterFunction import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} @@ -110,7 +109,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("Decimal Optimizations", fixedPoint, DecimalAggregates) :: Batch("Typed Filter Optimization", fixedPoint, - CombineTypedFilters) :: + EmbedSerializerInFilter, + RemoveAliasOnlyProject) :: Batch("LocalRelation", fixedPoint, ConvertToLocalRelation) :: Batch("OptimizeCodegen", Once, @@ -205,33 +205,15 @@ object RemoveAliasOnlyProject extends Rule[LogicalPlan] { object EliminateSerialization extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case d @ DeserializeToObject(_, _, s: SerializeFromObject) - if d.outputObjAttr.dataType == s.inputObjAttr.dataType => + if d.outputObjectType == s.inputObjectType => // Adds an extra Project here, to preserve the output expr id of `DeserializeToObject`. // We will remove it later in RemoveAliasOnlyProject rule. - val objAttr = Alias(s.inputObjAttr, s.inputObjAttr.name)(exprId = d.outputObjAttr.exprId) + val objAttr = + Alias(s.child.output.head, s.child.output.head.name)(exprId = d.output.head.exprId) Project(objAttr :: Nil, s.child) - case a @ AppendColumns(_, _, _, s: SerializeFromObject) - if a.deserializer.dataType == s.inputObjAttr.dataType => + if a.deserializer.dataType == s.inputObjectType => AppendColumnsWithObject(a.func, s.serializer, a.serializer, s.child) - - // If there is a `SerializeFromObject` under typed filter and its input object type is same with - // the typed filter's deserializer, we can convert typed filter to normal filter without - // deserialization in condition, and push it down through `SerializeFromObject`. - // e.g. `ds.map(...).filter(...)` can be optimized by this rule to save extra deserialization, - // but `ds.map(...).as[AnotherType].filter(...)` can not be optimized. - case f @ TypedFilter(_, _, s: SerializeFromObject) - if f.deserializer.dataType == s.inputObjAttr.dataType => - s.copy(child = f.withObjectProducerChild(s.child)) - - // If there is a `DeserializeToObject` upon typed filter and its output object type is same with - // the typed filter's deserializer, we can convert typed filter to normal filter without - // deserialization in condition, and pull it up through `DeserializeToObject`. - // e.g. `ds.filter(...).map(...)` can be optimized by this rule to save extra deserialization, - // but `ds.filter(...).as[AnotherType].map(...)` can not be optimized. - case d @ DeserializeToObject(_, _, f: TypedFilter) - if d.outputObjAttr.dataType == f.deserializer.dataType => - f.withObjectProducerChild(d.copy(child = f.child)) } } @@ -1624,30 +1606,54 @@ case class GetCurrentDatabase(sessionCatalog: SessionCatalog) extends Rule[Logic } /** - * Combines two adjacent [[TypedFilter]]s, which operate on same type object in condition, into one, - * mering the filter functions into one conjunctive function. + * Typed [[Filter]] is by default surrounded by a [[DeserializeToObject]] beneath it and a + * [[SerializeFromObject]] above it. If these serializations can't be eliminated, we should embed + * the deserializer in filter condition to save the extra serialization at last. */ -object CombineTypedFilters extends Rule[LogicalPlan] { +object EmbedSerializerInFilter extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case t1 @ TypedFilter(_, _, t2 @ TypedFilter(_, _, child)) - if t1.deserializer.dataType == t2.deserializer.dataType => - TypedFilter(combineFilterFunction(t2.func, t1.func), t1.deserializer, child) - } - - private def combineFilterFunction(func1: AnyRef, func2: AnyRef): Any => Boolean = { - (func1, func2) match { - case (f1: FilterFunction[_], f2: FilterFunction[_]) => - input => f1.asInstanceOf[FilterFunction[Any]].call(input) && - f2.asInstanceOf[FilterFunction[Any]].call(input) - case (f1: FilterFunction[_], f2) => - input => f1.asInstanceOf[FilterFunction[Any]].call(input) && - f2.asInstanceOf[Any => Boolean](input) - case (f1, f2: FilterFunction[_]) => - input => f1.asInstanceOf[Any => Boolean].apply(input) && - f2.asInstanceOf[FilterFunction[Any]].call(input) - case (f1, f2) => - input => f1.asInstanceOf[Any => Boolean].apply(input) && - f2.asInstanceOf[Any => Boolean].apply(input) + case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) + // SPARK-15632: Conceptually, filter operator should never introduce schema change. This + // optimization rule also relies on this assumption. However, Dataset typed filter operator + // does introduce schema changes in some cases. Thus, we only enable this optimization when + // + // 1. either input and output schemata are exactly the same, or + // 2. both input and output schemata are single-field schema and share the same type. + // + // The 2nd case is included because encoders for primitive types always have only a single + // field with hard-coded field name "value". + // TODO Cleans this up after fixing SPARK-15632. + if s.schema == d.child.schema || samePrimitiveType(s.schema, d.child.schema) => + + val numObjects = condition.collect { + case a: Attribute if a == d.output.head => a + }.length + + if (numObjects > 1) { + // If the filter condition references the object more than one times, we should not embed + // deserializer in it as the deserialization will happen many times and slow down the + // execution. + // TODO: we can still embed it if we can make sure subexpression elimination works here. + s + } else { + val newCondition = condition transform { + case a: Attribute if a == d.output.head => d.deserializer + } + val filter = Filter(newCondition, d.child) + + // Adds an extra Project here, to preserve the output expr id of `SerializeFromObject`. + // We will remove it later in RemoveAliasOnlyProject rule. + val objAttrs = filter.output.zip(s.output).map { case (fout, sout) => + Alias(fout, fout.name)(exprId = sout.exprId) + } + Project(objAttrs, filter) + } + } + + def samePrimitiveType(lhs: StructType, rhs: StructType): Boolean = { + (lhs, rhs) match { + case (StructType(Array(f1)), StructType(Array(f2))) => f1.dataType == f2.dataType + case _ => false } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index e1890edcbb110..7beeeb4f04bf0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -17,15 +17,11 @@ package org.apache.spark.sql.catalyst.plans.logical -import scala.language.existentials - -import org.apache.spark.api.java.function.FilterFunction import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.{Encoder, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.types._ object CatalystSerde { @@ -49,11 +45,13 @@ object CatalystSerde { */ trait ObjectProducer extends LogicalPlan { // The attribute that reference to the single object field this operator outputs. - def outputObjAttr: Attribute + protected def outputObjAttr: Attribute override def output: Seq[Attribute] = outputObjAttr :: Nil override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) + + def outputObjectType: DataType = outputObjAttr.dataType } /** @@ -66,7 +64,7 @@ trait ObjectConsumer extends UnaryNode { // This operator always need all columns of its child, even it doesn't reference to. override def references: AttributeSet = child.outputSet - def inputObjAttr: Attribute = child.output.head + def inputObjectType: DataType = child.output.head.dataType } /** @@ -169,43 +167,6 @@ case class MapElements( outputObjAttr: Attribute, child: LogicalPlan) extends ObjectConsumer with ObjectProducer -object TypedFilter { - def apply[T : Encoder](func: AnyRef, child: LogicalPlan): TypedFilter = { - TypedFilter(func, UnresolvedDeserializer(encoderFor[T].deserializer), child) - } -} - -/** - * A relation produced by applying `func` to each element of the `child` and filter them by the - * resulting boolean value. - * - * This is logically equal to a normal [[Filter]] operator whose condition expression is decoding - * the input row to object and apply the given function with decoded object. However we need the - * encapsulation of [[TypedFilter]] to make the concept more clear and make it easier to write - * optimizer rules. - */ -case class TypedFilter( - func: AnyRef, - deserializer: Expression, - child: LogicalPlan) extends UnaryNode { - - override def output: Seq[Attribute] = child.output - - def withObjectProducerChild(obj: LogicalPlan): Filter = { - assert(obj.output.length == 1) - Filter(typedCondition(obj.output.head), obj) - } - - def typedCondition(input: Expression): Expression = { - val (funcClass, methodName) = func match { - case m: FilterFunction[_] => classOf[FilterFunction[_]] -> "call" - case _ => classOf[Any => Boolean] -> "apply" - } - val funcObj = Literal.create(func, ObjectType(funcClass)) - Invoke(funcObj, methodName, BooleanType, input :: Nil) - } -} - /** Factory for constructing new `AppendColumn` nodes. */ object AppendColumns { def apply[T : Encoder, U : Encoder]( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala index 56f096f3ecf8c..63d87bfb6d24d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TypedFilterOptimizationSuite.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.catalyst.optimizer import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, TypedFilter} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.types.BooleanType @@ -32,91 +33,44 @@ class TypedFilterOptimizationSuite extends PlanTest { val batches = Batch("EliminateSerialization", FixedPoint(50), EliminateSerialization) :: - Batch("CombineTypedFilters", FixedPoint(50), - CombineTypedFilters) :: Nil + Batch("EmbedSerializerInFilter", FixedPoint(50), + EmbedSerializerInFilter) :: Nil } implicit private def productEncoder[T <: Product : TypeTag] = ExpressionEncoder[T]() - test("filter after serialize with the same object type") { + test("back to back filter") { val input = LocalRelation('_1.int, '_2.int) - val f = (i: (Int, Int)) => i._1 > 0 + val f1 = (i: (Int, Int)) => i._1 > 0 + val f2 = (i: (Int, Int)) => i._2 > 0 - val query = input - .deserialize[(Int, Int)] - .serialize[(Int, Int)] - .filter(f).analyze + val query = input.filter(f1).filter(f2).analyze val optimized = Optimize.execute(query) - val expected = input - .deserialize[(Int, Int)] - .where(callFunction(f, BooleanType, 'obj)) + val expected = input.deserialize[(Int, Int)] + .where(callFunction(f1, BooleanType, 'obj)) + .select('obj.as("obj")) + .where(callFunction(f2, BooleanType, 'obj)) .serialize[(Int, Int)].analyze comparePlans(optimized, expected) } - test("filter after serialize with different object types") { - val input = LocalRelation('_1.int, '_2.int) - val f = (i: OtherTuple) => i._1 > 0 - - val query = input - .deserialize[(Int, Int)] - .serialize[(Int, Int)] - .filter(f).analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, query) - } - - test("filter before deserialize with the same object type") { + // TODO: Remove this after we completely fix SPARK-15632 by adding optimization rules + // for typed filters. + ignore("embed deserializer in typed filter condition if there is only one filter") { val input = LocalRelation('_1.int, '_2.int) val f = (i: (Int, Int)) => i._1 > 0 - val query = input - .filter(f) - .deserialize[(Int, Int)] - .serialize[(Int, Int)].analyze + val query = input.filter(f).analyze val optimized = Optimize.execute(query) - val expected = input - .deserialize[(Int, Int)] - .where(callFunction(f, BooleanType, 'obj)) - .serialize[(Int, Int)].analyze + val deserializer = UnresolvedDeserializer(encoderFor[(Int, Int)].deserializer) + val condition = callFunction(f, BooleanType, deserializer) + val expected = input.where(condition).select('_1.as("_1"), '_2.as("_2")).analyze comparePlans(optimized, expected) } - - test("filter before deserialize with different object types") { - val input = LocalRelation('_1.int, '_2.int) - val f = (i: OtherTuple) => i._1 > 0 - - val query = input - .filter(f) - .deserialize[(Int, Int)] - .serialize[(Int, Int)].analyze - val optimized = Optimize.execute(query) - comparePlans(optimized, query) - } - - test("back to back filter with the same object type") { - val input = LocalRelation('_1.int, '_2.int) - val f1 = (i: (Int, Int)) => i._1 > 0 - val f2 = (i: (Int, Int)) => i._2 > 0 - - val query = input.filter(f1).filter(f2).analyze - val optimized = Optimize.execute(query) - assert(optimized.collect { case t: TypedFilter => t }.length == 1) - } - - test("back to back filter with different object types") { - val input = LocalRelation('_1.int, '_2.int) - val f1 = (i: (Int, Int)) => i._1 > 0 - val f2 = (i: OtherTuple) => i._2 > 0 - - val query = input.filter(f1).filter(f2).analyze - val optimized = Optimize.execute(query) - assert(optimized.collect { case t: TypedFilter => t }.length == 2) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 8e914fcb9d728..067cbec4bf618 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1964,7 +1964,11 @@ class Dataset[T] private[sql]( */ @Experimental def filter(func: T => Boolean): Dataset[T] = { - withTypedPlan(TypedFilter(func, logicalPlan)) + val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer) + val function = Literal.create(func, ObjectType(classOf[T => Boolean])) + val condition = Invoke(function, "apply", BooleanType, deserializer :: Nil) + val filter = Filter(condition, logicalPlan) + withTypedPlan(filter) } /** @@ -1977,7 +1981,11 @@ class Dataset[T] private[sql]( */ @Experimental def filter(func: FilterFunction[T]): Dataset[T] = { - withTypedPlan(TypedFilter(func, logicalPlan)) + val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer) + val function = Literal.create(func, ObjectType(classOf[FilterFunction[T]])) + val condition = Invoke(function, "call", BooleanType, deserializer :: Nil) + val filter = Filter(condition, logicalPlan) + withTypedPlan(filter) } /** 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 5e643ea75a16b..b619d4edc30de 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 @@ -385,8 +385,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ProjectExec(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => execution.FilterExec(condition, planLater(child)) :: Nil - case f: logical.TypedFilter => - execution.FilterExec(f.typedCondition(f.deserializer), planLater(f.child)) :: Nil case e @ logical.Expand(_, _, child) => execution.ExpandExec(e.projections, e.output, planLater(child)) :: Nil case logical.Window(windowExprs, partitionSpec, orderSpec, child) => 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 ab505139a860a..b15f38c2a71e2 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 @@ -238,7 +238,6 @@ abstract class QueryTest extends PlanTest { case _: ObjectConsumer => return case _: ObjectProducer => return case _: AppendColumns => return - case _: TypedFilter => return case _: LogicalRelation => return case p if p.getClass.getSimpleName == "MetastoreRelation" => return case _: MemoryPlan => return From b52bd8070dc852b419283f8a14595e42c179d3d0 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 29 Jun 2016 17:29:17 -0700 Subject: [PATCH 0855/1470] [SPARK-16267][TEST] Replace deprecated `CREATE TEMPORARY TABLE ... USING` from testsuites. ## What changes were proposed in this pull request? After SPARK-15674, `DDLStrategy` prints out the following deprecation messages in the testsuites. ``` 12:10:53.284 WARN org.apache.spark.sql.execution.SparkStrategies$DDLStrategy: CREATE TEMPORARY TABLE normal_orc_source USING... is deprecated, please use CREATE TEMPORARY VIEW viewName USING... instead ``` Total : 40 - JDBCWriteSuite: 14 - DDLSuite: 6 - TableScanSuite: 6 - ParquetSourceSuite: 5 - OrcSourceSuite: 2 - SQLQuerySuite: 2 - HiveCommandSuite: 2 - JsonSuite: 1 - PrunedScanSuite: 1 - FilteredScanSuite 1 This PR replaces `CREATE TEMPORARY TABLE` with `CREATE TEMPORARY VIEW` in order to remove the deprecation messages in the above testsuites except `DDLSuite`, `SQLQuerySuite`, `HiveCommandSuite`. The Jenkins results shows only remaining 10 messages. https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61422/consoleFull ## How was this patch tested? This is a testsuite-only change. Author: Dongjoon Hyun Closes #13956 from dongjoon-hyun/SPARK-16267. (cherry picked from commit 831a04f5d152d1839c0edfdf65bb728aa5957f16) Signed-off-by: Reynold Xin --- .../sql/execution/datasources/json/JsonSuite.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCWriteSuite.scala | 4 ++-- .../apache/spark/sql/sources/FilteredScanSuite.scala | 2 +- .../apache/spark/sql/sources/PrunedScanSuite.scala | 2 +- .../apache/spark/sql/sources/TableScanSuite.scala | 12 ++++++------ .../apache/spark/sql/hive/orc/OrcSourceSuite.scala | 4 ++-- .../org/apache/spark/sql/hive/parquetSuites.scala | 10 +++++----- 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 9f35c02d48762..6c72019702c3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -847,7 +847,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { sql( s""" - |CREATE TEMPORARY TABLE jsonTableSQL + |CREATE TEMPORARY VIEW jsonTableSQL |USING org.apache.spark.sql.json |OPTIONS ( | path '$path' diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 48fa5f98223bc..ff66f53fcf594 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -57,14 +57,14 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { sql( s""" - |CREATE TEMPORARY TABLE PEOPLE + |CREATE OR REPLACE TEMPORARY VIEW PEOPLE |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url1', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) sql( s""" - |CREATE TEMPORARY TABLE PEOPLE1 + |CREATE OR REPLACE TEMPORARY VIEW PEOPLE1 |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url1', dbtable 'TEST.PEOPLE1', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) 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 45e737f5ed047..be56c964a18f8 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 @@ -139,7 +139,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic super.beforeAll() sql( """ - |CREATE TEMPORARY TABLE oneToTenFiltered + |CREATE TEMPORARY VIEW oneToTenFiltered |USING org.apache.spark.sql.sources.FilteredScanSource |OPTIONS ( | from '1', 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 207f89d3eaea0..fb6123d1cc4b5 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 @@ -62,7 +62,7 @@ class PrunedScanSuite extends DataSourceTest with SharedSQLContext { super.beforeAll() sql( """ - |CREATE TEMPORARY TABLE oneToTenPruned + |CREATE TEMPORARY VIEW oneToTenPruned |USING org.apache.spark.sql.sources.PrunedScanSource |OPTIONS ( | from '1', 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 93116d84ced71..0fa0706a10b1b 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 @@ -137,7 +137,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { super.beforeAll() sql( """ - |CREATE TEMPORARY TABLE oneToTen + |CREATE TEMPORARY VIEW oneToTen |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( | From '1', @@ -149,7 +149,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sql( """ - |CREATE TEMPORARY TABLE tableWithSchema ( + |CREATE TEMPORARY VIEW tableWithSchema ( |`string$%Field` stRIng, |binaryField binary, |`booleanField` boolean, @@ -332,7 +332,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { test("defaultSource") { sql( """ - |CREATE TEMPORARY TABLE oneToTenDef + |CREATE TEMPORARY VIEW oneToTenDef |USING org.apache.spark.sql.sources |OPTIONS ( | from '1', @@ -351,7 +351,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { val schemaNotAllowed = intercept[Exception] { sql( """ - |CREATE TEMPORARY TABLE relationProvierWithSchema (i int) + |CREATE TEMPORARY VIEW relationProvierWithSchema (i int) |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( | From '1', @@ -364,7 +364,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { val schemaNeeded = intercept[Exception] { sql( """ - |CREATE TEMPORARY TABLE schemaRelationProvierWithoutSchema + |CREATE TEMPORARY VIEW schemaRelationProvierWithoutSchema |USING org.apache.spark.sql.sources.AllDataTypesScanSource |OPTIONS ( | From '1', @@ -378,7 +378,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { test("SPARK-5196 schema field with comment") { sql( """ - |CREATE TEMPORARY TABLE student(name string comment "SN", age int comment "SA", grade int) + |CREATE TEMPORARY VIEW student(name string comment "SN", age int comment "SA", grade int) |USING org.apache.spark.sql.sources.AllDataTypesScanSource |OPTIONS ( | from '1', diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala index 871b9e02eb382..0f37cd7bf3652 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala @@ -153,7 +153,7 @@ class OrcSourceSuite extends OrcSuite { super.beforeAll() spark.sql( - s"""CREATE TEMPORARY TABLE normal_orc_source + s"""CREATE TEMPORARY VIEW normal_orc_source |USING org.apache.spark.sql.hive.orc |OPTIONS ( | PATH '${new File(orcTableAsDir.getAbsolutePath).getCanonicalPath}' @@ -161,7 +161,7 @@ class OrcSourceSuite extends OrcSuite { """.stripMargin) spark.sql( - s"""CREATE TEMPORARY TABLE normal_orc_as_source + s"""CREATE TEMPORARY VIEW normal_orc_as_source |USING org.apache.spark.sql.hive.orc |OPTIONS ( | PATH '${new File(orcTableAsDir.getAbsolutePath).getCanonicalPath}' 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 6af9976ea0b8f..fe7253d7354d5 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 @@ -582,7 +582,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { "normal_parquet") sql( s""" - create temporary table partitioned_parquet + CREATE TEMPORARY VIEW partitioned_parquet USING org.apache.spark.sql.parquet OPTIONS ( path '${partitionedTableDir.getCanonicalPath}' @@ -590,7 +590,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { """) sql( s""" - create temporary table partitioned_parquet_with_key + CREATE TEMPORARY VIEW partitioned_parquet_with_key USING org.apache.spark.sql.parquet OPTIONS ( path '${partitionedTableDirWithKey.getCanonicalPath}' @@ -598,7 +598,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { """) sql( s""" - create temporary table normal_parquet + CREATE TEMPORARY VIEW normal_parquet USING org.apache.spark.sql.parquet OPTIONS ( path '${new File(partitionedTableDir, "p=1").getCanonicalPath}' @@ -606,7 +606,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { """) sql( s""" - CREATE TEMPORARY TABLE partitioned_parquet_with_key_and_complextypes + CREATE TEMPORARY VIEW partitioned_parquet_with_key_and_complextypes USING org.apache.spark.sql.parquet OPTIONS ( path '${partitionedTableDirWithKeyAndComplexTypes.getCanonicalPath}' @@ -614,7 +614,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { """) sql( s""" - CREATE TEMPORARY TABLE partitioned_parquet_with_complextypes + CREATE TEMPORARY VIEW partitioned_parquet_with_complextypes USING org.apache.spark.sql.parquet OPTIONS ( path '${partitionedTableDirWithComplexTypes.getCanonicalPath}' From a54852350346cacae61d851d796bc3a7abd3a048 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 29 Jun 2016 22:50:53 -0700 Subject: [PATCH 0856/1470] [SPARK-16294][SQL] Labelling support for the include_example Jekyll plugin ## What changes were proposed in this pull request? This PR adds labelling support for the `include_example` Jekyll plugin, so that we may split a single source file into multiple line blocks with different labels, and include them in multiple code snippets in the generated HTML page. ## How was this patch tested? Manually tested. screenshot at jun 29 19-53-21 Author: Cheng Lian Closes #13972 from liancheng/include-example-with-labels. (cherry picked from commit bde1d6a61593aeb62370f526542cead94919b0c0) Signed-off-by: Xiangrui Meng --- docs/_plugins/include_example.rb | 25 ++++++++--- docs/sql-programming-guide.md | 41 +++---------------- .../spark/examples/sql/JavaSparkSQL.java | 5 +++ examples/src/main/python/sql.py | 5 +++ .../spark/examples/sql/RDDRelation.scala | 10 ++++- 5 files changed, 43 insertions(+), 43 deletions(-) diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb index f7485826a762d..306888801df21 100644 --- a/docs/_plugins/include_example.rb +++ b/docs/_plugins/include_example.rb @@ -32,8 +32,18 @@ def render(context) @code_dir = File.join(site.source, config_dir) clean_markup = @markup.strip - @file = File.join(@code_dir, clean_markup) - @lang = clean_markup.split('.').last + + parts = clean_markup.strip.split(' ') + if parts.length > 1 then + @snippet_label = ':' + parts[0] + snippet_file = parts[1] + else + @snippet_label = '' + snippet_file = parts[0] + end + + @file = File.join(@code_dir, snippet_file) + @lang = snippet_file.split('.').last code = File.open(@file).read.encode("UTF-8") code = select_lines(code) @@ -41,7 +51,7 @@ def render(context) rendered_code = Pygments.highlight(code, :lexer => @lang) hint = "
    Find full example code at " \ - "\"examples/src/main/#{clean_markup}\" in the Spark repo.
    " + "\"examples/src/main/#{snippet_file}\" in the Spark repo.
    " rendered_code + hint end @@ -66,13 +76,13 @@ def select_lines(code) # Select the array of start labels from code. startIndices = lines .each_with_index - .select { |l, i| l.include? "$example on$" } + .select { |l, i| l.include? "$example on#{@snippet_label}$" } .map { |l, i| i } # Select the array of end labels from code. endIndices = lines .each_with_index - .select { |l, i| l.include? "$example off$" } + .select { |l, i| l.include? "$example off#{@snippet_label}$" } .map { |l, i| i } raise "Start indices amount is not equal to end indices amount, see #{@file}." \ @@ -92,7 +102,10 @@ def select_lines(code) if start == endline lastIndex = endline range = Range.new(start + 1, endline - 1) - result += trim_codeblock(lines[range]).join + trimmed = trim_codeblock(lines[range]) + # Filter out possible example tags of overlapped labels. + taggs_filtered = trimmed.select { |l| !l.include? '$example ' } + result += taggs_filtered.join result += "\n" end result diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6c6bc8db6a1ff..68419e1331594 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -63,52 +63,23 @@ Throughout this document, we will often refer to Scala/Java Datasets of `Row`s a
    -The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.build()`: - -{% highlight scala %} -import org.apache.spark.sql.SparkSession - -val spark = SparkSession.build() - .master("local") - .appName("Word Count") - .config("spark.some.config.option", "some-value") - .getOrCreate() - -// this is used to implicitly convert an RDD to a DataFrame. -import spark.implicits._ -{% endhighlight %} +The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: +{% include_example init_session scala/org/apache/spark/examples/sql/RDDRelation.scala %}
    -The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.build()`: +The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: -{% highlight java %} -import org.apache.spark.sql.SparkSession - -SparkSession spark = SparkSession.build() - .master("local") - .appName("Word Count") - .config("spark.some.config.option", "some-value") - .getOrCreate(); -{% endhighlight %} +{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQL.java %}
    -The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.build`: - -{% highlight python %} -from pyspark.sql import SparkSession - -spark = SparkSession.build \ - .master("local") \ - .appName("Word Count") \ - .config("spark.some.config.option", "some-value") \ - .getOrCreate() -{% endhighlight %} +The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: +{% include_example init_session python/sql.py %}
    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 e512979ac71b0..7fc6c007b6843 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 @@ -26,7 +26,9 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +// $example on:init_session$ import org.apache.spark.sql.SparkSession; +// $example off:init_session$ public class JavaSparkSQL { public static class Person implements Serializable { @@ -51,10 +53,13 @@ public void setAge(int age) { } public static void main(String[] args) throws Exception { + // $example on:init_session$ SparkSession spark = SparkSession .builder() .appName("JavaSparkSQL") + .config("spark.some.config.option", "some-value") .getOrCreate(); + // $example off:init_session$ System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index ac7246938d3b4..ea11d2c4c7b33 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -20,15 +20,20 @@ import os import sys +# $example on:init_session$ from pyspark.sql import SparkSession +# $example off:init_session$ from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType if __name__ == "__main__": + # $example on:init_session$ spark = SparkSession\ .builder\ .appName("PythonSQL")\ + .config("spark.some.config.option", "some-value")\ .getOrCreate() + # $example off:init_session$ # A list of Rows. Infer schema from the first row, create a DataFrame and print the schema rows = [Row(name="John", age=19), Row(name="Smith", age=23), Row(name="Sarah", age=18)] diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 1b019fbb51771..deaa9f252b9b0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -18,7 +18,10 @@ // scalastyle:off println package org.apache.spark.examples.sql -import org.apache.spark.sql.{SaveMode, SparkSession} +import org.apache.spark.sql.SaveMode +// $example on:init_session$ +import org.apache.spark.sql.SparkSession +// $example off:init_session$ // One method for defining the schema of an RDD is to make a case class with the desired column // names and types. @@ -26,13 +29,16 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { + // $example on:init_session$ val spark = SparkSession .builder - .appName("RDDRelation") + .appName("Spark Examples") + .config("spark.some.config.option", "some-value") .getOrCreate() // Importing the SparkSession gives access to all the SQL functions and implicit conversions. import spark.implicits._ + // $example off:init_session$ val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) // Any RDD containing case classes can be used to create a temporary view. The schema of the From 3134f116a3565c3a299fa2e7094acd7304d64280 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 29 Jun 2016 23:21:03 -0700 Subject: [PATCH 0857/1470] [SPARK-12177][STREAMING][KAFKA] Update KafkaDStreams to new Kafka 0.10 Consumer API ## What changes were proposed in this pull request? New Kafka consumer api for the released 0.10 version of Kafka ## How was this patch tested? Unit tests, manual tests Author: cody koeninger Closes #11863 from koeninger/kafka-0.9. (cherry picked from commit dedbceec1ef33ccd88101016de969a1ef3e3e142) Signed-off-by: Tathagata Das --- external/kafka-0-10-assembly/pom.xml | 176 +++++ external/kafka-0-10/pom.xml | 98 +++ .../kafka010/CachedKafkaConsumer.scala | 189 ++++++ .../streaming/kafka010/ConsumerStrategy.scala | 314 +++++++++ .../kafka010/DirectKafkaInputDStream.scala | 318 +++++++++ .../spark/streaming/kafka010/KafkaRDD.scala | 232 +++++++ .../kafka010/KafkaRDDPartition.scala | 45 ++ .../streaming/kafka010/KafkaTestUtils.scala | 277 ++++++++ .../spark/streaming/kafka010/KafkaUtils.scala | 175 +++++ .../streaming/kafka010/LocationStrategy.scala | 77 +++ .../streaming/kafka010/OffsetRange.scala | 153 +++++ .../streaming/kafka010/package-info.java | 21 + .../spark/streaming/kafka010/package.scala | 23 + .../kafka010/JavaConsumerStrategySuite.java | 84 +++ .../kafka010/JavaDirectKafkaStreamSuite.java | 180 ++++++ .../streaming/kafka010/JavaKafkaRDDSuite.java | 122 ++++ .../kafka010/JavaLocationStrategySuite.java | 58 ++ .../src/test/resources/log4j.properties | 28 + .../kafka010/DirectKafkaStreamSuite.scala | 612 ++++++++++++++++++ .../streaming/kafka010/KafkaRDDSuite.scala | 169 +++++ pom.xml | 2 + project/SparkBuild.scala | 12 +- 22 files changed, 3359 insertions(+), 6 deletions(-) create mode 100644 external/kafka-0-10-assembly/pom.xml create mode 100644 external/kafka-0-10/pom.xml create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala create mode 100644 external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java create mode 100644 external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java create mode 100644 external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java create mode 100644 external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java create mode 100644 external/kafka-0-10/src/test/resources/log4j.properties create mode 100644 external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala create mode 100644 external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml new file mode 100644 index 0000000000000..f2468d1cbac7f --- /dev/null +++ b/external/kafka-0-10-assembly/pom.xml @@ -0,0 +1,176 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-0-10-assembly_2.11 + jar + Spark Integration for Kafka 0.10 Assembly + http://spark.apache.org/ + + + streaming-kafka-0-10-assembly + + + + + org.apache.spark + spark-streaming-kafka-0-10_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + commons-codec + commons-codec + provided + + + commons-lang + commons-lang + provided + + + com.google.protobuf + protobuf-java + provided + + + net.jpountz.lz4 + lz4 + provided + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.avro + avro-mapred + ${avro.mapred.classifier} + provided + + + org.apache.curator + curator-recipes + provided + + + org.apache.zookeeper + zookeeper + provided + + + log4j + log4j + provided + + + net.java.dev.jets3t + jets3t + provided + + + org.scala-lang + scala-library + provided + + + org.slf4j + slf4j-api + provided + + + org.slf4j + slf4j-log4j12 + provided + + + org.xerial.snappy + snappy-java + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml new file mode 100644 index 0000000000000..50395f6d14453 --- /dev/null +++ b/external/kafka-0-10/pom.xml @@ -0,0 +1,98 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-0-10_2.11 + + streaming-kafka-0-10 + + jar + Spark Integration for Kafka 0.10 + http://spark.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka_${scala.binary.version} + 0.10.0.0 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + net.sf.jopt-simple + jopt-simple + + + org.slf4j + slf4j-simple + + + org.apache.zookeeper + zookeeper + + + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala new file mode 100644 index 0000000000000..fa3ea6131a507 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala @@ -0,0 +1,189 @@ +/* + * 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.kafka010 + +import java.{ util => ju } + +import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord, KafkaConsumer } +import org.apache.kafka.common.{ KafkaException, TopicPartition } + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging + + +/** + * Consumer of single topicpartition, intended for cached reuse. + * Underlying consumer is not threadsafe, so neither is this, + * but processing the same topicpartition and group id in multiple threads is usually bad anyway. + */ +private[kafka010] +class CachedKafkaConsumer[K, V] private( + val groupId: String, + val topic: String, + val partition: Int, + val kafkaParams: ju.Map[String, Object]) extends Logging { + + assert(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG), + "groupId used for cache key must match the groupId in kafkaParams") + + val topicPartition = new TopicPartition(topic, partition) + + protected val consumer = { + val c = new KafkaConsumer[K, V](kafkaParams) + val tps = new ju.ArrayList[TopicPartition]() + tps.add(topicPartition) + c.assign(tps) + c + } + + // TODO if the buffer was kept around as a random-access structure, + // could possibly optimize re-calculating of an RDD in the same batch + protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + protected var nextOffset = -2L + + def close(): Unit = consumer.close() + + /** + * Get the record for the given offset, waiting up to timeout ms if IO is necessary. + * Sequential forward access will use buffers, but random access will be horribly inefficient. + */ + def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = { + logDebug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") + if (offset != nextOffset) { + logInfo(s"Initial fetch for $groupId $topic $partition $offset") + seek(offset) + poll(timeout) + } + + if (!buffer.hasNext()) { poll(timeout) } + assert(buffer.hasNext(), + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + var record = buffer.next() + + if (record.offset != offset) { + logInfo(s"Buffer miss for $groupId $topic $partition $offset") + seek(offset) + poll(timeout) + assert(buffer.hasNext(), + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + record = buffer.next() + assert(record.offset == offset, + s"Got wrong record for $groupId $topic $partition even after seeking to offset $offset") + } + + nextOffset = offset + 1 + record + } + + private def seek(offset: Long): Unit = { + logDebug(s"Seeking to $topicPartition $offset") + consumer.seek(topicPartition, offset) + } + + private def poll(timeout: Long): Unit = { + val p = consumer.poll(timeout) + val r = p.records(topicPartition) + logDebug(s"Polled ${p.partitions()} ${r.size}") + buffer = r.iterator + } + +} + +private[kafka010] +object CachedKafkaConsumer extends Logging { + + private case class CacheKey(groupId: String, topic: String, partition: Int) + + // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap + private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]] = null + + /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ + def init( + initialCapacity: Int, + maxCapacity: Int, + loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { + if (null == cache) { + logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") + cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( + initialCapacity, loadFactor, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer[_, _]]): Boolean = { + if (this.size > maxCapacity) { + try { + entry.getValue.consumer.close() + } catch { + case x: KafkaException => + logError("Error closing oldest Kafka consumer", x) + } + true + } else { + false + } + } + } + } + } + + /** + * Get a cached consumer for groupId, assigned to topic and partition. + * If matching consumer doesn't already exist, will be created using kafkaParams. + */ + def get[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + CachedKafkaConsumer.synchronized { + val k = CacheKey(groupId, topic, partition) + val v = cache.get(k) + if (null == v) { + logInfo(s"Cache miss for $k") + logDebug(cache.keySet.toString) + val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + cache.put(k, c) + c + } else { + // any given topicpartition should have a consistent key and value type + v.asInstanceOf[CachedKafkaConsumer[K, V]] + } + } + + /** + * Get a fresh new instance, unassociated with the global cache. + * Caller is responsible for closing + */ + def getUncached[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + + /** remove consumer for given groupId, topic, and partition, if it exists */ + def remove(groupId: String, topic: String, partition: Int): Unit = { + val k = CacheKey(groupId, topic, partition) + logInfo(s"Removing $k from cache") + val v = CachedKafkaConsumer.synchronized { + cache.remove(k) + } + if (null != v) { + v.close() + logInfo(s"Removed $k from cache") + } + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala new file mode 100644 index 0000000000000..079a07dbc2bd0 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -0,0 +1,314 @@ +/* + * 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.kafka010 + +import java.{ util => ju } + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.annotation.Experimental + + +/** + * :: Experimental :: + * Choice of how to create and configure underlying Kafka Consumers on driver and executors. + * Kafka 0.10 consumers can require additional, sometimes complex, setup after object + * instantiation. This interface encapsulates that process, and allows it to be checkpointed. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +@Experimental +trait ConsumerStrategy[K, V] { + /** + * Kafka + * configuration parameters to be used on executors. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + def executorKafkaParams: ju.Map[String, Object] + + /** + * Must return a fully configured Kafka Consumer, including subscribed or assigned topics. + * This consumer will be used on the driver to query for offsets only, not messages. + * @param currentOffsets A map from TopicPartition to offset, indicating how far the driver + * has successfully read. Will be empty on initial start, possibly non-empty on restart from + * checkpoint. + */ + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] +} + +/** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ +@Experimental +case class Subscribe[K, V] private( + topics: ju.Collection[java.lang.String], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long] + ) extends ConsumerStrategy[K, V] { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.subscribe(topics) + if (currentOffsets.isEmpty) { + offsets.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } + + consumer + } +} + +/** + * :: Experimental :: + * Companion object for creating [[Subscribe]] strategy + */ +@Experimental +object Subscribe { + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def apply[K, V]( + topics: Iterable[java.lang.String], + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): Subscribe[K, V] = { + Subscribe[K, V]( + new ju.ArrayList(topics.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, Long](offsets.asJava)) + } + + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def apply[K, V]( + topics: Iterable[java.lang.String], + kafkaParams: collection.Map[String, Object]): Subscribe[K, V] = { + Subscribe[K, V]( + new ju.ArrayList(topics.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, Long]()) + } + + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def create[K, V]( + topics: ju.Collection[java.lang.String], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long]): Subscribe[K, V] = { + Subscribe[K, V](topics, kafkaParams, offsets) + } + + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def create[K, V]( + topics: ju.Collection[java.lang.String], + kafkaParams: ju.Map[String, Object]): Subscribe[K, V] = { + Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, Long]()) + } + +} + +/** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ +@Experimental +case class Assign[K, V] private( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long] + ) extends ConsumerStrategy[K, V] { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.assign(topicPartitions) + if (currentOffsets.isEmpty) { + offsets.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } + + consumer + } +} + +/** + * :: Experimental :: + * Companion object for creating [[Assign]] strategy + */ +@Experimental +object Assign { + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def apply[K, V]( + topicPartitions: Iterable[TopicPartition], + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): Assign[K, V] = { + Assign[K, V]( + new ju.ArrayList(topicPartitions.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, Long](offsets.asJava)) + } + + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def apply[K, V]( + topicPartitions: Iterable[TopicPartition], + kafkaParams: collection.Map[String, Object]): Assign[K, V] = { + Assign[K, V]( + new ju.ArrayList(topicPartitions.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, Long]()) + } + + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def create[K, V]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long]): Assign[K, V] = { + Assign[K, V](topicPartitions, kafkaParams, offsets) + } + + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def create[K, V]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object]): Assign[K, V] = { + Assign[K, V](topicPartitions, kafkaParams, ju.Collections.emptyMap[TopicPartition, Long]()) + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala new file mode 100644 index 0000000000000..acd1841d5305c --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -0,0 +1,318 @@ +/* + * 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.kafka010 + +import java.{ util => ju } +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicReference + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.{ PartitionInfo, TopicPartition } + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} +import org.apache.spark.streaming.scheduler.rate.RateEstimator + +/** + * A DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. + * @param executorKafkaParams Kafka + * + * configuration parameters. + * Requires "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param consumerStrategy In most cases, pass in [[Subscribe]], + * see [[ConsumerStrategy]] for more details + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +private[spark] class DirectKafkaInputDStream[K, V]( + _ssc: StreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V] + ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging with CanCommitOffsets { + + val executorKafkaParams = { + val ekp = new ju.HashMap[String, Object](consumerStrategy.executorKafkaParams) + KafkaUtils.fixKafkaParams(ekp) + ekp + } + + protected var currentOffsets = Map[TopicPartition, Long]() + + @transient private var kc: Consumer[K, V] = null + def consumer(): Consumer[K, V] = this.synchronized { + if (null == kc) { + kc = consumerStrategy.onStart(currentOffsets) + } + kc + } + + override def persist(newLevel: StorageLevel): DStream[ConsumerRecord[K, V]] = { + logError("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + + protected def getBrokers = { + val c = consumer + val result = new ju.HashMap[TopicPartition, String]() + val hosts = new ju.HashMap[TopicPartition, String]() + val assignments = c.assignment().iterator() + while (assignments.hasNext()) { + val tp: TopicPartition = assignments.next() + if (null == hosts.get(tp)) { + val infos = c.partitionsFor(tp.topic).iterator() + while (infos.hasNext()) { + val i = infos.next() + hosts.put(new TopicPartition(i.topic(), i.partition()), i.leader.host()) + } + } + result.put(tp, hosts.get(tp)) + } + result + } + + protected def getPreferredHosts: ju.Map[TopicPartition, String] = { + locationStrategy match { + case PreferBrokers => getBrokers + case PreferConsistent => ju.Collections.emptyMap[TopicPartition, String]() + case PreferFixed(hostMap) => hostMap + } + } + + // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") + private[streaming] override def name: String = s"Kafka 0.10 direct stream [$id]" + + protected[streaming] override val checkpointData = + new DirectKafkaInputDStreamCheckpointData + + + /** + * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. + */ + override protected[streaming] val rateController: Option[RateController] = { + if (RateController.isBackPressureEnabled(ssc.conf)) { + Some(new DirectKafkaRateController(id, + RateEstimator.create(ssc.conf, context.graph.batchDuration))) + } else { + None + } + } + + private val maxRateLimitPerPartition: Int = context.sparkContext.getConf.getInt( + "spark.streaming.kafka.maxRatePerPartition", 0) + + protected[streaming] def maxMessagesPerPartition( + offsets: Map[TopicPartition, Long]): Option[Map[TopicPartition, Long]] = { + val estimatedRateLimit = rateController.map(_.getLatestRate().toInt) + + // calculate a per-partition rate limit based on current lag + val effectiveRateLimitPerPartition = estimatedRateLimit.filter(_ > 0) match { + case Some(rate) => + val lagPerPartition = offsets.map { case (tp, offset) => + tp -> Math.max(offset - currentOffsets(tp), 0) + } + val totalLag = lagPerPartition.values.sum + + lagPerPartition.map { case (tp, lag) => + val backpressureRate = Math.round(lag / totalLag.toFloat * rate) + tp -> (if (maxRateLimitPerPartition > 0) { + Math.min(backpressureRate, maxRateLimitPerPartition)} else backpressureRate) + } + case None => offsets.map { case (tp, offset) => tp -> maxRateLimitPerPartition } + } + + if (effectiveRateLimitPerPartition.values.sum > 0) { + val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 + Some(effectiveRateLimitPerPartition.map { + case (tp, limit) => tp -> (secsPerBatch * limit).toLong + }) + } else { + None + } + } + + /** + * Returns the latest (highest) available offsets, taking new partitions into account. + */ + protected def latestOffsets(): Map[TopicPartition, Long] = { + val c = consumer + c.poll(0) + val parts = c.assignment().asScala + + // make sure new partitions are reflected in currentOffsets + val newPartitions = parts.diff(currentOffsets.keySet) + // position for new partitions determined by auto.offset.reset if no commit + currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap + // don't want to consume messages, so pause + c.pause(newPartitions.asJava) + // find latest available offsets + c.seekToEnd(currentOffsets.keySet.asJava) + parts.map(tp => tp -> c.position(tp)).toMap + } + + // limits the maximum number of messages per partition + protected def clamp( + offsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + + maxMessagesPerPartition(offsets).map { mmp => + mmp.map { case (tp, messages) => + val uo = offsets(tp) + tp -> Math.min(currentOffsets(tp) + messages, uo) + } + }.getOrElse(offsets) + } + + override def compute(validTime: Time): Option[KafkaRDD[K, V]] = { + val untilOffsets = clamp(latestOffsets()) + val offsetRanges = untilOffsets.map { case (tp, uo) => + val fo = currentOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo) + } + val rdd = new KafkaRDD[K, V]( + context.sparkContext, executorKafkaParams, offsetRanges.toArray, getPreferredHosts, true) + + // Report the record number and metadata of this batch interval to InputInfoTracker. + val description = offsetRanges.filter { offsetRange => + // Don't display empty ranges. + offsetRange.fromOffset != offsetRange.untilOffset + }.map { offsetRange => + s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + + s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" + }.mkString("\n") + // Copy offsetRanges to immutable.List to prevent from being modified by the user + val metadata = Map( + "offsets" -> offsetRanges.toList, + StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) + val inputInfo = StreamInputInfo(id, rdd.count, metadata) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + + currentOffsets = untilOffsets + commitAll() + Some(rdd) + } + + override def start(): Unit = { + val c = consumer + c.poll(0) + if (currentOffsets.isEmpty) { + currentOffsets = c.assignment().asScala.map { tp => + tp -> c.position(tp) + }.toMap + } + + // don't actually want to consume any messages, so pause all partitions + c.pause(currentOffsets.keySet.asJava) + } + + override def stop(): Unit = this.synchronized { + if (kc != null) { + kc.close() + } + } + + protected val commitQueue = new ConcurrentLinkedQueue[OffsetRange] + protected val commitCallback = new AtomicReference[OffsetCommitCallback] + + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange]): Unit = { + commitAsync(offsetRanges, null) + } + + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + * @param callback Only the most recently provided callback will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit = { + commitCallback.set(callback) + commitQueue.addAll(ju.Arrays.asList(offsetRanges: _*)) + } + + protected def commitAll(): Unit = { + val m = new ju.HashMap[TopicPartition, OffsetAndMetadata]() + val it = commitQueue.iterator() + while (it.hasNext) { + val osr = it.next + val tp = osr.topicPartition + val x = m.get(tp) + val offset = if (null == x) { osr.untilOffset } else { Math.max(x.offset, osr.untilOffset) } + m.put(tp, new OffsetAndMetadata(offset)) + } + if (!m.isEmpty) { + consumer.commitAsync(m, commitCallback.get) + } + } + + private[streaming] + class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { + def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { + data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] + } + + override def update(time: Time): Unit = { + batchForTime.clear() + generatedRDDs.foreach { kv => + val a = kv._2.asInstanceOf[KafkaRDD[K, V]].offsetRanges.map(_.toTuple).toArray + batchForTime += kv._1 -> a + } + } + + override def cleanup(time: Time): Unit = { } + + override def restore(): Unit = { + batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => + logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") + generatedRDDs += t -> new KafkaRDD[K, V]( + context.sparkContext, + executorKafkaParams, + b.map(OffsetRange(_)), + getPreferredHosts, + // during restore, it's possible same partition will be consumed from multiple + // threads, so dont use cache + false + ) + } + } + } + + /** + * A RateController to retrieve the rate from RateEstimator. + */ + private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala new file mode 100644 index 0000000000000..c15c16344924f --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala @@ -0,0 +1,232 @@ +/* + * 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.kafka010 + +import java.{ util => ju } + +import scala.collection.mutable.ArrayBuffer + +import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.storage.StorageLevel + +/** + * A batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * + * configuration parameters. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] + * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @param useConsumerCache whether to use a consumer from a per-jvm cache + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +private[spark] class KafkaRDD[K, V]( + sc: SparkContext, + val kafkaParams: ju.Map[String, Object], + val offsetRanges: Array[OffsetRange], + val preferredHosts: ju.Map[TopicPartition, String], + useConsumerCache: Boolean +) extends RDD[ConsumerRecord[K, V]](sc, Nil) with Logging with HasOffsetRanges { + + assert("none" == + kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).asInstanceOf[String], + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + + " must be set to none for executor kafka params, else messages may not match offsetRange") + + assert(false == + kafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + + " must be set to false for executor kafka params, else offsets may commit before processing") + + // TODO is it necessary to have separate configs for initial poll time vs ongoing poll time? + private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", 256) + private val cacheInitialCapacity = + conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) + private val cacheMaxCapacity = + conf.getInt("spark.streaming.kafka.consumer.cache.maxCapacity", 64) + private val cacheLoadFactor = + conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat + + override def persist(newLevel: StorageLevel): this.type = { + logError("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) + }.toArray + } + + override def count(): Long = offsetRanges.map(_.count).sum + + override def countApprox( + timeout: Long, + confidence: Double = 0.95 + ): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[ConsumerRecord[K, V]] = { + val nonEmptyPartitions = this.partitions + .map(_.asInstanceOf[KafkaRDDPartition]) + .filter(_.count > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[ConsumerRecord[K, V]](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.count) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[ConsumerRecord[K, V]] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[ConsumerRecord[K, V]]) => + it.take(parts(tc.partitionId)).toArray, parts.keys.toArray + ) + res.foreach(buf ++= _) + buf.toArray + } + + private def executors(): Array[ExecutorCacheTaskLocation] = { + val bm = sparkContext.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compareExecutors) + } + + protected[kafka010] def compareExecutors( + a: ExecutorCacheTaskLocation, + b: ExecutorCacheTaskLocation): Boolean = + if (a.host == b.host) { + a.executorId > b.executorId + } else { + a.host > b.host + } + + /** + * Non-negative modulus, from java 8 math + */ + private def floorMod(a: Int, b: Int): Int = ((a % b) + b) % b + + override def getPreferredLocations(thePart: Partition): Seq[String] = { + // The intention is best-effort consistent executor for a given topicpartition, + // so that caching consumers can be effective. + // TODO what about hosts specified by ip vs name + val part = thePart.asInstanceOf[KafkaRDDPartition] + val allExecs = executors() + val tp = part.topicPartition + val prefHost = preferredHosts.get(tp) + val prefExecs = if (null == prefHost) allExecs else allExecs.filter(_.host == prefHost) + val execs = if (prefExecs.isEmpty) allExecs else prefExecs + if (execs.isEmpty) { + Seq() + } else { + // execs is sorted, tp.hashCode depends only on topic and partition, so consistent index + val index = this.floorMod(tp.hashCode, execs.length) + val chosen = execs(index) + Seq(chosen.toString) + } + } + + private def errBeginAfterEnd(part: KafkaRDDPartition): String = + s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged" + + override def compute(thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[K, V]] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) + if (part.fromOffset == part.untilOffset) { + logInfo(s"Beginning offset ${part.fromOffset} is the same as ending offset " + + s"skipping ${part.topic} ${part.partition}") + Iterator.empty + } else { + new KafkaRDDIterator(part, context) + } + } + + /** + * An iterator that fetches messages directly from Kafka for the offsets in partition. + * Uses a cached consumer where possible to take advantage of prefetching + */ + private class KafkaRDDIterator( + part: KafkaRDDPartition, + context: TaskContext) extends Iterator[ConsumerRecord[K, V]] { + + logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + + s"offsets ${part.fromOffset} -> ${part.untilOffset}") + + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + + context.addTaskCompletionListener{ context => closeIfNeeded() } + + val consumer = if (useConsumerCache) { + CachedKafkaConsumer.init(cacheInitialCapacity, cacheMaxCapacity, cacheLoadFactor) + if (context.attemptNumber > 1) { + // just in case the prior attempt failures were cache related + CachedKafkaConsumer.remove(groupId, part.topic, part.partition) + } + CachedKafkaConsumer.get[K, V](groupId, part.topic, part.partition, kafkaParams) + } else { + CachedKafkaConsumer.getUncached[K, V](groupId, part.topic, part.partition, kafkaParams) + } + + var requestOffset = part.fromOffset + + def closeIfNeeded(): Unit = { + if (!useConsumerCache && consumer != null) { + consumer.close + } + } + + override def hasNext(): Boolean = requestOffset < part.untilOffset + + override def next(): ConsumerRecord[K, V] = { + assert(hasNext(), "Can't call getNext() once untilOffset has been reached") + val r = consumer.get(requestOffset, pollTimeout) + requestOffset += 1 + r + } + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala new file mode 100644 index 0000000000000..95569b109f30d --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala @@ -0,0 +1,45 @@ +/* + * 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.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.Partition + + +/** + * @param topic kafka topic name + * @param partition kafka partition id + * @param fromOffset inclusive starting offset + * @param untilOffset exclusive ending offset + */ +private[kafka010] +class KafkaRDDPartition( + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long +) extends Partition { + /** Number of messages this partition refers to */ + def count(): Long = untilOffset - fromOffset + + /** Kafka TopicPartition object, for convenience */ + def topicPartition(): TopicPartition = new TopicPartition(topic, partition) + +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala new file mode 100644 index 0000000000000..13c08430db6be --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala @@ -0,0 +1,277 @@ +/* + * 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.kafka010 + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.language.postfixOps +import scala.util.control.NonFatal + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.ZkUtils +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +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[kafka010] class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkUtils: ZkUtils = _ + + // 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: ZkUtils = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkUtils).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 + zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + 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, doLog = false) + 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 (zkUtils != null) { + zkUtils.close() + zkUtils = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkUtils, topic, partitions, 1) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + createTopic(topic, 1) + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.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) + // wait for all in-sync replicas to ack sends + props.put("request.required.acks", "-1") + 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 = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + zkUtils.getLeaderForPartition(topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.size >= 1 + + case _ => + false + } + eventually(Time(10000), Time(100)) { + assert(isPropagated, 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-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala new file mode 100644 index 0000000000000..c0524990bc4dc --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -0,0 +1,175 @@ +/* + * 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.kafka010 + +import java.{ util => ju } + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } +import org.apache.spark.api.java.function.{ Function0 => JFunction0 } +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } +import org.apache.spark.streaming.dstream._ + +/** + * :: Experimental :: + * Companion object for constructing Kafka streams and RDDs + */ +@Experimental +object KafkaUtils extends Logging { + /** + * :: Experimental :: + * Scala constructor for a batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * + * configuration parameters. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createRDD[K, V]( + sc: SparkContext, + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + locationStrategy: LocationStrategy + ): RDD[ConsumerRecord[K, V]] = { + val preferredHosts = locationStrategy match { + case PreferBrokers => + throw new AssertionError( + "If you want to prefer brokers, you must provide a mapping using PreferFixed " + + "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you.") + case PreferConsistent => ju.Collections.emptyMap[TopicPartition, String]() + case PreferFixed(hostMap) => hostMap + } + val kp = new ju.HashMap[String, Object](kafkaParams) + fixKafkaParams(kp) + val osr = offsetRanges.clone() + + new KafkaRDD[K, V](sc, kp, osr, preferredHosts, true) + } + + /** + * :: Experimental :: + * Java constructor for a batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param kafkaParams Kafka + * + * configuration parameters. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createRDD[K, V]( + jsc: JavaSparkContext, + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + locationStrategy: LocationStrategy + ): JavaRDD[ConsumerRecord[K, V]] = { + + new JavaRDD(createRDD[K, V](jsc.sc, kafkaParams, offsetRanges, locationStrategy)) + } + + /** + * :: Experimental :: + * Scala constructor for a DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. + * @param consumerStrategy In most cases, pass in [[Subscribe]], + * see [[ConsumerStrategy]] for more details + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K, V]( + ssc: StreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V] + ): InputDStream[ConsumerRecord[K, V]] = { + new DirectKafkaInputDStream[K, V](ssc, locationStrategy, consumerStrategy) + } + + /** + * :: Experimental :: + * Java constructor for a DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. + * @param consumerStrategy In most cases, pass in [[Subscribe]], + * see [[ConsumerStrategy]] for more details + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K, V]( + jssc: JavaStreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V] + ): JavaInputDStream[ConsumerRecord[K, V]] = { + new JavaInputDStream( + createDirectStream[K, V]( + jssc.ssc, locationStrategy, consumerStrategy)) + } + + /** + * Tweak kafka params to prevent issues on executors + */ + private[kafka010] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { + logWarning(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") + kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + + logWarning(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") + kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // driver and executor should be in different consumer groups + val groupId = "spark-executor-" + kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + logWarning(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") + kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + + // possible workaround for KAFKA-3135 + val rbb = kafkaParams.get(ConsumerConfig.RECEIVE_BUFFER_CONFIG) + if (null == rbb || rbb.asInstanceOf[java.lang.Integer] < 65536) { + logWarning(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 see KAFKA-3135") + kafkaParams.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + } + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala new file mode 100644 index 0000000000000..df620300eae21 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala @@ -0,0 +1,77 @@ +/* + * 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.kafka010 + +import java.{ util => ju } + +import scala.collection.JavaConverters._ + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.annotation.Experimental + + +/** + * :: Experimental :: + * Choice of how to schedule consumers for a given TopicPartition on an executor. + * Kafka 0.10 consumers prefetch messages, so it's important for performance + * to keep cached consumers on appropriate executors, not recreate them for every partition. + * Choice of location is only a preference, not an absolute; partitions may be scheduled elsewhere. + */ +@Experimental +sealed trait LocationStrategy + +/** + * :: Experimental :: + * Use this only if your executors are on the same nodes as your Kafka brokers. + */ +@Experimental +case object PreferBrokers extends LocationStrategy { + def create: PreferBrokers.type = this +} + +/** + * :: Experimental :: + * Use this in most cases, it will consistently distribute partitions across all executors. + */ +@Experimental +case object PreferConsistent extends LocationStrategy { + def create: PreferConsistent.type = this +} + +/** + * :: Experimental :: + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ +@Experimental +case class PreferFixed private(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy + +/** + * :: Experimental :: + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ +@Experimental +object PreferFixed { + def apply(hostMap: collection.Map[TopicPartition, String]): PreferFixed = { + PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) + } + def create(hostMap: ju.Map[TopicPartition, String]): PreferFixed = + PreferFixed(hostMap) +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala new file mode 100644 index 0000000000000..c66d3c9b8d229 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala @@ -0,0 +1,153 @@ +/* + * 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.kafka010 + +import org.apache.kafka.clients.consumer.OffsetCommitCallback +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.annotation.Experimental + +/** + * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the + * offset ranges in RDDs generated by the direct Kafka DStream (see + * [[KafkaUtils.createDirectStream]]). + * {{{ + * KafkaUtils.createDirectStream(...).foreachRDD { rdd => + * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + * ... + * } + * }}} + */ +trait HasOffsetRanges { + def offsetRanges: Array[OffsetRange] +} + +/** + * :: Experimental :: + * Represents any object that can commit a collection of [[OffsetRange]]s. + * The direct Kafka DStream implements this interface (see + * [[KafkaUtils.createDirectStream]]). + * {{{ + * val stream = KafkaUtils.createDirectStream(...) + * ... + * stream.asInstanceOf[CanCommitOffsets].commitAsync(offsets, new OffsetCommitCallback() { + * def onComplete(m: java.util.Map[TopicPartition, OffsetAndMetadata], e: Exception) { + * if (null != e) { + * // error + * } else { + * // success + * } + * } + * }) + * }}} + */ +@Experimental +trait CanCommitOffsets { + /** + * :: Experimental :: + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * This is only needed if you intend to store offsets in Kafka, instead of your own store. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + */ + @Experimental + def commitAsync(offsetRanges: Array[OffsetRange]): Unit + + /** + * :: Experimental :: + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * This is only needed if you intend to store offsets in Kafka, instead of your own store. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + * @param callback Only the most recently provided callback will be used at commit. + */ + @Experimental + def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit +} + +/** + * Represents a range of offsets from a single Kafka TopicPartition. Instances of this class + * can be created with `OffsetRange.create()`. + * @param topic Kafka topic name + * @param partition Kafka partition id + * @param fromOffset Inclusive starting offset + * @param untilOffset Exclusive ending offset + */ +final class OffsetRange private( + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long) extends Serializable { + import OffsetRange.OffsetRangeTuple + + /** Kafka TopicPartition object, for convenience */ + def topicPartition(): TopicPartition = new TopicPartition(topic, partition) + + /** Number of messages this OffsetRange refers to */ + def count(): Long = untilOffset - fromOffset + + override def equals(obj: Any): Boolean = obj match { + case that: OffsetRange => + this.topic == that.topic && + this.partition == that.partition && + this.fromOffset == that.fromOffset && + this.untilOffset == that.untilOffset + case _ => false + } + + override def hashCode(): Int = { + toTuple.hashCode() + } + + override def toString(): String = { + s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" + } + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[streaming] + def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) +} + +/** + * Companion object the provides methods to create instances of [[OffsetRange]]. + */ +object OffsetRange { + def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def create( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicPartition.topic, topicPartition.partition, fromOffset, untilOffset) + + def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def apply( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicPartition.topic, topicPartition.partition, fromOffset, untilOffset) + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[kafka010] + type OffsetRangeTuple = (String, Int, Long, Long) + + private[kafka010] + def apply(t: OffsetRangeTuple) = + new OffsetRange(t._1, t._2, t._3, t._4) +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java new file mode 100644 index 0000000000000..ebfcf8764a328 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java @@ -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. + */ + +/** + * Spark Integration for Kafka 0.10 + */ +package org.apache.spark.streaming.kafka010; diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala new file mode 100644 index 0000000000000..2bfc1e84d7ccd --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala @@ -0,0 +1,23 @@ +/* + * 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 + +/** + * Spark Integration for Kafka 0.10 + */ +package object kafka diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java new file mode 100644 index 0000000000000..aba45f5de6782 --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -0,0 +1,84 @@ +/* + * 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.kafka010; + +import java.io.Serializable; +import java.util.*; + +import scala.collection.JavaConverters; + +import org.apache.kafka.common.TopicPartition; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaConsumerStrategySuite implements Serializable { + + @Test + public void testConsumerStrategyConstructors() { + final String topic1 = "topic1"; + final Collection topics = Arrays.asList(topic1); + final scala.collection.Iterable sTopics = + JavaConverters.collectionAsScalaIterableConverter(topics).asScala(); + final TopicPartition tp1 = new TopicPartition(topic1, 0); + final TopicPartition tp2 = new TopicPartition(topic1, 1); + final Collection parts = Arrays.asList(tp1, tp2); + final scala.collection.Iterable sParts = + JavaConverters.collectionAsScalaIterableConverter(parts).asScala(); + final Map kafkaParams = new HashMap(); + kafkaParams.put("bootstrap.servers", "not used"); + final scala.collection.Map sKafkaParams = + JavaConverters.mapAsScalaMapConverter(kafkaParams).asScala(); + final Map offsets = new HashMap<>(); + offsets.put(tp1, 23L); + final scala.collection.Map sOffsets = + JavaConverters.mapAsScalaMapConverter(offsets).asScala(); + + // make sure constructors can be called from java + final ConsumerStrategy sub0 = + Subscribe.apply(topics, kafkaParams, offsets); + final ConsumerStrategy sub1 = + Subscribe.apply(sTopics, sKafkaParams, sOffsets); + final ConsumerStrategy sub2 = + Subscribe.apply(sTopics, sKafkaParams); + final ConsumerStrategy sub3 = + Subscribe.create(topics, kafkaParams, offsets); + final ConsumerStrategy sub4 = + Subscribe.create(topics, kafkaParams); + + Assert.assertEquals( + sub1.executorKafkaParams().get("bootstrap.servers"), + sub3.executorKafkaParams().get("bootstrap.servers")); + + final ConsumerStrategy asn0 = + Assign.apply(parts, kafkaParams, offsets); + final ConsumerStrategy asn1 = + Assign.apply(sParts, sKafkaParams, sOffsets); + final ConsumerStrategy asn2 = + Assign.apply(sParts, sKafkaParams); + final ConsumerStrategy asn3 = + Assign.create(parts, kafkaParams, offsets); + final ConsumerStrategy asn4 = + Assign.create(parts, kafkaParams); + + Assert.assertEquals( + asn1.executorKafkaParams().get("bootstrap.servers"), + asn3.executorKafkaParams().get("bootstrap.servers")); + } + +} diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java new file mode 100644 index 0000000000000..e57ede7afaef4 --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java @@ -0,0 +1,180 @@ +/* + * 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.kafka010; + +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +public class JavaDirectKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + 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; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaStream() throws InterruptedException { + final String topic1 = "topic1"; + final String topic2 = "topic2"; + // hold a reference to the current offset ranges, so it can be used downstream + final AtomicReference offsetRanges = new AtomicReference<>(); + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + Set sent = new HashSet<>(); + sent.addAll(Arrays.asList(topic1data)); + sent.addAll(Arrays.asList(topic2data)); + + Random random = new Random(); + + final Map kafkaParams = new HashMap<>(); + kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); + kafkaParams.put("key.deserializer", StringDeserializer.class); + kafkaParams.put("value.deserializer", StringDeserializer.class); + kafkaParams.put("auto.offset.reset", "earliest"); + kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); + + JavaInputDStream> istream1 = KafkaUtils.createDirectStream( + ssc, + PreferConsistent.create(), + Subscribe.create(Arrays.asList(topic1), kafkaParams) + ); + + JavaDStream stream1 = istream1.transform( + // Make sure you can get offset ranges from the rdd + new Function>, + JavaRDD>>() { + @Override + public JavaRDD> call( + JavaRDD> rdd + ) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic1, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); + } + } + ); + + final Map kafkaParams2 = new HashMap<>(kafkaParams); + kafkaParams2.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); + + JavaInputDStream> istream2 = KafkaUtils.createDirectStream( + ssc, + PreferConsistent.create(), + Subscribe.create(Arrays.asList(topic2), kafkaParams2) + ); + + JavaDStream stream2 = istream2.transform( + // Make sure you can get offset ranges from the rdd + new Function>, + JavaRDD>>() { + @Override + public JavaRDD> call( + JavaRDD> rdd + ) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic2, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); + } + } + ); + + JavaDStream unifiedStream = stream1.union(stream2); + + final Set result = Collections.synchronizedSet(new HashSet()); + unifiedStream.foreachRDD(new VoidFunction>() { + @Override + public void call(JavaRDD rdd) { + result.addAll(rdd.collect()); + } + } + ); + ssc.start(); + long startTime = System.currentTimeMillis(); + boolean matches = false; + while (!matches && System.currentTimeMillis() - startTime < 20000) { + matches = sent.size() == result.size(); + Thread.sleep(50); + } + Assert.assertEquals(sent, result); + ssc.stop(); + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java new file mode 100644 index 0000000000000..548ba134dcddf --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java @@ -0,0 +1,122 @@ +/* + * 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.kafka010; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +public class JavaKafkaRDDSuite implements Serializable { + private transient JavaSparkContext sc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + sc = new JavaSparkContext(sparkConf); + } + + @After + public void tearDown() { + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaRDD() throws InterruptedException { + String topic1 = "topic1"; + String topic2 = "topic2"; + + createTopicAndSendData(topic1); + createTopicAndSendData(topic2); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); + kafkaParams.put("key.deserializer", StringDeserializer.class); + kafkaParams.put("value.deserializer", StringDeserializer.class); + + OffsetRange[] offsetRanges = { + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) + }; + + Map leaders = new HashMap<>(); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); + String broker = hostAndPort[0]; + leaders.put(offsetRanges[0].topicPartition(), broker); + leaders.put(offsetRanges[1].topicPartition(), broker); + + Function, String> handler = + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); + } + }; + + JavaRDD rdd1 = KafkaUtils.createRDD( + sc, + kafkaParams, + offsetRanges, + PreferFixed.create(leaders) + ).map(handler); + + JavaRDD rdd2 = KafkaUtils.createRDD( + sc, + kafkaParams, + offsetRanges, + PreferConsistent.create() + ).map(handler); + + // just making sure the java user apis work; the scala tests handle logic corner cases + long count1 = rdd1.count(); + long count2 = rdd2.count(); + Assert.assertTrue(count1 > 0); + Assert.assertEquals(count1, count2); + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java new file mode 100644 index 0000000000000..7873c09e1af85 --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java @@ -0,0 +1,58 @@ +/* + * 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.kafka010; + +import java.io.Serializable; +import java.util.*; + +import scala.collection.JavaConverters; + +import org.apache.kafka.common.TopicPartition; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaLocationStrategySuite implements Serializable { + + @Test + public void testLocationStrategyConstructors() { + final String topic1 = "topic1"; + final TopicPartition tp1 = new TopicPartition(topic1, 0); + final TopicPartition tp2 = new TopicPartition(topic1, 1); + final Map hosts = new HashMap<>(); + hosts.put(tp1, "node1"); + hosts.put(tp2, "node2"); + final scala.collection.Map sHosts = + JavaConverters.mapAsScalaMapConverter(hosts).asScala(); + + // make sure constructors can be called from java + final LocationStrategy c1 = PreferConsistent.create(); + final LocationStrategy c2 = PreferConsistent$.MODULE$; + Assert.assertEquals(c1, c2); + + final LocationStrategy c3 = PreferBrokers.create(); + final LocationStrategy c4 = PreferBrokers$.MODULE$; + Assert.assertEquals(c3, c4); + + final LocationStrategy c5 = PreferFixed.create(hosts); + final LocationStrategy c6 = PreferFixed.apply(sHosts); + Assert.assertEquals(c5, c6); + + } + +} diff --git a/external/kafka-0-10/src/test/resources/log4j.properties b/external/kafka-0-10/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..75e3b53a093f6 --- /dev/null +++ b/external/kafka-0-10/src/test/resources/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=target/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.spark-project.jetty=WARN + diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala new file mode 100644 index 0000000000000..776d11ad2f648 --- /dev/null +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -0,0 +1,612 @@ +/* + * 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.kafka010 + +import java.io.File +import java.util.{ Arrays, HashMap => JHashMap, Map => JMap } +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.StringDeserializer +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging +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.streaming.scheduler._ +import org.apache.spark.streaming.scheduler.rate.RateEstimator +import org.apache.spark.util.Utils + +class DirectKafkaStreamSuite + extends SparkFunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + + private var sc: SparkContext = _ + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + after { + if (ssc != null) { + ssc.stop() + sc = null + } + if (sc != null) { + sc.stop() + } + if (testDir != null) { + Utils.deleteRecursively(testDir) + } + } + + def getKafkaParams(extra: (String, Object)*): JHashMap[String, Object] = { + val kp = new JHashMap[String, Object]() + kp.put("bootstrap.servers", kafkaTestUtils.brokerAddress) + kp.put("key.deserializer", classOf[StringDeserializer]) + kp.put("value.deserializer", classOf[StringDeserializer]) + kp.put("group.id", s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") + extra.foreach(e => kp.put(e._1, e._2)) + kp + } + + val preferredHosts = PreferConsistent + + test("basic stream receiving with multiple topics and smallest starting offset") { + val topics = List("basic1", "basic2", "basic3") + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + val totalSent = data.values.sum * topics.size + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](topics, kafkaParams.asScala)) + } + val allReceived = new ConcurrentLinkedQueue[(String, String)]() + + // hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + val tf = stream.transform { rdd => + // Get the offset ranges in the RDD + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd.map(r => (r.key, r.value)) + } + + tf.foreachRDD { rdd => + for (o <- offsetRanges) { + logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + val collected = rdd.mapPartitionsWithIndex { (i, iter) => + // For each partition, get size of the range in the partition, + // and the number of items in the partition + val off = offsetRanges(i) + val all = iter.toSeq + val partSize = all.size + val rangeSize = off.untilOffset - off.fromOffset + Iterator((partSize, rangeSize)) + }.collect + + // Verify whether number of elements in each partition + // matches with the corresponding offset range + collected.foreach { case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") + } + } + + stream.foreachRDD { rdd => + allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) + } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + } + ssc.stop() + } + + test("receiving from largest starting offset") { + val topic = "latest" + val topicPartition = new TopicPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") + val kc = new KafkaConsumer(kafkaParams) + kc.assign(Arrays.asList(topicPartition)) + def getLatestOffset(): Long = { + kc.seekToEnd(Arrays.asList(topicPartition)) + kc.position(topicPartition) + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() > 3) + } + val offsetBeforeStart = getLatestOffset() + kc.close() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + val s = new DirectKafkaInputDStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + s.consumer.poll(0) + assert( + s.consumer.position(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + s + } + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.map { _.value }.foreachRDD { rdd => + collectedData.addAll(Arrays.asList(rdd.collect(): _*)) + } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + } + + + test("creating stream by offset") { + val topic = "offset" + val topicPartition = new TopicPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") + val kc = new KafkaConsumer(kafkaParams) + kc.assign(Arrays.asList(topicPartition)) + def getLatestOffset(): Long = { + kc.seekToEnd(Arrays.asList(topicPartition)) + kc.position(topicPartition) + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() >= 10) + } + val offsetBeforeStart = getLatestOffset() + kc.close() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, + Assign[String, String]( + List(topicPartition), + kafkaParams.asScala, + Map(topicPartition -> 11L))) + s.consumer.poll(0) + assert( + s.consumer.position(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + s + } + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.map(_.value).foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + } + + // Test to verify the offset ranges can be recovered from the checkpoints + test("offset recovery") { + val topic = "recovery" + kafkaTestUtils.createTopic(topic) + testDir = Utils.createTempDir() + + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + // Send data to Kafka + def sendData(data: Seq[Int]) { + val strings = data.map { _.toString} + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + val kafkaStream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + } + val keyedStream = kafkaStream.map { r => "key" -> r.value.toInt } + val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => + Some(values.sum + state.getOrElse(0)) + } + ssc.checkpoint(testDir.getAbsolutePath) + + // This is ensure all the data is eventually receiving only once + stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => + rdd.collect().headOption.foreach { x => + DirectKafkaStreamSuite.total.set(x._2) + } + } + + ssc.start() + + // Send some data + for (i <- (1 to 10).grouped(4)) { + sendData(i) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) + } + + ssc.stop() + + // Verify that offset ranges were generated + val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) + assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") + assert( + offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, + "starting offset not zero" + ) + + logInfo("====== RESTARTING ========") + + // Recover context from checkpoints + ssc = new StreamingContext(testDir.getAbsolutePath) + val recoveredStream = + ssc.graph.getInputStreams().head.asInstanceOf[DStream[ConsumerRecord[String, String]]] + + // Verify offset ranges have been recovered + val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => (x._1, x._2.toSet) } + assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") + val earlierOffsetRanges = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } + assert( + recoveredOffsetRanges.forall { or => + earlierOffsetRanges.contains((or._1, or._2)) + }, + "Recovered ranges are not the same as the ones generated\n" + + earlierOffsetRanges + "\n" + recoveredOffsetRanges + ) + // Restart context, give more data and verify the total at the end + // If the total is write that means each records has been received only once + ssc.start() + for (i <- (11 to 20).grouped(4)) { + sendData(i) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) + } + ssc.stop() + } + + // Test to verify the offsets can be recovered from Kafka + test("offset recovery from kafka") { + val topic = "recoveryfromkafka" + kafkaTestUtils.createTopic(topic) + + val kafkaParams = getKafkaParams( + "auto.offset.reset" -> "earliest", + ("enable.auto.commit", false: java.lang.Boolean) + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + val committed = new JHashMap[TopicPartition, OffsetAndMetadata]() + + // Send data to Kafka and wait for it to be received + def sendDataAndWaitForReceive(data: Seq[Int]) { + val strings = data.map { _.toString} + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(strings.forall { collectedData.contains }) + } + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + withClue("Error creating direct stream") { + val kafkaStream = KafkaUtils.createDirectStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => + val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + val data = rdd.map(_.value).collect() + collectedData.addAll(Arrays.asList(data: _*)) + kafkaStream.asInstanceOf[CanCommitOffsets] + .commitAsync(offsets, new OffsetCommitCallback() { + def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { + if (null != e) { + logError("commit failed", e) + } else { + committed.putAll(m) + } + } + }) + } + } + ssc.start() + // Send some data and wait for them to be received + for (i <- (1 to 10).grouped(4)) { + sendDataAndWaitForReceive(i) + } + ssc.stop() + assert(! committed.isEmpty) + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer.poll(0) + committed.asScala.foreach { + case (k, v) => + // commits are async, not exactly once + assert(v.offset > 0) + assert(consumer.position(k) >= v.offset) + } + } + + + test("Direct Kafka stream report input information") { + val topic = "report-test" + val data = Map("a" -> 7, "b" -> 9) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + val totalSent = data.values.sum + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + import DirectKafkaStreamSuite._ + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val collector = new InputInfoCollector + ssc.addStreamingListener(collector) + + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + } + + val allReceived = new ConcurrentLinkedQueue[(String, String)] + + stream.map(r => (r.key, r.value)) + .foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + + // Calculate all the record number collected in the StreamingListener. + assert(collector.numRecordsSubmitted.get() === totalSent) + assert(collector.numRecordsStarted.get() === totalSent) + assert(collector.numRecordsCompleted.get() === totalSent) + } + ssc.stop() + } + + test("maxMessagesPerPartition with backpressure disabled") { + val topic = "maxMessagesPerPartition" + val kafkaStream = getDirectKafkaStream(topic, None) + + val input = Map(new TopicPartition(topic, 0) -> 50L, new TopicPartition(topic, 1) -> 50L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(new TopicPartition(topic, 0) -> 10L, new TopicPartition(topic, 1) -> 10L)) + } + + test("maxMessagesPerPartition with no lag") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 100)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(new TopicPartition(topic, 0) -> 0L, new TopicPartition(topic, 1) -> 0L) + assert(kafkaStream.maxMessagesPerPartition(input).isEmpty) + } + + test("maxMessagesPerPartition respects max rate") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 1000)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(new TopicPartition(topic, 0) -> 1000L, new TopicPartition(topic, 1) -> 1000L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(new TopicPartition(topic, 0) -> 10L, new TopicPartition(topic, 1) -> 10L)) + } + + test("using rate controller") { + val topic = "backpressure" + val topicPartitions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + kafkaTestUtils.createTopic(topic, 2) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + val executorKafkaParams = new JHashMap[String, Object](kafkaParams) + KafkaUtils.fixKafkaParams(executorKafkaParams) + + val batchIntervalMilliseconds = 100 + val estimator = new ConstantEstimator(100) + val messages = Map("foo" -> 200) + kafkaTestUtils.sendMessages(topic, messages) + + val sparkConf = new SparkConf() + // Safe, even with streaming, because we're using the direct API. + // Using 1 core is useful to make the test more predictable. + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val kafkaStream = withClue("Error creating direct stream") { + new DirectKafkaInputDStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) { + override protected[streaming] val rateController = + Some(new DirectKafkaRateController(id, estimator)) + }.map(r => (r.key, r.value)) + } + + val collectedData = new ConcurrentLinkedQueue[Array[String]]() + + // Used for assertion failure messages. + def dataToString: String = + collectedData.asScala.map(_.mkString("[", ",", "]")).mkString("{", ", ", "}") + + // This is to collect the raw data received from Kafka + kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => + val data = rdd.map { _._2 }.collect() + collectedData.add(data) + } + + ssc.start() + + // Try different rate limits. + // Wait for arrays of data to appear matching the rate. + Seq(100, 50, 20).foreach { rate => + collectedData.clear() // Empty this buffer on each pass. + estimator.updateRate(rate) // Set a new rate. + // Expect blocks of data equal to "rate", scaled by the interval length in secs. + val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) + eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { + // Assert that rate estimator values are used to determine maxMessagesPerPartition. + // Funky "-" in message makes the complete assertion message read better. + assert(collectedData.asScala.exists(_.size == expectedSize), + s" - No arrays of size $expectedSize for rate $rate found in $dataToString") + } + } + + ssc.stop() + } + + /** Get the generated offset ranges from the DirectKafkaStream */ + private def getOffsetRanges[K, V]( + kafkaStream: DStream[ConsumerRecord[K, V]]): Seq[(Time, Array[OffsetRange])] = { + kafkaStream.generatedRDDs.mapValues { rdd => + rdd.asInstanceOf[HasOffsetRanges].offsetRanges + }.toSeq.sortBy { _._1 } + } + + private def getDirectKafkaStream(topic: String, mockRateController: Option[RateController]) = { + val batchIntervalMilliseconds = 100 + + val sparkConf = new SparkConf() + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + val ekp = new JHashMap[String, Object](kafkaParams) + KafkaUtils.fixKafkaParams(ekp) + + val s = new DirectKafkaInputDStream[String, String]( + ssc, + preferredHosts, + new ConsumerStrategy[String, String] { + def executorKafkaParams = ekp + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[String, String] = { + val consumer = new KafkaConsumer[String, String](kafkaParams) + val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + consumer.assign(Arrays.asList(tps: _*)) + tps.foreach(tp => consumer.seek(tp, 0)) + consumer + } + } + ) { + override protected[streaming] val rateController = mockRateController + } + // manual start necessary because we arent consuming the stream, just checking its state + s.start() + s + } +} + +object DirectKafkaStreamSuite { + val total = new AtomicLong(-1L) + + class InputInfoCollector extends StreamingListener { + val numRecordsSubmitted = new AtomicLong(0L) + val numRecordsStarted = new AtomicLong(0L) + val numRecordsCompleted = new AtomicLong(0L) + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { + numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) + } + } +} + +private[streaming] class ConstantEstimator(@volatile private var rate: Long) + extends RateEstimator { + + def updateRate(newRate: Long): Unit = { + rate = newRate + } + + def compute( + time: Long, + elements: Long, + processingDelay: Long, + schedulingDelay: Long): Option[Double] = Some(rate) +} + +private[streaming] class ConstantRateController(id: Int, estimator: RateEstimator, rate: Long) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + override def getLatestRate(): Long = rate +} diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala new file mode 100644 index 0000000000000..3d2546ddd936d --- /dev/null +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.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.streaming.kafka010 + +import java.{ util => ju } + +import scala.collection.JavaConverters._ +import scala.util.Random + +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.StringDeserializer +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark._ +import org.apache.spark.scheduler.ExecutorCacheTaskLocation + +class KafkaRDDSuite extends SparkFunSuite 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) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (sc != null) { + sc.stop + sc = null + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + private def getKafkaParams() = Map[String, Object]( + "bootstrap.servers" -> kafkaTestUtils.brokerAddress, + "key.deserializer" -> classOf[StringDeserializer], + "value.deserializer" -> classOf[StringDeserializer], + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}" + ).asJava + + private val preferredHosts = PreferConsistent + + test("basic usage") { + val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" + kafkaTestUtils.createTopic(topic) + val messages = Array("the", "quick", "brown", "fox") + kafkaTestUtils.sendMessages(topic, messages) + + val kafkaParams = getKafkaParams() + + val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) + + val rdd = KafkaUtils.createRDD[String, String](sc, kafkaParams, offsetRanges, preferredHosts) + .map(_.value) + + val received = rdd.collect.toSet + assert(received === messages.toSet) + + // size-related method optimizations return sane results + assert(rdd.count === messages.size) + assert(rdd.countApprox(0).getFinalValue.mean === messages.size) + assert(!rdd.isEmpty) + assert(rdd.take(1).size === 1) + assert(rdd.take(1).head === messages.head) + assert(rdd.take(messages.size + 10).size === messages.size) + + val emptyRdd = KafkaUtils.createRDD[String, String]( + sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0)), preferredHosts) + + assert(emptyRdd.isEmpty) + + // invalid offset ranges throw exceptions + val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) + intercept[SparkException] { + val result = KafkaUtils.createRDD[String, String](sc, kafkaParams, badRanges, preferredHosts) + .map(_.value) + .collect() + } + } + + test("iterator boundary conditions") { + // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd + val topic = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + + val kafkaParams = getKafkaParams() + + // this is the "lots of messages" case + kafkaTestUtils.sendMessages(topic, sent) + var sentCount = sent.values.sum + + val rdd = KafkaUtils.createRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, 0, sentCount)), preferredHosts) + + val ranges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum + + assert(rangeCount === sentCount, "offset range didn't include all sent messages") + assert(rdd.map(_.offset).collect.sorted === (0 until sentCount).toArray, + "didn't get all sent messages") + + // this is the "0 messages" case + val rdd2 = KafkaUtils.createRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, sentCount, sentCount)), preferredHosts) + + // shouldn't get anything, since message is sent after rdd was defined + val sentOnlyOne = Map("d" -> 1) + + kafkaTestUtils.sendMessages(topic, sentOnlyOne) + + assert(rdd2.map(_.value).collect.size === 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 = KafkaUtils.createRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, sentCount, sentCount + 1)), preferredHosts) + + // send lots of messages after rdd was defined, they shouldn't show up + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) + + assert(rdd3.map(_.value).collect.head === sentOnlyOne.keys.head, + "didn't get exactly one message") + } + + test("executor sorting") { + val kafkaParams = new ju.HashMap[String, Object](getKafkaParams()) + kafkaParams.put("auto.offset.reset", "none") + val rdd = new KafkaRDD[String, String]( + sc, + kafkaParams, + Array(OffsetRange("unused", 0, 1, 2)), + ju.Collections.emptyMap[TopicPartition, String](), + true) + val a3 = ExecutorCacheTaskLocation("a", "3") + val a4 = ExecutorCacheTaskLocation("a", "4") + val b1 = ExecutorCacheTaskLocation("b", "1") + val b2 = ExecutorCacheTaskLocation("b", "2") + + val correct = Array(b2, b1, a4, a3) + + correct.permutations.foreach { p => + assert(p.sortWith(rdd.compareExecutors) === correct) + } + } +} diff --git a/pom.xml b/pom.xml index e2730ee1c74c0..9f3d7f003584c 100644 --- a/pom.xml +++ b/pom.xml @@ -109,6 +109,8 @@ launcher external/kafka-0-8 external/kafka-0-8-assembly + external/kafka-0-10 + external/kafka-0-10-assembly diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4c01ad3c33712..8e3dcc2f38875 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,9 +44,9 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( - streaming, streamingFlumeSink, streamingFlume, streamingKafka + streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingKafka010 ) = Seq( - "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka-0-8" + "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka-0-8", "streaming-kafka-0-10" ).map(ProjectRef(buildLocation, _)) val allProjects@Seq( @@ -61,8 +61,8 @@ object BuildCommons { Seq("yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKinesisAslAssembly) = - Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kinesis-asl-assembly") + val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = + Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") .map(ProjectRef(buildLocation, _)) val copyJarsProjects@Seq(assembly, examples) = Seq("assembly", "examples") @@ -352,7 +352,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, sketch, mllibLocal + unsafe, tags, sketch, mllibLocal, streamingKafka010 ).contains(x) } @@ -608,7 +608,7 @@ object Assembly { .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) => - if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-0-8-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { + if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-0-8-assembly") || mName.contains("streaming-kafka-0-10-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { // This must match the same name used in maven (see external/kafka-0-8-assembly/pom.xml) s"${mName}-${v}.jar" } else { From c8a7c23054209db5474d96de2a7e2d8a6f8cc0da Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 29 Jun 2016 23:38:19 -0700 Subject: [PATCH 0858/1470] [SPARK-16256][DOCS] Minor fixes on the Structured Streaming Programming Guide Author: Tathagata Das Closes #13978 from tdas/SPARK-16256-1. (cherry picked from commit 2c3d96134dcc0428983eea087db7e91072215aea) Signed-off-by: Tathagata Das --- .../structured-streaming-programming-guide.md | 44 ++++++++++--------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 9ed06be62b1a5..593256603f92a 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -459,7 +459,7 @@ val csvDF = spark .readStream .option("sep", ";") .schema(userSchema) // Specify schema of the parquet files - .csv("/path/to/directory") // Equivalent to format("cv").load("/path/to/directory") + .csv("/path/to/directory") // Equivalent to format("csv").load("/path/to/directory") {% endhighlight %}
    @@ -486,7 +486,7 @@ Dataset[Row] csvDF = spark .readStream() .option("sep", ";") .schema(userSchema) // Specify schema of the parquet files - .csv("/path/to/directory"); // Equivalent to format("cv").load("/path/to/directory") + .csv("/path/to/directory"); // Equivalent to format("csv").load("/path/to/directory") {% endhighlight %}
    @@ -513,7 +513,7 @@ csvDF = spark \ .readStream() \ .option("sep", ";") \ .schema(userSchema) \ - .csv("/path/to/directory") # Equivalent to format("cv").load("/path/to/directory") + .csv("/path/to/directory") # Equivalent to format("csv").load("/path/to/directory") {% endhighlight %}
    @@ -522,10 +522,10 @@ csvDF = spark \ These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. ## Operations on streaming DataFrames/Datasets -You can apply all kinds of operations on streaming DataFrames/Datasets - ranging from untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the SQL programming guide for more details. Let’s take a look at a few example operations that you can use. +You can apply all kinds of operations on streaming DataFrames/Datasets - ranging from untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the [SQL programming guide](sql-programming-guide.html) for more details. Let’s take a look at a few example operations that you can use. ### Basic Operations - Selection, Projection, Aggregation -Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are discussed later in this section. +Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are [discussed later](#unsupported-operations) in this section.
    @@ -618,7 +618,7 @@ df.groupBy("type").count()
    ### Window Operations on Event Time -Aggregations over a sliding event-time window are straightforward with Structured Streaming. The key idea to understand about window-based aggregations are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of, window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration. +Aggregations over a sliding event-time window are straightforward with Structured Streaming. The key idea to understand about window-based aggregations are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration. Imagine the quick example is modified and the stream contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). @@ -680,7 +680,7 @@ df.groupBy("type", window("time", "1 minute")).avg("signal") Now consider what happens if one of the events arrives late to the application. For example, a word that was generated at 12:04 but it was received at 12:11. -Since this windowing is based on the time in the data, the time 12:04 should considered for windowing. This occurs naturally in our window-based grouping --the late data is automatically placed in the proper windows and the correct aggregates updated as illustrated below. +Since this windowing is based on the time in the data, the time 12:04 should be considered for windowing. This occurs naturally in our window-based grouping - the late data is automatically placed in the proper windows and the correct aggregates updated as illustrated below. ![Handling Late Data](img/structured-streaming-late-data.png) @@ -724,23 +724,25 @@ streamingDf.join(staticDf, "type", "right_join") # right outer join with a stat
    ### Unsupported Operations -However, note that all of the operations applicable on static DataFrames/Datasets are not supported in streaming DataFrames/Datasets yet. While some of these unsupported operations will be supported in future releases of Spark, there are others which are fundamentally hard to implement on streaming data efficiently. As of Spark 2.0, some of the unsupported operations are as follows +However, note that all of the operations applicable on static DataFrames/Datasets are not supported in streaming DataFrames/Datasets yet. While some of these unsupported operations will be supported in future releases of Spark, there are others which are fundamentally hard to implement on streaming data efficiently. For example, sorting is not supported on the input streaming Dataset, as it requires keeping track of all the data received in the stream. This is therefore fundamentally hard to execute efficiently. As of Spark 2.0, some of the unsupported operations are as follows -- Multiple aggregations (i.e. a chain of aggregations on a streaming DF) are not yet supported +- Multiple streaming aggregations (i.e. a chain of aggregations on a streaming DF) are not yet supported on streaming Datasets. -- Limit and take first N rows are not supported +- Limit and take first N rows are not supported on streaming Datasets. -- Distinct and sorting operations are not supported +- Distinct operations on streaming Datasets are not supported. -- Stream-batch outer joins are conditionally supported +- Sorting operations are supported on streaming Datasets only after an aggregation and in Complete Output Mode. - + Full outer join not allowed +- Outer joins between a streaming and a static Datasets are conditionally supported. - + Left outer join with a streaming DF on the left is not supported + + Full outer join with a streaming Dataset is not supported - + Right outer join with a streaming DF on the right is not supported + + Left outer join with a streaming Dataset on the left is not supported -- Stream-stream joins are not yet supported + + Right outer join with a streaming Dataset on the right is not supported + +- Any kind of joins between two streaming Datasets are not yet supported. In addition, there are some Dataset methods that will not work on streaming Datasets. They are actions that will immediately run queries and return results, which does not makes sense on a streaming Dataset. Rather those functionalities can be done by explicitly starting a streaming query (see the next section regarding that). @@ -753,7 +755,7 @@ In addition, there are some Dataset methods that will not work on streaming Data If you try any of these operations, you will see an AnalysisException like "operation XYZ is not supported with streaming DataFrames/Datasets". ## Starting Streaming Queries -Once you have defined the final result DataFrame/Dataset, all that is left is for you start the StreamingQuery. To do that, you have to use the +Once you have defined the final result DataFrame/Dataset, all that is left is for you start the streaming computation. To do that, you have to use the `DataStreamWriter` ( [Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/ [Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/ @@ -867,7 +869,7 @@ aggDF .format("memory") .start() -spark.sql("select * from aggregates).show() // interactively query in-memory table +spark.sql("select * from aggregates").show() // interactively query in-memory table {% endhighlight %}
    @@ -907,7 +909,7 @@ aggDF .format("memory") .start(); -spark.sql("select * from aggregates).show(); // interactively query in-memory table +spark.sql("select * from aggregates").show(); // interactively query in-memory table {% endhighlight %}
    @@ -947,7 +949,7 @@ aggDF\ .format("memory")\ .start() -spark.sql("select * from aggregates).show() # interactively query in-memory table +spark.sql("select * from aggregates").show() # interactively query in-memory table {% endhighlight %} @@ -1144,7 +1146,7 @@ aggDF\ - Examples: See and run the [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/sql/streaming) examples. -- Spark Summit 2016 Talk - [A Deep Dive into Structured Streaming(https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/) +- Spark Summit 2016 Talk - [A Deep Dive into Structured Streaming](https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/) From 1d274455cfa45bc63aee6ecf8dbb1f170ee16af2 Mon Sep 17 00:00:00 2001 From: zlpmichelle Date: Thu, 30 Jun 2016 00:50:14 -0700 Subject: [PATCH 0859/1470] [SPARK-16241][ML] model loading backward compatibility for ml NaiveBayes ## What changes were proposed in this pull request? model loading backward compatibility for ml NaiveBayes ## How was this patch tested? existing ut and manual test for loading models saved by Spark 1.6. Author: zlpmichelle Closes #13940 from zlpmichelle/naivebayes. (cherry picked from commit b30a2dc7c50bfb70bd2b57be70530a9a9fa94a7a) Signed-off-by: Yanbo Liang --- .../apache/spark/ml/classification/NaiveBayes.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 7c340312df3e1..c99ae30155e3f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -28,8 +28,9 @@ import org.apache.spark.ml.util._ import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes} import org.apache.spark.mllib.classification.{NaiveBayesModel => OldNaiveBayesModel} import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Dataset +import org.apache.spark.sql.{Dataset, Row} /** * Params for Naive Bayes Classifiers. @@ -275,9 +276,11 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sparkSession.read.parquet(dataPath).select("pi", "theta").head() - val pi = data.getAs[Vector](0) - val theta = data.getAs[Matrix](1) + val data = sparkSession.read.parquet(dataPath) + val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") + val Row(pi: Vector, theta: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta") + .select("pi", "theta") + .head() val model = new NaiveBayesModel(metadata.uid, pi, theta) DefaultParamsReader.getAndSetParams(model, metadata) From 6a4f4c1d751db9542ba49755e859b55b42be3236 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 30 Jun 2016 18:06:04 +0800 Subject: [PATCH 0860/1470] [SPARK-12177][TEST] Removed test to avoid compilation issue in scala 2.10 ## What changes were proposed in this pull request? The commented lines failed scala 2.10 build. This is because of change in behavior of case classes between 2.10 and 2.11. In scala 2.10, if companion object of a case class has explicitly defined apply(), then the implicit apply method is not generated. In scala 2.11 it is generated. Hence, the lines compile fine in 2.11 but not in 2.10. This simply comments the tests to fix broken build. Correct solution is pending. Author: Tathagata Das Closes #13992 from tdas/SPARK-12177. (cherry picked from commit de8ab313e1fe59f849a62e59349224581ff0b40a) Signed-off-by: Cheng Lian --- .../streaming/kafka010/JavaConsumerStrategySuite.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java index aba45f5de6782..8d7c05b5a615d 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -50,8 +50,8 @@ public void testConsumerStrategyConstructors() { JavaConverters.mapAsScalaMapConverter(offsets).asScala(); // make sure constructors can be called from java - final ConsumerStrategy sub0 = - Subscribe.apply(topics, kafkaParams, offsets); + // final ConsumerStrategy sub0 = // does not compile in Scala 2.10 + // Subscribe.apply(topics, kafkaParams, offsets); final ConsumerStrategy sub1 = Subscribe.apply(sTopics, sKafkaParams, sOffsets); final ConsumerStrategy sub2 = @@ -65,8 +65,8 @@ public void testConsumerStrategyConstructors() { sub1.executorKafkaParams().get("bootstrap.servers"), sub3.executorKafkaParams().get("bootstrap.servers")); - final ConsumerStrategy asn0 = - Assign.apply(parts, kafkaParams, offsets); + // final ConsumerStrategy asn0 = // does not compile in Scala 2.10 + // Assign.apply(parts, kafkaParams, offsets); final ConsumerStrategy asn1 = Assign.apply(sParts, sKafkaParams, sOffsets); final ConsumerStrategy asn2 = From 56207fc3b26cdb8cb50ce460eeab32c06a81bb44 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Thu, 30 Jun 2016 21:56:34 +0800 Subject: [PATCH 0861/1470] [SPARK-16071][SQL] Checks size limit when doubling the array size in BufferHolder ## What changes were proposed in this pull request? This PR Checks the size limit when doubling the array size in BufferHolder to avoid integer overflow. ## How was this patch tested? Manual test. Author: Sean Zhong Closes #13829 from clockfly/SPARK-16071_2. (cherry picked from commit 5320adc863ca85b489cef79f156392b9da36e53f) Signed-off-by: Wenchen Fan --- .../expressions/codegen/BufferHolder.java | 16 +++++++- .../codegen/BufferHolderSuite.scala | 39 +++++++++++++++++++ 2 files changed, 53 insertions(+), 2 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolder.java index af61e2011f400..0e4264fe8dfb5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolder.java @@ -45,7 +45,13 @@ public BufferHolder(UnsafeRow row) { } public BufferHolder(UnsafeRow row, int initialSize) { - this.fixedSize = UnsafeRow.calculateBitSetWidthInBytes(row.numFields()) + 8 * row.numFields(); + int bitsetWidthInBytes = UnsafeRow.calculateBitSetWidthInBytes(row.numFields()); + if (row.numFields() > (Integer.MAX_VALUE - initialSize - bitsetWidthInBytes) / 8) { + throw new UnsupportedOperationException( + "Cannot create BufferHolder for input UnsafeRow because there are " + + "too many fields (number of fields: " + row.numFields() + ")"); + } + this.fixedSize = bitsetWidthInBytes + 8 * row.numFields(); this.buffer = new byte[fixedSize + initialSize]; this.row = row; this.row.pointTo(buffer, buffer.length); @@ -55,10 +61,16 @@ public BufferHolder(UnsafeRow row, int initialSize) { * Grows the buffer by at least neededSize and points the row to the buffer. */ public void grow(int neededSize) { + if (neededSize > Integer.MAX_VALUE - totalSize()) { + throw new UnsupportedOperationException( + "Cannot grow BufferHolder by size " + neededSize + " because the size after growing " + + "exceeds size limitation " + Integer.MAX_VALUE); + } final int length = totalSize() + neededSize; if (buffer.length < length) { // This will not happen frequently, because the buffer is re-used. - final byte[] tmp = new byte[length * 2]; + int newLength = length < Integer.MAX_VALUE / 2 ? length * 2 : Integer.MAX_VALUE; + final byte[] tmp = new byte[newLength]; Platform.copyMemory( buffer, Platform.BYTE_ARRAY_OFFSET, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala new file mode 100644 index 0000000000000..c7c386b5b838a --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSuite.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.UnsafeRow + +class BufferHolderSuite extends SparkFunSuite { + + test("SPARK-16071 Check the size limit to avoid integer overflow") { + var e = intercept[UnsupportedOperationException] { + new BufferHolder(new UnsafeRow(Int.MaxValue / 8)) + } + assert(e.getMessage.contains("too many fields")) + + val holder = new BufferHolder(new UnsafeRow(1000)) + holder.reset() + holder.grow(1000) + e = intercept[UnsupportedOperationException] { + holder.grow(Integer.MAX_VALUE) + } + assert(e.getMessage.contains("exceeds size limitation")) + } +} From 98056a1f8683385599f194a4b963769e3342bff3 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 30 Jun 2016 22:10:56 +0800 Subject: [PATCH 0862/1470] [BUILD] Fix version in poms related to kafka-0-10 self explanatory Author: Tathagata Das Closes #13994 from tdas/SPARK-12177-1. --- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index f2468d1cbac7f..59f41f1e17f30 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 50395f6d14453..26965612cc0ab 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0-SNAPSHOT + 2.0.1-SNAPSHOT ../../pom.xml From f17ffef38b4749b6b801c198ec207434a4db0c38 Mon Sep 17 00:00:00 2001 From: Sital Kedia Date: Thu, 30 Jun 2016 10:53:18 -0700 Subject: [PATCH 0863/1470] =?UTF-8?q?[SPARK-13850]=20Force=20the=20sorter?= =?UTF-8?q?=20to=20Spill=20when=20number=20of=20elements=20in=20th?= =?UTF-8?q?=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Force the sorter to Spill when number of elements in the pointer array reach a certain size. This is to workaround the issue of timSort failing on large buffer size. Tested by running a job which was failing without this change due to TimSort bug. Author: Sital Kedia Closes #13107 from sitalkedia/fix_TimSort. (cherry picked from commit 07f46afc733b1718d528a6ea5c0d774f047024fa) Signed-off-by: Davies Liu --- .../shuffle/sort/ShuffleExternalSorter.java | 10 +++++--- .../unsafe/sort/UnsafeExternalSorter.java | 23 ++++++++++++++++--- .../sort/UnsafeExternalSorterSuite.java | 3 +++ .../execution/UnsafeExternalRowSorter.java | 2 ++ .../UnsafeFixedWidthAggregationMap.java | 3 +++ .../sql/execution/UnsafeKVExternalSorter.java | 8 +++++-- .../spark/sql/execution/WindowExec.scala | 2 ++ .../datasources/WriterContainer.scala | 5 +++- .../joins/CartesianProductExec.scala | 2 ++ .../execution/streaming/FileStreamSink.scala | 5 +++- .../UnsafeKVExternalSorterSuite.scala | 4 +++- .../spark/sql/hive/hiveWriterContainers.scala | 5 +++- 12 files changed, 60 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 014aef86b5cc6..696ee73a76e02 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -72,7 +72,10 @@ final class ShuffleExternalSorter extends MemoryConsumer { private final TaskContext taskContext; private final ShuffleWriteMetrics writeMetrics; - /** Force this sorter to spill when there are this many elements in memory. For testing only */ + /** + * Force this sorter to spill when there are this many elements in memory. The default value is + * 1024 * 1024 * 1024, which allows the maximum size of the pointer array to be 8G. + */ private final long numElementsForSpillThreshold; /** The buffer size to use when writing spills using DiskBlockObjectWriter */ @@ -114,7 +117,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; this.numElementsForSpillThreshold = - conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MAX_VALUE); + conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", 1024 * 1024 * 1024); this.writeMetrics = writeMetrics; this.inMemSorter = new ShuffleInMemorySorter( this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); @@ -372,7 +375,8 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p // for tests assert(inMemSorter != null); - if (inMemSorter.numRecords() > numElementsForSpillThreshold) { + if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { + logger.info("Spilling data because number of spilledRecords crossed the threshold " + numElementsForSpillThreshold); spill(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index e14a23f4a6a83..8a980d4843899 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -27,6 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.memory.MemoryConsumer; @@ -59,6 +60,13 @@ public final class UnsafeExternalSorter extends MemoryConsumer { /** The buffer size to use when writing spills using DiskBlockObjectWriter */ private final int fileBufferSizeBytes; + /** + * Force this sorter to spill when there are this many elements in memory. The default value is + * 1024 * 1024 * 1024 / 2 which allows the maximum size of the pointer array to be 8G. + */ + public static final long DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD = 1024 * 1024 * 1024 / 2; + + private final long numElementsForSpillThreshold; /** * Memory pages that hold the records being sorted. The pages in this list are freed when * spilling, although in principle we could recycle these pages across spills (on the other hand, @@ -88,9 +96,10 @@ public static UnsafeExternalSorter createWithExistingInMemorySorter( PrefixComparator prefixComparator, int initialSize, long pageSizeBytes, + long numElementsForSpillThreshold, UnsafeInMemorySorter inMemorySorter) throws IOException { UnsafeExternalSorter sorter = new UnsafeExternalSorter(taskMemoryManager, blockManager, - serializerManager, taskContext, recordComparator, prefixComparator, initialSize, + serializerManager, taskContext, recordComparator, prefixComparator, initialSize, numElementsForSpillThreshold, pageSizeBytes, inMemorySorter, false /* ignored */); sorter.spill(Long.MAX_VALUE, sorter); // The external sorter will be used to insert records, in-memory sorter is not needed. @@ -107,9 +116,10 @@ public static UnsafeExternalSorter create( PrefixComparator prefixComparator, int initialSize, long pageSizeBytes, + long numElementsForSpillThreshold, boolean canUseRadixSort) { return new UnsafeExternalSorter(taskMemoryManager, blockManager, serializerManager, - taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, null, + taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, numElementsForSpillThreshold, null, canUseRadixSort); } @@ -122,6 +132,7 @@ private UnsafeExternalSorter( PrefixComparator prefixComparator, int initialSize, long pageSizeBytes, + long numElementsForSpillThreshold, @Nullable UnsafeInMemorySorter existingInMemorySorter, boolean canUseRadixSort) { super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); @@ -143,6 +154,7 @@ private UnsafeExternalSorter( this.inMemSorter = existingInMemorySorter; } this.peakMemoryUsedBytes = getMemoryUsage(); + this.numElementsForSpillThreshold = numElementsForSpillThreshold; // Register a cleanup task with TaskContext to ensure that memory is guaranteed to be freed at // the end of the task. This is necessary to avoid memory leaks in when the downstream operator @@ -372,6 +384,12 @@ private void acquireNewPageIfNecessary(int required) { public void insertRecord(Object recordBase, long recordOffset, int length, long prefix) throws IOException { + assert(inMemSorter != null); + if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { + logger.info("Spilling data because number of spilledRecords crossed the threshold " + numElementsForSpillThreshold); + spill(); + } + growPointerArrayIfNecessary(); // Need 4 bytes to store the record length. final int required = length + 4; @@ -383,7 +401,6 @@ public void insertRecord(Object recordBase, long recordOffset, int length, long pageCursor += 4; Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length); pageCursor += length; - assert(inMemSorter != null); inMemSorter.insertRecord(recordAddress, prefix); } diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index 2cae4beb4c777..960698f4ebac6 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -176,6 +176,7 @@ private UnsafeExternalSorter newSorter() throws IOException { prefixComparator, /* initialSize */ 1024, pageSizeBytes, + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD, shouldUseRadixSort()); } @@ -399,6 +400,7 @@ public void forcedSpillingWithoutComparator() throws Exception { null, /* initialSize */ 1024, pageSizeBytes, + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD, shouldUseRadixSort()); long[] record = new long[100]; int recordSize = record.length * 8; @@ -435,6 +437,7 @@ public void testPeakMemoryUsed() throws Exception { prefixComparator, 1024, pageSizeBytes, + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD, shouldUseRadixSort()); // Peak memory should be monotonically increasing. More specifically, every time diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 37fbad47c1458..45e5be565c131 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -74,6 +74,8 @@ public UnsafeExternalRowSorter( prefixComparator, /* initialSize */ 4096, pageSizeBytes, + SparkEnv.get().conf().getLong("spark.shuffle.spill.numElementsForceSpillThreshold", UnsafeExternalSorter + .DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), canUseRadixSort ); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java index 1f1b5389aa7d4..3705291e1fe28 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java @@ -29,6 +29,7 @@ import org.apache.spark.unsafe.KVIterator; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.map.BytesToBytesMap; +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter; /** * Unsafe-based HashMap for performing aggregations where the aggregated values are fixed-width. @@ -246,6 +247,8 @@ public UnsafeKVExternalSorter destructAndCreateExternalSorter() throws IOExcepti SparkEnv.get().blockManager(), SparkEnv.get().serializerManager(), map.getPageSizeBytes(), + SparkEnv.get().conf().getLong("spark.shuffle.spill.numElementsForceSpillThreshold", UnsafeExternalSorter + .DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), map); } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index bb823cd07be5e..8432ce9987e31 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -54,8 +54,9 @@ public UnsafeKVExternalSorter( StructType valueSchema, BlockManager blockManager, SerializerManager serializerManager, - long pageSizeBytes) throws IOException { - this(keySchema, valueSchema, blockManager, serializerManager, pageSizeBytes, null); + long pageSizeBytes, + long numElementsForSpillThreshold) throws IOException { + this(keySchema, valueSchema, blockManager, serializerManager, pageSizeBytes, numElementsForSpillThreshold, null); } public UnsafeKVExternalSorter( @@ -64,6 +65,7 @@ public UnsafeKVExternalSorter( BlockManager blockManager, SerializerManager serializerManager, long pageSizeBytes, + long numElementsForSpillThreshold, @Nullable BytesToBytesMap map) throws IOException { this.keySchema = keySchema; this.valueSchema = valueSchema; @@ -88,6 +90,7 @@ public UnsafeKVExternalSorter( prefixComparator, /* initialSize */ 4096, pageSizeBytes, + numElementsForSpillThreshold, canUseRadixSort); } else { // The array will be used to do in-place sort, which require half of the space to be empty. @@ -132,6 +135,7 @@ public UnsafeKVExternalSorter( prefixComparator, /* initialSize */ 4096, pageSizeBytes, + numElementsForSpillThreshold, inMemSorter); // reset the map, so we can re-use it to insert new records. the inMemSorter will not used diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala index 97bbab65af1de..e01094a7c8e3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala @@ -345,6 +345,8 @@ case class WindowExec( null, 1024, SparkEnv.get.memoryManager.pageSizeBytes, + SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), false) rows.foreach { r => sorter.insertRecord(r.getBaseObject, r.getBaseOffset, r.getSizeInBytes, 0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index f56b50a54385a..9a0b46c1a4a5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter /** A container for all the details required when writing to a table. */ @@ -389,7 +390,9 @@ private[sql] class DynamicPartitionWriterContainer( StructType.fromAttributes(dataColumns), SparkEnv.get.blockManager, SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) + TaskContext.get().taskMemoryManager().pageSizeBytes, + SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) while (iterator.hasNext) { val currentRow = iterator.next() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 88f78a7a73bce..3a0b6efdfc910 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -49,6 +49,8 @@ class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numField null, 1024, SparkEnv.get.memoryManager.pageSizeBytes, + SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), false) val partition = split.asInstanceOf[CartesianPartition] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index efb04912d76bf..117d6672ee2f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, PartitioningUtils} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter object FileStreamSink { // The name of the subdirectory that is used to store metadata about which files are valid. @@ -209,7 +210,9 @@ class FileStreamSinkWriter( StructType.fromAttributes(writeColumns), SparkEnv.get.blockManager, SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) + TaskContext.get().taskMemoryManager().pageSizeBytes, + SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) while (iterator.hasNext) { val currentRow = iterator.next() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala index 03d4be8ee528e..3d869c77e9608 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{InterpretedOrdering, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter /** * Test suite for [[UnsafeKVExternalSorter]], with randomly generated test data. @@ -123,7 +124,8 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext { metricsSystem = null)) val sorter = new UnsafeKVExternalSorter( - keySchema, valueSchema, SparkEnv.get.blockManager, SparkEnv.get.serializerManager, pageSize) + keySchema, valueSchema, SparkEnv.get.blockManager, SparkEnv.get.serializerManager, + pageSize, UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD) // Insert the keys and values into the sorter inputData.foreach { case (k, v) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 794fe264ead5d..e65c24e6f125c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableJobConf +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter /** * Internal helper class that saves an RDD using a Hive OutputFormat. @@ -280,7 +281,9 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( StructType.fromAttributes(dataOutput), SparkEnv.get.blockManager, SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) + TaskContext.get().taskMemoryManager().pageSizeBytes, + SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) while (iterator.hasNext) { val inputRow = iterator.next() From 03008e049a366bc7a63b3915b42ee50320ac6f34 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 30 Jun 2016 14:01:34 -0700 Subject: [PATCH 0864/1470] [SPARK-16256][DOCS] Fix window operation diagram Author: Tathagata Das Closes #14001 from tdas/SPARK-16256-2. (cherry picked from commit 5d00a7bc19ddeb1b5247733b55095a03ee7b1a30) Signed-off-by: Tathagata Das --- docs/img/structured-streaming-late-data.png | Bin 138931 -> 138226 bytes docs/img/structured-streaming-window.png | Bin 128930 -> 132875 bytes docs/img/structured-streaming.pptx | Bin 1105315 -> 1105413 bytes .../structured-streaming-programming-guide.md | 2 +- 4 files changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/img/structured-streaming-late-data.png b/docs/img/structured-streaming-late-data.png index 5276b4786826236eb9ce043c9373870350aea7ff..2283f6782f380bcb352f294635aaf4165e1611dc 100644 GIT binary patch delta 31911 zcmZsCcUTkM*1aGg7C-?-L5k8rK&jF}DN+JRm9ElDKq;X!A}TK+O?nNzLx4b#8buKV z=`}=%(pv~UKp^=A-}`;{z2E))Gf(DuCNp!^*=O&y*4YDZqup+#eN|8K=`SGaHg@g+ zJ}(gUl)ki-D~f`f@5A}M-SogOn8Gr?@!P=c0MZM@t%zkbUgow7;@j^1Sr`|Fkd z#{<^{_|=dz^3uBF{fAK`!`;uoBqwwD1*KcFOd^0;Wo+3fspluZI3s%YY{xkPqRrOx z&b6yd;f(`kng_2!Zt+yPsgCL6Bh*PCq3xyT zgCVneWcX~$-U^&CmLE-rZc70Neb^I(NxNp2W#!EEAc?y1`ugNJ1i7V?6mdjOM|9;=9B8x%DBmZOUY-*GpbI72S$w$j` z3J{~8x{yOpGV(aGvtMQ;x8swynsjH$FO`@&2`xFgsimt^_G#ZkXl>=Uk0OW zl1otCULOxqznY_DxkBERS+O2Qa6W*Mz8m|glP)g7nFow_-F_uzpAP)_n#dyl_+|Fw zTt-x;P<7>f%_06!Wz${U1quqP%PMF7bx3GC2XH+MGrh6M%|_X0{0vcbAmo-jGa0Jf|wa${O%7zMt_aGg--Oa|n!v;Tahb*pPnv|5ZT(8djeMqc4 zJTJsSDY7k~^Z4&K>rwypRLZA(cJmIaf^~qc$$H+hux&nhQ2D;wk%^z#nFI|MDRn<4 z9a0w_{<+VM^XtV{?XCjoUvcPdznp`pYG>NIYUk|W+ljU8O#Derg#|%#vPLpLP^k!ERzTP9g2q_!aEHQQ{vr>kw5T%y-c` z**|hNC<@Gcvh{3)+3sPyZ|8WgbYk8iGjQYL-QR}aSmKJgPl2$%bJtSV&!p9COIO{t zN!(`0toCQweXE!G`TEsa<^EN(%~4FuOyvLbj`#f;#VL4~FN7JzDt_Sz`L$(G|5oD^ z(FLx}t#i=H_^ODMhej^kl(ftSug<4Z&KgSn8G0#IsDWE7IlM&xrLyh(*_)kGSCMAN zakICg6q&^gvWn7#sLajEz2jp~?{A<=X&btg9I{L7+V4HK_1f4wKHR~c_$k*=Bky!K z#ooPkse|wR$%UZIYho7eIw_)bK@u*uZ!c195k7f3y~U_eK!CDP2^J9x=BF`-W>>%Z z@sLan&Qd1x4weIT%{rccFuq)~BL49nR|MDEg;Fi!@IMdM4(p6&K}XCf*JY&IAi6k$ zJ3@gUd63;mO&;)Y$J^-jX5{o!F?!Bt*{Lh;;HmoB0Y1IDY5YuIzXHy9ZM4Z~*efzp z>9beAAb^mu4MGQ2BV$9PpM3z!JY{RyL$j=rxw@t8hLfDbros)^*3$a zkzd|CThI~~n$p3S>76pnXRz#OYA!udlkVG=-<;UzyVwZdBvspcD;{7kVa%jTagPoH zHKH#IFi`MW^~}i)Y5!w!T^2YSMpE3aS2QGDFYb&^b$06Yr8ft_W3>m%9pxJMCLp);tYslQ{&rq3RqS2Hum70?&o?>}KUOCP{~eKJW^2 zuYR`8`bgwlm@Z5Lnf;dJ2Xx|6bzu=p3ymXJJD6S{#l9)gtfvawW+I34arF(6KjHkZ zG!YhmVST5it*;+$Fx62V~sjg(M@$|VGgP5 zWHWHjbnjmY&~cet;^$%M-fh>i$r$yz(!DFUgEl_^%#L%u?*{tYgvwjrQn*Ei-ak-H z=dyq@#msNR(24ig{;Z^b%{&I85~ z*65Fw>~i&u)1h#<;@+gh%N6CmqwguWaVb#|cX?HmJsRd!lYTOCGe{>cmf*TNfr7*x z(@I_p%MIlu&;AC4mf%@}G;76d~ao@v* z+2I~AKU1dF5wjy3^r^1e&C~7b3OUbw z<9iz_kMiT$jVTVJO-et_w2;rvqNnNRbSa-6B!Mve%nfo}pzc-|0B!S}((74(&Uyh} zIks>hL$){caDip5Rxj>j@K82~8~(3LLWlC!`Jd0FH-D>qap*u48+z!llO72Y%|2hz z$Il6>If>w~M>G5!|JJVDvvGi{+CNS!!M>BPt`!zL%A7tJ$Yg77L0!vKY;v+sH}@CgmUS;(3^jjhR1)mc>uWt8>?L&X zeaPWh9bsB`pIvL6g+R*{4?Tl>@qHj}_#c=6NBeZ&mg1YqB*=D90Q_baxN@>?bEG8M0f7Fj-$aYCD5Sqs`~Ay$gwxB%0`)1tn8+ly+p4f_Lo0#*fgfN_iHIg%o0!? z;QkEDohdB*b}K22d)Kp(!rZvb+c?wO%DGo5%W5p4Mio-HtGKH zjV9B1VwC)OdD^4|m0~2rF@zZSn2BV4xc6w$^4T$VH!}V)w+7l9c03>_Hqw+QK~Bzy z9WnWdYs+z0iJ_=5$e3@cW-RtTV1KTK$Vn0>ouvjCEF@d^j_&V7u6s2F>&Y>o#=Gbg$^0S>6ab=zb0qS9e=Y4Jdq8WOS}ic zFp(lJjglM(+xHh(Y^CPu&>GUxP4$ZJrD3t5eZ|4Vx9pG~x3pU77cJZ-Z3sA*MjOEL z%NBd$anZg$+)D=l zm6wSkwTf~pKa?kT23qZHb#qv?V)gDLPriMebWbo5JMo+RhCn@A>sEuTtCFlwb`uA7 z&t0_aZ5ZEh0!&MnCMID0Ay!lzg(H2A!Zv&Fs;al_8zgCUM~i8^y%oOix! zGF4$xGS#b1%9Rf7J}KBr9<(bP{xB0)|KbK~ZkGh{A7QNfxWgEmvAD|Yiytygh$<3Q2f$hZ_uKKNfm? zvFdfC5}$ihb+0+JJ!HM%p68@q3teo|p62gMC|?8nAT)bpZErtvnCL$qnQ-EQ3H4MC zCWV7rQ5v9}h;Z6BCR}I$b1qaq+K*Rf3w?-{!Jt|LcEB+l6*8#DBy5)CmN)#!Z-(J| zUE9gt2Ve?H^a?D@nwS$y!!-v_bcbRk7Hm}=MdFjW#{4meWb5~Tj zs!kP6WcUB3-#Z=1Mb2{GLQ`VnzQ6p==TFOTdNX0|4olX-dh;)!?QD}WwjmXoy?w*nODPRN zZJ;|!8tuuB*GqL)48DCC5@vN;R`1N1h;(|EkUWl7L2;nBUz7DRJ=>Z>P8MXOL#BMz z+MoNe6(qk>Qe~%W4}9UXH~H>->OC=}V+5z)lie52%HD^uZx9lam%_xakco`0D083M z|41FO=m}dC+}s)1S^vtd57f5pPmk>XEFtQHC6--N9a1A(H)qUL-D@7>G);i6P|Zv@*(^yDCE|v?QlqfYNCn$VQ~3ji`8>jCi2YD ze1Tqc(vIJy%I2eTy12G2RSiHb7P%gPZ}XFw!3^_=oqKo(h+o507S(g@d^kJ-JzCOV~s+WL1{mX-ty@ ztXTuPqdEgIx)vT^cg8G!8{n^(72aTE3_0TxNfe*7E$4U_9|s!5iY0?!N^7W(GMGq( zMG+I|xA&}WZAX=ye*PyjYp7vfoxXE7xp7WID$m?E{n!qItb*_E`YHximie)NO|fGO ztD&9$lL@w}p9_Y{pBz;qKOZGv+5uz56JiSvJ8~W>hzWdYPP42iO3t*3+kbsy*y~T3 z)D$`Ticl`!sVK8`%Rg!uRMxu##XY)tL7rSr8^{4}p7{756MHJXY@hz_P>1x?Iz1`r z&tCt4v`yRJ;{V?NvCA`JYLg!t9h3lDA_Qy<>>rPaWpDyQlWYzvwzdH03r zT$;01Wd|tDJ+Q2Ie#|osoU`aB&IO!OaQDPeHktYCeXOLsJ?;y3r7Hrx4qcQiB*;Il zzPT?Gw6^z>4tMB$|HDq%RrOz+=aSul}PH>MY--~Lbd@;G2|~HHmT?JCo5tlnu%;mEo=9(hywkD zjo?AVh`c3ZIE-eu4%enQgIGrf%>M4SQ({@US#+r>SW_Ei(;uVa&HU)(#mwTQ<g99F7qgljimvRC>dkotS2X0cV*W%)$+?D+*2&qdq_LDq<~1 zp1_mu@U&Pc0v$99b)358&>-VChCO^^%d>5O$nu^mO7sCdNT_rYB_NY~H`IYUl2B zW?53n{XkVNR0iL@NWyA6SW?m)L)-}AUsG(;}b{m6(m z^D$J}oeA(qBc6cG(Dum`C`o=%7)&?g;N`U4CsV-EZtSDo+sQ-s4jvFQqS3_El6P*E zMa2B6Ut7pC9#TT4?ei*&`u>mZg5v!dq&CH4qVk+xl^@zF02?Zn#g^KwXBcM0F@4x^ z%~a~n{x45L*7IpO$IyYVDI#AX&8OAfh}80-Z%^Q>=hBCwXl!gFkAQhV*mTE)=vt$;xlz?CK&vR0@OhKb=}EU+_wGeTbi<$8opo&NsFN2jqbvs}_8=&fI{dWzd=wq3uO*@a-U4?L2zO3TGVw({Q0KS-s> zrQkU`HRP4>nGK+gRH=xs4|n{xhCKaM#5^gmH9wZfQJ*4W!OsVeQyHg}SkjM9O22yo zA~^0BR`#YEg;`im4gOr>pK&m`wt?KSYV~>cFsOjd#H1l6lp(+C;5|!ec@@^p#N=fp zOgEros@|O~^&3596YZ!4_+c%JavT3H65ywc>3dm#Ky2W>=L^qOI+%Qna8JEs{j)T+ z*@8`4n};Io0y8bXv5``Y*7`r3k`J_ZQvJtqi+WnR&pE?dsT{@9o^c5CCgJ7{qpOp> zVzxJ&OiT(e{VU4)eJvJ3DqN3qg8_ZUpbw#>7TLrRrwh;M``V-#j}&R2Nw_dlQnS$F zp&87rgsg-u_VeAgWH0ZSu8eG4vjIX@uQa1f4(L?y|11Y8lqL<)yX#xFoY!z#j>xYi zTyIrkDNV*LWXyD?^`m`lr9wADV#nUqrOB1HBX$tUU;G|{tAH$w2bF*pZq@j^Oi$FBP`4V1Oql^dR%pZnfZf&24Xgz|R(3j4v}Rn@ z7`aas#q;TnPC~9M-9N?u8?#?iA+|roTdN8j$f`}{KDBE_xrW zYPHR2H+sJGet_qRQ-rD##$DE#?|9qY-Ub1+lZjoJM){81{FOVJDJeC{}?$h)0 zDlf=^O68;lB0u69t3ep$93}`OJzZlbMcN%>*P$}j!4V#2jR+rX11xliJ$Rx-!Dq#Dy|<#yYIv_))z1+ ze9&Ct>`vV93%3i=2NWf?0rHwpEWULsfsMoll`Mq_9ehhm!+FVA8Ti- zyvlUpO`nnJ1L_Gr7sEbWsOcywJkGlLPu=%-87XxSdq^)onG>>-{^;gi=dPsdBS@&n zLkP$7iD@737yD=@8%UD_w%F36rq=G;6xy>byDoS=Om|?#K_j*lF~A?fAq&$-_&cK! zRnx&}kR0ZI5O!>W+vV+nelhkEM+aZ3oVJS*EpE}!sTdFPUJ*M{;nF=#Tq~}F*G@MA zo)l8H1>)I@J2~JsU6Jp}mt(UO@MvmDfsL07XcaVH@W{b2Va*kvIE{G12-Aaw{RcP>WQwvU3(&aEOXx(u@K}unz_AW2Yv&1KwlVFdwLkz+rZVy|E;j~Mo zZ;{-thH}#@4j&J>)INjd1)=+7rWrFwYeIgtBjN3BGA@&G)0@elOpi%1=}0j7B|fFS z_{~Qa5ef5G0D+*e#+hKWG`7*)N?G(^=PhKh%n5bi*VR?z)Vwnnr#wZD^Qxt0#l&kX zD7cgT84PBHaq;pS&~x@@2w8L7ZqrhA5Vk0T1)Dr!iy$cy3PD-NkySOb7Z7vOXV+VB z#Zp$)*?!Zg^1aEAUpITjkdyNR{OIjoKPl)(olAfex10N>UN}=CO7l$E1r~C7>F_+$ z9QSWR_dmaqT0Z$JYP80QC;dcM^A-v%P0f`=d@pWM@w-P2&Ud>Sqde*{1yi} zj_jhIId9um=i@Sh1GZS;LG5?nak(3gT`|pMP(CoIFr+DNy;)I>DBoLcYY%95o(xFM zmMmi=J*joAN@M_x(+8VU5Ohkr+c~a0^~xPPfH+|MrI)IYML?-YF}4` zr)k3uuwZDGx7`4|t=GNT0 zUrTdCnjBz{$@EC=3g!NvocTbo=Ed&^^2hPqmEKGJ5NymuVTh0MRdd9)q**xv5EIN^qc^=6+4Q&dp^@rTwXvuSRUPYX2%YG<}e|NfJ z4O|xbv!gi_!RdDK+MRmzMsvf^EC0H0HiUsS_qEUy@M4@qB7468xGPM4dCRD6J&8%b zK|T7Kde6bvuUlQmpAjp_25{}dwHsqb-MWA3#&s|a-HlSsS9JA?kdqitxOtL!20WS+H(ov;np65c>wpQI1ED<#;DYaTP~vja+ly-( zaqrZ~0Orn8rM)QosG}G0PfGc>%@X#4_Gb`21fktH#yUF}r}WIsD$wBzu~{)RWZ)Zn zUQF%tHHe3*`wQ|X-fdfhu3^BiZw(P!rQk}*w*7QobKU%Kffc; zuIC7S9zm!;x4k;gcH+23n}wZlRjc8!5M;)IzsfQ2Crwu?k~d?B-&|v?^=1IGc?3Os z{&#LV=$dn1zw$W%$z#;{R_=hgd9VZ2oGx z+Un?TsMUC~fV0)hb|to_a=lj8SpUMHTt_FfYR4`Qxs>1YVeASdlUIiYoy;8zW(7JY?KdCsnk89PK|G`;G*?`aXb??5$mxs2BS(op9|tIpDTF(}c)ozB*|+pU%Iy z!7kW4jPecyw<>S+0xLU%QOCeVC$y9p^d_coifs3yaBt`4m{1RMx2a}1HJ${dQotx! zWcrLhndMPQp=Q;x<^1ZZ5Hoe=&L*#8V4uuAPxhi9ZNU#pNC-4DHaLrxf~tf66L)8a zAISO2A8!2b-MT-gZ}iFQXVaR=;tx4K=fFo^)9Kc!FJTlsR1?P2UB68hB+vk~bP}tPe)Vl{Vaf)5#WN27J=>uO3jBk<-wX)c8nN6Ur_9_s(L zfSjJXB-xPlLNbwckn+DDlY5D}yuE&E3d+PvY}#kxDXfiF*{3A-q*$zBsog5d`HaW- zfR~TCsW~>V7Cb7I!P+t)4X>4qpiHb`7rB;H z@oB`dw+w5tFcmsAeN1g3NfI4?nDCiJKKFoRN{0pxwG3cj)Bbu6}>v zvcPNdNYqspnaM(<;N}&8_rkK+2yFP*$av>E;W-?`>0>a^Os?z7|%^i;NOJNNVM?QAP7Z%xzXl3umoA-J+Dkc|P2WC zjdf_n$M)5|Ef(Nre@o!#Q1;nA4?SN|x@&&R_K>Cv>tzaVK}yNyxhGpwt%0vlXLC}7 zMwi=oGb)!Ay4Ue#+x|z7;1ZH|TQO(g#K=R*!O;5t#+F#@Tvt8t*Z#oBB27C^6?|l{ z^&)!-+ni)es%X|X28iKFb(Kmoqf3oFYjKBB@2JoANk?v!h5}6|={-U3lv>71#7xp7 zr`n%2x6klO4e?MZfJatTeFhIi`8~mRr|TK)mQ0wOq7B&SK;baD60_B zl6_5@^^@RGTj$vg?FE!kZA6LS52Ikvp}yga{%yhYq)5d$fZk**ExY*GsUkLc6)rkT zta`09cW|Von7p-bm9NmX0uNs@%Hd=l7~N|}<1;T%G&KCeTh}u*A!ICUn4Vyz!hF+P zkEagQQX&qw11)8+)DL&BmJfYRYft6x#T9oC-6SUBJj1KqxbSW5m{5|Hk^vKB$m4)# z=7OVGMV#G((JpwwN|_Q=#?pG=e30tlz@YsM{QLHbLd%*0sl-lr&6_((RgfT9zdZ_sQ-s#PkBGhu&|llQM^&J&!?yVh7QE2v8;@YNglT9SB&nw z^p>LAuS`!%`v}|jHEUbrD<-rLr<$z?MVqg=!U4e*q0ZFNch+NkP!&0q+U$Nn!nBKx zPcP}(`5t`xDiTw`Us`=-s$G<|fm43D(mv?H)}12T^*uL4UWL**|A0>6=1j%qJ$qK; zf$bi&ut}(Lb(%Nn`P$fC_e$VIV{P2(HW5?sv34Wz#HWqm8Z;1gFHuCyC0~c-7!-i3 zvP&&@_f)r~u16~E0YN(0zjT3E7kSa;r-iVwW}7G_bHCTaL|$8E3}UKXi|&3vTl05O z==|o`je;akqfYp7pa4cxo2_Jrq`4$}Z-eT*ylQ>JBfNFt!eRY{aJ-oqNb&%G^2A(; ztK##KmGy+9GCN1XjYy}YCA}g+LHy;fp5arMCqtli~ubVM~# zgvd7h5VL2N(JiaKcj);SN$DVl7{AYrBL!9dM_=b9;qXrTezansWCHq25$d;Dnk^AhclH z^3vVR{Ntg!CTwxzr54FNGUrzjc|QBRo+l zMZx7s-pmZH_w!?JITq~Qvj{LX&FLM3u7)`ReqR?OJAMxZ~xS&nGTk^^E+qLM_E(H+->K?_D5)A1u8DbhVz}9!{?oh z_-1o_CsOK{!UOzI{h{z7eOv`XQ%H|w!(`985?jews;p#sZn5;3TJF1l(RAUOFIuGv zm@HM^;AHa6xsvgu)YN*(g&;jvdw0>_UZNCnZLWQXW(hs>JCG`?d@Bf&$7g{UJ6Q`I z+Dw*y*SoKWnkmDkc=ZWV9io5XA5^!mu)=VgB2|5QU`v_w*$}-Oqp@cfwsvir@rDkp zg(g07`+1idO+sxc@RP5awbnG$UrQR&FSsC6V^~!vDeqAp^)n?5F$3FI2Zi258*4VI zJGQv|l84k4<+;-Jw9DmsoI%{`liB6}O{6V}W$l~+KRveBdDu>IbTN?5@MtcWzDgQh z{afXkbjnZwpWdMTqYitWdH21uBtI7?Wx7vpdR&dU-tMpuMhp3(QuJ~IjdJ?|bKC(X z^V7)SYQK;*v+W>)?yG$NSzRXsL7XCH4-I9uY#5GW*4JT7 zd3?B);f~TgwBxtDZ7!AI-6n0}62B?9(4vO7UANcaiGw=1Oxq2ZS}S$7u^3xMzr+NT zY=dEr6a!{b(|aKp6TCC|vadf#1zO?fFaFr|jt`t4T+S75lX zQ$)D`9GrZkoJHppC*b_`uUrsU$y07!b{&V zUtGwhrs8yI9Y)f`Abh7$2*bQ^7-V!vc^~2PYUc1o@JW#3hp`gK0rwjVmouS7_FWRX zBmPJc{Re_aPKD9mHJ!M<=Vj|{ZBX-Ku1sR zB?IQ=3jCw`{gUO+OHJFCE%ZzS0Ma-181Yg5?o77KPY zT?Rp?pHvl`=~i#3j}z1#l-(uO|86VxhR%&AZaW>$uvEM zHB$f3Y(9N7(UYUWE_NdCa}j;hxYVPIvc?04<#W5NAXWj6!+7&dtF?=DRz@d@K>vjN zl*X8?d}ZVqxffD%OJbU3h+@-u$R2823c}h926=yA*G(<1OK?LOw;zq^Axl9q)qeNT z<=zV+m<9CA(Hc^%H})8~XF+{YPJaks+uo{zsx|3gm8>v!eJx)j5**joCckAkwHoynoM|BL%kn-Oc&&zw;?qot;5eBOC*;M^}|w(+lL z0)^<4(JBkdBQL?l3_TT{GzmkxobN-FLx9K^OV^ztWi-z<-G&z~H0;v1{XU6eRlKcp zoiu*W@4WS(RQF+^(^UGAccm40UyH_JDu5$-B|pwnyqnWG#mlwAPXV((o(i26?x%dK zTUvP&U`tu9jbMGDWQ3WU9%XMEAWm>KVq9?OAx9}>@nWM%tj6J9+VC^7mCCzrMkZPR z#d%ehsU?>uRKp{8yxg7TlLrpAO}g|(3=-A<_)3HSi@9Cek=OX)s%A)3wQ5jXYqfqfM|er-#+Y9Auk!n z7gtvP`3b?_#5>7UB|Mr*l9oWQvdHry{&kl#>xX3J(haE>=7U`~9H!E^c3f&8t*i3p zOk%0dvkN{=S$XCWHe-%0^p&Pi=L_4xP2`BjQCpW_n=Xf?#yT zcA{n4o^ZWihHs5JhiwQLu*F(;&$X&_lX6gy}q!?LDUH4YJG3(96vy^7|I!$nWm z_pq1d8}pvV=o)VMz`K$aba3f%l7UZvJs7}WgDN@OHxpG%y=7jlp}Lzb*C9xj_V-~I zs~-Gk4-L6PN}TbD`+l{ISvYXc0qtI8d}E0W9DA0QdG_4p1%eNZM@(*_*3-wm$Y;;_ zcEQlas*&XH`~K)7zu{8Q5#aGG*QUA(p_e$c-!x{BW_W0Yy8~D--$+Zb8^;@8TcQs ziIsW`SUnFVOU2(UeD!1_;~&ifr%g^hx+=orHYSO-gRf?x4TeQ_(tVtj=UI}29-RHK z1QZdX4-*^-4ZFX z#%z2=6mCJu$cxd8gJu;gkH75RFAlyWq3FHf-Rz*(R^NRMC*8f8-o@UBH zU8-y8*~Qg&FyGc~=Be-c5=?rUVkZS6@Q~=QZLT2+4fp*u~4Cw5^yuSRGXz^c-#kXZ7cxY zABWB{hT%tiwd4+0mdb`!c@(wXe0H;BLut0r@pDbyqQ#N%T0+Mkvf_*sRJOn%V^^h0 zuDm-j7tY^SE#hSvL(6E;w#{OH67IxAcg0^c_BV z=>~gi<+t^?J>Rs(hiDd|6PN6zM+@hV#rhJZyVpgFa3$U(z1rmG@<1nr(pLSbySi*u z5yeX5CEPKa(GMxBu38RVPENcL&OsJHWG;}E#f;g^}MY=AKyOLD1cLCeWheYLrv&A zpZ8dfHCD!s>?3Dvbr!k(SHKupQP3SvSEzmiNq)Q!Ivc~_!gajH3P(-lRMxLum9p_* zbD{kGs%QVLN{+Ltgw`>V~ic;>2~!&ci38#ZE>$3&+sQh zvL_zfNe`g&uz6%|Xbw13yLxk;iB&YCL7SC{eDU2l#|-T6ta=xwkaqFNSDdB#0ol3r z(~^(&ULVQ9HE?+=xzfToYi-hiK5NZ5qWrwA0l~X{CVgLH$=@Sv1GupOXwu?GF-DkP zN1KzjS{Q<~8O}E%UIjNed@wqmgtvCb?6|;sL`@Zp_p(dZw%I^8W5J&LM#-qSD19Mt z06W?^AZLX5V{*r_g?;oCMNo<@@>aoUBXlYw#L#+zWRJS-0?jzlXY;-qwA_gOoOI*t z{mXm>XPO^gb)deJi^0gvB#t^B6 zP<b4rP;_fd%!*Ax+~C`;3#AA09fIsd`#}NSK~cUOW3-8 zquWvi$J(rxTv?89waW5yFn*%|C zNUCK_(t})u(`KK6HH`}Lrpz_AFauiW4-0XDNyAz**Fmkp`S;ZlMG77a-Df>@B5pDy zMXH!Im=7%VnF=KC59`)`zDmJCYxwIHTw2Oe?EPJ|M0a4>#nbekD^6u$tRb_)d-^RM z?|d9&UpDPoD4prtHjfJnB|Iuz`?s_ zvuvXoZRLbn#j*0f9}n6WB;<8AnmU|0upm+JPTez^xcxYNs;FC^E~Gve@TUJil`8SY zO=`fN61&P&0R6rle=YZukt2J3xpN6CoY3stQP_YwNjHXR)k1{4?JVIisa*fKo|}2* z@qnw=Ae#M$(%6=wcv(rALOH4K>JP_qB1?Ip0x+WDK|B_i1A0X3>?ncxal0=Pcvx5lFjV!2ug67rek)@)Ck zK@`2|Xt2&vu9vW&SxoiVMpQ(oCnBpF#TPZ*Q0YL&9%PirtuKhVx8kOg^Uy(TG=XHK z;Fhq;{uI$}I#5}Ba*i3t<|1tj6@1vfU}T!LMA0DpV^Ow2M4Rzn@54-a_DF<>%#E%A zuQ*~xZG`VEib%=3#G^ocAhlkN@xiCR%7;Q-e0!>Q$OLOCtzoynOpADW6NNyDfA+u>=1+X*C)zhoX=4!Omut5mB75?FPSO4uqt zBIq_J_SkK{fIDRL$aJ`a#u_A974h~5Nh25gG8r8jjPx0-nz5+ zy*bSV-Rcgyk_W?|P2Om@s+|(}cOS>?T$V;yB?Yl>7(^-Q>($m}9{{4?9pZxL z)R?e1W~>N0p{N^*Doii9`{=xjTk5Iae=$E^o_HlNXJWnBLSk|~+5+xb|J*UWc$=3a z5s)cz`Tbro%l}R@%tS=ljbPsSu)t+#Ah=g^?L6A)R^JZR&0A_<;_Wu=M>6U7NpbFv z)Q+KY{n0=1X|)9e;&ER1Y|*n*yG?{WroMMR}}p2uZ&rt?|S?& zNR(33hwJ-B58M<*D}T%JJ8p)I5z?M>T9>q+@7fW88dwKl7oBrF`GuzxY!!NBJWp#Q zey_+c@pPMBk5We}GWYY(m96((nhrDFWc{-SLFLs8U1^}d`U=ux z78)?Z(jpLaobdjh7AM^}NvEsYFjxzW_9ZL!YV28NbQ!T$u00ZAM8l42c#T?f?W!%` zp=finuTe5HvsUs~&|tbdATW@vZDhEbt5pD!PX}U*VjnVl8p%Tj@APE|`TcbvG`F6h znK6A*m~Ioo$m!Git9Z-l&P)y;e5(8D*-kv;z$%lq7s+aiY>W2U#R*q6_5>H1+)1<> zVQNfvsxNWL7x5St#cW3|36`8#k!RD6H*?8N^KemYu*Sab&@Y+eKa^);8~ZuHUMN@V zP3lydh(Tz~TXd=Yg5=Qeu4*F%^Kf1HwVfRMLrImEM4BH^&J;&+Zgtd#g`&8j@12Ya zO+W0jVtzpCO=Mu<1Z|c0p|RkS0H3wBA!)hRmEE>9G6fvrJX?@QL$~i1^pQ_%!x-2Y zh3GU$x%f}H%kJ$^H!QF%pq@LvLM5fWH)=R>j+s%f4)n0)s}uM_A1%Al>-69mE4nxr z(Xw{M)61nqt7w98ZM4=m3E>se4ATe{&d`h%F8_%iIx&Xs^cDLJst#~w@?*AZeUs6U z>czu-_=moohJxT7mO-2{xqg%TAN(o$3Xdz3Sn1vo)U*o>NE3VEl3`n|^sZq#^NKfP zwhwz)#%gP6T%cf|yYM#-#-X!RpbZ{fgqk+ab-zG*!|YTm%@rr_oAQlhV z*lO_CdOs@<@DMI4F-z`_DyTaGrZ%lZVojL5%WIH8cyWUgOiv+YfjGVz4a+5aW3p{e z9P|3|o(pKJL=&*xPQ+(Pgrcf@ZzrgIe90DGrLPxwl+pLS=BHKmmmQfm!M!rZ#*S=< zX-nuaZE~87*-pMio(2air@+Yna;|9YMRGNtf=u(`BwrQwX*@Fn=vv!#FL zCoZ71!oOtr30N7(-#)wV>J;%6zWkGT8_#R;U}%1o+-Vj`DDSczS2{JjBD+3d*RnV( zbbH4g1F8F9c~-gr_~nS@n4_jZ7Z<2GTQt))qd$9hnZau{WB1YA*9D)PuVZ%)Q#^xH z;!CKW=6Es)_bXhoz1w3Jt5VUkBBZFEu*C#)tE%9l;()^znqaeGt^iw*&~o;kuPS=L zwb!i8$`m@^Ya>0p-^0oyw){qCH~72Km>%>+y4NeyY##X>myFd(72lBJ<0)OvB)|Nn z;SblLr2)!$YUY_5gnaytdjU3++UpA!oWEcN72M;JvF+De_`ZL{`Vbk{m+5&p*?cD- zJr<~zfneLfItT6=;f_$cGR;W&(&byUJHc$$8d!PIM`6~~PN8Z=#b-l0G>v~?#YlK!9A#fF}hpTMd{PM@z?@nG|JT?ZQJYA4F(K}5@s#o}y>S8$usg)C#MB+9VvU=xUQ2XhL28uw2Zi=ljPq7(g7`MF=A#HmA$=r zu<0HxB?yo7>^Rdgns^ehKd2aN!>^5LzxMyLzN25+Hdj7#3I+S4H|K>l0$(R56}3oc ze^+;?HOj1Z79{c5 zC^xt92Ola6tu%j;okRlDK7WuEYp({l->5+wtzQTO?b7pM>Ra%5E8=3D%FT z(hr^zeg&?b{9Rej^Hv7(fmyz2?l<6g#tYKA4uUE{nQI;U@y(K&J}N)RT=cvjVK=L$ zH}3w$EwE`$9)#arJFWX>M$3e&Za(n@YhMFySS}!IIT{!DI-)M;FBV7^sO@=H9EV;H23xgt7uSHNDpnB8*Pj_-SyN<8dR>rw>y){N-AK`3Ty0DeNyD#D+zm}y6|GcTVo*hasq@?Uw`KUod(ZJALNVCbp)K?D z8=Aj*P)>eQ#GA{MOvRF9ZqQsTPg{ak>NP2x_vK zT0Y|bt+w0lUHyeb?HtD-1k8{wUVt)QF|#$TG99lO4zZj1h(iW$Y`1Pg)$7a4&Pxg@ zj!3<6yghmjQ&3+?=U(VBpC505dTr#wj*gXoaG;D5y6vlsYG;V8WeX9N410o3O2LCB z-@fBIfnBzr+vCd*378*fjxh|5D1MtS6uJ0DFtLyq=A!db|KM>n@CvEpu(<;dj_>#j z=XxPzY$i_T*w`~Xwu>E6@W>r^;m+Qi3tpGUaYu(?ldm-;C!N`<_(zi~jHR0293dit z7kh?vQHV8YrM^aMiTZ=H-@>dKy6f}uiJ6O+aidLcD{`aN$cK47*d;Il^{~1#qN;A- z*n+-^hGU7^xDu}J(>3uhf8}&S-lAFP<_k*vH4z6Y<{+vbGrsYCw7uXEnEEYy^hhvu z=3etHcJS=a+Dgf%iTAmP(XPbEm7S8K#meKZx=A(%UgnI0XAK){U+3~X<$#0O? zh+uaW>(j&chyz;umB+uXi0>UI&%~4yZ^lwmTQ%SIA_Tdo7S^SOrC?bWDETlF<_**U%u4z%E`SJ-i!kqqpnaiWWwUH#Tem7u|ryW>q{}c3jkO*^-s?tlIS3G_8f%%<`qpZX02VVX*;gn&tuDx9O^ks8;#!Wdm`w&?n zRAG-W5s{e%mNNf1@o!pL?rEQsxtC^t<9x1%KPNa9pJVoo?+mP3_+=j`09~Hq_}c%3 z@U?xs;vbK?XWdm-9mC4sUa5IRf77xn{wky9e~KqdPx7n-@mH;N$-Un;u_K0!2mh|Z zX7F=pMjY~By|QuT7|We6CsXwwcz&;*$oTRee^RAIxh@COPWU5Z4J>PdeCS<8ho|t!Y)^<^5f<7 z_u}H8-FOhM4t@TS@#t^4QU{8q#=lu1PVkK4Tw=;z8n8^ws7OsDab8#FZN-z_VO-0T ziq|?2rIp&Ra$k@C%?JND{Of4WkAjF#gC`!8HypK_z8MBF;i6vJ$Gfi@^qZt0u3Jbi zRJ?ofV~~*-yATn@iCP-iPgi+qb*rf-e~hIZVh{U5TuuEkkn;3&si!R?r2bu&_bHQ= z^@0>xrQb1l2X59y{{zqc4e7TCjYmXkGr^}!qUo=c65FqCsdc$zu6Vr*9QKKwUpmS3k~bWs)<+L^=7+$}H$Bm8}(% zntSLqQ}&7MT|nSDe_nho4m)ngCo9+Bxbgmb=?Ax(1@Zn8(Vk_WQJ`#@h!P=4^pmop zYln}2(V47W&G$ez&s4+^P*4ANS`3J2z?+*kQ+*<3lqF&WT%5FK0M~g%>_E^5Z^@f_ z)GqRD(y_WO(yv?(y+HOjYvU$f$K2>V0Ngn&$378E`|UE2VfMQ8GJ%Q{gnF3}rD$iy zcb*Lgt&0tomb*o^dT(^;*$@po;16XLmW|%%Pd$0NpHHhQl<8s7N2Y*=kz)2w#W{j`bsa%vJvvv?ZCt8lbta}_*F_wSm>4IrXJq&5I zTCyQQoRA@4D(e|ISNY}I2L+FdLzSMqU;AUrao|RT)VVLl#c@@t-%FIUb*cZJ zcNvV*d~m8$w5)fkC8vAu73HMP@(IFp-*|bQM>Q;@jhl6BuO&@RRrzHKaBQn|pf(f)o?eEfRr zIWfn`Z(VK@%k~>$DoIs?bsIgRq;>m#ah?s!vrYupwy!7jmlCMZ^H``@a0KA1i0acz zzZ@?l1lvTt61{t_65}XUdAjI@S$ZpaHVAZRlnglY`5Nz(!44peUx}#B3~7o&uPcHc z)E_Y%{k~%Ut9KWCLBH&=;kW$7Cl>B=jNU&>kdyY8L08=*x)Z!sGc10sTiI>sZerf@ zgL7=wDhfhZsUPmRy@`6*gT5a5jcC>DIDN^r>PgEefw>b0tN<)G?Ju6yYdg8zp*+c|2 z<=FMA!-k{3-jpG)o-RG@bJaepzWiZhnN4@a;b`T@?QBsKSiEi9_akC1?yD00Ng66k zbk^mM-7{@=1+n%Pqen}ER>jO^9|~LeF!Nm>`LK$K5peZ}^%1o15r%eBn{P{Rv{H-FrI-NV6C)#8H@{ zWNo`s;0<^WgoQY7MArvpm?h@w4+XF>DZIIbC6z~?UK*~nxz_p61mw-NkjL8Sm=3v~K-{wn%y2{AP8W)g9<(`>B}~aL9;!IAujC zTUYeMl^nyYGHEhS`^slF;x^|Ne&Q|3xno*jaj-(;*Bb4@h<)L*p2U7wQYK8+86yX7 zZ1x!=d$+@q|IPXSqd+0?P|KkftqV8yC06a?~frbwVY%9*%rTC?aep`a<_nb`Gya#55q_LXiT#d zM^WfWdSHL#T?5}OlhxWti9;5l`taTCQrE(hIhRX$y^Pj+x#O>3G7aY5u{_so@ z;7!Ieri&SDdDU}|PpPlRU;7Ov-zntPqjjk%YYOI6hnYCnyd>OFCh${4((X`~?V)iW zgF_a2-o&;mQNWLMyS-&R=Pl#2*n_r6!GtDlQEmH;fhUy(p^K&ZJ)+P`SFKuNX3u$f z_^x;*Oulgc|MHAv2%w-YsHYaH~1W2d@Su0lw|+- z<0B-Ra+k({d9(DJixYhhm8|Y5eH=@j)S7FO-b@gsp$7XKa(#E=;{W*ZGzD{>73XI6 zw0FTcu)o^QSMKj&H97qye8(TW-J|0tatO*crD>fzr9Ymf?B>?r5G!#gtaW*Dsd`tT zixv@Z6!^k$6F6Rmy8hvYJU6xnTLm$=a@azzjE-hBOR;QUurE0GdS|a6B=jm5S3b=9 z(UH=cl6?2ile-mLuYY7E3r(5skpkyad3);*n^(z_|9XQ|=d|Gn;PN*0)#q`!xvI<7 zEzOmR1%U}vM%CB5P1cughgtRG4}~_#)HGwOX6oOzd?aSrEWG}3_m?F;={nhYaYUAv z#zZ%@{?JKrSMn^qY8{3u>lLA%?;jWSJ1^7l?sob=EbTzf3YkyHgP+IJnm-y*DyZ*k zrRLeR9kK7e)hPO&`+@Wr>HQGUbj%iLHRPvEw$cyqTA zASk`8cisPRe~uVu)O+{MhodPnGIsi|VVx_-dBt`)NGhL@olfAoh>ItPaAQ>Hcg_jp z%0s9R{T~4k+26QGbE`)To5FJMOE2bkVwO?gL8CUEueJf0;A1TU- zz7OipPT8Hj?nPgFdiGy^d-wk0zi1wxV4Y?hal*w^dUa)-mSQ9{jaYcTq_K@k#dr8A zEmu!!e2Y@39l=n(@9XzH>;9etRH=FzWE$$W7b-uQ zlw;>T>mI)T@sl?g??B=n0tG#}CI{!6zSIKOlH(QPcQAy5@=R`Y`Q=8tK~Zz+pH~E$ zV740r$3a=wcH^73;=Ff8eGR9=J;bvhwlgFCYf1erujsdK9AEezPBoQZsWrAe>@n5G zV5F)1ig_c$YtHbI+sAti{KgfB_A6YvHtOX~Puuk`&sCK|9Eq+Ta)9j+Sr|)MBBXh( zxBOk&kM|7U!`t6s{wv>)^vj^q0d-|R)s(9WLJvLnmnWt_yYThn+(g=Sl5=92_`4=B zmFCscM|mClH=^6e>moY1Fa5&tSC4xCihbe@?C-r$b;vN3_2-Y^6s|>SdpHBBdaAdxGrA&y80j>Vj@Ub+}f7Q-%Y&=l|7qbz33P$wc>9 z^)Cjg5AEtC+kx`ET3vM`aw8r*CU06a_F=fYq8zTBN8>bMgnnsW3#@u}i>g2Goo3P2{rSiIX>f@vR|6;YGB2m9nQJGT1G2f&Rto zE|!OJsXiPp_vDbP`cv;k{Xw*$icanF{y&yY<%yEP5j+4^0HgLyZ~s_GK53{ycdy#^ zuElJ`GMrvdrHF_Dl~~bQQ%q$oy6K#b?PcOCe@)7q_8N@;3qh3D?XMZE zX}uzzxOU>#wP3GnV>ecXc#~~oy;i@AD0B#NQJZ$tE@8s`N-5=Hy^dcL>S`zxBA?_7>FzLa~6-cK`B;8+5x2`V<@$u$mB-+jqajWgxdfFwl*s(yXu;$k*v%?IOSVF%AQRd8G}G$RF9<&3WK zNuJWGG$!GqV%PF!*U)DxmqQSGp41S%r#X$Qd=jVL`u|p(U5u|lKX#T2om#Rta(3ZX zJ?G8PSmTqhnrz{RM4NN=e|L12yDg$>WPxh=14_UCXdfT{{sYJ5jQ{V$+v|y}m(frE z^=}t<-j^&mT~>O?Z!JgBhFt(zm-nbB`JDGWc@jaV6OAlv0>x|Noh6o@huX- zA=AtCQ^jFk&1%EX^e^Ly&*EJnDSJ8k z1A}dc@N0H>--3wd9X0gh4AM#8hyyC?v0au2$k|xRFPxT8wP&irTTgCCDi%8YAMg2J z>Y6C-BhO|AT-r`PTu1<4VPg|89(Q_`tC!W_80OM?BCIY@s8x2V8Cu`qHgCM-MqhWk ziu8SlmsUaEq5^+V-F8gY7H2_d8ax_MVn&s0pnF*ywa$F0_Vxv>-m%L|oPa?Mv??2% zJT14khNBDl7ye;wyj3JJltRgcQNo+jt$m9|EmM?%n4Qo3Y*OXail%`LdnalFs-&aa zVTajl?F`5hXbfz{1b*-S6xDlImr-9abJ$M-mWj1VW@P1u#$U4&Q9A{I6`nuierJhR zU&}ZavxegxGhpW#6|ioiX=P652H!cDhq50?K)?E8=qA{lw4wmfHs;KZEu&wpeQ~I{ zXW(;d4r1rWlSF%R$I&{%l37qKj3o40yFho*b^v~C7*CM4ElX&11w zYcK^h%i=<;=0noWsX~58wLeJU2GB#RI>S@s|DCOW>m2s3&tIoka#n)1B82rBa)*<_ zn_ccQNtw~rcG7;m@>zJZB46RZ8sN)Ff`96qC;Ta6{0?u5C`kEvHni!Zm^jId0|8!| zU}#OPo@**==+RSPYY)NFO?@K{>7OGt(B3~8nvdr&^8d@cCgu3S)u!xjo<*$%%1t zV7?KyJNz6Bpm(ZRU&N!cYXK#NZug)e??R!)o1I3q%hSx159iJ#`!&R!kanr+JB=sK zZsRaIGKEs`7CP5H8=N_Y%w|YxQpettt!^ngE^h{l^pR+ z)8?JhiiFg<2^-~L;`lDicEQ4T0198v$p4(@M2Lk{f0-&W&-tiiC`?>RIU`;gv6I_8CHo zUJG?zK8-V7Fh-suSIZWvil>m1;IYDN^PYzBHb)LU7))R^F2vpWf``VjuPnNh;)!sb4%~X zN*Htc;{RHHi?Q%lVF3&)_ds~oMp^SuhsVZXy2C?XNf1ztnSp_MH6a5JY&R-D6K#O8_yTq$DFaI%IRpL&i_IN)x9V z`{w<-d+@e;^uQYQfqE^r6he1~Yu*_2M>sIZ+7;o{=~@IAwx+`f+_zC~MGD}40JM7O z$WLV)xi}QLqL&1|?^{H<20dfyhWA8bf(XQmX<*BMuPu<8fsb&M`X|O%8+%Iy+bBb9 zIQ`3IrI|oU)S$LO^_vj(CGwCL`IG+at5f9Op*9fQWH7dO2EcX`vEdd+N)Ik?R5T2` zc~&eH55gEa?kgX|kq5_RuME*?Uzrabs2f?0gg|6k7E0(-JZXSxPlFVxbgLrpjhMEH zqFk&Wv1P44D7S58mNjH^WsESw?wTS6f7w$uLuWL*kt@?qk5%~B{qp~uVx4m0vDu)& zPzp<9gtci^WJJYIAzhiUXCs}c{-4cjr_6*OGP0t+-Jj{Rbzd>DI>!k96~H`)hQcqR zv*5aTiX#2n&yP*|71N4Ygv1}^%^uL6yLJ>%@UGd~YDL{lCbqZZQsjrqk(DGfI(vRr z9?ffxE5q#{Nk_NC37(Hjoy}Q;&CN~J9TW1WWC&XufDwTkWh!YdghAFMSq=rCV^poX+r{A$?otoKs})2rg?GpjtYWq3NVedTOpy<@5d3> zWlpa{WSeau26&o72}d)}1*R;Mzm&6-jVYd7#PqG_!6+7yH@Fvph7Ck^wOe-U`RO7KoqmY?4ZT*u7K zr{LM`MG;!3jan8^bCTP!JeWN9@3(^#d_I#=Sg`Z{raP&cISALKz~M!mXJWHg(jl4; zo#0nAFX`7#8=Zk=IuJD%w2(WV7aU08xsY~as7R+^S?zLl1}dVVwPZl~Rfvr<|yb%Aa4d(fj;2GlP6wt7AG zuj!s_!oXdQ)CPq$z{^bvQg;WC4qBW1w2ptxW^Qg?rOK6ID(B1IeBR-zzl zMnLEK;|79|J%-}YV=p+JX;alhf(;E!kqmPFvs^}w<;*K=ASd6)GbWC=RpOW0$sllW znncd~9lo=$u6b-gY4vhQt`oI@ruq4bD}k?)1JJ&L0jCcHuyOWnc%tT|?)G)nk)j54DWrjCFfEwGI# z_W;8|J8n+cC-hnA!@)iF&7n?qaNPPJ$?mljSsUryX7n1t&Xj-*gn(g+D~XL>%tp3D zJX?X`s*l75x*+Fb%t%n3T9v>LzOE%o^gwq!IXa_pR|ccr^zuJl_Z~odtGd4-x4BO5 zNI)nG>7l9|*thl4;Ee1mh~~Go0Rv(H?p_9A_gk!bWs86|59f?n>fI3Bp@kl^pl)Fr zfcCJ#5yfYWovQ*seDrmMUaMmLrkxLtAygClP|%#RU%A zmTOt{7USIYH$gQ~vU-(+;|26N%?j+DpV}F$`GRd2|ZOGYm!}4qSpOjI9!}S z>r7=-B__FiV%Q~l?p*;vs&>mIgLC1qr58n7gNzCKkNUL=7U$F6L853wmM=G=Ru(Pt zdA}3U9_CFNiAhLUOHXEys_(V8XE6IFk?ct*HCEU)0dxTum{6jVm81E+sg8Hawi;+~ zB-d9~r2qv7|A{&H_-Qx3c}3)FBAVPWi!#L!%{y4;It=8DVRgp+kT&z$CcC<_mx=9- zuK1(VM~Xu8oRM*F9qyosqG1o%v?{@GbMi$i6S3S?JOd{5w zso-Obs+$`?b%|YsO=^@0=4VlXu8=m7aBVt}i4Asc6-Z#jT4ozGl@(fBMCDCi{kh%T zR6ha{2{(S2x2$39N2}7-rfe+4h_y%L{NnPRhDEORR5ZNgz|l?V4ecRh!h7z=PGw*= zxR^4xmYvLi+N2E+*2LjZFAZqcUpixD@v|0QI$FvDLEzCn1;sJKmJA&Wo@XXih36TD zj9AktE5Y;FK<)On=9##m9ZQMuolOtoKnyI67%TG+UEL8lu++k%O-m2&Z!`V&bE;$c z*4A+G??;0tpl|%#h4UwThNa;s=4gHMF5Aq?y6*SwkchATA~Krjrnt4;*~3<(MwyWO zXH-g9fI5|USc^4Ql>j}L(6nRWgq7O`0ktEG@okbW1}Il&wnLuQS9gK8no2zMkL4ZK zws4lQT{QW~MHo<9EHJdeJ$rG+z?vMccXvY;dImTFx{UEM0&;!5OVHI2(`zM*K0Ur5 zHTZz*VGNB`UoukLL~cuZ&WSz;0tXLJ*Bh6Ybo3C+(U!ICpe?s>WW`KGv-wm}$<7_8 z%8@p(PX}STtGS`;YE)b>@djDoUhgtw$CrVi6T4vp5&4JWhHKwFp$F16d}qi%6^}p2 z_{ZsUvM2?YK5QN74?9@*%vkRdnMo()hw3QlL<}b5zJ1Sv3jo{jpd)x2L!HJ1Q$x<( zh_|{qsPOXpzQWBRo03+Nlo{2=F`pmcO#i}5-w*Les11} zyL{AOh#vU0dC|L;fRA0G&3OyC-mpQD3Wp30C@nmDYKx2h@<9`x|ENgYe|rLp2wa&# z*Vze7I)WGOB!FrmcoIy)%uAxHXY=$w)H3^1)in%08x7~#{(AXfOYb#en<}N3r=d#D z5=i*@+=0NXf9R;{5SA;aykD!iZ`Fzd1bQ?SGqkNeH3cY-r#7q6uhG?djDX|Z3|{XB zEo$S38b#`@^cM19YNG{=7zK5Qj@m`!&A;A6d;qLzL zEn;A8+@DdT-`~v;5)9tDUNuZxZ|D)p?`*{s6wINLYf-l}{$G7Qige~EFQoEK6LFxlzwu(N>YWe1I4^$S+d+QK2V72tU~1>w(P4PLL!OgihA=~T z{ze7`7~J98{5D@imEK_)7bH}XefbwTp=)c_U#O%RC5Ub)z{)Kqi$KUl6g7*0vROwF zV!+8Nb2i+HSRC9HHy08UPV`x`&I*80x@XJHygCA!*Iq);2-OuxeFIgoyT`Yk?C;K$ zMiZ^AoETYXLtYx!hop|oF&VjMiVKDb(%cM$rf=chAz=J0%F2k_5F@pAdu6Crfj78m z){v19AGx+PyHK&5Qs7jaE}Ub&8{_u0PPVH@B8f`6vPtt4a&;=9YC>ZfR-V(y&@&rAyG>`b4Ys`@@OEr?lU(hDB?1g|cKxIcCceiMwM%<{=eDt1>5*ry zYT8(vFT--NasfC9%vqXQdyB}WB;UEPB4Ovx;iRdJVjsfnhYl;w$Mvn@`!VTB zo7Fx9DDSS4xkva3%opZ)+NxHKP1YtP#EWS{1TzDPu8~}6P;U<#u`=@FWn@nljRaQ) z$lfB*)ShctKocesm=>NImG_00de6s%uT8^dF>tOWVCc2$YUV6CgJKi1q|ein_4s4) zfdmq*bZB9m&!72b_7j={H@dZZdE2Zh+nnwCH9<=yHiVuKtoM8wY4~4^q46F2!`Me2 z!V|bh@$QBNPvu1b81o(ibAp~4qknPRlCs^>-}$_JvuHEI2}V!2L2G5u!qO@T4d@Eg z+OxVkI&+11)sLh!HB+>ZTemw_=i*PqO)Q#O?FVTsDyw}N4B&nn`6MYcrb0Un>u4@H(lC-^!Dboq8My9YJej5LXZ^zZgmmDrxPPs$zQ zG*N4tDv%m14b!HJ9{}92U>7%(wB|dB)9W$fQjXlJua9V!KU`k0xE+#De?i$DE^Xi^oq}^M>8-QlqJ}r%Bx42}_ zf#4weY?yf5ki4rq#0%fWa8@SIV7rEqPQxBBNpE7T>Q4>d&#OSaODij@K2Xd*y`=PE zY~PdXcbX5kz)!XZ$ThqgC;-Sx?XxFOH-vwl;D&Tr+_%V1N+{@>3QjUABuIsoC`xS) zSrxF6rF2r0jg??wabd|uR?3ei>qh%>EYaGZR^&GP!{peR#n@WRXq*CL{10BdaSHOz z@4K=q^J&GhR3T#j-o1cw7bQHhc-y5u?_0%1h`3DUkB z01v94dqSa9;ZE9YU&WwXNvKXR(cjZs5q@^K!Csc|zM;UX{xbBVRTDyRfT+a^NYGb? zWlH7i_aJV=)fkcqZ(wew!Ywqi}@gD^;P6fF`8o9hqJtF`kChT_j?{;uKwjyJSvpw64u46`Nc{zLvSt)xa zq{@qE@UXiBOXJ1<$avbwr&Ef7JuSsUTb8agXmrB=kffUOq46U>x`y-gfoa@ykE$5M zmG32X-%ZVgH*iG)+&mBMJm(r~oWXtT{@0Ao;ib_Q5QnO)Ju;{p1wGe|N=Dt%QOxq= zRJgP{(|6BK8M#(;pUZ;Ly5~uWt6aNkLa(=7CE(qCt5+aZB26n@nyocJ)Do^ZA+&Myk!CfQ8IjM5-^cnjfA_>NtS5{1yt+8#DM zw!{nU-CNRv?bCt?yDnYcKf-w6X$8!REl)N)8be?u+%=zr#c2f_1rBbSIH!?3<962# ziNtj&HUt^PQ%n80XHQyc?K7e58+ORe7*ZiiQ{I^oqF3$N74s)u-N^o#RIdSkGmHnJ z+KNs8ac@uV;R8mqppajdtl!UcGb>{b)8_%=S`XQb>b|%R5a2OKoX2ow#ddS5YPSCz zaO(Es>1Z=*0XYfaI4}I?1V+T*970coh4L@K9nSLYJZ6x7&aM&gg$hwy$Egl=d!k5@ zAqzGiaQlZ#R1Q`&ysidn!rtvv+ov|#Hng);dhFMb=$z^e$OM(OQD;wZZ5sYouO2dex?AOs*IP%a1=b6n*gRq?KJoUkY0Z;w`l=eco z(cc0}dj+r4-07O5YhQSq0*AIgepBtc4T1?oOR=n`cm|V1D%Yvbzm@M4k}eFH@dLPL z#LT%NZj-Mfp#gmfPE(3!0>i)Om`#l+$5nkuHfx2h#AG%1P`z@yulRd)Nfs747H7P~ z?Oud;-k2XiSHfc|1tp9MuqX&L2<%$L{u;qvcc|ONlJuTicXoGAF~7$wDJyrtJaZsD z4qbiWQ|*4>9k6Q7CS8$2s5l^F^WLoI>5uumqr4fnr#Gtii@zN5ndgxPQ}^S zm?K4XHBpAy@t0$Q$7ay4`Vo~J9T+m-vKF=T5u4wO`mW{bD3A(Cg)oQft_{@sv6bN; z;r_mP%j2m!9lOY;GL}8km-AvmEW@?Zv-%#KKruD5l44|Kc_48k@6qZFIZ@wG%Qp)q z284Lg)Nwp*?NtTqYG8Oh{T{A1A|WcgqXOvxQ05vUIvIvUJHR_hNXA@RA`XVg&gmFeA=wO7F}}u}5yV zCW6J=aL5?87QN$IQajb%s^T3Nx_O$d;hX+byE^DY<zi$%IRt& zz_0;TyIq?LcvRM7!R4!w-|j!?S-zWO{SLbiUkle=S}Uqpn=l0rLj@m5x@aTy83~$HOqVZG{f!h4Pn@ z075`mW(#sLu*t)S4S8tCiNB*>-vZsaeL-IX5x6CUCId_raA{;K&*DN@p+lS_Qxg5P z$rITQ9e+;+LqmeV+Ycq*v{5;lt}<{j#x9bs`JI(K+arQ7`$Jm+VJ3e*TjOoTLTpNs9NwG=2i_Cj<`l;1A}8C#GmorpJ5jeX+~}M6Wylvr4yqF6{vAF zXr5MZCvFD33b}>gTa79qpKkgHyIEv0w;hqO0tnXQk=~fN40qs-L3L7+R4>&-G3M^t z%L-!7@|s!`{awXKbhVF@E1E$F220?Fvcexl14xFgLz>3%JQy*=v> z9oCxfliTRAb%ArAeTv;{cHCUk@I;NV=TxRhLcR8LC@`p!~3yu(dttn zR&NZH7PGS^Q@8V)GJdMDnD2;QL;vvIZ%6i5asI;S(w>bg8sAY!WCZLttZ5)S)!y~5mOgh#Y379<(WqX61SxcKQmp>9?-1Ax790x?*k z;=$Zd*ON<+?w4fiYWlBzHl(_F20HRyoQnI z_b6h81M*XT_*sD>=s*Z^e52;#o^;77f&Y?jvfuKZN?E1qzvREo=a=OL zln!jSn91GI-z%RObLM?{<+NTb>G!4aRINz;y`l{9t1eA_knsf4#jr0KY319C{6hD~ z-81qd5%{uj>u|g9sP*m%++SaYci2HyN^ zrE31<{)Q|qT1YB1qX*7vcSGeBPR~A%@tD=*xeciV+z?M}O}TFS2{HcUgE?|*d}b27 dVgs%a@0VsC#fng@_VNDAOssDMApnQ!Dh-#PE^JLk{upIPf!)_R_O?|biSU;Em7*{M5QjX3+RhVJ`kpm=S4 z{-Ao7kFDbBQ-w=l#nN^994pAi-B7WVYu2dNX2KOQ_H}kKo`iAmgYWMVuMNL{MEo80 zZr2t=6=|&Q>FMjyY7p_T-}jlh;U>KY#+l9ZOM-Eb+kt0H&(>%Kd6!kAciDqJztgTi z7`|0_k9fk{C8;<942>I4V}ZWGXLzDtPIY%=)_#b{8*)v96+kw&I~E- zcLtlb)v*)$pFhrU5phA8z&2FJ3Ihm-PfN4~Z-RQ2kCTriX|dCSl?%3}B;t#~DtuD4cMRrvfwO~bRh zc566>&R!tDH*Zs8sMUMqOr6L%xom=4+a3$*xTH`*>{KiuJgPoOZ8^Nt#>`5-abRQba<-~;{P^I&3s$h1 z(%ai@@htZDiLJOh%G9z;mlRIWo&2w#=;^b7FpERtYpIyO2VzbI8Y~|b#kBIOpPydF z=_?9q2OWJ2(7p-7XKNN8v8_V?S<~bl1w>mrnW-lpeQq zMcFda@XN=B!T68)ZY0@RrM3l>{6S6{W{_h{ejRXO(1ujj8ybq*Ig1Fz?o z0}zdQc;}rJ&MZqQ)R$?ky9B6 zTQq3nWs%q0?Toi(h3c#qK30D8=%lfM)r8BRBOiM&`_kj3eco}hkLTcL3;8@al}*6RrZ)WQCq!}$G@kL~Tj2N1M3+#`iAJQ+e$NNStEY{uD9Wd{IXaQ7 zaqkqcR~!Ho9Io#6=jfLz9EIz7ZhpGieg2y&OYm__yif~L4riMn;R0UGcgQ$joFacE zZ*F16U#oDY)u=MEJ*WYWHk`mqp_OBhqRp{vr8A_1pV}^k(>^uK+R|fiFIR*wyaosBF%m?{R7qKZYurC5G$$wLQFm!_HHdO_NFa=$N-j0h=IsW}<6!%U0+qrU3`&;AD?P6T~D zzg~&iTpIk@sgjve6+c`^wn1M&PXCC6eA}647(eUS-~)f+`{V^OdTaM{!p#?f z^Z`Hrx6Q&=w+w;1TCk^$HF{B-uvs3CBp{%!VQYm!d%TYdNOVbguU$*5QBUlH&6nTa z7(Bxjg{F3Yv22-OD?{Ecjt0-{l`j`!BS;c_t{5Q+hHD zklbcaqJR>>xPb3RkVCEj$<3l|ajQ?SRh_RjL*gU+s(3cX`{jEpY!gj+TrC?wHde9f znJq5!%n#0NKg?$V{jR>`MN!x$7tZNcPM5`smnDzhv5v zH~Ha9_e~JUMS^>*CgPlGQC3vl2=x|OS25|Y>W=oGf!98Ctgo4t*M#>UrkZ=S?L|ya`&KjEjupTm$bs|CwCJ3-{L<9E z`Qe<9vX7V`845I$p-999diuo&UpvV`;N=*Rv;$mM%c!{hf zJtL67la1ck$%BoA`qs%U+>aW^;r76zxA3y2F16^NB7qcT?#CIn9o&9Zh@H6-*qa|u zutT-sOcVYUoWNQoY)*~;4jXgvh6c#p(bEVd_;}t*`rtEWVpfIo3QB<54D^Hcr(D2a zrcmDFws~cv(5PBxR3D5To`3p+rPB_XJM)bs%_kdQAyE~RlauwhgC@!j$5$u#dU&hAgf++h2;j$&mwDw6T&>E{ zVa2$4wuSZ{$2qTD#~bvv_r5a=+WHPT2>**boUPHI14erf*_g_6HvypUbMP|M;%sFh}?HIaY4q1{#uS4(j38o-B#AtdQm+8roz9 zqF#e16ElI)Byku1iSdOY!o$g?-KlQEL}4b{?V|ic$HXxs4=*$bXTIJ(C?Ubcow6WQ z3nV`h5gr_KQ?N9xc5}m{!rM-Br%ic}k5x7rsbrO&tZMj^Yfk^2Ycta1Q6T&ix7V8~ z8+Df`isu~nMo~w{__8kY^LnJ#W9@*wh;f_fLJQcGHzV2#{daWe`gm}lCD~!8gBbjw z*V^Zkx9qgrotlV(*JtC3XfR{zKj$F+A3l)@I1?k72!1mQZLI11Y&J-aFUk z67GjCLwvM+QNMv%#2#_CrXXFi|^k%CO3G@tTc^;go0c) zX~7XJTlc!jM{7{_Ew*a}`rOQMk+sZ?TIe z^x+xsD4}ORyM>ypI9%b1lOzvO(6xk#qh~Dx7L)H3BpJ8VD&6J4HJtxVU}=TQc8yfqmc(R4>>jpUp0NZc(+1%eu#{}Zjdy_>_1~`PtWF!%RkG&`_=6tXOS^NxI zAHtVc#P&ayi|Oy)zStwwI8_zBu_s}Iu>kyXnkRJRCCNEF_Fape-(7&)j5t@pI@M9t~il?~@5 zb1A9$O*KK48qTM7;uJeP>F@SoO_-WQ7g89Zj$(R{dOiZF5gXdziw zgZKuZ5W)ah5!oF&1b;4C7KMy9FJde{ZW6YJPHYvIGtzG)aLe zdnVb5h%8<@$a^XFqtzeFP8Kd{Z~SH_5;wAL2Xe^Xn6wo{Mh|#@{P_0^9(s-XxX#d= zng`II|=_;XxU+HpPd^$o&~ zw!dG^giI{l-mjG1$5|hih~{Q1eIo4bKSMlo*+=+9H2FUw>}8l5x;pwy`5WG8Sl;V& zr4rPWl**CO5BEc1f4{hJOoMLpT>(A)H+5sX{~L-U4*%#dqSBGq$B{SE^|{GVAAl_O z(mLZtq9|s(&}lbBiR_#g!}EN6mEn0UvHKPEoBy>Za^i}0NIh|e(|dIi7GRN`lsY|# z|Lc};@mB+rZ$F{9#xZOtL2>RkUjj0f89r40K!Sc_-fsUGFx*|M^u$Pg?(LAGe|`Da zGUYp>+Sk64WnW7k=?ty*UYJQNE(GKQ_Jbj!i`MyA?!-JD%v-7>GP1pBpO! zOGx9Ys#76k)hc!$3i2hP^`~|_gdOlIY6Qcb{$rG$bXI8+2S?Ed2N3o>*|7bgACNo3USoJ9|2`kH&G1rIlaws|LxUp zTtUa@&M(#Gj`-7<{#XAVMZpGX#XV07YHo((ozADk;=PGor zX@0B`=KC~o|>fP|*Z4Zq&yC!;UBzIMCi#ckxfo;1l(SWCpiT4#oM z|AU7ESqk*~n7yoYAL56Id?6D(XKguRjd=+&Z|^t5xKOk7-&KM88~aAR!}DbwL$WR_ zJhxr?pdPoO4?}N!Iqb){9i&Fw5W7{?ab}#n_h1-EE00Rw|GbXJy*&u&?F8gTi^Zqq zcASb-ahfoJI!>#r6!KJ3Ob=m-O1*{gZ+H&G@)vUX6zr|m*)8O*#$qz9Ojkoc(Moc4 zo>VDy$Ke||1%%-G4DV*uu-4;lSNKw<_?v+fBw~r1%Vt_b^tmcd@9W7m zpL^@3dLpSmJH?&qIAgf>fBfQ5p$=~b99XJGxmE5hc}J8_vl^Jl?JVdHJv}v(+$#w@ zc%yr-`5mn<{O^YL@)CGi*)Zek;ZoNzzg{#nliR_iQ%Nhapo_8|Q3BQL!m7}>+@++ow<;Mn!FA&!Qa+ng3-$7kh8sM1Nk4HE7ZgSg-nnhH{%cV}=Zp8t zGxTMuSvg`}3C({xgFpQDPwIA_j6}FC==-PqP8zo+xosI@$KI}vo8L9xkD}GaMF|?Q z!sKmip+0LS@4mDd%2!V;U{I_~4onj_hh8PA;c}tF`_{x~7S3$orm3;i>dpfB4;9H< zs|dnu6f>w=-=~7fu)L>0@~tDeGw&krDDt$+=%^tODsAvNBC0STC$yyxwo}Y+Z+6r& zdFYL9t-if_abj1-8C3vj_fwC4(>}$$KH~%ZTw;QOH5?QpGUUWuc{x;1MK%Bn6QI%@ zDZ+0mlT^FEckfY3Xs_w#y`^L3qf2d!={>yHim3M9zSX$6CvvOzu|upAb^xX;t{IlM z2jI#z$O|H}6Otl3IP0p=m{LZ~*w>p=4Fn3Bd%QhZLYSKS*`=iynqI(r{VJM{Bg4*I z`a!(rSWaPj;~N-xxfef=Eh}lMh*Ux*7G7A^+ zm^Mk9H+t1ZNT46)X6&u&*QWWo+|QOwbs=82OU?K9tfN)h7)_rYHMGx0?M|1r-N}}l z15>ol$bLul$fe)h2$wFmT4{ikoABmc-j&PN?IC&gG2OjQf0jk#W@FLVd$Hj6G)0Uf zF)EYp%*lRE0hQ=&_VYns&pw>J62q)05AhVNerR9&%BHU!@U0^@9y?v=($IuX<-q`OnAA{YRQi9QxEwR=7zA?C_ zcNIgB!M#hHmTvqqe|uj4HwRU_RiR;3hUuj|czYFncH!|R&|R|YUu;9B-k&@GTV6}mO6^j5H8@Aiv~D!m(yVo`B$M94F9IK=Mn*@CLFPyD_VCQH zkMKd!3qiMzJO4Wl#ftxCA6w_mYL`WVf1;&KKGKA}5Z$cuoH#)~vpMj5W*&x^l!9T?bR`#KR&y#pjnBbNNW%t+#`}@8lp9@8M!U&t z@%>z!DyA>(h1xe>(JqAVJAa$WYG@%kc9xUl+@&;HNA$y3=Su7#C&&339a?z(|NTcc z_L?U_J~M$RaTA|dgK-b|mlO1rG|JiPR1ul@2usKEl#M=9liV{Ch~-R`_HiAYl(z4? z0rr?n6?dCy${9^oS5_RiF@4rBzA%@pngMSM2hF}em3-$e6aCqmSH}i#b>=(AX5U~2 z#!s;cnAkv(1h~2HjI#JQ3|ySpRiZ2I7j(L&q+g~(eEL-dr}w}t0TruN>lL@0H?Li3 zEMlMEljwUhRZ14BU2$Uvtpv(dOnthBdG-4l;S+}2mc{}!9!L#rxJ+IT(G;|e>*9H_ z!e(G3j?!erN=++fEzobM&Cs*cB#PJ?&db1n(QzrZ2BjbChchOifcqtcbg- z-djPR##10;H60oLBL-$~KRkzDm+!BTC(rgr`;yY@?)IkEiwsN2X4WkaV7Tn2;Xc(q zg-sw%DbI>S{4#p>4EJkbB(GbuZpML?l{HlVWL3JObZ|vy{uiCelk`(h7&~|2ju*10 z3r)|)>AmAMQ|r=R{NolZ+vBaeGvm~D8LrUypC1calL$XWIUc!(kmn~$*m+H5wuibO zfKW4UY7qxC%DTN5dLQ4x^{Ym!U&PfV0z&L8jP{3AwJCK(a&8b)q52TF@opi+l?lVo zRGZ1Zn=bD6bK}H6Y`NaIwez2@iK7f%JlS&Rdx&~c-)Rsn!V~q*(p@%s&iC-u?-SC5 z5bGE&Y^CQ#4jEq&upzhLMX9NM49TS*La5P7muh33u5~sd#(Mh#tNI zbXok~L-9;LOLe({imKLoEsIdbU}KYApOdcqoVI!JvrI+z(;j#KJtar?b2-+2ivqwC=Jw}(v<(%C&!b-d!}?i?WmxU!7iE!YloIAJA&7Vw}dMgti4RJ;MG(~E{Hj!x~ME#O@5zVGvj9_z5q$H&u zrWchTlTn9E^A3|G4&)M6d3A4hkoIu){H68p5l3^~Z&%4hiCwSb4UEM`+8y2RiuWA& zebgkrx)cKv@k||`EH7X8MX=vh`E>&B^>u{z^ES*%PWsaaaL znZTbJxM$Lws7c-Kh*LhW>F$@Ox?~9I*Y6|uCuMyw=!GrB#hGCn%dgV!D^v}iT@qI$ zpt^~~66glXa(PAH%DF(mzpUALBCb={C2`t2G7pamOZW1xP4@_dhzk{37e@NSi|lPW z3Q}FIgt(ixVl4hPp!Tu>po!1X0m!=o!e=f!!`=GbK1zGZRQnu(Ixv+w3PdNJ_PUA;ea)|6}mTCL-&=UytJpY4C!fMJW%4ixk^e)g!!{pV8W9ZELa?^!= z#ntR`Nja?8YjbL%Akc&gNvIEg#E}_r6j~8|z4QiLcgQ+_oK{#0Y(oay(V`C1UOkx#s;tcfd6%&sV^vL|hAkbmW8%v(L(JPyrrXaVE3Re$KW35+K9(VW_>YyF zA0=K}%XBZQwC9zvr-VJC4Xw}L{Hj44`P%TUEB*0F{P!H?@b!w|KsWrdP(`X+V`b+% z->P;CuQqv?Up%Ih{qp3r^*7H*9}PeyC~c++>qx0FHbJv%o%77hDqIv5v{m=tu!-Nj z!hD5aL*E})zK4^4alFhoe5XIrC%x`OqY+*uS=h9|t2(yqi zaJa$^IjC&!LjaGqHSstvja2cnx0d2w{ybERPug1#%<6xQnSvS6&ijyC-LCAfkVN8O z#Nug0jOKq8_X$yZNH3EiItVHK{o}>yKG;lvDcY-Jx|Ev2u$~G<)}9lJsADxWdE|Kc zb{$4bTSV63r*$*tisnk}1Lk&l4nJ1oL_zbM#@if;&a0uxedbbV3Yc@C9?HT$VJ4C9 z!0mNwVCWQ|iT{~Z?%3=@7GR6MV@iH!UEJU$r|Hh?)J%JRZ_C@>hN>v?JkBz3`@`YB z#rv}q_R4(>_4}B)+gGoRNwLqj)D51EgPg7kRfTtOvTsWk?VQGrmwSvEgkKecMadJ# z?RpOGy7$Fs#@U!Vr2OTv)KghE6M+8cQt4-tBd9sV^GT2${8^y9q?X?r*3bp_F)QfY zez8?r$=zY(8a?Yu$E%dmI zrCz)~b*siCI6^*X7}%gFOIyliS1%fvvXsg3JvkAyb&jTrZ>2Mvgv9=!ab8;<+I1(V zKTHWfempZvo9W&SSWf&6rMq+Tcr-_Y8TRuE2ei6%4kMYZe9W59zpy)&OefCeHvB`P z<$}Ctov#Qyp#{0d*+zvA6JJ}!XZG6j$6!o9HA4km?1f$!8lsX`iC3vstgP@klQ%|Y zh8(NY@|Kd_lX@W}w}q=H0WEKy)4b?TxsxWTC#du=n1~ke2Yba|os{BX6S1EyZ8dkf z{k87<`YDke!CbTRG}j0%4vidm8yLls6a(mKUg4yl%@71}F2A$*V>jc!Hb4G3$LcBH ze52KVIsw`~hGy4`tU2|P5|UWA6Byb;c@{+g0?}Ze9!G=$WJehHTQ% zJsMn_yjMs~7&u3yef*lZDDMkv?{VL1aEf~3=E1dUIX>&DNxN!rQtyjooD;@-b>~fK z1-)V#ss;9S$TVD`act-%G4xN1pc+8O^mLy(x{jXiv13JgJ3YN=k@W{w`<)usps>PT;+oLNGRTc!E6zgv z4MyAcj*ZnZ1{MpG1XG(nJTD=5ZWcuiU_;{%nS12Q#D`NGhd#Mviu$&c=E{8^GW0qs zmID%+cV9&<4I%Fc0KX0cN=n1nt81MWrJgjac15nkf0RPFtLgB)&65)vnMIv)GLEoP zi^fq?oGHosOq<@_6{ST=Heqqsp)KyE6s~9}f0FyDsfVP{goV!_VHs^vTU!=*n_(%L zd=u(8=w55aQf(2Z@20M>AN8fYJI9mnlbU-&;FXedYfYzp0j(>kN&DXSaR@(&40;W! z-If2C5B(_$K?(d>L2Ez!+d59I&Dyb_T^$@4b`w>4q!|38@PY(MMjzLaq`wN)?|%u8 zh?m$&B$`l7-lm;)*S^6j|CPh=%T9UjxfD`7;T0lqIsTY!upeLty4OMZ(%P%Q<`+p8 zGF?9Vv)!0DP|(i!=ZmBoI35^%lnwH$P!W`IF5E6udN(&jfY2fUC zM0ITWjnhCk^A_?Imz#INX?cqmI8iq%UZ>x0vgZ85mYt~ZFgNF$+Vc_Hiou~9EJ8X3 zkJO~iw?7}UzeC;Jf0f^)aR^&;A(Zd9a%XyUChT~QyjwWIwRkxb*EQDVseNI?2 z>e;h7$g9A+qqPloMi#@w^J+dZUHi>0xxTqDZ6h?WE+2X*R=>_&jMUyXXzT2fR^R5X ze0k0yYH9e@g18A_{MMq3Cv^mq!H3MuS#o!|zDJs7uV%_!6)`R@uZ;&9^c>0cR$An6 z0g+7n2FQWmP{JVDrv%}%*2FaE&Ujr^>%G1+y&75G)G>3g@*=Ab=;7pZZ0@3me|~#Y z{4=1i*6}kzzfWI{l%~^TAPdG8#Z$VFbuej1Dv;@zbnS*4L1nQ)sl!4Ku9jZ1R|{zx zVj8VIEM6NN_uqNJ`NDKMZvd6Pe-lFxUyxUlvtFz<>b9;~=aoQO^e193k-6jsh?m;T zTITuiY_>-_-TkcAfQp5y^?+X|i8A{^wJ4I7nC)S;Rlyl8y2xdIsHJw%lMh2Tz6HyA z3E&d=2AS>D?@x2_m2}=IsnhH@PiZVKM@Hi1pkjV$u~tcI8J7+a5~F^$@-$tG6)8IO`P`@5Gr={)wvD3#Khx0QduoFFlsaKg4F+F#za6nte4q*Uq(v&nIoHn;_IOe8{2yu^0)QjALsq{wIlit1Hc?3b=5UTcg(r< zJ5<#~EFB3IcNCZXA^C^Kj(r;LkKF~mTuSG(`K(E;qD-jKe!h9Bw($7PpM^bR8ZoZY zoF%%~Rqk?yyPg>|h>za)dnxVqLIXVQm&)LP>ntwR+TK~agY>}^N@?j?rj>}w8$~J} zX1*Lzx>yt(xeTFh4RzT2n)AS(?qwD>;cGRdwoz}~i29C^hDUcE0iLEPl&h45gaH&4 zz8Nl9i@~X%qMDavDD*iuFXs&u@X2q1-FD3?AC$l2@j3UWkab@fI5Eq&_R=eT0x61q z>*N(Xo>wv>FoWtbSf_Gl9iTkZg?f<+>UYv&a`#ebf8tZ#6l|1`B$wy;>6)=GES-`XrWNg zH>eK2DXPgfhdHDjMBsZVC|!E&`{ab384g*)2f@q(xQ{em#0=CtpwpB|E8(ggem9pq zLC^?}ULJL9QS=kiNHh$t!;5D5M$VmO`fIIBuT-oIt@S3%d=yfVHAP<%`&sFSdW zwvbFK88==`i!VJIhp6T1>eat^M^c%@3L9(lj94nc(^BP>7gud-5p%;SnupK3`$v~} zv`xkGIEv;|F8-2S@FIWPq{Yys<-D&2cboT@0^B>305cjfTM%4V{+Aa1Pm$eW8-K=7 za?ychyW6EPTd}z;Zdq;-M5t9#1D6ar{Ut{-1lbM?j^`k=yPQ7Qd?W?sg?jPKVq^3~;WD6s+Uj6hjCaY#v+a^| z8?aTXmL+0EdEPSFj7oKt2U``!J{uSvW`F;RyH|EkJRd%14aBDp?deNWou4P}-3oRd z@7C}mpY*Z_e#|?1N^}(PGOYr;jq%3+^?&Z89J?Yb?d0zAIH%b6F_rCL7qoe9)K}-P zZ#5TF(^M^(;skWyQE~R^8benf_deKbP|xsMhaOQD`p{MN#*bF_9d}A^qt;bPw6M@} z_}rcq`{|=p0R3Xa!u^zWve|MDpdE|2zW$=w`1&3Ex$D{}3Rm0sZkvm<5<<0K-6eIX zjK-hLbJN*NG3tiek0Sj{chSbWUZ?!1&;85`u0CL^K5;_fgz8g8J#6gcEcE-@GBa=G zYQj*g;SSM)Ihgw(FdrY9sAjH0IbYSD#Uy*e+G9ys?(T4X<0XDoXl7O);QVzt?|Z@P z=6$xgrn+%qbnSS?;G_Q-Kp#hjLp(kEyN6oFnmJ zVTmwFjo~-P3adonH|>sZWuGEZpN2;}Oltx&BS|N9zdey#mL8659eJy$X$LJb)dv=j z{0bJ&Bm?DXlWXL@Iz&pu3Y4xdEfA*&2OAU9LzXC&FPQJAgqq&NL_19~J%a2NT8LiM z=ClAzFg32d{3kBl9*(eN-Xan9-Q8{XL@~!B=yrXh=;!r=HGz_L3e!I zPNZ?(v$^o$!z!gReL`q{?`gv10`r4PWQ@}EW`1Wc z@bwopuS`@-N8RjK&DQMRm@!><&M{k~9mA}TW#iuqGykONmy>~`u>vXgwvYpxi7!7U zJ@pT;)*(H>7iUnqgzpz5u~ov=$swc2kz^o31g*-DwR~SL=kkQC=xWYk(wh(MS!TgBmQK7{G&L#l|6L~f_cuKIuH^lXjtx^_br0JR zqyahM|KfX>$0ye9_mS?x9ITuy=vSGcvcT34cLqqlprkeDk11zj#VV}+~w&*lD}e;qOuzrLD4*rnu~p^80PWUqbVs@)G;&cg)FLd`N8;Dl#_i_fEp?bwiMknnYwxr@soLnmN0}TUTOIh zXlYb9ccx>8PClctX)1u&+yO!Rx*g2X4DBha_K8v}D#f29s(>8I(SYExuJG-J2>>ayts0y4VYuPO3+pk>gj8n|5E>eWC=);Fetd8F595{Z z;Sg=jJYCsxw9)m{=HNW;}IDxU>z!^_<%g$L41-r{~`Nr~CM=|km3qQp4m zA3ya*2h6n#chwDIC*8l!T@NEF(O#QI;Ao2)!Qf&F?hW=Gr=I4RB@963))+F**0ehF zRQ_dd$bV;(f64pTWC{W9k$7SfoEmnh$B=HgvK=9Y=xNQu!tJ5N&X3c~QKuR>7s$7> zyk*PXVMcu#n$->cDEZ!)QisgX*IyQPUT-@b?#a+h`1XtBstfVUMA4_kDrkCv5;hQZ zzfelXdL1F^#%Vn4p}_ybM#-}`3T2=UTk|iO9L!4(sShRyAyv#Tv(kQ09Y4F`&I))QenOjl+so!DDE;(yjp@?)t4dand7Y%o0Vx z+`I5CGJx>+&V?MYT1a7}zT^AVt&h8}O&4f6vhwS?mtE8f+)sfkHy0JRjPlmb%o^Ug z%c)IZQDhrE86(O9>irE5_ZD9!7&x}*?%gQG)~TpKa-S~$R%p{l{I>7WIlb0&BiEWT zS;~#w&C{uAc_J{Hps-0p3tTRd)DvJHyWIVIb&W;4P+;KEXkE=veg8Owy;+g0 zgm`iTZCA;o9oHAtaf&+NoCQ zHC)6k+uoxTyNh;MU38+3Z`+X~hAd;9!Oyy$&`bS>=%ka1>oxaYr@FN}n{Ff*QvkD7 zeZ6G!g3h=glk_~Y;eD-Qy+$V%vCrfp80sZuc1^eI9FI_;p5jLTVSxRl-RL85(gd9v z@J_}X@2s}$k?^I!5j{^1<968ckJ9L-L1^*Un%V}e;3PEwE#0y%X`U(PchKDTVEd41#_qh1F^4X<1y-}giy7!I5qY`)#gR@^U_Tvhp33T?z?(m}A3zc>BV z7?cB{hHnb(3De5Ng5^<{a zc%$~TE7>Xv&@BWJe3_!a@@%=PLWJT`SNZN!(Rs@~Q9?by7&i&|t?6 z@d3-(@&ZCYsfshtrgcF>!ok3R{eBA6Gd6R0LBlHy85z}oJ+kaiHN2K<_-Qo&{*srj zx$voRwRWtror%TY!}0L4WDB2{!2y#c&ULAM9nY9h2lgh-#fl*Ex*9Wkpt>YRUTt`7 zYm|M#j|ng7=wLcA1ODl=IB=`-#}M6hceeT;rR1M>(;kBFzR)xeg7o#iL z9(NhN`eK~9Z8twAo;v;0uRu`};-Tv4D=B$ldtkVT3}Pt{p-uiL7S}yQdphX&1aljP;6Z{&e~~FMo64;1X1% z7h@(>>nK+1Sl^W{*->g|U!9rS%?y^o997z~I>^RtGC_KaKp7IR_Jc*=Aj8Uog5WS7 ziJFbkk9Qf58MHEO<7OyFi#Pq>D>wh}moeZD4Z97=)2iDJOiufw8r`qTkYi~<7R!30 zV=gs2f9Xy3vo7-yPz&nP2(48E&B(N*jfLkf>DsA#!%w@FChHr{T#0`GmEv z8|L|j>TLzuX8jU?m7k*9r_pIrvNKx7Eoe$&MC_sRvW(QkoPO=()|WsJ_OH`sTtHo| zNyx`Y>%mFryMzg#d3H#+pC9q$fAvFb54J*#JGT9=G3?+fi2IFB~O=+UFdIW+6@i}L)s`Y zy9#9(U=&L)MM){j`x|ql%E;tkpEQkkOvSTwnJy~zV|Iq!g!)_csFd-?a*#-3Zpz9*@af=kT_SUFO>_Fh z)xiK#lE6SSb#45`26lRQy=SG;C4o~cJ7+gsug{eaC(vKAcnhKBl-4|3St9;PFUfq9 z6#F3n&&x0J4f5K%zZUc|^+^L(Y9*`wrir$*W!DpY^2w{;-($0YyT^O{SNi!N|A)&2 zUVvb}ArtC@--5bTuX`WJt2H8c6w0;Y1Cu65DVG||gP(mWREu>&HlM4mj0ELLQSzh$ zH8cvVc-O}B!_JJ|@$0&aISSo7g)ryT@xj@1fvPJS!BmqMuUc$7J6o+nPdNhuPiY-2 zoBOTR)Bj6{zd4Zhjz`gspNMT9I<5J+Deojg^nT76bK#3A-wq>ms7Kj{*8+F4s&Kk+ zE~(=F*49q}_>M%j41wgUBeIx`kGp>yo@7)xm` zNgb$+X_Z8vrglRURB`LR8u(r0)K#$PaRSEA`*d;4_Fe) zqldjXWqy;agL$46iyHYwM!J=XEA#Ng;haHHsrKG;_o^K$fmLQyJVEn%#tIZVOuWE1 z&z`?J&yY1KSKGBd*DBNCLRtuud*T^0SzEYf^s{>l)9p*o-Nzp?tEY zMO9s(CPP)n>Ve@rGmrX_5KioIV~&n&ViIpB0naBde2H5&!Hwb|x!BaxCX1o44UpfO zU#$BEv2;3Yn3K1fJ5#waCN`?IeM|GpmqD*b#{zOn(!e}xaQvn)(<|4}iFI?$OD^20 zLT}9%p#ncy;{aC=&PLP4sRkp=UqkMU@m}^;#2YYjXma%)A~i~H>0ujMpVDViUSO2( z3!QvJ@fDH%T6O1vzC7J(Y%@^_#H&#$>HOW+f0cSama#Se3n#39b6{tv4|8W8QtJ z5u4;_Y{B%6Cs|qaazE7nEVH~+2!@(E&i6uMfb{@p?Qf!{;&uaJTq;IC>ckvKO6wFG zY!@l`wAby`+zxy%w?JW9Vnh8h8uO&VFnM|jRPqR|L@pjBcsWiI4Vj3UIiA2vc(>J% z<|xtGx`$F0`iZ)h;-|M*1{Kxhq0ja6B=70M@wHQqNfV5NWI8}#^r@-IwnOc#r3ukn zG$22Id%h=7hdk;B^evTHL`7O9px-79F+5;pau$Y`i0L; z=w3lz=FG8ONR>CyY7q4B>aRoQd(%`*FHQEeV(%|wtxwl&)U8pXZ)aowqTC79UW#y) z`D&wVo1X8P=oE8nr8D6<$Z38%tj}PZQTYjB4W`VhWR@N9on#`?j*UrZFi6v=S1|JyqfMCMkw59 zFK>mX@$xS<8myJpKaU+0p^20(bO^y!x3$@$Q}<6J>xN^6;{BZz8GfV)g)&B;ySlO6 z8CRpopcAcn9g>aa^~s@?lR@_`_vBCDX0kvL7%c0Utr{$4?sl~R`}y?rvi=H5FzmLd zV}(*m4L5cUF4?>No=ZpCzT|x(3u)0LdS91EzIW7lY$!fkHMS(q`@wyPww*8Bv$8{P zvmq=!9#=X}?i`DMroX^dOkeRUk6<+#t7`f@n6}Mo#y}g&vjCUrAX#A3vLmf+&j<%< z98>sd`@M-X2x-1O{?F{;y6oI>gqmoX<_xz$p3pV~X=Vu)l2A!n+zXTRwOfbvP#0z9fH4l~GNt%ueR$1s|W1X z`hVJb@3^GD_J90cnoVt(4NYx0Qd1j_+%~LmQEpVUawQH*LvtY6&Pq$%ij=uvhh#&bgk~^IC_#`!)GUb$U}@_%6>+ zQ3j7mxv+tQ?pWNUvM)7du}9KdHj$!yy~`@~A9H zWPkpZ>6|TY{;~4q(CzwA%)Ku{V~J5Ejm@$3la{6I=nQ7l*#{Li1*J>F?y`u75rlw~ z2@y)ncmcKM0QJt{wW`QxVa9fHXwpegPwg_5eteYhX6Q47hjot1nQVtpXb{pzXA{?U z?I7RxQO^VW9K2kwe}oo$UT-NV5G?KLTNCv`@UPU*sibCiD_9*?f1bA=sTkbtnC^AM^o^EqdeV_VAa})rmi6? ziK$N?dk*v}Nfl<^fuGv@?W$1mBA36t`X8@&TSE0ivFE${#}K?(&Sh3&+ zPMxY>&aSp!TJM&7%h#sjl8{w)=uGqP2cI$De_vu+eE5FOmvX!ez_d)fPO)r}PjRIF zj?Ez#A6qYFW0}_8lFKfbfnB{~TvkW&1*I!o80s{)MStYmywirY|mn|^37=BPY-T3O=6o?u*v63yCdiKx1)==Asg`G3o-_}crH zA{&uQy9?TP$LjKe_kby$?I`J10D=p zu|SLACFL*6&O7DBnAq{tD86`c-&g8MW_#G#%xzouZgVoFoZ9rKQdxUDDMkDSfT9=YT`hYK~caAPy%z-;n-cB8lkzbAgnOD#QZeyCny*HA|?QQcu$q#JE`Nr~E zncX&EHO8f@t$7xRT;pduVg%w*1Wd;K@V6E_-1M%1Py? z|35JB<3NuoqXqIqpf$sdC1li33ky_)>}?Te;}L@Ynfr2WdEpmO%aohlMlf&XDak<} zT>Np8*?`q2DgcEtX6)<_IffYie?s;Sw3F}ZVgCTSX>G4bqS`~UOt+G81C`GTI+pd~ z<3$B8_|G75Qy0_3nDRZNbBbU=Q<_}B_jC4s!)WVssUJAZ`j1+Ug&F<#hu7Ubx)jtr zv_OfTp*7&j*|9w7{e+(?W!gx^TP#U-}kML^?NnA0{L}cQhYJP(I+G$7Jevr?x6UchC{>Thbu>O(%&i`A~iAK(5d*T3PJ!23oxkI$ACWyS(cxNO}z38MP#v1yj_h4>URuRJ+2nJ$R7u zr7AQn)3euB6AZg0(F+SsTiM+U3k~w(e|@Es{%pZJIDvC<02bB%F_n4a@J3Cku3D#r zMaPHHZYredvKd-I|03Ixj65o<#Vbwi3&e%U6;2N5lP|Q(-jA&RdR984fxLl8X6JW6 zpDqTx0YUj!lGwz&JK@Zw(aATe{S7j~6-sG=0^%GK-f>{Y3CNTw3dz7MqwKn`^hTzh z(gu{hiTcP6IEBve~lyIqXXFHrUPMFD?Io(eBhpC&d{QB+)m-~%F3*IRu*j< zy=9qp{O_eoNT$p-M8d0wV&0VA_cM}2YT@TzHCWM^J3Y&=fYqCK6=qg3G3{&5lB7ce zjPnVogn|;|$#hnAIcXKz{|ROM1p6qo=Y4-e;#71O8`!h@qzea$Ysq8A#W-8ZEq0>o zNb8(uLq=%4nV0!23sxvwJCMS^Kpmsr|;uQQjF=v_Gg!Gy~> z@CU}(wHT75b_sjex%4b$>y@4-U$xE80hlOkj>f{ZqTaAU2j3BL6#4-`x(+wqv0_g! zAXP(K(mUR-Ml2d=VA@8HtPo(;}3MO!{qkqxbikcx$ zeA|q;lCPzaSXQz)%DMrf-HE`H;qlW3>~Vkg_?3#KP-?Q)%!3=J9_1_hZXJAS8ZBzW z_ugSA+x)w{PZE((_E2Ef;O+lRQ!By@fkNN!Q>2>Im*y9W#lZKhCF^+O9K6<4^COq8 z(t7tL4gKcXHno1g8Y;y$bbA{Gz@1t{XA7cij*kXXBmz8Rm5PEvlaJo-BmV+s%z z!;tNz8q@&HA6SwJwley*V{>`j^z&L7MPINIMT(%Pr)?J_EYCc2{Q97c0ss&R>^tw zk8-2mPhYnbR*uJDtianQl(8+nYSODSBGGr6eag=$-y5;iqH_2fy4^fG93* zC>~4bGy9(`!pU6I+S?9Sz}t?#1t3i+8A0zQG@K?4wNa%^@n6w!>Nl z{J%N8+dFs@!bC`f`hU+n9#1Gh0$wYF)A{;_(izuWCsr$~e0qWbxT zP5X82T&;Q$Q0Pf(yGWONA8r@)IBcy^(a5Q?xs)~H+~w~S2e#~O`8g;HAUh5a)TJu# zoS$Mou)P<5`}v@s9T? zBQc{V35Bb5Kdl^NhMpbe_PQlvbIx1c3%3+`6j*o=oX=3r9lBT8OW+sOySK=`9DR|i zR(?U=o-sW#uhOo@)x+IbW*i0m;+YkA5c{t3_9ks8`JLds>pg%tn!VZB)HQGBanpsT z+V(x-F%2ifhlp7~s}XTgkw&QpuoMj0-UpvDd}G16d7fR}i(qdZU$Cya?||OOok4Ef_(~j)*{ZT)JC{9!KsDt36p1lDO-GiOOX~9K!jNbVdjiG*LDZ-%Sq*R) zt4_TLU4gx;V=wyQ(x;f8?)#@zxC@ZxCXXz^RN3~Sn*CV~L*>ckLI>~wV$bpDqsp!| z`R-^nLe>nf-&{)d{unyO*fbnmaQO^x-tQU%x35iWuIP#J?9$VpI=*M$^pwa;!TY>s zwPw={_7`_=-2nP3EfDmLFZJ?9zdx~XvfMckYiHLRJuj_Tb*Wc)W13aLS879gRzy_x z;FUA>Coh|11+YU#Fo*11>eY8Yt-)pmNQ8gM*(0`*5QCKWpT+MbCguQ)+4GxT*|jdI7$=$T(o#Z`Af-cHW!``uOE`ak+^ z`0_P>3S-gs{u)PeMcoJ{3-K1exW+e@&#rWR`U{#2T-4J2@w55q=kn{!UqQ&HGmVw$ zQH523rI)w!GW+p)qeiz7KfQZ0pF;eO;r=Dr?c7M_;9YC<0dvEb0DPQKEWksP(xtHN zH4WAH^vFLE-jkx#Cs|Tjo1>FX{CZmR^YPoxE$BN{?!D;q`F6X`X5@b%+Za0bD!Zla ztW=Ax&hJJcN%Q$Ik+kiVT1Q6T zr`~@`zuFU^z6=}KWo!NXvCc0FCakJF>fK7iM;J*N6uQegU3KvB1&;0@`G)krij+O0 zbVZRb+8mUOtbSL7bmWtPrfeH$Q7gV}4+))9}jeo=KG`HQHjLcDWM znEO4+=14cSqzm?M0FPKukIzL%A0o$P_97-v3Vmk1Ol|6S8`1SdXE_IqyXfNR57lO6 z>z0!mZCPLAr2AHg$K#aQc`9{;JgHHKQ?Qdyd^O;zW(EYmJ9pkvY(H>&U0s?&z>`7Z zTFH5OBHg3Fd$B*YM!y|h)y-b#R5^?JzCtM=HxATVk)HpTJNP?kDyrI}(bIQw*Cx@v z?W^+v?|)KbHpG$YQbw!pOIxL$>Fl2vUHYc9YFfM%AqCR8g?jdNBbk-6t@(6kw)?F+ zSC^#%)?9(j^sB)E>TFmdQLkIqDp<^o0JJG8nnB)iSJOGjRr_q%j7zg{dNiK8Tq)X_ zW&R(_gZRG$#nJPde71kipCzS%{w7>JvqI?pjwFJVoQ z+CF26^#xTWSq(}>)!UOb%+HzjqO~O2r)G5&HO;?eH<fQh#wmoR(>KSaID?cW+TltQrts!7xbF-}!W4~n6Hs*QI0=)T^t3+(LDIRbVRj` zlsi)Vsl16651c9Pxb@2)yZb2tn^tII*$R*MdLcwmT}MNU1@Y$Um<_A^OHy&R8Q2GJ ze{MGW4|)1GSx$ z1<2l3zCXHxv{ztMpS9}Oh>D5&W{ezV7!*W2`{@<~7YV#t`m@e>H}9l|6lf>Q!7aqB+g+h{G?p+g>ReoC!5Z0+KSjF zsxT&BB3C$QwP4e1_j#-Jylow5;ho*feJeak1-Zz0B_#0(v|VUOH_u3z39g6NM5@iK z4#XJn5fPltp%x&@PS;Ian$;dC-`%-cHUDKLsmbh8&LewL_pu6_KeJ|vuE zv(%yzhw^yvsjpPDm~?&rg|D@@j~_(rUD*60`u&HQH9jZv^F2MkV$ghM#c2D6F}gcW z+ixn>XKD=#V(@<%`jM?2Y=rkR59HC5@)SANxrrZ3KPWvWpd)WN=Xmp-&w!`GI*uKJ zErl~$ORucVj*n~q;h^KGr^M-K7wi0f)WI|4uKKyIbX3=eXj{fTlV%wg=5NnZ{l(3fT=>Ss8ZP$+j z!QET%Y;v3z*wI})(5i+sag*=wSdZ4QB0H~HTFER1mzoZ&4rGkQu3t zfz#{RsLRKa@6B)8k7(49%)daEFFKlMt=A+{wgI-K@@&Wyg-$k77503H6QA}l#W65# z$NV*=?h4lau$I>-ztZt70P)j6mPNca z6Dw}l6fMm#1S25OGi7!_MrmNBH`!y9oQhK*iBDJ#E9`oKTrjEh+Rz-O{NQvb1A!fpG66~)$K zyU4_SP9~eR^TDBS{mnH1beUyX>s70_Mdc(O+nP9xtdCzp+Q#D)F15*F?qBx^N}g~5 zSiL4~XrDI-skV}?^Ra0yFmR{4aC{V8Y{@aRXS+Q7TpeoW!{=I4C<7AFg$7+Zit?6I zR(3gUI-0zR7i>8rUA^NNP5_Fd0jQiyP_9)@k53~R>SMwi-jasBcO>tf-*ouk&D#C> zwbU+Pmt|naDH}Q~sb)Nfg||(t6aW~y-|8q?L-+A3ubp3tQIQZDVPA@shba>cC5ZE89cD}f& zw7ZK~>-Y%={`1DeGFQfS_aW2nhkbcuL^WcYq(|zmVYYDWQOUQ2Gd=X_gXjtnioJ%l zlsG(CR&P7MoUH_jGWyqaj?wBHA0NLiIM6;o+)lhj#0r<3q|5=1q>#Ysj*39w{^T3N z6*)#<8GCO?(8BczDsrFfv+8ENSe})zbo3G6*Q`DBbj_F{P(K-T*gB5 z>TAZmaGu@S8^uXG#J;78r|J%gV%Zv+MoCOfqLpV0rjDw=)r*HyjQ zN~KVjUlVb(82*M5g2y_4ug;`vNnU}X5Q-YaC!x7hJbIme`~3!wn&r8PCgK19oE(7t zw%DK)XB`4aU|zbJN}-8gT&)DaB}O5Ya0;OC)~O;zA~T1ciVx->*nU;Bf=??MGgK3*l}%rz>`GY93E^Be^fD zY7R6cM=Gp6pU{l2RP<4TiBnd(m7goMwH$P6VUr8^K?kVagwa69aD~&RH|sZskMwBU z|JR~7ZD!bx#f=4G;9{HS@t8}-vGZ1Mn?XSl|hcuag= zBXbHGy16wwd2M0S3B-2|AFznwwEEo@sQEBX0j6uppXkF9-!6W)EATK1xi$&2@ktda zVLZ8MZyjvOL0j$?0su7<9`w=kn%?n5NCP*i8X@BUVvt*WZi-E|&nHd_EZx%9-E1gh zBQHzgJ4mQpnD{Q#-67+|<+T3+7(~mKuJ*Noquo6SllMw4Bk}8yveE8|d)LCQst(*+ zk+lg!kw9eAIsMs<0@RcfOB>{5BJM!(8m=h4Go)tl-Ln@(au40?0fr)FP4w01RD<(F zIMBv?m>nJ%J)KR18KnqEc1H;7R{m*=bY3(r%w82oEE@WoA|&q6jTPe}blNrLKe^kl zg$FG()-ZCYrDC@!76M0xVQUb=O8+`#W43T$Z!7rVpW#q&r~?UXW1u5roZQ;2)h>rO zi8cWwTUGfZaJ6!h|2BEQclc)0j#WU4IIoXBcqE!zDN?X&&4KyDrNC`^!Vp1j2)lQo z58M8e6!hG`+AD}kl4b-qX)~6}DSS}e55m>`nS<91Lx*3@G@jkkQZ=H~jf?79=$XjQ z4iLzfMJ7+MUY7E=lVn5x3jhXB3k_{b=FGO}W>M>ag59XQ7z2AM^j3pOdz>|W`Wu8_ z-iJ40BIKVE+(N!9SD(ea%<#=Ab_{KErDE<6pEA+Hj0?fu+&D4vX!U6jpc)p!#bi_K ze1p7$t}G5^%O2$iCOO(bJ_+22^+;~(kV#sk0wy?IfUD$y(JT|)Kz_WxJ9W{1TO=O!yKi3Ii zTNuXy(53YZFdQIw`cfRaz{G*fVCxLM;{sY_<10gS>eSo2E#**=S!Gf0n#1Q2WFH4U z?J~RC6^#%ik^BZ7=`a+&m<|ny=*PyHjf!{>ST3|YKr6Q+^196wJIsX=1(zU?z7Ci5 zf#=SrFm=rXyef8U}neI7oaserY2R&#sO0h|Dc-9X)`EtDO$~U(=eq1=2Vj zgUA-U;CB?^jJc34EgEi-_vH)-dqHqDgv?go0NKt*!)3~_VvPkTWUfEdv67uh!G80w zDkt3xWSXSXDRsdXiSjDiGwe*y(&Cjhy`hyc@3^?hd*qwWwoK}C>DLB&zO=N+Q=_el zW#;0}lqhY`x$KN&b9({$xt%rYxh@;wuALVp{*cgsabddmXD6xU2`%jhuT&O%`tAh> za+q9S<-Keelg<5}TMu4bZOIn1a>)xfJUgrA;i?%>3+%io%TQs)Y>HzW7sBPVKL*lT zJ<|2rhizK(U!w4FNIt*NpQZ=OF5)l(N^#l?%kluqhq*2womw<$)`rC(T+wB0<5Qfp z)Ad!LK=;x}XW;pxD%-8qnC2XM#ReVF-=P|vVyVb@eE zM4FYotGVIR*2n@DLQ(fy1aYyi=w!nsCEbn( zga|zR7=}L*ePGr!GT3lBfx7KFufel+w9yWr7%Hz=N;XY~?uXc!KJH7lS zJN8~=XS22TmpCgqzhBJ~)MX^(RSfg(8 zjJ0q-UZ1gC`cr$lpu0j%KaLZKuP2lRj{Mx-X)Fl1(p>9JtBmI;GCe!W=^l8BoBS)n z)|RN@*13Lh5l0GdmI$=E)WKyaL9fv(qQGZy>tNwhd0mBr{?GuRbP{*dam4c}66Adz zD=@`@p3czzrUjyI%7H8EYIV=2wnBInws;EtKXRq=%!6(+4f9B0j(jxw26WorAP>W7 zKSzh@4t%w>QtC**JERRGkVrCT-#Sb$Vzax}QanpZN8Q-PtH0dkDVGwUkXgpg;uRw+NCPGH7O8Zs?!s~}R?_S3Sqc%Tay zskNl!3Mu_Sb(@f3gqriYvzS*%!iAcK4lX2kM8QUUvHmnB$a}H6)1qGxY^h^k3rw|{ z9O_(Ilondrg;xN0SrjKzHfeSwxXRCwAC^>r)~pXn-n72y5JK)<+howinJ1{9#h$=Z$$mY1)`li|$#uR2V}0CW*5I zuI%F0MEooUxUZRjKs;8z5Yi|T--Yxq)e#qwe`zs0R&a1Q5jC0aRjnjiusd|VNWSzu z^h=J?VFHpQdn{s??Vglr)=4C^f?2_g$!y)K3zG=IS3It&oX60xkge-< zcFL(*FEc$<@|;%ny_Tgz(_Stp#>B&4XNP?@6H&`8!F z0nx_?;RQMeRpKlr&+?gi1{LTA4$2lc{GHkrKCixX&gg( zry$cZoZc)BOug?{GK9o?Q6}8ufm*QE@G2MS0(2{en6IfhIHQ)vWxrV^F5*Bp92dI|(5A5Nag%7< zKMLk(=Sx_*VH*d77u}UsQX`#1_b;bflU>N{vtbz(zv{4~+}S!+USQbjJ3(MJqnPOg zHzuQul9kxTX%5Dj;%J~;CMhD&L8Z1Z{*{kC-Jf?msj$Jhif`n$Ia+D^-#y5i6{1|R7)OTdk>p0mwsMgr_$an` zdvaO)tK1AHak&vgy*sxfhMPYjf~a4)0c*=LouBEh+%|ApqV_kvkm}(6F(1dCyH=5| z$nt>u>G>}c5J+U&*D8-bN678TiHX|ImfaHzG-5d*Ep1TcW2h2*vd4Q;1uo-LRJdtG zmFFK;wbCH;Dp!u$9-#XDl(5BzI*y(O#%3bSG&6F+SZV7vi(3KkD7(}Z-8=2%$@S1;cy2kmzVKGbkz2+4{^nFX zY_yB!%6mR-Tsd_Y(!WOBp6ueok0$zgkJTyDqIC5P*RJKciT`xhUAyixj7PYwZ)Awz zMFiTP&xH8Wu+TzHXS&40w4!`KqKvYm!f?>-OhX0C_5{uQx}Iyw@gnh*P%>rsh_W(U ze34t1e&km%z96G~@<;0iO(KReabNt%Zgm*nZm4{%!DJoiSGJ2IFPDSB@LSeoxw?XH zRNTanhDqtvb71DX=7Sr*5VH=N6-w<@Zg}oQQct6qRXVd`!02*q;0ae~K!e=Csf@JN zFv`fR)i8B>)w_EBM-vDnIn1%7ZZ9H)SFoNDQSmiM=pr9&pWZ$wL<9-=5grUTu2Bo5 zbe^F@!Nc$FJK2Q3965@y36%$zhWWOSe(}DSZuC7zuTL1ZTSIK#-<>XLOtz(oDkd6}e2YOTUHr7uGPl zZw_Op9|{juA&Z%DWMFv-m)3f0H=9sr)+a#)vTWyDl@+^z)|PZ^mG7k9(B1k8bgun3 zZV4^@UjB%c(CWe7da*HX{mKdmw%22LJU3&tasn8?TN!PwVJ=|>;X@Lb%yVAauVs<{ zpv$@2Qtqa1_F#u7-i3s_nA|Fhun}~bCZ^rRKyxxPEY%#nn|z5P@j`EnUCfY9$jmtP ztw7tBP9xeUEp*aCeg(IxKX+PfU`7QP42yMjBFo|tTa~n5Q2E5nw7YuBYZv>}nW7zg zL1E@TO;)9@p*r>FQgy2Dn%N4z8M^hH(p>D>NU*H!K08eFu#8y0UMA$#?d2gYZli=Q zCWCN?_xr`KSPjLD%1{&baB7JV(u4Rz6S>4%mV*>Gh(b8`VKi5dVg|=TUJMH;pA`Iim8_63!6F05s6OsvZ zDXKjW8TB>sK@6~T%iUZWc$Jgn_w~u$sB$eOQhqz#U$dv0Vp)- za*sHYAf)9*E4!KgMVm^~zCVEZfI~~V!WOkL06b(GFdQS<9^el7;`Lex;$hQ{SF}I5 z;3K*U4Lx}CJ}%WpQRXN!9EJQqk#ZaLWsO+QKultW9HzMmo_Bml``Kj<{cTzBIs8(A zAtpH7Kh&%2yR-ijMsdw@C4gO_bmLi)fsnffw`Ise;%n)4!w3rr>kzAuYZBMoGF_38 zMu&se&(K_!xrcZ#V6txV8D6=HdXiTd#naIsETefO$cgMQJRV-Tad&c5o6X`i*`Grs zKO4knwy#=?jBv@_;5UD*Fy|6&{5BS z2~qi7u}r1?QbXtP9KKV&DtVjU8Cxl3I*F*MK;@_u>ul>NvvT5?!D z=^-?+V!4tP>tgM@gmUiuS+V=K26p#8S7i4;LLyO=kSd=a!644dY}Szs_8MvAM+ksE z;KSh|1=k=p3;oF-wroC(aLny>QEF9cNG_i*K839=?dVR>rht?$|04?JXYroE&;Pw> z$Y~aR!dUz(f6*yaWLN543_76NP)}&nPywCZ;qOrM7D zW`?bMjau{3)dJw=T*!ZUi$|&|3)M7 zlSYi*E1Ztc;v>TipPh*}uZkUW=s%4Q_34d%AQYFtHNY?-547#m5~GNo^U z2~2hx9;sg09VF{BGWO$*?R06jQZOsUKLC`=>j~yvQ*{_+$S~AY8qg?+pcOb2*2{pU zQY^$4HxR;lvY+=(fT4oEWwwSS{K#TCoBuVT z&4g0UuT!BN1RI4!FxR?y{?P7VAqSue#litZu#NR|q~)9>Eb6J#ReEf{i&*+F)Ve-o zX%EdSDtDJ&R0NheUf0kXU=;U%Sv6sZ@zW;Jkioy;JzF+=Y$JXBT8ykm#6Lz#Hwt~O zY*Kg3KqCUK#&;u9;)9VDu5{&4YfH#Ydde2-axUockEQphG&!xfIWx;`{(4hKXdcmo9e$=v_|;P^u66qEMGE z0Vihcinf=954T5FAllF3k~MWvGF;?c?F&l=7Mn4CGeZDz4FM~`!#{yhY-}N)5UIL0 zmq6%p2BXp1XC72$&sP+pWeOKZ)OWk^M@xhV^F-+DGFClSFvcc@Bp+7K*=1}gHv%*{ zoSD8$oh{A^*~6|oC1`4teI8{S za;KhkE7Y$GqZFT;7~y>h#W5r1db(AERY|&CerV}WX#0@1mNeJZC>#7z8w+#yicNrE zy{>J@jg14iy(`Co#k;yxyE|GS0yn_$vEZ;EzNZ`JLQ9-d1r#jr;*vCLj9C7@aayX- z1jgygLTcb3EAU7;UYj|V;sw?mU_BMWEOd+zv7znjDCZ8CswW4?`(2!VFMJDtkgPnw zN}sYD*+!w+rW438eBa{hIVU$wy6^h{!8;?GLn&l#h05?4c@oHRZT%e#utm}AiBiP3 z-5sOX>bVBaxY2tEVo~3S6t^6ds84%F)nJ_SnprT&i$o1CW}mWVV#_W6fkVQ>H27_E zC14t2n$54y;b{&Iz0&6y`;8GU37uLpLoQ_IukKOdWg?;c3}^B4K((u0m*A3X9>gJH z-S%X=p-AEX#D*0ClBbomXTCeKG83l}iZHoL%7q1}-3n++H*|UESy%KQT_(W4`lKYhV*p6P-Z8x zSJu33B(K7~6e)fc!FNhuGje3Ba0b1(?1eK!4kBOKHHJDEtfxzn`2{ z)BV=g04NBo?_Rk!mB@~&M<7l;Pi0qR1-$9wlG{|=D)jB&I6Hq zDrkKw!K$i!kK7+mMp)nTAM+N?v>3~{Un%widyP&jd6ghYZ>xY#zKu4JkgYl_%)JhN z5C-fT+0^VVUfV<`5vr`c79^Me5N81axR-Ch@sh}6k{THN8cl$1_h2HfEM`sI0#vxS zVhpV4RqXka^&6!xBH$4OF;7j4jnK2%t|{iD*AVo`*u$ll^14u^>+iBx_?C;M6MKnK z0Fcocm3HnLs$j{7SRTapcM^(w+!~9BbZ|{0t&yKxvbx9*Tr4DZ3)7Btn28>Sl20FK zoGC&idkE#8_T5#Y4be()3(U}=aaKSe?Dmu<sDRTXtIjB*(gKPFCm6@OKky{29KdPo=&KBfP|W?DxW6!u03rU&!FZARuT>A{hs z6B@)Oq^_c)BHEgjK9^}X9EK2zEC`jhtGGt~T9jh7i_Ls*vI57l#_I;H#KiW1Q51?7 z2w=~X!n6JdKJW*lrB4@Jcqi%D^P^RV=Z0#P+4UVU)7aS)KHQ|C-)Q^z-9e=R0g-f+ zeIg!z1pQRFe-+(WA|t*=?RF{CuaxBh7)3mUICP&1?Ih#trL$#hAf|Q0)5_20PT!ag z58HS&Q3#+}b4s{NT@%K{w?YngtQDtUdCe^?^33c?PVIWVzkwoVY4}*Gy+cra2+gI1 z9aMH?EC5gmWCn@vksr{D-m1lZW?7X%nyd7Y170Ka&(ZMUI9TzK^p4*t0W}6`|wM+z= zMxvMTZOA4D7~+H~te!I6@^*W2dhUopJTb)YBJ(hyvgm7wkj^9?LG(7X4sSUBBefO{ zQ##EQw#$)0x)~aqHlNr|(*4DdX=WEszCzX@$6tHCwn6LilsMLqw}OuVD}1CHD9$S` zWBCFAWG@f-QlO*=xgOdp6p2~aq#VW-*UO9t$K#aQ(6o#s*ChXTq25of%$iz}227W8 zh(1U@oqut0C0qS>pu(VK+}xexwSrM{leng%n5=-dpi6vAI|}l} zaCoTrTIC4EX=S!FLh*3nDIsqu3htNqU5Oy2iMNPOjBJP3K5@R7#|V6}ggl3oQGS)6 zoEJ1~b|cic+)IeqA&yJ;gpIP=MzZO@ivkWHk`#n}@PJPuX>bQRDVyjaMnV+@H5KKf zs|#x5sAoeW0%{A_s`m2A(=ubN{)yLOJ4ryllmF%b(d?tY8Z#y)z^bn@LanMu-HQ1C z8t?zW=r=0uI&k{O*%Wg>vwtDf&BQH`QehEItgPc*x#wPJ6f!h)MYguyy1x+rdvr*O z#D7_f1A9;ZD1DJs^RMw6Av@}DdaLhtW`1R11f|)myLunW<_DRuG`EW8H-mZhP4{p9 zE?_ut+9~rPqRA@^zvMh(BK4;jBGs{a$PGuC1q`n(FaVXWhutcf1I{Ba;Lhq6t%|?P zX{2?YAb_Y!XH tzztO8v3;AXwkJOn)@vO01C9wqVDk=NaS!s8{t|uwX4aRn7jHiL{{S*~OSk|4 diff --git a/docs/img/structured-streaming-window.png b/docs/img/structured-streaming-window.png index be9d3fbf8ba813b9883b2e38ee3b42536ddd1646..c1842b1ca4fe09af0ad2acae60251f712c7720b6 100644 GIT binary patch delta 55306 zcmbSybySpn7cC{BAf+HBC=CNB-5?Ue5Yi0Y-Q5p}K{tc6N_Pz14MT@?x3qM_9rS&_ zyT0%Kch`CrYt}5*Grv4%pS}0_v5yfFj_Fs9%t{TUGc@wvibsbtXIw=r|j9y!0LC)Y2sP6Qv?}b?Dn% zcjO8T^55##P#9fzZ|%!l!2RO&GF6$?M(Xu8l;ZN@;yU5a%_$p1QcdOW{4a66Ay@)# zd*u86<4L+?*GxVqyYW2)DFhDZL->{pl>vuWm7{Md;>s89ey1El8db3SQ_lhpy&zc4 zxOhnZk@WW4fcc9EWF%3lHnfM1ST!08&lmt{)8=IT{@0P?!O}fWnZ9o6hGnQbBs?Qm z0u~_kMRG7XYuAlV0=DjgO=^MSMbD zCyR0(j}&DywPNf|*aFCh`K-OFCTZ6Rr*P?%vt(&DgP*cxiT8@Gto6ok=F)o(Zvm=+ zl9;z3{Ln})ktfrDdj8u>i4a~Qqgw4YbDA0KxHMWpqrB?ji z!)bl^vVZ&h5yk^+oZe(U4jgwmHNd~gX1QFq%0VviY$GKTthf-br(_Dxc^p4eV~>VH zg@kg7R+8~m2shc^FB?rJ%d|W#Bu_7KC)BgaS4H&WI=FD{)op>1C^5$K(M(o4X0S&>A$e(U6^8zU7d5l{P|DwWi#4CD8b1l8<^}7P@23^D+Cbb`hWt6J`2vNK zW5~BztsHtxKO{^)zj#dVyE$i*sbaI#Dm{7GeMfdLT0TArwjLq?nw(tF=ctIvFFffl zuE`9^%UJG?DlgmVNlr7fN@sD}c}%9jWMjfTQ?*Y;vJ)~uvMW6ri<5S?Kc`UVaZzd{ z6)}9_)Ka-sDy0$07>HSYgt%6tv*A^Tb;_ExMqToxM|;mANLj)Kr_xPu**2?v=RE5L zgnk6dQsH@`E!qQE4Z>0(Hww*30RxTZDO>hf878gXEgiuVCDRS26_*;NgYW78o_0LC zl9M!>Dwqx}bAM5&};}7;nY}5TtS)pVrq-1r~xnvKoNGcZcS7_7Ahs9O=D%NXyW{z!g6^f97k*iK<$?L1c=|f{D zGsxq@AMP{w?)5uV6e_%B#4chTkRkO~;YZT8 z>=|(BKUKhqTPx!(yOj}|xQZ-()T37tr%b(OMJT5fv9 z73E;F!gy3-HUAaTET$+C9#}@|U5R$W&TDEHjeKdEHD_6zh0e4XjWUq3XVz`Up3RpK z-IOEywOW{E*{YRC+4-^aI6gETepUErn-yBJ(4rZ`bZc+nxJv*s@M@EmX}E-|D^ePc z89g2f$A5XSLjJ8EMXAa=HaeH^ndN?YpgfmRnnNcF5HH(}u7Mn^A6+n`}LY**Xw`bbF zbv-D4kyYHSzRpMn+cx_cHZ}1RO8+=!cV@MTX;Vu~48^7X?v3YJfZGU>zrV1IfSn z#p31_^1l4SNomtUN(cCf`l*nB=PE;7@q6L9&%i0%tlVP7^wMUjIF_1Us&+H#&+v{h zdGFH7buB;oi~a`i@w&S53xo^I$6dx>s|l;&LMM3&%rw~XHqBwY-)iVe*eF9(l3Fk# z%8W}ysz|%jY?Lywl+XwFhtSQ#Y4;}z!V8{2vCBeM*7?+)S&?o-pg;8nQ0jJ z*?+G8FrRZ94SbtT6HZ!+Rp1X&KZ&=(HBvR$HQr1xy2(&jOI$X`H&a2$Cj(n^~A`@UMn=W0g znSh-D}yR#J~8pebep)Dy-O=mO~Tzydx8yOi$r;I(pKpN=|;& zp$1^NDwza6e4F5GSVH;k-0w5VHL|I2FW|&{l=pdWL&oD!{>~+Gw1Rh3)PHYb`mL{< zj9!;Y@b|S|wP%jiAkFHJwIUbQS_4G&B0g>+OEOe=3~1~RWF!<7@P4BpUu-L*Kf5Yx zG=UeOIn=5c22o2k-cmh|d7hhg9o3n-6j=*$8M?ap>4Zht&_Ym`M7vSA`q2gu2Fs=U zR;n3}+~%~sVPEv1+4&Kx?GKC*6RkEVyqwrqpO=l*6?oA z^#;PZ+P^P8>g^$h50Lg=Msh{d!tiBa?VCf55p*?cLrdbe;WvPCD6Ms-ExXsE84V(Q>Gn^PfdWRQ8fCor8#-ft&-$+dw!rDRF+dCfwoI0 zfy>rQUPAueH@v`zh-`Yw?Ki{2!<@IH5%17#W-A7C1V(2mfsK(cvSooLn=;dzL6IA= zlsaMKN7|LfeDEsolmt-R%g2cz$ZAvQEf>dR^(aa{tR~y!5tFx+fopB|nT@uVwsrET zupn?NxI14VAcyR6xc&mf1x0^^0+MMCy-ykI&P53PGt#4u-YC1WaHlcVOWUAU=Ly`_ zc!t@!)&n9grBrxmVu^li=yzhIuSo4IP`s*Ldo@%w*fE>@X<fZO3Sej`mI!W>B{t%n2fV18m#lZ-^<6>wtndWFb-W9- znHCPROH$qAsz^y4m+H6c6v~Ivo+&DfUm*g@Rzv%jV7ZO7LXBnTU#-(|d2QF5=B7{} zF5uDGiPJ5=9t>?8YupXATenY;HfRhNP)JU606qJg72#+EgoL>hF%6sR{qBj^{lxMZ z-4*c^cP(8%zqwM^y1!|pwC_!vn~P;?Z-2kAaTNukrKJmq8Wg-C=d%9kw>2KKe>z*H zoGq_7<25s!3~_qLt)eb>>gTZn0BwQr+GPqUnRQj&#gGi1~Qh056Lt=&H z$Woisuc4NiyIsYjY1WUZcqK^4uz6f}xA`sEh6Uz5XW0FTfO79nzi$B0U2^Aqa3>p9 zRvUPoE^&W;w~?B`7O7!tyxk(JTY#AjreZldx$7w zx(WQ+XpQTh=vtIUxs|s`0|y z_?8vbri&~1dZW0tO&TYCB}v*VisUXheHhE%jj=sHw)`<)uGj+_hh?0EGToPKo}%yS{> z#K_h--0b6aY~Lt&0jg0*PDn6G=%2@geV%>gp+?MWMfIik=;b~j{m&M^K^e{RRggF& zhFQ!fr=wS*V?2`yB(iA(C)3N(1GFj~;2b8_;sgs}MFL&>Fxz@<2{Da)Zd?IKsZ{njS zx0S@%yc=7^+S*!s1}?qaZRqF6B=TcSDmGVw-0dV{e zb=_SBmZSwRuFEg6iu^HEa&8NPOWx)b;bVs}dp<${t9o?M-OcmvELh+MA9G*6Mm1aZ zCb;(ZbBi-hD`^}{t_*oxLGhMEvzzJd<*{rD{VEk7he-glJ6O}&}(Pvo|i#}yt* z(5z^TH|@W9bJmNGs!JwHc02zqBTlfNbk1gZbP1xkcl_N?0ULmLwhL(3E^CsT`f zX0qm}{Zd^6pI=Nby0}aVjd2X&p(H}e#emKy>P#EYtgPI&sH9^8-j5VZP9%#(8+b*j zJ=d!O5Q(2i#A(Zz)!m7GTe~zVO?zDQV-DN%@$4#%lKT*@2AFVft4U7-Ge=f!nKlF? z{5ptspss@-s~^;>H&`PW?^g1#v>c}8oESoNpTse7P;Pv!xl-egquF97cRD%&%J)Ux z*Og8HWjFe8R|Ez1)V{=^F-d8PfSV$M0AOU2xvg8D9Z2R&2eBbaC9$~b`d`Vz1B4i& zvYytn79V&fuWOIM)2^Oeb2^pGTud%a0%OB+UTcR>Gm9S)v&t@mhB>BI3durN#_J^j7+$rU8(%Yp82M;}F1bk*hVeZ{elEx$PplLVN#*O77c0s1 z5L3G79*6o+4Tw@5V|IVT$3|4ET*xG$g8fYE{0W<-6gA=(+pEk0w0CWr%Np!{+kKJ( zA@9vMMX^ud7UcXoj8eae7nTiBC(Wi`S3vc2?+X}%xxMV!L)W<@5u#`Y~<-om6@a` zR#?@_MAONi2I04eQhmkDI71Y)$_qjJdqQ}K>zVLWvr-!Z-5!6+rkE@)$s#D2oL6c@ zhJH^kh#`T4mxPg!H_#j%V|gDLcS=5nn&}-Zhd$Rd8K8gs5eNM!#OT+v{LqkN_O_Sl z+@Bu(YEy7@uFP4ZJ8XG3ZU(Xl=qos=tQlqHDlEP_0=@lx!WHIt*4bcsJiVEze8t@0 zyxFnvPEOz0SX$0$Ui!N!bPeZJ@~cmeO^ov>gsrr!Oh0C6mIy!)1yp_{tR#M43f@Rj z2~_=hqo$>%1Lb9H)nKn?8h~$u^fYN3Cxz8Q$%q@;EleOx%#&o=L0=$35XT)kLiz|r z<e*I6QdRu|v!187vSN1lc8_*Vq-f;LU=Z+xEXP3*Y*$`wfq+iWzI zzWQg$V>V{-5M!`KB&QBJ{ddn}(Y74&Jz=$Fs|gxC(f(aH1#=+C2N~!>RKp8z3wAta z4y$CGOaA;`12cT#$HW_hOiiHr!&z3_A7Fm9&?> z>?(_Y1L}Q9ca?hGPw?fFK-R#bJ$*#g$euKd22@USr7+~%EH)wLXRm(Dk377JR^#yn z+yxDwLWx5-jhcm;7BpYTzmKVRVQ0CHh`?$~mo?PNbhJqz@;e!EOlzh(KWm9yIVhvAprWs4 ztt10=R>YTOBNLdlG$>2eV8*$nngzs=HOO0yCYW{t!~_y_qc^+w1=6bjDj&lND<8ji zDpijB0cyJvoZ$bHdTGcW%sHGPWwK;R0t*@!@=EBJV zEz0xF4}N`|f%f?O{W<#86QPG5!Gys95>T9-HqB-NskWJdl>*O#*C1+BNr;+pQv?~C z7aNX#oG}*JvzQl)#JZyVh|*sg1Mf!wRFu#b!fW-M8UCYCCFRMPX1T9qo@xr=oIV_h zwT}yVikX28ya(3Px}VZ3g~tN8er+*hzE8mMNZ z(2HqlI%Y`Sc9x&_L^I~^&IV(|uMBLfGWTDJvf;M|wkn;^!P(vyrZjqgwWaGm+ta6t ze2pT_Sk*-7wX>}IaF9Hz6on;KfL7vGoR;6gLd0Z{)Mj(KlU7?nS)@p4KE)yjprU$$ z)s6HJ8%->#`Sz!z`r=>*D^fQ=VWRitvY@<4PTSa-fEFAN*VB?b<&0|DGns-ZK#EH` zORg~eEs-G(Dx_$#bVy%9;G&A!&3Kc|TyYTM!IDP9jxut&frhyjCo zPIsGd89g=3rR{(QCLsaAESWV-N;LxY=BP z^c{Yoamxt(tCW~flc0cofpI=)y2}QVLoWQBFIaUYEXR8lV}B>6_G8 zl}1h!M8uWtjcZz4po@3>6_oeP7O)9b;2e*L0GA0*d|bl2CwV?Vk#GsL-K@2X0=;7y zl}-OWBVBsm&og*~`=SHNOyYKF0wz|;rsJ1a1!%=rCXhAD8R>f-_m2YQ9HDspG z{OO3kXri?Ce2++{KGg6vjj6=qtz<<``0c>B7YZn~%tk#p_gS`wL8`W%kL?)l6jrVW z_bDW_vU{j}E@r3ArWswU=4>OfxMf?*zg*;}(x*kSuhz)rc~7iIz}rJi1}7beCSg6S z38Dr2a2au#xftx4!g*YcKL_;*zG_>|f5-xP1Yv_2LtymR?S07Na)DlpO(oJFz2GVWTZWiw;j-GZfzd6_N__SEO(+cEcMp4DTV)?r zEucS-T!{Lf`7myT8e_Sh)T@ijcmxpxJsg7S!L zWx0Ym&vHUV;jxjuXGRaZ&Og%G^C^o#GTsP0iq72~QRIZ^nuB2AG*8)ag31l)(=N zEjBH?=(N)apK?}sc-fP*pbxCHvgy0TRLhK|H$N6y@|zVJw7FFRhXRnOKp=m{Go z?H5O{@_BlPAKQysL9s2{W~qPE(Z}CHy^Ipyj*DE>Gj6B-{0)A5;F+yfH!)inp_{}` zKo_p&z$(}uZ0KV236?zN68^P4`buXHC`2Uo*zMj8ZxOv>oP0tnT=%^5wtS-rGLZV1Z`-kAo5eI||{8Y^TpE50V|} z^3n6Lw|T8_!00+DQ!3$jYY3nR2So=Iq<}8C8=0NBbGDcwV7yC5oZAv=E@}>HFUAd^ zyr?7jkLX_;@BQ@Pi#LyL!lucNAMdUR%P!&FX;}ZkBojM)S=~GF&eNgj)BwH{{uESdT_tQSvc|0ZK;sZ+!@mF-w2A8AYT4gc zR=#$5$d}edOcZB4W<1wY+e6T`ZwnPO!Q^j~*sE;Ls>BUzF^BxWe0isY3EDoW>R}a*8=$c}Q*61F1Frw4D_zlMQWH7C3;k}20n1Mk69 z%<3#5>phx_WJL9ddU_vr5D&_y=5+00qk;#vD(9eVoi3)1=mfSr#MWuj3BQSX4JT$q zNk$@PMV|N15&p}cyg;$xjq4+IoG`9onK}|Tb$)U(qoT4 zQ15s1q3Q7Sgx+x^8wZkX6>?MaY@(s|RTW}6V*$K>$oo`+`eLxa20 za>Tf%9UZX$B!9f-dud^D;O*F(Kl8>WtZrr0f@PJWml&`2a6|hHFA^*KDk5kH;(QAZ z3LFtQO^O2s>MC=1d_Rr950q-C>WrDwtWyyD^FiDn*;JIO#H;JM<M@JEHw2yW&pl2ZVneYW`7J%;0|YSQB*<5dM+8krhx8u1zfeSKxgI;cy3 zf8d4q;`uOPxz&|za7fru^;l_XY57eWg)_%qR+ov!>vR(9f=Sl=3C7niqN1Z?3U%dv zvbAGMSPnHwKCcSh3>gyWEjl&w41k<7w_nx-$$Yo5JHf5GV^Dm$DG_vphV$J3vD|4GS z3(Ss#&vtR8L)^08mxL!E7Cl=c2MZJJIr7dDfxFLV(!a<0USIt4pz$I@E_;WkesD9I z6`AKL*73vF>o%u~ors5SQl^!}jX&y)>`cJ`#iY~1UlTP=Q=jx2of8h%2CaxqD+pne z)iXt+`c7+ys?*)zt`uI^k-7C3LYQa|{*s)DZUP!NbBtGcH~se?K0q=A~u1S_ZOzg4utTQIHA`AO8)H?IbQXd>#bL9t3D1YxD#{l{tI9gv zk}GeL>o_H9e>Ct7wZ+8goXmM?u18=l@}w&=mt8*1e`+KgeO9G>J^$W##dYHVd& zSj+XqtT((+z{X&!C5?HUsdusvQX`L3=F?NvxHhxki1Q2+jUD-q_+)ohva%ls%)IW5 zD7f4s9|67|(td(8$z|(72fpWu_)1t+7Jj>aaHQPjRW~wqH!w{)b)8B_19s-0?ypXA z-syJeiQ(Z1a_Si#FVZBo-`=~OgW(QBX^Hd3p9h~*-K3kDe^L7iSHeMSUJMKzrdha}7u4UBP1{)cw!Wfmi zEo=3%1KzC&KQCH zZ=k}ir=c3PBRS-lXjI63_E^pGZ{z+a2z`k01G#VnV@#&)PJAgd={23`CaM>ooT02Y zA;Tt+3eI0^X>Dy~a7)#8zTNTA;#+>Q;`RVisIXE{BZ*nx%}e>db*HOcDy_~cvj+6X|8qL>bGIO zWC#BBYspBaWXpd3_{`LX-DDh`o(16K8jPX`Q)`hi?vLGgJ^oEhv}F~*;}kJC%+4mPba;vPQve(p|}exoir6Sy7G^se?C}&1|AgnrTuj z9a+J_a9O&s+e@bg_7LfT{pFUo3A;ZkxO&QX?W@fb7I{nlHNZ4hEY~@)H4&2WxHXC| zQ%F2io;ox!-`|_rA22=R%oL%3YR9@6wb7mAs-=+DCvUJuuMw+WS@nD7DQ4I9p1~5w z9|CE6g4Aj`Y+Fbu{;~*a@qy$1zecA2%=gV((Nj|o;MhHp^Mo+F_oD?_QpA~Ijr+Xs zik?k9uMw+Wt?b3a)lqZ-A>X|<;h&Wj7|}jDFJTf}SywK5cE`QR7Z%Hn*S9cE15@j3iwA}H2R~J;>Z^NKkekqEgxkiZ@*P5Q zEmirU;t-D$=eE(}^CeK}tln{dpYQHuL3Fqp(v?5JC4$~ApHK{pRu zgF1P#2$-Rmn0~7p3>#bZn{2U`~?saGZ*^ z*0Dmyd8^$*S96bmjW0fF{IJXfCyBFUW>kU&BaKSm1tqnf^7r1cPVwc+9OJ%RLIb0t zNjOK!7H7^|`iQh2u)CB@lur+b0Mb+}hR2p;#;1#0YgRUoz;E~@ObHjr!W&IY4mQZ4*S z_r=?Z2WYweY})(I)L|eH-nl(cV1DqD<}l8qrxVz@Brz~L<;tvQU+Wn<1zf8^m)+lu zXuN97ujIJ>d=GNtm-Hpp6uITmRww(`byz~4V z4QL+*!?aF)OUpbiLp5sTa>y{zQ0^fOX2VDRe{u1@kcJi+(zyNUxlgXQEZ-K-FNvyY zhK2b!lONFQQdypbkx#uIpW6yeGMou{(Rz@S2mlTZZq=#}CJ2+w$r*9d*Pb~;gw<)&+z!MWNktU#?;$6;)8tStLt&dy*f>L<;Z1{`BbViIhfL2yLP^tNacZWopQ zQxuYWKi_7Z?r~S0@@17+=EI)4K?r#8+UxV_zEJaK&kU`_z*TjDR)a3&hXeio)DIm? zMe><4m5m@H`YFYm__cV#-fj*aI;H+(oyOB_U|J?GwA4ur&DAPoJyqd)L zemD(vpMB1*yAGOvy1)MuN_2hwWlpj&dhfDgyWIZPFxq|0*iQ#NIC-iaS4189ZDb(U z9y&g=WLn~o9j<0V4fc7;tQgvu#I94t*AMRj60@CZWYqodfXt zf{4O6ahdI>zdJ`hc^sKfjiR8VAYkWTW5aQ!Dk7A-^~I;jez#ID-sC=x!KGu=)3BeV z&X*49qyfL`>1&lyWt#3MkNsc_?Ma`%zuC?fN<%)latf(x-4V)hl2wH;6QoSnkX8#yC*65uI+W@4mYMV z!jzToM99B(bf)BK#f5n*Uxgu(bnIZGfsd8esT?p(hx$Rmbx%Sl)kDl}DtW!WU^34n zUpkq6jTs7Sd+`|hp)|y5@4(79L9m!-rzcj3`Cx~!&q$ag>qwA0zQh2%d zq$@DzDnNm8U$jE*@XF!1#G%}A)ABylS(fa=kl5N7!4AH6C$g4(UX{Fwedw|ET0AyD zx@XuHuOxG?H{EZpYl~Dt>e~XgA{>;Kcg+{e^(~W{{u;!EXoA!5-h|9~hF9@GRP*3;%H@K-}Oh}iU9GN6*jpa zyOHWZgO0be9`nGCU_1fcl|x|L`dB>r=$y-*G88)9V1&zMIqfG2oI|r{etdbpnXz-i z{yBv?K<~LsX760Cbz-C0hHh<$nP8vpt2wnI9Z5p@>z2+GtIHTUPj}`+M*|pf-?taV z*y(H{H^EG5;ooEE#iPz`zGYK?(BT#3C-6`4Ry|6o=Ia)IVV+U8AroI2@xGtjR;Gs1j3;++>oDiup^pqaM2U#Yq7k2;B~TQhOX4`*@Z!?(%NF#q*J!i zI_enMA!HlQ$R!+Uzk1mmW7bZ5(#j=D6^r@tlakS}+W#h;A?l(1F0~a8?l*civ(#gE zQh(UTYfn(pLMp#+a8F0@KK``d9V5wz;x~VRQdiaU7l6}*BHadY|7STW?X{wmFt2gdVxmJ4_AkIUcQnu zDuIVE#VA7wY~r*(lqRRP^a5<)-MuEURDb)25B5?*DE!zw z?8TF(siP?c!U|+b_F-8zBqqRIY%H7bR{bxME74?s(O>w@*zrT#PUX-3>pWYFQXyf9 zA7GQ14G>w#Bdn8os#TxAL)X|MBg9(_H%lF|Q`AgjIMg}EEZ)zgtQ#}fOg%JVkEKj_ z)BF%^#0wFVbNAcbs`e2j%#&=X1jo!doA9p9j_)hIXXRZ_B_t6$Sd4E!IJQ+``Ar|e z_L0Vjc4I!o)V>FmwhQ0a@1@ij`|d|rL@y8mg#i<-lz*7_ta>rBerE!|jjq0y(}zmu=oZJu*lx4R_}x5qWg z%E(~&cA7HxZVB>IC}2HZ`#_p!RF^Sy?;Tkn9v-F;O3O{w!-iY&kpF=qqA{Xy^w-Vw zXiBX)$pKBDDpPegW2{zQewWa{Ie745bTN#c_<^NXFS=!iq8;9=7(_ZS9TA88k+Sh# zqMegnn60Fvf~`4OqWtRs1e$)AIy6l@x#EHB}c^nf1Qr znu`IPW#2T>rTTy%hS2=3hQ(Fd-FE;|hm`Cm8hG!S5*K*hkN)4QG=zrlx(@cS45F2M#)aNZid*~DQOMcuB+>5hX$z-*J*2>ZDg0lkPYYu4d zqwVsA>)>HNnpt}`^TV45VaOPH{cMA3k=q}k_mI}@K-<>*dwdvPg1M`DF*C4@pr;wD zu{4&-rSuX~8IaANN^&*NfltDkCqHyjS|>$JiTw!iF!>b`r}3kCSNhcdfPpHkP{7=w zVaALX<-ezagnWi&>?rccF{ZTpc}#ORI;J^-aru$yZ}jgG$NKGuSmEcQZ;Af>26G6p zCK6!OuKOjAkEB4x##%&1Fm{4|pF#ikt8WN1dk7Rg{GMW_h>4sX#%HXk`{l=11`CeN)*+<+jEun?38UL4>2p?Yp!u{vGf`@s}8JOO` z1Ur}kHKufy7-*(O2?`1-cW9F&6>`K@WmW$~=Y1SB>bShJV$nXtlPiDX`Ewhdy$PUV zDANvbl-yZ4pY*6^UyxI`w~WB9lvQAkHktKXPpw8^4QbgQKb8j^FfuZ#{IF+aif;N^ zl;wRc_Fv>g#>0k zUA4fBjbJkogn@K?*TvCBZkLhJ=D>*C$n~ctrKQ$S2r`pZ_XlyRkM!`*&4phGE>q@0 z!DV0K9@d>UXfYx@CARbR(h6t4-VTr3GbdTBN1$(4KPaeMusCx$*^Fong@K@aAzkr+LmX1#j($_GVqpX_E96TxXd$}HZixJMu-TVt8KT2UK1{Iz>Q{eC4x?*X(blPd%c%f#LVVmoFhYpIqcaONQZ4+ep=^?nZtSFf^O~ z`R*HPsd_msrm-KsZ1u8;UJ$3LEH*kH=`|VEZ`rBtiIQQ@GIwx)t{kKa#%$Ev3OkfI z?@h+lI(yP+)C|(t_-lWP6#{zVgI^4X?+Umay+jDgRJ>UFxbcfS>g5j+TSf$k1vhu% zB;^b``Rq&!?wl-@dv89>(`_JCErz79YFCT0&wRJ8DRbTY8G~SZm;$!TvO_=|bgups ztPeOjv}u4!>#3h0w-89 z7RQJ1t8!i(!OQgz?0RlRD>2Ct$o(VQJwp{_D@lF7>4*h)$Fg2QFnhW+Vq;b+Fe~Fh zU7?(rbX;ZG%xARPr~YvKl(tRL>K%+2MrFEa=e(ep^~1GZ!R_zy>{s6n6J%6sw4Zub zo*g-Ftuk^Yn23ax9b_P@8Nm<*KqAMs)O~LQ;sBR3gFvC4CmzA`SHE@UOoIbV5frhE z@YodZODvO!G`D$5DHhGjZ}@K)%6kQp;!}I$Mbc7W5(M2hYW3m{iLY%w>v$&LkN#e$ z5KB)RmWf%KAp$UEP}see0Mt9Fdru{TFRHRK`nO@sG(V7`gU=mMpN?Yc; zsF%(MtrAal;hmq`FFRY*-XZ0I8=hff%W*rwovlQH^D&v;coee_r32O*` zvbDA!4%GonN9v3g{&d+S*EnvilX0$cv0R>H>unCTv{jxX_A|4Nl&37k3YT9Wx9hYSQNUWzB4BLNdagEXy4+^h zRoQiK5)RYYb4@HZFa7aExfDg3f9!`xRxF4)%dG54!s+u7Z~(&joDWgDGZZIUDHY{D+JvR^AVA3P-bcC;~I5wH{9G39u$ED`N2X}$Q(zua@p z(nRl>*{0dut0W%P_^`3vHBbX&X|L?{cC96!)ht8;6Ty$)jHpR zGn<-G2mx)q>=b*s+n}yiqNHG*$yBsSaKHf3*Jqw3?!yFm%2ky{uLcm9M9fbVif2xp zi1}AtG)9w8rP0Z= zefyb0%ZMlt%7F#sJhg^AU4p!YZ^BO*HC1>9wER;`Ue~cAs56iL^K-P!>VhT69FtnI zby;k3cibmNSbDUL^xl$>PGj;hEe^>se*^W&X;N?-8zW!*%LA>rd@!qxU7KQ%EYvtK3?AM2{k^T;S|fDS z8qo))0*zEA?2a}l+_REckiCFva#~2!UL3Avu5>B9pSyIi7+@1Q6k1AMkzyfEL*!zA zx#kPIfM#1_%lOyC$j<=0k7$T0;Heq7L+0t`iNV`h%kO=L=5BzN<}1LzsBP`+qDi^# zbQe3GG2l0Q{&(#Q$InOZ90vCdGHiMCdy+=~ z#K%`yG<0-I#q?gI{YeL$7fEc`e0-h2@7G*8bAhb7)#g^tu(~9rPhX#HU3rVhysNI* z|H8|h>T$_uIhlhzR;WG!J^h)h{PT3)mD6mTyw-cOj79rTXTyIQ8!GJqihGjdQY%Uh z5q;f(RE@2JbLD#VK^HB+DOSkl`JTLh>rVM63tY|e`XuqQDq971Ckd_^HuNU$k3?+R zvUcI5-_e=V5CJ?JAx<&YIv)^7Yex0Ce?YV?eq_B5nN|D4j{jQ{BM=`!H+(D_2UB~0 z(Yn7jU5#py$lpS_Df{I`$P}mDfC8r}@Ka;Pqekhe^_pD5HP=n)BJOm96OSsZUfjbC z7O@A8Pi4~efKGp`oQYaia{U0V1k%Zi1L?Nu#JZj40fUWL3au&T(W1(~nHS%rSac-hM zaukwHN+(M^@yz3tQA>Bahh*Pvl!`B@bM2j_Z6fs|Oi(XYy21}e6rQ?lzvvCzovjJU z`#n#tsHj+L@w1u3Y=EIRgqA-0JuNF6zgamz`ZO}_zLr*;*N^$*78lA?G15O8WJDAy zZh7LC3q;InvB71bnvRa_6qA4->p>`84I83)x+uwIFXG62{WyWs<+7@$%&^J~Ogprv zQ)fP~Q*(UtbFEf zB_OLdeyHzEN!r2SxW`+F6d4>EszUJj%a<8p69_-V(Yob&!D6(t|1&x3K=J2ih>R!t zauE(7q2~JJF#k7qiP%IjeUMpggiAQrR18Wa2AUw%P{5IQAAfSxG&y~P zN_@m6UeZjnn19tS%fD=muWVK|%l~K~&AsshOmCP850eC?Y-dt^!B@H=bw@6R-!0=x zb3sZXc>c{qkxqJp*Xb~ia$_dS#nEWhXrmfA2k-(F9S2b>{5F?^=>HFIfrJd;+c%__ zHd%h!)(k`1Bg{X++(NV*BcZ&J{N^y)c|4I-!D)0Un|RY^&F^!m*AvfHT=sGDtJ{3m zw2LJMBxINzF=uMDS8W}bv>AU*GyfV-nqNf5$8-ADAIv)Q-#w~@dR@%GW2QW~ve z6wL6Tkg{+@O9MH}2UMm~+p^PA0*E9F^@-z`Qtd8^qjKFXm{T;HEC0!dz2ElP;o%lz zR$iC00BKDGdGThkliQ`PmQ-9+_D%ZwS!`(~D+wReGuG&0%Lf`&`;raw;}d^DkE5X@ zHeq5V8$|9vLenNhB)XytDx&`)%}_pxVyiooY%^IsuwvWR8~;*Z4HC*9@VC-X*9g%l zi;Y?(>2N$*D=;7ziC^mxUIe-s`LU**zSgR!;|r&DMYr^Q0y#YIb|8a}GyR6!j$^60 zj$@BQ!uB?d#tsi#XvVpqpn^u46L(?YA$2<4$&-zURF4t#F2~tTwwl!Ah{-w7Qn^2^ z56CBni%+kswmq>e+JD`sARHt#TA3+Guh9{GC3^YpkX7Fzh1lm@8|b-;sR*E**o4ar zvMaIK&C*-?%oWBbDv-zKLk@Q^^V2~!B3Gr+8haWv^hDX;j*})mp`L)7s@M+jE=9=0 zcCIGkCvy*s-X%8^o!7hPYpd}1!sDgsQ95tO#JIPF3@!)jM{&cq4)LW`ya^}5qu;t- z{CiaqUX*I!lDa2pLgIhvm391I0}?(1mr<(m6){f-EW0lQxx!BHz*Y?x#wg%qLUw)D zfUMzytb(8dF9v>U&=GJty5d2v`_g3*CG)fo5ESL3)Ki!a#Kdj>&q#wTW2>86No>66&YUkUU! zL(mml0DfZJ8N@4|VW(?~MNN{^)Tt!b6L;%x%?1GlDUogvBoEyn-JR0iaHPBM=BZcT z_>cSL-uvY;_Bdm+4>8wTbN*(}qo)c7o*U9QMa*hJkNcO3C>mo-qaec_TuVy1zc-px zI5!qp^-4NHu*3(8X@zWCN_Z^M<3(_V9bO0jWNJ!s^`M$b9}A0$X7{!X0vj#~yx>@q3(|BJ4X!jP!fR zf{7&;q&-RW!8(uS#hP$n!5B-WRpdAZ$|X>*_(kbU)Twuc_hVxe9H7#PnV*LL)mnAc z_BUin(7K?2xj?}9H(&r`wBKsBITi)qbAuiE+f7+$f%&v>XER_6jM3ucFv>fhJ~V8) zK(rTv)9M&Gy|ta};tnipJym6bmMyErsB#bQjK%cAP8tXs{eadxBjMhW1hk*DU`x2s z+v(v|%$pO)7pC}tB{L592xtE$_8$cfxidyFDTee>M@$c6pq#&4yDu<;}+37Z&=U%m93pKbMa&}2r< zD6Z>g#`^|ZYVfqaQFG+NuVYriLyDxJr zlcPK!MA|RGGTCKXMdFZN`Lr4}X$?+Mbpxh5M)eeQ7_`)t8J-m+wOj0 z#6OLOk+bqR&OK$=-EBoABF{~eHGZuuocjgJFndjBCi!=L6QQcDIcfJ3{Eu1^s^#*T ze5}_TXy67Y!oB z6Dp-v`F&FsRg8~ZCS$$CXF=H~J-s_P0~{Xhz{`zfIG#DBvJ@?~vxwaQe#Lo&Wi!jA z&tjL_N%V{w!5U}9rKRQLUlioaDlOp9Njwh!SYgkfD2g<(gGipzb%utcN`uqFeEq5ilxxGg1v{XuzjB%a8o6@5Ys5nCvpVXOc0! zY~1g;eC$*2gJUOfWlAv)pNzjvWJ~kE()#%QxjbuTq)ce6NL`jJbjN**eO!BfD?R(< zO=_}}yxgg-zx&SdrMku9c8!x0Ie&irtkyCFM|x*4L2~&mHvXf=WSTQmiE)M)i-Pk- z%#*e@K%u@tjnHjc)WTbVFtNY;#U3Yt4uSIzb`BRKn4T~Evwz0WVWwKwu0ypV9gmDz zA02yCC)Igd;`Mx=LII~5#m}v;^`ez7^aT)?Zf7Tdb0eQ@UxtSV(*z_elq17tw14A0 z$m3#)k3PeA68;o{K8I2dUW|DHTgH*!BEPe@($qsusBf7EuIZpE<1FLi<3=%PxL59~ zuzDi{LR+kLut`ZWH3p1PsRKY4?|GSExHD1YGS|AQf0?W81Y@!Vf9Pqs?s5)xb_;Sg zjR&jv_Rql$)tKHdrY|fDE+kV}*=rD2)mzFjVm&UtllKzD4BxW*l}sf8kZTd8ORj)7 zJV#_KI~Am!F7eV&0cC7!tgvTbaQpA<+x!*dY5IO&1X(oB(|B6j$Kv$g3B8~w3a^!2 z<>k}_d6tM?&2aQIgwY>|dp!V~F}c8)|D%MPFx6izXm~_lB%9~o6JJHp0LQ0GLyWY( zZ~$p6(ukaCMX|9q#=dP^3cDm;1t{%taaLi}PC-?Bf8u2;0x`)Oso5E$UjPSQ`l+w)lQb>df)8w{O zTyqao`qP*`$AfRka-1nz3Q*7YoHoWmk;YVVN%W=j{TMM8r9e11*CKZVfT%*aA;u@5}4@5|G}rS3$3b7OZ*`T3#p zMmY!47C0t`BGxc1U~rB2B3ZqZdNjiul1+DGoLM^}eOQ%z92 zj0kKBd#KrH9PJRowx#`03MxRKkNmOsi-J2Xa5eUJXDs+wKkj~)-AA%HhUR|A7?5Wj zsMTxF(T#X2Y}5G7W@rejvoi@Z4)`h^Q@un2l`D`q%+Pf!U3#Yk2hPCI5bqmqTcR3> zdkzVPkTU7T6W$m;G%VvoXu0ZB^9qK79@70K)&wS@Xx`SM;Qe0Hs{?Dvul&&0n6^hz z6(+_4@BA%;B|l?0I5bB~-L}}I;8q2~s&4!SNAl2A zFvjXLvwSErlkMc7_9h(UeQtpCe)gE&X;cKyj)8FVh&>kK2pvQm62t@~+%L0XhJRRW zoC~xqD{nN;wUg{Qu(bWxjK+%KGy525F(@8_+A+usMc55FoZ z{(F}zIE9FoBJ81^^YPnceNuyDnN#UxvnK;zfVts$*3 z@bY17Q87kw7-+g?6$1?h_5CBnA#00hiBZfG0v(SwK53giW(@d|z#YgPcfu;S&5GtP zi>E3V!$h}cdz9H2GUg^kXVLd%N^V4cN>ZBybwY!@B-3`aVZY)cLw8^o$9hIMS51gu zD2Y4z_LX4+BsStovLKD_q^BpoJ%X+*$jGEuC5E+s zbYnc$>3E}}Z<9Qqt@7s;WG_jzmGku^lC`u;-fZllG~Vz-x1DpaHRV3z$)Iy8|EZ6* z$YW2-%8Kc})1#*|DkHE93E0ycYGsUT?wI4~Gr11{fqiW*xPgBbdzgL?S7XwUO7&(7 zGGUF|rPSCnPP}h&%LMU^`-38=d-im?{J$9D6b_d&XZ)0VA$zf$F%XaxP9?}zAAD3UM_G?W0l45?`Ay=Vjxtjs~IwDRx~oRQ6YDg#u^ zxVy3dVeT!Be0iUYIC=JgaeWBr93Fu_3m>g9G6N8wJy_7V4TVAzhDBBNNuKU;hF|fq zvqw1F%t&!O7I9fuIBpvaS1Q`@*o+|n3+KoiuaKE_i=)0eJXdmc5R{VLtO$9rE_;@d7Fm~%3$w8g2>9s9v^bT z(KATnGA~pICJSlM-jd1zCHlG zB{rl~(E3^DlnVMQH~kP&m5(sv}cY#Zmmw+%r^#DVnZmLAuZjGu$~B zHNojhV?mXCjUOz!du^NCG(;mg9+ts1eet8@D6iz@RS&JF&M~_ixSs#)Yc$>5b0Xdv>mGgnGmIOJhP( z)$G~0ALQP08O$oqx#q(sG|k=;CM!JEyY*B~d+R+tc-A~uknnoNBvjkE zyPSS>EAmV<<9z_?n_ZzAJVf0LTJ&mQAq|Y%WOg%R#ON0_NHLoZCVe|(U)$_T&-kNd zEF`3ye>3ZPpTAu>?=vkcEJ)}3`vREesUkDHKQb6#deM%M|0u#aH#P?d0CnF3>TQ6 z5sdcMW}BlUnt8}Y+*$t3ftW!zfpR44<|_Zm}V7w(P=tdRw@s4?!zPZ7I3 z+I&w2L-bdktqb0WzQj_k@bxzfyq$#M0uO@i$u$@TwRDJ1^)SxV1T}+n)Tpn+l<574 z()+%S45>?@WJ0Uso?@y*u-)S8#S#7x*atR~#0)D0rg_PB4%pDik3O*6Rs2lTsBx_L zqTp01(*jl!a~`jF;|K=3OZljsu#Y zLlmG1YIe-2`DfnrnXH%zYlyoclkiLtrg3jfe_<%IYj;tVsB7=a6y35+cOvV~n>X81 zMiiod^iV*!Nm;;ZTGbDDQ!f|Sn#eXZK%(sDXAgvMus! zp3giIa6gY@_U^3IQLfS_1)GJ^w<=L&TJAJ*f?!w>iA|279TkosWCmjmd`o`rAW$N5ZfyxjQdPzz=P9w$k8d!1^-Gybs=>y4 z<<}H0_Y<{8n9DQ#M zO*|A*FJo=A8=twv(66bMyl5aUsr1rHGG)CmRcw65nq^mA zpm|DHu{AWU@fryo^b3#-z^o^{-Nsk*KeL_?_*<}#K-V>o9NqhfJ5ROjlUhPNSixW0 z?03MsVgq)mNhVkZmPy6Tf}LQt34GjXV9EY{x6gL2f=K_{NL@mcVs@P8u(t15W4J9< zt3JKb&yHt8r@Jq;cQ<<8S5D7q*0=An`6C?;_oLpMH(yGHrq<-ta+D@c3qMIR#$;;PH0zhSfHak3c=JzVXV$d*qc{+i%qF{H*xSF>~ey0;xH z`xAQ$){4^^Icd z(Ryo5*D@5FXzPc>8OG|Np?y}^o#X51chzg}BAk_Tlzl;<*ce-~Uti!X2lim()#k^^ z!3y!Tt%bf!x9vrw!oPc$gH2w$O7bNOxFw(zAN9`>@?ciN+JP1l7A`oGP>OK~2`r$% z0ro@0GzKbT*Je(FD!?7?w75qszgE`(1xedZQ&nVHWoFh7Nc zRxtjnik&fz&}x>s)xxS)csRP@=1}lUy%Wi+7;mjA_j2ztLP+)*U04Q_#t2~5c1uwvoI0m^r$%AlsApkMpG*>-`iq;nf{&{uL~^;2Kl*eE zRb5r72$%OA#aLF(+qW{=1*cJ%zP;KVYt3Wo-z6@J;sv{^i7qb9w5EcBWRE+Gg3YJ~ zpJGMcD2?0b<&q302>`({teVfjF4nIm6!D%HU5%$Iy5;fNIsE#qvhJ!5^7%K^}Ag~B{>EDOIdqxQNxLE9cV z#b!{irdVLDiT1|N9#IH3rS~W|<5VqF4bl{S$9ivtwJm*hD(|}GVfWlIy*kNqM~9k~ zx!p`AUv)sEn-Rn);E}rkioYfu{pA+jI|ejEpB3E8cJ-rWLZ9}f-hCs3dO0ovO}P+e z$OU8v@^~Bv@*X13WY>@5F8++^=buMHpB-%Lb%(pWN*gw{La(dy^gd=Fuw71Rl zSY7HxLQUs$5q%w(s~;6bcv|0GWvyX6QMP-7^fWWbJUZN0n>!)dy6$6|hW(2U%coU? z6&4nG1#r9~6BhG1D%Ie0>u$^DJ2$l!N8y{Pj7#7o!W0-PI?B9mq_D!vMzmUi0-; zOFkqbW4DZm*IXMqlQaCuNvpxc<_3MdO4sw02AXjuid?byrA$oflMhl98VqxF)JwJ8 zuT&JkFH2KogZ0ew|9ay49$$OjZ;)cssmogX?Gp77u9@2A>!%^4Fmxi;?>_X_JnZ}^ zu0$!q?LfX-UcHs%MqQJN5M!k<=FawNT)k9Yqbx_e%&~C40ys)zVyZ0GD{BH-6=>6(Vu%sCLuNTV8 zMdPTAib;dK17T+oKlgYatQo(hBP`{erC#LaPKwKFW*Y$)k#+%jKjYcT%>L3R43&6o zbmm`)=&CQavz^lOkb*8m68h7BkebKvosQ1G$^d|sQyjv>Stb=ZKVxnDZvz}QbU9-;eWH{qk9AvQA|QZ`Z?L!b_&vBXQ*{R_&xi{KsM$Zdl3WuHyh)sy3MlC6E7mzO5jq;a14Vrt(w8Hb0Ii^675@_(=B@R>& zKEQL-li*CF{2aUULF}k?hlKT7n4AT7KiWZW>Pk>QvA25i4228PK$q!AYtzV#K8hxI zr19byXRwpJbZj4Y`%!?gNV^(rujl#G0;DKXmLJC@#%T_ zo+scI#qPj=&p-pcmUKD1%oeexfF1T67mZ-O+N{M6C`WEVnFi`oIm9 z!?>zhWONsC2Z7yMmC536eN@3nZ2*IWM(7K|+{dc7YW!!Jt%{*dh; z?9HGgO$pm-`_;V?CZB4Z%|OeKHyUAvMRTcGU5>}x_`J!odMGnFOe>Z2=rNuJv$sRR zg)eRbSB@xH;*v1x;cE(TV9CJg1=X$wAuEgOW{jJ-o335v+MgE<2HJ-Z!jQtMCem() zYK8(7@&87>tftqf&!1bg= zPi?405-1p4u;^cFdbNI&3Q$6*a2EY}QVlLzo*0w+JNcfEy)R*l?2=m)BK}%bQoZ%^ z^3CLPW5TLe=Ed5dda4Pul1}O{5#cCB;um=UmO^Vde(;gVT{}$bSNoZ0q{ZsGE>eB+ zBi{GtpGqO>W4Ha;spmwHIm%mgVZ9n2Xne?iQoi z{AcgwtR=(l&dYH|`9q6(zScKbMQ+`#oPtDg;Y&WPIxznVO*=`qOp{J`Q_NM@j=3ne zWqPD!!{`*Vd%F1#4HP~;_}Z|*1F&Z2SCEsWz=xTqxg!rOx)tCwz*uNab=gJK(Bd1K zxEHQ^k1ICV^BGE;I%1fKS~w-4fse+s7Bu}M8sTJod?d@^Qw~&j5}P=igk}U!=~t*Q zjy9BtG%7tWdHcO2F!oy=ZY`8w z2JPGsE!Xr0tUBe*)(R27O`rfjz_;m!*zTC3{C;8AKUjC{$-Adv@*f{QQ`&C#7mP!$ zil)3>y3JSk;`H;~J?;0Tr#c|wF7fHb*ySg2$&LGow%qcX32u@;wO*NAQBw32)=||9 zinhJW8dezaj`#D!AMhc$Y8TNvG>*?O50aG92jXRX5xO(8)|K+}K5aNATQ4{P`A(y3 znDpZ`!(m9Sk8%nND{s%b(JSogPJ}QN zSRgu?B0XUUba{H{(riSfcmQ9%_~t-D?7{E+^%_?23Ge)tH`=j%UlMSUNPKM_LpgLK z;`Z~?I}!61!Oqb3SEeJT)`{@F)z&Zk*7S4wq7^CewI+AFuQ59of-cgWS+(wHh8xX~ zBh%ZY1~C-zr{VB@Gm#hLy{| zP#rNIxBl^}pacD4(cl8mR8fgm@ly$A$+t49uX~y;;1luei9&Z$EwBOM&kDU9um1L%11TdhMFr21AaRaFX;VB!?K7`tu#=_Kmj(Fk)(Gp@EEV0$n9M>;{8by_#p6_4!7@(UnPL01@U zI7>X4;x?x5?isnbT~BB$H{%8hv{|Z8E?ssge26IrRT)kP{s5VP_Q7gn-@Q%%E$=<5V#~{lUQ*H6JAjr~*+VUGTkH zibJ&as5MF(J;bl==FZZ-DNyAmOc>{w1%x z=s`Dr-WMjcUOpj)2xAiY7f)b;KbhHBF74S=%}LQ z;PhvN{E_R6qk^*quLaHOclymeAxesmtiGgA&Qk#vQQMPLrpOit zMM|?zQNvchDdiOxm4@?cR47ouyHI&f%*k=jzA}lRNjO3ax&BSh{UUO2 z=!U9gSJr5M(8pJO|3fxX<7QglV@y(K8kXonKF*mts!?xCF>dV^|VBkrw)$RL*;~LYp`f*gl)GA%C zBV%ZgDOpuR9LKO`!JLI^BlDFOr>VEzgpQvJiB-M-L`RJ^*d#k1)vW}nw4PDdu6CH2 zWH&6>LA2h(7%u^xjQ|`FBLXlVaO?d$V*LqVFOr+2U85&w`5gdAD9?Pcu2LA^_B+cv zxU;g_qJ?^pXt>)#+29yvrlT3=UB~O2xQFOO>c_o#G|e51;gHfL@vfr`z2fJg-Xq&% zHHJD>#h7Y5|wkPaQ z!CNMJ&+z$Gu-;3j`?t7`-*wu*RGiq>qw2kss{*i9YD!?&-w?+ZK2O^;+Ng z(s&p=@K|ZpDbLI9Y*z+j!G+yTpx6X4-y{5l>5`E6dn*M~-WX6YDd+koYr%O&)2njR z(CfVUg^Tz$L`QzT1 z*N&RVj;2$ENrE@>BRU!j(S% zh@tyBH-M&@_GvivBsIDZ>RZo#;^^Kyi&X8rLWiks1ucyQ&!bRohxId6kMFu63Uz3c zw?KHdQDvuiZ`?-b7<2XhR;=jV+Ij8qH1<6UN>C&Y3MNdOGyZp22DyO+w(*8%P{6Fw zuiyq_a$?$}v0kU7o3v85rs4+}t`tH)2r$h;vz@T1NK%-9)DKLLkH-zU^m4Ys!-6V{ zHk=xmq#l}e*MQWFC#a5Qy(*xOjJ*+&tC#6)&MczUfI8_Fv1Go^c&^%bd&4ABBTjtO z^7wm61AXq%b5_#v-vYM(fYxUXSX6wb9388z5^qfC z-bQMoh42NC+4fn-xiZCj+kDVI#ci9=vdcz7Xyp@j9DHK2n^6QhbgXRR5^(vtm8g0195p^nf5#-fV#fp}>5acS_KHTsm3 zX+ElVXS+EZB4)X+#|bwS)bnLLvu1Gl`mL9ZZN)$7V`BkGY-hc}0iNL>3TU4SL?TuO zl>cRhb>aGgR)V@TsdLE5)m|`UMtZh(*1uZKxTt1Z(=xcfvee{GXogWhS-o+mvK^XT zt3{XmuC&7SV0oHLJC1L6Q&daKrXt`+ox+~tMdnr%d?LUHZgRhm@5~b3+Zm0zpVf)n zPCa7+KYx}v3due>eqAE&Lh z=~WwK25k_}F$4=~FhIME1V$m2YqimTi$eT^s=iiO)n}q-8kUe!z$E$htjlnmhCoNt zWwv64yb+vKu;ZdJC~^bgUK&MK9kd}<-yL`)9}Bnp*Fj-{7xwNyXS1 z>J@t7X@LLr<-0j&9U=D7at$S@Pdz3Cm9axU%1$ZdGWR9+Qx{m3wRy1$h5rDSKkWz# zH&{Ah4p)!~EaEsfGJs26vkKdqFwUHutLuX^^94`CgEQSPYz@zUjw*)01j;VHid1plbkAMJwV<>wa^HC*>xuEpwEUiDd~Q`=|pKk7>r zUASDl!7N~|{Hax^7~ASplOv}q)RI@IVUb+k&#l_q!N^LG`VH`L3tm@m)w`9uVfZmN zE8w}e5tdJ$^8`Cb2=+wpxw_kqTJZm}hI*nNq>9U0lmrCco1Z6hUllZl^5A(qVY6r4 z=TOG=w1=iFpvfbPVczq@M11tV^%z*!<%c36N+wA#5iEDVcYrbhGQw|K#?`iJ5O-Xc#^L*>Ns;%2s}pj{3qx<84) zXKA(W<~n(M-Zwp(b^#quoom%f;MWXft5Lo(f!iW}it(sIW3o5J(04(7}X!W0<9Hu(mdO)t}F;`s0m4CkHabLa`!L zMQJoGuZKua$C`P-zHA6!W&8?M=Bafi}X#2c9DYLLE-*pyHl9K;$L@=xp&(_eOFCeGH;13BCmT+(MhDB| z`+@4;^1i>oA|)~ezR`rz>&R}!|4i?wTg*K8MVjIY_15#s=O2{53~>2fY+d)f&CYqT z_DoEnI!anJ5ASc7{araKBEj~fz7Fhp>IMrC(a|gfSb%#eMsg3$60gkvb<)2cR8y*$ zhvdnJgv&Z6f--z@o{tGk1d-kk7JNqFwowm0wm4`+1s(s07>>bm%d~zyy;vxhbZd}6 zp$ly>ePSl%;tSJB!b>7bEatu9D=HvgX`P;fU#0u{F(6Gymvw6gt>EF9#I-QuXmJiQ z6z!k$9GqcWdD?&Zv0w1TLfuhc6>w<$vru7nMO=4qdHnU(f2R?oH;oBtKWn@!-8S7~ zQ`(IEnD&^itCugxjgkvz8UA2!h{{Z;oPE_a$v+jrdTGPw9U#9tl3n+4FdvE7=p%W+ z4p-nBi_f{l38vU?KSMm)$ z>vtx26^-iiVu8VQF3)}%R_?KPnKLdfQf(pzLaKF;gy9dIRCGIHa3SPfEdl6wzx365SFCb2E>op5Y@&^Hw*$Gt?pBz1qd{csL_?qNWM z(h?uxGJn%8hvAj=&$dcXzN&->0+< zT&y?Em$H~A>;0V2SDd{y4=jI8nB7jF3nfgT{?o^}kAM8arLrcs0YiM z)a-E^vi2Elj|GwkTms4tT7{1p^x1@Xl8^|-VJR)rs?jx%ZiMqBQKbSaW~u_V^J7Ow zF8pa!3j~nQA8^6d64+}>lUO0m!Fn0&9mOrfSC!mOE{V6ves%*&{S>Uwd`0}6eOid1 zZlA&(l6c}E%f|%0Gl;dwmzd?Wl9m-#N>`#sUWAuO)38(unsjDdTnHAZHKA_g7;~7y zFeR9AF^zuqG{=nP0_8AV*aC3NlgqIESwkB`(9fxBN=4-z$WWpx-Iu)h%NHF|56eDR z;GZh^RiAU|8D2 zV8-v2`$?p*_EZx9un!f-F~@m9u?Nt)>=#P(*bze^FFxy!;MEO|Ndl{ znELEd%v1oO^s$4c2Z^%83_sk?O~<7Dv#{VwPKu&~Ryr=ehFMe|^Ax>{<3Xe`Zj2Q! zJ{mV+iL(FG>T8gWpRK+Lj7-oE#aT@fb6T+5l^4Z_8LWe|O9I;7qk^A)#L3cfjd75R zTZI4yL6Aer=7J2!5MLp6noEEva-tDSPPu`@jK*6@?~E^A|I(*3CUW^GeZ7!x-OG=$ z&&`zsqdIM5lyGx8_0vjYO^a)sZT&KvNzJnrYMXkVXY)b}=SFfY#6ew|1lotfwi*j* zcB>P#EcUwF*g%bN%e;@OkGC(+`&nmZ*xL<^Hg>4}Cuu&v#{}7XM?0yWgmnG%-OzYU z3xrr=-y9eGD6Oj>WT3&k34e|dPM%iU@(Am!<*@L0glnF`3qI%OA3k`EL;}BYM$s@% z2O&k0mj4olEYl>G&N1Dzl)ZRLr%bN+0Z(76^@9yUn$SYKt-rlQES8& z{APr9d^T?Em|LPH$%Tp+E|v3s;4HTO;4^Li^Wb`u$AMRr7qsoTcA7E zZX^lCO&E2U#-1JlQ~~q>x^GCOD74^H#GG6^eIUi~ZS}?98SA-1t6q@9STz2vJH>Ee zp{@Me3o@RsCIu-`*hO@U@K7U!SYxMTti6e!ym-FRT!Y?L>+ojoItz5U57VO#bGpyA`V)58ywgDQox^5jLgFUoY8W`av{l?ttA-AnG~ea_ny z(wTFQcwPdvGaU$Xp=r*ANGvTOm8~hAFu*AiqC*>^)PxCR{*A#r1o(wYAw2fc1b8x#;IDI z0H>b3%^vRJEXslQ=T8Ru_ypIUBM`MHo@%d^@nBSL&#xJeI|K-z-@P!62mK#DI}_&; z@vX!^ro{x_4vM#?pQU@Q0+@k@!)}N~ zjhP3#>DXB0Q|p33##Em%eY0e)$#Y|HUNM=1&VR?#q5q6o9tg;N`-Ja?0HrLB zlm6ug%U? zo$PLHM)ZP1-cKo}$dpVcEzRIQc~IgZwnpG5CWl)~cg%q6+#Im?Oc?*G*MoppodGe1 zR=^oSPXU(vO6CgwXMAlL2~`sg(AUY4{ZP@Sd{I5y12N(O_I2?+y+aj0WU{MyuSx0-|DKa z;l-u1%@?;P672b4ytnb6%)&`C&`7#PQBpg_zAJQR3~_+*>M79GR$PpfY-cM)b0|bi zuCjv;AjBq8rkrW3Phz{{+Y^OfXqM1X4DcSaTl2UI!#8pzyR`-xEKFtiqokojrcvcmD3th`Hxi6YTOYoNTRE-{J0T zHj9Q{E8^s+<7YRvj758u%RcUey#mjf)=#ermXoV}&0{{nbJ<2utYsXlh}VSOxSX-f z)UGe!G0|Ph231--qM`aU$kx=&eAe;^N z1813_jhS-30ITCAVb)7iB9YT>wM2AZZ@cGadhl;(i`fqW1WlafIl{S}o$yKbsVXEP z)M>)y@Z}wZts^)rqX^{vaJaB1^VTP#W~e!w)cG%A^+n~-Nxf`{9!>ul_%RA?TFJq> zGnwTQZwKf3vfBP!L072xX(f3W6tt%;|J__)p!@^ltlQ&nA_I=ku22`TT+&UMMtI)W zinW*W-QZ;vevhzc!yU`kKYTHgt$u<+L}v#*d9=81iz0HNV%B^1y}zoVk$7(Ufr@d9cm$%wa051NX;5-_L7L0 zOSf+ou{jaJWk(F1DCV;vqpT`H+}DWctlVea=fcW8^}>iI%rK25aiz+OIhXvr-OoNno{&mdK@TpP9Cso$|_X z(mfZT!G?Lnhjod(%zci!=@6)2+Cx3`og=Y8^-nR6qK9e7tAaaz*haiZBo1)he%Ndn zq8l3iJSyMO%z(8JSj=yOApNd!y=SE22`tVt{1mh%zPO&iJa^%nJI_^;7 zZz<0jz8l^??V2Py9pIT1ffYuwMkB-n4k*Ig2{In)B=;p+1i1mO>)u%RDDib40s6MBrxF^NBo zcc}SJ{}ye5c)Dl4GwW@PLj%YU;MW>nwEqqsNTyQCK3B@eNGqB`(T*?k`xjYVZG zFG|O>JhQ)>KpuYCS!deQ%_m!1okXNeWi9(&uyK*~Js*o3Z^%pdYXFIweSX%h$k*rF z5I};h{b1E~whKFJu7|P-8Szwk zdX(OnD^$o;GtZuNET|bxUtry-u9-5@5vlo_iAvO4oNeEAF5s*gxv*S4h%mk6Bk|jp z8gewtt8XN#1j;MY5!3Y25Qgf8s)n>Tt-6>n^m&Kue{iZLb@Qnae+9Mcp}NhJz+eU| z)~kS>4IIK%DHdUhDulH;OytR`&^ct%&Tzv!=gtxj-ca|F5bY3$P#mR3z7vD#`60(V^XY-`BpH0gNAKd+?>tve_F*>nutYcKI#?e-#lN2`I7kbGg_1W4U9Tc-T-3p} z1?7beIRcx8n<{~NCM%C;su9|R)ZU-B?^|zj2I73<@_y^&r(X1Q9A+RE6> z5rrHXJdG{*C{uan5U5A=-in!z>!XRpwE?kF=;9FJa;(1hYgCT;XJvTh-*&nx-wD0) zL8B977^8i)JAiVHint+OBsL@P40srO4gF3opt~z??%O`s@fnJ-&Kh+xfE;kZW7tRCSF{r2ZE-JhGBPRM_u;P#0 zdrJlZD1|7eh`+tgsG683Y7U+wUO8{uU~f1@)j@*Z4icA@p9n9h?6z}cbynP=Ddj}D zXgHLikn|6q)%Oc^(_qL+Iyc2_G8+yGN7mGX91JcWkDl|A+^z4x90(GL&Hx({8fEvOj=tK`Xj~_DJ+p!ObmEL!lAsMm zrzp+NfQYn8_c}3rWuMsWeK$mG#&ci>R;6UKeRW1uLtmR=qU@_nH3R76Db~RAEpZYo zQ$J-*@a~YMmLCvaeaDA?m2@oHguPzjQL$5%tVZ1;Fms@A)t2~n#iaqc$r&*ah^&Ep z)Ur}joLhV|sMG@BaD?Q@HLFA!WYrXZxG5oV!-t;gBg^+Y zI#_hC(CKVGis{C*xw2oR_>;!`U%b6#T$J7RK2AwWNlAAo-Q5a~l7fV^ba%ttG)PMg zJs^l6CEXz)UD7Gt4Bhz~eIB24&i9=E+rKaP3@--nz4o=&TKihpTDL3k5d*z!&9jpa zZT*1Xmcs1%fUJ$@LcYv0r=f`+Z}M^YyH657=WN&7MoVMkk*5`dg#_xTzo)jROloI_ zt`186G@riCqi(FmZR_fb>m7=Wsoju@yUkQjdswP8RrlbXq*s~g)GS`9Ex0q7sHJl6 zAgM&*;cV8BqIo$(>61JkK%Y61@z&GkX#BUN;JX#kSBD3JE>f zS5XQh{e~6%T9oA=2qfiwTGo5|75Ctb9?~hE1;=5175OlCp0W}WS+k=<^xe?1Zw@>J z-*dhWRiF83*{^%@oRB@!IF};G{))u`6fF$ljs8TV#5}}dR{H3OX#V(wvcp3upgN?{ zh6QQs*uC_Hrk3EWyFfKoR$hrmX*)Sy_l^5WcxAg==b3!iZfOvzluhW;v$II!I@v6m zU#6elp^3F!4bV+B>)iRolFNO<_KagvdRB^(_RJTn+ZGf>5^q0)#i5(Ea=HQ#h1o!Z z+yXTMh4!X?^3w8tyAe5?l{G3F>q={gT5nwC@34i?``uK#%C8F9MbwtkfJ3x8eO2a$Xc&t5)xd9|U3nC? z|GR{gwTga2z`xG$klFCYhJ|i(^!wTYfAr`2w)9ZeWxx2_eNkk+YqMo0i{Br1NK<$h6_e;9f)O43&LdWa5IVKq&p_ zDp^HdR3naqkFsBN@J%F^voXzj+D@wBNZO%l@5(^scbib7P}ssaicobEtTv6(NV)-c z#_}$%-4VIt7Bwoc|6L^1JMJ>&5dW0oMg2m&jT*1tiU&Y2(}qjeR+uG9b9Snz5-a`? z?dlBMsd5E-_OtB8XGcZuwaf{th1wnP{;V4WG1a7C zd@9*PzxLp@NjZ;_ID9BTASg{B-x+f`;OE8d=vVghWPv`^z_>%P{PwJ8X(MvEjevvz z-c6R;Zk`GwF{6uuH+a9xhzXwS1c_B=tW`V71NG7JOBc2lL}3x2_H5k=j1O#(t;R^# zm5dWCn)9|a$h|7M37EwDIUSqgp2YJBRNVIaqRE^p`K?>?PFCo)7K*3h*M?WhUX95& zqxufC%1TzwJR;pw`}R@_6~;IlPNepGOkjTmj;VU678f|a?3lje`!>xi2F|LGNH_i= z095!6buXp4fE@?@bFam>1n?JT@Jhc+cS@KT3E5r9loU#-9oDJ8roI& z_=(bgG3~#p>X!knFY_6pT#KuXkSZw}=E(h{Ffjz9bcf3dd5SLLZr3A|VQV9kAS{;Fl{Z2CLV zo~uU{tf5IWm(}J0#ZYKxU0X&dXoF~6rTSZGDQtz+wvy)&xFnT|0O3(2NkF8YA0K^pG0OtjDs-^bz@@wi;f`TI^>)5#}K zuXRwa*zqA!%}Q4*ONtIEESIb-SqJn7QD3*)+Tuwmsbz)ibyIY0&_}zvDx-hy5tnzh zf;UU z)}IM>a@Wo6W1fppL`YqgY!hwap9!>iP&prd@9U)F>bd!ti3~ayolSO0OExDIeK=@x zHR<Y$JJcbj?~Bczn-RAwgHy(6|ActB0Vp+wkjHWPfs?&0v^lkf?}Nj5BP%uLp# zV{*lTd%Gbl1*hiLQ@(LFQyzi&(A_~^D4MkE{rI8Vw*~juq zoKiklOhvwyqE1eOmX=QLNh?+M#F60haF^iXpp{vq6J|}ab6P0N!Mp9VSJsWSeLZd5 zU7Q9P@;mi5YWw$2WcGPRjS@#anNXTcdGHw`U9+ECsK?+MPk9>dK=0aW*frU8zcagh zva{gUjlxf(A^2g-So=HBLxKHQp-SgE=5-nF4cRpi=dYC)#QczH4*$tvlUFYG7ecXPK z(yZjwIL68#i?%TBsy&;dw8r;S;kdeea2t=rFp!v?IsmupCzT4v3W~sOo6#5iaPg zn*Zgm5e<-TkhPhR5)}*^=RM$EYNA2s0e0Pp8_zp3FtLMvI$k(FYQglH$Jf}L-}XZq z=#V54iUS1V1@|>#tbCCCB8YU=Eindy(*-cm0%Nj(qccdOTc2c$?}?K!1t_q za-AK;DchU3o6XSklDmj>jB+{NXYZI=x+=*9%(U=z_7{5c=JTE=d>ayx_%KGjtoBin zmJBL!Rpw?tZJynv^L60L4WwO{r%qC|_Vhm_FTZ_v*5)rY&It#g@*rHnFaAtYDqYBF zM&JyQly>78?P2`nt3!JO z4~BSrTee3_?-Bt>-R$Zb0VCVM;iFGf!AJb1-`=+8w8GDmjlQu3XF~@4u5B{*P~RIG zUdw%@_Txhj3l=`iWi?0e8Sb+pbn&eXzk}T4KGg(s^^w~_nGq|3>l2nRLGmK3Ly^Nr zE*K}oCm4K`+i2X4k2NeO6W(!Yr2L0p{;U60;+H%TS?(Lrb3?JWeVS91!55;)d>pk0 zC1Yi=&;i*F^5Kct<1LJzt^**;^LK~=JjRWpDM_6fHD*W7!KTlj-7VzuUJCIhM+RaH zyWugbW|ZzO1@tj&vUXdVA)3jfG%tr)1p7)%pFCWxspXnHqYi)xo`DmhJb}bWP&|{n zTFAngt0>)WK!kw=C^QW$a7)Dx#1F0#@cU-p4vZ@6E1M;ZdeBd1WG7~k(D2@UO&j&N zHL3T_O7jsDE(0F8h@5b4QskpL%}Sx9Bl=?MZAZIMFlRvNcT^hS{uoO!M^XI4exM5H zCi14?rsJj{%PX^z2kSF$ovgB51G z7&_cAPfAv%>>4I)xJ0gS`JV4Rw32b7M1 zG*89gTD-A`N9BF^-8a|5bi&e|-+`zdN~1x&iRpP}T@z!8E>TLo>pXT{2T(SY~`40v@M5DkCwG2;r9r$LCfZe-l31xKT?LO z;2AECE9%U3$aCRHK>rc^AcVwfGn7mj1}bS~>-qcWZxy`oQOuJP!+!@u|nn>>%nO7}LJBFnVY zRclMwx04#)D4R9t`VUb3t~oL)_z@7m8D$qFGu4kZ*dK|i$z?c05ZXm+EN`vv;Ufzb zlu znC?&5!sGa%NIh1ECRRO(=nmO#885yOH6Y{T#5H-^Berx#f8zlIlU@-me_Uq9Gw}T0 zC!zQ za&CmZPjx~({RX_?*I&QE6QICFlY+2geL0sT!KM!NXqqCvB3@or(hQs2!_7L+(%lgF zxWl$}v(1B37w^(iF$Kp6@#P0js7)hMZ$0TRFqHpv8~-=a>B|%Z`=cdn^?Q~zx%|C< z;L(R3*=3f8y+UULgr?4PNdsxjiw9U8?DCc3uEJOIw>X! zTK$hX=m3>16gi|;+e92`7zw;-!50bpWebLtKRSbq9~J>Dt18=;Mro1LukEn`Yv+65 zXJWJPi8vo=Z~0-OXbz2%0wsC&M;m5lH0NoGXE=ei2(Gk*6G8x}*g zTgzz3{%&WmFCztsg_05mde?9r9y3Jw^hu6xCwb{7Gm(Pop;j!o_-iDwQ%BSz#{&wnf?}aBw$8(zL!#&&*46mh4>Hby(0o>?C z(R}98-a#+=JTsfJ#rZ$%C!QHo}eYKT#GcI+`l(bSvp& zNL5q}(gd8O&DB`CFG5!n@s69ATk9w~XF+Z{!h_2*mM>oDlX#mL9!a*+b112d{JQSD z*t$>fZ~dZNie`N*I0)BMJyC8hgnREA9`u^77Yw(oIXQI%o2E_hGum86Kh?2ykRo2m z@SlML?U%#y$3r0lTFqHb=qk~dH+mG|S?PddPS)0N@$kL$wd_s9mODJNQH@$K-4AIF z36V~}+1RNxmu{z#Y{0@NpaD}K5X@?!_hI%4ozg(r$`mSEvX<(r^Qz63yPtKEw z_@0nV3~Ad;VNijyjVLcf_?3jNr}e`YGrAkowqCJVe0#uAr3^j5Mdqk~7<}<WL$W`Gks)rm|i$!nWSy(30wwX`xvxO5-9*>_xoOlSh7^hfGoj3m;eZRdy)N6&ic3jp4N;x8$TN z(T^(PU|Y%D7gwo|KKV5CzIEWHsr*a$7^aYg`1+x17!W$2Ko8-^JLl#jJG!##-QUN` z)?IMMZ%RM357x$$M9b*h=fKJ`PWO+#W&S!8oWiE1p(R%wY6M|S%2TogY~<-iXWyAn zBwD}Scsb^koYyJdrEtYiqdFVQ*XMKG_o$EI`nl&X;iFW~qvW7e=Ku?ijt1kLDPXa` za;%z`z^(c=kZG;$N;m?$VX+OPgdFD8-+l((EX5-+s1Aq@{TVdH!C}Ma@6kobywyIH z=a7odeLFk+eV7C9*jyO!83|fDI@h|xc)kS2xe=xnC9fyIG+t=iG7mcy7EnM6Rd(NP zLFR^i9wr(Fs|bb_U4)`)ADxPx)R7%k$c4R#Z!oCaJD|ruAn3R{iecoC$-snd7e3zi?dI>*sR!1+58!Ml!ug_)r-HvAYt4MANb)NuRic<#oWJqNhXfj~4 z86i}h4QYz@8~TJ&D?H)W4 z>K3}^zb*@ZS38hhjprLkth@NmkvpNeROVWSTt-ZuGf*HfojB2UWHKRttzyrXrrKT% z_fI7(77Y9!{RcBXxJ~G*?=R4^92p?JreFpmaxUL`s;}9u#N7quKn)xOMz=p z=#|1F!&Mt(VU-bIZOM3ir{1Ui@{~DB2M#<(I$)P(D)m`^A4+15`hexX@d{61_os1{VQPW>(W{9*~f=Of^`6soa-zpYq>nkDjA z`f_=q{h*%Wip;6{dtD2PW2sb;Ju(aAJ89|Q&B$ZUV%651*a}56C_%@dBu!2AEl$V8 z(44I56^Hb18nbSor>R6ImBg5o>GClX>4#hBoCOS z#m*l(x3H7wDEl?C_1w$>EA54ix<;T=?>O6czTVQhV@J!P*h?ESQ8Z*-qNU_U#JU;* z3u>{V1|0brsKV?xv+%NkV{gor=dMlZ+27oBD~QHod30;1W+;t`SQ`75LU-v#>4tS| zipUPc(+qVrm1X%VwiP)b-IUFwpV*Jz>DaNKz6-1q ze4hw8&WpXg>#l61Jr>wowKS?*f6BC7x16Y%?6v9~?U|V3qLnOwl9+<>%A6E@X`1n= z0{d^5o==0+hzy{+pt=ZkaO#0b4W3M3FX!I&z|(!KeDz}Fr_HP`#4dD|q8;z6gp!!f zSx$jzl)-D6bT&zHkw-HaZ%Ln4!oA&uH2uX|wjNitW(YV2(6!+$9$7}Hcfm4|r zp=M8*DCP_q-MvrbhUmT40<{RivTK;~I0InGmXj_K7-y2x_*d=aWuVj}0w^G(<{2M& zeJw88ckLY{8S?cyTlWl=uJpj+0>enuBgmXK|Cbpl;b#208-p1YLJY4({OqrMOooxN zKQ&csesd?@jk9##YKb?>=_gcDYZzy}Wcr{3H}FC)Q&6EyuSIr9>L$}o5O5(+C=Fgq zElkhf2-nm%+Ws&ky#B^)#K>p{Pu=VhTZZYy8pJhePyjCf_*ZeLRXU6J>=Nxki%&1+ zD?>&?K_V}b-qS9Mj}Tso`8Kfk1b(~=IJzjJS|{4}DVYM%pO!>Yy4a2mX8WSl-5+`F zc%A#8A)sMdqU~_2gzJ1s5&Z(n8cVR#_#;@*#q>w`w~+|hWUt(BN5twO5tNPoX3k;C zS{3OMcwJqO;Zlj`G+n*L!&+hXn2U-^3Ze+9i1S#Ur1nY_Shol;?5Yq8YiXY^DW@rz zrl#K*x9bAFaGJwsFzg9e>l!_I5M(K<@&okMB?(x97))r3%x2y4Ft;{5{^MhQ6%;X1 zqU9MU;e(Z&=`}o1ZakSYj&bjEk0Z1`sn7M5DHiy>&igx$Rq;phFc*Tpa3qJSWTn8$ z>HDlgGql%R&HFsAgrnkjeCkg$H)%GXh7laz!)r~CHMik|uIA42zQsiVWO~_sS?*ci z@miQ$9>9=n(Jq@{%OV{=P-SAIoN4mb=mtQ;;NjHRS&rweGk7GfR3Tu0i_t9B>}EWu zTTeLA7homRSbK>uj|x|(l8^JHT2DqW+Z-3NHY_ZJSL@0MZ#;s}RZVT##o;^ThX zLFM^uvZmWB>JO82DRvgspqxH!r2T1WW-@z~q4YLL9TJ=mSk>>>oudlGWI-#I+26O{p9D18Mo0++$}Gy~=qf8?_&XU;d1C z0ayr-6~_w3{V)2 z0~awmwhov4q1&bMFb?=1+ynx5Z!KnH`C$)RK4()6bK~7l@&0BQ-`T+-{(ZHe{BIYZ zHHP7Ebc4EyV)2x4g1F5Qr_xFaNn-xU*DprY>nRddK${Pg97VD@uK$%Hp8ku7KAr&) zrRVEL+9EsBZfH_D*-!AGFSZ}}V%9<3WK{G!C#4uO%=L+m0FdtHo#F~bU+-lcUtiYE zh~XBm0wEjHHo%4~y39>^ zUUnt8<;;e)@oI#veI3Hr$Jdp1#Nqr)zI%+Las0QZ%<4yvl=iL;PzpT@ISaICk>brB zr{Sm_28Y@OaRi&ebpS2^Sl^UTq=;fx-6G`aANt^B27$OLS>I_T70K#Yh;@)BM+YX!Sdv4tmU2Bcij!~CPJ}gf zj~U@}la5}cnrd_vZofhL#rYy-6k0JU)dm&q+hv4ZrgTAqE{7GzrZt-^1L z+FJSsIqvrOQ*q4KTYUISV~(+&mf{`451NdF8tETbfrFBog&!lGuCPG;C1m+AG(d<@ zWPq8wj})D9hXBgzf&2^{+eb_92!daZgjKbU64ltVC1r!PY>XP1W#?<-rC9;D`#jHR zA~N?8ww!fMB{r;i_otjja65t?=a>I=pjf<-ZkA3hO&5AcEEZkVkAI>Rrn-PJG}zrV z;EyxKm7wx>WP0k?8g8-Dj%WYkl7J5_-1?k;DOiOwthCsPoO!>Hrn#CeoIl_=T4l^d zwf9AvASekm)&;P}XX$1w*Os`mqLMu!Uu%s2$lGJ=vRoC)^Pa?<(kq;JjM#`T}Zh1U06#WH=!3w zDlX|Yx!az9Y6?OOYyp6u+gAN8d`Mzeu6C!TCK=nMSE(y)^-$kMDtB|WAgF;d?9q!<3mWwq;z4rBCL*yx4KDOokg%M`?s@WnB}8t(^yA(222$0f}^v{lfg8?MA=p zNIDf&xPYuIy70ShtHlFXr=X$l&7Qtbfzdf^Y(KYv3lv$+Zu@t)SRHR<`Rf==L-h3N|#rIiCX$7fYVwikbc-SILIc!uTu-2np>^xWq16pkw zY?=;_*m1r$oSPzHzTDW$bBlSyo+{Rx+AN@z{im4^#St9)(;xU^IUbr4Y^vvD0Uc$J z#+MyTQ*(zBg-*F_qr9S1TtN0Km9&;gm#gwr+VELq3oMd?0{dC_BaNk#uLV9dBqW(XR-X-wm=3lf^9A_%u>22!UW4ZDRXiL=SXR@8^|bJO^{PLnl7Y1 z)60r1(&10T;qY6WtqNg8JAafjqmpp!0C z-#_F7TuA9A3uJGfWX=1Oo1$5)AxIJoe75$I5tTvCev{=?^OY^f=|!rkvAQ1?P5Pof zyOrtDiGAn^g}nGt|L4a&quU0fN%i$Hl ze~QC@eUOqvcaamf%KFUvn);0_%G1PF20kChPXf(Bvl)rj`JKubxO8vy{w-hqYsFdR z7<~D=>xHLuDuhVBs#c*N6N&(t3MDCH<4dYX@oOn8xfdwqTpSfDq08q%Ry0V#v{}c+ zT;>Z4vn5Nnl2Wle?RPVve_qblkV$=wYuP@sMZXd|h%Ngai1i zOn80}U{YmhBS{;TW?RPeajh`JL=vRj3&P?2Cl`{)0%pkD2mjGqDYZplp>n0Dg7qvb z7v08mGHK%{G2@$Ki$p!FJ%|icsz&bB+aYgviY7E)jHr-IeP-Id*NE6u zy+sL^TYQu-M|DL;Mye8Fj1ORS1g8ih`&8gHi9h{8JwrT0_7?G@qE7Yun|sL*%RZa# zo7$To3;aqfKJYPrGEQAP>X<~xwcRrUlVGv%ESy_ELG=5=`_8e&;`!#R57WX*l zKVbpKf4P)jDSsT?{3h{I$2J;$H5xY=i8wge2dDj%Wjr74gOa4KWqfSk&o)1AwHrp- zbP6lwivMS~a$jY!m*3OQ(()8B%dPy7th;_Q`Ncj4iB9Sfp|SKOX=!(#h1YXnq|OTkd8aQi)36*jy5lVI7LfrC0Mk ztLK0N$C@Ra$m`qp82L|N36kY2VpFkh&HvSZZ-wFDlz_#aKK9T!UFh@uCfDG>^!!=N zL_0M8b^nfKj1a10K<1&*SR4k$dg8fZ2Y}(ZCP=b`CN#p_G~35p$ z%eNu_V#S4jE`2K|q&>#1+0IRwZf>BNwN3xz5E^j61sD&EwcFd!q`l<@4_-M*vloL0 zk1pTQ7w9dTNPg-uab4Y?>|*nN#E1UTaYWFBCojtovm&{(7S+uaQ@8g8N@M-0u z$!t94kBS+9`!25yTSw0YANztxvNP+h@Z;4D8|#r3W-Gl>5y8})51O*CXwQy)ERve+ zI|C-qq=AbLeh!PqhbcbV)lZ;C_A->THK(!&D_wQ;G8EM2pTz}3?dQCNEBJ1`GAX-* zGaIt3!dwMK27Xe<*sAj`sFp;e;@#a88Bq>Ke-Kc5)ZQ)(W<5O?Uf9v0xPy;Z{QE@x zl>zms!Wx#fL*VC=ey4)xTf?ixC; z{eiL#xKD92*WtUNE*t{p>bN*Yemk!2V{!KnW2I~ zPALVU3sYQ{QPexHRs!NHprXq{Jz%Qvo2d}COwN3b;!gu6TrV%F=t5QTm<_z8(rwl+ zRxU5dIxf7k-VeWHi9DQJ)!SC^rtSb$xzciM|0AS*UGtZaRxJ|z?ppiIW%GpvW!Og7 z7rg0P^5xTbU}GGQ1^d-U95aewakRi21<}#RlBzqYou?n!$u5XGngR4>eyFnL$iK;5 zEfCIDrUqY}(05q>gx6`7>fj^3Bg%$qr(?Z~92Jy8zR*VhU^&A~7Y0urffAIP(zR2; zO@#hY+2*v>p%L`Igt+=7Z>6YZO=c;9@OS6!DO=T%aouO)-Adjjk(G{dkEtkHe%bG= z{YAc{`V;67NO!^SSL*O@`}o)AExaj;X{Yj~HN0sk3iX{MK-a;iAHjlNVs5Z(Au*P# zzgdpV>xpOfanFdsD8*O&=n#{~vnA+-(i~itaOPCm4!CtCT~;(>?|x+vY>Fj4-dKL7 zMUE(+H!Wg=bm3YtDn|NyLsuBaa)xp7m@6A2-3#JLc7z@;uL^Z_RSk#7LiB>At|S+U zw$OP`(aFd=V|@EXKF<$*a%j%&F(KhvM`6FdRA7O?Lc0xU&d!&0quu73tM`#?aQl~r z9M6Gz*fnuAwwO=ExN`dLTEMb`1%HfJ3eDM%78+h!`?mNgrZ;H3(QTQ5(qJ2%x4#ck zt-e^;nPbs?F!pH~Hr^dMA5L%0K6amd?4&>})UoVZJ}iP`??G=ix^c#kf7@Gmp(C<` z0ObINZPRjrTQ6}W59Btn?)~G~x#=VTkToiPLUEbGEo_4Xfu>xTr(qd|$TI>G%&xwy z|1p#HPt5SgWNdBFHJ3`qvn6WvbFXN@E=F=}GZjkc6QPYGpi$T`H9Ml|e>UIV?dSPi z&%ubSx|D>_q|q93=ZzxV8D@UjFWMV`<(ra6PB?MF2cV5Q(1}E+NElJ6Ww|2p2vZ%N z++r?gbm>O+nXcDZ<)!GuMj_p3^7-i##AQ%BU66V|(@G557kzexdv1^c}N{z%d2oM_}~c z>^c*_ZE`4>&2vBx;gABWJZG`>N^jR`)tpbVPtrAbLuA#uBYBdRN(nUHXJUJGN{_-8 z;Kq+nr30?+4>Xg(7GG`(IFqB9ZTmhXAWq~--C0t%9 zwx#t1X1y0l!gv2>Dl!t168nx+UaDwcg^Iz(5&q5!SxPNxj*mh7N_@^v*E?S^bisMb z-L{mP>65g~y~6mNUYUDqi}-Feu$Bt>bPS$fa0NO`V{g+>j~UsS8nH5OZxXh0V|6%2 zP}pmAA5yRTI(Lh0bz%%M4zqRdP&|>7jFQruibhl^vq8aB`p0MdvFWl^+T|Ioi0^}o z{?N|ggHvbj2r46}f;K+sA+LYLyschJbC7W~L{WDM915ikP!2wr*$1i_cD^?%wv>a< zYq5@nnhA_f=I7}LS-X8owH8=;o=rs^7#&WmFJXg+;ilPIE-u{di$Wa>mhFR zf~$;)dNg#1e%OwO67S>f|5b8#Blx3|<_CVELHbV9D#~LS2(cBrUm(sN$lBPQ+9=b* z?FCNdTY5Nq)hON0-rGt@DH#O!M-ebo@ufl z==Hd-dLm&_;@fKZS!`q9<|p{NVj7CLfG$ubIolAM#sMgjW@YvI^ROAbZus#nIm2RG zr~0N7OQhqsW1KL8gS|3gB%@0B0-Wwzp1kvx(#>D2TCfC7_N55!NRE*%Rsa!{;9TWK z-r5H^wVU^}9nW)x%&uY-hC)Q73T1AZD>>lZ-sk?!q0#p=t6=XbNyIZDCky)u+@Fa_tE+P;@p zLYs4jo_e$rRXh=Bw2|>^V$fwakz?mX%juz4SzpgS!8WNNx?)S_*LNzz=Zk?bvD)FS z>fD{Ret-g}%_qi#0Ng0yNc%}Vq0dd0L6UxdXpIoh9iUsr^}is5tqr?$Tduxr7KaBZ za1GZa+b(fO|Ay^UTjYJJ5dp(4iA%HN+-SIGl%59i?eFs7KWCN>FI;MvIyIZ;Lp*u{ z4WbfeWUvta%K23>r7iUP)QEoY#oYy^$D&a>g6-`+Y#!q)+Z$iBMK3=KnlOAWfGuE1 zFa9+>rk(1F+WYrpMCTR?!_QdD;_@9|CCXt|^)lE6G03qd;o+Yy}9xIU0pH_BUMzSrp8qITNAJL+41eKN`GyD$3LTfaQXy(ZFPB0Ro% zp}*LTQ?}r2PH39X)cDJ-CX=fVF`&8~ooe|~2PjV0^iH2#*)WX*T_=h{q3craMc!R` zKiM=>CjIQ@xra-P9pbGW(gG_hjWM-zYV-t3G+jmSDS>uK~3&TDiIA z^{4)8X{TqeNmSGuI&bF_w3{Gj=eCnXHK&l%^JAXOh5DH<@_q=2NGRxdG*X7Ln2#N% zV?*ba1d*5pVsb=~wqJJD12V^}c}n6pY{L8ng>U6uS*Vh>-YCsneU-gwbnAJ039I~U zv!etlF;#myp~>g_c?U^+EbR;@-E!xAxWiWCN0+0%Jrn|BvJLgbb~w?SaRp36h$b^U z#}ReiF?YG&T~-Mxot~P>GkP=Gzu-{NPaX3+``F7!CtJ@GBCw0s4AuO&%8gETz zu2_q$lRWh#?EBonK$@u}yX?P=)Q&zT`G^RK{LNzEKy(cz#lF!_xZ(M(#?|duKTrlmC zGI{C#>GggvIRmv;!3gH@ndSPpz|QR$zrAfzL%k=3PyWKFf%10K<*Ptm)J(}bPw-r8 z<+snr7I5S=NZdO<&5WIss5?&%rb^jrV0vT!;nOYo7Y;*J4Q3T+Rh2|ni5)JR;_pQR zzo8L`4u=K}+KgQ^VJ(f?(5vn6>FtMPZ2XHyRkQG6u(Yb7)Yc=m+aYt<`pEB{5q0J{J*J{3Dmp#IxWfvcWwD+A^R4wY6y^l;LbWbh zbG(uxJBl<}SPff^2W1#6WTdB10IB^A@a0=kZ(QD`)EGfd0g*)g6pHjlG0Ss7<(enB zy@eZ$8w{mWW=iKI{8=h%**zI%-N)8T*+&tL5*!5$9dh%&WjcA2hW*TOOD)JeTlG9K zZdE&(XNRW)NXq0DhQ1kEzKYYLUH#C>RWkLG;{-$qI0&+KGcoj23kwnW>u~rX_`5ze zY&FfG(rOsjUhEhv!hw7%lyM*2*=Ddp(>yx4dYyqbo-Hi4Sk5@=26D?SC3xiwDX9!( zg3-K$pjNxinl`ZI4B8-(ZymqZY|*UyJ`t)9WX=}Bs(DMQN}Nh?Ryd=y!NoD7M3$3f zJ-%_l@Li6=`S)m$Ghah%n$yfWvhII7K^ z5!X8Fjkro%Z|^Y;eq`;!|oR=^1RFTx6D-Hba1SA;X3BV4l~qlu$f#N zOVc-`_tm#Dtm>84q4v6+b{m^$?WtUo(_U;{ZCm}^lV7gTwdHKNn3TI`H$^Y}gtR$_ zD;`*XJMj_w*yp~V$T~c6(y{Sr!M)UN?CQy?JZSz>6`6hr^fPg6@f9{||Jy-J;@xYG z_hf{k>;+gXXLi*;B*rG+>!md2WNchy&3Y6lvIUr;tj~GJ%wOyM zwcOoBBvf6u1LXY6LSmp+#5x}_AuwJ%t?G?@A1TOHCGzQbnE^5nY%+YFZpc3V#Mk{* zRdKJ}ReC1mwuzz6)oH`zb%9c|H^X?(NdYKvz@Vy?2hi7Oa+YPikU73`^xVBJbg-I- zFrQLuD~o0wx3Q^x8dP2BEvNb%pNe|FUK_$`c=d6H3m!6&k^v^WWupRB5;BVTG0%p< zZ{te_$Q+!YpYbQIh%(v}tzE`(S9-JNo|+nG@vHBVS&N5mIC>nBN9Dvb%p2rv?MEtS zVrm#p$L&rR{gO{+q5Vz>3_VuUu!iIc>%rq ztoS()-g=Lfp8WarzIslwVDPsgpjc z5x3tAFDgaVv%))AAE<{WblEde+v&%6C;#5|k&{V5qG$^$>dY=D(QHrg9ogcXJ&lyS zvAW}KGymp$iPStw^0K|gA$#&4V-f_v(Ed9W*km(o{luFXzN3}3rvTIm6ckFaj$?Q^ zmA@`PO!19t_!XWgIn5b-leP~Ecwj^aI*=Jh;7%>|uBFF0-n-;0xqEjbgmIFgR4+SI z`xcXJXXDdC(_DRZ{p*%q=eU7W6PanrV`b&-Ij#bVcuY7kt%w6v7d}3Jxm}^UrVa9y z(|8&oq?9(|C#gIGG$3nQDGyx3^4heGpH5t!BpI(qrOmmMciNgKZT<2KBbgGAx=L;B zTRR7Gv4mkw5`u5qMX81&4?UjdJQrYZa~o2%Vb=<3|+ z5&_vsdOslbnzCj5iI}cF1fwd(KjI)Qq#;j*EP=D25sDC?H_&Q_zylHRR z2hfclzqfpi9Cccy?HW4yvTI8d;Jv4g4`qsfi5&+P_k1^kvrZvsnQe9_HalL1G5f~b zH*XgzCJbNuao?LBt6sEnhm6^1GPgeNesQ$`q=#)lDd(#D zMfT)Fk8zw11Ape5LaFN$Z3FXo z$vN|9rhNq~skOA8Owj!MGv=P8)wtCsJ)Ch%P9D733Vk;TOPI#`=hGqfrsS^C1^!jn z46e;hY}$$`8`tLiu(z=C4U-2$3SW+IPgX}(msjO`41v0*oXzoL*L6t+L!F5|Y-{~& z>MbPE%6D|0Dz97Q2VY7+b~A3goF;RBnl~_4-Cj`?sc+wE?dQXlnN$&}AHIcYeQu10|<&yR{*q?Ht#Ka6eU?(S!fd4pS4BUcN0h~D$0 zNkKPifa$k}PyP+}@N|$Xr!}Ko*R@_<4x1|SEvyBH&G}fBNKT>QO+`P5emyExoid5q zCdqPEGiH(rNPL)NIjPt8lds^^_oIHlmVWXY>g>enWud4|sYq#y=(}6V{z?*NQN}{{ zirT8KRRK!DyY4Yoh_OcgLKUSRu!C*)C1ypew%PQ$k*zIP1fLTm*VHo9chZH6(8M@Y zm{Vh25WTcbQG9-Sa8=qP%_XU5=KE$}e_of9I})c-*0s$Y2J z`sSwcQ;fivz`d|#%r{kIGVZ?X;{9{wNdcE)!P-K-Arkj~83?lhs5xm2@UE&NKfkyg zyR0zb|Ae}xpEQOroextNl}MmSbz!ej_}wE37ezM7TaK@SS6 zr@LE(t-Gatoe>xlUjNavzPi$5g_=`w;bxk-4}}Q`n08^*4A$?Qt7ODb$BlVc?r!^t zYq|AAG{p^ouwULezCNW&Y~<9o6{ca$KY!^dj%zqfclf># zP4OyoC`iQbeo#=`-b>n!yg#=je(i4MpHuNncrus_6N0~wNX%iw!{kJo3 z`Pq3AzOF4?Gm8s5?e*=|Szb2l4|Yhdnm<1EkTkr}jQI6G5pzt$x5CPKNf=!xgUaWS zT-{&Z-cw%p5BHNf)bR*=hnTXDC@z*2Xdpv|b~W_aA*%522xxI4o%cuk2o^L@+12U0 zSyAqHg_hfoXSeJ>yuN4{C@;-0(vDCd&Tr7M8a_HQnJM*7Yjop>NW;I&XISvyPPF*i zY56`)YrEaWcSS$h9DaGK-1+R{U9R~hBTYAPj3#ZQr8T0WS43b428>U zBR&RNap)WzW;|tH2Xc&08(c~|={sp3*XND5_W$SszaN3NYHPpQ0fRGJckWT58t^u}yf=UhDDJdo0p-2whAq~1 z?z!ilbMHUj^X!Lb&j5SB?|N7K)>^+69EOp(i~hO-iJcZmW}13({b(g@HVNc^I9tz+ z6!8*Cs*nle1M!ycOC(f$mX{pwK76=`tP!1w_#H!|oQw+6I)&aJ=Y(ZW*7=FP((agLz(d9H%%*g|mdFRX3LGm}OmU!IeY$$zp7}qSp<{ zMB_s5T+1NS;T@p#T?almc!=>DrA#p$Tb}8jk7*_ykb+cBl4wm+L>V;#$@axge_3~T zs~IUP++|2oeIAsB6DJ%|{EWBcRW+1N{isXMC$^W}!i@lgL;&(jh8W$CB@p!8is!Z+ zX(Umg^%^BJ;q~On-MN%iugQbe?%Ib3 zGH!r7yH1U4j&cgSf+qVH5*&F!@>oH4d(WTv`T**6a{jPa*S=K3Y#ND1KC8q|{dhe6 zl0!{0yp8sDpAL*I^ljfge}RMVsmuG;IO?qI_1U}`}iadl^N5;v`&fdJKX2vRix zP=B(;UebJ(ctBnaw$%C*y`;%ioz?yxrzVc+`B#S;{+IAL4vwH3LDumZkCe+&-|VeZ zoBE#ktAOP;TTFeg?l}>vFix5yQj8>{4ND7NtW}Qa&`rK>AKbMQ%`)CseGhuG?}qg1 za7T$&UnjUyKNCZKE(wDD5)gRoU={c+AzycE7MHWn(4g6+AGUkrV8h*h4LhB^nRQ%> zCE3>~$te7-ZSVcj>0oTg7UPz90(O~1`qxZ}y`}ICPVSdd+!*7?%s)?o|`|4 z%JAXg+q0FrIFvnB1&`3*zD;n#^MlOddb{gP@3|p1r7&Kk%hBI>s;3gFA(Of@Sca(p zX4fx^-t9|g;v~fU_6R@%>%A=LpIq!D#8^pp=GR$usF57zG+vm0*Q|Y=a2k|_9pVZy*=$nCjA_4G{A*keK!Xux-|$lIETvwSNI%OVLQ}4 zLDD|n#IDwqOcT}Ixdq$P8|v;kLY5^Cfpj&DT=Q^P=PH(sm8Fz2rF!ooZEuDYPnvc>AXyfyJxT@PW%v?ykPTX zVCd+$3!5^Z``AiL&y<(<$BT0t`)Gs4l5bntfTP2hy1V(z@hb1?Q(Rfl6eh~SuU$6U z2f636tJ=&v$iAc{iUUakT@_SdWC2!3 zjE(kmyNNitktl4VcSSV_9gDc4{rh8JV+XjMY3q-;bb{L0-&Z6XkS^H#Jj2O?bDEij zHBU%vJM3w!F6QZp<~_|agb%mnGKvXOD-WMc^9)GA$_&xjy75FV6AdnqWhYbjBZhRZ z4h!-$CD_uP26$yBeeb6DT_)_1e*|dhwP)_?1Xd|t?7p&OS+ehDvpiricy{l>W2PiNyNJM0GSlwd1v3C1i_cYV z#$CRYRPT9ke$Dl~dA;CX94fqh<2=FG~%<1uA($VCa#5= zUqfKand1AMbtB-Z3`nHjqLaS!p^#`lPs@tdfiK39{}45$sb)uSiVDRcvkQ9G(TRCA z%}+oDIjjGw9u}A_E|XD^?g%4Q3xeZ!lH+on_2vwo3!~)|O4tFQ!^hE7Q}<(nDuii4 zcGE1dR`9$LvO4M%*7_^?jlAbjN-vCkO`!2uO-0yJc)3?-GuP+SxVlU*_+wu>c+dD1^`*G%bWDsD^Z{fkV_Jwn=G@>QPif$!Y5)FmD$lTm4&Rman$JLIh{^U` ztBSI08uV2aksnahAgQGt`@j(U-l}tP*rI!S6%72K8X^Vqq58!3COT%enfrIN#lx3< z_KZ83t+G_FYf1S&*4pFqqM6jVBN*vi|IZDFv%L$d>{a4(+P>uV9`qEs1u*$ny}p60 zZXEIg8zL)S*NxGLlR3}1Ra=JfWA+Q5+PzTQ{zflAZ`k9}X5;e1heshx0+$Hs@cY`bv5<$Xd}B8faG4$`bpk3ORF!^0bB z7IA&;h;jHdtDInDT@_tbCdBtjjFRDsOJpyi3yn5QVEN&pufURCtGY78nnc2D_Jy+1 zoCFZY9oBCz{p`n0BDE~K5Xak{KzT8qj13jcwdZSImI7IhP0uJvHf+yOhEi0~#lbb7 z0(yuuxL~BiHhB`%6tN^E%y&Zl+Qn(9Q$kfxRdXds*6Cz-@1`A4n-u`A!;^PK#TPtQrxpw()KX$o&b3_o!7Kp* zC)C@7+boT7m9M}S6m##CU7OaR8Pw{066fBYTr6>7F|`bDywF=Emj$7qL`Fs~$C6(tEPQ=y_C2ZO1I`qRwa@u+gq5LUlv?hHJK)>r z(tG}@>00CbuyKm?6rNn0Ik$ggR;T^}Y`MQYXh=$HqwyIt3)RhHxK@FY+1)^ZU;pre z_j=o9(JIZyV{L$Xvc}jX2l+!B6T6-@E8zpQiRjnFx9GBHU9Y}burdZ$$R#ZVdd6eI z5m%_Qy?b@G`BloG{NQSx80)Sa0LsC9WAc+D2dR9ois3V6sRdzby;S zx!9bzrWp9ix)u0IJ;hMj$T7rg2WBGON=0g1u$S+K=al8>4PFdEA-y5~y2=N;J}SCf z_DbwV3`fvEfFykf27Xs|u9F3}9dR7SNxUO)%_;iEq&!CnMKdTEOfn?_kcg34R5IVd z({o@|NBmi_$XD2|Lxk~F`jk}4VsR2vx#0RXx zbfhkAak-Iz<+% zS@Gs+=dt7MX+~l8b!Ih2W#So?8RI(i9>N=_>H`iP8WRKH;WkFtMD7b)WUJRRlWW&^ zO$@x03_Pt08+eahdnqa)(r-^+nQ{g&2JV% zmgP%9_Lb$zqFEF(dL3a@E}T912q34oD-w5OVurQqGkOU24k9BeQb+r^I!H zLkYx9TT+tW@0eZdB$ZGkw7P1vU|M2S5T27K0=QTADBp~behjF^gb{0?y>)q0b?hh{NS8n(GcE#G#Q)dZK@N)vjNW|cm%P)e0aV+Q77!|WV7B-C+T89&# z6nct5lC8ohU_R?@BO}No#o`S^PQyK6iyQSpZDf>MmTzz1P_CxoQn^Zo|9-n*iC`pY8sCifNAK&=)&0F7!tD09 zm09`)IPDO%l#L_Po9nCn1Al?cxe?(0TPc4<(0B+Pd3i$;&nt%V)n3k>q)#O3aV*f| z*kGNCD0a%aZu&R_8sX0k$%)4cf=hF;j6wI~&??Ehcp5L{m%J2Lrc0WmN`QEV87~*O zOBYswdMx^*Cc`?H2Fc>4eKM=5^xB2bUC~c^ko*pf;|tp1+Oi;+?~i(}Ls38zc|F|H z!+oO&Y`lo>bpKd(^Ls{ch4qk*m!WUvNTKR)(SpFvqX9amf(jia6WKUA z)!cwqxi=E6sqHslh!A?+Hw?^cQP(P>H%j8?y&jb5V_w$N8xrG?I*KTq5rg|hf&Ncr zQj%!OtKezE?>ghs(bG$>322Iixv9K}hDf<)dmOjD-i=gwa>*o`zXS9Z*!*NBl#(~^ zxMPJobmaFbH{m_r#rS!UANIO+z+;VWV;O7M?0Ov-Mz; zPaMnr#gfb8Fz~K9RL;^&JHo5)s8$H~F;;D#{oarsTEuPIV2k(g{KdGV){sK8%3=j{ zs)NM{XZBYYW#%{HwKY11R%&NukWWiI+oc~nLYmk$i<9%pza_0d zB1z)4cGIQfVy?{kbrrhFz+_Zb^zi;-!*u9!xSo)~ z{*N%eBp#Y|oYe1(yX=Fa8GWQp86xI-Y1-i_SPKyx$Pzhjl&K1f(lzb4wgRf%#Kp^G zM1xC(x1iztx|Tq;28i}CmgNt&f z{vF}={?0_8(k9W1WTR(Dk(^Hf++7l!_T>wE@7eeMyL4>AF9Ivb8Q&6qu=v!Rf~>?q z8Kp_`eaRsE@bJW7o6F`i6x1&ZI>uN_$N#NXlp)`7Q+7oYR!SRHGOu0t{kkG=Aa%nq zocZ&z!6q7HjB&qr8Uq-XRLj=j_@cf?S4*ZSz#-`fioCynB^Lwd1}ixamsKUl=;PF@ z&f}Zz?;%!{98aR2AX|w$mS~S+v+QUWDJU`_O{Ftx$J2&_hZ_X+3Rzi*wtAl=1y|jF z>m@M^BKw^v65$PKvos~M>0BX-AaZaGwr7=D&k{H@yP0E&lOR*1_iaBAvAtv^T?I?lJXs#oiA5bS_uI z#ad=VsM4|x%PntvZhsF)TKp9tF@=_*O=yjkr5{$Yu+x9{HQtU*;>x2B(B7@ea{1M@ zlENV~V&E~YUqS@EOrnIyy~?semCt;wTZ$Em!;1bpZ}-K21Jpl~$6sLe_r0Gj-aK}k zZHvhmg-rS#9T(z}yVf}QOEfq`UtYK>rM~%U<>~;GvZ)l^VfR9@a@Bq;I6;{At#aUw zY`BEg#`04e#ERVbH(if>%RWk7G@1vlKcYLwLQAC8ZO0dopvf>@G|Bz%SkeO$OvC9^$*pQT=ajI^iMzy9YOPR9~az>c> zH|YR>ipo?BAs*H{Pr)*T9t1^Dv~#pN9Ps_gF5Yy`y5tE%w@>P?T^&an^Q06NROn?V2mUSjKaK$S<1orVh$_j0*AG)nFe@bBiy7-GpA>c19aQyR4xcCFYe>+6 zQP=OV_lhFTu;ig+otV|IF@Co(kIxT~itE|HoWaWHDi=~GRXv^orJTKNM{K@t%Mb}8 zB|B$V;kQfpr$#~^pwwgmTEAbqwl8LF)9?Fc`^JP+v-{QJhcwz=3`-(z;{ED0luSE| zQI(Gx1NWGE-73os-?|v-ZVOq)IWAv_x?-D}xz&#&=PPU2Ve$Inac+bV%#h2mQL4&l zJ~4lx?FRI@o>dmBJd(9A9{iP)bP&8FWZsypX%4$&mRSUFJgeSgUN^Q6&*Ksa4i|I&p_TF9cwg#Ie z62_}_mET-?ZSp)rBZfGCwE>x*KiRtpw&tuyG6ZPzYYQJVbsO>DIt|JDNZ172hYz`; z*EeTBB%YUN4vNi(`y5CTvmCckaY&h|EiEpRNtw0LK40hW`exmpgBg*BS(4C-`OKh* z@TW;@PLU7oV{$SQ#`~FNi-p{nTF+kNkOcdc)#Oos#^xE-CJD&EO_%H(9;y9_s1pwf zQvroK8&(3{8+T&fmXTfhpj)Adf6Q1Mc+xrEvikO&4Vt2#xbe~RwO$qHH62UQ<=%VL z4iqj=vzqaHgDkZP&xxlp{*8vOZn3$-J3nwtuZ*UB4m(!Xsb8) zCNh#m)4ty*H44&#avOfLMusCfl&QPYG^?X}YIEiWM4WV*)o4^%4N1S0o7pHRDnyuA z9QF&7DD+>7IJ{rUU_mll8SFqr@M&N3C>uf#TSH{YLua$1IE0+~sy(dl$zTGyM!fHf3@mQKI@}?%UU^8IIF%S9C@5cT zjX)ONlqzsaEKRJAF7n6lt9)RSZ@~4ACtjB5c9UNusGvyn-goSE&^E)Gf+bl6<hse<}zZFiqk}dvp zursSaN(8Dv-I#V%VslBh%e`{NSAjd^9&jAYhe$vjL{~*K#Dsx#k___nSJbY7`AQO% z>4Ma*KYN3EzT3!3(;|YX-x|Yfu@vHm9krd6#KP7y3~vcO)S~UhqxjQ20WOFpogChd zaExn^#q2&*CmoDr7B~v=t`NUzAXwMmHJHY2p!#xWHM;U#xYRk8ef=DY-;3gR$gdqn z{S&Si<(5CDEKdxu?vi~oA;9?X8Z61kIxZS2Ra^+=izR}Ss0$26T3wbIN8-px_v=(E zd^x45Dt3Fhk9p=Y%5(zGfyMOdM0TkXd`}!<5YCCA^Xz z(4$7}daZ{wgXTGvhOvgWp3ckm1X-E3_PBbMJk87Wig?*f|TWvpy`$^B;yM% z9*z2gDWrTZP(dB&J;02W)#v;?Q~MeWOMaolHO?E=UYP-|v1=d6IL1_``gC_4JD3M{ zTwXBhEWGDTu>ST%wG5jg(t8tAWUN8bOvL+T>MtO5Z6EV|3nTH-sSR48d)Hr#h6>k zBF3YS71K58<_oHS0;&_VHV;B)Qvf?K(IpDx$(Of7f&-lneQ?NsHEq>?bcHn!5`tq8nG)(QE7icC;tW~ zh%cWjApjE7E<_{k*DG3xkY;o{x0?9EkjdvEE4AO#W6T7K+m!#Y_&>J5#E3u`(e9uE zs*0z`+vrdEjI_v`=h^W95C$3)T*SS{W7z2O{8 z3W;v}f!u)r7y{AVU1yEO%^)3kAe+J>k>RM~YRR>1g^-UnZ@=81Vr_YV68Qc2^|LGQ zI<}3k^0!bD!i_gkV;|*WHA+kNsT&-52lEZ=4kbJF8+hjL|Fe~v2U&C78Jc90_x)Q!G>|Gw-v+|}vF zVM03mvxzjz#6+pSTZx*y(}%2_?8(N&LcYMG1FMd=HXKdVAkc9qGo)isjy3JH9-y!;3$}v>E$V+-_ zoOA3QN@f5;!h?g#h4lZK_&|v8PLB(AR|-PR;#E~upGts6ZHjPj+_eygmahIu@3~0I zF_00>t~({`e%Sex)a@o}|KZ{JX{o_M9a>IKPPb10Riyvhd)?nx%I7P(SmZ1sm;LG= z@gzUtXg+HA*48FaP9f?h-j^&uTj0(b^}W_vu(j0@&z^URp?rL#BEt0Jc&;ZSo-CCY zhhN4Zq{a4+4R|%jZ{sS$-E&nxhXdM2ArrY$g_xL4sjkar)+u}^bh5!zdzhJ8WRQ| z!gje9@lmx*9kcpLeTr5a!+@LA-6Rv4aJUmIWwrZ>p&xJfx*SLb@AI#+fNOQY%v?%c zNq1GzwAtvssN~6J*Sh&ASLCojsSh2VJi@TwM6$#GZux#73_3cx&V03y`krHoA_$+CwN1BqgT8~S% zu4Js+L{px0c`(ib9j%;hJ)0h{+#)ix`qV5| zY7(0a8N+URcsf0WYUXB!^3SsSoW}!*pPIFmsPQK!>)$s#^Fk2Qg!t=K+C)mnKo6BW4= z7sq%U&KQS;1t;)c1?^*Nk>1akIZxfp%B@<>YjZk|hIST%P4}kGI2Yh2rav-wS3)sM zy5n;VqxlRyJ0`y~^pcbI#kv-Z3${|AH}~nX zBn8Yh)t8K)YkgACDrdSIi|2R%RkFKMqc$2`UtLp<5w;HgD`Cg!=R7sD;ertDdFR5% ziTTr-@1m#1+I6Sya!T^C$f#Jd1yyXz*BycKwFli^4@?T2%Ws()`jbKz-yW_57rmTo z#YpzOLsvc`i9!a&8(u{&^$i8;MFvv)%U!f2u9g9QtpI((@y+uf_;9}{^WEY`eS|lh zIo{L0o|ey7-4PaHRUzl!HHt1>VJ773oeN5dS8c6^R=eClT8OK?<|ar0H(NPJt?Dv9w=6^r_sDH;=P3b3AD!cYFX z-y0sDYN}kScjlUXZzp(eK(p7dzmlaB`l-xwsn*>hsBe2kGn}5}Xl9~LU%_gq5I^?d zJE-n5v(;C(LY<7qX6*#H+LE;i>W8G_3y#FOg8VEQ5Nxl^+7m7J|BOO@6cUgVb5GAh zeou=QX7VG&TOIynE0#Z9x*&$)o1&vHJ^McvOUk`_9}+buu`HiA244tvcg;=X;e|y=;5x31xjn6p3^b4X&so&>W!* zAH4rVA)3`Vna}b&B=)8X8CsR>s>BiK%%EeAhISruirOK2n-d`bbR<0u!)@B|*UN8! zU_flQ{uSlpaN#SAc!HJ#XkQTz5z`CX4L%2DaLA%Pd790BGIc=SN0J!Yct}iQ_JBmUdaa5aDRiSIWhCQ^7zk7iZb0P)u z;ZPuXVu3p)PWH7q|vW>ij%tVhF+U@2+RV{d-E zR8Yl`s8gYZZ_)})L4BU% z!Idh>UwKhv@M7$`wC~lRVD;{U8F*XeoXg!csC{F9XD$?5Jx}XP?N%rb=LUg5-K2BB z!OYu>>eDv3EIA>@@h$8<1@jQ>_8mMhFB3@7W( zl(PyJVzX$E9kt6G@=GO`ytXD6>qaucRj&DcjrBMFAJ6%wIT&M8EWX{pr;I3eAp;Bu z*VfV*@Avn#u}m4n%&dKz-XH+ItaKB%5PU_sL4U7oR@k(iPqfxm7f`bf7r-& zpHLc&`_UL4k4?CT(&y<}$e+rSZ#b|_ps32!mknk&^)=6XpP7E8E9sp?Bs6Q%oz-(( zPt?3G`hacI^+mIG?ATy%jiBCC#Vb2j`YKFvg}BxqgLS#6p(vSe|HbC8t8Wr?-Hkt) ziw7}=rTv^5;4|>-IQ{Cs`bc{lTuH3&|J>riSia`8-H-GKS><}ZZK+z#Y&siyIz5}H zG!&L?Tm%aYKlkbIMsIuf8#^W|x2P5PGnqp$XCrQt9wCQS+DI1$IRoF#vu6R|*{Q?p zB*H9lwi<3Zy>DOs0F)G10pW&+Y z*Yj*XX!18-_@Mf{xwNswxF=Ina`ZKyDUJK@+E&8*pe>X|D_H9C2GwAH2BAn91*?uy=nGeVk%i%p)$+=Mz? zaB|;QR;R)qU_i7#`-w`p%=cIO8Qs5omo#wg9X;Dyz`*!HiYe3Rai8EFJ+fozh*nk1 z-%Ydu|5sZ*Uh&B{o}Kf+n<#P2HoX8Eufn9co#}GrPvf6{R6eQ;nfMW^dp}pf4>W$yxrz?xIQ78lL(X+CfImaP+GJxz>eSe_g-$RnZmyi4_#i`59N7xAlmA& z8Ws$8lOArQpAR_l<{yxJh!W*{74Y#%bF^r4!}?;BT7RbuOCb3=IyX79r_%~g3LgPx zINddi+E~+~o7I*b*RSSzE$aR^7C&mWdYpcGM_N5v=r|=!_PxYrz(VgJaFM^@mXO^9 zAUHZy86NIAIoX|WR+Jx()?}|^AX+ILu*alUu5&Sz({3-#SD!Aee&mypYk7O#1&fbo z+SGgnV_+i-_GPBQBX72SXwI3_bw4yv4pFc5GrbR;`nF8Czy9g@&bXp7usbSJj#iJp zAA<5S{EkJsrz%V5Te7HPA|K~ynQFGehsU{E8VH(FZptJP zvyN6`r8SwXb6HH4PVcA5?*1M+BXnf#Te8vN&NtkO-L3N-^<0_JjTgvvN(1Yo8H9Z& zE(647(g+*@!^h|uB`dT3Y=poOz<*+izv7Bt_wNjD%T%xCN`L%qng9IAOO5CcD1!Y* z{|r?B>*g;H1^xuH|KsNW_0<3ItUuSxOAYH_I5Ogqef_Wg;~$MlP$bF*)WCbEii6I* zxFPk65ToB~ObVsTj?VJ47S2g`TiuqkkIWKvSpo%u&Ttt5#^dW$3wfQ$@}{tWnwrCB zyLV+ML}xI+W#{t2YWS;L4BdU%_G$^gfAtVY{(>hP=VDPcsViD~b@5Gm5`;T6+z&K~ z0+YwJ%VFgwFgt4<`5cYR>J77drgt#Efsml0jZY`2ZU3GGNn#M4G;GW-&2NZugJVsAP=au%WX#b1H2VYOb9@xNb zEkRR>%9{`F_v3Al1p)TtVd=A7Ri-u_^6NQ-kDxVcvxJV1jNJ;sn?#f-szZGuVV*CI zlzC)yU2pgsf^6;9*ViKxcdnPPcZbz~Czem}5vzZe?n(MDwf|TB{es}3`_a~e<049% zUkvVJI@lwUYP`FxDf%|!&20|Mnl+KX5!Nq=|Gf2j1I0HXzrFlDWdNcz7<(SP#~{(5-dm~O{1 zDwZZ~|0Sn?IEH_z#3llhzob-%mHdxd{Xf@9hxhRo_Y8(vl>fC_2mF1Nzm{7_i*O~b ztl9+sTghH4V)mr+$ke&+7!OaC>L6Yy;D?ZzKnBSq+J9g4yQj#4?px8LKw-qUZ{I#w zUXJ9el>ydmj}bE!E)&J1J#Pz*2rWqmTNv2b6jfCt1H`(8ZuepI0rA|NCr|PJ$6_E% z7+G0ao%?2Rv%}8SEc+v`{qurXrwz{#&j|6@x3U`h{1&^x40gGj(UFC-FXVDV>vTEc zn06*ai(Cuj{CI&5f9+c>zf`A)pgR`MXzkQ&v&$*rY|s0Ejw<{WpO{I02(?JtzohVk z>?AqYdLSfigz6`xLG+dLzY6d6OVC4PLthbXk*iZKS{r%!PgyxRUTi-GG9-p>+L4@$ zW3W~(66WM;oy>qx7a1I?i2B`{@qur0RGz(?q?X5t2%~ehY4%gibD-9)_pJQZ(E(53 zvM^P-nO;%i*Pp%lDf~V-_@Lnp{i^w<6>5@E_d7arg*&SQ;ytr`0u^Vr4=^R85%Eou zNR&hAzw1S*;(WU=UuQVr?E7S-R1H1t3C<-HQs=TwQ_{%Ow{AQ5t?dIzRQ0Tn1cIl_ z$`V`hGhvSB>lP1BKZj_4%=UB@`S=TqU>$LB{|2sWHoaHbmsO_u_zynZ^o$MPVm(?+FG2Nik*C=QVTnw3IUJS*THlN(E1wUYqY=j;s%Jp&Is%!kMx&k%MlmCOi^^O=B z^NfJ=KJ3+0jU0pT;rq3D=#!i?-^&HYbx%*+l1ZrEn7;3+ec(91Aga!n_Ht~&@`Rt) zL8?Wn_iN7ruQ{E*l@H}95t4eH_&6<|Vm8n0&kF7;9Ch=oj3uFW<>7w}x$*nwPY}?7LjED|(aSy^BoC{)@Ll*geb{v+#@R%Z_l$D8#f*4Ubbx%eB+Y z{c-ssaOL-}y{T(W3~%p|>PK)4VS>OCR{ z8b%^hK0(#p`hf@DgpoDX)^T(M*8lSm9j9^LfI6K@B z)I9zZG1^5ZvD2)Hph+(BTn`$Dce)z_jMLTI4(&Ik)|GlARiOvpkJ5Xk>`W(j;ram{ z&k#dTw6`8UBFvoYuO9y2ngU`ZINgep+n>w0G5x8(1AnhQr~#xjM#laCZb)WK^@k@1 z3yqcvuo1P`Mz2tp-9fhbVy$L-Ve?YGEiS{i2Q0EhQ#?|`0)I zq_73yQaw;yh39w+NHG9U>D3OS*Q2=JYLgnFk^^R1U)sBmuPaYD1UwfJLrrep`tVNo zeWGK#+5t@h+6L=NM8$L1y!Nsi9&*4NKf`&RFt~rJ*2*cMY2KemtEoQ&f8`m1IJC|L zgowHluDUh;2*SYMcxjL@hR?v9Z|!=WFX#E$^Pw#5C|E_M%leQLLBL+Fa$2nS*}hp_ zAXwL>s|b-4n0;_YsjwYY?bghj=f4lnCAKYl+XHE9sebw=r0w=DCmE&c%D@3emfxzgKXg;RL$)? zk&XVphJHT^v3V48&hH-EicZ@{KH2@1KH_6JFWZ9UzKEkfEjJEEj3k4GUAY@09dzre267{w#H|9+#4jWu=B5Z^iszOM;KQYL0C1yz6|)-@U?!ShLG5 zHm$n|uVno1YmTQuBy5)ay4K_QcC4#J-R+x&K+r!%xP&FN-@fjy@(-Ql+-jGmq##lM z#MFFq429s3*vZUzxti{99C(BEu4iwmZKpoVZH$#{ZiS~HMn6tZL||vZtMZ7TwDq<; z+qSmMrGg)G58-N}UJJV)e&}s7D)_r){8|6G#2e`Cd{tZ8OvJ9I)!uk!9Set?F?YMD zr#1PQPH7G=BhZSyrxJxI^_=%r(cwC@Qr82)ioHcFZYEqt19v`0 z@d$#K!ZTMsa)tjMvj^os^z>3_vrb9L0#S5JqF%~~K9bi$tYb+7Ls4Q?MC%PDPTe3S zyFCYw&1fXG`~8^ZcNFAiZs~&$Ow)JOjwYNpX-U7m}+z7I7 z6suqErbH<&0Dud}ysmqk@{o_oNEwrat2Nif`-9 zYdfOin|c+TK^qkQF6YI%2sMI9BAN3F?@4A67~&e zXzSk>2x$9>XG)^UNX8<4fbF<5g&%T=USI8s-9$}>-sHu-+SM>v5WjH$M8CpZnV?_DAc=Q`T9Ytku~_miewW`Sr{-{_ws^(euN(GZfDu_rA;6-SuLEt79OwWDvH{FApt7(Q} zk0IuYaRBXw^JbdUa_94pX;?%|^E{gsB{^`!KosnKln8Guu^`_Cvo^2&1V}CSSyVrR zQe+$4H|7J0`k_vs&HNup7wM3FGLaI~4w+f~Qj}cB@+j&_%{YtjRioxb582kEPA-CX z!pPeFjUmDewom^C;sH!4E2qAip~pbRfI?O!3XtY0^#Yus-`qRG!zo5P@Er zGjFq8z)45+n;A_jnRfMvv88SeUbH|uAF5h9kO zkP$F&YIM)&5dnrj1X%XGKNS&i5+2f@eEemVCHIJFS_1j0w9!fNOyrx!H{CHTo8>E> z2R~xew*=L*t%=jD9*QAw%mIc2Zd~$suI+zjFw3UM%^v~B4Ghf6#t^KvA*DKQBDQmk zH=81*ks-xYYS{N~{BJa>zUpavH}yEae2fUFNdyD*UvI%o-&UH>*w2Io+HC2fMAc?7 z;eWp>P@!*At?J!>WTQ@;enXaeJ5RVBc#AVKr5ODjW*9Ywo?8Sb0w!+o$@1hg(X^7} z$r4*)%x~C8`fghR;d60*)!!K!FAgIW>h&M5up~1g2C8AbNmZRPgd(v++45Z-X}*^H z<}cB($q}?PyphDRZ@*`+|5E&kpuw9FB$D}{v!x$*BDZxUpo@Sh5M*Qefhr%}GrP8z zXMpz*3Fiu&xeU0OQ!O*aG{G!gHSo|#EB4t%lySMTp==yDy#5#YTO!c%Q+GnSXEy8-22gU&Fe;9-$mY+`E8pnX6 z<)aGHb8<4Qm7oS~2^4n!FtKuwez)vFi2tRJz$WuI<$%jcmUqeX^k0ZEoLNU-Jn9&x zrtC?^%s+sH*Gm*@5HgZyG^!x2Ij_AAW^XNRm7kxdTtmsiD(LUxzkZa~-C1ZQJbJd& z3;0_FSP1wX#u+GI$ZWrq6<`af2(QG|SL~02$fW1PaJY8^y~JjFD%p5oMo3vFe#a3% zQjluF9QGZxu_r7QI76G*ugl7|Mp%eW-hXWJ8-qtEQ$}M^`?y7qvdixFXQ(_Tdbc9+ z()FHob3WT!iW7H3+HY5)Mb1z9?La3HOTdToEi3&*eq)`B)l-ng_>3uGm|B6(XJ z1QtsZPjw?lb}`aK-*I(NML*`AgUG^%k!H$Zd}_HtKMPff~^b0!K>Sn3v%Q1yuRY80Lkh#?cEU%)$w@ zgMn!dCkHPycLxio8-cr}baYKPPW;8uB}>;D7GVWutF=5(s#=3rTuMq}sY)s`WOjhP zY+7|UjBmI>H-nHpLMKi|*l=DF|lTf>-EKzS~)f zm?ul#oWGfaJ%V1CUxMG<+?zqFO~l#Cfe2@i7UPq4u>GtBh(4ig_hC3+spN_!+PqFP zDVp}^Ofq2>w8O}0KJ7KoSdnDBC1h!9>{c%|*e1M~MOTE#ywLV5^e|L{=4IEs zWfKy3ilTtN^MO@0N}|-*nbonL|D)?G!=miEa1}uXm5>sVl5V6Kx}>`s)B)-4ctjMW z8zhJBX6O`Dq;cr(7`i*ogYWyr`<-*Hiywnr;M#kyz3RT#+TKg<3WC5R)egM}f_r7O zb|w9FKa~QUToQ7brCQb3_~IWH%jKn(ZXPPRJk#sVM|-AoB;+V+yEP9wXW?FId!G6@ z-9jfvdhAjmz2_hgNd#o^UzXdZsz|r*Z&ADIr*7iEKbT^?L6B7;RZFKNpTYn=z`L@P zI+vz%qUA`d>XxV864Kxi*E0L0pLzuceaRSw??NHgk3US|AE)B`7^QR`l$-UG+~NNG zg}?%5IOdYOWA%H)y@@zCE^oM;sK9_zs zGQL3Bpl1h^{n_ZY>~yVkE{}yDhN53F-8{j+?i_icP%n^PFd@4VW2;9RS1ttn8WGc7fNxp@~*KAhZkCiE!Fb_($`Fvj&! zYiG{AZUH|xMU-)xGLPAhR`EE@Xhj#EDsc7T;#4=XXGemg)vbE8d z1}o>ux5M~MLyso=Fx>F|H@rJ&ReKLl1g?ng)*nXanvc(bLA&A(BZ>TM4y}+CYg@3` z=QLav^P)U0y6tG=4G`~9>+=9~kd2O_D+dDwt0S}COB)O4ECV_UDQUL)Pt9RG{ww0^ zLnN5XpQ9)O0OxPteBM`4wl0~m^=r9Y&^Ck=ui*8aBy`&B#W8uWjt{@+|1r!m9mS#F zH$SJs{c!DZyvD_01DQ7CVPjYnzgDVMjOk4g<28b85UJXt-;nopzGB7JmG)V^W&9O= zr9Tz6$E^n8d-yc|6EXlPz9}KWzzwMgwS`&MFx01lT_N*c!F+XlZ5n3K0}K8L(FfkN zVua#hwhfDHn=8eCHf8~G#~We4u%|<~8mO%4U2u;=P+&ii?}{~8BFt@zLn}ZoZ(}*s zo>4gaa`xk%1eS=dwu>wp^@%$Pn)Z0spf*BlejRHg6NUqc?3YT+9kF)D&_%H z`|C^1)h}Gq{<1h2Il+AW?KqqgC1E?w$K;)(Urelcc&0iCc#4E`^$gEDp!Wrc&6jN+hvVNa zdG$nx_dI6wUv;HCupTcr^mYF z6dn5x@eA4~0`_A{yl$w^<8abAUU!K(viFH!MNVIrf3&CL9Y69WSigSvFH8wL_O|BK zW%D-sADMM4c^UdmydQ>Wi5wqMq|mimhacw0bzS@puPl4i(N^D_6VuefUv6@^0FupM zWsL^xWxCgX{^U|~z?AMOfjp3qpL?uABt2Q!{e1j~gKm?{oua;{3=fg=@pg$7@P)c3 z)qRjcGp;PfxEgZjoF)GeQOu-JKPB?wRbgeV_c$@bp7&*#%Kz-9cxR{04u>2h^z*+? zT9Gn>F_-Jh7C1NQzxqFXpS?`GcWUvZ&*$qo@b)zWhx_xF#h79q6ZWWhM~~rqbv~3W zJXvz=)7alU?hhK778H7C0h8vycqb@dwOMO1>D^zT%^NSSW{G!}c}=nIcGj6G;@zw% zvi@FWu%9e6bN4bH+{7UJJihT|Z-P|AWsQH~pX6P)%ql3!&-H&Wt3Eusd&OOaX(zPs zG;Vl)R=Nz;gERP(0#2!4;b3Fd;6~iZ)9r^{!G0cLxQ~R@RN`3D2|_BR4w(I&U;)xP zY*)~n5=fFU#bIs2cN2Nsd=IEI3{|g`+4zLe$@3Wzga!J;5!j$(98+?pa{A!(LNtXw z>`KpWjnlbpO^D?j^F3DYhc@W=7Fpr50KaIG8MFCyEyIAnqwtJdR7Pdkpy;0g?7Nq2 zs1#c7=+xf57O@Wh(qKPBvc8P7%{xrp^-~mLx3)#RZi#J(Y4OB85fkq7SOYi-DO=Nj zAenSQS4h(4+(~Pj16*)^9Ptc({8syR1s>o8l!56{`31I|K=r1}Y1)ZP`Huxa4Gkuef?}U>j_r*OCqCZ45cpvh(*sl$$d3bxV7Vw_z zRN5UegWYt0T_P1g9E9)kkPMnJ|Hv*HpK6CFlc0_y&?$*!Qv?iu|0eOCMSTkAOTWed z?!deE&(u*=2V1a)64E)0;mMv9wmD~nB0HiQ98Z_p&6K2here|Z+)$?5B*ydLQ>V$? zMfI|{a#kX;9=yM1`YYBC5;60kOUza@5)K%jcguT^zWgl?cAhv!SY8~=>8ChAu=PnX zawQ84o40YLCjE*>-SE;*T8=%vV& z20i6xDbEe}R*H_~ZI9etE;=xNYT5vYP}wx<4)Wo`A8!| zCekl9bN?+)vh`rak!}NpbIF=>>PJMn%|PbB+kx4L5hEhHXb)vV>GBl^lC{iFkM^Bmwbg=OJ*d+^gbhfnF-pUH6$X6PSn_x{ zpJK%RIq0Tlsk`Sdkn#k!Ieqt2&N6o5h-RLC;_V5#wNWm7l7kma-wnDJWWecMR{AwE z_3(8ZL3~pM&_s!eiL`Ku{wI|BYyH3YP`lFTLn)kPi~OU~i{#CG-jkAKjoOwZb^3v9 zxADTcN`gZ&_*2M}&l3)dC?3595$*YN?!9df2e_C0xW(4-Dl%`^HKFo2Fi`|@r-UQy z<^20^P!1j54d{xHy*k@hk7Y?y=Ov`o%an147Z|x{7gPR~VUdkM9RAf?!k^XNuqO!< z&TcG$#Pbusn>H{bRnfW|?tk8N>n~}#MJsRF;GOYCMknbt-@i+oIUaE5s}$B#Ey+E+ zLi20Qmzs|B%IiW@Ds?T@w6xvQD zD5zlbo>Lyw1XJF)JFT7coP(EWPxip+(M7*8{BdLUSGSLj99;(NViFPC1J>CoNeb|mv zJF-|r@#!BO*M$P=-tFE7jOQIJA(^02bRx^m%6jHEe6Ye4VQ%Eb%!5@gbAM)d*3w-@ zrgMKH*!I^Zk+2*%3xrxo$nbiqLc!yBodpY%nBf))O6=!g1G`veICA@cWr~x=LtBa? zEVv4vE5=uo=ym7C!{Qp{NSC8O0rrwrEXYq3`rg0N6Gb_`rOPqbV_j>Ph|#;5;~aTg|ck?FuLhW zsqh!kCMUx5;i9GCWJ!FP@z`KfP*Hw5;C|WO9H`qGMDT91m_w8X72c)DRw6~xNk!J4=)VSHwnwI9f z6ZRT0HMnfXAuDN!`v{%{D}9M4G$dobW$J5QH_Ikg4{5w8=BfM}5swNzf;%b(n(ibe z55B4Xgt{qC9`9YymiXT!hJSqi;EGZo6A#Yah^+WmOHgQcarF-HS5MF{xEV-LJIlVn zH&|{k7uNRf{Lo!_i@lGfDCp`YT20m)Xfof3JB1VdN%=n|g3c$Mrf2^jLC3$D7T`5k zB&VPdRZMXpQy+b@ohzT{-}`IOufBQZ0=goAL=;DlUZ$`%X|G%YoTHY>DJaFpi#q<( z<6aPxD0u#?=yuDZ2@(H_v-xao@Z%p*PW4WSh0_kbS{sZr=vF?BEL5CQ)7&2kHr@!w)Ln_X^N!o>eh;wRUyrX$ z{lQ-uiJd?7+_XG!Gy&S?%SoX z^cuPA%d=^VS#&uc!HX`ApEM^nI5bK_k*XQ*>|y!f=z=Wmy$px74}v5tnq_+${s!ZX z0N1erk&_m1+d?zy)9~M2B3`i;gtF#6UOIahvNeanE39$cTY7RE$Hi>-Oat|%z;={( z8#%(+8k2E0^8$Yi21MTt>%V3D@6QsyZCJ*W*M|zgqJV|KBjoU*%bl1?^-T7cT{bN4wHA z&pVEnZgQ5r%YS;?&_T0NvYgMc@~0)wNoLfzW^pIyihNe_YKB=}Y362gOWs8DK8VN~ z5owl{S2XF4TzdoE`D%eSa)Is*ua_BO-%ZACy(IFSaot2Hf_fe#aQXe}wF5IP1;yRU zB-v-HPoDEcENiq)QZgJ!<24H&j-Db9O`QK^IKR#erpele=xt3dYd+ z`Y&|$o^-%H`FMA=GZ23r=j+wwif%dHjJ|jCA_4)G>&|cwUe3TQPLYH+JiDmvG1zpf zWzyYEnsV?IHWRqrTj^HO4{||#(!2lPUTcPj$pY@O+t1k~N*d1{>MjdH%rPkP+<(83 z%mRtq71oTV;%=`}s8uj70AS63v$;s@$3CLbxl(>VR;XcKv+R|cKPc;@X8+}?2Cvfo zw^!AktEByOH3un^u0-8Id?-~nPG@WeujRM2c=ab-8h_!0E^$b zc>6J#*Qd?0rg5)}-yc`u#NNKjWh@#x9<_YiMXj1^Xx#}!cX-p@?_i3bC17@DVO~ju zb=<038kp72gF~hMp?cuA*xjg3YWJ@aD=&WOGU1v`%CzR7g000%zWij7SsXI^`+EIm z&#_N5w@zl0$p!fjqd;)fBbXaMM@~VbP(_0jFs`Mdr6qPBdbRhfsXfcQ8=$#uk5ya# z#2)zvlnt{QdX=&6I$T^~Hfl)gvQ_z8Ns#pbFr))}m0CEa?3JI(wIlcKVBV*n;XhKD zQ*-1(%x8ufN2G(fdj&E>vsa_+pMWk>#0qzgHKu&}#!B#C1&%*uj#J7s{=W55mq{@& zVsBvx`T{&2EDf5ui=>mJUsHO|$4LOeM0;D6#6xlRuVDm{`K&`|vY%E&VB*(=W`yY6 zWI>pL74w*oSIUTzS-6E5guc)*~8wr zQTsVQLI$segeG)cKswyz;WU%`qe|={C*UULhvh19hmm|TlcQO0bjFV$*?=5_rJ+uV zw`a7Jh~<{@on|@%&$aL;uS^-&Qi%gFNuxlT!*df~eKn4?`y1gW zP>{uk$nj<*5`}fXl>R%8dJ~Pmvua3!-qE@0pb);1EnmmJcJbyl49NFe+Zd0| z2K&tngVzS?QW`;r;BGuNAiu{v!5M#XUzm!yj7$miD;z&ILK481$4(NSS;ifCFF`f0 zS`?Hz+cphaFe!c$_~9Dmzw(N|gGVitVtWVTA)|Z_JPi}o@Z}6UH z1PmmK{Q3f}_hWxaxv}YQ?HA<@n$d!De=+t=g~@mIY;hT{)(|FH0h!yq^L({DRiiD{5X za|iKE6IHh6FP>SweNUs6e&rUvt{Y|e2QUl>qPV{)?fhR{7@Q5|glbO6c^4tNCK@5! zfE9VJbrL$j!-8(MaLO!A@`_A6l)Tc?T3tf7_vY>)`4XET45il>F3|70V0@6C7x9%u z+oBz%IAEomSgtsFnn}6D#Y0opv~}p)tNvBGr^#Ebs*@z|1v!E6X>E&?_e7`|Z~Q|* zT$#B7jBF5Db=RkXKEY_*kbAinM+_z#OwMbZmcyJ@ql=$G2e2sxPBpqXLbv=U={Axk ztx{Iqf;p||1q);3%G3AgOXlmwzeQI_29^gne}y~|d6{CpTPC&Olsx-*>yy(+k68pz zh?dk}^_VgIRv=tZpg;0#n)*6a4OV7!U#EQ9oU{7vQ*{qK&~*3BpQ;<+;|YJqmtXGq z>f_YXu*&??Ohr^aWD;>~MQ^-vVmuroa>1e+GF(CqprcoZeYh!Sp&wxiLVrmecL*$eymx>p^s7&Uc0r+jvLi74f z;ktC%_q$(C=-i*QS^2jtyW|nwi^;kFv6s8Rao@eEumY<|s!ucMP5v`_YvfHBbkdc^ z_wewAaf-CDeIz8x$pN~cyjEI`R;BDhk@x8`ku&Jn$R_cJXE54?&(WPdsTF1}{3o6U z{Rs)fqfH|{GiFU3l@xdGJ`_nqM`0}(*qummJM;P7LfE&s;#$Q-n?j5wzj+};B&@;V z%G7Altc~McuICRO#^n0ppvd*B6y~YEucqXKHcg4ZMTDs)dIQS7^8u5BMwgyKE_kQ0 zj#v^GJEPUH`SqOq^Hbwr^5(s^F{3`#KcKq*mm$| zYFAs8R^TuSD1vAYdr<(wR^1gv6S*|nB{@F@=tAR~sm9&eTi_oK`8$RCnm@${q;mO&(m;=e|N=8r1rQVptFI=m`=21><zWRWZioa;IyWG}}}_PLM_^yYH(bMod`oqf^9sShq>h-Dtg zz~UE713snjNy!FuycC!%iR6xNX7)B(N4u2~2vM_pT9g&G!@$hi#4t@c9(eS1s9!S< z%I{6$6t`7I!TQ%$8cX9m6Uy?CGy8kH|B~6)27C8oHSA3tYxe#TUUkt@F2P599SI+) zK%J+lAubw~;8AymVrUWsfa-TULRt#?IIb@A#(nQ?w_N1{tv_`ya z#bpt-cY_geV%z4cF$vZC>~KIFE@3^0t_+pGjy>qz|6 zSdexJFMcnC!(1GiYtJ-vsx5P^JkD*0Z@2d6moGppXXz;kZ86i8*m>n);t;E5ER_F> zII~9+ah?LL5FUJim1Hw^5#_r&>nO(l%YN$F&?h{QeBj6WoDmC*? z%Q{p3y(n~IA)5f`aL#Zq`%GA@(=KJFQg$+}w}b;3Yt%X?zx^iNuG;e^tE~{*=*6Ys zUUw?~??KwE$*fytOo)yG^R9{_U97XeZ$9=K``uJK!T|OkEoqC z*0+)bK;#u?`{Ss=`-O?DOG-9C;Rn8GUq~|#bx~)Ui?*h<)-;nie{*JvAGPoiyveBn z&q{qP-<&_V3mI_oyo#e)Av8K*4ySXqghXpp&VA(3svJMl!TG47mrmT)kmTQ*c?Y## z+E{{cBgf(|P5s+E8MC2#0jksIiSW=vyU#2_`2x+A;CW?-T*|$kS0pa7XpRuByYggD zd(^Q7TBfCXoqCdMoa1}h%vpepLelDtbZ2Sq#<13#D*k4L5H&A-!;x>BdmjicriF|q zpyL^qRT8l#I&z=!Q2y9L@FZ5z=j@OIjuxN(Z6E>g-jL&tiKZN+Y{OzpipErKR{Vh^ zs1))rgk31jPS|{bzr#P~rFT*`Ew1Ck$08_T?vbiQ3|R zg=CSTKj)@DH;6EaUY%%YnqD~86z=Z`D&nWUgQMI6 z=vbCz4rC>A9vAR6e|Q>etb6aP%{TKmIZwu9MQx>hg3=B6CDBfKC@Okd3iq30eifXl z1{DbA=`lr<&`*Rxbk2vZQl%zoh;jKzv%Pb5ga}rq1+$81Q^XJk@B8Q(Mt}K z4l&1dhT(zSS|*s9cyo-EywfKC5^MCjQ*G@UNN35$_l|f|GTNeqkO`2h>Bny=*m>Ql z2xHO{!ReQt?Ux;}bNkJw%hUa?W`E*07E`qM!rf<)-<|<$*uL%>{i?6Lr7sLa40#Av zPER)iNmgmGeQ0}WL!%m7Q)pP1er~al)LvFC`u@HShaZK1w8bV-DlIl5S(;rOW5tl0 zRPT+~)Z{m5YO9dt7i-ygcP59w4vwZV^Up&O=36em3_rBaw+RS_In=%OFr1LtFYlE< zj=Dq*Nx@*!c)_Akd)!s`Mk=O1lT21l`kPnSnTF2_yC%kB*7WOQEp1J6snT*Qr!=}3 zxAM{{&OM_(T5wE_3&97U5;;HwoWg$WKN-CCQ<;I&>Lef1B7zZbqjqXfK3<8crfT%Y zYuO~68k*>2lkEA7{BSVE_b6C$-EAQUy_kw?2^{{;(?Hi}e+IbOOgG52gf`G2+y{FU z?{TQ#JS?C_;U}l0cm6Klz2w$oDkU`( zNMUxN@2dX!A*TiBNfsjBJr&>{jevHPsOGf(y z_w;j@aK;-FGWBM&ES@ZM1Q}J4A<9;WQWJAJ=ffd+W4GRAl^2nFgsct{MG)KU=EpB8%W$`}0eMBD?szE9NKsm#3= zxm3Q9Kjqa+fs~7NY%U!zem5xd^(=$Q1CuZFJn zTd0nB?S_XXZAs@BX;^C!{e-T_ya$F|nA1^S%W?RBgnQFg&o0?CtXfC9552nxjx#Xb zjD}Y8V+n1sD^};O+tZesX-rBBfo6gbc#L{?cF3cQ)6hf9Lo<=wfVD|UmV@<;kCBJ! zKFZz!ViF8k73jMgAKrw(MKw`Ads2*=Ys$+wR<%_Jww(+gzv?_{rZ@{&?~jNIi+_uE zoOT$rC$rdDINmxf~s@|;voib4! zV^;`IS($NXc# znc|`pm%1sMmB8*|L`lRB+m--}LQHD(OcYC0T)bkWVocD4R?IF%=jw+J#-1(WDi|CG zd>H0Y%x$1pKuhrwWLdnPrk0oP{}Ab!sNCC?CeqNSEl9m`3UPZ1AEN;MJ%= zv<)Dl^^_AQ<;!T>#z~ePVPEcL?Y;|ujg6_)%S|XUSlFzJ5(O3>QUG`w&_?}PQbcL? zO)NpQm80Ug{Pl)IX^c%YcSO%%b&ONsVHu~YY-0B{+^PPMr}OPx<26YPOqNyVkzAq7 ziL_C?K+R?v5%V;o0V7n0f6KxmwYFGea@p%G70lL!9aU;%VdAFX-D)>)en|TdFTwa5 z8iQwB7jBaDHe1OxEWEkD1cF9aue_=FM@E2J2` zRd*TQ%kQd1S4=7-b2jK#$Gaab3c95woU+9Qq0UFm<|}9?St&&y0J#HN%8bQ+^Nr8V zv8oFjcD0Q~4%0TXGO2JuFxk=ms7670<4rP!jU~P5S$>WcSZ2p7u!%Rd!u(KcOFVYZ zL2px?KU;EANjaH>j1-ntxAQhTGa3itGQQp2PtC6qwZdK+M;bGt^m^S*-27eCVRH=c zgR03AniU!TEeFp_&MrE_s5jjVn}*{MfZNk09WRrw_#M%!Sk!9xlmlkGTikflvVnqpcK) zOU-IQz{MKjd6&z*!=3%+8QLW!QQjit1&fa=W%uSMce#BvdYNv=+Eerpzvu)8(p7L( z$lFyNOvjmJpW-rOVsY#8+sYRDe~2k2{9zPtD)+r^(+FgM2N;d`k}wp0zlVFV=;USm zvL|4xx)9^-XcCW7LA52bTI|+AuTnLYC}`^?6r)06u9iw*ejy#uGqjGt27g zuv3G6eve$3>o5i`FHqL%Q|KY?k}E@8N;A1!Klm(wyJ|wyI=*8E(y4UmG_ek`4GuA@ zuyt80p{r3`l^if)r^f+mGsj?M}VnL($Jk_0ceb@iHn>ht6p)~62hfh213yTUJn zME@jWkt^7MI^`VN&jVRg=hPOwUk6bL0cO6dXZd8CDqM)t;0Ys$E<_a~15t6%ujys# z4%3R{ZqR47X%VK~OAe|LF9^Z6XL*-xUSR+|kC4}^GH@9>4GgpSIyqH*aBMn!l?4xV zDtLE*`#xB1^g_g?+$n^!;Sl_XLFO7cACv;zk!^y%_O{7KaO3+|RVf+J!O5af^4&VP zFF0RiE_@u)L(cNCTB-}bC@5#x2vMtMg?4f_``tW5(js!RY$YI?dN<9%M!*nrA-kdE z1SIq6l?=l|k9(Q4FIP4Z+@<#=i|x?&9IIE;0=8Sqk4n2SFbVn9SYq3%mQ1$v^kji( zE0+PzJ*V!aHnnRwKj_uKo5yO@ulm^kLz)e4>pWr$C(Q@+Zn(gC!UsUzy)KHRw$M0r zJf_dJoZma~iQJ>j^fI2|INYpRIZv4mx$VJg)taI%H+=KLj$a#AJa{rw-#Gn7SorZS zsZ@A+n1Yix+*hrKt0(98i91)DNFH4`#RX^Ao{g2_SGGfI;Z^X$A$m(!uXW=oyj&i% z^3f8rl;>LO^rEP(w4}FY4#&(eSFV z$l#d<1e3g~uA=6@Te>G2P&B=l?`J=l?1A(cNjJ$(WJk5DjY*f(1vdi^VB_*_Ss!qX z3{QpBF8_w%H^jmfyVcbUTb5~QuA`+_8z#x(31mKXV0FW3Q(0|qo>1qluMuT7`uqTg zpZv8y8{@J|S^~?V=8+Za@`J@L%e~PBTlM;OE)kbYTDis32i4QZ=dLSyHKcFSWs+@a6goctW;GRNDw9ZJ;8K4o0B>Iw%a0z}J{|f| zi>Y{JC%CaRrm7_h>OIECmqdR8JMvv&3?ub=MO~xk6yqxq$1&RxLXNRzx zFf2JKt8Pd-gHHmaZ=w)AeTmjtr80w3=yAixdQpH`YjE+7tn;KK!+>kg#jx==FR3PL zT%b&DTUQ#jc*edL9Gg2T=z+vq&}^?NwQ=Sh!-p@v-1sd#F7^i zPCe_-V322{-|~%0%t(m!8E&I`0zf49;aJ`N1Ed0y3u)6C78sBvn8lf-;oB0Kyh@D} z>QsDcTNAOr`ABxIu~aTx+GqGH;q%pKtA4pc#hIk@{uIdJ8&}2#jmX(w(z`mJTbKn| zLW7S92S%ztIr$%E9Qr#&gE6NhL>r@zVrxaW(4{))5n9c<`1l@!Ds1D35%Awsw+94S zwp(Do*jOEPWMp*>$CK#&I$Uzz{T$mfs=8?yvv;Tobg2u|4AJY29%yFRa+q<>_M~!j zSNC}BAqJ666a@{T3WH@XUAsA7!UUh~!B0r;dMEh!B^SM=P*EPQW1XZ0w0Hu+z!EcV zu{o-d;qo(=#^g-8Va3h?h}>0? zO}HH?>i{8FOLLcEtUDDeTUa#2#OJA*LZ{No6kQYDk6uf~ZfO^`v3ZBmcMl{=b#C?X zfuoM7u4ctxuOb{>wWfG*t1F4tR|<^Z^H2n^$YM8_HnXZ6XIv^$ri%)v_mKEMuN(w7 zRAVwmo1Q&$ifDaVu<-+PL`vhf?D(AG zLwhz(5grC;g75$fin21on~tkEp~Y>q6sYzFQo_PWO6?BVQ< z?BHxo>v5Sq5H;DH3`!ni5WQzVZAw(HoxV`InDgcIW*3;@}zcftKz5-xZ&~>pam)cOL)){HTqFj(P8BJ1G$=2p3hxB=N{q znGTtLnZBJ-&L#!}x~ta=mhCa~SW!kthh#JhBiiDO|uxh>`jfbvHsjdem>aM0V-k^?4S5UC(*GzX=imj^9Pl9N5RZ|{TuJaM>kJd3y2Gr|R>lw0^7dh(F zJpvPS^SsCpFlF;)y9Ciaqo~H-3K+vebOPVQ6QQWC|%KE9Pp1wBKub<=T0_oluIWO>FV z=}Y5&>c3pgKnp5O9Ln{_*fxZ z;o!&~1ZjC$8NVvOQh?vxsW|93=0_TBo09uyM~?k6qjJ~9X6+G1S z-G%J60t}FQn-vc#>(Z3!faeMshp@bdzQYVSGW_7BxjWK2U#m!Mak7tZgb^s(pHSiH zF534;=@uCIe)13$Q;;2-^B#(~gfV`VbFq-cP8uivwjX=c)o-j7Jsv?rW*QLxuoQLguwI{D)E&>s}dWa4B^ZE?Nb4jG(XW!h86AJph>=9FR&r}Us*tPst5aNw6Eo!$N~^OG2$IWUooPZg1lT%q2ofv-b_w2m%Xqty|n^yCnBF8)vYa?e=5JK0reVIV5G5M z-SeQHtl@~TkKyccZA&UUP_?YR%JIt*E0v{ejodTTdQoE?jrv>w9=ufd4Ddi|kQmXX zFr_dy(VJku4`^lTx@?KM*}MM8`KvTh`K0VQ`fi;LrF_uGKZyv1)STPVsTXpGNU((m zXFI9%(wo`RM2O0@@kS|ZG81y@>s0RKEFg9PI+XfTAow>zPqhVn;9%>pl$dGjMR)zH z!)Ah%a@>_pAf^$fIW`^D(Mb09mB4y2{j*-jG?oWX`?&k-pdwD|d>iMpAk-D*T-Vsk zA3N0_?+~a#W`CF*Op{6;jQ`#rr}Lf~&q(Xskv$qQdczZ8SNrw;hz!cL>YE1{JCK7_ z-_^X;6E`GLr>hz>y*0HpGYNyckP(FmcAji&DL_9a*d_3Kaxuf38~Yb!nMw?JyaKA< z#s9*Ra%jqGFh`>`6K}&%MwLw~W;KPgE6mv}{F+hc-Dx)ws~gd+9r|f%Z^`rYAtlYv zAR4rRDp#s^nxo#Wgk3~yK&??u7-POI@iW$WGwJkvp35QS#jHzTIqE{HKv5DDDBQoF z0#(9f3+TJZGrEBkpLZ_Asu zi?IRfIbx#AaM$1cZ-saQw)N9jK|ljMQRS2mg#7^Z7Hep9P$J9<&`A`g^@G{$-QYs4 zlApq;t#!!lc}J?Z*2h1|1Vc(D6%KhjF6qI{_6gjVO5X3~!>=jl9~iDi2jLowGd?Gh0ddl`QA8-YNGKjl7rcv=Y$*{jp>CoumThgGvlJ}(rx z)P>}uU<2R_qO}0;46 zB{Qy%mz}f!92Wgr=8?3LoV#g2cPxS{3ifbJy6!r<<@aqdPoP|NG)^O-H?1(%*t-bmS zvP*;bj>jGeAPJBg_`!P$0htc_EctG^jFznQev@C#n$I?p8zr7}jDD-=q#=MJxleCe z%K^k^#41&bX1)0VE~g>qj>g8y>4HjbE>E^4EJ+eJ~d8sm9fj z-q>zBL9MxnBDqpQb2yf4l`20Yyr z%TJzmD~6LC4a?sNkn0tpFR^`iDub`*h63$UIiH==C(2ywpkn0>VB0dC091qa7KcZn zIcxo26us#;lpwuT^_bhi1@5rWL^o+ih3uufCpQjYO2C zOWg_R9w2t~c>E`N?8W^ms*y~27Wi>1+r=ghVsI88tr?53B1Svsk-I7=X6Md)+F*Ud z5S{MOfVtS@V z_4>Vy9$Y2N+b5Gl#}9+P8YHUe^2&0`PQnj*M=wuF>HuHsnN?HuP4y&534Qvtq8F*s zup8#zpg0=(spIiu^Cw7e)xsMse8Je_g3Je5Ja5yln}syb)8UnPH~OiP9Rae}Hmf(8 zA4&2^fSt2sJS?oe86T-ct@C^=kD+x_o$MrqGB52Xd#<)8tXqIq_~bTXHmiQqm0`^o z3D_YyR3~l@jnr#b$LLjQiwJ*?UKKfnIe%12fPlYS)d>1&fBrxA>+r>0qgJEH#)s>l zFPRR&)Xujrh~?fy(By(~$Fo=3(kt8F2t*e?*?tf=oBQk+_rE@kSB z+^yB3@_T{}<7GQ5+V=c{CTY6tgMn(?jH{Q#tUp5|#dJE0&wOMppoVZ4A%4rCv3A*| z!+1QIBF^hKtA`ycStlrKu`0T`Cxq(V7b8aqy8Jw@PQl_Q;S2*?Q=H5dIc01{Sb<#! zZ(C_)c4U2{Yxrf&H9Ne}mr2t|MweN4BzspA4!nva{RcYQGdP?5Yr}H02Ec*RfRg5v zyeiSg$ywZHA&C;Jj6J3)kK+E+p$}1#LZElblhw4Fbl^3{+lVSbHSkq{lKr;JJP*tv zzHR3t`dwXFep+eL4yI-vhQ85irUnHZ9rx&YM=A_-s{^$B)QCETGKGb@m#2>;=CS;X*@8t|sy46M!b2c#iEneKq1puTzVqN1 z(s=}&z~0_!FQTVsOqL!4!WaS*cy6-WlyAWvtF(opFL_hI1e7C z&4yC3W>`~_&|FD5G!lxo6S{3FLIc)+nKyH2yg&c8uQEXh_ZCm2793YkZ$U(M&-xS$ zrCmot~_H@XtM3{Qg55O4XZYW&OVUisb0#;_~ zZc`|ejs3k59o8O(4kWoU8p3`t&J~}}lg$^21Q;CCdnYwgyWEg)byu{mi6dtP?4T1h) z5%lDt$L(mN!p?x`EM9??}$O*8OlM&eL6NHIDSP9PL`WX=&xX>S1t|PvaUM-vgq>&XZ z(QS=xrnI3g>Qxn3jQf(E(2GYdsayG|ik6sUmXTwwVxANe=C%=tiAF^{AiLV#<-^lj zVWSv*Q%2gL4B8|>r-{XVo~;)pR|+2~dB`-Wi_~zbdLW0CLRcbxs)AsHk(fd1Bb(q=F_i;h|Gq(sK7XNe$lRHC4Lm;v>e6FCQ#YkK z`YA%VC)_91Hl5YP^Q+6$4b~^{I^VeQZS$0uHBNJ63II2QFVk-#0gZYwjJ~f$OU`kG zD^lZ*Y1{Utl}eQu$*UJfi2FL&_(JmMO-p;1RWW9(biRWnuz0p0EGt%~h~_{`*}Rs( z{zVjnm*{_oVXsiGAD_4$*Ev{dOYcp}G|!iQIpZH=KpaQiXeBB=cVg_SU;4R6bR3w7>K2-LKt%BOLB*eZf?K?H-!rb6M7jn(pgF!m+Ys$U9!r0R zl1%KqGi&=)gsO6CH6cWh%~%lutW@hW)iGbl$Q(Qncx&M z)D@?iwc*^%={V^{dqu2@*o%wvF+!^sm`#Z`y}iP$pGa~@YBu^Q($cNJmiiR5yS8RC zm(doMcs769PjadKnoC&K^en}U^kjo-bx$~1hno((t1TCJ#sOO)onNHZPARNmv71GG z9J7&TVew1U>X*JM6QwoMdamvoeZKV_P1{05vrnT+f4Hx_I@E5aS!Vvpxol^?p7`|V z$iWVJeo6mZ{LdiR4YHfAY33SSFF*Mbt`1tcpm}^d&5HRI4*EkDBk4fL2(ll}m$Y6H zCMNSadF{*gv1;$W%?@ud*IkQyN0`j2)($4aaA5dWvD<3 z3W=$fKH>k|&?7(_MR?{zK5i4WozNJ6WhN|HH^pj7b>@O+YY0g81c!!$3;;?scFs=0 z?D~_BVS}j`?uVWfy&!zLRe!Br7DTfWTxc^f|E9x~=%VatQukMag!|Cu9}Tr7m{U8^ z3%-F@nHik?)NO;90BQC#12a1gcT<>Ae2FxOb!@PdcJV2L!^);}c}rhj(9%VAT#EAr zCOOn{IxC>ReRNTKuCcmmk4cm97dio8*BbP~)u1aeE8BT&tk1HWn!CeW&yipel9ON3 zx3587_Z=RnHbD^meZ-H#3H1jhU=(I-1=ws&Qp^bz;XGokNLsZkkDdo>#GjaSvs$$i zXDw&9gkBO5xHQ?V)-H;Rbh#wa^_r^aYkxOjr?!5G@yLfXJgc|xj-v;l{X8<{|1|cc zfl&2t|2NS>NoA>IN%l3_vV<1fO!h6?NVX;o*~Su1Dk&3X46;sPtl4+QP?E-4qwEq5 zV=!dKHrD^?cQ4QVd!9FcuXr)eoawK@za((amZC`3{C?T>>N;K5n?j;xWXRAf% zD7vOEyTZ2c5VtfrLLqZ1EDc$Y(U2U^I)%#@>x-MldY1xCi?g_yIh}Orz?@LX>o8ZO}lddgA=O( zr0Ch!+HVbmWL2Ee&YKd~^!3m)q>8_4^gnvhC>D|P;eTxE@r!)sit@$`n~A+mvivk) zG2|ib2UDr3^5m4JX*-xrnn97Kfo)mOWqSNWhKLu^SAwY%79)*n#l<2mLLI#Ydl*Py#RGupLpqhX4zBhvn5D=#rB9jX1g;<^!5Ul(5- z5A##-G4X#fm{==(#=sY+FY$hcwERpX%th3U{@IKtSIH#L`wl^Do*M1e=hC9tS38F3 zu?7yn3DNP)%~{F&S&8979>XQE^=2dXbdABz?!_Q#Dq#EUmfwrp6CMgeoT#M%XP8ve z?COL(Ey&gLD2W#yA0E>U(bWuLXj2xvFZ5<2vS;s}V z`DXeTOKaMueW=UNpM);o6M_edem7HkE8?7TaEUz0u<-DKrj?vw4@;=>ag+vRewA{t zk1i`&aCm$`m7nu??k@SU(lTwh;(|dMw?3?dCSaMI_%U8kxzPwx)sx*?OcHf`92zxP z+l}eD52TZKVdAH>TutSYjtL*lvN?byexWZPmfmtbY?+`E zEi!xKOQaoSe=7mX`sA3Ku3ejl(~ggJPr#b;9((($-%z>w%JRPy++e3ClK5cn@<);L zm_yIwG_j!BuHcsq&ZXu-K;aZY6d}JOo7{`q0O1uGt`;pQb@aoz>Xuj4u`B5}>ajB?&oR0DU6#=MT@CcmAf)Xq%zFB>2P>=^aG<}p7*l)3wO~!G zl;xS!dZnHckA7Hkf)?06d3*$2bA;2S1NPJ zcF!(qx!#>jGO>!!({83m3f`3x$6EVolo{JW;A#L*h^2#jR5_W|7f2jqq%m$_M8Kn& zFm7HpY2sU$t6^;7Sv&oiM`Out(|-DxsC4*M|#3rF>) z#Ohv}Y$|P*$_(U_D)8zKmK;s1XQ;_Vn(E?WNx)TtxheiZOyIq`>S(h5*A3bG3-?0J z!}MYu@3r17Yc0NYEa1+BnVPd0;Gw|}dq!F+pyhmtOf?Bfy8fwOY~9>&U~NPyyP!?H zIHR$x?P83dqGRMI{!Z!imVRnmzZu-j^Drq~c~CI7As2wS?!xPE%uQQX~SJCMJ&XW)F96w9=G0bxNzW34h~DG;&@j*ly$+*4CoI3sQU1UJ957>&WD-|Db$ zii~Xwt%7;+(s@nzN%ytN$d$C~kg5sU&&$H5sP04e&6oWQ4KFa2I0_tUyG~0EA5$sD zcpA!np7!NKt+1wM3bZ^2H{0VgFh=6EK+M&XVy(8L^kh)#qk;5Xdt3ZgFpNX_3P`NJ@ z+d*78^A{NXw^aXup74oZ@7eZ8y)hNJC_WIU*Ih`PBjMOvH+m7HGl_eqp%72Z8tS{h zh2D)n{!nB5KBKuu&|`jM=1svz@zVv&(t_@3Ko)bUiavDeylI#-q>7Q#O1qT+dz)_9 zD8|P(v1+BT8yCfXD{9nWi@*Inx{09teTIEZ^R4k>D2*}2U`O*qj`l*w-fc)wQ2gPD z<`w(81&oAAq^0aggs-v@2(Em|6l-xfPhAKvYm?Smms*%WW6#_Tir_G-zf>GZK)L|Z z&0@oaAu(h-Sg6tj#V28NUcu}eUu|Q@C2J=yY~-6OloYN+aAtY{wCR$U3Ct?2v`XdR zKLpZW@9r?vgQdt1BImyz3Q<}_z?%<#crvmHe;~@m-%>m60avDWfPMAAJXcCgzDhu& zjA??D_l$%bvLqoD5Y@V+svqj7jY&wRpJpdWA><>pAU7$fj$f6t7aJ7V0O`$c6Rs}h@y0^;pNCsv!wR`yZ-3P<@)sw=r8xY^k1w-x^A0*WWJEzsuJDFa;~ko%w}+qa-gvWaYA?WtroT;SZ{~!7bCR|0vwd zxK9?_YM-hhA4E)`J)gxh)Bvl8{Njo$Ok$KViDa3j5!VlUl~{heF+_=B;L+p)uPtJA zu3I{|o;mBWa9P}E6{3_v@t;UYPU#$KfGi-VFGcx1A=Pg;h@}fUm7H;}lKt*>Izt|M zXJVBsFVy=+MbQX3BPUzqZQ`#u0@MQ1Z?+b-28xH&F+}D%@skshpr+x1Ke<%9tCmv& ze`ly4s*}ujDmsbgCN)=OF>Dv}-~;mI%;-c8p)_OGEupP-ed5$JUGAiv=I0xg@4#2i z2sRjTl=c>U>-c0wIG9Z{wHrIx`Kx=9dP7Y=8T-a$^dnv+DqQ1t9VM#*|9_i0^8+ke z5@56RQY``hVDE|bhiwNX#qF)qm7v`G+?ZULT8lr0mB?wem2|sci`xFqcCAwMXyU?y zO@j4Td_i3?xrxNcVDV|{nz)|k=wL^;lCxhc62*UN21>|UEp^FvLpie}FMSCX*G+wi z%x``OwND&Fsgac-w_{Q}#=DykCcP% zEgPL1Wpm0(9!Bg`yWI&TJ>1QY*}F`l?qo#tTW`vG#8Ox{b9DGH4G9%Cfv{ zXvLkkVTO4;+?{*3Ax=rJ8@o6e=lb3V_+ z9GA*%vTrz9)E+l+wu&7P_NE6i=33=Ge%v=41ImMVfJfJFo(<~d6rBu$3M4Ho{H(xF z$i1qYeQ)w5aQO~O;a5h<^~E}u{26wN+^ESh!U-lHHB!27GFvPWs}t(zsO8L+grCju z^p^VNN~B56z4h0ygVq_hB)m^NJknGBOH)y+e=KFM>$(Qw!zKs)Vb17mzORG4rb5An z3q%WI4-AN{ztv2fjFD~KS-!EU>JR`a>Heu=;Ge}Wt`$#u+Zi6Dk#20#ol<^pN%BkO z=EkA+(5RuGbYo(~-DxMzh(OIcra2rb4YpZ7uwwbEp|u4VuN|Fc^Eo+kK|xSXC_te z@cv*-EFpYz&if^&S*b6)J2S`NywuJqy|yB;1Td#a&;&dzj$hIuX%p6!-$oGGhJiz92_C-<1suCVIu9=GOANX-i=#j z`7NB`0iJNMD4qqxSxVT<1W(YoO+6<4J!NB8E4B#&_}`C#mVlWXmooSvJk^nR+dg((df|RlNZpNn zYVjoSQ9GTSc61t5#;NDv-`w@RWXF|Aua_pO47^~_n&zUnB7UeeOAFuAvmQ{isw~Yd zPr+^F9uUXg_gjQ-A!|ntC3prU4=!q$57rM3ODa3sR9eZ#(cC!pYTdmmdlQb`)Zo0UZBP?n23!hDnA=GnPXUNqf|1p117g*%Fsy*bz z*G|fd_5gC1q6&j0PQUNK9Pdu-?-%UQ|-!ub2tlh=0|Ko#^P0y3-0Fc&a1@D zF=#A(AyH7V*eEC03lt0Ly1E}QwqY;{+11CNR6kZRrJvM2`_03|$!hM7z;q%D%W3(O zXttuUX&tf4s-NMyN0dw(jnl5mxNXT+V6#Ch=sU0~U`BgrB5Q@|+JyoM?*xF;qM+>V zY&m=EP9A9pPJhM)P*3#ksV)!&tjKPv)p{B+clx@_lYcJu_qV|O)(sw(C;bMr ze}A#hCaiYXl9p~%9y^z`D~ZP5(@;*XTcqF84@LxYZ+%sLW&I~x`{(Bqud{F~^c?%4 z?v&=1Zb#>U{EA&ktA@j-q`vqS>d3d3EYhbug1rIo)AGq*Il<*8pmO9msN5DA zW7YR9RPlxQLL(hK0^n-WK8(4W=cr=vhbxVNFEO4$DJiyw`EC0v%rZTVBEY6c0Qh`9 z3TH#WCfAowhDqt9x8ByzR2ZgCRg}YCE+%K8;srfBx)pPyakjR>p(S&3)??tavRWuO z@IO)i>psn&Jpe-Z{to`@i&4iQbL`vdxn3x*a~}KH_$*Q*)7%ch{fI5!=mmj5?r62V zvO&<{l>E6SAMk;cUd6FgS+GD@R8)_>BINl??FZQ;ov2ga9;@aK=Te#`&N%!LbryP) zcQsszN<0gFt*iB2cF=sr4CobhIPxEv_<23HWLp;rV}34k>?{Y{W(GU@ZL5x2eTFpxdNZOX{0jkb>4-1-j{rA^#`CGkht7^me&%FU z@{fGpNqEi9J$sm?x32eI9^e1T9zai;HAc1udnHw>#8$>DathX7`b)0%v?mSm3S2st#Z)T)93IN0JXv*p)Zw?*UuK_3R}A( zpxKkT8TP2v;m$`6l019*&pukDdT(8%@a2)C|E+)pvy40HkyuM)_tFu?TTt1AewQ0! ze*54s)LgK@6letU0D8>I>YwxDlB_N$=9PDd=WB?FXNepPOw>!TyAo6t@io)*_2}T| z&oVP+BJXPX4{aO9g=O;#7^MIErhiA8m8E6iAehKuiLG~=hu?PFGJ|dT;^1A~n38id z7AwmnEm9WR#SvY-%(3wr)M8hY?OwTZKLGwL1uD>+xv@awu#A-8f3Fd}z29rJ#m%tm z09LO>!zsL@7^`_GEN_ZC;xcmv6REJ%tbc0Ha_lVWIU2kF{omi}c6{rCOqc|a>Dj~Nw}`Wl5pC(U@j2(-p!S$1_>M1Iux)qVe%L! z*jP4dyet380*%#caBg&<7dM0vZLQssF+%9rZIQB3;1QW%KWOA3|NI&DQ>a36s4Z

    >(Ud#RfXt*tBcTbyR#1g`U0OgQPJy$@yWm7>8WA-_cmm$j>NEk@Mmxor7B^$yCGfBfeewGZ0^uAMlK%GY@k&@Ww z)-_mcdP8!AyZ0SFK((YLs0W*kI7m$+z&4coD38lhIE@HuE(%PBM9+3=GH*(%QICXF}T^NAJK>1$0*<_AAn|DlQ>N)(c=L>rK-C1F0YI539 zY5Luq6jQ)B;RB>90JA%US%G2v?fLUQXnf*aiC4_l$NT}*c1+R#Fr1nwo7zeALV_g*JWB1BO{Xf5oB}Uss4xDUSJb*)prSnHH&WI(RFaY=l zkTYdu37iX6A{cwu9hn@H3FDfPN&JTiRwAqakqfMjzpetEVZ~B?YtrzxsmlXb#$la| zX`?!W>1{bazA1dBMXY1xLYXujw4^|rjG@N6mp9{Z=Wsx|UQOLvdbtc&ig4jsp>$`6 z%BLN_tcj#gC(KUQ?_7&&!{vxqX*iBJrPw!g z-v}1&Yz)wERZ*~g9o}Lcdlrp`X(GHz*mMGPJk%@^Ur_tpJwHqEVvO-#!kH!gxdxDL zvyL+DkMr~2b6)>R(jlSIeJ_~eRa+J0`TV|OL7c?fc(uo&)+=Yaq5PY}AGW1O2P8`Q z&wYD$Kso^(*2m( z*dA6f*~!{2{90v~vWr49ay2Y?w3Eimat3j!`5c-4+`PkaHDope~%SKin*X;YKuPilAx5#2Z$j>B?jNuQi zdQss-P6zl*(^sIU3o&L(pQ@*HRFDo^rtMshG6f8N3tg(u`T9H#6mqL;p!zMwFmT2a zTF)x@Z(r)2c5cHdvx`9h-X{i8%HMSRR`>i2mG7!Xx5zl3 ztf0+o@kEn#&Z4KgtBk!e>|A` zw(n{{uuG&IZ;d_XGxpR`-kz66{CvCP6w@0&lrXy3`9?y}EquYg|9-pu!3I2$VwsrU zd`xElq7iU%!)HwZ#}renKXb08630Vj=cYib%0tZE-z=Wt{ZZMe>6Fp2<7m^VJW(^h z(nJ=2%e}Pej64;C>euX}gFsRLnckn#vod@iKX_=%$){RWof~B&e%j(X4xO;wLb6lmfN7?s@#@0r9HuS7kt3asq!C>W&fxL%<|^vqt8i^ z&3Yqm#XPCrivsAr3hhz5Lt}(_WPE3|??VMIhS$&4N7E}Ys@(6xhRb$7? zM~GYl&8@@qw3S>HQE?sl+rjLOipWUYinb&ob1-q?;6;Ze8kfZ&(EyNUl{gx3ykn-1?B4VN{1#BYgzb&OBKheUAU{o0A=p7XEb zn`{H7E`Pt0IbSe$FVra%C1=zS-z;9YZZ7=wE;z_Tz4(aFi%co~9?l-vSY+<)C)XO! zOU%@|iz?7Qb56q!257Y;CyR(LV5D}qr2WoH{h6|vXDBdTm$X|`O+PO6QerO4)Yl|R z^iBHblIj;Gc;xzt_*Q28%5w}dKF}=#0DatJFXVAP!^EQce~QN-`FH7PEaTUG@ygfc z=I4C7v_YIKM{W<~c!rQZ_Im1NcBb;X-o=DxR}=>;C|wTEe!XpL;+f395>7L7$fWrX zyE=^%bR}e=8oQ6)XCNt<`s&xenr}^4e`0m*kd2!cIA`*FE#gw6B|s1A>y_)!TC5|5 zTWaxzgzPHVzkYprJ6y+pN3CC;a|J8*Us2Wj$6hhJpJgHeK8oy=SY68F$h6tK&$D$D zVfWHur(d{JmSJsZ9gawMv?1+dhcZ~7Au6QPOZS}3<$pb-qr@GiqMuU4D2R@3pyX-) zN7H)A?ATq)^7rS3Q2f_j#Ox(I8eDo)G*i1vE^ow>nbU~O8TKD#M!t{Gl0`hz4 zF-Ea8vr);^)u$|#lx51kdPM*of$-lIo0su-qBKlyKCF6gc8i?KKe2=vNL~n+_GZXD z5)}BEn}ZYejG-)vE4E5fW{4a>RYZ7a-t@$mSZCwPocDvf_20%zLxz3!-Yh?OzH=h` zSHy*>pYN*XTTS->C{om>;(VlMSb24UN(*@j+GP}ZC*&wqnp%aXIHv{Nc!%qrKK!kZ z0>cxMPHP|kH{Wx!05m{c_-c@`yPtLMbMGh#c~-<@{^A56sHGMgAT+vWrPjZ_dgX&H zshM=O|HGPRYobLJ{x*qoCOS1mT+u(JA_Ws7L0rmTdVI;E68$o=0?nL*d|$kHbc%`Y za6j^H{Z@%!DeU#zep@@&tg^_RUI4-uejO7zjtL(>nGYDyHezd``PGFdE;oOE;;x&X zp-XI>)9D4Y)SV{$(O8Gjj|bShQPVouqKlb34Rd%QN^ovA62P3-P^4)uduUaABYGc^br`Q3eQ39epz|x&O+kWtk== z)4@U(-9&u(9ZL1m9pSSiD%uVPIfDT{$B)vDBto5S9<=tA*s&^XE2;Pv}h|JtBN9t!aEZz9_ykr1d#U zfFf)nwo+X1Z=B_KxSoiCwER{7A!^{Y9}jP**K+^R5{!tM5w|$)`yKt!h)}GGxa{W` zJ*tpVNkz3Rt=*8tTHfy-Fl*$gafQs$@?o_(pk5giG|p9rrB$2Rpf~XQY{6kRqFhCrM5uUz{IlU0kbL%LB})5SKQPJ_Dv! zRVuJxoO%VaV|UWZwbN}5*@Dhw{7sjEM7kfNQ zp+p3MNr3Z5_ep6uOUj4G(2vO7xX%_|8nktK6n6B}k$?60QD?7!?gj2sw;NL-Kh8#f zmuJS?EADUa;Yv9mu6XjJyqwU6>6iN9LsbNF^y|8*V+$#wzXCCua_+puGo-0|(pP}u zur%;a52MVR9R=+@OQZ0Ka=(y{=6-eomBfzov-6}(l7&?}k8b(ZceOdy4(t(asVCHo zJMxVpA#yY-%AVK9tNVsD^O7ELIkDs!GP<(M$L#5ah4h3kWhWTkoxQzJePxDBJzVnd z)qlNOZ&&KOy7I*G@8wB8kr2uH+<8nd{{*n(Cm1v5X%5Az)P=b}(>j)7DH;iv8bZaUTSH+z+_q z#$~RH;3GTCs@WKcGLa@p{IR6&_*XT#?6(sBssCi~utpKL_k*M!nQ+zJ|mR2RSr>w)p^6jUaQ4XszoZQRUwNE#VU7Lb}j%^ehI zFYHe?!B1eynXMNpW$u{peQJxo;@+((YkT3TmlyMX(9}TWwQ)#FQ_8*nloNcTT5O z=ET8nUF+<}FW2bDLya8bTFo^<2HU%MiYFDVyybhZ{%u00n6js8`$m-(C>AK^?;x68 z4FN7kM#5>;2Eih46A`!lcLV>h@!{P)yj@IY))!WnuP9OPQGbg?4UHp{zA1w{q?ye$ z8V@>Q-936#nmgDsBz}`IzQb65B&e_Nf1q0 z-rBQf;t8swCm=EZDpYy^_V@G<%045;sKl>$NcldGY85-{G=7KDWV~L2Cv2$hss!Hl zt}#tf_S;S%YSdEzLZOb&Erq~O!GnI2w{--ve%?I&_=yR%C>S^aanULJ!D#l+OEo21 zHWwAPJczpP&I4}(%IQPlah;Vg!F7zf${FDD zj}|tC&!r%`q)bGen1>+WyQlK2;)mcJ-RI%nM84fGnf_mQCw`@iF{)jxU6Oun71Y31 zzz(QkUg_8SS1yQ0s%Ee4`cfM2N5*gjy3pMPn#jl>T>zYb5Sm+vOa)PGIF;cH-%mAEwOdK+tm$8^|LfJ-(&#M_$3)T)Mr*g6N^G%+}9+^w@wWU6}EvxLQ1EY!Oa^cbH!v zU0Fp)UBLDI*KX#wht1Vm?95Hp$P3fX#Na3g7+dqTJ01Y?Q@3J?{P5&dbV@N!vxifs zwlcWYjQFhu0C0INXCq8TUhAvdf`>qX6YyT~QJ)s2=#^zDuCDn*-~_9QHC{L-*#GXa zb{A>p$V;yd>35kVAyR4RGhSUwdV{N$IbDO)si|bz_Vt0H@g%j?mgYBxw0J0>e0(#k z(b~Yt7PvQ#_*9e+2?3)4xj)y<2(f+zm=&B@6R?@e7Yi@l_~sEbLX_!d%$`G2u-MM> zNAI!np#$zs^f^;hi%rxZ^gi^Q*hKxW49t3_fw99#AMgDtcQ0#;t%~CcGO1s9LEEE` z3~N?O5X>!KCS%`!K}oMq{icr6CLx78`mLyAgAioxUD?pHRJFpSP^JF0NFq}qX*;b# zllBnQJ!f>^@OpO%mOQb(7X+7}?&aPatUyjE>#j1?iJZ-mSAJYRG=De>cF84ecE?2K zYjBIA95X3s9%5gcE{YeqOQG$ve|Bqf>GmE9U02$MV!)iBRjNF|h9X>$*wXfkU%C&B z6G08+h{YpB@-MqsZ1bV{Bb%i%s$FDTBk*?qjoG$d0`FDvrSk|dSW?(L5_posjQ*;L z#FFZkS9a{)LwCa*m-J~#yhB|7!`MAsbq*+wK7 zFF{8EAk8+))4S3cSgA+DNK*wJ@XZpGkr_rbp zfX2TbL@=%tFJ+sIj9IkKjqEC&Dg@k=wJwQ|T_z*~)zvS^O_R+x$$rZZXB9|`AiCAI zjo}=yXGQK`8)1@oSSG$8BwzF2?=V7!3|;m<(0v$9spD@SNW^~*eH~b;0_m)h#oXLS zq`Y$_nUZvKD?m&0Pc^iT)G$kEz54+-3-7QDQ;VyfiDRWL#%{S);#$^Oz{?r*w%CXl zP9C>RlEfKXl8|w%-Cw&E%_KfuEUh|XR>bAvws2$g27Pr+qfpS>f49276v_Br$V!k3 zUEYCE{iqgHacQ3ufxoKB%mr;7_3xOXq18@uU$hl&Uv7yw5I-h0F4`kmB3dFIn$;uG z5Q#ld(#`n7m^}~2kSl?0rsLPNJ9b|-6REwHt$jEbs4^rUN*fWExp|ekj_w!_-w||w zzVeF9_^>dTpiSXVE=^ubfF+jSVVlDfY`j?-{=F^vlNaDU^foUX7Fa>bH~??oRY>-# zsAiLPg&Kw`Tz>}Tq(7WyT4a6J@tsH_$N=pncZYjHRV4l6=Id<1x8cjRyyJWJQJeUX zo$C;$qRav}y#8yc%_QGP_u+?0OWA=l{!XYB{ZR`QXMB<7M-3RPEK@P_=9K=2Wy_Im zg5#ucJd38u7Hn4j4(#OGQ~@QZa{CzvgSA{x(WAubXZ3^AYmU+-JBu$ePZ~l4Ab>p* zBO2};5H`W4N=^+mU z)-G)ahwL_OI47p{i8SY@rGOqtj2*FroSd0RL3u)Qi*uZ9KFWT1PK?=otCVhtb4|QD z$5eJ|UyyeeXNM7$LTvrEjDx;90?{2JkvfAx5&h*i|6MR2==HPBZFm(>)hu<`o@laQ z^9j)KBk)7;401cy#AO|xyPiS*vc%paifh_p&Y(Kgm3vktD%yH-gw%=7j3RjN85D9c z8bE! z9~UIz(4`3@eg?8zDg)&0)V#|5`;sk&chAsQL+%p;zAl6}$B!Nqe2y_NMnWPD{2PxN z14$z_`Zf`4W1)G76wjA9!0OJYfmMO3TP$SKKjv7WO@O1k4y?Hi~rcTohK zwZ)t(U@h+S$%JJAxx-4Vk*x1gAGEkCBG(;*TTaFGOUkH(+fIiXh810bRq~wsksm9R z;-i?HVf~x9mOpRAmTc!Gl(7AkZD5pbJ1`b#aogqS0q0GbP~$m5iI&e>g&>|C)BGrL zNnY>@%7;0kj(kl3Plzc|yDX<}?_K*Liw#Je`-~$jltuRII13N1ItX^lkqj@;*lO)pDUsa_`?*?2 z-xDmXYzMhT^d9{2`xm^Zx$fng|Ni?|s`@uU#h)sWRTugEz>w;$pW>-&tdHI<vZ0qvY~*049bEgvan$SIhefc?auXOT>o-m^(Za9L#NCX zV$~-&4QDr)b(U80$)sK^W?3gB@Z?J)O0QiO-R?lc%cx{huF0;a*v1M-rW>ET?iiW)g#F)-h5gfV zFs%=OWt-a0SXrJU=DdPu4-rlTN`d=7greCWHds;<^VDKp79-8B%Ve{Q1kXFVxM35j zejcWF9?XUgQbaM|IdO$I&SR1N<)BfYj&x>YJe@D;OyWv*&W~Vi$F8_nX=dnBAopV| z@nnm6(0{y^QFJUdEPi=84y~P+AzLyeq=u&gR_O9|pN7(<1Ez?k6dY;a%6brIazb1R zYAft#764N2pWGk4r8~n)#pvs)d++urs5?{j-i)?4c()06reGz@W4?Uoghb@Vvqk%$M2g;R$u`&w;N?8J69#90dFD*h3` zyYNEC8GcpGnZ@ctAR?vrFa#AnNeOo zBXr%kgL^DUlYTwx2P}p&jesMWghLm7&^gDl&jtlYT}NF(LHgWRz`eq8r=}*PxpqmA z0A~}5*&dJPIVFQRC!7SJl074|{H4?d;)KSQGNSRHElCjmgB+ffB#z~lq2mQ`mo6rr z_psGk06`40aKo2saJ3z=Vc><%^*C~*B=D9ePC{=coOe}NN$WgQ%G2?ud!*rlx;rxN z6ppI@gabVj%b(WCsDNePT@M0zQ)uH}V^H;1Or}3{mgwYtHM4q?FpGl4+i2w1aY|)J zqLl=8zo9XT5!|p5ZW(@hO%(qI=H%;T)+=-%-Yjzb_B%R3hO}wGsbE!aO)R(ZijQUh z!?x0oxx{C&naEyn;7|YDHCK+Z=etX^&z-W=HoF{gM$tEq$6dKoEU{ROsZroN<+096 z$4{(5NN~%kNeUM;?%=v$v*}rC++%G+!0xcLhsSWx4@9-o8#5dXGiZ z#%tpnE+Qy$j$0ylt8Rp|e#?IkyH20H$LBvzZ^hTzeJvqIHuaU{(xJp1>@kpM+kC+J z3nP7$oq*u-%dCnIU4B#s=*5WXzF0B;KFx;v!il>D`4kb{m9pQIMyo~hEJyUI!J}QX zNW*BZgQlA}shZbAN5>}C)$VGc`tU`ZieYGU)v$A6h_+bl2mJFe4tZ*DS6SK5S(0RQ zU%Cu}DwugNcdXN^K1lLBTb3!u!(z`*pzn~y)c&y*sAJMO(oKFKlssc zTcPo=M2GLbMDOn0u0M&+gq>zw2|*Sd#+=dyJ&^Y1ExCLi&8w*#&1mMFZO@k_>BYwY;nUt;iesj!Ky;-bj#2Vb5yL zZ|nV~1qe{10gjt)kg{j8>yh3-ZXM@!J>3lPxAt36Mw^H?>zFZpHNxTXvY0SdiaO=j zzs7D-N+j|h6Iz`bo^FsHtb#II@oY0g6_;bMg{vY+Zlgc&=TV%@P~dtBl|in-%}*F_ zC6rSFnEkmyfJd>keOG>TvI=Si>PdG0HmrHpl6GUmuU?vaZeFnH(PTlh`e`F92XId@Kap`RcxA*lP#&@cldVc|I@ zyqd%KF3|CEgZ$r|k=Dq(@eIYdh57k8k**s<9Gy8E-f)AK-D_mq>4a(lVC-tIjSdaN zXF!^Oaf<*|_IFu3-CgE`U!OX%!i!YMSEm`8V`}O$*DHm1vBig$#G;!@I}eHqIfFop zofCRe2bee|$!T}^t1DOb1uC^K!}PX-FK=MTHZX+^iHg6`-n+=oN69k2^?-qmZC-{a za`9jf_7t$J{P2pYjo_2S+S!%uGvJZ~&P) z0;Ea}NzUoD+t4nQ@b9!Q`*{}te$mJpUTAZN;MZzV8a+)L&2~N9a(B790nh}L(Io}6 z-jmp({i^PYss?EXZMVm^4_HMKv*R9BwBs(FR3qUbzNh(9C=%;U@2&Q=y0He`kFJlX|>s`)^Kt(Tq=BZ zPB`+O4BCSaNhklwpf<&=iLfb)#H2+K6q*qStFI0u1k&kG;3y11Q=dkkDEvVSa(h42 zo{<&^G$W6dXe^i@+FU_{$(ozQ1Zp-IRl^GFN@R9?T;lx@V@L zuIhu@+Jm1ndHF1Vuwh-L*DuVLe^}ABbZKk&PZm!%ss6*+hkW$=R^mzA+uFs|^hL9c zkJv^}g4*M4*+ej2%-=iS_Oj|Gv%XbKn`)@Z)8w4jJ z7?UwO5KdXj_+{n9nmojm{M@=Jfc(vmzacaIByALZNFS!d$5gV?l!`lz$|xhi8mx+p zMW&CUet`m1w4OyKAQ`?&&*%`09=nWr$JB!F$RqF->=b@TIRFfN0YV;~MWK%aRIb1{ z0fZh~9_{~KKg!A8KM$#F2s8=;S2p_J!%JY(s7O-asbK3|`YAvYA+*~2CLWNg^v0Ss zn7Z0*5wzNkbOEc!cvWO|)dJtCPZhDN3xsM(hPH=jLj5CY#4W|rGbjsz68AkK;r=5j zg8yraU}y7YZ7&h+>Y{!L+^^Hch^q#9s)x}$ul`I=I$fAA2q-YQUpG}8H0``f_48-& zX`QNI8rePWoJw1r2nV3p)VQhp)h&9`XXLW<|a6bJ}9zIVyEV8Ghv42wQz9TLX0fa@)+mq-v_L>#I~ZXZyzK{shGR{;Sv0bds%h ziO6Tk8FzH{AT=$0G4#7t4(p&U0rCk4HA+3dIryPGVJq8m40r7jwnp+9p0M8cfMjda8ZP!q7Tde8Y!MwxO9P zjh&A+@r4OfSNG|L2B!puq=<$FwwSE^{oPm@ti#x0ywp3;-J6-%m8#h!wLpuaD=uJF z#LoW6YE-f3>4+9NvGAY={&?H4$8o{6+-i-=-z1H)HKxO%yExWU@D0coc)O>t7NYXC z%aA@Aqt4lS{aTAo&j|X*JL*t8sx=mUDQu0m>qnE`FhJOD#s}O%fw?f@Xby@&adCUj z)r#WEiq2w->;68lQDlTFqoD!qVjP&ljgGsFx5ch&Q>eUsYV=64 zEOpFRYutCB*~0gLohoHId-E>5Ok8N-H`dp7?U-jUe#8ZzC-F;E_w><(v?gH@7&@Z^m!Xeec>b<$$HAS$Sd;L7A= zYt*;);=o5V+%Y79ha`zd0uCs+Y}0`y3js{%Fk9%y|@)#x_zreE9vXrwPq(yWx`Ve|AHv(gF%3 z@LYe(HB%Z5*jw>NOw&zoA_ThPUCXdGQ%u*H+0gAiixW4*2(}eqXW9A)?0o?v)s6@* zq-a_6XCcnOwea@ea#MQZ5dJu`PT@rkr&%BqzRLG>n3cIL3tRAY*?#-D?6`W$YJj`r znTinLz4B9>*zWcJ{5Bjia+9^s$JPNr09MXZi8`k+KEi}m#?-*AKC6<;%SIbYV_ib3 z30s)NGmUPxIrw>pFAMxam*rWc-zoNAXFSSOiKaA4fH@wj43|soWxLenUrl{-=|UJ z$>h6eF2Zt!(&@tWxiy2uDb47SK=u>Gjd0}`xH~hTB5Q0jD#f$##1Vf&a-kBj82SMm zsyehnC4EMa7|HU%%VB=J$%QGHhKNuxbIzV%y5ED2UhBc)m*UL1YtV;rl|Fk*Ex zwaw)s@z9>r{k6q#hp0)1P4gnuto^R+NHzQTcZY3-SvyrXX`6sXZsQOzsaIyig(??= zt@qDl2d1`8GTvlJW`viggs#Eto>!K*dTJ>b=Y3+NuQnj%==bWP$UAgJuH8iDT&BwG zY(=RtDp2F$@=!%gTWzgujDtBCz`v#j46!WbQ#2xcv_7O_;SVfeAQNylygD~UP3FbU)KEteP7YP>fbA zYfvF>vs4`wP`t*d)yDIV(azHmXy{St(4ry)QoZPcf8`wM{j3L0cOc#^3VhoQ<69#8d9T@WaUg20JJL3}jnuL_54Jp5ANnTivNBv=so8~87;d$46%tu`CQJou zkGu7HRD3G9*NTqKWNMV$+UevQ&D&**(0;FtG@Q*F{mIz82=u2tro3?5m|1}ZpgCQt z?7#qWS;WUP;+}xMXh!VK!O+3Gglq;Lc5hKF!8`1iKnxpIPWy+0X=beCEdPxV+NM^D zKa|561WuRDb}|oNJebKLl|N@HR8oQgjP-_{Whfy?OS3S!?ebf!R|0;XkuaMA_kU5N z4F5#uG)7?=GF#wYdPg}!Fx*LML$&v@jbyEUK zoph=(YB=8O!#-)UyW%X8=GhPvsmUKoQfP3pNdr5 zC>?8-tXT#^qde3ru~>SKfp1m6T2)1NTrC^u5wE`Dr$VP0n?im&4#ZXA$ib{nOZ3{~ z$fu|g(_Kuu4G<351|jNJR!eI+wTr$sTAEKfgbC2R$4|yYI~cLe)_HQ)xVNOAFSV$I zWJAvqxzCsCxduKxS?RXRy}L}k*ZR2#wL!g{sm;L36Z{J2x1=Ys=xLmJt}oE-<; zHZOZ|>>muzQ7pEGw75ZA;XF)O)DAyq)ltnQMwkK9wFzA{tAskH>{>znH;I&|B$KU*y-6l9}M;hS=~Jcx7=yf z8h#L=2$H-8=L9=$A4Dd3E^G?CV0DENM$Y*0`ulUgjF&h+V9`-5EW^iVgaRVt5(da2 zVJ7%&K;E^7Bh_5|q~mt@pB1s#&|`v_$}YRyf@gdL{mZfyMI}NXzv@x!NISA~r#|u! z7idm=d)tLg5>dQcv5sBf=e*NX_T;1Xa8iagNyrf)I{L( zd8X2fUkYTiK7M4yK&mE7QzwciRsI0RtNFDQTjP5B@{~t_MRbeU5-@IjI}t&7DHT(W zkGO$qf*i-hxSv<^+6}sf>pcGwm6J34>{7~4Dx953N+c8DpED{bc(Xe|fn~CNi`nU~ zKZFe3U^InNjIT<22>L}F>EnYNyN9_1RqMCrv=6FzBb#E}@Mg#kpjV~B(vvG{cW_7Q zq;T2a&fBruE<3~~aFDw+yV((LCU-OLDup`np!~kAh^xCZ<1ZJEu5T+M>OU)@aST~t z{s2sS``e1hq5OHq^@syRw6Q1jm4<#Gt#-^6JR#)wE1irnxwf{uv&MON`CEfQ%=oy$ z+^EC|j+At5nli){8Opa2cWde+V~^;|@wbcr(Ro`NJ6Qs@O%Az%8*OM}atN*tziz`} z2`cS^WXi+!{QK$MSq3 z$UlTY%?^MRXsT-fp@;ttL^@8mze2Hw;JmDUZFSV)o6&_PLtg(048=O9sVnS@@&53@ zpb^+@4+i=w?2d!`y*)o%IF^Otb#QZNf=rsqK-^{ThPWtZMO_WNZ3sU|T6gm7E0RPSUS2Cj-{*&D+pgStT zzCiQ2$VsupHvPCbV>~>seS+t`^HjNc9tZ#2?W#a;@4hnb%2@ zNW&NW7Xq6Cw)*fuErLiGVdlYXA*xX8B)=Vq$mU&qP)Jx`@;(Qn=m1Wzo_Ow@D6meT z?Kq8_NkF2sq-eWb5Xc+!YeUnh--^27={7Sq@FeXTk@0$~ts2i=>Hi#?6l*;NE6sa`|IXJ0x9tmhCXy-3Zuw|Iw zBGDtn?>EK+wT18{&m;Qf_%LcNX-aQt6B3@oBaR(i?DLr(+Ea=R7tZ`f87^d} zyA=qp5aZE0E?i8`prYuf1LS8aYvV0w6`ScDRoe;5nfq}(X?7^Dd#E)?T4ojB0ar>| zra{JG4j~}(cj1; zdKv?x8Z1UX8-weP9o8c{U}aZfvWL%8$+(^X-2!=zUsR*!s^%@NHOq{GQhRhxaN6qN z)R|hFV~KcN8C@X-w`JQU8lRMZy=_`NHl{Kdt7W4P?=mBR9@mByR+DGtxUrPxooWs? z|ERHF8g3b;;U|YH5!IBY6BPjL-t0Dzy?D3$^|PL)hF6%5KXeRxFcyX$V5L@#@X({m zdn2qbUw`4_$V2B$rjH01;jwg6m%y%vTa!~GhgwI3jaFrqY$C7XmDC^m5a7!D1hW>Q zuYi?+F1(oly<4-mv}3=Qr-I+_U7Gh5txZ zbtm`W25m*uJfLN+3Za)^$b*VuoDN{!`LVd0{IUqV^N>~{?Q9{;uZA@=w00x;WBpZE!z46+ z5qJjFvTp9L?NEzsF6kO1y3SBamvQ&g65v&Po4~W}Q;A+O$W`*fQkWuA83c?D=K3ftvZpK=V3xP+a+JV2z2&?_!amz}@%24qg-sTK?EqoZ&Y9&$3(cOD2VdmCN9$yPQ zwJD7?6rQx7F88OD4T2Pv_c!t}2vB6RWFO)H8~MR;mVT1~C7MB0hqxEPRG8?Ekcy|7 z6VO44O6jvn5s(Qd%|_Op7OP!bDG|W@kWo#ZBAy>Z|9&qXFRrxMZV(^3#`;PDlQ_Bh z$seNK#;aJ>Dy_A6V~y9`e_a%famw6^2T#CLh8N#Ey>ecbC6aA@6o9ShTUPuQD?&!x zR4%n=t}EJbFP4x%s_s=DXKVYSbW{t8i#YvriqXHgaMm=5ZG&RO^mPuOOd0`PMJl$r zO4ch>X-f5Yig;y3mC@aIyjNOaQJk_bR<>U*ng=mHP2zB3|>E5|m5sjTH_QXKwco%P!SUNwvH^0yyfZ-!=A&EKo7*OwUVWn6uVw zOAOnQ#Aiv48yN^tnHUrSIQy7fpc`4HeYmEBuDsDUdN}M5b;o*^q_%X6XSOZsj|%1A z(Pn@?f2PQYH7HGr9H$q2O)gT+Q8_)dufD)QKtR5}z(JJdz#%X|pgE=5$G#uD-G)TN8#s?)tCl}^!e_i(yM<%ktGXBd3CR6+Jt*fa!v!y5XQ^^8en3xL? za;jb8Z{e7Rz0u(O!3#^qAvRk0WdK>xZb0YU&lej^=s9a5yLsYLuhf;Es&+K>M_@{!63AoctQ6KKr_vHny)vSl6 zoHSUfY#B;TPUp9C<-r+1*S6ZSeAxR48KPr%RCD2DLolrifev~kjcLqZ$Hve)1z*j? z3$m5^HLQ#GA{6rn(utpPIr36s^Ci;wYYg)}YNlsi0hu-qeig1y zmwaoQ#d8bQW(330x&0NQ-q6-#&ys5Llnz9sJHf?VMvPmsu(=&}2IYThG$X@wFx@39B3Y!vzm`3b5;Z5yU?9nKn@(Hn^yByq+|3uEAl5zHA5(goNQ-^ zg;FhE`z_4BpNc(vjWyx|gdgs*HwZqZcQ@}zd92xu!)fujc+6Xkjl*a^Wm$l)x+9@s zIYofy#&G{VZ{~gZzaj_bJr&!KfQlb0GVwd&1!Ja|-DJ9{)Icrfxkej@MhnThI54%>q z31{WXdiqqaAd&v#jn@LbtaQKv9a@r4f}Ss1(PJ3iOFQ{AKdutF*!~HI?3=0PE!;-j zbmos<%Ey+Da(ovj!AWS;P)=lLnCDg_w%CAsb*;C$jfXLdK3OgNIC7YIcbn7XDvyWD z0DnSds>*lt;9qvi!7Nzn)Cs9@%Y@xE+jtaOxD#XACscTAB**W)7@*`Cx9DI{&~zX zt$xl$@27vAM#4?eCF`nvkB;3(>VMQ7pRAyE&+v?-d08Cw;}U1x{Mui!5*m+rQN!~i zI6@D)xGxWqm)zr>F1-R8wLZc^-2v{8ews_%Q6Qx{7ibhOttFWcL_nQ(49JQXZ zawqGq`?i?tI&S&CLcAogy%G{w?7!fwfFm!vI;BBlyX-#l^GyN!?LG2sBe8DXW=FV_ z?IP)&34bq>%{I`KxftiS6jYzUC22SrpL)Af#?VT#Dw&n&tgLag^s@W3YL&Zo>eiZt zcLiQDY0>M}{Gkhv{KD@nimqUotvdu8i-2Rw|ni4O=kZ+DHWNT2LC*(0Ae31ecQ52H z24E{)kqJ?NXb4#8OC%IbBFn*yvFoUPN)38}MI|#LMzwsOP-N*RG?R^1i{(k01B@1$ zm0qZ3&?I}2@3Ii_?){DPuM{q4g`2x@=ZY&CS_qL)y2lVwGC?Q=cqE^k2CQMaHsdc>a z(i0eWePwhPEh!xGs8fh--9PbAJhhs-J!Yvd>R5qNdABP;BllMFx)`P$C5(}PjFA8& zmh+Q)1Xo{R?Ff<{vDw)>f{Wgcdv4RhIi-7vCHr!wdLctcz1Yhj`S_OjathH>9=}*Q zP53?V9fP=9q{Z3!&%^6aC_6Ty4mDz10C<#=C$I4Rf0k*74>%P zwI{!S6kvLQFJyCY6shC`?H!&R{g#2sF=D`#_m5gj6m{>rf3ub>Fqh^`@*yumkAlUW z)gi1q~6cMvT6+`*qssbM$Z3!Wd&n#MYqQtN>XrCiPd1?w*QO`4sDdtSWtqDe*9r_M1>Rw zHwT6)c{8+Tbkf!H{!*&x%OK+%#AR0=Fp}pIMWtAbLF|(=MZZY`=;tYU)g2cPO7h1< zHX!{|IN}h7jR5t>R#_q2p_|7-kbaVH^k)h)zPvg-8H=dw26JV`BvCYR&7v@(to5mvzi$SeUT~j3b^XCt5l%#<5E?5M!7Pdn(NtKb2L&gSNdJTGIco>D zbN}L0F`$D5W=nSTOvr8FoL+w|rnU7w|El1o10&j+6WvJAxycCs?tTi#fIhA@=*5Af z)&Y5Hy7COXvD+W~s!}0Zv)Xxf3kd7&iPcPo^Q>Fz?L*mX`~nS2v+#-Yr2Evuu9K@U zS()PjzBFwsygoCN7Q@i2s9n#Rxp}sXRKOZL zLEb?ru5VE^BMdSO!ej%vO;TP5vW$}rjyGRd$RLG7c9(lu}Z<5hVCdmUMUHCh--T`;?z;DjQ%J21~kWOyo(0BGcIGPuaXvB&)*$Ql%m1DZdvl3 z#5FTq$*o=Ok2;KJ59>>((Q>eEr`;E)(lD;w_l!P+L|juU_+E|SPf>#k*>Z9ENn$yQ zd!%l*op!RUf++wl?qk%HjK0ye$J%BhkGT?9gid0(IjfvPHawMfIg`t>+hOkL8aYja96Yt?rJFZS=M4-(H7sub}6P2`CxU+4dX)`m#Ek>f*&uX+|$wx`+;^ZvZVHfDdcP|DYBg8qwy~Z8sa(EAH?X52zQ;)HdsjOwBn$0Q6awm_5W(mMBw_G4F#ybLvO6Gc~LDH7@5FLassqO|v? z548p9+$#e#`ZcwXbnq=6IU$BfqO`* zqS#l>b?>frC&}Kimk@cqrK^3hbLcwZsH>h$Ey7a|J(Y{WDymll#fsEbQ^I7!sEo3| zw^7T>L`KxTQ*9!R^wT+`WOT4s#8Cno9or=qaT)|=2H#Ft-a1)=_-k>;PZHR(@5lHh zhx*4;3}6QCv-+To6QJV1)OMZ!0Xukjd9HUNxhlfiSbR#W;QeAmR;;R-9*nJ<9uxZE zHy;}hsu(ONO)Wr5(X&~8EeD@8?eu{b25Bu%&a#>`Ikq!+E#m=OPpa}tla;^V>VssSbV8o_*IO?O7R*^ZTUiYjABvKqV{jezME zBTgsQ>-&^5i#dxgdm7Wb5{h8bT)F`74IaIY^xv>qi0v5JbdSwy;<*)ImL&)JWXS7p z$brqy%xw0}>|5ahWKtAA+)|s+(pK=c^4nh@CU<&$KW=AHU6wdLHZ*btu~pp#o6w4> z63nd+6cVOpoLCxztnoH18)=#>^NAr68F(|Fk8}5;qk>eJ$^c$p2KP!H1ip7xwG(Xj@QJ8w7;sk}NXOpbARwvmi6kauiPtt5Kz&9-<Sn2=`aeC~*lJ+61}zimd_>pzV|EDr&5E)Sh5#=|RGteMT z^P<=hrI$s*b!Ea%>K~bDx%MU3-LKa1!%qix@6QNpzveR6NNw#$fj&*|{{QxeYm%a^ zMG=~G`A{0WU%V}1&N(JJ$KQD}HCL2YkCqP6m$&PFJx!K`s2^`$xefeMq?U3r;(V?g z!kiVR9pbrWz32Caj*MxvmRO>%MO1TK`Dn^6WbfAUPQ5j8%$q~!`%NP9B|2ZnF`i$a@13jB8$wfsU|#7iboJ*Qj65kJNUJ_FESLHqXu4AQ6P7IMCI?!*rRaKLEaCG)}?IiWw z=*)%J*ahbDfvAbDsDZa32HvXQFMi^M`IE%%&S}Z!F}6=BL6hSAK#r)aPG+c zv25st@G~VU6)Y%iU*2qg{G3>@veORIvsZum@rPvr-eiL_|aA;HRF=W>kypUQa8_)E0{&am9eFA>H-+c6b zLy+so1qAaz^1L6iy*Z-zyjxE28iV@+i9zR$jgUL@vhDbgzET0t%>0@JmcBnbrxpsc zi;SKAlP9ZMu;{z4!s=rCK}6~+!?A*)o_8RSO7(huIrZ0$W-ucc^DIM_xC~o#iCL0fA19LmltR zDVom?`$2+Je6uvx`&qz@MuKCOl@>W}2=FWh?N5|b&^F70!Lx0?{&~@BLkkQ&>ui=un8(KW*`QV`Ujqx63;1H4iu6K=s-Y5{j8dLD5n=p*scOh8a+$fx% zDs30Zy95U5aQ7rN1g(ZOV=RY3@yKvS=Ke%4XSS$j{G`=i>Hg+#;s5yi3iq+XVco|b zK(?t?a00}2&5@DV=Mi&48PP!Kia$gw?CZcI-NAdQIY!*Sw%IiRYCQezJEP3{C-<~7 z|JXs~HOq7!$*@;t^3yrKuAu#;Rqj#TYn;vR1v%@e!~fUP_+RLF;`U$ooDK;M&DALy zy#7`9srEC%ySEPc+%oyI?Z5ohE$-{r$TLZ0pB^E52~)p7|Mvws9102(e3N(w%>PI{ z4hRk0P~2oe@vB_~*3BQsomnuOh*C^yB)65r6C0s1SfScxRchrn+>zxd|!bH?^>I@biCtD)_Po`*m=<5-9W z*&exn-jaX%wpYoq{XOg;m2VZL$kzh*Z?5?5GB+eu04swLT)+J#HqQ@g=Uz?AYw31O zYD>pmhXa7_>(ToI$GX`!P+TvzJatavObg4(-ts~>ET$)Y=G4+2S5_k*Ff#`a?1-M3 z>S>71hN#_i+{937ky}ecZ4%iiho;>~w=`NF(EcS@qyTs?5(!kFJa|a&PAt+9JV3hnzkAo)TXk^% z@$TKL1>^XOjt{YdIu?a7iuo)qD9Fb#g*$J$tAov+xkiFeL&IW&!elIGic{cZ)ZGOs z`v)DHQv`#RKB`-bLyh4{!ec(X8#c^G4aXzby00mEA}INehZ=cn{;n|vxbLhp6+O} z-%jiV?qiSE`f`yII2do|RN@J~PyJ&SC#s@ty!NQqSWBK{wyZ4{A4D)e7>sAud_e{7 zpwfET>pEe0D7Xfdzm!n?r6H0Ga!#e{JCclCrtmj@9+W=Bqs2PL*QJR5SjX~^=@S;y zB%h@U>3lopdHz4LXOb74=Hk2GUCJY;j^eQi& z#OvQIz>SygIYTqcjXofh=CjkRpTb?PUb}FtsFZa_Y8QLh+0Q%_aa2?-dq1w`bgZTRF>L zL|k6}f~=Jl;jepH1k6f@o0wWouqa8yn=x%`AY({Q=>%qZeaC(v$F;+_c{HcM1axh} zGr7^q6uh7CgN;0XX(sb`A1_r-CxMoEh11#%o(y&n?_HCgPA!&7Q4~FaCZN7Cx1-1t ztN_yFKF~&TM4@?=Z)|-Mu?sfKHZL!$5stGh@|+TRU2As*L$AwASuh>5B%do z3H&bW;fe)ud>UwH8PRr)0tXFS8-5>RU@TgarLn;E#-BnC*ntCDO=G;H$L>9n@gS1e z+aFXqR;-H~ftvj6Z-PyI0XJ3$D0YF4#_98}-KIlAFn5;O2}VfwoS4YZL%{eTA znNouH55aS{;PV9W5jOWw^a>OW7$VwWMb0w1qYM<*6+R3%SI}eCX9KTWZ?-?4f%P$S zu7rp`o)A*ahP5GUf>YxbQ9Sc#s(QDSgnq2iAOlJ4QY4)3fpGK)cleKGyYRoc{B^UJ z(DQX!kyD%Ovg;4Bk_cL8CAosLK^%*P!2RnBZr$$x$=~A|%60SM3JGE7FvS)ab`0(J z43%glQYV|k0<6u|m1R!R34W(B0^c%hp6;)(lJy^0o_Z56iF@VL^d!Z6o9e>G? zHeoT+`C%G{9ug@sg8M0Y#v5io%5_A`GM3FGiI)>LtEWi;8Fb~?F9va!n4r*1wKDUz zY=1Y^_F5}e1-#4YF+1rM?E}a*iy|YE^yP?bx`j;^m~POfy0-L%!iN; zNg#enc%3hLJk~)R2!n2M9SgpvoyJ6=4y=!+tz+}FdLR+~9C!j$)ok(HyIbY|xBP?r z)}9}TXo|tQ#hpYta&W&ehDi{q0ee`Y0CE8X&mc#0Sy@bG8k(d*Hw;w zw%wcqvkLEj*j}I_JEIj6)^l9Z`6+fg^CsiF!^6A3X4*cn|5FZ-WVbs983_>rz{^|6 z8z_2!KE|pUcJqR#WPye_O+2Obs3HnU?~=b_PtgzAhQ?G7j}_*9iNQg3%aDUEK&L&< zI!ZZ>qotzL9)DO9B(t@Tu&YoN*5#_G?BAB5s8+M1^DvifgPzxm=rlTw?Oj4cTL_%$ zjQ#YVwu~h?r4bn%TKZ0+I?tNUz%?VpAAqZ5G$7P@2q6Ee)F8D`Qz;3p;dRE4$(B6ez5kTjH>BOPOPCJ>%(_WmUH^_*Ln*he_C8dKnzV zpH0)hUHPjPjl1i)NO`iK_^%4gRkEXn%T_ zdq(|N`^o(PGYkPLgd{G1)CD6ry3N2iJO%!n%$nOr{|07)%{|h8z%<(x04Y+D(vAKG z<|IGEO>S%)Kjv>rL6&S>!yXwdGa1Y$8Y*0P2i~MCm!r4nB?Jz(TbTV=MZM-C zqwcGZa`7^LGgUd|xY9(Q_ zX4(|^|CMni;83++d}i#%aw%kH>`RvHlhD{Ek;sy=6EPt^WQ&rCl-*~EYx_t@5m}0q zC|?n>WEqXIWUYuoCXD5O@jQS1{om*LJ$It%US%t+{!kT*Y z%sJYUtw1Wnm+6C>7@lkcg@n^Z1itjf0qZ7?6Rw+GWxwLy=U62Nn7OvP8+OwImll%c zTh#-?W0Cjr(wZH$+)M_A+5c7ir*J+L_j}kE6XhK=)#9ys`iqBJr{t1*m8mTO=Mz3$ zS+_Pp(|$q>-TKW|N4%V!-xI+%rvI%Wij)!|W}|BOZ)i@P4WKX-ti@kv6^FgT2i=(= zsts)gvpx?lBF8;MV&s^%q@wW3(Rp`M*yY5n>U;PYeAcr14y>yrM_AaUGlpF_i3`Tr%mwTWye)Hrl6@6TUr;A5fPIrgO%M*{@u9(o^E?(XaG0_M@$n>yLb+P^%dC zQf$NMOcvG!y#V{hC%PD%EMcqbxA0j$VKvKqPI^lAuhTNZhS@KOFWm0pOFyl5mYfW1 zaUIC+5a{ipH)GAup5Y`&`otx8WoQQwy|P`a9I7zoqGE$od%oM0H&0l)zNUL#SyG(O zZ7!wI)hx`++Za-1pDZklwkpba$jPo!x*G9AS%#9$CLP87VqGeJw(+gIzzrYRoMQ`Xn5(v=27%sE zP%hpSS-I$^)gdKKeK*e6dn7^Iv%b$g#cz)8kvbvE_^x$v1d%yFz9s3Dt@zbQF5hBP zZ(NCM#a~Iv^Lcz^@3i$D^TDbC*=k+YD^X|tsZxR*Yq*K9LPTs>uOXa zayG@}Mu=1Ua{sm9u{mcZ=DMaeVn0x~qTdG)vDQmI0|2)BITw~bp-kk&XJ(W#ed4`8 zJMAb$uye)e>-sBsbnRx|{xfg1z8uu_v(wDt(i(cN=JM6NDen>2Wo7DpOshFj2DPX^ z9Vp$PBOKePsZixR=`x}0m!$MefWn;o1C>5IXc7OQGwlb_qtT66vbG*m-U^?yUSHo1^kV7sCzv68atJyGWpQ7dETB1XJ zrpJU?-GZLyJw2-D$owOb_)qC*!{!nNij^cPT1^#gB=QaVJNWnakdE+#>zRKY^8om{ z{HE`yAa6|g&DS*#Jri(4F@P4d8DtKVup76E3@2*y)SwvaB<^_O``(PSz`9AdE19Le z9Mv*;Ql)L@1cQZ*1kAloF}{S4av$eD?Dg#A3dymk)@+60twK3Jnx<1iNyrheJJ@R*=!kO~s?4PGe00HEnxfrV@zQD6d`j+* zh;^MOwAskE=c&Ba>);KE~ZHhp+}O7LWJsUhY42U(>T4LhN=u+@c)(W@DyGxtq- ziIKmDRO{p7`DOJ)4Y`?o4@p=e>MN%b#=Bn#GBq5fI&`5WKQ4E_T9q`Fx_Rsrzomgt zBVnzZK3E~~Oq(d-X}(2Ra!=K9Vx{LAPjwdW3^SwJaDFSM)#^MGqj2InRo008%!PiZ zqWJ@v4bQo$Ed9Uzj~bjNJUp@dP`>1gcPz_e5#_)elL5>B&A;9n0i#AhI0OU)ItcU- z7$7i0V1mF5fdv991SAAD2<#9zAaK@>fWVv*Lj{o`L8`X96;Of`Q-ouKlCPXM7vUhp z>A826Yl@IGf?QvEk)Mnjp2fSH_s{22-)3jFv`?P7^tIprQ7cD|70I|DAW4rtzT#}= z9CpxkDb9y$AspTM?5JV_^>cwNbs6cPZWoSZ#p$w46q|G5kH)wtuu9blIgyMml5eGazD=iy)v|aXJHJ9&DEaA?5o*w>^uW_$E7PHexWe#JNcA%(brkW!>}*8H zql;_e589`UOG~6N!VbuC=Q|@6Ii8$S!xCQ+!J5B$tT-}XDw4yQNnB6DCMH|BLM8<- z8x(&`JpS%Xw5=@~Yi*^ardlf*T(uzVF(iTh^o3f9>i3G^^h=DJVCSLS2RB->C2boiVVCSn4T{$U8p@$3fn1YK^PrYG z+_fX&@N*fVvHF6b1clNr%lWK1ai8h_{0Zgc&9L>Zq10rkdig^-&#~)A0=`;i=7m`h zsCSsu9#46slq2VL#*=4BHAdYGVL~PY>o#woYJl*i8a}hU8XEOe^)J69@qeBDn>gy$ z^aaWlX{q6MJPdTjcRJi^6*_>!aH2_*;=wyV&V)u4{Vo`0bk9=amasR(kHc{f3?qdC;BurF;A!$G75y+09gL^>61*h3g@|)mBz#!R z(e8{&r83w_5wmW~L!?*9n6%s9)0S)Hj_Zm@-!%)@Hz?56>pBuqHq;R=UneYkn}+hf zUtCdq)nwVJ7kf+cn^_}KPn%ca$?*9TMU-;!q$6_T2Dc?#je2i7po)~NZGFRB!W+Kd zrRuexMj{eA{KC41@egyZ#36^mvUz?c$bl zjte(Me_ox8(l=?nzKCo8A|7XhMo!N!MgWfS>VMUzyiJk*7N{)cp$ji14yC&SlSBXfl5sGS^$Xa zg$R2krm%g4O_10N;1Qjc;Kx2dJhjA<6}yTarh6b4r3guz7ijuPs5T?BPM0fOM0 z0e}a+ovQbQ@(sth_S1a91<1!^_Hu&z2`JBPJ1UsHl*N8VulMdq<(~i^%y#gelL-d9 zq`04^;}4E*8`_TMe}{exmVZAZ@!GB-V=`oDFTlN@R!7{Sr6v&>z!0bi#_EQhu z+O;^dZE-uO)diiaayA%D^ncEk&YfMF`XIoI+4(+(Xr0LYG}G8!TFf?W`)&O$bQn_H z`v;Xy+@+BRp+R??G(@ZWKiYcAE=`I8C?H@NyH^?VN4paMq(IY+F*Egb1~1r3fi62P z4_u}Iy6{Kfu^~VYq1y!B7y>jAn73g45TFge2(AwS;+P$e1qC@Vj4&9qwEK@H)CS^) f0a5hMvkDG_J%w1%Z1|(y07HfWaYl6aw&H&P+%ajC delta 20111 zcmY(ob8sNf7xo+5_QtktY;4=MIk9bLW1AZ%n`C3#n~iPb?)UfJ`&Qk5db;M+Om%n7 zdCqe_6V@$O+$Gjfhzbv-yhyZ&3+3rHv)q`7MgRoDAuUdZnO7 zQks`En4bpGylCfHQBIWKXG!zZunyr`g>^o3gTp zduq!(S>+!LZmzWz9Lsk~US155cSKd9creTye`BVk+0gULr27ksqJvGW7X0sNI zT{~AglLc&c$eFsO1k!!_s2{X-DF7z9l?^1#jaT3N^;@nJrMUeR$wM~9H9d*6<k4-@-cb)NNY zaN|wW(B=+iB(!_Zm5Z`^vRrWL{?&BZA(b!XHsUW8R+us|`?}=C_z$u~)__DN))ug{ zKA?_Qe0_Sg;D$2{C)s`=H&W?|kD2x)pT4abCl7hxOkUk$0ORWBmDj=B5`Ve%o9JJxgE-w)J>nEJk!Agtp8Se%L=%d)7uEdW=nghD% zz~V%u+K^?A)oi2+8{88{93WMq@LEjpZb(ig`sIM)Us!H84=+*Xd z`sBMJt2Taz&r%7|zO@=V-dk-oob_AbC+r`#f)lpKJj-bZ<)ECL0N&^u+zl&zv)EUk z#!(1COL}2q)+;r$L;#u8^hA!G(DJ5LZGKLvbT~_$1rDF3Gv+c4<_61Irw)Tb|JfGz zOUGNH(kiu$wKmee!kj|MF%1KkZ2R1st@8EjNj92^x$oTbEd#>cT8O|+%q6IEWngJ( z7f)4&`BjWfSbT=k{I>K@`>9u)szQ@ix;jsd(D1=f(!zF&A0T(oGKnB0(M$Goc^{ek z2vkfXa!Qp$SC%td%G1rcRlk~PQ`jh+0ahF)NvXyPIn){Dl&eYOTPk|=MFV5q{%d=y zFYZ#(MODa&Uv0#~kIr7M<;3+0lD>rK{W3qh%pe+bwb6|I@*U`w1QdA2PV)Qu&im{L z8>2eyC(`SX7+@Az`b+2CIJ8E8Dt8v!)ANPCd8my2Vbd})$Dt|1 z%4T^UpDrcwxz{A{F!UrV1C#|{Cvmd-IK2r{S+6}Tvk8KJ%l$MowywSYnZsp<^^8i;=$j5y2;j+x>$ z;8D>^ywOEl*5AldP({qPLA{yF6JssmWW{VHqPJKI{N7T-##k}co#Ez}#adbZ*y~!bdf2n_J254$`JAZBe&GK}sFjSx--r}<6 zm;h{N5_@c<$7^xkhy8uQ1n>~g)O;p$%!X?edOl2J6d+YvCnby3G$!PMBn^FCnHHEA zfqV^#U=zmJ;kmuTNF`eFVs&k9!yRayht*`j`y$4XjX5tD+BMVGY!I5GbEN|Cr1`w>q|GSg>kpUoWGoYD0Fzy1}$e zYD^LgZyx#x;oJ-q$HAFc2>IMxk^fy)`lRTR0x(VnOA@F0V?HI z_{=Dnp4r=8*lS+i-8F6E(w{Ffl*{KprDR%Iq|q}3#IEeWw>I{a7=s(y^Ia+Q^b4hpt2@CJ0E7;03HI@= z`!4@3+06@kYvLbhZboL<^_r8^KlK8fE;dr-Y zyA=Bpq#I?g$LSI0<5uPa{2i&w-l0lE)VekrDBx!7&y5I_jRXmq+(B`I-GhQ(5nd(U zz5NTR2Sy0V`$~2~?Mh(Blfl}63Ns&E@!vqqfWl=&Mr?;t0x6=?FBupIf)Ba_Bexvv z2!TL;9T(UEiVw;O)|=#+9}D(IH%2XI5}0snZ9NWqv>32*y9A6;d!fAceyTkMj8TL6 zUk4>C8x2*wG|jlM!cjtKf9W(&|0c=G!mM!LB|*oi`?tf*Fi;$!Z#;|!#8}~&k4SdWNc)L*(ZCo*Hib9;O|0-B z^h81e;LGJ6HxFCifD=?&Pr|&^b^OD%`vOFB=d3#1LQs)Gvrfs-mmtM-HhopBA4=dN zm@rXE=o@kJ`L+VF1q&{FCxjgFbwPLOYa4(Ba=M3Ap&KUs>OD2dq<^)chZwA`aqj=z zo1m3dI?qRpO$Lv=?zwG1sXGQEc)6N7qSI~Cuefz6uO8$D|%bLhP-M8d)GE~D8*cAnbo^1#AXZAof$2;EW z)lfhK{u`yQBL4j0b!jGLz<01vhBB6c%91o z)h&q4Htisk(xD$|WD2#+^!)%W1kz%G1f5s_N+cnzRuz>m(hrHsAd>-g`n*+SG^{#Z zsNe7G@yUz%7P_cixE#Ly7xj|zy&ZSFHf6SpIx{KAGJNVp|KvWEfwKquV0(~fOE|lW zi;IJ-H3MtgUVHT_i9jA6Iu~^yvzgv+a3~r}GBQkVB87z^lz2H8iUUBf@iJTxJ%?%q z-I^1g%3{KCKD?z|2!7RKF0bJs2?i?zU3kmFatpB*F|^ishDi#<@1UiJv*~kPZVJ^# zL5s6A)wB9+Hyz*e-YDYOwXGkPsx1FKX_xY_XfSDG?%DCrul>$ULXkPj5!uj05P8QD9-!m0kKg~B)y5LoxDQs3|xrgCe*Gr=}qEHqYEA*AR%G<=dmAiDoiiIx( z%ROG+;k#w%9w(?0@MezmPJCWX;*hV1A7XH%Aow2eVfKsFg#i7)pD3G+>;)0Vf8G2q zV%Hty^*+AnlPjaj+?vIqs}3$i!#P6uAW7a5<^*I={nHQZs@kxNljNjMZzfE-L|;J& z?=qK66|M0l#PA_M`V2~sy)|OH?@7{j1j>M#Zg7k_nJYhtgI|Y&m0?rAu@ovYR{gZr z{07?>o~g~`Y5)aKD1P$6EI)FT^>^Ps${pxu+Ck1Fo*S;mIv!2QL)W^$R@pvQ{^9OC zI1xX!au2JoC0LOc{#=$Hx?V#^oV4sGpcTN{7`{d;D35O)E#bf1)gCJ=;0jjtd5+~- z99o9W+;4G)g<=ePC4+9Cq<++nF< zayeQ*vbLepCoB1~@Sb|`AeKBGW^$w-Oul+{utAAJKODZzoiX-cbo)9JHe7O$r6Rrl z84lF@C9;>Uib!u*hhJ@Pk>!#b8tEOd1ujQ{>W|xp(DwSZB*jbhx_AxQ4EMiK>C$Fl! z4e&z*3jaN`_$GW;4hSf+9kHOA>+EL`-cMg}OA}!_XD2Z6_3a>ciTtLiW={B3Gnz6n zeN=M(es%}%ZBuRK`*Tf&38<0owPJG*mS=%~KnHBnaeU|37raxJ7z6G3U$L3GgzS*((X9=CA zY|A`2`{Nn#Ck&|(9<1gbO~jLbT0ZoZUsBAZp=agH?K!Z&*o&=MxilBL)dvXIKPYj~ zF8qA>!>p5V#7U?~dn6ONH)dRdyERhCu)f7joj_K=Z0L#Y%~&4Fh@6O8pp=mn@4H#r z8(_5Xc5OmAehT=j$Pv1`tf5x9b~+M1*giiu)+~;3xAh?Z{Tk20m1l#U{LSo&Q#z_r zr9Yxe`v`hIxwBzwk#I0_&gA@upVlWGm86p*6o(S=5|2G|f8y zrtaD@g&m~RjUElz!>W%&de8hty{fxzB~SS@`jrwchz#KOscU-XZTr~w>HTKWLu?=I zc2e1D6|yxa)wvD*x%3rRZafNQ_fgdxyw@>_-U@#CJ5f6u;!p4;Mb3k%?PN} zNTF&UYq>9T7s$O)sj=gtyf)nCEwInZ@savHQ^%)@eUuV~>{lzG%Bm_u&tm-5@kfQj#? zU$KXpW0Ujmv1Gl1V;A*Iix`)*?-X#&apsz(BJ~_UHy}Smr)Aeq+VbkrZ!_5rQ~F_E zKvV(o>*&NM@JRu-)=cNv9zs?F54*S6w$L5CCm~pl%Ir3uTMNxQ8HGOkUl{AUXvfJW z5=h+0I&CGdKyI|muO*uDC1TmZ?)24qeKi=dh*Qgb{SLJ)mMeaL&ls3Z{`&>gSi{-4 zWo^lrdhBKdSAQ8?uypTY`Vc)MT+`{$W@!KqF>F76T}@|G+r0RPMe*CUNK@htYi>GS znCw5%D-x2SVk*HcZISzBf^UP1q$V{cp%b-YIj#5o^~+tpn{KA>((`Qv$6ZbmR?w#u}tE?kP&wU{BHzpah~K}FjMsH? zJ2`fm9Yebqgtzz(>v>>yfL;T!n*O%@Qmpc5%eUnBk5UlB214>DiksEc|?Y18E zqy6M57sU0O(Wb?1LRZ~OK`R`sifUxWKBgk3&>G%VH~KE6tB|+zo1CCgw=2}_LQ13e zGs_E~shuOi>7}Tc95%3@(ClK#s+e9M!oo^}h(NAUKqQS@!TQ8tu2IsZ(*fF@Lu;j9 zQ86q|f9E=s2@%Zy5rL(8ScLx2m7xT%AE`0K_K1VHT53F7soC$52M=a@u95aEA~?1G&k(~+o2a70)8y3 zc)hD6Q%EVM5z-C%HsCz@lR&^*Di8kfUZ}92S}Qm)g5<_itZU#Ll;j37xY122==*lr zz%qS2J5-K#kS7sV4r?}d@esYwn@vm{RGpN*NrsY%Vz#4Uk7M_j9kiUiL^nys;pSNPj1U$}UcCHs8TOgZ|yl7tP*Pimuxo(nW zLDjqiOvQ)&R6X*YAyBH;u+Kd6JjlnZ#8i&Hd%01PYnfIY*W(g3fr#&Cxtn_?BI!YM^B!v=w|D0h7+En>EP zII()d)C*Pk*PC1R`Emv)(ks1MsRe$HxlNOu9zj%P(mq&Q5kM=gK*b2H4x9OERQ8!T za|m6dDVSz_Rov!mP=u*cw zYAa^mxTs{?xOjoX)XGKnz*to_lIp?Y`^O6Yo`;xt0 z(~U#jUol~3gH1sR;CHnGK^VkiiG|@!Ai-Mt9U>a=$!Vc~gF#r42*dxsO9>VfKT1{a;@nxfaC`xA3#+0uLI=gI~ToiSbY=NIw1fwN>Dg)3Tjy&Behd{mgO| zBXrNzw`3Vv?^fz$fa$^vm-T3=?M51uclcNFj=$>kSvq}Z%B_4U;JBj6n{?wPdxPR} zosVIrPV)8FithND;kQGV!>a-g5u!s+0N^Ya-Cs&FU4q3UV@2ojDHrOS8MTi^m z9~gk8v?~f$&pC3Rn=g)1c8&pVIwg6T3?Oy2fV}&$Jp!ts6dMtV{Gbl)iJ7^`BkvqwdKk+4a zC0Bn<;6Z7_He!0l;H`=gmzDvX2=Tw+Uw{D86@#;DAemi@3|w|zSOm-i+ymSeq6Sqf z8R}mc5A?rzWL%&RMB~2;NY?-qZi6H-=fBfvknG3v&f)-q^-SZuCe9Z6@q$)IHTvKx zf_^{+FF~>pqtFp+5HCY6Qw((qd-YC1q|Y_8%gXPK2J!04YM(68&|T4@(h|AXg;M9| ziXh_xQWW>S6Yjr)g)tg!lsq7#Hq`SNbHR-H>)j6xdN5&rS?)3wr-NzylIyC~Y8j*JjSJLwQCv6-6tf@@TW_OLts_8f;aQNTES@*%&I)=pc9Ngb zkNvO-O|UH(G$)&4|K+l*wj$@pK5ijf!Kcd5p*G5#K1BKg1f@ zmk2hUWok39YttzEjjlufb}~L|6I-vS2Tlfi-((MH!dalctD#=bX-MMxxGT z;)(9G;pARAXTiW7Oicw9&oC9(6$-4Vq(Tind45+W>1rb^tc5jvVCPj7{R!XlU{6}7 zc~^x~)`qBhjI;f{EZ2F5V`_nJ9iqKdmX3RdIh=6lt)P5OqhiodIKM0yJz^)7|z3BlNKUBztE{gE)Jux8uZ zxQV9u;zE`vS|LP)>&ZXrf12`;e8fmqAnAT+v=C`sq@AAkqnG&6tiPEDD*ah3(VO~p zCBafK_IJsPKDqHcAn6-%g$8i!j1(NsNmL)UCxgvcmqA@ebPMa**w_pv!(-^a@CtgU z;W!*?YS-)Y!1=JuHaguNXRbhF(;H;%;mu6#z>kySl+JCnW!Ok?TFg1G3B)iY1{e)o06fkh8FPIq7sL!|g z2+Z~n5Kf>T+to}zA&As-i)rLhnx}^UgyGM%zH52oR@cAzczXb&Nn0 z?TwQM$u>aDPE|Z6X0OaV#p13^a3Mle04)Snr^WhSwh~-=k!^hEp-UZ(N5tf5MJkEP z=|cjG1KV=3erq`&2)tza2?1psoCC!#ac8*Y2?GMss+NL6iUqLz4?F0cITVkgeF7zSNTXi@WlX~8_7<3YH@RsRGx5n; z(8nNcJTdr-i5qo~TZ--rh^r%PKc~(|B$EbfnbemI)2gFw0$yJN@_5>#=oj>I<7fFi z_sL1e|Ekd|N*M~y{I=Uq#l&WraY@RJCrWm?zk36|&dT>nCQYCLpP1Xwf_M#e^;|hh z^A?%qZodrds6fjYy*h8m3>Tg#ioOu#l{JX2Iw9>lwOF}-pULNoNs{E4H_=uNy7E%U zDEVJZC4;*8qLyOJ1l-z0Y1m)K+xX-?(9=cL^cugLp6OG>^P)D8n>uI(r9Yng+6zij zMu_W^$e!_rShF($=(4O{jP_TvrX_!C#h0-U(Yhs6FV{;&DdKnN2cYPZ;wcS%iL3b0 zq2;x<+`gsr*jk_wU5+<}w$5hHW34bfZ^G?oRW2LN9bW`{_XF#4{Q3l1+31~>Jbh8D zw?W4k(I6wNzgSy>-;6na%Uz0S`ceeBA>Zk~8#$(Zc1sfAGwvvqp2n*^67EIxg}iLu zI1Q5C&hP36g}#{Lv9arbFOij&h3h;&U87x#8*2KHNMX|;r$-5&m}4TEQuHT%lJ%rN zky#1@bg%+fyk%qd6-3Y2b zwSTAV_5utC#-?D6z!p=9b(LF_G)Y(3A5G3`zLr$EE)4I2cea7~NSJb3t!pukvF|Ji z0ZfERVu__C`eM!HnsIvW5ck$ph6OxatOA4|-U>IkNYmwiPk2O3dCi+y$Ske+pO_de zT;%|ofnz4jZ=^m=(|--^?%`MXP>*=y%nS}=PW4mk<&Sx3Zo9;>ysL^e_p?#7m?+^` zoc(vm*6S46wI8*l$kIK~0IH!lYp?YZ6w78 z@|7QT@3q$C6I^K82yjGKpG^`N&DH;mWV!?Rs!2Q3p!S$<-h?dOq7#n_;$U5wM%;*) zwX%M43Et^{#r}+yJNjb?=Zn8}kI2!Hbr2%=H#>qI>Ya~STUhv&x5j_{!pDryh$0Vj z6cyscm*YW4W5cIis_&6HMfDSOP?(WgG-|?HdOl`uDFjhN;#oquAufzbB|;~cpMJke#^ZH z+H1EG97}&$#PcUOL=U+@sEn|a{KElESsokrJ;FlWdEXzZT1wo}#~oRhX%?kUoKcGK z7R=m8#D&ILtm9cd_b`ZYI=#6$02O17W|B&D$d&=FY>BqI;~ArAxuhXY{*9UnkElB*k2 zot*ugT;ApCb6K*z2bp8aYSw`$xpJB;bStL4yylOVV0hgXTUuv3mT3PX)}^xPvc`|R zwXR{Ayp1g%3aLX(x2FM}pEhAw+B)yQ*C|ZZ)V;&2v4qvt0zr{iHAuP?jlvpQD^nZ$ zv7XK=&7xai@L(4yBzx55t7rjhPj9Y?pt=l>pKDXEEBtj=-r0oBhixyMV5zY3&G4vv zOJbmOs$*){0UQrXOqEql6fz&`>X+?&m-SUX;zyQh>grF1l~pRMO)y`Si#aNt`W5XI z?mbFRPhxrRFcp6TNyA0K-PNh9%Fh|90l44}C^of)a^W6A}6fzKo1}_VxryEww^!3PP~(zD-g~qB%u~*`U=g zXfFi<$p*jOD@u~v!21Nuf|3jvC21py?tz=xuin-Wg1(FI!;y2iV}bE4-FTvm<;IIM zCM%b#7{Ir4cU2m}u(@pY3Oz2gA1Lg%*I8%}h7l^+Yh&S6z!L?9*5BgeQcPoFm)=Pq zPD`~8V6GZEYg4LseajT;QB)Iboe^jI!ExH7@RFJ)_pAdQ`mg}hL;jSvdc%6Q%ktVb zz@69lu1#odTVhZ~w)i9>KN>)29LsEj<`waS4AE)3uFVg1`y$2GOf zKYURr#c>?3M5_ak3J48L(Iv|3oxAsZDuL!iN7Oz1)^ZWGVeGPDH_VjtmilqrFOVaIkIO`|&6BsTx* z@5A%maJD(r%Y{2BjR01%K5H|QrX4(l*i`muN`Q$xG@*6r_xKXblz9_UAUhBWUYmv2 z`8_5GoOw!Hc`hRAE`Y@q9r}BiRS-^*0@Q4^vP7Ok54VMYE@L%;qIPc;i#kJO zKLm|^A3YtTFM9w29=4VZfYbF_DQ)wswf!_{Yy0aGgin#(XULPW_3R}VBd|yZnVtxD z6h|FzBp}(3gqc%d{hjtXcL%m>&)$_vSRE(Sq4cl&w@&NFG|sla7Yx90msPhx^mvmn0MWx2BZG?YUaQ&D zlf2XL9s-eS4aC0H`flLX&XgUPD|Q8)9=3k7x3rX#O4li`SP|aPTYQ!@#q8A~ z5*PCP6#q&WN>6D%NLacmc@O$zeohh=LOcG-6?QHVRxPPFWd>N_d9=89lZGAS44 zEVAxt@RVyR`ujDP<3b6|QuN%_c96%&o!PY_hx<_xETMDMRtbu>x;r|yao4VY`y8UY zLZ0nSLrU6?^eiGe7?rzl*5p}wpD3;l{*PfttDYybg#ZEJLQRjTP@sve3NWJ< zfs_FYP9U?jOfj*7^6G3O(!V88h`x}!9oNWKNkFM`3g_Z2aj|XAj$#TZE@@yOqh9ub zTPx4@6|@XfPmxiav_?9`f>AN<9Y2&*g6r}5sJ1$sYGszmII}fQgTy&WwQql$jmk3a z;Us(tdez^dD4r5ko3(M&%4w_(X$v#NK0KDx_U6k%ZhpBvMP2jr{)}wdVA{Q8fUD7Q zlH8de5Xj3>VX-MBtR||rWAyq|*5O$33k;BTq?&UHZ`1au(=@6|u@%am202t#Bo~w! zOec-*VSh|{QxGwT92`Ac-6J1L;k6PSgICS>o<24*|;k zmaZ!vN3xN^-d%vy+y~FrahIKx%gD1J?fA!3oFLckER0xg2G$RS(i-JO5t1kj$d5=4 z0src;(#1r@wzx`lFPuR=#;aW_6{L?_HelWDw-u6|9yifowSaQsor6VR&SuVg#OF40 zLz&QNWpm~xpHt`{MD*AUD1G~B(di|O7s>+ljQ|2W-Jhl-urcHe*+~^6a0V$jCK)>v z9*3`E00jGmg_Xtw#URm-rW>RQ{TKe_A9*J*17CkHP_qawyla!(6bEZHzz#iv!Z@aP zygmNIA&aY8P&;O-|Jl9p>TmyEJ@q($?(izl1lYdZd%vd&MnpVz_$NfMv=h-Za5{OE90Owqs@UjL1wp7FO7+#{5$j|zz2k>jFKob*6VY04_%Fg^&v{K4s zbL&tl=|eN>Or8PuxV&a<4BqBh1CAnQKYO0cz|;{%(Lt`IU+fuGhOo)10wm zAKpFruFChL3-L=_;+O0lG{|iyRTodas*vz&QNrY<6Jkp*hY`u8>yLU}%I%~>8+(@$ zR_N0qISzCGmC{jSu*Ekaeub1IL@Y>1A2?{=6b4&9KpT4kdy$5e$ACVOQzS0{giIM# zH+nz0#kQS#c4z$~H$XZf)r`v;%L%0?5*rcuC>J;R?sclp*h)6FESRbL+HRj@5MEg& zpzm66v+lhzqGh0aNY1J0`cd51aoX_XMrWFXuKwD<4gn0I?#S!U(#WVc%5f1T;5$|RB;)i@@FP+tAQ9VZvMwWs_&MM`?!y?dJD0vvK`)_JbV$%FR1qwj|O#{q5@4x{aL06M=;z>epQpJwGq8!^;61ApY&b^th^ zlm`^%g_(0M!|557UCiq(?AebU;%9T!i7NL?PhMrk@-#C3hSQIwTN#W3_C2s?tyhMg z5Q&Mkwx-KeF086%TcC4#E=z&Fc|!bWd(NiF8Rbo`bu*{0prd z5I*D}Hw|+LEi&mnZFK{S8}wZo(I=*r1dhjY7YC$7{Wo_U0r{T;M2+VUvV9GSliW)I z1RvNh!_f}xrU^>7Urr)612RseK3O#_n}qUEN@_aex*my_VQ?EQ?qFxoaP72$+w*f? zBk8s(FxHO{N*%6S5=>F)%1vZ0Cw6m zWqYm9H6uvA`y||JB0PUeC!ATeqWA(dhYUud;>8~rSdCc z_!2?i)2HUC3oHV#DCvZ>rzt;&?Pv+#s-1Jmy*YbE|4ulzLBXeZ7e-c}eKNj9Gs^{3 z!{X~u7fP5qU^#m*^ONJjjy#%PmiJQv#3vyxcymi~rnufXtxk z*x0?@Np8K#m(fK73+a?WCWSuH!&ET?fTb`)W56|6i$gH}E2&yY|CQ7Xlr}=AfxC~=xO^;>B@gMX3gSq~yv}8F^!5qPwNcyD*>28Z;QC2!iLY*EtU}(vK7a@1PV(t5~C4N>iaVirU$H z&GAlC*N2T2;OqJEp^t69%DNLi@b$O@6==H9rZb?Hnr9br!YEgO3ywmMO&R@}{bM#O zGyo-b{@G^V5SYV;ML}Rym>k}(@9vrp+#M?|qi3Cl$eD#EOA&gx-(VKtxpg;i3EuiaO z;&4s@1);TylChQ4n~yS1NcD1%LTWWZS9-hMBWy>l-xqbvaxH0C9FB&6I@T2&0H%kl5lN5yFQPwd~RTZr%eyL`C>(bu4)%uS+vknOb*d@K4DL z{!{WN1YdQoi=H-qR0rC*dmxr)ri9d)fP@>yln!cp^fp}CNIx#oG0t!04$9e!!>L}V z`ptRw0Zrk)^v%)QS0B0C68lji-ASXdvg5nf+~)fRg)eEhSyt~mw1TUCU;pX;O~`1% zW`7(Wm-O1YhHN!~kjCo-=g~JO@Alc8lG&q8;XYcn*VXf+DU?#*H;A41iLd{_DrD#h z&kkYAB)|yBI&3hbY+)Z0!noItzG{6EE1--yD-#PXj)CGt#fK@8%g)DdkqXx>5m8Z{ zB%)ogO9#=npwhTlJa6X|N=E$7$u8nZs@k*L3xgIwM%B1(`uEG^>d@y9S)hd7f}$)p zDXN`foKNue{X{7ilVRpMy`DW7M=tJWNY{p8MC1n$O;Op%yBo~Bs_Ud}idcWkNKFwC zxXOW8WWa=IqsLPJkn!#2C`Lj(k%mcw!?+Wc6m}Tzy4su~HHK0UP5yfkaqPn3aKIuS7sY2xbE zU@h^0!K_1z#ba9S1-aKjz)_i3xAy3I><`FwIsWf!%lzQYlAL&$Qzngu4CxtHt5_2Y zo;Bc}!q`)7a%Y|_X9t@<<{Ep9O--{+ z(h~`o=}y5Dv3D25Y+JPRp3#%m>S%6_RgQ)y$!>-4ZrCs%b?lFvZh9@ocoNFIG9M>S z>vzS*XfJEhOgGIUyz<4yoY^DA`g63vuUyd1gyqw_g)62RMS!;8g<;Q@u8(L>s`u;)o#J$e6P^iZ|P*qUn%^8{_Os=awB2nZ4dr*`U?mtT=va% zaATN;$05Nh_Fsq)0I|yTc-wBn3g;9@*A!Gv-zMI$#Bn1Fh>1)u{}v&g3#mdGkwVNOP#ZtxS3sof*a3MvlOtrA{he zkKMypEdJ|%r3&%ohu`W9I0#4`0tg6c3aBndM&ddOD$t_26~E4n(Q9bEGcayoDf$Uc ztBQ(Kl~$HFC;GH4KWVfSrN5zMrCwRTG258nBO0%Fp%})n3_hv+r{h#4l}GHjDgq0&@J);G?Jlgfgo zQV8Q^w75@-CUfC@u8b3`G@>(f^x$jG z$39ysV)HKcV3{>e*jIVOAakDuA7lYd%gx4V4H4{Cq<2S&`B^N5Wyx-XJWCVhIfNJ8 z1Mc-5MOt}rgk_Q9i_Gc4*!A4cYd2?%k9i6#cM8sdZ2l^G*-=-z>QEV#f?et%z;13y zt-1WGrQ*CQ;^$Y2oniltWzq9L563jR&`)@fbQNDXy4hPW;=mOD%Xv4UIEVdXKCg-* zw8~nd?B1ASY-^k#xx6yjOXTon%mCGC_cnVdk5YA79`myzdaC%S>YpMe9sGim)hbjy z6l&{oP5ntvCjEOfb}sn`SM})@0LtW1;Lph9R19%kytq1bG$?&WC4xf!F_dSPl8p@1z=39ZaZ;EbRylP-FlprQGp?! zd6Qnw5Y~+jkMU8i~W<{{=$HJv7goly5dUjc0?y3B!R7|CL5RH{g z*E_2TS-;NLPW3EB3R(WRe(|&(bBmXgLM6Y89Z@I`KJ&WVx~xm~J~@QUkW$~+p}Rp| zdCtStCI;3EzfQ20leyY;6?+DN{-=wGqn{fPIsbBf8vg&w_1%BDo{|ua4AjIN!0#by z+;&noj4|4jU&V^oMkmnT<`xna z=11@@1Dl2G)Z?=rv?YyNW)d_RM-X1gF(Vkqd8WAHPzl&4_8 zf@TekXGtY>{Nn2rJ9-oj(m!H&wS<$c@z z#!Yu@q~!Dw0Fn_2RLqPA2wsWd%{ ze3IwFnF*yNK}^6e%3soFfR<=D7q2BmyRXQ-`G1|_KTjde;S=$nr(ks=0-3KUu9d_} zZaE-Kf1Z&V&5IsP_CNrq`coeko0o{}0=LDYmzdp*Rv~_bvhw|agpr0Im~lg374X&a zi{3lL1H}kNJW&`ae}yNT4ZjErFy@b?*2;pd$KzByXw@fuH1&~WG4N^*u0f;y79 zfW23h-Fn?Wi{C>U%IbeZEXajZthSt+@kc}RL_5?hsN__=s> z+1tFyiOG*X5>s!TjaX!d@uI1X_UfHG^hr*D_%YywbEKR}wNgCC^>muLTUCarJ^R*5 z-fnvbWD;Pfu}6-GWD=2rK}>&3DU2Blc6EjOf66!$aHzgFj*qcrzZy%KA%m!_jj?6T zn#oeiP870D2nk_AWfuk)sbncz^&=69P)w68vX3n>6bT{w|MGkOf9ijq=leY8p84GO zobx<)?z!*#o^uKD(7zSb&Fd&-`8ez@agmWXealCDLkL4P6}c8IaZr*eu+~FOt;5Nm z-8SnBe9}3u1Z+$H*M$@-P(Reh7=kr|Tu45V5P2Nc727Jh;9qKN<)q>nI#^Pk6U z!FwLjPR;$+?3-t+G%*Gbl9SjZMa-)^c(o2pH3<~^g?a68e-%_b zv5W)Z<@40Z9bSvw)cW{US;9j>UY~ERb=(i9!8%BxlIq_5{_*YSzi<0=1s@HxiOTUd z@F_GgJswBklZg)0Y+dB0cD=)X_t)_;d6>d2Ia2HN!MlDmRW*Z2*!XR$EW$d~c|t_e zx9cH)XRVD?XId-S|9)qf0-!5fU$+&QVs`CxQT!v**zTK&Zz81TZ0SFlxdzpC$+BSsJVRp25tp+FA0~BYiK)2kj3c~^{%%T>RfAb?77#%-`Sy^ z#igmxb%<;*#DnaUAxv)e%71Wt?O6FrQ@3oUlKu8Tl-)U%9P4NATN+;7;Ct@D(c4&5 zMB9B?`DP#Tympu3vI>fF$LHaEteeXr)9;?Pw%{DqN*oudcWd}oMTN^Ho*uWnjy8H+ zpB#a#OQ1H_8PNGnGKR8#x*D86B_rnN;(%W>EGE5>aQbxj71|t86GwIw3sis0Ep%9? z`q;&6HEeV71WV1we7q+n7loUg7e3HKsB_^D_7v=Y6QA-sapQn5InpAaOh!%8etbMA zgBG+$mjv76r_@Dn^KOpEt zQI{?q@ZS2Nb&YWv}j_Rh=+ zsP`tcA%OQjCj?4^OWfk=tb(($j${qcGAII9Ghtj@^uJCBHin8d8^M^P;j5}yvn2BU z?P}H&Sy}vc$uwuqaLG*5sP?Fr90~KbAU1ZfJ8y*n-%F%XO9$%FsRdFE)`|E6tTwKr zK$-(zS!2Z^U9CC|L-Q7InX@Ivcxtl|PKL{E`LUg(jYFQ)YVYO4(8Xt=wMO>sEz}gF z;T#NWY|Y4JrAr6NSBzA9shQb&9z}Yx^%2N}&aA4q-?DO=hK(1Jg_XV;>u3#;s-;F* zw6*6ds5(6HL3kTocFf~)2PAslw!z!>M)ar8mGc|YGs_0>M}&FolZ9WLex4QXQsO}d zN-79O(fO+4aN4n2;p%eHg;K7?ALcq9T|-tWhn)>AH*F9M!euD|PA^vMj>&rHuFDI@ zNEqh(y=XlYLW)t)y)a@e(9U>nfvo&1S3gE2wuCpZQ6kNffgOd{22&lhYZMl|fZMXl z6yts??F%=rHxXL9JngntW}u*oURu$7M@Uonl1-1z%QCss=!7H8sw!+k!kVoVugi}- zZCX{K_;LA|LC%P`n_Vy`m4=l5E|wgAM?GmN60X$7esmst*~wOF#nNWtTUkRfS&UqW zNnsQqndosLlcEN1de+6iW3s+NGu&VF%Zw`o{UyS^kxwn*o1d_lz7+ShV~0##=kg~` zISXC#H1?OOAW<_tm`=W94g1Lpf4xALtf`I#9h@#u;UTHvxvL z_Ho&X4t1n(Gw&IHv;7P?FkO8W@3F4jD~=N>2W*FT9-_7Dzc*)V z=`_Z23nhv~(JIF77pEEOo!7nUVtbc%m~ZVFm5n$VI5`wV6yth}%w;XgxjT#rc7a#( zHi4s~hMQEhaK!QV}g+vF59ufm2 zMo9Z0F+pO6!~%&G5*s9TNE{U!X~4{J`sY-0#-kM&GH{%RklD^-N_th84@8ve2$eba zqyz5!%B8mD(>=2yN6}^ZZr6=(d}(PObD9^GonFoR?8wq8?k;*36DbVW?Keno5wUvm z;zd#9Z$SgmGC7$jzmXjG%?+gW3F}Z6W|fmnV`67HHRT?E{s_m7GFr%6e0VCZ^S1p; zev8lH(QYzzKfxfYK~dnWKa)L%)LLjH?o<6Fq2}-(=|)-d=DlRBUdA zJ0~Jd)d~yl+eEgM=58FUK&(qtVq!qWeNw+y*NtWA6RjF(C=ao20fgz_6#%!riUr_v_ zN}cHnE0y*oGRQKx)O#2&7NGE)oj*0|*YX%)LLhFvCE-bV@vXj#kF>6n*MEIK_Oe3g zZHVMuBHnZ#<3WK+k1Ma~?u{*v`nEjesmBWr_zWH3{nokEE^y<44f_{=MZ8h3n)z`4 z2KNMECq~y6Y7%j|BB2Ekqak>N$#C771*iE>&6dz3sC;i7WiPb6bCj_5>q!=t4^J&~ z&i1Sgi>?YN`8cuF+B9XC)c5RJn54y%SR;MxB@vjU;UnMlq>3O;BmI#xlbYvswQx%# zB#GCxryU#eO8zN*OcTRwGh*0(wJ+B(yw?kNpQ0g2AtCxB+#TFQH(dy)`}eIJ#$_FM zKv>a6Yi4ELm%%d#D;;A|I&#m`dNEAXBtMHGE?ze$@_{9t_3L}I!QLmknU~op+Qy?A zvzdHv%dhCoywcm)C1R>?e| zv#gz4B&2S@D!AK$JUH|2_piWr9l$x74DdiFpb6*25EC)HU}G-;p!V#9g>l379z3eV z1XW(Jyb}PByX#hAe^K$^=(b5WGh*mP5hA-eGsxEg03e|o;DdXaZ&!jjp8+h~&tj*% zv0VU@rH1VKDH(&`*w%y@syCE*)!gg>RL6o#Z z{tsjY)q4PT_^REGiHG*gU`P*e3{G>^xh%E_FaM~#CDX2 z1r+WDxKX?L|G!bz%e$!eHoB8igsg&)1No=ba<^Ty4nmQ;DUiQWFL16Gz@q-l0POAJ zp!fEWWl`HXfN3a;f<5@R3Gu+))}Hl29ovbGLmjJv79RiEF*Cw0y0MM!1_%E}OT%_i zgMJA8BR00Thp?gB|3%TLKfLNcsF|o83Uad1sO|UpWN0u)_W#{1DQ<_782|)PJJUEs zRsJ7!FX_K6vVhS8fHeF82vvv>p^59?`(kAPx*c=VKyC`40q=hRnp1$2GNGI$!XO}m+L_j%I4Oz&27`8r?>XLl03QqjLdc!x84V0p22q@Q Reuu6B2Otv+m2KM>{s$7n9Wej^ diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 593256603f92a..79493968db274 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -620,7 +620,7 @@ df.groupBy("type").count() ### Window Operations on Event Time Aggregations over a sliding event-time window are straightforward with Structured Streaming. The key idea to understand about window-based aggregations are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration. -Imagine the quick example is modified and the stream contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). +Imagine our quick example is modified and the stream now contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). The result tables would look something like the following. From 4dc7d377fba39147d8820a5a2866a2fbcb73db98 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 30 Jun 2016 16:49:59 -0700 Subject: [PATCH 0865/1470] [SPARK-16336][SQL] Suggest doing table refresh upon FileNotFoundException ## What changes were proposed in this pull request? This patch appends a message to suggest users running refresh table or reloading data frames when Spark sees a FileNotFoundException due to stale, cached metadata. ## How was this patch tested? Added a unit test for this in MetadataCacheSuite. Author: petermaxlee Closes #14003 from petermaxlee/SPARK-16336. (cherry picked from commit fb41670c9263a89ec233861cc91a19cf1bb19073) Signed-off-by: Reynold Xin --- .../execution/datasources/FileScanRDD.scala | 15 +++- .../apache/spark/sql/MetadataCacheSuite.scala | 88 +++++++++++++++++++ 2 files changed, 102 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index f7f68b1eb90da..1314c94d42cf9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -111,7 +111,20 @@ class FileScanRDD( currentFile = files.next() logInfo(s"Reading File $currentFile") InputFileNameHolder.setInputFileName(currentFile.filePath) - currentIterator = readFunction(currentFile) + + try { + currentIterator = readFunction(currentFile) + } catch { + case e: java.io.FileNotFoundException => + throw new java.io.FileNotFoundException( + e.getMessage + "\n" + + "It is possible the underlying files have been updated. " + + "You can explicitly invalidate the cache in Spark by " + + "running 'REFRESH TABLE tableName' command in SQL or " + + "by recreating the Dataset/DataFrame involved." + ) + } + hasNext } else { currentFile = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala new file mode 100644 index 0000000000000..d872f4baa6218 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.io.File + +import org.apache.spark.SparkException +import org.apache.spark.sql.test.SharedSQLContext + +/** + * Test suite to handle metadata cache related. + */ +class MetadataCacheSuite extends QueryTest with SharedSQLContext { + + /** Removes one data file in the given directory. */ + private def deleteOneFileInDirectory(dir: File): Unit = { + assert(dir.isDirectory) + val oneFile = dir.listFiles().find { file => + !file.getName.startsWith("_") && !file.getName.startsWith(".") + } + assert(oneFile.isDefined) + oneFile.foreach(_.delete()) + } + + test("SPARK-16336 Suggest doing table refresh when encountering FileNotFoundException") { + withTempPath { (location: File) => + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.parquet(location.getAbsolutePath) + + // Read the directory in + val df = spark.read.parquet(location.getAbsolutePath) + assert(df.count() == 100) + + // Delete a file + deleteOneFileInDirectory(location) + + // Read it again and now we should see a FileNotFoundException + val e = intercept[SparkException] { + df.count() + } + assert(e.getMessage.contains("FileNotFoundException")) + assert(e.getMessage.contains("REFRESH")) + } + } + + ignore("SPARK-16337 temporary view refresh") { + withTempPath { (location: File) => + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.parquet(location.getAbsolutePath) + + // Read the directory in + spark.read.parquet(location.getAbsolutePath).createOrReplaceTempView("view_refresh") + assert(sql("select count(*) from view_refresh").first().getLong(0) == 100) + + // Delete a file + deleteOneFileInDirectory(location) + + // Read it again and now we should see a FileNotFoundException + val e = intercept[SparkException] { + sql("select count(*) from view_refresh").first() + } + assert(e.getMessage.contains("FileNotFoundException")) + assert(e.getMessage.contains("refresh()")) + + // Refresh and we should be able to read it again. + spark.catalog.refreshTable("view_refresh") + val newCount = sql("select count(*) from view_refresh").first().getLong(0) + assert(newCount > 0 && newCount < 100) + } + } +} From 17c7522c8cb8f400408cbdc3b8b1251bbca53eec Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 Jun 2016 16:51:11 -0700 Subject: [PATCH 0866/1470] [SPARK-16313][SQL] Spark should not silently drop exceptions in file listing ## What changes were proposed in this pull request? Spark silently drops exceptions during file listing. This is a very bad behavior because it can mask legitimate errors and the resulting plan will silently have 0 rows. This patch changes it to not silently drop the errors. ## How was this patch tested? Manually verified. Author: Reynold Xin Closes #13987 from rxin/SPARK-16313. (cherry picked from commit 3d75a5b2a76eba0855d73476dc2fd579c612d521) Signed-off-by: Reynold Xin --- python/pyspark/sql/context.py | 2 +- python/pyspark/sql/streaming.py | 2 +- .../execution/datasources/DataSource.scala | 3 ++- .../datasources/ListingFileCatalog.scala | 22 ++++++++++++++----- .../datasources/fileSourceInterfaces.scala | 11 ++++++---- .../sql/streaming/FileStreamSourceSuite.scala | 2 +- 6 files changed, 29 insertions(+), 13 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 3503fb90c3f8d..8c984b36b79e5 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -440,7 +440,7 @@ def readStream(self): :return: :class:`DataStreamReader` - >>> text_sdf = sqlContext.readStream.text(os.path.join(tempfile.mkdtemp(), 'data')) + >>> text_sdf = sqlContext.readStream.text(tempfile.mkdtemp()) >>> text_sdf.isStreaming True """ diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 8cf70983a4514..bffe398247ba9 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -437,7 +437,7 @@ def text(self, path): :param paths: string, or list of strings, for input path(s). - >>> text_sdf = spark.readStream.text(os.path.join(tempfile.mkdtemp(), 'data')) + >>> text_sdf = spark.readStream.text(tempfile.mkdtemp()) >>> text_sdf.isStreaming True >>> "value" in str(text_sdf.schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 557445c2bc91f..a4110d7b11473 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -364,7 +364,8 @@ case class DataSource( } val fileCatalog = - new ListingFileCatalog(sparkSession, globbedPaths, options, partitionSchema) + new ListingFileCatalog( + sparkSession, globbedPaths, options, partitionSchema, !checkPathExist) val dataSchema = userSpecifiedSchema.map { schema => val equality = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala index 675e755cb2d02..706ec6b9b36c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources +import java.io.FileNotFoundException + import scala.collection.mutable import scala.util.Try @@ -35,12 +37,16 @@ import org.apache.spark.sql.types.StructType * @param paths a list of paths to scan * @param partitionSchema an optional partition schema that will be use to provide types for the * discovered partitions + * @param ignoreFileNotFound if true, return empty file list when encountering a + * [[FileNotFoundException]] in file listing. Note that this is a hack + * for SPARK-16313. We should get rid of this flag in the future. */ class ListingFileCatalog( sparkSession: SparkSession, override val paths: Seq[Path], parameters: Map[String, String], - partitionSchema: Option[StructType]) + partitionSchema: Option[StructType], + ignoreFileNotFound: Boolean = false) extends PartitioningAwareFileCatalog(sparkSession, parameters, partitionSchema) { @volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _ @@ -77,10 +83,12 @@ class ListingFileCatalog( * List leaf files of given paths. This method will submit a Spark job to do parallel * listing whenever there is a path having more files than the parallel partition discovery * discovery threshold. + * + * This is publicly visible for testing. */ - protected[spark] def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession) + HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sparkSession, ignoreFileNotFound) } else { // Right now, the number of paths is less than the value of // parallelPartitionDiscoveryThreshold. So, we will list file statues at the driver. @@ -96,8 +104,12 @@ class ListingFileCatalog( logTrace(s"Listing $path on driver") val childStatuses = { - // TODO: We need to avoid of using Try at here. - val stats = Try(fs.listStatus(path)).getOrElse(Array.empty[FileStatus]) + val stats = + try { + fs.listStatus(path) + } catch { + case e: FileNotFoundException if ignoreFileNotFound => Array.empty[FileStatus] + } if (pathFilter != null) stats.filter(f => pathFilter.accept(f.getPath)) else stats } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 20399e190f430..0b5a19fe9384b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -439,7 +439,8 @@ private[sql] object HadoopFsRelation extends Logging { def listLeafFilesInParallel( paths: Seq[Path], hadoopConf: Configuration, - sparkSession: SparkSession): mutable.LinkedHashSet[FileStatus] = { + sparkSession: SparkSession, + ignoreFileNotFound: Boolean): mutable.LinkedHashSet[FileStatus] = { assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") @@ -460,9 +461,11 @@ private[sql] object HadoopFsRelation extends Logging { val pathFilter = FileInputFormat.getInputPathFilter(jobConf) paths.map(new Path(_)).flatMap { path => val fs = path.getFileSystem(serializableConfiguration.value) - // TODO: We need to avoid of using Try at here. - Try(listLeafFiles(fs, fs.getFileStatus(path), pathFilter)) - .getOrElse(Array.empty[FileStatus]) + try { + listLeafFiles(fs, fs.getFileStatus(path), pathFilter) + } catch { + case e: java.io.FileNotFoundException if ignoreFileNotFound => Array.empty[FileStatus] + } } }.map { status => val blockLocations = status match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 0eade71d1ebc3..6c04846f00e8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -225,7 +225,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { // =============== Parquet file stream schema tests ================ - test("FileStreamSource schema: parquet, no existing files, no schema") { + ignore("FileStreamSource schema: parquet, no existing files, no schema") { withTempDir { src => withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { val e = intercept[AnalysisException] { From d3027c45fbe02752d260aefff9dae707ba5c5d4c Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 30 Jun 2016 17:52:15 -0700 Subject: [PATCH 0867/1470] [SPARK-16328][ML][MLLIB][PYSPARK] Add 'asML' and 'fromML' conversion methods to PySpark linalg The move to `ml.linalg` created `asML`/`fromML` utility methods in Scala/Java for converting between representations. These are missing in Python, this PR adds them. ## How was this patch tested? New doctests. Author: Nick Pentreath Closes #13997 from MLnick/SPARK-16328-python-linalg-convert. (cherry picked from commit dab10516138867b7c4fc6d42168497e82853b539) Signed-off-by: Joseph K. Bradley --- python/pyspark/mllib/linalg/__init__.py | 99 +++++++++++++++++++++++++ python/pyspark/mllib/tests.py | 69 +++++++++++++++++ 2 files changed, 168 insertions(+) diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index 3a345b2b5638c..15dc53a959d6d 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -39,6 +39,7 @@ import numpy as np from pyspark import since +from pyspark.ml import linalg as newlinalg from pyspark.sql.types import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ IntegerType, ByteType, BooleanType @@ -247,6 +248,15 @@ def toArray(self): """ raise NotImplementedError + def asML(self): + """ + Convert this vector to the new mllib-local representation. + This does NOT copy the data; it copies references. + + :return: :py:class:`pyspark.ml.linalg.Vector` + """ + raise NotImplementedError + class DenseVector(Vector): """ @@ -408,6 +418,17 @@ def toArray(self): """ return self.array + def asML(self): + """ + Convert this vector to the new mllib-local representation. + This does NOT copy the data; it copies references. + + :return: :py:class:`pyspark.ml.linalg.DenseVector` + + .. versionadded:: 2.0.0 + """ + return newlinalg.DenseVector(self.array) + @property def values(self): """ @@ -737,6 +758,17 @@ def toArray(self): arr[self.indices] = self.values return arr + def asML(self): + """ + Convert this vector to the new mllib-local representation. + This does NOT copy the data; it copies references. + + :return: :py:class:`pyspark.ml.linalg.SparseVector` + + .. versionadded:: 2.0.0 + """ + return newlinalg.SparseVector(self.size, self.indices, self.values) + def __len__(self): return self.size @@ -845,6 +877,24 @@ def dense(*elements): elements = elements[0] return DenseVector(elements) + @staticmethod + def fromML(vec): + """ + Convert a vector from the new mllib-local representation. + This does NOT copy the data; it copies references. + + :param vec: a :py:class:`pyspark.ml.linalg.Vector` + :return: a :py:class:`pyspark.mllib.linalg.Vector` + + .. versionadded:: 2.0.0 + """ + if isinstance(vec, newlinalg.DenseVector): + return DenseVector(vec.array) + elif isinstance(vec, newlinalg.SparseVector): + return SparseVector(vec.size, vec.indices, vec.values) + else: + raise TypeError("Unsupported vector type %s" % type(vec)) + @staticmethod def stringify(vector): """ @@ -945,6 +995,13 @@ def toArray(self): """ raise NotImplementedError + def asML(self): + """ + Convert this matrix to the new mllib-local representation. + This does NOT copy the data; it copies references. + """ + raise NotImplementedError + @staticmethod def _convert_to_array(array_like, dtype): """ @@ -1044,6 +1101,17 @@ def toSparse(self): return SparseMatrix(self.numRows, self.numCols, colPtrs, rowIndices, values) + def asML(self): + """ + Convert this matrix to the new mllib-local representation. + This does NOT copy the data; it copies references. + + :return: :py:class:`pyspark.ml.linalg.DenseMatrix` + + .. versionadded:: 2.0.0 + """ + return newlinalg.DenseMatrix(self.numRows, self.numCols, self.values, self.isTransposed) + def __getitem__(self, indices): i, j = indices if i < 0 or i >= self.numRows: @@ -1216,6 +1284,18 @@ def toDense(self): densevals = np.ravel(self.toArray(), order='F') return DenseMatrix(self.numRows, self.numCols, densevals) + def asML(self): + """ + Convert this matrix to the new mllib-local representation. + This does NOT copy the data; it copies references. + + :return: :py:class:`pyspark.ml.linalg.SparseMatrix` + + .. versionadded:: 2.0.0 + """ + return newlinalg.SparseMatrix(self.numRows, self.numCols, self.colPtrs, self.rowIndices, + self.values, self.isTransposed) + # TODO: More efficient implementation: def __eq__(self, other): return np.all(self.toArray() == other.toArray()) @@ -1236,6 +1316,25 @@ def sparse(numRows, numCols, colPtrs, rowIndices, values): """ return SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + @staticmethod + def fromML(mat): + """ + Convert a matrix from the new mllib-local representation. + This does NOT copy the data; it copies references. + + :param mat: a :py:class:`pyspark.ml.linalg.Matrix` + :return: a :py:class:`pyspark.mllib.linalg.Matrix` + + .. versionadded:: 2.0.0 + """ + if isinstance(mat, newlinalg.DenseMatrix): + return DenseMatrix(mat.numRows, mat.numCols, mat.values, mat.isTransposed) + elif isinstance(mat, newlinalg.SparseMatrix): + return SparseMatrix(mat.numRows, mat.numCols, mat.colPtrs, mat.rowIndices, + mat.values, mat.isTransposed) + else: + raise TypeError("Unsupported matrix type %s" % type(mat)) + class QRDecomposition(object): """ diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 74cf7bb8eaf9d..72fa8b5f3d477 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -49,6 +49,7 @@ import unittest from pyspark import SparkContext +import pyspark.ml.linalg as newlinalg from pyspark.mllib.common import _to_java_object_rdd from pyspark.mllib.clustering import StreamingKMeans, StreamingKMeansModel from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ @@ -423,6 +424,74 @@ def test_norms(self): tmp = SparseVector(4, [0, 2], [3, 0]) self.assertEqual(tmp.numNonzeros(), 1) + def test_ml_mllib_vector_conversion(self): + # to ml + # dense + mllibDV = Vectors.dense([1, 2, 3]) + mlDV1 = newlinalg.Vectors.dense([1, 2, 3]) + mlDV2 = mllibDV.asML() + self.assertEqual(mlDV2, mlDV1) + # sparse + mllibSV = Vectors.sparse(4, {1: 1.0, 3: 5.5}) + mlSV1 = newlinalg.Vectors.sparse(4, {1: 1.0, 3: 5.5}) + mlSV2 = mllibSV.asML() + self.assertEqual(mlSV2, mlSV1) + # from ml + # dense + mllibDV1 = Vectors.dense([1, 2, 3]) + mlDV = newlinalg.Vectors.dense([1, 2, 3]) + mllibDV2 = Vectors.fromML(mlDV) + self.assertEqual(mllibDV1, mllibDV2) + # sparse + mllibSV1 = Vectors.sparse(4, {1: 1.0, 3: 5.5}) + mlSV = newlinalg.Vectors.sparse(4, {1: 1.0, 3: 5.5}) + mllibSV2 = Vectors.fromML(mlSV) + self.assertEqual(mllibSV1, mllibSV2) + + def test_ml_mllib_matrix_conversion(self): + # to ml + # dense + mllibDM = Matrices.dense(2, 2, [0, 1, 2, 3]) + mlDM1 = newlinalg.Matrices.dense(2, 2, [0, 1, 2, 3]) + mlDM2 = mllibDM.asML() + self.assertEqual(mlDM2, mlDM1) + # transposed + mllibDMt = DenseMatrix(2, 2, [0, 1, 2, 3], True) + mlDMt1 = newlinalg.DenseMatrix(2, 2, [0, 1, 2, 3], True) + mlDMt2 = mllibDMt.asML() + self.assertEqual(mlDMt2, mlDMt1) + # sparse + mllibSM = Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + mlSM1 = newlinalg.Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + mlSM2 = mllibSM.asML() + self.assertEqual(mlSM2, mlSM1) + # transposed + mllibSMt = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4], True) + mlSMt1 = newlinalg.SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4], True) + mlSMt2 = mllibSMt.asML() + self.assertEqual(mlSMt2, mlSMt1) + # from ml + # dense + mllibDM1 = Matrices.dense(2, 2, [1, 2, 3, 4]) + mlDM = newlinalg.Matrices.dense(2, 2, [1, 2, 3, 4]) + mllibDM2 = Matrices.fromML(mlDM) + self.assertEqual(mllibDM1, mllibDM2) + # transposed + mllibDMt1 = DenseMatrix(2, 2, [1, 2, 3, 4], True) + mlDMt = newlinalg.DenseMatrix(2, 2, [1, 2, 3, 4], True) + mllibDMt2 = Matrices.fromML(mlDMt) + self.assertEqual(mllibDMt1, mllibDMt2) + # sparse + mllibSM1 = Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + mlSM = newlinalg.Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]) + mllibSM2 = Matrices.fromML(mlSM) + self.assertEqual(mllibSM1, mllibSM2) + # transposed + mllibSMt1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4], True) + mlSMt = newlinalg.SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4], True) + mllibSMt2 = Matrices.fromML(mlSMt) + self.assertEqual(mllibSMt1, mllibSMt2) + class ListTests(MLlibTestCase): From 79c96c99977b0478c25b13583a3e88cbab541ba6 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 30 Jun 2016 17:55:14 -0700 Subject: [PATCH 0868/1470] [SPARK-15643][DOC][ML] Add breaking changes to ML migration guide This PR adds the breaking changes from [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810) to the migration guide. ## How was this patch tested? Built docs locally. Author: Nick Pentreath Closes #13924 from MLnick/SPARK-15643-migration-guide. (cherry picked from commit 4a981dc870a31d8b90aac5f6cb22884e02f6fbc6) Signed-off-by: Joseph K. Bradley --- docs/mllib-guide.md | 104 ++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 101 insertions(+), 3 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index c28d13732eed8..17fd3e1edf4b4 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -104,9 +104,105 @@ and the migration guide below will explain all changes between releases. ## From 1.6 to 2.0 -The deprecations and changes of behavior in the `spark.mllib` or `spark.ml` packages include: +### Breaking changes -Deprecations: +There were several breaking changes in Spark 2.0, which are outlined below. + +**Linear algebra classes for DataFrame-based APIs** + +Spark's linear algebra dependencies were moved to a new project, `mllib-local` +(see [SPARK-13944](https://issues.apache.org/jira/browse/SPARK-13944)). +As part of this change, the linear algebra classes were copied to a new package, `spark.ml.linalg`. +The DataFrame-based APIs in `spark.ml` now depend on the `spark.ml.linalg` classes, +leading to a few breaking changes, predominantly in various model classes +(see [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810) for a full list). + +**Note:** the RDD-based APIs in `spark.mllib` continue to depend on the previous package `spark.mllib.linalg`. + +_Converting vectors and matrices_ + +While most pipeline components support backward compatibility for loading, +some existing `DataFrames` and pipelines in Spark versions prior to 2.0, that contain vector or matrix +columns, may need to be migrated to the new `spark.ml` vector and matrix types. +Utilities for converting `DataFrame` columns from `spark.mllib.linalg` to `spark.ml.linalg` types +(and vice versa) can be found in `spark.mllib.util.MLUtils`. + +There are also utility methods available for converting single instances of +vectors and matrices. Use the `asML` method on a `mllib.linalg.Vector` / `mllib.linalg.Matrix` +for converting to `ml.linalg` types, and +`mllib.linalg.Vectors.fromML` / `mllib.linalg.Matrices.fromML` +for converting to `mllib.linalg` types. + +

    +
    + +{% highlight scala %} +import org.apache.spark.mllib.util.MLUtils + +// convert DataFrame columns +val convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) +val convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) +// convert a single vector or matrix +val mlVec: org.apache.spark.ml.linalg.Vector = mllibVec.asML +val mlMat: org.apache.spark.ml.linalg.Matrix = mllibMat.asML +{% endhighlight %} + +Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) for further detail. +
    + +
    + +{% highlight java %} +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.sql.Dataset; + +// convert DataFrame columns +Dataset convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF); +Dataset convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF); +// convert a single vector or matrix +org.apache.spark.ml.linalg.Vector mlVec = mllibVec.asML(); +org.apache.spark.ml.linalg.Matrix mlMat = mllibMat.asML(); +{% endhighlight %} + +Refer to the [`MLUtils` Java docs](api/java/org/apache/spark/mllib/util/MLUtils.html) for further detail. +
    + +
    + +{% highlight python %} +from pyspark.mllib.util import MLUtils + +# convert DataFrame columns +convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) +convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) +# convert a single vector or matrix +mlVec = mllibVec.asML() +mlMat = mllibMat.asML() +{% endhighlight %} + +Refer to the [`MLUtils` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.util.MLUtils) for further detail. +
    +
    + +**Deprecated methods removed** + +Several deprecated methods were removed in the `spark.mllib` and `spark.ml` packages: + +* `setScoreCol` in `ml.evaluation.BinaryClassificationEvaluator` +* `weights` in `LinearRegression` and `LogisticRegression` in `spark.ml` +* `setMaxNumIterations` in `mllib.optimization.LBFGS` (marked as `DeveloperApi`) +* `treeReduce` and `treeAggregate` in `mllib.rdd.RDDFunctions` (these functions are available on `RDD`s directly, and were marked as `DeveloperApi`) +* `defaultStategy` in `mllib.tree.configuration.Strategy` +* `build` in `mllib.tree.Node` +* libsvm loaders for multiclass and load/save labeledData methods in `mllib.util.MLUtils` + +A full list of breaking changes can be found at [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810). + +### Deprecations and changes of behavior + +**Deprecations** + +Deprecations in the `spark.mllib` and `spark.ml` packages include: * [SPARK-14984](https://issues.apache.org/jira/browse/SPARK-14984): In `spark.ml.regression.LinearRegressionSummary`, the `model` field has been deprecated. @@ -125,7 +221,9 @@ Deprecations: In `spark.ml.util.MLReader` and `spark.ml.util.MLWriter`, the `context` method has been deprecated in favor of `session`. * In `spark.ml.feature.ChiSqSelectorModel`, the `setLabelCol` method has been deprecated since it was not used by `ChiSqSelectorModel`. -Changes of behavior: +**Changes of behavior** + +Changes of behavior in the `spark.mllib` and `spark.ml` packages include: * [SPARK-7780](https://issues.apache.org/jira/browse/SPARK-7780): `spark.mllib.classification.LogisticRegressionWithLBFGS` directly calls `spark.ml.classification.LogisticRegresson` for binary classification now. From 94d61de9cdb773c7f3e0ed8909eddcbb208afaa9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 Jun 2016 19:02:35 -0700 Subject: [PATCH 0869/1470] [SPARK-15954][SQL] Disable loading test tables in Python tests ## What changes were proposed in this pull request? This patch introduces a flag to disable loading test tables in TestHiveSparkSession and disables that in Python. This fixes an issue in which python/run-tests would fail due to failure to load test tables. Note that these test tables are not used outside of HiveCompatibilitySuite. In the long run we should probably decouple the loading of test tables from the test Hive setup. ## How was this patch tested? This is a test only change. Author: Reynold Xin Closes #14005 from rxin/SPARK-15954. (cherry picked from commit 38f4d6f44eaa03bdc703662e4a7be9c09ba86e16) Signed-off-by: Reynold Xin --- python/pyspark/sql/context.py | 2 +- .../apache/spark/sql/hive/test/TestHive.scala | 344 ++++++++++-------- 2 files changed, 185 insertions(+), 161 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 8c984b36b79e5..4cfdf799f6f42 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -492,7 +492,7 @@ def _createForTesting(cls, sparkContext): confusing error messages. """ jsc = sparkContext._jsc.sc() - jtestHive = sparkContext._jvm.org.apache.spark.sql.hive.test.TestHiveContext(jsc) + jtestHive = sparkContext._jvm.org.apache.spark.sql.hive.test.TestHiveContext(jsc, False) return cls(sparkContext, jtestHive) def refreshTable(self, tableName): 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 b45be0251d953..7f892047c7075 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 @@ -73,8 +73,12 @@ class TestHiveContext( @transient override val sparkSession: TestHiveSparkSession) extends SQLContext(sparkSession) { - def this(sc: SparkContext) { - this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc))) + /** + * If loadTestTables is false, no test tables are loaded. Note that this flag can only be true + * when running in the JVM, i.e. it needs to be false when calling from Python. + */ + def this(sc: SparkContext, loadTestTables: Boolean = true) { + this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc), loadTestTables)) } override def newSession(): TestHiveContext = { @@ -103,13 +107,24 @@ class TestHiveContext( } - +/** + * A [[SparkSession]] used in [[TestHiveContext]]. + * + * @param sc SparkContext + * @param warehousePath path to the Hive warehouse directory + * @param scratchDirPath scratch directory used by Hive's metastore client + * @param metastoreTemporaryConf configuration options for Hive's metastore + * @param existingSharedState optional [[TestHiveSharedState]] + * @param loadTestTables if true, load the test tables. They can only be loaded when running + * in the JVM, i.e when calling from Python this flag has to be false. + */ private[hive] class TestHiveSparkSession( @transient private val sc: SparkContext, val warehousePath: File, scratchDirPath: File, metastoreTemporaryConf: Map[String, String], - @transient private val existingSharedState: Option[TestHiveSharedState]) + @transient private val existingSharedState: Option[TestHiveSharedState], + private val loadTestTables: Boolean) extends SparkSession(sc) with Logging { self => // TODO: We need to set the temp warehouse path to sc's conf. @@ -118,13 +133,14 @@ private[hive] class TestHiveSparkSession( // when we creating metadataHive. This flow is not easy to follow and can introduce // confusion when a developer is debugging an issue. We need to refactor this part // to just set the temp warehouse path in sc's conf. - def this(sc: SparkContext) { + def this(sc: SparkContext, loadTestTables: Boolean) { this( sc, Utils.createTempDir(namePrefix = "warehouse"), TestHiveContext.makeScratchDir(), HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false), - None) + None, + loadTestTables) } assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") @@ -144,7 +160,7 @@ private[hive] class TestHiveSparkSession( override def newSession(): TestHiveSparkSession = { new TestHiveSparkSession( - sc, warehousePath, scratchDirPath, metastoreTemporaryConf, Some(sharedState)) + sc, warehousePath, scratchDirPath, metastoreTemporaryConf, Some(sharedState), loadTestTables) } private var cacheTables: Boolean = false @@ -204,165 +220,173 @@ private[hive] class TestHiveSparkSession( testTables += (testTable.name -> testTable) } - // The test tables that are defined in the Hive QTestUtil. - // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java - // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql - @transient - val hiveQTestUtilTables = Seq( - TestTable("src", - "CREATE TABLE src (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), - TestTable("src1", - "CREATE TABLE src1 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), - TestTable("srcpart", () => { - sql( - "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + if (loadTestTables) { + // The test tables that are defined in the Hive QTestUtil. + // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql + @transient + val hiveQTestUtilTables: Seq[TestTable] = Seq( + TestTable("src", + "CREATE TABLE src (key INT, value STRING)".cmd, + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), + TestTable("src1", + "CREATE TABLE src1 (key INT, value STRING)".cmd, + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), + TestTable("srcpart", () => { sql( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' - |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') - """.stripMargin) - } - }), - TestTable("srcpart1", () => { - sql( - "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + sql( + s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.stripMargin) + } + }), + TestTable("srcpart1", () => { sql( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' - |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') - """.stripMargin) - } - }), - TestTable("src_thrift", () => { - import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer - import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} - import org.apache.thrift.protocol.TBinaryProtocol + "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + sql( + s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') + """.stripMargin) + } + }), + TestTable("src_thrift", () => { + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol - sql( + sql( + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' + """.stripMargin) + + sql( + s""" + |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' + |INTO TABLE src_thrift + """.stripMargin) + }), + TestTable("serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), + TestTable("episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, s""" - |CREATE TABLE src_thrift(fake INT) - |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' - |WITH SERDEPROPERTIES( - | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', - | 'serialization.format'='${classOf[TBinaryProtocol].getName}' - |) - |STORED AS - |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' - |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' - """.stripMargin) - - sql( - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") - }), - TestTable("serdeins", - s"""CREATE TABLE serdeins (key INT, value STRING) - |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' - |WITH SERDEPROPERTIES ('field.delim'='\\t') - """.stripMargin.cmd, - "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), - TestTable("episodes", - s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) - |STORED AS avro - |TBLPROPERTIES ( - | 'avro.schema.literal'='{ - | "type": "record", - | "name": "episodes", - | "namespace": "testing.hive.avro.serde", - | "fields": [ - | { - | "name": "title", - | "type": "string", - | "doc": "episode title" - | }, - | { - | "name": "air_date", - | "type": "string", - | "doc": "initial date" - | }, - | { - | "name": "doctor", - | "type": "int", - | "doc": "main actor playing the Doctor in episode" - | } - | ] - | }' - |) - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' INTO TABLE episodes".cmd - ), - // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC - // PARTITIONING IS NOT YET SUPPORTED - TestTable("episodes_part", - s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) - |PARTITIONED BY (doctor_pt INT) - |STORED AS avro - |TBLPROPERTIES ( - | 'avro.schema.literal'='{ - | "type": "record", - | "name": "episodes", - | "namespace": "testing.hive.avro.serde", - | "fields": [ - | { - | "name": "title", - | "type": "string", - | "doc": "episode title" - | }, - | { - | "name": "air_date", - | "type": "string", - | "doc": "initial date" - | }, - | { - | "name": "doctor", - | "type": "int", - | "doc": "main actor playing the Doctor in episode" - | } - | ] - | }' - |) - """.stripMargin.cmd, - // WORKAROUND: Required to pass schema to SerDe for partitioned tables. - // TODO: Pass this automatically from the table to partitions. - s""" - |ALTER TABLE episodes_part SET SERDEPROPERTIES ( - | 'avro.schema.literal'='{ - | "type": "record", - | "name": "episodes", - | "namespace": "testing.hive.avro.serde", - | "fields": [ - | { - | "name": "title", - | "type": "string", - | "doc": "episode title" - | }, - | { - | "name": "air_date", - | "type": "string", - | "doc": "initial date" - | }, - | { - | "name": "doctor", - | "type": "int", - | "doc": "main actor playing the Doctor in episode" - | } - | ] - | }' - |) - """.stripMargin.cmd, - s""" - INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) - SELECT title, air_date, doctor FROM episodes - """.cmd + |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' + |INTO TABLE episodes + """.stripMargin.cmd ), - TestTable("src_json", - s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) - ) + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC + // PARTITIONING IS NOT YET SUPPORTED + TestTable("episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ), + TestTable("src_json", + s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE + """.stripMargin.cmd, + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) + ) - hiveQTestUtilTables.foreach(registerTestTable) + hiveQTestUtilTables.foreach(registerTestTable) + } private val loadedTables = new collection.mutable.HashSet[String] From 80a7bff897554ce77fe6bc91d62cff8857892322 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Thu, 30 Jun 2016 23:00:39 +0800 Subject: [PATCH 0870/1470] [SPARK-15820][PYSPARK][SQL] Add Catalog.refreshTable into python API ## What changes were proposed in this pull request? Add Catalog.refreshTable API into python interface for Spark-SQL. ## How was this patch tested? Existing test. Author: WeichenXu Closes #13558 from WeichenXu123/update_python_sql_interface_refreshTable. (cherry picked from commit 5344bade8efb6f12aa43fbfbbbc2e3c0c7d16d98) Signed-off-by: Cheng Lian --- python/pyspark/sql/catalog.py | 5 +++++ .../main/scala/org/apache/spark/sql/catalog/Catalog.scala | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 3033f147bc969..4af930a3cd563 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -232,6 +232,11 @@ def clearCache(self): """Removes all cached tables from the in-memory cache.""" self._jcatalog.clearCache() + @since(2.0) + def refreshTable(self, tableName): + """Invalidate and refresh all the cached metadata of the given table.""" + self._jcatalog.refreshTable(tableName) + def _reset(self): """(Internal use only) Drop all existing databases (except "default"), tables, partitions and functions, and set the current database to "default". diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 083a63c98c43b..91ed9b3258a12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -214,7 +214,7 @@ abstract class Catalog { def clearCache(): Unit /** - * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, + * Invalidate and refresh all the cached 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. From cc3c44b1196c4186c0b55e319460524e9b9f865b Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 30 Jun 2016 19:34:51 -0700 Subject: [PATCH 0871/1470] [SPARK-14608][ML] transformSchema needs better documentation ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-14608 PipelineStage.transformSchema currently has minimal documentation. It should have more to explain it can: check schema check parameter interactions ## How was this patch tested? unit test Author: Yuhao Yang Author: Yuhao Yang Closes #12384 from hhbyyh/transformSchemaDoc. (cherry picked from commit aa6564f37f1d8de77c3b7bfa885000252efffea6) Signed-off-by: Joseph K. Bradley --- mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 25e56d70c2339..a1d08b3a6e780 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -44,7 +44,10 @@ abstract class PipelineStage extends Params with Logging { /** * :: DeveloperApi :: * - * Derives the output schema from the input schema. + * Check transform validity and derive the output schema from the input schema. + * + * Typical implementation should first conduct verification on schema change and parameter + * validity, including complex parameter interaction checks. */ @DeveloperApi def transformSchema(schema: StructType): StructType From 1932bb683fc11735669c7a4b9e746e2a1dbbcb68 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 1 Jul 2016 00:53:36 -0700 Subject: [PATCH 0872/1470] [SPARK-12177][STREAMING][KAFKA] limit api surface area ## What changes were proposed in this pull request? This is an alternative to the refactoring proposed by https://github.com/apache/spark/pull/13996 ## How was this patch tested? unit tests also tested under scala 2.10 via mvn -Dscala-2.10 Author: cody koeninger Closes #13998 from koeninger/kafka-0-10-refactor. (cherry picked from commit fbfd0ab9d70f557c38c7bb8e704475bf19adaf02) Signed-off-by: Tathagata Das --- .../streaming/kafka010/ConsumerStrategy.scala | 187 +++++++++--------- .../kafka010/DirectKafkaInputDStream.scala | 2 +- .../spark/streaming/kafka010/KafkaRDD.scala | 4 +- .../streaming/kafka010/KafkaTestUtils.scala | 5 +- .../spark/streaming/kafka010/KafkaUtils.scala | 36 ++-- .../streaming/kafka010/LocationStrategy.scala | 74 +++---- .../spark/streaming/kafka010/package.scala | 2 +- .../kafka010/JavaConsumerStrategySuite.java | 32 +-- .../kafka010/JavaDirectKafkaStreamSuite.java | 8 +- .../streaming/kafka010/JavaKafkaRDDSuite.java | 9 +- .../kafka010/JavaLocationStrategySuite.java | 19 +- .../kafka010/DirectKafkaStreamSuite.scala | 35 +++- .../streaming/kafka010/KafkaRDDSuite.scala | 2 +- 13 files changed, 222 insertions(+), 193 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index 079a07dbc2bd0..70c3f1a98d97a 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.kafka010 -import java.{ util => ju } +import java.{ lang => jl, util => ju } import scala.collection.JavaConverters._ @@ -30,15 +30,16 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: * Choice of how to create and configure underlying Kafka Consumers on driver and executors. + * See [[ConsumerStrategies]] to obtain instances. * Kafka 0.10 consumers can require additional, sometimes complex, setup after object * instantiation. This interface encapsulates that process, and allows it to be checkpointed. * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @Experimental -trait ConsumerStrategy[K, V] { +abstract class ConsumerStrategy[K, V] { /** - * Kafka
    + * Kafka * configuration parameters to be used on executors. Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ @@ -51,15 +52,14 @@ trait ConsumerStrategy[K, V] { * has successfully read. Will be empty on initial start, possibly non-empty on restart from * checkpoint. */ - def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] } /** - * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set @@ -68,16 +68,15 @@ trait ConsumerStrategy[K, V] { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ -@Experimental -case class Subscribe[K, V] private( - topics: ju.Collection[java.lang.String], +private case class Subscribe[K, V]( + topics: ju.Collection[jl.String], kafkaParams: ju.Map[String, Object], - offsets: ju.Map[TopicPartition, Long] + offsets: ju.Map[TopicPartition, jl.Long] ) extends ConsumerStrategy[K, V] { def executorKafkaParams: ju.Map[String, Object] = kafkaParams - def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { val consumer = new KafkaConsumer[K, V](kafkaParams) consumer.subscribe(topics) if (currentOffsets.isEmpty) { @@ -90,18 +89,52 @@ case class Subscribe[K, V] private( } } +/** + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ +private case class Assign[K, V]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long] + ) extends ConsumerStrategy[K, V] { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.assign(topicPartitions) + if (currentOffsets.isEmpty) { + offsets.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } + + consumer + } +} + /** * :: Experimental :: - * Companion object for creating [[Subscribe]] strategy + * object for obtaining instances of [[ConsumerStrategy]] */ @Experimental -object Subscribe { +object ConsumerStrategies { /** * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set @@ -111,14 +144,14 @@ object Subscribe { * auto.offset.reset will be used. */ @Experimental - def apply[K, V]( - topics: Iterable[java.lang.String], + def Subscribe[K, V]( + topics: Iterable[jl.String], kafkaParams: collection.Map[String, Object], - offsets: collection.Map[TopicPartition, Long]): Subscribe[K, V] = { - Subscribe[K, V]( + offsets: collection.Map[TopicPartition, Long]): ConsumerStrategy[K, V] = { + new Subscribe[K, V]( new ju.ArrayList(topics.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, Long](offsets.asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) } /** @@ -126,20 +159,20 @@ object Subscribe { * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ @Experimental - def apply[K, V]( - topics: Iterable[java.lang.String], - kafkaParams: collection.Map[String, Object]): Subscribe[K, V] = { - Subscribe[K, V]( + def Subscribe[K, V]( + topics: Iterable[jl.String], + kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { + new Subscribe[K, V]( new ju.ArrayList(topics.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - ju.Collections.emptyMap[TopicPartition, Long]()) + ju.Collections.emptyMap[TopicPartition, jl.Long]()) } /** @@ -147,7 +180,7 @@ object Subscribe { * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set @@ -157,11 +190,11 @@ object Subscribe { * auto.offset.reset will be used. */ @Experimental - def create[K, V]( - topics: ju.Collection[java.lang.String], + def Subscribe[K, V]( + topics: ju.Collection[jl.String], kafkaParams: ju.Map[String, Object], - offsets: ju.Map[TopicPartition, Long]): Subscribe[K, V] = { - Subscribe[K, V](topics, kafkaParams, offsets) + offsets: ju.Map[TopicPartition, jl.Long]): ConsumerStrategy[K, V] = { + new Subscribe[K, V](topics, kafkaParams, offsets) } /** @@ -169,69 +202,25 @@ object Subscribe { * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ @Experimental - def create[K, V]( - topics: ju.Collection[java.lang.String], - kafkaParams: ju.Map[String, Object]): Subscribe[K, V] = { - Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, Long]()) - } - -} - -/** - * :: Experimental :: - * Assign a fixed collection of TopicPartitions - * @param topicPartitions collection of TopicPartitions to assign - * @param kafkaParams Kafka - * - * configuration parameters to be used on driver. The same params will be used on executors, - * with minor automatic modifications applied. - * Requires "bootstrap.servers" to be set - * with Kafka broker(s) specified in host1:port1,host2:port2 form. - * @param offsets: offsets to begin at on initial startup. If no offset is given for a - * TopicPartition, the committed offset (if applicable) or kafka param - * auto.offset.reset will be used. - */ -@Experimental -case class Assign[K, V] private( - topicPartitions: ju.Collection[TopicPartition], - kafkaParams: ju.Map[String, Object], - offsets: ju.Map[TopicPartition, Long] - ) extends ConsumerStrategy[K, V] { - - def executorKafkaParams: ju.Map[String, Object] = kafkaParams - - def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { - val consumer = new KafkaConsumer[K, V](kafkaParams) - consumer.assign(topicPartitions) - if (currentOffsets.isEmpty) { - offsets.asScala.foreach { case (topicPartition, offset) => - consumer.seek(topicPartition, offset) - } - } - - consumer + def Subscribe[K, V]( + topics: ju.Collection[jl.String], + kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { + new Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, jl.Long]()) } -} -/** - * :: Experimental :: - * Companion object for creating [[Assign]] strategy - */ -@Experimental -object Assign { /** * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set @@ -241,14 +230,14 @@ object Assign { * auto.offset.reset will be used. */ @Experimental - def apply[K, V]( + def Assign[K, V]( topicPartitions: Iterable[TopicPartition], kafkaParams: collection.Map[String, Object], - offsets: collection.Map[TopicPartition, Long]): Assign[K, V] = { - Assign[K, V]( + offsets: collection.Map[TopicPartition, Long]): ConsumerStrategy[K, V] = { + new Assign[K, V]( new ju.ArrayList(topicPartitions.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, Long](offsets.asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) } /** @@ -256,20 +245,20 @@ object Assign { * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ @Experimental - def apply[K, V]( + def Assign[K, V]( topicPartitions: Iterable[TopicPartition], - kafkaParams: collection.Map[String, Object]): Assign[K, V] = { - Assign[K, V]( + kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { + new Assign[K, V]( new ju.ArrayList(topicPartitions.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - ju.Collections.emptyMap[TopicPartition, Long]()) + ju.Collections.emptyMap[TopicPartition, jl.Long]()) } /** @@ -277,7 +266,7 @@ object Assign { * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set @@ -287,11 +276,11 @@ object Assign { * auto.offset.reset will be used. */ @Experimental - def create[K, V]( + def Assign[K, V]( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object], - offsets: ju.Map[TopicPartition, Long]): Assign[K, V] = { - Assign[K, V](topicPartitions, kafkaParams, offsets) + offsets: ju.Map[TopicPartition, jl.Long]): ConsumerStrategy[K, V] = { + new Assign[K, V](topicPartitions, kafkaParams, offsets) } /** @@ -299,16 +288,20 @@ object Assign { * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka - * + * * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ @Experimental - def create[K, V]( + def Assign[K, V]( topicPartitions: ju.Collection[TopicPartition], - kafkaParams: ju.Map[String, Object]): Assign[K, V] = { - Assign[K, V](topicPartitions, kafkaParams, ju.Collections.emptyMap[TopicPartition, Long]()) + kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { + new Assign[K, V]( + topicPartitions, + kafkaParams, + ju.Collections.emptyMap[TopicPartition, jl.Long]()) } + } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index acd1841d5305c..13827f68f2cb5 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -71,7 +71,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( @transient private var kc: Consumer[K, V] = null def consumer(): Consumer[K, V] = this.synchronized { if (null == kc) { - kc = consumerStrategy.onStart(currentOffsets) + kc = consumerStrategy.onStart(currentOffsets.mapValues(l => new java.lang.Long(l)).asJava) } kc } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala index c15c16344924f..5b5a9ac48c7ca 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala @@ -36,7 +36,7 @@ import org.apache.spark.storage.StorageLevel * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. * @param kafkaParams Kafka - * + * * configuration parameters. Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD @@ -66,7 +66,7 @@ private[spark] class KafkaRDD[K, V]( " must be set to false for executor kafka params, else offsets may commit before processing") // TODO is it necessary to have separate configs for initial poll time vs ongoing poll time? - private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", 256) + private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", 512) private val cacheInitialCapacity = conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) private val cacheMaxCapacity = diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala index 13c08430db6be..19192e4b95945 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala @@ -61,7 +61,7 @@ private[kafka010] class KafkaTestUtils extends Logging { // Kafka broker related configurations private val brokerHost = "localhost" - private var brokerPort = 9092 + private var brokerPort = 0 private var brokerConf: KafkaConfig = _ // Kafka broker server @@ -110,7 +110,8 @@ private[kafka010] class KafkaTestUtils extends Logging { brokerConf = new KafkaConfig(brokerConfiguration, doLog = false) server = new KafkaServer(brokerConf) server.startup() - (server, port) + brokerPort = server.boundPort() + (server, brokerPort) }, new SparkConf(), "KafkaBroker") brokerReady = true diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala index c0524990bc4dc..b2190bfa05a3a 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -34,7 +34,7 @@ import org.apache.spark.streaming.dstream._ /** * :: Experimental :: - * Companion object for constructing Kafka streams and RDDs + * object for constructing Kafka streams and RDDs */ @Experimental object KafkaUtils extends Logging { @@ -44,12 +44,12 @@ object KafkaUtils extends Logging { * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. * @param kafkaParams Kafka - * + * * configuration parameters. Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD - * @param locationStrategy In most cases, pass in [[PreferConsistent]], - * see [[LocationStrategy]] for more details. + * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent, + * see [[LocationStrategies]] for more details. * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -83,12 +83,12 @@ object KafkaUtils extends Logging { * @param keyClass Class of the keys in the Kafka records * @param valueClass Class of the values in the Kafka records * @param kafkaParams Kafka - * + * * configuration parameters. Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD - * @param locationStrategy In most cases, pass in [[PreferConsistent]], - * see [[LocationStrategy]] for more details. + * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent, + * see [[LocationStrategies]] for more details. * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -110,10 +110,10 @@ object KafkaUtils extends Logging { * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number * of messages * per second that each '''partition''' will accept. - * @param locationStrategy In most cases, pass in [[PreferConsistent]], - * see [[LocationStrategy]] for more details. - * @param consumerStrategy In most cases, pass in [[Subscribe]], - * see [[ConsumerStrategy]] for more details + * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent, + * see [[LocationStrategies]] for more details. + * @param consumerStrategy In most cases, pass in ConsumerStrategies.subscribe, + * see [[ConsumerStrategies]] for more details * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -132,10 +132,10 @@ object KafkaUtils extends Logging { * each given Kafka topic/partition corresponds to an RDD partition. * @param keyClass Class of the keys in the Kafka records * @param valueClass Class of the values in the Kafka records - * @param locationStrategy In most cases, pass in [[PreferConsistent]], - * see [[LocationStrategy]] for more details. - * @param consumerStrategy In most cases, pass in [[Subscribe]], - * see [[ConsumerStrategy]] for more details + * @param locationStrategy In most cases, pass in LocationStrategies.preferConsistent, + * see [[LocationStrategies]] for more details. + * @param consumerStrategy In most cases, pass in ConsumerStrategies.subscribe, + * see [[ConsumerStrategies]] for more details * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -161,7 +161,11 @@ object KafkaUtils extends Logging { kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") // driver and executor should be in different consumer groups - val groupId = "spark-executor-" + kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + val originalGroupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + if (null == originalGroupId) { + logError(s"${ConsumerConfig.GROUP_ID_CONFIG} is null, you should probably set it") + } + val groupId = "spark-executor-" + originalGroupId logWarning(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala index df620300eae21..c9a8a13f51c32 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala @@ -29,49 +29,57 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: * Choice of how to schedule consumers for a given TopicPartition on an executor. + * See [[LocationStrategies]] to obtain instances. * Kafka 0.10 consumers prefetch messages, so it's important for performance * to keep cached consumers on appropriate executors, not recreate them for every partition. * Choice of location is only a preference, not an absolute; partitions may be scheduled elsewhere. */ @Experimental -sealed trait LocationStrategy +sealed abstract class LocationStrategy -/** - * :: Experimental :: - * Use this only if your executors are on the same nodes as your Kafka brokers. - */ -@Experimental -case object PreferBrokers extends LocationStrategy { - def create: PreferBrokers.type = this -} +private case object PreferBrokers extends LocationStrategy -/** - * :: Experimental :: - * Use this in most cases, it will consistently distribute partitions across all executors. - */ -@Experimental -case object PreferConsistent extends LocationStrategy { - def create: PreferConsistent.type = this -} +private case object PreferConsistent extends LocationStrategy -/** - * :: Experimental :: - * Use this to place particular TopicPartitions on particular hosts if your load is uneven. - * Any TopicPartition not specified in the map will use a consistent location. - */ -@Experimental -case class PreferFixed private(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy +private case class PreferFixed(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy /** - * :: Experimental :: - * Use this to place particular TopicPartitions on particular hosts if your load is uneven. - * Any TopicPartition not specified in the map will use a consistent location. + * :: Experimental :: object to obtain instances of [[LocationStrategy]] + * */ @Experimental -object PreferFixed { - def apply(hostMap: collection.Map[TopicPartition, String]): PreferFixed = { - PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) - } - def create(hostMap: ju.Map[TopicPartition, String]): PreferFixed = - PreferFixed(hostMap) +object LocationStrategies { + /** + * :: Experimental :: + * Use this only if your executors are on the same nodes as your Kafka brokers. + */ + @Experimental + def PreferBrokers: LocationStrategy = + org.apache.spark.streaming.kafka010.PreferBrokers + + /** + * :: Experimental :: + * Use this in most cases, it will consistently distribute partitions across all executors. + */ + @Experimental + def PreferConsistent: LocationStrategy = + org.apache.spark.streaming.kafka010.PreferConsistent + + /** + * :: Experimental :: + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ + @Experimental + def PreferFixed(hostMap: collection.Map[TopicPartition, String]): LocationStrategy = + new PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) + + /** + * :: Experimental :: + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ + @Experimental + def PreferFixed(hostMap: ju.Map[TopicPartition, String]): LocationStrategy = + new PreferFixed(hostMap) } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala index 2bfc1e84d7ccd..09db6d6062d82 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala @@ -20,4 +20,4 @@ package org.apache.spark.streaming /** * Spark Integration for Kafka 0.10 */ -package object kafka +package object kafka010 //scalastyle:ignore diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java index 8d7c05b5a615d..ac8d64b180f0d 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -44,37 +44,39 @@ public void testConsumerStrategyConstructors() { kafkaParams.put("bootstrap.servers", "not used"); final scala.collection.Map sKafkaParams = JavaConverters.mapAsScalaMapConverter(kafkaParams).asScala(); - final Map offsets = new HashMap<>(); + final Map offsets = new HashMap<>(); offsets.put(tp1, 23L); final scala.collection.Map sOffsets = - JavaConverters.mapAsScalaMapConverter(offsets).asScala(); + JavaConverters.mapAsScalaMapConverter(offsets).asScala().mapValues( + new scala.runtime.AbstractFunction1() { + @Override + public Object apply(Long x) { + return (Object) x; + } + } + ); - // make sure constructors can be called from java - // final ConsumerStrategy sub0 = // does not compile in Scala 2.10 - // Subscribe.apply(topics, kafkaParams, offsets); final ConsumerStrategy sub1 = - Subscribe.apply(sTopics, sKafkaParams, sOffsets); + ConsumerStrategies.Subscribe(sTopics, sKafkaParams, sOffsets); final ConsumerStrategy sub2 = - Subscribe.apply(sTopics, sKafkaParams); + ConsumerStrategies.Subscribe(sTopics, sKafkaParams); final ConsumerStrategy sub3 = - Subscribe.create(topics, kafkaParams, offsets); + ConsumerStrategies.Subscribe(topics, kafkaParams, offsets); final ConsumerStrategy sub4 = - Subscribe.create(topics, kafkaParams); + ConsumerStrategies.Subscribe(topics, kafkaParams); Assert.assertEquals( sub1.executorKafkaParams().get("bootstrap.servers"), sub3.executorKafkaParams().get("bootstrap.servers")); - // final ConsumerStrategy asn0 = // does not compile in Scala 2.10 - // Assign.apply(parts, kafkaParams, offsets); final ConsumerStrategy asn1 = - Assign.apply(sParts, sKafkaParams, sOffsets); + ConsumerStrategies.Assign(sParts, sKafkaParams, sOffsets); final ConsumerStrategy asn2 = - Assign.apply(sParts, sKafkaParams); + ConsumerStrategies.Assign(sParts, sKafkaParams); final ConsumerStrategy asn3 = - Assign.create(parts, kafkaParams, offsets); + ConsumerStrategies.Assign(parts, kafkaParams, offsets); final ConsumerStrategy asn4 = - Assign.create(parts, kafkaParams); + ConsumerStrategies.Assign(parts, kafkaParams); Assert.assertEquals( asn1.executorKafkaParams().get("bootstrap.servers"), diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java index e57ede7afaef4..dc9c13ba863ff 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java @@ -90,8 +90,8 @@ public void testKafkaStream() throws InterruptedException { JavaInputDStream> istream1 = KafkaUtils.createDirectStream( ssc, - PreferConsistent.create(), - Subscribe.create(Arrays.asList(topic1), kafkaParams) + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(Arrays.asList(topic1), kafkaParams) ); JavaDStream stream1 = istream1.transform( @@ -123,8 +123,8 @@ public String call(ConsumerRecord r) { JavaInputDStream> istream2 = KafkaUtils.createDirectStream( ssc, - PreferConsistent.create(), - Subscribe.create(Arrays.asList(topic2), kafkaParams2) + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(Arrays.asList(topic2), kafkaParams2) ); JavaDStream stream2 = istream2.transform( diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java index 548ba134dcddf..87bfe1514e338 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.HashMap; import java.util.Map; +import java.util.Random; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.TopicPartition; @@ -65,6 +66,8 @@ public void testKafkaRDD() throws InterruptedException { String topic1 = "topic1"; String topic2 = "topic2"; + Random random = new Random(); + createTopicAndSendData(topic1); createTopicAndSendData(topic2); @@ -72,6 +75,8 @@ public void testKafkaRDD() throws InterruptedException { kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); kafkaParams.put("key.deserializer", StringDeserializer.class); kafkaParams.put("value.deserializer", StringDeserializer.class); + kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); OffsetRange[] offsetRanges = { OffsetRange.create(topic1, 0, 0, 1), @@ -96,14 +101,14 @@ public String call(ConsumerRecord r) { sc, kafkaParams, offsetRanges, - PreferFixed.create(leaders) + LocationStrategies.PreferFixed(leaders) ).map(handler); JavaRDD rdd2 = KafkaUtils.createRDD( sc, kafkaParams, offsetRanges, - PreferConsistent.create() + LocationStrategies.PreferConsistent() ).map(handler); // just making sure the java user apis work; the scala tests handle logic corner cases diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java index 7873c09e1af85..41ccb0ebe7bfa 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java @@ -41,18 +41,19 @@ public void testLocationStrategyConstructors() { JavaConverters.mapAsScalaMapConverter(hosts).asScala(); // make sure constructors can be called from java - final LocationStrategy c1 = PreferConsistent.create(); - final LocationStrategy c2 = PreferConsistent$.MODULE$; - Assert.assertEquals(c1, c2); + final LocationStrategy c1 = LocationStrategies.PreferConsistent(); + final LocationStrategy c2 = LocationStrategies.PreferConsistent(); + Assert.assertSame(c1, c2); - final LocationStrategy c3 = PreferBrokers.create(); - final LocationStrategy c4 = PreferBrokers$.MODULE$; - Assert.assertEquals(c3, c4); + final LocationStrategy c3 = LocationStrategies.PreferBrokers(); + final LocationStrategy c4 = LocationStrategies.PreferBrokers(); + Assert.assertSame(c3, c4); - final LocationStrategy c5 = PreferFixed.create(hosts); - final LocationStrategy c6 = PreferFixed.apply(sHosts); - Assert.assertEquals(c5, c6); + Assert.assertNotSame(c1, c3); + final LocationStrategy c5 = LocationStrategies.PreferFixed(hosts); + final LocationStrategy c6 = LocationStrategies.PreferFixed(sHosts); + Assert.assertEquals(c5, c6); } } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 776d11ad2f648..0a53259802d1e 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.kafka010 import java.io.File +import java.lang.{ Long => JLong } import java.util.{ Arrays, HashMap => JHashMap, Map => JMap } import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.ConcurrentLinkedQueue @@ -93,7 +94,7 @@ class DirectKafkaStreamSuite kp } - val preferredHosts = PreferConsistent + val preferredHosts = LocationStrategies.PreferConsistent test("basic stream receiving with multiple topics and smallest starting offset") { val topics = List("basic1", "basic2", "basic3") @@ -108,7 +109,9 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](topics, kafkaParams.asScala)) + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](topics, kafkaParams.asScala)) } val allReceived = new ConcurrentLinkedQueue[(String, String)]() @@ -178,7 +181,9 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { val s = new DirectKafkaInputDStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) s.consumer.poll(0) assert( s.consumer.position(topicPartition) >= offsetBeforeStart, @@ -225,8 +230,10 @@ class DirectKafkaStreamSuite // Setup context and kafka stream with largest offset ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, - Assign[String, String]( + val s = new DirectKafkaInputDStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Assign[String, String]( List(topicPartition), kafkaParams.asScala, Map(topicPartition -> 11L))) @@ -267,7 +274,9 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(100)) val kafkaStream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) } val keyedStream = kafkaStream.map { r => "key" -> r.value.toInt } val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => @@ -360,7 +369,9 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(100)) withClue("Error creating direct stream") { val kafkaStream = KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges val data = rdd.map(_.value).collect() @@ -412,7 +423,9 @@ class DirectKafkaStreamSuite val stream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) } val allReceived = new ConcurrentLinkedQueue[(String, String)] @@ -486,7 +499,9 @@ class DirectKafkaStreamSuite val kafkaStream = withClue("Error creating direct stream") { new DirectKafkaInputDStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) { + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) { override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) }.map(r => (r.key, r.value)) @@ -552,7 +567,7 @@ class DirectKafkaStreamSuite preferredHosts, new ConsumerStrategy[String, String] { def executorKafkaParams = ekp - def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[String, String] = { + def onStart(currentOffsets: JMap[TopicPartition, JLong]): Consumer[String, String] = { val consumer = new KafkaConsumer[String, String](kafkaParams) val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) consumer.assign(Arrays.asList(tps: _*)) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala index 3d2546ddd936d..be373af0599cc 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala @@ -62,7 +62,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}" ).asJava - private val preferredHosts = PreferConsistent + private val preferredHosts = LocationStrategies.PreferConsistent test("basic usage") { val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" From 972106dd3bdc40b0980949a09783d6d460e8d268 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 1 Jul 2016 09:22:27 +0100 Subject: [PATCH 0873/1470] [SPARK-16182][CORE] Utils.scala -- terminateProcess() should call Process.destroyForcibly() if and only if Process.destroy() fails ## What changes were proposed in this pull request? Utils.terminateProcess should `destroy()` first and only fall back to `destroyForcibly()` if it fails. It's kind of bad that we're force-killing executors -- and only in Java 8. See JIRA for an example of the impact: no shutdown While here: `Utils.waitForProcess` should use the Java 8 method if available instead of a custom implementation. ## How was this patch tested? Existing tests, which cover the force-kill case, and Amplab tests, which will cover both Java 7 and Java 8 eventually. However I tested locally on Java 8 and the PR builder will try Java 7 here. Author: Sean Owen Closes #13973 from srowen/SPARK-16182. (cherry picked from commit 2075bf8ef6035fd7606bcf20dc2cd7d7b9cda446) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/util/Utils.scala | 76 +++++++++++-------- .../org/apache/spark/util/UtilsSuite.scala | 2 +- 2 files changed, 47 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index f77cc2f9b7aa0..0c23f3cd35458 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1772,50 +1772,66 @@ private[spark] object Utils extends Logging { } /** - * Terminates a process waiting for at most the specified duration. Returns whether - * the process terminated. + * Terminates a process waiting for at most the specified duration. + * + * @return the process exit value if it was successfully terminated, else None */ def terminateProcess(process: Process, timeoutMs: Long): Option[Int] = { - try { - // Java8 added a new API which will more forcibly kill the process. Use that if available. - val destroyMethod = process.getClass().getMethod("destroyForcibly"); - destroyMethod.setAccessible(true) - destroyMethod.invoke(process) - } catch { - case NonFatal(e) => - if (!e.isInstanceOf[NoSuchMethodException]) { - logWarning("Exception when attempting to kill process", e) - } - process.destroy() - } + // Politely destroy first + process.destroy() + if (waitForProcess(process, timeoutMs)) { + // Successful exit Option(process.exitValue()) } else { - None + // Java 8 added a new API which will more forcibly kill the process. Use that if available. + try { + classOf[Process].getMethod("destroyForcibly").invoke(process) + } catch { + case _: NoSuchMethodException => return None // Not available; give up + case NonFatal(e) => logWarning("Exception when attempting to kill process", e) + } + // Wait, again, although this really should return almost immediately + if (waitForProcess(process, timeoutMs)) { + Option(process.exitValue()) + } else { + logWarning("Timed out waiting to forcibly kill process") + None + } } } /** * Wait for a process to terminate for at most the specified duration. - * Return whether the process actually terminated after the given timeout. + * + * @return whether the process actually terminated before the given timeout. */ def waitForProcess(process: Process, timeoutMs: Long): Boolean = { - var terminated = false - val startTime = System.currentTimeMillis - while (!terminated) { - try { - process.exitValue() - terminated = true - } catch { - case e: IllegalThreadStateException => - // Process not terminated yet - if (System.currentTimeMillis - startTime > timeoutMs) { - return false + try { + // Use Java 8 method if available + classOf[Process].getMethod("waitFor", java.lang.Long.TYPE, classOf[TimeUnit]) + .invoke(process, timeoutMs.asInstanceOf[java.lang.Long], TimeUnit.MILLISECONDS) + .asInstanceOf[Boolean] + } catch { + case _: NoSuchMethodError => + // Otherwise implement it manually + var terminated = false + val startTime = System.currentTimeMillis + while (!terminated) { + try { + process.exitValue() + terminated = true + } catch { + case e: IllegalThreadStateException => + // Process not terminated yet + if (System.currentTimeMillis - startTime > timeoutMs) { + return false + } + Thread.sleep(100) } - Thread.sleep(100) - } + } + true } - true } /** 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 df279b5a37c7d..f5d0fb00b732d 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -863,7 +863,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(terminated.isDefined) Utils.waitForProcess(process, 5000) val duration = System.currentTimeMillis() - start - assert(duration < 5000) + assert(duration < 6000) // add a little extra time to allow a force kill to finish assert(!pidExists(pid)) } finally { signal(pid, "SIGKILL") From 0b64543c5ba6a943294f189b7ca02e0debbfad9c Mon Sep 17 00:00:00 2001 From: MechCoder Date: Fri, 1 Jul 2016 09:27:34 +0100 Subject: [PATCH 0874/1470] [SPARK-15761][MLLIB][PYSPARK] Load ipython when default python is Python3 ## What changes were proposed in this pull request? I would like to use IPython with Python 3.5. It is annoying when it fails with IPython requires Python 2.7+; please install python2.7 or set PYSPARK_PYTHON when I have a version greater than 2.7 ## How was this patch tested It now works with IPython and Python3 Author: MechCoder Closes #13503 from MechCoder/spark-15761. (cherry picked from commit 66283ee0b25de2a5daaa21d50a05a7fadec1de77) Signed-off-by: Sean Owen --- bin/pyspark | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/bin/pyspark b/bin/pyspark index 396a07c9f4134..ac8aa04dba8a3 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -50,9 +50,11 @@ if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"$DEFAULT_PYTHON"}" fi +WORKS_WITH_IPYTHON=$($DEFAULT_PYTHON -c 'import sys; print(sys.version_info >= (2, 7, 0))') + # Determine the Python executable to use for the executors: if [[ -z "$PYSPARK_PYTHON" ]]; then - if [[ $PYSPARK_DRIVER_PYTHON == *ipython* && $DEFAULT_PYTHON != "python2.7" ]]; then + if [[ $PYSPARK_DRIVER_PYTHON == *ipython* && ! WORKS_WITH_IPYTHON ]]; then echo "IPython requires Python 2.7+; please install python2.7 or set PYSPARK_PYTHON" 1>&2 exit 1 else From 3665927c6f5fa4794a59718fd2d339310c70a985 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 1 Jul 2016 09:54:02 +0100 Subject: [PATCH 0875/1470] [SPARK-16222][SQL] JDBC Sources - Handling illegal input values for `fetchsize` and `batchsize` #### What changes were proposed in this pull request? For JDBC data sources, users can specify `batchsize` for multi-row inserts and `fetchsize` for multi-row fetch. A few issues exist: - The property keys are case sensitive. Thus, the existing test cases for `fetchsize` use incorrect names, `fetchSize`. Basically, the test cases are broken. - No test case exists for `batchsize`. - We do not detect the illegal input values for `fetchsize` and `batchsize`. For example, when `batchsize` is zero, we got the following exception: ``` Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ArithmeticException: / by zero ``` when `fetchsize` is less than zero, we got the exception from the underlying JDBC driver: ``` Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): org.h2.jdbc.JdbcSQLException: Invalid value "-1" for parameter "rows" [90008-183] ``` This PR fixes all the above issues, and issue the appropriate exceptions when detecting the illegal inputs for `fetchsize` and `batchsize`. Also update the function descriptions. #### How was this patch tested? Test cases are fixed and added. Author: gatorsmile Closes #13919 from gatorsmile/jdbcProperties. (cherry picked from commit 0ad6ce7e54b1d8f5946dde652fa5341d15059158) Signed-off-by: Sean Owen --- .../apache/spark/sql/DataFrameReader.scala | 6 +- .../apache/spark/sql/DataFrameWriter.scala | 3 +- .../execution/datasources/jdbc/JDBCRDD.scala | 6 +- .../datasources/jdbc/JdbcUtils.scala | 10 ++- .../spark/sql/jdbc/PostgresDialect.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 62 +++++++++++-------- .../spark/sql/jdbc/JDBCWriteSuite.scala | 54 +++++++++++----- 7 files changed, 98 insertions(+), 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 35ba522786337..e8c2885d7737c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -177,7 +177,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * clause expressions used to split the column `columnName` evenly. * @param connectionProperties JDBC database connection arguments, a list of arbitrary string * tag/value. Normally at least a "user" and "password" property - * should be included. + * should be included. "fetchsize" can be used to control the + * number of rows per fetch. * @since 1.4.0 */ def jdbc( @@ -207,7 +208,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @param predicates Condition in the where clause for each partition. * @param connectionProperties JDBC database connection arguments, a list of arbitrary string * tag/value. Normally at least a "user" and "password" property - * should be included. + * should be included. "fetchsize" can be used to control the + * number of rows per fetch. * @since 1.4.0 */ def jdbc( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index ca3972d62dfb5..f77af76d2bf3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -391,7 +391,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @param table Name of the table in the external database. * @param connectionProperties JDBC database connection arguments, a list of arbitrary string * tag/value. Normally at least a "user" and "password" property - * should be included. + * should be included. "batchsize" can be used to control the + * number of rows per insert. * @since 1.4.0 */ def jdbc(url: String, table: String, connectionProperties: Properties): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 44cfbb9fbd811..24e2c1a5fd2f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -390,7 +390,11 @@ private[sql] class JDBCRDD( val sqlText = s"SELECT $columnList FROM $fqTable $myWhereClause" val stmt = conn.prepareStatement(sqlText, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) - val fetchSize = properties.getProperty("fetchsize", "0").toInt + val fetchSize = properties.getProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "0").toInt + require(fetchSize >= 0, + s"Invalid value `${fetchSize.toString}` for parameter " + + s"`${JdbcUtils.JDBC_BATCH_FETCH_SIZE}`. The minimum value is 0. When the value is 0, " + + "the JDBC driver ignores the value and does the estimates.") stmt.setFetchSize(fetchSize) val rs = stmt.executeQuery() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 065c8572b06a2..3529ee6e3b6ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -34,6 +34,10 @@ import org.apache.spark.sql.types._ */ object JdbcUtils extends Logging { + // the property names are case sensitive + val JDBC_BATCH_FETCH_SIZE = "fetchsize" + val JDBC_BATCH_INSERT_SIZE = "batchsize" + /** * Returns a factory for creating connections to the given JDBC URL. * @@ -154,6 +158,10 @@ object JdbcUtils extends Logging { nullTypes: Array[Int], batchSize: Int, dialect: JdbcDialect): Iterator[Byte] = { + require(batchSize >= 1, + s"Invalid value `${batchSize.toString}` for parameter " + + s"`${JdbcUtils.JDBC_BATCH_INSERT_SIZE}`. The minimum value is 1.") + val conn = getConnection() var committed = false val supportsTransactions = try { @@ -275,7 +283,7 @@ object JdbcUtils extends Logging { val rddSchema = df.schema val getConnection: () => Connection = createConnectionFactory(url, properties) - val batchSize = properties.getProperty("batchsize", "1000").toInt + val batchSize = properties.getProperty(JDBC_BATCH_INSERT_SIZE, "1000").toInt df.foreachPartition { iterator => savePartition(getConnection, table, iterator, rddSchema, nullTypes, batchSize, dialect) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 2d6c3974a833e..6baf1b6f16cd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -89,7 +89,7 @@ private object PostgresDialect extends JdbcDialect { // // See: https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor // - if (properties.getOrElse("fetchsize", "0").toInt > 0) { + if (properties.getOrElse(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "0").toInt > 0) { connection.setAutoCommit(false) } 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 fd6671a39b6ef..11e66ad08009c 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 @@ -24,12 +24,13 @@ import java.util.{Calendar, GregorianCalendar, Properties} import org.h2.jdbc.JdbcSQLException import org.scalatest.{BeforeAndAfter, PrivateMethodTester} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -83,7 +84,7 @@ class JDBCSuite extends SparkFunSuite |CREATE TEMPORARY TABLE fetchtwo |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass', - | fetchSize '2') + | ${JdbcUtils.JDBC_BATCH_FETCH_SIZE} '2') """.stripMargin.replaceAll("\n", " ")) sql( @@ -348,38 +349,49 @@ class JDBCSuite extends SparkFunSuite test("Basic API") { assert(spark.read.jdbc( - urlWithUserAndPass, "TEST.PEOPLE", new Properties).collect().length === 3) + urlWithUserAndPass, "TEST.PEOPLE", new Properties()).collect().length === 3) + } + + test("Basic API with illegal FetchSize") { + val properties = new Properties() + properties.setProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "-1") + val e = intercept[SparkException] { + spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", properties).collect() + }.getMessage + assert(e.contains("Invalid value `-1` for parameter `fetchsize`")) } test("Basic API with FetchSize") { - val properties = new Properties - properties.setProperty("fetchSize", "2") - assert(spark.read.jdbc( - urlWithUserAndPass, "TEST.PEOPLE", properties).collect().length === 3) + (0 to 4).foreach { size => + val properties = new Properties() + properties.setProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, size.toString) + assert(spark.read.jdbc( + urlWithUserAndPass, "TEST.PEOPLE", properties).collect().length === 3) + } } test("Partitioning via JDBCPartitioningInfo API") { assert( - spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3, new Properties) + spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3, new Properties()) .collect().length === 3) } test("Partitioning via list-of-where-clauses API") { val parts = Array[String]("THEID < 2", "THEID >= 2") - assert(spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties) + assert(spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) .collect().length === 3) } test("Partitioning on column that might have null values.") { assert( - spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", "theid", 0, 4, 3, new Properties) + spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", "theid", 0, 4, 3, new Properties()) .collect().length === 4) assert( - spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", "THEID", 0, 4, 3, new Properties) + spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", "THEID", 0, 4, 3, new Properties()) .collect().length === 4) // partitioning on a nullable quoted column assert( - spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", """"Dept"""", 0, 4, 3, new Properties) + spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", """"Dept"""", 0, 4, 3, new Properties()) .collect().length === 4) } @@ -391,7 +403,7 @@ class JDBCSuite extends SparkFunSuite lowerBound = 0, upperBound = 4, numPartitions = 0, - connectionProperties = new Properties + connectionProperties = new Properties() ) assert(res.count() === 8) } @@ -404,7 +416,7 @@ class JDBCSuite extends SparkFunSuite lowerBound = 1, upperBound = 5, numPartitions = 10, - connectionProperties = new Properties + connectionProperties = new Properties() ) assert(res.count() === 8) } @@ -417,7 +429,7 @@ class JDBCSuite extends SparkFunSuite lowerBound = 5, upperBound = 5, numPartitions = 4, - connectionProperties = new Properties + connectionProperties = new Properties() ) assert(res.count() === 8) } @@ -431,7 +443,7 @@ class JDBCSuite extends SparkFunSuite lowerBound = 5, upperBound = 1, numPartitions = 3, - connectionProperties = new Properties + connectionProperties = new Properties() ) }.getMessage assert(e.contains("Operation not allowed: the lower bound of partitioning column " + @@ -495,8 +507,8 @@ class JDBCSuite extends SparkFunSuite test("test DATE types") { val rows = spark.read.jdbc( - urlWithUserAndPass, "TEST.TIMETYPES", new Properties).collect() - val cachedRows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) + urlWithUserAndPass, "TEST.TIMETYPES", new Properties()).collect() + val cachedRows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()) .cache().collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) assert(rows(1).getAs[java.sql.Date](1) === null) @@ -504,8 +516,8 @@ class JDBCSuite extends SparkFunSuite } test("test DATE types in cache") { - val rows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties).collect() - spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) + val rows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()).collect() + spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()) .cache().createOrReplaceTempView("mycached_date") val cachedRows = sql("select * from mycached_date").collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) @@ -514,7 +526,7 @@ class JDBCSuite extends SparkFunSuite test("test types for null value") { val rows = spark.read.jdbc( - urlWithUserAndPass, "TEST.NULLTYPES", new Properties).collect() + urlWithUserAndPass, "TEST.NULLTYPES", new Properties()).collect() assert((0 to 14).forall(i => rows(0).isNullAt(i))) } @@ -560,7 +572,7 @@ class JDBCSuite extends SparkFunSuite test("Remap types via JdbcDialects") { JdbcDialects.registerDialect(testH2Dialect) - val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) assert(df.schema.filter(_.dataType != org.apache.spark.sql.types.StringType).isEmpty) val rows = df.collect() assert(rows(0).get(0).isInstanceOf[String]) @@ -694,7 +706,7 @@ class JDBCSuite extends SparkFunSuite // Regression test for bug SPARK-11788 val timestamp = java.sql.Timestamp.valueOf("2001-02-20 11:22:33.543543"); val date = java.sql.Date.valueOf("1995-01-01") - val jdbcDf = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) + val jdbcDf = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()) val rows = jdbcDf.where($"B" > date && $"C" > timestamp).collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) assert(rows(0).getAs[java.sql.Timestamp](2) @@ -714,7 +726,7 @@ class JDBCSuite extends SparkFunSuite } test("test credentials in the connection url are not in the plan output") { - val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) val explain = ExplainCommand(df.queryExecution.logical, extended = true) spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { r => assert(!List("testPass", "testUser").exists(r.toString.contains)) @@ -746,7 +758,7 @@ class JDBCSuite extends SparkFunSuite urlWithUserAndPass, "TEST.PEOPLE", predicates = Array[String](jdbcPartitionWhereClause), - new Properties) + new Properties()) df.createOrReplaceTempView("tempFrame") assertEmptyQuery(s"SELECT * FROM tempFrame where $FALSE2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index ff66f53fcf594..2c6449fa6870b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -22,7 +22,9 @@ import java.util.Properties import org.scalatest.BeforeAndAfter +import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -90,10 +92,34 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { test("Basic CREATE") { val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) - df.write.jdbc(url, "TEST.BASICCREATETEST", new Properties) - assert(2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties).count) + df.write.jdbc(url, "TEST.BASICCREATETEST", new Properties()) + assert(2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties()).count()) assert( - 2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties).collect()(0).length) + 2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties()).collect()(0).length) + } + + test("Basic CREATE with illegal batchsize") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + (-1 to 0).foreach { size => + val properties = new Properties() + properties.setProperty(JdbcUtils.JDBC_BATCH_INSERT_SIZE, size.toString) + val e = intercept[SparkException] { + df.write.mode(SaveMode.Overwrite).jdbc(url, "TEST.BASICCREATETEST", properties) + }.getMessage + assert(e.contains(s"Invalid value `$size` for parameter `batchsize`")) + } + } + + test("Basic CREATE with batchsize") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + (1 to 3).foreach { size => + val properties = new Properties() + properties.setProperty(JdbcUtils.JDBC_BATCH_INSERT_SIZE, size.toString) + df.write.mode(SaveMode.Overwrite).jdbc(url, "TEST.BASICCREATETEST", properties) + assert(2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties()).count()) + } } test("CREATE with overwrite") { @@ -101,11 +127,11 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) df.write.jdbc(url1, "TEST.DROPTEST", properties) - assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count) + assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) assert(3 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) df2.write.mode(SaveMode.Overwrite).jdbc(url1, "TEST.DROPTEST", properties) - assert(1 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count) + assert(1 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) } @@ -113,10 +139,10 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) - df.write.jdbc(url, "TEST.APPENDTEST", new Properties) - df2.write.mode(SaveMode.Append).jdbc(url, "TEST.APPENDTEST", new Properties) - assert(3 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties).count) - assert(2 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties).collect()(0).length) + df.write.jdbc(url, "TEST.APPENDTEST", new Properties()) + df2.write.mode(SaveMode.Append).jdbc(url, "TEST.APPENDTEST", new Properties()) + assert(3 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties()).count()) + assert(2 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties()).collect()(0).length) } test("CREATE then INSERT to truncate") { @@ -125,7 +151,7 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { df.write.jdbc(url1, "TEST.TRUNCATETEST", properties) df2.write.mode(SaveMode.Overwrite).jdbc(url1, "TEST.TRUNCATETEST", properties) - assert(1 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count) + assert(1 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count()) assert(2 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).collect()(0).length) } @@ -133,22 +159,22 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) val df2 = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) - df.write.jdbc(url, "TEST.INCOMPATIBLETEST", new Properties) + df.write.jdbc(url, "TEST.INCOMPATIBLETEST", new Properties()) intercept[org.apache.spark.SparkException] { - df2.write.mode(SaveMode.Append).jdbc(url, "TEST.INCOMPATIBLETEST", new Properties) + df2.write.mode(SaveMode.Append).jdbc(url, "TEST.INCOMPATIBLETEST", new Properties()) } } test("INSERT to JDBC Datasource") { sql("INSERT INTO TABLE PEOPLE1 SELECT * FROM PEOPLE") - assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).count) + assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).count()) assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length) } test("INSERT to JDBC Datasource with overwrite") { sql("INSERT INTO TABLE PEOPLE1 SELECT * FROM PEOPLE") sql("INSERT OVERWRITE TABLE PEOPLE1 SELECT * FROM PEOPLE") - assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).count) + assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).count()) assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length) } } From 4c96ded84b8f882ab6d2d42c343ccff8c972d713 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Fri, 1 Jul 2016 13:55:13 -0700 Subject: [PATCH 0876/1470] [SPARK-16012][SPARKR] Implement gapplyCollect which will apply a R function on each group similar to gapply and collect the result back to R data.frame ## What changes were proposed in this pull request? gapplyCollect() does gapply() on a SparkDataFrame and collect the result back to R. Compared to gapply() + collect(), gapplyCollect() offers performance optimization as well as programming convenience, as no schema is needed to be provided. This is similar to dapplyCollect(). ## How was this patch tested? Added test cases for gapplyCollect similar to dapplyCollect Author: Narine Kokhlikyan Closes #13760 from NarineK/gapplyCollect. (cherry picked from commit 26afb4ce4099e7942f8db1ead3817ed8fbf71ce3) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 111 ++++++++++++++++++++-- R/pkg/R/generics.R | 4 + R/pkg/R/group.R | 93 ++++++++---------- R/pkg/inst/tests/testthat/test_sparkSQL.R | 35 ++++++- 5 files changed, 177 insertions(+), 67 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index e0ffde922dacf..9fd2568078e63 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -69,6 +69,7 @@ exportMethods("arrange", "first", "freqItems", "gapply", + "gapplyCollect", "group_by", "groupBy", "head", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 567758d2e2f28..17474d4c81559 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1339,7 +1339,7 @@ setMethod("dapplyCollect", #' gapply #' -#' Group the SparkDataFrame using the specified columns and apply the R function to each +#' Groups the SparkDataFrame using the specified columns and applies the R function to each #' group. #' #' @param x A SparkDataFrame @@ -1351,9 +1351,11 @@ setMethod("dapplyCollect", #' @param schema The schema of the resulting SparkDataFrame after the function is applied. #' The schema must match to output of `func`. It has to be defined for each #' output column with preferred output column name and corresponding data type. +#' @return a SparkDataFrame #' @family SparkDataFrame functions #' @rdname gapply #' @name gapply +#' @seealso \link{gapplyCollect} #' @export #' @examples #' @@ -1369,14 +1371,22 @@ setMethod("dapplyCollect", #' columns with data types integer and string and the mean which is a double. #' schema <- structType(structField("a", "integer"), structField("c", "string"), #' structField("avg", "double")) -#' df1 <- gapply( +#' result <- gapply( #' df, -#' list("a", "c"), +#' c("a", "c"), #' function(key, x) { #' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) -#' }, -#' schema) -#' collect(df1) +#' }, schema) +#' +#' We can also group the data and afterwards call gapply on GroupedData. +#' For Example: +#' gdf <- group_by(df, "a", "c") +#' result <- gapply( +#' gdf, +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' }, schema) +#' collect(result) #' #' Result #' ------ @@ -1394,7 +1404,7 @@ setMethod("dapplyCollect", #' structField("Petal_Width", "double")) #' df1 <- gapply( #' df, -#' list(df$"Species"), +#' df$"Species", #' function(key, x) { #' m <- suppressWarnings(lm(Sepal_Length ~ #' Sepal_Width + Petal_Length + Petal_Width, x)) @@ -1402,8 +1412,8 @@ setMethod("dapplyCollect", #' }, schema) #' collect(df1) #' -#'Result -#'--------- +#' Result +#' --------- #' Model (Intercept) Sepal_Width Petal_Length Petal_Width #' 1 0.699883 0.3303370 0.9455356 -0.1697527 #' 2 1.895540 0.3868576 0.9083370 -0.6792238 @@ -1418,6 +1428,89 @@ setMethod("gapply", gapply(grouped, func, schema) }) +#' gapplyCollect +#' +#' Groups the SparkDataFrame using the specified columns, applies the R function to each +#' group and collects the result back to R as data.frame. +#' +#' @param x A SparkDataFrame +#' @param cols Grouping columns +#' @param func A function to be applied to each group partition specified by grouping +#' column of the SparkDataFrame. The function `func` takes as argument +#' a key - grouping columns and a data frame - a local R data.frame. +#' The output of `func` is a local R data.frame. +#' @return a data.frame +#' @family SparkDataFrame functions +#' @rdname gapplyCollect +#' @name gapplyCollect +#' @seealso \link{gapply} +#' @export +#' @examples +#' +#' \dontrun{ +#' Computes the arithmetic mean of the second column by grouping +#' on the first and third columns. Output the grouping values and the average. +#' +#' df <- createDataFrame ( +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), +#' c("a", "b", "c", "d")) +#' +#' result <- gapplyCollect( +#' df, +#' c("a", "c"), +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' colnames(y) <- c("key_a", "key_c", "mean_b") +#' y +#' }) +#' +#' We can also group the data and afterwards call gapply on GroupedData. +#' For Example: +#' gdf <- group_by(df, "a", "c") +#' result <- gapplyCollect( +#' gdf, +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' colnames(y) <- c("key_a", "key_c", "mean_b") +#' y +#' }) +#' +#' Result +#' ------ +#' key_a key_c mean_b +#' 3 3 3.0 +#' 1 1 1.5 +#' +#' Fits linear models on iris dataset by grouping on the 'Species' column and +#' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' +#' and 'Petal_Width' as training features. +#' +#' df <- createDataFrame (iris) +#' result <- gapplyCollect( +#' df, +#' df$"Species", +#' function(key, x) { +#' m <- suppressWarnings(lm(Sepal_Length ~ +#' Sepal_Width + Petal_Length + Petal_Width, x)) +#' data.frame(t(coef(m))) +#' }) +#' +#' Result +#'--------- +#' Model X.Intercept. Sepal_Width Petal_Length Petal_Width +#' 1 0.699883 0.3303370 0.9455356 -0.1697527 +#' 2 1.895540 0.3868576 0.9083370 -0.6792238 +#' 3 2.351890 0.6548350 0.2375602 0.2521257 +#' +#'} +#' @note gapplyCollect(SparkDataFrame) since 2.0.0 +setMethod("gapplyCollect", + signature(x = "SparkDataFrame"), + function(x, cols, func) { + grouped <- do.call("groupBy", c(x, cols)) + gapplyCollect(grouped, func) + }) + ############################## 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 # diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 0e4350f861e4a..b0f25deaf3e82 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -469,6 +469,10 @@ setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") #' @export setGeneric("gapply", function(x, ...) { standardGeneric("gapply") }) +#' @rdname gapplyCollect +#' @export +setGeneric("gapplyCollect", function(x, ...) { standardGeneric("gapplyCollect") }) + #' @rdname summary #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 0687f14adf7b9..5ed7e8abb43de 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -196,64 +196,51 @@ createMethods() #' gapply #' -#' Applies a R function to each group in the input GroupedData -#' -#' @param x a GroupedData -#' @param func A function to be applied to each group partition specified by GroupedData. -#' The function `func` takes as argument a key - grouping columns and -#' a data frame - a local R data.frame. -#' The output of `func` is a local R data.frame. -#' @param schema The schema of the resulting SparkDataFrame after the function is applied. -#' The schema must match to output of `func`. It has to be defined for each -#' output column with preferred output column name and corresponding data type. -#' @return a SparkDataFrame +#' @param x A GroupedData #' @rdname gapply #' @name gapply #' @export -#' @examples -#' \dontrun{ -#' Computes the arithmetic mean of the second column by grouping -#' on the first and third columns. Output the grouping values and the average. -#' -#' df <- createDataFrame ( -#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), -#' c("a", "b", "c", "d")) -#' -#' Here our output contains three columns, the key which is a combination of two -#' columns with data types integer and string and the mean which is a double. -#' schema <- structType(structField("a", "integer"), structField("c", "string"), -#' structField("avg", "double")) -#' df1 <- gapply( -#' df, -#' list("a", "c"), -#' function(key, x) { -#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) -#' }, -#' schema) -#' collect(df1) -#' -#' Result -#' ------ -#' a c avg -#' 3 3 3.0 -#' 1 1 1.5 -#' } #' @note gapply(GroupedData) since 2.0.0 setMethod("gapply", signature(x = "GroupedData"), function(x, func, schema) { - try(if (is.null(schema)) stop("schema cannot be NULL")) - packageNamesArr <- serialize(.sparkREnv[[".packages"]], - connection = NULL) - broadcastArr <- lapply(ls(.broadcastNames), - function(name) { get(name, .broadcastNames) }) - sdf <- callJStatic( - "org.apache.spark.sql.api.r.SQLUtils", - "gapply", - x@sgd, - serialize(cleanClosure(func), connection = NULL), - packageNamesArr, - broadcastArr, - schema$jobj) - dataFrame(sdf) + if (is.null(schema)) stop("schema cannot be NULL") + gapplyInternal(x, func, schema) + }) + +#' gapplyCollect +#' +#' @param x A GroupedData +#' @rdname gapplyCollect +#' @name gapplyCollect +#' @export +#' @note gapplyCollect(GroupedData) since 2.0.0 +setMethod("gapplyCollect", + signature(x = "GroupedData"), + function(x, func) { + gdf <- gapplyInternal(x, func, NULL) + content <- callJMethod(gdf@sdf, "collect") + # content is a list of items of struct type. Each item has a single field + # which is a serialized data.frame corresponds to one group of the + # SparkDataFrame. + ldfs <- lapply(content, function(x) { unserialize(x[[1]]) }) + ldf <- do.call(rbind, ldfs) + row.names(ldf) <- NULL + ldf }) + +gapplyInternal <- function(x, func, schema) { + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "gapply", + x@sgd, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + if (class(schema) == "structType") { schema$jobj } else { NULL }) + dataFrame(sdf) +} \ No newline at end of file diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index deda1b6f43588..e05e5c4f1a5b7 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2249,21 +2249,24 @@ test_that("repartition by columns on DataFrame", { expect_equal(nrow(df1), 2) }) -test_that("gapply() on a DataFrame", { +test_that("gapply() and gapplyCollect() on a DataFrame", { df <- createDataFrame ( list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), c("a", "b", "c", "d")) expected <- collect(df) - df1 <- gapply(df, list("a"), function(key, x) { x }, schema(df)) + df1 <- gapply(df, "a", function(key, x) { x }, schema(df)) actual <- collect(df1) expect_identical(actual, expected) + df1Collect <- gapplyCollect(df, list("a"), function(key, x) { x }) + expect_identical(df1Collect, expected) + # Computes the sum of second column by grouping on the first and third columns # and checks if the sum is larger than 2 schema <- structType(structField("a", "integer"), structField("e", "boolean")) df2 <- gapply( df, - list(df$"a", df$"c"), + c(df$"a", df$"c"), function(key, x) { y <- data.frame(key[1], sum(x$b) > 2) }, @@ -2272,13 +2275,24 @@ test_that("gapply() on a DataFrame", { expected <- c(TRUE, TRUE) expect_identical(actual, expected) + df2Collect <- gapplyCollect( + df, + c(df$"a", df$"c"), + function(key, x) { + y <- data.frame(key[1], sum(x$b) > 2) + colnames(y) <- c("a", "e") + y + }) + actual <- df2Collect$e + expect_identical(actual, expected) + # Computes the arithmetic mean of the second column by grouping # on the first and third columns. Output the groupping value and the average. schema <- structType(structField("a", "integer"), structField("c", "string"), structField("avg", "double")) df3 <- gapply( df, - list("a", "c"), + c("a", "c"), function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, @@ -2293,11 +2307,22 @@ test_that("gapply() on a DataFrame", { rownames(expected) <- NULL expect_identical(actual, expected) + df3Collect <- gapplyCollect( + df, + c("a", "c"), + function(key, x) { + y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) + colnames(y) <- c("a", "c", "avg") + y + }) + actual <- df3Collect[order(df3Collect$a), ] + expect_identical(actual$avg, expected$avg) + irisDF <- suppressWarnings(createDataFrame (iris)) schema <- structType(structField("Sepal_Length", "double"), structField("Avg", "double")) # Groups by `Sepal_Length` and computes the average for `Sepal_Width` df4 <- gapply( - cols = list("Sepal_Length"), + cols = "Sepal_Length", irisDF, function(key, x) { y <- data.frame(key, mean(x$Sepal_Width), stringsAsFactors = FALSE) From d6588115a9ec3178f7d1edc86418a9832c9b3ac7 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Fri, 1 Jul 2016 14:37:03 -0700 Subject: [PATCH 0877/1470] [SPARK-16299][SPARKR] Capture errors from R workers in daemon.R to avoid deletion of R session temporary directory. ## What changes were proposed in this pull request? Capture errors from R workers in daemon.R to avoid deletion of R session temporary directory. See detailed description at https://issues.apache.org/jira/browse/SPARK-16299 ## How was this patch tested? SparkR unit tests. Author: Sun Rui Closes #13975 from sun-rui/SPARK-16299. (cherry picked from commit e4fa58c43ce2bf8d76bffb0d9dc1132f8d0eae6a) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/worker/daemon.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/inst/worker/daemon.R b/R/pkg/inst/worker/daemon.R index f55beac6c8c07..b92e6be995ca9 100644 --- a/R/pkg/inst/worker/daemon.R +++ b/R/pkg/inst/worker/daemon.R @@ -44,7 +44,7 @@ while (TRUE) { if (inherits(p, "masterProcess")) { close(inputCon) Sys.setenv(SPARKR_WORKER_PORT = port) - source(script) + try(source(script)) # Set SIGUSR1 so that child can exit tools::pskill(Sys.getpid(), tools::SIGUSR1) parallel:::mcexit(0L) From 78387ce0595aa8e3f300844494610b49c8c6d844 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 1 Jul 2016 15:16:04 -0700 Subject: [PATCH 0878/1470] [SPARK-16335][SQL] Structured streaming should fail if source directory does not exist ## What changes were proposed in this pull request? In structured streaming, Spark does not report errors when the specified directory does not exist. This is a behavior different from the batch mode. This patch changes the behavior to fail if the directory does not exist (when the path is not a glob pattern). ## How was this patch tested? Updated unit tests to reflect the new behavior. Author: Reynold Xin Closes #14002 from rxin/SPARK-16335. (cherry picked from commit d601894c0494d415e7f330e02168c43a2dacfb02) Signed-off-by: Reynold Xin --- .../apache/spark/deploy/SparkHadoopUtil.scala | 10 ++--- python/pyspark/sql/streaming.py | 11 ++---- .../execution/datasources/DataSource.scala | 12 ++++++ .../sql/streaming/FileStreamSourceSuite.scala | 38 ++++++++----------- 4 files changed, 36 insertions(+), 35 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 bb1793d451dfd..90c71cc6cfab7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -232,6 +232,10 @@ class SparkHadoopUtil extends Logging { recurse(baseStatus) } + def isGlobPath(pattern: Path): Boolean = { + pattern.toString.exists("{}[]*?\\".toSet.contains) + } + def globPath(pattern: Path): Seq[Path] = { val fs = pattern.getFileSystem(conf) Option(fs.globStatus(pattern)).map { statuses => @@ -240,11 +244,7 @@ class SparkHadoopUtil extends Logging { } def globPathIfNecessary(pattern: Path): Seq[Path] = { - if (pattern.toString.exists("{}[]*?\\".toSet.contains)) { - globPath(pattern) - } else { - Seq(pattern) - } + if (isGlobPath(pattern)) globPath(pattern) else Seq(pattern) } /** diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index bffe398247ba9..8bac347e13084 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -315,7 +315,7 @@ def load(self, path=None, format=None, schema=None, **options): >>> json_sdf = spark.readStream.format("json")\ .schema(sdf_schema)\ - .load(os.path.join(tempfile.mkdtemp(),'data')) + .load(tempfile.mkdtemp()) >>> json_sdf.isStreaming True >>> json_sdf.schema == sdf_schema @@ -382,8 +382,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. - >>> json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), \ - schema = sdf_schema) + >>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema) >>> json_sdf.isStreaming True >>> json_sdf.schema == sdf_schema @@ -411,8 +410,7 @@ def parquet(self, path): .. note:: Experimental. - >>> parquet_sdf = spark.readStream.schema(sdf_schema)\ - .parquet(os.path.join(tempfile.mkdtemp())) + >>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp()) >>> parquet_sdf.isStreaming True >>> parquet_sdf.schema == sdf_schema @@ -512,8 +510,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non * ``DROPMALFORMED`` : ignores the whole corrupted records. * ``FAILFAST`` : throws an exception when it meets corrupted records. - >>> csv_sdf = spark.readStream.csv(os.path.join(tempfile.mkdtemp(), 'data'), \ - schema = sdf_schema) + >>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema) >>> csv_sdf.isStreaming True >>> csv_sdf.schema == sdf_schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index a4110d7b11473..6dc27c19521ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -203,6 +203,18 @@ case class DataSource( val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") }) + + // Check whether the path exists if it is not a glob pattern. + // For glob pattern, we do not check it because the glob pattern might only make sense + // once the streaming job starts and some upstream source starts dropping data. + val hdfsPath = new Path(path) + if (!SparkHadoopUtil.get.isGlobPath(hdfsPath)) { + val fs = hdfsPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) + if (!fs.exists(hdfsPath)) { + throw new AnalysisException(s"Path does not exist: $path") + } + } + val isSchemaInferenceEnabled = sparkSession.conf.get(SQLConf.STREAMING_SCHEMA_INFERENCE) val isTextSource = providingClass == classOf[text.TextFileFormat] // If the schema inference is disabled, only text sources require schema to be specified diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 6c04846f00e8b..8a34cf95f918d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -179,18 +179,24 @@ class FileStreamSourceSuite extends FileStreamSourceTest { withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { testError() } } - test("FileStreamSource schema: path doesn't exist, no schema") { - val e = intercept[IllegalArgumentException] { - createFileStreamSourceAndGetSchema(format = None, path = Some("/a/b/c"), schema = None) + test("FileStreamSource schema: path doesn't exist (without schema) should throw exception") { + withTempDir { dir => + intercept[AnalysisException] { + val userSchema = new StructType().add(new StructField("value", IntegerType)) + val schema = createFileStreamSourceAndGetSchema( + format = None, path = Some(new File(dir, "1").getAbsolutePath), schema = None) + } } - assert(e.getMessage.toLowerCase.contains("schema")) // reason is schema absence, not the path } - test("FileStreamSource schema: path doesn't exist, with schema") { - val userSchema = new StructType().add(new StructField("value", IntegerType)) - val schema = createFileStreamSourceAndGetSchema( - format = None, path = Some("/a/b/c"), schema = Some(userSchema)) - assert(schema === userSchema) + test("FileStreamSource schema: path doesn't exist (with schema) should throw exception") { + withTempDir { dir => + intercept[AnalysisException] { + val userSchema = new StructType().add(new StructField("value", IntegerType)) + val schema = createFileStreamSourceAndGetSchema( + format = None, path = Some(new File(dir, "1").getAbsolutePath), schema = Some(userSchema)) + } + } } @@ -225,20 +231,6 @@ class FileStreamSourceSuite extends FileStreamSourceTest { // =============== Parquet file stream schema tests ================ - ignore("FileStreamSource schema: parquet, no existing files, no schema") { - withTempDir { src => - withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { - val e = intercept[AnalysisException] { - createFileStreamSourceAndGetSchema( - format = Some("parquet"), - path = Some(new File(src, "1").getCanonicalPath), - schema = None) - } - assert("Unable to infer schema. It must be specified manually.;" === e.getMessage) - } - } - } - test("FileStreamSource schema: parquet, existing files, no schema") { withTempDir { src => Seq("a", "b", "c").toDS().as("userColumn").toDF().write From 794d099691c3ef71b25178992086b4f25e4019e6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 1 Jul 2016 15:35:19 -0700 Subject: [PATCH 0879/1470] [SPARK-16233][R][TEST] ORC test should be enabled only when HiveContext is available. ## What changes were proposed in this pull request? ORC test should be enabled only when HiveContext is available. ## How was this patch tested? Manual. ``` $ R/run-tests.sh ... 1. create DataFrame from RDD (test_sparkSQL.R#200) - Hive is not build with SparkSQL, skipped 2. test HiveContext (test_sparkSQL.R#1021) - Hive is not build with SparkSQL, skipped 3. read/write ORC files (test_sparkSQL.R#1728) - Hive is not build with SparkSQL, skipped 4. enableHiveSupport on SparkSession (test_sparkSQL.R#2448) - Hive is not build with SparkSQL, skipped 5. sparkJars tag in SparkContext (test_Windows.R#21) - This test is only for Windows, skipped DONE =========================================================================== Tests passed. ``` Author: Dongjoon Hyun Closes #14019 from dongjoon-hyun/SPARK-16233. (cherry picked from commit d17e5f2f123eecd5a7a1d87f5ce75a0fc44552b4) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 ++ 1 file changed, 2 insertions(+) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index e05e5c4f1a5b7..d22baf6a20f0f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1717,6 +1717,7 @@ test_that("mutate(), transform(), rename() and names()", { }) test_that("read/write ORC files", { + setHiveContext(sc) df <- read.df(jsonPath, "json") # Test write.df and read.df @@ -1733,6 +1734,7 @@ test_that("read/write ORC files", { expect_equal(count(orcDF), count(df)) unlink(orcPath2) + unsetHiveContext() }) test_that("read/write Parquet files", { From ab4303800d04c12828dd2896add3e84b2545a25a Mon Sep 17 00:00:00 2001 From: "peng.zhang" Date: Fri, 1 Jul 2016 15:51:21 -0700 Subject: [PATCH 0880/1470] [SPARK-16095][YARN] Yarn cluster mode should report correct state to SparkLauncher ## What changes were proposed in this pull request? Yarn cluster mode should return correct state for SparkLauncher ## How was this patch tested? unit test Author: peng.zhang Closes #13962 from renozhang/SPARK-16095-spark-launcher-wrong-state. (cherry picked from commit bad0f7dbba2eda149ee4fc5810674d971d17874a) Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/deploy/yarn/Client.scala | 9 ++++- .../spark/deploy/yarn/YarnClusterSuite.scala | 37 ++++++++++++------- 2 files changed, 31 insertions(+), 15 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 9bb369549d943..01aa12a3c9a71 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 @@ -1053,7 +1053,14 @@ private[spark] class Client( case YarnApplicationState.RUNNING => reportLauncherState(SparkAppHandle.State.RUNNING) case YarnApplicationState.FINISHED => - reportLauncherState(SparkAppHandle.State.FINISHED) + report.getFinalApplicationStatus match { + case FinalApplicationStatus.FAILED => + reportLauncherState(SparkAppHandle.State.FAILED) + case FinalApplicationStatus.KILLED => + reportLauncherState(SparkAppHandle.State.KILLED) + case _ => + reportLauncherState(SparkAppHandle.State.FINISHED) + } case YarnApplicationState.FAILED => reportLauncherState(SparkAppHandle.State.FAILED) case YarnApplicationState.KILLED => 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 6b20dea5908a8..9085fca1d3cc0 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 @@ -120,6 +120,11 @@ class YarnClusterSuite extends BaseYarnClusterSuite { finalState should be (SparkAppHandle.State.FAILED) } + test("run Spark in yarn-cluster mode failure after sc initialized") { + val finalState = runSpark(false, mainClassName(YarnClusterDriverWithFailure.getClass)) + finalState should be (SparkAppHandle.State.FAILED) + } + test("run Python application in yarn-client mode") { testPySpark(true) } @@ -259,6 +264,16 @@ private[spark] class SaveExecutorInfo extends SparkListener { } } +private object YarnClusterDriverWithFailure extends Logging with Matchers { + def main(args: Array[String]): Unit = { + val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) + .setAppName("yarn test with failure")) + + throw new Exception("exception after sc initialized") + } +} + private object YarnClusterDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 @@ -287,19 +302,19 @@ private object YarnClusterDriver extends Logging with Matchers { sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) data should be (Set(1, 2, 3, 4)) result = "success" + + // Verify that the config archive is correctly placed in the classpath of all containers. + val confFile = "/" + Client.SPARK_CONF_FILE + assert(getClass().getResource(confFile) != null) + val configFromExecutors = sc.parallelize(1 to 4, 4) + .map { _ => Option(getClass().getResource(confFile)).map(_.toString).orNull } + .collect() + assert(configFromExecutors.find(_ == null) === None) } finally { Files.write(result, status, StandardCharsets.UTF_8) sc.stop() } - // Verify that the config archive is correctly placed in the classpath of all containers. - val confFile = "/" + Client.SPARK_CONF_FILE - assert(getClass().getResource(confFile) != null) - val configFromExecutors = sc.parallelize(1 to 4, 4) - .map { _ => Option(getClass().getResource(confFile)).map(_.toString).orNull } - .collect() - assert(configFromExecutors.find(_ == null) === None) - // verify log urls are present val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] assert(listeners.size === 1) @@ -330,9 +345,6 @@ private object YarnClusterDriver extends Logging with Matchers { } private object YarnClasspathTest extends Logging { - - var exitCode = 0 - def error(m: String, ex: Throwable = null): Unit = { logError(m, ex) // scalastyle:off println @@ -361,7 +373,6 @@ private object YarnClasspathTest extends Logging { } finally { sc.stop() } - System.exit(exitCode) } private def readResource(resultPath: String): Unit = { @@ -374,8 +385,6 @@ private object YarnClasspathTest extends Logging { } catch { case t: Throwable => error(s"loading test.resource to $resultPath", t) - // set the exit code if not yet set - exitCode = 2 } finally { Files.write(result, new File(resultPath), StandardCharsets.UTF_8) } From f3a359939afb25c8b91fabe5955e1cdf609be521 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Sat, 2 Jul 2016 08:40:23 +0100 Subject: [PATCH 0881/1470] [GRAPHX][EXAMPLES] move graphx test data directory and update graphx document ## What changes were proposed in this pull request? There are two test data files used for graphx examples existing in directory "graphx/data" I move it into "data/" directory because the "graphx" directory is used for code files and other test data files (such as mllib, streaming test data) are all in there. I also update the graphx document where reference the data files which I move place. ## How was this patch tested? N/A Author: WeichenXu Closes #14010 from WeichenXu123/move_graphx_data_dir. (cherry picked from commit 192d1f9cf3463d050b87422939448f2acf86acc9) Signed-off-by: Sean Owen --- {graphx/data => data/graphx}/followers.txt | 0 {graphx/data => data/graphx}/users.txt | 0 docs/graphx-programming-guide.md | 18 +++++++++--------- 3 files changed, 9 insertions(+), 9 deletions(-) rename {graphx/data => data/graphx}/followers.txt (100%) rename {graphx/data => data/graphx}/users.txt (100%) diff --git a/graphx/data/followers.txt b/data/graphx/followers.txt similarity index 100% rename from graphx/data/followers.txt rename to data/graphx/followers.txt diff --git a/graphx/data/users.txt b/data/graphx/users.txt similarity index 100% rename from graphx/data/users.txt rename to data/graphx/users.txt diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 81cf17475fb60..e376b6638ef6d 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -1007,15 +1007,15 @@ PageRank measures the importance of each vertex in a graph, assuming an edge fro GraphX comes with static and dynamic implementations of PageRank as methods on the [`PageRank` object][PageRank]. Static PageRank runs for a fixed number of iterations, while dynamic PageRank runs until the ranks converge (i.e., stop changing by more than a specified tolerance). [`GraphOps`][GraphOps] allows calling these algorithms directly as methods on `Graph`. -GraphX also includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We compute the PageRank of each user as follows: +GraphX also includes an example social network dataset that we can run PageRank on. A set of users is given in `data/graphx/users.txt`, and a set of relationships between users is given in `data/graphx/followers.txt`. We compute the PageRank of each user as follows: {% highlight scala %} // Load the edges as a graph -val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt") +val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") // Run PageRank val ranks = graph.pageRank(0.0001).vertices // Join the ranks with the usernames -val users = sc.textFile("graphx/data/users.txt").map { line => +val users = sc.textFile("data/graphx/users.txt").map { line => val fields = line.split(",") (fields(0).toLong, fields(1)) } @@ -1032,11 +1032,11 @@ The connected components algorithm labels each connected component of the graph {% highlight scala %} // Load the graph as in the PageRank example -val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt") +val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") // Find the connected components val cc = graph.connectedComponents().vertices // Join the connected components with the usernames -val users = sc.textFile("graphx/data/users.txt").map { line => +val users = sc.textFile("data/graphx/users.txt").map { line => val fields = line.split(",") (fields(0).toLong, fields(1)) } @@ -1053,11 +1053,11 @@ A vertex is part of a triangle when it has two adjacent vertices with an edge be {% highlight scala %} // Load the edges in canonical order and partition the graph for triangle count -val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(PartitionStrategy.RandomVertexCut) +val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt", true).partitionBy(PartitionStrategy.RandomVertexCut) // Find the triangle count for each vertex val triCounts = graph.triangleCount().vertices // Join the triangle counts with the usernames -val users = sc.textFile("graphx/data/users.txt").map { line => +val users = sc.textFile("data/graphx/users.txt").map { line => val fields = line.split(",") (fields(0).toLong, fields(1)) } @@ -1081,11 +1081,11 @@ all of this in just a few lines with GraphX: val sc = new SparkContext("spark://master.amplab.org", "research") // Load my user data and parse into tuples of user id and attribute list -val users = (sc.textFile("graphx/data/users.txt") +val users = (sc.textFile("data/graphx/users.txt") .map(line => line.split(",")).map( parts => (parts.head.toLong, parts.tail) )) // Parse the edge data which is already in userId -> userId format -val followerGraph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt") +val followerGraph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") // Attach the user attributes val graph = followerGraph.outerJoinVertices(users) { From 0d0b416097a095fa771a7d5ae368546c26cb2d8b Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Sat, 2 Jul 2016 16:29:00 +0100 Subject: [PATCH 0882/1470] [SPARK-16345][DOCUMENTATION][EXAMPLES][GRAPHX] Extract graphx programming guide example snippets from source files instead of hard code them ## What changes were proposed in this pull request? I extract 6 example programs from GraphX programming guide and replace them with `include_example` label. The 6 example programs are: - AggregateMessagesExample.scala - SSSPExample.scala - TriangleCountingExample.scala - ConnectedComponentsExample.scala - ComprehensiveExample.scala - PageRankExample.scala All the example code can run using `bin/run-example graphx.EXAMPLE_NAME` ## How was this patch tested? Manual. Author: WeichenXu Closes #14015 from WeichenXu123/graphx_example_plugin. (cherry picked from commit 0bd7cd18bc4d535b0c4499913f6747b3f6315ac2) Signed-off-by: Sean Owen --- docs/graphx-programming-guide.md | 133 +----------------- .../graphx/AggregateMessagesExample.scala | 72 ++++++++++ .../graphx/ComprehensiveExample.scala | 80 +++++++++++ .../graphx/ConnectedComponentsExample.scala | 68 +++++++++ .../examples/graphx/PageRankExample.scala | 61 ++++++++ .../spark/examples/graphx/SSSPExample.scala | 69 +++++++++ .../graphx/TriangleCountingExample.scala | 70 +++++++++ 7 files changed, 426 insertions(+), 127 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index e376b6638ef6d..2e9966c0a2b60 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -603,29 +603,7 @@ slightly unreliable and instead opted for more explicit user control. In the following example we use the [`aggregateMessages`][Graph.aggregateMessages] operator to compute the average age of the more senior followers of each user. -{% highlight scala %} -// Import random graph generation library -import org.apache.spark.graphx.util.GraphGenerators -// Create a graph with "age" as the vertex property. Here we use a random graph for simplicity. -val graph: Graph[Double, Int] = - GraphGenerators.logNormalGraph(sc, numVertices = 100).mapVertices( (id, _) => id.toDouble ) -// Compute the number of older followers and their total age -val olderFollowers: VertexRDD[(Int, Double)] = graph.aggregateMessages[(Int, Double)]( - triplet => { // Map Function - if (triplet.srcAttr > triplet.dstAttr) { - // Send message to destination vertex containing counter and age - triplet.sendToDst(1, triplet.srcAttr) - } - }, - // Add counter and age - (a, b) => (a._1 + b._1, a._2 + b._2) // Reduce Function -) -// Divide total age by number of older followers to get average age of older followers -val avgAgeOfOlderFollowers: VertexRDD[Double] = - olderFollowers.mapValues( (id, value) => value match { case (count, totalAge) => totalAge / count } ) -// Display the results -avgAgeOfOlderFollowers.collect.foreach(println(_)) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala %} > The `aggregateMessages` operation performs optimally when the messages (and the sums of > messages) are constant sized (e.g., floats and addition instead of lists and concatenation). @@ -793,29 +771,7 @@ second argument list contains the user defined functions for receiving messages We can use the Pregel operator to express computation such as single source shortest path in the following example. -{% highlight scala %} -import org.apache.spark.graphx._ -// Import random graph generation library -import org.apache.spark.graphx.util.GraphGenerators -// A graph with edge attributes containing distances -val graph: Graph[Long, Double] = - GraphGenerators.logNormalGraph(sc, numVertices = 100).mapEdges(e => e.attr.toDouble) -val sourceId: VertexId = 42 // The ultimate source -// Initialize the graph such that all vertices except the root have distance infinity. -val initialGraph = graph.mapVertices((id, _) => if (id == sourceId) 0.0 else Double.PositiveInfinity) -val sssp = initialGraph.pregel(Double.PositiveInfinity)( - (id, dist, newDist) => math.min(dist, newDist), // Vertex Program - triplet => { // Send Message - if (triplet.srcAttr + triplet.attr < triplet.dstAttr) { - Iterator((triplet.dstId, triplet.srcAttr + triplet.attr)) - } else { - Iterator.empty - } - }, - (a,b) => math.min(a,b) // Merge Message - ) -println(sssp.vertices.collect.mkString("\n")) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/graphx/SSSPExample.scala %} @@ -1009,64 +965,19 @@ GraphX comes with static and dynamic implementations of PageRank as methods on t GraphX also includes an example social network dataset that we can run PageRank on. A set of users is given in `data/graphx/users.txt`, and a set of relationships between users is given in `data/graphx/followers.txt`. We compute the PageRank of each user as follows: -{% highlight scala %} -// Load the edges as a graph -val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") -// Run PageRank -val ranks = graph.pageRank(0.0001).vertices -// Join the ranks with the usernames -val users = sc.textFile("data/graphx/users.txt").map { line => - val fields = line.split(",") - (fields(0).toLong, fields(1)) -} -val ranksByUsername = users.join(ranks).map { - case (id, (username, rank)) => (username, rank) -} -// Print the result -println(ranksByUsername.collect().mkString("\n")) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/graphx/PageRankExample.scala %} ## Connected Components The connected components algorithm labels each connected component of the graph with the ID of its lowest-numbered vertex. For example, in a social network, connected components can approximate clusters. GraphX contains an implementation of the algorithm in the [`ConnectedComponents` object][ConnectedComponents], and we compute the connected components of the example social network dataset from the [PageRank section](#pagerank) as follows: -{% highlight scala %} -// Load the graph as in the PageRank example -val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") -// Find the connected components -val cc = graph.connectedComponents().vertices -// Join the connected components with the usernames -val users = sc.textFile("data/graphx/users.txt").map { line => - val fields = line.split(",") - (fields(0).toLong, fields(1)) -} -val ccByUsername = users.join(cc).map { - case (id, (username, cc)) => (username, cc) -} -// Print the result -println(ccByUsername.collect().mkString("\n")) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala %} ## Triangle Counting A vertex is part of a triangle when it has two adjacent vertices with an edge between them. GraphX implements a triangle counting algorithm in the [`TriangleCount` object][TriangleCount] that determines the number of triangles passing through each vertex, providing a measure of clustering. We compute the triangle count of the social network dataset from the [PageRank section](#pagerank). *Note that `TriangleCount` requires the edges to be in canonical orientation (`srcId < dstId`) and the graph to be partitioned using [`Graph.partitionBy`][Graph.partitionBy].* -{% highlight scala %} -// Load the edges in canonical order and partition the graph for triangle count -val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt", true).partitionBy(PartitionStrategy.RandomVertexCut) -// Find the triangle count for each vertex -val triCounts = graph.triangleCount().vertices -// Join the triangle counts with the usernames -val users = sc.textFile("data/graphx/users.txt").map { line => - val fields = line.split(",") - (fields(0).toLong, fields(1)) -} -val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) => - (username, tc) -} -// Print the result -println(triCountByUsername.collect().mkString("\n")) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala %} # Examples @@ -1076,36 +987,4 @@ to important relationships and users, run page-rank on the sub-graph, and then finally return attributes associated with the top users. I can do all of this in just a few lines with GraphX: -{% highlight scala %} -// Connect to the Spark cluster -val sc = new SparkContext("spark://master.amplab.org", "research") - -// Load my user data and parse into tuples of user id and attribute list -val users = (sc.textFile("data/graphx/users.txt") - .map(line => line.split(",")).map( parts => (parts.head.toLong, parts.tail) )) - -// Parse the edge data which is already in userId -> userId format -val followerGraph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") - -// Attach the user attributes -val graph = followerGraph.outerJoinVertices(users) { - case (uid, deg, Some(attrList)) => attrList - // Some users may not have attributes so we set them as empty - case (uid, deg, None) => Array.empty[String] -} - -// Restrict the graph to users with usernames and names -val subgraph = graph.subgraph(vpred = (vid, attr) => attr.size == 2) - -// Compute the PageRank -val pagerankGraph = subgraph.pageRank(0.001) - -// Get the attributes of the top pagerank users -val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices) { - case (uid, attrList, Some(pr)) => (pr, attrList.toList) - case (uid, attrList, None) => (0.0, attrList.toList) -} - -println(userInfoWithPageRank.vertices.top(5)(Ordering.by(_._2._1)).mkString("\n")) - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala %} diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala new file mode 100644 index 0000000000000..8f8262db374b8 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala @@ -0,0 +1,72 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.{Graph, VertexRDD} +import org.apache.spark.graphx.util.GraphGenerators +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example use the [`aggregateMessages`][Graph.aggregateMessages] operator to + * compute the average age of the more senior followers of each user + * Run with + * {{{ + * bin/run-example graphx.AggregateMessagesExample + * }}} + */ +object AggregateMessagesExample { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Create a graph with "age" as the vertex property. + // Here we use a random graph for simplicity. + val graph: Graph[Double, Int] = + GraphGenerators.logNormalGraph(sc, numVertices = 100).mapVertices( (id, _) => id.toDouble ) + // Compute the number of older followers and their total age + val olderFollowers: VertexRDD[(Int, Double)] = graph.aggregateMessages[(Int, Double)]( + triplet => { // Map Function + if (triplet.srcAttr > triplet.dstAttr) { + // Send message to destination vertex containing counter and age + triplet.sendToDst(1, triplet.srcAttr) + } + }, + // Add counter and age + (a, b) => (a._1 + b._1, a._2 + b._2) // Reduce Function + ) + // Divide total age by number of older followers to get average age of older followers + val avgAgeOfOlderFollowers: VertexRDD[Double] = + olderFollowers.mapValues( (id, value) => + value match { case (count, totalAge) => totalAge / count } ) + // Display the results + avgAgeOfOlderFollowers.collect.foreach(println(_)) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala new file mode 100644 index 0000000000000..6598863bd2ea0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala @@ -0,0 +1,80 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.GraphLoader +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * Suppose I want to build a graph from some text files, restrict the graph + * to important relationships and users, run page-rank on the sub-graph, and + * then finally return attributes associated with the top users. + * This example do all of this in just a few lines with GraphX. + * + * Run with + * {{{ + * bin/run-example graphx.ComprehensiveExample + * }}} + */ +object ComprehensiveExample { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load my user data and parse into tuples of user id and attribute list + val users = (sc.textFile("data/graphx/users.txt") + .map(line => line.split(",")).map( parts => (parts.head.toLong, parts.tail) )) + + // Parse the edge data which is already in userId -> userId format + val followerGraph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") + + // Attach the user attributes + val graph = followerGraph.outerJoinVertices(users) { + case (uid, deg, Some(attrList)) => attrList + // Some users may not have attributes so we set them as empty + case (uid, deg, None) => Array.empty[String] + } + + // Restrict the graph to users with usernames and names + val subgraph = graph.subgraph(vpred = (vid, attr) => attr.size == 2) + + // Compute the PageRank + val pagerankGraph = subgraph.pageRank(0.001) + + // Get the attributes of the top pagerank users + val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices) { + case (uid, attrList, Some(pr)) => (pr, attrList.toList) + case (uid, attrList, None) => (0.0, attrList.toList) + } + + println(userInfoWithPageRank.vertices.top(5)(Ordering.by(_._2._1)).mkString("\n")) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala new file mode 100644 index 0000000000000..5377ddb3594b0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala @@ -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. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.GraphLoader +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A connected components algorithm example. + * The connected components algorithm labels each connected component of the graph + * with the ID of its lowest-numbered vertex. + * For example, in a social network, connected components can approximate clusters. + * GraphX contains an implementation of the algorithm in the + * [`ConnectedComponents` object][ConnectedComponents], + * and we compute the connected components of the example social network dataset. + * + * Run with + * {{{ + * bin/run-example graphx.ConnectedComponentsExample + * }}} + */ +object ConnectedComponentsExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load the graph as in the PageRank example + val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") + // Find the connected components + val cc = graph.connectedComponents().vertices + // Join the connected components with the usernames + val users = sc.textFile("data/graphx/users.txt").map { line => + val fields = line.split(",") + (fields(0).toLong, fields(1)) + } + val ccByUsername = users.join(cc).map { + case (id, (username, cc)) => (username, cc) + } + // Print the result + println(ccByUsername.collect().mkString("\n")) + // $example off$ + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala new file mode 100644 index 0000000000000..9e9affca07a18 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.GraphLoader +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A PageRank example on social network dataset + * Run with + * {{{ + * bin/run-example graphx.PageRankExample + * }}} + */ +object PageRankExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load the edges as a graph + val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") + // Run PageRank + val ranks = graph.pageRank(0.0001).vertices + // Join the ranks with the usernames + val users = sc.textFile("data/graphx/users.txt").map { line => + val fields = line.split(",") + (fields(0).toLong, fields(1)) + } + val ranksByUsername = users.join(ranks).map { + case (id, (username, rank)) => (username, rank) + } + // Print the result + println(ranksByUsername.collect().mkString("\n")) + // $example off$ + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala new file mode 100644 index 0000000000000..5e8b19671de7a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala @@ -0,0 +1,69 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.{Graph, VertexId} +import org.apache.spark.graphx.util.GraphGenerators +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example use the Pregel operator to express computation + * such as single source shortest path + * Run with + * {{{ + * bin/run-example graphx.SSSPExample + * }}} + */ +object SSSPExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // A graph with edge attributes containing distances + val graph: Graph[Long, Double] = + GraphGenerators.logNormalGraph(sc, numVertices = 100).mapEdges(e => e.attr.toDouble) + val sourceId: VertexId = 42 // The ultimate source + // Initialize the graph such that all vertices except the root have distance infinity. + val initialGraph = graph.mapVertices((id, _) => + if (id == sourceId) 0.0 else Double.PositiveInfinity) + val sssp = initialGraph.pregel(Double.PositiveInfinity)( + (id, dist, newDist) => math.min(dist, newDist), // Vertex Program + triplet => { // Send Message + if (triplet.srcAttr + triplet.attr < triplet.dstAttr) { + Iterator((triplet.dstId, triplet.srcAttr + triplet.attr)) + } else { + Iterator.empty + } + }, + (a, b) => math.min(a, b) // Merge Message + ) + println(sssp.vertices.collect.mkString("\n")) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala new file mode 100644 index 0000000000000..b9bff69086cc1 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.{GraphLoader, PartitionStrategy} +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A vertex is part of a triangle when it has two adjacent vertices with an edge between them. + * GraphX implements a triangle counting algorithm in the [`TriangleCount` object][TriangleCount] + * that determines the number of triangles passing through each vertex, + * providing a measure of clustering. + * We compute the triangle count of the social network dataset. + * + * Note that `TriangleCount` requires the edges to be in canonical orientation (`srcId < dstId`) + * and the graph to be partitioned using [`Graph.partitionBy`][Graph.partitionBy]. + * + * Run with + * {{{ + * bin/run-example graphx.TriangleCountingExample + * }}} + */ +object TriangleCountingExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load the edges in canonical order and partition the graph for triangle count + val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt", true) + .partitionBy(PartitionStrategy.RandomVertexCut) + // Find the triangle count for each vertex + val triCounts = graph.triangleCount().vertices + // Join the triangle counts with the usernames + val users = sc.textFile("data/graphx/users.txt").map { line => + val fields = line.split(",") + (fields(0).toLong, fields(1)) + } + val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) => + (username, tc) + } + // Print the result + println(triCountByUsername.collect().mkString("\n")) + // $example off$ + spark.stop() + } +} +// scalastyle:on println From 0c6fd03fa763df4afb77ac4738c76f0b73e46ad0 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 2 Jul 2016 16:31:06 +0100 Subject: [PATCH 0883/1470] [MINOR][BUILD] Fix Java linter errors This PR fixes the minor Java linter errors like the following. ``` - public int read(char cbuf[], int off, int len) throws IOException { + public int read(char[] cbuf, int off, int len) throws IOException { ``` Manual. ``` $ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install $ dev/lint-java Using `mvn` from path: /usr/local/bin/mvn Checkstyle checks passed. ``` Author: Dongjoon Hyun Closes #14017 from dongjoon-hyun/minor_build_java_linter_error. (cherry picked from commit 3000b4b29f9165f436f186a8c1ba818e24f90615) Signed-off-by: Sean Owen --- .../spark/shuffle/sort/ShuffleExternalSorter.java | 3 ++- .../collection/unsafe/sort/UnsafeExternalSorter.java | 12 ++++++------ .../spark/sql/execution/UnsafeExternalRowSorter.java | 4 ++-- .../execution/UnsafeFixedWidthAggregationMap.java | 4 ++-- .../spark/sql/execution/UnsafeKVExternalSorter.java | 3 ++- 5 files changed, 14 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 696ee73a76e02..cf38a04ed7cfb 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -376,7 +376,8 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p // for tests assert(inMemSorter != null); if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { - logger.info("Spilling data because number of spilledRecords crossed the threshold " + numElementsForSpillThreshold); + logger.info("Spilling data because number of spilledRecords crossed the threshold " + + numElementsForSpillThreshold); spill(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 8a980d4843899..50f5b068b276d 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -27,7 +27,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.memory.MemoryConsumer; @@ -99,8 +98,8 @@ public static UnsafeExternalSorter createWithExistingInMemorySorter( long numElementsForSpillThreshold, UnsafeInMemorySorter inMemorySorter) throws IOException { UnsafeExternalSorter sorter = new UnsafeExternalSorter(taskMemoryManager, blockManager, - serializerManager, taskContext, recordComparator, prefixComparator, initialSize, numElementsForSpillThreshold, - pageSizeBytes, inMemorySorter, false /* ignored */); + serializerManager, taskContext, recordComparator, prefixComparator, initialSize, + numElementsForSpillThreshold, pageSizeBytes, inMemorySorter, false /* ignored */); sorter.spill(Long.MAX_VALUE, sorter); // The external sorter will be used to insert records, in-memory sorter is not needed. sorter.inMemSorter = null; @@ -119,8 +118,8 @@ public static UnsafeExternalSorter create( long numElementsForSpillThreshold, boolean canUseRadixSort) { return new UnsafeExternalSorter(taskMemoryManager, blockManager, serializerManager, - taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, numElementsForSpillThreshold, null, - canUseRadixSort); + taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, + numElementsForSpillThreshold, null, canUseRadixSort); } private UnsafeExternalSorter( @@ -386,7 +385,8 @@ public void insertRecord(Object recordBase, long recordOffset, int length, long assert(inMemSorter != null); if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { - logger.info("Spilling data because number of spilledRecords crossed the threshold " + numElementsForSpillThreshold); + logger.info("Spilling data because number of spilledRecords crossed the threshold " + + numElementsForSpillThreshold); spill(); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 45e5be565c131..c9a1f2293a6c6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -74,8 +74,8 @@ public UnsafeExternalRowSorter( prefixComparator, /* initialSize */ 4096, pageSizeBytes, - SparkEnv.get().conf().getLong("spark.shuffle.spill.numElementsForceSpillThreshold", UnsafeExternalSorter - .DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), + SparkEnv.get().conf().getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), canUseRadixSort ); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java index 3705291e1fe28..cd521c52d1b21 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java @@ -247,8 +247,8 @@ public UnsafeKVExternalSorter destructAndCreateExternalSorter() throws IOExcepti SparkEnv.get().blockManager(), SparkEnv.get().serializerManager(), map.getPageSizeBytes(), - SparkEnv.get().conf().getLong("spark.shuffle.spill.numElementsForceSpillThreshold", UnsafeExternalSorter - .DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), + SparkEnv.get().conf().getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD), map); } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 8432ce9987e31..82ee5b0d77713 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -56,7 +56,8 @@ public UnsafeKVExternalSorter( SerializerManager serializerManager, long pageSizeBytes, long numElementsForSpillThreshold) throws IOException { - this(keySchema, valueSchema, blockManager, serializerManager, pageSizeBytes, numElementsForSpillThreshold, null); + this(keySchema, valueSchema, blockManager, serializerManager, pageSizeBytes, + numElementsForSpillThreshold, null); } public UnsafeKVExternalSorter( From 3ecee573cf7c600a4bc62b32fcce39033deff082 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Sun, 3 Jul 2016 23:23:02 -0700 Subject: [PATCH 0884/1470] [SPARK-16260][ML][EXAMPLE] PySpark ML Example Improvements and Cleanup ## What changes were proposed in this pull request? 1). Remove unused import in Scala example; 2). Move spark session import outside example off; 3). Change parameter setting the same as Scala; 4). Change comment to be consistent; 5). Make sure that Scala and python using the same data set; I did one pass and fixed the above issues. There are missing examples in python, which might be added later. TODO: For some examples, there are comments on how to run examples; But there are many missing. We can add them later. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manually test them Author: wm624@hotmail.com Closes #14021 from wangmiao1981/ann. (cherry picked from commit a539b724c1d407083cb87abfa06d8bf213501057) Signed-off-by: Yanbo Liang --- examples/src/main/python/ml/elementwise_product_example.py | 2 ++ examples/src/main/python/ml/polynomial_expansion_example.py | 2 +- examples/src/main/python/ml/quantile_discretizer_example.py | 2 +- examples/src/main/python/ml/random_forest_classifier_example.py | 2 +- .../src/main/python/ml/simple_text_classification_pipeline.py | 2 +- .../scala/org/apache/spark/examples/ml/DataFrameExample.scala | 1 - .../org/apache/spark/examples/ml/GaussianMixtureExample.scala | 2 +- .../scala/org/apache/spark/examples/ml/NaiveBayesExample.scala | 2 +- 8 files changed, 8 insertions(+), 7 deletions(-) diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py index 598deae886ee1..590053998bccc 100644 --- a/examples/src/main/python/ml/elementwise_product_example.py +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -30,10 +30,12 @@ .getOrCreate() # $example on$ + # Create some vector data; also works for sparse vectors data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] df = spark.createDataFrame(data, ["vector"]) transformer = ElementwiseProduct(scalingVec=Vectors.dense([0.0, 1.0, 2.0]), inputCol="vector", outputCol="transformedVector") + # Batch transform the vectors to create new column: transformer.transform(df).show() # $example off$ diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py index 9475e33218cfd..b46c1ba2f4391 100644 --- a/examples/src/main/python/ml/polynomial_expansion_example.py +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -35,7 +35,7 @@ (Vectors.dense([0.0, 0.0]),), (Vectors.dense([0.6, -1.1]),)], ["features"]) - px = PolynomialExpansion(degree=2, inputCol="features", outputCol="polyFeatures") + px = PolynomialExpansion(degree=3, inputCol="features", outputCol="polyFeatures") polyDF = px.transform(df) for expanded in polyDF.select("polyFeatures").take(3): print(expanded) diff --git a/examples/src/main/python/ml/quantile_discretizer_example.py b/examples/src/main/python/ml/quantile_discretizer_example.py index 5444cacd957f3..6f422f840ad28 100644 --- a/examples/src/main/python/ml/quantile_discretizer_example.py +++ b/examples/src/main/python/ml/quantile_discretizer_example.py @@ -24,7 +24,7 @@ if __name__ == "__main__": - spark = SparkSession.builder.appName("PythonQuantileDiscretizerExample").getOrCreate() + spark = SparkSession.builder.appName("QuantileDiscretizerExample").getOrCreate() # $example on$ data = [(0, 18.0,), (1, 19.0,), (2, 8.0,), (3, 5.0,), (4, 2.2,)] diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index a7fc765318b99..eb9ded9af555e 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -50,7 +50,7 @@ (trainingData, testData) = data.randomSplit([0.7, 0.3]) # Train a RandomForest model. - rf = RandomForestClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures") + rf = RandomForestClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures", numTrees=10) # Chain indexers and forest in a Pipeline pipeline = Pipeline(stages=[labelIndexer, featureIndexer, rf]) 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 886f43c0b08e8..b528b59be9621 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -48,7 +48,7 @@ # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") - hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features") + hashingTF = HashingTF(numFeatures=1000, inputCol=tokenizer.getOutputCol(), outputCol="features") lr = LogisticRegression(maxIter=10, regParam=0.001) pipeline = Pipeline(stages=[tokenizer, hashingTF, lr]) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala index 11faa6192b3fc..38c1c1c1865b0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -20,7 +20,6 @@ package org.apache.spark.examples.ml import java.io.File -import com.google.common.io.Files import scopt.OptionParser import org.apache.spark.examples.mllib.AbstractParams diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala index c484ee55569b9..2c2bf421bc5d3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala @@ -21,8 +21,8 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.clustering.GaussianMixture -import org.apache.spark.sql.SparkSession // $example off$ +import org.apache.spark.sql.SparkSession /** * An example demonstrating Gaussian Mixture Model (GMM). diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala index a59ba182fc208..7089a4bc87aaa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -35,7 +35,7 @@ object NaiveBayesExample { val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") // Split the data into training and test sets (30% held out for testing) - val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3), seed = 1234L) // Train a NaiveBayes model. val model = new NaiveBayes() From ecbb44709bfbaaf3412127dc4569732ade16a6ba Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 4 Jul 2016 09:21:58 +0100 Subject: [PATCH 0885/1470] [MINOR][DOCS] Remove unused images; crush PNGs that could use it for good measure ## What changes were proposed in this pull request? Coincidentally, I discovered that a couple images were unused in `docs/`, and then searched and found more, and then realized some PNGs were pretty big and could be crushed, and before I knew it, had done the same for the ASF site (not committed yet). No functional change at all, just less superfluous image data. ## How was this patch tested? `jekyll serve` Author: Sean Owen Closes #14029 from srowen/RemoveCompressImages. (cherry picked from commit 18fb57f58a04685823408f3a174a8722f155fd4d) Signed-off-by: Sean Owen --- .../spark/ui/static/spark-logo-77x50px-hd.png | Bin 4182 -> 3077 bytes docs/img/cluster-overview.png | Bin 33565 -> 22912 bytes docs/img/edge-cut.png | Bin 12563 -> 0 bytes docs/img/edge_cut_vs_vertex_cut.png | Bin 79745 -> 51015 bytes docs/img/graph_parallel.png | Bin 92288 -> 0 bytes docs/img/graphx_logo.png | Bin 40324 -> 22875 bytes docs/img/graphx_performance_comparison.png | Bin 166343 -> 0 bytes docs/img/ml-Pipeline.png | Bin 74030 -> 38536 bytes docs/img/ml-PipelineModel.png | Bin 76019 -> 39228 bytes docs/img/property_graph.png | Bin 225151 -> 135699 bytes docs/img/spark-logo-hd.png | Bin 16418 -> 11306 bytes docs/img/spark-webui-accumulators.png | Bin 231065 -> 160167 bytes docs/img/streaming-arch.png | Bin 78954 -> 51972 bytes docs/img/streaming-dstream-ops.png | Bin 48429 -> 33495 bytes docs/img/streaming-dstream-window.png | Bin 40938 -> 26622 bytes docs/img/streaming-dstream.png | Bin 26823 -> 17843 bytes docs/img/streaming-flow.png | Bin 31544 -> 20425 bytes docs/img/streaming-kinesis-arch.png | Bin 115277 -> 86336 bytes .../structured-streaming-example-model.png | Bin 125504 -> 79409 bytes docs/img/structured-streaming-late-data.png | Bin 138226 -> 91513 bytes docs/img/structured-streaming-model.png | Bin 66098 -> 37321 bytes ...structured-streaming-stream-as-a-table.png | Bin 82251 -> 47791 bytes docs/img/structured-streaming-window.png | Bin 132875 -> 88102 bytes docs/img/triplet.png | Bin 31489 -> 19255 bytes docs/img/vertex-cut.png | Bin 12246 -> 0 bytes docs/img/vertex_routing_edge_tables.png | Bin 570007 -> 323162 bytes 26 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 docs/img/edge-cut.png delete mode 100644 docs/img/graph_parallel.png delete mode 100644 docs/img/graphx_performance_comparison.png delete mode 100644 docs/img/vertex-cut.png diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png index ffe255063035dc4c61d32e8989109d4cec3e0a9e..cee28916e8db7738b35ba3232456ce6a8d0d6acd 100644 GIT binary patch delta 2866 zcmV-23(fS_AcYu^hJOnONklOafwNF3++bLKcgJKK#Xbfc8O+o@eD1yk0 zK$`?mLi&C8oWq{1`v7WYTC!*6%)NW=`Oo7!|Ns5}f3{DbY=6m?Z26qa_VOWTZ4S9+ z#KNwp-kCh~qAVgAO4rOjx%ENgrEPb4y0)U$hg|DgCg%0eB9o>>gN7BWbaz41x+16& zxF@p6q$!b*`;o0vJq}=-D@ARnbRDXn;wj9clM1v-*UQZUv`)NtL;>2CDo69yy*54X z%yhCX_`b7x#DAcbcX-b2HGtKI3;F`ANE1N9b3e3fD!@8IQzeiv)4LP<<6C|-?71^m z>6#aJ*5|L**7k32)k|o5bLpNp=m*5f*!l!_n(`>o{!LH#pbZ<;Rv)6#NQv@ zwg8&8OMd|BnDC6#nZGW@M$1tbMMLfn2o@_h&u4u&|4P08#`578w8B*Nwph@}1Gnv9 z|7M|`ryq~iO&KvDR^^%wU{*t84k`lQA~~u-fVK%m|D_-r89u++P8Z; z)hdH4XVsc=ii?YRvblOc!*EV6M{h&F3w%D`+3I&`Y3Z4={fvTwg1-8Ag@uLvl{`p( zjuQ$vJfXxJ{+P1mOvaEYln`xTg4Fvpa zEPu&69fm1*G>rtEbM$NvKWh;!s_>X zEp9SdU)%PTS(dpGo1L%Pu;C~De(QH=e}9)?iGF4cK-&axL!yuMJ%UONQksS_Za{k- z>iv}UdcFAyz(BxWh3}))92?38pADd1Y>=d_^2#l8^SB1u4fCDdb;rPSGn=~S8kqJ` z0*^qr6XV=?Qf3AazK_jqL+$eWZKv`b^$l#s(EHO|07e7=ZNMgLl@?%guLc5vXMb$l zTPy(c7D9fH-|v4H0Q^FN_5kXfl9Gb+0o+-#?IDae4|N*GtHXHb$o4-#wg%%~_1O)7 z>ly&G5N7pnWnz3bhg}=%5a}1$=6yK0jDab%Z7IQT5@AMdONP8uX;)t?^w&611(H+I z=2U`*=XQ1aaV_&bjzwLD=Su)_Ab-cfrk~HtJ3eU+e$Vpz&07KiGeDr&2oNs8`%;=V z<)t>x1Tez^;>CE@D0$5LdzkTJ^!KLRbl5!^XCb~u1c;X>6`?o(0{F8;y0yD z3Ey&cCqinF4k9Q^ z+VhNL6^?xBiS{8vvj^Y-Cj-^H1g3L$<+Rf7Z95W;#cVC26je2z| zd734@-vl#z5Kow^`2#%tshme*88vFuMd<$>9tvcCCZ2YQ;<{9Y8XO3C^T~BZMdnrD z(XUE#u2Yg(O25aWk^v-hUKIf=O>tphvzzi=N`%nel};xgCCE!Lkdin*BakiWlV-SwvxxFSNxV;z0JWW;K&g1L*UwIQ{q@P-8Ljk2qII?w>;A-hz3yQOf|}$I*~6Ijz=U;HQ)$ zh1nbqCH*6`!Zv zJ=HzVf98Q?0jK80URNfE_f=BM#%NmDp3cFb&l=5KYo%FF)Y`TrhA@m91;3fctL*z` zkGX4T)rFYgZ#r<28*c9I&YiZ|>Ba)6cWWE0lWhy)Kz~{0eWWFEgG2itW0_JKB;Po==XjG(6LeBu-}U zwP<6`PY}sb)FBULH8otLf_+emp*;zl2a_I*ld-v;=?H#zs5ZafSCj#?CE5mGkf7~S zpwUz_IDe<$26bKu(q1GDf*t1d2F)=`UJf{^5P-r*QT79 zx^iN0o)Em(oDx{P*TP_n>6S%K>0HyCp)Gh47Jqm|TR=G?>Ra_7n2rDozdl9axmyEg zxgu;P-x}w%Gm%eXmZjo=1V72Z^&SSp;I;$tgY#Mmd~A}(;xVFcZ2b1dT&4kcl)>&P zSu zbAJO%mAtt1us#LwC75?zd>ro>2E=M?ytf4cqb^MV?RGuf0H{~$u_0vkYBxB0Xu2l_ zn!Gt(ofyz+o=I(bpSqI4^EV0K^vJ(kL@y(Kg6&RnU^CLmlQ)GW2!0armSte98#TEd ziMJOVS7jLZz8}8_C764;zJ7XN=C4!;-ha6fL|$1j7fuNrPYM(KgA?hC*6!5SFpD3H5@Mzb~N(hH~It7$5$=svd-7dBuM1nsCi)OL}l`tB!{$hO+V{~I9p z+xT~@HA%vsLmE>1WnX4Oj0)(K0A2-t&%}G-i3GG&ybdBL1Os(8BW?X2OlAx%+JE(t zmnTVC^nU^Mk7%C*rxH?^iYGu3gV48vsgsar3T0K%oS80R&avbrJg<|{zEMoLeA(QA2ZVqyZq5uE@ delta 3980 zcmV;74|DK^7}g+=hJO!ONklk97CLFbmD7?Gb_4gOvX$!#!)oJiHVAFqDy5oJ~~(rT0sQ5)#ZaI z3W_h9XLa?xXYcuOZ$Z&C_=uy4^!KOh-gEXjdw*x2z4tk{fqy>q;mZJh)e3x>`16eY z>oz#MT+EE!H*^Kpr|y2`qt7!Af7OP3S&SY~-hp%^Dfsk?3o)M#fisZ(`%)XN>O;>?Da`;rU57CY41oL*u%w zb`s%oDE<>bEbcJMfm}3UPW#Vm`^Tj+l9(rv$`J}m0T@bz_teU{&R1>FmjPG&!2pIS z08)G~cI`p8w>P{zzRuwbx~&4h)i0itSVn}w4pUCFl79v;I>x927=b9PD*8KS;Cd_5 zb_&ug>4+H6z#u5;YczMRU3Tu+Yb#d1#YDplFi4o5iWwq91ko_=1)>N<0RV;w8-3Xr zDCX=GF^yFYj2i|M0McauFm6u!YE@jx1P};u!(@n1v4U>Vm1SRkABg_R5P=9_f&_|L zhu+z-fr3zOKBu zT&^GGFdk9`Vod0=di5c5+J6NAUpvy4oI8HFLwY2DW))w}mzPkFpV`gHcVSSK5Lmfltvi9Z)bALLntG0hFNPFSziF?Ll{xk@EMdZBoC9|a4BrzsG+q_xvht@* z<$tc3O>}OTM1c>O$gKvBJ-QN1{(9@F1WJ7&@{X{^90|1e{YD`^QYin!Jv+t8nrcYAE zP*ojYeMh8ORaphlfdF;|um+He08&+FB7jA{@1F!<46`{|HLw6+j2TTtlReM-A)qG& zLGS)7CVoI}nbSdhBF>Wb)s+@D8$H=E5)*pZV=xMS&4zUaeohd}^~V)kpjoJ+SMEtyQZ zt5&U?!)$&A;DFG0g@`~^L4QO!Kp$ypYC5PYVgt1Es_5Cn~phsH&4ljFqhSH&}n7A@Ko%BvD# zKO!>Q8vN2ziQC$z3DM!*R<0+)OAwiT)vLQ)dj87;&-N8JSZlZIX@8pu5xv$IEeE2g zGo0L1t~;zK~oM6|DJ7`GkAK@bGj6$-^Gfz5%vnqe4G zfNyDO;hLndRlk;&7E{}nOeXt9)fG z>KlRY99CPJY(-ioli805ONl6|b8x?iT-@E=y`iqIZaNWe0nnw2YeeMcYNq>Zh391+ zW#;1ng^08j3Z-;)K|L@1J!12-L9T!N^&>XkY77sr1hN{0g@3C0CWFosAXe`%HU)C! zggNcoEp88E<^duCW`>B|T`HAUMs2rxp63AQC=?3M^rmDgl^X0gPKt;QR@DJS`jIL< z=elmGx8%_ilO|2t!*$&hfCP~?sM1;@TI@JZzO}WrYkP8uL?X7w9=nbR0^Y@S0afR{ z`Sa($17JuLQhxyOiY))BBZ9F+8dqgl z>tE5_-25Nw)^!d6!UH#vE)NOwrV`;+0FJ7{SM4;U|KkfLD0DazBM5DWD#8;5sT14S zy_uVy+j9gFmICMp0L%Xr?GQ7?(UVeNRgj7 zj=98ITcd=1u&b;4jB22+={uI0o*<$jRiy!a4u8NxRb?Xbh-jRMMoC5AC*s5N`TUB| zOR@(5`Xmtzuga_HVgQSX;H&DPM4Z~P#ju7sNX?($_F6KTY^bY?EhgfsY&U@S0H}zb zz{~@QFh7#Ds&XAb=QGpbu)F>!pD+GfZ;qe}dgUC&&%hF|Jc~LkEe>uh)5%k+s#_O0083g_`y}e zh^Pyo69F3Qy0NjPQmGxl=o*hDfPO?ogMU@gttwY+kFe0wp@};>ULXBm%jWka4Cv)V zP%4$msZ{Dr$Ds$9`5PhvYX!!blSS-00OtXa?(XiBjp4M2fCy_vFtu1LK3_Xl4biUys-&pmt)48aVm|;;5~8P6^^cJQ!~p(zI-MT#g_8yVZEbC< zgTT&E#k$Bi5YgV9ot^sv*sE$Z5k9HNJ*wt+d-QkL%}Ov=K7sVztE2;nZhszh|7dP7 zV|i`=TrSriK;stubg3#XP}R<8oI%VyCz(w4`-@1+WHLugp6uP*a|PPk+Li+PHW5Xh zLaQA|2NO|!)esRJy}b!CnamU-K7PyD)jkm2P;A&jihSb}^xS`ov+MG3S{fR~NC3PQ zX@;mYo0v0$zw*NMxWIHyphZ08bow2Ak6MW!B*Roh#CRv7k}yp7!ww@jg5`J zRFy}g(+-T`i#*Sp`dLYHRGp#5@JZHM1VI2IJ|zgc2YX(+s7ePh)3;UC2@6E~k}B>a zGM83YJ|e9F&_8&d_pquiV1}zoIaU285jYCM%dM3Qs{va?Zz0lC0JucdPZjq@@_A;s z044)CRu#>vx^H*_B7YF+8@}%kC8B71(?%j$)x)m+n3+TbA|lMX0tmBlHk60Yh?*2=U63l_Xtow!sg6+O?J&40{4j^qt9pBUC#Aw;B1 zq-y})CwdmZEEkdCj$=5g4?nWj{cx-8!#sRTIMHqgQ>9Yre(t&Fw)r16Zf$K1jIrm7$U zfPJF&F4uKeGGh!gS9SfMP$+EJYF5=TL{!b3Yh2ge>VrzLSp2n!z2BIyj&mGmN<%}# zbpr&cVn^G$lg3{b9haqQUe^vnGRqIpnrQu8SxcWdZM$l^U3NxSUR1q zXQrh@G%_ML4PXU;6NvOs0J{S8F%e!=rKj`x{BnE(JkM(+qT>KL48TADovO6NTKl-~ z`)gvc*inuH1OdvWQt1iL^A2_#8f`5gBJUN8#Zr&)ngB!>n)c)95_Bs}nlx#Tc-%QE zJS`XiHh*xO_`FJ`(oaNO00SJyd9U{VZ5z@)EmEn}KCWxp0cudyvMR=XmU!@Wz-Rs% zWsDi38rE1u_{duO!B=g>myOR%8nfl>rYja@05UHq_FTvUYRfv$S!uvgPx4aR+yxp~?75fkPKtZ%amh7iU*5 zDSuh!e^*F>XginB34xf>cQO488PD)Sd@&9fI zpJbUGyuIC}`1$?({P_HY`P@A1`2{5^|2#*@ z*2~({(cRn8&6V-?xt3OLKHjp-%(n&o-=F{L>FsFu|1QbZ>wm8WUXcIx8-77P0sjB{ zY;dd0?O7=uPe)r&^6m5G1ZDnR`Tue5f91&V-xmJ=sLX$F`tK}wRXKba{{P!-a`>8^ z97bqp2E}Sh3i|$Y`?*-D&s3)BZfb=(Old$9;NI=Xwdlx&AAlcBKO&L^({QY?NO{Kl zxB^Lbw#0RflbSPxxaQ7r1(t7W}AHXFGAc{)sk7c;(y=#rc_O4RSSdN ze#V1>!(>(@_*F}WbKL&<|M3L{&jo}_UjvcCsz!#wgmma6vsxbNT=l&OQOzsAL4EdH zg@OC^W|(1?ONVVY*KTfCn_tb@h*JZ6Wt8%YKU$!jbAVzfqTnhUhGmA81fKbjTOwhbqoki*KJ` zw}o7vz{X&Z=#U2a^Hx$6s61xdlW(1o1UJaFL7HRkP8#e276Z5fe~LiZl( zZ+C>_=;XwLy7r-@36g(T=Hu|Fq+9sU?s)WGqisI8W#H<+wf|=Bw>>GKFmO z~cRZQ(dHK4}bp6UJYpwi@lkvTW$kanKGOL-LY0g!hHqg~msX4C`GN)!#pC+5KkK8{g=;F*uG1 zk$du*VFY^HYm-eK#B5qwHGYRncGo2^=tUpoTKA*+`z+CybN9&D)J2?UDw4v?zvzEv zR*G@opA+pE!-6gcpDhwYP>i>Y(L#)(5p`19{*Y;2>-4NQ{=T@&90m7tp0%xJpIyqj z=~!}(42b7?KX^7iLFL^JroiGPF^-3;pubxK&kiz$?R`U+YaD;5r||@=^%8By4!KQ# zGtClltg;)KD^wxQsieq*gtRKfk}0`}xXjgwQ%Bh^n*m>pa{TO3#(@ zS#20dW-HY#jFL>6m^Kd?$!}?HK3*TFw(h&jfkK3MKH)10IzN{7JD|92$xd1nCBGT1 zT#)}%z}jLjw#J8_eGDN%7pIQupyf$e9uxYa3vfo5Mc<_}u5loJ^4YwtRgf~DC5@Sh zx>vpi@j%kk=5%|4m{y`)Ca^zKn4@C0$?tG;B)j9;SL0evh(E#tbSk(0CwYyPg+{O7 z&T{iMDDsS1B|*k#N1qC*ySN4V-}JjTt?P1Y@UJh=;>g(EIDTE{VCR|cyq25?Rdheb zhJsmWRjZc5PAB`ydwV=T=c}aRzkx>Bk@!Jj+xC%u1+30(`MS61!R#m9!VJfS2Elyh zV$AO|Uj;H|b|yb+U_+mbOf_Re<+-6KB(-m``|rt*cx0@MoXe=9RlCwsgK~5ZyYJq= zS9crlBL#&6j%{Lm&JIjDP*dMbAMlw-Wbuv+ns8}GE)QFuNSYRQ|8b|lXrq<%1f#2) z&uO`}MJMj*?fk~yKq!8GTGFfDWgmm5s-=$S-#*)~`G$fk=EepxV{aJhOa8?;Yn77E z!~qTlq*N0wGV@a==#cxb<)89XG<0QD@IKh>;3Wc&n9}M~a$^Q2Ab%_b~!baP&#Sf?)~2;Fg_B0h0ItQ(Jyi2_qKQuU4=s{0`{h(tZ+KXe{01P^tOB4 z&q;_MC2eI0T5ICHP>5sK%7WvHvSD_P!|la8R`8VN(?gfRK+i!4yFjh`60i*9p-J>F39yTZloM`?rdN44Ly&r3L-Y>g4GI7@C> zEanZQaBvXz@u9bi;13?(HoO-i@N}Ehxt+lwdwa?M48NqvsM>CmgI=3vCXL6aa|xlh zI)f?Tc1`GibPq}^RbHRL#7SYmO3GPwgFLd17yH}(yM-JjAYW}iHhO^#)xW?aXD`B# zi*2o-`+c{cxzljTKl-a@9nx_C8n!X`*%&zyM9ps|W2Jy6&h{zVka~{g-%2ci14(7m zpnP_~0)s+_mB2!>LmRDP<97pNW6-8us4f;g_B6ug$ zo-UkEJw{M4n?!nW6WWnj?Wwle`P?w&S?P082$(k7sa#tFu7Y~|9;6fb@H9^g8IRTO z29LF@mv_jJl1H;yTHk&5Y1%Ev|J7q0yX|#c8MA)vVT+ReE$G05|j{pCDi6H+MK|q}Oe}=c)o4nU< z!`=V4XFcLZy}5T)-;19wsYXUnJ>l~-(C55vqs|$zDt={k*)lCYX}{E= zdWe&`XVZK+cKCrcZk0|;6iwRoz9Z8-8LtjIJA`@&Fxu2@HJnWaryJyr^f}GHRk6L% zq**#VQ<)yVc)R<2x$RuzdFhLm)o=|JbS$X2AFrX%m-(!L;I`$q)0TW6zr$4)?p}i1 zcA&>=lgph$3jzt(4*>o2N<)p(xZ=tmxJJ6h^51w9dbg+DVBr7X9ModG>1S6p?N1 z8|gbj)1Ge?d29r+S?_-bXCTeD>tu@6Ma~l&xEn+D7!$`7F)^Vt0g_cVHMqs+7H|H^$bhq!hwzuo$-um3v zYV0+htD}B=4pbh@tln20BGy>tk`VXQdp6@BHKf|7?soXD`yx&ta4a15n1bMgM27N{ zlV4XusG?_wahovd68q)cuLgyB=XyQYl+fRLC8v0hzvi_;j!k#_-rQ%Asw)g-_?pOY zw~fkv3gc%_;rEN$&NhCR&w9)ur=Q+IcyqWFzp&=KNZhO)E{Myvvl8k}ApNS;>g=!1 z@yFE`u41_>ePJ}fpNg=XZPQ5Iw*Nxyx=^o3IW81L)F8Z(aWU!(3f1Qieh+(W# z^a`BC0=i-Tn4L2|t&&yNeP6JKPxlunD`t;#Us75*;H_X&o3-=%eC1(K@c3paP%HH<)HFvr*g_ z#5(We&*O}jhS5mgldaJNmQ+vUJzTpE5gS<7r7spWMxeIXH^^j)}^QWclvSa5q@;VD4|Dra}E!>;vwLiiEfS)?o3l zWtG<0Cpw+atoq(K&fQeZIE)RYNn$YNB4YCh%TgpPgN7T(l<{W{NKV+_##0ME%9#RBvs77Jkmo@&WqY@L!N4cMWTKqdc^?oLI z)J5{`R{a=`dOJbd?A9~&?-Xe_tUANnx@Nia5_N$m{qLB0XlAk6*fzSG+XV`&@A5N@ z4d=?n?X$nIBNB7fv9BqKT&}rxail{1o_VMos18O67pha=YWG*l#cM%k0T;Zh1+L#q zW$Y(^9#W2_i{}1udCr#yQ?5CYm2HgT%-t8TG%*O!{D#`Tc=K#YB_-IBj0wQla0*h;M# z%9s_eMd0?AH}B@1tq#+CUkrBSUp2b?C0SLT7ym@~^yo7_h$C3<(b|8M=o~J$h3wDO zf+GeAcZ-=dHp_n$ zS!Fwv_VC#k08@J2+~d|OQU&MmDL9kF-IhT5zcu(iu5Ea03 zZrHr4z0F?(VwGZ-OknWSSYob|z7kfGW zK><`wK^LTjAb66AI{i$P(TWi5efvNhl!~~Qiar63{q|-&`jeQ1?S>6&o$fWu_#G^? z1fG3Dg2nf<-p8uIs6qX#uzz{UWz{~o9*3B|=o?JLkxYyn+Xz74+m_(eEdUs|ja<7x zC2{n*VT&1Mi@69O0VEa&kVeqD*CJ^oOO$}%_Yi-Z3BWsPye}Ac zznM0@9s$iQaG#h?D&oB_3VD>mp<|_VoF1TlmqBj$^8x5A0KdzI*A2(r<=@-0Xie+(9RLI9|d`+ zxZ`9IK)?&%-g6sxb}k2;231@bAlpT<73ll$N(4YM` zgG?B7U)LHy$LNj@rlrq$hZW?|E$GEpuHrSCYpM9n{>#b2p43>Iw!HEIBjdJ10iXj= zvD$9rjp!z1Uawel03yjwGdgV96*U5bz+?f&**sWmx^BAcOa*m`ydl@Ldg&P#Ma(ej zCl7`|93}6dy;3TtF3RKEamO78tba6#{HM6sMaG;6V`>#Px!wpo@(V?YhYn@3&TsI%-2gz8if*Om!X$h|C;5MxTV)??ySKWJJyF}kRtPEyhO%I zFKV2Az40yoRcc`Ud7Sp+1L~g$7Wv){qwyWJLik!=;=a>=BPitJ`tlGAFtrBE4w`NU zr8vqFgHbRs5+v@TtKpt9qD4fBTix|JdxQxMzBm;eV^avjlzFF-&UbF|U+EVxWC?Ss z$KSnKOe4hD=1X!q((_a2my6#;m5N4w&%3o@MIyGv-8JP7UWsRzsE<{uOR% zjN+fT3f9r%^WZ&Anla8=03NV+T}L z5hLkonC~eG9g(DYz|zqj|D%6Cb46k&XZ?46u|!q-J;xD_*C*&+y>O)07I; z<()iAU`_G<{I_8g{=d}!)Vw9=ob2STy&WKO0G;!DbznH1pP{ir=?FE<=wHJSm`WU1`RkSr$uX+<1Le~39cN7-60d4`A%|Q4ccn8KYJ9(fQ zV^fN6J29s3!9r&0xc8&!xBpRgh!mL@kgHSj%q}!~Q_`IFI1(oGM%n?AUVrokS)W1x zb8eqAMybd!9+Fd$x;I-b9onbrQ?o<*eZXJp&2TC@9W-xJJmKW@^dPv^EDX*OseJS> z#Wb>V*?%b4#oy4M&|YzL8^B_&^fwemh?CSZ&YY4G4&S3cWS9^x!rx#ij92gS7a8T$ zf*X90ztiiJ^3uz2OKb zmicJ)3W;EPY*y>^E2r!2WW2#g^?OnIjXt|+JFfq(djPFtze=pXPFi@eDcA@>mlW)+ zd?4cRBu^$F=dl@JAE;k#wufRgcYmp318+=l2Pw(@BwV%=f2LL~fv+*!pHGG^?;Qd9 z;z*_1^)@B1u|$fBa7+MLZtX|1l%-)`9B+$Rz=H0@v9_J&UMP76wm`3F+6^(kYzXH@s!-p-lTc~*1nMkMm_Tr{j)wKY?ofB;8>mxG6GCeb5O*u!( z=OJqy3x_}#w+O(|^Yw1Kf@=Bqpj+bFkndJuW50L)N*-dq_IgmfJYUbM1oyk#Rc?}mFR3v5VrtMPRUa);8;mV1^*9X zC|T@WrS-Gt5ehso{&VtO*JVZ}_;g_auQ~t9TH?)-w*6$S+FJl50>x1N)`)yODE)2N ze7BlelujKYa&0tbj}T6T5)j6pt@aSeQgH3HAErOu&C4_Cs(Gcya7{W|mhA*>m8f2N z3O_6NdN3j>?15MKU91v<1sV9d(l>*Z$N*t)#~iay>OB1Yqr_`#l-I1eAteA%3-?Zn z1PCC~F1ex^H!-`?o05$UFRx=H6{Var#Nd4_+|K7?eFo(@ zVI$#fBZ(RrKbziHV{UDAlA5L^?grS--_SO;nJzS2A5_eSO9zR%&7Az$4eCsMxlYzP z-!gszKdW~eUfx+BNa4QuX<#1gbF!)ByQ}kc(21Ha^zv`Hcg!8v1bSJT?Py$&BYF^U z(*;sMjL5WWlaFR-0uewosalZ7DvuR;P7Kj(FT!iiP~RM*=dwa(Ocahwsh)4}R%hs> zGYf8zvP6{Y7AFy)w9E1+WS!OzMx3Hl+vCWg|~7mWhXOgds^8vbMD2_YL75pshHc+-tHhs zWbCn=5beR2v-Thz80w4ITtw&=|SGGm^wgV?R zTPC0?bL&8WLVjE61r9Wv1W+<#`O=J9=K!u}GiqWd@%(dakW=H#Y9wX zXIkx0q=f+v2kJW5aVVwG4ex9>a)i2!=wZ4li8v*M-z9Ir31-06=yi=S*b13-ROP9t*gJ9w=D(85m55L zrABmw7*l9pj^;@EM!3qI^#^MzPAAYw19o?p7!}`&TqV$yx6{s*s0WSxAk2cbvyo;9&sL;0Oa}LxQu`JA`VpP?WAyrq_f$@>F>rg5SE|6FI&%B zHUCv7f2m)j`tFk@2=O9*zsmKQJx~%kW~1?+*ZN{-8;l(>l3lf4S}aw17Wh|2`#57- z6QVc=Ql*TlBFv()nC`YsZ;AhLG35vzWHototaKFfvS+}Naso4^KIJ`&%A0mu8g3F~ zriMkZKR@zH*?+h}`sc(tGKK*n2eWHt*# ze2PoMYb-J>@thG)=&>5esMXRVE060DEcmzC-tg(puhLh4c7m8zf2@hPE;NK1P#!vf z%nS%DfDh(*ncpd+9V@E#yt6LIAWVEv(8dFA6+50@_5w5}AT>vyCl!K3IU=HB0OWv% zL~z@)=Pj@P0HH?t<^2F2k6tl{@z;lFbv}1o5M7CI_SOdt!oKS=a$p%TVbkRbd0NMlG`Tisxw>0m!U*3CS$J3gK)jciz;ww}-z8kFz6tk!uqfEXU3X*TRtJ9};X?&*m=g!lD$PQcyY$&`4 zh*Eq9Py$HrpybL|FreUg`Ow3kpHo5^9_aOC3+fK2Ci9!irFg|jqh;Qvva%n;#ZK84 zy7ow_6N8PBmC<>!fu76fvp@uzaSVSYWIGr!Vgv7^SKW3V1Zg`h&}h8sX0lKrWU=IW z2(awX`k~on4~B8*HG0|-8WF|w1w{C7d3-9;%AVpl_s{%n^XSXS*y(-7ydON@YXk3m z)8=8_oS!N+thKrG^!jq+D`qm4m;$$Wh|sZ4;nCA$?DOZLZw}18Bwc0P%>!FU+X-kHg)0fpCG?n71PLEhS_%KTa%W zJ?G5A$q52HoQ6U4&PM_=cA9oqUj>}DOpG>M20oK9%34mYWNzX80Pn!QiY~9XF=2z^ zpGapxy_N4B&DEt{3;A@j_olwW&&;)8a&)+rAs7&8fqHUi&Bw~SluUbfn0jf3fI3EL zdEEAfT^z1r5@o&x|FQ3gg`zg>8~#z;lcmiM>sW~*dQh>Iv;FF_T(9O+q&ZOXTqKDx zraA8_YS-!(J_Zx6646y$Q)+2!={yUIC1b~EHaoNp$qQj1g_Kp`mG1O+sM?_taHxz;Y@s=dt~fl|wkC3kQn5q4;a zKp4s(o7+zmoR*6V#nFFr+jFWCPhv&j(&&^l*rniBxI4qMl*8h&>2QBp~qU-0HvGbO9Gl;QH zIo<f)6F^FIbc{3(nMnrzkF@i`1t>kJz zv?5FRjJoBj!r0_*ER%%YFm-|Hr)@1fI>MYQbklCRbN*wWcfX-qJWS{*H{U>N<6(8` z@5Q(J_y<-}DPuO(86ENPgF{#9(Yu#Hfn@b~I})6xqBEF^N!`{pp9B*cPslW>a9I+N zp{7?R@Mue|ZW3y-TI^pqvQ#ehs%oD0q&J%U^XORkh79a?>9031l1)RUy6s)qf$qbE z);7*m@eUFK-Q{TZKVp7PA^B)=i;xHfs1O})uFcP1MJrx9_7I*HNhVWJbaE(hrd>G{ zvhaUn`H_Cs!gi}6E#u&cv)}m$RCK0`s3lUA&i7xxH%4=_uOAe5V33k)pmE%he&wEg zNkBXvL(1a7Si#^@RMGBdh0%9w-arrtdMNfMDJfn6Mr>hlv(ODY_bB(st>#3qPSO3Y z9k<#cQ;Pm4py`SU8?AQ79*u?g*WKO+2EkYV$y2@d`!6dE>YR-xt1vG<-A$(0F2Ku4 zcvW;(9bmX!s%>RvFaP?V!Zm5$okB0^$+~z?vqxc%JC|?!DT9(CjhY^s`OxdK@U`wt zeNMo?DOqLjMJa4mlipfO#;TROi3XZnB+!DXsk?@2;xHHep~G2;~gbDHrwq7YTn-IT^{*sca1Lb>*)ZklY$6J@_dDrWy zOb-%9wv;GxAh|)ZRM0w_|H09l^QSUZKr@vjR}(I#UUstKuL|hjrckPiOnRF@`R~}33k57dQm4cZY zTl||oQHr_DVXg#OB{nw{U7eh(aolnkMfJ>qk)L@Alwu#{h_}3YQR`%Dz0yCsPc}yO zX2srFuD_(d^Qwo$)J9{u&L!J?txu}2jshiu2L;_de&Wk$r1VgGO>UK+DR*?jy9*cP zUie$|1os}FNrpDTX#MmS{5LVZ49V>c$)(h!osv0aUxk26#TU?8XG>qVIuobktAD-R zBrSNKC8!Aq(!L4~!p$CBBf>-FEUc}eJz9;iaJWI)HpkEEEOgt484&Y46G?Pi0~x9g z{?tz(1(s_|kBzx(UwkEk!Yv$sxygmF4&pn>(p^Nr{extb6J3OoNnS#TZkbx1~(g(d?Y}PdhAb!vv`tPx-{l#PEy0Y*6 zEF8pz(zDQ&m%ga~2IpwGq5-Vs ztO_)4?@5q4kUX=y68mj>HQA%%;2h`16UuLCenknH0CcFj+zXKW4@Djuc**lsH0{Wi z?AIroqnXiM1_=wAl3weB?_>=UoFy}>>_^_ozRX8IUn*ZQK%|5bK>?HZm7`PV+aC+D z%die)HA0woyn_hRgVUtgUk>NE9i=ZagF7rxl3nHKX21W9T5wj+;*C=L=Nn)Q())9; zD7_8uD@6i70U1B;?XU(Hd^@DRAwa1lDSwKo-~M90tpoVw7w5+tEJWyC37a6A#N4%H zV35-p)_Sxtz^w6&@INN0UL`NS5+M=^gHEzz@xd(+>32=542sJyXX zT0J_#wmLC7&ZN0r%l?&R?qYY|*M?o9_4ZsHpJqi*3?U6g`H#~@ayZ~*5K5f?TQ5^O z@li`iYT?XrR=p?q_(sTFNQfNu-Lr4}|E-dGh7mC2ts&s{x=3ZDn!>|ZdNjz1 zl`$fn|5H!kWcXjJ0$)PdT`2#d8SPU2%eU&baa5zrtpQyv57M!inX<$`1@D!Lw(?zU zjR+5IX*&P1;`@gS{{hIHt?b}$G*r%F9T{0gxNe1 zM?KGht$`H#Yt*?(tPFFA}*ItPwGpmZAVGQxV2ko-oL7GdnX-8+1s2UQxIa&k(fW(mLuC?f3730 zZ99;`2TThgV^tM!oMffKgyN#ub@GOXev}zi*S?(o2E1LsQMV8=_4P&eM>h4;?TNy^ zz?_H6B+U>m%bZ3v;r2df16UQBI1A=&>4Ye@YQ&O$L&Y47Qphr>4G4L zLp+!$fAN#^9gr7uc+U@pTXiwMLt1^q&hfe~`10g@hd<=`vqQ>FeYUb9eQUEz{-VV{ zI`^!^9s0c@-lutP|BRbCdr!V{+b_cWGLq^{7dhI{l$P|y{@cr7=Dc$5p{?}?LXEviRtSIQTou0a!D}|$D z$Z4uO_6>`;PD0t+9sM?s)tBnf5bv+1)!5;$d~4YG?HhnCe<=VBTih7|UEiA<7-Yl{&aXMGy4zb2j+@!>Gk?UJr z(oTkjs<7ije!2u&NxiRw8_!3x3FXMyVpIb2XEFPzAS?LuoewY|uW<(SKB_utYKp~@ z%4R|AYXcXp=kPR@gN=~t(mr9|{H5=K=vKv?Z^#Pa~IO=^uuPwiG9jS6@Ia;*k& z8NomlxCfJoE>4O9V$@%^l|Cka7YUKYeB{s|kpDh514XuMV03_Bm`iZXgUg-~U1G7q zvP*GkFi%!`_sM9uom|K@3TTVWiE_Yb1D37om^lus?wEy8p!S9<@&y6Fnd-OmGxk?9 zhmQ6TCm!{`8Rm%;G?i;^F&QX8&3pMdM0EKJZKj4U1EVPFbjs)?s=0znQ3>_P~K;E<^7kpu1B7{){1ld7%%;KCM?asD#z?)}+yDVEw zEBp?I4Sn`S-vi1F=Y{7Gp){8)8)ZMrQ<*z%nrDkYN1WJpV$aoC(dVf(f8a`H6Li&t zM{n3ncM#BU=_&0=ZVGN99*@#MQniGNHHc1lP{oP&_ae+EyWNfi9Q`3LW&9EV z^kFYQFt-bCx6_zmVpEEyK^jq)zYxiVZ3gk61o*hR$GfFOo?b7^3cBpucIrOQ5b|ut zsXM>NmjN4m=OmcgF*XxI&3t%@Z9kmxj$H({VHZ^Xx}DhTt*1ojuG>t7Wjeq4-`UOR z{T`X6(XnfayOYUm8Z<~EU|=m5gJ3}7dAm5`mCf{>6*y2N+rY)(3TR6S@eb2rviOwlg&1?p z=D~S*F^LY?_$!Dw_750-OX(AuO?D$4&)LntgiR}{0Of*fjcH!>$6 znv!V$XQ?%JH6FJ$ZdGS|WjF>K94nput<=wEIj%5n9*KF>64y!+-Rck*ISRdWscS!0 zYfUNZ86fQ$Ga-l?hR2yWo`tkRy!JBaubv}`A=VPB z{2xt;xDjvtauOmLnhI(U-GAPsDeFKZB`Px8ZbkR94`!D!U{?R75m4SX1Fu8ZG2 zu1CrHvqVcn8T`W8yL3bD{ltN4w*tEboN67x{jA0USn+b$8i-m}dwctZ5gK_Sm8MG0 zxOdg;VwgM9n3Z}`yC4T;=7_?P#2MYW^}#ewD}1-yC5-@J&&%n<3fzlO6kuTUw7IyE zY!Vz#aB$p$pehKsXs1fz7E#*?zr<|pEE4ZlTK7a$WmFg>?h}?d==^BuzzXdkZ1YPd zKsnb(pWXEbVS(WCe5bTLCz7$_6&Rzy&=B2dHyw(P4RzL#&fzUFefx==3Mm_e9A}d_ z0Tu-joQ=&ji9Za`TgYJeNvW2^WG#Y$YShuNo&%EWs$;$~-RNaM_t#mHlp7mLK*iS$ zt=&(k5t{V1fE$1yW&CH4-CIg58Vx#XL3^7+gmoRS70YK4;M7eLWw6eu21AuMCe|#_W{sDHVuwkY|l93Hu1(T>A*TU;~YczN5k*2dfbp-5K z3n$xO+__!Doqif)4)t1Ln0n$XsXTe^$N)MD_fLdu_p>mfRSqo)PQik9;(Ct1v_~v! zQK!)P+#zXbH$PG^6I*${B&kM7siT5vxd+AeoZRh;QG`lr0=`=~fAeaEqLdi5`Z)&V z2VegnBPx1GrPASipwV*Wom`Zb!+v09k>LE~0&*pHL8~5zZ((G` zzDc~H+3}DFB39HmBo_8KfwmC{ExtGHuVI@cz=?AmT^~al>68=8Ric6BERapbrPqzw z=Dz&U;2R;;?yu55og_7Wbl*`a)^|)iKlUKXZz$w@rpecYfIq}VCz&3Xl5 zzqE>H@W~1EQ37a>T_eB(E~5sI{#(kH;Hp#Fsj1Fn_|q z7-%MJ5b=1EHDW2lYRBCkOXRfb@ZImwm01};@}PwEvuFyN3t%YN|8!R&mDmrkicO82FRImu(_dLkG{y&=C`noyzpcdV;E1(&bUEijlme5Rz^1 zj#!%<-`?fgUq*c-audv)$+6!)PBFD?GZqOGm$Gv|uCtd_@ynYpI z=Tzb$tB+g;^3i5k3vo2;S2zyw>(U#lAtvbg29-J98@C_b{YVy9_Fmf$%arljPWXYF zcO)_8*j|c?y4%ocr7G$538C3oluY!NJV zan3>__D|LV5|*0Cst;Go!ac1 zbirvg=hk3!TLnM=K5G2IkD+f^;i*YieQ8PBp*(l)KsY}W;&u?Fo^Zs{c{Ce*Gx4AV z5Ud`!2oAqj)@(!8Q znB=~8$5``Q!}g5d|3T-l<;!O4Q|fB2Z2*}R#iWxz2+^c+tuXAkEOkRtHsO=3qKjaQ zmCGpOD&c+l2*oD^AcdX9f>Kodj1P4V`U3oaO#5H7wz~)$@kW@3pTH0-*1V0o)T2N6 zksS{bhVFltQ2mUldL{*KTWs?4V#qk=W`zMmYp(JBVNX60%CMU&ARyq~ANI%b)T~jpk|+No3U0T| z@d!VBDV%`A!Y-7IElk+lBSAk*NH3GYM7Hc-j}r8Zi52OX>tZZqN(vFjfK+7)Pp09e z2qLT_NQ4f5mYXkAR58J8*4!;kR!ug#Uo(tbJ$bAt2z#6n%3z@2UPu=?(V6J3(o48P zs3l3s`gjKpuXCZ?TIUujBGmNN!g08L;m$WpkbXm5qd@SZG_2BdBoxIRxz+CS!!~6Q5gPk%s27fmR>7-KE zz0PhZ3|Bt@@&Xg;JQ`oVWe4vgwC`!m^ibwON#JACQaMc9i;y8l2p9Gjo9cPRgU5)# zrm1$1b)UdwG31}iik3aTMD0Q5)UOb5$bhm)?41B4U|vaf1RoEu5teFLv;&}~dcmC}}* zI?4?vbUvkuo{NIC8{)E~|9i4{AW6=pnDDo-{a}WuM)96e^?HAvs6oPBfagYkItSG* zrj5aZQ-_Wx4w8vxpyDCoi!Ld-(;bX5qLn@RErr9Gi9(e|d`;OB7o59*)QG!d51qV8 zJc`JVIgcH|QKr_OG(cv)rut4CmIKUnf{BhX%aie$@xbIHPr$q%b|-rI!C}IV2+3rI zJ-pAcnm511j?2DNP78ycVhnh}=u}d4-+i4=iJ?b0p2hmr0}pDbPuJ(j;cWLmJHN&W zA-~;z6{)2pq9||NLXY3zf?X@^%N%&4BaBC(k; z{aVZbyP-4}V5F_VMQ0+XSpyylhRHgX$$WGPpCd}r4AAL8Yjpef-I4jgy;{}!_V0RZ zMMzLxs`dUvnFF(tz__C~Yfx%Gl@!+_k0&m0oMcI&IUzyX^Pm;y&$3_=@+bAg^=}oK zzgGsvUyNN@5>%1JYX)9JY9eUMn3CA518Fh80~335DR1DX0YFxoJoe~D)Up$^)gczu zy(1L)vl%VQc;*{6qU=Xv$YZGMOhpIkmLYD~nJC10m)SepAHtV?`Da3&1IPa5>VrcB z?FZXSx&mN`QTPF@x4@M+Pg5K<2uzyML1kmVNt3(oWh$eJp5kCLv|PF{D1iw{U_;t5 ztWq$R6?P~}Zpw=UJUsgpdI!-{0qVQ~GuKTh_7N}vYqqnFXw)V7vKyBmIFCI+Y$OoUt?C!?!iP@I zPaQi<6wIpMxshHm0~l5iG9aV$%<9KgBp1@LOcZeL<$_0VPwxeuAF~m=0&4H?`{Is) zZ)$N=xs&~C8}a+hn6SLI-DE+YhjTx2}FybE?;4UNgy*0CbdfIiaEPmFY(n{I5ffx68)?-Fngb2}>hvcXw z9Owlrv3F*E=N^*lIdBj&9naLIBNRr`ecJslq2V-5cqoY%qnWhlHLRA;+?kqx)bIv# zH@@|!m7$;~@*zr$jot_v>2KUfw(le9DIkFqPB{}9RGGnLX%@bp-@`M}}}k6I)Es$yVg;-XPV zs)yRZ{>e(gM%T@~{vmnt4*{XuEDH99DXathnOjyuT4);fU)Mh8k>9)K!29>=<)`sz*d7N?`PXmWG ztBz!ll`K!$bYaIkU0%)vLhr~A*wXa{Kz}O}mN<4pK&Y(>YN~ zXgN?ocpWFw;M>!rBRXG7m)mAdIKx`{f zMRY88NSL{5EXAFowt;H{Nt`AGE7B;0f>#YeT!N!gkoTU7b4E!Hmq+<2RT4Dz`)f-v z(;VHPbOdDVfjpb7vDE@(y7fe>8nqU@wjO#`p8VpCN*BheU}F+L$+Adm-TkTeHf)%Y zM)+WRK?@jIgp#NAE93exe2VvRp&jgBSQe)Fem`(qXjlnAROjjJCU#b||6vNzN5lzs zft1tf606R2G0T-~ilL{vL=WCYdn`F3cNPsJxFJMm^P38}*8gBVwiO3lo_NR4gti-n z`7v3NQ;y=sAHcGo5NG`=)`9UMBV$wrcOn=jH0ghX-L;9UXvZp%ANteHi#6}RAV@N% zQu*ZZ_{ahISa7}Cd?W9e_#`B`@1g$^$&N&Ci?s}+eU`k56QQ~L-#w!y8ik*}pYN+G zAPDSM5~_O5ld^&p?n2HB7oKqFX& zWyw0Qh&fDcN_P_i2=Xg6LG@Mn4t}NhwhutcqJ+wwb2MfJB^&QXr(ZBexd_+W-L~jL$ZTwJ{%< zy|&V?`abzY@F0oXtdt>K#zjxI^T>M6)YFC31T?u)*j3-LLO<+1^a%40gPn>q!y~an zvY)_2@`0&2gvf~!0QjUvY5gKfh}exkiBOb#(@nJnChap-4|)q5D$J<7nU;yxc~0Y^}^ZH z3Cb7O*{_5E4A}r!ar@ZJ>np_0QSQrJRqGL0o9zBEd9~ZxWWCyVVi(hMKgv{{S13)cy=2Xo3{E}3EwUhh2R zf34;F&8Qy*L@G9QlJbD?x#94F;y+GRoF1J>)K|00VgBV@;p{SV85XD1{z|J}xZ5}9 zOvfih0)PU%YO1Sl8cCHSQfYqWY!5cE%`R}e?#+@%>BiQmOYtT?bb%T~BZJ(>II&NnS58JQZ?)rVR;&*h8v>bTJ4 zzL*K7R@CtjuvgDgCddRF59i5d-+AxtRV0Ah22`Ox4$GaAmTxmZ0%c7zjdkK=e(RwVv~erx^ywYIbOzVCbQecjjh z`}xu%n_jtfUF$N9qL@|txi;vm73gj6_Mt4%3h(0;%r;efzPrCh-r^H&e4WEr-MPTd z9Z(gIqdb4DOlyN^^!s|>+OIzITti_xA&nvC@6v>PB_jV4G$V&uvouef%(jq}G$}sz$os|R>_N-8W*t^ze z#@$BY!}DNFuL3wxzBVXTsG%`bXWMf&#?KXS3L72V2@CM8)Y^7dUg)VAw;wQaDx>e3 z%GX(N?9RB~uWL5vDxt@TS7`>4OUU;N9C$(VBJ~d&#j7%5djWF(m03&?$5CCv zyR-Osc3y|IjDnplOZO~^*8T!Z*cUbbh3zl+%% zka9wMEi4mUV9R!m(B9eGlBwH>7h5(JGFY-tldLw{1;aXwR%j>o7C6D;7=`n-&zS)> z6m(M5B*8tAUSd{lOQYr1nA1f3*{i*v=AO)`A^z>}9`7?Oy?^k{Y`M!1!JpjiuBpUP zE7Wh_msfbVd-T;co4o+}SpwV4Vk#;V>C(ECdTx%I0E6$(;v-F1+?OlgpsGCYuGy{- z9L=j%q7HA%+&wRw@1{BEJv;kyuIrxs3%mBI6RPxmxi5X+upCFNh&Xn=pYhrK%fGfr z%}uW<&$NJHm|IiIQ9U-DE~r*R(kV(ac!uy@A2J>5oG-F?pbM8s#u=mJ6bIqOmNcCDtRXPuKh z;~U)#YG@XFf@Vx$;l5)*KTJ_r1@*5{THh&xJaggJU68-iTbW!;p4+t|(oWh#(Lm+HV^b|eW+fq>&k`$@-lnCHJh#`1 z{pvWQ_KoMzsN?+#Fr}RjPl(G6=SZQP*xMW_&UPlgc$Xhej5(3(AeD_l z@oE5z*-Ahu8{y*gU-{%Oey)*PG%=x(QlpRvwPi_!kS_&BySQ+0v6{fxKbHBx@0g#6 ze}QM{c@TL$-NJ=&%N><+I^L4X^*ms$TrW2YJ6<1jSQ}`kQ%s@ zbH(2?dbo35i^V;{rSYXpVkokV_kKuLg2O`|KNsxITKg<{7pE!(P4=zcbx&J-vrPT^ z@O<{R#GTd0>F(dLiCj(MSMU?pe6SO$19UEXGy%1E=E;_0TK$_zK`5 z(a>FAOO54I_klHpa{<#?lndXs@6?!XHKQ}AY+<-fgWY#dYT_E}A*Phx(lx%9G=JAu zF$(Kw%}tZ~k=B?*CUImEkBMRVC@x{jtfV!^aU27}+ozYZxfLc;cj`#~27@-Wt1$w| z@p_CUJUWHSxcGArc~Ga9%w0r?P|<#RbWZ7~?LhA=&gvA9Zz4`~QOt6bBh{Dgq0 zCf~(W2{$uMg%!o)Icp!EvClu+D1*eTKj)#oECp_EC}`Exw%6NtluMVJ`? zovOe#4M2Pzhy5dkT|HO$hn>xZM$HH-lb8ZEvkDa|@X^t8=#8)qgnb1551H5t;zg?>RxKe3YEp2STxy9NLCQ^ z@xI*Pfmam>J-C?`&spLh&)joh{s~j?bZc`6@gOJ|46~sA45o0!YI;zOG$dC}&d^#$ zC><9DKW*{%+<%S^>cD(~HeNVO8A*WkkhDZ0gErc?o`a6Rym7rewa&cVfK+C$=a}G; zEe!6pEX)gK#xh(tUFsLd$9+X|?jv- zWtmE^)yCSL^@CdL3IsyQ7>lSIJhy)FIrxG$O zy8)IsrWHoJ6x2P)&vM`e)bxSOybn3kWN-0$dQ-1t?B+bdQOJ$_Ue=w&E@noyzucY3 zI#H-Z&Qc1Gc_O3mwAuj`uRuy!!d+c;ffT7NZ0W?8zXLbp4CH_>2L$=;GHauonzIsd zN=UObU}`h&o_0szGdIl$NJ%kkX@lELAz7hJVNh@Ym=*wxl0}TfmKt;@yF=h5;Q~vw z)jsn0Nze9SWF#wy2|jQi;8(HZ;|mCcoo~?AoO^LsA%mEdkfzznH#ww-W_6}Dr#zWD zwh@eFzytLcAlQ`xVef@)1~(9_-;#G})blSkXP2nK#)0@qfck;=3+31w2L`Iv#Ij%{ z`BVXhj%fUSRgG=P5N-HpKn+8hqXaH)U`De0na9gr9-Jo(%Y_x?=Fr~~Rgqxgh@gcv z>Trt?KxGKKxa;A>%A?7p^N(rhx-MFd!2=bU4Fn?H$9n}Cryp<35At>njjyCWOB<#S26_=`#g_c@W;oe<8J+3DH>Ueyii@c~_Yt0!-g9aQA_a zr$;MG~69mh}r&jiHSy@cD9v=ZE_bR3;(Gj^1dm8sY(Y zU(ZA0pP?#sm_kw_XsWXr*MhdbfhG^`$SWU~=U2n1ttmD!A zmd>j=u3#k|5Q=)h&pukKH|oo;&x1~KS?^dvf{GKZbN}gRmC4H3+m`^kMfPxY+=%dUCySW3*-xu6&t#zf3>LQoc9ASS`$NPW%Smu9qYJVJ)f#;f|*YSxwNjxE(~ z@LcBstVnps-I{IURNdlwN;(`KE99KnuD=KM#9hze?^&{yu*3D&gPoJjQnp}cFg3CE zPA_G%uzSv(+3umZ({0bPCym{8A;*d~@-rk>MDR9e4@n@HpzR9TKfPh8Tlx;DsJ%)}ySKU=p{F3Lz&zk;MAElv_d z2^3RxY4feHuhgNH9F;V8WZo`DW_ZE|Xa%<%%{6rl6`PqvK3ty4Gt<9tYE>Oy8qPif z?C?y`z;hMq)z@q!h`%4Eb>CzAa7UDoRf4o3MhOpAlDpClUcUHuv?Z;>ON&jltTupP zI}_N7t#TU&|H>TyDv!+PV;j90*ZxeaFt2m-_WrM8p;u~ld0FqW-n?zF4sbM3r`K)Z zZ1BPXT#Emt*zs?q7TAq3mY=i&C`!)Yh-iER98VkCy?f0zP+O4j@~-nLz1BrL3Da#t zB2izt-$N=`m#QB$W~V!P?AWa7awFSETWMZZCxIF zC4}dl-e@x41c3&;laPjCKNy3%`TpUluz7a=h8NY4J>G}Au?C8RF>`SSnw1J3MC zGow1G<_q=x57C=YU9*J}iV~zgMXRnskIvDx|M}DtLRF8x#s3tX|Hi%j-@mJN-a{$M zRlJ?z$p;_+hSm_$8vgT-{Nodcs5suhYkQD6d6Z=1dSumxYuAK3#EE6q^uH7h@(O{cft4L?79(D%11It_Vc8q z;}{4MU4(iQK-U8Z;5Z8AJA9KQlaiB?l?ircT9$PPLXi)^6FojwI_CD4q%(4ET|vHI zJ|rJ}Lb%xtG#9*)x%bIXU7KhhYQbX%a)BG9r1D#%G^uK6HC0*mB$`~Bjdg{{vDn)~ zTn7k`qg*@>t!SqNL_6ZCCat>CI|Gk>gAKmy(yqC=3uhLHcLFL$SB}EG4u}y;YTL2s zzv2_l(@`C0ZkViZ2gF^|u-ha{4k1i|O-k6=@R*2HwfNZW*m{CfA1=mtTl%!qk67Ft z@4$$`SmxwRs0z9HS2%A_-KN@hIm$wXPSMc}5+XQY*a#!^|Fdy%db@`zeH8NVP=@}5 P@MENkj&il4b;SPw>{IwI 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& diff --git a/docs/img/edge-cut.png b/docs/img/edge-cut.png deleted file mode 100644 index 698f4ff181e43d2b11aa649d9e0f5cc99494a747..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12563 zcmXw|IT1P?v-bGLJlcAuXS*#@_ROBQi$W&Y$ zEv@Y=prB}R&5VuNo!p#VEU=A@N2eI*P+UCKA|qqfjC*=}`+BFyCXFYJe`M(E@8DzZ z3_!toKY3S>5#suF4EG{ow^gCCqk`kf5*XU@%i+pTF))cria*kDu(A?Dt*#zW!HYT% z^NBXgs&Qv@K@T#cAiE$$lS0=&&5@roka=K-{(~E2hVg+ZKSY)F=g*iLgf^uSTVW^j zkh6gwHAFai4IVF?{2Pl6PYDYP3qO+bTC5SaGyXeiVZ#)Dx^w@vF!;4_FlhmluN(Oy z+KvxLjv_{0YGP$=U4Z7{;YT&1e<2hATBQS%=PO>_#P*A8* z|Gl7j#wMzvpje{iB*iqmp#KOVdx3`LQsMfsJ|jQSJ&>1e(8It`l+(h{=h8+BV|^<0 zA}^0bjDF4yzLXh4mHdDUM@r^Gib6|9MpmBa6eij#!R&Hpw8(qwLDP}Fy2;o6wXBCF z^GD`I%_*ejf%zb^=Df6g|YK`n8g||B}hDv~Le%}mZPg+6S-yTj8Yd&0T`uA%b?gzd-l}$ffNI260=ik~{ z2iNfL9U+U$_7KOiTDR%(R=rH=$?sAal>XmP6GRHE>kVHqA|(11eYd)? zbsAO2$HsO8<#YS{#q)30TO1=cHuMkbd~qXTy7}lagPG|M)ySx*!eGm_e&!m9T1=#Y zw+%YIux@T{cAAzALx7lI2!M;5Ot{A|8ek;9Q`pC-?99Zm-{|5V^K~!^ht+tR{HkvoigRTaDxtyKGMKfXqkB8AXGeNr>5k?f^-890{wuegPCGE9IHhL z?q_Oh8~F*(%JOaVp+gor8$o}x!otEGAg<7GbfS-AN|2Y+#Y(<^(Vb|wePhWq;BI0e zpOjlF0fFrGUpPjm3l&8-QaT_soMOu84w!hm*FP=WUS$T|ethREbrN`H6-6!9^9Vi< z7w8Y0o_X6$)CQ@m8FD(JMb+$qQO$EH3q65>WHCHOa;mq}c_No*?laQ?|L(LhgnYR% zjgE^dX=at6NK$|s1LH_xZ#eD_4OH?#Ozk_r$NA`$D8fKB zWyL6;!v^n`hoJdyyi1x~FYywLwW^n?2!=Dk?L=~k*F(AscpK1p3`@5$lU}oiN%%nC zuO9*&f^=HM+9Ta%Mc=T}HLG=pYFdAzNbn~%-xi$ynz7U)%u>o^51WU^n|}Qrn}sqR z0%0w}DC&`P#}wM~J|YR!fe|)pccU8^U@EG=W)#60g~@|xj>xe|q2Ki-4u?CE7x zdf%fTp1S?%Sd`MNIUO(J1K+19qOzF)Z_U=5?W-I%+v(B5JS+{7M_>^@9E&MG3u2M) zoGn$G9ZaORG~&>afbeodwa`;Bwto-B#JC;nId*|dh}Menyp=ILE;h#GQ|X6p{a^mH z2;V;XM8#;#F@nm=ghKaYIy!z26`@9>A&!U#W!+gi zj&LaKe9QOpt{*EQf}L=+LXk>7Wk*Z5=juyJndMa0>!y9n=HV0l&gEs#yJ7V_-c*&B zv@MDfo{J1BuY)(6X(8^95?=h)ufJsRCBJwXOX8aTz35jh-h|@R-<`I;sBXlF(|@FY z0G2L(buR-YD-&}rpR2UiZNS^pS7}nA5+#A3oJo7!`OKXbll$7xBAyq-1F#@*H;NM2 zq98Ph0Rr+D|yA&8&6MAON%e;`*cE-bo)aTlkhGiWK5Q0C&B70D!6HXcacdaQjc z;`3znedpP9OV-VUkqJG!+8wB-Zlk&)rrX)}fB2F3L-u6!xN`>k5CJw6?MuoN_ij|c zN~g5$8Vsr@iOH`VKJiqt4dU>vs712${+~!Sy7r`4Hovt3n+4%4Wsq`ivB0Q5$e15 z&9Yvc-0Shm1ebKf{VLi#IL(^XX-D+VpH_rce!R|XM4Zk>hv4Q*kyl0&TKVcX#^*$B znDFjyuW`9!RhXx-YYAoAv>8bj%=&jtr`yhWVm>uT+<@C%I!+e%?S_Ea0MFBfF-HM& zzuj`IoL3LuT)+Nbly+-fzC3oT^-}n+f-+9=?8Rk4kJ zSemv7kCbnFwC1WjtO%1;It{TEqu+FwY7EZ6s88Y}Qfxi(_+9LHu9Rv5F=DAbY1O3g zHpxZ(h{?W2I?^bnC=OgfNZrHM3x&0PyhFSGVZYqxX{B@Z`W4HUVA?#zDKnfZ>#GFA zYzm)RagxlWov=Zfq1pTxLvWOrpFK*8i$Av>r0HqwQtOXe7h?4*x)=Ot*XUQ|O`gZN zIo}K2$A_l5BxR=-xGq(_8zypubBWl`tm(SfonEgE*t4#y9bX4)|NKhYaW#*+%m$+* zykP#tV^=X(ZA=34lMd#Ihl74~vD@i79oNymgM@OYzWuZb)-emOXpHgoaW7%*4azFZ zrkq1gU`?x7m0U0pXr>M{+yAFAY>v1E%I}?@Ieb3*sU`Py|5!0~ZC1F6&E)mJR>?rC zpw3_$BBr^nMX$iCK15dU8}n4SecZIeU3rR@8ENXu^{=3d`rp;|J!Wvk7!p?N)I&m2 zca8J~>Jp+8oVdiFlHuIO_M4I%{~|e9yexzhTuCgw72CI)!R6#^Ph?c}XKS9(VI%uU z9hg&O2T>`cT^WDK)wGjN*x@WoXVV$NYp~wKhn8s(vz#AKd9LowhHG`)?*8^k&6{XO zgapq4zWFFPX_m4TTWzocO+YVFWN?Ip2OfV zgwn&oTSmLnBJVk*_^Mk*o}2DhfR?0fG1QuVYMDK4gBh*zpuZ#XPX!jWU5~m3mJMP> zNGgMtCRoX?wYG;&UN&Depx`DJ2!QcK><$uO+hfa_J=rW zyBNlzM#alTxl=y2AyrD=ijd6Bw*e^ z{8`iM;$cKMo7yi)GyU|u)#aP2RjNPMK_7O3dtKyvJH?YdXW7Bvo}$cs&TB)leY~$1 z!`LP%x1SxWkH7oYazcAR{SG~XppVSRgII{eAaSRxDQ<`3SVYL~;td=fbhz7?6^~`| z?Qp`9+NJngL4s)UyzU^z%}y|Cqb2udZu_6N{tV>2;A8LbJu#GZZl6S~s73M^CFOu< zl2hF{gIC*xO!E)z4uXBdw7SthUR8|Mt*qVru-u5u>2_psH?2f z{nD=cc5wcxQ!Vno?|j!HD^jZT!$!d9c(2E4SN!{^DKB$8o>Sq<2x*|*x~vi<&oewv zj&nEtr1m|of{jiv$$3hz%e~!gs#N(`_0lhO_MgiVM*%g77yZeS3#OS%l^Pin>3MQGJOnQ+s$>?zW82D;VtQRwWwvpmh+%Dj;D`mf`2R~ z>4G1xK8VDt`eA^OZMr;jTyQd$!Cg1_t5kcf*Zq5Fhn7%Uw#y7kZFpb);>+PwwjRZP z<7L9zE&71L-)LfsvG8C*RxYabHVG5LCYI)>Rjvl_Z=6!>G;qasrgv-u)YZPwp8=!>*97Mf5DyBj9HMj6Gt z0@};K;jbSP^$iW)cMIP~_pwCkUol7!uOwKzjU#TWFyl06am_7j8|yG?I~OQ-HNNWU zxG7Jb0sx`UZ{X1MRa<|akB-gquIWa${yo8Q)) zHT}tKCO@@;0WXAc(cqWj z=w&%?fie%VL&L%=QPKvx4btX*?h1BG7P*ryKW%$?=)I2cfu86-TF4W)D(#dpDnij zKn`tMST7^o$HmPhP&C2Q0s$%C6}{J5$*WjJwy_3X!LDbk$B^Sge}OKW57J;3UQYxy zM9;8JSp4swOW_tW?r>6U7H2DkR~_^hH$KI;(YH&vEmuPD4$TLmMxG6^9U_C}e+JNZ zIpIVZEN&FWm%^KM&0(%$l6voB^&qgC2gnaIm=B5v9tppvQ$`*@4)5z`!Lv<9g2|U- zgY{A|ABEA4mAcqrm?!^<#EZR#D2s)aZL<>)-Bx;nU2m<4k%B7=5ewsyFGn5Lwjn+z z++ODO=8S`zUU;8{xmMwhm$XhGhfDJtsD=LH66O;38jB3+^aY<7rdFH6aQO$Hx+}BR zw?_2z?Y-?Gl}T@|&sT54F}gZClFLFHz@1=JQrUY5-t z%ei|k*q}W3OM8i?MQkY~Z7V&%uC^zCFtQkS-e}EkT>rvAEB1Kf+3Yyq_~~nrI)ipn zAGVG@JSrmC``qn+MBdA(a5@$ggfDPPz5_Xd-Oj9kk>_SzVYFV|$-F)Y@I_ta6*s)L zoR3fg^H%!d0I^CS_=P++Gl6CDT@v?V;=&8=*`HMz9xt}KT~{`r6j;fd%zTyxVr1-R z(nGjTo5k$RW*2!KUKS;~9>LMm%3+3GKF~FgN+@dOT*eZ}g#s5%+@RvUYA<-!zR-U- ziBF~7UyD|?iF!m-2h1}E8=I;)4>{yKqJFoj!9iqbQvsz31ml%NSt~2`Jbm7`+a5h* z@x0xHIL(KMBCwW{n;Q@5E^ge-n36dURE)n>KaTd2`R~Rma3A6eFxpP=_r%?Ee(Rf6 z=K6N6C;APqMl1&E%T)+)`66!J7j(EUWwDD@sxGQfz1Yv`FFMJ`6ZDK$HQcqhmRFJO z#t$LK)Ht685_-q#LfV{UQoGRTmEW*mk$DP!D}eKOoeRgb!wtJCvm%rP6y%=?RyRiw zRChKeJs0=#PEdvC4Jy$f?Uh+)mSy$?z_lKnw%Xc-O`Pk&t<7Pioj;&1zwg7|oRQR8 zG+5NgGXkD+ zPnOS!4X-%%_V#w#>4jctaU=z(U(SeQsn`0!c

    W)7fI>$R3Z@Ps7dUpBZ78%JG3n z?A`xBh2w)_)tB$V*r{>WjBE0{(MY6ow>Rlc6jOe zUmUve!*-xZHb9Xk#>Y_*JDn_CF-TR+Jo7T23wIxxjJh{sYW7AFSYRlbnBo+F@TYFN zOfWGD3vV&>h;(aIX@5s2=1uhfJ1cX3zSgRuW|t>>0^#4aNFzNXrFWj=O1irs9XRF&;=O*ihtgM8w7IQj$nMX6 z_CU&O^xKklk$x@-nDoQl{dgG=fxo}locQE-C8k;Wg&@@$l*}K z-M@$JqthlD^td-g%I8QdN;gI47+fc{MM6%i{D8hV=_M_HiEy;pFu&{lW@;HRON0si z9?*0uNEZh4hj=YL7@|Fn^Bc$EyUv_2LC_2tuW>Jwg7;j#g+h3YJM?8t>VR11%&Tb5cupf^(Rrl*e z93qzD`xJUnS%sUhUe+x+J%DVil!FNbMfc!w2ovD9@J%K9j9x-7(sG7#gFi3u&D-En zfDMNrKU6`4WBlQAdpl;s5Jni_u;kyV#b-=|tzPz@zADuhAI_#P+>&)Rc{%525Y-E@ z7E$7>esyzeRBo!xpq6{*=i}4KpsQjL4gjP-eH<8C1Q3-9va*=X9I6rkhbpCD?oa4! z)bM!THq#k5(5ib?r$LjU6A{VdvR(ROmFsq~v3lhZV@R%p_KeM_n!5-AD;Tt>!4ty{ zBFAxJCB!mQSZ7-XqpX@$OI0Qcc8gGM^bB=~}!%V;mHKs3pw%sY90H&Gy2oa?K{3P`}p zsx6ADzmg5f#L+e&>)D7?0bD0Pfb)9-m?mppbjiv@rBm;$i)f0;x2wmCj!XQ9^3UP@Y+#y;2z1 zpOtcn_-<#uDn*MWyZ*d={ld#UWqOv(!F zq5MJ>BGnluJ26Jb`bV1(^C3NDUkiIZP}}SG>$6ZqQ>ZDeyDRB-K~8vb`jVthcLNkg*(UW7ygd|Tq zeVmU33ncD3JTEP^Wurw@{lDQ7o{ML6p0D77saLv1$(<>O!q}Oc<;ZFFGlGNJuds3s zKjOz|Uq%kBWRR^e4FyTBUj2BYj6|>r90QiZjk; z!HarPZjrUQem7&73c7mdxxN?BvEm^E_uI(R?I`aS6i4rx@Q>=pY8$D3^;9i%0|q=l zo~9>;EHF-m;0yA-HdnYp{{b27SCAKr+T2hK9>J3UP9_b=|1Tg8ju=(pb1p7AZH`CHHitgfzBnGA%B-xT}5+{YH~N}7r@$dh{qucsRB#&p8v;NJlZ z06D^|%jd2$@nf9?3CY5D)W=zYHKO=WPrsRU>WfwU$S<3}@5n8Za#%y7J`4j7^om1Z z!?_V}i~KgfK3gP%8p0bUE!CoCC>!IIT<6!#AN|$-hPoc>F#y1q{a2xfc}9W%ny{vl z+COBmuaTIpyD~vx(BT%8DMQmVDTF@pGTrU<#Zbt`JmzCsdpr&g#|C2^p1-bwKqCpH zETM@#0AMa{R34U>iW27Vdvwa%zopwrN!CRD19H(7GORTYQZAPA&j{q7a$6XVC$X(e z?D==UUfQC{4F%#2g<Hum#i8jn}n-E9>RB>&2d zrZMRql4ka7A7eX>E9hByZa7C+kE&k{RH&CdgSn#ZqsR!Xm;?5kHT8QFHloA%yd0$| znkLU@#XG+J1td(?EJ$WijT~A5cp^KPk)mFlLu#c6hIoMVhJIL81+EKI&8Pi z?#`d}N94jp%a%_JshUbgL73-3s#ZIsU7VOii_qG73T{&;Y%m$vdN|gEkQWchqMZhn zFf}Q9PFraayy(%7J#B30^;?~wRScjxBcX26QhIbBZ(ZTM$LnKBzn_r=6MY|W}l$!0= zhrNSI&41zC1SL=k9m&NJa`7rhmMt52lH`9wELR<2AR#5^QHTWb@?EnDN0SEn8Aoz? z;Q?vnP<6gf`iIM3;KhNWJcAN+f&k?x0CU-$DUj%7 zCUmEhC1(x|Gib?0X}nSeA6?1CI3>4&c8>anq?8>dO|j`A{r=9E9;46J-lORKcOSs5 za^=GBudWlJ*@@JtZ+i(Mn%5&k++M(~zv%lAkq@d>aF6`ldh}J%0zicLOs%s|*T(bF zxD1^it*SJpnp%O&l15>=-9g_DPr9`s5)hIium%m&u=lpWvrPLEnIATH>N~lEktsOQImp7^ zkfb()Msyn__TsJ%Q_i$4rjog)dz?P;Z@Z3-fu?` zsD&7KuiY>_zufC3g9=3gx&Us-DZQJU!QgaJCB_D6nmRxJx^E`r_?#@mh&s`una7u+#C1_3s3T40u-NSZzHh=cxH$#xL zFhr{V=vBC(q!_THuK~0Q*Yn*M7F!9jLNIa(l zdcL3OLegMs-lIfCtrnXA{2Ef+cb=3 z>zi8j&qFKoY6H8E{H4p4<)`@P!2>Q@i;BP;Ljh)%(O|SOb(~k{f=IG>wl^9xF4^o|zciRm>Hl`02d77i_u9v+APP5u189=a)`D*&N zfSUnoXoWg8$n3&fTbkBQ~vpgN2NW`e0odb2Q5e9U%dJD zs%%e+WWYUM*A(a5J)|kxM}WMssk#NWm@DlyO)0&*Aw2Rtom&ztA`}!(y-qJSA}$+{ z?Q9`@Onkg1UT~tdRDc(T)7mP_XIf&z1 zUSiVx1LY;V8Wwsw(Vp=Ltn@z&eNL!Y^i}d#0XeV&&;jiR%c;(r=ijKa1R}+DlNoFq zJqBW8L0bOby>VrfgY-C79nUyYm9)eNy|Rmre^2ddrBzF zZ_L^3OgpX1z3Yg!Y*vu?c4(7TpfuS~}m2 zAhqPg#-r)%>r9A?vls|RHv^JWjk6LgDi)S$tM2RPn|M--i1x*u+v|{J4b!#=j0~8O zNqJT9*K)(kzw%$cn6t65o%*x>eMoQ7s8D;A2)NkbK^90d zfwri>9^>L+Z!%^-?nEv!_kg0A$ZSMI4c;V0kMmQ?S^`Wxji25A`VS)g2U}Kucy3wC z$*A#DP0e$#vwyi1HGfQ6V2nofJ{{=NDM;Xgp8t-7)tkxjRl6tPAL9XPb;1seK_vbJ zn2`eRb*0X%3dkSQJSe0_0Z-VpmAtVBKgSd@E!|zVNQHo@+H478){MXuJ2v=)mEeb?VD3B%GbAzA0z0ij^xBBRv6b9Sd4me{h-_vy9n;ELmB*Lkz`r>AXCv2whZ`yblD$V{bmleQN$7#1R zqcP~x{12%DKQ{xByDAV~Bkv9eNVv058d}GJIyMT4fJd=dNo!SF4~_ahkOx3#QV7uT z-Egj&B?wU$JE@P53p@ZSHft+FzNOeENQz03PHA@cuDu(uj)co!vGZm0%yU4++e6CgbmWL@v)TNeH_;Fw+1!C+xaG+zF6JHMe2_OlmJ#D|K;4Q?)QnurI$ZvuEgv0{AOAm<4z~ zEEF=|brl`%_&!{~3%Y-f<4?`WL32M_;vzB9Z;$BQ7a?oz!zI?dFzR5sLkKCN$nQ%y z`NIhVmHQcvGZu!^maC9|{mf~B_!V%E)&9uuBV)B)sxq#R$Vu<4rSJE@NFGac(PL<> z-DyZXLv7xf%;X4RGSv&%7!F-+I6!|br^+dUd+oUFo^Mt@_N}~Tnj9N0FEOG}#Lw|L zF&WkG8pVFr-u&f6L`Bpcw04G%`w!S@RDcyGYL9zCQW0W>fqs2`owl16$d0Ra`3LQZ zZcL?gLQ&;U2vuo+skJQND^v_KaXV;Cjna%zX?L?Os%-8Q9*Nx%d4Fv<7(~S*s|W!M zn#e6#`Qz#3`XmrVqYnoNf^gt6MkI(GZ6=SItobUoy@kVew*hyf|L-$==yyQd`EjfK zjqunps8^CGe;pse#MK*jyIGkPrX-|d*jj5AknSBL$cv525#Z+=8^_bRUb?3mLE@dt zH2-oeEi2vVzQ@R51myk`_`ttyyQyqr(Jh9$6Kldi%bOuK80Jb5JK(Fw=fH>hW@XLJ zCO@axZ|btd1j$uiXC+Nlf0A(>nT!C#1mJgAYF3PX0qCG?(ARR=aiEKr9y&{(g zm+ixgO~3`*Mw9so7n`2jOz=jdCS~fO*+>_FgNjh{C;`}yKL526BZ<5kd|V85k^JmZ zw%F4qWNElMUh$~IoStT0Yxts`%jPTk3qp!kR-UIkM z2UJh_uzH;-dAXv`BgdeNtX?u2sqTS#v2?K$j*%6#?Jye3Fqvi{9S)za+uVj;K`kYG z4xx;1=FiTX=s6|w8vJxL)~j61)|Ie9MV?JKvyu2F)K2AA!0UTZP~+E~@~>dQaMPVE zp>k22cVncMm%lbHmJwuf*Zp#V6U+kmL0vg#N$^%BC#nHg?tL#T#>!geC;z{y3jGMS z>&!PoYxJy~v~{U0%EC{@qo3K~cmQ9bYLs;B*F68Fv|o}Yv77tZFrWbvPeH~b)S9VU zz~K5KJ#L~*Dc*~ls|qF97}s1Zu^WsH?EG~wYh4KWP|SktjmHh4EMC8|tF`MWptyhe z;HttXRY^!7a07H+U0~k8b90EbJGH?ldaPPbUfvXVJaVP7aDo*S6gb+bOu?kiFg1yR zK>cX)ShiTm_ud@vUaE_>E@RTx0_Zbb06iRqU&>{()!7VbWz^`iDZIYPCXnsoS2q@s z_{#o3nIa@uW8sKZDbzSIM9+sBQYPhhvl>;WR~xbpNWhXu>y_T3C$=8@IKE_IX!r?( zveD zE75JRj2lk}G>tgmPZe>@zbaAs!B21G^Y<7U0BP>8I=(`Up78pJspZLhv*U6y^I^Y! f-;{XWzlv;UvSNd6E;s?JK9ro4l4Omzaq#~E&DwJe diff --git a/docs/img/edge_cut_vs_vertex_cut.png b/docs/img/edge_cut_vs_vertex_cut.png index ae30396d3fe19c594f3ad715236fde4233fc0775..5b1ed78a42c06f140dbb2716a855e3ebfedf6946 100644 GIT binary patch literal 51015 zcmYhj1yq$=8!fyM0YOT-Ly_(V>F#a>q(eedHm%Yj-Q6X<=@gLeZcyox?)VqyyZ0Y= z49^&PHhaD6UF*p?=R=5!k~A6$5eftXL6en{P=i38gTKRVy?6%xPa74jANU8IwYa#7 zthhLZij#wdwXHb>LK|)R;RE|87e^;^tPdahh8gHkoIa}s2ZyVDXz%Fw*)dEp^kL{j zdeZy%YXq2UU6AL{YiKbl!P+7hq76mGA^uthHLqi$SSQ$yfw z(fyg=O%n68{PxX1jj$9k4k(P$sAxfQU{e{LkNmv~Gx$fsp=uNpltIH?eHSLY?7! zSg5EE@`MCB4;0-B9)tvV+pMfd%96pwgzT&@wmDe$Q{f?96)Fx=Ch5%J?2tdn=sH6n z*nCg_!9{jz!9gIux@9FqHQW~t(vfsFBhk}4F_FM*U}J4<-I?uu zd!eeTYDEG${qMt$;H$gq)3I!E#W>Yko9Uz^0?t={(EmRAz{sdm>Gu%}hfb|~Xej*N zvfh4qOaihAUJC;8Q~k6t5Dy+U$oQoNdWwjMIDzrz^=q4%AN`E}xzdq2*^nXl|CR|6 zr$kaxQer3e3->qjxemAwWqhu$uRmX5GgTytE@Q>~w3D~py}jf@Uhd;Lk__7AsL05o z-@unKSsor9_Jco6KGg(QG|QApX$ zmNTV}oNNr@ladbKGTr-cuoDTmv!=Z45P(p?f1LTDnlhd3b$wD{*eZ7W^a%V!79$Ji zd9ZH$%4G9r2Mb{;8k*ekr`=ERxgSx?QdL$~u00gi)NrJ+gGU61v6=Q%7JZ{p&Y*3z zTy68Vt-d{|Umdf0Iv6AOi+#ek&L8+)rb{$sWo0ADh0BfFh163P>m8)U!M8DA78e&I zyu^!BG%nMxAK9 zu-V|RYil34YwebsG=HU=55z?;&(v5d!-&C!bv!=YohlcIEY{f#-X3fYr_MXjJ;JBF z1UE&i)9{HhwS7D4{jc;^kMo#it@8JL|L03L7YBn&p%2$v{pvfhH zM_N+SJmDD^GxK-!*W2q$)6?-We(KzL`t|nNP`Dv@3f({qa^}gvy1Kd&XD-|M>VV>r zLgmbyou`vYR!ZmZ-~3@QoMH~;AZ4lK`#u33^pW{qYiMW)j)X_mj~yB%`9%tm6~pg& z_In2#3NS)*jV7Wd=V|JzEjL35q$NDk8@qeDTSm+kk(Ht%=$>m%v!Y(q8Vu1_}T z&dC=DftAO+=H^a7A?=%50^X^)N`i;ibd%j8cc@6zvTTAlu--Wb-~rzYp5Dr3t>-Q97X9<# zkdTo5&FNC@-1DcC@$kNgk7v-ffO1^=r;)zHWM*b|SZ&)__IiBq*7{UtKA2!-^Rx@3 z&<2y*sOe%#2V>*xDeRDvx9{7$t`{BJNx5vMw#RdeM;4ZwKaaP7HBk{1ZEQo@-3}HQ z{}q;Gg^nW}EY!-&$*Fk=zh#eEHnp)KA|;JLL;|~#xY|s0Fk2`CKDN6wTFo<*A((nU zwz}G`mM<5oSF2rN_!KDKfJinboGIjOaytHb)jM(Ho-^^-f-Pl1LqmQ|OZQht%fTcd@Kd8b+l+)8x$a`7H_S8NM}z^*c`kAB0ahI895 zC2ib;p!9ph$c0+uHV@bwMZnC*I6!34U`!4mvV=Z1IR0&jodr={b5&WLxOskYkr)mA z0~sVl1-O2)O>PpL>!GEtilCq%h_*wVZ<5j0?QvecqHI36T=lI`%SWlx1m{SAnUL%< z^Fvp6WvK<~%xxEK@QRGA{N~~6%H8d;KrwY};lFqO{{1`qG`G*)6{EvuiP&Ky+l@(jVO#BOsaS$D;2V*oe9PQ;VRjKj=Ln3S&a-RHxlkQqb>Bukn9 z^C4sEH;^vMZbry`uD-l$4hRgixc;pXqU*+ye>cK-O$V0qyZT^RyqkmslFjdAB+tu3 zO-Er|@hq$}#yG+CB+Wal|Jm@G-!2(r`~bQ0dE zc{|^SYnsn)3~G77yOH@d0lU+sgZ6b*RnW^r(i9>P9VW7MYb@9iX7l9}86Bus742I& zrjcqNZ~yJbf44Hf^aZ)>f2Zv)_ADu~G*tWqSeVl=ytA`&NpjX7)7dipUi*T)mH&kX zlxmczA1P^RX$s@o3X~&BY;OiJ3Kh`zeFl#F{yS#N1(89-6Q&96> z8&2g(?iY2%&B@Ij?|U*L97ck$xZK@$;$mVAB^<1*tb-Uv?LLCdkH>#||J#p;MOPr2 zE%j@oH@}GM99GMdYCOTxa>5bM2z10gQEk8jXXeyxbb71qT+&L@_!%qiHUA)n!(3(d z{^mr!d~Wg6F=zJm$!u=&+s?7BIP7fu#Vh41tTLvOOXAROatr!Cc>Vl}FfT9%QZOgW zlPSF*ea5PYPeStJj}kp)KNp5fe#yf+2=`T~TSLj;3V<1Xz-;>bZ{#b=!~LbvVFj&| zpQ0usfihw`773_1K%#g?VC6%?`{_gZ`^GtkR8H%Y1X zxW4Mmy96eE_km;Z6?&o5txR8w&llU{;-d~{JCij0%@nQeE;|$bu{6z%js1DbkMfTN znPa0n^)48xRKVmbRZ<#)gM&dWXa2V*qQ+vFh|kHa-REAW7F1`>9e!{i`)6cifJ&s< z{W!;F*mh`aML0Enccx4}mZ(<$aH%mFEo%x@1?Ap2VQWjzKc)y+`%z)0PtGSHITUu2 z?ib}}z28aSIjjiKw;X{8FDWT0I>w3t$*gx+jS*x8k)}kmw3@8~9VCbRV_*$JCn}ts zZf`tgXV9Uin~Dkw8Xg#Q>V~=L{I2vh7+&``T&00hUsVb@y+@$uySVe7IMxPLCX(m1 z0?V?OvE_x>W*)YvuOWUSQG75)-ReBB?##^0?x+<=&4qC;E<_5*Zq*9Hb^>FB4Xv%- z77-Q}7CpDf0Yi?l=lxk!GBA|cuF05dQAczT@EsOl#UNJSOcWj=SCe>od07SFhy_7E zOo5mDqJ|XpOw(7j=&Q&1?sx+`J$9N_&;Ge`8gI5c{DF+4{P5s|=;Bdw$hVL3W(jwq z`YKX<3(swq8mdCLDI^u%a+{i%u>OER5Wm!gj;6npegy;72H&i;#!49b%_JWmQ9_or z{0_Bhh5*!EIPbMEj<$$1TsnvPIGGdAT=VA&eq=tU7r6Ra>d*MM`w~Qx2;a&lVfl6S7VdG6mpZCrz9B_6=_3hI1vm>2Egwn$>Mr6|yV6>JH0q>xfOBS8eBU^%@rY|}A-7um zE`SlmN?%Z1ese^Tjt7T0%_qlK`x*Zd1v#btfCiJ;ZHC#3O14@!dlC}X7VxQcx$?6H zKbYA=4zAq0+KNp9nBnZsFjBcKpIO2aEMQ zyNn90$dJoXA>AlzRKWPwVT?t*hUC z(hFZ{_ciJ)^Z2(SxXC4vs=%mA94}~rw4^~5lh`7InEX7nhDcEigG^xT##!wX84qQQ zPu3BOU>Y4fBplO;I@?|SZJ=oR68hT`O>3bB+00)=Lsl+a`)u_lU50J}1GMLS*}Q^p zZ0$}4x9Z?oB8nLgiiLQdTjwi5*}79HPqmAjigJJ^wr zjIN*Czm>A{_k`<>Q(2<$*y7?L>qH~ep((j1x-@+o_L zXsflQ!O5@^85h0FEvO{VPYG~hW-E=wv44sL_+dssPDSEWPAu$SR2MhmUaRklMmVvA zI&IoDxx2f|uw-(!;wU?gjnnX8tKQ%xMxX@*?rlZ)We-ZYGa9xKz+*$Q;55Sg64L{} z7uRmf<*n4s+RCl{9Q;dn?P1f_)|Om+HdCg**ZVncg0iHj^#d&}BK%H^F)vASF-O%Eve4}f}j z8<|ZE4zla~yA#3wisZm;d?i-Yg442|%Y4s;?YH}=8dSw#Jr)z)d22+oWT;fT;^E=m zQ&6A-q``BNfFD9rFJ4IcK`GF!4kT$ZmJa2UmpKRJ@)W)GnmOt}hy|gO=HdKMD2|4H z`4}o^2J)hZJ63fwi1OyIZ9$VA#yJ6ja8b5P*k9}G2Vn&+9GKvn%jILC>_?;EGkfjL zFcsGcTiZ%<8UD~=yQeD1CBVope)JbFR?E*z&jxodLGjtm{(C(Nng@|q!~ki89{Lu) zy;_E%TrpJyH;YbXmd&}SF`|b%>dAtd-lCeHin;pEbrKu|hK?(?xq#?`r&!e~mNZUu zsVCx!+u&#(qhKkauM{fkf+H)_-+XRLh_b{TrCOjsSY}D;4~4*x&7W0F2G{fx9Wa0VI^i$U@%+&BR`z&0emDm(- z^9yG7#Ol&31unGb=EQyF{OJWM=AIWxpi4G!aFd%W5i{qsU(yi{;4X#*gv`ystg!=` zAR=_5nZkuk(C>CCk#`ya+_~ebQY&Bc=fq=ji{tJr=uC@qRtmgn1--c9NpO`m1-R$BNbanXO3^1j?zqWDYr6(85|L*&gc&#H|0bB$2x z;+2BuxlIo?>{U2}Da+5cR^>OIs8lDc9tt+V@ zVUXc?-hwEmwUdAi#oM7gUYn~jmE_M=y;W(Y3pYwa68jP(X)$O1Y6bbnAI%?X>K8+x zIKp!H%b<88MHU-Zs4I*y*6!4rlc zT}zdwxyAd|&EmTts2!%x>(zOyi!3?iOmRU=t%`cuaTH~WT*~TT92<$wA5bB?lk>h5 zx*t~a3KtWTV&bCWePi{Xho8T@g3eD5V5-KieXI&m;0Yl|?T>038fAK#8QN_dpHxAX ziFdOZC2RitZ{@X}??8umGQTFyv&o(M-e14?`1wPMKP zQ_kcO1`cax=I0?oeC?xefse?3dF!aFFhSc!rJs za|Uj_@$k+4x`pznrBM!AS{auJ<(g06bTqbizrZ8yp24}}8vNQb!NJ1%S_NJ(>QiaD z`UKA_YqfdJ5)1u{yBf<;Jng^ad`?vOg)^zB^z0S&NIGp^jUgIG7Z*LB5Ig!p!o$t@ zxQCcdRW?-|hpo<=6f6-$7&M9~%5NfZ&{rCrEi)`E+K|~n)0*3ibOn0wUrDCobfCSJ zv?EVKLi~nqJkLfljcqkyS+za%Z4+KwZ=SzWj^U>NG#9tru(gEqZOzJ^nN@1<&!3{~ zn{e?7xu)i*Hjr>a3=E9xY?IiBoYe7cdewXcPXu9M;a6O3CL*l!h%S|*_x+#~aJhtt z2cdICV}AMagZ zs&Iw8UrnxitVM9FdO0~cwzjrr3mr4r{>`<=|MK#7BcgKpRgKy}?&fzS_-ZIY-FW?lOr3;2YPDFF&!#gENm)%e-A z$N(G0bRTR7tr170@50CNNDRUJfCB$YEDnrNd={l~9D)T10nL(G!E{O(isC6Bw zXEb;#I(UqKL2*+u(xVhBW(D-T&YP$`Tto2Q;>OB9UaECIKekb8 zqURya%B+zu`!SfxZI_bFOsksX??cwLy1FV6Dnnt!3lV|u(@r4mUsz{&J&_BqMaxv5 z9I$VYmY$BVa?*Z(k(!_XRvEXsscE9IH-ebQU3hl^|biVa_cs?rpQt%818sqyz*Y zG!lPs=@J+tH?sktdtt8&{|rtub$+SYbb_tKPT^c7qC%#SK`k=@>jytFvTrA9sVOOP zYRNJGii_l};31sfP!RH(tx{6B_SzNpIrNrCP6+lK+V8cJ?38|zY1C&F`VS^B-QJvk zoJSK95?V3>xVOBt@}EC{OkS&UJFJ9{qn(QQ0&rd0kgH(&1#%LS(aBG5^ZuB(&upEE z)# zZk+P&=%&ERj!8748M8-l>I%&X>|M z;P>!CdcJuj@@pT0no!swc?MLLhk~FunfcX) zN-ne{F|loI4A4=pFD4;6^$bSsL1shn3!EnZ&Oz>anO6%K865Rs0O7@A@q+MoY8 zRI;~qeSf*Cm^T7YU)CWwRx;3gydk~?^}LLQlrve&!j!#$PtcpMRDhPGVoy3}!8nEogO&`viv;rKOWInoNx1EPV#&A` zJGgQ5w-betAUcCLpd)f=Nnav&dYXjaP+l7xvkn^0NVWW6oLQO{+N+P>9u;PQT)F2Qw3i!~hu?28%C zCywL(7L&6hjlABs6$-##gN)JrqD6?}BkpaOf@H0wOwY&r-~=U&9Sk4uj(f=&w zd%vIAhJ1F2f>Jaae%}${QAZJ7NoO%?o4R?EF6?V~!aX1YcG(ptq+`t;3az@D!~50(w%}Mwo+bdD@QWZJB~?UM5>VXR-jxn-yw#)s8{{H@+EZP3wxcy-eRxGJVvh)m{N$U1)3s}K@o4Ji0 zC^Q3Yegh-MW70dC`jMVa%9ALy&n9d^CQ*?rDXiP@DVq%4)e{ulsIe&~TC|`)8H^Ga z*!yH|^EobweS_?mQ&`P*>=Of-7VK{7IhG|FT1cn?&F~xrdCcYBcKycMghYi;8PyFE z4u{-4sCM$YU=okT&^Br_KlvkowFD64=eznp2YW8w3hpICkL(wnZu%>YyBd7OTzy=Q z-Fw!U<)awbrTB;7m&H7HV}@*QxaxOaT(|*Dw(ueK67*pOg19RkZf{A!=|`9Wl!SS44UKZ zM!Vw5r@#x%ZIX+p*H{%fw%{+{cIuHY-{w0gLjCXGLjDJ9(m zY*-S~eBeOFEa=F2@``DP0B{nw^0@XBer1@k=jGE6K>Sk1j(@}r!w(7|a$;Cz0u8ydn~QB z^yc1&W7+Ty0*iBVMYOA)7yG(~6ZJ3{r-3j{S{MX^^^FgIgGoouV-p}5J0p^DJ0lds z@^oCqjQB5E3hwaa96Y&x#wQ~o^HfaO<15p|=4XfXa$s%c0Pw;z52Q{eLb2G7vV5Gn znYuZ;yQqlM@p&bd^bh_@dX1OB&z-mqary(I#%Y@@uYUt7W{yT3F^Nx#}G zBwH_Om;7>PGHA&igucvqL6*Q;lTA^zt7Q_tFz5z}FO>nCS)PdmSoel?E;`)3pHH!A&2~&h7Z$IUGQR7Q z(mAyNxB&?&90v!7awYqzqUi5!!4~?aAR%PhAZ&hN>-=H2L;8z9u87`~>qL4=T;8zB zB?RAFQsuM+TT(`Y7CM50o2`laXib*PF!sAJqh>G7%v!o|h~S_V{pZWrLTIn^uTyDquH@ zgNRFr?JU$XXw&R5m*4#P6&8~@j)Ft$XN-O-LJq2!pqg2B1NIb{t}nmINd6WrbBT3F z3HL;((QvTN$zK=@w*KAM5J7Ia%D4+r!VwNa1&=VIMj!0b+euXR)|Jdq!27lsQ0iiM zqIzO?NRNu-%7>3G$1Z=;_s+0+cGO|BM79=Nm1jz}c5TG0A1=r+Wq+OA5=9H?okh5g5Gg1mb z5iF-$8fmFYm?Td6P&!FhS6A_$pOb~ku9pWO`NUs@b31P;ggYi7d3-hDnZziQ(~1pe zXHw1QE2YdT2#T!Bn)DkQ$fSEqMdhDp9mL&nc3KoSL1WbBg*B50m|u?~3OEQRI?Tsp z0H*tBK9MwT^Es`1Dy6->J!(;yZ6A*VgVr}Xqdnc0#ya(hzWT3f=mxRVY;p>PbjH^AfRsVT!MzJ#m2GA>lH3$JNcm`3H$KY@93;^z0 zvPdG0h}l10eZ&U=*~7+2(D&WE5?01I?fyE*l24C8WlbyL!-(~xE?b1sd%hWPwk!%% zlvfDWW98qF#6v0uPPqB~F%XS-_mr|9O`~(b1q9Lm9_8rhsHLT~gWN37Cu`Ql;2=6r ztjg?^UH{mKJtW{>^C=v>(13UK8ukw5-GNdPlA3w}^kn^d*GR(vD?^fL?vuyWQFb=b z#X)s{9{C%hjYoI6Y272hyf^;ML{S5e0}Pj)g$4C=1m2W9V#kY})pp^*K51)u38Ukz zzh(t92`om;M?H~(RYA!}r47_Y0ic-Ao#%9xg;&Vt5mvz|B z(xEb2^NLA8){1jk-^Ck|YYzT`nJtaNB`@>YFYyu+6LUWRNkaLSG$>`msJN^z50}|s zaI9iVnL@*7BEO_sf7z4al+H<9FvD2NCmRgQj(jpPQmPs8W#|Nf%X}?sThT*et%gFl(G)B_(vMkaB z)gc?*?9EmrCQ&apIKDyTf=J1`xNt6tjl{MW)hrURjaVt(ddm|cOkxLVluGH=1gxH@ zo=R2jMAGLHt`h>DMyv`LB;p4$D>(iL^K*%!JGs9rFC`0QQ!#Yxtz(5-DU?=PG`i*> z!vxpsj$K{~d~GDNnm7}WXFKXZ06>*1OUL&kBg_CuJ9pNA1@*ZZ$A`}I!9b62AOn%>7r5Ff%w)|0VKhAb-RJ5yogx>eS7e<{m0L!YAHF;g zA35tSgc+A5S%h0@(Vh=8RLNu}8fi{}5(B-95b`F->HUOIv;s|%t_}~b{!9eG$!QIO z_pkeJdALd;++U7N8eXP8k0o2;o-$Plu%Hj6k02}qbT`2Wa7JX(_fMjS^S%_jo?w~S znScD>CYbv(S^(b&BQ1YK^s4{JHCq-uAsi-ivtDdeYom#5r8e+VN!plT1Oi#0iz25_ zaG)sX<<}32+5ZhnNSXw0Wz<%@%f0cr+;^+DWr|_m4jnYfK(X}Pf>Tag(psK10laAv zF^_#xN=iXtME!s~A9nX|EzVB?ifylLc@Yp1i9SF{36F~s-qxVyY(`?GpV6JjJ{#cV zclA~9VLfq5^{N$@(rgv8>AM3@VP%=KCnl33w#{Gk*t1Ox zxJ;{2f;FY3rP(Rp2QpP(aaN;c)UhPu(F_c1fyjniOHImV@;s0g;^zni3R+T$onFLE z9w@$2xw)PNllun;MlJ4ed^SCi!Vk2&gk?HaCPl8GoTa6rrk*J`(9fW#miqV;pMQHW zqYmg+Kq=3)ZZREJ$RbrM-{2^jHxU^TQKpjpRffwNXiiY$2k;;iIth%roiGWqTa#f# zM(mUDnabYx&VvH`Uw}Rb9eLBAPQ4)5Me2<E?hzqpa^2sAZG7H{yk##@lgRs3m5Gd%O6gh=&TTbx!v&%OgmX0k zg+Nb_l8Xx-nJxDM|JDmhKFDD1g7?a?rfy^}yN zAk0gk(tQ?}En`U07$QEfCnLKMv?Nh4W(v?1ERKE%q-hgKSAgE(q*~nqFFqBWbIM+O zho#woacDHpX=C8kt5<*Q7Uw!zZS`zp&00<*dpc3uI8*ZyW03 z?_RiGU|NS|*j9ql5Lr3`k6vRNXfk>LMQ4S|)D0;1ILhIcWbn@^j(?@T z`KLf2c>_SoNX%u^z3N_F$DrTr42Y3hbAqXV{J8+=EZDpYjg9}v{l|MSZX;-NqMJxf z@^_!DPadneu~6P0=ECex&d%bMpedCVy-uRrds}Evw1`l}sMltUmZ+MdS?0iIm9M_& zAg5t%pTX860>BAD+F9Zf zn|fvTFS`5IZgu~AZ6_;dF|a8u=S1H@b9Q|E8#xDL%fUpJ(NvxaAxq8_9Tvlu;B2H* zkrb79LOp6H@q|Y6sS`ZL4V~qHKOuWAX-lxxdO1zYQ@FuBw~MUTh;gNHYARa;b|o$p z;~{Vxr53BVYd^mUKu?a|UoyXM$jYoQ==iqPx)*ckIn%XMGbg8Kk%){-aE+5?HxMYW?aB$}6W=q#H zT7~9eS&MmsmG2X^Ntl3k4@djD{n12qb0l5fa;k|NwM46|3CMp)zI6#i30?kM{rvfJ zeND}vNqXX9_z8M+V7{LP!{W0%9_LgG)vWG;0UH40$R!5)`1rJ>%E!~^*BAMs2nQc0 zXSzI+TgzVnC49@Yu&z1KEDK;eZjWU*6-mjia(P{Sl4Jc!i3k}Z20g*Y+4B4e4PAQ! zl_9@;MW2$i4|dwYY0(ePBm*YRdB$CVPY>uVKVgh`P;Y;{qEl@dPD;Rh)ct^KO?y!< z`VCPa#NET=M4=mWUgOZO%ymsEJPtnVeXpqgf>QJSlMtTwlV9DnvtfMHk86N-srbXA zY2dET<;&x*kWcVy1`RNeG|;E1|Be0i>*H@AdSz%$9%!Gepk+xt$<5mr$s7aPOhM0! ztULiohu_`9?LW{9Ps*V(F!5Mhrqq>l+?G$lU?HnyYq~GOc+$V(!1@4S=ylwKQff-E${E3?^ALk%`p^mWF|UY?i`hh^#~U$aJK_eV{O;)(2kA{P;c<^ z@f91T#T-&`DW1P0ffYx%C1tkoZVFoC_nDpz4DpA;n ztQ>?~2%AGJDboZVc_gxM#l*zio3&v=*^~bm3m)QdstOGr!m z@pFs(om)5$wRNF1KHLwMUblo_dV>FEDXxzN`(N5WKHP`)&sL{?_=3a0mjTj_&2p0~ z-~y?4VwcP(3Vqd6EN}Sk6Ane&EP)0;Ykvi>LyLe|12-x`$JPhu1Q5tLUgPfISFoR= z*)P}Mxg0ET4GS2s7&iBEU@SzQgan7K<0XR-$aW92C)r8A<>0# z(TK=O3ltjA?{mF@@4^Iz8@7)?zF_(U;9rIF1>-3dSho+>DZ#VHoN5VB@z7S^wpbBK9{WZ_pR zB>dIHi;b4dfK|A>j|eePtG_PZ=^JlGX73 zGWB?wlM{K%R-t^^`gOJ;Y2X|_=%`IjP`i`KIv{jhhK((SiYi~EbbTu_fB8MA<2foC``TVJyd*^iLjxnFJmgK` z%iT4YN(PuYTb5H_$A<)h-{D(DS<1-B6dAViJE-YbnIQ0E0r2`^_K6d8+n%k+%q1G2QUsszeH62U;hnCFm3^bK4>(7 zXBF_lP{Ad!XSw&yfiicX^h|Sy>7FOuf?h(AS?fkLb|SBj0S^ zC-quBWR^XaMp-HoXt#el-R(1FK*zk7HhI{{$Jw9o>R~HTar*#uuO1>;PVVZ7)l^h$ zr0*IS*adSjOP&*0e==Rm=vktc8lCj+Xsme#pX%3?xP98{f-z(8rhP*OzF#dV-5G_` z5~zd2iBxNXTCUy1&H|wt464bMN&>q4y0E1CXkB=t5fL3V zx%!w(44SzsxZ3V7^v|$?c_Hxp6SCvt{01HKm}^?Zc#S47-S*v#W9QVc_WYt1obumc zD=?yWd-V!k8!X=!T3N~Z@#Du8pKCB~qqZt3Y^|GS4eNu85SABwcLJ2(&=eRrOf%8! zSRg*vnR7iu`3K2Rrd%*WHai$gaxAhC@d;Q17(sF-{_=ymI(1@*;P7te$-ZzIa7#RZs-%kq3$*Z^l4o5?9T&0Z^|xCSGrPPQ3gyE6X!duWOa zzyY>F+M?h$Y`q>c->1$9__Qlll${Y&{iWubbHF)?tj!UK(SeJB*_ovdF$lOiF!{Gc z;U3-l7txKSS6=wt2|!QX|E&P08!8b2V-%RY_zw*r^YVwEKu=yTP+*)MpJ4!?x7jPS zyj^L>`08oAhyf(QC_6A}k(ESrt+{5*8ePKGGB2UHJnZZ$YBWSapq&`CevEQl>$SU<0Q|27GN#7e7=?B}`Mq03N&j>x z_3%j_S6D?wg}+fF7yuAZlxxBj1kOX%(N(B=AY~7trMA7m)Tt?;On-rBY3kE-n}bcG zw8lyT9B5(PS?y2FA(z?Bp0Q?Z6~aWkJV+W_oU0(`sTDADp*UNYWVlDAuHG6iIo}U z&iX+dB?JNxCymg$K)uLGbqRvIUeBPs+S}Ov1c+%={-~RA|jfZDc z1^H*^AD}{wfo=e)+YzTV(6)dX$+4%K@mOlPg-ws4Zi?<1@7W6n$Mr0NaSICN41t!e zBOoCaJ$$;Ev{??M{+Cp@?pBSyD2txX57`rC-n~eI^I;X_Ml^ji;eU*1Q%Evhlf$ zc|?|8m$e729q0xKr4si8z!=u8C#V%uqL};v;0l@8oKD3s=&}E!`=XE~aQDWOe7f38 zW0EmP_RChQ&Bx2wM=*5?L|n+8&z_EPRV6(R|IH%%q8}&zFQwzd`Pz^84r~S>y#quf z_$(giG?nqtFJ1c{kTEc%Ex`j!*E-2B-8V&pRE3lcCH4-9>6nv;Ym^t_CjQX$_V#KQ zt0DZ)jH;nWK9@}_%|z8^L+vE09{K=U$%|$<03va-#iNy9qoWv4HWVK8|2SXa_VymQ zb@tOWT?`r~pOdTOzfcZ6Fr}E681Q8Ah!i^y!lRxup`higAugJb0|P)%{%s(&!CX#l zYHU1S`C2WJ3o@CXYIIVZfJ+@YsOsiD4YgzUyI3vAS_uC9D2u!Y$Pbi*Xqwd5e;-iywU7M;h?RB>)68Z8BH0YS)?{02yPYu7=l2vpD+KpxiMlt?(@%KWR-D4l;Z(_mx@e!wLp;CJH zNt;|oQ|*6Sa@Jn%Z(IxxWlg1)uc zjzr8iPy}UO0PP+a**>PO1)bZgiRYk`+msk4WoG%RH5l_JrsjMLc-)3{zElJea-&rT z5M}iR{uW}@<^B2d=hLY`>{q-OY^)GZv(77B>&Yc-ju}?G+hY`)|J~1QuIJJ+r~;tb zy@qvzPU@`H^9>degtRU7hD(#GWpeN=Yk2}iWS4a60DIFNHUpQ#ekqwc+fy5{l)}CKG;Q4l0 z7%t>ZUO8Kx)B>tmvCa4H7>ViyIPhWU&WbF6QdJ+~5=S{7XeAvUK&7L8EhXCe52(D{ z^EYLyVmu_DVoa2=kJoAT;G808p?bp}tv0%ZN#?|^6^j4BO-sl(Ks9SHR6ShU&HAzW z&|fSIWHGjYXr505JKY@4rFs&)71%Tb6g_J|-TUS&TY~K+9^+iS13A+|kvr3Rwxx`q z<8r7_#)zsahm)_1o7;#nGsP|&@FSi?duHe_41>dpAidszzPdN!TJtB|KX@) z6kC83SpgL?7yt%{SSI-W2YztypfZ)ZkHF%;+#WZ*I{M9FF*FANj`#YVI)$rcZqr~r-3)gVi|ift95^d(@; zLE*X1FOLqxOp~FUSG5J!NdwME5RkthaRbG2dNrfkqob=UDZ5#4P!N*cFGK_cp#797 zjIVSxAsZ8)^ipuh*U1p);&hT}otVH3)x@9@Zo6zKN2@_I93@CP9Ssi-uB@r~8xoI` zx!eBQQC+GI7+pNQCcQpsfkX(F#%U6NYRX$m+)_@M?YtU?CH)5x@GB5x&l|0G04aZM z>zedhGW6`<-0}YA9E!|ZO$AaL5c)hdKnKyqv5S9!I3m;<-r5m4IwZ{B z|M`MSR@o@X8%e3DrK`q2e*B=NS{@t>yJ(MzjScunCLtviL-s?rrXYmdYAg#bd1~%x zk~A8gh+}arM81Ct;~(s#D%3T&UoKCaDxW{j?NBzzCckS8ZVzY-C$m6*I=y#kU}m6BgzWau zHA>>Jeq|isG84G36^9vXmip#uKPJx9-}osZ`(jPouOou$_kTP9LzR=vqe=6cvA86Z7s`6YevPU7_{wy8LUrRd^q(BwKuCvwzM!d7^{*3eJ=us4$-;GVY3>U+eS+ibE)A7w+SQJVj!9j_sfSfx zkz{P!B5EQ%VXj@*8RMZ_$W7L$yz}vmM!aBzRUMgr$@`_;4PtQ28*fUqr56j|wJjk?otdPWI zqdunt0e8`02qQ*hfqU(=Dv5%6<_>%4dO`ksAL=Z(tw*x~+m={JR$g zR`q3bCYGkBe!Knde|LFFsK-Kc1CTl~i=I4o3os|qWeDrBRdxCf2FJ)SpB%d@7^qrQ z$RA|iLb&irTMW6GI4|q4V@S>bMx!qQb?x6&0nhr}H9K4O2#Sr}5h@Lk^*KL{Y+H^y z>U9RUnHLmKFZCrcdR`n`aJ0YutK*s;R%g*+cZ92ibty(tXrICh7I~9NL*vbRC+wT^ zo&`o+-+p6|@QMl68l(U95nnlc5)+g2=^fkOV*>(vTGiqwzkgcRca8nv&sI$YaK}m+ zlGD?D%D`RqcYqfxJ7ZNjvW z2|4_7v0xUvVi@d8ys0DR=Wpxf&BC>pecr(vx$=@e%6YMl zuO8!S`CWMNOB4%EKEI#*G*;x?m>7w@vDLZHN^lC%M;^K~SlwKm&-A$!qV*Uaw(Pf$ z<+uxn;Iuxpf~=~O8G$hi7L1DgO!Yo-*_>w|ndGItkr3x#{;4`z7AD|y9hpF+V_ttx zdH{`3dQCVa=l%|z({?R%oDapn&9`|r!1F^GnAzR#QTEgWZW_g07>~8uFwZs#pSzG_ zKYJ#7gGR{vQSOFr0L84|Le!LWygDuoWr=T-rs*3X1D+Fo{o|r^=(O_Lmu6l0+Wdj@ z0HT8~#X#a$JTN`BZP*JGEBz6dXai8lhb!6^mgekKMgKR2(WKm!GCntNi}WR&2OhXN z3IEx9D1=LuU`^gXPIeI@|6A#uWNAH3LCRKU0Nnci^(G7_0T zX?A)JKa#i)HqpP#UWmZBuWx>xT}`E@nN<)Gt7gdw^bF_GLdQ@GRvSBc!+7L(%3jQ6 z9Q--}OD*m6XtV}GRzzodJ$!EnxNV=&4H`>X|93WjzqhK}_1B-4ZH`D((zJ)T((E6n zadSk@zah;T7B|u&>4>~hHUgP{{x8q$@7(`*0)Cn&MCPdu)XzWRb665LRLl{XF=exk z*4ipCcOH1j1lqw|Z_%1KkM?V2KfYal)W_1J#cwIppDW@qNOzVY9rSRs!#2|)@svCtUAoVJIBb#bSY{8WqF_c%p5(DmtY@i(E3S*QNpAQ>i{2oSY z{jO+^{~?hQ5qEyw7BN2JbB;^Pv|3GM3(uB8`f7gFGB(DN*O6SXWB@#huH{|TNhv6Z zhy4Qq7CbJ+3_6W0%3mIZK!>-TD;=UB9UA+H8Z$Qz^5 zU}7Rsq}qNZEf@K0T1@ZxOw-$4KHZJu{DMW0O=N!{n4l>YSqcnus`^UfkOUvT`p&F# z|IQ$3MT}EE9D14-$dadYO=cVWAe>uG36XUBao^aduPH@NHtJuWQ*Ziv-fMDanH51p z4ZnIzD(Lml{n5(2@N)eo%?s>J?@G}OhXuXt&af6Vy`IIx*nZVxnIJDuq! zaP^B>4NHo>-66QYQB*d-D~h~LXqpF z%z^i&`G_`Rj0cZ z$&9@}o?l&NiN=jLC2q#sqdvINIh-VmBJfLxhfzf1Y0@1FgmMyh!7|#?K)2QH>9yFc z;>-!U`@0{A=ZT_TxdDa3(w095cwS5sMvmGkHp`wCC)~$a9TNhl-`XY%qjI}{y(p8( zrU)Tgd{RBt!k#ShTv`IlRN;qI7Q_2LihO%M;;Uk)oDc%xg6n4h)WGkLSRu0@S(&Qp zl320dmzlInm?IY2+TYchbAhb?Ga%J8?Dnc8&n^esUOLl&LY^j^45iv57>jiLv!+Tu zN!Q|{UZ+396RP=jPb6t|R~K!@b;5FyGtJM9Y`@1ccpQZPX?4uadm9Dh68Jqzc8YEM0aN0GkH9Jl1L&by{#Op`4=>aBPdl~I5AlNB zhdcE9qooL3;wh?S7hqT{d;k*dC46JPyP+&hsMFk57VRBn|&+ul5bL5RIrbp@;ZW@Ul=*AS1 zF4GlbscD((er7v*p8ZkEKr`OEK$a_cJTdBnLaGlW0qVkDzViB4Nq6+$*qBPY0oL15 zZf7RcSMP=tXnQ}_)O2M|$LpdzOPa zNYb;w3fdtk%ILv*2>lWcUhzDC_5jOirap4le~GzZN2X7onVOAXujutwdqK{_w)!5C z&)t9h(U;~+rdeCFqEoEr;PJ8lJ3;I8kbd!ri#Uzi6vV;E)yc;Ntrp??O5^uP#HFn3ww9Ow0=33mKv!4^W%QxSil|9FP0Z78xK!XlH;Thj605c> znnA;760YX`(5G7CGO1fm4HsbhURojTLy@9qP_D;KJe_BB?iI%P&+EtlT2hDPI;!i= zGN~OpjRtzpWC({4Nh!#ox11Cw1+@wveq-gm=I#4zO(8x#jWD|b$4cBD3yu%s3Th0D z_CBQ*C8t$0t4ocuYkenmq*V*u)4e$Pg|f11PW={B0)_1;yD)K{e|o1_8)>56M*E=# zzM>9FgWxCQV(kW|CH`y`nVEg_U%E~Q+E*`&5R*0_VME|?ylsdsaPofw_@^%S*(%Z5 zqE{I{@1Tc#*);3u;u5?%h6JX8c;k?q1eYgY5bnYn$CcQh8_Enyjw5aQ6y;55$o0!# zry#HM<;$ULkMo-+N!XHpo;o1p588YoCXKK7JSI#G-riC3{BGb3=XcXYNohrYGN@M1 zghu9dd0M{ixjUDrvSBOEDT{4;{bj7;MF?s=J2h*M<~d-HhP`Kla;GUb^LjU+JA6_# z3~j2&1&)CYwVuAF%Y;2)??u`uR?+j27_*Yk?j?H8NB;CqmZVO7^xT+9+dhc100BFk z;Dwk(!tO^dh!ZurS32z|M@4U{LM*dZ_s3%BRbl>u+?ub@S+N#Vp{)@Y)9?wm?M!v2 zSo7z%Rri|}S^#UL_>IIAa>`GtvL9k%CZo)$xyJtY@y=a$K9%nU@U|yaE-Qax@W4b~ zl>721+t&vUdfl_$Tky6aJ+*Ql!6mb^v0D0mAvJ{jX2x}3aCkIVgKw_b>rKnel5;EORPXx8HFL+-d|Vhl8Qv!{HOihd^x zS<6m(P7{gMH=k;>i)7QdRI)e{>V zJLg_qXhI(e&B-5eBoh&lgBFH_!5N!K4%1oyn^ZD|d}y2vO62RYu`K)*u3$_mOaGD^ zbNi(T<}+TWhsu)_^QhnA*84H~Z`n+7N+ul|jeiD&M_yE(U0m*WP%i1mgC+D`jG!t4 zE(K>SZx3m)I8D;)Ly|5xw)`-Imh~eu^)FS5J6drmBdI6UWXs2F7`43q*>w-m3LI(l z#HSoy9vSG|i(jHkz0#dohdIum%Y~vMxJiQ&>LlUPtpQizj_0D z(sOhFN|)bSRQL`(v7;Zm%Ks4(6QgNb1%VP+t!^`J)&6n-k%EHae)S|O#O0~F5ZX#J zbes3nIOs@?i+vOeqxL5Hl9`zc7E_LwPuFJE8R}{KRNz0m!*yOY!VD+`YaCBCNa>Iy zcraDPDT@`oIH0A4FG411`r{kVxzGJ~^Msf>b#n-_M#xMCJn7{xerL04Y@*!t?Z;PD zU*6qheINhNb@wM?Sx}Fbp-|sj+W6HXEt`^5I-`DSnea%SP4~!Ac@*Zr8%S|;Nm7^p zM0}TS&TH1&{uux0>=ePYM3N>kMJEa7|3q#@K3t+gogxNyLCPw^a-S)}&mDIH3;cC3 zJC5fpF(LK5{)D1I*T;^(Q{K7_r}J~Ga+de2VtFi>jT6Fj+&j#yCVc|ozZY805Tur; zs_DiURwiUOovg}-?Di+hlEdB$FRmjBM?t#f7N7ae3xv9Oq!07D^`FM)O?pp)7W=96 zV@n4Xg5{4it9LlQr0XMvCr%X!NNs)el1hMu(OL@38dL+7DRDDjy*TKYW0K^3H!*vt(f9291eL4?>>&b7$Pxn)8<|(=M0MYZ3(r{*z>(Q6z z4UgrM$CjF2A_#M;3M0!nikw4Vuq156o_e&b#)9c&Mv6nDDOaRFp1&U#MTy!^bx=vC zA5J{qvOzKQl2kJeF)ytrfdP5~fM#bH$KxBo_{&86604 zIHBaqtMhZu#y-ma{6zt?*va*K^Y8kEgrQ{}W`t&pWY!Y1$CRjnV&>*Fm#z?7!y-P# zC6ew|YJazgK5HC^$jgjyf3;!B=liJ6!#iXkRqTf-PWYWl3(63g>^Q`g7qT5y>Rm_H zE%a8IPwFHtZmj1sn!Z6EJxb}VL1TAMBJ7U4A#Zm58v6L6Hz{hMai~Q;1meY%UDYs;LGN>A8^}nN!|c$Hu8KYpgS0 z*zu+aX$>oz64+Oe%@gl^F&|`q>B3{FDM9{9S{pJsgKvK#sU;s3}EWg{7RGE3*%J-r|JvcQ%e}j5iwY%;< zIrMr}EB8u4yxld93?ELV5AaTuNoGp)3dwQse^M*_58i<(I?(FS6i7uX*Yz#-kBmJ1 zFo~xs|H=6ioU-OV=EFB>^5T0l60u?S^;frQg?u zuH}|ChIE%PRY>baPbB)hXd_GS0j~yr1=4wJ9Ct~rSuOL#yKOaEGP293iI&Oo08A5@ z@eB$I3j2bl{PAlZQK2kz>q#b63`z^smg^dx(1!=X^@O2wj8_Pc_XWoT#%IzAH01Fz z_qvIoZZJQJI%$CX#_K+9r6n?5}H&zsA3Y3rDxew)6(YfkDl}J%ydyLJZDs|sxnTnfK>4g z`BY~_#FwLjPH(T-9DWs)q2{T7=?OIH-SI9=M7$w#Qd0iN;}oNV56nzW@!))=MGIke zob&yZs?F0FD%855b`Ug4bBG_U;o^6@nIZfOs7Ebyy`3T#LfGwHEs%I-*!3;m4I zZDOS|7>9*nBB#nw&*159c|w>|2Y+ZO#P zS~z?}4QQtsBBjdB&yUd!YBQ1(5+2Z6J$1lHC?YiHk&Q&^>ifgFd!v=vVR_q9-FwWY z-r+UJvlV>rjF-J);M9`qEMY?^L;J@|eRaeBb%~ol3w)BA5pZP5%<=zQ&879601vVx zLsA(p^}YE_0nZ;3M}iqby=wRNUEI*zPZLSO5E}wDaIGul3Pz3nifHItMSehQ;I75L z`DUvXUHzDOPv4xx_Mb?s!$~!b%jBqyw*^{fYPTSzEL9!T?XX?83tTHCd5h+T>L4TX zpvRPzzBzV1g5a8u-~Ff#JUowXr=#a=Y=<)G)qz(LpUj@x)Q0|=7drXfCKqv~Y!kQAeo%Rzx*|GqHoABQTH581W%-Nmex4)*deqsuv<-a@ zND&M;<3JR*iEB(E4oTxVg0+^a z*xN>bT?gYpmaH1z+ncy&1W?ZeLh}aFXyi*ibX@NMd!&aZW$)N>zZ+Mjp#Zz9vWG^M ze;8>#a3NtzEGPTxF67?eR)kV>8I^F}lM0t(N4;WQsNMU-0JO*=Pn7*Xx*J8PP22a> zapHeb{L-O9kxp~!`E(1&?JIEg#8gZ+0r^yQAOK(~rYxB7>gXmUUX@4IC^h9fWrF-O z#jZUWhTYa5sUMk7zG&kVSjMjZ_irVHEc>6+J<5PM;}AfYV+YcU>0Yb^)Ya9=zJcUX zD4&$;U3sRIK>;l-6mqN~91xkFKkOagjiT33Y)7}VNfQ*gOk^ub?{*s4?($$qZ=CwB zi|FFSph8~3&+ooEC2HN>_le^w^;Wf&(-3KJYbXoex{PgvkrRQ5X(*2#Jra*7+)}Nt z?3KuAqnOACg3~*})OC?WD*mBoxL(cnbP&uq7DXZT4!kN(xM&a!Ld7B`NLZ%>u`*mi zLh+AHnDZxaYQ*M|yrh@yiX!QMxiM~=?lur0YO-62e-3Kbb4?Nvv=`@LDrK>^jMPaO zCCaqDPXcG^<7a;wanx$I^9s_@2=O z3HX~ZiD$Xa!)q-@ZG1JGl6^JgUzS?@(P8(P@~F3~G`xHD+v!c#pxMp zpqtP+2#`feJeBSRW$xZRwSjW$UR%5js)8Q!aQe8m`x%#^+@9Wj{bt{MB}+f8Y;V(~ z_$yT!i16wM!>4m1>OYK-@q<9lejq(u>(r9Pu-0nNZ)!Cbq~$nju3AGNDi_>tP;MFey;7ut3c( z9YU_?;^C*x12U$QZEy{hlCb)-c&%Q`@*g-7+y;ax96R^?@$ew^7E>rM$N&Usr z)XAB$@U&N-QeUmt9j8r7R|fm=o=o95|%?ixb=)p>7pJEJ5%4+e|W!X@>G|u zYlRhBJ0(nX2gTs1F=|!Qx8o#*@XS2P4SnT)_%fa(Xo7R6M)F_0R_7b#5*~{&|ZR7Nu1w}uU zlAAIJRp32S*Rj%ib|HWTI$v?o(A$=4)&2NoNByOF7R&G1vnHMgok%wQcg*O})rD_8 z8jYX-V|Sg}dBvk)%f6{DMqL||%_JGopM9<$wG)%6=Jny%u)G6V&y_oa=;YaxYMZ}9 zlkHJK)-#BnZf?cO9ZJk(Q~lrZNSPP-sDc+n&3edT(Mrs;VWX63h4dg#Qc}`hbQuR3 z!&^CN@%d>-5Khj(1!T0NP*HfC6!LhYLCyz@X|YO#^o8o6ic{VP-g7u1a%PW*AS(2O zU_?AB0*s6w{)wRkO-z()Yh)JwtfI=`LF0(GIMX!6|erKC?0#squR z97saHls6!DdwW??sQ~*N*1uXrotJ#`W3dUiHGzh_^ne+BlutMNAz4sDs7 zug+&)9EYb5=Sg6)&LRUDR{4ULicSxP@a22^jvIY){v9w)QcG)jPs%n0ct-L3G`J-M zt6%TE3n;ayER8oZgr{54GWY{s>X@ar4v8r|qhNpnALkYoEi;sUPaGZ3wEp`Ro5=eB zk#Ra4(^jInE@q$=tenyj$$Yl&*;5rm3@s``Q%#=zvBpjh`s;RG(axLRAu^RDRxSp8xfscrBM4wdm>a+X8w8?(%24{P??9P)&ZhJlV+IH! z_wPnqN6QnUrG8vhWF1#`CypK5H{oH$BVZbB$`*Qp>At z%Vv1M%WeNfNtRUOovnQ0!nkfAzo!^AG9{o~1?T%;#;)iZ5}o9n%mKD_{fgBUBz z{K7&`f8c+x#72C?5TxzyKB@Z$hilP7)yjQh{&5qjm^!qfZUOQtKN*fHQS!{cFf{m! z)IS%Z;H6(K54O0uc5~_8bX<s?%)q|OEe|;iAF&mYio29Q!i{U!yzf>R{!-# z&l&3mL+RGBc%HAJ6?3O_#@kA0X{*rp`*Vkmxo;vTKV>&&K%%?3-X)4zGcF87c#zkryZDnlQmaSRS+WOM=pJG(5CdNK!| z{!#X=JI*e?sDF<$nrKFRrT~g_=>`#qeGl(dm_jO z3B7(1Pq!_#hy*-at$naX%KP*&9WJWgf^Js**TlLn^9G3DW=D@lXWL7%a_ef?U@cPX z3ufB1!`<^W-pQSF1&IcQR!_tQDepT>19Twwx+uy*xrc}O|1J5Zyg|k7%kJNPo5W9b zlJjORvZks(yN`r~1pJ>2e3eJz}n@~{u- zr0B?;W!`Yk?;;c5`4oZMS4k|I*wf`&?z4Y+1DWG*t{9)j+mIXn5-L@>32HV^Y@=B@ zjW{VCNgNzy4HA61Iwt2$OjUf$imHV@co9Nnd(vhs$1z+4VY$`{CmN;5zQ*lAmLC0t zEc#NYzTtGn)sVxmz7>*|86R*)(0*fTH4gBCLm0H#zHUcKyA+I`c)8-YOC z*!=yw&F9)9&Iih_nb;R317o(~ycY>^JVdQo9OlEq8AV7&A@g0uc@Eca+V^@k86_U0K?=#fy(=Nq z?HCdBLiyTx%2(xq{CHy?JUcWcPiIISQO0Q#hQiMZ-tnq!ES~NdR!$PLdrW+U#P&HW zvHYf8vYfQ{B~gZ)Tu=PBzP`RncA`#EV{YC~F?R2Z1=l57WW8aEc1yGdynxzxq}+4A z4d6S}(!WhTV|4Ob+1jFRW=sM{Xl7$Ve$L5f&&Yh=TdrVoAYA#{vj-BOb{a}WxxQ*2 zVoF%90RCtAfK1VAFmx?~w-6qUvSZBXXFT6*soHWjr<5K)rjxgh!GI81CS)ThWv-1e z?VCJKEJ34j7x3T3jNzllr5NtJ!~YhlUmhf=282I+-x0)s-M8UqfLI>mc4i6)qiQwl zpM{-!TNYuDc$PlN$VB;i&%dt6TN0|N%XoZiFh4tL?G!?1rr|V^WN1IqGS|p7aj13F z5t=?5;LCgU&6nnEGx^?9ewS4TzLKiz_#QbASGI)otrVHhWUzdM? zchC`h`+I^lVO@5Y0F~qB=BD)N+Ef>(=O|ras|L0f(e^O#5DRuugZd{o3f~)89;ey0 z=X}F9+Iv<}gc)O|Y-K%Ew!tJI=zWpD?L3t)g}um>@r97riG~tk1)HkWhVV?m0|LGP)Er{++TR;%Yk3SW48L57 z(*x#7qsID~(*vcJ?>~V2zkYb1_Zw;Mx}29y9#IRM&)Fn2RIjr-nEfL>voOrw7Ul}Lj@Nt~KC ze7FjrTgY87<^0C(e4Gdc`A8;eqDr<`2@I>YhZ1-Lhk2P97-aLkSJ!sP8ei-}G%Jn_ z8OCE9kE-Af!AX~vvZg61{xan?PTsq+59y!kkeuhNnLhkluD9*?5azjajwU{pvl=GyGfAKJ{puwhWfFMvph4D;)Vi1;nG^Ypl5UVGmCu~jnt5naA+{gxgKvr+oK z@>Ksn$qN}O(qj_t;?6b%!#8^nk8^N9x6Mt&FAU<_xam*hwo9t1ENLnL1GL)WeY{C` zxq<3IWl_siy2S9=N3S+EYC_+f&)9oFAmI?vPkoV}XLL6`O)_HA%PBL{q>K1*w)`uP zV{>tdWY|P_@aN?cuCq&fB;Kq8jan&=-D9~D83?Y{obV*DrF)CoR|YRu;XIWjf85&o zP|M$PJFZPyYWw;7|FYxh!?Q{hb3`Zko)(&Js@DUOWIfE-)X*l5-Ze?HmDgpj_`rvB zS!4?_K`Er8O!s?&JXWonn1bnl{V&%3p&d{k5^#loqA){szUuG2SK^nE_>O#zP5cnA zwx*h6=;Bod+h6$865VgIhl{60&+a>U3fbF0msNHTD;o&{{9lo4%4a2a;}1nV^y99P0KtT*&L2| zS`L$bIkZACsn+SoZ^AIX~L@X>~a;j zDt}>@epZtBagn2BB$f}!yTqsd%=gcRfc8(jrJdFT6=lI^vu~1hcqHER7#|oJ&;6MD zo2-%yk(Ry-xs*q+{zHs9Yi6NiEZ+S&H`CfG4btw<6!zXKr=F>iu=7fJaM3 ziK=}&JV`VhJ2PT^GQ~P;+1H0MHarrPS7WwI5ETuIFD~21zZUP}k*M28>#zsN`_tB8 z?7xO|C*tGIpw^B%Bd-mG)})nAbo>XD=`HhR%>+W@w|cE^u@2+)<|CrlhbOfS5^8hX zvG8USZ}E=dRK#YpqTOI}HeidJ5c-zONns+A{atpp-cK#t8SA^PNhrrZ=Hh4vJc|W= z`ag@Bix+A46KG{MPwEW}44&HvjZ`yhuegkxLF@clUJSZ7vph5-W^VgVbUIs z)z*-o5X=1W;>|~USJy0%f56B^vw=O+O(dpRnYLS;RZYx{4KS!=&Aiv!N;W`)U2MW0VsP{Un77$6yDmYs-j)X$@w~sJ_O3q^bXQ zZR-UCO*sy23lff;e#n`jfx`W=C06HqMtXV=W?-BMwAs%Dz22_r{0?K@{|h!k3F>^x zUFPdYb^K+ST?4)sn{$s!?DTT-O!0Zg3O^#Pe|+5;YLVqV54=5zsFx@{7v10xf}w|8 zTM)*PbA}%D9Cujrl zTg8V62$53%_}1Mkc=eOgP7a5q6Ry+gD10YpXyvT8Sr=3Jc5An`RLMzEV{69BDmV$J z6aVftXOac*Q|U$hD*bc|FdcH+?E8N12J_k^QemnuQqupnzr5ArmN?v(LD38MXTvFH z`cHzg8$oZIz`bCp$+6q~l8#%{?|Ig58*cIR{FBrM{ zxN+to(DfOvIY3(iX?CP!$1}*%#r&?l2PzjT~9-pwh zH;WefrNw!DoGs;FJALLb_REmqg6k*ZyC2ifW-QdoKeZ?4h6i2!ge842Yg-dwN4h4^ z)-U)`J+I}UDlWbj4W?2oUTzQnaJ3gUG{F3i`?lXsFssZu%jq%jxz;U%OuD0^KCNNU z=jc~MmJr%C4hQ4TSlPSybEY@34e0fF;G2H@?*=#~3E!034MN?79r6SB!*ZsEhwkXL z?}bprFl>{q_F@lO*$VK~7#>Rhvhx?>+IYHVvQU7H>F|-2^sCYb<36gNdRc6@=2uGK z2G9LKJ>Qk@fY&@&1}Q$$+6!t|9W9YVG@`~jNe_E%Ow5>n6bbrPd&8d+qLp&Z+XA<^ z;nQ?emC1iEb@ZqD;k$`bnVVj8cvYlwNhL8akrm?p+#Axd!{oERBNy)PqW^FNYd}=Dc|(9ijy&a$9!gOu-+J!sqZCM8QGS zeiywzIi|%$`)8s`sulONDC8@Jr7+!0mn-YVwWmKnv6^f=l5t-}s;OPl$9S)%x1UhD z2K!k_8Y#7u-z`dBdRtJ+{jyj@^#uH?k-LurBFK&;`{%S@D_M^xK)I~eD72nWb^oCx zXcx}ENf-h`bPzyG%-`&`Y=;&i%^)A8iT^pqZu9p1X_w|^^=n&(_{W5wMHm?^P0ATw z81wDZZ#4ywdgioyP?4FzK&}QFWS?rp4d#*X2Sq_sTi4rvVVH zhuwqsV|zUyede#5NZ`5~Gs~!jlze{ca+_|MUas4cr2RTFdx&H6fgGGE59s~p_Q7M)z=dv4<(R4v!TNbaID*zc}ImYN|A}R}s zckuHkkxgA*6-p9wZ62B~wzS@US9qJ-;gfQo6F2ffH7+ZY0jZ?f(VBaf$6>8iIY|>w z>&r(IKUbU5u-7XnMyCj2wc);QkJ%fEC2^C{lIQP1DF}H?yt%Pa=X>i@`aLF|$av<% zx1hU-ugf*9t(f=g(3?{*x54fgwu*8Zl<_Rzn@2!{=;?Cc-;^7lU(r!G{#Ry(e!^lL z(<;y+y8LU$S|o(DBI@H-3qe~meeKpqK72{%iVc*9V2+GoWxvWYCN|dsB-u+36ZdDljWXm>^aRt3!c^1@NjaDCvCxkJHDH z-}ykXwy`J`8H^VjJ{DnEl;h`fz=3u5@W7){^gH{hDj!vy9G5%-YE1DnPStw|!Wnh{ zLcY(vNPYFUK2pEgm2LcmH#*v?=h_2#1(~O3@6E5`g>dwDYyRr(%G`|Q@nt~=EBN=T zFZLx5k4Y=R2-U&95R0(^X=q$vnnu%mE;|jmWcc*RvWsl7lasaI@J?jF;|}{nv$7do zwD7F1BeWW9ZF0HRYgOv4XQ01rhCK+Lp0H_V9j2mWg2v1~GNuHuo?GnTXkcIh|EOdV3>?YiAC=nSl+F;}|9Ea|sFiD&z81eE{NWdD*Ww3Y` zC(S^Ph$bw?NbwaRGN{igmIN#`;eLOhEvLi%@DJ{jo$*!3JrVOdnZx!g4o@x*y0&v^ zSl7d@+tm(%K~}4Qh*+D)pRR5kwWP-tj@+%br#qr0^9EGbmO9W0k`I0dQ;CNel*lRV zTwN7KbUUm~L8x2gg-qpN=iBcT;6tPQvNSdd!O|{m40$k$64@ok#3%x4c5%*x+d+d< z)i4#~fvgi^(*>we@L`#tLeqma762r~S@5sDFV27i(=%A*Jgy^cgUQD8XGx-p7~#Ju zwSQYf)lZE7ocw5-hIl-|=~uyd{vXUS&uaawk$Y>suGY$+Oo(R}Dnb?DdxEQ_9(P;& zMeg{bDK7dqAY`Ae`Uud`2c6n((e-xhf%m~3?JJmOM@qHWnB8E#?P0@8dv;JzXSuc( zy|jXbuX$GHx7cI`O;NsMCti_X)|o2tq}kXXp}ZQ0S(&!4co%{*f&-a!-zPUFDymIL zNQ1R@1+ORX1Ht~%>o8b0&h!J*I+ppw@>9DmSMRd>9}BXKt5CMjnV3d9IL#(zTBuvO6T-8paN86Sjy&Spb$x}SX<=wF)l z{?+XR0X6&dVH& z&2ato3#dcK|G8eJm+OC^|0aI$7iW=hXHs-T(rvcTX|zr8ZZ+gLVmpt|Yc!$csWFPM zs$h}Ap)(v8E5X{*5{4mecDpMpIr}B_jK3jJl>ZQhqjiDJ*u#H6WZ5d5t7Jahz(YE>wuH#k5Uw9x zyuJ|_;&St}EokiXC><&a={V5i+8y5;D(TI2zdaRb2_lWz8vAZwOr5sAGD>Y9B_FcQ)xD$1`3o$2rYx<#-2)VwhI{S=P)3rx5V`kVe`8sXVLMumA(2AZ!6K(xu zeQPUufm9}EbPxMOKtK)?0t?;ts|?HZUUnC!iUS^T`!83IhHU>P^)n-ODVuOS>ZiQ8 zvTTnS8F)?M;iE;TSi2R^RbYG_?y%NZ)FW}b+w>n4X4&b*?k|ipfUr9z934b8%rO6c6W~FELuxoX+9AV^P!PY#$x3!bb|yRLN`O&s<-cw$eSIHVG;ed+ zH~4vmy(7_LlihJx7Q4~U#uKzvFE_Y#+t-CM#^a?a4G6yRv#uGb>Dv*%-0DSsZZLK|&r?{z zhtjV|pfj_O_GUiDI%0sAy$KpN~Z1Z($NPg~l4fJR6JPwbAXS+!RX)Ks8 z%iZ5CDi#OqM%3F(Uq6~+hA)ddLuYS9m+lY0P=DLwB4jNo$9UGL`XSfP>Ic;GV{HI3 z(G)~SL&|&ooxv^&lV?aUy;K%%=)SV8L-Et0d!=?aZVo~fGtBNj?9V$Moe}A>4UPRS zb6VJQde83jQpoezXy{GF*EZDkIyFNA)z5F=!z_mm9T8XYrIVQ01NT^^I!@s4X>vRk z+=2+USr7>Th(iw0{AzQ0ioyDHuFY6ZvNlMHzgoE1J;jU}xmg~Kt-r{XYctTueTDbW z(9Pi|_S3WfLJTXY&XkjG$5$4 zYixXFXSWYFnw=h&Wf7C*>2A<1Sham+G;NzKlq1W_5DWr`5B6(F<>z#(Xz@+jH##1>8nP(K^`Ujxa08E4n2NBK1gd$G*P=VXUMifn zhy5GS3q)JHCLdN+Re>3OrS3amLXV+-V0f!oXIKf;x^e$O`G64fGQot2hfTDdt>t|X)+e9$VNO<=08jlYc-lQjok)2m%f@vu z5_Ubu-v5v3O-?uEU&{p1zbRI7?LotZIF`j<`4P0%;B;Rr7lS%q#>(V{bvT@`?|T)r9`FjJm z!xl4T79X11oiG7a4mnaxcQ5W>MtBF2nE#fcH_#&`em7_cv^?v@-E^;G6{nb=+H&6w zKo4(bK{o~#UQ{2KV+ZWRyE10Co~~ShxkSFSjj9N&et9!;df8& zJr|%^RmuD^?i>G>G%f03uKdu-@8mJV53CVvaBjIhKBO|wF;2IK*TzB{6Ht|yO}Lje z9{?Q8OrM;x0BEtToLNOhMXU{vY_j-xcn)TbK}(+HL@!3IGklDQ9F)NUljRm?ESkUn zO-NVEwSC~2H%xqHZ01%H`kK6*_Qb6~wTqEgY?^F+<)t_*@b zpWlH|?AG57+!IPlN<#OVz~EplEiH$%&YwT`U}x1V8+xNiTllbxU?THyfZ>1H`xFfn zozqSQP;~~P_iS&Qqmv=z4?ydsZyj(^4?rE%%4gLScS;W0OEMz4Mb0NCeIfYF7$1Lb z8*#902!tcwq22sJn+i~Gn@=Pw52KY6)CztJF!t3_0qQ6YV*&>Kb{+HJd-xtL(Ds4O zR|5&YR+K!^S#W&i(V@W6EF&-f<_zIK0!eQ&Y)(7lKAHFbN{T4GJ)FHzhxoHcQDWX* zJaamepxmO%T8TsiJ`RQ|OT@f}TbVFjW>At;mc{6(R=stvRTjXtEUx?$B30`Qg_&+F zKJeW0g>DAG3p@Y4eEISY?7{;4p!*?xU;?~RzwOZIhb#D!f^34E zD~MXbGKZI2aCv%$6;ULc7Q7t+qD#j+K>cMI!r%MZHGt@&Y zf6vZB41wgsLw=(2v)|7S=vm)tI)>T8Y19MVK#!Ovgaxku-I_aV0|9#8vcX(aTf%T?wm5;1U9(*sFKSo#kD- z{B4d@#jSXMF%{SGtD!;rJ8#rxd5-R@v18fM##d3OD;6Z1P;QVelC(HSx$A~1CEVFScUxh>TF zk`tK2Y?g=6r@*E}2jby7)+CAQdlWzy#~)K&bz;6|Pr64Md-VrU+|;KNbB>GVcyKjC zv#~T(pQg0b79=vMwTku|M?hP7DEk?FP1(kG7~1$D+OP_R!DWqqsA%xd6@)4(cQ1>n z6pW70|GkJG^7HfIi9DLD1ArO)1b&~a7Ep)uWCl~%Q5XSI=CO87=5uAU&mQ`D!rcAu zDf#}B%?-n|afOk)?^F2VXQn_}LvFc#hhVwW2pbXX`ZBe#8Q=?P0?kJYzR6=@(RId5 zo4+RFXhrUF=k`a>D+KbGKa14BRm2KW|8j@$VluOEhCjXMvXE!V`;260KG(<$hE-zy za7W2Y$$a#7cA~K3A->&4)q3+zu!yQnkTU|#roY^^=z!2=(d+PmYWfO5oxtF+0>C!x zL<9VE^J$cZ|Y1mU?o%@lM_ z4FEc+1(Yf10E)fmAo4U=&&0)TI>YHm=b#*H4u>%mS;MTKO_j7+zY4YnS;Md+XTwoa zdEC1cKXyf!YB>!WjTv1#oerOjU7xsLhtOAf9&d|a7sy_K(RdG1PT(}65$}V5B5cvX zyhZwi;x`06axW>tEBkCCM!g9jwjJ-y#MghB7)oL{`tfivB&Iqzz;uw@YG83ObLxNE z`>L?2+PBY5BMs6hC7nuxbV*5fx0HaSo9^!J?(QxL1*9Ye>28n|BN^YZh7hP}+e11K=ftR?ex=}mcKh;D#>J4Gkb2wZ*;m(P%G70?3@<)Sm_ zx&x~?G^zMI5)j_P5G74bsZ4Y6o?24Y%e{f9?YAI~D3`H|ijFQ;f#rL?O8s{>0;K!VKscMi<27ak23r)cSG%OEYkJMB_OnTw{%=qs zN@Imqh0<2a*Kq*b zk-;!2Aw8`MZd=8cx&RjO492O7Hso@te4{Q!;;Lx1VGxRSa*-)CH1jj%1RU*0z^ zVFh!&@0v5ZOF4P9q5NFX{v9DP_ zb^OQ8kw%oobh+g`8_IFw}d2uj&^}%0Xp`~E5jY1>|~c^{@hmg;<5(XJP)si=HiXt zDJg_kGh`_|$)`ibW+1aVn)$z?+hY(Zdi)X7! zD_JU!A56zrM+&?F_C^Tmd~2&)AA7_Yf1jgz+fWaNSTd%6fhqg@!J}fJdmeV*W%`9* z#cle|S&IM<2rh+xU0#H7PKYW8%7DP0$sY*CT^-$>Fl)ul*2RR+@&W_k6H}iiwI{vo zLUW_mJm`nZQfDtSjZtw0&!C4SRr`5L85di+y8|#pjX{P8AAqbtv7v(O2<9S~$!`}X+%Zkc+w8cHy{yAY16CNey0S*2&y`u1mWjiLA7Wa(kRsM1A zittD2?6IMpi}K1{cVK;Dy=esI3JAS@eNZ)LOG};{vzzbMdW|g{oBG_`mqo9%trO+< z!L;^M7(wMaLZJKwwF5~Ih1fKG;flf}r6?D#js(kM=Zp@A1%Oo(M zO*63$8i6AWx);~Cnp(ghr7Z^F$VJsBNryq^0)Od?nS{w9_qM3<5Dr-Q1?3^d5TivV3vnjc_*KQFUY|oiCC`vC_|( zOvk?zFZINZO}S=g7v;op*-IrfD4nay58M<wV9s^0FNW`>D4kaT>^#Bpu9WDmKZAFOt+Twe^%oh~oofrAX< zt8R~^)$Gx;D(1%<2^cJy@qfN0Xm5MvHJQkt-bRhr-4LTf`fD zwoR5(L07F5Y|Ns?lB)2LB!P0r0P2@>hZgyL8agb31d1ZL$7k(@jz>wTb z5-3_6ht+ynH&ToN-LfqEdwCGdnTJorX@zU8^Ooe?1a$#b=2lgPm9==2>!w@86av9h z?0}go4ymgmx}(a_P>4K;&<7Y^kU|JsEvy%3d#$T;r5tLDDY6r{9?q64#t7&KqTqov z480V5DJ788Un|m^(krXZ&{dTPSFbwc9m?`uZRiE<=hi@9uZw*u;k|6_#n=k|yLA3%Fm=HN5tAjA<#>a7!DmZDQ`#3L^ul{8dz}jz&kKOD zVlOT1w(n|=cv6KIRp!8rycI=HEmeCI>!G2U?Jy1z0HUF+f2ROmdxkqFs&l>GE+k)M zvgI2zbqNFFe6ZBY3iqY+*pQ9YH-kDB1v;_GlP%J>9utIxjp}dba zo7Ef2HFT)GvmD_5;&FSF1Q{FB6PTWsY8;DzW0#ZHe0kC#~(^`{~YyiQ1tq8?{E=))kB6RU3=xwyCe;8k^Gm}MDG65I~nlL^5V+m~Y z!&Hzc!cf3M2sfR^Mvbga`6c#sL2xdJ|K)ToZ+!rAxj`j%x=8wXuOUBZ7jD+`jtlRU zr62)Epa`dg2D9x;-r}@}bC9g+AF+=XV(cp=L3g0R)Su-6CNEfph%bb^boHa!5(LzI z2m&ukGv05--OR`LeL;he``0~jHKKTjiEFw_u7XEWA`I6ks;(C; zPsL4XlOtX*(@Qu3c=e&F@s{Z8b)hTdW*Za9I^Eai(xR#sj(hFX+1f78I!*Za1byCxM>?nlBv?N5GP{2(em$o2_W4>u{2K(eNJ14m@!kF z6e6xbq>(v1ET%|Djx`woHI$mIHeluZhI7nWZ1A!F!eRt8CNvM;AFAs)O+FUMa$em5 z>%|~pnAZ-ZLS7>dHn@$=MI{EG_8L}amb73L7urSczu90D6Q{j(bvsC8Ow(9}p$(f; z(9hHaK)Pm$WtS8rM1KsUQo?s`KGl`fW@~s$u(4mg1FMxD)z#Hcf2xr2+0@p{+O^>z z*z~%h;{q2zYo2le>4!92Bg|S@!x4}3QEr>>!h<*0s=TRA$!}9_mfG;x)W{eD`_qnd z$l2>ZK+!E#Vmo^Q+KXHz1u4h~nCVu$a(gDIqhy}E(x86G4Df+4wPm5n$pqHTCto?A zmtM>{))6urbiFV}=&=Zmh%kMJq-rlyz!&MTHf5W6$ovE`!sv;q;JX7w*Wqr?`y(96 zM8PfI!ZxwG<`T7X{C<@49taR9X76?!J`%3YqI@YI`**2~ujpOr!oq^&yLVzOS~vs* z8acvloJq=o9}$#4=+`4(#u+?^1vXI32hP;mHZvuX zc}Wjp*yvlEV!$+8tOaNl-||o`$s%!wy6;o1~=uqa!fZs^`KZ}0xnF_uJhw2 z6m5T)mnDWY&$nu0I^~~8Xy6-Te0(6-GcF@%4kuJAqFvsPf}7=OpvaWoSe0_Qbd+aR zu{Jlq0^r&~-~#v%5*lh$)MUNEW=tmQ^nN|SajDCbgi;*_FLmsaQ>eG5DUpiYQ>KmD z{pFc}=ffSZIGRuGm*KYynrX3ryM8(^%;V|JM@Z6dlN-!%SkFd4W#Ye;{w(x1)qhLl zDZ~2grk8($Otn7KxeKIT=Ki?p#BajJYzZIPV7_#yn67T8thiG5U6FvE}Rno5a&c#R4D z4+g$|6&;qzjZgzE+jZuEC*jt~M|WAWA*t-RFBEqW z$g6;|mKt4DfOD6YEdDW*t1lfuRp9+-9W(WhDG6Ri(`qiDTa64H!PmgB-p04Z(MpznuVJq9rWB>IxBdp z(W9kfowkguhkpV|t*juuK#wFb=_hO~!e5|<0Q%TOb?OB`+;|db(CGPhNz#JK1I-Vy zM@qB7Ldh*ES+})ce>2wgzODjQha3do7j^2Uz;~eTsMe9 z0xJv!80m1!Q1s%&SH!hzxOjL7TIV?%ma~EhoC?O1*-O<;hckoNL_^__*EpaqqbB*2 zz@Tr;$86*74W-`&Y;S7`x2r>XYqDV*mN$EkU0uN)f-?{IF`OfCOK~YFgM2TTi>bIW zxFVCu27&?uMHczN%;d{LupE7;wD(>O;^EO#dh-d+5bd?=JVTsga$%XcT=5TzXQq|N zj>#XHatqYz4#-xz>`pPx0o5(6T_t zs0EmC_*Vd*RdqdkV`6_nw~35wz;77C zhQHsKt?}ctpo*u~qFfGj!4-I=`T_#06OEsU_M}%vI0(j~3`uvNRk;tP{fMOIO2Qcb z7`~v%&;n%Ptg~N|GE>vsAH`>`X(Rg7t{r9wp6X1~F?1s2iRsHue^rJl51D)KP^!K= zeBYS!4Rm!VtW%KMsQLKPm~-*lrAJ#sT>9sK-CxcbU~xwyVUiAd5_MZLyxE(wBy?{F z&=`@Y^FLZBBqN@qH&Ap#6_-7~m}~|5>OSCS0?+XZPhBg#wLYyS@2evDpIsqyDfiTU`--#8Ps%xUzlAQN}jpwvY&45cC;~G?-!) zjQNiC^r04YV6QZ-1Qn93o8_8}ayb99rO9AvZ?LmIUAat?56Sx4KRZ>_p6yhHhKBl9 zKo#4@bqz2XKU^njFP9N@S{D`z0heHAX7qK$9zp=CO-8??j4Ov9AJwZj`;I@T9V_TL zIX96gE5mFzV`r_KdI1U% zhk6v#pbKW!d^##2kaGz*-Ht#{(7!xO-J}n@?9Q_}0no22q`3 zXEJed@tU;;Lqk33qRwvnHCXt{1fcsURd+g2rWECbD@-IU)5rCeh(HGi#s@R(JF=0B_ zki$s2d2(^IbK&aY1iW|ora8)IOnGsd3fY2zA!=AJU%n^7soUP4Z-{^|Sn2#^nY04; zY!&iaGzg8Tm+ed`4TTekGE}*@K&RW zB;76CjY;5)Bta+|fj@jY_=Z4f_gDAIH9_OW(9F+Cer|rS6_~d?Na&00hkN=;nK7E3 z7+YE0o^8nvYH;d4<~^m#?H8q{m4|vZHa5m4CKLu>*|$Lavf+t5(p8%_v3!dAdM+o+ z=xgRffYhiNK8g=zYf1IAs&(ZB#PtAF;4?2x@T7t>1-F^>YCleJuKVDlZCcmX58!ix z%q@$9g9G@swAQs{Q8yBLf;W@<$4no702U~@E+`yw0Dh23_@=2^vtEFIp1nmJhD2Qe zUW9)?CN567NY1T{3`s}!ch*k)`f2jzN!iRI<)*F$FYD&1$bWkcX@ z(rRZW)>SB&%;wI)a1RHA3$Y3aAu}j%2eXB|Z_P#Cyp1g~Lj}Px@vX38-E9mi{uW1a zqt1~70uTzl2b(BpG@m8_gHgj>jwYay%4AeSXuTV^!f&>d(COFNDbR5-9V{b;%c`$C z>*T_<%|RO~f7{G;UE9c|;7UuU#YRv63=}CTPnsFOSTzr6hLHoj4DvBT00v*UXrJBd z?(?jZ;7Z8ZpO7=#S`8lW2Yypfn;71m_F-1AHkIm$I+<9Cw+|oQ?3oh5j^cxHk8$N( zd+Q&W0;L7&tGk3sJXnODLFnHr=A-AeR9xHxNzNO8dNJgRXR7o_@JQ^XW$)3$4L&F!*d$X z%S$>NHO~U$8CBQrW-BHKfd2})44B70IHr9BLSP*hJ-c6xwaBAvScTAvM>as=KEYy? z$Z*zk{E{?p{`cWb1C#qjXi|3xUU~W+RA&ch|WdJh7@M$u21Xa>px=GXnD#ky42uxWf_(gXVkiQ#k0pazC zsJ5m}i0cAZBM481fslXzi5Il~;1jm%!1SoX6~j(LYt_VZEywN9l0K7~y21yGGf4T% zY>AS8WAVG_G}z>a&EvD;!p)9^-^8Nu5PK~!y7!2pGp9WrSprjlt@|`H-=roUw7K=D z52I`to6WCG_a1j=LL)S{)h_^8Lf8+S7yQ5l7=wl6sm{7?q}T2=G&B@MQiiYw2LuLg z4aEn8`4Kj{gc9ud?CjNYr#luIL^u{WxxLX-Y8DmwqVC$dJyo9KkM6E2(*-}jg@sZM z1X|7C%M_2|&QZwiAyprb$y{TB@tOZ0LsDc6hHlZzUs~LH%u>nkLD4fR=SX1?Ih*@J zoeA0Z!TYc22N)UegMR#q3@QO;yyJg*T%l1>MfGM}S}oHI@|+xjMN@Q(<)UxiLLYj# zJ}UlFVg!CB_>45yBF-OmgF2K0LFb<9%2-0LdlU0;iNwzzO0L-T_~8B=LpBJR(T<~1 zi+RMM{#!nrc!*qhg?3|DBeR;h*2%KC;-tabqGz(I5;5lxaYTrs$mh?WfuKXmq=i;} zO3t0!za2$V-h|^Hl)1UN6>_9MT185~wIetMo&F-8!$H^zzH(ZGd8d_v(1HFMMD+#)tT&#k}cnw+*gOSx`R8w%4UL* zutT!39&ys`T+XYo4#d{2@enYCU4>UCP{bp(U6ao1c7*ob9tNGfm?(y}qR|`_NXOtp zgzq_x0n;D__7jt-ETooj@~4MGNV8A1H5(Hj$AbG0YFZ2c?R(t<}<&6|Qw`(w3mP%Xg$0c?Xd%5M|^Od-cSm zN2agt2g~ZpY)5F-1)w}W33Ux3Z~La?XsQjP_Rkc9A(O{$ad+hAhrfs_r(&dx>{n2;`BuZe(AvF~SK zwU!sq!Nmy7Gml1{nA$v1AD zVNmKB3rh;UcHN~a{So=T^P5wd45QMrErC_`9+`GjJRB@&)aO7xopDUXhDvd!KMjgT zTj+irwK;5|^uwGBqrk(%pK`@LTUaW#5d{N^DRub$uXbsiWg`Z}K>avAUshY2xWJIk zv(5e5Hf;nrUh7ps%!p^y@sV#cIP*zVl(Cna6_tBIMlA}a2+R%u(dBZq@VnuC7e78| z>UQT8@|1&2zF~$l#|*kW8RK^=8TH}@G>xUcC@PZ6fZOx)B<=Iq#p1>-8%a+5-GWYy zsk#KpjK9pP+F{Dp@A~DDW`572cBJJ$yT&%zt*ok;79(w>$v6W~=nt69uw;s;5%xM;6X0nbCwDH85 zK$3qNP?g>RGF=NmOwpiBGi}rWhP6Smb1t=)i!(8T#Y=;qzW8aIA?1ID0G_?KsaCkD z^X~2r1&^VC?iEALdy(igtx#2swZRS-Q&R_(SvJt7lf(FB;Q?llRFsr{Nui4|W?6P^ zItO7K4JA#V25(89xW-@8FmGkC`P7uv)a#2mYT=po{d8V1HOk%VtV ze~2e2fo+ygLXWQkh8J0>m23c(<09Ug#O>jA`)IQm$wa18_vCo)c<~bF!m^k@9K;C) zA;@;2A(WtjL8!f{F=wtiG96R$#^25BV2x-EIy=H72B9zk5UzNGH%`n z!+f>H9pj)cpc^`|3c&5-py6D?qGs;T&?hS7%EDL6Y-RV}EoqFXt|Qy(McmEr)h}=y zbusVzftfh~QNEV7OiSG!e#{0)g1r^3Wrgg$wYA@yL(e(bAi{Q_M|-l^S`PqyG*-ex z7brW;MUAhxojr@#b4o42R2rn+k&^gzp)H_i!yuPb$U#5xMTY^48`Akx4yqa5CNqo=o>)`9`IPfzX17vo{Sa1>=f`g za26^^SOGMlgtr93`@QEA5fI24z>@ZzbU$WsbpxbIF`;82E-Gp`K%PVrv2XYI!qGX> zSOM7SlfnB$ks8>=S}%>%gh}UA?>+-%nRihyawEJg!~lZnS%QCD1%iz0#?cWV^Be+n zUVJx}#KCw5uOqAbJ1ji9_90Mefyh)97fF(+8O{ds=ARH0886ofEigKjT@ znNsp=jkpIO9xHtdP@WXk1=(J=p%K#Ji2xn-@9Y1yHvnk)pI`R>-++bqKV1Y1*#E2| zu>Jn`^?&=#1_kw@uA*T~5IiDckR-Lmk+feozqFyERK%xnDhe7l4xYjBZ{PctMh6B$ zU%$CI>y~F>sj{^t&C>6BfcY}lM`?mH;Ryr5NeSu|8$nal(q9ugAkc&3)irS42`baswB?r5Pe7xLuqJI#OEig7NS9dPS@+QbK2_0GI| zDNoezZgH|%LH823F%|$Y_t!l!mOCJjqZk}ga!g$0ky6`G{+$8AQ6b%*?t_ZO#@SdD z;wfOVU`T2aj_J2J@Xb8DN6@$|?I`)sgdU8>X5>bfHRmi%{&EFuat~y!d>kru*M7Z>u z8bN$`e*=&ydilYHJ{PujBX>zwAYs%y zrVr=LaPUf!w&+A0P@AbLI0+?x*a?2kCMI8 ze0cZ&^J`H1b+6ZK5*jy1jnojTDZ9M7bDk_Rr1uYDMUk_)&tBd zdZ)itUami<9G(kc0AM^@8st?T!oUyiP_n&&5mavPwGW-#w$2QJ&#)aSM?hM2TmUm?QJyswa1QItLKey#J$;iWy?#;Lod4=ESFqJY>%0VMO68iLI zoBcE)&jbiow~n{_D{Xf-H*X6ujD@9A4k0{DZ?{EcRK+#z6?3}DV&mJ!=+`KG^+|2z zQ4okd;rw1Tc3$EhZsmQ;j0-sMC01>cZz07!T_HnwpiN9!jc{%#t!ip+SJ^gex`!2` zd{Y9D+dhu4tNY2BYOk(0MH-7`;+Mj{^bS;>ym_-x3J)gw%{Vp?7ui9vtH70DhA3gQxK{`d@?Lqp;r{Ntj_iG_I0GK zh>Ua4g~>aa25a${=Kb;XV*wr6eN|MuPfO7z6g*J#4^iP~TXKCY`a2xy-}`wetjtLb z<)7|V@{Jl4Ooy#(tj~xU?WEX5&q+u~o2(j8)ZJO5Cg>^;|4S+yxq>wo@eQHYbbBG)k9#3O)wV7-cz(- zG?K!sJw6ZQk_)+?3~(rk%p3D;ywUCO+ZAHJOS6r7-;)Ci*^F__l9O%M)A-PuYit|TAw&kFbOjKVQX&WM}SIT^2Sk`ks_258C1ea?vvwmK*0nvO1y_j`p~6rNBb z%&H+@I3jm1FLuvXB7f4OSg8Gmf~`OPF9wSsZ~54Kh8tMllz@U+rtU=h_N@-F{&)sfjU6na2>;-novQl3y?~$4gu1^%@g6uW!LlCT|$x zmOhNiB=>61-Z>oY6X2{LB^>8HFS=7v@$+09?3W&F<(ql9OZ+r&vRzDh$ePc*mfm;I zHY694@*Lj1nm-KSTqfCz2-1l0-_}Fr1b4uH_X*m5#fmVAV zDTV~hcpr_3H@|B^{q(6(&mj90I+u8*!}v>D8n)>iu}9{iK(E3OC8V zYVf&+X%L+EIQjDT(~Xoy>~~D5{|3Xobff}YV;njQp0{-Ui}tR!0{(UN12L`8p08Q$ z-YLY{eex8#Vr)@6rx0j>m~g-Xag<|Ldi)o%uRm(sO8+80ovAHPxlVG<$f6dHWt475 zVgc`@@AkMMY0HfzX;X0*-^|fnBF@hguEBL4i`h%br`E@3+-|Wv?qke)s zYcQ;&LZdl@@|nD^%_=L_gUNu|8AkT-RJ!x4l#n{|)p6|6w3(9}9tx9PuO zJOXSPqT@;4#D!vGMNNaZ7s3*SBx5@PX2zubx7+=Z4Bxh4u5}0AlNHVxkwPH-7|mF9 z?kREa&zWSI4n)ttHnw-S>TfQ&Lpipy17RKI4&$L+rLlc@=9T~&`Vb$@Vmf(0k8#IX2Cs4s z#}Zo(A&vHJiya;Cz%<`5o@KDH_tU@D{iTwm`d>RyiHJ-~@z>>m)O4hbXFc?<%h4mq zp%ld9XgebrtuV(w zvh<2K(`Ps)yv{7_s)2}^u5wLKHll` zQv~_i#|s@=t`;#{EJG$`*tgpkJwI6DpBTSOe8Vw6+LxJrsM?-8^go|k{4WmFS)-#q z6(hj+XHX|{m$5nL^d}QXtRQSI8%0>V!8$H(X*>G!8>hI)Ro_2_~DfS%nRH1 zW{D*G;O@QU#&iN=3!+Hfpa0%9Yew_$WS$RCOlJKP8$o2ikA0DNgqQI$Sb}rJpOu)2 zW@MZ_9^Q`SxXYK~O=@{^h_K7UD$Wa&8vLE4+5%HKzF){;m#*Kv@abEF8K3Sl`&G$0 zH;H>zUmd9MrCdJTXtnpb@Z7luuB(1toy8#m(QqCw)N%;{kevVP;@@=~WYlJ5=2m4v zvG63l#EC}WXf<5P9&>7>I4{iPUN_Tqt}XEQviqxTRKCiVEvnb;5SzY>f9>u1{NIxl zU^%^5%TW11G*jj|Y6p_Y3-T-KuP(Zt%>*b6~MqLZ-e=;nKI-=G6{9_)~ zKK>At(*z_!MTh3q3FI)r!P)xE_+o)kD(R*&+K3j?qL%|HI78oG>|o;*B195mAD)m9 zR+ZEzz*{kTcAr9Qu`?=0;XJ470C_8Qc@v(bWKc90bF>=nTo~QCEdPe_=iBcyLe1v& zq4Dp0TsqUm1t^UFNdtsEIdRh{I)=E#D}A$Y_RA*uk+0;N(K8l3dhwk{EPoBg4b5m&5k=X$(ZmrU~_3$TK6a`I;$XsF`aW7iNZPARH{j zCuwLkB z1Zk106+Z`SgoUcn0#Dfab9Gw`qBh`q+3X&$0zbCOHRgH*xeSY-gI#>^VY*j#h%Cp2 z)1cDIt1=a7Uafc`vzyYdqYfLIfwMN?EOIt?Cf_Rc?=_0}!vDuuq1ZHD4k?1n&Vv_v zn;9OmR(|%g#HLDQFnjI5sn~qm#?y>rM|No1$(m z?l!ZsL6NFu)i+3fbD!l)#9<>U*Uu}&%Kec{0Nmbk-{9@l>dNh0l=m!@&aFrJ2f^mE zpkrD>W4fNFKAZFNU{T=$WzvPb7Wv;ZDX=582yrjE7u3}vRi@S8R$yhdV`WvqbS0qO zA}f_!qxsYR$vFAWU$pO4J;~>u`vya5nZf>Nrp3dsp43a>A@p@vusoa!uh{>8m=5@ zYg`$p|C=zC)m@I}LUZa?XvN{(+e$9qRN1o7l!?Wgh?=I;=w=Fk?M!cP|CU_4yLEK5 zL4ZXlx2EyNih2E&*xihmGr>vE0UwJg8TRhMUO_7SVkfMa>dVrA4<)pjsgQ^71iAX8 z&8NZ&YGhlRaB9eq3G`Tl=ycAC`f~KqYJqcM73Z}4<`9(aG=k+X^YwK&y)AFr<$c%q zQb;5?|9O_krP)lWCITv=T~&Jh?%4(C|HSzZUc13ZnnBgWM!2brVW+1FS&kpRSY@~u z^~3*lFRwAMAn0M;b&FWei&Qr5tIj^ZJhLA#IJ(Mfq2`hyP|3K)akfB(B#_`I+|5|_}$e1?IBjEsWuOpw5dg@-EQMZmN!v{*dz z{MupYn8ojdpd(tnRy#t%ixr)qfOp&dSJENK?eduTciB9p19`@=O)!ug>n%cO0V*m{ z)b9==U^|{E#oA%b&-mvOLv8~j&LV<*(#QJ)B0dHo{TdW!h>GrQ%+j+E3 zeO1c8T_-?;{c%WSuY%qJ%|wk<>{N=0!@@fpzl3$d^uNUTr!NJB24L|6uL_m(Z-a+o zGkQ@UjHKuRuM@n$pL{gsmwgVI&$ftCJ*r-P_~+v%`ZCM``Lss<8Mk;sF%YZ!=VQ=n z6)g})^t@y~8#E93l+gTKHl8><3I6Gipj*(dAX<_xdHGAsV`(#Y2pK3j{rh==5gEKa zLykLAka_F0LTDuNsl!%6IGV_-e;Q~YM~QobCAtJ{9P_v`*Mn`saAtFfsUV<^<|>UZ={zacA4N;DKllxWfDpWh!cZs)?!h%#2@V z`rh&|p}ha^bIb@x89ewR;w8A8MS5&&YkT}C@_4egwzqNM%|gY*fG-jE&q2Ui{`>mh dc|$e+2vfgoEZbwDQvCG4q$T9Vt3(a`{s&&uNB{r; literal 79745 zcmZ5|1yEdFvn}p!!QC~uTX1)m5Zv9}-GjSZAUFgK?t$R$9^4^F-kE&&{{MGh)uaki z#hE>O_g=kv_3DXIR+K_Uz()WB14EXP7FPuWg93gAKZJt;{)bia`y%iQiAeVX84U?N~L;-c!F;HSB;p4b|*!!%uq5J5jb9Cf`@5lJLB0*6wMCN)TW zC+df-tRkYKE{-LEhAmM_M?Zo%11;|D5PWM037#y1K({%0eP+I@U@z@_alY}#E6;H% zQ?P^o&?BwG$Kr6p(+n<5Sp*E~|Gaw2pqTZ)iF|*#MkEsSPU-9KKTeV&E{q`d9w{px ztP>%9fa)}&9T5~K1poj3&=+cEcJ?>~Rj9yY(=R>w^Jfi&7Q59n^Qk3?5F3=r8| zc12ga!{a6h$jGGL;*pV zUSQbe-Q=(-4gjmdE21OioVj+8-YmSDH*EH@1xCeSb=4F8b3g6bwor z5ljF(&|6XWs-I2H+oY%b0<^Mq*=tp9Fl0Zk7c5+wz=1CfAd zU13sSYyDM$-}Ajb_BinDqsd%;Rm{J;U_^U)bUeIOBfE8&AE4Pbe#P_h`l`{->;J;z zcD9nr!-<9f7YO!0FHeXd68}e0V(MXPTsG6lHLfbxUvn}zo1I>may(}E6`p+HxCjET z2tAKXQd?_nE*y?qB03J;@cy^cQdlG;N;FD&CC}#_N*gWyuija#CgE$6(Er*FSyCpZ zI2>kuTGUz~0@T}Gt=jC=y&ktCGFdwKlb?9ZV>JEa36Pg0Hxp^Kd{EhxotVY3ZB+5ivW?@}04@O|@E^bq5Jf z^%E-8zECn@{(tw|hz8<|R;?zo6kdQxv*>Ka$M4;Y%N-uI`clwP0-2yi3q0d??89AR zKvrHRhXC=UZe1nx{BTi=u*U9kAV2TU@!$L2V-E$qCk<+)ZnL#|l{#Hpu12Z%-LLC| z*>WX{fCP(blMS9fWdz7~-*%O@f;(Jb) z8>1WyRq?oNGt@a>fL&^2R8ml&s^QP;L%Izy?UM3W3$3X5 zcuhBzAzo!W5#ZD!LEFl)j9b+VKanr|kr)hieOtO)C$&r=TUA6Pa1C0;*9@!TKXJAc z+avtvr5InOMz>l0tQsF39lh!YxBZ%O^E3!|Kp%aQZ^)Jx;&3Wot8Er^8TR(~Q}=m1 zpF~4okoSaETOHC*{`f0jyV(^dNN8ea& z(dw|k&l-YIi1>I)g$WRNG|F_whP|N|NPqjxn&pgzZarfHZVj;ZgANaR`pLo0cj<@|8)#@B_$Qr z^gtBm@neicy;-p4nGgP$zQ1bzFnf?=Sry?R&w6rQk;_>0Ghm%Aa0rxmt&X-%FRUf|##o@6J z{gfs>YjWKFa77ib3Ts6GAYByz<8{9#L>&^y(142_fl87#?DOaKVf+r*jSFz`s~_>W zEg}Dl8`?>M*HYeBJA*D9P&I`;i;L+zEto!z^d=;iqP#Fzu(Il9>ZVTs{8vgP5bH~T zgX^IM7WUxGMv5+Ff^8P0Fgx-_?dW}T{Dh4Ub*QopMAZK*y@wj=t3d~K0%f@zSW|c~ z;ydP_6bS+!oSdAlS(hJs#t|P&$GiQWL5ZZvZk0>b+u>SudNbgU!3HcgS{)mJR@DMN zKi4VXESZ*hS+RI9bA~BMjT8TKSz4$Qv$E(Dlro8Ram-adT_~7kvY8>!ef$cf7gZz# zLCc?{^vVT+Yt5hf!<$@>RS{57(#$^7D5U|x7nlml2B{XP?LJUJpv$#us`YIwm0=tR z7~kUChllc%6s-C$*JU(k*-1%c=91bS?o7FSZnE+nBZroo-GX!>WU{YU@c+$|=Hvx; zaEZo9JYR~Pmmw}(p03Tj7P;(JB9~Dfy~Zj4q=@=`cOs=$u87U;uv8ahY+^!3PTp6+ zZ%S%}2k{kf2zr9wnDq!xcv<6KX)@Q5D(5089Ls7j(jdS6gaSv_?dF*r4ied)U;VB?MCguMbjZw*)5SkE3bu@H~k&cN2sa1%2*0 zI54EEpAJTPF_&k-{*@9Yq(z>p`+wKBnkq(L>&UH44G%|{|HzWaN#pBKZM8ShNC&QB z)rw_cAU1$nMuR#blf)7tUN>V_?IbXzO$|3Q(D%w)STW;iOtVg=Vxe4MXe@r7*P}EV z;Mi{bAO6K6?8$qaAUiud_Zk^O(8l^0>2vBu^d8Nk8#Bugu<1)y4Dhs7n!%CL>d=6o zVdT(Jo@a-rRl~<&(&ZRNO9>C&+cUMEuS&(O)Z#-MX7xO4UKFnd=aSWLa}MKj*pS!w zqM6$M^L)L->R=-C9~$#Ni1Hp0-2a&bAD{lqXEm|jx7X)=zqUTf2v(D!3R4}<$0u2x z6mZm3=Q+_HS-{}hHoBE;yjmm<{RepZAk{E|49 zLPs&aylb>SmRcnufJvj6(pn7h??_UvGZNLwUBgGn;&xC3I9*~wy21WTsazHF_V#ua z7e1+x1Sn#h777}r(3SBKN^l`3A%1Rr<2kHV=Mk9#fV%?7NXgQ7ctk`BEM2$Bj3$sF zDnG^AApezWwx}Uw9|HCpew@y2Q>Ae`Y^b{dc$8h0YhZFXnOnsipI4)v`x6ZOGbV7| z_b{REKsYq=g|^%F*s$BTjhQ$o?S58CxCX;%jO1Sk{GK4FqJmbZ6v&1kydmJWpL6rN z?1gn}sD=u7y-V-R%%A_82Z7xG1p|0=6+3K~V0a)`vV$Vb`ACtF*%%hR)>r+Oh$_t* zd}tV$u~)P~Ey<5j&=RyT)J=^Y$) zD;%q?(ash?${YYr=Ka;42q&5m#8@ED)zuYmurb!)Ll7)6h?MfW5uzMmqN9^NXuxQu zg#9}&Wy901zdEe9OVB9hILcDo*QuxVg&}@!ZM_jr0x=<&;I#PZVky+hXaLtazz_et z)Ar6#CWfk(BjxV(LLuZSAb4DL_+Inbj7pi=4-Nrgtcsx}=znoHFi;O}u0~JGZ39Tn zUsfdJi1^(T7xYv!&8IC%N*M(Ob2q#WiFB|C3FVap?@~Z)tXdu{9D-qr*!2MpVr)xb z#fX0y{AbxSDA@^Kk5DY1Q9D?8v@b1bw-MpsP%r+QT{WVCSdVpE37dT&1rcz-UseaE z1Oa&AYfIW{oDwPyczcrC0QkXs)$Xk%?g9u198fBUra`9n!2#b#KD^!^S47Pd1VT_>9%VY1onSCwrxU!~D_6sPbXY$n|zh9DGI#tU^ko+;I#Pfke@2f$$A zZu_ttK$N3BUGN7yPU{rLgLFCysrZyt0FmcGAaREm`qo0D++Zr>6$_-54x2@0Ss9t6 zhke;b^B?MGr}!s}wOVd%7v0b5cK_yNj8J^8N2Fi+tWaBRm&se3Q*|26RpjMkH5pst z1dvmRcwy#D5(Fh}Z4<}J^Zl!$PbfzL@YbXPKB9&Nink9ExEa>R?!xtNPW_ejwR)|J zi%K2-N{gZt8QczejC5*cNU2PEn)d(qn*IcM>P|JZYNn81ySj$oHR{X5hL2L36PTDH z5fk=?QireF%8(8QKBu*KI zKQ>fbP8Ypws{{0u#cs7l;rw}meQLx_{XcX-q{kL&39-?j!@WWG6wmST@>3g=22UNk z5218ex+eXu$a=S5_DYML96-|8onWBvf4)3le2-$s7kIhsFPVUt2Y5{uuS=OLt#5Fk z9UM^44ssRuiiq@tLp@3kIoy=8S&_C&v9`5MjAL-Zthcz9^>EkSb$ zU`)DAmPw2UNC>>642&0%|GAl9aNi<(0B$_U{Wf#V!^GhOE}fZuFqN;g-r+%WOoLV) zqzPl6cK=z37cqEFD_JJr_Y|8!v!WBd#O@Ia{0j(Yez9jL{ce<7s5coFXE;rwkctdB zx=Ob+HYRsUeE0F=N7e>LslCJ2xm0?MB6^J~WdK4eMS>unN>UGrThuF6UEf~3&_#yh ziH;vO1Ew&B(m(pkh=n6f-?e#O6wEGG7SoUyKoG_-!h3}YW)QH{kNFwO1-l_O> z1El{Qx3NW*kL9*l@JL8li%8bBYR5snwkE+s{{;2{MUODnvg4FP`O^hZLG9@6hlWB; z!%YcOLqb4ALax*k@UzVC9+JPA^b1DYz+Ywu>|e`FMqaMDOht(zSWIMy<}4(}q##Tu zk&!agz#`*|=iCM6vrnjc>X$3#CfBO@yLY;uWu0P^8I*TU6iFwMtAEw!ed3FYjpZ1B zUl1V!2V9ZlD92X(7pgv+mBnfu3SEOIYFfXj19f1}t)`2ho2}<%U({fB$B*lO{$UQX z$oESFk{Siq2Qjh1E&dmgoeB=ipMg9Xg)NMRof#{w4z<`{R$fdx03>wX?t@l2KRaXE z(6*VcD*bgB4?|G>SOfO4+_$TN^~BKr+_$(|yPmumpjVxePwX_~Y-_CUIsb6t9yh2Z z$b?OKxd%eS=wTMQ^HRrq^7tVZ!FK?~mU;{FW*P1mTE;Y*lJ)m;d;NXK$bS>Vq6l5X zbUHf9qSYi?*ha!4xK)Y>l*{wy2iIKRahE&28XxrsrX=}2&P(?8_IzeSko5^~b-lT& zbQ%u?iD3DcBLL2RpE*vLK*`J;4=M+LUk#H)6|ZiNqcU44jtT4fJe(_Z4iPQLT0|Z{ zU*hw+fBuZ%gH>k{q_W-FYg>_o0DG5o4c^!rRQ$q6@2hZ${K3j}%M7J+t=Aw-P{;Gd zH`g4yi>LKBr`1d-kWnSp+ubPcQ$+u%S}z$PUV=m8e)@iY0_2IvF6irGECx+^{gcTD zLavG8Jyh@mo0PZ2sUa4m;E<3(?z5Ffmo1BUsh+&3FPb&v_tpCrruP_D8=btV%m%t0 zF>iX4vskkozRexqRG)!GAMYE9RdMs?D7tE60O@Vmp{DLVF*)6AgTUS zKMX2G-t+d*@UR7(9G~gr2U+3>Sihsfh&}({BvP@u=yAJV=RC5PC9X9sPyT!|?S%bq zUpqMNaim^0yCvc8VL<&hg;oS-0q2#Z!N=?KW5Y(cSB;_=NX2;k?x!OcWIK=lTo-Cm z(oVNY&xdsn_M+nA-0n@!=ie5OuwiCUVG`z{;wfWrbXckAxVk+3Ebl4%$5$7RcQ5@O zPHMlquXZg!J>Dole_AfFCx&7fMcOut?r%G=5pi6c;?8w;F(SLs7|M3&&Lm?rn~pm% z%=gM!SMoegC&T^82DK|=A0Ft(Bnd_g-S-DgiLMWo1z*@-;PoQlkIRuD>Dn4?r=`wz ztpNNG7>0cy78bsYP-dwwz zjZ$y^i4&066Fm)b)t9KQ-<~hhvvz2{Lm$tUCor0{^7Ou^Xl?5=B$_Q#z!b#rME*1S zndTVa0Hr{&DQ=dEW1m)34zKsqKxZj>iY{hJ_x!xi2|qNufo=_P{RhM$EU0MjK;KH~ z&-edLr;y8+x>P)=f^L|s+`E%XOIbt-9f6JgMlSOnj^`Ki*k^}Lto$hZVS~a)8;j$i zQj-+l@p=MZb#<&bT-IREb$;U`)2M~xIKp(Jhx7HSHxl1Va3+Ix^;=wJJ2ZU4kHPtw z6LoW{l>{u%AQ|!aOKmrK{Exj!)ox!eZ-69M08vKSIpS1K&(n$sK+hJ$$hi>op%0$m#HC4h+Q+q>|T9PKnuyyq`s6j8Ay)v?G2+_0gY+1e-kQ{gcjP z4k-+jzklhbc0vzO6bw|0_&i+SM7BePiz}Q zq$n*r+Qhg)eea*tCp@C;tQeS2Wub>R|oflhbAW4_klN7dPHjR9`xx%$3LMD>4U_IsHY}JX?^`Ix;b$>wd z(^LQY9LJ2z@_dZTDxj5WU~H?2i9~9UUMI0(SaOXk+~d4&5OPTr;-bDxaIxFQN677c{N?!KAiFH+IXo{vh$O!+#m zd-=7Rh6mHL!&St}Bl);*`iJZJ7N4w4tX6Jzz|pm;Xg5poUThDbhKsYvTihE@VN4nu zh4)QaoUTmC8CR?Fd>IA=dZ|;tC}S3&^jfkJ8l1jdv5&W(vbji+fNK#;B@jHrfecbb z{o2^OwbTOb`nrQYPu)$Ym@i~7_Yo*gGr@-4x3n?i5duDKesdCQ%`9fTQH3_|G#$m@ z^Sms`xQ{SFcjO~{13#sQ1ZQY9>V^2V`CWLu)2fqEf*TAIercBed$1cY<2fSlqflMM~=LKL{)mJS~2OlhJS2<18_~cluBL z8eg(?R{sE?E5XfDsEF1n%fNOrAdv~U=ImS|j3zQzDaA9~hbwPxb6#MA zH^u)aU^71VH!7W zPM2zIJ0t|*qr>G^FG|Y>cB}P#^iqLQ_X`n)}Gg^?1r$X~A& zniSL7kWaI~OB~dKg!?`)v?(%N5`qjH?MF8E6qY1}`-AV>)1lob$}L_avED>7G3k?` zs$7z$f7&qmN`c6GgnL?6lw0XE^CZ6%G_qZ}bCUw$uh=42YG7mXPzcpk=xr}8XG-AV zD`Tn9g&Ij6fm_Gef-w)&x3{(&AxVJ&+O1ZzL)VfJFq1H-_o{*(=lW?c_0#xJY~O>b z3t5^tNPC1;m%^Tl6NFUlm~=|%=6zf<5;yAVWbIpEV3Dfktw~eAJoCUWAn6FAR17c7 z`@oOTkPL)l7iZ@u-e{%rsqorzwF_JgV9F-x{j$9U>NiCAr=oS|w->%+5S<3deEHnm z+zTIcJmO!`B2f(D`=2*{ClL`d^Zdj}+N*u|?yeAk;bxuX`^2-C2qn(P4Ss|T5 zH1X-efJ#Lg@U8u}qLVsu_Qe#3rE00}Gg@1#RoMu6a|ZkK@z2dh^^+Su_fu)0w?pvs zxcAWc-H!{p3lBQFb?0W)p_IzX0?swBr%{)xF!`(U( zNlKxj@i-_xT_l0L%juL;21xp90KwIG{#M*OR1H*{E1&RfJdRS*4;l4Z7@gA4wali+ zH?l;MZ0e(N9Jx--%}YUn!cXY>!-Bn8sJ zV9`puk>JUGl3-&$)9?1pB=*`v-%-L63<`pT#m9$VN}SS(nJ}U>GJ&?B*;PMqXCll= zs#PlnTXiRDTpxlK3ktJv1-4}X{c@k&7yP=NU-80ZU$ssfdRlwy?1Tl?WGEJ|MQ4#2 z^%pU{fADNh5c(MTKSMttE8a{J|muxKp0WDV|g z+g&VSpe(toC`ifi3O4v~(QzS^{(ea2pAQgv8{C(@3`mVX!;uM03DK#(Aml;4hGhgU zaAJ&vpwoLKJ51;8e}tnes~s_pKb@*~RzCN>zdKu9BtwY5>N&qWXS5~UO6SePu`mWy zES<7@_rsj6wRre%f82^q*6Rkrg8;%EBf6a;8QkDTT%Vd07%|EHNUzX7ip67TGLU4g}{uPvW%3PS@egKn{fv+t3{i8++ff zaeOc5ZFM#l3h4pVv`%HvYL8`COyLHI&j>+0vmiq4!$D+`>Nd$-sZDcIR?BZj%{L)> zm9{7WYDK5H+K<9*XI9vT7yA`ZFfbUyE*0vP$;HgfsW*D@)wgbyY|KK-QJ6FXO;OB3 zOp34|bRm`roF(-K+Kksrey~eDjq9Fo&L`1@v z!gq6OI^7vY6hR$oWX_j;wcWqRcC@y-DyKwie1HP3zP|p!9qe=;2T0=Ipr5<)Qw-Wt zKeD~`ZmTz2q3TKL)_Q)FqF1gcL9$zJFuMh6kuf`^D#v@+;=VLz#rl4$&lAc^))U|% z0&R}oO8)&p|LC@f0Z=@BuK?bMnnmMX%T#fkO9qjlJFnWOei&p>&0NW3iR6K%{9buK z;C!_s!P+n3Zs=<^W(pJH3QKbjalmMnr~g&I?L+PNw{{wey0xWG0F6wkKM~wSdQzh& zEQm0eKGpB^^R4u5G^BqJMUnoTm}qqP>12ot&}#U6AL|q+gA;^)OY6rQf14is5YHj# zdZhd=e?F(&XFKfOKq=h89EvVTcnrLQ_XPy@0jKN|)go02dU8|c2A@x2qS~f9UKuas zpPsJwomVmGUkFEf;T!eRMj2((Cy}6zQMh)8;)(&#Y~P(L+zw@c zYNENXSxLwtmCPyk>EvygB)}HOO}-GeTwWc9m@`aO83NSgvK0||e%0;v_vsyHTY{l* zv4!c87tv^Eggmz3z8^HSj=ZINRwUk&3Fl6E8fsG#cfla9bQzrD+>(p^}AoT zXyYqe_BhCX1J(0pd3%5|n{AGpQv@GmJvR%a93YBxnZyzQw&6&%pp1=;)i-O%&{fo{ zKlJMp%|`RPEHL) zeX86DPRj(*`m$xm0>ln2tKz4$>X`vi2cH;<@wu{sI%%&&wK91!lpT{{=d%?~RuLsl zLrhyP4uVwJ)JTM;}$5WVp+5|zJWIbQka+r1LPu)g>`~+(p!y_uHe(ZfB z;dLHZ0MF-Dz9w+!BumiJ*)|I5@_FEj{j|?422Q{!V8$Ei%-rzB#qoarij=DZB}`S+^h9TFl{gIa@6K}Df%%alz$B14M zg7dGr$`$4oOHc#v_wBLCH>^gJ;do=hK%m<}5YNfSx1yYv-}m_F37Aea{qy=}hFebD ztdC`;X;H<3U?TCdJ5NRE4L5{>_jz0dvnU@*ri?$JNqz5`FncE}&tT;;n%=9aRriTf-k8$jJdq|&gAFkN0n1j* z<)gA|8Km)%tz+#O;lBXHiFfPI%i}X6%E}eWWC)0eF1)ZP;x)J4PHhm>!eql4Mi)9* zH=rI&rwfJk(1P@S_DG+~yj?yDYX{vY6LPX1m!b=T~z%x{U1qwEwPL-Oa+0 zxHLFt7?;^pmi+f`KsG%FLhv_o;y<X=K)NRDdqc;|b<#A+N{scp>XJ@HJ9a!ToHI?djQhry$&VkY`6Qu zQZh1VZFH*p{G8W6GYpP>;hg~ZHVzL^(@GLX4p#tA)1l#tkpd5xc40)u8iw+WD{6!x z9hU0F0cir4wh%86W~0eeY@vPAoTH1v>>2r;z5VosVZ~v%iW2)Ot5ku+!jK+EKZ6u= zc*qhIs_M$YeA2O)vUu$&5+FPT7_O50fqW`cwKEG2-l1QP5^mweiUdua{~=z96g%FU z09`}D+WVHK9BPL%N>#-<*;Mm+50(UFz(gI?^Y!=lZ&x^PNVdVQhYUE&xGqLkd=kYT zh5}Yb2?1(owB^5J%0B*dBtsZGRCU`T;Ch6XD;%&Z=nwDwlvK2!BXbnJy-rr`7gFjFkpxeopYRG-v`!TB!8h-PUH!Ks#@CW z`U-lu(%FZq7=cvtxIN0|3}AM6-I%Ak*JXWiRPHt2mw|I(L~QQdL#F-;n3FsBwcQ>UhF)- zcfrSw0o3GZ-f;a`ok*B({(Z`1Q*;>vg?FR@Z}Tm{o?JWHe(OX*3h;Z|&(gty>)i3( zg;s_D%WHfAa>Uq*Q^}vL;7ptAePm+&t2@-9g2X;Ha4HrZ%N+zMpR=_#cECZVTyAD* zN&5;&-yl1ZQ+86AfPg^G z@1{t7N!lodNo2*oMQI_GoshQCPlgYm^jZ^y15?t+SKL(fkH2E6)M zV!@S9@g|B%BbqK|tk<-Gg(5N|d~W-HY)&kZ6+qzI^9nV$nQw@Nr91E{n<*c%bvdCP zRGpQTIhvwT(JzeSMOQXG6LfTBIX%usNh!W2c1@e;{(#bTVqj%TyJZGT`0_J#kbc5o zl(NGmNXI>Nt&u1k;=$45SM?Z4L=eI|pK|y6M8%-OOgU7Z6Yx#T3B&H^bAIoOt_P%f zLQZQeWI`V0)*sV9loqE4AyCM@j%TO+ z5;VPkplhCrku^7;7vSa1A zG1+mnk_m*7`NZJ5DlWpBASBA<)X5?y2F6}Z_gj!NQK~#dx4$*Bxrc#nn=_T*pO>3V zNaZ|>60DTbI{{)?4t4Dcln+w9i_bFxm-9D?Z0s+X-eYCB`Zrfz z`uffbM=DgcjL)lr-z z2(govV~z;U_>a zaW@9?1-uC~+bqWK?CiMGYt-nb@?fLEuCdaeCVK#;pmk~w?a^l0jMP?5rAA;)w(Jmo zpf!mEUPP!)che^BJyNO)5oA93_sg-;$Y&Vwm;u!_x*&2a7oV=T5zd7i0A?k^rrr?d z-HLZB-tkc35L1~pv2zMgOR{5lY3wp2)uBbELt zp{U^>C+dWTji4L8v5vH~1s zpe(=6RoXN1VRBIl_&s%fO?A<*++L+Gmcl3=LQ+DT`ZY&OIxOIu+w?BjlBOCKf9V z^`?>lk~-6df{d(T1tS)O5V7{tdD{6ie|zP1{-2i!&1&sJ{*KnVaL0{KuerKUV(VNr zJ(fR+$`N)*spXSw(J_ZBvkf69}BeR`bjk#xsP9k(Z3qlSFGDf0Qhct&Vm0WG6hfZA`T zut{zQ+(tg9i6^9JQ{L!Rau@gagQyCkObxprOy72g~=-y*6!^s7{UP zfVUI?+3pN~bwUMoA^P@F98JXgwZD3LmGf_?Gt{c(s-ody)b#<$ z@!O9u;C^`qdl}3Fl8`v^=bhRtZMBEWG_3s0JQhmI(247P0gFjOd_nIfvJ5RluP{LE zF#q;L+Hy~2E1uIsvb}yj--Y1gH^*K}c2gKA*;IN7At50cJyM9jKGc!u!0U~4Xa4w^ z*>gDbw?_ZxocXKMZ%isz`UAV{<1t~U+KuSz%NE}vweG;t;tZ>3Ytv6B^V3(?-8MQ2 zL%t-KrJl^q>#z9tkGh~dGVCYgs)s&C~oSON%aGqX1@gqp$yGs*>f z`;$&rah1ML*L>I@zX#w}8FR7)rw|Lute!XD{t9WY3htTdYFvN=6yeA4mfwXju~`u1 zYdd4G)P>&FFMf0f|7=jA*_}TmKP9_z!m$bKaD`b%6VmGb|U;mGtE-U9pbu6dDu z_qIelI-5Tg181smHQ-fr6xJh(2N8!+no2f>)Xe&WN0+*a(D%bb{Z*iJWM;dNUpZOr z@65T7X{1)js<`iPI>rsdqOVN$a^4-<-|f3(Uw>}yTpJA!nXl1rO9u5L%muYuZ1L9R z3eo_$QgLn&;wv5i6rMDmziH!P5nulaW0H;VqEKy+JaAii$t;hf&HN zmQ;IvUNXc}&+jrZT^|;QP*aDGJ~om`yml(wE}4>*OA7#$B?CGLS?-)Rr#nlKS20iE zIDS^|JjcCLk%=1~1uI-C2=cu-S|gQq=ED9^X|cKTI0qlCmt$HaEQ#+=j`n);>RG^O zZS{PA=32|mP|WFZF8i1&#mW#71^}J>iK6(*yRCiNjg7|@&O1s`UI6#iFacHouM#5jG}AK|-67I; z@($(8l0%m9wR5XpwGD?a?8LW_WcRM=e8XVWO?`Inztw=SuSdM>9 zDmgmS_~j4)1PTQTD2&V4(mo1#dBn974W+HJ>Z{aM=ropnGZ$_h=2(3L+}aDpJ&eDF z0JoAS>(+qj<+X;QL%gvfKru613{7i559(rMyJp8mA6=Ru~0e8TSUl119> zRy@~k7du&PwnaL>I01wStPy2=ae$84y?ZZ%uV?7jFb_(MUT!>ue+h{|GMF&tRb!I+ z*W70*ZU01CPQiP2<^v9=aK=iYdn;D8@{3*Yt;|}$oD&Q1cEm1e2PSiZcNmuIB&_i1 zeWHE;q<@8cHA<78(i|%o5UHAsvDZSQITL0isHJ$87BR^zCwUCFk3wBRX|F5c_B9@T8)Q3|!W+n_y@}d7m3C!}4X62}u5oSH8O*y52+};u&QF zk!NiS(#G0CPQo0ak5y-Ph14*1kFTTT#YAIpnAY$|ScGXvJB-4gyv}oovp?SU)@-mg z173c#@@8>X7EsyB&)r~t+Fmlj>iu>Jli=1kF0E9~{^y0RF)A{Ww2_7X8`rsTiu-?Y;z|;~n+j$IlTjJ25O-E4tZr zo-K6)M%~pcTI7=cANqJfB;!@f4v2m z|FQ1;s+?@!o00J6QQKkO_??p@*k6tqBoA?_QgKWF&21OM zK%kwfF(hJ6@k~eZ#g{TwrC1WZ+`Pr6er5(T4-!Qrt~B%am#ZJ#n{$X{te}XK=cXD$ zsLx*4bLgUSkXKSD+~vm^wNt9d+4wH|KHl_T%I|4Ey%Mlalo9f=Bt_!QN3dbA#qn;0 zk<&S^A*ub!){=GF8U)OL2lLyuPC1K)K5@ap!M3b8NPlJ53Gu+LrN`1Bb;a{HG3-D1pU7Dk1r zAfC3D2nHLC2sjG!<2)B{o781xx+NrvYz{7^7fvQwC{e1jW$2=u8m>W(O^pg=QTx^U zosWfl(42lL!Nk6y5EnLzqR4uQS|SCN5n~%HKtrbq7|~KYIY!v`3%B*jX_qul?=>71k2$9QpO zbRZnP@-jVEC!~M9%`t0I&@fDT8v*xv4=mbuabp8&X6^E@-R-0TkO5_rq!ykE3jkv+ zF|Ruk{@;2|l$4WG1L?ja;E(^4>wK1=oW%IT=T-OW9JEfFNn&jZ)Hq1~sa$?7`(k5e z2*41pUMz{s=XG7uhUHSvMc4z9-5sxVoZ5Y;0!v{ZWG5A#0bw@*pqAA6uZpSyL@V!y z7p#Kf?er?xFkDZUrkf-o@^HT=o6%kiZZkO_txI=ymyW4zFVc*=9Kd6vDDNI0gFWA! zz)}R!=%33PUP!3bJxtrzE7z=^45yekQHK6M4jgL*QvfKu#UGWZ< zdS?krl}@7yi%G8~)|06-GbUhP7i;D#O(~(ENSKtLqJQbx8GjD5s$ zt?FvFIh*}{eb}t0o?d8-?FD)>8DU`M1^ZzBr$2_M#5}3no%VUKnY8kOu8%~f!J7;g zn=p|!yO_4QlN2!q%b=-)8kP$t&_2TeEXyG+_UqR65;vm!BxF=PK9CHvU5EWvjw1rp zz-RD6n$w8j_r9g&1bm-6ySw{pnzpoi^|dSN_Ih2;cc?Y@i1$ zDy2Yn7_n>A_}cvdL3Obyn+_{1skP}BG=u4otsv2Q%nr;mCA5jg;<1OrAmMItE%31Z zEdtqsfC1cDK!Mk&{A{9lfo&-8LTfHwM^A81xzzDXPWGF>Au&XUBa30@QgPb1V^xZ` z07hE8XbF12J6$aN_Sk&+e?${;IUPZr{Yo=@2;t=gwT_iv6Dl6e~puY?tf^~U|vRx1*5 zI}d=I(#RPN?0p_+mO(E8Jp>X|4mM(LH8s6JW`zvP?ElJq?OT1UjoKX3t8tVMUS}v% z{&32x<9465<+_}AtC#nhgPji?70Pg1w~HzQW>7)%BYN@3A%Bb4FX&sGTR_>P+*RM@^z^N+ z`;UtT4pn2q$?rjmqDh(HdJ%dmxwH~sibw@8%TOf^`<4x_E5~V!F_&WKR$MKaBuwY< zx{ORsO?}YiFSiCJIxT^%2U73x8p@HGEQr`T>pGWVN&N-}%;>vb?^mmcScq0-JJJ=n zUKLoM+;fgpJtULX17Z7^*FclYR_njmLP=v-9quU%*=evZF5&3e{LARwLz{lg3iIxq zRqHfHLb5DZcK-hL<+dMHIAbBhjAj8B5fm;wa$D0t-c8;HAr&}G)jJh6w zYl9rU8vm@;H~!i0%l9&V8Tf`bjVkp*!1}Xr?T+E+|PJl77_i8+jdFBY%GN}?(0|00efO-jPIbCJ)N#- z947iG#^>cK4F*;PD_~rnkspRO4~Vv4d7PFCWMY8~kWJ_`iyFI|E^nzV!~Nw7-iZsG zJtg$yoz+w{OKA3;Qc9MuT=@(zyEou=){ab#UP34=zDUzmHbY$^#zKENGE0w#KLjNi zPbdQ%2ls|0I9{1h$WCa|^{_vz2c)G~`_?9Nz}8{jMJ#9E$Gr8-KFs+u|UdOWhsle#vGzk&-bgoA8u$>9JPut~ar(Qe6X zMzY;^)F77uE=gWB#SWVatbmfH-Fv~4pCm2p!C!&BuqP-^vx>H>TYMl7pbpFq0twmu|M==q|&{PcEg`jTXF!vvbO4E{ZmD8{Q zkYU&LY)v#Zehm%@ln%U<22@visLwn^XeG(z328^o+iVT&~x=SC`>uzYA%<_ZaBtj0#LRMtEoWj)_Iqdxfao|B9AqwOg6z zjo;Pi%p+Vf7&!cCz+O3^4oq%y&gA~?34-MCeRRxSGYD>4!Ued|7{1}y16a%F|H3vZs~H&og|7?N3L+Q9BQW>+C<{v$<357HwT8JKs3M_@`3pn1@0O^ z&qV`lxQW)d@bcV(US((k2-irZPpAXJ?N5jnZ@y%9#=2SceSND}Kl@56jOr1v2n}O{ z`=eErwJD#Vp$Hz{mDs|6C-RVZj6r8()Gy@e1#?o<{y3Zm6_22&xaoVNu!AM3%IkiL zJ71%hZojTN%xDA&J8S^Ve1*-;*W?3VT5!O*J$znhu&33=1;|*rouOtJw9A;de6AH4 zg<`8}2t0!hzm`KxUmmZTET_qx565W#A5&);73KGKVP>eI86Gqn zLAs?&K%~17NkKZ4?(Pz#-ox)-?^-Ut$uQ4z;y(M{`#SB10&a)T^^0x|Km5y$WXJ{u zan1seS{q#-NT_kX-Yi4`?cjn~an)5$ReN_;R2b_v`*$3X2U=95(C!C6o`?=l&2StS zg`{ty^3_1Z%tN-we(AGrqwMt{EY0S}ZqP{d#p?#Ph>+d!JIYGq$4BYHL!%MRArhbM z%cVg^VKJDv2SZ36;UUe#kGki=Z+z4GMxHdhFHOmG_wtH_V0tvIfhDd-bTnh-V}-UG z;cz2|)g}*Uio7K4l1AqL*71FJUqOfhI&xZ(Eu`o9ML=jTdheXqhFJ~q2&EZ z&t7ZIGY~W-a{6I)bHc%A&?YGNw+!Kwbvf-@&}ks=(!AA7xuW@S#u!<2l*BDkkE*PU zOwN+;P1KdZ-5%RF#fdoHkNNO?R9j&+gn@mi2FBiawdya88A4-NA1i4%=!3E$@M5pt zg|@LptqN`+GP*Kt(b;;Bl&BTTC5I`vr=#5@jUyGU&--vrUr&`?u5I;Re%u;C$dm}ot-P2)7jya1Zs zv*8(6t0B7PV`pL1gBRhbnB&er2)mL#z6L)#9-l}~m1#yp?!Syc{{A%xd>335Oj{k`xkijg z-sXkCpI3YFbU@7VIYmknL87*G=~$S7Em`rt?hY1}S^cWW%D;onNBB^iS^b4Mo2fH$ zwLhihjMOP62GKogyxSY9Q%qJvNZAGRsagui#64woT5*5+D5Lonk;;G>OKVUBup?mb zykzKrGf_6+kvay?%;;9Ad*-p|UQtYg@68$~2!$%AS)xUs0e36WjFIK*EzxjNoL>ZV zhg&<&w)1`(ML=Q<6Q6CFE)^^E4mlJpW4hgwLMxXDWw85wlmuqpda^ zl0yR@&;oOB+3V(GgPBPC!~Z@JcxkH*IwFrQKOKjxhNBDt@hXb8F9LP}(kBgJ8${5Vb3YBFFqkg zxxx_XTX{EvkB<$g2LFhfqY~2oZ!h@#$y#JB(TdC*1YT6RnlsEk4V?R%G~=ZzrDoPl zoSmnt;Xah2v>(N);J>hy%w|v?^zAU!20{G4=pOqlQU8u??AFT6W&Y*-Im?w@;abZ- zD4FxDP!77hckHbQ->gE6-(x&s?^LlJHtXbos)%+fAe;l+TLGE#i93|kBwD@O0&OdO z1I}ZXl6}_q&ush2lviT4U-~;Ywg~SBuTvDZg4o>M@J=8KZa*rGwr2TQB!1!&oD0Y< znUw!3Bf(F0S{3~_X2;?&grRfv-ZdHJ`pWF{W-dma^B7aud`baRsBHYpsRpxMB^>-{ zNT{&XC@_u8(s8!)Y1qw%o_u*-nyfU811x=Hr3qV_v9<4?EnLZEmecS5CSodf*qbMx zrMcXq47BMsGLEjVJb4tubU0@JI~VJ z+l)m5(nos*R3Dd#$AtS|AR9Y7!bCgK9|;BDpNob)N3q8@iq3eSsO;JmjPgz=^*L5- zP}EMtd(|Q9w(xPajka56jUw?1Z^j1IflqKGu6z#_^^@gq)A`KV|C_=VJ%oMqE@zP> zL>(hD`IB42<#nQ&Ill{i=7n$X`WdutC3z7^V+=vAgkOoKTo4<9GFJJ~zRgg{XS0ZF z7z53BIk>(2q<-We(Avsag=bJW$tlf=wXvm~vC%0=$*vyyNdlVDF=LU%M=%0fV;E7w zm{_40|9uFd2wmcV+hq?ztOy+w@9&LP%>HiuLcBvllXoUZB-Hkd&B{^&uFlWEB}wL^ z6!fd!Ru}98{~MGvx=^5o&_!$Kuw;zzHSn8Xr|vYYGC2$qPpzP}9b+PDl$OTl8Y%ck zaof}ytwn+N&}nfUUQhAA_b`l8tX_wIu`~H*hF|fG=kZ}}VWPdDcEisv&`jV1mIr6a zW%$`jqaaK#04~@8Z)9eG3J$`F19p~2k4QBwC1Tf_4Q%!ljI~kFR<2tmt!qOIR-1g^ z!c@C=w(1MHNLRh`kK-2zWpMKa!Nw0kPpsrYz4{?a=zpt&hkTA!A9Ar-VSICtelxLj zf?4_aKpAu8>Ua=142%LekaIuqzPw@^1bx~C9xxIV9!J82jcAGuN+Oeb=x60@yI&Qc zsBymg>+;=p7_}`3O(m{sqU2XC9sc!7%(`jpr?KD09%>!HzVT!e(JUYH6zVmekE=j2Aj@0qM`4THaQ@ zrNky7kq3nruL%c0c)VM~{NISY$7a-|{V0WQVeEd)R5LTagH{zf_my@?rm;kAMvzNy zibvOYy+7`JMC4(6(rUhu1LRl58N%QYhGaVBuL270LrkPta`9jmFugXp335`8!|dNn zML;XV@xY(b9Y5>K9ZbC{XIe#)8}#d&>)m>7Nqg%ls2qQ5iaQFl?-}D&*W9|+mkg$< z2XI~}0cuEdiV77KRreqcv&LvqJ=cb4PGa9Ll^s4)0n&ABoKn zr1|Yph*V&!WK@2WKUh#%{XeR&Q$Umi)vMk~E}%RLB^mJ}sV&r0e)=Z?IAA zE>q<~LpdP&Rg(?x4VpdF+stCUoGydFW;O3FE%Vq(|L5Onl<4)g4Don=$jS-Bf0lG4 zqeGOTG|n(^0xuF&d5m7czuXf5$RK)}$M2nvU7X+q)vdS51*MnTr^-Z8Z74jvC5R$u zw0XJ|rj94z?ue%#Id64SX&-9UqbA+D5P2XY4!+PJ#Y<&3g0=w@Y%``!?aGB4Xf==8 z2L0!c5$)tgSj=b*=Q$K87u0UE8spljD;k)alW+Jr6qq{Wm7pc_#GxL3sUH`AdJR#% z#0%gk90meZFUuaOiNMrj#O#xc4&##mHM==uZp;o!E6`=Qz}AzQb+J2@4-WpgTVIoY z)cr9iXyjKfppT1TOUkJE0JY*CO3dAig_AyM1MHJ<^#{htm>4YE`Np_MRkZMl%YOzo zga|KOUZw!@emL(5JXRn?Sy-k`Kxa!oGs`_)X({Cz+SOp99ii_s-t&qqJG$-p93X{& z;KQ^FF_#7QW3XF*AA!;k5&Fi{Fhn~r`ZumSFvX!ejUO(yrGrC0-I1Tyn%p;;l5uX{bxV;`7I_cC19=d4;t^7 zNFQsXLdtwrcnv9Iq}v-Y(}{x%V0t1UM){G#rVI{2jxD|_a0rKwIKe-1m^Wz={uLE$ zKeL3>K^xH7(+8Xn0sD$xt_4S4YiXasW@HFPtkQx*K$>VNGkSm?5CKPG;%*sGUl|rK zriNfE^6&U1_=EH$a>^@*a@_i)niC2O%|}}%lGVBnOcr=JQTgcds*g|MYZ8NyvA*cb z$a+?#Y&;67>ePAu{;wZ+vThwX3XqOZy<;rWVS&R?87HE2kC=dfp0ImAd zYLoZVz&w+R!vOp_MUUwkE@tOiXuzX{uG3&wrhc*iVXf9{hBZm(-8#d%mQ*%}*c1vz zoT#{XpoNXBvhoa$QHmhX_`hScU=K)*^p9!ajk+IM!v}kSy3N5fO|8j-Z|1yB;5mXz zDBaPhy?QZ-V_K}zmCveyZu+V%Bc-7WP^I7aAmw&M_@`7pIRX+*5pWchtTT041znOT zPc*TzC1?e(qgssxbTwnDU5I_>fDEH~uLi$-a0Ika3lL?9cI*DfegDsj+kpGH1BCv0 zZ(RJH2{qu)R`oJ zDR54n?XYdBjW$?6wurwUVDF*;e#sj>ot%ZP-c)e3Lk);Wr>3(M%*XimvPHSf17$g^ zP$1Kpp3W3uqD$&kybDr0EZD72lLXb+@}^;^wug50Ne=Lwn~5O{i-M>OsDcPy?%QZW zfkmF^?S?K2he;Q{)-~{HZ(onC^%vQz)_k#JZUc#$$8I_2-td<)@rNa z6!~neZb$bhy2O|Vej`UJm59VCu9j^;N?|f=)|C?In`CA<#-lj>GP0iw{@hFW25Nab zh~!bo;|Uw^)yx4r=QhxZ(x8*h2)~S%{T^^$*f|xcm^0c>j0b^pw&2G>1U7j$eJVaTUrCNzBgTLS^jX6OyK9| z2S80EkA`@yC(^x*k?j{Ot~F)a$6NC?CLB#ep*X97LY?>OJxdK_e_wpD+U}^$hx5X8D5_ zVlVnX)5=g9>QXAHUJ)JTEFq7VOx`MJ%S1QdTcfI@Lwqt01sgza8)!PQ<8GIe!}ec+ zT~6@l=dc($qu(RWdofMSs5svl;{C3_N8VH3U{PsQ}3PpD)fSpK#=0 z)LH8fmaXw?YrQ#5&5~NAT3Oen{*>oCcYy2Ty8lb;79FG%zX(tymwmC%F*1I94=3xM zKSue6q=t&ynP~#t+>IW4cbH;55d@Y1S|Y$tkq>ap_(ViSbDJoEz(zhkS)%SD!J?7_ z_wy4sniQ1m>3`-(Cpkg{0ZRVL-P{vQvmamUoHyTnUE$G!O(Z0QAHw7*$JLnbHo4_m zc8K5N0%?#AS%(G^lh%fg4Y~m}F;0z-64GH7Btt|Qh9Gjz7iZw_{gwK?pJaO+-FkiX zI0)+kNib85c6ua(UjiJ-rUnjJ8$HhGL3UXSW>yem{ev6{zzHh_p&$Q|N5Rw;`Xm~8 z^GrP~xy}VWUQNw%eo};SBl>SybGh+Ll=&p$xEi55u$MHeg4d+*Q88x@4g&iX3;qtC z3=L%xT$zk_gqJg{AlU92>0fkW2Bz`=9FH5iF( zZ6PSN^B`i+q=C8cUoWZC7a{c0M#)r8Y8%OQUjbe5Vds73JOQu-kbIfA3MSO3JIe>% zJk(D-wx141b8~agLB$^5!0Y>RXIa!x9@;o=%U_no&j|G5l6B>|;$w@ul$K-}@L#q; zCSW?Eho@qy1F*zacv?_EJ^9Rxb2K_KijmW7X!QTDBuoKtej-5P6g>Si(MPLOX~k>1 z=+w1AGeeBp`u%i9zaiqNX z4N>ETbQGwuCP071CB5QpixfN(`sly`y`eFTMDpVg-X(7d8-;&kslZxxObXq|r?Kqk z{E2~ln~P2PcQy*)yX`fd^Tl`&aKTN4pbRa1M&y(iAmuG*t#sL{PZ13ly`x=IT=lQlU+04G+ zqf}Pw-6Y^byEnw@Vk7kbo`N2f2D!G7Iz-$V&Xqzy-YLaqV&F_ZmfTO>#R8N)uLUcJ zNlSNnhzz)dGV&WWLo=xmAS90(fKk{r_3D9iqGfB(Yg2y}H74-<+?hj?$2?#>nMxvbbS(xgtq$m8@Q<}$K!V){)sjw~ zKd$eCK6h~VgRdxQKRd4}*d;Vt>R+-v;?Q@1C1&_%u3{bA7X0BQm+U@`9!Oj^?{+J6(M==g>fJm>C~Pk%QtgQ|SVT?T`54e|9qU zN8%F`r)`kK=U%;W77mzq4nfF9SGs7|DjR}WhOigUNm0EwXrTu zE7y3V-I+r6cE(T#5g*x^MY6ovC)ZQ zsb`y|Tt*K73fqa!+FoWw)1!H9Mu23lz&aB{K3u#%R+y6V0u$YdGe6rxbb7)!4G^HQ(e|4sa_~w=Ujsq z@85k;`XK6O@b~Y6d5QA}*h2k2g16!W(6`TL4E+9S4%Y^Q{gm=vq_YDeE-qdS5W%Z$ zffX*8jYA`^9#2!-+uM2_jKnX!`A(|gZH?f+0a2r8>%{?2t=O_Y?D)cawHxKy>kdeb ztA7MU*iQZkbex-1m|<4y1L*X%&emRLP0h)lcEN~248%u>{o|$yZ06m{`Fe%Jfrj?C z`Q_kaRvGMKhvREBI|PfON3V-^fxNxh{1+M{A|c5z{4*s0u*GyfYGH$oSMVNyxQl_; z{Sl4&hA92Iul`5N>$WT4_cdKTuvfIcaAxrDq=8jZ ziC(w)g7&Nzxf|FuiowAj{MmjDH8>DZ`6~cBhfPNEApvOf;f;@=X!{)I+Oy9Bnc~NT z-(!lP9(l-1g`y=jqqA?RERSyQaiHKZ}g_V!Fkx0-oN}y5Z30zHx^UtGFO+55EaEHz|#+I??Obz~jn@WaR~H z_OG6K`DP;bxv&JFjc|Ql_t9NX_>2EcQnljSK|qwaqdoB}SYyd$PY@-ke^iM-MCYb=6N92Pa*gxspg@0oC@IMGH zF57(#sf(P&PU)Amrvn}yY`Tu`aSz>%Qph<@b$VC6btZ;q%wo^ZwdPzNt)@a5hWQzor2Av2||s!8)+yWC|OM zAoWUc*%_C-)%OE*W2Xb{gsGeyRZ_`zr77i;WfG0+Bis9p}YM>%_rID33w$a+SH?4kyQ%QqiW#rk~ z<>nMCzo3DH)Ql}S7r1PJD;=Dqz?Xaxya6A2^Ms6n%_Q(bXzk%w3nm{!1nYgrq8B83 zG156US_WB6R22cIFdXp}xDr4Ep^$l8EFzCGJ1>V4Rjy*Z@Q;49W80q=?=wQTn)LO< z_!hl*#mzfkH$hH8!Lv<~^o2~`ipol1;>|Lx7zSp;-7m!S^X&b2c%e^PLx~&_P{sWZ zYAy8LR}eAOeR$jr($?B}CXpsQ$0`>nFmyEty2(ZQa4IbkQkzRc(>yJIg=LL(Q1B`n zL>0Xc?`2>N^dWjZeHEfP>W-W^?(U9krKcD5Kvo?Rw+V=eQ|#PY>Am5c=5s*wB`Dkx zaxVY8kn`Kq?;)d+p2Kf}LjxMpbPTdR)J=m4I?Z;C%eqP6RDbWWSwS%Q6T+*G42-~{p{Ae?kQG!8#l5WX4 z-I~D?M{M>oUshI@BSK64SyH&vHy|`}xa?}+*AA}l9)@(Pz5W&qIYxzrHTdxf`2?kE z(E5>}E9$e3tFOjp;fK_AGtvsRMt2um_y3%B6YruI&jC&uqt)&Yfy||EdSiAaDA~9X zt2m{;|4#XR`0?fkHGKGo4-kGn+B&O$LbXU(@=@JF#7nf#XQmn&e~W2!|? zRe^sjoXDSn%C`py1Lr?HAkvf~2&wD}?&Xim?)B@cH~b~3<4&0pY?ea^`6FVQtbY31w->hxk$EKPQe0WCC?l@RWPxfm@b_vgfq(!kHjtVl- zog#PV>ZNEf=QJARG5l`_!05{{=sEB_5PE7N40<pk{xZ#KGlunyg0Li0cV`DSwy8tYx%2YgtY9;PrW~*Ep*8Q@eo7(NUaS4I|{1Y6V%nq6S9Mig!URTJK7$XS(Z5S@%&b3j_|41(q6s5ILjNFFi<8zWpm_qsFsjRp6+RdxHVdAoVO69!K6X}7K&yf;WV z+`2Tajx>$U9)zQLmx+vwFr#*qnCZ!6gLe(@-O6mly}J7GZDXI{;-Wu&fiU#EuV2%h zTUyEx)T{0Qc7!&WT7~E>GnmRnCd1jxr(q)o@nE4NAQ$-Y;WtsP^HiVeeHA993$?tu z)vCn&%Nlr9US)QVfYWc1N>~FZDJ5BJEW$|A2W+-~DBSr@Esd`9ZJVji&)%FRU+4eS^|~ZeW?L*$M2R#Y5Sx+(>WFI1s9+&BqkX0a8kZqko`B0#Hh^@LPD9C8 znP*;>8-i?`s_}K{!$I*2A1gFQDjpKEn5{|AspZe4ikitXe0@Pj0}DidOXoP)k^n&Vny+`lBLygJzn= z60(@t6P#|C9qkN)j@0kUdP(m_yWezZ8h4?S0gv|E%AYkFG6bBJL+O=JC_ddIfLL6? z`kAxAIx7<@yzop=U?4(G4mEE#+d2#%9uk45OyvyXY+o#TzB9n;bg)FH=!Ji8$`-F? z6uuZ{Ee|v(4ks>)O-T=;HLGW`(vQDqkzI`&UEh~QXkR@E-V?XTf>u) z*Zlk4P>Dyi&BH*$s}-v?4XZj&WxdCze?U%J(#&gAL}so_^0R8?KxIf(eLM;iDn^^b zniM(e7{U-~6>}L0FrKlz;-bN`sKZ9#W&DDfMl#_>7NX6Sboz(%ji@{V?%6Dw+Ei{O z19g)Z@^Y1HkZQ%6eW?XbxOacd%YeeM$~~DC#Dm?F{y3COWf>W4+AaL*kc0zD?6ZN= z{>aGqCyu8l(#Kc)YKz1l05K2&gcvHNX3P!}Ty+L+L-8KBjQ-GsAvMOKBH|B(g0%b$ zOQa?e{b0$JMUd&2B8~*&#)RwO5ZmC)fq$P@l&Kk#OWlU1LL5c#7e%jJ%NUL!+S>R) zyi@Jbb|0_U<*0vtWispA2oMhmP>9*k`0RMnIBAR|7NQ^EM+W(! zJ(B+Ky1^1T5qO#Qi_kWUYyXt~3)k*9Y9e+rD-xQBucMWNG8)A*9~?Gm9i(weLFhWD z#qCJWROlNHHcn3XQ+!F$`*Qm~rxFX86?syTTR(T5z8h$%0x#yNF+n+0jN4#2EP9)rq!GxyMh&@PON(XbEV!qf^fG*rDfDzzfelx;ncUo^c&J zOXlWi@t)IKcP%#x&ryfbfG-Zktc-{4#hH8=eZH+j**g8yUqLa1C6MI+kvW#D5n=_yMK$pwCr+Ng(V_DL7_B`hnrYF zP3kg-r5+XQZ=Y-Lg3%HM4XGmhkjD+U(@*~#EfGatjRoLb9e>BV2u3g>AbC-ih+{^8 z+fOw;!N{09k__%a0T~$#U}mH#2+VNE(&UB~$Qs6oLq8>UcfX>dWxZx$53PN+A#T-i zIct4&=T@fEh=%5%)-kciW9*~_yYVbyqLx@-2=mXl34b;GRQ4`8!4ya4%}cVhB1~q1 zLK$S%faI8DF$I;YAyCphyQ*qHx5OVEhGXcH@^Ae!>xKU@Q_u~Sp4USm{Ue@qX9jGv z9NluQpBsgF@*wyV%MD68?&kg=uC2#S7)Q2Jz9>Bkb)4cemYnUi_=GiT(R*tE-N=Q@ zu{mH+X*rPJ7yRTw`kd-?Lc5SQW!X4ma!__HJ{T#3ln8PvD@tX2pqL=~GJm}&=I}zd z!EPWD1H6bUBPjBcN_tMtu3SN&Shd;6x44C}og*S1igM=2ZKg)&bW zK2Mx=u52uyMxd(}$c^~)qz+d0Kb`t?$?9PHPTmqfJcz~sQP}gma!)dkPt$i&+CPsZ z){qMrD<29nlfsP^m6TZRO+rGY3l^RULinpvN6GU#q;pUNpQB+! z<~hg`lRXkBfN&}|RhS@Tq=}Fx;*P`bR`*5UF3F18d(o4xDLErppaViG=&GQ??N|^) zb-F5-70DFM&d?{e4D6ZPQ>7^q{PjZa$CLmp5e&MO{KK!#;*w#|b0bOSYb=VFiL(sv zk6!=o8X)fvY4c+empCs|@Be9bvi$^P10_Z?a>8L5q{xt2589X`M~42wR4G<|mV(hh zI>~qL-RVPCHoQ)6>mtic?~M*QF>6Gv=}KOUuH_=)OWx(UfT1Mj-+niMl}&9Gm5C)n zPmffBqr$y-_{M+(x0fQ(6691at$zcU#$Q^waXiNj`=A;l|6Ki``fp`w*rO)Yal zHwVTg!E=vEVNQ6I^lSJGVHS$6m`P^KVX~jeY2+W*EBs4mwfBe&uYCuYSQhq5N_2`$ zlefAAgwYz%FnxDa3hF-P$~S7B$)j`PGs7dMD6`$k zlAI`frRX40U_&MtGlq6O%Qr(oLs{RB&oRWp!?$@y(rW#td$`}``ggoMP|;v})7pJ$ zt^-jK4SLv6d_?7+RM=?5x?)?8A+gv{2Vaex9RHVJ)~TpuriwL^=&$PvcTXLMrkTIK zgSSKnC-Lt<@GTHhPDF>YisdraE%OHZUSb3;WkO<_73DwYi|df_I~G(LtI2S;2q{K? zk+BX>Fs2mseFIF^Rx13x#S~HTt{)g*70-X3kLw-j)A2^=_gHzn%#!=n@Pkzyd8exu z+HFSsZYh)4CNj5fJ7txE@5}8!5*;*3moc57Wh*vo4$6b;{9!yc&Ib}5r?)2=!sOuG z@mj2Ts zdpyfg%+&Dc=w5)?q$g=gu&+p{>{JcsQtbQqbdB-5%P)uc>@*ODWUSpH!vOqd^wH1~ zsuzrmlt~(>X>{?&FKh&5aHvFu%$23J+>|HnIAbwnt=%MfW9E4VvwE$|s$NC;;Ha+Y zi|bckv@s@>i1~_OTCW(F`7&X;>kxCot(&C4M(uL)JJ5SR(3M3rHu7-{#*jktu^+8J z77#kMLZr7VL~8m&vj$J^jR4Z5+(lN`2?Ck*sjqTW!{T1I2SLmObTq3j!JHc?R#b+rIZm!#d~ ziis?b&Z&0V0zB8$W31Bf7gOo-G=hK~Mfm&;<%UO|JB;?dUPx`(UhLmK;`sx0ter47 zf{JhN^o2njJ;drQN4omO2AGMi7E{k&D`(*hRp>S00w6L+-sd|lD&&{oO;z1tCB}@} zz%uw8pkPuY%;twUH3=xod3Y(QJlV1Oc_Sx5n@dAfDR2MlZQx|(#t|x3Hl>E0MWzM0 zOKRjtPX(i*I=0d1JMExF&pnoq(Qaa)-Op9Y*@utM_PXWk9)b(hOnw#H0~T6PF&w z%RF`T93#!<)HxtQ=`>gsX7LMIbLc81S9YqG-USm=dfVgxGfEXBtqS)5b6`s5=dW6; z^t?&~N3mQh3|uqi%8Y>-toTIg-!=#)$O|qR>jYSsA!@O*EuT^wsG_nj^AAeA1nr8O92F z*K{wUk6v><&qMVmu$5!a~mb;1Tw`!{!kGy#;!C_?6 z`099gFe?K(!A+Bm6w%(Fb`1D;goap`Yb*Q37eK1 zJ?2*zUns8EzHTeOdisM7R+~<)U--ca*%GsNaMEIcTG-+zt7e71ZL}HL7Ys#B`aJDm zcoTEkHPwlWL|2AB_fq6AqZ^vO z2E7`wXmtLkQp=#8A*i+3{z}W(=zAw16Qw}wsIw*$?b+=x7*lSS1uIigW}XFDHGm-~ z6jH{I7M;()iu1YnT2JkCkI!SwozYe|OkKyol0f}>4UDkC7nvosM5mtKXPb&ijw8Pb zadODcK(HlJ+n389E=2N?eUAb&Gg(sK5k%PxMVXqxWI=Rx>Rl z5ZKb4lT!Si)O>r!utCO;qtV>lRgz1}Al=)OTE+127o+rZ+?QsNY5aiqzSJ9KNy|6svlMWPiq?g94L@YRYbw=_P&l2f}Yq~5+QdsOT7K=+F zC{tvYlpx?CB@ z4h@i5>i$sFA|odyfIb3g<@5ww8mTK2FTVTDe%gURUM)SGZUrI`D;Fa{?%$f+YT#%F zN(CGSQpO(SIgL|O`WeO}uY>znEtF3++F>y9n2W;5<#QrVWUmPtGt>=iAl zNc_PLa4w)#^jrU;g{~W~f*v8yKzbR6B=i;e=Fe%fp61(aRv1+NwW%4h(zNN1P)`P- zH&^f+ceT54r2w|h5j)#v_x1B3n zWNk|U7r(y_W&7L1Hlm@o#HD`15Y~D(k+LIK;?8`d#k#2(BkF?nblCd_--s91rdRW6 zp+bFwT&*@G^`}jC@)^JB*+|*uGKj1lB=1qynR~tAB|*aRQ2!W%^%=qGQVc@KE-wDP zpkR7g{{ziKWyHJjFMV`FZtio>M1$l->(Dw4WHn8tcQcmuaPhy$kNSXCqQlgQElcQ* zJ`ooxt9~ov$?^_KcgCc#fAfbEjR$9b7Lfhj9siSHrwT3V*K=ZmG=>JWqhS z@gK>wtm!;?fx)Kx<>%IL?eEwQWW|>FnR@BRad#?NB^=$rXdG|cz>(Oif@HJn+&seX2wn+}W_pOdl9CdKj{R485EQ?WQK9ssil>oK z`U`Wv36l?TY%LJa5*^?5=;v#N|2@ODtpBhn^M+?(Q)R`;d^XX(RpI+!vej7=r!5bk z{^u3bqdJ!(EM!x&IB3;ji9WEszxbE<`RJ5@jom6kG!EM^?o`guO!NXK^X z=;&g+DyZZM>vzai5gif!Lpncivr|>rSKWi!yeJalIyN*h&3-eBXDBPz+CA-nDmJw|N`K|hR4@?gmqsXtbW=H0Pqhuo zUVpGC&S6Vu5Ex_UF#2UENOW@3(MdW02>_q*7@L#Tj{Kb(UpY9x#JNoltqmQS>N>`j z`ivMpO%VPm&@rJ(Rn_9~eztbtu(unz8;3m@uAz!2IKnu2pd-q8=^^?<_iP3lzyE4VWSgdFoLk=e&O5Cn0%LJkM`|;Lx_j3lFXB56Tmph za|fiMUxLu#+CgRpFy8WW6mX!gm{2%tK9SUvLTz;&?xrv$ z=kFadiHCfqME27%lz*L*UqO(%`qmMF^aH(bi(J8%UuMUIAd-AY}OwJPno%3DH^cuj{&3?RdKN z(7Rl0yG@uAG8$Dy|AX~Q&0X^bL25@Jmm$4VslCu!-IyXiC}}{vSm)Gti6-@|^!%i} z6or&s8*Ya5HR|jpyP=O|?{iv%(@5AA1D}!7(=iJ5fe!}@y_;Lwx;nR>laxFZa^0ZH z>i0kroVxZ7n{bLiN0%{~kmv(FL$XnUAFB6l((wSdW)G)g+Q1%LQow=2&NABl0#jF4 z3~2x^_~0dj>(ZftY0q-DIP%xC#fNAP;EEK?gQ3cSy6>ZU{FF`fjd*(GVsEp*KT$om z4mwl#QUHK+)ndao>4kv0_I+f;m6Y^1k52vf1%O~l{XxHU#%U_$ycG6XZ*Xsx2zqKx zz9h;a)X?*5kunq1z0!9q$#usdL+6cpT6Mq@|2dd|JgnhKUvj09c=2XtB zAMV8qNIKa6xZeEwWF+loJ2oBqK5Yalu=K$vxz_wMtb-ovl-9ZIi!@G86=cx8IX*u>@4mgRC-^%;ylqap z6t;cBM#DyjQ|5b(tej|8Wdp1uEV`llHoB*o)@%M8m*BS{v#}js%w3wx4Xgmu3ni9U zq0pdHDE6oi1@V{fgQV~QqoXR8ImvoLvYB5fsqCJc()O+XJyG1gJYhdSyemNE+j{;3 zCsl9&qw0acQ21LfSX$K*vJ9D9<6nF;sZ_{dnDN^>?n#X5MNaGH6sU4=(wXWLG{WhL9nBGTghXDAw zJ9=jJWBGemS7U4^ey2?_>%yL%is5Z=>=yvQ9y&o~h=!98z0XokX?x)3(?M`{01J0n zk9vKyD7X&3;3)6wIjtC;qEWPAFT|vvcWpE7;?0o@1V=8!EH~wq7B9lNT}Z6X(9?jA zpLo8&{he#uqONw$e5E^MGvEe!eP%d|#GOi4M;DARNzpNbf_R|{0$r1RwMqPh_#Th< zSXad*_n*7*?NG%g7xYL*r|3%xvi7&T5S%kGm>c-H^Vl%4Rqxd155HAq2w8irvHsPM zEL(75GY8N}>>c6|4|xt8_2-7MV;_7StH)zA)2z&Cp5NP#n=VP#R(=JxoD-4_mN%pU zz|Sr3W5wms&kQMp0;j=Y##{&uiTUcv0@@Xa{;ryg3Fot>8*XMQ?;)q-NgUjM>zBXF z2?cz|v06wbzn>_Sv51qmWZjw!l)P}_&o84AQNOLdy;ixMc1KGV_a8~$k=a8*9Z$$p zeBU!%K`Cg!&l>HXZ<*SxG>HuV*h+}^1!TRaZrx4KhD{t=BZ9JADmI+ViSGd#=*0n4 zf36&7^zQCh5&v#SDZz&=`I#2x;^N}pB=-+LX+{A<2Qg;A!HwX0)$9eie!$4(PUHO% z#pwFW7k7Gu@Olp;*B1mtSnF-$7)jah(HMB|3~ie4TG*Em7qacbKU4han`WM}XBtEy zN$csBOyji0GK;SaWqP>(DbDPAIboaY?lIoY3j8xC#4B35h>(S?=GUnzszt;54fHV7 zbqblZAtpu5zxh{%%*65^sy`)PVS6;`urWEd%GDZ^SP2=On3!Y=A}M+@zhmf#4tswQ zN;&a3jDa$-nO>n72o$`9=Mk%;1qcp*Re0T=^p#DE95uSbVuP5aJv9lCS+umYR2rXw?f^d&lou)0wML?sa~b4R zvB$@sps4(skJ~UR0*HRopQIfh3ph&z$}^zJn$XWntof){aXH}4-dJ>y?7WyAJHmle z*j|F|F}R8~u@laXe#9r=4oQ>HU5**?T&AdiD4-Y0u|aP`EavA`g=|DbI7zCBXSgd&zr*_dM4Em- zWlxc-$H4k+F}-TK^r3caGEefvEi~#dLyJrBABgPi21J ze(TyY(^pmyHXJfRiX0a@xx7(cDJ$CR`khwtBoX^HGsVZ36z5$*?POJCi;NA!TDh&3~VrJUrGlu#nl5x>cZqu;_bZ zi+EJ!Hq3mA_klJ4#XX$7RaQ>2k@B&0he zp2O$+ThChj3#_?w?>*=2_uj8QH}JrbFsxoc9sLY04yje@8Gx}Tr?P=`z{atvW> zd%NFkM(JynK~_Z-%K?Z|Ts{CB_{v-tFp=sBYp@HImA~V|q~h`cZb@}+|GcX7SFZwh zl=R-|?9XkYK+xUvRSNrtfnOO-NlA(8hc#Q)U{o1EIuPzfyQs4xJ02~+?(7^c!z+Tv zaRpfvG{I3Nej51AL4E5uT&FXtQxh}Ss8CdFhwTib=cZ37JJi6Xi1a)Bn8q|1m^o2M z*U8W%{ftg!ipZOMpw_%zMLNG6jLXKIz@J|I_JSt>bKe&$xYlR{@mqfGwQF_h&iNo3 z*y7_vnSXvwiW}c3?^-z)U|@5no6Ga7lo6V{1yauy{B6v?Dz9~tsh*p7tW+6Ghhn*t zZ*7^#xk<~Lm={2kz|3r_NQo?1MTweRR8J>?*OB+;n$oe4;T=C~0^*521r+)WWF>!t zyW=H%O}rEtX2w+HvsP??>MS}GO<=vM5$|c$OlBQmuzvWHte?~{Iw&u>xOAqvtb&IH zN=<_l+bl^;pGy+gWYT=>RxVh~kQZz*bM20y(38|#49;&K?deQ^A6wuk=_|kW$^i49 zX9kfbCP&e^=s!f$UbAoRmNSiS(yC*^JU-8KNfN5Ye$<7xeMhr#Y}6~P0cj$xl%V9W z;*-*NjegXkiHuInS*$rD(AE3I6=9W^Q5NC5L5rO_Akz`rMAqCC=xZjrs>b%6r_IS< z)RbvwlD<7uK-CeWdE=`M@o=9<;TzH+bZXih_0Oy8tf{cuHp z?e92-e|NXM^R1fIG1|Ti-0h;BXzyF*N=JE*fa|%W%fhm7jwZo>yq^?Z;RG1fyLjkj znS%|7@pBuoDhmXUKU3$tbm*YNwb%&^`Wv8zt#q1SiVam@Q!U#zc>#B-tqFMgYXxG; zB3Pa`Z`KbSnp3L(k#v8Xb+b^^2!gT^1k2PBM;Z11CDuS35uHz|Th`6`qHlBXJuDdiO?N zHmt40*4C1?o74S8{m7y6l-J(3dVIGm6&83<91s)yAg2^2w(mpC$)~kjZ0ksGrmyX< z)Oy5-*cgw3d#1wh-6a7~*q>iYhXfCc9cT%X)yo@ef4{$L z}OAz}dDgTu@I~m3>9vY>pQltAt>AHC5cSM~I1AEfNz|Xt(2vf>OCHwo9>fb&P zfO;vFBk1rjU&-`RiyK}|b;qDLc22)10GQhzF7o|*sm>-XcEt;J@QF1kBK!k?Dq2oa zLGv?pV71*OT3MeLjq#! z^Hq^?F}{8OK*7>NYYg;kpRK@Jw}Pkjl7TA6FU~masd>?6I7){VFfZRwRd&Dfo!Wm3 z9F_tkTI0bI^AH`Fvf~}ef$YRku1gTiKhimB7%k1UOEG%9ev~c%0-&GkV(0mG#FV`YBmgj%P~R`1C&i!F-x^Ttk}ZB7&^aI{^P}I8m}C-E!-s*8 z04p z0|%S?GG;<0#?_xDn`SrTQP=~aO~I8mG#egU+u!n1fLT;juuxR?R!Y`Ulp-+d|4HMB z^2WnAyerVv1nuv;L(SV)(1!<8dYg*SpTXX6C{hv>elh^T*25$`j{-l~{=H|3v^M7U zip8C>Y548qKaR7zs;iW5-b={)X*Zlr+=hA&`w<#clg&Tq*W=#q-(*J4J z&T3nO?sj3l($<=fv9Q)B*PIu1*aOC-S%Rg(-AgxHFIfZOrWPa&S1`18FUta5j!s1- zRs`wcVC2;egISPZUFr-k&s>>F@8^@N6De5!FR@b3y7oH8Iaj}8PM>toMj@kB3b&gU zeeYkde!MSBnY`E|OWky@-3bY)mB!}wU|#&pEwYOIRo7#^ULW~GV=N1&`0bi*7`JO* z9S(F-exB8FQvrXXJcYwNsxx%L!HljVF(HA9DWO8wm{jf|pFtr@>V{t_Oskns@9|1$ zk*{8j7>mz=4^*q`diabggh-kkxer8+2Q{61oJBR&I0(d>EVQK>8W<1(ijnQ#gx~u; zDt$`aGQiXjoK5C%@N@7GXJBv;CqV4q)lvPws=t^qVm@*4-z!#ePnd*}3){+3YDS{K9tqB?lIU}U_uzUg})K5f=rcT+1va2M(D3(H#(M-uwB(mx`2L znT*mG$su{GyS680t7-H9?L&#VGKz&E{7JWI69ocB4@I-!gdBy{0%anDoiF*ucGoxm zwn@U1mR2$uW)G0ec0P(1dmFtOQ)<{9bV`xs{0st^P)Mhi3gPC`xg}~v>!Gw+wh3#e zc1ck9nrcP=KdY+-CCMoQ=C*$-Pcs=Z)wL)%xZsNFS&Y3)Qoi5#NL-eklmC)gLIzS- zu*a7ykzt9g2L89iRg=q7dQHw*yRRzu3(t4t=xZ>kxAu7;Fahe&2ic#NUd8P+WOk6zXi~c41Y+G;6+@nYqxyVb%;2*C* zc?@!*^Y^z(Z{ap89|5-_foS)uuI50w(uJ%O>OfGs(TLjcqI3Zc#dBArq#z|z)- zn$I(eW2(7Flz9ma%>%GVY1kZpDFJU0wdC4V9X0B9Ag7|L0<{&D;$I2h0Jsyhw}ceh zf22OSG;q-~AwoH%&pu6S5x=lX84NwV0ON3DK+{pRQy1gE%xVnI`o-AwQ#bPp!2%cE zbROs5s8CFwBz!DDCeUXj$IG+Vh`UwWc6qOc9Ld1`2Jun?sdVNqHf%#3Yr zb$sEIbezCrpkrC%+k>wD`Gat3EaAVJfdB1fm{SA&oF0AC8ps2qaS3qCDvOkZT$pJt%%EdQi%=C7>_$neXSBVtW>i4M?A;A z9f3j2*#YbV2exKzbxuVD(XZH9TzT_oC$6j!n^g!R$FH*WE0R;cwB3f;FUf-~Qefq7 zo`WzySz?NxZV|jUS$>oin-dlV#JD2T2*mQYk=48nfE9<-1HxlbYutu0!nVdpgJ3e+WiNPAmu`ZJl$qzzdLRM zDJeA6;#^W8cVCiMS$qyspLpJ!3aTl|NgJg8mG{>xrvBoXS&Q4qr^V#n&C=ju810Yn zV;tHn?orfpWJYOvkJ`)H-F}BcTKkHAh(zcKUnU_fy#Y2Y&PctWkh9YB#-0>+;?S(_ zE`;G{@(wddu8_DS!X^G*ekS3{Pdsi~;0 z!C80=oSTiR4~ahzw<)zlMhg#OoDKx-^ABaLZn0jVtdKr0XrT#bmPi$eMf*Vwv!qaI za4!4qF*<=A?m==&fl1pR*Wv5Q)uhfr5w4 zNfa*6|6RKtlcnX@pXx<^_}~Y_eFqG-()YS`D7l=vtR*M7i#O*FaaDvGekNFCGCRP zDxd-H{QmapaAig+xSAPLVIyc5114T>)Eg$H|M5qTh@~3{nm(Dt3CI%9Rm&GE1t>n% z{h&Ya7c?Vtyt*S@Lhn78p{L!+Ck( z60UKG`aux-%RG0KTM+I)0k)i19430flm^_8fx1*qmi)Xte5<6=3H*1&R{_n*RtF7z zEA9S_(!alOLN}>Ct;e*!p>?9Lef;A*z}xn|vn4)S*k4M)?=jqlK%i+FBq_(g$`Oga zQdl0`;Ob)aaFtUJM~)%V7V_E<#eJ*e2JskWAoA2BeRKVJzWU>R-;C|aZ+==Xo^IkT zB#OOL3)d+NCa%mg*0{iT<(LX)P0Cx-rD*;s9loEhWwGZZo~$1Jm8;-{!QH8(k_zn) z^tn3;9wauBB*e$g2iJp0&-=zyR8o7S!N!(P-Da8SA(vpYS3{;>xFM`;7>|u_Csqxl z6A%*B?Z?)O3k)$+GEz1O6$2?_R^+OV8Lf)?KzJE`B4)J!6Ecz|#JC@#=I9>~5J179 z@PR7Otu~QIF%8vZM#;?V6XpCpRYmWp;911H=Jp-f-GhdmZtxg{2zexIMb^ZsejQh% zQ){Kph`?V=#HUKN2yN&S)0JMi8{MArc|8tNQ<`z@G~ZEmRd=)tm&(hwapsaGIfvY8 z__ii%$^83A{XV;{aS!&{!@Pf{aea^eH!izYn;y;)l_ls*G#YH()El)b z9%FIn_4alivk)AJWa2+r216=FPKz_^RSth&$TT zMNR!=!T+=8{LJE|=KLJFq)CH=H4mwdP)`k`gG5pQC7&5pBn3O)Y5^%^hcEl?jGgN7 zl9@u|;GO42@2?x|673oC&3wsY>!sfPFGCScA+uJK$9?`nvyRtwdjI@}JJ-g(*Rn?3 zBiwwr1a1Llw3VZVh(FqIplN)%Q`ER7^4=vSdAvEjwyy6j%IFveT-Pf3zOI5sO2C6 zk3ABEpCp6f<=z{so)aQ7k6I?{$XX)_@Cd|98u5Plm;5NII?Vj~jW1xK3ak1x$NbSi$_OiAKIlHz{`KZplgU_`SAaeu8MEK;LdLI~FxUeyl6qb0S7$ zR%5Wg9|k5Xr>mQYx4y;0m&WNOfm_C~1C6hZ)X5i!E+d{EEm{YBzkeALU!g zu>l#}cDK}*UG>&7me_wxizv57Q3e#k32qnM*l{Td55!f|ez&|g>G|1B@I1Rb=wZJ? zF);4p(&xq+{`X;P5Sg={C_9=@u^a%`bk5ss<$ky$+n=H|Y|`u>TtYGwD(vnp+w_h$ z(qE4yQKT7uOi6{WFx;o}NmuV5Tp)t`vSRLY)H&Dxz9oX}TSy1!6c8a#wi5b)@AVv5 zyWb;!(IsgZrl7-q>3V&fi=CXQZi<0ZvK;u%vIBw`O%82g&{-0~c*jw~1OmJ$SQX@3 zmli#XeBpvyDbmS1lCZT^Sd)?=3rXmcVQY$6SVr90j9hETroESxx6YK29!&Q_Yic;j z?-I!u3c%EV9fX_Q1*CqH!lyou)pZ9k)Qc(oP@Y>ls!mec1(7FE#r5hJEQl~9h&M0O z{xp9AU^SHmKO2Ys4SPVRkxzj#K_Xf;^jP@#v@}EFD?5J&m<9|2SA%a)itPL?QHt9w011YNNX#(&3uBt48q09rz@ppYRPl!2C5esI8%>|%{0 zo$!xY*TVzCIU4ZO2Ioerow7`TR-GlolNJOV=+RC=mi#!PxA1f8HC98lCIU;<=1OZ{qBG8b+FT3=7bedR zP#SAW_^U717rJR-*d*78l9e`jtgh8Z6iS75t36`HFY`zUM_7gEgYK)2C@#Gu<`clM z4&6E}WF~$PmpKI{5c-b-TScXx?N3EE4$p?}mZ-k|&!5yx@P_iAr^@EysrggR1*Bsu zqf3VOV&a=w7Gkr5uBS*k9V5Hwwwe(QW)N?h^4)%^ejFl!YS14<4}HB?W?RI{It)6x z7~a=ejFRo-BK&>)_g+FdrTfb^8O58xz!$^8FcFcxXSly(FH>Q}btNxaQ!OM4=~saxFL-F>9C+02gb>-{CZarmd=_S)8Q(+gqMm)aHO4u^jxy zk$fD+*Xkx@CN@L0IQNo0!Z9n&K{>gaO29gsR%iHAgkoxe#j|nb^r?Gh=6yHhfS6`V z`Yftw7n69yn>3Kdwi+_G9( z++15D52%3FexA2t2s4*3u`;o-8voM{*7NKfe^H$N|tPAKR_~VE6)WU*%+4OdO6=pIRz$ z?_@9qi@3Wt<&;Wc+Bxo)6lLMosKL+*cqJ5_LHO)B>aMst=wM!K^IF)!4(+>C7`2;_ z&)vSw_IW@dPHzkq6O0o^bpJl-HT?ErcnQHKn$my^GC+oM?@jkch>wC9#fa>C7bae7 z^#B|LtrL09;E+Vu>C1haCeb1n(u*`olL_4Lz^U*Bp;xI_io1S0?;0`Swc85czahK> z@h*QEsz7W4uB9G_E&hpm97%S{?a9~tj;y6~J;r6R&Ti>?x}&JSu&1_|TkYitm0`V` z`t1Zw=SM}oNq5AHUST(MM&5IR+KytZf$E+S=~!C!)~J8?0A#t#AyXcYjvXe0ok|o_ zh`t~w3NF4}9$JY)I5KL%vL*!=*MBr42qM0Hl$bBUpXc#!U+hhn1wSbwSP*yKnCCo- zBI{QU*gT&j;KPpH+PjJ-IsWejDe(wO!rTcyR*8{0^h(lp~;hvPcQ-s-10S zXsK!gFuk)mC3v~ znY)tU2_ywE+BJ=g58gTSS&4c4AkDq=4v*vcj!J9gWFLA4gQ91yn%yoJxbdIF#K zLCu?;*jci=TDRvYln=vwtK=RE{iT$G$-oJF$dl^g6qD-pKRqlaq9g18_W0-KJS?pa zO$aryk~AnlCBeyKjm=;5AI5X6NZ#J{rhVk3k|zaonB%sXY#y4-~)&3Uh8}8^AYuicXV@-`Q>U^obYgk2aJM~yN!w~FiAG0 zoD~Yl+q^zyGcys;-Xzktf-?cfdNCD&YJC z;4b`+k8R2EU%34%jQ$Kv-6pzx;RD<03{<0SUWk;Ie`W87T6%!Y=}HPlX3KoHs}!At zOW~5#ac{z|`1v03kDTT+367lX8)UBdWzcu^iiiyDgdyBE%jl?RrE(_yity-%=!!%y z&$cXh0`tDb7>mSyR2-z_=hNbvF-qZ-9(a!+^ggsyy#yXtKBh16YbeRxSgT@`2>45{ z)o`=MTNU5F*lAR};+M?N?ZsHyJ7)*%9k79kih6d#ma~F&KW-ETX)DN-5###+9k)pn z(+>;^LM0d7eacO`N8A2jffyCFektHO`zm^K>7F`Q=7W;F%34xhfX6q8g;VWD!P(gK zV3C2nXeTHrJNVB`FeY$(Qdq9vV?*&6_N8=6DHZdtuH?IimaQuCSlpt z)$ala`lzz*komg}{2SAAogy2tiJ+n*QShF)!WNpbU^e&#Gu0r%*Yug+4}NYI5$JKB zPQDkX!8}}RX_|0Fzi{$t`ag9&(g(5`2OP?Q8Zv!(bxe zmRx8kAE&C#e&IT2IO0GPi=Qx(^(ouGd^-SR?`})~Zx0w)B~6{yZN6J+!kVq?YG$D+9U6cccpI`SRPIb!Q|R43CX;y z^Vh!m@Nul0>A%W?pF*=1M0pZAMDE!Q7uY@1A}4_)go{htxx&;FGEVaMC(0X5mTOn7ic zYFW4v8`Jj8HP}DT5^%w+(fm#QL49&PjRbZ6dF{vXu2tSaTI60m=FkwTOW*Vo<7+@t zCcs1(ME4~@kDC+|_a|0cAli#pf}a2_tfsU`zMEFb3%B8PCzStvft5m#IG*oaOIfus zcO2pV`T1g5g8_&p(bftRn=ZWY<(K5meH2DJ0c4heZ;VSSJOt3@mvhhn$Z6wINj$7{+uN76T?~c_ZK7oagSPdiRyAM~_5rdBf826n>->DgkmW;6A zXOg`!8!)s>o3Bxp z_g{rzXbMUNl;w@=Ak+ay#Ola2WYuJ#Vjd_LYkhq@mRjbTlz(gl*MFz2Kf{2OJO#TpW6*0oPyGzYg3h=xKz&(T)!! zU3|fs1=5+`jS#UqPU}TY7EbyD=cA?O_s0x6EufRp6R*e60JQ37VH_25HM6$>9loy$6EXe?(6_2 zTF4{PBCQ`pNXZuRw0@+ELk(tmd69yKuI+Kxmy`c(P~6?NP%ze8v!4vO49F>s8F%ID z;5}()@2#T|9wPMvs*5zN^hu0wFIopYZ8e5Vw{~#B@M9DmF(t{*-`*jn#Tn($xgzNL zhU|-&o&7rU0C+Uv`G^^0X}@WOoIjVH-iT33>Y#=_;Hr-MmUl3?)!O_h9! ze3h70BvZW#mKA?QnBDE_jQxKfF3Q&pXN+}48ildlM4J14#QLEOVvR?0JGA z=8l-KE-FJkfo%H;7P~dsZ{z;C$!tl47p3p5D-cFV+mKy7D?8VyNpBI|=3XL>&V9=L zT=E0?-<>vj0R^AH40TvdY~{HFd4bV-Y2%oQ>8{3_8Ec!lgQAfM=a`!=K+0g`630%*pJ4{4lA3UDXRYUV5yV%eY1hP{pRb>{KX&+)T6=)nXEB!U-f!$&?r(~BV<(3Y$VdN z74$=s^wO-BTDklV2C$%V;R|wfZxnC`;ftYYl&B(r%^JZ&59e#pzRKJuC;D2Y#}8Z{ z6cx;#UcuC7fU)w-(6CUzBk%Gf>3~fiko?Y@3O^D6WD?iFI1LDz{BOy54H-D+h(bY9 z<@Ov+nSezk>Q!OrF$>7JY0E30mp5c`#9kb?ka{dn^Fwpy3|j4_=>~qnq|UQ9Vv|b< zeur;h9$u3R%35^eg#G!%UwC-;`ouw4;*#s2MF1%{|4x5qb$_F*FxOU34jFC+_fs;D zwiq(lO5|bVvXa$+Ert*gZi?>}`2H|ZQ3gV@DN#|7n?OX)2!JV|+OB-DP5%JDR%9L_ zsQ?tatOMF=*UH2Fzl)~Ji;H*2Sg;f*DlP+nE37NhS%@l)qr-AU#o+j2Y5YpW{9o}{ z8-P)X{}?&OqZjo$@9~1v1w_jV6fyykhKL~T~E(8BUB`Of76K(TR51R z4sYX}7j}2Zm`;RWpm)2F6VuVlL*gBYv=cd~f(i@odOUd_Q|Xht9D|4D^M#cVu4`~w z{3zoCswe-sze_VIZ0=fUok&sau)7)A{R}TK#4;2RaSgbu4AU1$R>bhT^)oApkd*TeO1c71Hzeh;k2@+QDb`)`1y$y;YJejoI! zu}|DxB^R?HEA`J_xcu7}xnzv$O>r@?^Wm2g zk&x3lg8^X#6lB<+Y)<4;JnVr~M*TLyWv?q;P(v$bOPJbA@;r8{6QHW?z;PMihF<|R zNs_+m=D9E2b5>BkmCn{rcPern_}>Ocgc_{yG-v{qGQzCd)^l5t18wHvipcjSY>SqH zQmr&BNNhezQi?4P|9;n7+dS>MF$Z&_8+Sv8A#37Wn=Fz@NVtXa0Ujw{Z!$O(Uw3(# z9hO6h-d|t54(h;(UnSOX)RzzAcIz{HS{u?ciK&eT+r5HQqJuu7EiI}zAE-3Ulvgls zlz8IgV23=N`!Pv)bKVCN_Tyv5;>EnsNv&4 zZuZwu^*b3Dh+;w&@J>t6tHLnuijNs zxvg6{{P);MOZ(OwQh3Z5J1=dY=h&h!e_6HD@4PB`aNoA9g6?jP!h{}!Nk3b6CSp(g z!Qss@5q8SZxcK2Yf?oq}(8g6?w<@FLP6}rRedC+6dI5G)$ zQo3-NS037GHX17Gv;N|{7Q%Z zXpW<9OTPgC4S0O`S5b+lh~~`K*xZbkbt%J`PZbJ*Y>sDhiU4JX<}TD?K5>RCl1@f1 z!YciaLb*X9tibMaU(1+fvg|z|q5@G38$4`8B!swI_F>V{{ouEbLCL~8;DIBcbVyK; zVQ{*@!QED6_wwI?#rwafrdJxeTSOglZy8WWPME#y(}4_Wz30pD3b$S^w5r@x(;834 zY4&0`YcA&G2?&4H)#QaZ;~NrZE@mp!b8v&28cTc8a9J{q0X7g*jP=3(<)4XCy1P+e z3l_6gwGJZ4Ptb@*B58XJbB`3;Og39lKTK#{*4LC4na_mCER?i$8v_84pqM2X3h?Bj zE^Bgio57f`ah=D|Y~Ao9l_?6Gx^HZdDKv7GWS4;v9QlT6dJ%!@QOjxW^5#gm$XR%iV!`{-NA1{C&c z=i<HX_;2u=)VnEd4%A`eIkNs1Qh0kI1Dj%?@-RRmEUPuOY@z^6Y_YFus<%D z+p2#zWyqUaSFo*68C`YLD^`3uiS}(X^N?e{@$2Ls`Fij}y7asTS-?e6UowWUd!s+P zUU%!;z{`Wb^;QdnNLYUbJ`xML*N6%uYM8v=Npqf|q@=v4!Ilt(FFDi>ABgbHBL{XK zdQQw@&(qDpC*Qm13Uc6yTvrHa4O;_mSh07w8p_v7-^tZp*xw2Mw~*42_?>@Qz&V?P zZ7w$XKbwK$%kjp zM4rh=h^iBSee)%!PB0nBmCqhC$DE(B_l!Ow=c*-f$T-`SIi%q^VGRniaHU}Zzs>5j zShILqolEtxDpwucWAd^JvJ7sLqN(a4Q~nzlW;IdUIp!R`pE&Bg)^W` zQVMT-E~DGr9EuE)J&Ircl9RenNE{dpzg~+;j*aRe;D~7Z-74jE?KsYlJ;Zy0HGy?! zb$-#-s_&?mnSdGa>`&ZmntGr}XpjNF6NM^#bm)2_je z7xl`}?pCMKp8ARD%vI%bq$fH+NpSo>Nn{l{CE%HbP2Vq)FHd91nZ|cz;Xn42Ds5f= z)-8-Sc)*`{Y{NEi@e~i2hktt$>$Kc#C4|U$_c!S3Q=C0mv`)Tta_?$46mGTmV1!1d zB}A5Cw}%9(xAJZFx%zl#Yf-Gl^M;&-o-_;N1I$z~_bp`VzDA8~nZ_=|?H2oV zJs*dw6w5RFbx_QS13{I7!lz;MM2ER>4#PqGd~H-$t2|ZZN*lDe$NI6IJ+Exi0(|_+ z-Z~oQ-r_wW;@ZBdRfZH{pEl)o;>iRb0eNSOEnO<+dmW7&`KF{d75Mm;4>-FX56T)0 zP@?O;egjqH+#njtUsQ{p?wIPxsO+gt$@C2#+!kf{_^jHpx`Q^}<;?-fi4cU%F`K^G ztFe6_MBDivs3?fURhJp@w0f~WD)h&?zflOt`0NhXJR%hu9Gf%6e00kWg%1bEeY2`S z$dodmZv|fV_!PYB4E?`-w7j%Pcg-%WK{1^(ZHh|r`8lbA+1kdPzN@av_dukq99q#tqhZWz z??R96{=&s8!CB~D^Smtf(TEQY`0`WKkoL77ouGh4u2eZ+hx{9z0`elV1xDZzhFbO8 zFv-GnJ6{S|&^Q36m`*-;-~GJWO-@l*#2W~0E1K1sVIR9k|BH*Dw zFmJ63Qx(C%-c+AZq2KfZT=^$)QCR@KQ3l2Ty-R0J9P_#3V41E`fV|E#fTnd+Dq+ZW zcH%jk^%09UH?M#~1Rs9r@v+dofIBKZi1FUT{y|FTW$=e;S#oGD^0=&mi=4TE`k%ln zVVgXAWw%C{UOfqYy0-y2!HQ@{rJd~wmiq5CzH|xSe{_|D7F7y`_8Z0`$bSJ}xI*QR=ZR*IW_RgStz+SQwX zKihk8cE;nFnm1g%i-*cp@)R4`78YQV?Db^#9##yu919&Vwm%IzIx8vB-`A+(Z%M09 zRy5J&kBm?YF7Px&IdT?S{ZLx2&U|s?G@xKoWGRjk^^9ymH+Z=u&u=sS z6%|--Z%pFw<3>L?C`L2}Bx$&Q9clA;&L;5K5%#T_?da-Th(gj5wmYSpx-#hQ6cst1 zkYka7_xCI%ekYGfI5-ZKG==u3;$OxVgMz{ar@bT?6VY;l_5U?u5`&%gf^NRXSINnBzimT<^I8@hR2 zS!shZn@n~Yr3PIY+2$QdBzcBvzt`9*$n@ExIC%4B$mg+dh(t`EY zukNo|6r-|K<2)_0QZ)Y6hNKWcw!uPOGau-_w`spe^iccCgY~JemK_jxJ8!bA4}fpl z5WFLrdNqC(6&4zSM$>CB7{ht}jqE`tna=^R+82uMz(<0R2e?`TiT@*L=1SqZxds~A zLw4)gvx>Z(+iSSEm}hWeN0R($#c&0baEUaC7N`(>Jp_rbnj*@_9x9!yW%!AOA+lAz zFeYDwm_iA3*%-wDUleSp2)qmo{$`75e|gTs2G(qPwfrcE1u|W!9PO!#z~8?`;=byK zkl1sRy_{3- zAZ3C(!`) z;gGpc^`YevXzphGi6~@+e9!7VC{UNAp1nm~3?d~E_|gH+z6T5_*VII^{&_grAQ7?z zLjom#%6dcL&My5rzfk=_F*m)vVj3sC5;YvK%HM3_@!PKYu1|{TnytRDICt5dQL+gF z*}vI|oSe{aNKfaOUTOGXY4xXyEw;XQFJ_+_lzm^rlLXQ+jvM{z#X)CK(1wmM6=vwN z*L=a9%jrU^9>7b0EEY5VYp*Y(TDA2yJtEGjdG$z;B~lWkJ5Uk!Ex&Ca?ZnY-r9QB| z3Q1=W2dmVlDX{d-b(ZHVgoT=&`Q3QgO~_DiZRM|CI$IPtI#eD1mx&Gj9YBDQ7QQoh zB~=Obi<};Hppf(Zz^057ILb1xOkax0DrWG;Vv@bxuiMgV^Gv0nq~t>ItNkMUxs#a3 zx||;QI&di!6~5N`QjXOm#8j&v_|S`i%n=sXn??5gg5T+4567c>yKZl12IJ%v)R*F73gJ@z?Wg z)#g8Xet!B?{`$EoKOVo#J511_AN*@y47Aq^(MfBtx>BsiU{P%N$By?HZT zLOZQOA7?NHQhDByat~jE3iH)h1@L%lqken)_2duSCx>7b zuM^>kgM9qK!<)Wd@oLc_8uw*Ke;`}o%^xjYqnKoS-#?}Zos1PNVxYLyIn=%*Z5+(K zo{E}5D>3;}F7S=2Rp_-A1L>zn0~5eIC@yU}zjFnCpKo4=w z5f6L)bP>josYbKplO;k-x!!r+2vxnkwQ!oDtmm6y-(3uPPya8@YGAQL07*G@!NbF( zd1<|MEv_o-E7*-4F54XQqYO1Ox(niuDv8~8m-BqcL|7x!3AaHt&UY~3Q z>`bHVxq>sl-`|t3LFzw|ADTtUz3DB_sS&T!NNP(YR_*<#*ZDK`OY@0F8#E`SD$q;l z0@pNK2HGC<%L*&6hxO^}Vbz5bMU^wSJ>PXsgpkmT5U_j;Ok60Rxw*L+2bZ|SL`GYU z&D*ZFsM1bdTTAqKE8T`c4qX!^Avw?p0CI|*LDuHwc!QL|sNA+Um z+>ZL<548+-3v*$Jvi|TJ-3XxfS`X$sBkRCPq1Y>%n442nsB&^jAe7WIJ^2qj_hB~) z1*cZhqB5%iSu7gmyookv5EVNdv!4uK4|N%t1aQB}wtCdN8t~Df$g@HWR1EwT{X-h@ zR&5=?v6QxsAqKC#=K+K{Wg(D861RRF*X>M~9bk&CZzONec&bp@gsHh|U{^jDu7k$B zzMfO}uJ$!urA7quzWqQ$-hk|W5!l-WwOy~HQ9Bxm`F{(gtMIY>UH;MuAPRuF z{jEDF1sa^0ai%cxvf2OgUxlY8vQD$3U)+kjsH@f^%Z+-ODc`DL<6f$Tqw^AeQ;ThR+n-H zqoJhjJ&Gx;om<5Cyiw08a*qJO$ZGCK?^fXxdiqzk@JeAzB?3w6D)`8wXZ9M{K+<@p zf<5cdl(LPn=n1b4113PJliE(txm({yQMT2sDA9|j?r{TLcp_(}oQUfc%K~Ptvp1!- zM-JS7=xmwboDTG<2${NRH_z{g;gK2P?9pgvYb?}$ z`FBL1`ggH*a{XTkIus8U9bFhMu$&y8;CT|~r(QB6&#$Ojc2$t6vu7P|YeM1aRLdp$LI$SquvN|JAX&!ufr+ zAljMu9(_<(0~m_RxU{N5SL@%$~wtO4+Aje=>m z_iYKTUB{LPIVu?W$n8Hnj+G|$v(Lf4vqkw|740@ogZ&PzY^avmWF_ubMwPEwHptR< zaNPaMtDf*$L`q@Q$deN4BcGdw!V-_wlKig|O3E4gF_B+Hf{P%vhcnmn9QP|1t_H@pdJDIU2r@8??>4cevZCqXaDwn zNPt%JTY~(sD+Tg)28Hw9SqFO1`RD3(M4ZllU&@`2s;lQO%>0U-jujV_ezj*bxsBcSyz=%OJ7yHyMBIn zJ8IDgw=8_##4is~83SK(<*iGk!J?Lf;J9H9B)Wqgn&&WN91*AsyfXgg2%FleuMcEdb?+FZSN#OHlid3z?Rd4$hFVSnM->CX{=l@-oZ`<*uG8{5~lJ7w=^M0o7 zYh>Qo4uPW<@pK-Wx{_P~j_P44g7ra_K_2Kso#o3!I+ZRdXw-*FpGWU>`~Sz@SAS*M zb={IuDoBcyfOIJ(4JwUvcXu~PgNQWJAl)F{4bt7+-QCUEw?6Opo%cKE`~hc-cMQh; zo0nJYz4lsj%{Avgs~D_h%pksv8n)W{L%i9*tM>AFBWi_~&Wo3#b00kuUq#n6REf{C zPvf>AtqXyz4+TWBLHMEMhGw@RYfe4f^X*uTk#?t|lh2bYoLuqTMFJ^mhlPi}o}Lsx zKb*e4a#6ZnLr~6(f81I7L|;?fQmfyI)3O(17%&_adabE8Ml1i^D6xsD;ZAVnpW@$* zbiT0^4V=|c4{}3$3kQ;-r^CFp zF2_FHol3^DiaS9G!u-q8nG(}0az1&uVvLD(@A{RGq@9ogcC7#60;cjvMS{qGtPSj$K(2EBc0B=4j?(%nam% z&xu0?z&Kt)?F>diHGQ5r0qf014!LGB=x_R zNi&bvuA=Pr$N=g-A?@VJJ8J3L@OfQI=I36`OMbch(>ts%>iY0QytC##(oir5Z=^mu zMGRVJL*>lI0z16$t)bLUJxvg|((Tb~Dt7ibr$lxJgroCwGQeR)Mn+Dfmd~S1I{3+K z{PC{HKzx=(3ErHniGRU)Gx2PY_i;+?=Hd1bLx_Wqkq7$08Q4wZ0izU2)|Q7&-p5o_ z-g4gM5_a|&7{VC94jAj83RSgF0E{d91ce+auRMhU(Y-qvf^0w_>|HOU{Fm`Iw-+F~ zuUw-I24B^jf{JG3`g9X3^F2c-uy|(!35GNfCC!z|6qPH|;NEv01wD*)=Zq%Z-_mDm5YjvYotmB@UYf}Wq@={z=;Ww{Ofhu4H zyX(h~Fu)ljq)@FUt#8peA(~b>9CU=UgGND6G+m86y+TwpIwN!@#)K!FE`YbhYznhQ z5=#5}))^gIf_LENBm=@;{#fzC5GC#pG^a(eyOUTGY`Eez{tI$Hg_b z!KJ`=x)^QFR)ivXHOL zq%YDIM*JxUM0J4%fJ8 zj0`$~;u<5{L2`|*&R-yD{_|r_S714re>KcR<2}m9;JusbgqWJ1o?~T4zKn_Ly_SuG(xj$4;gT3F zG6=Me7guak;&!3(8EJFZ@b5pVb=Vh$h?7)0Cx)*6J@+=qAuY=7hKOruPQ( zwJYrb(&Mk+F`y6P>04kfeVQzaU}UHz*${{sX%kCv+E}9~(Yc8;OiIm8`VyezrIl!7 zQz(1V@t%rlJLR018Wb&2rsuIBLFoIil(#jymTWkMsQOvSPwwEHJnD$y7bi9mQBhwY z8?%#%N!~$hLn3;6pft-n39-|=`wgXjY}lyH1I4zvt(BeU2zs?YqF@FNIFkCUo+ZTu0~(^Dj~Jp9+~B6+N;(JNmhTzB0}x zbrJMgvU9nZ9N|{}?!1xwoEOPiOlh=Lc9ZIk!;w_7IY1|?*5y=;R25Sx8<6M)S}oN> z+)mMe17KPj+=_kTgufeB@gR)E+#{C+gblGfT7Dm(f1m4inm1M}f@LCCmhdurJ*W0% zwsWRzx3NxwhD^uyfRD)8t`}34t8gjO0vL=>_@zTrbYq48C$57%bk$D7$S4C6W|Y>x z@Y_gA!BkMt)oqM#h16o@dw@ z9CLyVD&wiMyHBGqD4Z>xzk=L77pbNVRw4Q0uH#eas^bQfG z?M(Gx7`frK2pZKj+%w&pj)CSJURy^J{?`g`4vCvC4S$@LMJpCiK|Rjjo-O35Fjnyu z^=Bvz5$m1#sO{m`ual` zssMY*H2z~XmxHFR2_2z#ds`!u>|V)fx$EeQ8yi!6eFKzRE-H|Ic~8;a0~sBe`chFp z%$g~5Z}HzhxO|$wB4I@OCE1%gJyoGWyAqRVx~Li+Dh#G9*FJE6ntv!ibX>@}vGGP&(ug&CyH?|u~Z3;4P{GRov(@;x%;C6{y zxOE_MYh2F>*4YJENH(a^!jUohTgUhL-2Sej_lQTG7#Ba7ff5Sdo787C!_`MxwY=rg z{qGEM^!!69?npWY6IK-z%p1u9M#p}qp7A>dvivM6CdqMnl%rOdd#4$iIL|Z3>j6i9 z|L`u8U_RqYCjR93u*Z*Omu5B~^Y`j4oNOm&o+^>e4bLt4?OqHi^JAS`q?Ja|6v&(+ zb?mRf#4D%zbGaoIaJ*DUSBaUZR+vPsn7si?)`~<$s`2mc2uNZv^LZOmlYrt|sQZXWOs5`lKj{S)7nG3X&%2JIe0h z&Fi7vR+SooA7s71uYV~|ZZ%jwURL!J$Dz zV)SPbEnw6-R_AD`&J3$ze$8K%b*A%JhiQ2vXri=oO~Y(IO|R0ln<0^Z)8e(_NP{DF zO8?1jed_orO#PcI&Bew@@{dG${^{7o@JCZeoC%F>d<9HRs+I0>^~~{=d)%3IYajy1 zrGaLbg1BM`fh{BiB-|U2viPy%GZ{Rk)2LT+%uhUe`uGt0oFqtcfpFO#r`F1kyRflZ zkHW1?)>6Hh7<0Of^Y!ay-;oq*Lv*cC$q6=(W8VdzQ#Z%SjGFXWSt;DrFO~)ra=JKX z+2Stf_TPa6@MqxQ^R5IH_8(Yw73~VFc`PxQ5Y8g<2d9&T*b6&Lt7jEmVX7A~OyZPJ z*NQ6l_(C)(0cYna71LK>-2gn^hX7ViMMEQ22kpV7d}JnMLY9E}isjfZ*;j}CIEdk4 z?Akt0vx(;(tlF_+?N8|e&udOZpaQ&Iy%|r8_mHB*cP8?c(=lvYT7RFpE^_8p4++N; zyWN~>$KPzz_n>b$nwWrA#EDcnj;azjf~qZ=)*>7#_ZeD=ttd;0Y`@|Y-;tc8{)7W5 z*@7}kRd>phoBBA-JAp~E_sxJ#kq#{KsFg>m147cu@h1O3?$d}?`OfawE2vZ^I-C6{ z$=gO-p%i)f_47E%pCt$8^G?;J3f0)j0_1@n>bmTY_f*i8g?cn%({Z*joTfwd*&I^7 ze71d5Q!mbPoK@`=V(tO08dPQQ>35N{n)0{H;&Aj zQwaqg$!RYR@6uG0J71szSV+j--CYIm$2%P8q#Q7!Dd`-iKxXLYi;}dR@s(4qx35&a zY8Q_C{(*HqCaJMSFYMk2lfivAw%x#(bPW!Y*OsAciFx}GxhrMASrV2!L>7bLC9-9Q z=1PwL@S==$ILTwcbp8b!(QisAoJq?)rJ{?V=(r_)S&2O03$rExddbUU!j7*d^ z=>ttL47^w8lbZf?Ps94a2ZFJ^_(^^adSW$;lg3J82^7m5~f?cdoJDCDr9=m{d2aU=6p9tKQ3+ z)V9l6t#$nXtS*u=7Wk7t-=q^wD;W+9{PTAAJ|k>u@?Uv6K#Ltck!Vdu8kiJYCQwCK z4`?$zVoE^EE?=^<5eYFLi1e3~>iyDUDPi!Ad9qM&5O3kKd?NqqVuCz?{fk6IVhqvI z;v3bMuj4`iCbY53B)$UD3`%#TF_`D+Wv)6nRbR!W^{vJ(mby{K3~x{)!wsB~R*MC} zyoZe(R&{6~V*POk-dPV#kx>PF9{%sPouy})k1QCdg zXKf(eU+5|j(aO#Tj8pI6@mQtcb$i&s7Y=HC)-Cr!yf9}aTUc(&m8o9(Ft?$RW4n}7`afE(EV>Lf<|cUA~6=}CoHTs8+3IWTBY#@c$HEjX7JP`NJUFm6`r%bq0_>4PKRCT2_@vVaC;Z>Dg+99^i`qQ3HJBLDJ2oB^SpNE`6A4O}u18~AZ z)ftM!kbG#QAVThVK(@*dad@1s*y>*{sBn|)l6!iVJxFd(7Ve9?@bWmic z!$0zTtA7V6p!(dq3Px361HfOPlaJFg zNE&<65-Dx_lWJ}qVU62ge`er)mB$iD@V>MZs=FEv$T;%@{HLD$HO&|7;C=pe zQ5OsTqJb9gNCM`(ZNM*Yi{=&HAO`Ti^G(1M(@yL=T|4xq_gS#;(I<06xEA@LRwG>x zIBaP?@lz$lk4vv{L!me~mt&hvR{WM-hdmx-9&Nn5xb(1?R+HJv*`2!5T` z?~CjUsDg8z4{!pfyz=*AmXNfyoh^$E>3|lGv)tgu*z$$R3~^+{ni9erufd(N--!G;X}DZS2U88k=w(!n zDBEIn4$ZJk6~0KLY|Aec#OojK#}zf@9ZOiy{z#{S#%rnwv&v_KS7Mqo6|jV{#wwiO zllHWgpV$FnApg!$v%6pfxYwW$OIOFMn7FvPNV0U4o*H|O*_DUexFH+OFN(&PwOim4 zAE_(wRWH$}>KqwBi$)V;rP9QUIjc%vxd3tldtW2m-WF$u#!ta)0UTt9zVZpV*0ue+ z;xM$}@8Y{_e%ii_WX30?8^IMSu{;Fxzt~ywH5BUXj#t{1b~A?G3Vti#+89Xug(O?1 zL-mTGd}ARp=@9SZIE~V0jm^Hebf?>M@+U?#e>Ryn)L>vf0cTk&ucgQ{)5!GfwmD(( z{0z-nyWHBqf)9xGz-X=(XBBs)eH!E5aP8Y|;ghKEycT)>ykCGjgUWpSL^E6p(2v~g zN~2RQ<-e5SdRW#%M(=^K6unEoIeFb1&!v!5e}6H%@U{8KT;EUzKkAb>!?XIIqETnk zjtx4hGmA0g$g-R)B`0e@FHX_11pe>h1orK1{PSYgf#0II=x=1*-yXlRdT`1hS4cXN zk$+r1i51ZaT3AC-x2i{PYvo z;|O>2FzYe7SQ2F8Gg>Pt2Fm|V;&xd&nuc2?3V^||L+8x-G3gqLX__)^9sfQ{N9V`r zlvQAncW(^CXsLd0{R`janR?Ks;Q#~8{ zH8HVpYa}Q1I|W;+eg6s2=hW&-}Ith3a>4wukOofkeBurSUZmz1mRRh8-w9ue_G}bE^?~JG16ZWGl|1&TApU`?M2JF2PJ*dU-Kc zFA+B2hoR9t$4TPUeH22#$xM8MoGe~XJ3K2pYas7Ws*{VfDo3W>M<*U3X!1p=-jjI#^^uiJpz`H#x)2o)524WqQZ?y@=E>RBl7!w?ZZ`oR>v5<^0&ew;nw$%melBiVsF!As!@K^{$(;b$LPOKNdO>n#{7J= zr;s`^NP^ED8pT}Be@X~??^y*_5A8*fubmH+s%mMtr1;Bh zYNS3R*43l$oQN=m*`G^6?Qfa2ofyCap;(ggnmCL|@>7|ozWZ@rLgkwM8{BA?w%LD#OVw9^% z3!A*iOG^|1uj3p32uO=YC8?E!6@>Izp>H|mRvL0mCMn-OZe(JStlGgaUQ(~a=GfFg zWltJsA+zD5xo|2(^oh~6V*i0GUt7ZXp%P874@9|Pt-Dn?Cj3@_H{o#Dd#P_=08*6N z@1Ezj8V49S{#sWidgK%JPZ{cQkPrpDyf{@KT5~n&Kt_WYp2X+#3VYqZPs{`RB0+?2 zV*#p%wk}7XEV-}upc^ntAaSr*toi1$O#9r%VeoY0qqBzH{tOF9q)rX+EsX2V*`jXS z=wNV*Wt#&{UL8f$JH(okdxBS!+BwZm@4#g05Z_>z3@ zB~mGiQIyh$MFu=+gH|ZGAks7*^1^Y}I%f*Q`}Q zEkjGUZ|B9u(85*$+x1!i0Q(uFd;ZoZYa#{J68cmOV( zaBn8j!a9x3`xf^IkawhXf75tU9igfWFH z;oeG^$EEyvUC5$*7fYz^b_Zu`h9!Npmo!#wKj8E>GR1T>^g_)=Ui9~8e{C13>+4&% zrnE%`P2IUx?m?9TQhc?hq=-$H1*aB!?GC!^m5&i2YaB4Ft7Cb*Y83t@rlo}06LUdy z+!=p5V!CKy9-IXpiowLG?=r%syZVBN6F0?(nPP5RPE|FEu{D}3*fk9ds00S);f@s6 z4+y(5AAqLXPKeq-*TlZ=<`b{p?JxB1?n-4_Jtf-Q5=DA?HKkuh$LyyA0$JXVGQ)ZQ zsdn-Hys1Ht_W}CVdFzH?n-(4t&a+i~d8pHPr8^Ou6enJa4!=4Udk-hz_3qXe1{T&n zyqou?7FMHCw?7lmOEe_fD*CH7+}H^MU|hJh`rG4wHke91`NJtlp?Ll1phBZ{ZusR; z10ZsCEWIbW3I-bua%!_Sao1RVfqN}g8zf9+F3`yHA3>pCr8QV5qp7JG{%Az6-raXg z!oamE6Q3josBHCN-na2;K!iNJzg=b|g|c`Oj}b?M5Caoa?~pn2ZxJ=;%?DIKD$=Q9 zag1r#o^A**KD%67xBh~3N>Z1o)enE_SQk5|0rvlF-D^i^U^A6X5F{x&e$u6ch|g=j z+!BCQ@2NahO|rSV`UzN}jOB5-cZ?9J5Hs5CeGU*z1&us7%~!9U>p?Q~*Nbp#&cktm z+mR0PO?Ph>-=8dDtUJGWFKe*zyPs8VRBwZ#?JlvEgmfE8X%I)>;D)JWQvw2$Mz|B& zzc(J?=Om>cK1oZ(BKagFsay?4%3t!&@qw6-`C#Qk`K>_a;Wo+{S3wK$&#Qd$M(h68 zd9PgmxfY-GJc&%gxb0MYoa)z;QXt;B9XCn&j=jGZaczFCx=pM$LeHQUcFLLtvi1TV zC!%^im^*z1!zUAow~@L~+km$H4yw)|8QhxnD@W1C)vGVbO;li6~Fxz0LyDX+6x~IPUqh5jp--i3L4S7i%LAWoJ$8oqD zW>-C}Au+ERH2Rn8CL4}KhG3EYM10TwZH-K28gXo67B90A}F2Lg!^mC()LZ!MW|Ai>3; zF&;bnHSQ`;2RvOkjTT+Fluda$a=*1$;R3$KBpJ5Xlv+OEuB!ZY;6h`;PWYiI45q4c zfpSjm>Xh{}FThiYV7u{(KjM=UnVUM8GL$wdQCtKBq<;FXaU^6oJ)izTjQAQ6yG}{mr>8Ee7EEX~-yaV}xcgUtwy2%8l2WW+<=9n! zc&D-81WN$6ty2abSU@|Qn?l8uZx6aa>Qe2NE)1~5_u?&fFP<2+5>bFh8EC_27dsZg zeuQJBu-sTFtn9rT%+woF_&gU`I8}OyNtUhB=+K=oH?Ek2138QOJ_9m}XbgP;%kr=N zMw`BBW7Z)`N_Q zICSyj9X43`LE}WMBV0fx5Ta96;5yFAg@*gB&#TD@^%b+Q?$@AQq^H2kNQovtYkYQP zEML)5Q3QW#B@?t=HTq$LAqSOL8v0U9K%?8ON!4^YK*nlHswf_G?hk8dUVS6}^v%dw z54_u$S3N`sjgQ<(BW(!0mXdgnqOToX3&?|HqrSeY0!MS1CUfqUR~Lh}$qaR)R8r&8 z{IL>K+fALN1EJy%w4b(s`Jy^2)Nu|sCG-JMcgq8&PsqsnPhJl_car`#+5dp1TLQH+ zU-!P{(MkF-Jr|sB1Eg<3D`I~GA8W_dwIGwBQU&u2e;W!Y`=VK^3KC{3GL{|m)u$$G>GV>Y2)W*}amA2>-Q z<=SEduMy22l^|&PM!y>rwvvGLa>|I@VS4Zhm75=sK1ShwKF;$jw@I>19F5q$IYz&) zzSS%^2h%pG&CPtEQd)fhR*;jM4(SWd0Nue8lN-C$7xT4-6?1r%%F~m^URSB|vLNCZ z0ZJdJb1dF8YxY6L;YE&qgQLjQz!9LcJabljy2F4mvol?mT^ah%l3qqS3U38yl1d%) z&$(WPo*8I{ZrVF{B7LOI8j~+T&|#bwa=8~ZNv0fr4_T%gXXw;>G89 zW)TdM>l1mE%H!h7L-Dv!Ul!l`C!saPRFvp{p;=wT;~eFGM=w!z8WF3ZQ4Cet_oiPm}kFU#+H;oV7W1Bk$TP()Fi! zw-6alpEatM%AyEZi4|KVorJRgjYdE_oZUg7GRFYUsE=aqM~!5bMIH~?h>33Et4E-s zHFegT18d(C=%9YYSS#`vK=GTy_CTM3MjfoL-9t_7Dx9-2{ki- zXfl|L*nFOd;hoOhclq=UkgLB)iZe7`kxa1WzI|G%Klo63E+(pieeb9r^dZAG_~oPe zF#@2Xj8L!orQ*N%_+N4A2VxKrZ_-t|SNwv?Jx<~=iQnL`U%0Tt^G@Oy>)+9+am?n$ zmNH{BNF3&}v$km*N6ES;qg4)aXNeq>$F9k>DAu&KwUcJlLQQZm;mLO-owe8J5Z47%i~jvJVL?J#!0xauE7au11wG5Y zvx2qKY+9y@NBY0-4@oXx=?4JhbLaiIdnK_n=^Kcxxo%>30w&^B@|E7j^Lp?cV|OdT zu`$Nn!Qa))&-MgL}X|uix&wKvAVgrZ3DQ@R>r)#ARWS0GD?$ZTm*C@$vdH} zEF7LY@%-tkwto3uZsVWS(fC2`jx^S72rS2;Pa8+_n1d?TE3QzmX@|rWlGKamS69_e z(F03R9E=iwEAVj{uhRXq3~d2~UhQ&FBUE?oIDA}gQ>NeF=+NBCII7q-1Q-k^1v;k} zKqTDcwhM52J?2bi%5%vvZ=;kp;g``V=={kYSo6p$VQK;3HLh;KgSYe18a&0AL6B-G%eD{+Tv0q5 zjGUfc20-wjk&Zg)2(wQ*eKxTlMJWbVHmaUr)(Mjopdr_>gu{Lc2_j=iryuFJE)heF zyyE?#CWAq3{nAS!kz*LpdA5N>Q4cq}JH3n2;TJ`5zj5tIJi0q~UNHxqm#QYGNEJEgcu0SjgG(qz)o{P9@4|} zz=Qy;gc+NjY$tQl+X)~Q2`p;4p&oYp**VHVvv$lCNLC z_5{vUnV%Qrj5qV=XKkl?NMhO5bwXYQwlIy5pY5O*@I zm-!=aJEQN}+`tjhrtsE8N2qHR`z}vOJx||Yo*cQ!*r$z6U(ApNv`a&Q;&2LRvyedA zEIDQQ0{hct6m)9UQi^wJN`nkPcXpr0>-lMt_%)pd9@S}CYjjzA0h2WSaBRNMaZp0L zVUrT2l0DfAvx1st?9|BywReV1_?g*@i}ZK&O8k$Dj}JnCCQ^?1`fsAV#0X<%eBFx5 zD+Ny33Xb=OB674SQ@6Fs7^>)8cZ)<@ImO{p7(zHVH@ijlL_uS^Q?GvV0avj|Ye4CB z79Yw<>@?0eVt0_gK`tU-7Wr1-)RO)9Iw4;*QrV2V1lUDHltbyLV0ebYn39msZZhoE z9~c7Jp!l_>HgamyNsa~WVFeR}XGz*+XGkqIy%j%ABw zBP+XIwdgJ@u^O{Wr1L=L(S&XX;JT{Xv9Qh!e|?@&enff^p%gaUL{{^7;&opsNbSBv z2dAXmtT}W2Yah_7q72Fa*T-ZwUEZ8ppD`JJFG8ysk9QcUdz(O2{HR`1!+VGBewS)r zPak_Q*eSula8ZfSmOpmykRySgTV*0Eym8}1`6%W{~yZJu<-Yl_g`c5Na7+O zjo7{SiDUT%5sB$XzZX%-hCduzZR5+dV)C-qvzpFidkj4`28>sj1BmJjC}8c>x9vzY zhZrMr9W}$+0S$uY zt)61ozZ`o)kT%VyenM#PPR^rcBTYMW#c}gKFjn);gO9Qa)q>BDO2;c+Z8^<-k4fmt zPJ`rrvrPfvVn82$8x0mr0h5!!<`ku!@$2Ssf<#ioN8Y`Mn3?y#M6r5r@Wl1%PF!nP zBuc)8xf#MCAkc_8B)b3@y`M1R~e4@Cdro zB`9s{>yFjSq4Y~u+N)>kcJQ3Y`c1O=lwlY&+iV}>KxxSnQtyli+J{MvJuw+(EzBU- z>7NT>Sn*qlU8oubOc%=LK{jnLO3+lu1!q`r6IiG>_qI;QnSBhk$k=1q7sm;w8y2b9 z^4SA&QT(hm#&s&&!!p{d`LdeSd=#gaO1`h#3Ag7Oi3pk@JP?U#h|D#2vER@iy2{a0 z7(mKN2M#U(%=EK+R;#E~dK@@qJ|2W6;!pQcQ{|O7f~3Y+cW*Bh+fxz}-tA(iz5$?8 z6lMKor432_rxR|=nRY>rLHkIpjpL!&^iN^g=m&|;woV@Q^X*X~WeKk?Nk%LnxZ`957=C?T2wKwhnJjOl?V#e^`hg)taJ>@9kON9rn|J>p zX;0oX^VkT6;OBmgj&4825L@Wra^mDR^+crt`c$KM$&VtMN^Xhkp%f7mlH`>gx}n#? zNn%TP*QaM-;AI%lD2hz=Jdb|8TVB?BHJy%oeO>MN)Jo3h5!p1xvLH?IxHAH@sMU`r z+g_xEaruBvnKT^AS|BVUB25@@$EcflU?Jt-afy_5`$@g2(R}ka6SL*M#*Kba`%|eX z23p!6g+etE5Cvrei_fiK?xgTp&0d1EVjwHiy#1!$S|`E5#z$6SB69G>D)Wt2c(9ro z;Z-+9a2NiVqIgLHJV3?iJW^z&|on7OqbEYV@vhWODi`d~E+Vpkcq zQP;;=x3ik>N>RpG7LmM3uE5`DtWcd3^#J$K>=oeKFC2oQBn6!q6_1K_gZD)k{GaMr z)BI+}(o-185(T{pnhR#_!xhYUJ`T5pH*>(l!w&<#DQ5c<(}5rLfl$1dyg-CN03|U} zkCW%1wjk|gcJTIZmmMiBD&FkhBI3Pr%xk@Edgg$mYreD0lDfy_LtysLQ5YN147?Jy zwzHEJ@L^P{Qq+ujU!hNYe_~f39M3jv_?-9BiKor9ZgP*!;B=*3Q_mTfilOx@6WgxF zNr>A;(K~~_mpZ#*yKMmlh^e$o);oQfOp^j|rGQovXfxHOSd-qtH^{-R@)E(NIOO8zbFZ2G%mLX=g72I}dtq=v?$N&GiA zle%`)?EM-QV_D*O`U#~z;%^4YK>Kg=qCmWNWpx#6{|s!=3|j{yHo*S#0x9q*Ay^Nr;dU0~Dm(p;(mU!H!`@;yE!@6~|D=j?Qy zpuE+{rVXn%K6BwkMMXN?5cL5tTNP}>`D`6>HbcRXlQoN>K`2$O#l;RJ6z{2Wf|3CZ zKivX;-FdWFkUdX5x4iW8?=Re}9=;84Xvf&nWD~xJ+n$Y)1z;(d4Eqhv+#97$*g`1o zfPDTL;Bx}PMp(#|$j}cXhU-;-Pp*iEK}s+Tq7oJ;H( zpcTyelekr_Eu4XCh%3sf?8hKk4^4rRs2@}vxk@+TZgh{*TGM?@BqTs6SuR^5(RDHq z1JVft2$qOv)CiC_kk#k_c(HtnVC2PR>c~o5E3!AuigN^g66dw}-SXFT+#mwZXh6A+ zTI{t-ib2woNx9%-&Y6rIc`<_anfvSwnmNdjZJw)k}0* zqmqC{gJ5s);inB~MCk_;7NI_ZnG!Je$jM>tgz3=I`g|7{e2J5ZaSY;kr|Wm#2Yv9Q zu-e-Tqp-&3<)G_t#8s2d0HbV}=X=w#xY_u&KMRn;Alm>6u0FIGX!=BLKJ)vzF%&Q$ zY{vaouM2Hm6U;F~MoG4p;gqDCPaPdpECx}PrGE|r-xk$MIi7?gAs*Tn^wiR}8mNG7 za@)TwQ)VbXk!Cql?hh2PKf7G)rFy%z%fss!lK=kwdkpxL3G8+)ucC>`XZn~+Bk36! zi~#ARsCPZ9kwlR=^lOcMWK}oRayfipes+eUPzn}TPyKgNt`b|dH)vLvUcj5)cr|kKq(11|Js#$pW50S( z@yUXNJIcD}^a`j5Vy|JdO~OJ-ZLh?g9C9)W3XI3AqxfS$uR%M^&d#QJ{$l#6KNi$o zOS_7oM(ccr2nN?3K0MK7_j@m39!k9$$*pWV21l4}med)IXoi~-BAXPOyNaoiaT<(6 z0qM3tA;BtsGZ%!-BB}X7=5O!u-+rw)0E(Ir@)8R|w)-D{p8sT9-Wh)b z!@vD5*?~HUjo=;ipMK`QT@woW6hfQka)TH7yIcPKcDBLoSaImc{p;TU_sjmnw?Yp8 z`zHVGjyy*ppUti<=g$KEw`+n8`3XdX(=)6$f9uwNzo~4<6y)}IY3Tps5up%c{UAlc z|MRti9XR?T{g3>>zh4vWEu`z$H>wzh3E5Bo<1YcyE6FAmgpB_G&p$0fNClZ)^qT7b zayx8^;CBA=A^-Cs|Fa|iAA#;aJMurF_5YvH$|F$(jrB5;kCkBVjEP|2&53Q;S1|Z5 z*R&6dsW!d%!Li&CkNA?6_!5f;o1dVr3j2|iJ)Ez@m!IRrk+%J=Ch%g9^Rr7x8<-&H zFDc}wZFeimL?tYF1qMI(fD@%NBAeNJ!_(&IiN|_ zL{gMOgq@RoF}T9`sQc^6XoE2rJVHMI;EXWVF{Nm%HQdYMfu^I}$r`fNl`%4-B%BpE zXx&38zui19H`9L8?@zs!f{2p6?e;{bNaIpg*rx}MPVa{c{QJL0s1#z`Xr#`$QI;sF zD^+HZTSCIGq=A(2Pq0SKHWeesc3g(6yI`Ty6Hyp3)YpoyVPZ|O5OE|} zpF+EyF@097RtyXKzRk>1k9&OL^^wBb*k=aXw~?y}r^WqsQ-+0`#>MnJ^ZpNPX4|s+ z(-x8r*~{kzlo`_Ic``LzQ77k_`Q^;DPVf8&5<;?JJr1*C4{;*YVZTmM#xwKZr6UH2 zYYP4zxZl%lf~NeNyBxiQc~%>oVue14pD9qnycxK-mFiCb{8Z+X>D~mosj~%d%{Lp{ zFLxw~LD|v`?yFiF;bOus4E0_c>E1tg12e@-t_l9KTwZ)Dl>%~Qqein?UqiFy2SZZ| z3*x22Jb2s*4QSKwcK04q&P^4DGuTHqan=uVBcsU}r7)*T1^-H-@I4^GRc4h$w1Us< z;BuPLUhZfnv`Lv(hB}gFy+b0joNRcaWD;D2d3{Kekc-x=h64M052DJl$Gt^)U34;w zAO0R~s?Nrnfb0WYL4Zg~LPc4wN9WQV6_Q2E32NSl-vmTy{$(-J5?|2w)tVwnrnZ;+ z*M1qyHT1Iy7oBf^EMsl2Q$-&`{gXdQA;Kg+FbUS0u)Ce5+qw^8R28CEqC_)q^X}7f zCWW^$4d-%iSJ={J1(d9hm8|#t{3N|3I+KF}H~0DVmtiSnTt*=ZfvGjD$jGvw8^4L6 zC>)c{CdP_`2R6Fd2!T$=LUd#wKE`jz)KLmN(?MAR=2_1?k{RSitzP`yO)f}R%v+); zt#piiaX0XZ6?+_%j$Vy!Zt?3H+9FeNN;YS}AKbD3(d4@6dw^ z)Si432?PM5apwm!G)gqDvY$pysC31VHrrQNCFMqPyD;+|`4riSyt)d*#P9z;8GnjR zHuRfz-jrb?b8?{$-Ya{N47pk1R}RUr`TH51uZRN;(;~k7?%h=RjHTNvqT4H8<(t+l zz9dbDSRKOmq`ePrnK$)IfAXxuk;C` z48hU*xM(E=ZCfF!QP-<;dmYn6J5w$U0#t0<0=&=fD)D(M=NyFqp{bLQ#m7?&QEjS0 ziK(thdRb3pr!7o|-WZem@0`bx3QVexU_`s&fkAwgLD46rHn-d1{-Z$GI<)wyCf`p7 zfj^yip!+JgnnD{&e5Ej*bc@paHTaik>0GoE3L9)48-?xd$)^5Z3d~rLmoY3TBlHd`Fn}J_A(NCjwp93PXF9D5MLd4 z1yybZFbfuH5_B^Cn0>!WISJ#KOr~S^10-YcqOSR8!6HMd$YB+EFbFz{d2aA#@Ew)=hWLBV(wB}NF!#P6;^7{j^~Vzc zQ{P7C4Sil@mvY|mep8dq&CMV6riJZq)z~XfJVRRt$ijTqGT~%wqQA+8yj1ULI;wd` zzY!LNfyeTmV$a|{55H>e8~<8gBBw?~++ZSJL_*b^ayPrGBL4>Bq)@G?rE<)8%@0mX zb;%R2b9T~CCoYd+3ERIMHZnHpoL>wIcf}P(7`qM(=aVdycEx0`(LOD+>IpP09M9Z~ z?2u3z7>GWfAJ$pKGOtY3b~3g+rHA!&eDPU?NsUw97wogYd2o zHE?b##W~iozoMGx?IQJz5%cJ{(wEonbYW_rU!TA3-&qwSUS73Jx;l*{abO2F@f3rj zYZ&P1iZVot$YaJ`Z@NGHa20$wSr=V5aE%bJ<4xkjia$?jj}D!1o@0*AdS>(q+RZ>( z4bYRYm#rLhXbYnCkZ~7-cQz)1NXBG`#KfX2)Rr}; zfc=;^x4%%2*Lv^SU|>b04t@aX>>2ELrBVc)yUO?00UND5~~ zMq-LeNTjU_=^C4kA;V{I`+LA#aOowT3XWE=J8yNR78no zOLpobM^eIPkrn)hn6Svt>E8Redxk^bCJ9rpTMHs_v`LM%#}qi?!}z!<1Sm1qXnZDwjKEVu4tc-QY#+I7c^C>q<(3e4H zbZ4&UA0L}{8@z8bFU!#g6;W9w$tp^Tg%#S)iKP*sOQ03-s?J-s z7ATj0?uE-4@#-Q$;27RP$=ORGB=2K7-Mkp_YP!+<82$5~sLr!$fkHSNsx!%{n}&UQ z4^YYkBF!e-zgv`q7Umur&{;({R^E%>eH7f_N5ZJ7mr*zwPN4jykFZk2G?|G&RU*cM z@sUT83)iNSaIoEhj)scDS4H^A3YqA>KG!o$8IzAydEJaFYw=s9V@P9rW+cG#wYy#ZkHG zIqx--qxu)sf!QlR{g>U~*pFW!*f{}TgGcQr8Mwo4VcS^--R^B|wDXlDexQ-j1;30d z8(wfFUpE?P`9Z9rN>a7KQ%1SNG|YluYBM2;Iy#fyx88!ns43d5%(Bq(sm6URAYsm> zimz3W6W5mRnQu7N&eX`_GnWJMrEa2^?Upby?nwOop(#AOyPKDGv!^m4(xRl=3r$8= z^dVmdM(V#8)@D8-UR4|#EJKjhpRJpYyn|d@mXC-Fc zWykS~DeBc4>x>)q=qK4p8MA$9Oh(rdoB2M%)?A?z@}1aOMc=k(nD1O`GM{VTRX<#9 zj&Y+Fu|2S;j0TB|$Q4Lpjqqrh5ehLiB4tmbjz=9Rp6!)1v2I(Ca6U{DG(Z27HUQi# z@_%)8m0?kKPajEXX^;k`loD82T1q6ArIio_>F!z@q$ETMX_f|Q7FfEwJ4ITAr90m> zp69)u|NZIyaAIc8@0^)y=FA*FmB5lJFlA(cK_o?Ub?h62MBL$K{3)uph+R_7`dg<0 zZFMj5pcPUTS;G-CC&({Iws2Dg&FnnzpNrtPw~?kYqns%|~uOs^oj zTfra}p-YQ=>TVcR$p6vK`(F^qA*aV*+t?R%j)# zF|p>xh^INiy%{4;KHv^07n6}=a9~^i@^K;q;E!Iv##lR>jdgJjwr%1O=cr*Vf0BN6 zxtVR^19+yg4ZkKp_wWJpJQ2}$1nYiEspN07j1ucgjnlrC z=@!C?Q|`$T+surkefM`)0!9>ij9k^NM9GmO?cUqNav7ax7P%9C2UVt*(!qN4@imq3 zphSd@KI5lp^*cRf?5c5FQ&e`NT@IU6gPke?QnS+MW;J6^HS0M`)`Yu)zUkE1=#yZ$ zWZ(5}@0qj60{dV4t9fvk zaNHesQN_c~AwB>LS9mZ@{kM8#&C7P~`rfS_86NolU3GMo6;6EJnJ(uLub}$d7EUk3`a$B3;+o0k;iyn!=b@o~mxXvc7e)HqsJc?HLiEUOR!&_gz(yjT z5c|kC+^$z`tXGxPd*J08k{q3wf84?eB>NT$S?|`bRxXbFXGOE+BXd3Y1g%(uWuvkU zy;yEm@Z2QMnB%*~4C}Y?`49AC9q2hn5|1*WeiL+QS=0Fk^UOac&k_&xV!vt=&CBr$SDEz5M7h6*YC^hc()wcu6 zp5wabdfjRf=x}k!a|K4xR&}5&6@kkcC?Za^A6=Zx0e0SCBIX%OZ|GmTX)yH3lj*FX zPOd55^FPUtsFjNYSdMck&{8KwM2Da*>H*&N1Mo2<~3N zw1Q-dWwZK%diUIo_!XaX`S5U3sa1N547{McZYKTg63!#H3vx$7XWrQr>U`{m>!-plI~$F0d=%=H z6&0t#(f#`FGdd3CR?`kpFtZXs@BTf_KlixYv5*L~D17D1^L>LEXt}^7EM!U~023Kk z&WyB1Ns^qrS7gItSjF z;|X81E+-&LL+a=@HB0>#&CH9(jOiipcqKFs6=uI2thA{tOgoiXj`dWow^N#m57uWs zg<%wN3w(@{*M22|!nS(frsjuOHXm(FqykgKnryRyVPkzq)>v%3-IA4?So%8N+TR#J z%UAU4g8BU5*!_sD@#51>l!E#bMJDF-y|{XT^yKub+Z-o#LwRKv*{MOADi%ywt_e1^ z%Mo>BEJF}9$_J!xS^J)PLq}(rVbvd9??`*i|NBTbgt!L;Z6AwpNzjtL->a>Ave07<^60= z?p{(Iw?jwAlv&lAQGZNWtyGj9q1sF4Nm*jW+Bi7LSYtq%DDulPMNo{hsU8g`@D*5 z9>pgzt<@#CqeuDOO8v~NkY$DnvoJQ_lRX~uM`CHoXUy{)dV0h%h)bFK%B_dVCvu}u z`W184J4~*4w8=ln@$-9fes7Kn> zkcsspf~~DySL$TXuvPv0>AGBsiB@nt0S-qXvc9T4(2=`{r;O<2n06N;fQIjhd>Mo9vO<)Zx8&= zc0wThj?4_ye--E2r(YJXv+34HMW0E1_*JFHnD!5zhx60lD&I79=nZ?ncI@7WY-bel zK?@e7W;)5r$;p~uc=3d=j|0D-*K8fR^T=N>OfUY5qPY>eLvCiB1vITF0#O`;rDK=nB_Re6@GfVA0-dN6q;_@_%;96Dpw&mUN) z%Evd`s`=UHp%SJBSvCbTgYbuEs%!49yZO1mW8eO>LXp2^ApHEyi0)e}8|Q^s*o3D7 zqoI>$T}W$EUPf+Y?EGMYo!htD)N9u&(#gJyjfP8R+`fEG(p8_5*C+c}^-=tLe%w6e zFfP3ce|UT#7PYMzDKZX5nZqio6~$ml$0AYYo8eAz<0^=dNRBFv{@i1m>#%H-Z#s)| zh^*zg&}^N{w&>61_j9VI!F^Z{*8ZIU)jR*{v;5XQg(`?U zx{duhnnriDJiQ@WQTo??!{3P~k{O}s!uUQ?b<(7bXNcz}cRN#>k2-}jf|=Xc_VQw8 z)2`IUX_Ev{LUx3bm}Eu2Z$#=nRO~n9hMrKL95t2*q~|Uc2J~do>@4Crae8ThO?`fa z@|p?pmjW;moZ+a%qjuW2u2Uw@rV-*FD`E*p#(HC^{2J%l<;L3bQ$MG`KGX@;PR;@7 zEy<@k9YfaX$4YSH^`$pB-12gq)=#_DqUre9zbUt2D!2SihyGL+ zwdcE+jwxoIc|4;l{*Ek_%<^V$Q@K)}ocN`8U=CL7dZ^}5(6DzAxsd7fA`Q}Xyv0Y_ z35jJrkySqqs1;vWI)9PJ(vpwQmU=t^ZMQ1XwGD&EPw@44;vhqrqB*Si3sDb?J-4QC zLEBX$Iot&lLn%3*opmJ{mzgbg!%v(U7{53IIH9V9u-uFW2~O~0ly@)maz<&m0oHoD zf%ycRRwRL?|5+y}%TwPJzI(55l2*QzHYl(&nSEVt86rxfSNX$K5BCK0FP;<_LNB9I zOLjvM{7GVaoR=DJTzFvg>#(1ajs8^WSA{!K3w6=b9K~t&^u0xL=4%|`No?M=MEEws z?f}GRi4UQ0Q2T8}P;8F?wiD;Hy&Pu=GxD6-e&uDS?4QZ6e7j6exZ`jH)o)pSTAVf( zT*$w=jv;7@MKfe(wd0T})B_c>q8OcNT`aJO%n0Zup7{9Yi}h=t2O^bNgL7IP-C#LU zg!rotuZ0RS7Yole&qZSNInl6(m!OOvl|#D7r^;*|b6&D=Y4-wDJWE@0{iz&^3T7Y7 zy7N!Z&r5*ErhR}4evSiZWBK2sflIZGEa7%RFlx_c|lHe)dkDUjDX_q&Wv;?99lUx*-SC>8*bp=p)nFUZAl{2Pm+1# zg)ygdB<~=J1Pf4=I;D})tMrKi9L{i1Lnj%WzRunr4Ilsf?sS*4I&238ZC^!pEGSvN zRzo2Obsg##ZxNTz)W}wZ&8@7h!;D~pbrj1?y0{xHDlGQMGfZkQ9HF?Uqk8kGkz@mi z3O&r-%>w`o0{H@DnQfybjG%<5h{;WjJU<^=xK>7s{iv$6x)u25y26CkRj_86 z@b=CUx_Eawntj7HqYrd`7)-ws=-vIPZLs7O-GZ}lBf$=95NxUbh(=-9OX+7L=W6tzhte1@VkaO=UnNcuJa4bHsc?0 zjcb!wx}%TzD)Wpk(w*(ZA*gh|&AH>4hZxfI)pQ+o#%4VG?nb>5&~_5$TJKd+oheNt zgK>v}QX=?cj5_G8SZ(Y^RF`n%(6Q-We&QF1-zjHgr53xY>0;z9x5Gz#LQyFOtNUq^ zza9MfaiU^#dSaROGF<@hor6i+$CiMsmVu@y`YH_+lJvH^7szgIH z`pl37N-Xc1%Uun2d(yo3)h;~K;q7TC;cT}GAll$$IA>S&kBO0(3Tp5(z59|PT}d!% zuCaE+O`Y3q5|#%{2V~q&E;2X|BwGBQ>8ElG-;ug)RKeXoMt5ls4SmtG1hpH|mWfu8 zu)S%CxB)!!{HmanmXRoXmZ2R4^*oS~Y1t%NjB>{6T`ct_0`iqWb5yXz%NS;Ecg=`gJUB4W!4(X zm4mv1>403(9$# z?{k{fHhM#(6U{cGU(dH!fOS;f^?r8g>Ku*Vmr{kzLEiSufEJA~II>Uv!LS5)>B(I(fF(r;_QjIIeoG$lX5^e{zE&+?w z#a+s`PxOQHA)SsZfOSwMPL+e@)e`X0_J)KzQz_pWty|rGxN?CBCBYZ2LTni!KGp-q}ozI3F{5*Yc z5B1@-D+o<+v^6$D;e@>qK=|f0xLHp(x+!5>_1I!BT6H$y!K?B7%^u0!UmNdlDGPr3 zfcS9odD0s#53%*;QrB>dv;(U$rrYOl)tyv^-+_E$@jjO4i3vSg^8{3y#5TJKm_&$u z1n53(vxH$M4|35EPsZQJX zJ#NweBaKg}t5O8deaEL>ZO%PFJA^|Jw};5)WJ~#gQTvdVj2IFTfiujg)`JGirz?oj z4hrhtpKszn=2BI>W;#ROYSX3S_b?AeFO%Bh`69sjfXGpE=Cee~oWua`ZH&SNS_zOl zkI`FonRbVqT)<0wpc=G%(2W`7CD7jUuGF4y^tNT^vU_4im zAc%{RmUibmvJ&y=`x`>xT#9dxL}>7_)AJ0R=Zd8wmo!YcaY;2_S%a48Xvpu~T#$b` zB0I)O7)+T}0)^PKxaIdm1$0CP0ky??^LHFxbji zFd$-1<2T2-ZR>H%CKhKfT8xtA9F@;KT7dRBnRXf`sECP9_8_8P>w_#UAzqGXBAAA$ zc{fD%ak7sZ_JUO>j?D}#j;qH`3+$&(xW+Bxqh-J#3Vc7DcB4oze22QeG!q;-$}ZDi z2v)Deg}DgSNl)`_=x{-V_PasX)tnbD@>J44=rn9>uP@r_sq)mau~J&s5($}AZW3{t zFadFBr3P6Lbu!nf2}2bd+QQ?mmAn^Kg-c2ytOv5+G2SRaf(sJ3S3TIdmqH_TYmBa{ zZSqS>;!Su>>pDIt4WG^4zaGwGfSHcZYiI`7HDZBT4c->;c`>HtEdX+P0f*>sd{L+c z<5x|ptgGL+_Il2$qMXF)zV-lMup!TLLr-Etda?Fu=@;OdpwaDF3|#o4V1@G2cOg5l z1{yQcy+$pi{vInU;4zgM0I9d$k*pj9kR-a>lWgPy1|j^w0X%Xm>ZU7i46o+b5wz5O zU@a|7xIHtpl!ceWSBSuSEF@xjoWiE;^xS^^hUQz1p1sik&+n;!ZPZ^YB$%PELb}S zm4D;=_aMJjla-<$ErwN*cw%w`I*a|SE==gm^u)7sF7QhGTeq4V6$mfo1!C0cV8Pjo z*%791{C=jJc^&hAs1ab>hY2LInueh&lH70hGrJ{o<7stqut-HO@37MO8ee~b% z3$t?y{L8@axk)o5d2!3KG7_z7I{L^@@9+=EFqMjD(x*{S;3$fs6~uw>9N7P8(7$ZQ zk{cqQ@X_{lAZv2eP1UQ|AM6$jXt_QmzFT@2jdxz>Z?((KD}pEYclckTLjD@cqR*(7 zF%2ladyGD%JN9Cgl*iTh2w>T?`He2OTV+CMV>GsY`{U#7=H zF%p1YhLMC)yv)~49krBypKww3f|Z3L48M(n3mM?$(O+RoWA|s`Iue&?vM=mZUGPV&g2urt5cg9^*i!! zm#%08^m8G!$;f012n`I_f{V2FgmE|R*KdMfFELH_R|hHD`)Dys{7 zO8H;hAsZm8kGPVq*>*knN8JDR_J@)ou`;T~iTgijKuV)W|NGt70q}6%69zIA4Tx04Nn>U|=$Eba8TJ5@2A+%_}Jia(7aQh>TKTzreu3z|A1cV9a2hm|R@o z7!csY00epYd8r^86@0tKzz7n#tP3&=RgNH!E4iSk1gPo{5F4eW7AG?>Fd6{a+2th# zKyk)7K((2~%LAWc3jh!hb) z&=1t_!-imQSSk8}UH^&&1qB7AND+`wLklD%gp^G$+k1Wg-|x&l_uYHn+kLyK8+fyO z-#K^6nVB$-`$EFWayKTl`UC$#gdW1ch~|Nt`_N^GzSLZKw3c$!kn~IngeMLq&X1f0F`ln zf4}s$Y>}=luS@rqrP9|}D}C*C66ah(zW?@>D>6T_QJC(r%S;; zCrja^BP1(3C#|fNVd&MVnUmt37FK|FSG2ntBu~^K#v!=&l{qPfi^qrsO)(R-}S_D5YXc~0R z%R#3^(U5raV8TYlp9}Ns>6VcNYrJP-<|8u`c$N;&sXgs5pO&F+0f^p02A@py8a$Yh zovU^NJK~ub^2^AA9ijgY9C4UQrRY<+8=W*vb6{t2fOWmAWt+6Tbh|3%!YT8mVDeFt zm0zq;(4BH=ZGzz@tE`%=R$4_}2>il=^_Q3zHrCfAT^nAJ_BBtVX=syzS*J_!%#$T6 zH(&jz=k(9pk^^rG1#J~5Si-dH$9z$5XEPv%pgQpoqJqS*$fLgtu%}(0euW*ty$4EI zl_r!`T=7r|QCS(JlA@w>%8ZJQ?wpU(Fo0B)s_;}P!k)@c?WicJ9I3!ip{Xpz6sfbr zQWJ?~pn-rr)HrLqU*XvVun~y$vmwBvEd(JIJ^u;mMrYt^*i-1kkV-$S@Ufsc?C{Z$ zfgXf?H1NYBi0g+1?f7BAj}FE0F+#|9{ZR2fQ23F8ci~406GEQ&${UT5Dsb>GrvN;{ zdk!Dq*?53w<`hasZUG?X?g(>>@D#d5IOH9SHIDk?{Uoysx~K6p2Z9{XwgEEVyz~L- zc=ZnSmJfl#datDuI&2dP;!jv)L_88Lb+4iQizWFGIz&A_be(#dvStsj|d1l~a zc6(^2!|a_L*y|N^Q27#P53{!o^F8g`@H9w&2l09+;6~t03S(5ncxYJZGINWyp=X0{ ziXfAUK_4gvQx;GlnNSvz10_ObLd6;|_C@R~1lG7f1@`d)DlpS;G)PvJFq#8GHAc(A zQ^{JZpaI}ub}mHdH%dR`h!=}{p!D-Yz|s^A%lA~_H5(pJbVPy5TSpZf?8hhsM;B~F zf^Pl%7*anNq2TBQMk@@)dOP5^!zp}x7*Sbgs1j$HQi-En^GYx;SgJ4scoydf@t$2! z4n^-4^Z5DoG8KJKme;i1TayEu(V^z--a@5`V z8u)jx@ZI&Rq;1*#_`dCsqJz(uqDhD0JDsCiR`~~}n|68YaA2=c(5y6Oi1(ql+uO29 zdYkdIZkE2*Ez;k%6^k&osX}JY%oc29<+}ztOB-t7xqPQW#?}Y+%rdFeJnEEvPk~r` zA!u#``H#3qUXxVlh(bo?GbPG+6Dxd}VpJhU{U>B6Y7r)f7Kf4IAxvh>jgThlKjI#F z-DUYlw_FuTRa`bm|#}Xv^g%JR@ns{Lg$6X6+ZV8UQX5x5_KY&nlTs6{5fP1&SV6)HGpd17`Hw2hsMJIi zmahy4sIbxSsooRm!^RS=fx{ZroCs;`;~-uR!}Gs4nq#gC{$~5Tzv{xzr^|$c8~JU{Nw9VTHkm?I#xd|x%-?dr3aiX*|{9SOUquA zXK%IlJ*uFo2<+iJZ`YRPItSBJ|0Wdk8l4NyE*XP{ag1aYR|D2SDUViV%;9AdU_zRx zf=Q_`BKm84=zaaWH_NL_Uiab;SGYuaDcnKDRH7powNP9Chx7{zi{y)+{e(=LGHH+G z-%kvXRQylr4jSS7M+(Sybp@Wt-|z?4&M1HOw0m2(VcKLXMrpQ54@PTxn>Jvoq)D>M zpsYvjBe~W4N?y(W_*g7gcBC85pj3=}oXWXr$^Fu~~O-rPAD>nLLs5-lJEP5srC5H_k=6$G?gY&%hb?T8xqM%u+w9}*U z7;f^LD!c|7bYu~v^e*DYZ1_`lyoGL5IWXZFSKtb;O{NMNBBl7TLPs#`0_g^x1 z&VeERMst_IZ+KwS;YvM@TXT{sy;1t9{O_oHG`{G2kUrVRzqILNY0pp zBp+L4@<-1AuUro&-GD|pIWpea9b9Pwf81UN}e{NVJ3*%x5sBOu@j z@*k;>(OjqWZ?wJm?6q_KOVYCBUdgIHKuQn2NOFt9(=gF^((ijO2X?uF4l3oI=FQSk z`;2t0e?j^hR_icf?x^Y5=sykkBupLW_)3N>4K|2;Pbtv!6fjZ$0VhSH^rP;61*cwdqv|YnJ3rS|Eipj+5N7 zaXN2E`e|jIz~wz+!FH*)w^y29e^9!X-YpgLE|=`$(V?_hRUe@GrxK44Exts1%+Vfo zG(duXq?z)g9~KzdX?^{E$lDSrIPm>aywCC4n@+Fxj&gDDe&1yZnnJ{8V|)N8uGrI8<_L_C*EYUSE${ z));&3yHyTCnuH4Dto`4V{v#D6@|s9L>cNl6Ln7(D@z!4ml{C%V=~$qK)xK1y$;^zg z#q!<-ljN+!$4hCUcNAa}kL2A`%eKj#3)jk{uWXiH-OC*IIHo*b=FJ)42hcU;QNZt!GKW>~~Ay`8Pe^Bx&r)u1(J^-y`30(kSCbVW3fT=$CQpN`XH5w#q zxG7&tPe0BZXj}29bgWt^xwFoY(%EN8c1})Oz5->jx1u{$L32KbJ?7?@@06a^Pe}fR zLvYUQ(Yn%_3^IW-0gtqw3GPv*CVa6*dekZXM+83FVh@-84Y&M4zUL|EGY=mp*I#t7 z)NXB4*aHSy(IU>ae#rVz?XZnAz3x zzN03|G5d{{7gsgNhi-maeNjO#f`X2kg#2u0{(E|Roo1-tiH?lc9>uhRC7OQiLgA7f+xBq`qa zG_1D9s#X)qBs@tF=oAfT0tur{NE5A4&UV1xsCU-W;0JL+1&O3D*n>6&ZISi#_RGWQ z0pAIQ`^@rzd(~$iHeOy^+caoz`Q7s;3ZEMEoXsqHv))(GRh1R;%O8DL)6O&pAN3!y zb6?{0lZl6{Bjg_m5T{Xhj0mvtvVFky4UKZ+EkBn#|Hj_BF&(38`FZm3D=v{Qe&&Cq zpde3s@{S08>CKT;tKiS>DF1cqH_D?=ER=;$KQFJp@usX?y+%JW+S@xc-K8ZZGG)>P znK@&c%sXtZoN(-X*=PDRO(uE3d=k2$1lM%vzZNV3Nr zE~RtNm)rt(9ihdek)?ljItSQ+=xA$^=10GQMST0Ef#*ox*aMUeQ2~#LSG;xeTQnRB)!BuuEbVY} z$5QsTb>mvL91Qj17|Lv1+m@e;Gir0OaGmfzoTm;B}rcgj=GE{X1Y&Ryv^n#Qdd@M^Li-^T3prSMQc?9JW(D|4J#W-Y4Cg9+1;J?x3T4Jv~zY z)XyZlbDIurXJV_S$q-{OoG_W`MBDWHJMNa>{_!u{FIVqB2Xb|uCs)z-#-zjQd(=Ie zs)7CU^u*@Hcj-xF#YcQZ3TtNiAKFnW>GzS#fgM)RJrH{KoZA-OB;|)+qT8hW><%nH z&I{c8`*^Qv8xA9BK?CK^s?OvY%{jPa0jK!%!A~Z>eEYj`>?4kb%;@fwo~};Gttgfb zz|!$!;Lkg9Ga}E?20ozz4%2_MLH+?2h<)#eKh?Yprl5@>hUIq3ohDbs#d0O&>a(6)`8kY!uttYye2)%EC0;SKo1JT)mDc4C zO3u_1q-@Un^njCeZ#u%r5&XJC3YtB)j@Bk={?`|z1j9x7*nH$L@v(RJ?A6t-GCl+bPihm4dKUn)&G8VvrMfnmUl0h zsB_E7ddtbk1r;>iv$HrlKvp?k1tHzz@8m+*PuQJqLH}e@;=5W#e&)_OKyJAH%d+6e zL)B%l@}~!;!~8@@B@G|-Z(#|aUG$P%{)x{eEM`kacer-r$BvO7eeY{>=4mH0(N4s@ z6oxI#(eW&pzGQJ)e!831N#k?BMI(Bmlpp*aT|h>;OPd|%0OelWnubl%`tX;f?9dAW zi^77f72^t*A8np{;bq9xHM>@>-gy$Ef{8RRNz&u|*>5WU&YdUYHDjfBA1(kYKH_pI zsGgDj5HuE!aJbzHnnOn2=+!nn_Z!LS+$!aBFMv1=-r$lKtT1(N)=DlEGw*21F2mW! zg;36UxVV5@Q0NJ?W9V5^Q|OIMXsld&MM!%LO<~Bpzq3<%FsIbt-Yy+joR(Q#B^Bdw zRxNLlQj=KWJO>X9O~B(Va+sbDaSx*h9gcg)-hv8x+?Z-v^w_;1m?}TfI8*r#yTAPE z61n=?ugMb&pA8YJ9$h70yY^G^p$pz8S-7Q(a8hnV7~F_@?V z9*X=A6bCPMX?XTmk~MywR2+I-9)>?1+BwPaWZmjgqQ$-!7k$xLV z9578b>@=~!JL=a-|GKB7_?S?xG=OEcwz(j#%cn*_ip;z6zsZP2$KV5~351{N5 z8}x9BNG@I-;yCTpAa0)qEg}kH`xTKRzw>^XUOXw%8;x(M`QNs!PQLW58|0U_-L7R~ zbaJqj~szuPvhlr>AAm8H)=AY0Zhi#qH3kAC`VS-xVW{OQ-X z;)qqZ1|ptkdfEQ`XBAdR#gUgu!_z;LW?b-6cEFkGUQW_d7@8&94H^zl^Bn7p&Uze= zd44#s$(2F60;f{H>N8*319J6$kgGz}A-e)LnLN3Cg)?Gr=4O!g4(EtW0k-$$j-IK# z>G}u0Bqj4cik03|({t0tz7YtAt)MCRw&0>;$={@K(mcKK$K{Q}Kxq#Oh;w0>hUKeq z)^@!V7L`d(F>jX2!=f-;Cx}y+`Z2excL}3sEZw{}7(&XPv??7?r9#H=bq0oVdDWxj zcnvM=GtlT3l$1(ObF*}<+br#k%~CpT621_5POVeqE*0Tlr-+@Ja4v>>>DQhbaIeRv zIRhTA&v7hFd3V1ty_QacOoQQcPc%cxKb7(|U%ozJd)4WuoG9P>`gJmM`qV&aGoIw- z&)5^Xh_-)Yxgl z=g*a@7L*+!4eB;K5tm-|nLQ#`XTJZ# zkgMP5?gC9ulJxl>;p!psGX($q5G==$rd;XW{Kz+@d*3spnER&Ddr!|zNBB7cULz7a zv=r%_4tmeAt&64b_)i4{9N|?+!gT1r{`HlrJj-xQWEOkPSX|bJhe{aZi?~Gyinpto*==uIC{`wP;JC>cvAKI?ZW2z+C zN8j#{tN%c*F8s(R<&NLniX<_Oc-Wp=g;F`Aon{d~=H^*2j45e)FI<4v_;1}>C#@@1OHn@NnmNzX-KF!!v9>meq4Dc&!yUbtNAByugC4c6*v8=)9O~9< z0olXWci17U`ooZRe<$wz#eUWpU<~t2RQlYCm5&~E>#B9qhIo50&*%i%q}re+F!Ez= zgh4#wZG^ts&={e^sm}1Rqj?!heu92tQe0Goea&v~by;bN`~Zsig(v>$lrwo?Wlb2w z>82)3FP;wC^RcuM@}V(EJQAK;?_l@WzyCAo5v#@JqvZpi`kBl<{@m?W&WzQ}%tKC; zi>~>tj2^#_S^4nq|NNJH2dl-^?Bx9CRnL21#oX7{jO|r;nwljBKQW&74228{2|r$MS~%qGyTKn6HEH!@ zkytP3LBaRK4l93jflV2{1nXa0CXJixq!(Y(`ZdYah6FZJ2%Th~#QpFgtsYFf^AQZA zIKJ#RZQ3fAfAaHj5*}7oHBLTo_0OalL)$yklvLEnxu3jM#-djpHD9^@Mp^XYD~`{W zZ-elhkp6(3qZQ0^5EmTubBe^8YNf4qkv5vqxYO?gb3jM$m|ih`u0t-=%vNskKV9&Xc~ig2;!j(1YUxE4Cm-U(To#xi)_~$ z_eDGUijBAJIDgY355*J?BeI}cweuyU`~27M#5(uf0PuJ7AKiw zvBVR?SlS6`BKdI{wJ;(#VD<3#e)u!>T9}t5qe?M1zMD)}Prqz##Iyj6uHybTS8-d~ zy!@yFG&}Vg`7t-o|1k5PKt2V(*eFnGS>wQ=G=B8?Lu!%aB=|?_FTCP^1=P7Ki+VKG;CfIih@^gF8tSDoowyo{C8}81tU-Yma5}LIdf>c{)O8mw{?w_<2pu_Q-+GjSCG&}kgGq9D7iX77Cz|7RWPB$!P$&i zo*B+?!X$?0OtdX~P_F?l#C_96lMmPFm9*R?Vb*T43Ur53$H>N%2b0K#XZ4(I*F!_N zV1cudg6ZSx$;Kw>TDev#u)jHmIbnv!m^G9$AS#mQRsC=Ok4Yi+Q*(hCU;CXJD8Op( zjvGW6}!R zad^;QwlSR;Rg>DsrI;NEmO((KLbykXaurq1blS6W1zErAUh(D1OKT7?Ta+owp5ac; z0uJVZfrB{T111yeH%j(DLvk_2((=OplWxpS^D+@TiZkUPNcVbR>^K}x zv}_o+egs(}EU$z?+;flSht_82c?kh?BMjmpPrLBt`WtRZpnkyoGi55aQ0)%GRo?SY zyEGxH@7#22nC}Tbw$Fki3q{k9lb$#3Q`t?=4Gzvn@`vkV+E+e~MPcJ{g=Ed(0X*01 zZ@4wV-^h@wABW`1bSohrR%wGts&_)3Qd*#$rpX&M1E;NBC7E6IQh)z-(za=kjUY4 z50Zrs`~?><)M_>?|E9c1%k0!cutvdx%G8E=>7L+U0}cLDgng5T9(;g?H5z^hSKk~n z%+l=eZh1AB5I&OLX#S)AEw8&G{~I=KmcQTkkmW*QHcp*8`mB$I>~{6bTx`R7`N_Ma zxo)E$)oZW6Ay48mj-%!u=9eKI6UsBf&!Lom-k90avgBTA-?~b2#?4fD@uvj*6r>Sy zfNhlCUB2--DVR22nZl-KpcyOLuirTGC!|i5 zji<=#uxdLC^S~U^j&ZJ8jVCb)Az@+CN3V*TC$(_WWmZub3`;kpmFdMNcNh^p)Xlum<-~9?~IG`E}Ymk=aHlPP}klX+AcP&%*=6(-a@Gg9G z#jorPy6q}F3wvn~Klws=_)p&pMZwECj$VL<)bdPM!!3`NZ-btnI3aT|h^l}c3HjsZ zVBhM0q<_o|<$=%Nc)+(kru7}uW}U4KBCYGOg^0Xz3e4EA^~r%@W{I}$lR;Q<1 z$nf1@hBU;`t$*X z;k+WPAO2#Tm!_TKy=vDFJ9P{CO@`AS>Cg=y1-!((4_7|Mu(z7WPxdynU~Z~YKQj7} z&Y)~lNKH@;CeL6BTWbjjgZKc!EUux(Zw)hH5Fdbig0)wVHSc>{{{Q*lqY3#raKYIL zcDuu3zr#+^V-kaL{qwpg zyhoGQ{5}2rOL33DD#uKbs z$7>(b%Cqysdv-Y$6Gb{3q^o&Lx?xlfC;r&{b1yC)$tkIZ2USHrRF;FWJ_O#Sa`j?} z*hrMCWw^&YBpjg>2Vbl<6Q<5TeNo9e-|=?A-btA_4M%c#AyV0#bEORn$m*W@Ic^oi z`4be@wAp1G7?y(W-?{`tN7MZRWvZ+IiBs`$YgJwCI?2YTo+(||BZgvRkG2n$gvvVM z@s#)g_B?N!y41D^U&byZpSCiJS-v3S6*GJ zc7AquQvNe={c29d6zSf&Tt|S?jbZ}pAxo|Yozbxkm+w@maH;IU-Y*XXQ#&vCQYgM| zELshkktkO$L$16k8iLFV?fv|Dd9?;b^EycWU3Z)W;FIk&8R0?2@oJDx2t zw0#KV#mlhEoL8Y!^1vWk=JUGQ3t=#NXe~xo;?ARNne6TjZC&xgA$q=ICi-B$5fC_vBfX z7sHv58BUnYX#U|~`A_C2WEZVqq-%EY$XJXf!x=sxeG5xS6QSoU2mVcn4|FWgj-2Dg zp3eW%&xMB}nYOY~;|I>zZg5E!|9zMz&$6;emWCANl~5j_VRk*u8&R*eO729g0m0R* z>BA$!45o5!=W)R};jpk`xnvbDOsqeV3iZ@pB3BzXZ`I6d-K0!-?U8+am_k*Bj2$N7 z50ZHM9R0u!bI{Rc+h_EhJ0;K=ew?d30+N-5Q;)_1vMJJp^Srycfb7VRr1wl4vvv|3 zl7i;y?Vhd6q_})C8V26?>zPP{!^KxniPF{7D*0Tf#Z})>%uZxbxLiFQ)HXgLBRBxG zFy=o}(fn4-g2Nw$jEl?qapG1Vl{C_`wsyVz{9g}C9gY@Fo4uO@c?HFLGy{1BMG5gg zkK-Bn>{|2at|k9Ar#=$psts#rwBbaX7XF}j4Wh4*J>@!J2ZP~h47xY}SCuo?voY+>qJr#dC-d)3iktkQXW`=F8Hn1K<_O(aN zb~V&6#@j*HJ_h}V^t-R?tsUi*pekuVGhy1ajnlXIdDeO9LGP8$^)Kk2>F#~cmeTZn z=ElMuf`ck(7J#je%xPaQIg{U^I@3u?V!Zs4WASxEDQ0(J^>#jb&=jm#7WO3LL@`K$ z&C3O)!|7Ru3~1a0@^Qg%{ur0}PYla+b5U9^{e1-j5x%-x=>Q?0)FM3S}vqCOJ9jyL+QN zwRWnvWwYdFXJVmPvi@8;NpNRD@kKOt&?G2GRaG1?t)|!yU97 z#i3KaRM>IlC83!hUVuH*+2~O>zjCM4ZCxs5^FM}zs!G$n zX=B!Qz(Ex>=~3`|x;rJaeKWRLasC;W!BPoC%7Ufh%PuICP8_`6-;*bO*;LF3g4JR% zxN!*rO>jOVUfyxcCrc|A#J5T!sc;L2?L&{cKZc`4p`_W9@4@z}nKNg~!6zLrd4&a% zgNrpNrzWfc~3|;;6{`0h7%?;QU6gt z(eSDK4>x?2el&bEKFj|jk1v$(dY6(|n5Mc-GGT_j?S9x?A3sUVVm|>#u3r3s_t@(W zmyhM2u)nuc=lc2&{IHajl}T=HZo1*~I8C|XG+YbTzBH~{Bo)2uuqEhZzp7Ys733#_ zKKuma>SlM@jYmpGqFjxxs?@q|X*8HnU%a-;>VMSTZz8>>QvGRjLe3;XYv_3OFZgJzD>6WGcz@F(ZND+p-i!kJ!g{jW889oOFRnTO(uHdf6!jysvZ2r$z z-QWv6%8j_`0~;dAfdXtokF|$$(2|Xn+)$F}$qMGLgGJ5cda^Qj$_Ma7>o`64X77C% z((c6xW;}MZ8%K_IU|)8BPNtOOsL!(U5-8{#jgyOQm`|Fhz^P^6$MQOG{=*705LD49 z{b&VBQsoRLXp?lD5+psbr_WfqfkTEr*^`kby z5f-V+cQE%D6m(u$p7f5w9lp33Eln1>ShdwGBd?de1{tlsIs@XiCV_-qMYU^qa-^z>qw zT~9E>BC$@~6TJaPeU2~QPwzM`FE5o`3=?u5(aAF@(S+$cyZCRkLPYbgLqwVakDevf zdyOi%Xnaxn@t7SJ4e!bW92p}0D7`o@Jpv;%|79ias!+=})yf3r^xEo@wVnFjOA zqyXou*OupxKNzE~W2Nb(KTFTH&~_SW;4wR?s2tyAurA zL(>JV&H>}WdKQ`~5YrJR_DZnV5_2;bA2>70H#iD|8ps2$>+nZJkY6F!)Aep-gm~CE`Jvd8DT< z4#Q%#hW0iYf5;rE8eOG(NO?u14Y!z*(y%Fh%&(vSh=1b^Ax+eON_PnK?o@!$^hC0g z@EXle)I*eB)PG9%LGpjg1ZCIcR|%(ol(*a;&SCDY#naJaTOyg$7C7Z%?Nx+NERR!9 zJZ8kpRi-z8t5U@F*UyJ3L$hJKLpx&+t)lTZU6q3^v$_Iif0__%P`ZCHpw|K=uw|!a1VxuGds3o<3I0K zMd7HhvskQX46-+hMPe9YD!}bg#p7xuqqs;~+BhI2R7QjDR}EToYF z;e-=mm=g>Uxd(I=Cro@&*&VZb6@Y@V5!^od4c;!OrqRnE32S{`-2`BnAa-vYU{Ax48xR!jBdz z=~(4{!qwb;9rgOjr}~OIAo8$0&U^1!37i{Ia`m2fX&zK9vkjnJ0qVn)6Zed%ZbHS8 zUxPrB=AZWDgFXo7L(-s|2Z_T`?FfT7-PGOl%TVrN!iYOlG-aNY%{^bbUb{o;ah7*C z+GchF8E4ug=fHppnoQFLV$GW*0}Vq~E)Qnlpal(o@hD$C2%L&E8D;Rq-_70 z=)JZ}M-LW^;f(HHJgPvcq_MJEFVBF0@eq-IUZkN<5a;C@?uo^4=!owpnO_WRgKiJv z>gw&%4~x#8PTADdB4ZBNPb#r!qyR(jI!_L1(tL1;h}q%&&U_4am~#V87w4Kf|Ky4P zQGrnUap6%2(omcNuW>RED+gXYNSv`54C1uoLmvj6Z)ZoEd>FUEAa3r=Gx5D4|NG9E zCP&W;pMTxa+9=7CC9|}v%#vsCmo4j-hsci~TO;p0@mN>ZEF1f%WcdJhr4g5%IRA%U z{;@79Xg3!hiahAz1 z&(Urfq()Sw=t1IiGpIN?W6<(#(02%WFq{w%fkDMRddl!*KS58#8JR&ZeZ2UkV%`l2R#g$8yHd(Vakp=YlRo%tcO$Mef0M%hd6!Rqa9 z+@M9gxxG`mOA4fB>SQS{E5Q|k7#_s%b_Q2&Yl-vwkcPhwi$ltm=L$H~{2L$03&$f* zGD$wx_Y=AZcIwJVxEH)zXO{1ypfQNVTpIzvh2Nrd(b9 z-qZZw2^SceQIt53`~O{)AGJ@SIM&mVQLI!$}$q@I8)eGj_0JpOtf7@UC! z^}!EeR$if$&;5WDR85hVN4|=+N{dxLSbb-xv~e65R6%oPBo%ZH6m*=+2bx zi}w8ld(v=YI=iqKjE9kQ;&vKU&R7_>sz| zD52uWw?TXzId2B>mf)Z9kOt#T@+2#Nl?6=&Wg(tMO*`boAZxeb zgo-nrjE8aRV>qE;zVK){cERE9#rx$tgL{8<9p;3aL-srSh1>T2@oU`P)7lUsJ!R5( zx%9$wT^(S)Sr@!N=7qSQzd)XcoBzI!Mkr`^=!1Urgi?`yAqU78&e0cQ5okUd|IS=o zFtQ%2{V7wFG4c|r+N0^VEI>> zO+wFT59F>4Ww=OOChx4pnqEMy(M&IKhC!gBeNL2;{m+p0r+*}ki|^3|e(51^uqI#z zRM2D;d%anm+a$ZVO3OeCpMkx?j24jEqp)(IpbPSHrKGr6$|p^b;wh7)0jsK;pftMx zsX)7QpE8EMJ!Q<^w0qdIw#V#}L#C12sk));yP%jmp`g3*5zy4pE^BdfRUYD=Fl!$v z!3x{_JS?z*g4V57_|m52I3bRIA*!q){R3rhsQEVnDu?(ndd9i{|HG&T7?{8697rDt zs~vy%u%a0B{L#*wnPK9Z@x3(vH-Gy}psU{0%@uRGoFBl7;+zAnkO zcHQ{RFGzktUYPeD!-F(zp6iY533o>EcVa z>uKXSkW@iip;1BSbk|E};V8$~Xai3gZ0HGsI22$u_8Avpu~cP-97mM}b=|$LXKIoMzL~p7Mi`us0GK@x~Arq^%t7=9e z9R<4YIg5(fORKhh9!`-lu`x)0CQF2h4>A7(DL^s_BpkB<9Zd&aT%H2;I7+Yd><0m& zI3W?bdC<8yJN_m!>J<)ze3(~*mT!Y_8j<;*jl)2`dEM1aUdU`*wOH=>;m4)Dr9Nc8 zqc1!~@19>=E6W!BD}mHy7o97opK`p` zbbp9DAu(wSp94G!RriTk&XD%Za_Lz6jN|zr0>3}x>KoVouTR5}Jt0?@Uidya{T;_y z65`M*Ct)xzo;;~X;${pt_h6VHE%P6=3*tj{H-83$G=g}tEWYp9{xJ6 zRo$o~sOIhM470uhlZ*jWnmjr$^UZrP6mbG}N9B2qkKewa0q2klm5OoSQ8IbR)Pq1W zq?ZfFu%f$l%Qn3d&K>xvgQt)){6Q&0S1TRfh~+P`;TIs>_|7S3_GJ1nB&+vM!@e6& z5A3+#xf4Be&UJ5WX_uAgssH@Rg|cw*YjVMR&z2f)ugZ1j7YFw}NAs;c6E9E68dQ~c z{8;`5%72vZfbWciyd&hlM_6C!gn;VB2$)|&8g-0^_i1_V(fp)JZ`6M2i4;+xUW}t!bMqeeMC_UD4$`;X2B#>sQEM zZv7wGu=15)G>RWIdzRe(i<@=sIV%4No(HlRV*cCLJ}XU~ICx{iJl#gbc~;7?me)YE z-=_A)6P5|vu&o_AQn=w^DHwMM7Q5n12CYKc{v2*tP6>wpTHN9VQFVS(6&C$VSkBv;sEjTLqZHnF0x5#iP=icK%( zhH`L%S4%SrfLpK7lV+luR6EY_J!8|3yX=nBZLxlFy7v_b< z6UnP!_44FB zKa~gnc)fJAgy)=%z`ir5%Z2Bjy~`Vr7^Z@n7vC)nRYyq~4&LBCUv4EbMP%toL&pK* zi+zZ<_hv~CcB15V)Ji^vQ@MSQ;~b_p3^Fv^qq9yu33Bz^9+RsUjQ*|PxC!emj&eip z!33r%WY$gkw>oM#As)^3Ao+IzEs#ei@aog19nE0z(VYyXCqfP5;g-Nic-}^wTQM1n zl1EGHi@(#gMY*GA?eaRtq>faRR6+CWlPB`DZd@k0?Q5lQ61KN^y!Ih;6tv@efZdt$Be>C>wX+cUx&*ycviP7JlB(EubKy>=v0w*2uTtj!kDXOFB(JJ zczXD79veO9cBFF~l<_vsZC6&vn zLt`+An|mOCzF)JWh2exW3}<$HH+&#>`U#p;{!_UJX%QdHzvbH?{ToimXRz?}zoYJ? z5j2C)<6@w9pLwc0_u^t%?UivbXx+w@^77+x2H>L zmoJj1@Bfwj>yO`-4Xe;=_C_MiPY|wIyB;5tF*$bL;qeA!ko0QURPMnDh{J#Ts{cq6 zJ~aC#o*-3KcF+#jQdr?c1Et@|8ynr>0@5wrMN+!uaU54XUa|_xot!3P6IjY^qVGNn za`nP1kgN3xzIK&d@$jU64kN<6vSgV&`Sc5N+R4XBF&4b}CMrL^gNINi)DNJ^he3Kt z89v~L{!PRP6(?PDAkG&L2IF?waDFXL2GhW1!Em0Pkx^P$!&Ed1z3Kg=ec64|xp}GN zjy+iG@Y_-5aj8^dE5xCpwzf9e{L;Nr-myZ;=DbhY8{)6CN%^Y_xllGk6rvF=L<5?GUM>$4@k-B=GY)lAX>-^L*IDA~ zy-qZ!&7j!;1)W)0DHG-&f=0L;10|36vuk~stVSAFhl@|!>Y zbM5 zaI#hEo0U|qJ`1_J{jNbH+8J`S;D|%y@{7;IjrWlwr2qgx07*naRBb<(CCgU$@lD6# z+Q0nfN5at>r<|e{AX-8EpdK<@|B>)f@~Oy0-6P~uwkMJJxJ9lRM}+ob4MzE~pOXAC z9DWu2;E2+H3%(~+(ERG8VAnnWdl^*_lj2$L(5$n!%O0-wdSMgcXQGYgfeXNND-AMU(IuKwcn`1orYN=}F6hpTBn zebuG%-m~7RBQhVo`b)B9Tb9uD(M?jUI#c>{Rt-y=?2FXeSV57S{n8 z@`pSB4!QdFE|aUzec~f>;rq^3d8(^#kehz;8+r6USk&ejp7DO$kH04;<0x597qUAQ zPoMd6(A)@VlI15Zl)&JTvNTx!gZ|^V7lDgrkYB3{gft9ic6>KHxx33pOxjjHj_q?eU#ASn-xC_BeP+ zMP625TLoe-RQEc$9e52*?1>CDfSUM!r*8C4-A zP|9PlP-ygiGo>7hDk{p$q>`6mV2GP5whOqW3h_;wI!PXW>?y6N?a0q_FTN`0zxPZ% zO^nJW!Kg&TmXHsPLE_t)e~n7Xk**RYVUM;4gK?Yu)33$JU>evg7|ydJ)K_Q6yjlJX z(x*?$=&NPmCzuAyx4|KW55z(Lnm!LI?$YNRh||)8!MH7bbhhjQb>Yl{;XFIQ1F>l9 z65Pey(9|r8v4Cu6ZPj?!J5Q23|KAVgl;e+fDL^`>LH-_kY+*pI7R&iKij-41mL%s{ ze_cAw!Jy^aU?l%7Oa38^tzFVS;Z!J<(W;y|_slw!HmNy4#bXbB6W`ojB1M}X#k~SG zl2urtb`-Fvzc}>_IS}VKd$A*a+25g5mDg;W&$0_0Yw9NcO}1j6)@|4%_2^;oU;;LDD$kw}_~{O&Uo9;Ya8Nf7 z5|2KNMEwwfv7m%BbTd1?8=lPF=on1pe<<#nVR+O$gD`2oh4NnrSvvF76XZgy+^1Q& zX05bi;oxu$Z#z5h-Dk+JessND^^ptl0f?y+lNT_pv=j@qAis%n#a>Qeb@gE7%R~*} z-nQZqY2N&%Y#4pIj2SmU$`L=e4W);+hgXuOsH{wSW0_d5g1Z7XJRlW?&xj|e|KR{nQ^mG zWsmoSvp?w5zU?R9mwnJ9E<{6>hu$>L`{vo&+7rk2i7f4*yQ1ZAdK*44w$(Mt=B?Z0 z`kQ|y5B>W|zm$FlTJJvd9pT25IYZVcgM>7Ot6Vx8{O)-E9StRz;wQ{+i@UGA4wnIJ z!eszW(%+3Unz7Z4J~2$8=cZ@o6i8+s&TY;sbv*@~rB0KWAGP*>_Lls|`InqO=FZdV z{{7@L^2ig<$iinA$+9YHRUwhUf>^G_4x>YOv)p|a>2qQTP_O$XoGS1*+2)z8YRs5b^Ix5!JczAoSR!L6a;b>6=9n_Ipu9QtN@txl7s;qms$ zVsg;pBcw_6KeYV2(8+#>7k+!v(~mRhn_m8tNWmznJm%w)omYf>Fy6PO8BjrQ*}O?I zAG=1XPWUVueYg5G%Kp>>9S#vx=!UcM5LY49V?fb+hjQkS(%ciy@f9?OgAP7m7KX4f zB#zz-S6k|kBBJ1zlzu2cd2*#T&-vy;F>YDlTp@eTAIE}^T5n5EDQ>lT{N6vvtQk`f zZr}=P^X-H*;oKy%G3D*mfJg|-GogltV40m-_{!h82L+hhZ93OIC++KAfKtWt)Wd@ABp9ufM%Tjz8*f z{n&F~KC{Y>Y-R*nUOs0Q^%>mXi8x>^7W7EbBQnUU}sTez5igDbeE2Fq#?tXkuCqcB~w z_njesL3=oP;y90XB0c&`ZNlkczD;Q}>`?M=ew_a}4_(~Wwz&EkuM0y>Z#aEc-`N(n zympWDpo3C=+@~e4wE8Xk@tUw4lmbUV>Fa1fhYSsaD(5(Z-4KZ}I0^?Z5GUxakusda zcJ8`E+=P{l!|rpq^EoCRn12qMT&(aeEGX1DW^PGv;oTRcYY1KOF${;%IRpvM`eoXK zM)ixIxl%s zTR7u`d*`q5)Kw6^$Iwe|eBpP}+MOXQW3#2bV4UP)K9V0OZVOvHbUVtg9_7a}}c=S>2N_}8K(jK@ZguEZ1euEj~{ z8e2AYpa?0jkXSW%Ng%D9`7zpG`ywH!CjYs(tjRX zcfRngo3T)?NTyDnplNjRW=PXZx67uEJXuqEq7+n45H{Z2a>ISGmDpltu%e!vTbDNwJib`Kl ze!{pbLscM{uH(mA8xCK(<(I!x1f~VBj?;K-!}77HmeVgh;Axq6dR{y5+wb~^Oh%i?V@gRU7>_cDbhS~_H{{RND?uG zmX5s{TG!Ff0~;vqLAN*P$~jj^#^5m2!l3|Yql#aYh{}U2KV_CRqJ9Ly0!;Al6)8xP zjw?_{%!Z=l!-md1o`!JR7&bE#Td~mNDK9J5-qUF(9V5$EtdhI_;m%e5J+|h|oikgm zxcFSH6mCAsr6Hc)IHp;}tHF5OE{)rp|A!uXS}wlo^Qs)~f8;;%$De&)j$6RHk27!z z683O5Z#XNvGHS6mFf)ALWh(b&fyp))31mOm>S z=kD^&k1KNzmJKNc{Toe#UK9Wf?nt012=xHseu$Z$ENlbJM^DGeRUcO3M&#;_zsuYM z_me9=bRI)#BO9j_PixoK71F%q9@!w1rDfDnQe9oG4?nIrtesZQajxwQjLMFDCF7+(OU02N_LQ_!TrNJ76$hy(&^qGR$}s3|xTTbeixNoxZVHMGq=xha zUKI}RWGAFC`WykwK+9Z=1DDvrC`8YnGI!Ob=gZ{rW961#{T_AM4TpdD6W7Yh)wS|@ zy)x5H@o_}brw829YiTF+-2?fFDnM6%=rf!?qi6O8qv0(-zW*WS-}jGr9j7gRmloqb z-wMVasCugT`Cm$Z+cw;eFj3(=JqkI@NTrGCOkJfDn zVnIIUu%VnupTphndDkg24kJF__`#2*6(=ateB*|n$WknT{Miq_iuM7cUS4|;m0y!T zKM$agk9@IBv-~Ohk#@F6<%eCYBaJ;RTcqr;3(-cpDF@%L zC2%i;11jiPXOm=Mn-ay)E5C%SIR?fBm7Nc5;GEpLaAc?-(s1X&*_$0b8BTYPh#!`U z$h{b3%c?kAST#`y@j)1FU*P`e9yE--ST+9T&t554T>WKTM8-Mki~sv`^4LAU!#-+v z5}C!Fm_9EWKQ@xl!k8&Ue25a#{57oqFMXMh-4G$J`l* ztl4-E*0UN&ASu{1gx1S8rER@yQ`A)2>|t{+JQa2ecvOd8cC4pe3OY7OS=&ab!eXZE z-d1dVnBscZ*qg|E!x@WpfR%qNBaK;O_i&e%ME;szD@PZ`g0n1{jp3BPD@%*X^&E_z zgUR9=D1GsefW|OT9!Q^P5mXZ zruYmgts1MnW=>UbWW+wc;xVPMoy`ILxI(X#Q$hS#-PnDC)NS~uWW0QblpXSZOzXI2 z(UgxVv;>6#X~t>~{JOtTIz6Ee*2iSw&5mIWC#2D%iIJ4TEcB!^vUE}L^U^oU*qYHgYGdg(NRL5Wl^txXZBRmi>OjAiCb}7J z>5IC}QRsdrWkrf7g7*5DPR~`-qadVHtAdPU0?v4J#DnCuDWIf$Fzpq*0k^2$70z75jd8H0Q?eBk^uF6WQwG=GLO z-9`f+G?s?)!{3y)#w~tlM9J)hKcPipWG25TIn>#|_6ck?!@MK5=CBZH z*uc30B?i{%zk@;CxhDP^4JC#jqh<7I&or<{$GyS&x{|`L! zq+Ey|I~A~*WA`5|H(hq9{NeG{pdUA@*KW`SUI$lTJ;2nHbQP+lE$(akF;m*6xJjE2 z6*!f*DR3%rHfB^J!6&Z>J&?j0Y0jH0>$CQet+CNqA>Sg>xJr6g|695?y(axoOf*^e z*yDmTD;wGwFO9#n3mWrlFz9A@(9LjzLHa6JkZre^Cm>fiS#HAEuv{@b|AOHeU(il{ zU@;H>I0VVl^7u7hjZ^WWa@F6sM)u2kT{iVp$m){gq_BF1_L`}fgNKl?9MWd@aKL1r z^6uJ`4B6B-S_T#*f2+OI##I+0c-2 znKCK{ox7v#dE ze>WsvgJJP$vIcg%_A$3}qm*FJY4w(eCAX(RvN?avxoZrIb1!x#hV87(G(r_7V1zV? zTjjv}N=qd_Mk|?oC~4Ajl;gh?S+9qIPdHyj>B0F8hB4d;Jd?%rv4Q8%W2_X7HHN#L zTrGuMEz@{@aN9#tHubY|-;|6eLwslbVwwn@ zG3DmMaL79c$~gl~SQZr#wvuw|-!(YTdpu5G{PiF2(y*L9JNL4y<=bDp8Vk!V&^(ZC zVtedl+>edP4W!5Al?k)tc=-wDU(@TkCjiT1{^+lM?kc$iDe(V*nMR= z@V?F#DQsVl72>s0S^I)y^>?YV<_d8h;g?-hrDe|2va&ZkUV?@hBk zGDAP+&mbk83fUS-hND}e{8O%SxHlh1&35NDNmJu%a`ihW$j7m|wy~u{k88Z)M?aUl zPCQ-4jT@(<5LC`o%)7HJnS-1kOi#{2WevE=FKwOtIkKKNd^q#00(?lLZQ`z$ z9E|*3@_~2D#PQ?gyO1-UA;Eln_N(8QrOQ{!jo-LN*RH@T;zE30dOTpSuKvZ7L(k5? z=E4i>+useCg9y zqOo+W;L!7jKXDy;gxwfQaP*jG%eO&4B8wytcRW6k*Ri_twDYbADd(v*Me@f_EeI)R zrun!7sx9du*oC3^3$FR8bhXy&@PEj054^BZqjIrfa4H)%a2&4XJT@CT&SUciCeCFO z)=ZcxS!3tQwoxa_tEK-XuN0gjOBxDg{i@fc{@LHiw#R>n(}Dhu`RiAt9}5(<@_~^C zt=tV-xf`@%Fi5r-PUs#dYhjkrk)A#OzF){7%AMg-l|R$e+ptPnU%gv8U;3T2;8wQQ z@h8hgAG*x{u&KlS=hs|wjnA{-UjA@;(r>$)1E!?8NRKP&E33!L>dMoluBlZI2F=Hkd`0DUtOs}2icE9#qn;Chc)Hm-T`;M1J{p~ zKw4~?AHxY9&H{`VAq$5xX#&f#Qk+n6`lcNq9XwnS;&><^ItSezor69LbA>f- zND#w1j*ihKO&6AQNy8bX=uvj@(q1p^fP$ft;n&pYs&XC9efEXL8mtz__HnNA1f0vv zJ&78bQgOUsF8Ie-3^JfggM-BX@!(_fLF{kl9Fdum4yut`KYF-Sm%E{MvnQ-5&Xs>W zzaEQK+`Yt=c<<=OmN%Hz&R%)t*?-G9=UzCXdS2u=c}QOJ3C>@$!Q=cjd)YRJO~@y= zkn`7Lr9K-6r{wM@+hSEZe->+6jrr@xr32T?^|fuq%JDAmd~uurrko(hU6GK-iWB0| z#sx^S@~0hp`fbVNY8(!fz;Gdl5Ph16Ln}REAf$0+%|_Dw^|aOD_|-+&%Y3i2Zh2Ey zcUH>!(xW9dYJVxKtdS{Grs&G-7hil)!>?VtR;L0^IR(RSlwn4eHY0)qEDMugjuN5U zEL&te4)NJ6*&80mmcog;j)IA`Qd6mbTyeV0$<-eqdoDX%GsA;Q+#Ga5KS({1j%!04 zO~XAB8jsmAfQbvCdzfa_Jq&l}@}dpM6c@oRz&aY<=vLRzsPU}9T9$hrcobJw&eug| zCU=$(!eCk~ehnP>3HqnCrOo^YX+-@7$oOc~|6ueOD4PGEo#Om~@#FR)$i@oqoQg@( z_VVr0(~rIIqo!-!Gd{RbBl9;wL33`gW$PNrX;b$LEk+{oWf}^6mS?X zY)Dn$vBl?>Uj_xHM+s7!8-H=G*xwxPa5US91g z)*;g>WL#x|tgq`($2A+aNgje8Ip5vJl`Q<8w4=cD2`X$hc6_)!jEb2`+ZLMfoywc< zIjBTEIKs3(XNqjh-51A|R_KDW{-%|<_LTR*zUWRo!_*7TpT&OF_@x7uXO^{9aw~U( zCRaS?Jt|lH@M8nVDGj4xG=q@k-`l!HIybz8tx^w4`$6T-B(tuSRu8wwMy!lXP%Km4mm_-&YbDXRWOb;zUMeV*$pab;*FWr($L;7 z1?wMz&c@=&^6@HTmP*Q1d{Jw#z2c%)caj(vr&4xk3Iz{pV`(Miqy9aZ)W2uxWyQ~{ zg{QO84B~3%1q!-*-A;*Q7Z3@}K~vb={BxU* z-7`KijEa{g3Y~5ImprB)D_vXb-Kc=CGc zq-fhFsj6Qh#r=;u;2_2PCU!X$G&|lWH2LG{w7uigS=u84F*S+pUEe2 zp03u@xK_IBS4sCa9D|DOD~*|BWJ|_A(vG83^LRcyWT(>m_#kcvjWAFASoz`?zbKbn za*5ug#CR{e>@uOEo<4oLCWAamn~}i*mKQ%LS&o!fj=I##A+n^dQf97tQhFL{rFg&7 zgg3^ya&yDFiA9KU7lhY83X*36XuM&JJIC$dpY%z?mk$PJL>>_aqvR0kiDNhqoZG8t zLwMcQSIDpbaF_fEikXI2?Vf$%mGZ-HU5h)e|4SE`(cb7=T!ce0(ex#RH!=o6G(_6$ z4KmUI9;L5=2gpBzM*T!XW2Bjof@b2FDK6;A#Nik! zbYPf5b#u5e*@VNI7xLEZNQ)-G@A!*n8THSwxOq)!7)>SO{aCyPH7?ITz0lt5_-?q- zisr{vMkr|9BV1mN#ZC}v9%Ax|Yrdhk7V*lri?Ai?;lKVG$|O%I2IRxftI@aoa6X7M z?JnXP3}+vT71~2FbEj7b6?#htuAl4em)F-fV++>*XYVQi?JAD;Ja>tEh>!#eTAUVl zFAl}s-MzF>v{(zJKxspPQuu+E0xcAZTYx~E5G5`z&t3lSo838g?|t_rFA#aOcaw8x z&hG5&%%1aR&ia;e!o43#Llb9GiyU>-kuq-FmpU7`E5BT(ugOTrI_EoP7oD}fb1;N- z*5QOR+c?F+-f-f>WDXrRW{qbwRY+dd5-F@$DBUx@h667{t2$R5byiVt9JAR&GI6En zcf(u?*V?xC*yIYK%1~P&&DbuGUF%Ji3#F;zTWM{om-_5p*tfj9RAp^1jX35m*DpKk z4#$&m%h5+4E$_VZj?!5Dgz}{Dbyov? ze)g{N`EjmA8Lr98N~C|q$I|QL$FQa30lGmCrR2_3RAJl^ zhIs6eM5Ky1!{fdH^@2Tf1^BbFPMJ{7MFy5 z(kSxcm^3fq#h5t8^MOD3bez(2(%5_)ey|VDVa)&y!^-sM_Iy0>;$@6MGn7>~%C}2v zWv?yz>HUg+Cd{QIj#nw9nQx4->OTViy0KeWJlwu|!9 zNYJR*q%&(>huz^}w>a_K!fw)5&_`->2g%~hVKOgsl$53Qk>z!1QdwRm4QS~!E&56t zFrHmI>m#XMFhOb-Op)dDr^%x)eL}mZ;h;^iOIzJkX`1!6>$)ymD2vN$WnO)u%+1(9 z=H_iB75Sqit=lju>d{}iV~G?e66s_jOo-#eICf=IR?1C1sFUeMsPC0OyyU(2-qUgy zELb4(=g*h@_upS>Ns|g)rv~^u?7if3<;Pb#PElHx)>oF-XG;FUx16m%Y2IXg0HusO% z7zb)_j!iv|vt;;i>^STJF%@=gCtV`QIOE;f@|Y(KC`tn%ul5cZ$!k#mP?ir-@j$iaX3=td8^p+xw?`_p0cgcn^e;F)?}dS`eC{7GXkdPGm* zNRNvX!sz_CamA8lWx~x%7{0=bJbdV2eaUtEJ&kp8cclNQ;X}|0ZbxP+Ry0s@i+k!AIoB-n zqBAC>LuMB!Ww7N^=9oiW?2*t(vuA)a+7~ZgEMI^9wI+=lH%>;37$Kuak5(sb^eMRR zYry`d2+dLRIWp9x7s=e_ev(u5wY1Iu1RG}ekovZ49AXhTXxJ-`LGx59c0cQ%C~SSE(n~JzURFl;p#R&Dk z^1M^zt4Y)3-H+T6m-qf{ifjN!yZxx1PJj-6ciIqp>4Fz~&6dJJPyDt*HW^$hmmRvH zwyr;$?q-F*{PN54$Rm%)8E3?{E$x8i4<_0SBaI>hjUt^N>y^$h>=L)`aE@VH_c*)1 z*$YI-c+wCuANLlo$&KF#CacFVoF%(~I$XBk6F&&7RZkf#&057~uubN~N@hIs(UH@+ zQ;v%+x=1EXnk3VwPginwdvCVcW+^9~Sb1IG-kdWbpEch}eiX8|WEtShiekyDm?Q(n zzl1Z|ZJvr3Jg^0keL#}gPEwIC(+C+159!TTX}a1uXXP}5S~}+S2F>gmV%XT=iEFy| z17~_eb~@}kZoEwHeDEaceYbSm`4Y*>$=7EckgZeH_q{k7 zSdlbn8LjoYv_&fuS`xa7#mk{u^GA^Y+GBOlKO^rty^dj)5ckRcJ>@gi%SSsvdc{L_D-3)(kQe$rwC&ge!s{dvOT)N5ZW>**#rSTqJ#Zbe9`1KU)vkAU(&ExkYDX zRSoiSAf~+niRJN+;^8~5!v+F<$wD3O<$w4Bv$AXTzN4R#g6z1K_S+NOq_s@^f!%$| zmclV@3)fwDoqYcJ=XU#F2l-rhnSnG%X{~9tX6gK_+qn<+Rpz?uL4yYAUgATB43VKj zhblgJ@L+Alnn7e`alLj(OB&vDLlN>KU$&Y#+nkP;PTCnex5-GzD_P}JZpuufPC3bw zy^qQGq~g|H4cIfZ4=p}J_J;7?k-l5Wg#5i^=~8_7IpgRA zjm(1~utRu{83=i)tM#y5W5Uyg^yIHRSdNutFoB*j^U8sfF2cLU16z7_D@C_^FW9wn zmQaC$(w+v zZm{`9qYuHP(CHy-VTjAN}=B`ZY^N zT;qJvW1maCU*F8QWMP+0VmpjI{myLpzYj5C2~FxAMLF{8Q@7MD|Co>E9D3-XQdwD< zjLm;t9RKim(fF97Omy0GestW{EzPYp>1YXgI6@dm$9OJ{VjA;sjN8VP>5$2vWwA^v zKjg78oO#HHeCedQ#OjJGu86WtojO%}LHKc;3SF}X?44te3g0g~SccBJq}KqMTeuyq zStNXGlUsEWMF3Ho;GyDb#`wcXPiZh2eGo@#%4Co@c^V|%-br_p?r_pP*yWs4j>dJ2 z(vSytA7caLjt3u)bJ9#BFO}EhjAJlu2}4{Hll+7{6sPLoy{7cHx*+7mV9?(%wts+`K$sGAuHa| zbW%hAk~QNn@^+KNYz*2j$R|hT?V&5j$%5KVUZ8(S$QVlxanFn0LhK@H1)sjXdg#kL zamqKEXVTZxu>)DYY`di&H|&J|z0n#Uj8^n3|8`@a?6zKj_0D^2=33SG5@x&8l=+L3{6_KTNeu zO-b?-%14~RAn`OOs!mryh5hY=Yz6EZlheZ{*|8 zuq7p4a)xBOo8AV0y8j+>`p*tC7V`4g`SRp{W=8jk$a=jCQ#KR&vbWzA>$@hlCEY_N4OFDdaW@T+6%V&Qj<)1x?C42LsiP;?28fF@db%3o&gD4%#F2*|3GBtm3w(J zbeWzZZPoqs|Ggsn9ecJcT(ndjx$?9!LYN?*>W?@=R~O#is%`dqTRo6ZLVhjB;*z9) zAN{o~Ul&$C$`5f?pPXA%w9!HMJI$1`H-9CKb=8_?^|CHt%=-J%DQp@Rabu?vjp3J8 z6j*5tgdy%?26GNIz=@-?UfyrkpIDj(X>Bi`bSCWMi$ zD4(rpjz^ak7fMfz-&}ps8JG!Kq~wigu^xZ!RXWo<%EJ`OZ4h{({?}c6I{qLwbu{?B z>%}SZ>9l3heIoL+orlQF*Y6?+{HT9i5|>*Y^vk!g$x6Bm@5|YjE;6HTA`C>xBCq@+z@rUvlVvCF~`V~CD`KB-lZ<4LrNT11ALZr()^g`M;GTPgdrb2 zklz&Tg@a=HNck7fO7)a~Vog1}y;tx7Xi0=2Zj9-?7zjh$#kT*ocZd)db8`t+9>%3< z*Lq;j?q0BM8;lqx_uO)YY#cN~J{tR_?0V=)GGWRz?MaNvw6c?wZ||%*JSyShShANr zg6+kPk~$&}%M9@(tTZY?f3?rPE)*vZ-V^EtfcoJYcmADn;uufmGrc-0W=+P&IcN&P zR$&^)ps`j=g=guel;Iso6joZKv?=dvU1>Z{zI-a3<-gJ=m>?M0GZ6AQc-)uHyNILS z1V8)(mNSh#1NZAb{`lkL->a@8eoB&r8sPJ^XBsB) zPXQ-{qp?G9La#XIb{r$5(_egOzG!CFGBI zb&#%KLdZj?cmU(_k}ocH3$Yuk5Sy29cd#9{*#c+%{xlNnjdwnjTkiRtv|#y+=C^WJ z0NJhd=sh>;1_orrO-wjxtg&gu6NE;uS1M0lvV(3J+P7P57I)BLpmOm8>wp6e(76PW zpedwF)qr)6a?%)`^iDhNB&VN#dL-k&{`D`p@x~iKha!nk~i#S5#W03g}t@#N<9BJHly?FG&lHOb+ z%ip~ZPTFno;!9X-pq+zmYo;K0h&*#tzhJ1G|H4aztO&+gg{&%MtuT+)j_mXxb3((VkksToWZPj1?e(lb3?tUY5}Zk^Wv4LTHs@-HNP+RF(}+urq=b7@^R1O6tsWfXjSxpb=H9>9qPg8StZ0R7;fq6glLywU*g7PGWq((a`<-YJ({e~0diAG0e~HX$y( z^^mxh?_r2DO)sM*#HBPIc7`5ozK{OVw}klvmBI4_o`w_6iZ?j~Q{2m>{Da?Mf+>c6 z*K&v)w6!W*u}KLw;)d)<3M+{?gF$x}|4RiK2*1mv0@zQ(Z}hvQpHzczJp4_;nQ8oTO()1(h(cQdfXddfvN=&3QSwoE3moFw;09kiF6 zbksr0gFi2RJXfB1cXo8YA|w|(lAX2B2#infEC+5qASo%A?p$!e1xeFVk?Xz&*y^(J zXgX+KT+^;=Yat%rs}oK*L1xdMz0SJV)_pDga3$ae7$;@fB(i?!?q+LxAg;E4==Lty zU%p*dFJ&J;ic`_>fsb~U*v+~oEFP8F(O0e|2naDDD(P$cU-rB{HhC!9_ zFvK-6Nix}5z9jUkL;twCvAT-%f%V5wZ}DjKnpVx1CiAQ)B+1{3ueCbt1jRFyH`?() z7$?X40D#)|6tQ^O(EtK<4JzBQHQEYL+ zbOmJnG-4?NU(Ur#m&-?=eW_oSwme~5Y@_$;GY?^68P<<;8?a7#v5zc${!n@D@q1+G-~swK<6Q`Rvl~8)-qQs8&R#|~c(k_> zd{#yu`IdF{+6VG;f*()8${4tflx}hK%9o(6rUw(}pt%Nm(ey7Q|C^_!$95PKGw0k{ z!w@*Mr1B;5OG2+NJQmHXgWl<&n1lZFZ_u?oTDxO-K)DNTa{cYUmB*icF==tD63IQj zXN`R&rZY1q$ZN9kV_?zZW%A4mugHUs#nwJ@8P%_Uaiz|bOj>pVKYr{i#)rs0N1rW= zmXv9h9!1&m>=ipn?~>R!b5dSOB47J_f!y@Wc&Vv(YZ^(i)mB@{Yp=bge+)^o{Le&n z#*nTH2h9gyve9TST)0sF{O3Q*efQlLJsU2c7 z@}tbItn#U{e%l1eDeNwVBX*O_f}ZLy&52QK!y^ZnP!)8PM)O_hnTIm+;?9ZXmDSoH zS%96!ZomIA9I)|aR3djKd*F`+_=JFd4kO_?nR%Eb=eLwvFAfU!(NZdI% z2i@PlYb9&mfLDyaJ6Bqlj-e-zb8dQe?=9M(GW0y|f84q9IbE^h(dc4uzW?F3n&6 zT`H!%qkoV5m|2q{?vtECwKZdjl{x6dl3gv?+oC-XBnf*U;=We+yqHMHV7SlVD#X^x zku7j4%yO(Om(xEFJ`DWV2jl2t%tc18S0aOYx;>69+lt^j6BkQ$(Ap$zocsxyb@n6V z%E!{HmeykJ^1kSdpXnB$v*#{Q>hrGprHsaULT(evy}cuyl{tQ_PKfFYktSP?-avkF z^|^B8Ew0t{<*ahK?YW6^!_l!#ER=Z%KXaDU$c5Ov#Ez&W*&v*xbMK{x$%w)kQde0f z`6WHpNf&MhBB!E*8sIy});qTswH6ka%W)pTtXZ>^(l#|2HEL95rUgi~cAx_E@+wy%6PKQ7YrFv#-a^Z_;FbaD?jF+nSuE${0uIZMvJ=2nyl z;1U1;KmbWZK~x>Psj9A(pPhS+{QJ4zV+li{cK?L>b)^5#4%;6`!F?@H{oku9(_cUM zRyM)Xs+H-(kBqDSG)}7OV%^$1Z@-oN_V#O}1m#yx{ijrY^czXv`Erb7cCC|I;wt}t z{3d#@u;~RSjByP5=9_QU-QhTH$-^{`KKf{R=bd-7rOnn@+vXaM+70ddk9z*)nW4j{cqZCAz(L#qqyGoSNR`v7JJ5>Dwz&TOhKCMCgo- z9yf7gz_{ahFyoR1HZb^}F>vc+%6Z7XyGh?ZJ>?hoJR()qHJWzEgMZLQ3eOxUfQ?b1 zm||sj@IhdF$G7l}dTSnz@_aylg6EE_2L0>}r=)>fVi$Egtta^Cv z{n(@j%dPr=@5=Py(kp&pa!>AV=my$w1EAb6!5q)t}O6-8ig0eZYex_Ju*ni_gg* zaR!6#F7~VCqgD<2CmJJI1&ouZ5?jwvcK4>}t(FChP;r$P5rFB~w>1Y#6N-7V4b~9u zyZcUYV>Hja@R~d_=6S7CbBZg|pBo1}a@P&IwI~JPI5hX&=HVJ)p!^p-`nf##4;&aD z)mxjkvR-}TZA|W{bn+q-H(M#*=%mCG=og0GzQHq8zf!33}KumyFyHR z-hCJST|UR$IFeKDfXyYR=P;>$|30a!sahjWI&#ojJZg&@m87Mk=i5mVgGm$)MJC=Z zggipr6Ix{xhIl(Zj7XB-ePJb~USZ@v(6wEc?6^mip)U-}N>-ka)K?q=^N)(>t@238 zZs_hl^Mu2(ukFyN?CXB@fPD7VBzTS`L2)eo&7Ewz9xA**tENdpXa+lCk5@xdvtWoykyMdNw9E;LOvODKALX zoG(&N+A6LKLK;NAR~(z<(UdgS=bUqnY`Ecuk+fG{d8LdUJ9dp2gs6}dves*W&)&wR zZNi|PZQg(2deSgpSD9LJAXbnp#`@{UrGDXfI70k^(|D)>!H+>8ih4*q@uDeW93e0M z8YJ%WJE$l8VH~=UTOXrA(i=8sy+Qrtm)BmPjT6m|pZSZH$eu@@DKEbER=W>Kb@rek z3~|fb9y$q3a_7ul!%lG|8+5`VLbLE^_3BTKiS-3FB!BoW_^`^sFyg~HJAZY5G`Dlm z_%wrq<}xOTqfg6201C-CLWjq?DOgjh(nX-`S-bqtyo$%5t9ce^K?!l=A?Q^GOI`)| z-1~DnpayU9w~MHj=Rl(Vyyt%1+}D=ly*==CdHp44=-4xJHeikRiRWD_%dnh+GaaMu z5?_SV86_m>zs=?w%T1S^NfsJDpRru-d||S@i9<4uz54^1x!8^UFmaC`ZzKPC{t@}* zRam#~!<+AYg5_rqx;lmY6t1ZM|dIX6x7*yQ9CVX9Z(S8{WaZ4w;;>w4;hL8*6(D|5uQ&Lobp1$t7 z3gzN6kC!u!KRi08fJ?NFJNs(6_jhCT7)$czMQIEYCvSr$he4w=sCo4UJOK4V?g%b_2e=6TJ4v_M%UXaSkuSi=%70RX)bQi4_ar>8+<}gBD3mUJXg ze(qdpHcE<%^L0#`Ct)YjI+87HxGUIvV-q5z+6 zP)l=hCcl#)umqF9pAbA^3sF#mR)SqSWg2z;duNEoV_&{m<_C2Yb z@!@K9(vgEkNiN_751dpDeC>1*P*wd3a&Lg*UXpgeoF1X!?TA(q5l#*p&^<4K8bYWX z?vXf=Ts^oLmu$Go4i0wKYB48`p%wj;?YEQ@vEjUh*WP+x?)=Rkw40g}DBR<9dap?T z+paxdMsK*jv56o*1|G+Z$;-|@A(~~$#E0*^UN#;zJfb`O(wk-S*E4YMB<9~p-k_}| zyJUb=VcevlzSez|MLAMPSE>OTXpT!-cjqp<>>{U}a!MrPE3dpFx88bdYCJlsk^eq~ zA8Pg%dR_ZBlp~Vsp8DdFy^l>%Rwhzphxp^p9#l@e4XPr zNDE~+p~E4g(oq*Dufa%8FC%4AT|5kRHG3V#ncfDI^P?Q(%bys|I9^U>2QzbvaZKg@ z(m3vqQnO^%Y8!rvbI@!-r?+A35?$OB*oo$#;vP*VU?{AmlU#8^TuS31amE-VPTC0i zvKYhQ=-fkf+^oV}dS`wUCx;!jL;R8~>g;4B<^(WE{!Tux6OxtI!%S>EpPiMhTagxE zM&=1e9wb|C8XGsf9ZM13{pd3{mJCLwx4}qnq-dRiFaH1@!zLyi^Rcl0&_VK#C-0NJ zciGO%0|`OPt1v%L{_@D3Qi8Ky8ER_l7ak~qaQ`W(m}ud_S>oF7}dysC&C5^$EW$hX6H|Hrauqk=+>*h zEbq08Ovu?)md=|d%Rl|2G%lOz?1Q%r&PJH6AcPIlGzYEWYEWy@%mN`UwbmXI=R{fd z9&-ET61=egVIn_p?XHTI>}b~H_kybcN!dY zI$F@~dxee!%J_e5_Vb^00VsX&L>X^!tI|clt^(~YQ2zM(BcFh;GEMf=8JZwpI6161 zFH`p1yl;@o7m)HholS3prnkXRS2A(SCDPpa(cOwL9$hTgUUa&CT{4{~!klu^jWU1X zVmvs$q+Vx~w<7)f_vQA-+7<-QBaBtT(W~{0fuDcG)5JAvt~a zT-p8bGvp1N{HH^GN$ekGV;LqdOLB3pRA;@MOiuoITzyC&Um`!|VO%V);Tq}VEiS%H zch_G3Se-cDxdXS9?82UC{QOSmimWyVO-8)ZR?x-G$Ltj3uX9}nRRPpj1X*{y^DxBS zSB!gUh&yhCj=%kqLo7J?93O`}Sz;6`!@|j72W~S^aF-J+O&XRmjFkU- z>`uA;x(oCVA$nHesm~0MvKqb*PNLytH4FcE{av~3zQ>%5US6KkQiP+_nrHn-DyF`r z<0eL*f?cZyXq0VzG{>Vk26@v>H))HClJbxZo_xj0V|)iv;lIBItlOI(Zk*}Q@#$z; z)_@<$#NvbG+saIwJv~Nha9UPtE0)iqTGUPfhInMZ%r*da<{xXBbngAcJRI0PPL#D1 zv93D*WH}BSFj*+CtdgIea;ZG_)Qj4ZhA?sEIN4(Ml=&EEfn_mtMmEwU@(Xni@$aj@ z$!D-||u!OGeEq`5kVlyLKtn`BFX3y;&|qAI63Tygzjx6H{OJ}R#ks?Lt%d>tdHix{#>jZ zERfBy@yMAc9uZ0N{-@*QhC3d@3yQi_CyiGnDvyP-T-I|8xAZs}d1Lw~=_0xs=eh2O zwb>Sa@$28qYyWi%)q!w@g@gLaUGdVIc(=ka5d&*40xaMCwp z`R9e#-62=qdO!AgXwv+~Ka@fJ1LzGhFX;@1xJGs&o!nelz4|AXuam|$+*g{X{axyo zO#fcKEUr}^#$m-}K;nyVo>+U;6MVK^(_0dNhxQG0ByDhC4 zdGPV4KJo#it+gO(?R?enji{$Y8U z=;kgMx}(&5{E*bwRDS2b$@UJKE$H+tD7Nmp2<#yyLi?H`w10^-$?g*;haIuw;ACVG zPvgx;Pn^)eXczBBpJ}JcU10KSNuXZnU6Lw ziBdX7C<}whmk2`{h;y>ov%kLsUG8qSdIgS;JM#1^a3(DdL<#=o^ju%^*V|qy-?>Zb zE8Vtr9ZQi)SywecCvAq9UEMu;^w3r_w}rI8rB~CYP5aIU!T6-$532^uQFDCSwuEQN1fJZ3mKQUugtAzmF4L6uE+k~FcAnbOOR+koV5!bC+Rg45w(d1EpM7SX|9#d zMdPGU`tIfm)({rRPOhO}nJ&p+S#P%qQ z=4F$F*pj9Z<|ZcW{$}9$!ra89wzk#{lXNj1QsS)H8nB)g-I^08Wbq@88=Ua!-m9N1 zEZ#xB%GgoLW`8N=?2%U@_(^F^1@B8VmBB2=8Rl)juNZZnhQa$UF?=*zi-a)gA02S=a`?~8Q zaJ3#iN7Sv8#h}R-qv#l0uUA2o*wEA}(-&4p_aQ=_tE-EIUCjJr;ZiBLiAj-cw$Vtr z;Iw0+B6)h%B{$xM@o04M@Tqzoo$l1s#W^gw{;_&+w3avIb@=|f%Oz(WAC>#;-~J`{ z{r=CYE9tD8r(oz#l9^j5$gb|qH{VueXzTAL z^;jQmORwUDSjkPCgLYpJ7?Xg5roXVSB|`oSzp98QUiS4t=sc#H>x+1-e&g%Q)wzp{uUnCyUw*dix?^nF!?m~l zM&9}0GyhzvJVi>A!jXGPT2qabfAom%Ql&g2X;Vm7r~w)jTOZAdL0o!u=%I(o!3Q54 ziTA`4Pso@tW4_Z@d&DG#{Ge;V-V?3>prhslGCFFGPxtFTKpOgNC*!jBkR=P|;1G^K zOH<{1XJfq9_P6hBi5iH6bl2pyWuwgIPV93GtbDF)DTe*)UW!R<-O-u`Jon_I~j}dcWJ7fpP82hi{@S)Ujtb)=ZpsRy_gI43f@pOE+4}Z@dT{9xF3> z`HAT()}K5WvU?1Z?9#zf`Pm;=*`~%hXfjJfKLcCP5XB1}I0VLO1(A}$Jv0Z2Yn6Bi z_oiPj4munZEBK-z$Bu&KGoCEVv0|Ak4@c}cC?euXR<#Y>)h027I;?E;fQ}?U42KXMBleS4?7hZUwj2JN@ zDu}bZ$B*ZM3f6ZRC8d!6p&DQ#fi}oCIN>aBP9USRF6}!MlgM_LubYPAILnu$X2v`4 zUpS9}<^cBHV7#rj%2OiT5XS5=^H@GY+MZ0z@aD=EPMYnB)^rbSzI4bwyU5L#ovl5L z#_`ts9>-+X8!<#vhbZS?V*Fxdusm33Etk;Y%;EBopJ9X{&a`B~$?Rkg<#)N0ap;`> z%%4B($I`ZBvQ*Fhbd~-d+c{_s1aP1~xqzmsgCUK>>D;M04FWE8<{pN)eF1q|<_USl zSBR@h_hEqQ*MSPh-{Ks_ihuXoyq|D$5{BA_mZZl`n(Q6R^WF5Pn%rqDKQ~*tVbWNi zUftw|OHS7{&6KCEzCn(Mv#h{6!IK`cKq{U$1|86o*w!ur{I+_`7>(4U@_cRu<;-uz;YJn@G=NCnQJO>Jbt+Uk0NPTD4o*;?42|NQ4V zNr=+c)z!&SM;(QkvTj$SuBSmtqP0~6d{^kG*_!5TZ}z%aw>P2m+giTL+ea2JTBt`_ zHsSydb%JWQX~E1VIOuFkyaRV_H`*&_`dKD%`+k;}3_m10aE_dH_ zi45rLHr`=c#}U+A7t<3%o)QIcTc{I%u4I1O`;uSFmDK90K7VAurMxbhx8; zl?9y1hv@TCiJ zqcK?s9p0wrw?6QTD;!(=j{R~HTJU$Pex%9A%ySDfstTlW%8S?l{sZkQUGcNATq(FV zYar3+Zm_`ya`n|$MK@=faq zZB5tC;=T#7F=6-jA9I+j$#|PUOV2=PHa$kq-52@3=H}pViGp0+ZLxRH5*fY0aJdIF zyu)(`J{>nvesbvP^3^1sdGGj!FqFm0ON1d!h#Q@GCPKYnW+SMkH_-vd^a^EBmGx)E z`e)BRg=E06kdq|uTvCd7-5J<1PRE>^jBkw@t7hWs)03uyX~~C z9F0?8Ebt7~`yPD;FQoNEthoM6f5H$qe(mMK=LPEZ_#+O$@ov*!nX7})2!^=T zP#8x%@zQ1hhh{Q$B?5O`?18e`<;|hD+pa!O_S$(n<0f-4{IE;vO`?N&N_Pu*-6VR9=aR-jqyT#D5uJZhJ+F;v&o%8_SWDGxsO7reV*pgfpHqeu!5i8IY$h%=5C z2@HmKN9kG?eh3v0AlbDGAyJ5XLPTZcb(7q_n@Y{Nr#dxZEV+ZG7GS16)f^f3f`%{+0=VG0zZiF$}Sf-V0 zFw%P!^=HkbV=L9XoLskw3C4x5yYRGVmz^dY$#%l|H^|~8IIkJofiiTq{J1()U7Q{o zPkoe`gPCZ_KlRWrqod)Pnwl{leYKQg4{;0S*yX2ed4*(mcN>}1zJHI@ESslBzB6Bc z#wP{Wd<~efwcVsR5$GoD*Uc6+LsL_e9COSux^H*t*{%8K@k1(sA9pt4(6$TJZf`j1 zmVrN!$$7g<#jH=IYRaqV@oscZj!uoX&!E~t+Im_UM&cqc=%Xx2qv@ov3@cAJHE}zS z@z~1a4&P6%z3>#>jgcJKdcN|P_sjX%B9!~hM-3JhPrfXY7lXkNkJ3S`=_U+`h+8n+ zr|T_#{B1k<9re#&Z)Yj3y7Iygq471SbavwjR8HW8xRjrV zrnkY6AB`!O9^~U#N-n(`y50b}=7Ljne1^r$U${h0x#VUT)n+(nEksLiS6_^^I&l>7 zzX{^1fQYNF$D}X_)~|OjdFr>fqN~dt<}n|WGA_98PVIs>M&Hbut7vY&(URL|W2t%X zZmF$UnrcZ~1$80l-m9!=640<=!{mk=ZiwEidGqG!p&E47DJR{99$Xg?h3^g>HCxl1 z?agj)c6ZZJmk!!QzAiaX7FRY%`R7kdbJb#R>>#)rIIn>Ah5{$##WaJ&6EANx-FoRP zEXB&#WmtXC<-I$$2fgF^3-#Wr=kcdsl0$IV#{zYz#6HVvs@Y$RZI1}}4&tQGpOFF1IEy!^iXq-0voIKR zcL~3iNN9LKVo{c?N(bqkKocj6#bqI1%5VG)Cp5hblF0Cm(#eN(#?Rjpn3Lu|v_U`uj<*&C5*7Nm~DmZD|Uv@<=m+?%&wO0e? zq`B^ut>>L~+DXnh*>B=zi~YDpo!2KfG1YnsC)wsyLA|3R{-=Uy_S zZMal?^}IAJp6qg{4fL54>`i1#Bl`%tklt{>bgXj8$;QSinC0CaOR(7GP20rnK}QT5 z>%Lv>D&7@crex-PF(|H47@FW=^>@EoB)!a1`V~$u3z>3WshaS$&2} zUaz5&lix=&TT7&AQ4{FuWa-RB(!vc>0{P4a#z9JDG{jdy|J;7qO|5p3)Xa6mnarLn zL=HOskJWnsHTJ+eascflVi6`K(8MnarfFPf`G+F=yv%PGlfw!M$yb1flf~l6z}s}D ztcI^4{khv%ZcdgI!AW<=(t~3U+c$1p`0n35DgS&MN5yd_rRf|(3eS5Bh8s;neb1h^ zK;C-)Q>CUOKK0_8W$3mC%9LrdWWU{ZFfOumS$Tv{XA3&ezhKzT=$6Wr%J=V=`np>G zB^!~akSZ`A&o?RDsR$WyD`_Qu`Ssa_@1}8FL zdzr#6@97^%&Gff@i_>diwwRC?O_M=0QwEvEa6(+pNoV6UEH{a^H=H!*#N36J^iEig zWnmi5xZf3LOTYc$XYubac<{rsZdsMo%v>%li<=}Xuean69wm80Mo8YEA(GRlpJd^~ zHLI`)j=BWf8~4%py!D1i!SIoiQP@WcvPz|5@?xo)y-aEPy;+)=6;2CxDTIwzxDJV|JT{g++-TVHz(AXV3&SNK(TIL!4>JLb5VL zI^$!IX~s9CTM3>xSu7$C%MWoQ2;+#eY=b6;LFO@>FvOYW1C1#c@R!0tbMgL_=beb{ z@Oo$}KblUz^cI=rH#9 zFlY7Wc81Kw>G8X;F`PhF$VLl}tnAqxUG1=k*Ijgq9Ch%XQMHwoSIR*rULt>d_V3Yf z+tS%s!#!obv@UIyoZ{Y+({GUEly-;X&yd!}I%%#%OM3Y-w33%dFH1*sqUVf6-}r8Wx%HMLg-Eo3NP)#z*+z zjVBWgH{M|!@lZzKG}NMulGUT9q{G3tq2=3B zT_w%sXicM6kdC=&@#6T4adgsUaMHXgDy5~SMit1!_N0X+y(O!vRT}1&V=wJGI8rY> z6eC6jEry83SwyIFh)3!8`02ZeG|l*Thc+~cgGQMgCpSS1QV?IGg!V^mkfLf$8ze3B z8U(Hro;*~Xe2j*4s-a0w7z>(A8Az|y0H}D3=G&#UGH%B5PF;OVb0rU(m&65=h0w|| zxCZs7f8~;c!UFfzxW(vEa`uUbN1}}#H%Tu4`Th7(Y{ty)U=oAL7Rie&8t1dRN^{SE zbk*VL%LAyG4&)W0Jz@(0i4is6;Q*oZt zS;DE7bYktT+k5M#q+w+%daJFr(nB;%s!u-oL@vJg;;02;@hO<9fy5fH4?bHZzzJmi z`VW+)z4w&4E&ZhOtG`Nf%`)}LvAnb!49Py*^ErO`a=tm}thiJd}@W z7<9O*!$==iu0O}3dD>M&)l%JO!g3mPdtcfeA&ofQHVP-of(aQ6H({8dctijq&7|b* zeq8k<51BJ){-6LCoAlS`u$+#@B7KfWU!N|#zH=Zyr;xHxRRZzWr2cH(W^)r0v}*Zf zc-a2C$<*nyfimgq8S;l`{w7bo@S3*TJ*m{D623Ax9*5yse z_l4J@jGxG?#rd+{`2UmQjSrNp;=YbWf&s;+z1ko`hD7@m`3~k&0<0TGYnmrwr4{9& z<%}{hnQYrFH<4f8e6ieg_wVI`z4ns*ciK@(OVMhE4Q*>ck5C=*V{K|18Z>R_6>P7OC~nb<%7p9N1mD59)>T zGPJcHi3^3C4UP=5DajI0R(t(vL$iAfm)dD>O3s!?Y7bXvKa;eesT>LbP&@0Pv$?Ua z231cZK}*Ps+8QJq-h1ejVC&0?$^=C{v}l7qk;E>5y~ZW*&&RC4)fF@nE3+231j=~wL{aK)f^RB%^CZk2GGYpgP z*P=stJ+5*AZwDE{c>HWMa;V&i$rv*}dP)9x-_5f1<{N35{FyO(UZi)T{u#L?aMA}! z%hVU8dgh0z-NvF?>pBPxmyIKq;%r{FnuiV@8VSfvO(sp6q%G+vGKHjSpo1EqbG8$* zxKe;!-mQJNkr`QA=$JH5#Ztqf#=;vR;INQt9|;x5uR)86L2yvp#4Oy@1WS`}bZ;pd z3OzAbXV`iJ<@Z-zDEn@^we}8WkeHKJYtX{2uEcmVx~g%uL2E6#t8sKL43ofcI!8@s z&E-_wV1hW0`{hKiW-LF%xdg5Cb>LAWEy&wahpP!k?;@Qt6t<>I!@_c@T7ofal*u05 zm_80y-Dy}msedlUqGjPHYCCy|Q5d36W_o_th3WaSR4UCv2sOk*-YJ<+rRV3)VfF=J z6V4PW4IhZ*^5u{RnJ}Onhm*$2(iC^Ebf{E3Fb48H|CA$iOSrOf9yF0W)MEUWlc5;c zO6}3DG$Q0qH4TkTvhQx#Vop`13|^qMtv{sVsV@cCue$}^I&4YbaPg^f-nDmXOST$g zASYaKgS`3NL)yi|S>UX&j_Ow*4mFL`ak|-aj;h#KHTBNMQimfC+C#6I*h%b(Xa6CU zJj6ra0edD6@|UbLu)k~}#-yvhcmjR(`Ium_g??E_^$40&Sa&tR=Rl*%rB^fnTnBy2 zEw||MtJ>OHsjjZpO-$Z*s06?EvSkg9n=Sdtz3G@2%4N_m}rjG1@%^) zy$l`cpIO{r(y&gucF|PHT`#u#A4Q2Aw8^EAoX8Xv5`#p2oYy2VndvSrP~Fp#W2~}V zdICFjjJi;qOq7w!Wm(m+AgehTl5DZ@NO}F4`yC1Qc60gH zGr#fl$Ur8MjIKf>IaLNv!yE|$Cnm2B{V6hc`C|8TDK;wEV8l?ln!iw%0?-~a?cFhx3PuH7O`F}0Ax(gD+^Xq zGFr$?jt7ww_8s6AV!x6_%DIES4?e{jgb znbEa#Ka=JG8^`_qMW4zv+RkuOFtDP`_%W!6N`M4HT$<*g!!?VmVG5~q1}VMAmA50a z2jX-J9R!Jj0A5Ptp`-V4n&M(SSIww00W;{hfsPTpn8l!vW}GDxAs$6gAEgHbu4x`R z+$BfikRJc7zB;QvXEC}-V`zOrn2w53D7`X7F_%-iNh_vq8%8z$_KIL=Di0auzoY zDwoahPz{brTX_2Er{&>?AJ$fO>bY61q~MBbfbSGr&>TWxYnmT_O+7c0x#{bpyZi50 zLyYcWjt8@66^smqxTQOL;Wp5*_s<|@;MoW?Cp8O}N@i`NI%jR=q6f4UtzEsmlZF#R zVJo|Qi@PH_?Fxs(Z*Ij>GG7Mb(jiAa#=;vhH)q-W#rPm-QN3I}#q1sRVle$eokN`I zS>4u?^s-6qh6p1E?U>_Na}qcQWKiv_8eA<=4Yj&Q#1SeTgsHCn93*bC6Ixz_A#W8L5lkcQs2v}r z2QSmx;Hv0P9vOJ?b6`w4Tajl-pN>N{HXJd;)R1d_^;;YjIT6MbFEn3D$Pvm&TxIc) zxQlUcP3fQ8djrWEu%%SJgKqh<`H+cn@gbzbx~>7f`({iz7R~X)v(7q8Hr_Zki=VCP zPd{~s%BP;&x_)9mjPm(T@xzZ35jh^stESiXvJAU(RZn{ZjuqC2jTIO{gCTC|iIDFf zCgtHC)P)7A6}bq;Z+XO#@V9?WH!&5c>)7>2UHy><>74SK4L zif{2Cyw%jlcX`e|P#6 z%O;&w5*>t9oRiDSBQKLd`35?h-Uc15E8pqt_^v7a`Ova4dr`Ln^_!SnbHT~c@`6Tu zHJ)_Q&t>WI<^|++}@+4@|`=Ru5uavR$}!Z z|5Q_VYr6*27{HiXS2wqEkg9>s)Bs!3I>8lAn(q{^ zMcsCm+NyG?n>SA1CB9D1UJ^R426<=io1=-@TG}37Y>ZScEtl*jY$yWz-o`OvUYxCs zG2=G$D6+A#+AlVO^kNxUTaAld?YZ!kt(CR1WWf?|;}EHV(0y1KjOk7L+g|@PjIC!D z^~2^VQ+*w|3To$|Sre+D)*P9U&@PtlG7@8@84Ph38!v==5Kg#`_*uSWkaD;dp@SiA z50quZD#CHGIOFNQW}NKG6FS85JALh*auV-I-caWdx3V-VbE{m8?Y#b!lYw2`dAT@T z9c!d{X6hB^9*>v9rOlbQP|m*Uc6>Ctm&SCBmFLPC3Q zuG-h5Ns@x=ss`Nqk1uvOX|8Eyt7q`w!Mf9!jU&#UJzGva^;9)PsgJ95^`w4S<=T5? zv%A?tQ&iGjX5{TCwR1kfZ0K@tf1KlWw#ebMi^(t60n?nVYCoGGf6!3==pOZ{@H8Nbi(4-1sL#lcALl#RoceS0hH#PutY|e7huLB4%OIxt4-y29LHwiwYNT$J02Q? z7sTlp)`96y$cqXaROMGef6|9_;&4}2CX3@8d;T1c&dko2@^|i*#`@am{V@6zTsJkq zXUA?yws>erxrxb+JMJhaVf#-Due|b#+3aJq&4X9czLbtkf&Rr<;(Uwzyu@-ae@9#Rf}Z#=TFJJ`3q!1 zd8tfXmLv1$Oqbt)Ytb5AkWJ~zM3v~{0eFm{-N#fsIvK8)f-> z_u~(&{Z^uuHwD*j4bV_b!kqPfD7#^8oAp8&8#P02ivP#KlY{u9#?%@#WB-PQvnb8K* zX|L)9CM(XPykj)#C%mI)i#n~TMHRuIa$FCG8bT&?ip!FwK*v=W@rgGF=Ag z{953kBhQWshH2I;7Dt%uA``|F8bgC*keEM_-trpnFpmn5@RP`IW%QO^eUHCk3%VnD zjou*3Yi+KVs>!d&wAPU_BY#I}>^@rRdu$=!!=3Yd*zm1(Mj{V=9+7yckkXR5x?T;w0VwnoZ(qV z@}`RYVblQs?)mY@Ar!6_C`4EHf~-+eHy>ls{O22ZY|jxV4DrNt5b66@zId5%nG?I9 zTiZBm8;+9~?sf98LgI;+(S-3Xj_UjrD|CkUx3l>1 z;r*uR16@@wr?tzo0{ywG82ZcggIMJPzoac_S3xL@|5+%@C15wIhs4#{c^KlZvYgXa zoRzA=J7ex12Sg`dszB)Z8q~6w?1MR0gX$Deb_6LKc)5J67M8`#28p{s9?AZZSID2T z)Xo_z%cX}jz$x>{wSfZ%>alA)$h^4EQ2C}{SNXbWh*VB^N$RoCT=P?|P`2-=zwsi> zfTQDhbP-;9?lRW5Pfxk_f|IcIo!d`*$+G2g^2N7cGaVkPK|M`x2N~ydbC7uAzhwF| zuA}Fp{Gg}tq0SZ;>W|}Oq;P|MrL}CDEdS(>DJSjU!>$16r0pOCE|KATeG}G3)4(z; z#$LrdRHNPwYv~F?r)22zHDFGfOZxcXSJ!PboFp|{+IaF69JpT8g18046VhpeIL}5f z2`nvgsBqfYlmxCCjbQI&&B3;V$n*#mlriNs9HiVgOT2*GicN4pwS>?SGj!bbx zg%(s+!=g#Dq-bm5=J4jAX~a0Oh9?9OZ|?mgnUJ-URLy)}>gIkP$OArJMklk$?&Mp! z{!Sm&84T!*d8j$pM)yD~daF%0l(UXM)Wnq!KN~OC-}MM~wBw87UM8&*kE?FP+v#m} zK~^#S^;v5SF(->bt#cn2@zaOlW%hcT{u!A$I9%fZX`cV7lz-_C*I;R>ux@LBhSbKR z+1laKD|T~p(-I5szyH2mb=6gBIP`hZ#10B==-zyuB=0wE&lAh9-+g}>tq%9=c z7n%>o5jxyK&LL_>>$L&fc+!Ek0B>z%Yt}hwcuMsfw5MzO(v~e`XhoO0JG`hxG9e!h zw1jMw2lfw_${#@cyD`W*NTxsITbmof0;#y#J7}=x){!iL!C;6pP6hCl$%3>nGQ~d% z$P2#_B(8-TBn=*-3QVtwOUv?5apw0SS$>rPdVhxCBAkHL_{|%bZ1-BKCCy6h0)g)Yoiw{7`ChZdbJS5s$*#NZ8cFxS z0}sdxFT9{%_WCy!rKFHl4J5AtI%pe@rp>I*>Lbn7i`8a08{%v}fB8>sm526@5%SVC z&{dejLMM!gSuJf`mrN%OXUPd)Y{}ZCEo$7a3?|a^o_WG}!-KR|lY|AXz7v%>dfGDC*(ha)SrzfHq7|};BeQErO_b7AF9jlBk0%|M~KkoafP_oO7+mC zyWBi$gMomw{?_r*geWE!M_S9{agtil9L9@sSV)E^#D!Jnbo7~xjX_2SIKTGFvxn;L zIQ$CatXIBp418&bvwM);frLx3bYiIow2T@x z%G(N!PW_~6pu-xVjiPPjdsUI&3oT!)^Hr;cEN0xifd;ACRYC^RIh;l<29SnKJEwj>EbMk44DBqRr zbS2Ih=4mRQE7E0^X9}Fc7vDpSx+YA?S}2>reJSXbaw~(t*ZW{ zS3SrB7Xu;X;GrLRI8=jYrS|RJ1BYsyrpvKNQ;BtXCti57RN~-q{TgIGCXkPZ#8;%h zrmmsqpOM!M6UX+Ix-b5OU4c0J+|7cgXsNL7YCzvByytf2Z@+&1^iU0Mb~&TowTWve+V^Tc9OO-Jv$Gr-$vbIO81e10tQtVHBP_?6E1ciH=|3t8Lic( zR&Kl5gqhr2?!?{0;2d$U7nfXlJleucF;hLc3)V5wm2|Ecc-PN5L!^z);NH)ZG*_!|WKmbWZK~%~R%0xW4TIMJ?_{PWd7 z*+*lPm_kxDkfa9qUa^-k9o^Dt8QGD&AgS6ApT{g6VTdzX?Q9-e(44i6i(}GloFvAs zXP6|ui+ZtZ*n7mNtD3D~rAyX*%{0519@2)m&&x8kT&z_FPtNygL64`8=URNK`f&6{o7fxcnWcSV0%cL=#E(WeN6{twk41&E2`M07&ZMYNg5o-C=e&b{=c4Df9AmUVoF! zq@{es3H3smx)mrVjTPlf&(ab(|KuZN#Lz*Jc$eLBzkE5#Ilq{$#*t)JO2_qQ%#6}< z>vJcE@A3S3%v%27ZKdYDd!%ORoRrbG%3g=i-evY2ab`E2^u-rnEF(vbjHJ5Yf(zt} zFTPOY6-B0yR1GAl0on&Tx!jtWl3Ca%qDZtSOt3iO<_dVaUnVEV;_T%*bWztfU?~!u zbQ3l$aTB)Cy^UGb(FF(U#)v(POJg3p*l?i(W?qBn<__rH#wRYFv$mUY@dJV7H^4cU z*Vf3A>KYwtOh<#swqbU&WY~L(IB&t&x z5#v8X=j~%qS*k|^Yx5YOQ=Bl#<%DxQll?mJ!@N|}g3DnvAulT{j09^(^wqEd0XdGf9PYSp)pww4KH{$AVjba=rxv*F_(5%rUYYhqdr|r9!F( zlGT7Y=ccM9x>-?Hw^-vr-3Kzo?2&gLo{?DqLatunyqn_T1EdD$KXYc z3mTMs7q!ou&^h3ChiF8tXF6rr8XmF{%=otLKFKSVhL$tRw3c%-#;5tAfDMeLXi#tI z-mL_Vm79NdJ|ybTI%+E#tPMRnX@+(VT4$iClFmJ_L>6iV8`Rt@`EhMUR=WE~a*JC@ z)^^F3&!FSw$U1t!CUj`)nZHAQOgX^T`YMsU9$Hy+yu3OS%t@PJ<9@W#A4jic^v53F zucIQZkd<>YT?s?na;n~ti$_zkC7sVnV;GMfiP_s%oc%K+!wm2Fa?Uk(Nt1PTBgLdJ zsKj8^-*;L@7D#AK;1tysIvC=X z?kp}=%iula#oZK(dUcn|I&>+cTe+Dtr+s&GGh5TJZ@T*_E|*=#oyCksFUx**GtVZi207lii@{6v9wv zw(47|7m1V(iSs#;gN~dKH>*a)nL{#|Sn<4fg*zDHG1}0qdrNPYAGeyv!{qrxPL3B_ zb`}gH3~@_mWw~IfD3j-ebr4Kf!q&!WNylvC&U9}lvSWUJ0ahwNpf^70$+$$gLi1s{bSj~j;i zhj_#!0kN7S4DlFQC?RmY^$;LYe?0-KsJB$VdpCCGsYrEq$0~B2VK%>lEoTnh?6uck za{TefNAkS%(o1sp-FK&+Ra7G>#A<*Jh~u@j<%=bA(OC47ZHB*jvlO5*lrY3S1Lwfp z4V_*thVpWIb(5U##c%=@ zVZJ7|p5dSy(Q@XQ)U|NXOEBxYI-?Ey1{7h!ZnpYN>Sy-aa6LC2S}hbQ|_LWj8lj8g%Sk_o#Uo=QKs0KDM>0VkWryYBsY_`$*k+`?~`VaEn zr(da)CZ84RzmA@N{?HvID?3*z-}|-H*STZd!avkVnG~|tYJdiVGrMi4v9r%UTejFD zwmA;ZlYaNzch%6XHK9@kb@>{g!)t^iYnb?~WMi&LW*(Lz(H5`Vv^g4TNLmEiwk*7d zdLF^3AyVDaAa(T(+AGLoDCwl>MCo|xba}0GMz%U-2;K@HvO zb-0#RRqMyXa2#4xfHM-f%OhL)8oYj{D`AKy)4!=~j@0B1*4g0vd$VEH_70khx*!~x zr~~mZ#G?WzjjuqW_I2RC7vM6Grg5Y(I)lt$c$Dt){6Blw0hncVwNDa4_97&Z03x7* zC=T3K#XYOG+SV=ZZPmJPuiBy3T3g%xcKuzpwXJ{uYPG+uo4RLJ6qFq{A$te-pXc0r z-tT^2zK{SR0sGzL{myysIQO1&?s)HJs4gb->m8M8lIcbOvA8X8ER7^#?Y8C_p9bP< zI^3;GnYr;_Gd|j012)pO1&gPU4H-5{78Yas6;^h4HdXh)e;^B=HjOyOic`As5z7FK zGYhNE3$UVmXi2f$dga+@tP3?LuSq-S>bqsxN;hxV9sjBq#1rZ~#LYc1J$<2n(#YH9 zARM2EyI)`a0TyDkrYe3F^uYeG!mvh~i^lG}^G;nn##C|0@AT78m!(UWrgSy1YT9gK zXXDI#(YW9($*7ryg`j()^7ZvPVTgB^>cCGr8m+ZjM_t{4 z{m|^$vZ3aqM{sha&>jw7D{c)i-a|XX3GMM=*)VtD{BYNEZpVQn9a4v(?WXJuDcX9t z3?DTdJrk4YoO`ideB(5uEruG3~}#ljfHPO z!mos|@$|kEXe6WO32KWO3#Ayn)0vF<_|)D9QJh{7cE%@j_3J8G8dQKh>dYT^HfP>q z9fq;+gljw7WNk*a_JHk6CUoyD&batB#GW+wm2!x%V8jlxICnQ`ocRxAhIWuLIWXdo zf8ffihsL)-!)~~E zODhU?00TU#(0xT>?|~nV_Hm9kHyfP~?pPT&3X6C%r5cw5wXgy8!_RD>IrPmT@2G*c z#bQ{U9bHQq*`ZN3oX{RL(AqeI4!2Tq&bbvI0<|qIvaq&JI#9osj~K2;F6RKxz+5z| zW>-xn@R5X{kpIpm+(L-MRtA-Bs~^GPv}#-fO{Ma14(&nfi>ok^YB&Tq-I5q>{9sQQ z;*r;oMuxt z-b}1`W3VOlECKo^Mx8Htp;e{7QNqO$M z=j68AZqpCmu*^jsQpjgs0@OR_om-n5rSLG4^LOUF? zg*q*Z^=Lt!1GT&vp3B-Vh(llAQSZ9kq)infX9Zc*Ma+aX@+Yj z51`YEJM3H0=<_Jg8eG;fuew$Sk0_Vz_Lzh+P$GHQR-1)%MO;|D#-vBK#!n)2e%f*8 zLsQ;Zj1IU1vDjXiiDBM`tGGGu>}aE5(3o9=W(`|+&{g&vZjDv^@eKg!&=@_#)zaIW zo%uC4Kue49Nhh^4KWpXrSKrZ>+4C34Hy`>HaHhxH;oZ}C;+g64=O>?2hd2NIv8=4A zjp(z1UYqVF{bM0F4r5Q6jWieAum@6Ieuy+K&uF9{AZGmmEF8A`g5QAT$G@D@4Z)^eDX>C;7v8sky!pC5bB#B(XFj5 z((uX;G@s;*+8w+nsms)kVxgNE!_7Y{AL4{s5n--(kS@sLyzlIqI{C{xA4n_Q>d>=o zMK4r`qA|42ZCGq%+1RqN=0^h`uF8&vv3-L~J9S50XpM6lM(wy5tPQ>9HVkX8YHF2v z=s6co*h)rD8mC8l=Hq~b7YOMEW6%r81m%c0p*gz{wi$KrM_ZP*)p-Q7PlZ*s_fyGwcL5Kj|>DL55oO}aar#Cn2}=Z!!(bisK%R%}Dc zDx|99P~12DhIBS#KDv+i4|(P0{%r1#=Atp2*?q&MXGjIk?WSR2x^UXJW$wZyPLxc5 z%GARU*Tf}DU}zWOCZtg3A@1G7AR%tS0n-QPO2PI=%j$X0O6?4H16Hah9SCRuQ08ME zZOD_xIBmm4V!YLjhWYC3v(J_}bLJ>M0Le>5IY1I%xokm0S^w&fWl%>Qrl}4_&0*!9 zi87oJkCu54LtOoNKXEwf1GHF~$bU~g_ljI`*Z1ZBKAb6ket(8EXQs(A92ioA`QcVH ztn5X%ciwmRzW0>4QAcc-|1C9dpk-JuJIyoR+C@ zc$+@s8fZqeMu)g<4myD#Lnv`Y;u{qeFzgMg7keR0@`R0m;_zlr!zcbE(xAJ=;qWWZ zJ4r@Wy3=a7Z|(YT{4lPfAde6H>`|GAhKYuAv-jS_yKe(tIah&OQX;{t&yD{fJ?njN zp-o}sc+5vn#{TGMFz?OzXqY(ujkiR;?VSP4Pl(6dsc`K@7Z%7?m@~QcsiERn$0OkSB2dG92YBj=xMOd}y0V70E-id3y0X(68U>w})bjJff zldJFgfi}WT7Mtgdw%p=dzDMhJV7RG zvlZS+p9Yyp8)#;(zBg>7t#ad>^y=YzN&VUaX`1)4ZBUrWf?#a@u4Puzbt|c{nG%^Uga9kN-6D=9_QoB^#_%Q+hI~eD)x9 z%bs%w8|QyLC>f1an3g(PFU8QJi|T|>ar_!o8@6t=A;hEk#6!z7A#B2eu+y<_=)s@= zDH0?{2XDOMESa?XB*`0DE;*PVUY(aMwMb8OOS3d$4!H%x-5nghMkC#U^zvy(x;pT5 zq7LiiI~#B`*v;sDHz4e?`X*V0x##pzrBbr(Xel2vQc5xUScHxa&zj&|I6HTox{Qof zorHLlR}U3W;J>qJ1x8P6WNrDbNqf(vt37BIHZ$%nN}NXu6ADLhFk%o0JRn>Ii8GKv zQt-GY6?h_~fAd3}Urz>!hsLkEj;9X~p*N48aO4!(Z;zd#&`&;##?PA%X3o|X-h3@D zEs>ippX$@?J&*>JMmdCX@veOW*g$irn?r9tguER+XWdYF-peXO`tqlhcXk-@%j#1uJV3bs9X^YcIaO&)#xSvPMO$bG=|pTK`6 z&hgFL=3r^Y;**9&AExFxO@RXhSvgrg=1}*EC!Q!r9fdUm-aP#9!}7b|{Vt^^6Ub)_ zvVP#x)>J1o&wpFeTWj$VeWY;hh*mz@uAmYk3~^sEx%$P?AkOC7pf%<6$`62ri?Kb!D!}tWsnc{3JbuzsamqaP`T@x^K_YfWMSziYSLDW!^p+{~CbqUM3kC*dL7yTZHqv zxsc+rsb7*$=2q!3p!9pi?GNacl{+AV^Rg5g&3?eGu(-I99{j0$Ox*ge)-I9$-TAZh z@hx;#9F~M?0{!tl8ULBZBe4habJFzk1Cn;YG|4N+0SNfuNtuC>04o*FM>8LDTM94L zm^Evb@EA}U9%{ng^^QC4h^!cCdSDWpinUKApmWaXSk%9CpJe8hVmsBL=sBZD8P!XS z4h7s-1FrDL3cS`c$P-rI(B0X^lF3D^?|d*zPP*ha-6Co(lXu)sZn%tVYKCGvX|7JM z&~Hm?yJVpU-8QsXT5B=XjmEj72KOppkyj%+AL$tKPQ$2&aGDtE$Id(~ z3@*u+%)(qL0^R)5A}PcITi$C>fS!6T_F(hGuypv1R$4R7mMGShR)^6}`A*J%XX6U# zs9z>)wmlvPo#g4%XGGKJ0mjWiD^2tqbv_aiX+`W|SUI!8)=JwdZz_b3C615}lWve9 zJWgnZ)u5(H83IOVxH>2$U>d8!INGF(M;kH^ciMiOoc)!fedw-WkDVvT=_elP(&59C zhCWc;rtCp8HRPL>o81_cQM#Qh9W+jwWLlKAINrKhXN`j{mHo+eq$+$vC(BK54B>9gabD4U`G< zF?T3%M2K6hSn1F-H8sh}C!Z`0SZHH;aiG$gins41z{a_G(Z8ksnL8yLr-bE=-5gJ&NGpPP8EFm7A3ij8hr^^wiUme#tA9c{)atAtBY@CPSm`$Gd zP>iitg%}|$!D3i0h|R|qsC-;vQ8Kzx3bz?01tW(^#&DcuR$3_SXmnfhv!yjJOWKNZ zWpzn`WZ>|V!qF8{juqM?CXPdIybS9r3Z(>h8Yx9Xv^G@hFj&M$3h7q~ zu#x8CM?0^3!h{KO%{A9VY53@)kL03@F4Dz)k!1?mVkN+llA3q_EDe}*&KtX*z>&pvHJ5H`R=S#R+ zw-8g5*bRdQnmt_ZR?%ZX(O9$dF$|Y#RFsuUIflB+G4x%u{TM0dgW>97led?Oy?2tz zeJ4rf9uuVkPvtHXq+-HYDX%O?m?2U*tV}AfAgmOpG3uN%rX<6`5)jfVtGf>~48 z{DM%Sdp7{}Lr5fz^NI|luMRQ)jE{xgQ8cRRH5Q3Q%J3dnNN*$^xby188!tZ-E4hE< zi8yd3dd|Dy3{`#(IO8DWc3(hJ$Bh%E;{`S zdI#%(=%4wHM)~fZ|J;$gVoO|`)M5TB?U0)#3rDkB;c)~~zx^Qr%f~j<&4ppSRD%uk zqmMqSN%;Nme=mFOwU=CW*=4CIiT+T^CWwQ*=Jw_~se2haV;Yx8;f}}Rz5;iQutult z1u$QJtu3%RRVy%ca}}6YXDHsjI*Zzl7R(i2e(MAB$p8K`;(`lYufO;-*%yZ{m7r1P zt$pk{XW{rymU(M8xT1PcMh{WA z;lU_$3+Zch?D=!;1v)a;R?yygN$G$NxhsJPm|j`m)#9=de79ySx?9`? zS;-wbS{7#RB#ocGfPswIV!eoOFSn3aOIm1W{2SzAGvie+KgSm*zdSj@aNfzEmmRj< z+Q`Vw_dG0b;qVp~NXjHKhN`SKz5dBFjmtwWzeIBI`6Or5o-$}~j?}&I9jqkfb}>t2 zlBSH1Tc=(W4gT}W(Q3Gv}@`yz} z!;}>r|2-s*RH~gqrj>3U5?5yri7QQm0dq==^fa)C8rpEec%$^-hh)6C)B|xdhMPMZ z^}!f=`_VUI!)f^-e~1lS9hMjsoQzUdOOKq+v~tv{t>TaVyWp zw?WUh@l20~hx`+#9P|&25z;45NK^zH+bJRnnIufbhevkn3F+VoB?TY*Sxw8A_n{U_4#jAz7&?Hc|o|C{wH6_>I)Pc)#S1oGkfcFyG9@#H@p~vSdNk ziYW%^#B%7Nwhtb+@*LU`pYh2&nevsd%FA!XPFLI>9S2TF?l69g3`gV22DKPH=Nugt zM(r_BdN2P3>CQwAG(D(C3~}S+ZpB%d z(~gjTGzN(qVM6m~(EJ$;!_tm_OqW4TBcd<}hdARxIHl9coDC+Y@6ygKMMoaEmrNM9 zm0Wzr7c~wSw~No{xwO*`%HdpjKN-f$VH6u8WW~0)Xo|EfpCui2Jn0ME^Fewd=+o94 z{z>0GPG{uZjsh*fk$G&O$Be4P5Ynk`(>-V}T)bQ^n062LM`MMft|xFYxm38wNt4Ft z(~UTx`7=m=hCA1u^-o?As7L-;kn*wpN}878a*dy)<~k|t0W#D-KQ`@hm~rFA$xS!i zr1_gkTCiY&oORY&sl#RlNI&Zntu6$^^|;l4zgaq$yd#B^j+d;VV^QEyD0Qe2b)-RC zTVxPrSE~mPLmUN~g_=#8LgLR%e^U-Q<%;M4DLNgr&#rR!b?3=eIC!d@JzcD-&cphd z!Gk#$%&rFv;wRL{a6+aF52^XfQ6~gzjPd&Nz|9(iHP!e&MEG9dt?ecA)v~X&1@; zw_dLEsumxGO*~S7^2&FHCf)usjFZDDx%k3k1D#)5Df6;$>(!hWq5qCJT_oe3cI)Dw za#z_J1Z&Kc946w{s}kHFz4xvY<%|=Lh?su*r8nf;Kl+XD?U<|(lc$jgL!CMiQkEbG z|9lK5Gag4@2DCL&)tN1=}4aYgc|OwQoJ52d_koq>z4-0Q0eZY;u^J z(>D9=yRRPqX^D91si)-YU;lb=IF7lb;N~KMPHe@j#rgNO|F}a27gb9Bgrg*5aK090 zUp$di7GkEDg`9=jD?}8DBH6Ob1_|2dE?s56z1vg1uLlbS)oO1l3a`go# zqhT(=ur#jf#V|E{z&yQ+`HMzrc&Ipj3=+}!Jycv1=o1G#oqL%9(>O%#hso)Mbqybe zF&NU)NHijPVoCrtpvV2IhVgXTYZpsP^<1ebo1$f+H@#<4>(W4{VUxL@q{WEwSi)RF zXu)94k?A1r)1zsu6oiC+j!iovdRpL=Y0l$h9G92g_+;;3p zIdJb?A~`Y&Z6G{!G%P%N(71N2zYQ~D zvysN!3x~K*KJF0N9Si8peEUa_$a62hjXc4|lV68);g8e1!BBM;R=q~P9TEh0-pmu< zLL$^1-8>}Tlm62NW#OQN!=!V;zohp4|4sFzorL@S*FHA6_>m1XAKn+uMPg>|zWZ)@ z<{2Du3&rwDnSqx8^}q#wE$B6`eClRtpZ}5+PW(JpX-`6<=!SW<3?ivoG+8!*TYbbA z;%aFsHR$~TME5uELYy~eI2Qz8#@fwUUZ5~{nC*-Z(NSXnV0>r=j>Kd<1(ip z&&1^&#!W+rM_)ZuE5ix#Fxwf_EQoi5Auq-sSyBIhA#Uzr81(>ugzWG(&Ha~D>Ya#pM=G2hp#I@Cf^;Hz4~eUH4~c7d3&S_V(NLywSXTov;{G$% z`{2P3oX1;Iw7*{F-C0+)N%-fNBM+a?LuJaZ)FGIYzW$O^g$09#+a1om?jBjVbh(oU zWduQ)-VM^PiHner$zJ7(a4K^Ti93coG`$=2;Z5hHVK|}UBtn1ohgW{(7fd)p+CF|l z>SsNdYNRRN0Wi!Lb{G#wg!pi%d)P2{PCkcE&p6`@9r8}iM-PC4f+(%Q<08u%j6T); z<8~QbFbw7B7)i%D`1)OrZ*v;rkzqNc@#RtLOW;ONi!3SvLd6l5a1GAFuBmO5FJE+< zJoL-|Aq6#@YP^gG^-UGUz<( zuaDD*C4WZG<7qtRa{zkKOr2G1oR4OuR#;jg^RoBEQ1`!}7A$}Qk5PKzzc2XLI5FIl zk0GRD40n&f9>5!M^A*2XX;$JSwzIDKCa#fdm(@62KGG-S_cEtHufaakztOh%lMUNagQV%hq9nFdrEA�HXOB>YNQ81 zL5PxNin4F4dQYmKyk1r>`!`l-A0PFa-N!XYD%p?)5IDo?iW93pJ-rwwu8u}5bWs`{ zg_~EkSPnbwO5qfmnXy|{%Kf)oB0J-NsWPmlF2{T`7jfm_^h0iSvbqJ4S`ek_UL1Xk z$DpelT{;zKGsEyCrUX46rH|B@PS(SG40mbY5dJ&rmq;_#5-lGx#Z6se0VPLCd(${G zHE|6zEjUcbA#P<-C&0;sdzfJ2HfO5}!aJcpU@n5D5Dyt9(SK;Ccn}7@nVi5lVTA%T zT+^!l87B6)K^cT2T|64Uhs0?N>WhX2rakEXAL4dtV8)1F4VzZJlhmaTm6nCf&`u7_ zK!%4rhxq#N&&rg;OS}x7hnVcL!#KGBThz?F_U;V1`M#gxzEtK7?)~ekPwI^Fw7*cN z$^3_Qp*r80$zFL6}j%Z>r#uxB8l`5!g9*t>^2C;fXss>kk>2VU3_Rx}=ZD z`K(5Vs&UQ>xM^!I938rqPW7<&%reni6X#P2W09C(h&xPtcMNqA2pLw6$+b_shsp@8 z$xwI3+z1W#ew8fjEKbr%d=dqwaUr1jv#^B89{QubUX!+#8`={R(OtSXfjyOIr;Kr^ zduj20(z0rSbk;A5_%Xf=ZVdmNV`7{++#ijjK#Q@L^@Jn#ck}k1zyJE}U*)g=n2yR6 zE4w)~ii{W$$Fp)l;%}#OepGmT0 z8FWte*1Zpmip>_<6c4SG`Pq9&BNiLTn)ZnAzT}^Aa#&mMk47(=i^i_G z@MPWQV3N4(mhZ`|c?;dvD^;Mm#B{+l_2%gh{b%8v;A|X4*fjk+(o&1V=}}RnO#e#2 zU@-IFqXF9FK6lQ%dm{!pr)x>4F^|3mV( zJrp0Y`{D*F*85)Wgm_dB9=i6x*>#7r%+t-vzJ|xwb|>yyzU#qX$@$anm9};qBMiem z%9o#Y0=B&#uZNEe!{rvGSOdcj!eD=bm(qgY?&9NUuEqfv;^sebgbtLDHu7i>Q$pMu z@=ZhN`8H@Y*9AlV+m?SU%?&kDTXvwHPtT824sEYb@0m%78fd<-AgQ9zyrrQLu;FY% z#nqGhN?;Sm#2Q>z!n<<|h3evBB4P?=bY`FmjW2?X8-6tn8YYHYe4)S`&Y@YRn+w3) zBCr??=ppl@@lI${k|m;_A@#xeI(~VvQj@YtSk*mLnijkn@!i+_yUH{Tr;K4fx+qV# zINWl@8M=v|>^I=rwy9TrLmF_IIUBo8sekHP^+nxpg#L4QZg=TuX+%%D6<6b>O#e%O z6%%_8+#k&akX$Tw=bd-zF}F-jeSN(iVnWC~oicqV0m_rZ*{yZUWz|a$O5^nValysb zQn39ol37p|)pM?HjMcwwj6iZzS=04VZNCQ#CT8$_a)P49so^)VO$nJQsL4{P6D<8uh!myAop(nQIEAT-0d9tVX$;1{`3SrCV8Uq}155{TOAp_i9Wfe5 zJuMxl(j>Y1>Z_wPeDJ{sdR8~L-=s!5BC%{R!m>j- zwzpz+HZHeV_0;u}-q9#UlTMWEvI)8>nu~MokM%BbSO22y>4HoTZE>bSdh&R(b2@my z(c)z*<*2FG$zT5OrHF?S!-wKzuuFyW&b-5!!`j98Xv;!rVI5Ra1Z(7pKjf(wy7-v*6ugDL~1=V&=uAYQd|@h9Vd4NhHa_~bcRn1vJ9D#i;h zL1G7MUFU7PlqUa8+CZvw0-6#BmCgar`VzB>^Lr=#Xytf>G*nS!xe;$3}Ir-(Eo{m;^ zYaXE9`%wRu7qn22HpzSF@b%Mw?#R6)gQG#FXVMFB(oC66CjsUu`{3kx-CSF7+;PXr zi6@>IMf%7ikI2t|{`1s)bR>}tL)b9$I+&OG;j zIJR-JaW2ARE3ekg3!|7d{i=l4#1;#Hi4iBac+g0kNSX!{Z&iFFE4Q>?wpvpKiuyBkkyLlIC zAuiUq{H!m^c4J3HVwv`>|B`oRe4;(+HCVTxar=Bg+|ouE;&fB@9`*0?f`w0nVI0JJ z(mxxA+|heR=_iuvsl|z7lfP)b!UrcSExT0Xs;jONUQuc0YhU}Cy#4musYcoavL28h zoNSnzm(P}!Pv0Re@BUiyaO+#)_#-4EuLSiR3m*t3dst3s@5`$K{`e|aOQXIL!yn>` zpyHx)+)q6FsvL=)^P;6Iqm&;pWe>Ug+H++Tu5&EI#TDG&%zDLci_%!xw!kRekhY6O z>m_L+*h31(t<_MGhQLwZ(0V{6|kAKD84}RTWXh+<8 za?n}OQjPH#GRt^qqWIuR?Nn)AC>D_ZJWY!~5t3K_@#1k^D2Hn|HIow;Ifhf zq;=(N>10wxJe2}cXIKmo~g}YZ7fQS z(gQlgl`d&BEe3%z4);&YCDfos=B!+}B>qX8{-gNdN8@9-Fsu@Tn{PXM!+(3td}&_x zu`DP#RECxf(*~NOp6k16+eaONxCWZGOw3wdn1O9Z_O1wiwDI$hc;bV=&JP*W2c9Dc z6o2mDiwJJ|M@WAZn1>dpLCTMQ;{>ea98E`8JrVT}L%48?kv?rmEXiYWr(+0+hPRIx z&uTG%KTI1dH7cFG=%TWbvLt^m45_?~G(uNi3$9q^IO&IZd7OHK@GInpA>4`7rF z78v+3sA0T+ErcQN=`o1XMf|NTZF2c--@{o6KZ6zGf#DPx7nq)T;^BI1=TN=QF-H#{ z;eE?2vl?Dg7E7DP9i$~nAG~Y&NSwyU)2;u2hQ(vBFZo~1MPZn7oL6v=lnfuEjWb6( znZKiO5?fuH{w z^U;{Mr|hUpeGDh0OAP9p>4uR0L;VA17?+=XI8Re`N?g4-7~OUJH5O%@@ld`{H;@G|a_VrOnmbw$(`W?|p-s#>iLV zW(beGCWWLJBm{o10++&G|bU>Rw)Y5U!=1DO}p$#cT}!2y5~p zT^?kT#cKpHZ&X}ZAQzr;tn4yzn<&a}{NOk8(i`vNoL%mrr}akaA2eKEanST`Fp79x zNHYEt{O45eC~fclQks@RhWx;=FbV!ruYE58>eBKv=c73?bm^s+>S<#Z+wHgCF3&&z zy!H}Px^YsW9CSZ()xU8S=4PwE`}!-Ks=h4JtVHy2EBx7*IQ#{ zLut95`^?!3lunTJ_FGhg?GiEE%qkcDP- z8hg-gk(3SzT1=))LdDtWYAA;EmInRT4m@~$s$bP{ufmgV%$VW~s*A=AETZS}C>Ya1 zLzwI_H$wVw{t44Cim;}mb+enFq<(C{7#r!rvaMus?qoOAjiDx|566$=*>Ses{0Fj7 z3XB(-ju)R7;iNHMtZ~~{&y=Ah=*h!$qG3LN+I_NU8RrdoOzOt!AK`tWBQ(7m)JQi- z{~9v-$;cZjIoN{L{OW_!R9%(otwoaQAH?!A7t^qEfzz%;HMtk)xa4nZ)Xl{>+ZJAC*!V z;&clP)57`g=U;hO4n^ZUXTjpgb^kpl$-OsTgkw7^qQzjmCeob@W|8>{TVH__nv+3T zfn%jh7~)Z6#;c9!(!uN^PXS`IyNR#2{F5SO&^Twh)O6-b^ROwpor>!zn2%XM_m%0= zK$CE%_MrV1p12y{h7$>sJq&Grh+AgqP9gBCLNi*4;rZ@OC|srMX~yC7YjK9S`DHxB zlgEc@2J($HK7@6+>~U#;Hh)%SKCAtda}RNOyEtPc$KZ)gkPn0@8#2L!F0VRZqrK z2$@ogGsMjg^`2ci{qehJq_MEL3vSi5; zZKOGz`=6#4^2DYthrb|!P?NcRwAs9~vM#IbzpuSiX#`r_irGaMUzcw8gX80boB}fAqFUBi{rgq&3Fk(Im05V1-Ovd`) z4DaJp+o17nP-UMj)GBGPLv_==o&($PAkT{fgp@0VxeJc#?7(Wa=e!bfQ7G^laYm!&DhUAkNk7dUw~ zz6}~3gAtvO9%+Wg+#LO$__wsREP7X(YL?2PvZH0l&|x~-!_iF6JKINP7|X`saSgP6 zIjqgd2LD{e9nV0VP)!g9>6gMXs5oE4UP}T!Z6FxVKN^;P>;an8cz27NQ1Ji)4TnV> zP8IkxGLChGcSGDQNn_s}EU7E@I=H}wLxY9G#>(>Couz61t5F5EG5k9oDQB8^dGk<# zEL;PZkK;gx6c@;;#~m!6+kHpp6~EuZg0Ux`dsQo!jn}`W!O~%H-THIBT;8J`Oz$QW zx<|(6pWLnXkPPXRx>x^8S9bp=Gz5_hHUq>yI@udxBhCAw4>;g}C}mtQ_QVrUxO^4a ze)f>4VaA9-9qu$=`S{h+iGxP+#~-Gbzocj7q3-e(or)Ry+(Z=Gg?M-FTI}(| ze@rvg*Eh?_mwg>uss6`8=zivJUU!}xbs!Grz+$kW*yCJ;8}WF40!Q0iPuzPaJsPD; z=xE34e{V-WG>QJnEA0r4Z-eUJ2T-~JPB)s6=MZ6x{R@V18zAipxhx~GT_@#Ld99%Q4YSo zQyI|?mLVRwd%xX(r(SqnZTYkH$YDC@j&@4`06+jqL_t*FyeqDz z<=xGE^0IN+1v(D87z|;V5>1n)%j2|DqprX#y|iR{0n|T!3_8Rd=+V1*sQ!G|h#oCz zM>+u>xsym3UqW~~t6xQgd!3umd!Cthu;i9hYQxMCAJ)Olx6HqNZK4L+2y4YZD90UC zVu?3~5@9G5;)zdU_(cCbx?8FU$$Y}og&w@99sR=?PKZYfwTG^NC$36oAg3gr$Gx_* zRl{tqm4WICnSyh7tk6n_mdWChDbj)yZC1B6Y!v@ac1{*urW83hHTCLm>&ZG=HoRO-zDen;zI>b^o~Ue zZ&O)mlFX;ygLN6KC_8|ORjxIRZm9d(POao#I32Ya0hhHWF|v=G=4V~7YxpQXbV&>8B=aAqP)$E+I6hGID^3{V<{m;e&}n`S zT7zw-Kh#+`)SX*4PL^kFFU|A$r5j&BLY_lBtHdF(m0$b+3yy-4f}CcFDqa^-o_xf$oO?D1kss4k7*Y!MT#Z&4JST z@898z`QNFYbSSWXz}XvNp0-AM=bd+!E3UXAiutX#-Vz=U%K6#Pf*~JVoLKYP&!y(+ zTV)V3WzmkuVFB2-7^&)hyH0FH4KbIK4N@*FQ3gX?EtC9MSVs-JdC-&p(~~dCk!Rf? z%T{u2i!;X_vX9()`597KUMl5!hjWphkD%8?qOt1EB+{opvkP&HKeQvBXe?}Z{zINa zJduvQcSo-Sr!_Xtc}eEv?k~9&JLsG^S(;dP?&NuPsKK7)KIcP6AeE;2}`2HIgYWyA-YxG8rMl+;ub`ik^xo3=%0tpxv#3|;?~!QAL$4)Z^U)kaWM#-pHI6>|=qUV8fBIAT?Qeg( zMeBuSq64=d)qe1VE&y9wH(!c&I7V_tU;!8xscHio>m<2GhQIF5HNi3WME~~jNf_dZ z{*t}(qx!y|JSt~hbC2*yUmEV#xcIbV<$_a>!NDW>=sEKMl6>4Bk82`zF&JZsrDuK9 z8-`7WuP?oj=MYauyF0ti#^uuZ>GLu>dv6&$e8;G9=BOp}=%#F(Df74uTiQ6I!WSLl z<`yIO1xAm(G4yRJqJP7ka2z$WOXg0yctJTE_nVk0Hh_ndtDcrYg_V+l9$_-GF8-7GPi9A1u@{|#LRwgu zi*sK%Pza~~`-^`|cW3h|X+YyVCup{d>}mifeQKSW94Gi+*JV7|urzA3j`eyX`jJ zibcja5B-&|d_`(&YquD^us&(QJoCzb-YpHdn>?p{0!{!sEY>)Kuc&-@oV?O#S@6)x zg+VQg>Pd@=W_i+K_KxTQAs$(ngNHn}lV=$mea=mBjdMl$5V`k;3uK>Nx7TAk%Swv1 zoLl2edh9;Y7$mL@s)r#?HySH?gqBW&noigoRR2Bzc{aWcl7KpUNSt8}Djob7RD>_V zqv7#R*x6Epd!3(^d6~P&pvv9!LKm)y;GDA!YY#{RjkMtyYV^3cJ&Jpwky_({Zp0&F zFWy7QpyKdokYQAq9;%QHC&WV~8Z?E%T8+^$NXtZeOq|+vlOBc*A9~}v_pUF*X~`aR z`r6IsvJ57xC&P~`_M&a5yS@xV-HofH4XY1Bz71a&|0)AUNLc`=Tmnd^(2_S_@w6ar zz2a4J+ipElt~%#L^~Rr1<}H%1-tmwhkpNlH)pQsnIYdG`!Vp(~BmzU+%!caU!V_wI zfXW|U4AQ~y+))^^Z>yEMw|~1ueXxamZmGECX&dSuKYqMid+oImJ2Pj_l#4F9XtUP~ zKkPbM8)VgMzmVF0V4fL0u)>MQ$l%h^;Dxy`$Z*VfmiY{tp8&eXkiYH^OJ4PlkPm4Z zBpw?jayZ}Z;fKSASOj*^sn^JB?|zK1&g_U$uX}GeU$)zN1h!EX;WjGFHy7k!VJfy! zq0FaY9f~EFu@fic9}BrbE%5wmW}zia_62Rc8bQ)9JK_nC($*MUJR8G5<)GmK*R-JL zJnI>mn}MyP6?{wIKvxMxh3ceWmU4!NEhbkNGXQF z4%&NXIq}H-BS}5>_v!M(-~1Josk=K`)8loJj2`25`Jk)r>84>kBp&KK#5JD4Uz8vq zN+XH>jc-i@WSWQVSDg!A(Usj?Jd-m0Cjr(=cK$q%0zLH5L*=BCPKpxn=%bIygAYEq zSuW;dK4gBWoB6D)_+Kn+TlFbcW*?1_nH^ErqMr3-$mL($XRrC#4*6l8=0Z4^|9h-I zeI7UeAm+mQ{B3!iFnrdm?v$rrd_xX9^LkmZczGn(LqE5x+;QcZGGh2poDNomqdRe? zHp+JPV2m!Zu#+y*=V}GgWXukRxVe*8{Uh{z8#KNR8s7#p4dP>IWpQfo6QH;Pndu4Kz>EV%DIr`A|XzRa)wRAB8Sk5Kms6 zBD^Zd!4Nn9jDmP%NY@)1PB2Jo^K0Show!d9y}0x$*42NQh7eb}IO;RqH_$%#76GU~ zW+V>9=t@JJI8-~RQd+9s=_0R?=Me8!o;~QFd4fG@4t?>?SI$+Pd-743dmbIh?$+=A zQeMXwx?ZY5Tc0OX=6sEOGP}79Kp5iY#rzWT>8<`PJfYG+8Y~?K6X|7O;bi`}1El%g z-%8W6S(|;2E#z@a$EkbfYn#4dBh6m(q)C$^*0^8#)mQ!DIFaqZB08MCVz#V!>SlML z1r{B0pS>HQWZIBpH3Rw@%Wy(G%$5cpn|v7ukTf*8gPADrGfy~FzIN`Hqzt|0VVJVw?Nr=SnWb~DIBhaW8{>43OOs0^ z8*1|Bpu@?r=b2#)Ck*if+R6(_lW&84z`yce4`4Vc=pNp?fGW;)L$Y$)FRtv$HQ05+kle6$fb#=^s?oiI9K% zkBpNhwG>&!JkX=3YmfwK4C;#!bpACC__x0>4ZIWDde9nl3;LlBId{js(L80WX6O`Y zty?S|^>(Oz&-^>FIej=eLrT5SKgGnmUb*$Eq%coPag)sr7oDub-gIlnIo)Snd#|jl zse>w^OGL!DLfM->sB7Y?bJM#)5~G_M$B)xBa3e@J;*l}`k%b}_Uzi3LES~?2T+Ho{ z+*KO?^`Nx*GrQv{rQSA`1emA84^KAIx88cIeuOg4_V#u;<&;xo#flZ1#m7@;dyCY( z`5UQy_HM~Uz9|@ggbXSeruAe@MwTCa{#6IFL)Hb)G=%OkoY3WdgA7GK2E(vQgg;Ro ziX4x!$BQ?vy6eYs-M4Y3Ja^@lRPOeGl1lZus;$nVqq-tSNVM&I8>zQxGv} zzJBUBu;47x8&W<>R*32qgyHJ^Dt9PuDR<&}=~g$HO>={U(O9458@!db*v zS-d4JA`JQ75^>f~{P1Lth7I)Cv18??n{JBo)xw1f^*(7fJOgd&Ud8@@3}+XPKN9D> zZ|~EHJR6tcm`dg^Glmo5VP-c7TmXgXUXy&4Vlnhva5g&hUsG6F=e3P%+WiPO!X;`B>9d*DZL!X*CTP5l$9VPG0U z!_~rPK;Wbk8N&%f+8DP}O^siyM;Xil8u9DWeO6uPIQ|hEK z6J$R@>YpE;dU75r(S7&bS9tuVnWvwATCb~QBOT={bKFFb@g%NG$Qn9UaxtWxo|PLb zKgenH%UY3th7`aC2a@*Lup zev;5IL}-lHKIA`+KBDNTUn-5WUzAS=;~ai8&J`6n84XvRGJg$p<4h4GZJ?=)j&xj@ zvAWgET&pWc&o5XY^j}g2sT*bngT$SU59{A)XAk`nC$w+|U3j`X;{*#^O9Sa>=I}7Y zeH@AOEgpl3>9x2C(=m^0J?KQGTADU@NIlxQ#{43z9(s_pRnL>vO)HYhJLI3Z>BjW6 zKKf_j;z?sVA6<~Emuj4j271q($46pg1O3d)?`SVHjGKDVKj=DwUhjRO|0n@KEMCTA zI3b_z_@`WXh)Kune@Ii+zf+BLD9}xeQ}!Gk;)kc*AAQzYXUX1s?;XW__uY338|uVH zGxE@f224O)O)8G0~4n1R<%$mChF3#+WkLPbsJ6FcwY;E?OOH1C!+mK}=X@8XD+MoFGFT;GGZEU{F;8B}9-4!JjHbu^3J}rzme{^i+w?&V^$p zie<1+IEB0j`cuCUfoPO@ z0Lj8aTCUb+Ug8$j&82as7`tttS7-3JPc%^Y>M;c|3lJxC4@m?Pwh2f=#RC|G?UtVA zZ-xQc_`HsI{THW)bPnx^Gc5)+U11u+w81Bh^XnjS$E%0jMnHo`dP_DeWL#Xs9tXDz zq+@kr)7o5V#of{C^cX<5Xe=L>YLv*di%-@CCzNY-ZM~d*-FKv= zmD4HAOELXx`k)5Y%_h*l>DB1R>EHM^n1uhq#iL|!QH3Cr=2D(EiK4D%vXJ9_yNEn^!mB4%c>`*VYf#!R`^eekHF$I z2v??=MjJlh%h|;KWx({?0KVo5=-rk_{7TBpDMnu2f zs2|`FZSYEbQ}S;z(Iog69)a9gg*s|clj5Qs7U%+Er^$$k9hVypIh8!fVD?X9VrkY3&#&-l& z(o7#Em|ViW)j#)pvKP(y=n{OPPo6kVE<(j%=FRs%kz2m|D_u0kuYKys=_#f|)4Rcq z(Z9t@*q#23Z-Y(-CJRF5m9>&PVi%FdMN;?C)2Tb8L&5d~ZVMqfU&xQo$&)9`<(FR` z#r)c9ugUe-U$2d6Uo7Thp5sSYds_<*9Qki)dGlec>YprmT$6x#+ISOGZoU9%p4DuF z{0&;nKL&}bF^Cs+^Dqo+1yUPp&?Dr-4^4ymaf`xS-_R&$Tyw8H@Qc_T&YWAm`)gC> zxI_2BLa-t!#jtiErlHt456a|9Z4g6PqhTMgOrNtOM&r}0ka)sFGPEPq^f{co7~ckA zLC=i9K}8)N`44E2UQhg+EC|&vXtyqXUmBNuB(n;Rkirq$Yvas0=gqC>Y(hxPRcfza z5I9YY+pH*H6Hp?gKiXNto(L&yrA49R;h#ap9d%2Kmm*#xiN>554~;?Mkh>Z}KAIK} z&7VPfbQBy6ak@DZ+5*Qyv9*u^X@qpl_m%{$GUHj@h2^7VarVwQz~l`qqK?g*B}&Ng z?07dBgm8WI@9vMrAtq>~`K5mRVf)Cz`|KQv@F%~2N*;UqW$8o(s9NxOi+RD~q-U2V z7i>fIA4N)f;Y;*JThsZv`A5wxd0Xu%?KA!)Evpu7vLV&DWTsvs30S_?gLyDSdE$vD z%3+5c7PtN89cFvXlP*w;4Ieo-KM~*u8R(axC%#V9Bs-jH3 zi^X93PM(NfbD1l{AC;D3~PTW(<$H zZ#e&;?CS~#c{mc!K5tlAC9?{Tmg36o^!QHBIdgPza~jq*c@TDKps4_6ddbY!lf(k~ zk!NQJ@#xo-hwdwjm04^^oPwtEB7r0K5Vss6q>>fzXSwR(`WEu9gi zDUp9Vn4NmE7Znaw1Q`N%Iyb!=^t$o-=T|uo)#s|v;sT@rJ!uZHjUG`R#eXICWODJ! zY7EKg9%;G+`Jm7AAH_)eu@964_SBcz#l0l(&;8L^L${UYw|=HWs?0AN(-?%wOX2I5 z0P{7caBM!B74CJ{T_d#r2_#>$SXZ0J8`N8HhHFd04OUDDM&to|LocI|56Rj&u1K26^H zXik)@T`_WX|4rx1gmELW4x~u8Q03?GvJ1DhHj$puw)7{uD`BSD8>W$lFj;zCJwaJ) zTK>~8oyo_g1@WJ^|dm)0rX>Yy0_;KmTA89U$j8k%#pp%zF4>zdJw+RB3bhd@8J%_QeK^sm2_Hr5Nvm>I z=cadqs!#kFbVx7tA8HV3x8!dO|JfB2B^~E=H_iHIpAD&o$xPwhOTh9qKR#LM4jVR1 z?zrO)Jx!0H8X6kp)KgDYxV|4x^v}k*e%ABS{K^Bk3UX)3!J?wH=$>ZexgQXRyD7cc_l|nS;bae8s{i|n;~MFiAx$4wiZ2TLGO{3mOb=D+zgsKMF~SZ za`*bQ3XnOHxaLgY?u*eu#o=VoXc*Ku7@lz{9UnJvGvQa0!<}DHg%=mrAnEYWAaNRl z#M3Ybt;d~u8PIM^H@5nfL*030Tg%d{9i<6pb=y#P$iL~jFZItom3lp84vquGyw~_K zBjl>HzZi-2<2hAw^*4SZt5GppDOa6SSAC-YC@MwPrT_KepF@MWSk=9H*8fU#?J^y8 zvGDORrrtJ%1XyP=Z`<~(2@@u0Puk*|F=K{YcG+d?)r)51%*9}JAN*C?Klr`mk39g3 z0!JZVqOM5xSY!t2hxMKLHC*dJT08gd%9n;CEeoFrS(z#g=3=*6`~F}2MW$YRKNelO zJDfRkbpejsy#i&-ZK1*x7tYh>{mncX3+k{<8o`BQh| z=Ff2N-k1Dq!@@MGOc*EE_clx8C(p_9_5xX0a+s72tBm$Ib4?C=&YQdG(m-2RKQlW| z7fEprwmTDkB`FR?7^G6D7lVp(@el0)^M%?R9-|UCsbn%cu4d?74KY<fARa?Ag<-Ub_J4hOak+ecd3*GSv4nU1-T{~p*? z*})H?;!u)7)4M^_JK-Qyra2gHFTi#Vo-}sk0lUlbSg>N|(I;P$pZ^hyuKZB9>fZEj zFsXiHIyb!=ByCHt#iMo%LR!YR!O&ATxX~lzGbpD7cSw)MdEJjlYwMPAmv}dTQ$bu@}!H)a6&u`YmhiIk--oT;|OqVmc9Qr?1H%H)*r|n5B?TK&c(wX@~xMj zDkmMYKekX6%1}H-yzBxuQDx%73rjajt^VY!OoUyjCwpHRck9{)v&{4ctxf5oVg{>hP4^{=5CTU z&^pA;#%U0?TA>%Mt*X|NU?+mWRe&C<@bD8sB6RaG#A#>t2H|cQEBvBlu$VDggsves zs4k#oFd2R4s6iRb8H4^c1RTa%y9J*n7V;9}>}_y9nibiQ(xFmSc!0Dnn<1;)V*4Z2 zzc1*0t^YK{&r3BpAFY>appm}x^v}zLu_L2o-t?Vc3Rib?i-_fmc%^NG{-aoWKsJE? zoQfSKt!9=qE_^4os=EjA_O&xVKAE?z7tOU6S6+FgOq@6|it3tcu94SYe|?=j>8Npj z|8eP@`>YgT3suHoO!<)texl|F`NP@KQn4r=B3iY3~_T$fPCey zhh^2Za_oh7%I_Y39yZSKtbzM(JWmeV3&(b1DrqRLi7dkE?JS(B&9xv_?HSZiBuJAG zkG-oie$1IXJ3CsMgTemcp9#RfPltmYwTq?k(-&n<#w2MOK1Cbm^73-6?|5+2me6xn zJ>aKH15F2JbT-m!GO&`n0}V8_sG5W*3_>LO)#9PzP?ABXFiLMd9@3`&C_bjf4DCsa zUzrZqiW_*WL3g-Q6aC_$>bgrqVtf|2_s=12_Mm(E+O`yKNs#(wW@RHic*tm3I(ULK zFTk~wp}Z31LpRzH;@KGeb4WM?Pc}XjxJR19-JFl+77&`Y_6|A!hVRL$x(2A6i^g!p zV_)fCW3V63Q^>33m(b4mHrNgS=~!6EIm$IN{vfR_O+ntY_)>6FO2CG?xoC`Y((EyD zKANk%Y1-P_>?UQ z3sA{}nVm}$-admpXtGNEGQ%?Cavxw***?;WMzMY6r%sl=kmttgKRpBcvoX|`k7ET( zaLU-&QRTQv=SxvA}0D zN8@Lu?ER$SfDN7`7P%YU&j@6J4*?WI#SC3p1u`!Vr(# zJ>un;Sin`@rgN1uha_=cI2LQ<(pX%wD(@=3Y(-h1P8!$z;UW~ns2@;%9^n1t&g%itZ#jK1JJ9#s&`9JKR~ z5s!oP<4o5@tO&}mW(@cAG%o^=@+aaV&K(Xv`NQAkOP75^>Kd9Yw48R_fjCO{M9eoA z>CIF_iu18jd$0_~;Un51vUGc>^gYcGcbT3%k+g>)Zge!gboY;!ERmZT?9xD6`k2Yv({j*0 zxx-3SDAsxs;!&k~m|VeLS&;%k-7ALixJ=`q;>dgkLEMs(00VoY)eU{8NCx+cRGLAC z$GhVJ?iQaxgrW?vPX9AG#7(JKMaJPkZoL|W1x`x_Z6nQ9Z#lWB-00KEFZz1wWBs#{ zW>1=xNdXqIoPEL}G8wn(nEA$!9+l~DeCYMC@%lHq3YE3w`cKS1>%%{{DGna8wX}Ws zXKe&Ni_;`=a@-7Gn7=tn#8aY}zt2DaeA#{X-J_J=e*5k6!V52GzG6Ml(cUVJFMm(6 zibhBlbf@~)?53H-R}ri;th(TEG08;>O}grddA^aoD%7~;zr^ifk*2f4~dfw zSCwCX&oAWaZ~hEJ@$PGsW#_u{Pn1)RJy7SFhZGm+RFR$xhLzg7T7xuc3<6d99)@vQ z{E6vvy!OPuw&aXS(`!&=fw&F2bkIV5(02@{9iA{vVOkl&45wflX1^rWO>1TDkYl8< zv{Hw)+3BF(Ip53?6$@|SwiKAOfhIF#YRw=V_o+j?J_D-k@nf$6WLrjKP+wH|z`VfY zl%NXn4%9p!(lF@hDINc#%2h8u!+SjP<8qLr5#rv1DTz>V#9>h5)?7i)K3&}FxP_m^ z9m%E}gnDLAgBhBuy6cAQjTOjk(q26qAygqAQdlJ7Apze#G`$<7PZgsn#lTuUcNzj`iId9JJ9aCtdMv`Oza!xd`yfT@XCBlc)0W7OFDzo_WRqKXzT?1Ok>| z&B}Q9)te$#F2lSZ#qX2mW9!0CGX4WObn)+#V(cp4jLR$xkK#w-)Xpc^(&}G3n`?28 z^K-Jeqg0laA1&p>M?^<=a_Ww{-;x_=N+xLots;YT*Jc(($9+1j(nY02MDwLV(~Cj% z&_zSAH9LAx;d>b3bW37fP&Tec#^H_-Eg8r|08NXBE?w^H&+%Y<80^XqR~x6Tbwk`q zlJl9cVnxQ`A3KSOtGegsbEx|Rtm4|q6&7qZtwcz5Q5YkWhT`P)4t=H|ucE?g>yopqBuJ^h_1hOwiD%YD<%kxAo6 z<2u)TJy)BHEOW7qY7q80qx#bDZekc8h)UN%hx6CsBBU`$)Z^>Jzl~}tsda}Bc@A-x z21|!Q5H#9=?RB^+5(kjXN#9-?haW7%Fr>}-X0F_)uBl(vYmqLQo-NRHX`n4rlQ|w% z!pkC5QK~Em%ODi#8cg2{V&e!mwd+Qqt_OiE0e9<77<95TWe^(frRr&_BD4JS$eyMY2Xsg5qmyVUCncHEg`)vv*l23%IZu(aL zOal)vVI!TNhq*%xb?-iD8#(98v2&GQe)}W2>!IK2QJ`_U_UVbUgU(dv9wyblDX=^J z2Q=bnuLu8GL&xESu^QYL{CeDcr$wEDn^poYUu9@-io@KTliq8uz2w3RFN`9dK7G1; z{nl%6?ci>>9|za$+9ECZu!URdIb3%DoHIRKi=o*kT>C)Jv}9+&pyG6(9o-d+AgfP3 zUCcvA7c0s;*T^gHek=!{ag%&ByNa%A_QJjD-@0b1jH)Qt`Q}m#YZqWW30G^UQOan=E{cKDt;BEp5P)Gyc6d=?sJuV%@>*W~T?Vvs`Mc7*!68p*AGTYCXi zr4zM-^vg_T;2Vm7`(K}b^?iB%wfEJYKgEUkSU|e6FsH;} z9cLJV#$Y0@j?s^$)y2k8u{6YU<9u|s)JR*$YRRmetc@2H|5-HlQAWriAr^)e9a#NY z|A7ojTO)&VOTl)3=pPzEPny+>RWf5n4U^dmmdb)9D^&Eay*EpC8o#w{yLBZRVEex8 zS^tZXuU^-D_3!w6h2A=kNF0otd|bXFtym6;68{+c+pt9QfB5KCV9ig=}K)*&9QN|ZB!;t z7t5CYn+5@Sam5(nLIj>)TeV=W4EgsxQnb?v5T>iRO{ny3cy|JcSCCVn*)cqEexW`8 z`upurM9tEa`(JQ=Vv*m@mV(f0BUbE(G^;^xq5@uqtzcAZrI=BR?Wie} zv8pDMi~ID(QpW0TS^no3foit#Y&`e_a)Lb4#22 z;?Mt-2Y>Z=RPn~s?%i7nGUx>-A0d~VcC?fhb5W6{iJ1GEOQvu~-Tb=r$10lYXQfV8;PvJ)Xz=rH;KyL5foVYY{u#>E=J46URjSICc0DCf(N@ z7T{I^${u#Wswgdy@i;u&KQtza`f94{~?l;^XRZ!4;-`DXt{$5@myJlODv` zBm@+PR1hti&kJ6}M>=`xVlkD}ZxH^P`}J);JsTWQo&(!c&wYbx8r!J4zJ*ufIw+q9 zt_DWN=*?FHSEfuXnY?0y$er8IXI$~80Uee@J%hvaj*oqje#=9A>e`xW>2v?#eY9ii z1~#B9l`-cEHU`5qD${bt!+3=eWT57CX^NLSF^ROvGZuCFO}b5##LpM?Hl?`V-2j5N zHK9V;FQ?iKj!RxA*KnhGR?3lv@x6F|Y`BxAPrN`!3wBbh@iMW}hA|in{?KxU?aaBV zoWZWF3K|yxU?^xBc!i3SBtx%M`;Zmv`dYTF=xl1`%blUmv#fvk4Ob(?n1KvV0o4A-e+nVaw zntP0nboL7CUq0ADYg_BtP^OMP@X4je)%BNOWZ_EVsn}Wo{=rvX1~lIN0tMDyea?2t zt%NdxR`mG(U-6phuTe?!Mby?@E9+UOadn)xWvr+6X2#V$ONpzOSzhg->v_HGO5Ql; zr2`%GwvYV1?1`>?guTUQ?|v_BY-<)vZM2?cWiVFG*cr|#k5EcfX#%09)EH{~CRE?8 z_kx~|!#njI@)cm54Y@>y;2OUcer*65-GGvAzUO^j-vx?gOd~z7(DYCzy;gcH6*XTd z${A~FW4swtL(e5+u-W*ZwTl}{3r1oXr-|pw?sTv~ zeCJ(XpswB_XRym(z^(N5^Iu1;8!x8XrgcOH=Xx(>ZS zkVD{c+nb?2>g-*to;ftl>uHb2f;{E( z8mew+B`9XJnlaM^#$c;O4$rEfVJtK_I7kyOJxC3~5nlT4Le4#Rmb%Yo&sqGj#pj|D})9 z+i$pBfQn#Kn?sA)f`nq#LX78>Vq;x2Aqx8LRi~h#OjJn&2R&?$(9-i`s;sGJtNsgw z%L~naD@!tnw?KPu&;kybf*ukKhhXQPeTPrb$3OSYd3#}&4p(>6?KkZ9lm>6?v!6%V z@Guk(QT4U=h_E@AyXUfVVEIDPa*oI2H1gEfC>R-(&S*8KF*A1w^j`IrH!WWaRp@CIY6(ndq}e~;`xbINJ~hb7`A^i7 zw~j_@uAl~1%ve}PyR=tBtJ%rcS-ZFaW*7@i51*j27`Gy;cv_2<35I5?p*dd-e;&`> z>!RB}{5cw%%v8=5HBEHO`~C}E`_8+mq4hilU(ox4tmjxjq^5B#ZM*CiTDxTz4Rjr( z(Bz2gURXi?{5Q|jE}lz^3H=5f3L?GLt&*JpLK;(%Ns980MEPyIW!bpZ1P-mlfI=Vu z4@1csDjWC%+axqlpn%tD=Rf6{a@0*TMFeud6m-oC^A5bn`)6{8@8>v}cyL^}%N>FW zZ~4<+-q<6HTY6kQ$GF-=8+daM#2I|Z?li5Sa5I!&Tt%^-7b#V{m5M6rq?^uM^M99` z1E_FL_8p=4{)eb+-DN!Fz}>{a7ULUAv6Q<_;g$nen)ZT%Abt8H@1$F=-6gAnNy}N3 za}kfX8e_0@IWc7j=*o6@%co*hpV#uG=rz~%oj6m!zTX*^?J81X?tu@^d8Yns6k!om zUQfPEhl7_;ylJ<+_{eHi&fwpirf*VHE2`$@s6!%zkderALk;salfhu`-{|-hcWZw! zt8(6Q$(!i?fA<4Af5$b5$7#E^^Dd_M{-_%C4qP4YE#++BPauz(`$Lr8q(UudH)bJd%X~}G4gvn@Rj5Bjg zhJ)!&fc(49d@bvk^X0dEkZ%3pXQ{9l6Zp^EVGrJ`-+mX}%&l?Rfy(hp7^A)K6JO=6 z6k`IeG%yf<#MV&bNBq+%kdsyZ0~}y_8P>MNoWq&c%V~-#cr~AW!l^V43}@5aGw1GK zcY#mmh4Ez;XoEN85e^KyVKIi!-1mTVNqV#9Y~$)q##JLR|-F_DK&o9>E;&PCfCw5oqpwm_jy^_jGqIQ1B}wu#MldiFgP&` zlY0;c+-p7x0>5=?DhwQkr{qb#H^s{dPaXXO4NfHJwaVM5s$~GPu|z!aYbg8Wnnn3qGBUwk+_2(a zeO=tjKxbtQEmyeyIPVlQ_%sE$yEag$tGa=g>HLQuPO11=**p2KegTc+Ue`13H^_oE z_p&jK*DVLG1}t1%b?G*V4&tr&*EA@_4d5ki2=E%+@yS8rZ5d?B)oL0#5k{TGAILFu zZ9|q#|LI#)RM|kmvU*nxkS`Da>S@XqTn&#+%4QeW?!M4q#)_>49}WYSOrTO%f==`P zZ0|UgFhprC{CnROAMfV*&CgNi%mx~*zlNGy+ihbo>_m00E7`s(V&KfMC1Qj=^_A~sd>7r1OQoFkV_SFKOqaj;Lwah{d+&e5 zTBR(r=BPDD#9#cN;%C6gBt`o*tI`83psO2G8_?xc)3la)s;;3>_a5HuEGl`9e8-9~fHC)~I-CuC;qgmb-rNERM#296V ze-VT;i&|%f%yK`{iu=9kw;swMD06UH{?splS$;Gspg8cHQs=ucg$kSTOm^?1LnT*J zqV+0jSSd5SZy!o@X& z31vS}1iwKOdn`XF1a0o(>UT86b7>Lxg{D6*pQp@}H&V1`k95JRFq2pKKa0izX+4h* z5Do36!n*Yibec*fj2&|qS5Gpo#--9HN{$g`Q}#OzyTmV*L+c)gAJ?>0Dn&^%n4A56 zzG8pF_~j`Mv&yP`gTt6Q-(^ZMo`o|M{`f>MO&<9@olLgSiCW&jxorc%7z{0EEZeDz zkH};Gk8r@P!5mCM4NR?gqk&3F@?zyq5NJj#)l7Nv*}c>=;GXi-wB~{{w$^q;Z9(38 zp0>4R=bK#jVlFMlnq~u(ra|cpwwlZ{T`f{`b!v0hn^(^XDr~6bqCnNuH&JihjTGwJ zFXQ#fH>HMMTtK;WI{kXxOCbahrhj+_eo$C$d#X^~Bc zR@>qu%HD<0SrxQC3?-c^s^c#1unPoWK59iu;YTJu#0t>sc3%JI-k`H`j>S*B?%`iK zrspV_K&|vBm9%lIElKC*7L#7SoXWW>X>>`IHf^T9k{vvM5#!OAdS!TN3N$kn{|pkT zVOxHLDW8?H1r<#c7(c=aI%@diO`v~%+8hAS z(d&Y>90GmMQen*+&Ky(OGuEJ-F`PL$FRvc|BVz-$8a&&C!KKs&niFsUo8xd+3dNr# z{lW|D87ZUexAAKE^_>~-uz&&S0O0lH#Md0NfLTD-JFIb*}0 z2>0&faWy+w#(SDemf(lKgM*88?&IpHW5ql~8cq|SehJL@Q9OZ)pC41I>9;BKLgbu9 zRpu#`F9FBw?DtuI3B-?zNl-pwwgesLI<`!2B%wI_e@j7QtGpRlKPJK^!!<+Xrt}A1>o6;Jr9+6? zJl53CYiGmX#v5;V-D}6Y(?D|q&U+)emIA!O5ZCbL1iJURRdViB(iOGMbi9J6UiG{x zy)9C(nGZ_A66N8n0>rAJ^Fw%$@0So* z6b^iWGyFjr6u(s-ZNGJ0KF@yZ^K!2&Jks$r4NrvVVD;@(*tA`G*)Tm5J#M=2h~H`k zpXYb;E`F^-I9XW3hCq{)&ucK-4iXfZCz6z?<7B^mNj){ZE7fv3?26@eM^E%qDv1@u z06@SCG}EDgr%KG`9q2N{;X5}6fDde%j#b7Yk!{o+K1ppS_E5zJ?zv!GKp8ugl>2G# zndSv3oXMY~Co&auF5>D~7vqX6b$0BzY|NW}baPK1lilNBA&>c=zB@Axpq$FWB>jhZ zUQdGevgc`{+;ir2M^E&5gLsaAro70wX$b?e$t&Q>6a_|>L*5z%YK0+ABF6YfD9$a9 zQn^Pj@(F;7Qw1!pyPWq1z1h5D1~wx2JSe-^XcvY0_fuD3D@7W2^IGDyA{TVIPGH+_ zHMdz>cFHwYPAvG~yM=K8%Gg-;|06%JIDE@iG0H_;eT{MTj|;+;lb0wv z5OiO62KxTR;D9J5@b`OJ-?mb4H36F$(UmY16OAb8LCR3pK;g79Xv-_4U+%Z=P`}DE zF+7{&eb&28c{blL8eRQN`PDOA2@3PlzhR!!f28yVDrwwA&0Jx}R7_ZC!w}40c*Mc8 zQ}E8Kpz$s!=*a>Yik$=^>=uz15LoHS>tH0doG#n=HI6Nl7_P=>=5A>Di`ca)|rP$g@wbEdK5PAJJt_t3}y0%diGn6yBw6#~rt~A(B`ksYwU|t1{E(sjTl15679+1pcoZtgae{peP z#y1mjY$mXLE)kC{-^cwgFBu1XTrC-ESM`f?5m(q=)mEx@d)dk+t8px6s#ukJ6tKKQ z!xoau`W~f@XVA!r4j-ckFQIxgxPzj#7f~Y{fWZh1(=RdX1?BA9?>Lxt8s0e-G~S1r z54tC2N}DLvYgVk)+PGdWxe!Dg6>EbrB-0DW3gWp&aBh>Q>O-EAP zrF}D%)~}^DZZV_Pj9DF+KIM05&%wY`xHPBKL`LF3Ehtb@&6eDSykN-`1&XX4j0ExN z3|-se-X#X{7#-+3h3G8)t)l}cGG4lQT`O%|)69w)z*W@aHudwUOOkiLp?XSe*eHe$ z0G)4nn88X{0BY4x($Q#?#>U2IXsC%whF_q%z&MpOU%>30bJ766OSQ9IWs+9=jaG-{_ru;Qk(-0LkY_@rhC$;dW`R@*~O!R2y zR+UKU@8v44TI#tXYl$>>YG$i`5byq0ObK50u$I>)hCz*wvvfY6st*c!dU~1)CJs?~ zb3Gd@)!464JZb*y+_v7nrk-5H)uweuu6pBdcY9kELf}SQ&5|Bb*zB{eXFRWowy5`c zEti60Zm@)Sxz6z9FufMMg7Rz6rxrE@Lm5{qZSB(byw*QG0}jlqpz&^1(BZ;nDvu0M zVRf7JNz~5ZHFd0w?Tu7lT|q;{#dRC9Ka?dD?}5;$ zui+KTs>^A346E50w^DJX@psaRu>y)Um>qp!_3A5VCwETGkM8a9aUO~*4Ny%!kA>D% z&H94I+Nu?St4l9DPtu^cP?{}au4_&W%Qs8w*+Da&x5`^up_@u{JT$@!kGx95sX97Q z{uZjNX_Q8bvr8LxO0I8byxb2s9S+Q^puu!79|~%^d<`XrtW~(Od<}TGYS&i!$rI1n zS20%DdEhzPu>A@<>}+|)22WC7=YH4cF51lFB@NXk47Cf?E=?^X?ME1^=RoB~u|(9( z!+ij{SUKkjj6~2H)bXfpx{0<-{*nTvWmHttF3+k028DwF(OSM0VCsIHO;=yaxcc?; z5{#dm#g$@;Q2GiKUpIPen+8kNK1{3MkY zGX{%zsb0wiexYUtC_CK05^z;pRc>&DIP+meAAmw447Jju7~@k(_iW_}*A}Q2pb_mozkhI96!f0oJ(R(<+c|!=ksGc8KyM?qft6&po(9eAJ5dPU13Xo zGC4L#uRi^V>oZ8pdv3kb6=zrcxqgcCR3gGhi~DxMmIM$WiM;q z%!U0bnm+joC8mcME)X1C$7gDU31ki|Pd&nharNe_cH}m$ZhM0WOvIfNA%|2n*bizd z9cjupNLo{o35pliiLkRwuX!ichXeHvve%6hK^L;OjSI) zpp9*DustNYwb61$gUy$V!tEvfS&-&|;6N8Qx+|v&+gZ5se$Skt0uGBRP>Oi&+lSwM zLq?>3^W!h`@|tIFF@`4}zK^_f8sEmPoHb2#rrdJ8B@GsJrYAn$Pt!$hb}i8toEh1< zpaAd z(S;DT98v{mrB2c!VRriRD9&XGSo2#WhF6RAiy0@f*^Lb*WqO)oKm}dkAAa}s8K2Ds zd3F2sJ85kTRzfs!mdYRdL2+nhRx7w%?T|2`)-&+y^InSdBMcNrDjuRx*9$a0+)I0l zuBS-DZfa(w+}_?UW!K*xG6N<{9dh3N>^B)}Nt-la40t~F7dKH^iF+5Q zfs=>msYk!LVV{19>9(!Yym1+_KJDT>aHs zErDt9uXuv+)0VUGyL${+gHU{;j}o<8Wl;z4%E#J*p*81?slsYX#-jGqo3(^m4kjU}wD44_FWfz&}**Fm8ptU`0u{yp9eRUn$+)*f$n=0`Z{~ei0-} z`5kpRh^r3(S6kM(@UM76S%4qZ_)-0^(#03zlLM3pl(Ny>THDrvFZ=EcIRGV+Ea8Qf zc&a2guN5XQHF%qmnJ2j>uRd_cHMHY`wG#I{Uds~)oKiVKVS!VD26Ran!qb&QZ@Tpy z=}+=XkD-$<)6_r*?Jv5Drkbvy*0y%L)vQyfApd*3&Tncd%z#A-!0JU(=mnhu3pZW_nQCZtXGQSU%62I1X z<{1PGfsZKY7%qXrj^ve5Z={SW$6plO1WS3&D)I&y!4851&w;`ruP)ijxOzH=aP_5+ zF|K%yyosTt0~ix$#oq``6MqyBqysL*SQ{wiG0?WHgziJkBh|8I?(lz4g#(He>|`|+ ziBs|5b5y{X$}cDqo`dhsLnIZ4fnJhp^6E>FS7zyC#S4l7{8n+`6OOnff+^AlW+{bN z0t|JgTm3QaDGc#Y(?GJ0Ix27AvE~gj&WxTeOp!#d7s{!xoWa=TaSKw=8Z`(?fPXAJ zL#2byQeo{n7P>|y%RLGfrU<|8vTgLh&!3`bJU#Y|cN~7{Nd`|IwVk&^sJ{P9=)PzF zm45N<|C*tk5swGI`nR-seTxll1+EQd0GW>jcQ=N{`f0S~EmT+E;0w(&$&f71BF~|u zapbmsvlOl*8*3+U|hwP8CUoHEp6G*Dl$@% z>%c$G5oZ;u8h@qnsX>a2_0rUu+qes{LAnqj2z_^E9Dsbma-UczCpz9oMaeN%(xxFH zdBLv-m5P$(2igLR$!m6PqaOlSIU%oFIaYH!Q%Xd1iE{S9&lQF@Y5Fk>C)|IKLc?8j zldK`~>K3+XLoJo=53x$|JDubz^EhJZyYI+WmPSEC21FrTNz@bnM> zCEHQWi-Ej;>pFV)@#nbE06uBgb@)ZX(XwtEl~!=e=``Kc_#i$0!!OWtkAGK;LcFiJ z?^Ew57>B_*0j>9GSSlXj?$(#6v-~QmZrvo~E;^M!@Ac`Ka{w5EB0|j^N*a_UD4k5U zQo&>g1tyNOlIB)VfQ@|kbI2bXIhLooqKsa50pp72+b<`sZolqQJC51#pAmnHeK1}M zo!mn`Rcs;K!wuM`-KYwx$FY%fN5^kLw z?W0$odW6P@PEuuE3zcQryV>(f`*dumo1S^>+w{xtf1U=rkJ#Z(&;PjVUGz8axsju1 zL7TsibRZ4l5JC20P3~~_E7YIYPI+xtip8RCm8R)-zSsAk83({q?e2z>mSKcoAr0iU zQ8+wJ#Un3K9-jw_D_kEkd13MptJZDirDc~3S080u0Yw&sT=CQsy#|T{4m8~V8jUB4 zDAs<1U=14#_ySSbvnW3(-&n9DGQ)d86dd*&9jJQZ=~wB_d;SlF zBC~J*Sl6 zxsyKlj%(*kltwSycYr3Qqja+Ic4}*1BQ;7Lf0^?I{>3xm0Put!8DMH+VnTMn0!8=R z#?T+AxU`ar8@KQnV&*oc@?jQ6PyBHiaK(mUa87}U^SSVk>u|BrE}9-XL7h$Srk1tm z3D#gC>kCE0pT+nErlzK*L{9Y|d6C)&eoAF)cX5S}m-mHsL%|n<;W=%ErZ3@^<&sw_ zlT7^5al*Mq9n?5?2$~_T^e`Y%sf|w$^9C0OXpCy>jisf}SkDF`|6U z9eTX4ECi3cR;OejG z4J@y`^2r;Yc?GPOl{}OJ`4;6_62}TIp;W^TQOvN+Mt3%PuGAL9`>21O8yr}07dN7d z8Ws+%?T2{#gTU}!3Zx>u#xk#OJ&%UQMjCg%?T!2nqUZM>_(9_hjdQ6I0sV4gMDt z7Snh^I}OFksC0ZEr9wk&?@%sX-zs5&BeYr@c>2|yZ(UkkJ;1oSfDOXnAV`h!HLWid z=yTy8^PRC9(9sIEmTliI)6IZ22ubk3cZTW@hZYT2Z)IHFW8vy_Q!Q8y-Fludp>UVY`e|H|^}X*yGR1zdgduWq9qEU(&s5V;2HY?W_f^TAf@zRP_MoU(#Ok|1<2-3sG{eSN$V z(viogDRGR)c`o724tT=={#&RU9GRd8*~0 z`6sGRc2Obkh*ii7eB`q+nL5pDP~hsvk3Y+}dY*CBahkYl!_7J(z`bl7{83u?m)znpf z1(nyDv1TY{*f~JHp;tkB7M#!e{@moiX(?zV1>lcFB2qK$XW&;Idw?p-ig>*6e7>qR zKzE`u;qMk`lWeH<{NAH<=y)G>oE%`otuSw9AEOdpFSnwsnAfGP7t7sU7jEQzT-_GF z=1ZB7cE(_a7ciOT9a_dy<l5bs3|BCsBb^jwgRtJ+jCv<%uI^N6jYS5BbQI=PQf-9_} z4P5QIaQ!Uu%Kg6i@RoVEk)O$USjL)TY)E^oa3>{ex5=g!x?eMl%;cHPVUV*Grtxg3;BPP!aUU{=A%9 zx^wjQK}n;sr$Z)?S5rK-bIsHrs3Jea!wy?0P*Np1oK}+QsuxfKH&++9xx7fkG?NIkan5TL zj>f5@^hzqM+h_w=jdc7e23|(z);aQ2$EJZd9iN7xhCe(#K^38+)JC1W1nsEoH61Lg z<+-o*+yx#KLo$=148VC3nkiD#V%l@7b&4UA1g{Dfn;hibxrS*Z*iL;#+lk7WWF#Q1H1uiIupg8Uq4S4 zSOhvpDs9PA#1IYObL-dg=NBfH5Yc#uDy0W_-jC$N>&ON#jsK3(R3!mL5>n=*CR(UgeeH4r#hnAxV4NF} z-qVcRv+x>wqhd-+6q}CoG_Z~W?q=kQ0$MTX_5p~so6)abW?^8C2dR?Qu5Z{uVBKXL zXc+;Adawqi(iq!Eqz0)jFiu5@QOe_auvnfpz|*+$dC%ay08hE$X*U>$l(9(6shLU2 zvax6#&m>~Jk9mw~oRZN=g5g)9ppwSe5NDVgd4Kd)3i66)80Q17&~iZq5a(J61kPZX z;|t5>2ban#C~4qD4G=*S6Ft-(-%quvUNc*O$EfpR+klFkIFEeF5J+cw-NPSCfdh9j zUbj6q(aSC7X6nq}Of%(e_CD*7Y1-&OMNs9Mk2eoLSK#RxD`=!yL66pq8e$9&_EYo3 zGc3ZosknY474Z0=sVFLl`9TYS)`d)colyY19M*Sn*EY9o;^U}Y_E4BNU^t$?jbat< z?reZqTg?Em?^Z4cR8i~rA_Oj4)lk^z_6F5+_{HFiSEMUUPEv7d!cay^X&_Do6r%to z_%JgFQt+{&PR;}=JX1*1sS=9il~5#5MbTgl1$cKP2tp`i;0ZPi+EPZCw*u%}&E^)) z6g$9#;sq^f^umau<{^ug_(5t)osgavtdv*)+XhtTv`v8|okkCg-RPKyC8=0g_DYXU z3{sLSi9NKI28%XRKKBR#Z&1pp41%IE2pswq!Fiav0xvI$f=0SkNu%`wkYkJo$~rZ2 zn9hs8NW}&D(tRWg1sF}832?8%o`<3G&G4A)s~ex_r^!G)b>(lOc=>v%Q3AXelL3$c zVvIFw?Q))1`-R(5lw4YA9z|fI|S|fFEHH%Bs}i*Jluh_ozeb1Bw-R0dDj_ z*C7s7#1M>nAlxY=@eaQgqa4jWiu^{tqjCp~phc|)i10_l+y@srMr-oBs3b8=!7^TY zm=|OUma%O^VafdR%cHJO}=P*EF|l$J>) z5Xu?7fskohK?FAa9-7?4*J8wI(G)Zy1F%8h0qg)ey1a2rjE_<2)L~jrho~qoPK6bX zJO{DCtPF)NMX8oD@hWH!z|jSsN<`B@W-Hz#1*tpLMuS0KyrQhZ?qY=HEI^C{f>S|^ zc>3<#CFhxpi8X~NmgOxOxs$<36Od%dRDJ(X_Bv8aE%iwpmm=&ihi|c-D z7s`P}RnYjT3P{vS0c;)Pfx-sh3&K5Am*}J>>ZbtjG7~7R70XGi3?*aL7z-p`*PF+Z z*#wPd;!zm`OfpSICV4E7TSUP+>gP?y$AWDXFRGU=AJi<-g^dGX28ba)fvwvs_rF+0O?AMRJIh!Ex$w6=qXKz>1C zqcsgb!f?d6Vpk9!qq5X6RnaI_=1)^mYKrocVPnU@Tcyf!qB_4C6-OR|V{OdJIm$K< zg%l5z(ez9?jit&c6sV(65Up~vmaaPa_Xjalm9t(`5Fa}se;k*h8=T}jz zfTzzC7TMi9I{pVB!w;pbKz0gHjh9bXCkIqfD+L#TQ8;Iq1Xw>R070F8T-I~v{l0(Y zb3hffQ+{a|IFz+2Y8@^??+oOe^wKa5&O}&I^Dz@=31;DoB^rAAQuzgx%;)8h*-#M; z2R)#4A+vBmhCwl-rR3KVZ*O75BI|<)pz=Ib#P!w6|}NY6*WMvwN135)emqh zz`Y6@N>>##K&xQZFt1Y9!l2*wsm}qQ13m}Ngack>4dsh4$S?Ro)ejk`G7R5@UuBXi zXee8iYw$y+X~P2Ns*LryGl{$Z?8@fASy9l+A_Xp}v!EGig&~B zJD&qS2Ye2k4Gus~!Ku7*`kmM0CdZG;DNyf$+w{8Q5#PJYIdE1Lw3F9fAbZbouM@Be zXm7aloxb1afX@M+1Ix()m1A?o!710~y3fDnb70vNw9k5<13m|Q4)`4KIpA|(b`Jc1 XrJL@*@lU9D00000NkvXXu0mjf8a1o%OppYabM3kVQ-hF|BdP4>G7Wkyv7-k9hhT1|{SV2-) zm`K6V&eX!%1PY2O#^}ozW_xD`M-#L!Uj{~LsSzCAltMxxl)iNS{Q2wWDACB5kuT}V zIy!$og8pM22CnEiNX98PB9zknxrloL&4M)VghdNc{yK>tm39MKUakt^y1y5J3V^n>%~w9t3f z$N}{443qNIz8;y_k1-a}4atmCDSl@D{8_8*A6|RDgL2{H{?QBGPjFD!5QHoAd z_DxA(6Uql@VgxX8fq}V&MYd#4C;VFlUO|n_cfZz$sa)R^2SE+yi<&48L;r3WQAEmv6>aJ9<6EjR_0QL^vUP99e z3JRU;_0OBAF10sMQ0nuNB48Ew#p87NHYJmXeurTn{o3g%9cfm2D;e6l8L6_AYC7vr zmNeltec2|=1UDdu>KMW*%z$(94uTIQsb1$k&}bV`$XIXMx2iuZn7?&Eye?kKRF zcxQio?{3|$^f-67fBxkg|I0saG>;6d0}ARZ)VrX6zJL-00gC{BL8Jcj&;RGQ)ztU* zyP1A>>%rEDYkn_J4K_<1!^;&<0Q!V2QM8m+%sGbXLa3HTwtMmJ{$E#;MujIetI=LP><>E%k$}rJLA^0 z!d~D$ul~+@u+GX`%8*HO3`h}2B7<(&I1KR9 za3s*xxL~>vqA)lTSm00S6GBV2B{EYh7}@ZQ=`o=}4B!5AXurr-!6z zflfrWmSdLu2XM$jH)~9Wk>6^uxE!d!k^FN3_Z71?&6kO#CfrdtfzyTi#f?hBCXE{*dGAA&d zh9MTWVWjlP|9yxDrgH^;;-5R1Q-L4%3p0zX&KB#fii2TO5LScUg15FRiM%z8V#7ZB z;YiZ_vkF0MNb>M@hR05d(>R~oY5VQTba7nOl69x=lj&%x%W?f&EVB?O97Zvb?!T*> zh#`tC)&b7j|2-13#GgO@D0R~&+EeRh4cdOPv%yMWoZ&92|J zH20hyC*)pH8M7U_f7Q~CCf0w!ezFzEG;jn5f*>1qJb`wXgZcX=O*08a;b$-nm+iF5 zGfs>1evRb}jR>f8%<#DdW*Ruy*EqWfM}qTSpwZ=MVn5G_v19=GzCX_a_IzA8!9gM+ zRSjfP?5{x-@^~U_%q0Ae9hktxgb{qlVF}7y`p$1`;NmI~g^0x+jAG+cl4jci|BuDI z-uRj_1cmPkBv6ghFI%~Y+#GzJXVv0*GRC*`^V>Vgpj2^Cpv@KMG3huhJMY**NV#^S z@6Cc4;JpP{Y-!Ne%S{g5K}eyzncf%h^%a_(zCOH!FGYba9)=!jm3r;&=Ns9ARso{I z*z{Tfr_`^{qYqIokXbja*X4FWmDEUl&gwDqE%5&H4K`H@gb0D?z$p|3LGK1?H&`$H z2wJ!Al$q*H997rE2?8@34Yl!zRNOMHv9S`Y-!VBBOniYo^a7(usr z19*NBB6&w4aJ%VhqN!DCUp1ay?_xTL1j^?8J`Z{hj|+)`B_Vi=AZAULs-|b8ocS`l z$2eny+|)K9<&S#G+|K&Dur?ysgLz-`7h4*E_qY2)`a*o3S04+w+r4ggP_q19p1WCv z=DqjsHlzA)F#z}E#_GAh+&%rV*m|8fjJ)q~RK{{Qc#)v4XYn4qb)*pY?hlN^3)4Fe zFYJ;>jeYKKewE`qr@YtlFN;={+bDm|`#;ScFcUchzZ!`-2>gT90*xd8n9CXx0de24 ztJtC(_Bw5E#DR`^_umsQ@&e;C8^dwrE5Zxoz{C(9YdxGkJZ*S(hBRA+o2<#D;AXoLNF{XhP?rw$r5W#P!b-e(gO1M@z^(F$mO zC*;%G3sDK#8l=p8jc3HK(eJAq%z4XU#im(3i?36l4n%?nw|Uoku7V)p;qPVUOWQC* zD;(Q-0J>aOXYl{7Ie|V0vd@lL{l4aOqE1v0gnrf{?ggyg8z6EXfJ8AdWL)(thNeua`G(xhE$98_w z>7qa``A-zhA_bELqeG)cRkWizon|~Wn;p)UYga$E->!PaQOaX+Zvd{8#Hf$_zaZTX zN5Y1@6TaeXzGm?JL$n@l0|-rj3J0Yb+v~Pyp8@Mo*L6|t|4tB?+VtfYgt6o9?SGFC zY7vp-d_oxI+)2BUfv;Fm&&Ab!pw8p*@(9?ZHSf#Ff$-;3zZVwVNFi~2UiUL)c|Mr` zJ@*(UCJQ0(@oKhId`)?Wp<)^tCDrqT3H{~Z@VLN>_X76Qdhn-;rj0O&LQb#YYsMxH znc}*hR{l59fJE-^4@xU-H zov2;nd!0b5E|>fnd>0@bjO+u%HsTOotJ$*VyNz%laBJ7Z5&vtkkRmvh@-unA2W9Xk zI<>;pdHtMTscr*9>ro9vVFSnP$NBY~{-F*CQ`fMd?X>gd>69X_cxynJAN0Q)a9Yh( z@ZE34A&Nx8z$3sQTcl`IsG|gTZ}vr^cl6k{9%I2lrz(KZLwmN9^yThXdb{NuMAAmZ zm_+{@*8MFj>l`-w@NAn5c1BXj--Q{|tF)bEzC0F2SPlpe)dK++NN$MwF)|Wu&fkL| zRHtYF;(C#>I_$@HlKBN^KXD=))!+S>>fbeeoc5Dc`Z-go9v6Sc)Eh%8KGjkjupkf`vm*=N@ zKKl~Yx`L~u_M1iNJsAumuk}D|TtQHA{CIieD!1#=ydmS!ax+T;lHX<|M^EOPe{JZ) zVh3+v&0Xjj1d&1E*rm~Cslj}bH*8wOaRN zuK1esZ&|c7W#NBAOdmwT#s1DRaJrimE4=vrpt9qE#>Nk$Wt!0I=zL0=aR$IClPxPJ zgK?DDUc`~R@RUCUV6fb=|0IZ!LR02#DW0-Pv z%Z=TmTB@cvP-`}x(Q7F&$blSebh$ICiVxuIf1=^XD|^sZ;G^^1Fad?1n?v-OQi6d6ru-nGER97{le*~CqS^ZM|)!CX-W4yC{}?4OBwj3>|XL-(PnOES9r!Zv|0Qvi1gsM zA&!yoz8IuDyzJRLkJ8d;bv;gHvnr~XIN<%X6)Kk7Ml|K=nj}Y{;_G}et;e~Tt{S|C zI6MP?P=?y&yZujGlBExUHu-aLxzXlvd1Wu55@>e2;)dOVdrh5PGJVm3U?P+0z7Z-! z{|AS|7C)i6VB1v7RerZv6yzbkb$r&%d%BxF#nndiHr;;Np?}>AsD}iV5lNV=X6bw_ zryES!P*N5ELtyw(YjN&27)xGERuuG8o7?z~q^u!hs``4N*=O~I4RCOpr+ zpy{>tLEH%9{Xu{Qx6#7;gXjWP1TM!m+j93mVE70k@m|Nyzv4-Ygw|9>DY* z&U*^$BeI6KH%oTvx7JtTW-dQ%9BP*)o&-oj7m3k z*l?_VyC2IJbmsT=?rQgRsYdpv8@=~Zq6{ILtR86hiq$=+^r`8K;NjN^*O9LWaBHbw zU3L7PuK$w=df1~2&Iq=GKHvy#S*m%0%^L@MLHH}*L zitExbkkbnOths_y(iZ0`OTTW!w-IG5t(~1i*MGA8Wh;-cP$KXa0Qk^;P`G+VDpBbF z_$)+Y0EuJyr+hR<&hMr5N$yhswa~@$4!O4j9vhCR7PrfHGjH^oj&6i#yk)sBrj_O4 zdHVlTe!cEW&r~$w`Dje5^S)5)p0j(jO1b^npI<_|X`Bv*WN7Qp39f&2T^CGLu39ya z_8DK!f(0f};&~mAU}7Nk|3Z&16UQ<4-u%Ab^St$Z2+7#Ka_#5EAvB;(O%l4!8<6vC zDpE}zUWW~QdW|#xfHN?La1-<2hr`Ol8_pLXvwpy3l^{c{cRO93woau{O!A@mA4G#t z_3@mxb@#6q^)3SuDW}^%FW2+=em9EypeUADcqlAi#Cixf@y_}Okz?q{K_=qOe}e!b ziNm5r;W^7TX%ufxsgZQPl3bzF)<6tNb+^y$O47req@UdXJw_wQn0cYy%dG?dgX+ca zZpA>MY#2nW_@fB%<2*{)RFw(zKm>hdV&Q#A$p0!wXvGgS$s_WKDvS!UpY1%~;-m{;+e~;B`SXP&toq)ve7w#5bKzce?nR$jyy|mMfBJe*rOBBKe zPU(Whw})E$YP>y^Kq4S>mC_W~y6>F2BS&qi zkrg@(o3gepe%Lmxl%l$mDwObgr?!fnb9V%_ihb4D3f)pzum~2*f65RRFc|<>5qviQ zsB8ai9O=kBd2Z>;C+j9Bj_#*!*T*^jfiEW}A1AUW9x3EA9swAj>om&NdfMStjeT>p zaM^Y_R$BDaMbvt+ZhtQ~3ai5qh-Y@K$Lt8?ujR+TD;>H6BA$i3LG{$=o~^ra6ES~B zo$JJ!0PFJScCY(85;p#30P63ns&HZtMV$a(Dy=q|$={OPFcjb>{^NCJ5r>tXdQ_?y z1m5<^DlbiT$=UkBXnGQF+J4Im@ZlqudNaf$(6geQNZ2-SW6?Oh`@a()e?8AgJdW>~ zJhvx??ncfg2p5ZbK@RCl7?lNKEOqIM-Eld|Sl)+b`#NdYa7wEe^ z3_&RUsz$`8Q7jSIoH?VWzaQJEUV<$5;0R(`8kSw=bXA~~M!SC!0x1A%t0qN;CMBdl zXWgFa&#hNGVdK)aBX{v=fo_P+?n_~(TZ@iuWB=hLEF@ck6-%D%cZ#(w4}gAl(0g9( z{&=Zz_fl~cU$;G^jeT*VQ`kx#qokz#ejx@?sk-lEW$7rMHAUu=-y;(ckme=WljSIK z*0aVBUFU7EY6ppEDdHAP-J06@r}{)_kW8RkDAMG5(0h&T*Wbb!io$8*32?WmVV{4n zOIO($&#a7d%43d%F=+uXDR`qujtcda!xF71b!NszcGp2JUFN2Gle=K)=J6EskwQ0% zC&$)luyhtj&d44LT9Kv>Za9gCx2=$u+fgEFmWV{PdbZm_$riz+NMR69%Mi;~eU}mY9`S2cA5R_kIy}N^(0|7@4|g%pCxP)DH(5hCxKL65Oar?z(DE8rk+!qAKtA!Xpan zKBab_2}6P7$FhS7hkTrG=0({FGzC^a*Zm%o--;0@ET#qL2~5<~3~$rGkrVpft*hX} z0!KZagiT@O8>O72HQ&O+l-z~OGJz>+w$Fz>YmVjrjwob`NHyo}w`FlN1AVT@tsZ_J zff$uqEj!4-6Vmn93@)dFvtfVagfQz@zg&!y4ScdK9aKO`uC|1AzBvz#MoL@G<{U$ zxOeW}&YwBHhVaD5s7@LqMYedXMvFPfB&j>g?gdJC^YB0=x{}YRst)b1-0Ca-?pr{I zC{L0kbo)p;*GbXMng*}{AjL5YSiZ~)%aM90MHgQ5;o0qJf|?aJMmg4&^QE!ZO`J!V z`UesqNPvTdi6SX0HdGLkVQ+C67 zGaI9aYS~a`&^BNkIxB+0-VBLB+EWn=_yc< zJ2&zE`aDyqmP@k!9kP6GK5^=@Xv3c{zwRf<$Ebeo!^sZ`%Alm<#fLfI^>wgS>^9!$GeECuli31>Z#t@qCVe;dM6HU0%I-mzPwdHc8$b?29**W)Yf(+ zmUT4s_3yZ;U9ee(@0&K~+d`iY(N=EtpWeq@%&)U}HFQw=Jzr*ClZKFY2g9I>*uoN* zv-tr$MrGT@P@ZHQf>0l2&NET)h4%ZcO$h!$D-lZMG?f6DE%v!GwWG&gV@2~$s$4oZJYzk;0RqOm z->>wjfZ*x!1P72znw8w~s9m3+W-@dtQ30kvAOgTEdZVVN%I5TARqW>eYz>&$u-+*5eNT4&`UU(^&TH>Q}} zt0POrgPYE55_aZ@_xa1uwU=kN{qd|%VBXWV%jr)qR+a6E0$=+-`RSAnf3;Pw1NB%B z=z>dXde(H@Nmce!t*xJd&#v0DTwP3^its z2@xfYN$#M*pNS*txSEoFoS0hurD{~#*zMMTvbR5S37~C`Afl7IvMtBn?4Gsfo5niA z*C0|;nHHdZQByOy5kZE^2A$=Z6EU8KNKG^2?bBPvZJ8GjA3~n@(MHaWOz%_VI?}ZZ zPW0Ycx3@Wpu(vmxYm;r8f!OVA<6*qA+~hbsLQ=Ao2VqG9r^kiyx+<6XMAr6j(rZ{J z@6e4S9*hx=qYQw9g{tQccuh8MB}KMNiE*mJ{Tzx_cqNYHmbNYe53MHi$Dm`Rxa0QD zHu*|}iM0&y&)chm{9#-1A45rS zF10LT<9hem;|K)mgSdbuV(#uCJVOdg4(2Ii?{!ZZxk?uBhd?2q{z5~3 z)NB3tHujd3pJS@3@QX>aZma7nkZ^HHWT7kFb zgccN20*gUSMnV~08#Jf+LPBG&KT8-lWb!DT& z{nBLS*8STHS1n@ke691yw&}}P110qxwpzdeg0MxtyRCWv4CxJO#@}dS;YVKq=xQz) zrh^3IU{^cn(jlk;Yvv+{T)5`1y2hOmY$YR+w(KX24Z`8S*Pvj=4+TNgP-5ig2s!#> zN-9LD2al?X9(P&($Qv&*i_>I~fylfS+tZ0`*~ZvFL~NyV8#qH{cte?3f>dwCD`zM{ zUo_ve=Ww#Qd=GIEOA=fk1Qp?tJ|9sz%c}r9W8v?sFfobXV)zXEBcS%6Sx4c0dAyts zm&w5$OJ>>(#MaCDp#N<5d|m0s76qhdoo2_b0QfEg9B${m+HrnFur(Su;_orqY{GH0 zFcijHHrsd-TD4QL8{P!8YT;b^Sj!)x*t&=$E&+Iao~!~Fb#KlAY(sq?H|+@&4ZY}; za(QiWklXO*pv8&T+V`~MFT-tjJi_Qp>C&++J9tZP8^!NV7)*5z)DDHVzM!5ytR9tl z@@$?kSf1@~@3+wSAH;6zzCCwg+g3prN6h8q27N*@4IxRPxv2P~Tq~kH$zkTNTl|12 zBBgP%)+r!x@$_)*bv>g#{ESuqnF2NMGx2f7CM58R14iHyXjq@Qi4MkL4TM;6l$(AC*6O#7@|$loeL zncrGPHn1=f(CBXV1>kaA%%^E8f=Bugl=9QR1=%p4u+$I)Ni^-Yq;11&KiP zP5>qkD+rA^1c@w!t8+zzftnnO2YM0BIUb8rzAU>Zv_96>F~}Zi06b)3xC%43Ttu;N z{deocp44uBHiig`l1{mhGf>ZJGDmb-`U5_vLNKIg5zXmMW&4fR>1sQ0Uy6|P8kp*G zy#R) zbFQ)J+K-R{CxA&30^v!5sG0J0MDZw7_udf596E^IlWVtCLzaNRVW5(zcr?Q*2&eLU-a941HOUc@PDCdwI+6^18qnzCunM<#WLyMn6a zbMQYNF`6}S5I9t`UU;Stio()xO?T=sm`7)Wd;|GG4JlkW3ur6GV zCHydXb|UfP;VO{a0m{g*8{V))Mi(#blf|&0K-Enomr!H^c`9<6;-;8|BU75ps0wg# zbJysMu@sQh156OGO5&D0zW({tVO@J&~HmX$5GVi?ajSH7E1V2n?UQ6l*1+WFLEC`+yC}mh z68A_+_B{cx1H2ZE#in2xn82jGm9+>w$7w8)YxG3TGd661G=I&O?6E{*%cgV;{tsxB zec$1v{qt?BT6#H!sdbO^8dMd6o{R>07l!(xqs3nU+x0w+XVc!pB z?4Br3zwCKuL|XARwYBo$4bxnyP1Tz$I(w$kd-A1|EMu!i41ox_9HVdw0+hm^QRS$&>6@*&36cKtF|^JYvJM1K z8dopsu0}_+#7N(Ld?O+ssy=A45tFrK!J_J|M!|h1$eB1PgSkMvKNDfffP?7u;|8Jc zm$|F^i;NqL&mV?~vzF4%7BbN^)T7OnbdSJW z)|jwT{j)5??xr$ouT^k^%duZIN3F~|zZJ=P)YoqvuLv_svg6HEDXYPZ$|sSJu)(qB zUF*0#RG8rz#{3;nI}8GRdYOLbGa!4ydZAAzqe}{iHM)D}wVm+V7wc)k7IF2M4?41- zWo!;u(>UHrkAxkW8Jl85Fl2~>bN^xLf1uIE9EM5j(v2=lE<~6=K6V0MN*Y=wSp4fC z&JjxLL*VFA^1|wM{jS<^M`Bc_o04MbduQb}}r?yjJs5|nh z2mrnw+P~?k@GTZHz(w*BY~#`K_oUVC()hwVZnw0uLkjVZWE@y-#!xMwvKqgz9<1e( zXCCk4#bdX=W0fZv2Y~PhqT#MBpZJMW^ib5g(aq*sJiuXZa?arJxYGGS0W>|A94R2? zQ4J&to8zw|@z$7$Fmy~@$jSKbZkdu@iq)e-v*V)i0hicHGci{6Frp+QYA{>qZ=K43 z=JuBYBZhFr>PF^xtJEl#LtjQb>=Q_R7P1Bo*n2nK4QOYQo+IF~noj_%I?=Ri&8RM5{2m_jb3S`sVFE_WM>Q3Y2`Lpfq+5qLr zY?-F;jXXW9z{sU<2GDjphrl_%B`1;6@p23=|D{>|w`yecvr)qDBdoNx--_~)r*>>W zF?{z!)uuQ1F_wU9nvj8(3!t`Q2UQTA*ODcAa#5v%uf3KTT1QD@pT68uZsGl{8_o-l zO4VPV-2x8m?n~cWhF+Jo74y)R6bW)jGBV;xUBokVDq#&w4D>e8nlxYQ@X5zsC{oOA z`k1{6#HA~sL&no3&Cr~80v!{MQ3HMc8w&v-rWp{SWBk#3%vlk@(|ErNWaFIo$)3cj z;h_eHO(^Z8^WP?g-bbMQ7_oF~;?v6QW>ki`m^( zLb)4T%};D~TS&IerbK|<+ATSIn9p=``ZKL^Q}{ZtB&4|+^C&+NQAKN$BI!!4s(e}k z)5e_09RKkRBb^*tlP!%fNcxAzBk8NnPZr^eg&1K8yLY1+0#T|NWS_ zoX^=Ov$fksmh{^n?$4i)dA80R0NX@+CbJ-?AK9ljNa+Q5`6mC@KA{VF$PZFAvXEtj znh+ucf|x(hF(5HeAcQl|ZtbZ*Lh+u-a0Lu&LCi@%7z5#eS_ z>i_h;kDSVmAx3OpJW@v^ z#ybYPNAz;1z+1lL(S5I`_pz77uqVVvrBFPI@bS%2E`jsOqXC^(ox~7Hy2d3&Ev0{D z3ynJ0Xudy>&1_znZ~uCvG)=wuVDq!RlF}DOEPJVG%Cuoh{WsXAEBsvVLduQ}dnqs? z$V$r%BuYNsw32z|NQZJ7F74oN_@{hGB0$ewA&|%;7Z0qJtZW}y&~8V{6EBNsZ_p3w zDybgT0kyUf#i}Z3K?!KIKRCDV%%Y1-E8Mvo_TCqe_9LzZA`@`$Q39u1T$nM$`2A28 z#0K9Q>wh2)%Je$nx)%QvLi+s=ieUHfpI&VJEfnFY=U`_EviC-plxMQ)uG|JP8Tqj| zP7XXQL`FB_CT`*JbuMCYzwu(54_q1afL_{GtQ`RZwxi}yN_0ehItywh+04yAMqR9d zOE@+DNYNYxKM}HBQA@+;(5ySxzV{_@Lm-j$#EpP8hNj93gH2-PS2n0n7=^En&7Gy@ z#m7%TIfvjP>i2ls;4ZiF>?OdNd`i#cB{z2xUnCaoH)M?57-M#X@07^qjzq&d@L~tItHJl+If?+hlWGvhK@2byM%bXAx0UXCfcFAz< zXD#CWqQ*9StsjcLsn^vdNW)5-K{xZEV#u-x97yAL!%{h-V`8}bRi(>cOdnli6si{} zm30r3gWH>{buJIzqxPss%rlaQ(Q}eOB>v8O-W&}gsm|H9Do{wdxDN0k3%F*^ZsWA- zj#It>@*97FX%zlDC`bsLp$~##7eitv{;xS5&E&}D#Z(0n1vYMJ{RWNjH56e3kiZQ_&7Lx*>xL6db>D8}{1WzZ8W&rY48>^SyKh|e3>R=n zrwe(-bqNTDa&B}}Q(@3ZaHCXI^1Q#uWyglmT4a>Po7mtqcu zx%Ut#Ry+$gXQiutrb-*nH}MTQ=D)Xlx51zGBarJ*98{i>C7%^cWvMI%vbbIU^UL3j z0RiRk?{sMcE})kSe{p7gc|XBiVJn{z>v6y7y_X&Qn(;j`dW;1{DCkhF+Y{~Az39s} z?CB}uBNwJ?g|%_qYW_|ys121kpcnm;&1cVuaq(!>!Ji~aUBZjo#xuopl# zCB}}}{pot{bu+H1LJSJwVQek^RmleMSGl}3GwZh=avymf(7!QP3#W{hyG zk)oz&SYz);I2z5dx3y#FtMQFn=s`fE#aT7PdS+dG7g@EVB?lTE19jXDA#MQuJuOR} z#!_yM{5vMM=P}0x0v=bgwJ8Oz@TJPrSIy-0V0Me9{9*qV8WpwY0bRt^ zI46gJV3a`cL&^f1ZIPLJajgG^U!kHE*5qF5WbTD8wJBgCY;jfvDkbBObdDd7c7pDlaz$1AH8p#yUZ`>XK3kqe12Krqtlpzbr;!8rrNP^4?EhA06 z=+rSCSrT{OQpj@+D#(W|=rW|{W}UumK zdTY5G*2E`Y^-0z`qYm81{6 zp%@vnfn!c!SlwXSWc`emj%QCvKvPefVS?Jxw}6VE{Hb1j17Q~5-Ujlzbmx!Pbw~{8 z?BN*DMeF4#Tx6`$DzvWiBM|ehhNCd(YV}Gz9**_&n#>5-Cf=r^KatIV>e9vi8R)Ar z`UL=8pg^Jw--mrr4>y`f^?*n>tEhrI#knTA=`5~~!@b-4-H+Rt+HY+P%KHKpHw$XO zSp+|NV^>=G3Y}>M7RI>k`esh|M&nV4Qdel!#;ReenPOFPumWi_IRc4ZMbdapIJJGi z0lRt0)B4&~06au{`U}fAZ}+EFpjo_B2e|`Obe#aVixnP7QVhU*G?5#%qa1dJtf4{e zU&ZA6tYMPET5jD+lN)vpU#ny{5!8+ykZMnBnJORI-Z-8!*N1BfmmK_(8nTFB<|w(v0&WdoR?8a!wQgvZnrqrNS~}ZWxCqH(HQ(Q(T3Mc~pk4Q4ScA{9Lu&4Rv2 z_V%ycz!M`C@xtqG`vPaFfu{oadZGpMVUT4Dtw%Ttf1J-OijdE??Obe~$9Yl|dw=&4 z=v{Pung^sBbG(oyhfOi9t4vM@l0#`kh{i1>WQ9G}G}iteDn}@WcQVN}h;0QtJ@gXp#i3|>Eo_x>61FnpKvR(gtpU>H zS!bgZw3JNu=Qajiw+fZiQB%(puaoBBQ_Y%9C)4@^Vp3x-UyB4O4=3?@=lv;wJY6)A zkc9!JJZ3t4{%)Y4n9v;qWe%w4y(Z9El}eGyIjNCd!7;10j#C^$>|2F!ALs9De%XJ=boAbiZVC`IW!fU(`=?GMuVJ8 zSwpyb3unKyFH-P($`Xhm+cL+htd)~qeJZ7Nb>>!a8rEX6A{5j!1atoU_jEV-v}x@~ zJC}g}E~>V*Apk`@;v?9yt8*SGg9d{ST~+h|k#aF!&5rPAn?=?W_wPU%*N5H%NL}^Z z*8>6NGu6EX4wvH=rn*iwfN#_F=c(d#%^=C>k}b|gJPmn+OLQ#_WhxkIWZvfIA}*il z>gk>fR&3AG56Pi}tyotF_F4G`iEu(bfq%N{}0x9Jev z&E$Et7yBvJ_vt~g=?dUTY1?vxiDpxXOnum0RgnQ{IzxI$04aj9E=;P71 zfKEh(8~70*K+Ris!LS4>7qyekPG*)|Pvd^iaklVMepZ5@Lz~crZY(4-3C6V1Tn2bm zh*h8FsCwrBRBAk)KumgW!Y#Gp?`Sgm=Y|zWGkKMdJyT#u%*&FZgLyR9+#l48H9t#6 z^*Y&XW5L0n&oV>3h(l`NNvh2itt_mxNISe2mWR-k)qZroT*W-HT%D|V0Tgzbx)0E7 zq;5@lS@%cbPo5SjCC)mCST;~t4YCL82!kI1quKAA@(cHMzrWbVK-ez*h>(4imBM1C zBzacZ`K?kwFx9&3^%BNjtJ^ad!?*r97tZMSbY zqDs!1U^;@bs#?+zRco#yzE3*6+VR}kiSZON5Nw++cNty5KYXUb_{6Yi?3#0g+wqCrD%fAYN+Q% zEvcfv)a;HL+EmpPW7eX80RDqm56CE>KRljBtzP{2>c1o*x2j~eiQ>BKkqdm)+!Px# zFz8Jf^;E#v@loGS!Wm|(w?ByNlzyUwSZN%2qQZU(B21%94@`h^Ng=g^HE$bZmG z1g1&nweBv8RYFpwv5QYtC2Hs*$hzsNHu(L_`&g%2)W z;}gD40kqTd&+xbNMkq1R*yMXt`G~huN&3FcgwQeCh6NEg2(YoPZosU^*1+!^UKk94 zzeg+!{PsX8fG$!kM)VLOQJtDI)b_4A)t5U^e7WJFBXVCI1MaDkQ=Z4DB;^ z1n`<_P79l8*Ij`h^u8i%$rMEi4vVQhqm*yfdfGW4Ce#6Bcx4B(J`Y#KUay>Kyp<6W zxQqvkv9`V&X@+-jNYiiaK)mcrcE6ypZHlq1GXF7)z}23|6?w5pRZ@4+f^poE|UUPPA9MeE|2#Z9W8{hqx~Fb5T81P-0K zWknH+Rfu~{fFN$kE1ZqKh*=^uj00!*1W#hcI5%eK8!GnDFx1!XINM|(x5|(&m0xee zlrvT#ejg+%S7?A~IV+yR?FwBj)IQMXQ}*}GvF%Dt+2@8t4;IX2EjZ;Po3b(f5fkTH zJQv#oRS-PQ&g*y^0;272&(bMIDXYEZ;p^ii^?DZa_1prA=|z!wF9oM!0N< z=~SFEs~c&+QcQm|Jqg=a^^pLKWAvkpD3P+je?JXJxLhOK8K>Xypi=sl&DPuu@&{{} zPN}Y%*b80=ufGVC}g zDK!sYck|%n$wLUt3;KcKn-Ux+q;>K_he8?n1J#jX;1^D|0_FIpA%UkJZOWP4&X-w8f^)`bHi?DXo=7!$> zVB(u?3e`HS>ywpA_O`+3EEEFzB*4fMHzP1RLH3fNlz0w|gxGECod7-9O?~NRXv>;v z+KMKKW1X~KW8H3S3qi;8!=VlqOXODtQ1clvHYXic4KBa8JfmxEnhLxMaOIv^+Ah|X_v&xa1K4t_yU<9v@w02q-A%9Lc9 zIF^M0*xJ8Trytw3ZWU|X1{RowT)L#K0wledw!O*~VQkkyJNiXALTgGy-&&t;mTmzV zRjfK~xdj8z#=2Q@u*te-=FrK<>!90c;B8}eJd5eA)cSLM9*)(NzgzyciEYmqG3~b?;5>UTj};X}(DfMNpSKdcMGOO~I2msQkzY9m96M?MRv7PV&+ROyisbgNN!H zfmAttokM`0g+ntVPG;=zn$YKJQe~jU_*~+^@(vhMi`m6AXEF<(#?_8v5&$$eO{Dmr zi*{GzT$S&cOuIn7A+9SfbN;I0AJ+&tY?puGRs$mfK>LJeq&w@@ae>*JfJ`HeUUz1s zv21W@PrZbYSlPZb{A-CU*`f?ZFk!<;jWnw2qoNd-Y)kk|_2QNSI(NFSqVdvv zwx}VAN22OE6l9(XBB3zX^jnTcc2Cvs=}_Oq53`2#nY`%AITBVTffP_N&*zFq#U||n zaw*%EeF`mP(_Y!;=oPK z6EHq|S2}V8;a7!e)MzPQX6L*QE2J_PFB9T*yqTYHgGbx(k?8H62b!!w@;)+muE{@Q zS@L_>W4l0jqM*jzTJABvTOEVQ699u~Cz?^9~N(*ySmuJ2%x^Qd2^?h0JY zY!}o?UjDAVP_4&r=#euH8FU9tBKiOWFdOnYnXeS)cW8%JiDqlvdvRHoKyE;Nv4lK; zCGs>fTEZ#&!$DvZ;qXg-`jcYG6@P&02c0Di-4N87X`{%r2HIgUZSu{&j?8!$4ti+C zPdeV`yRdL@O;*0_9gbjUx(Da~l+ zT-a0=N;?0R_@T^UvmyzX!=_g01D3pr8_Ec|>GY7T2%sxfbWGzR49*Rs%!WoTof+y! z%%1=-x2MGrDk?upeh?s8f2Z=j(Aeb=;3pLQEEmT#EGF?^w}S?Zp$|-wt6-*BEt`M# zV1ZF=ZnJ2n)7io0z*?DY8uX;EIo>uQZ=HXK+jo%6Ln~y#=d-9xEWLEdNWIoC`!_T_45$2~w=e@G0f)mXv7M-ED zs^5t1UURVDW!hRaTTO}fZv@VK6m>VqD{tZgp{m(6HKX3=ekn1ekHk6tMg!4S49e5s z9jd=I9@BWR$nC1J7nkT$%f~*S&mz!8s|Nb|2vXVxUIRx@%SLk7`|qT&{Zm>Db$h!s69) zdD)gX%jB}{34t_y0p@(A^&Gu`+5D+|>Eekmk!G+!r(KOYgl7mkjGpDNC`DV{a}mEhW+@ z;P_9COY}PP`6~eeFORi%mAnsoxn24=%}W!om2GYFh~b|ab?VIRJ3GoG-F|+31GFZe zwe8x;=ZatZ$f}4NH}M)J2r%$D*R3^OBNc3%rSP^)R?BxALy<8f-%B@8hqQ14Q1eR} z(CBDKkiAs%6RN_#S>co#VS(!oaz3h&?9nxqmYb+s2H*&>Qr$BJ6CFSyz#<$XYSCep zFm^u}Sjr3i5_k?qNo#mn%7D$AC%$#>^|_6&HbQVESMm!t%kU6$jr4ytXl#*#VQX|{ zkyQV~>+rDjI1N>M+I`lNPg+_3ubA_UYO2}#I4CGYKv1NkA#{*P4Nas4Na#qDru0sv z2?7cTND%|l3010e=^#Z!iu4XrEeIl2dXsVoxIWLvcim6#m#m!RoS8j)&8)L$&wu}l z#@LMB-~@OKdupE66gowbh>`G*XqaA96f*p3UU4+o$fF2f;v6ltDO_ucCo3Bm=dhp; z=^_OcVOhRJ72(^2vC&)$LZ#rKc0;n#u}34nE*w=QyF%g7ESpIAyn;mTi58VKP;SC9 zu2A=#DA5_emZBT1$u(}oAoEsbJ^sLo*&b$S_HW{kkV)24c8rv_Vg)jnlR(04cCxgU z>q3#kcTXMl?(2PSq0ZyFkAItEhYsXgRw@-9+R&_j{^p0;J@hsyC5jPXY2@0a9nT!~ z)LRm+fPiC(FK1;=_$ z`ZU=es{^Ni)JVl>NXydq>~NzEZpu6+C5;B%`?g?7S}RR3Xf5u>z;45Go<} z=ti=QUIxn|zC?-9=cBuRsDgVXCsLEb&QC#%Yy}Fka*#caAFE9MO+NJvV!)!t+TTmV z_2N7Bhkth1y`m0XQ3M{+Gvc={dBF#dGkvlkUm)#Z)qf#$@)9$rdOKmhJ!C|wkxVDd z_NLT{FhV5fdV$ddov=Pa>gv2Drl8bOj9cCIbLlrqJA+JgejUzrNY@pK7`t<6jH8w? zSLlUNa5ogZpRi@YTZ`TiTrIf^rb4rZI zJvPy=E?5k=X@$Lsy^7da81H|?%=6ngy_NJhE$p#FOKqmbAwx0u6b;G!nOwmTAXrqG z1p4Kp`c#AiT?l1i`(19zm4|Fg@2r;H+8pE=M%QJ)c0qgEP(*$G%{B~O$l9~*Q`)~S!EZN*v=F2s{Qym@`u&O+DuT(|iTOlXOS?~OR|aEWFeAG@b-g4iI&z&g8I z`^hx)HiNdUc>Y$kM@(`V*)5Z3nnaOGl5lQAFF)42~ zQ|Qz*9+K5M)4jq`xR#(RHqFP?aFkOvgpEeMuFpOSssR&wb)w?>LG)KWnKl)-2AR~` z+eKeJRX+PeN1AIMEe6Orh`2HKJ%EeP=-|Z5%bh{SBR@u*I*r*mp-mhX(J33+zj?lV zrAyS5A6LNl=z`xUuU|Fu{aG%0dBM>-&xb~-A*JtU^T!Q)3!F5K6y7Rkq;=VL$$P2i zQ>X@xb{3S1k#=e@NN-JC73jR%L9S&+s5p!rzV>09%0_b*&PTf)b1a@TrTWGbwv71$xvsPo|J1-$q#cRVg2I8X#w&!IUOP z-Y=?)Eq#G{w0q$BKF;#@m3+3JpKhwZ89|6x%2&#_XY!>QmEVj^Qq4#m!WJvd_?uuy z+(b^iWAXGFhr}+dxux z8@5q!a3$ua$@T6V1+Y?D{gJ4-EV%OV$tCrM+sUsPs9XJv=O?Iw{tTqEDQSTaC; zh_x4;@)w( zAv>IYMrxtR?ZyIuW=~4G>r=PAJUY_aZ8tK(YR6*G#Ak0gYM8KUG^)x|<7NrEbLOsz znHC#WF{*~ayvWI{zp_I-z`Wqt%uh=Cc0;K|w51z}EkerlCfh zGcsL?1Ep=FBYgm5+r38ThK@29$h=y@KmUGclwPyeO0O->=#132{<#RfLKudU)&$x0 z)RP0#^tUV3yGx^%14P!ERq^L8--$!U6vC@pxYVsyk;6$GNWb9d!f9RS)iS2=Zs;{< zx;=A9H)L?!kIh_3T1M6C`?}v=yuTXLVq+{ZZVppN99K32}TymaUpDNE_e6 zxGyT0?vZ}(WDPEAq25qa!X6FXX6~Wb69XmLd>ekKoo!xtv1zd7n~N$uL>ON4TG&lo+;%oHx0($5m2j}eiK-G)veV}lSrS|ssf*Pi4IXP zkXUCtFAmboSbClQ{DcdVaVo6o_-{L5v+4ZfAJGno1B;C7YvSD^2pA#_dU8h4K+q#< zl49uhcG>7;2@B*pz1ighC)%n8=!8rSyJVTNi;; z40+pUFGfqe&!AJ5=A3ktXPVT7F>~8uCtd@@B8g0;R0@)=)4_z^Sx<-LMgW&>~<{itDb)!PTS12gd^ob!{h(WDhRidgX%53q(C4Zzltp zMybwj5QC?W9^gpE@&pC_+f7|+>Kh$dQj{%9VINV#JUA_W3T(jPo$x`7K;p|lecrKi z{R!K90zi5p$aOb4G>%)w+;u;XGRGCR@JOGPQUo$^nd`Tp-dk<#5Z0BSYE^`lFe`?0 zxfv2|JARlne1Cu?E7@3VsbGLK8Gm`|)t@BRCtR^Ee$WwOEep?YAClz?Yu#23HdTa& zOoJ{Pnmhztn;m<~#LNW&x)i#(;3v$M`?o}4I4~4S#S`^xr}xIiBr1Or8Wpy(7SjrS zhk#+*lKAdyI-}xhhze8(#Diot*lVFt(J&-i5Ieb&Vn5DMj3UFqL%v4Sh{Gi=u|3}g%-)KF>h5!ZpUYjH}nPb_~suX(*QHq@POq;Or7k9W^eI z8hMA?AXW-D-TG(47t|{{K1plm@)Al9QB%$(N_e>~w$y&+7+1KL%Xr6YQKcz}KGv0K7KMmnu%GKTo!t-bYmLoG^j1B_}sM?trZYId2YqQN!MVJLfMgIxEAII96^^)k-qx8mIUFnBjrW zmT|*FENiPcp0qN=7Ox9ov_fUY>l&jWQIh#4AzfQJ+zbb(3J-zY*Uj7m)2qz`o^knF zeHzoc(;(z@cK4D#N2*4?w7cLEu)+vsqUoHM4c=d)%`nQyGWfI|D`9) z9jWg$%@7eAKi_+@wVf3(f$EWuODlih}s>447Ccu}(f@q-Y;^p=UjG~GGba2xriD`k~Jn7m0dI{4S#;GsTxrC}v9M@84P=Df0} zQ9uo^S7zA_cq~kH3^hxooqQKn8urrVz?NVp>y>8_fnnACoj)$he^gk*DUS}KIW;+G za(l~|6*4m|)1W$4q(FPWss=7CoJaIDRi>+;XCUU+IbzOw&rK zv_QDce8=Rxsph5WFk)D8_I%~cmb7S(|DP?!p{j>-vHU?Y93Fd|bxDs8v^Xv9w>xVHn&R^J~4D(PJ zKB-yq!P}e*qwAM;v`UYFuBXSNk+NFp^4syS;SMK{^ZiK03R*VmotWFi84|CX^x%nM zf2M?ZL^W4mvT*8QAib=L!hx|>tyr}^FVc+mX-5y)h-+xw_h48h#gz$8 zgT|=~gd|P<^1HtvRrv9Y{enddM5Y?WSiydKg!}x!^R2Bx0Nc@26=kI%8Tp(Gj3b8K z;HQLukC(I0pbJR=f`@)1fI>v6vi>N!$8v@2xy&Yd4ChMsTJnNynj=fXt=33ID_)OI z!Eag<77B0682uS%E{aPIl^}8Hv*Y@3yOIG0MT?Ek8j8o8OFyepY4N8(8luLPC)%2)@eB=JXvSHsLoAwDPCgN;;!-vBtnB_N@C)`0T z(%=Ff$MY2TdOl5sMO(yg)MtB zCOJaD90EoEmH~P{TCYgIfB?b^0Gf0Y)wYsS#pC;)!srww%|TQ97;KL?p-3T0DA0H{16*U32l^B4KY0;j8iH_qXmfD`&&nrG<9zfh#)vWr z060%DiUi(mIZkHfgw8aKCy-1$yae*0YW&mg@XDWo9%vO!{_O$ZYo1Pcg`*O5c-Hjm zR?;)WrBH}g&`@aF-ygdF?PGkG2^@bh2h$1vHDJE@k`U(1ZoI-`+sW#WHY8+c4XL2u z*1|EA$jg6C?287fP2{(rzt6slFLcGydolbw9KC-mCTc@UWF7WVeEfDcyuO#<*=mg+ z7tWlW4V=y9_m3H@(ibyYRY(lT|Lbn~0nF9Ez2fP&@7h~Ny~B%YpH}>5l)yiyUOVCO zfbak3<*7%(U8l^GQwr(nL;ug0XY&UjkWOh&|0}5Wf1=v{FE^h~W8j}Ntm=Pf$3K@j bvBwviQ`cT2HS$iFn|NqBRb-KjvEP3H=0B5| literal 40324 zcmZsDWmp_-5-sk|;KAK3xVyW%2X}V~?k>SCxCVFE;1--f2n2!$x7*pfcQ^U={@{6- z8M^!JdaJ9>Idz68Wko4u1bhTAFfe2pX>nCBFi2xCFmM_;7~ma2nB-XC1+%r7n6iwR z7>Tlr<2!3R3otMmEHh(cHYYb{7Yhty=X5b_T(qyzjj$Nbs@zI!F58FxsmT*$_42NfPMW3(BF&k5JJFN=vY**;!cd!B$uI zDWQd(M|gx=q*XaHyTON;5a3;4qKUy9@8?N>(vx^#gg!$JF+uo1lz&H*_UFx<8Ui<^ z7Wu+P;vr)LJ!S}V@Dwsp^!ZmT1~fS&Bqa1`>Qjk&*v`am^5TXm^i0>cr=pOjqM_tP zP=Rjbqi_c<6e)rT=|>YQYwJRI>{2rDXxK>Se8~QxK6K3XY?~F-? zq%v8hLW27PX(1sei9<7Gm>~xt2w`AeD&SZ-#9&~+{b9oWUBL$HlpTLkRyP3=gm;qG zaRmdzZgPTre;=aRcgOetxettBHbeY~hzoKEF2}9VVN-Cz zQt?N3IpWHc%XyW`VsZniYt^EzMrK+^|C&pDU;C1m(!Gf^`UZ~^U27V_bSDUl$#jW2 z!|wA5`npsn41WLr1WULMSpC9r?nR;g{rx>1bc}dXD&@XP1qJQ8a(9Wm%t@O2#Bqto z#ML~&S$|9KyzeCQtm=pgT7itOm5d_X^p z=Q{=Sr=YB$NtW@KmO2ufkdvw~qFa}vcu#+Fay@uW`Mz%?jzDfi+&le00SVehH%83d z-eh~l&~AJuBV}2-qB@du@_s!{AHv()o6_@=-}S%uu=4vJu21G7oJwUkIa6iRS!A*r zHIpe;gn#hMFz=U=FUg$HC`C=v9+xvvR%XIr7+QgI%sn@E8-qV`7m;C#sobxI=)XizB z95$}6&$q>A*8Gi)oU$@*#*@w;2 z+wJ~B-!|W2;fS5=TijT~=6J?iYx!qB8$j4O_(0{k5dE(k@aEM2X*AWu;S@0 zGun=?u30FAPW~sLzwoEKl3_BT6o2_&tao2*!5ug>*f9=jb01GX~!{>dWCA3V{QfbtIxIZL8aE$kvR**L#tzRD z@HO4)^1ge%-A1G}awBkYaY@X~jEush-re#&lPFWnt#{rV>FX8~%)z78OU^EIV~FRD ze{gc^;WoQ!{ww@>~D@1!?uToYss{l$_{wmXeY&0RQv_``J+V142Z`RC$nD;%mB-6lOomLcpZ z?f={xFw9wS@O&C$ty=xyIjf=XQ`s7DA#_?I>M_%8t^NKqMzuvpIqqNmPVwGwcR@xZ zyoP7DIz*Sby1EVr;4o+hk&1*Uwp;K*r+4@V)A^ua)o()w`GG{9 zTi(`UpB`A)NPZR5Hv0c{XAs!!`{h9BO^}ZZ@ zv5hoM-XSLhI&$&1qiin6U8C0roaB_0ectJ|gdp>%;G7~0Qz|)M2qzT)#p3`y}ZDzc?;-$Ywn}%P5D}S+3@OjsI7qrjO0YA%v{-1hugiq zQ_|5cI1I;7dFq~2#2TSSQ%gfD zM>K-7+es-m#J`^=^*M*fHF|;4X!3u$o)ZE10k(d-Nn5F0Ml=DvoB zMYlTjlH8lLgVLT7-XG<5a^AWJQc=?jBb$>M%s7)zO5>Em%wDN757TtV+%tcyMgZ;n z0|ck%slIp5{)A>9spV=%61e|7qkEjMh|Dx6wQ*b>9pfj%LLw$&`XtBH4wK zR&K)MEmGPzT}5?S*RKZZ=*6v$+_&*=O`ZZP?BGB6hA95%;00U|h7CdX8*M5aXCLEE zwbC{X3JTsR4Cs;Gk0~}B15Y4c)2;UYMB1i+deouPJ&P^9k}Ni*YHZ0>YDRKut9uFW zgSHX%HLcEcIkU_~%CyY0+-RePY@wtybVYg9UK~NI)9EgU4%He}p==tRi;hxOR~LVx zg^KJ_rN;X$1Fv!Di_cl?$zyQv@JUxWCjadYL3waq65#9YS1n}Xn7yA=;VL&Z7WR-% z;XAc7`>X`U9DAR`6?1vL{WgU0g;g`@G#9r(dd#j4O9lA5Mz^uHw$2A$M^9K!W{a&Z zTr>PvNP-WCLFRwn59{^tqinyChma5KpGl;@VyxEgddZ zW!U6KjzaH%|L<`K5QZv!skV{Tx5p}DyEk?{`^*~o+kvkFx=mKW07R-d`u`Nda~SYC z8e@=zf^0?|wzyWx5x;ZSyB;#}mCkOJ`o`CsoHt0gEMN%&H@K;_NfVqo|LE925AX^J z(gM{}DSQ}f%%2fpJP^ogAR6TrUHiacJKN>$TCLqMkYehz{dc$cjbgKY9~bzvM|>HC zb2UBawqK-wqyGI{mYK^6Xxwo$o-mWwoescx+BhCx2cY?{7uCFWqo@eylt8V%3-c zZBtd#^&c$<*t;Fg-*Fs+Sgtn_(>aH%;B(pwmtgATbLHPFFt(UX>(p={Bf!Ip+U)e) z3w(Vfo=9h;Z{-Pm;eWU}i3xaq2)@ku-@Q4d+QEX34Sl)oePx^s*ta3NAzY)%*vx8x zi{6~>KHg`d*Qg8)d_FdE$zrpZNbNta>x~!iy(s|td!;MAR;~D_cT>V9#{ek(4=(&S z5!jIFvqkuF{&i4-OIYS^^O0-U?j~P#&VJOsx3-^h0lK{xom{fO=laM+C(C-Zkuu=< zT1)=JTN(9<5?27l9WIoiKHQ$6>QawH{avSjhXMt{P_2(nt2Q=^-Ex9$yUiQ&HKPqK zqpppa;W>CvXm4k3^Afk&b_F|)zI{M4d;_gaK8uXb6z09ilIi{B{_Yw}bmrLqo>y7_ z_j#q2U7&w%cD6EHX?%jO!Vnw?+xX;Spu#Q#+qzq3w6{M=d--)B3=#d!8-^8Yi_^|C z!8yu{<`qLVok6QQmNj;}oxb1#fg$c6F?~jX67f9HiVCeXqp#ZpwoA01_A z&s>;#bY4m0^SDVg+TFTdRydyO{docub*c3|<$7+WxVDD?tU{KNJFXn*)8VjEYhX23 z6n2zMDVHAUQfJVqsX;szrq}8KpRI$#vb3Ood$x`0Vr$T1mz6hUs`K}z2T_67(Hnbi zy5j`geTAjrvYc+!ml|lHymQ!t$l=ZYNL%lAS?Bi&cWkff>>UE>!L97l?nUqGGc%QZ zW!4eyo17G zMV)sj{03BQ^liKoX^aWK@j~T2n+sDOW?ED1XLW6dZ$Ph=QcbtYT?H;xUCAsPTy3&Y zrUX0AZ8b~shSyE{^m|oZ&%j58?0VyVh+l_uVzmdcmT+%?TB+WG&8QoiTyWVA?fQ1lL>9gA9uvzR&XVhbHKT_{{zFAkU-7lmGhebIHqF0PES9yH8nwbMQSgVWu z!bjwr(@lDqD=&bHggs9>L_X&W`s2So9SE^{og07tz=23gO6owUg_?Um|I8vJL=1ZR zmZBtx-b;PsrCgB4ZaMvhZ|eH*N+hfW)pZ(Do+EHFdObqmNJ2>|fplN%cB|I1FtGr6 z=X=&oee=~}Uk<<3@BXsL;siivlH%f_>WoCC9QN=)^n92=al(?4Ff?cro1WW@4$%s- z9qyH?r3jJMg>$gJ0SH%*&u!_&as*l(EEXn;-L~(n?A=H710-ZeZuFLUBMt)9eb@M`>PLKOS9rI6Soax!#KUdOs7WT&WJ;@%gj7+u0T&0IB&eFE8~^ zG+R%)n_*%%@r?7d{3TDd|%0Dj_gmP*v@9-)`A%{)02g3$H*SG{F%@ z>a;o-#1e3!09o{Y+#F-YU=Ff%o=p<g3YDEk(RPQ9=5tcSkVKn(_? zkTc)^CJR>ppa^jja@>v)V9*@i$2G0Ox2YREGVtUU@>!={SBgl4yz1(Hw7irxqgC1s zMUC(W|KK;oH5NW948xwP6ibemz1;1g`u&=da#_{Nf+g|-10m$1p zsA1Sb;bg=L{6uFqbmrBXi_WSaV7?IMPz8oZ}GE*_0_&RQ7pxoY@*`C2*NdqiYF5 zFL0_8R#$eVX=+QEh<`sPB*JiaaS)QIKS2(Lve+@hj08+_&+W%w8qB0t{o{+^WTVLV zlaC2?#%uN4$uwXWRa^)tY;kIWi#c4DqXs5ZIIe(X9XeX7nrpVzN?V%k5XksD4}?k3 z&k3Mo3p|gP6_9S3HiM7}ICmn50}`VX;`bmg`Z4p3s0Uqt8W^ zf4+yL(=uicz&B#?$#g~u+kE$PbOcq=TP8t4B5K7PG_K~4&+FWOch2Bs$-X)eO9ziR zG`VeNs+CeL0hfccgaIL+=P85k;23^MQ@9GnXl5oZYsl_Wm6ki@CskZk+#hE2gnSjP538Y8pLW>T*Eg6DO*L>pt4fG z=H%pvl6f92!vE$ax3<}W{v8tt+U2ZTf5XjS4DsJr>EL>O+Y8kxt@n zCM1zdMQ2;8ktCPp`Ai&Vn?HWQN%)fo0eIvM`neo*Y|(y_C`swn!A#Y_qTIC)-+Z1X zA~wTs@k^wiro`>)yMnV2tF9P;ppsKWq%@g1HFbzd^xU9GvwR; zzRBlI*1O#pW_3#%V`!a0>8};ag&{{dTnIWkIz@5^mihivl5|ZRRAC#`a$gEW-p^KQ z(68c4$`VlW${Nx0&PxHRS8>b4)dbrORvW@l9PIf1D~%m)Q8Lx1j2t8bSBq7Ra76+ zomK=~aQ#jX##qVX&yAjETXz7;`PL|8j)8$8-a4dKrdTml%Apy9%_s^KW{vASx~E?Y zO1ruoyJdcf8u2*j=)u69U@HmUQ-r*KFR>pvy)ej3DbtXgN(Hb?YKz>(>4z?LO(Ge* z#D{v0X;@-E5_)4dLTV~uD4(03v@frHZ~s6o0Q(F05kezUQts2x(4dNLyIg(IITcUK z|JeT?R>C%!ADvVz8?)7M+c1{Q>-oWbpSE?M*|3WS2BHwTmezDH8jBu**2rAv=Sp4v zVSJnA3=t9mClb(+6(>n*$khx1zT z5O6p1bUwX{I@O=AA-feQ4U?!`xk>_k2t&j%-Ry8LP>3xRU(Uy4f49@~csf|u`$DII z^C<0)jK?Pa#umUH+`VQ zb0x%7#(+ab)xzfaGov@rXuc@RagnuC!P(o$J2>YDJXCmneR+NycGs8@xXrO8{84aS zoFGV7YgTL(($wCNDlKBOQcE}zPly z%tO}hn>E|YGq{tlA8(r+w*}=hSwzlE8~y$$&_Ub4#-h?153d*nBBzW~Y|R5wVAOFc z%~3%wCBnhWl9o}S<91I`XFkX~vD&1m47+pwL`kFxR1C6!~UZ@ z0)5z68Gk#!Y%~U~NehTY@7b$UFKS*D*3gLP8>VL+Ltmtg>3( zCryo(yz>p6wbOqI)hs)Bucrt_8vtpbx|=iR0R$^yAwl=Jm&Bj~T51SUKB_o|KeycUYVNBvXmDz|;Al>1=XrERwS$hicD1$~Zt#cp$)m z9!R7+DSn(@BJOyxu&_{CWJFY@UO`5u3epcvm8VsSbkfb2&}1p)Gtt%kaQIOh06Rm& z!zKX5oTt2O0JxDU0m8P@qo2w*>Fx;JjIh=TX{Kb&C4KY%3J||JW(Or23j%IU%m<)Y zA+6bQnr*Z>yR7E3gKY8{jFbefQ*H2p0(&oVU1cbenB}wBpgMGmM*^ zRrY3T;*9;?4hVDJo9S-JMf5#Dr8CWZJXvpzIgOQo-yyvomHH=5jFuuP0I`U&`rg>_ z_z(tz8& zFa!UG4kt6|j>#VtqYvsSR=AJW7-VpoWto@c2x(8$Wj>D4{xc!#=TE!2?nVI0nSSdE z4TVw|js&FuAXY`^YV^MG^BxT>3QG+6mcYHBKlL%96J z-dGa3^WkivXpw5B|61(*xABysfu#yH>ULxa=@RKEkfvc)xsfjB&CR;wdkBxV8ww7837!B$0k z0O&rUnAD0S0M-Ho5vdB*>CIYjNSH_fsc)o4eOFv&aT#?R!5^{hOa1kiehMriTB)y{ z`e8R=FNH`tIFrLRou9LPF^)(;mx-PoWPQGuAcSpj2UJ$%HL(~@`uNybP(puyf6>6N zyS)T6U~bXo^flztCBnN=bdHzNg`lMV97L42>`RL|z7Aka=ejzeYRG4?N$g8j*d5at zRcSJ+I6q!aGOLuzVzu92AFm8wJ2x5W+SpX;&-`ytlt>D&SO%mVJe7o=o8(;%rgOC$ zOvCgA{q9(HcX!42j{yA|EiL_%<;=uE8laC%M=}Mx0s@bSEPhz0kjuBoMwvo>2_}kG zk!W*i>~{P07AzaFe)gBE{MSfuj)12Nsl;ThkMdcfC()`RY{^)4QVB$rlb$DQVUMSs zsAt``L=IbB6}+R+|Fd0UBs;*$B*AIu)XPaQOA=4p+uMug%~)`SYoltzZD(d?Mt6r} zBoOhgEjhFIU&gzYfZ%j@zWbaB-dL|w25^Zef92E>&uxF4#bwN>4ZPR0*B4Yo93~K> z?>sIAJ2d@|*~0hbPcqjpF5luZpZ66Ta{hW^F~Dy#bxO9}e7X+ue8nwdZib7*PyKQQqfQ*15JU*cB^Oj%{Q5~{USOSpY#W;0- z)Bz z?ER_C{ti))C0r^TbnI*r!48g*AGiy7iFu7*Gfrx4;Q*)ZNvK%`-w+k&Yr8k%7g8D% z#866OzOdp{D`qAa&!&vA&4g4Y7qPCZX}%MVD=&X-?Mz&iBr>L+r_0F#WAfBmgU)bP zvysT-8LK#gn!p9$U6x-yy2c;Q?jd?vhJ0^fO8G3ylkw?YTX7LHQs1Nc@*UC)CobangAJKSxsNO)BZK7&I#eHRoQMz_nFT_j8z7;wbTA8lE(T*NcJU1czl z-Rd*BFde|Td$$FQ$I1=dVEr~3{6FqRBo;;*KI}+wOtgBtoMht zM<52NgHwJVm4uG^4_igRq(SeEdb~Noygg0+s1(fk9r5)2utrbG3mu~DMu>0FR!GC) zz&1FqRX($-6(Eh1ax7dYL8K{@paXF*7jMK>?eMv#GxWQVaJI*-A*eDkn&te>M0Nt9 z7~Xw*mpfRIV$GfGffRxIardwgQX8XpnBFxiinYjMokqOLD*7Gz?QUM@FRKwGkTB_?qjv)n9Ki z=w^bvkS#G?#+3kT384uk-(Hp!@D@*|^L8g=h*5ioQesjTslClYj{mS&5e7HnM%b>b z#cI2Q5}#`qR*-|3i4}YXUHaaI1!=J6l?rhryMZ3othfrK?i3GXT`M}A7xYhn!GmtU zB}WQSFu427iiyDS!9UL*lufou>2>>XDCP??I$ScuouQ7NpF2@dP(&`*uNe9&G=(H> zWH1}S*o$`CEYT%$R2vGAV!;1SH8+tyJKi|6$CL((V^nE<4xk=%piKs&jYk>@=dVwF zr`R_(w++~F1&dQ+2X93id-Yun?VhGfMZS{~_fT^%w z@P*$6vcl;fsOi4|Somc8EB|txYxSJrAzIdvqI_l0Jju^)aiPh=p63AZ+5@ zTTCZRaLh&rM@XBp-Vfy^Ug-@BkyU_tLPHiDq}ewn+UR@hfTsX!l47AmZGwlDx(Adk z<~*YLz2_7TyKiGj+)g`Syl#gD_-BCd3xbSGlq35cetz-3oVZQf12gUF0=F=ZFz0h_ zA!k;%Fe#SP%04EgTC}|u`GPU1&z22p^e>9?wxNh~MZlBrj8RX*_4syPRjuCl(c`D$ z)N)-pCk3JvbnJ2;+>My(dh(SadMG{EYD2NdO^zTa)Q z&J_CIo^rb%rK(T;2CLCBeSW;qlkX5j9O=S~YMi;Ttpr%GTp99>h%jRw?7*(>?Tx`@ zVZbNb)wl~emPVHPU|1N1{iboCC|Vy`A$<}1cP5H#tZ?x;I121XNS&-Tce6H)j?!pUf_`hXfH%Py=rmv2UCa2Xpic|xAAz2Z zJu^fo@K(QZcnk5HInR0Cz#ChW?9EJBjEUDBu*e_z-+#sN!h_>@h z26JZObhYh^KN3b0_Y?A*%~R@kc%hduR1n4mwNrXJ(Px2?~Oyh0jLYXeT&={u1{oldb;b5i65VZ2ih?YcVh2Iv4qH zo6nk={)e94xZI84v2*o);@mW9)h{qadqT*g2dT4NQ(h>2Bg6Nf>V5)ej7WK1L(askSF_CXy#XQ{x3`E1-s|EuP(bM&r@GttZNv@D;fkXR1Z8e8FYYrZ+<3; zuPm`LGAdBc5IA-f(j7OPj}i|s9lqpzE^F{OC18nv}B0L2j%#__++kN4Akd;a1Z1`9geM!EwkR*B<2DGAXZ6-}>H~+W%F5{C&^_*) z1@}fv*QD{#ouzNF?<0!UTyRPN_r+VYOSge-<8T$tagd^l|g8{v`{`5r8TMs)E^ z=RHn4!zFJeW}@3lQaWg;n_KoKqT0y)csWh{e3D#3WKq_wCw8YpJPCu={Mr9@v&=>@ z38T~PN62q=P2)_qKMAS!*RVZgeqR=QG#3ePZY`xR^E;F-NQKP;zhPV{v#p6K?!2rB zJ=kUAwK0*y*$z^ABi`;?25ii8MsW7fFFGG8*m_&%zU>{Stoq`RPJ~}dxKJtupmr+s zq|Y{A{J3?pIcu~!s=TB`V)r>>4=D3P&jm`bYz9*tKq4o%LkP9LRZFo8(T@P*mpHTv z3BBNpau%XP{SOIl>L_z1GMLpyy+oK)^2)TmIAPEa#gcIt#gy+XZ6D#+jxt6PtIZ}Z zs;^3AGlH3fN@ew8CM-}f3VQvYsZ1JvRaVdt;Dw-TMC+98xvJB^8Oh= z@kXovnd+7=P$7B1>{!~h$j2gi0z{hxNJ=v7v<6LqfR=7%x0ZbZ3{%Bt@&tx$e()X_ zyTx@EFB>rG>Msp-sC6;4J3Zs6C_cZ5Je?0=j+@4jH;ChU zj18*|r2LGRDO8CQ&Rmb4CS&F>q2`PjK{BAkfhUBe7iGk^4UpUm6C<1JRcPaPnd&aD z7ixup7DZs7mz*(t4H0c=M|vH4;R#N+Q(&F|O4|mAUG!~Gh_^DC?Uxwlo=AY?d>up; z3T@()_wG*hd>g(vy&doD>^y7hgs3}t`|@};DNQ?X0k~1W0Y;J^KYkQtS5XkO4;?XD zoEd~u7sG(OYgT6Q=vi;>1B*x1{?x##njAw$QtJ{Ip4s6J5d=-k~3_sUc+$-@C{ma(amps)C`FbGX4ma9AwFR@d zKc7$pr3V0{e||AaTnuQ15`+WHb%@wE{5G=%!Aur)y3LicwScMPNK;crnU*A0eR*Xr z_T);caPPcGf;%?55%lWYkYC^h{S>(J&|wdZ;^CAbbfi={EH!%-BZ0>7g8%}HEPwbr zE&Z%TWBRLAu-&#DF9nTp4N$(4kFYMm@#KT43H*6NDcvRTh$6hH=E= zI5pcbHlesQ5)g9IKcqCHP@5@HsDhQasg1?*LT5U_&Zkty_;*qK;-XS>npFE$@i7d5jW6hrPxn`cfHr+s$WKj$rVPap5;NQvEuq?jWv&jeQmi< zU(f^M$D`WOgV18OdR<<2bmB8@a4h{chlok@{>ufinblWJPTO6rpFwD;dQvh@g*j{M z@cWZNbL{D4KfJIn9Vty;ni>wbs|bqpq>4XL^t+-BM0xoD~G7wNV#-klsXc3X*ULVwRg?@3Di z3idAUBw=}g_keABYLQtl4dB(MNHw-9wH#|cc(^_oZTj+Z+IddTu7MB}a4~=!l+Y%W zW)wgU5=}uY`W9YZ&g9VoIF1Swvd2Zs^{Z0S(!vD&A9jEN>qr=)X3URh0Xok6-TwuTvSTz^Ky* z1&SA(8JT`R3fjK(pD56$*8_*;9fP2s>7!*+BV$uiBUDn_zuCctGN? zCeH+nycN}T2B&lRs9{bEtb%!Xcw(dB@1TPhHO$bq{Bz_Q@x!Mv1A>+d0*`r_3`XPr zHEw0XWHAuWTbAhG?~k9+4f^F=`;~mq*iAI`Zpi%3M(LfaBm@!43EVnLomj(Hr}(@1 z21EGg0q8CWPGD00Tb>Dg)+nG&8kYDGFKrk(2}N)J(|HH}t3`h`!w&4%z;bia%CD>< zt3_rZ$^k~@)G=BsZ;gi~OpvqUn1v+F$oGR&uAqPvj;K&8izYqRYqBcrZ~{yaMI3ej zEC;+4Ea2Vf(mI?l3XjtsGSwGyDT!|bab=HC6XojLSbdoSLrdWXXAlgcxXjAlo)w>22^ihBPbWe5*q{%NFS|K`x9yYFVO_P+oF3NUX2N98{cYRp==zd2do`xoQ$qhU4$Y!8FY{5pexNhjPW;2s*lcWMS* zUKgf`VRnFE^RL(&VMy!ofh!oO^dRje<)lOkCQ&|~Hsqf$iPP`?V1Loyik!s7_Ic6a*Ua}L%mJTSQ!l^~}--kvc> zsLo7Jr*~i`R0mZ!fhuM{DiB=cDVGf0Zuz2eIV2X-2;2R3#6)NF&g7j$4;MZ@rkFFl7Of~t;cf|!Z*_8FT#bdQu2do%=kjVwBtyGY=5V8(?PJYlWT55CtoYv7Ei z%JAqBq>S(di1KhB@&wvrpM0EmtXq z!RK;d_+G;#0k}Dbr^2qTu7=gJvOcgH)-WG55<-|)2AfFIYxW_` zHT0F>CG|%}&SK!B7oyPzmX`-%=f1Hu_Ratmb_j4VlCFK-sUNWIiDl>9J8o!r>a=|C z+Xq-dt4+odNh^&B;_x{J01LXr@c78aiDuG(1z-r>92tObj9scVAP9W9FZ&Y9Xsp|$ zFN8x#80Y_Z6Ra-{xJ~_)3ovC%4jUVfd6jwwkfB^W_q_ESusu3H5*KO=Bq;YiMc7*A z=}$p*N%XmBL?)Id9Yv_mbL)vJ!IK_(4%vR*L<_Td6D$Uq_@WplHIIRBkNG8?@iCFF z;{Cz0LDr3FW}H<-aTtl(r%?Cl96L<7NDgDrkHmpAdU8UK!&#BoN-0fx#}dHj=A9|5 z(&w<*VX=UvmRF9${@c97ch&{j=)!3|KNLfLr7Yj~q0jpB&y8h8D}&<}FnXh~EGJ}4 z8m3^iM@){+NPEm$?;i2IE)VDrFFZ@4D$U{{7uzyga9rG2@mpr>Z_7+YAInAQc4Co7 zb?#ljHtVohYNR7TZ;F9=9m>bEo(1gHDexy@5g_v3T!QOP?N& zj|FKqQgGn4>pet0W92*?qzCVi?e1Mruv$$an9#?jCl>68Kf~0E`pK7}tVYUf%`$w{ zV@DA&3K|r05%DDF90@Mh!*Et#xL)kP_E17(CMUMnwMyCd?rdaf(G%;`x`!WKUg`R(5ii(Q{6mP>ex5uGvUYdtUm`O;#a%`X-0CT%jY?X`#8 zKAnwM3L)9gR}%DQE48pBPGTL5|6&M}(teHKh#0+~fFLes(`uDPEb?J7f1B#@(D0%j zu+b!kA=Na?mnS4fgHIR0Elf^wS9ojKx%p3B`KcyAO%w!#l0)0Ut8R7>e8Q_5aU0p8 zlqG^;P?#PpSk6mmDSxw+e-%&`G>1>J)+&ehE{}{bd6KjS@=8OM&bY}VdK0bNmHG?v z9AA#y6HWSCdGrtm?u^}f+fi+l_Tx7U4#QCT)W+?hV!B$x;v}i$TiWPsdt-)6n4gxs z3;ZabBzNPs(%muCOVl{XqVf^a3JK`^V=fzY@%=KA8nTGd{_J z7fVL~nGweJnO}l;1K?bPELYHKH}Z-VL341`LEho)fNRoE#Q5=XW^i@0C!Od8ckYSr zTg>{9SvnfT_h0H(czsCr$HBJ_BYpP<^PCvs`=Z2Q z`9psL^W$OK=zYYPAd%o8)P^XDkP%;z@Uw1V*UkKgCLRMmsP3H>o$-_7(CV37o*O2LzK|{9iY4#LEzbz@!+`xyhZ63Wve{XaiwP?;+jM7Z zAgCmGO9}{7FWpPTWKeviN(sn1N!E&ouI2lb;4;-%=D62sO%8s3h=mLI?i!87WePD& zZanZJUV~*UDZX+bZMPh&u&T4HQOVY6U+HwG(;srYw!VwyFSr`zRawCcwsd#5hf8qm zwW%`)%ZR6Sqzb-ZTOuZ91P@$@-ZiW+lJ3_iqK zcYj&hBCL16-3CeUzF{+mvpUI0rJ4TF=x;<@hcYk1%=(dZim~-dL>f3Bnj{PQfA?t8^MNE+W+g1BJK{m^r{*-uO)cDmEVY`hy(K0?~CY(%8ER zfw<3@*4!UHgfBJ~OJjb?;iF<{cLHxcyR=8?H!&$U>W-@4Ruy||jUD_E`Ix*~5lhI0 z2U~Qu%*LjsTzPd9v5#SFYI-PYaZp+?m4Qvrd-n}n81u%-89@V9k5SxlvCJS`Q&%t6 z2PsamL43t6{^Li(fl1G>@b)aA>G&-MK}Q?Lxq6rcyh=+TTgHe9YxRPfqAr0Eve1$r zpw&36_D6)2H&+P$C zV`-b($T9m-%$${TfRmKcghMcg2N$vB3w}X?L(CEvC6xoh$6g_#IgYN^D;w4Wvb`@V zPkBuQodK)w-4WA;<+NxsC0Jd;afh1iV>oj=%$#0PhK)~ikkVZPh1JsF=pBt%8#5Er zyI`|9{S&a{EY0K^c7}`(@3y=?-?0$IMhhq2I<~s@4QYzPJz(mlU~O;`bc$Ux?6}!r zaMs3+YRQ%HjjlYL0ZL*jtJrMs=wJxjcZbqhX!C&Ow?=$J(J`M`O;+n!Xx_h$#BK*^ zkQW>LjL5b{pB)awd%^gjj7X8IMrRCoI0gX8NCo&>msAFj<$X}U#QT~E7NmI0p7G;W zcJJ&9932dp(wk30#yw!UjPkbjYYby7wkwcHjI+Gpyduf?Vzm*ZGqPaJbm_KJPY}#YHepmQv&s_ z>~y4+zPImueB$}70e{}=<`zgTAKJ;E!1t)s$T&TSy2!cQ8E+EC--jj4zIB*T?Yfr@T)aB>oPe)z+b+>Pd01mh_q;*T ztM~sdjHLWhfn*SUXTABe_sc9%*325|&J~C83+TrzidegDMUYBquasz)`!V+6U=(~R z1%xjSujMm-_oW0%*ViiM@Oz5nrV&mgVKJ@z_fvSg}q4lDOiB++p{R+Ag-%qLYN zbv9Ui#~s>I!7hwLVnHF=C)k&x^_2H~do%Ff<=uqgetFm>N0l72z?8Itez-}4CrOjF z9v8jQ@*tJjO&d5%C^WgpXTK8j>>7f*mYg#hD1Tqj6c`na;pDp_i`7aiTetU@782Iu z-@xJOaAPcdKh;b-&3zD_dSI-nQs2KLq*YuUQ_bim`{`TZ1&+rKn|1Yw<5N+1l!GWi zrc8mTowqaS`)v$erN7PZo()iQV(2s~gO+W=A|i^_X%Bw!M!S6}mO`CXkp-aO&24my zYg0{*TUbE+G6F92?(Aa+$MdY2OlA-YT;{E5ddhMgj_;MIxzQ#oFiu)?uv!w;_c3NE zqui;G+>EiizJ!5|p@q!Wtn2IQ-}^;tsm)qLqbT~BD|f-}Uf7(&+^#`p^8Ek(F94%Z z6++m@(1rBXK@&8=J)F~lZz6N&A&Qmb?-f7HH;c^{J9A_(8gH3(K|>aaHdtC?yt{Th zajK3)}T ziYRW1fIINaMUB<-TMDj*ospPA5<+f{00{#1v?Jy+sk^aOo2Xk()$GGKTUm|hO8i83 z&u%@vCI0I)PehA!P19+_7ZX)g468jeA7eMF{qNtSWkJx>aLCBRz;|0>0moVeh&3?Y zt$eruFv?UeywnECOeB{aN?ZIWuP%#{v*49Qz*>h1UR z`SfuXC`1!ol+j|X(6grcPZ1tAileK(86QZrr90?}=Q%pvAII-h2q2n=vmdtEPS zT>v5jcPlotdt@iySx9lZt zyrY_b^gdakKpMMYVf~^3lNBlU%IhSfuGirHW7P+;p%&+Q5`(t2)DDDPWG3w%D9U$M z3AjxT^Jq(_3=xw!iii{AXCW?)eT6PXw04Cl*d^7^xXvRg(Xd6LaZU85yDaS;KfG{oLRVVuP5Jk-OOJm(ZL zet^GiN78*|I%&?4BMIviLwR@1SJ7>)5{#|vby7~f#5Jj4=j zVPCC(GuF0sglebd^s94MZqT(S1tF~l87SaouC>=pT$vfpYyR3Dqpl5cYwuS0ml@;t zghiw_e7Qm-MS0^@*#E6H-A`y$YN#~2j6vi{g>T>W`|}50Xkc%12kQ9dolbOvdGpfK z!*NYg+B9o)v3ybljr+iyr?h~t>eWPCPO{ZNrhr%iQ3ptAxo<#ZmJW&YF8k%(C4Dk` zbn9xl=2&XW^jFPLw=12tzzj)|HtfW2+Jf}y;s<++l1t4xyn3zv>b{^_Ypv_i$0Ll(wFcvl7o^4n z{NHr8)(-QNViVIUVE);yAhP-@sj|0&_L`?a-32p{bRzF{RPz$NULkaK5fw&+~qgQ5j0v~$=;r+QPu!~Mxh z(WLddiTv7@^MD7q>H~j2H=M1bduG3YV=^{gXiJW&{0(P!X*T{-sokV=*Sb}6TedOW zV&iW4lZ#WmjjjVo4h24b+|IUhF~gAQ1jFGVzQH3yrxYNFLLo{WffWY9lc%zuynd9SQ=LfrMa#D#u8>0ZkT=uQGL)jg%SNG(Eg zHaKJrCNun_r;jG|sWGv!1Y<|7<1No8SR@)95mPyl+>Rcum%iCEb^`e;Sb-)EnBMAs zZ4=wqmwuZ392kKJ%U6+?B!A^8?-@)Tbuko5k$kN_P8j4u+M9kQUsjqd4BFB$=@xBo zGELeFC_)pwzhsD_&mR4P59VRnQHevX(OEypiEQ+Fq!VoQQbwK2O5z%-?ls02vz4w*W zRaSI$V;>O&T-48qY!=YIS4W`^@=0%u(H{$Z7GI*Q*Uxn-%!WUHN-QBiT4NOHuhtp| zO8T-P$n{9hcnSJ#_rC32WR=0ONMlh+pAASrr?@c}zZ$Z+KT_hl?*B@>Me$G;d^ZLR zgJyg0Z9xFjV|!*uJ$Qw?cG1a-=894yiWz^{3b*%|@@OptrU2plp z{j50aJ%u+dHFrO-TXUHNLbyJL;({QRE|~+tnW2=*rDrH=Nz5GbBq5fY<5Fga}YMuf}h_heCVWVFX5_?Egn+D=&6^YS$%m#qT?wo|)_{h}1F zx?sO{-*Mt|8jD)v;NeUyHceJJBa$4Swp9{dTD%Ot>IVCOP!5no$vqz2SQJ_ih> zb_j4Z*k<+yh6m>>kk@gzW!Q^dcPqVH@#N3bGP-3yHqXY;SQHhysWqDMgVK_Jl#r@a zYPqk4LMu9c@Pko zn-}QA-?_l{;xA+e@QRSO=?^nR7BX~ghBX5xjsHmS4%wb%W^QsNiqlWkDlV6YH>f zemg6tuuN!cGPlT2Dr=X-zZ!R)AXpPzQqC-f73yoFShrL#_GRNYXcg7caLH;l^Q#^l z7-13!npsXvjjrkVT~Y98bu*895&V1zYXN>l9+gnf=TsdZOFm$$*Xpi|leUK;2%jGP zhoqcker})o3ctd}gnh`Lb)(ns9LDmX#r1FVG?rkWd2}b-c+H+N@m_a1U&$VdU=wbZ zCfjO{#9qzYUmY$1_7s~~=_ShSNJS*KQjO{(q6#4v>(uvfO4n^Im9Ryx8E#EEsJ6B+ z^br~IwmTImh{<118XPU+a<3-xhrD9tK47yO`z;ASKvYNCGOGXf=Su3sKd&bI=(ftt zha}^uzaStmM3D|YRER)hMsG+KdAT)Gxd!^H)Ad!JhB9Qvcr<=Ym{h^NI2(1s;l#nCA-WEwKnMa<)eYX4(d7@M$zEy}KFcZp=nr`Ia+| zc}S7OOV>vPRN+Yoew~XOhiS1&c`ZyFr z2Bk=|uob7)v|6@}vQe}aKIRK^Y0QS0d^D)^FR3%oDR$S=p?YmQY`;#i*uFM2T*Y<# zHN^w*it6CpT3&*$SDsM^S-m zQn?{t8LHMYbm{qPiWT%kv~GG8ymiUaf{(V}%v@)rEy|53CA4qtmqfp=MY#yBNLhqHqfAVSXfQDtT16JH_70Sc??|M?fIL2*H){Mz zDKXlkLPH)TMQ4NV*V*pgtDG)(&7YD9Tf29tbCR*vLv_wJy71J^&RG0x!t`(UJ06Gb z&vdR63YXv5!THCDYI*uVQ?&x)tf3_5(V!F$$NXx3>miT@-N}Y;gbRlO6wq#tN98o^ ztbfX!9?#_8zV~4%CEsO@2e2q`D3Kao?>BFrs+an_*dzayvr!qjS!@a#x`F&ovWWm~ z+l3%ICRz3DNk7t!G4F61^V?5(#}z|u5eb*NmaZc?v*}PE`S)fzuqGyxssLway~y0} z;_=QpF;^0>fx4fXd&dNu=xc+p@D3{}=8WA%J+rLfs^P-FYdw18LVJ5>)rg-e1t>U$ zX3T!?YWepyhm6G6m|0uR{G4tcmAw~pPsR-BiqClQch~_@FOqY0*R$&o{~-jub2yY2 zCB9^%FO5i=7#~mgq}}kEX}1}VaezsSX$)9zv#=-6JSk9U8+}~ic-rvYqAW#ERx&C( zx@Iil9%P&TX(c@+CvfPyXBCZ8u919JGC`zcUS)mJ6R*+7hgf^RwehJ=BlFqHg-|}V ze=wPZ1*Ig{qkIrPlvtR;{zR zr+F;$Y+56BQ+JL27y8~`g?`z)rZ!Kic;NSHG+>F)3dOm5yGz1q+XjOu+n2m-!hkHQ<@J`{dftVuBaTdQGH5L_ zM2n-=`G-#WVujGq5_b(9%InI4*+wR{#;F$JM{m%hVT#|q;_S~FVuQ@-V^U%zG=k#` zQS_UBgy5O?Y8|3DCeZQavkk8g!HMHU=1L?=ZFS?C@HAL@FYQEUuC1#h1QR4u#TGlh zzx;O%UG4N;?_HG5u6s#l#x5zrC%ZXJekln6Upk9( zZtc*vg|HFOM19H$y2s8Cb>3jON}i(m?gz%a{sE)n>H(+bE*shi;U3L|Q=P+0%fYsn zp_#H+k(F8ZS~58~I*-ur!%pBW%r#?Tc!gZCaAsJKlH^Sq=PW_4;%&ek4)}KQu*Z959j@aZa08Pt< zVEZxFo`&plg6{i}^Z^}RAudSq&F9|Z6K;GV zGOi0+R76tv>o_YEpEe#WfB5KQeX~;4HBMwt(0#PRrQG2-598A3i$KDxlV`TQOE{9V zdFl(cR;3mG->UKH+f@MaQW4qbP;)3whI+sd@1&w!IvUg{*?z24j;W>$X72`74|7T2 z%=TF^w-6xaLtFU!H&h-U4o_ag@Pa4fY}E48UN(@p1lTv*ojo4Y@eLq{Q@*SoK2w)5 z;PANp09z)^{jvodD4X4ZMun^Tex+Xf%RYcEc0psAciW-ke+ zN_^uvYy?HQqDmxsaKS*|ebq7`!&{pK=M5H%rt(Z4s&%lnDdnhl(35MuDASaTE(WG8 zgtx&>vvG%d5jm&Y{}!_(oeabkGdYh}#>H9VU1h1=$?Ke*Th||eh)(E#2qnW&iEM71 zeBo%Ng%|lAweR}SDl=D`fEViTl7fGa&+xFwcnU#)?8LlXDA}uu2N6=W=yvObTID@6 zaPum;GG2Ro4OSz+Sa;l5^_SkX978E@E9{W{I@LsG%HdZaL+WajrdVkp3G)|Z<3yc= zm0}Km4X6hz8E{i;s=#Xwao7k%e1V;SjooUVZnW0ee-HX$u3xmeYO)b+Jw|*kJnkcTN+}mK!db*DZlw%zxC? z5I)uff~g}Q=c3ye^kVoed|GBFQX^>f4}m>yHex_gW{0jA;o zm6XjbB>H6I9|TkiSQ+su#Duj*NMN7ZTE?@YBt^*gorQZ0zfCuK*tN!n%3AWP-cal( z74q9Eb}xHI8dIj$1u_?<8bl#*K_qInXL!~EIay5|M_R6?kQm?6i+l^~o$aL079{Ck zr2l1&>j7=#()+_1+~(m_+iqne)v@fU&!eK-l9HNQ3R~~_dPk+%hTu3?O$<5!Y~ky5 zsmo?>5VmxbCaO>@(a+8jXr@P5V}Z@7ba`9G=Dua@^iq2 zC{xyfr59DQg&<;TY&tIS$d4l4YwCDXL@qbj#NZ@LAU&F>@x2?ThonhDWi^yTu2Ylj z@usk=nFmvIm7g7_=cVvpCuXL3%L&y((Ttd<+trxpG)pRLO{7neIP&En zi_)1X_@3O>p0Fo*w$2guW=_%TKE(OOG#YFi1N1IVR5En{T&SoT{U4{MINNLWYVnnH z13~qBytQraI4(gyo#z*DEB-fT0|<@H0JS&Q2^N#F+q|kHh)_6+x3$R4@yY^QO9Z_V z0s#bNhIQ0eJcf48Etlv8T#Kja^_<|yD|tB5GKEC|_cPMut(PVDrJS@rT%8qVM)AIH zKK4%(|MXBJr1N~>!ixqa7Sr-SxRja;d01+=a(u6P$-jZuBI{lU8W6r3HHKTFY}y&) z1LCXZ#cIN*Tb6Tnm&j5k{3W4!KX&HNRli86L^57~{N(ToF8L}=kbx9jyZevx(zNA; zWol~rJLpRJLf}qG*Cj_@zezWD`i_D4Bm3G$Q|BX%(KJqMC9fyT<5pCmMCLotqv_N3 z-e~Nv&}c!d*T%pZOMZ&hoJoX2uTqusjy!g6v{<2!v6EDybT2hq`1c=&?I#>|2w)~# zGar1A$O-vDDRVAh-2^`r#py)zZ~`fA^)oISst3A0AIV*15P{Kr17W}!u@;{}@l94D z15hGr_W(^L6w177)86C*Gi+)1Hy3$O8%I|Oh97Q;*2n;j@+Dt+^{gE19*e$3aPL{? zehX`@-;dztoj0QB!xf1@n|c-yamFMjur@&uf*&7Jk$eG~Bkw(S4pdhWk(NZrK6+|| z<VQDR{cc|PhZ^GDI4oa810Q#XjhCpDymR_F`^FNw1=i{<|Ez>>_7!?MT z?HZ#AOoe~@t!w_Z-q!oT#;JGv03qcNC`Li*cfd7ru?ZqqJ%T7yS?W-5MNC4xjREq~ z?2)$(KE8>5NKsF|`BknHj0P_>Pdap0$71cgFeNhDv#e*$=mBdi&Fa=1($gsIiKI$v zxDpJvjpqN9IE1b1~lb=Ve@n! zbUuS@{X)=0Fm+KAznc}O8Hv+iQNLalr>JosvMfvS=pBHmAW3KTqVjsBQ|zSw^XD%N zK_yHpLH0Yx-7sNvh`ArBSYE<^$#Gdx#?*SrGdXu{Twq@;`_H4Xer~lomZ|W`44bvM zJ($VH5~KA2biys>M4DIJ|DYNGm}dw`XYZS1lWG6KBO#V?0O1Qcy&NWh5V46Dmzg=B zTuppA*tnzaFv%BJ#z5GK`UZI@@t*g6NxuOg^bHGN6?Q^Vt}$~0B#vg6nY*3aLb`A}7-)*?sv=%QAmXR;6*ikqQ%Nu&A}-oUghVFj5S;}; zhUrp%wjjhU%r1Iq0v_4EbnDL0`+J9*v|NGUf0qpvpDj_4F6lnF^XN-Q7W_c37&H7V z6Lkj)DrP%~gbTPgY!x%G475e+I2EY49L8cR0V{BB(Wy7T)!M%M`|GZ|uL3(Jqu@A` zms}NMB!XqYf#gfWAi4%MJBV<(>FQqJR9m97mBYsW9<+M}W+WxOP(O^Rw)hA-77HoL zkz`V9=BzyeJZ9K3zvc7Q1jR8=f0lHae5Y1f*T+(Qsipp`S-lK0-w@$-6c%B71qc%Z z-RSAemWaUwgnj=8y7a_*3GThfp)&DI!?qDD^RIIEB|N(9@SLm^tF8#1Y5|^(TH!uh z8gvV}93!V3%t&P9?Bzz=K%|Q(+nhQ6&qA;W17enbUbs{s>(UkeU150OdSCP{JcNvW zUtXGvs1;pEQ1BO~6yV@*&h!C;+p2H#l)%D4)z8;5v@W{mN`p_J4S2P{nf_#=NXT0q z#-e4P{Q9i22WnAuhGfkzOdf2Ka&C5CryB~|#Sz%rDy=1j4%wlqJX(eB4l2>_bF|V2 zn!#sPZ426M?;e)sA^?f#10S*Y+B@Zu5kYQcYtyXcDqfsOx2g)Qx0OP}Wv|-&?2pQY zs#Viihj6Wr$d`R#6G!eh)?B%66K>#RH3%@ZuYr!`E`Z%jEOsGyqv!tGCZT+F;YedF zBl0p^Xn_X$X#$-GKdBEi2C_-0@da&`M%cUsJEcZr_-g?q`fo6KfSJ}8d)TDFK_Un(joN2BmTzGaIq`#aYVS+610jEm)ACH&ON-Ae@KxkrfX_*TumDq*#Dc${&e|85@P| zAmpAHYmQ|(>3%v-R`USdp6QysuO??|A6XRRt*WAm%m8 z{i+O0!7K@$!y0`S?MxNJD(~UyNytQW9!61a81|^#?SX*_ zvoJdVz;#+03uN4A6O?9oZCm%h6~qx8iIHmp;GMdnWpy_liN$*mQ08VvK)a{vUe-$d57=ek3Yu&76usqa#78#PXIk z5^y#{7xOI281s`Hv(*mqR={*rn3xG6mNdC~N0yY!a0P;{aM7n}qXuUnfrr7kzf3Z9 zz?`Yn_{BoM*1MdOm42ZIlJ{r!b2C!(LX*cskXst0MY%!dfnSkZAm1_G?taxUP@(dbITyECYVaJJqSI$~$UmsLjZQFysJ0J%doDD!g(+W=x zlC=kcK#Z)MoL`+WO=oTl{MjrN5ngxUfdIMnrxN~_6Cqd9XtEXL8z6l*7!@FaZy-7y zBq|H7)}Jhm+?<8w;-JvBl*{}SN|KT%C+~mV!595GpT%mMS7h;UvBn}oWfbr808!v@ z3g?f5(OmbfNM+qIWU?uY^U$*panX`T_a!Z+7LW z=ZLRbBzM&>@UZzLww?4}0{j~VzdCq{8+Fe8He*rVtOH`9-HwjSEFU@&kQ_>TDA_Xo zr+aduT9^I9^#5U&HrnuAmBM_E69^l_3;+J-N=r?#!ih!bDWM&u3g+_rmn~eRpkX8g z1Uk8nTl#Ds(bLs19+#+efSqWXMlC59${>YEt%DIkB5e!FIZ%8o^~Z|87a$TR55S_+ z@}kk@W;-c#7tp`W%S)Roi7UqJ15Z(NzPJNKfsxF&8br6UnmY){`qPx}rqaq5p=g13 z*NZc6(;Y{G;Z6Y?16Q?^t}5giXakH)xc&Q!aL?@-mWvyPMgbO#cEM4M1_eRy#pS6H zzl<4^bf4%CGC{{N!^T56M79Hi#_M#q!^e&}WO3&*Fb(rFL*Ds_ERb*dG$lpNKtB==%c8t?C$^PYPEsFH2-X6u zJcdwbYl^YkV96(2i@IRjC&17H2=1^A{yy9zehOlX&s_Td9FOmBQ z_5PQ935j;U=uDvytxqJzfNQZM@~1a-Ue7Z!W6%D965Gq|=kKRH{Lev@%Ru8noi)`6 zPy9a{y4YTUgHfu$PzFY{2l|lub#r2QMa9I%A9B=**vt}E)zTWbirwXh0egV^W*S6O z96pGq^Z7S$9ng^Hr*j2v`tUww?28CctWI!b7n6OH3}-utPP6T-Q@qNP-wsa~v5nPtfRVmP#;fCq?^B7!8@;C zXQdKJS@TyWbX0*2A5E`GiDh=FiRFi50>LH<+CRmsq56ne$)srNt(#UG7VBB@3_nDL zzgRmQ?Z0Xozkgp-@4mk&EGWpy$@$i_ZU+q%;(3PR;qfrvC_#D0rd&w8mp*QDWytw% zM1ae7`&0CDZR-=Yeyq`Qt@pyooci#*mmBtzZQFE z^1is?ypYA0=YM5Pv3*rp^-S|TwBF9KOo>2pZee~|n*PA-4e?+2_okRa{;fimpL~$w zDna*N|RV6fm#j-z8 zi28VLE$v>WByM{6|Mn+Mtlwr#>?uRL5b3?u_ZT&CmU?|1t0hNzIA~;>5Kdv2Y-8>3 zFHTvWVD)O%uA0$n?Edp8Nl>aOs%Hp>URuuvE6aWDJ9O>}B=yw%)-FA;9K*(R zMyQkw?vucqD({c7A_4vSUJLsaYlax5$G48=t1DQTpONtJnfFoyOI@Fi!-iQ(6936( z_#L{ZqL<&}7iB4mTU*)BnW__hBz7sHb`mnu{&Lq!P2##kMTG>TO#QqB2vp=2<@~8` z(VQB<$QM+8N~Fu_#IP(-)yzEgT9iC|uLVq-*95ii@u=O$78qWdgBSD5U>^%NHhkKO z-xsJ3k>94zA1GHs;GZs)ru05zNV$*~_af{DDyy-HcF*!~9qVQXe}YQ?>EKxDjuolX z1)D5E8&ysN<^F^t(D$UmUSqKNz;CgvwTq=0W&S~C-1q?TClQ4BM$CKy@P;;YD0#om zWi#&1s`@)5Q&Z{57fFP!Q9e`X)GR*{CYJN>Sh)cbk;`SvQGNBdo}U1shc~jXKr|j7 zVaw^5-l3`X~zY-Y~8p{)b~`62X$wsqnfyCMsj)bf|(5=g*iB}(Zy z=e~;X;d0}*qH*?`Jv+tvg-KEh4?X=8Fll~!3WRf3AJ9~;AahEEkK-PV-=%_{C5*>h=P}(!3k+&5EaP8+gCG6 z&J&S5|B5yUGZyyGIlp_MI`rU}p#g)2O=Eh@FxV@JW~kiw`u3wTB$*oPrLe*}W;3PK zNAQ7&;byl$q5BVBDJLhuH%~)s1c0ZosrAS2`y@PkmX=hP9Zy8c0-Rf1Dd%Hzk1~N>RhE)tkl}U9dZlK4*)ZV8~HZ zB`9?P(rOZ49B7XLF61-faZ)fxX1Bz`n!aq8&Z}9%dJ86-=v(?6n%MPTkQ$-3)uD=L z!5l~-9WfOb+hzc`FwPUA0NVZUCw0&l8K6?X2hE#nGU#j>U+h#@{#jox2f)*Y?I1=oHhn7aTp z_n+(CU4{LY=aX)aA6TdAFTAm4eE|C|u+`}aT&|Yda@1eIDKR|vRe|y@{ z!^qQB^IxDz##j`4gc8!a#V`M3ky<{;-B0!X%&gK@@bq5NV@(qWqU}}8{V{=Rb z%3el3N0Z&zq{_RvTDkHqYiUPu>Xj>UwwG;1u+Za~Ao@R_O27ump6ct(m*C_^X5!bE zepBh%sp=GnQZprty@9l}f5Pkfo_$18Om~++p6?+0>v2L*g-iB8)$(YBN?H4@$$`^Adk}L1KR$d(9ctgu15g5FZ*| zE~Gph@H<48rx!Gz)-jMI?AgFvScN4P%;Oiynx#2<$~Jxcb2OBf%!;+kP#V3nmS7BR9d9foG~!AllE*k5wRFt#PY{A;?X$@ba7P=eyb&hznnWi#CS zscp+tY#Cs-^)@2^M{rurqCeU0V>x)7=n0Uej%a^1^Qf@g4#Zqw!nHuF-1&NHxF98K zduG-gJR6;uSilZ(38+P-1k9}B>RP19=#S?ZS;ZMTmh{;_)Hr>a{cWf8NSMcEX}rIq z>aHAe)HryEe_UeF{2IMX(*-tw@4Zr?kRXau(UyNKDluEs}Q>Zrwvl2 zql8&2%mN)+@DpdqWf5m+v0jLHN_POsc}{5w!Vo9JbWhN`U)0DTSdm1V<{*#xBBiDD zvZnLKL8NYj-T>4DU9iI<0gh2XT!G;!pxQk(5>%bUuE2I>*l`5ipP4WJ4X#&+aAs@B zUV3Z~Cpu%a4FauWk=qgtb1q|Bir}cIew;Suvf8sp<}*L1fyVGAHyv z8)|Pk>zLY}P^D_%#YHyC1A4|P;+M_KPE)t331%?Z4<_*+E!yS)aX<}Uw?neU`K;^n ztjlk3V)krgs%g^=+Iihp7zt>ITf_EbMU2;05Bl4=?i3NzWq~}j zoMvI=X`>pUH@mT(S38~t^wjD*e`Or2f~5td>@(?2wVHlUs487nu0sZuX#8uaeL;^s zutWdw52Wv>N)IClmDKn6eXF($=85ziYXoi6h1VW; z-U8M5j+ct%?=$-g1I;YFw%uSdYX_@Xay=9)P?bo#ERdws8<@P{sNBA?iSPS~$ML6s zqtHT1sUg+*AEsNwo!;*PRHgo#y`hIw8!ra5C#f{_$4@$IP;T%F!EYy9 zn@q6E59d1*TmY{S*ZrLC%JnxKaHvseF}BXnSL#-efCdJmIRQ%T-*Jey2qs^pjX}MW zbr}WPokp)?m~&+wntrwQ<|b|BGBs@C0lFcvS(npn8O(0>LXvqT?gnM8IWdas%(i5#}?VobRdA)rmy56<(yof5RdMLCb9_Pz7zy~h8kwzlh zA0X1o*0GAhhjxk%ZpYaU+}}r5YE~a#o)nR4yQL?4J(Q_@)~%#1{pGsK4aS{C#{1yJ z&Hk)B^F*OKaVY5Q#0At!wt{ev7qThCPU{Yr;Sp{Te5D^6PZf;s0#2P*u)~?{lTPrf zGd3KZ&Q-5=@#TLUnTiwLZ!+?$q)I@Nn4Z5%*i5iC7Zl5t%=Z1bxq+Wvb2?p5H z2!H~*Y2dn>u|}M4iFNmc=WscV`y_#Wm3RP*y;Es(s(9kW1-COT>Ow><(cWMurqY+g zD66bo+|3a^NQ6?w*Bh^c^@St69%jhx2SXwxPw^uYcUmPHR)qs4>i9N5a{f@D|KGyLx-fIK0*&C&?gPUwgT8hdzSgX6((wAQ zAE0S#F`YWre;ou;pU1=w5!kr}z8m#EXyNmUMU|AS|AUPlK zrGIFd(GUJ%H@yKWq%pw9wdQisu=L-^*sJ-=^%f)`_>@_sFzhK~7oC#zYG|zAzX19m zKgZK`)ry+wX%H(Vu$x+N`=F>BIhRQL!H@z|tMMrkC_ycq0F??Oqp7MI3QP}_uz#R= zilK}3$%MGa=7BGR`AW&elkLffP3d?k$qYivEeAx#s+F=0V#`!RqSB9Dg?OhAX$^te z_qtv8O=7Ix)CzjbEzN(FO=`J-2HQ;eg-CszxH%sP{YrVa<(bBgvf!?yqhOCDGRM7c zZmXQ7dWb++mRpUj`O9BYSW+1ILp5>rjVQozBQEOiX|&$MzQ(>36oSQC3p(Bf!^)bj z%xvj#viY^RsK(pj7OYktm}NGY5n>LAlW&p3g_ep6#L1v%P*6Bt z>GgbX*xXaW9{$kT!eLuqy9Zh(*8mRMWopkBABDU$VB`-9#fR`=)32m1@xZxQC4MPa zT*?vi{oB8$IB&oh41Yy^Zb+2`UT_KZw!ZFBTiJU)^1Uud@!16@TaOUA$~t`s_l zB0gw1iw}X$P;ryM1(_F#%%Aeu@t6_ht9nzW4Nu}HBK>AH@#Ig|eiVghMb;42D>U9H z;)M6<54{rYiE);E%NvSa2grC?!5G5^KPSk!xVOR1Pq$YjgcSd<{?z-eisW zTf^h7?{lJ(EWzQ<7pR)XhC9zDQfU*3tJN0BJtS6n2tr$ArltL(+DZ^ht&BMtO{;33 zjScpmHrukpjrSpf3@HqeK3Bo6HMtu!(Ctb3(-^R@@DE2HLJoM-M?Cs9$Hu3+Fo<-W zX`H&}n-DkRHVoxUJ$x3|D3odpkwhf`f^!{Z`Dti)eZ8Ctf6(r(TWa@(#KtpP#HG=$ zd`ms%KL_I@baS<+_iF_l`L1r@0F>(wmGvhP#mB~Sl)l6X&`}?)gUpv_8?7SB^C<91 zGgQ$lbyw?Bz|9~D%LLt!=YSjvw*mzj;Bo<{?s~Vg4y}4PJ4<%-TQ5dAo9_~a_ZT3d zy$6~@dK-q_kA=61adCNIrAb4t(f45`G!zM=DfJ=1fH^pTq_-N)xtL6Mx&)A!?SiOb zA)-=61#<2}-fG-iR%drOmR&EkQaVICyhoZkGo!c)2Yx3Mlc&~;Zko`95Ta$uDcJQF zQ!dzQ}3Q+eH5I)XtbBwcCvMzzQ%NJ;NDypJ6da6bJDrxBs@nG_BXvT)y4l2G>M z^LdYSanXyKawNvaRE4E7VS{2kO_Sz^OSqUAn-qJ(CArP?{P62CP!*2m8^Buao324y7pcY%yERFGK#>Vbsqr}^{z{rxj zyTxgtBPDty(vtaG?ZmlvI1r61GndAQhNn1K{H|JvU&kB#3*z)AhDD!?`7oIJrB<}X zhH5#^;nE8WR&zSMP>Q&HnI*z>o(vR6@|ffv16Hn1QF^^b%u^qk#tnj_%wNOW_dIR_ z9^ZanLSJMw^A!~D!of$#f#9c|EC&kEW$kTjyuaxw&@9!MmIBZ2wQZ4oi!_3r~ zt&Orypiot;r>jR+*Zt=^57)t4jlnIi0bC}+z2N+A1)lCheS1GuqD5lh6}q`jO4Dv( z!Q0zDkG2v|F$k(bP;G53Bhfiv6tj>coETEqi}yp%C}`|BJ{Vgf)Q4%Mx0uz2j7Jj@ zlpxog6&20-Qx!XgXgK@i5B6_}6S11xS3yEUJ?ten5PU93o^dL&-d0>Dj!~TJw5D#` z^+6%Eetv|sDAwMAXAVzq0Tfz`eLIQF<|hp*k>K+K(v3EMC9)_*x$sWd$T3Gxx8iGH z&~DxPnEo{aZn3Ay;I)StBUIELtwWKTYYdtN=R2)LNXz-$(@Wn!eWHn54~B9*w%f$g z7hC^p&JE5dy(lDQqTyH(PeaH>E?NxJ)Rq(^ld5Fm0wl0tbDOW@EHpu>VOcDgm#A1x zEePWFC)7kH;=lY(Zdu<@Xa_Q5Qd2|!lcU$p3Y4LXERGM1yqG`V@{V{lW9jNXY}VqS z6xDN)KZB=cMF@!`TI-kM4Td~**kQd^qj<|MU2LXw`-qGs2Nh|-J?Au*>$}=T8#|=> z(@%OGQ#fhCtnyjjQ+wH>A^jP`OcRneCj+!3%<}p~)~ykm+e{|8mgTH+gEL?0Ym~{r z^7&;j9mG~Pc5B|tX0{19x!9D+&^ykhsdS*;E2xy(n$%qg82i=tKLRUu=^QeRe zu>~Vv<<8~wBm~}At>5kw@cLW9;$!1x(Bx6ckS&fpkd;yNEFunB{&;aG(g(Bs8g1(} z^mZLOBF7y}2bK@I``;inrX`KN3Pr_y55c_(<3YZL;=#3ub#=qQ0=ZrgD&YjE6ouLQ zeqhazUaN7u>DZJNWP{~0pRKOoDgmM`spw5Q45Jr2(L3$?4-3n6hJnjxF(5-FWDu{r zOB_<&T>ZLXhNHk~Fc-Ny*x}aef2l^PY%})l0hOj(fARJ~anf`VYt*$9OS)b~#VtC= zwN?2f$`B>R>ottIT0W-T>4|A5gV*ydBP6DR-u{@ZYItvWtG01)?$W4}iYyNbt-}sv z>J3tRh+{3LkOKY>Em}rTzJ0^m6aCAlAlvATg(_hW8J-L(e^}Z@dierYLhgY8${xV9 znxbU2aFLFV*Sl4c$7Wv5;}u;`hf{}x1>BF`a-fbKQ$JI2bwvv7RoA)Yi&R;CIx#kG zuXsF9HK}VSt{XvMBZ;;D8L5`U( zTnJGzSNrj~{B6JVrGm_GuL*G2uqb%I(LXM%ViS2mzyq%$aWu}oFQR_T)zpxm(HM0( zNVxh{ImE%PgM2DJ8c%~w4+tPhS@FOg-85kf(!*!XKFxQT&(2EJRD#D>sC{m#cYLG1 z=_R9ZhB>T|ZAs+Z3R}N=c=y-WCD=IYy>8uk6}{9Eco^C8S)eINu6k8C%pLhr;uHAI z0hsf-sU}CqetTnkdfq$4Md)A{ zI>{g<0r%#dswdYS?2zO&!zjm%H79*8@-aBQwIY<1^VT0Yfeq%p_pYOFQ|=Y|s7~1? z)!kk+RevYoenw#|QPRi^riqr>GH!OofI!ZI+yR?Jdm?2#r@T=W`fZ{{s_LR;IoRRo zvJHlQ(KC3p6WE>#o!cR8$UW|Anb6v-4hVGV>0}b-v={e{lDMF(i_#ZL2;QA8P zwm&D5>GrbdHs7@vtzjzBl4uK~FzVxJp*aotbrB{(-6H$x_t0K@u%hQ~tNtMw_81}% zeblm(jp-{dhs5FK|wbIvPrq$8Raa5z@WN;S1Eg^(D zc68Rl2#<}*PCoc5~`gnccIdi^SuWK8obPI*K|PE`-(T0&}?*a_Ua<* zuZ)WhPmJ=O`R-l&$P7LvBnhKZk4q1)M3EleJ}n<*S%%k5 zWU9)qyNqDR!FSO@r}bF~ts?(exr9 zR)YK>DBoX5Nh^C_G4dr>j{aenlLd0!_z&b#3jA1Ap6*obrFVkVh&R?BIh7<`ufr1_qu42hH_Q~tfA(^fQPm3a ze|ypyJ9JdJ?Y4-phecMZj5 zYz@uxZNvsl&s#@EQV6f`85x%z2M1hl#Ha>&frKNi&#b1vY%b1}|7Ns5<%FO$S-Tyl zZ~&{p%72ANQu`CdCZ>1o&?~VPoyQn);ecql(JJ)Nr)X&y5<-__`sG&(+MbOh!Qhbt z1Z%H~0q{^HtO2^jZtHa^ zneD7$9UTWT-}NRw5!4J&7$8;dx$;s47>lV&e{cc*#gcN)tD#f)^z4C*S>-_w;5qGNv(9!@==}6;%2dSqZ(kDO9#}#9ftxm3e`o9~uf?(tS z>O@-d2&RkHASk(H0OpNd9a$nuJ~^^&j#MdTNG3z5(9X?We=Pis(oEU&7ReE$c(7Phw_)AlWq011aEd+{ z+`3@524R`*b5g#~Mn7}i%?~(I*c;Pf5!fT!1%E$5`@Ce;c7xT=evjsHk3QrM1%jsO ze{ZlVL=if3i562V%Cz=Sjm=NG7Ed;;!IV%GKk<2IzvuiIjk~ErrPc_+`+6Qyv)w?` z-Y$lx87Kn#igbwgW5DB4^v$O6E)Pu}5XSSIH|K)-lbm?#Kvdq^x{7rruGU^4SH!U! zlSEsZ^FQrf`9G9v8@G&g#8LLflEyBiWDAoeJ7Y;BSt3zF80(}NWNDNwdt+C!Bqg$w zC6Q2=Y}pEvHT!bVd(Vm9_naTzzu?U0{>|M+1^`w%A?FsU*XL})chfkw@|NOq-P-@MnLU#2pVu_aGF8QKZwjYIv3Z>AsQ_u z7S$`LauwB)jkzSPWO=QfcN*bGy+j_oL29dAb@JA@bcr!{yGJ0g_H%FFn873iOlAAn z2-Zp3=2Gdg;K%0KC$Hp--I;LW>fYKpj%eb1`E@KG8&xUIL%nW}Z3dm{&}+@(u51!A z1%*cAUvt7ub~fy5%qk2~cP|i=aGz{C3`iOQDypDWB<#9kl|qREpB!-_xu9BRLx9L1 zNNJ{yR0{SLvoT`3+3>~QvaFUr&dlTeaEm%6REwKEOZLeQC41EvxxmEA}2FE*23^_=pP3WcIyj6O?~sN^?Et zCbvxE#043aBdyF)`4irED(txjix0KO+71$hvE^4Qf~%THeh#BC!wDF=6G_?ljD9#B zMn^7Rkvq(Mo{Z=&(t)nwPGto`;)L z8y&Sg)btXj1=sB5>`UT!L<+8BbuYRMdtqRf#5wlatTo$8{fp~)$vdS`?moom31k3NPI+IJNMD(j&KyeS~0_gosT9vKQ7k4 zWz4!ec~z_YyU)g(4Ua{9E7Gf+sN4-^Qt^{UuYJRAzzu5DnJV=C-JD>w@RcGTn-U5Uw#O)`s-=s`cOs#j zOcBBzMt2Q+lXxy2Ds}zgef{Z$(V#2y=Zc1dgpNU4*G1CRuqW>L8>@OMHSgJMEGm~z ze~}6Ne!Ju7M@RT;{3^|L{Lk;m$=wg#Y3@z`S>m%nzGz|2ev;nAOqyAegYWf2>9$;`E!nFJurNLk zKdu$RQ~JmoH4@6hr6nGilQ(!6m&P=qqPB+-*3SlSaEUA?eC*WvK4RT2+&BfV*5xoeX^xSZ0O0-lHwzH12 ztG+TvHx2Ch=i&->qASy_<26PLNp{;vS4x^JP&)_)acE>3I^E||qweB&Zn!tWi;~0M zQ0II1XvN2#I8oMbav5JG4SjLz?WPRxxzp|^Ctx{_&YiHUgXrtXP7=2Qa)tFwCrjTc z%VxGR=*o0)qiU8PdvSVD%>v^gJxP{}b!QCpqa$fn(F1iPT3AS7S|ejkxt+XT;mobE zt;fkE9tw$Pq!rju&MTfUM@2kkJ@QeJN=^+m>oQ;Sb_hzbr$37E`|7nmi+JK``W6>K zPo<&NB)zA&mg3@SqSQsAPiG)4yDc8&d&XNK8%EHAKKwxBk?#NtSqZH#j{cY94U ze!A^o4Zr#CEVY8EE~&!)7K$IYW|qHAPk${Ra_ccZv6Vl1G?A@11^KMo5Yk2v#!ji; zf7!H{rGu);7_AT$Ty*)6%5tnJMGgA(?J&i(TqS9!w$zdMr##~{eJk6ta+r@c%;&fm zJmRHjSsv=d3qJYLl~KPnY(_I6aGhwoaG-ROMe2a>4#j6>ua74*4c|mm4#~l#Fh5oM zpQrkdn!H`;AMa@0^l-Q}|Km+Xfvq&!*wnMh*49hM;}LgOX=rFRTcqy<#l^)%@cc=9T^CATAslQ67)=ULz6?KMlYtjXs&Zf! zE3H{u<=@S+t>;}c_D{3X=FjVRsB3e4qiuI78rvJWsY}HZv>3RE@dJ-PuG&pz)={5W z^l$yBt-rFk_u8bw3c_^AaRprxs|2%e`-=R=%q2+`yWVsm&?_ckfSUtb%ycFd-tk=9 zzm2kHX{ykEVC&jV`ktSaP2a=x{UW?widh);_%k$R3KIXPak8JBgM(;d|NYSQ8Mr7-`q$+!ucxOuW-eEZ5f-hR$Bt8MAJ z85OS29we>8e2gmob7u=KcWC2UG={*Kth)kF2m(`jDM6!g-+BahOO>9XVGC$TZ5c{d z9`md;h-)1DK+;QgD~szuYrVIn?CA=eC3)xLEHUZN1it*->H0Y~f>|sJ7BzX_ElMSU zx$A^4=xLky7GJ5wM|;P?|Sp~a+CY1gRSWgjzgP-(PTX@j)4l+ZTEQvrx0F08I43Q)#{9I2-eA?FJlVN zk3XR@_uiompbvMx7{T={&1`;KUC?Jt9GWEH&**P`ZTJ$L?EePTYY#jZ0q zNS2mg2sGZwelSwbY&bN^)V%wn{JMnszuDy>O;0a}yuyB5Fb;9o`5`I{K5=+ImbWQcc^oJU3r5AevV_sGWR<%nP2Wb=@%U>5L z3$eF#?mw{{tV55ajJ(V9{1_#;iJW9YanoZrSW5!}ev~iEXS?k-S13?3NI~>>rZj#$NK=c^VNQ51lk!rDAqH0l+L24wcaTc^B6V(1WA8aE!@{ zumMK$4{}2ELwJzh2J9|x9-JFMbNz9~!Ulk_yAGsDs;IC5YT&d7X_}^X#-qy z3NG;}2e@zzZ5r+p+Oe1m7abL$z#TdbQJ@WJz=J4yYG23xbVN=TbbEDu73cit^(hJf zGnF-yipa!p>NjqC5tDZF3Da`{MKu{=%h|=z>NVWsi3=k&kZe>glO`v`S ztHSxi&RM`hW!?9tOUV8MNx%fc<5G|GNlP0^|LuAwvg^-c;?wp~-_#k~#DwbQTl){R zdve5)DU?QryfSCva{!q)=)8GipXg&Fm-l~k=piKGfQ@p%MCIwKjK=#!^Er+T$MTAs z($9whp0VFXfl##bB^Ra|5}$EkBF#+jrpfjcvJk$?c0utM`A5i(#q5cy95^--;cy}< zAMWF$O?y$G;FtHO$eC_aiK!lBx`&(AIcZKSlUK@bI)7TAU5||83g|rL5SFnJIjA)v z2ndH~LDW(AePZQ1CH~?yEm@LXoT%WwBsi234BFEm1*=+4rG5p*rw86Z*-3Q>{T4zQ z)Xqo^<+AQ>#VI`^D}NX$Z;^VaU4379UBgTsZ9LQEZYNJVl<6oH3;a!~ZINI;kZ+a$*Kw4eJOiN68_6$?5TBsv)# z1rPN7MME7u;Ji4`ONyfVG&o?V1Q2ws#WIULVCsL9t}d{fsgVSo*56LU?X-ZPA_a-~ zdw)kNAO?7*-+p~E^B<}e>40!xcM6sL2U55Yxt>sb24nlXYB+W>VJC`^!hc5^lnV!y zfmmKb$Ujso5hVMMz~27*@8XvxG4X&KLX)pNk^id~IJN+SI{tM6+3f#1;lE+>ZzTMV e-2YP@?WwU}l}(&G~a_~>dGX%?&71^)*)4!pkr diff --git a/docs/img/graphx_performance_comparison.png b/docs/img/graphx_performance_comparison.png deleted file mode 100644 index 62dcf098c904fb9a4c3b97276fc3d98b321fd2d8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 166343 zcmc$`WmH_vwl0hX8h3{P!QI{6-QBIR;O-LK-Q7J%aEIXT8VK$l{C4&}`+VOW=e_su z9i#Zsi`BJe&6@JeTF>kVB?U=DI6OEoFfc@EDKQl=FsKkPFo+TWH0VnE4QC!07&xP~ zsHl>(s3@_LlY@n|tvMJN6{e|?5zAK>M<;VMBcq>_wA64;?keHo(JDqgy}f(N& zJh0W(eezF2j>FtSEmA5R>0RK13~;bc(2+#ojZbqVXSBp_Xd!P3*8ypO3jXp*pcsDd36&= zgRozvw4K4g(7FHqfyec0fP;Yvfk}%At9gK*CZRJ*<$J*f6}+XT-PX?^N+F%v7=tZTfs{;AzzEMGiDS@z|{B=VW(tb5{5~E zk!+;&=%35$e3{KVfp9y8aLem=)t$3^nd5!IJ!Kn?421#){!b$?oa8Dh+?pB> z4uTl$pGH1ffDl0N|M2y2CI}RO_zI$=voIKx&_4_al2_<|x(+2o+z${U#xUk2nEdpQ zf`J1zWg{Go0DBL!`olq`JamRe#BW=dkU#KQ-#k)~}87$;tXM z+P4x-z@l+)TEC(2oQ&eOe^SSQ$;;@Je|%?EK=k2)>h&ruNsqW zX`$DyxU0Wde4+;Gs5*7;YR_4abB>O&aZe|tqeEn>A#>6-{fSyQzX)aKg}{B->zZ0d`Um3OV9eZ$Ip-RV3`k#F9&FHORDyC% zq^|%rokJFTKFiv}R9G2nQ!9=hKE_3R1zVNfJ^5kh1DBFH^EdpI&SVej7-{CLeNz<{ z8DJ+=evIQrR!5mxng`dn`$iopEHtRjR+5EPnytub;_Lb1g9D4Vx`vjpu}EiX?kzM* zMmwZ?*$pAYC9+@j$AZ@sGD@`VVJ%(W3&AwYvrFpMz}H6h47C&KqSLLC=2JRx^KA?V zZ38N<<=YBVN;+(xr|mq?ClzGIc{kT&hKcpx<iTK=@*vVzmd!#JV0E3o)zn-#nwa~W zxelDvYU?IOO*PTFW|BfiZoNWh5@2!vs*)Pa;TGOdZMZ{>j&hX~z#(bdg-JA8=&dDh zGN=+~znhfCR@O9hIb0^nrbUyQtSU2!pEmacyz36xUwPAOHsq;FTrk79!rS0W4Ye&l zk^M@mK{`!?9PuPBd6VNT)TW%2+J=!%Q&;9(4LrTYlvlXU;f_G2J^vrkEW53#=(Zkw zX6w&(YCtO8tae7J{2WQ0jcFX_fz*IY2$FG09ZIl5tBpd*JEE_nWro}l*qLP`Q{6)< z4f4l#VOhw79a$gdRuk=acJ=~I-;Oc{<$-I(`^(CV{ApB?r|Wsok4~>CdT|F!v4Xcw zRlUSwC*gK*5F8lb5XK6`mAVA?m)m)}A~!b1=>y{T!a@`~PT7(RHg`TW(ycz-475*d z)f&O4lP;^nSD(jE>;{nl0YU+Q!I@XhENYtvfIfd3sf;l>0^=%c?~F0Gze-ugnlllk6|-pR;p(gOwxyGE3nN3Ue7Ed7 zqWt)x@n!lo1Ms$1eP`luNvo8|QJ@0j7hL@5oDv%0U+oDplIXqiJfk@0z%?P%l`}-^ z?8(OQ^2{>XO*tl|Tlkvbb7>lrpw^c=Lk&iCs&m=M8S@j2M@GB(1G$zk-XIe=Z=n8@ z?yqtKEOLT=##wZA)+rFHnnYtswg1G2;-LfYa)@SMORQdhX?J~37n_%8*9a!E>(<^^ zO-&W`-e|U*Gma=Ue*a>m>~*JUk95l|HeK5sE@4}>RNQ<&phh;cq{sWXwpIxi=biCPP9RUBM55N0F4 zw!kY$F|A5(U-sYCYnZc)@laa*aktszk}aPxj&o17U!z@=%VIfE@@`UhC=Va86iyzn z``STUI?H7_{liSo>z8~FI|HXJw+!!+h3evQn)2M)EW^58HL~(}!e*1-sXfo^g4m&4 zM@lK8tz)|8_$+^uz>6!7yZ`e@S{Dnq9X zNs1}0b+N8$$lB-jk>LAAp4wOJf$^gJ`a8pA@61}DUHM^NOQQ!d@S=(!8XSffKjb8( z9|;h`7ns@(+2lKJF*eRv2*e|MKga%{n|?Yw^;d-4)lsx-cKjxIz}*c7v~J-6plxP$ z4)5$-xG&hskE&W3ou1%FM&o-`R8FOg*hFjf7bE&3H2CmYRTjG6(s@P``qr|uavq+> zM%r8FN6xiY{2oxOFt|xrSez@Ehh;82S$mXe_Y70spUUDkm(5B4CXhjXP~g4jX=U3Z z%=R!x&33*PAySG0(2B()2s`Y&cP(w}By%p^-4J!?N=ySfr=^)5+SIj`ilIhXVFb#5 zu&xqX+As{`k7E>gln#$O@2@qrZQxB8WBj<8&@?o0JY=|W$;bm!%Wngd`>CAm32#d` z-O$ok{QLC2N<*dQTzZV!J+hsTw=Z!ixmIm`1nnL}#|AEgh0lrC&@k6Vdc=B9HRa~n z;+Nr~v_T0jkN=ZF(;6o^_0ZxJfJ(;PuYKG{PI@&mcDObtW^t&-558usb<02a#rm5a zP(!9M>HNgx@R^xpy?4HrhYJe42mNc^w2MJ?uECF**1NW5nbOrc%3;f7F2$dkz?$i9 zk1!{3LA~wR=}v+`d8#dY!Btu}cOzi^$%2VlqwXavq7X&6A`fqxj zoVMns$WS720UL)4?d1l}>fLso3E7l-m@ijI_SGtlNp^b@FMh%T3XBQBMXg!@tHe~E zC<}GQP>qCE4mu6~3%VNg5=QmTKF59yJp>R|b`(t`%KFdqPT8Hl{ znMu35xUGC26`l{5Zr7cEtS9zpn;YfEe~=4oa#_p|b5G@3nzpTJg(l051Y?@Iy!}j_ z^5ZL!RCdoy@$tQ|S8=Ms!;CvP9iB+8+_hbQ=^ILq{>e3cZ=s;AYjTXe{}|Glr+VRo zWTSZ=&WmAx*a^Qj9xhiQV{X@_qN`f8u&_YW+}teVs@|p&pHu*^_s4?FOb7bW$@;!T z13ZXZe3Y?9Ma!bEpu1g7dePuaM5=JqRi+vp>;+~taqPxlUbu2go6O7x(v3UJyoZl+ z6pfKu(}lS6y#2r4(W@LG%Eq*no!#;B>&(TWY?l_9x0p zay~V!TvUDHVyc*wOyad_3|VK=7Y!?4%8DfRb+bbLT+bX)mD0pC>vMdxxlph&nF_i| z%|!LU*dyfXAPl>FGf-6kNtoA)=+Pt`Af2qLqh7SYxkS5IC^#Xq7+-x*1b*V3) zsZ9oz!SG&yrfdSbV}Yg~?ZPSLEXYYY7UjpH1}*v1s!Y%ta>@F$(F=aCDGI>AmPycd#ANZ1}{kCyjdL&`M(`nWFTZAl5J~IXmA$b#u(-;5Q-d zKUvw<_@Zi&O*h|0qS5xe4INBTMjElgbo`6%X-!`D-3-l72l(+I$2gH+h*4yqni@Ji zE(piDGBJ(gSD%rvrsaGZ6E#(87s5RyeC=XXG_pzWQHueL`e94Q({0GN1fpREgewQ$ zxQ$G@ztt`DapDs%JxZ739-iuq zF3x3dWM|eSYiQ82yWRHxnLu;x)~tdB7c;3lNWRZhQeNQY)}_Y2jF33}!N|LKV?29> zDqKa@LIb}2N#t2<>p^86dNKt4gHNS=Nrx^~p&-P+iJML#Aol&b9kq zFHz+{=_jS%*-e^`bUK+OJ-l%=I)o|Rx^m2yXxS>lgw;T!D9tUvy6(xnhDg|&iGKPo zsYIJ3ohY}p1Duw^W`#NVKqcuG`9F#A%C=_f5(fR7UGovLuAoa+rqk-cBJQrG9T8H3)?H103u$shw`x!H*b4Y)KU-NXd;$-V7>!3`w z8i^W~VF?!()+-mU1O64$9u)J86(L><0WB+w56aBk8lu#Y%o5ofe;Tod^w?XZR*AVU zrx{L&tWHIqve5nWdmYIs&E~9{ef$jNA85DVs3VyZcBU0_0 zARRev4N(~$`8-ee_sdh6x#kFyAQhu?&S)!}A=WzBAl1S00j%E~?H6fpT&e0&%g^}m zlGD#LhY%!brb4>k8(<&`#D(z3c~Jq280!KqlxAGHmfEI89h zuT^hIY=oc0l}Xvcu(srmb0Atnx|Y*WLfT+5Hy7ue$MF2%QRClqe+qm2^b7$LcX$dM zO=-^;s921&eg9cebAU>OX2#u><}BkYkds`rtP8C~i-n`con<`dtt z@&1a0LG9v)*6CoWPHLh9zRGbGa&~P{<+)7?uI~*gZoD9pI^q7pCMivn@t@)iO^ZGJhbY>nwY25?$Y|}%{=3*ONMIz zv1#mOa~I`tZkDaBF%{5%@u%3N~Z|2a>0QgbNws zT>kIIV*pk)kTO-WdzR_t%Qc;vLctRyg5I;exeu-h6`K?7F$*d8*KC5Nh*Aj{;~HtV z{#!ykPKM&oSvm_8^+cEo|$rX#A zBc)M&r_$i2h<`M2t}$+U_)t153R7NKSa`o3D7x^3VXToC{8jd);v}^tFfvz0Rgrw3 z?`Y$w1#iIr7LTP_I2c% z4V;tD@-9tR%8`UDs!Et7)LB5dwD}#42S8E9N83cMmBQ|3(~Hu#yVjZ((kek3Y6H@`(VhvI9_pTXMkX2&0M`@wL|H`H;bW>CaBnJ9HXav{i7hXpE#6Zx7?&c1M)-{D$TdM20X3%fh4&`+q5dWiJ82gWmDZ&_)YsLAwol=T)yYp+_nV9DXWm>A- zkz^{jA;iAsd8ubn4nk?M%hvhu%;2CJyS4VHq(-0~q_6u#rtTI5p2w6B9*ddOX;0wc zjK}HVf^r<6iI(7H<$^^e7LUp`mYtsq`DC6zV3_-fSVQS2yHXJ-6aX1`g{)IYx`q=x zzd44yP9lR?pGj_VONg74?|JuR6@5}kv0mnuj!wUMD9s_nee>2$oZj**g(V#!QGwJNpqX86=e!Tu|W4V!YRukL}Y>yXoYx<6Z zJHmtN>R%C|L~?~}HKRzg%Bdm~KQ=0VlwI!dt#uiGUY8DK`%I~BL8f3mZ8txAc?Wjeqng2G>QYi*D;2H3X;fw>TsYlA5tuh z+FfJdsQkbrQ)?5aD%^xQCl|3Rac4WQrj^HEjyo&1vDCprGE06}zZbESLlu8});$oq ze0{9HRtbo0cWv?6WVt&mba1Eq!E30=pC5s63nysiZ-q3_P#JvUDX*pYrL@&~ti1a= znHFK#QA(CTbavoSWhr*krFbd}wKY>-?Co%C(*ZJKPp*Bo3Fe0St;A2nNJSzEyS8lY zyru`bb>Y0|zGB=uQ0uw~MwKrEeZu9dYHs_kq+s2}UCHWrqSa8gYjV1}7;IHpg<&B6 z!x@H+yBJD;N-jiWr$F(1Eos@N#dwEO z)uZ*ks3!jD#Ex^m+_@7yoSQa+kVoT8Ptl!TSI86iM#sE2X~D8gRNVRHxcFA?>nA7( zkzQ@ZU#`Yo6tjC3*KB^`s2N0F8JTT{wriqQ)gm_=C7QhY^z#}Se!d6${bt)mH&6Lz z?nPTIsNwn%;!P$Q?F>@cab+d$a$k=Z*)6URXlkfSnsg=S;AZpeWOVc26Q~27z7rMT zl0s13H&$^Y2@?}Topg;Z&bxm_h{?$%wm{CZay;S4?C>?HS`2<^G1m^^F)`u#$z#U# zny{$vtU^(vfFmaOzEfG7$}H8BAWA6Vw801iq&uyLLsDozsT!0FAdUkkR6{8fBF={@ zoaQp@jEIPhIaV~euEi&bZ0GUqY0c?eOfNepV=y5oP*0+)x@9Z4kbsvQHosQ z{b#WisSaO(<81j*p=jDQJCCG$0CGB%EO%d2hA68>n#!@myw&ds%tGf4G5CpH#oyWSnX+-k6ZrzK!a z4w%y29_Y?Y*Uxn0P3kyT^-SmAERgAs+lKmgk}Qe~X4~M6Hl=`_i{E1>72?jcoP+E|XTw{9D&67gl)>;AN{Ln_ zymYpNMRu;T=3@b51}0do!?2w)^n5aiY|d~ z5d@07lnC>wm{MWINoNSsrx{Ia#*7xL#~cIi8`ETm`FYl#;3)fp0w~`?8iWnHgUK10 z&m5~%Si#oxihf&abq-#!lCNjYbgrbcjz_D1z5Th#@$64mbDD{W}k2cWzO%^Y|0H}!VePwZ{w2GjL6(?nzsZF;G?;-X5OJSw^dWiXB90i zsy0OH7VFG?1D={T3=f{Xybo^OwVsNmq)Qo}Qrh)LGH-mu0Z|KWIJ|6)+9)^X3e=0; zTf4{X(ziO}<{61OybOtn5+}8nG^TdpGlkEJMJmRwNdxT}tt!8i>4hQf+Fd*EZ4_zL zIk4vDaQ8xxeze$*v;{#8MPr@l(lJ_K(ixAJmPF_34A2VfQag?r)qu^NzuP`zcQJ<4 zeKTnA)0febK+mZ?m`&ncITl~0$DWcG|LIMU)?&YArTkP#mdV{6Q`hU5rquXVeZctb z`}Az7WnL=**|u1k2kj0fH(U0R_}+oJvU#_7ip~7)HJj5r^c=IU%=P-m%l4}IsLh`Y z4QXG594Cgnv7+TiuXZ+7sp}f{m!|fm_QRJ_<83dz9yy;%Y!ddKM7Zx*vpySKOB=Oe2Scz47meqg)|{3;QDTlKQykG64;JOsn87qz z&u}Z=XmJMt3%FuB*Qej z`rR6hrN07F*`V1O)7JA^Dxy{iO zrEH1RN(wBL9`Rsk3`$tO>VittQ6w*i*5@M%yHQ;}Oj}(ANWNF)xSmy$a?g3zEbGJ6 z8mvB*etI^I)bYV|tH4S{EiuNmdK8c}_U z&MiyINrm$os|-qHI`N?$z`^5D$zh*uOpf`C^wY#@!Nmrf>j^x#hLV!~>#0kMx)o-g zj@&rqD!#+_)z4pPS;Zg%nZ7XbFb~ZMreJETNr#^2+wh<3?LR!fWu&np|MI=i=s+wj z32oMZfi+L&D&?eK?VRqC89^+4((a(rJ>Lo;41RmRh~BJ%HVG8+M!DK2Bv|x^*z92# zP_k3?fhalrt!i#Z%Fkq){!3>9H+aDjc|FBvZs%$MWs(sl5NG_^pn*6R+_#R-2@E_A zb`Sy2oN&21|C<{Dkkxm{IZihOQ3Ter-E;{YLcFA6VAe@ zfCQ$36q7F8ChCN_KY_74p?($bLeL4OrU%U~fAXD=lp#ZAnLgXX_f7ofPW{Hfe@w$G z9LMy>`u?ifA~u80f<(o7*or9i_h5AKCyl+i)U^yM&n2lt%#_-}XeQlHrb|h-a$X^a zb2ikn1NIyBxN2IuGL;C`az{!J*+RSsVXcU>x%%xbIb=T&TXXDvg3Ao^&9Y8i6tEF_;IGl#_W+gJ#rpL9;Z3W_R?>;_^7w~wUr9YV zGusDg0EV1NDui&|v2#V=YwLlf#WQq<2aWa%QgOTEKKdT*IqMai^jIv_#R~*9HI)St z=8SfFUC9(NI7Xb7t;t8@*(!D4Hx(vu-{^1Eg2KWH@!puEAV><+jc+S@X{4lC8A~_& zSqDcdp+>qE_g6YC0!mm8w4ZoYP#1)EJyUQLTNT}lHqe*}wO%$bSD zr<%fBZd3Jza}EV$USe@v$udunW@07Nl1=n-oF?+MCr)l*t)i9}^-KiJHN@vknb!)wM zlG(thuiSfL@@meRou=_Qq}F`9SYB)D)J-ky=Jm zA^+q!J|Ph4NrKTQo{sesa~&>@JZ4ob_3LX)+?YCP$K)o_=BXZaUR)B0My?ZaQh2d1^BgNqOm&*1Ey1R%)Wy3641yjWYCB&jEuUXl(zki$2R!Fxv7xqWK zmf^z8%K~fF#9#`BBNQ2D71uAVB&1gZoUOaw)E#2&hD*%%!tD3ebdA`>YH$bbNmDEU z>k-P8Qtl>f$$3fD zdCK(X$Hu1ojG}{&b3V3b>1C@BpdPHdabq_ zd+Bx&oLj%O{%XTUCCsbS9E6~r_4}@}+gRt0F+Iy|8iQ8_=h7`wT!%s6A?Z$+|BzSe zF?&cJAuzo(GFXO<&)SCy2!RDx%v0~uHs}e)PuYMP|Beni%)9IT<$dWI7{@c_~&^FTr;?98R3|D2whE* z{;BxZb33}$Ewk2|Zct*NR#^5w;!cqao#{VH2qh}~6XH@*Tsd}P9!-SsE0C_PIt3-r znQpuZ5bs1|$M{FXu5zk1IYVXUB(ar}##w76F+bV+b-fYE1I>xBrA*zoDexQN;WJBA zS#^l*axOc~tz9SjlHGT}Z=L;=itVHjvn&+p(;3^x!B3_(LpCzE$!69pWQWEIGMaE> zp)aJA1>KEW((+6KGpTnJ&5Vbao*)Q)v| z?n(#>AYg)S)#UZN7|OltWd*2rdCvQzbdbu79pF6Zy^)-6_B(~=f;Y1&r8C;hmya|~ z@F$r7(mn>&L!Eb*U?V>Es?M&*^kYLcqc5hn&7BFt%CN_*qpTwqRFII+NBaI6J~lji zq_axS9qL)ln&bA)G;^FOo_gerHuD{SBg)7O_UJXi1MfZ3;V|9A6m`SXo|7K3re(9h zx8w;9e%|%Ny&nn}6U(wJgp;SX)o6*V$#XzoVhgI3B5rf%8kNMXf^^-mDbst*a%8aGQEHik)^r(-l}@h;-8|Xi7Ks2x87b7@6}GdI2LCE zPb5w#&xW`RwKhZ~kpUgzZ_rANhY31|=O=Qqkc1VaC+p+4LzWx0Jdx*J2QlBNWtz^dUddrpNgu!oBu;g$VI5pPk0+2*AtV z#G3dX-PxG4Oj#iHjIYgr(*596ME!I&6yExp!luVAvMhu-I6Ri9>|!(GAFayW1h-FX9`W}B`kx{_Xqa>fDISG%`dS)%T7FmQfR*iAgs088L&gL(hCM~-by zEaOv_)x(N$jlh>?fOq1M;A(>`CGQIy z(K0YK z;#5B=A*PB?1scehy3ViNE z>tiGM+`A@(IWvv|_=Lg80t2FV1%jblp;iSSl3I_B0ikg?8$tg7fCUo8Rb?>X ze-C%lAaKqL56#ngq3kd~eWCuNGkJ098; zYF*hM$qE|)6%97b2QF$e&IM%RSNUQK6QeDZl`4%s{x&3$)t*4J{k0?z~9sSq3sqQea$e`&fz2Esr{K)@V15^2Kx%HTvBLIsfwy-|+D zZ#g;U%p(|;G#+}aCQJErR8uXh!>?45oNh(hJuh7FP`>4k17IXUdv$>#H2n*Se0%Q> zk3SF3hM@?%@%xeC_>$5V#kFOcosS3{edn7+rhpfi2Fz*$>;p+>^rtKjsnlf=SIN_G zlU=GQ-j55CRT?!os4szF9m}@Bh{7KRj-NY~t|uh>9U$$XK&;%ExdkCaMEPRh$+#0N zQAcJaRQTnW2(MtF7qs5=vaTo21@`cU%B`X(YD)s+vDI#OaRx#Nk__mFf9^&gh5+=I zyFhK2v7!ER-+&p&7Dw6Yk^iX=0t^CG2n?bC@MF~K3*fJz|Bvw#)OV0*ZAA3XbAy}% z)OheUX;$o?8xP`Pg^0=Kdj2WOYMSfx77Zr>lZ7k5+YS1nTlU_~nUd}X{k1se{<_Y+g z+bq`tdt8^aEoap=^~EbHD(-qd-WaX!g@r+PZ=q^+d9k}*ZV}!NGi7kLu#d#W#FWtM zws<_69uZgYMD#fg#-=XU>dUrny6M8Sr6epygX)xBB;3D=s}bOH)d%Bp+4VR_Ti3a( zF#v;zS3I1@>t4&_;Y^DP@?KET&jZ+Qwx?WlTne;?^6!nrmAzc{7-AAURlxm(h7#&k zgz{J;z{C_6{CNGI=Oo5aAe3)8o$GS8(x6l>)3HEOs0r|W+;l&OAPjw0$nWmndXxj5 zWd;bXpzQQ5>$`7?7fHss>EaB=?4^kLzwQxH3kc)_fx!OKu5v2SZ64@5>PM=jas?Tz zmhJQiyI9!R*y!+~SPXhnlarIb5W$;a0+A&0OG-ZV=Oei(ws9jOA`;fCUvN^7y_FLR z9zY`V&0OQ-4%7Sm`Sp!d#|wYB6-gZiJP&Xc^&Ez8pPy#vw|eTc<#Q{K#}33eUug&q zLngX#6_EOuQQD#YxtmwCK3S+3)P4DS(QDw+S2tDsBbR?6V!t8t^0)}OC%-NhM^HuJ zdJz5YdW2}c#cutmXJPV(amNP>tWe zlc2ygX9YcjSm3a)py#-3x-QS!)O8&=OXvB&w*jm6x*oTRoTs=)4qxNJ|N7?&`k&|1 zy5lk0y5fOv$fs!vJjd8<*6k6c+f0dGd8k!+Y3`LYadWAbTScu=wjyL$W`@4^i+98L z4&@x*e*gLFOIH36NtB9VUY0N0&~?Ci$2|!eM#e`PJ4sonKqbkJA}i#-4{{-f@P8UN zeEotj7f-3)6ZNtO}Jh#=V$GJ_`cjL}_ zbvN9S)dV`3JfUMU-)1W@8I?tSlXUVD6VYU3WD3jj{MHViNgzQ^{efoWEn;hgNAe1& zSEYu64-9W12_gE|LIZ@3(01x}adC0|>UO$)U#x~AG5w~oaY129j3ZFJN~5CRA_k)B z+nebo(HaJ{^|hxBo+k%C*<3c7WtN%1?F8%HrT>R+Y}kLo<`q0Bn!E;~X8b$K{mBCB zutHMvIBOU|FuaCTN4f#Qjc^1A5GOG?H@B9P<6*(7{jks4#Yv9yExUu)@_{$y!q14N zp)vz&r5c98Tp_xL^z%K8{%Fj@bXRD&$ zkO&Kj{Qd+M78O+(_V`tjc(jbOx}9l(tT73u4Vv-f`Tq16Gj)_6$Ibz;(;s9)BM<1&wL-Rw;=SD%>oAoehBp= z&g)^-axsP7+}vD&w6T2wZ}uG#dIms9PC6Y+V(S5{U+7$AY9L$Q(%=z4ebXSbXp za3V)YRK2-vx0?OJk|B4u{qe5Zy-@qIkv4YU_PbHoMJ4%C)V76`&32ISz6BEt>*QtS z?{@wj{wP;(cKdNU#K`~tTnFoOJ%oKS2pkAQG}u?x)Rgr9GeC5d>wV!U8~MNh$KAa+L5{l|>=>SPhsF5Y%Gz2AECG^#n{Gfr4?xI#!jLTCU!K|o z8R*dc6gC`3q}aJOw0zYIiCpf+yHu^cXQLrdV!2msyGM!htE#RWPJ)>V5)`myj2JhxH#J)_I}3i-uH;)u$$zgtwaO=WY> zXL`Tbp1mXp_$prOicg_Hm+tqPH?%WC{Fhbz2?k~R&nmycf8H+}DuQI9=;ZhKIkkLL zjaC%;*Xv0!)B1v8&`-CJ3AmKTA#_my-Vk(bZ5QpQf5VJXeb%Gl+li*jIDM;CSy1=e zwIuTkyb8cVTNsEI2_6_v-b<@j+jf8#%w#3TraY`I_BJ?1Z22qABxUBjD;&)WZPPcz z$}*UoqH1_5i!%-y0Y`{bBKm78-_D_8HkUC~ku3HHVjZu$z-T<@GaV!n!GA>psa}5v zIwD#$hEY$?8(GFK42H$?`uo@8vb=@5;0uoUuGD(eP0C!4{!OxQ7*H7-pm~RZjs4P< z3vwG?eZQyua|)rDh=^|%;s zgs6tv{?7a2r-L}#GYBL>dM$`mH+Vc>U$ze`J%33EyTq$_#C<>3=v)D< zD)tjkw)aI_AOpk`2!sJ37<*n>x*pbSc%puZu%9LPUBCx7g`BwxlyE9HYe}U$=u_W; z)?G>DgT-i2Ee?P3(0Mz>n*6x>Uc|N&BoOAj_4%uT&e292YmRGbeE46341oe~=>r^M zkT}xv)k68p3Dtv8-Omxj{=cbRS-JO!bU1+Au+^S$aUonW0zf>|2dLKXNHt@Z@#nk@ z#WqkW8*8^xVhv=NykGQmkcYKl(BNqW*yapu)iF<`yBm**v8H!RKI^%kuZh2YL)n7zdtZJ20()uUq3FKy=s|P(s9|nZr&-RpUq+})B8yzP2UX{lo?5t zNs%Re1Y76UnUSprLkFy+)kp;QL3<(W*C&S*E|aS&u%IbHw~tZ9@DNYNcoegg_xQ1K1Cj4gL9d%(-ZH{vsXwO@bfz zPN}_7x3<$zf6?C+6dW8Jg8gdQ3kBsYt!)eI{CuwOl04^T@p4tkzU5leh27gqR!*MZ zb4IRTf6Glkr~>DfM-its3F5zCVN+AnA8JZ7bQ!2MQ*&#oW+K`Ukz@OO29O4bccA|! zvUb$*N2)%W5oLMr`x9|#Y3aM8qBx%Rvs>;5uJX3(2X0ogzxO5vY^1rM6jv7mN*3T8z(i?^eq=QrtPR zHw|by4cNgVgz_QL>J6oDMg-rPsVWc8W~8t@7(qE~itlzJ0M#Nc{^QSx;N5H6$J3;P zUl9?E$sq`V&j=E{KD73f;Idjk72@zu-=8keOA`4OfBx9TPUw<=h9G&f3?%#;{sG7k zeL9U35;?awH$QDTTH(`QE;{0J{og%JAlKfJ;`kp37@NcX^IHHo*Vyi$)E3x_9{*DP zE)tHuFUcx+81c#$sAOzR@&*DZMevNBOjj{ZO8uaKudp{9L;4UhMsoX0A9662 zBir+w;(wORUFoB(ogqv-NYik8FzJl3=y3Rclh$(tLDF)Wo}GQx+DT~s7aD>tNG_8e z>zPFjd^uJqssZgEVVDqcjP$x_rhsod4GIvM5zo=lvB~vR&-#7#9DDNpczZlq_Imf)nlW+}#>%FsEYWisC5P^?seWko+E*CU(U(HJm$zFA zIY%nlye*_bNhJMvn)Wa4_6^0f`)>rm3NF}_`liQg2J^3?a3)fy9+Bj`<3oYyx4E~+ zZAE!`J4K+5`pP%-z?tRkE^kh|^_IzRIwaH}lca?DEpX~RzKg4v*b)8GmE(+a!TJ{` zwC@DLTY*%-0OIMJq%oL`ChB`Ug?6#|FR3=YH*g*oefS9r=GL#0aQnSVdVQQvAI&yt zTK&o{vwqaAIUi^^IeUNWFTwloM3(`kHvEc!vHG_>63U0YyWAF>5pNpH>$zey?9K+^ z`iRAvp#AfM49i?=b!`7^BglG+qzGt4yx(zW@h~7k;UO&Zd|UkPZ!HSZMZ))L#y$B- zPpAASSx!I!I`Lp4LsVN^`)=C*oi+ai(>)mZL19)-!8goGpzRJih;D{X>mH1x*|+q|H>7a?e@ z6`$whGA~ryuS*FeR>l)f{EB%mt9sVmKlw`!LK)l^g_TbzzU zgZL_-Jer#?hZzol`jvIJ>JcLzO(ZqJf0x$fJo#5%u)M<6ArZaeeRaI|&onSPF$MfX zi1z6p%xr85dV9ak@_={Uy3LJs&Kdk#T6p*cNsOx*EM1^m^LM+5Riu}xf?c~lb&^mg zF=^vgQSiE)${hblfKKk~&v*OPr}vYC6Rv^g+UM=gr_P`UZkGRfRvmOQ_c$0|IzO+m zvb3;06d(3>Z~4Z11{NjtdHzhy@2+xl+_UO>zhJUeQIm?K9@!So|8!hiVrk}FJnT2J z-h_Rix~L?0smU@CcJ3Jym9p zIf7quh6_NNB%%x4cm64hedjr3p<10!rZc&{*D{xa63|)5c51UT-n`fX6Q)+8w%`rc zMh#^3+z+V45W-MR93PfD5E}cD?($y|Hgn^%QA}?x3d&|n)d??gEzH90L{DVZB%y)Q z2Mt-hIdRT-NyLILJa5O5z@YXOzHS~Uodm;Irarr6n<7FL$!<&5jocBtjaNTvHCg1# z^l!A=H&QTF`g@DWCWXOu(7G^}=opa8tq^`<4FXu}iL7PO7@?h|_G%S62dtL2TxuE` zlHA(tx1vFHn-jr2UD@2~cs&2Pp5@;r z$Z?Dl8w)F!_|as9gk<6L_m8?%)s0?hmzKQ@kLO>4Z2#*oLuCN^w!X(}Z)n=QF#R7I zRj#B9LY3?XgurF;psy#gzOZ=grsJP{kOSG+*~4h$_`)1;*@aKN4#g*xFo>YncAeOd{VKLi9ChmN&B3U4ZJpWAsPuJsiy!=c~patR!06f09 zQvN;keSfrPFoCHIQKW);oJnOn{R}|`$k~%;MqxkFM1D%4vo+5%! z*MbGrNy`^)T-NHw-l<%PqgBj>V8SJ#_^2I^_%GA%MMiE-Ws6)ixJdlU&1pr`G~Gn9znlIxl=^z@MkGBFIeqI*kj;t-v>bu`nh21;gW3FexNtTLwqXB$R=W}Pwx-DiQ zHntN$k-Qmk1gPC>j_3OHlY7?>HccCFvwfGKw?}nD789*?mc6?*AHnMNP~D+cB)+Z% zuv;yO$!N;w!)+PCjXh$ZB$tHZdcQuH%2aXS&-?pkB379SHSFcWo?;#MAXZZpZNf}F z8LCdcrG~b@EeOCBmqs@E7&?7sswxyR?WSgm2Jf_Au6ms1UtT&G;;?X6&53NNh}=B) zK{=@VsW#2af^h*5VpNpuyb99BjeR?i_nEFOEiTRmxPSw_!E5CIw^DxrCOvJ#efstv z`K9n_a&j`OgwNdRyQC!92CR(ghM`kM15~N~#hjg-<4OHoi9f!ZKc~;Dr&G<0n!x1^ z3BUVfiE8PLyROr$>Lu0V^VP(x!~GGspui-Q3{B7Dgwt?tQVSvQFes_9!NkUDNfw#! zqw!W_s_v%~z6M~Wv}5WgTH&@>Y{W^BWN&U7rm~rYYl|cRZM+H&eaFM@pOSSouClit zbitzj4YwnyuRz(8az)5h2YIP-vN9E#;9Xc%rJmGtv-7Z2qHWgsX-(12eRm~RKFUwu zLWg!)bt`lZ{g3VhR#KBJ(1SNBlK&?X1>!t_4xtz>Usq9sx+dsa@jj# z%vRj5a6)5yxgFlIewBQt50jxEMZhV|624&QpTE_K1}tP>;0TQpxWqwJ^z1 ziMG8LZJk@~!iuX>i@6UpE?9)#K1bz^W{=kg?z&1!N@kbaLyw;;Zw-VSzWBk;13X;` z@n>8sOFvkRx92^<*~D-+N!i?xD$)S0A{)}7s5rHKsw8wxfywx5&_ zk5cbj>HSMc8>V-J(JVCsvF2?xc zXC2;sca^m02+uOcFbnX-xZUqiHINs@1XV3bPTiV(eozb~H{%DL?)4(2Hej`ck_QS5 z1JVaJ`TFPW5jW&NqTy9rxdpg~pTh0t$_6kkh?pI2-l#rrsPw{oXURNjG#SBt!wV!5 zCKEyI9T+&8NX9^V3WAC$4xH4SNY2zYFdZlaqmv4hAv9hD^KpY-vBcBgi@MMlKo|~U zNZ>Er73;`jA!m1Vdv&ONF~F0>EM?&zgS!BB^71DIMa67E+vjETExa%B9>lUFXZFqA zTUkKdFka#7Eo+@JLeGlSt}@X@oDhpL{@ooNi_3?Im^^nkMMcHaPX|@@RU(pns_%Ug z8AdeBcIAl`@&H!l*YU-XV6BCnR&Ys3hi~gv*MMe*RozV&JgLg;poXpt8p!h^sARVb zw5RD+UOlxd&x3zx=%70fht;UcrRMRDV^nS9{{rq$p-xV}q$J#ca$>!y{g_TY?V$C4 z`i)i>n|;6&Bvm)5wfE%u`UVIFf7iPLP!$0hTDAk|dsR4<^`eSl)f#tZQ@T*V;o&D_ z=6v|op8-+@joQsGRM9J# zBVcFEgkM>VWO&RU;mg-)+*813xK_LW!FByL5TC9OuO=9Glr39zth(f{%aQ;OwISyV z%W>(rJs}XGI`Q6|5l;c!hf00zSvywyp%;R>j(rdSAfc=mmgjk-LJ^5g75IZ`VlAX8^jK5q^R;o_V*^V z#D1UZds5K?7bj;igWLSr3teH_rn2HQ>>pT`_ufIONCQKD^=oK-lY2X*MD+ErQEG-C zI`6T2M$}M^pCy{r&p$L8HRTwjH4;h~p^{LS;o@-XX95vH|E#i7QNKBUwOfavqNY9^ zIFw>}HR@_oaHHZ9@EIU>DJqR4GTD-cOwN5yPdCz*vGOqQ4aMwbVCJ?1aRO9+z!x^t z1A7wSNqRwDehpH-(2yyzXvy?T{@g4W552@~+-m8El<9T*o7|EFzWQZA2<~^e!V;>h z+i^B#mvmMSeNXcg>pX@7v=WTCA8IxN_`qP^*u&!oV8Ta{WSEbRan;@T=^iWvjVyHd zbWj9}y1zPTTG@U>5qZ|iX#I^SZ&cz%*(}gqRn|I$i4I&wbZ6v8_Uk>c?OMCD`imp2 z8Vp@`Rl6RVV8?62k+OzH5ehNSi~?M8PlIRRx`sImZ=9ij5yD2I9*cpimK#6;yg=hN z1II)W%&|4XkP*!BQh)eHzev%u_%#a=z0oFK#Q&h4xIO@{-4pw^!^gSzh+gJ1d5=|1 zJAEPDlb&zV`5hh@s?_TMY3_Fu>a8%oRel6>OK1|au5?)o&Sop!@Te%&Ozkm8ip-9qiq;t%A1xwKqlkiRCHOMD>mU9e-4CpU z#Ke4eN~1rPWoo#iouxc(efhYqJNX#hiq_WU&Pdt-(L;(@6(SfIEwK*p)`zZ9msw&M z2yPI_ith!UOa)*T;Nd>awBR6Zdg!L08IY@#iBIc>@{UJ?XML7U}5 zAawRRZQiT?*x$K6UwrND)T0c!5U%XBMexPNb~{Y&;SlnI?G!9`EjnJATknV zMWL9h%aF3#>Um-Ae3X4+U$ZKNCHt4~!NI}dF!>1-a^&lIh+wQ8jjV`M1T*+tq88x;yu%2(aAR=ryY{uM)2Pp<-x-FOhABeLTfh4^@oVQS&3tc@K5 zUdofFuYs^Wkq02HAW8QwLXb|3@{V=?6^aL1Zc3`91fXV1wm*$8p%IFVFRV+OVPHUE zugC?u-qDhJo(n6`Q&X!1JJ86c<{{K0WDet6poU}6A3W#*RKL47uk*3OIRBl_3qfPz z8$i2Y8!W3Dc#wA?i0H~<2C#_o2byN;V@I(GWr<=)N=r{?W&#+8w)?wb=Mq)qX=BYz zd}`{UV>19!XP3HP;_RL(cIjYtO4=y08-6s6+xKpZ1Bzfd1vRfg@|hf*l}8k}rB?xl z9z!=-8Lt#3uwFW0AxUb@CU9)k%e8hW8s3<#*V{-Maf4^CJqJbqt!NE>y?(-^W~ZdXFb3D@3HC}4g!vQ{sWqM8!hYA#0O}?$f~9@F@-`ABND_WozLuDhsfwSnB7QeWhBxN_A}}Gcz_tHi zNZ?0EH-|oF8&`@Ck)2SOPl&j!W~N&ct8i`(IDp0C_?I3UBf{f5L4g!p(f(-h@Bz@l z)j0qukU-Fz?_{ZQ!ftF09?bVNM3XsFyMu_EJXWCyxPV~Nu zb{mDtTC!=CWy!Rr9ZTbm*Vw`P@6!kV0SC|%cIjCCK!{WkF^{+Mj9AiI%omVooxxy#<%j-MG zA>Juuvygd!e*O;-3X}aG#Xb-d@|O8uMm&RfAA`|+MD#n$Oi=CVQf{h z{?cGqu{c~;rTQOZ1Wv#D;6NtgT0}lp3+3#UNRv<=nB6enbN!eQ>Zr*8(oOZV-Youc zr&muXCgrtZL_~zt{OeTA%1WF%hx5U-7!G5z^vDU0Gkquko9bn_bcVaVa9LxBxPKXo zS6p6Q6U@c6eH-q7TY&!a>3x3ab{eT9m)R+Q@yfCCE}E0zZBc&CJ)T0Jtk zE&8H_hzSgI4RxLm4kwf4<;TS#vy2rjhxCopPA7}1^$fkYL4XzFhh$GT9WLX=oSVV( ztlbzX{36T|0E*^nJ%IfZ7Nq?S`N|Ooy^@BEb5c@L{>4Fg20x+TMy9zR7Q0Iw4=Lzn z7lG_*C?iMWS&vIVCL2HBp|+7auh-)>IWF~ug}BEb+hut`yS82B+k72a+c>VBP*a^Z zt_(d6lD!hyJ-dtr$hvnl?h4!uE%+4A0&ovtY`_?Q$>M8vC@{n^lUaoMt`_79_KHZr2DEA^igE4_00DWL<*q6bd7{|**}ZUvbTl3&6c z{nNJPYzn+FbsI#6=}v@LDt#~TH;3@rx-u*p$7w)wl#rfYp1G+S$jB!E5D?zuvuI25 zmR43XxlOe>iZNCz8@IK*?zlxG-E+co{qQ_|s68RmQ?x|I-A7i~)}xZOxgW7GUAR5QhO+DWen}UH60h6TK(EAS3QnU#9x) zk3RRManFq@Co0$Y^Q0)cxzr5H&T(-h4ectNUUe^*@bJh;*wH8W+ufc+{LzTE=X(Y_ zH^#_msN^WbBym8!%u;KxU(_b+y$b}K1jM{*9sGcEi)01x6FgQo{GFZ_0qCff#mXwO zov1IXlgyZ_;sBvofJ(?!6ln}_hiYnS1@7|Os*@bps-lOYyCI&2#Zl|*3f07>~TE|Qd#Y@0Z#M?b893Tslc&Qtwz4( zTDw^Y1G)F>V}oS+)-;c_bTfbjdt1J-5hkS@W7(`Bc7np?(gC+}f&?t$;rR zWzMZ;0Cew`h*Mhg@WK1)&stG83lq9pvQaiJwFUvE6=huK>&sKvEN4B2g%>Vzl*!!H zvDmS^&trV{xYi1<1W0nK@HL}l;CJ!SP_j~CykY(K!z5Us9L;s8a8?2>7k-7C!(^#C z3vfUd&UnYDs|@>{9l#s^ofjhSv&PB4 z5R3sHR0Lb7#Kc6n)JGusrvb8K8zLWmcAl;-n80c9!Fhg<9s}A zW{2@`K`>yvWuNti@Rt0BTQJ5QeNoyJjVY(r=LP4cnGA#wRzNUaBf*L5`9P1dtID^0 zR<7IHkZ9@A0Nu+#($-rp!t>+Ap@HMY(wE$aJ|2UC#wSjU8gAr1}U_>}Leer7yBH`SLHIHnpU^*bR3EU+^5 zvt8Zz+*KICC%!;eE6#aI4{|gG{n5X8nd*Mmk^yB+1$_Y*LB2_&Em~dR<;2!U85R7l zUY0`d?(LNZn`lx6qAp(MO=s#&)$stM%DTva&&lfRZ&;#iT!FV|#}$ZnwZ^{_9^vj8 z>C}fyvb;MEOH8+HJXhT-GtEmF{3J_{ZtH&V*G3@cdAX03y9PZIRzE>#R?DeRAJpD1 z#m&YUs5KEPU~!{s5Ahv0I=jpB@;>nZsB(YNvNAt})LWzeckf*ithbw+Tgl!aAoZ7I z%mLmUnf9Vl^PA59vjtpXM9xl#zzw;d{_`jhLfytQ_*8UX;KAHd_d7E~Va~r?0gUK& zW7*TqQK35lJuR&Qf7Z~)+7h@@D@KhfU!a&!gDe~~Rk?v)wvuZT5qAl8!;(R*%x)x# zqmp^O@{SJ@{BpY#hT1{lE7XLszPB4MFECX~Si(S{*>`h)vE|M$&(Dx3C*W1#eS9jX zlH5M?^n7w1QIz5SK`(5dCHW$auC$>>Aruu<8!L%+yb>G4`|iZx)BDX~`HxYn&MYp2 zc#W5fZ4B-~Ge0s&QSkXZz-f8{*Cpy!qjO_?e9FeY=y~A=U%i9Rp^bv$^q* zZv!RPdPzk^WnUIy7FZ1YXvXV9(H+n}x@)a2Eah7D!&y9CpyAq@nwcTgQhzr?WFmnOcCH60#s(=0- z23VgX=H=xn7#SIzBa6Q$1B7Yx=c3@-JhA^XCfi`GhjCXf#?lbj(4cLF-o}?IT>a}Y zdl=#Lz(F$B=Zs`&zwbU1t0MR)pXVypgZgDze=xHn=?X@M{nDaoe{(%MgoI6|O6Qx? z(TCE6>al3RQ~9s_J`~#%3g~dpBTiD*=a%27h8YNQb8=?(2Hx`TlPD`o;%2qC##`3b zsiyucyE!TD*>LsT9J_tS$ObImC7r2Hy0;FrA@8DshdKCiVCD7KQzx(yCjn0xvuXYr zI*v$|>Rh^p|gPw>F-m3%3w;+-iF5ytYbI zjP841^_?1evOp07Pb@_z47+%tgg+M;s4$D4CVv`}wmo(UV^$)kdtBI>Yk>X`M{!|M zLLoFs`38v)xh{aMjQ@^p8+??_(5WuLN2U>Bs)F1Rg3r)6Cz}P`1|A=cID5GEK_L5T z$Q}ViU?#~<0J?yJHS&dYW?dZzFv^z+Qoc$<^?1c)ZC369Y_e4Yehjrn4aynLnm|2< z_-H0Verqw%;U|gekm)ofhW9bDDFg|td4i|dk@OJvXnrY$=H_JP0B>fiPV?IOW^Kk; zRaLc@gaRIoXkUuUX22v-={8~TumzkW$|XJlMWR-#+wR%i($erfnVI>@Wa$T3dLIB# zGh0qkhu`kx4y1M?3v zgG8SHghofBb-=-7_p^Y{{VgmRCdI1x0kubqItJDsr-_9dF!BgR#Lv1cJVR7+pcu!l zDaSE41q;!}cGMw^Jf#m{4W_9TG%O})iFhQ3tmbic+en32+vFhz8Qbz_^E5x6_n-${ zdLWB)15_QiJ+~El01f-`R0uRW{=5K&yu*Au9CEaABI-v7LK7hLQ0zI(^28UmGb?et z08h?=_^m}@zc&mEF84UQh`GOmXhGlOnl3&KiI49kQ7n^36A!^&>-0Mm6~^ZCL(d>n z<8qGsccVX&WrrojO*JQ^rwoA8<^W*Ja!V^(!sEZ9EH&DGK?pExeewUdG9SWxyfkaz z`4ms{l7mGR8J3S6NU+d7#yK-nR-`x|D^RJii4%>mAps#mzQtdwo@?yK_y)@hqm18c z&A`N@`sP0?M~Sm*kb%R8NH%cSFwWlE&LIm5Nmgq zP79%L2&a_2?eVZw3kSNzNgcL(PP!^}gxF;{8_>Y_IJjqKXNy4<^Mj?ee(S(;vSw9d zD7*EK0A^ znBWV{p&?K7o9UShL`7-?e7o5vDZTjW4>)?{A716L6@$Y+iPC~GQ2UJ$KK?51&Sk@< zA6AKz36n7i9abhErX0pdh4*^9TDq%nyyIU>KWhdQ)w^`ny``sPgOj#7Zqv2ef10AP zLmy#>0}2$#+10-q^78$pxXEdExJ^Je= z!ofmqA%$s)Q2n4 z#ZjJ0>xLEKTFt^GwRr&~iT1lP!1041;$G)JiB>bl*M#@(L9MlYhgCU0j8obeHEVj) z?|z4z(uSNQHiu2xKdHyZ<5tEe?}g;VXX~;$?#gTttr343OrS?9QYZrdiE*p<54&%C zxt*HuWM*c3g;v%6_zX;c94XdhBc)U?)7W5!?J$`wQFWYfjH?csyq?6hAdtiDZwc_)|wfP3RzebVQ(r> zKJ%-+;VN-7NiVZg)rDNqCM{!v!a4WFLF#m+sNlwwKKnlEi3 zA$J1eT~vP)6KWu&iTig^DlNtDzkyp znr41i|X_4PT$z7608bJIhut0StQ$(7gPSG`@%T?^0YpB91P)`24L^2U?`rWF$%I9L$I3N)6%c~EO6d~9h3IhGMOZ|6#j^8Y=Tqd# zpF2Oc;bnEKQvR7^xmGjvYvbnEer?DeJDuscMOn^)eI^JB1y# zHL-Euor3g;9;bS=Nf3g-*ofoMND1CZ+pp#I*MDg>gg+ozXJsP(Z7 z{q(npfVvmj7-;1a(!@bp7*&;e*zDZ)9n`untnv!j_nNIXk`Tty2Q(|Kgxc$@0BODT z$dX$C`_Z+z->}IfS$qFNFPxLs>8BrqB0nt#T=F8x$_2`!jB>mu9Bx1~1gqpB*_nJQ zM=W9+kPRFwO{7jVzae}JI0dTO&%K=Az{uzn(>q|(|MVf9dwVG9@wIg#;xIkaa$F&; z+*hDx3gnN_hcUCMwk03cYj()zm3R@UfGsEXiv>Y2;~*GFkF@!P$!fy*W2`r|@ z(xm@Xs@^xs_j;p8pJUcA67ti)9(+w^!Tpz2cCDa4Vxf8d&++ome^eBa+0#+lxa~zg zuB}vDZflQgur<2nvJ^QFY#vO68{T;tPqd^AV!f+n-7(&6`>ET4dAmbpw1 zV4`RF$YrJR*#P74?W<~V_Sd^*M@Ko_b~6&k(a+qZG_#QU&YvkoG1PL!Z>e#Mqf1Y$ zJUo~fdR01{EyJb1E_#;`=!H6t_oZPV5j)QtTppHI4g-B?lpsQyoMV=KA$(mm`aaOe zGHkz$ap467KdjawjE(BefZeVPyUApr&hLVLpp5eRTLqPCd&l4S7?XB47nwzry_}ef zF#td{CTl?x0irhgBizy1-@=$)_PF^{=gsr$gR*>s6KTbyF0YR1`->*QLaC*AVXJL+i5FVRW9vcBhWaq#_+fujQ(@&=9&4D{>I(qs! zrGicWu~E&MX8~mV*lTYS(p5;pSX`WLoy3xPOh=shB~}{#XHqLQm}*2VXeM0OSeQCX z2o9U}yGZsg!wS0^7l1KB^e40WH&+&R8)mYwqn1s@SC;$3ErBszZ>cV9iQogW=$7<}O46 z6Hg5qM$X92h#R;nZDAPgzTNMNe6N>?A#ngCVEhh$nC^8WB?w4ZD5L}uq`dyt``lmaMk82A_ij0< zS~M#5E`c=K$Yi!?RB^O;MD<))3=#gU#t&_PjNl|-zkh+^s(cf9z26WoZXP({3QxHD zLnQdE^wlMODdT9S)y4J>Jju2!55x>BQ1DqhJs~C^$`_hcfqHHEMQn>ckAFl@r_3wD zeR!c`(qGtDJV9!uLuF*5Uzy|z!G~HVMZxO8@J6B@DiFez&FHR*HhHbVB$k=y(Oy=)xYH0 zd<(ZB=1RfRoa}bEsI^imG^0otwc9s>s`VBRI8R%iNeIk=TjC>;jv4%D-9B3pAx1zL zOEIomKPi>Vn%?Zdz}p!xDoX?5fibFUHl@`ot4hb3*Zcggg=P`vvA#*I-pHHxbQdv?MQ%{o4Ivd#|SEJu4O7P~p6^*;B-E zKhg}i=W}30)?QmQ(dK(HO|Oq+%CsIya?*E9D?O46J?AWauYFeoem!P{u0JsM zDV`+;yKJ|w5B{25ZcFy+wtBe@ezE)~1EeryG=WB-)#p8g)Sc}sgptkVfp^BimsxKy z%O*m0=DowOP9#hKjF$9$dp~qu|JJdxR=kSsj(Q4{5P4TeI&!*(%a_aEbxpUbt8>4e z7xXd2CzbvBLL(L^11--@fm)t%=+KowYSQtWW$3GEbZ*3$eE~{KN)INg6Y2iwg%10` zVHfeA>k=4L3T)aj5po!$dv}FF9LJ!)vxGKHU0{eKja0*NC}rp#g<>TKCXx0>ejrU6 zthqDN22yBqcj`(gsDl8BfP7wV(&Urzj(+)S$Y7)J>_UHY?B1Y#t;7y(3FapMX)ul; zf?=TpsE*fcr3^T${p!hkIL(Y=T z>&NJ}@q8Lhy_t3A-XZf5=pB2sNN-g-y-0UiUEpUajzsdo2VH#y_c!7_;Zgr-FuDfX z8AYu|o0C%zg23&nhwa@L#mrex;6$feq#{V>M}nlC!p6L<-8(1orN;5Vy*5DKg7&l4 z3)%hrGzX-kZTHK(PP`aMpVV{eVVxjhhk0P3Zd+t@H<)TXr$R25<8$-ZV(l-~|AFIC zi0!I>n!D8Hdd2stXMqIb!zu(1xTdpbD=r2i6_u+WD;$gmB1Y}^YGB!nhW|QPc93I6 z<=SQb-U>ztk#sx|H1tjL(SkQ`upGn356JO_>U%Hd!PkOs5AqkoP|O!>>O<{63tO>8 zAl@IPh{g|=bES}vkOni2OU!Md?5ud-l)6=*u{Z>M?N--(-rAv@bzX!Y73vRgFpeh?skgJ{6JaEwHi;77r9=BrQy zX#V1{B~V0v1SU~~!vZSLAg(3WgLk1KaZ*tLy0ng4KW&nP=VnGZYK_Y`Xt4ta5hnQ5>nYk@5aJ zqM5aDj+0&rs;dHC0JUam{E}HYbclmK6g*p7H!nLCD=Hbznmy_F1RboC8`X9<=KkPy zvR30*kEAL45~BLY;DMT4}FQC z{Q>rw_ej=*?!(D@uZZ`eet?4%o%lQUNRsM^@xN<2hqp=ZO?y;q&Pu%iiQxxb`J{k9 zr>Qwxdqt;xa!FqIzna4N)5jB`Yh$No%GPA$j*f`h;D>+7n-0SaTx?}cRn*b(-Fkjp zJ+s=g`zD(HIO5yBCkaIVCC3_=@WoM8K4u+SO+Qgtf8m&0ZFA5)NFlQRZ8h-@Op0$%T2Gcs!OkCvp(69^L&zEyIM7<$P&hK zE>0G>OC@7t4snZxgJ3n)%e$t_?Mh+~=krdWoiLu$>a}q}HbVpD`aBG3TwKkjtD0p`y_;W&y!U*gXXW-z z;KSiZtdQIs-ED9R4mUM^-=m50I%_fTJD}0e@hcv3p9(1UsQ61Ndvr|L)4{h+A|Qz| z{ss5H#V+_M6h7V(oAJE{8R1`v#v&;s&Na5p-mtlMeZ}BOJ*!%oPp5#mVddb3oAP$b zA8c*{%+KiMl=WB9^7MG>JVz|wt+O*_v3M)5>qd?}!R`_oI@>A2I2TVlW{3C$yfQXZ z0Pm!`ZRUi-EmPfGp`%?(vHnCIocjooFI+Nu%fFK*GzI@e@eBBF?srUf)?u>$ytt-PTVs3#?_@|;({3zoYC z4v0ldnZ;chbmbRrVU_IzHae~j(Pt9-4j%FouceOZX71maUN1yg%|BaAm;ZfUiy@f% zJT-!^PEYJ+S~ti2V>Csj#bh>^UwPpDc5Ay6rA?oRvAnbSM1S&0?$N_25YSd>^*gW1 z5fCj4!(so;vEPl6nol{n92@R1URCbdE^8Re`|*r!_nXP}hVM6rTq{?ya;~WkM}Y`SY&wNA|i7BMvnpar8qv_%mGzo)|3!FdHphBldbJJ19w!pV`{?Aga7O zI$pfa{}*J*{Cj94OuRSw3rFaENi@4z9TyQC0}?F9;cRKLeTo|({1fa&E27r$na@{c ze>UBSC<`u>5Byc>ZHBiIIPLZj4~R!!rIo{2POEb=sk$|loKmt>O4XQXdU7%2SWN5D zj7*3~Nu6dCsX2HGq{kytFJS#TP)|uc|Lv>m?*$FN&{w*ZQ|>@;j3N3-BPvPY8}IL> zGyjC%oQiSCFHTo~bXcoC!+d}@ZMvUVh65Ju}QqU5G_0sV6? z-6fndl2waP2KGd(4xV+X>n;@ao8Hz!_k}aScSPa-(#^Eac!8IdxfQ*wH1_Sll6g%j zyy}`f)?5rRg+GCt7J;uT)u@F^F`!j+;T^a-;4}YMp#bH?n0R-y4O8K(yF&AA|Kd~vHn)C(@Pr9WPUL+ zSilE)@v2>Al^rW(aUcgY(X*-}II_X+jrwsD+vEO(wE1g z4skUM=@{eI2gL0xW}XaOF=_24Iy<-2MM1+iJ9?n~w&IuR%De1>W-W>AtspwF6kOyi z(yX5{$==f1N~5)am%}zqN$`H%C$J3?vL6s`Y(}{aQkqv6Y5sXC(e=bAQWQ0QXEOKJ zH*!2~(Z0ZgdsZ}94Bj%30a+K;u8R!@Y2qo)v$~BXRwY)41_lZdXG2Er1G4~dsEE4j zZc{BQKL21!2__Gen1rixNw@F+NIgNCGnk0`=5##r{DZb^wMx}J`b9J2>1VxT%r|er zy462653Q2@-sgonY~Ddu%OA{R$og-|w@&u)d};GP!wYd8{Wqhu|@?A?7rElCy}jIR1C1o`q5P_8ku^&D+0L*#HId%KpF) zo`m&G()$Dymk87B3$+su<$V;DAWxsvTVfPwmAqg(B_VbX8>CkjzlIFm@VK4*> zzPY?5(<5!$fy0GtfoufVldMwCTX8iUyQNv})IN`__ZV+$`9hsq}8>S8Q5K(4{5u&5AoYI+Wcjx9M zlI`DgGq?Wx<>;QlXuU{GuSY%^p6L_cxc02kALXUl4#Hh11f%6nt*#4vG1#I*a4nJ~68ZF_Zw-&l;Q) z@bw?QmnqQnRw%)A%Z0O&QL7n9wxj$(dsPoSju4Ra+CWv}1H=Gj@`j7Li89Vn?9S~W zGk@Iw+oXho#Gc#@hy8sDdIbCZ9UmabWlYZg7zm*%Y@Tgyb~J4BMgd42E~tsX2#l1X z!pMQE(5j?8j-{9Hdn5qn9A&TFjGx_#ybR5#~bAqsN zajuXi23*_KIB&nsuk&&KeCKM%ix4D!2%xQ)aFAe-}YbuMDAP!2qpONCo z22)em-|U8NX1P+`_i)aZFJwOnzTO+D+)#P74n4H-eBYr@%o)DPvuX2@Y&<>LnXwrd z4x|CaVYPvV<$U_D|)sMChr*mWD{6$QZbWe3# zdvr>%Jwy`_@Ew>bI=PAAoe++nZ&?qqfAO=dx}Ox%-4!%U)m#EEh$#6HJ1t}`iqxB@ z9{hwyDuywh;dQ&8+S!%q3k3|AYRliM z?1`RJ&zn>F-9DOHu3$Mij~B;&-|mOe|7{_&}NVdWsQ50%45beZCc(D|?j#{250%^wQdal2vJ&?-lJ3k)4uG5fOd zM-KA$${fR&KcXYx;>QDnjuCiP7c@>Mla6#jpZ3r8Vyt%wAz4pn+} zI_0MY0EOo6Zvz9}$T*yq+vw$=@Hz<1P_Y(1D02piE;~N*CXT=E=qlThC}e?^$l%jF z-Kb0u%UTU(&Hvppe%vJrAZALP%kWULIr;K!*Q0>VB!FE{$RzOc7h4yax0ahR$*Hx5CMo*r7yrB>Wglbu0A}G+iB5z-%CT z6HeJRp$LRZ_Qv^DQaqkQ1F@g~Zbnk~@kGG8CM!SpZ&eCR)*2K^AlGG+%m9gn4tkkhT zSh4xsWN>`bZqFc17GEKRut88zm6BHcgOWfL2A=HLSqhp|{aE3Oht!jddkO}2O!m1k zOSi`s-uu@Fuh?usuX*-!&LrC_fm64Yjv1BYCtfVc+vncWj++MUAj*%uN0oPOHNeIG zT=IDGn7|>`>U+;JH7=%6qO<^08gg&h(mTv!w_YO95uaa;S{`1X&PD)1|IgGbj-$P> zfsmhrq+e(8&yUUfvo6;3g*R;*WK#fWnmZoD{5bdf)w-|>KY+J zplh6(ZNCDL2!e<`iaGCYA7A8B3n#5Y7kr*9x0u_oJ8-oqb3t#E>K^?(Q2$H0g=DKi z*p&^4$KQeT*e3gMgY)Qn1@^N7BAqkGwK)lIzO3 za-+i;U{+|ℑEMWKCH4z68tXk|l${=y|m+bkSs}v=|Cx-~5C`i$+%vXY zqhi+f{Czv|jjUJW>kq19Mj2$^4ysW$_v?Lw@vE0TU^p6EHyMTPf)EVFK1wY%GA&IR z*8niRg)YU6_B=lxL4gXA6&b8j+7Ay!SBC&$^640?`-32HL-1CMv$B|Z$wQ7Cg{^?S z`mhcpm2c(`F^ZZEKd1YA#VGS6^C&>gZ~T;8?IZv9B`Y<^CVK z&MK_RHtN!8alrxW zeV>?fj&ToudQ-muH_3J*NSivmcy!DR`HqPVL!<>&u@iOCd^it&K!KBA<;R{0(>NiE zD@x{z;@Wk7CpOOq0q7;p)c)5qEpLMk3dAW|G!G%EQZTv}yGI<8-%y{cX1{wYPU>6% z9l(|iLeb>tNxi;pHlt`@%|B0h#oh-}PEuh?%Kf>sg&URFKye}@q&5HRp9eL=z9d7~ zb5tqgBx#lYAM)+XW_cIuz$(74_B`}D8g%g{$7JZzYOpzynrQ8f8F8*-%DAsrAm&y@ zh>m@t{Bz!mc0zmMX-$3c!#XLEFy}+g-+F7#+TbbA(TU;muk%hrDse}TtRec8{{nx40DQJ zM}9wE<>5ppNxe3)^YiG?o;NN@7=Bff+HAI739bFpNUWNrb@t+Q4+^t5bZhNbm~R`K zB@uM8(1jnM7ZsQuR~nDbfVT-LT^?$d^axoMPGAY66zTJ!&_R1}Mql=&XTQ&c`~ERg z*mGH9^TkXF!Yl|{+#6xE-6Ej;dCjO?NQJ37Eaf*3HAe6(j#P7jmKuO%KWJfiHvXp< zsWwd&{^SJFS5p=_#O+y}t^KL!K*CE*lGkq-E?x*ew}eEI$u~O9c+IC^xESVjHa!>d z#Wrignf)mvLwO8mk|0+ft>C{giOw&qYuDXRG5wdnemidYWQuo_-CvmJjqs#uea895 zaH&_DVk@PXJlw{bM6ekqNBiN=-e(O)>fRRf2n90C@f~nzwQu=v)PJMXAoO7nxctgK zDUy_Q5M?$Mg9px34DU~qNbm#cY~e}6Spx5>+lhD1xs{r!PYY}Vvy{RbVQ66z#exwU z|FV0EQPJjnyuwiOvNu)^MnH>(q9!rMbp3o*KRm#eKrg#yt_AUmu=*?oTkJi)V7nMn zB4e{F@3M1Cyl7aH6*rWp94e--r||LDhf7=5(YIlMHP0B4B_Shj2qsq)Im}fKYgxe&>!=`HO#KSE;{Wvue79z`& zt?k`2$u537hI(^K$ls1_lc4)NhNow*M;z$ZQXiEAmiLZnCEr(!3d#iqrWyS1-IScm z&6C|!(^!x%G#W>Ou$*1;gGyz8xsvdxMmm4TdfMCByH^e+4I`!o7l|F7i)fr&U9@IA zO6%Z3;Z;ijK#oWo*q&lHVlr3h*4Y#Z5x43&rB*5_oxwIWjb*an8czw}YaD z>vm;qQpROLUF6jfu*2IuL>f4C!1s9bZMqB}HL4}c%Un}wvh2QIL#JEAgNI3AZ==)M zKUGgO$RMHj&nmc{zV2tBHPKQ1wye1_0ckCaN#Op*P$F{K4&CV&iyXLu}#UUu6pmyvDyVInW|tOON3-M*t-n%{whN(;n1|%!P~H zG*UhnvzdU|xth~Fgv3HjJ?oO|AO6o94}=NW+5LdjlCg?%RSmdIL`QT-84flgkcC;F zhtBuE!>poY2%7>6{i}UuWQYvPcIt9!GNgtnyzl##n)z@M=C}D{i>J129<8u1#9-8@ zDP7NrDfG9ghO~PF$=mM^yKr_+M}a^HCp7C(-CyIq#*N=6sFZ7B_+x9hheo)?HVSbw zN|Bw{=)7xCT0{hBJ^IIS<-vdJuKW`R>u~vxhgOYH{LMQaK!w+xUgwWWl5_r3wdyw& zs$7fr7)S0$Z}#?5CmmQr3!|y<_^-Bfd-1(IuARD^SV-e1eN=IYpe;Z+?L2Z86T&RQ zY8e0y)Mv}!VK4n$S!yd1-+fcF{OVQ4_$@- zkkYovG^KUfdS!S@7cjht8tSQkFQ^k3M&hJRju<%Sr+86VCE7ck$m6lHD1E>cGy4P} zgKV9gj?ZQn*Y(%gvWbfd!8!gA0k?2Y5Cbz>79#|TB4vk+dG((L!fuGTcEV$^#{;JK{ zH!N=fFxhE$3A*nLP^>vBvjQFCgGUu@GS@c@rk#$9flPY;nzG==(r?BI5qgD7stooph}MWFJCYnop%p>8(7s*_Swo+r?rj8$<4 zGAoB{x6mxR;(bq&#%rA)Vb|-6bJ&xF+2)oOe5A%J8*U2A@CS$PWxSZ&^o}r@Kjl)Q2fWbGe z>`QR`E3)@IrtoOSOuF@UrCnx8KJemx&aAJ3;ytmN-438lyJuxXoB$4+Cz^SP8PS54 z4Sfu&q5@30$IU5mN*-YhrHBx{E6Wss?gcEtEaCdnV9 zd+P^ZZ@1j{O;C#rJd>oQ^%r_(7Og(wYiyA)e&@iTtrxWlBk^GCtW)?L`sIE25)gN~ z#yOFgSo6HzAkBkUS|0B-9^Jb8sRc%sQ%GGAQuy-92p7@yFr!6i(!|aoSZiKsa!rc- z-oB$Ij(<8Fquwx2hKW-DJvJr?SEeCj4(l<{Cio=ykpiXcL6!nH$YU=`@F0&b8I zv(*WZdS>eRG0b6VM|z}nD3$foeHIy549dDjLwAX62)<(^spPw=_^s6Pwpt;u>$JpT z5gP2i?#y#X`sjoy;(KMcDH5eVEauJfLaZq0fR}*(o4V3mI<$syEz%o_%cyX2y^_Xg zc^%HHGy+w$xV_vU;(CE(Zzvatip`MDFqJV` z3<~IcO8(UJWQUF6N)Z^7Env4lKV6IPZ6~U2O#)iZLM`h(esN9`qt;A*x!U3ouGyi^ zZ%>ODK@_;9U1Sf#!CYJ}Dsk+QmJ%S39UAzBKs=V@ALUQFDnFd@?%k(w?a{c6>tn>| zB3~Hqql|AtM;k#K6H|*ncmqf{r#RHf7z+LfK)TDUo&BCpAP+OcX`IqX^Ra8L0O8Ft zj5+2!xDjo(nP)ELh*qEpD>l%J&{klwKPusrAeibI1`k- z*7b+FfXnvcV7wrVy=85iG?m9b-Fv;anf?oT9~b(wm?=^WStl#D3d$rpS4?v>uCSX? z`9Jxh7^+1<$kMn84Dx&OKT*F(?_N95;6DX_#*M~BcQLkLuP{Zk`Yzf8Ae5=eZ2x$d zc0|cvC4|Py5MUC{dy_<|AJj&>nwqq0c?nkth(ee-;nz8aZCAZ+-0T$h!jxsiF49#U z8m2u;ix;%iScOmgu>Jw}QQu8pC;Rb(_r-P_n!F|E-UU0cuj{4L*@x?Fvr%QR#qbO_D-H}yfxkD%zHQLgJ7-U z+1WebE>F9@71X(U5RAjtiMAVp^AX$v)YfiIQ;`=! zx?X2ou@M@Flq(Woc9)ows4<#swnJOsd>Oe3hJ7%lO#eq!eLckDX!+4&r6#s)sHI? z)3noPiJD=FZ;PS~2m;J@Q{s@tYgVtZl?_z+#W&B~o_x>)F-g9!?oAQc`x z-_Uuk-T!wlsjmzuHWh&GrYYU_V`(o-)1e8zLFOIJYde|@vdul+!>Y~g7SR4|KDeHl zB#S{GDobXCuz-+)1irq_QvwRie-Y+1e{|Um#H>Ek!w;zeWn@9^R6B+hp{?+h4`|qU zww7iC0sUF0e;LdWUQQN8FR|yhhl6iJe*Hs_H}V9}f4Droj-hJ}I6e5pqS~n{$Zl7Gs<4G!jat^ zROf->%Hl+IoyYeL;_Ro@*ZI;W$;GZU_cp@w4^}0v)4~y|tiq0ZCwDEENWN?tl13Hx zc=U+kd;gtvZK%xP*bN3&8> zuI2w6&3;Ag%neH>nFgJLcW5bwKBeQGr$mmwZ|rs$Car|3dWJ%Cund>jsn?2lW{prn z@KJUOFyX|9-b1Fi>;6-k7%ma<4hbQ)rsIq`v}25abxiJ?FppI>+vDicj}H_mXD+fo z!ts+76wB>{PKf=CIvZ60Zn`_f&zx4d#$P{b#@G~o=|z2&_Zo&yAhgt&giUOVZ!Cb~ z91SKThKOh24(-6a>kh5jcoRX)tM_6c&%3kHHOJKmSrWO`tDU%U25tWxQV|-FOcz2d zg7_qjg@4uEd~NZH@N9E&)t&GCAKQd)nFk*&P(D#~zsr4gBDnlRertKrg6+<%d3~w1 zI*6OHyg5dn|M;mGBAXG#uHrj-E>8|;2nWxjzU(2;SkN!wvd^-7wDHX^Mvhz2f7i#d zKS4zBZG*pSogIDTk{;Vq>ZnWQE_hp_;@5a#SpIwyx+AtiHZa*XO&U*rgh5lUd=`Rx z%ljzBXko4WO+J!UH0G^zAS*fBuJFd#qx?0?a+}P28~AhoPq1pDCr{m^J?qChS5Mnp z&O8pWr>%7X%RV)6RhCtdy_Ig^9{1YoA0Pw8I|b8ygnvxiZtd{Ig3NV7%9s0Vk0N33 z6jc3QdH5kX_`cgWTUYxy-meMfii!r^yeMkVtg)y5=gcOr#92e3gv79uU0j$WGDiW% z=9WA28NwjEYqz`?;8Sj(r8)Dl_VUX0tlR$Ru=s%=#d`eWulNzB7@p%hHhMoMfDf58 ze#_hZ)8j~2D*+*fFqIV@j>)frrW=Bcs*?wE5UjJYzyKRgA7_IGu(yn)TVJ07)@Z61 zfKbPQlDFzJx9yDVD5qm05yJIXC4+@OdYbCDww2AS@6h~s2h>r~(5^bfp%zPPA>wwm z=r4+87RQoEfx%zFhDm{%B7MyAiSr&A_C>vTw~4)8cyY*F`AtB*uvB?bldN9rWRZ zPNY0;Gh6g8yJgKb1ZUIP!LGG&V=^0KmG^%0VQKlTf>2LRAR@_1T&iY8F@t9?*Y=qz zfu@;8$k#pj+15Y4fF4sZ3K;GC!%l`%s^XNsj0)>$M=RW9nXG==W0748*a%ShC^quu zmnFNi#OqcHWzmNH{0?9OvG;Vi}j8mqY#gq>>kb%k5uM{B- zxWb$^{D(Z;>^~Pb)@MArVW_4kF)}U_T^H#5>~+mD(s)&0U(gb_7f|GzQv5q`-Z#ZY zZ=x-p0+R>#eEQXn<^mlssL3tjqU;a>8MtU#NjuOlqIN~(!qRZ-t(kxMro-E_g#wZN z^jNoyv4ad9`B=1kQkPfyX>$KX!0kBIecEAEDjLrENR8I)RB}Fn1}iC`Dtb}+FQ!H6 zbPbO-g!L=N?LgIidu?u=v8}_62$}Dn4e#e(dH=_5OSL6hx#oi}x0ey5Nwy!moH|QI zfc)EG8}Fj+3nK~VPX-N!!PfhCZj|wlU5(O`plnHEdf3nlTaQ7~T~b;{CB)_Jpmi~j zG^J5=z1L}9LeXg|Lk=;-H|y2Q zU_`mZ`iSznLy{+vkq%vLw%z1d>;_oCgg(dKm-z6rpyi#lk35K;aL78nG)$)(F)5h(;-L zRqnjm)9CtL(-^U_R_uzVRO^Uao0ye@WySAEr#@LZ&xYG&71rn3SIwX2pLZU!H@suu ze~K={`?R_X-4jK7;w^W(ER3UmTMsN1r$pPdEUo6N^rPk( zUo|2`ZxX#-3E%UD(Z%cwsQ;WIu`ofOl|B9qe)GJ)Ix3(HF$*aq_FM>jevF}#SJ#I{ zqEk+$ciM=cO%JC5K=YheVX>=;W=>No(ggO&U|civEpu~8m8t_sCImjss1zH|C&Xlx zlU{lbdEq=5{aEqZM0=l7gCoLib-cCESUaq5zfjxHt55SuR~CkK{bTV~3nY=gCa8Wv zYVhbOeKwIt#k+uHD=}29y9g3uU1sh9>8^??&@o#G;f`)M9l&vn=H3(BEQ-wV0sqM+ z7!8wMdIS#J>XsYs>WCg(|6pIa{Rj=KVD=O6&6knFy>_sn(LEyGenud@zJtfpH|kU= zn|ZZk6{6IT^@X-`*E5T`9l^8S_l>wAoB^`^)3}(&_u86li_GIH16V${lED5$5eow1 z+c`XfaG?VL(|Bk4CS86+)A%hoJrv!@OcIkjkfzx6r9)!H5Bhm1Fm$dq5l!&AkM(XqJRg$gr;zpU! z&Z`jrIszu~Z=T0Xd?$q$8I2~;42J`kCit{kFcgL}9Uo*&iN<0@Cs8QKb_l59hd!hs z?c@%H7O!}ZC$s)8Tv#~(~0d`p>F}g-(0~JQM*ju%r(R{(wx>Gn%kr zF6c+)Y%pJgOud1B9WOjN1lW~|?oa8-SG!uExt2J6d2oq5@p8^omw;1eAp5f{`x+Yy zR3N=*_#}8w6KRo+RD(Pyr1_z+FTqTN5IVjU>KbIUVJgtN&Si}7meyWoG>R|w*-m^J z5ah6i4c>moc8y5tB}(dh$Q}_I2 zIk)`f#+-D?NLKaEC$9B9xBl7N)KDg+Vqv z`%hg2e+cwac-gKcK28CwaVD#QKM20TErsTCs^>2@TUVR@OaX87vvvqaJFqq2%!ThZ z8RLEW%3FxS3B`0Kzj-}ZQDYWsK&xi>ZTwF7m2+>dX{3|(?#1>^ zf!^SkFcM1N>G>4{*yYU%kw@qBfkM_|`9T|-&ka(-0ml@4kNaU8Bh}VT7uy?wIgGUR z<*a9hb8gG&pN2*b>c$QVg1txUnlV)sM&@$ZT;a!Zn}8fTW=BPM*YZS%w?V2e3C-*J ziw*8~%V3u8Cs(l|ltj&@^X_{)<5JyAH$b=Xp}&Ek*qOEE-&ejDwhsZ2u)fmDKK?ad zJ9{s%azS(mS#RMrr^K6iGCy&qXQi;he>t@jkpK9$h0pWSq5khF7BnFX-C5FQRQeow zd$Bu6ouY3t#Ccvv30C`%-J>%m$2-W{toBP=A^HfUKYbmaVi$8*{#`fL2HZ6S>oRrN zpGfWx8e2d55-=@5rB5i`TQ??aloO*74wd`^eb!)t%w8qd00+h6Y8;R@nMUa zl|iIl@9brhZC(PWlBFf_iHv%q^1apkB*E<3y>Dam-Xkl4U1K|c^RkZ1n6QTV?JSD9 zi4ikJRf+gTCU-G=pi;C(!Ke>dKU*PfPLD=8Y%k>jM^ATss)8}UEYFJuDrvmkcU z9;BTgAhN-DrP%sJ2V9rkPRbgV3~< zRrkK8{XJo|!Ss>&#g5|n?EY$23Fn>gdp60CL3*`f{EKaEUKf?(YtO3PjxTVf5cZyF zkMuXTjzS{GoMeZPj;Exk44!rZ<=l|}Ze!u69Y0_0y1+q63vY*DdwafZ@%JSYjSa+) zUJeYZ+wDX`gQ20Zuqx1)%XZIu)T8|C3dF=(8JqC|;LVzJ^jM!!&^O&oC<2B7opre> zf}U{Lv8k=D1u-f4aF{I`sp>9Sr2K_N-b5|^iZo?)f>ieFsMhX}-JK=o>HFI0HjTs{ z8PNpp$?DE1;sguq+17351?3L`Uz!DuUNu_W;7qP{^;V{M#EDRMl?YRlJl|rdS!{aL z6}Y_`;YE?=-wfH3&K?&m`TQoi3=xP4ciLrG45}pz(crN4pcsUyrN6|nTM$ocKK*<1}%@-k} z4Y+$ebtJHU8jD5WN|`s(Nc_C{pJF7@A9<=mZK!i z#T$Ou*1ueR@hhcFK3ZwNo@sd;d*Iz)eATMRDQmBfZhNjdezzU!1WrtK?UZ|~RYlQ{ ze|O;y;)c8pQfaDo_L&!dx~2c~^HQ-&Rr2g#`qH9rj2UX-pkX7kDWR0S*b_)!DH52% z^-ceQpKitE0Ao@dTsQ@!tPtQI@hCpsy*9~5l3@2VsI$>9+&O9xMappy;0-9lS-HE! z9k$47y*zNklLq~wYLETsO{+{_Ks%`o8adz)Wf5=;C-zyZQt2c|sMK%#{!k;!$hVtV zR`?8mJ>cMe?Xny4iYQ$Qz#v=)UDD20u6-O%bii#FyLe_L|41A5|NY-@d;==jKRuS+ zv?2~C_H2QhAGmP)B1``Ibm$C^|FR}tev;cxFA9hMFh-sU<+ayq%*gNz$UId;39+n8utQ^7tyO1@!f3?Gnn*Z+COvJ-z!{CtpGuLzSt(s{c!Qthf)4 zdpC2cj3j@)N%ZTypk24YSHo2gzc0G-@jXdqCsv`)btKZ zeToMpd5?K=Vc%Ama ziy~LvRJf|@jS{^tZr<>VkCC$!_)|wDNi+D}ZANCTQK}xtFvaoXdazv)SDubEZtW*o z&*KK_hP{gelYyf&=YatE89f$&Sj-~_jy!=+4OzticYkIify=p?fS|CPAYR_)0REu6tE<=Ui+7JW=%Aymg=g! zN!ZR)Lg2kbeJP7TyQ?S-2Jnd*&k)6Ad_JT$;X zynRAoYp18HTDR9=b)|jzc%#o_^aYz1@$nndII5g9e&HivX;4~vwu(!|w8U!-Y#93v8?vmKk5e6l*8a24Yw+it2%?G=udApGUlqdT}mM%yBKV zLfKXGVv9@3HKc~#%vk9#JSNmt?dE$cnqGFl;u5vlHLc@5%5;CX=Tf&+Eca$qpN6wF z&H`rRvPge6-S08OOrBEGdEc#~&_h`1;v08K^^=#ETy^FyCbPhz%J~Zi2b2|kd$r)J zH2z$7z2e(+5pW(BW;WxP?CIPOd;h*fhz1C*VhTt6MM{i$(df}(F= zIskn#t%80T#Dcr}gkd$J<|VA=3VXMM8henfbd9QRs{8fC%Bw&rt;{JKVN&5uSE}gZ zzH>4hYciqFN3?%NY<+zd&hzSK;zzVrO2fBGwRRhC`yOiP%9+>7-#gPNbyf9QU?k1H zsbH7Ep`}fa3+pOhC5W0D>9x;k-%a?D%T+j>&Yb{U3BT1e68iPot;XJ<3BtG460#?? zfIR`Hf>%q0<^Ck?d_HH-Hn;6!@qoif)5l-$lOJ zvc*!DfqiYd*;^h4&0fxleCc}YrcyZ=%3rl)uKc=vW5u^w-dyXHNIEavdU*|T5=mhn zLa?44o%b}1?o<3L8l%C)Ga2N=>^%JR7vXS7V$g8mr@68Jre3P5)EoSn`cV2q7@148lpc94sL1bmdO!h0 z0W#4u9B**OWofhCKHgu?PDW3FjILg^*-WfX-Yl1G{qz(_SNmH`Q8wy(Y8_xY6dsu} zu!wO!tWrI4jLY-(WnzvB8`Y~kVmNV+UK(QvvA#-dJTLPMNREglfgg&(K-HCM?a1r> zijVI$p?Lm;aiF*|dbzl1oed;hCCcAlE9AWxmpn)I6A<#gt$utaFTw+gWu%}3P|GEw zlkmrLpUx(W6fG{*bFBADM;CZ7$E7%|{MM0TY0~o<5zygrSpLf+!5jNondcM;ko(iQ zDa5F~oJfxlz0S65MkgBgA01BU!Kr5L2(cXQEz^2P|Ac*dajUUZ*uS$`G3$+ZpQ8Bw z`P@+PE6Bz}5}(p1OYK=a18+-51 zF3gynuV}C0fyo^w)4j-L?N6Ce?xa!LF)v33yJ@ll8Fq`xoa-T%SM%nn-RB{v1tl_A z*GYq(51u0=8V5Qx#e-u?9=49HjnSzawXO>jfdhR=u2a^h_-Z_+12Wc|M{oX{G%GXE zIRTK6JeamEH=b`U0$NKye6Tr6$b;FBC$J^a4MTgf>zH9CpXH}iY)%3<@e*2ER=3+% zkL~TBh2#VhTjrfZf@KsS=(4=A;?gM{*5@iBL?3APB=kKyA&Dq3`Aj4>*>smHK_ErnvNoy~tJ9Bl z@|*i!;Yy3ys#I9vq>#g+F86_KmM+%r!;15U$B=XAwYTiQhy5Blu&cK+P&WiN&QkVX zOMh%Nqx<-=7)UJnga(>0`&z4Uj}p$)D-D`F2IPB(@{MwjTV2XcX-R*FhYcorPZmy| z-i}Y&3Tu=8K4Y5a*1tOryN-0*XDX?8v^6xMe=0#uhE_v(e?3{lgC{c@YyV!WrReH) z#TWFf(fIn|%-J<&fqwi3uD+LjPI-Y2D!6;FF#s!F$hSU1{0hk9Zt8IiR}PiO173xA zrAdDaef0@4R;)5I_s^qyTW0%xS)gN1EO`&h??4o*Fvn@b+FOr2&B3eY+MX9 zBDt>ITY`k`Y~Pdt5A+z5PuL9p64S*K1t+D|Ww#wa7<%=eLzWuP*cz|Wl;iFz1r3hd zsv|PT0YKooF^hQaFBwZq%RwC9XAOMUVU~5Z3q6xO#$Qaus9?J(ueBN%=R%$X9WYi9 z$oOnYseQ|BTq)XDDRcmgFiA>T{QW?)Vxr_ez5wqs-HBqRw+kt-Q7ls0_K`ZRG1+vv z^)Rq2`Qd+m6~iGB6~Wsr?mOB!Rkm<-hmcD732xY8#lVglD~ zwD4=smzxSNKhjg$^CKbe+f?Y(Y)8b7jFA&qUQXIoqQtoZLqo+Xv5Ml#FTQB=K?~Wv zW&JM*|4j|y-$UE2C3UFM`~})fF4P$U%gpsb{qc<|s-W9$eJpdUg<9)lVi1Ht2LegB zRaz!4nLjyV^-5o<&Ti8uTaWxe$6JKzNz>Bk~5`QdC*P}PvVhve`$K_<32w0hg%ZmXTU4HR|v>9>4nZWR`f#~~#Ho{SC&kJsfU z`_@DqJkr_tNkh-_*w(WLIP=}k21qOR^`-P#j)rf;vd_~6*?;E<8v9YPCR<9Oa)j0H z(_CL2Q5<2+%QJY*>}FMZK3X`ebJwio^MamVPrhtk%5)`U^=6oVJp_B{z`X+=PX>g? z7Fkagw<R%IQULVfuo}yyqrhqnZ8f|v;&Ve^aVyF4;1CXe zh@R^f&i`qk5$JF4;J9AXJAU?{aCET+lZpTDUw%N23HQXFw;W()j5G*azS<8JD`Q9r}uy zyefxQ+`RB0#5!`#@69vuOC0ae8MyYbu0M?P@*dSrZh+_k1=%^Nl-&(D($*jNX0S>O3SyzRIi^62PpHZqql_^EoC6SRvZp4aIp9pbVLuxZ> zg=fek_Qj9%`eQZfV~~P#Bci@V2tG^*slNL{mLbP&e!}KhVfy9R!^pSsIIr8~?D}Hs zG9^1D<2VPulixC5-@D$m_P0#2uu~&f={mY5lj&vwN78`?sw_Q|)oN=CR{oO9AhX34 z5~Rs(&xDKQ(FybWg0vi=2%I=lrsYUxDP7Dda@eV)N>}AJFe96bR~MY6Af=+fuqkX$ zjJtdYQ9C&FH>{AR_@3!8Z0caHql({ktU=G9@F8IIfU%~H{9)i zbHJC5tXz%-{ETTa@uY3ygYYe7>E2MeMcE2ROQ6Uen_q79-$hEnS+w&d4MZ@MMdtoS zC!Ot6AFf3{Roqbf9earq-lHRU4>VN>;A2IeyePiEK{Vpj22@B1&CNS-yuQOwLd5mr z^8;gsXtr>U9RZ_us78Pog~#T7aDTXNd*&A2ype+xNTFfhHG+Zg#18`j*B`$g6g-zT z&^#Xh=n~`U8V0bH^H|{1ql>BuIHsgYIX{iX9!UJ=j{XqaC~@4htxgdeVSs^bPxKrb zSkQc0Muo|t%RNFo{vgJWoz6g)FS=W5t#1A`f%$+ZRPT^nKnvxOl!4Uqm+43XRVEZ} zIrW=g2d-THr~ft$$S%3BtyoGR6ADTv?sa~A2k#qyTpyW4lkY}fXnkrkXazEC3!0;LTH{7VF6mytDGYA6aGA;JMLAITC(mwyi6(Bf^qx?3|^u|o8Mmf8=e zie(FHU?JPJV%&xuv+LX%c04^OYQoRv&M*tExzzyw$cP_Iqm;BVmzJLNe&?|C3tx^n%LKVs<@L+caa^VZP5-rK z+I_^HA#&C-GV>@k=6!CAo9%L6<`2W0D!)f9e9uC$(&4kNI-?Gl3u<(P*Y36t=xd2* zKuFDIjO*?kaEqnl2e@RpWAso391cso9U%(PlT4ZQ7O*_7@YxD=I^hAIy=TGuc%oDS z{IU5KR1U^LW=I7CT95i&?7hSZ2NZt2Cv7(jT;1VoVfr$v`PM#B+Vm%FI6MjhJ5Hr1 zl>$kB)aIQ<8vQnK`1qD=MuZ&u&;JN5=^EHtUCx)UzSkKexO$!0UO-*q1gv_5!=gf? zZ-EJ0;hM20lzuUQV^tPVyZK=^sP8`Y7h* zeX$TaPUM|06!1IpC9x7_=ISR&9N#T%(+D&ya@4;nZ9~E`Qv-;=ZhCNX-(?r{X5jLRcnXqf$d|zYqEMD^;;Xqqos)UH7 zaoZ*fpmCXE8sVPLX89dbS!R0|%VW239A*U&4#t>*bi92(10%KF<&dLozes7u*M@ z-d|S}i2QwkT+;t#0pzG4{;e)-2fG^s(0&9aLj47TP^C!RqK{Rky)^b($wY84dDU+2 zyOq~)r^^5elwuWxTQuNUV7}0-QLHiy%P&*ubYZw0hbE1uN5hJm%A6#hCx~=^uT3a= zC2uIW->$zA&y)O0u31>jTQsOz61o*wnbcX{JJN5M1>#8UL|xV#h(77`@cMRf}OI*6d>acbi+p z`i1G9N@%H7na#y?aCrm;Mgv*cigim$b+J>!jyg02qu%68X{aylzg; z+P2aAyYvnBbKZA{^XCv8K4r7~n7uo0Zd32slYgc*{Kb2UIs`9q;L`qoUTXFszS;q} zmj?uW^&Zc_EK~pBhFP~Z9>@e0KA`ZlY(z3HHRAH=3MCXj_TId9L75zC=D480-K-3E zinUE*4ZGhVaNa2!4Sr`%ztA({yBIoRz5ekMHB58|{%jUk!zB+o;6`cX)8 zf2zuoH~omR^*>0(H0f5cT8%B8I z1a8RRwCvy9tN>st*g@e$za`i3#Vcj-D88hf;;E3My2xUg!64%!M}z#M@V}v(?nWRy z#bq<*T`}|0PYK=yw7)w%ex|6mLGE+DlmOL#5&Tymv)kP?jHwoRuyf!dDuE zyLe^%^XbQ9PIOaU@Y3FS<#Te7!8O`?Rr;69*aBC%+JNAVlj-xe?e|LR^fA;Y|Dqe8 zAX`WD>k(WlANKjhOUGO`CTBU{yQaGdSMeuJaC8n|l%Ek*l7(RYAq5seYXR#KDe`|i z8IDq~gu;<}5G5%ia5+wc4vSI@zgp(_*QZ9X6`PjLK2TV?|JL{TR($R7FpZgYn3T4; zwH#;YLPXKm%LSWIr625l#fkrR{NnZF(19?{$t~JdrP1@i8UpHXGfC+Deo7qGUm%&N zP_Mz^6RS21`oF*9RG0|#&L)LH{o1JcZy5%P76S8~v1jTCD6S0)vB)(C3mLDI{e5E+ zrq`7eM}MnHdB}wrXDU4NZux)P3FsZ@BZ_kw?k+}6wdM!f;nx%lsr=1-jJf@yR=Ob~ zen-E-#AX|4nJ_}c zNL&_f`t|b9LYiLe^Vs zi^~LqTH|W2?#oI8!R~P*A@R>j>CAzTByV*2LZS?>5z-##DqK#a^TZaj$8|d$9`$vt ze0Bfm2YR&v36$Tj-$T^A&a{7;U~2hxGhNO0CW>?uRnto`efz&sf5y56r_{1D5^W&h z%>LdDO%!vwdJ%O2;wYsRdsUlBq%F#tdn<%Bi`7$jY4Ha3&h^q!mpI8`?w68q_H*(Q z_jxs)F=ot|g&jbCso%dAS#N|_^$U=Qp&}6<&Az^k$;?Q`l2!dzL&y?$`+#ttLQ&UZ zIe**dj^qLjFKO3`;gh@ltuq(+_M~o|QMiI5SJ{C!%Q^imQxN+)Vhq_s=SLn6e|^X6 z_s;WE6fRf17Bgy}n-`>xtstxYvx!ksqwx`Hu@MJGKJ6$q+mTcPa^3sPTltgXj9vOn zhJMw`QKt}T-G5r@V*NRDL|vdlwly3%)LAE^8$+$8f4v2Z{HFv;h0y3{_%ZzHd8Ef% z^any7JN&i+jf7Iszo1az+NAp~fzBs+O06yfwb`33!No&~yI^}cf~^ScXK6H(j+O}o zr8Fj=ZPv|#8C4)ZxCNue2UL3cos|3@(Q#8*_haW1gz!pIqLNd*_GJuu5DV83P`nvq zlHQqyEVBQwkhs_s6?j{$n>)(-|ugvP?~0dRGqb9pPW ze$c$mR~iqXY{vyE?yopV_cOZ`5q{9!7sWy>kox z{IGxjMIDNi2K2Nu5sY!XVhlN%GC4Q0QFW;1NXzw#Eq^)4A2R>>U6c9g(6?$q{HuBz z<6+AylJ6gThCwYzR~lX7?~+6bC%%a}iOluwHQ%BXk{bY+r+<_73k~DjpoU*p`TbGC z;URB}N)8J(X;}UJ==3&AInS5szxDN}##wrG1P|++Bw=d9wl*oD8Pe3it@z~>AA**! zsfQkAlkrekg!=25f$!+Af010`TK{5A->VHw(q0|JLq{_|$Hl+DkM_yv-(~YiIQSo% zmaI#k=MZHTc%>-IZlaiPY~snbAidKnFN!UrM>E;3i&hVn?2b>$gMq^&^Z)k+3#N)` z1qCGWF{XCV!1}i3+cWsSaeI#~`-nG=QIQMwjU2m(E&%k2!9>EI(K7U=mRsA-(g(Tr zg`h~1${?IvDw0o0f3$si`0Lq7rsc;-9FYrV$>)EtKWqtpPxVCiX;U+NI2+HRJuja- z+T29GsuZ7Zdu_maa)~lB$OF%Y+hwKzokAYOCR6%Fmk=pLwLnds(JuF3wvtRE`R`jx zEGHi!MhX6XLCwLr))mYPwXL7FISH2qN}T!WcSnf*H7&)mvz4a99+?XFH-^^Z^-8~7 zA_TE{@S>*qzn9zi^Oc=&tGvJJ&u2K<)*$}}hAa^k--mm0F_D53U%!B={gIadPK^U3 z%g#ElE;YIqeaI1}w?3Rh)npi&wykWT0ruAl0a-W-BxR3Pf6oS{RRUoc%yUD3mG>if(83wHj}Tm)TX@7k)X*&zE>(TQ5C`^8LCMDRS=mKXknXP+VKEHi|R2 zySqEVVQ_*=&;Y^RAwhyOxCeKK5JG}m2<{TxA-IL$KIp$W=bm@pU$0&jRZ}y?UVBOR zw|e#J{@R#aDiNM#^qc&*ww!RjQn%!{TEb!1$L|}Ljn!JuG=;wp82LQ*?OASGO!w-l zp*A?Mv_)K|d8paaEE|Tf5GkaQbW}N{O%Ex0<&os+Rl+X|0Js^h#iLcAtNu%Ez^ zS_b0a>RFvpB|abWW^5 zJzKyDT_!9r-?Fem9pH}K+V|`VLG1tZ7vSJzUh}(%TXz8&`56psr`m!u*oq6YTwaHL zkzNk149_C%IpDlf>)Cu;4W?NmKd}&cy#phmKy_ zH}a98^}N8Q91vbn!^rt!H>#=1YjPRX(!5v zV(5?tf`f(QP#fju+nSW^oFVac)DSTauhm_1^(c=6P(ou2N$lt({}Rt8nUv_YsH4rF z&rFCZOJwk)Gnk@>fZmq}acXdyu3@T-d$`WB2i0b8Fp!b@H9lK5c>5SUnJf>gMJAA#%LUP3jx+rAdwPq2#}j*`n* z7apOELdK27VK~7!`S#b3w+j{tia^%3Bfju2y7x_Ck{J+aBTaLpOo|r0 zfrXZQ`BX@C6MGFXC2Zcz@nP`ISH~({e1a;Bvyw;v)ERch;CZ6g0|*jXTBUpFbqxHc zWoo(hjB$+bu$a(!?(s)^fy8tt1lcp)(9Ruz@%R<8CIF2K@RQ|IK~4^mymNqWl>{r* zWoHy6STw|8&T70%6ck@LxnMRf%~wNBxBHPPcS9H;yr(w_fR^Ia``*C&JHO)iHtu5i zG#Sdub)SBBF!iV$NLBO&Cnxm-s`n%E;$MtVDS=nGdjOAt`5i(-X)K)I#&lJl%46NQ z1fR-hP6!n`f^2y#@;|8>yZAxD(m~SR=0SroAWv__Eo2dj^%$}CcI?_jy>&S(K8{)a zN%sydVnkwOV%8j)XCT1FO_L6xTZ$c>U^1F}kJX{`X;9>)N(w)90a_NqEtVpCB8uHZ z`A??aps?WroIOu_HL<{YjJ|i_@&k-I)yod=-rPA*ExB-@DhPXMg?W$SUqveb`Om{2 z;_GdG7xIqL`T)rhGT(}x?a7n~1X&6k&O9C+=DvS=$+NamrpV>Xi?>7KlnK-)ijU)= zc?L?P0~|nq?19Kzob!@8?Ny%scnXJ5MIiI^{aG+P1}%bI3*5DCsN|vEVVfU93^I^) z(1rsC)X<(48?Z6}^nDMF+l-5r zfxT70cGRS+D?m6nKBi2?NRiNoMd*8#(o#OZ=uF)j9v%({4LhREV7FMo*b$Hj4-%2; z2x6fJN5|qKJkEP?dafRaoa4BVuDNs<*h zj5Z0qEwc_nGz11|@%HT;P%R_1fwziy zb>C27_A7uvMR*=9u37S~2g}#U8lk|0Yarb!I?u00p-fC|($J{eY_Zy9TgG89>9KSw zl_Bl(pZuqx9Pq--jI7DidAc1?`pz1WxIHq$FHWaw(mm(oG3?fS6}sDyvL!Ximsdq1 zKTpG>u5F~l|Kt04ui4AB8e+nyHpB#+IRu|0AgxV7YL9mQE=czO*yk|O1K=szYp&ogsuY^^+3yHfkSx(bY zH1OF@&zo#z9~tsAZtt)63Ay;XHiroJZTl0rJPRjp*T0p&$bWc9ldy{ zfoty$&mn^E(oA(Y#`d~B2Rfeqb_hfpRZ@3Zq7tP@+rroVmbH>9;+4=fX^2`p$(vftF3ac=BNnErNTK za)&C8>MGq^dgEj{hNFYKabV7^@;1aApDm=i8;9++Y(U}hBvP47z{W`Mt;m}G@~h&E zKaZEx1;Ygw0|rg8xnR$&_%DY2{TQ>&@SA7~`Ul5k>?9A<;Kw&>Lt>XYBq@kdtITk6 zGB9}5?75T<%d=3%aFns*->qN1eRGm#cdc%yL=bUZ?<{xY+3%af`Kc3u3(8l{`-%kd zG1Ay9P~&OogTm^U1|7*QgamW2VTql>=HtUd^jkTOays&J z%*?R%2*od?Ut<%UsuFb;&QXOK3{A}jGNl0$&FW8f8n|X?+|@s^zl!%S2iiFNj&FJ( zp5Cf{?Onim1D_gs+^o z_&A<0Te^CSF={t&cP{w3QRBImg)#@HaT;fFE#!RYDM3}DR!MVHL(yelSD|5!;--!u z)6EhCxMbg-u1i7i(hk==mz{PcPp|&Y@AY9e%WNQ^(s@ptWptl)5&e{c7=?c^wnVLJM!5t}e3 zMPUy=p6>oCJifKucuLQvS}S#ZsD57Z_IMMp;dZ*4-8#yL^nso$%T<7ANkX@>=&@Km z$lQT`o0xfDF(2NOOkyE&U0t{hyeSJ36O?*@tGp&;%nm%m!uY0E3*^Z@ zIbi5zq=DXr;j-#h&Dai#fS1_unAiUhazuPwZMQ=EBDo5p(ZHIxd zaI#R-Lv9`0Gqn@srIN_m@!(!uF5_<+gM%-TA=#zQs4H}V#1_FV+;1SwUH3tHc6HUc zF*B0Zy{Aa&4!k4o-hma)9uOHp|sSady2hC=&@Fj2ES^7!#@eFMM-FW|;2&`WKrAWRRT3>OxC(>3E#n=FSS zXy=i8;h2Uk`1){>&w47cr4|=tgHcXG1((6y6=@ENrmn;c!3E*5Huqrgrz&MC5~X+$ zk6N_Gf+zcMp8*P^Y_2d<2b$GOwLbcYU1W_x2I36f^@)fOKJa zR71XA(}{RRg4NzFa;%mDp^!~!l^UkkeY;Fh@;EgC@!kPhlwqpM&YlHWR1N09f($&s zzGVC+Bnlf%y})D8ZAs`oJWORm6B?Czf=tPqnH4?seftCBL_*-h)@}~vc4<4S{PPyx zH|>6X^|?a{iPt8+v=68Y`OnaNvK_4U&d3V~dA^KM^H$t3?w%GkqD`^v$#dO`QmU{+z}->gkXnLhREoPbcbD>YgH-G?m6@<&V-aztnHz z<(aZ6PkTLJ&e3Yoc+tm?_vghYS7}-c9)cgbg#!|$_K6ZT6IF}wL)amrK-1aGwQ_Sb zgY{)}^*y9TAWyKd_6@8anslQTd;==+qlNsXPWZd?rF@Z+dRO4$JW~?&WyL2M@y;*%Ly3l(cb-rr*=CIRI?j1E$7pEgFe@Tj%zX&_H~Keo2KaKuCa6{d!bIzm-& z7wLg@9EzJwNVMBTM#MhG!3roE#qLAW4UBS3@iQZ1+)f|lC|ux0gCg#P4T5@cM445- z3~_CfnRG%P;3(StuKg=o38a$f+nPD&>~vh|$~{N~X9$iuju$=D5L4eYX#S__7(2l03XiKH3=w z>kj%EMjK9ItL!q0p~{CDDQ0A_tJE>H^n9C)MwPdcKeQM#Bje99kd{3VJyGyl21RX_U@)%LP2TDudd^P(!@6rsNc>pX)DPmJ9+PDfh z2NlFA2P)hvtjUQ>Vy;u3qtv4LBe=7&HpzNQwVN&zZ_;t5C?*;t8Yh}*tV-vZAyVP5 zLD4OfnIj5dVuv|bVh-WL&U**5?UGCqM`$~!)2l1qSf;DfG`SX1LD@b(@X|eNJV^Rr zEH?N--RU|s0tPu}V26=}5kxJs2AMRPrB-{W0@{H>mMBzUW56tl?%c-K2%FeGonH87 zDMl3$xFd7aAPhSTZ-WK@E|3;m+v6k-;Ule?515iC)1psF@wg83qBuzb1_Sf@J6*gD zN}}!_3L;ink{M(B{Cu--9dO*IVZtr(fAra{VA)uN>GOjWpzR}b*|Y_w~5kK z#AI7S+zVq12GPzV{S@=0G$okZT%n#S1qC4KpJ&9|Xvxy1FsM5)oH_YGBV2(~5Im8!Q{m=VY`+~Mu`y*2wZd~W6+Ll?N!QeZtvEA-Do?dvcMrcG4!Z3~BA9c&L zL9W49mAs@0QWA1C9KgSFc7Tg>6+7fJ8Nh>Vfc%Z58<SG`-G7#Y} z=pRA0NY@h{s7(g34hOjo6AhY8Z>;c#!?;~#ywC7_kPa?n-r=al0RoSY1%N#Pmhw&s zAp6msIZ`k#uRixkx}5?kTv_l@pXtO^>BwNwSKYcyLxO{ak+w3tO?ry-@qa`!Idp1F-EfH8?KTVG(3{Z#^6nm304xC2}budrsh~DCIbee$m3AJ!ND=F z-Ec2wYbai8kEI+VSG7zgM9e}bDR4tom{B1^tf&gq7m9m36e?f7j;^wSlw-9S@qQgk z@b*5yJ%Jln{Qz_m$)r7%p&wnrz7nHAf6^R|9APrF%UA0Rv}077AwD=$T; z0!Tl5gSH(m_Ni#!Ot@$SCaSu^6Ob@E(+bL3LwSFtI{_B7$!klZTd3zCeLeV-VL_r* ztqAgyyHvOo6g{vc7!8^c8%!?g;}1n7LCoA9;bbnZu22MEh*P5%F(AYtrCWVFVUXjK zYNAeLf)MTU`mPCJ^e~^Z)uQ1e9oyr>$`#y#%B9P1GguXXjZ*``S{!b=K?ynH`{rM5 zdrmPpUg8@TMloL32xCS=;u#C+(4l0NO4A53MnT@QZcC{ZY?IASOuCx_@8sAhv_`sh zN;CTSP$I$JtE1+KPw#QSYYbLrO!TfYKHO1faqHB{1h-wkIv$3S7s8ZC3tSbqH!M;#p`Rip`)Xf}Ts ze4@LELC_`DAiTyU%YvM9RO~r{w8q%&O`_d81FziSi_L&2hlYOa4{qLgSMDMhf2=T- z`ra>Es2%L#c*F}<-E^Lz zJqTzwypz(zZav$m0n!Y4p=V4I)pR+{iuxLBoIdbzq@mf7QE3!zR<&@gDrvpO)t{o` zRRXXTx@lF~6|q<7DZ4Frky(#%shk6r+^I?#6Q4514J;aHMu(31bN5NhU4kWPef z1E3f`b*y2-GHpsssBO%NgsHVt!0qJ06zR-Ceo(se;dF)5^iyeuoI%mTO)OM+*0Y^e z?`A5Lz1Fw^&K)`Z8nm$NL7D}BD6RWz=FMC8*PC5|{2k@Epbh$;8qFIaQCeH&;FrC= z6tu6`UxxqA>ia)>cqj`qvgo#pf|;Ac3h83RJndaXu(#%fd!m;+lt@L1S2r67i^7dn z$-q`$p%*)LQzQ@lgpzuK!_^zU@IIn3;HcChvjgmAOETdFamNsR3-#2#Sol z>ognwt)U58D3E&kN9LDY3)D?4^c)f5@#@hrZT6MHf2bEj z!gXGs0E{vSEw-mD6}7XF&=ucW}4_!v;%0wyg_or#D%+z z<893l+R8!LH40DQ@0s!`k%xPeC>2QEm5ql!nFH0KA*ft~qQsvpT!{j=y1=-)$5I<* z@_?F@AqNk2kY?uj0W}H!-Ie)3u+jC4J}CaJ|Qq)-TKzKdc5=zTZYP4VJI*wqO2lWWY6 z`dxO_2W$k zHX%?QU@8p&VW5+MKTyYUZukU_W?kJr5%Kw5ig3H=gBui&A&O2LU$$8x8VQ4LWNt_F*2k(wuP{=-M0Wy-rrwb28v2XIp!*Pj{4NDBBb! z*TdQJA8-YukfQ@Q+=?dflT5mi&*YP_Xk&_gGQ^i)9hkI(SL6bsTGjjFUCrg|Vmh<{ z3~W9H0CB&2K6v*95AlPO{&7aIJ`UwSC{A8H_i+V$YCnr}!OyMY&B+Aa;*ky&HlxJ- z*jl~0IKtK4*ALK?z#YgRj3nE)J%&C!RGSlJafz*%K5);;1;&JR4oF<{+zqJ*;8q(h z8L4%6c>vj~OHO>j1H@_bm{=3f9~-lgdbk#+ffNS+y9Sf{1|IV-^Nw=V`F=Qph0Luk z39AZ^ewFeUWWOlfih1cjo?H<5^2Y`Ot~%%x@h{a|HXWePnNdNkq1M)Iw%>uZlEh&q zLEuLDxE_ogowf$V)8^%QZ=zg$NGjsdk*|Xbu3?n4fW`q-<7W?wKzf1*usMkN>4E!K5+4t8wbSOQ9w(_L z*B*YYS+@L%qs0b#T=|LH0;-td$8YlUBK6fjt(vsYr~|R4F|vLzmp^4ikg1-_=Mo&6MKiOu>X31yDi{-qGvtUgyeS2RsL-+(>_4Of`z$z z1^r@-L(B!o+R-1A5)KG+u=$Ha#GJ0;;h(w6fPesQy-6Sy21x$odUjteeLjryQ@vpWp!mwj)?!$?O-R#q0uM$q^O;PM84?zsEfp=C7@nmFD09|~c>8q@Gk z+sj@Qzr0H_ijJF`TN2<|!}<*bU0$wuj8C4rS|%W-V;MRC6?>KLEH%-ml3NGrlV5&w@gE^aZ*!Rc{DEbBKk+t0Gn)!|0<8U;y?l>a+h+z~HEH+h94N5wx|@BfVS z|NfAa9LJvWd^sKc-@pI&t&?0vl0e#{vb&-Azn}hv?f>_;a_QI0|KDH#pC&!f0ipd> zgF3&-`v3D5;9*o6C;0zulY0wiUm5e)t(ZO1%XmHM5=lr3KXtV)8o%bq1x7=Rushr= zLm<7h{*NB~BQvJ`bf>p$3gwU5SXR85>se=+bLhYeJ1eC+2lGDB9C+_i*@hCx(8gG+ z!ZuI?@4}`Idh#E)@U^uS690SugEch7UN7cw)MW;1|QUiCQ3ZlG;%HT3}WT9OlPYCIw$ z;@5Bi(W;o-czeL;Wgv;yc0a2V5r6=mk9NoZjXc;O1}4pMM3$W|iD$>JwJj##X>&u8 zhTj3L3h_lvNgzRvSP-#VMw#E^&+r&mIlRgJB*~h;B7vc*pu&=PMV=Fl23(HPh`cZI zor@RMrs4m>Sp4a~=7H&F#Xd#0(x;(^)J31We|$*Ys`q#%@fQUXbh4=fBL!h&3nQQk zrzm0QcjyaS-Maqq828H!N74#rN2RThdx0%X*h>#1eF}adA=|?pD=U7$W#C1%C}S+H zI6co`G(_N6wg3$DKEX&M;B90M?Qc4k;6@lL5m6s0R`S80RQ-A1qsOoB*6NZl@)W*pO1HA?-m{1W{ z53k7rQHn~` z9E>eY9T4JdCEqJM;{Sbr3#)Wab5UnDT8LVh zxlH-W=6|866SUBk0f9rM^u*S>7R zQ>p2z6p1S&hP98}Lh81pyhVxt4tCuU*r)4RtWUgz5Jj9Qaob) zXEOgCIY~m0Am^6gDT%$EV@ayt`c_Lj&D~Qna)|%q6O3~|krLkvxR-{(>_)K1TJFHa zlPZoH%GX2-VYEHIIRKw$39G-}(Xa#PEp%}rY1e;zoS5f9GN|**T&#T>7Nxjc8N=&h zEz~+Xslbk&`>_hRX8s4aftd-yItDSCiJ?LK7dPc*re+3BVEz;dpknmL_R&eWN%laxAe#q7>6R^Yp_ zMPnuSa`Vh|W)m#d+iH$NxmFpai>fS@<2XC8y(DP_DUEeI8gYfy2`9ov2(Z zX3ncUuX3&Ok_uhEub>o6AdE=kbsbDM!}Ym$NJUmUHpZ1E7&DVn<&S}M>6@=vZ{(CXdUN#t+|gxDsZfio;FrejD??y%OtGt#sF_1X5i=1)r-zpqE6Sb9ROZbNe4wp& zEFYbCUMz*g+2ylOXqTR4s#jO_=0mu3R*&riH<|D|JBfC&!b(;vfKcJO6lD4TH*Ql- znxNo}!)feR^f~*lUwvGjQI#;_Cd3m?K6e$G@lms-N-Y{yC~$nG)i?(M9;R;=ySSpZ z;T3NQHA$)nNhKc88E(K%x|k_`J`FI@o#b*sI-*_rv^|E>aka=PRi!aRZ}{@9n)htE z#Gi)GsEt^pN>K~7DmZ@qdj|h!%@IsUwmzlTKsGl`-cBqjq3w---lAZ++j#SFuwH=Z zMXYHNa#nb1k(c|aAuEKu`;irI(>6_M4g#;755Nt2dZ?WL?8_?wkZ7X)GNWl@(p%_V!8YR0PljyY-7rSz5V{I zzwzZ}_UDj`P^Ch@tQU_8Ga`ciwd7wJy4}q%IM^@>_llpjNvYB?!h_rlZh0EDhI**5 zjIW|4QD{uV$((-%yUAa66C)rGZ;|)#9zWc;2+MvRze3(aNtM>rq0vTdDDn9po9hKV zVPNb0$cniIJdG2taD6M;<%!PB5(o%@T#UVhslqMeuK56^D>mxJ(7UgIy79ro+WHGI zk!Y-nnCpTJP(?3g3R0s)q6J5bRh~%>0yeUNeNA5ruP-yWVOJWbxf(RMKx@L;TUAz; zrORQW_N9ye_r*ehQW;W+n?k!c-;gSSG~X8iey;zN4}fV#IO`J$_1zX z_FR*ePf^+wlgZ~N!9ZL`CQV_>vq=*<(=q^mo<&fOpH}{>Vj`2mlFv_< zQq%Kixb|g%XWBrw+7RuZ{zplG4{9`6QBa+7lUX{$$^>@iH;W0(vsnNa4Em8JJg+b& z;SI46A$9E5AOA6$JAJ4VhpK5Kw+z}68b%0tZ@ZjXsCa+>j5Oa#<-fd!q=^0KsU!z% zQlFsjn!vsg6m;~})5t72;`d?!g*n|;(vI>{d0JDY%bG9eLanpR*veXTW_H8Wa3<>2 zR0Y?sZ~xk6l6JH~xg_SMuMGKeAY|#8O>vTtsEMUvW~*E;@f29*UUeLr<7~=x!q~4r1nQg zf$TKx>fGBdwkY#m(pPaPAIr(Js7*6~Xl!7&45C|2+_dVXyD8 zUH`@c&UP?Vy&o%l7kathaM*raSZ{G$oiTImmp?o&8~a&g!}T-MaKzU*;nN8YVOgJQ$J^$;Ap@}U5N znS1z3$rYTHtkj;o1k9KzKSDF4XuJflov+U=SrvQ!pzEBTlUy3#?0flV`ZoW)?}zs! z>KugIePd#%?u14(J()c4XLaCu(&j{8b#``^aaDGJqS&fM{}^ywQYV#f3qp@J`CM{x zOjR)capC>DWhTo}_ux^BI>IyEAOEfh;ilZu%+B;4(teF($5WxEJ$+uP{KwQ+^!I%) z?Aj?ZCy|AuEmYuQohIY)yRz z{_d}kaID9Wq>DMtUs>5IITz4i>HTjqd;5#IB3!~$5B%}88X(w9A78?Zqu<1k3^ycZ z^qfa=i`8b!$q>YuWtXN(aLprv|Nas|S0fLeSNH>hyHohKjl7M_s`e%a?C5am$S>UY z_xPp}r!4Dv=#2WdM1gzc*kbhSdz6I1#@$5A;MWTV^iGOg2UM!y10XsGgX$`wp9{Zz z;c4=cDXT;J3~fh{v_s?8{9a*t{(QL|nfyGH`Dv0H2uj;1eww@udub!=|=%#fjo2JvU=BtfrlsvI-R^XdblV^dgAH0u1H zcBLda8>;z-M@={i&vIG3l#=D=6{HoF4y5AU%o3AjL{10AvS>L>WMI>6cEo}hr2NZA z%M0-0?IfHnwC5Pkk=B?Ab-v_;$uKZEj*6pG_Aj{K2CjzQRA4ZQpjaX-rV^JI_Gg#) zloCHz$rGQ#m*?45mbvPbVIVCSe}*<9WENS#eT>Tz+j5e-%9@y@<|0XFBrUs&?W5wN zx8?>eBG+)+zvIuheM0TyN?Eggklu(nqEa;(aS_FL9keOBRp^mDiwRG-Q`l1IPESo= zq>)9ZG?y05huQtu`pfmbQEd`ft|MEI$z{_|I7Y7#VJlwx(MT)Xz5=z-jXE~E z4^d8owQoes;I@0_w6^L%wul<}{&iwg2D#YO!~}84*|r5^KtaE3-rGq}3bXtRqsTIx z9jagI{Lho&R69g%3H|fue9~HW)%K4L_j~Q)OENS|;4|HBu}sOfGRqV9Uge3I-Zk0t zfTD-uBc7$ehG!e_tvK_=+v3|V4;C2{#E^kSsQB|?dM`WMLaBfDHTw0jk6qIFgQnsL zAy?s(CAjTe4Y{E~ck=3_HbO<23O~A96TY1oZT;ZnL~vC|vQ{iF((_kA_K!+g1A?Ck z@2@1)loq9_o&I?AY(dTY#T5rsJpa-5K@sz1b_ZMediq^L$TEUeAm&mb+T?0J_BgwzT5RJ_k*L5eph*FAet7Rg84KL5!+{w98?Y&yI zk-V>d}Or z&SEPXFY$b95GnY_MbR4LO!i_tY0@d8 zgmt(igr>g_%UZVxO|K1Es(u^L)jPmRrcR|%W5~J@dMAe7sWt!h87aPi3lux|>Vd$k z@bv7+?6n2n%0CY&80%HTF}+z+fr#6DIVp9~T=adO@cY2tEpo!K9X3e&DSwlYVo!l& zK^PK0^4^HCJIF9{UiX8M)`OcOt)`qZQPbpxg~I;5A?EbSTFWQg;~0y{Cm)*QD>jQQ zNn->~T_h-5=!BQ!b1)n-5(V20E*OQXG?LEDuQjyC0eLGyD%l3>f$j{af7>EwgTZM7 z`5WV5L35}2e)Cj-NN8jhgXZ?Gkg=KA6O(!*#d)%31 zT21Z{Cb}t?tCj06%u^<*Xi=wXtiF{0z3mgO8gsXz*c#`P=0U{S(WI;U3GlIL5@Zi0 zp?+|9L_P}OYmq008b*8ACDeK|~>znvuEPjjpp$(k5vbW{9@^+0oFLuuICY(wvnkI@0QGr^VZ z@n?Nu{6C~^L-w;#JyIDVJw}A(+91;}Ml}$^E+Gch(>rW9`IYn9eD8pVa2C%(pX5RJ z9Pf_?lW4zR8yJfd!?HR1Y^jNX0}T2&`CV`g;_=kE*gh7Lv3yNiqb)r$x$n4)`sr`8 z`8av<7jWPUMzIe>eNHE6n}|!%x$ls($9?Q*Z;U78(tkamo?Ak7nm!P4-}86iLp=6U zNXv@7Su)NWF3049h}5Gb@lYJmavoeld`2aIK|E@}s~~Pug0qn>C?=}piCRXbO3fuR zR%}tegY}pE)#od!Yn17v;v?i6Zs}srEs`f$pxUTS=fK$rSsn$~`+J2u9n}kFpKBqE zjtgoG_U6cnWljvVx1H8IpD;r0Jf3WHR;N5l3N8_~Sk{fJ>lC)sfkC$Pa`F$0-vn&_ z**PcfksF~x73xf(J1kmI{VDPAgRpgi`DW!F7~hBmdOFHX;Brx4XjQt?ns2-=yO`>{ z(>pPwtv?r9VG}~t@{k^erQ3RRxEB9HjBQe!>a?YRE;PJck3vtptMQSw5;k<4kRwj6 z$$*TFM~kG0#Pk?45roE-TeCGxxHvnU)ikB00`AFkRWWhvk{ug*M2!O0jL;7Ro8PXu zoUmUTY*i;4CaC=s|M)3R$J>zAF5(Jj4ktuiO#F*Odi*8g(KDfcXODiUx5It9fGtcd zbJ$#@)8-LRx8!zGagwJ#m07mJXw+xDm1}Uv2Z^gQ14-ZbtBD@2#%bj52J)$?sln#2 z8~GGjtYA>1n=bkbx-v(?lG-B;>*e;)4beB?hN{!)j2(D_(=rzw}M*$Y;%3%Unu*O8vTnP5dr2-fGjkTA}m!rMwMG#i+ z(D8ktp+w-oDBb9M$=@lQ){&S~q_9b!sCeZPj3vTglX)TOh|aX-fZAmTY*+1=(?;zn z%w-FKj;PNAex<+Yu*}D&pWJL8P@kA*^B#Twd>*GN?fW)cZ?E{qH4nRJ@9&9gzEUw@ z%ooOFc{Ye6QB}ey;EldTIhMoQ;J~l+6Wye3cIvM#*a-gep-Tofj)}b7omik7zK|ZP zKR0ll(Ez)3hcKLiqhs=PT{!Jiy}}j~6d(0nyEv0LN5>qPG_o;p`l$Z->%$u;IBJgy zY?{`o9cJF!?!`^$%~Kr<5j47dW?i-0JgRpQ5ckjj(Ov5BNy`rkELtP)T3Av&s2v zU!23q#ifu=905x?Wg(CCeaoHAMm7|lg((f))|`m+DkAp$?Yo8J_eKY&({79UTeU(R z@88Zf#$u97yuOie3wl}~!_YR#=H@DfF+D{WuvHsRU=3vqu8aJ=FT*g58v@_MLb5IU z7^jeCIfQA3kVFRR8c>0u=xf^;4-W09hIvoRaA)nd4ZH#*o(L{iKZ&49y45L#l$v4hq!WrOQ1?;HVV^4TL$**+AMuO^EmP#Y5H0D$X*IkdMf#l+TL?CgVK)V$TSd~tGNHT2 z?t^tcwQ7|m($!M8-D0eqVTzdbF#wgX)8n5i9CZq~CYNKnYQq1e(CWT0W_gmw~!QiP~Cl_3BRz9_=P1JcZCL=zIJ(S{9 zB?wK19F_KqgAnQSV8N%J;*Ska7-EEI>y|Du3SLz&o5!FSKS~YqIfk>};_B{$u@IyD z@fScRsQjOFbQo-TPL9AIABcZH?!UX+91zebVx{yuLmv65z?TT!6&{8SrFFbCFrDE! z1ld1+qj1ZS5oqxGw5GwEwTCX~d1K2nZssOKUlZ}Zj^ic=9(x)ehC6D@a%EdV~D!=jQf&_RxqiHMP76zrH=1O?siTqOv zcDuP{H4}e6HRg&?(jVecNm6NTjgeNE!q-g8&TlYlsijw6u_t4>)kz&ex)whXc}mLn zGQ`kwT$rXSz~fGx+laF?JoPZr9lc{-O0Aa)#cjQHxc7p{&p!vnn3|uuKXSH?BexlU zw30;IpqPPW=YFw#erZD2Ru)tqhv;+n4@;=P%WD6g$A;<_JQG(;#GXd!xTo|yT5<1q zH%$6s*D2chIxox5Uryh- zKNPZ0`jCw1KB1i=Pb?-3lNdY;i8`{lP%L|yayj`wqJJ}5ktC{eL`ji0NEPc71kU^j zofF+LjH`N&O2r7jY_^a1ZB4}#tw{L=tvk$OvQfBj`!wmwADa9=N!d-H(PMxJOE6I3 zy>JBFPc#ksZOc8HqPd4)L_b!zaC8tTy?qA8Wn3hG-;o_!@qm^cK4|TUV^DRzCT@}e zVnpKJ8;$SBzlS#2cF_W0uebWApmC1%x?S?SVu}iF+jnFy36N5Z2eLM0NZh9Ut-_f3(P=PnL`Emnw(SY7RiS{{= zdY3y7=tz5zrj9AI(+4T;F?cf*TPz#gI#z$*sESwST4}D2y}zt&+^K(_2&nsT_}%1t zLEi-#q^?K)=&#uoL0=iLVRl945n^^)Qd=_n4*Bz=Xmer~rzrKC>WQ#*_j{jOt~^Mk z2yVie^b$Pr>I=+l4|4o|pF~Yc*^=f)GsTD7bJ~YjX~8W68jsryE(v_-IVup0Nd-o?!_mpT0cilSM$A;t*;F+4Ul@&GGt_ zck+jyYw`kZsYcS+`5<)>@5H+D!<-6sVo2pv_iQa9EtYh)4dNS-v1D+#%?~|R-V9c6 z4u~n|!o5B$=og}Gnrx)dp;DjeZs;({hnSJUff6 z^TS6ydSGCvK3V#^pFAP-g4Hu$e8;9H{M`FO!n3-eYDZ8g<>Mn$wUvLDxrpaE@$Z{$ znip4825?H2YTG+Hkn_|Io1ll0Lw+Tl<}(NbI>$6C_k=}?B5!}r-=1j0X2=V+iJMvT1BS~utIG8v#xm{2jH6W@PLb2)`MN~ z8myl@+P!y7Kc2!FmwMqLh1Gu%3N|7W3cGMbJgkE}4^=gNry9qZ-?bcZwT9W>&p%b2 zkH0~6#Y(0$7g;7xd?p}BooOtO!=w^dS@U%S`=1s6Eie5dp^S+fG7!?;5t6Cl>6YxH zLChz`vr+j|kWH3*P_U{N0}XFL!c-N(y4%gk^+F{=>>~e2KNCTaSS%v0gFCVL*uZJ) z5urcM*AYw49^$4Ky{`n^Zx+1KuEthaE~%|Ejx*7;R*GhYi|>7=Jlvs+eux;9}dD7!u-oY3j6|B5m$1LvzcZZj-x9Tdh0 zLf7VuL*&PW3fn8%qm4-Q;K}$@j&i4&Ra`Wi#K!E5h-E1 zMDR1hJ=Bi5Tw4l)6_Vq2jxriwZkBAo!x`pl?2fPQ6Z2aPlOOaZLMd9+t zsK$(AV^menc^b0uLu_sSAk!y0E~(&waV3|N{Aq=-H#A1gO=Wyl(iImE)AKL)(rcae z?W$lmW9t7r$}eR=QD<)b+X}J$CAa-A;#ESaREoCm>7?cJYv|fdi_EwKAGH{*ZY;;i z|Bl*hCeM2AaIk|hu2@K)!1#6urtsbl1`%GCIY0owKffSABS6@&F$gb9qk!+E@+EyImz@IZ8cPI@9VuY&T)Kqvf~lWuooBJb3YO?Coy)$^JC)PCJ7w z`bK$GjMglzrYqM}Q7r@2y10gZy(~?4ZwIQjjfVJ1pM#6Gf00Q*L+Vec#m@NP$C&K; z)c&vXZ+6Q6mS+9}NGeHtkM+`HuN1&PzWK3Ta}V(=2C%!q2I59F;JZTL0|u zXh&JH3Y}0M=W9G#>nE%; zn?KjUrl9^Ue`#COQGQNw2DSgO%8&Y}YG9?%xen^v_Dw3g^)INrX^~lx;XjKl5BqD~ zOI0qH(?>mJQ1|I1J-IVQ$5Y(?Sc$T@B~F`HC1_164*+?*T*dX+Z>rPE2_P_&%P$Lf zhIuZK366%y?pu%r6Can^x~Jhm`rnWB&^hyVSulxF6ZiMa*VNFOX4bM*RM55WX`taP z=ZK*V(z0oSPj+7fo@j^C-mHDH^HM%Lr!_|HXE7HjM){?zGrRt6EoI~Dm%nXs8y##v zNDsbnjM`W!v~qD%Ad_v}srglf&?Io>{=uCC4BW#jpt%h@tn7T@eK%|O`Q{3?XhT9L z4S86*C@{cgS5k5i-#g}kG!7JOIT^`F%1W{B_;XXUO&70t0@3Q%i z?gpPi`M0sdU*RpZK>YGJ-L~E>fE^y9A3xedJR}QDvX2!+Z};)lPwQ4!(eMcf%TN%{ z^zzGO060q94KYPA1t+1to3&@xXSDutltkL_9+W35>Wn81I~li~Z>Q*vTN`Ojf~a<3 z9bMaaj-ERQzG5U@p>X%@1?o@O1XSV*EfinE35}*7ZO_s2Bc7yT<7VS!Gd5$}e}J`? z-`13^L*JfVpIZMHQvNMou!Ih^AD|~+J;{emC(IW>(tOYxmdA@h4axHeM{AIk2HC|UwgT39<^Zega?3O>{6>}A#Qpj~Gl);umdbv;p zkp6`7i(UMqRKkG5Oy9U<*`|70;#LM7d#9UzycYvKm;d1rI(90>*8nSod8JDWQHLwL z$!72qEze9TzvK(`B$y)8;x5#8gS}AxZ3u*f_((GEL8nMJZtQsK)F3^=xGh*&N1wQu zx0dm8`|GZ#rrpmb`DL`ED-^M^?S8wGvAsO^^0lYz@HQ9L&GNqTnkKrYb(p^YvmWXU z(#lTL#=>vm@`qtz6Fx!<#g}kGeIC}HT^`Ht{L7!qA1=suMMazzFI+-Lj!eY58Jz?abV=x4X@0=2^Ug8#F#^ zFa&f>(KuMUCk!N@P?H}{Ey$0vyT+A2KYqz%0B8qwu5ip!zoI?H zx(2_dZ=oZ(7G#36-FCN0^nXcxdY^w~{6=~Szp*=f2~^Jb2Zf`q60FNRr(3+ zuCPX$Am0v)yruX3C7e*-&$o}|FZF})uuHy>3v0I&{%{L<*_gh<2ps$n!V5H?k+eZ! zgtyRK|JeNB3Y9^Fd#PvZ-_x)h7@Nf-55K|vG_dP$`SR0~G!Pzk$@9!CKXRFili=n1 z`S!8=$5H;owN!=G6@w|Rp|CD(%;~Lvlxo_`@0Hna-7~%X6^R6;hlXgF&(G?6^?F(N)(kvgWZGlLbsGv+C69l-V}mr(}5}$qb!5&_mBItfcqye!TKp{-6!# zqtoQZas)Su}4$*IYa4l6!D+ZsY8L6`$SRA|q( zg`sVF_!T!BQt}~m71dsLnPYOCdbfVvDT9DR7#BQXfFyi;L`%E;7NM{V(^UV`*$t}U zkF9s0EdiwVY<<{}Qg>lIv4x?fzQQ`pjw?JaQ28ap5n2l6ud1u#EtG>aeCTQFX;@BG z%dc}X$YBb3gv)jmOM`N{Fnm9A$K^y#y!{_$sBf4CoP5ph8MenrdqY^UgBmkq%kR~J zC&`L2E|sEN05}~aWudfAUVz>0L#zrqZzWQ%zYT@&QlJw5e9(rV3JALTdm1cDi zRtm{#A;c|~QH9VXkVIX|Wn_=^B=Gd?nBVhRXrlOIPZQcv`0*fz0|gUYzDIO45E%38 z+;E>A+jbbZ$s-(LY)sf}x?qKTz(`&|VI;2e4&n-rC`BA`Gj_f~T$f!qWd+dkhnQP% z*i3 zM`>)vSJJR4X8(#`$S*_`A>l5F;$G76sqlcm*e$<_mcM+>omA;Yp+h_GrXk)AGMCDK z!RmhiC$7K@akE9;O8ri6{a>*1n}kt*lv~;`?q`1akw+UdEyn>`nIu zATVBf{*4vE(hoq9pDCV~3`%A)=Im4*mvVb$=>7V~^MB#|#ngWE5N&<^46RziU#DAv zzHYHAwo*tHLZ<1T-w)0LGv|SE@^zgG(O`EogM?mNi6l{3Sfn((*gZUETOZ>Ro78_b_Y5t9X@A zj5e*WqUT@YW!o|laQLI{1b^_7>Z)u=&>g(@w<0-2Z=NN(XhobB@RH9;_vM9=p)?&i znxtoUr>J|>`j64_W!1EK8Nc;5acv5HU4yjk%|3d&%N`Ub>EqLh8!xG+P4#?64&-Foyt=ATfS0mp(ssdGnZpsobW}m&cl856n_G9 z%=QXa0Sb<Udyd7eQyJ@7y45=d+I9bYFL{b?qiX!204k+SI^Uzs^|qamlLV zbo-@@b<^lrSJcMo#*H9W>=rZ#!E}mtgNl1TliD5 zwmb|cPXas64bU5VY#Jt?4*yCnS60{|4%@|H-{}wXW6FacF|IsgwkNN_gypgIi}b_# zH%&+7A7krUf6!hGZor)J5)7pZd@A)Wlp{EAM@?Nz`v*IAcl9rPBnIxqF4T83;is_v z=g(U}1A_y!gO}VdXeg)U)qIu^%5UK1}`lUk^A373bo_nm<6b8!n;r(MKuWzz1(v^N@dppS2YW?DHtThPNGk zkuv4{1vloBNxn_}J0A|3udV-Ts#xme?fFxBNiT?@)+f+b=7?Mas|~wDaB@l$A^b_Z2gOO21pKakQg#9&ly{; z8)aW|fYW|=MKBcndrG7e_(-!lT-$r)7W7q%E*6@$$QQyo_?g zHO27`9!b)+{lm0wQys12GU01+Seb;6zfe3*AH1@j1{h{2B!-iG&e0nw8gkEZ8E547 z#XRL_j3hk3oDJiWGM2fE7V&vPM-S8ePadSZK3YkO=QWs>LdE{_m@0(OAkWuuqt2jV zqu~k#FZ62Hmx@MYas-ji%py`Oy zvSr3M`-g?>+O_qxc6ANC@LVTt;|HlRu1(;<^=miOliQ+gyfCQ6r#$0)VCo$=@Ru82 z@1UO@3N<^J_^)OGzmTQHKyx4Ms!Y&-y1tfkAf}uaE~%woU6P<@wsg|oXfdR%yXM=l zsE&T)DjswrIv=Y2=4&`0n(fkV;qKeL^1}-TQxZRjb#=eY>xQf9gL9fXd0QU(X`(4^ zv1{Q^m+l;(E8p8nSGTy)J4cQ4p^cYc)=X>W_t3v?=g%(igX$BvG}4l=pe_L}@1h#| z$f9aG@YXqca35n8w111$62@xDJRI5=HH`DT1l`8h)hBxCftQke_@7@#u@LK~UdFm+ z46Hk7dvA>1by*|byv*NQT<+?|=JP|G*OU9Z=>b-P7fjh9hF zeI>P3jnLcMTzr(Zs){~wb3q~Gh#2Z+Y!e6aQa+!Gf9I?fHRj{+;Bf;{^A??^YRGA>s0pq2c;Rrocvht z@OL8RXQfH~wI9R0BQWJPR-VM_mr>&4&r)LTP1O6`S9xoSE1@YyJykB*!1217ew^wq zyVWW4nT-*VRaCk3R=ze-DZb{D6kl~4^}p~pG{CFUjB(|UA%62R#(Ewrgt#`H>=OQ< zFtNIe_~pw=su)>A!_6D`#gTFLerl+4(FX6noRV++ocUR&So~V5yXn(Znez{S5QwQjOxFD;- zaS0N0Zb9Xm3~)FT4V7W-uI__IAmI_rw}Zd%6hCppebh)?q44+ZLHG>4yeCDM^B55s zy8OBps^58zes*A(gG=T(s!J>Txm0e(PiFju?NB);&-g2={4u)X@+P{z%@Y~|437`j zU&Pn46d&~4Lmg69rop&;Tw_Z<%fJ~^S7DKyYdPI^V>4aSXft+_&k;TAgQ7 zIcNvq4D4K63AE$S1FPFB=g8lXpgcQ@4Z*;1OYTo(?Gq76Fnd`XkSF(}MRkP&4CGUSCbWaz}zrpGfhW=Opbq6Dnf> zC03+J{b#VG-kJ2k5bZvgq&=t8)XQ4@y5=}ty1tfH)$^#2-!#7G`c|shc9yo_D`l?! z@!&GbAdZVwDm#u0Q8#~yprNIjK6rIKK`yUlE&hUo=jfDI|K?37q-R4lefF{%qi{HS zu!mkclBVUn%I1p25GCa_k5|U*aqR=`T*@tU#YK(u(e-w8neOSQ?{Dp;v$l0{{80p& zSAHYFQT~km${4ItHePN-D;8JNdoQV_`ECF^c&e8kKW)pOkuNUr>z!+^YN2;Q8N_cR zo_W2G&haXo_g-8}{6dJasG#aHRtDYCXzB+^be`y=XZH@!Y1R|XUzFgmHH>47R&8vi zkEJ{4NBs6f`UQW0$iC{}%$`2Q^?7)ZPIRYfxV)TJ^2)3$`O9&5WZxt~m9`_C-&nkY+7`uW{qh9W3^`c`Fc_Q9 zyx+E6+;$#kZ1$yyzYI(Bfqw*>A0M!F730C{olY5qbe`PaOYMwjGY{M@V+H2g7CzjN z59`ffXL)skKIW7`Hf$fY^IOAaRdCiHHUXb(BzR=Xr-rxob6H%y$a>eOaa{L3a z5IlOp%hr!!XY5vyl8?V}^L`4Y7wHN)T_u{IobpLo!H7+czhC}XrS<1DNl}i++`=m&|9^`BZ!5XZa2IdaDc?>7dkGKcRt> zhbh)_3!ja2qd6YnQe0~hF+Zgrl(#ec;Q{UGAL1obFjPy{Z<`!zW zn(-MvK)ui0P3ekygVo638J=iE)^p~{ucOA>KW)l@tfo&qP09VwF^255@J6cMcpbk; z;+GE_zDV`yzop)T`wW0m=ZGVALyZIH-_)tCP_Y}D$NzlmUlf*L>6=37+4{Hb0P&id+@x!g2n+>n}G-Fw!(?@w+^G=cu^)i+m_tFgwbn|L{@y275_pMFR1ADOYoLc~o zPlItek5LMaEh$t8p&m?K!2(|V)wnxvZzf)x10UM`Y9~F?Zu|Op#wB6(@-{5w%doM1 z4c4043c7549bMffFAzuQg_ryIP-h;p@@lN#_FnXMp6aI;-$~K&REDZ~#nfhA4SvZy zRu1ut(%Wufx%AXo+SYB$58jtvEy|z_?R~qOesPfTVGZ=Ui6d07APbT^&j@iCwF$?u3qcuv3ug^>=(x8o^4q6KYSz!2*Dv3(}D ze`GaU{s!K%j`M_0bavB^o;gIHylGW%o{$M&gdbH1O$J0XR$Qjr2?ay3gL2#F28qTD zDh9$QG-4MT`oL!U1bB#hHh&`+*)tF<63o%O9mxwH_0ON*kZCl$mLK&dbpo8kZYwM= zp8YBM?!&`$`&Err$a0+dyyE12xOi#mu6@05gS-`o-}Ps#q$SWXimy z{Fn3hL2qcYZyS0~_0V@&IT3z80x3y)HdXlz%MX6mb@`>O^m~_>=d^Kv5+%;tzI@Yr zrjP#3DL_o$03(>%DtZ@>wFf);=wF}dqa?d$GDEcQ&;Y&fy}Wdll`0#rX?4n=G;QOR zR?nTW129DIKF~*d`K`?dZfjvh3S+{bNL_K@937JUyJSwh)kzQRb;=NvMw-s>w!D|$ zs-O?w%nwhFTh2{HWBpcbJ>BV)LA|H?=)M=ASQ3+_ zuAU*6Qy@{1y{I;ojK|OtB*&zL-os~T%KN)J{wbh~wIwIb!^K2k1C& zIeYod9${+eRo93zC`~UtFW8{a8OElU_8myElBqesX8Hb3I>hPKv6jy0o%)h=&z2tZ zRtgOJxXm8nt94-&)%RJv8aQm7@l4XaTYIRBVL|!(k0^w`4#6CbDPyC!Jd3KIXoxGsZHVHtMT~%C0Vm7&}Sf_U2@Z8N`}RMAko0FvyG=O-Rx-@w0BN7xDn+Y)pO}#XY)+ z2HMf{FdHIRF(?3kU?9cGN|HUhOUNScs zIzg#d|AL14{~I-|T}H9StErm5Vo4+I)ce?1_zWbrBax%`Q#T(#-f;V;xJ{H(V&jJ> z_52S5Y*Bt=Mh;NdWB(u5sC7X7rjv(g`1sE#wet6<{*uk?!tt;C0uB89|1~v_7fKd4 z=D=wgXh*AmNr&l3@-S}+GgV5l!Lt_ckrb~mVLa%h$&O$DRabw4Q--k^rWixh9+Ag` z-=o2UTdDSz|3XzYyk-58U#E&gUncX;#ei%g+RPyjCX->?3d)aM%+dC55jx-U+t`pb z@!A@Sb@3~uu6CZ)FS1w%op$uf$S>znSz^R4RNO7KZDs1_C2zmhNj)94bjKw%)W9>< z^6E;uWId0U*Jr4MU$i~PTjBTe@dje#IVNrG#9JLa{(`^cpA`+ey6KhON&2Omn(~al z5ORdKg{5fU{vO(aL5CP#{rTF?PO52|Pw(b2R|~$-$6r4Sp3AO%^P!-5$Cem#-+q0A z8Czo5B3VLwD`44)SMPuBCRy2@rstpOq!&9d?z7DE*Oz1vZ0xwixKt$QGas(Pt2NVy zWfuVJwLFui?c2`L3q8nC1%327S@oYg=I!cF(+PeX`r?v0`pv8GlCqqxy|#hgdbpc< zO!mrY#e6F*4!_z#4;^8N#$}7~u7{W3?B=VFHZ3iqQ~kricC#3NT@0$RIIrS4&7gGg z7~C&|*e6$~&F3bY3-Gw`sBY@)*K~<@Tz0g4V3&F~PWV zJ~$dYt`Tm~LXDlZpF%TihY@-{ph0H9fSQYW1~ec4NCe~!5hTBsmt$)9TKjnYC7e(# z|9RH`;4nS*+*$gck9E*C-tWwde5|Repi3@pqAz~9m9F7cERpK8VOWbkh<1>t{*UhK zqWgGDlrg-TQ9^)80e(JljXTt}j@IlD{&#`G$FcG6;%?e+iU9!gX{iSur%o7B59*1rWBnneA?( z4C3@V&LrvaXU@>SJljj%9LbbVs3bFxq6b(R1O-Y^{tVX#KZKu7Rne96>_hnUo-TTp zS75;)TzK)P?fK1$eb-Y-SFGZblsIi%@~|J|2VTDV;n&RcA5V1BVfT`F5uX{f(xj04 zW3hHJ1g!ZwbGV!CeQuDhm6cXqee~VudqZUq0>OZCnV#e~IXnh3X4U38T4=lwrh;x* zW8V}dSwVL{x0w(N$0G;}c3l2iUg4C5jgxikC*{wK4AIYD@~~N5?r^X?v9$(El%wYC z$pJcuw{gaVVP#xqio?#p^vnqk%d433;){SVE`li&+okTTm!xCMC2j@F@9`0SC^zh3 z`BnZKOh%DUzF7SnYH2#qYrcT*S>dW$)eEl~zG*fkTEwyJ>(~JFlo{MV- z$eg4lZiceIQKdKvJLd5LHCeK5TsoB4&222T5-D_pzu6MtsBf3}1}N|{fLWe_8~7Wn zEH!q%7aGKoxQ0qanCv;e;%3W@c;hL5nVcq#X$#9jtJ6UG%nx}+{$O>~w~N1DX0eyA z{gm4pm1bqoUr_4oIS%eH0AJXdjvW6Eb-nh6NrB>yo_@;UETUYb>RzBXQ@D_(bRz~D4{}8W8;gu@+$6q7#%B$Vk${+!j=_wv_z~C22 z!I&hS-8f)u`3pN9lko{+JobwYdKxblB_aowyw@+TrMpBK#4BbVe)z0W23h{L9AYOHUuSLadEf4Ye6<0Ke>gt|RLOsL!+e zSg$=!0giD#Vqz_?=`l^8e=IcA&-|s_LgBfy-JLz;0F(TKf!KwDz1Ts0JN#gLHwoj@ z**Gpkxv^y(X9xYq3&*MbWcJHi_)o^@3ZY4$Cymbx>CnjK@ddl5t;K`PIJvYCPqcr= z$pi?Zb6{r&g^BvZ!AmdLC&_~X8=n1PcbetE?))Wf(I|3vV~%F1YUAgm&wd_199318sN^eZ1|q2*?5$xBJ!=;SkZvfl)l zav_eCPwc`6?#2x*)taMkoTY!;pQg%k_C$L`TW30y4(&PZUn28cpy!_B)m8pmc$C3$ zZ)~hIONxo&e^4UmO${Nie>ow#VNb~vA_Jp&93?IZ~!%#CjHFy)PP^yw=z^zVF7qFK=dU?fchV0*tCc-gQc95zY)0wB@@pe9`= zE^nik-{xKXrSGZyvw)(-sr(Q7f6x}u#9+MkkK6G12gRQq1()$ki_~D65@xACA0W?t z)3!4c-8NT5ju>ghl9WjsVFHIpgvhNEd&r_OR` zhCkTAf1~A>j(&%d>%PicB6z#Pf{Q4#Z?DC~aNyOWuRX+dm#_Toyv3`V;)@z6bLvq_ z$`L;t)L`uDWBA~9%B=bTD}w^Y=|k~e8@aPy7C4ay@Hp?=^z_T2i%v$ZtZKZT6^uDz4sY)&e~P8YSgG2|EPY}tU9W@aWqbl zD}|34@Pg1WuE!^*FIW>ykboE-TYi6-jsT0A@>?GHz~vmDNyo=1y#7pGd?nsJ{TV)= zh+9Hvk-2Dn6V{d&*f7wgrNNI$z7bZKqR-&V0G=l#ALc?gJOs0sy-EMXHT~=z(sA+l zew1!M6c*mH`VHxi@e9d^_%fUec!&GOO*}Hjr*q8sI4{8MZfl@3FI`86?-`=|aKj7= zDDntnZrCv9Z*IuLdGv9a;!{iiMc(cOocIen)<5sK%I9T)sHH_;?izlA@f`p7g3ufR zDff^1+S^HQc{%fzR;EbzfA1K5htKmt`C3T% zN3v)X0fxMl>hPVDBf!xRaSh{qN{Zrx>%xMsqfGbc}v9(!saY z@jzmG58b@AlYYUU-`&ffnC)j7tjqzQ+|hCR@kl3~BecCt+eJ?>?R`eeG~WJcnXX>{ zIV==;EB~-ztp0oZU%mb-URvh_Fe{j5t0#Xhc7&UfN!n5P5uMrkAmiP`*fz+`i8fG| zK{pS4ZIEWoOw{zq!h5KC!g;1Y$O33HSxKXa&QsmXwhQ=$Z7)^^%tk&fkHUnRTi)?3 zSa3lydK4;kOG1gig=hI0@2c`g%_YvZvHC3esPU)1{CxXLm|GQ|zK^*CENaR>PPxaQ zppNrTPUjyeV8#U_RAt ze7=Mo0j6_MnY*g~B^+@Slj>Z{ee3HV^pX6~J4CCgf5ybhJffN#O2Q-mqnp(td@8HNX3@(VA##7U*h8w?tb7nJ@9x3z2QZ@gkvB(Huumh>>gzR06+jqL_t)G zH%!p&hrty&2nMnuxf=41?jPszSG$hC`Z*VTa80@V4YZSAnw`gQGq?7)@J+G!cr!Ip z7Wol&fKtbngPI2^Hwd#liFZugNatO?j<$pooJS9j(5JsQPGiOlWhbf}z{%2-To|3% z@YJMvfILm(B7XzV8$x@AGA#@gann1zIzpRy5aat`gcK|T&oupMy8q`ny5Pb#YUPVs zdigt4d`$WS-iCg`C7nF3Y@w^J*gzKzPSS&XWWf*i+s+^`hWwg<=0e^0J1hNj+$5f5 z%xqy5n6rB=mVeB94j(?qvQ5+V=l4-RzsP{wbSV%l;Eyi|EdV@JGoD6Cqic4;E_5i# zbL7wl!kW)e;kcA1Z`NA*;5X~^ zqSTCvSEae^%trqlqlvE!=VLp?BYPGjTG;I+r6h(exrPz)2(-m5Pyj%`J3kRSZ-t5 zXCvBEm=?cL{_OGAJBI05o_wqE_D?lscmS2uZEd~Pe{cU6-v5n{kJ0eR5Fb82NoSAU zN;$sDJkJOIH}k>v7}z#q=c1^9HDflk64YWg+A+AL57fe3{ulI_s-?PCGK*)n@tQ4v zrAOaw{M2m&OiRn`y8J~JrVjih^Ws(xmX6r@A5cE`8sf`RsCoC5d_dZhB5D}`h>NHR zKYcVBliTcuhG(fTJVveScpD_|+JYRGXNwct5*7|<#@ll!!ty9|V66Y;(Rh{7NHmoX z`PBAW$TNRZ-f{c}Uaa8mx4L0|#j#NhlU+vEI>+SbJM=l5Kl4YZ=~O7A^b8}T0gvYX zO-eTQ6PkExKk@nBGZ+ml?!Q~~R4zX!B)ZcCO>VQ1Kg)B~1P~Wdg@;lor{E3`Ca3Y5 zEq|p?^c4X1FfI_64DZpoh3z6Osxf)uNIiDhw*QHJs|! z)qA6s$`>At&TM#BbH{$hr>Nj?J4q9qWydb@ZFD1(l16a z8g8UlzH}3vhPMSwZxhJ4BuJTIRx|0}@-%v2gdQ&O!Dl;~sdZ~Vz5V&a^mlyDP(kX# ze9C{^nIempk_a>7pEBAIt;{cqeHhz<`OxCr&;)&#b&&0NwQi_y6fHQ@Jb}Ocs@H#8 zdK0)SSXo>I#Ce6gvzswRKt~gG@Utzsz^j_c#s^IY?=q4c@LPsn=3(bYp;XnX|0y<; zbcbM5Ewyq#q#euu;17=R4R_6S=_y@w!R`*?FZ8IJCvb1%-DJ0a^B6s7JDzxu3)%;N z+CrD`Vdg1qC)2i+(UzxJ`BY8**e{MTll-6qHZzs~m>1kx`Mv!|UpSxougIF-%+BIn z?K!43x7k_xdipQ$85Bp4KFjOMB|5$1X=*;MhqO0x8;$7zCivDcTWer;HZ5)n84E5` zC{%3B4I4)5Vbs;w(tj^8vL^r z4CQlb(DP*qkh?5Bev0#x$i+mUQxkN~4;A^b2uzLgW^vN8r~b9{quCCzP`)^S@H743 zatXiqW`9{*s&^5Hy;S^5sLL2YEm7rY@Qud5ESSzg^@KHkm($jpZM# z%nzh#>Y0e)(P%AxlD`lUxk_`#51P%y&eCVo6aI)6;$`WTQJR9uZ}?{E@eE^jz5HkM zqp{P!_2So-UpeO|CU`=~XY#f0rhLl}6Ww$c z2D4bWVVTh{n~^#e;b8t|r*#ne{3-v%J=DhnT>pyo)WzpLe*c(-$8l7jS{~x?)Qzz4 z3RtkPyw%q~?@+g0@o4{2mKnDOfG=M?qM2iuY^@mnc^)%=`l}PPV|yo`tJz7X;V1=u z@ptZp>uKA%A^Q9K!(o=?qRCT|-#7yi9ptDlztSL@$!&JTN}NbmUlca+w3 z6zBrpwT5$qkbnDS<(tp5F9^*CHH*V_qDI*@ieO^^3@3`S#W6qes`CetwJIo~xs_jK zEhZQ`CE4e@W7PCKiQUVaweir3geFSxLS9{5i= zStw;Zo@2Va&?{|6ZCZ&{MShVk$eVJm?wUbv#y3Q>JlCTS z(Gn~F$d{?O`!}SIYv21Wn*1u?xD|MUNSyp9@q#1DfXSN5pHNam$9l%&prn=fa$Lb0 zJlNn9pO#HK)IGLeUV=zJD?UOI!%-o>ky*c7O@5<^q)5HQ#bss&{kLw>{L#d5p8PYd z$-r?N`%dRs;5S9*V3pN>serw~Y9OoX+I-j+#UcHGP!OOrriYa_ZrICTTYWQ$04gB~ zKBjlGtH~eJCyXpU>cj|~3R_dSGMf@zD*-2X9!6wBdPzY~!hw#i}h5k)$ zv+L1os_9=Vi*uD9`7Y%0ROH`yLmwUMznmIIHPgt8-^WI98xRPDqngEFk04`^M!*J#F-wEBfgaK5uBzuM<(cA zyS;c$dFAA1BR`WxY|HXo`C#$~ok&FUtE8^=rZVHay)@}p&C^Cha!o8S~OF4HCmx#otlj@W$u)IW{&tiTH|n< z;^w4-EoEFoXPqA%5HUwobgAr-ExeFg?SC`pwlZEUbL>tjXXdz8<=@O_``8{5KDVF$ z0f*EWK2_y!ZlmXI)7YcObHvB-@vQ)@v`rQu_#onMAElq+7KXBa8QzKhQ-9M$dA#zc zY%YKHqoef6a3fa}-%|CaSFfjwcooC`#X67O7OryRsoYq2Rrza{fAB~-8jsi>l+Ur4 ztfif<+k;aj_@C7SeVXBKoX0esYPg+tPM2P-&dYvJ;wCe+?Pcw|<;r5GIUnR3x#V37 z-y(PY1voUE(_)zw3-A#i__CR%vo7ojX_e^NL3H@jG&sxyN(8f<*YGWAmRcffqJItv z@*|E6k$}yqQJgfk@J>@#t|mc}MnYerpFcWCe|Mk0Il?#TJdG0^Qra~3&wdiqwglRc z#tiZs4#vssG>yn*x~?LBM3qR8^)ueEV|SL$p5rnfyDhw&OC`s-m4D@GT+%cmm&>oZ zt)l;9#}83~w``x(J5D=!%XVjH2X*p6r}(w-H7}pwS+C) zNK4Y=2eXB!bQg_2@&t#nX+E9$Ui4eKm}dEB@43nUEamv~Q<#=BUQ0M8&2T`%l6J7g z-F7W?Zr2Z+r$?Vs*${wC!vsNU%CA%@U5FE+X?lT1(u?Jf<06oMmRH!~Zn}y(w)P5T zY4GtRTNagCx+RSyy^Q`_cbuG%3V%Ey9rJD5LVSFadcYHm*e(qp(6m@5UAuP)E~a<0 zg$3wgLI&iK|Aa?L2X3&IH2J}7ODnNCphAAcW0`6h@P?}w#jWaqM0f2xk34fDJt@}eESEkp|0LO zzJ6NY??){;CzJj|qWEWtn>(c_cOh>=6Ebh^lxk*~D`ZxK_jyYv>Qz+)?=&@%x>1n!jSAo{`@w}nC zo4UHXc>KX=9#nHHIih+$EUbdKRFJ8NV%_A|0C1)kENLA%BMZeh*VrxO2Q7+N*c6xV$l> z+D<}H45syQNte8*4R*r`>>xf^I~SgJUk3>xb^I8~gQS!esFu_2X^rLGvjHa^lMmw^ zUZbB7x%ws3i2R@tYjN$eDY`*>Pc>6~6LSL_U!ISN;J_s9vSyc{EL47;Xmf?vY&2{3 z_)(-9I}-<$uiA}wG7?o9zF@ipVYWSx%lmBRBDTd=Qz7H=l6(s`{QchF&Ge*4A@7Ki zp~W?|hH-y@+iu{Z%_?B3Im`AJuaGVK0yNm<5@fe zedgQOn44EAfI~OhzcH$df#X`7e;Gmo<`AjTo5O2r$r!m1sR9{RfbsG1!EbC$7q(N2 z_=OH29r+djjp1eukhhFBicKF=tW5kHWa%ARHyfc14F2_#2~E+@CIkIkT?@EUFdkL` zy^X%4$9TNwgUoHC;a^M}C(_8C$qMfouPiM_tPk;>cW5DsTdTP12Rvd&G4Mih^)Xd9 zD|xdbqD)sO?aB9sV4%jiH0}KKyC;B+#I0x9vpMbeCXW=UKO@5ZrNrxH$Dxkqbt4--}<33gz&As}z8;ERpLccwzhsKx%{oUzDx6$z7 zW@W!_Fa7{AnGD#=1g-U}-OMEiO7uAnNqc4wlkgDx37HB)mv2#`Q`~iYh!UQePD3q9 zGhy#+zhGH111__s=Y;bHV)Xsye^(*;2v-&0pMEQCHNM6_P3Ls*fAIp{7Vya3Qu2lB zxgD%Kv4Zk1=c1P89SvtIEfG>mPR9L;MfZZLl!xMT@8x&1r68Nv{su-#17zduLi1VN zd)HnfcymP1)}odh@AK2k>HTJm{QYa@E|)l(h96+~%?mQ-BgpC2ACq-Dl%!Y2W>k2a zXLY`W*LMF?hz9-!i_gxWCS%JrcvILy&5!(8!o<=_z{iN$Mt%iBK8bBhj6or~T~Y z`%p@zxN4S7^oXWUX>b)ThKb)0CUt|>e8N4`YkzZ;>+ZGlO0?N8Bc=nTxwxC`J}*5x zf7f`k&VROv4IsBHFCo!!j^}s=!L_uF!n5J68O-WE2=CY-7kVZO9Uy7GI53rp3ZBCf zzF48|lNTC50Hsa+a^usF>=$l>@YT0p{{Frc!*V)6D-J=|p=Yz{ehwdgkNq?4exH=GP>Qco3ijQ_eVo7j%(}~cpsPCf6Ou@ z#}_obLki~JJ+c%a!WMh+InJVDgIB#LYCpIly-8xhdkNQ%Uw6b;vD>+yz3ey)UtOWw z?)23G_cWAPA340d9b&;|KHHW?YU+_Qjz3hCr#l7UzRmM&B4Pq+Kq=j>dBO!WG`RC5 zh_CE7oL)Bm)cVQ%`?=66?c4TSLy+;=n`7PFHfIWEpoWy-Zy)U2&KtRf))e=-DaWMv z=78WaTJVdjiejLWR7&9fD=-R;%yS|d4-S;4ox(qZ_L~2(zsbe1>45$tu|tyJfgvP= zHatqaSEzN;;5F2hcq9B;8nk-nS}x#ZF^Bg`EM#UY>~7HxF2+GIv}=0x`$F1gA!dQY zuCrX@@-!V;u#z;#f@Xs`l~NEY>q#8Fs)hpprq``So6to-=`8O(o?%Mv*bh{LAwx<# z7{$S{el{7flb!#GPe&N%gmSivGpC_Q`3a3&=~1+M;AQ_8|LZ1$dnt#Ll-jr#n7Ffi zj5O-d(Ve_RmoA75fa>_^A$G=+rq3&0_@?@INtR((?8p6DA~SxdB~6;*!j-~)hNR`f z^9SyO`+ttx=pK-6Pvs}ojQDv9@c zzF;>|)hV8%zm3bfx+GVr7sd?zjlu~4QN|6zUvWON(BZ!@g!Kb?dr8>A@4f`&X=1i4 z4!5yD(&1)ZC-P!Nbpyss^!8LRX+i=*Pc`J#6N84r*7rB8OSYS|YYD(m(4Q6Y`XsEa z<41^xIA*}<5w1`WyndpxM63yrBtEp3Tzq!>b&x5ldz#pJ4jmQ)lt2Vg^DyTlJw(#| zr6iOMB}SfJmL%L|&vHu*P##he=(ZMJ#8$;hLFO56;^Iy~j)mQ$yjx%9WA_eAPokQsUp=V>?jx$i(AnO{d^;Nk;V#rp?WN3rS+j}UNSU}$C2JRLp5?3@@k+j1eOn^ z>5K+FlFrLJ=4KY~8a9;T`SmSaJz^%@mzWPa?Lj9;;Z)Z&EX)e)`LL3VW=MwJ(0EsdcZw7fMXw zfd;+Cf6NnDgRm_4&k1dzxMFehJhHiebVBo1nBQ#v zbRK0mSm{C8Xm~|qBNo2zy1u=#NIK%Rd^qw#@HRxQsX(VUoR!hfQ^y{?s6-$Dzf$F2LK}FlnaTwl`JOiE~32(G5 zrp-?Lj5&1)a6bfclva!@b0y>=vh)@z9B7d$HEoHOg4MBK=#5Q%*%?Awv~JcM;AgT_ z)~;vfhbEE!I*3#Az@D{~ts*WK2cfPBWr)=c{nH=;mMLHl@A9K~>}%U7z8n5YB|NJ1 z?+n~yJA;KKpGW+dnG`)QO9MTz&Zm;ehTlCe;7>_M#F`sIqqLJ-O!cH59XA?wnC1FeY7(YD zzqB=rrJ$7Xpif-Lr_TS_%cBB)5Di*m} zpgRKsv}dLcwX5tE&v}ww>0Mh5fessk!GMo{M7s_l6uDY2ywQzo2TXf~*xBBq0|B3p zNK#2!cI|#I*>B#sz))Oi1$?wZxu^-sxSLBeJ4us=a>Xh(?w=)y=ZXU!Z6FQ zwVhphn=Y2y?j2GdWR`g!FIu8Tj`is?&P|l@oPi~8OOBDaTEN^;{Z}vcsH??~>DHA@ z=Jv@(6`Oq}9X}XoDEy-0>`!;_^+q!c_QHQ4JeTbSlyFGVbFj6LkP6}SP2x{0a_?FG zVv@AtE8*yjryZAqMDS9srrtzjic1%Pe@Z~FZ=maVb4}EnH+`;cr)FDd7{Qnw85<`| zo8G_aHV3!wUiUnKct6IR ztj8r){uFg{`Ro^lm)o2RIGpe}JHmrhrjeYQdY)$yF8T^S5<5RLm8k99;6}ebY}ADg zH_n)(V@@?HfgQ~y`nf&bk|pm#+1hrYk8ZOhH#QrmAXzn&mihHRH>{gaJ*?COW$Ffv z6R&Z(e_n;u#))92}2O9~N+x{`fUKF<6cI2<=0j;A_j%XL{akFG6A zgK!(l7ePf&DmQ(_Nm)QZZ}2oSa#3rnFGDNUa?R6dxT27j^K-eiaKnX`rDVkXPS2CN z=mG7^Zz+i6yn$NQ-Q926qK8YNSe-LN4J@CyaYx)oC30D&B6B^Ogj{HZ>R}#2${Zuj z@cVHf1}<#uHjb5*Ab+;5;cl&akx{CKRveIyJ|Hh@+s|ZgI87WJiw7r#V*wBtgRDNKV)V(kt2W1dYhwyVAj1bFo>^RwE#HnAr8JN zFpLGiBJOnC{Yb-T0vW`X|r^J*tX`sqwHIvdJ0Kt%;c^1UI5lju$^72d3g^|%r_1y=~qfw-lV@KyMDPmI#@ z1``zSrcHmM_IntN1xl`M6TFUFym=1x#hd5OyC~7ppOe!QYf;@FY_=LuYCL*15Jmbt z(4y}Ph?}z1);dl1%*2y?^Y(edYZxl8^SIZHembf3XP$YM33nwOhXkAfhf_7aAfw?fd)eZ#=gdIf?V((HU^9Q#gT124kkXO z70xi(m>sxWA6#cCuMX!%-g8Fxz%bS#*l1!6aUL-tpYly^AKYsjg6up|h*^U;FDb%EsduzaM7Ps`W6$4Stl$ z>TU&Bv#6#5CGMkJ#^D*D23sOC%AMlnJ(R(OPXH)_V`VfYfZgzsrMk$|U@YfFH}B(k zHO{BlW-N#Eg0gvw*5pBmqGNMQXtEI-!`eueANy=b#W4#|gwN$%s` zy@0@0@4_Z~IqX{}odC9j$eSBN96H_n-KncI5T6!fb2QyaG>OsMho~Athprsf)!m-4 zblq6?vLQeC{Q*8`QTQ=yS4)O=v^s&0M(61kQp|?#F){6B)_-MZTw{C&swS=aI@f)B z!oQdL62mFO_GvD8F-1j+Qc~+l2e1l(|K-Foo;EKc=31|fbMs9bc!BxxZ8W~?Kt7Je z!HDG91IX~Y&w{_vJ#2B(?tc2vl_ZOU#C3{ttPC-1Rngph5e&n~oxashi0&F9Zu?Pqtp75p%eq!gXnQSGLqH5Kp-xfU*1*-)Jbh%YSFw@S zzxMMErZKb7Je2gzi2KXS+3uI{(N7Z|z1MoK6OROixKGdQN^s#wu*dt3FYL#U^&v25 zS(3)q2sSgik4_GJc}h-0d>SRPlmp_)C(auFbnea(3Yw5H$e(e3F)Osixex@}b~fN? zx_beFdq{@7`l~F?X0={aMk~Oj9Iz^GcFMo24;$R4aE(Q zr6(Cty>1gF;%XedQU%%IXkTl`4-YF4JvyOOI1qVyA6C!gjcTbr>Fx?1a&vQw@I0;5 z26qJ4(?!XY+Bp=tpD&Fvr~##bCF78$=H_ucb1`Mnai6Z~J4xd%kOU2*k@?b(&d$l^ zl=h1cCQbN+K1ZLTx`N=~?);-aubA{T`}algICT^n{Hr0g-=HretY#rQSn$ zxT|8`ioNASc>NTm{;c$sq=abm{nO8_Ed5vAGwq8&KLBKfKQttKl&|Qi6-RPZFR-#x z&AWky4$vfCl#ZOV?^Dk!6?TI=t^B}0g8hA1fx$1mv^e2_+eJ}(C&6p_6JSQ`KubbH zGk-VFM#^0eceKQ?*KzcwT4$n!yNjzks2`}}8PW9HBJE_G2?dT~$pFPXZ@wa(CP(5ZSF0iXo-NTO*PYrZ} zNK!8TI5wxNe|8E|C=bA8c=_&7Rks4`G0wH^4>S6`Hbxz{h@r^y#?E@nS zC_+wp0epx5scI)&oFf4iUoLBS7SU~@3>cMs_`E|t8i4T<#)mTm5g#6gP)bBk|Cn7) zrgcp$g46}L5}XPvaVx(PH?qHvM>yeM1UBA;{)~JgRRSy?g43Q0c8luB++4PqE_bq& z)rWbvz5$~;30~0QM*;5+?y?DbG)-`@)SV{KkX`%3M#D@S4Y4SjifUzw;9-roE_xwf zn7M9b39-5bcJeW&$CR!SC089?lOnBCSo*gi=`mR5;yv89XQNu&8;5z_PnAFO7Y

    ?0_%>H@Lo!BXl&j%Lne)%ZgH@7WAV?rYFs-|G|7iGJMo&ojvH#Eib_d{ zZGlsoYYk@9E%sr7^Vh?IQ*!rIy;pEAkALH?rnLaG?5B)*wKQ7R-i5h16lS1_e;MT= zonUSI*ZxLG>3v;`&3<(X?-lnkTAqtf62sx~z1AifpOWFzbq8hffS6<4wT|{aGU*Hg zP$-DK6a>W}3FY}Ui%Kt{hrXZdg_9c{RoEJNlDkiyV+Bx|P!vdHVWowU>-zl-3jDtB^bTa`yM}%RtWI#dT87;k6-Yxw@X?%e?02B}jpl(Tv4l zc+RKB#QVmYJN!fYZ}$Q0?6jXk@xxY~Mkq0etWYTtwu`Fy5 zy#=J~UPY)JBH?`cIVly3?b&(rnv{$H8HQ`8T(X0g^qPlLX&YQP<^-*Wq(7my9K@ZW z#*sL9u`um9JzA0)EqOq>51=hrPcg!Gv;2=*Ao`s){*g6$Wt@66K{ddfLEZYA3Ix4d zExqB0`I5Ph@KfgnZ*3Ureaad)W15L}`U`YOHW`?O(`n{5qU3q;H4tEv_)Y=6ys`W( zsa?hnuR`1fRv#Wc2aIcweoj|?uKQ_JJ92PCzszYZ$Yo>Z0~V=kg}!765*z#(>8pXp zQC2&$&qmk%{2S)TGcI@8bv{fUjgSt#UdVM9vjatjC0|*;Nzy&Ng!M^q%&`r$aZ0)` zh1H!|y9I_|K6~q^8152(vN;xb-d*>^VGE6WG0>*%_UQYl70?O8#xrx@BWv2`_`lrWug99xf($FQZfGq{|Tw%i&|P-2|YF2<0%dxXs^MXItF+b!kA;7~!cN}7@#djt~5_sw3U6GXFLOOVgEsGsM zxcK3RW(oZwB6;`<-Q)=e=QV!Ql~MCbKCfq`)4s-d73KC^olG86X32EidjkPQ4J=d1 zNyoSQ;EF0ag|_7TxZ9vHN~wxONcFj)bGZ&MwGMV3K+_iBpe$w4FJ(WA6|;!A{j9whdM-toFrrn*Y$uiP1JhN$I)M=1a*X~#R@yn&nh-O0td^uKZHxT z)>NvcHl2>Q4=KLYxB_weo7C9n&%*Y+?=J}YKcLg3L~&8LO@f>Jr%Xx$Z38Ed? zpwl)6`B7+HXNO@YNteDD;;9k5lQji+X;*@WA4KU?BcT!WQr|x(vE^Mw{-wNS1|)SXm|xtx!;hWUODHey?>Y z98#%mos2bm7Y0vj(~2d)9(m7XJI5&xsQq~)uvsQwtp-&&PS%Lylh^iTWP~I z9`z5KjN;o=&+ZyzG19rj08|FS^nArjjGWS0X21Fv+Y;V*NS~CzlL;vAiWTpApNZYZ z1hpFsV@LxreYh@-di%O}0%$%QHg2f(Jjfy?8!XojS%a3AebYCX!Hz!9!{#(4JX6@i zgJfeIO648nvqK3^6OK3q8lJG;yrqbmgSZWll&Bln1*IobbIPotFxdw*B47x3vf93r zQd(ai`mC=0*>cnZjn8ZijKSw>3mDIP9V7>cG=>cd|Dj=bs<*0r4ehn$vTz^C)1+p@ z;F7A)evQ1YNzx!bDgP&5AM4Rn7|gV75t*gT{iZdct=;F4z?#x?<)gGOq2RLkbjlWK z$Wf#=xy05>?WCe66w7h_lB8)NUvmyn56Y(yxi>t|Qd6-X?5QP>-h! zIz8Iz7_uEo2^-}5oBph%U}QO&{h^q_a2bgCb{Wj#9oFJhQbEh+Xkj5t?2^{OI+{CvJe2H0KkIhOS z_^nb9$@(%5sz>YC#I^Gs8z+VM#0BulqlQl1^uowGLn5orqJbe~JPs-82DRi58b#^c zjOurej47hxbrTHuD!PC(Ti7054E4+FQR3o{$+BW?8S zij44wtB}1s6i%uNw{t=!OSnV!JE__xrD5C~1Mmer% z!Wci#(|wiRU{4`DZ|356iXb;Qz2JL_A|U=MJ-5KL2FpUoWDqJWY{v`F0lJFbScOoq zJ=)f_!4Qd;yVHQ5Gu04{<>QF*WdZ|`JD>PT5HL@A*wRncZbRpZ(VHNy8%Hzh} zF?OyK)pTXoS#LM7;-Mh^v#u0!>{$D}G+v6Y)g-0{k)$$3j0)ydKD&2pYjs^pT$Mdj zbkSm0L|FTHH-?S(UE0D?hw!&JsB{rB;W*+ylzFtHzPiltX;Dzr*y^G(les>x_($`9Cfct)b z-^KoRE>#36JVyxa*13EysA-!g5I?dZMjmA6kO!|XF0`f-DKT}7mI7g(PkLasK5iQ_ zn+WuYT_3p8`!z^sHcJ{3&?k<2!!Fu{wq&<}W9!$uzCi*H^$(#%F3Y636OpNv1ar-3 z>kxT*@p!z$*6SbNUtc$@MMl0(Qiyn2Y#cp_=ZvINZ0-gpa`_alSU*!f>3MzbQ;2*> zTd$o5-1lX>+`WXqpwQ1WBs3d=Xxl{asvvBZa><083BxOF3je$Sh?b% zQmpyn;6@i4p)rXqm=YAdywyG#Cv`7ent30%gGBjm2=%I7Ae8qR6Vr_56^&vNBv31{ zo?v>LX%U|gKK8evtbeXo`Xs<`qJgd{u zGdmZt4$r)QTiLb0x{_|+(75N@{qz3H`MoYprLv+oRwxIXsb5)>v`#6^yS+%IUZCRFM=$**Q<=*tIXL9NZ4A9z4Eu+mZ?|=dvoNg zbSA&KmTQ?U@ec%4nTz>xys8hiTuTZM%aLPzT3fM=#(Gjr0%A%nAW~{;O~M0EWJtb( zm_AiNfg*#a@iSoUw?XSd(2%5P-EQW=J6;ZyU1A~*w8^6jI(B;tQygtg61nFuOL3Jo z9y(weu(fwMZm@1ce$&VyDV6^78jTWE_XwOftvYuAiQh1b9A|HX5LidQxvCE}TyST< z{!w%Ia1Ot0^iV7>khHvNe)l1ZGkTW~*I5XcQmweCuTCa4mZGl*+j%x#hf3U%=;ys= z=&8MWexdQ?5qCfLI#_ccu1@Wh7#EPlFyIB{eDS%VdO7R%SGr037e`^ZNo>^yPZbxh zukL?e8kTXtycB-OfTKx82Q z{D{Im(``6;l-T=#@rvj_4PCmEJnaJ&o!(uH-&ba2RqqVs^r_nzM@WKam{&~*)-G1l z#eu$>RyW?5Gj(hD8-50)JMLpEqOv;dSO{kShzqtnQs z&0d8Z_F}RX_Gs(YRAh>R1)by_9UN4yYIC|xg6Z=!jhjsqma%FW$%}3~;h~k%`Y$AY z>dW@L7ISP1t(~RY9|IaHqk-qV@GRM~-*!(9EfMUQG4s>Q!rrrQgKV(EtDMHu@+j zOUoN&)~Vl<+ZBGn$j~s2V9n4dO)`QUmhwn2iOv888}A(0#k5K66F4z+^q8$bLmBPW zul{(ye;KebPN+?X6cf{-J>SlII$Iw#x{6xkwGk*^HVWGBF#24`FRVQGKC8`)x2(B2 z%@fo86}Ig3;V-(LeQa4*4kj|D+@_Rf=Q*aAQSzDD*~wpIUf|i@9FNS6QvaKthzj{g zh&k^^^&~uKGA{CW{jxePqqc9pxN2SZx5~7R6580m2I+}XF;t2X^z`R<&?yp1uUWJ+KeUCk`>Dwq5~xg}4n zxVbINmq779ksG7Ps7o)HCj2{B%H3Yc@}`@{GIgLd<6T>@a7kH_l6G8K83A zDibQpct|07($f=F-l!s_Ax~*?IrDX`(90mGX=F|IDqwx>J6QZa2|mn%EW+(O5`ec1 ztL;xfZ+E8H9Xq&c7j^&#{Wi~araZzD?C!Esh>pJk_2BPTp)NWBY+Pw|A%uqhJOW&5 zWW0igK7MS?k9%bNI`c?3?ihNwL!Er~|=J%#_i^G}+nlT`- zX>g6y*0D}8Q|e@=iF82`9V(|6(ou41@#bF?{49syjwW?;Sf;eUzb{G??~^;!V}-2L zEp#TG-BPtrJYSC45t3Pg?Zs;~FCBh?Bdag(Kiq=2k5|#JSvvnBb^8!ow!Vidl22YP zf=r-rcc=${nn$gT@*la^7C}1E0QC=8n3%}k@066hH~wk@)Nk8GBtKap^$^t9sW8B9 zbv|?8oC}4Y;}xXFYtqC(9+#^)W74MB3PSn){Alr#m=z_6e_T;Cg^`@~LR{sb!PRsG zYg8P+t=5B66l1n$hW2EHvmeE?giyS6wLsEsg_C%!R9d1WzcwkH@oVe5 zaJsHOx*??}9nIO*2Oc={(K)59)fg!3a1hkhX0F=)mmwLH>p`V#;xP>1;wdE+!AOR z)hhU^o{fuL{q449He(>OTOTW($94; z1R*$Oqp+Hu_g+^d0n}!#3{inv3P`$-`?ldOh5(-pNG4@uXzh>}-bMDpUsL%%2ya6d z09Z6=8|jNz4yd^a_5_`taJUyo^Qf~c;}yD=T*Lh(oagQ4d**qTD@-{4ttuyTEBx(0 zpNFv-4rWP`?Reg}+Tp{YAaF4Zfka1bnpb8*^|VFGI!G4zyI4q#VOfWKAN|~*3IAWm zeQgYP30K!90s8o!E3Yz9Z3Vi0WFve1;LsZ9fZRH=z6x$$+-^N|IxT}ERqRS8Qsr*b z@tZnlmw&`I9T8EvYi4+(_GJYg>*w05@)muHH)oC(*SJm}?t+h&>SqUP5+zr9q zl)ZNvXv|aXu~7St{syUn3~|AJ!L>m_`$eCulik=f!5+2TXcwIU#h$)Qs0sT^_-pOa zE!(n+PNg-51{ug|DWg#7H|W$7b+gPC-Xw`&sVmBBLg~leJ?=Bp4paPnTxjT^!6u=Z z7(xpE5M}77an<8|gXaU2yXQ$Uv3A)Y_fuaP-{J9AX~!4eU)S|}t^r}BLl-aejHmPe zg+_2ts`)_JgMf?7k28}Pks318|mGVFx-Qa zR2!G?LlU>fEs$0ph@ek_GeaC#~!Tw=>Ag;%au?H_7J` zm4(!O2LzX?jrvNNUo3yY_k}GhZPU8&r-#{v)O&(ge=ke9-L=88->r_A+!Sv;p=0j6 z6<)BX%*)H{AJNcuK$#PERU{|o?G3%bqtrb~kbgYXPKS;F%>J%c1%Qkrl-*oPc(fKDhpLPecr>q185W zYGaMV@$kw651AjvLs+jW@R2qDt2TT^+9oJBNxb?vHu+y`gVVlIoRmM=mbVshzma2- zcBIzY77;)~g4r5_?nZ@y(hcIe;fzumJ;xD+tDYHWz z(KL=uJm6OzeCUw>sGs&}{9a=2X{EOgjByglx0)PX6%0bGzXxRPGE0n(R) zA{%^-B0p}eaFA-{SDCWV8ORpI7IpP&Q`WkCFQ0gCTAcRq-}AxWWyuomB#?)h^9k%% z7?ksr=ruuqEXN9Uj*-I&)So}GJyfkG;nPGF_$3CRS$ai~;OS-Q`pTm6u&!!B!7tZ? zd7k#!nKedb9IgHXr!N(Q{J$|c5Rv9Pc5E#&9r+HCvCF$T6^`xLM*HG8A#1Vui<;B^ z;qlhX_V8W)n@8}&<4BFN;j+41;PxCy-~z@;++Z&k2ltaq(RCEp+pni5Ln)yzt2L+d z8Ee`Iq~!+0LX0AdN&13jyi93v9lBC|ARcmGc^%Q;n+^wXswts7|3%^oG(>%@y>6!~ zdlOt(gb(GmnGYydeEAx&{FuzsXd*C_-mkE0jRfc)KJ$YuHdp_@vomM?)?S zp8g_i?L|xn^~M>tjDZ2%^cam2>@<}LY$}GHa=jK3m{00lR@1NzSH`Nu?URlFd*skY z#@HdA-D7pc$u8;aU~FwKdPD9xaO>ZAi5Fzy7d}J5n#UFNA4qg6qzjmo!S%CHLx!gk zz>Oeuof5LM^s$dpv?F5Jc+Mkh+QwJOv+!i*&<$k?3CSu7Yksp%`q6C~yS&jct3l2R z=#-1W9I^7ML05`V2V2yIoGK8NF(7o?B!WVz2u5U+IkmzgBPM+2KzD)i*8lgfUj~b5 zTJW-gK}r$)|5X#Jz(d*mKu2O*4wVtOtIAVVLseDy+2_Dg7`k#yUI9|sblE*Qg-cFP zA3@JlzO~JF#+>8%oBMj`nzen@)hY(ujU;AfKil^SyaE@itFH_*J*M#N?54Ye(~HZt z9b0Ufy9_dt0IhQ+)@QSeRu28#rvH#&2V(*=aA>Z@Znevba+_mm3$C0s1R=fY_Um`5 zgGVUk#A5JysR&s?3M8*=iW5>f-5P!;9vJB;6M^bO=?s%|8EZcN!uTnI;^jGW^TNIH zM5TnxQd`21-4t(|zF`k5$8zL9!sdhmSMf@{GuK~Bg?6h$mgErQb|Fd9`rcI=f2Fh! zCtQE|w~8U4)>WH?=hzXro)54NuF)X0CnO2xF%E&d&QmJor*jz9@DaO|<1@XnC$Bw8 zt@EoDB$b9)&d3}G#1OSy_5V=I-hK!ShcEko4&t#rkupXKz=)}px%7=ALSX7Hwha3_ zm0UsvBNfH~)j>RmmcX(`F?d{u`0d5V8|p|~-CM2=%x^t|EHBHul|1S9iFp^?D}Gma z(~o>U6$`<^Z2sq(Ycxn7&`ZLsjf)$zuskcRmAmwT`m%ELOZ5tQi0TlJqK0V4wCi7u zi0#>92)IMVp5bBCGgPibF8#!!ic_mZsP%C_JV&OBr?aI+OjyC_4Pni_t^4CeWsKXJ z=HrVOVhHvMe0{vX0(35;WZ_Y{>{k=>F8>%r9Y!N${?*LQwYSfQL0o4Jj`7p|JfOPJ z{KlIShov2n0yvvFDB*HjL$|Q8Aq%}dCbsWRY+J!6mW6x5<_TLWCv}^my=DJ`Q5l9d z^-zn+ahPV?#SMwgURzw6YT2=e#jdUo;v&zag}S<%#eYcpm$DoLfuG9X4w@vuGgPfj zOoGA`hM;51NZa?KoWwlJT~34Ic6tQ5Mu{|=fB7HC2fw`Rq)i`N+t@^|7dmHBQ5JlX ztG^(LT&Hib*Ft1Yarmp>?(_l93S4PP+b1WiF)^y6Eyz()O>|~avQ3H|ZA?FN^|t9h zQ`Q4Fy4sC{3K0U9V^5k~>_rC_a=N&;CcXL;?wGBD9ty1L>V-;nHc8L0EG) zOcH$Iol(C#`UbvKKKcMrhue({^AA2cNg!(3aja&RBluFM1(6^mWavYS{C7D{5~BE? z{}rzPx^j#J8Fs`%G>ewRr{VnPaYVVhTvsrBSX(|DGVBIO=({g|X)qs8ZEN^gr>kKm&3-9-i{{5^QKNC7h#L3lj4=TFxkiTgg%uJ~-idhznymU6N!58LI4r~$Ad z;5f(g19S*aURWuvN8fYvl@}>iXtC;q23rgx&QA}a6aLqz>_Fw^1WaDV_g5SIRC?3O@8=X4VjI{-v1A_|9fFZG#GGo5SPAY z{l9kezb5=?@pmpk+v)4S2kQU#?|Q@uu22iS1CG@HeTDpgd}vSpoy%q^_xN9!@m~r2 z-!uyJrkfFu6QLRW7pf2=1)iQ-_f1dEF&b%$Z7IbGe)9riutjn#1KFcnDM zy7c0f7;?sn^^6{xYu6&)!o|f^!v9uUbM$vff~B8G|C%V0Na^y=L?jQfA2oiJCo1R8 z4zA6`w^pu)c0=h4TREu!{24x=^H59g=@dyysfVv|hMiX&-530Os(-To4_#j!6;-$` zED#8jsyR4P0)FC`E_^Q_5JfdpN=lw>A$Ds zmwN7O$LgXY_>&DU$qaV=soLB6OhMo154&iBKyfA|LQ+r*>uP*=0C zPOib2i;$ZX@>6ubL-MD&PQgQ|_z?;Mpy5f|Zdv}6!JaXRZ>liw_xC|9&sJGNp*Fqr zjIO(GT33k|D^lvA-qi54oo0^j&?L?^5}d3)XuVsdBqgV#P*%TUvh<*PAUusC$ygEm zjvl;(cevP=a)YZhVK2sy3{*;M1ogINJYJg|#Z51N);JHGd#!HcD(9J-_s!? zTu=~ouT^Q|UMbO6veUiAD!@Na?Xj%L9fb^^J~Z)JDB3eGy-&Houw=K~#KYIcC!lHu zqN6b7h4UNywB9A~DWVOboEY*UDSMjJ0@@!?jM#ubKLC$6m6a-x`FT+U9%|Fl@4SwK zEcQt&I*VXOAfGBYrRH;^fBo8L@`DgKJ&W@`(Tug51mqMH1%rdChd_~aV`C#p2IR-5 z0!uCFgBRyA$hi(*2k}@vKm|p3CFw|@$Ifr>_{34!_WvFx-SNwp>30XEE@I@HDnea@ ze%6;cI~nSlk=o4s$fMLm0~fHbS>Rm zi`8Ku!MNRHP!sfz+m0Y};2Oj#JU8fm;~US6-bU_`o0l3JN;Vt60^Jt_ebo(z-4k}lnw#VH^7*4-l~Lt zx_V0GFEoH0c#BVupD&CIx$EWLKPb&F>5jsmn$a^;N5D@J8b*zdS^SV1Fzm?VcEuYz zUsM^7Fe&lzq*arYUQFzcF3Q}w^;pOt{wMt5Xw*S*K^lIZcrX`qJhxxhPg7*f#j=Ll z)Yv$AcPAnVzuWQvr0}E}wAXAL0$YMiT;|=jk)+*b@m<1B!DiOao;@RKUR+wTWiY$p zS)va|YJv`Pm05oX?vVOq8myP+U%`{unU_70BD-`{0mG|8S#o}x5-;|lZWa9f+Bbj>b zxQZyK2_9r%kQ93ax=$zwB0!$|m$KV1@P1&m(Qmq@#eV4sxfo_|76YV{pV82iq-cn#PWJBU zj!9hNRGUHa;Ai-VBq;z;#9h*No%oafYmcj=-@&%+d}9KJP21|2m8 zsd*W~E;1m++(GAYSNlbYL|#kKVF+}%!jK^J^?vbg<VWR zus=!|5?LSV;MprB^W-SB#U~03!TBY{P@dOskTE|Nsh6|y%R!+^gh=PCxecQB;;MlJWr;d9$<36w6 zJ2luZBE*pYn~e-G&<9g<>X)}~eH7oPKJ~!}yhzFC5QhaNe<~l`6<@pm7|<9Qkq@+R z@VncHvT@salmC#)flkjLrD~*#E%$x|u>^i}VVIh&HaZXnUkV?f(B8>p#h?1kweCkr zLkl<@O+E}V*;v49c~Gl}(=Nl#2KD>smeI99eBOLthQ_&}b(Wxhj?qLD?9oI{=FZ5v z&SKn-8R7fZ^(!Kc@1B`e^x!PksnE-cTLcFb2M1@M z;yz8!l(lD|e^hvlBTf-~h5>$^)9FrdF~FEOz+$2Dfa~HT-)osKGp&mYX%ic=~Kkq_+9D1+F?lSDP4PeRH z*oGS7E~vHLXlo5YmpN<2kjGX?kx8r91c3bCzLz;_3IQ}iLPE@Y1RI~J|LBMMJ}#|k zkQuv%2z<^k3%WPf=Mv?V!lb04Du%;@L~qMXLz&+ICQUvia=kE!x-t+>AcSqxHz(Gy z&688BQ>;xGBH4ov=INAQ_K;;3H?Mw`&!PYwOD|?4)t56J@|y0ypRyLjSAEFF-!2V$ zcopVD_m1(37ksUO2(9Hn_K(j0;CdfYaJ1G}dp(^0KfDAMbqmfS!&ybJSP|jKvGuVS zI(|Zm|0q5C`>yn@0MZow{D&4S#(uj(t6;ywDxTnzyXRW`BzKzlxHuzlj)|^qP{OOH zWgS{e_g#f~H)1iIEKe#0IF((~*qndZw3e60zz_1_F{U^58G{?Y!51uGhP{V%Ur`({ z)(xDzs^&lS!}X@@n-nNi(2IMCjf#8Xm3h3_oj#<4O)7ei8lMIG6^>aUpAjYpssZ9} zrTwV5jn9l=!POpo`{Oc!-D6ibr2U&+k>$&-VLb^fcN+wztIbdM6ErQ|H7GNsvUpI1 zq2@BSVeij_y`Lxbs_%==15&yFt2i5%Df}1QA!qwIU!b(0SGkgPZ1RKq&P3@r{37MY z+$!8*%?_4z3pUi9#BEwdJl-iACXeThNWj?z4K%5eO(h`)Eivh?*PnS=?!W`JgGfr}%=6&Wz{i^jz4i-|82l`tn;}TN+;s*fy7&1_a{k{H zdW=cl4rD3K*M;-vxSBqNy8XOU7hX~LtJX*zg%#Ho;pCHUzINNDj0)y%>aDgSDde-V#qSBt|g1$u$&4#04&!wf3Sje(-8eix}Y_+xbper{q0R2kB{ z*G07AH{7~&+gT&2bo$Fp^X2w81IBeAUL84vNrnl_2&|>knbsW0K|K z_@u6`Zcsa)kAUuBW+Fz{pa~oosc0A7e_s=q7i<=CYi${VA<+)Mu9-4}@0bx;_gTJi ztep>AdG&MRSPy)yXS0Ft83(-+;df-T_Hqq=)3hV=VHx?>nEcOO zmd2^Zn6<39nU7_z63clAY8>ZbIn8b(8nr_2yi{$63^UF6%i{N)N-FmhUV>J@k@0fz1rE4JNsyDGM$8?xostJCZF`m2z&%Y{he?!VB zjZms5zX}-(%2;*lzO*X}NYVY$ofwn>&!00=uh$M!e0eAmjf1KyA%c57%tTKbaX+T$GXEo-T21<4B5|AJMO^fXzCJ z)CPIyVg}p$ewaYNR>~L=Z`;zPV*qV~8d1o1@&jX(93Nlg@x#hd=Z_#3$jc+tXSnP) zc8H`-ug-_Fhp}is*j^|LQ)T^~@1w0J&XX+v@eZTue2@J&;il8vCFizcwqiSEQ^{DLl_&)|dfEIRi6(=$-C)>RZHyZuuEEiOeyABQJJ1c)I& zY?>DYX>`#(!J)zRSlEBK5bz4KU%pe9yNfFZ@4PE58XrCN^t{2ace~fZGc&BwmC+2Z zMM36EgdFxVcx|_<{;xbF?zdAMaX;B((}lV&iyi*3o9e4Uzj$I4U+aWrG$j)ntRGG0 z8X(@NV?nt9Sgo=w`2iow_OA|rBoGoP-o`fHexHbI7uf)*{@DQLyVp+~=p9o+8IKpf z+rCMpk=*aX=Ej+!PRAs|fOX9Z=-&rRbT_g(x^@4QRmk<2RBijpRwdQAPxACBPowN7 z?4X#FU$>?$m%qgiT2I@p;#dicYFFi+#7viIG5Q^rmu3=R9iq#IyP1z*lX1l}ge&09 zYSV?uJhTD)@42p#o~bWW;MIh3_|Mwf*F8m>0G&s{&!VKqFLs8@0U3BTE|0Hh@r=nB zjm&ij=hEnwtZ}~NoCmZCsw4aJozQF`#yFa3p!>P!5Q87)~0#Y>#FFBdjW_NrGokAk9pj_}T#5JrY9}Mp= zyj_TKfI*wqbRJ-2&8%0C(xDMj2z`gfX#AJvZ{T~<0mOPX`qv}*Wqb2-g>A9|-};3G zyb4uY3+{{_I`q%T>9hF{KE2?3@nX^Uj8otwuUx?E!077m(}}<=`^d3}=&?X;4-04Z zvr%P6X`upmb2QS z7sF-MUdHC|jt9(i`PzW4o7UP|&m-McE+Ov|V+gsEMvhRzBW@uZM@Q4t^_>`( z^KFHwqD9j}Mjpy^JW=ZOmT1B%smIrr?!`TE1|7wOIqPMYLBX@<*vq_64CbK_nHfvx z@6V4rk4VtZXYVl@{SZsHRwJ;@H3L(*HN?YxX+C1tefm+sv%R)U8hrgYhJe>W@UrlU zAAN}*B`gbO_GnKZmmZxzEjf9E5z5g_w{Zm*HNxC^a>u`aQ1<5Ki6?yYJ>tvn!EDrp z_I|KLDLg?rxYLa(i`t#O9g-FBnt%0BFwl}iAQ@v{_iBpxz;dgtY+Rt>MtNFCPNi0j zNh8*U?d4k~!WZ1gwb~&F+9ZtHE2QR66_@J@Mk6vZ>rB|u*03-Ly(4DW40hEUnI2S> z#xE8}qz`}RV#9scu}epi(C9>J)v@r>x{*0-WI&zTm{t*Q_B@cIDfLM=&>SkFB$Y?$ zoN9G&i}4$+|1u1{IJZ`XB(r^{NIdGHbTzuvbsWm}Aajavf~(f;7u06_pC($gdTl)+ zORWX|m!wn-9(csaek%8eZ?u{6S(>!&4T-<$Z?04*e@3n`9-w?P(0GVeyToctYylB1 zRSuX@Lw-Y65fAXWG3ircZD~XirtrP<#HlpUbLG^u2XwK*u~u_(TN8v>I@Jq=*Lj21 zq)B|>?Cd|r3IZ-|E)qqxs?0Jhmb-rXn!QHoiZ(a~wV`{@hbEj4G2T~eiVPZ$6qC@_ z6w$u2uTe=Z;ft2B_RQ#1di#Tc03S)e&YYc8g;c-IoSj$&MgOjtSSG$>3=c#tq65F0 z()3-@?v9-I&X(vv5Dd^vN9^E3F;zQmUR)9KHU@88aowsJ;9;J=Og%Cxe@>DuA7LR5}D@p%k!NfP1oJwMX5F4k(lOm7K!H~ z$2P%LvS(H)N+#S(n*I-Ty8VB~h}={e8#_Ukh-?p;Nh*Io6Brb`PS512+8>+#gmFi? zz%sk=V*jAU;Uq7ph(dk?k~%Y|RpdL>qdPS7|6zr9uZ3i#5xAg>ERe*XuZTd@%&W&S zaO)jD=|!O9x+?jn&6q_cmG(SnLk9ivT^(Wn=jgU#LdK=-VQ;B#Ox0@$q>gRLv0&dd zG`Bp&P{0@Z&jkb$T8@))EzNqB;kY{uzqeBQcQcQTYpR=Pla$X7qLlt{50vpkH2X8K z9KMAYCXQjIOtkrcilaOwIJ0X&Z=WC~VYnU!>}J{4r{QV9knJvV)}YMgy80{79d7W3 z(;F45R-%u}QD3nKa76Spn1WR(gdE9T4E*5td-vPCBHBv9*SOIHA}{;O--GBZ*W)mj zN?!{n{~+2fINHkxzoY*7?fv_e7D1OIF~G~R@Bo5G$b?dRPHR;e24%gR#dAf6cO3T9 zg0FgIEiJGqG*nT(Q3^Y6r_A%{rZQpCd1>Qzkgz0%kau~B-Qi#v(toXBdGn!m0Ex8+ zpQD7ZM@Xd$;7<}OKFGX472wNR?!;V<3;G);qY$ z$KtRD#%l@R6#9!!zT$h|(Wic64q`$0aRXGCA55z*U7SgOA$HWOPQjNkHmk%JlHuMl z0nF!naf3+`#*7N9Mi2Ka#*snrC4YGcLW1^{*4q54H~y;lZrUH1O>zU+V6~ELB%=X1 zPGd!f4^R|HMiy=eac`uQx+czPc3bo3urjqgQJU2|LNV{2u!cdAG-o2+N^9KF&lj3T z_0{3pX6l)2y_I%(D;Y}t4VW`DvxU39pcnMTO>(+)PwLTnWY*dP7dCh5hqtRH`=Vp4}6>;8`!eP2@P5tbf;tw_l)ral zVHnlBQb|&YMxp5=ck1C*>B%r# zW7k{vKdEsQ0A{&zE*!m*J-(F>xiPZy{uVH_So}6)SmxOJH4FM(nM~7oqK9e)2 zH{4Tm2f)`w7EMj1^wgHB9%Kpf%B&Rl8n*Ro`v!lN$YKHmp7XySR{6&PPWZh(&F=w9 z-20w@GvNB2eVUSZb8ghsWGUopn>#iRt3dike(s?ojg{NA_Zl?2j01%Yg7&E&vEzSS zjpZbm68FW6i7!9Ac-de zZqUO)+T^Jk_pj9Iu-Y`tbZNSDne2`W^jQ@mnVt?qbmf__wvO}5LF7im7)1>}gVeAx zB-WG{9c&=w$uNCZ{19Z@h4~?Lb~+JAWPZ_Jp8rSADsvK4z86zSJ=7W*VT`9sE|6Te z{-5jmEr72!%UP-*C}3i%SA#a8SVYo;eZ!kESvIR!QEHb=Vh^J{llDtIUO7tOAo#+n z{3rO+^;ot3!UZ2`59kEnQ49`Zj!(R2%-k^CTSHRlNT!UHjYVKsUgU+R7Vp3gW~&^1 zn_2-%M$pxqf&&r5{>Z0_&?<&giXfL33hii-h$p^aKG>kv6m^6g>wbq#G0$L38V^bUo@sylGn; zv@l+d@L6oQc-wN>;!yVFu|J*kfmC)SO5=;xo^X-4AKpfryip0f=qys0MnHL{=pfl+ zuzZx=Y2=4$@`#qxv+o}ThY5ot2&e?t+{YxZW{q6#ml{G>`eMv#V>{h0WDAw7WgI*D zHOI?6_|}BWmP*tqZ%k16kdMI#*liVxf|e^KF^3En1p49OZYyzNgHMIS(L0uG+$>f4 ztOJErG^mf1$8jPw1!$V+7f}(x`2LCzu$;2S`geObcya(i&B!nP%q^BFx9C?w{mdV2PhS|40R7K8%Q3>S4&5)OsdbRR??tH0v z_FmIf8vTR}(VijEJhyWk8Ze06?0*fvJrNE>T&@)4gX!0V=}&WBj()oRrd5#-XxL+? zvVrgzgzr-y$n!_|CEjD@bd$qi!GcZ3S4x>X#5wuBPt4wd_X#bqh&}De2Ke+hpFVwBi&K>(<3k^w~Mxf(>_dN_Gi&jLY&>xgz~pcWv_{XAw)JP5MT5`9^UuU8F6ld^fVGbZZW1*aB7@ zx`N-54nDy55~*O&Zg1JM%Qw=9QSqK-$_JP&Bn2&Q0O&f}YIs831G<3~uOX68tvb~= z>HH2#yU+33!QCE8i{JKdY4eO?s=(v5=|d@c^`02i>C(}7;;x#e{u@#InYAnvw#21= z;Tkl!)C*xFHHRKO78_BI+rg-@n*l;oZ<_Qi_x7c|N=rPg3_*iDQVD&gN&#FRw}S$W znw10;qc72rn8LMXRc$J$kO*jN%|jBu0A5mQ-s5gRbOl`KV=zw0DXpV#7b*8=EcB?U zvuYGel;y1??<+N7n32)xFUW~A`xDNX57-Ir(OVQ{I-5q^((i>Nwj zng+}dSb?2`Mz!mzGoR3<0?lFJDc!E0pT$?7!f3m}-*pA9Q zk<;Vs7!?_71Mu~m%VD;#^LHeRP`$#Zk;ohrYWN;-|ADz~a^Zs>MPR5)`&$ji4t%2< zHDkkGGCrDif+8Gz6dPXmo`_w|kx=DSdLblmwGN8e!$EX}rblyHEHtf=kntzX?eSEA zR_iIt&TP}x?D-KiYR%=usrVN1ecgLC_yL{>4@eo4gF*StB$$8RD*t&L4!Ygn1K;f> zKf-p}V8$X=-&P(TU>KjTAHm*%i2fxk3iqv&+#!#etN1&PpShnipMQV;u#>4ZmFJfZ}O^LT%e?|@o}z8!;_q)q4L zcY87#j?UqyjahcvN9n>-a5?=(^L$n?N#2M4%PHU%vHmqL#Qa?SwJ?>1RRJ2&J`i|4 zkK^I)*l(E9D2gFwdH7vSAIx`m88U<4cfA?MeSwXN+!y<1eKqjbg!!vEZ)!%y9C=&APUqc#L_ z{>Pj~7KFg$4=Ou-EC^}nt`TW}MYpN1a4-qN5ALhnHS1eudn+|gwWI5(M>1^u< zw+oW!WFVEqY@0?lk#T(kN?}xA`wR3j&0=NpyR9bTijzuJI^fP#?0Wi5Gpl}=Is4Oq z9aK#w_s0-E{j%T@%!ggK9ar27J?h z&#M-;o8v-c>gtMHh@GUac^Urp)N-?X)_h0=VO3hvT%(hbSg#{mw@@ojA>Oei z4&AcM^UhL&s+wSjz2CWZ=bODAJ8628wD21CvrGTw_HEhL(PTdreF`(RwL%> zBv+SDpDLNv&ESz5(Y4j8f2%-_<_!VJ1dA#hD69s|B3VN=cYc8a^ON zL*AI5>aXdlx=afJ+E;GFrLM+>XjUiD*;K`T+^|`}8J!|M#eTuV?V^9Hm2ew0SB?hF zB4LErjzL`J0*BYSop4q?k1QXga`D}jZX7k&sJ}@`!|*)LyGNWXUH9q8 z5&m#q$v&LX^cL5xKWAdsln6@Q%p#F)*!(qDHE3#V;hiiu@k@Q9;yo{}-O7@)Af-Uu z$+Pj0Rij!VH^nXC{XSCDTp%*Lc)-UW9Db}0hNxRc1 z7Tf9-wddUo601SAu55w{33HLiu!9y6W8&-FaK0ASg(omn#MZZ$8Xw8)y}OEhBWjaH@9xp{2x) zEPIf`YJZ9X<;75S5;yrP$C9Y121fq@D4pTm{!WVy2S`(N>Lngy6iMC9nK@ z)9E}?Ak(|Nb5A%r!x6`4ZKIWHtorUFq4o>P{L)z6(XVlfJ5yBX_2Q$Co0x~B{$83m zBXK9xa+3X|AK>E#YId%r#y4*$8TW1Wy`3biEicmuVtWFQ) zqJ~OnwN;XyIx>czwS|*$ES-13f={n59Ld-RB~lyS!JVdCnmp0f1)@z6@6bIR^5Q?f zpih$RPdIxO_DZ@GZ-l?5hdS@7llhr*qmI4GN11vGm2a|4z}jR8%Pc?x$P{K!^=ttP zgzQUnU?3FW)f$2LoInt8U{$2{vv_UiaV%%)kH4_8tAD6JarxYW*_H;t+$kw1XLw07 z1X%CiMLz`=ozF7Q6&a>C9olSv-j}GmKpIXU7k}#BbC*YWixKm%^d+QeuR>A2;sxvY zmxA8ij=po=#qWbsjpAX<6GL|0)Px$mCSm2YTyq z;($XSMxgkGnLaqBvsO}8{+^&CgH`TKtuY|GLAL_5PLIybv5Ja#dsJck075-iGL(Nu z2mm$w^}D)AFLRE3mX{#b)hy*N_D186M(19H9qw;@^9t?QRd>H3jmbRUH)y;L0dZt{f0Xp1;h+wHS>$!mUE>AS`%Tv0|MW_nXTE~0QII{q7Z5LE05tI$^&h11zkWh0 z!P!IKKNCTk=V0<3{K{XA1m)=;C2J_C)a6ZGb z|Mj+58BZmdK#bHE z?K8V1Q;w|&st9fU%_=uI38r@FRxj=gfA)72Q~V1guY3Qlt2fxe9k`YR0}X^8=io|D z{6?qDdO8F*)jOqezH62uN!ytO>$wyjs$&ouh$WXny2|p@L`}EIESFI|s z!Toq=sw?}t(ymWf`K{MKY8TeTKGVwmg@18s(+rV@Tk_A;nC0WWUrxf4GCv)%*S-FC zT6@j)7pG*7hN=Kg?fH;%;uq}LE#%WA---R!uiKJ$>Ug|%`aT^ej5=!g z$s|UEx@336)nksY__L2PA-Ym6PiK}tUtK9bX|c>RxKC*I`cd)Ju^48-`zsV}E|yjQmXZVD5vs`!fp2R%g?`@)vk-u063I z09$ElsP++0OHHl)K`=na6i&_uEBdz${yh&mI6s9-|8DRJl1p-%UJ;ep{+F=Kl4K?m zwqIlhtxwgzDmA>inp`%~P#WH3Q(EQVDT2<6RGR6n&g|bvX9d(hIQ!N+G)Xo|JHyED zImo4*?qc76rw@O2qKJEI_e@UaIgKL624OS)YvP`duPQ4s=d?-bB?`YTKYVL5qA{U- zsLp@BU`lb2`|D`s;aiYizCW4X-uaw?3vG2awd<4PnKG@BM|W6>`YX^eFh(Hei{JHO zZ&!inzvvYZ4oJu^K{`kSi~rzX+^9od2H5l*mTJQ?!awOFLwS%tMHQLvK9AeCNWK#;d zO@wgq?7V(g(lkxCo$L`$QnF0{R*o*btfp_DDB<<6-DUrnyqt&36Hi_kFiLbtv8Lx+ z+w}D+mag(TH;17Y8kP1bU+CvOo%NtBX{sE*9}ZBjX`9{ejL^$&WCLU4S90hvv@AcK zVl7)|zx^M^6B7TI(e%-OYe9F79RxV+`;D%ME5Kbgb-M=El!o6iO1qwXACvOvAyghe z@2AxS(ogt)ILvW1SZ(zs6eUVMM3-phYdjt|nU%c~dZ+POBze{URI&wjvs4b$87|EC zVqG5qNzcW~w1KIsU_>Ar8(X2|?Q+UulZ$3)evsJp$x4o4!+R}yn2w1rV8mVRNh7k_ zA5X+vPTL>L+_w63Rw8E-{KRPXr2Xyg1s^@A7Q6o8o7u&CQSqW^Y zz6a;N;bv`$q%F#uwxF1>St>FacbKG-mziv&(mAGbw>eQt6`QUtInld@5jj)z{q}UwgWV01W9p4>Y8X)JMXmnCg&+dREqyLKomnHn=kn-0P9N?=`%qSwQKB zUnDBKa&NVt21zKOaq5mK_$26F4nE4UQ@e$Q{V1HI#Eww{i$$S(faVC9s>uGuA(8|^ z&;6VSvNi0LHyLY3p>5s((k6gdLjNpjEvh{bh^BTQ;hhHJ20)L9uLK;t<|g%ncU&?L z0P>3QrJ+f+JdoXN2PpcDkcNjWNT(?&$AMOE_>#wDqHa?b9>bDHXAbV8uzmYB*Nj$I zK+^xF81O;PL5Ga(gL<9hfQYx05xNDQng6li!(nC3`|HCQfp;UB_KF~jq)dJT)cdgC z_2CBHevq>q8#b=CUhxM`Y*!D8IDL*V8{Xj1eIgkAoeh} zPo$A@x(}uZpBZl7#+Wh)?+R}{A2Vrc9fMLZ*C2le10@8aRF6}O z*!&q|S@Adf8G~Bcm%QrE%$)icBgFe#5ATSI+nv@RJMnn~PVaNBb`8mCto?2oGt?I2 zA|zrNf{~B2h9=#=nLhTpNQ+lRBvAxTKSt5V)Z^8Q9>6&ZgaP}EB{6f5WKZIbEzsRY z*q-FUTJM#+i$vH3Hr6jCVE#22oXy{FJjVJZZ&%Q6!|&PZ54%?^jZVh4pyICz+JZ`5 z?*bq*K3a4Crf|++g^c${+6#snE*n5wGBYyz(|K#3A551M6QkJX2MUGEV`A49W=Z-B zjb`$ONj?7gPF|S;IIQN{s1J%$G_Wea`OTS&mR= zjwe73mf6F0ie{PJ)PB@;ZyedBkZ6+JGd6rA0}m_N5hWE)(MnU| zEGOYM^P9%CGs{&*lb@lCr*qd=m<0J=@)6YI1w0^lqN2-4JGtReF zKO#>h*F4ZWF6+1^nZ(UjLJThU_An%@ISRjE-rSa+E4q%P@B^w}j28JTWlVn?K6P!< zRAm=elqGVaPd6vL<$p~4HiL>RuAK_rCh0PLJJ{-9;)7nw^EBQ@HeaX+$uj@=8BV(F zC3a0^G_V=Q{7Y^s^7hN;@u5hO7DhRA*_$B7xO}abLfoZ~Q69yFeb(=L+(ygpOP9*P z7|PgqFXaa)mD$&z#DeS#ktkQ6U*~`DJjx1-F=hL2ZF?aRP!txVl?x=pZN|~aj5}uO z&sb1C>>oF+eN;n@A-c=H-=akfEcNF39kSlAKeven+*SwcUrXL7JPB7vkunwO#bS+C z&*F<{bX<^&qV^&Wc~8V%Wyi<=8%sJ!sI`~r) z!FXi)w3xsW^sLDEqN}36;q;p6M2v56Q_>`m_o5_a67^OXJ*iep;ljEqn$QyTJ^Dqd zzu|7M*az zvV!xiBPW78j)HsDsbEFmdZc7KnFyu(aB}ZGDXivPvfHjeH&H?% zI6TU$@JXH=P2t})%^7UK=%|`=`EJyPPj_m?ZGPOwLoOL488S zfiEgwcLnTsC9-Dc((?>F(iX&KpKEiJbAB4v|CjM+kM&n4?9K|;`~uXTbbKbN z&l2<4mzg(A#iMwO8)badf)mGWKt6AoMAcxIO%NUYt!bUk<6OZ-2uVcfUK3Xqsgz zA;^v5gxHM6&fnz=6>Np%=N|NN3=&nCn5AsPOn2-$g642(SvfXl1v7143j#)(Q!q;5 zcS%Y6-0eW$v>j5|=bCl=i27}q9gf3i*{gPNP~KRFiAJV%Rk9_+l6_lbdB}YUi;qw#r97?^$uzSy%z)@A39?UU4@JdHntL_OGwyt$4`T9TVJGI?v2?Z z1VwtWk!IOzUxwmSCTY8RLF3GzT{S5AO7mTjWY8r6$N3~`lP!VrFQ4Cb_>giMDnMYz zE_S2Yf}c%WJXm2sdY{L*EXKrNU9~o~bGGU&@*$Atizoo{TX{7g4>L7o%84{p>}e0J zM(F*u-LcpffX3SZq_)y1SHQaL+|Q(^n_SEc8yzXcOZt{AE9o_uP||&lf$Uv3opcI_ z0BztE;n)VCx(>ySGJX^Z)sVb?D+WF5wmR&rI6%N4h<^8(w&oSr;Kuxb3svM8-yzo0 zaUh2#&BMs*H7}KI5!Xq4N<)ttUjCsw4?YDh!w{oAcwUrEU zAd~n;{vmP?NuNI4;)QSBP96W|q1D$A=|VO8p@4-OCOylm{-SUR>cQshiJdZ%dpHK;8>G+i5$vfeR&({)UE!U?7 z@8m*JoQxTR@8(?~u=%y$ZvE7Iy2C9;Kutv0Z;8uSeW*TG{HKBJpU4x~ zcM!9rsm=pA9TWh`+>d#z`tGSV(t?4MLytkG3?9)tKpZ`Y%vS0p=Dsrn3OEFumqpBI zFq2XML+FLLaDR-~Pui3AM`vbkS=aUFjepbRcHicRYF%%dtV;pK=|*=n$g+(_Ao{7( zTD06_cWYKB@4b%0K1$pH(i}Vf=bXr8Y)4Kh4Qg%%>lhwBs-%KqioY_BmD6RqkXLR7 zD?SdiKJ<#IRE_)a=2uAvIeZ#F$0qXfE7~*L4UG}bBcsPTw*B-A>G{}uB4UmnP(Wg) zH1hY6#AVcgMA_e1`k&|@Xg*a`jX^dPLPitdlvVTx<{Y^^KRzu_>DEm5rp5mdieLLk zIV%hBN?Co5*Q_v;0EDLi;iq>;bKi^nir?B^7d9Dixol@ma}5%krI_U@+$;Ivkq~ws zr>TJiomI+RUTK#5v;FmE{FiFnN)hrr5u>%)dokbz@^A+CUI2ld{vUu4w1SI9WUo{@ z&i881kXTVB(XCOa47T3;6ObVu*AOBIAT;~Vfk2-uWL_4Lh|$7J?XUu|T^5 zY@!U#xMEzY1w4kAq!qOaqT5OGnRgF>-Ma7+RhNxX`Qvka$g=NVuC^c>x?OPlW~$)6 z3~dlljmTWj^9>3O$QJkJ@;rXFt4#k=XBsW~D;;yET&l%+6gjv4MtFor)3u5FsNfag z3+C0m+#O}Si$zv---sal^#AC33#h8Tt$mp8mXJQQl%#-kmz1EiG}0Z?ozl`E-3Zbk zNH@~mogzqg{TKIr@73S^-fxUE7z_?)pS@SiHRE}nslF0W-z=niO9;d-(_4y@B0x&lQBm`%VW+ z>dM!5=;uAB6BB31RHQ8;FNA|G%{G#;{0bD2*qnpHYq%w#j8qhlqaE|ZQH|&0>7y3A zO^}|cJfEWdjn7&a=B0Ix@5mAx+H)xMyrm=8uv};>z51d$I3|43V%e&kHbOTjWH_>X z7?YlRJjLyqrrbJzi%9GNk!YxuC#5+tXQzgNxYgXP-zUMtO)^PObSk*9`=;TKg$^cK z)6-lgNbhW~_~I;>R71~m6Vf6JmOK>-tVu=RDcwwR{*`l{4HF*Q_TSrg+utL+MF`D9 zWO`rB?o*B}HIX9~l@DlF_9wR1i(k}OfFnb0gXq8TSW^}pMC*)!i^c$mA*Ls^Gv`62 z@2!2{krf6-nV_jVP+1yV@hKq*dJ&EQtuS`r{osy$WEd?1v0hO_zKc0r4JJQ!&I8E~ zoa_4^jAg7El$(79(H!Nr$dOXJKEiED`XcbCU&_nP#%Zs6gziQB5ztW>F$O!P8sJg! zmF8=$T=g+T_kgculntk<0Lb9*X}0ai_fwUuG!SRK8dy>sulp;L+RTQlt<>a=Hhwa< zoh-{3I6|g1pH+Zbkcmbnb9yKcx(nS#=M^4p&#{{f7dRbk`sWy-+T*G3wgqVQO@-SC@g_+X^HQ42#s>HNIMT3e?7~GOFlrob9OM z_#dTZm~$=XL$I@7$=CNGxCSQmH*p>-qmB|8&W|6V(yqGv&~|;I7a+^@M31&RS|pF} z?kbdzZi&QAg=9$%qNe$Y#|Kh(QQiy;r$k~<^8shBHz!#d|^HUgL8dhl#Fj^Wj&lRVX*bji#eNS9RK zY(q-Pd`Mz~`&L8+l5v=-Ur(b$l*;27HGJ}*YB79Iq39t-Si4~3MwAA0a$|W+t`s_Q zJMPWIJf@={CVHh$@b%DfsQn?ZjMVBxZEDJZURqRG7(y$T)aqI;iHx8w=Zoftr^4Er z@-sixdTrmRJlZ$@brSh!y)-={RoNq^09JBQq&6ZtgSXhDP`{F03?rqx@>zSzd)v6$ z0D&iQ1oM=f#c~t2=zQ4wKDM;Tz~af-!LratG(WW3Vc+sLZ45&80A^q7rZd&r4d^LI z)5>sP8sbD3iD4>bFPmi_D(c~tzpKvyBw7Ti{EeRz!PY@xP9!CkgY`_gKI{eRdVAv| zv54Bz3^xBeqYu&=GkK#b7vmC7nAZ21$R+lTDo%>DC>`^jlC-99J$Kc*#*4)~cey7! zSV+j7NJHlQjh@khI1t~Jt_8@cYrS+zHPdgW+g3`qdrvrd!IILa@_ieqC&6}`l(DBw z=KRs6f~h@g=}fWDDb8L;Y{S~)!0g)*Kn%aPTOn@J3|17`l)j)+HDJqq=IHLZwiO*@ zUFJDT#b)GZ7Hq>lX+Czze0aH6*gs{sv_fj>BV(=)JIJPGi+zkpCQB~pfVk*BZ<;ULA%JG5OjV(GV}ATT>nyXpJ-{yOQ4}v-1n(II88xu*d;~N`DKtMSvC2KVqMYH zW*r3<{!?xuHaiUFw3jKz@0;bz$};2^*W5}KramV8sq(b`ek`Lb`nti-!MRcez=rs} zf$`{b+?}AGphOUdul2~o2@w7UbS?}!1JOi9o)S{yb%wIls^$@|#JL=A^vOy4D!E*3 zCeGHqR>G7*Z^oP`x zomN#2_4ljrh~qn`B=7z8{Kkzf!~B(Ndvx zf_QU1QC4}6G@dln<=33+W>^=8j&6z!@(%gAG|s1Oypabo!=Tv!JErhlfaoZdonwEkca7yOWb`oId0FTP zVoz~W7S@stA9!v)wBt~k_ef((7|O!lPEor+u0HO1J(1wOJu4}ZF?$>m5v9cz_-g@U z!rQi5!UkkI%`OENcd~`}#qN%W@oCW8_03; zW5q&>$!$%;^jnh;CdBdx`icsAC`i8#IFc59vP2y=gO6TYuDuAfI-Y(ECOdrC{hPjm z%t*a!Kh)=$06)E07Y(TU4t`xrFboX+El`ATukh4gK}R{l*hWcxTcLg)Z9Ia$hVJFS zj#vH`jfE!^9|m6VQxIr@Oy@y!!n|>g>XiX9b8V6BB3#H6xN-ho>S&(FeRKg<7NNHf znPj)<^#v2~XZ1e;Jv7Sm>FSbCOOe@lK6Y=5C6an{V!WR;Ta$hmahY0UHi!3Zd|GZ3 znnv7BMG+z?C5u^SFV_`NGt|dU*LgJNPpX}ou6Te(k{R;{H0Kw zAYjvUhR^5Kea^~57xWfmSIzMTGlcM^o1o?V>ukFa6%(C}o0cWI;+Q7mLvnZ_X-JXq z)Cx5_)(f#<4EZnP5IAE6TNZfOUh>blK{*?%u*jWQObFtF&xqx2RGHca;LI#hV|_~P z4l(1Iq5HY+1DCaWkFV4C=umlCobjhEDGZLWsDx1#A}2mo4_^pNqR4B0BN^uDU`p1v z@PT&`eipsOej+8?ahkg8rRVBEbL{1tC|u?$y;PG+hqYK?cQn z;mZc!ej{qDuTF|2P7{0cUmvB){<4iKJG}i8H7jU~L5nGdvTO*SPALMW% z8~I2GzdVo?%C94G(H?$?kl}daF*s|t{Gpcf{Gc}CSMOZG*67Cc_Soj4rUddPr`ao5 z>vZ`vJ{>XA;TKx{Xly5~Z%xJuzseU$e8FeBIS+oH>Kt_Lu<0xqMZ1leqaPG?ZSMav ziwU+!L#-FTG1`BPHcR_M^;{5Q1t$|GqCP|5MuX^y7{7#e`_`VuaTF52rS z0@GBdJ(Ki#+v`h0lh18ayyb=Mp;or6=nr!7o(C(`91FvT!~x^$D^v_xFuxO(tID_6 zbDEx-siR-nu|}|;FdQ9Yt;3VzQ0+J8%5AU_NK;|t=Pg8xk|#)|32SaFoX18W9*Av5BNUKupk z>YqK6p11Egkq#+34`@!Bqn{q&!A9aqpOrnmaBlhJbr3Bgf=7BCl|#tGy%ca2R+7=2 zHow39bJxbZ96t0u<<7XQe0$epcaD}6b5wpOVClRyp#8xswGf_J$}W8v_i z& zIDGL%Du$Elr?96wgN$d@#5Sh^oSM*HXd^af*gMh| z#HZG{3#cL3(QxVISTGfana8rXPmh_XF0j@ywIzZR&wmWY3o)DerJfpS&)1HYmqziy znMKO8O>;|xU_-iGJvNOY`!5c5pRk1*7vj(nN0q7KPV5}bj~<&Krb0_b>3oIzn0ut4 z*#0@o$4FrTrGCO{MB2gnR($Ro5q?J6&&+e4ig6yV=Pt=p;Rhn6Yg06Hw|%LcvHcE3 zA~|q=`VvcK@AV~89;@X48#vE~JtoBYxb577R5ERU5JbWUtLR;pv^kSqbe4WYVF%4} z5TOfRwf(G8&v>I6dpPaCZ}OD!v-tD|R6oh=7QW5rR+#%ttnwGEi-+g)HXP|7MF}C- zkGejjhhv8xYO7#I&sCv?oohJiCQjyQ1XPGw5w4Qh5zLo)OtJfPW9T^fozcEc-M@~&G}*UNfZU1yu~L(PY}D}FWfgA_x*U6 z_SK_?NGOvVT2-W*tG98){ia#6d4O=NC_wUtT+UU}_Gbb)eKK9WMZqPXS>e~>h9uwfdE zOFsI1-Fv@MO)y_6{B&iqKr!PPxk&t?PVu$gqVFNdK?ug7%{96@#1)hBIg7ueYw)4z ze`LDjOWLM~Qw>zn=xJd_rk4%aj>~gblOcWl{rI?LNhz z+e;z#ExmX2AJ#MFSefnH_!Uy`Wtr9!)VrD4eo3CS+l@v z#9Dn(B#az#UCTP3E|tU8&l=hj`GYJa(!XCjEqFn=wSMA#@_a|q z>`wFBl~aRZcB#8k*q!L4MgE`pGol0eWaI5d>U}&fcmNjoUIfIAx72!-+qu%8DDe13 zVFgbrHA210CeU&5P4~Js!w+TBT$_|;Auq6^XvMA|-WtwO)T*;St5O4^3G@B8Xt7vf z@p@!qKr)zgwb$*lbRgH)`Yod6`f5h4$!V*(YW)^o$eYepC*xCShmUrCdmS=XAd`Up z+23%U;Mx|Dm>VehG0QpaJ3t{;frpGtp~a4uh-rw3tD5Tpr$eKLxLh84{yUbipKmx3 zzl-feV}sLJeNe|Kfv!i40$V}2{*Y88iOj==_e0Nx3$Gi)tEuNZ;pA8xQ;s^yQK2Y0 zxSH*B=@?A7uW3;wzY;)rz732UIw2(+5`5nKg~;1^fO#EOJ#AAb^qTrrrd3Rt>0T|^ zS_JvuX(M$b3oq`@o}&2)x(pblT7WbeO9zfEQH86oTujn9#46gvV7MTvGh51Fs-qHO zw1;X%#VqKXnR+rJ2_dEDHj0|YucNnezrJnmDq?Y?tf(d>{P4zGBZ~WdZoL2tM(sQF zZX4cVauNLt-eXHMq73DiX{I}HH?X}S6!6vjLpGRQ{QxRKRy4lZ>)(iAzq4mSJ9|3e zV78!rlOyl=i+E@0#62Zv9lLwv>Rp$&5;n6|brRRp-z;7;!YV=TPA9Y@fo|>U9);E7 z|8^UCMd8M-v7e#d=zjSg5O_-!#V-nY-296m!Q1GG7Pd{Vl7DE>0`mQPTBg8NScz@6 z-MhIy`!!1m*NJr;*jmH{(a);{a0aw z`Iz9MKK^J9$2gszbC#L2HV9Dszip~A924XQoceA?==62e%zGDHg=F*4*EN%V*eCNQ z3>t=ZFgk!`q2l*TN|SzGXC$yiZ?7nRMPgsW6I*VP_(9Fq(#cy zmenyj8m@e2ri`8PY|N=b@d#&R`go2R;%aB6zZv`YJ!l4rPatPpMz2M@EapBe$r4R| zDP?BS5mS*o&B6{mx9=Wf!xj>g$QVI75tE{DDnPEX&h{!GePxv@Dd^8#rYXGs{Fls= zyx#+ig+Ns7R6hTtK=IEW7$ zcCy2g%8`HUE!2V$2zpWVJ}*>RTTu!D_OcOvm}c6C?X&OtQ};{DhQoH$tMt|d zJq%%4--RcrS$(l|;}pRpYg$L&w0`*QBa9~nv;Ly#-D|p7I$iO{WDgBIYRg9Ne2P?F zj0TaHeWkoS^~}>@YD^9h!g8KQ^TJ%UWSv`E`kjR;v_U@FUi7P^Zz>ot@EcSH7zzW% z-GypMvb~`Sz5X9o_`j0Kgb9xBPny`35t$=V{ESVAV#e4f8+I7FS|=8^b;Ud{=l5BrM@d8QfGq=jdR z1f%2$yt-{`is4#k=)cMZNB)Lg75hI+s)n*Bzo@B9e|?8cmlHws;v=XGW94DsOYn@M zh~qH%uOuSU6;iMW31x%^7ose60X;YFcKm`@&+Dm6l1}u{RY2;$JX*xNQoskUL-p%w zNAL+(3(!{zJc5mZvB&~6Vm8{DYdn^4V(!SfTC08+KU~pwSbl}5XzdE-{+ooo{OTD3 z0Iaf?igFUkTPcAJr}S`tXVw{l-OOP=U;=U{%I*0329`7U2)Yvb&=+*QLRg^?ZhgGB zZ$oa1OZ`&c5x5hNg|%H}7R>#ySpQ z0U`yo6)Geux*OC%!s@(CEHS@bjz6*3QIfVb!)^MT)iuY->?I&NA&iS$l{>2IP%a*A z(CDtg@pc|LFOFq&ex=wusJ6zx)tt3 z@5h-YWY50nyiNl^45m3uU%db9eTX|YtI(USoBHSOe5C?kT#+0>YPcL884k9W8XZpz zDuA4X!~Ypp&Vw1W8QS>&6Ik$Rh}580Ow+{ z1_jFnB9eR%Fb|*Wu32COz;6Q>F7sUh)2EGihnVm9=ck z$L+TS@q02~oU3!mKkRo1&Iwz4Uo5)KH8&+8yIFb)X4=8>)~<`lWp~&UXnnJ@3Aj^w z!nLqANWAl1ek)D}cgjY55W`ZTo<1E-Rk_-?S%bW2KKtznoYbt8Cd>0Vxm3ASK7BKb z1%(5X+6%6}J_Sl7A%@42)h*R);|yma5pfkisFIII{>#x6VjkWB%K3JiR6gXM6{X~Y zBu9n?K?9Ufby&8+cHkpb^Faf(`EvwJ>aBdl<+1SJDL9SDeD`i?hi|{t!2c^FzecIl zl3tU}=aRvGVhm_k_VfFcLe}Wah0kD8)jTDIi0L-1zLN}~u7BM9vU?4I!O{y1-jDxw zUr`OOAG=P5zGpy=s`3hH-wS5zPX6HJd_jPBgUzfhZCYqk6v`nYalqYdQTa!yn(WQQ zV*7la{{Gea?r`DE=Mk5gGH-p4+kInvJ9HR0{p$-ch1mRwAB{hr>~5ztY{gsm_F0Jz z4!-)!08Eeo0C@3BhaxJFEcBHXg$i?qyJh!{U0>09#U#v7g#30L;2i?a-CMM0a^}g|mbJst&)v=DmM?;#I=c2V{x`(1w|E z#$^0X>35rj#Gh{|oO%S4mKG@Tm4N^=Sy{c&o8H_Ea6I*u$Maz8jU zPl+#3dZ9wIq<+(>hZ*(H_;51?_<%3kOy1w&g#=)A@*=Tq5wQ$Bfa7r(UzCR$I+u<$ z_yDs4@+0Af((R2~-k%uu1t|a>nO~!+#UAx_aNgRO7V^OO(c$M4jY$dPn_NnehjF!W zc*<*1y@m6`$My3rcB+zjE=?v&6ySpeJIh}CNThmui_vZp;ITxRFpQS$zJh*?m_c`` z4ZY=J+<&KJ2lwW$xqKt_IG2%Cs<1a60RDO?#%_R)CzWZl+K7q-Y)0c6Q73Y7sbYJJcX%O)4Q@~9aMQnARt7l;yE z@smgJOu_6}GXTp8$3q~fFFGq-zI+O@u%l|#(DCzT4x(BYBPgL6a`CTegl^MV)5HkI z(EW3PN;v2nj<*SplHdQTr~W~%{x57J+7;aW5B}~szdxtrf07aZ``;RYkFbsxq;rIS z*4BU1irAxkuK#l#BW##Qz)us9lS6i=?g^yB9TWXOK3dH7_=xL$<6l8tM)(?OMVyAD zKJA(1a&ww9P@|I;@Bd@6|Fd=exu}0q+8(wcj)RhaV)@)hJVw(tKX_#0YNbEJ@b63b z0*=o0e@H@q?Zv;JB$MQE$6$<9fBgR+jY+~sNZrYv+4A43;P0ylV?RR#PZkf$S(^Q4 zSNzw|KXQ+dvv&F2=RW)&L-W6P{lgcG{3ewAm;auUKNnL)wYK${^zNB;b6Wy?;F&F`2Uov&f{fF1wF-^;9aGm<{M>yfr}DO?YA-iHQ+$bjV54?BIgQRTgd`@rq=aIlxLTN z;wblWSy>Iwo6XiE!5iZ<+~ZXz;`y4C$eMCblR3DW?!A2olMd*{IDGM(D|?iHWcZH%m}>_F4o;N7Vj z>w8C34U$$)t$)lt-B9jo>3wB-vSl^Vxg(1! z`!TH0R4bICB*fm{fu6fyjhW6|@xOK;U<{|TCun%QedMRT5rZp-Z0m(2RIKxJ9TSzL`$=-(*tVRBH&b^k#;xtzba}VhGscnsZs&VAs@Vtr zF)o9cSROf@o`WUW(UZ+3SZ6PT?Tn(j%6HX+Sc3Br@kF03{c+nZ_Sdc)*ChlBBP-NF zE$*&S&CpzF`&q-*l8LGOYu9B{n(CFv@rY&*O_JiAfCLto@aG5axGzj6lIE?ZEQQP$ z3Zc)MZgn)PYri$3TL}DP!}%_T``kCyrP2wfmMtOe0VWUt?noXqo?tv`npW#~AUu*+ zJj0D;fD-x6bj$s}ApMS@!GT)8uiw*TE-Rva5MgoZwL*ELzN&1qTx^K{b!S@Jf(A(qZdS4fQ7|+GmpqxQ&px?_sOeR-z1Cr*=HA;$dNA6IZJCMbRaa?XAjS z>vy4IQDxL-lU&jyh@W1QEQwv4{E*^+Zf#H}T#vVB3lty1NJ(3KC0W^IS`$OBdOYDQ zm}c1#D3P*EXK~F(H&GN$G8*}Jn}uHMuPQs#$I(XeOgA2zP9c_!`3%WZ+SBK%3h<@4 zMLy^Jbk)t1JLDs*TpG5{%NTJ*eFl!NgEq@c#Zj(rma1&kLjv?4M14}}|mQu`HrWnn`AATJ~W zwm+|*!~EBG-^cWWNfVT(xvdwZ0YimZi$^YEZDk?BHj^ilGJu;TcClh0jTo(tk8vn+0BdyeURMBG%Js@ z{FM-|=^f1O=mb99&G)l@?wvzMvXd^NOZwDj*LyS>XhavMzc}~mdQ`@>`MKgof;P*> zFHhXZ#Kq7eelJFzU+F!0mrhH#$yt#2&q03sZyKH29roqz$?)v1F%S)cLw#zw%mbir zg@7zy4IDBRK-WPI=mm!F=c5E-`_7|W(!>2l%cH`4In8T8+4_QxRo|_K)E%ebBZ0+~ z|Hu;3thc9ml$bsRN-Cp|dh~IV3-<`61C@V7bYkOXBe2z8 zKwK6=<@w-HOc3sUwCPXKn5}*Ptp#8d%{)ad%E9Jqa2zO%qtS_E?Ky;Y>-+Q@jpBl$ zyT_Aly5~_eKDn5tlHCk;tWo1jGjMbp@c#4O0BP{c=)~p6W>*g`USeFZnb> zKqO9{w*c8`pW|af$phY*)l}@362&`*VUKG)kFpJ~KrZGidU$O1-cd^y;uTM!PlD>+ zbWYYp9@>S9@_eV6?g=FYT%M_;NQNSumc5+3?tRb9$%>^p{&{FuV*9Q<(cwB*yvao} z!h?Zu>*c{zA|?pZrMplrI?r1i28cNodNpg5%qE|%w_%#zZ zhBuL~XMG~)`B{K~>#Z<9yZBLKhBXN1G_(fk$9W}J#?djN8yF@~4V^TtEb z>&H?(#zP>1PA1?0Rv6<+=RL}PoiBb^4sc>uGfmj(77au{BmtZE|X#m{>O)2Z8kk4FjHM-OjwBFRkVhl5T?RPh-}l=SL{qsB4Fc z*vP6Q1ZUiEg3r`?FdKbhJ&$hLob9%T2VmFt2ZP;(>^yVID6{T$l)vl4z#y`xn#n#y zCP3{v!^_t@VbSnvF$ofG)J&g>H}EXyPv1W)$Z$0#`lrWX^a4gEC=cV}iToHlPP>6%5nq`297e`fB^3XTi-EfND4J4peH-IeLXs^%%fktcKTRFY-$)!r*^6)ZC2cf# z!5+!iF1lQX5)6bI)Scz%vm%-cMBnp5qU>dBiNYw+|7n?!6QQb$@*VrLeFO{jz}GpFz*qASo4Gy%0zZ{iJ5pq>)VrOmmOcLKR^SI_HX^x4 zbR^GO$YNLkg}E>ag%9jU0wZ^<9xR+n+Ez^XU#~O`-XOmU-V~War=(+x+y5l^RA0s+ z{1+As11Y&E+kTGzczWcrm}I%4!@Wr3Y-*1L&+Hjd3Qs4_#Nv1ICm+7g6r0?O18EN; zWB(nSI$dFR>-&#*J<3&mbEpZ4SlT)k|8&|5y&gTXyDd3)?Jgv}BeV>zLlXy}_>beU z`bHJ>Kg*b@Q|gH%^=uJ^D4?LC&Mqqf;_8(ni=bJr1+JDqK8wzPR7HpD3#J5HYWH9a z(n1k`btx)F)soCKz7Op6mFANS)|kx^kgx{R98jj9M@ROO2S)l~w9>eL=3{|I)%yjm zYx2FEFKGBIAdVy`)vAlUL(nx= zFhr&*dq|ItZ27n)z=(8xtlDvN$fxXRUQ3&T!QI>?Rjz zy5cq)93~Gg%bC`@{=UA|T1DQU)YkyoPe{Qe8V4*o8Gj_`30HYQ!!u{S)cEq9>t=jo z1loMxitf+rUu6ZT&%Z&$JFI)#V@|q-rF)-*j>Z;N_aybQ?de5rm^vq%A(2;%9HEQ& z$$nmi52I$b3y1$ikEKjE!>zkCYBkHCLZtsKn-D@pUczV=>ThMM*<%BBvh^97e)|?t zLux1Vi7XdWPC+_j?nCp-RFwDlZpv4jHk(f-UWCVLeD}F_BJ8$b-x|z;Cbi!Qd}|k1 zkleJDXHp`R{d;>nM&GKSNU`gmhgdO&Z|{$a-MNVbTwn9RRzOmbUG{zuXxkoqB-qIA z-$at~?Et%?myMt*a##n8_ya60+Dm4?nvq8Vy$#ezTSVVC%F6AX@<89v6gUmkr-Vg^`BGhQDaHJ^pDK!br)MSb`62yUbRg4%G<;MIw; zdEQWcw{_Ou(pJumBuHxNECidsnYz6NGTa!T7Xu(|3nbFA>7@kqALGP#!^lD2~Hu>M0PH=G}-RamdyrEs&cxHzhdrtYM-Q6?Vq@E z+Rp40wi9LuiZrz=Tr%h=OjqxiVdQcX1GQ0M?u+69|JCU+A)DtbUg3mJ&I_gHuSq9S z{~0ZmhzKhZ_NNq$L>e)|=z1K%KcRyNr*K)N#8|IMUoSPeS6QnM`E0b+vRW$o;|;O( zkca`mt}&)G?j?tx#OOdW2XbVR2)=K%#y4AvC;ZlQ`mX*@njd8=4-;6_*IP}7!&sW7 zABnxOe1#5x`a6;@pSrLM5aRtgp69c{rAg&hXa!%h$gi%h_E2|#TfADC^g{uGDP9kG zvf0y}Hq#b{ID{Zt+?GF09{O%Awr=33w%5Q<9ER9$6S)z2s>$T@Pgr z(G<7_W+>y0!Y{gr=om~?+{LJLmMD}(-J7Y!sPtPtD3RS)Ud!o@Zu?lze;v(PuXBJR zFa#WzPbXgZ6>erf-9HS{vr5%lv z{l)N?rUbAsO0VH2L<_T#%$l7JbfmI9wUkLSrizcD@|tsx*aP8)WxsHaZbx{cwhGBK{H-izni2dM@W@|*t;A!4E|yWw8aSyeA_D#E zDBeT_rpG@qVUJZn#u6LHfgwp_=9Nx;&}zz#&$DQLRrT3BnG(gGjHl$u7Dq>L22#gk zDG#{F6hEw9H6bJspy$6fw9f582^wkj6D6AePP$)Z`viU4({L%&fTM|QfKYOfXvWI0 zO%R>@xmw^vl?2rRWrpAkS1Rwew1%`cU~XmbC!k6W5z1JdnNh+~5B{y}1Nn|UBK2)A zoqYctKFDdmMJjIv>cO(DQ}mbZc4_>EzX+M!L7lnt*^BZp8{ALfJW<403Z2!mU{Vj-I6%9JzEbiM=!fL$PEx(})Pl6}}gVy*FvAEB>pzY32tVKO)-tmYrnn8eyFJ0!wUs86 zPd#Hu+}Om-BMUmX^BNwi_HHkL72}F%8qg$j%8)a>sYWrK%Kb+wO4vyRvPz}vL>O?c zg566E15D?Zl@_p~-a$L zA3T1ZOZAXP2)k}C(;a3{=Gn%J-8~$mIAxf7z~ep791cUHh)We~l9Xpt6<>v>yYmEi z0>|0tb<_D0L(}}1S(aNIDupU2itrW$xJrtKF#%8dw+6n6dm;vkmDK-z#2X2~(DPp# z-#xAN|0pbp-Fi-YiP64hc7J>22#Al=RtJwXiS-2uEF&x=0(GRoz&Cxk6YyE~psHQ* zK*kf+oxr8eLJ^>GXe)x^UG9m7k+W*at0S~t1g!yg%sA_J4j5M1j{+CAdv}kjtXcz+ zg4d?jnD_pVf}PhuuUFAF9G!0byUh)7> zjJU18lc7M`XeUHb%C1 zoY-jai}2rlv%7v90<-rTLin}s61|~9!bY$HO`+I^e1t+M4NA`2rxx#PP=c2$c{QSjovTf%yxS3U|(ecqNv@?qpSQ#XYxdfuM^|p zA`>6cao@+AR(tL0c8_O_f&e6Z3y zpMP(roccV&9)rDOM)(v;!v3X8DnqqdH!VwCf9Ev#=K^Yt@EiYyE0>ST0B5K942VJ_%tSNfv? zNSg5qvXf~IKjy=Jiy3o2(g()?VIaxqM}Pkp&NF~i+l=jb)YUmh=d|CDhDo<>P2f<6 z_vL3`V#)=2#ZQ<*ZE^HC*dg+tTDi^Rx>7AF0`ytfJL1rNGkTSH<-ARIb*9;HQ|S<@ z=tJ&EMjCH9uVx<@86Z$$8DbK^WBBuKqGoK~Hf7?~Ki#7L6%?96Fo%HwRkimm0_FA^ zO>QslRx!QxUS$ElNUiVC){Q^`ix22v*YB50ir~YG&6pXs9qSQA7CrXUrDcfZ`_0o{ zWE_vZBJK`&O{c&JR*e7jg+(0O0iH*3cP+1_Cm%pJAj9D>g|cp~oPYlo&!JTQr6f+a zqjp$-6T>3Lh0%px?6a)x8O@iBuZ{D;KfWI9Do%N;2`=8}GU|?=f13Qz&&cZ~wVd*> z@JdCkIxFnvLyq%cs?uSYd&e<6=lyLuXUr@lTca*rtf>CO2O=5U`kf_;2?tAxS7>Q9 zaR?uN7$aUC6}QSX{UzH20!iOJDdCK{O7F<(U=2U05FGW=)4kc)G)i62(bjQF!{kVn z-E#FKfi2O~T8Gusug3^jj7SjAmD86`^_qOGR1epwnmtx-i6YBa36P-Ags(}wZ;#rF zKkEQSdK)Fu6wRRooVQ>+o~_%==a&xObRn+48-6{vri<~i#Os*)Cg^!RQjl@)eLe)g z^n?CxUGXJ-vg~W5ahY&~&q(V;3gPrY{`Ax9py9W6yEip1J7;|Y+nXS?z>Nfvdb~|# zxuA~1tbDW=IlTj+&lNy>iv!n-xf=lxKAbJOXEY68F=y<9K9|3jYtENSmtCi-@OSBr zT%3X4Hcfv*B2W3GO1E+X#%fQ9_>_3(g=!!bBaW$t9WtfsYXocgMm&MTrKR;^3WnTInm7;ynb%jV-L&!uQbXfF{{4DJ z27?>$(S&K{6AomgK#}|kur(<_aMW=uwE zz4u(nKorIJEdK$J!Pe8;E6%v)EvcTNPO*dJb)Dn>+*?ngRWhv-2$4wZH^AXM^s+HKj1 zYyGVNJ6uF&JsnU}tdD_GazA~+XPlZ|znvPOao?Rc_zQMj9Z{=W8Ich|0*-ijp2xRG zH&gfU2$g|>b5=BFc1ddYBysyy)c!lHFAIz~^X_i~R#;fRLR(h`JX~yEG#NjjEAn-V zpi$W=R5AM*7!Pxh$H@@W_IFVA5FrGDWJ>QAXRRA;*dWT34JC((95OUDB~&$4 zpS#BEusIkBu=PIcF^J?wv&^~B?|0*pTY9<_3xyp0?-%Y9XcW;dP9n=!GFX>=ieB0G z9xK|{;LKK*>gAEV1B#kxmO{0P!q3`|&TP>lVrwnS@dbWOtu}6VLQe^DFFMA41})}} zXH7DSAn1{8HI+k1t;WV$ zGV>T|xt^(5LYt-M3CQo8(XwP47z_O_&gP){P!va%hePOPV|}+~*|AOJCpd{akALZaw}eJaOjY!heBqIU$k`?4uhw z;!=kuLi=!U4ArDEQkY%{^Rt}dV%e2pqyiTao_YZDcj`~V3n<4!o02PQJpR}Sl-kTI z0&ZtMI)PkDQ7ljvTYC9*sp)*~6$2yV*+xaCOX?~R8H)j07ow#JhiGfKwKF0iem0aL zYO?``Q39$fdqk9)AOSYH>PP{f)v%@2`+>Q#qMyrJkaVb8jkalYomsd`NIT;q)R$y{jAmnf84zof1!z|6prNjj z%-WjXq`%esBA6dRn=$B*?(_A1#X*NQnD7tDLf&3UlWz673l_7kCN;m?IS~FB3NQt|bYK-<<@Q&~_`u12 zYQ<;PGXGw8Df3~rU7g#COQF639hoKQ7EAqhx}|t?j_rm#*H7BZoc^nF=?*Mg;XsIp zRYXR4!u#KCpOhphKZ#D? zqS14w@Gz?ZfR#RE(IlJ0%|I3C)n(LdtgouLmF=(Dy~3&fS}hP?IbkB|==Y@t^6_GJ5hz+>Fkp!P=M|9%1)tT; zSGz}uTqw5js98wOPOV3JX{{~Npb1T&AVgR9*qCkwP@i` zFh*~(VSC<#ao&jXd6%T++dd`UGnOF!yG#%L_j$?<(Z?!rzUhxuPL_CokB1?xut*_W z)bkrj80u&h812SwT$85u)~TCffCfHs?-9fo95wn63) z^um5mBi|wY-}nB^80M7X&5xX(+P@EqzeW}xT+|pkLgUZO|7T@_Uozvs`Ubrw4CldS z{6GE$f(mw1rB650f8Xr?OuW*M$6G$$so4L&-;ag|TU|rg5dXh#`+r6q+&pf8tn2BF z^Z)&RuOC=dOqpz`f6e$`gB-{1Td8c#Kd>M74!G!vB2HYaH?A}AV&-9BkN%|1G(gO)j4sC`9{K$cS{i> zBf{h*A%wcl(ysp{7;s;@P>7kXK@pHpdE74gz+_EUqs>-G2b&X`WVruk}Xis?58 zR9ZEb6u80merbV!&0)^t!hDU%!Y2h5IlAwe(6_M51ZGX?VAcIfdOz0`9$$r2)uN{p zB_(;BEc5I>2(>Z#76#e-MqERS&EKwrBN-5CW*cg%pIKn5EnWPYLJ1gnv1}8qs?{L0 zj~lqgdIbH~0Q&?(!Bb$OE&XdbBh6U7oUgTE;d9>irQ0$lIFx=Z)Vhha&?&BsS>%N9 zBZRs4?#fEzXDD<ngEU zG})o*vaIOB zPk*WIJ#ZQM*Vy}jm1p-a6XFxqx)o9h`yk>(%zb5WG}V%Hfz0m}*s1IHHmYQ0O)-N( zaQ&EHeyWjv?pN8eF8Pg3hq;IDl8tay#9jc-P=4lasq9SLygud4C=-Y3Q`939A<=T)zu}ZY&y&E)_

    fyY_83Iix1_3VgLLH_Lg^aAk#&I;0cR=MZJSz*V1@jAEI04V7n3PC2TOyZ@~l zC+WNNS<2D`sZ~T#|77&S>0XOIo<3rbf)U2gXm!(|bi-|5ncwW(!<1KhE~~{9?CG|R zq?~m7wl;IseVsPSzcQtV7Idb{5^bhS!{Rh(t>)^m@5LzVLNti}tUuh3P!FVu6lO>? zKzt>GA4H(Xpdwv_WkldxTWD1~D@m}&chFe%*?G>49l(HQf%j?UD5K5D$Y{LzBc9!$ z6NymJ!=>8_H1-f1tPPB?AAB+03&dR$?e68+7r=kp12UmZfkN8G$*|BcwS`QsCqTLw zZ}i4g8g!vtooiJg(%=9nQIY48Ro&tS$iK!NAKMeZ#hC`~wRTySKKu)PwP?sy*`JW4xERFxy(|dpz~SfjDEGZyl59_jFuS@-`|o z4d&)5@$%?LP#3D=hubF?!FL~eRL_Wom`;|4QP-{C2^p>@Mt;S%cVv9`=Fk_iAt)BZ z%eBOIoLA>S!%nrnl}p<1Bw6AMf&0Z3cTh@|`O{2D^XOi#e*3GI@#)5FGd~`?t1(U@e>}8x1!D2Qbow>K8|{0cF_*~OJLWLa~Bt@c2s;1^D^1unLf>) z>85<4OdGW|1`g;SlX*$@hr6DCQrl7Ma{e=#-*xFxIT--tfa8K~{g-WDpLb&Gw9L$( z>TDMf#6LeZ+UU?bj5YucX3`mqv0?zPL+`0^xHD6#O*PG7p>+ngDqAOI2Rz{Gs%++} zN>0ru-5K0w(%Q7@)!Xn>)`xYqG{idMc0tML@Ed`HQa+tq8KwbHJB9!#gxX`j^5ZC< z29gM&zso$xSV_SsyFzPWz2<};Cz$61&_#zpALm3z<)X*<;J21>@&@mc!G^uRuE|=M zZ{as5V{$IFE*AUTSi2Kx*o2SL+L<0bw^t{_7{!AYuqxd1Mpy_fQtlH_0`tT>N<>T+ZJk&Cg<_n z-L*<#F%C8`Yz=qVb)S@>9soT$I>0?P5Sa3W2W@i3IB<>)0ImeZ>m*#MEHmxlH2YS~ zH~vbF_z~ymzPM-AS@&{U$rxW=T{NHANgL_YEc5GjkN+{}! zW}UOKT1oA7Cio^(KD>C7Yt>({n(SHT+7g2B#Zqrm!`b7NzK**EK{tHqEXBB>WLf{` zj`yR@$F?j^3)ZW)==wghk7h@N>u=urQO&+cq<=x_yN%+%^=QX?g@BM6|DFNOzBGV8 zrE76Uq^2~6Mdf##$}JI@?!1)Ar}O2(yHAIoL{z&X^i;^re_zti8E!D_Oih%Q$(M)c zy+7An&7Av`_uN!Fl)*=f_l#QwL%#72=R7bYDYhinj3H_H<($k+X*hGGO)=*xo09As z(%e7OB4J;Dr7(90KlA<=QdZDl)qPuQH7%?6VTbY@4Ko1?h5vHin%V3A=0{r+m~AP( z3o6G6?v;VihWjOifGt{qGNg*j_ul3$QfD1Yh{6(tq2u;=wjfVgauU2!s8JYJ{$wY!ce8EE}w<9kU*2BI$L(;B>sDNSN zrD{bpm0bVf*t)y@+CG-D$mIpzEi)qnqRpL?KQ!qJ{h=uY>zBn$XQebRM8~FG)3zp+Q=3 zH<~g?xR2XbphWL*sd;uFEO)2)zg_=g-m%8~z3hlXiemD1I7{$iNx#kZwU=$Xc2Z0I z4-L(^pVxa_6yugEZ>J905JG)JMIaDA!$!e7E^BA-sY!{|X{^F`^&3dmM>K@yJl4;c zU*Jdf_sDC~TgLC}BNE;A-vlmqZ#n8!0#0r4npJsV%)K zR);q~fi2;AAX`Gf$t#p(p9f#J;XM>0-hMT0yex1#OYPY=;KD)~PAR-|8|SKYi9z~0jE?SqSmOyofgzfG2ipf^}?7sqV3)j8^&^SHF?TNs6OMNF|aUB_N~ zZNFPd_I!2G6#d^VX>>C`Z#JZi1aP{mA1jX|iG4gg`8R{s+P5OHkYce( zL$bZFiS4)iwcuffWeCb8xY{2FwQ9n@r+y(C+d66YtxR>@u3@8aZL30t@KgP})D#W{ zWqg83`wEu@$UrA+t<`w|oH{OVm;nLNldz%Xd`7$2!Y_#Z4s$NZ34p4dJEGPONE?L% zufbfed||rcRc~=giClEu$BI(`x9sI%6$ApW(>@a@eEXmMAr$ib%hMVgd38&SJG_DD z>zAq3w+oSARKy{$gAneOFagvXIyfn>ByOg%9`c}s&7jc>gCYE;e!g^Dfjr>4K@j8~ zKz<+x(-`U&k4ps&i0<%-T&W@8IHxKcknG!<)8ch=Hi5Cc;!%hRRYxF7atE+h>tIR~ zM_=}h?7)oJeLZ$}*b}6J>QUT@NXBqUN3PI@2jEVVfJdur>GB zf0SOLH3((+wP%L?1iu@$UOp*;nq+xhC<|$$q5`)vqjP&`KXlRWPx-g&Plu0TVN1;- zm#;rDC038}z7CQL(L}CXk;v%qN7b^Cz^MMtW$l^Qqx!5H`!f4>X=wra{UR|fRA;>U zcxckE&G-2@3B>M)ZL^JxSm-r7XOSY9g=gK&P=#u)>OE4{GydGKSMDm{wE!o%U`>U8zZJB!e!O!zd>G za%i2}q0Z(P!idPm5#Ha?CPsJ@)IXF4!98qIx>B=1-D0Y$Kw9bTPvdmnUy#mfhfQ;kka&a=m!H@6(jU-HStVa)^hphT23#N$ub83x1 z5JKchNfwLPU=g|lG}9O)9xm$JzQkOBxnWVV>8({cYDo++byId*1sxRgQLyTCo=p7P ze&NP9v%xMv_O%}n4N8*)P_;IPQ^?6BBb4@77PiFI;a}nUoDpH`qE5Yfj|yQR5O@u( z40nncHP$l2g&GN(ryKW|%bx&+VE4{n0&SD7*ANWFtsGzwp8#oBfN1QZXVh$<3D<$z zNx2&gA9M{1iMG1FKJXhBE$xSoMB1ppr4CjXiPbMJF=&RW#S9_Ap6)~oxp*>v1P0E} z2;vM}KflKq{zR=tcF5{ZG^^55?%VXui!Zv9uB;3%1)b}?2v~J!{`#>a!HO2UNIS{y421fh&`)CBw!BOtSx~>$6Y@(NZCaEHdU#jtbTIcXssuP%;vq5@gly&So7Lwn_0n6Kb#_zvSmG6G4@HlIW;~?GPCB zpMKtr#yupt2y()&a#@192C=IzfaOI2(s@q%6t?_hxQnLtwC}R7PM+M{3Unx ziy7GgsGHEKDH6H^fUsjV9ZU!;yrHD0#&OW_?D|qeKO;=x@M-btZr;4|gJT0lz3Cji z)$vlFmCK<`ddtn!!qx>o<_ngAUXaXaE`%^Dq)iw*w{IL4)kxDy}jyqWlO?OmKiX#&uK@GFhn4(%VCvA8nWK4BDy~b%u zK5hBYUgtcP|M<-2`f9}dsU(aY{A$v>+PBe;)GSl3&#oJ6aUt7qQX)?6V3#nTUnewW zx-ag*%<8ajaathSE0@Kfo>Ae-;P3i(IwuVE)GSVbzzD>>>mtYo(en8l(?{G#w?2Oy zHLTe|=#DN}c%dQ;M0*X5yWg&D81Ss#9WZ@!%K&h?OAM=>?+?`teI--{sa*(WZ+rr^ z2-p&{*wC@iIVcQH2576u6%`$;gh{lV?el9~ZP$@nDZ<~hfMKzg&wk@x8K=yQx(&5x znsPZ8q04vW#!?hf*zT~;5aA>?EdCIy=eE#@cp7Q#^RS($o*W{cUGEE=pG|gY(W$o6 zI)_4aV_pRhQV;@X5V{ou0z!sBgY##Q2s3=iILYLI%Mp&i`A`LijGyP!C)jtvW24?( z?DP0Q7t!Gew=hA{?MooeJWSthtewkcKu_jrEdUNj-os{xi)iws3L|k(4^Jb{!IdX4#j<8ORyYkAtfs#Y;D4;I7)X=0lL4b z(RubSSai_tv|HaKe&TJz-n%B=?YPX))(5ReL61XnOY%*IeIGABrJj_2PlxXx$Ei#Y zGDOjOsDtro`9toHIS;9ftAanZwTmK%?mh5`jS{YEU3^1EOQ|_>*jZr2P{}wyxzl~d z;n8!nm**-<`lW@?^#}XTT^Cc8eY9?#*g~nvTH|!p_5sd%6y&Fr+~3v!vJo^)$R=f` zJH5d9QT>mmtRFj=JIF#jC-cAGJcD8e_%NsL-S#EK3l|?ewtjYZWVC^xo%k z+-fmFHV{Mt-9f4hY$>3x@DLET5$n__5r9njcU$u(cid?%6mdpAl+cseEv_ZET)X%a z1bI%BRnig-5)u^PSv{JCF=<#Cd{gy&mJ?225Y>)^PUO3POo&4t?kWX;Hw@26@aJS2 z;aR`L3DEf|k1DRD9Me+Ga2n+at@;9kLuM$0iFq{V*X{<|9Mj>F+Q4c4&XU9C)sQ>+2LqNeY{FOv7S1K0*+ zyD0%AjG5Pil^d2UoGAADUh82WY=}PW-BeeS`AMenf3vD^`SrC!Ju~TWQ?v%#c4>aI zfhF_AfBxA7nU?3epcVuITTWc*vc6wi^9S4S*gF_1As*APJ-cFUi%gBj4hM|FkGX~3 z+4kG?sjHk6GiK4S<`c8@=|0FQacvX@sWCDs#MYEi@MeA#bo7U{^SCWZf4UC)E9gnU9%T_ zIxlzC>9>9_jxXcMG%xrf>mRGfBXV5)ftLG-Jl0@57tK*PfrB? z_-G)&R$pWx=I*-X@K+n0i(wPa-M7H!78rGk9fe9UB+!L3p-`(N+=gjtUu}g$2c>}n zN_s^PauSQkLU;4^K@#;L+oJ$Oslf#FG5p;tSRmbcK~eR)gOf|6O&Gr6;b!l2Gpne% zStrAzD-}+#pseLWD+A5|`WH$kSxG(Xb3l!#4l}9X^|vRXhrb^3oekyr=&WbW>BHOH zr^aTDhlzRU6Ws8O6!jDp+p5CKe>_oc`0P_``6c&+rBg=b?+QeB9gZvFEA?AW-Syq* z3)4QMB`eH+!$Sv`0OkOFU1{Z3M#=;0!IL*Nefr8_C-}yXT$(|>AtIv%jlQ%dxbsoB z?kSgLMOX_3MLBWqtqyBu2P#3XZ zpH`Druf-&9ygd68&W@u_fawklxM)_rI2_Hkk3ETz@UY`bQ$qu0mT!eQrke zpRD!u?Ps^0XRk%|lX|3d$70yL0w1ak$oSsJrW-9mwwWMmCyX=;7&}Hy5pODHbQJeZQ7%`47 zmbASYvKFl9cigfTJR;eGR>M9ohU=V*)DXpTRy@KcWp5Dqta-@RDSZewZYHJ^68&Vo z|JZmimSA3wJl_DD4U%)ouTfY^MNOP$9;b}VgGtLal+Pl1#!S12W2ek@>4@T|+ss;j zvN+Zi;QCx2lft9}iKPBxNmP8YD(CzX&iwZZYFul7c8_vE&Mu+cz1*I{6g3*m@f^GR zOO*LXiP+^zHqRDeR0NC_Xk6`y3WJX^kZ8}Mf+}_QBe|A}sD({cqOn>lR>mj{4E@!a zp3VEVwt~9uLk<=+{TYs^0+WVOgiTW^j0V$!et#9odIvFj-F1gQ!b3PKRv?LvMlo4= zB8~z^K(zzBqZ`WBceexJq;vx4UtgEM=v=7cG)Owj*Yz-dI?va}o`->Jo#<)hnCoc1 zgom(muN?DVR0y_KsxXp!`;H}apDnLmg1U56rD7zlfsi(jpp4y`$tnEJ(&{5uL`YqDCL8x^M4@Kx6ygh~C6I0o zb1~N}TqWd3Hi1$at$uJ&Q6HszY2Yf+w7U#KVSUtQ5uQ81FI*__9k9!PDt_$vI1TE{ z!q)63xS@+t<{$#Cx=SG)oVAO~s5BzD+qQ%0Jh7%41*h}6>1L40UTWb?e%|lrC~8Ta$?L#kwohRSSlw$go-giT zS!m+4XXk5;IIQkhJv8J9G^ZoNJMLSOn%r@^m7Dsup;^akw}Oq=RRob}^=6B{RdpEB zE8N7zMu!XoR$&ISt9+07AU6Hv-ydZB`%|jGx;hI5yN4J8SE1pDu^|e# z7Y>7x_zUS8kPf`w^^$DKNz97*ujGCarm_rBhii z3m4@xcvXe;D>VLUNsKnAmuhV&Y8_p!tT456NtLG|IlZe_|5;39SWfiDc zuXip;1wU6eOFy3{tJxt$dQR`mrTN2W)2DuQ8qTCdkdP_hcHPYBNAL7&T_15Qk3kge zYwLB7KQl4$r^25M!yUyiCnra=p7~cGHDS1Jzi;j&nn6C=`giV$Q0<*M2SBdhrNPA} z1^pEnE^WO@!&`z9m6ZhCsCXx$adqxr!H4AOuxk5;fGzt32&1&6+Js~9B+0wmHy3)H z%1MKMiWikb8ug2UANq>SU)mXvGiLaz1ZjNAGDU_PfrG|*0yn()_AQte&?Dk74Xa{l zCI(P)V=tfv?@xO~n~N|vDXC|Z#QhMg-k7!K-Eg1UZlRsh(;dQ}LOEr~7Dhv(_n!|l zud{8ivM(E=?|BIDgVl+w$bHBnQ~$}=>)5a7V9&3($EI_hFB;eIW5k=Li*c;#oUP+F z7>PjLoAfm~GQ6M%V{ml7EG?ivl6RFZwyzM-w~^%EP205K+g zabhvTPEO*(iQ{+J88iUFMf4$^ryF|R4|OW0!J=uq1aGl9r>M`=P*m38!^n1 zaW=G6&kKRGm=fJ4H2_t-!`z^A-X0ejH{(XBkE(N0=2SonILD4M7J9rp&DHL*nLDx$ zI?x7s4SWv&+Nkp^%_4JgUdL!DK$Q^Smi#XW_ib2?TQJYa3WGDq1Te0St}pKUOp-lnl92>?&JRa!u!sjf z*7EiHEn;Z*l%)!ZphR;Qy>wa|i2J-N`pnSl?|7kv4iwEof!X6QEKvPL-|(v&;c>X2a`nU>LP(q=S+0aRSe*rB!`iqdZK`- z2Uf381U{UFSVP?)5G$jqtzhDJ;@{C2(&Iu?YCwsQbnWPIuAH%^ zpTNG(4rgkVzBd=y_dlewF|2{@?r{E(7JwR`TQo*zJC1Q&@e}!B7dP7`;vOJ}3ZeGx zy-(AC*o!xh8FEyq))#->Z)>}-A&{tyI?C->+!2Y=HZH_6TY9U|{rwWvoJU>fA4Iu9VR``N*bEG_ZYIvSO~u)Mk&S~iPKgP zknR0CPR_a!9FLKXUng0x?2^Kn=Q!LmS**It>2Nsv%o$ok~+hBK8zU~`lH`%M! z>X0~`;TcXSIKJo`ytc=@pP|7mt4pBr2To>++~Co2`s_odVYCCq1>p7-k->ChhP*&E zt}ulY4BnTK2cQfUQm7uW;dWY0?u`9RkKbR^$sa0-P5hLZW8DLSo>GWhOG8?J7@mg{ zr0g#M4o9~sY}5JNM59Iw^%#~={RKU$O^4r60E8ce(R1;V+$dhzmtfU$?_to>%@x6OjZ%bh{fQ|xF4`RjJyz#Giv+9#_1-BSlu9@!ILI8F zSz|qNp*r`D%NGi3^vujZSVSA0H#sJ=8QoxDhM##ZRj8g!tLrDml1uG|T3M)*J`=~* zc>cv9*u3)P57_Q(`fs~1X?_**{a(J_qMo4_Ka#Ly4wLM$?|!E)Ctx(xZ}qMuYDHYQQ2>d z^^)}a@gz&tcq|F4!iT-3CMeUpbQ`n5^UX6H;>~-2Tkp3po@dkY(W4tC#%q*p4KO1( z=2kmN#v59XuX275l*ED16IypOs@$tDHyct}5e_f(~7&(4| z^tqd+;0}@&94+ZX^xxz4Norvc8ZR}g6i91i8*{(@6W~m>Q->Ww|6BhjkxNK~tyslO z`+y9vy-a?Ic328aE!H6l!yLjp00lyMB7BKwg%>>J>T>|d1|$S40ca5jHSVLPfKrJi zh}Q%RD-N=l^Un=o3ax+Px-!&}a+53(Uo89h=+1U&Q78*%k@kO<+wKz8gSk(?>r+EP}YxVd0Wy-@p#xg-}{x=%IGB z!!{d8Jo{Dj7HS^hbcc z8tO7?NeFgAK|&@6y>{Wq;(871Li1{yG26kAnS)T_2$z@nMI=mY#8J4X9vh>O=8Zrx z^*|nSbU(n(BbX%bOy$0bFe}FARgipmA9J>w{w-@r$a}2RXj_SgUtTQp_m)E3#w~Kf zzzc;qLt|BWR-aeGENEC?b7Xs?8)Uo}T@OE~y|$CM6Z&(2LI+yqZ_p4zKK4VP!7|_V z5NpR8ghC!ebxM9+8#bJd9l%Z_a{Z~iIg(ZwRch!5J%qAX46DY93R}@$-xuwXhBQQ} zp(+gT8w6JepMC#j9v(5>8S1VN8b;&FgaGh}3r6z|kDqlQCCPCbG!fjaO)pa03 z`lCj@{y2?zV_8~ngO~3tK}>SB0weSW6^3;G^W%FIioF+ChD}79w7tac9T?au15o0k zA(=>AHm z))OIGrC9oWOzMP_tdwM^$RvHX?xb6Pe#^C4yz`h7Wo*$gRLxHhSxNquRrbv|j1_XV zPWpE9F)>QJB<_&Qo>8HCQ#l)(_GsZ5TD9#Swts`QgpP+q>Z@$kEj`EI@XdDd9KB>N z@&{U)<_ye3gLQFrM7%P$+k$1?wa$A!pT3tqlzM8<&^|jZkmEgElqzc+YPM@ywrSC0 zthL@di^ek5c8SUGT=;?hV-)w#1ovN%I+G)$N7TMm-f1Yl0J3NCIWOS>`&WDN4GaU4 z;|>?Ks+Zv!21^ie`ddh{AoU{Y#MZZ~q$nM%G-x4c1o&qq=xCpz2d61%3=W;@}Dh*ebT zLLeAD;GbbmgJCMtzi84ukfG&cJ@_RuDd-S70mcD$<~C?JGX-{GSaZtNgX(IVRVLccE)ZmuLl z+r+p@h*UC`Ahp-r#)c+IupE+@Lk&10RM$3tMq-fo=gm`!0J0Rq2WIbw~st34^uIC8Rj3J0x)G|)#9U~c`igI?wo2@_o+3QKg z1Q3`==WhUMI4|wdYnLy!Qd++xON}>`N8;PViTz`#$198>Vl>XzbWx)lsoZxB%eJaJ zcSh2{H~?Jrz)A1FmCMs*Np!(oPg3?gLG$E;eRWN`R-qQ&(ZFx`Y3DzWdGqHr@Clyo$X0xg7~odZF;;T;wxrh(pp6gdnMBClVusuk zhN|!(wch*veJDx;{5Q+(DFZ9p|A1h}2=9Q&(Ay8jz1aT)e@+8656V@{j!*)K2EbUYeO30p!!t|UCK^%^xalLkNskH&v zhaGd1!IJ0WAC+ddlVbte9rJm+150403FNU`DX7e7lqkp(o4x^jgUixljszkCW9mMshY2B$slLrlVQi*`NnGB{Erp5&!~M{jXe5Uv z6{Z=03M_wfDDtpCVI7EPo=1|v!fFP#IDohVoJeF%(b`-RXqQ#istVm$%?&LD<2V*9+B2(f4q^#2h&{@#n@;-IBtMfPmO?r-49 zA2|9LUrJ$-{AVBbTL z*qcBU_%FWfj~|jCqBH6#+$f(e_`>fAAiTS~%dhB`rU(T9G4bbnKenn7k`72AmO$-=$f%w#~RfO=-`@{cc* zVtgEY430DS3h#fuJk2X`cQCM{yMI3M-#>rO26PguZ8U=?3d=wIQ8IXny-4ZPf6_4i zDB#4N_!Q&h?KgkMtv@c`4(<*Mm%I3X5Oji{UTwCN@A?0TMSf3Ymsju9{)8(2ai7TY z;MIynL+nxiu{!^T`};vX@p%kZO8!q8!hdfMe0Z4b|92l=_!xcm+b(&^yAjRFYE8DC zx#~i8>lqc;<(oXu)>gNLpD~QLclRlddmH95#HQ17?it)yR4LaJVKu2(!LG^8mK z=p${n;+8Y!{v|S#zh4b^uFFs6#YJeHKg2t#jR~?EdVh6qRTfO;5gFYzZhbv$o@cIj z+Tyrs;O_XaMSHvG>|P#wt++%-?Vob}ve)%p76$VtsVyd%0lC2Dc+vT*hHEbZhEL19 zY&(~M^%1X5U)?s(Md#KZAbV$#+CO9l6ILR*Qx`|J6%K57uL@>yh`?sKH1O9`%(d|w z?F9V3^1v6*hs@+b`qwHL9dwT;JjF_1Q9T&e)RdA<5hPO z>A@i7X|JV*%#d8SY^SdRhr3=O#@Da^y5X&OrxnQ2O8=FLXV3GvvYtLeCIH<>@@wB% zIOX{whvrf(Vuo^JXo@OQ#%9SyWmpU4A#OOdYb|0r-SEPogA8}?NpQ-?xvddzLSshe zxF%=!(@S=)XGu(ln6<~3>@XyXXZ{YVH}=sEi0D3S9&vq(CztFfG64cHu*E4@Ppe5d z7PUy1o01AzHQEWg%=wdh?a(H-82vt!X;Jm7=OlMz?K!wOulgCWKAfWRIC7!E-h4P`|W%QtdNReJD(iV7X$|%mVYeXvKu+f9B zENtju_EDiJt7rkeBO2QXQZB_AJn_MjL%8gwhB`#@pDwM;JrnQCN*xP^HTO%{@I4nB zr$?J}nN(u=4jgNXK8L2{@ZddAF8Nb|C9d?d1mMm;U%@jt1#1tH7#mhjF1NdIjDD8( z>^Z`Fn5DY*)*W$^qLbu7#7KrV->fmZW1~ zWEbAHs`T^iI7dhnv>2eUy{H~byXWq0)OhP^d9rzE870Xmzx3RC?a)$`J(R2pEAYxA zHN>JzppfMIyM0YI@Ws$jH3aPC#G8v8Hz_JUlmSH0P^U`Q(_kQTX6ZER|N?&Q5#KW1p zc*{dIQCFd5{-zthH?4s7v<&aiP5_skcFAfGCh{-=;r?aW;}44yg;!{6#Dxui-@j{( zbY|WFpNlp45~;B=Xu(`Qic_NdiV2Y<-ODOMJ*#etB1PQ2ex*<40VDk0%m6rG)kwZ8RaUlD3GdOxxXeX zufNW0!+(*Q@RJpOZZ}EMmF_KnnH<(78>@&PI{PYVflv7G`BQS?p#d0hkP z1uZ;Q7fr=ep@A&cZUd*1gpkG8( z_T&V??r2g`?@*cuz1SX6)yyls5wo9SnJs&>v9~Rq6y7h!NbhNJ@$hw&LU|F%?rLr2 zzO|;p(9=4n58thMbDznK;i-JV7B$}NzDS65CHcDYafczfYq$&5z4~R}N+x#Sl(Spm z+?$8Y0!X^oOBYQa+X^jTT~H3lp{^c&d7-b|vQc@8{H}_i7|hLSu%TRHgONTh3CvU2 zzWqe5h<^IX2ZNJBzEAp7d)4LA_q6#B!cSYcUXNV%W@L-4JT`zS&Flf@i@;OdU;7lY?>+QD&&%4a5<(h2yO$PWv)I!V+Hw{CYHKUC~%951S9x^x0E5J%8 zE)&TIAu2@nm_L`~Ex&O`5;#RMQ`;MPH3N8y8>I^|i(9Ht_!|7jCljk0#}619Zf1ri zB{=4HK{pWH+6#Bd@T6BuY0}jiTYzrkUNTMaZU33+TAq3Z0_L7EuGwF|?Q{jhl|xCZ z+mF%LE^pmlPiNN>oDoPgx*jBuS*GlN@4*Tb)VFL^PUSwdyC20cE~S0L_U1i4zlfkK z3tiVa{pQey)JrBhIMt|6Di=}mo+k<{Bdn2ND~qj~{y(3dILI{Iwq14-(U& zyPNINs?AdgR@WK16dsIn&69~ZRmgiya^hjfHf1O>48hdP9j?jm?yqm3Z7c;%=}dU7 zqSD$g$c;AM;WwBag;q@~z=e8e;1+*#SAlzdOCxgLm;b|Mns-0e?(wVOzi4+RTTx zR^!>p0-KwG+DKgZbWILu(a!R5%VRh5&Bj&rYtI{>m-64FEt=F66ZNB-adiTw$w5Ra!TBt+5LpKK`sMYb>OTgy8roNN!DkkN!!CEN6)tBRa3vj z^2}@!<@k%$m?F8y$qxCwl0o9Ya1#8okKfQ@)(Av<>j&_?IVR2q!@XL;KVMdM7Oo~$ zcCwjAf{iL6`g!Cx+_xw>Yp(vAx3*fBEeukrU%T>tHX+lgwo+lXZBIPL`&ntQ==380 z#`WvMC(Ha1DaF@~lS5q;nVAHf588$Dg*QA5{}R;um9P z(D-fJZv=VhPCH#icnBCNmF#4Ayhw64{;BCb7dW?S_nrI@H~Z($pzELJSzNac!+;@H zjS0S{w+{H49tAs}`j@B9G_eFJVY2`IoIRrLSD`bMbNg^69bq@E!gumIEIz>=gYu=~ z*yInG^0)UNoS@XALtF;)Qh!ZS!b?5HS=ieAx0$t8%%BgTaI#B7{C07Z0Nw`)Q3;3Yd?mFIygU3SKzX z=i9b($*OF&n0*-z(||h{s;Q1v*+1}Z3u}MaUsletUng3Nk8c7;xA;;?9}9H`Q>!R= zphA{khnAV`1u6_6Qt;++KgM1ORlKBem*ZAxC&P$Y~M41Nl+9fi1+h(vCy9JHzh^SywTi){aX~0`H zIv}b^O)@$K@45zY=jy$_v#i({8pn&em$a;E{xSXS^oM8LsC2}zrhvKJx`$b_u_Ygh zJ9WzE^V~R3MK_H9)E+Ii%wKQW-GA7#`6%w4_OIxwH9s3ve4}u2y5On8(dlD76H$I~ z{{VBu-uCX|GODdm#4~Iy1yhP`7U8KJw;+M4MIo=){7?rGDhk4>uqZ>iIN z)N;g=Sw_VCLJPC*RhP(o;b=9}6995%9SXhlV~&^4%Lh)Po?Ejzr(*aH3+w!j2!$V0 zZ9TL+iOMqeQ;hRcbYlxl`AB+VCh$~UOnEgn=h}kA98-dm3-44xNv6tCbofxIt><~_ z{CM;fMj%P>@nubRAD!y`)8NI39~^=8-Ln1gVqJyPZZ}PnDd<@I5ia9ZEBRs6 zVi=7aKIeFJgJfX652$0C92US!}M=;Gn8127QcYLAonf-XpN4am+y{g-bOWwmG z6Jgd8cD^rSaYDDFv2NBtzp%+Tou_%h+*Oz}sfF-s*C$kE`GyYZ@Rd8p`=T21VIhO* z!P3d5#NLJcr_A`gE*tt#ebfn7q6F@kNJ*KaAMo9aRIsqZhFv9kXv{06nnNxeH_|^W z6DO*wD!91F3HMwKuGl9O3^kd_$|b`c?q-W+WC!f}jLB)m7%1YoOBL#MrX?(6ET#WD z)dW?(gku&bsPZ>Wx+ga?a_SN`0+W{|udL-BXXNxzS%<*^V6@6%!w&i#8)NgJSrZl_ zJaWb|(Y92qNT`dX#y5q|C>mCD5M&c}S+2vPFq2Alr6sUr?Y>dR`SfIn17As8G>I7k z3Mgjn#FjLp#TFgS=V^W@`umiBaO_QvUUhb@7+c!$c0Y z^z0ex6!nPWmuthPsmL%Z@B8^eWcO3?1%}qpg)VOx zPkS1vKJ;H9L;s%k!tt*Jf3_jdV!0QPjZez07T-Q8aVTWu%WHH(I+(hXwCrr|usNUauYb3Il#JKAp0U&{$ETX?j!R}gvzGk658tp8wfWjUi(&Jb zyE){@9a~ykp{B!-In{qvlK$Ib`um5^@Xy(qJ2@+5{?%XkzyEfp0Q2nVcbe?~+-Afi z2aUWQZaN!`fByY?ENEu&cbZQ8-&cP6AEk(=PTX2ih5SGNK1K>O`cB|KN&Jsa9Ltm5 zPeg7f=l^H~z671RL2?~z`G4vn3PbfsgH9bw6vBe|KmY#!fB63oe|Z}EUdhUTubas5 zb}%Y`f~dp;S*@Vmp zw0D5P@(E}dmTU{Ez`!M#1zj-_2zfV`=(p)HwVb`NoXC+#<&oGN&7cRi5^-RNHSqn5 zis@cyy#gOI#=i#~K*Eci?eW|D6XjP!{{id2Hu%*Ubn+z20Nw=OEln|fAT&eX&E;X1 z>Vv20JTOpR2UD}RmAk;+VYCQ8jf(~&FaYQnD%8ze(LbT;X@Sxqy$p*f;7>XNMF<%% zAKCzP?by)_J~)?pWquC;FWT6z9!r{^*J=P3yX#5XcCo-TL_SLB!EFv$(c~8tNO}Ws z0ZvId>Z`Km6Ipklin0M_Q6a@Rr`7=V5(A7_bbz*9`8PMgB1Hr6q>ca*Itbtq8^BT^ z2>|N8dHYiw9^HRDWEzfr0_6d7q0CC(^gK7oe{XW(UT6!_f6v80I@6y3f!&VE-U>tm z+e<1)+yM)4!{_lfODp#7tWXG9?jy`JAoae){Sotm&w2Z^zY^69R9%ay>2F0A@i?HV zr${Vz<&Om?PIcx!S7mwvsNgb!cSmCGKsX2HJW$(GLH)w%*FA4f1-8w5-c z@~)3BKJAq?F{Kjc=w+8SlfV)IZrBMRs`djNk++^tWrZMr8H&J(rq&Y>l;~!k zL}Df`PN-NQhu0b#8@r#QcN^Rp=DPrggF&#&C~ED*;jND7ii zde#Igh^G^Q0vi94>VeE}*z8{e9ZWbsAk?Gsq#vS7h$9vLrXEU7tO5bJtCVU{3ioSO z8cFO%z{olDV5!3c*kjsTrS19J;wSASw5v+mnnXDzkCS60Y#mgcarG<2lbh%ML4>{g zhdz%U9ndT`mj3TTSp3FP0h+fi6Bin`!h-KtAjUcXg{sAp5$kA}@k0*7!B4G3vvekD z{YQV$0{&?@=FhwLuYCvltF!3w(lr(_u=$#_o@4ws+qt_}z$N6q9D&}_j8U3ZqsQR& z8x`bPZYO*U9~>zsY3}=vb*Na~`=oYh!&9 zxzZvvOv6dnJ#2e?r~>+_l5_qq2(cC*jv;{#gI*c8CQ7RlsuEoK@xwZ`(8T4{GVl*@ zAy5S&`V8JqV}1piDuOxvRlubP9MRjc9VX9n1G!%Y1eYL`r}5%11L2?C4cJ8kM68p$ z1ACvT18eCQdKTmi6;;RvLIAtH@g69jZ|gj#Lu~!rPWX^udG@f`;a_ z0u~R&nOl7hI3nqCzqN zewHn@&n9~EEQ#LDh@t(5Als-D(He#JX);D)j*Htu)J2ZoTEyHnFyYw8g%qF4W_Yy> z4qzhk6CN}Aq@-%#+|!qyns02}_j2t=i-&8UhKBgURvSxa<|dnHurc=Lyqb0^S`#68 z@>O--{R0pp85hLu=wu-?H4r2(uxN{@Si@DONbMz5Jj9)7XQ4jg0jdu<#?W+(Ll?+8 zKq^)7zSX->zp~~zOVZ@_jSF`b-mxoLJ3Ez>fUh)a znpEDsmTtc@cR_ucD9KmEy~^xodEvqdOv>%n7^aldGxJ{u%Y@4OOy?=v6|2k(GeF93 zM5kFcVVPgUpNtRIQKK>kmf!r&KXMLpwr6*AtA6WZ#7Z=~P=TMo2>Kz1FPks>h+yjj zIgY{})$SW@$(ltRt7(6w-j04|!LuJKdenluVR^FwIK!b(rKfo%*`MY!NWXTO^jaT9@XQ24?P`@LK zt49^NISc~nA@FaHQWPPyyMiJnq}FIs`kuQWd&Ao?r26?7Y1}0V4EM;NpjRfeeAeTIr(afs*?rm|na&W15f9nkU` zCUo*q1wv2_2^|n3zp6uZqDxEC(JObNmK(8w3{$++^3u}t+S2|ckKoIKh3WDSa0=k( z^4QY)E6svka@YlmB0N($nB7|{Z9z$kM80+soz8YSle6FpQ2No^$x(Zw3#H1%B!OjJ zo9#82&Tq$;U|5s1x5qfTJL#vS81`eh14Oh)pG3d@eo&+XK5qq0je3FqHOGFM9?Q(1 zDchahCfb{-?pI)KmLhzXCSy0T8%S#0Xw@EXAD4OfAqXE_f4SN!m}Uj4{|K=Jkrm}( z{$3~vUVyvGlb{T-16ang@RgkT`jHiF=xVjAwRSZ>+JW>WH;s|Mc1Fs{=(hD|TpqsW*@ik^F{iT&lzMABBww%(v5*bx3Ke^#-&;S& z60jVu%sQn!^_mScbN*XwOm31&$J0}O%An|EFKumdhmMuXC+~>W&d^;YhwIpu$98Q2 z*e1$V8YA`{o4|2H*~{`?xUnyjo?v)8%ngaH;~S$XfF8Gh!`Uuby#d_WoX)egPGQnA z_lU2!<5d~}ZULbi&(?;`<4gXWojId?gu1LURhI3xF7vQ51EPxEj%!iCC_ zU~IQ|8zNmv#C0=qHVIx8fev1mcnwI2-5Zvg|2(Y=*R8=Zicp1-J-9K_aRx@#C4+fU zexFnpU97m6aAhjM*QZcy`+lg*#nd=LU!(E{4aWt z7Qsgup=KR(K@w&(9~!-&4;Se|O&Bx_3x#AF!NW%(VMiqSVB(+IV*_}_sgTKK?YGwt zySqd#5oE995*E%3u=oRx*>Sp7NAv+}U3 zpD~Vz;MY-yC5=;D2JG<DUv!)GVI7qQ~*3c)K&&*Su6O@XLN2^K0B#&kmW%_!-}8G`*IGgD0q z`=ywu&1kc73bQCoPv}Y5h)ArUDi6D^(8okc%1e&$wIYVhGw|5aH%vFHdzhQ!G~mTO zsUuW}az1HIA(hxmR7FW^Hn4d$7X&|;6B3yPvByA{_#->7ce%))QyZRM5{goT3L5}H z@CBb%?84y#xzzzaUZy4y1FXV^@J#-F6K27|T)!H4v# zi;}aN8o20_Sq8ySsP{LP>9VOOM-+zMzh8lR9KD3L-(7i7H*;)v3^0Gp16daJXI zWp^Q1pQ)tcz=+rwCHHc+xGP|6ay>JuP-RKiHk-GQG^)V|(kXadzSR5+nNVK<_RGoG zWT|wAT2`@8P4A=Kqb%6N2c?bzC{4ofNpW2Q*|F+~BU4`IMzz&4o9@AM8T>a!D___8 z+FxV7EMbfd!Uvo3ST+u!hvez&>{I)zjW++uSg$f@_bsW{B(ZcvHApw)bnZ z#}Pc%EEnY(qppS5g}@cz$TNIQRH=eoDW-5RRpT>m-cTi^a#ntQm9xt}HWPk~tj`*r z_#vBJqCNM-hr*}>k*eS={xf->`&T+{o8_-$eMSvb{|G5~9{LEW*(lkjM08PfE4t*g z2vp}v*$UG7UN`l5=AdhDGkxXP|9nD zxImeYWG6<@sRBZiZ1}||PmR1LV%P+q(tPpr2Z9T$0TP09age0Vu9FGTwonlv7*eB~)I2O!S_|!clXZxZFe7Bpp=>LJ{kY{?zP8Akeq6$FdH_lBzsS zQIqu_sVp)}Z=!HJpV7qjq_my3`U1JnIN6`LWrjFud1D0Gf&;{^l{WxFpl~-S>QFpX zTWjE3vN2kXI$M##-PWi|_W;nbW!AHGua8J!{bi^#r08(bNw&R*r>WFccy)h!CS=mR zByQeX()_bEoa2|ti_ z9Yr%1q=yw(0!aGK04*S7sGsb|Y6CbsxMRP!ycv^FL?0qC4o3s~3ob-;@d zw%wpE#2XJ;t6bmd4hsi_BfM6)fe}mB@DsihH6VaOH%7}TL3M!YM2mVv$2Soa1QLsZ z!sC!q|KJ1w<{)*zwf_%Ee&onf=Ro{%qU_3{~7#*#& zPO5|SwC2~7gH9QTf_nWI9Ujw%8mEF)xuX5p8XJKKO!QtM+9=#SE@hZ-SQFG zuuF8m0l{9%`+yUXXMxIRH_>3K$W$a7`HDIqa%91`=XaTP@JQdxE#OTy0pj$+pV|y> z^>*tJver-Z>lNcHZ{|N!eDpi18u-Y)BDB zAQ-Vv@CsL6W)ENn7X)_e-s7Lg^4vnQ4w&V+YuP$Dfna=Z_|lc#lJ}~Ohs6WxVkj0C z7P$-l0GaC~P&NKu{Y4hW-a#{xJI-CuBLW#sF(qCsADXUu{ZYTr598k2{=P1*{lRl- znHujx4H<26Q^%qBZ*sym3|l`IqzyAqrHMFFbfLO@323>>C4eT)t*+$tTtOhhsWM~F zIra2u*p4w<0HoWjbG%#nfI@;2T}CZ8??U$HCqfBmbs&IoiV>UQBsYD1mXmOP`V%=g zEGx-SvV4_usDhlqg17)z7LkJVF?F*!_5!h5;inRS9*&q!*cw_4A(i6V8GO7b1p2_+ zsVc+6b-jz|X7vC&uU0lLDz(N7^lz488*~4)CGQdxM zX_Ky>)7v~QN`nS5tfA8w1xoSbivhw-pJZN)-YCQkJamYqeQF)AhB(33FmR(0WH%5HB1b_ zdOGHA{k>rA~$v)tez&uv^sfX{tP4F@}*~vR;`1u97Fm0*wWdIvT9OU!@Dlw`4 z5E%BO2-}pvwxvVyNqO~A0>ibqgDp`|#^Mdz(XuE)DL|2pswmYFz&4L6`2nzVD^M#^ z?5e#7Y6BXs-y`LNt_ml-q@{;3_+iT)Ye7$|_6TLF#Xfw-k?DnP9owSJ8%Q@E?=IY6 z4eLE6k@2MAXssJrh;2lf{HcDO4vZx#gYi%^6s7blRnW_PdeU1rw2{-`*(+tAalHj@ zJ?A!VsL3vXM!K&>MkVun*>>Id^Z!BsuV5(KJa_(rAq0{r00rv}nhaH_jPFKUgR+Co zqkt)X+D5bwu`3M!VBF3bM)C#lg%i#}7#u%Uv|hndD8b77id~t1pY)va?yZAm8}Z4m zg@}WApD%pc#}i(FeA?_6g(jxOA9l)oJLvnRq90#bKI=O%BtagWs89nc=GXjK~M zL;Y|EpYA5jR7U7i6^mALrl03?Y3bI&VrJqv`_cjJGq>~?gxF*i->EY+DAPxkvfRTz zKZW8=(yq3lWQslryC61Nn)_X<1YUF~WJlojcxf{J73{67PAQPON*dIAS2Sla-YGfa zM??)LM|S6qN^%b8f$eh%%W4yRcZ_H~BbaKH@x&E{xZPLP`G=~P8ej=8+`Bj3DqSE) zRKjoL3q8v@QX)`${PZ6~#YP7W!Y<(XpU!pwXHb+4J&yKU_8t5#>0_h54z$J}!WZzT zrupj3*+23gAi^j3o*!*rY>p6^r_^L4TIlz-Z!xkv2^_b2@9dM0 z;2bE8=B~vZCSQ8TRbk$6ZpxYSS^IJAp4T-~cTXRCw{b}cD@fsu=Pd0}*J0yn05J`@ z!EtN!ao~-%u=`O~*n<1;N}2;qXj`scM7a>vWsk+O@+*p8DWW^gzabtT#aXBd7d{}5 zB0E!(mBr4!KOJ6VMy?IxVWM&^I*j=MWYum}&loI)F^P8pW>XP$tsmE<-TOdWS%lI= zUQ5#@=A>sK zEVhBJ`e=l8a28bjRW}_^7eqq5@ebo+zbB>{EDCtd_*j?b70-^xy}6 z%1kynikHjc3u*W6-8;&oG^-RcU9;KtWaWsPa-O$GT3zwm^dl*|=jl_HJ4D+3i3{W7 zjt?WAwigfI>^P*EN-6}BPv=`^eK(8WJ)PL>7|`mL8959vX-n+nWe^p1rr#VMVaVTj zS9{SHtuic}ww=E29Fp52GcUcKs`jR;ihsRU^HIvBnhufKl$_q)-h~Gn8Fcg@Fm&52 zOn5aEv*{Kc_xn2>i5}$!F^jHB+K1PCb+}L0rax*5^7R{r=5isOd>7r_7evtnpzCRt zue(a4cg4DR_lDW~nKr^O_e??Jd7WfUZ(Hy#2H9tk4?n|WQ9!f(-2vxHAJ)|~{!{Gt z@4-KekfgKiTQfvjTq%8k$%m_)8yVRNE z+8@aqnv`6NJUKYgqK|uk{=wQJ7Cjgz+2=Iq)~vsGb>LcEH1a#o+)wepCrjg=PFZ$G z1>d-#g32F$E_hApFlg14V1^=@@K~W-w6(S6TrY_IxfLZ4=yB(>0FM1z!<|OvC11RF zF}`>Fn4Dacu8r2n&WH5K8^OUEni?9!Uiy-fl2EnDuWtmaLidb;wL~M!3ttZJnT0?X zrioD1&`AHbxJdJvefjcbZ)Nx$iPlgH|O5S`- zL&Bvb`jG#6nS1kRy(FNYt#9)Q{a1r68fk7)pz~)z{WCw>i$3h^;--T7e>GGABmM7Y z0zDpH#b5g``&|V_Qp;y#{{3YBW&Rq#d^OXwnSboF-(wP33Rlmwas8*g0|E2>|AY9; fg6RKK4`PG)J|wX%eCn+<9q>7&Z=zSMiwplB($g{` diff --git a/docs/img/ml-Pipeline.png b/docs/img/ml-Pipeline.png index 607928906bedd224cbac4e2ff4069ee3cf2da261..33e2150b24151b6ce8a934e78a139fbbc0cb6de8 100644 GIT binary patch literal 38536 zcmce-g%9>9nvM;T~gBBU1#I- zzR&xe^9P)JF0N~4*t7S#*S&hJUx@8eHA95h<6I4#21(#k4R_FEGcii+?n?R3aDhzZn zT*jiU;V%eqaPH*ENVV^1ekpj8krHokavmtZ`J79}&560e!?}};iquxDVlQQq$`0@j z%RxrR83~Dm;NcJ1B7Fp4Z5&AUwYd8GncZ8g`r{Rk^@#M%YbA@?o7qH<1w`tg(D9{0 z4F4!UG+TDo#uO?!Q=w?xgUf}F38z$`LBo(~rbRPD?rSrliWWqBjU*lz0K?=$E)PNb z`uh3jH&|dAVq}zG&t9#ktou!icGgzvo|XF4-!xJy_Xu}Z*>%_MUSG|h`xwa20@OV$ zXx~;(PySY9dfj3LW0;Uw()GLL)ce&*zgxfitK#^B=C=}&)i)Due)kxWcfI$%M?-$) zgLgM9VvQcBm6JnU>&f)`4^jdwM9ie(@^rD!n!wHdtCG-8b&+uQ`|2Ocq8F9deqWTp z(0?vNg5%Fa5w16l+Cu)FE@F46h1~CVGmXBrwG#d3f{w1Ot*v7c4XV4kdukL_Ve{pd z+fEB*G2H9Num>4^SZ|FlJ+4E7#6lMdBNM%ZtF{-K?xxx{S~)@g{NG>okI3Oh7PS|) ziITAS$zk&OeRuxM;Lum&|K5Ra1Wh?H(&oNNt+nLS!kLir&VA(B3&y7RbJ@ke9kaX! zci%5^C3dp7^BdJJg|1imZg#i1y}zLJgS#a(F5}T+|4R!UY?b~~4>iLZqe$D5d9xHM2rmMt+nDn23LNgT(oKK!c!I1jq7M_F_gzw&2uEr%enl-Z?O?|P)UKrf>Y0Fz*>S_KjzqrK{F)cmZ-ix=y z4=(lildZPrb9`ZEaqx3v{O5M2e++F;e`9OmUhf=ro}G#QjH#Y$S_+V!S#A(|e_!#b z`h2N(l_%c*Vz{TbxZoRMM2gmLjV<`(z2%!)x7CsSgJ!y2ae;2v{7zao`X3(GR(m#cg(NN7SJ_K@-rqP1AINxln-02F1qw8>+_8JkeiE61MK;D$ zZYISE7#AZ`EqmN-3N_dpf_0+eI^AxPRTNHds+Bs;ka_xhtG&fsw8U=C7TV1MAn?Zp z=5|&Qd0UGNNU(yZjuPPv89Oc{Ml~j)Ase_j{@X?KPU~`q&b_K;2WNgK;nI#mW`D}A zX0uM@F{Cd&>sr0=$;Rkqj0j&NIBu)?~2-^(2qiHW8ipy@4mlsdrkqB|Pi$R;7|K+Ti z*TB=ovKPxw;O{|-l$am897-v8*Tz2I7D+>cOy?hMlyXz);NY|d`m8XLVdK1ik>H@& zh0W3V29<6x^qtmxy}=Z=-ryoOH3F%bsF{d09KuPFosBaV3xn@66&OXixF0J`=dMo9 zwrK4@Y@Z_?H!rpLVC>&$?dSgBvPR2VxfqR?oSuA~`2AHf_Ehg1v&Z(0gh@!@IC3#$ zWo^>w72$w5alZK(KV$7?xfw6?6?JPHL%rZ57Ap))JW14rLa6BMP4XqIHZY~NXx?4p z-mE6CGQ?tMvptQ8sXDO1!KN&%k zia1myrmRxzuJFK*e4sB)PU$SkHv7($ZsL5H8Z<;+C0P6WiNcVh+H16^1!Qp7MO(L6Nwd zXazO%LOl_M&juVC-57B0T&XEZ^zH+b9NYMg7oTOkt9TWE8xd;`*gJlGg%y%&tV@^0R-yP-fej6Y zn_=y?GZExBi$V8>Q;NNTnN!cO4kv7@P2~#xjNta#cOx1W?){B4%Donu4Fu%`(GeQp zBnNi=`6K=-s1M`$D|&J~%gPdnC40xbIXn==2QnXBFj7(d^USo@Xm&6zsi9Ga!%3tC zpH+0@^iEv^^4{$Uq)hD-M^&^t>ia5&cPA$!7*A{<( zJgOlo@CW)>=|_`j;dr5vQ`fs_$me=+y*->RqM4^3-l24KRg9p9=Az)mJ z$T0&^W$qoW?b{byuL4Aa0sma=F;ot+BTie7~GoTZk!>{%&Zv{#C0@O zjEZj_{MvqYyEA=H=}t^^%zBt$(c2$iou0+28Cdbrmp-B_&9*#a zwTA{nv}K_9G8Rly_AtF7P7y?gM}wcT!m|yg-2}8SR^qL^X`kF-Q^pl}o7c7&M+r{j z#L3aW@-D+}oy!OpT2Ly$v?2hGOzRez$L1(#34{e#H~N!>L+}jz9`e5U^ig zu=k=4D0!}sbI>xl9|S>q`yraV?0lAlWh0ssv`HIGi}i5@^s~lVGR!ZtP$Tq4&F`nD z*UH|QQ#WW%2-T~S-2uy9HHTMd33*q}kUZL3;bbMr=>@f4pSKRTg){$S1njY*I8CbY z_kz!bayH6rob&|HXw2Z4!00G!7py9Ip2vLUP#XTooGtfb;cK?$isv`GQDa?}*r@VI z@Fy7bZuG4sY-mV8tOiE`{5BK`T0(F&GO>;0l6#rJf=*&YlQ#7my>46gDKp%l-tODY{bjWci=#ltmVjJn<*keZNdUPDeUdvhoE5sC?m!}D~KMUyD<@4?X&lLYK9QfaBv2TCIFtuph+|ICQl zU;~c?(g5J~*YSLS9BgrU;r^7!WyQ?a563(>&ebba>2YYBP{)z9R}UNRGkL0|E+p<7 z3c=)HT9kkEhR4Og?sK;AU;)G-`$k3~0NUG`>QxGar{4U8^sPj!8IZ2;LtqJ!rwTmV z$6>A_o64nn?lXvHBWOvL6qD)Vg+9CVG+vEhaaSBl{#~I|d_3CSuPRQhurrNv)h~8j zq>Uc|(txA~c)`GcyJ{wSZ=M>)O2K&(P&yl~pthFx&OQe%|7$6v;{+z*xTXw2G#4EW zN1%&VW%4F%D-XsG4>{@w>3pH&27O_WN>DfnMCnv#o(nhE2O*I90wJ(bvfyt9Uj|9! zG;*ze(Vr^FKg+rBBhc0St-1@Q9fHZ=HCbWa8GNmw=PCZ@I2Z#-8Z2pVBpU7|SApc5Po0)xsVQ}DBO zs=2`3&CmjJo|-pg2-uPWvXJupvBz{Hw^}sj+V#r4d#wKZ;ebigP#mXSlTZVvOK-1I zHN^sGvD3A;NA`D~J`fKLH(U5LMT>Xgg%N&RH9jEFe;M^4Pc49KOZR;up5f0@=9Y*< zf8P@NLM6jZNN<3(9Fh@={sIDSd-rrVDXYtZfznRzeWkjNmb? z)i(;3dA8BEC*CtzQMs71f1Ed5`P~S6+9fz}XFG^i4-Ie@xxZ5kAUZmE} z^DTnLWiRNLCMO7XrGOm6;cu^-xFE_G0_#?OHnl}K3#A;ZJPJ`@q-N@tu5t+u5aVLQ zX)@j2zk3gL(RTK20t-{$;Lo!}Z0wJgN|n!`bHe>ZLwX&oEU-6HlGK@yoE9Sg7u=wN zD{#TVCTcQ7MNak^>2Z7!d&f9th{1taZM=*pBEbdoI}dNYErks&5G^A0)j=|hf){S* z7Eze91To%pBBAVC}nl!j>GD%lzz%p4T{hQ!S+2&>2kbG^)CQjlR z-n0a8^hnxh&=N@5UzIi$0L8>UJaF4NCTRDR52P;Sznzcun(y>;&BeiQK0On9lIy*i zq-0a@AGz;l0wAo!Ib(1$D;!16qU&;WWlLp&1hR=!SC%m~wtGsIhi09Ger-sj!}Np* ztFdrf+ZxKeiha6BT~_&=E7kb7mh7LI0KgqQNCdVC*mt|y_g(8hl;D_xb7GXFh-s9O z>5$2N792#TaNo>S?9&W7Hyo{8I%$lsR;MR-Whs<@Ii*A(wc&0jDJomisvxB z_3}9t9+wP)XtP3L5qzn}8Q7~%#vvqW@#lR~8)TOzHP0}!O=vr>{T`!Kf*Ilm95((M z)#h*T!0z6^7M?Q-0*AB0ouN@Lo4QyB6lFfHS%1v_`8mm8=Qq_Si<@wCZqVH4MzW_P z2{#28#?UAzIwxpWclDq~K!Sx?KTET3wI@P{H31!3qK$Y(%%3t)5(|IL8RyMYl$N2( za^gih$=M*TM{oh{k*~c?kj+y0q8}UZk9LHaf*DvJiVq4c83-768LUyUMXl($oXKb1 z!U>gGIt#sADXC8SY5g#Sf&7zaiI~=NAzoGCo&uuXB%n;ST+RGSa!QH3Nj>$L-%i}g zC|cjY3MG)}1;ioS93Absy&XRVnDppUZq&^{K4EjwO$>^!Wld}JgL^A&y{1a7_@G>0 z>Jb8#sUZI2xInKCd4!C+d@oyNK!QPsW>?PXL@FQ>B5T6%j#rZS*xMT|jU~3u=vCrF z921*R;Ce^?7P%p!3a~7JIsv}Fu<#Ig2v`;Z7X5ItsU!#(CX^FdhjSf4Ll~Db}0mXC8wYx_vZIilbQtU-5ib8N$(ha*q7K@0N9@urQzC!&pd5jvK9MU=Zq@*|%{i1Y$A22YC1t!RK^KxDT$#5pDgmX97F{>0%;E## ziC!%Fyj{FG9*m>}E4?BP?wRizMv>z_o|u=LaMPBQM3n64UgFbhBy$K@IEFq3{5?!k z%*`hn^x!K9xO(N=qKd63@c7Y{f-7WSTUenp>!Csq82yeI+fHX^OBdp%u1dSvib%kxb;x4yAEpHSvF$qSD4Ji1igD z(rZhkILT&Fm4Ms|y#-JU-p(y5&_fo z97R(M0Frlc9YENADeLEqd1e~Fsg;*Gc*1Mr77RfGXnxwRyr}x769k+pXRtva3;WO> zD!ELhb@u7luzn+}?s1xetEfWD1>g%?&|mJyo**K8PmL8@TzXL8fwaDh--ndQbtgF9 zRgD&LZcN3G;4<2;nJC%kJCF`<)|S$^LE1vZp&_`WlUbH({8{ zvYKo617%2TgMM6c^iQ`}x%29$ufvvV-q~9w-5DxQj_5tgJ$GV*sMf4&o7=q2N!%3d z4(RD1--pk|hnW}*OAFg2GeS9WbvRiN{`}k4p}%d$-jvSLWlPB;BAR?WOA)*id-AGz z0;U>~JFH*!>x*n{3IuEmVy-L{Kho+(cpBtQQuN1v8=2Tp`QajPH%@X_(vg_pxTn~H zX$EjsrRhqS>zGl4NJF3VZ%TortpXh5i+JPKA-?Cd^n~w=W`+PKBbAjhk3Oi$0uqJb zx$G=Wfnq@+RZ_{7YJIQ4S~=u~6Nq2}@%G64XR{>&GZ(c>mIMM@PX?TAz*6kkh_fyl%6+H(zUB7o!E6X=ChL?7G)j|x_-pz6X37wDm2J&P z^)>TAR736PQj-@@RE&Qn&)91&;oW+gK2bbFW!C~WkRu@>m0N>_dCC-Nw772QL+zklZ#vH8Oqv&m!@UI01vx4LrESn6yJTOMKOfPM z{Q6$eS?w)_VeIBx2TN>r$<`sY8coBaP0piu<%OHKx=K`~Ok+^s%6F=-EPRrk*osH%w;xu9X`XZNP^6|FY;DIPF&qZM zACIbWCAp6e8`w7~DM zDd;AG>)PvSebzag7}%H`2?KW&U_HcZf74?ip9z9E{906bL4HLb)j+EI97ZqQl(~v! ze$n6RfgI}E?~2ks*r!QvEbFl0&~kZWJIv}Bl#IfdVqpi-a_$(ppl|&BbX$Zo0RdA_ zf}tyj<4}p;OXDU5o(fKcb@z9Z{6a)Wwc98(SY1>HO~Dd+iA#OiobDC>oH{0N$Vw@CJqKp_RIY^W?S!?@C?>29MkczWZjAQGo+j zhT?-ySxS*BRPpF9M(Y(2GYyw3}Bm7D%ekugaDG6qRpwZGG z492z=ognM2ht97Nv?*bC*lf)&IKEqN(uw@!o2 zBVezpF%humNq@5({}+6qG{Ymm`8?dB)t#)9GYgXEIn*VX2pYGtf$?7-+_$M;NY6VQ z!0Hf6twis2Y0HGBXMV1cW*q+A+AM#A+_xBAdah`4p5vMKT19aqS6N#Mg^&y-Ci4xW z7SD(*UQmdx1yNrEgfx!fZ#w`cswgghQ0^Pj@~b}W?dB%w?)`c=AE|4F9PMCrw|riN z^2<>^sHbx{=usXLkN`3X^yBQuS0FNm)V94%Vbm!|I2?N z!K5ko$8TiSrBB8Mx;_ErUU9|0rIO~SNhn$*iEOq3(XiPa;`H78D!VUSq;X_lTd=uB zHx)S3Q*+*;qQE7g`19|LY)&YZISw26piAOSTehRm^!$R11eMstnsJQRhRh_9!?PyJ zRyir&EgORweqaG+{=*if8YrS!xk|H_biory*`CMWr0_%WuiR(cqt+)W2G@^P^S`-hrM&;YJARG%`GB{TRH6=2q7@}D-Zz+>nD|+%aAFKd-t{uZ?bobox-G*U z+cyLlNA!>s?mkzS8d@UkcDYfCa)`>aQ=Fr3CW+{EVtxAQeNX>VACA>SS`A5pArxBF zK3+Z^qmXF5BGLM;lH40s*t{pLoFHNKa*~k;(MR-}K;Yo=8;v^ixOH3^IP9Bxakc4q z$y2~=IJZC1Qcwff&I;zOx8xJ5aCvt}vI`XC${?J*Ph+A5=*Ms-FsNvy8U*M&=8|^%< z@h*Ms7@)%b#7Q_=bBL-K!YQYt>FwBH9$x&9WTNMNZ{08+_O#TGbk3lSV;kURV$*5skLw`?+Fxjd=O z^2J7jPXQTj2Ziy=$wRQmroCrCRtmTNvIy;z$X{Ng&Qfx@sFx)@`F}-!h*E^gs(geVT+yO z?%bTKvMrG_S&LaC$AUf>KrYboC|a@|n71@aTzkfoA57{N0g zu)c1DQJ&<$rECcLE_#}6ViF=RPl|ceDOyTN0T;y=ZaHDw#y?kyV_=(==%(-J%DkH4 zdY4jT7-UPPdrs*Z<0HE=B$@hkT^G@KcISdLOLh`#3*ArqLTME)7A-Shi_?6PjPjyV z4_?SRY~M(h)`JaV49J3iC1TUBX@6Q8%{GP0YAo|Yzlrhr4TPYC9{<9VTKOM^?XUuu z(La_cjwY4y9ouK$-WBaSbp^B87c^MCwSR|~!8+^p;rAoI|3r#8Sp0KDIpf}EbtT3$ zlczGlO!DnyG8yPlBy<5#!IQ94W1#4?D3~!w(fh5rQ@iim_lr6oj0^tbP9ifpd~C1N zXVr%0)3%XG*#{`Eid1$RegVlJmoO=0vND!uIJr`o#6V`6s+e~I&p^ld&S#Km5^N|j zsh*eZU9ev`HvZj~^bRuW1k!_GSj&E7t}SgMdrJ-Yk3d!Na%X)`56=Z&rrmi1xs8CQ zMn!yuaH>_gfU&K1n5b5@VNdpVhaYqkavy)yT6!0$e?4g~P`p|iNsl^VyvL@E=n5_2 z;-x}l6k-{Atj2Xe^;AH-N(gOPvLD&cBI}dyk|ti5>9_9dL1|POwn=WVl5c1a|H2-> zyx+aJXzbl!g=$ZcOF%ys=T;SURwP?Fp7Tbj0Dm?J{c3A8TdfKY`>$U^D2Vs{Br>J= z_WA4UZs#<r{G&|P z(E^AH&8;r@=}b-E1*=3 zk2~8+`$eyQi<@xup|P0Kq?u=?gQE-r|IR>~4k681Ki&KEi_sIe@L=0by0%7Pw%-9~ zkl*UD(K#a4sx-w+xmMuOVO8cxj&xHJr9)gtR4 z`e5Esz>rSViBaiYsEMltKfIklmVzc#@_cXaiRAkAmYabY3a~nWdd;G75NY@TIO5>9yy;tK@#I|B1!o`~%EChQ z6J2vvUw?~G-N9$1S^Fj$f9op0;TGki=1ga2AZ`%m(V8HV%c+VK4-#4VW49(L{tJ6X6!6%v=}8rJb!Jx zJZQJkXh_K^&jW9jsJWn4N|1d@FvEavXxff#LyUU%agGoIayxtx;(;NOI%lX8v7=}i zce(PY`F4N_9#HU1+TEwbY_N4_~>2Z{V3>g%aa1_Tz6J);ps(c3DX+El775|R}26L6aVnKC?E+p^gEea1KIOglkc zPGxD%U!-{&52>|DqD1y4?&S(jyT869f!dbJ${+Hl^)v*lqMYxt;rqn=APNO!(ntvG zo&V~x^KFiik^KMC{E#Y z9W1I^1ie5gDmg=p6D1dTOqzZEP`q*)C^CxO96aZ10i1(0@yI06>(S+q9W$wg^@`B8iY?um%s(rnu1tBQ$<}qn1?9e(u5nLF!J}p= z38G;bL>V1Kjk&k+_;DfprAYT}{0l+U6jfNIgt)u|q~9bE8DeTGsqW(*1GG$C zLaU}Kt3ih%8hBtLALwpsu#zsZ>g5sxkNRmVnE;b~;@F5HG$#8q`mx1gK|KCI+wenw z^|gGq6K}7!KAsRt09Ply;xgWJ-T^ss<5-51Of9>>FSRs0HbT;k?p#E3KFOad?AVRu zjM(Mq(Qg*DG)t5?gvVe0ZWx|=hE=05ztilTXO?Vh0TBg`E3xpRj$MpnN_94h{*^4tnSWYOOrT@ znsLAB$8{0k?orVG>Kf2j1LX491U!@J@n!Myq9n$iJIk3v-ykezs&A!;931d&W(HFO zYvMSM>!XNEDg5NecZdt~OC}b!(vg9{J=_ImA-Hrb9!Th+PkR>~D4v#L4pc6p|FCHt zvV6C9PW~!))TgvEh20v)omyhC-T#?5QD2n@n9JY+DG>Ez9fsW{x&&V$|Hz_jcoO3u zMDono{oW;53E>j#NJf^p|7FU$bZ5mcRW`TzJI;4?Y{%da1|wqHFD1Vdm3a9)#$2tfPNiI59jD`uo~X#E4GOs=(X(jJ=w?z+1hg3rt2te_N2cl!~L{3 zvdu00f~retOd7DAX^*cJD;h7EnV(Hjf&;umeUx5YVa1M&@II7&9^~+Wa$%@S?6Yi; z6)<;#v75AwmPe5UOh^!XSQJO?eZLDsX?KA(Cu9+NNmhrZrX0J!4pXGJqZoRntOT@w zIrra>sGD)|##pyE**cv5s&Y8xg`;rtLl-$OF9}EXdE`U>lwX;(*=inz^Fae?579@+ zJZqbt)2t=a!(P4?nHf&}C>7A{#{iFv=xBK5yWBEDR%V3Ut%E+b|LYYP3mD-svO8@$ zQF!}4^J612f2(3Os5M%?5d(_U(Mz$gx?rDvL)040X@Q~GGU$u-&!&KWR}oPnnhSct z=_b%i-b2k3FX7*DtAN3kT{}}_-wWAQ?i>`BwP{en!~X<+pAERI4M??uf{Vg}>mhm_-*)?c%I4#MA)!Zp|n)5w1Klp^(^R$PA?{yb@Gnv=o4*V6QdFAWW*NI0Da=V-wp>tlco zAoaNW*nI1@5`rn;>b)H85BYg)_b^uEd-*CuKCSeLffK(1vb3bBq}Hm`3ta~eeRKPg zU)MVq)8lmXI4>GiJ~tXzzOc0h1w5+GOVuoZ$3IcAEP1;L?8Wk7FN(6n-FNf7?0bm7WsDsjVK0kxkehxL<6gY{{+elM`hDvO0>x>md%!%D%i_5Ym=94KRMx>^4<#8 zD*QBfU@%}eXH8QX#U>Tl;hyNf;i2D!>TSsvY|sfJ!PCoeOW2Q;5hlp2@ot0A|olIP7q-+ z`^C0>RPW1Z|6q&8R)FZb4!CZQiD;qB$YOktImgX|wq0KUmu!@*BL%r!l+kU;M~+Pr zAwjFb_qa|vJ2uON(GR(k9r8K$!-khsKn07ws<~T8qP57uZra(}Z4>#^>{?zM2|1*1 z1w}=7{bzTahr9P19#on_L5{R3<$b~QlXv*Wvd^i0a?n7(h=903;g4tRn&<7SZ%GZD zq=2@i&4k_IQlg(51Ek(b)X2&*FZ%P={*Ehesr{R#=$x~1q(cO3Lz7E9KsKAmNxxpp z;gg4xCKO*!lCA6-35Y>?lTrBA*xw3V{wQm#3{JeU@0@&-N6t)i?CsP&u{Wu+=(~Pd zxt=h|LT2+km6pU_VbT&Sq!MTPL*N{$ifdxmUC~dMV6p+G_nAQZrk5n_=_Chu$?j7vjR zDbWNDE&6^Zo_bE8ZxmG+dO7i3efz8Hlgke{M;;JFbM*~#^{JHPI5f--f^BJ#4pnemdkVrZXc`oX5QoyoC|E0Q^-cj zTCs^=qT>1MCLZgzyjR+!MJf@pjzjygLFIFAz3!b+& zr&g`g!r3C%o8O`yec>iqO!9Z6F`(Fp_R5eSD1c-nef+wpTgWs&x+J-UMP;OcHZ5h!>BWx8vr*R~VqETBCX9<#+Le(1Pqcz%` zf_XraRiW{J)Lsv)wrj%H%YS@AoYZ#rVt6{~_S$BXv+>X-4|o=Q2Y=1gHh%|b`2$l} zBW3fV1)rzAOAIcu%FDB^HJ&DGRUh1}ml)l=TuS+zw*9hbyMwCBbwkG z>PV@MwZVsKmk=vO`a0XnKGjodr`#sX%YTpdY4LvL?(EutJ2NqnzvH1T1V3?Tc_g3H z6pd~<!JACEfp5C( zQV5R$c+>rQbKx5@@ZQ}EyFy-H7ccGkU9U0jv#6l2ugD|; zoP*N_dcF~!knk+Rf3BZ=E`R^^k6(o6%U5yj`UXAvN&fwGX?%N>c^Y41kxMJ_^wt)Y z_ePJ9b>+PXQBYk0i^+z7C9^-_`O%45eq1-8?ibE>h186GIc~4HaMV)5nh~;)y#MRq zF-~{Pg!SgwDlp&gZIo&5MJ(#7CK#Xf5#4|@D_4XOXGo=7$z-%d>BZ}!VQ&vKYkaT;)|lDs;f&|@HPby#;zx=+6_&O8 zAhN|!U^wbgA_@Y&zOu)|X+8ylHT0_^g#+Wh`j_SIpQNKs7Tu3}ELca0HOtW5Z_x0B zN2t`zA_(wPt_$ND@Qi$+zjs|OuzIK5qdq1W>CkJCH27EiT*Rm@7&3AUdF5tF+0yS` z@bn~cb;|x?J*{(hEWOjsTaO9hd8hY7d1vG9HO!qbk0>Fc*ZSih6E5xc-{oVvxl!5| zQG;>J(R-7!cqB$dTVUH(+x^z@G0iMukF0s5IPTBoek=$)eT;V+LIo4;PB+K19xbsj z#3|ryGgrrEtRb|sLcpkkB8&OB{X#Y6PH4o@r?vR~PkURR7Vhdcc7S=eDfR5(r*|-7 zV?QbrmY*n`%$7`KazdXlsOY?(`iyWhzccOhWJoE=IM0Y9r=A*~H8!&UaeD|rR!ZA^ zZAtLo1#BM9Q`SpUODFXGTJ~xXBZ5v_qzVobyNu|bDt4^8rMx3h?^c=zCV+{Ifkb)Wu%&{ z_E~YIrYf)8(wf=DiQ2r|0SiBhVg1#&Ap_g`{m9|z^%)|YW6w$?5HNd)Tf!n^ipkK0 z+N67XVE5VTko0i5f7WB6o=8XsJ($5OH|6;rwQ$Gb#rdspnd*iana5U+fh5;BfQb@q z`mr6_5^^j-r~iwy%)TkZczC<7oz%il6hMkpEiWvV^;JVG^=gy?ecPn8UyumK4?}*! z{!oxAp!byd?tNsjpQEC)5|%m^T!gQFq85L}{XACtZK^)%)*amc~40KQlC#Hxwz>bAM3`4 z>16)4Qs4MuZTVF&1GMsf@$1pWgc-{SA4@Fd@gd{m&~PfNe7ATv~e~XZzWg_i68-$n|#hjV4opCjb{)Kl@-%2-hJb zdh^7+t8Zzni89V;Z_rGi4Cq+@U9U9zL>35@?$7q_Eml_()&yd|w<2Gust*+ctiyU7LbgZtyO_e3P%|I$>V8lUbP zO+W@f4Cj(CBlnNRd;DcqHw^6z&U*~RJXbpfmIFG`s5llpZ^I@HP12o@2r=$@+x*V2 zA?~?{MfX=TLFeCu+WyNDDb(F7Ul$B^MdHPPP9iZ&K4jj94E*nc{ZfH^V2qr4%*9B> zbIy7GzWV%v?*HH_OAlJJ*F|o=h`l}t>9i4^znVSMVSM(H_TS&vtIP!c5Hrdk#R*`d zg5F%3L*?&bT zxf#HL0-(*b`-x3u%JR;)LJg^b6%SZVOwB|1;$7gqY5n@W5AXS;GO@dFk!R@js8Goy z(8c|CVWa(*EZ{5ehqm{B*5bc{1bjjLUzrEqU;HaKNJ!29$^`Ha(f=MK@c!Td`}d#a zkN5xQBW%cfd-DgIO0K2|^YL!1PZRJK1EeQD*%gRd6*gXsF#k(SAAfOUt><;5R-@vHJ@c6mg1jn z7Z1}w|147M37-5;AIHnVg`k`#k&0gRx6}>C9J(HJ6n!NBa=ZDnj*d>l!BJe(+VyM4 z{RLzdD& zBK#A*Us!)=z_m2}m63fL9Ss@IB(P9(WPknZcZA=y@_fV0|Bfa+m2E*kM9S>`XPx_` zhyxnTIRnHBc$8KjCT~48`Y*G7^?&i2fA9Tv1;x>fZnhPflv-K%NsM672-$Dv{r_{b zkNbus6KM9n#pk!wMT#VUjVoqM_P?vppYXwg z*md<^x#9z0O5Wo*=eAk=qMJLQHuWE-gd_D(QHTC)bS2|Oh9iq!=K8a~zZUZZr1xJa zI1g{PQHQIWVx=ZeQg3=?I~J z;?kNBioikg{I1B+)c?4FgEgLV-TWm0qH|a`<&I>IqYG-AZ%e-&;r^E@oNhh=^tHEQ z75UX=fglj9gOkd@XSwcX_5uh{07?BHzi_;$8&lF=Qu!xlT+}qD-ncpf7?1xf^tId{ zs63^Bj13-_WFYlDKNtDglN9K8_Y){T{b!*<>sU)p7c1D9Jdn&u%TRZ23n>22?Mbhx zDYZ*{n!W%;1lPR6ffw7$v;6PrNU=2s#rG|E`SZ7`c#~*-ym+@SOOOM=&;F+}NS1h@ z);#6Z4}}=8wadE5i|UhwduKi0e?jK2G>dMHVYz4RTHm&i;U3_0T$>z^rFM;fwWX^^ zX8{u)DD>nxF#8$lpwvusb3*bj4N-~8ezf>d)%Mr=gq+(llV~@S<6_$X+CKk+r{4Mi zf`$sn~3}aiswY@wsf7^W2^l~y4Py9 zmZ6bOi;@#C1DpIWiuEpQb6XnP;Rlo`k(w9l_X_Ke^3J2LYmblK4K|(3U~@D{wPht3 z?2k{zbNxn!wW|C$&RNggM&=CItE#;^Io{)-uF@0ptM}b*tK8lzma)Fn6T3XAaet~> zV@ntLcCc<4n`GW+;>lJu`MHtLY+H4%vSsk?4))#_b1lC%47&It7{MIM$`YElIZ%-r z@#9GXip&*ho4Yh&kTeS!HeDcKcBi~8Umx9s72nNx2ds%Cxr?1O==t65Zc_`Mdo^8b zWs9*)`a0>K4rC2BHtak6__@kY&Xam^TG!}ODj}JJT~xC_vL-n>cbTj=-syLEP_8U| ze3W0zv8;v~g_8KQvmjy2NJMhD)%D1py-xOi?Yz>t(Q99;@ff-l!l>^+0(v5C8?qmM zvVWrGi7OJ5rd<4Oaztq8dZ=#LYeZ=Ddifb@6a-(}FcXM$Z^!C4f-xFh4n;D3-N$qa z-&XUA)vX2(rrzEjz0mSI{Jo#*HXmp7V*jXEgF~p{DwSuDgk{~xTl3m+IR2R9W&Z6? z_}yyXF-K;TZ>8sS!eHHv{JMchPmNZp_rN$~nva21@1st;nvnSzkzYRs8gF|OZcd^p zsBLa{9G7J+$dvF|qaiB!FZj9FSbEg?237M~hI}Ae8lCQf+0odPN@K)k1`i=mGLi?- z%D%N(E|Ru1S*Zxyzf^yXHWalws^@!>$R^->wnJh0^!U=bC>yz-WT0@gD>(S|kF@K# zy`Z~fs*^}w^_QC|??l}P&Xoo0RA-J*$t5ToR^#bds)OgxF0l7zoK_nW?q!;8=NT<8 zv)2n3%aO3L*}8{)kAFwFi+LVC!t>r8Nmosw4!O4%xhl{TZCFJpDc9e9DF1($d+Vqw zgRcz~>6DO^7KuYhgGvbqNXMZ;ICQskr_$X5f^d*-knWJ~F6jp8=Dr90{l4#8_wT#b zea~8+CE~<>XZGyb^E}U#d@o(gmNQ|lp>*7dIc+{&qOajwuKHh=r@-Cz|AhaRkUTm< zS|XQc*&ps33R)HA?WJPOf&jNd;rw+ni%>SD@b#}7&9hM!E!MXWel}j7{K~ACbje6c zXs7K`&AFfE7l*b+Hs`(0oe!-TBL)8K^<5^l0B4>aR8KQ_0MMkYu;S=vPfj%p?)PQl zU$VxP2Ab3i%z{aG@sNRuLWn!(8ozRaL3%M(DP(~dF=tN`{=rmR4-Mvcx< zcr>QltDiny0^H9phPQSqPl~^}Y+P29&_5Xo)iGso>M|0_JxjO{Iwq4{Ugm@k9mYER zd4ZOab9>9{y;stmH!o!$?&lG58wmVI4T%u(ABPIq&t)m+-EbS}8#x<=$O9u(3^C!W z*5-Km-xXz5|3-ih{l&@pbE&!gb>4Zbb?$59zQ_-ao>%C$-^3tS-CDZF{iGI)eMcq zWJcu@dkD9vqT!QvkurJk;lce{f7Vta&opk;{OQ0slJBN#aH_{@jcl8qg7O6bFm7Nn zosircC-klFOg%J|2u@k7?}q{lW4iZ29(*-++h3QRz18qQ3uc4^k?$xYTD_A^x# z=Y9l!bWj6;gbAyw2oC+n$LQ;HFD2dy2^bFXN~d8LVVWi{Dw-^~jVH{RazK!_u>oXp zSWG8AGaKN;!_rOC)C$z@vKlMw`dHGQ3V^#8wq{4{=jF=fSM8iI$&0^Xnzl{E__zr#5WSJpE8ctq%S`E$U4Wx-NO}yZh6Mc zjKWbxlhBug0XXJvbmt3Vh`R;Gl zEoWWT$IE~vxyK^{!|m71aKB5JKHMT6XgNU8VI!L=8~gAEcZw%o6^u7GjH?x5Jam~) z1Iz+(%n1i*wh@=4NZu3icsoej@Z|97T+=d-)M8WE04fAI+WYF|z}ZOCN{KRSMq41i zMkRUGSD*vJn||2%{Ul2kcPU&?SWIrN_e7h|P90oVp<$t%d}=8vD8x%x^yP1C+X6I2CGOL>seNBWJvlwT<=pf#7anmiqy>U;LK zl_TQKj5oMML(W^M>11t~4-;M&>+YqNebtaO`WZJG2EY>8TlNEfATvys$ryzff|` zttGir_$~f?NZ@qq3Qrin6Z=9zP)B5DS+ zOlwvOds`bv`Um;%cpe8ylB+@#hT{49-Zb!3q(YE zvc4vhwLRre+Lw9{c>=Xx8nn)mGRC)Ln5R5&d}OA(1I^%1s|ePOv~=^ti96iY&Tu?` z8)3oKCkbsC>DY=v^K8tC$Wum7vuIa=ZRaeY%ReeKmES|4tFb#sOWCs73$0{8RQCq_ zE^>A7{&(RaLe}}1#wU4TL)@PXnm***#255dtPeSwk(IJ~t<~gHjRf*>O<7sfBr?J-^1_PB*B*Ez`7i6t zxqsomA8h;_i$jHh)upnmhvS0Hae z2w%*8Vd0~oQ6{0JPr#k9^MX%NrjD~S2`Sfeyi@x*tS2U3Gux|87PNALrN z(ug!2oYa6#44XdK_9$;@&aOM{{B5B}$M(QM2OR^$^O=}o_&%Dd@>99z(;Xa+4EiPlF#cK0aNO;2uhdwX7Z zMs8IB8ZiKd7ge?ZgT+E?3F7Xl`_A-2SkAOq>@-3ZJ?M6#MIot^-mXE~-TE35;TuFpmiSmx+ zIoy8r5cL8W7!(y2=yexI#a9S|uJf*3<%e>bqeXJOx2{^bee;U$ehWtl@)T+djwK=& z_`oUne5zj+<-o`D%S+)5T4ATlsY#v%1$3Vw%PVR}EIgE4wPpt(%IMy|>5=`~cs5t* z9W~yvIe33xrqy$~G;7uWTlcP&UV78}wq+G)>y$ztKt~2BpyL`)bt0HeFv7XiVIWy z&QJ|&ZiTzYWaVnyF4A+K(*l7kITAu;!ev8nF6TFzQM>J}ab$DJZ=@6+@Kp&`{4!6} zalkDip|j_2*G{spp5CuG*F$aY>~`6;tm#hVJad6ycs!Uv7w+!INOfSvVDlG0kI|(# zlh0o2lD*(*`DVAYhfCx;43>lM&7%&cRq(VF&^6RJ`DiaJdBII{x0==J`ROa!h6$m` zL1$;A@63Nq)bh{JUu7{7ZE=c;FgmB-12t( z*Z&5%6L{DlQQ$n&)~I=PP+(2^?0}>1z@HA!Y+iJXNcVM(+_ev~K;@~Ah}-i6Om=S( z(HALM^W|qRo}-Ptr^U(!rrq=Wa~YMHNW-popEDM-S&(n=0|iXc*5&Fuc!+`VO}4;~ zL=DDCs+vFSktaKgZ<9C&?VJ$%ml3#iedkH?^K$r@-fvx13ENX3ZXct4%^( z_}@%;`>OCHzv(pCrrHM}vEY~ORS&gZ`OD8=Y;it){YPux4*OY6=?Q(MS!{2%;#c`T zDl>NYH-Snwb_fDn&1TO7^s#$eWH@6D$<&E(h><H+Of-6ut#v^)r=9v4!Ny=5PmqG zk@L>p9OiuHz5+N+#Vz$u@84QqGt&b8R117`{S8XBZSraJqtlXmP224^79L;M_z}4`+txCTfy4L+D#@5XCl)?Y!f6RP#^rD)CK&2 z5RTD;-9?Xu3BY`oF`zFQyfgmQDu-pd(FqiLo|ho5^31Dnc`qlX9JefUC6x4A@?Jl$ zVUQZdZRa8RaZbV0H)r|&%`UV>;ryi@Z)BxM4SVjpz5OT6N98_yhUAVwehA>rZVm$; z1`#Va$O1^^CW)1&`l`mNFHa6LJ<(51X4{YybTAON3kRb+oT-+uG*uUG&U5#WYO zg~4oAz}f&{*1!Ny0H*H(pypCd(G%=x74byJ&8vcDRpG0ZuviZ_rOImwW$P0d-*e!O z*FC$4z`MrCU_}?nQ7Tfk;C>tMm)r5RAKF6jOTgB(j6j`ad0XiRrr*vrlxjT4Bgd@1 zIH&KEZG8sl3sY58NkP!AYq|x~S#QdRWhTy)9UP!J{}H=^6K`K|UJ4I=#EotA;=gQl zM1V+tkw$6xQ%g3~2}Ro~S}Zx5iHHSQ8NB`=NR0^Wth_E$^HtDN#9!Y2Ys>Z>1N3Su zPALsJD%L^Md{JP~z(8omL5VtUXhct-a59_{#n34{MK?qS3?NPk@4dO=Ck^C?)l zfOv@>HB8wf3GaN?qU7eF9#ABpij#V!HGYWFV$2lE+{23?jj-bMDtoZMr1`ZWqP@M% zg>ZAwCS%dC$tua&OI}6Zh6t3r02yJ1J+)5#j9q2x!7VJ9=h|qrdHbxTJ(h zMz+MI!uJ5laW-0y%XUhycG5+d+<`o_#ayM9IqMxNVoQ3Csx(?C?&~IP3(d>|{Th)} z?0`D`=XJ%n<^_tOHp- zZzn7?U;0>$NGpq}Y+ph51BT4^P0cb2ch&2ly^mb=?zA)PXMabH9|r(1GvXBTk(fkH@0RqQI0g6J;jK4;boV3jSs5cCN+ZC~h z#Fs6ez5)Q(z<6LoqUyxg_PSbMx<9XZd`VoXeip^G$S_R`nPAYj``3@i?@m05WVXq@Izpz09}MUozeSDo?!er>Thdsz&PXw zLxj^oK+W>E7Z3MKChHQM=LH69e}!3KB)?c8o45&{RdId!Q%rY&H?F4wT{rHwO3r-b_loAa8Z4vPMu&Q96 z$8Sjeo2|(2F|ge5V@be>f{g&!=kdFA!Q-?$8aCsAp8$|n-~*WTf4--!t*=+R0LuM8 za_)aB|IfE2ul${z|Nrz3f(|no>?1DPsPhooe2Ec*6fXsY;J^ps^uG)0k72%~d|35n zPI;^`iTdf=Eviw(L+^(hUAJvonEOof+m1#`fdUEpM`sEv7BHXXLfu*9{Z^#uu)}Uq z4NTwxXbr%=gMG430$DwN*LdI5EM0c-3IZs8VD$P}8tN<&?Ph;H=>9uE=#pYnKo{NL}^`4m4abH4#@By;ZOxOk6t+ZtbL z8Qvh`l%LmI9MdI=@J$pP51uC?|w~!hp{&Rl?d-66z|f_u)l^V zRY-y4%Lo@|72dTvkV_}_)^1TCcD5#w!_HLzb43l{-_|^wFAZkk7B^j+!-_!SLXxf! zt&9}`-a&r`=yI<4ZZ7$9|s36+H}YWyB1dPQ~D4`u^NdnMnKszr|kUsqiRIW zLb$+;L!tE;)|ecKUic8&8(vpAsjX0|ECCEMkE2$_$I$?zj;nfc8SuUeSB3Fer;WM$fY|yY zczMP=r-fcyqzTA3MOp#*rVOKQ(S<#oXSu!njS9SEvMYYOLb_$Y5`W^;8EZ*q$<}O6k=GI$?fS#|7U%%a-of?H2Qy5MK632;y-g zJ&rQy=kF1384c0D0Zf#Q7#q)BEtnr7^8Q1l2xhaFWm|`~L52d8uf;w!%uSK54}xW=em?ccqo_Ogg%MFeoBe8l z2-Rae@s64PtE~v&34vGXW5)A1vLFt@bfV;+=>Rt9atIQ@5%HbqCq7DiZdvUO2s)_G zD5|6>K>CsHBXyPDT?E<0v#4Fy1ug>BtqEEHDz2|ia(YzO-h+oD^$oI+042RgF9S#p zJdQVDdHY)zCz8P^BG?#RSr>0Qi1=f#jwRdvN%oPOt6Y?3otb$#eqNpekGoqYfo5*Y zR7!~6R=7EPxpn&H&y^GgROBwQM6Pfl!-6&o#7ia75zLQ?r23$t&k1Olr)$cO(h_q> zQbxiI>nnrNAmy10zW`@8ner|moPV4^oMw*}hCzpc0S(V)1%-H|K;}NFNUmLywVwRZ zqhI2>+i*3exn!0bO>VsU%ec_=C{(s@^}f)}3GK`lqLs}Aj|gj+1;;CujoN;;iT}7B zd;av(y(L39v^g6RMV||R8w(wxjSzyCbZS6rFa9*exCy%`LZ6*mHj<3I2%cd8kR5-0 zuj#*b3_zD#5U>MaG!2lEvw%E$VgNSSBlCb^~9rAqiA#WShr?lx67KSZV^kb-~L7&W2(t!8Ryh%QENsQpAf^&1q*GfMH^?!{1US@YKy4Xlh`dyV1 z^u}erK3cVE)w@=;SE0minBIgNY2qi?o|~O|0TzV7{5}N#tJAC*Y!!H1Jbc0w2x0)p zz%~ccJtALGsSONH&vOHDS@&of9ag5romdh*G}Q+}WLm|2lP4>(t;0q7A`Y=qptYYa zCCg7w)(WBD=D7^yKn*ZJ{o|N~qgg+j((M)lM&hICWabVx;!^yq<}^j736u@zB1KNN zrQu>w14{szxh1CFp1+a=)DidgfC!bF~Qq(7V^7=lxgX zf*6*bGy3-ZC;A3FoFWW@L%)969WEEy-w;~&t7tB=$sa7)SH!RTt)&_>j2J9oaI_Ho zuh#%)9KXIEPcu|!)of7HFH67ha9QFF9Ah2*YQwio01fA~g!pDF~iHAB2Q%ErjM>Yi`L-f%M1x#TQQN`;7~3k3t}!V1)iwIDhwTz^>c2ygi@1 zlxvDba$SB;(bMum(>guA-_yz%8y}d*$B*WSGaJ!-lWlDpNs{GR!3EQtRkEU5?!JZQ zbw-gJzmhc%Y7;Vm@fUr+FElpQ#*{&94-aQ@a7 zdb4?u@p|v5Q=R->< z+!x6|qb)5N4J9Wc-H^FHA29SfoYY-4K&t2zz~+~&IRJ9Thd;*#Ijzy0pcUU76e&uN z%KYA}M(66>&hs9;UFp{{ixraX8`3_U=Wc$u&uCo!(p!|PeHlxi;YmWtpw0lMA`p?@ z#fT{!dD*<_8es(rXUzDz3^*^i(x0~*lDVJm(}0YApJ3;h)2vZV^4FI$o5R2%$D2<2 z#M`4xf=o_4~E)xYy*KF(XFLf1O zOWp$c@ddNQh1>>%gNEJoD^WTAA08xU127 zeiUyYJ`)w>Wtxouj$(6oz(BLGUt};OIqSGw0mzrz0W_kXeaZn&Em$`0c<*IluBq(s zVEsYU;qM!=v-N5!AhT+|x!L?@mGsc-)M%=`8-)^qwc(ybt3Z`;Ia(KFKDo zMrRSa{8ybeOAEp8tlt1xU8t7l+o?C;g7dehC*y6_yxARy3M*gXnAg-U^d-%_O6>=)O ziO4q8Oz&h-BHax*vT1L+_CJ98;v}Oz>#>z7ks%S{$mV@J>+$4}fISGh}(TImGAe*3n{S5stv+fmefv!tKsDB4$SV|!Dl_9Rh0gjdjIN31uR zd~VT`%*!p->)nOW@6;O#aAwoV9|URlJ?Ayx7{+PQ^H7-O7I^8^eFyG`b!vK%cOjL# zdWSS}nfBSIsV=Z&s@Jt_BTdY%kZi+eFEZ~!fwRIBM#1MmQn$^)N;Z1;f8^E%67*+# z^iYAZ)q~6e&VFx!kL$1&7#iEDft_*&uFOr$X5!}k7Ja`PN^>3|_n7@3 z#(MJQxUNQI#`~b|Ft5PI`*L?`k^akt!Yt&0H_r3xOcqrJBQDwX`tA>Hb^@$7m!V7F z!}(zdoX=mK&h&JfBJO7ipX^#7ieNM99j!}yNu50{X>7j?KF4RlJWWtL!hKv8fvsfA zn#)<0Q3_APdM@37vU0J4J2(#WRTEiK1G@*b2OKYD zyuSYAHXMPRf;@9LU6J?oH;}JR3v2hK*L6F}Q>)5l82U+jnjXg;lE5|gVK;>@%$>p` zh0Y2m!6z_K$x-l6e;+>Pu;$81@pb2>J4QN!=aG1R^#{)oH`QE(bkWAcXBpRwuDtn} zA_ey6f-9l0ex0)ZgLGBQJ=?B2eY)^*V9!57>>n)U!*lc&oaW3)hmx%g3?a{>?YVJ! zucKIr-5LQ84RA9I<@Lui9P{$cQloQU{Ke?dZgPy&gY<*!gUEv%;qnS8#m_yuyPY`- z;7p)q>84|d!Yyc$Yy|rS-ZNNI2AC>Iy>>&pefXTW$9gvr?ioLS7fWPQZUeqVXcK_7 z^*VdG+3(ZG=^l-3E{@3Ikn>{M8`2|i$_9M}plzl_#$rk}I-X0NLM~C_azb_8*&P8~ zm~Qu-YQK@-KD3^wrUM|s8}Ab`TGVeI8vqvTp{we7C$VmwdS?3Bwu3cmI>FMD85Duf zlc>Z0JAE&|&pR}7Gp1;p2g`%O;s>~vFPS7CG_fTsDDmwT+BiEN;AAj&><%mzrAr#* z&ScZQx2rb4s%?93H*f21Ek*m_z_MYF0mnZ`Bv1(3XIL=!__Q1WThJ>&;ribb1Un`G z_^|)}%Ek{Y)c3G(^zrfe4Q%mGP=J8=?}G_Ds}O=Yn#Tv?0^}ZVVX2S5Pdk8b--twF zM@pA=IT<$l*v-Q?9l>|qXK3cASU6#RmN2Np+^sPCp!v+GT`c2Yp-?!>lMVWQ?GIoG zw}Z5WvWkh*P`h$Ad~-(|L5?Ew0U(RERM)Em$|?6+_6x8P{96@}q-TiA=di<1EFq@3 z{4(zr96azdhR2yR9}rnIAaNsZXJeU@tZj=>WnHu9k7l*8cCqdgIeb=VHu#CBoHa#R zJETo*Z_o@ia8TpORg(0cOViK0B(#-pL6LFS8)SO9gYZ}pq*cb(A~n;6?FP?es0xeb zNZ&oiuIl?5ys)wyh49vH1A7{HBm()IOk))A@jT62g}g zzBg9&3FI>2+BA18l6KF?i7Uw3BYN)f(Qq-FV1%V;k^Mb6De&T8k2)1hw`N~4i+)`JS=zODMJR_Z_7lh?AU-&FMA=&A0HVm(?z&fm8YXPOR&X@9UFX0Hg z@fqSfwpZ(t-I^l4p7!>37@O;%cj=)UQWGWh9<o%Z<`Rik}V*rxm;n=ef$33CH{esxZ_-YQ&C;Mru z$hu}KveGq{W&w$K5D3t}yo#;&mKrsS>KKcrlE^cj*YEJ)>(@j3H)bK|<;n1JKU6N?dEjbEe#FiO?p3Ig z%4`PNVo5S&BXUfLD^(%M4@RLc!|JB^pdNJZQbwXqhgd_M&ZWD6mY&R9^ZS@Fbb5et2)eVh>;W`cZ0I-Vb00U~Mn)NOm4;i%hUN?bx134g zoqAf?+`#SPA7`bW%{>9O{*RBkHdtYTjE49etXjWpRsoK&dhg#`zCI=ch`>-hS!eV$ zjsA#{u|DDZ;S~rjOu?L(4QatAR@Q|0nRxk|s~IGEkR)PSggQE*2J`qtRralTY+4j+ z8E(}8bBxW3105quFG0R2$l zZ`l)#L~{`7`&wI0;2G=lf9~Dr4-=~2SioCJK(y5+1TRbeUbXl8D;VQ3TX&4ag9N(-2?#P9eHc7YFaw>OPGz#QCW|Eh&(&oVQ@uh_e zvt=w}!VR+06H+>6{#2Z}vm2cM+)a0Gi9fF3&9+T?>4Lv!`60Q&&EJ;#0NBC(sII5$ zj1V<*(ldSAUEeyEiT~ce@vYZNoQG&F-{M(Kv1%w(U3vdJcR9&ESe_Fpo#dFjRbfy? z^to@|acK+G=VO~ZHpD$Fm9}5b((qfL zNOoXhi9YNGm76wZJ%-6M)51g6_+Wg6Pu%RZ%^$Y3kGYHFrIE+}jaR@e)q=j6(sEHR zZCTEqV}hhV3;}m@U?Sg)eDTyWOoK1Fub$L@@hYjQ2-hoTY-Lr}7mze=HNxF}w)T;1 z$`a{>;~HEVYJvXM99dBTXUFM$IbnbxI!Z|XJaB0CrR$d9szWWAkxeR1W4+n^*Rl#+Fk;dgjp>$u)T%{!W>mKv$vvmpyINc55+7Rw(LUL&~&@B>8>l zpH8z+^pCwnl3bb`!aps_E@k<7Qb+3I2;X&NVUXZbFD1B$Zy4gLhwcaUNfUi)61030 zsH9^4yqZ_Rwr}#+m%BZGX{kw$=j9kTV}v~8DG7@KFA35wLGb-pq|3Ovg>Rwwax~PVs|` zFAVtObaC#qq85;D1n81-qNP}(o3kPLr@%X>%~FU7HO-Uy!?Qof2H`%wrxIMCxh|vDarhK zusd1-?R%#|K9#1S0`aLBFBZcw67+IPd+ZtDT^M+SRe19 zDEp+M#xUy5+W1R9TA>50v*7)yF#}{lAK#D@BWV-j%V%S4(;^6aI-VPgubo(T08?=!%|@rGaL!Gvn-E zQVRNFZ2R2`LwhvEZ&@qh`HYDe(Kvo7rOL0Kdt*!JI}mL!q+hG9uE%S~=@d%B@N{Sz zPjcuqrb-Z4uO4qn(G%i{8I0O4_5h?LF+-nvOvk;<48#6gFU;@Ek-f1)^JbHibbUrg zsonOes(=7*Q5($ySrF$?tEeXnMJQiQgaDs&uTRGkyqu3j=WKW`a8447I_h>|yC!{d zrEOU(eU8vmwTD|1aQ7^Akd24Jc#MQ&pRzn~UqC=VXCC^3R8HH*aFnEuy<{VnXqwI0 zLnr+1mKYp$2=4PKA`341_tsnROZBiTrr&&^O}b%Om$d|BF-YlCEfiw_mDdjOQ#x>3 z(iEXq-FjX1Y~iup46>VIBdwZ<_o}qgYwX6|EqhT0!jO1q%`((yl*C>%zC&8}1uETQGLg+}Kjxv+Hr@Dp34!pxY7ljAl&IAlrd~oHieuHbyctLU!?* zDZ>(LnjS6fZxr{cf(44L*_5oL2qGZ)KAxJg@8UWxf%s@ZsAV_KQ7GBo%^s4=pzT^+(0Q`&#X-STy-Q_kO3lDwittVpNelb%;$0WVP)>pM zJ)rfIrYB?W^X{!E-)t;G4U3Atx)$EBJ0pNAj!R7;2>9Xl_YJ>;5%uOVek|TD267@r zufYFNl(|mrV8;3oV1w+ezMH%=*)4H~KsL2R(~JXME_Mel!T?B>lK<~(S5)5nE?mxN z{=IgES149IldP5jp@z*A{kW<}`wng!)4&O_RP2>ThfcK|-FH3VaXcZgs^9x^>VISW zi+`k+u>s?dRvI5TN}3aCuECAEj<1E}NVYn;$Q+t964n)F-Pl3foUKq$W=7N=zX}L_ zr-C3v&7aHJ#Pdf61bG*#+ZHDhJ#365nJ+6g5wbZAjTfXdmlGrjKd2y(H7b7DBLbnu zB!9(ijq2p5r1afHUk@@jvXUIlP|_J~#2wdki)d-p{7_?G=FCKtwaa5U!%7V5%H}-9 zQdJ5UZN$})XeXj|3YG(jinr5Mu)H7rc#HhE^*HNE_AU78L;a~BRJmCP{SNro`NUrY zk5!m9WZ_ltI)2XEK4Oy`iQd4H&cPLl#mouctEqGe=YDM=aOQQI`iONG#iIw8`@-Zh ztSh>qwOZ%fVvYw{wA-_Qgw-`l8e_2aXWylMhmYNMA6fA(RI}_#3`kOD#mDaUPdv2) ztVrB8#ZsW8K}Ph=uQF-Pyro7wt{^kt#^kk-H@J4D*SVU6Xr~WVgpvguh}t%LM6l631LO9 zkV1}5nsiZN;|rsZd?debRkp?Q{3bx2yO3NhfdGBSvU}f|=j2V!^|}MX{cdiv`p4&2 z>`CMxNjp{ud*8e|BWt;{q4hAjwUzR?$azCjU^zBBL zX~kfTSj0b-CPekyIvZ88N9u?s&#CcaI1tr4;W!e(pad3n8DzD?c*>M^$1q6x5t;Pi zXR}p0MNL z=BN)fkvrdfS2Sp6{gO?h*-DdIE8TX!6C$jxqjP7m2ry7j*u(y`Sbi5Gi_)!Zf?z^h zERW30ZiR0yswvdSUKBC5q9QMvQ&S%@q?QJ7D6FiK?5c+h5p+V4NfF+epEBQ=f$g(W z&;8r8r9zktSDj^z*OiO&<#Is^I0nAk`|(n*LO~24QZErUiKV_TicczRlF0bUdoh+O z-vF1ZuB7N%IBaG;!~!$L@j^(h5qCln$LhDlFCy4x+eR_K&5u^ zi?cO@*ylecbeh+8g8A@&@J?GS6p#I3&ei1(6mCbJVm~`e(z|;DfImz0_U{ZBL{W_U z0fGJs)nx;Y+k}{!Jhy6{k^~12ZT)3GDwG;&bNw=aTXPmRl9Ex9O`IwfS{C9;dx_Xw zupxGh7hTY~93k=>Ck&jhfOA5zSEl4clmgO(NrVE3y(ErtO+m+k2ntd%WrHlM?!>3sZ>pO@ zsi&l`k|MhQ1e%M6@00llVii*-&|EFscS3)=xv_NOyCi6WHuOikRCWljnUkJ?A)#Wg zxRLbVJTFVVk5pUfdpU5(VqPWB_*=)H#Ee)sjw{C;G`7y9L_42$;`>UdU&DV!?A98h z(FzE)30&|4)=$o)MX_GyZR+Es_y~*;2xpg)YOMSZI_|CMFALcN1f_-c&*=aQ@ZJ~1$~a*?)F?$DKr=Ny%LL`@3mR~c)IB{8RsPXdFlDH z(9?%+9>00@3;(B^0PSp2zGfs`vvpZXw4WfW8iR^Ol@v%__BHcBG9L1Rpb|quUyex6 zI{OVqrDjGsX5(O63VcomNI_p#SR`ZZ&VplJ!=GfMYD(=Y4ZaWEG>9+}sUMN`H6Szx z?`*R@-m(C26k680lm0hc25zq1bur)mn1l?85~^& zS0_Zws%5#Y(S#SWW0liZ*o1~PxX?q0yyF%qB1kmgvusKF%95?`H_cJ)v>U_ko`C)3 zeN)jNss$po^n{n70Y((UDEGH3i#IW1sEG`+x>~3 z_J=d}Y0FI9068hs zT)!meN>>F4!=E)?<8bLFWIPqnCTZ1k=EW!?)`f;3Ibr*8+D=0%ECREboz~ z1eD)|+?}aT-FDB@vqT7e@>WTg&@B8e9_lw6egeuFEp(AN9n^HVF+chdZ|G*(t=AU=v*oP4xf|IFToWEO>H^P7v6)qng85? z;zdwW37D362w1?(e%Tky;1{ezG}|sy0^7sGr`IHN&OH0P_FBin6T1_Jd_rlf5gjoT8oRK);mRaN zv<&J*ov|Jh{Pdb`4KoQ=yjMJi=!KK1)52s*4q0N^`R@){Vf}#S)(Yg4)>@(~NJN{^w5`wL%6$M9S#~-5N|sUgdLMRxkU748zS(Uy*Fphm=;Z zIXaL1BqaUy-R=4h=F81U_X`L;{PaszH8rzVjCdZ@g#_;_uSiDc>;{02;S(H|lSF1Y zqJ%_BOj-s)Lr?RJzW3XgQ$jFSadOVBEFe^fA@9eO;fT;4z*-|hjei5NNB>k|lN6pE zmOcaAIxJtAj|upMotKS9PXYdk9LiKml}DJ++IxbrXo#DvFJVm={WDe0sLcxHcgA);x|Ui``Wv1CO$8Y4eOV7M`OCDwWI)c zC}~)|_Xj5*F@~cJmjU2fgy{og=G>1p$vTzOhnRFAVq8HSkR3e>+6@)fsB9`Onkh`d zvEzDi!4&N*lF6Uj&t}x}JT#~m>X%dFOSw&lKMid6M{MYyHzH1RrQM%QT5eBOsx@lp&u zPob36s`|3rZuJ+gnelkZYASt1D0Yr0>Yb``uU-v@$UaYDv4lL!lwhnodl74Xerg;s z!f4Cxy74552i&rFo91b&p~rODg?J+4we*2ADJ#3y1>9oD87%vL=ZhB#X%{}G=OjeO z_4LmKIo=AFn}SN`F-f0LgCi}^CO=&JpUYU7CXxO4W+I^3*x`Oyp{wb2HX-9o4r@QD zbXh_43AZD@!FMa);5hACx-}Zo3RykRqu6KWBCpto;UUlUwAGuKPsNY@EOYod)rwDd zunAYuJrH}zf+jC7KSs{n;AkgBnfZ-mWMC@08fx`hvh>8Erpb1C5krtL?|0xTwxlZ2MJoRb-OI?Om)V`C52N| z%InpCt-1K!mjg}&UHm4|QX9Odqi`|w^eC$HINt87EHm0fXsCHGm`xLp#6DPCu-}C1 zd=2x_^Xefx1)K=D-csH)K+jqW56h$}ji~9^H3^VXS}d0fi~P|nr-rKZ>Bf^K#f%#$ zbZ4p5Wh(PK8q&1?oW%YiaP`;o$B+297uVAP+hpqjsd7x+t3_@uG0wCfGJAr#b1$cq zUkJstC;L$kJ?|Hg*DL_e$+j0fvtIKb5ihM4&AD%mtLk`Kl_D|20&jcf&i)6^(9gUa zJmx~7GrBmb7HLJhi>dY_Y;#YH5rHAaY8x3IgHYT3Y?P#H;zbiOI<}ur7Y{;t@wwMXo{U+b_r%51+*KPbTl(v#)Tk>PbEL?ZJ08(tTr(4ZCi z;%OfSIL@)MB+cM5&f9y=8KeiaGu-4BK85J)InHBPu%(lXv(OyHZHYRl=OmRYh7fCN zh^J20x`_cmn29iar~asPkU(OZt!sT{h3hloe%^C?=C{r`(qY)(r^{Q2b4f)iUzNps z*xOO(O>m<9?uA+=CZ>&KaI2FsKw^tO&YVQ83$^1R@ONEZS}ZboSto~36&GS=paQ4u zKEEPz;DzfcNi89kAlJj8zfvtkTD?tsOJkb)x)Lz%kWXnP zLqn0ugRq?XLnw^x>^XTj{-k({gnRV)?IOh9I-d0HmPjwYr|FJqR{?tawSc{i?;ZTJ9|f0SHxrvm_t2P z51F|oBeum9;@L5Lw+tT~*M>k+@at6{^AGiEz@NV)$LL1Eyq4&KH2Lr zpCqVT*dwmw*#4vqS;8jag|H|ehuLmK`YdXZY$>2YMmfet6aLsV zpA*?Mm|zhmOzG%)UeBnkj!ftZmP+YS0JGe~6uW4&i}2=ZBD|0sKsi7{x|GtMNZfug z*eb)No!)OSJANG$G5dsufGzf8fhPi=#*6qf`ltBvH4dmd{ZUPaU%ZA(TdM}ndIIXI ziCpOE%N%M9C=B);@z1|qz@+CO9REdH0F)B|Tk+rL{~tPHz1n$~xTuhblE1%Q76 zEQP7T<^|Z$C*$MoDDWG=QbJq~xdrmU5PKJEQX7LIdn@TFuBve3r0T@0C)SJ=q#N zW%Faw({Izt7FImmse;AT+5c0+mB&NbeR2CPm9eWap<&2tUmI(SVq$30%UBY^*lQ5k zvh|XTeJhM+=8YQL5D^(MS*9^5V<}s&eM^dH;r9%m-}CSDx%b?A&iCBUz2|(-0cXE_ zNO%ZuYuF`tx{CclU|2$JU`25IWejOo)94hkCp4=-q7Om-V~kMOAUYgwTquzZYs<~_ z6MZ99C91atWg(v~OPLd(JwqL#H&H`oC>t{9=DQynSC;ZqT?E;C@37R~#(CqYIZtPb z4AKq_pn|PGpG9`sDqmauZYkK>=9`hbYr~oiqM*PlaTO6OPR(qb2YNVtFG&;^L4q*#XA0 z8L3xa-vMf7CdIYe_wes#)bupoDRXZz4_RZqHfxn{^MinQUeHsqzl&9^x8KbT$J&w2 zOK$SwoykJ86U73@&06=_OOoP&@sNJaM5s%3Er#ndP0`mALf~ki((Hg$m><pzp~Losm#>J`ZSz4TH-uvIz*O|d zN~sC*jef3QiYgz9+>rzMXIZqSg1K(sxjSa||L5+6UnvkxLh*%}M?cB%d>|#ux2Cj@ zg7gb4NPlf2H1mg{m^99A|xBZ=^k`0&Z=mB z1~j9|!!`#;#pC2yt9(4a-RiV->qUovFLohB%g?$3{_OOiH0N_VG)Qt?w!NFN>2N%% zEzfr;J@AdD(S?>O+M7B0IU{$iT1bSi zQzw(uaqXx^ezDJ>T<@;-OJea{7eMLI{yaQ$JEg1IM8&2)qdR(Ei1`+zpPBeCwy-mm z3bIYpp&v}2J_OUDY{X)@!}nfw(1~&x4ho9b-pIo{TDnLE=A=oj^n0-gp8feU|1Vkn;;`(cc$h6FYoRQTM5u`g%c?lg_Eb=0bOGNITd9FK0nayER9A? zVOE^A?^FNFHjR4v-9zlKX;SbUO0SsdwD!$)k>P6s@ zONWePclD9Qk3!I-4aZM?wA}-BE@$F2XjMziV+njYuOi?z7AH{(l#=g+h3uH&Cxf;z zZYzJDXc3czzpDLq%^=&@qgD(BJADFj#bcX+i1_?f(y^vX%%F`>6Innb?A1S>#CeZU z9c%k^1COqXIB^*XTPBv+9tcy*Ka3cjFR&CSPY;M>q=zl=4^Tu=&zO9C)spnkjukFn z{`u_sUCROdorK>}m9z;=VP~3+@&(uTg2fd@yC16(Zgrv?7p45y(G=vhg5|GxQ6U*y z>R50n-oYSXP92_u2v+fbQYZiv>Tq}@ESh-9gs&4QY1A5lK)umxF1|mN`}dLGbMK1q zmjtzY&Yd?iteuaw+pFjG6FnjJV1?78K~3A%tMTXuc7x&pp52TxR%st4`ow9n^zDH= zc-%m;7%KNW)6Ywjmo8*#9MoB&qEvi{&?awoPV7 zozI88*e*NGzFwkqOt+tZu*OpvmhlcKEyXttC1a)-Wh$ovOJY{pbd%J=$cioY6oGc6`ZLjB+Er z*S+5;`6Cc;OEy67dCAv%$)Cois9qW?e2p8jqpOp%8LFY-*fcFQz}Z7NGBQ@#u&=-WNB=a@l;MgoYGq@dFX zmtdQuGJ9qZ=nw+{))^|A5VYxep7@-W$Q?EG9b$+9%p0uo5KhvMJ9Byn)RW?)|so;J4zTimGBcLrYz=$L_-pp%o4#t zKLV)1!6ykrGo={82f+xSpgyW#nAt?2pg{eg!u?!82J00a&dF4Ac&Nxkg6x>c@KiF>T>Jj=O9Qh3LW->AUtv`D6L3HW9HWt#O$|LD}n;m z99?p|^B&NgLsHKi$?>=r@+$Nc^&2VroynSP5S#|9kcS{8UgsIn&Ap?BHbI zh{dN)Tn*L?WF2uS7YYm!n2NdR-U|j2(QQy1OphFfV0KD1S{MkI3FSl@8PEUR>p!az z5y517e=pj)N9X71$T07P8p0qYr=$oXl2r*nQ+g(c$!j+>&X!bvCI@vMc{wm6F8eo& z7#5=-yThK>fAsy#;Dfv)9~5OxI41@Rk|N@SVEytzTV6+$DkH9A zseaVZf|lV--f-K&jVa)vs=Jts(((oV_mhA_l8E|1n7p4vU~XftiB$5xkzF9n3aswy z??C&*NCI0vvso2sh%H}QbKfgiS%YVga@hpcYGE&+eZ?jQK2MnGR`Wy#ygz8^dVQA4 zSPj6h;sqEaB(8X7+*c;Tz()WT9U&68_HPUm1jSLL@aQIB@Q96(=NraXJMQ9J|M0PZ zSJ2?Pt{E#n8lVyPw?pvrywSW!-#Xr>+{r^K$W8ah`6Liikquc90Dmw@%w)zHSBda% zR0sUxc5a~1D^EuJN@6Kwy*F(c+ zBmH;l5s^E#ej0Fg@qRg+eJv9c98(?Yx#bAmEKkl*LIg@}Pkj5763g-OKa+ALKHqMKZh!OtN|$?xwv zf`MqBeZ#RP0jzs}BFtU$O=Rr(p`oC`e^V(Jz)&H>7$@Y`BSD*QA%}XkgWwnd1~zbn zGBLlDD&{nSMfC^H#XcYwo^`vycUc3-E~GR_wsC^aR`Rl$!iQ&!Of{R3rDZ=4MWUa9+lc(TIrrIjn3t1#CZvWOvT-9`rHF3ZaB7FfKp7U*9|_~TVn3q9E_Kr0$3e;q|8&?RH0Eas zusUIFtNG)fheB$g*f%^H#EJk7eu_hP5_sm$touM zrVBxb@-YA0m!!>CIFI(Urz@P z%XSTWB4TOf<^EdC1uRh@DAxjDDrz{3L;MS(5$AKFTK$TpGDodu3$NYOy%!ARxDz=Dk+Px_wt$~MTvWF11SI-}|e-VAY zn3Et=IKspS-zZ66pUjE zLil*y(6g{A%>a$5Ss-B0qF=x#&_@0Z!_Fd*?b8XPZ{3Xmlj0XVYryxy*Uv{>|CS3} zh*7S;p??h4cOKf8;|jOU9*Uh;(8YE?bcAx$cf4@rCN+|g>QIFOXf_`bh7++=Vr5eW zv;n;sw7_?cLV@;udi^&YmF<`{7W5(Rl|r3$gq>9=UpOkFfy*L;8lZW;Rm>1Hwv&S* zeaiy7*9b=D$;nmdHCMQOO9)JY^m$qLF@Gx>i46&0nZ$2?QGkpXQS1K5IYSOW^P_m% z0rYxb;{yB-heBr1Wu-yV8DIzAbb^ujz>&cS=7b<9a6yTgHu_$ImRQZ-?n`l|zd6H& zMaU2LweYt*CAjS;rlH}!;@@GgyJ_vfWKXcuSA+nC7f6JR&aI}T0MzvG1kiJIk*}u} zG(;G=eO`*t$JjYJ@qdw!B4c2HdgS3~BktZ1zGHrSKi81R#Wm|{Cqeejeo6SW_ZR}y zR)mj5L08*3%TPrC&l*6f_&mWDGbmMk1kVolQ1wRLQ2_ao$r6Syugvet?*tp#$Y%sO zx}l8u_AqyqeY0nvJIE7&HH&-oibYG?!fjd+K7VAw`-yvsH%&#S|B{z1L+>!eGC+kSSncC{P8F8}j+#&VTjJEUFP6){` zwCq3{a+t-w@9~gC-Qkq0;4!oz#x4b(+25kQxL}xBt>Dp8idg>>nm@!0hx#nL6c#+U zBQau^imFXXRhtKHj7#+8e>i>_y@=PexuNF>ojiNiiwFJQZik1?bg&SUM0L*wU2EqL zor6vSL>`2oi@shc;s)q|sd!NTz8w5UUNmV1dmwmWg%*Ey&!EQXO7AZBb%k%n7*l?0 zLCj0|5|qcND>6%I{QP4I#J|X@egkzu{qYZa2ytBQINT5_gT(`kPjsEwZ1Clxa7s(Za1_cudv0 zQi}mW<9B}jumZ&689mSkFJL`S#ThoEU}qS#!eAt(SrC?8F1+dX@~ zYZcX)zh~PjL|D%)n2u}<3gA9q$aq>c?kUD_GSXo%KDrfkcH&Ii>0xsw!m(#fekSCr zFF0$Lmu*btA&`SFvUifFiU4CEA)03%BxSOA?5BWjz{)ySGU-wRBVHAIJVbCv$O`|2 zbH&-2155#Vr-~!KS{}MypZ8cSt;SIK=KBE=dx%%4fW|P%Ni2*kHrQ6T1>jPL;v*i* zA6m7K!TqN^cF1^QAi3iy2NE%{SeRH$y5V@Sm}1#fh`EPJ|C-6q9&(oMd5#ObRN- zj2GO?lk?*bF2pF97%`>~fKrhF^-*?1M|`V03m*9nRid?#Dg~*3bS`+Yew$M-_bD$Q z3)10Qy32{GSqcc=zV>-_Ks-8o{-Rl&9&ppZE!S0AOUaX%(9K~9(n#}LbQGBps z@YD05`@qP|^4N7st_LowYJxu{vC4TS5Ktul=RPT!EF9(k8^QoqM}|{6{mjQYmvVXfokpTq$7KS_DLyi#uu@MeSWFb}@!}56E0D?7Jpr zIgn#7Xl9wQ#}_BK$)KU3vV&6{oI|3V%VL%R@|fK+oxMEkxvOPYmjhuj@978lCRQ1d z2zkI5z5NIgE6yq}g^6FO+)qLqV=?vALtILPf(;F}rODtv0-(t?t?}vg#PjnXIdz74 z{p0_qv7>z*E4VHQ8#WIG6bBegEX>7`Kb&>n1gm;}#g=;F4x07@lf|b%!LhI86_jZB z6R%XhKa+}J6BzWOxb;)3;r--t_H{^`xQUbfO6{hLNz?q$+nl|DRj%PVY^>#XAq9(O z$-Uv~9MPlV6sw~1Eck#>XG(GW-?`z5*N&mh8b3uC2l~^rQz&EzigA$;Uv<+Vt0 zAFl>Z3z4Uyb0W~w05pewl94guic9ABXPBoQ{IV~Y-G5ve=7k5iaA-;!%Fvog29w%e zq!!vHTN|a>cF_gcG^stYe$BPo)m=o#%OJ0}zF1KW3#4dsv2>NrALsrPt$sQoHqiqx z8{hYjfjMD6Ul30(*=KEw2PZz>CAnf;5~`(r+TF*R9_dRKw{lt~9|{aMm_e5ce|X-9 z70ud3TDW>LTe|gWZhAV~-jWR8hGfhrTg+H0l7wX^{ zH)1Pn_>Ov0BdQFv@w22!4p@Is&;V%=lZ11=*GH(jeB5?Q?t-{ApVLxHu7KTKM$WbD z?~3ie8dnV1KF!Ap*xZWpYg{@X$=tu`NAucGYDWmBSFcBvIaw%fd4S-^FIWq76Wpm| zMrKPXxUQknnbhBg^Y53-Krr_xsB>voF2WrsRQ_HYSsxa^+uWO~Gn=`X$23A&hbx}! z#+Zmf3q)BzV2V-(<}DB~K?*PX9C{q1;|?=mL-@)mFtOA@Ic6T*bTC!^vTB)s^71ay zNhR|sXG`#b?zj(;?2s0)RiN@DJFcVrw=^0gMGl{eXfI0dc{_!Pk+_TpKR?jWPGSKr zPKAse#HxrF=}C>^fUmwU_ZXiN`I1`!A@${L-tW59WrpQl^Jr!si;Vzjo9F%#GHX)WzLdyl8K_S5=4f!>4yzX+m6Tf~_7Q*kOQB{Z~Dhl!DYOx$E zLtAHk&P^UkX#UTMQYHpkOecrfq!~m6=|o!`@voxx3Zx7u$SC5-Ts}CZ;&15We6ZZ@ zt;Nspn7NFA`N*PP?44PwpEVIOu8N^1j5aa*!MD1e-SMC%XefQDpx$>~)>kRVg<4 zk1l6S<2E$d7c>k_gJG=0?M*Sn*>99W4e%CQ?VJ*;@(@3{DCSs-xPG4c$JRH?)k2zZ z+q9D!Wtp_u=mlY3d}a|l7oXxiO5!)tq!RiObY>w^UYFD7+!$Xv+g3KXxkD8+;5C!Y zd{zm(CRw)CD_=8sv~ji5;;n@%`|{~EOopQQ_y>~X(VipQY}%X6thl~*CSIrMkrcTK z1M@LKWG33*E|5utq(pTD)F*@xr3F}Jv#9}=+{~>^PCp`SF`5S2DP>wPzP)R5|`1Y2gbx1b{ZWuPfGh2=+oZL`diAy z!EH65HOA1_yV;VqKW`V=ERO|1o_(p&XwbLgdqr2`Qyb9}#0E>vbcQh1np>Y>hi} z0!>I<&~PQkX*}4Ytb)eB|Gwg3Pe*2Ghu5W=gEZ@v$p;mpW^Z3ACW}O(_B%R3h);v{ z7z(Xw1~hBCh2eRSY+?Os-cV56q09RxeNfYu4{*cWd-`#pCUcuAFks64;=u+N5eOm zOm3S-Zr&f@619aTn0OlHL}%k6fJV!eZ!SkhreYPICy^NYr`KA2_fn%~JF3o_6ywpi zvXy=>`jx*qjl(D~yk-l>QLD{CK6yz|S(;K9z*llvR_A<-kp}kCJ6wkVan;dFB$v?8 zeA!{NXQ3=WPQp>zaAii*>befk9zJNUFhYM;zh}eJs&=@JQiD{xRlVgm;VBDmb9{jC zffjPDN}L)hsdJ%^3EVDMMO~{fgsN6IhdX<@lm5aC_-shPSgOi9V2NxzuhKT2-a7tH zA+l7cyVTo)I4OtCW>HMrf*DmzuhE&(J*rIvIaF*YlKrags>AZa_&8u|l^j)u@}H6p zhkEg4A&{J>r!qcdSpzl#5kituI4O?bW%uTQsC`Me}?!O_hcyXCuX1&!+NZ^U%v z_oxj`_5G8nb;;Fz#xREPG;+7BOzuaMrR2E$N$dBDVwH6lP*ork21*@d#A7o2qbo2k z8$DD)oHYBw2bGT&kvc{bxf*rcv5H3g4ljoYaCjtzOsCubnw$n>b}t}NJ=S=_WOuFd zp)^kpAD2-p%;Ep|^(vZvD5cvZ^V<>gANSp%A6Pev+F&b)(D9^{cyOK;HKGn_DcgE= z;J)J*G;YQjjU}MnJl<@5M|F%B&YH4+lkER-;(s3YBm?Q!Kj)Mw<25my0WnJ2+As*~ zGm0la4>5K#Ho&Kj-Gcvu}MR%=m|mJ1bS6J7|ME0M{7%Cmx@! zg85YpqH-dv+-qx!1}bp_w(Bp;{+b)z#F-~vSkBM(F7TD!*56XJV=eso*Oi69Sd}$v zWT1gDrk9LqYBu$&XNFIjvvljzbGlk&vihP&anz)YOw7?tcyck323^91PjU)z%DyEB zX;PUTX|0i^B3e2rzC;rJx^fo;o_c|}X@I~BqrPaKB@#pgXC+`xy-mCuA*5HW)FVB4 z^U^kDt*mP3F>hsY*(qHErjdh_zgFY~D8ZqbxAHaTyBwRrUg`?P9_`JUEJP zEASF2hkkOEv`~k9BkuW2!{CDrx{Tlgg8;g>I;=u`I^BT{^k0Xw$?V}g>!t1(pV$0ttlw^()JKti$T;G)3w&d|P`=D_1~O7W z(D`3YnOORuLy^8#g=XhJxpCn=xui?V$lSPzt8#p1kIp@J!)1Nsav5s?$K_s?K8X1yoH7E^td%|LJwg>2rG?aBis z*pUFKE%6p>e)XxSIDYqIb$u>_6)^H80^^(XA}nB zN=-{k%ZcgP^o=5$qd6b**$Ol7WnnhSaTc_Kc-Cj6&cAyUv37-KjpZt3yOOPp>Tj(9 zDUtsVYJ~-YR&1s#9Y^Vo z*c%Zw>g=<$VsNiB2+jM<3puTDZoFkUBx>7Q_E}NuyYs#UF`oZE!Mj+tD&H0eFRttz zt_qU}w5V-xutDdMhZc^Q?=#wTws1_jBDmWVK)9xfaCE~wq($srqUpKnpln z6mbLjcAqPf0*U%6b3yUk1{L zg266zYRNJ?(LVWKsx@8i*;LG!nu@eHHpH3V=2cbiMi}@aFbTc}uK?IA-ktL)9AvV`N)_ zDfT#m5Rs@OY*!gQp!kW1ncywxbXW`~88I}2$t-Hl$n(bK$wBLyly}Y4`cD^JI<2fY zTr74=8{ge(kMP?S1)`^(41LH8?07DUHsv1jL9^^oyG##s*Ynl=DVhM(QoB8dzUElV zl;r;f2tiRIf{tSVc{t}W`CJ?6bl`VEj_1yZTe|99h`R8aFpu0RJU%z(Rpwn8|S%s+AoD-l!`=Dc%rW?CP6w% z{!~21n9oH++1Vj%{&Gp{k(73mz7uq?BCxt#yPdWQP%oPMB zQxk$g@4G1CYLuJQI4;nHu|ao?(6~$~mkWev`*L~cp`p_zP?_y4&1;Je)KYIh;5m)~ zHEt_yBujGi8pB3oo}PG9;Ch_edQfpOQB8R5y(8}JZY~WNy}kUg1zi`5%X-PYBupo; zo)@tg!5%lPYHTM}B0f#tv-8bFn!bf}OA&Pt)Hpdx>zCK1xLtTRe|qkQCSkv(J^51z zm~nu`sNW{^w`hJh!dk-^u`VgXs<3|C8;@{yQY;F7^qvIwl?w5oh6HIwO@qFQM*B8# zou{{HAY?7{UpWH=Drm`^B+wea8o0VP_G=|5RQZ0Fae!!?#Agh<#1h8$`4-L`n zK60vc&ES7LsqOoVX)zJHK^ZYvjXa}%J<9B^7$vn8w??CanAaX9x?@E*5K4lU@Gw|c zORUt_rNlD;+wRMu7}QSDw1Ua`W4xf+J$^AOod0Je8?p^5Q2 zZxZvux5;h2-WC~SioE|$7|`PTTWyvF5m#Y7w8n}4>ITLHkINMNeAdu95*i?}E#pyj zaPMv}VOP(KH-gFAr}QHr(VdhI&1ALbo+nwW@pS&E!D3c_fj!HJUB{pm`oY-d5|?Z= z(p-qzl{2^2aHyhwAiobE<&}c2_;Bs9&5a?;u5-Klr<@czW`iA{>F7+JXR}T^95&C* zuKA&h*Gt1IP8VRBItFN!M-W428+psk8NP3iN`TE)(;`b5P??BiK`nQ3+bC{&G81ar z@sXr>tVA5Mnte~Cc)g(N3H(ZSO;W5i>1uaOA)_(#VVcTAY4+3fDj$IMn~b?0!*-ejd{;-#fYAK;%MO)oqFc*5gvbfgWQ^i;Kj3 ze16HAUoy?DZTu55%=5wO#_s!^4*-XB-;rpQO_&0n!E1vp57Q2OS8pkm3{)axHP7C^ zH$I}f^A!g_$#2ngu{`Q);uD61bP%1?)fu8R$saWLqN1>y(3H#PMMwTr43#g4Uu{Nn zx}%4UOh|fyAY(~Yi1Vc8z&|@{R(gL6*!h?;or^X`gA*K8aBiOHh?E)7Wtx#8Eug9y zj*r$ppXiK%e6b(kA=DK*td{L$6w{)n>-K;oV_nsp=B|zPfzjSz?cg9s{6PWk8n6~K z%I2khYwNt)zl1^Zo)=kR`!`u2>-U@DvoaF(;W_$Ht;hDiX81+!JK*NEl}vVNOGOQv zs^(h?_k6?6`!Oc1DmiMhL_meWT*liNbx}3I(zHWF zbA#ICwT&c6)_bpY#zhc=BSlRcql&bn2n*}1E2cIk&r0Uecb_-J##|39xA8RRLzrOl z4fHUU7{3|AvfauM_V4D97+MNM#fkBl(HF`~1P^5t?L*Z~=!94E>bPK7duKNUZxzj{ zmtKEKy8!_x1)^$cVXV)p|5VwD>6xxb`^TV18korj+YIjNyKQ?A5DkjEBXv_N(?;p{p{NhkzpuL{rCd zA$JH%Ig7eWna)uI(N5HG7nx~yJH;TPJoNS88;$5)lwHF+djWDA?yCP0r9zR$ld&kw zt_oH&bUEo$em=6|7>ou`0cX)OqEvWcjR_-JQP&a7J9b17=MbGkGREI@;E15M>ggnY z6vst}pR8DGQQB*&8^n=zHFsXXl_cH`@b6FS)KE=omB(|%XIOtmX4gSIo6=jvnLfL% zX!XBirloJfMx|9z> z&kV&`Gx2=eBm8t+16CAcO=g*Sb_5-sC#XIhvtK3@a)Ls=r0hqD(eUnol-wwJ+ix~e z&rAUqhQr}e&Va@KQ?>sanR zfpp{2%Rvf^+|{0G^G0SrYJhyuOmT{`A@k&&W7iaNqsd}^eeTaxg5d!35otik&HYKtMP4a(Q|W%w6@TpWZd{~Fui+$B5lU^1FAGTK8+eIZ?l*t^ohScPf8{_ z3pz%_KlAfM8Zk+dk0~V)Ib{FL5rm5s;?0VW7FlZA4aYUn<}^O;m^zgw%#_oN-H_Ra z3bXc2u^ewEa~VU5)oj`VsW#QV1+ONWR@W2dT!Dbi`l)s<)2q&-FK^GY0r_8j#`0gL zMHKDO>yy+n*b3cMzTce<;wKA4$|qe&X)gUtkUNo%k;;p~jX^bt_Y2~hiJkFKos9k)BJU}WNsXK0Dd z`8HG60%D2e{vl7$aszjGaed8;J^6!XhvcDziU~gvL(nYbU2+vC7-3+Nvfpozb1{J< zo0+_KYjO|$3n)5cQxM&W7IA>z#Jr)X$I2)%90vt-5^G2j9nZlp; zgb=kpJ7}1vlGaRtlLst6n5)J%+ORr@84KHM;4EeXf@ifF7d#`qjl6s%;NWv-#w(P1iXP;lCv->hj?y zg}UiHHZ0c{{?mxUx;g3Nw_u(Wz6s03W3q6^H!l7yP!uJpAJ!vher~M8rf;aj1|U_G z9Q%5RoFAsyHeCE3IX(n>M-)2(3Xd4xz)dF~9OH*rqO4)KzKKD#wh>cTclWff5@R~m zVFYD9WBlE#nRmGkf*dHo>(IbvCm>Lj=u}i-$mt^eQ&gn>_w=7FgmAf&z>r={CLH*j8Kq9-bhxG0ju{ur*rsV5-=CrQM|ijx1H?Ed?K5i zbdig*6T!oDE2oVkunW;otkK@JL*$eW5iblxb4?ZryF0Pvzj=4lhipa#NEA|JAH{aNiuu41gM^nAHNV-@fcZv{gF?T5vp?vYa&ijx_7i3=wANXBh)$#P`+=i6y%du*$2Ys1^GvL18L}2T|(3WcL z-hHdD)u#RDGFlLug>W-m`=K}lCN>r1Dlt#iXDmK-eaL|@a&KzIPofI0wQ|iD`jy{g zGJ-q+K4gI9(5e|lMjjtFtC8=3pvle6O;SODPzSQI-H8G@JnUqiKrKu9JD^r?clFE9 z1G8+YE^gyo#xHIP5ES^JeQqOAqot%CaDGDa9etcEe+PwNC5U3bWke?==V0V<%H&7c zl5OE8U^_TShE*O-S7<@pHFIt~W~td*QXNi72@B8D^T(LlvmDR;S{B%+LKGrfkD;9y z*4GhEyui^$?JFW*;o%kSMmGwv<3$vG+)%Qu&pp z4ez_tdHpE8dqlW6DHkc)i4-6}7x1E(?f{!8veeES7q0Kx;vajN4`&O17Sb?m1Z+i% z&gM;yL_bR!7X-9NFd?@6t4Y)j&sk*K9X0ZMNQe1ziHu#nGmWU)aKU4DKPrX)i-qQb zy+MLq?vgmWcuQMb$yx4~fn!cTZ*=qFA>&KAxm!MxkU^B;2Oj*CE2?rCQ(qF}EC$yn z4p8T@m&bb?yNl$$SC{?f{y3h9IVJyA#LlWOv5SL=a(}Tc(--1x35VyI0a8o$Zv_af zZt+PG$xbNR7%|De9pR-l2&Q zlI`aU1}+H?i0g}I#0ag+f7w%V<6P&|OPG)@^I8}{Kh(nhh*98=Mj+P~>#WXo#(z^| z2oFAVc7ykHk|Se}%#Lz%@NxoIW~Vw&8<@Z;n7B)gQ?qK#tD zoi0we%4?V#_1gT%q=-0|tdB;iKYQGW`%TtE2mG9+-X6p)w#~7F9c%jJUev&b@ z0gx3S)$z$3{;T(sffNMNc9GGD_v(RZXqR;B$&~CAe$jNi*Jw8;?g!_NArRf2_q&jq z-C!qn*#648zgqKPouI&1;-*JmJ$+20s3icID55q%H^OO-gbs)2Wxc`kNTFeK9GhxU zCQUDvK)?%L>yM*mbodOA0l@|ZJusjoMV6HNPdel&IIyZ2(*`wNXy3qbX3U!}u!iz*wI66X5r*ArE!<=)&=*p_{P>{PYSPkCAvap5 zb$1vntDsdp7_ca*S&0l?nmn%B2*sAl$RIx&s2XH59FmY@fS_&R`$Y^$jktSfjnfNW*pC#o=)=hkMlSDlgBI@; z{ZXt-ZaOr1$A2r6x%vjUSCZrg`!jvt*C;(po_7@fmkxr@bN+~Yg$REsxx8qtW_#lr z%;cnQM^NJ)J@@Tm8yesM6mT{hCXsbXJb3e8(gTiS9w&J1M7mY#X%znuaXED|R0${j zWmk6I_F3_e<?(Yt)Jrh8NIgElK27_`=V-nl4h zZn?3Rwr3-Z8VI8(HW!;jw%gJQF8IsA$|PX>i5NSXu9QEFlqBV68@eg-ZcP7<!s3F+KfC^f?JM$FPOa78G&9)Jk;dgReR0LWyzvsxuIM1f9F zv0Lh_F(aQ4yYXED6pHfz>#TuuIt=MY0x=y8tn9PLMm?tkiSdc8BAtBt2_>#Z3013O zQ;TOKPeYRBuh{Jjx&wJ#3gzTe`!sR+7hv8o7l36D%K=L8TwF9FEA|}ldB6{=o9Cl~ zz0RY2ILUWDHSL^s7%MF;Ng0`xzR>WHq#4FQ5gwV?0FYzWiGmNp`Oa%_O+MG-gKSG~y`c*=^>H{^9!!~Idok+w|} zHLAoo1Z&kVR)P`|c-9klX&9&cWgro5x=f+SlFn!le|mZ)p)cH9p-hUJ#I{e0@ll@R^D^gZh}5>vAFEcQZlrgqj(_Z+h)t^r{qQdK z$I{F%8tHEzs)2+L=SE6uOtTSj{b8xNz66v_B7YDpL4Ywoj%o5#Zj&r3 z-H6dxwW3V#G5W=;Mub8oKPm1dU%Pfv(iqMr1SKssY~$YS)|yU&|4P0QHv=a&G7{zz zBRs#PPbfdwqe}5LmdbK56=1 zF)0l{0c~c8auJpi-`TijJS$*Bt0{chVoWtktlIc@4&4Wy>`V5WOyTD|r(9LLXMUJa znh%0k$r%s78&t&YXj0&Iq~AsBGBPeMcf7XZ^1rN^X%&cq?oE15O&Nbl=~!zNTC@0%A|qjqBb-JU_w-gW zO!1H}-h7WN*eLvuF-_m0+LPMCwAvVnL{PJ6NiOUURvVkXbWbsTEfyTd`Bo*mLo-^d z$&_5;z*ES;A8c2X-2FAUIsp>Yc?dDZ*%904sR%v)2Sm#FseG$K@WJ=Wc8FOtmH{V| zV&Bu3nfn;zx$y|U)0J5NsJxeE`egi1-n8e%ca>?WG7K(b z%Ebc;lYd~|Urg`>tJr{Aa^hcR)4xw*sHlnA#23REh*J(QWvAU?t>F_-(yR*^-NSpS1+i_hP5JcOJPswO+y$ zgS#&#Z*J3p2<;^Tmp@J2;AR?7Frkk5v_5y$y7zmFE>rU*k<{H-PoW7oj0j4CYzg7} zAv3(N9?bFaJF=Hn1351>s{U_@I$I>zC0kmL3Myhjinw^<2w5blw3PsV^I}#;kqW%J z1DtwRj9wye?@3OXx0e0eC{iyof?7vgS~odzux_-jX;M*?Vstp@s+-M497tuXsxpG@ zNcm=3m#Bd4PCp}7gVD^kaZ-ahu3pP-UkDqv|A7JOAMz zmoi^j0i5K)#KsENJUG?P;sI=bJb2=p%Eb7+Li4e_SF)i12N6iOCvYrrpJa$lUHHFPM3n=MB)Ac*$M4ybZfw zu@$t?Qmbzp7~j;M^q=?}4s;+&xT)2QPAhAqRs5@19Npx5L=Vs*8B z!=0>Di_U&xV5PHS1FQ^I-p@xUedku?ym7(;cDYKOo`fnpeaP3_m%~W6JTxWeVPJsP z0dZaV4=))A^AsDX5KxW(dO(UOL8sFAl6EzL-$((x2cRXYm4GP3qrzl9JSyTeNuLlJ z+kmR`(QG3jn3XT1OurtD&X+;M-oC_9zN+O|7u`MQWp_WFl$4NE)u z?*$}Zp~54B;8&4VA#TpfeO#%buNkZ+E{>`mg#NYjE8-|!*?3rKhR@sVtJ800J_r&d zUA0i3W%*$;^K$S6YPD5~IGTy4_9NLXbH};M>Acm)L~D=2i0HtwD>aj+iB(AAD5gTZQUGhEW5h?cDnso%IZcQUhK;3>QZYN+err2id|y7 zOEMweaB+t-8pK>`T8vSOeVlB$-YApN9ak;H-@ck5Xzm%m<-)YR4nx9#kOQys#l)13 zo^U@O(RFoy@o^d6n}|KMEtKYl;+i9-!^}EkZ(J@pw)F*HXSdvM7HO4mT)S>a35<~6 zf3a%=t$(-dv5Wz8j|j@mfYrwqkB(;owjW7g zvJJc?1*F0QVLHgRZ`BvgWR!}&eBOy*8ut5}me{J4zfYwf`#m!1^B9MrEXXj*u@=V+k72+`|`>em!edj30pjZ%Y1hTHAeO7M}{jOr4_ zg*ax!?|Wiflf%X+{34lC~aLUaB+iLxNUn8HbWglN-e>%kW<)7c1q6&az+ zzPiVCzDg3S;WH{me!9RF)OFR1R}k>gc?T#_yd9@3DD3Q9{bu_28=@HhLNUWw*r%bv zRiAT6+kJn!gY#EdWsEPf1%7=(w)xU#S*ypl>7w^ky{E=)@tTFNtR}CMne9VPtlGL; z5#Ml>Vu_!Cih;qI~=AVdOuc+Rq~&#OHPIxqYR ziKBpOHN=KEWDE#wP3AONeQr8XDDEvs#t~77j_s*vI73O>Ro$apy^Z&^g!53aBotP#CTB>oa-{k5>?q~7!5Kw!Hd z3Tb;)PRrgcnR;h1c7)H1rJ|XLziXR6)YYSS<97SkoROy~zHx+k7UcH}#F`O-`v`$I zqp37b-zO&ZI^B~d1E7e8_Ckc|H|^(1^IoS{{XKaw8@E+wkVy3>5Jq2KY*ht54LSr^ zwOZZ-qqtThC?cFm7B&9xwy@Fm;~ zKodHBuTErNUPNkRs`V*?#`$zhf(%0VF%?QCyKpnf!+<~FBo}Tn17;7U_e+3Yh^=@; zjq)$Y7nFn=Dcwg{HpQh)bkA}u-9T0cLDLEq#W_eZD@_Z3Ut2ZQUb?>Dp-4k=zon$o zfk!JtcAHt!x;>n>^6FS!<|aGR$J}#k>b;m&J6wy3(R6nuV!eyM{p-%sp;7`;piARj z44v8H{3%W$Z)H}1?Z8v@n?Wf)j=oxWZ@C<19{9TZjHWaVDv?AjNji#|{$&Qotj;Z4 zZoMxqtpn7imzmFgB@f7d{4W4Xz>{6W{-PEl943V)shgT+UrRLk!Wg3;NQ^^tV9`4n zJLW}dyK)7okdbg`w=u}RC>cr2YnH{Y4q%t@+$ryytfz(*;O6a;#B@X+9l6^r^&uRD z%XkAgvE1Ga_fD|`8-I&}j~|<8l@u8nDJYtWqzgTzmgWdAp)5B+9#7l&jc6U3gCT*n(b85J7r4wcUhDG`RXyK7O+o~1O>gA~$ zv1iJ`vc;dzJW%z=e+y%BuBECLW_PJn=VGC+szzAVtg-K}_B3*JD$77r>XG?mytAwe^k$PL<)&kVL*;Uwxh)PGgI(dvQZoIEN@9cBRw2VCFi*i37 z-Al?2(B<>2g?xM{y%EGYj5P%z{gE`=LG^z};?c0<#xdcRDWJ!lOPTQenyxlz-+azD z1g0x%^zOOBaLt267zaVPFPri$nlGrPu9K!zXO?KZ!vBZ7w`z!UTed)JpHz3)9|uXTRGedB>|O4S%OYtE8pqcgYh zXv+tGb8oI)m}dr87UX1);NBxCUUPN&9)G`1BOe>r4#kF4CrNh$5w$fHT4w0~3gnHg z{Yt>M*m=J|GOXJCpoxr3H%r5US121*2v`zCA-!$$IC;?Q7(F@2rY{;L$8VX7#5t4G zPjM58ftqp~Nr1`wc~Prh0GoB61VLXGn9l_{9gQ@*KFUzzi?cCvp)qNkTmzMXpeC=G z>_tvmW2{CwSsN+An^rH?1uZvQ{kBB@^WQSM8tfZx+MmvuqH(ugs2=Sdmsb+F!@Nwm7y&Iq%rfu=WDjw{c_l%9%`l8-`W(m>vy zQ0Oinb}Xn^k9mn*%RDhS@nS%flkiLxX8ZY5i(O!#X+}MlJfKv z@Ch}VV%$;~i5ProOZDGMP38dpHlJf>6T-G?@sZ|IwC(B>&CsHGEt6-=n@tY z_LpLeMn^sY<;I21i1689!f4onKYHxvB(n0n955X>LnGaHU7fRhx;R8jnG+~VIW{`6 z@6V>Ec>58tj@LdrADrZcd4eXd8>^3?{K@reQ{x`t0IF{4;Nv@y4k5G|#iDLyBWyB* zFYZJE88P2I^kF8K#}je?iqV@YQ+{4nEGbwsuj*&5bPwM;8ZzHEsg@1D?6qanqWYnl z-+YPa4S&ZaTRvN>-Z(LC`-3hrmG0p?amEd7eW)`i(ps~m=xw3r_Z5iVCnskf_`*3QfUawr^z(4C9%CRkby3fA z%HG~2F>bNr!78{98EFhZ**WJYVy*j0KaYeV@tRn?mx6ysbtgd)msCeotAy^RM3lbN zS*0^#;Ud>N{vF@8t-Qmogw0)azJ%vtabY?XhQ+h;p61wG5A*k)oW9{)joZn)>{r6h z!s@qq>MMbjaeVbP-5(|@WNXyJGn)Ouceiqwjv|x-q;ePW&A#_QY53SjFW$ny}h1Bh#r8G z(G7kY({SlcL6E8P3$J_(nhDtt3i5Xe2oU$7Angv%Yv$^WJ$B~{mw zUkr?WpG_7510&_lfJ)bn2zi71*_1QqQN{hUtZbs(#_>?YaKp&n{+Q$LrtNq5&F>l| z^7)&0+9i>+~8)q$5#S>jOTLO?HJ?9kEa`5hOTRWR`_sfW-&L! z0zTna9j9OqJh}M|@&KlxXS=@X1gf@zowJIWM7HJl)h+Z8QB#4M_~~;@E4=Dybv}(cfQ38Fj+DN|VZnwG~NSUR8~Fds#wLq{Z?xx+BcRvmEEoF`|5gDyzK5XKK;r z3{Daz1vC=X^p=tf5kgmqfzVA1{N{scFsE)0UBrC7xRl^&nE2K&oh`RMjgsh#{x~iZ z;yX>=t4TJOEw$Ak1-a}C%<7Fgs#pn!ETwMU@^$0f>dE>#rv?bOBg#L;K}|&5pecO# z1e0NwV~;o3n8$tCb`Bm5h-i@6eiHs3Tv>n}8~W+J2N%*$e&YC~Ei6Q-{l1ltV+O8} zOYmk6Y}jec>1!GO^H;s*SDQ%(y~3V0ZOTOG4)P8d+qITm+1bra%D27wSU9wuXnYM6 zn6j;!DS4-{tnX_z#@4=U&%SHFem2lE@XnzzanlyM#i;fJ^}=5a?zerf5&r5jJ*@?g z*G3Ru)21lFr{{#gzy{MXFQV%FZyNzy)P6ajp9jXead5`H4n+MDoI&Z8^eCeX6&~FqTQJT z=(3J8+~Mcinw2}Ps6_eVWp!Nm-%>76HDb1y#Rs*_F}6EQ={W@g(k?r#@=^t`YR!iD z%&(RH{hn=9ivxLr`V2-vWOWGO)KiReKzgfA61T?SM@h$8h^`%{?{7 z`&#d?>5v#3JMvSF$~>RqeoYs-?9;;(d<<7`apT_P_f$5O^ zeXFq;Q>%cRnqv#;9>fQSLoP}rAc?4WZHN5qWCM6rwpeXs+)U1`heR?oA7dFF&`yrl z^Yjk3_xr0@$Xd7(y)uFw{I4?nYj~jp@L;?$0DRov_y@Aj1XWpD!S(*PnGRU|i^be2 zd+*O=%yJAHTH4=vi`#-`(+_R1dCj5rl(1RzXx4kT zh`DcSJRqtJiDBX_lt54EWeWWq-)5Agnz0X}p^N*bOWR5?;_o+>nuCO$tZ5y`R8nP| z(9c(>{8^2LlhJB{3I7U?NWjw32C@5JK$;uq*Yoxiv_C`JxL$e$pdV~a>18AvR!G^G zUV=;)Sby+*MP$*bN3`A0=5taewdDS4i-@b=S4qGsTVGZIY1lMfOA(!?FeRC2+dNBZ$dgH% z$L9N)!adCIp86+f{?Q5<_c8G>`3^N*x~D{T7QdMV-$8I^ye=jEud+;N8x*D!uSp!8 zG2SunOG;fZv-sz74+qnp!JTT!R+AK)(mzpsACK=TWBG5eQc+>U_F{P5c_K2Mm&PE~ zm$uoE7ou@ULpF_yXVn^B@B^InlJjkU42x12x(yEBrek#Wm`fIqd%eIna9yd#n%II2 z433!4ClG_h<2uWes0TDiVX&U3ZFs3?ns~obJydWhg9~v=Nh`IKY%v51g=eJ)dPybZSHU%%qeKx<4slf4wd$>Hu60CgC8_pVK`I zy}rJLn*1)bxq;_CzlHsh+GwYO3Jl@-!q&?7g*d?yZ{Gq0O@?;miJcCc=AuO9+(}`* zZN96xx-#)KeYQsV_WjVYxByZ<*UN+ib#H_xgf{?}AlZ-ebJzi~B zd`2S-3EG=}iZ7nreLu5X7+;T9uHu|CR+;iwSt!nf+?U3)e;0=K>hp%R0&c6tDy!XaGUVl%4`~t{oe1tljVO1ZG<9r6Kk4p@){sVV+L3O%_-#*l*lQtGNo1 zk1|!CUB_sKU2D6dD`rB0>rJ#vZOBY1H$KyPJvzDJY{oT|Y3Yufyb)prRH)+oq_>( zg2wp&)qMPM4ZyJfXOhOg@4u5Ye3a>EytV-St*fpx;0ao^$nJX_%(mm0N2T^xmS`E( zo;8w{r11^JhsP_sqja#%kxw}#{NCzn5yk#jpVY|z}1 zm%R-tSKws=iXgbnqViDqlQZ*Q9?OI$nIx8|zwgsWq8_j%)-^94-RW42$~Z?t;Z4Nd z5*A*6`3arjIMfEoC7ALm3*VHYGiVM#84l-kek#eGmpulo$iJ$3zsOJi&nUm~wB1MRb^3iZL5*PWnB zsvIZ!b*6v9RR$m1T<{0Od;ZJ;C;M&{o%JdT2hK2S+TPSUc$R9^K4|lW4gS@Cjo|ocDSSN_nSG(1a^J2%d*Q zrf1Lkc*_5cPUTi6)k1{AZT+RLT=mkVVQVCF;hTRE45C&ENUH#8N=#Dx(A{g)F&pYd zDLzQ+C)(VJv3Htw)YNIF)nG-kqh=qInn`O?FOwEayy72EA4FI`2{I`LFGF%3Y;0AU zOc#tmf#^SbH88bUAqfwc7ml@ak>0m!JKJZApCBm%<*Pz<^nc1>wdcH!2hf*BwMgAQ z1H!oOyH}P5<~&b2TPy?G;OT2{trUwuCq~fl#|3pJb0rJ*QiETs1jHDTh?wWMA}xrW z^2oF~k}zTlD!9i69{#G#^~Ea3<1st*^upP;ODt)`{qFu0qOIBe#Pg-}Kb5%zDEpJ7 z<#Q#K0nm3A{ieAu415#1beBHb>6SL<@l*?wVK`E9)ErbusHPbJ7$ttkkfUl$1}ATTsU;D#I+CcTOuX7QJ+l5|5S_<)=Ub^`%g;|e1Y(#|>mL$NVja&!Ulrs+ zlL{=A5abdjYS)Iudp_6s>@|q)^)}jJ3fA!sH_r{o5U68io@-<(9kGn3@Ce?Ki?e^@ zd`^IHmQZZT`5L(TgV3nCGF*9SeNY9by$5QmS$xiv`!^kUZFa>@C#BK+yTbU zSU%cPDY5HYuQ>i1pB?Fz(P%tWgoiI43egxk!iaxEFwE7SRBWxNbN=?v8fxaOj8)!* z#%$<6NgYc!CMxFB)hRChI?|z3HVU-b9Ii{>{r2qk`P@A|=XVz4A9E!TZWu|^ni+fe zrJ2xNwA&6aL3)|dp#~WXm_ALr3>tSGi6z?xXvs4A6#ZO(^Z1qBfW^c1@@B1&CV{Kw zG?0i4O=njD;Y%w(#OGt{25RDPuzJ?+^HA*`I6NqciC7QGr$*O)QN2zz?S|(p^bYNj{JCQK7^#e+O8oc zKBprI{F%4Z<{@tmLeIV8IZ!d5R0w4?s>oAh^>nADI2`jjXPZP#(UT?#tV}Q>DvFh723n^%a^;)r6dgm(@9<&D9$6cH}0cD4}%;&xlA6tKy zd0t5NC#9o9yZtl~3PXMdQK_*nIUo(K)j z?7o%Es=2ij{a=28F1HRel_=lJS-sS8ZBag=ukWD|%$SL};8zannLa6KGp0A_L_Mj= z1n}C9y3(f$H2j>!l}dNw?H?%;?_e6epDbmMoc#R-QIQX@pq$vobq!9p#T_=F<`4no z+-5kPrZ=Bn9}gV3^0mVD;+@UG!w-`PxBilQl%~q9=kiI<(6GfBaIfuaf>pQOx6zd) zalPSXnEW<%^raFuy0ZbL19>fq4#be;li0A?Y1_C`em{-RBIT)r@9d&Q!@vdid|K3C zF7_>&CMpun(W3HO-_vYdNz8}oeFfGnT;q|b;D%HJvf$J3rJ|Zu&VOmeUM;BuaEN5OfHsvuYSI!am=Xb&b^(I3;$W>`T;3f(>y<*+jS>I710Z#+|& zq9;m{>ODNHvA^*W7AQwGYwBA_uTV-$kKPD^XRInENw@RxioPyczlYSqo;We1ttU55 zr)j|RT=lg5^4OuH-^r*sPRdwnn=M4HxCbw_k=&P*3Oj6erH${32Eq0zK*7ATb)>ET zNKpSaLqh)$r6#H@baC{Sv1cCOP7&GjwrMf+ZAH5}WB=)P9c+0ZTM$#dZbX!ZHc9#S zWClq|RCc5{ZxU1F%(<>IUFZ|X$2$@g6XdwqyH=)L&oX_^S`K{KTh}>NSZBsBb{q;G;jo zw@n$N)WHxbf_i$uxhf!$5wjCOz*Y+O$q`Y1p|QJ1uG#BEj+weAo&7n>X|+sRzWjiH z%Qlbr<{L}jU>~s*%%eA^!-HPukZkcrh;N0F{wE}46#Xs)cYIC9L#G0d3*Ihn3-PL! zdMRl#PKO;N(hZgAQ^~7?^_bIQy{^qWvkiNU;xtKSiQ{ogJA22<*7g8E1x1J+mt!&j z_ndnA!01H333#PipV>F!cUf7R`aV_Kv7xoKHCn4;teK`^F`b6o>V0MuD}vGwZ&RHz zWs|*j(k7#m^4~b>H%=5#?T8KmACHGkD`g%=5}OaPPk6nQ)(bfYD-L$`)ABoD0TT!3 zabp3@wfn0_H!8kUBW=GrJkSWmmu|$vXgyEp+l1lA8|SkJ&XSi>TT4C;8mtjajI=>* z=%e0;uN1c>w;TomGMf*{zjk;YL6OUkyMmJ2JlV;oZczsSIWKd5fOC>1pXZKJY`pX|>Lr;qJt6JO1sye0z%xx3gZ?tdIKVk4hElg^NVyaqdw zMU;)T$Q*{vGKq}>w7|(J=fzT!pBE7w^l7SleRHG36tD@@#jtA zHV`7X2>EUrPvFhz3f$k-mq*{wpY3{JAI4I1c6;WpvBfC@u74TN)cXts4$#XE@J$&9 z_m`&}WM@>3%P$$^-|`E3X;}zQ&)qkj2}dcLKN~+`r~G1JX>Ga^Uu7d{mJh1Np8oCH zyb~=%Dr60^^oS)a+B`pY_cd)48lH)dWX45wfv#JMOEElsZi^?~o+E8Dv#0&rOE&V$^_CUuLj<@UbGirCp-= zrp58V-M;Su!T7Cr6g}--mM5;lN|)5wFNWAxIa7xb>A`|*c)1Ur50RnvWR<(sKG6h_ zy3`#R1}pGn8f|B|`NTxe%bmZJ;_k`b#ZQCk$0{;%1fbGD44lHiF@iwAC2zU@n@)Wn z{{TqhCx#+M8U4)BSSR_MqyH~tp(-)$FpZ_K2_)Po`yoCOAnph`IJ zb#Bd=hb`h>VXxBJW2{wr&}I6BX^6rCn`!yeef@TVUb7YN<<-?((Bi5cq|ubD3s+5~ z%PRdt0)i{N$)5Kk+GLQ@`2C^&LhhmK>S4spsBPSU>GqHD-|O_CQtU*J76U_v436GU zQA0kg>%u3O6at)*^Omt3Uao^7+bUQ<{eiyM${_6ric&IJ^zW1kyEfwYr)j1l9;#wb zXVkUyihcW1HtKsxcm&2j>D(y!TxtX;3d_D0&7jYiE!$mIR-$SkwBHM5Gr7v))d@|i zd$hZabVF<>Fi>;)Mr+0Pgp5QC47)gMfOxWKDm$lfM+QnQ?W=wR&$X)Yc$e6d+ZSVX zw%_2kO@HWj7IgVq0jE$bagNmPJRCQwaCY-L#D0iLaqxP#n;in7OIvr$>R#!E?%w)X zw&?C`G&giwQ`r!nc8XzvV5e=0EPtE0KGT(XMF8B>msMZQ5&KeETmTjm`K~G3c z&?7?xdGdV}cO(cmH_!nfiZM8d+Ec-1X4ta4dpY9gw#cfshU5E?pQa(U^?@9bE%gca5gTG)xcHsLBF@dm?F+oUD0QcI1LOL98j%%1QH%gHd zZ^DlTq{@w@PBGDre5P#X1+kZ5S}Vb#LP6k80*h9K)_J4?3*TM#Z*oaXi&x9>4M^=F z+!1(sgtW!RebaS9MI=*{m_TTBY(O7Jgl07E($~XjS2?2&nYUObw<+3!nakyyDd@t* zGoVd?LMKT3lR5?}ne5p!IKOLcGlr8n(ubl$mm6xM1z~{!Qw&gNam3NPGZg7U5LhfD z@eKuzAs3&as(NRkO9SjD!jmJ3+Mo$89tVdlzbt^Q`sRq?KR+4f$RfTAdW09wp1F5; z_A^uiv%;A^X&*pgy`Shc)iE53p5Fl;RFEZ;3bVGEc9<5jP+*j=h96DZl4Th2ZJk8+ zEuL$Wldm|}yOg|*hSL&qVYH!rP>X+=HeIYXMrpL&h&yF(4_s^WFx?1(5pdteowWH- zw5<|WMe#C^j#avzRUBe}DYfWVzDnzOS`yh%GWgglcWJ6?lb`0`#~uc8+h`JOHpr;F+49DU)->D?O>xu>fm!4>%$ zafc`e*<%H^GNL&Rt&fHqF?3Ere1mi*+Lwq0Z@iEkt;xo$dk03De(^`Y#B#nT6(eRu z%Eppn9^W^wX~LSBF&M>nc0{6)jApQSCaX-gaVAIs$Fu>$^w3%v%Q!Ul4I;%Ki@S~OPJF1 zsct$alU)%s{qRm28QL>kBccjpGU71J1sf0R&bD=}g+(c^0-py)x+@Q;1@wZ1QN%-A z0!gzs4gGEVNI=a2$t1LcUjuf!XpC?>CR$MAx-B-^@xB3-%h1Ja=QQ7l-A6F_RbwmJ z1t6x8caXX8MT2e?<~B~tfOg&IX%D5P5RT%=YK>beR}F@Z^u=#1HI2{)&^f0BkR=WR zVUO5$Y=1l9wxZUBmrs-kw$}UkC+pabC=J$Pr>yyDPmD8m*Xe4RzAHWU@We5|tcjL6 ztwE9H6lAE_s3J$s-64Fn4)-XG)43cY?)<&qQR=OE?}o6^Ov z>-W(Qb}(o5(M>WdnGyF;KKJ;2-hMXc`!-$$RMn$qA$$s^5II=WtgKJG9v8BCRK$!w zthVREdr^_zf553?i{^-cvf;15a*M?8(J;ghK|Fl?ra{X8J!7Ypbo%Sp;A>PMnN-P2B}%;;4UAk_XS|H%PhKy1*WK@QXzVJQHmv#?JA zA+RgpaJ`p)eworj_thyIQoPWW+j4xa){NAC$1i-TS^xbp-Dk|~`Q`^6YJY1P>MA`E zRDkmj@@R4wu;g!c5RAg5-Qs0ZHqZcL_4j5xKwBct?d4wWNTR&_5(juh6oIzsLQt-u z(6S@O5>Ci8oexd>5GwF69CO=kj!eE(gZ7}7;<=kZG4`=uj$3{un}5_S=M9U$Kl`^e z-!286yE3zZV~)CeNJ0`OYv2kGt~mj>~Z8&HApj08_>9;yJ6D@?mSpW zapX82HB~d>fL#Z7&y<1h0#w1QYXhuE8D+_@>YeGcow{#bxPNJ6y%Ba_d!l1*b+8*Oha{922q$JuU6qT zvOS}j6As%$+?i{GkF{0#kZwCGGn01fR6I?)+wb>mhUf3u&LA+AxrM-#qEn4B3D$dt z8=uf`&E ztx7;pexO#GHiOFXB0sZ{AQwZ z&`zQI^8>qkz1Q;xf#c_iHaG3t0K$GpZ`A!KHRfvpzoHNAt2R{O?4Cw)B(m=|p(!ssa_A=R}LJ41jjy#X6-%FF=$f?J%VTYMKv z^{RaIn!Z7ga$&8(V(aJhjZUZ0CNQ4hSk%^Sf7hbBRQXIx+hw+pDI<% z0$0xfMOCE`tiE@jqhMOhi7|joZF3tB-=|I`c<@ikq52E7kF|U}zPePF>%0i;D?7oP zn4v+`vbu5SUOYLBb8xKm?7jof@=5EB$GiboUG;uVj**-_pDIrm|Gocs_Ali&t`|rk zdJdTYruF@T!Rm`4204{0oHwU=c{fMSTLVS&AS?W0j%oF>)2d!=HSpo41*%skmAP(Cu=B#ef3 zVJfRys@V)bx-iveE+KF#uGw~{v_izh+G-6w#zjO6;=7AQg@+}T4 z0D;{;wmVhj*@+r+^0^S?OSC`Buc1NI`WY|5zy4jaSP9bUm1jiSyM8?y69%yf$_yK{ zy=Nv(?Ojr~cUWJNrKa3kfT?$TCbVpZol(cJ74nkt@lJ%)-hdcFvA5&=dFXm6UFVo| zY|a?SZ0jmvrx?o46%nQ{&tivzUg#EILv~CQwHGE$;=J@o-EM{0V_IEPlCqon%q(iG zWhxw-oll96D*5y9;tA88Vcx}Ix-2$-c;4%GraEmIZ}?%@xKloX|^ zP#NWZEdrR(&^Xflv&RP?)(8@Y=`0CW12?4pw~EUe-4eDQma9q-=8Sn9)&zv311oL& zh-UI*Aa2qo8zy6?tk?w&=eyr9Cm;6NxbJgR!+uwL_`v!%&xSS00L?1~=l~iA;x6s= zjoh(-Ez0xImB3RHYD%zqGi;bdKblDbQ|y5AdVX8iyKHD>>zpAeGVB{o9Ko(A--p)l z%sX0rFE8d9(Y2UF*;SaD(cAdtB08*9b#@w#{*JoJ2%<4BJ>QDDR_P_R28+{kmcGC@ zoZwjB8{gkAouc+l_jg|t=q6J zBu0%%gv0{wB=_kD;zJImZrJWpr{@D6n;BL2x7=BgXMlKO8d}_aFzl03QvUqsusQ3{ zAC1KH7!I;=gO2dStuXly8>k-Y{zo-|O)@gEk)x_H-)W$SLv}Vc}SI8IVW5z!B^nr`imJEd1`U^^Y>tdb3}FhhMfm}z7s)9 zS-%54&Yqu8^GN`I>Xu<-gc+F!&>LAz)PGb!8lf`AcjG|ZI41dB- z(G@Ytp$h>g#WPOvxNWkrmU#Dri3R7zKVp z08u<6j>ck>ab;-akCd>m>FKBU=jDXqm9yw-G}EJ7=5II4Ymu@6-dM#EIHI;;8&Hab zRedE`sCu5na4QFz(Ww!q0jHzWaC_TGy;E#UGnAFa>vZNQjkh(OxQ=*VO{vCgv`Z0N z%_EA#ik0V#_j$5Br85_}w2CEk9oh`?VV#W#On-Q?Na2e|?wX0;z2B4NhjtMpk4}r2 zB$8Tfbbl~@sN}NM%jImeP4RJ}fm?-BL~!4>?(tZ~SrL!*z7WXcF1E5arH1pS?l!Vj zu1`$?;O0WXF-So61CjopKc_>x2SN?sju~!f?oDG5h1CQIVfd9FZjTSN1^#AncX;Gy z(W~I}FCR2WYk{=)+8~#47xL-1KIiy>0B~<3BLF=Cbi{oP8&Wdcj|UUv4+U&WM#GRq z2_S+yFkcP%+kCl(gyiE_!6|4|nRCT`#!R=NbeqWn_%+ z&pA`SW(UQr*vnXBBSg-@el`KZwI1PR+-D7Ouz=IZQ%1-ba%sDb%D*B1{GU7e*XBAf zt9gIz{QShS3i466K5y@myieA=7v_?--gT9Bv@cB4`;`f;u$L5 zfbLLj&jJ9~iZ$&~mz4oIhW}qV{rk^z6$y%*h<-Sr^+98O-cildjAlO8`OJFA58PY{ zag;@!@sh_2aT^4aftsDOPGHcYzZk-nrijmA*b0$_&wme?YY1msB&2u$cwp4#E^{Ri zk5g>48}N*UB^7REZ$S;mz`+5Sl(4-zMrgpWjOk6?R^ur8Fugvhj(q(#P}I~XXi)#} zJ70&rMv~&N4ZXXd)m6^vr{$c{vKc^Fk1`U1XnP9aY#BTTJ}5gUj=&qDczmHJj5s~G z!i*c>CM1co$g(T?-|&zK^2dt+4+8rAXcmq+Sq{Yd2N3LK*WwI&>>Io%hR`6lZ!jD& zIzFRme`BWan9Y_-P8W{UmH)PzF#pHgCID$sZjpqh6c71Ui^Gx9iR7aeqDx+Z@!0<+=CO(&OK^*?VnQ)rA_tBz z=9HTX<-d3Y!G|Zo$H5{!5rl8R(RPb%eB_Tt({P`Gi}9i1L=j}y+*r$ao2fP5{rs1N z{u@< z5-$oXAFsx7s`-DpWyR|#&3|H*|fj z|GkXdZYadXQ}=p*ZTPFme9MQGwrbx?s&Xx{kd7HxAB8KZUxDGd5{Srz`ME#^i7$k8 zO9HPh5u^7adBtWlM~y3jq>_l)XGYf8AvHa&joMm&8~@+SrnH2+meh68%>0`u|3i2Q zOe79M5^4Hqjl#ce_1~5=SSzq+aZ}WzT=0L{)BlVB`3zGUfI0|2Gh(9g-@N~~?Ue}H zgd8w@o>OEtN`JF7|Hc<=JVa@duC>}4|L1NzA&5jhzR3wd_yRubamwO2 z(2I>dKQMsuK@yk|vC#JNI8`KWsXQv-!d)eO{YZgI3*wSGS^6wN(~k2&|ESCEE`WI* zS%7C>99Zm4aCJaU{@nJ)>Mz##E(Z(77us%ea zAO)=x29{c55a@LD9L6}dXHYE{1CTEZ0AJdPcKQL)IcNMptdJonl;A!7ktIQIKKf0> zSv0PkUzWLR3%uG;W)aWR4#Opc1H3SP*BnZc#e}URm=VRGT?QbQi~SRgs+Ud}yqDsN z1588Y6o>c5xSrt!HH*+|4-DUCnsU*=SP)!>#(4$nC5d?~G0?H^7w;i+MJ}`cLGfG=ei&b?8%`Uoon)(&9YGalN0pyM9j~ zh2{7i3eKDUe9(y`d&yjC$?EhaoEDnhHmci>sPiR@ld$Y`SpA=@3T8E6>a2Q8i&~DJZGymJpZkE|4`dOsF|d5E8RUi;bAD6pmB8)_h|iz}u(NY7e`!DiKji5r z+gVaqgB}nIfLG*S%{Vj%24Vkk!=3=M;?q=cz_L+I>jEnk`8}{$kR_-MSzIMI01i3Y zR+~P!HhZkOrn?7B5Ogna8psmBc7eNN@sEQU0dCOTMOsxM>x;z#z7rpluno*&)1*g* z#XmT@_!*!Pt~sR-rTO}ntiUayV7R@A=rh0l)y)k&F9kzubA810ntI(%mJ#^=Jp=zoZq(b;esVa#r6`% zuA7f3h?XyCc%$OWc-)R@2l>!DJ^QlG_% zk=d=bJS}nph^uoBmu*lXdxUuB{YwwQ|AFLYgAHuGmUSQbPgV*xrklBH`QpOs?g$l& zyAk|^gZjI<>>LLUUfJjnlWZQp1WXs8Wjhoqf45$9q!*&)K%Vwb<9y83&#jn<+jvfC zwOEUyIE;tH__cYqAVItyocl{M?_u)eN(EgKNXG2b21iIuZ! zJYsY*hlrvM3W?q;1xaAPC&}+?Z(9S4>9Tf1ry$)!j_U&M!Pfs=!TyKKAOTJF(QPiS zY26KDFz$U2)3g)|wpkY>g)bZF z`5`@<>&I*-MvF$su+825NJ5I4n+794EduKf$iyo-2#@Xil>dog5V!U9x$CUM#4PkI zZH!|wH-}I&5Fm!Q`ow#)akO$Ox;Cfl%kcHn-xu&_v`5))^SYgWpR=AtmA;^yLzt1j?=zGm+WbMd)7{^{3B77$J2SA9pK^a5G zj#9NgcJgC{F>VA~5CZ)DYfLD>WDTu;8J8Kv0jwm~E0)Mb@qC@oG=w=KIoafZ3-aIB z7XOD2d*wHGu8#JLfTkRpe_~Crl@gBvZfm<6d?7W~1V#TalSGgtPP5~TU%U3F_zZi} z`lIu1h~;62wQp5OX!+370;8 z)`#2w4r6!53j(BW3IHc@K+}Xdyy=Va7}r1PFK`3))#Qmx`P8mc!KdA?(7CdFVu=&~rfry}(}l@&=l6}! zp$rcr^;?8|N#Oeju6LAls_l8GqYC?5Zy~mdvK)4NPT=N@QC;_QS?v1^SK(PTo>nz< z%BYq{mUU4#@;D~6fr6D{qpxK=ffW>~Qk;as9Fro*koz$d;1VUU;e#(UCO??Z>x$j~ z!?!NVF_M72eZ$qwv5bKALyYc`O)7{*AV(6Y{A6OT;niWnktgdu4hdM?8%nGz%>8Q0 zt-=4Y3-#|_0HuvY3r7uSh&WIHfV)ZfnOHiqeezc=Xui7 zOH&DQ68B-}bBHj)%Wc;wt*<&kc-gDmZQc72<rL>Ncm0FtJBr&hvNfL=e;Qgxv>HzXXS@`4g8sAP^vm^;P_{Ve@R@l?K|qVrMByGG+O}4dYvj5ETn7pr#I1eq9vQdlG8b zOEW3h<$j=?`=PYBcE^2rh+azc&YDwU;twtvc7F~nXC_b{>`b&~=k>O%8%+8UjZrCo zuL(bcl|1Ju*aEY|T9$6eL{UD|b6{sM{$jMP-M|?l`nS59zZ@vu?kAWuy)(CC8<;6n zTb&L59rEJ@O@mT%zUxrgLYl*UHkW5_oZ`-s)Pw9*uZGx)C=5r>^gS&%b*_8QQ+ZA3+k})Kd)0eqtYZ_IXLMTplt7t`W`oF-78&4R7-CJaXJl@ujNaI1u_>8bCX)C! zm0EcSOA*FgMpQ6(P7wq>eG~`XGYA{RB*y)m|6UIT;K+L6S)y{6{n57*YHO&sE`)pP! zC!@v{<{(1@_gBWS<)eo>!|Ra#j@cE+3b{ z;X2=yHG+U2tn%&xPU4RyQlRn=gQyI-HkS>fQGXacHPPfCnA{VCh}5r3oiQ- z>$2sL;`%aYy$7$)r{MG#2?KElf}pTD2Cd=5WZ-CC2ury;fE44I(bx;_7WVHZo)qGD)8{+wubrbA z!eh3!#i7NsjlmjGw2)KyJH2DYWQ?lQmEofbFWUkSGg?y60-+yw7|8$u)C>%bPCK;> zlxa0S5eF7Ve`Llh-`t0;@xSD1P*dN z(lg8nTFEmV9aLTa7ypA*?dIS!PIr)J!kqr!`MZFIN^2$?7|d4YK^eU30Tks|#f<!?|^&SVChB(Q{mV-~j0LJw5SIzdYt=8Eu_%k7m>*;~5 zkFf2JaYW&90i}97`$}zCePJ9Cm@x-wZbcF$DAL6n(fl5`o5dsx-wPBQ`EXinE^97r ztX^$eXCh{@3uA%U8$JywF$e*1cA$)EH#JUDjnUq84P{gFD0ly&Tcz%d0FeBS!5^B5 zz)nW2SIH`CG68!>K5zwc_FBGuE1h2}Mkt92WO{s7xgCv-GF(xkV;TM3j{w$DUSoF@`)Vz7g@8D@^{7!;$*P^U*J5{Q?#KL&_Ghb!|$F2EIr2iy|8+iA>5Kv>Jc zgRJnGewuDD*iQr2c+W5O8CqmOJLGfUjpWPLNiCCs4yAc+5rG@nr{?jlLc7?Aooe8$he&e=SE0 zsiE@HSzg=-m zj7qKaocqR*#u2T;#4|+a7}M8%B=|t)+$%E`@!+-G)hzQ<07b?g1otq^MtF%C_QK~L zoc*d~_8h-Vq<7~eiLTs?bl1Fih*Ceqmi9V6@#<-}l_4sJE-a<3lmwW<+Tc^nW8^4R`~2IxWyqMuwLhzHJ@ z-~oFy7$p~9Nsk+vZm#eAxu!Y$r*qrKP0E)PD*9v`4@@($OjG?1a_l8xhmvmi1;_FO z4Pljh9pTs`DS*`dr-#}Xdw!PEe$&^1rznA>_~z2PytLX*9jsUcWnnvMDeFTpOZ?2H ztha@>X9^IcRCa?l1gl_89Aa8ao{dDB8;IKT$0#O--sx**4|;=Hk_>w5y8a)!-ZCn# zXUiK7?g=iz9fG?<2PXsz?hXm=?(Psgc(5eUxVzK12G`*3uJ0lLJNKEH_s&}Vq3G3| zv#V;?_TMgEvy*q-lz<%+$YLNI>#aS#_Vmzcvlky|9qQxX%tx?%K9y`dm_}+aM^lHP z*|M!(Ym!@EiGBXbly@WuZ;RGQsbZh`r2r|G-lnTx%YLt&qi?w16_t23ZOPR8bc=Gf znN#~Cwq=h$#5k_1-!yOELPofkdaJ3>sU@okWmo_&%wa0>b`9x)-Iq-bk3R72PmZv} zMJ~^ZRs-Qg8p|!jHt`W|ET8QRrb18~a$i4rEuQZyU2P5;ao;FH@#9K6#BPsV$w(yc zUMb&@@vkYTGZDpZx5Em8RVN%kg!mP-1NZ5YZxJY?^u12iNW^maAB-^Pf)pU@?=pBL zi?Bf^9J}wYNCN{fps! zdhR*i?X<2u`?p#Q8ATVPdD%xe{qtX=RGfd{B_TYmWk@udBi3?U(5+6#+Bau

    qaP{fO1$hq`(*Y7&Uset(>B?GRp_!>8BDql>?eZd3ZW>G|~uA zTg0b^s9*;cg`83>un^}%-v8+YtWiXHe_UBYEk>Sck96#lz|~pvn_{3z7cCw=)>m2U&oL3Z62O#P_L7+dK)5; ziA1VV;2Ja!-S-wer{1y+U5ou3Jm&5R%FSLprcOE0Y1Kjfwd(&Z%m$Kq`N=?<#uL6y ztn#ZBHeiU^L37(DEW9%-=c?(UAP>;r+7$A&UO?Z3l_q>KJq?ihL~0&kL=%IwWj(F3Pb@JA!ljSdBq#eJ2EgjbWwg*$T<0W}C`vLr&=xOY?OtvIAsy z=EN=}FQ>XW`X_qFSUO(goB77|f)T+YXih8(`@;NgV_Do7yt1C^D+5a{5B|boO+SdQ z4<>E*>}=h;-SO16JOVE54OqQn3aJZ37Y1rep%EMwiAo5p9z0uv#&Ae-{a06@c-5bS&h1-Vf(&^$f@QXR#t#+^qPe7A5p^YKBkX_HDUB$1XyPoH1?c1J{3%=rO5 z0#Vi8W6Axc8XMc4a>Y>6m8e%K#Ke$#KDb<7xm5W$@!sQQ9RoB#7w}1u!$!SR%Fofo zz9N1hy)<6$e8_~C1fi0cD7)YERFLDP)`YvHgY9p?ruh>iqvGg|9V(%FmEJ!wgr=?3 zVwR3?QaYZ0fa;M_?e>^Pvr0i`9TXpT!*m+csK})R&C2wKie7z|%BnPrm8t8V{EGK* z3ZsZB8GY@xs88fH+4+mrY}@d9JmDvo>+jC%B#&cF0ja9XWT^!XOK)ZxK2Q9nLkrMd z;+NFK!tldnjtt@QeKFol>)Iq}7%Brx%lA{Bp6Dt31ogYJKWzy^XAwE#DN2pQQm1Cn z1F$~yK~~eO*m7rNC`uy@kxcBmbR1vLjJCcO8&d2pTx18`38Df;HpPDD7!T8Tvh`Qj zE5GHc=E|2TR()a>6O5wj+ z+1W_}C9(>=Rg63FgUPPjhn;5&a0@pl!u7@YUYQ>+I)Vnz8w~`0?}> zc$`6w&9ph+E*N#6yjz3VVi?rEzu>B@qZbL{(P0#`Try|&d;u#VszW8nES&{A(*D9) zP>zR~|ExILAm(im(KvXSQg`{h<$Ru483+=*`azlhp&GRodj7D4g~-?FZE1XJ7< zy#AOWw;|JF+JuBAkugQeI7>IglWnB-o%1!CfxZ&Zaa%?`{EWSugupeui$!~&holj$+0vNkmM z3h@-Kz0eDa)!PLVcha;#f*pVGMXgvtX@Z*8{Eb<4e)G}{I8wl@LtL&!=J9FfqR8XR%XYP7p`LAAFG8+3t$vYF& zaC@qiyGtDtW(F6+yBuC$Tq74#!_y0KY}_KxHRwS$fvW8mv4lbrSka`0H9I9gLRYZ_T8X_sxT} znG}gB2eI=s9#iu@O&WMYHl_&L?yK>}tSn`DW1dCDz${eR*nlYYKWzPuBX$ zY55_NECjW6faO&stz_}=?2~GzKT}(Gbr@t>Ywumpx^0q?SC8xfULYAi4)`4$9T{wf zaj%78-z!t)sV|$4yci|Z;6LOaW7$y zu>9tg@*dfj=_NL$!V7~NlM~#mM&APIc9wtSu3KODTnKD7Q( zIyoHeaH3^BT6&E-S^^lm+wSJY^sGD3)-+K1kGA)OSkq!cmH{Rflx@e)(2G%?1-(CIzy3B}*AyANAn zu;dZNQ{{3$=e&oZ7mVB50jb_U?IplpbDO(fZl^Cp}Z7#B+St7unOv+ncfPN{sXy#nsBsjD_EvvfXPcjOhaN3 zsTi{kg9f20l)G_`5V45Lc*b!u(Bf%2I@&$AtbimVuE{ZeBkWO7^W$*fGUAkxu0Hh7 zl)EU!G}79fVt5&N%nojt9Gc>_#eTpL7~WB>0q1hWH^$tSH3yQ8rp~M@*-udiOCbkV z{ISoP`+yAF>WAo}@lic7#oW|+O;Ut{U9Wj9F~#AW#r9zn@73uectXoW$0ci9D_}3EIF8qKHQev?Uu}4%WGo(`MM_+W4cfL*S#I@2=6D8s!iQa@{$ z2CBt@=*^u3NGE&4JKmw}Hw%Kn_m{7{u*1%#zd)RyfvXrs7<1~oxs*Oe)Idq8kGyw@ z@MIs_baRpUDwTV~>yDimQ?cdf2FX*mn!fu{;oXD%^~(>BHXj)_GV1|BE8j=N7~0V@ zpriRA{=)SA_ntVEWgeF*YNR?dq)|+TQMsGR_4Ay)S60HYc*x0yzGs*ZHd7g;o(C?D z8;bKoKzT`~Tx~Ek`8xx=7HaSrb+5yVcce%b2WPUXqF3zJg1k}H-S}ULKy+m2o>0B+ zBSWJwoM?IhEp2T!hrJm#^H0b&07UF_snm(0R~DBibqwjd_G%T#`sLQ1^6c5;M{JAL zgTAJP@%>4o4xHG-7jT-lMFa1&oA{e>Q56+zObW^#WtCHWHX}is<}XAOpFGYH+X$M@ zjml5lfX=1&R7aU#|KJW)PycpUvmA6IT7NB>Ix?o@zI`7eP+IKd|WL~A)W^}wS#0bZl_YJ<~; zcRS8~LjxF;ojw%&qaJ3TSs2X`jvR$2vsp>z-V$5OMsWY?Y_U=8{}%$wLZKJF93&`+ zoQPxi=i{6<56a+-h~EN)s@>^Imr=8&{}Mgs5=@W3ZI4*OYBye~NvE~7D1;SSvkNeI_s^3ET5c zcf_lmuO$u4%D{DCr{|t6$(z<<-w=sLiG0uWUbRPCLh_ zU|=A^o7bxt?sP9?!%&_qI*ziD39$Zt$s?|s`%96y!~RgnzH{PZcOO9#7L5ny8rCA<;Ml5G4I` z4~41XgN~orVOnb+=nskcKXH6zt894&OB72g!hKRU1B2#DH;Jj8>Hy8B9Xmgv>)z3t z*)eM%5(pOg%PX)2engPH*83x45I5Z@2?MaWPPFZQR<5D|!~q#Lw|#Rkcb9#%CulLCGJbjTL~-0(Pu zd-|My0Wdf3Mpbh}Vb2w^nY+?MT?{JJf+LsPzr}n8185sJv$wIcG4B1j6$a*TE+v|a zrVAVZT-Ejq^%YCzcRp-Ca%+-31w6``bHx$04WBpqA01&cuzG!<_(m?Su7{Ai<^dPV zHnrM-(qiEC_jbz;Kkz~+55oyWbuP`>k%i4mxFPhq>&KE6{XeGzs5@XZf__C1=Y6}G zayo45<2bh8^_Ry^Qrf|qRb()g4(?ZT=8vk3gMbT3H+BrIs;c?{bagqS59toax_@*N z1hByL6mfU0LZfvyAFL6ZYOvYHFDP-a(-`)4yJ5#PtuPTA-IdYIU8;tkN*X_1T^r6>NmkwH8F0SE zAU?!+kduZ4s4n#E^ZnKXcPClSD7v;N4Qb+rf2Zd$*a#b!mN}b+tcRT$i}ng0c&)@R zO>Nt6ItZnSr;Xz?1ax$LP5Zsfpl0^*oByK=3IIseV3hm{A=}#!a!Td=@_PCxBXCN) zSV_~JKz% zvGDSf!|f8RZQe5*W+rrD2VL%(um7#FGbo3f25Sa>U4xGi!$rONwt5PD?Tfr~>A|Sm zh+Snqss>$|@p~}4kuegeDO2#z&!-Z+{H;t3r5QuMkUig_?qULN90|S^>1f5;{K@sr zutv+UsHC^QYvF4iTLx6+QTq-FEZ>xr@dr9Ypi&pv{Hyg}A?u%mgU|0o_w=~(oC!6T zBNPbc1x~jd=qkz-G@btXC3^tEO=UHNhN<-T3V<)M8LzjF*bXboWj773_q`LRqV1&< zL!+X31Nq$`tI|c#XJNaL(G?DcKYV~2@drv3GaNh8Uc_DwC>0sVB%=0(?Dm^i(W9?H z1H&66vvWH1&X?gy|Do60z~IKC-{PTKb;b^q zd0oa2I*tncXcoCYs2UJ>eksrgk)iOyR2Hk1tmW(Zd*#Ac+92&}lv~g~-JUZ2yji96 zx;WL;X}lpQ+S4Mc35)1F`D4sVE(AkRw-oW4>NTc7+z(p(m>wYEmO(xJ_2qf^$jvuv zk%*f}Sj%qfXv0+leqb6DEP8Zy=%l2ixUvPCCMJ{ljd^r;w1i01Ru5)}Pb}~zLaI04 zpgX8fE+!T7T3dzd-z^2sX9;<#aqz%YrZF03e-4uu?FH_@V>n<#9HRupyk~dM&|d}1 zstE0Qbdii>cEeLBkpytpA}gq+B6WfIQaoE!P9ROO>BG(f8er+@>UUJkc+`{ z%jsuqL2V4n8Oj3D`19sTdWSOMbT#gSZ;Msh3jR943GdMzVyocggQ} z@?tH_O#mkk$eUr=!J7~g5+=#Ch`MJ5zRELqrbEXBiqu28znD<|wNS_s;qc?oHme3} zmB^Z3hR5fl)5>J2FT<@WNTafW9Z+pftm8|P32ZU&s?`L5UST#ipv?<<&Kv1Ht&`2h+rM+eH2`t@ zDZDP0^nZm76xF{{Kra60Pylf7c7ix!HX~Tt%RS_X1>i9`SiDX<=mUd;Y>rzWy^cLs zP#W*5IsulwL3RV!qvi!N2!djQ3RI)`H?#~VMQ62kbZ7~(bkgJw4KbDS&z4N4>by_XczYSR1@zcYC zf?%n{Ibb(&-i>i<44!HXiCdlhoLP=A8&WU_O$Qc0NN-cdjM@Lo9K{!eA_&uMaG(ZW z=8EiQ0@RL;LIY4ig2bMe#3?g+mcK@h*Yz7Vd&BRe%is7SPXiZyJ49AE&}5<{Zx6Wf z3FJ&|e?kLy@k#jdQwCHdz~_A+cm5bdK=|oLLiCIB1{_^n12XQ!QRW3nAbS7|1xE$6 zJej+Hmnv?otq6d~^K4Yj*zZjicXKYxXSUlYsYqXE8xFT9clAQ&X} zY=FRgF{2IaVDZCXfidhPB4_{BmT3kNO*=v^$7OhTH6A<<0mZE zUYgH%B>l|8^Q=O$rmVH*dP6S;?2AQf2mCKWMg~GYz?PbR0EP?R>=a;*HWXI$1@$!} z!MnjcZ>FI8>jaR~U5lHe6jUX9CPJV1QJSvCZ4IY6^vXS1dz=gDNj<|qgypos0Vpe5 z!(;`2f4rZ?#a)pst-L1)B<++w_X{gm9-Qz-EfaP?2?shAu*e=FN_^?E6~4RJQt-tD zh4UXC3>Vtgo~C+=G6$v)+xr8oKAuh>Dc8VY!7GxzxU_sJwW+k5G8PFp z7JD=nNH+7HEAa^n-qXL;BGz6{u3xA-^6(*yU2o(;&vgq^)`xIOWL zYWVF<0kIm^$baR=12Dq>Ut(P9lX=pV^UiRqQTATZTTXh~{;+P0t{yfOts_W0gzphb z-q!&t#WKHIgMyOH!GyoJU?nJh^3(6-7@dJ<&l+6q;VJV|n_bU{x*^7A;>D|;g|A=V zdW}EWVB?@X0TK4U%%uoxVPO%MlEOm=A^912G*^Yd#>Q6ErQ7I)q#%4t1>C-5cXML6 z=scraB>3?~vn#A7sq6XoOf0F7u3Dy4^`13ermsBo-i{w;`fKpK_Q?T_Atxp$hg>|) z%~lv!IPa?sp99HSn#ecd@E+jD&7B6S$;UCwlUmq zC+DPJ#so>`5^1lL(@k1$MTrYv8~8LNn?%lm<#S(S!e~F|ZGXxq=snm}Z@KnJ=eWRy zXF4pv)=y2hm+hkQuT=OoZ`MLbh1)rXm0sE*M$XY;D@`-22u9>0Kj(H%wNIOig4wzI z!9peO^)_Ck`iDz;sf;RpH$L;X)1IS-_g<8G8n-ZvTH9*hV&XaQY`gro&*E^!F-FLW zEJ8Wd|IsKw**l9~RGH6ZG5Nv-;Dfz;uT}y-KvIF)ZPCvf4c)~G5`-wH>A zeoP-xYp}agye<(5@Mfq~6=E*gezSgL@AwB(YU-UhWeJe}Rs$jRLMw$IX(Bt~{;8*% z5<+J<56fz;e1G^%f1w}%=Dqv9lv>kxuZGQh&-Y}i#9ip@gnc;J3X1fsN>b3;9t862 zTe0bIhl;<<$$Qo^aMlX~jIhfxLt~0+&6PE6bt4OZAS>nf9?{iTAc@M86`cvQVRl@ig)oOG^zF{BCcPK^7aU{y|MXxlrM5k9O`2Icf;d zmQbSAhna6((xC6tb#A;>b*5JxF;}I5sffWzVRCR54B;hpJ69I4P_VkBN8kS`ovW>| z4zybtoJ7tp9#KE6Ab~X^u1S2>uVhAe6D`Qxi)?8LnxXrqiM80mG{cucpxbp_=Uq&&@y$ zji2K+_P3ZOjn2{4Mk@+tUoK3ByYE_Tb8{;(f5aY3lu9N|4?nVC_#EmK#p00bJNGdj z@P3ML+Ymit4CDc+f;PLDP{{8zFUnyI&{}E>!7p&%6J%C&!shLm7latm%UmDy zMNcbrOh}1i#ee5LC1580gw7pdud>@XowE@`(O`K(S_^lIrN;7PkT>x0xUYKtbzV`` zfLg)!&2F^utuLc~OehD@``;+XEr&|4%hxX~jhgZ@{wByZZT0H=nO0&__|NOD?QRvf zCJGn1nof3RAZrKOp(o5{MwN7vzK1`1zG063L8`QCcC&^k!hYG^P31O+B6+@*$>2p< zoW~G58f;~bJ~W#0IOPI6h#v_0oWNbV znWSKnAEhZqB-Zms!S~yPbPbPlM(5bMHL;t%#2Lf&nz2{2?O7F2c9@%xJS0FHl@|}8y(1%QVkaxmFO%2EQ z$eMsPRJT!bR~&ZbQ$XNKN6t$1gLX46o(O%9!O&%J1FxPlHcDXuiat#IL{-J* zjoGk8!}Z|w5GHy!fa~o3VnF?mfYSlAJK7A#sEaaLHC(OjO3XJ7^EKceJ~15~ozFbU zB>)gsRQ|<`5G)wH&aJ%lW=u`out#TjH|o3g72ZTaA$U$Da$YMXr9UdQ;yGu)(qcSK zuSiH+lS8m>T2C?d4fx$}S<&eqp zGTw7D+TTidX)-(pt9}&YcYZg(D0J)YAoiptH*c68LpM`3n7gFQ6(8rgbN_gr`i;Tx zOgF7dwf>l}J{!QKEIM|j{f)(f@Dag!EpGwybHGH;b;(&BzdBV8Fortc1GBK>9kil^ z6IEW_n<{qO`}Kp>YzV*k>24oiz{Tn_w^O+mHFQ`Cmq0f6`U3*<=yTnSt?~$PX9x`? z-`r*re&9Gqit`+_Y-GRjJwM}u9X4@=tTLZZ1MdoDY)17!%DA9oxs7|Rg}zjP$bw;e z6atAN{gX&WM``1A0_&-w=_4uzA4hXZ{f)G?Lbqg5>osd6BOv@=b|xlBH>r0e&6ZI0GD zzS7$-Epm;Q`wbLP+pXB!qoP9_RvVj3FuH5vVC5d46yw_aD5E2|dau1bdpF-&$}jVu zuvxCH!eu48EexL?>-pxw&_a81Dwt*Uubk<7_lwj7^46=Nf5cE2VM={TINVW`X=`|F6b&-!XSWjcYJDShT4`xy9>&T4 zNU~uGw?FVfQRAmivtF{sOW)J`3AX9MKaa@=6R$CxyPKnF?YyMk*Y#|Vwh$~RG`Mg8 z8?rul{-2mW;CJe*IF2`cz*)Foq~If&e8sV&;wA(}tx+ z6CFbEQ8a8s>?+J;*Pcu@(EUh?D}%HqV5Im>vY~42)h++^mG+3Pz)BT6D?ch_0??P? zJo^>U?(#3KFf{F;13#s6&|h|X`L3N7YB~G1`}g`s4u_QDUfZN#M$aq7?I5jO9@DME zJ$tz6O}sBLEDNU8LqNY-sj^w{rGoMORVMFUE3r${y0g=>Ja{1e7;xndaQ5n4+^qAV zz9?@Q-tS9s>OfpL;4J}eFHwXR_BBto3^4YdgeRJMi$1FpW1fQj>s)~(nDqc31>Sh- z89VhQVe}8*{EOJXG_Jn!|IH|vT&$!x?(uNiTw~ld`(=)JpP0e)*3Dp3SjE+Be-^e1 z$y|Y`Hw1EaGMk*Y(?>zm*D~=nXyWI5N#me^&9N-HP-oGYlufY0Y z*Q<Snq9^{`Xt5~g^0BuMO(K-zg^yvgiuDJ22s;v#ABHNE5Fq(=wjZC%JR`#n^2 z6m=?)n3C5yxZqUJ?hHYo%~hnlj%jYf2q8A^svLg%BU|R*OZz`2^MoheK7&_=5OF3H z!Q54tvfR;(!;yHA_+_Q{ZpgIn+U=s%Wa$yVl^LsHuq{SrT?*Pc#lH4}7dcWlM7Zw` z0j!sXW;zxablQiT$*s#X2$P6y{V7^^U+eo&2DD(4GnZNZm31FTe^0J-$`x!b*TGwG=RK#c3s(Cuk?7fTtCh>IPp2Kan>Rzw z!GN%S8qWKMg!Z?l#>nf%lyM$QbES?ek{GBR0|6;8%Rf%i|Cky!qLzQ6z)_dOMi+;d zM!bGlLhR6I(sfR_*n`=ShhMUs^O_;Ka(V$CACBl!yyMGnZd){5&L5e-ER`kWlywcQ zsoVdo#CH62kI8(lE+DwC6CjsmeC%PO`CMh|@lCjx<=%rRf8AL95Swiy-Y7b&X(Y4C z&cw1brIG(U=tv-x9&yQWF!{fdi~lhCKbzq1gSq2{-Le3KKXE!{c9~k9+r;?)Zv#j2 zXj9r$II^bv-lRMmOwQJ;*QfU^S@lM@3Ab*wu+Ty!o6 z$0@*B!u8v4(WRtS00^|#h+f-NX^2-FT#DUzjOnJ0H=*h^vsyHk+)-1n7LOcJ&P^AX zL?2Bi9os8*%N3BS24*?&h^^@U03eWA+iGZ@agR78z*jTZ=(yZ4DBL{G#K3B%!HTcM!oVd);3}TvijF zy^*&Y70?==AWdCj%95*T^uED_v{AheO(<*DSy$4l&FHC*LM(3!>dq#6th=95fQF?H z?z=Rrm+g~NOu6Ya#BdYQ2DaG|NwoV*eoNQ`&HXsh5lX?MAy zxK8olCiw~-vxOn51?3g+-Qm=cog$_*Ze$B?z=Vs+q#Kbp% zt?ce8zW(i;s~)k_w>^m*npoURwqJyeTmf@no3upkzl@&h?I}Dh{0t|la3||^H{#;t zJ2#%bm!dLf_+_pS@MF4|`cs`Pm#Mk9Ux&cL!$L`X#0mJU;E*#vBc6}g!|n2(1Mb`0 z?Nh7;rPe^;qd<))xb0%f=nqT6FjKeOI`>=3>XAlK4*B;M+2XTtm&-AfDl8Y7VHXEF z&$NkH_pU4ZS00ON$K)L94|;2pGbKFMSA|yk_m0AQr$W)^?~}M?R@rF`n&=7X`@{Px zeNe=dOBvJPK)y{IPI9DaFNmo+1iGM@lG6d@yWc=sJoyQ10-XE-FhWrnq;7ymXMctj-ZL)SN8OfTq%> z@l9SZkJ=Pb2PKnB!{_p+s{HxE06PwIZ&=AOv* zK3_WUlReN%-?|cF+eZD;Cyv(CA8o3J?;DS6rxsl;M9N#eTjL}ugQBM_?db2iUPHpkSvu*ut1a#7fMvw$ z?Lq9#*(~WQ-zk)rIZRpX>%Nie3S}|uH!VuH^=EtG_}ml-GGbVEkVH1R%tWTOp4W$Y zs%kHbr33qk)jnPqxSHj#whGs=aN6RW;=mdmC0YgpnIh4M8|fnRm2vM?Cb4DwD->3d2RO{qW3GcV?01EwKj5cPZGp?kxV9y!biV~F zQO2L*6|A%RpsrZpX8vfVcsIzG@zjh%q8{n?>Jnqb z%YHE%uML+aj@tf^i_w`gUcwDW)}!>9ydIPA-Q$2^ea+3=#j~kmw&v|u$#_4jtU_Qt zqzuNoqYjtKnLmC}z{1r=R+3d>c+D^8tORpGZ;?PIDXY+13zvGO0a{TUeOhTU9qW!pdHr(W|?rU7z-}|JA zX*w}CR?Yrk2r!20;RoJqo6fLsK%4haUABeJkC#6i6S;;{J9(RrJ6HA`zb{0@9eCiH z<=N!g#Qp=94pb#4N9jCg6m*9&WqL6^<+GxQ^}IV+1SAEsF7&|su#}d7ea406&Y%6Rc82$Z#?wFF^9m~aL&7|*NBjN>H zaJx$*^-TiWn-auUYrCis2?#BD*jC=D=(^Xw?*vT+>@Kie4ZacmN&2pHk-hsM?iCY` z@~klK{Q*`H?-s_MFrHJ#EjFC+)t~ptjGgFI`bjX<3Uj5>1QAB|r)6#(Tif~ef71PL zU@8@;#!#S1Yn|7wkDbHZi(&LVpc!(q1kQuqdg{1X&5>BxO-Ih$6|Ot?f|=4-Ip;*o z(V9L9;(ut8WE`U20voE|349*{O7KU!H_A-b?hdL^7T9WEw2uLz*PSL_OxW!a%;^`t zE!;P`jbs<|CtCsx%ed&d;F(E!66pr}tLmGKTTDPVg&!-6f`yp2-Yc|`l+vQgbjEd? zP*D{a{H!m1OA#RST2;9Jm-I(@tO}dSI8H%b!DABKqm;A7oxQJJUtOJxd|wZa5|ERG z(1ylxyoXMKhKi^-y~SxE!Ha3W7hZnOA;_-wj&eE0P4tzOmsFds!OzzB3kc^S(`MWo zh=yI0^E0w)4xb=FiUVaMWt_R4E|ZF2Wf5fOX?h9s%@xdJih|YMvYmb1F_%k6AHF;X z`|hmi>)fE!s4N`$V-rOrYzqFh86J zhn*(vmv+{CQ&;pu>-fWiq$3K8%~BaF(AxbS7cUK`Lv%>fym^nvNj?nIyj75Vi-BA% zES5seO#wkX(s5~Eu`oGWBpLP;)JL&Z9A`k@+T&H=b}$)m682xc0N!9_8H;s$;m(Ip z(*$jUI~ScsG|5>XXrM61(08t~x@c)zyV>lmT^5~@4@EPCP8$$t`Grn0!q3WeblPrL zTrh0~GqFfl=0J3?rmxY40+Iap^U1|n#e;AaMFo)HxKD^pPrLv0uSHW<>`ec}pzr%o zXHb2klgYNo(oOTaWe89A_k+V?12&RkR13DRYRs+!CAv)1CbVA4f<81v78=IS53@;D zlT+%JZef?heEe=VMN#yvm%~|N{Y@J(L_P}HGw(|yBl8{(3)*H)LzQ{s0|~*l_(!cx zcu>R1^$Ow*%R`Tx|Fd>;aIwxt@wGJ(DX_etBw`t5kKFr!^8RSR;F_Sx$=DvC;2n^f8wqzW&<(>Tb@lp# z_a!!*Z-2Jmse-+%UzGRh_QSVg>^GHTUewKf;zaZ9Uev9lNM+guL_vK3*AM6fuVzqh zQJI#WjcTqLj-&tCEpm%RBc_4X_`{XVsv=0n{w;Ju;wN^S2Kg4_=KFDA?xzR;*J-UtuoLvmk%OG8(qe$s(V5l`HEd_(LKUrtFLl zL1IkE^D09`Fp1q9{s;FnQNCOCYaBfFE^&bYTA3inN>nUTduv`nKBeX_i9ywWWTbJj zIWex2B5ihk;f@ks0U8)d2N{q%ddO`Q5D;`=2B@+gQC%-#s_mU;<2|EJ<_`(Ms>vH2 zu*(rH-t&N5JD#q8?4V_-Cxj5hE+6n{nLr+^+Iz=W;gsO8AA=;)flHLWwN(AVj z?Sxbq^_Myo`TBs#pYX3y)eIp@PSMhd2-2%(lsSlL16>50=QYsI4|;3}nb_wlZX=kY z!Ig2}?+0zX{$v+JpC?I$;Vzj6U(BO~YDzRKZk3MdaOs5Ms7J)Is~1G@Sq7#4nCRJW z7|19X(hWEASqEec2>O&JN2km*BJ}+?^>BpQ7m@wX=k6nLptHv-D3IdXZ)xHfo2f-} zrEKRvHM5Q&6^PxEPO5)WpCUo#wxT!KO%08~1iTPuv z6)58}ym`#4AgopF@l9-qgHnlxUnT0xO8)zUiV2NvjL`q4#|F^g-SQv|wh%;&K*Et=#Q3wo1@Ss{DF0FkN%>VCU|9*Iee;b(MkOCj3?EXn`*A;ZYQXXr49{~t8`cY6LIDWYOKlJO9j%D*Z2-vd8!KO-<~ zpv?bG5ELe_K0tOBOJ%j}|4k4G8q;5#emFnG504}kPx_;H#Es5TI^?iZ#x;dBkeUl- z_ZK}LYIt29QrqpJPlli)Ji+ax-F3BBUt3s{sfgZZyEW91?_(}39~@Y8R+#Z&xyAqU z%V7Nd)IXc8i4B8oMc{>uhwGhx=P{Ba;YPwG=Tlh;?r<|WYi`sWu0#w>z-mme$0a1J zKbz!hDmJ?@2;v3!)e#IU|^d^xCnf;`V&oXwUc=jg8n2 zIL-fIhx}zTK!n_C$rR+|&btFvNa`>kP_n*eI#s);wG(}H8aq6P`?@Zj%n<9ug zs~o+qu69j}&e41K^;o7`g zy|$Cil$DQn3Fzjq2p%Dul~Gn19Iq9MObi`E*2jGGTa1NQv-9Cr~S_B8^Uus?bN%Ylm8|^#eNA=p2z(&6joplTnwpz;8Ccv zWCbBiWQ+3%yQ?pdWE`)Z+ZZ~M+3${`vw(E-Og87EE2DPbvQ0FTswJw6L`mU2<3z9z zht>UAIj+i^CMa6+BaRge$c?{H?DczmIyn^jv@&2gJoyawX1PKJCNk}U+m;KtLXFT$ z^a2|L%J0-&(Wl@0`H3~TkM`+GDhOX2bIJvbAT&H`aEEzGRadece2)_4Yx&FA7V{Cg zzoKS#rSdi0ZLdQ`?V!VdJV}9pAjlC{Us@}bk-54RUyx}P3I)jw>E~bifz1S859TV0 zy%`Ek-)?fYWTI5EWFTBdBu$itD;O>|CRiEmdexdwEar@|2ho3CsH21@S0kvyaxTVg zPc!XMzSosR7tLJ3!Zm^kEh9B$OD44Av*FcGcrdb7W>qWON1LzcL0$;l_T`mQb_-rd z_d?iH@@)?txpK+S;gKbBT|Z1wsZjaFI>j%k%u>Go{=C)0YNeHUL4Q8FHZ@nc{^S{G zR8y%M&63?@p&b;w|6bY-Js87bVE5$PH69>1kjXcv2)+`vYM(mHf za>GQyp9cKvgu`ThBc_`Kz+S+j8XZV=wrEt*FUQrkLoB9V4N*Rt!vXCNOCMN`LAz}PF73Dvap`hIBFZA};JR8PIw_?o30G||ma`+zRtp(Ln6%Z7q= z@SQlP@Q>e@Fti$2<^Q;UDbW7;CLAk@hkaRIf8w|)JN0C{s)PjS_BKip3sdEwF_3m1 zih_TvlgL0-m4f#DP2)DX%B0+4o?OvrMm3oUoDB|9*@_({4|zUmRhf%eV-u1~w4OZ)d-wptA<4{SnH!^Js=tFDmy#Njy>6%NG{=*~ z(AAH=|IZT>go3OujHzknkJm*7y8J1i9iACxOE!HCc$j|E9Mg{2XM}gDT4AV}huw)P zAJbY;rHH1|C`(9PKVKzfCSF)!V`GQiWCn_d;dr}vZ(u{=|1NRs{Om0C0j;+}W+^u9 z;l^4sh8dE0s?Df%QO_UH<)tT09q3PE61kH5UOf^VB zvtf;WR$HFFR-{#BYP)5?Xwf{6+@#BF4NCjNi$za<0?ktC1b&z8HTxUYD= z)y}tXnAfs@5Z{I)RX^BR{Jco+DTVi4N^1INudbVV41*;DDCR+#aIf~eP5(RkI^@{J zOol^9+(Y@&F#(~3XiNuUmEjsZkU9K};%$fW0j(1( zEayglXY28>-Z*+g3V>Z}>^`O)U$vWPA^FS@F@4) z_ei9sB4;osLhdm}EUrO!=Od7&(~VlP{_t%leP$}iTum*Px1VCJJh$AAsB(glj#)e6 zZcAE0OpuqZ2|k@^nfeaj+iDz=^floJvHIMH)Dqu9Qq$+&g1W*LSL5QF7EDacZd$Z+ zY9`zf>b|pW21u?KXPZ3&_XtYTa67bnbXf?TG;ln+Z0Li)p3f*66vK{JXe;l1R8G}I za4U|jx~0DQ7$vL?QT!*(Td#tsQQ7%}UVdL?sCu;H;r1LRyTOsou~!9q%aok{J1U{R#a>4(-hH-aIBX=kehFh3aYq&IkP z^-6*$dwnkMC4VKO@kB=-M90lMzR&6>2<{>_XS)l@cSG{RwQ#qnbv z>KTJ{>s+>@&#}e{5bBM+7@={!*~X>w2dv)TX1Zp9wxFU8frCS^`rN zRg0tsq1|jAmv8KwIjAZ<$4@TSKiL!FE^AXALb6N*4`($isQ$yL)!~QD=u7jWJxaj6 zdmUARyRgCNGuiVcswo=TBt zh3Bx<$E+s7%90S(RAG1c#Odmh_$}2%Iql#llb&3yIpYzqRbZQ~spXz(CIpOse)kcC zY3v;7YWS`+@@J37$%5RxNZ3Stg$1osq(KsHy7kvWHZB>D}xO zAmBh-1v3y8+X>g_s+)!D@tB_);A}bn zwpvrd@1=bGjpt%E87<7Yqw1xPNu#~2s744+g&ilt$IhzkYH$;FRZMTlBB&t)(ftN1 zsi&HU$Ox&?$uVbbDME|NP}u3*9Q|ciHR*6JqB4(@Nk+R+IwgCM6LRg$KTo_-#b+>1 z-F*W*F+hdU(tRT*rMdKI6OX~T#tT9;(LtV}9o?;3GtTTEgXFG@%aD#r3?C>Vjn^s9kQT7@WC` zNLGr+MSHZD=!%lUTDo;^AS`Xdq#~t@el-|KBzlu&m%55b3mHi*SFB-5c6xihaHHsh zvyh%PP#C>1vy2&Z%zrZ*Pes8}Ar!ytklCwn7uXanL5&IDYpxHRctRQ=v9bhm*WO;}SzMMphu;nfx7cpQnygCLN$^0-5mswg>__XAD-Y|CEq2)X$ zS;0z+aproKotvx}X8ip#rtA*lAC`P{cY#2*Mmt@anr{${4JqffrQhOOUBQl;U$nV* zGfYiq@tM^dY+{NKX?0cKOeM(j^YV8t%5=~s+~eNK1uP$`;WnVpqn8i8$IL=>W{VSl z_+{m8*II0FxaaAWR`i9yng9uNq|HW^>e=b8Urvw5=VrrGlfpknEV(ug{1>fc8R~H8 zSWgx#B3v);efU(v7w9&bPg99q)@hqHcjsmT3qKxIyXs*u7unUBHyYu1JG=PIIU0qk z;Vze&7eilBZGRui?U$ZvP8(B}7u@2XnMN!bj5TJN?Yb#idcoD*O~f~&oi7tX`UMu_ zym}?+mw%hC=p080Utl7M6v0T+LoA(_9KJo+%p)TX_7?a*EUxYz!Yux5S!AoN?*DCzMjpYaVBy0*mgYvvjt?W%YC60wB)zIno4bLu%p1J0L};h1YV=$Z3uWnh$vc{e8k z5JE;U`vuweGg7K1gt;7_@X0wg3oLL%yx%aQ5OQ#Tfw>Vb*)FMeE49)?_ zF6)D+{HSA1Wnkd>_5}kivzh7l+Zn9+n;JGf58uah<)&sxC*}$?tg(o&XYwDoMhBq$ zNJ*I;dpR?sZc^WiQsQtP%i)sreE$jQ*sdjv{)~7F&w;{;+IwjH1)bDnM203NZRo|; z7(_pDOo8E(h7Hc_E0~?!;KoCm5{h+Ft)i5&d<&W}%H?9ywQq?=ZC2riQt(w}!;;#= zE$OhrB2SM_UBgg{MmG1SdE@D?5;i?b{3h^31|8;P+bp;++dzZyV{D&a`Lvmk?HwY@ zb#5%mHLE^ZTpSWyCev0NKdP1`r{}2cNlue&goQ$SV$EIpRb${j7+Miu8Lu19dv0YL zyY4B+>D%Z0=iu{hy z*kUJ00JF}(DY24b_>AP~5Byjhsex`0b{PNmk3kj+grRg#U$6U59|62@N|2E37 z>Emde311;fyv`xTy=ao=Qhs1^DE2<|yro`44hVU0*RW8HhrLuggnTH9kZ-K&= zW$E)paHe1#5MjDt=vPoox3mf)?K)`Jf{O6pFrr|p{1O7w z(`PUkzu_a_Ld&WuhVSOC2j^Z5*?^;=KtGJ68r+jznOWkW4LTn6i|)yD#gH&8@`$P1 zj$$v_l@Pd#r}XzR6JBH6c|TVQH+L#Exu|55pl_N-52@n3Z;Ze047NqcuR!R*kDF7v zO0{Xn!>wF5oXr(@l8XHgEcD+vB&`Z?%iJYu)$Nk$PZ%~;ogbjx55y~-_)nS|Ww_}!3-Hrs2K`0J zszuI1EZYu*Pdvhon&%N8lTR7&D$R2iUQ%2AAhgT%Zz@>}vdB(%wb());g#YR-OD?k zaW$?-DOaSH0)0iXnpgi5{s7{0kA1UYe?!_p{6?NbuVS{R{VJw7)y18QtYahF;9b~H zOOM{x+9tJ&rAr-HHbVs~)v>r7I!YLK{qAyTX_~@ zU3Q*RW=FIzxe7^CF&4jLuJ|zc2n=~hP#&>B-⁣ufA~3NA5OmqX@k|?6I*(5~WdN z7-FGq&q_y=1~&Upt*03D>Y{j9qi4|*z)RX`o>%e=S6ExBht?Rd(v(l*CS8Ni)239^ z#5pl!q}^m5H1v`~yNb2_wz{ijo5h!xUlchp5$Xy=D>HYdO7gJA^5kSF@%A4O#7R(j zySH`oY-wfqahG_5$M{VuC}aJjTl92QWpnGZy~uudo#^^m9M zpdHl2aA6Sk5z^F&WJqNcl$0mBSlq**sBQ%EFy{qM;xL7nEtM($4PcdmfR@+|UHSMJ zq7XE69TRHg@f)U3Cbne-g|sC&CwtDE4Z8C;Zfn$lMS1>KgV7L`Dy5<2B}Zo%LX0MAqr z5k!S{H5!IssSe0CZb19uFb8&Pp0*MOq}+#A$3zeXXjxGr&4B z*AY?tpAW=%sT%5ORSLIv!WPWzEf=gF zH=9Oz1=GasH7Xg01e?t6z~IE{+`VVFqx&HoZ!ERc9%Cs#lEqw@KkqW!0{TR=5eipz zgTQ6Ybmb0dfu^E=^Xj$}Ge)xs8VHysBblX-VnUvgQHX1Vs<-@V`fQU*YBhNuT)3a-m&|H|7)zo<&k`zjkwz0;zyo?!?E;{= zb7T$WZ+%TnKdNjR4vWDH6k`YBJK@|nVSjh(&zi1w;agn}7u{7T+!9N~#9e;Snen5< z+mIUs`wlm|`2Vlo)Qv_E6`gjPKbpIAsA>Y0w_Wq3%2h1w&4_zLt7Y_@?NGI1W}vyR zEMYezr=<~4Gh?7uQ>Ko|_$Hv_zxhxBtmH4&u<27e&G53O=6AAJt6O8(&0*k^KatE_ zsXR{oA~E-yxjfacJYRoYqaDR!v1%waK|)SVITTa^3xLapkT5qCuRKP|)-*pPbhkJF zER9++j*-Hg#dbL4I1bbeMw0Erp;c4pe2sP^X?|xLhnE3a_F8dW2(aPUN5v0bEH|9| zxF|a(Y`UI8?{4kRdEBaj*qcSmX*UNe)Qk%^$YR>t0CC?jmA$C&+9yjcfDBp<@(L3g zsy7H55Jt$o7W9(hc&QI_1{>`+(X?)Fl^WxS?S|4CUpKY$(8#DwrbWA&^@V_wf0@m! zgeEoFW>13!{Qdq;^_baT8w-J~B^g+}|ym*6_64nx{fe!NEj{xlrjPExeQrOjowdHcfLg}!v)&<7k3M}Uy z?V4rIB!_!3G)l!OL_cejS(NU+)8SQkOV2L|mQ}erudQ=A5Fs6ZX9tj%SCM$1J@XVP zA^{0zMlmGCwv8a!s~9YJ>@K|GEQMdtSk2ur9sPu0lf-x@v|}2jZbp^kZDkn)Z_5{| z^4EpJpPsdY(>4Mrs+OQ4duH6l>^D|AZJWQo(e8a$MUCN~4NTh)rS99+irIZWMZZc7 z7q#d>-BhB5vGAUVMkxD9=pA1#nXTF(lW6^fZwV%;0;y>JwW)DSui(8-MMXc2>HQ6J3__29BL8}Xx_Tg`41-g zbr~EbNELw7W={@%%U!GdHy2)^u+wC+ZmQxc#oSDBFRjx5m0kiUl+ zL*HN3A8!##c};PS>|8_U)EbYyAFn|tbgXxEaTV~$Le@0%i#iCTrjhiFJE?4;U)^T* z7JVDreakGtdYbD-+~8NEa%#-I8-`O{&JUHZ)qg;%cjAti)m+fh;5e5-<8DS2d9oIE z1;zHcRbHl2)(yHtT`%|~iYnORPB=MfoMq_t(AIi0aASL<*I@z`F6RR>gzR9lHypXZ|C`0Sv6SZd&f|o& zA-o$_zGg2gGxl*j?m?z}=H7LT$lL?<4r?4xl>UF$6McGq@F7~>r=2-?!abn^_lZO< z_*Co9+vassyd&zl8`jAck#GHSCoJb>^vEw)22EvXf;E&W6LfAv#x6xYswYod2t-YD z%I(lN)>?^W$t{WXRg1dl+nk^Z|A0-T(y74|X=B~}u|_Nl%O== zl;a3e$uw9aH%(kNuUk+K846F)UfVD*Gc!z485w5smmH4YS9q_u3V-G{_5;)=5MUg7 z@=sE0=Z@?`TCJ1D@GVzjGw~SEt~;5HlCj`4@m)&wlpuiiNM~?^WLTp(Sz0m=>>XV9Y7jIbdr3%7|Lh@;_ne zfPocrOS}rq@>ClAn1wPT7nMDJo-25iG&6@?gi?KHupc+wH!OUb?#hmsPMTIykHi_M ziJZz-ZQU)E3?IQRT2SsyRYXE^$LZ2ny{`hWZv{Vj-o8(F>Gf512g?yLa9A+SD&(DU z18GUjxa!Xb@*A{_cY42GncO@ps9!DX@Uk;qzYP(W4ggcj1&7Ib(O>YZ9gq(zc(2co zYogrWG2g}COYVotR5jtfUNkK!q+~`)WXMa0lPBlL`WR*nk(3reA+o$K1}3qBE6EmQ ze<+8mc%#LYjtUlNwU6gVrzRwao@?I7IY`Q*?Qarjrs4Xy)JEb;HwR!?Y=)xKaW@L0 z>JQxE3~g;~w1tf4gZ~qJ@8iS2{i(lJomXEywkw*vxPpcbjmSbVp} z)iVTBw?Rmrzy|Y-wd^m2@ zavryNOz0qc3Zq@9c}))9RU3M4|2Ebc|Hac`>4B`zUj<)DI78qW>+eX&$U-i$O!52*U%^4{xR0$Y$pjRN6de$$=yS_85?4RcayC)|2B`mYRXpGl>?E& zaDt^>&y3qEk`?xRDArMezGq9TTXU&Emd`515Llvx9)i1DRENFrEx7KJXyUeYP~*}Q90ofnHF1FAR}rl6@&uhX=2 zl_VT&D_AbWNMjq~9)ojU7~sSb0|QH9?vmOVz}igjmK=vm$xX^NFJ$B=6s10{3tmTs z`>2p+s6An_;E%XYIfk)~j1w9vx^mO5%4W6$lir2kGF>A&1Rx#FBh@Qq*Bb!|%}wto=)1iToQ`h4O*=x(VztynxW9|uJezIXTqb_Fd~^@} z#t6zITMv)@Jy?MKEMN>T1|abV!bx>!t;he$s44-rOiQMQ^WN@zK8w(1qd01^q3_R9!`38{2X3Fp%8^18R>Oz5Inxx*LDD)K~otW$;gr9k19>A!Lu zA8llW($bxPYP@h&DP^^*oKYdR^FhWtGK0bmxVC4>j;pTiR)`MWIAZ!_$o207?>~3EV5@xwZH=SG$S8XV4S+Cb==vbPYejI1S}gEKGIukBb$1{FLXdsCM&)G?ZzD zh8{5Cv!+Pl$C(UYZUAi5g6udocz>kg*%Z;*`@a?wFI6ZXu?yCMX(fV?1Xxr#* zR|9!-gr8b4?K3qZfHy=>U%+*T%c~X5G^ppkZN&LCNw=u8-Bi`2+o<}$ zvU|uLu5y;2cfe=|(x3_na2f}&>1_7l_6xV`vPC-iOH$5%*>_j zwg)o7^OqchPln*Ga1HUcK_!ws)1C3OthxdmYaXL6wq_Og4S}ZD=g2hA*e3(ZT`a1` zjO6YwNTC~yWZTX$yQKsY93BrPi?qJJW&1(^p3542RXueErG9RG_G=RvnuPXGCvjM( z%*z_jsv(f5=YDX6Wg&M$+??dQ}gr`VgwK$c5h=8E$_~g z^y@?=uvLulYCtWd8;tN;t_(i%b0#xOtQWQMoc2Mp5X0Ta3+B((fdK6zDCwQ8sZhOB z(3BTSDZazAK@3W_ZP1o<{G!jhezLg?tD+A;qXBM%vt+wGFYMBNaP7zp^f~C@Sf>+d zWowAco98 zirsHj0UjP7z0&7Iau@VMz{IvZkS;TdpD2_P$hsrLz0JM+sujvN6oT5)Ut^KPYwk;4 z@v17?D~&C;nvsk`GwNaX_ag-KrYD=*aq`t-Wz}20zO9DW@y#)Y#Y9IF@L;A6IL}l= zVE*@*5yT6cxmAhk{<;bL zNBqzq*L|9m)wfco9=6p=29!cXF=$?58wDz=+l?TKg8p7(`OzYFjiX4nO(O=W+EqoQ zABWqvB!T{BR2-#=0=V@bQ;$WG=O7I<<9^;NHXAl+NP*z*I`sK=?&Ka{qMhmZvo74?b{n*wOGpJ8_whf;~G>+q|f+VleL+8@H&r<-+n&K%UShL73=L0^9>Av(l zy55z(aAg-v?|gMwb`j1RlAdAuQ}DYN;|BJOTK z8#ah5BZ>*vek~kUPe<0_vZs<(5U#)`WXHeTLkyX&1H4x>G0#-};#}I4;L*Kyi#D%e z$BE;P8=Yb9-2PapsVUXAH2@dXYLVv2`BR#*$IseQM?JK?#HqYAtX!WB+q_1%5LN6h zlb?7+)k;e2fnU`2qK``g?~pdB-(V#$Ss_!58jjallYq^yT#K93h?HqMsx!MeQQAx#A!d>?`9Iw0;EQ1B}iMs$90U+?3K6%V>OSnmI zw^0VRuTtvJPxuYs#mjaWtS zhNnps<|wAZW2Z?rD(g+`5Sd=_{32tc8D=zw*Ed*nHnY$Qjb*0dk8g{kXj0<~ z&YDwJ2Hkn7T%NdAS1DX(tEUthz@tGG_VXeuNNYpUq--;s=z`3y_5s4ldd0s~qY91{ zi#Jj)PBNNL69NX==I-P8D&jU`*Qe=4Mfiq{HewdAne$aaynl>3n0^Rqka&V#ANLJ9 zs6f7s14ID;N;aKOrEr#ukLu0BllM7x`ru>hswby%QqUOa=&DDX^l6yW3(JaMqoop$ zUEKEB`C=j>Dd9sq*t60K8bLE8RbUGgLzGVG~3GKlVcQv0vqoZy? zHwqK*Jw1x!YqAk(P~w2E=)|pP9qCy*Sd-70gk3z%_zicVMKcA?vNh>s!24u=7;dkV z=Olik!q(#4Fz=O}u>e<^lSfaJk1dJXxl z=g3s?C@n9NQVTcDrtW8ZUBgUMtELe`Y7$VJp5&`*g-4&Z8G(;snL|+3sP$dNrro@4 z_@AaRAWdtL^(bGgwHt-m+#OCJ^1y8&PTs1W59y zD>UNWURZJI4=0J_WnC?%NQLrfb)91iO7Bm!{OE*ZniR8_x_CF$R@NY;H(m?PkQ*#G zt4cD}7IlvHSq=dW=}!0#Qz*_HT)i>p1m599Td^l2+CJXT`gCkeNw*K zhSfHvOfaXzXiD=4aJoA@sUf85cC4cVZ8@mMfc>sNjPI=yb>~hI5wi@Zl|BD0TE=B6 zaLgYs^(RXu^by4a|BHFos^^)dId{CxB2@Wr-;7S!9B`o?7Ue5>A^8Pmf`fI<95dvD zG#Emms{sGFPFQPFc=FoPYnKxW2%hgAzepRbC{`2YL(N{#fEWm1+0)<%;oAn@? zP&@5IepZ$^a(arPNxfU?oOF>Qda^1ye6PC2ohhra-w`#_$|`eCo^a-{n{$d8P(3rx zQlk?PXeY-JhyW}+ed6ntqazQUtO8U6)E<-EinXVpr^x}+fPp{RgV^>-g$XmfC_lU; zUge{(!Sd{Kq=Tu-T_o0x+1zD-_bomWTReKgPFmyIf$ z@N2Nd9M`KXW-yQ6*OM!$dbN-UeD0(P9Cj~KljoRW!I_{9T{RvhZuG5^wNZj!Q{DP1 zQeSol(9=`hY?<{m{8m4p+(T$RLX#O%wY!1Mun>G{(4-loD9cnSTfHetmEs69cUW*1 zr4?>x_|)5XHJZwqcGr)rRw9N9yeK`3$r;kZcNkL zTs`c~Q`O+vaa|QIk()Jjsit^l%6E;v!=#x<(-5~Gbjn!X{78J1qeVIB5-P`7I{?uL zK&zH7)9<8|Fl*4T0snmSc4ZS?)j{D&2IEswTsjrW60^2MNZwHlT z)n3MySWS6F^HHMaf;9bt=c$5}_QU%ctZzl(7kfSQ9uyTCNGUuxHjRyBMFg|Q;2)xF zH)5u#x2#IcM`v=c(Z0N89$yf`39S-ih?s-Wq^5e0G#+)x0NcZUBB z6a&3Q72IVuaeX5KS!GC$-q*ZZbQAjZwe zcSn_+==D=HNB)mAj=;wS2|!q?JVn|bqq~{eS-Qc8PL$veY7WL=#a3yuQ3#kJw|Y7v z-6`Z8`xT8r<#sJ@04;ts`p|U9-ioeAqwJE$ycpTNUjhxxZD&1jwN}bN#YrE~$lOT? zNuca8Cs-Wv9TX1$ZUkckI6cawib^6{)B3QF4dwh^owcc%*G}Hd9?E?jZ^jQkq0!V5 z#-XBfjyZew+}qn%#5=tl*gDXeZ}v@5oiBy;RPEN#wfUZR`~GYFIK&fP0F~|`*hdRD zG^7}ZkkNSqrrbHu#zs*dp3L|MBO551J46_HhsUHMHTv(N1?m%y zjubJ;nImx!3_YP6|=R&VH%hZsXlc48}R>{ipD15T& zp>OzZ5ZFgYx<5pD5v%$ppVv&LU+zB;xh41P;-a-J@48ZP%$*jhw)&yI4N@ZS0dl86 zZ^AQfoxc27%aU?rx=XX3@|Bm|aBz384oXcl0JCRBuviZI?>uQ=Dve@1XwqDF=rXUF zx9iKbdB=vy7VT|5&C}Tyf_v#{vw==E-s<(MveH8JM27;B9SKOPAy%z?LJIL~BKaY_ zs{}S)h4d7E?~LCYiGx#oxC-}CK1hdXq3+^Tl>WvI7d4bC_L~K(?gH=uk^gF`qiW~9 zp5$ow;*&PhNG{;>)omuk9PTMGt3#t^6}*d4hQ(tCOz!z*>8vm`a^oz9*)vA{^1IkT z+MbbbL|=O_Xy<1)!Vce6-z!*>S2lDimA+d3u{oFUE1=P|ToU_B- zq)!(*JZc2SRO_%cDgIHHWh0a3)vev>))jJbJH4sMR@<^CNjiiRm* zok3F|O8GQ%D{SbBj?X*|StG;zY3wpN`|_T#;K`5Mv()>mjHjFC_ia9hHgJ-XL3TyT zd57-|=+I4$JCm;_H*{e~dD8khica&?NLggnD=%1=bh8v3+!`j3fQ8J4AS#YqSyTI> z(uD}0^XhWxJio`2fnZU!`Rf$?ve!R-Mzn#vDJn$t=aqokpyJK*`|Qf@=*?_TZw!nr za2QpfzTzPJ`(0Nia#)=Or~5Iz?UnP?uErDbwoNA^Xv@+gHG&zd0}~y@hj5I?cSdpV1O`M^%rH3F+29S&ReY$B5r{VqBYVZ z9!F?ghSqMwn)S_1kF!0U;PEhCBg*=%RLcYac%tT7;JPvpdA{ro1mrl-oV(1%?g{2q z*5I3{;-N>^cw*Z?ot{H34KRQ7E$BXcw7=_DOB>IYL;7|#tu{VOQ^|G*(99ZC?Wob6 zk56$?A2`z6^67~WsyCu5S7YQiiUN+zUC#PV)Qgs`5v};C64O~GT~>Fw&m&q+(4<0S zc&{>?GYV^s$-ux0;CD|zHcYweq0dV1>&lQFD#m8YRgwURcbc*cfAai zJ(y%O&vOR~7Z~}U!by};Zax9}$?lG{elCJOD)*xPHPOO-jt%E67bV{1F@M!COI&Qf zd$!%??g+!ARvtT2u(rwk|B{CRyKHx@-{} zCXLURcXk>~={3W3Rk*o{=1TphVJSwE!QJHA@7>|%oc|5g)mrtmfGkwo3!uC6c|#EE z-;S@u`fxDenfKu5yVNwcu%@|Hqfexll{6j6g2P?9$xTpYh1{rrFnOGxLf)Ppz`AsZ zvAPCGl!iEKZx&t#rK$ubWMjF(!f1VgqO4JnbaWW%h0TzNYSE zQQ+!V^@6kMKz4n8@@53;LgTmGOkU>F7f8e?BoOrd1Dv8~{ko2<&@>UTVq*g&haD~m z1@LGAZ)jU-pn5DH!d$v4Dmu>R%U_3%u7cL;COV{0t!leS+q=G)Tr+Rr_If_vaqFS0 zFDp->bUOYOMX~sdhcvz6*D`@});g>?V05ZnENC^M7IVr|I_yrQx^Xsspr<~5m2r`) z9Mwj$yGWYHxmf9Hzl2qkNjr%8LBpIw+A;C0D1vX!V~-q~oCu3;LE-mi^`H zzcb@9o@*sw`uD%h|8C6^n`t3}Wx?IEVzG-sO^?8Ro1wj55dGFr$As4@&GkFWM)$GS zjAPH*Qp>vfwV1XO>bl(hW=wb_^Bl71p=O)!I z`c0Sh8AouRcV6c6eufn70Bai%>}Edza1FXvsY*50L&As$mCm}^fws&13H2F}80Gn# z4aWAOW+^a&4e}0<_#0Jy`GW5#Cip|~Dp2=zz7t?hf)WB-)Fd;2E_({Vy5O~(0+l$H-31vevtUq1Bv|2WxG32fpFjMoBI$1y)2~--50`=OjK{T zuouaE6meYdw5>W`wz%Mi?sJ#iTVhJG6!*4E_xB+4a7($Z#!E!RDs}Tp_>p+vG~wtU z6W=Y}_pQe#mvk;${FCiC56?ilPgn0-IsI*8L(hNSMUoU6nf9MeD#Wxe{|syapvwp* zz%4GO4q(a?uoHaJ%XgRkhaG9x7qQ*F`R0Dw&FXL=wvsaX54mpCXg8Vo@w?nQ+j}`fEfkmW{X38-9s}&NB%t zcwj3&WO5}-o7pJ}Scx)2v;1Vr!MCH-hh}S05AZUBvCF9}XVUPlH+#zh0$h^7M*zb@ zePFy+ULfzs2+;TYttq`V37&Lr>=t$s{U2v!qawt;Y3Xn#-vPin0Yo^$XNA%@?k?|C zY*0uMrsw)A>CR&f!yh9V0-sC>;=|Qyv0(DkMyRO^hX7{~R>+)3>ZKlS_|B ztwShVZhj5a;4ggNDmzq-kyQ_z1%DN~^E^f^i|*bgp;v#lfA1p_xNbEd z$MdpE_f5uHCa2(GEx1+hVGgRsG5TN__ zGH*jmx^Mc59sG^y7DN{kb#&E1O|%*mElXmrJ|G%NU}HtqgE03_bsJU~Zv8=ENs;pj zjWO!lwyk07)!IZM@rjlbRJxVp;XP)3j|beR__8+o0I2giTAa zhyjDd@|~h~hnqn*KV5=)c!~oD!kn~`m3#fV4_`pu;Q@@7>$mB&wD5C{(RW{yW+KO_ zy^Uy}Z7KfeWZs;0Vrqn*2BUAvEn>lrWBs`BX56dssRxW?VFX`WX^gneIjUd_`p@0T z?oJZnP@ToL^^pMK?~epA@KN#oX`(@3ioHT}bR-xwiZ%i5>@n4;ETUcQD(;r0{7z)Y zhK53Z%ZEe(RZb0vbOuWXNXV19etvaqn8oza=Nv>+%jNdDohu7+4GUh>y&`kZNvu=z z8jlbJrF;JJNwyy~kZiRgRaY`brw88bmT!L%&|6|$quPmi zJ%xjEy)Fqge}vOD>E<#^A9g7;svO-Ef2PWOq!1&XC9vhjViuR)SV0OC$vQg!@b(c)J+`O z^g;%xggVyu+V4cPhXr30c9olg69T0vtjJ*D>uIWf9{{ZWNZ{1v5ldXFjLS6Auw7w%k7ueWj0OJt?HHc~z%CJR9_+`XhEVj(d zD(;6AeM=dCm-N3ptK7Wf5SxEp3dahqCkrwAzRYtVJrI@jlEL12a^ z6YPV+$Y6(*rym3RTefup%A+LyC|wwN!I=D_BV&mCpz_n&|ME>q zM#{#p2n|R9M9BZ~0-V~2Vz4(BxA+Gf{7+YR{Feb$1?utts{zb2;Az-8nf3puC;L|m z3IUcZsrq`P|F>rMCw&=Me!k6T{$Jd~|G`G2C*fmQM1f}fM*+v*FGA1>c-mkRSp^{U z{JZ1+_suT~fKOMUYO>gW?jr#E0Pr-HPA;>*pzHtCO_Bi5DdvG*$^TokgX#Xbf*pui z{Ko+S^wj^y82~tHkxoYapDkbh?kNSp;7Zld$NlG_1;F$Y0H31&pQcjC4YSBUz1o*% zjmNzQ5d6LHUkq-&6XX=b#AttaQ}a0_@pHe^iUPqGz&;Btu1l|z41d_EGmW1T@$Yfd zJOo`|R!t2AgZ%-s=*Z@(`)!uaff5Js`TT1@rx_EyLH^`{$-l6t0Hzym}I-XZ>k`}hC+`(~RE`Pz-^I+TaFFu&g=?hs&5TICZTf86AVrvk1XmDOExX+6V_ zGz;*6kaltlzZlj9FlFlVJ_N~7B+w+VW`#|@ZPa_xac~y2W)yww#o6nNyK&RjH{yvL zQ4|QxyzQ7_q%dQplu$4%!rMwAY6_$+ZTKWA2Y^RAIl|d9DhLEIZP~AhiS+^GlHH~~ zQVsbOlem93KRDzIw}C-#KR=**ZiuEop&bqabN?r0#jUgXiISB%b6ZLRJ`XYhb!)Za z({l+RCjajx^94LBHHbJ&C^!v`m}I<<#~EI+eL^)gdY{gwhFI_B(KrrDn`wbJ;E#3D z_gebOY6RsnB@d{Wq_}UZ6a%wGr#s(E%*O2vo)|4;m8t#L%W=9Y`1R=M=)f^^KW0Z0 zify1&&E1R{{_l za~&41%Wkem!q(xyc)*(Wv^7Y!=xsN7H+r(t49~AefM*CSXNtGtDn1En+9Oz`qn)eQ zQe~T*n)sU+A>f@-T+{vUe){9qWyr$C5BS`P0DpARUk$+qnD8Ewl(14mCQK2`az_@q zQiTc_4LrSm(B5}l*@24vx4TOcjt}uf@aeUsKp3e=*PeZ6u#)^?2_3L@o_Qkl6m@2J zFXo*b6?CpSP>gwZ=?i1DTsx8!-UQG^WE-T!0Kewk5v@;O2{+2BMX^>SW&oxtrQnYB zQ&7%1URRX2+h~3`_F7URp-m}n*(sqD-ejOUb+VbzYgRUn29 z*kJ$1!sI|{(&~P?ZY#*yRR+idPhV(zOo?4{3B6gU+JEm@2*k3sN1Cc6Sj;jkf*!Wz zC~Hj*kR|n~TkW3d^;bQM1DMGgdD=7N;1~a9K8P9#&9>kKINOA8e_IP=ZbH*fkVbA; z9hH-lt30XG|EV+)#+E=Wk+xb0<36H}iFK{}wyUAH#7eGCSyjv@)Uu%q?06SL(ymR! zZTh4lyS+%$n=a%y_)1vx@KbbLxf1MRwjrC%w5t2l<_bnPxAT(iw_}S$4=9$cY>-8^ z5%Y3*T@Qw+N_|k;jOEU7t_9VyX{%j0lP$HyMT(g6x#99pds1e;BU@xiR?DH4pRLCH zepXNn%`Vh5X3TMxCh{n%k)ZUN_U@mY&7bZ=8+IZxtGiEIvffvpURNDkZ`1QzZyQ%$ zRV4*rG*YnDqFux?wvx*fFaOeT5F!E2S0lu70&G{rvA^s_B3*&uPy#jEr>|ri=bG<~ zU;+$%V6t9N;Y5$n!t!tnR=!eiI2nnt{P#{nz~xiUuQ+BWe3WI&F5g<@jNxq+kDJKh zK_^j#$;mJsxM6==G!p3b#MG)0)!k}?Zys5%@AMPFsm@T`26YodRkd1of4wZ!1~hqw z{djO8bI#O?JEE-9Y#rmAKY(fB)$<}bGc(X;F@n?;RsnfE*SKPl_0wT7l`aC5^lM}v z5|W@b(n<4NXBpeZYUtwJhW@7Az4K~<-7&dH+cS$ozesPvN%BQTGYdzg-4nCsxp~k< zNWnh&GRe2R<5hn&Rg(vRj090(yrf&hqaEM*lEznlB3i^&tcK* zdt2q#w5i2evI#`}u!4ajNH_>G7Y~}yi)-eGmOhHttriX{$F&Ae#d^Brtvz!&NB>N< zXYbl73cBo54G%xWMyjYB$-hI68zfz0%XwL5PCzh&B+^Y_{fg>5g(cr#06kj{zY&EZ zkS%kz*cSLK5x z{RUkOd8LWY3Nc~seu&{2*xqlVU>Ofb5@6r`uP_gdBwFjeewcG1s}cDVLV=?@%!l;= z2@H+PY7!3aR{N9lQKCSaCejV12&_%%TLlI)>};qNzP?~ZQJ1aJmodw($imx)#dmczLd#3O$fE<)|y zsi$~M>be)vY$5KPTq^8hS((ghC72x%S`t86BcTcXA7jksQy)>BIM(u*+1)Kg!BILv ze+S8NP<0d5T9KBIR57PuFv`I>5k+}1ExhP=mapz}^L2R68#buXrOiq|XGL9`V*f7RKmbI>TM! zY_jYD_UO|;?XhiwB;MtVp~97Aa<7v(+o5O8bK-$rR11)@M++==TimQBvLEc<*kn2) z`&Z1-ECNZaXCVYKt<2lxjn1*v6^QG8>&r!2nHl5(rlc^SJYa(Pf10}Ta45Jh%ouyf zE_?QECfgWGmXApDjddat#>kR&R3wDSHg;KusmPKRdq{>bMj4VNOj4F1OSTea$@ooQ z-|wsY-2d-=?tP#4yyx6=&$*Tf429h5JBu~Ok1>>0sq5%$9A%#Bqg?vX66I#!gj^}p zh_VHxW6!mP>^5kV(m>Y>?G4&mqWue(hQ7}frU*eGv$L$FQZZZ;#y1f_5@;l8Ynza2 zd*X8b8c65l{@hw`KUPwPJ6L)8UuPuNT6%BVqE%A8T@4^n5dW?PBp5F7(>?OJ&Z23!*lmhZm! zb{F~u-+Hiw+jn?1xEfHC@MKp`-&G1}L!%sd`$J(z@gMr@EP&BFGoBDM7N$c0YcaDf zq2!G|c!O5up(1W++gWz#i;F-1PwP9e0Pw{R6OEbQbmViowilONbvFu1ZeWxc*&Fl1 zS2&tz->-6<3VnV6T}{5aoL~B^PJ?q+;7n!_L^V{-@^-sy?PAwqV>!kAh9*SxhT{C* zcm+N`zAgKP?u{mZRkze^z>5EazM2YVbmSOtGhF^rX=HQTe!=W|A9iPPlbUh5NX8+% z8>_rBDlv8~O2BY%EK(wv($5@1Iqo?Of2Yp*az{m;AW~brV&AB>;S-bti0AqRQQXgt z%=`)}^rhu8Oja|l%kT&=?zguLN&Cf|!C(6H9T_onyq(c6RTOR5$X4hs8#bB8KJf=& zb=d;;W^Lrw+??PtvWN(ThsGI@E1W#dNSKO$T+h8|LDYw2JBDk>FKUDo;%L5C?r?2L zCr32P`ZfiT9<}?FQ&}#BlLE#S(QaM70tkD_i4>TtnD3LB`Mh4KES}ptq3V%NV%}jV zGozaPFehE$4c9h>f~+4D+$1w`JzUNT0}h_e7xYC zyS+>a%}>axiuG4$rbpJy%?+Z#93%z7v!w{5KsE}^N0CK4Jxeic(u}3y4^Lc!Uwxj& zU^P2eeSVa8vXQf$XfzRCgW$f@iX=~DXferpG{D<&BFBh@bD?ir{A8`U$PkyFdF6x$ z0a<35;8ogd@a=Y)fsc1c=t1RNxy2ILAuW2SAq}o4(afbtNLtwQR@aPT?uJ%CayCCl zrRMUgX+PF6Odh}M(K~Cm@yhCg?sH-Drp=ct-fchN9_SAh>{>g2hboL$D0WBu5Vue; z1yoPrbsZ3_dcxoFk8=u|1y28h&-*t(k+k;8n1lBx*`xG;sB5qI!%I3twSyjI>jV&Y z@25PAJf2;*cun|t@%0G*RHAnbDTvT0@CR*TQ%zW4E4el)ab>D8trEj3Wee+i*u;`% zg{{&yCDs=X@^#}9q}gs(JD(4#fI5rvMVzhM zBs>Zf%TgPk^lw7dI=#WQaU0~Bs3BJ(n0ki$`#&Kpa_?&_b6;&9JD-&kbc*bz$}MG> z(jDp;pk32V%RqKH@d^ZYy>2^!OA#9u`YCErfT#|GxmNGrj;t=LN zJFl7W!rl@zi=Y}ReSJn0LouuTMr?HNYdBh{IA;jy!zda}#`XV6>pN`k+hWsK!c=JK zSOzN9H?!rf->ruVN$Ah){PQP(AWZ-W%v0q7X2y%_$0@=;9uJEz9$5bYJPrF={@OD7 zf=SN^)$}REeMFc_VyId+hdHx2{1rGsAQchjA=@>}*CDU)XzEnJ5%M>o+-G5V>f8DP?=m8P0H@$6SHP6N2n{j_YW z3ElaSE^uY_US#C(_4DA$;#<@QsE|?s0*^o>EkgfX*K7uj#kxi>8m?7hEY_{x1nT>iXvc^{H zUc*C9=zSZlql8!~_g|18!D0*R>$6Mg0S6NlJN(G3uN5YczLBP2*eyR zpsSUP4}!k9uN~C08t zu5qtI>(jP_1w}h9Y3S(bu_u~SIMaq{^dGFU;D~Lwhmb{ijWhpOw{2&oe{D>sDZK|V z_#zQXzo~vpq;kftJ+}bk=?g6*WGb;HG} z-E-J;mVNhm=b4#jX5RUHCQwESiit*uhJ=KKDJmi)hlKRZ0txAfKj10iKSBl@uMi)o zO#}sHL9tWJvVIVmf%nwgkGpAM?~LBv0K)oXuynMZlt#SUrF`cI9D4Qf953wJ zo-rX1*&NG<9$7agL-Fgbf%U+L9J(HfK`L2BW=2LCYl&`Yu`hrkItI~xdd0&&@qrIC zzdwl4iAp}n^R2HbT3<%w! zT8pUIAtB-6JpMf~N*O_@Hl|io=#v6$dau#B!tP64s%QRqUS-`4{5s!%%laI8OBrLmt?7cC;kCXm-GNsC__{)W#5YlVF-K-5PR+Q zaySP{itz#fdx;I(Z(KO&?{DO~3FhjS+?TXXF*4wC<7ne+<8wVQYQCAHy?KI=4GD=N zGrdsDv0wGrXYApWduQR{*#}J)w~l#ZV)yNBHjV<`yFTtOR(L8)um4{DtES<$$7M_0 zED%gEdpzwsLDDeIecL{!W`=kHofVc+-bi0ym_JLFlxfBao#2J>@Yp&e_y`Js`c2e%_r5Xy__jv;j41l3o&!@G%CD{ zIP0q08H98NO{{*nn>y2iG%`hYiFwuNOP=418 zE|}S&ll0eEtRBBU<0x*tPjR#CBGkRJb{)YI#WgAz@TI9CUXe>?ouzrB6^iMm>- zW*glskswxD$jX3kTM@PF8^FjA_kS1pD%JVx$7py9eXSM7ar`|WR&;YdPI$3*+VbDV zzj}vO-FFr1XyV~wPjxLV>Jif{dEO_TEf=fIF&Z2K{z|{NSaMNNHxbv|DuRKFELwcI z@G!2^QiiA7l7#{K&za2k>pdljz3+y#RKl@SMxUa$TP)blwmlqqsz`6mwvD2||E`3@ zUZjmOmHN%@DS8*rLbaGk>+Y|A7NFQmDacPRo_wpKhK1e@_uQ_DcsM;9FVrK%a~ECF?({FNZ_n0ZM~OI2@S1x%G3Al}E$^;B0}-|4Cvv)L$)^`p=`#8DtRrd* z?EF;D-1MJLQU}}M+yX6d36_1YXTlAWX7|HQDF(Eo3$d`$t*_9(Vt(aIVc=S~DsGPB zv{p7BxQf&>5o~Os=OzB9kReKkJiVc_-mbU?-;VXSPF#N`pDDf6W%=Lt8zC>Izzm|1 zzfj{yv*z)xdX1uu>bRXiNLWj2bGU}L^i~>vtIBT zIzhAQ_0H5Qy-nO zgvQN`@|R75etr?jSy_iQaq%0l#6R1}@gSEq?mxv#xboS-6W?pZi8zdQnsPexbbbYT z@=tm4%h}D=Fdz7ynms(s9^=nO54d6$M5?iGzH-y&*r-bL##b&^-O2%)91;~ZSpa^w z?hl@?&$FERDB1PTh>_3d;hd1x4kda=!yD&wfMNFPIpB@OaWM;_jsN@K7Z58GygAU% ztb?exYPRolIL>LC$z7@b`A}BNg&G#zdWsF^LlE!d!t@GmfwdJe72cF&Vq3li`Qx>8 z0xArXGQZP?s9EkOIbU6P{9XcTUSs}m2QZ&!Cjfc!@6gG%Mc_YMeTBwmLc>Bpsv)MC z>O?X$U(72S7-M(qew;0-sQ`8on?l-MvrGnPV zSyXWIvK4+a=;0t#?Jq4NO&-M<-Sh?8(-{`e6ioDF@*Ufq8P-={s=Ph~{BzS^;J!|9 ztSVQGDk_<5P8Bl$jM>KMQPgldjRrx-+s|G1sDW8LKEF)A*p_}R7FjcUo9>z)t{E4m z-@W#t&k~$)`p}|ZHLJ1u<>mkUV-c8j0HiC74q!xr8`1a;8s#7T7Q76>#KYVuiVXBy z5ie9_zr9(0d*I~W&DoA_${^e*i+^;lU|=Kp^#7wC6IUNi-s>>P4=YaoS}pHa;o&5362uEBw8M@P(D0EC&goOZYD<#>-j z56lHX|1FLL(uoJPY%A&}havY5kZ6zFqDp6>+|0x~cWDJ#+(abFaW6`ff_^FmxW+I} zo>mxER-K(2JL6)u{V`S1f@@cZ&szr#C^CguvZPFNa+E?qPabWK1-d%A{-UMKb2d9x zP11Y~_gC|D!vCPD^PyH0i_h_FwOpzAa#wZdhr>59RtFEWLZ^x|*QR>MnIy9@9@r6@ z>zt;RnVVkgOM_IE0+ zwl1nd@U~(kzHMkJHy1Tf2p{k_MXLo(Ocp;JQ|w&!<;6ZblKDdc5zvQgM<#x|E0yTK zq3Ew<6M$sjJje*kp9z9FnxJ7C_z-dGZ8;98cock=CyhCUR6>f#0vE;}zd;?bOD=B$v+OJuCPg)+#I6gwbI>BA^yF z@|66Y8)#@-)Y|hR2h2cq3%drZ3Kl6+nG%anu;dQ)#VXWV+iWArVQ@{T1{3jrUPIMz zdwfl8R6H`c&K5K=oL$s(X~s>ZXE)()_Hc4g;|Y_zI*WEvZH6UY2SD-nfZz8^E2J@+ zGruH)ukBsUIISFen56kS6ng5T1nTtm;LE^OT6X~)g;Ci(iC*yQBfw*f#_ulHQst+5 zGr4suU0i;Dd{Fps_V8%CZpmZHV)TZf`NWE3*JQpPeD$kZHsOW_!&8^Bw*@!xf1r?7 z80hEX22jF@PoPW8N6zOCk7dmA5KaMUHevrBCz}8rXa(cbq?^l(ixe(vleWTOjTtj|KEIYfw=9Caf*hz8(@xt~l-4815J))#ytTDe(Q zDaBVF8nA;}PHB@3X7hF2P}gIez~QIixZ|7TK~&pHa_u z-?lWqO@n>gxCcZlOm$+n-IpXbD^hVTPf|t2w@N6C)NA%OtQZMd13ssh>^H$-cC30< z4HaQcck#J9E4Z1=FEYlEr_Fj5Xg-E0!Vo#*{9V!I2?w$H4{YqL-@U~J_4}n9MA|0PJ51!T;bw%wmgl?$p)HOm(@*BwaOYW()eW%Y%Xz(5PE%Pm?*#4fU4DZp}@6$SwTAtlNV(F}iUF z=oQcnP#L^hdl%60DiGGk!SSt5T#pa5=Y$jIG7-E8{NsRoWMA}-IW0`GzFA5#YcmvT zeQW*&`vh=vVHQV~DIJZ=@dW%g6)=)g@^H8`ETWAUwMkHMXMcoC*kxaVufTs^MO{%* zqGiD|#KTJ7ok`t{6mHT$y!;iWcA;-xE2iNTI}mfBNH}GDjFrf-^k7ee0_Lwvix~Jz z1kMBd*j#>sbec5>KXV!{1CbClg)4?=`sG7oH;}LY`}0^}bLBX>^Twg!&B{cHjrW6! z91Hk2MZ7eBn~GPQOe1n-wG*!ZOG-f2Gj{0jIhWT_VFeCzflpoZI0Nq6o3_B{IG7F^ zZ8=_)#|}#DLgM)-aY5h5J&T9?>LFg+&r~x{10o^|=`HU+Ll6lMNgK!GkBxn>z7^vD zY9E%87YBRd$?+TsXWr@!pG1_83MrSE<{ftEXN$B_gkQ9SH_rsot zfQ2}cZ{0~SFU)Zh=OqPL2Zsc$o^hstQF=11qclN9{ zs=+otd4XK{)nqftF@HFFYTw?zP2FoJ@NYJj@S-(yYHUe;X85}eUK9}wPPF4&+Z=IV zt+QtoETZL!M`Cta24;O%vGx0qkyk_vZj8f|vxqqH`}^xY-WZLYO(8Mlpt z52{ZXTER6oQ>YBe+ZU+?+20IEaU}(PV<6t8!1g*pc-j%$bp(Sg>6rhVMbO;bi9ahnWYl}F_am0_0xeEV|EZt=e z7b9C1V6LxUY0=c<9MYh!3b~?N^(g0nbk8Mo-DlUUkxba@Q5qc=sv6c1NC=l3!b*`2 z5&0@r{KexP=w3!onQ^?0vds+}G#LiqXeBkY^nx(S_ZZOV@_shm0F8c54YQ8R zAb(h0k3&*zRP$$~dAyy9Ncz@DRBat-j`WZ1fOHo+59j^(Nt+N;GQCl@%m42tNJE=+ zUeHZN7Fpc$*9p86zmzMn*9I6u0hq4H6R;+Z7~fN_?m*r45#u+5Mk~HaEbp`2u_AoO zF+TiHcYo_xgiMEkkVNM3(2f`Fx|$JT#(5EohQ0U7X{iUx!^%lI@e#19eIn#RaniT7 zH5+bO)3W0L&M&{mq2fG-LVb=lm*rQ6R(^Qsh)D3LV#FKbDK})`Jq)D{iv?~_za22h z8oyS+?IIaH9STGg)41Go8dy>XivA;Djp5n0#agP>aZ$&~IOR+pIWpz3{#lfS6r#%{ zA)SrzCdZ!hU(NFgdmT>^wIoPq-0du1%l){Ua%Hvg^F+4PbgCvv-{yll9b#R& zN@&fZ7yA9G^u4~4!cz_IUtpo7S-ZiI4qcvHoJYfcE#>tZqn=rM{<&GjvQSq#)hI8N zG;6BMb8!}$WL|x0nLURyt;#WtKgnFpHxh3BbPhmdS0!+A{_M?&n!pscjxf1`v~88T zs8yhkP!qv5vZ^B2S*8~;p3G=^bl%KcFsYt2N%A)x&3*3_bY-{tmv#f4{;% z?;Q0p2I~+2Y*;eR;cUto$tR7a4f!F#~E%H04J%Q-9|xPfV#P7N=i6mF8>7qiQ3 zf!^KhE!vGa+S`>amqTvk{0RmF`T1LnQlYHzqtNgDa@zAvsRF`OI`7{yukjnAl1KBK z6&noIIevT95hm-=D@L9(FX-#vc7!Np*>qin~&G#2s>EUTdJ@NalN> z`+ZA;4k5}S&~X5~`&|ri24Fs$pii)|e9cN08pJav3reB#Zi3;QXoqeG)C()Ei!AUc%dB_114@7L1#S}Z4d71meXi+N4{akTk%aAQ5n)ZvDgCE7vvR$zLCGVdXPF&S!2rgGY6d!cIJKouv)?vBwNJwZdJA^RfU&bBF5`G`QD26({C zK<_l`-4{(fo}!SB5*bdWK8*ay_cVHl$*4$(R{syziKsY2N(l$>`(Nf1ieB>c@r98_ zr}GS%w+P?Ma$&skn-DXOi+bbPqEjia)63+iCgl=NH?XMtWD(feHvOB_Ohw%Q(#p=; zDwT%_Knj|WRuqtqjHpq(BM@Ff$#|yNSoNtf{UGoPLYPG0g2L9VTQ|ZUqDj&?V>y0I zV5Hy%DGrutlY1L27b@Zty}@8H{ps>ttI~E_Ng3ls8&?M(mEh1&_~nC%0_xb|6N4~* z6I5c#+6u1K{i%d^Fgg11J-6>|L&)MP?qRZx!P`R5f1qndUT_MZsge_yb?T?bR=#QO zc+FCSFuD&D_tQ6Q160v*JB!JUe-M(xOY~@T2A#I5l$oIJV$W2Bj_x-(DMy1$cS=r$H=jJF6^UF z@8v$Kf25%@(4*g@m3$oo4AUL8p^2a2d!tj!^#pei9NTDVWX`IG=1$6196~t|pxIjn z^Eo>^qcm!pGD_K-`E&49EcQ#x0JHwYvSInKF}vEHln8wUaWcm|rImRVn|qf(#Y|3v zoqR&o-6lE_6F1*bQmdWCI7r9B!a_Lev|>~WJuibr4z+#!0qyQX|O;$R=R?>JJpWRjb$*6kdg%VwQND{I#wb%Q`&1edY$oNA52%OgXUe`0vs2FiSO)M zT+_JMBH!ywH?e799vS1w1Gxpa$FZxRLpzQ*Jrey+-r^g&-|uF4-;m;=Yac z54t5)+$gB2VN3EqhkXR zXkPxKLd3&|=mUEA?S3m9Jp_~=1Gd|3heocyPJ_ay8M)r8N?UwmN5=j0?xCr5^8Jt3 zaX2MGQW)C>$>Y>v_j8eqH zI9RBy;@1_3M1%+4{zNP^5@?%>x`m5+6j_&1ktzA0gz%+%`?>L)E##~ijWLBO= z7Lxxlpx%=DRsO5=yoK+5ejX=a!6Vr2pnJSZ`k5%MPA#Ohy`?dex1va=_)Imr6Wk|7 zz6h+6b>7Hex40Nav@G26d}@yYj&G_Y1P>z1QAzsSgI^;u>oAzg87)x_)$djOG^1WN zdjT9@VtRB@tlZQe$-J8E-cwF&GhthLE}h8?dLykW&?V|v8IaW&@TtPIKU66=Gk>7z z)5q}7(A@8zGUYBT6;6FT$DjL3bv9?Sc?Uf!2?Ko!581kdj3oSi7*c!V^rHUH$mxFA z{Y-Xgf!#Mq>%49pq%{~3^>t8b4%kWDi-O(^D%80s?Ql(m)gU0Z~uaxPiqAjL_=2xg>vLkd83uQ&RsxFz{{m`Ug7y4EED2F{nZ^ zuiKf17-M62B+m~%W@DT`WI>J<4;kbb#6`4Q*n%m+PegHvwf5CtEP1x-am`(nhvNH7 zZij|I6#7{owS>sa*`2j`jL^_t;^KY~)9cdUeq7#+bVkJ#Lp2SbUz$M%r>#t0{!BBp zZa#b@2q|cENFiPlU#k*AR4e6ZF!S_kRjusn8iy-k_WbkB$ML)tu9YU-cbA)jxqFc7 zSv<4osrNDSUu%ahw1j5EqiVMG^?-2Ele=ig7-R2lMd24fbca$vl)!Xx3lhCb;FiZ8^3d2 zSz!t}c4jzpoh~xnbl2pxyblyJ7%N-y{Z8A(H=Y39=FP=vSq%*9Ny(4o_qiQi_bm+W%i4{meXos9hLgmKaLF<)_jO#aOV zNm>*cV|SVs7>{#`PX%=>VppWBhFoa{XsYxvyn~O};Uq)3?x)0W2emg{9U`?(jtV?S zr3$&#Q_zD*b{7y+rM^@@>lwnI5Ot~jwE3D6UE#7xll(TPa~fa2V;$gABd+{d>Nj#5 zNAhf$OIde5NNd5DRw7rFbRr9$RxchOodyp#CYR{eoJy1WyUQbgIhfDYF+g4x^<)p& zCr5+p2zPe<^MuoNWlr5ZIHIAHd$P!ksmt?&=`6QZl=?%+!%e8e^9xD~obC}#fAtCKxI@UkE?>OK zG$P3qyZGc*PPJ(bmnAdEO3jTqv?h-~Zu)cOpFi|1D@p6G(fR8J{h_gqVv`Q=mo*3J z*}Ry^;YxU@mgOY;$A`iDLZjdjm4OM*YJp4$a za%!AANw_jnv{S>>wt&N*Pq09&=fhV~N8Mfn91s6oj5XopS5Ap*QdlU50G1;_$ZS;v zouTPc`Z7{DTeHOzwT>UJo|NVhO5zogh4TBl=^y6Em7(hMQX1^y_>rre=ELr%UOXyk zIZ)>uRd@ILBD`?%>X-6QRbsyQ^BicbYC zc5w$#ip@o%Z5NHi>LvVT+^7_f1u3LexLWJ_PHLHt7WXxLQ3mmBN25fgNE0R3LMh+C z9X+qf?_qDj+<5`e`!W*fhj0|giF(vbAk(N_cEqxg~T(>|+?p6x$L zb)l}_OWV_id`q`ZH}b?AXj*I1_g|FHrA=6OGTkVWisNT!z7)U2Hv8$KTCQ5&2JCdU zn`9B9z|%|KLN}-}L|{9E@)-m!Qp=fsL?tNP6A%mgv{LVH-7udx7z(*EABV78AhTYI zO`-Z1iXi=LLpd?=>uawAAM6=~&{ZE$JT2Dg7%OIeHPN zjFPuiVP;oo^mskhc0)g|Cc6@$&dFVW9X(Y=jsrfhXl4-0TGa3QSbL{Da#H7ows&uY?;-$S+1%qq zcP@7J0mC7-78b2~i7fD!?DAhs&k|D{X4~ekXkx0QEPbcAJ^nz(rc-T#>BTU+37WM?Gh~06fRW7CrFF1>6YNZq{1=TD(iWS&7gRkD~~7hTB8`V z^FGHQeq`%p6-i0(#_#CLuc%VZeLATKNbgwDoFZeiBDm)W9X-`N+;}R6bDkXr7rMTQ zCErca2j7xicn$#`KAG9avCE}cMdx`|RVD-36qH8ztw14eJ@$76OqKG$wZ5mj1aQVN zcAha~*JHYjo=*E~)|uV+LUU_HJ4f&G>=I3vvRU}1%s@IcIEcvInmTr3m7(r!v9><@ z|J9ELoY7Y06-cKOM@I)DFwH}dOGim3AtXV4WzXnq?5 zN&AC~&sNnG-_+|m`JGYo@}QUfVEp}Ervdc_c{(?l6l{3OD;%A9Uatl1kCO+=nVW>m zkFlh1n)PZziZyZ?6&{9o6i*dW#)rS`g;@$#_Xc^G)!wFNY|X_#v#7_pq#_k*+Cg*Q zZg0=ru64??_Hl4>!II@d$OrMtujom3AsutW_gK4D9b_CLW_g4?Ye;{9kQRYyh+tPW zWl1w%1oFP&WPE=|U=x|!r)u)gZFu1qjRpdEqxV-FrTCd0M9z`sZ)8C zYWGX{keT7DQYNbsle3j-l>~~1X+(UHKLd2DS1LNl<|@owzs42Ve+EAsaB8x`*9WZO}0XwN2B>`7E&um%LJUeepz0xVpi6f%&(@Q z2wj?A$eLR5B~tv=Xy=xWd2#>w4o6M1HII9|iVI495+TzB>kb0kc9eR*yKv@C60+z{ z!aK#?g<8Ad!S68vX%!O8HF5FzxW$4QjVP|f-}fa!W%8Aa_Xl?NL{^uul*QNc_Y_&e zq8rz{jAMkqw@Ku(4Tr&w<){aviu94soe~1mxVjS%@RWJs+n@2T5H06e#e-UtuCq7O z`Us2?(uxn#akX&Fo6mz)Nq2KnVpor5$Nhv_AH-Fc$(?UW_W#pfJp!yXd1?-qxtRS< z8>rBvTr|ENBFvi-Tw?F3c6X_90(ga)nY9g^ys*I{YgKne$xY^RI#Z}3yvq`Ytk=91LIG(nI~f7ZxyTKJ|-+|_pcvWMgW^L zZ6gW3>M3re!nP<#zrew4@}>Rk^#*%ZuF*tYb|KVfCWdRk$U?_9=xY{w2&lzh5Q|G6 z7e^0akk>r3zlGyYz6N&L1T-AV(s813-$74Whr&EU0q^y z+0ZLdBY@nh8I!v<*r_Y>%%P3$=y%(n{@jn3qU?JV*!EqH1H} zW@T^R@rJdW@QRh!m61$bvN7M`b)DL)pXGG7w=O`3Fr zvd8|n6GpoM-iz6!NPyh|@O5$p@l7 z1xNq>Q@?&`(W9R!*Y;**OZ`)x4r#)9hCHa9GW2Zyu=OTdF}#QZ=P33-m0piK3DKT^ z>HH};l@_y-u>GJOddq*HVf%SNp`A3~qn{Hpxf9L&V7l@WDVHu~JshPhjcQgbE zlHct*@Po!$Q~5$C+2D-eK*k5qr#bwA1n{rJysLN1zB<6S6kt4CotNKf#kC%k0N%pC#BBLH@bK;aL>wjZqREaPSiH`3W& zRz)6!45MMQFxTYWYHR4cTZ!o*s`V6muT@Rqe6rca?D+ra;~c+3o+)yPiZBGJd28_@-|4hb6Thaa%_ z5g(jf)W(L(lk|&5c0UxFp%=PTZeFGJN1_;Xk&G0|I*WACere`*5L0=$=)&2u9gTw; z7jQtS8X~L0P>oacj?7E5Zfe?pXz`qjw*AE#&!eGXagDJo8#ll0zp@yY_lr>g!<{{L z`TDC6)wL%x?bhVxC-FRUO9XCLJ{R-s48TJHG@Qr`Vg*cv$qo?o*Vv^DYZflPih3!% zHq)5w(q%|9-FJQT>Q9!1+n&$gXD{wGiHabE-MIowc0f8SSC;X9+VP5)34I|Pz773< z>XQ`9WfxM1jjMQ62dx)siX@fgJ`b*EsdD`eRZ%m*&-Z?rsj&UyyiV2i1UW>pKK&x= zWL?C!T0&!o^=(3?#Utoh{$9N?tT<^iW9cs`!?g`8bb!|1&|f9&SlNk4?%6?%-t%pd z=*=;_t-DpQ*)0Z*)p(8ulG_Di0@ZA`oFH2xhrNB+CAI`2!>d@tFPc@eL{5*VTcvfa zCL{Lz4!DkpUsQ(O)Q9q4s&9_zvcxOf^?TW%nPH+$arXV$>RT-qyMbQx9>30``H9a_@IPAtI?cbRnHw!}O>I zfl3&|wYT=fauw)$n!1UDPFCG_1JWky>x%p-px^HN#&;KlJLCjcgq}Qh;qBT5v3VPtm_A)b-oaEcK}q4Lah3 zUkvqi`4db4<^Ezy|4A0BcAf^s$8hh{te?t>6!&#w;CUBA`&zuxrF3LmJ=@x@lrCE$ylzUoeI>vS!>xrW(g%AP~LTuT73t#N7F%`hRcSy9%sR znZeHB`I)jiUe;EdS+?%XNH_n5< z3FDvqv7iTtmR!8p#Y(B~9uADGrADwCLbJ-j9!Fnhtt_P;0@B4V%;5~gePIalkMYx{ zW6V6VzgUbUe5gOS1pw~*ic`S1-ET;8LtJ(um_KIfRFS;(P@1eyWL(KUKgw3{>j3o% zH4gc};f2=7cpW(VS7GwJMKy;q3XJ#&k3 zWF$R=%e3gn}S+ESv}mo(>bV4;`#07TSLLAZ*`~ShFKb< zXlBR>*CZgu6`-K5s96*S@>C1@&VVSmBs5te5LUvpfb+W`VOt;eoOxQ zHZ9dMuOtLJ<0FD~#NRaDGv}nU7F24_phdSj+GZU9YkJ^W2f)K}r&Y+d=NSX^X`)V1p8ctdNK9L73+_sMg40oO z0^#Q0aI%@ntl)V{8Z)?|uGv-VLcSg4;EI9>yw#i!%+Oc1yDtEuU*oJhyNI0PZ?y&j zKNLSuiu1}YT=6w>$D-nCZsmlBCwSX=J;L7``lg^y{Ux!vp_*+)6e$j5CAni1t;qtyoWzEvO=**54hak5Ntbe!y|& zv25LrjQV6%lpb6da^LD>-&9wt<~t^hq&>FhPKV<#S0OPPMVPAQ5IjHH;bL_O6?gJc z<-DK^fPKHkV)s4yw6U|N*TA`@K zrNn5E3^TnOysu6d^Eh~7$@1uLFf56Fa4b0wBkwSNBbyeYo!tX^kpOn=zcI{@zzR%> zalS8L21rLqUPzN^@GnupnL|=!9aHf!dSA3H^cY*DC8m5z^tw5atbFO5qQS^!(BiH4 z5w!LCia6_ z?h&K`(??3j6FB2%xOP=|OCYj)K%#!>D*kPj+eX@$OaQEC$%x)mY)f@NMa1{s$JW?} z9P}wLJ0Pl3Hsm|=WT=7JlJ{+@Ij{%O>~zkMXFX>80xtnNFL8&su)hO+qMacm{c>!m zqaZIr*7@Q5Yw{iuWcVc#PdCe8Vb{EItwQTW^JhxbtgrfCo#r&k$Zqi9qw=WW2Lh;w zR|B8(70nizHqGsJ$61@Sy2G3{3Zlw-2Wuz?T3OI%o3%YcMX`Hn%raK+rv5ZUw-kjC zB|}!`dEET$iv;i0v~Y?c1Fk>L9w{j@*!0D>l@q`7ci!dg4J{So{0S?v7rqiI5_x5-qR1 z_d}km9)Dqn(y$1O{%T2T05$CjceX%7_AF0hXbCl#7O%!YbbM&xL$VZ^=!L_`jiTXn z6i=UrnO^_;I^-A1i${V$BM&GeE1rd>y33O8_kGP(Z(Q-D%d@9)p^46;S-uh;>XxxZ zq1qu#OH2d!PszwZuGN7CQ@&tPA4NxlDv5S-vvjTdcID2}Wm3d3ND_1oz7u+4IGFjF zBWzP=Z4pp^Ijht=e6p)@6M;W`SUba2-5L#A?Ih9Pk)h3@r>+HMG??>)jW<@ z($1}>IwIZjLeo}WRL7EYg7$-ssy(8+4)F^A`fkL1MWPJ){x|}glS`J&qONV~TBPuK3-p-|1?w6g4gMg}lpcb$*Pl5h zDvxP(p0%+|QeSzfYy5dx_QjxTb0CADbW&}A^c9=!6YcDpRa26vUqKTUnj0EI*7uw% z(f;FFZ!Y*F%FNc);ANo_#ovpUI%rqP#qsa&iKFY9)32|a{Z$+Ba>8LglBabKh8t7I zo2ll7>o+RhtxfX1PZC9^vMMs`;;$^Zs!h7UeZRp}=|jp=BnOJULgHokIh2tXBFRtr z6S9Rn{1LrqN$lLtlNZBJ^$A+0n$p5uEhz$b~AR zUBqn6g}b|{>rVr25FozZlc&{1)&=e*;y7!sLp&1dAT3UVQsnbOk_)Mp%V8uNOL(P| z2}T^RqHCULa}1|w@UQKDnP6~lfJxqR@1Kr!C8wG(lQNI+tR;Tgf3GR;CQ32a>VTMg z`y&Ps7`jcQEP7}EtVj_xc(tG72mtfp-2_G&uUpRT0tUp{!)BZ4=o<(V0Kon^IU6>>* zT%T#H{2^@$t)>Z#>u!F~2=&&Q-3$kK_YxZsiy2?&r+K@*h0N~WIok;d$ ziuOO&K0{?T7&rH zWK-Q*h=qS%@I^Ka@5p0umJr7r6kFvP_`reL_=VOeJoozvHZW+e8L7Xuhx7a6h%bVX zir_rx*6Pm3QA+kx7+zcu=K`1Gbotn(|IRF-MPmmFuo?RU@>P1SMMEVTRm4M%0gA;@ z(jDHAzy-uShV~8UC);*WoMCuJ$OSbj;KhLy8RWv<+y#M?J1gqt+Ox;>W=0gz1&Nh) zQ`I6tYSr`t^|_P8+Go*=62)n=qV<^!DOFChtR}Upy_rXfy3#T;JSH15w#@lrdPSC2 zO23ZL4oA?%8yLXR98DiZMo@XHS)L6N|f&jj9NNrmYTJ5jcN9nPvZd}DU*fwh0l zwN3d4=Hbfrtu83zL$??B)L zL|KgC;sP4#kvBnk&k?dAy1p!M8RNI{@@OI`rDofb8XdC-wq}Oeu^1-^Qt#`)k~gnr z0oyass|vloVtpp`F0PvZ=+|zDR~hlI-;f}~{G3T1@Tz-TT@)(F1KaA#wo_vwd`h=o zi^P8+!hT^W-J`W6h)@aInl(?Cai=c;`BG5366=D=ldDh7iubWtqo2&}REzEh14l)- z^l(l!HM-!fW0C@Ug9z>&`Hyib@;eeP1OTXJu@#}A!Ui?0)sq{2RO%t%WF2@Mo~iES zyGcQ~UDNi)g{XIX{(}t+B|rQ);Y*(gQN@Avr+)I{joVgK-BK!0%W>a8r|p_c_KXEb zPjU-9z7_+H6yc0Q4*!_pPC`G*C}FpHTV^2ax-dS2**I#6x)(amLcE~&`$9S5R%mj8 zUjD!|$_D(!U;g1rQZ^DE`G!?L(8U`)=?!*fgjQuN-xkktD!MawwF}N znCr`B%t{7Cg9HH)gSlNB`c%z>8FMa5FCek|_-8upi)>u{o+6kP5X z7g(NMYnY_Z{BCc9cp0_W_Ay0dXDFOxSapGa`G63!tJs&mO$R!4QplXxVRnP+ z8&bciLZ0%^(>qfLx?e7oafbY1?V~Gpx_k_#qS={hcd4X!+BdDh@21M~^f-SfG7*V$ z7SRytq#ZrOG(!5AH#^1O0}h_O`}vTi*z(4ygbBsYbDZ*7Q`vB`UEjgIxzBAV3}vzk zDudT|Y2{g`RHAOnxXg~r*dA)JWfd{J=JU7(Iy7v^?jayXz0nEcMJfC)dX&EZ4ZF`Sc4{pZ6fr*jEGgFrOH$PB3DSi)xG4&edhT zG3G*FP%Jd2c>=Lsl zO556Dv#A{Pv&R1z+_2+b1-Jm8-H~HGJ9GFc6S`ti|6bH3LO#k{U99R*u4|H3x=T>-k`R8Xsx z_!par{6m52V%m;feZ1w)HSY2_5aXCT|0d0`!AJ9XP)_E?Y)I?9CEKe4Akhqi&>*~I zxj$-iE{2qE64bf})J;CrRCx(R{|>eGFvucwcRxZb)_ncqrYN-ezh3jBP+LdM_bI%~ zE_eL&O98a);9U;FZ$Ze3g4bamOjehFx^kQ=v1L4MYS@Z?Y2%MMw3GZ5)wOTIeh-zP zcMIl&>7mEH8(UzL8o%;SHq(X`A*gWOJbG{R4~9PQsgvgE1i;w{tv2k#76P)s?qUwv zUmu$Nqs9f#r>f$>Bf#fNk(}VEGdWkS6qx9#CprQDH(WZpBLB=ET2onNuEy?z7};{6-+3D{RF&o#m>hKF_L(kOq+PEG z6?;#oeoPcT5QdMAe{yqgNf$S;dVHnM9wA2c{(bTRVqxp0s8F@*`~~&&|92a+P7m?i zo7`)O1_=$A7l33;Ap7&VscU7e#q8{127ny}dELV6s+7s|O<774a~3-iF`JxkQq>m#vp~ z#Hn<*d+~?U;Ew5s1^O*r&$+R}WfbnbPmE_5YKLm7nBDEd$%6jP`^K^#+?0T^ZvA@X zI==Bx-s+LV!HlZpU7UO>L+(xOrRCli;@1KKkAp=#qMgfAx{}yLBk>_}<2yLvQ{faO z!#$zT@UjLEa9MIcz5z<>AFoMC-&u5{{cPf$Q%$>@8La(f!-7|&CyV@u&>&t}{c{U8 zbQYI8)hcFYy>HMgF^I;SS)TKMqv4~% z1^*9wZyi-t*9H8dqyi$KbVx~uAYGy~NOw0P-5m;ufNG}1=;oYn zgFf&3y!ZS2{&UCOW9VS-v-VnZuGo9+x#s+>CelYgE_i<)4mxQrN$3og_y+JAM6a1c9(jN76Bc^=Hxk>(RC~KTMT|T!^;~St$=KLq zwQ9TAmSoQNA@6?7e|Q9fFy|oJ&g~;Z3s+EYeXw5aRb#3(hrDHH`kZvF+0Qjc>RS8$_U+NyM)`|0qxW1Vg*asid8r zjJcoaWVS`cPQwb6fAc&4ZBvAZ=F*<^_ewqKt4o{>_FVZfP)CCc-=5b}6bcqk0DSQ8 zblqQFWBxzsS3G$+%uvYd0e5nR;?V;J#T1m@saSSI^QJYt%p1 zKN4x*x=s-z*%66=l|UC?o!u}xzMQ}0`M|~X4;aDN=QMoJd_Ax?d|m(N57F?q0kfG| zMd`;)>Ln{+aSHd@5b*@ozcLUt6;)P>`ZlbRS9IliC4!219y|+4eSP>`wy>#;QtPCa z)e{KyEkdVQv-<~ELG}!j7=F4dLQ~ZQ1Rj?uAUe1D3=ORjW;d(x#GOqqJrBmWaYV}D z!O7#aNQ+Q}?ZR7+^U#8U)9SOot;fL&7-{I)bYq`liy_xE$vqD{B4xzTW36d`7wB+GuL(psJpNmHDc^Os7UDX7{E?{DC@&@ryMe0z zO+2r5M=IoU)V{dk`XSmseHF9EzUbtNQV%in)_Ps9)N7=x06nLWm;P#d+Vddk+BM?@ zCqx&Wvd*j5ssQ+K@>UR~AHNWY4`NPsVaD7`aBJx}43n`iL-{5lGcrO37A`Fcc|y$`Hx8gkV$ z4L#G^g?GTt;marJY1OdBygh$4|BbwXo0S8?+JWm@%lhlE%5EzZz)Fq8LRC;;&E83}JyOLO5%=0^KEivF^RS;~n zGuF0NLFd)gU0(IGzG6P|%$lEiR9iPheLKlG?cq}P9y*6-4Rl_2y>*8lC+YgT91!YK z))afL<&iF*ex36^5T<09aaTC-POpc5V1^v-6GAS&mdqA(UH5|g4(j38_8Q}3{L@H~ z&ek>j0myMny0E7IerU;Dfy`_)iAz%iPMZ7fjitdPAHUOEq17RKxxSlD*Rvj_^=@5? z8ptU=apKIZ*HU!)S(?utof+k;C*rBRl%r5R#3Uq{LSXmsYJ}+&626_cTafD5H7@G2 zIo{t;cv#j0Jr3>ZNoUgYAAFh)Jr5h$tvjrUH}Kpw*EDdhsTa{_@u~H=h&`;8giFAE z=bxtc86MPNmYiSHWAd)BvHKsFvzxQubZf`^?0)gBO09)Oy1?yDLi(%A9WVMs>sJkm zPKGTdo;WEZ<*s6+?cL@SCE4yxgV5}OPTkQ#k3HBA36k zWQ#aW#^*@6p5|2fY=Cuh0#82p&if!Bg7e!uPjlEIQfLx~^~F4XR3B2=uP>jnq3^t< z?Ugdggk3GFuG+h@)ofC2)>X&q!S_^8XBkfzDqz2G44eu^TAkT$!qi^mzf&O=eO>|- z&%}seMCiiJmF}$yA1dV69Oe(7`W`(|B##-_*7Q9aVn^)f*7;qgN~Oi^f8*YvXhg$< z(O?f_S4Osy*o?;4DbAj;<|_?h$QIBiW$bSjh;qrb3a*qe$h0txUi8XA*wj6=oXw+- zR_pvvmkmV}Q&B(nPhDQB4m2D$P;ROKvkTF}k-f$3JykY{+{%KaweIXYTl)ihTlOkl z-A&ST-%ZUc;W^)`i&Kv)rv0!_t;Ah4t7&6jr&QbVTCsoy$cb%|B< z`6}3S%ZD%_Uh~h?IO3a?oqA!tae5b<1-Wy~U`TxLXlILNTFp0l1xU4sPiHr>J~cmd z+)u}Zx>~-J@JFUNx^z7V&+s?p_s#$@0Xjr_A8a4yrS zy!tWviI_!}nV)qf)UhjeTwm$@qkO43d(r3ED84LX>*P820aRWgrJD715d6?HQ4W~g zG}hB}rO=jo61hD4fc8y?uJBd+Q&mm0!DtLcV)ia9kkId8LVFS8HpkE9zMJL*!Oc#a zfhQ-FH-{HPynaH3pQs=8*B)CB4MZvQ+NrgPPiXZ>Ms6v4ky;)oo_G<_?D_r08s1}a4Mv5&t# zh5}=SZZ)1;c8rN%!&s4~$p!p-<(mE4=&=f|vl`w!edfI*ZBIJkFrw=-Ta@+gACi+3 z5_q(WxJX89QGHL$sB(nTXt7iXSoF&HW@(gvm7j`DA%$MXy|UK$(7Z%Vj8BIz<_|`I zr_CQaDoZjRR!4=jZ^lcboCvQQctdwAI&y+m-<%3Z>t&nt(bjv0DfbDA4lq76$mKP+ z!ehF3%7Pad#q8_<*opZvg9E)Mnz~pZ28ls?ln(>#+V08W};-YLQk|4b_uMS6ZL_rBcCNUR_;J4SRDPkX`>wBtR18cer5RU*pN z?B!UH7xqpzKhI>Yrmz$}8by~hbxzP+9+IJ;L_m`6*u|So(8IaHKcEsMg`{O6-re0F zZfW2&Z4&(MlG@Rbv+uCR{yL%OVR*2O3lsT$ne+PUqm&kIDR75a;1#p5BuYiOVwX=JsJx6>g?I1w>|>O!M>Vw25m{R4Op8f3 zSFk4lPyY#Qki!k~Ar_eRY5amGlN=WhPiM`#rQ&rMGiG=*GgLn|`E(^522SafQ^GcP zfAGN3YaZealxaq>*c!e^5c{l-g2iqls+FSqGQX(|a8ST;8dIN-)?v9ojmsKl#{Zy_ zn(hhby)B9x^h^MFFalc#k7_p6T+9$LXRtceNMLph zIPydat$xhn)I8>Ons;CCV29?HDsA60CtN>btJC&bl_8w9w=Jk zps34zK;aczC(<69%$cEZhC`lH`I@UB#hE05bxa_M2*^SFa@pQ@TNj7_Fvzrm7wfu? zWsx81&>}=n1n&D{R+FBs5?9NJwzz@w6h{`K8@cqItGM+}sE#!E;RDKDcJ!#rI#_ob zrhL?IiOB};hQUqY%Rm4xg0zz>#kV#?nxpAB@ggtX!Ozt+cU8B1;XQm^B4_(K=}ViP ztZ5iU!!~j}g~PtF(~~=qpK%4*3-0g2gt+~OBwTB9R@>ob{yt8l*P3r8-|T(qQ|{Kk z$>Liu%Kri+*8p)WEVO{4udV@>7wORlcYmXK-W*Hz>CUcdUyA2UtD98XA?TPlE3VBK zhVe#W$T+_1rPqMzu7UzkNdIcAirnGcThwbPcpf|zU9;l$pfeTW8eASGHufz1$^#~u zclLc46P+x@r{lpbCIN8WomjBtd?d=_-5`{?(&VxQv6tE7-Hg-|;jPFzB#PawlW^{= z&Oy+|N~ERNHp~kPtn4=df!0Qj!p#d5A`4OOWS-r53fCjN==f=tdWR<XoIY=W22riFD9u2Rq{1=fGab^c+1iAx{{UQ|OvZlL|>9(qXZDnRBb!j~G1{_f* zS_(UD%E%F|I}FvGcb%de`rsT(20{wo!~KLDQ$=|Kara#)$M1*QIt=6;xo)m@;-x9- z8N?Z1D|}MvvF~E<&X7i9%Bw3_guI#a1Q(D1D=H-(D3$FLE*qIrP()#_{ZeO7rkt z6ui4{d6<>^O!4B0<# zU414NC8Wp@{bz>9X`p$|!cshA+NW>FP(n2(y5t$Yab%D6w1|nX5c{%LPdr#ZdYbJ0HFci9RH1~=7=3iv=H7TZ>o%5S_|-1r%MqQ8SlX{4 zSaePN29u^*K`U4$L`ISkoFs-O&Dq5`TL`>6WnWo%ZQm0Buz7n~h9XOAIyPdTHN;qBR#FZ^8xc9 z6i*a$g5A$6WtL%0qVOzU9QvWl#6Wh4TyA+Qvc*-=?%N?*gO?izezkmd`^}Qp-jzQ` zzG38y5~x!(yxWuuw{%Ht!WU8YJxkOKWdy#xfJGT34k$&_O_1=4Ln4jvb*pm@jM~E3 zRLxoQmEpQwqDU-mC*#%LlVykZ2-lG7^{cBALB*PLw&+4w-#qz*%;gGiwi;MT=SjP9 zhuP(TQr^cl;F)%S!B$CcHD^Y|{*|6Hj1O*V$iLYv=(|0babGs#tL-zh_vXv4b-l)Z z8Hg=};f4D}LA})6rXfE2wT9?}?T}*MbwK{Te@9uuUo)FR%}^<-lj zB-?|M_0v%W+WHi-9ORts35b1UX7db~^wW9Hgk&p!>Bq_%YCLwtg@_P86U~Nera0HEB#jVbf)~;65*eIgqrv)7Ai{@>+<%{FOfTzA{5JN z#XDP4YNz$qy|wi7?f)Dy*4_(E=;UrOUyguJ4E+9R%a43FeS3Q$#nBIVtaPtfZ8U6c zOB7RYE+WQGnKm!O7^8^G{h((PTy3-z$B`o(mA_GdjEu^JoAo|O1psaZYuL9gUbpj# z^}D};XrfyX4d5KS=Ww-KRZ?8+L;?JI3)=x~#Q=Y6ur{!2ij!~V5Aiml6h@p-cos4*fb4&Pk>O$F3&&gD&QV1 zGzF21mHCU*c`+bL-KbyqsJB+*2 z-aCcVuG*MNXL`J0&y@`n05llT3vPYN9cc9Zi9-+-s7k6(03Go;)VzrbP_eXw ze>Fxm`yq?0wRaa?eDx0w;NqVqrf6Ekh0j8-+%bhkZX0d?Rs7#Ob`q*iz z{vBZ+D71gYpuQ{Q&*`6DTkajyz<68o|E9Sc9}Boqw*oY)0JX((C!IM^KiYSh{24&8 zQWHIRcMASrIsYRXOQ_VVyB~WN|77>4zsW7#o&~L*Vn7jBP()RD<2>SKo(3qTwDqr2 ze@5RXE_@VvsY$8-_%^+8C+YloyZ%H5qd@~250&aP{gxfC&s}Fu>+q24fC(rFu zgMzA;rjX@PtSl-}M*$Zr38s+;UUzD$fl3+ld=A>A|5HOjul5?w=8_3Nh*K|~VBB86 z9l(IxMx8kR=Pf|aihPSGC4W`)%*`mTigpaAjP z-gYZ`T84(}w10oisB8o7i=*gy!V=!y-qw7Z z$?aWtx!wLD^qRqowX9qw(_CpjSFS2q#1}h`G&A z=6rE#Hrk3hKH42_z)>{TWICY*&(7rZ(|XM`R|mqyeS0~{ZfC*_qTbW`tKaSq?+(-$ z3$WmUchBvk2VmX-EV|KD-vURji|zBG5E!iS%oEUVChB^~)% zBiFr;w3-8sQWzvzqBIVCcnl_}7)_?uIkeyP?VA8`F}Kt)V9ekC0ldgCK)*W~O?S0? zUa_ok159RXv>BVpbonR8vjQ3gUnwHxD^5IF7S}u@y%nr}keeq441com{rQtirPYQo z1e$1ndJFiwmFrevBc0fhEzKeKv?RkMSubD$-xUCSkl&8*l>aC4&ILf;?TP(C-p!_N zP|AW_B$7MueymSYy)%j3la4l0=N-nk@pH=Xz*EJ*{w$tngLA#zFloT@xnE*+%#rfq za7akyc79Be5B0f}089rD$F_9*KOe%w<1Bw19XB(^`y?}C_OwEhb?6P0fX_}T7BOt$ zmCmyi7v+ZQqDS#hPf6CFK;wbW==j`9;p^<9Y{dRd$(p;XJHR>pwPSwfS1>OqEAA`A z3&QX0ulGIjl8HHDc|lp$$%yyGqRBn6>&a{6Z?3V{cf351UZKQ7&q=zB|5C=vV9$Yc z?Ajt=0=YS~*Q_ZoeU;+Adlre9(44#cg&ddh2(lXxsKAHVCY;+xihmjxn&%C@c(pi( zFM1JC1>J54C$DthA*R>&Sg#rPfsL=(16HE(>dUcH8NwOI#-REA7 z346U%serlQOvVOpTao(COw_EhGgLBvm0ga;;_=o;wo@GdMiaNcRJ?!-TId-v)sNau$g`}^1JX(B?` z{~_=_6M%Yc#&Q)`!&(*R-1Zkc*v|J?8?zLkdi6W{q8m=y#%1qrjMoPtYAxfoWiN~08>*Kn2vUJr0K>T%Gm@T z3^D5|O$556-<)*C&p2O8L46vjLfC67C|$guH^09g`7{Gj$9AlOCSkf&$b+VdNdj0^ zQY~HIIMpgYAq?In-n{f(t@rly7t+TuurxZE%CYHAPNxC_LIvii%CsN$P7|s%zYCT( zdy{+J!IB+xWVu_3r8Cav{EdjS+kTS&f#xB$t=YSPNm~L=8K^KX-i0i)8LC8{hI}o~ z^EB2fmYzph`}#638p2Uck5-)7Q+-d|6>B&4+|#@c?uFEwiVahz?1$;)Pe2=|Ie*X6jR`Imhb|MZjF32w0cT{I9VHpTyy52di0&un|AZx$?X2 zuoJn?29bGe2(H(^r?`$BiG~Rxf)v$qdMPh^-67L{UcQl*?Ds7CvY#}r=2`LHhElWd8NW62W?z&jE#3ah0iA^W;6+gz`uae_JJKqU^*7(llJ5G1of zWvYF__N2-V*n8xjSMSCA!)oAWgq#}RM?YdFo}^Nd*F{AhrNq$Uzr8}_BCE18<)hGY z2UkKZVaA(Z%?97cN$%9)%4uGwNR*_6N==rw6eQ+)mKjm^Y8JB3o^TPjcans-r+bc0 zgS_Tne=&O`blgcZR3moApm_-&sLj?4LR~A%GrS%-5sd=i{6_^&rtICaJAgrFJeuBL zWvL-&h0wq3WW<@>k?v1Ck5&2P4Z2TDM^z2K#sBaIA3Ons)C@sBCOl8Ai5boIl@S#sl7eDbmT?m5Y z=npU?!DfU0VY-R0_7p?Ue57h3gFLTQ-l?2yB|0L=+naK75m4QMh+ z?h6AIsqU&Atl2ew&&0HyKy;2b_eMjctJWwivGCt>iiX^(qMi zHBllNiV<8z?$hchRexsMk>;I+@<>j30MU#Cto$14Mk(081T9FHZ&ZMcN}_S5vN;%x z(*}ES&YlqYl@3jjFgMv zF{&a-s#VI5FRZ9WkAynat$xWdXXX=-8$S#7BmSb%WVpdc@YV)vF#Ke0zg4G}X6Bi} z2EWJ#hH>XOPf01@T|C5=yzpCI6{=P03lQ&b7U%~$D^5nRGdVVh8m~(e2anW2f_(PbDZ`ELOjkOlGJz~yu;DzjmhKP&ulgxWgKAhdYtgeCIAAW@P1uQmi5zjM_?wdlvFL%z!mzL&3GC>DcUZeKB!+T3 zg;vBu$Xz>eVbNshOJ_O!aY+ojhh^)22f(A)4xtB6!||(iT-BtAZdYDo%HdugLJFd> zYTmmRJ8RCwuK(^gL5NITJS_kA^Bb^qjAgvHEV-BimdTyF zV5M4lyx=>GrM%-GjHQ#^B~c7ViMn*{w?{A6dwpJ%Epc_KQO}v{=>Ty{M680jqWPaN z7}}((U!%WPx%9-`OyJS{_LOt$=S6m~+JJLzEiSGR4?c5KK0%*GH&=jtyb3(?o`tG# zf=`Pz-Ot`I7{7g9^OyM)E9G-nhU=28T+5r}fF% znzX8vFpq)B5XjY�UIJoMSfj-N&}*K(Xl=yL%HNoVq1?f@nN1y~Au;2UzYDb+Pav#WR7OpLDk0k-2! zO;HkC5cm!;^NQz^*!QSkz_wtnh{06`v~{%M#LVp#FZZTrn+4;lxHY5SI|W%ITeNS zc^vTCG$CEXLNo8vsfjiF&;}+HGDa=lG)a-o8@lqLLZ5x&@C0md z(FVtY3Vc#tBPM89(Z2|%Dh=?G^sY{%V0OX>z0LiJ5{DRoQ@l%#ElGojSJiCcDp!Bh zxyKy85VmZBKAf8-3Luj^(=HvJ)tZ9O7I^*Y# zLUnVFUdEudjC->~)?0uz^YYs@b3uQm0i_3`-odkE#sg-8im={8dV@ccrgOpCW=TX_z{Pi$9*>+A`Dvp2>T4wVfS1NYUt}rLL;Nmh+$lYEa$iXd-`V`A*F?@D zC=K1Gv&J?2X;}-e^RDg7LR4UOwcB}BKDu8=@c}oObAzi*O87PZuFMSARp zIt;r^(+5}U;NnAW47%XwUE7rBGqxbvn0XoyWLZ7B9pwKdioXE_W~UUVcVQMCDJ-BnE1lXKlI5LNBX)DVm22n)@T)8~mae_h+3Y zNo2oicc=U+A2EavuEEuwW$me=bz1;hOo{@+`6znVwoR~eB^1S{o98R#rOTW9KgA_G zwWQ&H3#32pY|o)lQ>0@8=KgU@lku7LiTKY`X!tWJq1s;<>HZ+42s> zNHhC*EgC#vi+Tr9)JFvLMuoD)G@@%aFhq-kpKKmj`ZW|riiAliePuxdcdPrfC&;e& z*A5-rxOX^clB-Q0U?jgdjni7+@>6QJYuzP zR@T682U3jlvmc@jUkr8$!g*;^!(mM23v_mPo5IT|hHfRuH#2{hpCg&SXkNaW)OoF+ z2;M^}{OwriJ0S!RN-;3zl^dLw0Yd2jvJAl=uVaFlJKX3MMkFmTi5cyh$j9EMLHm08 zvbr2UTNnY2F!^$+`uyc|daab{*rW6GpSKfd7dtHDsc<5bM4di^n#XdQsS3}4@VzXx zwhWd0X#m*~Vvm8UOD~+L@1m7@vdHe%HtFb)mT7n_>uWROa49 z=bKUaTg>v^QHn&-WOGsE2NG9m7CM; zPE*NiNuLN6$Pn;C)fjta_c?UmmT%1DsOZBM6zKWSvy+~@u)ab%2`u};MUtcLWUivd zJ$ww%vF{F+0I8@I{}%|xy+IVZURAg+mjHBZlI&g5tOZYTf3@Jpecdhi*HE7h{@$1W z4cE$)A)vn)HcBCTIWM9O^b3F^$@gmZgA=oRfP1LSElaX{E&2w3ZexlmbNUBgu~c_i zjj8T~;Lka-f!k&Pqgx)qVfwx-CCO?Fpzq%u1dTKsT!7$2MD%!}3Cwo$+#p-mf?{{t z;YlC1c$qxB1Fc52YlY|9Db+xm?Z}t&oexD?Z$Vcg7%_m8eTs-6>y9>YPF9JqnlodU z-7X_n=Hl%b`BvdN-phBnR)#Ep%F#jrEGlnlx=eBjIB6d}(>|IeC{>q_xHKx9s+1%& zT@uA^uao8ZP`Fl3u z;YSObBy~>uk0MPp39{KfcJ)BheDRfhoBKUzqlX7wGB4>+-agN1)z0BLT9LAB`we|^{@AMcRbIt{qY`}`-;`tM)ED@t4yDboJ@_%Q?F^} zy42u0d7e0pk-q`=KAHBl|E=)otH`MZfMC2kjBtgT+ZXHtWgyd>Z(BItb^!e$n-{B} z*QIE7Yk1EEn>i_WV^il}ti9{7vLaClfV3C9lWs|KE%YDK-pX4^`wF!i{Nj^usg6Cr zd2u8}Oltjx6Q(>rJL&ux2Y_|5*+m&|lP>xBI^|yhq9=?&uc(FpoCTm-SOVW|((GwP z&ZmFwK_UoGJ;9jW;@C7u68HL(3@N-l(hDUkk@UGX%g?UJzZXl5c7W)qV+*Qrdc(qK zvcI<*Eo%MU@DXrs76&Z*e~$mx{GglJj7Ws_VOzCxT)pN3Y;yLRvN-#aM+g3&;wSuV+|YSh2J;SIROnL60Ia<&A9J;hpO@AL6FnpZUsPyQpY zQFIf{4%x1XS6hg9Llu^k=0ICWh}wI>kdmf;-ww zUyeYnNAVuqxkSVTE9a3M>7V=v5@t_;?d+5bFh;$`wVWc9WTzfd>A!zuv5Kv4?S9-Q zT5CY{La5rThxuU)9nNN$WD)S51&&4KKx8zEEsa!!gtn5c3YcwAF z@prdBVqx~!d7;p88G6+W!zy>G5v^dWAiL4I94fQE*9E1O#Wczs|4qu~6=Y!i9e3%2 zsayqUm&j{Y;V%e=ox%(3m@zu#p*UW8A4-2dUu59Ek8N8-YQ8kHwK?nNy^JoHWOx1? zcV|-h2H?QBV8rFS3noiZg$$Cu*o*WC6OwwrrDM5I0}(YQzRY-hs4khYy16!D6l6qf z<{hN4ed$sr3Sj;6;;$7N=kw9KyJ2ZVEWL7_h{H?hO)dK3L^C!Gs zN7fLKyvy_odJfpfjl?(vSBubnYug|V9398Y3{r!;j`+w(*fSr!P@G3Lyok=`iazzN zP0LZyhNN#t9Edk-GPiDTz0xXaLv9V+9!5QD9RFX`3gMzOm%3o`t69V`V85k3_t(Zyi!6S%&$I9wc~I-EtLHpf=) zg{{Te&G-8)lM-awlJ*w{;nB-XfFSJqKe*)ZPw+Osf3kDR5N$8e zJ*oTR!ep7B zIo){5u7R^HU9M1?^%Ab+PzB)zeHcU--y9U!cl1MBM7ZQ9jYFev;`XG6((<6ws?}qf z3_mDmNV=ol8xEp4ds|KZL+QSueRnj+hkH=oQKuEsxm*d=%gjf*1h+7j{7atU;@8QN z)OReFdz&1w%V3p=?}ly^=%?0NU(z(1sXCFKRd33m)+uZ5>quh4yA>F@)2ESv7}XaJ zr@0^FKkt2$QW8)r6Y@4{G1%5^=UVl&e)Ex(&zv_PHrB*i7nPFnak8wIms8qvytPpp zh8Kw9Nk6FZdn2j0sav7zB!2P5x#GK3Wb^3{Z~Q89un@`j2z6EXzdUqNu1zkpVfBeh zXE|zA}zhOI67hkuKSl&t87u zw$tz1!f`GER<_r*_362Ycz%zk3{Z^wW*!=LfDzx4O~yt}dqNOn!>no%WT2mnL*f)8x0W0n=%% z%5C@d5v-5*Fe%A4k&y)qWV7EnfF+a?R`-XXeJnrea_7#F8kmkL06LMx(B#BYVlqKJ7O8u1vv&ur(vjs*w!=Cb7C0 zis(^1;C(qQPP8P|uHKJ{DY!DSyw=snzN`MOk^@q{9+FAq z*u(Cs)4|J#beK2IgVyhg1-q`umm!7j)5)`E#l{jhH5u)`ngA_Oh5$JvzuTKu@dIpG z<3S#deECK)jS}|SnWmIvUfRL)r=ja9A~Zr>-==!7C?X1Elw#RKod6(5KAt@3Q4d|Q zWF$j_F0d1Z6OHM-_37&+j|k}tPyey;*FfIF^c_E_(^0-%om?t8(fQb8HHf$bZ$4tW zw}wTh((8_^)ZxCNv`-vfpg2-k@?@MOR-SXO%G>^jlI?{?RJc+xm|nges`J|1KRyG+6cR6(R&Z3BdSvPQc9Ll{QOtQK z?Q7QL{tNSdwbQv1rymUp>IqXpHE-U!ZAWLz;w2WcDSNmPGQr-Tj*MESsa5A%ItHmV zs4*q(TNrShD)M~au$YBDRy^NZT-vk9e_Ag}(Dn3Tu%6olbNG7l3{9zEk?zZoRq14M zxdJNJxH+|}-F~{nHH&rDG&Wv=_eQ0YMFpmg<|WmB4hMcfR9#}kYHeVG)%s8aqX$X{ zi*0_^88F=~(kpyfH!b)+z}5rBJY9qm#PQ~W+Yy;iWxTc~VVgNe+wJ++nGGz^lh1=f z&Bt~3%9B3{GG+dFJ5~+*D5cblvILhQXgTBC_dC_MkCzis z6X=eSej(H*Q&Q~0)f2fjYs;GiSI&N)da}|$l4@rgMj_U0bR)}m0gT2STr%tkPOg{M ze=FxBqyp;mOBf1G*V^#MB^mxX+SHb$=RI1lG(-H_@)1_Nj;?<rnEI8oA`$V7$|DnU)*M(P%9yCz%DG8~9-Y zuChfo_4 zc}eo^xQb1eOuGo0$D{ZgJ6MVpq*3`lQz_L!Z3cBOfV%PpIM0r)*I-NrMVT(XN=IPJ z*t$tFiwLp{qFJBh%QyKd_Nsj@vZQ`=1UOiU6el-gz@8eLRHheUrs*w^2jLfX;Q|7| zVp0vK3Qc3S>R@Jk%GW=u7y;^$CK-y|K>o~L@NZj{FBZmhKO6&UHm%ygtGOZv;{x6a zvI|vN1QQjNZ{invCBA`r+aH!*edqr^d`cE>E;r$!?mmtyLuhj08))4j<*Kap`%|=n zCh5z5911lCJfQ`8+N2TDr_q-l1LIv8n3!w0BM((Dq#K9Xo@d#=tyIsCLDAj&a2e?t z)TlLeS4TUAyQ674n{*rH1+TMWlu-+E zv&btJylg@dcNmcItY$t7m{hn}?JP@v+(3Xyq<613R(k(!ytSZ7ooTjERIED_E_PX% zbEcsKMj$d>KtHZKft6CG3*j~RygEro3camNs_^H+Dw^&<%gj%^E?{H8qo_Ut9o?#Y zjXpjCzAN1DqAx_F+o_U8DlF2E?JQpDbUjZ80&JqNk|a{Pfl+61u+s0+hSIC%XPfr{ z^BAMsT~gs0XN^>X0dFHBF~hEqCxwWjz%ji@%Hh$%jUeTokbn37^exfI_=mUe+cFE- z`T_KLnRTHBgO{>XB1;)Oxo+N>DsJKhfRGmH<*2wMv7##RcdxNi3^ zQCFUqht}TMDTwr#n*EJIfv>=JQm)97B8nq8k%QHWvozne=~VlWREP6xRrF7ttBBJU zy>dA@<-VGQI*8(bom^_~Pi=uYS`?F<+;sb8?pa%G^}U`d$!%3_)0uWMRP%20V0580 zn7W!&;SQqH&G2CJLgR}V>>d0Be8B@Ux# z!5uFlmes}Qz{Nd3_*KMnzfRnE-qgwysw_jDYQME1I?*u3+H63zMKT>w7Ah8QLEKLtykjICizs3(+^jkr)74~d0m&q?>7oAdEo=G7&!x- z*O_mTN>E;Zqj1QOb=xzGAyWglD&LWrzWibKAWutqdc72#J;#3q7MKgsX76m!b;y3d`a}i4VH4ymKj9S0Q-u=wp3M z1}iLX5LfW=j|vU^G4ZW*M&V$UfL-zvVT(Eq`OCz8mGFG|1Vhm8({P-ypmKZ-L=|VS zr0%5+*hzmwv$K`VJE#jp@=Rwp{EF2Ihm50N#m~0lgYGjBha>N^-!$TOGn&;GmVsbo zE(ZSf`=*oOlGn})+UkNVZ4DcQYrpbx%*I@7dMg^zzSc?`o7yJx`y8z?$jKaoZFC}) zUJ3}5Vhp_T?nsMUm5y9U`k2o@pFgeMVtGcLgYzWL&|)c|-LJBE{TKdeep--SU7BLu z8oTuAqoZwHvKIppFHTWZ>ULycYO-1{zIg=UswyqBt*4CvL%ZyB_sjK^1@ih;-dcaL zb%HDJvk@Bc_cI3=T$x#4I%dM}$1Nj;Ad5n=BHw3}z!V;H`C}_PuQTy_G-FeQxy~$<2#)xwpkEol? z-kOEiWLt`PxF@`z zi)MZlk3yAc^g>Bqn=_g8S7ps7iobQbm_&+->Fm(^0TL% z{`YhgL#@9&Y0uC7phWooe($8TTC<^ z`Vz0Y*j(@KVH%u`=!>#V{o-!tI)E+rmA2D>c-dT=B03znz-ewUJO<=Yjck!Y$U6Q zisD!)=$5qLU|z3RfLhX8aWCa-x=zpcp9P{!_A&=UFM>1ON~8lzpnGsXnK1NF|(ak7=)55vF!oc-MBL{6Z+6- zkFa`%C{FY6K4!t#+5=HpLr&{dBvG@K_W>WAtB2fzH~Hpn1)e7bOT`Htp93Gd&RZpR ze|P1PvvolULJtZ=v)urX5 zaq(>|e1QDV0DnYr&n~V;-yrQH%C=oxD=a zg_O;ITISiFkE#7thAkBjU;ZevT~}FHdoR0YH*I*~Xs^LQpIBgYmx|X`;%NjDrfOMC zHpe1=0eN)XJwXvBR`+TA4Fb7bm98McA5C?f)LR+N{D_)poM}OugGgPn4^I!wn3($v z1y;}X>KhqQ+{#_$V8itskz=eq z^_=&yl+pL>z5yS0tRRk>>!i{Tn5@a`f~%N##gdc!(T`~#o3@OJPSEuJBwbj?ZMO!- z9n$enJ(zOHv8-ESm$!m%Qh&p{|I^*I_%r=}@yRzo$;z#W{8A$cDa>Vrh)T$@^E%IY zzfaN(R1s|8oNuyFswraXyPpxoBnn$|>QF;O5$k*$Z^plUdT!J0Pj3E%G_;^|z3+aD z=9R!^Gi5i{si?O} z16=HwagIjytv5^T)nt4^LaTU+a#hktsN&GaY@gOmEhj@vB-*`BG4ZN2{3fg&MYI0l z*XGQ2_{+y*UX+kvkee1Fybfqcf!7H|W~V6l(TOA@?XVs+Y|*^4{6iO~>kGEHOvOS~ zLk`;V@JVnh)Wby<(NaN@#k=1lr^c<(`E9`sc{>CB;h%FqO!jGKAGUlj#GG{z{}h8Y zC%I@%gzLYw8*7(emc1j?p$qNJw7-^CACc^b6(-R8@Sj_?UW7D1y0mm%hln5Mzx*h+ zr5Px}9O!=(VRc0CF{q-*)yu9ykm-+i5nZB1cfTAg$E@T2O2iCiIxs+NG4XU8Xk~w zzzabpk@5580)^_{Bz2(VkyqIS=K7Oe&F58pL^zSxvhahjigZk7FKpVv*e+>X7l*%1fWtVTsqh{zK(9lV02I>g&*_I@s>jU^qvI-F9La^$wK z5I___tWMpDIf`Mm%l9WL)HP8n2Wmloh#d3y(JJ{DzOeIvqNVd|29p^73pee43LX#} z07zFgY2MnrLXRWktFFh9*cDUOi9#3i3qtLde(U|vDbuQobt3ZsK zRY8^KB+lioHWaZs|3%ZPcSP*}D*uTJS~~BMvIsHZC!D5eVuZPr6N$K*2Ab4nVYrih z-$!HfhRjPfrsZQ`K!?gt`^y}lL4fzQ{mf^?4$vK6-TP;w2*X+(z1O`-b+xScJnIS0 z=Fb;0Thk_gQX$d4OPniJawkS-7Bw(MlLiBeT|509?ZUj~&Z-38KIQV(Oq*zp*woC% z$t1zt5iJZ(X>|rC^*;Le6#7I`kXz%F!!LI1ub_7!;f@Nw;gheL2qPe+Qw0i9s;vqMKr0h8=S8D4H670QLV0m6Cw(30~w7ec0RQ z`L$zcB^`PYLl(}LXaGo}xkb@(>iD4^f3zPQbr>16B)TrVy$qNoHbiHe543+tlPgBs zFt?qE;MH?n=bzv4;+Wy6Ozpj8mv!Qe2@%acCW~9v+27B?9Bbhw*20LK(WATt?E`*=L@%-zN#vV4ByOddW#r-~D1?Ah0HaIxnD>~;CR zC*yy>sH9jL_ESO6p2Tv;XI658pPfX3RBQ#&5_*6&f^bUQvwD6?sw%?bf@)@tWA-NsHUFeS zU79(^?iQZ`9mPK;@5L-T2l#c;5(NPOV*GHT2be;o3D-tMBWi9<4;To}}yTi#3_3cF^|DtRg;20_l4#D0P_t~@273Ay#*j~>}wP^@dY2z7bnznz$0g!x$d z2kitsLk}jsf`Y;s+RD$o?_(F+e)uPokGN=$eSH;hLx0e74U3FdZYekK8us|0UE5zL z&yx++39ol%;r0rLdD}6BEMGMyPG0l@M7h( zTZ6MoVx-E1w-{KwjvI@xYdab}mdvxzL9yET$rg)Budyn=zoNUp)vJS><9wOsV(kheg*@OG-{;pt6Uku7wE2R7A&A6oc zBDjnlmG(jV!WXq_tnIZXl1#VINk7 zF2I1kytV1o74JQGJ)X0D&C}1+lBQ0>m@&4bK{PJnRY}ZN?)u}Sjpg&*4BV{0CI6V; u0XG@!=uJ&^rZ2-`qQURaN=UZ3ZFiG27dYk5)29c@yZB3&mPoF5z40LswY#eOujF5G82gj%>q3xU%LP8=GbUQjbyE?}RMs-JZGm^Em zHZc%4dp-eNA6<(Gu+ThO`a2qN$sH76J;75D>BRa?#0|85poWEiLVm zeipFp=M<uMhOz?>!w0E;oF+V>CU8Oycm?#*fx3Yf?!kz8awU)Tg6NZf zU1TC~5;y-m_#J%z#c%l6=uHIjXJRlgu+IZOUUHQJH-~@6&93Tyo@m{9`Q`WWt2b^I z^`}P2vp_QjfDrmCVZNTJnOQazdL9u-7(|F|7FgHT2!#_kVNfMSQ*9yauz?|yu`WTc zXfmUypKq5pB^cNUen7G~Jy;JYE;#sG5hNq45I8u9CwP#j{imJ^IqMVB(%O#*LfMF^ z+kg7>1^Leph)LG;M}P%Bi3@*KasfGML(o!MNO^w;`Pv1E4r@*b4ww2Xp{j4MWJn~f zi=||2{xZ!p9(u{N1teY?ZAByu8z^)YA6KiqI{R%Jy+yFTte$$Pp-Y-NH3-Ed^E&;$$$o!zKl7SLkFClV8!G10 zUoJqXfMPh@M2BvVq<7ucEG9PJBZ-(FWe}jB!*>xwSKJNbZA(&5EdeR$P&GL661~PX z6Ea}R^cV_mM%)J=_Mhv8^kS%h?d1%C?d2|on&Y~;BQH4`Al8ZEdttM=huv*+pUUvE zO{)iqPyn)kQc{&HlXI$eqFf*Pm-h+^^)W2vby*CWzM?*8z=X0RVW@a{tWl@*7%X9T#*9RUDM$Ct*w`I8QSW z6kSWUrwn`?L-Oa8Q-mW$n$Ul47XTHK?$oc++mEi;oUCc@*%?jwyh`z6??p!nyFd!2 zIuNWkY)C+5URamwfefh)st{209p++cp!3lQd)))u<@B7bRm-F&5dPdv(h8;CIC$IF z{gA1PyJ*TYQDkLm&3iWUZ|hFzOtW68G+QxBcOS8@n^rjEwni8@TIr5qiDw4ojsJ~_ zJc<0^9uFs4T)ug)%vRe`!Fw*%^!Og2cW$s1k~Vyv96zM?lsn}_b4#`&__tMwZ6+*O zKJ3kp96ZT&AoqA|u53ULgryxiD`p`oig$X0da7W1(fd80KF^`Gt`+OZq_%loY>*S! zg2OdD1&CNvMb_suoELKbC8J=meV1q8{mW`|HQV~U!3W=+8k4mp4a!pCqQ!M90lB)~ zdAn>cC!J^>SD`A)qlcXp?*^c871>)Vk|sC#lcwBm&rzxI|IO9TN<^1eIP9}!hov)b zMeXcbVfTCDEhnA2BYPe)5M)R^-~I+m7XqCP)}cD_&aDD60zE9ahAv{h>NwL{T=loo zjK73hFx3Z@6Lm{|D(_11d(4&?1exG37p?C|5^%9vXhXNfz=GrI%@Gu?Uc*v~(3g-H z5e+RU>lAu`eclwVrs5TsV$suR@F;(Ci&P-z^y11G`2T0|M_x0SL%uxIS8RIgOcl4B z`r3G0Tcx7b8YT;Z3dstjF1@{b^5a;jn8P*5zBj-r&#|n+d(y*t!Z}1U+(f8@{bidX zxIm?8*-ACMF+ElCj*j0e3>i{?O6?#)#dKDp9Mmk^Of*z%=0YMVSGItG2Y^yhsX~{dgr8#Fi>`j z1o4w^fSKMRsTOHr!&MKP)`6T?@SLj)&N8634Qu&H;cp*~z?YEh!Z2li@tQtGFPS?S ze1%X%_L2mV!FR<9D)W%P3*wNQ|#e$paZH+u{mgYYAMS>P|UJY9}LKN?Ja0Yx`86HrVV}5G}3AQrOY&e|7ELQ?f!IHeb?1J+buc*~=X>Y&Y8BFO0)DJkZLZEXzUW z6LR!_O9W$3`d4L(RrMkbJKj;MDz1;e;fD$cG}9)zQTOf0obsx8MIEvv7V{BVK?;~C z{tKB9{D4e%`)0A)bSRl_ge~DItS#1w?ThW$Ucs{LMvUIPPb8?yYA*5@cq3s*2c&BJ z5LpQ@>ntU~0tVM01hvvhLX2l}g-7uf<%^LNtoi&1C?u?ySN)tAMV8LB0vfE5^>rEU z5E7)$qX^e-zoS;{*%N3E&&gU8gAXvRy#+(A`3r$|s*&3~R>N6$lvo-KTb6-{r5Hru ztGiE5FGhqX(LovLlaR?~hAr7EA~I`~QO<)~MrKCy7F%)8E{4}p5}P!hnz>+AT{A0d z?B+CCe~A;NwotYRI%cpk$M~HOs*T@l;c*Sz9l^NAJPTw3X%>atUC*}q+pb9g=$t^} z-X$N|p0OUM9#;6D!`&I&+d;y=(DzOEh3h5rf64bO5Ty;lyUt<-*DPY(_2y##1?0Ma zv35M7@72Mg_E1!{Tzi>- zoMU!YD1uyM7R9SKUOV(e4xVoQwI}q8%~Qtp!;PZu?_SE}1SgclJkleHi1gfDSPX_K z|Mq6VU@9$(--N=M*QGC+QoAN*@9Dj1&<)LQR1C zp8ECOEK3@r>(LVrm-k>Y?hHvp(OO>hZz)U!s*+#mqKi-Pnm{K0M0N`o)r zU*iT=4NX<~S~4^qu<0QpLL^RWHcsNg1Xr;BiXmYqt#1WO@Q!dGud})aw05S#@BN206g5&jK*1V@krs5h{KTZD}B7U8!MjR zCsk~lH{z|I=qQY&e0Vi`qdm9R3c~8*GVjK9KT8S_kk;^j&ZX?VoV+1s@Df!f*L~rN zZ_a4a%3`#&U3O|52Td$zYo&8=Q@l!}o+g(G9f+_nHrVo0%9E*PS|#4Tz*#9!Iz-8 z+v);o*i?pV(&QFA0f$)54JzrLdj*3`yInLB<;xRvd_lj*Fhh{@7ftZOg3O|+{I4s? z5NZsr(?#3)Yp5pOZQ5c=R%w9bm%`uKoD$vVsY*5x!>$&HVpZr#dGI2sKyS>ri<`rT z|N3bo&_U-D%}YZwPlLNu%;ibh%2J~vPA7g22!WN+ipC=G^9%Twk&H$}H{)Jw@=wJA zvm8pq8h$-Zt>|9pRVIY}uyTbpIf1_a-PV9A2<|+&?EgI>AS(VDDWi6_q)UZLq1_%g z?(G({!}zASc#{w+02TxbD3niK2>BOIi0MowoOA@WFsVSyU+k<;6+}5(a#tvj!}e-+ zLjz6Gq5d*Gv(GRl6Gwh`E?PX@EOXWqmhm=^N}WyvJ4Tj8VFBcfIFw|6)R}6!=fUcD z{nDLI!eIA-h`5}0r73?Ysl;@pkP*v4+dEeMyXJjPUO-XNc6$sbtvz-VJ<<;pGLJdr zqj(}kKM@Z{&1E+=At{qK4TFvo`y+u1(OIRIU3zbhW}`rkU3Wm2c~nsHQh2*$1mpObSe4ss|MbIzso=Bag*0IL6Oc#A@EG~9 zu)jd|#~JH1CDjGG!*kzgEvAM|y&R8;IdPe<)uYTsZ$zTG9z8a)J`7pQ8mP-O!*p?b zyARc!uTqwH!(%JpcE{N}<;Y`5?;BHP3@5(!XAG#1Kbw3RpDQ$3!jji)f{}cRtRAAI zRI9e*#dAAhkzoUYIXRS9)0__aBYz=vxZhN9$})_%6hfQV-Ta!ENeqdLDMjRq_)#SSL!4mei2ecQ zNpd;w@t~w*OHM7{CGqdyo%B>;Tf8TaK?3q6*x^q{vB zZwllts^~}X!?5*Egz|%nDnY+TbhsjEcbj4_=&0|I%(18lfmj{0FLK?A zH+@TeEoNDE(uw2g;VSDLPhaQj`=U0gplrqZa9dMQ<>J|_e`l^`$E89xVkaU@NGCIk z_KHa|6nENErHF)&5{QATO@z`L**!AMLsTeFdgC5e{ z7~6U9r*u9zsac^_^F2jwRa8ShD>906;YPKM(A)c0W{kNpn=6 z!Pa04g_zzTK5PnMXTIjZDYLT8rT`yqW~OquSxJ4nkRKu#-c}uiFxu}D7)sU9g9ahn zJ+NpL6A>Iic{!ecx$V*D4sL`vXjALsNa?%2Ct~GQp;fvdOfgNl&&12a@DkrIE(of=}gjZil?rWT`G3Hj0|vY}b=#k{KasMf`x8XwRBSN8Uk?cXxy6~On{8+6WVhq>|AKyB)srS?%D?*K@^2JfA!p5ty!lKcHT@OEz-XD&JdKJ?g2>W zpj7_~4nN5*E=jT5n{Juj?$K^BcP~j8-Qp9qB-;6#VmsA{2_DVgyTUybc%##^W+waO9kzN0s+`NE zWfN_VU6`J}iWMw3=sJt8SldTRsQdGv#1!4N>X!SWI!juQRJcJ>f>pl=>)oEk7iuf{ z>Nz({}5`XnkeGro~$@q6C@SxPb`DCU8 zRe7*1_b0&4*a8hcIsg)#o=!5nuUEavgcnw6aP*ngTkI2BUDOrkX}3?Kmzk)It#x8-2f>IT zs_0we`o!I9`-Q8j+K6vi1zAMNkpRKePm%S6YrINPe<-XUA7n^pUnjsG@k3sfbNz8l8T`z-JRKSh!8DeWj#VzGB>gbL1^X`b(0d(f0GKTTQ zZ)bxYw@_AyG@o_Bl#GegDq^9HkmZ-@a1FGGA~8F4pxZ^uxk<60KW1!3Mu2_YZs8$& zK9;mPLphV)8t!|*EMVdcqc*`+JXE5$`Zld^1I(236~_t4VrFuug4c7*i_HVjI8M@b z1uyF!PsOU#Q1KvA!7plP-ROuor_sc%9Qa;`gBLGQh`f0`DtC%`^z9$~jOpAMU%pTZ zoYzew`_07jTmr@LYQOQtZ!eIFFo z*>b)VL+{~O+z^W5oAp9~)A1G!#j;qub>X6g7!m?m^B(km*nm>Gb-etnIx*1MfZrZ5 zl2m8lV+#@ziQ@?%`}Q=+39i#PR5(_%Z>!{N8n0*88J{*>M~?1^ULIPXXxv(wYGgt4 z#CUhkA9BRZwrCmEaYPJJMQj4RfUSovBZ^6H$9mS`$iFOu8C7`LP)4=Kg>*(Je-&{0 zUD;VUV2|*Y>xN{QWNy~wTV_$YH3qg7$rPzbh5fN^tdakD@qU{T+ru4eLVft9#a4ly@Di>6|aAz>aWxmbtAC$?A z2Jq!=R%~%7;J^7g{R-@V#j2+ThLOL+InzigH}Y0GITahfNSJH)l5U~*l`;Hue2@!W z%bQzF^F%2vGr733$kJarG73?wm3Y_P0#f^CXk?arv{7`}cs;=DctpKZ-^WM)hfK3@ z|5iq@Hf|LvhlQkZC1gnHKXQR_haB~vUJJt+U@p7D0$ylZ<#4pm|3PP~+OU?DvY_!~ zLD@H#P-;zYOkDgnCf?V;5A6d^NBb*ScG~d8N!zlnKPrYyyLhJ79csXgE_HCXCo&@7 z?A4899sRA@`%!23JYoky!w!Pk4EbEIRZcKnz$lvyBU#Hxh>>f;1MFLn^q}Ii59=3w zGX+9e5tO}6S|zSmtich<=X|DB%6no&W(Z6XJqDE}T8rU(Q>h+1d2mI(%+&^z5oEAS z@yTGkAwf$z(LFme@JghnZ_Zmm7W$Jj4$x^%XR)eVi|qLZ^~KP<_O_3SXcQa2X{-d4 zpD|0NZ1CLQA~fy3T*X2F#cJ)6VYzC4kjjgzjsOQU1FvDB0x14;_ft{X8h^l;VI303 zO@(t?#QfbeMWK?BuQ9U0!uRwxTA;g@!Q)wTIeRtxy@18UnHVV`5j1Kle&MY#R@bIq1=F; z3V%A6wC~>N%4AqN)ScufX^g8VMlONp)hYeEg;2$}N6)ydeXk3sn@|GDX;_m-bW%{m z{q+I&KyY=I84ZOFgfo0t%W+hCdHT~S6;_g=fz)0%47+N70i5;(`WFK!=l4|bdNq+ zAU&9$v*{Cw_I?MP~M z|1WWFr*>jngW?wxbsMkZi162MKNah4zY-!+M&LD+Xc69oQj(f{&zb7M_v#wH9xj=D zsMh{xoMN9RP>G>9zDQ!TlGMXs-&_qO4{RYh{07O642cF>)7~gSevs&V8bQ6*-`#7( zd0@ewL&87ePUw&H*=S&L-B}L9QrQbUH%^<}X8*S|7Us7EM^m7cuz@G<(eW6^cjX|< zHmLGl?|#m3B_8A%<3i@qF2iq@brhxiiy>itmK8wuWINl!ax_sk`WZ(d#MoA>wcCV2 z1=W4aZ}SZod4|EZT*mWX^z8EQy|MPB?IHA!WhFH`urv0kcHB{+#oUx}tb4DynH+8v zlzW7bA=N?k^qo+=k%a2q3o-sQq7ew7-W*P!5v)|{XuW9YaZ;Qw)_y-Uf-VksA-c$T zc)uy%zPdVSepKQ+_a+QlV3gENA}+#cN@gEJjE7~_MF?LT!}UvX0`CQ*mu7`Wv8NYr z=?_ixVcLMCJftI|ZY#=L9z1A`%vgBCoBrJ7>mfqPXWX(fm;IOL2Q>gqe0@o|L}#r! zbIcFK{khIXn@zmYjl!F?RBhw_H#jqmISqJX%R4URW~jO~(aKzJ{gOHTuN9D1=$M$H zg%r5NRVB4(+Q70#n@b|TmrVmCcOVg8ybY`IxR{_|dV?Ju$X!Nx6uK721{~#g6T=-t zJ&w$%-bODYE%mZWHMgmSFO*lpkZrX!k=p8gQF#SmWo7%9Z^2GQ$!@nwQfYKIVV64e zO|4b6SuNL0RPUm$)}E($Eu=7$|-YZx`1IV%*l>l$d{R2Ss zD<&pg|6MWj!qKqn&_Ke!tgUx6rTMdafUk1PSf|pGqH7u z%tRxAu*am6aYzoE`0b^Hkcte1vgr3j$I0k7(444GI!TeubUEGsVAdU^RyF;cLcPVX zAr{XMVHaW0CjiO&#L;Qoj#%_cCDOs0nC9G^?f;gbgOsy%GGh7 z=Zdg>F$C^#tEOA0F*P&iJ0p7Ele;Yf(vKaxQ^4rp5o=^2PT&A#lMg(&gJbde21(6x5P(4me>dovM&5G9b>e+AFbfr~Qxnb&7uZH!VP-SnYiKYSH9Zy{2#wEwtI z4l$&=+hJ(iB991dPoY#(qgd+NMoGa^UycsJGFb{VTf!&07tdYfDo@|{n>FfL6|8KG zkFd`Id!8lU4 z4OT<6O*I2@rwZBY%vh5TAy6>Rr`;WZ zwq`DIPky;RbEc8piE-GdEZ!+Rs=;7|DN7GWIWJcTrWn5if%-}bIg<3Cd>o3!MvN!mtUhz4VU1f@+-Nb zW-H5Yc4`Kd2^agXI|$t`SP%18Qoi%LEA%4PiB6lWfTDn1T&0CZBcnWT_#-;UEZosd z{DpDd-Wr;}344DufkuL#jLYo?UfN4G{zxJ@+Y`ESnRa+UtyZLhIV+txJppOb2fN|P zz_x>=bMsiIuykCKY`7-4Rpx*dD(_;T??UKSNGd=tKUngP&0X^&{{$k`&UZoa`^>$-t8Q7wWWxM zZB#!jJvV=dws(2P*ANr=|9d_$q5*Fg!Tg&qkJsB(hnCDA!NMSuQ&$iD7>Pt}p;w#z zy8wUEpDfE7sf72iK6%$j6~%-^RN#je&OfV|bZql)*WmlA91zixLS~h=K5>&W=0><+ zeKUg6K2HwZhPbakc)~mbTUsx)NNctJ1_q~pSG%%mE+P~k6%M2BzT3iE;O(m%C?hd0 zyTy>>&2CU=Ts@p{;l2@;=+-plL|KkYlvdWU5VcXvO#J`$cim+h?zxyA%gUAU4~3F~ znoMMV6A}3syBicKE}{Qf93&ISHx4q3;|(-E*&3^xlV@h5DTP3)FZ>T`wluhEwCG~j z>^Z-9TE10=@CXD9607!wMb)EUf;mBXh6tb3Vv5PYPELQQ_Bm~JI^c+yoR~9laWRP7 z>R-h7omsyQ-dsPMrxiFke#E`I?@Z4+so8pgFD86EoRv=KSkl`a;({7O9p;W^og~}N zGRQ-TXEJ=$f zxj~#|iLZmL;c;}Ugl$e9$`jZBFP9`h;o#uZ+C4a<@cE-sm`$i;uR{Lx=OUCKHOaSj zBeV^hfnUCN)|$a4wgSNQqF5Q-5qYo0!ekrAzn@iCFG zh$ykfOW>OQC-bC+ZYJXi;*991sNT1lxXXFJnc#bUX*H63^07XTs`Yi<}GYgVs)FP9>rDP5t*`c?(1Ex zRXwIHjaue;pxZdCj#S{wwFRCzT{o~Q4}$2o!0(kbl~mFkpTx>O+uU{bEK*9+(T#PD zI_*l$qtbgZ&;Sdnyq|d?XAZM+AJhV9HT>Hq((ACFjpkfdfyI12?&yJpUd`#p4#L4k zn^Qf4QMWuYLzeYq^j`8{Qf7#cS88u?~gCRHKf}fgQ3cPh=&nWpcU;; z#h$Dn+N2T1Ryo!GoLd?Is>q7-n+AEwY|HsXR80Y@6<}}lC8DLY4vp1)bz~22K?mma zWk_Xq_{wA27TSG5*x<66HteT7T{f$GCC{N%`b~9M$KkNM0ukE?MRiM{jHK>?xWP#8^E=) zXtY#wQ&w4(GryM5H`D_~W9#I>DM~;QXhQpR5JpOWEy3`P8rJ>3nOXT5gkYpW8!@e7 zaomk25Ot=2gg5YQ_k^ql_N*{7IQNPCXrX6Ksl__cGC^c6g$rBS&eURuO|9Zzh?3$y z`+qYVfXvUTs@R~F&Z;T^RE)rfs<*|~%N=s`!A3EikVr9@K;^H4n`8F{pS@|~CmH}u ziKDjcU(sSH4dla~)9ovApz6jwFPn_|>@16ZkYYsqj?6h(Y}X8ZDUc~fIURtLweDM0 zb{)0{N4W|Qlh^uzP**4&4l|j{rciwIJ-48NDmot+=9E%p)(0)!B4uvWyEluV_}6X7fh!pOP(*&FUrfeg3wk&+ zjL|KpleGHM;CLKn^s{~PeR5^Nay+GG>Zu3JJlb$X-<_REa;=S#<57zhB(%YtW48P0 zAj=lr|A0qe6JW`PZQ2NzjxG5CJOOMVhwCCeK+}7qx_`>7Sb|NqWeugTgFNHOfM%IQ zB1J}ZChP`Ur)S1nastH(?g~MsCMX2RVp{BkJ(Zh`A`!5)O0?x}%WQW+pGWr}mSBt=-4I0P<=Y^x#Ow3pm`bGp zZmBx*qY>i|e)D-BE-TS0ExlL>yOF4znu#D{hD0FCDwAyXs^(Ku z5UFbQWd-7}=((H;l}KXI;;DtDkT9)08d0Gn!P90!wTRayC&S7n$^Wq+g8=A<%LWlA z{EtC`1q91VsM);`=TAH%AVHF*RB#KasCmoyW8xJm%4>pc_or#YdQZa2#3F6TA5lh2 z9%m`tLAKo2+AzZ>R~zt4W{|&`G7TiA+wu>iqA=>HVriC1Q^Y{@BZIF9MUuzXNb$C? zgM_1rGZOWt*ty>)n5+9i1(5&|nRI~Hv*rEm{%RR#ma2fnRof)}yK}##8AHeLHG+D{ zwFXHzctrUcMM6;v1d;sNZd*gn@L<(J6P5iO4)Qgx^4k+LbtuO8Tds!7b+2waEZ349 zqd?!fdfXOv8lm)>_`C=tD7BUn*heO_!lG{MBdEc7aGq)jPJ@Qra+EY?1?>eXVJ+zS zb1yvQM?2y+#8P-cbm9ItE$*xQ5AVqbc!QGZ)(Jw}nIUDZemGq|4w(~XjSD0)PgW!(A|Sko+}6Kr#6IL1a7i3pT$oRAWT_5rLtqHyd(dCY^(0 zXh#$qOL}?#`l$t?wVFv5;W3duO~1jhqIR$X1EaT}YX)ywaxwL=NzcnF8_5Wqt5Xpp zgW;fbI5UH~F%ZakMY3aWSwrpRUq`87Kt1V4^~mKFIiOUxj+vZe+gmbj54J~9jJ(fV zOZ{!FJQ>gu4ocp*yca&M{cLivOzIa?1UgEa{^h)o+8mzA5JiFaGcosNFZz$+TIzC7 z$e}v}>U9ofYHniV>SD`!6Yo0BEyZ~yh2Swso^Mo={p=FehF7&gw~70A)DDd3b429Y z%0=3O02RJkO)?lc2tpV%L-9(FTrX<>^>}hD%FpYHoh@us}54 z;OLM&(EVk--85*{l~LZp;>H0Zv7!@t%gQd$CXdtEjRl@PvaZWoOum4LP@>aFLJ6D@ z>@InaIwHII2^(7Oebtyk%Es%nVY83wu8II*YvY}$m?DP28ta|sL*nzNq{#(x_*2)F zi>&Bj>H5b-I(C9^_G@14GMCp~ES@GIv`Q*&7?>Rq^@Mfs94|zsbKl|GQ>v>;I!XJC z?&KzsGZeJ|QW%XE0F}{(G2s}iKLa$IK|^OM^PA}7>3AZ!s8CNiB?Bf|;p z(b?IZTEVJ~`^$svXgcSN!8)UJ_p*Otzp+*;Nq@io>*CIkBBQ%i*u-;;^6K@`23NGC zz5p&IxJG=zSoi7_h<|q@40^7pPpS!@8_Z zcDvBaD=QGxWSdvKyho&gR!DX|r!e%bK4vpYB}{AT1E=46X5Oi^NQxt%znfx6Ex8aJ zr#(ESk`AL~eILG<9Yj1}d{!Gcq?RN$QXGQ`TDQdL7$xR_1>xS?Ia{T!`Mq1lq?)$~ z5qqIEN{zAcyxj$f6?y~aH82-Jhz~1#fohH4Aw%&XBfFrXqP}Rwh^pt64y>Tmt7>KG zC5ooYr?(_Wd*CZDNd~dnLaYB0l7v2;{Kydt45w>Q0VP8v82-eQU?%TNfKc#u^ibn% zYYwcSQ_XZ z_8=1EIGV>LSJd*D0W^hwusvPkPj^Sq3t77r`apsNZ+D@oqRtC%C5GEdfD0AxFpYX% z>o6XjPXa$KDrzm>KQrNVS3_8E;=;O``P#?JU|Fnm6I|5@KbO|lO7JCU#5*#pLamMj zilY3ufA40JdGPb~Iy=`nv(Pm*T4B|QZET9}8Sih~xX9mvI7Sfr6Ir4P3JQZ}(miGO zbce@HJG}l!rA^hSX}SD{*~O2PrBY6OhQ&-*Hak!*7(3%BY_2NR_=GcZiVoWheU$B&NxyQEqTegaH2|(d^!yyh)%9ZK`G5w=?oy&%s5EqUUPWR z+Zmc2EnW;Zsy3Tz_lsd&Xs0QGQnSAX?W?-`3E?=94z78+6h&4u;y|AmghgrxMBn(kT>a~+LNfU>rA%Th#>Ie5CM^atAPH3EfovHNJrwS zjpH`JYt!_$?%{=L>jdnSwbIXix`Dpjw)Go&rhM@DPalTh+`}KrQOY&$Hm%o?i9`4i z83m7=zX3$#?u{E#N|?4HT8bZIT_M>{uQ^Tnmo?S~`C^4DZns?iNu{4u4=6E*Yi|UG z@JnPKk1$MM~&5Uqz;o$-KI?7%v)wXTV`#_{M2Vp z=okHCIi=U`bPeN->W=|un&g^RL|S^NL)x;Cl4lCc^K_H zA`u`a*g-@k?QTP~!1NuNiGzF?D-VrB9mh! zWm&{AZ(CChg$WzB^^>3FC)`n|Py)l^c5RLcU~_FwNJ?NI_||y+Mmyza;+ilc#|cmG zCjC8IaaTHIw3FJegK=o5r+x0$A0>10b}C^RA~AO*%RE)$F{UcCw@~@DN-bOKE|*`7 zvvlk0RYG2qGj|le!XQ)Lh&DSQzN&tsZXrsO{_IbshvA!?qDG?>_dX7*))R`CZwsfk z`9L|?_4sgjoNYx3*B~VAax;vdaG!$s9QnIqbx`}AW%X%bokXh_$F@17O5@}b_*Ssf zqSm0uJE-k&>H24 z$lyZ|{8r-fp$~m{JdiKtOt{~gp5u54qJ`Ma=%x&{UM!$s$#&a>xpefb`c?D4HNTVy zo!s7j(-*0}noATye}5MJNW*sy#w31#>4)cj+GUg(>zujg9@}WNLill>ehG}PB(la+ zVX}FMIp#nElLN0JE7qDBt^wrkt1zlQvO=ZPfS%IrdXREpHI$B%{FxeOCW)l)r-`;H zpAPo6$Y?&tom6CRTE6>7rBO-oo5nCz+KFl6opcMq;}-1)vGjB%QurAZWX9uEmohlw zKZeLC`n0*}z&n_=J4O>jLz;Z@e$FcxSMs+jcpUJ-xwm^Qq)i`(F%!wy~n^re0G-yLO(gAd)lZvFdb^jC?*uLud3y z%+(SsK*c&q`|m3CujwAfH6Yi@BQO5*bdDQX)(?9%mv*FD143 zwM~fBDjXByS!6%ZOK0DbA0P$E5|nC%Rypz>3e9IKe#kDa`&iKh}c}Y*yU4;&GgbS*xQ2H!)s}JKd23m!+g5W*ap8_c2mOjMmpdzaKPIMP2YXO|8?^eW7b6|&;zvLkWirE#Wm%-CtF^o4+-9s@t}hJ zfo! zIu#lpR&xDi*HHrE4ujX3$h*En5iVB#iAwXhlB?=V$AdW!+fbOXD%+ssDLj7t1qbk3_SHX-?gG)%9WBM)$RDdc2z2&JAskA&CX>t%hDeUdV<>bNUteV ziRd`%EilJd61$%r?xcTBc*foNt%0 z4G#qf^eJ1g909i!Ox~t%>7J+df&;I1oJVijg>*H#UBhBL;$hfXIl84fv8dg}Hgtn> z{Wh>-bkz(rB71a=KXoZMaro3?p7AoJnNd*l;oZ)lnnTAlO0QNvMfm=~Uluf6XG&i$ zm&&Hs9BCou$9-1{I-F2U%|A;pw8fPCZkqgO1P4!6q(tCmI<$Z%{h07>E z4*Bdo=fG>PVad!}t*c5w!;(`Wh30-c$@^>EopgO%u^WQj>^+UBR~_D%@@KBwgX5J2 zvwAd>5%2KXC$RyM3a$i<1fyyMapz?fP4#whQ3TQNjFok)_lK-A&e+Ez)jY!q?^c}m z_S6Z^UM6Ki(@yfL&aZ@N7i_uHj3dVL*xMWlP}3$OEvl#+xQ8t_2UZp_xh;Jj7hMp= zcSS4MZUcIHdiAc?drRx?=e*8BLAur>@Fn>OeC;f?_YC%-cie=D;O^Gi))~^R#S1qE zwi#|X3l_|(2}{+H!&T|m&v_3l#w*1~bGy6dH%p@l-?%mq^6-B9>fFNqa@HBFpQ~$F zhDS49Ih!?OxW{CqsQY}sK0lFNv#yeUe4JUbLkP=WKdiEbov(eM>y-oUa5~t+GWhJx zQ;rMvl7D3vV{u7SURS4loniA`Hv5TkZ3Uf9%hdh-^`4jA(#J$5t*(Re86H;0`=P{% zV8PZI7%PLHW?9E&!1cOee|taUX-UOJmZ?;ysbilYogC(Pbeo}~=WR9L0)fT5Dp|MP zwpp7q;Z)kW;hrSlXU|~N*QTa-9GiT2uc~v+m_{iuu5aYqxKhH7yZ-y0R?;Ybx{Vwo z9LSXq<@mYBiwD^tUe9`mqmV%hT8T94vG{`y>4rYBd4n=hUPEr_8}Gl6YRa$8?iLSC zJ7GnyvwgV=O=3vc@}d+-vEQ(~ZsYTP{bjjFx@AoxO7HiTB@-LjQV{o@qNU&qxuB19jz?+dHhA@I=F1LT*!R< zt@t}ln*+OG4gyM0rVqwQPE{4sND}?P>xXZfiqvK5EI{ zfWf=jexX$6TG)}OHA=a0>%?T7#6SLZhyF`DGoml(va-l;l)I9bKRCg?vTO&T?%Kwd z?NBigi|pR%rg==_Z?^3U$#9cL%U|II`WW3?ulV+6sHR z=C#KJrl$v7O$gLZ^Me5Ql}P1LEj%gpBJ{wLzBhB$S5m)%ii|d*%|%q!K(f`E``m_p zj`7fq20QPdELP1*Nqe{l^a&i2O$6TyI;27gFWh2J?FPh0iJ|0i#7KT1@Di_eNQ0 z&(+y%8C){Y(1Lq|30HO2(EHHU(l^)Bxu!H4_6YS7o;4J(;@~K#Jyt>>sowP}b*0^e z24}!y!|vSRcd6cw(Lp9F=7&}vKKe!qW4wPZ@!pqxPejBK0e0XVxX1IymURsXaziZo zwCLA8N7iTEbnP%zg-?+*{P|R`SKa@_J_bzl-2WO<>IZ+sVDWA3N*N!cFc9H@P@)Va zIyms7y^~+|kt6%Qd@7@_OVjBD=qx)VzYxG= z{rF)I0wt5vB+=&To2t8mY#$xNEJS)Y_Y< zRy4uY7jJV*iIpJtzM0>-6cyo*Tvgnk;#{4f{yhj>6q(T=#i4vMA13AMB+JW`I}#L>$V0{$Zo50J?K*PnttRlc<}63 zX84~Q38$MM-|zHE?Oe2rCOA`Ga=Ry|9MeVd7fZtsUhXAOxOCrq<7FvceKkFwIwRHh ztS|O>HL_SjGe+CvTOrhLgJrv9&__Wa{&@J+;($eg7=^Htrb2#6u$9frb(oKwX2)2k zk~u5V1QXI9Cg61*=#C%g9uRBZ;#TxCUGL)V5bx_g)-^->YCTZ%qH}~KvKLvnEw5|Z zvsV{&!~_ofH0#HiZ+JJapmTgi1(OzY;IRS?lFgW{ia`2H#tMxHJ@$UNS8#?7*-bp zw))9n*`MyE{bz%l*wAE_(8yiCC^l2s-}pB3h7py6wOI@lSiq?sZk|u_3VvFs3qHRf zj^;tGjlvu|L&~*HoHDFVkrGLpI(AlFP4|NRN zw-J3W*8a?P)Of47HwDOwi~ryM0$7huzt=cqF8SH5qUpSLxnfF1euV<#eu)b^+7*NPxWyWB`HTH0KM{RPrcbQuy$~QugZP^f->5dS_#{ST z!_$&nj(4zJcBt_ZpP1|M6(1ZaIp)oetxA#Ks#kY+*nge;5~^33<5l>cXW!9O-%a-Y zqVXtk#O4IsGXDfrGeLb|CWPPZd3gJ{RZ;$O2}l=zXf}%#Yvc+sM`xQ;G8o5rsjrq$ ztn>sus{FW=KuLO9yqu;jUorfX=cKfvdN+;dS(rK54j62H>xn=`DlU48_nK_9o3w;O zH+N*T@T<|4iWUqy)?dH$_fPjRYV++6IoyXF)L_Vzo5x>_(FP5Eb~*a+u_{n{BG~l; z{8E7)++DYXT$Ym~$s^6#*?Huxce1HsPNno&E-{bc0bXZ?mtWYm`OZI9Ca27@l^8p` zc);Iz%`s`r9Im}WrwBpxcbC}2MZQ%{#{hIrnM2>nFoF>KUU|2OR_6w92lR}G|ouc3py zhnj+}M4cl1eZlP6PY9;9j^7Qyo>wdN{7^{@8zsKt)4txa)%kzec+04`x}{q4uQtqo#5{7?(%icdCq(9dG9yI{;~h`7`toLDw$PtVnPwgRjmeE zCx54^ea|^uDJ~HI{wq(o?E!Ec5_s6|nSI%g6MmRNvn4t|O4YD1A6o3uhmVNj&B#oY z+=AZvv*N3NnDsgx8Oq9wQ-^n#dVNdq`E#DIj@9hE!aIReea2yF8HtSY z@X~u5hipGWJR)|dzsZjm<^1oI9Plrmlf|A6i05P{QFkIl$r|=$0LX?(96gwXPZ>Y5 z(H2&f;Z)Kd-0CI*NawyR2D&eh+NA*v^-TToE zJT^3noM7vI9T~*@6C7H6=I0jHQ#FTdcqu!b{HCH3kxfSenMbiGd2cqPNjK|)mRP!Q%30We#X;B|UfG|W{~}3+*OrA8 z7R(uqSnC$bA9XL`Bz6j`d=`6`cuYKY+C@wUTZ=+&x0nk_8Q@=axNNV zXjwX^%3n?GQbfo70)GE1g*h;5<4zKmynyA=eK|oiy)Kq-C6k+q6tDCdSxqo}ra-cv zl0^Fzs_kiNg=tdrv*Cf6j5sVr8ID|C#?LjdMp?|52o)r3&gnHc6$4_RzN`xUz zfl-CybBVq_QqWILBEzIbA{r4SZYAYD#lnp7`1znnw9*EGfa_T}#08Q%OxXka=rR|! zBIEjV!}2RI<|l4~=};#MU|&Y`ZBRZPzP=;%i@mUHSe(KJXjz~laeK70Cs^UXyeMf% zEcu}2bk8%(jmXUvJ9KDHTCPMUEi^DQf@XT<2BkFp?tI5DU(W>FP<20dVs)-3ZMhN6 z%=l5)-Ej0%gLYc26e1xp;2qg-6>f{6eiNZ-xwsa&6&LJ*Z=LX~zDlQS5k&(YDn(^7 zUMV+>M((-$2PI?hYlpI%M{afl9(hZ3!Z{_c%qO`h+&EHkzh|;pdvzkRiYn1Z(^9Zd zK!#i;X7E|gAf@a8myo4*XcGR}UI|GTxrF}ZZ)4DG&yMaK>HfJ-G7^jS>Z-1D(UO93 zVIpb{t6Yk3!oLz$sMax`Ew)e}hrBFH1%`>}C-B81OvP`zn*|-$7g`tNN53tZqVJOS zbOr54PrKNI~vaiv>_j2jbe$D-7?)5JVf^ z)}77-B2xXMvH{w3!Oapa5DF6PEEzZ^dS++2y`4^Go0@(1uk>hJ6|P~Nt1>3A6ptmF zPIof*G1WZnmC1zS7TdV@Z}Pk&9cDn_P|aaLE>W;P*ZAE z*!dGpQ2SfOL|w`3*5}a8vU=*RgHKil<+aebM_XB)99ZepgyWDkPYWUsaEX1&(ZA6r7mVE5^w}PBL5ZsHw)QGb)t&t?G9+F_gxJqn3BMy z+vOpU=jx^RQTh+In@4t|)!@bVi(g2{IH>nMfwU)*ChFSD!m&PEw}V~Y-X5Jkj$a8W z5>Y?LiG-oya^(Xi-G6VJr+U0ap^Aw*gGTLtA0taxNMF>{=$-wQD~Leps}(w{PAdme zyR((T)Jn8h(buV(Ys!X}OE7(t>D=PFT=wC4CcBFIH5`I0S5(|aXkA~XPpoKi9}e^S zk9`=rPMsz!X8CU8n&00V*il`vzF!aTEt2VqA0*|h6Rd~2IG>Tb3&e$_`>j;6xIpQp zL528#LIOhg5jG;mVB#sqYvwc9u7~$h6Zh>?I#fA1uZkS?Rh%G6$s^3BEC+6Q3#+pB z8n|^xy11ZE%q8U)`EFPLNr2j+!<2FOmuPyLaSEwZj-W%18dmy+M5V8i=i zjfR# zL@v2V;91ua)96UBC@x^`zMoN1gVb~Q`CU7gm1ULW6)%Clco2I((d0x1t%NT2m~q@q zn-+Q?kJ#h`=tR0_L=p!b%N=%hspSb7H=}9tfiS2<(D|j`ISGZhHjtKHljbi6kcJSd z+^70p;OFrf{jc0}7Fmd_*{I~;Ht}pI>gAECeo}w6wQ>&@SfaO@h|VPVq7g>!L8@VEwfbRQW)M z1dNN7UGtEY_YP~!MxGaI6?Tp`%0lB@jWTML{5FS%hDQFw{x6bB`u4PJS#u{oH2CzD z92ns$M=*DhG7@?DYGOEeY#J(UI*(3E&pZ!Vvhlk;{j z@WCc3p@fRsguX;#YC2;-;z%2kfE)g`VMw=E-qSX2=jQW{s? zdB^lCk8L?(2dmLe5}+=WM8W5#!#81k;SIfpMPbI<$oniunI1}kDg!y&n58rw-2#H- z|2|zMMuv=-Sk(%x)Q5Sw+;;0V8Awev5%?q_IETP{+^I@G`qKwGN0$$qrYZ9tsSf}8APtG z02w}Fa5A2?Nw8IFEN@(OkK1I@-eBMuFnW3fC)?pq){DJc%J7Rb_=Kf>%CBN|VZH0N>^zJaKSM`^2@a=X-UL7)GsdsHiyfS{YdW~QNHSq&UYskhIi^Ci~6 zDuXUGc9B~5SLbOW9ams?3w-=_m;ssz71)!|P3j>YDS!Em7xBm4{P!D}wNs=13>#Gm z?37VCINMOKVoL)!kf9ywVDxi89h&6h-o~k@L8*sJJP;B3uB)2bKA|wWX3IR`PhJZj zq-D*VKU-Hp5W&S)rt2Io_W9k!?c2A}XYU%S&G$O=UB>wiSezu;gbU^9-Xn0+MJhY&qR|K+{-NRu(OJ+Tv!GZjwbEb8A|{&?;?Dc*Q3Ae{ zfd!sVT$c!J5J~~+=zs2*8|CJQQ8R_e*J+aLW`88I{Anep3Xx(VH1P*vLzhB+=*Ipk zlMmMf6mpej5$N*w-Mh5Gn6QUsAZ&4G9iU*DN!7nqlu9HJ@Ch1$+S38zMxD<6P9Y)8 zO#oskA>JwUAa|{_c)?T3x5#Mks&S$l-;QA-0{rpzfP3UY9aSWF4BrWyWAQoE>AIdb zEy@v{+GaWf?r7t&>T;@btt#vEJI|g#e3LEv^YS>&=wwKD>fw{*ub;L>_WOjTN&b#2Ufxd-odHiyOdmd?gR@AowJ}-h-jmjTaCVN z4H_2IIxntVGmqLr1bKS$FyaCzL2wR#0I8WqYacYFJHG3k0@!htm0S?dykYvabSI&L zKb)cSvbhsMcjS{+zXRPbB;I& zbRk--sPkR1Sc2{q6ga2W;@05U*N_?=H99*C2?~Y8!O5Ru6TcM3Rt){H(r4N2i1aPT zmQWgS<39jho%#5&(VQUAE)M@>#=+A5fkGoHK?4`Q34%(Ti}o*rLM<_P->;nW&e~fEF++J z>d$h7>sMdz*V6pXsunu18S`-wl`k1%B^8pav3wB2<(fD%QXs`kkVxuNb|&vX(pn^s zkoQDQ0o3nec)#8|LEE5)k6v1r^axUU>WJr_{hI~?B$5)5SHByJ!I%RolSC28h<;RJ z3&Wh{F}*-Ux#5Om_Nrd+A+&>g;r?ozw$J=Z>U;ehFKO_xc^_JR7C1BGdeo>3c)TV` z#=`5oi)InX;1~VHrGql-KXl&lVwaGdY|;}nWo39iFCLe_g}@i>8P6v-%m$Ru6b^cnVc3}==vx}@9X51{=%0JC-e(* z$=yBKZQUa~eMiy;z5bpohFDbwY^$di5A5d*_8+#qQ7qA013}B@#X4;o+6^$KlKk=`;>|uMfTC9>N;pF5w?y9<%7+@Z z2PJ%a--ZZRr!#ub+wh5Emj7X?q50!-m7%aI%oh?&P-$-^IW+=#>+QO5{NTz+20ct` zwLPh?(R^?r$Oi5ZK7221=Ds26b3*s^DV|%)JtOU8^!Mg1YvP;)9gFJMtc4|yzEjRK zE(q1ca$tk@Df2$`BLQcZjie+T6_me739fD6RDuJuao*3+kmCty&{4<2>@I zm6>2iYTuZP%H@JRvo{<^p?FJTPNa3qTxi=Q|LmQ^WT)7qHpNOae}sx?`g=Zv z86p)3T)OAuyjv$Gb@dw`sbnFiX0-Dw)Q$pmXAJ0*fT$CQ_}vk=ELM(s-3m69 z6Ew=c4C5nAJ+2}bOihlXPA}kiS6zD!txgN$kiDQj-&fxeHi;QRs{Z3_Ac_J-&G)UI zPqHlha6zB^eR{`{Jbl8(GWp&C&T0vW+^xQkBy1`?GYT-1DY9(1h$dC?wo1bODVP-- z4ycB{mdyBuz7Y0pOF<9I>*6n=**t7=X+cLb)!l6gY6VPsBnanH0e>5l9E#03OV91% zhh`*To?gHGc>{qhqdvg$>{ML_{*eY>nl{`AS_V$*v4m;r@>XMPay5S=B#o~g?v`Wu zcn5D})}6Z=ZTlq}Sm^t8XL6ow?u6TtrXkn%AIjsu%OWa~UU?~3ltgm>Tq;yFtkBt8 z%KOgTs>tiV#-jp@p`weMF5(+;q$tk_AKjN(y84_W5&Ng`8U`t=9Xx6zY&*%as#pmUcOVN`6;e>B~Hz$RGK%T1ADgwAKXBDr=3OnU=!VIW5qY(TdL`dlE&i>dl|P1Ksr2jK)q zm>=}VQx^PzAW;{mM(+HME82pYDM2<(Svj(#5o@8lWvC@I*Z(=Qe;|baQJ^N_fpm%n z5>aRM6v?sIl=T{a&=GC52v0lZhb8p zprD9M&4l~PsRBt>wBr%qdM(IlftxZJ9q9Rg{?~ugUPZ}rOzHt*!-4K>SIvt+u6}32 z#5!yXTdu`fUyF&HJ1u|u?0QwTm8$Ut!zzyl>*wF4%(Mc?`ZmIM?~{a0wD zfD0QG8^^{jk~Rka2L_1Z2glC`to)&_{k`E@_LaPm4aY9N(pBidurl6^rH3W*m!g@7 z7bvle$kqu^tqR1b{xTE!V#EzDJuGb}Xe^rFk`DbKWuydIq*uteewRC;n!!b&DrK~a z0TM@J3@E3X+0HFwbE5q-fn`-k=fPM9@s~yCMq%{lsp{*ISFr&6Cfq+xh?@_ocEhr) zl%+{yW6kvW)w=Fq7(UMt9bMtxflvvJY46r_T@H?c7{ALPnx@MU*bYfMV! zOU7x2j+~|>0#n9C$SxT-s80K)CHpl>7xQ+v-0bk6Hm|%A3x`RjjvS3203C<%#`!g} zj_54!RQwjSMtZ$#*{6{0AB3w&9?LYVKKzJh*Ha;43)Dd1Th?_@X8G+HmJtQwrvAb` z$VP3ye>sL|XNxmj|CywbanV6>2VJBezpD&mqX?s;Gr1F|q;-5yuxl#b`Sx+ThUBF- zm&tv(A;%_76)OS`m*NKpiHeC1Nh=l1Wz?$jf@)psB3n_|Afm6ixm3v@AfszrIDOtb z>cTjsULVk4oK4xnG&|SVs8a$(sU^Bj4oQgy4o49`Cb$2T&7Yd;c|du=d~@AOJP3MS zcbt668s1y~5PTr~;DhHQE?5oReL!DcN|vd4ZB#ekn7=+!OPrqudiR7d@`{1MKf;-! z-0711mQ?)I*wU(s@){>gBLY_4^(Wq+n%fh5YS>dPhEDzZed#4ztH+h%mF1%k-U zOGhl@o{%usua0@y)x8}S3*VBKx!4JI21>o6dueDT&C9T$ZjyF_Ar5r7N_r9EgyRK1 zOxOxda@VW9px7Grj``i!-qfZfuK)+gMx$k|-|Dn}(V82CI^@S@X2Gj=3)p&GDbagZ zyA&aRNgrpE7W%Q~Ts%^FtyJTNK<9LKP$Cq>IC{_>mlm#^7So!g%19+}y^SVlJ~a6= zjoj`2S3eD`ie2LkT|ySD#s=N+L;|mr#>m5tT!}uUc4j1vi2oG0i6j&d-s^H4j3t70FqXa1SE>Ll|xsN}C%aVI;n)zcktQ zQ~rG*Qy|Lj)VzP4#pi`WjWC$=l$>h074LaIg`GX5EE)%)GGyDq zwWIjjmcHL(@tb&h^H6UjTcbD{<2@xy2(@dz>VWtk75Y3F@fPDfg!a$J2x4T6;hM(P zu0(jomCr=BX%_vv{K;R4#Q@rPG}_U_M=@O(E_Y0t#DrK{+{{=r*cB|d^5urbZKe?C zuLsGiV2dp#i|1>NK9wZd5CMV6gl;Za6-)88rf2sg1oaGb%-z#@UUCNS##1@WeCjX~ zza=uYwPQ09?DYdyeL)LeMaGkST5LD9reaF^6-@Y{@|BYP4qpm?2_YkB*o#jcCkrMf zSM;BP>H&0yN`*vTsczF>l#j+tg`c4KXjnLAf>yT%qVI{mZ|x5H?ya$`%r+Ec@W%@7zE1NC;O=Q7)`l6*o?PlBAv&8qfsll_6Kvn*$E(xL)L{rzRQd2 zZKZsJSMmedTm%QoOLV2>%}s?^grNgxhJ88PV|)+>39SAIL$bczVa`fFr;)H6M+%i| zQ6sp~sJ>AX4CcO~1N&^SFP)6nwWqbwi$HhsBrI8xMSjkf-a(?U$4WgF*f*0$dq7&u zFv6|Ab`u6Ook5UzzxcYKyHJ8XO;5-)N$&o{tbg{>G>Lie^LCk>?OJ$omIq|B<89z7 zSg?PSR*O$#eK3*`k-H18#8aXFD_I5RL3B)IkgqypY8zEeqKgGc#+zS96O*)K+ZbLc zClI`Gk_!*01QGwb?Eo!^NRyH4m)#Yz!I57&+;mBQ-B^${y*29zURHS4ur%=Yw&%Uc ze1QjXJiy5#2?j54)oRYcg-_Js#vWrRfi|K1FLPGFy9ELd-qnmPwOf94O61AS z%;weka%K|ZJ{Yb);Dsd=vV#Ou``!(6nc0tG^u`VYQfwto*JSD)fUyrvN-U6e%gq(N zwi`^hxy7UDNmsAa>#&(Ja1rp({psp^a4!&$8ZuOeh}(E3uUFq%bCSxqcs<6{bgI7C zJdN9<_coKfx$#@f;+Uj9byn9MOV3!tqXwHl`DnDB2mz<+J}K97Tp^>XAz$sdV#^H< zkyArE6#62q1BDX?XtWxD)B%N#x$(O1u(ZJ>KsSLR@JJ)0uf7PB>fBI$9AmDXNLXr} z`FcV96dSOM47q-%d(94t*EE;oWbE79QKJtD%{9eQ{<`__2Q-`5k2i+tS9;P$N|(BZ zH(Bey%5b30A7yf0^3yZE^Ff@H;Qa2_S0=HA65_q+v3G%d_x5_b@9He9VT5Ft0Q=WP zI283y@i#@rEno1iOM)Vueml9 z?)kX$*!e;`Qkyjt5X6+au9{i)yL+FLh*={g-&JbeDA&D-?z-#3v}(!^rv>%LR1HEj z?zuNcC+8~f6t116=C>eXPV_t?YBOz6S}`jgnX<2dRiv13EH*X5pLDWAKwm#1j`=zf zXm~tO5RbU*lo)a&5LH1)Qza}}wa%_UD&Cx8dfl}?96d0RYCP2cIQVwz;U@G$MYCNc_v4(ZtOz^N zDnp_ZwxJkM9D#ufF-!eEs0psS@OnYao&WlMQ(PY={@@rhjLshei|N(Kj6Ln`+-jAt zH;AAL;gvt4tS8j!+j*9Kz}!XK=4NOY`NpdU9Z}q$qGFQte2Uq{PwyyEINeTu5T;{6RGsRfkH-2Vtin5 zwpy+zvN2J2S=+89KvA=|0A;RxxmwWgUa$rR|fi4vHeWYySrwo z7DOPwk-s;3JHXet8V}oOZL!_O`RUr&cUQUf=5%;;7s5ml!F4eRp(^OFa$(I#{NwT= zJ<=KB4I)M59eLg#zmGmayTs$hR3w-`lIy%b|7OqwyP<7GN!?s;w_3CIL^Gttuzu$u z<$@S%A+DtB+||i{&M(W0U)5Z39shEeR)4vn+f(|!V%!xJqISSh)-5E;HUjaHrVU59 zlXaWjSEBek4*JZVT&095M`lP+m@N=E+okP&6o@T*;_P9bh1+L> zeb-KdWbdAO!oe{`DsN3%AG^&7;o06V%V1tVH|OF&fz0X)77D?Yj>P4D@YUZEXKDf@ z&+$M2?%)5r*u9n95=G@}Rc#?Ch)nZ~NuyUY3_j=Qw!~PhyH-x-sVtPDzXJR#Y(Xp?S9ej=MIffI9rzAaF zJvKjm#yerYNRjD}xGUw*+s%dibS3Iz?Q@vVo2k}q%*_i_Ol;aAfcH#clhEfSOh~U2y}Y9k;dP|DrK$z|#lQ>R6=xU! z1f{bAMf8415r|Gq6B!M66+9*(@=f+B)E*IZc@Ept=} z&7eUY1&2XyzHGSQgMcPhrRq22u1y3meDG-9!(#G;GDcchy&aZ!+|HG6p39dU)@qd@ zF^~A&Qo&hkRcdJMt*O<(OMXEL!|=}=V}p8y+MWv-LAfOr+e{jM<|=$ohn#S=L#;Dm zsh4M54k4C9TDR6%F}d@6^bWV%U{hL7(9ozBxV2atu|{R{-mTiB@AcEEt4I{k6Y!Qu zHV~?V)^)y_ZatdP>lL6t6tY-#!DRA46n4LSNkOX+TT5cJR%m&dyQ5HIp=#xrc1+9OhG)%qua`l!L0^)ig}VoxYr&I2X)KZ^^JUTs)MVg;j;lMSvgE=M(UIoGe^xcG4c zgR&mHQYT+Mh!&j(H7Z{)TN{(`fV3M15sJjVA}wDy7|x6E_=TRPnuHU4!$zNRCArVS zhm}b5AnRi{ntPvf0BBVEE~m^4-%;x{@=6>9cz~1!1*EU(Zef6nMQRjJH20ryVW?_^7_AHDB>ypv?+K z6utnZ@OY28T8ONe<(J#=Uc{S>+)q#~iklr5DSOJso5|@A5RGLhlZ}*qg$944`{ufR zYRxeiG=I-tBg?i92Cy=wR}h6q*>x_ZyokzZ#gWOKmz4zzHx=RteE^bur%{4tLOj?H zo=3lFx94g`#Kh!~ESd;PfYz7&xCn+C{%e}Zo$g62ucP>oZS}7!z0b&ppILT~OK9{% zSC?|9RUwi({qWv#nseh%mh5+F*NuLWzxB4SWr7{S(HR{Nzd@ZwA<#^Xf}>BD^@hGU zgAKnP;o_Y#Gn)>)!jPtiWB^ToY_O?9O4>opiimT}<(`A<-TC!PUQwg6D5>O9qvM&n z&hqSHAi6%#43*9GOI+wx?llS6kgJn~czgh|NTTCPZ;9Z$c*R?Bs@4gPDuZ_GCrZ!B zyt-S_&iNuA?+R=M(He>FvWMAx7-xS=%x}ksTy!#boE0%kBcCixoYI}@krPx~$8C!p zL@Z)EuQi68rKgTwH-x4bCndWZo{3{2sF4Y0y_s3v;CfK5e?lZjoSu^6GL4@N>_+!D z!9mfa#G2UZm#?(2o88;`?2ac367RL6a!1b|s6Jo`tN4MhkzJx7j1P@MeSWoKKdy3; zjgt!kSEh`m&}51cupD@;W-}tN!}8p5!|>^(M2VIW zyq5C&=z-da(_FS71h7{9=8x(!`$Yp{)#p5^@(aSy02%&$Xq)!EPWd$<1|+;1b^7GZ z_IPLX`aS0oVWI{Zx8`CM)5YYB4?HcnN?%=N?u#y%kh1fyM^AJ0`>94ueBdg_Er{AaYPWYNGKAf=+Lhnxg-ZT+o@xULz^$|hh0cwV7 zHKA1}3+2##Rk8f+LJU!rFPT?fIMQN5LDh(pY&nf(Cu&qCQr4 z0<6tOK}q}5{@9PkDk2vo2%@cLPh9Thpk!ZyP|lO|tI%Zqtu*=lNN{g5qeR)}!iTi0 zH>m094Egbw0_3KdbF>mqANQ1%VIc`uz&E)`Bz;~pQsRw;Bu=&kOWyeOjhDE{Lc*Gmrrxj*Y7Snna%z!Q%0`=^=j-$;ZgbuvBVUP_2z>jiR z`%K+1b4os?0w4zsKh8YFWL(GpI2WU$GT48ajjLcNW%X+CN3E_0?xWw4}E83?~ct*mmh#~51;bMV^& zkjKla2wnlN^MDY_SU$&)?fYaV)!D{vziT>PO>_gXNK{he?v+05)l{%T0kmhb(#CfU30TB= zIM<`)7sNp4(!s?*d_2DKM>gKE>UN3_sc9qbfuKQt=b|yj7rw`ln~-)N&-1Un0v4bOC<|u@&%7t% zWoMUCzahhM>(T*d!UtThGTBpR+Ws>rkfw`dAl4Lv6E2DQvy#gzc^CeOkn;~Xkzag{ zt+i;&fVokgW!G_XE+!Tx@lyLTkT89Sn~fj`q!O>qluZ^!m#yCI-B) zeE?_pV4cR+qnR4)?g7O5Ro$rOP9pi}WoB*5i9@FfAKiKnx<0$^#(i8Y>GpObsRhW) zCD^c*<*9LtwfFY}LVC*!lNRH{D2Ra(SW&}v5K9p@5{D=ngG!ZTNj`!Ugl^WX=GF7! zpVZ6F2!kq&ybeVwK1fV=LLV0iojK73{D}D5{nT-7ZcR-t7~J_LAodl)k^Dq zuZZ;!A1HfW#HMd8Vp_b`{;slj-^QPVr-a8huJ~XHgBa?yiI7iVNa@?4dIL;V_eM3` zhX|%LZf4$~NmWM?8KY!fcHYwZQ_#xj5gJ6pO4-d^sMY=B3R`?dYoQJ8_J~(eTMz!i z4+Nl&sT8}h?eIiGg-RR@{x-nReh#tR_;<9Y68<7d0cK`g{})|ZSE91u@}}@u=8)~ zUV8;ZXEwjU_eF?^6kH|IHWSf!T7bf+W@8N5Zm4zekv{!ra`;=(6*dOnL+!SXJzbqf z_@8MQZtpl7g8>a@bP|z2yjOvlpcdL9wB-bg2R4*KB0__G)`wzoBnF(8IeYs#FSn`3y=ggoaJvw<4Gf-r7ok4cw zxVEDfkcz;)^~jDwGFd%h>paJEe@^hY2K?x*C;mWpeA57h7Im{ZqD7NK{9Lyi0hj2U z7pxodzYWsAu`&=~40@&U_Ywe}c`p9`UYeV$vvhpMQ}+XU0Q#%ZEEn}20FcP zzzvhp)^w$HFo=!~-&S*^yxN>s35ck1f0(R&1lCp-oOJnHEEr!0%ylpPdToCi)kS;e z-Rc@A}2Slu6*{3{_qHnYaCn|O8?->C>FVehrF*avxQyu7%q&Nnn3yjTuZjI zd7F8j?>V&k?d*u&vfA?Ls7O17+a8_q22pV$@W@u$g@!Pv$FouUjb%Q{!_lazBl_FhqQ zk6}LJ+Cxaw$`?%^4PW1nX5=YlS?T72TKhfXwW?LjveiG?SW{hFY=k$gG;R-ByidwB z3V*Z+4oeceH<^XoDU)Kavx;(0g3yJ3TmD~w<-cAjm41#58xH7)Ahyl>;ap2fodh;fU2IIB!-u--yB3s-v6P zusUAb0xlhT8(N;3hSXelq2v@M?a7VGxfo`yx$O06U#kxTDMrOH!#u#)Pp9LQG{+!9 z>6{%aeNTUi4qX>Vqb$Lzc0Lr;ZuCuvKT{q%rgC)6ZWPk?0T*0M3hRM)98#Ha!ehSV z0Fg>pXNR5n>&=fxpDlTE?<4IvBu1M#g+@EAXe@dSqh1&94%7JU@e)1X^efgNKhdEt zFx(B9MRNTEYc^gl_D^a|m0sMoixKFCdg@srM~y%Z21) zovj?@z+jO4nILgA>043lpSo|(6aUB7|KP-@@EkgI4lCT?)rOk3zT{FQD&Pf^0%(4B z(JSz=s}z}8jTU9@2se%vvNS*w)um;!)pjexE{OvfN|M&Rx6-5PJ_vMGgfdxWj&lXE zD3U{umS9SHbCbI)_%(E&0ufvIly5|SC(dTVAKLsVh;`BV_jy|=_Ule>`5!G`?}&-+ z9_6@wU+M+c^lXUZS$t)&T&fV8%nAW&ZRP~Xh7A3#=4Lxl+{ju{x)LwKcFg6jBq;YD za}JRfbbSf6Ew^!0o80OzweH%r7QtN)J>uw6LWnQykr1Ctt9e!JF4Lz+Ky6cyqn#%% z4vMUij?R5Z+I3{ow}bT=o+j~oJPO$5U)P=wf53ht5EI|oK{NEuW>BoPo#*ceD7c@< zY%kvwJwbJQpC~*|U}vniLiE#-5dRpF{FBGIsQ*r9cf;%kb6u_2aU1R&!nf@HksR&s z`Zn?^_g{epWO7!*&bMCYATv8pT8*$a5M`E$xhEl2#k7qiZZ$P6jm~z|~ zf~;-|Z{DbKn6XmPDY`i!tRT5GadVeL(LcAePC2jZ<9s#IgJWrNvl6j0V7i(25Y`X& z-5RRVH#p2qqkvkHC2MhOYrg4e6`c)Dkf{@!Jy77;4ju!5qcTQ@!i{&{Zncm8|DG}Z zC!C9z!nL^$1@oUf2jO>q6p&!^O=M=Q?u z{M2!;%9T*{LS^o`gVLdYY$5_ASM27q%e|1DEbif?`RJ9AlZupFYxdUC6+mJuP>L;u zeX(){nOEt!nei78W8DqD$s#)bK}&FvO>xL+_za{Jj7UYY0`i;d=L=V4K1c{ zX_%UVU2GD51J=H~FIiK|ydSc$yq?|hglDH9j`Covga6oVAUp6dyy=fy+Y32IUmo%C z@TQFTZkOi-41*b-R zG@j+88?bnY@aCo=F?_KRQe1mh%WZJO^x#d?XVdAcg|Jv=MysuQWobpp1?>yF+?eXc z_d31r_qtcDXOFm8#<)wn5912y(SAQ*b;n2z-&f%W3D{xvQ1M5}0dh$R(|1%qm3l%d zv^~;S%nL#(3tS+Own86f39=>rgk4k+h~aG+RD&me!Poneofh4+cB1O1kGSn5WFXp>YYZi2eU%W`7MxwxZ1rzxk5A#jELWfc!VM zoTiWZ!;X~|PNi0h))tc5bSOtEw+>`CMPf9Hb0r`Qb%WDs({~{jOMZZioe1ADxU8@# z%tw)#6$G6SqMW_hA0Dkj&)!?7HTt=T(^Oeq`DjBG5^SjY9DvF>s zRn3zB%hB{PSO%f&HdYCnqGXrJV*^$?2o~#{Uv^rR&q}%+Wu4b3t%tF;tCr4)O4iyx z`bd_fj&Yv1=h8gnV;SLDy72khr7?6+#-XLBT4}W2+YGP8V&HiKU)Kh#(7HaaF=AYp zgSjSBM)xLrDeEpSg14uqo_ufNaElvtIUP*0hiYEI@^WT6bMVJ+^D3?~B*pWx-K(hh zyg~2!pQ!auDU?$U_1pk?ar@M?Y(zLF^+6=u$P~-I^RtTya>Ui>Z5e4w-?;^Y7Btaq z&1KcJ@mG=TO(ek+3iiMw5MU@)lhrb(M`u$vv>z+f&hrfqCkz@ZL4S7y(XskX-UsLi z4Sm2|BC*7Nv}{SCy_;S16Bd1Y2d;IM?=`J1()iLsn0D1&rGNx+uBX3jK%Mj>{?Ih= z=U}Mdi)J>o^y`m{IBjD<7TRyvf`LD?^k@H%7Qp{~r%BSFo+)9H&PV3D%=D%e{OiM; z8gXgZZ-bP6Havsb>BQ+YUq-U&X)P3&$(B_ICgi{^orylHcEM7>z|DK2yLtLjVk^SV zPgR`xm3K)0mD)1zv7Qmwa9Z<*X=|Gg5Ch- z8J1Kx4o(Fm@MN$a4hUe~S3G*e$m$NrD>tMY>?O^@7JQpn@#kkIjI=d0rP+MB4wN*@ zsjCHxP9;20B&`|L+1U>lH6H98+%-83`k6eJ!gjEXe)-W#GzLrs{0iQCIQ>J$P#&BL z0yO?t<8fQ%=i$ZYS>ljQO&`U0AGPtLiqXGf5{J}CfbAnXX)XMf1We&$F6xFhg{OV0 zNopLR6*_&~)n7x@U7duR_S&LJ*93hY0;CjDMk2T@^WE%tW@H#+v?jNtKaz)& zvpskwVij&8nV`VeO`3|VTd>gjOos6w@!3#~a% z-!URYUu!=C_8Z=xd)_ASiq537Tkx*)DI=vhF+jJs$Iwr7<+7T5KlZ=VKFF?9 z^xzowu&d+iZ-~p#W%aJo=pEk|(lj&-LYNiLSAU;a#h-VXJc-6J8Ae(ymC~eeqTHBc zb)}`oU{Q+B(=Ufq$K2f&q{sQvCqZc=?Gdmzs!>(UJMB&ykbV0$iD`R~b>SCP>PyVw zYi4utwW#ouE3;v+b2`_zHS+Qrw{?b;ic(xo0I~+Cnn!9C@?$cIDpWYcW+*={wDMlRJH1v99wz zQtRETQ=!#@Nv~1eKaXxmv)JyOM~&9uszkE5aWWnc@lnN}!k~Ep0s5wwPnoR+|BrVD z<#$D6$kjaK8(M3xs_q9IQ^&`5itw)lQ0?tZpz2e2y-b9^hXcnNna>iWBbA@b54!-D zeUVo>d%h+b8b z9)h{OC#O=_=Q;#|00>|O97JUxE7+P zIxkl>;#^iEW$Ejeiluw}veQYau6bs*i#1*JeW4Z~DJmnLO_>of=&=MpLN1-DpzCb;9;KtdCnu z`u_UlV{dF?zQ3MTS(;5FX2;k!{@>;o-l@84#n){5NaPO1tOJ zu318^Mf}qmU54o|ytFyaCAD9X@y29B7F5Q?7_X#c;ITL70Dp4pb%F7KGJ1N4{h7({ z4W-h$i`9nE&`87qr&0A>*TpN1&xa*q{Z})O_0Q;PGH=l4MsDXcrUi%BYjKiNPdh6N zoib|Iu2tySGVe_K&Iin-xx~hDP*Wm6N++BjZ zyF+k?Ai>?;-5r7lcXyZI?)Gl3{O6f_XFkC@Yn_$F$~k*?b;+-*tE#)|dO79a3TXbD zUb%{9HZAzmdQAO@`P7TUdE+&F*Q-&#$XIL>4!S{5Y4j6$DV?!?=&6M^x$?fM@vI-; zenJ-n*Yb;GFWf)C6HtQ*%1pJXRVNvp#`P*M>)stTeOWTsPbv)1g5e!eTQpCO#_Mal z&oE%G;e-VW!{p`>?f+6FfIL`GAunU!10+5k%~o*ECm8((sE5=0BtXeLd`Mbgc-(uS z3Z@!Q)r}MBJgHiQ=!ykrvHj|!6t#z`{|V@_nSBtOTT*xDJnsWwr@al@VHoD+B8-E+ z0K@n~b2e9D02=NY7m3fR694|?8FOw^qYvqOJ29=xPfMcYDkuHunyOZt@^!%N1LQXnpM? zt3^N3mtfQvZ&In{^Z%PL-vlC9+tC3+^0YLziFnHT&cg$&a@~_u%Poc4im=%?={^X> z`NC=Bt{?*((qanoT?(#l2g_$lX3_kGNtO@LXn#~#{r7jSBj1g!jNGm9z{8Fw=aJ7( zx{Wk?WkDG+s^8rA-2DJup&i$;-`H>%x$7SC95FXk7W>gr^NJgYMS6Bx`MC2)A_U-B zZ<-rWWA_h*5sd9X9%I=mVE8RpX?hUWG4?_X@6rT|4sp4`0R2r-XU&bKRPC^5Ah;t| zLE>)hX)=^ndJpW{ha;-xJKs`&*_Ykaf2uRtcnRpz>v-r417eaFPcj8^yz_dxbz)`V zd}t^i?COo#FB@YyN}KF1f2r4rP&92QAU6kH_K@&$JvNd^4))$ImRa<+lq*{+BCy`9C z=yGr&2}Oq|B4Oy5ZpLg6WA>p`)?~YF0$tAv@ zwMy--p9*7?-AJ!*FT3#)<(?dnB@YmS<)@(mNU2jC^8L5QxN7lUH{D-)nfI1hL!8>z z?Adp4$>KjiOzY@hwwrbFP}A$%aQM|tXm7pqtFMwBqWV|-S2i;c0v^v^_tHA0O5|bP z!t7upjo#&LM`fm3@(}dW278vk{JnD#y~wdF5q7cnt7T2;zZr^)T3BYku`{PJFTdN> za@vAu)=`Z+3K>(t+XD?N31QVX@O{aed1DrXgouPr1*|&SAI}9y~&9VS}CBec6VLF5FL1-`8xWR$k z8x}9JwV-V|{KCWuVLU!d*a@T|{{_N-*@Z5?4!j>H9n8O`@?YePf9}Yn_JaQ3f6*lh zxMXz2utxs>GL8TElz#P3aQ`b2C?5#loT5U8#83Y_rLc<^tp9o4Kh0VeKNK9j7A|K{ zN9i91|Njh1XEL<*FF^Ru_daw;I^!&%6n?5eIm7?*k~c^x^(!Vqv#gJQDsO{2miCt#ZIECP&q6VP|FELbzs#n6`uX z)F?*GEh(2=p>+W^L`du0h^V)ZfSewF;KzC8U`YluE>v|W@}Cf#NeBvTg#|+nfvP2x z-^3sea^qxZ?;Tr^4?cHpD4>0~h%N!Ywwlx=>PeEa^Fbs<12eiE>I(7IDkeDz07V&x zDts598LsKr0?wiqu6U409HJY&75h{G!MeFAsdmN>!NT_DO#GJeS$rT-(6r#4$%E7= ztjo%;|A=;oX?rhDcyMrR2O9&Xgdp_;X42#8 zCeZbDf>9O7|EGXsQ?piKKVYfYm6ImqpQNZ9xXWlAG%KX-6`3nZ1Chu9D95qqq3OC- zTl2T&PL9+juQMDb!LqSXlW2bXbCy_(s4UvY1HG)a37RqvBH1PX5Xft5{ja_)4kA1y zLM^-xMvQY)zt!!}CX}4p=HTuKi5g%9f zeC9$_)9Uo*rs;_RBwG48W}*+ zDigGdbaZBO>r&aK{()jtLDNSnT=9W?bQ(5fAl%tSRZ`l;sekAPTffB5p<5#+iFH1; znPMkV8-8N^%YzjU;+xj z7p>OiLYdf0*6V4XW_rU$mg5BP+F0=8Is(L> z`^cA79`jB*V0CJPH`qjov?d@{1G zJ@Ej03aw36xYL&1l#PHi=0)}UGv_(wxL?m-@w*_VW0Gf>;n)aSD{C)XVGcM?i%-~! zXfq|O^q7b4)Ak+|JRAp;QVx*vcZB2j1}E*oB$hNrFSiB9>@5#2()V1-rDl#R*XQ!g zE?bNxo}Gz^Se+EKnXf?X;6-+(V&Zk0$s(%1zxHS!^AYPnZ$zx0sX^UN)2DZG#5x;j(h~LZfzV}TGG$3R2hi82WHvHkL1v_^;=H!R)HYRY zP;6I6wk2hpj@xc6sTDI{$_lOPH-l^NFo_aN!d;$;GU@l5Vck8SWHktV;QkP;8ggMQ zq+sCA;+3EwwRCz5RhgkjA74mwl)v!tZrmfsjE812{+=sE?c&r=%)$c1Y38EG#c7Z9 z0?m{`mvoN3%hWwF<&DLrv}RLgrE;muZK&^!Y-YN|n5E zzgcpb&8YImM8f#C^Z0Eq`AN2nDm*Pte>Tc2HluN!%cv+t>>zPQ#urP8-KZ$uWW#;< zE9pbAkeZ)kYqWkmcEz+C0z29<%vj!Kb!y)Kwi8>0Ud zD-~8UrA2P5-2H$)D~If$|8sN!F9C;qvBm6{<4PSEos(<%Y#hJH)Yb#=>=ksOmCg&r zp3D1iMTrEZlY%@2ZQqj@+bYfMeVW8R!*aAUo1ejbi6dE z4}CKd=1hTWs8OtxaDjDRL^DGCWsHg~Tc%OmKk~JBz#CSXBp>j;mT|wWG*S4O&E_t9 z)_dmVg}=g@p4Z8JDug4>N>6S!!SjY0{^CJ*_!t zZ%BGQ8rF4Q1{Y0QxsR%;6J?azv}JH778NrAr>OC_WDM#1=tz1xD5nuj|A}tMZ_K7v zGC(<9Rfi%5y73mwPGQ@$H$glF)cow#oqnr_#&%|lGWEph()*mn^zg#~gr4lCIYt{S zg!SHR%dK^}xVbnaeAZsh$>7};{G-J1`nKsu+IP*iA!m|$sfLprsjVKDZC)$b0R{+Gb!bN@ zn1=nBMkg1TsVm~-o0;;M2*5J-W0V`c3@|CPDTu)HwMN+$TA7Een(?!4eC|rbgO?1>t#XAVcClCP+3MDLWt6&Gw-;?1pit zo(|<5V6t%ilsd5hB;!5$PzZwF&~7LB@=~FrFbU;y!FU5&UtC`{NyoR+gQZv0`votx zkMcA&tl zro$Nf^$q_?)CV2WS+Q5?2P=D>`IO0Neg)SIJJE(O3*ye}(=}|WnN2)dG60j=g9dVM zrL@R}ZOiztbR4;-YE5eExlqm=b?r7yQJ;YcqC*w>^qJ)%ED_aE?}F8*o5@AMkR)3y zu1BRCx);2I_$}jU})eVTzzkz$!h`SZnr^!BpD{hQ!XxIHbb%Z z(=iYx;UN1ExZ8Ir{0X`dC@w&30k!KcuVUxJs$y{b=uWH*rlVn#YYL;$`t_{y3>l(} zH&Zoi9ACbS@snAtrZlg-J!d)7Z6Ky@65v83E`Sz)oFQMbQDDv+TKi^@kXO^>)I3~h zf$ck(Z|DI+n_k^~ z2C)PUD!906!8fKJdp_AEdfX=iUOA7q0;bTr?9rMzYwi|D`b()3$%cVGjY6c~+xYjO z4#um$;$(PYedHL=qbyOlYYGP7is$bh>xbY9Wl|uwasoKn@o<6`gFQ?NU&t;f7WdP9volQfY$imxz9r3dtKDa}nsYKVV?j4;5m^24WgxGHjJZ zAvrM{;qoX^&j4eXQ(I_`l4ulBh`gF#xcMfVzMXo&*J*KvE{~POHplh3x|8Uy)Zq19 zn1CY}5k@+Il#la3G~1Rt*zM(DV(woT+Q-)9Vlazn$_1iu4)9+uCzcu?1_)rW*}Yn+WM7cPUPz( zOYCo^d~pvnufN;u#Mn)zd%phx76^nGW9}qVKs8ZJic7lu5ni#1nx@tV+vKboVW+K_ zilXM^H_HVLc}xxx6%UUpaB6C~X*EX*pYLc3W9GbDlYpS9=98kWJ@oeoUio8og`cl( zDi6k)8OwrJxD2QXdFtp6)!CqE&&19t7FOe0s5Ol446DM@3MEw!yF?leYPNMv8Mg!Z zyZMxpU3Z$)D#x+H;DDPC9CWu~(c+s4q|JJ*6yrS^v3tVu@wA0QFVsxYnt-GTVIi1- z>-q#Zj*2Z&dFh@%ANJR_NoJQ8nK}IGCzQ6R5P)5xkMsv71-v+siDjpYuhpd49~7gm%lbJ8E#^wka`jO; zAXS8`FVYWreyWS}>YcHFKVe}VRI5kd4xqw&@FV47+x>a-#&4~#KPC@Y$UTU#{p==m zqyqPB$wFXBQpo(KiNxqwC#H!qb_}b{{f2LTRmtGA9MQmF&Pd?yE7Hy!$Bx7iX*Shf z3DK~oC6_C7I&Pp$#~@2NWH|ErmjEhFPk`U|gVHYC=~|;{xfL_Q2x?tRs#W-*CH|7L z30IM|V!EyfGk3@$8cgQ9)Pc&vZ}D-^x`T(S;N^ZZrIig){o-bs)$X{#Tf}45s`*Kl zF3;NGH_V{{rGZ*5UO$%WA--{1TZjqy-s{tCQ)TE)#JE#}*(Dk_RWQD(~+mB<0EOyvhF zq`+D^E#(Na46^JTBM-A(tgoHw9v-8gqspC9>vS9+={Pmyk)Io&ckLb>yWQ(Fl&mJp zhf|{fP4S@wcgvx1vn~O8cCoIb^yItjXXMd*4yM_3Z%xoTsk#gkT4npAZ>IW%=x=4U zI7s8o1$PUVib>Fma8%5+D0w6N9V*nCfD+f-g54_gxR&4pNxCD+8NF` zi?;aeeBE=l)1=TxVrnd1rH4`=%zhYyAFA88Jr+jYPQ-&j{!hAMJO}h(0_7SZ*EMMv zl_*y$r|wGjRH{fo4l5&jsa@hwe$toZ_%lIPUR+?as4Sn>7=*9u>F|oI1M8%R4?Y@n zK*)EtG9i%|4Z3+^*(D-0*hNionbB=LF1KfhCSJDi-l4cf%EePP*Wghn=$?3Qm@wu~ zi-(g{${#aQY5GcZMI$=`G_{nGr&_6}VnaA+RJV;{IFw9%;I3(gV13}&55DE)CGR!n z`C=Y48UkDE6a9EiTx6xbG7@$lnR4uL&My~U3HtFvrZHQypV)FE zEDKy0%<8I;u&>y2^7uWy~>6=CeP1yx)$=m?zPGkI}U3x-U+8%SiO9uAu z)*r>@(ldqq9_hvjP7dLFE(3~rx4Vja>if-OmroT^Gt5OC?J+4Ti%nX-!zGuTYa>+5 zxIV|#Ot{hJQzR}66-sWGI1+e(W01FKr24jXMYCaE%Jc8*TO6vXY5e{Ybjtbx^t+}EU>F(@b<^-;fJu$-?7R||DHjPcnErMs{ONETzkgsD{H#KYvh5-4!eYG(;-#)9NMeBWm*&*h%jbKNGXn*-4v^TJC` z2Vt^P0@4Ke7MeQNep}UYsg=1ryL8f()k2y!Hju2bwxU1lNLM~*K6nEt**xO)HR9wK zG0{*-h6a(!&gV}1tjG^FwUD;jME!00`VE4Id*3L9Y*bJ^l`a=-D z8bqSn;iLJkg(5j=H?9lAmk6a^(r+px%Zu~*Xmax<_2%N%2Cw45w2!I=yJmAc(J;Cr z0NTm~1Rm<{;XO<3dVph$nA5OX{8oG|*_1Dq^euhKf>lgrz+>VJRUbtGhY_>Z4pfU5 zu}wO#i0gtRIG*&hRk|f64FF{k9)$ROUuPgfT<>KL#K33fMZfgL9LnF;6>;>LH*sGB zPEU%8kpq|ng#L^a)r?KQeL88z^mWnRqQdCyG!TZ_jpJCeA8Z6mKa|Hce>NwuN#|SQ zMW%U6UyYjsV8I0hOe9$Uh$eKYb~v?0$KU@ETLaMQEqIhI%nPB0Q(S_2y@|pCT2?C$ zpTfyN_>X|@gXH>*vbO)!{!z`dec#_-_N2e1QI+?o=>H1S_JMOh9@XGVq(Hrud;F&K zLDO;uxm~tL0Bi721X-p9vF;n6S$?>Fl?Hz)mr}qh^eI2({s7-v}45_tqsltI2o{^;x1TMKdDjfM7#Z z{WeP}0UXhKW5{ntkc6Q;2rEU%;E<3lsrj6sRH{(j4Jx;R&TJ}4$?!ZgZEhNX zTmHdQJxN{&tC_2*j2EU}@zKq%nNASswa1#)C%_4JfF3eLWKcqpaD#Og60>A~;3x#t zvfJrTH6M(|urgO;Q^2v#;nmW@MQ>zI2I^k82s%zay7I@pIjV0@>uKnWBs0sBUL8>o z^m=`u;)ckU1%?X=4z5LQz)q!trc4w9T$V+Y#@CledXI;1)HXxhR!iHP&?vua0pX`OR59yhqzC;V)#$j>J!To z>78QT#%?yatsN=Lr(Ty1y__rw95{ZSV3YAd2hP0R@0|})RtQ=aMc{H_)>^=BY$Y+LUJ+G3IDsoGYB6PLg zv0mjY`*qB9y9`z^weSG>6#w=O3Y0Tw!&HIH+(V5SJuD`~|1X(7BmRw#gKVbLd{ z0K#+xlqNtkj!pm=kQSQGW+`NTS@EAWA3*u1&)?;rzgypdHKa9c#RCs)!_e7a-8=Ln zU6ld4=y5pq^DoWP(*hnye1$>;5-rpK9SX{{Fu*lWwFJQ5SiDckpk*KEOqQVG+yAc2 z&2)NGYaHVfpqk+~7RaYYML*gT0F7SH@m?oblh(D~*mxBM4DA6vwj$gL4EIsnpX#q9 zAU+2;r_WMqDz^LM(pvB37!Mu}gg!ZdB#CIwn?FcxHhZ9`R4U)yuDV$dy8$$z1aSev zxUtW8#P+z#w5e4lCNj>GgZBrppuG4mS8`ebniwS59m~1|lh-;L}p~ ziX}=}_4TNl&5v069mP|_I;@^zF^c`4z zTHKan6-`H`W$7!Zisv7|o5_m@L#MOU_o-tAnCl7+aD=-dhJg{*a00yGw<7R_TU^SQ8~~hD0Q%_4v5lzpz&8m=3Wmx4#ymMffR1SeVi~>owSI0g!A)rm1+aAem!~}tmpu3z&7bgfM13xvV|<^zo8F0HS%8ta^Pqt78X5O z0)f6>7{n5!l8K1bM6mM#xf^IGK`K5yJ%UWk%o|~>t3mS&wehx!^^J{H$a(D>p~y`>-YJY))b zd#-_M9#{Z8R z8y_3A#{1orZmtl`!8HJiF}}I|HA6DQPC%BzHaql(D8C7Ou&f$E4nCDl{R%$WOw+L3arz|$bfNpNNa=uR_ytvhV9|1U2SMO8TB68?}tJD6b}9YxI)JbDbGC= zBrLl3oVFK^esAcHW`=5B;B6LA^T&=yi*+zCs8k(OGCW~`4$On`iaEP;vPA_5rtUN(v7ulLVFcs_06>92SA052A*JS=sUnP-*|GFN|#53dAvD+=`<7@52zhZh)CtR-nuQ_V8HparhhTp;FOQwFm~e9PhYiVD>v{s}rhtBC!hqedO|teH&+fNd&6SpB@7)mp)G zdb_j@9~&Y%CI(Q@zF^(->^Ed7EK@@b{Yzetpg=F*zyX7UXB_+g_W&^$#aWoOyWgG* z7sYT7xef{-1K|kiVRm4d*Rf~gsG?^5VF~i$sMqqM>IyAb07!a5wSwx?-`0+@?t%*x zcKy}_q}{TqsTu4-0;7;YqZ9(64nN~S%xr}P3A!gfXX*q$!y`;GWrtmTEt`M0v`?Ty zBJn3r-WdU5zCt*yn3G+M+PRq6+ov9tn(SL!i&AS$>L8-k!}B^6!=|JT zV}Pc&wS}P1FYXLBm-<90{!LY~_P3E5x3FGQYaVd#Q1(SOa4+TlLch4gh385V!rQ5S zuc|EK5$9YZBSOMFuJ*{$qq3YOHk<7koaV4n(c`Sry1j;+p$~7@y!?&ve-ZUYUr!|O zp1@aTYhA4eNr7>OMs@CB<<-FV32S~(7+Ij6duN9&!IgE(G5i>gYhFAgl2VscABJw_ zPup{bl<6<*pL2$zRB6g&9p2kG9T-^5#mRcOAB|9OvJrM$o4OfUl4;(03~Oy9=xrEH z$I}F~Yq)(qdT}s2>(n}pKvZFZ65x16n>4#wd^xUdW^e!ZX@fUPgYnk*UpCl>MIb@Mjt54)xb3)>Z;)LPrA+=d8;%Bp-EH%F@b&;cQ;KhXZ+co)& zJ@5N_dzU>J{c3Lu9lb)AJ>N_Q09ry5(!6NpLh-b1VKlRLxM65hY_79i++)pgNB)cF z7frGBrT>{@g0;K_e)YzKJWQyZ?TG&VC(52}=agg1T>~(-_$mz*lX#`N6pNv>+t-qf z!<8B3!)(PivU4W2m1n}>wJ_m5c6EuV0k6D-&cTwTqf+wJgrAIr4}!8!k)x zcHLn?-@ zSa}DMeiPTA^=E&5Y;%4+LTRBF@gqe>+svN&kock*fCpl1uwXGwqEB-9SW)pQh8*4q!zA`C zWsn34b#9B!MfFJ}ZOeA&2!svGuMRSJxY~hb4ytIbBRM?;itxPy3zQY$#R+EerY^w7 zw6fiUbK<67>a4LwJr~*6S$u)>7<>12Kcgr7xfppO)2l8vt*Yr={)I)I?kZzUY`lM! zexIHN5x52M6tnRlT-C5Iy#S{WeXEFCQ|<8Kp)zR6{iN^WoO3g_i1hd0yIrz&w~{M9 zq;$q*qz8vCPVX&dFl$_knav|xYuoyLf`Plej+r!;1n8G;UL4iNXxoo1eNqXYb@y36 zoDXo4-^`O5YrU6fg-SBDINzk-_yWlmxNNg3f+)Tev*6Z_tQ>io%81m&0#aj)3fc`2{B)||AH+#M@WL8XR z^CBOyj}JRAXkxlg=@%Kd&rz%B7wK|=bHF9q)_mE3Y}TN$$_iRTV!XA|y+?1b>8=p> zN(#i>A4mu{DwR#k;#pi#qbEglPOdbF)aH|Ev5Wm(GQG8QPN)gb@Lp$v_~yJmp&^_G zlq{6La^!QTY$DU&1vCXo{Eny?qz`%1#3s-K;qwb?l=XpE4%nC-X(kOepJpIW6P}jc7hu< z3suHmeDeeTt9Bz_a2`84`|LLTRGUKbT_Aca3wj{8(4N5s{mdXtV7}-SmC7z1->WkB z&JJnK`ZqYckq5%_WZ+gh`<(7{CCHNx0GS_Y=!mE>c%Ej%$eM}KOa(*;?I#=Pz6~}^ zc+DI15DV?CHRi49P|Kz;1I>v!H{cC4?5Ts>IWb?)>dr-@>=oWp+7?g-a0h|E?5%lU zeFT>0jJin%x)mQF!>cx;n8*<3+)uvzp0l&$@OT<8q;xJYyB^lQJ;YZ#eXMyHZwash z7ly?tTvFoZppdKYyq$M{Ni^5B;8FeYv-uROWbIeJ3u9GnPhzN3xdoV!QAXElNDCda zAmJ#o+pO+%>ZesHUk@@`)sTgY#n#u)s4^E4q8Cpmj%!N({##)03CwFW%-lQ{h$z&a z!(%NSe^G$IqcZunkxH2eFHGXWlP47&Z zG&T^w5u8@W>zmY zW=}1V>g}0gw}#nE?Dj^l-YU9tc#KO2EJLwPb?!SKr9_*S)RIV@kLYX>z)60GmdgD2 znQ45GelPd&`S{YQ%;Sa~dz~z3?b(^r{PgFG=Z#U?+Cv%e;Ymibd4EPLg~I^S{1J~z zEo1{1fkxZiL+#PGmHpB$yHpdpW=uThlLByg%9-)#=iwOc)b~JFlPi3kW*>6Zq=>p9 z>%Lc@2Xw|y7UZ9d)-j$Zo_8XU>o~syZX2vOe>%CHzhB!qET70fa*zl=y8KLoFF8lU zx`KpFfHpGLtJa63Qgz0%jD4Z)xxyU&i39{m6w2q9lP98=W;4?3!<2T`b^d$0lC!Nl zP3w|SDKDtk9SL%BeF`JHn0`mby-y*j4WO*XJDWGlk{0BuWEZPfskIWhjv<%sR4I`J zQKvGro(d(4YQYQ^*PvY5L-~@#l)+bz&uJz~8FnOmfyko^QnJeRDTBX)Pg6UK z`jiiCcS@gsTTv}wYNg{fAKPTl2AKRrX)HYvc{uM6KtSaXGtP1w$u}#v}7>agnA!+Td804 zgf|194}XG5RJeg*|RD8_RgU)))4;79J*C61j5sAU0>$IDBGXw$OOx`h$%K-bb3qg zt9!P)`@P@cQmtZy~jZM3;s}dqr5}ai<;dLnEw!u_rqVrBI z;C}1x@S63N0oQ2N@GKyt+dxrKvUutwxpuG{II1y7ItyI}9Yz`2tUNs@hQRwSAIpIk z#|MW1^p4vXTSv8>Frc6x8T;ik#5U&w)!upr^38Z>1MJ@EJ*FiMRvW`r{gvDE+4Bp| zoiLBZ`~XxiVOP8p-pS+!5S;MBia#k91=G7pD}_uNI<%8X$VCyo|`*s`pP+eJsy-f8tr=8 z$4t?QDEPMffb2-{BNhUzGEF0_&4x0RlXwbuv%Ccso>AEwieR57phdC(>FQjkU@vF=j#3OhZli|66Ll2 zF6-g;>n=C2U>NzV^VKu!?J2b16;BE8iM0ufS}RP6Y+VqQtd^s>XfsQh+X;fzFXbqkX1gcKi|@XlsaOZh8UjU8Jt3QdkM8w^!Kz{OF~D~rZD7640cFu z$~@*@NiL9yw*6u_%2ziEwmw$3ttnL#&xj%fDw!abJA^19wudGA0{3=0mfzhHR*Prq zXw_RhaL_~qnAO7qEq)~JLH(T!^dE74p*KHi>1aHJus%!>pXfQ{;23nKH9UDUY z7)JaBsji%B9+$q$T~oRK;fq|6RWNb7BNoS~j(wCO6#e^>Yz_@_Nl!8`=Af6`*^+{6 zQk%{w>k%+?K${`Y7iS49L)32a^{+o^n+xYEY^16jWaCc0+YzDK_;v*yZM|U(Cjq&hQ(N znaS4kBKQoCl8kxYdj?+$PdWE|eD~1Xzrx$3q>Q%TB;>r^fe@8Y2aR=0Nt@_^T#a%e zwYwsRPw$L_NC&asV(=sR-sKe`VAu{mE64al+D`$b(auk6o1&D%Vcx@dqV1*|8*w1M z_e~6;w?au+xI~FO#YLQx29-AU&XM>jXUnNcIM`JE5gXJEqw##-pXU8bjijs->m#c1}j_V0e$-D_xX@x9T0MQxAfF3B+_p@Ach zDf-<1D3r9(VsWF|4tJw!_g0qZ*p7a=y%3QUy_|7}dPfUv22U!##6*b7Oj;*ZE2yo1Ve>(u znL7C*@5q`#9rKzG;>}LLEyJSB$g0A)V&lPkW3#~r3UqbaBGw6CIZ5CW86GpNz8~<8}0|2&giN+x8hZvoV_po0xF?44B|^B z1}iUzgp1v;Lm=fjFWGnLBlAF=eTG;9Uv(ndB8E~<;9nX%)q2&bZVh$k1PhCDQGJ%R z#O-~*Su(3()b9r_y55U_)`>9Ok(_r4idXY1-{w{6^>lpd)vcAQzO@gFgaxS?W8~QdkGOUoyKPe`yr+A5 zyghTo)h;0Vi1zpHBWEy#;Lz^)7Ki zJIb^CHf?QaULop|W~A4KHwjJMr-K5_Uh;skFR%KcL&E`4%>@xR<-ktM28mKii6?Ib z8#5(931^uqSWSL4gq3?FmfOBuQe%Y%S!K8QhkMlJN$MqtJFnuwFQxN<+bmy zY`@jA$ptKn*eND%{Q7jN5@0R($-ihzdjI4EBphK8cDy+4j#ct9LMU&iG zN$?blwQ@-Q@N(`wgtYEbuF}IL^{xzLYF-2cL9j$at=)C=FoN8!8UjDn8^tE|2a)Z+-{3CfwkB#N(v8s1FYgDU=0dDz!UpbRyTg zPlermKz!L{>3DV#O4x(wcwTM|F)~8oU+`yJFK9DEeu`bN-e%Ip{ldt$NF%_L-%oZ4 zfkhOIqMv%|kEm-Zu?c@#Px6DXZ#tv8ZyS(&Q(A9u6i_2CT`MzP&*|=?5Xq3*&J9Xu zWfaWr>Z?%&D~<3g)zgr!Ikck+!J>^AnauX+wm}86?+Er?{YS3DFMFuA(kf3}uk>t;gOv zCK6p=Fw99ml>4;a^XCS*e(~??^GsxWaIadu`Jb1<`ibJ%t(bQ><#Cw|IWP6g>6rU+ zlT%5PR*7H`hUv|OoA05*rAFvY4J?vB;iT*ar5X=9*X(qQIbZ+yt*dC6zgERnWAW7x zOQc;zX{VPJ*Gu%08S@8$bkvxTbPwxMFzAk?@1n4#ot9b!8OU{^PmL}oB; zY_s)EX6FeXAqp@ETw%Zc#Nhha0Dox>!;#NQn_p|mk`n1b_PHT9M`z(Pc@?<=+M$ub zGuu^OS(e*w0&>Btk}m60ejC(++DchK!dBopijPbEsb52LgFtX&8 z%~eAsNwLERB8M}ZSKdV6^#oksX5qKG#Y!!AYD2JMz^tiNvGHKTx+4Uz5Wr}cQatKw z`e3$w3=L!KTN#YY`*6tOrgfiW$AxYY+$a~+M2v=ASjy#{mPd@37VHT7au61@IN4gI zS7ouO%Pwr5fU&YzKiK~8EwsePrrz3B0d5=9ok{N*I=9-=5wNi4(Wjl_osu>T)xD|; z-H5LvlMmMVX?kj0%W=-g)#bYxfhkybtF?|rj<{|Nq46=g{$-vG?Ni8dD=AJC^=m2X zKhL)kc)sKrWLopbi?O_i>4RLaEs9d*59ifLsv;2{EHtDet>#KmCd!Frz#qdvUA zb9%|rchev@z)hsgJ+1se83CHR{*RvyI0+p|@ujv=7C40guKX{)=eRkRg9fY0wRXW@ zN6UL&l)YeQM(e*l4@Pb}fbXSz$}6x4x`~mGSZFb?98zgPRWF!eC8$}lpkT@PRV-Gb zmap7k<+$pWpcj=gSU*5w5Wm-AA*bMhQTX*HO=Tx?q@SNd+Mbc29dI7}+S_DmxXX(u z``qi`f>yql@|vH1@U-WXOV#7+5my@$WZXIRKao0_q?jrmer%_2)x*A-+eB? zs&&_(RBV7N+Qxbu)?dTPTZ9GrO8$>?T&@J~k+QkyOR1zt4w?32Da^Gf*yvIeY>hi0lPzWc)Zxy`N?`Tgwa)CIsvu&=$WNw@qZP%{7Q zQu=qhQY!^eec8o9hG+R=>XMU3`{8%gDGqT|#d_tIP^TR8!aepB4~^c(tHeJPlv?hR7#v(WlnZ@BuD0=K zIG>0}N%8WBAM-5LZfA5SS);)&T1G5E!81f=U<50}L0gSo?BQum6REOo*GoDq>@wCz z`IU7gwSs=Ow0FHZ<>mjpST_ITLz$M^GJQG`(4CoHLYVYVTm09*iRs|c%5U?jDIQh2 zd}eoKqfc?5o_Y>fJH>O~q+9&SvUB%E?tYn0~oirtt z0}G=#MYy3KbBlXdT(uQ{y@i$nhGnmq;?l5G|McGBa;1Vkdbv#2R=N${ZcFRNYp+>J zR9O%3OUF}9yz1PME*zMiQ~tZNqizICym~=&E1_Tf&o0Wi^WlXeIucsWxO{k2ZS2$-`2pl-nsw!G*&?omDMt{7JV%f08~ytkZmzQ>%n6{yOblL zbTv#qO%5)#;g%}>X`8=LIKgISlXgx>(HRk;vV^5M7P{GX?MC2tE&a|F{B6$lH2u*^ zJs7FQ=id%?ou0_mL@X3F7L`T`GL>CbDZriZ6jEYud(63JDAmU4qb~`a7~FF0zm(5b zM}!XIMCY5K7W%le*=ysR_wRDt6#hT<-hwx-Zdnsf9LJ71W@ct)c49kbW@cvQn3>&X zh}~vprkI(TnVH$sdC$3bzR`Dbf52#@t*ut8du#1lwF;lA>VWCI3#PWsSDBx>ug~nP z$AN?wu}>6G`_!(U!*}jTqn%ChbMtp*1+MTz*(HMW*1?*TttIc5#b{}MeKMl*(-xpg zkjeknF8}9rTXw3CtCq|k0|ggz#nocqOU}o^#jLC&>nY=Ry*VJq1xs}(sJIJY%F|(N zgI#){*OPW13oP)J*p8srRIgGpY)>3TkI&Zv136n{)wM%C&XRo_e5OO^$632qQLOZ4 zlRA8ZS#Wdnq^lX#ODkp+d!bl=lDu0N|37WbLiZZlyeoG(+CNR+hs&Dz1V zyU8>KhQCQzI_HgvN3@>kd|(}@=y>mRW;)8d`3qE4-g*k6#45u@2B@|1hlL$ah3tNH9PLBzrDiD z>EmL-aco!2)b?$P;=9#NnB9uE0*bdz&1R4GUlv4#Tyk_M&C(3-sjcEUTcierjPl2O zp{H%tjSgIMp&h6W55<^rZzi+2$Ie zVEi3rWf#5?zF2*YOm2Y+bAY+rtW;=y8oP=3ZrK)Y9kPqgMgOV$bSILGJ> zVUvQKOM7d9*k&TH(l4(`mr-$h(-ti8bIGHvR5EwEg+N53x!sd&o}pr_3Fa#tap)~d zc;DS5F_T$hXgI=l)eR%(mwS!G)n8wNEl0`3DN7S*h~$TekIq407i=gER>2z%FDz5afS z`=!!W+K2~Y8Q$Nwm&2FkTa+QDYnvcGFeSx~)qT*I6k3|Tm6(r!7$zm9xoKx3<=cQp z4RxG~*Yb~^IETg#Y+f4#L3tvBJpO|Z^W8&k3 zS5_=9%TqUg$nO6r2C=x&0s5YxWJ;xO7)lXD-zwW+&oL|RtMR}!#_MCMomV@op^*uh zFMWkK=6z80(D+TQS zN7nfFmz}I+F}WTA7HZVT#@hF~A5rV)uRd1N{0E8oZyD+X!h zh!*}gA^XqN@)9ZCBTB31vF-G!BUEa>(safjf5J3>yQb4R6BL%O-xy3L5crktC3Zy-%tMQxPdW4NGyoCmN0v35z6TQW1w4ZO0yj}CQ%q(` z+^k3W#k=ym0P9ATSe(y)f6hL%}{@H=-!HlNwlNNoh{Gb08~Rfz}w6vG>a=j{FS)6n&1a`dYTL672i)qK^^pG6IBDt z79;Y}Zc>99crLcyuX_u-lwD_h?mNt(HIOHB>e6y@qh7w?z8z5hRcevMK!YhR<@;@+ zjCC)&p-9x@)y_&My`7q6>C&H`pzDqgOL@F=M& zqlZiWcBV-f;KAkt^U4~ov#M-I-5WLO8rjx@O|W-sK&dwdZMX`oY-%QDgf& zRp<|%{tOpXCg&>spLq-Pj>rVIS`=13t81k!Rp8JCd@?uYO)WT~iuGUXT)YLnSp=ip0znz?zv%1Ss7|L}HdsB!^Z~pf;@ug& z1NeD!+7(Tlid;<0KtxCn?#sEn;eq57f49fS0i`oFJNU4Q`EVPA-{kB}#$#xttDc6U zHGW!e=6HAauNWt%zk4N&+c5`Hco9BdJIopa14zAmvT{4Xv*I-yqM>;tP)o*6eERoO@Z)?6p z8ph)~aF{$?2Z!oP{b_3)?qzIoo(Kc^4u*!)urahu-sBSd<-5|;YHyo!HPEaPI5SMO zWa<$joJ5-M-rDd}%9dT($YoG3jZyoUJ^RXWWVaNseB3pw#<l+;xKk|g3?0T*Q9Kp%Jtkt$d9C`iK6!@VV z0p3@fgJp2H9l`fP%`!)tb_YfWA8OB{LL>dE<@%{+>BeL))wj_m8X+? zn{%>q+?DuR?5s+=?*F=c-hKUk5Kusw-r=S*i35>y|4WS$|CN4wBL8+Fk4k3Na`m~A z3gLh;v(xj|o#W)ZZ#oU8id@@0bl(oYCox+eHe<>6Wb?W(JOO`=b1 zX;>kx7LVPzX~i{vB{B(~CjbLx+K%^A#xg*z(JkuaJK@<+y9YwYjtdrO_F_J>Tds2l z+TW##A9?DTR5}R9%MY7usOQsNPilq%*E{~0gQ7-~|D_{+OC&p&-C#@5tp#QEse?^+ z1YY1oD(tnAd7)Mgz`>|DpMv80VgM%P#o-Nc42PB&vH$2Of{!1E#Gy7Mt27Ynb`y$K z`Y>j6qedGW2yVL9<^%VBeB)Ulht^o`ZbAzus{V;jf1*VVt)YZzk#FC%3$}7rNHjl< z0+)7wuB|Hve&=God#>4`O}|FyHfHS?m$$MWe4a(y9|cgpEts)b|ArP=_Gwp1r!!2g zvneU__wQ!gj0*7Ce$aG}s_7z1O`U(CG?G9ljpX2P4*3F$DBk}@Xvncg?_;Vqo>p(} zCD6{Ycs<5K@H}5SEVO%N^(`F6C3ESXa7~0%@8b#X#HLXOX9{SVi=38Qr&g<{7sPvk272dT1nihq zqv6tG_tFz8U-_yXIrmKYe_lf002|uiY}%>MLV8H$uMW3%+1=*7lIR0b>Wy*fz*oo) zg*ta;BT3^s;e!Y_!)A zYmAhb`ZmiAO{eHV^ybTQx-j;|4tu-buGnA>vMUBu+|%>8?z+kyut}5*%mHmv;`t3N zO(jYpH2zmQrOR+~POyz(&jseUpq3TAFSQc|Bc_-!$M!ZPHCWsJ zM*=*TrMjUy=8YQeLT*nHd#OyP?Iq2Mh#g`-iYbp_}s~HL_J51Vd3RQ`v$WSXEz=PBsQ#4Pfw*rT;;*O5V($-{E z!Y2=$v8^9j8Up*NPa~Yv3xDX zy~VFhL8ys2ODw#wuqH~Sc&AHq2TKWT!d^>Z`+g&mm)aYm^G%hkMTOQ{`29p`bOvj~ zuwII3@eSHuoP{VVaUs>nTC~Ac*{W>F!?c~v!EC@>4nxA@^B(s30+YemTRcgu%=XB<@VEpZ#no!c)ObZI40nP$p1`p(#=cX+ooaa{foK z#c9vB8nJ==4;@%!~WJPvV@{Esh5GgE=$ z7lo##-IwHSK!REgP2*T(Y^6>lDYL3{w@c8pV7LI7@>#U-P1&_79c|5twZS@yTAri@ zmR2S!s0d9o?wg$`GCQMcr&lCHD|f7n{1%IUr5!^A=By~>&RcxKu7P*B-mcR875QLy z?-;93sLM5H%>1+ENh=D48P@@SPFtU(fbvK6d8DYsV1qqV#{8~VJ)G5cXlx_3dN6P>l9_6K1ora0hpL z&E{E+>t+vBddzsr-CID4Cf*c4=CP))a)w85OyPMiL1_u1(v5yYQ32&wNl)xX@Ot=M z(hcd0045K=E6!)TwX!w*@Ni$#yPS0^H1^MENs!sk$3q?YJ>cT}Z*s%p+>|<=2S=in zFUg~2)%yfGp$jWcVw^A#;~kFTEq9sYNBk-+Wmyj%7h&?K!48-w8lP+5vOcAb>!?V< z;A!A0Cmbz;eSQc}mEYB`r&&;puHyfs;ENTl(YlXDH>oJmQk-@o0b-Ev0_-@}D&K!9JwkKbz z#iFbfZq?WNkN1lRCLO!nOGml6zv4TZ$f!*6m;gO8AXi+Y8`LZCy{Vr7l<%D(i_zsV zv6i%PVL4s*tMkxF-gEz{N#k^M+My3=AT-~{_OV{~woTm(DAudKbNpZ^PHokubt*es zdB5gm;yHsRoQGLe{D=+3mWk1GZ=q-*@$Uraz4NPji%T?6mOFT1xh+4+WyiYps!fYF zHOF!K=yPK0&(9@F&sYfcDAxTjr($E9@Xp)maR!5b(soc`d6)brinT2{J8ymH%2bH@ z{;#LzFDVl%2+@z2$&dZ^B~jNa5KqrH(4174r~N0HYGiw1J^=%RTQLSa^;mYlF#eM6X4>CN%ieM<(n=P!YOw#yG@zsrfH6(9lQT_4?gxJ#c z1GlHy!$Tg48vNtN5$uA`T&^@ZFlAmpPRvh66w7P*9xiquI-LUPe8t^f%Z-ALGdcXr z>)BilDcyoCDVhx2jsm-@B!BLfd*gi4!u5Tsb-LMHh}xY;>dytR<^k%_*IlJPZvO;Q z_;MPqyP4J(m8-w(9)Vvz`}m$0W9co19A$z}dZ_!9&-AVYj9hh@re<{efbt%&Vw06zYebY|c1D}fJEx?GzdmxlnE@p=6gyDamWwGH- z69ox)IWUJaZvlsbowqY;sInW6F;+@!TVVLm>-2*aL3l$Tx`!+T#&9S*T)gNR?$xJxRU`&k{hhL>LxvB*7c<+ zNw%a|C^vf<;iUs=x8yiWm<&2Byvp;@9%Z!Zpo6T)uIaO|?-R&}Y|>NakTlCXtpFwE zDuke*_xt84{(v0WWQ$%(WXKis;ogq;S$d6k?;w$2Wm_M1WdO|_>6J&(kn1z3d7(o- z)4*cI`3>M>m9*S644#mRLEhAxZsp+s-bw=q&VPp|Y?F?E)u633JD+E}uw%0~H(N@T z<11y6)R$18gCR03Hj5(K!98c9n=hTY%%IJiXTC^%VT-d{-cX>WP+DHCYI4@7$+2W?n#p?(6J`~gf@{fM6tZAR!G;LSb?5o}p6HWin7*KPmWP=+)!CYR0 zLH4*1kiYK>g*c!U6g>NgcS-uLzT&3AKgL?r@h(2-6bbdZnfZPwMq2$TtDSrd8kaP2 zojc@gta2(B^)zGk0AD&6OBqSRwhJ2aoWY%Vm%QNDgvD20h2^QWHB8wLr0a2}e6X<} zpfshxylb=E(YVE@VgT&uFiAANO^%D3s(xKBJchXd-Se6P5kT3Ce+Gx$gP0}L_2A$) z?@b{ZsNI@M27JzHg;p~XhkW*WbW{FjY%?>ntHEwMCyx1A7&qVJy|xXL{>i&95qCiD zm{>p{$h*ru6(QQ@v3aif5ub7N$&R^x)6$ZaVa1&zPG_ke_)+*GrMo3I7I}|Ia&J#} zdUNgvZjSGaZNy&xSEv;V@aoHh?CB*#t}z{?{m>^yy+@I1iNyeMXhTvTQ&== zxp%J?GTLL2jv}p+hk+=7@{L6+vx(Vdxx{9(m3O1xwV6>fcy2hvK44g31q4Z)z9+b> zpX{d_DTz49SZ1+q70nLCNPXU(D>!$oLz)L%Xq|Cu<@M0AbItG!==U zOp8%to9O~ua5eiv{KeCwHz~Z?bE-+NuKM?)YL;(GUJ4nf^;`IR5#aUkIjpX^h`c%Q z4vS_?g|-%Q!i5lP`O++SD(JxJS(e?q)cS%5ZNB&N&a!6wr;3ds7}IsWTfG(wwsJp3 z8$F(o$eK{2^O?WK;d|+VWcV}dV~JgW5Hi^pv&^9B=Lv>uwA*uWnfhvN&4dV`#V(tCX2ZK8Yk`YTNDZY<21F}ou1 zYG0DJ3Q)AWy8ji#j+7XfbX#YD`lJOdDr8Be)EbSDmuF8hU1CE~i;&glnd|20?<)sA2P% zq5Yk`WhNQP`B!Zz4gi2kK|Pzx(nRhN=2@g%voe~DI8Ci~r66r`u*^#B(zmn7?oAg5 zq`a9u?Ml?T^U@zj&62k)=Acqz`T@p{<$JS8ArGkmwagFkDB_+9Y*Eo0d^+VoQt2`8 zdOJOiZP*$Nn0xw>T?$^P5v`_a5VXv<64ge^*=Tc&*LZ8qh4V>D^iHp>@oB{)huL@fP!P`E*{?9rHVY&>!j>1aj;>+Z21>jh2@PfUO&I zoT^^pxCZQTv6SG8;sh5^b5Gh-(8LVokI7CZ-g)&5T}4zAkHdwfSQ03%CQL+~IP)g( z1jCKJ>WemegeOvWDxZxwe3X81UbrX<5PL`IS63=EMHM3yBGM1dnUi_6g1@(Gj+i!3 zT$p%mSJ) zO_bit*(UUtvR&QQVW6U67E|TZV~0k0rSFP-a|mWMtjfhd8hb<08uYRQDkx^c%Hr2%2#6n%! zoVU}@A&L}RCy5z%_NV0XvXLXlrHIKqema}lqs=~gkD;?Yh3f+|#iT~=$UeN=f^qr% z_V}QYND=KldzWBRzuTFdT47HGo01%1%iW0SIhSsA?{6wm3j*?h5rxeDBFh(gBsb%um~-A`wfSmhKNuFa%~*!Cq=8f-2T`FzFd#0v z{Gz%;!a*JJ6a)cmZ|589w{kRP=0zh$wnP8Q3=fmUYpbr3KVVqvl)ySQZOqc(*rSPH zqUWvg+=5|{Az&!cCSvE}LYq)8&4^W9&%Q(_;YB{cm`eYRgaPi0>QfQMtM60~hFXvQ z2ov68UvUTKdryL1&x{RNb=;z58*kOsjOlJD+Tek?Sx7T-1O^T%|3|dUFW8uunC*OR zSq<@wWnPVxI@)k4vr30-L~GDSt?ROSa%<`+$H=j~6;)Ht0-jom;j^lFp2d-Sbf+q6 zj}Dbocqag;P4GGNRcSHADD0Q|Cx<%xf!dj+5%UEK_A+Iv&;++prK4_|@XMwh?6^g_ z`wnozN<_@>&iD+-K&m*#r8k$XRK^ z6O}I?3(Kmhk_>2SX*29gE67>dSuN`%Ya1Np6&O4}eXVSj`AXPWT}(C8pE6ff+(l9> zTf}9S;RaJJ-mBa70O9bt<%4@IuGiDsM}16eYBp6eH>u5g#~F9(R27DuqJ{zKygJ6Y`X>T{1$HuJ z4prsJAGlj;;Oy$ktjVv{v&B;MI%oZCOG`AFAnPY0SRf)D1rV4M-}l5iD>T8dn3b!d2APuWp-F%tPdpvEh>TQ zDh{TQ_;c50&HBV}&a1FYkQs&Lnc=wjFu6l2G+wp6>V_0KRaUI56ved0c-96`P^#Rf zAA@1TT3ae@T1dk_WFf6VmPB)kZv-}C{u7lre)^gLygk zBj#`SS<>xsq*2Z0A~x$ftH6zXpkZ=B<)!joR<_~?v(qTX|5s?QLAxpue|&wCLEEc| zGIgmA$z2KZ(n*p`qlzXSO$zwA9dYw%%hTkF3j;| zlGh@k_7B2J-&y?Bir3eQ6TVj_lZ4WuQ$@njq};#<;zwp$0}Q`PTAiMmCVwru&tvKo z*`i!Mg$idhTzOjo!(GlOV6%o^x5K`z)>r{2%SW69!bSkh`eeC=e5U`jKbC8^m(`3z zd~#wcxL*R=a?x;%zV8SiEZfGQLA+P)`n8%-f5){&<~})ItrdQ2iqyWb<5@n}21fg4 z3!OjgDGK(6Jp1h8A$mO|7Y5?#a>_0bu3V~-`Z6nll=qJYlNZz-tA=FBT7XO&`|J#Z zuzz~aSMnmwZMS~)=&P|PF>n6bd%b>dddRR4wH>A^zo!mYvan(G&gV?E`a%5oa-muG zZS!jOd^h?eurVSG&#B)|`#eAAzKc*4^(d1fbou_VMW?3$;GXJ4p>lF2%+j`wjmjcKIEs@mSxgsqKh#bpXYPvOi%Xl2hgK_O|E^D!8 zw`S0~Ct7~#TebsAM`FXXq(6vv*$w8}reoQv!XRNV`@*bxVcftQ*xG*ee|kmVl7L3-gTa^KU)?k*$f21>p~irWEB&%;Fpp;Wn z^ig~Jq0&Ad(r2~2F9A+rFX}4H!bi+qdy<*{?e${ppa?dj!NJ>={k3Rm1I8!$eA@bud&d#vP;!+Y`_gXo%O3gPu2Brj%S-A&;YVezANtVaE3>Zgb(9?CO3PccMY-zrV>s-u)x>zP#-7FP zH{Q*$ZaA5724%E<k+Cs;E<5ryOhVxHGhq3K3%=1K5-WCEQc(&>P}IU8 z5Wof~cIZ}ImuRJ(XIj3>IMhf1^+@&^SwWca(wwBXI?k$QHaryxX@Ls)J>E;xE2R!) z8R@|HioaP%aHEe}zpm2W=;EW)_dCgzZfareNL}Z#k;X;3Mct*iKvDAFrXc*ho3f*Z z>AF*R&Qz7Lnvk#&135v>!}z3ZJ+21Ju5dhsWjEu94Y|1tUljYBGN?=b;D>t-zFd(H zeX{Kh1xmG;nfnJRsf!}#rxqBPujB1j2yzxF#Jyz+C-d8CDjK2R(7s8~bf7*&Wk z8IZ1E-%}QSXFn~Uh7jFMi>k(Ad0T}aX=rCP(EU1QL9_D2wOso|qrpNS7m0__L$56x zz&a1Tqj5}?5zd8cTVDsMs$FVgzPgz#4Sz0;_%` zcxlY(W(|0uvhF^tnDB>gai&y3xdnn^&qCmutkK(-xP#bynn(5JLpLEGWvZ1h{Dcd> z{@_^fhF*U;UbS<1lV-awLY5I>GyaC`sHWn-J!8giAp|y`_!C%)wf?*wO;}13sq=Jj zB}GN9C{Ih>Yol@@Jx?7ej)H!slUkB4BYkl+TpIeUf$#Wu9ai0LMFjHd@|;i)jW?n` z6FSTaP#P_qhP?wbv@0=jBb6Rb`gp^p2vm7Z+&=Lkv$fsn1et%jI%E|Vt0zQGL%7e@ zyF;%s>c}sG%cUPu*+3}0rn zqm_CceQ?HuMHopSF`iHhTByNBRnqoMKFAKHXx+8pZ^!m%wcOAbksO{Po2?FKtdWVIxjz zjwTtVq)X;yi#rJf@bxdqbW`76y4aQJt89=NEMdp`I4?pzR|bD;K}zbn1EbTqdHT43 z`RL$m@TwA}f6yeVzdri4R1ucUUWMb6L{4|_*?WVEETywv8K?u`cjet*Zs1_U=m+ye~uI-C;h_*axSs&f{HR!L)xUDXp{Y$ z7L8@%9)2XfedDz^CKHcW{lIp8XwF+=xZiaoELie+cMHQm#r5F4S=OuSYfBfHAVDzM z$-0^M_iPRANH33$0DuG1tXD)Y6W1SnL-|6H-1fJ@N^yy;20GttD;A(c($Bc@P2$Az zocG>u$sX(Frmff{3FcAkU)b2Nb&d>$&zqq_@>m zru0A8eyi`zi%NLHo$0`(I1$a_b)2}tk;)bA_&Q)Lv13et`ef{prGAB-pjf)#7<+s} zYUB07z`7(PCWhg3xMmc^WeCZ+=3M{MPJB;z&&=FEmCL#CGGq5Z^)$J9%|@4|HUdJv zzJTo*2jTx;|=-hh0e&r zLUbEfou^%My{9K#tG@{?(!Tz1I2>EaVJ~2(my0-byP#`*3Blv!`H2dIvNcFF^NS(w z6ms1!ZZ(LP$m_p^?Tayf1dji{-(QEa9UdBss9=zy@B7#t{Mt=?xgL1aL=i)pk63mJ z+X6v3|D@A-7SfxeN$+By;Mye++D8Bd{zVm%`0%<4SPysVB$4w*c3i)u^WJZAS$MQW z5K#yjG^(1(8Ipv3OoVVeeUB=^A)~?P)#C**n^QY_lR3Xa(TL|v zq=rZ;|KU%FfRZ>yt(I$}g6kx<`OKBM{krf4&Lv$*v0BSF9)@hG35JMgqm{T#D^EroPQez$K zczsiFUC60)Qkwa-<$ztu;e%IQGs0-dH5ELrtFV?jAWL{SB)u6zxyuo_JXjnH3Kesd zq}d%Ys&Tr(MJY-bYNDzgVlgD25XsZlkXep%7>!#rBc?3=w3M6MBg?LgWMSjQzJ>{t z)zqXJ{s7|`XvHDlOd(JsoQ(yPgA@Jnl99;^Dcn2!PkKiJlhIvnkbilEKaq`sK+%uAPZQ@6i#aMq^%W2Nozw*g&r6yefL;Rb?id zkbET}h=w$i|}r3Ht9${TJzqq@Nca&22{p9zsl_P=vv-R9Zj2*pMCLMEWglhhTq&;jV(dyW*tq#jY8RHmU@>J=OC$Wd!Y-mQb z+KIxO`25w$Eg6r#sfPQ!T_yZsti_;aZW_sgJcWYJ4P`vzI=kSMrFgiktprj7_j$q$ za8SqN3bKxD>%O4S=@c2gjx{=?!5A|>l)THP*N3=CNWnz$*L^ApH##xF3d{}fn!_@Ae5<%&25fnAM|D161#Mfyz9NDV3ksW_wB9P^{;>9G)&r@+NT&RX~lNMagi`S{tR-!iG1|TPQKh(yo)W@gp_&OToBa-5;FiLI+HF|~2tfk@8{2sR4$XJV8M0|{ z!(somfHn+ zX#;t?&JxOG*yc|X4e3XJaby~R=V)STE>ef;%`!uSzZ~jfb6}X=8V+)|1)jd)7I=Ge zJ!z4Ly)AvS)~>o*)&|9I(cq{(sY#SQ@9oNi#I`C(+rq5EiSNO1?O#*--R8as`s=Ck zM$geaEnCp7IM*KfaQO&iltJM!%TNw(&Uc;=A1lmSvW-8*-K+$ zu?Jme@5R&s#F!dvo{^gKVBVUT@^m{CdTARYma|3~kR8n!He{w4>)f`TWWk|Y;-vkd zvjIgQ=TH%>C|$PcsX9*GL3a)pCIQAe-OcODD6fd1hDufhij3Yv-i#Z1sL0);@miSY z`Ghgu{SqaLuk90z)@ER|airEmb%|@Sve(oxpyMRd6gO3R6K=U`ogm6n4tFOp`>D!l z(h)>^)h{7+r%~ofaXC1}pU-^+{^uE5o zbhPu{2?y8|31iwkzjd}ra4?U*t#VA_%4}jr0Y>h> zdeg7XoJ7?)t;@FtFZVK3?CQrVq+q#Z3G4WokIvEr@jl$~zpapM>)|Z7p=m^ae>P70 zvEA%=%Mn$=CHWl$oFR%Cf&y4jQm4#jwFo3}_OlnU)k9TIYNvz99;GbW@(dizR9sq; zOS_tXF@2iz0-^R`=GGhVH15B%V=$IzE5E~)W+uxGCy+D5LK~yesg&XwDIa*Al*;Z` ziJyd>jr$G7`w+hgkjqm|uN{Enl#g3{IjVn0Zmy#~8vG84Y^=SgdT$cuRpEhOe_j-h zQhn$>>jQ#}cCZ$2I-M^@@ycb;;(1?`|KNi7L;BdCl6^clZwi3W#AYv#uej?i7ni+^ z?TCJWQ7s72EKSapXXj1b9riKbcl(O+@o31()uUf*noSc8K8^To{+F*jx%)e2;%C{z zP_>MSt#0C=1`6UJppH-*(Qt?Yy2>WG-Hl|!Xip@*|7sbByD7VJZdqqdS+i2#X}Ag{ zszp~4b+q8|;L9t=o6Ccx}xdN8c z76gyzI+3M~Q{M0b_FdiZ>Mj{-%Pw*dRBW@D;S=cCfAIxrs9bgBFhwS5f|iT&PUE@F z<-Ob`MvX;Deb*K{*M8_+D{gbUVg}G=+Nxc)7x;}}Rak>!nMJjcqE6yE_#*R^&(MkU z4I7n~3|+(@eMd+i420jV)e9QG2Y7f7|2BoFepD3>J$#Obgo?-#aQ8*7(=(DFNPB%a zOF6J5ob`vQ3GsvED{JJHJ*I1^(0Eay^O{&mSv#ZXevN`k9b7aTwg55iHbG3}Gdloc zS}n3Fom3UgrHc*yo)1Udgm$(HwPcUHl4O*oFUg{I42D~U_h)y(Qg$)#>za0}tts~0 zNtiTzN;zWt8+y^g2=GHWmGCi5SQ;%%!?vPHEdnlDUVnVrRh+E5>m|KTYWPbl%?XDi z*s4ETU}*>XIA6WSwU^uS2n{vspJ1dWF_>`liEy5>GID)6<#!QvDCd*4aPq8`@Kx^m z&>Xf;Zh`QYt4^}+Ul4*kpVseQl(HsJ3NrmQVv_2iWbn8mzR85;up7{6J||#Wl#MLJ zFDD(f*WcUN5tJDsi%(U9rdXi1JDzy5$es1el}GT7U>keZz@>>R6HxmA*wn1oRL}%c z3Xu8ad+oYV^&mi$f&OO;<(1`&G3b-qu#5o8i09_&{fRlM!$AB zp1_l-tjxs>Wby+>urz2V)nz2_SX?!ez}?lcB9DrxfYsGgg{8%8N2}zKQNFRq)wJ>Q zhp~J>VA-hY@c|QrXB8{P(J|A~j<=|2v|w?Srz`1q-Q3=!UN^5t+-3vgncnzx!0P6{!g3n+@2(9i68R&1z#+Z(F(nIFH4Cn-$t5J~cwp;f zfo9$A5bik%UvKY zsZkp$!j31YFlDY|GPv(Z z0}>Ep764U*D+zwQDt{XGNJvu$PXZ<<8WB`|K|uJV<4wlud=}Pr1p7{KWD=r|dax%52gv8L+j5Lu;nkH(VT% zPw}A%czoN2fyDVbGuL1CwanAcwrc^aH5;r*Kf-nWUU&J zsBDA><>ai=Z*BsRdp%t;%Gp8Hlyd5F7LEg1o}ylAxzQo)k0=AKod$}4rF(MLQMRjT zfiyWSWB$S2xawsv=p{BIq_!Dnd5Zy{bQb_ClTj@>H^=~1RU2e*0XkLt9iWytwI4-{ z_2roujg)CwtK}69{kOaoB#I>y%$( z8S%+CcaV_?WU_#NVVN&{4WXP#B)@HNZysU_-7aNm?#qe4xy2;6px#!mPZe@EbE}(N zF*3l)uXQ1U4$h8>7YQ8J7DJMC2X~w=_EXQofl)T-zMcloCsX ziX%!H;yk_+qUW22@b8aGBVoQq}iV%79YTiUxE&NMRyejnKTrzI5$109&R zYug@LkJ`Y9ff60;G+qwM^*1$kOtHftP@g6aR@cD9?uX&@#TrJNv-AM7 zt^Gdkf1N_meIDPpDtdpb&MxOFGNpnt4csXGXACL^%-GoQH*4#=Fx#5nsQL*1Fv#H{z zMEqZzo9ql8WYI)sakDT?oGa~$mD zR`VYFwawreJx%1Sfjq{8D5dVZd8G5caHt)|8~kiw4MRKNSo z$da3j+l*wIWlvi43j}7(%nZk9woIg`%S!QIblBIEb~cVX^w(q#w0O8cxCgfM?|j@@ z$&B{nT-?-HYEX9;>60|v-O@mJm{iLXf&=wwbucJ_RBb=a6( zT#K5UQ+})kwz$~d2>x8hS#bU4pC*?av^~_|E!leJ?#=Vx^e6cE>vqv8^gdEMp%D9~yrxN`Sq%;!{EbgSHaUxYH{lP0 zmccZh>28Wao9Bszt?X;y`O1rcwggE>SC{9G*dr--W>3+buJoTeV_I4%KDPzbI};1z zh5s*xb|FVdrFlHPbIpphixeRcrrXIycd$!nWOT4AEDo3BMaV)3)3))}HoyV|qxEex zfj<`!kxmY-)w70Y!?m70+B~=E&0bBbtxlp*!`Mg!)PuDkWC(O9NzT}B{F_j9Bp_K< zmpfVw+Uz)7kr@sCJQ~00a_jYhHvQ8P>jaWVymtxas%icG1_gU&}TYh*i zXPBIAL`Y~8lLFlfJp~l{|IPXWAp)1=4PDt291jRmcI3&UrY2Fmz#iuf8GO~kbPqdA z7rc7B2yvD@DHXNP=H?c1T5|b4fL|u~iJnKcIN{6MJZ;WUcN&fb@FtW)L*jK@X(?y? z4?J|^vP!?)eHMg39el(aQye z;LT$T!9ax~@o7-ihh!-*;0$?v#*nx(^eGW9o%75`*M|x8XNHdfO3Mc}h8l;y_q~BieYN6B^CqPn_Iy}JI)=S3oO@T*yHtLKD4AW5Et18#u7W%o5jAD%4M{ikz(OXVA z_4|h9fx88ahJ7FJ^Y&F(7qD5ksK)x~y)e2uRoA08m6@ff=jHZ#WktclO(?~aQs!nl z!#Jh`cec3~fql|^2Rn*HU!S*4RK{Y%=YNGn6cUHb|2ml6&2}n(gcEdO{DI2Hs(C7& zP8LtzAm!A5o=EwmP^#B?J_Yfe&zmp*pmeUJHa|i^&;|r+3lVEQ5iZGKD!OSPh?e7Ac zCU(YWs`c0^${M`yFO_>pwsm%k-{(?}cbt%N^2CbMJvD)wxnwJSUKZI_~WGVwTK|go`UvxpP)p!!ty1BD2QQQQB zHk(A!DUNY$-rg$F6N~PeD!KPAWc472lfjmMeat7{0$5ZoJWj2BEkUwX4=(&aNw)S0 zl2OVLftUGQO3~XPBFPnQW+#lq{zsRL9yXYX@*+RhzC|n8Mcj8Ilda`OWqacpCQVDr z=-Wx$#jhJbTJPD<@<<+SI2NMg1GU@)k};Q_#V}Mg`x;p?TqdX(6?e6(?oih+MtcPd z-cIPPVcfbYoL+lzk*YUH_PRfK1;}fuKt#3UU6-R)9dXL9L%po6( z>{98O@JIIgKYT?q3sIjwQE0OL@a%z+%UmGgsbG5>s(7mUuZGQnI!^-!&0J4+fv6R8 zl6Yx;lK5E_bxV^<-+al1P5(PLW~mwj)@Jlqu>MDb>99_;CxfVeH{?E4ixsNNpK}|g z9-(?^^peb1$`@?*^j`bgmR2PFg*_U8v3-E8fr#V>T(2(Gss5na?v^Mnt)T0M@qiG!u?h4E zu@9c@TQN0_iPra7vjj6notJrWVnN+tt2W-5fS)QB^c&e8QZnx0{2110%U}tLqoprD z(lR_ol<>UqcUvhfGYUd~tEYWV&)j!JIljhB+zb#r_bFA=iG$pyZhV_Wq#i9_@D zGq2%{Q+Ao1s=oPE#;L|%H4*RB%V~aL%l_F@Kfz}fTU@SvWjIgtstN?qr}-7v)wnoH z?uEW4FmS7}2)wQymU*U>p$oBsYM5#l=SVi(AGP2W~CB;Mjo z%^xLphJR(hn$0#J!837oolmTdYN)UO-n>+N;e&OpLY3VHGM~HX?Ol1P=iKRpx1IB~ zvJ%-|8S4Z7jgzD0X~{osAJ#%91pQpG20?Uz@0FMfSZxU&#f1etpD_*esID>U6z`EB zI!6=7!x#72I~LxmSI69qo~nLR5IMFF5^+|}`7oGh^O8(%Hj@%AKt^YR_$E|)5Xq|S zGjS`naoxY;yQh<9ZZ1c+@6$%(fc~xNBQ@v10$;%=zix#WQaHi0(&%y zWvgcP@u6_2Z;i&d;EQJvTZm&&C+{4}`LOgSsi>?`)tSg?RI`9Y+^I^s|LuJ8X)a{a1J9L`*jM}_ALKkdG8I)$2+M*JjYeEW@1xlQ*O zwp5v8YfK~eLP(H(j8}cm!}dC#IRM{7z2`9u4Jw#whbCg{%XQ@D&hqfBYDqyV&Q zdOCf07*c2P8&otAwP(fP?y2OP6uG${#Z#JNj_VxIbtyLuz}eN5Rnlfe zaia>5Wh4pM$UVQF7ge!>0?>ygio9&icTz^A_nJp-w@yE_?1$~=1En|TlAP>647+Q- zs&)ep_X4ilU8$owV|M;AFux|Mp<8JN+YQ1^iK7hUidu$~hvC{IQ|_9@yd3-ZvzP|H z$KO+kk8GvtKS_AI^^mq(jzw({rk@ui`MaR4@9GllVNhi>a~wsZ^pyqRH`?TQJQF0 z+|1S-?6wJ+$tP$%uatRJhO`(5d?oB4AN~*bjL_w@C3db8_aAG4DE-{*L7ZRf<0_U! z>QwAwPD$YsE=7||@iM3%E(0cywx5;@wr60brwa-fz?!Y&Z0=mYI2=}k?9moHX)Ft) z^6yqlbrr~iqNd$B7-cJ+@U& zC%F{{L~mHHP>`eD=MGr+H20yb7F#);O?L z%w*~x2M{wZOM`Wo0nrQdxlq3 z+^oUGz%?bTi#%kC8{V@p1XKb5mR?WI`mBSCk+Mw;WZX#GhX|3#FcD{!n_$+U#Hvng26ICzZt6iJAT|Bn3R@Wi&W?a;h z@Q3m`EVPVriPZcIwu&%ae>akW^(Ds-$*mq-warsZ3M@}muTHfBZ6N2FDE||r6JS8g z1o$$|uK0W5v8jesX6D|@(Gg*D%a6&;H<)WmR_R{y_9Swxu7dUSH1D%dX5UniOCUhC zkdZY_YxZSsG~P_j&vf!Cn0a@!r)69s?MZc55a3Em9@gf|bBEdJ!FBL*=rtzx zwg)%m6ci|uET?yeMj}G-K`F!@b7E!*fL^XUd=aMBSFT?`U`7oSnaTt)ho9K!Um|=u Mns-4}>UJ^z2ga@rr2qf` diff --git a/docs/img/property_graph.png b/docs/img/property_graph.png index 6f3f89a010c5ee09c2c7554b405e707258cad8e1..72e13b91aaa8c220dfe23e9b1fd350e2211a8c47 100644 GIT binary patch literal 135699 zcmdSAWl&ph_cjVGuEmR&7WV=TEn1+sdvW&!cWsN6BBeN_xI00DJH=fCL5jP(pYVI0 zzrHhP-Z|gSJ(EeYv-d5_*R@u{RFtG~u_&;RkdSa?Wj?7PA)&+|Aw9doc!s!A&Ml{e zgoMmuBO#$8DZoy*+cF7_XsuP*0#3x zwxL&p#)HPGN&5PKU%vR;iG<>H?Nx~L@-O@yn$!zF9Wry$JR!rjczqR(XE~2Z5LG!_ zC!Xievyr1y4Gh(gh^>A+o?<=&;1-@=UC4Y%*oDS}~ug2z~sV{=9v8fY}|5E(EDBSJFa- z@lEVo6OoRH4}>^45Ax(MKR>+cQt&2!NxH?&eWWb;y^x%T8)J)?dk_2^sk2<$&pq)J!w2{ofDZ7WXwrOq^Jtc2Yn^O4=@I z;HLV~@eOm?-)D^#&#+5U9z8aalT(vZXH%Oi8+}E~4|h#Gyf%`p>_>zSH&6Ts7o{^* zn2E81#Sw2B#B9jQ$p8BMbcqi7U%wG!L2&=M>7RRF!2k0h#CKT8)6@T+qzmxx4^MAE ziU0b6*p%}BW@?r1p#PZa|J&4=|6_Cieb@iDQ~!^1|My-0zfS#@C2_(Q^0{PD@->V~ zi7Aa+V{hpcFT7r{oNYwCxU!OGLd-@ZIypJMRBfFkVHZcL>d%hravLV($iWA=8TYO`wrQLr=q>Bxctdv^jY8=FiK7 z{I-}sY_UgT;kF+lSulU8WwG}9&<Nu+jd@WH>=qG3TeKH+Os2^^`tVNo>O>Tvn~ z;{x0?P#}kYV3Cp(lbZid4^pdBc7~xIOcMOVom}&vl0nf7Phu70kUG^cN>N5N3RFnY z2_<%`fRQdfoB!1#Le9d9kPA~iK4VIg+A|^S7FiY58^Ri5OD}Y2S58h&TpTe55m9`# zhWRcW=xIoR5HBgM(#AEJ!OEZpjESR$)5)Q$B*|;a{148sgdl{#0l{;ygwonNeI*Fh zo)LIzzJyD|LQd0u;u!Sp2Vgekoh*+8?Gy#5YS#2LsRgaJb!cu1I7(7fHKAtC&8Zs6 zFNk0)gRB;-nGD#iM1Y3Ka&<6hi>&Dr6<$71{clnO!owT)4+dDZITr-3E*}s#LpY!i z#oVKIw+|8Pvn#A1IT|gv%Get-lLjUSiw&E`yX5ZuY8{-pCRc)1hC)aqE<*I2MKSJJ zR%}vUM2WmDQulmL?<6OFM92?c1Sg3luBH*mGw|gi>YcC9oUjCjm54P0Ij=tLjB>0r}J2 z57-Q9|Dp~ZG#DSw>}G*^L&r1pzK>%gbLclYe7&SfNLW>RkNRVS%0w8 z2xkn+-x1&E?pg2X#zb&e?u`4Tr@`Geh>n>a*n}K|nNs$d-QDyd_!}(biDBVv!0i6@ zLHb1)kP3qUUK=QoFrZ`QlGdw^SBZk4FA4d!=RpIHTl>XP<6sIbU4RD8Mcf7WM{e#+ zz{ZX;;<|KYK$3}C3HHP)(&gv{VqtGQF$6VUGbGk?2StUu>Kt5*vKvG+M0fo_u-DKJ zA#rgg;Mmp^N`YAkAebIA1;1d5eN-Fpw4ChhWD|ol){t1Wi9p01UE*6o8!EPg?;y`} z)$btH5YRW&#<4ycQ}|lbT~PS%y!h2SUc_ElJ1UQUf2X(Hd8X7;BD>kFVOHB8PU6u8 zXu4tUf?9V&=}9i2bWlk{kk?%q^iwG+7sJzHPW5x145~lq+CsmHE(*$$b*`dStVF>= zv}uE3q!zI0ghUF3+pK}@if=%?$3|&=Jn)n$qzyWW&Keo5ToO1xTT((rr@XYD^AqW0 znCd$l4*K7bZMXV?+0Q4OW_^V0MWx|$5ope}jekY*9mID;ARi1=T^(+HcdHsJqT!SS zdK&Wx1zsLHx5r+C@Dx4QX%gI7WX$x>40=Q|BY{8jlfGha>yz&gHzWiEEQz$6=;<5U zBgjW=3|5a#V7^Rn*1;_*DcRfGE7=no`4iO`v7THgdw1jR?QQ~|vN|8Lu$eC8nfMg$ z0(ta<^$)tGQWRTi= zxY=BV>ewg$UJ(1eh3owb5BeldLl6@N*bwK2J@sjAZ5QJ0(z5=h8@!)0u z@PVOrO6nWflJ8}07!{v8p9|Q50m3X|VHy4fhtAA`DDk({;KY1xKbQox`{l>`iRnB= z<+D_8wUmaTJfThhbi)VL+uR)B^I*`dteC)Kg*ETOaAl2{31t9HFFoD_?$m<4&9r46 zYn>$FM0Jb(FUKV~p@UzdW^DWpuV4IdgasF?62o84`1X-!eMK&!cVQw0Tyl$yhv@g> zm=nm)Y%z_cEHI8@=5W4=bI6iZLfgMoaA+K2_#+P+x`MS*%hoz_Efa6Ok8dMEbWQ09 zefSQFVNj}S^D_DYi_CG=iyBJWIIUMu_2Rplt)4A>XdxUw#d+X+k9suRcVhL`De*1W zLi!}ply~@&qD8-wgS^=nhuJP*_LW6!(!OZzamYJHOH_WMA)R5iZ6+riKa2x+X|UXl z#qXrJn3&gNKB$q$P}!zh)X=>gtf0Y9>wR`kqx7sIOGUmRvgZz}zbzLj;5HO43QdX` zS6DCMNn~;`ywR)y()Y#vyl9ReCW11^`wJ%?O!Z`z2=2lc_rCdaf{OI}Szw25R8VH~O2iw|ByUg&t<-l(VH|L{h1S&8DiRQl@ z7^$(ieweQOW{@~tw31rgwqX^54ZmxGOKPU)OtMF|%$mB)-00(lw{Yv#Iq8C3#t%S- zH#S31PSs(x@v3*a!frmb@6;Q62@hYtl@FaDQ8{LAQ6>lo0W}Ub6%~_l5vX1OzOD?I zV4egMwmj!cw8C1kzDsu80ulj#Cy}rGd3o0b9c_7YMP|UU$SRpT=Aq*yv0*UGnGN8T zycQJy01bZdm!g9zVgl9Kc0V}l!m)(A#TYDho5`?T$(sHcaOV`w1$AbdAkjOpOa1V% z4(rTr=VCEn6ZFAFlOoIWG%u5X3(d_bJ_DVU>*Q-DCbFnD)_8Lb80e}N#>EvTl4pV5 z+J0iv#siAGcn61W_TV^w9pP*B9`tleF>ajugiVFI*TB&QBZaO1Rr_M|N{UOumwM=q zl!!~4b^a0{dQWdC;UH(3spEVSIHAVu`(kqI3n9tR6|OhiepDG5pH-^;KbABp-uY$* zog#sQi@hxq*eG-Mclrr46uez}!`T8L965&=^~*1H7SZ||Q{L&k20FN-1%gcKpHxBp z8JuSTT$;eC{FRJ0Ja}hhoU~{n*Rqf?&yGrjP#NT2e+DE-is6{)x97;g2A?@cRRL}3 zf^z#td{^V2Cy1M98=K}$uixU(+s+Zk_j87e%>&llGBPx|UgrNQ8vo$8|B-|Fw;BUm zZRY!CWd5IHnna5^JnN?uSDLYde+|x0jeKHvLPV|xmbuj~!*ss}l3KaYw3fm@$X-Gx z^+Z)c6;}nZ$Bo14C(&?~<15Q|Iag&fKWdj-_tvy_qf%@8ZohuW#h5^*2>Mm&r+vx6 zDz@-u>P6HqD*sDp@Ch}Y>KCRwN@7<$cnUU=-%L5N!9BMD!WF>d+MsPHxV%{3nH979 zERCLf)0PXCE;6E`xvCb=dj!jCC=B4HueO%m9ltdt`emYmCS8hKK4B zYPGFeQ1GE4TohGltp_9c2mZFFM}3SIM`H3Of%ZjaRA#NLF^shAd6102vOacvyPlq% zp)Ha+T}Kos(@*+4wqsIW!4)&eaAAz4EoxQ8SnRtA(-;++XQRAiDCYcnAPN!&HOB=KHd_?3G1|}Gni+Digw9# zIwX2uKv;S;Z_b%feLgf%yPFgpCi>MKg2w2qSXwd+)(>XvZ4D_CIt#91C-k=xL-m9Y zK@8e}?nyX3j7-iNKUzQ5IddYX$xDP|b0O;0V8iUjONB`30+g5*6 zYI|h;dv2qDn9u|L^m&$rLs-wCfFC;h8=3yVsI(zclt#FsUE|xe_IOKDV8NXDa;7rj zpxeb7>DNviQL7W!72La}H&CdV&T9aRX(_+<+Pb*vq+yPyQQR3_+RQ8l*acpBr?dWBG>E;Kip*K!Pd0FezGZ;3ytdvr%05Lwj6%;L7Riv=MChYOV4QIZ zcyr4se^gk?)I{&S5vxfTVCfKJQBrvvl^bQUNdBZ-Nu=B#g_9or#;^py*0uE z1)DQF0PXrUlO2HaSJBi(HFM@gf?^{emvWn5%npB3t3OP7_?(%5M|||WXK7mx`M3gw zR11Vwh3KBVd%=SPlkJt>q(R%|dFqsCKY!;s1={M{Vud|hO5kOHdc8EQR zj{kBRX^V;{QX)dLR#+aEn=-HH@uzYw4p|BroUo;5?cCO32N}llob8h?PFQzuzy2l1 z(UNjvO~fV@>6MS>ng|qV&g=x3(CQZXQgXD~aC|ZV9fp(QD`2`$*G(j4Wx#b#)`FIu zJ=OKDAVas!)cQA3O_PqAS}FzBzl`goWA22w+rJ^}CfjI0E&#{3!GzP#i47CW_UOkZ z-|bih_8ZkQs78W7COrVp`ywwpj(4|i=Wjgf5w__#90aXkNXt2`;s{;}%{Cv8Bir`G z6;C^;|HfJG`KDKxRF%6(jZmENxFFtG$w3B!iKwCYH`X)tb88*pKj@&>$#qM? z&mMmzxCEC}8!s#RoC|gTmFIlum{{%t9KWxA5m!&w0a#|o>l=i1j891ov{6oH`G(Qg z^BM!>uo7HG^DlecBXRG2b&eYwL#;0<1wY;zx#~8Zg?g8 z1xA|U@rk)2lE3t0Q-822P`+Io38BvykkZ+E1@uN{X?h5U;?d%IO;xujwB7ux_@c3- zaQSrB#euZQl{Rev7O6roPPkxnuQ52AV<7vSk3m?^&bsOYO6WRG%S#rZ{Aq#u>nDbF zUA}*UB~FuhFD0ua>#E! zpi45ZByMZb31>3N{+kMn^Y1z46^6QC|4 zC5`%!MTh_c#q~Qqw%Hom$N82|hyKuJk{QPQk<*8_E=$@r|5+D3;K!N5(*h&7K&`P#|sDilQLFO3o~cD=je zjni`!4EojdkE??^wqpL?5GO!FVf`8_A;CP8W;(9jmp^id+CQnAwRskT31=?1-kxvd zQnjkFQETQ%A?u%^Jq*DNU5#}#13T8kHPSgq2-*P-mrpr!u71PdC)MpPhqaR=GWzrP zJurCJA(`A~e3fdZulRjAz)Ie+q&`>0tNi-wKoDJkG>YGt*MNguwcu6V&98rXU#LtI zCJJ^u+QGuqpYkE5LQp1Wy>axH+OzA(5-h_($WPFuH+GDA?gUn63r>_0_nN3G(OhQC z@*e-*v%uE+04wqYO}Sz@}jD9@9D4}po(qv0S^pwdqKv#HE+L zvV-e-Y{t+hWAH?ijC*{HLkN(c%QcTM)uEY7!|1+~7e*gP6Ktbauk^?a;cWhq8_tKi zpFu*+07X++4sCf=K>!%#NE~w&*;$)T`R|G7pQ@4m48pEnJ3JZ5BB3iCrYmS** z-68g)A@j1}N427OD@N9rzm_MlbZKI%4NUA@HKIT!p3*2TV{okNRRy9NQf;JBnvH=- z1cVC=XNRip-+vPaG|#4g(l?l+|Ej*U@xp4H@g*&nGUGQX7Fz?E)`6FHCf=qo((Png zKJ;#()~+u^SM z%a2IXz`NL3B(@9RvB~w?_7VZCE!|jpb56)zqo60mCiUyNfktZ_Z|L4G7;Eaj*8vZx zTVS;und-%`CKGWnPDwpev;e0D5DYDxQ&{uf9CS&UsjGu;H$%Y%IzPhgT_}ox3 z{C(jcBW}b~q$Z?=$#zz(%HqDrh>WmMu3tbMvUWNxOhoP$*o&`|O>~+Wfr0e#O>dqx zY(Y2B)2udr{p7b^oZ?HbHe$-OQFD zX(a>GLdYU5B(nS7yowOaMC?$%QL5iV{CTcMOMZW53BRIRK>P zlaUtRVgD%8wGL;PFu>xrwMj9eN%0v14!J|(xB)I__(?WvRwPts@F}PT`(MP-E#^N^aCSGWkQ||y6x0)w0>>0ZoYGYQ zWme^FDfvXL_>Jpl*q8sZFB2c1M@A!aXb@65#J)0Q&J_A6?UL!pn7K$hI31qgf$e0n zKJp9+VdlOYzK|{4-VrbTCGe-I_Bp-nK9BY3cr5*5kk=w>WIy$I&_5Y7f;5a!RoUd4 zLT5MY5thX!lBI|if%)7bXx1m@zP!Vc4|be`3K%lnf&&D@6T6%u7W0?#fQ4)J5)vO7 z?CcVdZsDb3hFs9av%o3xghm6Vjl7&QU`<|A6Ss0|d>ZSC=_?O~J1^_ymf9x-y>-b# zDt?4~=J*)!lx|JfVub1l|4|cA|2^au{UQxHVlHB_Uj&|~&ksfZ2 zkNlaI@D}|i>iG5tAc4@?t}W?u{3_keqf?yTm2~sj{Bcyr79ePjPt^TcZe_#lPDIzX zg$3uOrnmP#$c1k+I7f|8t$r}%WN^}Wi{mYae5pfKU58ICoG-bBH#sI5eU-Gdfqf*v z!Vdo0%K_b`_dBSdkQG~ykzST|o{j%ziBYUY<(u%wOgo0ZooyTZ;q&^r8^L7`Jthjr z4FF5eB?5q%*2nTc3NaLWg;P1PwSXPms@?#@@Q}5Qccu0$Q)SD8h5$Iz(3F$DPxf@-Qp3$O+4@4>*q|8MycoxqltUg}V zd~@M=6jG?{uMNJ6m~Ak=GPBezs??S+m9uSNEvJ$q7I@=X3wHBjsoYAkGIzk#PLxuK z?PWcq8hLr(1D_ofG#4$efd{xcipV1pk=W$r*(6BAkF)rJQx*1#fjxs<~t?Iv4LKHkr zVkhQ+^tPQTrAC!0VBEpvs=u>s|9#v4S)u(O)zSaomTpt#8isaASzk8{5iR=vN4>ZU z5T^!g+;)x>Ec-cImK8kJO+5|fH=t@--VYU*R3qcmfA6|AdVRVnsKRU{;CofkZv5At z<_gt5&1G|VIaw@Ig&=m!Iw)rI=6n}{^U+Kpz3T2k(2khad> zx^pzRlW5h<5i@#8rQrMa?b4(FGpyXtwD()m_2A8X(R-G(7sOcaSafS_Nf5|cH(=|? z$rWG#uZpPXs#(ClJh+FSSCY(J_USs16FJj1J#yMv06%x%Quu@K?m9Uy9`@%BTDdoy z&LEW9!$|e*HElkG!9TwEUp6ka-ySLl+)QLM$8q`neNKCIlGr-9AauGp5(?hjY`t6f zHYq)d7_ZHi9$Q7f##o(h3?a^DxsWgWw(*I0TPbW zCPm+CL$@GLh+`;W;43;ioQ{Qfv31K6&Uk1MV z*)waP#Qpk(d?*^*7nJ{OUHx(C@$&J0>G3w8zPwnwJOrFyq2bZ8i$*~~p=%Cwn=V#4 ziHna%xWO?cz!wysUAnD(2r)+`wV|NKBaekQfOzf|<@g05Wy`0>D?-Z9>z#Fv7M9uR z)Ld_IX{rnEOlBqajU{)_$Xsp`&Y-37x8jB6 z=UG=SylK$Ol3R3GbQv1PjzDH;$m;OTik?nGp zst=bw1;Xk)srtVs1?1k&ejFXya24>ubNEJ+=D*L&w+J*Q=m4!rW7Ok+z2~%&0KV_K zWtFokh{9c8?UH`D#VZLK7}LAl=I(>Z*AG z{UU_<$pAud5#uSK><>FRKXm>;6}3AwHCk)N5#ak`KAOs)CulzO&2CrgsUoKmck3xG zEF>td)J%fwc$NO~uX&6!yH}aJt1Gehr!ffqKW!X9iKhVsXTi{fKLiE_&g|4ar^9K( z=z9~8SW7m0Bv0kapiR^$V(P;oS1?OPnJsjAg~L1$A_5s9z8uP0mZr( zy)1-A^?y;41l0i4_FSv@TyDtjBL-9=nOJDD%%Wqi#vM>SAsV9sY`Quq2bzb0NOHL5 z6EJOje&Ghni87vus+{BK#|@`&0Km8c^IssxQIb!+7~etG*49s%x4O)4Uc$d$)6&ug zrpmctQck@T0R7HeELTWK&^$Vt8wz7H$5LF*8?v46xxm<>!{&*zB5(E+8??0@x01?0XvXbLxt`VS)8kNAo`-AY zt3Y7E=w8DVJu&$u*Uu{j<~cG^Y6f_>;=^rmT1fb~)6taOOc{d31m*NH_WzUPa%SMt zo#gh;;{8zV*yvmC-dkgmF?`+h*)*Z=oFvo#%R ziy%4)0$KsjIpuD@Y8w+5FW94ZKLHj0gAvqlG`zBET3w5Boj(ZalPGSwFYCHn08rij zsJXkO^(92;6(XlZr~_g=Web=P#oL_b4f5MetEACzr?F4Rc4-y}X}Y}Pv6;hrn;H3# zsOa1}X$bSHZ|zooxW&(Tg7*IF3MI%{OYfr&cg> z!i}-bJLzklBNUnA#jegf;r7N2_Sw&qqk)^Ygr%~H1Nss5Kaa}>SFoyLh7`l+!Gr$m zHGkG7vZ#pCK6e2$WGePCea_`5xV0)jclePcmHxm)ur8c-vs<$Wz&+LyESVTjuTjZy zX>ydaOlp4HT_|ScMYP!pg}LqC!{1niJ{uBETsA)8O64!VJ}m$A4@)}=EZtbOi`B?M zUx7(a9UP>uprz^Hj8EZ!snzWiDm(r`E6$E@F{HN%!3W`$uF|0+L+2?|2e_o9FB`op zH{UKzYtuy{&iSny`q1A4oMQrMD)X33#a8(j^kvR02G zX^jBX-qL}aoTFj-FDFGIuzpLjQcf%od(Oi`2dk|DVUGB$H=(An5xhmrq|MA__iR}OxR^R%fw zP_e$rOdh;iIXW8Z3Q(x6%4Ej7jK6U_!X$L2wRgR66r|=a%?EtALP4mK(1qk9vy4^m zq?DN+{6*N`sJ&n4st<0r%%k;gPRtp1@;T%x5Bp6-1>6Yc4m?FzPi3YqsAms!7R;vj zZm$bE-rxUjfE1%0*W2ePLekySoB`dY*e;1ykk_)FtIyjYJ#k3U4fUX4VrF}G$%WVQ z^k(Tu=xL|hpyJQNqu=2)RuTxKOZbQ2zGccSyjFXYywUgOthFHewoTdPh&>P#IDQ+$ z`~7IpcoonCaYJ5T+H8s2SW4$3-(C8N2(h1ZjSl)p7|M`X?K51&n$*1t#ZlZO7B!2+ zr6ySW9YwM?*qJMNa2#DDsMWyL7h}pbEa{v$6n(({F2O0$O;d+o1mkT4NV`nGu(Wh& zXvm<}-oV@YMgm{h_3{34$z#b65tgK@sjJiaosE$hdYTL*aFCFYY>uR-CndENNxiH9 zCGY<=HeAPEt&KxUkMFxo;n8RQ*QAwq z=*NxkAKkImFcw&b0v;cv3E;k12ZpIiq3ei7wI}Dt5du*x;^h!^fx?CR=2(O~70YLn zqPeq~)eNHE!L_|zYsM@A!<$~@eee2mM%4sEXVh7#@V9Av3XO9Zboorcoy%VuCaXq7 z1br^{eg@pFVm@pk;;OdV*86=1`Gls^VZqeY)X)1ExX}H8Nq6Jm{l8d@Ya&!f)$72z z-Qc<_hraa5}Tm?QZi+Yt^W&6*!c$^uN`aA(6>0?n5ic#zQ= z6*Jwj3)yDmeuEUkrx=C3l!D~zw$v!eGukC=_>Ra8}TH57{hD{bevkCiu>McT~$fd9Qe`Z0g zseSM)frYz39=BLrFw*jzSAF~kW_2HaN->sKQ%6$b_V!1gg-1U3Q8qQ`+g(WcX`~6O zTl3SiQQCw7l3{3_v2t>9{>R})^In^&i@RzsmU+{!2dix+G7RoQMemk9T5lH|{C4$> zZpITu&NA_@OG5AAt@rQ&uGV=tJe=g$mx}#zVj&GqGx6YV&}$fA;)A&16|Vcnr8d)+ zmwj!yw&&9HqKc6F{T#;(4AdXu5UEeus__=sA`TM5gFy$!7KKb2`D}fiGEP*JT@BqX z^i#HMUVa;y(@ve?QgqdZq0SE1jrrf?xg_naGn#j-E=_=#QnI6Epx0DzHLtB?} zayovz8+p89=RauJxej$j<65vN`iD@Px^ zGw{j!V*z6+9;|fx^Sewg$9l87M7L|i1a9ET%AX;pg`=Kfq}ti>SJv9 z%|$M*29k`BocK-W(o^_$NBz2LIU4||rnP(J`PNeZ&mUUb{IT8p%w=0fYCTlS@h}*x zR^*vyslaonrjU0h&!+$pkjG{$W*_>k*LanO1;+TK=4cwdAlkxpfCW&4^;nfvEGD4; zESED;AHa!AORyF&`swgktGILpQJFMRJp(7GQ6{om&2P|LGnqWLIW-BfF2O_hwUNiG=45zYb+xuR1(uSZw=074J`8SmmCxm`buU8109^B3l z)>7uZ&5nFm0Nz_E-B^#arDtJFPFAq6H)u77&Jd89b`xFIC61|SmNNZ_D@yR0HMf*X zRy}~N|6JMMFK)I(c|TNU$P>tKt|}BYSun6{W*8asuF{KBV(qsuKy$sf9Oc#L89IsX zZJ9remq#-slGIo4KVd_W34!x<$3_q9MnVp?T)xZL1BqPaB_%=ck@N9M-er%~`QLjp zOE9px2V4&r2?z*OsAUnu`)SBw>5NuTEJh8}Z0K7G2L*Sr*Y)POwR+ko!u;g0$8Qtu z&f{D0s_F-Sm&wp}HTZ|AU2uOwK&_?G8KvG_g_(k28mn3h@wqfQjg5vR3BS^;H|kA2 zpC}K=Uw`_rm?rx9e$cqob}cKc?9_$jTT^Q5P$a%f{2MHS< z)%zFb^~l8NoC+E0O3n}PPJB_{;}uDE21LTZ1N|^WfiEtvIHI5|FZ)zF!R~ATRc#I` zrtz|ocO<9zY0391{LuCRC#c$FNw09I@qPC1-e?eMt%$_g=f&!N`{T-1>Lg~p@7bP0 zk}{WHp6^hD1E_O#uo=S^3x z0up2#IE)6fosX$|lTN=53SDY&#*IR63v_Sy+q3W9)2Bnb&2qxmr4Y#S6b%JjcTb_( zZIw?hlG5nKM!-VkgdS*cCX%T3Gr-PQ*Og|E3H;Le8M6g{Ev4T4ru*N5q0~pWVdMep z&WXhlI?>1ZgPVR*WicSU%Ic?i{c+oKd<|{w(5887JG;OWvC8=`Cny8BUC_cGAb=az z<5!t6K0QwvD;5Rie+pQm&hQEC(}plu>HBJMEu;fxX1mSauhXNqT>4X)o%pLcsRo}b zK-C2vp*Q`*yIadPw*vI%;$|*nIt5K#QnOh%H-)Q<+mKe3EggT;p}^kx=g43xi!iQ2 zpDK*9Fxg?j7SSCsdsh$gHmj9gOHy4E>uQF+t7&ozjwyq33OaZ!qG|S*qC}^^6%<=c z4Ldu7ZWr4fJ0+7HOUqKb$0bTfty+*`}+Rdi*^k}?eJ^{l~pbS=|Tg||C z+J7D$)D6~2718&LqnB&|K2=IW8Y3``)Zc$UI4OMr(Tt8Kg8P=fw{xeJ`yPSSc9-h{ zQi-=&1hA&L2A**h!@XP`>yH&=RM3LSEO5EZidolZG>OVkqLnxDYbH#B5i*QutoywRQ14Q;7x#fjr0 zg^2*>YhhX~K?wBVq*0LKI)lU1yNC)+x)adT=YcwXbnyQAvh)(o3tri*`J5lmd$0KS zP1`3VpxH^4Zyb%P*KUebv`J}*(jw{u>+GF11{9aartB4E%V8xBzYx=}$CG%a6ak0x zeV0d05vBp@(85plRZg%w{GU|6hFdqUiTAYoZ{%oG1Yj4_dJbXzxLYoPA0XVkfSB2W z!ous>5r0Njeq*{iN0&ExgzCtAP65fPE@Gk&;$`S$FaTByEc{H{L!T*>lza&** zHE3>iv|++%e%jUv4aJ;-DW|*~FVfgb5e4VQz z6Cb~xv{q#^r@yiJ5vero4^8dEzusYc+}jABb%aFX_8HFKn(}YM1sN)E`y&yUZ+Jg$ z0j6YKW@ve{&*cv^+i&4>6Zy(!^#@i4pLPr1Xn(^5Gz3YWLhop~;+ zJFgbpv|h(ETG#_!9n<&#uhPK31ZOY=8`*y;f4sQ3kR5p` zL^dLH8c{I+B@;#FJpye*f>YfM1FEo}phq=Qq@$WCea%?c-+26@GJ+Y246oB+-Iw;t z6`;rta&KY@d*I0`^IaQYOur?!=$oYdx0F;n4PVu{-Prpr6e5LU@COfQlaDM*KH~w^ zaPRbT7W~&Az{_7(ke_}LOP}$wUuPHn4yFO9?mMpc$E&-qo9Y1Jjn+s4^ZhjDo#?m! zA`t-Qo?LiF`g8lGkl~eQU zx0+HF(Fgq>gee+0@yA4Y!bXi2u0=mq>^BX)d6Z!4@19SiTWa;H4;tRD3PS5qE{z!q zm<@!gTW#M}!(|^9W@Cq2tpqww`|iR69&Rsky7{5)-@AMcn-fPQ9a^p}HP&BftvuY! zv<`Fe8+#oT>f0kM;;%)(^V$@5Q%f?0tikCyYWJ&ZOJyzRrm}(@P4r^5fqF(7sg2oc z&Djd$KU`D&se%Jw?w!4~r+t^j^@Fsck^bkTYSGAG~GJY}^;+f}FnWKOR<-;LKX42%vA(YyR6y1Xhk?~iW- zPAhezAD{n0f$uqAi=GX`xMeMYpRs3sLX6>`b{VhlxSU-t)unA`-d;1!f{ia$k^?Mf z=2=M6;gY~Dkw-rtn(t;nc(V_G>-_h*u>l0i9DoRHuhzQ}NCKb5lM`q81{yjz08r6~ z!AR|3d9KlGk@FT+_tf;;v+!=pz1q4gwjk+cS}5jBg*(w$0$M;fZHk_k{BO2;=G0GuoXrbYJSnzOd_};<)l(Jy`lgroJo89to9x9V`^sxxgv@rIq zJnc(nzb`@6xs@iY_ispktj6WWjQWR2kvXDgxkwwY@`5Ky#Dj4{#MT;eCpYHC^%N3k z1e;PDCYI}ZHv%yBux{*3@(oo3aral_Wj)v5JN zz6>(KH7ZDwGqOAE{W027vYq3>30EWR)CdCJ2c0H7sy2pqLpKY{Ltln8fo9m8!>d8M)CBmR)J$U~0$aZ=?pX%u-v>ccG zE*Bcyu)iWmzzM;n|YP(gaZ>QD~m;maD*nd6P7a7YZbD2Q)Fi)OtC> z=X#%V@Wc!ChNC@Ev77~UxftxaK4vr#i1%~|U^{3h_3MoqdLJfF4 zPb$JEqnUTahXxU1@dcy?0Qa<r~OO8KstBTSn_evoyN0zPO!DyIe398mO^cns8b0~#0+5g5)_OocXNqS(pb^<{ij47WXPC%9+9Wae+YQE-ZU#)3zoRrk`~=LcxBW~ z0U6S*%n0~N)|ifaNQ;ovvryFMK4d>*@CqQj005ptEGm$XQ{-A0{9=@2pv?bdAk&wu z{Q2s%b?a_-29kRBm+%8JtG$YZu+BXLYjh6?Ov>RiWFYl#DMYXHzV9?O@U-{KAmkQMN97a(LX`6EoXv((-g#mm{s6eu z1-&%bDEWjseEd{U$RdCj1P{c-Uwi0xQsVUW033AHfW&y$>Xqfm$mh^h{shHXdDXq( zN?x;sbUzEiHykdW{TXWVqo1u&KgWwGfMqjFqaL`P-(c|5w=wbJ9uDn_G z5sBN>#~bFW{{@LiBL=OvDc)MBfhiG7^*D)df(JTVmOz*4IuA(nkLnxyJ@p1WILx z0R$7T(ju12I0;hbX&MDA_wyS&m0&UoV*WG~)UQX+Ji_j@_v&Q7M@2e_LaltQKi8qp ze40r^9)rG6Zw)Zj5DMPDz&(wela}mx?|6rD-b%?dpD%yX~fv{Zg_ zt*fKq){h*n!sRIN4f6!AZwZi0VARij%MzAdj5X#ALA%ap^(3eis0V6*o7Hbd_M_Q0 zzm~!(*pNNlklw7~Fq*hL=bR%|0tWtg!(P&rdhk?@j@%A25%f4%%bWeE0|bEc`Jf~d z7^)}I2s&MI{W@MfU1h#D;JlA%oweh~pKAS}Q9EmR&VS=!dOE9;H2k#{f69So^eXmo zoT<=&%EwOSa^=g6FP=urd(S-%R^A=}9A7Ah{qfH1G5hdN9%Q$0-&5BP=)L~^;UJOX z$}$46`t|j-e*N(=0$gssT$DHw1BOf1tmis)zItbAbErRUY2fZ%1nklK2QM|AuAOb( znR)gHhW3Wg%5nIf1m{_`Ht{;WkGJ6OwBMOU7v_YAFaYR4&P_A_&bJirJS3i1xHhH6 zIKD`CiDpoBz5oVs%6rcnUWIzPg(6Q-GU|%vYjeVDb-NxPw3`i1&#qW1G5d3taZCtN zq=Q;mSUA=Gd4fFOJ9gH&@9Q3XZ+W~wR3<=o8izr_FM4TVNBB+USxeW9WIaouYsOS1 zqhnebnE>OC=nc9bCk_kup>^$0G9%w#KK(71&4+*5)h{}*xg4d>I%ywXl;L&Y|5K^v z5Gd_K;T_VoqtLB!E3J{5{pu^8NH4rOgaCi>D&rS(Mw>&aFC-PSp7Xl%*MwHE6)G=% zq83+_kT#P~Pi&9;-5A#vA$qK^dNsQI@yi8Q{Zvpu zU?l@1q{vY%#)qmmR$1gjFl0aA_Qzvw>zW9nvh(WfMv!RSot6?yfs=ZX@gPTN4WwZ@y`5-eiHjk%5fPsCG>0TfkVJ8d>*8#p+U@_%N;O{>$tr=k)O{B z9opqeo6#Obn5{@!_3^dSnm^f`5FjG`&S>ep)#(dJ#eOhp*-#vPeye9l z!gYVHX2kEnMZezZVo#@J4ACx+=qGpqe+}%A#1L;2SJXUcX&iF%%N*nwIe#3dF7+G; zCV(5a>pT#GoU~p9`Vby$`QNS5_Vi?R`y1;`F=D~p?;S*=!4c>CN_3gxtAOo%n`lgA z8FyY2+Soavk*hnuzI)Eba?ifjn(xN&;mq4r(whlxig6V;{mF5x3e%-O91)QFn(5u%26rk120c9~ZN{ z998Fv7JnE31Ai|eBJH(gL}f#CVQGmr;3r|7_HuvZ{Jeh2!IpL_tYGL|{lM zrE_QykxuFE5@zU50TBUd>6Vf%9lA@pVPNR)?wbF_eIL*L9xosG26N56_TFplwSMQh zXJN-T%OF49>Q06GCC3&Ce#3l!$ylJf4fJ#p+hG`#-cd}vFuMhcgh&-PH{R+;wBM7+ zNJv^eS^gzBDp)wXoNg1@EyaDFs@fKt4Ry7dsJTe8v&L#>1p?fTr^(P0_gtyU;LVhl=Q=2!Ni)ZbbwCgvNNwjdu|MiK2 zPo@>K`-mlvdMxG-Rf~YXr`+md`I+`)_~kD8SLG|Uk`-!oz~mBG9o8hh@r34UMjCYn zZOR(Ip6LMah6#w&ME678OrZxB+MBbfvRPSP4!JJ##}z+OR;(*a-+5Lz{*E$5WJikh z&JhOVQAVsR6L)3#Y{?nEmbwmN^QMk?g+>KO@5V|)8yau?x&p0ztaIgxjt0jSchf+sR)TCCzpkM^m`8=J6APRe#1X-SqBEEw>UVqT zV7irO8#+sLx{93FK_L`t3FO=vG>nS6dFT5It_UVvi4r82=#mSb8Bug?-qrk=9PsdX zaaYX->=!yBJYWLW)&ifxy1KgabJxQYem*`R`8GDuEuPgJCTn5&l9-stC(#@J5PFz+ zBgpsz{^8tudz zo3pZ7A>ei^@kBU!Rldnn#Q7egI}TLVe1A>&mKUY`pk@WPOJVY-gjPLZ*zqs!B7uuQ z(UE3ZEv?@V$a0Q;`$Atac)%ivl?s>^PJDW?A6XyLI|dBgq6sl6sfH%s&yPRP8tJ{^ zV|?6A%T95y+7la$OFl~R%j(VaxXLA8O#&^mKap*H407WnNNGHwL8qdbjLf~|Pbac^Z!J+zOg505^cZb2EwADKp01_FX z`(*NV(Ka1``0gnSXyzVZ*l2<+gMbi-EhNyF*fwO(5P3bdey)98y1#&mfHcggN74aP21gtW=e-Q8r+tHWHn|B$omV+Y~`lMU(jDH)}~;{^4ECW z-|2>kCSIumu+#6q@Mhn&9pz&Pyu39WDv%aY6hKHw_@%;YU@}ipN#-!OgIm3d=e>`e zvgZ2jVs373CofD)30Ge7Q3mGS-Q7FeMF)yH&+VO^ND5)u#l6MWKn!K$7pe1wg^Va* z_hlm^T6SF<8zyJp05qAFPnnr;ozjxLysussVWFXIb!8TR4jcp3V_NagO@o(;DQ_1s zdhYMftDC8DJ45ZGXk}}GU%=Yp#*Og??3twR&^(tj-%~cdAO0mvaI8)6NlZyDZqrK@ z^m=>`?6$90*VFs{(KAR2j}1|Q-=W+udvFqi`K#@IjgPT4W21RPvTG~4CLWr!+lFNg z$C%Y}_Tc9WaLMc0znr$p6Z<(H}$^1_A9@f4_unda z2dV+~_Srt^-p4)VL;)5G8W!pEH&64{kD2G~8UC=`N^m&`et;Bf{`ltp235y?RNZLv z(BDAg9UpOtTbc$N<9L7*LJI4e-+$7`nKg>*kgMM^tkxI&e3(9|5>Kbr(Y&Kv!QA;n z4~Z27vf3NIylhgu@^5+ppq#*wEMGBgnURt-oSw8-wu(NjPvx=oCw7tWl=<`1F(sv- zYJ6TDf^yWqIE9Pv@LLGJyxIMwY!gMa$nlPIm~_iHFczhNNrwg~gLFCjAV33=_Uw3>0aMd50q?DvHE1ynLjo`-Ji{kfM6K zi%Yz#4hkJf2|kf>1lo!p7T^4Qfv8zyeQ0W#lK;QyV*t4s%eenwO#u)GNPOYW+$#Ra zJ=?yp7QOg~{re;Y}57S{B@|2^@kL-fe3d zXK{HKW#tv~@y09}SP3Rqgp0NbP2S8uNM#;)$lq4oFrY%1k~4EOERH9k5g;?3RKK*uYZ8v{y~z7Bq3-@w7_!$$E*($GM~8e z9MrSM+y0(f0}neZGGTpo;Cms6Up@z<$P0J-Zb(4yFtYp=r%b;d90QNft*%MM1xDoqW1?x!f5x zYtUN%?h$5?N*=&UQAT%wmFRLUh_;g%a`8$V8--)Srv4*BRY?&oZm$bmTYi-EQ1Jv0 z0JZs*3$aBRMV)cX_cs;QKxA%((A&NGG0}n#%{(uu3&KmG;QBKp~e(6;Tm=@fy3Wa zu%H2fiNaJKkeao=TcYa-e7q34$-Z1*djJ%~Cb(UsBrN(%fC@wDLgtLhbcm9e5HreR zC2=Je7uT3-?1wKu@*>QF5kOF>70wXxNxpr$v(a?a1w5;ndIwz64u^;+wFdEb0}bAtrpmqUc^w+9;Mti>PgQbSxp z8_+CVs5>mFdtgO}Ha3Uwp!kF1a$#8+eLHQT`S7Za|W~(7f`C4Qg$!-9*04(h?uG9&Q7<+ zva>RGM^92^@_x}&!S=)0n*P>CJJ-_AkQ%ehO;=f-7(yPVM~Q}abynW5^`Q(UiZ2gX z*GLRQu>X!2Ky#X8@;Em*`F^*Zs|x}atgieo?X;Mv9|bl<5mpX5F5Axg6E@{b0|9^) z!XPE*D+>VCXtLk`Wydv_mGF_s1wN@|`3AZf!3`ToCWFNCYt?&_2=7@eFj~1;nPkfshkroS0aKQTaz@Xn zAJdy&x=l@4ZG)4ucd&_fFc%N=%_kL~07eMqd+4K+DaD-2PWXhd#+&_mG8%u;oi`pu zR%6qw)!Yh?3k}PEz}ljJvKO&1F=2Vm8+j8pO(?5s+#dOt`%3;;?Vaw2UD6i39-fa) zOe8j^kpQIbV$I%Mv453cohkuC(-3Ey+3loYs4u9_a8{BZY&)o}EBXWwX}3-imC!H7 z+qFs2JcV$PhU=Xb)zt9m^vZ?Bufa}X>`sV|j*eiFQ|ihp%XAO)1;m>lN@+qKF-*MN zqzb>y`i#xGWaxt$NBO~Ebei0rd87VE^8|p$MHX27^+TuDs$Weon2&KvUYjkT|9d^q zHEP0zIlbJIH`S4(6Cw9E1*wVfS;;6VMaCbG_GmO4MfoHyPsX|GrNQAH)7`mu>_su4(=hC?R23Mgy@oPCIq@p_%qWQ(a}*)|Ve z>aVU0H~X1Q>J$6;J2dlQjpiJqGRaCQ_fYYL2+`$6>n zD6&Vjy=)r5rh=Nw7S|FC*L!ioIs-1yOmy{rF@<3}%dmC3pa37HD_opj6RB61JuxOG zkzG+$5v1v0cfdTNW;<=5p*CT?yFCUHJp1|7F}6T{?s-)~TIjS{KuMj#^3ya=)krTk z3G5Zox-e{NCkT4w^EgpK5-^VBS-uDd%NOo9rY9} zMm!Fe3xYxyJ(N?EbOdDk7z^5`KgJgsa~c))MR$X}N=glIERCc%ykKjJ#iJ*AmqsBj zX7UUUg-0aM$Pj#Gw2SK^It?@e2ravL<8B!8Dg*g(Ir4GP`=d?|I)6vB$)0Z!CV~1r zG`BuaoYwor{D2=P>xACCpUK^?3;EnbfSKzSq*Ha?hxRG(4w3TB((3qQ#B zy3X+J^iGb}UxVVqx_ZgzmmBo5SL>jA=yi$d`qp1V)Y}?|b$(|F9In+O6l~)AiSqe; zTCaazt3NZ~;hlQJhu1@pFa6xuHQMsa`Q+~RCj=o%hUHfC*ga2BPUD{Rj0y8f8D0LUN;<@{dsV{eUt|@Oy5mwd5V0E8K|GH2sKqh85VGoh9!Q;XtLfgW zeQbvQo?>p9J5{Cm6|y@8<5V0n|!d~424m|&sc`TouN zui10DCh>;j<731dk0o8Q&Z*1G%Wjc#j^o{1pi&d@6-Hq@V5FY-3U|86M^MvNO+#aO z0B{w_cKrR4o{s)_+f#-S3Q7kyw9Jw~<$+tWSfOy{`N}}%Etr*c9McAtWP*X@8|DkomS_Z151)8uCKZeDstU@eWVqlByQlZO&!tkIfYQjly!NZ|?fl5hk^>Lo89TJQq9n z=*;xO_^7Jix%C{$r<;Xfjr)z_1^+v$Pfhz~`Fg+K+5m1OB!h-FU|p{pyC^5GFrrGb zZGQ}k#oqAw`T3qbwqc9PKSLeN&3VbKP-vCDa%Rls_dU{TCR8v9 zau|tv8?{2x0629V0CS5`H56#%AiVHKd9N3auLMs&YU+=_yJ+WDuk?|{Iu7@?bwC=J z5!|##wZnMK(;OpkUaYSZ-|fq#vS3iK&kg<>mXo4_chO4YXO~J@_?(14f|w=@q7ldI ze9iL0iCtU{J*OHg*DDBRs(tB$?>VlzR;MxxjGnF-P)|U9~ZhD5!d{;yk8fG#z zihD&QBh%fE9VR$_59_QWzLL+t;s;z+ zR!^}K=U!o$4(>V!Ri zj$97XRSpEpc%1dOT!2w2O4WxBc$UyJ4cyJEgN8eCO?2%e1Q=zUK4VAK6c}~Iqt{QZ zrG!nV=XTQ3TYN-KGT^g=!=x+6@anQ$qr^Tl{kdlug)Y3m;Jez^z!Zk@a54~)H9EgJ zrR~8HA1I9#ndoASM~Zrul_q03Kc%>VWAf(Q-^OmY{#a!s$a0+F@7A5veE}DFBYOkbb7tgY>~Q;< zqt6nSp^-r|m&KPXrqB8!g4A^N%9b$M?};$j)~H9WB;YfbTRco9d}9C9a>k4IRmRc` zedeaSEn@4H<51IK#8!~oAi5bnnER*+{dxC-&k;1{1aIT8phRI-doIdvJ8^3pe%Y}K zyQG47c+K|Lal^WGOeG)q1F@UT1YPr#+<>bTZ@kz*e#kCLa~z$^ANlT&ZM!TuJ~5wZ z`KwEE^z$^MNbD2!#+MgRtx1|mpOM_6ep%{A9ntY7i8HiQfrQ>6g({oi`S@-_Y zNz>3R1+qzkhzV)+w=Rq*aP}ofb+Y=?4v^a2VyD|sqw1XI{OXL~GQBB-9S5~J@AM)4 zLD6uHLI>h_AWiYf9qRjQv=^acODTH3ise~tm$A{vv_FAtFaL-hNus50@4aNM^TSW;w7M=O0D(?efJfbL>f#Vq5FC?;Zb zi(rNFsw;vCZFQ9fCP7zM!9_UA{TJw!ww}X!!UIkuo|xqSF#q|=KPOnG3{KxCfli5U zrb$qiad3WE;a$}Qhf|m`>hK27s=AIZ8d`%B|wh}a-4+FOeHrdY269$ zvM`#i0}gFPleRyqU8c>+Thn-(D@uSEO$Bp8=4a)9Zs?MIDE|u)P2t3?dW6}8wN0=MEW^UO3?m_4^GSF&772od(M_! z>Sn)RlJTVDP=+Oc_c_-ocD%CnQPk0567b2h&2J}C@0l7I<_58!Mv5zoy;D-=7 z=~lI7TAHyZvuLbQ6ic6B=|ydjTLsCsT&I=fetp&w!r2CT&t%%pIj?z*1|+FBRDVhY z6-iE7xyZLpi!O=d>Q{2FqAMCxob$#I$i*e?U6PNo! zp2FlclCE2ltD0o#RuioE+Q;@Abc7C+H2C>J&vH5o(=P0K-lC#({@#^vfjQ63W~P!; z)!7JC&+6J8|0k^01u@G?BK{rv9itC`Q|NQyT@gj6K`OES(0k7SF~~^^c$g+qsEvmd zGFQhVfTcP*1O|g$SJfCA?v1Z+=2un-KN9N!9XKl$NGk+X$Zg7n1ceRQzr-xmX?-DZ zbP>J^o;W>h$E%+kE|~b3D9)KlO6pm93QzaMYWP6G#>e?3s4Hi+`4Ya#W!*b=u!Tib z^Ex_}EDXM0UtOXd9!QIicpnHA5t3o_r}JW6`A9?U@q#C-$P8`lPh3M5IJ7+cC|%{9 zMr_}k9gPp1EN4y)ZUTR}@H<##2rM>Ie$>$#gQK1BxJcl6Q^hBfBw_2XU}vYN?>Jk;6w$M*u?8zVL%(%3{Kxqu+w zTMuKeu}ss%{_hfyFx=YSu`Lis7UOH~*;t+BYIPx7ZOKX6pO!W5dX3)M zevIHKEXO&Pbb}6PsZJMsGwbiVGHAT2`o2AwUJ#;7kL@OVXSy zjQ#$Vf$_xo;#(qSZhB#4*?yKqSNxOXI>&@$`HaL&~SoMVnk!-baI`(T`h;7%nti&d6`|Y+CM;cIy zhW10aKP^Q1q$Ht56SHcm{*Lg;#yrjd{EJijE9u#MwD;9VjNh-j8s9{e`eZCauu@qrrBS`u4kxq1)7u$c0?q7UY3H5qhyDy;DISuv zb$d>&+1-*}k*mI!{&pkUujaZXXo&c`P`Gk8=YI^leKc~rk&7xn{m0d!1)uwMPlFTudXIa%8ouZs{IG|j@N zyIT056ejEn!zxuYv#6nl`S;<+4aYgXmOd07vuba{u3*tJeU)RNy&?)D*}=@Kkc0Gj zmRRvR%Gr$#g7_0c2;lUB9{r}SBvs2~e)wE(+1?ruiN6S!FuQIdPGH2IQs{G5>Sxla zf5%#o3j)h3`3C;MSbvtFm%`(fY-JiPO00S}x@C|sob;!1+zjus;jNGs=HDc~O-ari zIJ)OnZbD5J@}ZAMT9}<9L0LD0g6(Qu5Hth(RFd4~pwLeI5*(2*Oi8sW619-@^tmU8 z=-69h@#|)JX>BwR_^=5aLzr{?g6(QSU-bIRHoU7X7HG)j?2#>8t15%QkVQ?5?5tTT zs4(DNadS3azFo~&YWy%Nei!BBrGL-w2JBs4R|PyZ0d+zx0tJ}#t%bvXK*CWqrzIg+ z;T2WolkTkH1In$BNFb(PNdx4%2x6!7iHnjHX|3bc7S`#0Wc2V5z+p9Nw)%El#?5P~ z2U4oyVgU(DY5P9}$5S+h2!-E`#ZKAi2urEO|I!GJ$1+r>{ic!Kqd>$lar$JZK(SEc z0|rm} zDRk`N=o9-7&5M0qYJQSJkywN81Q;>k6sRio+Bb;XopcX}rA%NfW&xh5{NEz8Z`Pp7 z2cF0vxf0-s+?#DS@l-tQE3>%LkMSa8IOiedTFhptv@wW>Ra)yb$FzHUf6<(AieHse zf}!(bepore2{TfleE3`XZTq_m9t2KK3B#i@;tufr&bV=vYmyjyde^LZ#ZrxQ2P{el zj|&ZIECx{@?vWN1D@(hjt(G(mh7GD^aZPf&BsUlvRKo_fvL1@xeIKfJQhqr%bVc{R zQ)QZ#MGTLOms;HxHnpmKoEou$l>D-RLMnodEwXQ0N&E6R&^QHw!V(9}|N8p6=`@$k zfv3C~(Cx(Pl0OzZ!%8*y?AJG8$JS&avw zpe>$0@?L#JfkJDPg|B|)=b!!p;d4exxT|dS#p^4eivD&=C{LgZ~Sl z^!IS=tJr{L7=L0&&7%bP(m+Z?WmftHPa9F|ZS(JrDfgyL6DhORZZF_h3OKrl?MmkQW@cX8;PUyWY)9u{cV4F=7e#f&sVY=j#=B&d< zgrr?gH3V-ej88|lPJkN`68+o!~BTk$}zAV_ORPJkRk%0 zeZMGKMAwAud$oPN2@t9Qy?%thF>0&V+(rO{hUbXoAvjNVf@b0oL|(s$<#o~FK-Wlq zG>{$wNC^MbeCC6|dN`Y4Y5Ns+UEoFCobAE(_Rn%CBB`wozX??|O8Lu+(@2`1e=)-| z=r7_p+ArZZKT>{#)}g$}SiKFNc?u){B!pF!vM4Lj6cG_h(He?KkJ$DF=(V;JP!A;P zJsM_FLdC@(oo&PsR&GVgpKkBpZqj)#zVNf*E_@w5ymT+X8)pG`c(1u`@ldtZ6u773K*6A$*m~=KmgC3}+%O->Ii{U2X(N`f*L>d^+q9RLZO)e zUi9Glpog>j;Q9a*wFr{lGI>J+KQVH{(&jy!6CXWTf>8aaegtM-^m&Oi;8mfD%iDn6 zC<)?akVea?ew?T0LC*VK@FH`TZl&9k=I0Fn4GR--#M23Kl@IS;`S|$UMbk*t7)sYx zc$}<1H3ePrF~HYz8{6eTp?N_hM_$=>4c|LHkqOT%&(ci9XXP}C%OFD4<|XT`AcDq+ zamcPC{A9Mi8fSKZX`N%w7`Rxq*lgMl6-OuQ@x$NbCedeL`xa9)f{ ztog7d>sza~Dtr$JTAHYz@3nI)rXLb$)Vb`>)=HZoa^(_G5G}E0_t8rMH0`uGyPHp6 zP>}cJi>}5ZDFAV`B;3BEG30MVWmdvvgU^xQM&)WWuUoQS{4z5Vv^fu(G0n$E-;& zTLHz;e+F-zByb)6gZlKrizmW65ICWRKqJeuyvb_0-PkFA>5YU)cUiXa^@*Aw2op4l z*7OE_*SRt(w3yQ!03sI9cQzdcu{B(87a2413RSRV2HPfn&AniGB}spi1=`A~*mKV= zyQKt~mb27hg}eOE5b3wQDd?$QAaJMFx71F5i~yIN$)MSx99meT?v8u)IXYjmm1w=QuBD1Yu5jeWuM z8jR3}lp3U9slXkabN(!t*RId%%bjoX`NqlKDzluG!R}LDu=cv3!oRhd1pCjfjvdF&|u1zK(`ZG=aTCI?*0kX7Hhn$xOzKRpi<9?`cdW&+4#V2S`gL6nGZ zj7&@q&{7DaG3E7e@PmSSf1z1a&BYBD5kW3^{+A|_+3!z$8(4)ps!|A?vwQgb1%<0# zmN1rn%;!vkv?hK@?;=^i7qvA=^Of6sBmGxym zITd%?T>dgAQ$6y1R%nW|KUaGW>}r0v{`V3(8em$jKn5i^sE)SXFZdd~|JS}q$|>?N z%d}!I@xwhext06&79~^AUzcodn1WT!BY)Y{9wF6C5=m;i^|1ECg;3ZL>c(D5^`>{k zLZzO|eu#7yzA+?4*bBTi44L_Tkg43{3{aE4+o~DMwiH^;j{!39B(4l__v?QLWZGfh zefABTQ0SC2G&KH}<2)`h0@%l$2hS8CP3!AjOVN7gEfp;xs>HM8Mg95V;ltaLA%KWq zpzmSzlE8q+caOhJBt_F7BYnNOoJZ~-Ejx6h$ri~k{i;>n@~kJ?NkfIntJ&_`QRwu^ zJbV4K6yMmf9`=+SDw(7rVh#uCSRq|QvLq`Z;o^}6KKO2fK-;bRT0=XHo$s0<@xq<@ zXV?3DwpjWF!kkDr*!;;8%!|K%$eRU9N=nVjm$S5GjpxZ$>b9k`(sFWWlL6|9fp2+R zA_44ny}Id)MLO0f#SkwoPa%dvKXyg@~JfPY9*9=Vo>q-0wHp zzQ`4YJii!G`Ep>lIl{*0Am#+!^fJPf=m^t-9BO zV}%$0H9C3XrVmL{|L~R)Ch45)ed=a8G^bz|aCQ%Tg!W6?@Kt@)U zK;bJj9JZa2zrZiFn$OQ#d*M-k_C4@MbL;BJAJuYXDa#46x3L`r+Q$KThYC|)Osv2w@mNp+pv4W7`FCLqP;HMMnv;b6K6IQ*$%$lpMZgT{-twn<;8 z2pF8FQ6%iOdj;&Q^KE%H0HUAVL^aaAkqUzT=~H&BswblU(KH^IERM zE6Gp7>b5^p^X(3zLne2)E%@{mEWhGaM@&E4kK+ z9r@i<`vq#a(t+BjWLjW*Jb|@5fwZe4kPV;1OP_B=)cUqA5KEdd{IAl9x+^vIfz;J< z8)V~O&uAb_mjg6Xb14u}0Z#cTNa!@7^fJ5Fwtl1#VRGVi^8K$E@7{Y>%Z6J#2g8?n z*|d1xv0>uYM!=RRI~;Z&SLPk$#g2uE`DmXcP-Tm^`APw04WPe|aOl?1jda&xT!&UD zEKyPS?9y#|>qo{Eud_JH$CVH6D=t%S&eO{yU)GH|=#?)(M0kF$d^u6M*u)hR2&_eC zn5%H9FGxoufDm8H1SM`7|AH@e5C1-k^=tnXxJNGN5|Dt*&W3Dp(_UcVA#8a_9&cn_ zlEvnSS&{|ZPF)IVHii4$Bhwc`A2JSL{KQkpAS-5nbetR=0gIZOB|JYq>JQgLfX5I+ zhW9Z51TDsVX@iLeB5-kHqHfC5i0!ceW0CO#Y;ArF$+z2l+<(8cIm5^u^-Rz|%s06k zp9+kf0*u~#xPY2B5yF)#CAPy3bPDmc{t9#O;NyJHAamz-LtTB9gcNgwHLfi{P`F*nk;50R% zZD2bnWpJ=`q8!TCU=C*HS`gbT94t~H)kOYbcK)NFd^PGEeUMBku%o`g3mVv8PuFv6 zAA_PbUfI47L`<#~3Ny~ePyOmJYgtK{>!+Tz6>ndypFz=0iMkk8RV4aO+-CZ`kd?-5$V1``9DmLNCHkzFuv6a zqE*YUtg6ZYuC09#(-cAORvAP+fU{~#BWiy%+GwB*Cojn>TLLVU=aAU;6h8`+cJYQX zZ0*K{InEDAn<$(u!Vg>HV`sU#YIB{o6a;zV2R~Y2A+PE2XdN*7h0bj@kO`5(zUzjU z^%u8Z3Q}M^$D6h6?0%S^6eLAfz-^ywDjHYVG+!wLcW2*dp7iq7QZJ6Z^C{5z4LHpa zW25{5D&PNRSxc>hf>N&FJcfx7gdaW!6d3;gxu%|u2*eyPv6|#wH&LZEm`23vn@-t% z4wD{qdWNVG@7!Qul@1;xM$Be2Oj{5iT%$OBZb!?$in+1`#ko2(*Na7XxC7@*YSez( z6T?Xk>dP{3uc%p%SeqSuLQ=v1=GRCl^mXejF)qSLIaiNc3F37QT|ubBs*4^b(&zHt zbERvv0up<}g;G+p^1WNC5c*HftF00E>25W;ypa@V-wHtMo`}toStI&5=IOH%&yNCt zt5ft8rlqVIXCL`_WH5Zajz}xwI~N zv&7f6sMnm{BND>*`;sq=LA`=!?fSg#&mDJQV;*7aEa>f#BB1f)m$YBi;9w{9yBZqDWQwLJ<{@l!F!Vn zQKTy?!r;WtQe42i;PlC--c_+gzfUW6H?IMo<`KZmsb3fdXdw??XZSyG$$coyUAfN@ z8c;zNYszRFUi{m^N>$+jMZ2(a1^TqvulNjDnp36UDSlDrAp3x##&0^Yc}b}-$^kTK{$=uqCi6t4Ul`Mq)A>rd!^_$_e1 zoa1jaejN}dqV8WX@zu!W$WClM11+SDtAng)iik85dPkDLmM4I zi-og%OLPyKd~doJLG-@^)QnmS7*s~UUVcXPA{6^I#N3*3m#W7p5PQ!(5f%$v!|AfH z2=lqXMziG&PCek{z;J&_6N6fM-AUUCRZN-dFS>%R+4nWyGI8Gfu2s7nBp8Op$VrKa z?mJ7j^8+e(taK5rl0>2MMW<#!L3ix&IRyHelD3@q^XeUa5ENC&d07g$z%2%k+&JgL z4s`Z2I3i>Gv*j`c9=Wb988wmsYHLSs*5!!UJnl{N6WjWb(#|%&J(| z|L=#0$t#6UoMy=rxQnu?T$xqDp!&L&_NoP;!QT|mi|7$DEd%wU^j5T9GNkmXurE=IFnQ*UTX`Xj8 zN~iS~6X(CTrq&AM-STWnq3F7kBeu*;GZ2T(S!Mfe(NNzRmnV#$q?*zNaNht{!pfG^ zXX6M+6$L$MB|D7H0@ zB)X9k(WN@{LI|BS2i~z=748`8aYY}qs0DQ;yAAzQrF^KOER!R~^XH3qR}e(t2l4wy zGP>rzT)daKK!NL#9MC0DA|1p>EUVZnAlD4ybv!0@vu1WyXwEQLWNGqbUA^I;539eWt4yH)RV^OmBA3xZj1r>ZaB1$ zE;u>CJx%dA3&al8mrOPebO$$*IF-2g&tP#rF;W!Vp{H_UC-Gz!ym$ z7KAuN`Kfb>bJ&rO1yv5aQ{(*?4xW_ng@=mpBig?HP_HzmyED!GK(^2HO3cH%)tBi; zMw;Ymk;hkPjS_H8ak)8%R>F8j9^~L}r?mi2{)AxjzIcZWV^rJy6TnSh^#D$^6s+h2 z*O=#aU#w|7v={dN0CsaBBcVx;=)i7@a~Gw zpThI1hh-81o3k^2@8ErIIKO;2x_SV#j%$TF-0uwm3{({gFZcJ7eh4oS;o%Y@c;$KE z;$ns4L%>IE!ifE_5jm70nsK9}rndSLQEaCy|2}vYxNv)wwWW{PX!Gy`rtyPngg6wR-Zi}`!n^)i9g_P1Lj> z5WQN*puYAYzk8=ki7(K@zL`HZ!h*B36$qr1$NCkG0GU9j}s zpj8nqWwYPyQ#@6_*lY?t^B8}f7AGsF`K!kX|Ai2CY1-x}cAtiaN6(l_;8r~zN$$1n zfN6Dt`MsbEE)Q2a45!4MasD$YOA>IrJPe;~HAwnJ^`iM9E_*})!u`|EWXWvRl^ycj zPPr((qo13XxA}58jNK&G`>X@3Y-zax99GKxkiND|9I%3L3Mi5MW^UU1!OOi@0l}KX zxNTep^&vM!39dv*(3Q8yZ_wx;*-34%4~EO>%nAuuz_kP>x+61YU#I187z-;ILZIo#9k9vd;Kc@ExiU=Nv9% z{9wk3t0QfvD@g)i`Dy-L0^*;<-4@|srk`-vroK`GaNzubq_Bx@zcDDYjfq?F-CrN~ zIGS*)O3hwRz)IH0H!}OSOcE0Is&y@e!xa6(=t0CY5-+Zm5#C2pjJA&fs|d&KMF?I3 zmof~OwsO&Jy}7zmUw^x^MCy>p=kNnLQ48EYFA${yb(TL;Y32W`w;BgXV&WhMnRyrk0gk_`1RC-`0yuBIzJIbsu$qCx6y$dRD=bR}LMUPY?Gh z0}7xhSt?BC?JDz{NQI9SX6G-Jt{8AO0G}q-aLIPzn1^0imI%2L{A-Y;+8VNU4`8xr z!@$u;5+<#&91;}+fTW`dgOfY$o3%qE)W+lmqNRI_(ev;A@9(m}6>C@3Y~Ty=N$7NTUeNzVmU3*#`64e{t>UP!r(2XUHVmKKrLYf z4airyV#Tuye*OBs%i3IQfEa%JN5ep>%Q@!Cx&ubs3M*l#;vw3eQ)&~QGRPBsC+%a@ z4w=`C?&<8AZwK~0Q6LX4rL{Zy@oxmuf9Su=wqGLe7m$Grtb^ry1*xi9Rv)ypjDeOG z0br6tfEOdlkp3OGsKV2tcNma|6rj3P?!e9n*NRiP^nFoVYHI_mt81P5W;#`lEQ*yU zNOSd+hgZOs3CYGPV_lypQdnk`p@cQlf7O388>6oUx5L$%Ev6q0`3-8v>)POw!~TF` zl2j{;EU3L{IN9!|B0Wst;tYp^nm~LPqfkzWaMARoT+lAv3Hu2ah zka6Ev?zcNvm-}u9_Y@lp(|fu?j&3d2fj=s@wzl2}OaqQ-eUD=^8XVVTxi{h?p;1n2 zaH9WX>MaAJ>e{wp6hSFb5JW~mBt%+bXb_N+Zs`tThVGP>mhMKnW9aTia$xB0?tC|T z-Ou;=!+(x@@3q#s&N$*4J4r}mysjZnxnUj$+Xp!R6*iR#qy7xx5ai?_K=Zc1^wbeW zc-2)hYg$NMg`39lP5@P0l-+%*R|wi$kTX^F5buW$mxrQVa6&QvqHkr90XA*2`gL5OpPd^S>uIry(CF


    COqt~8Q$*18q=J+)V2v4_Aa#_#yDwom&dsV3E19HvwB0pt|!hz5(3$<_40aRuf6A-f|1>Y*NMpKq&w5cs0WJ=I-#>cpdi z$xZqcE5EpeaA>4iqQWkwY+Zpc;t7q8`C7!JgQoF;O2-6sEL4Ijpv`Uk+a16Ak6+gd zZ|+GT7ua*kvYb|%7!veSZ&!iSuB-f?a@G)!f98*$h5*ToNd^+Q=vP!>U=!(XlQ9@R z0;gwDB83V0(N0SUKsjlE9Tx~C%B&kTa`EL2mZ$}1CkTkrpzJ9ShsO_!;uCNF}mw^Xs=5q)ov z%cD1EpR5sJ0&-O0GMO)&$e;nwPZ5`Swxi^wbS2`L-@&h}>-ykkFKklcRmbyY(MozC#Ni>}PRBt5Sav=S*itfNILrYe*ZOP7s<5thzRHnQ-njUWjp$mMK5mir|RgD zM8Cn5L?^-|{LM}}R~%($++lm3MM{DueALsR#BFMJ%!)$JU6Xy7{#MNlAIrEcd@ zrya8otuUF3YgF*e=lg1Y>gVoiKMzz-PwQQmE@qT1Ik|D$bpV%`3;g68@ipd|OLzQI zI`HX;ID_TT!UI*lw6ql73TyMjrfG*oaeUxO5SqzK1xEFCzUgN#YwY5}&2Z$D`h7cQ zi`kw>kP+6KxCcV2i;c`PPEQGXni6>e;rJ*3n|Vc+nq|2HmKzA994lba=y*6K79C*d zu<9yMK~I4Uw~?Tm$?22)rP`sU@N&N2rkj?E$-fjDX2bUGBrU8Zf~L}N-t?y+pLFnk z`5#HjBOuZTsJhtL$}JNhyf`$@=vW?owM+eYPCnH=gE>q$l{OQh-y9Q*81CC6FEqUF z6+1EXhp*Y2Bd%s=tzaQo9}%`bX!aT^((b>04&7lumKXVrCmQW)yTs#Xlo>V;&VTD8 zWLCAnQn3&DRv-|jt^msRXX0jHdX)i%mM(tT1DqvDyTCPe=x%Dh?;0gQ0jWtoF_icA zH8$Q*8i{9iKW!?pIKUl-cQwU8aw7^2VhE4qehNgcAZR8&eO)i?Nd+T@HAB7`3W3h{2-2{NySYNzZ| z*0JGNRIpFZDXUPTXVMT1_95?kT%5=kl_mEj0S@IeIY4pX2I+%4Jx+Zeh#ExONngHb zaMjiKgzo~V{a<~^-(W&($t#v+%^ZxFY~OsXF{rz2k>DUEU!tXX)e03%ai2PAOT0I_ z2NX&505`KMfYmn0^MlK7=Vw&n)1OcqwV~QsWl_;7dhkh;d(7x}WnemA<;o5aI3 zHfnO1Yg%KdD}(pU_uPSSaE(k5b58;Aqoq$ufDAE#gr(vXZ%In8&DDruwU(*w`4 zD(An5n`ygoVZvU{mIVv=m~b#N6y*rc@wFxtwY0`B^I*# z6!y=aF_~T(2FIYFu9O77+1_aA8^lFC9%NRbWBjqY9Lw!MX4eigZiUgZ9e9sW7t6I> zGbGjHL*%$6tlWc1kC}lS{!{k#ep8C`)q`eCm&_n?Cs>+rvC_X9@vge~Wk{Pj$s&(U zGzJSS43_QMA2j>4T|6+wDl>M-(~XVj0!Ozl>}Jd@z;Cl z&{S1ThdAo=t}>5PgPzrr!dELDl`ZN<4}N|Ujy{;2b2L{GnzT|8PJ`H zoyTCf&uDcjvUjmr!gtUE5ye%H@F-;M-uc1qb@w-Zn`XSO6WO(?)Ug?MEOmb3Rx`$B zbs@tBxyEVV1aaBJ&i5g5H#~RC!_B{W*W++`c#l)t>sL~|!yOO8vQiLN3HL@NRdjYw z2p-G?6)?LIEL%{t34YZFqh~ zPMOxfms)?K+$mc6eAS<~g;iLp)b1bx9Vvu6PMK%*LQ2(^^>FQ|ZB%OgSfZ3K9365E zk$Qgz&g!-M>+`wz;%tsI(;Uhfa|gr0xq!d&&!;`$lXo1erNQ|=Y*(LVF(zR2$vyNH zmAZmbXIY6s6p8a)9TeP8uk_!|^aq*v*j@=CB>DB!q|dl1&1-}m<$hZ!7MQ23)Iihu z!#z^y55p~q>Z6L^xPNFY3APkLqnV7mc|-ojI5}<>#RE#?+MflK--M9u@bXFsRas%D zPR|L0jl*%OZO5~y@}f!Upyu_%xyH(R4GR+Z{aNv8)YCiI-I~_mlkakh6r3v$^qx5P z{U(|ozB5=yK%&$-Abbhkt8w0~oYdWW;n@s!C#l_gUd&e30D2d2KdjLp+SN4Xu-YMb zW@&li--cHb(>E<%w#P~Lx?pBbi|-RPo`H>N{VPK0CRfx_=n=Z7;Zh(uYw;UtRNaK) zL&Kxt5#9b+3PKHbJy}D3$-2lG3X$dEwfi;4R1ei0VV-pk9UEq5XPNvT|B?L#mkd4_+ZCWYXtwo?CaerfD0+ z4skpCs@s;u_P7egxxoXKSv+N>kpNPDVtIJk30>-NS<{Mq&ey1|1@v%U&(GiR0pMDU z={kFc^Y#bwXo;%c2rmB*_?mm#p+HNN5+Q=tsI~)pw#c-QjbN!Vv9ttXS5?b3VwS{h z73XX*2O-$ntz1pf;T(R8j}Sg=td)MwI6U&l!smQAjC*D*x>F-9O&F3uYilqPDlHdH zJOrrVxD&<()d_t!*6DmBrBzUL)Ubqq$RavE$R@^>LLQOPK7Ni=QusUiP3~p~D>oOF zWmpyhS(V!8sMafJ*RF`hO`1x!*^E{XI1hLMR*+d{vYf@G@J{K`B=-BJ*4xIA^SfD+SCff1 zlJfK+;Xun(s0qLt-}Fekg>#$P&pBqtEc#gbLmvT7m?Z?l=ublZ2Ce*S29mhqpv4yO z2~ujo9LXXTv+(Df*PNSvMRjo}OOio1qqh&=jQbnDJy6gg$b0gJ(o>S#tuUla1tGXL zpCr|ka^E80cvYr&yPmrX>1Ryl8nPN65D}xw@`^BH&g&+h8c8=>MSn7zBHb%%92m<< z?DTc~^mWjC)6GxuQ~crON-?!u1lM1yimjc)#|W$!k=`%?ay?nnLtXW>b{z-aDuLyJ zTc>mdrZ22hUphh79et_@;$14#WLq)9cF{ipKO?lc|w2-QQ-sO>8}saKYJ=P=!O1v)7*V^bMYCiRvl6g$JPk2=^2Y`dNMEMadmtDllmo&dgD<*q6dFC0z#<$I_{?tTR5Og zErYzR-mr{=*6_w3N#`zV)N6+&IiV;7b~ocH6Y&D%1??&xR@2ZGV2eHVKNIJDb|)T$ zs2r{yxE*SGC+WuNf}7dX$CvcU{}%nI&G*rYJB7x>$UwDFwKBkY;-N+H8PD;nZo<3| zsu$G87^TPQW|+_#aS9W#bjV z2*EcIx_MF_F}c?6`LZ;a?KUXxPeoRq$l(mw`EGiODV9-h(^oflwRHzih(rB#L|r_u z{bh9TbH_Cmb!s-(K8X5CGEGMj$C|i);revh3`hDrvz4i;6O49IiJN93VM-`xGcCx} z!3Z!$TCgTH=Y`Q})LGsYydi$t$fUgfyQzDAbo`O4yzX6Ma=j)Q2ee-^UG-zFNq> zit~J0{Br5}KblUs8?)YlhDchl6# zOvriUx3xM6(SkCKgLJEJPse=@>f%E5P!V0a3C0NJklECh9nRNrk||fscay!h#MHQJ z^ZmX!j=!z|HuWtlkeA~J`FYERt?*yBf8c$TL7{hL@wgK#KxVc~CG?ZVf-mBrhDhTr z%#sg%xteUaNo%cZ9dvUpR0vHZ&c03O0yJLJU%>$!qMi0W#r-kEpqI#9!|mQa?CVh{ zTRTkb3tQln=ddrFa`%(Fj)&|c*qa0s<$hbi?h2Kjo@4yalf4rBt{M|+!V#r{`OHc( zhIy=~l1m%I)10{af{5WoyR4Bpw3?ABIG$6-!D6;|3k>DA%~fagUOZ|<)6>26%`!Ft zm#ga3_0bhQRlO=$$X8faE`mHp6(Ibu+D1>Cdw&>u>vztm?7aosHHO60q6>H4iV3T% zI|7Z}7MP;rE5Bb4^7)y|J2k4VgF4G4OKT1WQxtnmZ(;r;`pr9Ei`VyN?i-wD5pGc{ zl9q`U-xwYHx&blmiI=iUG!7(-#4HD2akfR;yI>h2q15fL6ECcW1r#h3)lR}iA>@wBg$~i9oZeA(O_vwW7@)!c2&U|LA)#k=r8E17~ zdCtL7X)O`(xdm?IFutC=w>@G2o8GV>xkpN;x6lO7F%-fSU%lGO-L=*c_7eLSGx?;? zv2_iUg)itZcBx|(6S#MD5#<1@8W?{qJef~+^#d?)Gy6hD<}XqZdOS*I2qr?HCtUYs z(>yObAMp1BwQtH>PLNmvXr_LzM3fkJ3WP5uaPJrmvoZp^1oz_ne!$$VjmLmSso3+sY84nQm%7VofI?g>ol*IxQ%HiM-Ao*ws~VUS~Dw7Y~}(Og{)jj#0}9kb!Fpcs1|0?B3dfO%O7VQh!i!x$&2hfj?7ZxYif8xwlHg zYO?T7=+SdfW7{=##lms1iE_Wb=>->5@m?(Qtr>2aNZ;vA-X!RNRQ0(rn-}=q&%T`gy%(C#Z_TAhNd3pH#8>LIPL3cF-CW{ z@E8e~MxL7S2X#b9AebgCEGRy@^SSHT8trS2IC6M2qrFDqO{;PH;&uN;2W*(GDmZKG z+xqWvS4;_jgDni{YoIccE+?H2UcRq%(c-5oUX?H%x-wOH?B#)0%t1zp znmpSA=C-LzVXl7yW8DSb^y-BmpbuCwaA0HBC0|4I*Q|S*KD^{4uTFq*EwO~d%&Lck zuBrRCApT%h7mwr#2jzxnm1)4{{hmQ5yR2;vny8MahRQi?z%4=>G-bYN@R*LRvV$ny z<>+`#`L1PyguzL=YyM;v-~xxAjBXOb!%%_E{HN6iqsGs)#S`Vw#@Is#fol%tp&Pd% z{X^W+R&snJ`@wG`d3G0Zk-xz$J1%eiXAA>wC@Q3&KWVdV z!0pITgRK=+8}YUswousdnev;Inv!8WlL*lV_*uVFGi+JkWpj%R> z!^}b7-G|C2@R|W_?S954WXC0DRn1OJ^WV z7Vc~D2&o>2Jl4tFCko7;?NF!qS<`~n_HeI*h!A!C{cuB2Q?qR z8P_{LM#mbh`9B<+&ZsOft9q_YS`2BGmgtI>Wg$82d zAOp6t-l)rSAKLWiAn^)bW4#XOXanpje?k!GdeFYHjYp}t@yg_eD8mZkrS@{#2??P& z&HtM4zJ6g{uuD6kCJh%OnDG^1SIV3dW|5S;`+YEo?f+mRmOfCxp|uWM4qCCPHWdDq z0GJobffnxfia&~HCt)ERweX<8*gxwyrHB_>`Ag;Y1z_u@V`+52gw4w6awr`@?0O=* z7k9PF;EuKp1C(!UY*wO^GSRV4IrvC8nOk6iU2|;$WbHnkh%xWBV3e-KtrrllZ9o(B zADBH53h~POP(SzxRXbpCwJ!!FEhB|NGOsJ@o{-1$nTqL%ELZ0MKyAb@*IcyR08`9Q zN%vjznd%sklZo%J)`u6p;NXwbRpun;pIrg+k6-doHFy8jmD3*MV)<)`7r@hu3w+I> zv^_MeDJFG!Icy2HL&FFjUc2|N%At|7S$LvOM9|I2&HdI4a(!Ctj>U%QjREjNY5~#4 zCV&cN!8Qbdb)K>f72L-kCq!Nv1AoFw&0VjZWsd5PdTBoSLy?df?-#em^M`*Q%W3i` zqV;ja+}Wvdvdr8ax8ed&@)~TMrg9z5{rnAk6PisZ1K@oQ2ES^zcgM@#cSgVm?r|0J z9ot6D$5VJrJ$4{$}hAzGoCTLK`o|6~m)GrhO>!ki z6kY1V{dUsbQW_U?(|_6P6a$f3_*7V_s;cU~{EIXIuOhs4+~|J`10(}_Z<=iJi#lj7 zov2tAwvbdskPu1*-yX3^K%u*~uX3C*`Je&9qJ^a z?YBN2nL%EEgpRe5?w261HGj~rroeE!tTx0m2tF58dk$Jthh(^JtB={s481(nqwzl` zxb<)yo(3F2W!_8LS?h4tyP^fBgv`JikELPZa66{src-W*QwjIwZ&7;$LBG87k$q4r zPJgX&BqDrUYoHG=`H8zK_?e0Ju-)_Wcd4j$fXMi6+I04Odi;=by{(;))OG-nbcw29 zfaH+MJv(CYAl07diuxssdg`3&Wvv%C>@&)tHnAGVOk^QLL(&p!BzaBdEZk{%3_KRE z>iY%r9i^qQ$HHGC-WLGH`-4083da@j*3ajZmaiRdTfwn&sL=J<*;#-S53fwwvv6K! zpR8B}$H79g?c=o&dTyKoP!^_Lrmqr|_hC>yxnJ(xaL|}8DMwu>Atox|5VG|*!TpPc zL`VF10vG`Ja<{OH#R>v!rkvayNp~+3UaiBmsZ|zj+H5EasA$3U_YNJ{CGhT$4*dLD z@y7WD;z*L-{Oi&e^SgNyUrKBCOY?nB%<*EtpF$h_!z9Pe$GHSIfMLV^ZX~9$okt{- znW!{a%edJ^TpPJs$YVxxR#yCUq)7+K?U(7(mlduD5^%#sv&k0~h0kD7-`!Z>fD8L_dlU<7z&Tv6Y=DdLF4@F) z29~ILw&v(b(l@6MbPO*w2BS+C0r@4L&iUwRvO2AEWu5q&2a|=QORYwf`sSAO^{zQ; zZcGgxqoHJ<8o_1=qX>#gPd6k?!ODv1{DsH)eXd0}z=jrRc20Z5jZ{T3lL!#?744FW z>~6sVW?jI#56<$wN4PESnPibw(XYn~2+ZOy-%&Hgv3s&)@ZFmyf4GI@TZbr3jwN&2 zqEGnvoeC^$eQtShyZFl?3M7G$-I1qh;B(HT?!I*yK&~O;b#YMsK0azbA$!U=ZrSW_ zV4jq%v&olKY7yWhP(6uU}PZMsF!!0an^oWT7_RYh32CN%w4$G;@qH z;@S%a8FVIg;I;V>{i*y@$ruj!K|5pn~Fty z_eHhrq8(38p%aI)NN}_QE3NzTi50A-%LouBbNc5ykzyw_UMpk`OtL6mCIU%D_8adA6CTEKLyO)3i zMDq|MB+N@YiS<%KrnE5-gd05i$^PKyU^^BL~+dq z;rh}kaa6GQsp4G_rmudXTcK)~bv z{0ec(E}}|0NpD>bG~>-%ie)BV&9R8@{MsEj@+SobYrBA4*Hqnwnt=#v^V`Kbt6(514G zG1)S_`jajcSZ!oB!Va-?M zJxoaW!As+O!39wJ_7Mk00l0{688LQc{Woj`y68lwz-gyIJ$W1z6B831jrYCx*lu?o z7E$JoAK5g(eDT#y@`8yW%m;Zy?tIp_xl(CVm z$U-Rm{cx7dkvb&=29P=ze-v}XHl}CT65x5KYGN3LS46fgL1Z>P&a4h7kn|E$Q6=!z z1w!w`Ud^lniX04%CVP$7W*e;WCPvL#kmah2UvZ^Buw-;Ui||Ae==YP)tXotmD=I$H zdz-50k1lGlm1>SPNS-K7<)*XuNuBK5cicg6Bxo3crbp(Hl7*qk36r-w@lhBxR2Wp^L5?x1c?d^5QqhoIpv|MhZ`xz_8U~Vn9)z z2+2%syl&Wc4RtmFni}jV(vK_1EP&@Tp;$p3;E}ob?Ei>|uVjh&)oY6sK_6ny-L%IH%(e$Gw-$80CrEon}`6)nW1XvKs5bd(z2leN@Pg8)2 zZ!+$~H&kneN5{0{#AoB+$c+P(mr$sXX~Y{t=0o>=P3D;V0fxbEb(S6B9it1!^)q67 z*(Sx=CX@oLlnO{$7h`|~&bzF8k1i*w8c{;V4ikTt>;h;e z2LYL%&^z6P-k$)`{VxPNCJ}&Ozn@lN4!rAXeJo$G%JB`7(OvjwjL81yQdcZ|qoF#j zNe%P3hp33@Jy3iC&Pk(yVH@vB8g^LrGT?;BNUIaH?!5u1eEvPz?cjE>d!bF+znl0ZZp&Gm=Tezv1W-LL;njHf_g$C0)V21OfD%5?$pSVB8qdkF;7o?)yzBWQ zT38z_;)C=O8vnzulfo{~0D1f6p%YstN0xxj=jiRQ7fH?R8?JJ-gZxv%Odj$?%Z*|W zYe6s!2x(NWFVSdSqH|L^NL}RR=JG!@k5IgzTZT{PaO5!%v9->vfc@ZOY>F(YhTOW+ zwD7%OVuR2PGbY>Z;G~@Z%`;&=BR!1bJ;ae;`X~#)(H#QU5W{mmR>+;mC*MkLepblvcuLWRY@6J8H$l4J(U#BlJFM+ zt@;-Mo&B-exHq(82zNY;zxaxgxt#Ft_CU{|*R20*M9%#R@VBxT6h!`S!yA{3d48As zV7HmIMOln7+LNR|kt?wd2$cTDNiG#oG~GE`UkV3hl)Y1VuX^MpnctNbH$i*&?Gk$B zc`B6kIsMX})-5Fm9Z$@$2LInXMRay`JrH*-usiw~nu6BtR00Vt> zjx62!s!2<_f#=wozS5BrhOtHqacjRLjdGzzRX35frO(umu^UOC`-gzCEGDau<=a;e z=S3!`WCOsqt|$L|@UH_5{#vb#vwddY1YK0WN5* zkd{bly8_BT0lTPd1Qk_9T1;duqu&EqV77(~`tLuO=YSnG%+!bxH$A>7Ua}F)R$mjI z9B-c}8U(UCEOQ+zScN+h8{}+P11Mub!i+!FzC0itGXVp7`fbawz3dU&V>xF1em(p4 zIYvC{H50QUs}x*$@q8y_8MSq;xc2+)F zq|s*W`EbEv~xq4)_2gX&WGrRq`byfR`_PW6gQ*a!)%j?r!$1eRS5?|Zp&29q$~ z7ff>n3K#Rj8n&{6B&^JNY4SMQIz9H9nMgQR6YcF@O3i$`wO0!5U0ZAKIaaLwBHecx+ZMC8gD_0(8nA9yrpo<(HV4 zy0d*N^ANY!dV3%sDqUlZf=qqjo%#YH@XpWUQ56f8m0G~J2S3I z%z$E-iO8!zlAQSL;Ho%2&uXg12*V0=6}ijuW$M|i6-t4#6IdKw%4*YBzq7^HJ_z!D zHt+pC4US6h5P5mG;mSm%$+iUXgsQxv1gE%BQDHV?=;1-wuND%SEgj(Fy3xAL1$XAX z4GoL<37{OP6o#D+J6_j3UG(F-S!{2A@?XI7Hc-eOL8chrNF?WVM$`x4AA3N{9F{-5 zD9SJv;TP^aTB+$_HX0BW{k8rZe?OVvQt*A8uOUhAIWjZ|(uR%BA2;yx4NYWz6 zk;#GQb?byoC%G01DBxqtM|R_kzRUy6H3k#5W{JNT(#;oysq}y5uV@B>a$VT>!e>7k z{^kQz9OILd^R|;3D<6?2?-#;A|1JEH6*zx0n|hpmwGOd_iTFCfWX5CyfLFEfv>J-6 zI$Ec~Pu=(&b2U*B#-U;DU1EV+8ZI6>YOVyO!ATroBf*9;$*b5FKkwA~=MogtyF93}S38l5dqQ2L(inl@0j|Kk#4v#BVRXaFk_-&;Yj}V1g^~ zK4IXj7%**^k(W_W@Lq4v&dbZIX12RJ`OZxWOelYKw`i*c==9oGb-aOn;zeH1i*NOU z7L1CT49})l)(F)qIZ5iHN(p#Fd-B1S$=J%#u}hB-YwHA2x}e=HuNH2UV>DM?kn?w^ zo5q;D2k7YJhFWSgVIegM-A_HVSBR3TholBQcx@n|i|27D_tCTD7Z3RvLf?NAD&D^p zf0X^6cO5n$$LjkC!OnNY%p7pK8?X+$421%M*pKbPil)?TcuE7HR$k<*Y#)0ivW6@w z1;$j`?Mps}@hfVRV^GG-0-p$}|Y^9RURiHkPPaMt0#>QUe z_?sC>Mq{=okPkuwl_3~25t%UoOvpmRQIi`n;yCxE!JEvVCg^qh?t_T3zO{oVN{s|y zMbe9u1)xxrx#KpD?N(s7{07x&386%68dj9qD6b`qe!u4BcG$}%qSuMFu(6-g@*N(>) z?fbV)HpR0+tWimBUobAd-PBX{M@sE+ll@K9JkU5NQk}8P@0v5mP5w$cxb@1$CP!zp z)sf$SHyxCa(58e<9T@tvLA+!g+neSC(nDV8fw<<(0oYrcpWCX0rn!IVEHM)#BaV?R z;!uGJsv`xxC@u6qqr|Qc+i{hP#y0nRK*8+`hhS?Ha+C=QZX%FXQ+m0#0{a%8$wSgdIk_n?{q-OZs@{#|I$>=Q^4)c(wBb(!fm-khIq``kwN+arK>k!0 z)N4MQWg9pSO

    3Xwe|IM|gn%gFyfD&&_ZQmbu#J-@S$BCJIlhDG%V1CvY>wr5q&&Em5w% zj?pxPZrT**!v<7UOv>cfW!#e3<`~6&{ygTvcP}XnlYj&ou!aAG=4^Dp5^x9qBu6hw zwOJS~f17tcVAU+eiZ<{5ZhNr6|8VRR?ycgmrgu&hh6oi+3^fx5 zeAvX^>LD((bmcy1%8!#_gD~`+5H1d(IdpwVWc3+qMk^B)Cl+l7Yex9n12<1z$Mjwj zvw&WU4H<_IA+vwff$RC!<{B~cpj!rmMy|Ns(8xhqzMfC^ktKcLFq_@Z%W}h?#1-GI zmDjdvqmo_X9CZQsRu6FjD|GLOE^1Dsx%c;cM6v3w*z~Uj!(jhO3V7Pw%1fxzkyEho z=d3bg528Sl550{Bq;CczO?QC)FhtPj@p!7PhiaQU z^r2=(*F@4=ZUg6=ku6pvFLg}zk!>0!AJ*_eZ+QAvOlkFJHbL5@PCN2Skhf(#pRtLIrgC$?YAmHN^1W zTs?C&9wgna9O?R~P2;G)a4^k(D=v8Djyd50CE%`gJJ_i8$TLE#*Pzkj;IxR6a1OME z10y}I?%BJzA(u|wmfZVCTYzepdW}b=#ozx~GFLCWnt%iBZ#wUIsshk|G?qPFPXhl& zjz}aB=Bu}sa5@9&$Ke|D_wCxVeS)V`URN=GTX&x_jbfn(q7&M?foIc|4bJypZR>WO z95zP$0Txw4g|21)3gDyk;A?GV?`&W|@nGx*96_Lypl@2yzfc-$SIqX&(1a?P!#-ob zBljgDz3sb+ONAs*_)?M}fC9!DTV9W>IEGK^M=+BnAE*N&qL~RTY_OBm>cCaT*vm$h zNSu99CpjIE9#o#shOD|VA}bRb<0DN-c-b# zeHRaq{PqW6Cw*XMqNxvj{T9GtKm;#owCMnD`P62edM`Af zmEKcI2<&2F?wiEj;c}ijlJuTW1 zW?XLk#kTaEOrY&8;Mxx0I>bux+uoPCH3JX(Jrt0Bk@Wmo>>eApR-)OcdJ>b5;*Lr} zE~uc45ce8$0oyr{&yE{rUrkLspD3!_)+89?!PbC!`FYGFmrv?HSKzaD9g;K2Q~M(r zB98i_c|!|Q7DI@a(aJmCFGtj>Q_h?dpdY{z{D8DJ~B=Mbtp#ahWm1bo* zSN#+yl9o2=58L?#BM4xPO{35a7>3~^;r+rpPz>F}GqIX(?DE}W8!;2rv^p=h)FVraD2(1RS@snP zcLRq^LB^#K`6+}sE;Dk~zrGf({r&?w%Sl>4TT3vvsyeUuKI-e|cj)#{Navq?E%5t) zP$1q@%~)b7me{+#`e?H~!p`si-0O)pLo)M&-w^kZfW(*))XoFtuV9KiC~(73UJi?c z!{phqJhSMA(AnxcV$4`_V#*A^dF_KaAT`&qGI!q)B;mAP!~DO)11voup)QCTXkJ`R z?Zim6x`4%X@@_YroE%ofx$W=w7Q-@Ag5ySnLxaiN$Xec@jgKoy#zKD=Zi#Wlf?XGy z^%~+k#uTbrpFSPvO7Cr0>F(}+ki*=Y^EG3uCu#MDt`DVj;d;uYKiC;sv~i!o^*|5C z;V{?(IA>-cB0m1BB(hrd#bNeD>VuPgr`y@FmbI;DJ6OTN1FNti-@&!|dA`RN=tFgk*2KkQ~Pj0IL$p=OKx7YLc#OD6PCii!@!0vWG1!QCS9{I=>JdhHB={(R4ekJ`3-6#GP09Xb# z9iAP;%Oh|w3^duBk`=qPFsiQt9mMkgF_j|)W7Yo((gAZ_yt8CRFpo8hYJ)dSMtOnN zz-1}tG2<#}wbORga*Q`vPszG4jXw~AF`}|zlI`H|wuR?s$mmzbs1207e9!)qbs!}I) z;WUFfAgfpD2Pm9$+BVZU3Ty&d;dJr`=j9I!s{IC5O{uNlsC?8+G`0wqXH%PkRkHgD z!LJQ7fPL*nJ-aV7dR<9eWiWKKI2ZUiD1ue{Xymsz<44b#5x}%gSwn+p4jZsNijKwy zavcg_oKLm-Z>G-YO-_MV#Q zcQ9sX#UEs{-5t1SZA$Q#2`I;9OzCptbBES-U(6XkpQyC0c%>G5E5}FwmC`mTqrt?U z*%3fvr0tsYEj0tfUzpjy6Zf}V@W8080k~2NtIR>5aI)>EQ*K}rUf`1s?D~2K1-~WN zBdaNTh&gQFa04)tK)3-6y`*WHdIa^1p>kKtWcK>CVIJbpzB6&~S!gsM8AD;ucMr(U zBAc~eKY9o={|yg-cq$Z@YKp;#=9CCo@YO#jQ$F4&$x%TrBJ z)LgIISN`cY)aw$)nG6HM{RaZWjO(eK#js{bl=svKYH!T*9d~{0`K87Yzix%9WG_<@ z)WVl)pwzs)tq`ugcWlSme7BqV%Ee0$+W|;l{)rOfYTNd> zza~z6T5$5Wfe!TeA3!ZXq5Tf^XzLa!caqVmcQ%C!*gRbO+aGVppvTSuckS>Z6rQ@h zeH6s)`rItWcjwe9_R1=Umj)ErBWE(jYYYwXe&5w*>YhS+P8-Bk#OoT*izmff?(77!nr7>jxg4!*<&&-YDi*)Ib@`%b3(U@b0my7c)%o>=|NNV2KI9^Q4@dA8;*EWX zdVY6itql5_iv3+}@dn#%UHG*f#reL}CQb6BNEm|TBjPyQg9n2Og1Bz}{6;Z>M7uP& zXnT~1m5~FT5sC7v>~WS7yhodRWBm~%M^h|x0+>EgMr^%)4F8e*-jn3B84|h3B=$Iz zAZM_?)|#|G)9Rb5z}#=#%JkOjk>x3CjzL99b9GwXv7zNt z3%*}c1nWp0U&RL7RLbU89IA_qeAwELJQsB`fb#Q!Y;{vseMOk$rz>V6VO_CqFl5gn zaPqcjUP@s~zmeA*kTei4TumV%Hk`G2ZJ8r)4zG&%e!VQb{eVG=Xb+QU_G_};R@&RU zUArHei)mI?Ox|49tdy@z+S?rqfmO=;P7DTR6xvb}b)c|R|9I&MAv9p^e*cnRBJ)Lw z@z}%x1I_^Y6P&VrjF^ru)uN9{X*n2kd3WY3b_%nN1Tl_e_0b-X{{&vpI2zrVQ2O?8~*`FK3< zk8^gT(O%xn!QDIUK5uOlPhDCcMqWfzRua;GJv7EFdbRtLA9N6g*9i}oA{h*n-JG2O z5OPgPm%YZ(CSTg+T0aGcyxIL-SrpPf4Sjq9o>6_D4axAg+PgpYs{1{#l=JhMEsrpS zxgX!fP?$8zLYF&rE3fdEFswD5?ePd%QWhRQc0Ln=bK^uYscckah_X4MvB726HJJbC zvxDE8n;x~b*S>nqg+1SA+B~FXx#Av6i;?3D=wFeHlUG`s0`I4YtzKot5d4She ze93z8f{o6nUb#v^_c5gVh^VxpQxYx|Qdf>aA6XWnhK^m1r=IO2!nxrJFJI~vs3IoZ zcwJu4|0K7W>`K}z-WPvt;E3J<3E2(it&HiR8evtR=>~*{WA5UQ{@WfGPI|sRZ{dP? zQ|_+PxjB)>a1e(YBi?+@V-ZS5UakCL+0We!p>aQzWMjM{SL|;4{X9s0gL;Nq;45JL!fPPU-4kKTO1iq+4DyfE&*^H=sh*m8- zSThFUtCo)E9o&5}2E>uV7Pwx|g(tOdOr&36bva-1Ju<6hKM)~i1tGs-3&sf0+Rxs1qPn{LMzjFG9W?1>^*3=;m$DBE1T&z~CVm)He`>Nl zsA9C)XGXqUHPk~IcRIPRp%+yvm?yq}2FkP)C_d^y`t2_E((!ARyLe0`Phqmy_%oev>HrA0&{M186B^n+%=VU;6-fL;g zzE41Qv+iG}oQaJ$qgc~u3hfOz#68CN{wg&uu1?k&EY}5(lX`}(Bq0yu`%lkpRWYAD z{vq125PJO9i>ZZyG6&D_^UrB*fpf;n4z`DmYr7bVns{PUR&kB|f{`rO657f4#MPCu-BV?ot`)a)28pMvzQ zh6K**#8dGYt(g#KO!?I}jN;QML@jBzMUG{%MPJJ10c-Lp8I{1bqAlNwX2%5}CexZT z{-`);fHh-gE1mx$tB+W3hjmyap2f!bZCPM#uaf8l16W!#8;WTUyGmr+`J%{OoQ*f6 zVrixb9>sxQnm9!MYomaal(XRuCr8ogwx4D)qizO1iek92L`~Sj;D$z7S=oxZ%Gv=2 zL`k3-ap$?Ps85bEa(+r`dWI(Bm*e#+9iAuSi;LJY?#J)dcz$E~bROV3aBe+K+=P&) z{APB=;8hO0-OxX8y=W~y7gBWHCQaRVr>A6 zvbrSjVpBFJLYPi+vvzGN$?Fm(U zq%%iZv?5jR%K=F~>6fd{oOoAPkID&tzlK{uKYGQ*#Vcx-tl2rCp*}uVhupCCB00Og zRx#2tpSX7HAP*_jnG4z6k``KRO2*sYt-}|DD<8&6tfvKqcA`E<@O`tsW^QC-KNBpp z882yfwd4IzRnQ@O_ap-twi!vD4V{hDWcOWCK8Zi5g4?gKdw(;o;UsVS{w{{LFTL}c zueT-nEelr3D8wg%#5mr*tL?$d*c+> z;adVft~kqfq0)1pL(ej6YBqfak*^>vs5=GOb`|{uJ2h6pZV*b397J(3k)0{XZ1MeP zRH~S=wh6GV`02O%q^)H7?NPtxQ`9x&JFQ8N*?lA(7dXKw-?ETTb?Fa@VSVU0kq_H- zY$N7+vTxRvvW??XP~B2`HVquP{VZBM>TSc#fA&#~`gB{!44G74^md^cjy_Rm7X<-Hj)eOizJ$k)C2yQ}-&I~?&$>qeC@JR7fVzIqne;k8%n$W*QYTi5-#z`WGq zm9&>8gh}aP>ZsopwsQWuOi;*`H<9~(y~@SGhN8btGRrlrvLH0fnNr$&qIRcV^gX>h ziLXm`nUL%a;ge#L<>n)-tC`V%9h+~Ce=$%*1UYgAxdlE*exlbN@XwEv@8t^yW7ibM zGWWPK$=Ebq_M|s0I&i-!+i!Wix!aRh06Kq-9?fJP%{OX*E@K?|7nXiqE#GG_K}KfG z;@G9a#8WXi|^OvP5RP4wTNA5dmLP=f{1j9 zn;_<=LFfzZy8GbnaYIwQm{ghYr%NKdLJIZjh6BERw_mj!-F?yH?Ga#Y$(Zy5quqGxcLL|VRZPlkDZLcCw+ZYK+=8=SwREMyynjA_ zZ)*B(`m#<(;N*8Tfi#-zQd?OH^}5Ri+%Fg1xCf5b^RDn1b?gG=jDBy<``?qvD(7>y zcsAE7V*eWEwf=HJ4K?r8kYkEKD>`i5eZfN|ohUQpuhJx$+UM+sKKrK(bsfuvyQiFU zXhxhI(_Zii=u_6*Ih#Z*@AenI52ETZq$a)kPxN8NsdBWJ679YM+s|jp`RlGbuSQL1 z+%Z~|w%s3m>cWs7QVUqy1!ry17BLmI=VNNRpFiF?0yEG>K)eH}aY*X<aQkgmq^jj?bAZ0DnRMBrx! zhvL#Zd6}z>nO=s|zm-i(a9VH9swt0vSA~o6)v(hL!tO-oE-8DiwtS13B&w>U{ypY3 z6>ej->tP5;x+5w>@6q_%Q?e}`wtd(BF7)=?>dI2T)MDnhE>=G~w{|e@+IOjh*AjXQ zMV|`}*hVRBGO3JdYcZ)yHyt$Dlfg7@veNee7daB1a)#uHhfj1>(vnUboDP6}bV$kK z`_(Q<6550UU`HZSoL`ICRD>gc%o)vJ`6mEhK==RNJz*stI1xI%NLvf{#J@$DUBx-KOt^|9suM zY&TRBbV<(jK?>u;Ye#}eTkoY2&5I2hhxXnCBhKVv!4gJMGPR_Tzz)1p@7rON*}KA* z*So0+Cpy}1RAGCEQM2j>Ed@$+BaP`Y{%W1%#RFRhlzKH41IJ^6!G3<8&Chr4tW4X; z%#HTs3+;niq%upd`<;yBh~`#4LUCuR)Iz@U(7TBmPd&j0lky)UHhf(J->ZLgz99EL zR;WGPlkcPbqgZk@7V8mz6i0c}jlyBHR(VSpI31FWJ7tzDH}HYRN6`r6k4ng95Oq7G zY#7y13E>CV^v|m6=(9OpgSK8lZhH={e5sNQD|*Zf8`y6j={8iEoJ%VsqiXMn-8Zm3 z>VeCA8XVR8&`6R#Q2>rhY`5T6c=>DA3&W}QBUwXeSDZPC-Wd3T{kTE0nT*Pzs%!Nx zW_HxY1PA30CUwd}!NMBAOI^O2deK9u1jl*2xw(1y9=JNAiWZ$Sa62^>ItP^gK#W>sgl99E6jK}>JbG;1aPo)e(2Tygf z)tKAu!JeMod-gmOYX??}j*qpu=_%gYw}VlK)U3mfLI&2qWK`z;Wn~v{EPqE8cWhtSpGP%?_fp@a(*qiljR44@yh>NMjl;rS;~663VtB zlFg}G^m*71KUWnN#>p@6l*WyNzy5Qr0e_}@w$#{b;4>p)^m6Qp=hrp9X=}6nvh{k6 z{WPDM@4d6YbN1Zka<5?)q+tYujWjLQ+(XluTXKxi%fZ3b!R}Kkv`S)2xFB915ar+y ztWIJ#EL5#Hxg)$ub}cDL9S+UiKkv~LwGU}I|9QqxIJrv|AtMfP>tT#N2e0PM-2Llj zfi-tQc;Eu##9a-*`aTnjG(#{&?CC@c>?eYbKE!XoZjw-M0k-M3mMkD{BuMni_O+By zaaWJj(iKtYMVr_-K1EuTKUdAF$Q z2e87GHoX|Z4|}4&b))$A9`-+IS~aJCaUUCHvy=wTV;U1m@gdR9R6_jj7)71&P*VdQ zsrK{q?RqHZen$Z)e%FiKLS@5ETMDj}$GD*reY`#i=7~KPGB`5$d6p9l>#L6^4BzB8 zDM}kRoO=z0+xLWzmptEe=hmV_r4gL|F+Xn4Y+Ko>QM3hQYQyzJZRB0no1EjJjpW_ zGI);h*@8q5^+oowfDUDfNZHIvSyQa_4y=RI2#kun`GEy;k!+Dzn~bHc^5#3FE7PTq zSf@^Ush6w*1%@zaPllIG9R|lr<)2-Y9eC&e4TNJ}?;#32eA~hCxpJFrTs{V_)-~}| z8sjqM?Qq58RWqTivf|20MhNG@1`Vwf?|P=JrHlBPf3vy{AhHM@v$7ak#lpc(Shp9o z8UM_NwC|fW2Ch35tUh6{Ly8dJe+@nZ1z(~V2GnUs(V*`xaGs;#&J#dJ$uH?u{hYDy zuoUN`bB^I+Wz8lg_{!3A9+ryy7^@qN8lv;D=bCL>q|YXDc$rvDrWJIrKBK33e~jsK z&YBD=wO?s$PWp}uzu6G!Tc-YJPeVlA)q#Sx&h(syoW)kw&AAVkmZoP?22+I}C7jf)V1) zFLEHo4DQw?#EPXNg2(D^*K$*Du~3oNOhJ4uO?F~}Xj9IZ$BJ#c3P_du6lz;}!S?4p z)_TSxZ=aF6oF}EAvC8s-ffuFAXwiQp&Zb01bsCk$l9ro(&SdXl{ln5MeB!V>m)uBS zrgmsZcXWjFN{*U2@9RWWEPm|?JutvqzhxptI<{Rnzy874&(>YFol)n)4z1;zLEQnc zB7GQz@J)3hBO}&}+A$tCWhNVNl{pxE98I7-rb(zz3_OrdhJr z)tfPwY|0*l*-8xD?5Iht@NRxCr(oludE0#B1EpKN+I>7F@2-Q1H+;5*Z1~`xv0NSg zrGCUGf4TG@MH|waYI9?w!>klRS@5L=*6o1bi^uyX3D5Nl&(=IM4m>d7s}k<&^R3!7 zU^S*&JzZ2oWzjVHB6pIfgL;+QJlvdiKQMRi+YIcRnW2%jhh(cL+hZHwSt@AwfejSd z--D8U9`0N4Tol74O}NXsOVxk}{J!YOks!;~&Pyg%B=9GJuP2eX<6s(pc-&5M%^$1 zEdw_$P+zd)<59yMeOlZcGb0bplsEU{k3#!tM5&Q$qEyY#!p(Tfy|Zg7+J0T_sh3`l zmX(woLMp|&PW+hb{}i_Z+C&^WhY}REDGc$bzACH}aeUGf6yW%v-tq!za-yGNNIko7UMk`Ko%tx1e0%a=?8}ow%nO+<-lrV& zqYZ@+o*mQJZ`bbkkTca&Ga3%xO4WEM+PY%(D6|i!#FK_gLe6u z)2v0g++iQx=>9h|KStKv$+Z52+;Z>JUV2o=Xw(#AqR^N$&-z6HnygNpu z4B$b?wFfqBe|n=u6r&4&-3;5V22Q`0i~6-CX)$JZ@G#pis<^W8k%ecIE1P_@zNcVs z-vpjtg75mX%o|Vp6cjDTBxS3ZkdyUsjWgr$7el2pksd$aRXrF*e|T!wDy!bjh!~=X zV*`D)ldj$Pa9>$%7)v>DbHjX8zecYb$H{v=$Km$hGud{`>tTaE z&7`Wz`F)T2R}X7Ac@oT&@vP-evox)~Pf|P2UYB6SNWRl4Fy~9P)F__s^&E|iKmUQU zYhj`J^ShuWXR=ya$L7o3Ut`J0@YeRggEMzWV}{4>=eCfsGQT-EB0;^GmTDU+9!`eI z#atYB#7%nI1Iv`Ql-^lAoAHA6eJi1dpW%iUF?%Y+-8jpe+ta8n38C}$?kBRJ@8Ob- zr75HqSMUO4{#502ao6tbVmBNGyl0~WA!B=u!5)mmf!}1B52`*aWR;rV%(gM0-1~!_ zAcFO-y!oiiNe6>z25*4vFhh*LrtaFkQP_n#?X|V}ZA-4B6@)>KA6L*$XQC0hrW#8ZZ6@uj59G%o zfXVYX|4hnt^vTaLj`u&KKKSXDz|cxEwnm|Bc#k=%u{*KwD>DU$`<5vkgXN@t_ zgA!*(Unn$0>X6hpz7qTmc!Jy!ROEiA9UMSW9j=6cNK`uU{n;>zF|OY4?20CY4!jtb z8TspW>4x_l+ENJ9Qf=|>(RIA7Xt5aKGQvOhV73%QQ&Y6{SnqHFHMaq!L6n`?uIn3n zNBQ|ltOL7=-+f&^8`Z)>6S9)vzp>6K#R`5hDrfe4b8#)N&$D)=EP(v;XZV2i<#$8>U9E8>+A`;d+@7bpshpA{&%~a@W$9 zWq=$|;d`ON>I@htDTRyk$jy_Fo88*snt=#CR5|G z%BIJTrsf2?8hrP)3n8G&GNjoOmlx_yVt;5%&)3v+G3pB3kw&3M^FtiR_ogX(3j+rl z`J=Q&^kdVhG^w`=7xgbk4-2}yF%BF4X5dd9*EoQZdDU~_0FLsl3zbaybEf|~u=^L0 z&AtE6?WfJoIJLD?R0GZ%@Vwn;TLh@T$a4!z1Y8BFmWD^KGs@WLfjgyNU1d&7{nLh9 zxnqstPq4su(`iRj7mQ|vq)7Dap{=`Q|HdIP%wMa#QjrlLYbNJje2Akbwd*Qzr*emD zSlgCuF)4Yys0s20+cBOpseUQ+iyW;SFT^ZX#4a zt50-}GN1x>4xvL=pUw>y0ONy}k=H(b=wl3{>}9|WDOkwG?9j!Brzf8CRcAm_(v4no z&r5DciV_DdHyMD`F<~aiDzAha5WE_j@Bg!bw0? zmRsxCkPIdG{6S^E0DS@pR+eR|c(a!MR9g>2Jb-SrTS1D9)V6`ZDWU0X7e)ROu1>wqp6~A9Df}G&q$#@qACGjkb1IxIv5RAV)?5hqkkbQs z3=s%np^Wg-Eh}M+pG=9i&t>>tzcv}(VKLuRCB8_hma9BZ*)Yc0&KDjw&0M=H&x$t- zp0W?LTC_9i9*N>>Z>l42?9wHX4!1)68k_Rpckv!fX_DHO6R zIOL!dJs(J=eX&1g@CFneddzB1mjp1Uagwr-L!xiiSGU882-5O`@HS^vOCR~Oqn*=B zG|{oVx{oYxJSDvOeJ?>v&X#eoU+$p}_?J?07)w=0Y>@e?7}8n0U$6A|F$d_;e3%20 zm`U|dvY4i^{cVAkGf*5ESr4jf_30~M3d3KYCg{cNd#;rYS#_dbn@7to<{$K>-3_M^ z?*yQ&?3S*{<7~O!6wRf%q&hcyv`7yzpYig1!%<$J@s62FdAuVN%J81=koFH)FR3vk zHXc6Sc-OLS`QjwJEPw1Wkq!E!bxT`@Zl}vcSgV}V&PCY^t25pAOtS*B^kK40rx|wp8TQdx z0{Sz9&9?DyK}eCJgo}q1o-23Y5I5j~{A?;R2`Lz*nUXGtLr9YxjY<4D9nw0a+Ca+$ z*jd`99dU)vfJdBzqZb7xR3?otbnoX+E*3Vyytxiwz};n)RST+a^aj^g=D%q-J^wka z_tH}zV>mB1w88$ZtI8aSp(H0fw2LiaD2A#0I)3r8v5cx=cZy2!Nf$P1Q!JB7D_%Dr zPhy0ha+_}>bhSNM{H+u;kf8AyWdT1(U?gS9r%WpRGs;Lu;-l7nri4t@QoZ|NiGnm| z*z5Syc85>4ZlCz8A<B6H_nGqyL2GG=tI?oL6JxAtsd>5?of z(pHC%<*YsGdF~FclDSFNvR;d zVa*pdapkC+1g<%>nSy~bf*$bbIqQX>QDSNWW#!jYx^@LhW|Y-(B19=6cxTn20#R1} zeHdb1p(Ew&9`Iu@81-yO&E0k-jc-&bhn>O6>2{Oj@u<3i ze&|9gDTL6`GRQEy>5eGa?yzwA;#yaPTf{n79}RS(Y8tBL@g}#EYYJkd1X?2IbffF1Knn3-wyd) zd^7X6W;Kl=+mwHWJxQTOAH!(4O-dOKVnCxi7d;kd^(;vW;@!dVIz~JF$rR^tYq=G@ zsM|M7?bMQBn{zN=6b^EHtprvY((~(?TO~gj3omp?+KL<;E(kZ|F>Vxpi8$Bbg7-o) zce8#8mDnQJ&hjy%lw?PeCFYA>X&!FCT{=O1q z1%64JO@8$a>duGWRzNfAg`j}%K?S+PYLnnTWDZ9U$}Py;3l0(AAM}}eG8IW=a34@D z;Qc9nEamGzGbb3dVyrPK^?}8;L3Mm7{RSD?;a4q~a46)aE$O+5t?AdBbI0-`151j; z!|%-t9<}Znop`r>>RnEGbmzv}`Qg;_LjN7v6ovpv>@DvL@Ha5n68dS0Y~7-fR!aDX z$0bZ7DH845c^|!3ayN^H?vCk@s>U$|P!_Xz+Luy_Ec*q#4Yy%%w_EE5^Mi9bOEFkh ze4$m|cNGJroTX4w&cF+oMWQ+StZFaoz5(K@ z^1Z`60fyo6s>Vir5_Do~Qdp%P$q{na=CLSOcLw=WKq>~;2RS&1x#GO7`FB0qmGFol z)Sh&HZ-UH2H0(mmnpBrgLDD%m@OK@Ll950p7v}f<01r*cfCiTTBYv zH?g|K3=RFNIbHHK9~KKbDe0j+*vsE%Q8gg-#U)(u;yv2JK%xKAKQ#5>_l01W`1LXR zk_LFCdeToY@Zqa>fktoEv!`g$OxFVQXj)sY3x$TcM1>3X_^1+_oLS0 zCAB;-JmESdG+c5<@a5}2v-{D7AIS`RefZV=SQbY+Mr|_4-F7MF<3wt1Ty?VD9NTwN zswc9Ni>HL%=fE1q^&OeeZBXk%yF2Yr8(EO62+6ZZNtthHJKJ*p+ch7dv1^EzqWg3A z)F?3Ka*Ess54k!GGTbTEr;*jE8SNT^ihVzownp{Rk|>`3OO8hK%afXcfr_hMw*w4e z<@J?})4V_{7eH@^844NJPAN7>%<;e8?X335j`9?E{RW;pcbw7b%m>7* z?E#*FXezs5#rvgbGtVND1$=S;Lkeupo426J?jM-$xv{7Jgk3F*v184uSX1B^E)&Fv zZ7+ASoAHmSKZ2(k@Z066Gl)3ZD5^q!Tbn9r#%++9noHY{;2{>ZxuFla9SHWLcW7)x ziMxJ$a);|~!g3RMbn_NKwdW63&E!xH32Q{n@c}w)G{)Y~96lzA!N3y9m26!IIrMMC z;G1;@1GJ6m11)nIQq|DNzlENmPftiRPcKnHxA5Zu5wgU^7s9GxKX%D%q27V*Q4Hfj*H6 zMh>VyQI69)ZQk2hA3h;P5=8OKzLbyZxR7n3eMA+( zg?^>F%Tu(Om|{eu9VJHJL*caVUN>X6<`1fp>hYf(p|=5PudMaTAELyqZ97>)_Mman zLb=;$3f`Bi5B)VUG4W#iSD)pZwfLI)(G%p>o6g@GYlyC({cvUHC}R#7@UUT&u`0$U z{$I2sNpx8&Pdxk#X&i6MklXGJT~A*&{btQNU^{vj^Ar!nlXL^CKpZ>ApDFt31>=_z z6zAETsuZRq=kM=6uePH#n}!tKD5{9G-|uYn$Sv$9rq$OSCVg@lsVHW25&L=?XYT z{wuI19@sm)y^fImspdF(c$2O#%7$OshJCdhm00z$rl(&0_KQkBuoeZCP0h_6P!#)L zi)Cudl~ZYPxwlPCkCeDRJHW%kGyY^{`qzvK$7y>SPysgf^oj+^mzD#kffVq-WYk00 z=K#`Uu&fZ0bmY-tDv1$(i8HX}BLJhw$Z2FaZK35K;t{plz}<}&qR@&5+b{3Id0SQa zjvp@EU&>?pe#-Xq{XDhnZ_CT?gGy@)j_ZAjFgb6J+yiG!#&1Pi-BCS-~-shiBP+kY!w6Od{M+5dH!7CW4y33q-o(3?y}2mp!7;wAjscuvDxg1Ih)HlPzKuUeh27rfSTO} zK_q0a%cLa1x%zaYIp|YQ0PNP|OSUGR83uR-CT;bM`|*f%HzYsiUqBa;)!8(Q)H95l z8@b!hXuL0j)zi@>mc6QCWJurjR>eM3$jr-l^~VP*Fa0d`?9=`i34AwbtGHPF$;cVY z4-CF|Vc5H8$RRWI{hjqPFC6b98h;Jj)X96DIo*(Z)imDB8qbl zfj#MtZL>c9RAmH{CU7jj*oq$WAlO!~#St(bLo?L8HjYICRhS}+#`N{su+Lx-28H&G2M+V#Phpf{?bfg?8A9=gJCEZT;9fgm zB)Opfy(^eAF z;fWGg06fWBz9lmHAZH$;GzP*x!&I}B+jw3w5@n)8@{=K5Wru09QC1I0mBN+G?2#jf zUWi>rt*N^s8Xw2TCe1YmxgLDTblp73q3H$3`zGG^OZyY$)bB2l%*s!_=t=GutoGF* zMKP?=)E2u*h0pG)2E%Mx^#AG_IHGrRoNMQdW8qbYN@qfU51cmm;B}T>+-1Yo?Ybj~h}OVgiq2)k(xTW_HVO&I z7fDy%_DN7O|IBm21x_2z^PZd9dr`Le)2H(%qd#|X%KkX_>xkP|P@={EV7~ubthTfX z-kTde>Et4!p>a#7#yr=UfPoE@!#f>XF=jWA`JJengABTI&RA>nr>DOns~T4l=AU(( z?DsSv3HFk%t}^)iNN`R)^rlV}lLhSqLfXx6G`H6&=rDb*izjJawVWMBfO=ZX2 z9UPwkQYd!v|9BVzZ z#zz}9sR`n;1$K>Y=vS-pvpo8Qalhd%XG>)YO~hUb8VtW`qc3E7c3FF=7MD+ zD*Ri^F{8EWfoqT3x9`^Y%sPSf&X3*=Ze24^^F4g|Pw58Nkxeu#G~x+^780x}csf$4BcmV7aZb(JuuNPS&t$AMG?*Oa+->FqmcAs0kN z021`9?e5eCB*Bzh6eBupLYXd-l#*UUSYI3hn`Gb9b1UG(?%tq12eI$fCLV&jpTJY! zV>_KrxRxOlpEu<1_MI z50HVtTsMf%RWnG6%R0zmT7^xDL#_dr5hNNAv@)j&s@i-7Cr)##0UA3oa4n@GKD)u+ zygh=}rsslKa1Dr;pLn@#b64(hK@hASR(i%I0h^7D>dj_G+mNb?SOjxDkBt$VD=`0@ z7d$O$82I4U$&##d#mlT=6n`gYA5oSssV>a1`vSr!NenKUb?XWXQQ7QXMgv|#lubUe zK(x}6^PL~^S-ZE!`UszTM_#d7n1Al=ytuIR(&14d;87s-YRncb8-l%yP+g*eTcN&L zcJ62RJ}%Bp{<-~NHtUyP@AVse$BuoS z-EJhCT2{Y?z$ygZN+q;NMut=_42yJ742&+>QghEb^{TnUq{^)7$;1aq)HQQ(3RPiQ z=0dXWa1O9wz*)0xv21s81AA)p+Xf%0IC$?N?Z|?(olX!e-nvc#`TFHLJV5oX9c;Ll zd*G%Ae7{jjA3k|EoUJQ^k>F*V_J(_g8Bs}t#Q>)@f~c-!q`@klvhS{$4Ca9)6udt~ z7N>sG`Y<)&T(jje$`v=JijZP0-_h187P~E}N~y^%{_#(V!SA5o1q*j32&dvmf=z!vWSGjrs-Z11qa}Rgzhsg(xH5c1N zIUWagL?r6n#q1oWX4__cY5OHV93qNwEy-atb8x)-9j{l06zK;N?mr_HWHGMb{ijSi zl6d&O3A_ndlv3U6EJuEkbJD%B=}JFu{?~Hu!gRe|KTUW93rW*ga=^}2-w78pWep0T z*K5Hv`NesScYtPJWfXsZ#cS zl7}9_`^!vg9;}&BX3EV=6m2`mJb~(@obT382C!4Vf{9h&+TBz8?bra6b~Hc;DtEP_ zNKAH03TI z{@t6c8vtLRM^A-*7RB^p%B#b5!cs0X=X8p5Z1SL@92;ef;y2d&+m5*;Y6DSH2GcVP zAVV{N{-;gCG%V+5f6H*Nld+rQ6&p0!|0kIAr-o@X z&e$Kj1JF`P5sntoB8PRmyFpiQl)QjsKzVIT0Aj6U+pl*kini6DZzGCXR5;Rb>3Wl@ zVICAA9&Yl3fnjr+*{-z5=5aS0UYfz$f?eB|U;w;bG4lTSn3j0d@ps@TRD7#ey}RDw zg7pTUgfXeU&}Hx2@AVXmm{+pU3|9v#+>*JrYt5&tt3jCpXlCHN5s==0uzf^}D|~)i zj`!A^at5B+-!^X8li=vFM?)fx{vm947Ro=hI6Hen@A7Y=S~n>!&hwgL)=>lkHfJ}C zF_gYpi?qin7;|#a(p)lxP5aO8W(yHJA4_r8AjXX{2Fg;6WVBo>iUF0grB#EYVj&VICO?RWHvpkq9U z(uEagRHN2_Qo6rbBR_pHUHzTIQNU28Aj7ZeEqN6)p=UCr)z=&x0-c=EwPEsMQ5v+9FTIH#kJ)07+7c?VzH9tw)j)JZLPn3l6g`P_&K5L)aYvv+AxB5r(Az|NP;0OD))8!R zDD@Q^*Z_3@|NpM;|75!wze@qM8c`N@-a~?OS2>gW=i}mtmmiZ(CB<17@YJ5^s*KC3 zVFK$r%~jn*=j|@WV5r$PGobSZSj6FN%t)jFv`fO_a!{VeL!N`Ko)xr>sR^PXe%GXB zRc(V0qt=Obz$d>%jWqvo{MVrXndGiB*ucVCpiita*ZzqI@B)mEvgAW`m@x(>4@2hx8=G4(?l_3Sl2P7 z=RH;D^F+7soHy)jb(NT;o&fn~gyb095icM<|D?fZ$0TyG-Bk@bEfvc7V}O*4|*b7$B}mkRHgOqbaU z_GxQG77E?R**@u6##OcWd+rTZ^28qj4WH>Fa<~3+q(A(fU**Ap`}Twls)M$S92;*^ zNaJ2sQO?_H*$ox26mObemqQH-T|F=(Tir|qj2#{l7a zOW@Du`g`Fgn)joD*0N4hd-P{a*i-AszPeX;nn~ifH%tZ@eK1P*LSYohZrK}_zJx9X zDl>zoya2`cUyAxu#e!3D`ro0ljZ?OAKWk;XK#hC(h7?I=#F$jvis`x-fxLp>HIrXm zytTiXW(_rcim1WE*p7*m=s7p*LDeqO z(xK~!lqtXV>m+(^*>lv$RmC^eA6hRLjF=OPj21SP8n-mhgGRcwo|f>KwvlH~7;-Z- zqe`YU#!(e9?iBoZ(vxK0`*14NzlGQBV;FaZ*Qq|5(V#-BJK*j}c0nUmSx`?Pd7WXj z@)e2cco5Kd8c&g3PmN3vfCBq(LgoQ*?KKD?^9S4*@@O&%Yx(BZ-yJ(UPd$D5bh0B| z-bCo<$_~knmY33=&S2E|w5_a83=;h22S(&UY{SpkJTXZrB?>0%kdSH#SV)^VuC@_# z_{{s?pUcjhd6OKkn*gfXhT&#mTCQ;E+k+&yAqAPBD2X}Bab!n0w%Fo*h7Kv%1hyg| z6TL5d8`g9~GpY|7X!mtF--x4E4sF>MshLT*cGD%OF-R2hf_0$%MnY}Cl9$8AA$A8@ z{aGQmA%3Gg=>ff3bJopyazgX|@(Ru9Od{*Wz~QIW*bNpbzcu_S()Duwv|CnsTWgD0 zXiuNuUKEnoxWVYGp~+yScdlU{=Q=vJnf>_d*E%{dUiFT7I30<=kEZ!E?JM{xkiMAy*<*M}rS~6p0RBE0+vTt{W z^iKm1$|y45hICI7Q^0t#5lVm&Sd;$BlDYSOz(KX4-UNaJXe6T?M6pEwiimVfp%3W+f%6v&@>5>E}cf<)=8!8p27DoBiU30Uo;cH#%6 z{Ko$L3jS7WXPx+!L(A!xyDD>G;g_f-MIWkC>k_x7Sm_y?3T~x?uv!D2PSn@8^`6)M zQP%O8yBRBh_?o|K+AY^O^{y(S)I2r?0xs-w&)dIi@aq>mZwT){fGaCR56c}n5Bk;n zInXDH2_5B@RBA@u~1au4RL77zbV6fBReo4#D+L|gdtqSw44v0Wv=6ZG;VBWqxl)?2t319hR5CE=` z$Z#b=g^G@w|ClEx{?|qm5q}vw>&_D`R zYkj$?KE6V(bh34-kEoM=tLmTkb6qnQM+BRyCZN$naeMo%tbn$*7E-B~iK8<$8o_+? z%{minQ7q;=L5y8qZmM|J4ve8{ikLpm049p*J*Ivb+=&fN_4WVrjPD#xymV!j?fOPm zTKXf`!^Cdj1xi(6>ld`W?6IdHe7#Q<*ZJ%9Z z$BkZ`>Nxom7+c=7y%AbHj6HVucy&l?T5XsvxdNNt)En|y>8RXt)N9Afrsrf83TgpaQTh2+`g!UxwvC< zX!~syXxKO0BDh5`u;p6%lO>MN%3j!Cuez*&Kc!+r7f}IO`iHlMH~((DR^#a#Efw9H z-VH;qPd7F=%f_tQbJGq`Jv`bXSZHX?x0Op8k{0z@?sb_}9NO=1Iv- zft+fS|GLfC@vIi91egau>AFyJ6~8q|R-)ADg2Q2S`jmd;uH&Gk|DYE`Qor=5JI9K2 zFo1(YI(B|qxj%7*zf1&^e)#Y*C>=4lZYHg|o?1&MGy3Udlo3`;8x8gNU$N3tjS=b& zqG#Hbi6rUU3FdL{kJnp{2h=qDRHk+DvNcv)^pjXyvhR4N#BkV!w^D1@1`>Fe&g&m2 z=jPnyiVNQUR?~g6CDa|x9ci*A1)#uDAI9|k<`gpC^DbWaV4k(w(W5^aq+~RZ9pnS6{D4KW7E~I>`?av zS!JU0ZA26;k#6( zEOzw3sVn$kkhkvzVyL^TYg}Oa)+*s!Q39zNA$KHS@s%g9N#@@ zOwxklDnW}LBA+Qt!w%8o)Dacm>umuf<4MXIrrHo4F(ISskwCHhyR6=Kqui%tfn|yZ zqMN#?jy_{l_~9+Y<8<^xfUB}Ttft!BZGjrcC-zuMG+8f<9WwM@E<1YMvf8ov?{vn# zH#HXN&U|K+B-Y>ab4^gkk~3I^N820X=r{-|0$VtW4cbb@hLBOEB2R``)W2fR+B;<- z5Nkh|{fbJEiB${wz3N}Ha>`Hjrf=_c0)cQpRczZ|`@a^|{P^JSqA>&VHGBUoz^6)I zhvO3)t!o3$Fl7rwE?Tg=n4fgi)k0Zjc?3{N2Z@7}BVQwhq<_KEe{%LR<4 zOBi=T&cd;!!+#2Z|Agf@$Cgm%yzg#nyQ?|8W-t+KV6#Y!3}#x?txS#?EmuoxoXQ^2 zKhRxpWd<6H>}eSLvA?6g(TK($dVUUlmZOV|0;LNf2bAYRNPq#x>zTDPG*_u0btge@ zli&7yWo(AnhiN%0WRxWWqmIrBM9Rm4Fi`qwmbIcaMNr>TXn!B^ zFbd@Ka2MPk7Ut#c%VPW=zRo-x%D#X96H!P)wk#E63n?;og)Cz&WX+a!aFI2VBzyL; zWtUwMnK5=nBxGM=1||CzA&T((jC*;y@8|jc`qNRz(Oj;%uFv~?pXcjLO+6EffAjV1 zX(b7ku}-eZP#T%FN9t1Nn}Lx`MZQv$7YWu;)3^m8WKkQ+!J9BShB+oa>g~eXU>a88 zz;bB8#3Wv?e7F_YU?2WPwl3%@C0Gj}Zb1X!aeNbqcf_kSd4P?&D$U#JmqbA5R~?Th zCoQe{Y<>I-%_c5x%+406@L;h9Kdi4J(snT;x=>!@IX8vS=|Mqc()W+ASCJe+F!m>2UD4X@}BWh88xiL-9voUf=m+?jRL7%KkdkG`~iH>}zhu1zYh- zO(`V)n@GwMbiBLXbNUFbrOIHw)F()ugSu7Ox zQ5pSKes>g%DiUA^HYvp#Go%lx7R=c}pDG;T2xfKFuvLau8kVl8NIHSmZ1&_ z=4%?TBH<)DpOiYdQatmbJ}P%l>abqfu&#Ham13=oPH!sZDy9zS-V3B!kvX-hqgZ?E zMHHLx)3gaGx}*Iqm&d+k^&{rxL|UT(D58g@LVRVcW~29+;+xUz{3vcmUu^VItM;=8 zKibx>tlX!P@vo0@Yg!yRs9h`#aVRY{;|lxXzWsL1~|3mTMEMbR=_h~TE`!l{v8Hn#c`!Mv0`?_?dt zED!FDkZDiazIUBmr9SmCXu_Ud#6yG@!S;k#-DLLHSmAYEoE==0j) zhaTqq`O=U)867u-)TWM!%&YULFuyZib{~hg<*x|{3txTs%iGV-j|ofyZD;*&XDSId zY|@z#9>&C6-LEdhMzukXhtvS8bVgYD_t7nK>4Ss~PrIuJ5s6jJ-8law&u?u07rxz6 zKiLCyX6BCnhamtQ_uC~OAc3J7+_OKYgXT{tVE%kC@dlJJ!)1hSO zyKpBdt8)mKVIurQq$nbE!ZL{!Ay2S|!43r?il6*{Is6RxV2_5B6Utoqlue-fkr*?oYrlY(orFJJ93sr1>O42;-KhuE@dS4(fCzI6o#LS#+L0fx!*y~9|S5i4bi>FWMA|l|2+SK7j-Ey#@uqIyB?(P-%lWHspllj%4jp) zspqs;rOnM}oIaj0xR`-&VQWwVhdA^!xAt!u5fu9b4mbu+KWBLwvf>&PoK)SCj@KOw zVJ&U*dq=@+5NU5>YruX`X}zN4Q3H$;Ha^o%wu%ZKi2feZKviHOOSAr@X(PoNHB`WO zlE4E*kk+pUEDp<((UNh!Nr;;q$Wkt39c{atcM-^9q*S%kaJMS?ZY={V^-SEQfR&Y% zj0LA_E6&xEhR+qhbw`{PSuPR$%(&B41Aj`I;BY?hqWGPV9tm8A2*J%FmH@Z z(~oUnmmm<6L0$*toS}i;I{2}!kc_Ad{-}BPd0R+fXe8)x6c3qen9YopzX zZKpvbS`8^0oUwh|53D%qhe1u`3&PqJgx-~;`9ba zA3YYqE9W8n)Y76c{O$OK{t#=hdQ^}kSbyn9qAPhtGP)`}AJA*-JXOlpTJ zmla20`xs(-<~Ou5ewdvof`gd&3dGtrge!r2JX?^xGUdzmb%8#ijmQi+6pgJ<`(=Q%r=w-u} zG2io+B_G2{@NN#Rcey4J1YXHUVQ^wL_AsBW+4;Px(K8@HN73xKWPW>1X%!r-^?d|I z@@k^M-Vo?e#P5q5p64qw;06hx2aV9#@qbQd8de{NvipXN*Om?m_1izoc?fs z5>F==kXgXyK@%KI1dUi7xNg3&-Y!VAFVa|TsCXv-v( z2Hbb8Qf|*ch_-`k&fyha!C{IYT1cf1Pf5FSqZs6LjIOJv+1myJHVjCO0Ebb!u?qNW z8V=x!Rl4e`t@qzs!IKF=kr;UhepfVRIPQ_p#%#O$EHPwB&Ph)*nS~W1+U^7*;Nme; z6Bt3^>2t5G(mz#2E^ch<6(+%_=~Vrj8hhSrsm&!X1#d6QOb`lyq3Fvmu*ZkL^0|P< zTK%RN{FWKE;vc!+r|;i)TP1|-9kR-`12yzcY5%HLyruMt6I6fKLlzuq3^L%Om<@5T z!E1++!7kPL!>p52UXOtE4DLfYYjyWK36mEf%^#tF@7IT$;LQv?5HvG!01@JbTb5+J z33ws^gOd-D3(@7iJ0W6Zg2kew(sVdbj5FZt!S!U`YD`vS-_n>vG8G?tG_k<1!gt%5 zFD8hKN|an3`s{SB(g(!9o0~)Z&JhthZ_MP9uf`V;eVW24^XUCaHO@@ zL`&8UiTg2A85c}4UxmS7t!V2fDP)%+yNnAh=<7#eAAr#ZZ0QqaZc4AVp1U>LDy5N3 z(0|?uEp3AX%Qj80;ca7e(+a$tEM-1Un;%J8vahR}0{uys|c@F9H< z1QRQrB|BKU@Y(6I9hiWV7~Koa3YHJxk-F#vECu{e^Vbu$f!Z5@@kxIb5)$ISKj&|2 z*LlmpG%-FtF!^#n0(h9YWSrBsapC%gXzur#;D|GO)cqZKq1RT%J=~+EeO}L znwkc0o|}0XCF|ot=+g~QgtZInz5l;jl7=Rb^RVQ_#@t0<4t4B#cfwO{Bp(&*T;tvQ zyhuOiG%y8IlkdE)qV{5Y8u`|vLJno@cRRLnO*5`!FL{{#$>2L(eBSs0YpO3W%QbHGe&Ok_~gd zcm#WTdA~`DE}CDk>{%A|^J~CO(#!_F75il}CX9e&|M5~kj^o9Dq>?%X_uC-gr#-V9Qu~!j#zvyJ}QW=1e!Az zK{QUUtKOG*ccC&amj!Z+WGCHqyYkT1*{eXk!i&kh@=hzI&+}g&{r8ya>B=ngAeeo~ zW(W=h<^R?@@Z^P8zGqg>mOy0_KKmL&WFi-&q9GTd)J|t6IJ2T}uV{>T*y4Y=ZvIwR zzuCT?J@*7Il1)BYZSR^Z*rh}-9zA-*KhA+-o1>FRxE`G@rZXZ91fZt#>GB7w6 zTG`OhfC)pu!rVUIcBhusZv0;Ak|E(rBK&zW5-=sU|0@)_R@5wbpK#4<_6SS^8el-W zsnaQC@j8IORQU^SpVR7eZj!lw4Q0Qn8>Lwp@bN)JeiamatWu~BP&1cB>Obnlk!N|? zZg@Z!yz-CU<56{c(9=Df1xM7 zrR944{W#qB5L_IIDL5qhoX4u_NFmqFG}rY=oZyM6iz@@wa@vl7)R@&SgmeNstFRHD zX7u%yeef`t`Q2sthFfhQ|N zKwhICoo7A>W`gni>r`PwA)()Jba-+u2QCwU-APMSx!`qBj{wa=-N1A~1$2rqdtW8E zW6!J*1OR6rl|_oXp66jvz>}b#7D4wCnAQ5FbTo&xz=4Sd2wfGWt1vrKRyx%#M2bdf zX({k}*5;^WP@iIE=F>(qnRy=(HVslEe9i=MYAz-8x7-OJ^`qi^hg(z3srF1TiSz2ktjp7q3Cs&G*wHwTiRD*Q^6e*6L4vwSYg;w0n!yj z?u-!L9cNt@>}B>0Q_WOeA3w`!CQAa*Sw;n*5?a6jOndgE zEuh%{7Afqje3A@KjMCO|*5;%zW{9*Kn0L4!lwAq zm0t&6)cwvr?Xwe$X#OCOXvDw~PfG~mLdM4-b?bY~78epP%OD$eL04C@k%1TOPW>sMm1AjVDtgw)piD^`PtBe0O~}9=&EjsFOW0Gy|AzP@y#a4;t_=7v(!)kC1jP-?x-oD&^-GC7 zzFN;acoRc}2qa+ox_E>jH6M_&{_xdYYTvH0#RFlg&Pr8HAAO)mS!13 z%~BP?Mk;iybk8v0MhN@x*iVv0yoS$FU=zbGz~aJw5esL`5PU9H_@yL&wbPvAcPMDaPW(2d^1lSk(J zY?J>urS8lOAVIXySV&N^3V_2ILvYt$!k=g2`lb4nOS!<-*$pvMgsG-RN~LR#p_ zpzR_f+^QdkgoyVDCm zG&(Q*;^Jqqq5B{XH51-zg1-LuqRhX5nEYLWSh-Sfy#xF~BNIv9YNy5#J>DPk7_PME z(3PsvhF6KdO8fvpHj0o0&Pl|rQO~U>aP{hVYXNukvLay7IFO2vQZ#=jFg<1oznG&b z6eDPMUI$qAY?%N^FQ3mCKUTt(n1FL?=zhnAA6!=jOEM2!c%EeJU@CbrKX6XI)@Lo| zaa?jz?TF`kh5#G-x(E~!ltuEsd;P2PwYQ!gFu~qe2E4Ji%Ru?jLZfk-SKKS13){M$ zV8ulvXE=sHjup5PmZK+M{uOvT-0U!$VWWzmJp2g>oz-K`^&#ZGn{D2(SrgS(A2*JVb{y`)n3@gS+Am4O&k>#&XtHQ zO$GKirSjMl)FwIF!5jktKK8`t%wyLbtia)31}*!&&`1XiJ~MHQAlb)-7Xbuq2KoYu zrT6@dEHAo%ZJ`+K%IGqpOjO*8p^_BIeT+3U1pLd6*_Z%oBtUDV(HCGmZP_V`VF#Pq z2QA%{VUDDYp~(U{SM9;OiC++D??RARfTM(gT(Z)PM4zI25GKk9bA{8%KhHz83qn#X`LaWS+yaPR-$D&_9*63h3W1PZ`lzIh6$vz~1 zI4!gD>^U)4Xy)+#?B6m$e=SBk3BD$>HV5Ng1gO(f&coFC@}e6GCla2b->^ABvHWNOXZWhO`i_-@A-s`ta$w| zPe#MlbhwdtNghZ`{{LMU?;_wUZE2rZ=C&_l0VBUWZg8##mu`18;RtN2!)n?@qxil$ z`o>~fSrHWMtc3<&w&sc*3naRazAVsPE%a$opYF*oAaBfs(0a4)^W< z@1iKWXg%s1Tfe6#D9)S>EUm)P%`_Qp`#=@F=2~&f9vhYB)f*;i22UwuK;o&yc#(B{ zXuT8UJJv*lNj;fJ*CI}+D2A%y6@qzIuC0SBF$qzcRr;nfKgv)=#DD=Otw+4sTQ)y+ zN$DVmXVbxrE$7&GrxHn=@U3U!v}3=X1K!kp5Y>`Q9gSU94M#AauuvrN9qjKXLBQXP z8dl&`@JSgNe!xcmYGX1B{GvfC%$kDICM%>)^xAZAws%bdheVhUkp{6d@|fX7S~2nI zpLjVY1>ik7K)~LoN6qzNF_6Z9{4QW|SjO@q8|ju}et_`OZn|6x(<%C7Y~ z)%DnDv7A+gTl52Y7YItnJKenhFRO=QJM`kwA4q=?V= z)`+zuE6`7@D&294UUZ4jo18`mE#UdGOA8lhp9$$={6ZyeAax(3#QYDL0Qrd&W^q)nA8D& zT7l~w4f2IQgN%LPntTtNH5@*0%(df#FtQyh`&MV2!u;qE5B)n*(P+B&X3_v3=I$;! zVnxEgQi7m^2`sNZ+^$=;hOH+hCI%buH*MG%n=|eQw*+_`H{@I=PU1 zBrYzL3B39H5fjt`RvdOfo1U~t5H$xl%`f$#V+bKqX-7qoZQr)keecdR{bR*q1xOcVlIY5Wt;#^A-Vr!yu(Ds%U2yMLGI+N>7qyeq z9`3$b!evx}m~l!v(G_ytPz6ni?2_tl{g$N$0jL*W%2=y>f(m6u7u-qp3{df&<&kl> zwtnW(F*WVd!2`FGo<{+u>Ga;u9{(iXw~xuF*65Tr1pscd4ceGM^iisNSUKOZw!9o; z>}*nA`I)_KjsnVR!?Mt`)ft%BII~LZyitsGqGeHVe2$XJ+049^{?Aa+Ir2~(*_Jf4 zbcsBICT3k2&-kF#-oyf%`$;S2+7zr8_w>yXX@6jDDS*bhO%54cnS|l{IqjW7KRb-= zJ9S853PvzMaOhI}ONbaGkPBNiKeJf}Al9CRhp%6AJvtwpOWIzc5>toxnJ&U^bOekX7> zJd>@zmQR%LmK7Bhm6fd`j95=`a9r#OI`|QD@W@!L{yOcR8lec!(_rW0Y?94b1!^iJ zHkv{TJzqEZOf-KZt_ z{Sbz7DSL9$W%wGPF{E3k=t+6P#i}&nEX_vsWGn=z8#TCr5}8tIf9jTJL!cGRDKz*pXS2q9s|bHDOyOX z&Hp9`zbDZtt%C<@O|3O##)0HXaznWHPl1FtA$HAr1-R>%ta#HS)2!c**IKomw8$)H zVvHaIMrybRux-f2e>C$IlBRyMb9N&5BX6@hp8R73kh3L`4igy8Doqz;GqA7CeOSTY z@|k$}^Ow(Fp?DQj4}9G-DF;W}r1?G5M%xEl4W&KDfsj=ki1eRoD1eSkDzW!|7<89l zBklE@EyG4_M#Qmfoa~Mc$$^#5U#@uh)`vTW^3aO^DIfYZDtVqpunH=1rsbK!34a`^ z;cmD7;;fx^t7y2zmr#OT48PcXh-@B2-^9R;atsL9PsQIs_A^yd8&cOZd(qdLf+~`x zN3Qo=hr@rcSohU@S2CGoy(%-409yWg}0!-{s>#RV@M&JuQ`{uEzQy2C&7n z;E}lS1O#YlfCk!heB2|`sM|#?6e|S8&e)GM`t1S(i&qf0(N>8|s@(ZL;_ z(?eegIPfDwg)+d!GyL(i2%wRP->m1p7xz*|1V~za{Rng5lzTDd#3GXum1?I}tO-`{@o%jvjePRdE4Cyo zw-2hd9O(I>g6dFk=4lji3G5~}Iq7%jN2$8p@MSI}QGgbnhTD4|6qxeB84@`mmy3avKp#cIChT+&n z>@iv51yLpZO2&}^EXui9oF|Yv8mNnN#dQhQZEy4(Cs>ECILu|x36@NT&B&g=lz@vn zC91!kPzpG6bc`FpCVv!3+@`pLPJ;X0Buu-}<2@hMG>M%2p^ZYJ^ljhm=jE@ZeE&w@ zW%LXz(eKnM-+r^}@cFe=282^3Zh`b1)D3j;1yAs*Ah<0gYpXpBL_Z*ign&M$`SoNT zM=DnI^E`tMH}}qtj%apia?i~2)pM1eaV0Ai`}?4;i){XNCV%URL>~Bj&&`L!p9y5F4JwA1sy>na6~rrv3`y{{|qZ6W~d7FxOFRY;YZw(YE*DH zx&ck6SLu>#uulD9F!M&8*$b* z9XtD=_aCXAqp%>;bEZz+(c|t3ut&w|I5F;lW3uOy>Y>g1&X91b(%DUWfKJ^OEH1_Y z#vs_VxF3Qw|G0AwFVeYR8N}-z+9eYlE)9~JTc<7At=YGRTRZ*6TG%p)d%fz4e`>xzns0_5%h9`WO)A$*I?gW6zuPI-2s+U88v8Qz8~*X=Ms z2dPo8OB3qk8eiToL?4Lo(t*CBGIs^SW9&nm5TGv6zdC>Z%q)*3d&;>CReLoPV*Hmu zy7-dN5eZg3&&Sj}PLjt%%BM`h=+O7;dzrsbjNCl0w)-EQN zCv8q_ZG?(;b3Bft$YtL*NmP0_Nmsa*^7`(TDJRd1EGp|3>C$NbE&KhNsr;3!vZQ}1 zE5%=hpU%ETpO!o7aFT{y3hO?nrCQdFRF_M?#3c9n{N<7KpHF~0MT#JE%OPF}8Vo-w zLGT#*_Nio8SuLY0-ZPJ`4-w);Cvq<|G&`T#N!7w?%PA%V0)O>xUhVKGtoL2P-U9WB z7Am^S<=80T#+tx1BXy`?kK92(bZ%4zc>CIy!*NEG0}GvkD`c1&d{S4W-5e%?VYi6J14ewK&+?bSzZyJw7@_$N)qJ> zs8-xifhLd&i3#y84}{vt(?NhxEa3M#Xp&+@Pg5`|6M#coAu#+{yP?!utEbfAs>2UK zc3#3kk?y2TI*?MOSiwy|PHIP*Q5wOnp{9mz+vC0ZlR{7L3t9LY$Li~YMj3yZEk&>A z|8Zl09sknc;%lM-feKVdWXj_F&Om{4=OF4$4^Ix9`>GL$&;L(1#(%C1(I)k0C4mtf zq=X6*1e4^XX>7I7w$40~8Tbp0_WTwZ7v_3ajIvuGQ5BcI;v7I2Y@1X2PeH+M0It1( ze4TU%UH!96`QZ(K~SU^?&2M>=c zlg6WSu+{Xg1IQPIf_BEDC)QpEMj-8x3YI2ec0A9(cyN`l0X(!wh_Oq`O=hyzBs1Y_ z+=p-GBYUE(bWhhpHNVk_>9V5fzYRpJiyWF%n)(8QMV4KdD&y7&zF%upI5yYT?$)2C zHF)x$I1IfOdTQm^()_V2l7@TbI&cb&m433Ci=!^QAI|{4fg1tr*uV4Be9ufQE$68Y%S0Lh~q zjvBA2Jv)T(JVI#`YO#K#&VO-`zjrKBnG2KrL1SDe@lp)7mRsXW;p^)iT%t1Ej)ht_ zz;>+;H8qET^Od&6!ZaW@rXV{;ua|7)+Jf^K@KjQ_w|v?39ytMWXh*1r+sYr2L*+|K zb70Durad4I;f)_o+!^7#a6v6o>8{Bdc-g`p1@!@zv19NMz#m#)j+vbvVj88V*9~D{ z(`SLC@9J$&4v%j|rZ6(HDE!$Ou*QkhOlP`xVk;q(0l6rM>;*vOlIDWQ{v8KQp5>gX956RdZIPwi;qQL^|NNl9Pv((F=a`GN!%#*~sSN3L6}Xnw}NexO@E zrKh+T^S~G%TONkEV9~#>IwU2s)f6&PIJx8n?ZR&Bys11Z!8#WB=oaWYRB-6*>S`cX z08Q?9-)kh5_4Q`we0c%_*%V;Q2j{N|;LhY3>7W!&UAl;1anb-ECHVXjop+8DLD=X5 ze!ITQiX#P|Bt{^Q@&2*$_!zAEU)s(ZgdYZJ)sr+7Y?DWFNfeuN;v*b163$8p_-YZ- zheJ|D6txsIJoNOetB%veTyUJfQ@mPo86N6cQG9;#vW=KQkj7R3PV&rice)&f$E0KT z&Rv4bl$N|34IEI^Ca_T}t+fZ_j$M@mKfds+sC$t*bdep}h?;r~s464aD8c4Dd;!_z zS6U2=Ri;~#yFnXub41h(clbP{PUHGu^h8;WhX}cST9l9PC6!efOC)HtSqTPYQqx2K z$etDQOB@a`C?A~GAbG%pNj^GDK7@wHNDI91=-lN+y)YS=o$GRX-EUp7bp=x%otKys z51#6o7+J-m$7~gIR3Ym-#e=qV{9tysKA2_5{w^!yu^D#*EC?yPJnwu>(KS;aQ80MM9$ zN*gW~Q8o$(l1E1R!;7BW-K0ab!|Mm8@!HfvSMb1wMP>gb0jVDpdbn`9o%lItFsJ$tS~u?a4r@%Y8D*rNIbZO1RRZw#`K>DCi+ zftA=4-CG$DI&@w2IVR3#$ySTN=kA-n9|XB5!akfx_4+hOl`tP#rAmb5VY-Fd5sGm%=Oy@{9nRa|WWSNG3){ zx0n0IaHeJ9g)#%x+ky#a6=_8hIPJR7N@x)P{ri`qB&SA@TFgj4g1#G z;Oa(FthC!jK0#dhaSC$RQ}N+($|X6+&iCeRS&np;St9P1l>A%=#nD(F(E=}!~g)gr}rVkR4r|A6cn}DvyQ~dP-gwI`GtMFykcQb;X&KT z4l3R30{xcnm)5w=sQF~CHtv3U_5`D0^_5O%0Q6^TAFnp2_ry^it&O;_}UqW>}2uTfr6%8kN_W+_#P){GiPBCuR z`06?p}i|?#;L7WXg*M)bv0uWtE)?NUKfxziCt#aH-}O7o)Od{+}c2Cf2Qm~fJb4tV+tb%?W}$Bnn&NQ<5d9%uHe!i zp@2l&7R3t019spzNWAPs;iL@+P1&8MzAKf>eEu70pMuu4>VFZ{sGI-6$%6$M6g8H- zj?SOGv*fe?TKgRgvYNmQD3-3)(^&Mcgt6b61Nv+(c+ z==gQ;cBX(S zl&NNFlI9Z`SJax?YVY=~TRk47|SGeUwKPXI~k|nG{zM$1pbGAjEcf0u^V8X;dOLP8-i5i! zs=4-x#n|B~c#Jv5QhSGSSnDGO+G3W@kUsOAU$6@YQpjD}C2Tb}v$=$V8)Z&LH(foRsCo=S8_` zrA}ePH-jvuu8nML8$w*Zv{2E~exQKW={t#IliUz113o_{BKuP!#Rb!|3`tYVLeNIv z;VW|fFf8=}I#DG%_#M}7%MknYjj2UuZqEGU{7Id<8FVhoyH;DP-3hS|R*!zn>Y7Us zH?_~oH*6gUbPaf($4|jST_79RE@_#{DrnNw`l<%BGvOGY@NIn^r`kfl8FRz6a>V!s zfV1AwS7Jp8ctLQgdj?I}Kf0CuQbq@}>mNHB`{0!i%b+EADcZjvEsg@4UH zExH97O;j{}_v&}Q))Dc~Rho+%peMOR)$O|v=eXD?s1J@HIJ=zRA>`Aabz3!>hdO_} zrzre2BWzO1iF8Kh%>SNa!5E8g{Z*ZgjD0ftLB!TxDnD zP?4`oMd-czii8K>tAQ#*eUS*j!0NVwEijT7d7<($9uY?dyBl}}v{8)A*xgWZ>0;87 z`xiLgkK3M=RN!gC$@n(~77w!@)4pf8Q8M-c8yH@O))#+IK0;^h1qkvTCGmLR+AQJX zwsVYRrOLVsx74zfmy^j0YUQ?WlRtUnbfG7-P4md=Ld>f09~Iv9Yo`L0e`CY9XTWv$ zl@HW(@yZ;VH>O4(Yq&khBG}nXU}EoNy;>*nG2I=_83p_S6ZEyiX=xJ&;Olo>>v}mh ziIK7sln~YCHA0D!gN=!Tr%Dg5WN8dx=NZOxkmsYwU1Aew+FKvO7R(+=x!w3uYj5NT ztrz(v=Q^(Q)oI$p85{vOWrf5Ab);zI?=F!+8{>yJV)gIItVe6iPqP8U=LjrsI$TcL zKhEf_k&(AE0EkjET27BjR>I?cqr!t!781|1!rgQxXjHvhPVKhb;W{7wcgW6tc=_`kV4aq59AJ*n(0J(mMgrI;I#NPN4#JfS3| zlN-WZ@-sVZu>xKmlRmev@M1$W-x1iM!kY)>-Y0ENA43`$SwG@LGw&!Ej2Y>xcs{lb zGF+HJkDXySSJkHU@UjwFQTG#B<0+G&?U0$xd%Q!ieq5P=jhN*&Rw%oODQSl%v9`{= z&>59RbN0{2CeH#@q0fnOK==$#J^GSWno`)@>Ggx|hd*=n#_TqV@LTmp)429(h@%BA zeJd|ld2(6ujdZz{}6e$?eb z8Sk#$tIz1P0mS42c!~)mByyS@>8Pva*8*ySs1pz!I*x|T|Ry7vIr=-GB`PH--KwB(yhDPVLztWMARVRw(v63Fqtx$U%JW&=2I z9=|&b=NocuFuv@oN$99ApikYf&W_G6Dqt$FjMW~VlueIKDDd{(6vv#of!CM)jHQm$ z#?lLp43$-|0+#3DR1k|w(VXn@{({;;sij(|?Y2*PN;8}J3!oTm+Wt#^+2B2_&BK+-}uB`qy2BSk;1{1mQK&iOI_H)O3X_hHLw zao2T-6+nXM%zP{cbh01B_LJ94nDIE05^Kh3mSX&WS)H0+yRUdcgxCuV?ybA%K@wP4 zc%KPVw{Hyvr$|sSH2T5YieBaFt{rpW9MmDn z8PvX_Ocp&a?qTb-DhhOf$@0`i2i4nVj}w~7BXNG?*+aQr~s7nn=*@0 z%7kK2xK?ol^hXlI1=FQ6lIY4>v3FM?HMSRUN8Jn|%PSs7#9HJ?g(a{kSNVq=<#I(; zN6C-JuSQW*TzO4Q&MrSCHirX$s`sn&J52t2OXf3vZ>>Ch3x5NnS9iZ`^p0yh$l0UQ z)31xx7rBTvuQ==Y>2g*J^nc5S%pQ1s7WSg=*S|lh^!!L*8~}Ue_{a#aZ@H@m_j^gd zuRiHRI4VY+=asx5bLEnS9n0LPGqBkoKUKmag;pnPW5i_^%YyS(lCv+x@CH56;w#UExC zMzPeeli;g$d-$eND*UM=0v|bTmESjuquZJJ4NzWzal$Q3 zp$wM6E2(!b91n~r#7qhiycbNgA^(6wR9`pTUaRvWgHL!bz>902&JX0X_XRxXSASC^ z2OHB`+}S?>jv)uS{e3`1` z+_NF-$KxXXIvwJ#{WzU3aJjfWI)b$M@;GND>)JZ&$eyA)E0`yeaL)0SMwO>+5!ecHrrxdPv&uDlE4 zJ6A}8Igaj_9n%9i=x{4B%{Us)T=gKW4Fb%{_25@Rp!8>9rRGQ1S=v6KnCX zY4Vqe;7>|QN?hELn@85P`SOB3DXg>B(@_dengSZYQp2N+JwWIZj+xzBA$&Wd_~Yas zieb{&MMkgcGqHoC0XQfN5-_#u^a&t?8Q!xql_{+6MlBTGeo+j4mQ^>kOW_6thZpYX zX=DtjA@c4~CYn7bt*}Dp3Y$jLV&;=@O;zxqBF~4HZ88@>mkix(lB(n{?H)=~aDTdt z%*Mp@BkVMvMlh-98D|WQc3Gaegr6-@XB{$8r~C6MtVwv!sqq{#K1IKz2QU$gw#W)< zeTybWp#2RK3XIXcutz@nv-25%I|-mY4L97;*a%K!tHm4e1DQ|Z=~^RnMV>*LcPdx~ z>7S9)otemHrsHFK@0i`(bXDFu@RJA)-Ea^2>PpgCP{3x)(E(CllVl-ll7H}^HP1MX z`#)_Q9!omgU*YMaA|Tn`V=yD}v(>?+C*6AQ*CQ`uF|;D_T%2veVPYEPDpe1Fu3tGyDX6av@Vj>P?3ca;Zl+Xwgp&*zb-xULJJ zu3ZZaYDVq#Qv?TbsEdD7q!;G97%c%b(HyKhdb&ShK?6|ZDOfMrpc5=Wq+SSpLFicd}X7XH5 zQuqHM!GYSe3yJRqR|OUCs}NvAsJICq3i7!@Z(EL&xd;kNNx6ch%^+N{3g`g$YI}Js z9VdBuUyie&Hdh5yoW%S~b2|YeYHr71*PPSjU~h6Hk&ngL#F+24L!HB9uyQo7!(78i z>Orb$$y<*W`jr$$iR zUw&QrDw{^zKcQ-Nujag2!mS=FC3Frb1Yag*jRCMs04*2}090{cX(cC5@W8D$#H-a8 zK<4yqEsH^GvEB3dqBp%kITtn6*=zzbyc(&`<7Wja2k?724cSJdnuNi0@>7K88Rpwz zxSyMccg1aDeYOn|6bXubvW>{{z{B=B?9B#10q+oxFLWIwFwiXrqIXx+=^QSVci0Uf zkc7Nt@|F|xEP{|Fm*pMk^eJXohSQqKyV2Tlw^Lsq?r&su2L4zm%=(fayoSVsLgoe8 zPJq}_ON%6EajZ(%@UB8?ocAKrPv2jZ?pg6pnYdU0N@LyX_~6d*=A%?L|9`(#peCWw zYc|N?G)X-lc>-7vT%hMPMVVZ?YwzK9Gpd~x*B6&v{$Y*NoX+e1HNt(T28lP=uIKJ+ zG{SBNKb-$Q>KFbyItDi7k8zrwfR>U37k2cT0AN`obg$CL9nd6Y8ib1JQGc6wy5N=^ zfdCXe<%yA!icE2eAH3?eOZY1~cbbDJ){kqYUZE%xXkjRTIFMSP3keXC`<)h5-0%en zx3ZwzRY)Ue0uxchcCS%3c>=TxE}nAS=W&yjjv2+At`y?cTb1q1T(@^6{06pp5Qqz4 zQmRnPsm(WQ$cGBBsvvKEPEcyp(9~{a-pI1RIOf<)@VkjKOj5gh$3f*|x>)};owFL^ z64+8>J7&?2cu~zNY**;vtq+8Z{pIva;zFC zHb>q3%Dgfdv->6BuDYz?^!xtkK^o!l~ls(be;{p%$;eHAS{!^8 zJkn-xO`MT3(ja&2#%>$m)17#Z2Zj8y>tF)#YtmB6(QN!N<8`Zz5eBF4+^|n;i|BNZ zv;Jkj68#G06$XAOtMF8gCOyU(yc^>gU8@y`(Q(V?(G*mh1#c>Ja{2e-m-1&B zFFiSxqCYCejvY==3O;(`-1n6uknVbMxW66KiP6+t8ch6^o|^`zHN5IRtZW{CU%ZqF z=xFhQDEh@}yiaqynE;{1*l7^dO2Kz}} zQ%|t&Q2oqjpdU0wXTQJL#!`8+Stw^*X`n2%J`vkA|oJeyT*gTWdc95GG?&nPGoF5=SoW&?$Ft)om$bLc^Wl1i?CgH8vWqm1FO8oLddIx; zn!CZym0->ke60F;T3lZ{3Po|bZ}M5wPWZ{=^VTZX7jbWyhlWnx^IKL9W zTV`ghb*;6|^ZcHlZ}ANBX7B}PUu4ts228U$?doaX_ay7t=CG+o+&V#~CW(cu=M$gs zJvCbv{;K~vV0+R2%^xnS1p>%o`HM79vFLH#|7<=hCEi+MMT8LTH?y2xrvQAjpX-v6 z&ph%C>U2{Am`R;QG%x{s`8vP^cDyVzE#aFeAy5oINm2$T1fXsE-IXr$PU`WpT=qBX zDLntx7&ayL?8l$OS5c4oV%s8wO`Ns0e)E93VO4&R16TGOjIpm=DzoqX)p2}lD(ZNZ z`E@%xvUX$rcd@Ft#;WZD;IC%xYzr>>Qy(qneHqDQi$b_Zo?+7*b4cmq~2i3GCEH&DR=C9 zlVFTa8AvF?#JA@lD8{SV`Qntq zD4a-*WX&X50#J_!^6B}`WdEU|!NI}II+-EkNO?oSc+5>6#Qj?c&;iJIM_~(mc(AnC#~k zZe$d=I3yXMhB$apq2)lLApc6r=5M$mor#MgqZ7aqIA{`~Sxf3zQ~`&*eyceVBW3dmI2-Pk zkHeWcreRSM>Zpqk1D3lg-d#V~`R1d`O?WC3i89z88syJceF{k&fvBF1g000MGT;}&9f z`B)LiwDCgkJU!%U2{YYl!Y8`a!20WX39_Q1IhMi-y$eiq4c&nGBEy(-pQ41^#`GgH z-qX#G_1mxit$B)i^)lxYZ_2Iw0x`W7$VS;C>&4oLGvk(^0^R?{SlJi9YSzofUz?Lo z&oY^hV3?9Y;sERhdX5IU!u1wtKyXN{ZEb<$CNCG4NYqu{IHnB9Bp_o?aQsW@EMw?j z0d+0#H z7t?4)G3(O=5g5^*?iikFFlU4_2~2{l{^0k@S|Q2x ziLkTvRS~nF{Bx0@kz5nrJ+7&$s;aBD`E8I`J1Y8NMddBUH_9B=fU+X{I__o+D}otk zy|#w^&BKatotS`992|Z7#NbiX<-w*|p7<>$>RZj`R~6hnw2bG*>pUA*sRu)g@$hro z1?MTn6jD4_g*;vXX#0BUBQl-8V`L``9_Wv9h)MEI$$<>rmc1JFv1*MUE6-!sQ`lo+ zPq4wnM-F9=*t@q7jM!or_J6bKC;P&%_F06Og=HZ+m{cZ@{(nG z_hCcy+B>G{TP15^mqlKzWnHcsVXSdod3_NvEV<5x`vsEkLy;vl4my@5ckjPDhE5}d zJECCMYYVegjc-b&J!cA!yub3gZhl}af6+D6qA32JvfA@!LTy+9kR>$08YQSRf!7-Uy_obGJpae)3#KN) zgfB5^&E!Rl0g*B0J2|Ktf1Ie&gDt5^{L@Z`E&E!C#33LFsXMo4tLMDfgms$WPf0f9Bizu zg&lu?2mC$bI}8#|KY#stqO7d^S-wi`)f)P%x2Ld-kIzF}!ftsZ5eZIXQsaXM&Q?tB zGOtZCMencGZe23C6LLfO-%9$Nph$pB&8=}(^7(b35HY6|G^^&64%lM<5q$zr(+SO5 z3oI%(2>%M}&cR6IR1dfy4qoc&)eA0fjC)6UMey-i3ITVNG&EcOsK+MjRziU8<6E%tg?v|BkSxdd)KxX%^K34PNWz|?L*xzVU zr5Ip2Lyhyw8VL8-9Mppsbr|?O$%x5F$jK=vD9A}jNGV9kNiMB#mr^yVk}D-ipC4}- zki?BR0h0W7qYwDKbnfuitu71QHzw`V>T&(=UJ{+uo2;2+n)37$#xwSp5HbRc%NVnA za#b{;qq;v<32|#=TROHK#j!7-EG~#dG&h(-W-IgD~YHpfsUrH z9!;Znzg4U#Sl(-r59s+S{)N1iJ`w$OcyxEjPKlEW+w>y@f%4Y)8Wk!38h2-{1WfhI)IO zK!>OD<4!oOGU;$K#;!)O@J+f4CEw+M)b77I8z4%X3l!z5v)I9cF&cP4r!g>ez{M%8 zP%J`mRsl3S5EfhE;9&2F#gcN0I!dP6>p6DAx)PSAVd+;MzzOj9TdbIx*)OIcB*xRs zsbej>RS8)v>%hW%ZEu_L%KVP(aFBB)-1Zp2?&LhAV|n%84v}E{(|BU;Q=wb&a9WY) z4Ucy{Y6E-W@-H*1pm)ELP>N5j0RLrD2{fYkM>G8ZL;#$I=ic_fo8UT_Yji6xv9rT^ z@jPc}TD-RkbS8LWD4ztxNQ9cX_2_bfkHE`;9Fy0Hul_jG9o(4*7 z3goEE#{(N4l9+k^OeIIROpWu;A`K-%13EF>uoV!r#mu1s0rd0=PQzQ+1BDoQVA%K% z%CjT6sZn>O_ro`pd8^wE#ut#JuH0A?r|2O=PnTZpDn2FPa2 z-XX=!*Y?Hf^cQD6LR!REK=1&-4#0${18f1k-~R&&v5E}muizp%sFOOeeKlT-4W&1= zL0Emch%#WJ2gKZJfW$hEdI5L}ml3>z7Is6cJo772!k^Bz=XR;nQD(q=t|2A>K(Q2J z?${xk95Fzc5Ria1Jvt|6g-pp{D7|sJJKLB&pD+_UoV5u(+P~YtotT^ldO!<4o-+)) zfh`n<&^nPC*hT?niA>DXu7s-ZyB}tvG6ZQ~V)9d2uAjLsi+D~4><8T|0QBEn2s>Qd z2={2yubEI0=%eVxfK~BW|1Tx|Ikl}99zegpWO#NvvI+R{r0Y@46HHlP?~o5|Zz98g zW7$}M8_)~?Id533T-hJ>Tq#~U309$8&Y1VzA_W19h0WH)p?s?N19wm(jt1%J5TL1E zMpVyXdpNq`gP2`-2AI9O|3U?V-a+R`@1tFP-;uVhf^lXP2WCO-P5HV+;LB&$AB2z1`)(UOm;0$6khY@y(bQN{~ z`K=J|Znht!-Z92gB5v~i2P+4L0r26ctDx9r*pD_erc}pH?@kByYo?(#Mo}EMuo|!D zK)3sE-jQ#(-l)NJnv!99ZbtvG)iR-w)#4X3vO)*qX9rz3gs*~$sz-u32IGAm+ez@) zG4-RkWid$>Ten=ZP*iBmI3+`KE3P|z4@Z3vzae90( zakA>>I5!0REu3C6NNSfnmr=$#`myyuk@8qRE0Bxm8Jdwmoil6v<>YT+9cobCS%1Q* zq~{9b?&Z#zrGk+d@GwyWAx9Mi+zfDW{>RyW{&tX*drOj#({E%XvlyCUx`|32Q=JWb z4-}tbK)Wti3@vEIf&Oa6LHYI5<^(LBiU~6lY5Em&9hxT)eSeUc}C_Oy}J+4@J z+&R|{9n_5*>+gHmm-;(^Pp6b%o`4@v5?=!wyKK}8wE6fUHyP=Ev_!P&MZOrMZK^c_ zU=0&H@o@6v-=oYxdyNV<=SzwcRR_H+S8Ru9Vqq#U1payB?D-oY$OD-ib8rgdxMfKzAIa0IrK1 zM>~HFV#CF0L8yW5GHl8nYrJ}XN?0+y{)9ws|H&&df9m~{r=ovw5c@71v3!P|oxic2 zum62IDc-rITx?57UV^E^gdz;V_}&UfTR%s9viR;;N!6N{~xj$SjY}oX|H1pR=S=f z;&us4a^{Vi`}VaPvS29#-=efSh_!;eAI|<4kv-zC2a|jaBHaa^!p+s;GRXRjA3q2x}LIWoT z7B~%B))6eP%yUC-POSIKuYTYbBMJ5WmE_RtBU zpbQL0Vi9uUj7gHDsrZ6Mc~`S16XFz)qn?al;E{kOS_!42c+%Ct7SKP7S_GINZJ1Dl z`)bwvr23bmNR%@kBTK$$H;__q-H(V4QC*9_1-V3`Nx zM?vh#c@QWX1EVP->_=--`iIx*ap8_tFHBG)AtAxxb`S@%*ZL#D*M{CZmz6HyrPMdH z-|;?l-t7WzwUl&w%fq`@@Go6G6+YejI>82Q`Fb5LW;@#X?v|xCs$)lm{~w^qCqVpc zR}`)20LI9sfN0_nLu;M*3#-2_%O3&j`SSk>{;9R7Kg*W}h#*j_4dd{03uzGS3p#+X za$FGTD+THg6yYV{=ChI_=INkB*_;TPD8M9wOhPP%F6SFWS`O}X+}t*LMJSywXh5cT zbe_g!+F-k3&9nmP$-l&G>ezt;>)8+B$$=xWbO|I0^buei`rBI=*b6uN;ag$Wn*&Md z&#B07^vkU;cak*mSz$T^CIAiBL2N8lsfnDaE`{3^Gm%9~;tq>PBar)CZ z7KW5mdkQGHSjGmpNle;;Jrjg<*w^`wzw@ouW9xT*6N{*J6WDK#NfbWvt^lEJ6$JVa zsJ>vyV<3A4u&OUG<}y$p9u8Q^K)=a@NRy8NC_XHZ!cr;b^AxZ~p+%rUvJOTqcO60IABKv8Xg>{K6`gV}V=v?<#1d8V3j5G0bprE+{O6O9!#t<8!lcaNGEgHDv(q7lBE?KtnE3YytQh znGiT>!I7SgTAW6`c?Y9KWD$Vc$$jicg+~e@KK|IV*BQknhnnmhl-caM-3kha{GkHmADcD_n!FS7sCIj96`TE zXbk53?5$)+Tmf1=nV4tiWph7v5@4JZ3o`}=T(c;U4{Yr z*Z8IPG6BsJL{kN&p*UY)lw-z0%xX1$MLH7>dR58cOFRewz9s`6qyuoE*)WQQ)ym&B z>_rXfM1#N#O!Q}Ag_x}OIue*mYwfvTzLa2AKotW#yZ=7FJ{Pn5Sc8K~Sl zzGF=}-vfJ=1cre)3up&g+S-18Ir`{(K!4|61GqiV)b65fmVedH#_Cn4K@xrqrk58~ zgwh1a&K990`fdQH+I;g#uljJx z+xHqZZfijo+k=7#C*$dPpW%F=q$sKHfy=o#=a^sp%z9mLROQ~Jo4twptK-)1cWvy) z87Nl(oNu4(Y~$D#3o4k{KA$l%Ql`Zs;oBeH5|kBta; z?LUc+lrOJRjo)}~Gk8joYINqg8;rnW6Lv1E>|HP?vmL))2n`kLpYL+KzsNaM0DZ~C zgl*ev6)+Jk(Bi}J_SVF~T|q(NCur|YN6~~f9*3b^ z`ZHZBkQ5wDlNTJ#oQsc3IWo!x?8&6jgB>j|{Z;+ag+63))f!8!h8#VFLVddPn#DvM z@q4oy88uR$%o7=J7hXJV-VEt6Fvzy)E1l%9r;S8^Dsw!aM=>M*Lzc4*7RPbT!e?)= zXWpC=rr`-S8uQD13&>_0YqN`siv`&h9h#m^DmXu}3}N!Uxf*|O<4$Ypn^Xzern2V* z(b4-!3^YtMw^9%Z{c-mfpQt&og(v7GW!;u{mmWgFIkw-F;X+3*ZOu5j>BMgJ& z^DwDbz^`b~0xcVdIW&c-YI(i4jdu;M(VP>5LZ6F5UAfW6M9S~AuGZFf&~gvDg=U%)lR$Cz2qa3e4ERX!Jpj|SVjRtvAmu{tfQmjq%bz;Zv6wb-T!<0 zeTSC0t@6l4!-V*FlKv84U8jqdz$a*hpfo<=ytmM01pLddF?$yp;NJ z`L`&ieMo0>;0w-`;q<{K-nj9rU9aYiw@QBQ`0-zKpl`ha8;8X0=seRUYW(x`mrv5(gIcAb9&P&1ge}o#P zE(j%pgrOYLT$*~cyFJwU2bl~0GRGUB^|?8yy3#w#@NA@F#_q1POY1>2N?!;)hj{YS zSJA1Sp~hC*t#ZIFLEw4IHFmAXj`;>G%c|B+=@8Phz+xwsZ8cdd5*)UPd#^57O1yII zGAuU2?Id{~*M|bx=>kk1;R5CR$C4zhlrZfYd?$ZRfgD4;X{7zcS4Y26Ttcr4mfo@% zRPH8#APc~pOw^!)P$50x>zfS}F8&o4WzlnNG)q9%{Tcp1cJFI@fu94s=_CvcsaFI_ zqlhUS8Lquf(<#d(Q>0?AU78F(|31Zp{A?Kid*dKO_aw zrD%y|4=cEBYtc4@yrKtS2M3q~huT7J;~!2%0JG0;k@5WwTF+^WPVXE0@>Wn8j5|VYS`#Mw%q#rxUyUi zUO4u7`)L)Kr-t21%KO8Mn-jVw#@TuR?CM?lGpC4STTN3qrQ(LmNiNUQkeyEbZJVWA zJ95H{SGQjBMYMI$Iytn(PH*2dq4PAMQK`IbW+O7f>8TOx?pI;@7w}00o$$tzt?GUaTo003&w+eki+ErS*Tj?uh zKIFdMpXT*CBa4H(yR6U4w6tgOL5+hhSB7NuFHUf9DI>{qpEtkv2>&uTSDw*GIk$K{ zAfQeCcKk?TcEBa$2c0*3TY~AJE%nonNEH%S(znMg4^~#j1I?TG&Hvmy9^W|dbS9W{ zN23G%>{LzmA|j{{dS0?bcqB9bs9E)8d#>@hq>pkPov+a+2J0E4DtTC#_p6$BuW4ug zaC4IH?Eq6oo6xWDYBy;Lv%>uHz1x?+%aNT(VXQ9-5Z+F-mIjxt0+9wa3cT`JX?7TF zG6_`CcHW~(UL%a`RPK?uyh(HR+b1N;b(x$>dsXDm7g!WTZq#A>%b&mn4#FIX(NANR z8kXu2?IWB}Xmk-p#FyWJ4uyWW`L}X~h4O!f6?5ddEjOvt-OD{{vCM={WH8rzb-yHs z)5XRNmNxKSGB=H%w4KrqMilFg-tfkbgcIjk9?z-6RGskVyXx$g9eqFJ*+kLO{G+O# z}^OLtW=RXO~*tHe!h+hZ00Ed!2i%&`B7t0D-$MB6pb) z|Iyv1@`(PV?D5gpd6C<{9=2p`WJI7K_J{ntZ0xJeC-3&;tnIy|?44dC$)|9YA0A|8 zbt@$X(Y$7u)2@s<^F9Pb*Rj4mA9jcM>E6o(N3vD8?pK3}AT|$-G4tSeTz;8>&O$3a z3$Fcj)?1wKNY{ADmGoTQNHWOs3QR0B>XgfES`ZGv>WAj|ko?!q4mNrzXC z=;Dtq{VmQ4sA|AZ1j~RHVq?THVX5_9uG?$&WpNnF&nhVQOx}gx-4+um_>s7xQ9XQZ z<6EE@s4M;I%Fi=#p3-t7P-J4s%t&U-;_9&&U;bvFrL=6vwo2&HVW<7M6}I5t^LX0- z{j$gyq4T(0_VUkdmlrGgy{N)T*@IsmKL<<#z-FkYwZGm>F-v;_?%A)ich0wN5E4iQ zp}ZB7I6g^@1z?)Z-U5S|wd~%hykWwr}=g z=qk+OL+{c(TYLiwoIcGjq#Vm{SosbH^}CQ}FImNXqG!)Lc6iwmtf=cfw@HMY2rF71 zF?iqquCpS)KVD6_MsEemA6*88HZXX=YQP>q|hH~k9a z)*%>y9U%*xzA9|`)7A%@w3hd10+H;YBsOd!Bz+*mpYrmJWS)p<4?I0CzJFqX_A}#m z)*46SSa{bz+#=t-JiEk$`rJR9>bh)2K#x^_(d9>%mSaOj{ z_d6FY6Pc_yf4y8erLw#KWa9_X#CuDDnbY``xD+kVDcN_cdkZkDq4HDfZEo>xyS;16 zyJ^y#OmihJ9pLu#F|o(@TTscE^E9|#d7V69m^e8cs*l~mFJ%H-CGM+jQdd>4d69|6 z5ec%(=+z(6P^EN{BAKXN2CcK+8TZW8nMcaPXyDx#-tXS?Qhiz<2a< zLDok3n;d~$mW#1LMh0}rS7cf$6m>z7)IjiE*vPuLq=jq&X;;_d+@e2o`WxTa#~#Zj zm#dD1e&o7pVqU-mZAt5vzGw55t4&w2ck__5uBv%pHtIYwm1V}@UPhl{sV8&mi{&G` zfw)%74{R%&v213X$dyyCjC3ZntJZp}i*Zl19ouu`_nt&X1GoA-%Rmee#5mcSd7?u> z)}V&xvE@#VlhBsoXU?aB*MCsRIw&&1YskrhNTd?79|KhnG%oedCO5tXtp z*XWqul$ls4uy$ohbs46%3`WE;+0sQHAj)w!|{)zCTEk;_VN_eZQo zrLRFs?4l<)s*_PdOX_k_szTPZhH@^Ip4T7=_wtN-OH8-mwA5RahvD69?5a{sF?>gz zz7xhHuopn(qJa5R(dnEHsT#O0^i(%yF)zzdSSBj)0lM8(*@J3#H_fUcqR-Y+D$166 z%6K?NnKo@^E!UW~&SQ=~+*Rp`!@-JSq{OqtMk5eBUWb8xUJX~5<1O*Cd!2;O<$K|! z0h;6!QqpV*O3+dYYDW>`dN=YTqv=l+N}mKhCr_-&8GkfYmL)@2BOppFE+m zTPfjU>X?`En0_{&w$^D$>_yn+MEp*?3k=wUJGH)rNhSP=n~7uMqfmyPxC3C(DwWXvzhAXi#gpg5oV_0P~a9pXx< z`S@5LE5fZ@G1?$?*sw3sv*#W(74#+N4~0YU=MT=8rF2Q7u+Y)vVwzDsCfUS0Tq)xLMKR{kGaR2^8rh;sLS7ib=}E zWM=Q{o|w=LMn=ZU-;&oy`F&5lK)v{e3})d_SWvVoCm#TuvU(}6Wp#;0zAUDCIl9Zg(tS7-$e~X6CBZ1{qVjkCrkilexcscEBNqd~Z3<1t zaE0+R>)pHD_hEM$p8gVE|6F~kEBi;zzTsG>2F1yMaXX7n*Tq1`@yJYnN=o zOE_ZsjbePNuy>$)kP}59lZcZs!eDbljI)jWg&ym(LuR>;S{@-7W|4dOqfvHQbRQKx zAHGR&tvE~xQCwtRksu~Xf>ZR8UZ<+57UcDv*$nh^b6bIpJb*;Ls*a$EEF^DNRI%pW zU8y79{9HIyGuXMY7YvD zUVcWEe=NE-GIFcBS|v+v6cesFxN>Jckdx`ycx?vdBILsT(W{n@w}SV}H7g*#Y54Jh zYgYXjwD#6JOyLz)P=*v*G3E3LG7++enY=z_h}oc$p4Cw%ANzb6 zyL0u!vzd6oxZ9zuQvNl7n~SOfW~qe27f-wTa6k=>fG{{fj-4TwDM{Z~Gq{|a=O(Os zhnOHhI`)D4L(>5vRoud0F<7IDK!0RqS0Acr-^$I>oawSbJ=ek|tJO6bDx>z(%HlN7 zF_Nwms-=g{^DzBBR5S%)w)tnb@tO}YSE!C(9o4K}t8*O-F4yoLA#FYl-(q$ci@OE? zbGYEFN?>pMcx@-W-FIb`_Zec>S6=#_JT(z3dX|BTHoX`1USRdlz0cC=6%U!G5g$v7Bi*KCFk6^9H>t#@VMsvIC(iqhl!_N(L*PV zit!Y1iJ(vMIkXZY!Q!6G=zt)Oe9HIltTmK2tnXF@eHC^_ag-hXD5n)~C!&Y%w9(!6 ziJReyWB@8f;BVx~V=)Pgi_P@(n8rU+g%oc5;3_Ism^+#{d)P?`Dh)tahE1tRMWYs_ zh!6efADCI8^}Svg8&;^4oEld0dlZ|KSS1!sg7UAxxH}u>gvqYb7*u}UpUPsN&wpMA zB2eGAP@;m+CHe!qpZ9yD1)Lxjame>Kjap3}OYIv;Ad41sTOGxkYg?6nS5s%&*$F5eIyX6NpT_ za5hOK-Y^2xuYc9ykkEIa{n~%gqMhtvVkqaG5Eq`Nu{-V~-)^%P8c~zkn9KnfB#|A7 z0za>vZAALo!V@=K!?6GSpVyWzTJtoWnryPMs6H}7G)AsfX8X2k!?UZaLTOjQ$H;^X zeGX=MDn^@j4-;$(IeB=#dGcdt<)2r9+kF!l1v?YfhuEbfre~>it#Ef(JMNjobO6NuD@er(h_Fpe-&35bZ@7YMxhQa8ja!j%sTq^_4cmQg-L7&qJWA{ylF(jIb8g7Bhn%FSNy&C2QA$# zT|T?_1YfJs`4Hiue%x;QENv(}g2>OyT`W#MDAh z#%$;7Y&_m<%8+r~*hz5dFqr-=3$P`DNy2Km^fIvI+sUPSveD%&4^b;@tDJ(3!*X@}H^rGY`cDf6`L^=iUh6uAq~aY^ENVjRHly(quHF*_jQ z^5u2g>Qff~Cc=l;L;ov9jOl#X|9(Qikb%O5@~Z^CJ$^w!2K$=-9#Kl*4bJDb-YK%? zEO^x&BlGVM$W#NMc4(<+Xt>OT3zXJj8)wrK>DH>i=i;{+|zf zd2h@ub@Yqb&)6w%IxIx|UoY&3W@(CHSNP35=6iPXkypRzh#Rk`9=mKt*mwEPHu=Db z`FGt5`{O`T$by4|<5jLaN~B`U74zk3_WI4xe}5veW&6aUw5qBDY>UunGGJ%dHvcal z>s38Rr08bI=tped&B|pz21_)MM4XWQ`{RQRbUZ>I*bIKZ#`wTmT~3Z@+<+qL-ZM)Mz*O6OT*y_pk*YHBGx{aYbR~)II3&uE+J-7+J zbiGh5xr>;DzZHYn&D|I1LG_ecn?7{-Qqw0kWA2IlsoZ|Wo}MM^^53qQ+-fM?{{VbR&`z2`Hn;gw2~*8y<2=42Q10#4cd77%g< zdeZ&aL~mS&{jhXog_t%o3xv(3BPlflkWe@M;DBDUg}A#Mz1*T-!-ndTxTryo+i7gg zF7KwFO`l`UQTk8@>mlyuYK_9}D7G8WmK&OmmKNqVRW+Z|EJKaQ-%U(H0xk;-Daa0n z%V|Bx?qcw$6-ODlxG+HX!9LaNpx#<+7g~5D(%muHY5!U0LwU^dEYrBVN*%!(@x?1= zvI}8>_u7;xYR~*?o*jUk!dm>1Eanr7DH%^*4a-ZhTzGfj@<|Q$LE5rMqk#T%<#FB2 zC6#f9=zzcGoawr{EKvk8F;}i_H_w2`6Pcd$X?s+Ga zS%Ft>c-|3wz%!B@6Cci5Gu<>nT%B6KTd^uB6#-zCAMcHFzf@{laq)^;Yn zf?RlXosZ=2X~dawcFiidPb=fd$om6SUg~dB{vXOLaITZACD-vLO zH!UTnY)Cg2frM~^ssyj@8vnNGIbTiHR<6!D8W*vR2d6h4f%!7<+K_P_J2{>gZv%-8 zKJ*~_=bt1Q?C<@0PLZ*4=sBOXK1snx6_?7y_O7ALJ4tCs8+vqtnz;4{fEF?#w&W)q zy93PO-m~a@uXnnT5jky0i`@^gFx*eV*0f-q$Vqi_tjhG>8Fd}Au+;m(RBN@e+AqfL z-=qvKV^0*o{{r}KH`v)_>~<-9_od#o2l&dJ2lU|ZdxCjxIiM+l6n}F^9`np7#1wVk z*rnNi!aD#|4OqHP2^G)oSQVSHXJXZc(ALFU$|dEz+@}KR zhc6J1Fr7cmj*3j3>T%uoeK+Z((OLc(VII)|FtWA)?TIvKk?NT>Y2j*7SN`LP`kiK; zB`q`PrTx#ta&{erqTJydl!lpxIrJS}33m}AiO%EYTYilKa>=GYqV7L)dF~jQY{qn) z)RMCA%r5R&(@AO^@}BZmto?C3N(@^ppS@F>5QeZ>`}~Nss7}i;TE;hS)dQc^hRMl>4si;1}SNfk3_?py3GCzVtGT>vz8-i54K(8)Gz$6khvwB1TVai-# zVuU7a4;O=#MPk01jnJgTjIX+z=p(BrkR7!aQvJVrR#Qb>=Qd~LM0-&^vvOn~UXZdl z_;%@M%=D=pLiFxWMi(}3|5>IMu+LnrNs;I2+_XC?8+5&9wLj^&)nI?C7oOVM`&nU@ z@NA00cVV=T)bH<=?=C1}9cGoz+6f9Mzq)oQtP&mbZL~Y3Buq!vlW+{vpdG!c@<+}F zhRO2Fo)41>sbkVMNTvLb|GTaNQ?2@mZTRsW9;B;Kz6WRu-}XS zY6}%?TFq@%=3H+~TTOUT3mOE5zYTZKevnNzd5j?FD_ZRoc=-w1*6K-&dA8X?|F(M_ zM4~6fq{mAs!==%m#}OmB4nx7)kw%S8UihJR=@t#+F6C^I2PM(TD?dZ!>LIg+m%<=A-j@uBIYDg$4RTK~jpR4`yq79ld6w3s;mi!~5TTe%J?B zE~1Tv>94B3Y^kfh-7>!RI!&stW?EJ3ZtwD26nU>JIF>|#-Q(WjQjrCdN1vjAhK9z} zV`4uB>4pj0^NXq-sVAG>a3nEe_trRwOl_uRVt(b-{>X3kVD|DuJ^0)K^vG3pr%}h< zHDC0+ksrCu4y!8-`ru{!pG{5Nr5EZ;gAhE!ahl?~7$aLHa*ewNRN9HIpvY z6&09?%TNt980n7SJD?@0Di9Q&xeNWy5hhsusskKconXSbG(ZIkEYNPnz17}&uHEUd zSF{@r1nnHF(p@xlfR1ZEu1I=Gz%|zljTt{m7 zNPm5L=_Tq!A5c3Nz_f1NxGLoMdoI!59OF9XGW6l&#~D20zixkXWeYiBt*nSMk(_)p ze!(niJ`0v~=r8N>58f)0xn8W8jWhF>LW|YX3!M_KM8Cn(mXK z9fza3QW#^%%2h%nw1O_LnDFtGm;e4*;iGT{lqg_NXt>tUAhE#3l&{welr6qTn28^d zUYR-hIPc%l&qFc`U5>R^o7t4FjFm9fF;d5oZz;UdgsuKIa&Ot6x3t36!YLy^&ga_u zPL@;DT@Rkw)7pXI;5;834uV|VI5eXqsfDw~>CWJj2H(Hsh6qKE}jmEBko<25#<*8tn))oeWCbv5-f7D$e zY>#mZ_L^;yqh#&_W5de`KSde>`RqW6{}oER2Ah!zOQ!davjW=#r}6umZgW+K?Fz?e zFw@gHe;lN0yLt(sVUXE|%HsyCm!G>ap1`VMhs3EcwXXfc?(kmi-m6cX@p+H?ab7tp z(;e(?l8f4SwuA^wtBopN@vJ2^C~irC#FKGbYfK+Dei+7nhwH}PVDQYJIssHbe22Lk z8t!xKe()|VEbLJgpY6m~eY<9$emj|)Gq2V9`<6|}bgexd7dJgx$ZX(dp45F|Aw?H6 zRo!VI@y;ss>(fM z+`bj)ifob0YPfAB*`;&h$`W>Rb1GGkKKdeTvtrx$UY3Bz{?FIP-Lt`gwcppr8E18s z+>8$n;}d9RvMRY9f@hF`b~VP)tpa^z0s24Pdbw} zT7TVQUSG&KGh#S6erx%0BBFM0m1wp1#MIRkIaYRea7%yv^`#x#Bb~jmo8kRp4t!c& zvX8ss{6>^OfS{DU1+>`Yi5WJ9Rjo#MzH@@fJiU2_^N8njk07xbbN7qAZ1EF>T`!2i z)_DOX?@HE__@=^cLr`%QWD1#*#(YJAr4$@Y=&oH)Abazp%l-`2)vw5c4B_|8=(swlSlXHSJ6+0XIJ+e=%p3S zhn#b6VY&oNhN_KDL{_z#K8ki@7SEJoz!ft$Gb@?#(W6b@`u*up|5E7ZN4}332Y@2Q zYj3H6{XIKvlg`?TE)QP)`ST~CnlO*$tl-Lc9dwQS0l7TdD;3q=WBR0sHv*GNUr{gk zrBv|1-6QrO#Lb>t`OB{i`(FyecHGO^E1aSRNd&1nRm!LRS;yHrrE zd-TGNz+2f4M=%T9JYpa5Ke81;A`m`|oxMuf_yVE9K?lyyAK;dp0^z{CS%jU5*TRTf zqSB!2oin!B_xP85{?!a{w|JMd0&**XTT_ENvr;i(SAp3`36UVXf!?)0+qM7C zksC}E1i20+B_&f+n6#DncDK~z@x}>g3=3kiQwfFH*Q}thtuA*f{mlFZVE606O-28! zSJm3Yso~j<;j&%>f6U2mkjuidxo6M)y40;_B0t%@ET(pMK1yItu_6+Sb-+40WPXRh8{34COP2haKu$iC&B1>D_us3gTFJY4u zw$k<+1@_)H+Y3Fu2R{wJf^~0TPm<%Y+$S;W?`u%$Fa_FoV+~6HKC+gHmGbK=K%%9i zr{^=M-3Rt99}FIeeH+j+JONd{H4rwAe)SgFE}G6~#b!m%QySPj5i^eO^$FnhQp^GN_}PS=Bd0S5`N9>5V==5^p0 z)idrfpFEJKg`fUfEeE88nhbRu+-dR@jZ{K7Be<$9$>F za?${4J5u`U6;8Q}=E-l1RByVJp6TNZkgEd57}nqkX;y%Kc6FVBHNEy#^l;U!p0qp! zi=f!d{_0Drey4f%Vk`ctA;EpJ7GX=RrRBOWU%qU9TbDODU6To=EBTWTa)Wn7iOI=# z)V@kxr}S1ImSA0t_1uIN@k))j+6g)S8ZOuG~MWhfRD^eY?H z(h}9u8kNHz)k00}LhGzTaGBDSmO#dUK-Pn%h!G%sps<8@kI@JE^x}rm5{ZrbnB+v}#4pJ0`%^p`s!gj>6@8?ZI|Q z%oZDEIDb8;!fkH&4ici;XNoyw(u{sD5}Rzn0RBPbwTq?U>eZrji4;e<~l3NUos?@u{v^Qgas(u=LOuGdA~D|9@P)WmuGL z*ZxhXAR#$|fJk?D2-1xpDbfrwG$=@ibayvM_t4$l;S5SjcZ0-xa$WcH+|T>}rf!=v zXU%c0W8Z&kUn`DXWu&Xm$aKon#*1Ie7(L^VBlxaB{tS&L7$blk!8e$SA*RP9e6m)Q zz4oJJ#NC~Tj#_Wd^7HPjm3u&V`&SkABRH?a>4sqc#J~O+giC>iXNqMsNkNk*a?(pk zYySo%-C2I|Qw284Yh9gXd|wR6Qlmo;HIneC=fU0{Hp(w53rwYWmnyL3L}p*h|GC^c zV33l9KV><>BNvV-oiVFGPHfdY;j>@PI-Mjk)_{q8znsAQVG@Z3PYiYR4WZ1d8=&VC zH!l@gc?g&Pu_FdHDV~8WHd0@Ogk}CV46T-=%g_?E@tlmmiYXpRZC%C|R`>%~8iAe`TOUKdH zAdq<_ud4d|_`B!cJZq4HF5btN?b)V-h$!9SgJWZ;`jQ~3Xqvd87lT+mK7Pr`q&ZVx z?=Kck#Qx7G;lj`=Mm1GM-`$g5yHc{m)EFcF05S=}t#K7mon2gVhJbJz1PlY(XXKck z7snU$3_Q!#rsx$I0Au*CC58It+A7Un4E}UQdXy5VtbgN0NjG+~IXCWHxYxO31gK8U z&e9r8;Wt;U?5X1I06vQEbI)xlX}86}@$vp|q5u7$Y%m;nU%uYA#AX|pL0GQ|Sbg)f zaHo!0@OWSZL?N(GO?VzU)%A(qpU-rCB_aq3_}Gp7Tm*qr@p%0IoDwiAL04CP-K6Jb z!Y4A&PB_q~;RR&y7pW{hvGz|N#g6cry5p+tXbK9o7$Yy8BhOzr1%5nJ*oZV({uL(= z|L4O5U}$!YPs3bhKMQ(X+p9G=SuJ)WEaR~tLS7+p9({ed*#a~FT>xmDkdTn<_Ls-c zw*TBcc^{~hKm)@+@8g_o6k>>f2m??*Jj&cB0Lv^NN~$}+T_1=9m?H=Y2~FNnvgme@=Dk_St61jGmbpy&o_PE z$;xV$Hrg$VwXVy-i4&pb$KxIW+{CP*i8K&wR0i4@zEomR67VBvpw^T7osUH^4F&4v za`3F?L`iB8ZiDaI7z^f-2nP^ltNYGMpHqXSW1276Ntv}9=)IQ|t-AY2IK^hWGPY0ibvw zh=own!~z8UBe&rqf@vHTo>Nz~>A?11|8@nb=R-q7=UhgBAJBRPoN_u;{)HU;E&19= zkrgvxldV7q%~C4&G_)F;9^qe#h5Hw0mcnzV(@

    9WQn>q>77+`~5g!TBT3bV~vp7qu`$k6N2rb@PS|eHzz#y zyS@iOK7?Y&tZcu=kBf2Dj=4b&8R&tz&VY7xOZPImt@xq?p`349{!|(?ae1&211}_% zOcD7GpU&Ia9M5X56c*qgpCAF?x;;yv1&DT?X=pAA-}OMQ%ZbgWSfc%?xq4PPcy~Ay zm}(oKg&tq^`Q*t@`htMOiQS1rU5AV{kAQIy=CZBFLmqLB@Q=M&2QjwNM}z+%V8m(o z2?6vPcU9Xpr0Mu-P7u*hKDLRW3EedGI2WQSdhsITaPIeSO5BJnl@0&JSh_e&BV>$# z4|raCArTQcL5jpG8}ODd>|)5k6^@3=C#;^r0MtO~CIi5U@Mcd(Ap{J{PcK#?s)%N3Qfk~ zH|jLMuxs&bd(S}$PZc+JjgQtE4s>7GPL7k`jnT(RR<9YTvCDr<=bqd=oNZ%uT&=EO zQs?p%nB+??I+=>{iod1-wi1s-bP?k8^g-^Q(4SNGULhc@5&_LO%Toxk#$3~+CMX^! zVMeqc?iWp?R=IuZAc$ann;e3~U)sn#dT{^`dfoY+V4@r+x|}Nr-UmQiGf5EoD^co( zL7ASC^`Y^xotl(Hoalmq3fF4fl~@48C!Rq?u=3rpO(I@Nbk{#Ob4nX*hLj7&79|O0 zDW;~Zzc4Fcu!F}{XfqhJx+Kk_;(U&&@eD5o>8E53k18u3sGOKdYB>bY@EJ$f-QC@B zfbT47$z+6LN*{()z5FVXP`aIzl$1oSSNsilY6l08n~s4JIz?{;h_hdf15O~vDh)Z; zhGV&`Yj zK`eso4-f}Iob23a4X!3HeT^cY&pROQH+#fq6jql?33(9J%>qJE@Ri;>2-n^m`|>PV zcN>J`ErB-pH7_9{;X1Z#DJ`d~)c$_8v6MlFN~y^XsN?n{FWahAm|#1Em8E3 zxi+@`p{>gp0>@ij?7|~UOD>#{aVVfjB+xyal5uY3M=hr^Tjs+6SpUoB#bqEdrS4YD&e0rEn z6?950)WS};rnQ+S9$ntT+ieD34AVKq@W^qoa$! zG+>CyUwWyf6*;YLR#KC3S62TuZ@qdSW;6;$@8?KfD7PCP?Nz<$ahI^U6G zK`}Afy_dmVomnK*cL-RRHU>*SdU&8+odAS6 zk08CrW%m}a%prJc#5e*C(;D``Yf1&Gj!zMK_Y$1;89ZWe^iCB$!_I;qZ_0BZX~YGd zD!5BjUNe%eY7(*d+{YP9K3A@Gr_}?ZZEImyx@Vl_p%#~tBwCdb^IV?5#6eah7s62; zp}qek@CdNhQpbE5Fj4dn@=L38Lkj*8Qi%*4ZW*w1SG%50YTxLYNd_7-xK!xGn&u2x zL!dZCiMKl=*d7}+mcVYYBV+=38rtcr%lIZUktSftEg!29dCbY97IB^u@Pvel4lJU6=>V>ySnw8)m{?>KFfdfY z&CRinHNW-gxUZW+!{tIXs36!5FgiF3*R8q-%b!rkz}FC`iAaMPK5}<6BEw&4z`^oZ zM2+Bb3d8ypWF0iRXS(~=v`Oqw-ZGsoR=xR5U{MGGZdp9^;CEZve1S?4cCQ#b2o!bp|$OM;_{CkwWrH zaiitvJ(j$;5*puuLCX$F>^f?Hhb*8I%I$$Ea3~kwPbnCblNTKfM+fo83I>H`4=}2c z7idw8EbGhri?^rGDEO>=y=DUraUFQ;#8H3eQ$ePfr0p0{CA@eIe@RlZ$V>t`|-D(!|emp2rXZy9UCi>xFoFK{I`|niK;? zMJNlb#(xO!0YD&0vwoMlU%iI;_*7t$pVLKO$bm5Zxm%O9xJrhK*jR?w`rKy&7VqGT zRu4*(FXWA5s+jHmB$|COge>JceNNLaje&Yr+5F^sWNF{H`Ai=ZWdr)KR-{fzuacf^ zYxV7$a629z2EUxDYSF>FQXJAO-zX!bJlgxSiyMFn6cJoKn-GsuMFsrF&UlIz>O8#z zA&DqSGLltzf(me;81HKY#LK7-#7>6xVe>ZfR{Hs~;%s{j z9rqB&?k7p;=`Np5fHV}A8iTEh%QVV)qlq}nEc|oT-jZ;IS3ax9O|Hs^Sm@fAnc0}g z`zu&-ocxZ~Dgs9@YA;33==4+w20e#xYY()1VxbIf4>!Hg=f1&+h6tviu3lqfnY(AM zU(8z;+O1kwN9ugUacp9FG_tom16?)SvjV;*ig_j* zTJJo(KLwTufY%8gI$YrB24zajg1Kl-?p~F=Bb=@6&dcZSEILo-`8~ z!a*N#kyg$eSX(q)%qnxI6z_0GJ zSOw+nUtPdCqR9uWM?m5D_(dE6rcSsZ7+Q`r@wYnidS-$?92SH2VolOW@!`V|y(W*J z8t9*E3~bOS8qWEYqM2=-zsLZ&5DJN570eB6Q<>NG!}~<~t&v7UKCeTog(-&21ZJR7 zp8KZxAAbX;2C7|zc73>v|JfFceuBcgtSCP}q;p5$;aY4>`73k@bPA$JYFm?$3^6DG z51>?U0?OUKrFc&f5;*PgcO1@}km{-NaW!Xy@56Yh8SJZqa?{zi(ftE5&Bx#J2+VOd z?IGit)RDaU?F28Q^T)O2`39&T_mZzb>{P^gvl$C9lux`G0fwZA2$|q+7v;`XS|JYAc zEHrqw$AC{B+kXnghoJ^5l-N{MvUanFLGho2mpAnHSKg0bJedB+yUNMF1}2(u(6ecb z^+#a;#2Zr(dM_^Z=V*$OmzOslYS{V}e+)2|&^WSFd5B3(L`6j@5Xg&7Z!XiyFS9PL z0nuAg<@TwI{ppeZ9|i7&6{6o9AGT0q`>Q$7x#ij(sR}FP(e$+>DFKiMxYXziXs`DE zfOHD`h=z_%V(yC72+aHK;4@kd!8uKSc^k4PWLlI7eriUklSmW z@u->wT2!EQP7|9|!PzMh9{og`Fx znCFn^R0zee5jInm22A^Ga?Cu30H;aW?SiYCCDv9o%#c5Y%|#C_Ee+Ft>+UY}{~QZI zUa_vZc>wYoB?zP+%x4vSicf25zKdFoq$)O4j`v(jW{Qpm@6ecsoYB+`0Cq{#2y2(h!~Ng;3D`DGle$KqU$Re5 z4t4Y9HBg56^Wpm@<97f9sO95s<{R4U)$+?Z_)HMjN%83ro~p0_pvwB+=PYZ64~Q#N zynFt&;ilytjO{kTTm!4(pj!c;^!P^Y_wTcnrKv1>)9l*i7Z7}&zrsG=WXm@Wr%|la zeq@$t+0M(&eJb<+M}%H`4nwm=KN0`55|@=`r)md?OYUtr5i&9|-uw3UO7q|&+VTtvR{(@L7LC9O z;2!i)*jQV4mqf6Srslh=KI;VTh<2H4Sd(epU?IaPdkt4ODl4fXX?qIf#f zc9k0DXs?=t-Mu{6kDnJ76t(Ot^UjDORsgkO0C2L zG4vXs=T%aVo;ppe+#iI>RS4>>4kg1GRQ)YUDL1=TtEbV%*m%grqB|`3N#>Uhn?BRZ z_}2({i8^~BzFisgVNeJth-+!q8_ZA4Tc(% z^_5lhO+Wa_rgbp{@idw_M?^gIXs>q)U*@pXJ*WlAyX{iQ##jH{mH$<^!OVOA8ql=J zzFi#_++lvtx*4#J?|nBJXHlx;#7PnW-HBKqP=K3UamlO)w$;O@r>sVcroKvj;=z4k za70O}iwFDe1oxv5Xuy&5b%LvPd)k6N0TZ3QKB7tJcgwyAjvo~w#&o+!c1uo8ZT?zs z>4dfZs+Z#ZOMyI5Epc6Ex74`8ZwAAIJXtE(UNCIY-V>di#bLo7TZv&?`(jhUxA-$# zTv_hP%SX3aw#f?2a|Z&B8P|<&S_c+qv^Rc{N3r%&Mnq&VAm z#m&$CXpqT1N}YbZ%yc z59?0d8naaXT6&kM{ZPPdc8jG)xDeMcMsR=T^5Au-l?O&j=kQ1hp#GGM1@3z;cD=H_ z{X=!4AaQCk22f;~x41}%!_ho!1@vt4j9e)NHN9*(zU?_DDx6>~a`WhZ zCZ)wWXr^NfY`2~Yaqvi#*e!zyf|#592Rc}gMrdf{DF%K_wZh*WJ2Q!k6UR-bp2)lf z)sHpVQ^&Ors5qJ-O%!U6-%cwcRFyinD8+T2{ija@a2`XUU3+md@()W}V(5tNp6_7q z$y)ysl_$?xItlR>>=Y&{S;Wn$a*a#+Cn)a7B-hD3)7)`S$3ALUj z4huwK1ey~5UCRB|7$L|n1?+upg#nxUR6TA++bIJQ6M|FAj3@x&2r$kA@8zIjASN-s zLKN?ecySE&UD*=ZY!FU#QTXxY?X;kR&mMazw_OZh-(mT|c+ibgaKxQ!-48RM1*<7L`BKa^`vjtAqRfCuH|vVSb{ zGviK?cpO9pivC!WzNX2Kl%jevFr8I9)Ki4JMI@}9yj^B?d7_06mm z2w@FN?fL#wCjf%`Cyw0jJJuk|?-fvzxIL%R0o6Hu6``SGMNL{4yjzl%t^((td#~!# zKy%m@K4X@uDyA1&%R^<6Au4^DqA*6!XU-}!ovu$DB6r6xt+B8gA*H#?5AL(Wy{_@%^9@hAGbtJK58qKR-9gEd(@fl_NZ6%mYA7jUU zNO9~v-#g%n(<*^eKVyuHd&Zw%SU5c)`V?6VQfo#eL&)Rg<119-jM_GGf=^kY1V4OJ z<+ivpFhpOX;2>}Ry95xx|KLVg3K-LT5}ux-czkQVKziizsi94*F2e6U4zvj?p3Qhp z$w%Ylh_c@_rDX9e{x&^}ZpLRH8Cuu8G-5N)k5t`55#N)?tT@Lcx{vw* z6a_yM@+S8#9x%CmFc_@QfjU8RQ(H%eRIRy-RAf7Ag3&He_FOFlH$ui|JImAI*q{m__0z_Qb(`s{H{DzE=ZLi6Q$hTjJ2FY1xWNg zU2r@i_eOt=;TL1S?T@ZrOX#~xxHBDYIiO=T-p@mM*TY#zSAZ3hhm=9o?S{~V7n46SvW+2{t#+*US< z5+`~S>&S(a8uRG79Gx8^Hi_ZYi8)U%K)p!yu^V$f&SzGA_qtByIPPcihDMMh${%>m zNuY3+qD_a|{%Jt;!^uGM?`K-FTie?VOMox8`W@KMssWifukNk9@)X8#dHnE48^aXK z(PMpNUT2>#JDj7DMhK@;lFm!!lHGLk%Kq;|4zL&56%~he51V$u=$8O+^|8ej()EiS z!GjlFVRB&sy-kQCgW}F-DHSu8>n$IZIYpp>4EIANQNw8s zT~-J!ms2a&|6MUtpojnzP>ha-jqL2|%Eki8qtsDVoe2ADZKT+es>!|TR7|-AJlB2I z9(t_TJC-MT55i8#+q*6w^o$F!p00;+@ZsU%gvTcLtStcQp<+NMCF*~2r*BzKhjjXItrTqTRp3Kn;P=q_Q{1~=%RZ?YLR-Jhm z%lLo|jsf}Si#syLCZYYMOgb`1Yx1{NWIoaP5W4c_=6nuee@SSWTF!i6l@_Kx%;H}w zyZht~sJjl5t*)Jx7T?aSl8R9yD(Yq_Sy9)-od!;RC~b8fM&jCsr;(>`e( zy8E5iC(_=}|M(>~DMeU!XWbIQ=qfwD*RhVq#KyjjKnt%QpO`R`fT4K`NMJHR(%$9e zm4kD_zk^UWr>t0lr}FA>XePV&(7_%i_(^&)Hi|^|scWa!8GDs@I5mx*jXT)&4UH|{V4LaA>KXT z1T(kS%O6~6Dmsq2?w_7cU6}Tl3m`YrzQ}`w1Q-8$5g`3|$zRXbDJ*zMtYChLGlWEx1TqEU_``i%fu z$ND<7V{TlhjzKT9j&&QHR?P|7W1^PSZ?*jeE3clgiPUWov+f&y6L|;ul_=`8N6eg1 zK|@=v-@6$WnqK5JNl|+EhPq*T76Hkk8}I9R1fQv<0B(P{q8= z>c7Yb>nZyaN?zjkDMQ4uJ0(gjA|?Y^m*4W!l5W4{_w=0z{hl~0dN51Hn&h*O0#pKk zR20WF; zK*ulw<6Smhn<&P=b+^z;ko$T_te%TgAA<|{%>V#!{T5RmmJsnE>* zofpL7Zr+8t(MFN)0)R|le6MG%FT@|##ixt*o{WppPr5oV;x9jLl08^7#bh*O*utH5 z0#8i$`#DdKP}xw2n#?L1`C^PjZAX-zHzf2Fk;s7!Qe1CPw+If4m;`q$nBz)#u47C2PgRDQyK0=@klUEf-D2+ux zEy#?TPn&BL=)v->V?_a{aq*_Mza-S)j^O^Y!0cFK@}Y8>j^c?F9V6`I8h)Asu_J}n zg~n(1d%xjE(WLG`XO1C>3*^CM`NgqVit?c-?H<~L+9wn&`v;@do$9|8 z1%};`qlGZ;!^T7ufJJmoM=&dN3!nLn0a%Iqh}XEplWtWW^?r#)LT}W)fzm8Vz_%rjI5P#QQbhjIr2l1eQ53Oj*Fv&F5nQV-* zhS@HYR=gqbZKObdlm0C{Q{(9e#`Sb9Q*AjDhw3Ww8kzdrdN!+~2(nK?3 zT*ECS^GiTJRg!y=Gn%5{ep1x(tg5Q6cyi5ebTWQjc78Zx^qkcpAIDbi^hi-#8CTo) z;t}QWhj63|(}ml@@bxP>@a75l;vnSZCA(upmAX^XB$TsSC`UYFcdStSerL5B5Zi1K zyl%Yj2D~vM(8_QSmVG|SKkt}^>XPCl6nHO|*jv>m(alQDSO8+e)vZe+HV*XrQh9Bv z&ZRM>DGTOIXbPRk?xa1FFpH4Y;MHN;;--k0MVs4ak(uHtB3~pKS=8Vc25GaY0-7A= zO?KpIhlgpclL87C^qVG&zYR8GV-F(GrX^ER8Rl1xIGsNn?1!HT62W z>DuX;C_$agP9^df!+4fi&68CXWh;gee}L}Ifem2juv)2@A>j# zyhko-%Xd;-^OPRTIrVQM2&GISew&mxnJsJ&A|c%K%N~F-cAC5D3o@qD!d9^M)Gt=|Co)@m=$q@xaRH&0zz)s#_lUQR}Ux&nD0>joC2;ryI$^hQ42_qO3r5(f5-$&eBlD#c;itL~fB=gCqyRx5! zSm=I_aXe2&-QxOHjj=RvQtTecc$8bL13skKa&`U1;gG@}Ic%piyBKC5__6X&DCHrU zrBmc(jqvwddd0n;rhf`=*;Kp@2Y#fDq$iNx%raavGGPLbe`cF=mdEr7FR}p}AD`>* zAZ6GLlw4=ufihfX3LVzl^fEt%zQN`jKbM#P_*2T?zev<>F+$mbTQOh9%cqsMK0+`xMmXYO$gD3jGgO!mHaRS_KrSIg#osJ#6R(qs{1QGVgbUTTQ|HTm-pMWn` zA7}j=aqTKRu7V@Z^{tMIG036x&vgaeYpSKTb86N^dbhE;6`E?-mA^hCYY(8u9u7qfd?mlqLAN+I z_F38gV0-yyM{J>d;;>S|b zwPPH`yoX;uJLCS#q*#8at6D1__=Z83u23_ou(~&#gsDbN>3s*ydC*BXi;Tf^^Fuixl*Moz?CUB`25QxQB|% zuvz0&O+)+;V7GXTwvq>g!8i{Qkoc2|-K{=>n?)$jDVA7=jNC_5x*v zq%54=jr_G;ilQKq?q1`7L~Pno>MJ@&7At^+ss!un<1T=p9-2X6-C=JG_+fEl<$k(m zbMM%|L^(-ycjE@r%Z&hIbl3})|5K&~9rJ~b$=n?0LE=r?02RG+ruTcv!>{Fg3Xe-K zKuh?!o(^3?0U&6+F*zUnm6=KP3}r zQ==IX+;l?Kf>`WtK#MyW5hJxf@vqFQsAx2f0OFTf3*#bRJPNzQFTT^z zqEuc-{8C6@*KQ_aLGn&#!40VWEf!+f;)cAA{a*!>;bhw%4{ZYd?7WMK_kXW|eu-UR zLJ3yj+U>>OC;n{ob|@5DRfU&Gw%F{#e10q|Ev;5Uhmr$v;wLA=Nhc{d0*M3fuz(1^ zmvcR@`Wm=QLGT${O7br=+j;{(ajU7v*$lJaRl9LjI&A*9%$2@O}O?;_kQ^PyFQQ^Lg=fTF?dBfIBUG{hnT4ja$;bW)@~!LI>hy>cXx2@c1e4L z==T(#ITwF7N89klEz0fM7;Vqy`CUpKYa&YnbB8NvX1UQ(v8HsCOT$)(LULCWlJ?<` zR0h<6uFqzCcv!E^omVbYz_BPXOr<)b>QsoahDsTL4hK zf4fQASFrOCowk7ebKaCsVL5r5s(g_Q_s939&TIqKIH%fvN^CO>vP(S8>-=B+*MZEM zJK?;C!_h$g3*bLiuQNgsg91@J3Vnk3fi$R%_gOPWe*e=0Pm70ZuRNw-?*fKsajUFp!g+ z=fm$HBZC$m?EwIuTiY&($OhFWs4RHsM)UH(2%qB6Sq&-cpBzckIC6?O=s$7R$R@L> z%s2}&K{BV@^^P{-q)5#{W);+J#!4#JZjHt!p=89o4#^X*Up%}BDB}E0)UqsRr9XyB z614cxww>?&XA$F@wSZxB+(U6e4^acFg|8_xGoHi?$NuLt5$c}TBQLAP=j(@cz9 zc%QDm1Tw86Gwd9j?%07Qzo#q+gIA8O*8mNKWp@P(xm^i3w(oApbmJhm;V#>OVxtK1 ziKKY{KGgXj%Zbjkxbi0TvkZSkHFK^iVHjdv|gd@Qh@^?{|U9cf~BFH+ddbxM`4%RT(5(9 z1_bNBh;l**!cJpbrqsSz!?j$ckLa&u;T52t zBJoh1IGfAvi@v!*igSLbQ`UHyrQx<+>*zpg8hKW8*-KIPOvVEaVX28S&=$L{eM;HG z_{F+{_ZPQb>96}h?_;nVsbD>=RF8&|~*t0Xz|3Vfj#bVV1OM(?WRIYB@Wcd|tsyFWi>=7w21enWo!* z2#S$$a#rM~-1|r4q)i&-Pi%ub!BZ= zqs?$XFGPfQrwRn4_Lu27G)FsqnF*cMRFW#dz}N;$WuA%76{UuzHW>_A1g->PGO;sH`*#X zRkVbx?7-k)@EQMCOPFi6*|tQ8uPPb6Pd+XZS-t_3dOWFF%wy%{w-<4^%+3`ihQK`S zKTEzpY2^?`*5m~DeQn$$$B-Ep9KJl>{xRm8;GnjcJEbo1k$Cm`u`=VqF;nVei^zVN zH!;cJhJTB7oP;;>&*xbaA!6Ml&O()q594kqGjU%v+3vJB_u z<^r-6FeklJq@lfC%ra^@1v(Zt#j(gYPNo23?e$aI8QAvSU{D^Z^2`I#)g9RW4NjB! zl4S8+WQ2Hbg08%lwj|Dh5mhQ;HID5et1(O9S6w<(>SJ$Ut0emavm^1_)rv* za6&8nPC93vyaeYJIE`EBB_zxmqxKNGVq}~4y;tYgqL4m}xHjDIhIOpR%^Vm!IK&i` zlR%02iYIo}5YzxkN*Z$4-#B-N{F#ZSt`hV03BMp7sNA?5z!?>b-C#+Oh4D?`UnU7*m@Q6x(?dcZq>7HAOe?kbwsGEtT}(Y&tv=;E6&;xbuRX8MNU}-a(q3z) zucRdlw|+ZXVzB#~*2O?D#RH{D@*-IAKIBxhXZ>;*6sZR5e+U?*wd5B__50jFB72{4 zHmTWCtZFO>Xk^oT(kq=rkTmu@WBQZ@jD(tv6hpKti_ZZXArlzVuhr1W_S{VFh`HwYbN6Ya_ z5u{FaC{8_+1_g`_ntS~pq;ddQE&_|sOT?r5myF0-YHR22W0B`>OEO+c>DSqOerw=v zV$eOq{B`)stlVVN{AVxss0_fKdgVgK$GgJcQe@w6o$piLKQP(4!-I`aO?vFzIcvoW zbdKylQ8avqd72YmRZ>M!nzSgxMObURb;6-d4@nst-!UT2qcs8R)9d2J zNsq*vTrDl}HYLZhlU<*8rCH5hzvRv^F19q=R*Dt?NechE4&oJ#){+H%ZIPE)puqwm zq%@z8fSpE5u{kdLu=IyKBZ;uRChjU<7`Q~K&dF8Fs>W?%{ z>NU5`0V^sfSR05L&< z(6-o;PZ=T9PZDvGA>}SYLAq&e28d=xu27k<0*fs7r^c|7Gpti$VUaU6$GNd0=vAtt zT}h{_5Hhs|3A5oYAIo*p%};R7o8VpV7&Y0jf|ZwAqY@Lghp*xo=?41gCHW(vj5i}? z z1?v5(q&FOw5a2wq;K_Di#X>~MMj*57`?7+FMQsxwGV=lQrG7Ilj2CUYQI4{;!L{<5sXe`&x(ch7A1pZ4u*UU)5 zzV|IL&u?Voypks--6CVi^B&_0`yES^y*TjuRhxtgl1DIQjD4xoDU|I8ueU9~=a!B@ zBf*KSem3*n!Gz(E|Gnd%50{ID)XMsYx@A|LnUkW(UzFZbFdcwFKeLL`(}@=c@Z(%Y zmByxK9R}A^nglkT+gxQPE@$XvO#j9Jch>?rjw{i~u}RGo=jQuMue!1Sz`3{c!Wy`f1lj}ux32>SBY;HB#U;DGE zh;*0GORTo$aNnnJdJtjBs=F5hcSC%NZzo}+*8lC#fg;?uk4VM)Sr!yj{`TIGj=iL= z8@b z!14BOjBkswuLJ3k!I|?#`7)ob`gdgbp++(M4wwej% zi{c*G0z3qOeIkYdEU_dxa4Nez1a4&VV{Y-?oTZ8x)BtZ0OZ%MUKuTIkzc5tMCgRYl zXVK^9)TsnvMY)qr_#fP{lu!?8h%=PzO7<|4!XnBA%!YL^7Wfy<))K|z6A#oJK#VCZ zc#rDYL&w!fmC(wyOP!EvXI{|!vNx84=l5zw1SbwN&FT54)W6ZLl~iX~wR_0DR)CP| zT|@(OyxkUSKXS;}-zOf_YuzA2UV3vw6J6&=thv9?^@%VJ)`x@G2EW)NCd%{;mWGCl zOsEslLkzkf8*Kk}K9eHuL*tAUG_|?eL;2I7<*Z#GZ`!o|+T%(Om0!;BhC2e90w#(m z=7h?{Hl_4cOot2SoRzt8gE?cGI_IKJ<$~vrC&J z>CSk-_~0^IG!rpQQmb#EZ?ty|`GoMmfjM|(y#=d;MD9ru+jwS;BHg5MBmIO4Qnei9 z=)(fMD=bY~QW~=Nx=s#1E!ZwIr?OVY`o#2mzLa+yyL%b~8+EsdpYP*q$AW{^$RtUC zUq?E{fiwT>ecRT24Z=)q(ReG+4l(m0bpFJGUAJ{sE9@irMY!SB`%)IhVO!h9<{w1e z$P2xPFYU=K81W-OJH5s)z-dX1yqqI>AR@>1GyB>I?|zY5-p!!|w&%uq(x)ei#Fe@v ztUv2(TaV?O$ZCn9CGB!YTqq;ZITI!umBkJB(=`5JV7w~`GILRj8PSwFX8E7nj?!7JDIgKB>alkD81v{C(amVXGi91^&`#q( zItZrYfDK%Jg%-mhq)y^|D^iIc1TL_FTGYlW6PkK zX^fJkgUF;Nv-)`5ugcIVhlk%b@jwus6YP>?he$p~<^A|GN5;R(AG$r4=9WoK_q{%h z;GG@0arv3u)~#0?q&oP&p`0I5YWM^93XL~~Go0!e>`m-1?VTDNPJH`~Vl(Hsu%UV1 z0Q~->Pb0+uP8)L}BEI@ca7&NrZ z`I5}}K1d$LmE~3IEN}kw1vqQ2%g&l~+;@E*%-U1B88Ze6%W#Iz6*Q&QA)EU@h7oz= z=Wd{Qugl+MYtehhw|{JQnKJ=}=(U>|&RTEN8-NFf4W>EVEIo0XWVAf19Ug!Ja?=zB zEs|?!Ic5WxG$0=W)nu}|z=dZg#d}0{(cUJ}lxBd^?zJS{KwQQ!NobN#@aD;%YN^n} zYNc0uK&$Xmm4gVfsUpMi-Uj~qPwPKxcs{4w8}V0_jXIlnqK?^|^if&We3~x@k7I#i zwMFH5XINIgGqOg^{g={jKOIo7Slp`6H};Pi^sl#9woeEK8Aru+W`!h?0>fQ#oW{`^ zqeWdwvV_vZ*v%xWR=2xG^NZm39>YvMMVDIy%mWLzgwLOWK{WQ@wDcihXA?b&r_T;#E6c)r&8~J@XwCT#iGPgd-KiaW%}uho(hoWm zKsvj61M0s_=5i}vJlBU*(+to>5RUJ#x+Q zX~P)*On=V4{Ll4xQO6ojZdmp7=YKCox)q4fz)absFim7E1n6-Iq=5+Aqu%*n=^Cwi zcJ;H_=4IuFZ(Nc5qRv0~m^6$aF7H+2kUgC}EjV!g*vZA|A_ZP#X2=re=vaoX)_Ie( zR4ZnOZ@DdPPZX?e5gj(L4|v}Kxm9KZx5yuU zl=Fbinz{W5Tu$jvGtdq>BI@~Vzu!PX`Y-T~4dR@qcTjx6;>IP)T`r|QT=|3U2r|m} zo2Uof%Ru`#!@w>UBMf>YLFjg(DZe&D+oj4M8^0_IaIh^RPK@I5=k)6tfZT?Z3^kN| zqDwklbpYY#-z(>l-(dfDm~5U+H%3c4oQTM;j&Y@bQo%7Ce3i{|`amtpp1Naj!RswYmQx=u$vXOe~KI84^w@9=OHS zg2{Zmd%rTmCQl~VSCno#b#+)4Kq)rhC?fPBdM~OC65Lq?vEo5e@ZSGl!XJ|amiV3<0$*6>4$N7v3l_&`adWDx@K$#q9L^ckw$ zZ=<`UyM8zNJiq6C|9|({f2-?r?LOx|=Um^|uvbp7N^z4`Qa?CPobXE+pB{YNtWI>j zE8acs=}Iu&6(}J`|Ne@)1V1oB0A>QIR0hE2m_U&P|H@R6Du)iDk`IM9680}%j6BtZ z((44Os$08#1&JT~8ZMTdJF}Y}KRFm26wT0K&G85@xU6oysjeV#q7SdJ&o_%j5vD#xOw1xTID5*9)2y&uYTDlF{d4>her;%9DlQ z?sM}6s^lO-_~7qfYYPJsBb*&`5_lGz$zt;}zMU1DrJxv~t~uM>S`iP(q+AG1wC6PS zz(1X;g?;ebpM^pX2@5X9Fu}xx=v!@2_C^;|Id>PD#k#X;h-V34Z1;A<##kSaDU~C> z8r>_dC3P+6fa=A#RhZof}II zU3Fg56%XAJP?t-v@5{r7TJJE}5-A!iG#WrC4x-Gv-ft#c;k!ox(0Atfy5$VzmH93J zqrc+#1cuB27_~cA69@!iYd4nf?S(X(@4J8-g*#gX(hQ^(8yhr|IdP+1c|~uwAe(pV zx@)fYc-TrGllL6A9|W#*jhd&qcE*p&YeLnwK2_&}0s3S6=80BH#09YXIBFOpDQX#L z>t|?Hfls7CY?+%#5R-e&<@0(w^_XS8d-{|!SV-Hirf+a!(lktP0D$FAMAau^*zb{e zcPgq0CoeiTw%b*A^jY}+F8Q|E_2C3aaBFHrTwh4w-Rv7lJR+a=bpCjtUmxT#uo&*ipUvr z5YBt#w0KkM;7kw~rk$?5Q8J|=0;DdP{hRvTE ztNRt}|A)xTo)so)omT)S#1BY(6H{+d@H#J(I+RZ-%FNqjNKC+`x22=P$7Ef_qogZm zQWRM$WpYQ9^Ls&!7nD@lbCRpyja&u0lDfQ{LT@njkDSMf)J)WWKs>Wh~ut(`X|FtU6h)qDJxP(+kX_IJf*$)iOqr?D{@| z=K1+y7J7Pn4eVR0&e!HdnV!ZWIXk3)cRg+f>Ua|7oM{CbZ85g2Ns&{VS||>A@tHwQ z{0{g5FJtApZIxlGv^2hJKGHG{KX#}UqWfN4e$!*n#LDZi6ktWJER;<$A{ui|?f=of zBURSzgCO45H5VKV4E|11CO`C(&K+;AX0-LJvuQN_W!{5wEl+i-p{<`4b2eTstJX4g zQZaS2)%lsOR91(EOn?G@&7<#DveU3t>N@Jpr5S%L81u;{-X$gg@?WzI#v2ay6dyZt zTdj3Vpb{6_{}len;Z`)GP)9}=Z|Y8)58oagZEf%aeos_p21CdQ)+c3i`GL1i)_gS0$Gth6BOka}HWGusKzK)3K?=wAFAs7X5n5Q*UFWAGYtM!G6SA0 zuw-IKv9L~P+2^>7Oy>7W6%a`7uI}rcP6wV3F)$cQj~jBBzq76W$ZL5Hr&X&Hcy%66 z1x&T6{Z%mDcHru>!_5kYka6EyvAvh(c*#f6LZddGlGa0yQ?S1i-qd@A6IT4Xe#mRM zJn;sas@b=JxHyo52Q|S&AP%yk2Nb>+i0>AE>PbNQ@c#W784SwOmDC)RDF}MW2(+25 zzK5zKc!Ne8jvb8;3pN+;A1e154adnJ!~_jNRci@S?x@DDeGLcijpeWecT@@=jAeGI zDK188_uvdou{dk6aC{zCfnn?#Vghx(*>B`a$>-ElVjg*Y2o|}_um6%N{Y-l4y-Lci zIprv?#7KONQ}(56)mH#m#@1w&x``m{L#3jkD!f#8c+n1VYfwT$@*gnd2;-!SM6x0B z7FaB&)mh1>;P4~@;$2Jld!cOC0-XE5q8{F5dLH6R$nGH-ha#%)-tpJrg7`=O>!?e7!%#roELxs?U?TNfpNA!K*-C>%HcDDe1Mu zJf|jw{Jkz26S!zXh8qj7tLL0wWGkj*%^Y9Xlw62edLJy}o&US`%i80|zNu)t{Y(L> zKiLyv45GLv{S@<^C^!u?Vf^_BYRG}kvPGbi3Q~t(onSwSPp)lN9u5Eh3JxHK;Gqdw zbeGl?He-g;W5{Dz))pm7pHA$_C=v{ecKMJsP7JlOV1`u+a#Zy(FHTy64Af>k$ZB7r zNBUaV?_A!)D8>*vN6l2?^ZQ$Acun32QR%P{#364PGXErRL-`d2hu=L1+jMDZa5g zCb)IFc@sl~0Ix(2*w&h<)1Vih=LgyaG2X+tSE+qW)mVpN%EaxH=pU?8cQZrIH`

    O~jf@okD-36&J{pTVrRGL5Ees;p)qrlghvj{`LrBDHk#T+3RudRjLEW31)?#@57JrR%4cx88LQ5-xHSTzgv9d+U9 z0GI?<#SY~7>n0-D{Q_APgwU(&CsCeu9xR;+&+pB6?p4xQ-PgL&wf(tYq%ZP6Nj2bZ z_nBe!;R^E=K31$F$(eT|POUDJ%5K!{n-pY;SA2iIA!nA-w}`)YWBoId1A6$nF)GQp zue}eYFO+WQd`+p)QWCWihIFkN?^G|Gf$7J+y^x2T!;@QiBxG*dJ*cg}TfckD%jH9t zA#WF5NoLA!XxP?RvhJT&E#^;FNH#VsapX1b``GS64Yu~-RL8uw-rRQ+WZZp4z1R&~ z2V(Fa_RS4Iotw-N_oHxA!K|EQA@7gIffVrRtyhd%9zPzFc_ee{c-@Dad0|x2&w0Bz zGHA+U3)t_>l3VMmWnWaxyZvnZTFY@c-EAabz9SaaG7>PiwR_Nh=b+ev<*o3Av6V|=szn#3tN{#Ep3KGEBYNPHX>Cv>Mv$zW$_Lx``xcwVTw%t5lmCW z1e{aW);6p4t^QKP^p$5C_w($$7#8IG`AmDk8Jxs8ar=dFByDI#cPy&Dv}8MRk5XLp zS3wAY^tgQuN6^Ln*=Lo*SXT*1J$rVG46a-1&|7aG%nXE!$y5x>I?3+)#T!)xuXAg< zFkr5=2_8D1TjiDS6i`bU@EOo?Yphh8lk>NZLfzdtNsiMcx{b9d0_5ERt&-R?c+KDs zV_douCyfctLTR4b&mI#|;lqjNWo!Vc@o&w>58n;!U(T(E^Y)tC3@zD<3yal4;e*BT zNVrG9y>$~KO0SWaCvC{HN29|JB7*_S_#wX%JKZE22vmxaVxvhUai)D>eddAe`nX zr~26x6ZYlCg2p75P|2eL{huWIE}}!JoTouog))9c_YY{`qRyz5K{6L?sNNR22&&C* zb3x{tt7VG58q|gvVOROkCXzol;Vi%%4LC*ULq*=t$QCv0&w+6bpjaE5JIP0dN4K@# z#8?&0m<r+Qqu99jh+SKI`ya|60{P5P!@YrB~^3ZH?M0l?HBe z)HQR^9fDE1YHhT|jw*J89qu>pNyb|_^?`*{4^hrLsZajEkP4^ho*Qh&mDK`9d|1< zW@vDO#%)Hi{1a~XeZ9jmgM(vAt3N93qV_Fl$|PFO%ZiBZm0T6gkI@GKxd%b>P-76H zk*Q(udp}C8f2ZDVC|4!Fa^SpRJ+-v@qD|AK+P=>8w0+fbG7?H^1_q2yyTeg;A@nV| zcjmAV9eza;ujn2~$e=keQ{;i`SH9fb8useRm%*loeuZNZY`t}i>pYj+?Er0ym{6gt4!F`$@KpSDhijCDO_ z1U;MA8)gPH_`D%!-o0gsG?i)d zOKTi}Gsrx6mv97y+j_@D+cpZAM_qx>v3;nXo*vDmciS*m%|#8d(IQuJaHYD3B=?4^Y8{^HX*&=34@j^A)1#{&dh?t!PB+K#!&WT-nD>{* z5Q?Btikb#M6Ka)8hPHsDDnOxtDxx*0q~+9SumP~5{_f>BB-^1j`-*r6pi)w9dRJ4w zN?}`Yub;x3kmzK`-qtv=SGJHNOXIF3NA@@2rKY9fQ;{2g2_>iknSdjqNWJ3VYPzES z>PUde772Zphx?n-J36rWrbDCm%!d$ecszHjxhBm3o{WgdRx+ z$rCD#%(5!w!YQwLQP?}P;_<`l3knoIJ-^5F#b4)wYyedDsZjUG!K-wGz!eXmLsT@* ziETGo+x>{MwN(~24QqPF+ob|$=i=d(s_e2qUD3Vx^}{o(F+IZ{DSbu|m~N!@Fu88KW#d|-ic=OUh|I~=D%V8nBL$Q=qbkm=UpA*PIj7+?PP;?!HK(LLM}ad+Ui031+kn@?lI?UacYZ7f9dO#5Mf5Ad3n=feVm6&=*5n8 zH6Q;wS_%HG#J2(p1pZqib}}V=&U1huVGB{v$;rvP3Tt4BY6?cJmo#V?V*2}W+=$4< z-@5^7?gJ1d{Hifmz|!LR>vu7A@EGk7qub;I5K=(MYN>>gpgMY*>ue?H>o3(?MbM*wH{qhwlkV1K5hr>7U*HP5$UCY33joa6nw zse9>QYF-nncU-aWVPVO$tp>6Dom+zOnuUKMhx?WbK znU7vk$O+jl(YuRNdzouW)dZY926f)Yf)0=^#PnxTESGmX8}1KS(^nI{;)B)U&kH*z zbPUL6AyBhsMDY50JCDv686D6chAx@uO{a?P!e^f<2tnfIA6&xalsGmOp4O!tyLkjU6lUtizG#-^OC&?u%O)?9i1(rHrOw>4lSBuardk0}E{#X()g& z$I@*zH9bA(XQIV#(FeGY-q!+@*b>MDYjwJQcFaAcKkJO9tfm` z{aNmPQZ!uQL~JtfFQ)UwK_lz>0NX^`(i*OvANL-%U6|Wgao|6WZI_l8d?eeUlB* zH#c91t=pkH%<12%>i1K(C5p-nIkC64#>JyN-{qhWi@t@X6h|L46OnSQp%s+nf0iqt z4{&HA9)B;69X26e@&x`8OY}$ta2vodrSlKW5YAbf+@o!40&G^}40w}i)80O+j9a)- zbQ`5}y}TsIouwgGTMABlM$WcvbosW^@=!>+sw(!mlR;;Y8E1G`XJ>~ZXEhx^F~x7P zFdjc!oa-XD0^8!>mDK`!$)vZOM}&+`Hm+l@?3{$%?}r`r!0J$!a!Ez>QWdi2@>QIG zWQR3-IokVXBnP=w>Ps$)`sFP}j9+D9TiGC*?S{(?d>|GD9B&MuvH-r|*N|uGTlWYT z2`g|2rP8H+FBFY_f(Ez+&%!!(<4%@1k~J3Z?Z1Z6D2yE#$5e0l8Dmi5?IC zrCl3X0_-3304pAL9<1A*K}++lBiD=6L%?&`-sM53ErIps!dtp3jxIOF-c4m`w+D3G z)ywPS)||J`*Iw@tN%qM3SX#OTSO;ieOu$*06sxy?L= zk=w4DD1=1-M_5-SOF&{>v%5qVY2Om`tP~JIksKD9EKsN0>QHbe z`3TnZpLW4{5vUh5NwPeKA4mQUp`z-)#oyNp#ce)z&ZWT>aN8j?_BILQcZW5H+1uL_ zA#r$QTG%PHz%$#tyu85*=U_50V|h6}zJ|X>yhf%;NHZ(mD;pTh4;|cmonMW1_4d&t zdWy@U-}C@)!~b!7+c$W2d%MzuxMlLY9~}P{SOkulpt>WaOwfZ1ya_4gyc_R9k30M_ zgtur-7HLH&+9i6}>hM1aJ6L9eXox{W)~c!$L;U3Yc!!_M%e4vy84ys`UVd@lbc<~Z z*xc_FCmv0(rfH)i?3)d=kb7=6T_`<;`w)q*JW{|WLUxRCh+~LzC5fnqaG$)Af|t$# z^3hi?4~QO#z}w1nDPz>i$+Q4@?jeTS6VLdM0162o`oRBfDURIav^cK=$+Uf`42?69 zY^k6FU6#$qj3>-3Pi2OMN?D&HydMxD#|H=TDpiS7)#(i6SiNN@n_w}162pB2sWgk2&16}Rh78k|pEd!qSzo3zX?``D907aV+YWO4bA9fud zwQE`KTWZ8*(MI~-)+-|fe7LM;iD?17j5WmHE0y{=`ZZ7T_Kd}}>L`0+sd0T)-X1IAsmY$p>ijEqxoaM)O z4=+ve0S*&w7_lbul{t>_?a+R|AE3zKC<496Z@T2x;4G=Iem^Sfc^;~!OZiXop{SH{ zB%LTL8=KsbmL@|$EFoZWwZajLJy}r%0Qv+MHSkIj9t+2t%I^IT%L~HyrUE}(BH!aF z63CGz&=)4WJL0sZAOW#~*9lk3IN3S-O@AR*N=sQG9yr}kVBPTYLXh$ev-B}g+vm_O z4FVPITQ7Y%YRl*0l+E%ug-sLME?xxj^Jt1 z1OLMbBp~f`N!=V44VW7`5H%iDmb`kw%HlD9-JPDrYh7TV#>U!udTC1XsLJ^5{8K=c jhWqyAn8J+cE(of&M@7w+K`2POJSSMqj zlfBoOYfb$o)(Mf55ru=sga!fvf)f`LQUC%1ivrOlvMEC?_r` zh%e`0YhrF?3SO?I1wl)DELtPTfS1@_ULZx{JH`7RQvC@+#heb{ju{X z*&4yGe68qU1kn5hh5BaZ<~dMk1%$w%UxMwkKznw^$Xy@^f~v_|>Wg7V4UHH~^zi#d zlj%i&`1klyfP#+U1tg2pg7$&nKtOzyK+-b{LO=j}Lj-v{0`*nN*`AS<*8_GC%1%t( z5eNvG{qqGJ)vE#w1jGj-;eIE{nfC67~a`1oI6h8r^ssBO~zP}%&oES7DGIGp+ z*)Tb=UH1R7U}6LWU}$k*eFMV3-%WhLf4RQ=J)``q)1Ow>RTehnc2swrrL^+eiK$-BbS# zoAqxBG4^{USS&8ckahvrv!1!Cxmr{xZ#p-*+GY?(B4DH=8L(9NGf~B|W-!P+P2^ZQ zCpm?-w5?eup!-}z4JO$CE^e>vt=( zZ%{S&;+WyTi62lnZnO+H&PDNM5H(+whf{OBU+tM*eD_XF!CoK}fwfy%3%RS(A5k2sB4tz7bdcO$n!$F*zpUm_%f_V7YB zTM0S#CdIGE4Zh43T(uk%+^bweJg$-*+AK?}HW&pOr(6hLb{MO%)VC0GOAZ0!zyg-B zJ}o9noaP))rIc%Tz&O}4`(-_^*DswC=FIsJq)u8F2Lu%$nQeY4L#0cH0|gqSpX=>1 zOc-k=dW`vulLftYB$P!3<86Fc2?n@n{h^pL8}n{mQh7 za_P25l{vhL5i|iRGO&0J*}+^uNpHAoC#8Nx_0SZk1M3(IpieB@n0Jwf2?sPtw)@Ta zBlFKnbY>=e+eT$8izbk>3X0*(SqVCG^Vh<0?H^6wu|S6ViT^T9I|yR$16=5|Gn9B5 zVedOA=t`St?X?D(6rvyLPBJyji2B`}{*f7z2aA;=P3R&3o8@ExlHQL&^o6#m9L+VI z6~Q`F2%r6Y(Vo`K+D2gGFkx9<BA;vEL=Tog-B&>D(Jyoje`b_3O6+TK|99}8#Apn}`S zL!o$P1`)(Heh6%OLtn`*Nd09#?QuAEdB2Z%aFi?x-d8vVEobBb!Zq=cpL7myfyvLO zR%c$%6yuGwwU82HdTGw^m`hvZcqzDc4546z;(CXx;K_D-3}1y;qqU>MS{_Ekp!DoV z+^#odWTg0SvQffi(|)8lej}{X=XE)EL-ifg4K-GgKL!h!O}L$w5&6rMyOBi)j~yHU zQ)-`TOzPHPbm`bz&x=Bwd)c>?_$J*?-*1N|Z$_a!HX}!ShY|N(LO!DK7(n1g=?`uY zMfUMs<4e71&<{c!97_HG+3Qy#o(_Suv(OyJE=QPue;8K{YQ7tZId`nEY;utD7*ACOw^?c(eL~N@?_bjiw<@T60PO`>_I3m@i4HXOgJ)~EcfX|mm{6Ot{uB2> z^I42!XLE^siA$#a>MBIRU!jU~21k0;6vWfl0GW4wqlfGQ4zg^laLlV8HCt*99~A=Onj?joUifpbq=m6fvT}P5IXnq7Ste z9vTrc_dX_=)pO<=yS#Ra2$|Q?{o@gGfHy=OScwTN*seT;p@vfnUwULx-$f_(w}YzJ zpVMB_vu_@7+I{0xeM5RMUc3)usBUS*cBI9|R31b65`p?-7OlS11E#1_B!LNr?9-or z^N&%T@IQXHzE#a*u=sk4d9{1mq6xj&KRh)828g(MGxr)>mo>(R23p2rQ-iW?khw})2T)Q${nk4PLw7HAf#)N0 zZ^6*pr_DgH2B4Q*>1OS*sG*}e(KHLH%@!=`LUaf*d@qoWKHA*4#A-a>;bd#`HXP%7 z$(Ye5VvT0He|(vCSa-fAS_-VjTU*k!UTBC^w$*uD;SUF!u3F7VzVl9RxVatEf5?hK zEMS4jTvR!%7Rygcjt8=-(^irC07Qf@1hQc18}^0jXN|*H=3wCY71|L(+PKwU)(nvl zf!E>tO$2C{o{}M4IbtRy9(mr-_4XhR6un*(t{HCaujZ%=i=?DUn`?3FOPS z=ZWJhc|yd+WGt#4R~vH*N}!~vX+o&w=dJ&fRT`prdLaAAhRltr(aVE`_ahU@Q6vs zi0DgQQ{g{&IHhdOk(QY6Q)f@;xS@Xb-^XG>Me}_lLf}PnyggH6*TrL13g_)o3aL*q z>|?1nKa@N;o=fQx5vyC}@a7fFwJPcqfvtvq?LQ6&cM|gjj`q$Vy-pLMdChsy83>b% zjm_PVg}MQ{PKQN8k8?%wFLt@)94)@kw1wH+GSTTa-d(xqa$18w>yCS_$}MoU#U9jK zcY2)=yUyM@+h_r%Or+`CWgcvKVKLgL>0Mu0(mTWdn!?p<+xM|ytoFEmtJdH~GEU88 zR9!%Hx_WBg7j7BN!cu=MV6rCDLk(jpfW%hq%G#KanCo_f`ZQE;{6|x*^8}7B zooJC7am%2iCt#NhZz#6U6qp)!F@((vy-1h+>rx?nZ)C&R^~oW1GzqQ@cG#Lt7l&1h zqGg)fk2WJzo+Og67YNx-C|2EM18WzxoyHDLeILX+xUU;56y*$g8SJx%0Zk7q=T-e{ zw(X(5@#RAja}B)-eVHjg*&R)GL+fMsp=M05431yRR~Bo1Sy?JQ;bki|!WCq{v83U! z!`KW_vr6?$^U)^JSlt1fP(QiHgxyUXs@#g3wBM@ifi zFUGZ(G);Y5AaS;Bpx+wz)(kQtpk|OKK7LG*GM^76CN1tfe^b#nuKB27J``&*HF$Ae z>f>=nQivsb0qxd`S&}2360b_I`vN7D=+|2z2+5@Kn{ro-gV%lkUcV`?W?ie0s;jtg zp<;dThRd{rueG*gCN{r8WMrxyW43wK+{#hKCuP=Z!~$;doI<{Kmiv%3(fEGtj9o%K zqCtOgcB9W^E&DeLN`O+ATtfVMu0z*DOH(&@p$)Ase{vcZHgsY@d{@Nd6}T+bULjmX_m5U z77|;>1XO~)Pj8UOSEmqf-pP7k{q?p|u1UQHql%~zyMTp{C!;2dK)Oz6Z=)*1WU%w$ z)EmxsIJc*?aeTRIan)*S?uF-L6?yvOO-JZK zAVK;A2@Lm9A>!+G9`QqIidj_3^{0E^e#P?a9KvfmXocgA%-6o(zt~qBV-7lZmxvH$ z7T0Qb{P~hg&N_4KZUNDLEh#fkDZpV#X(~pgpdJ4SE4d7gg`|tB z+9rSH;wtL+i{=KWaHR|Q&_d#AbBP8gapO9^&HQGkQ$wiQ!@Tq6mznDam0VA+I)!v& zG%0~&^5!J0jbyPbjhHOdAsuF5eqal*tJe^vJsFx+O{ zYBhqAj2DN&1xJIdL0xnJ@rOE5gHjY$tbFtsbZfvfb(wYD5j?8j7@5 zwBoCbJoFQl)pj1k8P{<94O>0R*l6}8goOjwOl5^rj=k!P(I7R~=SJH1bHh_Tl4hHW zx~XUmtl*`{Zy~Z>93zjL12r=bJx=1e#%iE6pYFBfg$#*&WY~P>-A328 z?5If^YG7F=)b?uKD!McjY?(T~r$`8UN5r1)a2QXGqAn#coO4ol z!kjRZ4BavhB-XK$+pkk%UjT66ksin{f6%dqo?__F3raRAvkHY;DPAYINBb%jmm8m; z_M%`=k^z$l7Y zRDJw-*0EN-;eOyelhBb*QBYhm>-^HhLdm>u5YV8nByacrM=mh`ft}17N9TeFfF1z} zoFu*A=ZSCJ=*?)(r=EVi@iQ774Ez5;DqTOwQ>- zK6_?1Uk&0t_+_S$6gA%9P^^_pN1;*M?h7gUc8?|TkrT%o&3U=b)!ijlGBs`^`j9gY zIE)?rJ<0gmav+g^jlbS8{3EXkWqX=LRKyy@f4d(Skiu7 zAH05RO&6}es0nw2@jB|PgDo1G4T%~xBbHBLE1EdWpIlmrPQDX5O{Yt=92-|21x@*} zs-};Gx9>WN+wE>xE_u1!aQL{Q;oE8)pTONFS8`*Q;gpFm(IAoY%aXs-F*2Qzg!|K| zMe`}=sMEnac_b{%bBqZhYCf1)B1txvJu>FE=-)9*)vqQg2kxgwKUSXY@`)s+U{bfn zGNa`vJeb$@qLbKC@J8!33&iSUM43H&+OCeS>^62_YUPihzvUR=paKK{fgbrhVf2v4%ot6j+=<0LB+gXU>MBQp%2`z@xvB*8Z30!>qd8Gy zIbqfu*|@VQIci0PgnDEdmuF=c*@3e}j$x^+N{#$x34r0`FDby~1x5m^-+jE)3z0^` zTBzIltI&-aCkj#AF%pbRr;yB0iP~$aiR9+f#QhX&PDa;%v6}w2hW8q7tLEhuMDDpu z!;jw4$Qqc{!C+d$88JM`VX$`zWW-x<-IOb~#8l_pTp`(su(>fR!8AP=BZwv&(2=xk z!kOV9!yIe4AAs+X!7!UNahaZ8XO2r2B5b^idcS9NX(d!{O+}~oo?0H|bN*O(@VY&+ zTuB;(vjB6cAk=|1>og7Z$v*C>u+d}Ew){gjTw;B`pnK&})fjSY&L4?XUPbPNC4Bz+ zD0+2hE}9-U!zHce@<_r*Q1UafV<{}Nd8N7k=k~jTvn!sc5zDNWYoUVDZzP^G9uWq0 z3Hipb670ciW)*+FHhYLi!#QQqvfX2mGh2z66&8Zdr;D(bQpei`Pn0?an0IdFQhThR zbee#cY*u=QA+B*8iPoqixIw#uPE{wRFZ zCGa^hnJHR<>GMKK5aMZfNUKQ~?%=rhOg+5FdKJ`^S8t!Hh>e^qf+*i+Z#Z?4q3T$o zIaWZ^;cegXz|ajbE`_UY<_v={|zqd?MF*H zTl>W`(`BChctA}gjt~9Q=mXQHm*diU=m(2ou!G|k<+@wsEwv5^#AK=}-zbHW$WZZF zdglqLkPEs~543Is4`_b(+6bCx4CHjk-|U-Me_W+mWDO<=KUTS4(+0^RrSZT^Flt=8 z|M_hPTXC$$4*X|_d8@+-4<%WXbK_lrj@Id1dU1qE?7b$Z0_1*3-8aEp^o`C#A^adF z%D}-%dmtQzI=J&c-YiR1-c=3)*v~48RyuCHdHCVZj37M|OaPre7szfYohAeWe^i-b z2;1Ji^fBr1b(!!9;&O_)w5g%)Z+HW~5(C zL3+jKuu4mS=FVbeS$%Q-9#qY1>ozp6d%Sao=kaWf5EjqTl)#89YMjP+G~@^b-bBri zKQg#JCRkuaPO5jQ`qat1-A|ubO*9oE6#ge9I*ro`=Hc~hbNViQqM5hfk(%TkBDkS3 zeJxcRFK+-N_ZJ{4twR#SW=A!sFR(;@kKY{y>e`Uzye|48q3tdlbyz6rwK26kMb~|^ zcW$N<(){vho83Jh0&cQKYN%}-_p7lv@rlzHN zdub4em>VorQmb_sP@qj_guo~c5we3kCbi#)(((LucrX;akU-Av6}5p>k7qzd;YE8y z33F9JZ+!OkfdRIJ5Aq^#kE89!8|alCv|fp;>dT97=)x`wf9KbPuM~$n zgR%)a&$?8?6zJ^O!BN~Y{?+A5en34zk5UKwwF@EH8qO9~FD54|L7U+lMNll^1!`_g z!(}wWwS8aR6Xuy(I`aTE%XV-+ zBmIUO#mYT+D(K{R06=_eB=Ip+H%!~Jplb96uz>FsOLf&F41VfAw6~z2o~GcxVnAs4 zwA1%UaCNK6e{p231z{XqsWGprAzKTgmhA5f8W@zxcSBeIfG~-KE5bYeC7-m;4K(!n z0uOF+n=P(Jjwa=7ZFx0Gwz!#XY0KA{%ti)kL4h3e&+-VxO9;|y2$rxH7ZNhEx7RLW zh-Kaarlty%Pe!U$1U#2lHd0^^-UC8jOeH2(?e~2Z+fh{39^#^MD&d6dkJCh=TEr5@ zI%y&(IgKB(-0PC?N~+z#Juds;0?-zMl9G~Uc=~(WbyM^hit;ZZX~p^y+mHd?QrxJR zJI(xx2mI|48!Z(r*ZOAg*D7HXjj~;BtG}FEu%;Lj;0$P&8My%F3q%Xz-TbwWxuj}~(Sg?)gH!2Yl z7LiiCCq0gnOnpOk$qvwiifUf1L(|jdVk>J!hat!tk7%d5c=B|P;wzyoq9idek{trV zdwk0-vu0%l3T2r0)>NuXSHPEW*(?S${TsQh8-`a5n%)nunm&noCLVAY-A&Njrxyb) zE-SP|E+!b{!z%ieIBl)KKt(TUdnS@gOn>wm}Nd7f(q#!{AUAn$xWPt%ISAJND!Si8( zvGs#qd0*K)g^)QG53kKVo`#rmr61(g99F4n8vDJ*9fJi2ZZC!=Q%-l>Mf5;bOs81m z6C=vY^0+X?E2yCkGiiPBQQ9)mY*rt1soL$LbEpxoLQ7<2lals0tAs-i9e<6|d(K+L z*_|@Cn2WxqBg4IxUHc4px?N^8)f70vgi1zq#Y(}Qbn{VpXK^BcmXQL|y?-S#tbJ;3 zU)oI`9G^BO_By0GYy^+@Zl&%IH&_}rJeUhZH$b$@vUW^#gor@&_lxQ5seHE7B5fh0 z{4D8%nbq$6TKLriJ+EFN%jqp>m>&c(a=Ym45W>66LYtJbk00iTd8neEa-eq8!`(Kh zs@nk7JcxIfwT7UUJ zYQTKUU@{zQPeh(@Jf*r3*MfaCi%NXkls(C{!qnAe2bS`UQBAWex7Stoj)rd~!t+;P z%61ATQYg!Z`i+7`6q#MK3_ta$a#+LdkOb3S(i~Ppx{~x0 zh}9L6+K%dG%|c&-80|jRw&)KxJr&JzOE(q;kwG^IF`(lE5pUj!5g;JmI1}T3x0UVijku1geB5=xG+{zDN|!9hiRkLinvtK%u;?okPhL(BCAM(TO0NB zxMBew>rA~R*|l#$jMEV%Hg5b&i9`2AGD_@v*Hp&2UvJH5n!ZkN*+Y?pqAQZK%BB zR_MLtx80li8XLmc`n;cAgEH|ST~m|!cb^>?dpuZl0|R-@Y_f1RiT! zuiS3Cs-$Uvykd|>^MKZjlomMPK*%~os5vs~PgqHHnu4}UP36Wtep~H)nP=Md=okOl z=jMiWBAmcRl& zH7&H$aXs1P%-B;6etLl!I6w-Ev8**0j@qOf5pUF|=&G2y3=MYY9a%wI#|+?aMnLy) z*+H>l&iz3mjYi0ZjBz&9pSL;C7n%iayE!ApWmCU**eo|GPQ!==f9huaby!8@81-Q8 z{lJseUoVlN11{^;UcYe9`fUy`Y3DCiZIEJv(wAA=_Yf@)jocRLSfKztDY8GRppO-vRgqBeV+5%UP^M``CpJprCTokPQ2u3o}k3KW!X z`||^1Q1p11PkqBeN*oae-#L}K`qHt8{udkxz`|4iRn&5(jiWtzx-DDv&b7XzoV*8G z5GgwB!R^=ZJTnFJDK-pMo|75J@v7NL!0C;=)0(eL4yiH77+xB?kqSbc$_fTQ{Vr{v zK|THa3L{NU&A`jTN$c#U_KTo!a-^C{-vntI&~3Aug23Y6AFqV|#X*%yClUuSeQ7~O z+du>F%dsMTA&zr?V~1_ZCfY&8NB+!VYNh~mr>n**He!FRjsZU1C_ zH3`DHE0*x{ZhqPjB2xp$NH44`OHGq5%+uBZlqQz;+x0k_zChZpH*qcW71gDb(I^r8 z=}SkZ6O$AGk2ijHKQ~b^V6>in94>rxH+H2~<*dM*^H==p^9h0~o8lXV z?q{@KD|dY(a0qhxlcpUZy=lF&g#l3V3g4SpS?{jnFXuRr*Vq?6)t>afv#YM;S&U{s1T zNZbKWzNd1Nn-3|=nKk1;Ad=Z~4H9c=CZy4Jqvc?bf|8iK#`h(?6NbJh(`%vKM$7gP zj0+eBUb}0;v+&Q z^-DFa#1*EB6yzUn9*~kYGq~M4&-n!PBRc8k<*7FSV!b%(JPj-I(wDTQn0E6f-Uy_z zzWSBMEJH=Gm$}W!;^KbSn-78g)i!BMM}fO12ip`BU?m)H8If)M z6siO}H_+=Kkk2F$eDJn`7HQbNB++b{Q*n{L4UnM5Xxn*AqfkvU@}szn&WN3p;lF^U zvE692*KlvemV?xh!4Cc~-3r~rQr$(Sa&EM(eomu1<-{HYCm1h9?0sZk|NdEh&3;8f z=P(blTTV#S7L1WH1 z^S&johncJtTGD4~C`yKvJP&Z&3s|_l%~%u2h}P_q_RrJ9?`hxx}!hJIj#e1O0g z48Imq93MByw2G+$5$yVh=9-`SvJ%c))DS3V=0L!X;sD#%y~Lke8(f>{J4;(9tbvDM z(g|2}-r2@7ZPZ`BPTrB35byZ{I!=>ldas~hs`vQJi0a|bHvQD?f7<7vV=#9+I&!i! z*3j?vjwILLO0yXE19Gf_WwCPyx%RR#Yeyx~_#Tx5yX=h@Ug zbP6KPCp0uGnDKtie=1O`Cs=7TF276OL%lxW+I6lD*DAs^0+N7+jOgN@9RMFt!RCJe zEfJw?wZ>Gv1r#|Sbmu|uup!j>oq%*zsa6vZ@?qNLB0C9la$4%ypLWDBvi0^>h?c^f z6-zq%ZSG?D3M>r13`7|u(D%S?YK!h5k~#WO8=o@5eo0~I+Cnhh^d)Q@M;Q-mc-QEE zX0D>tslocwBr-Q`d~O50G)Ly;LG7kias1zptT^#$e%H~t?OM%IvhG*U?v}P8tXqH# zBtONqsJ*Hf>p4q0u3}aj4}id;o3eIk3l4*mVAW>85NS}cg75zrM?Dx4=Nn*wBPj~VNmUUf>4aF6lj^k(57DQ;!PGcmnPS-(kf;`hFP=c z$QzUR)0tnhWFWcWg090-Wc*z_C~OFBkiG7rr*_At?ak0T{slj~1*kCNpYBn*-S$ORdnZ@2!Hof(*ezx_KSy=f0LugDJYUq2s;K z>%Q)E+{-9bUM;<~xO)cStxoJ5LpN(UE4pSv+8vj_6c#U+J~B@GLD&4NX&ro*mE}6I zV|3taJEp|TI$-1Y`XR~JIRf|$g*yr1b8q8Rb+&BI>6WZjN3n=Bovqa~=VyHe%DM(cvjGlV|z1PnqJ9Hq?IX zzF+wrmZxqbhPN=NROu2>GIe~NtxKpavY|z8UHGeIv63kk$L1)|5o4%-kT+!S*#REe)LU8qvnkutPyrGw+OoRxuRj zHsMUFEv4p{<6PqDvsbj=I5v@^4AUirlcl1`he^SvGiF<)v}K-xyh3)cc&Lqg{i#5r z0?irq$e7bbljfvt9o(S9>LH*I$myksL3mduVH!*mHEoS8>mW<*CMSeH%fQCSM`v$sqlB>Xp&gBtMm z-nW~k*pEAWE(rXupzz-a)da}F%D;6~5)+H#MktdUq(vR?mTTKF&kWb;?V@B`tw^|M z6zP^^mT0zcj9ts|TCog}@^sDxzkn+9aWR$T(yh5ZXRhv((3EA5TORfRdK7rwxRVV+ zEh?z5313=l3P~#;_|a*ADH{Z{_Az3h^72^Ur$7 z6^;PtA}pk|mPiw@>QzwOtVTgKtm;t9z?F`79BP`gI6zW2O@*0iZC7wRObY#3pWE@R z0yHL!5?v%RepD9 z%PKsyV&RR3E6zti3M3#NSzRsb(tD3|KO-9Ml)!%nW7aNIP{BHiVSLmC{T5rlhdANL z5I{6ZMT~#~Xx`%A3Dq*V{fa`4LqTjEwP<*~$Wg$ybOpBgVz?~JCK>Jo*=Vj(X&|9B zEFfc&iEfVQFQ`M?A1KBCeJHA9^P$LepAxkb1$0KFZo`GdVrafQuapfkhN>G;JOUM* zR3_%a4l5N;*tSfep+jwC@`|`Qzz28Ok)4rDmtsv3OYRGU$DvS%LOf2o^}`uRfyhyh zbDE>n(30EaVPdi{vAf?mib484k+IJfh*`d6lR1EYnSlYmDN6iHBTBP#(BU?ioSu-P zd9DdiWd7kR6Y!$7;t=0}1*wY|5$ZM)S!at}9!)G6Im&)qHnJ`Xg813eEr9=%y3jr9 z`HQ2EaCNMBHFN81+UaEQ(>#l+bZS*gqX5OQy|gaK!gs6iKD3OA+dN;_HfMZ8;SoIV#zh9y(%Wd9M%(h|AxK%hZtY7rubh$i>5b*MJ% z#3Us-nqZyfPoGkZbH`kZx+-(aSzHP))Zk;wO2_7g!r_&b*)C5E_Q24llsuewZyOe< zv|SC1BT$JwalNZc3MQ>D=ivM>`$4%!fZ*Xs2(0uDP@1sk`wAk!SYZhO5Mjd)%tj$G zM*I3N1|E?Ls84zxEBIZS(jl)ulXFYyY$ZM{tTzqOh8YCfY#T&<`BP^#bjz%^w0;{a z)ff3BqLCH#prRyE1YLykG6cOJbXtn|1nTS3ZHi^a^~{ge6C-f=vjGD7qm~%HJE-8m zpN}fuKTe|lgwdXU1?lhz;fUHEj?p>tofT&@yi*IuWoIe}Cr@*!>IO>dEUv;?!&)Mu z9x>KX?DLuT`5<{%@`-%;`9*jm7kpRtk3ZSYJYt^xL34~4^u6%+N(---RSirz&07kd zvS(~vDEM5clf|+>e6Sb{utga>*c3luBOcT>sLrhVJ^-DT?W!Rj=HYCW@2_8bf_IAa zpRQ1d!JBiG`MM>-5Kzq8YC_h9j9zyyR8wTcXU{zvn8x=%HV85OukSbC zYk1Pgg+JD`|=l?Q`pSjOjG};5YoMh4=rR&&Wnb03p*P&1AJ8m&xQgAq|`$-MR_5Qzm0oVQ)B7oanId|)mz&O(Ruyk}7`lg{deO@IaMHql` zlK%a`jX@%S?qqHzD|9F)X%K7Ar9K-qEOEdtM&|+6+4G<4j0D_I<0{tH0tma|fOv$< z2!w2@5X|~sUIB^9S$0ecGwj<0|J^Of1i&)nM(6?P=6}fOzpkAa0qkjA0WNC!H#YzC zEy8zz{^Z0O{s&+G;>D2(Af9%v1C??7cbWR@G!?!QV0r1xiFd|36v%Yu>;9W~Y57;wRwQCIFWHe+kE5Z~wpd4xmtJICiOka{vF( z_kUgIAOTK|eP=)TcS-%{>^7eQ$jZ6R^dGYJ7r6p}mIw+ZhjG!rpM4`7z!@8tZgZNy zXH9?vKF@qO;AtFwK6w<=29Ykt*i0Hhd>WcbqcCQx)sg??CtusABGV+Ri|0dOb9$hN zrk&>0H;A;FC4K`1+uildG2JriAGk-rtII|KoiIfHq?3>2YDN@qr*4X;Z#qYCrc9Mq zfxw?eoG)|l9nj&Sp^-Y*Da-Q=#>~vj_WV0;rp?vXl?VlkxVf*>;`7&I$fwbEml>*S#M%)Y{$X*$7KKu`gw;3sB-Zw^Sp4?)&!CTz1J@ zAJ3H0dJq#6$OOYd>UD?4OYV20X|pW6zvzAQeSHUscPO{|MKgBt0knG1bOuR$Wt!FX z?vsWf{wk~IhWl|G0o(6aHuUYV275q`KIfFDud0`Umx{P5vmm3j*H^ypLA_3 z&5Ox*2B_jD{RIQsh`gBS07~OeJe&SURN7(*xn6t0Xga-U^o-c;xkN_OK~ z*C;GP$cG9&nU+#@n1;(?k5|%LjI~p>HR8j9F)xT89)o51cDwb(j3kHSs|@>S0Ru@(>Dvv^VxuCQH%?8+L5%DWv%TJ706}#BC`ETTV{W> zVLI)3g&`oj%Y@{!RB zgfNyAMBG?JvZ$@KJFm@yV1gS8q_#k}Rivv^SJ7VxGi5Sn@wjw9 z^bB6LQBp!kM)hgI8(viw@F-3zqy>%CJe<=pElx;4woz1QN4Q9Eu?A^ob= z9p}m78lm<*(QZa%=HSlgP5H60)|5`jQHB?T-5n$*<>Jn83{mMe`C3kcFvEe!f0#uC zX@IxL6;Cq$B?N#s7MA9{Gqf(bqE$S>mz;c9y2Rkf6bxcLJv&L2Z{$N-l_%fKom4{+ zT0f~@KeX@4k!pY!&G7J0Ot<=(^-|`gDWDk1c0R^J? zNH%pbZ;h}Dowxgauw2#3P(I}87-1gn!`;!qWo?pb7A8f-(xMxMtm8~UjCiPqBwjdO zPr5+%XE>wQE=As^w+f5or-!=7Gs9%{vADA8;22=frAfmYP^t#YLa<+r0&pJ=mL4_7 z>KbJ)miCNl;C<`Qlb-Ucmsj0@ww6?~mxeX*ny;6IQ=@FxV49~kQeCBPR(X%Sb&i6BN> zQ;EcJHdcNNW3phNuE6FdCSp=OK<$lqBGYau@#*tno|TU}V4WIJ7_*PhDN<&}>#(k+ zYJjSIDG1MBqWzBMQ^S5Lhdc(^z;#zgPeUC%FDTRA^Y< zpG81C{sjBkphEs&Y0oJ}G%BvZVqa3#hM{7SBSqB4Lx;M&Aj$mhjwa36g1YLU_3!~@ zk5-hdcFP?$d`NJ0{BJ~i^^=4&y0NdxekkU5QIv5gDtA#dLBr@1RV^A8(m(9?=WopQ zwP#~0EghdLdLt{9k193N#ApC~O7}=%z$O!+9I2m8NV{0N9lKd@6MWYQ$2@;B44-)y zNqVsSjRTVj(|qGaA1T2woB!9Twf~8b@ozrAK9fQcaJY6qS&c~CEB{AcB!t6vb^$$k zXVhQQxt3Iu@41K`=~tNhfM$%jx#HLR_Q)G0ez7NKeX&CN!ln1tvc{#F%qES@^_-*< zk*ANmq`Wo@_gV*0Po@Hi_#9R-+O?J~tt_3$3y;!``Y8n+DFAEEMj!|OGhz|?3`UA~ zjd%2sgX0KepTrUbzp>s4M2!BrSkmUM_@*zMQCA2JXeVpsRf_3V$ZBA7ZZb>T#4e)N zA<8H4$mz9&Dur~?H0FCet;|%W-Adi^zQ`<=A-;fp__Zp6iuvbXam~Lj0t5ld!FHF- z`U3X57`LXlw)L1W1okNRS7&fcV(-1`LCh<*yE2aO+f0~TQNajtB)akwqK|f7G~>LH z#*h@Vs=9}_1w_(S?!N0WoyBcFgb9L!Qr9L>nPr7sm6j|SHRH73EqL z9&y+R>|fE0lMVC8R$MqnEPDZ42shAvHUIu5%OC!APBk&9cW z)d?Xc`dD{qI&i1;DO)SARqZ!57HO_s?k97?W4j zp22?Eqfbc4j2fhNj6snCmXZX?$mE03>DdifM1W9PPrVJ zP3aw^@a+t(A|>eqqWPR&Fz#@>pp+{dQiA=77B|ZdUWHRKC4iR(CosM3qkPcoZk~oJ z)uYvsmOCSgFKTkhsIl1-QSKKpI~bsKpYk5FrMO`%!RJ;KS*Ly=xnS!BQ7~bi?byO6 z3CmZ7C6@LT^aYN0LOn|*b3W*6^pjrkOKcPAy+Od>e%9`gn2Gp$irxd>+>& zeE&aWy<>D_QP(xvNvDHOI<{@wNyq5e9d(?FZQHilv2EM7Dz>e!`nk`0@Ar=Tqkhz= z8a2*Yd!M!Enrp6ECI}?Xtx1meRg*n0qyAoFdVb6Q3Q(u$ADZOQ#X=EUHkw?e^sN=G zy$r;lFqC?=5H>~3VH_q2#YWgei>{)w6Gcc|3t>3oR+bC@RociB5Xsg94U`4UF^D6y z$bF|KuDHM+n;lbIBRaLK7mrN#5`Uj;B*jN~E)17OmGf|DE3nLipJ1=RE5&1;)Qc8> zbtlaWrlhS@%QH47J#(&CTPe;@_2TNY?V(=rs#}fbenIBHbK8;%on6zDfUCGVY%EYH6En8;M@m9R5Zn z;+fh&<;9a_&cy#$X&`?0m2`2~rEh;Ypic;yl1?AH1`bBLY%eXfp*6lNZUOs*t!5&Q z@*vw zztVB-zM*e0jnv#uH6Jvc{-hh}E=6&rB6ZC3U%*Q*l`D+nj<}OndzIJ`eJMQsIjk^u z3J^hN%RDIcN>v%;VxRDaM3&=`j8@#xF{pH)Yo^hJy9@z zDT?67qbSOVQW`MnA7;X|e-Lf9+F@C)ac!^<2vME9UH;fv`Up1+%Yk8C^g=So8^KV2 z|2*J1f7)OD>=~Jy4ozXdlr@oXyU=X=24rg8I^Q_7mjM{RcV!#o5Rsn`eyg;%I;TbL zc&yN4=*SEdl$SPcXi|86I2Tq?LF0eFQ##w~BG~e{{eg{*ZS!=chs$LLaeaM#v`~f- z7Z+y(45iS&+U~*R_uv$Z!1QG@7$hVjihy}Ix1>G26%wR|RlJHZurAQW^WVLK6$b(6 zu6O3Db%&*mEfOPYpbDS?Y=YoI6qtG#MrC?Pd>x=;`XNSr<%Y*xb$5CmS7_!nk@#wq zQAST!gd4cKI*%kanWK~Bz-Q^U;RaKChK~E%*vrH}H0PxewFy~fzHV}NdK_Wfs#_%- zRH$ohfAui-yWqWY^{y7YW-C#;F+g(&@#(B?YW~z-M&?J=6rMe_I?aFCC&Yp@U z_Zu&gqTS9j$!36Us{Nt&<|ppcOz0#d-wsN#_aW7U^K#lLIe?@1 zHE3U%r_JakPq*R9eBElGsz!Mh4YakBz+HzGtk1+qOII!%su*|A;=*SI& zzkHu+l<^LWO^fC@y=g}AmfS7Fi{33<4dIc0mzveu^XEL(Xp-ot%%k*D3)cIj#KJCe znigO$fl1AmIn_KnZRCmK?W($gf zgyn@qOi(9#%VqaR`}@NhQX79X4#%XcXBMw3bJY|Q&u`lQv4rv?vAK~n1|w=-|8*WM zn28wscU+w^PMZNz=t+7*=k;+XQ1f%I++vlWk|{#WPGl&$CjzP{_oxX656Osf{RwQ= zigzq^>g4(Eo^4q_W|(0tK)dX-P`PO0x$lVge; zljr+%wCa0NYFpa6Q>^vK695C!bMp9E^+xMN^_|1)0HW1l1!p`*XK^YD^0&(X!f82hajYiZlnXg9w&`WmiK9Wo!_iGhNR8kJg z-trEdvL8-N?^Gw;zD5l(?kni7^LlsVeZQMqZg;Iy_f{v!-ELdN2i_}xg@RWW2n`KA z0Rh$PZwX54d&7lQUFaBGO#2f{su~(F&(FIApCJnQVmphP)&mn9TMmjnO!};z1GpfoX*d; zVpj>=c>>Gq53~tg53*UF?{}Qw0MeI&k6&7Ux&-ZAiJ~5OwP6XKzXUraazY*~bSWc%^sg~V|^$z3ob1RtSnCM&MnsY_qcen{7 zJx6&waQ`;jbW~iNHGdfT(G1}^lnKk)>3=EUp(KMMaHD#MSGiDuTiLf$N?h-uL+1iG zk0;4q7Z)569rRpb;Nnr|Nt2X|eb_Z762cM!b0k7n{SmrS^fQb9HxUtlm{>e)`7C3=-0&QGdRM*Wd1nE|QpOGYgHRPN|P3hM-u*#Mg5Lu6*qn(;($ z;vD@z$1g`%D}?S>=B!wu2xmN!r%Er(fUvaZ!D7M!4j!35%|;c(c0wV$*H zH(8i@czSlUX3bLQlZJZCcGw@vI%10%-ty$Y_qh5wp2`9hj!q{eB^5Sx z(^B^iLkXla2}304FQ;+THa;GmkO{H*H5La9D0-iqVL741u7gKS=Kay}TDoHM6rVg_ zyKtircu?vofoV?-gp?Di3h)L>l;wjdLdWR@!&9rhhAKsgUlPy@ZpD`ic({|$cM`Zo zru^l9oHmPd70vpfl90gn3Ghu*`jjJb0$bt+H%V`nT5W?#iu0Ud5DUoJA zvdr16WqCGex@papNbs7oBa6!o=2m!jBRx-#{_$S-shc^4n5o=D$9>5`?O=E!c_D8j z_Z}L^4&y&u_to+L@Ey#oD)|w2RFGEKI?E7UgBt3PJL3P&jS{PHunp3lPE1Uk+-rn! zc2-W=X}5yfMo<6%08_xrLt^PLTHr$+kLtIMPSp8YQ{UXZhECr;n?@HugI06Q(jhdy zHM%IsEvefVZweR}TBk6R8u@2UGUqF_OPZoemots(mM2ygI}D8ptimpw3G8b&dHc%; zQAf4fuesJiYT}XkwXAL#fcrI2dm{PJ{xJWbjg!^@{c?z>fKzkt&+zA#P`>468bcbQJ&QTkY%f9l8$xB zb6FCqLOSAjM(R)82<#gKcnJ+FHW@QXJqIJ?N2A832@K?JR0&Gr2J@Tf1Z06rPwk;L zWY(sZN|5uyZ%nN?Z9tZl5jY>8Sf|WU?{C>a51-zm$npA$6L`)a-aj7-e73lGb{^%0 z6We)y3@~`*vTK3>qx&Tv=)(=?Ew)q=0>R%MMx5@^EF8b#QaIGV3ZV_ZU zl>3i`kgrbK2n_H8NjYR|u!+1T#jN9%Nca3oZveF_A}4E2491fgp)%?0C34*a-fy;A zuKS5ZpAy%`fw%(+X=!2q+MAyrk6jBTmWs68=d4s!ZR}h!7b_;&1E5z$nB#k5TJ8LDCnk1D#sohm1wgbY!q z`?IrY$yW5i;|F;h*BQF21CGp)WRm%E(OT7EeC5MT>E6~f%-j2=w_4Sd-ClIj>1`y( zW{8@Ot7r$mz{9dZGcb%M&(1Tc_Y>5tFA95ee>nT;`8FTf7 ze{W`?fC2HRKc2=iS!gGk@{9A(N0S!RUtSBe-j+fbVu&?Tm_yjwz-6x65UJJR!O7nLC_C3m%1*x-Jb_~WZ z@CaMagpEnw)MCqL!XH&r>lDnyz16ItY>p!u z6fXTPH?-$TDr06V_{!H3gq|^#l%A!a=uZ_3=&!qzm}cMGpk({&U7geNE;ZXkd6S$h z+dizcY++I!f>STTo!jEdIydz#D*!>w(Yl*K+`=iF%jAe?L?x}Qzu#ww#IZDrB!}-q z?CkA{FjnblY5R?mwIR7OD>@okCON+%2)r@w(Uj@B9h3da;G9uz(6R?y0|!nLesKAP zYKo)SKtl<=DSU%CmLeM7F7;v}lX6}-RSQDRCsQ{6k=SZ z8Z1$kv~LrwY~i3q36(#UQa8^GVqLktd^+>%VXGCJz#>0g2NaSiCublP@=mN#th;78 z_%5!XE{8kx6)r<1&KNu|vNuiM?x93%xSn}(P8{DF>yq<=xSLJ{-0pW3 z?z`)G!;)J&|Jd=6uGcf!V!4LDG($(dHfZbjoNs|L%E`TJU(k;eRPOxLn-|*0F{&v$ z0givdHQ^k09~6Gc`>q6{jC*ulpoAbk8X6HK(#W-!#D>`;iFzTUBy zKG89eK@9&ZQo^On0LUJ;-RvVY525`x>s5jRChwVY2Je|e1g1N^mwvWCR5p)~%8D2l zxQmBzmJ;*}@LuI_tb!8$x*wD^uP&|4`6^_#C*B9;>g8%yx4@?*YQC0x#@w5SUC59R ziYzLuF9E}cx52!1x`-#ppgy{vJj zs|3tU#KSWZh(cQ*)J&Q^)p)kFZ|F5Uu3Q5;s2k}TJ^v1A6dCB`_C1W34P zF&WIc`Xb%RLXxd*5OKub{_%2plE7{7?*4c&02=hSeZOCF*Xrv_N%}ZKlHbE5;))#G zE|m%;k%iMm0{W>JXrN)Ts@;cEv9!_zZJ8QzYH4H}8u~ z=R$I5^eC)A)C^>Mp}B>$*UD-HIk>~MNx(@YYVyG+VuVvNS^f%RfL~f?XUlyM-%GUY zYdW2tCA~xj?QGj*n57w=;4)IJpnl&D(RM_4elL{`hEue}@aV>tlP7d>ysE{c_x!EA zM;QGMazI%={eshu+PPYkaFvp*5-jJ#y}-3+X0zRxgV_qCTBC5%^|KT!!R&?cwg062 zZvCAb4gAh)g*S3}`&v~Yh7r$lN#$KQw{n*Q9*C$N+O1NPVV`boNOLBt=q5)&LCbgL^O#69D!^t zLDOseD`ppJ#%Q1LfRQXq3$!Wae++*L%&)e6*|(deRg=ctJS~?0=69_ zqB}TNlWR=3yUq0V*?Z|0=wktHjRI@ofw6;Fs-_B0$mFOy-hq*~rsr2Prm2m8ty9MzeJd@qlGH*lZw)U^yHi43q&yV6+$%Zrb zPX{?gH`~|&M5miseSddqQU{j>cB`u1)xw`H@HlD?{?bGcCo*L-bTxp@*&LK)MMsw; zJTawQ=O|Ru^wr@A$j6`2&~&2<1DU!yzm@pXrr~A#pujyBaF}Y3$7T0CkRa9=kYW6q z7$5JyooYMal45ZSbuQgwLAoD(R%><21CfkMi*lRe>jLeG+YvVbpvkK1w{N{l)5@QwGP@TKq!`eyN*c|}BUWMt z5gom;882%^as+do>M&Y%BV;kz7`h$`9sD@rEg$FB-u_x@k~-PmSN;V3Qt>PBpgc;6 zC=nCh%E`fFB_U@UbOcgvT#0YHjgNc?M8G^IXSsU-C zGoBwdM=n-aOlU_a@}l zZz{u+NjDtHBeG{DIl3Y;b1i(hEqP{j#Y{$_!oZ%Q*7@7PToqCoIH@qS`Z&=Rg+wINB?3xEJn3mjvg-jRp;2wYuJ<*9Tb* zHrU$kvt`$Hzo?1HY$)OTrKgk>r=3$wvB_#FW7gV>`hwZRz@v0)kxOrr~cEY0*u zUy~hG{|vnK8F{?NGj7K=i^FWLtQDpp|y&(p+kWvQ8G*Z2$ z{fne#dRN0tHP;d0kv9&z$qP|Wql#-qH)24o2F8XzyYaFbLO#joH*dH&IOaF*?(X5* ztgietrlHPvdzkLL0=KmWS#ofEYVOAyxo8R=Wr}^;rf})@7WC| zuTrW~mZWanjW6|$>l)_`L_!y6!ZUlmIAjQT4s(r) zn7--fw&V$Ci^>x6Dv5Hs5^_Z^8dRXHC!ttMjT5ezZIm7wA#m`Of$8r@1NMkBd~W7W zY52bVhNSSoJ3BpABUKd+k0xpkS ziOa)Nx3@zC-sOiT-@+!@zyl$C;}%_TUg5Vbc;)@u!5#jkgmq}J?`86Je8+W@)5qI@ zodp9CMVpeNL?x566Sz<3w%T#or~AKkdq#&|a|l076D4I#BCM1Tw09ML6_V+aOpxiS z?)NelEb%S+oy}|-Aqq*q5?2hSTS#2NpZg?heo<=VGM`6?-c%aV2n?^K;nchT)0bJS zG1Fp_3JccCNq{tbJ=+MX=Hwx`8?a+t+)dd#Q5{I)TJdoaGa`*ZM~DcBfGkF**P;aP zASWfIRLwj@1}9!-Pd6SuA|$@4W(q`*r8b#mq_>&;G;|}r9m(83(OYzMe(4hZ=A8X6s^3;UFGo%lW-$C;~ejeTW~Okdi{t~u6VcKR;M%kd7y+y|lV&6|OnmU_I$qPuG?yX-xw|Mj%v z^I|e%I=I_yx@Q=q7BgzQXuQ^hwA!lRXF+uNe1Dvqm}+nNGBp)Qwx}sDkNA%)9ZzBE zv-0}5o0A(5gE=cKh!+zik#|9))EIm@YC_hv;8k#eEE3N{qGW}B%CS3VTr-7_>gknv z5F^$sEKroK5d1`7*9^lEJP$OW8V_g>vo4CPxeCaY0z`rM5Xi^7>V&DG45MJ$!^&a9 zR+Gt4uCYX36qtyJY zFJLvXxgCl_KKrGGlwUqxF9-a8ONG(AHUa0KCkO=}2$%H5yZk!*AjOYM~^)A_mU}|Vn4ta?Cq!-SpPxb_h_vRY<4`iU%n&1W#Ij@*7Os;GTmcomzVXMe&MQgOqg!)xR>3?YV?02%@nVe^?$w3~ zS+v$0Fb+Ku=`COS5gCYDO9`N5{_FiI1wq1(374hpZ?Nd7Cxs9owZu1^=<+rFD)kz0 zf5;!B6Q@dhMl`}v3(Z6T_r5T|{GVM&Jinpg3RY_Qal)PbucRWLR96j)uw3&|<+MWZ zhaQa4GEAx~W*QG|z$zp7gD zijK_WFrm2)xhSOy(ZuB^8qETBA%WXho%u;_ujrdkv1yXs9Tdtn@^_|eaDSW>Xc}aL zeo1{YJC#YUQ_v+`MDkfF6C0&Oifmxz%dsK$UTI^V{C|r%BWS|`{&(L$K zv#a|-s>!Tg0qnm;1`qpkUde4{#uI)+GgBjl5}{j*?hcRN6FOI9n-&GDM!!AZfC-yT z&IbB!y9nXcMG=#4a#JeRcM=ja;4|R1=5kmvL%jP+YEt26Gv zzBGpIK|j&i${m^U4|mLo3z#AIdW*7BtUw7 zy$>2N)0IC&d30KFcRd>j+%6yWpNNmfTrrN^zG7gmfQC~OHP_=aw6@z{a=fQGYvvD- z+;_O4=tj(zfT0+>YXfHs3i(tV<3ojn0v+l0XcWY7wALc|I&wG^! z0s);&^b=lQUfk2ADyT~_3=9f_2QGB+!0%}Nlj4DHSI=ymMj?zZYUHr|XpP^M81%ey zSSGosWX&(2wyN8V9D&v$0`|Vm7H_Z~Io zGX+vBT}j)x#huj`Wa!o^gaJ-sd&PYyBoT45H?o4i@>;~7gu-!WSsOlkHw-1&*szB% z3Ie%?f$(u#^Gc0M`8##22yHC%x%mZm)TQg#U=D&qdG=e^F&UQt`T z)xJfTR%cj2DL`wY95J-t_vqlC?pp!6#GYLMTtLYt<%MNBc^v$d@SX2QL3_VYITop; zaV^BVssXUo!3BGnc`yUDYE(3}IF`VyAcJOAu|iVl&sw7;F|=dt0ENrha*fMwBu5wo zKQF`QRUg9k#!;nivL5N-cq+-kVV>X5*yd@s7*T9r<354j=79z#r))nxg29KAkeUVYuD*zQAm?C$IT9^V9TYkot^ubm*adYXlP)n=2#h z)1XaTvdQ?K>PxWfw{O0*?xeO~<|Z?F(m9TJyTQIfqGEx-Mb1PT8;UJg`I?+u1}D?T z;%b=gAxF3mD;{M&?4j!KG_;UlI848Tj7Y4KMy=_2`U(+P7c3n%jt2%3P53eWOS0M% zXavku7P_^u48Vob{y;U((kC|=q-wjCu#1hZaHx6~ArMXM0Cf$Kf`BcI?VP;po833AEn1Dhl7 zM-};MWJPl}Gcg&-asN_tt!pf9T@1yLx4{x@^UIvGL|C$|QJCv%t~!jsTu1`CJ46jm z_RsN6M{wBMs8rYA%%Z&e378BHXuF7Ml9|uz?U3`6Qt9oB*O~y~C@$k%ma&DAu^D)GP-o)H91U8P$DYm54Ne>lp_*&GQW&c-7#BW{ zMDqG^JvcRnlvM&TfEWZXQ&x%+dk76>rE%c(E5)>HeM*)w`z;e%Tr1yOI|yryHiPc( z&J1JJtHbR`o0#-nO~}e~SBETb9xdGDF!N)M+$W<33OD}apy1N zlfYpw?4YqowZc5Xf`TAwD$i-wDe}_JNiBgP-EI6h>m(PGIT|Ak%1Dp%LB{%r#brW0>X$Q8AAp4$jBZ7lH_qTvp`M51AQg z<2lAj`VErs*&;#VSFI$)z2|Ok{nMo>?^dYW>66kVttcGcRq^*|#Rd8GF&scBMvR?n zS4Btsbx{UM)2d2#uKEb0nVRecoLYoKS0e`1nQsweEMHpftgLKbp*LxkpExDOy^)rv zeu`4duPp38bAbgf8P0yKcpd89DM7Qa31st7d~LWzT(M91YK1W{Ut=I% zlMuc8P_hNf@Mu0?jO3!GkO?}6z#c_%0-D$q#XObanHO_;N~PO`?mc*r5$?kK{hnc~ zT@Hv-;H9G4J+kgSi!&P*jsBfnJ`_?ayTKl=1d?t2dp5Wrb6`=~Z=dB`zTY-Nf7(}V z_f+WqnQZf~88t&ySPc;S^R!KLv|6$+^R9ea?~T+KED)@kt13sj5IWvbwjS;L!5cuQ zJJIttGbba5O{12H&UdxE`+g-KoCMuy5Bjq{++zRM4*o%B6#&&mJjv|6W|61B9RcDF zJz-jnS_pi6d}9p- zlI7}$C0*}S>$m*;{2p8GMyrkQ=JFSLL1T0T3^*tOkv(zYvdhFhXhB9o>KGwOs*n_B z#oTIdsSEbbLP)~wV>J)f^QpXs%`GmmoAkx(kX`EhLwOd``|3H>Gcmb=_OU=Oam5T# z{xPIH+&v@KkG(EQD3HODxaSI?m=U>77`eg1Jte7uzUYD6UD@Ly}mUC5n=obdMsE7#Gsg*yqF4y{Pnd9T5#6s1v*@q?#XD_P_<{&}^ z%Io9Vq%CJ%6at>_SEQh?AmLpJmqb2ES&3GMO@vE&bfAW1-ZLw>C^XpbNj77gyutOe z7|BV}I2NiRJ6?r4BkNze2ICFkA)Qh>XLolaWmg^V()k`(pB6CqZAtA(X8u6z0r`vq zL!zN*9WO{IE`;7p*=A?5`jdNZyei_0Rh51+%$?u7o;!z^2m;caP_K;~;a}zJ&i{-H zY+@u&X*25cy7pEn10Y@sm-w!Szei}A%n&-T;@4?bnv{$Ytq|&e$$^usKf0v&F1X2= zKDO2_>}K%zF?n@2w&`e?s3iLy_DIXq_ycqPc@b;WE+o_{i4P*oWNGTE~p8b74 zS+{qH;ysz4MI*2xYd5;yd}$IF-MWI$Y@YH~3#ADhjClZatWkMV?3<{)Rbk=1-|31x zvArYoiV&UG}s+6lDR7p6CC+iIKMD#vujYQOVi$ zEBsZ41icU9a0LptO2bjBlmrO}_jBoEO-lXU94GWtKnWP6qcWNP! zFa}jH!X|KF#o7UX-yCwEO8b~xFY_hVPNy5&Qa~`XlG;wIM%C$U0JHp)KTdT&?TVQ8 z(&-1nF)*?;a4j?oL0TQCt`9f!#&Bika3}g*@EWGz-8~z60CQ6a^D&Mv#VkiKpFSnt3T@*1z$H+nqZkYj@= zh5pSB_lf@%#+-jFD5yXlxFS=H%o3e)FVldvLOPFzg4F1pFUJYXmLpv!UP%p+T#ufj zly8#y%I}H3P+Ko6)iB+{Ll~1wbPSTj3|w}t`TemS52eU*>BYNq!oj0fEe`?N6ZFpq zM_RS-*N08$shSeILl?8@w4;WdLH7C%4U@?Pp)^up2H}Yd|Hw8u{G-v!nEygJqctdo z4XBR|!tSiR`SZ-)7jNKBCVd>E(E(IyssCAH8$MJ#H1$M<3n5c7R zLp{yrx%H{l!z^86yjDS!C(lx&zs{@1(Hc7koKXws7rNhs+p{qaFL!_%R>2)qT-=DJ zO*iSxlXbr@P+J#^73mtap07^h=^RtMOCk?Q61R+g7PYw{w9tUTS(eF6yxwJD$I+|! zjf~fA_TJw;?=``3u`Ctsz6rrGdmQCB{Y9%Qc4Xo9vg3V{H>I5kSz48&sL^aoyBw8J z+6EtxHd6PLtz+hLncPNAd(FR6nN!#;)4D~ymr1Usg%LuxcS{qJ%^B3I1v5{y{&R=_ z^Mtq)+8mpVKXpIEAs+d9;hpT|S@D~4yitf?^5MSxfosed8@Nz+09vQqdf!IMuKsy= z`aD{eXn}LJ3UvlZVDq`L>g(%|dr`<`MbsK;i}!M1g;QQXtXcPE`_-59*Tex`ifA%x zBL$gm#H8-aC|3&$;*NuWglH)EV$3FKO>JsibHv86qf`$Hzp*QF+3o%sl@le7xoKi0 zuMyOC8O{VJs(RoS{JtY{sF^A4Oqt;Tq0a@LgX8^~lnAEN>owg^))!eDscSm9>%y?m zoY@SF!u}dBeFK9DB2X~UEPl<7aDkt7>>rw})dU?Lf+`;#lf}aS=7xVuZ3=M%=F6um z(?=xTwS$r^jMuiYOPkmfc4r=y07xp%)?;io!#wS(%SX=dh+vYACVKEE3N#TPStHZ#e z{!hmIf1<(){_NXbSm83YE*gmz(yfH_<5ncTyV~pNU;@9kij42-xH7zD)*2X!cEshH zn6TkZ6||vCqxLc*bPEYq6h!LPHsPzKlEw;9jP>}Vv^!<14LkTI#+$t(8$zeZ2d%cj@AQZJAjGnQ`=15bMk(IgN$X{Ys zg)QU9J+1bcQemoZi~K1E1$KY&o(dHtu~P$TLVu|CTVq$kt^VyIik?5vyKYvYH|17n zDMs`}K{)iV`J|>;Gdm4t`7J=@J|a;3rZP`lETFT zV;(u4YFtH{$U?V=(-LDoJuV!~d}2dy3RGw@9rGx~o73kNtGrWf*=89OGGHj(l_UGA zXqU#|O5(ch7Pv?W>K{I3p`v|7)^@Qu<2JYXOrq1w15s33g-YbmF7EDY6H~Y*qhUtm z6OIl44bHzr;~L@P969-P-~!jF(e^_j!5N`05NX6f#jc~RyL$6L-F5sD&LZ{HbJbb! zI!?@k-X@BhpGGZde&O=9CgJmWtXSE_((j=f9a!fOM7miraU$AR6X!6>4gLwzswq@z zan<#J^dN;u{j@UyG!O|teQtpR$7RPT5%z_eDDF(_689XqS_Qe}nR?=w1`*tUm0TUG zVp%4xd7#qkBcYA>$_4okt||9AoXG&C&WUT8sqNB|3s6t}?qGe!S^|A#hyl)abhvu)T% zFeqSNkhI7}`W4n40cMJDG!$J&Qj8u@N&XdX4Ko8zwTk#JSBX|uh#d9acPin1Qn_XN z2(4A7;oFc`g}1y0qGaj78HoeGzn#kTFB2BLEBh4M-}|1F05JN<+P4MsUxXSlC%-j{ z=*t&FXPlj$(sVMsW$s+c>Tyt#k&&&qPS02A610<~%4&%Fe6v&2hF;|rn6fz zPqF3xC+YsL02oI3WGbzIV72YTnMR#Tm)HBR-zXa5+a*O=$iCWKjEofQlgInM8bi^M z;G@O;VtzEfk%;jp!6(;k+oFxBsJ}OrX~{yzzXugXDa}~)>y1S*A5f;*%dS);S7><~ zt99lVoSp10Z&f567DwlM4B3l&;;!*5@-#Z{G03V7`@?gIlA+%AZIPwts>|cKxVnA+ z8b`4&+T&9Y^N(-0y~wbXQ{@w&F|18~JYU1GZoh`a`u)rURg>_#>3)e;|MBk_`CoTr z3c)PiJsER1fT5d0_XO5yGL?1U>V+}_!xR!KN602&1P^hs>+j?9r3tQOkO#tcJ?;k% zJM?nr`-3JaDQV0o|MQQ0&BrNxvbT=iWsYCB_bQuGYKrE&a$9E55j-J^k`zcOc3I;h zH$SH_04hej6atL>74lXYl+83AIBY#K*Zs3Ie)M|Ao z2_=hu1|cxUDYjTgh*GsGcy?UN^=1Xu-z|29>h!}pG28%%+VJj!iFDC;a+x&LP*=*f zC)1B!-!Ja{_?$M~R*a+Rw*nihrYlWrj*mLbeqwdQbiL!eARL zwRJK$n^r}+G{mro`Z`KU)(55Px-e$M+er8DSlrV#l$Vj9exHE?Bk}dy8wiP}hVVUY z`%~(6w!3C6^~YKl?c6Vz$Tu23xOIAUAnT;Cf#*J3C>=g)V8!L-=hs+maP!^_ldRxG z94ta3g8TaVfjR2hLs7T#2n^Mo$%)1SQYE{7R{{pS=MBs;0 znLQ&Y;uEiUb0mPVB)UJK;5;8kCV1=<9z?oh`5RX%pEM6Y>Rl}V1Qvg8X{fXAV9nE* zz>kQgw^M<%&7VKu&q~3QZ);I4B}TCyMmf77zc#!DYz5iEF&HVg{)7xjKxMG4bg3oR zwr}A9x4;CFnYRZ-0v`4Z7}&fd41|$!Vw)qnD9iwwdK~06OTrcw<*vtW_a{cP&p)*U z>d?XujM)Wv=F0J2I@tVpr7{=xwKbM1wV6z(39EghP^P93+8>DG}tOY9GVV5hwP zn!`*B5RSu(`1TP{C7^3>M5&*Tr0e}4uLRckobmib`F(`jo00x3Hld5E$T}MU|2UyL z(c-MbzSin!K4(53X8mDa#POS^wW3&d^W3~+&D`L;Tc`0*u;zTbp1}80@q`zO^2*13dgZW-XHDVk54ktVa$s|yD^?^JB5?;HOGVOy}IC146oJENy$T+YG z;>dFrjEZ9$Rd>i?_C5+fw|cb%TE7Ses{!H_?& z_x!0jmW(8pY}MD@$=&GU?1-J;L0XDIj{n2fJ4V+Tc3Z;@+N80K#4cX72LRsvyiLt^LlF4~RjUhdip@PbWYi^~3h~NZz>POB=+|Lfa zu0vJ>V);Dh%@6=x7X6rO`!tH%>OWZJ|C;{upFbH%-<}S#3hD92*-1(Jc^NoGKAoSj zA!r!WI!HSsPMpk_nc>7D5b!3KC|Toa)|(>(^|(*~B!GSg&i8Qqz*3cxSc{k5$scT_ z8b`f+=eeAGK0bS@O_R7}(upD!S`A^_3qI*D$N`J`wE!L1tl87^mAg-t)sn zbfIioAq%m?%SGo40wDiXJ>^~i!hkR&aO+Kl5$>#F=8OY8Fq%y3U4E~7>&d{Q#Q&9b z0^x!`@h@QTb=%!Rc(za1d$O(B$ieIt_B;tRD#bsF;?S+uJ*)>Ku#1(85ca3C>@XQq zld6F#9)aKM6%>K{CKwO)e{Nq&ZfK2zh+n+?Z9C{r{WraVu;;~veI?JN?itPZAUt9I7?1}3o)*1GHtt{Z%3V;L!c)|LI zhSEsxZ*EwQz=d^|0?e%3>Pl9;m#NURwad|Qp53Dh{C*#mQ&9uZ)5kRv`MxM)Va{58$&_P1AZ%%^q3<L{H6DR^1+Gk#&;FVUthF8hOMVR zXe62Dp>Me;0Hs<0p=1-1e`Gj$-h$5^vPL-z-_ub)U3|Idzu)|yMf4Q<^HRdF3VUzH zZ)Y&_b&jI1)99W~z&#xwea@gX_bBkaX<(WWDTS&1PU+ zl!mUfLS%!T=4z>WK;zj}zBw*q16f`ik!3lFFHxj84Y zqvqC5t0p%kxHT7tJT2kZLdIxUw_j{~D`7DH!zjNeq|Dkjc z2MQd`6gb=+t1_7mJ&|_(g!VaxNcVe$h^N8aR0(Y2Z~d92M%V4MCYCxD#W8src#UcI z&2vlHwr*|$S^izS_#Yi;QxBR{hbt07@x}gfShH9?;~iaix>!>9cXM*UsuiRbZlr)O zu|2Z3TIImeB5v{OD^ooTRl>ojNET11)kr3#egVpyS23@8E~8trJG$BAjHw)7Pq7No zMb(9@V462d8D^zWY!b&W9NsWj0~0%`bldc%e>+bN5$goO)}u^nzvD^GI;bR1%wVp1 z-P~-b6J;$`MX<)-$zE%~tD$VF3s_L-f48=_hzU@j*7u=Atqko7{vG5-Ls0{FCkqv- zsek_+G+4kk+%({sqDG^svU2p5BMkexj`tHmsG$Me0Y{?6gw8hFb%OykFHx9WpmX={(@Pc%hf z_JojzHO~pDSgg*BgOTfD-AeCI*;=%nKDG4R>&j4LxrR46AH5qUtNK|x{*CaL-EmnY zt=aGkbvVaryus??hy#B@QXiS6Iyvdg`+$BoFv77e(nn4QlPz=@nF(nI{rQr~xA#IK zg=AZEglS9-O?yNp=igK;u3->n5QNTS@viC(&$hPw0Vje@AVlcR77S2mZ*ImDy8Hbb z^50pQlauoZq-bjkr-eoV_W$TZ@=^H`%50{5)q$C*uOox`~TE0sRD)4dMk80)`$RRZ#! z4(n%(bn>}BnHRs<8}k$DM_*ZMEAjGS*`MZumZGf#)iqp#CKda2=(43Nm-44eP0&(v zh~A=uoS*#lm#4G~Q4RW1bq>ERPT$wZ3F!&cubIVo%Qz$$=L1(0bq1+%_p*cd(NjGM zyda_*Il1)e&nb7ew;}~n2{UZlO`AYQl%w(YZ=o!1+jAU+e6Dm7eY1ez;D0#FVFcix zoqQYhiayDs=^Q~+Onm>+UGcR%j{w@kvHaeVQX z;23Hzi9VCkq+G2vCSIR=wbeII{&Xlu+4h(X)#+p~1vi$tw^D29Yx0{6mP;G
    dX zv6h^li5_FfKme4=W|jKuX^@+ZZJw6_S|su>#8uZ>olR*a-tXwu-KgzhYdtA^ftXF8vD3^kyQ-kH`cyqysZB2?3S}{U1)! zS{)vQt6M&CF24I*3&-f5V+{Ei#lY+t!}@!^bpalWq`w!+BAngi>b=>$H&-|t3`&Wd z$i(PYXsFoKlM}~5qv)}Sz7h4}lJ?CkDTzTOsNb}Z! z)2*-%jg`GS7K4gXi2iW7W}36z8aW#Yu893u9u-arox8LB7gk+eQzf`d&TsVv&s+G9 zNOPP-v-Qbzkw&C<_0SQh{Ntme3e!xQ=Ah<^M-yK&Y3sG}lILnDClv=v1=zlPkT|)5NZMqDw5y7<4 zt-c!5_=waNCdVZ0aT)~bW93MJOcHzZBU0L3Jz;Sp_a<(Sb-!l>A0#XlnDvym>{$-y zyjYF$ltuAF<@vJTxJRl;DkH?M>u!dZu^>Z%vY^hO?+Y-!i(!pXy?<1y^uF_;`M#WS zwB7u2*bYVb;Fxdhw>h8UOrNT1Uy3^fs=tg|W%?VYhs?0LNI(?WpUhmaZgZ;A z0ciM70rJLw(x72DFhWAY>KG%y-Rf6E_vF-H|1Vnd*TJs+ic?chPWaAMIR0tEn~;ee z9TH*Y3L_N}^Bf%XYg#0KARtlv`eH}$e&Av;9&bRqQWIl?k|;Im<*6W_HUFlP>Gk{j zyf~a|p^U0@;5P*)3a=Ju5cSN)ER^VI8*Dd}g-FW=yZYkM$stNG9R--3wxARHdflEhVi(e`4RBK;IK<2!U;X$ zuqk#(Y33BorL|q=n|LH4Ta}MQcR0X<9MogdzsWAF3z`@x%Ws{?Kbm5sNnHa91hYjA zs3a7(FX9OJy(Q8P!1)$#PozsDt?T1mXILmX_(Y<}>3qr>i?v9~u*0n$FpEjrbamrGyXa5v%BkfH|i;3NT@vTTv8T&Km+Hpe(I zL8AHhTM#nUn2lr1RWPA1V}Vn*4nkwazc>PzdwTE+?Iw13$x}(EqZzVG+vv|!WOy4; zI}V8U5J2K~@;BvjzEl-9fkvJEzi>}9NJztKS8x4@H3}D9-fEvth7>yJVhem&2Jj@e zs)Emg>7JYZZPsb`i;Tp&lQ5pR=6Iq0L2%5-pwr@znUQyjdG)Crb&YX{Af5U1S zWpvvJ5QA|B323q)@bMd<=}5ke%}m}@d+iHrtmlyz;z52c;R{$dz`zcDx$=Wgje}Cf z^>%V+lRA4v3c$N#+YcXL#_7YW2n76Q9|&2_Sn>&+BAa?~l{?egGelPLxE$ z9ro#HnZmh?)^+RNhs*MFVaFU)*&$B8JaTfiHI7FW*Hh>2B+Xn3f(pgpXcFh!>1xWSFg?} zrzM+M=~pHurs~eCbHD)toAOfqcz56AOm0`M=7nH`0lXWVn5 z7B{pPp@)a&8sEi$t^BYc@KB6A%mhC2m9y3kf8DDD)f9&T(Q9fwDl>YS6x$8Nx@D>PdbjY=Zyr2t9eQT4%&i(C7(` zNd31y<-Ae}&JNfy_i|%T?}im>ktnUeCa6Z2ORTY@JF39KI_Nh1rT$59wTb=cY=p<< zu4o?$J@E#(PPatU)Nvz78DW&RJE+f}x0czkiG2V``sc|%&c^dXSjv>sl_c+bz07IK z5wkNhwX4ocwI;4tG2E`FA*c0Ij^e3I($Hu$YP8bcsgfj{QPB04^DtmwU_T((C^EBP zV1S|0NKV*p!!LvphJ%6*p{7Wo2)NeXJj zH?4U3H01_85kxvs`VY<25l}IX$I@tmIx*{6R3a$y&5S1M4Vgo%&NQi)?*y9Um4{{_E+{96ZJG*bAgGPC+qRKK`60C~17HFD z7+jVI@zs~-tT48-UXcQ*Cv1S&g~)Wk3gaGAfh)LLI;k)U7fY~sdss}zxe9tqY*G3P zvd?e`Kz9Ao{si&tY7r^`dy8t(va(K?WY{~+M4qLZa-&_e7$K449Jx_sh1a{FQJ6Gz zI<$Q?!DRFqNm5S@we(`Z3g4>4QabbPM7d~oEO|yUbQsJP{XG8!9FLe}iJ|K)GT3TP zvDf)r3_4kV2!%{4Te(Q?pAF$}PyY8ulF4+g0I3A(mY7!TVw5_lD!01t#kcSBD|l52 z>R$+bZ>B=0@+>OOTgI+JS~7p*N>WKo^cU7!-hizxZ6kv+jnW_0E2FdSGcgXHNgj|D zTuB(FwkeNxLaz(F6%!HUMvHfo4?5Lap&|t)kv4y|B;J7m zpSp=hRpdFS0=d~2)DzOb%r4$$+OVc&Q;{zAjR8r5>yGB{*Qt2M`r;{iffO=k;_oB8 z##D$BB@+s5DQN=0#7E0hgnR~pE+=UPr3R<3?|OsQ5~>v z1$uc(*x_q2HyKx{>X~{j9uLv;2c1X8D-KaUR0d5Nbqb$?X;C-8CtrcIZd=>jFyKts zTkchu<=#BR&|K2)8V5p1TXiveoxmZx_p`a9aEC>YOYnQbwqMeq z#CNeg7JE2ur7ul-Pm-rnhZDV*u^sgVPqdjE16(@$EqVM+Z&+<7d5hWAVr zL5?O|3&Lw**dN&R6e~@hSHAsqEiFKIhXlv@z#RH85AKTo2>qb@_+)foo6jUG_O>|! zR435Bu(zSJNc#s%B4&pW|FPdIH*n8RE-yGGxL_6_g}Zp??J*Rot@#pE|GS2eSRsdQ zeYGA<9vU&F;87lx-@C;P;~3~$*`SL>7a_CC{);!!G>8h|4T5KX%JaKgio`Yy z7MXdDO^w_+)l3&>a+?Tnlf+i|P&wo9g~Le-mzeW)uDvq{HOgLy#3n6>AMc0$8AL>p z%rs}>us*gNtw5@N!FEh1Md0*&uD?xB;rHVULL|%_LR>Ank~@eBDSv^}^Uj69SyzMr zbJfzilJ4#qmcJVJ3FvM@rH<5M0gJ4=kIzdD;aGkB*`p^>#4u<$-x&9JV_aX%uq8=A z*VA+PK=YN*j=>#`1_7<8K&diQ-5ni$p=4eq%vIf2+(%YL2kD%TK-tFQJ8E1kvfCSX zrGifUS}=WRo}mSyv$dWoZ~J8cDa9N4Dx@5$ZOEsgD+5Hud$hg&8z()S*lntAj%q|< zp1(s1&b@O3Q#%`otC4>kCl`?PIk7?|r&i4d84WT{qO}6HCPtZ3_VU}K+-*&{j}OVL zU(ev8vI-s%#J0JX)7l%Qo@Z8RSCMsP8*BZ$^q)Drq!NA3Il0!4N2WTP#;5S^9IRtf ze4}1dGI;KjQeMk%it_WfaXcRNve$eN(woEBc-{p+rzWki7fTGjUtiu*Mga^3Po)_G zsTAF!TqSw2p=r-;^R*nBpvaJZ>FBlK7ULp#`gXbwMG6{AoyQlq06I7mhc6WJ&CIC@*^e^l$a1SO&Zpm{KtZ#2DM<@M z?5&%n4Uuso)jZ-DU4PlZQb%1NPfZbhv(Fc*2+3uVzj5=~Z+`k$=dRhMTVPYf?@t%m z9re*@aYRy+OAO5LL==u|6Gm2QLHYIe1ojPs;@)OxCMNC-PGWq`>#wQIv4SxB_#g#; zcK&q0C?b#C_R#mDBmd94li0tH6R2}#sGu&;eH9G8Ty^(maIOR1EHXTlxyz1|R2jUQ z)-Ux*Q+9w&KhHt>sa4BQsd$;5t;?Kaoue$dAM7iz9q~xPjo!z_`rH6LfKoGkcKT3X zZio;cH$2X4YU^|$T%&rQ@pyqclHhh|Y@zLUyp18T++DC(w++%IsgdMHrw^0G3~3Hf zAVTYWIp^WM4$%=xdkMSFa!629^8UpASno>#ce0EEFC?T`lZ$Ei!T0-)X5WB*V(0Ii z?HqG?`dp8NfFw;8Nd=tb0sYWcne33mZhZvmz>VWTtw}%cnd}!|IcxhLzO$Eq)>_D` zp75bgI5D}8ce?HPSun=0C}MK5B3}=N6+GT&OuV1Eb{5+o@LJcsaD_^TIePc+oUyw2 z=$4@&fB!a`Ok8Uwp!`15CaWFz$fa9%NT*p}Gn>co8@hO2c)rZ^%AW^(ji4fq?%S(S ztI^H1M*We{It^4&o|pDD5=y9>i?)sD z*YhDev(+gb2(Pv5*vl4Jk)DNFJXLp)ftpZi~E#x^HeGuvt#`589bY zKhK?onJ7Qxtqix^XP;_^rZ^kN7goMPkPyI6gbY~VPffK)Ts-6@U3u&();L$_*jvn1 z>f!s6aqXwPosu4qTF9V)55$WQI$Fm1EP|YKBRF)%SfRR={uSGtVld%hsIIG!{Y+yGjm?-E7NbX+|%eu6pJmL;!48tOncp|hfslVJIh_^WB zjTC3F+Fu5Y!h4c%U%Uu&ib<0_(zz{mS@s32zn29gn&fY#qi#$W&IANbEN0tY&rh^( z5<`UjpkPvws~>KW0Eu-_)=nSi2<#~#@Z(8(07Twqk)z%gi6a8NkYMWg#Hbx)L|NKt z@5PbiF2;zBsUgfr0Q(m|%GraY+w=At5agXbay5|I-b1r&NrP1JvxHoQ0MlC@)XO9E zzVufx2Z)}F)dmpffXDW#`g9R~p$|2qz3nMR56~dob49d4-QZ?Rr!I2r z1QZW@$YqaMewH79z@E0;j1mRnT-gmnJOu+OTT%hGn9C&B)oT8zvHv401;0Tv@@!2i zO-tqh-w4GxvxpOYEM?QY6BQZrMwm*d=%WdOb2)wW{jY>Nk+LEOYHS0{9GPi0er{x6o|Q=y zwi+2xA&<1JGcAQk6pWE5y}TfBXt3Ru`TwwyC`H9^1!N_zL`cw@&X!8t+AQJwb0wxA z!gpBZJmlM$e`FEXST12EGuf3W7%uJdX|T6InSQ(yVE?e~?K{SRqiRq0?#Vz^3}Qe` z<-28V4c=>vVW7YNcSn0(QiWEP@IqE3%LAS z4zfK7kklLiUe0@Gucyig6WqLklCUH(iD;2GMF>V4=Vybf?O?d5f zr*7ge;RISniHaARQng7!+=3`#+)T%G1{O1mK+pEW~?TIHB08fejTu#|3 zgj+eeIRqXn0?G>xu^%g*VH9z=&tyhJ=>%9UBBN36PCWa5o$&f{r$jIaKGUD`FfT?U zqi)0s^~vf_cwl+hX(5BR_kV8VBI-=|ZtQY9t(l7&7fN0&V_5`HXoZo_#70+hh?lK+ z+plCG3yJK`G;rFX>SY)AC!3epy&OYN?w96)vrU7Smn#dSM4P=nUO~gvb5V6(kJ51Z zb!Pjb&*~HTa)xwdhBS1glOg7B8;g|MV8uL&W!POtu1Yy!Z=7j6($FnqxwzJ=egWl1 z#+a3Ku%sk16J$(}VUb(Wk2dSS54BS_)z{5}v?|?A%AC(CxV4YUM^SAg*bg)#Fft#~rsOSRf> z{aQInK@EI-7#;jo#gWw=xvR3OS4RHo?Q`Zgbi8nOxzadEq1O6sO4haYrg=WOf;9PUMnf*U*Lk+`gF!j<#%4BHXG06x{98{u-`S87LL(-P&j(-xPQib-(+ z+S7g)^frvm;1w>P3}Xc!(ct)i2o-osEH+KU2xq1Cyc9(Qn?lhW%+8DXgAOakPQ+b* zDtepMu32x%ZHZ3q+NM11;x#dNx-}DEI)7*{qw7SO7P}U{#y|=l{a~EYyQxnMCE%GP zg3Xk#y7)Mj0te-`S@zH`XAvbs^zeY}Kj1%+d^!2RbuCp2Dh)^TvL`mBUAb5c48OsP zSP0D5R`h1K3`RqZOD2J{4Mflq8-#9VB~>Lwgl8u&aRx{NaD;q185qOqb~5uTw)fmm z78{zaOtL*Ts-@jSV~z3AcG;ra=?D%a*1JMMf@yX+M>_3$YVC6x^-Ky%tOt|JEVxbT zBQg37$}OrCg##;^Bo^=uCp!MMtH6uEiodc)@zcn>sIIr)r zudi=Es-OciAVvP`a*j%1wO!>@PN7k++$>1aW1fP=>FHwOTm0GI)z(p;A8GVAfle1^ z?~oDoDx|*!B(@jmV!M5nA_)>Te+4t%tP<19TiA}RK3bS9RF8e0OkOn4l_0jiEUP(W zRTi5$y&Zr1ld}{~ism^jiU)R48fi|!ESFZ}G-tWewE)(|Zyu0y| zG0;7C{wAEMp^jFAv-#`(MQg)bnl2a7HUhoSq79gPL{t*y$NulW8cOT~`E;PMXC>es zw$|sO7!uiZpy&|%!^gDVET?#S9@N{tokwx7nGd@%o>IXA@~Mv)+okNy>lMbbz&b#f zIPu5ae@*u9{CU)3guJV}!_>Z_k5T^IwV&KTZe*miU80-psck_U7S%KU}_d*L0PtA#|0Wl`6PV+P`LowvW&+EiKo{^W| z8}4ef+H$L9D~Zxr(iDg98EPmB+l#o&X3JN99`2dd0^7R|f{87sHvFsAruhtHRO zzU<{cA2eDn1*kS2s~`1p5&qg3ws|Y%t7w(b8!^gL)i3oUO5%%u&1HW7#P1hR%n+St zR<%QG!!|j{+35>hOsmNMI0@N;5D1l2vu0U7Uo!Jx9k+q=H0?q1Vj)$vZ1gGefP{^r zMCXo{&{6HdF-fc^TVc8D~Fi(`}gX*Xz9(2wQ>e2nCl1KlT2jruaC!{wWwcu zSL-sg`2F) zL0ywW5b|>I*UR#K`;r+MkT>*!!Da-#ml&}&Volr{4vcPaC`VA_vS_^9Cxs2f-;*jx zF;2exEzpRTHv*-~a(Wz<9-YNHo>8hr6u}oENS&ANRS1^8A^+{wpck9!g{mCErXstd zjnS^kH8nFPhF!J%;-=2EPN#}#blbolDoumVfa%KnX3gh z_4^fbqe#YYI%{R-+$sFtV=iG(VH%5O=0Ba8M5KT87mgvLs`36#RLqGhVlJ>G`PU%y z3*Jcbhu;BNmmUzM+F7+>83hN;UDYNTyoh$X3rabXZbIMs(1Y8Z=1-UEKmcL&3gZtt zA<)72yz|BOcaW(c&A}@HC+_FRFY!hsZywh$+bBv3fvzPx1vZ9B__OBSgT$RYNe*GV z2;AN*=fw079(5tyqg-Ei0w`G`kiuM58J0r|uLGVmFbAeu?PI1xHPKAs?Dov$i4aID z+S4^?d>8*LD_RM6E^~#flrjs0fr2d+;|KLiyt#BDLvy}$B`K13hZE@W4%iBM7JYk~ zn9e#6`03(nDo-E>xVZEJeTD;p2FPnZXcM{Qymqx5GfUDAoGw&gJ@o;FRnbkEY4aNp z{v=b|ycc6j?Zx90uNfgM5#`4x2kPD6EyR54@WdOfBiB}rnc4L8LzLhZ4O=Gh%ZR)3^r$+#K z=ZJf^2Xtkn!H=hKkN)XLqzRS>HGYZz74Yk@F&$3O*aDdgWO$Sqqivx&#ZDmTGzrzD z+LEAlhrosFlAPrv7~Tn&qNb~Xf!}L1Jp5VKvn*W%s}+|EmUR@ehjnDv_;|`um0vA& z?65j&Nh5lW|g?SBR?1Aakj9C0}bZ@YL#qEDm?;ez=T!Tz3%-m%`*6H- zb?g}8%qOHQA8uLpk|RXBx*9(7t~Ycgt?lzq^|0cd&4W9jkOj6GV&P=)g%kn=SSjZYqBd2kZSntUVZ%8oQ00`y(spFaqQ=KB5Z$Ok4HCFnnm4b}-wXwT;c7xINmR;G{zpnQRY#QK%_FuE3&=Sa*j~{RS?r_9{m{hG*EcK6J z3D{X{@0)0t4JO=XGBI_%u{q2e+;exzOzDhz<+bH9Rnr5By1jPt@zv*TxafpRW@psI zP6S{5x>lgC%6+G$hQ7S;0Og7r`ttca*JIr`ZTN=!LqKisMDx3PVYw6R2-ER&x}ODw z=&Jv6&1Nc6GGb1=Q2}M7g*`t?&z$?T-*WP8eew2D?%yyu*$`~{sI~g+6!H^DTa!@5 z5k4Q9{)cwXe@=KyL}!N!M|%_?3&D`DovHfxlQZB<+M)&9#7xQ_Kl+#6!! zf|SHdqN#U$(a&l#lOYLxs$(QCOVXeBYepNHr)?AoyO^;CD2@wkY(Mh;!oUE)nGJ*3`7|^aH9qnQf`r6UlH z4Oi4B3zX=DFgg+`fq?h(c-ojt5Bg20skAPsu>a~r>u-;8PWP#q@cpk%H;Wo=rclP~kt~|+zpXQyY0Z_cZlarL7CZ_5{=h~g<)viv6$Fy_ z6nW4u#z|5e+@_hOKeT%Xm&zlK3s*M}zmyGgSMi6#^7=*pWiW_f>yTR~Rb7qRw!9{z zXh2%Edj4=>n|gC0?GaW>M3oZpIuJhl1=`SZ+oZLe>4SVPRySd6qab}HIT(@sy?9ls zt<;R*q-T~WE8GGkM3Z^C5VtIZFqTkj!-YPbEaJESSS%3%G))4fOKUYMI>q88;jzrp zO7E#@T^CfT?qI?-Eb+u}C$`lr-#yl)S#;>q(gg?8=i8jNck=KnYG z90rV#m)E%^BB4H)Kk>~|RTXV?l_Hxsb;j0F`AZ{h{0@y)fK>PK$oG5>r2~({#0rxz zQCKQBY@0?n$@1H`hYiGaQ?2jTPD_AtWE${BaxfY?I8U|a*l5I+_Xw!4?dNKmxi;0i zHqO|;(!Q`kKU0eGwBKBnic2-cU4DM-N$fma&xe#!%As|utTOtDS_oL7CP>3OA8X+q zF#-*gCZTGLBz7!T+fxkzdh|09SG8N0Q~jeH01+E0jQ*ZktG3;6FficleWrji z~cC{trA%2EG270%_5o+v5eh z_`wqb#7PcUEfVo5^;bx}d#^Kbnh{A^D!s3-Vj7LgE>du+i)5;mj+ON@uH(+cym=vl zYov8@rQ^E==hCrDRh$Evbd#Rix2m2cI+i7q%}&B$qb$^zUM1Hf&l>8C2aRiX8!V=PJ5?PXH7k|EfK122x5Uem{U@OsPs;cJf?-;uyXE z6y<03+9dM!eQSLeZy=5->*}@4)Id9eRM9Lg6<=Ny_fD~(6*G1Ivbv2)vcSYS&9}wB zuFyc}Yu$a3XHqF80@(szkydyoxs9CI{c7b@*w*>rdhuCW&>0V^1MNtu{FfbuEq?^r z4kVE=NAXIL$YBbw7Sp>HlV>=0*st z>Fvg}TT+=$H(oKw*#?*x5sZHO=By|t6}?7>)Lj3S;US$;<>=rwMv#m7l8PgT+ISV63hYrsz$qw%CJ@H16(Kd`Hshy$>++l`G9q zElkHW;kR^6cy07m(Iyhfl!)EKc`nwuGnmkBjU$+N(7W9ixH9JUwKVkF`e8MM zORX`M@E+ud;j(7-szy8aNX8CVaF&`fY9;jFdMR4yc+svU{%`40{}2pNxSQrNAZY>c z^zC~i%Q)!E%s(!fno&vTRr_M27OKob--d!nF_sIYF)#bR+QF92grZVR12lb^EWR}x zyL)SnoWV$TDh68>t^@x|_^ue1Gxw1DL1D1jC`4R5M(y}3;po79<|jttNR`#k6a~@K z{vLP)iOnA>VGDKl3y{{2Y{=>!EeV@(JvD&{?WDg)%WLzgY(OUe;|1Uh@+U3HT=_~s zf@uSX?>=t1N^Ryl)K}w^8Ii@P=ozwSnxAxyVd#^LVPxPF30QY$O3Yw|=e@5A)$i%p zkIJ9RUyl9-N-dj>(0JcHRop|UKQAct{9I4U^wn9pvr`_W)5UpG{yw&<~OfO4)EGb#}%(8DYMv7$v!zq?n7_ zxRV7OCymXALo13At_EX8$CNn~r+;Q)pqP9@dKIgDRFNBKa(+DqoI~5+6MdWIzl*cL z0Egzk8<41(9|7>+>pbAZzu*-_RBjdIc|oX1Xx~># z%occhy3T0R7n_9#5HfQq9gFM@g+&Xf*+motz+%=RB)@(A`ZP6u-fq_wEA`F)wD-ky z{e1j%y!|5WG_9%4ZaKrpVPnpODa1SBLBPZF8@%1RBQN1qqh^aUWSN&1bfKA!m<7w8`P@9ZwfIP!q7jhgx=MM4$_ zsxt|gy*13VdNcC!dLaTZlGaKCVaRFe)1o@6)3u7}Ta&8+7UB2b)QzlM`PA%`dcItb z)U) zF_aS1;gd65Y4);51I$A=--Lt`$eZN)Zb5$sNPPZM``4LH#1^2Xt89Nyptc0{n48;| z)@fs<-bM&D`kP@>8S6Zv%}8*N4sDP3-NEVVSh%pm3u7+a6K5X^v1*~gJ{zqy>jyd8 zD^fP^!5`_ioE3$W@m&K02^9eKPh01Gr)oa>OjVTr3zGfuc zjE}|7)U-M+O4_m8Axh8|bTczVo-HXGhrZk#wSFslEnI7@v&(&~f-sOU8978>c_u2* zC~AY0&R-1qhK`&D-w6m$_q!n@EYLEZp3G3G&P%i%v7_(2fn) zy;-$Ed}Wi5q}!z!CKe}JD6V&<3+8X~V z=xDHY#=srk*}*o>6$%At?0vxJeMP{D#e3NUML4X@qPhS`69YK@R%)^F)&uOS&3SeG z0a%81B#bN%*BPMHMSbM~C}hydbNcW5Bj9y3dl&Or*esakkctpe7k3{8&(yHOMs{g% zwFl_8V+@UIr{g#f0~gEv5a&b?JD}SC*K2eLV#^KJB)bo=xQqIyy#{m_w#oH?N5`;Z zlUjw=7Vu2{_h&18%VQ_M&2Un_`@%UtynuSR5cL8eL&8^jyxJN=IqS69>ZNmYxW5~$ zYgbUqovPtOcrA1Xc9iXuPtH%bNu$VJNa1!i7MNg=Dtx&kPEMkq#z)v{JwSZP70Yl> z_fWJN8&TL2Ps9Jx4s3TGqd!F zKPP<(_O9O+n6ib<=~%s6n^b!4QCk>*!(bT#Pnn_PR_SJ~DYf%T8^5@9#f~QBD?;U|LJH3J^X}@5N zjhr1m*r~E!Cy2En+@bqTy_n|zA0s~k@zO4?MBBlo*^k=+(5Tv)w}No19rs3mS_7@3 zn}IjTQk*Bb4mM+@>3dSlGDCkudRD!GQKs)c13np%{e8hRxCiF@@vA1u+VEb6^_%&> z>3ETfzVi3Lr;#!|T2xI>6X40BPXFX>DN&mILF=T%#s=nhXlk;1+SAmIZ7rl)Pq7X;N=5cw52YdsqbgV@w)&u#|5sMv z!B!HwV{rvqz8E|DT)Zf|?|IGrg<{<|p_OcQkvpi6GWU;VxDEe7RjEWRx%*q~wHD<& zi3>O`IzzW_r@=OPkPaH+u&OL96`crce!v>p=q7ESJ!8r_F4=W)@7xtOCJp7!!oPl= zh$*u21^)%}JWmljh)~jiomcbtoMMb6h`Gn7Bg&ioK)u3|AFQs@hH> zHD`M4%LJwGAjns+jxuP!kThG~D|ciuV9Kx1Dc5qD`s-TS5J}38$E05QBOC07^*3^D z91h9uE!l0wDo<4fiNx5H%TC$#sX!Z@>=#SM z&=04ReTK_tUNt_IDEMh~B}OdRRBzYf!3d$|2`Yi?Gn0Mo7f#gyv7&|cy%D#67iH3_ z9!{`|~|hi0hQS2zHDO-ogw$pk(zev_*sk2N9CA499r2*z8hR_Hoi8yBRDLnPP^OPK|n5Gy*%Kf-Ya?~D-9kV;gwb|Yq2HHXd-r4L(w*;(YLlE$V zm0u_Uts-KCp(r-A6xE4|c)?M4@wS2vH;rLSOKc7K7PTa=u_)%X2PVU~=fliw0LBcJ zQo=wVYHc=~)ex9O+j7DJ4_2yJ0EfiH)iUk#+D3~L!nRAmfL8MO&N|Hhw}OKJHMJ&G z2=?~&z1a}_r}(%y+Zh@5t!2x~#6Lg(&2xmZXW#b6#?shF*74eW02gOnX)X_kQ!r2k zo5;a<#uOBH)R|9xoGTg1-DA+@-Qu_BiP>4uqStCbM8Kr${g+Cl?|4OlPCh8sFbukV zcKc0R|4m5wVU~<=mAtEhkkopfDYoS1YUVehXZ74{g-%L%7B2bLE}6&GlnW{;;39NP@YI7`qYsX`4 z-9B{DE2Y35oJBptc9d3m&^}bXSH~kvm{syUQ2W5KNHK5O#}%l|0EBvK0$)|xXxrYW za6xSf=AO#S;)Bc-Hh-V?s~%6s2~FIiCZOy?+jpESXkIDYdT(pq)gKcF&;5WmD7pSa z;w|Osk9WWeO4}@O5I;+f-cM{@_hF|L*A{#iyw5Vom3t8QlmScY46CFlBuV@ysyfNcpnOP#tZ&S@ zJbu#O)Z8wsy;W_%-Mva+frl)J*n+r9i6`r!LNs}p?Sx1i^S|Kp+=X#n4& z46AViRpMAs_9NtJ!lxfyP2106ra)plggtC*WQV*7+%3iSNG4>49H{9F)lz|N)>`UF zXeDe?ACJD6^*Wy}`a!*%M276$e%R>*7FaJgvxKr?iU1 z3l-SBJkzwbuhZ10^7F&M4HRY@!P82er2JO8&)ZRIfg6;Q>?DvA|G1_kKh$GZ%Oqg* zBoAi;KZ=wTkMS*)*FqzujX?{;1>0*ZopkkZLPX>h3iDq(I9gWbruESTWp9Ps0-ZCnj9W<@EH_BbZr~bPU&GLXJ1`*_Fg-oZ zU_2O8Mcy;Pa}!~e)K33QLPoZITv~}7J^>{+43wp5Ej;(h=eb?ZY3w}rHz?i_&vu9O znJCD`IGlsyq+>gDPd=wMH}mS`5&Caakh?o8UX!iyh=0r3G}HA@DplBAix;h}`a3Sz zFgVdyQqRe8)r2z$?p0V`|E1n+3N)ozQo6 z;K}*~Q1Px5a{LGjo-5U-2E8R8si%LtzQ`eMy|mOsfpsDRs7iHCv2a`NV#aK|U?dOtD^vcTM`v4mu0NzTl3&%&_@$aB#gQIf*k1F3soM*d{uG zO=nt-73ZDY`d)0thW#_$NNJnFfigGjB4i?b?=OTXP1o8Q`&KmkuAs(4i|(vgIntC%<--{a-7_U+V`+JQLsqMS zUrF9}L%i)vvly8R3n5> zqX1r-E|sd5!#i##<6i1Yn5ei~AXmEW`#j~RQ0T5kK{o0tSiYL&77w;vvi2PcT3^eQot zaxS(yt_Gnh&U-`|gae@3(-a{MSH1MZcOdCAE8CGjVkRZ0j~ntbfdvZ08s%D`^t8Ka z=P-gCIe)SYeABvYGI9HLZ-(=a`X5Vo_*3AHqfM9PZLAM16>nZHNQb5+CjjYoQnmDtMjdFjlJ>HH0zzb?)8 zqNSF79j5ZQ8X9#>caoIR%#{Cp#(fN%;c<1x<4)W*LX!E>8|CZqa@&WveJ*mJ)W9T0;Ob=0j56~@uZ@^kCu*=Wo`G>W}VhH9Ph7{#m`r_nZFbVLCh|^ z7tQSbO)h=~WZmhTQc3sUpNz$kCZ|!)$l_YZ^R#n@*JPQ5tVxgA!#8foKgnJAh3)x4 zH&#hQvHIglFbuTanpZ2ks5Lc!j!kYpw&{0$yzF1ir)|rNLKSNZ zXogzqZDQ#Z%HnBczMhJ$2pZ=8{=jO12lk-EN_otl`n@956dI!XT}A4&ikQW2{A0(P zGbXJ|E-c1=KUuitwWK%=zm&<7hXdu`Y2jSu;m;Bat3YbQbhoUSz~wI!LaRDer5<);J^o-XDI64EeExIqv^Pc)N_LjQ&&=it_C<> ztmEAs&j~@T+a{qVeM?Hk(BorsNltr|dgBJcGY6klHL>-Bp|ZHYW^*iszHOa#YpHsztjxOls@Z(Q6<)bPJsj-cpbdYCoI~imqZl-Pg*0>`v{?P>Pcpm1Z+?4Gs$hUbf-O&V zwIJ%OfMI}l)A_wwY?pNW)_0(^=a*<1Lj;pw-e4136d96mfqKMdTm4BczQW@s zH`cnmM)P@zoW@24ds2!#a`Iyts~PvSPP16Y5aTBX*TgDe{#yIoYgK^(8%lHag;&~< zJdH|_SBA_@uGLlri${-&Y6W2#LO8f3TT@o(x_quKCDo!x*E+Jw=LFduWonY7CSV*s z!t=1x*%B|51%gseiYaWnM~+iU9IrHGX?+^q_C_QSv8Qzkv7V{QAh z-_CT`hOP=vR~hsei^ex3f4U}2jF*0GQZ}pp!Vlan9$!VgoSa9r@#pm}(m6>ds+k-) zP1Uc%Ma+WHpxGD=DgLb#y*1WcERPZ@^$XV>VInm{eVF>STm2}&DbA4aL?=J2xw#67 z30KlCc>~?lij8R$3Bm3(9@0xtp;-nm=2M@QHM)}axy9mJ*5MceE|>|szJc0U)5z!Q zQ6pVTIjTG5z7-Mw0iQw)cnY2G!>94> zDsw4*SWSo~oC4h;+!Bqjg!U%r`G~=>V_2IAuRvwfbL1=5&&HA~84Pf`zK~Xv4e}o= zdvKr3O+LJS$Z-6%dsuja+)f5Rzaz`4rNMiPfB9dFf*i>4W7~c@d$o0s1O;e< zWSl7SlqN3e^Q=h%fOuJ0Jyzz*dbE-!PcL39;f&1?+g=c*JO z85D@84J^`_7SYvd@e=7@>K6LD)o)A01Q_P;l5HTt4?jO&c0yAfgP}AX*S?^0A|N4-;JZyUf9TDS(scjR}KS( z$#2PuNJ8%9D1maljaB+VNY$KT7^Qgg9GT#V$ z!AKq!@2zVL+%T$LLxpp_3iV~6kluO?yFFuy7WKn zb&QS4%j{J_|FlmD$U)o$DyVCZDl^J#kn|{E(rJI*Q>(d<6_pCA?fnp|e6W0baj27j zr&BCxwi&NDU@rP9d-1u-`IoU~fnqBvB0m669%{{5tl~;#6Nd z{CGtEmgEV3&L(gaRfOKY=M0lqMc0^n=u530`qiXIb~^9nei~>Ss%M6;Gl!ZiF1k(1 zfB5R{_HiP?9-%|nZ)03zyZcIHI}D~c%825?j3vq595tN?Z!}l_@g}`V&=voNn#7$p zH30*^VCyf8MknR%J)35LbgRWhS1|K5gVUAGYf>zEjkMv4q#>U8-^0B@%ytQz%t6-4 zJ{4eST7ysd$Z8uds+qBV`>7pX@HwyL&Mk(DkTYugy&1kOmTbMAwW8avjz{8M{2DbV z`Fl+|k}UXT=a?V~{nqHr`7BwuW3qPi_ZhpFWe5Bd$fEa--3L62_J6tHA(%wISYy-T zONu|J+=v!msdG(Xu%mfW<|hx!@Y~%TE2N0g?BlmTX%Xlmkh3z`Y&4U>yh8#vZiL!x z3e|+@t3c&alDz~TDq>C@lw$ryCi))5 zMf&@1{6bwC-$!ai>{jTWzg0X-RNfl6o8f0J858d0qzZB+vLPhOf-f~cCeq(4W$20( zz;$?!c0M!Z`TIeM+r3jisO8Kyl#z1C{(SMoX16W81f(k}`#G55vL1Tt=ys--qT0A> zk8pnq#>%fQ0z`>ad<61cAKPIg=H7lADRLeRtFc3tfObR%$ z#ffJ;G2EiOgzz!31qw#trqOWIdlf9{DDlho$Z@CCz%Ng4#0u!3{7%6LvzARROI{qE zELP*URS|AK6gw-)S`;84f}oj+CR!4G0M|}l1`J&PJ_R0KEX(~C&Jm37HLUl#`mcTG z5}!g3sQfI9LasWRtpRQ(H0{KF*%lu6of2Rxv>c_u#hf0C=2;pSgCq@c)Ye1O7A+mf zbYHs4g5}*>=P$Z_yjh&=>f;z5)ms$4WQJMNv(zPFvtpElIVVL9D;AB>&9VjU>uW?X zvn0Ul@b%?BjYSY;7#JH{=nQ&$g^k%`Euc`!LGo|Q!i$-o1D6OE@s6YSN~F(v0+&s1 zm0QtU68=372!NNmLP9}^T~T;om+^mX>BXS_mBV6SVs{ACOOnxMwJAkeDv72`Nsbr2 zw!BL6=7q+8F)u^758j&LO2jBm&?Zq`fZ2K!U(}=8DLIsRs~S#xehqeQakR-XpllmMD_4 z{u-ZqiV6L^44){x7i{7}YFih2m&8cHe32wgAcC~W64LvY@D zpxX}*gJJI%&!@d)fs6E?Ejv-%zi+~VR&9Hb1c-@~Y#!EJ2-vKb(Ml;z@do@ZInbYQ z=N%p%PPU)5_Lpc1g!{;>zYaQ*RZt)rD$bQKyjVgl>`DU&MHzJz&x8?=)o`8TD7R6z z?IbUhZ!LwjKuXfnhE}0qy{|uz@+}=WDwh#nt84)UivZ-gW95OqI*{VYrzGp$;KvZL z9U1{z0iCb$&G}znG{j6y-F~YH)E3N5<>-FYWRVqAlFY*kS^QqK_3eK*|+$yx^HGX;NIVN^ZkRzHAJvMu0td}fgz-ZShZ2IzV zi+FF-?`Wpb2-<2Bn1EK}J(iX2u)52t-$qI+|AsX?9AK2-Oq6ZsgGgzgJ&dtV2u74} zFx51pgpTVUix$fm83?XW#1NBlsY%EVjJMLp$b1_Pl@ZkLWxI}a;;#mj6zw!Bjb|5@ z#EZgo+bwr4Q&kjbyTsw1kr9D+#SjL^quIV=K0jYTcv(LC%>)X(vf?KJd+In|y{&Hf=gp@KZB|^4r&lbNV=4 zH>CWjsmm9+|NStoZ+|j;qLU*XE(hTo&vc0d_lDivXq1NstoApe?_-1b3Vd2V6L8)Z zWMsXh)?j*EX1O$ex^Bue`PB-Z)*bD93BG7B+Hxrf)N{$o%8G=$2u$k!)50VDPBPm# zth7W`4S#*fH)YWpi&GOlk*I1nr!b*2XifAY)YLr3=_jr)YoVs*}-FJFy`nu^u4Pcvg)T#s_K z(|zn^Sjtt=2kuKE@v_e10LT)n@s_BhUXPW4h1M%tv3P+`Zhy3l%XaxhELAfl*YOpi z@(I}&X6<;%q5I$afa&?z|LJF6-~Zf`C5e4F`;<|1@)d_T81|fJaG#hO!)?Cnb*=h?DH+x#wcq8Z=>QGDB~A3_J?@VxNvow_1f~e0 zZa4db@3s}OPuY{6p;#W~;KLgd5bBPcE@_G1Z%0;TCIn> z-8Al`9ov};GDmGoG}n#tSgl#d(O=az7+H)_Ew-^@a*r7+9*4fHhXIdT?6>tKM;F^L zFTC3N>Ht~Q$7LbDL4rQc$Zy zBn0j+vN46AKRnW5)g|j_p$U+A*f$)biw`lvTNOrsH=sW0kXa3eRuccc_D=jH+v_3$ z6R*as-$m#Dkk?D+gOXy^&}Y8r=xNYcI~sxCIShU{efn&sS+SG$Wu!31#P%Kjvl&-~ zk>*~tym6Hy*nn%^KdTy-BCRuw$K8oZh%1jC0fsMlW935Ep&xqmb6n!P9)g1}r^I*~ zrxJ3bxZt(b6h{8-S~PsBgx+ALwBfl%%Y_Oo4qJVBRhTLfli|KHpB zKZmX1#YkrwDkG1tDMj!$qT8UfBm5cZ4j1iezleEC6kr(Mb3_M<^W*_h7ibc&4rcARO)wu1s3uKu zN52!ryD{*aNTCYh19Z=OKIy~cMJ42D&UADe-Vz*k6C2^bviq)_Zv6g--go_MrccG0 zb9v)0?Zr!SH2uDrKIX~Ran7?ymsevJ*Il1S=YK6IfRH~*Zn!d%#aW&%^z-B$eG^hP zOa5`0t#{It(ciC#W}imGPwlJgh7e~KP+WKTmcKS^nojw|Cc!C5vr)Y?cyrνH4_ zyA%G<`S$P+NZ4yD5OX?{*?GbH*dChRdpHU1dAn#!;4RZ_$C)XRL1>cWK4VFTrXc() z?3XVThX!$wy7hu=c50xW*wWCnFlYg%5cko*_;fE{F)nnjB$r9 zDiCM~G&Wc(iL9&CIFb~M6g6M&Y}?FpMBzz(==8hp#2C$d{xEN{n|*Y3kn7T_Y;25I4o%^wWCxzrQ3!t#M9L+rSs8((eEvsD|>O!O4af?FW& zI@(!os~-A3Zkr{g72QxvW}J`?s5GQYOoXo9asieg{nHLoLze3{oC5oGV^4J?0%-7G z{_Yblh~08t`WWG5a<@&eATokip*r5MYChB|atJ>1tHm}jvJ{6h*NFCEh;g9`OJ}sk z2tCJ6WA|Ie)Knc+g*T%c;k-gK_LOd<_I%dMRt|>{9s1DQT`6<5ZXLXr@Uq(eJ;3L) zkeB=E5ZJAs$f#06(N067G}bb0<>-*p+fx}-1tx{vZ(*~JU6 z$`5Qh@V~>FrQGv1w?A}=^Ly7>#?DRp=SM;LK%Y*V-opax#OssOYsLH7&q6VV2Y*Wi zOCKj?l;$hi$>uB3bceH@UMJM^KMgqkEy^(@Y7(bKzI9N<*t;MMc!FQ!YBqH*%#H!^ ze`@siAgWR6{;mLaAs!!-R@X}+|4KlVMS7D?bSBV8fuOT+A9|T9kIFwy2dVn2BAlVWm8g&lvFZu^Mjc-a8WQ zZ~O3%6GCe+9;eDA>qg~GCR^QsAC7iQ!$asl=5L%Bcn18c&}zkmqhrNxgpIx!rVKuQoQp@eawm&jk`; zvdY|*THWK)KSGar$cY~`pmPTrZG*s7tXhtP0)8H&7T~UG{Z4$QN z-87O!viPEmp*kfgg27D(D%H5IGYP-vFNrtc8f^Q}sf{k<4#sXD{A2tgx81D6!uvv$ zlxDmMq@r-k0+f-yO0ZAqFmD~L-wuxjryHTelq+I;baJpu*WpAKe4KchtZa({Rbiba zH&W~SEj~p9z_~&*0_kdO60CteK3%=BV@PfH&lj!qJHkxJ$U)Qo(VwR6eN2X%J4=O! ze{MY}T(;O(3Q9-(J@EAq9PKLNOXXmE5KLw#*y(tl%_umsj>Xry5X{+YBCIfz=^lkl z&^rm1=$Jy13~1c;%QKDT$|3y<5pRnV|@sKfFSW+)hAJ>hxLD0*(j zA0<^-LflD4HTajLuN}?)r?$WdsWL_JJMC4zwMTLeFNz5C?}=)=aXhl!%;WN=T^#ba z40y2BGvy9;=QFpKh6s%ktt_?#_L3|Z>?vRa+5Crl+^kDhOufs`mwg_Dt1uc;1*s$` zD*PTJoR_UX0$&EQR2YD}6`e{D5K8_t|1;Bby?sYDCC5YGT*?dNm$suZ_WGN_J`W@vr|Z?p=M zF^|0{urQ=)P6C}#!!K~JC5@eWoI#b6(u)S$Mu*0Yt2E@OX)W-#$4eji{*PN7TPogT zUU^gef7M#m=2JO8*O$z)B5`e+h&U=wzYI`F?8f|9g`)pnBiu^EKGTPVQr9n{1E$_) zrUKO(D_HghT;)4AG1`30A}5d1S5PpMMT7KAPsOI@{sI%NY38xX`*TFO;QfYaCs=7f zf07X}5FzWcMO6t%l6eM@RXt^#_m;!t>XpQ%q#v^{RkZ0v!&te&t`l%dKU?Y2ZvFG1 zBfCb5?`KKA9y%l>X1FZ#U^q!xxr5bOR+rAgRn&UN9A_Rd!A|3;*DY_OOG9U*+P#|kp{O~M+s1G40dN%o}ZhUW!QhZigGT4+RRJ)?4fr4I$Z_y?& z!XZ&`j}ozkt<_b42Yt%h`46;Gdg+0AhiVEgq52@$9O&Frn9uvmMpv^~1b{>LadG#Z z?dXBTj}a2=)sFz*%7n#GRpvg-8VxXBty#}t|4FJ7T%wm8I@JDtF&G?aGw{t@Twl@V-7i}f>Ag?7q8D>}gjla8L#NQRm3*ESjd%V=k9Dba|43YcsnzxRWSVxy zZR@mAkhp@q{tSQN7YK^zN1U&jfx()DOg^^iZDpLJ)6Bxm!CUhxoZ! z*0Mh%ElPRhQ{PRqIpOT78m|yR4|#BLg51-ln`qk@s}S92f0GH&_()YudMByq%_?`K z(xbs%iG+7#j>{lYo^X3s66h!Ho!A#6MZ%JJO-Yd0_ZHKKe}lj10^FU(RS!bM{#{pB z_j8f=VbxzrmXnm@LA(jzX?5+Bc6)*D!+Z-;dTi{?QqctF`GgJx`%vw_Kh;Z3mP~VX z<9f@x5_)_ff>B$$8u-ZmG&UmL6GfBl+&)XZ;hvcB$Il8!>++FURYJNr+V};f zyy*UvR{d#*>9vIwiS-w`(Lxk&(Lx*Tay=+~m-rmH*2PMI>`YX((K+m!RE_tm>4*8# z;ScmNl0{Q!cr8u>Jt@EKtK**|%Iy-86u)I9glSfkowAhaK2gtGum0xsFqf4%gEe2c z7iTFxIA=%KQq`ElA;^mZ4r(l6t9R^UmXUio$MSjHD+`_V(@worAs4B{yc<#002hS& zOl-9M@}*RL>BxW07DWb?FK3(7K6klL##iVxaPZ@YUaiC!u=kA*F8M*z-$urm)X1vY zyjMRXtPK1=)^|)q73op+B^WI+b`MeO^%}Y==kS)umfGV2d7kJSTf;q4niG9uWQq_MM z@4a(5cH`d-%5!MkoPh&w2EFtL&%m1L*g{gsZI_zYdYu#Iw%bVQ`bZK0S#T)k)^X39Es|67}W& z5BAMBJ7%3B*b5dc;QEaZ)?_K&`uRL;ZVn~GJNWJJE@tNINo#yN&IO+fK@fCu!$+RK zhl`d?5Pqeig6333#f7H76u1$kfNHcz%c666NlvhJ>KxmVPsVY#Ji_=~aAo}RUX_X> z@fku@PYulFt?KakHN~r%jU!ty*kA0p(Cv!uwMq$dCtnc!V7l4WdOFqV*lk7#4v zVam&q`0l5O%J({FgT|C~t^R@N56#3d=HRE^BIrB_9PM1_#8MQ4N749PzPS5TA=Ao) z?ron(_9@RL6dd|*3>W0fW^ko&*b#4UZ`U2i1iaMLjScKm6~!x}zlw3OlLIZ5_CeC# z-lS1u!Ml~cxGGe@UAh5>>8BZc@=uX4xt`d_9wNSuSc^-pM3wbZGTn%o8Sh%B zhXXt^{f0&@r8d9|V4WS3yz)o*_`=JvM`9i)B|Vw#s?oikSTfJknm2_H*`SI)zK-YTN{x>bC{!mHRR zf*kMFyYP!~wmB4pCUp20TH-(BA*~g)Qm@bLERV27411%bD^3+i0eV@nt~h2hnQ3*iW?UNZ-|lMmCef(~;d6Wg<<36$foOhpA3^m=gdTb>LELWr_5Ntah&b`|yxs2FGCJ-`%wK_Y zA4I3^8!z}X0|1~F?ri4SaWm|t5tN(xznDqFEs1~qF{fM@Zk_P`Pw`5x=FDy;tE_uL z0xG{8KRt!TRVKVXyLW*&T2*ef&=O~R7V(R4T3fDXuOop#Ss3w0UCsSIC&aPp1;2ff zNe>I_)Jr&>qNtN*ihof?PXlv47+C<^2uGIrp$~9UFxgkn3vbUqYV^X~5O9lRRbSH% z*;>egKnb6p8m_(_Qpwu(aBNnKUqf148tmXz(^h{)YuuwInew`tGIC)kDW)UJ5IriiXRs@geFmGh_E)9~{T@Ode)iq}TwlIm z&@uhx*afGfl_a@jt=Ar=aFbGxkeSX*lpwra@~C{dyMOTLmXSwMDtli8@YjYA$v@+p*- zVe{POG_Bfqir%x+E!z-X_N%s0yg1$)6VgJK6+sKBb5{{3^;K`Ae|MZXRo%(wh1|;Y z(cb9NnL2m=p0}n~ddy(*SgU9!?u@7{g0lV|wZ%Vb+fp)p32dr)e|-qVgvUbc|B`Ul6)TR$`MI3)O^^jsDBKOXuL?`ut&*TXWi0Gq zlDV-IhA}HcJlc}7QU7?RyBdJ<#CP%=71`AqxDe+B)69AG`PJ)V_8JE-VmJPxyd>l38M5D zs4`?obr4h8F38ii10LHH=fcE4+#*R^2uHwAwDmb+64)_bv5-Dxg2ysH2qr zr2Dx{7v~TA~Nx=YekYH1u;r@OftR?*uj(=CUvR&q%ON zAe6d3K`x-@_uSxA+*1pm7jlgA5RuP_9{V1MT$*Mb!qc;29hNrCi8DT3WaMVwB(T#D zDmsnGmp=&oF$NOu@CeX%5#=|U&lT@?1XMNRZexIm?AB)M~K5gCTzGFEF9I5Mw z?30wzOIk3NyJw5-V9m`o{z?jGS;$rE6wszlz>V1++G^g9l3c3m{SA_HkHci$W#(xv zD=#l5uCbif{>m#>ICVJ2JTIxTJ$0VKjA-UATTG(tTp*b#zOBF;z5QgV59Ru|nRL1I zm$9#jn3HIX)7{Wv?Z3NqgWRy?iqufqZ!JTZHT#2yFm#jph-n?|tfK>#*@e17^@vu; z!gpy{5cVJUN0uwxsp`A{d|>yfjqaMNER!a{O0HMY8vMwB_kG8|Ky=^l>E__y!ibcU z6Z_xfZxmFc_gIw{f9&hAcR!ltEHXR}fzc+iFb&Lp|D0#WJfTNgG`?_4MV-e#^B5He zit??b2)_4theD@{1BQoR?J$_unsV?*CY!XUu()rQYH)H})x?x2qP3`rPq>hJKs^y|+XzKj;I_iS0rwiAxt7{fR{ zVX;x(tU#$Z?27CNxW4Rp+d5n~_+*@V{5r$+HkVW~$Q}iwVqe50HlKg|XoanY>9kF^ za9y=uQ9~7d9?MCPS@pbm@cexiF!A?Q4K4FL4DBm$O^>R^#3%D4pSCHQ4C^_D@8|bu zPX4rxQxg}N?Ww<(z$5Sei~*z9f$&(hDkf5!rJpY}5}aEfG}l#YsJER}o$sgdm6`8e zJjr3t{AJ7Z*7ncJQugB)-pw&w|9dUmrwl~j+p(oB0>b@41NR)l@V zRK?^ZdnggXeo*gEyv^1stpl{7Dzk&W^Q>gvD*Uc%Kp*z>*8SFfxt?jk7Rs#!hz>rf z&>k28@zeF=-GU@HLdZf3)^DLwrCt2RW(RU8mKCh@MdHPChj!4(r5|fz5I!-k8?(`+e8B z*@)&HZM1ts!sH3)(9hoShm6RX8n-Rb1(ooS_6r3V1hbYwj$vnW$>q|>YIGvYd1g02 zFqLyS8f;)a3rJqfo4wvU(Y7PfK7*eNFt96@u_6w2JgNQmA1{CdRCHZcSqS%7swRo> zf}H2Kc3kxFSeZ0;`d3f*etyP}Ki+sgB?F&y7^LzqGm$+l%g5`!xlEQmCP4R2WG=%SBz+Ibbxdr3tfd{%_qU$ zSRPH}>R=nDrcY-W$&zoICaCakPXT>Gw}B7jp*Y)90gzpA zWBdCvIZl&#PN;>3+fed(>($TFsDtwwLp=Z9D1armP4#*=!U#Q)T^?d^P{H9^)B#t57=Bf&E?ffmgH2MJdJ3|k`y(m^}kby?cg~08ck}pR&hZ44eHx(GC zsMYxSf`a-3e|!*r0eXK*`>Ra4!;Cu7k=+T4_Dt#KANrLfpD45U3#;!d2&1+htG%=S#1)N4*x9C1m7PjP$N^|Bve~Q7z{*9uO$% zFL?|Ly6-yhd&X5N#W`8*lX`N%lR`?IpUXF^cUNA7_Q<8 z7+tQM-BDM6ts2F%Dny_c<>=~g=a!XKN*HQsOXKEGJZ@UAx(6;fW{hRNPBPq{xssYz zBGEe0L~_Gt!KopJnwZk(d)&_`&k~a@qW26Aizyz(gT{G79PTW9>3_I0PaepvD-9yQ z9QF4~<0$G@$Q7yJ9XM7-Iy>!{ENQN1@>nf3;BL??hh`m;j>AM(_;G0BC5uI;pRlvq zfaaS&1T`#%H*v(tvl38a*z}x^dBm!-V`F|#{!O!NbywcEQ~)}jIkhAio)kk@F3tLl zAaNd*Xp!>lU>?u#mtFxcl%Z2auY$U*>YoDrkx?^+d*)^^TUkW-7mTke-FpBp4cI-T zp&uJcT6vJqPV!{;Vn?UcZ2+i31v(5Gpk>ocN+>NZ9+Rgymils3sXI9X4z#7nPG>}q zB!+LM3b^r<2z5%5wG&eOJbVxq=Tr2&GORVS)OQho z7yt9tcI~>kvXIqTXm~!>w0ord}_xrsekWwzCJu7Kn3M^u7b4P)QWa(G{=+{(Z@oCK7}6U z0+}B_J}?{D-|oy@R+yvCHij1nc9D&H#&4AEHd1@Xcj{?K&*P2DX`&Jrx!3gMUOSww z5K$7m-(a;62Zgu3+Hl!;{gd^V6P|sQJ@Yv&z*1JapId5R(N5G3YD-jb_VUk+nPLA# z$F$1>1-Ca9UB`4{emhZyvinkJR1xaE7Zq-Ca7%(@mN@)c5t?VQ8z~Mj`z(xwX!eRV zi{(_AU;!Vs+CvvQG+z|!i!o-ob(vK=`B&jKC}z%*9PDNNjhx`>u0e^Il3?ycMeOA{ zbZ{uVXT6WitNTG2+#A+qnMVeefso`tec!Tb{qJ*bBipdQSbW8$mdwQ74)#zyQ8!ff z5m}WfDo1P%xuD&3NGfM~(uVfB!=~Be)@_@!D(-FC2igJf_diutF<^oC5jw4cFs$A} zD|Q2TY7Frp%3c_ecVvY3ay60Q2G+^nrq_+;XdZH*?vZg*?=C}!?WVZ>W71-x>Q*IT zvb`G#kF_TX#FW(-qOWvw*hQ)$GpfZ+IVwzie;T0lw5q@aD$bU4JW`FbO8u>{ln(-3 zcJ9LTnCn~Gq)VQ=b=X5BMeJ^ZyQA(z=0b{?JEt|BW)lRQa_T{(=@6U1b+w9 z$Nx$VKYXgLa74-Fdh|Oji_k&TR@1H=%p}Qn5XVr3eHyaM9}&zp&ODbRoycl<6Vh{LHxc?ws5}vS7#!4qdmI@K6_PRKbj*bbjIFUT>#k6Av z_&^Gvo@k=YLD*XM*$t}Gu5^+_g+-{@xF4wdRTy5%XOl6gqJyZF01=e4hU#n*dSTet z98tX=w}O55%4)|yE~pfAOO>qFfTULL8QJ2g+)^V8^sDS3vYrecFuI;*w0%2BkFFME zV;D@N^=>+=o~Bg4~$No&PKvAkflSPZ~6L6mXkl=xQ{?$Na_S^C}?FRXgGjzIK( zAJfMvIqz;0XcxtCgx_SQ9g-9_H^_d?U}62VM24YrZ^kNFgJ&h9R!(Jb%{Ho4Sn*L6 z1kz;Pqv@izPW0XR@YJBVccwxAJRzUJ!SJbY(zrftqJYMu-9EXxJ}M7~0YgQ{19n8( zJ@4OF6Z{}OybR%O8?JP3kxSTy0o@F(h)j$8o*w6BZ1X>>1}z(kjOyVMan0sDG{K-<%HY28kI{4kz^om@&d(VuGh;qMt{2329Flk1CyMT)Tg84E?pZ{;3nc z>kP5ml4q+!P(+b@+p~0eW_Vfq6AH+3t(;{(9-dzBuXAgrjNVC%R?}*8)bGimM>2AE zF1DuDnFxkegDPsTNO#%KFTZGlH=XmeC;3oo{!m{rNn307(9DgB4=`U?b^R`HUTo&Z zr1q8>=mpN}pPHpHTIwLrgNC|5T5zVPUHdc+F7XnN4Wrt{dbN>{;+a;N!|eZP?}~O> zO%A&sQRg*l>Jrfk2gx4Obl9|&$h#Itf)@`K%bckbob!KRHIN6ZgzmLK7$avj+#Pnh zo|pYsup1rA*mgA0fXj(dP`7keI*jA~I(QXeAXQIbx1|YVMFZ3SSmCIQLi;g>?wjR} z>mM&7^YSI)$y}kLXA7p_@%R}GVECS2$cDw`%IpS3KmLv^-!I%Kw4CNtmKW|-?vltwnq`J zB+Sd^5&V&#yB7J$oq{Tr*V&Z@M#)5end1jmGb zq|sB_r)t2ox#^TxkLnLV)nN4}s-9f?JKOCobs+W99slE)jFoqtSLwX;s9xLOSy=1T z7kb2&d*{(ZN6t)$J~l@C8*xSRsp0MEG{#WiSb@1C*Llm_s^^5yAtDctn*b44-qws5 zT{ncyj`_6jM6&67$=9jngeP?TLjuVP9%mrkW{XQB3nW?yab^?v#*J?>VCjc>^r zI{#G9NbSd|P31euq{|J2EWqD{_w42bKH+Iyp2Hj%-JyLxN!0&988a)*`j-F4 z)mH|^)n#2L1cC)?g1cLSyC=8@cXx;29)i0&!QI{6-CY`K+!}ZJIy2AAJN5nQuIgLW z_nv#smbLd@D_Ula!?3?`?Tysv#2JIbHBtqm3e7_+cf+vhwg*%X5SD`rt*+{pn!!M@U=Anr@Y?k^@Dq#it(+sXWzGsRvqkBeI zPI*&}A^CH~bn|30+ZA?m%H1S!H%qX_yKyxyjj%eNkLUKtSi0yJpM;=!2@O{dCdd>j zgbTwqUK)k?&T^FIi0lqEABbT0Lb?-2ZVYNy%BK(eZbEn@dJ|N2aH1hhNEnqJ zkF>a~=bGIgWobxAJG~+#5XwJApm-j=zQXKt8i9mRVT0+@L?y?tFw%b3UJT6LsF~}2 znY}J*IWXHTvEoj+yn1%J-@uK7n^~%}-*paup++pcAQV%&Y{=IiE98kGWNFyl_F;Xx zhUGyzsye%w;$;>M`8<o=Pr@mdM@Va+;YS zQc}toF`;gX8Xx%fS}V<0<+ecQDo*a)Kd!pe8pM3sjR+}qPZEk-1%W;$^A}9&s2oi+ zOrcb`8x3>{7aecB@-ckEsTw;dqFln zH}Ya8Fh}_1vrHRIC2)pA?dWF5u=fWK4;5kL^jRVACrlQa zucwY$X|U~0@IDbWitn7S9t1i{AjPS(FOq+ltvn0QFx3fB-FU+Zxk(l%+Fu|d4a~3T9SW$Sjo^mnecEDu>ZNi`t zaUzH<6S&m$2~xsDe>?~Tf767c!Shhpq5T+{st3c+qy4g~2i^Y1<|gvovWt5>gBM{^ zRvTZTr=Qtp;N-9%2Q2Yt9Wx4R_Cvs)d>9RAlLFMuF*&hi@ILov?Kf-guEh6EQavn2 zuA9<~Uab$){qyC1$Fd{6oo<;(gwibS0n<&aQarPLN0SQJIJ9Efr#yMg%lh2%-TI7~ zhasw2`Ef?EBXQilJ(<VaqzmZo)|euwq8(TxH2Xo~!2M{qEB-?5_PS8b=V>^9WwF2QB|^vP zZ7QP4ejevwy%u(*(K3L0zLYHE>|i?%!m*M2c%Ae`G<&?oR^yTE2nif6&+1^h9N)?m zkF0Y!=Obvzo%vC=!0b>8PeVZ10}Z3$k7CS~SFm7GG~H=Wl1&%2$JK<_x}Cu^d^MrV zSe94Jz*NB`YnVAuJblBO{NwAcENhON_kdq*LCLrLqdvk~Bo)0)xb$2_Y$&pNC7_EfeaQQpF`D0?zAp&|qv zXVo%X>fv{Jj%P^>GI{Lu{XLtdll>*g zK!flAeGFKKzA+t6MjxRuKX{(~)lTA0!%T5>h`sk&Tx)3eE$j>DOYxV$YUJEjkZnn{_hIjnF2%2kF9lm< zRl4?PL91%8^FkzxNmJf;lq`;4dH48GPv6Vpj;F9tnH|iRVx4uQ!J2fOw8BbiJD8R6ftrA!pEw!!{2n(2Yolv8p+BZvNY~fgo!xnPjelj%Z3&91Ivgf&_$C@ z4L9)ES1TS%pHt}pmu3ctTF;fIuBO{82OAQlSo*1P!bekJZeA{HoD6Ex5aCMP?+Wq) zLw&erl}*%-@a7X%Yjt!fB%y`{K<&z zEFwX82Kn|OHBWDDWppqfIc^hSM-Q5-YB$pPy1V8T8z>%l$vvmU$l0SOB)4m9k zFq$}ZDNhwL=X)YBTThlX*3V(Pdhz({YO1;_+`J1EBaZSJ1B&zzrPu+NbauR3r@EMY z_yghjC^R- zN@v!YecxetAGb~$0$SsKz8%!b>^G^d{Pw?nr$Ftyz# z7n;RZ5Zak4{%*5{t!JQ!Y(_`M3Q9-qJn0%S#HF`amXTJ}2^ed2@L=aVEc0!0jMevR ze$@^Ml_7Ou_{n81V(S;Ty<%9Z)*(!8fi8VR6Z1kVO}-rW`-Fp4bYp5GHQGn|@5pm( z#_0n08Dql>dFeY3rHY=SVn zOAk6-x7^7y-ExGArRiW9&cR}V&dQI?1v(s)CAcocD>gnO9Y#QGW?wF2C}Br9ZTt}X zfJ>KM4gukm!VCyPVu~;!70Al#j+$|QgHlH|#ci~=H}k11mGwiZM3sP2+^XvKpKKoa zR;G4eI;+_t^6OR)Ld<^G39Ln{cfhtrfsgAPNHq0h%s1PXY}lqqt(XM#t(auYvqMR% z(G;!4ShGpr#Gc$`QnT*5&>WvJcp7a6kXJs@Bt6FoKXI_<;aJ?-t4?Wh0-`4NUkSn{ z+HguatVlatN0V^`=t={-|L=6Hs~%*IuIYX|jDn=lu4Sz)RNp>|GR0I!3r6z-vVlnBMlom zu^gvmpC8tswF6Y}5rDkCoR0*4;A8cu8L3Pe1JPKyv!?3))jWH{NXX4}f%eOiHOxZAAXZ#~yd@U?LSEiz8uK^JuOmQ6LvXR#_rZ=tuU5*7 zB9X;fm39TQnB-8*^3YO&o7o~5g8T!j-^Lp!PX7IcQ+L0`^!w>_J8tD_@uPrw9GeHR z-ln08V#Zk|J5sA-6bs+@h4@{G%YMN2BcuqSV}62+mqX9io*yozou8MEUOe96m{^-> zv88)^N+$CXAneby=C5~rf5B_Fm@AF^JQHBPCge1llOn6pB#Suv)80)>Yt=uPKb~XG zhvu}=y~G=4*uhEPQ}=KgIWarH_-Dw8*krv-@PUwQP+oWLg4M0dSyf9cw^swmq6p~~;m?bpLi#ekW<5KKyW_aof{_|Oh%TPFYz?j>1c1)h0=^*y<; z8xF#mV7kr^bb%!cpvb}LT2}bvp8ltW&;#EH@YdEeiwKK%oQ|2jp@>q0!Ev6@W@!1Y_nd5C z1ut31<7VMLGtclga~B0K;4=Bo@yQZjS3=}*=bH_g=;VYRExE$?qL49nQhxn-3s&_ZgaEjEN)vi;Z25Lr|o zgd!L>-6PM6H%o#h;0t6Psv0?oQW{Z;XnA57jb1xaHoz^{mEJphJT~0Q9N~#^P8Oe} zS@U2#y(Yy>3ovn-^7&vWJm&GI&){{L<Cc8hSDUppeVjtZ6KuBj2=ll7{*42v#bune0b73 zmtl?I=J@`G*}@!?r=gu%SLs(eVez~4aY1C%bvD!ONXqs7Z8Fu`fJwckzFy;F_igS= zF%Y3&HV59%ifs|*7BAc_UQQo7GXdS{>f4RR;PnXJP_B2LtcTJ1gRcE)SY(?@F4;9G^HYN*%%>W>ZwKweHl#cQr2}tXc@@yRQvzvf~6Xk7HsY)lCX*<^PU$ z`Ln8s=cr9y?XZo_>Spb2nIqZ>`ZN+v*~C*g%}i`wRLO{8APqz*k?)pHFnuQ1%Zs{r za)Xzoiqu$!O-!oAB$#t5}iB!T9i%)@#-f5te7a}MVjWQwz zeV!brvAq?hx!Fb*D<8#yW2nI&8+Vc9?KCcGoWfOS8@Q$ade6y{8Yv|ySd+hjm-Gu{ zHhS+a-N=J3H6!scA+lZ5;0IM|bKi|)hTJiABu@pr%^6xXay+hjrL(odbcH=qouxY7 zX&L|6;25L;Tr~ADATi@T)|sY$-pd-*s64h*!8v-wlAgJ8F*)p_qpg=24h3m5%BV$ zPdU^~+knOps7A*Hjf=-V9g2--9xn$4 zSS@L1-i4d(C7Mo0bH$ZMDjSbF%G!_5)PcT4c(;dR;47tB5k6mJuqEM08VmHN_d;0_ zWQ-?yG|7&rbQkaFJ5JxSbK2fXZKVFrop7#HBJvtl>#Lx_b-ahS!I;Hc)VpS2=%RDz z>$b3)Ij&t8R0XQM0x%am%s@f ziJi``6(H*bfZIFaDZX)ywOc8Nv=J8NTDcvET2eX=h&G>)!ZwVAWnC<|jGnuHgURrA z_z+0As0f6K4}U>9i}LbO2=t6OCjNQ$i z^Ig+HGVtJQ$|KTYfK<@6usnQzX>?4ALV4MKURfo@@bP5_gB)g@N~urE5<%o~?4EUT z?JMc1eYn~TjLv~&dQ?2(LRwPCtGAZ*$x_+l0|v+-6!A1TA`ESvzMZ{!i5HXoUpwK3 zY>EJ#$elnpoEHG*%`X4*o8Cc@r{+E7h909B*gUns|MyE{*C;;91&ykj4DMcsH zTExM_f*chMpS@Xg+(wud#+THc_S5SJeeHJe@u^oL8UqpDpxuuwe0hpfy$MuQ=mNRS zNcGy^cm5+cUA`YKAVvv~ef*ao!W-?O*NA3}q2neC{@pYND_~!B>OhiQXB25tZZ1I| zaS|V6_YaThQ1Y=vo}_1^)dw4Q1Eh{-KgE!R3 z^QTBh1d2XK*QN2>)uo5rI+=SYCrWNB{l{G$7B?t6ZIM6dMRks8-437JJvu2LEbL;weFzy7HY<6J?<}(tF72FGTG4g|F9$H z2CC>0KPJ{+9r zs-N^ea%!uDHu9_aaR|I~-!aKj9W=^=F9~qVzvm|Q?aB1_qoDV!zw!W=4Ga>@GBs%; z#O|D0!?+o%QP9PtJM5BnzKeRceb4PO(sf3fM9KacGZC~JVO-M_;#@vRF}n!s)`#w~ z|6-l*nIr#+#vU*ysF>%umpF2~D?7ho4md5DJ`8)i6$lBsi5g5|AJ{>$yNmlM6W8<( zL!(2;4RezHx}y80((yE!!mBTxM$j}n@5Emhq5AjE84-a zWvL73$FP<>~gvx7NzcIhwC#0OFuS=-1Bw&_)h{SZ(a|V1ZZXp z5eWx45ZvmRCU=dw__@3Bd|zq9haSC;BiZ9ZE-hrf~JZtfc7V zTTtPntjy)#D%A2t^9tLd`ZmE1PM8#Jy+=h&4ST_BB%TJFP4c!n)sh+7HS@vaYoQQx zsK5XI2qy6G2$zQ)W|@U$=htx_^K%s&siC9e7o=FO&3@?It$`?+1ht3u)hR1OmN!gp z>6Z`8vAkKH_a}ei ze(&P#sDcr=I5}u)S9;5Qs%}t<6E4FTd#~HWC^S5mp5jB0o_K*9vYpQerH?NNBmz+v z9nytOp~Es3L!hRrD~iMh7;ZID?c0GQIZZG8t0zXUmX_;>7WYgygWk4Cr`1DSk(
    sRLp7&$W3Ou@+B}hrb!8QY&Aa z3IGcE$vo9Ox#gBz1=wYKrBSB^`Eqwi_qtVQ*le=|xkDy zj3Y!mg}!ddSADL0SEqg3SS+o>K(e~>=u#{^DD4KyxQ<)ubYWU(bUoknLmAl5^oeu< zy1_o3D83}Df3*q6FoR=7hx#k>t>ba3EW`8kDn{hx4@>kUaB`JPq$*YW-}MUWLY(CH zZ3QtlZr2lky$U+9B?~DBF4PF!hjG{9Hi5@-^Kz;IeBf^%EO00 zRbrnMcW*p&ZyxOvup}R0JHic$U#SU5NO~H(G2$B5hQ?Iok5mp!YJx_=^p0ECM6@@pJA zH%+e>x}CL_P*U=+;oc{T;|-fzbZ?Ch*?p$)EfA@JbhFI+K-|JPHHz1L^c5)aP(Mh{+;s!uyPaaa==MZNihZzH46 zXPl?R+B-!(qfb?;c<}&tV8ek?ulkND-Nl1>U{Y4Be`%*Qa~vH4v>)xW(Ve6bg740g zm=|Ra$vLLk72NZwtz1w$HP;?EU%UanV@%Iwpab_rvApVmnI*0ejO4r zMYs1Q3bpIR*t+0KYYEV)52`9rA5Vlr`n4&$Z!wq(wCJi}flV@7Vr@XGpE9a;_dA|6rQYcTYw zV+{}mDN25;KF~Y7g0fOT*G^uC8XKW|qZGfHfaON#8ig?%3G;tlLl>%fY{CMc5>oOeq&ZRy|9VV%PAgd z?}PhSeHnvv&cH^Gg`|DMfq+Bb^iIlJ(ERZlwP?oV0)OPCPUGUfqwBFq)UzRkP3VJ6 zM5w>heA^t2IsfuL(wMdG4T?DVZ@w~9Xs=c$1XZ$&MfTV2ac@I^fQRUl^BUOWa+vBk z6kB&zjL;3*7!5xY_68HmOzORgbpi&_?&^+d)CBWHOB}GL zTA~8y_O1}mGMC)T-kYW)glj>lulFK(_Zi=()0f6}%Wh@nWp6`92CK7$v^3S(jq&>l+>Dss4)l*eZQBY5y zDUD-RPITL*g7w6{vzJ!gH--Ayy|3FcwHS8LD_{zs1*C~N^m+Cdz-+BtAL6X1SHX4X zO-wo*gyjzAZ8F#KIoF_%*-+~gdU_Q5C>zr#&>BnSDtUv?D6nUEM4mo0)>Ye9Wma|G zNBuU`J;n6>%WP`7PzHbOa=UD52;tw_s)xaid8rHsbXRkn!GthNl(iH^u&swP#NnXs98VR?ScPf2pj-`tknc`~4 zhdq&}f%Y+JT3ku(FH~(cBl5J1)_v9?gx1L{rm|{sgsG#P4oE`8_~>|vGJ)jC+%*T(#I_Q9 z3RuH_FV3TbJ@v|Jn$f=GByqH{HEYch-v|XJm^%SL1pn7 z=(Z?4)|dOU&mXy6x6+9qLT=Zv9a^Q{@(?SFGBAc$sN=-RDl8YxaVrL;`*Ae)%1OV8 zZ;lAV_d0G|gQ)Z+lkB^c9 z&fhFv?&6a#4Q@9<&%G8n{*}T%W`SAQNzBLFm^H)|wrfXhX;W0(&bQP@W(F z#JCa+_bu0&29eiBadf(&Z7xa2{wkH>q>q!{#=GXUAtb&#imeY&YmrciP{9~RY?Jgq zz^b7M5%#(9U^T&B@!pBmSDf|BRtow!@F2~Pp9hPWg1-tX1<&qqx0rJVlS}kVui=#) zUQ#6Kw11>NB1;}%1ob_bfv>E#H6hY+>Q$gwa~NdusLS`sAE9NgysuD=Gf^YmI7q zotX@Ktsc6WAl68FV#aob?3JRp9Xp&R) zitwcQnpM?|Fq&k=)L*2&nTv#45uM{vI?Lp$7vG2xiOw6-_w^@_ zHIU!h@sA^t)w3GxTI-EBoWQ&}qJ}%-jOnRrgAZJR@pn9n!WP`T>P^N;h>w z9C{nx*@uW-=})IE7v-#5jjth72>GKHpCogLVJ+3e;cWkEcFgU(1*+cD&$b&7bFVjk z0eILukOs8cS^ZKxueA1w>3{Y@)*s^NDPaXGyvuKiaVP@z`nW_ITufX30p{K{JymED z)d%$xRheZ|`$5*D5+QH9X^R=Wm(1n0lqSlGq;?R!a}d$ny;#vDR_8qoee+K2`3h7y ztnZRXvLQWKbpWzJ?}I(NUV?UjXOd#QfjEy*nx+X>nr1IZ8uy9vYFwDHd4c zon`*CH`F#2Wn06r1f6x63A-gkXod4;P7%Vok0nle0}<{BukD&3L6zIQf3n@LYgyA# zeSR6WEO@AS!CwDtoEo|J?bqq9K*rRaf2^(C25{Wa*PQKtUP>405)O2@c&N~%z#8+4 zoOGM`b3Mer+>oL2(1aj0dXJ9x6bSyJQ^nAU#Gay&hJX^ z_+o@DUy@IT$$ZwiEe%xqNl`a4UQ?N?xBdg||97#?2q23+_sh8|X!YYSBT zcs5;-k~fwHtHm7knky&*5{0c$LU~I^QY5XI$N&~CHUdM!e$9J|!NosUVIGH#IFsj? z6SUk4MNQm_Yf}B7oPvV0>I-6M53GmDKk(SJ=QR3p=4`xIsUH5|11ToB=?iPPt6ndA zP~Q@^QE@k<&};o3w_G5NT|xJ9cUr0Nb`sgC)QxDnRJuM{gvDX%sqY^QRkNU|D@iUlW_9hMK#OD7#B(=tm_HfiPDiU z-grd^&$rAeBMycN~#i@E`cS;9;*VSX@lo zkC#9H4wO{PGUNVRQdB1G>aFjD=}H&0hjj z1K`61w{mc>eR}bW-|VElz^|MDc(ps=I!OrYRtAdQFw;{{I}bMJap)@AHxr?UWEEeh zSs8N07`ziw+HwyL%hDnbO{Sdo+xES_&Nq1uP__fGSG~slybh75`$;Y{qGDWK4{!by zSiUM6!BP7!CsTw3K~OsIAPmtzlX` zO1TVt6+RXg#Yqu}-j@)zsc(T8@oRWq0Yf({?tc=Hazx+!THAiGL9Kqx~J+;)T5Su1B5Rt+6Fo)5#Xe+R5fkZqJNGGnrHNTr`Q( z2J>=*KtTi2A8KkLopFQz;Q}aHT_lYWhwJ)F+(-0sAObMG9 zoDDZ0Cl9r(<(8x;c6vm7M#8yN)OJ_(sEJ6|BT%+`^=fmL>4CT;*)NSKs^i72Fuoyo zOq9Gncc88WVrk#moi*b5BQLD9y4L+2R$`m>6gP}U1^mDMtHIt1?24^U84WP%V5@9eEq z`jAln8~VvYBCQgp91(CF74@zj6IZOAs#Vo>tnkWM`)FxK*(d_dKItRF-ZmA{%kbLn zqlJNR-qYlpoKx9+C*feXBBzZ!AnmTtM)|6iA&>Cpi+D#A;g+-iP+4FY91SSlm+gfC zF$g(&)f-`RqoKS!e8gua#)`So-OL{Z4@RK0SUYt;gc21J0I5!?XDbL zhC*BK`RNa1c3p2H1-1voHtu@leGWL+JWmriZ#VgJ?LVI^-g32 z7h{cewXNz5Sdhe82=*kz=xsFAf9=LyE3t&D6N8qNlCp=WDC(^?SnJ;J*!?U zP@xR|ZL(LWpiL8#1(DWmbenK3LaTm^i12WZ~<2-^bRz zE-_)(5%|5kewr5!C-9-C3@1Wg_{mx+$Ae`U^(T0}=0hSC+gDCai3QqxxRC23BYcc# zo?QPf>|Ff1S^V(=(T)tP@HC}6W!c=x7^hmoNFt01FsP0ovS#YR} zakomL)mWMvpQs1G`~X`z*%4!c$C~unev74&8}PKD6c_3G=VFoK{sIeVrf_k5e3Shbu)07^;cwETZJ3a*Woab7nTpk$BnOoyVGUFeadv>P#I)%Y=6oy(|DXQ!sn(8 z6y`R=r%XrJoWR%ObH<){yYeUsF4ugp;B_1uf;^VfNq@5Ex=ixssU|a?YE&mIok3bH zlUv@{pMIfn8zW6daMuSP^ffFLy{!^}vZ-#1!;Id)aMCPmGCN2$%i#6-L7(FFO5hD0 zQl_hc>$A14Wr@|m^r0I^HM51=sf4p&HGoFO<3wEsT+Fm0vR&r4GQ1No<-g`Xd+41s z%8ld_%A!EKe;)=^&x48N`zgE@iYn^Tqrbo5*Ngo{H~eeu-q1)Q6DN3p1{=^R8O(g- zZZIy!az0o~6iuh@b)BK<1-%x&`_j77K$OUVgDq?>)UeMF+@Ap5$!e<)a@DkKZD^HR4|rPqqxvkODs1bdrK$czmQagY?287eT}K-?JSH=& z()b)7h1~nges)?Rurs(JMrqw^j_pWhMA>%Rz9PEr*AKJdx9Gez1mJIaPrnyyscjTK zT{_n3A{r!$tYBK*r_`$>^d zG`DPr2~$SPp`f&v{E<^yne>X+`wE_saz>4d5u_$vRYro);!3_qvOs4ScRLdLW@9#f z?+YKh1?8(M5%CJ&Z&=pS8Gi=J80oHYr!e{DD!uZ&zqd$uy}kA~-ru~FYjpSRyS5fl zFlcCBk#2OzAYOMm*lK#NpKp4p+oCra{+j%#RgV!*0V3;`K{RY{-U{3iYRhd`IBAmEVS^-96ipg)b;bcn(!?Qy<)*qI)idRXuQ-`v#F&COo+|DQh(u4IX3GsVW5c4X2nu2Z0%Os2q^)fs*{fAa*D^kJ`NI=|7eA0jCvY{&{>J_8 zQs|;{BfhR*GIiVz zMfw;CiwR+dlhMpba7y|^T|FdDjvoh)!{!rR_oM}wQr(GS3^Vn7obWi zN%Rz)5ps_)i@>)o3jGrThn+@Ghd@C@l18a(i#0W1ve@SP($nGIS*IL z#x-NFCzszEO*f#T6$v?q^hPrqt1^o~XVFWyu2)hs;cqf^N@h1J!?CsQvVlZ*34VJ8 zH>o2pvHycyL=(d~^1oZ;cZAT|9ZPH;%WQ(T=NB0$;E+|u6UDk1`j@>LPdDb~%`6A> zPNr{2P)e{+hc$yLJMKUG=$^wT(rW}Dt*RJgRD3F-Z(PSXCKmo-O;6Dn1^B#Hq`c~X z5r%a(V4tJ|Qu@@6J^8kA!&XsHQXc+ski&NziM}=irzU9l+Zrj4*e(dIRCg8AYGiRO zx8k1RI!aYOji4R59~}9v!u!MhURr$(aT&jt8C~Mb>5CwUH2wBUzuvr(t>eunJ+%6J zDjyX}7?tV|y1)h1Bbrzk`k!PdhDIFp{CP zTx+p0tRdks`40r#jeZB`2_e@Yj@i-PpZIez#xhC#_3OA;%0Ia>H}2nW)?j{!=N9dN{I#9kxzdQ*A#g^uUDB&W>Z-zCg$5IhMKE@%JK4nz^)2WVs7iTKxQ56 zn7Yz5a*`2n$DMd_VHqFSHkOPSDcif|fY))WR8vk$Ef=&%?rY+c=~k2)9?Z>UmM4%X z32W+NME4Gln88RsZ`Fg>^I2!bU~O&XU{(6w*ZQw!;Dz~4aJlb-iK7$F3|?ZT?{*2V zny#OGCpW<$3V5yxs@-uQgHs*(g@4<{R)4}_Qtv3YmTk9rm2$Irbzmzy>I^HE*5-$e zcm`%)uWB(rzZck1wf7xhxN(d(6mk=qN^X85NfiQ0=-RU*)(V$@*px0e9n zZQS?j*4qJT%S1P;!}r(qDO0mQ6W0PjVz0PhZ^fyx{04=a7`A&}V(`Zs@G2Y`%k)<& z%_|J667WdQQUCQZ_);~h{E!x-d0qRtZ8CA5d@8FcH2!aU1aIQAGq1leSo-YKR^aeaeZ)jsW5ISF3If>v8V&|J=1X8+tvfLC+H%HxX zONTgYHwU5!X8s>tZxs+%qooT53-0bvxI4iictLRYpuq`H1lQmMDcm8!-Q9u&cXtTx z?uAVKr%#{TJu~-_w_@+Lmwxi%g$Vy8>-QJ4SYMA{xEJFm3)MfBAH7gOC*OOOI}md^ z$>XA2LZ{nI9Q`jm_#A`>XIwlq>x%oM2Kn9n@M9LqjYt>T`Hmb$z|g^ecb;bjJrG{< zGt};-`J{HFhvbWe#pk<|rGv#Or*J@U z7ATH+KZ#n>(u?yq{n4@Gn|&=zFbP{qG2EOb2O;|LGdc?Eh@z6XJlOmtU3}diYPlV3 zz~@-=)2DK0_iNR8X(~n+*lA+lh&SN8ZTWM{7zMaJJ|Rbp?nd5b5+}==CdnzHyUcA6 zcG=)~nx%7!qE+o$#|AVuI1C)7mB* zmyQ2x_RkJaq8EEHRFCil#{Y3MgIqo ztA6IzdoHKTR#{sc>BdMiPqj8z@mX&1D&04xR}RDW)8cITe1gBSsj7C#v#$cLXOtHQ z_>Ki5nWQm&i!3?zt#j-R1cvUCf8!ard}7B1l*JNqo_?L7UE)xi6Xg6pj`r9s+~#zP zP|Wa|i)g^|;oFUlI(8GN%mx+9j<=o4ufDFv*sn#pp9?UFpxX7^xN=p)l+AFzCkunX zV%UxsAZ}&rxPPLTnfD1@xjzVEAap)96XY+~XG0uY*z@~+HIdecgz(3>5ZhB93)9JR z>{XlfNq1Zt_rp;xINE6=J*jC)*8n_gl`)R?0Ur7_L)HEW6_erpe-5Zbi({sDGfbC0cqW30tWpkPaWzTf`;xEPOn~FBBs{pRl{k*7! zt(gJkzSk~)GsFZxj#WHAdW?tOnTO+>GR>uFy)WiB91Pndw%_<=1kvZzj!wYm07CnK zAVGoGd&1467L(7era)d3Ph{=4ZJAH6yAp=42Y_fwJ{t^WvTV{M8)@q<*Rx*YkDX~L zuV@2C9v=uQmx=073gM5F+e23Tmni*xE_@jTU)3_*J>K3YCr9?u`@WNdz{Fe^d2c0NI{sdlDGe4C=a}J zn?oE$Uzkh!cVP4}V#$B&Jn)J6^cKXvE>)8w(Q8j*JkIq6_1M}W@nXXIC57JWDL}K! z!|`5CzJskVRQdE0YA+wwe}BF~3AKhof?gfGu&e#nyjJ>IdEoUqU~LXg?u*%caK1$Q za`PsYskV#>&aYkHDP)SbI@u?Npa&92P~p;GWFWN*z$sGt5FWEb1fOQ}X0uEP6C+JO-ahUBC^-DwDzsWRJcJKILZY8e%gH!P7QfRAlRPQ#k>{bL(NEA4IjT(x%Y}^;7cD zxR)gzA9XZ9(oBY00#?KS+3=&!QBiBPK_L3#IDq7HJ@#Qvo1!2;|1;Dz`(Qf6KVr0j zAKq+edfqBy>7R`0fDriUM~+^oWlnqK3WrEb`cw_=^pK87sJ{a{E{=@^A8{Keygo|c zDDdXhL8AKyLSe9jN z)JlVEbOtllC`1HftPdha$w}cXKoa6?=I?|S7~9Of>%u6`G5%60(4Cu4`%{EqF2J?C zgq3Og4<{NUaq|tdY2L4SWbm{}$8J$JxL4^G+@j<-uaOgZ$JMP^hR4fG#WWMec6N{~ zoW@Bnb7`*-gQqANHaZaev-r@>@x&Eadoqcj`y1^2txiEjXD3m4T0xwSyUvt44t5U* z^6ch)=KH&j(;qOp^RxQ|pSx!k%;hnJr5MQGQVKi{#`Ukz$iHXgKyWnr&v5f9xIBYS zlp^KeRwNrQ$NZ_O7Ghr5?i{V}h~SqEIDr(;Wy|UXjgD*j4mg5P#IP;ySfeIkU4t*+ zx52Q#!aTdg-}nX9yBG0-TIVKx7ZUELH+b?pF&yGG#A9Z<%FSvmvPzT0q@=(9fIK(e zUhk{(R$WHenxS(%blqIf4sV>)#PD*Y(}fcK1a&FmblDk%UQk0NyrnjwfA-IoXw-b} zh83=>Xd%&Q)Q0wR#E*}E7?ZnDLhv2c%Hp>GT-sblU(XU!P`tig>1U$@c~X4_BOm&Nxn)e;Yw%$_GimwOg#nFd4%)q zs>BWskaGrQ%OBV&1v_@pHB2pV@s>ihe3%TLW4cT+=qL1|6OQX%Sg#Q1&tG^vaaROH zy6lBQks8506^rd|JPJ3#x0;}nbwW|sbbn6f)q3UOHG)jih0NVy?M6O3uugV}W7cpD z@L|ev4MOD|=37kmnBgapO%|%j?#orDp>uOG*AbJX7d?b~>cbWmme_L0M%MJ^uCo_364F@+j${PS#O$U<~0C&QIAQ zEsrGQyrlm%xz~%qtMg11$eBFWi_MJwl9tQG#Sphkw|5dIP340*fB74h>L9`vs6_9D zu~@ek#NBGXncJ6j;fOgb`T(hSAx|VAz?K}Ozt`*C&qRv^bI#&Rc!2> zd#Yh0`{KB%wr#GcPtzjG;Wpv+i6miQKj~JBEfn$MP{ZHF0D0HJXMmnF0#8Pqq_)CA zjg{2`iI)VX3%2CfFW3TvjOxSa<0NSXut|9B6NZd-PDk8Q0p_%j`vMe$-g!XftS5Re zQQI!r_;CE8<}&&iYpE3@DJkj6rk?ARibkh#6wYOaHk*Rp4*zushsGn7`GNoT-DRbA!s`w9$z;&{QcsAY@my1Bbn6y7nJR}%zT13n zxveSu{G0fy*sU*HqSN*@yaB4glxOaRZ=ny#Bhh(v}K-j)HuHbdaD{>!O zp;W#x%ZC7)mgQ|ei|zh-m=ePg#jTKr8jG=;r77CA3>@Q&XMS8q&Q1L_LVG$qpOJuo z8~zƳ~!o@h|1gB>_|Mp2sz`SRmyc%Pb9fxUHtf0AL%-=Ta3r4(~;+*Cn|kC)V4 zpXo}NhT6lfiTiMJV|rTfYA`Y`lm;dqthGr@+XW>GKtKO~Aq@T-Xkd$V;3nYMX_=YM zv)iOm{i}b$yn)0HcRDW7udC-r*o5pvaCwJTD;JV|s&|v{8(P@blwFL|-|GnybLWwL zE^||1fTH;`y`9Lw{$Wg5?_ZyT=>t1D0 zY-AKl2=fYvziDT#^v(Yl7DyTMQaq; zhX5>&mqX)LtW^h!in)YW&eo$@+%76%tc^>jk^K%Gu0ypi7}$ zFKD@yr_CDV^IWosP@Wi1!dJFoTx%qLq&B(Kz7WEHQU&rJ$WAd?51lr1n^$R6v5IKfn6azW+z9S9kf5vxxffHFDGZ zF93uuBnZJ^b|8v2Q_B#43H`Jzmy=~)d8NL~7v>?l=~^MGGDQ{94@|W?miE5>%)<&O zK`!Z)%5Ca1P%u>hlyV`)YdRQ5-F$CMy+8WDF}?r(H0y<+FdSPqCA0Sp+VJx4E_+Yy zEiEkvO>T5F%O5{sESv@ZsBA;l>En}e)B=az>Td;k6vDsJMY6ErwE*M?EW~&eS5jidF!J~F~o_4awq;h04HOIZnF8%8X#(%9gVL=nThzPdztsF(Y^fcv;Q&dzH7xF917v$W^m_d6YKa6;r?>4h)^ zsNyWC9Y&xOn7_fe$`U{z(E8yYbvxhtwGU80#ER@ci`UaTp5tTL%@wCEt)GW8MS1S$ z>uz5-p*R_I09J%_wexnrZ8GHafaCRC;{kN$JvV*Q^Y2XYLS4N{nPXD`ZCdj>p}2#p z=n7e-AMb$VLOAkhgid;4Jt{@OPx!T781Xz&(}}e49CX2q$F-a}J~x29a@zh9*Nm0t zNB|wUr=E)P8fB!9EHklaL?O#U@294Iv(mU#E2d>eGL84f_McF^`Ucm8ap7|jA1FEs zhT!*<9Az}L-2Lm{kovvB;xOS-w}@p;20N0%vGC)5&D~@hPnR9aey6H7?G3d}uGFoO ztjfvY%hPZF4_V6zz7ujLT#*a?>1|mR-;~1NMdU};B1MY%zGGyOmc3sCXM*WyF@XR* zY&^2c(}=v{b0kKbkB)0`2F|pznpBeM;roCH+^z5g=StNrPvGdbj?m-H><^G0auQqE zS0|~RGM#F-V`d9VANF|Ronmheg{+P4*S9;?M(Yozdha2UBtRKM{P zbNpWgDLi+u(MWiUl@{$~Nnp`sK98ny5pjN&Xs|zlQ2O3rSGNCUM_abDZs3JB{r^I6 zSo{TSS*z78R+mDbC!;br|}}|mInIC zqY1``14t}_D)py{2n)rKFSoA{b(oLkS=f|!R|%SL%FzUcnG`jVm-x=9cgFm2b|n|< z`++a`I);HzGFwg0ksovk{M~u)e}Z`alUk3HPBD1&Bmb zls2uvP?C9+!ks9su*FPgQkkFv1j@<|pwZGX&|KXfFJ7JDCEBm#uUzUPCfAik6hK8= z$e}2EI;5lKq-cZXXX?LV>puSrRQn&iSX2Nuu%gLfwASdyyNDDQLBlqW?Dn4(pbYyj z$FZ}cF)@kQfPo55z?#%F!-M7!vHX$8sWpw<)7JuUA9xbGXQ+Z7hdfGomm z+)rnW0D9=FG7;CYfjAN(^(7>IlT~lC)-DVFZ}jND8xK`<7-4(ifL@h==Ou1@1o7No zv?}y3*+j*mE9@u@QmBjGegJM8$X! z5+G1T0T@&%{mWP{S{k5mTP#-TVFq*7D;AfX6nZ63JRJ35qy_llSFc}}Tv4|;;V&jV zake!T-E(AIdaAp*ZbBmvw%>KehP>$ir9jd_3nYquRQ|`qy(Gq`BRA>!S2!)OLq!vL zLOgihm8kapD_)xSv*yO;-1LsbF+!0ItGPd;;T`+1j&H!u4aVR}W7}6gLvq@_SVt|? z!3^6!z|87T3QvqfGT0_vF@=!+YkuAb^u7;n{RUcJfiu!d9OOAhZIanA2MZ@l=zrD?=$X zR>A4u*WX{K?iE#^W7AxkzcJT(Avb=!cHYHeBIhr25ybfhgCiV|Q%>N*5<@m66`oqE zBtI?BU8|)RlYm;8WulyJFzx<4r;1im{z%~9?C~vFVyocOW69*u z`&Ki{U<{;M@T}Q;JzN?@#uNt#jOC+U_2U@n)Z_2ENz>mjf9L5kej|TE2c2Mwq_&oS zMTHjJI5c!_?~?+Hc>V<5M*gmcY|i-5Bz3aDZGG9Qp!lRGFcL%FC}x5AApH=N(31G# zevo}418!fE{7=UeqGS$VElcV7B*`mA-Jki7e@IHfXorVME(iRoHHUW4dSwd|d+-tD z^$92Fzu7xt@$~c*xa`C2b_iM*z4#4B#NC&4x;^^ge_^)&yRr%+J?!#33~bTYY69W( zmg;t2#F-)z&dUV-F>=VTEvIT!_qL%AV#MsmJO!uj9%`xyM#Gz??_F$Mqh69KcvZ_EVi0qzH_87 zD(uNANoFep_i$hl3#85xy{v|4!;IEB1yf;FVPq;xAH(g|Mzai_b+8C?l{9$J+Glb- zJ;yiXa#SVfiwn@OnmZD(jLBQ3Pd%jEbeF119Fr9BrNnROvBT;`-d^ZtILp02hBSHqkHzaIglKYw7DV0 zn4F~~>_hbx3?6MB9%J7ARqHvGqG}5tc2VY^h`D6b`2GUMbFS35-kPmquB@Z`>+#WGw5V`XE^{RwXd9P(b6^u zDEs1}?Ht*25UA>y7T?rDz0J%8bgdGy3^nfNSoQD10_`pH!J2Mh+AiUhtv&1RL2T@9 zJF+f`>W|+y4gc@4gO(N+xv3!<^D#60tD?anH^nBWq39!R&e~5#amF`Vqot^7ui}P*@RT zfg+dohnjJLi zl*@N~HD)rp^yY%--D7`r#NCPc1j#ka@5#3uMF^LeF)7%!!M3%r_^Gs6r;)O0L_ zHfwi@O*U4`%`5c>>HPnzO9<2smypB;X8wrdWMpHL`5W2NL!HDFk)If?+$x$z!{t&D z_8CGfP8!<|<6C%nLX%Zw=SYX0sQVR+=WOwPqSwbb5Jw8;lJ-eU`$}tuc+dTb@52(p z(SElA%z1%GU|)eHDmbwV23i~dZn1BwfQTw@mA?@qd$!WQRa9GWN~~Yh@qAyKH<`oi z?#c2KK2m&DYJZF!Hyy)Tp`vo~DCmpLIN{bpmaUXFg918^)8t94N@>7kz`pmp3~}f_ z{i-pKOHcI)F)vYw4rRvC382eIm4;%akB=W)OB2O%`gFR~qJ0+KX4E59_19tkM7?2_ zbliQEZ3?vg;Gk2hu4w7-U=fF&zhIv2e%U|IbiTZn(UuVR>gNLU`Z zE`Of~o&f+&?||D?Qat-8;;dlHwKDLl#h$rcwE z7Gv$-fztPGIn|{z@kXMPtAfPZ9%fa#ckmvyM0iN?AIsF#>JgIC?nVTh*TCkaM@RL) zDqNcH;Cj{)Y08?QjnBRcSlZ0nPPh!`5V=)JiBrccRIK}+O`e+2s+}Q)%CrD>NxSw% z6-ZC#!`rhvr7Zd4NEZ)p*ur&6+Rqg>C>%5d)&mf@DHzHC+4p9eFe7co>BmY~)~D4K zRJFk&TQYB9^*Kr|UPf|1j_f`kJYP>B>d<+>js;{q=c({@)%mm4dh_e6VCF4vAPDz8A3>ge@=!V;AxuJcJC)uU zm>C`Jnvd)HH6DBCOT3|nXeFA$jG!KyYGoo76imuqsw7oEMM|((X4Xw;8P8Lzs+JkH zrNjbeGFSuEN_p$yT&HWmiwO30su~-IKDS`zmU^I0x7{Q;1T6t*teG9ve)SunCMy6U zCiv&>ausmI6Y+~4VXqh`x2sb?Hn0o~hsp>%2XdGLR&ir}n72{vr?|7YgKAyJMjvCL z1V3jPVP47+wfHD6U3Pq-qOx-9Jfb+(vc&lKVKw%A+rNc#@@HHUI@Qkj2Ifzw6VmBo zzO5W6zL2{=P#z)ybijI{**vcZgr1nbuZK5Tbec%KRva005{HIh?_lMY-ym{(&gr zyN^WybZMm8>P8MI+c(u%cRvt%VqUsS#$8v5ZmF`4`2V~fKI%9uY4sD@kR6z&#-D~5 z3&}dgkcifCE>Pt7P+Nw_wE@FLru%USyhS{YTcVpzX8r>(DC0($Je~dHSd~WZ;`1*P z1PgB31niuRiNv0(KkPycbHo50N6BpqAUo|yWjQE`g=4)ij8P+p0YP$&8Qg5rH>slS zxhbS0qUrdAQbOKFMbm`&kw6|e2=$6mf|56dVQXma08+KG#MH=$HI<@kd>P5jES~u+ z-=*0QcB3BGP^bxi)~n`@+#^>hvi%*VUqCkmcCW0>z9*n1Y3Q0f3lMQ(DWcV`EEy@9 z@scyt8oFsl+m)9YTJ{I>JeGXIDV7ssa_Q99o)E=edJ-MJS}?0?$LEFY zr%=<(RD7}>?I_@1OrehZjbv{CLGRRUCMxSnsbfP319=knPf}6_zr4Sd|EyVlIY_=X z#8>h+{c|Ok;mSxzM#8i^vJkcvF;}gl{_r<^Ko|<6LcAPYrxDzr#K|iyM%8U3CN0tf zH?PbrRt@j>4wi2PY$V+PTBYbj0k)n)x3VIlZ$ZFXBl*4K;?Xx{o#V^6yHKHQF1M`+ z1^@>K$Eeq7!@`#M>$W(Q$4pBPE7ZxKdsN_gtol6hg-(ucIvzF1_$XZIo6|Yy(+`aD z!REcE9pzV`H7Lq+=4{r6_rDyYLK?vUng)72(*S`L< zJ@n>wNC2fXkier{3@5=kx17ws?>C))-atsk-`v#*H66zJe3j`Y;OSxD%>*=P>Y4+^ z(Q{kS7`JTW8NO~pR*r=aD14r7q(Q4=x?T5cHt4g0F zNObH5ym;B2GKT&je`77_(E>UOZI*7*Z~6>k(r^DrR{YQs9e9f5pICh5wP z*X8mtOISW}4>h5LC-q_ybB{8eXYg!~%&}Bb*svuKHbs=4a+m1Pu;sb*JR*P53vwfU z8$m5@c&plMOz)8+PCw$L9=BwtlJ#t(h{)pWlVR<86~q#^lg9|LtGod(<_;N#OM~-| zBPsh;Wp|n6fE&uQ3=4tf4B+P@hCOEoW^TjBe4%`w+Y*Msk!L9LyHpf7>}5^jfMy?m zY%Ts1V~Sy4a!M#SSI%bT7{PdiexOltb7v(TK zGJ6$^VlUTaqFc;UP@HTlJo@xF)l~Fa+z0VHLL6p%aMfOJKBvk#$MPD{k07%#9(bJh zfm&5-!6)5evcP=ofz2LLr-9QC#^L6&@SN=+Qj9ek)O){be$%ftk&R051Y+N06Q%Vz z$!S_;{Oc7r))t0DV;{>Jx0>7_D=6b`EB`PdnJfwzz!>S_z=gHIYvqh^>e1D*gcEU2 z!l>WB^C}0*X2$1=Ne426RTi4Yyg3_{*~pcP#r_~ZoYD{B6s-)yduDxuuY2dac+Uf? zZ1RjIxkGg0iaOjf)B?GyLym|M8r#Hg9tu~Pe^%H>ungXuO~D1 zzKuU&(AN(u#wP&q!|Hkt@#Fcq1rdQ=vxwoo~o(=yMHLK|^MA z@%j@Q#}igBN<2^)pJZxJ_xJX>P5a;h6X zZo{MiL*7UMue_c@)XW@Ev-^tpXAiIP_i}{uXhw|0l9i6zd4uNbabcy(DN+wkV`?j8_q61 z0(YQiC8!@^92d4yS&YBN*Q{rbA1sf2CGLl%C6f`y{sm9~itBB_7i#Rl;_wsJl?Q-5 zz_9`w93Z;b&5DH^H`L6*@QG{*xf7!|Flgv}IjA+CbIl!hXWMNO#E!{>=T~b!n{~saH25kXo9@H+)<)Pl9nS9qtfxOGD$N?Or0J|=^uy1 zY8YJlPhTw&xuIY$iMQ7myCGBP0oM(FabR)(#y-CxD!(iRg%H;ianIlRFFC$p+w;kF zdf~m?k3_>Z-uu)14h;7{PFAklbu54IaewrdQZpLWX-`bB*kTS?;3w&(?f2Itf^8{%MFTqGn(BqD#D zgt!bv#QLPlm_Opv+PzXCcATC}^6IoUEk${+ytn}%eLc891k`M)xqg4-z{EW1c;4)H zBS6#1c-pw2YP)bM8Tz)EJZdKCD|#yLdaxmct7t@V11uiHW2Qi*vi3~bC^!>zMW;MO zZB?W?iLEI)Y|->t%rT>kzNnQpDG20JK{+2~$`*Gw+45TWB2}xY-2%5fQv1}hZ7_=Sl&?(OZ8>EE*4D;miH@=y z3Nc^5#~Q%qAUg+bg5cv)7I-5tEpI4t6^a58#Qzx^vqi z(mWy$BWYUW-O?7dVyHo`B+rGh4Fp_*d@7W!j^ax_GGg zl>qex{`x=4{wpUez3~%QnzW z?(fLIa&|$g$%^QxLKb0Qi@tCpJMU$C642MQ8rjOix72DVVI=7?3Y@B;K#mkqnvdpuJWM=ysLG8!|W$P#* zXX9KGMi7YBbFOzAH&YpoO9_exsSQOT#5oOzo$?kQg$Qro!My!7`m>+XHH{+`(csfb zy3V$tXRCw$IIuLnDcF|0wxt zD5NUAVpqBK5OiZtjjCW!OvX}(Pp#UFL-~GXdAiR2P|aMwaKF3Ayf#~c6Y+!itFgHz ztwlEUE!o4~3-OOZbo3FvT+6s;kX4bc8ibNqv11 zGZUpVNYha)B>jUE#e+8gxzM+haHxP34NV?fXY>3*&vUYDzPzDYT4f*#_Dv@!+4U zj!b5JJyFp=o7-~CdQ{GELRp?;yT1KzEdU(Y9&$HTZz$RRt#Apjlw7*O+xTr@Dy4<8 zFGV@)2j=PKsaJRUMSX$tFmjUEbz6jVQnvlrp*RD3jGaaggI-UjW_X!;ZD0cW(4Y!u z)+DboUyzQX9nLOY4h6hopajp$#g5Kik!{2G&bCS9_@g}Q-;jt(=BiZURexrF!mxMF z9I<%64iX@bw%itXs~VUK+Rfij&ESnA09_SITYRD-*Mk)SpA9r9#Vc#1*_*9g;|`|y z6FC$(m5ivWl*%k?Ipk}@At0TgoojwoL`yym`$rKInbDbN<-p+x0NVyr{^M3q67b_9 z@xFbzL~?#1jkxazX_qi+4ZMCFV0uAl)$_AY=GPu%ufk{*zl(0f8YnLVqIiu<1mo@` z4TChWU_JYdI;SYTN~&U&p*!{XtoSo)vUhLvhzrey6&CRrOPB$~JMq!uz6BB>({Biz zT+_i}*)GSCOg|ZG|Yn$8BljpsHc~Je*+lwlybib3scEdm025SX) z+0PT}dP*geAY#~PIjSoJt2KVP>l~;o!Rf`tdQW?Fc;Y{L0FD&~@hA%%13HSyG%6T7 zXH;@LH)NbxZTH8OSM1_Z3Q4EV8ERM;B?twcgB+{O9I4LY(-u1*xq)RXUvRYU<*s`9 zYCS%=i_kB5l%vL#Y$9HdiREkfgV}*s3A~VuuR0sl0v1117hU z%c8+S?Q-K99J>Xl#C>HwyQCSo?|0AGPoR+-=8V2rc~U$8p^+|vja--lBMZD-W?=-j!TO=(al9z;UHx1_xb~JIuWh+XLED#AZ5IpKKqyI z!YCulBC{LVb};sUugO!8-=_{qQKwvn*j-%V1uC@w*XDBz>o4carL4(8%1n7=4a%|i z5&r)ZC*U1`<)cwuiBqowwf#s!e#ZclX?+nbo(cGRM=gzI-Z3L!e31Xq2OrY{g!7l4 zkQpHKU^1t!lRNR4nCqH=rp*5gudsp9M?2Qbu@9QYF@7KT+c5d>wfCd+eMDGJwG03o z&qoAZD}?msQ+|Dn5j=-n;nxh3w;Hz6j-O)*9>M7k!v^_8yOu`1@qKC+Mk!cE_6$a+ zz171S!7k6?2?F9%yWOkGG`>v;$_R}4lyimMaf~Rp9-i1q0!0b=U>@D$fn05kn-cU{ z7Ud@=8JWo0;@-C}=fdyGnJe!n=9_#|0OU5v%sAn6jPn8E)YMa{?;@X6RLws$7Wd)g z^#`QqF6Np2BEkdWq09P{)I^zm#KXr@m6nh=I;-`)_wPI(UA-r8H%*~G8LvK97Pj1L za}RYN&q!n09a^fbol~1;rjg>l`K|@@SRo-(xDDYM7myNzoYWP2?F@oNSeTfRA_636 zuzu_Q31e~NmA!}|7BD@Dm}giSYvX^f7P)DjGE?q7%#`5D&*7|J`YyxVE$NJ+@U}f?p}x_2@)=dCQ6VgI6y+;GX$Xe^oV%HDf2DijIF84xy9FjB%hA~ ze!d-2o=R#HOy>J}e+<+ipG`ES)IW|dO6J{?xhh=5@2~{$JSn{1ZAGq{WtVe2$G$2O z&t#CH!Az;yC{Ic=4;`j9$6l6Ef)cS+#0fj z+t+6)*+(G#tNt~Y?!gh;krjw$%f8dXu&L4N@Xv})MCX%jwlw-8a$6OOVRk>Wh|E?K zpOEQvL5-jVHv4h}l2gC809AY~OQjtIi{Y%w7Jzd9i=md%U8*7kP0RB5`-e zy?LTxxr%^!@ZhPCTuf6!!Rup};oB}6y6K4Vix|~P=3v3s;!rxs{9Tu=9dYDeiy*h$ zei(H38xp-Y+XHyKABdp-Am(K}g^79vSsY3dnstmC*y&W|yzZnfdkyi_dNw2lr^d+R z?@6I7xoYK~Gn0Kpv`V#osF>0`kK}C!Twvu`@qY*7OEi;$S{aK)T*I4L(BBS{0zZ!} zD^N~KsoW6o9`OlH1`q%qMLhSDINWO;@WIm`k#}my%C%Ux^ZtU=DHw}!9o&3=eB}}= z4W2qWxVTB>L1`aqEnw3z0JEAjis-9zyBta^gX~B{99&wU*}ElVjPU+oN(w4ILy?2- z3=J#l&$t<4Rj)0f)*mqGXP2JKcnv@H1?B=jh?`w7i8r4({77G5u4Q4e^Lg*uG4p3m z{v;yW>4)MErY=@w)=Dkrk%#ADrs=0Q=0!6*+~vPnE5?{w!HN=8RMkpItiDc-m-m9w z%MNE2XAaXisc^8-A9K~k{7!-90*=KugXVJWml<6)3I+y!V@;UeTKwxlD>t2mg)OetAW_vwA#gS^o78w`!s%fV2J_&aZyF>Z z(``q=(|(<%cA?dw4Uyd6rxa~PT%*R26_+6m4jD?V5JM>_Q_R;eULDPO+p!jWg|)*Z zo%0XQtTka^GTJ-8uoi^WfU6$fx=tcKVUjNgouAj*T5IZaPMcrHjl9Se*}tXC0lA;(_H4;`zi| z$n}#Rg$+d@vwYcvI><5x-xHpQL)&~l^fcgm_peNSX_f(GSFlUpZ2|`OZ2HFzMcY0y zg><(7a46=u4D;iaC{xY{rFeC7ujX$$GHT8{>eRUfZ=2Gv_07n?F~>>iJMsoCU~D$W z-$7?kj+@j#6yq_C_skzpECJD8Y!TyJ-1O~1{Z6NE9VbJ42y=I^m zurkX&bGlm8KJrj!XbMfh;0Z>86C4Wyc-Y_4K#K%wfgh-<*RXWyTni_D)|(F8EZNEpd{Hqu&XkuM zZ2m%-*zMi)jjdhP&%GBXjq8@fWj=|A?k$y=`_5zIK|QTinjCm!Pb_f5-1407>nn4r ziwWB$pCvzIbXaYd9d|8<5uc3w{`R13eGCCTsu5OTFT$Gx+tp7wfjAWKeS+;G#? zkSR~N@m840YrO&gsb6vFiSFu7ktbj2{=v2!;IQyJiea#zOUPKrye$ni3|&w!^!XY@ z93eHzu*N`XO5IP4lOC$VY|#@r?#fn>g;W&6V;y`_wP-* z^Y`Z)!lThr!YF=FT$Tz$e5IJzQk7V%l$hV1HnHCw`q7TC%dfLp{@d_-=bJgoA@M44-F~Gxze9ya@l!#A zO5*9P_GaE2JwpR}5G`MFc4Q-jXX!U2?qal#+~Iz%hMwEx{8sqfpIZD-ePe|HxOT$v z)rj{PirA)=M3!rJ6#TX zGF(O@YGa!hOOSv=YVEG1o^p&pqbRdY*7+P_@=}9S0vf|vdbsOAQLnXVJpK8;J09G6!o+W;i32t)JgKKo#5<9g$O<$R!ccaRXo20F3GEZ9HqAaL*vY=b>MPKnH z{c@b5Q6)>S`3;Fa1L*F2I$%pv@RR?D%WK4Y4VjCU*~n6puyLQ$;wg)wR0k&H+X{K{$BX~L7r zqb+Lrkn?5G{6mhd`rbO#cA<}EQC}-+)QboT_ezo1sfPywexnbwc7xWQ+aYfWKz~8C^`-3lTvop-#kn*M z7ESi}u2{`|e95rLGxFsJ>O`_lPA+`SuO1-5J{=yM3_&~;X zY}!tx8n8r|t_ZrDgb(cgSl0VuTnRr3+(v}`eKstSR=i;mNQA%nQMC36GtG)30EP@G zVhCX~>MW^R9k*VSiJ!@ATXKy1mG3e2GY^B2+n{<5?z5u?(l$mv6I3>_zT8a z(rcdeI@~Ja{l{Sa>2zrvvk-8yGlUc<;%`VVW1UqYTkBGfexh=@=Ov;n1~{wudqAh9NRvp>Nh+_s!Gxp+jKQu7QVA=wMV z7T9&Uj3-o1mwZkrppM2eQELpHEsNSx4pLe&e?&ko^X9Lp5H@MNU~2J*pF{6>G$MZe ze&>eUI&Iwh%DgR`ZYp{@ZLstGHiIsfd4%9qr*b+;4XMlXYbM!z&Cf5X1y>gi+vVFN z$IM~3Re3(bc)q>J8j4nd&pSs0yCVB_+20PckjIntgVPV$nz~*Y4)sQczVLdAVt_jo za#lL;ss3^)%B%m4-#<@^sPz00^D<_oorphJLm2Cxh&ia?lum<-5baAqs@S!iJv6ov zCgvuVP%>@)W{wut;}Rwpek7H@l2UMl)FRonKS(?EGFQ{qTGt5BK#WFV zpN_pb1s{#B=>aplWl0oWc7qOJVmx3CsFBiQ4hS-WWP0%gw(srX!x?npAXhCKzwK@0 z7ap(xN1Rw}1lhtVsWrkB>haM(!_L?WMr4S4ioCn<;^pd)xnI;+H5c@>7c?W^Fko z-vJ?|h_%8@Z$n)HfJvj%R#ex;<(gJpJZELP8h~5|hKlDMg8-th58FX?U?1y^&W?ZR zLnmDvsLm3(V()5i(*F->7R6l35)vXR`L87Zcs@3 zOkiT3!JcyHqk_a|QtXT5=XKu#H}aU^P@qcE`OTr+^q~#~E9Ub#%?m}7Z_n0;oUcLS zNv{6uaSp;I+z&xzx2r_KLs#86_981Tx(|^Ht_5h~F4yCwY>JBc@<})3pwgf&w}@ri zqLqU&VR$?|wBqBW$H;~BHN<2JKox7OnFmzS904PNOt z$A*-_6B!o61yTqrf3mTK+~1l4h$>{g-<3_+cffL(RzDTL3Zn!bK8Oa`9sYw{AH^0h z`HnO1Hz{lW6p9eRsN^Ke4+O(#7%MbMkqKLrE6e|ft9Rh8G}@LxW3!WVY}>YNJL%ZA z(XrXFZ5tiiwrx9a_qq3;H{SaN89VD+YtC6!vuX;9FiX)gTG_mwQh=Te-dn=sbI9(^ z9=xK&K`6rn?4q{FW}ke%(DIcAj|kUlcJ@mBz*v!_jdl{#6;Ht6B8&SBuI~&FV&KJp zN?gipib8u3Y_^m+G=bBx*XkMDZ3;)Uvjrel%9;nG0)q<0x}FI%jNHMM;j0m?oM9iS z+ZwxdzJV5e_vuLPozHE}CsQv=IBe(y_B?Ut!xjIcwaM^OUPssWCM?Nv#cJ9OT$DoO z|ERsF8^%-A1UQ%eKn^>q9q1D)=3I8rF*G<0y(d1B``Sf>f?N$8;NR;ikn6bG;P}Dt z96mQGINWRHOR~NvMy1`O!{_c%&SSrB zj_ETL?syh1E>4|wrr0GMdGC2DxDg-H4unJ1@7wGFxe53lh|Z1wO4qb5O~ud0T#yRK zTxd_1q~d%QAkU@@mqwQRb8P@U}2(Mp6gESqZx+L0qE!Wi8U3 z@${c8TuY_lO@U<*0U_$0n8dZ<=Gwibibh;`2Id+uHRVqb5`nk9o)`oyECQJ$c0&-2 zJD2=h{*116_5Q<^sLhL5eN5%1tZ%&t8b;}($zs`0$u$V-v7o*?db_fDMq`m5JY_qC`ztD(Gja|Z zY&Em-s&!{nVPIfL$jL+TGvkAkC0+E;CZW)Jbp6ef8LEun!Jzy)lR|37QEZ7#Il1dPpx5X2E(z8-_)%JLbDI zFDPTMe=l8d?_Ntz%M#CXV|I<>-4b`=T@U^8L!H7g=U7Iu&=vwk9Le#gomw&m47!l{WOj!5RzE#(3{!GuZ%xMuQ))8 zfw0GjCsA=Rco+?y%7Xa18)__biby!Tt<*zi@pmLMih*+qJqZCz*!{Q6vAJNEfjnW2 zPa1X?5uTQJ^_scVjSuIrh|6-{)u@M02{LX!f2a}K*pmKO0VH%;dhAED#~E`x%m^Q* zTUFn#B0+*-1p^%~Lnsb~b6h=W9{2XPDi6owqi09lJ9+;a?CrgoF>oRbX=bW}+o#5= zA($0Vlj}}lx1B#ccH^pSgz6^3F3eu4w>@}^QVU#hZZ8zG4r2_l`iF`{L#d2CoXfjv zuf2^VBIXe9nitvxv((zTfoWYYs+*LfnB2^2UyIyTMYyCUKK7pf{l4~NvKcF2vJLF6 zlIc%kEIBa?#xw8{C!WG{6f&Zvjla5skHQkKK00Kx=6~<)NsG2Qr!CQOiihLO?{T(S z*1M~ZJg8&l8h1Gxd2JT1NCEgdMq7`{wLvbCwKR*peZDwjZgH2y!^;x=hGc90S{vWnV11^abk;DEmW(9||>fm~7A?plyU`A;QHZ zMd%D^IMlWJyU(6e+vZ-@1a`}?FnYX8|xHc z^m}BV?QIs=5uH<#9%%N!jpFV1%O7+w3x5{74GBQZd6H&v+GT=5RfHKt?Ff z&*h84rmO zp3jW4JMB2>YB3k8`r=VXsIeQH%(9y}Us!)^S4z^1#vyET64mAYwdgGOJxD%cICr}> z_AuFkYx7>PKbrvSCP#dQs`+UtA<5Xl1oG(SmL* zs@*d>w_@fn{=khAYxiV!PwFKZDOU5g-Z@aLhK(a>N!kl-#O9Al%tF4m6tv}(G;GIv zuKN?>w`x<;!*w=kThLY9PyP;Jn3y(AErc#<7>I2OaN(=?pFwtUnOpp-kY)!Xb;i?u!N`9!8=&>PyOmMuP+54`kkGcEaU*cJ?=*rO zO-gZu>^RqXonCX#s0zym4Df#-FPzxhtklPuI&b~mH!5)9*%_SNM{95NmnPoPnFGRo zV}z;Mc2MBQ5kdvAY)@QJmE}A@ulc2A3^^VikO*}(bo{BT6ijA+l!1>F@=51fg zrbfKz*7{VgT5)iYC|?{jHQw|6^=>lF`~2|lop9v=aWB1<*)On&f?)^2T>lfr4*JNJ z8aQoU^^+Bf=3#FGCwCAhY6ui-s)-^*&LsTEEF#?W$C5A=$T0pv&=70Jq*(wpiAn3f zTA>{&aJnHm_qf71sncT;M{d3*t6ZFiodtCi z``f;z9G^$e_VY5f=#b8I1l{lL=TZh^A#iC5>DTd^n2GweHwsyxy#(t|e zFKAH!%VQ$$$P&+`4a0RS98?QFTkUbiVjx*is({cJgZ&OEn15zBt(S^W){8trpAwAd z=B^xlhNPN8=)E&h`dqWxyl<$Ps#P4dp;3h~jynlddd zP3L#giA0|i2~PH9jspwq<=0}3FX?#d@b@_Xb?5y7z#!_m3IPDA4-*?8K9CVZ?33cr zghJOpweGDDn-KZ1oDl&V?0=lS*ch~XXuR^aOrkfv#tZwfkT*bUS)wiM4w6N^NVox- zR}heB5rf4F$38xC&SKcZ#CI{<@#-`&D#QcefFZ)T^rPi*>$o}GMtZW2VFgY@lC?!` z$2qK$!sFceeD>PU!(Dc)nKqZZgXph8>f^DzZJU*+Qddukrr$kd!r$6+K)g~)ZAw2o z7WvR~d^#7po+wr4njOV8`?2ac%MY1e-Ow=vzcUVQ@Gvuot6OZ%HbGf(#|`;gTqn8& zhh$$%x7Ze|&0m-oSIp|@-m=2QFMXG?-YuuJ)f6?FH`UOVsw|%AZ2+f)fyW|d&e)6L z?*SCq3uiT8Dnk&E8B$6hJ%@-z{|bqN#~;*TU6kbam4G_!DFL(HsvWWAqY{BWku>(} z`bj>%vAVx-XJq?CjWD_BZ5AS|&|N1Jk7bfHkF}buZYGVVdy>Su54}>TZ+Fd}$VTTa zAj$WEo)>Wxzq$$+GR1z3f({Ok+SCDfs~IEu&0Vg3mhhSwotlu>E{5Z@iVl@hoLQ&XbkyBCGm0O9buwCQS?OQD z8_9Z_X9G#9lc6eWUv6w1DPL*qOo@z;o{vNNJAmt^$l&(-*7LzphF#1ZAu`g+50FfG zl&}w{EQQ9y!VEZKy(r`lCnL;#`t)lFD+B`hX)fAWbA~6D7oPg0-x}YH99YKD$~AI? z@4=1j6UEJp7Lu@uSU*R#JVe0QA_L1>`sWn5a!m9Qi1RK=#Ln>8jVcb&FU9 z0vvEal_0^A1^;t%le+Pd_$~es7I}q|8l7H#sLL|C+Gzf>0=syGl;wQVyR}cKMGG#8 zx~{cccA8O4Ja$RvHRoB_@@OP`7HXEb^VsLHR0~8k(%Aa72k)no9G7OX5BUy&hHP#He9SugA@2PrF zuM9Y|+5|%&IB-xeCL9ZvmgJHDI%!FeO;N%btJ2*Shvxmop%RN%eiE&RDwX=V_CQJ( z#i0{^96Y@q)w!OZOzf=#cX5NK^p#?&gcg-C@t^kq%iIW3^O)1eBUt#K>apb`8n?fq znk4lDBt3SeD1DCgF3qBin-}|=5+5t^4^AHbb~&wJN~AkYks3g5EO_i2u#AK=OKqeH zYgtkUlu!7p1c+VxKRtv!f@-CwwEqZKErbkp(9CNBc79N8<5DtoK^C!^V2*8^deGLd z{Bao*85}g8GYy8vt-~$bEq6^P>ao4g#>Nrz;mT%hvF?nPAZ6r!oaF{)XL1i*kqQXX zpuxo4#s0@(a3ce=S2K_?h@YF+5*Etd$l+vQHvu(bpeq8=ayABoaKI@B?tJo1BqAQl z`Nw1QV2^z83yr50xrR9fkBz#LoQT7lM?;`NC;tz$W=;T71bZ0K|29$;^KsDgEs9{L zNim*6i(=hhecs>RHwQrlWegmE>n~6lisPO+>T}ifs$G^Vkx zEJ{sWzC@o;wwy!f#t|x3!WK>)>{#gbowva0vnTTE-o>d=KDIP!W+#)3ZkA?b_3dPv zy4{*0+xcM6`PHX=7&NFa&Ltvc^1AUo8qI%3bG-c+p%{}%a14L85%z^TQd@_D`-#(L zY0i%M*-}<-?9=yBNs|mueKc;RJT%5DUmhe}$L(HQCxhb_JE5ObTCH;QeXVxtHKUMr z8Y!JJs_ny^h7=x7`=z;NhNHST3ydAh=}v**G(_V9+#XIxbE&Xo})s)32QaCt(<_M#RPt+neD9eS2?pP z8?^33^FhVK=R#{h>Ws#doW6f6GQm#J>Ni};YJSgcY8VM&9zeE9GBBps=zZ6tc0H6A zKDO&Pw!126W|17n#n*o|iCZ|&7)Z`Y`i8MUvuH>ZTpB4i;S5Fibs&wVpA2G0YlN$4 zR)62#*kAq%Z@T&wYQ^0cAiWW$3>NPoN5ajrep47UonZ7rFb+SQ9&L zp8eZ?&mX3fpk5%mRHmuaI*%N(edT4B?(8+O_>!pwg}qKpJUMEnFIh+mJy~6hhg0MX}JFG1HiHK7$lPywK5G*T-n>eT&7e_j1NcK6k%j&?Lk}QI%@(%Nawc1L8Rmc8&}g zOB1ChpJ-2bPPv0kKn@Ck=Lqy8QDjl8<21%by%NZM-IRhJ>?+=R`;_sqcT+adbC-6X zDF7UX{z4f`6HT$0J;(Oj_Fttiw z;(x3?4mH$JQypzl_P=jYe!k%oGMoKjE-)Lkc z;(g{5g3+p+bxkvZSorkyOUf)VdxQJ9bdNuFeqV{gK6vS7AE|Y!PHJwKvpCX49Ke+z z(;GtI?o|r%`&IYsV_rQn%R-fY>#tLt<9<@L4s^WG_iF|In_NB1Im|$Ll-m`MLM198ckI>4Vy1Qx)r`+X97 zl%2P~Kr)WdK8KfRy(6|hLl-K)g%Fo)X;0sFeL%d=bbxW~a%01j45cCQ#|k=3S{)A& zvqbq2;aIEMp*S^Enx$qjsK?+uI<#U;tH;!9eRT z1H_1xQsDmp4hSLYJ?0GHNY_UhYfdVJ8=Z8{)D@~2(Gf?XRoiLEyrm-!I|=ZAV}e+&0@-JAGm6DH zk3$5XO?MNh%3d^7nQ64ZaZs$oG18~1URDQZYDl+gg_GT{LwPl(9Y^CA{-beTcKuU? z#VxR@uF$cZV(-eBBp>!NB#YOe`-8gMRFOl#?;xMO-L^tcmac5b^gh>moi6M!IoaxV zkCBnKCwj#3uHFFry$!WklyE!niiS_UYzfy`{BfK6o8~sBg1%QLmv#L>l(LeiWASPm zk1qM-Q+b)VfEdp)UG@>&sc@EyJ?OhmTl5F*CIP04=Qq=q4!Pn)>H^Vm3Gtq;HL=qW zGGC`jS*FUKO~n0h?s!jxZV|H1T9NA2e3UXeH*H?p6}o2vMc3wP{&MmIKYcoP*ESxO z?rbxuRr{V{*)-Y){MUkqiRT2Ye=s;-Ue`KWtk_0qDRlDP70T3K*NthDnkr=EQA<4N zCdDk)_KXL&&2|@W+%~%_fVVuYS(Z!6^b%(OtG{hZa`G)GrbUfD=_qSaLUvT*>S9ia ze&uULQ##UUEyO~uLIDY-q|ej?nK`CK0*UJhXu>>#{Ib&O&A_1&u#w=ofw%wCy@4u< z$!zQF8|q>}vYeb=2;Rbws_bH$t=QHO3 z%@IVkOCXdKi#ERjOzvZN^Y~%NIIgdU?w(YQK`SMWW6~m^ni}aaUV5YXj?>*WYc%6> z?+<1(q~B#!H#-HuEaueX3P5V1Da!}cHWA)id*Vu2%d!A8)t&9f z6)&H=Fm`hjp|$0T{2vBEIe-uO((YWBms=XE)z8<0zBni-7Nh{lM6yvy@>gL8(Vz5s zI@j3W4D?1isVON^ii+_`-go%nv{`f(lGr4i&^=|RnPt=hoU!d~C{9ceG{>UFE^-4@ zinwHj(J2T45KinzGPSKMlaHCHk`;FpkR)YP5FSi_5axpLdw4yyMj+zHL}XU!#2CQ| zYSwLn;LgbE^5EB6%$F%pHbpT)ES;tx{9H zlwXFZP|EcjVUDcl$+CnCYq>Rlj7YQ1xd_#Y|7MegRahTFxi<7*uydYBQKu{$rD?*P z*|ErE{VCycbgVmNx_g7*_41)gOEK@(w^DmEM4ww$OkSIaoZa|cbM>9cv!>q#ei_Qc z`|szRvP%+LjWW&B$umlBT~MvWMY;5Kl9zyPd$P?blcs(eWtAh9Nz)sbc9Si=M~1=c zBpKV2rZtU^O*EhnDRF!JK6nB3okBB5=f6z@$m`D6;9)pD4)VMXL-=)w{-sRfxNSH* za?@oXs=XY=(to~%XTmtknwMcQ)+!hj_G%b>B9bP`k@&Rreuz7xg<+x9KB55VgB6CK zDunQyP+I-Qr|;8J7oo%DxC^38YfUN9)M^EDATFLi|FG-vgK-p$%wi;(Ie*9!+{uv) zFmQ~kO(oxP#+Lb{kfqspQWV=44&n+AVDq8>!gXr~iWr#{F^fVLeGZMm9Bh3YO<07m z3iqRWZM!t-$40zQnQwiV!s&8~8}z!`nlW`7Ev3+V?3XywU2;w+zD%3JPAT1%(#6D! zX|;ADDuX*LX_#r6eI9(%>Su5Ko2#4d;lFU;mMXK^ceFiqqz(Xs=;i>i$cKTj8|&mM z44JaA00$X+BH#n0Ze6e&f2*)*;E7e9KMH1Yelo%K9IF_6Y9@|OHz%ciI+e@DpG8a_ zT?kWfl2Y@;ujAt!p#~dq2ylR@zd#He*SYQHmTU7!q4L2Yv$~HFtX99EL~^H^2v6x{|mO zO^3kRxtUpLmLuXy8JxvxUB}C#HoRZ+f8Nt>w!MFS>iZJEU7bx-`^X$)any;)C zLMdKuHOyNQ8;w+|KQkZQB9>NK_fa<{#gQB{cnXVUaTQW1ayiPizZA))F|hOm4#D9w z!B&%;#dbsTk7C(?J7Uhzr;cb>}RJ6;~W@U7Hwvtne{s@U04c zq~(&Y?{G^l>;xtCvnQwYwTVjFZ^BH1Zu_z=b%Hv^SHzPN{KX}Bo|B($$y=#?UMZtI zVP54%*T!`Y+mT=2izrlNg#Wi1OCgwNAFHf#RAr1sGh<73>G&~zsr?PgjPP&#!S)OU zxstA(dH)%YwIOxP78wPT>xIVYGW|z+nD&ppCBoigmViaID-u)|253$kDjgf+g1>m) z*syWq4hqtEoEm|*VT^y0_yy{p@F-5boa2l?A5s3Eu_#v6xX|j5n84;F*Vp{C2)Oe? zmKtz4NzUCU;zVUdoa>PeQj66&nmcN&t2Cz2oT9xQ1quOrqB|4ZLfMO#Y&8Z)twEYn z{o#q3mY{ucsK&+#I)KRn&_x+QM5(oM)qsLDFBufv={^K@8)4d6NYgIvMutQNN`8y;eP4~zy^}MH7@k`l~ zuZnMwXlAurX3hEQXNfO_uIJ((0h+g$CxQIzk^|f1qd0MGm&k0c_v?F!*`iaAqHq82(S-?w zd>jOlUsM*Bm5e57o~IK-aW_q4wp|u(?_?NXq=jCnbP29r#R!z~@R|l!iE-Hm!dZ=^ zjBIv|ua|vJZpEWw5;8#T@Cbn!feb-?E#57iL=5wCp}d_n-9N}`#-xzuiW}tp(+?R# zHeq5*6u84bp4LT@3dQ4e7mdBIu;B8EnBIq)p~q{hvZuK1!o*LRno zdZCK=Gtli8fAGkub^bEUeT*wbh7yz4YRyc<8tebQpUGcnBHW0BK$6r@^;G%lyOOwj z5P;{bI77?HO4*JD9cooFe(V8CqKddY#KvYV+Yk9ZwJRW$LqtR5t82mQQodu&h`-ik zMUKe(ppey=O*Q}=kQM4b*@J_PA!f_7-jK=AJWEhjpghcbo-1kVl7H}h0W)fyb)Bw5 zscOZSlEC?$(3ghtaPN=mb5oh*+Z0Tv6ES*1wHg`|e?fOPGYmESA+O38lm0giVn2N` z-nVhk9N=0*$%hy>*F-AvZ=3{hdo$KZ=G+or&izoXZWq(TV`i>EiVdea8tt3n z(phL|RSHd?hZH;Uj2UHj~5?`(S$LzGsmCwWf=EoVfb3SGApGk zVFdo=#1OCr7vior31ue!!h|I%2No4Yl6_$c?#w_SnY?9?&n^XLAz9FE!xS5jnQ zBd^+B%va(89*0o<6J~yn5$W@cRngAPwgi@I+d^gBTuZfr*;2^?z_VWdE<}gCuJg|( z(H6Ph3f%@cY?!Xrdb+#Y1)de|SX;!XL_vQOEB0eJ69MEW2wA=yNhd9XW{Sxb{UN7r zL9&GRJNTA=Ue1Y!+Sq_U=(umCdyi@8D)Vz_^AGXc`1;W|O^G(Dad z&&`Xyz{$xlsyT5@iZgbIr!1$|9TB-wT2lABxF-9)uL}`n+kW)W&w~c*AX_tB`g6VP z{1_{gjJ3yaNwswzRh#=bblD5RF;xUPpw%dw)3+Is3Nx8vqk8mV5pgtm6A!dQk@QtP7 z2J4rVBr|&+b-roi;bs)^09={>QT3EZhg)~4@RD+{%C{Fy!kv^}jc9j&TJ)|W{^MRE z%d_n$-RhueS?^x+bi1eLpCeS%e+Uf|q#cb}Y42L6jhDK>l_O#H?#5lrPkH+qLfXWH z<>}ODX}-u1oT1EHj(90~UuTT{x)(G`&oA3D`>8^WYh{wfGBe)OXjG$QVT|QcA1Qy9 zKUAZ`nI4u?3e;{FPS+$ihg!Z$AXqe>VgpxH*FdJLo#6t%xSE z^4JiWLB)*rC!!>@)mOj*A7U+R+Dueg92}ph{hNY7zCgq*E`ZuJvO@9a` zi{cT4u)Z2cN4rWfXit<}m8vl=myjO52z(UwS9^vgx8aR37C!Q%mrx1;D%yaDs==_J zF)nBBQD^jYOVVS(uxSUGQu8JIEl@z2mNH8|K64B5EIo&3^vo|?g~zX>3i9nuzy7NY zJi+HhGa);Ts%|S)(lV0Vq@P>}?e7>nEO(|W8fLiNALF*)M%^q*pz^1ckvPPoszOxJz8cWafloXTM6fOY{ z931>^gg7>CI`uaPL2Vau>AB`c0>Ts|20Df?1}GO6VW{Nr#B7k{Qs{(XyX9;*m>S(n zMpEU%HhQ8GC!3(Cul#tYUaRU9j0>3MuWHn;$9z;AE&)w139)5crH9pHwqx?ri1Z-{ zAq-=|7`>S(ccuAktl$69Lj9|gPND=$4sjb#9)?brU)Af^E6*mCTd8#&wy?tEu23N0 zOdnf9nBBt0?D%g7VD%LxTT=CsN9f$$0hfn4`QT0Jra*BX~$! zCb?lmELpBPhqlUMi-JX7(yrv!zZJ{$?7qWte6)<%gXhmetV=L&m`I5TNgaOv%w-S# z;zgd5AiMwoy6~`N*0)cv;Ips*0va5egOTkRWsZSvlF-O~wvZS;?6KN{t8I~r%v$IgCe2P_uESq^_w7cFP5$rlF=*v*2i8u3#Oo4Kdi^lG}Dybaq z(d6<%7R|Xmb6y?pNHfA+e}JyKw%LzcTQ}q#;2KOzp5^>wI!kF34q)P)<9s?dXeNw< zv_c0Za(~Pw(O`xVBqGcbK(s&9I#fJ7Yc9LC6i<;54qlMI z@r7zY-4_|~xpi8=63c913RhQ;V!R)kZh3?Q-Q51p$n?^*;`@OfU9}^yVc5iyUJt1% zTLIBZ-PQr8_7LH>))lm<=Yx2Ug z&_&gS>ntO4=JjjMD$FEN@xh@i~R;G0}-XT zm&Y1ES^)hOIX1=jZvA>l?}vFPOhiySPUwG(v5$!_yZ<^oQCSdC$LZ@`p)Kv1(LABz zefOt(b_O61$^0h|v7X5L1G+UtW}aK%eRb}PEA46KvNkkK9@f<;^2v742$5X9YRw0$ zL+A|9$^Xwk|G#HpP6WmBUS(r?(yzye5&-aMSeFZUkK*!Tj-4Z^NtvVnDJ@n*m(Vzj z+)i|`{g&>|;Ep10QEMWED?-CsH)Ss3K~=DL!ejqpiD_WtJK8TISOXVEG7U1 zI^4xMr3YMMB2~dXB=A12PM^ST3rE*qozZ9;BR~k6mztw zXjE$T?I0JQg6vepXgP_D&XEM*UjO#DeK&ayDl2`gnrVTTB$kMRFeRy&sM3528OQpN zP-?4_aIT`lEWXt^UTJ7{?4c*P4alpIP06-PEAh9AMxb}fGjF+wi*LrJaiC8b-{4|~rMpt4hYCnhO|rK&*2P@X$7jBuoM3J{Ika0Z_@ zXU#@)MU+dE6E!FC-28kYHtX8O6+GDQi=O)>tGmZ?v~CW-fbu^xg9!4U=h4uMG(QLB z1~-z~IRC{yb%_kYolQNxCFjePIq33+zkLZu&js%r7nKlWZ&uV1rPS}`2wP2x+liaN%@G1Sk_6UHK! zrDoxH(aYMEm;i7~G>4B37LM?m$2p;AFxc#7BABxuCY` zlm}~G_Oj`sgta&R`rS#%v;Dc|4llF%F1!%||GM#%_KIXp$VT`vebvMwmD75t#rP!z-w5-<=aFqrn#aoiirazFj4 zgtLN?Wy`csCnMgCFOS2`RY&B{_6{T|D_pEWrADhtS2cep;+u1PvIINbE)3@jmch{o z6lfFY6=}Z{*0CP0=K@f;{Pz&J#&HY^sI9pTVaH5`(-9AIQe% zJZEgZc)z9C0X|3?yC;f>X<_#CaWlq?GL&<|twxZag7w^u2}ub2$$>%3W9gXk6`sp{ zTPn^V-cvw}&d%9Cz&&0yOu$o<0onvkb+twu3Xds=1H`2#GvY;GU&0X~L&H2koh!kaOJ6S!T1=2Ev%Oz*{qW4BOvXF;l5S@ z7?THGGk$TRCM%v(0udu5Hb9Uh&N;}nz~({*sRm$DF?!mBEU^r+>Lu)eSXQo`l84s7 zJ17h9Jj}}?eivUKNO}!YbmZ;|Mbzdb>y`%2%UiP4-&&PSNXH6fbvYnDLM7lJ4VV1SDyN%EtTqV z9M6TM<*P0YyQncbPpdho)b^wt@i3W73lbD0240O_=$SPqti`03Ay2ZtjOBVs6Q zZ7&QyxZwB%At+eD@6kai60q8Z%Jfq0)v3e_TU`>$W1vy~_JX(Dp%W?!C4VV?GT7GQ zCoa)BXXAxRKS`;jB=amzkC_;btFc{Ntwnw5)2RC*#~Nyq9yfN%wC{Y%5Q)27WC|uu z4!hA7P02=_H(dpA*2Q+0YBF(vXgYcN9zu#ndYhoc&UAcaWCwD6t+tC+GiUG(NAft+ zDTYR)wpThCkRdW^QIhrZKNZl5IF}eSDB1}89#_0MM#<^LU_&H`aY=NVNl1{?HMTzU zkz#K}ENwqXA|+_T1SwI_DlWybOi}S{e6cqG`!!9$>f?{k2x?EhP$b3Ae2pNX{TUG` zStQ_0&QN&(eP0~a>v5d*9DTLRE&8H&3$Gz0_@CLqoDGV_OI*n~Y+h)m#jixoo;xzWRlPS=CUH$v7OtqPN)~3jH&{f5jC3NS(%~SG9^w&HQ}4lpDPyc@N7V zahckzzm5ET5WurTWh0l8X=}rxOoMGECzkgvP-E%D<XPBt=XM0{6RRf5J<6^mA;x) zlCpBOdGAS8r&*}pEm@DH_@`v#h2#>sk~n1;W9N7KEejcj+T>%b7n4PCCT+P~W!dnW z!;SQR8(#U(1e2U~?T<5tQ2b}0`@dK6A4o=mRBLx6n@_F14$TT=0Ol?XR19_r+(iP0 zg*Vb6qBM?DwDWRKO8qw+Ogp$If>|zk!8M4)Ze;R|fO6&sGn2+(B(uSejKU^McBj-p zypTQF9T$;Vi-5|2d|SsbxDB0tgT|OCZ)wj{iG077(+)~1?WV#54rVMVxQ`DNUBdGdD({5<|#|EwTRqQ*5WY0b1E8J70(Tx?RO zZ)hz%n$uN6z0!i@ ztq{m@RVemX_6;7oYPE8NGZP@m8A5Q6z~N_kEE~0~fi*w>sGu5l?v)dkr0#^VA#b{N zcF%}lbj;gg-n^lfZtj?kJ#dhT4@UeKQT}h|&H@aU{ux2?epjRaVj)_J8F2m8P9!Po zBm348l9$dOpGZgP6^#0EQ}U$(&r9Q#j_SuGTQ0S54-`b++){Y$lW**{sr>u!biJ=S zIUUdl)(d(4GVtYi4k*rD=ar<8kq{4wwt~(4QR%kMLIB26Jp;Y!LX6?2@jF=FVi$|j z`%-O_RgEysrl%m(%E%5zfgLn1=I2g8E&?W(^^Bo+t1cWaJID^%Bv|FaDPyvzUIQ~Of*Wcv8HW$(uQ{SAM(>OK0AF$?uWki~= zJttPMX3JA3npasFbxeeY?lg@z{{3;xKQ0J~i#i)DB4E%W(965cW^3baIAi}?UI=%Z ztXz`_%_@>~iB5a29glQ0XX@}~&ws%In$GF$3~13J^8%x#^cHjC5u+bVY5lRTt;;i7 zMa*t~0VYa)tbTX==|pFujZmwqy#2!uO$r`2TMOmCQ}X`;Az)zXU?7H~KWvF{fLMqO z?EfSVZ|iaZPpsTHw}YS0hbPQ^0Du#^m(PkTMbyG z*L;s3Zk!vuncHUbnevXlIG$xUN53&XfH3PDsum~>}lLo zi0&M>K%-4-^y%^k3AC%%p@4@4%;MqI%eW%JALJUtKQ|GwH$ZQ5O9CQXZ}ttl-WvZ>)J~2-T*XGHIGpN+vP27GeUN3yvY$k10`r=9(k^#c~Wro5W}pt!cQmGZ>6j z9#dL*8pPB8UZ4Mc#Coq{1puA8Xb55zFf)%qNUwPwr9i=8vQ^T?6Wr)n!*3mv>#f{S z#+um@psC>2NFX)1fko>-WZ0%tjU^=SkO_-))0o1w95RsRK5G}mX8s%n18Zh)lht!D z>Shp8;m=|zhpqs_WkWmjinzXK{KRrq{0h)AfjO8p%ja4CLE$XgcwlIQ6~pt$cnY>$ z#Z*{?%4Y3ila&;RVMIgl=>5C`2oG{;PMad6Op$hIhrdOwoG*YpgC87j5;pxqy{W|71S|!7(9_hg_(6 zR`5HYl_5&Toc5E!pQ3lAs!B`jevH}Q7#;Qm8)3P+vN9<;r-$+|hmTm&;tisnxBYC4 zZU^XMLPDgZGN&3PJydw;Ejk92X4#A*4$e7}6@M6c3M*Ok=RUiV4sg1fzuzOO=Cj|k z#BXhdl=fxt1izg+Yeel_5_8NfK?87nsfSLoX6kw=LKIKy!}t>6t*x1(_af(=fW$~g z5h_bZ)+`eJBiw4UPUi#cwVM<+j<{MXXao|!*5)6+#Q&1s9074+fHA@s&+~+kD>46{ z0Ox`Y%HV6>VW_SsT5JNUuE-~tt00TCC{jI2Y?2{-hct3o-gQiv^9as~Opu>a@V@}Rz`W>Y3Nq%eG7V{=xO3r83Q&(o)<= z2B#@Cr>~h81{z#K>lc{x?_cJil)Kd*upDr_-eb#j@pGErGeZDiEdvj$-=}a2-8oRV zWhKWBj2!MNrlvkwf})P;*ReXW1q4^ER2Tx@2nqWysToHOJAvbAKBd-8CSq5jPLvn ziiaa4xTzu%IF=>W_IJq4jEXM?Mux;0YBFW$*1Yfq1vp^mlzKnuDygi=CsZum%BJBs zqXJ>RLDn!$_b4+Oh^u{*cFQ(=^m7+65$jsA(TTyc2E^>A)R{1B*OQiSPfaUh!F%tL zE1OgQX!R%m=L_~ffcgKO$Gl+fDgg4Wii*O;AblI$&g6P}+~zjFShi+`nVWb{DO|7D zC>h8elEIPMK@qa3C8naH)h|OJJu^_FA3A^R_y3g(ya!LltU>fy9i0n`xdub2$NuUT z5BnB$X>ISC^s_3tZ+ZB9tBy%Gd_h!c&S=SN4^pG?`h@S7^8~k@#+AJxzt6D%qSk`N zwZ}+P@~tltdLpEAC(lisXqE~dzeHu3CUudW9zfN(8)t)-v8jIN>zPOHcCeITHol~h*AxEi-6x75Oyq`i9M%4%)ZpVBO=PW*Zt z{A=@Y&8j9@UC)D*s_*aGvaXx%f9R$u5rXvoWHDm6JZ*FY4? zi(EAdJ&Qw+L96ZsOE1B8)k}%n)I{R{+1&d7zDy7xlfI%qx-aHubLm*G<96w{J^c)^ zX9V}7L5r$%Zj&WC@RYg~vxG`nGlVQ$5`V(u4GY_1piq%BbMIQ*{tQK%RZ!%7QC+{! zxD-Sgeh{x0Fsl+eFU3NCj9OWb`mu09B>8}-y%w?Et?zfJUleN9lG=FjGa|BzZ7+i0 zVE{B7&*(7Lb^Zz-zz;@i!w={MLAZ&HBT?4~)-=YQ8>b-{+jwaP^O#Tz6yCSX`W|jD z)|(E}lYWQgoP>+okQ|S6JnTGZ7n%R1*#b8g4Lc9;ZA2Hk4fV7FU5C`VJ`l?4@KaSS zv%D|`=m=NuOum3dBTQJ{>}@#WZq(TDcY-|+;>F#w6aLvU7_^801tn9*Z5TXOgm35! zJA~{OQm1l+p0)O248+OVdPNvtKMPBa4`dKng{CK%D81d1P-=Dy9Co~-3abT8e&4b{ z^3<0;JB186aNV5y;V48nOxD>vq%1JOIKZ%={~u-ae>XrNmy#fNJ21m77ZhmwZ4raP z3rPQKQll2*DIl3pe;;99($+0%pnhW2sL7H>z`ucyDCNP(89S9Tp?sOndC^oOPM*Ga z>vmXm))zR)BAlcGuW{f7gZejnrN%9-zQM{aDC7@5QT#9M|5&+scB^O!a#t~zz~A(# zu{3691)@j<4aXej-r|Z?dN}cMVh$kY3YKVJ7Fgg6P41YKqk?fImu@?YwU64pnM|T; ze&=u(c4APEDv>T7yD8bp2Y%13=n*w9aZ2fh?%|iGO19nu8=X<08j72l0 z8K-jCwrQ@)89X61b$C(YOV(y(PHlTtVr7iY@j#;bNk|##s_R+rbg3%C1h6gyFyM*f zaARxa#9{VbX}tyIq__)c$j_lrD0OCK^(@x-UN72-Gy0J!Var;Iy(XE4OL7l{JN+4Q z>&?c`!sxe$KSXt0Lot~@KzL!9yAW+9V&LK=C!`c9OI9lU|3jn!)E(MC?;2?Kg#U-G zcMPnv-PVO0+qPzGG>xqp+cq29O=Hd(+iK9*Y|z+Fn>1{U#y+#}+WXsQuXVoP&%gV* zF~+44ffA4XvUVN^wSwqM)0fJubn`n=))`?;s#7BSupLgHf<|X}|^#P(B`5 zCTO@WfHbcBG=wLuhz;B8hk#Dtfmh|10bkSm;Q-*W+;z!}rp+mis{NKU4TYrNDZ8EE_0E@^PC$@1>NkQ+QhP8!zvhH7{S{#i}6$PD)`h>foW~znb8-y1&>`69GU& zZ^A=24%<0r|eQZH?|obZH1JwOi(S9?_Js8X-{o6AjW_l-UdbMsS>a6>#OUrxP$7M^dxBe?zb-8lNcl6 zANo<-)qY#uBSeE|Ik&`iWv&LNJJTWA&D+5;Mrn-Mx5G&PO3z z)O0=WXl>59`$qd=I`|a_@Aha>;bZ$=+CN>wPjVxX%iNb^cpiY^F(z{UWU>WQh*nq?GFzEUXEkg8Y z8*)1{5B!K)+)90qZruI&5Zux18g)W?ADtX_^2U5tkq-Y4+o z%SESjd)EMdC+g7mDsOg20!Ocr&YT=Rk|&IK!kY>(6`=1{zc&cVE^`ij&t~|azXB%I zx|ce73K61l$G9#|gUKb!x=KLirmN0sH9{G)F=0BkR!$#7POEsU2pbe2qpVNp-c~Mw z7zZC;O)Otd2;v#IK4YWeEGo$&XoevZ=uBGkD{HsAnK)~4*!68Q;}G_Q10JDe=OS9q zQnR+d0)%^WOZE@8)}~NDU%~SPStaSk*;5xi97Mq%qi{{t!%!kdxIc%Oh_>i|Z%L`Y zV~E5&zYt$zN}4Mf<$Q5DE5Nmqi6FfRL3(VW?!lYN5l=b2p`YV_TzgR`KUzj}S!C=$ zVlx;^ZrC0FR1Db9k@FEW(ALIpZ8dXP-`A#wIH*SdI;kK-yloi=?k{&rA#|7s$#zIz zYs|Z0ZSx8TX(RK%TsuYh`zxHUW4qNS!`Q<utClN z{NKfAy!A{#;olK#g~y2>zwgv^muuxY?7CoHx==r4|M6l`K}6a-EAz=BZH zLH%j{Ka6M=vMSo{@Z$8ZMEb+d4OGjEYH3{UUaL!dTBRexcP%Ya#>?wb%GSA_H*xL9mcI;fAc*>67%7@SS0%XmHQ;7<(@^5aZWh^%(umjJT?F`@P)1@A}mSxUaoE(#e^8bLZLswF+ z%}9K2^e1&}fysz0Bdc)gTwQUBShoTT&h*U1d48}d#Lp^1z9o{ecxZhmfRb#F?NX~bXFcDGQsMJqlI2>#~hz+xS6&3xfBnlMiU@H!( z6!-`kb>voUj`KWB)-qRf>nI~N=SOka;vq(NEHvz>5>#0f;vl=0wf9X_yp z`cztsuS)wsbElnqQ@CxmfUa$T**bv$Y%hwrHg)k{HJ2YH3a<$|JYSgVl=|-yPAu(V zWc;L04Zd_Z9-mY^InTcUr%{e3uy*%mx15d$}BVMvP0r$h{+4iC`NdGF?5VT7$Y;(=TfP$M#R zvX#)r1JzE2Q&}&OkCsdaZ55nTgtafb?^<8)U>rb?J)||%Bf~utE`e?esFoY=3R_QO zCCUVTMxVzvQY$1{cGO9f*a=zre{5|is5@>)jf1hJo5HlKVIlJU^%?dz>$Z%{K9h>> zaIwhGOZF4`DT2k&HSZt>H{H_u*(?@WGY_nP>Ve=^4{BU zFW<@=8r0>N0vM&8e@+ftwj95JnZe43AJ2|G{*FA_vA~SpP%(1vfJ|Vd5~s4_u;&X=$<_QZwwNVN|Pu= z4G|IbFhDqLBWNhmpa6>^{&4~x$}&_`tvF|r145-SyD-kYC9av232%eAMSjhi;-8nH zH00QBS&H(EE^{%Z#Ouj=PE%AD?i+AqYfw~OK6bntv*?@JEijW7ox&6u)suPL|4otq zkM5x?M6A&SB!vPMfxrTHCai%2z4_uPtzLAB$&btBr%*i})FDP|z7P>(TVLZ|cc@eNtX5^nIqm$Te5w#3)?^ z_p7MMJ3KIb?cE(R7ec8lL~xK)_4VbSmj3o2Ct?`Q(9Q-$=xd2Gj_mFI%)S7BH z&GA?>F>J|$AVU)%98nK1?BU;QB7b=NHn$bYFBV(myI>8&O6Tu)F5WUy;ntjY2ya~c z+g4jW&LR8z&n~AE{TNfCiE(`1_|$yA9=$Wdf3j%)zuxIDn6OfVg_r;!hItI9fJf+g z+36I`mE$mR2?C`Ru5?qgwhsWoSKZ|6Z)IPrcDH_loFsBJook*;Ys9M9v0jyu+OZJ+U%Cf>D(P;{zPOS!Y=3Um>uH>5*ntvRjIeUJa zsSnflR7MWj}bD& zepi)zwa&*m0)JE z+Si0>$*Q8fVD6)+4`W|L^H&a2`*g7uymn-%(Mt91KPXif+A(SV`h*gw4D%B12^aFz z7C^^l(0K(jD!ucsevqcnU9Woci4$;i1O!nnTI;H{JK;$x04a#f4-)ykt8H9{Gx&eo zp?RO;On>;qQGj_~%>RYwoQ0RzJ&P7CS69aQR4V*;xMj&~+5H#Vk@^}J%*xZt4{YMv zC3#ItK~0Kq5;=h`i77wPmzBXo5P`mMIUb}nT3@Xr9_`Jo>8WwVjOK7!<7q!_S!XF; z=5<;dfe?`sPA>l{`lw)^yMASB=7%jTFB7;66@$@vwy zIkEK=lm$GhrENHTltpr;S9ta6B|(eAqT@f|Bz)4H`D{dd1f%+s89Zx!_bo!KwYBQV zgt|*;fhOSkdm(}N0)u&MjO8-#h?nsBXt_?o@UA&Q%4d1%0vo3VKiIKFl(piwI@b7m zR|R6GdU^Tw41{iVyESdD43?%rl9ZO9|n-B{B0a# z(k;#8>9(EXh};By#c{nJ5a$~A1y+YK-;GJe8=)A z^uo!c;{jtJJH%+Wu@Wz7yA+|~3euZ08%RrsYvR_XfLqVSf73QPc^e5OjkRE6Et5B8 zuCJ}guzseB+LvJt6hk{v9Y&tR#Pz~_K60eCF8GGfuPw+~>A@1NSeCn0naXpJ!^R_y zq=~C7Y6l+~u=Dz`G7m!a+%Ng#^!{hrCaJdOyK|6M<;TAP9opq@!M8ndbzhF;pT?hZ zlP=Wjpg+a5NQDdhVpy{kgnN<`T-1khI)ouQO`MCoCp<(wjS}F-bNtRk7E3leD{nKX zT-A(MtzXq_3&q?d4~K_~2@U(F3;23d(|p(dI|BC4HdceT+w^K;^U}ue&b#U#H&*9? zGKFTp_;68N2bJo9lz6zf4AFy=4KH9`uSEf@%}R~(?e4pc*sQier@0cK8b3tyBFhea z9o_XNw=klWwbSUoxcRn$cJYd*2j;g!>m5Fm<)u77`5Gfi%o4*tLh8b#{|0jKJ8xaN zdJeQLOIVrhS*Z1O*_hCmrBDgRx^W&*`#`NE01J`b3Cp2ty(nDY^(E@9Ng_jtn*Vj? zNYN0RUK0X&3^mHXid&=}jQ5nkpNLDnr*KAMxF6^XV@ESKo+amm<|^A6rY+7B`F!n76y;^nP*iag?p1Hn@Jp!1b_Lg8VYXK zs`)K*UQ*{&TO^d&DvS!H8Or02a$V1P&ElBLTBQh4^LCkLahpd1z>FPS@h%Z?hEi3-1i4chlo3p`<(%7qXf#G|IFn-owNO5Un+ezfyoVwB6@MLpiW20v-7Tx?)rtjDn)i*?eO;&=DTHZ1`AL=JGs=MeXuX|WVW%GpK|CrL}ce5c}i^wGjE0*Bf)jJQkwRhXq-A$ zmnU7rljO+@UeVx+vEfBw+_frd)aT>^649C$oacc=PUI##UP}v}sP(LPa2Tn(aCH)B z`AJ5n<>yL0l~q&(=msPjzHyGCvf|bwI5Iq9&bd7?x(;?HJ;#;2oJIoZcurU9VKmAV zLG(SBqaJBz9mi~cM?>`xPCVbB&SVZ>B1Y31q zrCJ1zA*M8Tf-aM~2>p=DgM|D;UnE1!@MmjP z*Ycf;lZDv!t#MIKI0b2RWAPZ+CM@SiU2|@+H!{;=G6LGdASF^lhqsCJ=@EjJ`zgyV z8u8(x#M97+htq!;t{cIYY5^?`rp>7{r8s*4nNB)M@WbIf7zt{l zOjr#_=BT#;Z`31~_3?o-74WGGCCRlQrR zWgken&+M{Z^A_mruJ3A7xFMmJdcLb(>+;S@RtHpT`uFzN5x`G{27?Rgvn zuvpcd;P+TYEf-;Q;hZX5`P)SFT12{=q9ZK+eYgIy6)Tv=0S4fthFm;39^3^*qwc2f z#+?pr$svR6YW?K$*V6X$=jc4g6b{@F2}(I8)sW$6*IJi2^iEO zUNo4FHirvCqt~CcNLI;Fv>kKG)-`0&4h9L+8#zrnpM+Nl4n>=GgnT6ovXW>9qi8JW zIQ_XsK&$}sF0%FF5kzvLp2eyh`T)aROvu1wTU9!zsUDD4N3C;!3WA;=Qn3@i)T2Gxb36Y zW&4w|vHOX>Z-3I2(qyQDzx_@+jTQDv%M3Mm3+WKpOuwU;6Ap}UU&^}i7XWSsjgZ0z zMv(}`QvHo;`CmYA0x~Q#2%z?El=P`1AN7`1^#X)^oP6#r#H0F6vYFM=^q{Zg$7dZi zIOjePVXe~5D06h)?UsOgV0yXLRD;?&YK3`N^lq1L%#F)7A&_pdHVDzSB5lv{TvFy^ zXlx=MWrWwCf&Pn9DEx7mFh()jbYX#XFo%K{2qg?+Y<9S}`Py(7(QI&4_Ej2s)VZ#PZjHZMzZp4zDO&|l@K=Sf`}_CP7Z-HIYgSmFWM7Ix zOF?HP6=?T;OlQq}-tmdT8#(q}4H~1l{kAygf-j32yjl|SjCh^9h0xDWl&MS;udMZK znKrp82b7k!q%D;hU!AQM^bpJcuS)S>D|;?Yi3xljp5_w|!$*N;$*9wl2yj@HX#mo| z=7s`560e6N!aYCe2;TQbMl9zgO{TFfTtbCsXyL7yeozM!d~n;#{3<0$`YZ%@slD zGx5y?z+g`W#KcS5KPTC)C7Ph!{R;IWAt9NqyK>@b4+ORK^Cq*Bxl3VwpH4@Q;Ev%7 zG7iZ$8mzJ|1)v}>p4*UT+BYaU3h0eiK6dUJO$aX2 zZxjolXq>+h^E_|KtdLm(lCg!k95inZ8MIOnE|0Ib94}#0TN>lJWoQpisOc3b%0$+( zd@N(nOTJh=ujv7Q?RmqOSiSlDMXQd24!GYf4Qsru-M4v2!3%AvaB;*v znZcTUlw_*QaK1ER*jPEjH;_6jCv~DQ@M|7V$)i9laBZ;-@tBW#OZV-IEa&mO^RLOC z_rUU7<<33FT9pK0MvI>_5x4X*+yRt;Me6GU{~TF;&&e z>5*+2GE}DB53Hv~RAll=dR)*C8nY8{Y8dc-ig!J; zxj>e?`lx>Jm@ag1RnD^kD7V8)B?!nG8)2VyrFETCXoex8%-yqyT2bd_V{bIO+-2AK zs*OlAklq;UI7;X#wC^`L>qx1V##t&&;uHHmGJJHlW6>2JJn7ZYNTui&_2S>fIOVDy zv2$Ump1f;uE^H&bN>X^cI!xc>YCW*uRWHXZ2)vWtRs=v6epT*=sh~1Lv`hA3C<1w3 zV({}$#VI6K^}F!h51fAsIb|MfHBjmEC;dPBQF(kmSG{)iPPRy;op4OtSOX@#XR z^?@Sxs=VZ4S8HEvs7Cf~-$17;@?|7}Ri8J<=o1IYv0t_#CI4}ge3hY`kOMnE8y=Z* zBJ^Gu{F*V(#vX#kjTPaNRjzvrKYyX|61jA4$WfANpf5To)}dr}$3*@fge+)spQ-%O zTukk48)oETP|*@UPSR`kh09{}rjUXJDXA$KUk=3M&xY1vr6Hz;syf@aE`x?WRXe=SeG z0t4}y3f9N{s%(esIG|zwB$#5({1hEDu_F>cVd!sfFqilR7w$$!3*bB+t+%%}sjE13 zG(YDxzM&Azk+}7`_YMCyB=eTu2~FK)!0R-H7wwip{r{)v7%h<8P{u~YMkI_Ld^apd z+%qe4KN6P%4tnhQQ2oZ3C1rd=5*&L}o*;P`DH_q#Z!@M$k~sFtAICtb*%gR5=l2NRr7`S5(AW6B=^r&+b0DmgWgvW@Tq(CCLFk zPk?m%5;D?zZaQec6G@X+r`+i4x^eqBdJHCxdhi@uzpKx)_ai#1{tRv{waKO}d89sv zX|jBiK#iuNDYeagA&jkN z)6Fr}|cEx^Ep|mAxpy*^wnvC9K6#TMUvQQ39F1zt)%x0TD*L zj^iw7i%4Ztnk3Vjq)&NTE!!a7UY&0r0Qctj>-b7rPCP7jG@$nb*nW7UccgZFB9j{- z_dMaF_o8GOq0*#au~$NWj*znDjT;pgSFGM&LDv*QJV+ z!&Wi>1;NhE&#)fIhq9e9jv?cfx?(y&lsc{k(_W~!^_hccz z`h{OCbnwygxRsyYOH*ROu3-^P&b6?HC(E}h9wiZ)Tey+NR5Tr{^TpV{B>>bpYf7w~ zEK=9zxt!`Lw1LK<=^{-OOQnVNrE-7}G`QcCtifVZa!gXRaXWV09f;3M{&X|+2sJbg^E8?6o@{pN+Z4y;tu zT=YET|9T3QsZ}GLAR;ah#??W|e&`^-puks0MERXgd{VhegHt3;r}Y^9HQNb^)q>Cw zG&MQoqQ!OI;ts5iX>0ibDz=M8P^wNuD8kMFC~5jy`$6$4tKkoattlh=-uHr9dlIh& z`|}ZI;?HwMlB-WtK{S%yP@!RQVr~A2b)1M`vm`S>;&&rSM1nTeTC0KSS)Vn8q??{z z+1rlqvjxfVEEl#LUC5K=$#TRcNBD8?R=-~WZNS7NfRc;Qa9CD=$ROR*OrPWqp%-<- zXfkeT-=KiqJl3OYLv@4Z(dyv4Q3SW>qOFl1PrO<7-ZUD#KlVRUl2qIr@A0)*ZVWrP zX>#b1u>*;X;{e7+Zr2l|cn)w*Ak+o{?Jq1Cia$C>?ELthbg-k_qRQ4cN5(6i>?={9 zu-1zPlSvZBbpzcs+?(2OTfVv4vI!HFa-mdKR3L_jhmVYp&u7X70kfxaeyyy@K@!W+ zMcy8!6-2wpNhm(7lXF?hk<1k`QO-PJUgSH8{(1R{VO&@D2QryWHCf85y zpB|{d0a5eo11EecMHhX$Hj82OUgNNrlS}U67vtz@)T|IZ<@2RfL6_5&I(AD~i=PT@ zjx&5Lu}tXhWdT%PU$aUR(+@`>N=)O=;yblobj%meXM2RxHY=oH&3KAnYfS4746}zC z=K7;G$?FGEz%qdV5+bSoU&z2xJUV2GQVb>l9tM?C4jt&y7c%@EAr5cOMB>z79poty zjGM)y6 z?}RStFWoUx4}{^q%XdKwsW=}Vs7BAETd1`}SO0Pz4DYt_!b8t#{g|?FfayqA#!)VpdUu4p8!&coLx;q`9Q#vF*>6VZ z5>qfw3XJYSLAz9M^qqQ`+W*8DfyyQ!a>qiOmqfNFj)g?9F=3!Kh+^Rl`{5<&IeE3M zE5;dy_rVyW*UQupd~#JTDA{RBN6un{$k)HpSK%>s9!aU>iOd< zxLuChvA(`Y%;#
    rZGS@@JyII2l`b&wsLgRyM%Y|5b?3g>Acb&3y7OuR)!?Rg7Ub z=Q2oEx>kg(H7ie0(R5}4$LJ8IeKznZOE=%T)ZjZ`v{b`qZu=HgJ_KJ+XPC~-`Cm*` zQJ=F=Ka1z*#V1i+7B~7evH3dnC?ioMC?k>1)3;kg<#Ja#{H|7x{s3AG0ZkTdQ@#6e ztj!S+b}*vrY3R1Qijc7;+KIC+BhR0`L3w%+pX8?Dp2cFvER}rH*wFqa3rg0h&USFe z$4+m2pl|a0EaeB8d)!yv@E}Ef)Q>=+Nz%iX0|PBf1D*HzaH9EAL?uzwN9l&KF!Ti?+^Wq^MTLF z%cN*wPsFP@BC}GH(_QZ))D0JlLUEP02>_0^c9rr)_H%laYg8Jb*q7Xv@?QsLCJ>fs zUpU(2hX(&fPQE3=M zFCK~Ueb&j*TKIHKAm8p+oBz6L17X+9#n;kBvKL)a{X?S5{iNJwV*d@w%SqDRLfGwbl>DRYkf+Q44Z31h781#Sm4d zL;21{dsTDu$FepT6UyriYNZX!*9rmH886;_$ffFK!v*QjiC)g ztjOdxRX1ryj0Ufem{3}-VI{;$r{DclMfK{;7)eT|=pfF$Nof=b&4b0Xu3@zx#@+$P zy;2;`DLW8ezIKwaF2!Dfm|Du3-eQP&k;z$dvf6%@B2BN}1CeRNpM`>`+>Z4p9={z%~Z?PgK$=;)8+B-tF zrbS59V0TB#=|6CN3VzW*3VCT$>F-H+lV&ehC?oA%hJWxAc9XQsciaKPSq94^^ZKP%GVE z&cx9-f-5#tyz=HFCM9}kKv}$R>|fG0KMflRUK^{7*9D*Qo$5qlxq)&IhyQESu&Dfp}h9F^7ucR6n`> zQuSk2w^^2|-BNK7I%VbW(rpVhm?g{`v(R|Wj>nSlvpt`m0U*U~MbeL4lR7fSNK5Gf zxCV_fht(x(cf5s_L0`cfyofYzEI+72Ev?An4U6f605n=bGd&w)R3D{>j@J;$C9Cxn`@M5dEspKN=P125 zuQDhD^Zq!VhM!i+J8`Qgp)4@(!E`yH1Kc)y9GeRZB}<#*A{g&G;eXw)ZH~)qpHp^L zpz6W>nLr}S*rt6HS|2jVMN>Fx0y|aH&kpvf#*Fdv2ZFW|CtbM6u+)^ggV+p;18F06 zCuF8LieS;|gHy{;rQ<{9IPKy`e0&){AcdA>JCqZcABQ9sP=3W68X96?VTnVeotpbI zz6uVlCRX5wD4pRq?}tAW31GYYAc$_vl0Zfztw98w*VLL6{?zHR*^AIF?|aI;ouD`t zX0_tgJ1SoXxQg5V9KUOO%3IqA7UV3KOb6W`pU(&#om<6r_K!^9*|+DtX>gtwDXg#L zW6p{=2hNh$%%#r|x=$!9ogV_ML@is?Hqi2h{%4c_w}4I-4mwm0%n;K5355#aJR1#% zoVM)JXLAX)n;*p1;OEDt^LSYicZF`4a2ewV(~%xvQZz)9|GDcj|br^ ze0c8IqPV{lV0hSMcZM2HYO`r+!vHDQ<<@smUN>a^KI1t9W zDLQ21s!Po`zk}Xz4r444{B)$5V;$5kR2cu@$8)+t#RMol#LKw0Ze{ryl?g+9TbShI4G z{w7(pDjd;2>hp<#RhVQ_tBU1aeUyxn`UErqiVNScEh(3+DB$+sJH*H^0qVy!F`SZq z`r(+>2-1~>fC1Q7NK|RBIj@|k>sLH6$1O#?vkaE%!IqZRBlmleD@WmSyE1h^+Uw(7Rka;Ve|a;= zY(ol2n#0=HAD~y`9pU3Ezh(K+husm)gnOyZRz`??s}c*zu8h_jmTMt#*}t`5?x>2I z_7OQo7VpbAIG*l-M+L759#6-6DWaAbG4ekkT6WpLw=*i~_1i>8lTy!v!fqc0Jk^@D z)-A7sNcS!u3iEAOOqM5xTTnazo-|HA$3;J4t2i}5sX_-|KHcMTrTKnL`Tm)A z9J?MZlVa%C(WZq|_tGX~k>d|6mCPuyOiWbHQQ+i81&WxZ!X#ffFX zsnq9I8@)mMNgIRB4l}xM{F&mqUw_m+sQp&4n!O8A{s&UaAl^q9U%YGH;YRddbVP_KDiM#a zX$U2Mn(q{8Ky{84*ZA3Tcq<1{%}n1nH{!}DAN{rLm8I<%WV3ndZV{*DEtUo1alfcdNY=uGF4d}EY_%NJYr+(x@; z;SWRQyLS47bZ#px$v>)<~!Uuy|!+S9)T4-Uba1nH%q;D zZOTbZju(M^IMv&T)l^Fi@Bfno@)P`rXQLGBIXLzX3fl4x4NF8VA`Ic8PBF7o)mN?r z6QklQ4(1O*>mkQn&8snI+d8KO3VyC0VIk@x2I9wEjJ&2U6p(S3bfl;Ua+eaT^-+JS zMFE-hHc7^ZWy_T%k#3=qfmsay43(O^Jl<$DeEtmvV=vtKiyAX$#yvJc^DJo=_jJfd z<$aD@z>*q_SY{nq#8g9I*P#X^iBl33FIyC57`Kiv0Rg*-BJuG6Fp>hyv77)=Lj;@C zOw{M-6f5Ih4|k{K;Kq-&xrvY0&|ULy10(5ra)2SZBoNC^?DCj{2qtJyPcPlrkJ0N~ zS;YCqVDk~5PC2Jy2l?+EP{55iq9jjGf5+-<6s)qo`Z4gP(f*IqLU)UrduFjnB+H{L z<0UQp^@!x-OeI6+wInWTV=(;4)Q^?UkEu;jCCCo4bkj_}mFM+Pw!%Xb)K99VyR)9s z!gLNcr4BP|F8?NzLPWd{f_;3RYhfUqMHKJ37;l&U@RKT9ykVe?msX_hS&27!Ke)%> zD4Z1ZZa|KPsq;YGO+I)AK6r2Odc#ZX6(*0(*yD(;20s$t@`7&XfKt|S-oeGoY7v#GBbaXX{g)cw_D0|QSQX$1*Um7; zgm~J0?l*Ul$lTt*sHWz2FzTC)vYc6~jgkAJd&TQ1HcvE#+P6q$iD#JLCX>r?o|>Qt zrL~Vc8YU9_4~0n&o^I*2{p@ZZ5k3E`%uQQuMajz)Lb zAGV_Bcvp8$$e*ZbdR#mSW0EqSZe4+(Wf>sK1oQ@!12j{8j=o=g$pWb4ZDEZIQV>cc zW#tx~qgr$>R+h?$DK%|oBvc#*ZwPgotzd>JGrrNefl^WuaHJ4JPNl@%BWb-S+s=wy zDbC-%rQYm$QD%;CDMrmmPT3H=1Pa)~{*ZTkd~DUOOSdtW8<%Co{%a+oS%U6VqGjT7 zbOfA8hD6hjh=LhUI;XJH>@P%mxMRa~7)NKE-c&`-iA_XX>O)7j#&RBG&BD+$l3=gB z$&%-^f8aH1Z@nvH3_sw?$Q4VT{}|qNAfrE!G|~2A79lwP zMpSQgFxv65^PqpH_b85=U>UIT@zL=4XL{}6Jrx3SFt+a71WlW4t?O%0t?O{t>iu?# zd>n<|+NJn4_UO^%@x^+1Uf-L@Eh0Xv-t2w_v(3Z+K`$R1K?239dx`#yHOt$oy3lp9 zf1T~>5${t?MNX(n;4QYMvIxO|JjS(gi(OH6>ekh)(z~!nwNHIpZBGdt2luol#>CPi zt-<)IQu5fsapceQbNX37f=I({sDdwoEzXyiD2$|*OZyX6F6ibJ9vuu6PL+kZ-0>Jks#45I~+88*~j+<)%bhl@$ z15q8nXX|Vdg}xZ1#qEu4NOJjnCqlw>yL?VXF0$Z`%t!}#ejY|z(TlCdbBm(4i2}#4 z18JJ~vi6XiB0458B@5-O%pIl}D;NccXe9z3l$w3_6-yGzyf*_0(oKTeJ*;R-HvZ~= z3ne4H7Nn9;AiqkV#x#Mo7^OOoWaJ4^IkQp^4+hQ%^_ zn!NGY+&)EIL63yf<;Hta8R3@opNemTIevCmY7_G;aEA^a`{10_@^nwVN46_Nx*Ddf ze{~#woY2Sl?F1&G-U4gIiQx1W;&J@T_a;?iPH#ye9lrw*@i4h!uPO zqfx}U@e>f$QK!R(Ama>5ik0DzZ?{SIeS6^kLmo< z`Yb!pF`jwhX*sr2Mq~@{w4DIAq!`A1!wPFEsytFcV_|74;M~vY$t4`(c%0d-vNcJr ztRNqETwamBQjSK8q{PDMdCmg@;@i67x&7p=I+#CX?8iRQ)hDP(8tB}sEtKL>Q@!Io zsaY&_0etfD>?nzEjN$VV_Lqq2Cw2%Y1efSpOO=(D0;gDXxw=TpYBvxWZFB~)Kf0eD zP4rhB1-C$I#~$SXR~hwVw3gz#4R35g?vbvOhVeSjvF{5z4 zbe+BY6nz?iMC`O5qN%XwH0?b_1&CY9kw>`vkRBlmBy!sfCTc(+FJ=Yhj8{x+%F-YN z^8$6!K6y+ARbWRCUN=wOI4qJ@Rm~(nP?7j{iR}K%ZOblI@R8=MXPYPP2s66r{HDg3 z3xOa<5RduRNx*b6OSU(@x1YA6cWtkUmBOjcnW_{lyhx-6La=W4g};qkkxp1z(S z0DRoqP^2`ju&-&Zy5`tbS5vb{;q=9$z9BJKq$5gSRj>o=;EEd@P*;E@iEG9Y@+-LX zeO8@rL_W+g@?o}W<9_1E!>_|XDko+PulK#h`{CzBA0cij1C~O!xqu((Y97Ozx<=-# z_8*+oI_dTvRZA_6)9s?ka%%?I-e+))XT?*K)KiDXqG!HzP|7w$@;n6bUqzK*j{{OO zR7Ujed(*MUn-3)~e@~!rRzK=pD>%x$!e= zWIBg7!W#Di6g(`p^|K-JdU?g-mI}0|;D&>v&xbuRiWX)U9UhgdWpByeRLM-0VimF*n@#W zrD|XhX0*<0b~?_!Ju16jl#8e4q-$UG1wAqNNkTENvq;MVuD16ETaiH&s}xSJ!tYq? z`I(;Eib#{0Q*mJ4!C6Z5>~IIW@m#RE#!hn4=Ng_0xZiq)!tk{VzOnZnsePDgB>gF2DXz`8d#|?G+-=5^#T9A$0D6 znra1_HStcexXy>Qvu2hs)qNKoe@WW1JHG)i8vPR+XxHAJ7C1z=eU15%X}q z)e>Sc1v$UQ!FptV%1J5UaFay_+7bd(-39_Q@7~yAWJdm!UiXiP9&7KO214esEEQIY zgkBfzBn6&SHiR}twvspZz7-Sw`1{k?V0{>3>gm;1~!Gj~35 z4RXn-!Mg2zeDS>cQZcEf7&5q2n9}knC@2v{ReK>T1k=nv)(#SJr|#$uZs(9v0mEQ6 z1*LGZk7I%f+t7Au_7vYZcqph~Byoz*zavtDDJX#HG4--cva-^gML8G}-a!exT zhT$iBy?D|q^R$Y9eGBp_d;ZI|^=H&SOi-VtWid3t{713-*1yeNYv3D1<2`>*3VxDu zKb2G8Io`G@$;EW%PGX?60{>hd?qg70?8o=T3}-Xsl*c>|xGdS&jq%hMbNb~!z4!MCig zW4Uvx@C;QgxAc#3(oIZy;W}A_k?NVo0yhNh(7M8xBaL#ThEC%Be`TLJUhDO551-%LPY|N$PO5lkLOzFgV^xhD_b?l>_vqv zA5(76WPL7k=T2+l$4C0AYvLU)DK7H23ocdsYRc)Orr7jZuIS2f$Zg~g-~piDx>P&YR8t4P&$<-P@|bx$H)1s>wpM z0Lw!MN;rKV&yV5T7>M}xoeJIk;}GuY3V$6{c8)nqOqCxTN(0K^$XJ|0vsvTU--pu+ zSKZqOo?H&)hDiJ2|TK%1{vv)VPk=UgaujKD^Gyr(F zIsD%6vBzO=EF*=@EMlzr!pVKjlUh;WT>P)rMnnYt3?OguV?!{j5`c;_IJ~<4z@hLt zX2&hbnj*7-@!58pSdFv+ai+@$>88C#^Kq1}3r~U~R;vH}1BJ!vba7RlF#4qd(QV(OWoAxJvw_$vC#zdkL^{C@5fSwu-Hv{GQ58h z`qBz!FqhHpdGrllO%poQY^4iyrh+w{35zMJ* zxv}GDzX#y2`L61O!wTVlnt-m5`}v*{KcRYr3(4B&NAAxE65O{RWaxHLaSW}lKMSMN zck(=j^&*E;G7)S==3}y&c2fDn7}|{u6n`%Q&MQZod*ZmeRWj1&MHyC($kz_yRkQ)kdJWQQ91gH=MM92m@cw603Y* z3Ak*-zWpK092&}~NDRWjJkUUUk+@whT^r8mrNVobbmb}Ke8%J{p;gH2@3$Z|-L;cC zHZsC?Ogmnz-laNXYh=9d?#g3^`V%{H-gp(HmMcW2bsybl z)OVq?#c#XofV7*zgF?{ghtdP*#>6o~bcwg3!Aiug)lBc9@PZ@PsFnp93SOv&INZ2a z+9brJ1{P^*=Gm@{Aau@NQV&NJOX|<)m;i%4An9q-@*4#Fyf^F4CM(#CAC*u#artiAg?*+cte~(i`#Tj3j zr3kM`DSoXJ$h@?H<@0K&J%GRH=omQ|0gGgEFE=w+Jr1<{Y-zVaGZLeJ*Vab-faj4U% zGzmZn$T7;p+AGke>8|t^R6NYE?6Hdr+A;}3)eTnHqSTSv{}1Q+0Spmwe0*D~8*}(v zPfYFlx>O02R-q2xHk42?&`a&*6W|cO@2$SDN%}QZQ(!04Xpqc{>73QYx!CB1`qGx6 zkj*XZ$9>e|q}c95oTR^uhzY(0@$xk&l^8i*-=3>BeBikl*RPz@!30;*AxYFuO9{9} zpkZQKO;co01AY7pOsamwU()%gjIVd{xASCvxF0R?9&4?_+!FyNI_G$4Cc0WvUM*W2tVW zIc`hzXWN@P2K_4pg8sY0Lr<-yx`MAS#H8oldlcc8gRn45lG$qcj>}YD{)}(onL zvu1Gq@XtRX{C-|gU^z0}7G&hE<&#|7;y8GsM7kKJlKYZp|LnD$&W8z0(vA?_^+@22(7SfB5#E>a$_x$3O?w*19s8@7OKRaTd>EAPAHQ=_kLSC?jr zu}Io3@bx>$TS(9Lm@9im<6C7Fj&Gr3LH)*8hvCvL8!gQ@5AVvuZd!E%KNz*WH)*@{ z3mG3!xxhn8c4Wc$9lm_~Ddu!zIqCm~goRXr4nNGM--VJ3%5k~5zV?Afm@!s~*c6fQ zJj|y=#GRc!Y1AXp2VsQJCoRN+6lN^MX8hTdh6|aX)vtnp4joxXTf?(=rMZ(Xt)(?x+>(Duk6R zk`0ehou-RCq@ZLh<#Yf~S+p&*Y(jgUTr>d?SM6u@kIlWE&vvuHo-!TPjf|V!KiN;+ zf7?%%r$`K^%!lep{TgjIZ*chzpCFwPQ_B85M4x?KaTE&xqW*Y7X`6MM??MS4li5Cc z#W=_vl`5%^NB(!n5QMp@!>#ZRF_}DhFl8PN zju8UaQSwg%(&VS0)9-i|iU>g0C;e3+u^Vq$ujN*GhG16J2o+&fnfV}# zxeErpD16E3BzwPz7wy!M4ix_cgZ%L2Ua$x~o%P&}zdk7$^-ko7IYJCs{z|K0V7$D6_TpSr(3BuxRGht72n@5sJ_`c?K-;xk9 zd&w-Kc_7=AaKnVH+5Qd1_4luVNp~xNU|B-3n5!$N+tEVwQa^rsy`$Rr(rIAB$_w?u zMI*LiQ){OUR{klUkB|T+q}E%K1PCbsbVx~6BAM(=SVT&@e-ql((TOQvFfM<9j8sn& zYta>QDZYc`so&u-Jdh52HjN!hNjIaAVI##!;2}bhOm9D^Q_{L2cf>=ovV~DR3KIhU zI1l&iHHQV&c!zM{-QLwNg91pdKAJvwN+5quT>6K zvK;;~cJz5c(~W7!l$XHHnOD!xuM<=2s47oL2UaoIMFB11vve{7u%x}Y<<8~x+dMxO{5%;g3m12PA zTwV{0u7D;X6lBp9x?3z~SUh5q9X{I(v5q3@(b>(ASw2hiPLg=H^InlpzAmsaJH$t_ zi1;%O%3~N<-OeypOS3BqGF)Ozs?v@zvj=Zru#2GP+mdd2`!VZv;?LG1*VkpGi+L+Z z4eV`A7~^>HgZ{{2RIRhc0zdjO`o*1zgprfo1fSG^hwxm{_%WTVko(nDqw8unw?)s$R~&3vV_<;hA54}`Lx`yWFX9tx_ZiW3qM zLNPHq{?Wa^Ml0>B%qS0smX;-edEkK*i<0u5Q!T`rcA1^xq~4Fg03u95_8QCSf|{62(* zh>CKAn(i-_u$?ZC)VDg{8--~O=bqm=gV6kX!aj@X?Q3oI?a1VYHx3QcZ#Vn;oAa+W z>UKsFzYndO#zD>dnEQ_TX_NZyB`{UHc9SJ0FVBGcM0}{7CN+5O8cT@8V1UW;hVnng z{u@Mc&t=XDQYb)xEJ(pA3qG33b{F$42^K z{&YDmrSP6wu~+zIOZZ>jLWTH;`CMQQ6cTG*tj*U*U5DGcmwdivo@98WF!S9YiMyUJ zo0QJfiFc0Dhi8ycu#WLua!<6DeYaMa(2y#Wd=X<|yv8sUY9n`1+}NI(-VmHvD9Zjn zgn%DV=nxk0{v}B&26{f=x*xz1(+qu*!c`F7eat1}6TLf&iPv^&1<65W3-N>%c7 zTT~sf9|WnZgeE7g1?+zqmsu2}ThsBBprlH7ql@8hB$54Q`80j;SYxRZ)e?zBQtRVZ zmU&|^wji6m)dxAl^Jt3JDv|xMufVzpv5o45lO$)g{fzz@LmcE#o1NWvJe~J|@1w2! zQwi-lK?X!el8lSEor__i^A;J!p?Emv*wlia1((&E{MXpp{?5 zl7Pf!Bg}A0s!W*|mg6y!*ouaoL4_lcn0B`B1PJdArXrQo1kqD@gRS$F!zh$U9o^!a zesSMxn!xis9cXoRcfA~TNvcBpEIl>?;9Y!F%phlRXlPo5iSa(QZKJJ3_`xz1$3(}h zGNHImp35%~jeQkZ4cF`@XGBr_`EcQKCfai08Qtw=-C26vHm<;Kv#Br3!+q#vsx}sw z%n)E>FRpg1c{xljB3$_`_8t2kqBq6fzIg+!y}UKZPv?RT04)G%Xt(@@Q60elTWx2I zkp5a=qrxN(DxxY(HVOyoeSkE1Z-W_KJ+4CJ99(^BRr)^46R&#dYT!Je#kk_uhKOAA z`LtbN@t`upCJx!y_l=DEVs0rB1yAM9+OZ@e_Pi7$Bb`8n zF3HR$%E@cf?Yy=*>-q8u@B)a(eflaM_XoSyp2`*-qdPI>aB_e`l)zPJ{u*fE+hT?% zlb6u6V<&VMkt9`&ik8G-86Ls36*64e(V2XzU;+0TpJgaj91W2&N#>=R(QqC@opc=? zZ7H@VEOttuoJ?SB1?2KvhRH1o0JoFtRYGn!e)GQ((QCB*X_jn|6nGfxcSS+yzTyZ! zfjs($A>$4a>8hXDJV<%*UzagGw;r?480TR2IzVmdLnqFyd43 z=g+>=QK!FcZC(@2lK6#J8(?i{JB~t@b2#^c0J-5-Cq zQT$B7n|R?!LRbyfhRlM0Ovh8vPuz`~!~M<`VL?GcFJ$r6+b?)DS*?Sv4KRDCOW*p& znfpKVquI1Qmbx~Z%+WS`$+Wi+4&#i+71cB-w(#6D=!i58B6ihauY7N7s{Cl<>uNwp zJ+~FIsm9Wkco7vm?9gNO!C2esk<7%zT@%=FP*v8~o5W4N3FML_WG|j|>L1fa&59k% z#x{LqfR777ZF#N>#G`|`>_7j7CL&p{BTf|U0bX!7+ZPeADJ$qCkW`2lEj6qhxlm86FJCG9UdOJTxTx-(+;-6Gb*^ zJkz|V2(MZClY|tx-9V<%>etrQf$TF1Ztzk5=RrYNJ~m0EkAuyU!ZOV+M$h8g?XV(b zN8Bm6rN(1tS|d=XWnMMk8N4p^Kw3DaQ;qf`QcEaU3DriB036;2j-92iwqI7@K~8QN zPeD_XAWHIcR;S0b!p~ee6H8ww_6xPhKE7$Sn!ThIpW1-BKOS-`hV;#RFly)r%LqJ@ zxnF{^1)|WP1!T5KKO_7KM`iv1%Z-elZpfy7Rvs6O78MHOEUzP}z&4=F5bsU}S_9qs zJ=hE^0u;(;g2G$M>zrNMV*!457u)FJfH8>sx4N=&w#BY*s;lj-#gMZbrUmDu;?;}m zn@qhg9oR~c@if5mVZUgraoIld519|bSB?F88*P}&ew#V}O9ukO(&mUWGk9l2*9$To zWVWGRYb=|lF5yEoC2v=pNO+4NCQwV1gZzgCGz{FBC1!iFM=!G)#Sc5P#SeY%5OIL! zJ2I1nzPQL&~e>qC`K;28(-nUB1yaYMg$E_Ow6-mGFne1>XchhG;< zzaN}EZ_;Kg3UsQQBc7e3z(NeCbPWaWBu~iXAW^@(BM4>f>Tn)yr7Wx};4QrIM7HPy zKFg=CNsET+>iOS5)F5d|R~N1+hy?D9P~s>xrW%^iU3SdRKVDFzqz=^>XMR(+VzCP( zc)aN%ZSdh0r@UPZLRBX+izHw$XVt)1mZIo&Kch4=Kao$Mi`;eH!hPB)1)1FH3?4OC z?h6|C=hHUzA<>g?e}a3p%evj$c$k#SLmBS!lld1W>DPzb01H+Nd)+eVfbF&TzYJn8 zBIp+CCAI0?0A}s4MNJX~Vwrf)V4KNi6ULJ)?yaxB$y$a%k6HTBtBkjwrv{&y)#>@p zU3&Mvk9XP8mZC27Owz>thK=;G@Xddv)um4M2}}c;9bPH@@6@Ouo^gC5|9Cr zl-am|2%L4}35%^e_QR7LVm_H)N=xHuan zYDRurX_FsaHWaVBi|lV0&(X2LEcU+l8$V59cJ;dUe`pwj08u8LMmy0jDp{36nTcb} za$`?bDB;g#v?cX5sy|G%t6q_n5%<>gMF=`2#w%FBuyA9BXt{ zv}BVh(WJ4ty#n;VEn)ymx?`)-GTA^Ep2?qF+_}FXX2oL3|EbD2e}bBv=Bf0w8=6r^ zhn|!4QyLsnlwCT_@YDGcRYgToVjdjU3^3}Tu8+W>)x5`4-^5h)T4D^ItMudBx$I|_ zHRD?J!3(KjAOO<{5G;)jWZA^z>WYdcuy&q9H`3pgBoAlv!@<1f`))px62QIYL4ldY z=URG_h^6uE5A%A$3T9K>Ln?8fo@1Ce{LM!qdhpbFilipTNM>BF(8)DHK&x+zVN{M&?sdNY=u;UmLmVy!T)DC3_|C zL%Hit0uEW6b$pO_ny=;d`srI!(6VBgpg+t!phFYI*2fwv=VnV#z=A%w)jU0%BH||k z;V2+`uq^GS-TR~ZBX+#mWKpsDBj1N;ZFpUJ#>yV`T5!2aEbn^P{^QGNXR@PY?9ltF z^{%HWZeg_&diJfG=EZ zUeXD!KVdv2A{vB!-&&;t7X<%y_#@m0`D!|xnUrr0V3i1DRff&#+W0ss z&LRaW3&zgXWu*ge*!8YtB{)uNl|CN8O%YU#>uFzXL>+bPQz}5!o0)lQJv8Rc*+E&A zF@%5wq_N;|Gyj;p`sc!NT>}(=gzMr)QI9wsydxIYpT$thrRGs+4HR3g0s@+K;r{+D~PtcS*w}NARr#NmDG7-hI1hL==~c zoOi@5R*+gx%U!_EZ`VG-D1(iXMx(`xoy*?2E=WzURizY^YMs?xJha_eXl{#-W8@5h zSQ<4^vSR1LzB!4I#)xE5gA!aEetw0+W5QK7mW(sTMW?3HR%BD4DsYV==GdqK#nRGb zj>x6qeQGe%p{(M`DTsEezU%J>N@wZ{ovG;6n*=0Q-9G$k_lG$+IB)<{CTBy&>zLbW z%l**=AO4WN(9wVMK|8K5s?hHuNqm19y`QPC7p71dWzG#A6ugGM_B4NF*hXbgo*qSM zJPL$S8omitX-b|oqiNn?!ip{tH($gxjeUSmkk1TfcpaBrmJi2&=uuo0_GhS~CO{gM zsm0kPhqq1Sn@g}Y4W0VeT-Z~@O)u&QuKuoZ5%9Zv>h?v}_x(weF8pX%UUh&wsoG>m zbkgxl4VUBdgHYp*R?V-Lotj^Ll)Jv85L}{`TLg>T@ts3L?S#KW`MMc9>+NJrT(E}6Q_L{S)k-R>8MuF;?jU;D zQRiIQ$LYLOJ+-)=K+oL;>$BR@)08UN!H{X;Qs?&UW}!??=b*>qks26aU7N92>`wrHJvEg9#Z@afb-Bf{>TrY$J+JSKV(7V(D1FzB#N0cp zWkmGv0TDtUknk7P#@1tqyL`KH9-EY@Ibp8K{q#fIG$uVfu+(nhv7c#MW8~{I#&S%* zyL&O}wO;jG&hfR_@>oTQ5C+F#z8qCUaHZ~zy@cFPQoS{*DXW9x0_NnVuD6<}&*hCJ z>xHKUmpix_B%15xeU^g^i(UBgW4b-8f2X4^L;+^Z6}_$3_i@xgF00ZJ=B$#@#G6l+ zvOzd-FG&_$>u5#(JX3_uarYfW2^9W)P9a4d`a zWGB>Xr{J-k^GL~4QRaL`q&IOFA-4j4W7Q|LnW+12V*$oTm<(@MpRpP1krIZ1^b0F( zk7<{u)l?G}7Y8es3ni(4U+xPu=^&OSW%t6`L&#&v%*#i!xeMk95Z z7^)RIK6EdDpwNizK99%wh}L6$f-PYe8|+uKR_c@4XxCjvueI@WxzOb2J0lX|WRvru zqO;(dg7YEb;oIYJso0;m$V9;i;!XQ19(F`+07L{_@n-ao7bEu~KQHi6@5<{_%Omi9 z=5yk(X_vu%p!$W~4h0gLI>C>QG8VmBM8D1&IgFyvNPdf*5w)ElKULdUN0lE#dN!T) zf(2pgx&!!32hpD($OaD0+c*rGKawAPgfk&^E-AvyI}2CuFE0eviucUh!%a&oXfJi` zpdg#ICwdM5A(=QfL~0gb5ZfM-&YC4TmVJv?vfd6Kefk8~Q1y&>PtpR*3wYQEHoSc> zuTb_qHpEZ(G4F<{o)lGn;h9-bAaNAd>Ld@*xpgdYPyXX_uZn$|bV=X5O+U$`EHVh# zb7vm7doJH4VROWB>+-{7BH;8S+59xQvah|Z^&v4iPrtmE%+tfUo;a2eQ8QR zw7is#HxoA4#wfQ)(gteP&ZcVe%lHtVrnN^<_2FIJ>gK-XHcTRgd^z-9~JZ+`|%QJ*@4ksAN zn3p&G?fx7gXas?|isF1Gg)31+CgXjs$@2ZVZp?)hKdBDsDqE0{zT64)BMYSTO4fGK zP{+^cp4!dS^o5%LdP?()2(=LoirlLTlXo&Ihjre@E(^DyuQG}XzU=(^RMZtM`oTkz zl?iI)VL?})-^xW@hhvj)Buz7*MQ-z1d=xzLBBZ2v;FZdGt)Qd0km@OI&)sJ#oH2NP zFfK)1$sPs=!xte!y<#J!aKpq)X@T1iheZwrfP(VRmjn|B zJ}6_#fQdB}SerGqew-d~HBu_0(W|M4Yc>d|bpVaZedRV@sI$+qJ)lt8=C;%otM z+luDSg?NkpCxM*cgh>bsFLAR9AtB0m848dg%wZCYH%4j%gBs|S^tE>9P&zrbR5WC|3 zqt)|8ifGRH6~1$w(@whx0=FMHCiaAUw+e)ve zAq3*mx`!?u&kTMlT(-eP{5mxSxE&d&A5W)!Em%{swzp3@y}z){dhqg{l-zZ822Scn z9$l^G(yl*DQ<9Ql3RQn9j!PobaH0Z>_1jnK} z7Uw0UnTotqqNMX>;LuxeJsGS^ZElpn!j?jm}z*Fo-_y*ReQkh+Gd}jb07^5B3u&Vt_r@b@QS9reDv)eSr9^ zq)CLU776LR6N7s5+piv$YWC>>-M1AWrToO@vjr$7=otF(m){k0cU|#hYInRHh$0MfEQ$r*3axo%w;u(mYXn#5t@U{? zH-46zMdgqlIO5}PLpKY8HyMn({rr1Ebdp#DzSg67>2Y^&OF4(&fp-eycO5(EJ3CU` z+C(-?JRDS}xUmCyfJmeS9VYT!lQLTrE@4UjPmG=I&>R#U8h-W|(iW3;_56whLn@HV zvzqJ@#~UESsLWzq~5N4^psEf@?@OWS;E&QQVyYF473OX+GDGGbT zz0y=}j|@O{;PwZz5(O{EchX1%%p^k*;AlExl7c<<_NH&z8dI$=Tcuk8>EF@Nq0l(1vS(hG=GYC#@9^?=!m ze~mnusAk{c3yt2ju?^jhMa|Jij=xQ+iYX*Bp$41J6D8U+CWq%v1w7!V>ch_K{tjcf z9UULQxYzt9N(2Yg&Svyhw-$$+%cGdZ8ah5b3&HhS&gxFd~yP1S(^O0x0uK9_@_$FUQhM`d|8qK5l zQrAU3Vyu|d{x1B@c!Imm<;B{b_v+t2^QT}o@mxf{eW6h}jDFo#&lvBO&@qeHk#|X?z zpbAn5uI=51To$S+Tsg@urPaho0|lnM=SW^&wXTvJoKY0cVrJZc`Vm5A!m+?N=Pgec z!{;Y$r;;A}uCp3{ylnp+I(XR7&w7!;N|;b1K+<~rY){EG2hCpdEYBOu8rNNt^2qc)UI>0+a&AF zdXhq6LOz4ZKPt`QJH+#UBg&nhTEHh-vYQo?gi++Sehyn2-=EX-&T(`fzxe=`)n2+< zX!uPC`f@u0`1`9gP*8_eN0{|zTcsB)#{RVjjbfV74%?*ah$!Y=6cs~ZQ+cmnzLCXF zijT&q#|?HVA^_>#^ktXxSC=Q~hvRM>P!8^gx)&Z0;>w6n5(32fst{W%geR ze$;b66A+{h-0YiMkXMQ;E61(9m$}(+@R=>pxU}hy!qcuX4rcf8ybRrDjeCQ5M+4=n zmY|pE3a<|TC}g=l?UFjfKac;@b@U54p3%5SNSL&j_HL$?{qoY+&XT2DT@yVzSrT(9 z(2B|NV_k;Dx|@PvBE7jQ{dEN3_RKlkkHTppd>>g3*%vuW95!_i#s+~9ap{`8YM zSFtixS~IhfMv3~Mq}%V!BpTuaM26Kb&@Jt2r$Q;nyZf_SK)+bO#>`q~Mm?bjJ-qCb z1uu^L`_(a~WH-El+cGn|66vy8OkUUJ--7<8gf~(IWw}O1zSr2>6}D~&NJwJF#^fJA z-Z>|Xev`taVTQixs42gxCu(T{KTm+%w*9Dnp7h8bT*+-xfn zFLixm2MXA&)-(!MI#9-+jVcZ3TS>}%g^~UWBP*9Jiy(9H#u!Q32^kf2uFkC%-{r@z ziI{`b9Jt!CIacEsDq1vja@3Mnt6(WXfb_&T=YiEvuWS}q`2YbH?S^u%*5Jnd?K1q` z&(fJZ<;_g>4|SVURkOibue&X>58RD8ZnuLxtshbc{6#~&QuweOgZykl+If3269Lbu zLdRShdnw<)M>IF{n3|<344Q4%S&U1DqIw#s>ADs>xqn)Z| zZ_QjOW{j11N?SB9t)T{6Ncg$s478BJ9UAL$4GPyg>OjL0-*xBl`jf*YD2;L(Tq75; z0W=L#*KJZ?kErkRe({hXc!P+gNLo+ZD&<|u!PM_3ow*d;8+h=;OjDyA)aHc+9VP(~ zfd(k^L2F1#gW%s{KzLt3E#}aWkc07aq{_aWYQ+P*Y&f>@5DNOXLND|z!zG# zxH!n?2~%|Q^0mCahsPif>V~_8DCZZ-wLAe{R#aUnvmF7aH5P=FMTYB&u446jBF7CR z1<49o0Mc7=UkVOx{X;72li*z@n^oH#xNzr1y|{R?w_&;SXMpku*Ixw1wYKxcR7j zJ7FTma<`h{m;&2x1GWra`ue%IQyAE zb-IXHJoS?0xovrr zt^hQ&{=b`v3{%=01xSl!Bn3|~FVYv;ueOd+x4lp<`28fhW3A^#g>C3BvA&MW;o2PI z`;aTtaB)AQKKyGX>r-N*uqfTh!*;PpjIPk0Pia~6HCS`t$a?VVL8M?c%bCs>6_0n; zLGs?IU6TFi-F{33ieEIvwKsXYp_m0>gU$RLdxf}itpy+SNOB=J? z{qaTubUFGb$Q@w>>bzywLpzPaZ^+4kI6(0zAPH2|xA^7S1oLuld_3w|g}OP)V)sju z*$$;`f=gk8b?dTgN-+9kqaf=B3`U52NR&kE0Nw2sq>@lj>!8>Xln$PVH8t@&9kGA> zvt58V;Q-CM(LfKS$t9U)%LzGYnM>0|fScts+sayV6NS9Ya?SfwQB#?yYNr2^bO6jy z?&Sx|nPjAGvY$8!;-BJ_8g=QYHbDs)@9fODD#ldUW7KJ4;qxvw$zB#;B1hwN$q-bP z3TYwql@4DTe`B#i7PJuwBc3wJ9;YrIDML9clF=t90>WR2UOxG9Ev9tEsuUHLE0-nUM2t6i$-3K4{~K&H(%t+n;i9 za!p$A#uMa`!RLB~OfNIMJZPI!4l<}rI2nj02;DXvCqd(0b$Jhng}m<@PngxGI+Vv6 zk+-=WgN#aL+`9^CeAamEA7d$3ir15a(waV^FhA?x$XV;V%>S};;kN!+?NTTtn?Kc^ z=)&yiiq+<(rO9~wJ;MsKP<%D3FTw}RsE1_eCJ46pSDC+w0Em^ABDOmCK7HR);swa# z7YG&VxX3ej8P4G|;&b&*3VNe2!jyd9R-n7pYzoL;Wo+n znGabDb6Ycylz6wM<_Sc$$&?ka%&Xl@JHY>PsxtJieFOo>_(-~@qHy1{5y)j^+yP%v^>sqToMG2q?dvL1x^kI@b28$Qt(Dp z-Kg|$5XXDs(q~2Xl9V5T<6;>xozvBB6q0H>W&W$>G4J8R!%dRYZqH=0)OlDubWAZD~f-KZ`g}+(8_wTccuMPgtlx>P6O-{8aD*S07{L*6v zjkofp*GP{8GmCP;v(j3F?%`@$X{&Br$hqHVRzvGoszs*b)SqXSP6B=MXnNwURU=p} zjwdBPWgI&~UfWAfx%FE|##nit6FP2$gN-k)wD`hirUjYvfK$VjJpi9`maS26knbt4 zgu<)MQgX^L(5m>6Yij^zqq(XK}=i6ew_HB0LPZ^)e-qo;%qPx@@NM-rB zHe??9aD8ZUzS-y1n3ORhqCY5e%2G?c`MnF^;g+icitoTBK z7Fn^T-)p-Wsm-ly`qEuM12?EXzEfOBRWz4w7{OSQX6jZau}V<2JKpNEMgrB4)-AD2 zYbFy8SmwyfM-36~&td3)f6lz2@1F~fZ-eM!q3%khx61qy;V(f;JU(|Un_FAgY~pRl z^i}PFniaZ>GQ^N}_X+eB)5x?j4%13J2Kbm6j`veaj{U}#oKuLADZ?hF`tbGJgk(bo zM%FM-&Qb}tPATOuRdF(AeLKo^+U<`L*KyLz1dgQ6eDxCHMg_1c2_vQquz+2LlM!p5 z*p*^hy|cl?wCR1ko89!sk$=1U-yX+_kfLYDB^)xe2IIR6XmL`}r+yXKSoME>UTFG~ zwFM4B6AI%$Y3}(O&<=ixDX9X5<5O$G4Jec=mKahuKGVjaFjmRYa|BsBrNokkjsx_F ziIgrH4DeID8`Cj=`qMi7AfF@h?04ZnI4ym+e7hg}Vkqcyh0)g2xD8cVVd-dv#R{!< z7}EJ9l-jguCru49;}5WQlH97ajCLhorn`uX?mi=`y3K4A{rdUk|Libis}ZoEcbP9m z@K6R+%n;h|HY4xOdeAb?-413DncL3-6^i4tcS)HN@$(F! zmNwuAvB-%JE#4BxRDLfM&(G6cDbb|R=H~IffY1Mu8DjGKEU!ci{XIwVmVm*8oH$%O zKTzm<{r>W{3~+z->hMNs$VKe^mhQ@HwM3Wis;~ zr{r*bHaA}7o4lTZwgIAS?mnT9{ zzs-99pt5t zC8OfeMhyh(z-;v8IKCy7$#3H7wRM2?1Ld{?l<&;(75EEwu8q>t_9{cOUse8}CwS%z zjRksog3;u>H6zJCUT)&=>4bc1d|vSMj3~ zqUcW@$!>79N`$PCtV1=0tM3L7yCSwqs*$prV#c;!c!;B&IbM0zb1QG0`pD4wIL}+@ zM8VPWkSO`DD*TsB0q7zyGyvEMF!EInwhC29RX%#xNpXT zK?NKpmn33x$Wbc)jo`g?j#r${gEVjAWyO)DTC0B|1tPfa!tUqrRh5mb#s9Yk^gnCpwL*Rp^?~?B-94ZBpCWVc@Zdt+ z2PHeauG0l1AtB`x`t`Ovck|IqoKVgO7^E4Iz>$Yd2((OSHZdhsb2EQm%hub85$xi< zn!zzfxk*D{Tx1?dD$r3lu2I=0=8p@xu|@IL(oL8R7*ma*IBt+1ic=Sv&shnY6l?BTC{x$w zjE$|`BaV5cUNP9j*La9idN%^q8w<###x8zqln$o00FGS^s!$2t{})|f6&2UEZ5s$d zLIiiG&>%sAYjAgW2<{NvEjR@PC%C)2yF+kycXxY>?0w$d`@H*FtB?GUsx{}DL;C2w zPYj@i`o9sC0K$WWhp=|)epCp*t7h4KKcp6r5QlRBvDfPrNop1PB}slrfCy3V z&U50duxff4f41$m{rVudfiWPp+8TCBHw@9K5))qLa_c(P;@KnhN?=HQINs7~?>@Ae zCX*Dw$wW*YUMZ4faiFa>bzdX~EfD8f{znEud~S7`xhixg?CXE65!5SSjUcR$UOT}K z#87t}%vK~3G8vCSSvKEhI~$yHT(J86DRbU=jXg2v8$twLr3>I7BdS@jcEfTeyYLKG z^MBioVCGFSU6+;!mQqZ_;zj#OF!4Ll=CvW%EeyHfd38{b-qv4S%b7ItXlTwur2h1~ zesqAyC%F$}^4Ghl3bWnW69)fSBVZwT2(_R6aLLzKSs(*ilQ~^Jf8UWd!+j>gn3wa4BA1gmG|TM5R5+N5EKrcE2%ahtJghd47L^|{ zlD2=GIPC4S$WYDajFOcLe|mf-T<^8m)VCvH$@96(SCnv;gMq+VSZparX z7D*8r>N1%xriKiZnk3$_5i#PJEq}o}F9iwkAwYnY7Jlnnf#Ipw+t=5|dA;D1Ap~a2 zz~*-6u&7;y{&W6$Jdvy>Fc8i7Iq1_v0)V>3DNi_Gyx;cf;kB`g=}tnj&E(pS?5~>> z+@G{-5zZQk$*bQ&5?pa>%*4X>w1P#(QUdtv7Z^T?#uhjunMJ^D%X4lfP;+2v*}c~Q z|6Ni2ClruF)mBnV4}bFvU6=B>L*}~OKsJ;wFT0AOko(2>Qu0EG@$p$C; zZ@inyroKPl8A3;Ht#S#LUbv-%Em?ni@P*bAqFYqOLpBSyHdPK$ z$$avQO<(GL4h_oaBB~&ZEYlBWVawXb;_WFeiT#=9>J}v$ldqRUk|!}BQ^1|fRUznAb~u@H|DDOx z!)@(Odhp-tAjvagtL*sko%2dd@&>)m}tz(Yv!N>Z2LJulPrQiDSY zLnb1z;7W1Bk>dzy?1_ zd7An?JmWwQ!WsQwXVj^&U#(b_s-Q@vEK~gVsOm>EDaa52=nGo(sANG7^3=Pv-_|dep zZ6Y;Cd`-h_M(LGM`Q7w73U_9CTzNVk_n|!p^joiai-z~lKpq&aGF5_tfr05SWeF1{ zEj6EK%FNDQ>;F_z_)Tht6D}%*M^F%&$zmZuv&nfC02C%enJt$>n%(Y+h=?GI)M{wr zd8bH$TEiIfsznJU()Er}0e8Fkb_=^#ZT0{G3oHLxAd(n(G@eO9ZFmMEGztyDzAvIH zWhl7BZao zN$!m1uW+(F^0fEr&WoTqZdf6lc-S0I2)LeZ^tkci7Ow>$2V!Zj1!{JX;ZQ`hfiaR~ z63`Xy-)Am-YPmmRCrd#Df>$dc@B&y3XDx?qZ1nxCy+3M*h>zb5;WZ$#+neb7lai(V zV3mnNl8rNe6!~FNQ{<_L6qlP!Mk{CJyi%!%%nTdU9czoiBow7!-Avb5)<@6M8!u1<(E~X0;Y|@3Kp=bw_N0CY zm=?^WNP2b_j9{D;n>QmAqF|xiq0ib<297F&R<`=Ij8?fo4VwLdS%Fs;I|W}pPb@2H zVWmlSg0nBFxnOj>bA40L+*x>^A_c8~W|gpVb#9i~XQ+2xdojwGNJx-CN5$iWkSmk% zgNb-rUgaoda~0k^oJRz9=9>3@iQN3qltox$ny|_wx)WzV5=j0mURYDhybqtId!*!3 z8?J$S0={zD8%^e;KAnO+X8yD+`8W_R+2Nd4rkY?`s}Pn95Rr^6{~KAM$DVviY~!8) zX+gm;I~<1mg)x`R`rfbAm~sMhiOij|9tT z@+z{m*_KAz=1ysIywv>M)&~SzU+0t+4hn`pP@odm4u9Qo^!&o<(m(k1s7ycD{-km?A8v%SWs`~^DtM; z?Ym2`ml5#>OUP>7jhRICqquCPnmU>(L(9|r*HS7)#^Y|8JoFUC(I$vYbX@Y&*Sc*0d{u($WUIbFERP^aE4as zS55Q#F)>2WOzAWW#CiA_GKZxlw6pl`gFW+c;xDl;t zCmmwhW{h4XzaBiZu2vvxbt;JHCn7))wz(&gsOnue;=Vc{Ar?2r&i^qLkQt^R<**4V z5@!vtjY-`I$~P0K!zw`e_YT~!zf{8R(2u>(zyN4%$>q2!f>;QrL_csolzD0A57K|4 z1z8;@7lVi~Dg=drTB)45#ujhN6xmmqPodI_6vL7JGmjRvaBCQfXw>z9BQU;N5OK`9 zfm?rk%)b&<4T*kzBATiUxK+Y02D_U!$#`$~oqNQ8D)E=~;CTgf=_s$rMzwwrBpN?& zb3a=zG@bQ-qO;qR|5GWwf-{0l?>4Fu3j9p8l`9_43u16R8w7NkvNUU9)R4Qgf^*^2 zYXP`=vf}Kz;GjfER%c%_YQa3x?rfT|OEXhJ$4I7X>O3kqDSQK_$ml%Nv*15v+J9PF zNFSheY-Is+aR*6p-9iz8X48OCfpViGowtwAUq@27m7FeyCLM>?vE}Gd2u+MQLYzkKLS)A9HH%@M1LmpxxYG*hZIj(I(w^;xw7-OoQf4stdSWz z^jDRH`s0IN>5I$}g>nja1q`;r$yvwbnVE$8N?pqhDMVxwV=^Y}CPyXgiYd$PruFDa zyW*xoGn1}5_Lh(M!~+GAiuCfQ>MB?a!4j1JhD|(EkW<4qw#k0uFW@toVF29v_1e4E z*B=RJXoml!DIUme?{-G0h|tQ<8L0S(`T*>-%bPzTsg@qCFv2I@i5=mTxghH)Kam;Q zODyqSRUW?~88;fom^M0-S;jB38Dw`HIRV%A`LYr>X@aEs(biC*se+Y7iFg9#Eb-r4 z;|XldE0(}!4qw1)7E?q-BrF)0?RzU=)bUrrrjY2#dA@CSDExlyfK}zl1fGXz$8iH2 zF#^SA9!ezIk z+&EVGM^W*2CrQBsnxPM-`S-T>h4453uA|(AJ-FMUWz0nq=BvJ!A^K-FbyAII|0t-dv}qiR?i3ivr97!* z_b>d}^Zj}Dl4ew|vd5IQ>%UGk32>s_o#tGZcY&^U^@W@vDEMRj08prT4kxj*|7mW8 zd0=OBWlc2dxaIkV>r_<->Bb@u1I$jl`39rrj7NW&HT4AW&Xp>d?FIE__<$ybpoo#9 zSTfV=mGLE1r^6`~zl<+{LJmMhMPIj#Ix{QccYC>s^O6<`l zN8;b#g}wqYcuiR1JI6!@e4*6P2xM`78=f$l`AH=C@jig!r{ZKj1>DXr(fJ_~3WnfH z>InaXD_za5%RK9$?~#==zpFEgpn$4qm&@)spLViGtQNE@6CqZ z-Gsmck=MB>!K3us=DCyivhJ+LcSR~N5^B}F-*dzeX9+eM?_{M7cYCf@c}!yDaHcX* zi9~l3<`tDYwgOA*?gvVK?dO`5)s7+|gr!9=P8}S! zO{aKyYinOd6Hk0@O|PQ8lm^wyd1A@g^U-;&G`=&T6nI<&_)jUV=>owg-Po2aVd{>n zevX%uN=bj3J1>#e09FIDE>!VvDv8wP@*j)+UQ1Z;R=6D#?VI63DR75kP1+2qp;aII||i!gI-Y) zH-lKad!=S=`)oSOTK;{zDNZ9gSM{juq@D_q3%~G}mxJKDZoT=S?K#i1j8p%#nw217 ziy;{fT`#5mZ~c_&97g4s$_Jo_bYFb9`K1(ZO}S>eTaugPvtgLps;!S03>NVgRmNAk z;w^bja8x&f3b(ciz&JFt_3CiG$pEkK^%@|x^@HAvCH;ekm%;_a5v2#Y@_)-ysUZEz z$0yUI`RO2k8isY!JKT-woeC>qFRL1^;UxC&%GwNhTw>+G1ie7cDKk)6ks+lB?qUQv zk777m%o4E-#puv4%CWC=bx@%BFqA%_H{-(-cJFdG0LPMV$t<37E*{dS3@j*P5IG4K zQ+oLd1u=Lj3m+*!m|+iG+)!h3F=u@#-oVutquaI#lD|y^wn-9m_nLm;(P$I zzY`(?uLx9(w&~RHY=xn9y1i+pywZS$EVt0&FCUYn zIG?0&lQ>T0?HOg!`iERrVGFljJ=DF@q~|88*HwhpAdjLc^<6_bthZ zp(U{Dwj?%%&_oKPkA@LGe4(<%o7uh7`){EJEHV`&lD^df!pbb|6Ka(D9P|UCTtKy` zNAn@k#y0FnK9yG#$%sz&^iBqy+pcTpvLhyzKcrz8Zfgv!qPFS*Iml^J&l{>gg=esE zQrXHBG@Q%Hdq8k~UpU@lebQ>-T6IM%o)E8%&f0U!*HGYCj`VXw|J^%~G$K?z5#g@& zcO_G;&9kV-i1IP;$^a_9?%=OzP24B*4erm5>vEe!q>h*d?PPGj9*$lwR1tKYnX};0 z=-#U21TvH{W=BlB4ZCQUr^sql2A0Q&Ff~Ju_tm>Ov5Bd)I^*Q;-^3S!_hqTe1nN@PZSeur@bZBd{X)fRDpo>KR zn)L3Dvq+@e(BL^HA+eHpf}-DLddv%MF0>a7){)6{x`oA^b44FB1~xCn=HT|xc~@?l zJ75DFTm7HdIYMUF4Ouj~>rWIbzsueVxzAT{s%k2iH&qs6yMAP&AQUkmA+$z9dw2Kh zqdmoqX1vEp^iMIuJeyh=`I7sq!;Wx5zKbOg{b#3Fu8)_~qd2N!9&(X*7bwV5z?*;lA z86p&O5Lo){I23e=DV(|n_p7a#y06$GVcjM+V1rV66ncljnXvl$28tJ=p?b)rQ#{E7 z67vkLF#W3|$O7w@NVX)H*9_)ZKVHu{OmHyXZ|7k2YjC+@;Tc~_qi=VUR#$8u_Gkxp z@?y{z8bwM%h&qlYU(k57&kffYgG z53q(8A!3|OEAIlS@!PhIk4_9JyIsBPhO+C3E)+SrN*mHA4dNO;( zNQl;?mDiwVza=Ww3KT}~Y03<`C!R5k3ViKo_sx6VX5nH#_*dwLJdJfGbg`mkf$!Xe zhCi)Vd&Baw_iAjEnSoqj1GPUimu-E&4i%#e98|K~l}e$;{%rp_;P>$tmsFk;G7Hrn zVC8{fMRfVNs$q$}L)3(kPUnt#xD?ThPgP=y0>eQ|#+*@N$=+&)}?gPDslP+d8Yw<(6<+XGp{u(ep zuZJ|=eAWBJny1wLeh=g?rp!DWe&)Zc75Kf(MOTQ%K4h)Tzh7Q$mDvBBIhJ~ua3aCL3I~Bm6+Y45cgp`E%d9;c-=kfmSL}mH zhV_PDZLRqy<=DcUbkzBpEmp>3_9amgy^$rp+FG6P`lX>a_(*tOC-j5vYj*K4b;|~s z$&@$ds$^0)J(&A=m^esbcUAhx`c)&5&cXX0H3yy6mx;>X&dI+A)u;Zx*Rbz4#$mU{ zYO9(AF0<7tst?Fjs<1Mc&9UC`tv^OZ@aAie;ue_c9cO~Mmy<1Z+9s@;P&wM0&0WmJ zq}iWWG8Y$W%3$YeUj*7-q~R8ay9l^R2%=mPMkw?+oP1G+|C!d zr0HQd5gDB#PirDvvh$!+9!+{z5Sk}`;{KBjttfkRxDt3GVNjbn}{N`{%Z+mpwLDWfC{{$aBop6w&!WoA!2N??&nIQP) zT(YOKj_;v23y~~)Y(SZQ3B9{Bo6~!MJdhK_8*UslbG@-(5&g0I0 zbiuW~?xptOr{gKTB~wcs!AQg1U*jqxPjO!^Xcp#cG@9?BlOCS>W z<*t{48Z34LnM6}5sBrF{7{qTopc)OpJq0!>*@G_7;EN%tZfacrMJJXlfSvTZ$vJw~ zd%4iqzt~OvIVLSu74q5)LTp8HJ#@83cQ6)@p@K_UrhS&vSec8vO^@dmVX@ZAzPLXt zc*?}Heuo#Sj+3W^HRA`0dj#u4@<9cU zIA^~NhIx98jQ#&YA<3pGPD|9Kl6G>gV19pgb0j zG3id)nVrhnDqYd1=~B6lFtfH*2Kz=wAkC`j#4)($TqWRd< zTi)pqoB^l_vqYkfYAHIqH7I@ua2vxwb!)xAw(-?sdF<%jlsEAElf46pk2|OajF}uP5-nCZA9|)gQn}jiHK3-GXBSKwsrO?%x|1JNpTO&DyYO z2;W*CA{d+c`!UJ;sTh#TS0~6Q{3ZjU-RS1tQ*pXfne%{-(j0R}SJztN(s+R=@e7CG z0q}~GeCG52FxejYvkd^{d7_C|l1aXe&{MvLLevJ$2nEhpA7*Uael6`W#_u9?SU`8p z{HPGk10_gm(Nm0PPB)aS@GUKlb<1E72ifOM!E@LR*I_U-F}V(T`Xcw(Ry!7_6C+!$ z2>$()IIllDr>(G!!ntI%q8M+JZIF0YPNPpZr~g(FIx@J7leTr28k>$g&vTc;NsX?j z2M%iba*%7R*uZfs%uJR3{nf{ttIxU<%}K(Nt*X;smf1SZJfo7&>X9NA7_bi%Z*k?$ z2dF-Sr}e#z9#U82F7{ZUU*F|vBuh~`?7zx$;p$?eh3LcT{w)(gVPr2l#(2@|##=AA zY?sjPW06+)c4D|)xBRA0{HD`CcOpn!5PN){*#dub_MqB zPqa%sU~93Y%0PZX3nOOjeOXXFf3zI~e#g2w`g&Gk!gl_#C|Xcl;p)yWLqKr?e#&`cZc&3Z8hBfQQq#^ z)oOK#JtOT?K6$;OoVo7bQY;e zbYo{Z@2Xx4iL}!-p8B)fehKIr+h}5`F(A}*y%HF7@pwO+G#f0SXZNvgkABw@isopk z=2h(Y>ErZK&Xp19ki+K-Ji6{sx9uL=d{Cx}p56R}s-W=Oz@VN{9~f)KOLDG+mF8T% z@XAL5SukD~uYv{Fak>l5`+`?O@=PTI-m^DieayqO(=zLGAbL@M!PF}c$>Yu3DLF;n+S;nE5qLAG7q zmW|qZX0xxY@56O7S^9Kiy(nv@AlBXdw1V1r-JXF#I_JwyPG_)%eiTmw^ZA_1QpRv5 zYW7C|9pxo$Vox5w1ytNz>&fWk%+AXafe$7U!ATflA9e7bbVlYrO9xjC(;O zrYJt|@mI)zhmjjq*mdEJ;4JGjRU8v-taXhY_=L)NL-G{4GmI6m`uG;HaX}%;@dJ~` zlO-8R!~jj;iD)cq8N-)l6%9h_xgEDiv-?CPvvzm()ePW{7A|L9*U6%djc*)xI~idv z<%R2qHOqpBb9*f|fA9u!AxQd)o~N0C#c$7qT6Z(k4o%$H z3{3}M0h_)1+U+@V-yJPK9|XM7w{Fm(-3e4A{6k^9xk``i$i|5=+EHrk(gSGC9CuvM zc3~G57)XH@vC$V0?ekD_&Acl+ON(zz88M7cpOM3U9Pni!GrPxgkz}Qq46_pY zj~75nNEAkiRG7D95;`O>s})pN#~B;v^e}${mVoPu&H_)dTeQ{?Kk$%YU2VBY4lp>rRCLSL@L< zG53gw}Q<(3c=1?r^zXPED)djx%ZQ;*_ix=(Mt3tsQjQw_!o}gLquzY2Df9~ zp^6on?}x>^Tza)lsO}5%u##*tox2^c(rdSmfdPXRzMTW9N#Urrg!bv*fD}!#z8PG# z{-Zm?&+cg?x968Mi9uy!DJGk`4INEWF6-VjW%4wfN8^_f&hlZuMXLe7F)kRbqLad% z@LxG-lAVV#x62?tEFrz41QFo_29^B1@LFxo?8Q-1DSX+umK7P~=|_9WEsKUpl6H^K zZ&ll}iMT2R#?O0c60%sUUmCI^5+HDHH6s}UC8;2(L zC{nx)cOPx8cgURidCO$38?UY9t>u>NmSHdR89C^-+vx>`|9Q??3{ozTaWve0AUot>rvf((rcjcdkwpY8f zm%U{c0#-;d5|>i-9DUue`P{mtOr8Q;GH|x4?4)?N9&t+iyR1;EYp=+BT1z=2TxTQU zaO{cq9j)%iU#akV6K5p?Hcw%-6QFFkWwk@A!84!KK61` zD{p0?@?!iAI+{vkAm4Jsr>r$))g^|FVwu^~_LA%VtFXs=C@)-gd+m+hh%nl6?ml2b z@Viupj|>bDQXr64#+Wo;gI0u~CireTGil4xETl#fOCpcnde$}r zk)5lp;b~4}NBTNvK2)TUwpS3GIOXjLnIRCz@tIP`Jgn?coJSC z{8G9q9RJ4nn6VEZ=tg zBGI;3PXBCtqJI<;*=0J(=TcQ^S*JEg5${JoA=z=od9)>CG!1%=|8}Grjk!Hm-Jf|3 z<0cjN83GW=Mz8O_T-ZyN;yU*Cg!d&Zt(Za$ugo+1r@M3rmF5Lxabjw2N+2lirEKwy z7FO*`DEIw5c_>#eF{B~RE)^`&K#L5ot#&ro$-VoUtXb!JA)5H4D9{XdlGmx8zxw@ zw5XH)dz!yT^4#x_P9mwzsZS0lq$-AlwtTdGa%lZDlEhMH3IBwI@3X&rF*n$wk;Q*n zD1M#%6+ojvB|LTyz71h?_()j>Bwd?_dsS&lC{n@w^AkVUNhaKaLS4?Tj`I$1KkeW+ zPE!(OEzNB;am(>de2OVm^lUm0KU_abf`L3B(t>PxIDGOeNrR6AsN*+w9%f3OO>+x|MV+##aW;0l~Wr4a8 zhEyEPJ9bkQqbE{Z=j0%+$%tEH-+&Ri_hT=Rq%Ey&#wFd6h^J4mcP@+MWYmy1A{ex>ztOvA-pC z)YuC4i+2L9m*YUQy{CZihz3lubwsV6w|&5CA8W38f~~$J3La)%40&*MYarXg=Rcl4 zST@YQKgW5la{c1`L(4C>`w}rd`9unD!K>z(4qgwf){qfGE;-PyGdESZ!pY7xy=Ots z+F|YaM_9LX%Vd~u7`rBp_xGD!4GP01~$Q5s{TjZ1A+I!!paA}E@Xd`Q4u|9KGEVyg<#!mLTmKoo8lYr7&aS9gb&!taQ@lHrAM704 zK2p4?v5n=|H9Lb855n66hJb@~ck9u4$VV!~PGaW+yL3nRqAVuCB4J12AtKjH2S_Dn zrd^Hwq~VOVjl}aP^RaP#!tM@4Y%ZoelD39Y#T!r|K??XeAcqb5s;C}z+Hx05{!gRh zbs>@689sN8elx0F$%{a23&ufyg9OX>a8Y;Ta4n;m-gd^yP`c1`N0o9_5!s!pN7&we z#yTEU>6JEqva)F*l1*v>1jscc{F5`oMeo?(T5fOO=EXZ1o}kLtls*{X((;jkv@0}> zCvm>Z;g|d_G2nW}#S>a{jSa}e;&{H_w;9?wEtfs$^uuM(uOIkz_q>>9V03wj=Ju;y zJz^7nOWUcq@gcH_%zlIzbhr$_%X_7tXlZPosY;-u!&k7fL_Lr8HJ&zUJ=*pr3vihm z?Dl``j@uS7JBYrhzX`k=Unu3Y$Xns9x5Nlx_fK_LdU`?g>#&9M_Z;aqkBvJVxoQSM z=Q{{XPOxF5w3n*hilyjz0*%<@DKTf&b?;rua&LSFdSEMD6M3i%NX%Q~zq|<+_B`lQ zXq2H6PX4t$C)g>W{8?$ph@9#n%4<#id=9E)pk4S*UFFC{%@}vkJV$w2tV!&&|Ht&H ze9PL^9R9IeBV-4A%;6B^>uzqaR&pb035!evu0e?lURMeYf**Ua(ZQx`9!r3qAm@bH z*W*oh%^xIXWDFx7{yi-ket}x`CRaG2ZC$^{4qK~p3ndCl!JSxN z0FT>h4j6KI+@057Hj`6) zc1^UuaF4-Rs>bq^_WNPpr#`w(#}?errRhdM?vea`f>R zEL?WTV1u5$n$FYu3rPG|fdq^uyK@(+=4=}EsR!yS<>R2jnQP$*Qy;Re;;w0UGJ;}5 zh^CrjKMLAG#ew?d_V+SbmxwzPw0OsErSAe;4rNwClEQoOgddJl94$pToY7Y#bCOMr zlg_qtj?uIoW8u}S`dfKbcIj^@Tp#ClT<;GjED1DSw&v6a<|Nk_>y|3AlCkD#m2hKS zNJ62%2p!xMCo5{XhfcZKc*xHW;5f4EjCCx!m@k%)Yn~Do1?P0xqn!PkXgkEDb=4j%Da)UUo&k zUUW$L!&>r$dP{WDOAOZD8jP>6wO;RLvEq?{F#r@h!~|k3=+vrEvtp$pKi4E2uHQh- z?i#b$Uyw@+%M+gky z(ri%+QJ`4lZAROQnMA@}RIYN}mSOTEvA$VfQF!sk5MtVu7^S9gPnsLdpa-@nPtklS zVadpv*Lc-N-D-L;O?#0 zewu*mD&O7GiuhqwA!w4lvoo3ey!-ZkxHHYCC+fIf&Ud;X_^#^wE;;!Ok*3xodgVbrR=J zPYW^9sC!b@$71!PZ%U1#4%7)sY`VyB4}a6`f&^kKhvQB}V!r}^B<~vm&A>Vn_!b!v zg8+t@;+_mQ7)6YqL(g}u-!of|J1vW^G!XkX$!jZzBr@Li0JQsHz=Ly|@ON>NDlNN9 z_22JL5YbBVO25`baC(=`KbgruyRvl6x?mai`lz#aKL{x$$aGPI_FWD5WDub}A5;0# zHoM{LZ5M;R_EnS=t_!%?9(@kQfj(ayPH+#m#-dH6IBkm5RPfoAao z|1Ft(sb~SA8<^+A6pedF!F??{v|`+j>~>_cGX2E(qE<}()#tNsv8-1eXCBT=anr{# zA%_}>?aJ|sgVO*l)3r=KX^_Eicm?1X{=hjrS+E^R1MU<0%O&L>&+>j3PeoR)eqvcuAW{t6<(~Xkv z`?n$juijp)jb;ex0aCQ#&z22`On+o6fQ&loRcv*16vqBv<}!cLt5F12&OMsFhKfaB zXD|4LP0bT(%x+>AEq-8##H%s`&)j>5tP=uU9p(WtEc5Yqoe;C>mAWg)UehUASeiVZmHTu>hl2eV(Y zf9Sm1eU#7&L_Rrt4vQ6Q+2wx5rW-G6iufVfk#fgGi{7w34(peBE;8?b2Xvw-pLIK7 zdolp(EJ~PCBj#-b_`1jYuV*D&<8|)k zWVKQZ7wp^R6%}-jC)&SQtlLlB);a@gkLvgTMy8}?A=T#;WpxXs{)f#;LWW?odU<1W zUmwYQ(Sdf!zEs%lY)j%?d?j|*QNeQSe`dQHbf3~La(TMliYfBsdL_Dh|5J)9@N$2e zy~ySIC^RSe>==F)^W7h-g?}p(fJ})h2#6E>(k>-BJ}DJv0NU{io5KHv!ZHg5(B>3pX-YkiauFZ_ z_tFXb9Lc1hIY5OZjL)cg3HtyTzB&H}F4BNe6F$n zM9RfHo#HRF4A6>f5QS%}l(waX``?gC{{$q|0U+H-TtP5iC~!-}+#d~zaH=@e0+C(( zyD~bso6lZNE<^az0LT7Hhjuri1QO9k*#?N z0gJGbH0~+{@`&p#XJY@pJND*5uPeKa`0oWk0k;CEFZAu^<2slddilF5he4qNqu27| zk;Bcg$8&H=^IZ@_WKee(78gJ(eR`}tte*c3JZw(^;K{OgW6rD|+=G7?`X4zD2K#Qw zOnBpaK(6ZZAA@fb~)#68t?FNb{j=BOX9Vhy5xV9npW_5z4$M$7`z zCS#d_+_mEPkDfe>^>$UUiwzDr;Ttg9!8FdW2Xj?oWCt!8Ad{)WymPECh5p4!G^5rp z{C_BDuah|)XMzPY_RqtHUK!zb13oYx1?}yt<+gzbUQFQXv>uQP(yEb~p?2&R9X ziPjo8pkF^cmoqpP48ib9alh*;L&nE1xfZpd|1Wjze}a*hE>dc^ytwp}vN@poQ!f7& zxb8e(ZBAJ6N3Ha)|NXDim&yp}m!p3q;8)I;>%(`4;qE(9nnoidBO~JB9sLn#@IY>g z00PEP7f>{r%G|%yB@oMi;H%GJSfI|yW@`|0!R>l6JOejdD&@m$YsCK+iGd#=Dh6!4 zao?>~gAkq*R3fvbJxAl^lp3AEAjaQPi2wKf2lm7-b4>btl?k}S?Zb;flOqNuX5h4z zM^Z_oT>v0q0ZGfeoRIY1vAQczCTBYk2(;o2 zPh)lV|69kgmvvON;hLghXU7J%t^0Jn2X`dd^-Kh)F8@+3{BNsC^|G3N$sendSgeVf zZ`R;@GG5euKm@e^E4cCqAg_#72a$^31Bc*$KGL_qA1XYtl2^b0?_#^{2iXx`0ei{| z{GF3@AqcxXWaH`nxQDjoVa@U6H{ZwAt9i@6q)FjC`Utr|X4U^b!1XWnFOX8>fE{@Y ztsb#u2;8WF`bm*RgI`F9@9aerC!9(TTYD{>AM!Ly_U~ZE(}fJ1e?v3FCxCDR6l{Og zvdYRBOYZmPzP`RQDt{;$e{Ef>6LGk^8Gf~yWOsFxR=Dw{YK}2Vz5TIwZotX;77TlS z5CFefPD6Z)w%KXZkwCy~Ye^Feq+Ll!;;?x&?=csMe}yZM-JI(!Q(;ezrT+U>5f-!H zx1Z3tas_MG>74E-ufhzbdA@DD>2q@GNHxM8`{E!6@CYBR7{ABAXrSS&jMWH|6~Y3c&d9>u%&;|+9-st=#QR&k z`$IA5;2E-?J}eU5W%c+;kyTFc_YJCPp*HR=oWbVmz zDe*ZkOjH8UEvGs+ZtH1^gSMqK6q}$;pq5Tfc%nJFU{6nfwZ&o(P|~q$zJY;xF|)~4 zQ+kBx`Nuho=1Dz2aR{&4Qn~0OAu?9_O7z=sBY394xqedxiInQ09;xwLENF!=yqUT$ zK$z0>h`46+6NaM+16nSWePQx336Vs64MNX$SnK;Y0mAKsnM-eWY$h)qb@wV*8g=N9 zy03al*=YqHdHm$Nx}%@YTSn7I5Z*m&JD_4a_4xzL*o{HTECFZd`X%(;i`{X_XF%86 zfvTjw`bQN<;MEUW(VXk%jHEmlFSSHs8G)LJ3`pOy<-fx82a|Rh^-LAtzBS&S6|fhA z!~+uQZF&_6{~*=EWN5_)4zqkoD$Y$}ez15c^a4J9p_-r2T{Y#I<`yZ6YzeD{#|=f} z_vfr5v+l>3R^!I=1JL24z?a}Vr6Efg6^pQ+(sgNt%6xzT$;XBIdWd=XU7y}Rj1rc~ z5PX9Z@J`*BcXd{fXKLOeZzt+VI+`HV_S2t8IV~aLbq~^w7akNl9-qV27F!FL#U<4k zZrw0>y<4^)$!S~<0j03v-m;$qR1}@zU#yZk6$V2u!V=Gx8+PIU$q6TRkaiiN{&_sr zCHJDwkTI=fa3gJI6`T;n_i=}QukA~5l$)XKOEg3Brm|-{47kIv$%!@x@+b|a@`HS6 zHse@{^P*OUl`2v^o-kibwCW20sT3NGdO!ed^J%A7`-{?83IP~WrLesJ?hWM{)k;I< zHmWLUqrzZC7XeNNYzID>b3U;xHR@P(rZ1-|q=IAD2mD*D8_(BdKXK!M7biGU*f!&$ zt#P=`9x>;^nQ8V-(!bau`~uLZe-q}j?#XZbbQHw0KUe@S!h{hrh>2WZVx_w6R~yad z8btUE*qD6Nb?BEom~ny40nvVfMMT%|9hSMZn~y|oqLdP1>%_(KhS91wi(3!O!Dm}$-RGp2HEO7zuZ#Ai1Yi(^M#_pIc`a(b3a&ZAvA(c9nn`z zdG5U&kB^D1JN#D3XK%(!OYeC&wyO2JrX}!$(XQBuKqbo{oaFhDdv%5On=|XGvGTW# zzLZ16_6G=#KwoqUkWg873rd=?JnqI*{y8lNtyDmjj|0~uAXVUpc8N2NS8`?mvkeIba!`$ba&VHL!alI_q^YqUKhx{_dR=N&CHrL)>W&| z4`3y#*gve6n4gNrK6yS|BR%DbY$A%EVg!?%w0#|?RF)HjWT#U%wZ1tD`pV79lEWSA z$094-l3=Vhc-Q*ngU;Y4_keS`RBG-UPD5X|9Z6w(8&>=QOjo-dmenKWz62QlaT$7~ z0Jv^-Qw5|8(tJQImR5z1R;DW17gGFlb-s0!M!rN$a5#MEzZW1m`UHKXbxmz*iN@q+ zX^tpVlnw4;yJ8&PEWlk$>5;7Vohg0Z-;IoT!Ep5F_c0?wZA6gc@}ZAM*m zDE*`G(Ss+}$s}=1idRkVFh3}MUp>u-oVl(F>W&he6q;T#JfA+l+s$~ty2oUR`LaM# zvIh8uYY-WNd~#8<6y2C{9*VlN7hpgh zVE62)IPH9weg}zjTiEgy77nb^<(bwFd7v0)x)3kJ-G>Rk)*ChHP6p-X%R&(}q!hrD z6TbdTF>B|k%wWKxGb@sqdeWB>X=Zfn2o|rdHhbPg|4l}1S*>?>TYQ`155hjX5cHlP zU^a%nIlX{eGh33Mj#kXGUNva3;@#@>a>7ns`omgn zV;DO5@*r}A$A>+bcCt}gS7+DDIW+BKuIIrNj%tG(xdo-dp4pXH99~4hdb)S`LOLgI z;4luK8bAKNt3R9hUiF{^HbU$;g77)DbOw&&%E@etKwkZpvfMn*CwUa!GNMvRJQY?7 z{L3p(?bR?WdLJRI^z1VNp4Aecd&G54wu5NyFg;D?`4`&VaFvt0iwC7-E}$9~>&@@g z_OFIo3@`<|8HAzxPd&$1;0?v62sB6%04Sb1P!L+P_B71O4mDozTs4UbRw>YXREe%8h6pCn%J=qKXdL2$nMx`{;X?LYh+NxkR5kaI_+3ut*0PDx@k@HZl&rG*24A2iH{fExnZ zn_iV?g723|(oE-=2q3vma|PY_dIe_YqkSVt)DY@d4kUaXAinS zMbNcqgi}r4P0ZWgH?)B+m!;zLMdjnFc^yis57t-uPdAG7olEEiSKZM{jhd(DUK9BZ z@(bZjLS-Udl(h_FvvI6Z22#&fcAn>5KMTxyX(%uK4$)vk zHV9QLH_=D$%EAbSr{qeU3~#gNZ)I)m3PX+~Uwv7n_K=_n$s{OXB``A3(7|D-KPo6tDWVZk zZf}4AoJ4|;;36RCv{{5%nD{y&BE?crML$rbDE+`EHw4U1(wkg4QgvN8xMIgn(s*14 zlTU3;Px)MThI5^bem3NsDP3+G;-;;_%ic$gMAq6qBl+wn-@jUGW8Ltz;2hU49 za=N^gjVc;6p_UzaShI~KS$Ja-u+JJ>7M9cyOpgzzxX;o*4!Jx+66>}pPquJySj~~- zGWl(c-R`#&C)#{`Hj^O8+rg83X^_|RY|5`JD$8XE&7kj;CqCFKSZTE6lA3w6mo?z} zogsPhiaJKHnjQ@46D-l+!kD&n_7wllyV0yIp`h=YD6L#ENB2OV`R!y2nK*hx#e6S) zG^~2PK)CX4>uY^qc;77iki-7Esz|zSfQHAKQYon{#pz~0eKoQq!{$!O3(Q>Ok7Q0)lE1*&X4PxqJMMjw*iq3VN{3;HJ3{pu{}=4J-zLq5AW z7xp#hpLd|PA94?CZzZd)RJq}TuBerNolN2lripZ#!(q542s23C33;*Y?myM*UgW4Y zg!ra@prU*U?saS0`_!&dZr&H`#LXX1)6^=Q>QG~iXpYAUvDUJncw}r^E_escOkgS% zA>6zL4bb@KGQPc>c<^w zdU!GykrI@_iQhNcqWY*47E0qCYTWTXse zi0Z_0l~p9wg3ok1FbfWKmd2Rp#vI^2=f6fA>NWCb;0_#32j}ij6$F^>!r^iv0dKts zRHM!st5~ff0;tvadm?as`4%1yL1)n9I>tQdc{L$!4iw_5OGY1{BM$Loan7*Oiig}Rkp7vCBeHZ4I!YkgaK&t#C7DD=Ll+)oX9iX#^4YWkM z08xkUzZ@+d3@$c0gWkIV>y7~FrjYe@{gOPxfyB>8$!`)_>m32XC}aGOw3OjsZ>MTU zhe=ATtBfR7dji)mBaiX|kEp1+Y zl1a#C>)L|WGyL&kIOKnKC9bv$wY*e@>WYwES__CX^=H4}&{xMeCY=<%EZp) z=K^QXadQZzbAGxzZvfghRn7hHQ1S3=(kc{w_!IHu2=Kc2Qht$AWe zr~^0g$57kB(=r6%RN?GG>$(JbANcx^y?Ia)z9fMMX08P+fGuUatfRG17%`UU2^XBA z)9fvRcfQ`?FLI0$e0)x6axqhXv?ft#wiRE7PX6BN@t$Z+#_!e0hViOImeYO%G8=_f zEFp0I5xUJA9?IpoeOW#~N1{y5eDMox0)9c~MneUnCmm$Cf6517AMlI$%__>1Gf^iX z?!WQ-J0IrCFVx1d8CR7=p^8OfvmE0tyHV-7Or2 z6+j3@bMsN4e&&HT3n67+yRCN2Niy5o1nl>aN3U%w zRIg8hcUEo%oNFRNhIKi%J_R}MN3x{G-(J{!TQKCP-fesQz#ubyLV+Qd?i-|Z9c*>~ zWA$7CS5hp}1LT>}L6Cw~Rl0YXdc7-WXyekGP%LCYK-_;fJKPP#dk^l8rwmt|V~~LEFf%;Z-pcL+Bq|tbF6=#DbSjVy^55uL=+x zmQ9RKCC5J)29$uj51BLu0QGcP^dry048l6WhcNRp4(WaLbcd;73 zNx&UXey%1NU_l0SWLo#MiW-e~71E$d%ZlZC9D`NT-eJ(n_I@KRMW<95kf;7_peT-d zoJpfw|6cC9Nw+0;Pv@`qjFKG0*YMbpMo@D`8U)fNUgGi{3tV_?B&r)JNd&y`Q;@D7 zt%S`Cpi#xps*#KO){*C19Z~bi%|sMu%wWJ&l6#7dn#4k>mAv(<_-`Q)ASJu-PCUiT^o~CH@Q$ zG?U7dI+$NDOfg{tG`oe%Oh-~S0Bttk)Kr}><TOrhrHOwd6P!l)i%Mtv+rhLZ za~M^TOo8>trm_z}R(|Mdk! z^uB79>Lp^^`ju82kI4B1FL2HESUUPE$(7WkX_L$G8L(Sbe|s?gB5mQD2n;~ze+^^q z2|))t_Sh-i0Ay|Y0A)S6Yl$^7>4cxi1B9KRZ$|tVPBemd!z+GgzSwT`(jRez0%5p= zB|(8AkNN~E8iEehQbmkL1QzDDt$ikVFHW7++ycoLniH&N`Akjtn4$7a!S*@fqv<09 z+Rc?%t@vbXi}~oy*eX=&rG;EejqJwVR;>K`w#ln|C!Yb8%0?HWfJid4$@CAU9zf6R z9bZpwo*T6%AP6(*UlGA&`u33;m_aVV)x5J$1~W+yNB_YK|HP9RiglFWT%cj-AUY8?GY!2J7KKw6mkGH4t**8AF^#T8i z6b=1=WdXdO4xpuHgIoOv-1U!u4@15#FUz0c{O6h9q4ryWFQ-*@sg-MYbOb=<`RxFu zYfje_+|JHU`^{c>5fKqWc6QZbSqosL%h&pP4ykx*2m%5E<#KI}QHAVnqnDSLLY0ya z7rSHlUy|MHJ{cNj`vHnAv2Fmu4>!{Et~VY)L8DO}mN4@BgOV-54Gj(L0ng6GJ{2Z| zojR5&z|gk?bS{aRnJ4!q1zw*5-q4NlKn$f$W6d(HpUZkFpw)<*YC%pB`5lt}KOr}8 zGT$L2dsyMgnG&^1?`MxEpxZWIY4o+3Ns^`m;9)PfdTF)x?}==W{4k%$HlHoU%J4X# zayptv2Hac2y;J@c5X5n|X_N~IO)kEQw5Ih7#K59b$i;CN8bu`k%|{6N`SF+P?GiT~ zB_yEDFW*&<)lWazzrJ{lA5P{;bK0-_s<<$p#df^DE!zXW?@*WH#UB#U#H#Lh|DUDq zlB_cdPXem^Kb`g!e+4X@tTaXI$Z4LgwuFY`a171?SS=E`U?N6FB)%^<-GJ0f^oD$( zDd8sqWrg5eKxt3&UfOc4wPmKkprgs(wqnis1}Ef%Dq_SkB9>j5OiynCvz*LgwskZyeUMLP zSJ8X{>+kQsrPwfiap_JT6$`${BR_}r&wFbZS_fpbA%5LvodI7#4iKCYVrhZADC!V; zYd$R3`EYh4Cz4;P>S|af?g?)Kr3RQ7%9e~JpNxde%j1Yb{{G}T&J3q{Al6G$EkL3| zfaMMek4=ha(gw%|E9R%>)PP@Bu@9c^?|x#xShZF{8) z=f3$FPo*SA&vLlh(sV_loFxeIWq~%=ZVQYxqmR8W}6vVQt_MHxNZzR&uGP3&xkia8C3?pM;>JQ5cSb? zwb@-VOrmCrY($~a?VOn1dfpdM{~R*0TF65i0nH=psHrAfNt&AE{@iYZPy?EpAUriM z(|Te)fcqy({b#hr6($^lmvo9{HtK~NAL+l?9xC$?ZDj$%GvVCWP#+BaTidgRWQp^!^^$A^`@GPl>;tE(V~Y%?|?i!A$v0&Egc(zV~{gealC zip%_)ZX<)=oUSpLc~8dZE*?+fVzXOQ!mrlU_`_|qwzlp}aS>0^r(TI%0xmdeXw|$x z){(GHnzf`_E}i@0W8gjR&|d)bCv5#eLtAiuw6&H|6F~_=nOH!~a7@p;-Dn0vUN?yPZ_w&>R?~*M@$#z}lL$$?>lk}w2b1(rN>^2sH;Hu%d@qD7lR zK!5?0!#3^8HFsKOG%9zrB*`U}qz%7DW()Y(@p8Xrxc=P%;H2KBUi!zTs1fucz zQT5nwfFc!K>=Y~1^ll=87ap6HaWY*aYemic(aOV6o`ZpU{BsN7(*_n)os`5j_9k); zYF3gwJ;yKjKC_FzBuT|l8gLL$Fu%dAs?c}gN~|xFJMVS``41up1nA8qO$+R}9aEFz zEgZoX!()4usw*>Hc5}}?fGbY6@gHLO-zw9euowjin#@cZ?H82rVSIL&v_uf&+R0%J zC|VY1kpFO!mX>*#nw_5JaJ+lIUE98yW5pS8*~i|_NtWG8x;broP2lI>GaXT}og98P zP6i K}x{so~O%X5d6`Zf?%av0?t4$e^c7)+8e%9cU4}+Z&Ek<}%dy<(9tDNhXp)=&qY5o^xsgXq)2&2$N6Piue)U@OF-w9PmnGcDpAsC}Q05_O>sgY8Y+X=+s z@n(0&@$gXC)z#I{I5rnc=fJ1T`IIAefc7UU`{7au@)%AAgP49@tdU*(g7jzeC_oG14uepE`I)bME2ky zi=@5@exl;yI3uwgK?z3>kB>MO<@Doj(l|u(w)%`7SLR`OJhL-ReGmWjJr$5e*#b9W zkh85-QzIDU5Aog*_@DQt3L>QjRUE>x*#w6~4(2NCW-9voC2B0Dg7M6?K5c*jHi9ea z%g*Np3{94@LqfyBbvHU63bIf*Y7y#50&fh0|LIJ)q}&UrLnd)K=Br>6{5Rr_VgsMF zk-+A5q6C7iX^n2Po5Q_wk}2i(&N_g9qoRtMDJMM=pfLs}B_;88`0@+@f$|bLYlSHD%L$i= zs3_Lp1l*^TGNaXvVb0|DK_3GkHAaLy?#_N(CD*uL?r6_zouK?*^lgvp$BGkEXT4y! z?dtDI9L*5XY$&2gfJ}$@!-NFw*wF&oQ(?tz!+{8tO${d<+ReNnDW=A<1nBI*s5^#N}r~ zA4vc&vm;qA)zN(yw%sKC|6WgSs9>@$JvWU2o7Dr?+vM{aeLhv?x+A^Z`YN0C(if~eU;uZe3V&fPd0s-h*#+Z_ zIz-B(`oxFy_XFFq6y-a;waK$V{^MV!P)HIEVv_(YX7qk?6Q;XB6yd1@>Xg_P?mCz} zF0_D;H`50uPcQ&yVg7IKhX4w!6X^xA;qc-Cj&UH+*x>Fu{{RwFRU@X6@;sfRVd5G`U{25;?30T8CfF~ zd+76(Y{iLmA$4G8MtW}kvXcD9prexO?I8nu(;U#UeP0QA>Rm#(TR>`;ZbXh-agd@- zC{K(e*97nCy^+^&TQW^*KZ}ljBoq}!vB{LFwpLDJw@HbQwTRD1G)jRu1Mq4#kV-*9 zHg*65WeE<$e{71724=ie9NSgd?WnTPZ66qD40(&%R|hjg5(~hluH}JI34H)h`q7fC zYfLM~17+5uR*?^{h6$duOn#;0_U02>n5-=76?O2C;Ncz9MM_MjzT5TshlyR|g{3?@ zr|-uOc7E}bN7aHG${UV$KWnC^lIuT1u(i=!rAF8O+OJNh_8_F$4f9Y|ZX~Dn+td~! z)Bv2MnO37y(S)off*`*yCn=dirJ<;iJ5G9w`z84c(1@iN>8E;g@_sTX5;8L2_eD9! z$5i9Yiwv!Ipf$1NUvt3$u5W~453OrCYiJyM)H?8N1aFX%8XrFZJN3+8$$HB~!Ej9F zyR>Ogk-(P+vKLI@*a;fL?!Iaqd20DCBTd_Ayt^NtpS; zaK878#Zm3}%z^N~nB{f%jBkJLCY2UD^D7g32xi~Jju?Du#)U`zcJMe>dcXz%npOySP z$tR{P5P)+UBZ4s89u*~8nSfdlDYUJ@cH1R~)m?kVjuxL$lO)u7g$$+6*a<@ zSaKnv_Pc|^Bv!7ksHh&}-|vVYHCV)NZ9(Dhaa*9$AL|5kTvv^r#>BdEj}@BDF<$|Y zY6D0fnF+^ba>c^2vHEC9$pp1*d~O!ZdSO!77)=Jz{wX&Jn!-f+5L`AW)|h`{56K0g zz8A|jVW$T6S`h%nE`_hSe?y+2EfHaPpORL>=F*K$DbAGGjoV>Do4&*EXe~l}fzM!m zG(2c0%|R}(J0WuCVQFb*@ILNG3a)hu6==x1qEF@9P}FblLeyShSgIc!I9 zgDm!x0cnC*FW@cTI##_*e>|1nFp~PO&t!m-nyK=ejDH+`*%%hPqQ*}$!DWUva1bdP)FpoZ}yGZ43cY9o?qz zLnw=^7I#w27uRmCTxY4D7f7?mnK|CR$hU7C3I=V)tY}sA<}qy`!0p#iG6|A{K58%C z+(FV@!FM$L(?zgFr=akp>KD%)?Da_*!uQamOr*()X3>|bF4*b(gEh}C0R1prsHQr` zeV1nw{O>h5ArX=Q!v+tdc-hOoAYckWU?&9+*md8-jSL_Z0NnwmA(KK z16~WP8S}a`&&Db|F5gc?D~{84JEAW?k57Hde}HcsQ!sGg6L9L&EWkg+dr;1*DiBYM^}58^Q{ef0=WO zEmtpy%1yhr7w%#k##kjz!Go%{-E6V^21;;ja{RAPmu>*k0iyV%*gCMBMszHL&twP4 zWPUy3ow_BmyQGBz*Zc_H`fxzA34T1y4hXhwCI!_XXE`Lhoc;o$C9R;$n)_~g4f>8Y z@Rj6#Z$GczMba&%JP`{maj*BMJluS5SHX(DE%TNEix@t4^J7+=YMghqn<-uw#eqpV zpvtrpBB3^-2C!RvMGer*X>fovU$}JEKXHq$!d*K(ia^wQOd8E6?A7gJb{Ls*-4S~P zf>=PkRj}o5i20|^5rI#&Y_$-y%T+Kd1n1*&xo7y!PGr=OQOKq5Z}15Gux)QE_h#rZ zsZ7;m-|I$pI6LQH&LGpIeV2d5>}sv#C1orA@G$%+PjF?a+abXm7zkbZykX^(*)S+k z_2G%prB?#nxd^ODfbUjrJw87G$=~ef?w3v9zf9-HYGX+2M`VHXa5q=d#T@lU`IctA27uFrhX@OUXTHB zC2(Z*WQOzY zF624nZ+wQpqKHBY822h+Y_|F2gxjVvJ7u+M+pIWrQfAt_v=Jjis05Buxt$8*0slO{ z2~&+FIltqjrElQt;_=X@WlsigBdRm+hwltl7AC>x0dCm6E_h|WCRnm@C+3M?iW8Gv zLuzT|fGB&?WbT6vh$0vq3)zsX98QeEki@9YHf4VR%f}LgnLG1gxMmHLC#uN?E~opo{7*Qg9hua3jTNJ?-~CXP^Csb9S&NKN4Qt zMVdd(8};w%BF*4H=;zlE)pnFR_x&{2pK_Y+)<_maKx%m5Ag7&KbpImvB~E@a7b*jX zQAnDPA43{3*E&nSp%v=zajE~2r z_Oz`2dL(k*fkhIsBrx#M$3Tpc?A|~^P`1Re4vJ#VJ4n(n0cHGM=v-;%qCBMj*&!d( zHR_}pm#W+m@hDyod^Pg!sKDNgA@qlX_-i~4c(cf0eX47d{&&M4Oze&{0yX$*wtePV zFMV z!{JjmKV1x+ok@7XrVeJ_b3z^p;0MHa)>x)oCG1-QMV4=g0!hAgtzv2{I%b;GSvV~~ ztE0aZk$}G&=*9?YDg5>Exry(NXY)b17QFZK5~qK3{r=gd_0w=bUY&7UdXENNrm^PZ z`1H%((yr(zR+^)H?ia?fC$vDS9OVrPfqXW_7jZ1DhXa9hU*z;5Dr73W z(zwdK7W1|fjW#x2<5f4gfNNar&?GGdoy~FI^vvtJUd3ExT21r|oNj(C`Wf|N0=kk{ z4E%RaA8PH4W1dYNr!otc!{VQa87Ewz92ux$Rh`8#oKM^~Q3gnIjI`zc7rNF zssqc<&)?v3oHE>CyTZ**nim!}2HdXx*LUGVI2@X@HLiQ(AgikpS`}VAPVdQ^7vy}3 zt;7wIYtUK5r=&R{GHbD>zScQYo8_|_DPzUF>9Jn!j^%vb9vpBEo&u64iFjWe>bc^0 z(!^9XEY>4tG_7o&jZOR+CQaV5RqjSM)n>GCTkj76N-_RvKr!c=uWJ_wr)w!ZeHxG7 znwq!Y`GuGf?31)yNApS|)6b&1ezMS%P0J*~UYO-asA@CI7k>{KuL*VRgc>2cD_kZ_ zlBN*|cGPe5*#TQ_D8f|VBQc+2;2wVx ze?Y6E7Kf^&sI)K6f2>)RJ$@u-Mh}VnMH5-R#NEukeNJuqYc?DP{$oYn;sCx9#sV90 za>(!V#9*WImz-=F%7lm)WU!@3(M`i0T&bI`s;GJyG17PhwA)Ky>X%5`1sYR&iQQ(h z#pAn-Q|u(Ll8k3ThoNX>n#o*~c=wvztih?PpmVlND>}(s6Q@|t{!i^p!mk~e#}s4{ zQ(=sxp{6tYjrqjnLnx&xZz*C)GFt})5#EuM1;SWn8g1#tRf-ovN(@1Dy)kReIY6&0!31E78$o8DJ)=pwehUyR|}tT12Zz6y}6X2)FWb2!Xg0#eUC zp{9B~KwpK^^lv@?1L#g=@l`uc}Et5>{N|{gfTJ4bugfIc^a`{RsF4B*-kQkIPg4W@S!J}otlQHeko1K(2o`Sdb9MKMbEd`^kZws#oQl5CX8s%qdttf z4MyluIO@ul-&D{?l?--oH2B$NxmApv9j@&n8{B+1rgquQ-=V0v-g7BTm<e40`x8#VXRmMb`bq`1FM{4>rBDecN|* zn2Vy?X>JQI*`X_p2d?b}TG`0V(mKWOy%55CwJvW>0u1*nVobx>?N*m5%}uRItKQPe z1k}PQp|;lLM!W(yVu^&yZP+P^_9i0ovMIanHtnzgsz(|*qr9|m7~{0|9@!E&ipest zjE|A5%@2^gMPm&xJ%J{5vFIrFN}0p0+_co2A^T)jHD3_Y^O`YfULt-LGyL8=ZQMMf zm}t_$hG*%q)W4Y^OUlPJI@OKP<@3lfT$#mum^p1QG*Jk7qrg&We?nZclA}?vMnFL> za(#ZVDS@TCN1-K_-lrfGfL}^x2FfjxKC+_<#GeuKq1$k&pBT5Oa(55&2TfFDI6U*D z;yVF;i%EQ4T6HmA`BxyxUX}(fjnm0GA~I2~mZFp=+yU=t^4;?LvS9j7TaoOApZduH zZsXuj6VGZjRWguL!h8#{LFxciQdQL(Xvi-E`PWu|+8bh75N~IGPlaMA@v%69wMrhn zoGTLbtVnT;Flx$*F)Ua^A2g<&jW&ws>-PYB)Ss4$i>G>#{7f*iJu>+uENc2@r@`h1 zeTWLXle)>f+6@T75w^ZForseS6~g1Y-|5G-jr$BXI~S^6JOY}2IBYnvZfk%91Q{Kb zx7_OQhpaK4;Mj=Gb{1_0wdI-pWHv_2xS4rocSoP0uG{UU3L>9M<_k)>i{k_REH=5( z^|fqtB7(gfGvbvOYVkN^oPQmM_Y^z=cZAl4tBKfX7ATebrI8@v;Dp9f$YaB&^r4b4 z6@Z`i{oH?8Va1N7m14u=e*fU87mXPvpP=!&s{VPW)Yw!KVdV9GH#1GeSIY?0J<8Xi z>tqdjThTEm8tUFY(J{g|F*0+bnr2sXJ0VF>5p=`HuEUAXT{n|_KLz~L?Jtb$WT7Rd zFJ;>v$Y`FPj{JWSQwv>pdoRls< z<9{e?#0_`hHXKG!AUW>)o}Vo+D~+CeR6@Fk3zDi1qcv50G8X_%f1c!GX-rC@Ld(So z>n-d2WCWoGa`$l18=FUSzKd7(U8d!Irts^@j~T|j)VAE80&Z9lQKU{jC($yauT&9-owx!H;rE*suq(Xhl>pEw3r@!0>6!x(`osml%`2-iROq) z`qkC5o9k!Nk4I=-uk;cl;e!Y)z^narScEAVBZ2Z4*0MU%k?~4pVL-CS7_OL^6eXg=CxKlR)z$S zq>KDbm+29mr(YacME0INZs5bywqjtbmWXR%%AF=jg84Ar2FHmH*>##gf_M(6$vVW& zVhWq=s^Unqgrao4ASK?46`-qk{84HOE1N_q^I%iuQc}iB3~wz$&IW29p%ZMP#bMuR zT%tSv{hZ?XHhTm?)`5@%hh{~cwbJnTBMP%KwAib)Zm-#r6X~os^M@{Wi+#JX(+KR0 zI`=zH8fXdOn9q>1=n3@c84N?VDr@{?D|Ut$&*X&{w+Dk#_p~D zO?zD-ULihzWJjqtJQg0RJ+xoo&-A?#*~#vuE}ksU;95#w1h1=PA@wi<7Y!Gc$GNOe zm_O#4JPst{4zIDic|2sF3JRmXeLLK(JSy2QTol$bERb6WZ<#ng_Et6HmOlyvw5L`M z_`SVokhZzyJD0B??RWCGUs*RsO=w@tvs>TeD%6&mkbiowq{mONLNKzb<=;jxlJwHI zX@B00A9k_~v$AW+k~eO|+gp2CIhlx972*3dPeM$q*MLH%Z3AyW`f|I^7$5!1O0Urq7c@ZtgZ-zJwebW-2@#ZSXsB(aBMHV0!xT}h5#yYs^F@#%v6rCjW- z@iRwC@k&2%jDc%^^W|xt5xx=*WuhFvlZWD}5pa^I$KU5mslCSyuG_vRpuYlXN-Ca! zUaCUZLi|2AJP~~I*%9Jgr{;nE;U88p>Iw@cY;_bAG^ycX%;}Xkq~UrSv)R$v#UqAk zZ6>kVhRlN}>I2(pe%Yu)i7#R#z2Dd)A5%=2YE$B7kneWY%IpaC`s~SbN+b!+?}zGl zsmjMcWiD^raNeK&;`ooNqxU`T8v}+LrDr>4B~><(Yn< zl@`MbCeDisg%`u7`~8`4r0)eMO7luFLwZ2|`XLcrwe>)Ht&67!_*NL{B9;V1%E_q| z5s?#^E?c=4DC1V%gIr!Koji+Gfnr=dz_n}Nf4jn^%I^I**^fVl#1XK~zz$R$Q}EhG zWJXHiO@NpzLx>Kl>t^zaSMF4oB0=G*YS43G~;B;xk~5vf)-9RzE7w- zAx)@SX}^}xT;74K_s^izDrODY;}48su+y=r-3>O-8(BRoS7(ZUr9gCg9^?4-5b5mh zme4!J&;o3SL2^)f%3KW<74igy^}rCmuj)}MQo71>7x!#wBr#n@^ZE{uf9?Ju{|*~g zRavO;T$TD&Ks;6)XKZy}Aw>m~a&T}kwOc|)1O~>DSLYQR;P!vv0VgOCy*=wI^d51{ znz&*!SZz#|`ar2b=!I@?3*hhzS`h4175kz}4JuiOt&g-Xiey(p^$DZZr9Hg-m-E2l zw}@mxa_9#4A0k+K=Yt}6+Ho+(Fdz;}RIT5x-l~zl#0u|vh915pa*ygC?Eua5YQvl0 zadFJ5kOIyUDtEllC}f#tPR?Lo?M2g!pe?DX$6LCTveS3O>+qOgZ(0*l>HF4Y7K`cR z<>g&)GUHGRfYQ8vq;E;Y$WANn>zdGbl_%^A>O+=+s&Y z{u}tmCnAKGTn#P|SqD%;`NVZn*PT;RBOgdB)+$%Lg?+?+!rN2pGw&_~suJte;KQ4u z%F1ZNEauudh(D5Ei<$6Xa`Y@Vlr2;H%-)`4%-fUvPp+e!7!ZPowFsANIgX7Vunz*u zbHS z*0J1S@$C#pih6Xa32&W)^A4k73F!>M;DY33?8_`Ghu z00Ri?;e6gY>#ckEzkds;w@UyAdShA`bGx3T5%H8Dz4XcdT+WaYtWX_2lsER_)M#C)iU8*y zcJ+V!we}PsPXx$+BNBNOt3WAG5*WFEkdSLEybi(Ej2f0 ziKGzAu7)v`TuNH3z}AGIdj&5`vp=q+lg7^sj|}^Rum6{S7DWtVA#rhgTR&Lida7Gu zw9ECaJt!bq`*OeV=V2 znK}<`;K{MedULu|FHaQ?(AX}ki~=D4iv&Pu0@^#tUVWMYh_~cw4}LQ1VmHjLjCI0k zo@@D$!z!uN1ju@UAJ^mx#gfa${hc2nlm$;=x3S#f@qTJxRltFfBTv&~!v|rRp&e#d zzZDfISb^?%;^{N0j`fOG$NL{^d^7$6^7Z^0tA#2uUw0`lQ1rw}soC>T)J)7AJqi=85xS07_g5De!}pa9i7I;EsC8Ufg_X}mb1<@)or zABO7kjTinaKya(&v5KTu#T!^D4kwIqZ0fkxkIWfA_N|}q&bYaUk;PX3(E62hS{A0~ zJ0kw?opf`N)ktycrvTO!rzhtt6zVw(@@rrfAo-|HB*b11Q6IEh!>%|HgKHes352}s z<{|yk?)zHYN6dyFGDzmD&0M^Ee8@N@;dlUd&XHeOao7N^1mApr+n0bbPx^+|LTQtk zEx&gMhn0OTR9G7{eB=XLOzdpgV;5#BmAvz-D(B}N)6WZBd_Z@wVOxB|pOqaWuqkR} zvr=O;IG;s_3@eI~_wlGYxjd9cAG8h}px#@jcK6A&lR+P?wXKa`0-YYkmo+;dU)?t< zaK9IL7|$UB9=G~+gee7AEYbeim}AyRB8uhB14udzPO;X0`T|%Z)2E7Wg_uWt4q&fo zq3uD3x4BGIeAgUqwv^aOu%0c!bQk2fT=WSHDoAWJ=)#+tCJN=oM4P=qrlDMq;nERZ zzai6u1%C(yjW;@5y;0T`3jnH50J~_`X_;f7{qV^KqR|W!vSJ6IZ-$+Edki}2lCM*Ox}Gd| zG`n9a6e*Gk2LTixsb9Ku0=z4WRU&Ks?QedKURuBBV59-l1^&KU^w;@Q9ngE{bUuhX zLL12<^1+ouMu{-W`eL&wX0p24ggHOXqoo|nK^4uU-n5(D{OL!s1X{=kD$vu;pmj1+ zNX%fU_l?8*LPHgW^tEu(0|PUeCr?6r2f>=#dB$Z{lFs#IYwY5d4+}f;$Mbb9(+Be6 zFCq?Ico+ncUv5uHfPv{%-&k(ojztT0;y+c&7!2(ur(}3$z;J$RH_P-2{vrGoyUcF> zC&(D@WthVeClScunq6Ro(sZ`b`d1^@bE&dYoXi<{w4Qnk>s2r0N4eE*Fb5f7knj=F zz{>DG({nlrnM64X@K|J zJ~oDE*Sw77$dBDq`euZp;M}(eou^GV9hRD7Z`}PiR;FzRrvoTJ_$#}j)@e+LB&JdF2gTFXqVpB@OZ{cGHu0wh{g1qgt+<1*1kfk8z=AN0q2PEi4by(Z zoKiNIy#?JPilVW)uwv;FJ>CNJ+R#+W6=q|vgp?0d#d1rX50})(F=!36!-L ziVP~>^OA@b4iaR8eK37MmxNl>2GCOMz`Iv7hA z9I?&~@3FBWsJ~Y(6c|f30D<_PK$1B@GctW~3OwGF%2Xl(Hx-z|6lG#3XtIauW55@) z@*&`zlL;HKg$Q#p5BblS!x`s0v-k~QUusfa0t8JhJf0mwxKME|+y*(!B|0q^ z5Gff?Uz4`Xo-v}`pzq`Y2g}HTBG9FSokp`qCS$Fu~AfYx5Okd!z6~X-um4D zO3_G|F_isqUwSZQn?_kNET31DhL69;+>`P&{;i0ws34?B_dMpBCiyCdRY#yqb5bR7 zi>oZ{S7~YDK@QwLU>x=LU}vjPHM|X;BEmX?)(8gwNTLJpw3lQmSb&nU zOC-27C2&2h6f6V?TLvx^T1&yPh=1gSuh=tmaTld!3YMf$F@$-WS*m8&KxoV^J zu09n$nme<)m-2gh4R*?6`p#hH^(vq@P0ovWDHv6Sh=z=+_E1NDDDf^)zt-w0pZ4Km zPoxhLyFSSJn7&B4w;f~yRbjANY}9~J#%0acYW{}L{)Jos9X`!Bo~Q`mX%qI9Q_}+c zI79?rM2lW3()ma9`6*lWO!9s9%cXxO9f#43o-Q5EO zcXxLW7Ti6!1$TFMyF2IH@7{B&<{w2BGqc&dyWeN6PLOj7iCmrWD(!#b&iQmYu@+CzCSxWc9#g zwx_rn7A_$o5^$I?U|!-0JlE~n{NLYRaA{WC+z^2inkt9P+1{r+R7L(N!_5aot?0&` zWi17V@t;a;9J3z=*4B8Pntmu=14_RQ*Vl7vmn2^&uNUbyXPV`~+XyO{H3x8f7hY^= zH02&^E7TDM3@I*`XcZq{7yN#b-eEoYX)I}?Bgks}U!EN5~% zT>_dwUa46r2nB4@2L8>Eg>Oq-e=>7zRfg?Rx~x$&1r+!UFo zWQW|Ar; zWkD{lCK`+{e&Np?3`xVGVl_#AnSEEVZi;p8%yXy$udDqZl^%5Y7-K^%Cf0v$jOriZ zf%|AOyIDL~38Qu{rh%u?)fb*_ZOD+}SBme^LS8?@CzjmaW7lWILj=z$k=oW?>TX@7V zH&Rv6dCCouq{7q7s4=5Y8K1cdRVa+Bpadn%nPus75V2asm#YsXS}JhNNPV;UW=dQk zFrD|O#Ly`{@$TBA=wqH%9ix6`+u{8gWc_#(e>ti+(#*Ywi}5UBZx02Q-8=o!cF`H( z2S6c8S+JN4KfKA~oJKQY^4L&E215%Q@#)}1u%}5rJ%9!h6+;IPz)S4DKh)yR5Gr*5 z^bxl2JXf;%Accq#yZk{>t>&YbXM>DMi)L>cZPPCgJFILQrLZ^hw0C_Cmq$8WPX<^BqLj(m6Wx2@PI6SN*x!~lXRG* zAAjB}5>!q@GfCo1yEe@lcdT<+o}(NMWecBG5kT)S(Jy1*H!l2OuTT>fb|@flJWRKL z#4(_AV0BWxa~cB@;hO-oArgi{9=ZQ<@$s(Us~+{!8qH9|Pi`I0toX5zs9`E}jqDBYzb#a1l=}il(RiS4?Z>Y`_PsQ3uuQq+ z-*#Q>p(Sefrw+r&&z5w<-tEK3pFc-w8%r(UDRAZIqkdazYG=E|c~ja^A;8Jl<*NL& z+wGm_qa%=gUx*cxr5f{%vyHIvkc`6FBVzB^ zT=+%(+MZC^E88S+TrA167#%e*;b$9KWc{SrGbE{3k&vMDgfa+0ntYGHwIdbN$ZphN zJ9a%&edZFfOp^0AI()=E-*}uuG`!;Fb9|DM%Dg}psV1!?_go!U6{rYy;5Pp0PJ_1K zk7Y3K2^)P^xpwmWiqhGr=RiiJc*mT;$i6^^NxA5e`C9VZJ#~{>FxkP8Us(RYrfxIS zdka$7NmG6#jXq*;6F9PHzjMYn%*yj-amf*VsbgGV~)yT;}{wkf)$+s9aT&; z(Z!6jsqNp@wy0cfJ3c)bY>i(2&GNeT5|NO6p0NKPgAt(g@&I#%mo9|jUh@T_#c)x8 z0hZT#qjQbRRn?z??+EPnG)amZw=A-m*vSdiH*Q!$R2FAfb_6Y9S~O!#w6+6tDbbi< z?Kc^ttx3A6)bvC|YXWZ#qdL=aiFaWLuwyNs<4N-1kg?^}(KUfewl+~*4}G-G^KH&- z-VWW$<@ijQOhoV4@2SqjI0rF!JsE(J>7Go*NwzpCiG^fWisfogQ!%iyseLH9!#uH- zMBj4ah>zxqc$+a3LhB59-C1sL7R99I-D`~Do%C=sMr;8Z(nn#=O@g%N;(rZjxTXIE zmbtUC0S9O;FN|&jJQmZZhl>o|g8jnsG&@=PjL?XPzOV>4x2UK_VOSJ zbmVY0|H6HhT8)3WT*dw07Y_Z-3vn_c(3>Ka`8f8TfUonb4aPe4?BZ$ohSR@;&Hq=a zwD=3v1(O}?0H|Po?8rkpi-21qNg_Sp^&`=Ze+XTPh^(2c+(8SvDE{}|( zgGWjJiFKyt(@Kr{WJi$UVV&HSwQQHr2K2Aw$!oKAO)S^5Zo&UW(*0+qCNzQiYc+^$ z_I`-aFB0H`vCX>P?zO!E;A60s_Uo}}5S(;V0SZ7VFeJ-2hZUhZuaKDTglZ%NO^X*> zbj&6PX(oq{csdtE8N}up-IW+8g&dP>&fA3#Br+<7r!N><#K!N@2NQb87b;pOn`C+N zj2ySmsqeCzs)RW?CdY-E2prSyr8HTI6nnOO3?3*;ap}d#x`zTiFpzKe5yMOU!HR6Y z_={K~8qcEipRVWja9Y&YEJ_R$X^yKH_~Y`og?yiZ{T$>&d>mJcm2VQ$ZvK(4(+-G8 zE`on8f0y2S?StvNs>S`!FY%{Pc`%vdaVtPdZ%GbspQiuL6>z`I9VKGgD2+v?CMN#O ze1*6v4}h4}Q_2eHu$nl!w;DJmVhXC(O|!-cc|x_C$zh4bluC4ZbqxF*qbxa6DJ^%Y z#$_4ds;&QC;yjQv1p)CU{3;Xo18)ndNMEj$HtxJ0b}DVv&=jK6e$*qSvRR-2oU}!0 zJrKu;#A5E{a)tCCBxPrqUBttEKsw)kkcbAZMolhgj9V{ZBea3*CQ_h zqGINlbqng`EJskK{BOkjKNa|r0+FgHO4(GtIP5=vQ!ocG9TXjk#&@0a*zYWV-Df)&1fAwOnF%` zzSs3ux}~#_y)bk8`EB7@Qlb6Gn`1AEl;^NOV^_raU6lNJAmHH1M1y> z#G0d88eBv1dsm^qKGu`!W^J0(<)Za_m2y{J*fwWoCu0LOw2vQE>6cH}KQ{0rh*;>* zz98Ta3{*5BQ8rvsDWNptut|yv9(2k(V=>usX+!^q4?1D^Z$9WrjcWdx3mjD{T4-+E zip%F7Pd+%TR?(MzZA-D4NG6fN|IfDP_(BOpxVvS&Z9)Yp?F?zPeqYs3u**Aoct+)* zk)n*cU&9dbNK!Y4(wG(v{Z#-@_;^y&Wh@Sm-tnvn9JZ^_T3`pK{tR5>5@f9s>X>;L z1#r5tO05`HK6Ra`>Ib(c>U>IlunNheG4&r}Mhozs@f9gEMm$Ou%h)GwhO(Zeum@9u1_!LuF* zM}kL^{-zEN|732tKy>&KASWZokq$q~^t*g7@9zg;8QV8+b%}soX1t@ddml_4M)e+< zOAc!-L#Eog^n_Z2FLq_333FVPhBEArcgEC%rip*z-O%~$m>-2(nOy;PT4NwbzY*Gl@BsiE{6|kb?rOry9Gb$J!>+T=t0vvB#9!2%zB84nw-#oogU5|qR_V{ z45}E~#TG+~LK|v#7%dfQ+ zH8Xtndp4S(aLV#ER89;o)vKm5G`!eWTSU(ksKqDl{Uu6F=HB8BkLnmR&B}Z*9;{X$ zY+NkY>7kB>K73Ed+FkZ%cZPKd4|8%4BJqfHe2q@GmCYOq3!AV93rX5lB8fpC1mfkbB?s-zt~wWx7ZU3d)3 z)rs#PtSlV74|WNAwAL?KwV4LgC0|$hy40;^1ffqbV#Gq8DMY86Ex0=0VcS<7u*|Dx ze*MK)+_~0sz63q&naR6Ftc*fJ6rXnnDS~8o1G4Qgb;7!ny41I(2`KcF&b$ovct4w~ zE(}#0Q0-5^<9!@J|E*aHm6mN9t!9812s~Q)E(Kspl@&LrQVI1oE4Kf%FVC{&fB@Cu z7_gr>n#Nz}8E4uX`ZLrWQEcY9D>IK6$1@uek2-l)j(Bn_W*>7o(O_gChLdZlm(?5h z*Y+zFb5Mc&MlvAMir4r??!}<6Bd!w_(t}idmD6R^nfJ)EROs$d?P&f!6U#?H@Mo+O zR%HSmc)l4gztB?$s8js|nr< z4U5;Dk5|bFnegc7-!g8$!TWxYpeY;0hXbEy7B@2K;6BzXb@F7fnYEGx49{GrO^r=Wr}gg{1=UHKC8b#N^(M+-NuP{y`&-hk>p8zGp3 zaTkf_{XxB->zl4E&jr$@<0P-oUN+Acf7Xny1{M-fW*0Q_4bkzZwt6iZwnxJxNQh{BznUJ^`5L6+pWL;yIQ4 z7VDwfg$QV+VrR%EwB#+}PpsWf%`irvgRE-*g^Ag8#T*9RmK0hl<{&BrMZA(iMKw#s zF-nuu$%cY}>^gNW#i7W%p1I^)Oku+A*ygqL!d!^^Q&T}hF zx6DU0E9yQj@$V7Qv06YqD2tM7A^M@Gu?lB>dk$qt4T?ygbeSG9=tOH5gGcJueal$=!a4wt`v-6(w@-% z9esC00*r2T;79jB>;B|-&@cs@ojUeWn%|LXC0Hv0cEnN*?*$AY=Bv5Jo;Aro1;CMV zwGKBx7sD&?-zDY|=r+&%odr%NCCtr_qX!}b5Kwl>+nTSGYJbzoQHffRD0)qULS3(D z7*F@OO9dBZzP7tJ^ zC~2DF(-uSP-?M(B8g>Tx&sE!aM}q9VJx|chMQ0pke`S4;i5dkXWP7_u|2zBT1-;rm zW}tj~&er`W%c#FtgrKFwYZVnB6@-AQUPXU8U;GN-Fb(=ck+-#!yZSJX0OgrThv%J; z0I#88rW*?DI2%b063t)Z@r5|H;nE08^oY(S9LwEELY|t*$2b@0G(&1w*3j^xJaHY5 zic~tXnWrWKTFFfsiot)AmAOvfC!xz2j8_!;+iRiH$sWMqpyW8O6>rv0Nf9cK=%|oO zN;C5t^PVEh!0n%~ciFYasISdjpRZ#YTu#(fO273Sj&yiE?6NfZy`AWDdES=Boy&nm z#YE#|oB|h@Azw+X)DBOSy5~1X`DoB}lNjNRG%XIIiFNY4$1{i|&2UCHg<7pknkKn1 zG+d=}&=@`A<+${HG`f6{+m#>e{QSAHNR-Prsf4U${mU>8aJn9g=O2-g@;zQ)tL-9F z(Y@8$^|Uov=Ol9i0&uU`#Kb90kAwK4fHUUw%(Pp;*t7$f#_AxjWB=(8=(HN3f%70S zKy<2HHuAY-@6F#EHL4<2EJ6-*+nA&vH6U_N+2O3&nVgZ0K7;T;erPkNjRcgW`V`bs zU0%aWgUR^GjX0qrN`4xW@A?27o;{5DD>OGUd3l1te_Q~Fo)VHrb9(OqPt*OAsOS-U zA#GFifVD7;Yg--+A!KlS+b2#M!axt%>?b4Tm1LgUc)%s?Ez+0!Ard;j&umu@Vt0mf z;l*~|&Lq+C)lH2?N#|<)@KuB^qMt{7;wxhuN8U$CJJP`rkuh+h3fxKspJF#?36IsU z#L%bZiG3u0i`7={Ue9yAx^NKG0yl9l&yU9A2DX9Z8I=!mLI*jU3N{`)Z#hqw-){>V zNTM`lCXHcA|gVv34s9tOu$xG zG}?XRm4#tHjXV7c!#shg3{NQ+>*~!Q%0GLOi%VE|#aHr4=(XC!oNN_a{xIS#vT1*^&9(Q5uOl3ApDf@a2Naq2tw`aIimBDg;wg_| z&3Xq0nz2xCkn{}YA~B;|C-tNI)@8p+HL!9dUEhiAT#GEQwpQ2C%&XHgsQm#4mA4eg z33rI+Y0JHWi>-pX(;}|8aH&ylHz=A!Aa_%BG$TiXYzB9`Cds7;+2eCc|6utgSvwCq z%O4gD8W3bOPNr?J%5)RX|3M!!H((fon33zA<(IW}npq_<8>@gI`L#W}=Xu~|VEkpY zug#hm1prlt7&dRmrO+7EkRHza3*-jEOV89lZ>uq6h>}!?qDx9n1d(QY#}2fG{jSLM z1-(S~Cs3i{hOr)0Z*DiPghpVxbqvVls?8sll2zNypctO7Ny>?kY!8g;QE3!%g}cvS zZ_x-{`l}P8F=lG5X|20EuxsQey1~dIvo5)L!L+$+CHt=Z(`5h?odUa!&cz0c1pnaV z=C}(oEhXVmdN44s!mFO)O(Eenz7n@q$G?orHVE@U+yg9&{Y5fiurJ$Bu~ffti!xwj zb)Pn0LNS+B+%SW4=S1^R9gtkH%Os+Bf~MQsm(sHoYDfwHBwitLvJyadAw{v53}A?e zxQ)w+JnxA@66VFMfIHmWe2OBxVd1u#y>)wV72SanzdTGbfAx%%Q%|4B7GL{TCbry# z@~~mL1#7PFw7>x3fX*(}WhDtFYD~9xE zzzK)9Ro4_zIDM#=ck0AxDVp46NHvn`Uf)U21Gl4H9I%F!eIQ(Z1qk8*LB2ru+eF8zHjo8RR@gZ;k8Uhsr#xLk#Rxv&M;_EsDsYTvPw~)ZJ27=z5pr<9Emkp z#SAur);w4EtMd3CXD|5<=>lN0iz=-RQ;n&^C*;map`%&nM0zP{8C+Cgw{NRl(7?bqL?))X-BCw2a@ma(2%(ut$!bq&+l z7Wzkn)5tjJW2K?pztlWqij1UU#+G1tW>l(K^_k5)cR0%@$66=%O~M#AoQ9E|k5+}C*h)zc9~ptJk$ z0~4;AkCw+1wNazm;eXw2(y{$c-~@)GGkOd%zg`?niv5ykRO4mWk{NWv0eKrDkRp?C z@%wn=)N|j0Y}vJWyxan0Hb0x(WX`dp@-=CuX4TV+6RCfXV|*-%ddE1RQ*zz5BQi|Px&OJs$=C>f6>Q8+s6=in!gQ3f}&_2CZ;oGF18cj z+>XWm6vcN_;g`SvWOXYJx@LP4^*(Py9@wMKCkbu#hYldBgC=F_JC&WBnf>5abpaxp zv;2MCOxo%_S>@Wm9${|%2Yy`_TY4^&pTUdj(?3wULQ*(6 z5*`xPX)-us_-Mc-FE>NLJ#-dkwJ$*AzHC!ZQ$cLQ+}>g&ETx|7Q)oIufr%wu=5q}a zA*SpFRBGID&E5J7KVvu_e-%JUo#g!Ji->2YN?NbOEZ!`vvurPr&w$NMH};q$#Qcl8 zA~#AB?-=i}Dn#!#5Y0o!xEE~5%%p(W{Ku7j>a;)tZ6S_?>;%h^1fMhUw#10F7~ZBk z-$p{XCn2byp_APtbEwCM_7QN^; zP8dH5?{Cek9T6Ea@=cW;G0Sp`-fI^DZv}xXUt`=M1nGWa&04AQyDfd%qXv%hEy|6t zyxr6^A&wSfEBy79>0UaO(yF`-WO>DxtG9Ujo@wa`*}(7_sLJ)_pD5{ZiDLCRgHw^? zic7|Vs5{IyGB`?VYDOT?6vtH^_Z@HkB6ancX3)`)$rPP(mT8)LT-KVZC>z^=i;yj! z#Ebyk2*_HtMemDE9WJl5Y-KuJ(g1UX4uvyHQ+`V8x%-Dry`kO5trmoQJ=Lhtvpc%>egG!E5A9(Xq@FKlq@Zf(;`n2`8YlZMstgn;X%ucJkq z)=f71+9{DeOdF>St-Hl{$nEo#t>Ej-3g8{U~1jE+{L4W+c+NF+BAkaH(pq zy~)(qlY>CK;>-YjytJ5SLklnL>dx=j7~@@p3a{QuhZ=PTqzqZltaL1mX8QJ!rV6-W zU@8Vzqi$O@Y!AlMqD@AM9(*zve9UuS)t=Dz30>IVY3LQwf=5G2ws*|dl~NUQy$9c~ zt@eqF+F|b4R+37je(0^k@4D*;h-U2HZmrHENV#zq2&A4nOU8e9?t#SoVMcYea?$m= z$7pUjV<*#+$?4s(9ZoxkYwt>T6(6BlZyIC`L+j1M`VQ1v^DJ#ZG6=)$?b)GlO55`M zBmFDm#81-m$eUbRuGqNf8V51=pYaSVW?x=ruTm{DrRE3j+}|rGLi~=7Ev<2Y{!wyP zuIQC4GFP|wc@J3RMC<1U`3YZO=dkK;AAPfKe*kCa)N;-M%+5v4_583@hc0m;Fylf($eyuz<4u^GP`?Puv2m=InX3y)&i>RdSs4$ zdoM}g*O3YC=1Sl1xJ<#8FsUjliI((+0awx=jy!i!Hd6fbu+c(E9dw68`#j1HXLsl7q zT_Xr#pj#Y>-v&M6o>G2$udELpfg4u0^#S)}5aqwIEGXprBSjH}DNN|R)hF+C+&}I% zPyU3YZ2MQpL*qBxlaW`_bwK%7opiC)We0o>T+xqmBas`Bfv!jzN9 z9#wRnv}@WL5qv{f5Uj`6ctxVfUI_(7_=fksnNMi%y`aZHWl0%L^mh?7!_Avqj~>QgXj69`#( zUS^|n36I#5-95-*X1H$$e(xnb9cL+BujJgeA`ATs3ky4CCE2~eHck1_#_P)+!Vj!} z$%T2SUs=-zs~ujP?{_tRWj!;1PFFDOHA1$ZSDET zOxE!iUNk77mJ)vELwW~{gWqgMBkHHa(`V2hD9f9Nvh{GbRO9kWuNL+WI&t@BPZi$o z4laR?Bu6aCQ^Ue10_xs6@YuX?_z96ZG)aD|bRoszQ0Xuz!73CFRND7}yw!IPdYy4-4EwE>+22 zB~OZWybUn9I#h9RRlXFl>^H&#`z$8r~k5kZr`JjDXueSwYPwH;j2Mt4r=m(u(?=D>`R9 z+ePZtR~7`Ph%?5{m=(5YYY!=yXFV9H6Hw^e{t5$c(Y(Lak!R+h=nq;@b)L!BKSP-| zhgr~xbRKZedjnMr;1_3^?G(1;kuwWdVC;BzZ`={{+)PWk+*AxArtz`4M4ZQuBu$FYe34l&ZE?BO!R+`V_22aBy(CM_Jqh z?Oc&;mg*pJIc@v@Egk{vZ@~@7=Mpib%|@gAI_I}av1~(V?$~&fQ*p{xD#gbtUP|6H z2)biI_swcO>Jp<%a@qS5OTixqIM#T{*dcVwa=b{Fvop-gD^lC>K@z zu0Q}#ar&3dKDNAd#>44SAaFdjwPI)S%sDP)@&$u8VHxHR-X-BfJ+(zN^=XVvbp=U= zm{Y(uOG26QOu)j(()q0$-|IXq{2wFSUg0cniF=vzwDutDgFI8kt{5l=!sJ^1`P(gP zW&ShJ1G-Hre{?eW!ES$EkZB=H3-d!cd|6Ir&L^HB;UwRCRq(4KC`i-^n}en>MZRA$;3v=J_>=FOK?hJ9`$ee_k^3G!YcV_F1>=#K(BpUx=QWr8c;WV`s~7Q0y1&aWjiTu0%Y)`PQ`WVJgR7(86-@c1H>N*bqR$EZUlwgr zxZ)vL^#~jQP4N!kb>`Ot?0J0mi$=hP>#y#8>UNt=>$GZ>&>o^^MN1oA1P}v5kg5-^ z(OHT=ZueC;vb=_i#T6iD&D1ht9R+i~g73d%XQusNW6KZbo$Q_LPqRoTMzm_CW6+Gk zGS@FOwJzthDGNO+RGgSP+OLDMwvz8*EH1hQdr7O84%-KO#JTe4NC6kU|`O>jNGtuv_QO*{>!t7XCLPRxn9X^k6=ipIs5gvv}2Y zy?a*xo9X4hQMs9mCpa%7hs|;!gWZza{d=}US7G;Y?y1hdVWQY zpzPb?atZ5F>0T_y=mWIk?dGjbtlQ!}PKn?i2_hXHDSJKKR>Uso*d())(U^)sF?B0vZ?E4%ah zW&JbAd~22&d*C)fiQ|VO(AK_N56z$|n)JXf9WNyXjQKwp@X%zKTg5|yy&-AJLY-up zZj&T)A2!vx+ZgaUZ^8?;RHp>IUuI7CV^UI&54M_|g3rJ!xH4VVCYiR`8|)epr1i8eeDLCD zIy}5nLzBHMGTynD%S|Zeb>U9$uB>_{oXA|F&a3XuLEOT2GMF;stl#<59T9D_3PRq`JpRhd6 zD|v@TG8&G%ogj}^l?1wK_aiNrIZIu2NbuV-f0P9pUJ5n{S3vm6;!1(~6;1ojl$Ps5 zMfvgh*ZSMc=AIN4yaTLVWR{ifJmxbFxh|*YPzr4hviFcF;-zaZ6@4i!gQOw8hz}-d zd^|#oaqBR4->Fa6h1WD*A<_LrjG+rh01aIASp+9wKQQGK4`Q%QGcS$o9vTY&%-(ZL zB;`rgHXx~~ZK;#BEu(b!=C|Efcs8dlRxQ{ox&14rD`K639&Gw&H?kz2zaZddci;5? z^3VD2vb>UE*I8zY!aXTR3D)byk>!0olFDop%oDm5SJe1NEff@w9b7J53mW`&7Gq;7 zAe*9|@2ta944;=Hx_jWOoxh;>NOsDmYq)c@IznrY7xgEh;Dr6=j7F#cO0t~%3 zh+h_rcO4cU@`e?nzxLHz2yPDhKTCe;+!4XL|04Oo1b=s2Y3o4Ng1obbl%nTZ*4oVaN0#8< zFx^}DB;CVfnRCOtAi1(kIH=9=r!$}$oL6f8ib_#hgb6I*porr;NTS&^;;;oI^vlZ;go}3EPw6Vo0i0!A6AeX70>BT6GZ5ezacbmKo-z%H>zNw7C7Fxl!n0-!n2FJa)}vFijCc$HbUL z91Sb8=hJXPoVUw>C8SLZ(U&U_ay%5Q`Jiah4DX;(iqQ*L&dJ}4_z~|2pH>-WQ}2ry zhVZoBI23<3j>Du=y(vewnIp=vV_OV@KXy4qHafkgd|1C2Qw?3UgS)#^&9*qT*4*ZI;ymEZq|Y2sDHr+QTX6bwxikAd#Ab^ zRUm7SslF4uDSRRK+?lm(ZUqPN!KHOb1epwnUo>P4UpQ|L@+L=SDkpLv418fB~6Q6eVR||QDxRsTW`4645Nb(o1;TY=mVYg zmz>MB=kwO+5?Ke9=6k;(A^Lhv$hJ32cD_f@7*CmgPq^M@vNqmyicX$rQWa9&yofzvNkNgbU^ZKiP) z2lyWU3(b$|5VOs1cLg$|Oaf&+(L~xaJf(hv-Ogs+b!JZNddpH5F1u0O1Je1ZO~-K6 zI%8<(l?D4FXZ4+yKMW<hv+%{3(OuM5N zqoS*N;4;o4)y14-+dQys#Av7XJ^{)oBw$wH&DVxC5tM(t%mL{lBW*w#FUVCeWHlN& zPCYgs;rlk{Oa5W|P1Z0fw5)chUtc4kPkNq+q}T`-pW&Cy#O?Af$T%S=+qzg&5j5Wj z`YSsR=YKGyuwr19(Pvq6V+ju+x!f=m)+{F7nH5Rap0B;wBPB4bg)c=PW-+TT1<1d419qHmiG++Un& zmMI-sGAzrWyPEGv#r-=+T{J z9=24L??b%(aQV9okDE`3DLV2y_f;$(`j<;)eDEwk849S;A3v7Mt1JdNwQ+wPw@F;8 zNy*8aep6j8P5zB+}vS=WlfN#z{17=&=vX5&b^Uzp-9I-Xb zr?IQ93i7Eo<07@O=aDZ=Q^AuN;l4GnuqPVuP|Dd6hJ2=-Q&9{_;fQM|e*W^HzwN0+ z5y&)Q7&SNGrvZU({LO@gHX#nu)Kg;}~|j}S_dJbkb_FgCBGbkh|&VopW^e?tO?W*g;C z+J~Dz4IpcWOZJRKIn)8}3Tc948uo_~Y$NH2WEbS|DOg>=Or*TU`+P%T)S9(J!`S0v z53_rcX-(a*%^L^0rwz-fWxq-7ZCHkckfa}IvoT9Y;pk!xv9a-)GYp2m3zRQwwqp8V zX2AAK5mEg63T@NY?#jCAjPBEGJ(tt>nnTU%)?Wu~qi<7cddI$G7Y*FOp1s zhG4NjpR6X9gyv;m@3jLex0(-coiE3dBp@-pVuB;pv!W=J2U0lP)9ZHUW?w!>xrg&N z1-H%gs~67MV_?i=2xSvGZtx86wl$qY7}JgZLF(UndL_zS5 zY}>NK7D>doGyzYz`BySIP0MV?!lk4_8f$CuRA2Alu+z`xAzLfjtKcDYwI$whD2=zI zN8`q>qVJfCN@Z>8^QSKUg?1`=nf`$i3rZz)5WP-o9av1eUht4W>EQJl{GYJ?D(=@~ z+)Ixz*X3Kn?C2e*J3VZ=h-}p&-n&%^PXip&)E+4t2l~IW$abx3-!TPHl;picHvi^8 zahoYj`C9v#@aq-t6nFHLDFmpNqQ;stBH~<{uZbhzKfrdrtMLq@h;w!d`_`s98skjx z1>Z9ePvXvoucJ8euSNIgok4nt5B7liEXI`$YkuFalYB=$M1UaS2deq4q>A}^8|X!u zkVVtG)d-(?<`pc5RcQT?oRLjd9Ii>LjK6-m3XGq*`i=fNP&crpohfmxphYA^gV<$* z-yYq4F0+~~M8vh+;#*cjqKYrL+(n=Ur*LoXZ)>bg5&&}q$5Is{=R~Wt{cX$-%uz#| z`dkwu;qIm_YPcMyl+LUNvhyrniEwLc*gm+G++xE7n#@PrFdcOi_=E9=ZbN>#n=-T|38V9o{6<%^!b&r@^L&iVi6#M`G}I z1*lD(DwOu|#xP8wo2`eIRNwK$E$^rC-MY8G59~A>!OT|mW;Lw`jUH^6WJp*L!@H3el#@+k*f9co+^@rJ3_l#6p?DnA)Wn(eH9 z9g%XIot=s8Lp+hR8&9}Z`m$_vbT;3fC^p$07cJ;C#S#+Hh#=P|JnNuGCod=fnsg-T ztYN5;EGXcdWd;E|Pc@&%JxzU4@V}1ORYd81=VoS&ZxByz+{Z zJ+I6{He`fXHIJK`@I%o%6YmNolk9IDj4R&F0E;vpO-?Z;KDp~`^K9{Is3~d05Aj?t zq$v*KvHG&pIKCmgHj7mY&`z?R_3Ut8o&9*FZz>1fXKd#>vK43(t9VrAgH$vdb&CY6 z?A@}xTx{X}6c&!S`x<4r5svAZKLB2?ja|&_ke%ac(bkko~uxGHk}j;!`PIR z(4FDGj=r=RV-0^eLYhWWUWq_QL>y{{1n1-d>&g%85`2TM z0{+vTqpWS?&G40v(617Y8)N*aep;o+&mRJ~niL##xsi z(IRXaH7q-(jwBuGL58M1HjV_4XUkh|J6cSAYpZ3+XdP42_LB3wnH=UtfeX&LdoF@} zMO~~vSX$)!!7HM2)Hly+jveUW#o{fVwlqu3@>1n1DoT2VJ-!l;jwgI6G&uiMzAhrq z2$TcS8>?1t9wCVMbK16|C07fMP8MW7Vtydv#IspgSG|@7vilnGZjvco=!OSt%N3*s zpQ^(~i&=^8#eMhn=BWETUTYmF-CX{x*UM?o8o4E=47#zB-F5?6^4}f@44#eR6F9CY zx*W8~dr|4xHV1#=S-d6cgMU|6b;mFIa)sgX=g#t!D2(@BIXf2Rq+8|n!N!kRkQ+QC zTed%fltbxJ6@!9}{+5LCBux5YM>rwuq{h#vUP;e7v$lb zNKVZ4$R1OP{i+oG&?d$N*wB;2^30<2@>*~Q)h2$*-zh>A;Yc5FIg=sz6{u@C% zE|ikhE*RO1PK=xS5C67Fd~|7xK8s(JFgS;a$BSX};{ijNS>!y>Y0c8$a-a)7zA`L~gPhAMMT-iVUL? zYIi3f4R>OH6ueptF z(6s?El<#XDa4(zuE)E4;m)!xp1J#RJ$xWr+BG@I|`wV}DA8C6(VCJEeM`zOTa^}?&AW4WIx+|aeRXmNnNn%!U2DGgQ7o0K)5h{L zvw=tl5QHQ&>w1BZAIf1hxDD1Mv_s2&;^0V<1aY#X6Kr%%`W`k^#qh=+95cDTJvHDj zO(+(>(1$YGgInC8kuhbxEsw8ODH}_b+6|fQg;2s1QK^5dt(0FKr_`xp!!KIm)*5@T zcN|eJrI#^43bsXajj&ALQVxZHnxf^)Q;l~Qvx<2TqG-n#odkxixuwU>Lf@4ZvJMK% z@NwzDF#K&rx^FVCosRf2isiB$!uGIJQjgYLB@Vz(1S-0(ZK5u*Yg|aiSx_kBbNKEr z{G@(2#h;-xNk1#={O-V!jQ}B12_U?U9QEts`x&v`$FC_FJAce$qPtt0(7V2Hg|v$9 zJ}N{~n#DMf)WYk$Z|U)UsEX3A07r>pH8#tlXM%E!@Eqzmu+pr`tgPE{`&@B{1N{Wc zPBF$d7DE(WD1#L0j{HS-Os*%4Fm*ba7*C1oi6w?J>U*+dPuUSGn|#rIjYB44n{x?l zLpud0Gx~_>3CbL7^#`>dIHl7&e1Y(l}JK%|EFN7tD ztX_|(bUg(bIO*wqb{Upe%%G}Y{32>Sy{IkIv*HlO@k>QVvez^^47mwfFZ2pyxWN+` zO({{-8MbbOR8l?fK+TGjR|`*qeTg4;YywMqdGcTjOHq&KCrPP-FL0S3vE0aK0W0<= zBap>^{ly#}x8OgzA%h1x7WBfYo7|V}Y{f?IqujR{-AGb0rToaX^b@98Etupp0lm!K zd&;ls8G2dLza2Mhb;aws^6kpxcjWFL(;FlY;R|y!4QsevYD=>Rul5^RQsojPrh3$+ z{QhBPZ`c=h2n?LJ^XG#rG3ybvn1Z_)!wCXHYy7}t#YQZ|M_+{_Q$CJ^elNQAr<$EE z2nxib)RQ;wqnni?GD=o#`W-rJy8w(`MOz^d^?LpnI3+hZ0^6XFmIO&h%Y?*7%%4lW zvi*K`ciRcd3fiN56WF(%I0Lm%cFxA!=2ZoG67|@qqWQtH&+Q$}Xm(D^(1SbVaW%)g@EF^&Uazbaha$-pxks~9Z^*1qD5!pm42hZTG0V6=>&`9vKUi|Fc8Aj zN!YxbG1;_0*!mWsm)5zHSxH_~mQX$zW6a2k&O8Ywx1uj(z`OHvu zE0GjP{>uqKTs9*VG6)>VW(5gdcT2lpW-MG5NPgg&ZCte*zD%~+4GCUw&_`1D{)Eq{ zyzC8g^t{c7Ap{$(HP~o9weQ4<26*LNtKVZFq#i9HTCLF(Ga@oelo` zk=qeE(|3v3xsFI5nOIXgBe6x8^9CNs`22s`ddq;go+W)a0Rkbo1cJMJfWch{cMS>d zgS$%z?(XhxgIloR!DVpQ;4b+Ozq@;P@4oMRm@ntdboV)3)m2YDRfYG5ie`Svti-#O z{0H5o-7r~Q0W(IuFAz1e{}zUXse3ygXZ^yes$1A0fysM6F$Z>t^>OSS?$OLBmf9Cq zOOE~#bHE&leGhnx^=RSzldf9G@;2pf9!hCx1+lL_8o%g2^?o?WpXh6As_Kd0TZIAe zN3$!;W(Ot6dTkNS8}wG9mFK&ry2w?w#N4e>FrA*r6Udn$}-ZtMoc3l+%jx*5i<`AMoxY zI?azTy0*9sDRt+8MUmSjfT}1mDun4?Am@1C!PNQ^L@j-DkkTOhZV9c zZhH?-^th9uAdd*s185J+KepW-R>jrT=21*538<>i$9%3D?1x7hoO8&cNBk?bPYDL| z@GEriB`0A$^Us9NV0csSj_Bu8G%{97kP^ZS-3@KYr+A$hw%?7$r}+B*j@@=4A8WhE zcsgAh?fv8dv__DeP{b$4rmupq#4ttWD6;+2m8yjxytOV1U?OeN%r&uhq6{o ziRSaD?2R(VD5+dCx-~YTj(10cian>ku5GCYJ7s`fY<{o3*%a6BB$q?6a7!mm-?;1T z-n6#q9K{cPI$Up@MLWilIZ`z-AJP;HY^A5CKiiuR8;Vbj=z$FeSLYvkVaXM-GX^4KOwPxmc32NsVgHiBT5Xjr$8WQXU94}KbMwfo&FgRx0nDmjP3m1-#z=7Bzp^z zxSXy=2Cp&ad;6-eme@%@W(a_D6D`QnXXwCmaVJ*|zBM3K24Vh_Tjyt4ncJMQR+v4K zXs9!pL;JRbKuRTJGu$+6~+znh`WI%@U(T`;|% z&973nP*DYR~~S*JY{?_ZfrwSqv%}w8gN)- z+E-jaCM1M=kmoA)4)CW$;_qA77_+;(8?T^8+eeB&U)mRAU4JH6q$qVqfqAig<=GSa zc#3Jqd-slf@C?4u3NcYE2VGeDU`REa8RxMI1*#CCJeGS(K@JwL>RCG9o>ty@37s7+ zdN^zG5|h-hj0r5nG(5#1>~k9@?(yRAEGWSUh(927Dc=l}`gkmFNPR3Wh2}Zfb-ZDc z`uzkWy=mk^dv}mP+Nk)hu;Zk>2N4T{h3MRinu%-8^I+wv<;y!gbt>ISl9-oPZV|YceJ{VdsE-kl@j+A^{_hcqD*ZoZw{!ieq`@6)o$7@ZInHZW{1)>V76%7Wx-kD5>o*nVx4tsED z;-G8)$Ov=rT-U8>_T9;0`D|8z48t83L4>%nBEvh893w~!VX{4&l`K@Dk;%cJ-W6p$ z)uS@)p2J|Av-xp2-cC(f_@fA;&erVJro>^0tK4SWyQhz)eho{U7DeoGH?f-xp zl~(KY@iel}xlPgc`M&v0J@EArSz(Z3t%Jp^DY%<2?>ZZqkiF@;!Q$GbuwePt=l#Yp z0Wsf8iwh~+K&UwCW4xmIuk0y*XIp{%kuS9612^=WN4x3q_dgY5zF=9ter?UIR$9y<7nQG1vaJZXEy4 zFuzqA%bOW{j7({pF(f1tp9H@7A~XQZuKEKbUZN zkXUi_OJNBfyI;T!ZU;)HC?xz&^XyQ(PNfD55Kq<;emH%asOI3&0^yQ=YgJg3)z@)| z*ajq<~* z+7RnXhj~GNFz3Hl=)FNy%~$ zOA<@}N|(>C402aWo7A!5sm0&w@5LJ_LUHAQ2zrL_t91IegKN{W2k9f@sj>Z|z zC+@rh>G23ks%XQ4U$4>FtIwVLvuoWsH8sAk?))E-?xo!`aAy#*TEh!K8?r1gkq4Un z_w@84h3D>4-Mo8^D;~Gm+gpwClByQu6bi?pK4A?Ih8mg4^;pm-_5-Dm#p+ZIvGjuc zoGDYYO1YJ>&)Bw=l*e&Sv?eX+5q%*K^`D(YtKU;rb2d6?+Sj@~rW74?M$)U{FeXAP zA*7+Jgr;_9YBW_vl+1U13;^`L?gT@)zARuxTC@j5+$Bw(bE{jVP=5I#iB2@I6vyx{ z6-$E3N~tS)!XZzWyutSdZK!1Y_r|}HhudUleQCy;y*~9;?fqJ*KEoGxH@(HR#_uj| zitKK5jo8KuSuWiLHBmTM1tq7(6qL37ZLplcmcjZ`-T$jk;_HLPQwuqg(I+>XR0-*` zx!a4Thv04#+y8<glwF!n9~0J!Y#w-*OqpQ**YH|!$gjGDJX*Hf&6?8mi<=^< zcU|huXh^gqq12}MrdGiD#=h?E2wnV8?Af~Y4y^-;s3hD5@5q$JZmW!3m7K^Fp`%iJ zYH+9e=u9X1XhMJ{Ns_EO=XHyHxe)B9KG~dx+E>K!`hCp_7IbsTZZw(<-zax32tvI% z9PjItwJ^rdZu$bESIVGw zROZQFhs4w*v-YR6l^g#NVFkDfP?U%-?@7FGK{~-f%@6oz0<76C`IY-1r^c13JYNvh z7QCzLA^(jBH4Pu$;JzrFkH|scrH*%RYXc1lv~BQU@1L2@I61Cx)GIQev+oup{cUJk z4;5kHmNB8=usiuUPMEsNpMJUm--SebCuopemf!I-OtZ(WEo|`+8um1PcFF2nCTJ<-Mc}QEIF;$mb85Z8g}_N!!0Yt-jqT znt=PeJoN$^U5gH8IjdBH;KNiFUy!yz)k)ZI0MRBZcA>C!=nic1`7R(nPz$mCv5j%Rjc=A@N?J0#B;bn^$kBsc`7N zi$qfBIqjc%(ZJaf=t9?{7v*yM4Fx20tA}S}sK5iwq__CES_3 z)py0Qtry0hABAnpEK1D)O(q;tvc#aLZz#c#g z&I1v~+1dG(nF!_R5{34Y@iw!PU=3jN+ye}tJMzj0PVj37s>zPJ<(${){O%hLJ9vh# z#D99ixlKmV`2MzEPJO;lHM_!J6;Ge5dHxTDZ(-50n;3`O_4IvYo?tR~Rbk3HTX;aq@tXt|&rPrJZ_=}7?t0fTy>ttr`{J)WhWmY2UI zX7N00J?)@z{m-&Zzj?u~>YleaQ??9J#tLfe{K^$I)9+0zfK|LdKHG5a*}Sud$NbS! zrnrh)T3ue3PrHzpU>~&<|Etey!Z>d5Hnat=`DWW^q23brrKNV!LU)du!QDKXCRU24 zbwAqaXKMDak*Prd0hWR+Br0O1qD#}br*r!17NCC$)lmLb5`HI%pDF-48sNm?$1o5+ zAHTMhu6kq_OZi*A3-vQjqi2`KB6$b3<@qby)25+&3?z}B)z{7czqFeZCya7OgZ!#T zxGEL54dU2Fiq{ml8qO_s9KC9-XZo-jYwgU(G*h!v z>LMgC)6Q{-%5$ALTLRWC-(6PaG z@uZ~Iy*6_dg~$}9tk3u6Nqe=)M`xY7z0oL41jX+jj~OSEgZ3U0n`_D*IDM7n1ynXt zN~s4ksHsd~s2hNa31i*ya|!%h`;I8KHvfrFW}>e7B&I-}F@5^J)?QZieQN&CmxevH z)nIX{d)I_438hwiZAvD4Px3zudY)d^E&nz(|JftmZ^_VCq9_BHy}A-sJs5-{pw0Ds z1CeH(h;H2XTiLiDN(kjfVZgq!e7U=YhTDGP#?+H^e24jnIjy6?AFb%mv_p}xMeiMT zLsLudCfS~q5{Rk~?}v2NbTpzT<}?fhGIL#4V@pHw)lGd2WBHVPtpf*kH)#TO(Q&1o z3}NqFH@U-EQR4)>wuV^y2`JF|n!yKGF#e;sp3%UHM=;89YY&73U73zx! z(EMBAGsJyj7sti&7I>B)RiIG#&aaTlS*x=Z^Qe5uVGwu7I z34uWEw_YzmJVA&$0B1qKKYxfW_a6*il3;0z)0A|g1%Ag3?Fef$lV5RqLwDm z>wCt2{KfQs^gQC5H%vM}aS=7P+ZyZ~I-KAV*zQfct1Eu&*pBGS*{#BloK6l!aAmYN z(RC@2^MLT@E&SFmgI*KrRMDrp!AdU*x@BYfdb>R$jq)ah>BwQ>mE_(dnQy#a-@C9zrc2l9K-<2{`5Q=&&x_7h?%Sggi&)B`pb2FCu2-)AKfMaMH@ zp9ApDd3Ls?KhW~D=#l_e@urc^0((Mx!l)Bd(1Y3X>)8hAHYNj;zabY9YWhuIg$ybU zh&qP%3&-8^w}y2w_pAv~4C22_l7MIRTJzQNeR{f+bBK0_Imi&@#f_=rT=Nb_i=Y7> zh!zk!qJJfHJiD-cytI_BZfXAe!OS&F{-4t?T=;Rq05b95zqG3rK?V_wvGx>0<%x^? ze12MA7g1?M`D+zrHKQw8`hks0ZB1`8< zlJk@Ak24*aCM+l??jX@OZC%v^Wn<0@%73J=cuY|RC}CBliYKj!NUiJuHr!08fA1xO z(m?r1HU)(SS@z_d8guK7Xbfj4JW3Y`1?vlJ&#XzN$c1W}>s_nnBIJmRGw9v1gJ|6e z%UamJjRX_Zr63ES2e*>)+%fW}<-m6eR$<&30-oqm`-867hhBH)dQ^gIi{DeWxsi*s zWuzjbGHqFckXlks)310loi$c!AN-x+d5r{ zFeuT=Zi}&rLLslbZ*)0*?jhYYl5sQDuw^;BYu~yAWc=z*!NNQtduGd5e|q{7?%1n4 zaEoA^J`~2m08xlDHrssIdZ443y0N|4475#G=BN@Xq7CNf>8jtfzexBQwKza!OaL6X zu$4XkR#Xw7VMl}rQ3zfl+aS`%on#?XH z@gM5Q7Pq=xD?EWJU3T{$^6?yeo7|}Y(DmBHuqv8{tA({Hi{8o{WuIQ z%^dmH!rgFSO+eBxN%TA)T9SF%4~DgOa$!_Kas8F(?oycq<%m$4yrekJ|6E`(r;?T(wtib-D1OyGDutfIEp1u7oE0siC`4sL(S*4-W~yR!abO?W)pzvFhd zLT^IQ4>-|l!zGCa@lS&Y&`Nu;j0tqjfv0~u+K*e_C2k9oq+kn6;6Yr5FW_p!qYAOw zNP2MUGE{=KYco~dQDMcpgRmEh=sm6OfN4hR3ckxxZB^j0U0CJ11)+cIxqrbzE5UxL zI#b>-POc+w57ixU7egb3$)2sJLw7~|9WT)^arX9=Z4nG-UO}Qb`$lvAMc*W7G z1a>*+j~#Ay5|wj25ifiU#w;}bV-2z2*{KhAa9`>^@OLXv_I?|ZwTL<(68!3&|LNPI zI*9u!g%DYpykGyrCy5H6tO4D0m{BU9rT56p7-l&HYm+krWNmD@k)?|;Knj4%R3 z{`)SJ_Iv53l=f2dp)C}0JuCOuE?KicQ@LK6v`MNCtN!4_CcDnizyo;x+t|@JGCxE3 zS0tTV1kZSw+k|}VMd7FPKuAKfQE|#m^D@ND?AK~U)1AKtZ6hD2G5JpVEPkesl+Eax z6*_);?r5W7JH6^TXxI+GY^c4NGTF%q6_0u$1JQS+pIroRKX1mwL~Nu+AhH%pPdJpl z*svr`G15;)4{`d^3@+c)_+IelcKgxpc0f10fe?2=OkpRTnm7K#-w0WcgkmH?`O}xgPd`j=fIt? ztR0SI%fKm2?r#^^kv&Qq%pS6G-@_ygcMo#sT^5#1-<3h_=KsPxU!jp4+<$B{&Y|qJ z^$VIJBZa1tsJ1Bev_DiDfdoY+_)#CxE<;uIxL9@%IxWphmlQY`Y2e@SoQHu8T&XlB z>o`&0^*Y#12L;*>v88T%Z4m0Mjj0xLwh&9$rOTNaQpFcQX)wlxm5EMgoci^OBP5|h zv3w#|QP2`*YB^LE<@93LnxGhDdJSiz-%4LX*Av=QMn3L_1S#i7W|P3$eHST!HPR1LI!u7{Q=RFrsydx z4F319z+E|k#o~`7i7QR7h^U|O4ew5{%3+7_a#H0AXBjD&&dj&Dey0TwC^uEEekI7{ z9^(+DVVjm+qFY0FPPEa%&zpN6Rv`mF9$wH@YW5G6@(5jNf=(bFlW&4ms2GzHl(jP6 zI5zdZrN;=`w*nN`XBb>I90;x*UT6e^;zs}uSb1VKJcaepM<%vclO4|iX3kQ|F&{FrXWgodE z@;@9(Xh0g#zhuoiz4&2TL9%RXxb_2K?L_bLMY`dts_5z`N1fUDXJRL&z3M}+>CC=m zGz}u-6)78uLA()7Y@_=$$|rp}>L?-!n(WEr%!qcM=EFfcH}6RW#%iZ?NACUazkad> z+dWuA#B6d5&ZWnIl7Exv6I!m%_gOOC55_Nb&l)1&ypNqN+jV@JlI^Xee3>#|Mz&%r z)V7-BDD~fAiEZoeRBh8aLW!u4dA{k{#ZrG+$rl)3MG6T+-!nJM>a>2b5WHwq)4M<6 zLc3x_Frwv#MZQQ)x2N@7RkDVkykjC2ana~{VjKUY@Qy0RcqRD**5;rsR3+qhl}ocAYynLE z=!o(^+{4m3A53W35uG!zX|O>vfvXBRP)1Ip3;RL51#uKWrfAk~I*&5OPYmrT!K@tD zlpmy6U~!5tI;dO*h>HXywp8jGUzkNLW(HnYm`vUsU2?mKY(~&$Q)6fCyJ+o{1_Iz; zRuI=V8v4RTC_xrP!E7^0jdk)vLuBAd5K<5gq+scXzU z54RkNkmS6K z1_r6s$^_4U8`3v-?)4JuDP&)>^jR@WClb*5MBqUP(-~)YTor%)7~3y9M*O`!qU6*^ z68=}xF%66$wk;&`rpte0k){AgUq4F_o0>mptns}#4NT{-O2W*r57k%Xv`O(j$-;tb zsjmw3OLfPn*>`HlL-Q1dEL@&zZRqAlmw)uqj)NPP-FLaFrg#!l*k{=K;(DN!tdDB_ z!vdN_#s84`o~e;kbK{9Q*!`jHfH+<6s(8tGQsTLw9+A@MwKY`4y5MYr%1XBs9MTY# zJl1>e5qZ}rh~Wv-B1s|(NO}k0pC%qXYa@Ntt?18ts;o13Nl~A}KiCcZIdQwPdUj+u zkvX;=m7jH64EmW@_JQPhw5K$1Glmb0zK3nwFAKRwX0D?7tc1xEyt|RHAvw=PrL|!c z_9m&S?B`YdzqG6@l^Vn$hwfih4GRuRG{>GM_l!nh{~4`3`aSZ9TvYlIo(iVA6B}H9ddgO zd2YP z08F=64c_1iyxnYpR_-;uxi@8!B3>dfQsyB>*GQ}XD8P1m5JZn3DWEiWBa{v!Q zQ+NSHv+2XgO)Eb zLL8(pyUurg4WNB?C&nSEs49*?yR*22#HhvMDTzlM_35iW@O^ngpgwv}*xXW7MZqqt zbUC7`%otG;5m82J5?yKB6O(N#Q9Z2rcpWA;--EAr(lRUfHK77Rfm7x~x&WINZrDPn zatPvjCed2X@Wf@xLM--7v<

    RnH2`$~A>O5L?6?&uoQ;M2L9G4s{G;MCyKU z$AroUhz=r4mb?h4{hcp*S)Y8@Y~dt$I2w^(u-%NVm+g2^wnX!lrK^ERdk>}mF5*+c zuF1yKWqrntYPo3yjXt10@Vzu%68LwJcwbF3*dL2B7MzwEC_<=%N#85C7^ebW)bgrlKXjGq%?Q5w{;~h!^H*1`8BY^hb7fI>tse~m-cv<^=_JU(29$S8 z-o5&x1Q~JHvvQezh9^<$+Nq`fuyqDvQ)JJ4I9$##C`X%%xO4m%5>8bthv@*V0)^z_ zRgd7x55_7#;w0IfP#5~rPGsw&7tVY0tR$VMEx9RvF+aC4I}g@ZMr`UCS#ZkSyk6Uy zWirKA77dt=TGNcy>-@w4_$+`R#8XI{r`yOFRXsXh@96I>j(V2}MywbuPZIMhbAzBG(o9e2e7}qh)aNU7i;|_Q zx3kZ95{@;rHWM;t^7$!5MD>%mICT&Onwgg1KnU}ky5Xd*VQ##Zplc}4HwNUs^we)M z@{fbBFBKDRa}h8N%REug_prEKw@&5aXXNdi1u}&w_;`P`bn9Sgva*hJ$u&FgK00GBLRhJiU+tgH9g(UeaHn_a0=kfH$qTfRXf=k@cnul7{ zQs4Wo-d?^D1?>-aIDUAN)I_to@>GJF@1FZ%$VaOyVNw8unDoT&0S#J|eeqZYJ<+Ew zTzO&csdUU_PeT@UcY&F1kFcZ@r3f<`07KQF8)kb3HnnfEy%%t2t9+1iWe-fr-QOkQ z3dZG?)<5U)w<0}k83f%GM+*MdE$di1lbH9VpPu9};PfI;l7|zDm|KdxbI$~;q?27J z2hBZjl|I;}JUqv9FjpLLwc(pnC(+cb5ot9>= z5LX>DqdK@e_Y^5;1TZEe?mDV~xV1%y`TY=Q*SJeZ>6{N8?R{S|iel2R%R3Pc2cT7R zpn(v47;D=dL&%q%d>)PqIT^XiCfoEoi}@Q~bG8~^a4M>V+tXsCTf|bNv6T*Fm%9CY z)zz3}s{sTW1pHW{wmdzEbX#6vcn5$eO#R)uZJL}_er{Zr8IaEY@(Dut04Y{pc728v znX%omC!m)J;eyT5pFa@!*psc{ZkYTFO{_V6&#_f0q*pX##_xAYvl!`nOT;IOH}<|^ zkDHeN!TRYaZQkxiEZskdMF07uccU}cuvQu*xF25!>~-G1P@P~j!uaYKErhU z7osnkX7dO0vbH)NyCM!jst%YPKf*0UffBGX3>K_gyFJ)dkH0^g5(-fKSi2w$EKJ#X9=Z;r$ z4ESZTDV&2>8Y*n)HKVZLm{W@wF04u3Jz=9^NmzE6ZgC@nl=|ipd6*pv_M9j=NHoaj zs*sd`aCD92svQXy0gW-B?RP^qgLD26>bWwp2L@fsE6>3N+rrM0i=mDsmVN`_8A5dQP!e4#U0q5EnY z6Vgn1jT|OmVZyi6ARqNs{bqW}^cgzthTu<)yBlr@VYi>pIhjl9b*&5Uaq6(0w0L?vz#pUcaNmxu+4_H$_GZcx zm)P+$Mi*r$nOhV!4REa5ePe2fgQsYUAPxuZnt_^)(g}1Ho1!4p z7>(X-qu}s+*<*!py=kx}gfDXrprvEKXWv?19d)#Z0CFTI(#MeUyqN4?!?nuZfy_H@ z7JM)9kXm%~Yl=H2fSF~mUwd->c#^32HNfCeV?^Z@#CZ%6c66@H&G?ZIuS+u}_;Ze7 zC~tE<3xSSs88&G&fqT0UK}ls=Jk$FobLs*hMq|spVr%0b8UP18()v@9Ac?~77(&+~ zQA%dP^bXfyFU#~(>dg!$s?10P<=4%^WaoYS&ji51$|EEeI?aKP9E-z|-<;)xOeYxSQXHfOhwo zXxK3pOW!8Ua>i9|davz!2iGRf0i>Eh3dRt*D9M+sU#`3BKI}}l&m#K0NtLuiLu#^W z7})_&0sak?_P*SUyg2^jQu@8&zHdopB&8amRPzBRAK0XPXlMeXq?_Va$QWI^@HrqK zKSHqDYi3m1fiK>z2=Ut%g5@==eBN0X09NX{@X^0>0ZZR&E z>X=j?!VgxuYyxX{!PJ1YA-l@&-cnLQgWz)j*H!j~#f(4M>=^ffKa942#eU-PLjSC4 zXg60)+&2uG&0I;?F~ivZ1T&U?-ZYY>>Z-N`2S^?9g03wfJTrzA#b~MxPhb;T% znW^pb2Z}Ve{SOCsd>^DxKq4MGVGlNEu6+UwK#YH4KElQ1-8bJ^T*%Ma2M-bwu;rM` z1_`X5&mNB|KwvPQ6dDSt0H?OM)SaiH8Kv`~(E{S=KP4AabXf^u9}$Y(IUZ$db}NhR zw=#~-nHw)|%ZrUvcT^Z#4!$dxPuD?_X*hpOezw2NdK*8|Sp}*ooS7=2P>o+S@z1_| z<3$}~e&*&4mu36wP}cYn3aq8LuhGJ4+;PjLNW2_Ka7;t<+ghOGa&nmP zA;9BvJ9uS#es4P4;vWp{vv?Sk=cyu&6ALxmR=#=~gd6`8@-iytr5+aa>(89xMG9A; z1VWMKU5eJ9HaTCx#E?vB?C8-OgZG(?-D@Z=ycbk~Uy?@PUyr19Qc{=HY@XV&MZ8e% zyc)vn6S+AsPnNY~MB~_`ak|lS19q78<^D1}wyDKn=YnZ!hl^cjY?9R=M#x|dn~{!s zxZ-$k0ef|C+ylGa#5)SMY(TNQ>x)ynosGaQ&O0K7O8O<_aA6t>rRV$Ulxmt^8$(u# z;xj0Iym*Qu%rY3C94U+94^5m*%#^?M!0OAO5l3s+NA=>-5C;Jv7fXf?> z8rBaoOFgqL*xfbd;NZ}A^Fcl&usq-SgPoQtI$j`5uVZ*H)_lJ|E5q#K^Jy^L;k(k( zN%LeDo}LJyu)^r#u{GOMMnn7tz2hV|Pz*Pe(?l zbV650CoRq1LE4ANEAm=PQxu#^^jvGQ3s8)iG>o|E}E=C-=qH5t?c{F0tqk9wN;7^=bOL z)oGjp+Ooq?QhXRk3yz$ebw~2pV^gl7NnNGy0h}{IhR&G|x}>3rqV&n98!wodQ9I)# z7b2EIyC4$;_$?&ya++^UFV8R%ACAh7Qhefs4EAMc{G}}7%c|}B%bm1ig;`Jdg}U}P z@VIKkedFaEoA{-;=t4;+vW#-qA0ZiN&AqQ7FOZ~Xo}zH*AeVeLM>jSizjky@GpLNr zWD;S!5h?V{&C!M$jP?UR=ZrK6DwabzNnKq3L@N={-rCO%I}XYwFQkru-Hk;EsXxw9 zL=|oJ#cd@M$;<%=_)cnfyaxgEzZJER26kuTK~c(UurGokSfO1s7nwY6YceA((-^-} zA|S+B#fvse<+ls_SCWik;lfsMX$^O^%;gOoUkxR1VZRBUrXF(A_u?xN zlp0@Fy&j)^IJGuVODFbv*9l4Dw&>4iv#^h^F^DD<@`kFJ*bps&@7&O+5RaW=Nr9ns zLIn%UB?_H022xQB-=~fO^s_E*l!~}HN~%1WQqg59%7QOkIVFH}@hSDo7GfvHK+(;c zeSu@-Q=8fB!viYZIE6QwR`~4_dp__H-nSXPMhNWAF!cFG zCkO|RIo8dhv!${)c^RuQO3&5r!c3_X8|%gzW{o0_ctF!h*TN8Nn92pM(9#-t@9zr# zjO0-QQcyG>Xl~a*9tA3UlAquDbzync2Cgmb#<0j&x8iX*;}1Q;Rod#lmf|`t)$TZ) zKC1i)Fgg$5YdFI?9SDklyw#AXScWEoOOcilBCJHYOyH6pcA|aO+KB!!?FH>EgL*8t ziE63g0qc^aX(X$SIZUw6#62d5xrVf&tU}0Jmi9cIkMTP@;XX6e^jpIXj#IiYUQd|!MRlkzsKiEm z7z9j*rejxtG+3|lgf^@lv6xJ(y2)DXooFTTKeMwZ8g)VY*A0hKt|7fJ|8gzDrL&fx zf|~&feF(MWIFDSR9`a#91;9|r+E3_tI2NRzzOWq=Ll6k;M%JK)SqKhX-gxZxi+&5m zFf6VM{;#+X3YT()Z&Q7YY36g1;28>?5jo6(!g{KNX*!$h#f!pKw_P?#fR#Y>^;`=^ zrr^+M3y}|Fg2QswqBfT7bt=~t4F!E_gBFzI6fc5jCJ1x_`+*I4;vZq2j%)M%SimD* z!_){0mlXqVpVMOFbpr9fh1ZQGMuyG-ZtHAAjM9yC&5KgAtsV30#!`pgn#5yaU0)db z4E1IU`QB_AH-KA`y%&h>?@P=-QWFQgb@7hPG1v+W>E2TmDp5d&v+1T{4<_nMVzl}v zK7y|IUc8VZlrL|JEs+CF2cLjIy*SZ~dUkd;x3iPjqJhj+!-E1vKh50nj|VTPLITBf z+X}Pwi>w1^dFB1t-Y~AO;L~DQAg$n6mPGwQ43^||e;E&tR*xg5KpeaprzTXecero^ zS@#4zf{$Q-?g$%6L^mKEikKPIe5W?7VaWYZ&@(3B?NuCG4@`hmPIER%VsE! zkDGe&^}NNxm%&P--ddXNzI5w#rS*5D z_}uf@O4d65YQJpQqvI!Dy#La6Rr^Ar`B{D`cRp*Vo^DR!&09G1W~rQ+P)fa&Lzr0> z6=$;Ya19iht7O}i)B=%k85E7H44OEvn9Uu5D*od-Kbn{sWTr^MxP6u`?D|`# zsL`Ituk!aOl=9o$X~Xf6c%v8xHAXY&8xfDCZu~Bi7ZL|Hn^9X3AE7y0*LpnUix}|0_5F@3%c+df3=()Pakcsr=J?QWv%+NMv8!a z0z(ezNk54|OcX;7C4(5uzmpq*sQtqTpeqlIrW)B;4Ft1-I{ zlV?dhvyWXVOLwj&&iks>W!rskZXwnOuGDllGQ^g{#0Zt(q01GlW^eBS)J|?_x3e^F z50BxS6OVBhR1_^L#I==cb={|z0C{<0@IV(IfRFt)^t@ktC0tYctNTtd%dvH9y}IY# z*xpDT1R5@TPx_n9e~KpV%My$AwH9t=+I5Bu6+VaGuTn)SV+adn|@-r9SO4+e_dZh1nT9XS%r>*s2v+O?{sDfC$$6)N%b*UofC z*!zl|CrG>(tZR&A|49vV70 zM(v6P#;qHzVTNu~Y~9Ns-^QFTt&3YQa9Wv#Ct5LJy7aI8LuI;BX4l2zlPO~WzsXs! zRr_U3%C+0pYoonf&52o52Q2}BZWA-@=!B?0w+|?7K85FD^-^DLo?}O% z0AaXdIVmvDXqr<$gq~+|IFrC&l0Y#F&Y#xAAj?lc(VfwD&%UDp1tQr3>&lN0B{f1PKI4X^Im`0b@e(dnrtc; zbuPL_D*lprI_+%%V$81Ivcsf*EQwgW_nkXlXiNtiB?Q{md(B9ZGMsHQNS!Er-peG0 z@PS-?(#PBI=Axzj#HYLiHgFrht`^H&(!sOJhn3)n8se%4S|G!`heO<7M&;=`NDt)Z z)1jlZAp)VZfS+N!KN7eZbEh`>PFrL6k0GjAyCp;`+l)sny(je(Zhq6W@Lg9K3o5e0 zuX6i8U$l*HTq$>ug?tUh5Pu!M-b(*%tRESbIdM1}O8RLR_5NYkd^ bqVa7EF^Q& z&V52Of=|tdjQdN*VJ2ET`J1GxumWu{Dcjb<`xzIWEStnQP7LZ|T{z&bI`0ClUlli&~&TX>yR=l|bw0t1Td+m}RfvAH*m^sqvbNr=i_ z^N-ly0IlI0w)~Z$E74j)&gm7&oddVhSQG9FZjRY#ZU7e_MAwq9JJW<Rz>gE{Q!YS;E|ZJNdKPz_FPHGPPCPbGKE~ftgg#SKvnccNeU(CP9mg!}o;B;wFxh zOv$w|$uqa`uBwpUAg;?m+4`UdMt$Qa<&G>;Qo3-ij34~_foa-EeCHKfl7P^5=~6Ut zNm6n@z_xkeb3R}1m2sv>ksR3*Cf>4U#17D{+y%3oN6O!d#PG{?weowvE$6=x*m`O~ zngy1!@UDc*&@{{r9_HwZgP*yXsCM-L;#mSj86FFYW%Wqu`?L2B1k6|!KDvvcIR?5Yp z4kqLX$oyr`r?fyKz6LfRngcZbP8FnIs8_@V{>FIS5m>_q!X-rN+|Vhz0pzO-V*mDq=rgn01DU?t=Vv5x$h z>#&ZXe9(8hxi|3>o5lKZohTv5vWS#Fhurc?(TKC>nP?36J;bmxQxEMv0LH5IpyJhd@-n+acHR00K-E^iydnHwN|wk)Wn-%YRG z-U|Cyvc?=1Y&A=XlCk9jpl?!e+JjjGs-Wt*ehq!%JBogL!-??B;Wr)%qo;^V6$BNj z;{V*_#RT2lgS17&rrC6xs0$fccI`HGJ|a{OrGzi@C3ji0hr)A|zb?JNor zD=M2`mK2Ap^|V43I@5oLCj8whc?_-@B?0T89gJp`X+`7WKme3CVPUK<>=&Gt5s^0XfPyY%^c&n2`%=CTQsDhXFx_#;%oDW$N| zlAvWY_utRrHUq_LJ1F}&nX7KkqbNkS{oam~w)UdNeDBc*p$69HmvS3(@#*1vZIgR@ zIiSqZmu-1$yP??s6#bSJ6yA&$nltA7N#fIO=9B_>>drimq%S+}&7Thi^E-@dnPm|j zlv!nxu@1tV<#`Xr+lK#!8-j1><+b*@_$jB%rKEAGUI8EQx%lS1EhOoVR^Btv3F{Ze z>^vFD#Bp|DV#vxAV*2Dw+&wrtY8B8VAG(3TmXnXq`DO6h;@snl~l2|`e z{>d13siJCN-$;Ntic*V|o1j6t8ux z%vb^~efT(PVJtUd{qYNZ@hnbrIMd4PFRd#e+XCa|djT7ECEu%m{X@6hRSQS#ygJj# zTBl3{v6zX^BBL6Cl|A!lV9^xuV+w#xjgV;7-jnu)j!cs8dEgLU*{1a06O7-tzhAP8 zb4^c<+!NDP6}L9B-Arpk877ikDO1&ujbA!0vMB^bq(9L_h>Km2B%pjv%4;=AEHe`eC?? z^t^JSUS9#TOlrnP5%2GpDA64iek1^(-3naJKgP3Yl)yF%1m{+_cYizcMj=VshAw`V z-l+@~b*plHSByeiN`f}?2MEA!v9%{2qaBj!5%EH(hg*>tp>M%XF zaw462M)S<)E;tOMA*Nn1!d1Jg}~xQdNP1BP`ol} zzGSINXr6N{*zF4ufS-Q%d$nl# zMj-3>;j(vsL5QK|s^F?h_WJtWtZ*-R;e*tos6yxODwY%5r+wp&=Y+<#3qSi&50bjJ zrM+pzT7QN%(pHW!7Hjp!<9&cPnR+GvlA(8;R&|M(SU_cL)k$(-avfbnESC~Oi9b+a z)-&$WjN6o-_xxPwuDK=O%m1t|Yc*n5<~L7`Cx%|ahs;pXXw>x2q9yX1_AtMj&b`)0 zW3l5hKWj{R4^m<#74s2Ybz_PJHbQ>dhK7smwzeb6$8HBzC1~Fc01(RXmB@&JSjBQ2 zCbkG9s$|Rh9{+XQS;1eu3ktlb#>fm|KY#vQNq~0YfmZI4ar^;xCHcg|pZ2t+xuc^9 ziSkI_9;BwW&AtWJnFCs6R>4(-)xC|8n+DOaPbBU0eGC{Gx!V9u_`JkLRkZI#)!5+E z_g{MHoZJayu*-b40k?#2F;*hKWPYVw7!+=Y87n?>o_7m*Cs$g^eYDYIXNtGAXnItS zo{1o<-gzM)fNFrOdcg+TEjWK_QP8b3-^%qzrunRO!S@2ID03|akNI*|CvM;C=^e|Z z+28Ibuqk%MNH*!|k#3u8pC6f@>!$gcj&vI%bc|9_^jA3FD=l zFYVn-nzbh$0G>-HO0tUWtBwLIB-tnM{d-Y-xd+Nd8Wt-T$iV94m;EUY?pu4t|tmeru#{TWx{H+dX4^Bk8RN>wha)I7+RHc2byIdCc!p4qJ`a z(<&Um3AM>;rP%E_zY?=I>wUNCAD_Af+2KG7quZsl-p{e~m+;lgPt)&iLYyG=>2kH@ zm6b)~ktI4~N1HvT%G1=-aNCj1>NlX`gve4m>jmKiMmk+a2`42aGX3zUP^ zpZv;8Vuep$?QM{PtM>HONS>D}FRRq|i=#S$sVHL+XJpMoj~yfN$VVoQg%6n9K<9zz zou~Z>SXvr95}hf7_(I&aWYuO9YK271ZcQwTcN7#@L40mU7kNw1vwcBvmEQ(QdG1Wf%92*Tb;#vD_)8{7*0~I_8@;=FXrx~^bL4lFbsYuV ztF2SE66JdK11vyQaAA73UZ`R?J>9AN(q3>*P@B;!hDz+=CNQ?ETPcgdbdJm;@`-2? zsY7p(**9@VFQ>i0V!hNzCY@ss9x;`6mn=Kd^1KEr^yQD;SB1P8k=GcAvF=beQ0Ba} zcpx3#rHE5<%x@Id*qh4lp&~#RA!%Fu{^Kn0T(A&1(>KI_GJrV&Ed!Z}9S$o$S$OT} z(RjLNb?+&J5C|w(uS}W#^UwW4wylUe$AS7bob>?aKJHLB_$q2HC2!9MlrV2AN%x@f zJg1ixjSp^?26HAv?fxD`-*tNiLW*yB&+kzCp*WaN?U4<^a~FC2n!v}>-BV-thXMcP z%6|`hylr@=RNx=_bEiRTgE;H(_*KG%uEMIE_)BK@73GE6?J_)=r@-RnE znwrM7UIn?v!OP3btL+wi*OV7mV*^ZgK8;Ma{P)V9{!$FR$W=RTor$}}j)<;=9qd$$ zC-|u1b+_)Yuo9jB8RM&^hx0%0-u=K@%wS-ex5-FVPW({04{&C84&4|ux=w@;9p%g% zK+O#$uNW<^xIE|l9>!rN-$*29SM#63wj>gmhu(JKcDL)^1r|i3lYDG}$*=5z)Ii~I zkC+W~`Jb_GWFl@b1Rvy@QsWr>r2bm$y-41le_HJTSE5}0rc?ctboHr?a^Ud;~H8eG@8Cp1=Hr-y_T`lJLEh;dgu0mKorcCrAVVYrtP7lyc{Eym!5&o z()06pQo!RB+rT@}SvU|E$TS1~HiI%3_@OC%*^sgE!k(j}U5!Ue2I4j4#r*TXN*5W) z!sP&c&otR?X&NU-w)9}*EzPcc6x!Y-lh->9yXs7Wb_gRy<)1DW4Pz0$9yl=bG&qr) z_o^!Y)a2Fn1(q*0l8t#m9K=%n?q5%`3@Js)Z?nkQcBvVg-uA0VxJcd*mtE=mKfuhd z?UO)Sq{nAYcZE3nwh;|+Z_<0x$n7G9gI>UMA^T=$Py)+npOazqJ4{fxur=%i(s@{t zQ66-TI<&!{UyZ&&PoDm#h8?BALEn3cm+Jo}+w3o8To{#F2K_MWBM>`nK2`u85$dYz zibVfeir?ema2KtTu5Cg7aeq8?o$sY?=*553eY#Sf@cgdzLL#wmbMh)-blbctt#~ zoT7y?^P(apU6?@u92B!V?R_7I#0TH!I%Jy6LGX$G8e-GKcNJ$K@6D9=OQsRB;~XX61d#5TWbeIS8j z&GMQt#kVob6Zt;+|1}>_SugGF(#(Ffr7It*uId*?*Ww@j2(QTZ~`Uv%3qED3*u9FI4`*$!VWXUyXDj)5@4G z|MYEen!#M-2P4>k^V$$_mzuF_2BTf4oTlK5Ev~jbD&3%s9+vIoM`QzQ)J5kMr)$0A zw3O=j&>G~>M17*GP~=^}*6g~+#EY>(#7RK{7+XQx!{tA9%juXR{?N z8Zd4>EE|vb8D$F{a$qzzAs^Du94yxoydorgbN=Tuy6gga6h$bOGFxgVje8mp*LckN zip}tZD*gPtoM_W?`B4ux)oIHlucZc_Kj--(;;F_%Paq=4TM)%fw_=-7DNDxOu&CUo zRFv~ZkG>f|!HHB%hv33ZNO~W`hLS%9`1u}?$>`KHW7*rx&zWpO_uKhmbnhh2-a^Ha zc)c)A?+~hWY(i5Gz7e~mE!E($2uZUu9VsR)r``BiKXG~4{n5^Z>4Jl=qr6ra+X_np zOE=wVz{cO847FuHIX3M}`5;EHsg4G_cc*+2NdbflpI){vv7t5Z{$|TNKtGqr2nUz* zgV?T0&$C1O`hPZLN(?YU#Z!5MTRQMdyCHqPg3b|$Pgvl))4IUs#sbNK#^>W=&(*tX z$eHl5JcOS8J1pCnYk8mz7M93(lgVSS!)T?6g<%#M~9K?73~n{-SPamBSntYDp7>X7aJw{JZr0m5^XixH-zA2=WA<P^R z3F|By`y+YG;~B^oMGFU~JF(r4E(O0U$ym|Hes#~2VfaWe3+?EAW5;=vVK@#Ux{eL# z4AKR~6%pwWkPLifdYw%yaFc-6{P{B@R?Pb*R} zw`~7b`#x;;JeVtfJ9Md6QXc(A^adxB_;G7$e6sfQ;Qhg>+ItK+p}jH-b)VV`UF$|! zx9;&DVl$MGSa5m`xuk4fm$ncz7K+3!Pg#-%xORuVk`0U!Bh#Ay4lg04}MHW0S~Z z!>jwR^uq&8PpH@gd+lf-gb8w5+6mk_6S7q*u43ciK?h)`{)DMX_>Cag6fuFO zWz`=KW0Ojl#>XOZb=%4iKu6q)OQ37ju18tj{HmkO7(;(!CiF-O%^+b=V@qaZc0c|T z^-f4qWNjy=h{~(gT;HO!(R7mXqyUt`{U!u?JDH%*|6ZS@yC1J#9T#5*^T3d|+4fwd zmVSqG##yPL>4Vg1?!^~x9734P%EK2851#vMo`y`v!p3%Z@23=bK*>zn<2XpkBiNBU zMvNg3@^j!Wl9tfWU|&8*)O(032U4=S+6_EZTP1fmXorkI#kpQ|%{C z^Ff&gd-R^Qsh#J*>xX;0;ws-?g~S=b?~tuH&I1-#7$;Mrx7nA=AP;9zoTQdk?t#Z_ zH;_OULuDOBi9dsf#Y;?hr`FgH@va?NpR?MCt={@? z?ML1n>(vu=k|*_31&6>b#U{b0`&BX-pX{@k%JqIQUzQ{TcySBoQY3F=wlD_6G2P$8 zP5aiE&AhfsBH6E#^*?lNm@_y-mtwiLHBqYlf8KC|L9Tk-{w_H`b}TEsg0Je6+`SlaU^TH&?7+RzxJg8JQqcuiBq$1zhM8?+*n$ z`MaO(2!`PU^;b|0u)L%dEVcd}{%&GJU6?=V!|V-CoPT;x!K+;Pi0?*9aZj2(7r5Lc@-^*IFg|+I8Oxf4%ms3SZWn10+ua%zhCKgzPBFdGvy%o zLYB7GR=tGdmj64SRl@hX4@6uoF}7PS7}Inr^eW=u)x%B5^Q4+OEy&Ym*{u-Quj&~@ zaD0h8enAV2H|T)Z+BN!yh{&Cd1IWN#NDZF{I9;Fol@gr8&_7bgGpr|ehu*p^Vdm~NUc>983;`*Z;`w2N)-hPaGW zk}*i_{$Q;w&=RkFy)4{c!<5libp!v>gr{Ffu1FNXWbV~GNpbwKe<_Js_0==x+Tb?uJwBu+><<MHCHo--7;Jb8+0bU9zA4(x-^r*=Uf! zRityk+zgZ8tQv9M1@r1XaH_~|K>G(ea195E+cZCxd~rD?`9T*|>SRiHqK_AcOngKK zq^MdZ8yku>DlQ#GeCUtn1)uXd}IIK;NttrDlIR2`R^;WG}dqyrWN@&9!39|AER?7^#n%}}k zBS=VVpJ%uqx*Vmo z{HoDk?)OVLUM-0HJ)nfZ^ZJ=(^z&X$nDMJ}dN4cV-$rhHOoGui#wj}V*iR_7RDXi*61Felp&D1{dqN*lg2 zthi-)Vhh;klel=Q54lZ-wUz8mI?JisVU)3czUI3g8Ef5Z(tnAwBh=5^H30LE4vReH zyq7Kk?}_Qvphlu-ti5?r%a)GJ(bm zmV0c)tyQLopt6K-S(W4;Mqc5sKbUew97PCVMF5!EFNn1&EF&U?eDq7aqcuErONoI7 zeU9bOUlc3r^sri;iivIr{*M?Bc4+0&PuzDP+y zvH-OB30H8`GfKYHj+M`lU{vAkWxv9edB~FI?M}I>VG;N7KT38XkZ{W6xYCRuOP%U1 zG^@5CBnX|vcn7|_h)D{J8=+NZ`WJgNv~4w>#u;IqoYj=x)?-0Mc07}a{Kd3UZZaXi z(`r`~c6ZgH3NrQat>#>w*ZZKBxrtb|0rXv|dGQ1>$=DzGwzEDYae@~|f#W!`$dlE( zn-@gf=gDL^s0q6{-g*k`qi`5tO%b-a`|`9{{jXdjzrera8jDn>Q=_j(nFJB7l)OtB z1NB^ba=qYdrm*#`(`!SP#{35p8{~6+@r#2w(iqn4lnoPQJ#MQQiXrQ3@ zhna(hSzS=*xU|O=K1D5&JQTw!Mp5=1Kkh+sa#h%t1}mee*KVFUJq%g-nEs($&}oBY z)EzQ8%i*PN#lnePV#POFQDp<*P}JcH%CYu4c@y8*6K0m8w{H0nN!isnF>hg3+B4*& zO(qQ6$3pgTAuG8SDjv%|c5`jpq_vCgUKc9It?!K*am;?qX_R(h?0jJm|4~Ah&|rst=M)k zwa|-ptV___ymvQhFKmEyo7j5lX6icVyjJpPWHMP4`N(y+4ZrcU&d*tudxs}f!m^l9 zK6iCwu1V^{@rN(aY#09rdzmZBhqYPwpK5k~Z5KUq}ZTuDINca*AUV zqm>9KmYRB`{+P<{-I@{jzP5ZI;HR2nfprUPPDrM9iSJZ+M+qpDs;?s_w#(_UjZPj( zD)eG9!?T175V`xh^Xm~yNlH^Ozkq7z|H=ZfHFwB7NsinfqOha;Jd$N1cIJoYb!CmQ z&mPjeG~T5jpw}Q)WueFgY^aE@mTijsiJSW7sc$J}!`S4RiHX>D4DYMzKJDaQk8epo zCTo&0AGZ#eMgylutyYt9;q;Y-Qss0(FBw&hQiE*$vH>d+`a1I1m$MAnYHr<9&L{t} zd>2S%%SDg~PdXSVtjT_O;N^eWR&U$-P&F7Fj`h|olX&FCePJ0*32E-zx{l@T1AoeB z^tw)J+e1LifAPBzeOlZh)HU(Jm&XmC(4kefrOo`CJFxjIl-XgNWg#1`mn}QjyXmDI zC|!w-5?=ZiK5$wewdjW=_n`zoi^1+eLyb!vU7d95TxAz`MN>GWhu+`zGsc(le4myqC`|MTk~;Ql0ysSaVeD1tHx z2Y#+m;47@9sZ($k&w?wdIYngU5B!Oo=1f1P%)`BubD8y-w~?cMiP0f3)2Q)LR6yS5^BjD^@oL!d=fYWppk9k7uhqg**%ij| z$%mPs($|H!^80#HI%pZE$^csy8Sce!iSemSK>Iq4r?Q)b!Md;<8YFO<;itu!+&^vG z842=6%E=JS`atuJR2HuVn1g*sE{S+-yOfKJq5j`o?2WVRZ#T$Yx#4(4<-^G+1~62W zJcY94-ccynNjEiM`NQ`t-t?m=-Z@|0lK1sNIMA~^sz@`haS)7$ahB?x-q$~o>N??o zeQO35rRTH7MQ?h{4=asc%y$-uK?Jq(F#j-lHoAOptfsg1K}7Kduz9boo&UoT4kAZIAZ zmM`~8&s)#R-L+`JGkK!IX7In(Ago1%-d5H~L$nNpS!Ol~B-|Vvv-R}AH@j41eVvhl zLeOXA;L{k`G>&JBq8{|WmQQl@uXMSJnV;0mcTa5p$1UDE8HZ3g=QJv+&gy^88yKMm zf-V_k8GgN2QBi?aDkK9o!j2jY?MekvAIci;3>{TP^z}+`Px# literal 16418 zcmch8WmHsO-0q=MVF>9O6qF7DrIA4pkRG~QQd)Whg`q{HBt@yAq?-ZhZjc7)?ymRz z-*0!_b-&)3HJmkvbN1e|_b;C3*}oH}rXv4e}@4*dMY#|8hQLVkV# z57;&`Z)E_WG=}KL3765$M008j`0ItAOhz$Vn-~)gyQveW60syM_sdeh& z;6HH9mE>W-{eSPw#@smY48eN^JvRU#BKz-yF)|l$0{~Kw-@s(FK1}bXc_-0ZRo%*!)-~waeq+PdrZJ%C{yBEYYh1b&C_wgsQ!%%>K$Bk zLg=L}{LxAL z%&!|x3;Y-!4}X?FK95@bix;gwR^?GMWANa|V(6z6)7_hk&mosK+VMW!H%^gAKF@@e z>KNzoz zT8-d;FF-{3@masfX~DfVqA1MbG5_8n#wsN`Yn1m%?LhX-cY3|!CvzIPd-y`iS2%Un zV9|G}<*ubYCAC*ABvbpF^xe+5?;{O1-|t(%Awqu4Ftos|KY=1R6W0>z!JGWDl&Z(R z8t-u+@(XC6yA#2XdMO!v9I40H0>SUzvvCt5!l;KzsshC&?P`3rqW`zYZyPm>bvJj< z^yL~`N9UkMLbLuH^8Ym`q(o&+kC~$wAKo}mQHTgDHa;z-=_K}Az=cxWe7XHDE8O&f z6!E624|9RE<^AO<;wh=%^G8o=F|L2Vxsjat{pCwz%XgdyZ!q<>WNyO(+DH|PXGyYH zp~|C5R@Qjf&3yTsXQ?#RMb`_BQ%pFvbJ>To4&O+Em}yz&*&tpPH`lu}zSaDXEzwrv z4Hq|8$lA)@-N1O0eLl}RfsBv23^`KuS%yw#r2N=A?sL&;MqzqVz9@^Q*T0W`3X`X0_G$n4d*W)Xb8_BYpo)n&gog+U=%jdm5v8YTHZnv8BnYcRsblCAO4^JBusF z-Ts0#5w2pdi(9P5r@&xd?)=hnMe$u6(4^JNCgpipK|m%Xi1R!JzkF$$rsD^}Lvll$ zLfaUK(e{JkSan$<(|&LJH6cUs+q23*eEI+8X#ALt5lJiBPrfH%h6yX}AMZLLcMd$6 zs`5U}=g-l`jN;9uGR4HN7$>vt*R5DPw43?sI>rA}zTo8mf9j)Guj5so|04exVg_Fd zha#N>?C_ryq`W|}hp+~81@w;89;&0Bk3VF7u8ghj&mx>(Ljce^I|tgGHe5_@L2rb% zA96WnNvFwL80j=+HD9hco+oDeJBe>a^E>l9kOLVg)LS4rQdBvMPzNZG&NY}~vmEBw zqL(Zzw;X(cBL&XwV7e2qEvhwTY=8B0dLSlHQKU&rHQo@o4 z^aKO1d|hU~-%GqrkPW+zxWgkRsKy_lQpN<#eW62w$?pEaF{wPQ<; z2>Sdvp8e`d@?T8`@87O^;$fLp3jZhSum>5=$N?(&w>I)KJEM;UEdLr;80Clc81?r) z?<8)#;-eVFCDR>rkr$CKlAM`5@4P!WJk?(5KoV5F)UKEpZ8ROvc(Y?fOfKy++E=f= z4|=C1dS%!qR|PODBj!`G!PR-SR%+^qPEGNyQFzb~N7S%L>U1?|-Dr)NF!} z5NV`p_f)U5?O?D^-<15Xfq<|e+8zScHxwLYJPO{$EmU=d4O*MFvvfg;sDc5rHey*y zv7!xDqhi16$3Ddf)sTNK?1&X|3~Yt&txlWSy0sta3!n9Gzub~RC7d$fNkSIGJ@tok zgT9?l$n6dm=?%U2ZyWOJ)jQeMCIPHpwNb6`=ooTddBOW5GWGses4N^MZI47mz|x_1?OJ>8H3lxtAp=Uw$swDy+ZJdH*#dNScS2?}2B19<5y-K1 z5J-WA-?*>-;X2Y_1|4K@ek1u`{0ouiai)jF4s@J7ScA6uoJjj_gaA|@?EYm{rOEV4 zU9II37EmdpUsI5fl5X;(b*D7Z9YuaF+){ed^7EH1+qjicj@zyQm4u+_QGfg~-9VIp z|2QSL{l8`V1E!}gJBnFNsw6m4D<6tA!gXjScm^*4NdszoVRFr%nFS@$WFTS<^eV*35eG*xgyt*i0P3jU}gkhQww zZ*IIWQC2{OJ{`C-(FdpNruVSAH#(VGF!nh@F#8vc`FE|(S&3 zPV%Mh_(_mg#ox zSP=Z>3!<>LVd<%czZl{?PlLmJLz?YXx6dErXUb|gBcsknDRjk>60Y9Mf0G{VMbN4w z`0CL>NzHO8C48?O?U|nHc)s`-+Q6cYQ&jJ_=JM!>M?3XL7P~xypgvrk_E&2Jlb?lB z?5|k z^fSlDNo3kE}|3uz$HO^i@lI z@xa%c2OJAb(E%mjsiZi~H`ofoT~T48p~F^I&z#3OH-xp^HD9e7cS*`!H(b=WoV>wl zWVP;k7WN1CZ+$U^EOsX;AUE8uEv>_M;7jd)P{ULyr5%ME@VlZvV_tXgN*?i$Ydx3g z6nZMIU6DPi(y$yB6+b}}>Yg%Zn?s-TU^#ZV!H+Av00YuBtRlUP{B4w{2q6I&e*s7vm%}p402?xtKRIx}Ru8 zm4bqb-qr@ZPTA`cByGRqu}`77kdWpc<>bqvQA`MYzm-(?bT67#vMzNXNJ3|6;A^}R ziqeM}oN%+X!`^5bBj}YkgJ$c%$l~PQ+`Qw%YtI8z45g93^qc*H+`3hHZo3)L4Mn@I z5#{` zVPl5YK9iVF-vXOI=fca`+npE!`9?qgGz%W$I>*V)o~X`+i6-0Mr(kjra1eL(wtzV< z$*z7?JCi@}GIqBLwO@7SIb=#YQRQU_7@F3kdN`gAs_|B9E1d@*cxdS4n|q8=wJVOr;B8y* zO){Xxd<=#~_ImM>%4~18*1Bd!?U`8p(^t|SyV?jDe%y{k-Fdaa!=FrP!g31x*lId^ zX*Nk}WZwGCOo|JKZB_b}X6X3=rEHCLvSDw;D@s+%@+oP(x8z}MTGsmbd4c!Tu<0j3 z@}7IMml5%tI!*){DxzuWg(wqDQjEpA;_KRpi3zHpDs+DSrfsLo8rc#BoF3OU7N0Y0 zhr2hSvUqR5&Qjp=UFQ;cQRR%G&RuzBrAPJifiT!VON$E*>!9tP-CDyvUaZ*&FB%IT z?KyvDs5x>Y4m=leYgB)-e5k%mq`ORA>@yZ4@Wa$f>?0$^czu=P=0|Ggr9*3g*tlx$C|+LA+rAq%glWbXx?*H_97jw&ib%+4qnC*d$#d z_LfnE78j2v$8CUTwq7kJDQt6CE8dQiN<9(NpwMe(v7B?14++q=;82 zzXng^u|Xjv5@}yFx3LiM0O}dcA(Yq{r?|iwP7(AvYL`eobCFPSUTKk=Uv`qCFWiHO zcF)md=sg%c(B2NG*(47hZj00#dS@HHSAjR1p{Ux|)mmw;0-jyuYuQ1YbrfUNLy~`H zlQEG0Mmk~mzlAM43WL#UxzGUWa9~WU7_ittCws#NEl{FAXFWc@S`zF?>h)`xHD! z0r0Dfq_qI8BH@c);M>NMRIZj4bfSWsIZWP2-*Aj zl|VmKZEbA32^L%Rq0+FnK^jaJtqVdRYR3~&Z z5C2%m=!NS8R9K2sc-tkyY+oZAyH7WpFKvfBSOIy!|a0gJD0T(P^>F zokMWkyPPDPJ+6BjqnX+q!qsnG{xRrPs?YjWakkrXBK->$&o=s|u2Tj#J2@{`MQ9sT zc^~tjSmR9wXsrigi7QlsR7Lg-?oZ%+$Hn8kMX3u1MWQ**-?p7w2%*ZkjDk*czQE0b z={BQliz-`OC;2cvw84ndMVbV6?1joX&%^yI9oTa+7vb#Dd~NARVk$fHYinz}l@2k> zFYep>EVL+5S5i|u;q03ED-kHshl}q(k9^12Ofus!$C~`Eu<{Wwzm%(1%$ntcV1}k3 zHwNyCRkJnrao>vqc|HD@ev*kQ>`5mK1DCL83JSmy?ot zvyJWhoaR2C#WB`Vxj=(*Q&%*@Hnh-&c?re-pU^-TbH8n0M>2!`RVb>W!8tI!>i&)NNKK<+G|PvQ&pSU z8!Z-7N`C<@dq0)OWHz}P-BSFcuGV!Qkt(cz%mC%{*70i+05S}NiiURG%z7yGYUYnU zskE~Zog3>?rn-7{{~U7bakeuR2Q=)Y{fHy_{D%c=!B3*fKsV{7b>z!tYE4y`+BiGx zWIxs{&PQVx!m^ZfKe-&bT`j9*;WXNjOt5}JBpvJ#HrtY(3vT{=im^Z$(OsEscFU(G}WSNkB z{g`V@329|l-6~8G+qhQSj{c>QuLrvjK#bPOV-+nH?Gil!5XgUh?lz)n;we+LM@ykh z0ReoD4`S9ZqB98!G71iJxhGXM30DyXXS}f0{!wrgbmA|?xMHW(#~4rqX4~*V|{^@ zd}mUI*-jN_R!B5&yi50~$#jO1$8hArK!g2hwB2@L^5S5c*rda3%^@w2!I}k$C-l2d z)8adxf;x$(Io18n&aN!_UiP(nWhQyE&YKAM<|t;#7AWyR(zyuJ5Jqe&cy`NkPQhL#g6! z>vEw5L8-gk0%71H<1D5#P>pNIH|&HJ_Tq_k5}|g__qx3;5przlHz(|3idUC$felr@ z0Dr!gxnw3$)RJsbrTy0ScBnFXjH$c*a$r1B6^+qc(vxODCTH|1W*h4J46zS4uz8Cu z756e=wj+uzuuYlDok*Sg@bi%^yiNqPG~UEsxMkO3<2U0AaflbMV8H87RPgZ(k8KEG z1OTwH{RDkBawa&(QvW==(ZcL3{f9I-*CQzhVy5l!5>D8T0MrUld&2f76i^0i?bzaI z#uJ@_^#XsOm~c#!F3Yqyy8C>px3flFY;!8Fnxdnldx!Rw5&z~3Uon^2P=ImO8ymuQ z$&_IzjyQWk7N(4A8&pEHsKwzCg8a>o;Gn8w{ksjl(8SO@NQH?OzdyQ| zQG_yKY!v#0z9#;K$Bel}{L}XI&BwdR6NAaSs_S|flz)!;juz5?7)YMCMbzI)VN-(K z*dKdT#xI-i`tgOKI^rN1qTa$cg6-kwidpS zurQ+nOy^oKlgGNLAv@cunwc=#vRK zRVQ48GMnalU3&p=#S5jKlu&>`kXIspw084#c|{dIEiNVy{y=FOSq2@7)9B<8__r9$ zn$quAxY^G^IRDc~0>j$_f2z3nX zoq-{Y547$UkqaV)jHa7rTKuDE5?-o^b_r`7sKC#9E(_ef433P&YA+E@Ba`mSzqjWk zD%*z97ofOG=VWxuB==0{k(?xJR*YO!qH*24nQr?}xnaS7)y0iv1{+{*;F{!?kZv=< z;VVAx(~1unZ`~IOExFcwWA&!*8&z)>9P!@jx^p>HTT^G8_3qE-MLfo#eZga~vydS# zRw!SHX12!CYarvPm!zN=>_iTtQz*9%kyox`(rub~>FDUlkgv4`wYQ>=tt)A|vdb?i zL)u0mje(~y(&w#&VG8AyCn9JG?%Qwvag|5%=$Kp4zZI30@qiystinxFvLH-A4o-xs z(1$0Udd< zPfWH)DN5@df$j$wC&ucx2>iRM#`0sj@_;ifyv4>FhE7A-PpS}g|3D>ms2;j{dX9>S z>eWDUiHQq*K?Wf>KSeYBO$5KJ4NjPg15HZ@j1U1AOQ8#89}fThV@(u!4Nc3o z75aV-KmK<4h^=Q%vDsN{(6VMy(Cw3(07Y0##LeFjzF`GTb1ADP>Dk>k#o+y3x7 z0pzkq3nB5+*E5n%&N8Vy-K{F8IUg3m>>!oH`uI1fBT9Imx$uKx`xuKg8boR~Dgr~{ z3k^a_Qw?%m1YxaeoFNhB=gLW7xfX5`9A}ZiWxpT3wsCWFYacxVr-)L}Xqz)Vs`88* zu^uShZ1IFMJ(wwI-mN|&76V3K#Iow*Ev_)<6Ol=tMXcQAOAUw#g_nD6@+7g z%b{SBAX$yiP628;Qu=v|Pw)t*(9}Y9V;IdmYfOrfcUM$FxWq^ebb0ZYY(^iW( zPxL!sHdl~zwX*Q9CZAbE$IMc@+iX1R?5Z7@&d-I8JFN(=Y>3MJ)*K5mbYJq`-THKX zd(ffx8?$pUNS>iavlK*sahE=nz`&s@#PpHKkQ70ZPI`8&XEsHn^Cn6Z6 z)hv)X?eytX{5Nj0Wv-_;A)&;hZCU$b{G=%9PcN;d@1qce*MzxOuDEWdMXCA)Cc0pf z>XlJ(dZ#zfkwrArnmCs+Z2)N;cFD4-swn&*prFUX61_~X0{duPfr{T<$_1etu<*bX zHb@oWEFXAM+?+chuI?_^OAz@-uxHaXd&c*@^4QQ0{-_+P#d;No(=W-Jey3-0e7!+Nr0aXua64vrPKkf z)PL|No-mH%@1M~Dm15n)9Fz&!@HyGirg^FRWjH10*>@qhXqqI@oFcbk3-u91guz5# zQ>p_`;O?ZuxX??n*ilKlvAo=CH{_I#t%E~kn|WzB%BRBltM7|Ua;O+78a1ON$|pD!A{RpF|C=AD85e zZh9HmXPA_lw|~i%RD4V}nshmbXd?eYgG#cEtj;_P;CwYGQsp%(4JTw^U{K8-?UpJS zk(hE2Celjy>6fPfWG$=aT!|5sxxym#1<~PB?R|pkt;komOK3y3y&ZZN(jJ)oL#~T#jj*4? z3Qv@<1UmeFfNbH%yvt1Q@W_b3FZ;`vZfk8-&I>*X_(r%JGfD5TrRUUbf_$htkWW?G zT;P{5j(GhXl*qbk?Sg5NF#v)bZj=u}>;TS&?fs8EV_#g?IJT_AfX8W?@Y6bX(b}Z0 z0N<&y36E6+H3_%%0ihIOi9lRa9*uZSfZmSSV)30oi@tq*UESViD(8b!UPQOdHrt=x zICj;5A}Oksj*bpU$B=5+3-x)h#DI^y+30TFgE%#`F|?Jdw2U2xoBL4&rB*h1GltB- z`RxWBLnxYPvGuRW+04sWg@(YHs%TYm7goW=&p(S*RD7`?uvmXD3ZL3^-@<_08fF)_ z2D8K}C<}#1W^*tJbVP1Oe^!gKvMP~&x{Unrk%LDen`~jK%CU1&r=(c&LN@T%=w=@X zgqsvBt@rQBIHBVKoLlX>=d3z36S|GATd%9!1jjJ=t{s_nl?OnOP%$ z%mBn2ObnejsYqJiPuD1dxj(OA`<6LqkX+b@h!(-2v3aVw&chPu;(4U5Ve%f z?bE-6&l9E$MXdYsp5$wx9Cq$AEd^-U;}!i0hNSq9cjlIF<2y+l>fW7hjn}`Vk5|mb zl8|`)+g0nd_O6LDp77p+*o0z5&&*oaet&yw>&p6^A5XWe$QI2KE<{*%`jFKQUnr8T ziYSzTP+x48TGy&F zIX-?1wO#z;3QjaH->51;&qvQQeCJPZK|GhH%x0sb`C7g+fiC&7p{2v+-~{j`3qU5+cQ!q{q=kx>6crWe zCmPe2Z*=#AJ9pRD7moH)l1^hApId*T4ESDMrkuFxAE$3J9t9xFgQt>jlA#V3+gp$^OqF2j|Ob8Yt>a%N3%n^@4L{puE|C&@!HZB z=Wv_pLB)8@^OXL(U*ESu_r2#EwZY}&leiZ8E9u?CrB~*$Jy9NnJt2O3GAh6Kx3n?7 zOG#88OR;b5!m2j`+)&wAbz2+_u31KHg(z6s$hsZ(aF<*6iw;vaIWyaoru$P3v%^Ez zUg9=9)md*GdQS|=#&|4uN_|mpr+gxS?^w)o3t?t!bQ?r1T~279lX{|` zVMvyvSIEcT_dLs`XERUabCb=+{O7IhElA_+A6Z4T&1 zo&}_sfG||zR)&_2j@84XR6e-D@J3*|Y--jZu6sqj?}QyHiZZ;uY1w8fwqGdsyF7Az zBX8F;qLe#^kv_9;+LOa*Vf#nL$kT)dZ9|^%7OV|Xy+LXxLGcaR4!18&M?N7Zy(rgcg z_t}x-?-174k+RZWhEld$g6tW7N7&F=&5uLQq{D9KRYO?m%_Q0zA7u((Hb$1L1$-c2 zE#UY$W0H|~ykphN_+X`dlBwJBh^XV>k0N$kMzQ1L5jkj$*y6XSk>%?1KvoIX#o56GGj1N$RE@&Py76MGKrGC?Vj8UgqRgI z@}%?!)+(D!0Mte;jddhL_v9@Rccn%mEinuCib_hoPNTD0TfbhAdQ(VGW#^%Byx0-J z$(NCcSRZ4|9bW_e1QMmJetrJD2W=oTaqc@AwVS+dE>1yzLk<=Q_m`D;T|}TJaJDMW z)d(^pVDJ4UGZsi!>e9j3o~y-S__oain>3&6McA#SlG2^ns><7JvU9!^H;ypRU#M-Z zLl6NOz+c0iYggt69M-UlCeay#fV-b?%B~=VJ~Z}2HNRh=w7n>2Rz}aNtBB6@9SnNN zV{s8fN&FfI8je#z=GU+Pl09QkiXv~tKS`#2d~nqjU7=mfsJz&>$IT0`y*?RLWzyR` z)y!3i!nnYN;5_mHQs9So4#M}PcAppzh`*Oj-9({0F&c#2aQt(Q)z;=gLp`pqh6XPV z8VAfzO`TTReP#p_xDBe_^(I6UX`QxY_R<8_-CIMv4eoba$CwAh7Gg}aSWf_Hm^13Y zo0#KdrZAr9p7Il8p3I|ZKgoLrc;k=hQ_0^-7lVIGEoK-ra|fPnn-0YVlSp1*b-Wiq zA}*vBkME-YNCcz(rG@Kn*ikjYw)g(+O9Y^AZ%u|hj#rG*9_PhnsPNmp5-FeBk#lrx zIsbb7v$4@%O!^pRY4M%~N-b-1a?L$FETed6481D1n=$TY3}`+AOmiz;T_iK^HMx5k zRqVb7h^F)#aheCd_^>2GMelbou@7A&TR9oEiNrq0kV)fTj5=ALnEq(}uPsa@tm|v$ zt<9P`%-t_=pJ5z z?sQbuTobP!_mopnAL#z*0+-rW#kbf@2&btLmw42T>+e6##3t#A*_d*~b4~tZFJ*AB z5tPF**5y<3c4O;CAG)7x5<$sqd-nGqpCqS__L`O48-E;qXDH$@=l7$r!57|Fb@U$!DJGJZUnW{UU(^&!osb@v7~%6*q3N| zBHW8v+)KlIocerORYK^w_ty866i{>H4Z?3)$Suv*co(xfrC9Y>>u|8DstV=$BsJh_ zlNEXLd)k|w5D*mvF@Rbe<~fTU+Xe93=X8q!cEbW8cnn~Xn&lYax;njiOy&Z`#*dUH zjZG}7G(l~GS<(82ZKD>pAvpDe|D+}JC9amsXe&we!*}VTkjq9CZkeyOwQz^>hX1U- zFFwMh<(ut^ILffhVWH_(8ez9anoY=ejBS0TUgXDpMrp@!0St|d9O;+p)=9${i?lEN zEfVujPfH7~#X^y5)UCs=R+pGZ%Z(6um3VDzax$Duiy@`N;s4+chX98Bb=0u!lDzC( zt}a)V3Os}4N&r)B;kpLJr~<zkC}1-I|82Rjl&u%o;TXIsSq|N!#aonZL@PZeTgFf(S(C^H&24{{GbcWj z(PyIoaVMyIpLzJ9K?mLBjRYF>`8 zLZW8W)*4~@Tdc(dP)cG&IW9D{eIbwpKGg1R+yYfvtT#FE=u;`zf$c;s2O;NZw zG1n|-^!7}(vloI{Qy&xJ!DCn>U*E$Kdvjegob(sfILuz6g00AwkK*`aaQ#B^$>OOt zEWM;DjdFL%3)5BZ3(fZhVZPDF7JdD)3>uEke7}N70MHA$p+3*VN4vs{PjRkmUj5+9 zUUz{gKgr4z30ze>FHO-g@%NxdNcTK}&-IIg%x4Vi`IAh_snJ_G9~wffXL(lUEiMlG zo)E}e4zC{zB@e9wO0PW46~7$c)Oo_79{0K7yg2?mx2JbgbixibS<}LMoW$U|8i&wJ zqo(cIvV5STIqUpzN#U-_rVjL0zC|{*1snM=21dM6{o9P>O8_7bGQ|I;dZ(s@d_Kg5 z+bldOR3CLAXx&GRfjwicIo=P|h)0J=af*xAn5&vNI63i95`Jbvc8w7B$Caq&rqJSf z>4YXWLjN=2ulJ0ZQh8ca>r!fQUQ!{_YZ(Ve2k=UE!w0YtYIn9*zv?#`V3x!_tLEDz z@VTbytCey2kFQWzA3S#cmevyla6O_^)w=sU(fEnr-43x z^$FbY8I(aL_XhS{fBdFP)$l09vIJb$yb&R2UESAbXf)&A)rLJT(`w%`Ym^ zr3cM|m6hp8Z3p?hB>S<0Z#ZhK?!8PGICIiI|EwJv+w}w0Vptvafv>zg_+xW9H>sKb z`Mq21tUlz{^@(PHwUbsnFJ7G~lVu5EfN;uO4r=npzZKdtr4Nmk1T8u4eSNT;9Ej63 zYg{)yhFYe?2Y`+tH^6al8Kp@1qK%&RpiRFSz--52-qO65O2$$>MlH0mXNAEi{a9CY6qiDrcxgECh-NLf~}2E3dH|n+zktRr4&NJNVS6 zPdn0_J_pv2L?qYlAeYbUv#VXBr?^pe2*V9R=>^vby{M1N9V8*3<*VbHI&H7h?%1~l zXx7@B>>Hf=UH}}fo$%~a2`OKGthoZM%CFf!%sRNsZ3cM?8fGW)IWE4bun)Y+)(A|J zxc(*MA&@cXMK)a56YzMBvZy1l94}xa>>yfyp<+JOedJ9jrjOh~ANOC@vs(TQU`TKN zLS6hP^GWAR!z&XFYVZF*ssW%)B zy^2?)@Q3+*s@&stuHbGelTc=ha|ZXN1Flu34Mx@$mL8Mg~IOm z!gj!t;{k@zv*A293Bkm)hrT&gXG{~*cs_w(nMrd2%ux6=)+CxWh$ys7Rboq&+W7Y{ z#^Ke#;I4OS4tB+FE>d;4jQY(s*7$Nc!}}}ZPzleSzs?Z~N}o0F&Yfm2{;;DQuipn`doCMeO@x1NvM49N3ePi93v{UYXI;xJwD4D!6V?w*xa zkR%2wC~f+4HClEPco@fS0Bf++`>Rt!o^y?Ro6V6-AI!njWXUAOY!1Rn3DUQ3--hxX zOLQa|d6H4~{WjiX*!rGKf=Q~d^ybrp0P~4(vo$PaQ2jFMpZB*Zy>Qr;lSaHD=+|HY z8OE)mkpJJAgF?OXoL?GY+gesQ<(CxF7I@#Qt_B40=vpHhZ24OhK;+(T?gA0aAamE} zf)0`o`_FaWDDZrgeS7X82w{-_g-(<3J30IZI+Ugrc&c2MLJeKiog{u zDKAgb8Moo2dv|cE#c|l(azbzbrId#b1jAm$=Z$IRYG`vVG|EkZ1!D$Yg8a6mTH_LN zCb)92hZyb-INH8eUL+u%sRx?Mz8c1+Q!yFEf*zV*#p_EX93truc`OGNeQfLc#^lH0 zz|~bPtOgXt=1NdfwES5^H;ciJ2G{kVgc8Y~RZVV&k(o>G zBKNEUiC=6ll{mPLg>&&ZOAd7f!5;dBRkDt@SW{vMRZ-22lOk>5#Z`PLU^9^V_iWd| zI{aW(BmV(!?yXa-IfDIo|gh+FE0H8WI5vN5|KQ`TA6+_q6zp?RypgM z`EB|L6I@FQL_UzipJMCPbdhe+(LG0<!OQ$wqIe3%W1a%6cw%X*SASdv%eEQ^EH43^2HGdLjnoLKhn3SPgc7wjQR=DbRVzNLYUq)YGa z-Rd~v>Os5}V7zwr_A?+$?M>jGd7O3sG2EZl^Av@W^fp5uD#yrT^qaoJBc0-=GG&X) zCn{@SbDnAvFo(%!nlky%?J-wjN_@ya-;*VM9X)&K0eM%@hl+X4KT4AF{dLqcf4YK`#mQGA$gEP>E-LUDG6M947$GaaiIP_jG|Q_S(37#yZY<@ zwD;?i0G*M>^}M3A^40HjZcAy?$H^<}PmQI|ntIL$5f=1ZB3wv-nAt zG!+p{g#tSnIBVZk_;TAFsQ5IRz-L@*XB7c3{Ic^Mhs7$a*vxA$FK!5>9*je;`sV4a z>g0VOjpB(00XBi_-kP0@n_1^509o0pdfo6Vq4#Jw>4<#d$|>1agsRg&gYR3GU$e6;(SQUHaMlP^)S4SWUSj zvhKRVT#T=)tLvAprg2cS$d{--5^p#5Zup0{9LgKV(YLSTVT7P^GNjbWw_DtiyL+?% z0~WR!UI-`l7WrQ+?Wh7bM!(!uK}F{r`U-WCf>IPnjs%787l^MM+OL0Re;5DjOYwDZ z-L2T37dRMEBZ$cmdx-KWL0XngOa2C2NyjW zKd|Z=Nc}Yh$9VS`E#FIBTi;3qNAOn18szcl?ZT;K7c3A*Vv+Zgk6t0Sf z2VXBmAVs6K8w)qs5_=^IITng6AFbhS`KkU;yRekXrM4br{enyY?o5Dnj29^i_aw$o zYrhZK^d|C}++`x0l;8%*_UB*t%X#^9ZHf$MN4`q!F`l;s)2e_G%D;YDJAAw6bfRZkF`aX;I|>cr z4(rpyQzsmVOB!mPXmJ;ICEOV{hsXTo(v9@W(Ss+^U~0H@mjQKFj%XUDdQ8O899B>Y@q5C8Y-G;@ZM*zq5grx4VoL6 zE{f0YySWY{C-H4ldS=v5LjPB3C;RGZ(sSzk+CT)2b1(xIarIL-U@cZ-OarD+1iwV^ z+$C5)iE_gEEd!r`B-2Bx={^6N*%vql7U@+LdN3dcEd`Rd!6GHhFJ5_B1jfuGhrpIk z6ZY8l--qyqLXJT8KZFRTa2Pb^gz_i4ptnDoF={i-%6wilnw6YgWmfim*oHYG^&|0z zhIRQlLmr(_S|##8inL-$#+KkdvF5ENqkEggA(VnasX-6Ybwu-e|2YBhs*5FxcYc=XW{E% z;PVRaq2YqtrBL@@RHAzFd0$D*#306GEk016S&upNB!`|ANBqTu@|QN5juZ>9vq8{| zD_mFmlLgtUEIOluBijGT^od6+n4=t`Vi^!k)>?5WZTV6#gF^aXAKVo=4yo^FVYwFY zmFRI2nguHsf5O#;=^~^m8O9_X6}s-4>0TWK>)TwO4gdFG@wcT1EEj!#)QGYPy*mNZ zq3a43miw&+7Hp>9aaenao@bFQOk`vuBF^|N=UybZZ41VJPiW`#ihOMpx&m6krov9T zOyAHPn1eed-*-ntKHU@?g^>DT$DCYYa{R_6CYUmrTE~%MjFE>7DHUe?iUObSZv4BB z6L@oRXo1189WC~k^uOVfUpFka9=9-2aTyeRVhvDG08(41_$z9P#kyUTA0rP3OlQSp zU!Ad^dAHbDtB-q5^($l`@v)!g2LUgd?AJA0=jOls|9ZLJ1dfk2=x_R7#DfpfKXI4S zb+fGEG9D8Dc#AHOIcUokr9`v17W(b@8ywfFz~ VhJ}Bk^56}?8#xtN@hj7y{{gD*=nMb= diff --git a/docs/img/spark-webui-accumulators.png b/docs/img/spark-webui-accumulators.png index 237052d7b5db60539c2259a1ddb5c949dc9cec7b..55c50d608b111f476cb61663f1565c36fbd0fe36 100644 GIT binary patch literal 160167 zcmY(r1z1$w_x?SMbP7laNOud;B_$xGgw)V2-K8)Bf^?^ZG$@@zcQ+#4-QD>f#OM3} zy;7vv4 zMM0pFFti)}hfm23rIqACAa`022)=FH$WgKb`WSw4+P?m2Z0D|lWN`z0w18* zN^3ZPK=?Ed|KVhms1HCOI1O_Zbw_o1IetSMD^`6Y8v|ojS1ViKXb?!yl^=L%W$dU= z?rLRe?ZEFUMEUm!e&F@P+iaBNe-ClA5TaCsjqBL`KwB=`Gb8&HDb>U>Su{ULV$;ZdX#?HaU!NCF?!Q$X%?Wpg{V(mcn zc#{9lBVp`dXm4)oXl`Ro{%~G>0~;qtAxg@J3;plkV>=zqP5$4NtR4Q{7I1@X51+8T zWMyaj-?@Q56?}M?U(w#&7`XDo`GsE!{yp;l_ujwj2(mp~{QorO@lOA~3*4$Osvz6{ zx=k2Wu7yqm1h#?6NQkPqPVFQkB&m#_cGA6u6w_mqe~}cQna&QWxaE;g%$E5LtF4ta z4515=UEXWf|I$7?Yo%{DF6n$XpG=k{_-3w;C_ic5RG9sGt^T?&#l!PF$-|nOC=lG# z7}X)Er`kQV`7^EE9ZHT85bn zx1NQNs*1Qnl&_z)2@m@!HplWjvvhM9$w96BR6LRDJw$+)=^;6@OmS`$Zvy#9!S&%? zDS@#L%J8UsZ21m?7Hxca8mF8{EAhzLucdlQ)>ZEzB36108|E&U+4HmRrCXx@g-LHm z!{U`cr2VdS1kvQpa)@f8c0teW6&V66yp8Vhofb?llt-9`BfA%EHQv8Fq0p@pPJQcX zGLeeL5rvp%Qe_sXA5+fxEZVh{sY8e&nIcb4TA&C|2J*~^dtc(W(|~t+P>?sys1hxw zq~DZ6iLq13ZwC6p1LjCS<#+jWTJQ1`C=;7u(@W!2nBPe4^vd04y)(`=MN9}2cRMf? zpf51#nq(&TJ3_X{3$%zs3a`(8i-A8)d>J;Wy_`w|y8eIN%Gj5qLp_!5T<)3q!V{9KW^Nw7XF~PS*)P=7 ziR=ngO^H~HjrvdhQxz6t`MQmBD8-MbhnU^<59(Hby7b0liv4`!oVLKb_4KaaV|6o) z>P;{Scm4tAr;A){s{eI_;=*!NhF9IS)$hTB#f@CEpRJeZFKTYC(9ljNXg!|23S-`i zDsye`QDssqoZdm$HF}tPH%xFkwa&XoS+eWQ8l@{fJWojwq+awja_Zi=G&cJnVUWCX zJy=M4iSEEl5_zIfGWVPydkG_K>Jh<;odm#5x;LH0rMyq10=OIn^`E zpt{bW-K@HGRL0C$Hu~Ljdp$E3gU5MC-ElO6O3DCfPeiW?2YVsg<2BUO$vm}6uemf! zHW?&9q5zG;U64d07>MVwX`U`>OMgvJ^LoAEWFo15IS)eRDXRCh2{*- z%0Uf1pvO9&O-3&cm2>10xGctIsvXu1$uQen`qkgjypQSKdmM+7(=X2arUOX{oTdsX zg3rU+jmO))^|%T&q@e;;-&Y^UKmm%z(zIV^@czeo!`)3=p0ep+3PzDurR8KG9@B?x zLCCzP%c!dw0~!mgY_UeJkQE z;@hc)&mMt_lhYLzRkY4a5L)AP(q#FZjhh=S$&Y=HlPhCI&x`FXb7#CBQU`L~QnoCx zR7}1j7i&F6AJFDhB&gF{F96hdx^$vZX1itlMIL*FCmXc+>0;48LORo=KQV$3`F&G+ zxD+FX1Z2Nz!}dR6nJ&`2J35+h~&S14odC zPjfeymWVob$Hp(gkm-k&-!%ii`KbXVs5>1wnJgn|&D(rR=!(>3IE7^*xNV!VWFapn zWy?9MS;GEQccQ zm-e!LzCM%hVYkaA<##lTW7Xz&+8Qm;;t6Go8?+?}Hx~Cp?;aRfM=j~aLvOT$S8r-{J6LF5Nj$i=kI5`E zWc2gRN3G{S2D(9}h(<8}g?};dp&wMVGVuW`v6~ZX@DyMB4UpSkV7Xv$0=L;7Sole`v4U<9r$@Imj; zIZx=vLF`eU#W976+__T1`OWEmI&8PgS|GJ=Q6Amu=b5itPw^<#u^oLc^}=l>qRu$= z^I7$FVXZ_Z#W6gdz+oG%PuQ~l-LMoH#l|V>oYQg$&tBvG-OXsd$N8YhJ*4V(Tlb#w zS5kn>q;XukF$h{jd7w~}FWJRX5ZeXrR0m$@ zyFlwX{aym*nkBu1`p@UH1YnIK9ifJxAAz0TqCSXr#+Typ7u-iS9TE0ipOjS zx(fqNMGQ~+oDtkNpu_Q6!y6vvv^^#`Qf{tH@5l%ahD&17(9jst#78cyJ9>-bUl{dl zT)#3tVsY@(y4qWplVPc2$~&VkAT1sz&q1N{13C>FL;1v4X*(fAT3BiyE=0nKX^oAXqqmwM4`Vljjt|F5493LGOSc8Rf4t^ktgcU=a{mQaK#(G zHQ-Ii|Wrb1yf%h+ySx; z8g#eD)A+@$MRxJPc2ma`RL?6Jcvm1`(pH7PG1%_J)>Ef3+2RGuQRYk7 z6AO4q7A$6kn+DhL8j6QWuH~UBsL_J)Nf*|WT~cz{D$PiWSJUMXVPIn#c}(@C-3z{B z{NVbYw5`~y#?@FYTgH|lB+tcwE0?sPU6kCLI-FU3?Km!x)+fgNl zoc;-q3`!imDqt%P#lcxn<0uDh556?)(38UBvYM_e__ilWpbhWS{ymQKOSvR|v6pJB zI^VV?KhQ<>%uT#3tvD39&cq(NyWGsNkS});yogGcy~V=u)Xc9UK;qWb^6-ROztBhD z30L+qa=CI5C<>jb+Jf1?HOs^nAh)>l@8DE(-fAVx;9v`1mbv@1@VT zjrX@$wN)|R3llSlv+a@^H&I(PH%Gmby!}p)Y$sy#6py`XUP*_;2r1GM*w8bQ->gYV zV6oynR?ij$cOVS-f|U988}A$3q{=qcxB^7K8q54f^yMKe$(N3e^(|lSaRSb_8PnJWFTVqtiip+4>r_K8eqP)~kiE z-B}fQR%cNA>k=z#la{%q%+I$;VwF&1@A4|PE$+^Ef zZv>_#=}sKGf~!O?^Ub7bDgp|7?A(G(b-<)*661|8Fn7(VHGfX)^}=3>Lrbzv#Gv}U zFm>WXACSY>-3mg5#XoPLMU8jvDe}}!w!>{gO?k-0>hwF&rDkz95OhX$o?jr!CL<08 zxa@TLWIk6dns=|cMap1MPQG6t2ry*YsFW}2qG+!Haq)4y(5k2-v4D+NdpU2iqvIf*0NtP6JUO^nnrz|xleG!kIgSeAqxg>M}L^QeRvn7LEqd{AC_?HZ90dRNb zRn2>(pAJNqyPb*8>A7I3+`n5CjE(TQR#nn~>9+CC4xW%Guc>#Z0E43GskAFcXq$cN zn3m_|D#2cJ1#c(_x8N=7&^wpDg21UP_VH-6?QLa6A<5)-YQUz*I>gID4Cj}M`#Djd zTReQs;Aw|*zv4>Z>@a0hAb7Un^R<0)@r3I!kJhu=C)>nb8({L;~(!6A}-_o!X?g}3eYmQ z9HtD#47l!iK)|q!h8N$u?+pLaRk=d~ja|z*?>R3Y;>Wt^wCs3&D_vHIiU-sOK3>8I zj+Q;%+R&dtk=Kz9;gBw+;)a>w|=X)vLUvPeB` zJnCTr@|>PwTk|?3PD;a)lq)YHBN4e$P66=MZXY60i7@y>{)FBsY_ zCWh=_XA{!tPfTf;CV!=3Ra_)x#_Xe=piBJ}iR-gpcdRhuH^{q;e!S4`rVfc?X}gdu z!;POt{%l!gSA$i>qEi>ot3`3DbTdXyNjQ~{Dh;S^PP1U<2oI~ zp4}__y2Lgou<{TXJ>gkBwIzodFKh#lr2=^orb)T?xBFhL@9(e18x8P3I2tc01b~il zk&s(0C!fg4Q2c@M8-nQkpW0ZCwt4%NGPb(1iH{n`XOWW(NPJBqc``8a%=g)|mcVCS zbTr0DTmpd%8SpdfBX7lF5Z_b^e8 z?rob}wFz7Y&je}nOLc;@(s%OOZ8jPNFF$NwWG!eFgAA3{&}K*xd#(7$#^9Dnv8!E>J|AZ3v3#@jQkyA`wfw$2JvaxnTI0Hx7NJ*oZ5umh+E zvclFrpTRvqpx?-DUxfdg=>d;3ovtbC4V1#>e8vGQKMW%`$u~Nl_TE5mMNKaB0?&cI zurB(gN6R*Q`O*{g-#(It$9>)10caQK1gqH2H(T6w88)@M-01QAs_OU(FfswpG_ReU z`lUdo3os2S*T-izzS}^;RaV+BX58)9@^12_H0+}Tg1=E;i>vZBrEvFC%THn5#gpZH zfdQ|_fT1m`=Ria+0H_LVzu{UMZjM#aq_1oxR?a>+s>k~YcVFyHD!~)Jwv1ZcMo*Vc z=;k!@;0kV{{KpA<4f2I@>N1t(w;>K(NC;7wDI?V0EaL-KfgRk^xC?>jk?TO5(_G!w z9nl|uNkcguU)qgGYhkZ&Qo3OEFY$dKT~aZ0O`D;jjEp-B^ptb$?231IIT&{zb$`HW zFwh)ktemmjmPseF5fT|CC$xSv`$E=O_!D`Lkpdine4Tj=t?z=wC2Tg&XNSIY<8!f} zewTycyP16S@8t0uCLfJ`5Wcm!13Sat3qDM+Bv1}@6HQulz$Ud$-%2HVD&r}-=|ybj zd1j3|<9IJ6!oQR}@D22qkQ#_jg5aAoSxzM~9~Ju#&u7>nJJTF8IgXqVXRKK5=T!pkp7k2m=?0#u7u4xk)ua9!6!n)^5#9$w{+!e3ZDXpKh- zaHIewfc-rb0lf2&`#)}O)&Fg7($LRX|Np;eSWI_e8dB`1=_Lkd;5Ycx=sA@}_R)ri!Eo zb(mZGTHxPZ{sh4}qXCVh$A&v2SO4D~Nkfm1kCio{>wJIV3xPk#UjwS0_+Q^t;=y%* z>+Dwp5m4|KG$waVdck$h1;E8Jk}`tdm?hM{CNP3#Rw1yNc#8kz(ZrZaeZIy*ooen7 zBco*ZTsBe6HLWEm^=EA0>?@g6jG~0_*q-9zMK<|U3)<7DpB|a=jK?WSKS{-|6r~jc zN6!Oay$KwD9{B3@$e7~KFYiW6V%_$8>fR;1(!H3Da?a_SrpXkcTcaxgU@i`4UOIZM z#Y-as6qC7~ByrFV449in41-2ZImU4V>9#)KTwSoZRWe;& zV2oKkPn%c=`C~6sSd0&(igpKdlx)l@I{d7(vdiq&55gmF9%^nLhhSR1o&Ge@)ibg zDWg}{D9SngJ^fjXvCJA`6|U#Dyglpm1O5HR0fGBjduZA{tV2U5b#;P9fn;GT|H7Up zl(E|Vbc<0*{p?&x-HiQ1LQECJ+5>9pvdeC|*s|IiTOu>T1I(vY5;Ou<3EM>o$x74n z(3$Y*w^5+u%KvWd364qP7Z z1yKs@HD{pgcr`2&o;RMjVcFChd*Z((E9FMjL~o-2eL$3W-#}$?Uy7ZB$E@B$c@Ua4 zqs`!GO)Kl$5CU7VIpSX!t4L07Y1r)b2#p0Ms;F9@A8{krqa=aid_igz!kf$E6>X@Z z(5gk!g%$Ytn}@aW_txlbl7_T@Jb)$@Fc@MVy+*3{zv!?U`>>238Uz2oCIM^r@8>AC z!{;DVF(_`UmfD8jd(f>?b+vTff4D6i9bEC&}3{;LPD(5KtQ3SYN zMVT^xwP{y41sXmxJsrv8t-(}K4q{n{njN66D8+^O+O=muCKn(<=}%(c1?;~eLvl_I zuwMD@dO{1w$b<9Ys^NDSEu8; z>wXS{LN13(?ZYetW=#H_u*Z;VFA41oe(tk346|DXa?AbfRO#+&Knh!o<~A1rn2MvG zCO2Fi6^Yc9yvk{Brdk>vwln(O7A*J|Si-~1;9)R)y>8{;tZ!Wa-FrZcE4>Gctr^{C z=|&f(2X9A`8STC+Da}T04Ymof)EGFlKEV=vO5GGdtmi1A8_TNg{#%~M?ihgpLhea? z&Ts7jAR*Y7=Vrs{k}`CN^AwLNIpY}$`&Bnh90UwE2Y?793?gc$P_IL&UqEO2AY!P; zrE;2vGS~+YVZy@z;!n?>%I@d|SA`QIo8)N-V|1_w>EIO+Rk-U)KI!gtR-ur}(Z4_I#ll_n(T zoVE+9?2iEm;M?~H?@tWMHqZR0DB=NueL_XFqwLM_g`YU+rc0GD0CF&_ zopD{y%dbp8u5%3xhN(+>_>Yu?vvh&9p8iBGPBdH1kg+*ddGz@vB{iFe{Y3%V>kVRK zMuano!M-+a^~3PGzn&L|YFHR)*a2f9`sJJ611uo*x;X@R;C3?-*{9VQwFy-mxxm%# z^;ECBIgkB%0LZ1dZ{-2MX{_LuZ2@l&-0n_SsVQ{{N469Zj)lZ)l^UT$CKF$sZpS4j zha8@T3hXtJ5i>dJ=#Z}9s*y#0K5Qpo__|^bl!}1Vn~N5)`lHWWy@$iC#ezJH(ox z4|)n0+EF?GkbPl^+^}n8DbVMqa;*N1oEa*}#eE>N55R3505r)3pf(xIl7_N3RWtUo zA3orJlmSRBOe`2jg<$OVK@^VSsM`&u2xUbYz*&!E24L&$Gyz02@)b?J@o4UQdP6~A zU*OeF;IWwlu*1zpMp!yq1X0bqnGvtNr+Q&vV94NDX}}Pm_CT#4kyfnB^0MJ{OiR-m z5TGT*Piw>Hq`@mHPc*BnlP{Q-2w(a`f2W?$do?N<${!|L6fX9V-5nqyZ{=&0Ir<#w zz5eJRe$WHpG9U76a$bZ(I$UUr2*|$BLw!H9j$OUMftW{`;))Z~(kWIn-pv?540izB zCvbVv_n|k{{)@@`*S|r1kkBPj!Q(zo4)WTr#~Aq@2W+YTmK6ZC=2FEPg+B&f*tKE% z0|4%2Cy5X0XrtG?G>7Bvlq$TsO2X;qJBjzhMH088TI3xyBLEn`lh79*4wOZ0ctHTl z+CeVfe^ezAAc9%2r3d6?VNbH3*%#EBcR*<~!)v%7b3}^Njam-JZxSfa%-@|=rRNg{ z0(GG7^A81@w*Z)2p;Kz=d5db$1s^B8onLk_@G}#c9bYpR4*<9X^z=7TFM$e<1S|h=(Kqz~*j-{hzLM1t=xn{?>L;IY7tV7Mi}~(jQC!)qou!?TWX}cu8RPis`pvdEz~g`6d*Pyd~lCyUv0f z()HlY53e3rEZ?AgS{>@|gR3n^-DVxa^v7Kop=z=rNB{+e8bBqXVp1=C?df)+SKmzN z-CmtW$PcTNd&=e64maMN76F6|K!`#*zj@|dsg%C&+ac<+>Nx#TJ&>76iAa;ynv@O! z8DgxVvf;puVGNlV<4jcpk;Hygm3KkW@5TV^Ka1WX8t4fycof6I@Cq8LE7__`i=Y*zfQO?x>6le&N{Rz_Mp6biK7b^nj zxD_f>wfkcs7t#ao`UYG3$Eh}=5+U0S#@RPmo8YhFFF@eTA}kg3oLsUvL|OEl5To$@ z6bE++R4zIum^KkF0(L8gDHf1wQJXEe=CihQDbAW)+gCuNXC2VNF2R#a%{a%tHB`IYHzVJ4D|+2*m#LzNY@Y8 zrXBCMXePV05z3|pq=(WrCt z?Iw@pAbA0QC>~(=3wbtJJ?;wumeKSRW<+8s=;5m|A9VwoX0NMFI&D$sAfwWWJi%=U zr2y<(JO+gCY9qr3p72oquEL#>eDVQhGPmxe22eF!Xj|agysBJJSh9^s9t5D?FhQ8W z9xovWtu-w*^J3fWQb>G-@(Y>$e==7EjQ%X?>sJp;JcG=MFIk^^QWcssd)K(S80Uv& za8M3@6*cs@^np+rP>1qXo+81mrJv;fRuoA#WE-*~pW~fLL$*^#l#%Qm0z-QL$~@j0 zyfaWxP$&sssoU|I>wy^b(AGqk=~b8)g)2_!H^IKN8`OKFqI}5-DYMd7-^y40fM~4S zqLzd7gqJfZT=|<`F{er2(}Yh;0d&l{PfEFxK-;^p=i8`$y4ZRRdar5i8p<5vW8Tj~ zo%ka_x_0e&$Qp;ozZC>9yeJT|@zXGBUT-666vG`-tUQybld#sy3BlF_zq4q3r64s* zQNhAy5v!2Uzk}}kfHTJ$c>?8{TF^S+TB_*`E z$=T|eU<27Sj8Ja}y&+2oO@m5s5}H1qleVp#D%mjXRE+*m4aT$-c) zUHgl$ub#Z@fvDqrAxb}oO;bN-U%%J8@_se^3vHpZEiux-`UsQC#M{;GL)COd_@W*>DUD@)}hK0_{l0@AobxZzae|$ zgEc<1ouRe*ZZ_Lgd#kkX6F4(IVfYmP9b46P_Fx^5JJJWQ+sY~NmqapzN=UYPQ|l10 zjDm_A0qJCb^afcBYS*=hI>6^Hz@aT0?>jz%7c?3M$6dpRN|{G-g%|ald{rjONj?zE zM-{Q_yR`65azR&{*H|s}p(OO>GmE$HEKSjS^O3t&1ibEUHcW~r`bLjnl|GvZ{>Ap3 zS{s8SQxwaOYY3%y(=sXJOJZ7lAM!L-B9AANwX2HI-Q43T`J9gzfnW>mQ-hU&0}t| zwWkG1lyTD;Rt7e_!za55&CoC$A;yB{H;&Bv&9N1WKrh?#CB3dD-kNWKL zUrdLK9K$GcBxG$q&;N~}uEe+t2G6+uv>V)L9=WpxedlyzuPgosr?RA>vXb>O9$rJ4 z*q$Xoq5@1CfDYPlf+0^|Gx8DMROre2&ZN4Ei=wvOCc1k0C7>u4lXH#G#=P<{w|Jys z{twNfy;A*l7qxh6TrdZF0BvB%Pk7k3htkdCm3p4Q#g^G+sg2w`@I;j2|FJZ>Ce?)h zax)@$_2$%vq~{;FW2O5_LmBsQD2i>u)4#>ozn`cegCW7>>UibFK-KO+unU199q<1S z1C*793ZVkH5gr!(n8N=J&cK-;AVZ8FXj}alKpx@lSj9(lV9U}!|8QSPKeqhD6G-)! zH#O#12RE0>Ir7?e$61XvuU)KpEaq-#f(XQ+S)J|=#HVb>TayTmwE*weNEN6^yLL>C z`M+DmBX6?*{R{OqgQT)H%xpOLIp>Et`)Un<^=yV$*)AR#_v&?7-#PL2i2R6M{fxz4 z>eL-cW8_%zUw4v#Ea*NGznxlY5BlXD>6hjKpecY4#3y_g0GO6B55x$6fnnD(QP5#+ zhROpM9L^|xS5Y+D}En5*o&B`5Ywdl{_q@$%te6li)DX%z^d^pUP$_?pS z6;c*IN$68lU3d%*^nPczYdIojDOVmA-!TGa-Kqb%f!zysual2{lG@$ zRqJCsZxFxK8RT~C!VTD-dv;l6k-?))GV*=@tGcBA_zXfr?I$QGJRWAkvLY^)7@U9B~UwHuzZX5u!1v|o-SnKFW7H^cOQ`YUw(&jP|29D z0j1*wmKq~79BU5#|HF3*P$qP^lt-r^gc^KP`G2AS9uySj@^{BN6Z)?hpaA^^hI1wW zI3T3eC zqo^}S^M90+Q30A|w=~~r)yx!7LwO#pb+!Q{XHu)-nxyc*F2rmvk5*U4Y9k(z*v|kt z$dBq!R{zBl027dKTgLDC-lj{10FDuWA{YW-TFCv?{n2NDz^smapL(c2&^4{LgX_5A zC>m++qPGG+hj~EkKW&ZX0ch@=t?|Yi9*Z{eL_0vnZ5?pZ0_*bqCQ7mPr~BJ;stDx! zbcw@fiF1HMkP_wMim&fbk93gpPTr%uDJOqtI5{!F1Gi(iku1kuM7_?ofkkPWMnD2)j@0Z*mjhJM@ zb$~?@NKskO4^UNU$e%15s2A&fvBFvN1CZ#Hpr`_b<8d?3oq<Xr56k=LGA$z0OR<4b@C%!-FnidA@#xi1vj_GI@BZ7c2qI0V=)CJFy|BW=U+o4Rw&ip|5p|@-U74a_4F^LkL!AUc6R3WJqDOXIR7Yq_gb112 z3U(VSfghWc6j|WAl`oHto3<%lO9ReO$dZRbe<>$TP{abx}4omFVT!TjTZ!!{Zyw;<|ouJ!^8rH%rTWRRjAWD+tmPJZ2qQogh({}D{$)udYDUSw-v2#EO2Xoy@NRz}r|a2$ zuBPLLV6OVKHJn1c7>~?jr({M1Sa3k`q#gi-qTo+}S4^KK7U2Sz7i))Po)=%vBRLv{ zZ%=+ho{M4fKmjJf`k|H~ePM1hyY->{yzvZXizxHA3Dm!05}I#W!z){pb+_!gnQb85l- z>pnbT`e+%En`m9f(5(lNqX4bSLMumRXYK5I+gFCq9=^TP_%&n65dC|>hXNw|1YPVm zmnF!m94nKz$3?3T@4UJJg|Wvk%4mrh|NOWL_bg2!^=X@H^`lBXn$&TrQKv2c?6L;53JtY?4>sA7xF z>#`THV+}=1KqqtjG!X!oSj#SEDla`vqkZG2%t5|&l0C)lFKBstiSo3N&+itwEg@=8 z+to`b&NQk_P7||aKd>b=+Rx3bx|I=xkO)^Y zsHtG?ct9V*pYbAC8$j#u`fonqq`ie#Z`@*NWAz;Ny84}p&!&?PMMdSy{e}iCpE9gSCAbVTR;FuC3TpfEw+xPQz#R$>v26Eba$r3#nyjPz=D{hNMo|K=Xf%Z` ziS9^CJh<3lSAZD)&X)hBQ_oNAV#S0E4$4(~s<*;Bk^sf9+)x!{*)^1T3q3aUGp=)4 z!NwSGAAD{%sGuiUUZQ|Ykhp&{Rb?|z>2dlvsJ79DsvM!9ZC^~x0rCx$9xOPCL7>JE zjQWZ1sEaC~+*>|AJYCS6L2TU&gg{Rt5&Mj#p>tan-b*w z>3wotZrRrFP-|MW%p&ahDIhHVHNrd$^$7Ve*z#_$@#YBaBTf>&;S+psV88MNc2$~qdM^Xt@NJSr zrFUOF7QP_CzPUIiA;8#AHm@*p!wgBQYqf%xdeVdV2pFe%aGuwg8d;8cDr<4|3Dhlm zi*K`7>A0wt35{{ha!om|oBe)~AzvBBori6uBs9g#aXEA)$hA^>{DC!T&{_YOjF}Pc zuQI`1uzHG+=&_>kWcI~N19-LYx2W6~L*iI=WjlrHb^%gf`@F|+k~AWcDthB8WYRIk zjMHGus$0S7vTXG#@E147gPndP)DK3-3+=Mzcx0`o=5xKF2AD7bfWjeRk@lr_39l9a z1*|?)?OJE22R{L@orJzsgf2sIV#uabvNzIz?dy5XQ+_-;GkP$7SQIkD4FK06`kb00 z;hL6{{Z{9kqb~1){BX<8lIAQMt69!<-6y_SU`~5!_X_d6nlAX!jrsK%&o9k#nF_NH zP7jZmebShBtu0}}e9g5p{&D>>GR{56Usfe6AtjY^hi zww?>nQ@|^a&9w+@g#K>!jBK*Of$r_MT@08=5B_hmU^wEUz@NG$SEV@JGA#gW6uCF3 zv-+Klk*+}t@ud8a_F^Ka&ae51a!HIS7SGE7&U3GN9r>tE7*Xlr;RwCPHZMO9hpkA1 zTZd$-=)qvzph_vbJ{cGKFf^y~_~I)v^G>yg1FpI@gUFxCOrC#(WCM@@HbNuq*Y z!fvq*aePSla5^iZ#Uu}*S`>;$%wB+?PNF`1tkzNd5^?%#x^BDPY6r=Us+`t{#=K-* z3~Y|{Q=Gl=Bzvs2xqUmO!m*^YiKNdvQ3&}z!G;Qm@8#&8XE`m0MTCa$(gF>KpDx72 z2fK}9>9!go@#Iyp{(AH3#r~h7GU5;W!#}w7ORj`m$8)h;X%)V6f8J6#D_Y!>ij<^j zVtUX8c&Q*NXT}xM1>Wv>G6#SB;gB*Mg44kT?qwB=1UoNUVVt-q*z~`N@e>nH;U^kG zMRumU+MUV0X|#A>dr%y6vH+9I!S48Sw_w3#!*Kp?3rewYD`x)iCUllJv~*vn3f+9% zeI|aG(`3CrfhWw$C22qf9j#dEA#LD?E6f!wRzevAK*`qVXm#P=g$MK&japJLC7DBstT#xRx_D0Pjoy(Tf_Z$CS44dKbpTt_$=_WspKvkrnXPRP#U^!JoX_?;-8|B$-X1G5 z^s&F2Y8n5k?(@$aaY>l@Tv7z_QO^3emSCuM7Z}35!AcVG683)=`hwGBsCZGR;o_Os z+T5u#B%(rDiOSZ%Y@^@n!}Nra|E2tPu(jROSMfeyuSq=)qOs<+`RyPS7_^@l5M&mg z?LK5*#*|H6h4%KCf8DQt*8wA9&lH&13h%AV(|jO9uzyC==ddx9R)2NU?Y6*5xYLzi zk4B^>YjACab4{IuLUN&FWgb*d_iDcJoHRlqLb~WGqGV9G#CGFH1yd}6rjKEJK#ZAH zHv9xj6ClYcoBk4`zr95t2Cv@d0H7BVU0;He-1iw*SVW@*+S7YEsZD-!lNwyhBxyV* zs;R$=uj+g*Tg+wt;9#_vV1#n#m6Ud~{?`1nuO%?Thr?hr>N5K($0LqQmO$KR9FMg3 zWcv*8!JZk0lY|BW<@+y>j+MXZG46s9ygD6)K70z`+XJ3L^2Z<^tM$4s+^-Lza-4tS zq7_pi9==*=sZ#Gzzr}ZL&2#c2dEMxbPF%%jN{qQk9Gj}j$-Od6AwB5~p-E!7x($lB zdxtf}`GR;(k6XXfZPfJ35_*lVo!I_Q_5S9<&)`gC5lqA4kO($bJfIN;(n11|@&9m_ z6*hH~@z2b5*!oAbuUvpKD>3W@p5dg=Q5&>&ne&}=c1)u>aW$=Th}Gv0A(t#?q?0?ZN7~3|_Idhgy-wyv0+uU;2*uf@ zkk}Zh7oRR1YCT|_M0iW$d84VB1%R5UQmzD&nJ@A+z}|eysLL+$ZW3H z+U41sq{9y(X)w7g<@)M=TnhNN@`rkD>e`SGPB05hS5{dd#h}wW#kW;q?9%a=x+ZZH+xX-LSF)BucpxV7f;v>u-I)#`0Q7pa8K-O zoGhmEezOe zmu}@VW;O?`Q(ILF@&1|$q^(u3`13$J~)?dP}G+{zumqoMjO&@^!ZZ8x1X z(&_RwKa+@v`A^(Nqq&F69T7s5P7$j6gqTu%zA$P()rkHd{ahD9@Dx=sfX(X3EA2xp zlmF;z51hp3-Yb?e1E|*0eQ>|q49w2*$=-C8jkEI^o7J?@yz`U7fsE3#IYI|t9V+mr zo`6)qgJKGJOuB)`^sY8@=Id|7)-Bk!8(h+q9p~#Qqe{05`=#{ZaXYfxE zdFI#zoGq0GMN&rBW7a*YZjCqb)+MUKB);!_kcKwiIUaOHEO&$gP98?)m!{WJM`_DTS?^+jsZA+yiNu{$;mDM_TX%*|aE$VE18rOUg`ZIu| z1jK}Js}c?uIfq{te~FPdXSOnP*T-%{aVhuxmt16EulmTd7BoDa8>(dVJI z_O&PTvSmJ{T)=pamExA~rHHw)*i=Nk44(0Xb`V7dN3=~sx!T%`246mg5p~%=(?Qxv zO3zyINEw{z7zz%xf)gnJT(DHeGe*WL)g2Nm-GuYRx>rV-P&rgnn3cVJpUrKBxTcaB zL3NEqI&-^xFDmb`qiiKJ#=4z2zSV!;cw4Plr1SaubbH$W-eZw1q#7pf>^yz8JMDF& zA17clU|!8LrpJ91YNg_0K2}(w95JHq!^1tPZ}(esIdNw;ie5v&-u=Kj3NyFF@m|_` zz0yr~;st~doOt`yh+|j1*C@NIIDpfffvi#3vugK(r690haNRto*Zwth9<%I!TPev> zlVy<4$5Kwr{cXVRi2(qoK`5M%jw_;=z>mX*vc&x$1lYKw7a6w^a_v`ueSc{@X%h9h zI@$a!M3;j$wy?9q{X(GfdM9cCYcAQr5|_C~s46Vj8shSPY!Bf}hL7^Fo-ot*9qxj~ z(9h>Zb8h$A_8;FMYP-+1>3XUs@d%{8<{sXl1@BDdw=Uu@pd%lM{m-r#`V=e&`q+|z zW1kT9Ws*?tB(GWA?_}>!*?KaM9fxh$l~XYy7Hog)aJdsOwYv4Jbhq^{tx(4w{2;{G z8?N^t(C`6P$Mx!ZGFb!LI!^)lpdi=Q!W0^s`9aax!ZXj|uc;LwKi9c)xnXzK+6JD( zPX@1#`?y64QaG>2XK>6qoizP37;*L73fiz2ydD2vg(Q?536oz;F)TUomFvb(gHX-6 z=gyauYbkeQ$v37hi3+dRG|HZ32TiC1rqB?{feOTkX0l{E2PUlqT&%DA)sj;_l~0?~ zz4l2PYQ@UqWc#(|m%rOjlr%ym7}+bBR9NAVUVd{jf5q+Hw1T3@-0ol_E(Yn?k2Q#y z?Xm>h2DRrf0uUc9LPg3&!+>tp5aKwDxo3VML|NmRY&>1i{NT@#%Xs}?*dwtcQ4bvo z*P;|7Aad;t1)6g*0hCSTlVb%O*1QqW*s(LTTB0uF_Gbdx;|&9Oi;#%~KaS>700X-{ zUSt}K=+g2GfK1=X8n!QPIA`yi5nsg*hWk~Rm_F>{;_eo9?PEL#dzZhDRG0*;|V z!dr9G_$PZMo|mP%4tt)%K!fzKydUjvfR`KxDVFL4_-=?eHr>!H0!+DgX3jSX$sERO z!Otf=Uf!gui$#vNHznV6_miYi6ZPE!PcvV{AfE3`7Ffh7OlqT=9}J2(xo%b$t|v1P z32rncJCuD~=P8W%XmfzyNz0Yjz-^JE;ZMrGu~5&;AOE7o=l(LT&be`AF!kEvE*VjP zL^b*0>Gxob>L7e(^$PpdtD8?_ClfDs{`d784_`n*?MWV<8}t>OSHugW+<9~4D;j=Z zf3M+hw$y5ewI5c6>Xnul>o}|H4#cTqshc-973w}}g)4XMz&9ME3b;61@0I5%(*+ar zxUY1j`5>L|j~A(hktW5DRx9T?1A}|H*@;8o|igZ2P@8Ijr z${frEE{j3zV_Fpk3&MX65=7*5;XZkZUt(>z?R67Acez|0i+hOG=tk_h->o!N{7#5n zuVr^T7+8|Lsfm%ebM^Fn=RRT3Y&oK8J`ZRuEZG*hU!Uq@YPdw!ZP<@)liUFpx%lyozk#k}T9<3g|~+QPNrQ_lHJ zgKQ{DY_&mG85yCVOlqG}G5mP~%B}lR(#_=As`uTNh>c|28k>=B&u+vtebRraAOC{T zsFHFH#3a$!rt|u)-`3d-#_DrUTO)Nt3I`|pU_7C2u(NUOYn1ju8E5#?4ul& zMAmt0I`tRg@z4Eb=7yk!vD?A)*($2dlMHhplP0zbO7JhXu?{BO& z=$xfBxKmj?pFNxz_{anuS8x>fgvfcQMG2NR^5*#k@#h$eG)ruF{^mqn><_y4hTzzCxKp(sLC&yKPDI-)X8Y(bpjE=%adScALYmwD-r zuV6uGtDPNHJwJvPdIXb^iau2$Zp2u)7>ziE2|V<;XK#Ykuk`0P1FP4rly=vv)p}p8 zm#@nA<|-EKx{iI0)v{E(?jr|Q!=|>Fo8g?j92ZbMt569(O_7|Un1`LmBTeRBUEJ6l z?DL_x@dERP?@iNtb{9GI9-}M7Sl%yN=@Kx5=Ga7wMO(KeFoQf7FOT}aqkhIG)Bdgf ztVQ2P`9;!2Q}EBi;J4MQ)`zP)15eZSI{5{Xtfu?)O6);SeJ7V6camHq0`=3;l&Bdl zw>>7P0v?(bp-`xHQyt3?d|D4~_MM6y65dVkJSLk_PE2_!b7pJZTJD?fPc?jY$hp!i z&X^(z-v`AYI9Rri{>6ESwf^{_1+VF+`=5kap-&cjjr7gvQ*rW6ybSmd`fNTMogp=k zgK5nF7geO9z%+bf0=uo2FjOyPiY{VJk+3FZYE8vK5@%M~#HKcJ8zW{|)yfW{FaG|( zx9;m*Q(|j2aYL;VuBql{#seFToULRPD>M7v3YJ=38cZU}B+tFGvZ1IQuT!YIU=L-W5w+ZLT-FFwebEUcI=lz9+B%Y2#P&e0KA7Zn0>`y(` z=c{U3FT-S6|6Ug$eQV#DQ6%xHy&7M16KK>YN7-QKXjaOhmUlyHfpze-RgO^`w^6hi zzTOfP_`X&px#7DWsLywj#=P9i3i4a3A0WrHcB61ub5lT({m-t5$>22!cjZAmxlbq> zDVx`2Z3L#IVivdW{#ZnV`Q87+^wIg1lqC=_Ii~9}{Sll@GWcERc84j$kp$h=x5ci&q`I~TWs|CM-?=)p1W zI6*1qm}O+e$3#8mwat0zV#OZMGW)OEd(q=x3$C-*TU~p;&+{vu&XDbmgu%qIk<$Kr ze%XB^vT zyMCWGDRgo<^tkTp?V)hmUzh;5sI4g#R1rc%oLwvEad0<&O|)L0-wbynShC#+QUL5$ zy*LYor9G&U;^f@`8KMcSI6zr&0FFv#BgxZ+l9X!qeE|z7!7-DqdJ7j>RTnsGn9`PnX(F2Dvvs z&Ho-AY|n8Ev^=jBJsuk^ZzH#PXk0LSD9iipHmdzou!ip;4MHmzev; zQcja3g6*znD`n1Zs}nh^pPu!?aC6Vxu8#_Of(RGD^ZLqSh%#a5eeQNA@4k1rl^=6G zbG_Aro1647eqgh|3o~!KNpk9#ED729@rOh+gbB2MoiU6+04VJZnj2c4gZES_pT!M) z&sP6A9E0;`-yGgj50B>`(^H-#lPyJ-Op;OTQ~q<)aCVkiyxrDn_L;AwaRMy?I3-8HR!=qyyV;SLL|ZuUqQj8 z7C7Lgihl*JrNy3DAV8y9clhO6dP9&72HF1Ym5nQ~EUXE90%r_ZJ zJO1w;7Uozmw+HzfK$#R;y1M~{!EBPfSNcMH}i%p?nQiwusN zAexJJ^FLXY?JxeCE%@du0SK6ce52AO*Onn9q@Hr1g~Bmb{MQI((7&BBRZ zbtj$0&3hZi=f`sS@p7cn7}yCp#5!2qrRy#8j)SH(KabhRj(*GYyo)E+n-12+-EqY7 zO$D$U*hzrfj)En`|0g-g%fQ0jPewa9U!ei+kN^cmeM!TTZNc!pv}llXyX)x0&zWz| zZYRxlt6v}7uCFw9PcDyy%jgmAWc(X0j`acc!Flj!;=MSUIQ6bk3n$uV9H zqV2j``SY=`4s!L{FfMV=(U$NhU~i!hIuo)|j-i=c|px08e!XlTDgjWsi zdC8lkH8L=@bbwk7f;+X`K$6YOLE%5&EUCmbT(q!7hO$3awA~kGf`Wtc6B5PT!STV( zPU!B-gN!LOL?0s6QOLRtJ(d|zZ&4IS`-Cq+(P>0z3eD$ zG_z*cgk|l^_T?nzpFqH`YUf7h*4(GN&}$%^af+o$-?FJPa9$> zSf1bh{t*M~9tH~(b^ctfqG8+yz8gpR-p&gASOmyH#I8~~AzB^SrsgOcPTuF&qge}? zpDZ6YI>h-UDf&kUE)Ks&!QR}wEUA1H-5g&jC`mlF0zhfb$fQ(J)V zh6>K8z%i~F^s<~OEo{|v9pB#ett$xXb9(fXyY5@euYE8D2f{jOo6NI>)$*rqC^u7Y zM~!AU1HZeqSi42$-!B*GNzwfx2BE@uQS<>;lLz1vofm_3AHanGDQHIqxJY!GtiPj> zg4$do-KsmLLj%tSKp!RG0a8br9DJ<<-anUWDi!oSSk&?yq`SGNw@3sLt-uVlJa^x8 z`ShpG-Xc34^UXG@ji!zM0tq33#$+(;AaPb>^{|4o&gq-ZWsy^qPy{sIv%`!{pC*3y z`x>n)aDju(6x}w9)h{kH!_Ka#?JtF~{5#>4K#_C%BUT{RugwV%yK;d4EPftHDk$i+ zG{YCfiRbD*U#>Qq#)^w#PPqX%Z!~i0 z{#+iwEIGB6%TBOHFVMjZyZx){l6X2^WwN2 zbZ{}+UK(jA(|EtEHr?#Ckk#Q7*Hl2Gma_V_Ge*P#Qql_LSe4TQWGF8VCTRy^EYA<8 z3&p`gf7%uS>wY|UKS;{od2YMt`~;7(|K>Jt-P~+md$Co5qvhbT6pq!Q{vlZ%#aj}w z1uki2Y4E23XYJ+ySA~p1(2G#8O0EgZNe5pdfF}Gc$uz*kIDiNH8I)aH4S>7zVnN42 z#Nb!^3Crog4e;!NVzk&N&;H0gyo^AT3t(;RM*&_Z+UIw`Xx2~2cAqKb5Ao*^K(B5= zWqH}^FCg*s4HUAUaP}U8x$)gBljjfXaANCw3r~lC%h>>?EgRP4R}Z}xsMdh+GYr=K zWTi>v;JpnSXY1qWq`TmAO+gP7+6r}xbWW3&0M_ls||I5KUK!`OCAM6We=U&1;Z4c0&L!;|(CpgcNJ=ka2vwKdz8M`~)n4 zUPR06EexD;OBd);Q*+z{0%9f`}EPuHeE;CEMxv@`Tq+;3Miy{8AWT;JrY4cOBVn` z7Yl2GLc(S!3<}a9V}7#M77yEUw|C&?@YHf*hP!*JFBSA~reUX@)Lzlx7LBtvMrG1V zn0eaTy7$c_leG5ea;uO)!S^C%FKIwBs#S1luqJMkfy1{20-Uzj-LdHj5F=ft#CaL! zaAQXlfnA>hhA7w55zx^3&$QmA`u#R~cUs#A>>y>lzzqe&U~r1t*GzpJ*%^N1W2vON zIc<#uf+Lcopw*z~ z$Ekv+uRPUy4fHL9Jog|B7;Gl*2Im5*4uff#+N2zlRK;n_By z&%=Zan|;8@<#i+?4|G77KDfgj##rx!)4VqIVBO9W&yRVG!{%XF#6P)78M&ZkPh5qdy8Cdj^=*g+t$uq}NF%HsRhd8+!@< zg8A|co?>7gu)o!1r}pC-AByv2*|{$n!Nsb)pd;^41z+?JA8pOOYxPZ`@e;=n`u;LP zBLA50NKT?xDKT{1zXvXLgzbh(U3J9S+#zS0?V~;>ULw(y| z-#zIG6f@H|`{P+ZL^0M+A?=_wZShmHQc{Y=#u`TxS6^r~cK204vGuV5iyG+@s9yC< z&HneWVl{1u%nhKqoCMJ{oiHy$CmzfSFm`6Y`D7`+JNomXp5&fq1pn27va_NkCHb*UVjJ7`u~e75%}5Vh3`I?`VwwSDlzC z?jcKnoy-xyeZ+Z8Vh;FH5(I}fK(JQmeZvB@p#HD`ox9(l$iMEVF~P3273$p~h&;|3F>;9b{nwNL~?0X3D zh*n;F)nX6?WaZvynBYeuwX*F6 zM`G%YSGb-XiS1i~NUu5J$&^nvzIIGVP;gaFy+}s>9p;Xg9tC<%O zMXl~T>d~V}$k+JGSaYU$g2Rej{`;l)VT`dwXx|EB)7R5```wM{mBwbOftVa)@9*TBy=FIU7QH z`a5dy6zK2#Y25hSiM0?d)J0YF5kDELkzbkm+dY7+im4Y&b)dx@+yKb*Mi?T*};*wEUzoKToAzuC4 zO(GysPnqyh@m;6yo|8s2m>^LI5?opM?StLQ>gSbu+1Y%1hmSX#gJVb^sUb;%Q$;3n z03cUvVD`%obXcJ^Y>Oihs~rI7?!ee634&}Sbz{D0*dJAp1A{NaT_@s?`l79CWWg!~ z0JH{W+kfTq?tlRyG3|x#n*DZ++%}w$boW1xKekE^90;0tG&%8n@Ce2>ge}?FKl0A* zYP`IOmTEl#GmDn9xwfk~As3I|LVA;rZ$LRKFUak4iW;LtPzA2Dx1;)1vAhlHyG{i1 zdYZ3yBQ*0%^CsXuC=*@WQ_#jYwj2q#59mo>NewAVWChu-E!X_>M6;$0wSuMc1nhJ9|rhj>{dciN@39ns|oy~oAD}5}A`q;_yd%3mrw7FGd)l;gh{-5lJV=^2; zr}6hlYT49q*dGGJ^!Y$tzAOf!@D)>(P}r8I;>VL3x3bqtKqtmR= zQ}Qd;&KM$=@Z9XKi5qsUbR$U8Uc%WN|OMK$&@-P3eq*NP$rHRs+>Wtc4B#I z%gy2I#$I`p^D&k-2+A-s64l(z78ItVC*@{PDztyf=N;wq=w);{dFEp_@)mDiP|GQ#u)0J{+_7s3MKtY3@Alr`f*6)|urjwq+thQ#5_b(Fs&DNhYm3nYX}<+NDfQsFYtnh|T(Nv<(Wa))Q2Gf2ty;Pvh$CA-W*$uIerihJ%?hbDHodCd$pXS*mLo<{y!vg-(v z)O{64&3o^^4ddl?ce&Fz4`OH=ZiCG-&c$al~NS7Ji+@MH3}_~sqIg8aolFp&a{BKpxZ(D;G>F> zkm6d?_9bNjoH!88AZn``2s#ZBG^ZUL3=*@S{BI;Usv;9~+Ai0NN8yMBIXgRx2LT@k zI2qp@25_}fTF|&2FWq-I@V~qASbXzLM)20~W@rhvBhI1qA^t03OrP$N>(Y}6fvr%3 zQdLO8&|J+s0s+@!AhM%T%mCtw(aDY8=0Sod|!CvMdm55$>IMI){ag#Q6rG{oTtya*)!SwR0x z1pEjT2M-90FOQ8*i2Nym~3kr2E!9f}7IhO5)*LYp*}9DaBE)iUQ3APhL4an^fw zRcPr$J0gfUKU!!tZ|Tb2cxo&4()H`HWK zAyhP;px1Z#G;9peHQ=SE1HqK{@JLF9MpeP8suru|f5SUaFE9n8E7}N+pcA+=zXR`T zki~c=3`#DW5u}(Uh#!_4bQW21>N^@w=Gstt3WS~vI?{4&Zh9N4HhmvoG-zdZSsROb zZ`m<2)?Ge-?+};hy7Btl1EMOt?gl_BCmyiC`z%KF6)!el+}HYWYHBX>@>q2lFY{iu zO*}G_NIRUIb2Co_3KmVYB*jJM&#$&|LtoqtPPi=>;161`7gbR5mLNKTTNIDB0w-wFB* z%8=Xf-r&*=&+$kpQ~6P^I|1>MPxnt>B9g)>#>U@rOe%d8e(Pxa1aKLKzacLk>j_yR z(6_WHx*PWSAXpx@5B^Ec@ix&g71!{1Rj2eHSh;@RuF;k5R%a*FUxCKi;z$fY>bM4eG#<)*T| zP5O>TPkmZvpNF8wi9k6=^9u6b7rBByZ`s=S3@RUK976j8uU}!Zq@#563ARtR7f)$4 z=Y9W1cp@S3^$Ky2426&*n9%n9_NI(Fc?yEHiVry?j!p+3T&s7DepEHDgUaj5k_2!b znz7g-Lk0E~o4f3`Jn;PM*;LXvAZVhPXltYUd!Z&U)GVT4%xG&#WG%HJ={nre>-=4F z>V0MJDvey3H}&<4yhK0MMyk*AM;9_gAxSFV@zKKmLXK?|_<0^De${`y^Hm%>kQ+&~ z+o(0MjFgZk2*)7nb6kMO(_a{9H2pv4RUvX^^AIrel%AV@@0>S2jrH@&+BDFm2yf}5n_af5lN>Ark~61R2=sRv}t8F~VH6IHLg{f7x*t@#P{*Y~>j+lKw7Li$gq!I%vXHHA_tA<1>jH3ngWMVUL@6e5bf3_Af2fhyDq@%rJgXb4 z19=(8+X_6DddHqi6tD2Z)WS}3%a}r~-|MWp zi|Onp^Hp6KTWj_?uE@$Eym0B?!WeXpN95=+4NvF9l2p{xbA09*%C6ctm|LwH)vQarRIq6JkVLue?@_^RF z*^*hK$iR}n34dh*SDlPaF=|AvqTS{M0qt?$$P9meW3W46P-`lSR106r)cwwPbn-Ta zl9CfA$4#6ZA{Pw;RofpMMthKKx^l01dkiVa?l@f|=x&T{lDtJ+hNo~hPVbLeQZ66F zhBfq(QX5ztzXQ`EhM}amBsO_>MqJK_ynmrN1L*rvN=x1y^@gH5)=6Aq27N2?BWmp* zoXd7rW-%Fe_`2;6PpJz2dWEi#r(i|@=jUIL2XMT^EoW-iwZbrTGJlZBe$^oXc6{(n z|NQ)m-bma1f|UIK`&Dc`ZFdWZ3IRegXhpz@8xJx{ z{}#=axtdk4-ZdWku-7mjxk5o6bK)#khv4X$glCo+P=K+8d_r}xGP;|-li2beAAL)v zEL@!e@~;h+We(ISbZaHNb^5Pa3`0iB_n+*Lxi!?x zTua~T_+z4Q6t9}Js2wpx!ti1;YGF^yHvUO(ytS$bB#h_N7MrZaJRVsvLXt*s6mJBV zplAQYtOuDLak$hx@d3gs2ox zq2&*NqB`{+bzC-W%!V*D-S@rQ-JnADI}3E90O@F~Rf=g43M~R&AmEhiHd-pRQE+cV zrwajw;611(sQd*cH)%$|-F|)m)s!w!clkBgo%X`W2VFR1Q<)H0wY5J>JQ2V3U_0sp z5I*4PhT7r!XQq|$4EaAmabUbdjMwhcNUNX*Mm z_7gZ_18od<6YX8C-5Z0s=}c)TV7HFrm^teLK+Y@nso%G=*8ck@VlIbM5I&1*NTL{2 zVJ~iV0B+1qN;1K{`US|DN5L+kASQvnDh1NhZzMt0X++7!YRF~%s=N@W6CVmC<5(P` zLc;@Ni3OBnhA!golyKYl1O&2Ri&N$^1MjbavG(FSuECTkP_KFbZ{|0U;@I5Jv)0(sHEC@0lKgFzJTcyRae*wrgvUbNrspNCn1ZM?UsneO<|cc|o23VKqd_L-NTL#JQ!f)+BA2}3WWMb&uAdhr`o#yz)3 zV1}m&p^W8y&qQ~2Z-1Mjta~Lb?r$gK3o=iX>}(0n#eUCEpEd#c2QU^uev3RY`?z?U zhWIK`$G$D3$PKUxpTw6VVY4gnA?@yky;8{llq>G5*>W|;dFvVK-A1$FPw<25P!K9m>|B4WSH%4i*?oz*ZSm$6 z{P1|}byqSKPibEmjM8V|lgT?UW@O?hxazFqGF9pIfz#jdzR?bW4VSh3us{lNirppK zd#&6t;jI=&GI<$8M%gYX8Mgw1HPssxgyJzMMmm`Gq>=F4NRF_^qO9U_xLf6+YY07H zPN8xq0c=E5UWM5g^kOnl&DE0V!33TYdWC&(yU z$pLB^DciwW`Gq1ggi~Z-?wgyT&Ff}Ll=B60r{undgoU{jRVm`t3Fl61bF83%#>0L= z_94Jx zpaN#6YxhzK13EgtXB`a9KX7dfcC{c+>;{U@KZmW^QO0zvXCG(k4*bBTOf`! zT!H-HtQ!?Y&kV>DE0q#PHvfhO zz7RLJzp6gh6F7m=;TylYMy1}n3MpbPO58)y8~rr3ABm>x5#O+8{5XiJ#B(?}m{h?M z2tN)AkI$eblX8_K$5;o&?oZ$~2 z7>AZ_Vtjg;Utg9Ap2ARRsMy*i|d-mxs`%NUtMrOyf$Sa)uE8XDls`r|W zv8uj2>V^~$HVn}IRLwHS7l&`d2~=9g)7c9_X!WN7 zxJAlN39ZD@wVzNw{6u&QigB0yX zZ+prB>K}P?33?4`tIT+BFa{Vum?GdSS7~$c;|acgEC&JX7b9H@$K`R&)uM;rzlhz} zeuEZSJ;-;yBlM-`toITW5Ce&TX7Y^h%>IaY32h_J!hq&Qp-}#3X#?CX z?ziEytT$4w>hjFd0Y~!E(+IaoX$3?UnnC+_@!XAS+K|yMGzaI;AR8xU!ZPx=4S+

    W%|cb?W&f))o(^{OowFx9FleBq&y$FOucwEHdGY8mci6_= zCZ5wg<29rMIJ(0sUYTq`&7NIXBC?fEoYvB%-5loCDw5w?zYXU)+({!tDoa0<6CNSA znoHGog6cmjOv~GpD3r3l-pQs#W`#C~LIc_Z>3|JS4f6avT1!qQiV?D`PuKEQB?_9h zI}Ungl+?$Q4VDkyfkpTps$L)poPMdgq5g?o)0I-yYxt$JHF-{>{``9OgO&nX9Dl~WVZV%hj0 zZMz=nGH&3#*wBP=o$-0dqj6ujK$&02$3j!&A}32keoD5S?&5;@S9vi`B+ zy(Bkm?3Y?iNW~c9k(!ApVXt8SOD@-l%Y_~ysq5ENjGH4r{*|7)(zAsbP+D)GMLm}c%xa#vo3X)oz@)|Jt=`6K#t0QOicudm{ zvh_{Ua=a(){Wc$T5B{YYR`9BI@Kfj7^2$dU;^jY-K~T~_DI#t0xAc;$;Im;tC`6_i z1OdI9k-|$>E6Yi)Lofib%pJ|**_r4xw~l_re{0IR!we?%W#fOp`;Bs5>g8QC?*-Nd z&IuiLH3Te>!6WXM43&r0@7zl2{nvV-<ULmQj}r~-^WCOj#;9^E;hHg>bqYyF1txEa58sJ@$4pjx}9_v3|AH{#lA zh4A~?1M5u3`#LIsRec>grd|E%QOZm{&}2{tYsJu4A=hB3d1-_5Vag-EOupQ`-!ZyP zDrRMtpAms7b0chFG~CK5(n}9l_K6TU@=qSV>@lkK<4sPjkH%?o(c7E(L(>N6KlWRB zO*R^=T+2PCUrD2j!Br+Na&>{UAPWADS2YPSJ>SP@o$?~g*zu9(0jdWkN>b2ZX@0jS zkh6USfh`l-JvJ)4BK{P169)cQph>=kp1`|m!CsrUJ+sz3qVzo_U}*D|){MStXBh=| z9!fUuD%G`=1vwTeqN&3_=}}_q(PiJt(y-$LL$79d=;_2cWIxa#NtVwy?n!+BdVYMC zw6-+!`F*`5rx%;SW<&emgQL4dS9aJt;Ht9X3<`zfn1Ue!KZ#8uK9#dk7@v6iewPkXUp4vXBaQ87czt5?0FD;%8Fw9AMLZ7XE zRQaD+yeyYd_}-zj(TI19Ho=X43Rg)Gcv%@+XxO&vrkzv{hV2z;sy$q7dzZMTJXw4- zsz2DLp~RB0Oao5OPy324#L29sFZ5<4W0V{ptGLH#{@RTxHXL-NM(=0pA3yms2}QNXmDaAf~Mwht77b;XV;{-RM zvKkrLU!V7g6N->y!Ck1iMsj}RoVTD4KgTn6RXG&bx(_P05pV7F?(e#NjcOF82d(G$ zfN&qu#>m$#0lu@`{0%{saz`H}?3Dt5cz7qph`sJ!jul7*vCP;UZ-YED$JPJjM z+-&2lbt(8+WYY?LtLR5{cI$d)tW{b{JZ43c=J3^ZRHr$YrxMV2VGSdPaV23JYUR%A zevDJ58+E~3sEQ3ORXTkQ8=K8703qQs3P&+wmG)X52Dm>O$B=imr&KVJnSaSa;o{(+ zzei!vGbd(l_*+&sW73EUnMG3$FG;)2uGI2YM`6DVVlmwQ^lVw<4AGxP`*zHt_ti)Y zEP_P(FsfZ-A*T#tX-;j#ciYcU8u9ckwVp}1a6dyJZx`^F?T>P0N3SH6qe)z{I?2d9 zcrc5@#Y)8qwx%=L{j`9a6|T%SBfnt9faLWd8h`kC7%Glz{pGOUKZzF%SHnVO-8rLpz~D|ya3V2O zB3CVjJIK=1`af#Wj_UU+;!or(5ZAk+2oFmq`A)C})@F0?Gs57^v%0jd5b-Ra;+`7v zE)ntpp!*lhM8FetTgGFu`Eoe%d^yIX?{T~f%A0&aG4IL)z(uz8529a?-~;Qu{oa|@`P9orc*B(h%9jZG z$T~~3#JbKn8G^ZJ-UiTQum7AQcUTv zn0&q^(FDQBWB79t1Ja%K5HeuqJLavhD7znkKo8zn4v9VBE$nnsz z&XLTHws+!Lkonb_5)?R)u21)^7(bIu#JbS?!~tSuN{XfN7?1Fi@Qrhy;$pQyi4A_X z8#>!0my>S0o)#}+@Z7VOFdOV)qzQ!|eqvh7@!8#g(QpT#^Q7}iw#PiO+FAW@&hLIw z8NuIiiM!e|_~ZUW%aG$HAfU#68{3D|OFNq{w}=x~)y|!N&D>U*d!sTp+SQ$2qPxZ) zX)w&-9!PqdQE;8}i?GRR&$zN(x#*OW*G#^%=Llg{!fF&5b~S9_{OZA6hru~Ci?Euw z$DJBVEtxDft9@NQJ-%_K$TcSe>=im$aOz^SlN~^v!6c3tUw5tQ1k@qf`V~&A+MPV! zDGKVAkDEsq(Uq~9%yQO6J9me%_M|q}(5_ni79A|-r(c2ASzQ|lzFK|K=IfdRP@2E+ z+P9a^HN>eBOKH~7mJC%KA)Mn!J7a3V8NkNEjR4m^pAlkGm)q4LS4n$pFqpm>M^e1A z_Y7Oomre?aPntcPA@;l<-{0|z2$qC)Y8)XUDX@r!9tygzdk{ObFofdi@Eo&ra4Ux zHF`09A{M4D{q1&^rBb%{z1EBxJ}ytj$SkH&7hOK`9Q+0}9{<`};|v0-2JiG^R1?!N z;kC@Jc|`W_8&se3yQUD84wwHjf6^oVLpF8Ii`B&KyRF(Q5FMlFBD-tdq12YQe6I+` z8Q=Npm(#=~p*{`>R7eIhvvY@#eU(*CX4&Vv<2>m`adMgt!W%Z%~(@yR|4VJ(Ze_zT6 z(7mDLHG|6-jyPfOe7wjn8T-uj;HaNk)R!c|`Z4k17h?Vnu-!$>JvXsOzhdav)k*eq z{-M_su7INqaDO`3((TLcv9%BvNu-Tge4`c~%U`Yw z8N1y1rmxN6S=l^hd=%mPDvfHXA)TAP+O==Z0?NJrf<)(2xC?Fta`ssLVwkvm7-nBW zC+s?n(GZ8yb0JnhD&2R|&`Wb-D9|4SV{5X5GEdCEf~+et&_!&I2I)A>5>JDrOV~Rr z)@t;Rv0T$!{uImIZ5ioH(@I)>To!`1Bcp3%Pvl;4s5M=rx7|P)(_HigMUVY0JQG`n zCh9*r6K)9Mj~DJKf6Nc%dgq=+W!f}>sIDrhnHAEyqU`Sbzl+T9ke|)*vA6|~d%kzj8%m)9d^}Jz07b3D;Vg_fC9G-!%s|utt zH43koL7~bXOfsQ#tF`=rU1DOQ>X(S3H`#FX@ z+&;;3zZ^bJl>>ku`>px`6%;dbAI!2!3_Je)q0N#mo9WGtS(qRN$=a|Fa=kf}3zO

    vz1zvZ8XMFK+S3YZ*P_hGCm*NxDp4NS3geA8D^QrwnljH7 zrq)y&L^5~l`uL`QDUdB%k2(Et8O@d$XKdMQkQ!~pRGLq(sAeB+rRJ9;*;9RYM~n1A zi}F`~_;j^YBJWPYAgLcgX;nrqSun#qR5(K&?~ zYU}5I@^P5cNGpml*W`au4SKn+=I7d5LT{`|)f<6yZm5isUg8G1;fgY_OHc&F6X^BiyS~$}fR&F;*K+ z`fW4n_Mpi5Ep+{FS}(;6dWyu@3mcR6*A)5h%e8*^!k# ziqV~Vk8)0ylVNY)StA?XOb6F$#QT9Os~YjRO8y|K+v?cuu;^C8r05J zo}1U7syNRCQIH?r2CL+5@!+|3%ISbjxXJRAo)cm!maYH;xim7=+?(}4Sc|wOYx`>& zY)wtoH<`@r_v`dX;;Cvl{D(!zmtjyQ2kS?33w^C_Y6fTL?m{6fVe4X0OoeOtW~T`I zw2#NSk5*ODU@V<^au;KV{nX}5=Hglxa4=V$t$;pKR=1%RD2YR#AB$oLjhNpNsC$uD z8UNP#9I)oxJ;3U(Y(<)GMLTo$^}7XS1yG6_;9K^&^vEjXzem2Y-^%+k<{hoa3gTGU zzg0PnjSfH5JtQB(GtH0y(^+Zxn-D(y#SVr;>4;BM+$TQNN-S9=YO)?oW};15FKgJD z4;r#4!V`L*!{*)}hsaH?tJ-1k!0FDg%4zG%Hbj3zh_t23Hi_8;>h&0%*>LOZ!@7bd zCYB}X3|k9`-`JzW4D__#JX8eA78e012U2@N2812lXODg&j^Y*t=5$zG##|t$n!USP zSGhI*tC*K=fACg|#{*-UsaJDjB8BP+bd4B*&Y0}Bclp2bH>Urwnn3ODi%DgQP4aKN zP8~3wp-oRiaksdaRoOC0dXdASqjR%9_6Hj-iGZoxS79+Ur3~1~(~+8!{~U0=4e%5a zw^wD$ZKs6}A8T+6?yDI?+%!|xVAg*uIcJH^yL}K8{vv1K@?QhPa#hyDoCvp zGGgtLo7Ev}wc^fmYSp~;TGW2}{1v*H5eL=@UDQn}pU&*vUe`}7%^juqY}Dm~B&?v# zAmg=q-q{v6%DSawP3XPpGw#i!r*=!(37XYy%rbM4I1~26U+&mWo@c%yb?K~itMak_ zD{rCS+{f9Z%y5(8nnJUUcRhJ|Jqgh*lGhR%AG>U`YnSkcp2+UBkjo^(kLj{H3;DaO z38yqyoLXq(UcEfLt4EWr?orCQnHCM|ysa)R>J8>8~Tg? zWp(vLF`iUM368&d2>*hVtE6bc)UoNf?;N;2R>|`p^}W8`_9gpBNita5M}Zvr#7fEQ z^;K1#wJ%dd6ZM)tUD{;R&c-x*e&Id{n3yU)g5KOpKA5z!bD!g(LH?-<^9vK>PD6~3 zH`TmP`v3{BIK%v+%X8%vj%A-2rt`@}4$S;#R*2;NiKnf@D#6!jT3oDKe!UxvNm*q~ zjqOtux67d!3qr?Nd~J^DzFVUiJ@a&8FsHKO!~&%W$bRk9{A6f(?#!9HKi7p;ER zNaQVFvYfANnJ?i39Pc_Gi{~F>x@?7?fI>Z~bil@eTwb&7FZp-7p|<^DWs;10jD`F z^BkUWF=gR@F|Ho4AJ>(nP5bf|=k5V=8@3*JvYE(LZv7%MSBb`{8Y=d1G1F~VBr|F9x;$J1+m;5bm@?QSTpn{h{P)Hm2!FJadQkUA)i) z_wD`FPyQYu>IAoQS|COG7~>*Z)hK?pz{n)Q&wBou`pwj|fIE}GS&~Imiu<7^jaUC< zg^+95u4x4(3dla^u(n>g&_lK8;hd~0QE0fx0WI7~juV)%6cqN>InGHKM{H?)@oV2$ zHpE6G56m?mnB$L;4u0p!dw(0S`itI&!+>#aV|Jgjw5$;-h;J{2-^tVJoP=+{KqSpp z7b=$kE>0e4dgT}OGL1?jjr?*U6|@`5uJ8OqkR?f7%(@u<*E&qsOG8kis*5CXMBa2=;49X=Lgw<=)2XPORZrt#y6E_}+-&xkB>Pd9K5P zEg?p3l?=^E9&h{qPF*xU93Bk?#Ho#JtT|Oo_-iN212J+5)@q6lP!{_<($W$Hbu-Pg zo+4_E*ql)3wf2vLOv~AnZVe0fPOM(88LHGB%i^cUBOVj*4&heAp*uljzE<=FTThi#PGxlkr^k zjO6erE3#}mZclSYNsc7ON-#w3O2!ACy#5jiU0Ess{K{%9WLv}ZrDZ*$KC+`b3qAc? zqc*BIbK)aF=-SyYR9lWH4;ZWI(OAo&qH!b2wLiy&>c!&}ET}VieUp!c-(0-&UDxJV zztg@s{bi$9dH{4W3b=5wZPxJ3ebNwV=b7a5SG961($t-qTBiJUeYW8!-L5J1@M)L9 zVuv@NM`^J#%R?MSE}3rlMnxi-qYK!gZBBzB9hq4oNA77lHl7EfV;g!^AUoS6f>x=T zP#hhsq)hLCvt>-$cY$`Zp=nir?l*OdjqbB@unk~oT?butf$&tNFH<4Y#3gHB61H4s z55n)}%#v9=)m%4y+)5PCP-0hC_-$jXyWhyJ%y{_wm|#$lp6{F@D%x{@`5P22$o^d< zYUJrIs^^itVd~vq9$}i~4tNhV*Ij#+k2t!8U|%+>CN65Cl)G*FGDF*soUId!57l>m zVcWV`d&U)5NN_Ic1LI77Xo{*ZL%~(w-LHPG0e|_}Ti-P`*-G}y4kZBh>AHAGQSN)& z>5Tyth)}u~71QrZgOEkv9(!XsLeiOE7Oyq65R2(%@2U{?W(tADmh7mKB$$qjg|eSs zzSg}3xIAJR;wgFL>ff>Ys6qM7yhLRo`egrptk}L$?a6#1}baQy8wp9>4BvR6G_>%>!5C{+zQo}C%)>kX#Th593 zTO2euvkSp*qt<|-bCmM}lz~-C*CcPPPrcDw5N(gx%Hrr5p^F;{c_n-0E#?CF&h_zJ zgX9K}wCtFYbI5_kExrf*Aq1QZVl{r`^-|%!=6^mtiys=DY?ih|gWGDx)4)(qzYe$0 zFnBD6<4M2($-^RWWG^|Z{bA5D%B1!RH^Hn(d--uPsU-ycttA|;_K?EX_ZTHBPdqo< zDWouKA00zu6Ng+x`V&Mnbg6eBi#mV|7RP3G+0Q~P^=o(CpjM)uFW3=$Y%{+{wcTna z$$0$?GIQ#mC8)o&`{ax5yVs>2OxKIqhgo>*o5EzRG#WpGn!m#RbdQ7~GkbP9M}6l% zh6q)zR-^iNZid97GH5_czr-L%Q>!Uq$Wp+@j&wga*YcAU(T($g+X65BBn+lRm%W;~ zCxG?XG;%!eR*0B)p;}b}-vqG1VYl+io~FYu8u)-)DXHO+CZuB?Vk|JbTeXR|<|M`i zgUe#CC3_pv(c+J=VER;Y@lBKk6H;dW$XJh7E{1|N_i85*kvi}dKjtOsF; z=95cv*R~`{$(^6p+DM!6#req)&95rmD8%n;eGT>MMCU7E;vI^#!HNoUD_fMi zQUvxO!5`rR6mrI^ZBc}7+Tg69$7iIVIElNrU(dAJ;vl^AV;(E$=?Q!rbAaJxQv$++ z{f?(3BA+$(_F&fB{GKwTP<&eWqRMRp?Y|lkk$f@W2^9gQ>C9dAGgONzssy&WWbxkB zu1DUS4ptL|+y0B{?)4T59*=M_`Iz0WP<2R=aW6*VP#NO41-VtHx#gEwo@*f)2B_}+ zYmv5N`?j|vA3jSa?|g#a4$T1vUQ{-qjHiUMvksr^ zBXk8wGqQpI=U{IUYjpVP>ws(P+}blveU?sB0~cSvT z9fmX^^W?9RP-?M~@IFw0WBOsqp&Di}3*}{*0lN){FCA*NmlT2N!2`Z#Km0{H&F4h7 z0#=0A6aE;e7U20LlKD*boH}+C2vfHpF@fJy_5y37DlMEceZOvI6M7P%7wr(NF@Pz* zzv(`y;`$!EL~Z@Fc{dsNSofk@Fu`Ub(+3kgRU}U2ue-NFy6IV3ak-wwN0pL(#|b>U zJ9H4Z-Mi&2ILBFFBnu8EChTS^6jjntFMB`tN!O7YfKOF4hq(+t&40{>Wx`x9^4={t0M!${u*YR?3 zIP=m~LuahgnXg;}g-TywQHC3%h4IA4zovUkxz^+8T6`8(<4g8#JyBgs>D@PP1O|G! z)DExx(76u28i>G0y~30OD=S!+{gya=)WG#-X17sBhtXket!mnPD5JR^{d=*Ahxh6b zCV35QVja|{N&D00ZPob=@df7}70L5fSLc*?hbCP7RM&L{>HCkW>;FwK(LDxQM!1|_ zJE!rmW#^NBLh#gS)0Lr>z>go0sVvX_X6&Q!ZkFU`UUyyzhu2t_FF({qT)u8(UC*$;vV?nOVlA_Nz6Pu;-l1X2 zfa2tRD<$}c;QcFtW9_1e>Q+$P02mgEzz_aV#pqfMn+MMA&F&JY;BDfWpd()N%*iy~ z2J+wWim^nZvJ=$ga9il4zfeIo@>EYBC`Im~7;x4wcmU(c*#KKbV$iiyTs>@Dt%con z;%m~jToz~q&frrGO5ACM7vHI#IrLDy7H zH*^=UO3K=NtXLGXo0oPjXjL3R|qJcgY@uhI6$ z)s}_1#?)q*xg5RA>ywQk*g{g)*IEOyT1V;s7*^7@pR>*Ne=sbHhzX$X9498@YWD49 zgT|Ib8OW5Sc-hdg#W~crGOrpPf}KXM`0L^=56hLfg8RZNR3e9CKIQ}pM#`3vGE03< zWTdjh-G33iU!l~DxiD0a5&7fM0XAH1#83Td%usH$urk+rpj>&)>lbDHI7Qw(ec88e zuhN1qnC-qDA)tF)WGc!=aFe&#ZKIhC>*tcxB1MXD?)t zX#!{!?^ZNof|N5@LzwerKBi92%i3 zAhF%d?SqFuwbh+>zHiSIdQvZXK8$Wg&+^sJ?sC_1p4p!YA}+n*3L0I?sm)!0jiP&3(Y+`0CrxvrZiM|4v-+BZ-d*vsT)J18 zu`qHCnZuf~{T1aZc+jrxE_gJ-O^jV{xrL@zeY8bfarB=i@ql| zMP+enH&<&8)vb3-vD2=Cdz;#>mc)LKGKgcx0DI#|p^=H#it;@&38ewuIfnQ63%jgH zONV(yV}BWmGg+C7gM`R6#>2)%FGs?34PjCZ{nX_N7NIK0voT#1C=1N1A4kx1cv&P5 zX8L`8{>0p(S}X=^oN?DJ*yDrMfqd>RkqSnNV|5H}P02r*#C6@Jkrd$2XufHfsXbBy z%bwiHKR?Sx@mnWIsf#23V6ILZwe1dhNOc|Z*Tuu)ll!i2QTElu^)e!Kw`>B*x}*4= zvk6J)=DTj)O3C5};c0hBZECfhrwgk8pj4AHsxEj)$p)~8BcBD+)#jvgu(LgXcWZ!* z6Z0Rz=&XCZp^g)VG=N|~E3_82$QTOJEu5M-K+UbaU;fy8bs$m0;q^s z((6eRgK1fSHi4%O+2alS3R;x;AlSIZKBd*9>f!nvl~H? zEL!NU;zN4uLEFB?rN9XbtRu;n)4jP`uB~j#BBQzrU}`gw;t|YMPaa*0fT{Se~&Jk3Pdepo*(?ib2_7~!(NFcKn#@U_K7QNr!JZt)a8c6YqsQtoA^}Z7)RVeRYvqXY-Qyi zdN=F-*JRJj$la;`OQVlrD!)i}d`ThtMb$(kHXyLavw%wYzDN63l*BwgqV=OdsO! zPLrqRWz@R{Q85aX{R^^0_8?QJxj|N_Wiw%Z2k}SNhPMDp2K!LdYyy$P8vAO|kc8oR zS#`GkI7R_GV|!#h37v;n?UCOe8q}?ZP7HuO5C35FgN2f}ieTX*4S2wt<>=XzjQt3a z5PE%td>f1y*_YBX#&GSAr#AdD;~7eW52rMPT6K@=J-e0aiJjBO22G=Oi8jGj!zqgO z`#zN5NHtg);~P|1@;t0k6IZ(}O^#eh&#IN3P65_OE-~iHWQIzol+gfXdw+X!y2dfG z>cnQ=l}?9+?I7>$4gA` z)^d5s)G_ROKDnQfpp9-Na$thz4-1dfHwH?Npr&+S+rC|l6JtU|Vg1{8p11^KuS^eK z^Gf=+0krrk-jMMsL;Ks|@n<8`&p~6DCOa!?sVtrbq?TnA)w@DXbMk%ut5g3Yy8nM< zFzI3yk;z)97KnV$`igBv;vYDyR0+p96qTN zD@^7|J^$k({8v{W3wzH~7hfB|5Mgm6H(s%ud$Q;IEv#J&NZcLk2$Xh8Y}9#b?aP#r ztzc~W|FPu12RxS;cgRdnZ#OnUrssQnP8gPKbQtbdT5Dh;{(j@}Y1i(h1FR+6k5=3c2Iblu?kU@tI$uRplQg^7rDroChVDrJjcWYrGh8P4#nFN)sHzAE zRqxZd9fBS7TJ?3ThoV0u*Mn*qn_yl>^kT%}B=HaQSG?tNv|R+Y`dP+;Zp{&6MB8o) z)@69RqsgU@G{|6rj_UfI;Enm&XXwGDN)*!1#2GffEj0~}$&_J$WQ0PeFc z%>oh*o{)<*8kPxpNDOQlT9xXov9vT$X{ee0vI}$qOg#y7;hTCw5Gq>Z8?X;5WWsO? z7HKIt6wa5mFrtJKSnXPxxRDA# zWQ&?Nvlmp8vOj1wx5_StC6`5X)XJ)zaVy)K6)Hc19<68Xe;YwB-9D_~5~6>{tj<%& zn~F*CWaz<-ycuji2xp#h1U^Ve3f5i}J)n3>IRq2EF!=eeFq+r4L{K43q+$NBeaECO z5*CjyKAMTyQd&_z+L&`O#>v#f(mX!fFMq&`qO)2fm!5Cm65C1^mQD*FO&9;YMhg%^HqlC<3V(sK;G>~U$>b5Sx8 zqh_BB>L)yAx1?<+SN;Z)o49}j5v$n?@)$OpORbi8NDD{prOtn=ONvKsOu+X1Dci?_$B(%KDB;&WIOlldaf*RCl%|u7!bLC z%473XTBO>7u<>js%ghl2CuupaZ+RUnB6QH+vsyQ`TE5>Gx+rN?4e5`v914KNDnx9+ z0uCaYnqYw^L+C_0HKu;Q52W15E(>FzCkx|^90inKOmSF|n|t!Cs4IDhVBBMbA7q9h zzeHs@>w~GeY^+{ffqmhMMLQ}*gPTUl5jU(T1Ig|g)HMeslOPAl>CJP~e-?M9=J&8T z3Io)$A;xh>aW-aZIp!LddWK&=aN!=CXOLidPnYFhNl$6UZvHDP$mr(gB^-N>1`)#8 zJD?w6Jrz^n*)v+til(0DSby&Lg#LYebp#(5dOp58tkg)q5IpJjetfr`r- z)GO~_kps7)tk4kLV8-y;2!o7yN9Wvv5&o;dQ8ve4=G|AXJk1s|G%|UwY72HG`4F4N z{&99F@jAPB6L(rxyOe*sV!Q|H+l-+@9ubz7wyG5!pF-7%zsC7BW*Nm{`$i)xbBiuk z{S)om|5du71Rbk%Rn0ekhs`HoKd{_M?_QYiFbLUR(x9T~We4jFhFZn<(5}khuAXD0 zY57U#&0akf-tXV(yA8CFB@_dIsklD@Hb@&)U%Cq$J4S|%FTsxDHN11n4?XDX4Q?9@ zj%!;a`=Ag-Gt?10?pEn0xL<`ymk?44dP88LABNJ2J^s)Ue>Vn`j?8BKl3)Z}G*T2g zEEcMT;2A#gdNx^yQRVSJ{5~Z*2}v95F>Ba0^O)C+%ECuAkfX+L%22I%?!{1u`(S^xtYo@@Hh# z!3=}o+H(v}bLV|!0O>SR3?&k&brqjVo~^Lmj}nOeUCE#z?{5iB0S|sBp zQ%9*9I}VCO#?9`PHG5aVpKx0`@z^HOvoChVvII~ zq!^_!P})#Ut`2$f`E5N*`U3I@KAao6>On0S#%43GYgX>2G7=8lNU|J+1-BtPqvd4W?s2zP654$(^p9+SOK})E%!F|MJ>;d&S-H}Brw>&1lj4!v9mb(Y zwW0e&cLr*B>g^5zNh3%X;`OgKt#j~E?(TaTkQkBPM+zKdk= zlGhNpGs}PyryVb1fVN)>2erA=+B}Z5N<&w+BpLWxzLL4k-2El<;w?EzgR8bp70@Q$ z;0IasWZHmL&Bx@tGqnlEv}g|hDTo8uevsfQ*sfWqczcj@i7#}Mi`F#E2@5@%FLI77 zK1(FCbRKZgXEn+oFd&VM)+pqt_SER(*1AK;lEXJw`ZWD8)Q!4%3>{5lkYV>HN00dJ zB&Sa|J?|X}?ind7{$wd?c?D8Df;_B7j@!CNbM`*jsUkv67WN-A@R4c9+~cXE4vhnG z2FIF_3YiwV<3a23>&=onc&GSxbs}Zxygg0o-x~?wO+$Oiv$+w^#6}tRwZXgv^-Hqk z$Ov79u?O>~#v42YtBVjha&5s)hhs?USPx^Ep+i;o!;=*VjdmqXlu0uvZ;|9vIM1p=Cr6-@FW?*N*jklh zj`BF?Jr;d3gU*UnW4t`*G-+OgHb2N1>gj=jk^TXZJamESEdeuH2f=_&(3qS{ULaY~ zx1_raj zPr_m4MQ_gq37PKQqZ!!WD@L;|y^P7BSsZQLfV2o^o4HRSg?~XVr z=Qm?>T?9sOq!8g3f30*r{QdY|!H2zwX~y_Lv*2jix1z6xLIZoj8H1aN$#RR*+9p~0 zf)w^;uM8RqWe|DYURHE289sfnPe+!*j0wDXJq0(86Loby|ae;?euUK{n?A~FQ?EgY;A7T9R3aCvE-zH zw$NtoFfpUdw*eR?8>*2(T0XCB^skZLC+r@B$MMucfh=<3g5ZWYX_^@A*Cas zTe7sjdo(5b(lo-E`{(RUkB94*_zg7*(A_|%OW4X9;*14*}GXt@-*CiyA$8eWc83?~86|FoXA?6T=Y5W{i7!nQKWw%&)z|hW&ZJ&L_v&KG_p0!FNPGND@v>_XPPg6CI9Dh}6x* z3Y)f#;o`2q?WYs%&@Yn!S5IrSUDzW1LU^oKBZ}a6Ez#3-vJid#x7zw*r;yKPI?@rF zsnHfh%rr2Vx`4uHtm3_^49a!>lzTW>(0X2E_=(6?ER2i6rTf$EE&IT{7F;5>rewWKlJ^|3t%Xsx2KDS9{9ZJ@%ngZmm@ z{cJ-W@x9up(%x7A6l;W3sUr^pH%fZdH!ck3t)e>zLzZ*~2ayN)goAsLiWMXk|EbmB zTcPni&nbPGW&X1-u|Q~_wJw8jF!AWKH@&=JZzQ*-tuIARxbp_-n9poxzG;A5-7$H% zGVTAPbygW$&ZqL%q zc3H1C5Z~q%P<0w|qezs^df;CoOUzpRVQs_V@82z-&80zSml|0w(~j|{mHT~9+X2r} z$+#0YI2%>G`nJ)N<8>B%=grzSU>}~Pi@C*CDoaUEOwFrLWi85k35uUUKz&QT=U}!Z zTgMsGv6Al)UTEHSu06K*B)405_J_=R=90`i;wY7Wqn(RCOUcaf+Cf)oPt3fT#E_I> zotlcU3xJQ&9UOvdA|GJ4*bpSFI`B_DjLyg zc@?X8*0VJMoZg#48WTLL68|!t$!Yr;O8M;krL`Rrc7>YA;M~T9q$_T~#r&2(8}mG+ z9}hN3_9rwAlo9in^u$yy1@)_TBmh6gs~s@8>?iZ}D&HwSld_Umc1 zc$G*bV->p1(znmPmI;fy-PaC>oAv&&HfsD^E$KO*Q%x!~2`2={q z{ix+A`(fdspBGc_A~%Ma0ajJmdx1RVX!nmDqooKbldQ8fIx167ZE8u2DJ;|8mfG@l z3|_v8=&F=-ZnAt$SD55^Md$2t<{-{y>}uo%20GEjK*&${6k`%2qx3?q5-`gT6ML+wSp zvz0z;<%pSDt>r6t_IhQ@o{_qW7I{~5XLe3#;3Qq5|a7Nuljx1)nxdLVEosaZa`6}5^cx@iA4QFd0{p2Z6^B#5!+2Rc? zt}C^%HAaPhoAkb%g~k8=w3=2JQ7vfR+b(A8_ArF<(SvoEQ17Q+$U|K>Epu^IZ(Jwr zIf?I1bT$8bNNM+5)_fR8dtzB=@}Kerhp$kQA1E0vjsrtx(|1Ekm3)SGH@`kmqOFEf zi`Hx8EpEcRPq=n6_pN;_S^gGG2rq0OmdH7Gm@8Wyi&{5_wpuxMtzOk%B(J^8>mu=y zYE}da1KFwX(2KVmjURP95LEeOJZ*H+^Y5VSu7cn1aF-8-(3*-^&}N_4xxvcQiy|U{ zOWaJEsPvZbXX%6tpkZ&cnA!Fw`z2!~IF~w-W7CNXsq7aUA~HREY(=~ zV7wlw?FQr8Md4SvP9d~Expuu71{yz4V_i48*l&z{`O|2v0NX~ZwdWvp1)KG!wA2%j zIK#u@Ci)tZTB^8I{Ibm;koSq|zf(iGJRj1$&y?5SNG~j9FN`^`WqM~I)LzH0+>_K7 zrXo5r;8I#ZIH5KYGR8L&5y~j4@a7P5R`EXH0Q9J2-?s+p0_V%5u&`ZW2#ghDP+g2m zC(t9#Apg%+n=Nv!54p=L_v>7WJ(hIO4coh3s^P>zoyKd8K9n=EUsKL#e;2YsZxq)Y zz-;Xr(^stA+_bpL!@V9fU?l7OV_uJ6;TXkwf2)+7f{tg?C^xU0LC?YJ0r}msN6oUX zl=VRo@}U2a)zx}l4yBoE<=7A8YR+_pOiEwsan%zKo1!Di3=r~RPdL+iNW#qnXoFCi zmJ{dDaD<3SHV7`?fGQ3<3i*lpz`3AqHiBA_F`B^H=M1!rMHYwajQak>VByUKUJE;o z;4V4R=mohqtKBDqY46QPvu8-?0xYk=iO32tjIlebxduqHUVB@+{DYQV#6ETk`RfjE zxAHd#tImMxJOX>|%CzCBNyXS7nBgnjqvAD^y`zml$??(8xoQDl>g2Th{hi@IcG(wA zjQaMy2g>jMONj+evaS4$Ek?OYnzo%JQmzHr$L3={jwEe$3fX{%6+b%^jUsAH1`Fw5 zkX?L7X+SiVf130Xuz&g1V-??@I`-pNe}>g2z7z;Dv(H%PAJti;ZV037Bd(LxdkaVi z%DyG3b$7eC))8OE<7)gl5|O_0pXHXxnDrg255bMP`seKp(uN-8E%xqC*HgdZ9WZq& z?R&o`cQ?J5^6=M0eJfqkxqa1Jbi_x#P28)lmRT7sqx}m9E+i|qe||9HbKg-kKg{zt z5wE`st?9T|ea&?|(HwPi?S5F_HX?X)+2~xM8@`Vqwr&!Bt87dSw5zh4lhLTd1mOfQ zr?_YpJ{i7_R-YX`{Z9z3+GNYi4FiCPm$qa8IlxSL-sUEM&>a$FJ72(DCr@YdrFYa? zdxyj+recPsskao?prfv#TmQ>YfiSI-*kz27sD6~T~KBj5DOjif*+LpOhSj9#8y zej30->FCOxFrrqANl43#*b?i!FS{>D6G2zLW>qC_wWGe@HD2e`?$AhYAO0qWx~_e@h1^RprB8e@~b&URlIm|~UP7Jh%je=U;nQ5)IuHtb2XiIeMSC)^wv=lT`YrM9EZHz0jp@O_d$6_e-` z{BObMH@jz$AMO!9A4^CL$Suf0bLG%zeauN1{l(u@>k#mGDZMaksDmsrB_VYk$^B<; zCCJSPK*yEl?exREBg&MGFM`WWk(tTYeV36I9FM{18zb|7xJbQ;{hJtCA0Nc!#x&?7 z#LqC5lXYn8*eM;`%q~rQsIik|s;2NlyxZFX<8)Vr6Z5MCaqH(}aP386DeKIJ%H*m% zQ=yJ?D0y*hVgOajT=UfhRsPITE&>}{RdeY&+g8Y;@))o-0efoX`K1b5Gi=Xuafp3s^R|G-TAjg@stOnLt+BBC zEBa#Of!xN9VpAV?{L6lQHF$am!aHIm%R~8jEGHkQA#6`>2csONT_Ue|ZJbmvm7xp1 zjg?iD-i?j<7kVE=u1x}o-52OF_&Sfjlo^6UUW8%gYPL; zElJPyyxg_H0ury9>~9YsXAy1e_kFL+y4k4dhuXs);IuwfZlYqVS3nvM25`Dm9ZQP! z*`G1d>zB81_g#H&i(9U;ElQ;Z|HcqZeeGmPvfqXNR#ZW}egP+0F}~!P?EIwJv-E89 z?Y=^=4d!YI*yz$@gZ8K?0q>KeYB6q~Lm-ucPeNBD)A!EfFY&2wfzK#o-JjV`HGj-D z(Dk1e4VatP4LHtmoLde0Q&6?p9NVIVWcIf@=spGID!`o|9mwul3+QaWvXL&pwj@_9 zKqXJW=d$uwg>=%HuDnbsd&Cc0@Ws4m=da?A+G z2o+kb91k*sq60oqFJ3pW#!RziKJZPcfEx^zy;82t zt+G?YzdaV}`H;&^)jQ<-Dbya=u)S9`rE9(+YL7mvpf=1j2~r(;=lQb!+Uapl1O=Cq z_m{ZD@MuHPzaBq56lb-~toJ()EgOjqX>?NF0-e_e`CVN%%u?!YPg(qio3uh|0V?iel|9IgR?v~H5FrrF6BIhH1IY^&2^9#Jsvl6eO(_U(H6SVXv z6(@7x{qQ!aIm%zd_uQk(^f_jb1T*T^eSOREUlsj|6|F{OZj+Vi#RW$pn7`Z*+uFZc zezx_6pFc#{Y8M+Zcq+^L8ox2WvUGF;7tybS+HzhY{RY8*IbDi{^!#A)v@6O8k>G>- z;+lTi?;F`LeFEHhQC(#&;^*MY&YL5My%js8gZTIMd#JbE(`STej$k*eJ-{R<*!z1+ zK>lMt)%4Gm-_VuZAYgC7^pvG|^eDHAY3$Z3ytvvdyZ#83V(H8_s^_(>Hxy;p>=(AOx3!R9KQd`6D7DTz8~VwST$BXE}C$BBT}a- z{kVvrbXo(agI0Xc7R^EmtJ}vG3~2{$H+Ga7iGAn!BF#vkMz|DE31G@O)k385WRj7M ztAATUbY$lEr*-mt!~D|Me3kyki@Vlq&)I4K6`Gy;s*b$k!=%~GXkMuBcQ-jJJ;YWi z&%Yj-c+n>U=ro5;&0JihbZJ|=+9L$yx6v5sgEaWPKdbNRqMfU%{&m>0^U@0&$O?hC zS~aKh9Z;`*P>S|%@E?~?M-i(9$*y!M7@a!14G zA*Y8bPGVy0hjKPj@^{w;hw(0=abZpGo0&RN7hYG>9j{i^@%%h>nvXgjt zo-)EM^%=~kS$8|dI!(bW;`Np82is3{ftimZ!Y0eR0Du0>pO<4Y4KR7ccge!bd-8Hh zRqaRH5@*9G8FAQ2E5YmI>^rwDd%1JUhV5hTTU)0R zrQO>1nntXo7XY!!nR>6=U*IyW6Q0+`7D#Wgl)O&M3fsTG5T`7c&JqR35j>)9a4;E5 zwc`>`!pq({z-2+#qhxl`b)+F%7`L7};LbZm1wdJ!9~Gd&o$+udumxK3!hbzWwr_oA zuu1e2Iw95}>W1+t{hNTiu+`iAr+AFSs;U*!gHe~~xD5D3#0qy|P~6LQsS39%UI8Z8 zwWrt#OzE*UNkF*%%Gw!Od;?9(f$;HhMP3lU1MVA3EvW7un{km5sl^s|SsQMv7|HVS z+z!$}$gFo1@C3R!*FaCe4?F}Mz%i;J%1&h;RM^zOoF%+I-!>=)%kNs^;Osx;Bkj@$ zoajwv4T<*PX5X%Nd8yVNNl&xH`F|JQ!CbXNR8;k0`U(dx_k^v0ulauQ)`hL-8J#YW zmbO$TkMc%Ia$JWpcACuL+als%CN;>g-}Ut|OZP_P^0;CD$~{Y@+^c>`Hjmz$4QB1&3)y`T<$#=P58csdZgYKW z-3}uZ(2?i9AV7obPb~k@H6;(_-q%p#vWAsxy~~hmE~@xkcSeBVbwf->*qr(qvDqv$ zru;W(x{aU7KPg--cCc^N?t-8pzFpET-DkAR-`HexGdC8&LqVUr_im)!Kek~AUg(v= zwAaE92YR))J-;?Ne2Uz$RKMN5NR4)|MR>g}?7p`#3}>RUx+?tnoPKb=F4JLY*)=D| zYv=sVjrlkHV{NyL=VH9tROdx&Pf|i~H&$T!+)$wPk-{sL7nd;C(_fuZ?ULy`7&WB| zLz}DK^YF#%Yk?n7+wA!dohR)wHm=p_^X$ZrUZM7>0(ZI2r$#hH5U_1fzAyYMOMk5; z47LkrZ--?)nK4&px;w#TDAvC*mv-rJDN&~~b|HHZ$IR(s>HWn^A@z42cbdxE$$F@D zoZrev*Y)hy8Gfw;$Jcke!Rtw75!zY7E^oDCK{pPxQsFwM?nJFCvz#+LrNiOJ!4b~& zIsrV_zsJv(@uB?ll=5us!fe{vm0GbK(D~QoPiF6dT>QjS=1_#ucdTU3_p}FB>O0Wf z7d%VJiZiA@d{Th16_xvkOQ(V>THix-KnQ~c+dIboc9vW>8`zo0AmX7K@RO)Vh$ZG< z$P)eA?a!bcNIhj?W!?z>TTa^W?62||7o6s`3B8BPtjXW;^6Tkx${I2jHA6fmb4Qy_ zZ_Y$2{ES%8C55mLr9$|#s9Bwdwkawy z{r0NmY)4XBv!=Gv4;F5y&tJtqvJjXY3T1QW6kq(oH>Q$lSA6+M6eoTGDX0G@j9oHh zD1IXk47!kmUg)2eTAA5HKTS>+ykhKnEyWrfc(p|z0nl4`!^eb=4gj$n5LByGGHrKc zQL)l*h0H)Zf)Vu6hh;QDvT0gPAbp}s28_(QaYU{Q@rh?!}e3BDN` zb!)p+=ZBTEhTBcqKV)rV)juzb52d&G={ko4S2>=>zO@Fd`qQ4sG zLBj6;Q!cL?`M2JaGr+!QraCCP*Svr9&gZ)xc|q;H!s6A#&F){zLCK&3u-yAHljcq5 z!Gglh%I!XbNkgG|Hg#*58^Y^VEGosQsE_}^EHDy@ZmvFRN`*TFoP$<4y4zffp(!ot zUOR|!no2w;e*Q4Qzao5aI-tdgGdj)stAA(0@E|E<;9bZHBg72d6GXxITWQdK$pvU}M3g*^JJA1WcE&rI};j^iw&2PZV zw;kTNzEE^_#W4l6d0U$|FyhX~Q#xL?l($(kVnA$UM?IiyT#Tae>mU*39dYVe{T;5f zdBn%~dA+2l%f858^{;nTBFh0vocH18;j!#YcKfq>*4CY%I_@*4Qp|Ei%p;pqCgh!M zotn}G0KJz~{R=NAxP-0~Rld}Ux!@e#pI>0X^c8ugu--7>I2~o{eb3Qb@E~k_<+Ms9 zdt}nzaF-OIpunBc9!BQrw+DTVcJUnm`}6z;=C*g1CcGYE-aY|PawqS=JNq+vXzR`< zH`AnWyHPSfbwQooTeo_zcY|IJncAkFwY}S7GGi*N9c84AxP7?L+Ocq($)Sqdb%bN( zp`zFoki6HY5+??oyh+J~X$x_e2S&C()VtE!A}q3QEoa;OHC>Z~M;ui0sz~b<&b>=F?)@NY?V|TWFME`G7kjhoL=s4)+R1o`UAE#h zaJ;Y=H^ud{=rLMN48ng`3^WV=9H}`!f42P2srd5q+58n9zUSep-eTJKQDu)eTNh2H z^NGIqA`4C`RnzAZCI67RJy?71a^$+UMiaz$1qJ)NJ|iqc4e5H;gL@Jh^QUDE)aTQ+j73bEZ`R(5l`u<+vXHpk9HT<;dLtk$bFH+Spf`U0Xy zLTe4)G{8=lT8lMwSajxS%ykkE)Fl#R1cP5&PXAylBwmXg^q493m8fL^6xb}*+j{#e zDMh=|j}KF`Bo3!ceL9IoR|_EnzJWm_7XJ6x^msbsxlSnZO8s!t0$5=8+ImU8$*9nV z=HYl2AFc1ny6fMG@0Za>wvoxLF{Gub*@Sb4s>7Om@oomP>+StT9`K$V@06*o%N7x}v+PIFqHhor47Ss-X97b+Y7y~+T{tVO6=ytykBhml=xWqvIr>fv)X zf^_b+&xwG}CqOU63D6_EZCOs+H`zx=qbISmaF)N;DIaPXj2@p`yF3P!9lvN>e-}@z zco;+>ylbSqaW^b6;Wgi^Q(g?Z!7p=g5WQ$kgr*>|ne9Lx@9U$~(mz9f$Q}Ag66nR1AgQT&bMKn3h0Y@6y0d_D7JrR2+!%%AP;})LGkG4zwPKnudOlVa z=igN(d!G@Oz7khcepZ{zT~pGh^Sy^9Gp!;P7)yvZnEUW~5T$$-5i!^2-HH7*;R?(# zpZ@tH3SkSv0(J@43iUa9o28hv1SxVm-R@phskgw2k3Q-MwX`#@2k&%$estbqi=TmW zi@h2M=MHFk+!g#--CWktLEMxePu+5pHilc6JU9Bkh`eT?yHb^uiv+%6G_l&HH zzEF4GW;)~wWP?RH3-oC`QOVB)sM1@QlICQy7du%gF@ZhU-zv_v3T96kFV0-^cV*^# zG7@by%(wKA)P+domLl|nt0+IJ)fr)+*P!Pz0#>_PwQJ^s=HRl#=2kDgwmmQtt}%Na zsDG0LIIUzh>ajaxt0%;+zWBw)KPljJVPAl?Wp?O+Et~R9D^2uF=qmuioMrkMoAotA zRNoulTij08{W&dHMiI-!@48u)4?v)^ys(R@rmh&~CmE?sSARYbJ9F1ZT;fAn{2&*w z(A{{v^%dHx@Gv!dDd?=Y3GH!?-H%SMMce1o6_lr^1h_5ZKAaIh@7dgy&6lCreltY` z`$HYo-KyHmsdrmHA1=O*X>#4Nd!8V0d)l4lln%iYZm@hUotb;#_ZuJV0#o1M(dGr! z>(bMIO6SjJH|ePNv31G;W**H2eAbjJ%MO1IlJ>?5`CiL}b%7B!Yu{9s#MPCT1Co+H z>uqNL3Wo=* z;zgSm0ABBz_Eg zYIb-fwFsdx*+lMt+BxawGiWj6fX?^gwnBi*_Y2<-c#X6MXMI83-&{2I^BHfpP$#4l zWv|sD;uiaMrSFT&e_-IrTlyld5c%a-5VXD#;l*-23^`mo&g<)ubc%=&fuC3Vq#O%k z0#V5m*5=I}$oYcwfIm#^z$bNb1Ql~3;4Y&AF^cQ@QAP3x4f130>2{O*<686gpwi-J z0lt6ZeQ^T|e9Fw7pEJ3<6jUysnPIMgr)SEWY#t-g)?eIjySK;vNLF3je!g^Hw2|fd zc>QE+zr$Yi;6r*pXMOzBq=jCmg|9aqPN_O=#j<<6XpI7I-wWi2 zw(Aru`2WE2k1%k!S$^IpdM@AD8?rax!62Y>WfGGtfI3iQ$2-lgL?}pOZ|Zd#@X4t< z^Tp|XPJC6IUmu3z*O;ddjq|gP8taleGFpD=nvSFni(XFsyt99hQxMy)&7VeMX;U3bad zcKbh8^$w}!&TG5^N?7le-l}fP_pYBJGj`e9gM!Ak#MK^lFlc1m6MsjYGFseIYL_{d zhSwgqu~UWpBp#ReD_$E*SJr?JM@o#RFnPar$R5F0J7Vn;D*Dvj=Z7QV@W#c|y^elW zIZ**ke5__8<^fgj%ODmirR&NBO{0!61}7-5xtFWgoKC~NhHx*^yuBFK@ic6 zKA7l4@6ibvy|>YcKBF6LFuuvV?!D{Y_5EjAXHMN`KhHU5@89!_CEf4GbgodauS{XTs{U zRuJ%|M)~|1-c$SXh_7l~TT1jw%0;S6Dl%SGo5=*k)3A=B=2n#qDgr;l>^od)V59Fncy|7)o=?SF z^=XPuZ6&HS{yVSB5^Y+a?b&fJi)Z`bv}zl-K&&!|r;RH<+OyHoG7DBK{bW$6J~o2x zv)8BObRy!}$=|vy2X|+RVu)WXvK%FX`oIAb!xQ(Z*Pc^v*BnKjod00KKRecS%!g9~ zhyVWii^;dOMh#;UsxxpdU4{EmUuy)|v6}~LT?PBiV-Hv3$L+B%g=BYs%1iI}MhJe^ zjL?jJ2V=Ese`h#a)uZK4jTKR?c9hWjnx?Efvi`ZPb^j{|+vb~3F<6-`asZAkVEz z>VGMJX`$9QPo1-(`(yJS{Mb96t%i(ELsqA2T;m+exIiV8RZJCUJ50xn|EP7#zMxbp zOUjZ^_tw&}xQB}ra?LsZ8-65|4ngikNX*@R&PCOW;nCOEn`2$;ZY~?2!bT{ld_;(v zSpAsCYl`dQG>xkS#mcI9?Z~=$6{^A|Hb4<)6ZZVJy#8JKOJ{1!ts1+^=FF_GoPcZcf_qEJV)Szwlicb%b2K)`iq^@?}s9*RdOaflCCoL3MPVd zCXWNPt};t~N}`a2b>a4sLsgZTrB+vuxlc7SknjYUHwyz2D@Dp0q6N&1`FcCGRi6oFl2WZqcWZ3cPm2Q~mj>jQMckLu zYYVguA?%OJnGXtPA(@G4PM=j>f4;c*7#(M?(U{-3h#!7kn9Xk{H<@jj^zpn9ouVb3 zTDFqIO>Ns`bl1!L!_e2F+g&r3q+mRtN-3u$JzpnUO1yd6!0MzWAjd-a>jyovmZ9eS z--&kF$uE%CTzMwL6ne%E8tzjM$T@ysBj3=K*!kn|1683d_Kj1Y^n?=eOtCoSfd_RZLLxO9!kgcG@aLx43ck9-zwja9ZaI8DcZfG|6=|F##8hkdD zOLR4fFpeJBb=6QRaCu}j&T;UDDy%3ySASm5%K>itNp|v6Q{^koL__dK#;TEqIZLMh zGj2Z^rg`%MKR8W?_74z812do-C}K_1UsMKpG;+^I8A*$Tv6?%^8 zbtQt<4>C}0EAI==nC(Z`rqT9&+ud$n<~l47wGRpFTO}JN-->SS`bg)OeO7lHx-+J- zA?|vdmRm5WwcjAT%EO`)(Nju+A10k?dslK&E&gNeh{ltuR*R_vjH+UJyb;ep{%ncE zwma5b9o8}?xz~I=M9;vU4ed3K=qq9kqbx}OHFxE%=gN1Pz@vFIov+`S7yJV#JnJJJ zFz0g+*}eW?aV!I>DNt5kW#6m5v;t)!9p1HA(R3QPj~{b|eYXKn%ju3I$8Jz5c>w(w zE(VB^)Bfp{Y*OnZwvZXe2Lju*akXsdL8fj;(q#6fPPye#{j7;?okW~?7^nnm+}qct+T1<nhEtuLG-!qC92Hn6Lz{xTpqyd-huKRS+rlZk~_ibm1)qS!d{`19K6$Q`=1gX6~8C9G?JwJc~%0o$zNQG1b^|D3byQU4Ibb!ES}=U+uw?tap!15bC~ zo7x3Fdj@u8P?n2ryS?^cI?z=wFnU>CfyO{~*J*d)L?+g4eMYysIvFbK+1BF8oYDnP(t< z_6xLmYr(L9aZGTr_90W`?jwPV9{p5IC~M;O`*60YQSNqA(8jqE)B)K$koeenHop1N zqrhx7Q|^GU#$5vld%Z2xGTpkuSU$(6Z0~`35EDsO+A^!-8w1?*3mTH;7Xg$9rM)?qvz9?z9?%?U7uYdcbP8k zablT!uykC`!r`i+;o;g0@^0(62bSFreOXplQ2+N*?grj+Nl?L6)JL)i6YHnFcW=#< zlk}VPgR*bmq_aLLeiY_2IdxP_HldHR2GHNX`#k(MXDO4gl}P*VAbx(kAsgIvF=WZE z#ow=UV7|fCD*y8t*i$=0#vN6xRRGQ?^D#!9f(@9?eAy6s?YC{TvRx~J=W3vSV41kd z-r{Dz@^!{f0PX7+K80tzO12d1uzF!&Xk6F;t=?J1B45K)XkZ~ zBD-#<2WCRhKVL*X5I^zl;1OSI?@zaIgu?5%4WkH4wlt<9)M{3Gheth+bZ*7Ny%F=? zW$8i+nvX~_f2T}TWaIc!R3}ask+CE$eEX7qgLO1s1Tm*u+&qvgcDs6ZC z)uDHPP`h<_RT}kOd#3tDwDn5`|pZ>vOXvC|DuSdhJ#D2h5I`DpsYkuQJ|C296oo0cCJS|=%TMzty&soHx|$K z!JxyhMw?(kuXZ1brk0>TZ-z|7Y^f;G_Gb_-qAxOp1U6$oZvRkn`*H&a^QPr;B<;`j zbsO&s_05+Tdp{0@RnO`ru#TJ0lUsCj5Ot=$qOpqK$%5DWvU@k9F6!yrYwor-^uZ)` zg&A+uiZ-^5eqM@~Xpd9(qpaLmi3!sXx|On=~>JO}5Lqa9BCnQt0PS~=={BYZQbR17~Z zQP06Y0&@+gT+P$&6QpaiU*+m0qZ)0>hhUrj#U}?w39skkOy6;Y$%PF=QG-xWly;v> zM%WT(nvEQHV$?eIap>?YM?Z{nT6jqY=t$tCGksiAXO$Rpn4Tz!aOD^B%})%mdGm@ag!Jcg}rgO+Qs%XqmXaGq?SSn!%Tw4t2n+Y zVziuU{zyp7W@VKOb*W?(n3Z%duVkac^n~b9l@E9>tGGN1Hl=_SVegG#%|tghE;4Ee zs9Gy)gQ&Mrn~j4q0o7;cxx0UxHEI8}mLT4w(!sQzx z&D6WX0Xxz%Y^j@ZBybU#i`mRDhFO^+o+5Osf7y23V@K4~ug2kO4^*9efzY1k<2at9 z$+I|%CyqyG+lB5CYRE=IS@I=w{lg3^IrKeg+u5~AH{PZ1uiZ0xX#Cw-_8#8|xB0V2 zyLzeYsWscOcV5=WM$~(EdZ11dsUbbkpiUi#SBX~H4O`t~NtbL6wAFd!B;`KDT1gh2 z=8->|w*)oM{Gbr-Sy6aMA>cGQ4uO{%9`Mn+1y9(b_Ywp}2lD z`~JY^h&EBe_I7atsZi(>w_O%P#I*W&IYL-lRH6JBhIO|H*5kThd&;1*BbT<=c>sSD?aL68-vAV16xInuSrfdCA;T zNwYFm_o`N+tJGoXI=1@&5}ChIc|R4XWtDVQ%YSlR5!Gmyd1hk%f6Slu zaA`%1eSGb`;NWK~qqN_4gaMYlV#U7kq-Hv=|4Jm`v&(lr0&Nco0<#uJb1KED7v7L` z@@e~qiM#7}0u!RvdpVAZ<{0^slf}cW`p{@gYqa-n>%$5rz(+xXUrZ}3_R2pVj?h?> z?l=Mfr<_9{_rm1aWh^_7PUrFe6bCR9Q3i%8{?{_%04f?>*)vObeMaoRlAPY!e<0P*AW>=<4|7&%~OH%P}*Zmk%^t1n2 zJu{IHU7*@h9}?DL^G*^#MfGoMPKd^nn*VI=zjmj+@9_xH6(8uiMmzC?5sWY4gQ4DR zQ+b{@uJICbn}DBj5`d31f%F6h9nWkYN&N3aka|fvQJ{vp4F38IjqXtTv3+rbXfkcl zDPhTs_*3m{OUh68?|0m@rKO?m+%!@>5$I=om)!5oRn-{eZx;_a(x*fu1$^`=^>wGw zK_DILqjsl^iEFgCpZ8`T8i==jU@X5D?0nR17uS3$yVr@czu#0k#ZuR{1ec5&$F}tY z?iLMTrIZxuFhHtC1hQCdS;B(;QQdGdswyU z;D|M^;b7>P8C5{vU~IFHTs{q})@cT6(p~*p$qCcy_EOS(Ayb<=@I~|tr_s@L|C1@x z>z`Rt2H;YMyi&4L+%>12N28TGB>}gKPrpDttA)5nW6os=h$2S*6gedEs(Q|=&UvBL zV{(<*p-H|h&P=kb)_-5+vJ;rI+Q9y`QWwKmBGa~#LJemSxQV@a1gDPA6r` zo^`lh*z>tc>#3XrdW?DrrWreZo^A;tkdNK+EIV9RSGjb(M+D1|-Lmx=t5CQzmLqBc zsMn_om~z-Yaa&#f5HL<+@(KcP^Vku15fF^V7XZ`l6Hb)OABM4Y<^b%Ne3%0VdK>A` z*qSnRTU#{N6Dj8qJKCuaR6y65VQb%-QZE8nrY-XGhzXchBLo|*CxrQ{J1_G}pY+Ww zp4J)e(n>Gh;Lbdg4P1Kb*4nMwG*G1kRZmwN@6RykT75WKY1#L4^gR5-AXHV%*Ka#& zR)M;*r0mLGGrk8`JNny33X==JF&bXFgVQgn)=jUQLwKMaKQ3ncm?p8Qzh)NYXC8iW z{MbxgHliII#bkUYzYozohHSF7mPQAApJwiRPd717X;p+mhX*B><`=wHqZZt!jh77l z1}7K7wG9?CiWUbY-EHf48_&*K_t*EmR6ViI-5xxdzAm0uhuxk*0T^jBx0Miv=(>cX z(cei8@uWLlDJ#6XtHcqGjobO|8BKeQdnlai#s_VWf`>Xy4S*j@mzfcbXvcB%JQ@pCuhVdo*>V{iG4a)rd(Ga4davesM1e}VN48gtaU%g7Err;|?>_M~RK(af7)6NLM< zm)lA~*(5;f<;Nl3{P#Pbj1L2Z678H$?Xu?-B?-!Loi_5Z_x$ZX?a`g#HD>y4s*Toe z*!@lh+C6HBOlB2tnHS`0Zj2;q8$sUJ*Nsg>Q2Y`EPhN8Bn741<-FtFRPetTsWU1m^ z3XSN6u5nAPH=gzDNc2zB%7I6p~^xg+w_T*Qe6xjCZL`gQFA zRk%ajTuU=;pXM|-ebonqimhEXeu@^7*Jq_?pK5KZcz_c@x{!w+p8~u|Pwn@m6+9}k zBDe~*TgQCVJq%-Z#uHMEovT1APRTLBQYo&pC73%yjO2o#jnt63BSBf?e zY1V*0wr~x+76(9JJWui*Xn}wY)i2TA<+iO|SHJ8NX%QWAwtqiDcF(+7E~$KCU_PK} zg%+y6CnAb%;~$;mO1k%;F2kUt1KXm9ZyuWnst9eVpT2uhfZf!0Xn^F%o%A{0N~&^S z&?1SMH74CPYO|ygDCWR-RjBv31*R?B-P&rPy<$7VKbISxx=woUj`+B!vsQods(vR}xLeLgl$7IgJLlPIe-W7Ly(o5blqI!M}O-d9lj>>p){#o=1 z8l;J^!2I1ZW-(AHE{(zmNt35UW(b$hAs>SV|O*gp1__th}sJ;P|A&a_Z;tVTJia5wkvRA+c_pTPK`?`C1|CD+}r)oLU({5aU5sJ!Mf zvGBTlrp0OB{7LEs@wA8gr3#v#`7WU6g35Jj_WK_dUQ>N)WBBg*$xl^SUi3hcY*kkV z0>FP*y+;H+fPFoCOL1z5i1V)BZ>(2m)0-hMp*SbwNgr8o=@(eQ$Czeynq9c1sW(1& zB!?X4!yNSVxY!HcZH$?PB%C8&5Q zn`=B(h-IdX_2gS^SZ0ju-mh%vUfx=mai*50Kk6&Dwz%6XHr}edn=EzR15!W3*2zmc zO}WEDg|FIJj)oeRd1bNY{H5B+qH>3f^1WQ=xB2XTMVnc^WDc|Ozr}tPY4fbdJ!|o1 zFSwDoPI-;eWbW>vmHuorzE>+SDI*u|U#;_XSiB(&n=Dg?>M(={s~YYZ;fu!X%K0%) zmBSB_uf9`MBYxy7H{DT#=LDomDmMf)?tyKlPyF_Mj6G*8k`wbC0xsv0olk@ZJ7#$s zPM3gc$u!Y>8OuS-4m>Ev9_0^fXZq7r%dsik!zdb+HNk}7JQa27xxy= zT0K3n+XarxY(~?@0e)xHHwV-fTHY}}^EpyTl#7(%&1L|u;1bVFoke+oWp`_+7Q+0H zVfIq1^rZ)`n!~sY_zlFYxfkyX;#OWLusm{wJ;vu^ZEXkTlJ8HDItM*j#;8hid}^k4 zrg?+WT<}>u&a4U?>HddRZST3lFB5PUw`jY)*wg7(?xETgjQvg~^Bl1cW3J2_7j_-A z*Eq|4j}Jh2B1{q(;@+0nS3&#mzbyck;wJne|I(Lth=*5IgreLiW7k26JQR~v`dfq! zoBhP`QINbI8$D-+EG-*V>36rN_RQ>WR=q8TwzWzu5C^US%T|(plODd!qL6 z=RXNM+SX_<{txleo*fI7{@HTy!GH3Dw!5%on`jvN)YdSp7q^dju{6@- z`BunlJOyXJU4IDw>WyH?qotsa%br-Wt3due(;U#(!~%M<`iMX+o^vcw<&HEc7dQJX7LNT);n;QGUminvZJU>789`KS}v*t3WZ#X?}I-a zpXF5%UMIWT@gqZ#i>k+c+Hm?Bw}0b*HovJ*e>o1i9{+@7KZ*^zr`viS(e_I!%-PyQ zDY~Txnn}$7UnVOCod<)q{?Qi!e|T9>_y4(ajpf*%vEb3znMM?nw*G%j>WhL4zD&rZBm1?$v3|lS}XozZdHjC&Q;#M2*(q9d$z3`wte~B zg!1Im_*<6419YMVnw(jq8gJOlobx-D3~+TWsWlxx_(CH0!_J==gBZKTao)a{C*)@{ z!#OjLzlU$!&hTnXlWLp-{`(UePxZAHT>fMD##E(RP#?;lO3lboYDJBEV1!nGGZn0A z24Kg-G!m@{AFUqL8ei7_32!_98_~pZ9?0?JS-2fH`aAGG;&86*dQQFKJ^~nN9k43t zpL6-|P7<|rXL~loan-@G(|Uy0)4rJ$Qf6P=^y8bjg&bOD4}ZL%Y7;9X%T62CUP*!L zq5UgxB@fV22-2(Gs490f4dI_@!umr8Y0LR9S{`F6IFF037VJ0*RUA8OJ2??u6X{QieJdTdlH^v>nZxL4*}_oa=IU5I zwm$`0ny+n@_-+kYPTV-KG~Kf2Zjm|u=bv=Sm*Q<2sS<$0gHeq>r`7O*3$|2&_Ieq$q}N2KSOey1}FW*nEuXgf0?IsE)LzHPKEREnudaQg=B zEfVBdIX2BX@P^}hl;b{Z^|Kv@@V4&K zKmgL4j99+9o0Keom|<- z%VMM~OAWTZNIA8+dWnBYe%Ji(lDEi!74BQi`8RrUCZdJR`mgPM!~_CcK_BQ3coYu9 z^R$liW|@g1?^r1cZ%zkXPpcDvV;_Xt*cxn>|M7Cpx3dFWN)=K3j5_<*LKv)~f+rE@ zFJJRSQOhl1@+A{B+Jo6KKZY#-j$|q4Rn?FFvl;s~#x`y*G6evfIJnp8c^rGCzh%~T zWYs%kvpBF>d;P6D{f(P)o6>n~>qKfqjI6fLiX1PJ9C^tZa6nZA7OJof?bDI9>5OaH z%8vM+6Hm;9e_B!AEv*qH; z2M~w<+)_z!;IdhVq$B@1Q3%l~*k20rTaeibCe8AMP;>RS^E00I?|c7oL%|v{aTlGS ztImus3;$3=dV_S8hf8?saw*Ydj{)Dec>Jr8A^S{m^%Z#gkC4Cfj7iU9*cvJ|m~-YE zId0}_K)0u_s)S-a2Wpe7Vg;a3LsU8LlXD)iN_-Wu{(kmrg95p;SC45T$W$ zHjvjN#rZf!fehz|_az-6yoBV~T>v$|T_K>8IM!}wE5_U)WAC0K56{(hBBWF8vKn+* zEgD)LaEmbmV(xIL=>;y1zWs4lV*jXQ z$VrUFKdWA4Tnzj(;V4ZtV_#vFR?kC)xh~`A9rETOys_5)Wh$Td>Tyzia~_iG{I`g= z)^X0`+0x_rRZs%SeowlG!Q&Y3I9|6ES%i>V3ZE&$4)4GZ zY7Jjv8Ptpfr8+)m2urSf+H0Eig>-PzBmo?ePi{)_n(C*fQbCkDS5_!O~TDY<5W9_3oLrr4ry+HxyQc2eu>S$k~LjqP*L-7KNo zDy{Ttvbs58F$&5G`f?6?e6BNIdI(cHIJj%)JJWCU@k-rZMNqpgvXsswsriBIM0sbB zo1M^{jiD%2s`^%TuvhF`Sx5h4SigEEwyV(d-KvF^ zjN((y=AMTY=sf+SzQGkxM?^U>I^*?h5ZO|P_v6tUnNey&>(j4STPM$cFtcCbY}7J1 zy(o60tB|n=$zeeEh4E)uqW@%}s(P?w{TJ z2UqQNDEGT(iITqNslLZYEK#ju3l+V2E=#HoyYEVev#cF3f_ zg37fNG9#}X|3XhJfXFVA3!JV|_XWUXIPx`1Kh)00bYS@ni}x8~lw0Y6(oz1kn8K=9 z-=nX%z3XZI6>BlmX5u8fubrybHbsK4A$Qw(K|xM`r_lggLL>E5@4X*CZ-^c zIL?;~h{3dPi?XP(7#sh|kfiK3)zOqD>E}jFl|uKiW&2NgBr^IA?^^~E zBim`C?UEnpRGcLmb|?ngb!@v{DoS25Y-&81P?^|(52fa8gtt?tqQr|iBzAKzAzra? z`K(|>pA^@5igSL>UQ~<^iwEqLS&~LkT|xtje72@skagL7uYa0~dAF~ie0FeyS%?GK zaAzu(?&FI7nLfJ$5g=Z|G7_wdO?1&dvWCggLgJHKm&-X&Ipdn=>*x?B<@IfK;L?__ zQqEwFG4sxiwXrc;q$7+UWdjPgtLT9Hn!ll@9;P)~3t?mV%Otd$t(8x;V5vOcAAI~Y ze|Z>~(zXUnPUnQmjZ=GQj$~>aiz?HmAGvfQe*_nde>RdLLqaM!g;fJw^oE$Vd61;$ z5a(6m5EEt}3QgCJB7UCTL+D!P(G~7%-gx*usUpqx@l$YC)5Cy-{xz2 zy3`Lfy_4auiv|#0+?rB5g{JFD9*py}ezc1bu?lMuf5c+9speIrZY0%;Hr$^*P@CY< zF2lsNKmo?Y4ru3Pyr;3{26GOqmf72GQ^=T`dj}#0+zK?bef!SoQQI3K%hM? zmcb&(8*Bh0QHYn$ZC(3pWHYQ@WO`=`YvLGWjH0rUyku>j=oGUj)4M#SYkBnwW^M9( zxP|65n;X{DV1`6ne8YDEeBPn+&eGm&?vrQy@W!}B2MNb6a}H9*0=3dXUsT@m;bo@+ zHC+1{qJ^syJ0Cp&`vb*MO><+F9CNpLW$L~}zqr|wXhf3O^K}m!B*AlE`}roU>o&+t zJ-2<#3=9FQ*7ZCx{@pg^dHi*l5AIjA-XWwWyHj6x9Hgmh@2P}lZrIsxejVU;;{v*# zynS)-Dgf1ZiNbxO*AZI1^bloIy&dmr-?`Ieyy##%%`Lz~;79zu6H!rH6Ro(R9HkoB_D8kq z!7JS_{0cy3RwY=(3S-P6KLyoIC8^$5ey2r$HJ}{+<;PR<4b!UM+L>CmD4m)^53E`r ze$YbET#f0G#1ls(f9pH*=MlpxRLdQQC6W_v>i)AH@tVt;2e$>+?tq} z-%ViW%F&q4meyR*T5QjQAF7+gU}=dp#hBajs%gGny`ESBp0^YA&#F@s0f%<;(xUF7 zfO3)sA7e%(osQ6scEj-#NR*+jZX4w|;Cxw>ME4RWRsT@kaH= zo8jBz8}AHVNPKpys)I^5A{06SQ8CW0e_zNKutl{f76DCyfn?=2*Q5(2 zXioMcu|2Rr<`T)10o4mDxSvPms28-Cbu}`6L1qVy1zX5KiaA=Nw~8%-7_E`s(Z%JN zOvX~@@`5}TfDn7!p_^!!Um&lKSuiT(%W~veeOWr^6nCauYMHEjL_uXFdpwLvZydmx z;o(vhf0f1in))VOp{T%KW^*EWxkPdwMf#gGcZ-MFJu+vRpuzO-TJnzu_t#Ey@-kO| zr(4nOr2Yp#w1S?eYU2mVAgrrKoS77~8wk&PUKyb+X+yGQf)< zj?tr!#fhH9Lmc?G${5}=VDOch4eW*hxD2EvN6S?<+FdjnlC{Q_d^iPcIdD-s81Z5X zSbeTJBOE9t1VA$Fs)2R%;k{!LH5~uPPB@hon)XZXg2jutycNeSLK## z&GtbZeb-vE=oDjWRGS2WvbSwxY(#@$F%rBx3|sOAzC zbqn^Bm80O^bxoN2Fk?qYvq`{uB^;Bg@g>UKUr%;Gg4cE6oJG5;Qi!N&r$xj}|2&=O z&lNGmt$&j2h0bAO2GJ{7tXav5P1#|=;3&R-yK6!oAUgFkUtaro5?KM;DNOe4{7Q8@ zUL=qqa(NRFS9|ZKk@5_y?s&doMA4p7CT-s{<^8ZoP<919B|PQX2o5tLEI+C%U-<5H8}n>>^QYTm0M4J}z0gH+>u zu1ntPXFJBFrgnlX^i3+|09*^rr?$W9qsoDjwuFY3SN3(`+s!9|H+>HJ@Y!1( zW5}omEA#Gb8293`?yP|F+>WfID{=W8@o-3Izedg{6+|-pJ4Nd)JzoSD$+QxpsJikX zky%O|W3qVB3E4z4QAhkJmtbPJ@S5=5iVYdwkn~2xB{vW96xXe$_tpP$?pbAhsyQa8 zfPGV+BL}-f^`%R#er7CMj=a%)luOIJmNx<6^w$l*gIQy%(|_uSS4~+8FeKyBv zp_z)a)n1a`LwQlPXncp7(au>zSJA(gZmyS>*ie%1$y}a0T$%jQ4ZNj~NBa!jCszv@ zNu+p2?_XtPjw;NoS?u&ADsX8o^bAYib*l^%0|NEd#8kHI7w!1P&bA9H?^ZnI+x%v5 zba>BeB_R*AZM?S7Cn0B-D`gGb0E$;UFzY*63ZuY8h*_EqI=nK^b1Y)ejSN{1DmPpS z;Zdld({V}bRHuQ-G;^`|7nqNGcX`ilyyFCRQI;F7ztEKqDphEy!SA@_PY#T6~na-3C;FoYq(?CDbC(^JwC zYF0Vmrqx%t92^rIW2zHll=pi1>q0DM{K`3TK!U~QxWlr`=I1eX89LaeK>lMkK{l#5 zk}|YdW`j-v^V?DD6_9|4i0_n%P00-3hhdY8>$?>d69i54A9?MR{;bWG@folN8 zr0J!d5|@XF?>KJZavZWj)56sjqj(|RvSc}cB(r^H-bF&adz=;7Lbr>mtOrXTmt*vU z?ivq7FxSnpj@FqZwWpt`-XOOdquxOdR2y*1n%#%l%0#P2&}9?W5^~_~RCrG?_v&&U z)Vw7p$C~)o$gKG#b9HXp23*lv9*D>^Ob#lo3l_NP3C$=w_d3juZXopZJz2T(H**w} z_N9SmBZB01U=J9Z0o(ur+q6erLAZ7T`q^wpGeSj;IKD;#Wsh0r$L(qyp|*)K=Aj`B z20Q6%>$wb6c2w7Qd+6qa$(WGl$WD>~A90^M*bTmM40_&QAP%V_#DP8(B;FlvRW@P2 zE`Zz7TyAZ8A}MkA;lT?l>N>K=K8YK}$+`Zoy1id9wB(AB05-07f@+6l5N#&ikHQ-E z^TLvAjpQ!b>;r55fm|#|S;Ow><>V3fR-T2;{gvJ_y@(mFV>ZVcf8aOyGfGYz3?5YB zk-{<+0F2e_N|I$0IAp-EP|E%x`BuV0#MW9FT6w(DDly;xq_uK6U*U6SbH@9Co^ttp zHh_oOYwbq_DS!IA#2~LlqhXSTV-5t>KR=6w{>hd5kzAhSLE>L9U$s^FhiER%3nId6&o!+?<);1lbL4L8JCgITObKao@iXx1Z+F&Ies@8};jN43rN!y4K zO2X(lAi%9fz1jI~ZEzmxnA6Vqu$~bF5$Ve1^~nH0z{Mgcx?)_=L+yCR1U2Y2F}t5W zGNZVC3vaomi}N@D8hqF09x&4tv{9W9K7&bn#Ri*MDyI)-KwZIRTd)|7r?Pp2J=*=v zGd^-#^r~+j-NGSW1;i_Lk3OQll7en=x@5LCB9gi# z#D|-K7ee$0n***GV&~;m3|#JpAWrg%h+U(EO-%yglfrd+);%`eaocpP{obh zk~$CqOd;uy)_BOJc8C=1DENDLf4LZO_$qFiXVV!19|3OxpH@gUQB$>#G+LV3rw>Y| zbRhE;MS-1%p~;kCjv%RrmQ>ZpU~;JsU&Gzvcil-;gP`40~W%Qp=eq zOhPP7qU4ODvg@rujt;Y|Hm>z!DXmkGz3uN%I`dEteIjF-<*vMwK3*sMv%B{40=Hui ziT}GDlPOlWa}`ifi0`Rjof1t~^WC2O{dmffT9r8zd5!eL{wWUB;fG|Ltf5TJUAH(x z5utIQvCcA5B~ig(ghVW_Q3y+U>zOH`8PFKMRcUqnWqw1LMa3$;@h(B|Gpr$_;|g_2 zc6V8aSV>RcqF#N;t+<6rRmC^oS$6<_?oms3aYYfpGgRZW=BoW(w*xQKME$0;cCB;z zSG%$-s;}usH^b0Qy0rbOeA1&~of$bu?M@a)zZSj0e1Tz&GFGmsO4g3F&Ddo2oLZzq zQ%Q55Qz4Y#!JOcO*Qx-qgKCFSU9_lvXLN_qO%JS{@)AL%PtweKm(|pGf+bql#wqM( zB4y18dU8z_QL%~S2pJSHGkm$x$j<&@)9L=j=!{cVnEzBiv*jR0 z*PoQgk}hya$wtdOPQ@y2lY(?nI?m#)$KR*DFn_A%!BQX~|8qM37KE>WDjkr$d=K*5 zb}CVBbZ&n0a}HHlWmn}v(|6%9?Lffsgrz?}8IbhlM9L&%oUF^l3ACGcxR+51iq~Ph z{_3u-xWziZ5nNeKT~$_qz2WgS<Z`|HJ-GcN6rRe|w#jm;NBy-LTB zQQqs)i#{=JdkuJ;Ruf{{cKqqzIQ8vvm+WMlsDXd?a|Dk2L)Dglli$7OX<`UiqfIbpNNftLtbA{2 z)Kt*a@gvEe>1?#RQ2^5EoWQ_jj|0pi-B%yXq*rPN-NT9@v~fCj?8_u5rA;=7qYv*T zNEkO4t{7MPAv)@{4?G`DBBH2@3c#4Gjr~Q&t%zEAbWkPczCEEeFic@WO@5cumu%5G zneACY=1=@zCtjAUJ!%ewd_WFEH`${&4V-X@ zaRpn_DjO^+)R;e|%t&~hTK2=|GMhFB*Ph|y6jn1gGl-e{dLBooOy3bx(z>oRe}>?Z zZ0(g2?kjHS)s!TmnNZw}t~*lnG_Cv*<9ErHdy%mt6Gv5I8?3UtPxjV+GDvk)=4WsX zgOfEoy$j}y+;J7sk0{xv`{Skcg=~%7+I{=U=@f$8zZ>TA8PUk3It(6XcgeRUYnu95 z3>&m09Mh4sp7wKywB_lY&daVC4igIfIhUNxE+?{gON%}p9jUrc_EaxybY=4l%ymh< zgjt3o;a+Op#ua60o^~y3e=Phxlde1GA{5D}I;A;{Wr@B_NeU^k>MRc*)XMwV^lMfI zAR~BJMYT@5Qmkl!LA#HH_sHy%(cEZQ7b{ns7hT-l*s2wCxsj4FrK(;*!Kiu8YLhuH z?_j&}L7Lt$W8sqhWNw2=8VKV_(uOL5Whx2SIAg()s3oJeg7oW!cTu^8j<4O{42*Cx z>V*@MIrE)YliqoAh^zP#qjOrAT#xDTdTvn2jMBD1+lwh8dU=+*FgU?ipw!ihnQ5(* zK7Sam<3N=uO~o-{X4(vJ(BtHPbCPk4mGof+I8qB_($78ww<;~KQKth^M_f#f=N?1X z*!#rMGurn|Xy;A-K2N2Sk><+YXC>p2v{S4)v^oRawSREZdwxgmInXR&IE8g9!L{Vz zUTo~LxWLToJM`R&G)i2l$ma! z${X)v25Yp4ad{f6KdFJ3bD^gzyc((77%};|Jr+)6oFCx6=;eYlJkvFNVg9o29>HK>(@}LU;@% z&j6n=gO+b%jbe9q7Qr2w57dt=4Mw=_V;>e=Z(9dNZhY4$0#3&8hxD$1yhwkj6S-Mq z%p&PE`X>6r<%GKBI#nLU5ac~)_ZXU|Ngp3ZN*)Ia`t&9>0s+|F+^S_t{Y}a4+(^eV zuId}*kI$HW2Z|0f6<;=JZa5n4bbb5`yZ(P%-E~+~;rlRfKok*aB^)pn1d;9@7>Iyu z3Q9^jQet#&LqI^KL%LI>TWWwv*QkvS>4wpR!8^X!uda9h?mB1Z>^$c@aqjzbm(z>P zc3QEKN7||-7pFsa%Z%dl)~!^Aj;G>6?p{RiiM?EojrJhR#L$uoK6N8x2pzb5tt!4= z)8{58yJYsnuvrDI5s{c;Q9q`Dw^8_rWe^`^N(jQuS@VlhblFia)yVwFM9JbF=>QRb zA*F3QvNkEF3*x0EH5*xp|AJJfvBFMAhA$9!tbtsrVa!ywJu*{@#^qka&rxYI`E-4ov8 zp87?|zyv|JNHx&ObmptG`TQV-@O8i}EeE`waX9=4ZnziR*_!8Wd?q-r5^EY2gdJ}z z`75*+^;USWZquYy22rX-5&_95+PvFN%oEwEUHKb-oIK|ZnKPs&NqoU(^s?o6`cYHT z!+&<=PIUs$W~%FI=9|1P-Ja0kX8gTn7{qRYJ;uyrt}mIHg9(%jEdvHnn7kcLNNc8+ z$FiMTY84}N#V+U$Pl0@;V%E)OS}>a!O~G})tVgq%R#}@H%938)$#5|YWiB+4BvJP8 z&pdIGsoeSRVe_(-6wY~DXpk5zB4rup3ZR+SWd|_9rr=AxI@M9ToKiEb zFSr|$?=zSPWe@2zOGm^Xr;Z@i4u{li%K@b5pvjh1WOSbTY{^kn{99 zntx0t@+~yfn9c5VImg7mwv!TaDMOoV6&ALK5jq)LrMh>zo{7R%)4g1G)b2&V7Uus_ zMw(b=C6r;1>ki<`>$Eh~6x`)j6mC}k=H~L)Xm#aIx#BHtms&6@#J8~kQo$A-=TsU1 zfamtXpv5zikO~SanpBa>4EmdQO)!$EeJ+GrHv*q8#Gs;9+MfJMaWN~J$AMGY)_H(l zOfqT?jOq@qZl|jThn+sDylCmx&F6c}O_%tWR8lImXf6Pxf$6&kjbYFSesW|PYTi`yWEM1*l@kkG;ykJ@$&h=W(cv$Uxn=US1!+_kJNR)DX9EmZY8u#-HXY*L~N7 z6*w7B8{$LRth@2tX?B^qOSJ8evL+biT^D{j% zUGS;y8Q1Uqd~(y!&_k^5zZx}F#X=nn+!y+7hnse}47%UR21nP0qv{8E;4yo8tG`Gp z4<~|Yj1e(~heqpV@2zZR<_C+t>3vU^mr5<$)7Z^s*K1iC(>uzd6QIvis33J)e;W?p zDDc9cEhHGKdg88Mj)W^F;7-V(4XZG-e?6rD+(?@`NlNNMUv>HLxw&g^Q)g^-`$R+# zk8$MmgYfV=Gjs1`9E2o7h#_IbF8@L4UGinBG+4ks_r}=CDL!0z>gB8f!q~X%+a0<} zY1-LEyYg?>p7%!x(}CZKR7+;D0RW%E$XfqIx;*4;x6W%GME!k=wr67y?PjbroszE+ zXD1z_WEr=;IcHG;9rP_aCEc<8yQJz^Dzxf<&*MZl1RgW);2q89@Ms{J@s&8p@kvGe zlbRNEHSi>k+_7?lE0k;1bg*0+v-Nvmr#(H#8^;x#5RN+exwF7Qt)B!k9-Oa?ep;*X zB^I}AKtX~U$+?nr&HeK`kamc>fUlV(N}I9{APDQcrlSS zVLIAZ%njT`>H6{QEOBsEngGnEvk1VfXy@XGzQcf&^@9!YsFqo9}fpt}1{MQ#tmfhO!g zvM&9#;XX5sBv4Y%Oyc-@SIDiI-*2Au^b1R$Fbjop-2R%BueVoQT(1miE z<`=@FR3g45_&b53r0iY4|49k7wCPrmw#?pRJNM6E2FScs=T`mvLNl$e3Z>S5Yc?dM zq#;4bZ|G_-%W=wI{_1olPsjLnYX_He{_j}?xZ-?GmGg~VC?FMc@Tl^ zYh2tiT|?2^gAQITQWr^Uz2Bj(dOp#CPOEuUZ7?m_0J9m`%b7&?iFmis)Y=<5<)zcM z=_Q2~Vh4k;pHaLE3Zb*$nCn>|+%rSlokg#@*Ksm$a?oek*&flihC4?zZ3Ap*7&}*l zjL%acum4O0hJ6zr;=7qzVg?<`!}nKetQ&qX z-gG4U?A*mFTjG98X3)z0p}8$9KabqGQp2!Vh|IP23<(VQqo;Shmeb~oS+`)G5OOz! zz!0}VbDITuE{M(GsUd)i7qFd7cHqiONJhI!{tZM%Eh^yLnpMidta`ZbYf}4WL~;B-I;X!EBd0 zrsNj2e?R>`rKm%7j!8&+AoE3(NroA8ttO3JVoW)g&E;-fCz_iabrefi`PK%Nh zIEYq=f^VslsHV)dINmY2d;d;DH#afrAzvbXnCIwDZ2oh%r`gF!6P?ia$eniVb<2By3Nt@i$zNVjpv@42l=^u(zp04D3QB~8W_PzQ}E)(uU34kCH&AB zMSD7yTQLBtPabo}tA73?YxdzFIE5}-%I8M{4=5Sl-xSTaNsk)RP0E>bU_JY1eyw5r zj%cr%M=M=>S+L7&F4+x`nOuo6@DYH%zlr%h*<$-Ugo}%^3ATp$Q(*1M33`H*&}!r>$SY@$*4} zGicJT2?}gTp2QED7Hs_3-&=KFrq6dCE%e=ZMON1Sb$b}FK_}Db0^K1OALqWHZwfV> zSEPI~{#n1I&-Ac2|2$AYrfMKp_H;#M#qQm>d^N-q|BgIi*ODW#?3YV^u?-u!c&LA0 zfn!A8Z2JQq_AIPzx>6nU!@d2W1^*^cF8g^1b(*%H03egvdBE$Z-Pk=JX4jX|@V0a-p%$|im1BC*yr+n2gVhA3jdN$3Oa)N%W7h^lsQlbjNR{BQt180f#N03z(qyC8?xN3;0>0IFG?fG# zTHBd>S;p$x_DJi7jg`z;!WJYZyQeQ+v_sl+byPGp)DUu@21XN)GOB0lC!$&#-EwBssKk0OX@Q! z_Conj|g_4|+I))3+RvN7QuCwE44bbXdx2Ra0 zg;QcG6M~BRD(Y1^~u(F#758;@GYRzs48gM z*O5VQ{H_--8s0CnVXW~Y7%Xg1KLPb7mmL!$+`X+$al_A16VGu`8`USG%J3 zytAbRBZZ3x+TH-f`h0l~uw_yBD0W>p#Gmn5`*dL3b~!8Ay_KUDH$2}P+ulbD*3MZ? zR)wxneId133;gU$LTxdWZvAuyf@eMBEA=|GM@IaEBbzx znhyMdycsbUtlgaC^C-gXqWDv0lM%P{4|_p_m4m0KaB_9G)~SziTu`2$&>}i{xf0s# zdr}L=8?Xo4m(AV!_fk3lDP@drvRZ9{7i5>4@h_z00m1uc%p5KwzjIJ?AtsEI{y@3K zJC*l`GiJh0?#b)pbBkYCZdfk4Os_pj0w3}7ph9x?n0`lfeq>B>)Q%I78>wjv9PtI3 zPeh3x_q1|fpgr+V0@5TKNXkxE7_WK{3`=|;R zZB$ZEUtKQf;N-@vJX)vExv2QbwfdBomNOAA;pT7!3Ln>)9OKAV*Lsi9hTQ25C|=Hj zvy)V=D>%sOL*Cx|@SPY0lfIqm0adZ9nvba!!rcF4->(npEQ3sqYws?c&ez)4TW&mZ zHwZL8a`&`*L?`VN>=G<67gUjD`;+<0ve@Me8?h=EN7*>WO$Ss^^=)9sCHb#=dX4Ra zRJegnVr@V6oyy0ZZ0r4?zD2AuWbAF1?EE9Sw6U3vnN}&LhBx=~YFh(UZJUm&I(#UF zboSq(2h_oejE+xt7?S{pA6|C6)c0u|``E-5`=Q#Uj)dCAh?Rl4Zt!2*Q0TYbVSPXTn=2Myf&f7S%vPN9~f zn6dC=b>hzys_$!!ckCu#v1KFolO6&asY?1I{rhwp$mbd4%Y<8|lTMPso$w0NO7yKo z=~{-oflPCe@_2mcqlGx-(dBOLuhA=8I_oSd3u{~nN`7tq?ds(%m3n1*{X0F@{#hE@ z!AO+-hbIRC*Pg7_oC~YkPYVSL_wH-am;)o6jF7c*}dIbiq zb1Ep*&rXf^svUerQCi)+ZE?-{=WOx>o%1ev^y<6ynD3lECjX$&d}`yirLW3HUWG4p#*m{&7QS>d-f0FJkr_CcJBu-y z;Rc2FqqSCS;F4L(KR`)*;4`k>Z$7)Z!|SbZKHkG#cc()CX9mst0o+$hXI8re_niyUb}lvuFBsk=6=8|sa09-q-wB)ovri%&I|HutBFcY-^2d7RCI6C_q^3-ReoPc%IIyOlX%1XxQ zP{EBX?>d^AF@pZU92qBe^-kmGqR?C=euM;MmoB0Tr60Ezs9}c`yZM|D7=N>G%M*P>i~KBR=gJE_ZG@$?z&`9>SPN9yu;6wAHKX;o)fUVa@9)v6PQ!THb3pA6Eopo+7@_9^dw zjA^>Xen=V-57S;u+HBLNh|s}6@?cnPRE$i3kj%$$t?gg1ei^nsuzZ?y+UUK zAp0$5jQj-Rxk<%Mwqd@p&C!n)A1Dr(W~lH#Yq~zoAHANpeB=@{{iSO$`MqUR96+%B zkuPokM1#jKD#*9DdN0bmOqB$VpiVE0>_p<(JicZ0-uUUJq&81_&X}Bxqi={`?#z(j z3CW?Z3~%dLZeR9eY=0WOe|Kg%S!CBuij?6%GU0$UyjR_IcXzKHs}tL>7X_PIO$p7H zjEc#J75^&{n#f`{)9QGig1&dQYBoXB^kK;vGumRnUcOqRMbUlW|6zNW_Q2ua;|M4d z*c!I^o!PLaQV85Yq=O%{&W!lJe6X zy!0oGyWeSUdYm*%j?J$iH5GrTuYZBHv-cAR9+6$Y)lXE{BSICFAl!tH>e9|^5|w5n z0|!~}iZ8N_Cwxc64g4sQwRpCM?VQke%ReQj{{cjIqq1co)gU#Eh1Eb4T z=K3Dwu5TxO$Y4v4vqE(jwxlyjvhy>=Jtr199Vbzqnn?HVd`%rmEN0y-}tNnX%=$g zV}mY`NS&}k#MhvUfM_4g*kjA2s*P5jPW@-xo)Y1!s*8OR?&#-Z(w_ zIHU+&dJ_cNO_*QRaw!Iu6r2sDq~RC0Lo5}WE*1PM@E67tjRfnG^_?AT$_&&?vk5Z@ z9NT`+*4{2i1S;6q2^6|pF87X$-RnkWPPO3v)NE<7)j^*rTcy=*_lQPGva0AL#n?uL z=E1Q9@yNJ<{D45;xfi0@!oBjod|CHC7%r(Yytp-a1K)1n88P?u#-_b<$NeOZd?#Wdkbr zan$wmEeNkp{{6})I<4$ANM_2CW5}0WIwn7l|8P;&@hc{+{qlx)&U!TV+Y_1^NwMt- z2Vg>GIAz2a$#ZFPHjbvPe*z+gqDQWoa>*dsy zjxsS?YyhrGzwJLTZqXB1%(W9(@wi-;&&1~zW`C7gz)W9OExQnx~5b)%~JkZ zF+WS~vR-p6huXo+0v7$?6LbOjI~_{2iL3w60#g!;MN=J zrSSs-Rs{@i4wt5A=Hs=FlQuI{G>_|l4cP<8CZCz)T>H$e!)?ypQSDVv`y_l`I94jZ z;C5DJf^4)$Us75%3tF}dqL<=Bw-Y4ok$V{(<>bt{pOvd6*-jg}7fy3~V`N~v6)NVc z2@`W|UrgXqH@mm4xvFKP2)mItlH|%>&4@oR*YsRMfrhj}d5v4&K|dF_AHsbTl* zD4ySt6~g3;714IX?WT874lEtZ{ztakc7ie=q2g7&)>2sOE$pFFNvdSK@H-}yga+R2>aM&_SwFa^^6mxH1M3;RJY_MNsH|87*Vhx+*hS8G?GKHfOBq<+Q{d@Tea3Tc z(HEo#*wJstBlXWD3=xX3lp=JWL0+uoCX&gBDBaBn!-%j*ak=V2z7%cWLywWIB_zYF z>S=!5hP(?@ytP4Zpke}mp+z84f^HBRTqJgsXT#Sv%567Gh4$w}zFBA@r`zSlB@o*i zUWbTo=M-(GL{)4`p&adwp7^8yc1KU%uTS1(AN^%EdxJ{>I0Uh=(I3CMDETuMIG#bw z7XKM2o@Pr87bh@qVJ-Bu@o;$kIed74n17n4V_E%Ns!$G|%H3mO)!N~z*{EkO5#BdM zvr)Xh&!6&jYOquuSh>+4EkG9cgO)@vGdfh|zkHZY0di?AFmp(XFw2oy;(|oB-N66h zV*W?4o4fxIP4=0>al|0)KO{{vhf@0Wqod~g67>H=)Lc1uR~t+|olEzj@Q@p`E8jlw z$#shEz2|vl>#&gP&p}0`B?YN{{l6+z0RMMyO3&}#B!W8g&Z`ryc&rYrjb||715Ci$ zo!U3xz5}}HS_2*33X}+9?az<={9pfuhfa5S=j$e=Qewn^9kW9^T`3L5#uVX`{y9FY zV&61+^>wkcuss*GZ{L0N$DX98HU^bMT1VhSe<`rJ|Gx0-#|AMwsbb>h%`-34iIo&5 z7qHo||AKys9z+}tNE}O#9vdvcKYz00q#JesDS4Hz7FUnqGU-z~Vb)EgEdCMAFN+x; z*<4S0W-{(!As_Ti&>+n8>};*pwDw+>%*bJgdU3%6^XPx+oNqR)i;@lrPdk1d`oJEb zUEE=zI?-;jWL2RrSqc5FUR&CAtmGyNe_c-+SdkQg)ru11r?al&MR9pI4KU~_H^b?9o+{O| z-b1{sH#X`viuwHe(3O?0}+J3>4aENs;xZi=HHw;Z|KWqoU_!m5h z!p4u<@_SM2w!CA4H}o?%Qr*4#**EE}8`A=mD!MA@J}nglHvc-6ClJBGwQ3c8u8OpJ1T1W8^@bGuG7@G zlG_JbDg9>&`M==Kahe-ASISJXb;xe;|bHMCN+@!{wL z*0lyLQ;FDIJy{4B!kZHX+^rfDSL{+U? zzj1VuoU9bJS>7Z0cEKddo}A+HGQqeIf>s_1&6JZJL9JkDrn8co3m zmS2k5{_<4#2IflJYauT^^wPSiyU=(}N8+-(G+|S+QCxdbw=GDpR44-8ssB|IjWzbN zc5kKZaOX*N8A1`&uL_yRbiWHLD@oFQQTr)7Q&$AtRw!0dCC;zmYp$5#>dbejn7Ny_Vdd`FBpj-S0as&T)9RqDn1rL#i& zZiQ+vF1;e2H;5tps8Whf_feAt>^>%Y%I4%@D`Ss+Me+ zS}3Mf3y2%~7LW|DaUt<_ksV=>+Y2*udtG^i~_QABjn8+O(ui zhg@xvD|KE`+lX&|Tf23~-qj}@H%W&27VrpsSx2$4)MgpM;xe@NoM}?Fuc(RQRpLG9 zq2#)t(MY>v+I=ODKY7{?OeynbW)2EPu!Sd9`v)lNGYwx9h2mb0n0> zD7u(}db008D!$Kv=ODUxA1CwJ#bM&)AQFX}#k&z+-MlqjXE&3s*ib?{Pq%N1Or3N+ zr0>)fd^RLvR7Ep-tWv47Vd3gS-pAo{Q?gdDK#u6o2^MG-r35x3l}t{!%@hx;XF!E& zvpULQo{k1w)0t-|q7rFZV|8j-h3(d%@HG!1xJ{1P6W%L%|G<}8=`r=e?zY=B^^|ss z7;u#k!p=&44BG4q4gm9fjAfT>&`4{ z8>l`v`K)w!cW&KHX@rB>w|OtvWXR*HW3Tsb-aR3J1yBaomNT*v?59Bk&`ykax8^>=vdFQ9dgyBnASDOC@{ZeXDDgQW8MW^NGX3{cajGe z!|o@M_F-Gp|@y1=WK&ZPn6si`GPzaYi%wu4W5Dgag#FR46X#&g`3J#COr_SOO!lr zMc!W9jg+#uHXP_=-B}<4Y%)p7tCLdYe=+AS>YhaZ}ooSg!33` z`9#4ZMGh4pfFSfJ*4i#Tm>Qf?DZ{O0)n06^GM-(xj~;2@R8NuGA!JuCV$GrFor?Li z&;x;;xd+1PL1E7pwVhqwy%pV^3ii^0mKorpO>Z|%NAW=GVm5x-Pbcby4`?oqcZ3$c zCK2uiX|z1A2=9Q-8%`GbS2>V+1+vz7**d3{r&zg{?B9ObiX4fbhVSOaS zbs|Q$B8lRl&kQo3vyEel*tG*RdWbHzx4%hPQr(Qa=qSRVlE1|^YV0}xC(k_7NKl~9 z;*orj(c&uw;zUqrslcVkJcS$E;4R8_d7RnINM$XA z)uk~l?b;u2-EA53R$1skh1HJ@@06kC!aCO!Ppg%Te9yXY{?>E7f#?af3wii+ zxq>2Xq%eKwhvfQWXSkFzM`Ln*<0FRc>ZybRs8hD>H;2LxTF~Ex$r0W!(tq3}d6{2= z!mn=^!4MlS={Q+#J)FAIlKSQMd(Emg*lsr8O(jaJTl8V{mIs9u{p;IKarM0|=H6Va zlYAu&<482C@mX*Y*&mqL}D1^Ju=ZL2pOsMpI)uYm!l^X4;+ z|L&Hog5~%AZeJm1^~9A0&xPz6^EgB83C}GXV}l7(xZ{SZ=24~OVwY*UjaFP4AN{pz%olQ4}1K9=X2Ywpv~4>!MkZ+hOZgP+ipf*f z;UJn`E_rE4eBkSohASp;W;amE#Rs3~GhygH?b^0Iy8=31cP;sySZs+^PtHa?R zTP-LPgP~jLVoUSaoFT@ewZy^QjKbmrTHeDL)s~$lAs^4d@PX6hf$_&BjzWLsc6mfm zJBXbPy=vfLo|%Gq_t3!Ao>OpsivgyS#>>~G!2*O=#$066^>aGAm^H4N1C{7|JS@?S>#QWTaV9`UQ zf-bUGmk2vPm!T-OLEXdj>B>-MpD!iWp&gKfY}*Oxcl;`_eTGQGC81q`{%u6Em+cS@ z0=;75{nME7PzK?e5<3Kp*NYo-MSj8=`p2RE^o+4aqhBvHC6w0**c{pz?I)I_MsTIA zc=_A#UmKjj8}Mx9o|*)d_(Wi{(bLaG-;GyHT12&$e<~y#Ze0?O_8M5d3dUDGw63UvmOJn@`3x$^2lCGnx!T|5488wAOx zEhuOiioy4()E@Z;RLz(gpS8oz+o>n_qT(8|(G`qD`Hnbm9P0qN70vd-78_6dU^8>m z6hbPF>&!#XE>~3)z5cOHm^hu>(Ed-u#K11pz@_^UXCPg@c3)e30S4kiE&HwiO-@6(yYt-FDJ)avm$fn zS=fCnp4u|QcibI;buU5?rMcFZauAO{jF{rz|Gp0~C0#Y-;T(QZbK4LI@La&$=Rn2h z-Rc@ZVPpXG`1g_B?2(=M=2@s`*V&c!DX|b~mh7hrj5qS`g>qCJ{V+cL0q_(;?gdLZ zP^0n9%PG@x)i<;OaOZ#Jm)8EfUJ zKDzP%0hvk8#OB`A@Dxh{?p8Q!3skd~`hjo$H$h84)84J468xQRX2~xfpA`;Yk=_!e08_A)>SK2I21kgHMO zwc($bBPhqg_K`NEwd7*G#J)4aag~?~rFWTQ5^F<@6+nnv&h27mq#{3gU1&6!3N1-g zW1>14ys)Nf98EH^=%#c&vwUOhnlamtY9fq;%reVEYKXao#}VE%b*0rCU4AU99Btwk zaD525bgEfwH5*znzF0C2`>ECQZb76$MXDEY3LY^%X0XvNzTSMwI#&D;Nmo9@18CX4o zyhFrmSo^Bw;OG+Z{e?O*wWHW-QT>_U0g~CnZ;$ zo8fYW!n@_FDz70U@8(aBfWwLx5RAuIViH{HETg88uw|p8aP)RV*1cr1Xc|6Xzx4;( zcnYCL%13`ro;cu2<5oOs75HHEX6;~-FkTCN2)jGfe!P{IiuZdKc()4SjLDxZQ)mA z)Au!p5>7f4Q6bLj#5JBrHiWL^stan2z7O6SO1cW!zl%@1JWUeDX`pSu%j97VJ;x!t`Zx+VyKtw51-*taVmLvhx`i>To+`sTkBLI#i z_JgMuv9#dd_gPriP+YU;cjm--LauoccuE`@>BV1EHFd_z~9U?E(%Ir&{aIH9T3<85w%YP zF-+vQ>2R^(G<&{=wH{7dSf-uTu5fIz$q z;g{j=u#fM|$(w8g&!XSth@U%20MDX(mU5KVvkO1gI2y#&mZV92#V5%&p-QZ$@-%2b z?3Vjn&id)+rvpAnPmYUxPt37FPB5Fx*#hy|g+(nHpv?V8#V-IadVKM=>TBt^`%t8re&PGP*XVv5%(LhE-*uCw87sQygzz;y-DirHIlN)oIf%Ra0qiLLIe@7lxXpxmxjo zvDIk8om~wBiCD70^%Jw#!^|6^4Iu?_)K+NqmPhhDhI@XK#O4N^rP8j$@uuN;bBeG{ z7l#3M6A#17yjuY?Z$K`o8vVhVUZ~MVExi1u%0$1b?zX4?I13?je+WrM(jQ*9XY`Up z7%IlCM=aijt&GOmB=*fLu%h5{`-wvOc`;be?G9WAF7R-D1>I_#4mmSQ+@;)-Yxq6> zy(oWjZ0qcxEX)usBYE+aWDcrNf<}F0wI=47zrB@02rxbgsDhk)fsE<0#E~`|*>HM{ z5}yNW>P+?k_m|H}T=N)K_-Wh)RJLVS*u=t&X5^!lca`qvU3PzT3|YzvJM)hnau2hS zc@1_gV)z3YY*prOMoCHgb%l)3eKFgL!iv^cgywc|@ zmlf{7-D)mnLbHmK6DiR$o0147b=KDIl?kTD6vp4ZK0Iqla+h(r(55+r{u(zX*%@3) zrbhEbeQNq_2i9~fH(4^Zp=5~XD+FeCjWne-OoHpkvq zl@ZycyT21tNYWW;3O?@@DEO9f@F#;giPRRUL#YClgRclkVf87kKKdR9=Fc?HbAy~D z<Ml)f6$8a}#5wgZdog)@gJ9L?Qa^?;lw@m6Q{r?H%}nkiEA3&p6wg zzJhptBXwvu`>2G=MlkS|zBh~`R_C=%I*^tO(`E}&*0E>}xS(uKxNq#q-oy>)I1|r1 zol4qRr1?@q#<%>k-kKK(TkA_E58j;1n|orIIZghjyem_%vrct3ccA4#sq5w0Y|}-h zogvNRB-})7IC1^<45={+Nm8X1OcILZt9(3p!EjEyT}hl7sryXS8Eb=AtDMMHz&6~? zl;HLu-8dDJPXqe5ub=@yWLDR_C zs6i&0aeH2wtZxE`@hi}pzi;m6y#7l-lyQ0x)6-}(gF^>uC}VM5;or59-Cq8vRnXYaW9HHlTKM)u?{RoaA=3Iy%f}ng_qy`VDQ=Bwu80| zq`MF^4Dq@Pwo!J;aB(Rc?3FlrV3=!f^>7lyu=pMm$)1RxOisrz4& z01h79;l2p3;dZf43at1p)F2S7vz56su{bG-+FLOYS$d~Oz=V@^8MK9GhHOZUbU)N< zTF8Y{PGQ77t!p0RzSlP_>ovg^&(xcx{c)e{2{TDf>nF7okCen>7;hZb%cnj!a8Nh6 z4tX+r!W+Dy+y-|pX|OPAQeQp;TSGF;;ZIZu%<#IK^EYVM{GV)B8Q#wS?rXGv&1|M7 zb5&ec@Gz#sm=0Nc%>Kyr^F@Ms+k^aBz!TrF#A5sP<_sDR|9^MhgMM6he_XOp(>4f% z2h&y_$`h*`92YMIGuF+itoFRG6^RJH0o4qt)qkPCe)>{V=Eqyt&#%o917*_L&nd(B zyRk{QAfv}TEbGThQEp=elBD zrzCkl-b8%a_XNV}V^=#xp(4^9#IVVRX7GXs4SC$9$izJ{-}4LG(-4_m5V7p;TEipW zdsOL!TxQss(OfON1me_?gi{O<&$-dUtp8!&Y$rym=12I!mrAEP+r3c8NwB}>X$RQP zSO#BXJHn4Dp)z)iT|b_~A!HYbYn3ROTX35p_W-r#pUC>qUO;O!IdnmXtvQ1{Ggazjgv_#SvH8Q;&^F_|luFgd3bQ ztU|M@2aVbb*U%xeC%RxC+NG_4jAZ>1&z3Y9^_A0YAo7RFoU-|PE9pfx^?T`vGyShrO?k!mJe$b;HfL<|9)OpGBzKTTF1RfS(yJSY$DUNIdS8E>39uWiZ=&-$iu z0r#e(VcTLBjC*6XSCy{YDS%%3qMnkx6i=+b)m^}-{%JsOo+Rf_JgK>5@e>uQ@@CSU z7rgjzvJY6t7{B9moj{Uw__epYw{Ouz&))8l`E1!9o5jc(*V?m3*ID`%KOhQ!m+VN~ z?8|JAfi&zU)qOqrp0z(2$zcW|fP`Bd#|~MXmAdkMh6inE9gypX(K0Li^=Dbvqble+ zvipt)ov6y%^ry?kKrzz|6A&8VJQbz<92Eljr`>Ox&gaGam!!`J@+$A@?>B6}2WmMTFZ*p=>J;&KdRn4s;Mq$_@yJ#d_62%$_-CX7Aa*k&_facYS_p@#18ptKK%_+H2Q%POf&EdEJZQ)cAyG8+SJ+3WmIR zN#&@Rc{R+dKCnOW?lY=L#-+AEhC3>Y$`>!f-m=a=jCVfJU+M|p!~WE%ivbj@(zf)E zfY)T)lg!$`{6ao2c5h7&KzD*yIhA`t9iMfrYKKVHMfy;L!#IN}c{?Qp!3BV%ad8o* z%R8Sg_wQJmvv{4+@qv{h)qs;GrFbCK-XA$g!f7XkGuMxv( zmknHokEK3=my&`EHsG_W7nwH*+9fV8sE-PnLqdVKsj{iG@Fp2=Qe17Q*`KfBvS&#MTECwkP!Ect_lfj|kQ*)V#c7_SYZkaqEDGtbxjm-(vJOTF z3z}>y-tZ66b$JpLeX-a?MqqK#m9Xzt9I&NP4v3EZrk zrJ(e&#XQ%Co@6o9no~M~?p2qP(HY#_H*cHmYR?t=ft|ZTR4DyaLGJW4^Pzq>(uM-v zk+n@U6!skrg!T#MYZ;+c+By?>=6XxHG9mKRfXHCqfaYd(z6c;wp=-AaH$wO8eN~l9 zx?3mJv2VfUq}tQ?r+nYUWw(KUsbtWbpFM{A{L4U3ynf_v1tnHlR{->($L_CjsWYBV z7cp_ZX};TctS2%}PWcbrj(p`0tT!(t? z-}m|m^2*lf5E7Ft+$>DbY;i1T!T-Z4_{%%Fy%toqYsM#zXuzZB$jjhp%Kph;PdHN| zsRScDn};7%aD4C?Yp6)d$XEq7DD?yifSNd0YK&^Ax+$?-cB87U@+}f!FhXuI_>yzp zvxf7V>n&nkdmMbTL(b}-N&Qw@QGVQ^oYOWZZrl{E@#o{&S} zuP=foyK0t>X{Z#DYb+BM*U0-AI-2DMwU9JTlDRt|4uCp&Gc0tM?3g*H-OS?K?&rXO zJoZ$55wF1c>;3o}=27V^wLzuyRA10u7w?-oqKoxF8sv6r?5+Cv{&kr?6QWgQXJJLB z3eq!7@te_H%aL>KgL=H1Yg_?YwhT z;HY~I&E%F#Pn1mjU9tj-?Z*?!$7oG^xNGG6j^7uqKU7tBc*vE|`=fNY>fm$KQ}f}O zv1do%Kg|QCQ2j#0H;M0BhjQ&i6KF}{=m4#0?@PBkO+7w3Q!DV5in2AfeK%t*zC!zJc!^ zsNT%*$bs-@LM6OJmDM5c5P{#WQFCh=H}2neXe!SsWrMdnq<%R!wkV}kX!?*DUG`Wlx45w;mlVzgYlO^dA?Xp@4~ptJtc<0ZIH7opRp=Uhb3-K(L7_{Zc7*o5J$clVlX(v| zAFDU7LT;wzwP(h2+(gTS{ae^tdy}l9E3udMBqVT)EG41o$8*`taryV)>|sZB?`}08 z2a$*7YXI+ZAAMU^zfF;S`2Q_kfbMGzak!m5CR zVCz4bL2k`eGMBHZQl&xWeJ!Z%KlFEBBzpeGLFwuw_a49n;n29-zF+6fZ0zJXRE6f8 zg?{F(C@ME*4#W>Wwvp|$XIJOYFuKuIYl8Pr?5)rK*h{eHy;o%^{p( zu`^-V8Y>T0ivBM*fd&1Ou1Sm|`RQx<+?2iUNss$!OR#Yh73zUI0O?=`|3xHyjo4D$ z$GMh>#@0@YjiskhmVzf{$CkyeEgYOGK97v?octRnl6U1CkxGId@I@7#YyF$8|I*`K zb-*~C;DBm`fnYz7L5@1)*3kU&*?@$k4f4rN{|r{cGCM`J&OwJ=O3(?t&ONahTJ!yppIp>v}-iv_5H&YF#3_qhG5L;8qzgw`5>hO@R*AN>t`=h<2(cG zYcjQ+b}QSZ=->KHpJm6DPL{@hEGs~%$+FgN*_^DuN2;+#5vJTkyJ>l>l0)5Bm`b6@ z;3r77K20URF66GjpKpHr$thTN-KBPwA%&6;Hx|CaTq`NCA7b$~h%!3HJ?XGV#3GFj zx)$LnM?~I~f7wK0c`%2R?DtYWS4qsgNA{pIe*U?;SObpTRv5?I!77nMqK!4Ic5>Jb z2x*eP%Fw3Z12@jw4`)}R@B-G$oCamiuZOkc)zKgZ0ijPhKAdURpY0gAy-1py5R7`pqgG%lnK<}~ghZOM+OMivdfT5jBMzp+2! z1^GVHCTF7O|H~Oqce%gtCn&}$`M>#*U!H?0l9togR;&U0GX?72y$=TkvDwl+sp=2I zQ{`NHof=pWn9uN`;i&OrQI+sSjt$5iaY~w-_aggP2ZCtlUwR&QY=~$G4B{STq?gJ- zAyT)Ub04PK{dddCMKXix?%F+NrVo!wm?hotlqHdZClP0nl_*VP@5JGF&Iq1xvfO#b z+n})pFwbF_>BdIM&f7e#w5VK?rjZ8_pRO*%?2?DjP&`71%{5o|icBXv_%!Ies1DD1 zDj2RjX8Y9xbg-<|%KLY$y}L`Euy~!YQFy>LL0T5K3{s{O&ef^-a5$nFXj&pB2!XLd zqO>+@gkx;vsC9#UD25DLuQ~tRPAgJl&OI( zRF{~5Uy=w76N+k_skTquQ{S$od?;&l;|7GWI+e zH5ez;b^K)gXyo}`x?XRvN5{BnSt90Z{|`H+Z+dkSDds@by7;CPx41H~Nihkc{0+Q! z^NDjsyvYnUt32WJ2^J$X2-;p?(L*F@#2 z(o~EA?IW;~?I~MT`ZM;R#inZve$+Cm_h4BNjO8O~iqsaCN0FwWg#R!^`H(Nj7bO%G zH*!VK*GtfDHd}@K-Yt9|c^g9P0OC!ZD%C7cQ}B6jl$Lq-C#bzzv|bGM)#*hr(1rnd z!UKQ&+Tj32S~s&>KClpc)?TZx`!#{3bG<%aZU_3eclu+&YN-!)l)EpD&kAbGGM~vl zaAWo(+u14^XFt?w;M$0N2R{>e1Fpgzs(jr^oh!7PPHPTD+1`GH;dli4Pkm*doR`YE zt0drx#Ku?uq648-QbFY@*XoTS+{Gr`8-^;&gfTo^qCjrZeK=aUGbj6i1l=?T z^9#&l7b*s;{D!s%?WQLlVFUe(iLw5J4oYA? zi&$6bs^wV!EazI|%^*oL_DC%Xl|gFz?i!FQKbhUd1V^i%An@iVNpbhLj;6RUo+Wi# z6Xn6gn1xfDOqrlGm%2o4SO4{Au&-c;yEE@#*XCp6(tsU3jLn>arfaV)dZd}(w=i9D zux!pw$o&(DRC-LK6w@1QeSZ-Z_daCJsK9vk`hx*}PGyAdrGBJ zz78$&+t%Yh@4@=VhdW3%mn3QkpD>Fp8p+mLSNBv~7V)jaep2*}VFK=0mi$wqQ}S5r znf4u5`|>1Feuw;}qDF9DgE(v%5*kSkBLBR z;^i2ALZq_E=nVdlSKWr7^L&x5H65*Pyh_Jjvb5Lw2r{yh11Jz+Mw3vt% z6L!5-{@YrR9i`gd#Oh+v@NVaRL>uHB(7--YI{%e>i4An~1IrU~HpTUm`Qq9hKHO$g zocxJl;Ia%ooH%S=2{!}D+M;U@ridUv1#c}EL*Nte_5Mr%6Tevi%hp5^3Opd_kGV&gnwv zKgy|x{{s-h6~YJ|Kvxoa-rjM!1^M=qW;ckCXJw#Tp(yc7~m}`S!$Mb9aR-_WvQo!$dLW(Ql?}7S>_|7Xx#4BY(KKI>kza~RQ0Yc?5XqN zq@6q3f3Elpl^H0-w7KZm&B=96=RA3h4>m88Yo?+oPpVp`;s2t-In0V$l)>^<0F|{)Bj)P0Usx@wB5mP-VB^w#? z?TX6sCxlbuiK4cLl zZMD>4t@p<@+c6j4nH)u*BX~cGZ*%M=@e>a;H?;~EUrj^CC8sjApK@Z8sg*ML^k+F6 z^0EgYM@E}yLP4uiW6#P$vwU3gv?nXcDL07{J1P{ytM-=dY}B+}Bv+{Z+xjROq>dN6 z$*Ln5N`RG)P5>ohB^Sv+yI87_dFIuSIe+izVp&=$ z^HjN;fHD?((KjmikhqPhhb!OOhR$Avvoyb@4JoB#a+&6R0kNOtmW6<7GotyznWu?# zo>m>nl*^$5Yb+1`>tWw#9r?otr`Y=Kmwih|EW?bmByy$Wx5v(P0+?U$6P3;y?Y@f* zXb#}~#{da!rxF|#=QeIXwsH!>=P3aQ%u8+45ph_|h3|gk-7s}@ifpYezPM=VL&*GR zl~g{~w1r?^wSVh8b4&{rF3=!JtTsjPZ>o(*`dLZQ`Z~H1OGC+b@3bZR>K9dLFr{~B z+oM|FcBCVZ=cYKl@V~k^V)do&%-`kEl%Ts!&jw1qJFhPIl{NP_RpvUapddWI;%4(r5i2K77-tPGsg}&Y@eaI^E zeeTZcxx3XX0JRdcy_@r_S@RiEcXyi&4-rk5_39>E6L?*BsU^-R0B~-o3s4ycgOg`D z2I|BO?S6~Qic}1ivs96lv|*u&U0uF;;&FFm{jF3;V2HtSq<}}L#Gc9w8~z2SdmM`( zVtlgaRe0--dJ2u@2qqTMJD~yY^fTn~gSzLxOsJQZ964gBz7%D18mK-lQ@u!7V4y@$ zuRW}mMkx-f{w!dt!-qS4vTRl-ZH#y2z`(M^Hl^Yn>juz5218b2L;EDb1Z- z)OpXhE2Y+VI*7fyX}T);Z4c-((Q12uYbA`gF-7a7PXN)PdQUb4GNvGy-Fa_q>R$>~ zkFjw`jMELn7F)&4)}8~ZvULl~f9>y^{mI6-xx~qE6hbNp%O(A7|F3DqZg+cES>h8k zoMWdtz}v1Keaq6~>$F_DDq2CWmDdeCc4^Why#Nqg4m2>Kn(pgxXNB)_#MyOYilwhr zMWAC9_7n0=nuccepIjvNz-jPsddLnSVJO86hU~JLK#W;sHh0}! zT=pD=w)vs9s?UEE-m_k5Y6iXd;IjJSgEwkR zeJ4rt#guTI`=HHs#GOiWPayY2A!^YdmlQKFcKWVoo88(~bB6j`_nN?&@N*}W9J`j! ztN6u`bnHRrIYF$g4eokY)l#|maEF~A^jyPEjHcz+4BrOd6XJzv&I4Q@vMGvP#6#H0g!7sRm#-gh2IZX?kZLOh zoTTFqwNxx$Hd3t|cDKCV%u0QiwUZPy@Jst-tUG{?Pb{rF@7wv!8~GljRqhfy3G&9L zobD}V6c1#I>W3=K8TP4R*0Rnofp8RxE%R-902%+Mng`~Q`(OShIGle`WDB=NA^#c0 zO9&Px5`@m7H^!$sMf&zbqN^OYBVPV7hX&^1K1y^syubZ9QSqi#6T`4 zDv`iKZ1;K*0++)usbTmFG}(LP@tk>+f)rAMI`MjUd^zb9&tlEG0 zUlu_2$xjVv+Io+QE6w4$kx$!tk#8;s@t~IDpwiH_cJ+0 zj!n!vTsa6DJbvatzVjS!i=xtC_sylcbJSh%xzgV?gl#D&fX-3W`vp+#J?DCRRRV6& zaMY#~PM0Hw{%QLqT+Hlmjqcy&_dUg$N}v%z)XlF2o7c}(owROvqU1DA`MFrNXm7drggrxocwJr{auCxz#$;3Lpzy6v9L1%9W?V{EKyXF*}+MoO+D$B4(H z040;uUMACS%^_iP{s)F0>XVO(`^U=S)kOZrG8iWh;%job7uv?=)?-BvNrYG%$ zkaFaWlK!DH^b|uSvQ}+0%{U2kSf>l=j8Ks8mYr&b0#-f;UCN32P z>rnUfXd2kb50~o~A69^#s-8VM#7W=ttg@BHd1IyOy3`sMQ?;eiKa4LNAoJfuA>gw0QC`MSfDlA!5cYbb=dhdX53aB z;0oR7Gul{wAFjonOAo(3;H_}58kl%b;_1RD_evSZ$hc9tLs2go_uE_@AHiYdLo$dk zDBF^DP?67%h*MT99WLQ*Mn&opw;tWThC6t>9CIcJUf1-vm=FVqNc0xedapbRPHoC_ zBdT|R79yd7e#SH3eOHRdg%B~FPJTlXsxNh&IrS97`VBW?PG4o6y|P*7G(O*%_oK&?Lm=akkS2v0}3<>4Os}0wDle<=aT6C#cd2 zmd*R@Gs|j&?Fh36>Q82#0t4p<)m75Jyfs0^Syv|(k}rKmt$7gt)htp^dXlPTGyhCJ zQ`nuS;v+^;LKWH;|2aZflO>?(f-x(W-3Bn)^tgxozh8DRUN%IB#UOa`DfRP}yJj?N zm?wg9l;>`k-u~%dNtJ4iNEfNgeaXEqjjnE8c@V+Uede{KVRx$@2X}mS$xlwyI<2Ct zxXagj*1@j3yz@<@$!YJcFH2^0StADSM9){^>-+h7&mcCSh0xrf$jorXrO@5`C9#_$ z5=@U%;gHd0>qd1ib=kopDCqtvdO*va$83#q_IR!G&K?w%s^o@l(&R5wdqdF{o9~}N ziJ%vED>%Q|R4iCG5W$q&vG!@7&b)|uEbx4YwnQ@4;Sufpkk|e%R&%LDA^5Fv{P%6D z2D3Q-y{D)`Gve$}J_DY6xV{W~E8KZwP;x;Zy3kTEU&JbL_OrF;j-pq~eeX(Z?TVd) z7jkF|W;I67avrpa(<*)!Dd0!E*`wdu~z;r3ARPAz!gQq6z zp?)<{l#Mn3DX+)02#fO(-b=0DAl5VcabH!ND4j7mpY;@fDc)HdN4Jm|um#Uh_E<=# z5uTQv`$^yui`m3bteX(ac%C>evI(tQ&=GuR_ zSdT+F#OfrVG;CG*qZIStCM*nSqdNB;Ma)W-oa1{`Th=`Bg7`sz6u?)*UKL|;_BG5i z`vM4^qs2Stde+6A{RddDqmoO)Li8Mwvejj7QE*$?>v+dG@C@%(aw*W7>ZCUt_Mq`5 zh|k#t$$HKCP47?G(a{qsG7n*fXS4?3+-a;Qqd7)1z9Nww@xyii@}p7PByh%$W=6ix z25)51G(iz7m9!*dIVk>4&;QxvvI$@}fwb1}5{kxR5_8?jDId{5&5IFX<_bfi&p1(4 z4F0%szw1%oY#G9;;=A-4JTfGl2QB7*KK&!`)vZC0+%jYG2SWS=5)z$JI6(-FT(gTGOe`2W9@ZVp95vk z=9pHkKCA@IoTgI@?z`YDRw@dd&1z^()R{t%;s1E|yCD=ahkJF$1IAu>pvPROX=%e?!f8D z0@O&KZU9+iOhdOMbUPVWhGmlFf2?x$AVzd7O*QTB`@JP9QqGfAc%YbaT%y!p5;FIi zM_SCcFtfv*2d=h`y91`{q2_ik1?qcc6rH<8wCJj_+2rX;T?tg-KO z_}IW#|KL`jHwQo{2?)-Ot_a|%gw;5whh{d=n4|gorJn`(N%T5%ef9KB1^5mHoK9CM zv2yVId( z?EV;Cag5a+%rwPUd$`RFk8Bpk!*#f7t8nO}>FNZjjYw=k^>s|`XdM2|vx`2hGhyFj=#{$9S7L*9dOtWuFZB&CRd@k0K$E@? zCY-pFD!z=c+&{unr0cZKZc=D8g@^{0gEpsFl)fY(;1=mBz)og!0(Zs_eObDZJjT@SAoAq5PER+jycud&nbdJ&$inZ*_YwkQ>7;% zcWS$?X+nFHPiV&K-z&L=t?giBwv`K>PfX7nuEHvPJ>#*BxMvO5;qHfl*J030B=Ov% z&@@KS1GAsSy>i=sT_X+taSA~iZP~V5YmboON@v-cT#QygvED(PvL|;Y2v^iHo)oHV zZ!gksS(DYJn>@nV8dq;^Jq0AX!4zP%^H&;B5$DE;GWG>#S7WvCmG)xW(!7__mnh}Bj@r`e;9o`FzGE*F;R7v{#xgF|H2-_za0rxOeHZ9Rl^h8s>htk zgL7}3h!@VrPMqR39ahG~?>#vI4GN7pC1(rcn3@6fLve7-rhNm_?nUE1h)I|X(; z4*(=CICKOU)y@M%q!SYf#V{@yaRd?LoF8EBbbZc9cP^e;@9bZ$kg4AKM#b?faf6eT zCe=;YP`fyqhk9npF_XL848K%3KP}MT;Vl)@%3%O8z)Wxyi^y)qo8j^Hgl!O4_KthCX73x5m0g9j&JG0mij{_0Of zlqri)T3hp#2bi@fWV$X5ZS=b>=Bf6#o9PUYIqf#rBikvQGuK=%^2d&6TW}^vd8VDE z!5K)nerj>B3J=gn#gwIn;%(yOQ{utsfPR<`eBQ8q>l;f@(tT|fe|0z$X?l4iIpAm~ zQp#<6?5Z1r>6Du^0gwP)h{VS!p3uJby?&zrSV#HUa5r- zxwet*i}i|Olpj;-4BP(^#LS?AVSiO`KWs_Cuw=zJ#l?cBT&Je;Z`q#QR&+IOi+_-s z(H;lSr{NCcb;EYYkB;2*Ivw)YhtmN@>SOZ$KPZE`<>Xd-0pCFn;T9`RS2DP|K0t_e zOwuAIKBKlYnNnfCNpw}gVuK2uJkAn1Dd+Pv-p0}&{_*Os(LX5RWcc48rE1y#`}`hK zgD(HkV#eQKP9v`OGE39;-JrDrDc3b4eHz4|=4Zw$Re$=tZI~F!?YNU~0Hy*dq(l~R z(YvmAxT7piZf!zbP?xDHIZ4n`wDV-bFfu!p;ft-XxCT>?n=dGuIJv!7XxV>w40UQq z@z@`XKK_2ep`~D)4{>rxkiA%{y!|2h%fBslzvCgLeSciKM3H2!={CU5`olBUt{)a% zy(C>e5D@O^a&Vb{Mj2FH({@4`ZW){@W0xB72+-7ummC*3edfICI>YlRo(X66Y!2?kist^E-;RWj1HO-=%;4VbfzyB*teCTr=68?%ZH;#;2Smjj zKWC2u6Gqt?@1-E;>IbL^7S?VZtcg`yc#>BE0e+yxdBiuKHBL$i=?SI>@Uh26b$)S3 z$TPX+Q`gp3k~AD=^)mVF)n%cij6=)IgUwEgAo)Y&s$#D2caX1N%>w_Kgin{2@Hbnf zz2uJDjHvaT6p|}?dD2R6e%xZ2z2pJ_b5=%eMwp*1F0_}C@gCKrF+?L3h0~7eR~aR| zo9S(glzr5s_1uFg%OPn`7aqCI-=CDpjSZCU%?+hI<1M?GURCls4EI#D8hczHFjk(X zA4*%k1L}3DuU!b^3QKC(vL0owz4$oJTwQxXT)3|9RQGlQwfFFh`SfVLpJUs-+k~OI zVFnjwGDjR!V;s132{o$^AP%QDUc+lw16NI>0c+#ljm2|X=ER{rsiR-)j0W+BE!+=n zQBRQF&8StS)!J6Zrz_jun#&qvPUoEHM*?G zyDZl>2uO?rC62lYuWYvJG#eTnFQ*riXz_4X(N^gX0-kCB;VFlwy0xHnYRD6 zFM-+3j5b%Q>0RaZ5D5b;Jl!G2Qn*2)x@c7&*G1b<2*^42z-aryPW29QhH~vp-zfPM z^SM{lyvos;T2?xM<2E7LM_9~io_rDHO+db~9_h_8Rj1*AN%>`RQC9zDz9rqGyV7I2 z+I`*tJS%duWrX{ zUvv@Lz)3$@{{&{TZ4lKae~ghjN6e{X%#SdODf#7)3VXb^f!`gG~X<(hwC zlZQO-`}rOfE}4&AHzC5U$WA5xe#8Z$kH%L`Ffs{DwRohJp zI!H{Oy$R&k(&Vn1c*^B?$Q8fO-_fBp`*p97G;Jb$K5IgzI}ZuBWda~69qAxE+=%hN zvmS@xOnvACnba)Tvb8*2#D+`t-x*j9i9n+;1=Fe#;J8B;H*6?Tb3xsT`+|ZlV4r{8 zp>D+!+Hz`UW_Bc}*WXkFoV-q_Bp_r%GqV(a?b2zIBqGSscU=?lg*_~Cv}89VV1GfY zn=48}H21Pt0%bC{eaLQA`~<1xza{=um&s}ySY0gT^+uO$$RJrwga01Mq(<&QDw>tF zGdd}L{9YFp5EJ<;>dZQu*~gnHqu8X23VrOdT6bL+T}u;5f_1s?dN~K_xjwUI}cU3We81S z11U!`rEIj8?I(uCb7^VKs+L&Fbqq0mF>jdfK6qGT>`$ux>g6|=Xyk#?Y^W;qyTLuP zQuMyOcdi18`FPI_Q^MGsy!jU6uo*p54XZd3QbN|2U_EVG7Jvb3Xn_YPq)0}R-|6zL#<*UQ+^V%DN z+M5JHDX(F%f*xwGvB=!#vfm{b--q&jS@)aCc(yA==&7M%B(zc}R2zV@B6@k3n=XN6qHwsdh< zD4cUPuRhP+I|-XT(>N{fF(x#;Xznq$w3(i}QDRcOV;SSjnzp=(8J`>7GxW2x4~GRZ ztQun8V(1!8nM1W^Bo#?9ElSfQWn{Cvsvmb=jB(zr!c`ujmqFuszA&Qd&J+-HI)9}A zb>lo5m-S3(p(94LJ7rRvRkb)bF&*&sMXi|M_fyGwm@(=iyQoTg=#AN_JV~;|%|x^6 z>9Q(dm%{%&bWCbfFDraFh{U=z#@^aPYvQU$_rZZ8Z_YMHW7dK9>1w>f8mV$Du%toP z1cAx7Js?UoSK>B{G3&dB{d(tyYZl}=#2x>P^+m#vRoZ29>RcPPs=LGEbYN&;%$}ss z1=}%ov%aRNJ#<-fbRz#d4Q_qwg1-->j6yMq%91^ z8BfeQ=Y8&G=sj^xOxUxH&M^Bo)gAdC+{=~;>C4nZ-lV7e0Z#JUuKce5pCk8wei(GI zV9MuVOkaJ|2c(U*?!8BBO#*h*igP|KDu@C_rS*HgEXx;3lkvRiR42jeRk=cQ8WX{Mb88cxBX1UT=hOj z?%`&PP+!mbl|%VI4k5_R0?IDSaWL3l%D;P7&?K!aJHKk`zS2au)C3??d`l@6C;k6< zkxZ?@OA|VZ3=Jfd=l?jRS8e>!A>D1hnu~P$>ECN@c8gE>SHiE3LTG8D&jQJzNn%Bd zX{pH`wLr7&EWEUPkEJ~LjT>ID{)ey-N+J{5KNY_|Y*OEiE}VkINXW%CMpnfq*Hu-} zIu~z5E-)2`4y4zmKuvzMZ&aOsMcJRQt*V}mj??SZ=0xdNWSg3lbQDFNRcUY39%7Rc zV4G1h;j?*FIU89}!uZbxKKE)|(YGONnWJgH3-5U28u1ivF z2c=Y+vw~$^-WJ)TZghzW6VF$|R1;m2xky}dNs%?IlR2yJ^$Dz^JU?nc%F`^@3H@7n zG)8h7b{o*W;grdUqwbf7iHQskDS4utW<1t7TLVG42lXT;*LD_AxC0{hZUhFXi!+Bg z7t4@@Nya3aqCo(Tnr)7NT*;drF0fVCJjLO4WM>{7`=q^vGtKADiIHGJXx*b@Iw0%s zUK3Ppm}g};lM4y|Hn#b4=mnP|e<0q0vr|^k39)Hn{(38n%jtJL?BvTT8l4hiGP6-H z+*y+~UFJE@m6D|AG0(*KQx~zHjT(Z-geiuL`wZArsdPu0`4Eqlk!rC_N5LGpc287v@Xu<~={R+%nNE{?vxqq)1YtITzlpEq$q zed?CkPX%AmdWY5f(?WvW#qn_Ek0xO6j%7i)QPPGLw3KnC$gh-zYv7ISRO|f*xnKhn z&o|xZf#KqE%r_K7p_>%3I;Jn^rVl~VcWT8lr}JIyT$cmhJGLZMlsct{R-=krBRtUi z-lw+i;#{ZkybdA=NOIn${BiC^+KT&`y-$UBi7q_dOY5|)j#jz1x}2)09&k`^Kb5w) z{&M)y(~iC&iD=24!sNIx@5vhR?v#xVc@M~Ivak4#Qq%=;v!K~+YHHTAhJ?7Qb}7!0 zSI9L-2S2b?MC?YGoBDKqy7GcvL_G;xlz;3dw9@-(;4OeNTf%oR;=T1FLpix#4b@hg zTj^Uv$37`j(>VkJ>S7bc7A$e*-)7+7w$SI*xObP)*__1Cd99?yyCkFbKg1P}$2%m9 z@Mxcdn$c3ugI&bq0YZJ?=Z8!M|bI-dj6$%VOj>*Z>(SA)ZO8%SeE3rhIb zZ0e<~ZsL-u6#!Vh;;HdKk6YNqX|9W1(fvKrc;7HL>HC*o{Z^c-Q(#sW36VFrPcR;Pb(D>GYEx%s75x zV)0JBH&9*HNHz(-jwSJ(*w4bbIF?qyJ3a};J*|5njXm~WJB`rY$GI0#s&=E!mOy={ z!tLTA{6%d)!YN_rih}3SSm|CXctres37ajpyWLt~=KhU;DRMQfizk4G|2rigd1j{L zX$EY;+$zpV)@xVCxZo&$Sb3=kkleO35l5Kv`C*Hl5JG>;Qz2w`B*xklTWyo(a&|cT zz12ruO9;Xxucm<8r5)3mW+{3^2$%aGr3?*alV?gZ4rW=X!b+F8q7k*+GE1VSJmz4k z(m4}Wg3>jsA7?&;sVW%{H1^3=c%1rKf@OesuAQ{%<1fXfrR%%Z{aZJW2WCT+$SbmD z8hfhJTIVEv2h@R&V}J+J{oBl#OW0r_z#CKQv=bAWSlc_?M!we-)Az~D`!zVjc`dGI zuGF)4)%wKJ1EPGH|1b20wZ;BKuTfTnx2aM@4dDK5~FI1Tu#Zyv4^Rb&|M2HE>1iLr$GJAy9Uquq9xO;)5+uTNmCx5lANudf2JWHtXckl%GG#Q) zZe?m;Zj@ZSKb|2WuNFw;^aAvO40K5*?225(yX`l3`sOyoq`2`iCZE2a-M^XrHRm1ncAPGWYZzuu%r?J$MTapR=~Lq+4lGQi zMC0wocrXFG?18SwixL-$5gY#KTz@p4oEJ~tdPPJ0!gX=AI?O7Y8_Y5t;$P9Qz1g^p z*Ec`VPZ00B<-5OnB`5Z_D=etpF`mTcTmaAQfeR=@0*F*%B*dLE_C{ymQvK^O`4+gg~=I^rc^6jyPvm5wo_E7qNGiLU`&9 zRs*(I!yJlh1@yzNh=YL0-cu6tZWLH)sTg=@8|ZmFYJNWYT!DyD%H+Kyk=#%o-lV?l z$cJBU^(dfU{sJkr)uKT4L9buF+{jpr3M7;u0KpQFe%KeRRn z*0(Cux8j>xlz%?OtlbC(5#;+Pb|+Mn}l zJ5&vWsv3bUg8q-Q6M7c!)zYH3zbVzMa}kra=peYV2K{<22R~6U#j`7bYkoHgHa$cB!XE?!qE#f!ouy3v(B^xs5t(GPs@RZYG6C z$&qh~Y<>6N`tIL-A+$PUl68sxhRnVy9nK`?Cll}CTVCK9Nz)N+AJyeKQz>TmlTY58 zb=3!%&%qYI50g-X<{(p#QGNZ(11>&9%0^B)foJb~$H)bV_!hABjxo^*KqU~kSsk$X z%k>Xa8TqcznL#71Pf!&8N_3;!CnPP|TDUR<_VZ36$)ZVQSu>y0>m~T>^s{w49!7J~ zw&$u6e^;;QmEl*0Sga`R)Q-X??Pit~rR?nT?7R%^x0F&CgQNiUb72usn@mpg&G5#x z^3qXTsb7knGX99Gd#uZ=f?S%^^|c`Sr!G!J_gzg8H6LU%1xvqqU-)XK(*JuF18|CH z%xnsE;>vK8^aPv(&da+GchUZhq|yoDfoC0og>~X*a7JZr{GT^~gq_A-CEYGGaO%4& ztUZ|JKeGGtY-Y_rujsIO(PES{pZUq@@ExSnMqMH#tBhp(A?pu5R%yJvp}E(?w`_IU z8FTi`1IWpU9HxSE$TG-0w@uz0G>QAfPZ=Q~b9YkmoBK4A~sWtZ`8YF_x18O)GjWP$zL zxcN6b5s2GXHOwdVs)yQKNqQKG*a7^ZR$_>|EEm&(7!m z?7rXc*ZV&XX36-lYat)u3}Ib6uC$af_u}BDiZUHB;Qx9BGi&n-ZM^S!)5rabV;agG zm}W1R7{v=ReQH)Ve2q{kkUqgbpB_WBWksazMR(&^FcoM(){xU;@BF{|)&;~>h-o~^ zTe-8!xH4rkh&!2-*RJdJ2%W92Ineto^wNI0lT zQet2mLX$w|W7hX-zWIZMZ~lCHQO!wjL_~_?2(~xLIECfebvyXI%t;r$0w|&ZU8Xoz z^lpw;5exb|APXUE*`lt|;X^`(CnA1tApT9DsD{E`wSt4s{!GyRs%+c3>@z`VxaZa? zb+FIok$3R9_e=~BlK=z)PrH2>;SR58k}dl!UetGN7U5!krlnDz9bt};bbo?~o@H>3 z5xRxw`X;0OGQ)`a(2Mc{|Bvyg-Nk?KLzxu&_RPeAr-J)eZHHICp&8Da?vx6r$>7X< zV9a7Z)FC^2TVgbj8qLHZVyW#&RdbwvcB(_8l0$nl-8M#GJD)hMFcWzUv`#)5bjRY> zclL{@94Nx)ejv*%s1C6g+5*&1xRpQJNf+bRQ>?K2|fwsOc4rZ+blJ0I=+u zpNO05?scj5=ILfXBF9YpHv_*@n2OT6%{LA7xciUC?WtXQv_bJQGTZb+UP9Xij=qd+ zx}W4)amEsf(PWF4o{FVpAqw0Q${GZe<3h7MDs*7rs~T{LJ9OzI+jv&u>OcYIX&8Wg zXJhck_=$(iQi_~)mmmvh+F3j3!0)2vM zpEGI;%bL)CNv;sjFx&mw>$E0)`kG72(b_TL>$J5U;dr-C=|ctrBnpaj3~@KZxE;r# zCk+X%tLfK$#yut7yk$Cv*}2SGTDsr&+hR6=OYR55ymd{g8UX-(8E!0Y9mRLFKXG}h zet$69Y%;~1dZGrh^ntp%9sgQF>tcL4_Ji*Ba1MK~W)PJ#NF*ke1(!F5H$=FG5{+ea zffjfNZF`sZPy?ss*H>efQ8Ay4c&0Zc3^FjOxsTAR`TdO#9$w(U(HBj5t4UEv%Dhmw z>MQ8EC7FHV8MX>1erHi;LJ|(?S$i91(TYO(lo>I?7F({ZY0<4!7)Pj7G<4rVYwX@3 zIvzksXaW0`)D!ADI&P5<7%4iUHQzB>QD2MdlZ!?J(5{MFrU@E*Bei*S8qi_1`V7A~VT%&Fxps z&j2X*-)iW?-4b|t3+;spgAfSR?V^1YRsF&yOrPy~fcGmjx@mwESB0gxw;FU~ze_h{ zO08CqzjD`xh5Q`trfb&Dq1r`VtJwxNj@CSN@{q;HL|8d+0%tgM$Z6R=|_P8l8A_4TD zVlPa7@WlXnknc-pYcyZlS5X!K1GR_~!J~m3Q)X!pyC2bPIUkGHhAC4+h+CS*ZdaNd z)o-~`7kh?ZhwNr@KR(g63BQyL8@t1R@7{rjKXv`EHkk@vFqi;*|A0Y-yYb6Uh;q?gPPukY` z$t;;5m$6~H1fbL=Gc#WTEsoKQLD-}y#Yz7K)i~uCS43xdR88=H?40UVygiyKta)Qy zjreBN$+J16%cAI!$e9&Vs+254AS+?_N6WE8u!yuw7Bij4af51d{bmkgpU=q!pqBB{ z%alVvmCDM_Qe~j_If`6zqKy%{%%3_L;aQO`GBJo?!-~m{fej??_Ubg$iO^^9cvxN|y{+Ex)Rp7h)f0h7jWY8X zOZ`XemVW!)dVQF*kPlMD$Pns=i?RnC>v&~iX|LU*-mjF2_UPUnrNb_AfnFiLG?{r| zm-(meuZ+=P!*!}fK5mOWvEqSWFK|gB9*Lc4HutvU0tMN3^7pp@WD%wgt#fghf zIvcavODYNj`1Ci1PiGsF%*Yufhi1_qnG#w8UTk?S$`@B3ft!QHgc(dzvT9=+L@s@!ae0aO!^`hqP@x=GiHaG9CN6KbmRgxvp(l zAS#&oQf!(R?LQn;`lIJEQ4t|~)h{8iFyzf*KXbwsa z;x9q7aeUo?HO5t1=KWtfwWi-oFNvu@F9sdX&7N?cz!*Gy_Ze_{1Is zZKjU`NF#tFH5~#n6N1XpER;%2>pyWgP<(?}<`(Q~)IE?;Zw!JV>m4EAZH_w`va&E| z^}j~aK^Xr{k?oQAzKccwcZ3KmkSbf35A(y~d1DU;_L_>zhEya#lN*dmE&JkoO%0SxQw1zAfOKTbP=( z+vmdfR0swU>yxn`3AwxgX+NIp4;ADOU3QNEj9SX8gy`fc2D!`s8u(|;5_LoN>qX`$ z6{*PGeJmSL``SjK5;ui|4mTGQ)Zc(&GJCa48cbfXr!B1u{L~M70u`iSu4csw? z>EaqRu{xg>JyLxCO=c_UPJ`VgLJsG>WEish(qb6g?1oPb$vyVz7G4?k^#7R8&(lIr zFXHPnu*Fl-6;(_r>W94#^VUy}$?SXJ8*h{8sj|@TwmwC0ncxK>r8fi&bF;Vvd9>BW zLF1LZY7w8dQhcr3zUyKso==*mX?(3q*uN4dVw9YRa7<`}kR^-*N1=XZ9ly{ycDqH3hW zg4|&CacvOh@$u`bNu&Gj4eA{7mAn+x|B%FRZbi4^kZO@@t0>)7( zN%{?I+)U8>Yf7Jm0r0nkT2OBlf(-exv58(d+XPwUFl7)O2LT&)ZdxxWA`Ot?hMWb2 z+;@v0$x9D+eDs!zo<=?D3-tS{;iVxum0EPO4`tgCurw2BDt=95FQxhL5s-i7{#?N$ zVUX(?5nm020$(G7_OeUyx@*eE;@yyoC^0&=v>rJ(Yg7}0Afphtm(l%_jr{A9gN5+$ z^>0$X+rK-uL!|7Y8&WUm>zd3vlYz?HM*b7#DX@|GA02-CY0&}Xnk_>jHY^mJAKDxq3$ zyWaUt%m$`cMbkpTV|`Ri3}Kd4=8{=B=OOBs6Lkw|Q1lw z&H6h=?h94hm9xT8tSYsP+8*H*DEsv% zIb0V*%3{QU+lUe{n#V7OpYe&LB`PxNWYK^JiBNQ5X zQxqLws}+yGFV9F^v+(qtLTETIdw!dAgaXh$b88&vk$B%vhX!xhhwEiN#Wn|tf@@#r zf9RR*$BpB`MOwp8fJ5l1;uoEgo&L&DcIY>0WRie_Kb@{QBo<$p^HNDsngBGi%Yl_5 znVx~)%)`qr3UD1(0KRV+X$UK`WsNK)Pu#k5UZ*jBk3HSWv!9k&%jf9$eI<@GZ@l3x zR!;A*5SVyfp7?l!y1r`%USs&^Y^Q27vrwFx606;g ze=|YF{IhGks4a%$8058M*Tj2Mo~&{EnPbD4lc`-Ag;Bx6)7^TM6&<~_V^j$at&FcQ zWzyC+uX{;fVgPUof8D2IpTa2!BWu;$aepfH=}iAHMFN7yKR)kotyHbP<+D8Ly&xy4 z$VPx5E!Vkskkr=DqSMJ4xGH~l8~N8}b>g~Fy%w5|(iVBB*{ zc+Q%E{82(MR+nax`2&C5-0qx7fpOnsW+Rvc(;T}5Pl$L#Frai3HDm)7Z!1`6`H+YR zpY}N!m3gI)cexc-+joyZzE`0z&+ltim*9<1cIn_WaZ|{R>dNU!W3Q;qmPg_H|j3a$g>@7fbr0cug%^t*U!Yo`*? zXXTuhNqF}xW5e<>8*eNgO1$RT^i~_S?~PXU=|$E{k;b~nv{dzfr!}ByxDWdp-Z{HO zSoJcIX@s0OK{OMCR3XpP^@jm3p^EN5bmQtOWl#*CwJeAgHt~ReBkNZw?Ro>YIZ2a* z2BcrKJ?SN(d%Y?o{8!)0v(Cp$odR5Sm0uu=8>QA12*sy$gPdh z(B5*?5xGkeVLMu{E!%6%NphMe24M&?LLTXLuO6q10P*3q4n+?;J}mp|MY|Se3Dddg zj$F9TOTm3YsrDfN&F5rB_Sl;cW2FYs@JU22T z12l2_ustUSn2?w7Zb-v2?he#IhBsjxrb{4g6>9P{Pp7I_$8F|#xm65AXc&+v>6;ks z@eLJ>zWjI>y;~c}?OHk8eie@D7A*b&T=iY3svo=97V@)}~ znB+AH_|@G^SG*A=QItBl{a+WZW6~AXy|`xmVa2jq2iKk zra~J}g$^;z0r~7b{@DmVv|)3C7-bI$9)G8l$lxWI{^wiA z!Ll)56+w1uXnaHLxv}3=wOqxKzo2B3owmYx(zeMW`}u|x#STdTi#XB_!;Wntlrl#( zFDR(zY~a$GLlRX->XHkn3SPT|JW3~-$iNWwlb(#X7{FXO!rsH7>{Oq*HT+(UAA7hL z#o|JUI`>K_`@j7uVOIC~oh>CkK=VQWcOtwZLIpOcE9b70im(a)EU291{fDR-|JVfS z+R)1bE?waRuEE zqiLum5pH>IKv5ooM`1a{`#gGuF)L{w&cqNK@@3wn#^7}1x1SxCw(l^h%y93_S+RhG&`iZiDu>psi&2o+y|aDOqxf6 zE^6(cOP7aqib@C^=eyG@p^Kgay(pv{drUgsSakK{xA4e9@9+R4{rFT>RcO%)Xz|Pe zqX_{`$B$W4&s_@8_20y?hp@=jLT8+A_~R?n1vFx zY*{xA9Yx(w8Zb*-X)uA)685O2SQ0e-4DG}d2?=TNHWm(*}xaP44kp%rDy+vqI zxJL+GRy)As{JTRWiKXYZnsU_VX$;(7^^wX}sb|Lr`_%N$s>=oYVNBCojCW4Ml z`+rynnU1U~6@)dkj-qXPZRj4Km`mwK#K2~GfCwfqWhD(| z8KFo=Q^ai4;@m=Q>kZDm&-M|-<6lsx)%W~#lBzd8|F|W+U8Q7||7)+^Bm&9#d`k!W zw6_M#d}go)bom79=VgiR>dhJ`)>0O%I`D`0^s8%Q9HM7h6&ShkRrY;s?h7lr}1Q(O~oR4=65{zf7oW>`;(jMx_u35`VQHD z%zxEaNW!i?Rs3~Z(KZb5eI*YygnPI3J$b_R6$vWByIqLrAITC@@X9XhrzA2ZsSm*a z_)BkJ(c^D_afblIEZ#hZa;}V3a%erIcWpmq)4#x9q|apBl`pXCCn%xlN>r_JbdA2hPliNZkQ|dQ6PX{^WaEX%V66UH zasM9+AS{Fc7*SnXF2Qt2Z>rz)$DZ}ZW>P&;=-OX_^xY~gz$=5i6!Db8N-XluYm!JD zMo8Yi=}I&HH4=-P9vMla#sVq)Y2*f0ja?>(w&Em2ZCL5lX=(EHoGHS ze(F#$#g+#ftE^84KhV;AC|#}8joRof4s+ig(X*YqaCvvCj}BGXen*lcwe<1%k&aSN zT;=9tORT*5IgkAXok=V0-1-NDwGUW&Gtp=6D>I#`NN+f$I?VJVJ`21A$={M@8P-B! ziZgayc}@GWcYWd6nI=MFo?A|qS9#zsG1l9wvT=5BxfDD4q{@jn-2O;~nm9-q(o{UK zsD7iYz}OQa$6^e6%<7|DM+pB{lSk^J#I!n3IL>sMd#zmUT9szaPixiFYu&uD+YxR9 z2NB@giv572luHsLm6(+2Wg^_0aU7+6>eJOnhhfCK>(Pn67kn-+vxdJoXuW>;bj55> z^)gS#JrimGUr*AK?5zQpC*gz4(p6WZ-Fgy6kS|<6;ke-VAhk5Dtg=%K+iiOLMI6t5 zCtX@;3=FAgS2WZ2ofbk`w<}02n4`GrJ7Lm0fgTf#_l4QcY>>Z|=6VInUzNRee>7*< z(^IV7fg_Sdp9zm;$%&C)Fso*~ynKk?c<*?LMYIasE+wA_!MvPNgz#$Ala(5qxAyGF zhRtzm1rHIQ`?BXbKj+68>gs&aaYv%-Ii>!XU4T$KL1(tZ?voWgvn{KPgEok?6^F`bUEvOK3YrLSWrLTj!%`s<^+VOUm z@lwW4>!oqvwAo()Kps5C-F?64(u{!5OGkm^QEUW2!g^Dfc2-Uo-rz*?q^ zJ2h!D8-z}o2@g>HypLrvKM*slgT}xe{UvceA%a*Yg>$jeD_qmK7FLSzEB%J+C8Cmy} zEv=S6M<6w#{o4thRvos(+s1k^f{hXpprr&}xCKHjEMG~+Y&^py4CzgI=+Gm4M;ge_ zghD7{BP<`kI9Mic>;YvtP{!EQWx%53K+zW#S7AE$z528BCwktywbIo_^_9SmV|^?e)i`L<-nM%^|h|fg<+olVTWEu zfdHq?Nzo-6M32d!%u1}r^!N~%Sg(&$j(6Kk+V3@5Mr3{>-fK8?5x8O`=)z#i)jlOV zP{7}?l%X+C;3(xc7$EFj^C2cKHc{PNimoIE`C9TOOce6c8DAbnOXyi8n~?)aL-;99 zZ6oHhtCh|xiSbC~I)%O~!A#E6T7-La%9#Wqk&qh<)xA{}3G< zCb`b*qs0}r@n$VdC&_K1?&Y6im8tXKe{)i6+_r`teR~!?h(GCSU${9#9^w5+wCwEO z83~J$-u06ZAYh???+)d9-f*v-vohp8Jv^nD_5KRyhC#e2IY)fke-OQQ3wG|ccJl>w zUyZ~Th(ezm+10KTmPD#61|ILyHLfo9;JuDGc^>}=O%Q1@BNN~z6CKVm=Xh2BPvPcm zVe5HmHcUL=v*b;ubKpDn{H&5&>yZIb4I);;KT;{M;J~w}$EwC?u}-ijzTCVGbU~T! z+V-m*tQCD1*j4-VBWAH|9;okNAbI>`0n#sc6IY{g{_vef;)Yb!J!NR@87fGR;fv$WbBNz z_M9Gk8OP#z*KcN==?L`)3Ut=FXz($zIV~cZ4!zRWCPnk=+h(iZXTG+22{OJ+X=%fS ze+Z)sNNGNiD?seC14w_wn+vI_4nGg>Yf`FmUc|=5p*WAl)hu@l;236_iZ6U8FPc}w zmEy#qu#JZyGT6b_=S?)eh2BRgFxjanVpg{74>VReT6vZ|sBmiNruFk3+74+Y!OO%* zvl|KO|6;e(LDK=?;owi4OHG}Iqrr|!ksaT?RyAd7ztf_L<@pum5Yd<5k3)h*>M(2< z%k>z?HRn3cte+kekCqS2KMZwcORP+xo(DHqTB9WzYKovONIICuqtSjh>ln(WVSuu& zj85j{69BH;k?GwnW)w}n9}gMhNxl< zLS*@LgM_0SFyxY>(JE~NBCODKotHyatFNSp}>z5kgX z$DMCzvS9qkK@Y2|CkqBqg8MqB^<~ZdHwJg}*phkCWr9Al=!XH-J!QXXuYIgc-YV$t zG=k?$hSMf1?Nzt!vKQwco@LX|=3|P zxw*cC=*R60@W#wM*mi!gN&vAv{oYM|S;VO)w(JvkI3*89zV^JQ=WNc*;qLkRieizX zOHN#yg)!c3m}KTp4NJ+6P_LbTdi!TSdEO%46QE$dxRIY-kj0ACICm;s=Yx^9bRWaI zx9+W|D-EKC^+sK%9&|g;>LDj**Q1h3}4}34u_^zy4~~Vd?BO z+9L9tO38UvB+%n30tsgzT%7 z3dlr6E{GFfS;DfLi$==xcz{$f_P3vv5*yMO(i7pn^BG5-PE4|HX&j{v9I)$R^;-Mf zMOh>!w#pyKZl)DUjAF>di*p+QjQsUfdyiJfHibC5`h}IbPQIz`V8k=GUn$@- zb#se#g;Pl(QZb#Lv_Y5vRneE6no0&}zKdG^r?U5oWm9ZneX;-pw3gDJ`oVJk(8!V~ z_pdfpI#i)Lcq3xc>kSBSPnRYu1_>~jE)0MckC4I|qLnrl+d<|*K zCvZ-7K!yY8P{wtiWk7=FR?**^z}eU>nJD0VQB#IAt&?V=^Lq%zGbygRr&6QUKkZ%U zX<$U%gvu}QzmMOiefRLbE$eHp{m<1upGGM?=$DJjJOFvOTi)!o9(uI;`DeQyv~Fkl zue9p5dmk(}f13KOXI_^6zO0oabLbug`>xyDB_kSN=W!u~O^?97#Spqxf|v^flDD`I zonpC_cthca*3(%3b*UOs6yUy}JE}#CB40jw>B-6cH9YeuDwN@DEold@E0+Xzq@e*O zI{EH%{EHSYde0I=70dW~EMAw{S>+YO;Nf&CVx!U`nz#1Md%!$VI)viT>FJPOZPVGf?HIg66A(qzV?niw(n*tP+h z5*^lihS|Ga71lJ9-q0^SB8=(&oAPkwz#(G~<=~fh%RJ65i|R9l2y1o5yEN#b$tOO` zVo54ikIU?yo!=4)doD0+X6+C|SpXy_dpCFsJCjtK8DuZ_Q!8joVDE9{yB;qG?!Ykx z)jRd|S8yntN_A*Znzc?txGtwtpM(DI{(~*d?lnsgjaeEyKFXrOCuYR^&Ha2clkp1!z6!{-I#V;@@-iG-d)-E~!J zYbi)XX4J$sb35Sj;|BmZJP&_LefseBn4`qNj>$)7F-#}1^H@w}qvCZfFGhJ!|6v|c z91q~JGdGLiGK)n1ntHPUE=j<`DjaG2Mot#PkRg-!SrhSd=;O?v6}}eV(-A(E7G*|3 z?R>d}=d+G}<0lsn7|}3~9ZM)3E>)n=>&cQF zmgi;iqY_+lH`@7T1xL@)z0PBpSB7asDm886?SP|`hCx{$F=+=A)Yte83XE8~wP@{XW!o{eQ59^4+Q zirHx;+D2?5yG6yJv<9_Ggvqo@+iX-Tg>nYIfJ%4j-K& zsIHY>@VcQ&wC9@J6!G&h)SKoH+@ja{3;8E+*H^dwJ*gfsPWBUlBOEx)>eANK`a8;& z=aZLGL+sfTb+0DL5SLT#BoF$5vxJ z&$fJwaJZp})8X2mocM|mj3JkMxI+T__|(RwB9u0vy}jsRjNOg-_)Qwvf^sLkm$gj z#CNgd&;slasc!>I_-$o)^w?Uq%0ZQS0(_?uZ+CPRfJ!qc4jdwbS2?(h8Z8A6EoX`x zvfN_azcJkme&wx`k&t;i&K)6W=Cj2jQ*dnz3$n`025wEZyh&2vVGHswUJ>=; zjbkV|^XPjvX1mUt86ukb$oLpRD`RszeTc3;I(=4cAvZ4I!Pld{nV{htG~WmNRC*ls zLpY(R*pCpfKSTl@(Yqp>=vNi+Jlr$=20}-c^Qv|j8}@t+a8s9M-@Yueg3S5JNGp#aHy&bYdoW2ji6?$%+avzkZ|=E_kA4el|vmVSufK}RokF0?$h zBVJFb)KA-;N-_OLbzxiXD3Wnb*p$a6GUNDw|5VtlMwx9`{$`dv@C|!cc}XpI_}5b= z1IYVYb@oUgzy;aABstnaN? z>;DvuE*v*fobOV1i6~93Jjk*}G(9WYEIE#;7Ek%lTD2P}9A7%1(r_%zYh(S|ELm;p zWAmr#s#&9C=djYvmscrjm%pEzs5hsy#e4AsU(gBe&(!PK4Xg$0NiddFp-e4}B;@z5 zj^*?9<_ISh^eFD18bQ5Esv67?NwhV3hWS(5A4$yUMl0yr2J~XR5E9noZ6mr8wIw3= zXSPMYmq#hXcNy8<#X1~mH-feYdc5=`ava#LmD`?W6uv4MP1W?uq;EyRt*hUiF#d|F zv^cfTKC|DGgR~AtIO@qgnXdW;o_kz2@F+M%L26&5yyNf1JBdq4RFhZ(uV57ud=flw zBKs;F-b4?W{mm#^zDH~yUNr|7+f>y3wIfu>-MwyQ3iNwTPqw#R#Z|T+Ux_ccNw+@z z+wpeBV_x1M^~>;Y$AM4slWP7cGek2xxqEQyRlRNMhNqG{*2=ZdBBoB=g^kLii2ItB zJ{Z#5?_sL?jOe(L*v1x~wr3HwEEQ&d5(@n^Yx4q4ne2<(Or8UwaN15 z-Lra|C|A4e`j~zXe4Qg6qe1ptrlP@*wbOa56YtVLJ}epaFfm2d%URQ!dp^1NvA)V# z#wu5Kswnf+d~Ojw$j>fy+Qz8&zWb~BU{2&Q!dH2@E+}hI#b1wpB^k#n)svA~nAlXX zW)Mt!1*d*^m^}}S3U#6<-hba>$Ty8uHS91iS{a8}eiY8qGs8^5=%2?JAM6APb58rO zXJ5V!kbYQlJs44_z|b{q6x3vMn-b@=RyKlmIIUwe_+;i`y>5DG&*Aq;4*I;2*#Y&G zEks#(vfLgyVaY}TzWQzAX=XH&PZ^zxi?W<7Zr@!XFnu1`>wx{pT72GZQ2A37lI^%> zN>P^#3jXYiAqx~fC>L=`8Qw~-cS9ON|B8=XOmB6Y2Q_70qQ3|RtjYw>y(`}~-^@LC zhnEd)?_M6Kwzo`|v06-RuNl@0L}t+gHU=2SEhiLGotMi8UrGPiB}bJq5!o-yvr%3S zX=1~U!YR^JfA;p&_t7jutGl(D>Y9&5Z0Oe=O?3+cjid)k_L8Xsl36at+m4grCUlmj zI_>An{xh4&su@P+`yaA~n|Lj2h4}*(^Z3+H)3e)1$MJm{7&i2Q5MYg)(;uB%ovU04 z!K^<3YPr{g&)DD2OtJ7oW7w?vrCK%V?a?BIN#PxO(lq6J%#eEseTLTFy#m2N`)(a3 zp>f;48pJVi9-1(7;fk4(p!ZZ2(@oWyDRt#nE&4WC01oME%QdL5=b7GudICdr@2#LC zl6`mBvhNIsz$H@}!KmF=m&ctB?UyY2iWf~Y@a5}5$!X6RVX%UXUCWo+ZL<;jb2pmS zXFWB)`F5f)y+&^@eRd+dMe8SCI?P0%l2~N0jBFI<5&+jt#pRNlUZ2&ufK=Bn(2XkU zcpD1ZN!csbqiH1-X-A{vF6gTaxBLZeYxCDsaSlzj{Gf3| zhZs!BlX;1iVK;ctd?_Oi+ChF{uw8+Ryle|w@K-o_qVIXoJ!&9trU1G6?R-}^RbqBE zKV|&tYOE4EBdOKu{T=t?PHdD%QtJflm*<{YTrwmTM@FOt(zH)rD$t(anVe9 z>8T0E@Xl~*FeFH|O_%PPRU|Tz_DhS?1_nY3_LvyRF#(4~KYE|K!;qq@!!B7Mn$-{i zde-p4A^V9mtcAYYtiD9WR|uUDAII9N05hn_fGW=?FQ<0qWaW%_>bPRR)bVF@`n`R} zQ=Z=2U$6W7BdUy@#@^f)!(a5;`cdiS;`$qUNNupRG$P14SARi;eP7*nAgsb~b4+?? zCviYI)yq4ssO0v*D$dbbsN`5`cp`N?e)yZPG_0ot0i!~yomJ()FBeamIB9)b$hK>Y z+;L5RzO*&oOMM@)NUSBJVJ?5J9JC)Q zXqaqHpWz^{r{GM-xJzEfo}0R&$1-j&QvyMEq;3-(^6tN|k@+3`yV;xrjZ-W|QfI!c zb8bhm1hOTS$)CDbLH<~#3LDw|Iv3A~3hXkuly^6`1zWhbhSL`QS>{sF&{Vdn< zhLZhB&AX|YJotQ+BgXjVFM=rGC*J@%>Y1|@1XbiDN7A=*>LWv7dsv{TreXyHXk*fuA$A}nqZZ{-6^8-!e z+X%Uew;Up;RfOSZ&){y8jcn8^w96ZL)mr;G2R%d$cCt4F)AB(T+`txWU8m~OO;}`8 zj-Gzi*O^Gu7;srCPZ~E|xRcR&5fN$pTqREj{Zp%_F@@(eyZJXQTY{~Sm{8CJ39vK$JjhR z42fbWyAelJm#*x@RLThBD1u$^d<)Wo5w>mwW*lh-7a^=PY2w;lA33;GjeBoCr3JU3 z5?p7o#P547PnubKUa`j`{8evzgq%SNW3l4<`QNp{`t zJidyDU$5;Ho8QGCFl)9CuLq>A^;X^=DB3g?Sp?{(IoICTK5BT%HJeYpWV~f-km)5Q zc~6OsC@dDf_Rwr>?BR=|hL@CPeBMySH%U6=n-yy0)!G65d%wH(0bsx~4}jC~ZFwEr ze)@FdEi<%^2`{{;#ET9?d3j0~*jufDsB^w>0N7^{VNj^>vF%PoNXZhKu5PoFTP9WR zyN>jLI%41*lk)}bWuJ1^Btq0|_lr5|56560vgR?tvPWPBzL2%y*B5otSrWGMU>oeN8KnloB8#4hq|4; ze3ACfZn{W8ZEg(ss&?n*1~vp?Xv%Bvrk~LcWOa07H z2&YyvwFop<^c8Z8DJI@w(A;ER?5JOGIFRG72KzO^#J$nLrK29Lbbm#;&(N8*`49=q*SI7)DxDP` z`RuVX{Zk3`!;g4L-VDBBg9$CI6uwkuoyLX`;{yIG>>m)!E$^g%FKVUS=S51O88^9f z1ip);)y|Wf(787hA=prJD1+8LrMGqrq08gco8uhk1IY&p9?)x*tZkH0pjvc9&~BFs z)t8-T3gbKJ2DdeVoSQ|8xtiH-E4$6M*4AyV*#S4R31YrRexvSOBKs<`sdsWcohexq z0Ax%Vt>Z-GfqXnZ`V&s5nI-y7v$p_q@WrD~F8iB>qK%9Fj%-M%>z(MyD~) zG%hYcNUXAPv|7`)IB-0lQu_xfs#c99Rm}G}*-u4YTVwXiD>?V7T9JaG0*4Lvec$gK z^eR@4hjC%L)^1^nyO~wShJGm%eQelCc!4THAnFf%cu(Odx0}yZzMWlm=a{sWqO*oa z;M#srW1`;QqoC+yagSaO^~(We%5--lSt-hN&#OKf^yRk z?b0w~vjc9$Am3Pp$**aO$1h^jHasUKn0J_{{fv&ko*4h`ab1-=x*MdVm^onJgz@No&0NhS8wKx81eWu(^bdLZ#rP~jN9$Az>VaQ z{76|#KkfYch!Zz(9cr2U+>yDDm8zsncjez`CHp?w$GYwvN`IL(I64L zHeVtK$?#u(2l6P~9AB~h_v+e&NpoQ{y)D!$_Pox^ZZLcOKGTFmAk)OrYy^Z4=-00B zbwT1@?#@E~b~Sw4bc0mD?#7F9JQ>OBJWm!U(FPZs$Mn`|GXhu2-Tq|=95-y4G40>9 z4YFQmDdCM^a;Q5M9gg!gY!kBhZieKIJhEuFnFDo9>@Xif1?t3dOTKM_7l;#58fjx1 z{+?sAsuJ2JsmBXoDdtuvcL(G4o)$Qsu3z~Iot)-wHaKpcHk_Xxp2o5de=$Y*u~EQHj`r>6tFR*Z!`_Hjh@CDLLw7Derq7CwK$#A2?mc@1ua_(Pk@e+B0bJ)mtH4 z(b(#-o$9CVEbQEYoI*5=Bou`A2vX@nOGyPBZou&6lZYk+pR)@)?sE`%C~%obuI$0R zmCz0)i~l(r_JE*^#GAVUkjNs;Bn$}sp93yO_xr9>_qyRnxu2vKtEs+V3ixeje^5_j zy6}92eu3dR!u^@-b&+d}^&jp6o=U4rH**Qi+e*UjP5sYhrKkujq4HFpxJQR0I>t`A zYG*sJ`3Ai*?x#(kx8GiCYH?gTn-O|VfBRsmV8BfcKz8ORw1(9MI4D6K@? zoU{!v1)PO<*k#`!r5a#V1geT);+f7<7ZM6K1=i`j{AyJT)aA~H70U(1Y_oobWcN~0 zeYlp7K)3n3cj~Qm+(I57Mb&LXM}_d5oD#pw{O`#8?;ZZ%Ky9;zdWkMAXS-KM`uun3 ziJFBib-(=y+5aIIw7DEpWh7 zjHEeX<#u^mN8@)7YJU-pslEy*Av*jU$n|zms$~ zK}%V&QQ6?NoK71T)wi9N%~1A4mZH@7_abo?{E?PVhki57Jni?-B0C(+ujX%rSzu=o z9kRwSwPzpI56v*6xM;p6=Xywa8H zEy+XmfN#a+CXFKUw-@V(yf_6J`QturL^uvp;xT%)EGZA*(Pz(i56&#~POd#{UPkxq zwAL@lEcu=+(xAgZ%({;`0O zBXPe{K@Zj$OoQhUSrYP>{b2T{pdB&fRxz3}(K)M*IY9w8L`E}mTvH7!B>$-K?>)H6 zu5a1ow^XcAG_aNKOm|&{bcdizZRJatOG?+(e`w%Ip#5t8DY8Gi*8!BjXxk~Tnl4+N zUMMtnOdeghL`Il*}KR7kH0??c25x%hWCXh-TlJeU#67E5J3r zWf7a>JLZ z{9PaBy4;_w>gj{ss*XF$r)h>>N>WNus8(i1O5A*-{;S$-K!GcJv;;z2b?O*^Xbh^k zq_6{BvfTMm{b^zSqH(U}m~1A_XQaT0uY_}UAQjNT96M|EXC@W7PjJgVH;wGHSsW@N zKKM(|Y{n=CdB%W^no4+PO=vF|+ zap2nRgn+}_Qu_pqNgXWfoC|?QGx{wNHah=WQDv6T=Smox>zqC*He-{*oTx^;rVjsp4`(mjly1nO74s zirf`dSiKbnE$w2RL;KUJC9b%}CHG{XEVOA` zq;8?*b(Ku-Kg^i*R+`{XO17-rlACOcJ$mAzuSY~aoD#TS#8sZB7k>O%rl~rAl z4MQwpIpY#gm)!{4})v|@G|LgKZcWjS-24p{LgY}RJE;O3GmRKci~z5ljME$m1k zYx^R_UY`yZBGptc{@F+eN7YU6vfwHjIC&kBga5N!^q8h$c|Em~6f+fAACz%<%o5u2 zUPo#Sxyx>K!Ig@#g&yVp1A8ygRIVgn+uiTGcY<5%{5_-#1n@gr1%S@^ts$Swg z_SfG=K^AW#UirYjU*;S>Ys_-XskA z6DEr$R}+R3=2x(jnn?#EJj%CISDeOCjR@!cW1C9*`Wct`*}XSH4uXw+O54S}CBkD= z#nU%ya-gULAE))nq#-Yd2KQ%`)2y3Q*SJ*Pp8Fo=TbP9-s%QPrJSLs04cW*~k>zwF zRgV66>@*yYDaakHWKm{+piq8hQ>#lE`mzVOI3xXVhg?><|9P@Dp^b8viu&S1y?Sm^j|#PekpU zqShLA;4XS&C^e$i^1Y1m4N zY;j{e12tw3@s9@+)lzbGAW5n%cj~I0hg|M%kgHcZT&TD*z*(U%2Fn`=cTDM`c9; zm)jw%@i3$5dlS>#VGH+p%kX~c<+*2y`C%T!&}@!ITzN?loGX>9waB`Inz<;xJZxHT zx2L>&OxB`(iGRp&##*52YG9NTLyLylY7Wt_UCZxrhnl@gOgk-|R6XnG4l)WhMdUDM zsvU1!^ulX76KO*tNvn}X*2X%MGfk&`E=8V(n8@>JHMel19*A$NZe87f45`EPf-CSH zUyD&eMe4xz>`moiGpN$s2+59s_UR$owMNe`gv9|K zsUcb7enqy56Y-z_c71vAGyfD5+g~@7x`grfT)hwz{mH}Je?DGiibhNwTiDD|RrRQ< zbKhHJG>f85?eXwRuQ)NURB>M$>|*ZQsWkJVqn&AXSG!g={rnVsxu!N+I=@1iN;O&0 zx2Az{`R%skg<+<9ow{Qj-euM=zg3KJt;?3=x(GuM#DD*Twou!m?2NF!Vz##K(@daB zt}UGfB%pV(l`5PdA`@ph$8=Av=I1^)Y%w}C$&N>PcQ%W2In=&qVA;iQft_`7`N1Et zxF@d{*wJtGs$Vieu7raE5@6oW1)HZKTHH+sQ}J2m#jrkyJQ23>A=~9a(AR`Ln z%?GfK*|6-c!;NYRu*;r@I*2uKvx`=X6S|FJksq&{HWMz&Mzi7g>_Yz@ms8PvsVg&4 zh`!bDQ`GfVM&@xr0}Rmr8XioqCtU7XGzzh&jfxYWG`&Bh^KTK)QNtb2A`lIir*&V-kwYzmKeB)oJel(Zi^g3(_85{LBserGmJRAmNHb38mk&xgBs#m*mJ@&pwY$bxSerz66rmQ{PnD z+|e-4Y7>{aILy90NF^UQml^9zE{+m@WfRWN$Sa9OoZi#+V#PxyQj6Lf@}V#TI_#GF zA%ujrFp6W)$zl@N=$#Hfu)1k7UY&)5|BQk(oq1nwvtvUGxGxFZO+qs;-m&>gtv;Bd%rlWBZKF{&FOP2vs)DYOkZB#M* z&jXx(7`9S;D%)ghNIx0pvO50RQ%*aZ8&hMkr{mw92dHP=59VS?y*y1-Gf{<~0WTL* z3qDsXcSGoP^JmYK5c`ge7H1BKt@S3-_$6NZqoWQmgcIq_J*=*9Ii8~&N07k zI;gPao_aK3^# zkm4F;%EdqV&QO8c=(rT*BKf7jzZHCPJzwo0gfze)6(TmyZ`2-w@f56(N$ZK*Moe|W zr?77x#CBMBoQq#BHKA=2&NsgPOd9yIL*HFckTz~d<~5q&hsC7PCgTOk&V`$<(f0Oc z8o>ka?c6rBa6V(@uWbe!h(Fc#?TQ-b*c;!ffWI95ySSc@D|P&UNRqy(5r9RFn>~$= zhV<5z#%T^4Cy>i^3$wU!ud#iJ#DSmUB%;OTMt&+m+Fx@Cf)w^lIc%n4Dln=zAQHf& zZnawMr-x0atz>s`^xuWqxGYXNcs&5j&)%pW+ z3~q1`hW+v2eAkW=G|{tZd%SR0o2^#h^4~7^+1Z}d3LV@h-1LMhVV0oeuf;iBe)N9d z#Mh5VHuCbELevhkNw3U&IkE~|PUBs2H==vLVV^2s4iXvqe`=cqdH8*a<~|NcGzKI zU+gWUVIs~sK}tiqQ(Do0{?gNIGD5O@MKPgT&t*qr62rNqy5O%MQ82-~pmvdc>0!wD zEg#-q2T`*VFI~AjBwwY)|D&!gaCqNgJ9@%CF{yys{-LzVw-sV;R)UzdLtqS3d28R?P%wuCWdn%b<0WUhFpUq;eiT|8cBA4fcg!7WzwIbf;JyRC- zyX5WjlSk{gZ&l1|E@xFPNufNu_dnhZ9C!sfG+@j5ZFP_j%(hvOzq_#g3WPV1RONUa z5%%)ykuVkM70a=~1XTBb(Bo0;qS9TywuA2p=yyWw8EY$-=cyJ8l}*nDmVHvr7DiI- zm{o2i3n9qn$Mk_OVrM4>tC>Hu|N54>nQ6*D?9Yej)U;J?SXb6%?rR#yfPF>JGa9zR zeYt=zZD34MPYB1oqK}SI?G;WF8OfJ>tb7%lDI3?F@557(ZTt=dIw4q{!<%-F^=$T% zLQCz1*zi(YR|5oAG^ns%;M7^e5Oxqc=8i6_ce-gmqf7jp>kM$78JEUo2 zAn++-8F&73{ca~!y5s%vtpWx=Ysyrq7G1<1Io$BYFw7n6F1#-jsbg0%u5XJv)#4%_ z;dvP47J4mPZBL8T^@{$$Fg5f_LF+AClgq`JOCjp32k%zULvz%JR&T#fZ{4L|zpF0( z0k1_WPoV)HHp(9TBu$f-?_n@eTP-(2y926xm+I+7oJQ)pXL^6fHKQf3#NT=_%N&#C zZ^nOmxz`BJKedZq%(aPL9-pITrQ`I>JJJD7`F2+~VezfwrkOJyFPl@nQB}N?|LoXT zXlUEIT(RGhFwp5NW%e}1Kr-hQS#04K7k_1)(rAnvs*FH8e(B1h_d$`2nZble7-);l z&waGY>8Y7IE`O0Eg#g&$qw4+>J5>(xp5GHG>`U+a512~WYCPM{ zpE+5g_#F?7tlL$V+lG_HQ#hx1I1b}xU7Al=E+;#azukVx#p$UZI_k_EfEVgYrOHN< zwwWSZ|M|Q>NkbMyoza+A3fO^0i}MZ((}oEw5AhW;!LwkQ|3OaqUM~}ZAWrIt*jVl? z7!F5_d6NGKX+m*q0hq~!o~n#{j+`3ZsN~?#L;Phr4_LqY^HlM4;0aTE^b4a)((L6~ zo&S54h%5*8eJjZa`ed2a!m!f=f@*Hs@V+QAwpb|lJNe>Wgk|P+ndlllx*O8SaPU|H zmDKhH^;>g$4!mAi@K5J0;PCw!-ZmzOw&)BAoKV< zFJthjLhdcBRe{e_`5;?8*5$qNFPtPaade;S(_D8wxyNu~bF8dw@1G8szcTD{!q7DL zct&i_yWAph{5!4XP$dvoFZG~+^lx#NdsPphb+`xqkt~s*rxPyB9{r5lyml}60j9(! zlw&6svF}3C>syfdX7`XAD!5m7UrRqbG$2!TAKI1cn^a66d-&L#%N$6Fqs5V(t?&Ia z2r}FX$r8nMvJa(jDO~Lo2F}WtCF}376-YnXW-&otD7xr5q05&sMX5=CCv7^4! zzu_nUZXRC2EDLwiTzz&b#NefU8KUG9jTi%g0NSs1ZrO!+W&AqiBG1Cm{4()rXi&Wl(%~c6xoF$XNTp`v-ri&7$ zH9H%Pdey5;W|G0<=#CuohN+(4h^@Gr_qasVHta46@|g7Jf?jVoqO(@ABoz)+z@Edw zpJ7i-DreqzuYBO@t{K|DKK=Uyq*p6S8p0p!>il%Ke}5a7k+Avk&S!@q7)5-a+u^yc zG5MG(J3-51Z&|4Sn=HLlc5qSFt!(hwAwjtUYD@TZyMnbAhT{KQoJ_6kqYBLyjb9yM z*2A-!ovb=2+lKK7OpM!ae5IOxEWqXFuRf3WwPt2AHAfK=V~YC1^FJ=O@U#d1toBn^ zqV4GU(Z4_Sn|;q$BMkr;Cs);s=@0c47P9(EJ{>#h)hV39D4RFnu{FGE;+wsEi$mM< zmzdcT_FL{|-dnc%(c7#p(qFLJMr_u7FVG(cz7G?HdG@oRPU5>!a?86nb=f-J7!SAB ztI8lg(-y0`Z0_din(u$AyV1?Wx;8`q^nL;i?1$&EFY0!PkbeDvsX|!{R@bi7nJk>? zJXX2J4BdSfU))h>TXO_l%G5v5NLeH-To?Pb-_}`|?T8+iIeL(mD`!G_pu=A2CZMzE zTG_upru)uSl-4P(A}jC;s;DJ!s_=ZSX)P4k;MmD|z=miuF~xd|1C)lhA=b{B+UGn) z(UB-7rsnCJ*&4fwdQ8m~bl+BER0q~ghTNZMU3SGv>Lee;Ynryc^cyl8y^E{b%w!)w z;50Y6u7b-_kUgkfXNEUDHTGpd>?a*&NJh}lp^OP` zGG;Wb6+hDLNrh8f;Ia4F3uqI+`a+S$qO-=n#tlepRh6C5k8uOL z?5M^o34fEUm+l6w?OZf8LBNA4?~x^(0aCjoE_c!v`+nY^5e#(Mx3`}CraiziNtr0M zTA;!S-MMa4{#>OOOHpb2sM?SF8)IBRJqn~Fvb`P`yb`ceGB**wWun;e)IcQ{E3{i# z^Z!}^i}lMZzjY}OYej+7&>WM!in{Mo&``qa0z>DShTl(J;V}MWtnCsF)9+Z=RKDA2 zmgAHV-4<_Ds`pcJ6ftK=4U8M`9R?}+xTS}PtLh*74TGNpVNm^R%1W8X&|Jhsuf9#j zOO=-uuA3PA&wn4jZ}=G)h$1P_t?Hv=pAHe#;Xl2iPBYs$}-W-&|V+Z zo;_G_J#O>=!<4VRKDVNTAV|f2nvn)SHs^Cv3R(H>u$Yn&O*0N@B90z@i4EZve~Nn3 zVyY^$n00JpU<3|;DSYqj7IK8O-vh^p%SY9t8%SNoj+FY(QrEWvct6W+gU5asi9`MFusAR)xqMYRERr5$?mbJoCG+-jH=uX?twigYcT`%H>ePUMN4E-`j<=tk!(&b!C;6yGWqfQFx)% zr$S$rT)hOPuDs8MthB;UYdc5~m% zu3L6p#sy)DC$RLJoWEp6LBlIPtwkNRba!;foGn_USOkJQ^`aLJ>rosVvRMe; zq1g*wzJ(%XA_!kz$z51jNP~8(@KL->MtAlb7|loD($;1XroGR#p9C` z*e){bF+%XM#5TZ#)nbWJ+MaAc9%@kMPiwGTarGVOC;NKU*zq%mJLCwf{<^35u|vT% zx{|}=mAJCGH}=;!C48>%5!aT##hQ1?%A96Z1&=YxlgNV8n7O}K5#XOEp2Cwr>T!Sa zoE0v)bfPJUyJ?MbJu^0F*Yj3?G|9>_--dsB^bG{`tP&0d z(=EOzl6#1@xt~HWvEtb$o!D7ZI)TpCKSg;%Xwc2ZhgqErZKQFns+cXb( zRSdBXYoym3#ekoivyZ5RjE) zK+MyHpws)X!dmneZ={?q)M9ON`5MuA_FL0w^NEc}`T{Xf5+nCH{0h3d@7s)jYrVYL zE4W-0`z0AFR{b4O9v$LccNkmRp>j$9c^P#AthWqC6rdG1-s&>a5PuS3HFai?fk@;I+uM4>TWssu${9@FwN6Bi`~uIWEprNHGl@yya4 zz#q*}D|cfxm+Kte50{^4F(rJgi)d-3s`VyNh!pEWaAQ&mQ_Z=4ZyNoX@*E%Mt*}S_ zH%qxNC*l2URYrpA(;kK@s8iqQu_0;kSyzJ#b$rLGL#cT^W^tYDK+3v-v1;hN7`Ohy#f=J7?j<c>p;l}xvHwP;kj_JviR^f zah)du@04XD=%T0p*rk|&!pC7Ww$ANZDHRsP7fEXSl;YL_$6Bxks13(Wr1oc7Sx(<> z9doy7KW2pi;S9i@5PbawHw)t{9Fa^n@iOiK#n z*G-Z2wqUhAaTvwX#LL&1zMWNNia}vHkB`HQ=wM6n{|Z%O)uM+LuZXp@HAvm!N5m12i+-*vCX@D8Z7I}2R(EPWE z6T5Of4|PJym3xU~IKWhIA|FvsH#DJncxQ~`<-89D&jHh7n3ZS42=?JOKo|66U+JJ8 z#QHvpGFmty2L;NN$8LLJV+8bTXc zir;Q1^ZXbI*#-}Vx|Z+tup52@lMb9}1jkK;CUhz)IM1YENaZaKFbUa%-yF3;DMijw zAU78eU*7~bLJw<)Zdq5&1f!@At_dg>P;{Tk92AL009++n<|a-lD`j_)$Q6@VVPgBJ)jiYa5DUAP(!i zwyqI_6NJ3$F|_cO%1H(OjTGOx&-FnWf?a+mL>GAYEj31MhJAf&Bm!bend~PW1-Tg+ z$)Bv=6e;|3ZpEYPTaCe!Y*iF2TmHpjLu*^V{jYqP>}0|X!5Ce=g)|N4*~s}TcbWLJ zeWBu4?Uh5IpB1l*V!l0Fesc~Z6`b@cyT=*ZG)w( zJW6nrf4)L>ILDKLOTbANY$TAt34Z0e-17}ooo)|HNVaLJ>q#HEtK|m(1CDNc8QOjXglzrF-&*f;cIW62NH?qomF~Jh6-B>1{ zpM8v`Jdj4IeBOh8+H(^LD_u|d`)b;IIrLWcde!h{*|(gTaLoLkvE?Ch4#rm~viCuC zsX0!6|K8260#W=gXPdM)D$KhBA+ViYbt#s%ug6KHZO}*Ap|?uOrM4qhC zX?%d42CT4Ry$<$+m6FNQ{3=><&>v&xr2NZMSWHUKs^qj47GO?XMygP&Y>_u(_ z{6@__%Ol$u+h@vY0W}wJDLT;kMXOGH0@n&Av;s_bq}a3;S3EIFzc{T26PY@OJCU_EZEq$=Ol}`MN{mvBQt%ufTc+L;t%;<) z=D%*W37Q#@Jn5_SghB+g&iTZZeYr4t`z+$KnM;W-Rloa}f!3c&6;O-histcScB{Ok z$`GmLlBE|^zI*m+sH5>b$pLMwTE=>w3pK^bnJ= z%Gy#i6X>&SaHF+~mB_LlHfBV=t^7yPpy0kTSkK+ZBenA_x)|!(@&fTC!_SHRND8^q z6ZBA$e%y5kYTuV#Z0EcuYU_@w9e-m}=+c;#;7GA7WSzI;b8my3{HYH&Km$rMr(;5& zRrQPGaLl+LALuaRHT)SCqT9jCIsd%b^ov{i z*N?Kz*FniF$t}c+4Q(2p@hY?kh1O(Mj>v4=^#~R(kROW)Ci=&(auH;};(#_Mw>-LVl{(l#GoZ0ih0UO}G0*}K%iPNr9xOS{K}6G5d& zdvi0Ll82SiHpu%LANPdrBa(!We|e`5Z{3g||GP43jp?fWC5E*ATxZ1bCidO_ z*a{C{;FnKLmTl#FGrA-e2q@-rnHPf~T?5mKuWSy}L2rKAXT74x5)e0%+>Md0tJE_?_6%6<2KmZ>WE9D@(W?j^FLNg6R07ETfy>SU=j3{iDH7A!-F+5c$ovaa-iag_eMM+s1}fi~jOR>XnEMYmF<4ToGpcqMbqGbCBSE7B%2HzG9 zllN=z+K98je5J;?y&bOUs<8{48*O#81xPy?uioS$-t}X%aw=nv?Z{b(t?e+nYz!#%QwL`hPKUe{#;dY+tm#_dyrL9C^{4Fsc<=_>JY|FiXh(Uggf5^8n zK99j4%WCu^;6nI*T|qvTNU#EpI9#?)VO(ujQtZDUxI!R+&7Hz3!V}=Y6ckb`<-noH zp}4`{!r>$)+w{@n!Aytel*G=u_#PcegWq=xbg-Z=VQ)W`EvBQl3+}z1#H_%s&RlMJ{^tvG zM!?m4qL70=m=9Owu=u1?=kbI9BVqj_eLk)GOpslcu->l|v?=k(g}Li~^UeKGw;Sv4 zWRj|!q@ISnSihazVOQp=#u{ty?!a37`x!J(y&q zP5|0tFz(j1;KpK1k{SD+PuuWk2)emG?3S2jD*w#cSEdO3o3GKKXvzsk(wD!n%8gpX z>vsG?h|C@)K8%F*t+p7~Z{rG7;?pb#zAbBv>=q(cS5J&q=~$LAcRk*PiO80o zcb9im|Kh$rctModrxll>k5%6gT{7}n)`kbKq`UV2Xk7(9U}s7)51yhifm(^5mbIneSWdqlF)Kn zY*{PB26=+9m&pb9L3`Eaq8)I-96COu)f!=a98ItQ@mBm~$)N;ds^f?%JNOM3TyeDIXr{@X=n8P6rN(Yim>w7 z6o$7neeK-Y=r9xzDg{uuB7F=0uB~@*%}yG-luT$rw7=!UQv4{90n9~Tt?-h1w2ly9 zLDhhc1-XlL{q;v2Lt%;%=x9Oit{pkH?za2yDCKIUXR`WIdaHvQyQa)x*og-wQ2e_N zdFyCPW|w_C$$Wa?IMW>$Ho8eCd4-AYd_k8-zQ^UWE5 z_m=d2Odz4Fl@`Hu#a}?H_@2}q;@4W&rzw>Tr>g}%XA>2Fc88d@R@HJXW7~ml{c%k? zrhA>AG_+!8tpdQS0B5M4G*+U&W1u@HE3%RYCIWDpI#i;eu$eFw(l9%S0qrKfD_D-R zUbv$w+NZjck&QPl`Ln7q-!&9GqNeFX)_?5%5nwxPSKA3wESTT?6EWus1l zw?M7vr^-{rxV!%6SiBC%3ytPgX;Iz>@Y1Xlwtif1A<6wF85<86Yq7!UtJwf)Z6Q}jN7;3oQC75vRl zSwSL(F09V^9PvL1IJ8PO!aYtso3l@AISszrX{zAvDz{CsoWlsPo0pCLL!RUv@llyN zy&1$W@Sdlb-_}ZvYb9eTbgiEW0rOn*WQp85k}M^++CwdMX*FeHYV zDs(`+oNu z!bp@rl-X0ZtxADdHRTZWXSQMK#l#11WElmW(J%=pWuH;c%XBVZ%Y36=RQrR0@CR=EPD4wse+BX2kv>g9y4!( z-R#qO1m&Bv^if)cpg%I0{tGdRgfZ#LFF;EJ zq~9*BpzI>NbWOhVqj_BxCp+(A)akbrKH2xYl;e>fArB&Sidi{IJU@uL!bU@%g^q$1 zX{MGvSmviiRG6O~5^n^K!aJV7eKfjmV$wgQ0`Hge&7^Hoz_+KpXadkni-m$omKm24 zi&A5IcltH2#IN^-J?B0Qy{U>sG`e>^EjA}^04?@e=ytjWEvmEZP>vRzV6KbtH%_Xu zs3ezBCQ_rxV)zXVZ^#XL1I%OVNS~VD48A}6_Y^!gY)a-N^$7sU_?jWngWM230r*Iy zJH>7PlACUTsFTKI<+d&waUTwUCq=&K(yQ@CE=#h#mX7{+0lq=88$Y>BcJROUUe-1t zBatr_k2xI|YN@!b>BT?5zibnZRs#4_-Bv3bv6DXtxRV8UdS$4Sj4?9rW;ln=zYvt& z`ui)RGknnC>+#PjK7`s4b$0DuRxX6-RuUBS%R=IFn`*HCd1Z{{v=#mQqxEK9=>B@C zCIPGLlDW{UQTd6JGwb|`3<(0E8&3=*peiBNZj}n9DMuu=-xDQ-yg6^7l|5z!g z_k)B-FQbG0dd@7?87=*6a~IRUK)xW>o#G$GJv?4DYJ`1Y&~8HC*t|LBujS+MO3fq6 z30V=Lt=6B&W0N$CjMQS&$$QwBy4vK*=N{C?vca;!n)-wgd;{!4BQG;XyZ0m_c-qr? z($q(%eoOU~ExVwa!wTRa{ z#Cj-bWm3)<*hILGpAhl7ozn-qm)0E)nL6D_wf>ukJ+EGTNKYHj(n#Dhe^(UE$d!n zi*+e~=NcrybLw^%Y$rktmj?MPMgi5>8!Vlo-&u=6;W!Mdc!eNNpjAC;PKx%LY+JmV zC{yMj`!ROcu|SgRx;dv~=|gLtTt_-(_9Vx7r|rY57mUG3Xv@P|dI~fbHJ8QwUIROg z_)_b)qSIQt#XA67=;oXEA6EW8XFfh5FF-*(4IS29P|Fq`6q@+hq1`D=e+Cqh-0~3J zMvDdM9{V9v#qCF^p46&35MNz z`5e_J=d-rEUsr57{m${f;_-WIgOzk7kDvVty4Pr+@7xX9l84n6`Sbnl{HzbG&qLp$ z{tR7EC+TfjHufARS&B7*?~$iz-Y2WpbkuNIJ!Lp9i(-jbXd^A%a~A1h`{%)CHFP0`s( zYih@F2;6LGV}WQVQ%_y@XPKX=7_dQQ{RpFR2pY@eOdltYi-Z2^96%{%`JG#LF|$sG zUy15O$?6fg3J&gW=(?UHf;scniZO3#_Rc%ScE^r;F0YvrY}>uR0v23MnqL2&N*$A3 zT26G^b0pnDLY8%F_(dz+$C2W2Vi%lLID&i znRFha);XS5^isL-Nr-of`x*xhIdr&-{J+R;j$ytrfnx*SbXpC~u zLmz3GIx?iESF@S4uR+uoHH%g0RhMstY<#Y%ZEX8!ZwhMc$oc8^0mDan5fRlUvcbLd z9aa>f^$wBS9heH&K_cvhA%G-rUs_od!v7!vWunn8(ZKNBuQIhVX*R93*7vG>T($Ns ziWnM&I}ccMIg+SY?31=L9DjpD13<%Gpn7Fz$LGmEPOjxYc>@BPBg$7fx6o{azxI#% z7TRg9c!3h==>^nd0{Vi2Wcf_ZyBF9x|Gbs#F=@`dYkjYc$%mbxc=P8gg9v5iR^4`U zFx+V3*QDtOAco^|BG@t`W*p9T1Rwz@*}f!eN~gcKU{2LTcH{oL#~1bFlcj+T&o>06 zk)tJHuK06g0)Avr`IM#Q>!DBoQEf8Qv>Ejun z&0tT(tPWCIf6vz`9iv4HetP)s&A8XcMM0}u{JX91sJ}WQ|D&%eS*V3wUK~R%o;QMg zPkSuP86c7piHq}xfBE(WWMkMSsLQ&0D0Iag<=MVhMvu>KlN&+s+w_hH(OP#@%3lm0 zK|dIvtW=kEA%yDF7>kd1?6t%v;dKW*hwg_NCdxcWrRaoS(58UX0#jQ~8*QD#-0Eb` zK?)%``Ca)p0=G`H=_x)S+1&GUDHZ*U*)fYo;*;<|BW0bo&Uqf1M|*9PG)5NhES2@2P8yX+-CyCuxBN8k*o_OKgHZ!29w}QtTm8O# zC{66rU_+c-Tdk}Qm!%4Ub)VSirz2PfPG^gT5kx8Bj6E&o^S?!0$_#7Yq+WKCVTUAt zsf@~}`%b0{X{0RhHTqck222cx^X|tiCQSJ`6NX*~N!xg(tA?fcQ3wZ72CiYfdB^eD zfF$F0K21iSIR)#t{EbJn+$e+hYqUES-0C4bwc%%tcd;Mmn)%qTSQkqgo9zh02S=B+ zEnbOZx&WmdQGXYnV`|{2K%A^ZYoY_k_7Cu?IzGjDKzJc;92dl)FCo)SCG2#Y>Z66l zR`Y$13py8!0ZNaN770h^t^H~g20)bAbt>ysC)fY8&T<_B6wqUs7rx?XqQ>QgR*%_^ z9Uh-rt1|C;fGrMxjxFdUS~`>G#;@Usi8Au`?tj?omFc1f4Oh=R!l(AM%zg0vHggVe zUa0$(BSy;JyNwJVNV4GTL=|Ko0TbzEV16CTU8W!tT42xaUPJmDMH60$tZi?p_QE}4 zgiG2f%1gvcBc<`gN*;_)aAAdY(zdEpCin5$nO}rhmGxsRL?6>T_NTtkPnm5m-6XBk z+u0Ee!%sk409&#a&>{GSa1cNUopR!aTHLSmK?LXX$4$}^iX*N7 zr^{#hXV-nj9gJ=61(utaH40P{MiuX)O}p*stBy)>Z{G|-ysk$pL13-YZehdR|Mt70 zWSDR;=;K(tqXD`=W8*_n<>j)YtBcV-Hi7D)Lw#@MT)9UaPH|RV9E^RFu4aF}1+*!N zJa^*HSbebya^kizW8pFa*f8Ua;n#u^+r}xdKiV}g=?UN2pNnD`PWU#1AL2(eKg;oF z1Tv(AyHlDv-Db1Wl_JJ;6b6+rL~4h(@U{FNdhE~=9|N`b2ny)a#QfJ8byViSZnTD! z0W~TNG6R1A^hQ9Blx;;8nRR~&5dH2$lqa@@*?lXz;nm{h+`OYeg; z)H#5qbZ&&KKVZs8Qp7(ray0R{Ccw?beQY7Uou%s;$Vv4p{5EIFnr<^I=mh`u1~xk6 zQD<=2BjJS*kGc-3Ak=kGF$jeOmW=6NCn{^Wvv|*oX0EdPB+VnwK;|Gf7AWDi;r7dG z&hK^6Mh`F&H&y1Zs&}s4O0o8GAnPx`q7-{nN$xSwbvxd26mv1M;2 zT^yf$CCWQduLEx5Xzgf;EYX$({ShIXkoRestW6~%dlzLv?UN6c+rsszVNMSQnzb)I zA3S7yWOiV4Xz6ill~r*|ggeDb<{kB@h0`Lg{>s4bu;{LI!%)ZlXOQ%^gZ4*k7kxaGq($bJ`k4HC zwF}(J2YlucY|^dcKc2)_P<)vmX~+Pd{FCm$*LLHN+~;&ABy6BZhy2kZfX0ed~Z4 z1VL{Q(ULRAB^gR3VFjWytHY_r$?bH}lJ3i+e}+%#4EF04Y8j#@{p|E^iVw%zu5ry^ z+8UkLm+n8BZsTV9qtN>uEHU$RylBiBDY}O)C}6I1z>wm#`s7KbiqxwVDj97@G#Jsd zN7m85QjKFKGJa(s#-3xK#JEzANP z94^RKy$h;VeLiM_j_0hqCk?&MJ|1TcZdv6^tu$r?tTs^%AQ=R;E(*Olbu*JQo$9_% za`7C#w|{F@AP1T7zQ&qfY?cy%&*M6uNgn8@qyFWBX-2lR;OFSfeoZ3%@#g0HVP56l ziq8g$u!@gsvhH}S^XCpYU^QnJ-_5^%*SsNYfBTV3`i{~IurFI`AZM{EpelY?Tc^muU7Co8_`ouC&98u(zs&I67B9%CoQuZyhyqO+J>u5+PPT{JA>bKjA%gYfIu^rfwVqD1rV=(G=o2}Jm5Lk zh4HmQ8KQQ)m+sTLeAoMR?Dy(2X9OPFug}TMQcN#DX%f~qED2JBDxQ{izetE3jA^@} zS&y`U{$d@-b`}M1tpf|WmFD8TqG@UT;q^})ma?dn3sV*s9H?USEOfFN-iFxfgDS8e zSY`I!?^h_1%cUITUM^l&tJ4QVdB0|utPant0~D4vsMsnLNZ9^NPEu~9unN{zehnLP zRd?j7Ki1{&flU?q;Oc2tNW60o@K>05z80R(4=?Q3&B>XXudzxnd|jgLhfrDnQ6MhJ zeT2CQr4`=G<(tBL3D8Y5mfcHH%v}yq=i2!cyNgYAwihR0N*| zY7ANG9(R3v{Si!SP`oQao{eH4UUt2H^)|Zt)@zu=ew&m2`(n&zF}gj+r|7Gu=u{1 zyVz8hB{+*3?a5>Uzw6>&k7RefLc-kWpYdoz!7IEV9{ z!}tB~#&_@ixWAig#jAdyCCBNY?JXYQQo8)qG1;mHlWz8lf+;tvsa|D59`(p7aMCk| zRf1xD2&?)@_+HA@V8RmY@)#2av5bD! zr^cD~21%Jpu$BGfBUj;y2PRUF@_LN1 zvOW8&5QIs)a{`R0cHooOvP-U@XzKo2J(vmXJ*keo_LXXIwItyZGsOUWC4nU180KR* z$EJ4a8753eTwl$YmG|MX$@-dtaI$6e*!hTs1r~LpcCs|ArP!frMr-6JPH@=7w&v>6 z8ihKPueg}^x@MMz*uB<{{J50!H_Ih91ncXK;cS zmq*$`U#Bf9V*ivuQ?DDu`&{|*Az(xO-X`w&(?ER$;_cD}IKtoe=ADRCGw-(quwF-$ zJgvv)P49)cp3z*5Mlt~t;gE4t^a@HUMmf1vq6gg@H)pH>?Nrl`I6-3&6D zLwM+l9v8-jB&}paBWUl6Jo4g5GyeliD@H^hjakgi^$4|C!$7&LrrnM+zik z9!kg^&vfaq;yk5g1cT_d4Wj9H+bE7fqFlcWpB~Mvtrii}6EjmmYM`8a+-%9rgoru7 zTV99HWW;)ngR@%`XY?$Vi*syi zatyid6r+|Wh9V!2O^=+R4VHi@CE&r+l-#<98U>3V5XatG-h+%mZ`e*XkF`t%-GSAx0uo@TYsMvEf4JQ%FQW77{7liY}Pt2 z4n`e}#|9`P-7Ovyr?8wn#Og8|&Pfi#v=M(S_qV3>JQZ0ye|gZydDn&=gqs5fa(=2H z+fV>6=*_Ne5Eo&KcjR$R!veE&nYkV7II(5Ch|URJ1e%T=Yl6l+D0m#L-3-8EVLeUI z=4?$y3CG$buLY`#2pj*!OD0{^@aIR_BH9^v4#ywmqF?~O4uksWSPEh=@a2c8a-7<5 zn5rew6h`+6mCEIoQl4cG)FIo<8lN+3Pir!!W+|7;m2)e?WpES`OBp7ew`@5jSD)P(0+iv zEwpd1#15}Gdg~7Pb|=nQ z@4K{N-F39bTu$PCpeQ>xd{BJ<)NQynJGrcxkFAg) z#r{*@lweWAF_=aSlhs1KcD0$_)1)GY>53ovY3VRxpXAPR)!IDJZkAvGzN zM@h&wjC`D-!T#mLDY--)W_;i`uyCe$b>hM6rC)TAsy*hiVCEC9-ei3j`@|AhG+e@j zvyGIa*9H}~ZIz%LFyeA!z#ZhiJFNqw=(39Bm$V%!ArILq>}=mq@zYdVz%w(GsuX!i zXx%${U|_?~E*pNhXZCCLYa|o5vtJI{77pz=eyq%G{8?LNvE_jSh10*?nb67d~y)h;A_bPO=g6bpqJ{Y_PKhvIznwf16Wu&?{ z8E<-h$=$cQU1C{A)h|NHqB+EV-n>tDV}z6twF-KY@Ibil6X?}xD@0)H57EU% zK1s*w!?IH0ISJN^+t!o{OUC*_G`2)Vxpcmk#q!w{%DR-LUf9vN0GcQOgfpAhgReS& zIs=e%%GKu$?Q%NA(RM!zAG?xOVEdLcq6rg^4}>=k;UD7cP&C?@soVvR!i+6TpM61b z_2!Xa{(z{gY){)6c~=eWWu5~$c5JnOwqOc!yuJ(}Gj^$krRdS9*GgSS12&Biydftw zt<dyh@AB}$#5|1dTDg=Wr8t++E(=yfynLd1BH`||{fkmqa=(jDGKu{JWXcGqF z-|zjRh!w!tYW)ZM2St|gxzHRdlO|Y2*An zW|&zDW}S)lw^%@bc#1A^BOw>KwThzZS5U$oo@r-4X2!hCGiMtoSQdIt*ma4| z=#TP#EtHMN;s9^eKPU*7LnD_eKm{=|5%6#{N6Wdg>4`;WSS7uozJUM_hmEZzWJDP_ zX%zLL>j#((C3fzWMcpfvl;TNNkXd)J7?WUE>NWcqRY~_6pAP&0_o1{HK(8CLle*vO zvMVEcPB!V^^?{;Ap0OIx<<1h;$|_0QkkY^{sl`N5n2zF?b+c@^8Y8!EH)F7thn~rv zel9Yt5|!{Vp|chOX65)!;uDlXLS7v7D0R#vTQ>4{3Qq9_(+w=M&3bY=a0^;s!%>0k75cvtdcy5z^<{S?H0w zhbWj(?SzcVvc8&-SO(B_vDR%Nh9Or4QOHn1-{-J#NZ#-YE6CImAH-hI_e76Rj_!>r zavKkSOh7ES<0l5#&jVhEW`()LVE2N7h=4_+%&}v<8BTt^XZBuiBnGOOIq;ja8n!;( z)A`QgAq+HVeAjd!l{>IX$vp&~t5cd5G=g0>b%!6Xoe}Dd)`>Z|*#-Q)+)^M4*n~;m z7{|lj=fyMKrZVpl#4=xz^cUA!rfPtkpKK`ulAp)&6nc8jh(ZEIAZz?VS2=buHbl3ZhRbPQm45&3@hU2Qc7!cD~S;M77Pw$Wv~c~i~4?|vpAE5 zjcpMj7J6r}E*-#nP#UYUwrTKtvn*TPar`!Hik`E0ekNvP9X{C=sok6j>0Y_uhFA*% zf@0(n|GNt0hM*03=)u!9v(Lp)iX$>_@I#2Umw0EN!7@+_dOYlD_b59ZH5m}Y91CSs zJ^_xT)~_X0e)?D!!EWYmSRkhd-Kliu^EhOGWCrPiCqkV`&Ba53J`gkr78i@yZFQnwF4CR>N(Pp+Pi}MT05Ol%kcN zmmG0=tW5G+(Yzl@0Yf8Cq{MyCIs$i$L&3;!Y)GJ)Wte?7WVQhpzTV7?zlS4OFL~-z z0V{$IkUc#IZ^MG!ZoPz5HGfqDH9+#^%k?__-U0c{IdLdn}YSk-?!4 z;&`^N6{%jFh0`?`7wk9-I*bb=luk&|tqKSB`YN}{a^Er34BShn#E}$V*-eIMB`g5k z?j++N>AfZB0KHz}|K$dM^vudKTa7>%tBzto9{p$EkIt2uAHNr~{o9a|-A z&lVy!MuhlO`?$&&4!NwU6k6I@tWC43wdL*7-l?*>t;S|!@nHgRKV#gEyr81N(V8!FxnJJ zdeKKc!g2+zmzlIh#OZ7|l1nU;U|szeMH!HgC6neJUGm1qF>Dj&xiicJA(vke?Tp`T z{Yifc5>j6$mfnxg3ELcvp|PhnLl1AN8BNHQ%WGA5UNXte59#-K5gZs~+I|p5j?6nE z;TusfoRt%j86lH zA2Ic=v#hh12O3RXA#P!~Z;+pLE+VNBwwJmLCJX3cMqp5iW`d5M-VV<>@A$+(bUmR7;OT-#iALg)Y;SZmz%@ z1N2gx<-9}$|3X?0a@+Eohh|Eq2( z-1vv8^j9seM=CuywKzOJ8eP80#v_E5{U=!V#>2aNvqg$;L$yimw_N1@(1SERy`l0g zs;CM1sFD8|6NNVRKeFbf-MFf&4*5eXG!o;tVa{MH8UdUTs5Y}_k* zc#PD3(0O&c?oT>3^5brKch?dY8g>@x5_6r=Rk;V6mF`v&L+g5-EI=SB3A;0=E>5^0 zpNL_qoNwghm-mUa8{QxD_4KiHtuPO4&&to414H#3tttF6ZNa4j0LF@%4=ENOX>PpW zy*af*WxZ+Lv?GAXX+(ZLwR~V+Iq4dRZiu-<0}@g#E#)BN*UFH5q;>V(qN(|!7<~_1f9`Is&HKMVG}Hm`1|7QVL3mxFPC6`E_-#@nNz3NBMA$X%;(viuSTXAl_W- zzfqU2lJen$1N$^3-En#NwX36W!*Jk{)ZE(((K+KEoxi=^{@Q8T_|1}kX~slafYtj# zhOSq9WAd#5fB)>u7fC;RM#C{QIvd@n=DYAh!~wGnb$d%c<;ArS{V3_U<);Kp#_H2X zeIP(i_yWesuD_pQb~Q9 zd(xLm(N6Osx#w1Z>$phUe6uOv{adDa?!ZPDdzG(>O#_BXFYSTaXK3f_3EKLJOUzZ2}{UF7VgzD%ek2D`pF5V~(*s@QWbd{reyB(gELTo-2wW1cj<5BfhR1W5(& zs^@5_|8?29YV%jn@Y(LxeKJ?RW)GCr_iC=P!f!5ApJ6(F3f)mh^tIYlJ(O*C zmns>|w3Y<@kJx}P2lhz31{JGa+3Tw(wpiS*xQ4tUkpYp7Pk}%dvTcI|)fyVS;JusN zcR7MMjg+ajBcECrcUP*Ihs!(TaR|l#lipW0CtsxeB%yi*VP^x$u~S08B=+*3l#xQJ zXww2R19y}-bz3#%^DPyFs>%;8Vx8WRk-ULavn_W2J1`*sYSAZkBub~E7N3tEdzLtE zUjfGLq5K(rjyQ$1kxK3>?bHKo3MkT6?HcGzP)@a}Uk_K)822Dz&lWGW cKx9xLrP!KZvH~~JAmC+Z<8-F-^wmH91M}JpasU7T diff --git a/docs/img/streaming-arch.png b/docs/img/streaming-arch.png index ac35f1d34cf3d0c94d039b16f2ab8aa0cedefabd..f86fb6bcbbdd8c00232b7dc8278fbd700440e979 100644 GIT binary patch literal 51972 zcmeFY^;gt!*Dh?(iZlpF*8n0Nk`6sX4MVqp(nCmtfR3Pa3?ZE}bax0yixM)FfQZtK zbi@0_`?;U@{0(QFA9T5JvFEe*zV@|q!eE*z#6)yNw{G1chNvpSZ{5O4xOMAxJpmr@ z6Yis*bGL5Y{cEqF0D~wfFvDKC*w{P0ymd=8EJ|=JAO2^1DXISG&eMKz>J6P8=3ilr4ksZ+ur|BF1kOgpE8ij6;`ix} z&EnxQ?`X*8ibv(+R@j~;+PIfmg!*qpRef(4*ydATHV&lT$m7^nah)6O{TAv^8nimS zjx%gzm~UggbQQuIZSlaJbb_x&cTqM>u{FE*+4f{XZzJE+-wQ0SOoYsYqbcRDP}X=g zp-pl7&I)D0>GY{--`wREW$SoyhR4c_lU}wmD87Ce%^{NgyzBn9q+#W4p~Bb?_uCj; zwRP^^RZoD8!4n>a6<9uax#hd`rpv86nYeT^MO4{-_z&53@S`*jijd8|6=rUf!i(g{ zH|oFGUE;dfBd}zzTu194MGRZbP95yaihSo?V0bssl&Ne%xl`(o{az7ta7^uZ9+z&< zo8^Q5x$8k~^4;CDiYJ$qyn9vhPd@qAY^;;3OBb~l%@iGu5p9Q@A`U0HgH%CN={PP8*;7UV?J<5%rHR`=Mv-+sg~|7 zeR}0>@^Qusa-w^pf1-V&d18J-`{EDE)#wD$$oiViG%{m+Va0lX^PQi`xR6GRxsCuP zvev{8@)gRjTA|wWC2Ne27-EfO9NYXQ-Qe@dhYv6PbL;r{QI$jm()g|>E|m}X1D`|< z;!430tM*>ia%Sv7jghlpcX~3_D075=51E_lBiqKrtVVoKw#Jou^^B`zBD+Yc3U}sY zNwJavKbG&l&jS3WZE3a&x6<9BrkeS8bP^_)L;{NoO)C#N0*!<q)nwM6d|zj- z++l9dWh(4A=BwKGW8a^{lQ^LHWhBlz<#G47BT4poo$AB~@A;an+olrP;y*+F4r#Fm z-u~-we!LrGM*AtvSu~C0Z`=9_&hv!@oBgaM8}|jZgLXXE{tFtTTW&rQQ=bd~3Ltb< zHFCdoi-!E>&u!Z*EWi-G%@9SfuFv#l2L5ZxmuGz{&tS1ABuGis%gaXXnFQ~;QF&~( z)l>dwVpIn(ZOzlH%uMq?r3D3VqQ2j$-xpagvi*2V-zi-theCA~3CAIa9$ou;i#MkX zY!B4G__lL?DxLLtTWaUi{^r2ur}lu%l|*)V0+wLX|M%Dbv%&veI2g&nnoYS~%~$8y z)hdcN_(So{NB-+$)%(fE+l{}d^g<>F%6NPnw~7z5tPh49i3%-i} zvDj(C`t(rzH#o@tP8UPWc&MAN#0rX^lZ4^ovOx@7opINx|QosO1LZErT+U)RZ@YRD^rS;+CO35 z5zz&7^3?|XGgbL*M!c2U%luQk2x@@0{w&34!MHh-%cZ8BPTAh}Yn8kayIrSq z+r_{;{uP$W+jk5(%+p2;GtX&$&V`dC#Ym6JT%*iCE!^0hP+m#zk0@VWyy@Wq5kxfU zNro$XZbI8}J2g+SX~@BSdw#la{9o*oQ^d*SN7@C}jK^WJx%StQWX^O^%4&h%%DQBB zYF>w$m-bhfS@Y-HZpa5WWL z5oHlfQ8%-t$Z|OpLYZ`qg;2ALqCMA~*H9QmDE^)z!ygPq!ovv+kKnd89JOUzu zi}6|X`Qc(I-eJ>W)3uF^`f&JRspINa3#~rhhUQ15KgQYFin*UV_Tp^k4R+6N$z#2V zh21N1V-*QlMsRt*Tq=*jurYWHRXU%v_;21ba%0kPF2(5x((u4HO?Q8oXH0UAdTKXP zgFtZvyJJVmg7eu!FY*b=Ap5iX%18Mh30T<08kzXF-|No%yAI2cJ~Q;Y#qG+l#tleg zH;!K&BTI0qT(M+2Q@=1F#9rP15oF2KC5vpukrM5-zW6|Cs7hobzNa4DS@#Sa7AzG8 z?k3|DB*8Tv@(VisV)0!qm=r4}GSp9x^xvz#Rkva8G0ZO&Zq5osQLbQ^CIfhluBeW5 zzvUJpCIm5RZ$a6$OB}rQ5d^B6`zP=%=pHy~j3+ynlXO7~aac zHaosh^Sk+}yec~0jtJ`a2|aowZ1}hfeT}Xe{>b5?n~)rP^zQPJSB4^(TWsq6uwxf*?P08jkqw#(59JbSVk2MrZJYhJ|^e7HfWD(47gLT%)v zeS-;6CL;~8jar!_$KiK-+Y#D1V(Iui*Bkhp;9?BT?olR$LVh|Qllj>@ ztGdk$`auAQ1c?j{d2aJFRpoe|&4eEB-t8DUAicrP5CB}bFstYiLxvbh?Z}Dj_@=l0 z8!ta3xz4X#x0@Gn+9$^yKi_D6)HL%TA?A_t4q&3!^O>USIEp-Fb__<6^X;aWQ9uw7 z+LCL4i&ErPKk@5(OJNn~9{k9%bm?@kl>6+}(e}3WLE4?sdqw(|B%O@!5#yWwX*-^l z-d&QX^@8*{=##VZV3~XV1W8m&yIhku9!tKZts3+_LS0gT_t^H}0ZVYV__i+_UN*?P zRkQik(^dt*roob7`PuBQ;l8=cty`58_s<@&*JAu@BVB-D)ZK)5n8(lYh@VL1hXwC_%P_zjp}LpcEk5C=AHfrA-kccF~C_UC<0EBGjA&-6y0Ta~p3m2loL4S=o?hOe0a5vy}2#8J5cJR+r|sOaUI#O4VP}ND~g$IT3GPHjJE5 zT;DK|hwDcRzsKvG0To6gh#G%@X;&HK2_S#{UjrHKWaKyRV6h86y={hE@-INNeB?h3 zT66ZN&66MXT$=OXmVa2(Vn<@5DrrP3!V;%7ox66qu5Qq>dTV!8FvFrENC_wvR9}6U za118S#Rd+N;O~Ta*#0v)p;(v=F^0y&JP4n9NFg{BPf}+uT)+VO>r0NW1qpfY%4q(RJ0c-`Tv4`j?ECVc^5jl8kY{rGIOHF%e#E;@ zKYPC zB#zws{y)&++++fB)3W_cRoZVtt2&SQT9kzNySt$dhL@JuRLbgjdp1)rU%d^Zg{x$aWC`7h#|jz8+je>XV?_}ykjvMLf3V(wslLeCr&+3x+T!HI7f{ zND@d@5%*@D+{EtuT~Fa(BNy_mM4lp&Sm9t&)CDtijZcFQb9nLKrY8E2u(ii)GzEcb z1?{J)V*Zvgq6o&U+Uh+_?#hOjY|>7S*fct4c^;C87H^@KgJ#wS3@$Z)BpOj&etC^xa@&D|1 z!3EgwfV9Yh-gA0*ytV~Q^yCx4Il)o*SUnQ;JX5np+)CoAyB)~gZg;e0_h`znkCvCe z-uMDdz1-dRSL z#Tnldl(M{j6Q-Nee%AWymX61fLD5v3(C5bP3Tr8Jl2U`pMy=WhJ+-oXE=*+q0|V|Q zFv|pOYJV5pFN9PcPJxz|D?8Qni8sITKk)t{HV~m=+?sX3M8D$sgGzH$L-}_>yvya! zPu@~>I>FWcDDNGP;!kj>%E>(MJHMifDrs;rHM>83bZ|IjVyiyF7-V%55O78UXx&x( zhhEwy@ey9YSg9gOE9Ut|yEV7%UCjPLx4SQ$XVMhVQYmrcHc<1f*8Mg7=9u$a){n!} zE;+8iVRZJUn9l)>!7l11_p}y1GPHT{GML9 z5?^^p2yO$Gx9G0^!Kv9jaM9~-ynl*bv{?3rHkc4Vmzs{h^D4$vpVp~Rsh4?0XVzZ~ zjEp)C<&W6?nMz*c)#OfYt0&pvA(ZdAV*rVO93YVy6=kVoA(<>i-SJFRN1vPKzO%~P zYmX=$sPhm)*%%Srpe0$Q3Q}_aeC8Zj$DR=aN+}kEvB* z`nrYXSZpwccxRM%vG4rj%&g@=?NZ3;sDt@i(M7E?uTi^N&{S>eQeMIMMc&^Iy-VMQ zFze-3sq*7=-106yhrY+I8u>s|CfHE?cPI4>a?W&J-AFGJFPZ(rAu3$DTe02MVVe0N+5-PHcUQNjB&)gh>_gmqtVe(aPF>*lrfZTp^=l$L%Qw_`QfbFjr z(FgZ;6rn&^?MN3_4&kU0T3eH(bLuDtrJw3;aPZURX~7tb4$6z zEZJEwivmWf9X4(`UQsMIy9XxgURXEM%I+UaJApwU_j z4n@BTC#p@TE$;N6-R5TS;Zp7Cm*2ijCBXEpQpsC&?Omu-d7|3NghMs=#7C$_lh!8_ z59pLhQ1y6Zkf4!^)xYGeC?ti6`t!OXXT$mQc~%y|{O4zTEWqetQmC%>4D81lozn0n zF(MUe0eg{+L9!ffE%-(d4gE>XY9_c15jJX!F$a2|F{;B)evbCaqmuDfaQF2YPA8oW z==E_i1Ww}@SIcqJZef8>yWNWNi%ijXZ{-xSm(ux3U{6oVa1xWm+pae`!s`}IFk5P9 z`uY_>O=6U*fmCa*SwwMR%4!}8QE08_{0nnIWrn&QtQB5&Zm%s-9}3zmUxwX$__IgF zKjAjAlKpar#3gN8=6bQRspjkm2+)+g9*lYLZ8cFh{k#74#}P4#yd8a$-fE=`$pr2qiIuOaH?g~r(V{UY|ervuIR8I zcQnp#<)a5!PM*NtTi~f7)~Z@Sc;}U0NbG8jgZeNTJygCVYpnR0H{^Xw8z^8QTu??c zM+GwTT4gb#Bn&p4#M5UEhod@#Ki^|T={Kl0q_kbI0?cgw4p1E)SPj&UvGvb2v@161 zX~(q4k#Q+^W;cjNv&>968I&qehVHIMU)+uK|TSwXy1EQ35);#LRvNLBtLMD;k?5Un!LFZaG{8m1v{M;VYzN z2}ec&5ao8mNbh?@p%qUwzZLOm@D}o^x7>@<)*7eAw6;x*Z}3r42ep+;Y?Y65q^xMf zp&wj$#HZ=fBoGl1JzzORJmjiLA0Y2^j13N%k->Cb^t^0kyv{x`q~jaWNdM;Ca86)8 z6%33+5GtO5F$ba{_Vnv_6q&~F-7XSn)vPVy+w`4WhA)hXD{?cKnk9JIiu4{&s}_CJ zrzEym`r7#L@*08A_FHP6kHnaK6Pk2--2cPeAic@OOuHBSBc~ZPa7nKw?B( zc)UZC#p^lGr4Hr}!{rda3KV+zH6ejo>uCx{@FX+g+di6vbO9jaXt~o+z80|sgYOOi zW5*~xs?9VY1ONINQcj=g)3AD?+x{yeBElU6(uT3JCzW~}(uM2R;}Hlo--#oQC3QQ< zv^zijk%w2tuW{~>_qLF|o11d=S8c8U+v1&B9a5uOKc}->=;2F0Gy5}QfN}17rX*#{ zjzS6aX4XUA=qA4nK>ItjYIQq^?QGC$m_SD-OZn8UMdd7sp#6_QguIU*e|jHp^dD3p z??Ei6y-Nyt8e>?=`8)^eXgaZ5H))(hufkuN-8|+)?`t9`WdcDMm7|%=zPO(vTY#G# zFZ>Vl>Hsxj7Uj{so@>rQ4;>{L!z(IaKidn{457U)9{w2ppgw41rTD_yuL7hh<%qDH z7Om^CU;5!Jh=YD<0u>e;r?I`bRaMe&@k}T@*4W^2aOFX-D2Ki9384-Q3J0&9+JFhK ze}&wc4S1Te^dboe07V-0V*H^po)hGqyL?I^7?QXL-loMwP#?peq3tsNu_kpX&*tH= zOCK)v2kD+>v$aXl28;Nt33mXPJ$x^FahpeMW>Txl?s-(C#7?;DXb)$NQy=q`@M&hf zznXjIFgrN|6<&|)t#_2cPEOFIP{evDVXUg=P0w{Aw{d9i_{>5l3wkdxT|g zL|spc1@oCb0}gs8F4v2h{5FIiYOe!115;82rDh4>VsuFpUKhEr-B@b^M=Y6?S?(a^ zq`_*rit>nhlOkF5T_DV%^`s-Na>kM$SD^GROIYZnza_S&mP@%O)i}qV*ms*%|*L1GA3mf^`LKz2biA+eX+9%PpTn^?nB@noD znqQpXdur5h}L&G%M?XPxg*&S8$+EA;k$gLfB44gFkQ;_}%26NyBoo+H}+B{7W zWlln3MJ)+r2kRduQS|KmtfOYLWv$0Sds9*}niyh9 z?yTDk^0+dgQ6zsqOBhiM1qSdi$l-FxTYU?!nhlD!%jtTH!$|hFb~^E>7#)vd4!5s( ztnSWIR_2xge!Fc*q*`|T@$DIemKa2LRe<60#j>fQ68*|!7NE!h-K6*lJql{F&gWVE+^1N%0A1G!(|&$HI# zIY&-x-OXNvXEu?6tf-+?X?h0eVTfb|G$&3EcYAc71_HzWZfbOmOeNnJBA2vpio_Um z@J^E>k$*be0M#~C_^jddOa+7!eHp$A8uM~6Hb}3xHq)LyR?bC{W{D-_Gxdt8=`VZJ zf7KE{&B>;O!N|RE0lvV|<^+7*ct8x` z)#b;JePC=L8E{2x!YArtQ~n0hgksYM%-5>(N!VUPZths9KKdpFyS8B1CMb`ME1v?- zEL8At9Zx>Uu47BmjEye!JWQ=F5@Fk<6lGC9#bMtR4&YyZtY_`n1l_z&62)D+#;+<2 zYy}%<+&mb~Z(;Fc|Ks-SA~v}{iyx5i=U~Dem%cf5O`9-?cD@V_#%7P>QWdkW9xX!& zwoe~AI^IwCEt;Ib%wrw+4dT%e0N8WTF(*4Wq_O$CsHk4FOl&qzNrr%>{NxL{f98Bp zI_9Nb#v7m!@a79E%U&tB=A3cH+?GUM+j_6pR>VOQhJLX04DU9fQq62`8EFnP9)qbx zLOpd|vU$h#P9YzZw~+o_VF5NXNwoaQL|q8&u)Nc>3oLjhYD>J#^<}h~D15zBOI%dd z4ZP+X_2!^0y6`}PtEL+b8qGwaezWzPF2c}AtditR#k<3K`wm9|TCS0p`p3$qKK!ln z@I7Ngm_W*;%BlC@*> zb|o~12*daza6PFyayC~CUP-66>PU06dyY6Yoqn45JQyBfi){bBqa!&?1hG(0j-QGR zQvq{p*|vmm_A!QMC-AMQCQJ@Et9;u15xdrG*E48K4=S4b9qm&pk82$yDEd!SMh8*39cnc ze>CDKwf;Duv4|bP8UEU$Cg*puMRsBiSa!aHNphdu8P zljtT|gs!v$d}BNUzZT)*Vv>Mr`cF(>ySGCol|#OI3N|I)$cp*v0C~AcWd-I3A(XYc z7E&|I8E;LZ1S9WRFX0H#hHP4t?s@TOk(s~i7X-%5ilN8A5SFr3x%0gL6Y3s7ZLE3Z zo%a=K&d)l{F`@TUZ@*oASB9Z25l?#Yy1fmw@V%M2$UfKbICZ{OE~L6$Pe~@|{a6jL zXXxu4=nG^{^PT8z#~C_UL-vX75Zabi^I{unSt<6U9}jaWn}u? zol9;E)+=3|c;RZgqgrMV$o2`3;%4F+M!HiM+lfRC@wjM6c48Cz{VsrtiZBC{hc2gA z^CMO;o*E}LTL!zH;%F<>k;Zv}O(i3dliP=~U{1p_HZsdvFQ?;JMxv01wG-pQ?d1Xi z1I2VQ=_qowQurHO+UWSD(&4|bTJF6Z<5@mg~o(PQnT3c?M{G+2;+|Q4Sw#wAb@1a!yJg(a!tIa@F#_MrXJ3;pCauW|Mw6_ZC1q#;)ZV+!T`>Os~zklqeYa5wr4 z8<`)~V_%Zggmr(18H&51%HOqO7uSJXHFq+Zh=51lR!q^WJihN#KxqQDj~J@gr?{k_ z!hnya$n^0_1Xrz}XwjR$3lj$bP8mt@+@MPl@nOS~YlH*x8dzUBuzOphO~0mwdQhzleuV!<&OXSL1W@e=pql&P zu;Y_GJ`V`_%^=g55NcrRro?n{?my`NEgr!^0{}-AyB8tUl9zmhTS)K0r1-Ii)J#8{ z5$lD$>NWik(5v15NCg{T+MrV_YH@N@NWrZ9FZ{0zlMlyx+r} zH(uNOfd01HnpvJP=~F>vd~8UGpdI&I77o=s0$-PkccKB1dFw=dDgcLO7Oj6M>zWBCd>^lm=k zg(Q%x*n8G0^FjM}LLY0f5j#EIl&TU+BaLOHiHjxYQbY^wScK}CuuOc?vq-->Zq7BQ zu?(qyq^&MBfe#AzoTNfu&n2M-T<(TJ%)#6WV;T;M0fp!Klpe#2MPMAtgh+!|UH%nz z(+s^eJ>2O7`69a@kW5+4>;h+>tRSAOpOUTI@%|WqX2bZ^MXH1+3G%}Mm&T*`K!M*3 zaQqq()QW@V(^1^=t8DC0=OYONF?%d%QLCX|ZoV@$7HxWD~X zJRhUNOim`|x9VD;y!Ls<*oyg9ohgcJZ46#SP*$=2Y(}Y1zDGq8v&!@Q2zLhhMY5>G z-NkTJWT3Q^sxa_kT#iq&r1b{-cgJiK6B8Mprf3+=Yrx1yAR3Z0QDRSz3<;o<8=EUG z(w;y~J;5&Ni;Hm$FDjcj0E4*`Z#IC@1ZkYOx$CCZEk_xXr4xzuZ*?$8GbaImg$=9?J1i*h4&@m z$^;R2IHgGRKDWZ=V-FPQ)v$9H#icD#w)ndrkf^HcMMwX4V!5@I~_z^J_!-D$O8yPA`s|w`e#Y}nSS#9 zn|z99btDqz{|QfRDw7=uWhV<@cVG!iRow_5rDJ9dXstpU;NoBiONg3DRCr8|Rz4oA z>lIyBGP?i8%;q$sCk0B`k1Ye-lywlXtUo29yX=mQ29pm)XMaso2VVeO=Z++2O_j7xQG-HJtn)GPQ-uRP({LEM=wHR@C9_I}+slC;Lj;asI2e-dvb&rPVMVv6LzSHA zG#>E+6CN;kw6m3tr7eEd0yi*P$jwIcgEZ15b{~E+C?HF$Pe@EW%msMRiX}H*p9bB^ z&KfSSVLP+RTO4)JC{w8=o`jaEtuHj_ufE9SD)T&~3+-#!-s1Zjddbf(%Tv+@x}+j7 z$48c}zW-8u!-0-0;TdxiWV-Bv+1W2NVS6%K>0w?PsR;AmFT^>&HR)H7hYM#!6Hx`gG_N}iSZxx$rv*cTq2c4VGmkF^Nw9M+B8Z@pQBJKr zo*%;#ro7naKgX^YBM|tpIZ~(y! zU*?GAya8vG-!B+|GA__5;}eVT<%@7r)7NxBeMA0%f8XB*Ogdd7ivxy~zQH5Qud0{0 zJe@|F)9|+`u2aSI)xT);5W((l|IkmW5(dIGcbfux=>-P0*lPqDYh_XeBjFx6m~Z^; z(hP_OdqvDNAqA(0#tREqA$kKO9P$u382Cf(Vp+c`T_ySh!vjkvRwQ>?!Tky_@E&eZ z0NQ@vH}*<)tjYJ`mJWJ8?B~o9*jb*{soH3S+TsBJPOam??C(ekvZJVuD6P!}>Y@w> zYanaMfeGtKH9w!r*)dYKt1c#*0#`S>cSj`F>O$7&H`L_G68<&_*|<0g0+<3fxYY8W!OON1QFKl_G#|2LAEG9ec*Qa$fjnB5f@ z;H@6jtk4Di)kuM=43O?tJKcV!-(<8lx|t0)t;-)|*ECT`((_i?A_bZC(X_1=NBKQm=6 zL1!lzBT*~t9mRZ-B;n$~Q3<5)tda;gZoWlkxRDy+_VLALh^m+HFo8Un8{k8i8KqR) zkawYWLw{a~VMsmKq7RKgqU(wxGj5f&IFDMA9-xl<&a_d-A1>$4y7RZCWqgR?KunKO zBjJx9#}cs1KWlKD4*!tf-udLc2I?S)SRxGxcOZneC7QO4NrSd&#hriN0&*}rUWuv9 zE3%wgZ-9Kh@O~ns$c39#abZ^kSd+XyJ4{gJ%b6IYdL5>s)G;zKWpreQo=R(42yS9V zOl=!`D`8NH0nK5OuQHHtN2n5D;*cJ(sMM`F@0MA@CKM}Yjp$YU)W~QO)!%hy^www3 zhNh70gU@@?ejJRzH3SDJY+7;QZt|C&tw()Glp$X>!Gy`)gw-^aq5!#QYTyHkZ?(TL z`)!7jH4#H?JL6>Wcn;H{7Wc0T>>Jp*Z&F7-F2|f$0oOO^6X3XmXK#t1$OPyid4idb zn+23O5MyvNNbeQU1RRpO%XV)ZPf{NanF!}>ZBT>}ln&yy4s}ShnHZ2uLkUYhZ<%aC zTjFhTfPUNz+~06XnGw2781^)!%(vyGM|XAm9-u+gG7m}lx*)LBvjkQI1GTAHBR~e6 zHiijRM=a>V$PXihM7IK&en0%`d1i_p)igQL2u={Qo4yeI>=rq)#{Z{Fj2jff2?RE! ztsN6(57afx3n-M4Y~NndC}|62OighueyLW{!?z)(6IpG%x;%H+E5)%@x*EA$C-NVh zW7e1ACA84RD)Z%h7j@u!`5M2DU4Km#w0|niMu3Z1C4#t>3*$Y-SD%<6>doGT|_X9`3 zS6i}2USL(!xuLaiDYa|fMw-U%l5q?59GFkg0a|%^1H0d7zL#-Iymh1wscaviLiBC) zSyegZ?E9C~7v?Lq<8FTO$9a|-kh24B=?{dsm_p!W1l@LN zywX>|MLtMlU7&ycsV8`X0;4PZ9@y_xakN)o*_pbO80V)?e0nwvgN3Vr*RX)LZN%RV zY%_L;?|(mPkVFhm)^d;A>#+D57=NFBol*Z$`$pUT<7~rhV7MlSi|OuKOOPaS1Ic;+ zNN8aQzla{SFB+b@ zYZwO)mB&U+KIpRg3{^`5bag5OYT99jNC}ShhX@{xYd250AS(%agsW!U?L#E04 z*J}?;ZNw4r-~>2BZo+!R=wt==kaUXe-Tz>CayTAnNLM&e{P~Tu491h%#Z!sbpR412 z5=qH<1^Z`dN8S$G_xuzb{b;7T1JE=H7!oSlfKy4&;S!LoNS1&m#lb|V6F|LHK(xKHO7!CR2=raSM&W5l6#LUhdjbTIz^8C5(+mZM0mfE^vz<1j>s zL|u^xLLXrnGdj8G@4)R@Ipvh+>fCA>kH6IW`cCL20<`XB6`*}~){OnF4ZT?|08`^( zvXbsyr<;tx3o>)-p$e$nI!J_`Qny~;oY-Qoao%=5S)I6#A8FxV>?3~CSV58t8_33v=iNHm2T!6etv z@y)F3cGBHB$LYn@U29RYL?C8d;h4%726QaYvih0Ceg_-QkKG#2k`~oeTo`8@VF5pxkp%^yY%?B*wsze&^#f%z67%XN`tIgd;xIH=%lCoM#9y)5yjZ8em%f3n zh!y%q<(FW>5mkooeJZI}M{J(vn(SFVaMygjcCzQyYI!48Hli*2Or~BB+7m$_j;Q#Ly1NfHQ>9cutLVB*MV%?359=IR=wW8Fas92Hy%UM?{}Y|W-!u0I}Ms4(Pq zR&blntt&sckR;HyGAShu)lm<8D37K40k-f%&sf%#cg`R<#h%bUVK6gE_S%0>z5wD- z6hag|QK#?M$^|fa1{%7B?_)n5o7uvwW*1?Iwctw z%DLGIkw|QOs@Mn1OV4Su+m7MZVEabRG|0^oCFD+bp2ceE;vOX~=0sj^yiKe}h)MWz z$BKpXGy?jd#j|dZGX`1tIW&g-xn9ngHA35)3W;6Sc)1oczhk5Q*<{RQ8kQUg4A4A2 zsHV^hy-*cbEHsNwHCAJ0-a63A>-Jr#H!0hJo|T|_)hSJOBzfo;&Up+q3O##rk0^2z zOlkRFKb2qa)k^=5N*nuGl4OG+VXI8qvGnn?lY&z+-4MAb>Hr1swxlAb85ir_Qv`5; zY=nymTHjO&RUa&Q6gHnZxfTl9OcF@pA_Oh=3YkwLQR220wkLs08_kf zM`u$qMgCPkC>s?hnj=#aT;tM)*uVOkbi75a7TrBQhr-kT+(PY{q@Q^7sH#cia%UrA zRJrVlY%fLBs&2Iw*sR<)6lSu+yIi^&t@WG6Tv;rKrs`hP<0MBSc%!anRK!dH1e4d!Jl zf+73aJ|UVC=Ln!uqWPfUwOp8lA&q~KLKe?Vh&-BM}xe$#b6u`r8d^FtBi0VmcAS(C9@~H)=>#Dp$+3b_AZ;B zq^%F9Q08Sy0aNr~jdC&6vzT(FZ-TPkY_kgq0q&>G#B-ZT4 zA9WIfz|GY6KJuu-F&G@43nWi*N!IZ#bM+H%`Y?z*PG|77z||!oPGOK=Zfa?gP#Wqo z=xOlU@v1De%bFOr9DdyM!C4Ag?u z2h9z}4o{{#BYV;YmAG3Qa)oH0izP!ss1K*a?aRw- zU;UFiB^{pgMk)j>fS=M#qrR|RWzbkLjf!;k_kpLc)Uh$!MtHo3b!3O3UZ<_cSp>&T zy%iaSBOW?$yb3~}*7VYJArE*t11O#!f)&}T+WDV1KudD~_qPvstZrm4gWXtgF-MjJ zJZ{$TtEfVI3UvZi)feP=co~e}8`4SCW!4dw#Gsiuj%lZlx^56LUOz>D6=g zU|MYba4Et{GE}v2Mc4B;KejavqN2)P$M<8iR%*0Q_SleE$~$$bbe;eg>SIX%phU6-7{ouc!CNT`M!iK zhj;?VJ-mJSO)xcktx6DwB}k&pKMJ^732dfJxM@?(j+=+>P(dt#1J&1UXL1<ppl7X(oy%Nw+M+Ew(`!m>p{PMLEm5u+J_>5l%w8OH4lw2MW16z`EJu9t&n z77<`U27hUT%5aJ?>xjU0#dosFV73o)@gFz5Qm^Lybh(v zrh@ygM;~x(tfT;#whmHU!V-Dt9>4 zJs7Mq5`Q>ezZ-rViYc>bzh4tD<8&zmTd;XRiQYcoV8)V_RCVnhKE)YGmJWqE3(^$+7)-*TFjc}712vY4mPIfl?UL#tPZb@DKDeU zvas)^@n9#|>_JQ?vK%KfcRb71_b zXe>ROve*|FG-k)6KBt>8XR_SD0ZgZHI9}sxHqD3gp0U3R`?6qo(pXwr`Z4tR#0esx z^ax-`_F{@PyQt|G4<=vyZ(QCJUDKuxiez+jZeif7W`Dng%hm^2psHoUNa&u3(P$e*t z_>UZ%o(At<5hKHja%dv9_Gg&?GWpP?b=3Ltm#}7}L8MvotkjkAu-jC^R%M-6+F_7p zk5Eu056xLPtk3CU3Yb7VTcx|}w!x@ab4i~&u~BHIi?MyUB0nxv*uzRw{j)+&k-3Y` zO#VLn5kP*z|iHrh#{=&jgfanhPPg2jCSqv=#SZ%-1 zUh~@u{5#FW4eH74sMdlBC2yX?QOsYz)Wk>uUpd~ASk!zfc50et6+p_0wIz)c6Q2xv zb+*|@)qc^p<3HF=yuP#fZv3eddu*;LTSJFim|X^o<{@h+V~mxlyy6-4!qeaPJuiMW z>-s+u!%&x`U($5yu!I|_h*}ZKkKa03j(MP{YMJgRZI$6L8*|C1=rGM9^NmcjhVq;0 zb>P^h6UzmI2nBAzvqmg^L}(Qt+Ef@n$S%3%yXV$?&n_AGnQCoO zX(pC-MhORIu5k9*~-WJ-A)eGoIx zy80L@H7bDbgHLsz#iMY$3(_%ZZMEgMKWms9)VHty2_#sMI(2EC^v$ zVmwAhTxZ82x~*!F*x6u~x}8|iayb|4^o!GqU#r3jSt2Y5XGv#lCIN=ZdNeRPp}}Q6 z9D9sVTu-a?)F%HJoXj3vEZSpFFBY>U8akJLQ)cy;!m2xMiGk{xA$G z7|8Ak-VWi|Fjtffc&}Ss1*g3C+@C0z_vem-iEQYNibSiO*-#irg z3QAbuWq&b*sf-^C`-k(5+Za~OdUm+L{SK`|t|8(v4>@-9@gEFeBUcE}-!2&!emIEy ze@wk~R8`#@E&Pfg4TqM7LzjSb3mhb+TS`E>yAcFEbfYvJx>Gs~LXeh5x=Xt2Tlo9N zz4u?n8H2s{e%6z7&WE@|om19hzE+c%XVj|XZV|%Kjd-J zfwFVyWFI;C*lnaDGr8W6|Lv?^E84!Wf-68g3E`rypbY@ebnbMzX}4mCOTH+!sZJ*u z^rHP~fMPr;^c~VBswSsx`z4Qs80XMA6-?WT{WVLY{G?e;(XZxAm>J(J6rrAdE;`Ti zG)Q|XZ@2>>J}vb$_HyN=lU?pIZ4RZJgMO58isX}$nlnhw#qDE$HtBF>{L-HBPkD~| zdSl0vt-cx${ce7xRNi9Dn|i6^^aC*rH)_RwaDxmY6f}fsY*wf+FzXTt;!7}6Az|km zsuf`9yu|(Hdr0QNq(#*JHj{F?7ZytXR1iEJT?oW%S*sD|S8`!jWTi2Nki8kP0W4C4 z#k5*2y6Ge%?MbJ4UI%#C$z=Bid*;o{2=~yoyQ&<)+XZ(SQnsD&Q8IyR^%bNrMd(}QGKz*AdV6wc zIwN{@xToU-0(S-nF55ADdAMc#kCXP_F)?vQQ17WPA(<7+8?6cy!Hq!TZFf+#`W-ll zy)5+j%?pMcX7Cb=oP#4vWi?eHN0eHD_sH^+wxz^e z&DoYSSBGM?pzKa>A503kEQqfob6=YL~Q^Ju^!F=Mi+I&Zy7n`&dE)y%iLlE{D_zt0N)x#BChxxV1Y7 zdS=Mys^7(_Ds3)TAv7wT;S-t+ZfZ}0Q|2tZ+l@N{NNJ-Mp50@ajZM%4o*X>0T_ zU4K@@BE=K7<{Xm?+rDhcdS*jZS$L{qX%D5C$@ts059OS@m$O%hlMZ|KT|R2)=rI^@ zXG+ok_6O}mh8@h`ln9Rf2~fvg)2RPBFgVCq*R#!+3(NHc#VsU$7!m?u?ejm zAPN1Gc~@#&@iH^C(5}@_T#)6VE?S7bgnmFJuXP>HUV&KG zKkp$R5$J$y0|M7^+*`Hv zL$G?k;CG8(WIM_!;Z)^p{e#e!3nl)-ov!>dg{;t(@m)2>tc=<8*Cpx#=zv(H2};?`8Q;S*^3( zBx3J1I{&vVMq<#}A!ij_fMYUOX)moE%9AKygM7?&t7Gs~JgjS`SmN;eS!a8jpoYq$zmy zVtY}WhcCfz#n@Vhqd{s@xL0R~S(mZ|NB3J<;kfW{v{|DJj4Fk+!;?LEp8`^XgrQLL zwmwB`>Y!o%nWbSavmTRvxpqrtnCs6DN3GVeIoE@Wc&L=0SwX!OuO-zCM18kQXw&GtGVyt-ey#QzL z6l$On6YXts*?5DhbGmLA3z3oxL(VHI+BC>WGROq|aS_N!LYR*nu8{pU?wc?L54DD# zD^{rO2RTWEX&uc>o&eO^=y?L`I+0l-E~M#r_R-@fC)irLs)_9>kic#z)FP>PlT@YJ zdMD85{4ZsdMKl~*!*Zkpf#IYnlGff$=9u5tI4-oOsKeFxO>ZDJ-DYoVIV_+_Asw2f zYiBQ{$fND$qn%j|--jpi(%OVR*M?EtGW0}d=RwtfDVfKc-WoVtWD)f>$$+n#7r5Rm zyP=oUDAvQq{}0K?ya)BDhgdIynDG$11ctaSKQC!JF1E(W^#X9)zJinWcE*o*;?{p1Ud$wHn@f zPrp^<6jQB|B|b;9O(>`Ye-7>4H(;r&( zKHv`7cu(L{JyW2t0f89(3r!u)kK_1tR`op++Lx~I05K^_Cd4>vKe_^=xM$hbhwpP8 zvCy*L4~JVtLTPSU60*%$Z?a`$FBj`x0VGH4I61BKgWp#6Z~0ni7o(Tw{(ZnY(?5;Q zZA(8W$pR~+wrADstM4@l6GN<^&`6rtR&?$h?p8U5yk|LF^*kci$h|a3J+&oi&<@;@ zMH5vu$x~v}WK^Hg257XZ8V^s4bYc^^npPB}dJn{#eq$$$6zAh9Z>vrW|Eq33cp?+c z6TN)qu{%6#7k(|H#9Xp534C#_Rk3g#h;2}QU?8b`T=2_WbV;=9OC7-tyloz>wW-7^ zo#^W=*Ms$NcWrDLLU>hIQ}z^P}@{=FcV~QiQ+8r+Pf3 zx#g=XJOpb3p2={}$tR=g#tLiOCtj^9`nEh)6P{rx7BRV&6)jyU?heh%U`k(%Md)7l zIOv3{4f|B0OP2Lo544vhJuNW5{iE^U+`$xwXNWMrnhLs4?m~}{s#5rLN|3vm^8$g7u#D|yBsrUuZ#cpbcEN>pw8ojS>+?x@ zrLxP!t`on*@fi21R7;GP)l}n9$z4~|xsO7{?Phf;X=T)P1MQ}M#{Cx|!R zf(KrNMf?nKmT1`AcTxQW`bR>NXUZRwu0BXd2tID56=OBmHH8DjVuo!XBa<_Opx z!@YfV&9s(K=1Kb^9)FRBX7r14BkrdGs46x&l)>pi%pqE1RCfNm$*-|>ynbvz2ZK~m zpFG-wwXlLrM(N5rUu-RVfOUvRw>JjhKnRm6CguHGxF@bT(u{ox8Eew`+zltzB-DK= zBmjq1_U>f606dTp8<@}e zs-Xv~KT~fJ$5hmmI>FVJyl!DHY{0jq9lgOe{q}beC8Zn{lHn$k&0NS#VI0_^Wkk*1 zCyS=Q53>ntRt!kqTB|}Y zgYgmX&#sZbK|-5%G5gD;EVn6+!WCLH(Y#s-@B!`<@&Io<>_2TlF7*pT#p&6s*gYNr$kg}RJjSLn8%)u8?~9@ zXROjP_4vGr=I}~P`^(yilRV31xR6$;8B!iHAFYoW%jiO8 zh3;bN!}VM;sOa>*4Q;Pl6*j1@a-s(bS3ZcGNEt|TuAqTK73W~CQ!~_~U~B9R%O6H0 z;s}HPd#lmiiQxnxu-Qd0*Kx?*QlzbM!#mrKvRCn5R^c1&&-d^;n%7Rl z{EO#aN<|OYG<^f8iyoT;xTr8?@!RedRrXr_z|3zE79wJ&5$whY@1H2uGWM)L+=Nw- z6I!WE=M{riN!1tT-|h2w3D5)J5Y6JGtQ$0yGiv-!Uq();v^^*biPru{9;%M} zH$Km=_hb~uCIee1O6gpI&eP6-6?V1>msiPVK7Zi@r(g5FkNy=Nd_2@3zHd0oMxw-} z*$R$Hd~u?;r|`^6HB#P6=mA_mYSOZXPncq~rJD33hfuqle`v<;Hh>UG)ogU(CpXWO`4WiQO5 zQ~8kNUW&W|5REIxPf1DRsMe9Q!9)WdKJ^L4xRwS7lRVw0DWY*2)I#RoPZ0~+qy;~{ za}&?5wKI0Uw$Oq|fARw6s&e63yRKIju-qBOefcaXyRNm%+{4FRTmnUE28VF7!jX1c zT-8;)4WG$GT};|i-JcS4>4L&TD>}B|#hz-aHpZ%$In;j5ujyF*qfmd+J$z%lx5$qQ z`W+e^c=Zj-Lb>vEOUHvA*QQ>7!!Qel5Z0qAW=XHUBIx`u0u_6v-D6g*`^3b~{hRuv zD2ti4_lH1Aro6+sph@!J2^IC#j7 zRl-utS6n?77AQi$0%5*s3_G4UEvN5_#{|l#ib{@TTFL)iUN7aRr353Qe8ksMfR;+k zBJlgu@f}Q#{JSlevZULn8YD4bn2p8P92A`n<3LOqmdvWt_Xv^cF)k>s4oNq>z zw#jVdZmE)rMege|+_jZs$HzrS1hjwE6DSMPiU1;vI;x5!?=V#^8aIPTOIM{k&aba= zFYzSje-`Yn%RTpX|h8K?|(ZAMB#iecs!i>S?qI@=`s8lO5zI*vJ+S=fT#h3`PttivaQDo3h3Dd1_JMnDY7$>w zXtw!4d!Kb|{5l>YQPNC`^7jiEf)-&LGvd93mpHb&lh&Y-SVlnmx12BO>O-?ZkqV!L zcIiDHXjvSxaBzHCoHprL*IQa)(I|lNy-iNg14%s|CD1Eu_(GV7K|IPvPZad|ssrF3 zwZ7d0qYMm>h)|iRSVi&;1)uFrtsW-Le}1;~6oOhYjWvpg6~hd$6dbQxp}g;W$g>j( z{1&xJ2bG3|bBD9fY3lt_3Cd_1Shc-s%QhDza*Z)1U&g)1P*`q^Ri7}a)f-&!LLffU`+|z}k93yC|0T zj1mhV4a2=9=tpx*q1{DCYmk=Uf;}NI_kdL)eKj`OFN0XD%(tIY^KWnlVj{;a(X!_8 z>yi_gP6zclthH@4;*5xuR9$?aX=T1QJ4PvIrKY52jPBS05=9o4S-TzO7iF{)TB(Zu z$5Z+rfB2C73mYSkK?Qk8QLl7dRz-j}(Bu5SlPO9Tmn@C|k(tnQ@TC|kQOXzq_qYv znGD$*(o0Sb|7#q`tmM~i&TzMtF9iS^-thU{Qe2I$#M(^SzFrdYUA(uN^yMh8u>BFU z-Dq*BmFbs1$!4*v8*IY2N@dV8(ct(2e9ft&vN4OKh0%*jI^=d-;62a$EQ;PIW@Jq8 zjBytu#FIxb95d_qFGV7{e3k128GF9I@e0g%vz^wE#R%(YqeTpO$$}sPr;?G8+1MuB zyb}OI2mlle_)h>AvHZ2;5}<@}0YYHlVCgo7%1&pn2mdIB7RemNE~_YLRg*n`X$(Oc zZiRnx%5o)0sHu8Xz%N$H?9by52rD;%xhX`;!}|e|mGr4!TtcWjOY8?}icKY6o);a( zOSbtoTrrUM7z$50`KsI^?3Eo$jM+!$qrZw9N!E?!EF!c$B2f4SHOK!#kS3o*ARWEr z@1_30<52xPJty1z7#2|e!d0xgoS<8DL}@#%Ff;zuv||E?rvkSU2R&#WnLZAbDDl^R zH+*8NY@w0*Ku5YmZlUtPHw5J{I3oCojW~0Ag<&rWZn#8l_6c1E>g=j31W)sC-LC~2 z#3pJXr|?ga>IUsM-q(+EJi?QH41UjRukbddW*n`}oY*6VLutzX8s)R;C4H!J;B1@EA4qHmmpB8*)2w;zlwE*8bR^op zXZd;)#F7U=382jcxez38p9kCt@hCTjPxO#PVZo$bjViKPf6zb^V%!i(3#AX_ew1)c z?NTTUI@>VZ0=Wu2}_9VmH1F%$hk+G-#~7o z-I=G)Uzca5N>UQ#2TF_~N?ehJh-0$pf5~tlGZ27MWwtmuw&OfQ+0ruaw||Nm4YIe6 zS3(xr#3wLzM@P*0@Dw{{-s-Ilr4vW=i$H;kLK0;&s2~{e^Tk)T=!&6PZZyI$NvfLW za*oNOahOxb(xHnoGa5xxSU6$@zXrJeO+WOtudwhZ)cq{X3|L-V_X&VF^`)4V8C&uV zJ*-ce zPQ792bPnLWD>oUUm(HAQ*IsPKW`I}-7a_;$qOWdFFWFVo@~+D(&RFfi;~&5JuY26; zd2qqJg+@iFFp>85Jf!7@=j4?B#>lZ@wo9z}F@_7SG~v+sw{uSF`rh}0hx?e7Ya#<^ z;x!aU3UY#hJ}bxbri8a^Th#7FSj{^jT1SkMn!@I+r`Rkm{H1FnRt2*7ASih|wam#b zHZA%O)JoCyCz~fsWAaIbk(LsC{NLRGX(c_}m`Qx{kAb7&B~W8km*N7W2fC#lo2Y$H zpt^GbdJlHt>ZvQ5lcqKPQ`VT7@#_R?IJp&r_bxODll<~rBut)~$5P|Bo33q4!aWvG zgETFUPjUyF1mR_y+fWtWN$Yi zEf@8Auc{Z9_Gk?sb}G25r}r3@4Wnj#)d1F`BiF0tQD=E0X5dTi7DM}-)v}m!9l*zUr#R$s}%`cVGY5_P)+fUT$ z^w~LZMXVH*j)<49IpqESb7_Tw~7dtcCltA^py@5E7>F?RXD`o~NCW7&e4di!XBu0)X4VbH6jPo@mjZz0mE z7gG(0&n(=%&9CEX!*A&_pWvPmzBaO{du~v5co{+sBWL+V+t`XZvMNN5$b`Jn3bINr zJe+~nU6P^v6!6uq=E2JuXJ>rVZ%dOI6^m5Z{N=wm6}sxZh8(yl6nkJrF-;vi=&wDu z1)7Yj|4L=Z;xW$5#H!cK4Sk5W49(^Ax)mo{py$e$4<vw{#YAP~;TElX?Aeu)_TLRk-2qOTxK#r@JLIw;AZZIQ3W)i=G63)@QXTJOx9I zV@~GeZXL^1x|}FXtp|PS9mD5v17`P++8AMJE5J{xvaqo9z8h+=6@Sl{z@&oHRH1+% zS?jj3VSq4rgixT5vHMM#vCkt zE(u>utHs>DUd90NvX_iP(m*phN?BLiuR6iMpyU2lB65#hnzG|*8bqFEV(;4eMv5jL z_ukDfHEB{cu|+Rix+QSw%X`TkLR!QAz*Yp8U?(D5rnb$!a+Qo1*ZY(XW;FMv@q3z1 z)eJ@WqM7s+ZAks6fxPxhZO%V2Nb8>Rj2f7r;7HU5zR)j&IbTSgojD3~So;s%J%&hL z?SSaN+b#b@QhvYSLWE$JV{BTvlImp{=49ya`$^&3^hJW7{dSy;NIfB&8<=iaO*Btd zAhR6~F4t{U3xK?ugVYi88w_W7#*rB!@`tM7$a?R3PlMz-(_*`*XrE5iEY!WV#7*Y? z5?$pom7<*1EL3t`XU%Tz+()Nw*RM8yxYh1f6C2X9YsXd}C{QTm!}fCJyr5MtO($Xp zncL=Yk>>gYB!FL(ZE5?ni-gp-_)!Qf(y3_s^cg~{w{a2EnY@IFt#KO0a`V*Bl~$he zZt}+6S@NG^TPMxJGFayE9vXZvG<}qh%Z5XSR{IGF5>tC|57+~Ys%D?>4pj-f`4WGQ zYI^wNK%m|7mL)#mENJR4?bMHUOaJ@Z^3%3?I~|z!0uc2*%#ANvs~el#VC*F~p;@Qz zNxf_i|2~Bb4c}GkC0C{GuBWb_7jQTfwW))LU#M*=Oeq;tBecg^KYs?$q>nj_LxplZ zk`K;v6jP!4nE{FNeJP(m{Ac)()GpoF72f@X9Vb<|aF+=Mbx7mwoA;bN7z%krfDbl5 z;)jC@s-?YRdN7~YBd2BOi$rN-Kr)&zOOM1x~(1hT+kpD zMl?zip6^sWs1QCSBMVDYQRCsMa?>4nK!%#-^k}b=r6@Z@llwj>d`9&p$UNdywT04+ z3q-QTCt8G_sKJ!?F&ryHM(*SI!Bh=4NXgGZ$a-{L^eB0tX+Tw371dEAx*}L#bY9 z4&zo6JBNG@k(zdtHnEkOQCOx~7jdsTBs{lHh|@ z7T~?Gq=xRCTxD$&=p&^B(vB$$3pD)G%h}qz2Dzw%ESmVU3DOnPRQ?gN9^qbc_mjq< z$~&~K`-K0*l^q)%G9IV|^L(+tD&dwBFfH%S=tmH0yCc`kE75vg83u2gOHtceY>HJ$nzYeIkbW&iSt$p)zMSsC zDQ)GGO%`KY7n0eg(Qf5|XyVL(>F3z@dyt9bf=fFltoID(^n%s{`|HMa+lbJ*Vpj{d zfkcEW)~((1j4O*;hiaC-{r^>NgO<+?WL15*Wl$In=jQUJ9TJpVaQW=TPsJt|unF9Y zw(}Pyl4EcscQ3g5u5lQdw}6Dud>Yx~4m9O4Yr4_Z?yoN|*NZ9@=2LY_-Mfbg>J4pf zSK5ja+^Ze0E8WhVJrW=8WnODG;lOuypIa5IrHj*No3dE-)gLxj1bA;N!aKfBQ+dtsA=nORYHNFR zZXBenlfK~`U8z;qRE_sTiuah!OyxJNhN{KLvRCA8uvJT)Os0yTS#><8-~X;S)Bho~ zeKM_F7VFHOp4p$)n}iyF2o<6Eok!T-L#~{zI+weiEq`J})n`a+TKtXduQp-Rczzj` zG5A$h!CX9Z%=?GPZ9eSD9J&J~O6=aU3k)Q@ot2fPY00JR9x6LXMhi#(kFikb9*)Y+U54f_( zFdR>OQn}-a5DTq5=r6lfu|03 zXC6;wd3sPFJ3b&(F9i=Rk%FH0g#p%1BtuFzz2GuTW{U~K-R$Td3zorwlT|gw?j`sd zjqL8rhg3R0bG)FM8SC195;bwtP@djZ3~DJislu3H!Y^nvKiT(#FsT!6$iR#W-3VQW$?k zDb8I~LK4v*b*5c01!5lV8y0e0JHVw$%q`D}O3i(sC93IH`Y^ILC=!?UV#gCTeVMe@ z(5l6I>!QL_kL*1fgSIHw6&bY?k+GF8`sE$pN^JI#|IXS9)(a#8Zm!I>F{LBamj-S6-ujA@=Dpr;#gh{G~ueW~*9-XH- zj@og*x^czoF3|a~YTxUH$NY^{coqk9c%tIBhgD)Xa2cwq@rc)r&(tMZMBaTpiZTyq z>M`*v+d8lHAd}@eSGW>t+4BTa6bfFDe7DxKZ+Q2W{60=fQj)#z;e&WzyvnAhNla>P zJ(c>5>q`4{aM+n}znKZxWpSYiw$T2ydk?9>-y@dJ`-~oEhKSGbPsR?H4&3jHDfkwD z`d+I`-7?XEA%vBdkJzk8>Hv0h^*&3=5q9inG=_hkzeeLEitE{{AV}^&$48_I0#0+(+i6uj?`GX8TklA^e z&)Z3{q?Dx#RnK>bZOwmCcZ(Ghlug*%eyZ(_8Dbq0~dGYLr+BWc?X zuzep>)$tn_?!8qb19?L{BzTnF`R~)INp9lAe&UJ6-?ri0p;-F4TsZL>5Hs_J%i zBP|jseZqk;FgUjL`cJP_$_gUNl6k+rbeIeJ*KTq$lD)=7L8ruOFE6fn3g0er zs?M%MTb^5C*(^n>5&mM}7qSgOFc%815l}o(BiqrkaF%v)p%Z9mEU(`auh#cn$KtqG z5M~o6G#v10Uj^%$*1lzva#E|U`S1dnbQcesotQ$Q%P4_~I{-W=HT75nuK}5iKt4e<$$Lj7Oi=@BM1IeCZkvOHIh>-g9>7j={=jhxVG8)GJafRsCHFXy))s(iIWit zCSKKMPxI?)6FKN-c<8#0AjkC@E?w&n@O-O(kh$0$4@rf40)hDL+y2Yn-aLnYP;mS*pm6nhN-u4Ca(yU`g#ogvaxRWCQb(l(}h zSP(>C@^$0fCJX&QUoHNX#gB|x(Fb845DcGbRsR0#`q@kTE@8D?fS2h>uzKA&Wq7vv z_||dI)96=61GyDRC$$c#Jg|VjlXsQE_7w?8sNB41f&#&}KgqU1#s*xDx*N!7dF~D$ zoiT-@>7-5Fxe}Q67t`(v%&a`M8O25QGd33kqA5du_pXOxDw z_P+yZ(NDom!XPn{QEgT#9O~t!NSpt&Ly%!leP*jckHT3;iQOLw0*E}Cq&3m5*yGvh zC&BS4-`$tc6h1wfwv^>oNse@N-5`s+ee~-3mKH-GE>b%0Bn;||_&a*sK4c5BdYJwn z=ZEzB>wWE)4jAG^!xkqghJso?0fuEqVlHdcH*GKJG>kEMg~FBx_L^MXzuuhpuGj#N zBwhbY$=+kSV}%TR?F*pMl2Tz^EA%4EA~XS8eWxn`32)E8Bk%X$5C8eMXa7DE08dY7 za#yETTi!t^O)Q&Us$)O7gft{x=}rkU{q8L`R!|9XhP83-2}?}r3BtP5a;1JAN>bH(_Xb~|aZ_CQsGPd5O+ z-J59TT=#vcwy-SNY-c@$wy%)6Jl+r*I{JxuO6W=jH$ZJuoGOJ?ae0r z|3~?+eK2GXh}xIN5eI5wpeyV!II>{8Nn=>C3u{3-@y}46Ma|E8jt=97K1y!Wh5P`gS-sftH}hh8x9Fh~d85AMyGPXfTBAV^3x7yvQnCCue&z4Tm6sWJ z9j6%tQ2agJ-jcButf|0-di=QGj)EDg}o_>RK|1ODO;pEEZ`>TCn5?CNJNM4keR6KDY>l>jUU}z!cG%`o~w7{8Jc?$};mH zJmX`MyXE!#y8?OH4$Do##SSWrm*(~KSdx-(`uo`aagDZbut3u#gpLwic=|AnYGJ|hbr9Wt3_ zoDoCRX2tmo@d)n?+J1CKUcr6jpx>je3Vr!?ffJ~FKP_#}AXYYhFqK!;25DwG?YPgF z^(N2u7hWeL-9J}&cuF|Sfwa1%=wtTZXbK{6+r;CcM$1mm+15pjQScsh;{GD{vkz%x z98FS2%LzQB+ylEhPJWMvaJlD=xsYBz`HY_}ap)~dZc_H!tZ^UIp@{X;pHmRm`L@({ z8T~2z;?rft5vtb$S&g-P@OK(`ou3BWVp~i5$5DlFVNRVSRj>0W_!1F(I1lXrpyq`J z(pz4jo#iLM?fhHgaUEW;YHsh!c**i3L(9m98eqnJA|5cLFbuyNojsmM zy9&w{xgX#+#3NjvzGA%LFFwVUYBTYtcg^R6Wl+WCbGF?jd{%@~fwD%tOc0ZgiuBAdI+F89LzJ?*WNPP+9fyX$4 zoFzDjInUw1Gh_XmGiE^8u8d^yt#irtE031bSB0#cqkJvy)zZD~A|fI!Ol>j`!)$0y zUENslM&)hlMs*C{moeIz>%J`Q+L|u&v!GPPOLWMnM`Wh9#!eu1bY`<1VH-YbUSl?4 z`FHoAD;vazdI~J#RUV+$8+se;NeEr@0ERch#c3kI>0H_cB z%+hX~1Xvys`{||=1&UayV0F-ilwoBKQ~mXWHu4n=2c^XQB&c9dz+M!mWp#jvfA-_g zJtCp;z2*=>fSE>>5l{)K#I?C;rx^_Zop!Jz3>o|xIlz4@uX&E7^j1>Ur$8oTbZk5D zP=hAV{%4@cw0!E$P=;|kf?A9KwKFc89z%HllYY36Me>y&%%%x{R(MKHc+b?Z@tKZI zQ$OOu(+Dp96vG$m%rMqD{G?Se_$02pc7S7#3rDU0FD;7~YYy|t8J3-s!NW4EWwM4@ ze`LR#obHWl+ULD-XAe@MQgSqaH$_drsI-Z`;4mRdFs1@D-ahN@5uCp6Zb%=V*j))p zj;((b{5O}QqQjbRVVKQXiUM5$N$t8cMe#l$Ee73Bh=LUa|4%i-b&`{!hVoFs}Uvblkv z(F~D70-}QLU(fH0tA3 z*S}SmJAoXUK%753Sls|F-!Urh?#r8c4OT7u(<`{4^D9o+o+!AZS#Q;_D}2Bv{$o=$ ztFLVAcSn&=rh!TKL*~lDz{eU8A`^xDK0U?8_q!+8sJl;eshc}L_c0{I5XX6QMEoWxV}$st5gG@q_G$2z^SiSD>FKT2ZCV(0vmoW<@qQ5+M;$@qz6_ z?e&;87<4Iskw~(*zoVAj8dGPFiKu>MEN*^Mx7~77IN=xCfy(^fu$5YMd1EH)F}3dc zcZKi_Z<0vY5T_8Liix^ZO-`rskSWcZRoPInO@?@-n;kKoR7jN5e5I}Xybh6{-c|*# z0a7wxVFpQ%SW{8q5(6q{rzr@7UYxP0^i+dzY z6F;;W*DStG2CmGjk02l1C>P91I~hD}!o=_Z%Wdh?u4I70^nd-~S*%<#J6AnJ4J=oV z_h(7RCeMRT~4vF8+m_tRq&v z_ENkV4Mi7q5Gp6&WBxmgpu1^$wKD6zOrikpUTh;#!tMkn4I||kK|%mwlHG*BCJTl_ zr|7|)30_VaC?Fozo8cG2H?h7c|L*PMG^>-xKy3j57yPmCM2{CsWpwSqV@2_X?qEHBd8Hh6hN|jpLa-) zjp*+eY3Gq?&VC0z2&JH`n!3p5d4ZvhX@2G#|>4wT%r=(epp9B4HiN_se2$ zM5WP3m}I})&AYk94>Y5&I2zmuegn3Bws?JS3_R;U#v9;u0U`PD?e{A$ri?)K{7;M( zxFZ%;X3CL{!24Vd|0@bxASUs;d;+Ou&ZR{<$=IbRnW#!&cVc%(Gh&Sd@TnBF#{W(+ zmJOM04(GJNG+%Ow?A87DIZ&VpL$hJJoi58xG}9;;HVXrKv-Kz=@*Ts#3kCmM+9o#& zUWm_EPflAu@29wng>~b18bAeBYV<1rex~#PLRNCQ$2_!3(1^obD+<@0`D6C^V!6^1 zzqKbvSHgq&h)`=loHXqtg<(+pUsE!Oymq?B8iq!2{42en%{(_B_Cs)94|pQ~gY?O}geY#x|2jNN#p406Wd)~`KPmjg?y{y6uXdg#|Eq?f#98FluAgZ+a~ zm%H~;yf~fOgbMrI-zoP0jvoP>$Fm_r)dP&d(+Kc1vp-fI3x_ry0}crb4YNBbXvhi4 zHC@W()r7Lre~GL+fU((Eu_=V5XFBMfGxu1KzcDD%ZA#X8EbkU#m6%&HiT((E<0KaV zCa{@=DSm8%;+rs0NrHfVSd`%Z^45`p-i}j#HV;(z7L=^MJc=~wdt72L{Xk6>rN?={ zfMe}<@zOVvp%*0|$HR7F+M}zPRh_%nSkq;Soy`ukDiQa}-%uDPRG=Yw_1Pi&^)@&D zETmY=Q=fG&p578a+9P*=%Z;8F4UJ!3}JWCrwvJkYsQm=6D@&3$EB7F_@D=`s}w>zYKnzhmZ^tzdI0`mOiaQOTQ6maGh8H3u`mx1It%koaO- zKf@S$KW4NduPuqrmk{kzbd>O@i(Uc`9YFxpl5r)Oh0PUorO3-1#!V8|(n4PpH3U8! z@j*t(d^yD=*+W7`W+{mdta=>9(R`NTcLG+_oeE2hPRa#cmwROyLFL^CIWxVKWRglD zc;0O8gV(8D5ruBgw?QYtYw^?1mBq@8)Q>%6aQuu}TcuuB=4ql|`N}P)CNH>q={Ac{ zRd?Q|X~cE7=$7wKYhK#o@zHl8XrQVDA&5ChmJ!8sn7Ei~2;L48o>crJKnW&DJTM#G z+H8=o&ALA*19~XPrVd(5S#CCcqDFdi0R*B_Gr^xeDQ|`s<=@vuYq}WP*MWtf2k$B~ zN4(|43dMR7^Mch$00)DDeH=S@SX`?&fCQz!$5iHc#rOQLeH?8 z*Y;*QI5YNrPtu?ZEay9u5>x;6kExK^)ncz}Nuq^WN)aVO40*^FYz3IY{0Z13(Ix2J zrLv4|GEvW7m~Rg~NbleyKdf?9%51Q)>18i*)A0a{*;)Usl-DOhRI=*YOFeDUY9T|X zs`Fq9>w;1IkoL$Ntjsu`xu%c1qczae_gY61t>ycc=1`>ey>8pDF!7%C zqKO5Oy=_W(UFu%dYqo=Z-!}r-9?Wzqz>!kzonB_SjmnF;?)K&5qyePb8MWX1{>rg# zXyE?Wj!F1X_jR-Tj7iH;sq2tfgN8`-sa&Ck5T+UH`~5pO_dS8xb<4#cuKMG{fJxPB z3LzJB64=j-2=Q*|sa6yZYVp$7uDjc7(iD9|$H&J6XQY-7PCQ+CNA<8N%oQ@Zp0r%K zaqo4jq_szdyF$2;*tqT#eM_3P!EUh6>GKzz|1HYhJ6VKd1o^R&jDfm)yNNUyDi|qDp-MJA884u|4^=GW2JYzdAWK|UkV$TF} z_ssH|DMQx38`+V_g7fX1ltix9i+?9De&k+{T@!({BlRrr? zJc*mS5yQ#KcKMzkxas6<&9&zIx8@&gSpd2l4=-u#xq77ukb^4{Pn<8Vt#m#-m}bfc z?3^rYaETKKh$9#_kqrq?>!;ZhRI9EPm7*=D_mU}UuLU$DST6FWJW^6_;c^6(TseMk zp=}OcPYG7tPHuHBY<*!px%xY(B4ZI|ykQc`Xe3iZ6>{BE?DI4;Z(YiuUZNv%9&2P5 zwftDrRQPx%klg6eaDOoM6m7z8acn6t|P_1F<1AirNGQS z$th{3jJ!_G#`9~eZ8|a3E=wr%9*O9>FguYV82`#g{`dYHz&p@KT$ipcy(PVjDN8;s zL@#hCZ>JiVxki0>`JCPAbX-%+7F`3vlG))fAIvBzz z5#eXo$2P#?BWHY_FSiuYs50*JdNVj)ZUtk0AJQVmls-fxHM%!)W8mz!5TV&&wP4jL zW`O1bUFM$Ipc&lr{4VbaqzLc9`I}&t`IEBv|aVK(G_=o!uHc%L(Smz-LOS1i`2XDv;QJ@m{8K>^gH;ssqGoP z)z|wm=?jLeVfyJOg|;^&)6f~5bWV8*P~f~xXBA&d5i0-L6?uEvaWLl>kFN3 zTyy>KKj|#E z)b@vfZQkHf;=HbN1>S{KiN-!4j(iGLzOkhDR0?*j=jHvZkojFN0kjnq97YiD8~D>A zC1F}~CirRltHeT2+P`qV{rVLJeLHWx!AhmPMTb9HD-7un6%V()^9==q^!?JJpVa!m z5+7?7wQEW!EKRunT-YK!qT*53Ze;>>MIBtAy5@m{05)Z%;AgR;*zj|2?$Lp#7K}hGW80NLKf^U)8dzQyED9C0_hQ zWq~j)0#nh-z{U{DdCS+gXx)2zxhHmM;}<;@4s(b_J^womr?UJwM$Lv-<98}6E3+t6 zX^T*>G@Yzh^QfWRI_z8u8G*BL%Tf_~s%dK|^(_FFHu=EGwzQ6brx7 zY&%1@<1LYlY+bp2G4OaiEhbz8Nf=Zc0dr3dqs(`(6gui_^ol>KKPEPav*{oaB;Nk4 zU=g!xWGC?UA+*RZ<{jHd|8(2UrJzY4JADgHLwr*%M${ZI2a{l8)4Pu8>`>E$(J)me z|8QShPg7-kk1Exk0+U!t->GVha3>@X0-*TO;BIRNh_%JNP+xxoT=QjwYb{GES)HNC zTg;-t$zh_58w6aKED;%p%3_cF`S2BeK|THIll-hvJ_ETaGrg~@1TM_OzTHT_7#SUc zZ)CXc*rZVUwZprXO(DGP=m&v=0#JtwV#}7{N)p=m8p3K-B4pfmnlml3z59Cq_KT-K zR)iV|AnD!nmFk7*oK+*o9u12uXdg_|`*I3dg!Q>+;#EB6IEtljQN^>Ve&!R;xi{4k zaNy+m)Q{q*LYn+mM?~ZS-poAIKZy0Q>uqK?phbo*0t9O7MTtq@KEd1R zWwn#Gc-^OBcp0V(*{RxSmL0K=%<}*VJP?(D;p6~5ep-ZI%g|xH!^P>IFFRZHL+PM) zE2mcTxM4NYv75A=Dn@q`3SJ zlTPZL8B6X>yOlTnii*gnCYg*Te)Z`nByU7i8`mZc5L9IanRbMura$^#-9!#jeYiuY zN*b7{Kig}rb;XXH2N3EU;M?(>>J!w2y>k*S+Pb7cAAU#_5Kx^ z%rtpraN6kkO z!WRi2Hn+`HTo*XJg0HbCG3x3aF>No_EYovabA}B3QUWz?0Tao#dhxF(Lj(-l$|g2n zD$6KHM+b43mryWk`(GE6rjeqZ+KN#scy=q#D|{O2{h|$)wN>wRpG+Rhyp4T*B<34% zLqm5}_ty;J(4UX$5LG|wEYy3LhGS*o#y#`8rL%|4`}O{2(0i!1F1~?MJTkQp+YZfM@p5+q|MkYk9@uh zZC>?`>yc{MfJs+Ump(TI1i(?uNFI7N@p)t&F(AwXC_25~`RI&Hm`}p?d@e9*rdy#aB+92XNs*e;&l)+(+zr$x zpFuTxo^F3Pa-?eQIsZ0fkhs|mdkl$Kh>Bpj_?M|Mo${!KtobS_{K8Ymxxh<}*h%Mw z1B(}a^-H;6ZzuQ~Tl7yG)%Q=1aj?72k@XD^BX@7{r^mkFy@!-X;H68D83lzE`;iQb zI;M##do# zT!&i9uDi}<$%{2NLmTZ$AZ3U8I`6^)Q%E-X;B8>^f$*_plnJ=n-h1^#!)X7yDqU(i z{pLB(mgS^;`4_XxR*5q50S&)#>}L$Ys)#soFe=dmij&-?f4}^wJF)x<&W?lT%0Bzl zAjA&$$Wh!YAcrRqq7vT)t%!v&bSa~=|Mo`CBp;I*Z4t3sX3BAyg$pMWc@;{loF9;J zQ?T2^$De3z%4e(MP)Lk!_Ca64t11)*2rGj#Ka549G~TP1xXHQYOB1vA;ZkQ7>szxJ zb9I-RJX8-4mqs%+T@~Z;uJ8WwOPd>Lue(u`UrFwhBJ`yvu4x|XbswTr)}fu0HtQz} zxb%tR29Jd>d9u);)LuxJT$)3E8OlyNAB=D-o2FT z@q|-kn+F)y!z_Ytab30(YSN1RJa@Yh2wOHkqQzMAZE9!+d~rJSaI@%6Wv3NtG7I_I z?fDRv+xuR!Y6`a`B;={+=Sn2d<&vI^rf6~!6dE&d{`bQ1I((P7O(=2RjL@0ukAQx{ zcaQY$m&~$FA&P?OWmK^5LLvPL5uaqLUT7l|g4tg3sq8A_D5QiYYvfPHzoWi-QUKvL~n_=Id}h zZPO(o!Q288l8y|Baf_TM9}ax&{%tcBrCVu}8R5pl?~8H|4~(wSc6c_nFFc({B)St) zTb++L{{!UXOx2~6Nm36_{9gmm>$T#N22PHY#Gb+oce>3Dux&v}BukqGxiE;%N>oe( zqLNc4e*VfveT&(q{N)PaR}{af0#M=ZXeMRaC}Nzvtg-akfdzKg$@xfro$5vn$-Go| z1H$?JEfp4gIscFF&*{PR$P@?Uz?JNQ&%ejQ&X9(}5=kCXoEC;R?73gIC>$ds_$vn9 z&&cab_WWi1pdhpotRShD{p;b&#mZCXHis#22z8$Qg9!{s@q?lLGHy~7tcejp`hymz zn#rBAo&~YX29^dMpj!u49!v$uUivg;jjZ1ELxHg9av;g{r3vfU_9|5LQkygmTfzUA z>vXRyhe(%c7m5oR=ky7SY%S6qoo##acUvo!%1Bw=hWdyh-bJ`lIwj!(;!x%)KsU+FXyPBZ}QLTK1;(yPZ%Nl`hW5&HV#&oMAI2yAAu#htL4Mu8dK zr7u4iN!L%cG}K9GJ$}11r-r&wV2ApCk5+sW+s7d}1>*6pXxJ;)O;*`P$$5OMnGUY+ zkC9qWGJWT)whK7m0`cmSg#4Cd#N0?HEp1R=AM!RO`)RE(Mt_DEk6xGlM5v~pk6@__|9d+nU< zhu@q~oxA&O#fPLt=v8p;sRrLeO7N!bWGX&tYcHP+M|j*5MOl@PZ@}q|F&b zNQl|8wuy97q!{Dk$ZNK?+Dllk5bBaWUNZTCBpXyZ3HhJ3DT{YhZDZ~Eqooyf1SH=9 z6wlAv+ehxYRY!ljP_RG5c4#L4^*2MHJ{InqDaQmd^@dV-W%q)%$8Z(XbpRpd?; zDweq%_cU*iN-_<6MAT1X_aq}+pjZvVTSi6AD5aMYqo;4|LXAHJ1rA3E4Y1mtw$`ZQ zr$8uFr0hPc4U)8Khml9yyH2mky8JF;WVtt5Ma*e+p@v^dYg*uGI{JE;6F+&I^u4tB zr2-~2g$?Y~e2JE8rGQ41=%w=zR~8>=W_Dx?Wn?V>gR-SA3?>GB zcLR6t=e#o+1L$cyPDmswY;ihZV;e5;kiV!SQF3QN_M30Cx+(O)h}chn!L`@^*Fe}& zEJP%vqbz-ySWqWtIEcQQKIv};RRs_A^@8PKYc95X`(OY&k13gF+KkUTlMEF#x+vlU z>SQr~E4bnUT_!JgtvF{zufSoxNtfxr{HgBKJzhzp4dWwpx3)aORsAnXlb#9gZohYn zTvQPmz67?dE+eSsev|+6k;`qU#HAH++%?MF{-JH{uPM@0A-7C>s&{Eqj$=1(p)AH~ z&p+TP2H3Mv{KB@*pCf!jVYR^ysJO7J7W9=)TSU8XYP#0vRT z-pbV?bdMW&{&nc+jwS^95rIZv?R@D0|k#RJ6}6s$rtuNrC~ z+2g&#QI=RpwW@u9bi4oE8_jerl!_o_ee~gK!eMSc=(CA8Sn+TFQ$@#H0C{mtX&InV z^p=8Ur!;3Apq|CJl)BUfYL+KYNUIn+mPBf%R_L}p8mc%Gbwsy{q0T$GJl-$y@Ftj1 zKh(=cYpE*&t?e??QgEByM)+MIIM{$jdsv?}jTO*{ATmCD9F#rV7f1xTCl)Ox6xJ1< zKh*BjCrUHp%Uv^)KH76n=tYZKvRsJ8w{BT}6;9o1>n9)m6B5{j=~3NC{y)tmGp)k! z$}RZQXgU24)UrsVgA>-0Nj>6PlS$%;7)X?tc?mXIEBs<2cdsl}N2~lEMzAsSTbNcM zYeDwfRK4RW&n_LWe0X!s>PYq(o+47HAYY9gdMH&M>$@F~{f=TDUkMagUe_msi-<+y zu<=c&UPmsrW;PsC-vv-NZp!pM`5&Uyk%(FvyPF(#1+IYF>K&Vj?|d6q|G2olcrm4c z={6f1LMN+SbOr~gXavbbw~mB8-fv-PU4bz_pRTKo~^3;2RT^UAX8F8tNROnOdRgcU#*Qp;BNVLH=#O@ z=s=!j_jxzjrUCz?yAG~JpEYkiMYNfhF9Oavs?~tkt!?SI?}#?dxV-3$8jwyJ^u|F@ zGEmHZvaG$Lmblns@cg&XZfAi-$nZyq_Lr9<7w7fTZ9Z4b|C_D_6`C#~S8aR~q0o46 z<)ey8MM6*<^3`*2hn2Uc=Xlm#E${OGo75nij}BD>*yViLBD4AHN%iO7bM8$RI>()3Mj(npFQN-sG zeotbEQ=R&C)kUdzf2Db>?*Z4>NSa-d#9CKaiGD91X5&(lS2lF8b`xT_8_dmC9r=!U zXL}DzJV{A-6bni&NaC#FHa7tNAmW`a zs9bRh)J*GlCvcla%j0w1T6}-%2pP1j?2)in>CNmEbBsc?NbrfUk}l0T0JfKO+wAUO ziD+gr|K`7>P8KU3+b)+{^2+uU^HthOzdZnLD0|F68m1sSh%8R*twpL(<1Mz4Z3eb` zU!=>9$nG!x^2-|aN*n@}zt$N42Sb-IrU4>Ah#uDqgh3ol?23`byBRwI*yQQZU6vgS zbqcAd7%Qj-W4=k`?6hVRoH#rWxl#FM&BBjQSU%Qad=dW&LeTSn|j-{z}t z{WU-Ip#S0RwSGG5`&Q6##y3V7B@d<9#vh!odw`lPo}U|gn7PL#4yKi}gDfsZxcWWse^v?hM04vxC`fu^yb}^#b$eZ#WaTPHApiI}tADcP z=;3@pOL9?{hh@r^)(66lTsu_3KgaXXWv=1z$BKyLnIln&I_}P;P|QUF!OYg2>G@8K zN!J}#v!tdi1*1par|0n9uIdTNVqo&@WJ=ZwIOF5DQToKCma^>I*N@c^NW#0$|XFtv7l!Q z)|OwN>18 zPthHjusiwKb15;1`@FCF3ZPwmT7?#U=E^+%L7qusm_QUA5fMFq6sZ+1sDJaA)OFQ* z@tv3LwA&uSS}ssC`+p|}7aQJBx9QMNU6(HOY=|(I95S!9Ja@Z6Rwgu&ipPz1`{(d( zayyI<16A*EuylPypU1qk$NCkh`z^lt5v-tM_$N_prh_szWe)MzL>VRFreD%ZNop~I z1urxqQ{-ZH%m$v$tnLuLf{Umo+1JR_Pgb)Ci%C6MJ~?6)0I#iJQ(dMzq8x*irdt8m z^b7^#{TJOrMA?jjaiRKpsE8C9y!ppzOISm@Srz|tPnpG7!_{&`w19@0S8%-=f#uyV z8;5U0b5$x-lWV&i8I`5%$|4g+!-W*rIb+c{KlkfQY_|N5y{{-mcdvMVR2bO~nhKyx zS=p#QXZ#qOTHU=ljjv(%IUs{<-3I8M2WoZ@C_)b3W~rXngF%6{gzYU6;p8+5_+F^+ znf}|}dJ_5HCVQWOeU3{Qoh$&+LSvZTSi!(FWMGR0?HPTB_&bskZa{)QA+oqVkpuw5h zn@OSgy*g8R-5K7W@C!ZHs>r8TTaYNwbJ~4rduHsNGN1UA$1iVWH&JSurevSuf5T)1 ziPO8nrP|vy#j#8O+js-b^Ii`1@~`K`XYWB&b%V9KPvqe? z3WiFLIueeS;}o1w`-yQMdB>Kx#dn&^!4LkpL`i76r|0@WXZ*6U*xy#m* zVGYTY2NT7&96}!z=-b_e#3uX@+5W08yXCJc>(`h2Lu<(I?uH;8T?sF|$05?Vo-B6I z)2pbkS`VGw21%&omv#>o3x+%Xbb3o_wPILxRW?!@NB=X&p;%HcI+qn0ZpQP6|F;bv z#Hu!bXsNX!S?#?UnrU1Of0H)opDd>>!TAMjj99esJ843?8)=zV1xOj_Mwfg?{(fdlL2`aG!S3-o3KY#rIn zk00|q6DBrrI5ifWs$PU>VZc?y+WQci zk8@H@_Oxj6uL~2~3%Bp5@8{3220i^1K3zr**_jeEwuUgW&6bWhtAa)O8goGQh#h3Gzg@LKPf7ir z5a0;S(_S}aeJqHYCcC*Nes`G7|9H>uUd-#ZY)`9SakYUT$akZ50a^GJ%K*-ZPk{t-t)rfjlb(`& z8LhS`aat)@*}M%_6}iQXIlGut&z23{?gTwv$b_!S1{SfMT9|e^Pyk$~iczGW-EK{C zt>tp^u##0KMcfF~)Y8xxqAGZOfdNG@0Xa-q}4e%wGvMJ|c=9jnRGhc~eW zv$ElxOBTE|@JFa78%v}MjS$pmjpP!$2lV@O@$WmS%(#_jSuCFbl{XC!J;Nm!w92^H zPFPOt{${wjU7Y0kyVl#VI&JOxM}u-UZBDs13LH0h*XgDYOtS-XV+gT|_l2J6M5beq zF`c(HB>Q>`o#v$XH*W2{y9=6L;1sKgHgaN>vTI&0sbzdsX=U_7M7{lR({);c&6ARi z1THF7rnDL=qjzfJ-^HqYc-+iwzhT5vt=isLnu^VAj`SP4+hfRPsP=xO=!;iYc5CR7 zvVP*Sar;*L4J>7;yXZ%e8BAy2OSTQ+R?doZ9aCX}r_P|+n6>(|$)o#7PJ6g?Jl1~V zErcBY2-5Q-r7(u!X}75ZS8hWMWV_Ov<4js!efs0Xu;$7s2BW3m+JiCNW2Ewoe0Hf+ zym$ppf;NTrMt&!`;CMFV(A@>3<=lZWFGv>y**?+QX5Pyi_44#KhjiIBl1Oi>9|vr}_LteWb>cLuNr77`bjOo$!%61nhXI*E{Q{lwxt z#HI6-p?P_5b6`p+-xH3tB$fU-#e$Pz3pjn>o=uv)dWIEVLggZfYFl;*&8OAyX>bTf zKLUw@2h8j8+GjIwTb2Otk1PsYE)?B4D9BW$UB8>pB7`F10ej&vkL%~0#p+#C89c(s zFwO1%wUdaq>b79S2Y%f%tkJe51duU#|mMxdOSZ+m2nbPT!UT!QL z?0LcbnVXwiDy|wLE*^!>9}-~=*l2E!wtgN~lkJriFW5!TojYGIr-HWMII@)VflG$? zcdJJECRj#Ep7H3Ec}q$a5C_+3arXg|Jcgi)AUMTvs03sY7c$~~U#Fnc-tnrVbw0$A zXg8RRzItn-)QEKVQsE^)(63nOLg`l9xTBkn-oMh^)Gje8myj@LNl=GqHgpzRx5hSg zwfVCohQ*7@?dK-v07B}9vIzGkNQ5rg=p#bXMcaudzAWEg#|)TuIDb`pZST+;vRG@@ z4biRq_2p3Ovh%5IL|*R$-{Yp+S^=z*99z5|nVas}4UXA^&y}oY@P9Xm5D+-|daehR z7WniD6DXdZpJ-$1MuLd;N#V*#L;(s`RW%)Y z1Q_rj2~U3cT3wDNxh282$1mh{e*tV(h13OZfNk4UNvF~H6TpB$!UJ9c-mL+$=~_ao zlW`cnO=sjeVE?qUK0Mq&>XObBj6G9VE1)ON+6CqR`*a3r!9TL~swQYd-(E#c&|sPn zaNUG}3|FoVT3;-Ue0C^t0p845+v@UUajOUhHN_mJ1_&0d#fC^m>`exVKmUe!;qCVI zq_=F9ooRiURn8ic7S`B$bgV|!ZVKPrFyrL4vp*ul8Tmr^XO`_|0l*Lx>uYE0YS;HC z8|uVeD~=SsLEG;DJlJO?7P>yp6bxOdH0~K9cI$X4)%|6$RyELAI;4ZrHs3nOy}E3( ztGuN2rwg8qV#_v65t_-@a;yorI>NzWUR1keoQaeQ*ik!_Wd>!zG7hx-)hLU(V3P3A zt}a%+$FKXoX>QH~EzE`7p_sn^8^?n^GITc`;Nt#+NHUXXYZn$vQo9*n_TfoHj0@c+ z_P9o5F3++CoKAR`3Mf}yiPsipAdt7XUKanXChHhRG~EIPp?Ux?1ty{W0e^`Xljf|% zQc${FShPyfT&!d>YUkpGgV%m!>~k?)9b~WZf&nZbAU=jziNHt_13ZoM2qdG3vtBll zDg*GfJwN&7R3(6JWAsP8aXUPG>HpmKizi5^e1QE^0w7nTB2Vgxq2^QXc8rX|pNA~% z@9bTqSb)^v2jx1C{uc`dY5-Hm(C28_hTFt35dgrPHV2WPIAVi|cTd$QsDH{EsbJni zEYM54w;3Ne437Z&Fo+0wi)Gq*O}Vr+RZHhIq?;oCFe^lXPow_bBJb1M?uDkSLbq%Z z>^z`qJEFw?ah-5ngR7DnrEeI?A}8K^sd>WSinNv3JJ z$&27jmW^v}f%Aujc=gYPwDac^2P zG5;mH2Q(ol>>&ahP<&b-j&QT0g9f>0@`@@|uY7lpC#-SkS&zhvA0@Vl`|;0GO~3~r zZ5|!fT$*#ASgt$C;0M>4ec~xG$tU_vM*%1hSj;_Jy8T?i@f8CH*BkW5G#ILOjH~O>Xsi`?DeJ{tKSW_9EBcM% zdcbbXYl%KvucTc|!G`)v z_R{cXj1k_iluLa+fSH}kSa3uKy(_~37-ZPDaAGAi>)iIkJfA5au~%+A{Ld8g8_kQxOrK6|6{#KJ>rsF{Ljc7<_=!PAMqG?Q ze=q0YB|ewtAB>rV8UHi>=NYy(w*R|z5Hr+pcM=S)ezzLH8<}17j7IegS(nVte(+<& z<6uL(!DoNzLU40(Ddq|I-}tLuw|W`|^19MZ7_Jm8()>=5)$8N`quu|O%{z^gs%ix>! za^~`)wV0>%nC?El^Hcu*NN)4P1h-2&%=l+h?1)evwMXIKF)Y1I{GxWb zoLuWM3cha4b8U;lD>9U0UFpXFDT6*ezi!VQIEmP*^-lL|MJ;>q2ao5=XUlJuZYP<=gGTNfe^_NfFq8f7Kr~F;aI)#x3}rf^@1a@y>6P8rYm;N zabXmn?!Fw+b@|x6VfRYwUY^@_N>c2@n6~}=FgD8}7>n56=>}U9^RVA_vpbBRP#2$B zj?Vo5d)h0^(=>CJxn7kBGK|{dd?v)IQi-!@1bD`M%(~g9Wg8IJIp1{s*4uP-jSH=} zOG)&uIKGT>6{f^sh=Z_`p{xTO&QqEWQY$OiSMNYtUza#T$!bT6JFqa@+o z++T;26lc2bvu6MMuS$2DS|`d^cUlwg6-7 zWJpY{(+I~TLHUfi%srpfrMTZuN- zbhU@BPtA%hi4z}3w7VnOPPt+{K8uQ1bu$g)vwlm;9%qN|c?14BNMCH&K5p|h7swAz8KInRzqv$A2Ms%T=}FgtjCWxR~{!H4J! z0h7?w41s=UNeGu?<0{&*yj5{dyBbzr=8Af1_&33nul~y8zd^j1lw>^FEzz9+c~E-2 z3q;xo@3cx|+sEcQ&9(mbt}XB`oSbC0nx2uW=$tmJp2Vb@TW-@vId+zSmy#@@?RXBXV$utC1>0B>&uVknUK6R%tS9`Av4#wE&AiEcyH3 zSETgxH)@rQnal4Yo-%6>!CE`?zBudN=Ao;oepzU^!@VbZ4WUj*h33^{2!8m-nK2$BQ0!f=f#^LRLcM40^DS!Z@s^p z#x)e{e`y#lJ-}!*laTHfI@O06%+A{^o5<{9?0BX!Qp-@CWYn(a%;4ZsOVdmWW@v?i z^_`n;Z1Ft8&f#~R{vY#UK6rL=!fRlSn88v1;H1_B9l}lw_GtcEp1|K0g=<5WiLvy3 zqfiT*h=`|Az z4c$6iv33Y7s2F|PglN6#LZ{*I-)=ipl-ZacN-N}Jj-4#m8qVWpn~|nV${xe+A6M03 zzK%x_&;G$FQ!${pq!3Mh@`$dTw+nMLAh6rSPc8!B+Njboo@xFA7R+jmc_7gHv-w8# z-hF%frH4C<`hgXXVH0aM1@vz%C#B;}e(}pCg3DLa-&$%W6iQ@2-xoa( zyCY>62`WaH;R^|in3D<`r~(B}5j+kqljKDcLSwVMG6G&gGEbY5C`^0uuNeT;C$2Y% zLLH|0ffIr=j{D7*;qwmO=BdoiX!@%fP9MjsEKfAz#(C%+sdd03R~Fk8E#Iz8_K=R` z&;NSz;C&UG(?Cs5j@C9MQSUjJJ_VmQuUF4(mQ0!}-8SCL&HmSk9!w|9V)|Y!Uc_H41#(RS)q2!^F?Qr;H~Q4>bx3 zGR1aJFwg3o5^ogS)%=}x;;=hY_hlf<19Lbiunv*MVL2LRo?WG1OiPw}I)7ZfLh5^D zEJkL+d1wS2Tk(d4)P|!(hb`+=XJEvqAR`K6r?y@epWOK3R5mM%X?fG{B6;P(Tn_J% z&+-=&#)i;kQT9` zkrUM(8e(Q-HpOZ6HkH|9)3%*}=6j7DW#|Y|$1Z)S%bu2Rw{1WwUh-qlO=Y}jM}wel zvR2GaA%R4!)cfCkVqPGwp61OV4me>PmA?HpqB>a^z{0&;aXN{+e+X`FL?k7^HOaB= zkHsQ@maSJ^Df6}BCrN_w)Kh5JGn^^1yNZ@NU-xsA4cPmvxyISl7xwRetFPmRtnsg^ zU+Th?n@$NLi??%i!0^tz#~}%&r`{e`fpHHcOA`W4B!;0h%-F6fOmf3aq4iRE=-Hdo zrGk@cY>QzDaPBWyVb7ikZV!A843nBhaf@hfR*8DReS^G||2qoIvHq-0od1%4HFhQI z)A`q;-rfta!dS>G6%z}=PYG`Q{^0+-)0Vdx!+O&R#O;8F4_ja}32D7z#_2_akml^c`Nm^CF9HogHwajQe_M z>s36!+G&osxzlVa++W|Os)p%*MG}-XDq~c+VVT@K^8Q=zXndB9I}dj^f58PzXDoeZ zlKwoH7qVjrS0d}BI12yX)*o_2Fsus|>M4(IIkstQNT8i~GHKhB?BtQ?JhQ_qntR#+ z?0K{TtFVIejA@!TjVAZ~JEu85gvi7lw}JY6LO{R8U_R%NPCJ!V;nUf7ZnNw}P`;Q; zBe=uy8brnCFbm$|$8U}ax69Zy5dtIU5l*1M6pT|U$(mTFIF0yFDW4Kx+S`gNFw|bQ zdcJolvs3?z)-M=DRsAw!ITG*m_w8Ce1dFAIS1E?1Yg z9o4Z&7X+Fxn5+2^l>{l|Y~G@PAC*3JeXuy}aLSD4JGuCpwI&FB^xsP0el_~UVHp=! zR_%_W<3UO36mG@3YV>XSGR*Kfc4iSMadF`=D`!)Zr@_Eo2>GUGIg6pxX#!VKZ(>&(jU53EZ)}BG9DsOx{+Th}9C@yUM+`3D4mgd(1cb6}yQ4E{E-ROZrN|?%O z$0!b99=_d{QW)rLpqg#GTL-XJgZxZ7`TA?9WI1K~LO|+a>h%#ypz&b*gy5WkG?tNU zuOEHe?qTJVMAU4+i@)hC_8$ksf<9TvK*=7m(TQxeg*R#bK7Z8a>&(8DG2}a}_hm<= z6bS<>vx@(1p;42E6Rn1_rVybg27_<6)9E@RPtDU$V7EH-za z_*tKFf`Pv0&V5|XbObC8ZcWlUC*0fX`@lbM1V|DaC7+lGdi#6M#X`nB{LkQO=7uuO zgrU@~lRHD0qy01j^ew3K=I1djQAQ5Ch{*iI)!$uRB8NmUxikZ#PlyAbB)Hr#*^Z<7bH>#NweIE6F|F`5$dZLF&(7{1hAP!j+W*C?jS8_-jAeu|MfFmV) z^0|_I5y#{QYDMycIV$1zdJ@2)Wo5j_9(Iq|snK5X}AW1J|GW|Nr@aWi+nlAJG!s0AILgR}Bm|G2i`v^y?`;`{Mu! T5~*%nOb1}Hs?bttv%vogI>yps literal 78954 zcmeFZ^;4W%umuXkpg{)+9^BpC3GTt2;O-W5f;$9vhXi+m6M{={x8UyXZ<2FP)qD5; z1+VH>ef%<2Gkfpe-K$r3ua9tLMJW^{d?Y9+C=?lK2~{X4U?LP0bRz;BWZ-q&;q(UQltJFW}HWT1i^ufs_ z6dlXHE!v-_quAo(4*x~~zqH?2s^kskhI@h*S%5*479d4|`{=!J(^W6?A(XCEM%sI? z2aS~K7*>6~)%_#&haSD_hhg3|-d+A(gCz+;L#u{#JoT{TvdXM6ill^+w~5&b5tDV( zaxl|n`-+V5!=oUkAK16Q4%dky5QdNAPhJ#n$U8qpE>DD$DRtRn6ZzuZA*+l`+ce5y zb;3X1SvA2eUlN{8aRC8kfE3?Mkmp1{wV)!YFk;L&-63`bLK~bw+q^k*ld&NT?m)u2 zx;;S)nR$F&#TR4ev32HHrzX}{G^Px}S9hfh4J)u#;Z|Y2Cmc)OC_Gxbv3e8Y)=XSe9;ObE z^US$ffa(fy0!}sV>gP@N;=c}ju=;KM`-iM=X_{fWlg3Wa%5a@d+6aI=*0i%^bSnTp zK)Al5?JMl)Bp)|>{@V*VO&j~lZ=#Ep>Dnr9p4rm)4Ppmq?OKz%6N!gWTio;ThTqEe zP3%+oSD1gYz*wAzZOz76jLTV>JaUD@^_mxM<73`|BV5v-R{hvm&?U7eAwjPAmht1v zH(CBdC2(I<+*Kvo9jq8)sW%Np< zT|ZZ~eP3hu-F#II)CpnRz9`qC?%Avz=?eRs`#hQYmgM08e zIn@v00ilt}=JoN?m;9G+_FoOZ;!LmcTLXbA7A5l#@`b$~Dk8e??y6u9vx;9O22A z9YHM}ej2IN)4#zEXv%K8Gq|IwE{3^&Nx)>` z$aW{X5}~i%{D&Hrx?eGUkI}C?@fSnR6n%Es^CRJ}TiQ2v3#7N;;y>_JuRRj&%WbT< zHcfa2+^UaJOYjqO)&@3bgvFKIGzGpAmIP1@YdgfuV|TCe^ffuFs3c8r!K8TC@L{j?;DFDs zZtBDH!wh{9u5Y3bJG5}Ia01A=+?=!9e1XL)uoF*ZzC@^qM(%jmbP5+SmR%r*De*8tm z1+y!~BTMz6Q?ji84Y~Eip2?1b^$C|2F=8Cl|9lF`(WQeY*HR2I?k}cLgq@=ANwH*( zr%*DzD9|YYI)|gmOwWy-g1>Jt6=@WJR&7zf!kzsz@%DtTL6Gd*=LMU+Nda7kS<*Lh9Ux zPd2f?h&r5RNpBN(d5}g$`!EuAmy)J-Uq{@8y-By=K_`C;}+4~Zg z`hWcz33!J@*<@y%FsZ?eC`Pr2_SmL@3RDwQz(yO4gb7XIc#i0@7Phs}|C99=?(|G= z?l$Wz+cfikoi_mJ9wkZ9iNsPg)q;E^XduwtxptLHwpaEnzG08PcR**t+6?QHf< z*KORsaD3HfdBUYrapz1LU?fpENNWx0tvt@tglD~h$;lk2x+IZsF@VY+`H@i7ksQYygs#lfmAbiG@RqT($|KjmYP%|XKmW@117wci0wD5RkuT{V#Jsu z7AvW3)BapITe3(zn>m(zqU8HI2iic`DRNTXHG+<-G=Fp3*ag;qkNGuJgak`g(q!0m zEW@GvYh1%IGDVy>(EZMBp4Ywru<7Z)n~OUP1oS29)a+N~9OGY#bDzAM85jEDC2%sf zgSlL%Nmrq{Ion=2l-H)R4 zySl%{(C0_A^XVn0wA$=Q;CGTaf zUa{@@`Oj@TM+Icx9a-RBtdN;6HVfv@-Z%~I$tW`<9Yu6s1({8C4}B{UDl2b0-8H<~ zy3ikH3H=uo*aH8Y@tn6B=kFPvfM?YO1yhQh0_k7d5VH;wD+9GA8%Nk4n$|_7M#}uy zb1HE!|4EY~B1&FYHiY&+pRds-tSmBNUuA?twaFLu)$Gfsmyvv{VfyPI zU)bam{>c|2x|q2?R~uh|^Fn{jWl93=^4k1Dn6v2b9lDs zZMV2q5FEz(H@VacLhH~pJX@eozdH&B6QS&*?ZZ#RUB|_ej*0ca4=YRK*1o-9F(n8B zLjWrpv8N=<$<@s3KF|@XwW^a|nZmAn6)YU7_i0N|C#A)D=%YVI@jvc?FG})SYmp;< zzMDU#Bs}IeO1$p&&EjBtM>#93Blr@NJVo$Q`hd6iA9G3P9wR8{O}ifjF4jIN=)>M~ z7V0wwGD!L5o$BEfo}O*9$^a}{JWiVKOf26&058p1ct z_}zbi02-JpZ%wSeu^}TC}h{3^y(M%t$WT`~+RCY^nl;^MKiM+u?w<-EOjAK~H zAczq|L$8|7zLRQcc2B=^GSy>22|38$h^R+|rb}D|dPj2eVK9bqe&G)Ld3Y+O<2>5# z4-Yy|;=sQk4zo>Yg}d-aS7Ez{(OCF<6;#kQ@}JrERgNfJYn$|G67m;onZop5MxxbY zn%b%H{^1Um>Hydj#%wd2$HYIRTquzYkB!Z2v4DCsKx)EB0+BzR5Gt zcO2l3ut2F7l^N?kXt#>oDYpMXt+T-^!^jR|t5H!XIVz6E{Or0&Hkm^^NnHy`Chv!; z(FxSUe~8b#FJPTk=~XUtnsuIC)8N+(kR~Za{ooGMQ&d?wVG!ZJ1A`z3)in!?YI*@Q#Q<&bh-bD zr`&ZQSpp98U=x|t4^`T0*gO>}sW}HP1X;?~^@bXXe^)vZ0#9f8xOWQ8jyD1;MK3oB z1_(?;n}|QLTZf~GcJSN_!I8zZ_P%~$h0jjm#TeUbfD@wJ&x zDS25RO-8NcIn-5dwUBs&!c!@XCUaKIKD0dOPW;2;X;r8d2tfe)Xy*pTR@gV#BiFH` zeYAFZ-z!Ckv19h%9ur$!K*v?M(x`7qwjSP06`e^Bx0!ku#Q_6@aWd~*;sL|E|6rmQ z5dsq%VY(IIJj;D5LmPs)cZlJS7wv;W(qS;CSj2SM0M)92eRQYdXOjWJIMcX?5AWm? zs2#D25@|BNtdmNtDpDQrMhFL3khc47XNxYDDIWx0om{PH3MNoMSk}3>qTrBBs%~3B z5F4#|q8ns30A9eW9t+_}Md3{j&E{A5BGoxlw3?v%+P$kO!*IPDc{%&8 zFSarNQQfb#bfh6i)aZTaJ7Q$ndB4q)@ZVqvOS-|<++v=>uWZOEP^UKDZbQc#l z8Tutz%XF-H{-lLFloVBTk?kBRt9VvownjA_cvZFhysrZyB91+k^4AIsYr9$5mZlL62eJOGntmeS`wu{?)pLCrllidLq??86KtD%5G+U39lbP5xohB$tRm%-%bkcGHxr*&7CH zqP)Mv*lm0|xO1HTBaeuX4Bh~onXR~#=D5W)OkgHslr0Vkx>c5bB6xl`iRMgWO1M>m zo>qwbxee_cjy3`L5+k15IFR(kS+4YQThDOdleF);Bi3D3RUA)_z#G-|;gqeD8)1c{ z5i;;^j4(yvV>SUxpJ9Hswgo@Z(^_(eiyhW~Z%`0OmcpDJ>)p)oT!r|3?v%LV?#t~B zK3UYiaD(+H`XX!(XZWEN?me#2*TgrJTS-MT4Q>PU;AJdC>xu4ZF|iAm#BTy9emYC^ zsD5KhQ}Io#--`6n&eEI{eWD7_DXK0R*9^;n#_(6(7jwB1)^8sn2X~>fO#M2}ivpFs zR~ok=&l|9;kkeP9D)nTT`G>b{aNO_LejK=>|NOLVTEgwWw>>sLT7R^`^=5MELo7{QPyPm`Qnqk9?epzz zq!nlY7m!r^q*0(|;ShQjOi;#rpF#5v{|f-trUtxg{wk(xO_3#q}${6w7QF&{nitMzWc^u#SOrJd)^CJPQ53nE1Ok(sX|n%o|4NLHf*+K0je zoQLkbETsj(neJzDM@emLjoEU&0PZ1WFm zzeKN2FTtOk>;ul`KIatf|8$5?QB>Jvq|*>}5G}OTrA;Ulo+8;GTqfuU?NM2c;y9Pz z_fM~6jRYyVTb}i%y@q>GY&5< z8FiyXX7Lj4{xR`Isx2+slj?eHicwM~_ZTtE?Tga~5ASJN=5Tpl)x@TM>q}^gIN|NI z-udfenbYwzmopgl%cYI(fd3#QJ?YQ#5p@dmn1(V5q#K2c1d1XXYz`sRexQxVPm`c_ zEN7A}>o6_GXN(J;Le6eAj*FJ6JW*-G(_uL(_KMd*KM2jTQzYR#j;eF~AYH>Zo!k-L z+R|_@^%M24RU<)NTQA(Ja7N!2De)u2t7PH+b9{|Ij!*xKtLg~mN+eO_<&59->c)cX z*if>(rCTp|K%B7_MlubD{g4>-`llY3gteg zh?(4(pL-}*k$ZYIzi?xbk(y6>Ygk0EuThiKKI+=Q?f-N(41^~?UQ45cWZ6T$1d8z% zV3d=O9KqG1&x(!W;mzlYe|)wc8+^7^Vi*#5@>*Db%J<@BG-O5fUp}q*!|oVrj%Z$$ zTrW>J^xILq402?~$d3&T++i;J+l=^r+S?)QfK8FTtI zxC%uZq;W8)`}uB6VBb{jg(>0-BH=5F*HD6e^q!ohX>wJbPvPmsmYMUb;EHJ?P$yPo zJEV5gC};lXQvR>0Td%4AZZt>4uT}6}H&^FU0(tLni1KuJc8dJvf7HHJ4nm_>DCb+| z{G)d01QcX_il~pH?L{{}4p2EzZFt{L`itn4;|ZN_%c&lqhpzUEw3$z-6mp9^{a8zJ zbsOSTJ(V2mMeGs2E=*9--k^%x1 zg{p2{B6CRp$LGPzq()W^1Dpq+v&d)rjTdzDAY|;~|Al!|2tcvCpeANzj-zQ-t)Z|< z@~@(abcWm?9hrtA0AD*ut~(4F8xlG#?s%WvEE$F{DczAdSwvwk!_Z zxkw&{vGI4l(ZO5qjw1)sm1G-yj9Q{L`_6ZzCY%0Gc@NGcjhp-NiK%5Y+{79^S(IQT zzJSghkpE;ikk$!6P5c9gIx&BmGK*go_=)q9oRjt!oCmbn5lxw(85|DMF7(G+8N5c8 zf7MyEDs)-8luI=+MT*Z!6bn#BeEbJfz=RaJ{Q~U`9}m7&ecbDxL0(huCsPboMM0PR z0_+>pNh4lSTV0!Qw*x)H@e65%;Jm~J zAD`vy>hu1-WFU|XY#~4>S>^+I;a6>#A*!>vK75=B8BRcaCqsgRpqBiN*eH{8Sg>I> zX`Sp@j43M1WLXqyHa*l!o8+1Kud9G}sL$4$i6W?Cj-AD&b*C;HTG+}fY8q#8aDt8} zPrS@`F5a|e%{*1qN`fqTpzkK-?isrreu|o}Bk1IC5E>iF#k861PY^$y|Mbw?6o4w) zp{HBGWhr7pUy{>dxI7w&uSlY* z6?dTvRrd0E@4>DYr3y?LC~dSwBu9n6*%lVMVk}BX+%Ex0lYknty6osTl;+L`-Z9bSzZQ1+Ul1DU zV#M9|Q5TfkEJ|Gsq?+)lrO8lR;Pc2XvQb5ig5V?>4nvYmT6`yN!Pl>rWiPcW%apmy zpIpUz>!&(`t_KZdkW$}9f{eAl_^HMV5VYrsn*WUm+P^|R9FqU5+HDe zLK5JBZN4t)PFt0hGN%VOCvd)ZBSp1lr`W!$N>nvCwjsTkA>9~b(RW$!5|FXk(0XIY z-1~E8Q#K=((QN9YiKfWNARA!ZxkOv+~hd0H2uE zNJU%RgQucGb|*ok08vem|JZ8;M^V-QLD+EaeKEs}FXVV9=-+?;+J4rjzG1?W45lrZ z#ZR`KJ-x7f)V}*{O!e8rU};Q>wx^)RHmPPP#+t6T=M z%f}QN06dlV4jF-z53>KdA$hejEq^A9 zN{tB6CK_Icy=^?(+&{fz&W$ix6rXlDh6*#@8VKHfs@y}ku^5|Zcn4P1-(I`7tM|Ny z1bljh>l32S>CMUljLS?|YqFxQYj19OP8EE2sOm)^-Zv?}XK~8iF45a7(|55HaUX#7 zZij^la(#|saUXIXVt(K^GA_p<}{T#9nf1w?2+a8iqg%{tt-lmL&9fvX!Y)k!6k`y_U#RL?37qBM71NZx@lXf%?!Y0Ld6cCC0 z9P^S09*DOW$p#~vGy z_JhdR8Vr|S3lapz(!apzqR{8xd| z{IM#^mI&f+m8%xIG{k@SLnomx{4T~3?@EUm(d!-88W@Ty^e={Hg8yIlROd?c-GhsN~QLQl#_O7_3?}4_MLmxRaU#NT_Cf$C}lXQXKEU%LFbu|B^u8bmX2DoF$X?h>FBs~B5H1`Nf6Z=R4YfUC;2C1lv(W~sm z*7Hxb2hJo52%TX4|#mHR7W~coAY<@xneVl5Wf0s{V$b*fbUpiQo z%g_3V{SOB_k|pWgz;YnnY<+8tiWt{*0vf1Nd%k2S6HK1r{zlh~gxd1caa_=90hS2O zPL;AP42Un&Wtk=23}k9oorG45BH*s1>H29iJzm{oSMo}sx3A+qWp5g7ofHgdb$I zGtp80m9{=J(e`yA&t8m?)MCdu)9%a%!@4X3#O;~gU3Z!XXR*f`}#L@XxNtNdV$f zu1CNul{FxT=)M>N{mSpJ<|batUSC<1Zjleim27Br+`Iro#bIX0O)GB9Qhj4nGYhG0 zojG}A@V9&Qr?Z}xS_5qGiS(Qtg|3}|idG8SDMU$?;7V5GXiD$oqD?JWHy^OmSevEo z?PJWN?aDOf3+ajl30MSFU2}DOGC4U(PNJ3#cR{|B3*o1WpG9)9WnHof=ID#FCOq{$ zZ=!Znh@iUSfhDqb=N^4r8CfD?qn^fUy-~Z8?cq(oW2L%h+iLoS&dln#6W(LT5b?Xu zFE_iajpZ{z0!jn`=?xJ!ehOwN(`A&DlEspZ;VKJdT5LJ5INe#B;xzT8A8WjcG`47R z&9mMo$Rw9a%&xX|oiL%i-FVj{_>W6>AiK-E==JmlWUk7k+&E@A&;g6)Lpe%@`a~BYB)r}JG;gU6% zb@1GTLq6|w`y2Yb|Mim9TIqf9Ri!|ZgTPSZ?^r|#KxzgT#E_;`sEMwAzrlqcfmej; zS?$dEM6nX`=&CnU6aI!FBoP*Vg(ausp$?<8YtB>_RB^CPzUX1-Zk*!a!`3o<%Y?tQ z&Fpw0NNsz_f0@+5RakL3F^f^{gOAQ#+-kE7fH54&9|K|3A@6&sO0e$G(dOOsIt(_` zMnfk1KS-4~DT!r^GS+jwPk$pvlm89EMoM&^V9>i#@BUfJmtdJmv;Zy(;>x9)v51N$ zA?Ocbp>@XBv2#fSio;)zOpH_ z+soz6&)NDL(nD&mMSNJwSyp8|l!#_?W}1>kQE*W^QsJWRDlh8Qk4C*#2YoZI7~p3S zsLP-j$8ldx^uBJ^I1vI6I*}Nfv@yXASiu4TEW+s3`bX8WqO~o|l6IIt>a&0XvNQwi zAW&oYo`dJ4h*@{SymWP&&*d%c9kl3H!eaE&5VMSlAuj&;eV6kLy2@-Jn$CM76OeJq zkeJNSm0`Wr@X#B+RglNDN~4)u)v&0?GUt0O#oK2z=Di!qo?s!mjT5w_u@~!BP7dn@ zUnILJ-t|(&n2JT?W}WY<92$Fa(d?-@k4YU(oXyTe^~%+ItEvh82#CN`sF(S%pTi~W zJEIx5t^uKgd7Nlq(XTsH(vzjfZ+iTZ?h#ZqaibG(MES(?(VgJ@ISY4^qdjpi)zzSZ2&<^G`ipdz0l#8f|WL-^psI~@QDE0y-1i$>$9 zd?YHk_I$D%j+IpJT6_E*!Nv|B?owLkd=PN@bo*P$3iXM$J^lmCt`Q_@=p=Ss(5v;> zR>M(Qrwt7|b88_b8PA@FH<53j%)U0bM4rS1O*0?3k5@3*l?9S%Y`m+e_$*ctS)HM0 zv4Dt*REUtMK3X7h;jXx`$L(Gq*R?y@`CO7p1r9);-`0)X4e8W#8RZl|{k|C!5fBys z6GCUG^A<0C)P)0GeUXa7XG!Xn6k>0b3z4fzq)IO^$nr?=4n{(Kj zFO!WS1SqlV)0H>&)`GX;JMYjKh^}45L}XMeir0JRO@UJTU7^?5?AEq4FhOG2DRch~ z4wHb-)dZArc`O|rwv2ZbB68K%P@F`LCh2aYxGUhcNcXNj z+TRP(B{HM7pnV+`ZJLgF)a=o?negN_m#T49pu-TBy5P`jHA_si)`^2NZ5p@i*575RV(eaMiX1FP71LfnKHtea zunvIZLV|wKa3EiN5=W|&XF(8I14ZQ~VI+tujnn_6^C3%;c}Eci_oa)nfApEfHdrwL z;8x)g;Mn&-@4&a`B#&D5uER7ljQHtpGjePt&-iyA;QW(MdtIjfC2c`^?<`&K1%&Yi57c13kT9P)%o#@yTv z?Jdu;1^Of3F>iNWClL@Sej>3n(|4K}bm7{X3>~c5VFE`grq@Z*Gg4J^LZeYh{qi*% zG2q1GefvXjJp>VxY9NE~FZ|Ej0q)&Bm3ve*?2z6ay%dYJ43XY4y>}E2>C+X)^O)JZ z=G|M}-hjK&OH@%GD%HW#c(E?j0^5PKx^LBbZz2{#+rDl5kv@pTAsS_)hphGP^9;>} zf`M${fIFyTxOg*DU(!&q4m1qn_b(*$(U1iml&0f5V0L55#CvW(ut`7TP+`tUc^Nw+ zZ=nx(ly27IjpN1Pkb<6&iiw(^xU2stV`OJFQV+bJ`IzsT5$>RCjwENd_ z-;y*6$oLEQn^Z((^fWiF&V47r4+2POcxDv5UIIsx^y?kF`3Uz1mp=_C5^x^|I&6E* zGM|TMB=vI8rj{F@^~$vU;gVu^I{{xV6|5F-ILt8}tSYW7#c-iegl7jWXAIbWc}P=E0ze$4cFfR!kP=7hn{H&ng+2GlWF)#$ z)}@8&FwR?T+1=!}2Xkjsz^O$|2_5zeEbN!WXtgA6#G{jDHsg^GmKRh~X)<)lxUm%o ze)t~4IL1;-x>r)Qjgl&saB5MsnPrA1`3R!|yk}@K&VtH!Y>0Ot`EkmVd zL%Xy+G>E|XfOA3I)Lfr_kHen_A2c*>4p&OSIFPDgiv!&K6eFHgbn9%LrmwdDN$+|= zSa>1Z9+2H#ITyHX*cbIc^zI}5IeLb(qUzg=i_(-DQAXf%{39iY2y@HOvN5kU8$)ED zHFoE$bAd-le#X1Q>+xCc4ryC$3+>}#tdJQiE!C79RV%mrQc7*Ik`ax$J>8a`Ibq)4 zR@e%f$q+3n3&0QM@MDw%*dyvrC6sD>~tuEZT=~qA}t6$YL07TWT zNtX}5(Gg%RXb2Ji(Dy>>fbfkOlm3`?|C8nBeLun}!ydgq(OrKlX8-ilIicz8@IDv+uZIfpx8Iy;Lao(w&4Y$?%_54oLZ{!W^t*oT z<&hC^r(e>TkdRlRrwBz(C%=OVr@Zu} z0CNCQMOTwK?pGPk?WR@NDb*N+C0DsYfx5XhNnbxak)oW>Fm@!pz!6J95#i!a8%?)yR4CeUfA#xJLj=c2CJl-!yj;I^ep^)9*$ z7TbP1OWY#aL0*&iQW_Fj0%u(?hTUZQC8;>_*HK<)QpCQlM3bH8TdcAGKR`k}>7<4z zgWR0h&4sJTMm}MM(u5nu(DV+xjGx83APVf#i%4hME$lE&uV7Pko#f8&f%_cs9_MgU zc!)G3F_4%qR30Eo5g?QI+OOz`U~}R4Vk!U54B+U2-ap0)nMiSDcM({8>uzsxWdeWP zMYyBqQ)@NX@|!WQrvSy^1FLgSJcL!Hb~46iG+e37j`xt|`_ zSYf%W|L)g^jQRpDhggRNAy<{rppIEO<22K%>XuBxi31C*h$fE;zXV5|k-9`okLk-L zg^e9jv}Au3n<4>G0I!76?=G;1IG6+a5@sR2osQ(?!wB}yXpZn}!r#^y7^sn_WDxwt zwFan~(=d*ki1CdvOzgmEOhyG1t}5f6{>J;FT}7Mj!9NZ*PA1M7K^zq$SBy4P$QCsT z_*q_rAr{>_B&`1F`*C#JM!I~FeKw16H+h};sLt-731o;t5{gisx?Gf?Um2=1#7CC5 z@^hn-^Q-=VYcy}Zgz9m%EizN2JJk8dN3X0}8YlvkbC!AZa%q>aFZ4$l4(P#JPAK@4Z#TYGj*P zoE7@(1-HZboA|gQ4{9+kFC~_DwvY)a?BnJ3o-|1cqMA;!8*g%p;30UN6F+(t?5giz zA<&CQ2@OI3fd3R{W5bci9t>`U>O1djLtUM;Of0D94x+Lqk?6{jq-$MKCM02<2D0L| z;q|%wR=>Vcxr?L5f2G1%nR+Fdsxd^H>%b6-M-BV&<3o!(a3aB{eQ+-i*E~Lr5;k~dhhDy(ZMe4uS*$dPQHkO@2SJA3KVEaajl5giTp-1V!K-o#xb~qg3~Dw>6UQfpo(QT{`I2z8`?2|BJL+JIhtmX zTG)`-G|B~-$O0d1H^UF`m!61QK@^H4W3|+C6#hy!twpH{DmxmOeaHT7wDNJktyQOg z@H)a}$>TJ?yU1Od2**XK3}uv8B5U#D+U#+4e8ApaDU^ z{r!CrHa0en(bgk{RIGOov1IJK=jk@D%SNg{Xl4!A2-7KdKR{ysU8r?XFM&?qA~>n0 zcgj@oea9irzoc2Cb|bF0Av3|2bukO2nb`U57WW@1-A@VAWz_pF|IhnIkjXS{YEGGl z+Q(?8Z0b$=p4+QJJW8+#rsZ7a3fl1SaAz>Qlu9YpPma_Uv|U9e8A5?CA}k0D&+bjA z>oLC|10w1!(d!?^!$u2b>cAq|{_iH;AHfu|g>;nidcEs*sm4?Gm)(@Ioc15vx@i>IKwg~?O_WjRZ^8QjXQbk$4tG4_Euijx!dud*D=mV-<=8Udr zsM$_2!f8Rxt4z4GJYqi`s_C%wl;<@)B7ih8k)8UNGcjsUrdu~*Xb$8g64j97_ek~z#B(hZDKWL0k10TWt5khLAsx3=y6$kwo4sY+MJkT z@}SH0YHyEGKfwwXSqQgW1RC%e1P}4K-)r;h8bX)lfnyHeh6z)sd=C@*rUJc~$|7L9 z8(Lpm?-X_Dm+80+-{4;9PUsO8!!OLRl>EKvE;5?=O2699>b$_ruqlLXFiP3IADk6~ zM(-ln*AS~4$t9C0(7H z0JJJWB}<wC>FFtA6!;j^M6?zR=x-9f2!7UsH(c$U|=K z<=JP)J-EGi*-UHdjTh3PsrU1fl_poMy7>p1kE5%NBZeVXkh^<8?8?87rMd2CqBy}U zf@Gkj{)~Pe#Ci}o#&TDAU^GsXbU%@MqdRXWn08|n`$X*hg)SMhp`7)%E8v!rtV;}1 z5r^{U^j=Dbh1Wayi|*O@>F!)ThK>m^s%d;nFOT)Z?BHZ(&nSx}6M2Ik= zvv)ALTJZ}Pzqo`ZOOeiM&$;-3Y*=|%{I&5rOATP)XN>_*?N05zKXpq(MPj+PtF zCTFf1bNSU6r{1Hx^8V{`>GHh2O>du{($@=lyqA7*wI-jblTS;N)RIKpnTmDl*Y%P5 zyJu&uq&r&N5*q{t)(US*<&As&;zV3P0hlNLqTgK6Iy5;i0g##!yw>Z`y{Jv-!5$|f zhp`tN@&4e=2Jz`!T<5f=Blj ze0$!Qfa$ltej++7oEFl0@?GqRy&yGXP9lNAWbSRyozSV@>gCJdp=?1Rh{s?+qT8tBr0F7q!~ICB>gNez zLHGI>ksFM81FI%>Ne}4~%WrlZ`2nmfeA2k>q}I#a8V+lBK4Y*Mt3)hS8<*|u(rH%* z@uei0hOX+y(pflF?rhoi@^ErDwkC~yeJu+8D^lJ}s@VHOtg(=Wl@svd^3n`1eni~{ z@Wk}ub+I+5j@oL&22lnTW6sY8$S3fzN^p?UG0p|@oC> zRER~!y(8JaDd`o$K!uJJT>1I_f!1?#UNHn*5L|z><;>#YJh*|DtUpEz(4R?WM4WM> zM|NGD+bH2fK?^)>MU`UkEuZ`(vWBXkQvR;OAc|6q8P#*|5Zj8$GNHW9evfEXbyFtq zgPloPt)kc|%hKArQpcQ=`!lN7r7f;~egaA`33@88%Z}uoRvX-9AiTilKBNYw(jaBS zF&{tyUY<15cbiueDtI+B>ju0oYxv zPSR;ozmZ+{*U&rJ2A4i)2hyYNN)Y(L^pweq)lzMda$#<$ThC;c$LNI>XmQ*p$-FpE z_9?`{zF^A3XLM!qx(M}BoTqfCoV?DP#16(%{eA5MJz!z~t);#}G{;9O@+#*BE_@B) zm5rE%F^$=Ud*kOwWga1m>-qe(ig)U+7cJo{TfO2LA$Ch@4EP?zvOMM)Jg8NiT{0fh zhl=4oha)<}XF!PDGy%bAuF31}rG2X%*Cn*fc!ELW3b|SIw-xdSKg^r2Z^MP(d}H=O zgIyHA`Rp2e1rB2yBH$unB_!j%cO|ns*3~yhj`9D@p_UeuZ_QdrPO;z-E(~KOk8U>c zKF&T4hGZ>d;7cP$91N*y3E?++K#(3vbZW>p9Ypw#hwub+bF~%`)Xa}sF2n~_Rusk6 zyWmJ*DGgSm#v6+M0!tHIaCH%E6?>L*2cG}Nq@ks4#w@`$KWy!PnNB4*) zXS7AO-LujY^|frMq9u{310zgTrrdA2v_Kc#O`f@;g1kF99SgOLze#H4QKwF=W9M_c*fgqgOamQrPygHpF8C*c{3SPau9-f7^L zPL&O19DZ+HD~!Q~pSTuVRo4?p9TR-?=H4z{Y?qYz3)SoH0R6>QB-_6707_gxsfTD! z%f3Il!}s0{BY``hDK(Bf7IUw3!k8J`-pz!i8kWLe=oZQKd7kyB0`1t-1R8lw-|h0J zk&0W99~xAAw`u9NwyHKebaddl>4<9F47aJMrZqzDA(=se9Ey5**wH~IEmSPcl_S&LaIzkM>4t9ahXkuC zX6{=8J&NmbMfz|R4P-9B_Aa8lROhy|;@_0^iuCkiZJxR<5AB{xh? zG5grHtVIML->{dzg%Lb)9p`Gy$M$~x%0G@1-6`r0)I(rPadU1?vIm#V5yPt_B(tAW)Wo61JH`AZK z+T6G$3JHu&VqVzYqL%568VI)RMNs0n6Fp4c;Tz&BSeNDDgu`lheY{-qY=3erOg5J3 zr0mg4$|0_m^l=o$Vd`A~2fcm~slyl;9&Uwo365>Y{#bdYtHt4QyJIGx^ImAsRcTZnkaPc5U8l+jd)< zZP#YouE|?%=4L+A?|(h>VqVO5Ud;8ujdP!KW{byuEH`8B6jUzksAaN=_&7072>!?c zPgRv)G*tmdC;+?ZJR$$P%5YYwO38?miM0(UmT*((m1BxjQnT(r2sWo8n}KO9?suIB zTtc$b4kVMaOTi63;_ZV8d@qw;2^)O+atq%0=ZQ)&9m+|E<`QoSGH+Fy&v1O1pTjcw z_Fzqxv$dfTO#$|Rmd3{0hmN|buBN9Txf~^Yx4uXre@BtO_o)J8LB^;!4cARzs~j{V z7@0Nvd_JN(6F-P)sAkqbU@rlmt@>r1whrzGk)kJ6>{PwdDQ$3G;~BNvi>4I^$QORN zCsXdd39JD!X#y_D>G{!Hs3gRCzg(2r=D{#B+A68XkNH3nDt)htP)3OoImZnJ0 zZI5RY!g3z^qj_J|>Ym~$2U$M6sn`&AH5#XjQY#^q{?_kGkd|S&)NVHF&R1UYGUyFN zxnYe@He8g|XaB_oAh-wf^=D+vkBh7#JY(No5PS1g#C2E>A7RVxOC%kJ#*-v5&yU=pcp=SI9Y@~ z!f6Z>?uFdwZ4|}NQ-wW}qAXxk{C%nKo3Q$K{b`p}0oIv;K5at(a2s9W8>5q{jp2B( z9X>Q7o~x$FS`SqL8coVzX|u`>hZ=UpZGHSqc-lglJK?^ghc?Xxq+E@h5pU0_LuXH= zS1vG4p+dmH!v_Sm4_NF_EbgYTq`YU8y0)p?V#5{Zd9yj?otNuBrX{rnI8I}FFh1(N z_{6XRML=`XU_0G@C!{a`kc~$8)i&FgAU5R;y^c@&l~%rgcM_*uL|B&)#*kENs+z1> z(ObuI@fbMkbYZER_6Ad94t^vVA6KVNk{(lJl-e)NV+R|15rJ|7fhP8KG?7?OzoL~= z3IrI#u!9)nGYtH_R>mBs(wZ&P(kO+Ch1~}TnA9v`>IL32P>-GIEYxhonxCyYz>~F+ z#8+JXSWovlyVa5^pMA}m`HYIl@dAV@y2X7^R;S5|6N3kcEeGJVpeF(aA}Qn4M?&#t zN-_=UKP36FZ7Zca)m$LJMv+IJy!CyY-$fz0t0AFWJvc z8MU&vsG`ofk_tyd>3}AWt4pbVgFQOyXx13uibg?LOl96+`qFxbVp1CiOE61#H+Vvn1xc9R;Ua(T!`Z12Vj zrZHp{f)uKDMQa6ttX|i}OFNu@bv4vexzx{k>2=y_t6Oa|^SmJ9b&G-0gRaxKkmalg zaJZeOmiN_2Oy2p>yEgg~q@Y6;YPeyvp8cK~+q8t?cx}$M`QI-e(Qq7XVte5Uidl7k zY>YpXB->LM>@E_78fu^LH_%$lPSWjPPdDEW`jyI~53zsbS)L$~i?tFZmIJaFTegc; z$HOHI`S%tod{T3n*{?Ps53 zW3v39l;&*(*Pz?_7Q1o=1scc~EIFQ~mh^DH8hC4%V>?eXw{0;kB<-ZX-!&~v42Olm zY&sJ}3_3N(igZiEl@?sj%@4M;5~c>8T@^bP)kmvI(5Y!@Uj#-TlN@5X21PuOf$nO! z!n=L_AwC(yMO=VZ4$u@+okk0-3g|c9;LfzWOJ3^K~xkNnWS`T!TX;1WLy<&FZH@jW?`65v$do?5z_lmwxx_asvtK zFds#Vv3BRqrW*^EXWvlx!L+yk0$TlOBsIbz*8Gpn4(mN*1dj~jm@huLMg*~V0?UTUoQSHI(ftWD9+ zUE$TKsgYClu#)~EG8!L7g^7(#l_AFIAXZ&73>ztGvBR^$dH1XBq`Mrz-n$U@2@)w+D0ZX-HePcZ=K`WP-dEE0 zmrt$Ck3PSVS@`ERB#g}V;iqZ7?F_q2_7DO6^a><1l!BN_QetDU=>ggr(Z#1Xn(rt! zO?WcT=+^4oPrKdlTn>!Xx(%jb^o!}0c?PKH@>Nthn$ z<;>sX$2W?O*-JS27s`1IDKdCM%tYu!=b zb=lbI)p6vvP=Gc@oLR1426-l8Z0Nokq zKz(dHQAn&dh&eGZnCCUYxW^xhGQuF48zs+u7*p$hHEpJ*ez@6>92EfJHKc%qn* zp+a6Z9!Zl102hJOkm~X*_r;*K4SoY|U-x z(niG1uM2#!0&|rHufJ^Qj-9KLuf%PrP^Dh}hL6)g+lu)?ZHn9QqXW@D{#F#fQ`D(s zWhm}l^Yf2e-su8>a4+|{Iogk(^hKLYlbMpK7%8Si#V$Hprdd5+GiSfH-F5hYk?F42 zGUZM1OJL8A9#o-LmfkP&m{xwNqLk({wD06aO4ozk#5nc*Vf<{Do#94k8380r0a8IS zx>{&HVy)j;?!HT8ZKskbZTEPl$=tdNrVQrIwIB8%xr^yyNPdR#WgV&NFb!^eH_7cHWMJs}c-ml4 zXUNmK;glzTO3Ebu*8ju5&6EK{(nD+>ca*s8(^moDl$wKFlCo_&Q#=0st1VlHq1Ue8 zWqIovBj}(su62ZcVHp6+0OD65zcgDS$U{d30MU*#9@1?WrDV#8dGK1o5 z$W&+~q^j5;AROy^`zA@C!H$LL&kb>Tqy+hR?NoUn9P48YWa0XcZjF=ix6*@daKDfh zRR>Ap|FF>TF)M^{jywBvU249WVdB&XCbvpUXx9BX{e;Lx zB2vm+TXMs1>uzw`m%M4Bd_)WnM1v6l5>z;fbCqzT%VewL-n$BL!qe}BwygZXD=#1? z$&RTt>OS6IJ%Wf!>I}@&`+|`I93=T_ZH{arRea1=+eMU_38SfArZHh{&HSMsQ7Bp-&C3HJz+_0C6EVj6A&r$|SfErkU2BLYB?<6f zTgBeMqEsc`prS-A(2HA1kY~V2eT0!hr8nw?S45Ld2+1Ut$>fBJl|JS&IS(0XVwJXu z?1ECo7Q#Ns)JEo%u^1<|)foPOi=}=iW2H19nJ9lGaUDX0K{$c!gMIYLq~4Bi+CFrk z8nAxU)@Gho2*7mFiw-oKqDgOk3<4Xgl&cAWzQbAh{Vo$W@W`Okp`Noh!04jD`1=U+!9%TRZM#G$thX zcOPXZuS~j4=JG$Ja92AZR}oRnlOf(9Jz#kn3?uz7bfb{XE|GF9mUZSCWN^=^Bg}1c z3XD$?Fx81SR>X|EU4QuO+tKo7P%eZmbNI>|8uNIqLs}`qhqP2BW<)DJmzU3OcdS)2b<3-Q}%YT&r(zoz}7cN0%1Jk4_=)nPP2#D zCQ`99oZaskgARj#K@_5yjFoMTU+ulNdWpuDu3gXG&02BVF)Q@h^`|I7v*%dag8+6{91 zCE>}&_;98MHmCDQXsy&rfxfJXi8hStoL?@sO#hj3DNvBeZCo34VXXS}#$FW7Q`hWd z!=*|-&@2qU&0$Sl5Ab{q)v;i9#nw_{`dvmeiVIh|np2n{|ZL-Ba z!v06oW?km62!y=NV>kjZD^l8^>OSSSD(9yZBTnKc7j5#lM#pP(nq7N1XR{LvdNe?J<&UaVHh zIXR^bTZEihR|WgM+>P%z^uBjU_m+IJA|1F%2Asad0xEkEt+X%%1=NUO7Y)=)uvl#! zWqlu(@xn9iF5%5OHd4(Ap)^#ZHFDl^0274t*bZ)7k;`*h%lWu7B?oE9wEA4r4<$9!@MeLtIwuluys5X0Ve1?*TPu(E@9zmBc@5XfUU2^19Qe6)Jk)`>fz3sV!1}ccAi)3TDdl{!eXbm#DAQ1%hO6 zCtHIp!el!XsmF1xWg`Y)v}41Y4k^P-9aJDMcAIUFS=VJD;RH8lIaj;F7GhHoufl8qy-duwwHCMm5HLiWm026NAuX5_33$rAO zs!En<5o4W#ZqLg1yTzF1vLA8DieHx;s70c%07O3hvzv*D3CWvdzq2&`YPP_^;o&<* zug1~d$HJi6a}Llz!pn>2@c)Ch7|De=-G-zk9DJ_k;h`sJ+n^Jp{Q+TXT7Vnkh?7fe zT-ZBBP$?SoVVY2Xm2HA~Ami0q?Q6{al84~Tj<(XzYVNozznBN~l#{BtkqRquUuT)Q z+B3z7idPZS%O3KW&sGf34bB|n)S%ugZ$Zmn;TC^MI+-`o{nimG9&!Cn7C7*caM+Ja zpIU3ouL5qu==<#ytBSAdO(}V4G zFAsMCR<9Vt!l(cH%Y88TDLLW!p9m~(T(0eP#0dV!p&aFFzm@q5JqL++`m0*F!$bUz zgOh>Fk=kCL+WD$Vrj5fue^=+h6@~PYnLYuYWcOw(u$3k7p@}Nj<;sZ|dLJ74)cBo} zMvKWrA%?9UKf{tI>0;4}_1}LLC7vdJWV4usYx}b#Bp4f~!{iTZ8&0UA@ z?0LsKA^su}ebI$UP+^9W+hnAsHzT7icDjiX?00)R9@|+)zq~ku^3gJH2Er`&`VXOL z_7_j#6PiNHXJv2-rwvwIKLo?)7RI>a zGAq!zqOszGWWin~`0^*_e$1B9T-9nV|Ia=&BEdB@I@_agyJL&_Ia{20_#mi~!?AkD zfO82U2Ac_WLVn_WI{&R49y#^UGvkxuWB5+_9$?urEcKO*4=tC&bGL6Y=5e#Bv+arj zHKIkab5BgJ$DpTWOZ}Yx0h?PaFW6vA5$wa{c%!EeL2ABI`Gw98JIRS8hiYH+CmBq- zZ$(JrJ0Z1~eQfA`EGflsQFKfU`SiNwdHRZr)p#KBQAG9nNTWmetqmHlOMJ~m%6!KD zBJ7CMFGQx*ZT~gm6J1b%B9wdo* zWd&+k5PovPO;$2IeQc;Z|aDq~W zNhn}Su5(}@>_wF5Sy6-&Xb_1@jk4cIe$%Bh<>jK~YSY|ckH$&Ug2`#hD-G^E#o;t} ziHi}GIU>lMf=L1h6@Mj+i|n2^&H57%4$m$v2-9xG$D0aq#0`#1f#{W7HL&B`5>P0& zA|{I+_a3h_>`O>&b*50wiKOiTBO0!%vLi8nrGJ7!J7ushH{3T>heQSw2TU^D0deq_ z;jP?ob`E69LGyvLBgaI234gt;pXee!7e<+ z1kg#+TPA9A4Rj{26;TJ%#pC?dRCi+XQeF5%#jY6 zFL9y(Ff+~b4f~LL36O_Vr6&>A_0;k20>>axG1_`T92n%N+pH2DQ@vaELUJjZKZMuC2h)~>T2weM>^8#A3p0^%X!FD2 z+w|S5n9XQkM31~0Bq`+&^0gz4Sq-eyaZkWy8yV1*%>XG1C^EyLymp+Dl)90@@YQUB zJBsl`mFAkRt6>kyp#}G2=8sP-GsH%vBLCs}bbJfft#@SSx3GZau--_q&6sQ`!9g3x z?+`5VPEb|*V7fIIADnY78oZl#-JnuJ^E2lxie$c7|B3Z|s`=hmp2EjbVjO^-lv0|> zp=T9WrmPFV@FY_J$xccD;|mq&$uj4cU3UoJJtxjTGX+~q9q12%RvK0GC1#&@G?VuK zU^j@ib$ka&WOM8mIe}e+tCVaCYiV0)wT51PC%gX`+IvD3;Z|r-Uw`u13^3vT7LJ9X z4`tLesUoWN$tql@bBml_?pKbW27Y3*2)MVVUb0q|Mju=V^GD#Xjt?Uh3(35at~MbA zglz#xVA&HJE5Jh7+^jXTULyEvuEBOsb&%M=Vuh1YwVSQx@v2q4h@-m~Y4H5Q&{+sC z(T4Ke7oNson1SD*B}rrrMcay{DP!s~duVi$fQeUeKn~4EF>MbI5BVU{o;c|^(W(^C` zquc90nmF*5rk&i0a_CbhSEMdKXkU%Kg*~-6ox@jrr7$_^Q-WZ-Os$iQ$4@KND^*i# z1X^j#e#}T_Hn7bvQv(qPw+Jgb84M1ORj_)6Yp9-juiPdepUgQ3u8bgwu%Xm-bGDY{ z`eUu0R?qr4^N)&fiQvaKhe;g(EgPS$PL1?ce1b9fk7-`X=su5YP?K+zm=EK9_pU_> zWEpR?Hqpv}?@AG(x{9~ubk3YJ=LsR>@tAiNc zwKo*9qiFmG6*w_2sAM3jRS3+J|Dm1jg@}}bBB2I5mR}k4M%Kj-Ch|Fbh-)B(ZsWAE z#D=A0Yp>(mEd;hEaEf*CG?I4KFv-|SKbkf&e?V$%>eI%>!mYTrSnjXS1ZCPAzQGpq zax!k%Icg0UOOgP$0RwcCbTW>>$=|7vKYveF!zG+&ep}o7RfGt}<8(GxAaO*}GEN*B zj!4IkghZbZ?1h7f7qucKEBLc`O^F|-vtI`@yAhU-ysPK}@WcRa zxQJ@|kp=(IDica)PO`eESBBnsd>*>4(h?Sdv5MpwhX3Pfza>aGu(U@4B2?9qeocTp zx$kR*jUiM)f>Oy+kNsg-wsLuGZsNl2eX&E%9#waVoX~NCo?!Z#U}F9f@!iKtJPyrH z@npsHRqHf87zf#B*`TAkVz_u*0Ngyj1%b;*^`Qi<+D+0H5uTzm5U5qvN5*qP44!rd zcYM5??A_m^b@TcQR%ft}RdQYp1B81~R84cf#qON_tbW4gqLKnnDzr5PDOK9ib67b>{100CeF8)HDA>wTMP9RC(5@X2<}aGz-Nb|{ zS#Z2CAy=4Af3DXV^&(kmF=b+C8N1+`=*^1C^V>ti$?IIwv?L=5CV7lyyHLi1_eY8n>^S>VkS1=A0y4CNU z6yKwme(3pkF*Tc+y{Gaf1oK1z54tV}==p&sVOR2uxKTlj8tQLfJoRUWz;YYtWk25&d>Nd4q^=Qb}rdx%%ZpS&ep(#mM&P#Ap z(4ZRF9n=jDiMi|alS$dsR8s>rvL9lCTsk4gci~T6y-w(W`cVp5%4Kx&q3ksh4sl$B zO$cXmt{-PiWLDy?15nZqw3F)Q#Su`QgN1tx@rwak29ngcfx87!)+uxpE zzOk3g^=}vuwG1|L+$u*)lNjHF{M|w$twfNkWu!e7>P!EF>KREG);pM~264vHw9`y$ zYfn$Z#1yZ-))iPy63kiRf3OMZZu>K}L|%k9)4|U?5s?egBpy zld0ow%qdIC3VYbCWof;dmYxw1R*b{bU8ltE)~9!etu6vTBgKG`dwl;Z;p?7YeMuRn zpekHRbD^ntIo4bK_J^X-v1hWeKdvvn{Y(0<9pQ`sL6^0=OmFB1?$%ru3UIjXU0z)L z9gH_Rgs^i?rON}zz-1EreEWA9)|o@^`-2zEd8cQtldy3JT@-mU+_Zz;CX;b*`P&`3 zQ^IAx=^R%LEyFvQIeWA^9&huty2*Hq!WJv~iq znDgs(r~l#?{&T!@BXGv=jM}-YHe&A2FZXBG01d(=xjP`&&EH=8FTMyazZTxDjqnnA?z>5? zM1~M<)=t2|!~$ZTHzop!(qOPQl;b16Leza5xGll@Uk9;{hul<9B|cl*3)U7C`b~ds z&HTdbLptqFn6{x@N(4U~Kxo)-cN4n(+RqDe>zFfHIXNMz+OPFLQU_*WX+nr={zQ^g z#{Q-lK~rwCI#*Bb#rTG@w(#_LLo&n$^JR~l;ul;Z%|XZWU@9^%gQl+}f{qfH?T*PC zgdbMhYX`r-T%v{f!&a2V?ho2g!3t%pIr;zUCp#Ep8@(+cpv*Xo2Rn(m!MnV?ED-%v zm&%Sr2^(~KdwV)CF!0=Ajm!=<61LVr$2Y&n8+Uq>th5ULf?S_@zeNJ!z{to*>42$J zhtoE`MpGMs)mSGAZ9>we=8L^sOJ&xK?5Z}irlBo@=Y+%B={9r_d8X7{@iN3FazqY~Bon-?-s*p0Y6{=`Insu_!`C$xz z*YA}W<$;&27souZ0nUPW-8ZZ&nDsxyZY3ssZ$ILG4R(lu57fjJ55e@2!36IG=J88AMQv{QSI^}Dq>aq3N-2>AZLvJunB*63Iw>Q?RXK-7!3p6u&pa^Ns~@g zPw6bHdxYtXyts%4BcyOqUy}&a;>dwlz>x%OFAtfQYoU?S`dB*dXM=O?6tlgNw{=n- z_Hz(L%Lj(%&ezU;g*J9*(BDnHVP{8`=DTtSOd{+#1*J~hUNC)hKRU6PiIWyy_-W|0)v3zM&`vATKYy8<|_5 zRFR~#lK2!wh9xI6Y_wgi|AQ;_+KMOa?x~^jG}Xd;*hIZ8-ZtnsAj7^Pw`UO!GO#%= zMDhFOc`;1DZ^G}9FtGMja@s%1qy6KBfL&$eF=u;lQNt&NszQTg{ zAU|2atfA#KaCn<{wb%TRW}YwIWgpBk>Lobb9^UEXUNSMb53>7c=RR$ ziU#-OYUi2-`glF1G&KV}eui7Sk9w;bym$SApsja9q|CT37@{EXu3DzbUL}BULq4L!%8N*no`2A*I>Kz| z8eoI@-;I|>-M)?-DG8sj5s`q|@`3t5NLg3O!q!16@bf-Wv9Fo)M!n>~JxNloTdXh@ z3$facy~rCUFbT^@2rsy{^oP7EaW&sXFy{-+%=!!&xPVurLbhe*nf-gI+k z%p48%SN;k+%!CeEf*B$s#*+?fVKr7@+Nuv)vAm^F`PKLyKfKYVb>K1cE2p;XlR#}W zJm0^ti67Fa*G=As4@~Xf2b`${1-q-sM3nI_tLr}=ex9V(-K7s7Zb0KN2?~~D^x0DZ z^oG9+E2Ls@M^+1hM5UHq7KP$Vq1Ha@B9jE+Yc?E4-dI@ESh`qA7XHES(<`qzkGvV# z+1Wv_?wPCYk|n4G=772^0HDp$AVuc4{LzmtFitv?A1yvisdAbdYv2N{1$ z73?@$2qdWo!1;00XOQIKvYFjyQLG~Q3ptD6BNg)(G>bE`JoG%@da%sz;<)1Y{7G@G zN8DlJ%t3`A4`jhKyfvkJgAvSf_MVRrm3LTD#|v*TN9kK?;~_~aHSYT`D1!=h5SDTx z<&9PW|9-?zoE;a~;(Z^U0DW|{V~X>K6os}q&PRS=)Byhm{aXDrk|T-IvLt3!gt1H9 zmc9L7Q2t5ed)EvU`r>@TqOeo}(%BJzyUpRtXJM>GicCCVNlEq!G7aq$KGyck=cVG~DnCWIlz89iv7kayzHI<|A1Rjf=+f=4MtD z$r~6J6{l9|x`^_@=2HW+oOAjn`9r;w z+E;f`c7?(b*3txcp4}rs1m*R^5|k|SPk0cgN2}!cLpX2~!YhfvbRh^6Wz=qd^J*{< zT(v566c{{>Rz>s_l)7XBtC6e-5HE1eh=ajnQ+Z8U*}-JQrUsmk>k>2Zg_mgAXK0Sh z#?XaxWX97<4UL_~$G8kC6XDf3Wke6G>{TQFa;^wEGl)0Ta^esESIL`|^}2+-oOux+ z#{pP)e7@zuUO2%GYKS?~)D%jE4hX(B-{_-wHpjAlp^F2wWiMOx{$D&qV5E0_pG7)< zYqX1B1?jVVAyz)EF)AP^^P!(ot61D_*wkC$@FYqh*#A+BDdF2}{H{Cp@_|KV-A&vc z-0bWnn>(PSpkNr_3DU}uWM-VI#;^GnXdP-LR{4xB^Vq_6v}Tt55Ez*l^Z2EjUJ933s&5T>1W z1VkRK!gy{P#yp8B>dFyN&FbW-v{#+RA2$v|{mk_mAZ8-&<6D#jA7qgP(qpDNO~;Th zocE5Uw0uM22NYz3LWcjQrJw{T%GjoeoC=oy;&1!$#`N8MnV0`p03u?F49%*vJqr0; zAjpXFn!tEZiFckppU-+IF&lFPR#3akQ7uTrUF2l6+MHK$29z8;Y2b-LOQ}Uy+#iSe zA7^s((Pl5oRM6TYcl4JVs3%#RltR5#|CYId^~<87Nczkkt62k{X!s8~`r3+$b+mTo zsG!8us^XdpC@0LxU`M;Pb-}VG^NK4Ea)nh%lMWquGFtiY^%X`G#eXbkbrg}-db$r( zBw>HhG#0Q9STGnw4^9Se&JUcFu(jB))V$~#E6;6f}aS{sT9w#WvqgP?{l+Ms9W1~*)3!fj3&M(!@)-2 z1{IUnADkBSib%0}>sBpMe(Nq2Q_ctM@k1E#+1*?Gu zrRg+tn9#Ch!iPtON*9X~c6Wc+D({tV0W~5c?l>m%RBUX<0TZVU((sS3kVDA zs2NF=&3s9>z`xZB532q3(mgi^oHWH&~^n`p*5P?ht2DFQ)%mVsGU}n1CRjS%1 zoSvRSqvll>J28*Lpj3pONIINY-(q>@ykXT{6_A9OqSMH}->ap^24!LLwrkZHo>@h3 z@aEWWbpJ3Qj@%AFxk6Ry_knVmK1Zf=%Yli@3{5y1hNZOMstoIJU$8M-M`47ba*N_W z{>?oMmt;V~4j;F9(=>eb_Q+;rrWopctjVr_15Ym#dr^bF={#NL2*GvpI{P zpM0iA7d_fbpbd9u&vqS&e!dXT+1f_zMt=_VRNm*w;EGz$+>UoERJpg}JGk zd*$KZpP^*_dl(_{e=uAzj`EA1dQwJ456FdWR|7*mo5V_g6NbD%Hx!JJxaOl@ixk{P zGzZq;Q+b@AX(ND9$y0}c1^J5@oCUv1RcZRbC&7rm7Phvw4!aNI@-%AI*8&B8ocV%j z<#?HmE?Ueu&!={4?n%tefq4Ol0E^(HXVc;6C$)vQ(?>o_98FBD)z-&X|KX`_0|9X7 zYIJ);Wd5V2gJWh?*{uAE@RoV1KckLnujr5mbm~1del;5aS7vuH6SbZ^0c{*Z?+xDfr-ggX-y#b^aXC&UK3O* z+hAqBJlr)~FJ|=`#;8(5BZ~)0mpFJ_`ViXUZ9zxh<7c6`88aqeOqjzx{m`Johu3 zl=WpL2ePUok=f2UBtG~)3g!G#vd9?}&dlL!Qg+fHU^4Gm?>is{p(gHg6r~Q0=OB~x zG}F`lO?3W-JZ0bZ&feEd_4#FLs;34}8ccuXtk^(&P*3(N5uu>SqsV;Zj85Qep_2?& z1|d*5!5QeGRy6<37j1g45LjbGfOMd%^zzViq1Eny)SyPN{)QPk# zi`Tu=R>zY9me&%yg^So*QAp!4D$^+db(*rWAqW0S13PL~hwZLu=10wp@xdF8+fRLK z0y|I$tBEX{L%9L8fWD`mfxPevu0T(a^QUF8$-bT_#53xX@SGT=z$(2It0CpD!#M~DXiX+=3XL2J^nf|>()?Ycj~1=wShQZ2A~%x zwv@9dYt3;E0AcYm<|M{Cx0NnRJJvS@O@TTyz}KI1_dI7#Aq60bVY965Tv31Rp)t!k zTumh(2s=+s(y|e8bTlXzANC)91C9!;E`|Ph-7?)HNS9OrTXv7)tPzv{wN&I_G%RbW znpPP4vrXh6@Ig!wI&I>A2#g^BpkSb=vA5C#AQhu7lhH~x&j znjcfFHN6l0M(tqM{-Mf|TT#HKwX^4L(4ro(n*mtT99MyTd0pquU5hk}cQ-Uf*N%Gx z`sKMvEt7e6y}*?+w`2=Ns*bW2gj14fZ^V^<(m)N8o6=an>_^pI4Qj{jMRRF?$6EyPXn5cUyvQb9N`|Up-NfL9^MAlOW zVzYchnvE6CrTXW8M**=%ODT*g0pXAxoVOcA%)At`JqOBg#BNI@+ERnL^HKqR%py-7$ z3qJVZTV8+ojJ7U%RJ82!+Q|Lr4-v)o?l8e0OdM~^X@HLkfEpu4THtGa?^9lTX+^~6 zJXe<+R3G@T*(9WFg#E7GVv$9Hgij@kd>10qU}r;()Yu%$w^X{qczpuLohPd9*3o`Mlv2s@1pM5{+bq}4 zDLnqU>Eu<=51-}LcAAA@fNzL< z{DPR_t7^$)O7P=+;~P^{ba`J$hI=$fY1OfRC$>m}{cUhw z9>EsgKvxP2;&AXNw!pnk92F-ZSay{p_vpK1WMc_xItqe`9AEnZLa79uvYz!Q8m!+E ziJtt`&SNxK_mpjaas|rc#nU*j?5y`eV~u@kT(12yO2ys7r9({@Y_Yi;-o(m+<7ZS1W`_pr_`J=)*I6<00ARbn_!*<5x zUw2i2Brr!my)dV)!cIc!#@Q7kL30t{#cemV`fnHLhLy%yx!@GdMNQoTuFW6@Q!6s&90dH`JKTc#K2~Wik`ZXkOK0@hX zaS3637py*NojAj952=ymj)J^|Uq5@=DOOM#4>ud_HjqdJz(i|7_Au?0s#znI1~Q!J z743}HTWnWDlN&;tD|Ga*vfELCw}@27b`A;OCrfcq^a&&>R>21 z4h-QJSpOk~^H0se}?mf{NSJfk+6{eYM=8Llq*#ZK6zF`CMn< z^(pM}yAC#XLGjO9<|lJ=bh860?MIvL9j`imfQ0K9^ zna*pVE5qOQuOW#nvcxyV#kqppqv*40^yYIFlE8ey)~p-Q`4`4)r>V3CX*Tleb-Ud! zj?yWKM=Jj$ZQYw<$}W;i2osx}0BSOcPOiMLs!$Xyf$N?s39@*Q2VQa41CZ{DvB%+a zKR4!JfE*AK#kUXp;17?Nd>NiQE0`7T4Ym)A4{H26$8^GhuqB#~=xd>rq{1P|3He5% zLZJ&F*NoKg2`UIE2G#jAB>AOKpSt))!>DREMzMbD5DG7R>ib?)Tr4@e=b3 zgi6y9m*n?r5ybfTn3$OUVt`z;icM!#cGg$^E72A!=PF0oaZ9{bW(g}{fUIkn1SOL5 zJR%b?5v7q=SZk!S=@7SCA3J)CUL8Knj7s8V)|bF@Pp6HOm9CkW&B;y;SOFggpP*< zTaD)rq&+G8y8lU*AKULGZ3398@e1X@lOV`I!k`@yW-cJV$Zf)RE&2Ydo}x=P#V?=6 znm~Z1f9%qHQ=iqU>Bm$gv(!P%R&!+&P4-(K1MO`^dUKULT#rMFZKFZ2a^|gZ?5!1l z+5N8trwH22JeE+SG4#3C|Coh8)v!XXk*IR-BMT&}K;7oBq9J_=`qnB$6N72}=~qzk zk6vb%GYqm~lLW%u)Vvqt@*;QW);yk17}IvnZoXB_hi9r0;rfHWan}oOC-S*b$$iN; zJa1ruNjuqxE!YR?mK`sf;pH0#4%EB51+!*for@E{qkZEiF6B%28?+K*u|2h`V?jej z1br0t@-&(Y@X;qhbkm#VTDPl`j<=Q*^3v+g;>u@U zWvwrnRaaje1CJcfKYlxZ9jc9n} zf$v#@KW-OWOU0n0)_lFo{l4H%zE!@^P&c{2vJrSH*NoN%}x;J?ydYfj*1Gcp!t`Jo^u8hHvu_JJ58 zgj=P+=gH}f+HL|Oi)p#eYf#W!9mA67jLyHKt@Bv+DvYWZMTa1;U`e9wLmS;DyJQzC zqyy}lNa4Aw1C}~!Aj`RGs=vO0!Bs2g?3i_V(QA$FURR;ka|>=;!W2e|ZvZ6EmwpR> zX1h!uM{h5)z(c3rQfe#=C$}y%EoT`AXqjnj|3^305b3xNqGqsql60b4!$a(mqgdHx zqv0|XSQ+)2qubTsVr9(Nc*|CPd_Cxkq?EpFH#wRsPh8>M;rIWxCb9Zh zpI4OiK~>3R;A7PpfV~rVslv95+^V*g7B}&!;VdBmgHGkD2zo?DHFTJis07<^Ydc6Vemz^CyI_Aa0OWH)-!vbjJMt$y zzS>{}_tmgv4`&yL!KL=~0P73@$;z^)?b)uQccM@1SgkPqY1uOicbIO%ihfQ(bE2Wv zov@Z->SP#OPW#dvD)7_wtps&g{dd{y<$axv5lF}wl^)cYZqINarBZm=WY~}Mq83BH zA1>2wDIwSop7tk)}DKPEq!sU+Fii3x2TAN15UuK7l=?6W?gyo7*e=JCI)( zuL-~*C59PF%W&6in%z5UjLh=@0kMrNqwRLlDtJ7ZrS~GvUNQeTb|9)EowvQ{Xz}j6 z>M+$#>|l`UVi`;4E71sYTg*42Q-K@QHoLahgP&cP6LUL$i}>yfLC6IAJoq@Rbbl4_ za}Z_26au(Y+M{CKdc_e3aI}r~c?HgX6L?%Va}K20wCR|q;V-lPjt^2X=0fL$FD*`wsVWwd(3!^#Fw#p)88(t|D0aF#$bis7OYOi;k3e^ym$u`Nrxkm5 zC1OhO@D`Smmae;KODv5sHpY`BuDNzfZUEUBV657@u*I0apyNfjanz99jeh23^{zcb z={e6l3L~B92Ani`V@W5egE7$={=h44k=> zQTk01r1=Ka#pYv@fFWxY|adiq;{NLmW`;acF-&gL?Ixr9U<Edi20(lnJ7}D= z$s1d|(n+1WfRY#C#q&STI-@ZZq2$*?eZ#zEn>v_zWv4WaAl3hjcx70l;f|08h=oOm z?Lsndhw2%KOV(`^;*b;(9DL$|iP48b^e=@1g$_ljm1JB2g9j;S8@r%)8*H6y++nm! z_-kCL0z+GmeWQJ0@!_)+YjXLvGv<|ZOod#RAwoxJtU`g|#{4cdkqe)Xlp}G3lLXJr zK(AB<>kcVyQ~3N%{xM{rS`)bsg4Hm1AaSGPk;$(N`rCg_04fiv-vaL@GWJVN`!&GL zG*fefRe@AueuU&3${NOk3EF3ceGs$beT?5-r%t^j5qn=-gwlHw^CSX6Jn9Wsd<}Yp z+v6*jya&k#yAy{ycb?_d`;)gTo%&Vywe)k~f~*A{cxL2+K(T%1Yr#0877U_kG!xvR z*JAso%6I+nEvKq9?VUwyl?a~e-l8)1NXGbYP5H%+t-}{#U&$_H_sz{A>o8(-x~43uNhwS{WU->eyyGUr=k9JkSxtB>vcrY%->itZVvuii*Ytgz2YUb zw?cYGJSPUpl%(t4GavSZ9Lcn)Y{J$n`^Q4}>@)TKDCXc`8>rcNvrmiG!@IU0gLwX%<`RkRyg3HRv$ z3=xG%th>d~3k&kvpZsg+lD9(!=7SnYEv_g((xk(U24RSZ|Ax7_IrFIqR`f3sHAMl* zehgMMG)#mHQEth-E2Tys^XDwI(X@vX59QD97`Dys7cLeE775h@eUBLEdsSu{6fi%G_+F(Ks!HMrc{RbYQYBl1tljGDA{2{G5Gy(p!AX(^!sD$j<+NfBdzbY9O&$L8^}(>#?2labvcg8@D%nxqnNbm(q9>CeIk~p3DaY^o8FTm-RC> zE>euYR@RdRA)5_!7NEl9Fh8J}0{PeV%`G}=l;+0Z~}`QxEK z(Y)*S4q%^YpD`3hX@%W%VRJyG`(|9UPHrvRNiEQ#0Ow#^7lJGcK{ZeqQA4+bpSwBqEteo+z zS>Y=N)eWWYZ7kp7do9^tVOj*$cSWJV9|bKh>phb8{Ay@vB9w$o`(sPXEiZ@Ux@$pkTfz)vqPzM< zni*ybw-iHnpvz^>NC#^6-~F4?W|LJP_R`IIZVmQLsJk7%RS;EdaxZ=Wm_v+AHJ~>k z>|$arL-RsB649_6yalxc!b-Ab>#tCIY_i#%nvoDy>dEa)`-@=f!0kRwm}XyhW!u^A zD`nY=$}9sud6wLD?tvCw0*$iit!j>oB$}@sT?G^D@_825{HMfXmp@ye9VS6hu)`YO z-4NOa1G_$ZQl`GdEIP!<2>FCRnQ<`A_U7m3yN{-(u9=Re&I{MhHUNs%lqldaq3?xK zbXG?H{IObttKcyBYqFE4_g8p4J{jr^qXkD^#$$@~3z;BwNJJNVEl~p%2^+`JBG|#P zw%p^;x2zf7$LCBW0 zZoF=^<@@ZtPh@3Uow*Ys__N{h0v;_Gt*d4Sp6iU2!(rbh-^HI`TVoNmU1hhsw9i5Zo+f#g8Q^f(LTVqiEU_S~Ea(5bgBg z^gW=$Lp>=UMFpjXCe!*)^r=MFD&zI%Eo)~` zKOUqI!0qPH5p}NV0h>1hI^Ba5Y{cib-!!x+r=(__jm}Tw1e@5jTpA0Ok}dPzkoXl~ znJt`Z(;?euP881ZT@4zyNi=`j(1>(@iRfXGlemh2&q;!WrD|Ycke*rDo?dXX28){`HxXmF zc~<`8hc&7xd=Ns@$%1yhBkF>c9JwY})e`aK6){Gi$|ClcNC|+N_<9x>gv0~qhfjk0 zm8PcKIRNM&GWRXsvPrD1UEUqE#g^rDaRPU}cDLSsF$CDr38a@sb07zGLBQJx%Z(~+YG0634%`v@hM#K7|UyeA}7+<-QIOxMfYHUPz;TM}e7$$aUj z9*T~DDI;=5EEN)yWe*t{o)J_LToM;K;GNZ*DE=67Jn#cJ!5x&bb|%^?M&yd=JDd)T z(oJB7Nz-jCG^iePF^N3l7kHBDB$dX<1)IX1!*&LQn&y z1ndwaYg=eTm(Zw=XPr-LLoiRJ#E8gP&nc+=On;mxhiw{tLOVZMX3rPyD3~P`*1v1b zG8JHGL9-}y%j4pbd8y|6T#U*@tsuY|B!3u)C{>J0e!TOT%)p0}*tArX5@G5P6c9N1x)-Uy)W3{;myj9Mbkx}AWC#+#4A-)S*$|$FN}FK? z(f0%d+MLyJJi(Ba58o_Kj9};Im#Snf9)!}p{l@$uVis!9#Es@C8X6P zc1O$VEAc&?gI#EW`LA|kdKJ=o%vIhQJTGKo)p_$-j`@?~3NIKl49A&}Rq522!l`!= zvgumGlkzq8y?ZWG=+i*Z?%!ykWpD?fJvC|0nglOAP!RS&v>(>~=7ewE+$XvA&*>D{ zvmwbbh6V$-?U?90rNC@!D5d{X7?2_C8iXEV{Ygv!z+E7SajTFRqpmH`-|(mN{0U{v zu#+>J6XG+L%%@d*H5HZ20*#@0@O>?8Np^lY%oN|)r(#)pK9V5_2F2Z2>hK}{bggR~ zNOXePyk>ckEW*`DlzxL&OC%IGAK3tUdR!i|sj=rx^GvlXtH-G{>SXZAgYhm|XJQAA z^$2c|3DQ*REg(_xXu83w@=P7lo&{wl8)Kp=L;IZ*KkgTW(o1F#!3-dvGz+z5N>RMc zKW>hYFTsTSuNuI_DIraV)NuX~O$#AhFo5{12FY;R^fX|7DAjIeIDFGX&y%aJ_{g&; z@eKoAOhGtVnxwI=j!4n79euzKG-O|nu}k`1RUzUqxFEN7Fm}VlR?6)&EB4Ja?p3?o z@xkNH+_LFd&3)7I6gS>D6Ls1bQe``D!=M&(4{AoSBymhXSGxC67VylC;Rg1&|LF;3 zEfaXDDD#)fOUNwKYV;zn>8JF(^CH@(@Y8X=@4Q!Rfs&|c&dJ$KLm}c#_=@MNK3iYU z6vqC-KJtkV#u0bZ!^z*?tg0~!owPMXTqk$wQizmfl~ZB`p*h#&>X3?|?L=Q@M9uwA zizC4X+1>ZOF>Mk8okiz>IwgOC0~v@$p*iMDw>}B%piY(1q+6#l$U208m8XW!ZJJi< zp4u7>lo7S%!|irUZn))0lKT7N%!T9Nsy;-6Pi7-`rlU_s$rh7$RrF5xw4(aNwtc)` z@z0~8MvKiB7xpb;SPFWcNu>TKF>f}7q&P<123#;V5>-ouSysob&Cgtr3dl*GcfjuA z;C&J0%#-=vs>Bk5$Qbgxus=}Ltkqa=zbc|HZt9r>uBBgMw$R;ANU=D;v_}5CXhLn1XgqA(Z3T&Vk9E6_UZ5$Cl z(1Z%Ew|+*$5VVs17DaUWoo5vQuGlT1%&pJk4yqnVKDNTOZVraH3GhGWuHR*0W{shC z7CS~d>)+FEmg-qxSzGY^qVqlhBy3p$Z9i=ingQ?wA(rPWA<-`}k*3lsnPzz05crx2 zdBFWTw~Mt3T^GonZ>#FI`$Xt7(%JPse;Pp8qRhCJmxu9#B3sR zGe*?aD=jf;BHLJ7%V4nfqiEyFrRV3*y2C30fvAESl9H$v!Mw&DX~f3mdJiHw(iVe^ z`!i`;Z=nP6$l#qKYer7<)Q}eA{?6*Y7E6KMKC!`13rk5l)c2bKe9EMKB`k$DxOyp^ zI!2zq|WWiqI@3Jp`rr0uLuyZk)OeS|rl(lunThl?^)$H%Ifwe$}yKCCnqZpQYRI zN+Hh&Fp~YG(@la&kiI4cJtVW&e|v_=nM(iO)jIJ}D!Znprd?4dM=H1E(3+2}mRn;% z4TTQ7fVf6>XqZ4jtNEmBQS*(FVWgGUsGFWn(*2j!p_H_0X`N-8U=;VlKVR+%z*m{( znmhII8op@uTK?Jlq)bWp#7QoL5Awns>`Gum5}z`j$Uv5DLJ4g7bD-NaorbCOWksCk z55BESyX{jSOB-ziqg0BSRFdlLAkZ7$6Haz|_7xa;4Tt8G*9A`_1EXJ%QmZ5-)%i2$1)ttb$YqsGh}~##`1VE!o1%9?@W5@0rMeMq zVv}mDvroO6fq9eac9U9&IR$NVGlB?hWw7hhFOPfPp5Tf#cv&#%Py43Zq`T=|uD6??D5x}R?w z^<2kGNx>@Exa}#oAfE@#MPNQuI2Di%v<|(9{?s^sp*+(*+l$5|THM^Q!I$m?eGMP| z=;gyc^_0aWx?;-7Bx-?hNl90ykwiXP-An5qJ^DV-_~P?(J%2hT`dS$QS+&D=%D)#K zVLFp>5&;>l*oH}_ zfGA+{ySY>m3gt3umd!#$0L#Q%vy$B}baM(Qwpbzk<~}4?|LFpyOhxaj6AkaK;L{cuWKsA?+lZs@RQQzrsdkMJ6SGh{}O z$RBwIM9Y1t4pmpfZk74v;#k%HbSGUvaO7D}`4vbztJG4ml9ZC-T1#IpAPMKro?1}R zERcW@E4dYAK>VP~d!Rh4@Pg9V^h7%dR#^)clZfZ3DmmweV8+%%@Di51u5jLYJ;Uu3 zPpR?EuyQYpzlTGRo%MMQ>tA`Z5ITvF-&Z)PKKyyr0l?x;8tPDr7J#hKaaLRfthdiV z>%^al;t4u)etS$=+YKm?y@^cCaTeUEN;RE@v|-3rqH^kPcMp&q57HyF4Z;v4)r@dY zc?Eg3@Cws)TKVA_iRH^hH$LM!=K(B7rtgX##31)-C}f1=4RVAPqmP}!pnf!F0B+H5& zeu70${{B8T;T=6_=gK)9;lUHrpN4G3H8dDd6{Ews-#w_{frAfvNOV`8Oni{>2?f4U z>E&EcAR=xN{0X}}iCgj7Ns^{Md?9(pkt}(SX%hX3{O>eTnuo|-ofaES%$ywZT$7tv zR?2e9Z>6FS+xgl&XZ`tf{SfIjkWXs&L>59gGZvftqKiVN)T zvq;e<0*)ighoP!1b2HHE+6oG&YP?j-+J23_3-#XZPZVJgf@Z)N8Ko_0PdO=b<1M*# zX+A;X;D%0!{P`+6%8;ghSn6Utv{%?es(3wZkKEI{7t+k-!HOk)b!Q5_J;iS!O6Ss= z{T~^v2_oA&;8r$kOp`GTO9Rr^@6v_iqC)@1=9Mfuv#{_DpAaPFHDthsZqsg2+pce0 zc;ThGe#D%YLRKRfT8b5J_Lj&%HQgpZOZV>MmKwZX_ax@Ub;G%~DZ;f9ggNN+$8g=k zfrkHWUMgyn7)iE~YAQu^*dkrtM#c}--ct^YRk@*OA{~?Vx z7bZr(hCl5WT`nbuKcSyMA7%8*Kx~IRys>#jo+|FFl%?kIV@?#aIQ-z(*OQwaI`GBp zu+)suj~ShBDUqEz`M4$h`z*O zEN&p}nNU}m$Q=1#?xqVzd4dyQ(a_IHIZBl0E^Vz|*vHY2;`~5p3kBtdS+OCvNDSpK zAio+=mri_eN&$*E4bQ$NE&Xn|j$TM2h#WCUa9|L9b)C4u9qk~C591hbjA*(Y`E!|& zZ~3!#Xn>)xp>wA2F9Z2Q#C~!_Y8I#)pi%V+#6VsldhVKt)`WeTypmF9U+C4vI>;zM zbmhEiKg>Yq*qMUWNc(gYEMJnRAzEJQt^e09jHv-e5b-te4dgxPBkQFlCEe<3>d;9Q z=Gtd+(!`>PLrkVta*4GMQ1l#06zK~7pfn@%;+jd3VaC|d%o6XR01@b?6h$i|q{FAf zsrLg0D+vC$50Oh#whja}W{-=khe`#uH$3ZN8~XY{8f= z9xzruL7px2L!Gk++c{?|UAWhtE6UqBpM!G|i?8i4i-`<8DRBA44`p#p=^u!GE@1@r zHeZACI)8@vD7QF{{o+Xj!4q;90RdAt#Hkf~6LtOrTtYu#wCa|y3hrCLX+^U?3kEK5;12Y`z?e2t{>ikS~k^_wX;8qVj8YCwNOHH%$CzT^+Gz z_JSXMjCm25yk(}9n^S2i% zqm6U-g{z2&NZZfiBaA32Oh{eCTbLulw(zsx81RA^i2OclV}I(;vx8T@Tp47>Fo|bXL7ZEfFBiqKu>6EQv<&u`Igzp#!RfC z((PB-r>ObtjHmqOZBC?e2@^V6CG7Jk^$aLnkdVaJoRD>#4ukepH!$XAbPE0_l-w9B z@!xpxVNm3=loLapYTO0n$NB~dsl|Hjv*}N<6&i>P9Jwv1xgqB#O9d(z2(8b6$psJY zXtE0^DEr)tzQXGm^8Re;?1ID|fV?j^vl1@)5Sc@N#tsjMhz5+F%r-y$?>zuxQdX6E zukQ-=@a~LCmK-bU)WR`hukw|0Z#M`%+w%F@VY7%Rnr*6dM zJxNM#mkxS4+SgV@*)SD-Uk5#N$T|iRGe%nAp```{K`LGlp1=|e?GVeE;AgIcKAJP( zb_%=(fe62|V6P)}$f*!%Hxyn%{NbodyHoLf%8Bph3}jR1KlaO&OFku<{moA5-;e}# zeOkqb*CH5LMd!p38LZudErN7JtiCx7JX&^5#qcJ?MGw7!0}pMOaNgANlmc5MJY)Fw z_F=LR1kBC7?D%DZh^TWh9uF_v_f2TpCzeG~yADppf6t4V3g`*c{7`ilH(GA8-}99R zO{paoR+SQX(wg!F0QXxyKzwJSDSFy0*rJ917XNvHkMDzaVu(MNn? zno(dSqm_;GMZEDfmv=>Uph@V>DU3I*{vbuM%c15xqewKheH7;rCOAJtjHpGrC37U$9D&BFij z%Qbh3JRg6Ul15yYt>Q!Z(h#bUwmRYV_he<>0iQ-;#}{QP3;u;DBxljgh@aJXdn*jr zEHvRWN*%uk!}(bwL{gv5;`Q_~W5KDOD&3IqeIarQg*$90p?lXA5}4UhhY&RQJPinn_IjY0l-z40Jp zWY|QgR1BxlQze9_*F;_^G}j$;tG#Iu1qb7{Xa7x1;NT&P2Qw_y?cAtEs}#cByH*Pb zrcoknhe-60?Oc&$Gh}RXC>LTzC4KwF1mKK`6Jy~p(WSOa_HphUxJ){PetZo>R|+CJ z%lY=5S|M5`m@j2@eEWgpkEh>Ogb&!zx5D28-V-vz?h&dNB*7ILVP*jXhQ;XUtRjpU zVg6qpqDts{0an5EF3Q+M-$~&z>Li$FHdX@@H=%T_*ctZQyhwZt335E%6BZnCqifUB zs7VGsx$>T&iTR?7h7XDk>uWemPBh;Pf@MRy>;rlTeua$NbBq;}AnoHb#qHQiQ%7fL zi5d=Dynfsvv(2!|dXGBbe&}^vaS*SlvE`?nOF~v+=ybSzp33R~gSb7>=s>qbA+!T( z5%p}Iflyi2W39j;N#RVNDUSm!(F}RHEGc0rF;@g6VWSI42t!gB-YBK5Zvi84DQeJX zd2;QQ>+gJp|4X{1B>-SR!5qA&pVUl4wvzH{zTD3POP%k%mzk40p+BUA)R(zcs6ueHKJA`fWXG$*I{CKs1lC{WUy4+l#?+v=Z!N< z*1?+fn6xk1>poBXJ37xiGJZgk$h;Vr$RI~bAbLnR1kNVa=01RfT)My=he1oY6ZwCk z39JI91|Ea`{f*|9w)M@V2^Fb4KfZ)36edY7(x+-`2J=oI9wnye)bqGsW4P{4vG_gJ z9*(XvlgEey`uH0z3qPi%b!z8<76Z} zdNbi&Hhx6Bg1h2NmnVK!ShM(Dk5a#)k&hufq+2Q zy3MlSN`1E(Tk=>Cx6&ng*Utf!{j!%S^-Dd5mgZ%!>iz%sY8wMyFMoFG`3rW(T4IXO ztU#XSiO=b)Gg+y+tYV8fJzq|4{$RIg_h(TvbLI(X%|&%h`;yWa=yItWU$qnd!Iv4z z-b*-HmgDPSy;!C^b{%NMMb`$cVac6u(aUxF*UnZB@yz5N&s-Qf_wZ+ea=H3m@ZU!T zWVru09!y?OD$J;XZaHeq&e%Ec7G58suauV(V4H23uwPD$jTJd6_-~8jz3%4|foN^n zF(ErWu>3JeeP5H6wVtKu>AI1c^JXbZ0>hVDBOh!j82s+dw+HgGTo+}lOzRUkyB-av zV+2ekOyj=5eR9un+N_a3xri3ZkPN4;tz!i2!*LQlLL|U0)86=tx$B50dDvMOd=8b$ zMZnb={6jr%h@IFM>mSt>MZ{;HWg6SpF;1{@{=c~r1bNINFiS|d6>g8F zTs*D~Fui7hJnc^>S3&gL5u4$~u)JXCbD5e3hwWKW)=uWr;hj3hi@&$1=D3u4j=d^`Ljvjcs?A2{Ey zYiU!)wmWk|#>S5{mHaVfZGUS~ZQ!8PC_q@%C-fN&+ zhHLmSjq>gfyD5Ldao=wn-3^F<`4(i3cvC$r5_qkT11I;^s$0ns2z1!<|$ZE8)yBSKmr{EaNNd zMnGVlqX1ZZh5=;oaMt^I$m`lsR@{sYu=K81adGjSyr}yjqQjsO&*Kq;?xXwksVJ_X zjmHEQ%2%$3qA9*d>fcw_!GqeD3ytlCeOs>CvDf;vi%8IA4zsLZ!{or;TLpup^y4YU}xqbKUN%t z&;d|Ys*4*Vqw)!5(tbH6J>Bz8ge!>tKNvF+0Qg(przG4Clg9Hu1>Z|^)hMQpXEl*f zYf*ePefKThcdcZ zyo$G$`2x%6Mg6fSB8zQ3B>{VnRd5toCjba&K$YwaQvdx$fvA4{reWS^ruGA(eWscA z9G2$rYXDPhWy=)z(0e*6Tq@F4Q@>_nsrv;A*7%EK5!Yok^*1OZ`=r}Q2>v9C#?5#= znq@AqEDb{wUkV(Fz}Ia&Fpna8dpWeYkjV>XL({yFYq&0WKWO1ot;7V+xRhj8Ifyh+ zBRCzo(P=-i({iNhz|p#%sQ3$4!wYgkLc+vPX{07l4XPEjK0&DcznsKZBdF_qLxTJy z##1)u3tmGt-((xu1!i6e)U)e`5xDR@@N%J*AoXdyOto2Q5kQk!|4%k;P3W6ImGH}5 z4AF`FxfpZrKjb<_UtxRu3r8B~03A-==*BL-Vm*Z|yD4-)R&1a$0};-?dVu7I-~U@`)gHb3lr?)s1$`*ebU5y;JQ09Nc^AdTtY@ zGmU+gEl1-0eBo4mHQgRz`6i5z(5PoAC~^m4nH|g04mPJY7hQP_Y|6gj$UGcfPnyKY}H>YeG2VnQ!?x z%=5U|*w%0dQb$@GZg9WJqxoxU^>x#Z=A%ZUh`7A&H-g0py2drLEB* z=n!3;kn4-Un>N49!LBc!O@#u{PBTZW@}&5(<&E%viqAGhCUZkrVkYFGa8boF@g66E zC$fqhtV>*veN_Ql7iTv&9SyULBk|GFIMliRs;(ZxN#CrtZ>QfT8WIck1M6e~4i}MM z_&Nzm5CdsqCC~2!dG4{^y#Y;^L-f;)wrj5m2!KcE*4mnyfpBo<_nu!Th?4F&xylu{ z_C~({(wx3sf2&Q$;Zht5EEzVHHs!XxXW3DxVIbSVt`q^@d^o8+GeDAM>T5(rXl=k3 zLaIRKZ^u3B&u~ycV~kk;v1fZx+(sDomu50(xQt0~;;ZNY^5FILR-U4LUHE-Fu34eC zkr@p3Ih14iMy0nh{vJ1O&2+r4PAEj&3QksSnPwu%1jh$HD6r2YD;hPB4#fp-h1vb~ zxExGc`gcI2VE~JP!Be$o3J0W?N0NW14rv|88L@2$8?!yhNv%uAn6GUkDjt7gMkgCC zbWC#Bm3JQ341aezTYV|wokb?Aqv68Uvq*O~j>zK1uMbRoC1vttj}|Bl>iIJAuV!`_mdW#U@UXuoFqHN=%N z(+Gef0QL5I*I6osM9aO_P@jKKetJ*?d=d_Uu}Ez?m{u=sXd3&+*Tw=czqCWV_@*UL z9A|@GYmK`7zyrxVs(*Z$fAH*K&=J4Y6?T=r#4fY3)Hk1KZheH z7ORQXo&B(Xo8{Ssqu zA^0hy*3LXS_*Z($kXuegd$s?hO;$nnA=K(5udE)iOsd`rj0eB#fx!}rbJ~3DsM!c-4d}!xoFQT4`XU7NVfzc|EgM?CP}<95 z)15=MLMp4}WXlT=_e>m6{ulQ}kur@#vkn|@(iSim%?sk)>^|F z#PWcNwPe486(R0q#|Ab1mI@}n#tFp1kKSN#D(1H(;{2rWmMqZe5FYRjxSipBgcu4essr3IH5e|TAnDS$9+E!HeA}iAd)#nyltd<8yyR}t6d)P{@E!f~KYZS<` zud)7}JAz|siiME-6pmmaU?R_Oe}>&gakwVmfrq>x6lwfN=+!k9Imx|?ig!Ye`ZV`4 zKcMQ1YEUe3lP>P2w9+_5aBnv%RFoX2Q>ri!(=mX3)3p$UI@d7KNk6jAbk=q?24_rL z5s70UM2n9a^{=rGo+7#TOP}OtK&CkCJU_IJ<%fZ>G~i$R1`V8)SPa6=e>p8) zc7sR>sP5BjnK&C6JbY+mp&l(#Y}XxNw5$X&XnScZ2G&1)L&`bxfs^p{6QdrUG0^Go zcpLyj(ofkUrJDWxBSpc3L-#GYZGnn#4Qwnd{!Q5}b2jt=c{VShVN^!593z~Yi_y?`k|~@;?Oo*Ks0q8 z^ZcC4uFp65qlAC~It8m|?f5B0^6_(KxlVl_i_=nrwRpp;MVAlc+vxM#u|Av+mc-oY z;es?PgbA!n>oC;eK}v5<#)W)gRtOJS!8F^;mzgW1y+K!+56U9&2nB3pUv;hyh=RMm zphfsV(b-SS&%d=ByEhax6t;eK{^I=QR={ORz4CHiU`4&suzhSW6_`Jza&Nt@%-e%J z#fk%4z7A-RmfHW?`FET1n{x$kppO#VXIKI)MFM*!vdxSu32rsXF?0*OZo{qH;4$(^G zuQ^&PY9DbTNzqf+CiOmx1A2j34GtQ{*T<1o&jBFztIH1Q?Io@(f6az zz(045AfpjR`qEd_zKyIGo6J|L8f>lk5k)gkB5mAV4)-m5xiC@qp}bO-hK<8}JLF~9 z!nL(h)^`d&_@osB%>qOZik(YPyWkfU1!T0p++vr>tD5+#y`zUVTz@$`{xGbJn<4A(f%6H+WB%(J| zL>Npg(Jn|iYFa^7Fq&;Zk!zGv{~Wh?dAT#(NsG^Jo)b%11vADCv9xiBjQ7}xGl$v zZS1nf$Hj;0eaxH2I|&IB ze|BsFua^b7kA&CF3A^MED=nizs=u*o?YcRi;py=Yt(agxP|sQGbuvA|7T$$<>NM4E z2<53AuFbeFKW?j+YL?$OH`$$+`V!jjn7Lj7uTCOg5^t>{K+PK zPFwDe6n}tZJK4zfX+hZ$cnG<5`Rsl89lPdsod|dUgUpTLeA_|c+hef9GvC0Pg08)( ziMt=j5hdwKU19LhYwY6{&O!GmRkt&7nHtdI%sDl{jMGnuMgevm?3SZRo#!(oOB-;3%Tf-M1??FP zKwfpB6?PABPJ3?N#?_L5KxhaNuq#Xkg$R7G)xwl7D|;;&DUV18M+}T4^oRRV3B?(AV>U=Gwry-?YLqwOaQS%YKH}GBErV2Fp~PsKcBJKkUX}IK z&W+9l`Icd=N_hDYX`c?cY}$Yy4E34GA~go=~h$Cs7o%3fJ& z-~ar;A=5g)xF-xtP$5C$Y%}V^(vr^HHSCA%_!X7$gTrufZa;qnqe|YJ>5wo0pMZec zL`yfJ1Go+oA6V{ZDitYvwu1guNN=H5!_3x{9hTg%dbDf0U(5s$7bZ-~XZ2$v29mx% z8jZ+=Cw->dBJpd0C_H9V!Jwv@gQ27-7(>L14)qu6nv$vxJuoLxC|4@_ z<8RQOPWfE)InH-FEZNlLTzFxK*6AEZr5oPZjI=aLr{G_w4`c%An}*&mjw=(8&<5R- z72dWbw)ZF(>WUA)K5K;5F8@^9b*$%a6hUF4p`kHsv%|+(dUBf$`!iJVt3v{VYm2??4*O%60YRT*s`e*RQ@he(kxHx8;-yv@zlomDR1JUx9rRfr z&Z>lLe@UiJ!uCF~o)GM#aXyUetqJ^;oUmTv+b(w|~D&OWn z(ZZOXgxbwzSZ-pH{hCoDDk*W$Nwq?_<;F@(NZn`Jw=s{PqT zx16DkU&7(n6UTdBGN2@8SuSjxBTB-_X+t4)0o2*1)J5PO*pDh6vnM~)lRj>!VA4ei z8%k|)+38vP2bTZ2Ihb-hr6!Mw0|p$5Nj~_6tq;J_3;Ncd&};l$F5HVCs`^ODx74Tj z8T=eeqxj_uurP$^(ZSvPba-FGnQjwP@dP##j>w*cg@sQWsQdc*8Bz8O2R^QfCYEgp z?qw**5E-b|8Pd$0Q=F6Jge(z?8#eaM^)&`zBMaKwe;(ztu0X>+)>f|Cn#k!^%B_^5 z%IUyyly(JTsWS$LeDW#!ox4<*^psAsRQ-7f*`BlfcVX?g_mVnjS2g?)Nta zumhOMl_|9VO|P68uDf`bt&??m^AyiyYFs6Zb2^VP(PQ)|w*hwt1LibxL^ncH(1WiWE4`CB@E62I5s z?dq@ki%7neK`DIahKkZI)K6#_@7_GXp^38*;gfOi%YE%NQojN%ayM|us(*uPK|weZ z%FPIgnL`-mLK~tn|d$yhl-hUPnOze4$ zFIzWiBo;P&72)6eIYv*ox)#p6Ab>)%bCH29zch1gK!CAjg*W++JoH3lvesdy`63}vOL*g9k`awhP z@pW-)ll=YtmtL4=-4-%Jc8OUoero<9c{25d{!9(;t19t*F)Q~e+`l21wFHDBm%B-h zv<5u7sE2E&pyw+gx*aJka9vL0v?GccF-R$o|5I|mPO~=G-PxLVskr0s=r@YHh7r|5 zhws}FfyWN2R6O~ugKf{$lbYQ_xQrf<_)QmsYE%oJ0j&GxT6o<|@!p_7+_LWlMtl=5 z`=+XiGES8fW9oJ4qyLYkbBxZcZI*C2v2EM7ZQGjIwkH$Yw(U%8+jb_lb)I>@^E)dm zD;xLjuD-e|X2e-k3%D|O1i4x^q7>owkStWDZu=!Nm)YoBDhI18IZagzMc%*gKEk5m zB*k-<{k^5%37Z%-q_uzT+xf41Lb8;Owgk_~ZfikLlEWGHgfcF$ |>)(lOPU3c{5 z^^9dhQcTX5;_IFV)jhIbSCy>vDa%my1S^=sHzt|iDkF--O6u1s>s3)#mn;QhTkU^3 z@1QWy&rAOyIf3!oAE?r)09PX-lWp2TY;TDPk$w*h9Ozn(ka#rb4fG`GRqU0er2Wkc zn_E1eR@q^UeD;>E>#^HhFd1-4o#IqbC}Qx_W29@bTC5$wzDa^xc2Sv)BfCb-IMR`gwhUO zZEj0|tDgaYLC|@FJEmQ#Y2(9{@qvdf#>|r07)EsHP(%EKgX-kat#4bTUg<2iq6dVk zsV2;nL4j?u*^DrsF}+Z<1opOd%`=XR%jZP=yEuJ~mtyjVXKh?@_Y(Rwe?mG7x5V9J zyZ`;-+dO7_sf>62w`v0ercSq|uGzPt_b0)unJ>tMHE!fw0`?{_F*e_MxF#^fcw zO10VNXo(>9&i%CBsoyYL`(be0IR;RxVQs6~LN@WGhrb|4EThR4@VH$Rm9*9p#*g*e z!y+PKVp4=6QI#yk+RTsA@)e=DKRa|r0*Q_siTuA3f1#mm;@2`mHJ%}sEKfM-{t%G+ zoIMfxE2*0z*QPENc%+jrIv&8)_>(`TW&cSlbjJ|!ImF6lU}~hk2gZUaQSqvGzrVk5 zyt3kWXmSvWDq2uZZc5X|!JsYEd@Vfr3|5Yb*}A@`Bo~GH2#24y8^k zrU*bMZ% zX6YeyeQt-=EdDAv97cnX>z?1=-yf($Hpzss)Wrj2b;m@a2K@fnvy2(xi$LM>rbH?b zjinVk-bh+E;}aK!HD&dS%M=*t7|*)x)s6VZp>Qv)#del%%Fi}>_lSd_V5gBbHPdv6 z@T#mFhlb6@f{x+2jvtXGO0AXyu(UP4REdh$3LZBGxf{he-H_^y4BgOfuAXL(pJ-|_ z_#!;+1=&hmuX^qQPL>fWJ6Wd6nnj9E#jXOizPR=$`-4fu zc#(cz!Q!*>1^Q=S7oNUW+ljU2k8o5y&mja* z865Rx&naq)~39nYLlc;?)9uhUh3wmAX*n!z!o$$u~d2 z*zI8bue6{4INhr#@HH23VT2__3yQfuW)%d2#&R@H#^0oxdeN2EuWz#%{;;zs+b|Uh zXDMV$k^cIi2w%a_P@8(@`BD_w%FTZ`-A^Wn$ddBhirVPvMl@exsdBiH!@PHN<6TCA z$t6VZaX)dp^y^u9Y636}3AhvbU=R~*%jmhB^%Yyhsl!5tJ(gFS?T_+U^+SK1XurlX z1``>-sEH;8QqeN zrRUqa168r9PSQwHQ&$jalbUM`iC7HM=rvf}K#N4@hb9{iwR|^U%WZX&n zNCDJ4aD$G<;2$l>Ur~@$Fgw5OYczt?4>DhDud&B=kptA0`{>8AlKLTS)|)IrI$E2T zhaBsCWEdiII%c8TTEQhf8VK?q=G#X`)Zv?;nE&7+*l2yyw6kN zl2eB$9&q*C>Ojy)meS5Cqoi^))N>`Eb^k(2MkER12<}Fe_6DfjL$6Mc;XyP=sl`mt z=;bO_vjq~RH8wSMYE-b%Ux8-Q5evz*4Nh{({(O-z z_&VWb*58}@yA_&}Cv&I8CMe(g(Ozvr1=r04evTDgAKl%ENALs}*<%d9Y*gMeA@n6O~lUN(Z& zBd9{I;+g_lkx3ew^r^>k=erMN!|aBhSMd|o>P#lT=>+E4#lcJ(%9;{rA4MR6_{OW- zdm86_kOS?u=$kg?`q4`oCfIJPF;JnXM*?mztYmX69wYR-)&;PApWk9{W#aN4AQmSv!ckVe;PJ3EhI!mNxV zEF$$C_XzJi4@ICzwkO9E*}Tk*HiPf9D9v?S$&4<9h(HSSTs(lrlf=95Ydv4WE=J72boHVWU#< zOa4yRZ!aR!_5Ce`q`I#k_Rhv{v$`wXX~s9x1z!a1NJZR)mrN~@mwW=!ORjB8ESUwb$VmYFKJWJkk(tN#Ku3GJVgDZ;XIegy zE0nnhTovvoY!sFQF~+UMa4KF*+{nLjHq1wiyw<*S%tfca=gYje33A zp(Vz7*`6QSN4ykfSso}{$dPd-&EpjNaz)pda@txk|nHoZe{ytl2PGkY^h>y_eYTk=P zcBe%Hga66Qo zY(HRZogUj+vkFf~jUDaW(JH~$M0O3UIh^cu7@6K_q4elfR+AH*E+hr139PZ1RD~-G z47OaO=gm?>qzC0Ou%IB&#SPwp;g3-?M^_P-bjlS|Cgc!;BuN5R^!sZn)bzogIr^8e zdor3dJ2IX(FYeSgu&jMhIF%3OYIsX<-1gI)s6W9?_by6r3rGYx^&1%ey+I*E8;hbj zfxkKDh!oBNVEJGN4`rms8m_9h|1Z1T@SK}?-XhUY18IadvVEl6yxnJjJQKt}! zghXiB*x0p`6)%6$E<%aNv@;9;u)QI168cslO;pSt2UB$=B&yNGO4Z*aI;OWf+_Ug? z_Om1eio+l4?8iyyv9S0;+u;=cRhtyQOV@yvJmewgcIc2Px;$lOOzY%G? z9@pkth2bhDx4qEl_LWsz;@F_p&jc(dQmRv8P72ot(!A_@4)l~7IEj1H`0GBpG>{e` zeNbo=R)yuY{bQ2tCXCWuu&NI98xE@ZGUh!76xQM_=XgV@;1D_QCs|+|2qHUH_SZka z7}9cNgRgkpkQjgqa&nZ@QcT3r|6z~Hh`k4NXB-r_;#5>{%_hY;iRW$?FdRn+ffR9a zxw7;DlR1ltP7Bk`LJ;y5i8h~3co|~t7WoT=Ax%D^K?$Plm3X5gA)TxI@W+#PS zPrJu%p~PD_h-9S~X+O-O? zzhlb5>UwW6KTQXFjgVPYPhgf*f7CO-6;G~9}S z(JdxWYU3tJY2GsWOkT)XLc!KhVC?|#V4dTj{pv0xYQt`$MXppMGmPj9LFX%T; z4(9pO{9n^fIVjxnHm(~miK0gnFh(*V?SQ-pFQ$ArM>AVt!=VVY%yW&}E>M)ua)-{Y zu|Ld{@1g0j;CE15z>dVM(pUT%$d>|Rh5WRg8mY-@-z(m@I!+<2vz{+CeG@fMkr*qg zujP$x9ydZe^zUafl#q7LD#&@cL~k~HA0MzZ18$_P9hMuoNm3YXNBK=v0k<5rynYK9 zw3>C(68h=c*?32P1Q|@QoI>a$V! zUu`AqB(n%?oN_)E^ZJELU%2~+so`0QS3k%rBd0~PeLu2Kbtz1%_P`mp%S`Udwo$Ba z$(g15lH!Kj64*)VE>aDaKOgREAC6khKPZzO^!IeuSGf=wW6#+Qdvof!<1D_VR-G2$ zBKNyYnlVyf4n$~`XZidOn?vCNj=agJ&mK!H*WK?=n98gc%(%b&Azl7@B4viu6B_=! zR8dm%#Y4ZzG%?>yxa)VG!2_9Wb)O2WHKwa5dmUv{zzu&%NUa=#RSR{l->m-jo5qk| zg1F6Mz=TQmfE58K#{4&%RQc_a*cMnce6sD65m2d>i76vXwwK|>v7dI_kwa0Xw9MRIf%|&6VPTILg zC*N=}pz$JG2Lx}yYP6nrdq6_4TPgJc_ZIAXuUa zOU&@6TvFkDPzOD(mZi;ZN?UmEFws$mR(PhtEQN~7F()>1I2vr9`CqO*s{1BSse~xY zZPTKW21XeiHm#rX@4eI%$NPE=w&Amu9kQ_l5#y)e;W<<8xRHF<3iU9jR$6gU_n{Lt zgRY4El(_;uKclVj$?EMAJt+&AX4X=w>gjR-2WvSLiN+CLr%-35OS@VhTc!Ht$>T;~ zWD|n18h%5_2^%Fy(6J&Ei4FMXR3kH|%t>viS3wdq)|CWB!KTbEzVqurtiV;dqC)7j zGr@oR4HgPul1hC^Z;-mt^!Np?iQPAD)#QzZtb17?e!w5JFSvCmcyL6;D*PFm@nR@S z8aJW4r@PmEJ3$7(JCG%LD2+vPjw^zRUCNkls8tx$< zU2fDMA6Q$2A(G;QwT$Dk^(W3l!}J6mFxTl8z&GGrlrX|7ZBvz%uf}mdTzZ09YM9UXZ*hyh$=F;bM?Y8J)zvvt6zEW%VGP z@W0eu^iv#Xp_#kD?Z4q;RsKu}tL^LlM0pdYCk+50~rkj z4|bqhrF!*ge6p*#`bwuAr#%TBO-rL8EwpHF(r1SYQ>%E7|ML6QQRa_{SvD9N>T#uL z{Y9oLH3BOwJOzNXtiOs7QD4@2>@}d*x_;*05rw`g5(!OT(HQhx_@VCAfjzS7C;ntj zQ&E|#e&jdc_XcTQU+jD`UHz<6n(LGDpibE*Y@4s6r+-4TeV;r1m1wMY5I#(~{CZ;r z(Ua|3n&tNvG;}c1T4o%(zfaUQ?QR#J2dU^US5PZt^SB{lNvwV&P>-y+eGA#^MO)d? z@y7H(>XbNQZzQ6wzD&T=?YB3+D6(3?8+m3%v0f4f)W!8a57dK>}MJI_2L>ECcwx&r+ES(thS1!wkf9B{I z;#T4Pa_;Z`DIDG7T8SW&()O?PT}C{~v5ntf*L(c8%Bpb>`)@EuPwjz=juTeqRLWRo zmVe0wEJdw`x{*s3W#Qa=|fiVzinA5B{L|Y`P2k)Gnb!Hu6EW zKGz(DY&S-&5rb|>lc)5g>MLV%Kd%jO3!||H2U-2{gNry#`Ps@3sOSWm3b^SL)*xkj zZ~l_~krN=^z2cdj1^`?xzSGq%@Kji&xK*%^EFoiCz$yudhM~#{0dk7Xt%W&!%^hDs zFS>FmWt6s-_ad64p=>?H+CRPS`H-rp6V*w+7}K_46Ng`j>#ROmksY=F`arf=x`P!= zMkD#d@S=5MZ6e5y*B185FUukRQEoVQx6MLP0S&CI?dJX-A@C<1G7hODGzjL1H{}~i z(uW)!H=F3jS5~~yuz~g-O&We;N`_H3HP^}y;O{hjQD{KE5P<5SS1^m_ugSC2Lokr4 z87HMA>JGkKE;dtBt35wDhz|RBRPvQVwnw46O9WSUH0sO08s|}B0W_dyk^OQYcW_C6 z??bKYNY%ZKJS#saLXBx;29h(1>(^yNzKcXn4i_~5+H(6j8_2g5(}<_XA;4{YkxI(TU zaX_&P?JzmQKCc}Az~&O^DVCKj@8avoo~yKWimfW8ms;TjPXP)1xevJ5x^f_hs|2oD z)l2vI`>oaurlO}m`~_ZecLR5+41I~5$xVw5lkLQdN{I>0kg0OJl}r_gWtRzhluVzA ztE^%kITCk>K0O>1sYD_wzYMQA;-<3>(hv*NA15;RAF1-k{*KkL;{Q@&K^EJ@HBf zGrT)22q*;nMPQ2iHo#bZ^&3&^|V{8b?sPYPSQA-P;tHQTyS zTds^>n5LdpFnLI)ODXIR4Sx`@f=&Wls!?oH&W;KW!ubpi1-c#ze=V^_bMwWTVM3fSv&Ed7puF-Zo7oxF zUNR^;gjAz#o>|2Ih*pNtY9d4!Pp0gw^S$&QU?qr}{)Jg(lE=$g)>&~3U0d7VOnFWm zu;3tgD&PC_Wj0^rz3t>K7ds&z`FgWqkuG-Swc(SZeJQ`Bxbd6R71xS$4zt{~(x}$$ zh6M6mL3yIgsOVYsAjXL5xo^1xtXuU_J!lV`+Vuhkh6fi|lj0NDIcUckFFk~-&z}`b zq6_Kod3LcFytWE2f*(PZJ(o> zT@B1ER=sHYh7mLxB*mfRuR~)Zfw2aB#3=}ss4P=#i$r9mU4YL#R%39<6Q%+S+^`$4 zQzYT2DrbxX64={@dTv#EHXXxrwOZ`a8xXi*xAck79+(vd%fcv>2FQvce1`>!KY)lD zzObQ16l+cMtXCd7$yN^)Uc-6h)ePy_HZWYN$Cf6;?F|25$|vytb8HjZ3}O?@OV=!{ ze#6?eRZVxm|AO>gFQv~Qd*AX=!}p~7ulCIW7+}BedJcsKkB6b7;~P$V;0DH<6DY;j z1;LrWmCqR9Ywag9D3~S>j~|@)w7;S|)vT!WA+~;#>UPx)M##1O0*smyzDYGN;bM~I zom!d5AqrEvh`yy5={IZ5B8VMtb4A;4NUAlQMxdWG0sI?MJ@P@aH{dkq2b|^}I)@+O z!n^Y_48`WdB38a_vDFvL)f;pG;_Na=5K3DMh`(4*`#}63hD-!mCrm}eNM4+Pp>$58 zJ!Km4PKtRyT!csm=3vB;%1YzkIp$|_^z{YK$5RP3AP39$%&mavSYQyvb zdb4I|03&LYoKo{C9qA@MEl(0NW@C)ZIq?(Aj}t z%xYn0!&?miK;#&%Fp=Eb-Kcl`OJ&FfW}xtHVX?OhVUbTj(?c!6ley?IS(nrrnvBcD z@*+bB&h$lamZ8)3t{^@93Ib1d<;y&NGE=K{kRc(p*-$)wzGk07Emja@`iwNl1BX#QX3Gup#6#F%L!=DHNpH|k5d0Q5Ugokf_Gzj zp>;o;z30@X9MzG4GzAAP!*%uj3G|04VGCN2f$PL%CJ)rh9g{TwFDY3l08N_8iP+dB z*skTgxk&_(!qa=JlcWQpr*>)6xVBK*K2F%Mi&70hh6N?ye*hsCE|e#@7u8aGBmzhTf6+7I zd7BvN>2Y~sQfLoIgtJf$-KNgeXu5IrRS)gB8j2COGCvg&D7m(=xLpvnT3a%dm;8}y z_V~7bEHQm0Zh9S>2_UEukojk3YAiQ0;SB}Lt%uKyi zF2OnXv6DDFt(h-eKR`DVcN%7dF?3C<17;g6fCQJY50wPuPwN~1zK4ca!kf!oM9tz$ zL{C99hkpsZ${4`vl*L!=`8l6JUjkwD`*ePq1J`kjMg{z~Hm^I{o?f$2Am2q}my~oD zCBaaU%Zh;Ss9bKF>>ZK^5KI^4H+CAYKD+Cgs)2n|;!fA}QV>?f*3=|BK|()n@O@o$ zwWP8VVLIcai7?lG2_Ul?jh*Y9{M+f_V|>lBG}}u!y{QxnRzI-zPd2y=vDY6gIV0Ny zq#*xX@YQR{;|Cp9b-8w7hi+`WJTq8ukUp=2j1($5vh586ZOdxJ(UR((9I#3FvW=!M zGuM3CTuq2!rGpk4VLzs(yxoruIdQiB(ha9aNKfFF1z%lV#=qSrF9sezHWCX~utN8y z>C2m@x1FBJR}}<$7Ug9Rip^(u;Dti*lW+xaRjF%^{N#ARUQgRl_?NTuo^jC*aT^6h zkeeVS=F?7$RMmg+HGky5;3I7RxCna^FDc)h>8O}saOLNQGgnTkPjr5_Oo5;uMEea{Ttp-Z$Pps;eygd#BW6p*xNiAh9sqHnTyb-vO zy|LXDt|PFbxsoW1H}7Kz_y%r)gO(Td;7>a~U>%<5tN+VM$_rM~OkyN_a%sQ&wd+9@ zf5N}r!b0vp5$-+=OK0-qj#6AW@e(u{$yMw{>SoW-0L8u7rXgsDOVtO>w8JbhC;Cl+ zBMzDm_?ByzqxU3uBq%J%wIK5%b2s(@jNleQe@oCH`h z_b&oNnvPvG!!?DllFGRmBOi?dd^mPR$_z%67|<|KY0D9obhLi`HNp-w68;g}ENdd* zV?-HB60{^eI!Uhf#56SHAqBBLb(l|-$9fdPA!yik132Cx;?++v34PK5oI z%KybJtEl}nFukK~9SX2YtleF)phE8hz3ec?i$571?wCp*^)Eyk^~~?=^eN0qrL75q z+n>x21wR35B~GOei!>Gm7W5o!-SP9r0DJWEMU~*G{DlhthU|KInp86pK$TKO)y(-LiCfk_B=9s+<43v&B|_Cv_eL-+u-;TNvu zSLOE2A`*uNotc1)CILJZ_5`(~A-KsaP=6 zmCMe;uAn{~L+kuQyfPuO52lxEyiT0Z2g;*?_CRySvU-fEF<4Q=weP`1XI>)yHad|xyoADN2LMc!&OvtG ze?IT5oaMQnw_G+v_k6*|%1zlk0IMr(jbS3|masl4$ZUB*<`rd1Jk(+?&*9x+xf_a@8xyHd|9-YAU zJFs?qAKkb)pV7;dv29;&$hax7Ezlq5*KG!%YaW4fx^Q9K*nxOAj_|%p+FFW=f{~Ub zF+zsC8QGO=dSydYZ-i+{lztX8rUhJI=DS1i;P`|jbSXvhcy3_-EXV~EN+ImHFENukNA{Xv z>{6Rt|1U0|rSNaA;-Ms|c9JI>ysT7;%EEgO?NUK&G|FiD1A&%k)Mw+1%8yjYN}W64 zt^8dUMloK3PkxX%IAOf66CmL##)gL25av^f+7|aU+g(w?>j&8z>WZQXea_0M@uK4- zWuDQmL>p4rf76=ORO$lK1Qzk$t)Sr%Hacy8IS%(R2_E{0#$&HMEcE>!Y}*x#EfNPg zg_MaIR(%+~=;~^r*JJ?B1&d&x1LuHzsUioEA%QfDQHfCi?HkVr&yf2;D|%p`E%PF@ z>uWmiPx00neMWSSmi!et+PLf8r_#FT6834(~ z1YaY{@VS*F<+kUGbkoRR;8!_bq?xk#hj3@f?P$E7r#~djFMC0zS)uPbfa%#%2tvq7 z9>#LtVWqMg9pv*_%JZx#YOz*xyvU1=AsdE6`k~$3x>BX+HCNURnp4>;&R^)7x*30d zDrgcxB~bFTS?GLp>i6@qsaF#O7%P0Tq>!k98%04_mmM5htuo=-BhO=|8D2;)mU-43 zLE@{<;xW=U!nP=X#W?3C6Xe6{&rF2;8&4c{mTro>BEg35p$oPW3i5PpNfSp#Q`t4e z4MAuV--dNv-H@sA%mY8+f}_-q*;?f10^=#bdK2?AZ@tQCq%G!fbiFeve<*EY1DrEh zOC#N*%KK)!^LhRItYy%7X`6Kpu9#yVpuP|ceaCuNn6JqgN@lwjl==)JmzP9sO_@KA z_#c__9HM-Jn$ki7qV()X32j-APxSGtWIBWGJotA2C|)2C75GCc8INEIjp*q7?`53l z;P+sE7=9etnW4zuP{U6w8BdpW1r(n!POS+*`wGX4!m1k)EzO6_V=%VF%Ss~WuPNV;r`&aym?*G7EqNQKL|j9Vp`{baah3&GglX? zLF`c4k@b7}8+4bY4MItk2jNL%jQiPdeR`C)N>Ms@K{}GI zLr9Dd6{!>lWJ{1IV>utx1yniQ&a}v)1GW<(&CVFK*M+1olVxjlIp`X!QgkEEqH^{GH{l#v4+t8-# z5;(sPTPJRrToD0#*mTE`@{vo6r=};TV=M3?bhD z#d=GS{Y6O*+87MMS*WspWzgyHKUCfY0tLWxp_irXbrf0;RCAJ~asy7eRH4YtP^r>Y z*U^!bf2}RT0Bf6}5yNKYNjsKX?Ug!^H*Zfh*y*uss#y{tD-O*0=rX(y@R*Q3#QB9Q=;?=|5=%iy+w2 z7~I(;uo{A-lm)9XyuIoT>2ns0ZUl1YAao7^mCtq z{O8Nu!d%*K2X$DqoX1I+IKIx zKvVQ<2cqJ7>Ul>EOMaKHyKUY=eoGxHj4;6o3!K1UrVdkIla3r|?32ch;#RE_Tfzei=u|jtrEsId zMXp~6jE%72F^J3i>NQ%SQ6)NM&)Z=6Efl-0&N%M%XAn$QWr^hZ6u(RNk3QvsnQgee z2>qdc50J~GMMhb+tf3?aX_L@HI48}JVRq#Okv~`7d@KB|UzZ5dA3eG*Nud`mI8Kl( zNp;U~qo{@{k+w`l(p%ZYjP^T`Sh5&zNivFY$sJi!4M<5H`&$*-LN#wNj0Dh06cvBlJ% zC=FQ`DyKZ=Z*eSHL(|e$C{^m5ASGgpdtJ~UnwiaoT;d>^W2~f>p4&g_haOOM5RC7kEG5GlThSh%H zfgB1%Bva70?+Z{ZYWm}k+qFLqmVx!mtT=v(%#0=#K>C0gL@1O30{%=|-?(7k%puaL z&tgXd4V6iwzztxQtF;KBdwNR><%)U!kt}$_-({n&8IsvR2sDDiOEd(`hda0yF7MZ@yuyGLT{so$;URw?5t1NL zC3m6@0&W*m$Gqnt;82csmV<`x#EV4?O_ea73v~Uh4^%>sC(T0{Zzi1<x@}I zrlZ*|YTia=Lz9U(^eH&|c|%hcxZP0H#>2DI*-@}x*?vD@dFzU=)BH1nR1h+fF7M(Eg!nz6O<<$rwpB*G^3gL5}vStyRHi=kqt>D18lAx&eb6!msp zIrKh9^;-%sFhq~bIS3irq$Ne3Pexl}3rp+g1YaSXYSotOElh%r#pPl) z)Lb^xM@ug$>_MqD&f12D+-+U>&Cr#AMm8mp&W#W3<+wSLk0X!Xz}i}faq%Gg3jM*9 z=0-Cs6W0?o1L}=btyB{me~LI-8ff{+NG_h^G5FTGaKN>v#di5JE2TyWH;;nku9G;2 z8?mfT{is1AwDYS9X&Q^%e9|6k#i>G~mCjwqinGG|kXjWrx|#61 z>9)^d^mRrDG0`n^JWlFhn0%N>R^#BhdiQFVyROD3h*H+?Ug=mM&>qx#Ny?M%=0I)} zPNJh8NyWdMp{Tr+1LzG7y?`-ZtWRz?vrnWKd^g9iKNQvW;f8QXoAwS{L+3npiGG3y z5jX=i$oL89w00(o>%KS*i<5{9QB$}5Z~#D?F-)>Gz@z*EXa5CBa4HQ-W99HJb0;@q9^Y}zOZ*O3Hyx7Ix5+ylx#Rpbwben-!0429Rb;P7{im9!o zMT2IEPHuDsmf!JPhb;)eYd#5~uov&6d@BAo4I z&F6>h7L2D}M-NbC=@dS=cDAm%ni@1r4&NDjb?Mma7-wqFK<(1l9cu?15j&&^0C4oU zoT#X%%|tPia$e<1A~7p>jxc)RF{t;2;aagy=A0mzM_0FvM-2e5JS4&c(^MqR2wO0Iy4wz{v zbR^({j)l!I86@4+iS(X1Y10>Amb=X#oqmsE)}Yn*L9W@+Ge#E$aroq-=y~U{^W?1F zLhLP%BHvsf=*`NLlfL?k&9y8J28}nq01uY&UMc$AgwQ8HPQjL&dqXlR%1}%2!~R4u zAc=LM?*ZgpJnSI`xm+C5jlkm4Ol8rhSnBViPpw3^%^wil#~oQ>M~#eIw2RBQ$mLes zd>6iFTP)}K@t=UL!bI~A$fVGs+HjY7MqAsVBEbl=_ybjnElxjIX^VtY=@Lz?&*Q?o z+r#FKLgokccuxI-MD6Xt+my$Jm6k;Ud;fKc9z;zSnjUbGdC-Cd6!NDB!miE*(0Fnn zTAh|C!0ueIK_3X?&T3u0jPK|=w1%5L;l^Kjk>gq+lFH>tW{!ueEpKN_ODesSv(TO6 zcy=yZQkAQRbJorQrYyQl}Ci~5j5Oohe zQDZ#^w;!UIn5cy4fV%6~@KM(d@nh6jL+7fMH zu}Ysr=AD@`H<7?>fEm~fWNVv(Xgx`(Ttf8rfNzi@a>8X%Ll^J4jU*kcNPDCs zw*-SCX_`tj9RShE&yhvYYPHeGwf*pSki>)5N)OHwz3u!h)#^-OGKxnN191~`SuS(F zDt>caW=e$Mb9Zp>9+lu20?0N3%kjCM1i_@Z#AlPW^uSwtW5A{cfiSjw^x zxl7JN360Pv0No(8twwgfVG!>VkqjQBc2+uJNyS^4fz{16>c@;rTqT3t7~608(sK|p zzZ$?l71ASj8Te-_W#QHvVs8wd895tdn@2n&l~ZlHOG7|~o{t1on&+b9(z^}LHY$%M znpR84-JWzB-~)+52(&}xuNDR+b{=~|pg+5YCe4fp88!4&oEHK2&z|Og+ch4Dz3TIU zLPR?QkvQwz$ccG%7N~1n93zvxP?-sfnyZy& z5<*(`S2uAk6;4<@Pf60{swAO`&M_EeIAbpx3<*q~kA+oN*zn-oCiYTSOB~{f zF%lR#;pNST|Hdta&gd_q?GQ{%bUKq0D>2zb-%Pghg%A*&K6d>|F?NZul%Xq`?O=iiye{ehgPAci zGc$Q~5PdN&zA;F_d>5GRhYFn>(ss2Pe8?yW1`A{5$+(6Cn49QbAk9*5!rAIXN`zNw z<#HHe_rO+rWcZO=HbyGF&flnPsypX~%CZkXP?WNZTzKP%pu-g80v)llOLdZ6Fkh)NGzO3t+@^t;Lq>I1b9uvrA+>+KWmKH~+ zIb$N{2&w$OuHwYUiTjId6U5u1ZH>in(LTVKlqbuQBsZ754xkp;gi=&1u7alb;^m@5 zLL`ADR&;5lADsy*cc^GI6r{)9<`qV!1rAS_RJO5v;TMw``f$^mQjGeA-%+ukvzd=B z=aOH3ohw6fu#w#cB1!6i?ZW2a@LRF1#zhB{!~kE+2@aql2F{%jJfK=W@X#6EdBE{Q z$)doL|Nk2z|633Mou({kfOO>vws*48XyoVsTtKXEqwscht2 ziA-oB5N(8-$QQ??(nX=G0FvBkdLk~x(b$yX9TQrlpLqY+ZZbU984-!u#Dd}a^08P` z79ma9dEeVSaeSNiBZ>N(G3f`eth$t)x}@ov_{#50BB_d8Zx?|sN)uFRu=h}jNw1@^ z){ze+b?Mo~@v=*w7cxL^xWS!=3@_yDoEs!X)Jc0KZrm0 zlcZ+csDV7{{{P`nf~?YSFX#X`tO!?e7>u4TS zu?^w&rpFMhOU8GVa-8%GQ4I8$nuH$~V40N~Xi1YJCjf9@1lRpsu=F9j3&1id1 zOf&I&4!mD9v(Ic-r`JB%M5(}GWu#`rq*>u^rB@1$@+Ovylqc)?K6y}7S0Wr}nhE`Lu+0NtvBU#nTuc?kgn`R+@B8eN zp3V)x`YLHBG6Lq<09>WRi@cLJPrM`HqCEeZ%I5u3EbJsJ!(`84H3f+y{Qp*7!x2}z z>Kq&jVasQLVG_5G(%x!98Z5}Rg;=qbAHYdJZ-|{Ba0~TChk=vHWF5^s&8A2klPsUB zqGJi1Ef_JWZS6Fxrh8m+7KA&vV8WQ;1C>z5Q}?$v0!Di$d&8Y;sD0Rw6>@>F#887E zg8l@Z1fM>~)+@htSs@g5Wg>Utl8XcJ- zT~l1Ql!d*?_$t23*v2`9^Fs=L5(6l&pnbA4(C9>m4KQ*sWGWZhIs<92E5Ev%GlZpj zhBU@^%ueJgtQD*j zUIOT1UxC3@;PY?U;cvVn8dtCN2oHrMCKW zTFNZq5g^LH5l4uXNj;Iaz$|1i96Otkz<+r8h^EZL2L^t2zF$5M%`+s{|qSe2L+0SsyRoqMp(9ox`! z4DGxt9xl3IC)PI-8aVii<$q4eMhR+@1V9OawzO0;%f05_i%)(qS3@NBPK&g+xNO+A ze#(GU9#;H65j>9)m{8=V^ShatP@a~dmYkvm@cUkbgof2~ohfV~sBbOx510_o2}qJ6 zVDx@AVyOn~-_j{Rr$`%&wG`P-q;3_5sgrCr$7-Sbzio|mCjZ$BkII4#%VlvYv7t;) zD&3X+|6kHY>OHI-WD~(bzGZ7wXb{`&WSF2yZOX@{ABSw>0a-n01nP+nCWe#o1nGYk z-3J)UULL; z9c~5?_^-ruupdeP*WOjXMG<#e@HklK<+?VCl~sA~b7Z|C z0x~m}!xw-ojw{=E@kf|R<2zfFE?Wog!QE!&>&7o%GA(BSZ>{MUDqd^@t$%}%3ubcp zJcSN-sxvi>&ku`n67#L+)0IzuwDN$KqmkX=I!#eWK0j=$u$vWW@r+bX9QWuWEX!(K z@vWK(3SGr{ryVc-(q91gm~n&sga*F@?C+QmZ3T}ULljG=s+$A})KBb*wi*~1P}}Jc zGyn)7@1lf|U+viE2fGFx08C&{*eg;sIA-{5q(4o4rQWBNV42G)aES%j>&p}v6FB5z z^~74Qjh9*`l!2zuMe6&TK%Rk@TI|f_hWXgPNb(V11fTTZwDUY4gCBgKiXP^18F6Qr zhM_ffr0PXdpTShV?LevKb(La^OASQ}N`Ev)c{m+5Ho>}%HbN$o=)q{!VrvMYAFs21*xOIdPB7i}gPn*{j_&m0SAGst8XLSWK*7eJZUz4b4+YBo z35@a2?034&jk~!!#xtJtb^$Vs6avvnW@C%FboMotI>oKyJ`yu4(qI!mJk$5N^kum*WEdh|)7oUrwNManYeavDp%)SV8diNaQu%Mg0Lu(Z* zJ)rL8(HDS@LuRNb{?A?mOT&SS6g5JKEbY3*QJmtsJU#6{t(r4&lJweN(l#fe{C%V^Hf zqw2C73B_h+u)(+X#dB1qx1;a&xB8y=@;AR*6;+8w+Tq!fQMydjwn_ESz0U9_*Hy(* z!if|e!5gT7uBcLqz2EZe1UZVUxK!nd{lecrwp;n9%;qo79?L)lOAk>@j8CF2d&vS3 z%{CK5P$KQ>1oKhqkXPJ%+!UnmeQ6c7>9b3u=F1!#hg+yB#Y>wWcqEt0&_T@>xAuhy z{;OZ)B$YmISokBPfEV;`t_?BOIM}#QwZ3ZkbibWRCQgG*aUqyJ+epK69#cJjLG#TG zc?|l5;!vh)uOR6~0fmd2Xn-q&B}5QOlC*4~9erZ8<7d%bub&?-w12Q4VxA#w4QwnO zm&FnrnSwCenKZSbTEkKpyHn7o4v0B9C|_P>GK@(4{Oowa_=E$oXzIJ?RxU2BLhF;B zE;JW(P+J!nLluFvbTj`*JToTzH$Vf-){aV;_p(V?v6cKUxe$e`ad3m=V7`9#dI9N) zl0X!GpNAo6OZYKMNwJeK2@xglGvAFD?L{%|MRDgP(F1&u53}_>$y-DE5_%vgqluFe z+Z)=r#}P@XVm5TQF3oTtJvc|j=BRfwY1&g>R-FH2^n=%xJD|j*g-2?!4e8GCAZ13S zK0a9Pz!>$gPcfjbqy zFdf>AqI-y^6AotMNXOrWT!Cv~SM4F@3DPBXQgmPnrY}yt#cTw1#B9Imzaw( zr}>pTuc(2mg=5q;;D=K;@Z@DvX@=8WG~-3Pu@lBvecVEFv!t(l{pia3Na#=I*r~E( zEyl7iw?%a)Kxh^qv6@sYwBsAzi8NyvCiK<`(r!jE^*tl0powCc~RDm)Bk)GZ*(@i62S|3xt=5UDPj z|MJb*BUD?gyIDEYko~V<5Bf_AD&hBMFNDWkSHVs>U;CeJ0D6;ifyy;(xSXFx18hal zPli6-_>IfhAzk}~(VF&cr$`@~y?_#X5f+!>pVDO)!!?J=KX%<3w;)?RPrYnaarp~! zuwL#c`A!4ph0wBx1SV{KS>o=A8-o4q`Rhs!&Vf=FMw2xbdOhnn$oWShaD4;x*#?UZ zG>lG=17j5KMiJ^EQU!fyI0P(f5TVO09xjjam5m7_ujLTLFEn!0=^3ZUyQfgnd%G0Jg$yWH zufV-ix`4A$X~I;x&a@#!Y6|2GV|svtNFQycE;TW~FAy`75Zp50b??P8DtmpLp4m_o{aWBU(--ZbQ^bisX?eCE!&6*uoG?!jl!^F7%!G+Df6tVZFNeLf+uZ=Mr-5) zhz)Wczf{uPxhP<+B1wdPrOm)!!*wI zKC0qFEy@Eqf)vD;GNDQ7-Kz>#?V1;sDDV06Tv9ZL=AP8S(i`C@7+*skdnMf#p~&@S zR!xI)aL!S^Eld!)G+{J zQJ0haz(Pqx2A3kyAG%1VGe{~y)>U@@P&Z7JUBRE=@V>#U6_N4S?PHQ?R<`AWNKGhJb(-FTC70l7G z7kV{{KHx>OuKm(?)N8y>BlS2W$-UEAGuCUJ7f$73uV^s&=kGfQ14&1Ry2FXgI~D@0 z)bk9HiTh4m_xx=30@C`p(pHncb_%pOXl)EWmNHB&5R&<^hP@(Dy5sA9;tAHai07J^ zs{a8r$_8++9qf(-nW3Wx?nUjP&!u7l!w%-LBOsTJH(<(K*2@Zc36K88L^BddN zl#G)-$$b%yqAAQ9gN%#=L(vkO6=iQ~V)1^QkdCJCx>cr~0<<#*r9xZ~KCuLYm$z1$AFA}-KU#Ti;j%wD9p*0C@e zscIVk&8wupC{av^dHPZKYrf@rl<0?Z@ar0D$tnj!w|nOP;hbW07$btM*FOxvQz_vH zD8s#gP@fz!L-HB@mGxb98gVg$PTnigbd|aAbXwh1RnVwrjF%R_fd}!(`^$plYxk~} z%i2u5LN_Yoa zgE~+txV4<}tgdo>aRy!AHQAP@X2?ZG-|EaUYStJq*>Wb%K4w-<>C73yFhmiuViBiD ziej|q?MN}K$9p->N8Q(-kW*CF*Qj?tHTzGE^e`^Gu_t>by23$oTa>Xwa+^DIU(o^R#$fUa+iq-Pxq&`qw>C1i?pfYv6HaT#b-jKR*={LbHU@9e@U zrMAg+*$0Xh zRhr#;&|UG9nsPRCnzvRMGkda~H!VD76%t+W6fx%%Hx`qYT%&oWeVa9S_;6_nU2D^h z(Vbg=YcpAMdT<_rkcLM3HS^wMT@ldZh|5Ey7Bttc4dpKjwO=WHEH-3R+BP zML{6=At}|5d@jdZANJXupX3%Ct9#>lao#cJz-n_@>eGph8*vn3y0&?U!mU@y=!BRv zfxH&ZcD+{%^0_~J&8q6UPEW%^s>=VHUi`t`{rGyzZ_W~bmwFQv6+`EvsEBPGY@xLy z=x;ikF0@t=`lzUAn&dCM4aB&cz_-rX7S0G1oJ}?g-gpYCTP6WhBk?{+zOcWW{QMh)7gZbyU^s&}Vz>zxrrt-ary*EYzk56a4ax3f?Efa6aty=r=2EZKg$c)9R2Z zJRW~$GZKJh`<)*b>FX{AIHAoF;G<3#^VLK6tXbB^ylfmd1X0V=FvpqV|K(>9*Fnuc zD#SG$<^(emB+aYeWe(#{uF1}am7rZ;UuAXwaS_0@c!OJR})4dd7npyG5KA1S%&0;9nU{;+-J;2XihQbG4C<`2(U;>Rx$7OHcz?)>I0V=6aIkMayXu8RP}^VEW&YYp!MNI#~a zc$RLhu+Z+-_#PEh&<^WeNiWb;rduecr^08aICK+xgO}X`YERs{Vbr%uLCG%v1?Eju zWclM?3=vC;CX7^1v{`FLS!rfcu{ZggL)`b4X^nOb&J&|NHcexXW5LqL?K!rN?4HC2 zR6M)wZyf&wI*XB+PIk&mWGjED`)*77(F3lN$K`vO)aoh9qc2MfX{8pUXeRe%1rX-a zD8xjHY5vi&`Wr%_B`C`NKxx;l7LS^RUapP#@B1Qm*&K&n&1)-d)e^6h3Jz6B=WR1% zej3%Bv%&sM9~Fb6FJvFR_p4zTT)$KK2rvzE3!*p7WS`$kSCwQ${dDZE;BLApsW3zQ zs@D#S?z#k?FUD6EEn+_hg@lDQh%tc#shD=0WY7lj9W|*Me{V{a-4h7Q53;CJDW1a= zraQN42hdD}GI&fkALE;5g|3qPKK+2awdud{ZU;7#uV8f7Dkjofv>ypn8DD^d>zKX_ z^D3TA88JQ^SzHE$KWV*^`){#!9B7}KC`WL=yLeX&xJZ5?pyb{%OIItSJcbvsPqV1j zOV?XR^K?G8?S5X7X7Oa9Nq*8^fWgxkhsWSWlM@1yBWFV~1V>h1Mlw;iD|oA`38V8QJ z5SZZF7*Mcx`MTsnf5HHLeL(1&{3bwZQu$71UBR+{T*4x`lvaPNTXbtt7N?t$L);Y4 z8yWK2K-r9P(&pAJ*|Rk5;RPG&-9qrm3l8m%?-F^i9)2)aT11?#>UJArm1&1`pP{Nz z>tkRp6+Le79H>>{VScd$w)mPxkQ@-BGsJrIw9WSOf1>+}t6Lg_E14xOJ$wZ2H!bS! zie|kF{VNJ!hCm-1iX+Mwhz@DCO@fiKptntYaha!68L;iFu@T{`hdDrOPYUw<7ALZ^ zIF;@oev_?bSzNjqYavkK)m)2qq zvZr}`?jlc=>M8kV!t0IHZ95AlhH^Jq=V)EXW*yY3MB(8}QxtT(D~jmA42g$1Ir|uJ zzN^yILJM485y=D%Z4N>AM;51{vvVDCZ{usy(BT(gVi?rGi#ulC0(KBX8>LOw21LQ3 zCf9XP>y!{T_M2QfJ{GQ$h85u5s!PJ&mWl z@$qo7bCuyyXH>y;kg73$W5wns^Bit)$L)j7uZF84YlxW}pBpNT#tk)v4vWtn*Jq^{ zO9wClv(Hm!zReWt5ff?MnbnAU@q(#V2l76!Sno!$NnY)Jiug}fV2y?uZ{+{*b4E|qy zF0FGJBFXCksa8IXw(4)l-mcQOMO%Zw5JKTMGOA7tn$KBiWml4 zY^IttpDZ_a!gCd#ZA+@GjW9bOXqHu<=;VvkJvch`WTx{UpYQ>yaNB%ab!>+&E_zPF zHb=c=bXGBpGO1+)^=gUff3PCGh>wsbsKrRE+L>@=BEco}0 z0mruUnz+1grC*#~6>Cr4kl_ifialrt!(ZXqdF@xX<~pA3Dvh$Gj@-PT)+|n{Iy+ux ze2XOI74-ey&-d+edqvN&>;K4pK~1hxo!+B;6`HHNT|R9v;4>R`J@4A8MJJj!ufe^$ zlA=Z5?6{*{epbMYnwjGje`RumkbpT7HEiIOKHFL1 znzjcD_S2qKURH;BcgYWI$LhjhV_4&k#z(T63D;JI7k(mCTKp)u+pOEIR=qmGXY}yS z#;HmUdHL)xvz30ur*%gnqRWF1&b~t;10py*GV)4tvx=!ftKZ*NV|E`9I%Sr=@H(WC7R>=hx18 zfm2v`3Up<*IoYRD%h6p9kh&)cZ*V%qtt(Ne}+wI5qFZH@Y zHgSH&pwNjKek_I0EN8e&nagQ9u9p2hR@QosIOF% zs{psNCR2=K;CT1EGRd;AJcU*$xuPHO>r?Z#x|Jl;yadfuYl_E@fMtD=hGW;dT5e!P z5W*B3P~t;2`){He^hYNz{WNX-mpxI69QL0ZZDQW)ZG6ny+*Tzieu4a1zb3N2GW2I3 z=5n;+StWVY6dhh&X>gim%z3gh%hACelFCl#$GICtBkbyt(0wlUdH7_#v{ryWze*Q< z8y0xNYy@9_l(^h}C&GJ0`;5>c&3&{B5HW!6M|d=T8)CaQnYFI%(Z~KzNzEL)WK>F-fZs1(s;pMKG(5imY^&vX4SB(Ua0CHy zB9MJV#pco_pN{|R6?$06z31hg-c4<{SBI$NL$kzNoQLL)iZ)67NZ~l3ACo#W%-@P8 z$Sq)I`s5Q`GGk+L=8a7KpSxHkS?!jw(30{9rPv13KjMl1-SjBcUCyO0(@;|QxW#q< zVW=oSlaDG|zrkwU8sCDn?k?g_t)DLIiPYV(ad_(F40;IBe#mfydn-DAaIQiR;iX#3 zM50Ot&hvWO0^8V-(;shc;M~T`8(0(3Om|i5ePnz){kMY6Bv$%opPT1m-*0Oirtj&xSETKJ3b-kr^vcdH;nlg5 z)`<~uaWu5I^%eb2Qgf3B3Vt}?r$J5I>%ss&N^)wlmC`0J{s&6twt4^n diff --git a/docs/img/streaming-dstream-ops.png b/docs/img/streaming-dstream-ops.png index a1c5634aa3c3ac348014f4a898aee1bbb6a5f23e..73084ff1a1f0ccab1ba851ae9682d48face887f1 100644 GIT binary patch literal 33495 zcmeFZWmuH!+6IiGs5laWF-VPs(lV4ZGNcUMtuS=A(O69PVyc;89ojU&OK=ECj9S6o#@Ni#2nR>XKUj6zW>SSz*xyDc}twBfqYSl2vz)F=n(oS(7P`a#svmsiq!Xw*j5tzAw2&8TL+;4Agv7vU#mc zTKUz3VcHBq#|aVB$fJ)rwgrXCwYsfU;e|-v6&IoHelC$V^@N2uY4DqwJzcKU@X*w_ z7=AU#o!(c^4n(pt-Q{X@Sg*6T`bds6=|jHH@Z6eM$2vHvWoO~irifaq3wl!Sn!S(8 z<}V1+VVHe2wJR`B!AkF^@%CNJ0&^1wIThhc<+bMqF@YJQ<&5)m&6gbPr-@+@m)HC=9s&YKBu2Q+Hr6|lWrHr!gfZo8{{%k-i1wV}g$E^R|Z$O4OM>1H;X zhUF75J@UbBeX9;#@H;k^jnxOsDZ*( zQ5hE(9;Gm(6!q9Y>n(xNrt8xCCcEaidwHYrToPuzbHrPp$P*n&d^Xxvs2L()h`6Bl z@EKa0@Fv>80b+aG!3y%gUW4iJgMA^+KBff<@2=_;34PM-JokaVLgc~mLyPn9Br~Se zcenm%A}EW)-90O0y)0tdD;8y~b1z+AyAK!0Zp@y{J{-c|%FQnK3qKPqM>VQS8Cn~M z^$Q3|g!w+|q}FRP^6==h#p1d;2UXG;bn}gpm}xxwb?d7y-OEWT>FC)g;phiTcbDqy zU9@8-ouyAyPSj2mPvlPYPQZw{2wTk)>8ht5G&(`a!wV~h`y1YF+QU%UT0P|#UxUiD z-K0BSvq=?7wV+dnSnf$14p0tl%m`Gzt9!?Sm-I*d@bFRb%fgRZxN2nqFbek}FwE1l_$u4-jhHYCpYIKRyE9aX|(ZP2wO%7RJ1=kW8!8O*tHR_^m z>c%vxZ{Lquiuc`U_h<259z2@S@Fko#`^MnrA4H;b;FFo97gn|pl${PDBeeG zS@VlJI%+iU6-xi9X^_PCoLsyd)nqe>kIjQujfL|=DXux ztm^v|_Yd!<*UmWh##uiTKAL+^zZ8eu0#9D{eMzs??RAkJk_lRKYpPGLrBDrVHJPfq z7Q79fE|2xXJz@1f^K+MQs%^cfeZs|n^IRJhyj46uonCXn>8SE3m3oaMNFi?V9AjWa zCo17GIkT90(*r41-}bAZsYC8lTmbnjg(CjjPxJ9hF-Uj{RhNrol*qVMf1l#?@n?yf zI7ZStY9Uk(X)2UChBXDcYNu^w-{#$EUW=hFbWqXNS*Kfm188C0)+k*Jc-RRKnCw2( z&N22$w&FDrO#{YMiWA7Q*UKOHD%^8RI)pRbL9E2NMR8{)DJ#4Npwf*=tDV<(|qq27-1e@HKx*hZU zwZQq2(LFg--Oh}YN2QqJ{4P(AI0{FkDGDNA=bA#O0D2gYB})K-d6s59iWNw`PBsJa zp239Tc7+=>p5s#?IVym;ZIGBSwub>f)~TigLDz5)w{f61z}L}4h}+b#Fvxr0kq8Qh z8io@FK@*9ffM3AZae$}4A71_YD-%Yh1TulT1_#b0`&;76E z{u+w|Y~jz9fti1g#liX4%KsYs@45eD?CX!^a5&*YoYVxJ92qdzzxVs+FCyM=1WCLe zXGsp=@tXb534fmnP4u^Q`@#@tCHfdbTpX~H#KqyLKBmrfzlsP=M2)@Lp!ia(zSr&} z7OiF`E%~g00~s&64WjLR;uD&#cG( z&r+6(F$1qGx#V{rpTYuSkPVkd+SpNjB;D_+rB9Xaw`LElI~NjaLDSf{&XN8nRVZ;=Ze{`ob~#? zXHig{xj{)PjJ+sr?1FaQSeY%*<;VU?4?Q}bakokNj1Eg%Oa=BUsFCSzi@7*mq(a<= z(&Du{Pa9g~m#cMu^on0e4PWGxyk%!oE4Vc?yEmtgk@!(zq@Q#=(w|DTm?v<4N9vG zE*aHxkIn~G@AvR~I|y$SHN7(jGLxw>V*r6GzG#p->yLw^@%$)r~%T=;)fu+3TME_lSiG-8ynwSdzXqw5_VL|pI zr(?|=^m_N`0(3Fe{!1qT;gE4OQSgJE_GhV4M#%2FY;`YA;J|iR8kQcOVrqCJe}r>X>E?KN*+Y73y8-!SCJ7WmIiATx?&aL4pB(on1>&;k|C%W( z9>k!|8R9K8_?jtSjQ$lW42c#-6McILE>Vy0Z2Xy5no#O-SVZR+(}*xArky-?(u%cQ zXKw7y-9imMH?I9E+XZE3BV~CHYOjJG|A9+*lFaXN^h_=(P9rnCzSe1dnGg)6Jvsjx ztG5>_jCt26o5+nRZlhcerQqjYwe!MWx?s=N3y=97fBm#+ygd5;@`lTlop0lt?;pnw zB}2$pT{b6%XRT{DFOpqm#)Xfk94-qRF?>rVu5&(*KJJ?h=cw+)mnCD=#;mp*v0jZb zv8Zjpzb1g(h@x%6y#Ay4LRA^&)P&1QeA{YLM!s?VA_W`EmR%1;}=~*c1MZB3O~W6D%h*GY=hB(<$8XtftXvyGlg~)Bi-M zoz#N406g{%%;AuW-aFp^OInU^z$X)=-( zaz+%z`2wu;=yWAoU2pgNV3p*8e!F(N0pq%oydobDrE}H|<{d87Z+tVwJ3Z-3*T*r+25;xzF5E%jImnF zCUD*{{xE=Lt3MdPEVf5_;}^zMH}X!*#<3fQLGDSRNcpfjw%>1l_e^(}SU1(aoCdV* z=kbJiT1&`jaXWqDN$pmR#fv9F7&&@*Xt(f@gBf#f2nGL$hU8vJ#Fbxpt%?$eGnj4T zU$By7Hi#{0VDoe(RlZJesL5kc zmgnBUkU9O9+vs-gK<$-ryx~<8i$dV62bVNK79I8FWJJol0=fn_Qh1Ov;G_?FR-gw zse2v3WiV|2n#Ek6-(veZWZtq>tMu1}=)@b$>LwjJ+r~7;Z?NbD#1W4lSHJBH`{hOk zKQtz>Ix!E2_{2cmK8tPEB+#v0%XF;NoGW_U^wG$i5j=XJWY(R?BY)f1W!95NqXMvi z#f$i%vH=EPL|3fVb8OJG5jx{=NBgnp`))yx!vBZkyR0@)XwHtOyrEf^ko&J zM@o!juz-=~hC2gcZj>>M21Zr^GDDxXr4-_4*m<+%niUTm8nan9E=2&2=1@I;P+Pz9 zI=yw>ZY%YsA5Dl~Jyl>dawxi0hJE>!?jqH(i#Us%j-Kb?uvqnr1WA?b(lqu0aU5vI z+m0~!e&|*skdz2lNzMRzqSdDUNP-R8SD~eD8jq0vvLAY>6Ts0cz~OxUx?OnbDNG{h zadDdK7a*c${XnXJT^=nmLbvdu!wMb3Aa@OqW_xlIJdIyPSfw21;Hlh1NWDTxm0YVG z%FtLPbEiJrQA7?`WrkN&mu&7cEEHt=-Y3WjqFWzKwlR{ksbA;=@f~%Gl^9ygxaX(e zvZ+~rJ~XRi_DI)JF~4lmrcqEzc~%7>WrPkxB_TO`J`UWvM@E`hFhc%XAJ8z4GYZDx zt5Dh4p))0zmbvzN(yoodkV&|uLpZ)8OmsO_rIG1Opn~}^Peq95LO!c0I;nzcuRI$e}+yd_nDa*4Qr(!Qw zILfyi=haD8;zp!o6mZ2*8kVE6gz&rsaau!O63K+eYpUIr3;5yE9xXu7h=Htif~^#b zC#`Gg&dk@pklo`dSE)&8(FUsB6fr9KV5-l{!Q*g}SZTmZZBVT}Om{qFlCMMtpNF-3 zSy3HSbxe5^3O&v+dm{{-zcUO7&QUf|1s+j3MY;P9cVmA?oUf|!K-)=sIH8;(bG{QLxynD;i)ra$2 zDcwmLj1f}09Ynz;`3KKZ-aReuIvTywCT|A*1<^}*dL|Z#*>@;2oIf*`524FejfA3k zwX2m#`Vgi=4*3fqBPljUtP@s1g6s;-fGkpJzl5vdV0PQ6NMs|XGh6<3I%N(oHcQIw ztFsk?+Uc;QX;2=gH9#Y7c7|15q_f-C+13RiTqWl!5TrG1HZ`l6B5`A9z6dA!Qm-W@ z7$Y4v${WN!un>ZPHitj$P1x%}4dF3qqYhkRwo?tr- zR)i~$+jhBVU;!+!4QeO5QUu~79$Jay9k7vj^Rr4(m1RreyC^r0n$gSSMyxJM>#b2o7#FhRNZ+Q zMl#IBpRm&|vr8T+_et~+0jo6;kmv|x$17A^ykElO@m+g*QQ9Dz?)rfykGhs`6lyUb z;uU_g?S!!oY3*4t;X=1JLz;8!EcHoN`gcZM#%ZP`R#W#L(7@JUsBdgv&rDd$hm^ic zT`F>lOb3BE?jtSECcb^@gPCrYj}jc^6W6VmK%Z@l-6I1}TM6ZxO}ZU#C7U|y_C)g{ zq+HQq_MO7_Ki&5y`=sRAqf0NJLJkjI7jFgf1K>cf|LZ_YRI>8xKDa8i$_?64R8w0a z*z;NJNBf9QKvlsTu(Y%SBdM>8srK=*t?=0?4ctr=Zq^yNg4A_TDtt2=0-Q|oQQUc5 zf_Ji)&3q)fgqK*+z!ja#JJ{t&LL%$2!i7xqUMi;q#F=(>Dy} zGVdDnYI&f3V?EnG-75?C76H8L(5vs&Eu(|@y=VxdNTq52Q;kB^Tr~_sR?>YB6p;Jz zX;e&@=MIy-xO;VPcfm>_r;?ZGdfT6Z^&i$p3sYyam7gJ3zs3^89Ip;O%e{OGl$OPL12rz&As_}AI6_t^JLAHkjl$)GMY{r? zReRno`^~lQJAK7%Y|keN`VsDa zZc-)-JBDyb{Ac>$1Ok@9B{B{}pdcm#xT|(I~p$=8q8FHiYJWvca zj*IuAKfadSzVaF+G+1Uf__6X984>g3x@rF@?WEr1>Puj^k&yA_$k(01Kn?D&Gh(+j zRn4p0rqnHr4*L-%DQA7+&*LK`mZeTJMcw#Aej=GBIGIazVt)2ah;J7fpkS zBYa3tz+JJpfJfAApvsbV zk-|q$gW~Hl!rI>LtgZ`5phU9KVZNzV`^icjRI4E$Kp&LCAuge76`zOpBx?+(c=m=; z!yyXz(8ZZYa+va~hgB!e*jS?Q9zy)8Y zqk%~{UDp07%l#jB1_4~-K6LS`-k=Q2)iEWVQUifeVUS}X$Q;MgxqgAi)3E=Mfb(f>3_e~sQ!+%TAV6S)G zC=O{1`7OM@e%)KK+8D5q$BDIsbzQ8uMHp4^OWqd0Fo_|`->UO(2=NLc%&E~gAQgA$a zDn3_e=umR83rc&K&C>w>THx(Gf<(4-?D#&t9LH{yKO6kyEi-gC-v60wl7&Gw=xBKC zhVpI5f8v(_iinUhQ!4;W@ixt?hAz>MaGrH4ahBqg=4cuQsJ{(7>8(LK=ecfMa~pVT zpNYguX(Kn^pAeqZGU@}$j(okBCXTr(W7#x{-ajJ%_A813X6%pSFbNTMKeDtgdWr>v zM1}&;CTiI0JA*lfxHMd?2oBW(;%BihRIOLhVGw%EW-|pnCFia%N$Jz;3tx+tB+1QJ zcl}EXv|^JC)9U?`Wyv361iLj?BuSn$Z&+}dP+VmwaxVe=#M?a&07;BgHup3{DKdU2j z@YG9q?b`^TZ0A6w|FQ98KJfesFGLP0N}d8jC5C3ejih*&e6s$jie^tBUc~R3T43X!FjMQCZ(;BZNn5 z&p>IXP0N#?DlKl&t|!ms(R?YGI0qtBIE0QFMy7=+j!x&_s%o<7PZOUPs{54|i0`J0@6Tx2$y^GzO}Waf z8n!1LKh)!kCkv7qm7iu9Ie2 z$rr}m2{$_dRIokmHZkJ%spQHSAdH_L+7*_o+W^4!in8Q1%cM7jPTSdGb+!>144ncz zcQ9!8V*z_C;`u9jc2(=s4x-|C|7yL%F)HnrJPvL+R#1GMIB*~fwa)gU7zF61(vGN~ ztKnyNseGb&h0YuK%B*TvT5(&b7r+fRk-y7soBBhw`BxUpQoW}70Qg5Ra-~#j7j=ir z3M*MWy!CE}qNpf;keTVJ$w;0i2Eg(jdSuG-Ob^)r*qa=1J5CI=G?F2p8N@75JYayU zjDE-Oa?mgNY_I=`P2C3&!z*ya7S9nN)MEf4oGd}}|0$3oU=P$tEZ~0qjoh3k0={Ya z0T!c-;>@_%2bAR$D7*57&4<2`br#m`b%LUqo;L-=+v&;<>pu5YgJOia)1bZ zTX{(h1tMi)X}#3C2s(C2E^VYpULt!@9vtcHTGd4%<LE0rwXD%%|RD zex5L*wVX7ss8JmU*$(^$S_#0=KYDKdg@eZKPT~kU9%ypPNln2HFOO#l2V%9%8IV9K zC4Ic##nF2s)wTX`Um)hIyTr|yRiB7*?estB;azt&(>re9KEaEthoCZQfpp&v^jYW7?o$AffP$p#4ezq zx1w2s_rDyLvpKo68&81`IYuYNWGbMv!%nNywJ!M%3@R9DRI4>wd>N=s)KD2`!NI83 zX**=E3@>)kohn8Nqi7*MDGCBBX+bjv_Cj0p2Ua*AEGHGplx@%h72Qw$cXK|k#>)T{ z{gqG+z+X7z6fflIceEI!cCw{r8*h{YlcZ{IrtICO{~!KnN-_!X6X&X zIh^%{T~(D2yohOFoS!YGHn{Se4r!@V>|g9w+J348V`W%H8YXT0m(?7Q|=y?5xB zdCfQsG6JF@g`e8W)^i?o!wZ*1N}eRViF&4?5)^D&)# za(r72=UZ>Ez7)5Kh*RHZ#@CZH68Xb{f2W< z6lsh~W3k=bOyg1%vZS6+h5>8ZXSXS1kN5koBKAT*`3SbDELB)`Sa(FK>)46UJFMP} zxAIII1Auz(?_nI`Y@;fAs=2RgDWJ5JG-A%&b{{d~9GsS5d!JI>W2uD9X$s=@jEGyI zi7z3RHzOEUTS3K8#M_fX+8(hz=Wwg#e77_G8tV@T6M6hkH96*5?vsa=NGHL`R)@tmh}zxR{L> z)NW}l2spA?7q>ioH35WPtp(X>h%fnl=2%l^6iC`h*HLJTcT^Qq{4KlW5kW>$0Z}lB zp#UU(_+%Xw{mTw;vuNS!Dc{3*q>A*A$c*z8WAKv**{?)kC33LRom0u_P5{&-4r)sG z{N@qc>tv_xZu(yJD;vOAyb2^p%v3|G89diF3eXa854W}j7%@k%eNO2#n0+-c@p06T zd9MffAH3^^bd{mekN}hton-76BriH*|hWUf4<(Awnn z;Lo~h&ddB~lEZ5~KJISFL zjtD7Ow{XHWlk#D;yIN*rhL`L5*u7M@wd{|&kCJ_@ivL2qRjMMy_`K+$iXW9zA^YdaqeIfFNdAp%qlaI_Xy=|VZ}j0yk=?H-ni$=GTV>!f9z z53@Rk#%Xm2#Hb9e5!r<-gfT1hxaEl|y|SRw1pIl{uWA zLX8PfdbSi9{Fh#Kvj`sqqo5WI|wP>#Z;Z`CW93!)qunM zFq=y*lEi#J^9^5k+Xj&J#DW-FF47k)E(oRr-TIBQa@2SpI$5-tTQVz zuO7s%bJf-_<^u7-ozmn5bX1!%I!rZU!^E$&$6k8G^#vSD307)^E&_p7UhBhv>Ce}L z)Y>t{hHZGA3EWm>x5G`xZrK9a>OjNvfE(EjCU8ajQigQop4;af`MiOlW@yM2%_^m? zykzNv2hFfI@+(@HX!bldfAy8~I5;{PU){5OVK1~Mo$ zxW%VR`5INDWkGQdlNtaJj5pV_wdF{Et4Ve5c!3o`TeQf_zFac$p$AWE7a_2OXS^ec zdJnx{xDI=NZY{PPV_e1~>2+7dZEo#$i+qh@4wpc6$=gk%Ag~TeR8_Z2s>fct*q8NB zByv(@Y;`-L9liE|1AJyxz%W#)>Z6>$e+6+Kij@0&tYYx6a*$`U3|Ql4xWYy86(1+_ zIhh;8Zx+@AmCLhk{*@#Lxs9^aLps>`GfQIp5Kd){M zOpO`sh(z+U=tZ=L-DKz)>I#+joMg00OdDy|nqH^EXG7W~N%WQGqAweF`nto!CYLHkAi zrLr3f!y8A?tPmqKE;?+Z_@jtT@tO?k*Q-q*)G)Gw-`qjXiyU_QGPq4CptBjZtUf)$r6#>{@J)?J zqd-E;10-ZDYDNWNr&$WeZ;SWkp<3nUsunRIgcMMAqhgQ)^}jmGfC6njZ9LpEm%(L& zB(RmG|FN)^B-@(Om-zc!YOi(jid)IP3$X$O@=qHrln7lrkva$zkfFPlksQ|I3MZ&M zO(Q^{n?ew#Nljc@{kaKiYUgnPZ$w5SNn=t^3n)foIwlT5s9f zyh;ElFdP7aQ$-(13?sEYbGXn`K+ddf@np#xXf+{K{VjwpVa8t;MS}T--!JBkM*-w7 z^D2yL=XL0O*^`F=?rn+6vtYkhSlHQCU6r<+P`B4HS`=V(0g*%@t!q~4u|T-WkG4^v zoIpg%eq5-qnQfpRT`FoKxH~{7c+f{S?Lw<{1={~QiX;pIUyo*ljW}!BU04}(OP%TU zU=Cojv`wB9RwAD#+p%c~>GSzBdReG7tP+_x>cMO!sjtIXo zz;}?~zDf2vTYWXC(Y>n&2d7<6E#GBSFqa?;jw*bm=p^JO5=tq^w`rpIs*j*5ry&m@ zVejCnlzXNQA9E@nys0?e{BVo%7ZuQN@*Q_Ft)WQo(82;QFRd%=%coJC7%0Pyi~!IVEa>ymj~fFm5z5)HY>fE|0J$;Y+{i;!axupF0Ns)1`0WYx0X zf9c$4e(Bi3kaZ*V?RtJCL5!K(EIe}<=;niF&?l%JahP1zM&$AIYp!4oKKTb~Y1VDz zArZ8muFp0*$c_9ybxhgTo9PYz%(qKgB3~CZaeZWRdMI6?`#pCJWjOl8AH=`{hT6?H zi+ybqy1m?+N|kYs{N)RckxH9cvdOXjiiI=81pd$W*Kp0h-lGx{kXoOqceh**r3n=n zumLLm;D#8t&aWR+&qd@li=dbJK-Hf(7D~Bwp9bjIkQo`3@PYAne|}mM3g9F(>^V?RcFq)DXb^abryx&8C&xTX7Yxsp+=PYe|jsAnYD6`b)?L zuFi0{8D_m68aTNkzyiQh zk6XZjqY+QJT0dul+G7UJdqu{4%wn-IBl+4U0uB)9jSg{?Xn7!Kl7orqbF| zFBY);Fo;+3MQhF-cO5-yEU6IIlQ=%Q>TDE5MJdPEFdqlQ)NhWI0w>RkdYa-K`jnMb?x>w)l zE1b}lSjve-Y2ZnWBtA&lXX`0Buh03Qe5tU<48TJyfJeCpw2^|9^r1HxpZsnl?NCO2 zQ*EOyG-oCZPA}Gacpfv=_a;|4VnaKhW#)yQW5VUHxI)y^R%>^Sm_PZfq!>!Y_EI_g z__Pta{%og5C>Gfx(CGTl42q9`wyFM~eXT$T=r$VVj0&crc{x`oFc>{tuerHqKw&uKzI>2Pf)1sM^Gl2m4)J zj@Byg->EyOnz zQx&(MpQzVZ$lik>er{dDMr^R#X_%{i8=MYRRM2JxkMLr3o3v-mNxfWlNa?wQ?4G}S z8&$1+e0#IQ>5P3K5?g6Jqp0ISGD;Wv<5yRSJ0@uD?8U6EwcTaW#OX?cdlhSS>vVdp zkh0flR;Nls?bQ5$w#7=ScFND1#S~&bLI%dC;J-MpFZR_JmnhQ~<|Wf7@6#ZxdD5?XJk#$gV8x$)rjXo2oZ z6`}I}ai`9M_Xp$-UF_>(chl+d?2T{+27$iGru5S`0i>I)5e}5GR+q9{m|ouMdo=Jt z272p50O%U}8sfId7C>t4G1db72o4VZjtCHN76EOdb^+AWHF_HOiGy?V-RGw;9D9J> z3gbr;y|3%A2GkVi{dEC_a0m`Bpo(aKnGn%4;sLA{N8}piH{d05z&;iMW`w%1(JxNHYBpuOw=TQzl(ArC_>>mi5_SFbDfG${xKn@ z8tFJ$lR{IYqwHphd~Hes5#KRt`#Co{E^HJ1r-;KC{(f;&Ak<{DvDG}Ov}-vO6CxFpEm=L>v{Gs1YOMOnIH_&YfjT(Xv1s+9>J z@cu!&mlmAGa>HX1o4fCo5${|x~dO9b(T%z^>CA6 z7bqmfs@jOj4u(R`?LMiqygib&-}3cY9jwn8H$1?S@nR4bp3vzK)5=^buxs7nsixDOkC z6dRu*zGvY&tZOV6s;QnYD0F9#-)H?7R^LA_w$qh$Nj32P<>avRpe0XJ6Q@;Y9+%OT zjsKUkSZ0D{`aZZI=r6IZ@<=GnEB1d>m!tOZ8Qu77m~HKs@TE#rCsok+vPz+wmj2tD zZH+Am#u`9Umfb4fRlv2+to$s7^Bw97Pgpd{z0#) z!1kIN#YonYx?&nOXZsj73jCWc#xcmcoC59z*xZ}g!WGNPV0FPjxMSWu7xP)%A4X(# zhsU<#(7BS}ABLB2(~PP%KG`NyJw5L_6cD}}OA)>3wmP*L^fK4sIGX40c9;+fD*o{^ z0=<_{@u9Cv(>kkk7$=SKAjcw@-#d_k)=3S4eCFHvtTm~LG9*r;UDMjcW*p#*{No(z z`E|IVCA~U54mVc>xaI8Yo%eH0W`rbw-K8gyqBT; zg%?*b;C}{_Asr)hjh;s!h@Df%E2}iGl!kVGtS5I3FWsrYKSLHj_BddCYxjlKJA^CN z_LN=E>28d*Tusf7u`RA!kA4cUITy4+*e9*5=h+I)JEpCw_B(*@SN=N~cB`Ts_vW<` zReSUAOvdFxn8)Z!8W#WPm8+>u#*@EnlM(v$>)^#^5_*qd&YnpI)0;~s(_G@~P^Su9 zeecUoF1qbKxrv+}*O?!>qc8p*5w8vr{B{Y`kG6M$X%b~xtul{(&LR1+L^gH6PJ#=s zn^!F_>uVHtEzhbR{+#acV@cmlRabe!*KBL3p;o-d@NVHU&NqIEfpb)+iCD~vTgt5K zu#&`T1U;ee3O^w9H5c#}-~aFf@?hvppiOwOMU_rz1!q7;zL9~Wn3B}fgQFZRt2%Pl z^+)4d846lr+LmM4p2O2$T@i_N04V++Lzr4|gqQ@NM7zb&_ zbSm=xKQSYb^2&Q_aELB}!95J;xme|!{!viX4k-Uw_y(EgO#dgFd1Y3`|K&9r;-{;` z(GdQ)$Hwz+|La?kaFoc+e@OOxA|T!HGVbDV|CZ7zXU)PKF7Zz03NuRs4w5G(u66YE z6;j3B_veHvx%!`WHvlb-EKR-jIs}si1_l-+GWL%~3ID-K6@nCv^l*JxK`rm+2}x+` za=weFn+gQG(r+E-G`BaSD{K}njb^E z%5v?7mg={-(4W2IT53O>6WZiM`XEy&GpWEBLEdE#+F({L_Gk7^lqZ1Q)K zgk35IPaA#?Q#L;Uob{j88xGFU2hggSLAY`}a;#aI-s^VkrKLgTFGy3F3*3XM!~d4- z)Efh(xFhA~f@$Q7=%Cd(XDs_}InB$8kP*!$&O1@FG}qH~t9k_IX79r55c32ip>Zca zG>;igQtM@d?v1KX)XvRH@*2=tcm!@_r?GbcNDK)$|R|tbzOKccWuqU zY{Sq0Sk(26CVk${`IkJ?wG8R&0%Vz&`eO9tB*b{@AYVSmsnS*fl=VU3YA07*rM(3+8#CV%SevQ3tgMxcq=CzB~& zN4%95T6xuTWU&l^X}J#Pn-WOd=Usby7OE#pj4(n^N*5dkzCH7t>TdzM-u^pBc8G=y z{=h%!XBesG)mdrWf;1VG)-QU;O>9w*@^;SLjy;sItE;VB+oJX7%cbFn>72W7HD}rQ zik9UGc%UWMnjtdiVNdbYmXI1sCmvA?(Q|tAM0t>_B9Ad?9DWQuc zw&Q5dm0jH`Jrk4oXvj0#@#TOQo;O?^cmKhG>LAcHA=Da7EpD4lPF@a#t6yze`Z#n} zB6qfzi5bFKbx!T!RU6)H4)5YGmZH;3>EpA!whB+F=jL>7Gle$C&`%$Hq)>V1l{?P-?d)CRrGAytfzEmU8 z=MOpM{3xgpvV||8iwU;l*a4l%@j17)mz_X|1prq3E&by&Myu=Xy;5M|aV@WH)&TE` z2W+BH>pC0E?mRlqdI6iXqA~s>n)O3agnb+)?2+)xRUy4{(D=)Um|zfZPww~xU5HV# z;pRFs0Mt_d!%gIZil-%K;EKiA_FTbV+5CxJDeRN^Kc!H(NlTT!ot(B7k#U=}uQGkt zrJH=}Nkokus zAkSvLu;{*5w7P?F+}Ean>dmL6HSSW9tngiPIrZA8(&UOl11O%Z>ic%~qD4^edU(Aab`Zy2)|=mPRHVK4*GJJn zZ30-)4TRsz!R8-DPlbk?lj$jKX}`ls*2M|InU`Ct?WEgw0JgUoN8ft3MB%+0XL$ZBQOs4E0UC^Sy#x)%kgAZUULcf* z-c(O9GVs%(OL4>OE~gN>@B2CagON!ze_>{S7Dad3i2`d{%cP%*L|pyWv0-cw%pCG8 zyKf?wtj_Tso7c{urNX0RmLK~Xbe&dQLLC$IWG2PIJ?c=S*qt#2~jbe#5;Ae-9^tZm(lY8`q%wXM= zY*7Dxu6m&!lP%qs`|PI%bjnBOot@BfQ3_BJXofrc|hnf0wHRD?F+lg91 zw*%95RiD>Cfv;M0eoUQX*^e)u?r+;HYYqwSS-xh1b%tQfe(^mCEwBNmq0iN8`VZBcIPU{QZ%Q(WCiyNI0bgqGA1jC7+M3kS{B{GbH_j>H zT-K|Ck`hR`*q5fgWG!G;xxPHJ{jmCYCDG`KF(V9q*4Qky@TvvvH0<`@H~ zc^uoJf{Me=@+Vkp`IpUgT*nb<`JCCp9$oIb45@LZhc?#@&+udx@kKJ-jegJYa_fNu z7Fl3?9*-tcek>?#d~Q-zqHrhf9Get(v8$5_rL|O@&x$z3xd*%_Od2U%aAiW;!Kd!8 z8RH27T#SKmsGps1=-EI!ks!Pqi+o(VDXXLU?FOyjkLC9M>qd=^P@pQ!EV#z*GrVV|e)WSant0R9XXG46F$)J1? z1&}{H@Wl^ZhE!26f2lEH1Y{^kW1L78d9NA2k0`v(ul-h|#YU(KwE4mv+2u^)F{>YR zb%$SS8R$ddgYTZUBmD5zxUmh7BDtfO@E^n=TU!@XiL-&HI$UPLZI+{nOu&Ce6@4aB zFSz{urDmy4b93kIbaV zf$7`5j*Ykr2Wu!eZ3PZ*IP(F=e;&BK)cW!9YOxCxY%m`ZcfldOBv|kq2SxIz`ppmhKwP=YhK!S*@f} z!%=&&=n{WBLgi9Ro`R<;wjU0*szy}VPZwL_4wn;t{94ly|0YJ;&vjQ5-tai`EhavP zJr7vZ-;~E#JKR zu{;Lz$8ncw|3(M%79J(&2vTDm1)02mm~dh}%xBDb=GI<)HQa0R1{z_q|9k3sxofmR#R_Do=0Z^d zW!T&K8R|<`1WfocbJRlHOQ3@q8_1L7kq z0F>SKc9znP*45p=fg*h-CtLT9(Yt1^0>)iNqKcau4_bn?Iqftl2HWV9AAy5|iL5xU zf?6)xwhc!3P^6%)AsVtDP#Q7JaE=ZBC39sc3)rUDTJfGz2HmStz$)W(o(si8>hYrP zJ~PKu(aT}f-fF)CS~$1@q~1H5O>qu44%I|WB<5Ya9l~s`9r56z3%=568-+E?Sv&{A zFiQ@o+s~f<>daqZ7Mp#G0sy6FhS&o!;C5=Yu!=>W?X)%h;)M%+JEGOr2P<5Ex2=T{ zYG}ELlew&} zefZW{fQ zizG|Jd0M3(gPK~rMmV@Bl;udl=S+-xaoGQBGtOXwTRG~q=PE+oZ;SJ6+2c@7GS|d& zTjlH0O?ewR+kkYMSmkmqKRog!-c6UzA^a0+yAQ;@n=zle4g~g0QPKpx+YK}p_OnSC zqJ?-*fu~PO)cl^)RLp+uqIF5$f0d@JoWdHi7>}gUy6#atjWwf_sD2kKl0CQ0f%5p5 zK9wb=kI6RRB^e$rbr#j5d+i5pJUB14hHQRH#zbp9-|y+wu&;Dp{p#jdCAlO#dZJZ* zurH{2L^?{>y8Lc$2qB1nS}0nzkn-{>V!UZSrQn3XE-)eMILS3>!%6tie(xfpin^+G z{pHncOGU@tl=q5+tMB&w>i5nUj=wsei`94bx1J=)*Ltmm>9~#k8j&}N!ogaQZhfxk zH0Sqp=40F4`oStETlQP`#j2-;)AL(2I7zCwjyHD$*BFnJINUmzS;J`UP{T*RL90X}JuSH1ZG zLP!>~>I5-?L|fhbE+IhAgxNHQwvlaL=s>)RQrHSC?Z%xvW}z)yY(*n!J-3^GgR0g4 z>Zo4Ua{p>y{#j6h@BUW@6jG#-H|lr)LI?f-))H0vtu;QDx@A?9;LkS%_S)a??27Sn zqV9LXIi3HfpMiq-&(8#XJg92D)CEAr`QIob=+`oyZ-eCJ$=+hOC|Gv=4l?cU(|~=i z{y2b?UF7fYxQOe$wII&RdSi58ouovhyI`G!^p4=L(7R47Bu8naCrI3_uwL-G5+NR8 zJA^|z4YULpmAjO)O0nq2?=7VEJjwKKBe zNXP^I?$Q83rsHPg3Ea1Pf)1;HfEGCS!)nKkRmdQVxj3J>2m}3p5b$gNH(cmxmOh{n z+=KSbi68UmrnB4JaHJaXQG6_oU2wqnQiDq6P}=5skk%pCY@l+Q zEJ{5)2h@slgJ?CzF(=?>M5l)c|A1ITP^<^18Pub9Cg*_J?>9g_@xo&cL)+~%lU+a4 z0#t-^{a}7ZzMcqqph^70z2S8cB)MSMA8`86_WoRdL~TYIF`!l#>CgP~2MyzXI-;k^ zs%JNcaYT+*)G{FKuDsw{F410D#FNe@DdNHJ^4Z5ewx&hcR1m#~(78A@e#S2z6R*#uLTuLT&-V2ef7FhQ7VrgoXHrPgKTRHqw_wf zn6MP|TuO2}>=ioFyjUwxRP{Z0B@D7V=PF;+vCoDbPc?Kay7v?Jsz?#&ZEwWeZ&>vf z6Xh5XPgxSm$>qqfss2zwL7bPV}0_@i9iGlRmlAf?& zdbMUXjF+kqj--Hu-%@Ak{FxlR@P$^txe$jkx~#LXEF8{afDr|)+><~lrPzsqzQgT%1a(C4rjV7mT!q5OEt zwv$~SH~k>VQ)#?xY0>BEFVKi{x57=zOa%AY=5EBvB$rPYf{4|z%S-Obc+-r-sFwFQ z+x}4`vtqb5u5u6L>P~Q?H)#U5?J)8HNfKQrs`gIJ0-vd^Pkz+#%d=U%Icll<7cU;w zmLi;Y=bt?`jXN~(%r33x)Ni>j`jJf(5aUQSFBZEKu9G;C;gg!ypDha=L7AqbxH=u_ z2)C^``=bXZ*&dLGXEnUwdlyKg7`CD-5$Ln??JbQYhfVyLT8i-*yhk2vp?tmq@HDkR zyei@5a>w6C`b~oSdHewt!llgUQ<2vKVCqx~0rw2VWh#RI0w?baw^oGLPC&5GJ?Oe-^_^yi9za)}3vK6LvSZ&5 z9Xc2;)2g)AeN|OLI1Dgb<2rythX#9+Q)*BHN60wp36K|%Qeg&CIZHGo;T8=FR^9t8 zawnqfa+9pVvMUVuCey0xLLl^dQrCp{!nC zTD0WRlvB#ZpnOh23QMklq(%KKSjRX^a)QJfMHI)GNGkA$zrXJeSW84_Jqzk%=PX+g z5GfD>BGKo&V^(K`-A|8aJkBhvQKnj{2Dya*V8<%j)N;>?lC?x$_+-AR*7^KkbX3Z9 zG0upn#El{OzUm!$UubWX`>4!`BCdTMZ)Yt6Kg?|uIXuI7H$UpA^`KQ8@ zf-n=iLi(ibwz5L>8$JJ@*C#Q{6O0{)-GvzC%m(()tM5}g!khm1a&4R!1(D#s%*mz* z3-S4u+tEBIe_RdVUD#_wxuFTy2SKcpDt`|kcZN+{-nG&C4YA191Bz2av6kaF!_rz> zR+M%)C*eSnUEQc2h;il~^o^Eg$e4{6XF@*YW>Pnf%Addeb+l4I){hCq7~Z=l+!iC(qOe`A$0A*|9r$I3-8-$14p+kt5>Jka?qtJYsQBu$jh zQ~@eay}h#LbT!XkUt;eAi-QVxE;bW9qY^I0o^HPZayq8d*8>72I5F z?;pwLx`v?@Jx{VVksqAZY-+>q2kZ}sT*SZYYoZv!$RwOUDE^jY(Bk2ICfKQRVEXdw zR}ZA;AeVX3FQ4yCn?~)x4(aquU~Z6d?gKZxsW?$}_(1ZDvZ2HuZSu+ z_Z5V=?-%8RC9MBq=jjo8uU9@}3e-alaEI4LAOOL1^BxUv`TN3i(St&NQi-|K+K&(j zX$9i8nb%_~IYp6^2P4n1&6QiiljQ57s*p>GTV;I*L1{Y_k&gZ1WWzFwpCn9H6Fx#| zE9L{VKCY!pdX#C3rEY26UdV<3r*U8En0><1sFu^z*m<-JzyaKS1j3EIci|Vy7A4Up zvRKODt7-7;O}l>7ofA#muR3M%A6+LbJDqHL-=4JyT27K7rKUPU!5$Qyp&UECosfuy?T~V;H%Lqm}@iOQvuY^qzw(D5gv(Mei$0`aVGLS5& z?hMf*wu}P%Gq-TrRWU-S6AHU?cULT`M^B~+PpV&dUYzOZ#l;_(#o@%us6HP^RgP17 zJLy+@8j|huSUf>(E$EMpxtyJ=`;S#hnnhF=!;|c^G;swclttK3#=pP$Q+7 zmuNKlWk+fm#<3hhNX;HU5@i*UlCnY*s3)SC&#nu8t#VAmWutR^(VwusV0|*BeGplr+O*#3__hBH z#9y{XL8pJO|6C94FLCT4j+^?u-9BEb8BTEW*=1Fu}6AT?eR2 zL{>fw6Y#jQhcg|T9HVKsqpETgv%$TRO>}aEta|2;O)htxRTksc>-?LQb6(tKzYol) zYC9Vw&2Y`9oOUA??u!O4SF%Lm+00_o?LWzqc%yEQ%e5r`((J=wm;(nv%tfe#$Y*Sy zr063C;s7T8tO@Zf4FWilSTf7FE2NUy+G*F+(aA=`w+5Vx)78@PYiVPV6B%yE4;nuy zfk$2Mn}YuhUCGVSR8^N}%NM}*-bOJou|Uzs3UiboulN~N%mUezj!=ajL_EQ(6$&>F4Pb*lkra_yq0rZ(x+jx z3(TUo!aP@JlbOXZ{jpbFSS@F!%_x?tQC$nR?3a|1{OvKbf?BWW2jL^(x4-vLrbnw( zlWo*~ij!MCO@>eu1G=c4Qwe)?@~7J;6KxR*asf;&nQV8s-8T_0n{~YXImpzET0guw zW@*M)+K;&QH`OPWM$z{s?oZH70BO95v!_}<{tc}r2dC}ZhfW}(q}{WtTJu9cW&M{j zA<-Fgh15y=ZMl;LA4L*V>URfR!^`6$bnC9a?2(S&IF#?FkuaKd&YcRcelpLN9K5$A zi>Lbpqva7 zbTt3+USK2NmB?Scl}aarV+#gKA$npU5;sT%GsAA(X*MtMc%-Fb`r)AWjU+)0NyKzp z)1w}N9Lw(pvS{814-EfMXwZ7Xoh2^$1TLnzmS3Pktw?oq*Xf}~co1y&6>?&Qk&safs0w{ym8vZMmizUPwllx;rI9XYIZ zJt7!51Z7VCt#pxzcbr!#h0)uM{)}i8-s}1?V;==VT0z#J(@&!Y|K2$%7iaYsxC05i zR@629#h-7M_4EhC{U`*tab!qFD2|QtuW4~BZXO{NSzI(ZOaYKIX3&acN3y6BNv+@P zdpJyo(+-sSbJw6t3#8ua;7#t1V>=w6jE|5Se3q&XU9i{}$zYuzUIp$wZK1L-K!4Py z<2xnq-?1ZAK&Fiw8kzbZbjAf^RSL8VfYtPRQiHJTyWx^fF^wqA~b`@V^nsZq+ zxn1DZ?|Y@47?N=n@782e)IA}|jjf$R2HeO`cy~@|D`r*Vm}i%j*P`v9W7f-A}k^bzln1W*}-{0aNHWY3&dzW9y$lffDRmcd+uC(yNnI zKK%-9i1?e&*FO7KOn*CA8+}CnR*^)HeblJUupT6RqG^5F5I%KAw%rJdH#PvoiPv)2 zFDstwmJ9z}u}g8g2E}Su1zpR|owo%W!_&Z8g9llil+z_qT=e*sdoKhY9@nduKq2W{ zsTfHO9{_Svg>JefpM@xYk;zy`MxU{%HvX7PXv|bTXZdjFk6-+zXJyA7fFn~hD8K5b zq3=*>%x!!2mL!5k>Zg#LFl}!RurL1Z ztDn+k;&CPiMQtml9AD(00gNUNw&59;f-zOCa+|$GAV8+@jByAUxjk9ZY%{X~r8tg> z=}YQnS$Wb7hplNUm_i6VLuoJMR!i=F&a)f`1_7lTNy6?fB1Dx|-((mrCIPF_qcfVF zcqw&n+&DeROXL;H6-5@HiYTDbifyipqi8!&#xhM)PrM2s6~vJdFzijHN~{o;yer33 zP%+{{nnjmj7G_z0Dl}mAJ5FIx+Q`!%^&Y%{k4Y!GZ~jUZHc|ZeqoKY+{HC9hVkX%x zxRniAVVETWhf2#x{ayp7@Sw_r_>ZIZnbp5vLaVT!pYsi0=OMsPJO*W(K`D)pbv9TS zk=x;WMU{THh!_->3{2TJ_qYzZ2lz?E!-se z;*Bz`ToD3z#^#^vhNdCx79qrj;5dJj4CZyr2N1YHaQcctm^LX;l#5JpRV~u6gHuld z0;QjOgDm7Lm{k3^O}$NZZxpF2tRTI&J@42TuA{M35)QFHj^ zw*D2hLb-R>_*Q~HoyXS~Jnp>c(t8?K1Ta{k?h*OXYRF>yomaFWHjT9xz|#rwp?a<- zv*n<}HpxNqhXgjTvIs!S07o@9)L?b`CuVEFf>;K^Y;Af^AAxOd!85j~3L=qIOoRHd z=LKdNZF_3#-j~PeUjpnUAXz;<(kA7AFTM|SI^Y>=LsF|M7Y1!8s4EzSR^TOkI zYRjvS)}vyI&<`Z%)Dx%U@r_-;ExE%@tvg#s%H$9?PyKs7%Q6~H^KROSbo4MZnf zvECbUUN}T^E(sBVpWYXh#W>ea?uGzOU&QwraL*H0MCs<8OJuornn;+V6T8RmOC+w# zZrr&fj6wVAQiX;-eTR%Mc}`JG7)1nE)P2LLEkNL!HX4b=@mrLzw9qr*!2Zv#{uM&v z7K7EnAGsq?y&N370 z#BMjfLC$g!%g6JLHYvc?^NRe-PO2~U{lAaVl zq^BbGw&rd*`hzIDkg&~&@>;l^Y$@-j=WhvcFOnDY_tO3G^$yG8OMs&orofa4R+kMz zBbhH^idnh$*1i`9EX`X`D zl5#!;ETAjRcV*qD{i=i?$t^TN#QU@`QFo^uBY z(n-WhMg*D=fi8TwgaK0j>Bf(;yncao0C<=1jVy$cg{>;Vpk60{V)g!K+i3wDEX#l* zm0QSO(gkmD<|t|+!c?!J?%mDUsG6U4Xr@Bb)*;9t$_UT=iA>@Lf~@ym zM9u33i|i!F-2@&pQZ@~sw+1{AgU*yBeNg{-?^ytGp&qQLt~s|YrX&lwMw^d!Lm7Ua z4r?HBK7lvARd=$7_72_4kyMp89&j;pLp_wL&|#|z=UO2Cpr?GlOHyG0l2>ZXBPv@O}cNM1K1>jp&>)4Qv%&!-je_Fd8>*}`@ zYW^+p|5&t^sPDc}?qo+8EfGPz&je5+y7y;o zsqKu9k4kh|qiAa5OGXm!@{id3;+H%+3h`&BIsm!pLR%D{QO}-)na@kiJNpFRMwvT04(!!z>Rn`S8>H< zdAYwlce0we}+S797Rr3TYAS^ zs*UneJOZqy{=Q1V%`Ioj+FwlmT4wYhPI>vbNxxmnBS4}I*mCkp^$vb~=5w)wiql7+NUH=2s`ojKk11dW12TeS~OJzPm=HRNY!)s@{0M8V% z26(Nw!gJY+f+B4Dg0+hSQqV(afB$v}PSr~ANA{U1SW%SqNMeuuXBGsDM6T4z7GSCf zJ;YiPxrCtg^W9o~@@n*?)jb_ddtk*m>vYlqMB4CG-X|QBXBCON^5sXm%nLO50w{Os z9su(L$4UVcR#>!gTH!kPA`;iqT<;GEazvPSjUBRsyyT|0VS<4IY9eJk8CKaN- zO}D}vg0QN3o35RADplcY_-X>ekWhG*iIb!lJZHXPj`8IUjrT~IYEIF7JT-C*tH1SU ze|=dz`;oP^b)z?9x0)B+vKeG1zZISdOyn)R*poV4Wz@DF;THhLbsNv)akgxJ6HlKn z>KqOkA2GEpYD)n7y+|T(mKxUAr}4{p`F*U6@3L;%^U-Oyz3#W$3evb|uat3v%0|>X zJ;84D=B_m;?Dmliq1Mv(gk#h>Zq5cDe8Qx=+*qZZJ2VNNgH(ybPZe10Z=i05!;lO; z%!M_z*Q4o<Wn<;EFoU4LC}5Vc>SSm0iWuxiwiKHqMYcL zP_>6Y-fFpTvHZc@J)LdJkSRRlvX-@}rCtuU7(s7#)mAo>YFsk`Pa3lzUI#x)Td6|L zv?6i{SJ!`1xumIj(wdglq0JX+TP?y3>D%#NQaHhJ8Z zQcoZ#hz(MTVV=g}(LGK*vBv}+AyN@#?f1{-4~Um`!)q z#j6`Yg*;%;meb=1!g6`?L#cO3kMdsKpq$U#j1SMjdtDO-TG^Y*PHV4m!;vKu`wmSV zC_bXVq8K%hR_0$%?p?)dC03=%c@n$-<6p73i zV^D-5%%%dsC>GG<$eV~8;O4X9kn>zK?Fy~#LXh%=^*GT0HY$VG3`XfsOxd(wa}UDB z3^l^)ND7;%9^8QY;W1Oe243xj7=BZ;C>!=tp*VbVaXMO$h6Xvk9Ti{F)nEfW(7Cb! z9ugFNSa6EJY~;_OrNMU&md+p@9mOH)!Z&g7qG+1q)BZ874|$wrn)?-G+iU5={$4es ziMva=r0kx)Yg5$H6LjXowo>tHdaZOP^0#0E9PQ{c!(}e8`DfyeyTHG=|~*D$|b74rz4JMrY4Jy=E4g^?Y&m z6)X9fA!JbZkDxW>ArZG4RB%5!3`(j_6=xNuXl44?a=)-U>Y$|p(EPO)X>`AKt#qff z@PLUoPhkoKCrqPfg<~g@QF^saeHxzKO@^##6lb)9oPcMyz#uBad9i#Lo zlRF$FKBW#P35WQ7MvC$CI>qN|@0t4}2qyF-N`6&>F<0OuYwmFr;(Fs*^h)gWflqY= znwn0$r&rO}JZ^rhv1T>LBf&R?ORa_qB`6SJ*3XSLU*dx=?A(PTqv+3Iv-u^O;SSVz zgY}XSjRcyb6`_*KO;I_(-4<=IK33FF3|go-BJ>o8S0+(fUrVhjN|SLKWPB}b`N24V z8tTuh#ChC2MS*hw%Rw+Oti-SJ2|QXffz7}2iOo!{`YYyccYTBG@5(wGV6K6sp^U0( zxxyuLOxty%>}Q48gz2+dJ+P(;>d<6fNO9Y2Jnce==u1cizSE8Ix7ShvBUk`>j~HjPeWrgc?zo^uEZ}o2;~Ha9NEe2DvYAnrzyYn?j0gd6 zs9$&RY2yGy{YB~EgRGe5`?A^x_dnaP-7EiC)zGx3J&aXkQ!`cdtMd9yx7 zcdN{$1F*u}O#(xZ%RcJGyI%qftvgCLGa593P{IsHG8dS}YgllN=JDTO!PYe+3^FNF z@I6xuAQ0r)Xmz+R`uCucO3lULMA(upu~EWUYu@;V!Aq*nPykUF*T;T8tv4Kl*YEuU zH=CddvDRM4$EY3pd6_~@M2~D?P zw%1kCpEr)1HjMg+C+(zlE6EdxbxE;5ia1c9>+2l9bvI?vIY{ZMR3~&E6{GSCfd-#TgAuj@D)J0qcrcDW>nI7RBQiCKQ}P#6zWn zxRS#mX?_{hSuEzpUBqMu6(gnJwS&)*pshj&P0H}$2+X}GKUFddtpZhZ#mJM6>s*Fi z4|6Yf0>Dr9jlOkN*YQ4S?nVI#Sq=uuo$@zPRU$z012_H-#Hl8*IC!l8MCD4E(o`p} zu=uovo`P$c`C`(3P!0nY?Ha5zpP!b3=~CBPEdZzXIjR%L0;CxT4{T`;mjRhm30 zPl9xG0+)TXh-)D?4|R)r{}xpzwJMWpR_u@wwSCess~dVuC?|%3doxD!mq;l&?l0>`J*Q3#YF23_ zJr~V4h{?)|-;JwXv`L+U>~<7?$L*U@Rli&YK?GG}j<1t{#HVMMqi1&wj8^kf9ma1( z@e)DsSmbPzzrRd~`|Yk21VZW_7(`c?4+v^J#VBJ2tokY3*$<*$U?Yy+C*L}#umnvaJ%mf;i$LI}U{}+*M?$c{?fr3C7#xP+ zre|u0Z*uvB!9+DRDoMq5XOq)J#(Qjp5GIm-0-c_d;ULd&kMJ4x8wXf`8Lh!gXtVNX zdI}!)EZ5%Upd=Pu;A^m#NHIVk@0e^gg3^*Tdo4{Ssg?s6TPSuAQ~YZ#qXwVu!omfC z(4wr{FO(lml$q>#%bkXl6`-M9uKe=2etOZ(J2OcGO2gNiKxjRYJd3lXmb!X{WztTU50Hq-U}-zTJx;S;ub$xrDeaT= zW*Q&#{{WuMYJ`h&NpGU1o+`0M!m{P8dV$OOo#q#eq#-6>bdAjV-#yrnUElnWAdCD! z=0WY6=p<*pknzPTEyXTTO&bh7KS!^w3|s3_L-i3<$LqBZ;N>n8k` zO2m(y7}-qp_N>2d$a2r+L6p0{1no-&|8l>|q*#B>%`QJlj|2L`s&zS3F9QiMA z_CGHHtl9tNKnE>^zh3q~-2!OH?0+{F{uwg+uWg}prc?p#AX-v8LGHE#HDSHLS1?1j zgyuLA8pw-2Yqp|D#H(8OU%4E}yP92RY7L){UaDV&P+LogeW1rsqv$!6y9A6hM8XuE8b@d`++SbI8Vk ziTJt)U&L1ML!V_nRt6<`A447JcZFcZVUHv;C*y6}d1#JaA4C|PT zFklh|O$k4+75HDX!9&r_xvz9G^CxzwWC8rdYPr^gY|KPWrt63Ms|=Qy?UtSk zjCr>$KulhXt=M(i6Cyf@aLN)M)9sUtp>*nP?Tplz4nuSIrp(FVHlLkD(DFb3{$~$d?txVMV^5)- W9|52C-$y~;kQRR_h7x(}^}hgG7Z)7> literal 48429 zcmeFZWmJ^W*Z-}EqzD2^Nl8mdNDtizC@3A$4bm~dh|-Oev~+j3fOHHD-O@2MLo+k~ z%lm%f#qWOmym;2SU(8ypVYueH_Bm&ty+7aYA?&@H0s$T+-h&4Z2$U3MH6A>`lzH&r zVIU3`@Xm7BkrMC*qm7KrdnFkey7w+lmNxbl4;~0T3;ML`wD^%!Jj_WW?}Y@NjzUOy zZ{;UV{bnl4@;)r3`20iV7x{a0uNgN*1jCA{uow3lO4Pvtf>`$tCFdUzDvHq)V3~sF zFMro5X@5Jj?tnl$k?VnmEu!qQfWm7 zlrb^B@I^wVYB;QRO6Ad1>7E*A-0&zdSJ(5aG58vFIL`24+!0FclA+TlVre4mmHKbT z=hS`_*H1o-OxZQ4kal9ZOceFvOU57LIOy#r`T$SKe0icoZfg$d~itOhwzyv7%UNp(0iH13r>BMbs!}fX$&b zu{(iw_-TtrHpTD@<-UnMW=OfEkJYnv(XF8=U}ytgy#Iids@@lgEv%GBzaYWb7==SDqSL7PaosCIEXxdJ?Y1Kao(ox z{_%x~+WWJ6Nk7)m4`cJkl((ZXsNmEp6?O!J`Zy22{vin+Hi@;nAWXc_&Q9Xssn`%! z>-E@Shn=0HV_ns^PT6+)hm@;`@eu?l!LwRcD4 zGIo4_6}>C9a@Qf}vHZm0NMAMt-7N~&veqvm@jE2C=%ic-K2h`C=F5q|zG!J*-^o?n z#ER1tsan0G-cwm$c59sQ`g-*V$|5UDD@cj15Tq0}euaK9VR0^@`@q@Do{F6WNE^6c z6m?x6JRl?f_j+iR1p~5*GXRm;TI$xI2FQu6q1)1s4+|aD`rym7X-s z@(nY6b})?(3(=dmA18UbB)iGz_`erpNgoR=BW%N?x&%6qD2TUJ`Sjt@q0#a(c;&9W zaAL%o|6H;5hfzCU_$y2dqOT7y=^i}9k$mvKKmKnV{&y1m?_&7hOz{6@Mhwuvq*P}5 zf%OpQ9m|))?_f@zx2ZK?=5Wg_8WkK2YURJ@)&(gQOP+_>8zOF%4pp^w@<$CbQ^iq> z@&p8Y4N}ge4+Dm9%yZt!$n@y9@96LEXJ`xAxL8nm`X6Nz9(FjGnOQCe2fvFMt0Z{4 z>w-y{Tf9t3nYm&$ndVqp-w^ZCTO-D^u%kKoXgMi{5)mT#>pLz#zred8x8bGQW9A;3 z!@q{Lt-LrVx$m(UYb5`+JOhsjGx2Sy8ot~4iiIf>FDWB~IQyj{_#0Qw>6Dfk+K0=n7Q87(RD>%BeIf8l_xj}=$9h9QTf%^7`V-g2KKD0) zZcpqkHBu#+Ny^0#*~>53GD=%CsHk}LO>l}U%wR!w=ZkuFU^CKbgABVUo(s+xNwb3L z%dy@k$ilWruVk71n8Pi!az8}>~xc~kl*T6E%V9G{a|TkmRV z%BUhl0#Ta1JDRg4dIbfy(J@g*tT0H+jFR4Hd3if(QW9OfO;W->UCv|r^UGTrr@l&(;YyF;0g}v==chdvk+)+lUkVS>GCVjjR*@$*i224`c*z7lv|gG2{D~jDM3uil1lJdJ zm^RTU;cJkQn5!j>K`!IMiX-{U7o#J#qH*c(nIIhtrkU_WAPG97WQzbGH!mUB5`jx7QF3_>f!KMXn*- z@O1T?2`_bYI<@qF?h4~6m~VpJnEfsZ{~aRO`^ zzx)lrmS1l5jB#u|Y-rm@-k(D!z{l;$WZEk1F`!GGI+!xM93NE7m4BF{R<`x1#g6k+ zR?7@dfGKikQe7gd1C6?)w2_;%g+Wqz=?ffiea{A2CL;O9$jjT#hE(w?woqmpcMq{& z5?$;~vdjqYr1SXzpGd2V!@;Qw2JY)EV(taTq57{EJ5Si;@+*n0hAqj{2CQ?T-SZdF zy7UABFz-x#KA&<{at9YucOl(PJ2*eTRM9G~gVFDk-k9B?cLH>glM5-e?-xB5TrWxY z{^wWPIZ7(3+|!Nz9g8_1czN#Kai@gH`3PTbZj87Hclm;L$IW)yz=da8!<=b^b*A@r znD53{$zSETr27)N@=@V4x>gxQp2sT_+Qxp*FZQQP%RLt>wk9A8&$)xFaOdCy`;DDS z#TO{oJ_;+@WCIr^e)Rphg6Kh&3<7m`tRQeiM`W#nfyipeQ_>}}24h@UJ`w&9`gF|H zR`G95+$QqiRzwfQcmcoYt#LLv4Z;8`9uq!$oOH+P7|Ks>DE)%!1Bw}>E5anpoHPdM za1*q-jb7|XnN>Etx@R$zqa$LK`<+Q&|GII-$Ic#S9vHbC(W9nZtu(22yM*6U#hR{f zovam&cwHtwQU5}JyjUx~8=Ce(Q;4`(`$nSfR?+gMW$T_FaWF-qfdSaisb#@Qc=g_? zFIK~Wz=n$LOWsspSYBAm-2c2rPXcH{tr3lYQlnIZPHY~iPIhyY&vgM`58`v+;2e_THh0?y3HRdNr|1Q zyf>=pTeF=g_qI)1OpzbIfe|8$q81fMb!eE+$pj(!4E3ea#pN5Hoa)QBP#cXGml&rf zv`^?SJS=DA!MhUm&gn%KNRpyPVVNOe(?Yg*>`*86`iEwpilc}xcdLF5K67S8uV(8J z1joXkUlOHV*K< zN8a>2nb1(-d_KLN=0K=gH6j8pvuH@NC$iSZh~TJE6f>4wOj_W{pMZ!2^MTpaerer7 zXN~U+$!9ay6_OI7=_Jv6I70`IF!FKT+n0AVpu*kkE>e<1EMIyE1Z9zT zS23C3qvj=fn#Fctc`GCvdg#8q`u&Jd_B%FV^o*WexpkJ`Wte>Q%aq}?r}rm4gcHDW zBROn4ADf`GD|>d>ypuVxm26DIWGE89xCE^731I5HK<;aIhVK{)t%ow?#7LloQu39q zf%FjpKi|!b>9gqxCWx3d?5{d>pEV>SAb&%53#5pR25-l>PuhlWZx>g^C^?ORC6uyT z2|?P`O>60IEqL`l*Cs74X&jk#3P~dt&@8DKb39W>#Sn|7CK0S`9%s^55Y$a;Dyr=QrjPs%)G5&tF?76xN zLEjm3MM?S*S;wF@z$M=a#f3Tw=Y7vda~Wd9czq7*XYnx3(>434Jq^T+(~isiuUc2- zdO`@fe+tSbO+tR`G#?lG(q0T3qj(qWYe#ebj*<`MqA`g%=-*R31go3Z_W1ik(QQ5~%L^-So6sEf< zrx)O|^10lMlvQ=(o!%Y;QEBttORUOx>XIxHb7mzd}fr zHBIwG~NSC@A_S0r0!*lJP>UR8S_@i7uYb~*M)GkE6s5P0vxIK?yeCv?m<&`-a7b)ws zb9NSd?Q<-Vs}M7`i%;L|YD}VJc`v$#+7)-+%TL~RK2`p!!;ah;K3*L?mSDIggAj9{ z`yTDu?@L@U-a&`WjvZYL7WiHvLQIj2)U;!95Y%}Vg*WH+7)zG67JD?FBm+`n%@i_vSnW!jg>V!DH}Y#6~=eNX+noz<`6O;6XBO6$

    08>ZrOw+W0EuESarXHwl*Gv(V_)JCS4Hudj74dZq+JxfV1o60hcZ6( zY>(kLgTk*HEQ(gK9^Bbb)=_T-#qV>Ij0x~;eov*VWG-eo+pm)4lrp{_5WVk{ZX&t; zJ05~|e|=wfbzTc18@uj)A)84{;kPcJ)_T&7FXJZfG{)PBJapRdGPeId_hvn<9@uBO z&wI&?19v02zcjA;gB{!7OFH56`mo1=tA4O3w^az@H6N@6|0Z*xvPNuuixj6%_r5>v zclCAzR&$e4N2uLVLgaA$wQ0xw*>Jzj?MElB&PUk>wG)fsk>dG3ofrLTi5!)8fai^L zGT7`QE(8IIk7BiG|Kn&RInS%gwo9Wfa+pA1k8|Z8at$c5f9F1&z2YaF!Px)C;)$NI z$I!st>m&T5RiDzIo)0kk_4XvlWzL=&M`(5TsOA&8iYxd=N4*s0MN#_!L$Co4f+%bS z#c3x;x1p|6x6c|!bvd{i<@HHDxNh7dg%1Vh1fZ>C&dekJ&{WBJOy$##RX=LeuvV>r^$1V$;GPr;xbdMOYV6qW%b+Ir#S2U=3ooyg%ahKaE%l61Hy} zWKR2Yq5GxlF`rW>(jzoF8FIftyHS@X#mX%sS`Kt#R8?zD+NMv6cTgm^t+ZN)CW%Q> z-?4gkT2~L4n7))HqQ3JmdL#5+JS7xN6d##~v)(+|Zbu`JAu&+)=FOc?%;_=I?#LTp z=UVp}c{^+1eSyQiCdTc7#JwobLt{px46FN&X;#1)>AX5nxpU6`=kY)u2t|h|e!? zTT$SeX>k7KQ(RL|PxZ0e1cch&fd(dqgS6xdZ5++|`OsNl=~-&%hFg513jVq0HD9C6 zgnsFD1QiVw32bP6gJ(0LR_n=Da$-AQIgop!3+Bi$Z0oIAK4r(Yf4|s&>MBIcZLmi^B5Tn99O4^LIHz_^!yoo(T(Wyo;=H&->zCM0?uo zR@C!zyemheQ_OCG(7CxoSFgehk0A=iMZAhm;@xDYAjYN62?*ZA-kcDHG!{JH1hv@7 zdNz8p>h9QH(rj0tJ1NqjN8Qp$1H2ZN=hI4V_cd1E`f~!E66NXS`k}77u$_`q+^uW7 zKe8>|tp!(`9S3vahol$D%OtKvz6kyM;3209yf9jiA5K;GkMW{DM2(4^kL*T^9);Q~ zR!hlBWS-7G!b_aca$cJZ?Fi#wPIco_dwpe6p(8wa%~)$LwpJlSm;T6PFKJdEjq9ht zHFv6=;&W-AMkcFdX}A-aogW}6gfgL6-;)TC-Lpv5-&@L?p$hnN-vc|u_p<_WYE0*x zM%gIPz?g9jUb(1)E;<1uEU zOI!L13Fw>K2Ni4T5S@anzUPJ_EU26qhIe4Q)Zq`b07lrN?w^>q7jSlb-QGB0*`e?I zet&)+v~gLi!FQQ#A;#5}$#kbOd6w`hI@u>jD?*|&f`?JQt26r5c%^aY^`t+Fu~H&Y zzv4RHCbgEK=B}boL;*Eb$?O{uGPw)q+IpesEFRSv)p)uXYf2nkH&^>OKoU$`)Qd8U zDKXSoLzs$Wr)^)V9VPQnkbs3QnoY8yDqz2w@-g3O}Nk~?KXXnd^m zvWX{v?vZwN_$G7g%H@!)19;HM%)nu8eHt6+UA_WaoRm3L`^O~%ZV~A0pQb_^A*5U5 znFiI2qMmbB8>NjT_9xe(OFsJ6O8h#GgRAFq*sKnqy%(^*p!IwZXeRC`nZRyBwt(5x z{-qS%csAxY{kGK?b+RvXJ_qaq6?#mPp~t-SQj@b>r^D)x8hi(meNoGip9AM1dA;Xp zyX-F1k%fEts!f6^0goxUatcO0@n&f4ovfnnlb zTW!`aKbZYy3!^GIjF~?g@x056M0H#LsDN{<|Dv#WcfnVNLZ;$Vb_~CE)st84=KR7M zS2Jejr1P3^f>8%e_C0c$AyJJi*0gpp4w~2JdHljEafXr6iJ1)bzFexQR+a8~f7-a? zMIagiQ$x-3IO>5XJ^~{c7}WB=Nm{&*2>cuve1JkF^c;bfr+U_4$HPSf_adGZ@lxgq z2yKha%3F>n@oZ?Ko3U{knn(Gl%|8)9x1EV**!zC$4i5HmjHA2cm{;7d#)hkaum?k6 zm3jEeJMnre$v&o^FfpjrejEQ3{#UMr{HmYPW2f2AfnxPtc%6?5GIUBA0Xui@$;@xZ z#KKG;+dAasqJmvTv!&QgTFCJA;h!0OCu-=|)s-lrjYvMAltAC_NSvB~rgCa?K7cvJ z5K*~pr#W@(F1dcssBX8DxiL(UEWe+~WlsPwMME|efgAm^9 z1B|8p-vmi}vo*{25K`Z3Y<>C$=t%vAGe!T%6ZkENP`Hh5q{=GX`-9au+ zjD(OzOV!jWUcH{Z&}nHiGG8DT7=P(s+LcDm20Fn{Oh6pe$Agszw?8BCZ`z;m`4&z> zMUUI&BG#KWq9eFVOsK$-WgWNsr%yKN$DCf#`0QqT!OGWqLi#6Gj{WZr$$;7zfRs;39n~2 zBuBES=AF;hVns=%zPZpqJ~d2u>qT0wGxK@fYJ=K>R9Ma!C?BDC{xlVA#c8MF8IsFc zRVR5x=gR%cAe$~i*mhnkuh;$WR~nF(f%p2yd0xUK|Va`F>C~=5H@|YD&-KV%M#ZXoWXE&NLs6H++6)8zGSo zKc2GF$R8l_{=_?<13yB}jq-8!E+meT%`7vk4{Qmv(gZoOW;8at9KN>Y3Tgl-=&{+; zUhyreeOS`)yM(ad=sC{H0mtQ5Zc*zEs{Tzmt<#}T47XX?6Yu&1UvmkgTPA|p@7c;c z25RPXgfFc!6|Ajr-H77&{44|~FgeYtPOwxmAQ&l)UrmodYsul|J{JbFRv)Cn-kn8L=G5A@R!D0ur$?!pZx%;0-fU+nB2qvWE6yR@fn+^HVZ~m5x6xFQ(>I*VOi{>% zqclL0+x^!zHha41f5gDE9*mb?r7GJam0752^m5- zZa5F!(~d+0RgA2Z(_1-P{7G$fnC=GnU65>lVw<8_Vr>ot(pK)d49%e#Fp2OfCNgTs z!#es#a>9IlZ+h{J0jAIreMGk~3c=59&Z7dw^*?KsEz z2e4P|3f9feU+FjHj9uwvtLnGJyo2Jhm$adRh8Uk>P^;2SBKzqHqS`xc_xu-IAfzP< z)qbQyDlaibA|+)k!^cMBl@BZDnipYkg>1i1HG~&e?H}c#`3WWLxZa+-an0-`Sxm$Q|Z4jtz;5}7a;qJIBWmzR%e z_uoRCQRRHtmw>;EDMH2GLap{C&Vy4O+ZUWC?){Pp|NCD4pJbN2I{szwuWAVOn}3Ot z_gh&d*)&;EL1}{SVx`3!kdaT&6MD_zr2@^`H40eq^}O(ZOsrobI7nH0;Pw8VPRB6N*_n2YEiVir{O-r8n+)7yqC>fCtddef7QhRe6DT?3!V@A4R1^k1ho$>UW2Ht@P zc2~67)-x9`Gz#g&&IkEqS^CK=v>(N(hcz*sV5P8k3aiaeR&iiuV@4)?sdP)hpb+aE zBGyG_Bw;Ig{t)NnWQ4Pw5f>Bi$lzJfJ;f-e|3zE6+kUkY@sDaKzL=jO?QI|eybQY% zhDWiJe+3v%fE6Ym&!Qf^onn%1v{OrUjiAGG>g1%fHXB=X?Dd`?(`;=DPSBW5` zxvjd|dBY{OY?U7J`if%|-zBrK^?fNZw+8VrBGqi~n3E@`HF&yOo z0>x`&l8$MZHK3ka*YYm5(yfjdx(qT+cq}@Qt;#|{S4$KBifcokz5K&=xBweEn94CU z$;fBVx;Q^6+-=P|o|i4p6=%v6jhiQavl*`_NH_5fpNssekjOYnW%lByS#+3|fpf1l zAU;M~0*cRnsVEhI+B(pQt$6tC2^L9Jv>|j0-k}895WVei7kQuLATAB>stXZgVCfa>HD$_IZ_9a_!)&Xe~qc>0N+= zMf;;}c2b3&NT9CG{;522DqPLxbqltoa%TPOb;pl9?+zQGm1fs@{!T-01FtokhmQbC zT{!if{?8NJX}&-1gzLKADxk!J95?7UJEtF`dH(wg+Nnuh;X1FWY`zbnVA7tp0wo5l z$oQ!&G7&iu$y8FKNfsL-$7cb@9d}$J0FE+)#xO(+~r$i?ppYEKoWAMcl=$*X9iE1K5BcZ+hP!Q$a4ST4IoIZIb=p&^!`HS4Xz=#vL8SfSpd!8yWQf3P#^&n zulk`sAfH4N;EIzGikAe6 ztpJ#GFp*XD+7{6sl~LRQ$07AUQB;8OmoxZO-&o?F^oJ`HBWXTwHJKHHw>xcM& zSaOBR3D7K%SOL^8!XJ>UcCR;LDm@T}4gMEX5Z_MAm{@EYFFJ9uw_TR^`u@P8=bG?F8Qp0O&ix7He~d#ulKdGJ+8)yn5YBQEwT+q+F&7}` zs)Eb>m)8q+mF??|3o9G%_(n+qP-HPtRy#(i{r9w=)(IV66(Yd$*M#>Ry1E2%OFM3m zdpxu6y(}nFnx0g1m@_H9ogGMT<0dc34JqjdyEJKQtbhecH*B= zCWPhPPa^#x(7Rrz+ZkgBa*lN^n>}cALV@;TuKPM?ry5POy&m$i^;m$#Be$-!#_0iG z`vhwF*q(q8CUk4O)ax@hUfJ3M^3bGlekOQn>37Sijo&c0 znsvfeZdXRP=C4h;i8})+1sx51p$&jTqIadaqq1C9jKA+H3JAA;*Bw-OsHDH`mN>G2 z`F2NuH=ME1w`fm&Yi7gw^;AJzc-_QrJ*CPfp6ZPDS3hdvwWL!7tPft7=N-JC(Z}

    @+?d|ZMA(ZCWjek^|%x-`glTTQr>YDnWUK=1v0qQ}22s*Qs;g1f`( z<;-Jt?XlkhkNA9whOCK;nQZ`G=WV2GvIU4^W4qnS6RH;S^0ASTk4{@w&Me!qwq@lb zyo^(Y>OX5W-&<}5lkMC9n-BiBbKbV{K)no$hLO&B>1IRx8dEOzltIhN=k1@VLCp4} zTjh*&k3Lnr^Ha+Hp{k$W$TRs<^tk&*5b$O?)Rj{-+sdN(q?g-Y-t04RZci))m+wiq z#JeZq}=}e<4 zloHp)$Om{qdf~jLmgY~aYgC1w@h<$a*gZ55p6APN8hDVJBIImZk$$-^VS=6N^orq; zy#YGuviH4~>o9jGM6eSbxqA#ZQVqL>f(B9NXLGG-b{-LvD#$)OJ>QWZ!<#qyD#-q? zuZ0Oqi_zA^1J^$-EM@ykUL?i`(hK~b;pm$?TpuVO#?-=tChChvr8VSR$N@_ zSjjcRHa2}MD@`KP5G(_Oa^wzxPL1?M=7w#{87_uc-e{C&yM8pS5_(&`d~_oxFW`h? zk2^YZ5NrA*qSF#sXQ2C@0w8uL>6;B*>y5=urD^&zZ|yQKzB)rYa_w7#0vVbplN%0 z_ReO&HHg9Pg9wa#oz`&GD+oV|fD%wzR7ox#rRZ3OQ^?%M7ARHnQpRUZ1bS9abYl@X zF>r@PTINFszB8)Dd&=Ga+Tm~B-bNnTb(eZk0Rnff7nkaW0$&Ht?JF||DK=pMFR$|= znA@xc{Bn54as?mCodd%;B9+PLY&1&sV`H5^V(~R$tdL=aUvo>!-YV*{aY?3ZMVV|_ zy&5x;o)g|qZS&-`+Rg@zOXzweDllfF^U>G;rss?UP@|I0 z9P^`1vwtT?Olh6p+h`NkHtGbqTNGLe?n_e4?;|?SB`2F@Q3PLRRS|lp^g3J}e{dC? zFiA_|;tBpJUoi!!o+6}A4uT@->&GGQ@?k(8x{IMDUIn~VBfq4+uXp5n%eWAFCQ`2r zxtKjEj?~uPl{j-7xrZC0hpY#*e7SB>6~b)2OiQiJPrOb0EG`ch1sN)r-YJ4FrnULH zLf7tET;$>l`nCqYb2cimVw))rAMNZrmpWd|6%jk!a>Ts|&JSdj7(pBPNQI8W`mpGGZxjni5(0U8U;=wx)2t0(TsH3cL9h3{fpeB?* zvOc32FJhTt^!G)h*ZTBt#wN8D;F@t<>8G*eP$}X*{j_3SZF=V2EjnVzoty+%1Lmh?9iuvDLXi4v9?!_2pukw z*&#L8lY65|Y+P$Aqn%W!Z#liRPfu9#Wm1o=K<4vTH8d@^w{Va5+?<(SbMb9HIweMw zNn`J9lbOe-Bk4Ct6u>jGLYRm1e`#tfrZUO*a4}wJEg8XD^PH3#lN+?gG#>^X1zvsd zJ+@y%-su@7%K7kw72$WV6!p;t4t@UX1!;}2}Jvy_WdMrhxr;$F%Bl)v#JT`j=F~Q zs&o2Iqf)pNF^DCK>84m8T5C3OQstF$zXK&%5Zb+w-K36tw?Fj-d_^lUGDyM+OY8K? z{GsN+A?iei0j#zKxR^a0H3JK%+pBI^s$krM;e8acpL^Ap zpzhY_15MW{pzsog5Il7-8BeydL|iU9^<$BBN-9Pi%YYg)ju}9NhOX;mPdhq%#LK=w z=9dKWJXJA3EjqPiN_DgHUro;fiOc+d+~&J!?#XLVDRr~+OU$A5_p>=J$E}Oobj^G6 zYfmq?y*@3pD~eOGrgyaP+mqCYmwl~|or^b<+Y5#nX{Z#RuLarYrry&ghkA=XQ^WeM z5PlWr_9{RllcJ+Z_lHx%s?V@MC+jU@QhlRQ<5YP)Ynvi;_P)7GW4#eKNh6*q7->r` zU7VAudfq#waS$Jq%Fd)V8=IxLs=f1yV)C!CE9SlBj8lF%1$+r|@RoSYESc;!2K0`< ztbCA5+6V|G;^X%^E424|7|S}YEwTYh0uzWV@UdJ3{|{x7O#x%RL(c9jaF z&xyg=^OKWUaet9>4@2S%-G~FmjwDt(L44n?AI}0U0R`CLnglBFzV1>JntgmNXLy2x zdxx*G+ByI8DA_IlkN1I;Dd7tU0d{~S8%92jzlwu)sm-AF+HukZPG_G#SLtQkWzRh` zY!a|=b4C;P#S7(hLwbD>uz44I-;^Uy!f1_g&4$Zu2EoH`nrd$qym`T zOnydjuLLnZ;XV^b3VavD3SxaWtAP=88?#!#6!Md4xFLXd&;7?cc;pwr^|QKrTl%mg z>@wM%-%8nvKgLf>IN*e(>=B*3e1Nv@ z)#P<@Ffu%#oJ{|fC$C1v!5ESzf1I4j1bSyHtYiCKF@Jq_zsv4bdwIy_$JhAJ=l){$ zHfK&U{~Cav1ZaUat+;L0So&>lK#_$AZuL_Wu}S5>5LIFvr&8qaZJu<_=Y$MZ*X3ID z)!;nZANFoGhjL0JJ$+6D2csH_-EZh5(u48svVH$Z{i?@Yf{0m#?rcbG$@qJ;7gbE{7|^-YBg!Ar>kBQWRi@aUGL+!ozR^nHfEDCI@Le40KoI_ zt$DWk7^iM2P#gTkQY-QBxk(BglJTkPooC$(8mzNyw+fpNOlWl36iL2>rE9qXujepv zPy7=5f^P?}d%#qM+DyxIFcH|{N&Lc_nv^Nj;N{bs+B#vGkeWJWO1*k_vSpzr+$2gp z*ot*`+bgsF=%`dGJk1g=efw%TvAyCDg20H6yl6CX72R8!`JSgvG$r$Ch*;2EU_@Ry zDrgfB?4xt>M*}oiC7kP(eoYC(K5*j6)MWbX7e_Bx)ZW=dtLR_Rb_yd1YD4mGRM+2gh$PQ$^K9a)KS^Aq)Fr_WwSWWt!fzB-k~i#Gz4b6kv#euH z43Sr|makpi7R}wuG&Q?I=`WS^0*YQ+<4)Q>S`}M^@DPv?W_hiLIcG|r$Nv#sGft)- zPmH8j*ee=~1J%gXk1$5pgmik%MS$2-#}?uTe&Q_TGCXnKOAK`?{hK!bEotV=Zg0oJ z)}z*1?Qt};qAfc=WZ-&w6SxPMjTvzPL1Ky_JV`lt^xfh&^r8IcjvM<&lN=qu`G3_vO#L2X(+R7w)Q>${_iwJG;{VtOuA0x&nVy!d3=TG|K+s6VEQyFov1&VE z;yKb(SvVo&T1L#dq(9s)(|Pjk@bG5c z|1*P>g0ic@LJfM3dfvFq2XHFn0`@;16I%tEU9BVbbe150GksmR`LxDQXq(~lH(%H; zTVYzr{!aW%J67K?PTa#EDqLa9cOb?t8g0dkXE-}bAVd~`0J;BRw$yiDkHv|YNgBIa z)9HEho8}bP2Kzwhfq%FGj32@PHyQ1~z8F8`Y+T~KjKs4EO6oYe_OFQzt~7C|9Tjl) zv1^aRKg?=ccs2ehcyR8~dgwuRHQAr$;vR&Kso$^eStS;hb-c%h1uS&+k(mtxNa&{Y zB(sTXSrYXVi-q^K#fX(x`j;Q?ziSXSdrSM2Ox1Modzl;LoG3(d(3P@tZNgvg$UJ;9 zO=h9eY84APYTn5KER1{$K(D-E!rZ|c1ReK~Er$Xp5(fA-SAjzT6F^h*5=y-eH38sS z@x+#CfOZ^jltjZSN4n{3?OfIe(NWipuLMhS#V3QoP%<5e&mlMDM#VP{p<2gsbvEk=*nSUoXxJSqiCSH0+{UF&Swe>AQ{dDc zVtn3bh^)upYA@f4=`2G6C(L2CcroA*59KWs0Fbm`Pqs2_01RydpH4|NNc6l#qx}`?Nj%cv%d(n4fHo z9dX>)CIO8wJ0?^oK3E%xKF5E&3cpz)40wQ_flT1B-9xVhd;R! zVUEj&yy}>nbv$4@cX?&(W6&ixUsrJSKzSe>@~|D9B`q7Ubulm2MU^j^v>)BF&*!FV z@qhm>fUbw4hBuPvcjJqhV$yx^OS1M)1J*fpUwrvmh|KvL<=f0apLpLH11g^Jidpqp9dAx1P{&otN9A3qnb>Ua}a`aX@Jl@M>VslUu>5GNEZK{pZZ z2fR5J*+JzfedO~Ai``hs449ymNH~)4-_z2}L7nT?pnZ9d>Mv!J*jtSNm!}QjDpu(8 zGr|@xOwAKN2hbN!F8x%qKr#tyZ|~wTiAXM)E`_mE4~pN(P@el2ug}e=F)rn8T|l>3 zcd;nqPO0Q1&hpyW|A?me;;ibP$)z#>_t|^9QR-qmr=R6@1P`L-tH_q~YL#~qE7&eh zV(C2DPek_+v#sK8<*F6xs!fs=9zo%6x?P+?Q2*e(q!Vh%#Z#sp^gXQwFI ze1WOCSm&y#zHj~cn)w332kaNBy2W8CF&|qxwalEUFO$^3)Z}t1tZk(T^Wv)w6aI=U zxPz|7ZNXIkwdkHxp1O*mI^S8P?ipZ3{IDo`NP^mrF&)lMjI^hgbnn7ZvheMF%6r-aXsMd8vD(RR;kd4dcaYNrTY4wRi@u)ws4{b_dXs`9hU zrKuHMsROlLU<+gnYQVkucd~H6*aUY&M8;RA#h!4U0>F6py6?cDb+TeheH+rCFq>+h zB+ZJ&n-%bJZ+r#T)iA=tqqJ_g8nCMAx9i169)bSLhO@BT;%V$L-Y0n5n9mH{PexyV z;L!hT!D)ckz)%~`gQ)KJHCaJ;K0-f-<3GHS-AcNoJbQB0+hMvH>GLHLlyF}#>@=^u zHTrx>EkMKIM{C&y>U{e8Dn~R^+`}iHGKUu1$>)4zlA%Px+$>a0bk*ul%{Hu6ft|DF zla9lzMfIwVy4N9Gs)c(Taz>~Cpw8q|fa&(mdK>q{M$WJ18tneteAD8jTMHwK(m?C7 zl+AR*`iL%QB}o3%T-f%@onIM>mjU;ic^h-r8uIoI1(nh3UZUcBs1vDD!$HVUuaH;S zkG#DYZiT9Dm+~LL>mZ#X>;7M+cI|RzC44^nJg1`MqG;QDBcV1`y%LSt39KNed=;hdUVivCZvur@R^E^)62^^jQ8OV8ZcC=Hi28La z;uq-7!NtP|Z0^Xo*YuxdLRMarl|}6!6eE_$R4B;t9R3euZyi--(=~7l9vY+@X$0v; zx|A-FmhP5Li9>fcNOwxZA(aN{?hy z{Tu8-y816?P1f$E-9!x*12vM-q$gjKlC0C0dht2jlc@930tEF!ad*=|fYK;H#o;!@ z!6sHjCRB>M@w__H;mR6k>Y>pzyH$*nm_@A5Rjz@5-a zEgvo@8x`lBCI8jrGq%O2U6hGPAUM}mfalrrb493!?f&n5^(jKI)&=T(@ylrcNMc&k zX;Mn={2w$Ib0AN(xy|oGzkYIS|MJRWaUfxS?|1eD^1hD65D}g$r(QI+ zaVhL+h6BxTyZqu+NvCBC#lxT7;<4Tt|FCU@7J8=`^e`Dm#jSaDIJ%nNv2`yiF`mn? z-B+!~3=XOq8rI~xVb}K2@?dPgp$_X6NbIu zO&@b;?sTMW2xW9CZ~Zi!vmMMbG0}+n<2VbQ@@p94b@2`)!0TjS3cP0n2bqgaVL;n1 z&7Js}EX}1_qe|NFoYW${Dv+SB7`^1hv_!IV*7IxyJ$0u2nCA1pAnmL?1BT)tMa)3P z)Zs$#sdha z>F6|967t&hmPE!Mj32xSziLaC5qtzTBZN3{`{+UBF_d2cPyKLgZtl9uxUN#z zbNANKi#>`JL_U__d$g=D-nMu$kYKwEsrWo8&(jtcL4^v|dezCAswLzx_40496gcc@ z;u8=O|J&pqdYas)<9SE_GPzacfbrkcKlitqc?J{+O#Gy1Qs;j=nr8^ie0qsq_E!rb+nxG6|7 zD`e8T@IkTh3Yn5fQv5)}RCb|>ZLj-D;n#;Cqx|?8c z_k6C#KdK1)$@CN@q?ww7hsD@2hi$9xGO6MBVy{2b&!#gq{vA+uNuv1yk}-AWnh>X? zOJO9%+S*zQ+aBwofSIf(C%|d?sYFn2W!TiR%)>N%hzU>4{GG*>HtHXO5XmM7Bmq~A z0hk#TB(Po?DRzOLOxJOuAS1~YLJ184-GoF?rVPIdpPFTKY*BudY}=vDM4ftH#e7|4 zeO(5n^9r;rE|IH7Hu)^o32#?yzDX?n3}@FPi$;efox;Yrq%M>B7ukeTqdbnc(WbrNj+}T|yG~P&x*QbSJV1 z1u`r~F%j(JdwK38p)VEV_4Ces?vxV+`Cj?lx&jPn4iT{268F^gt^-&T;EDPI?ysNe zFI?`V{85$krChe;ncAYil1VON7;Xmk+&PfSQW%A_366w*$lIY~njHrbk#O?qBnCH2TypZvqFWXq`p?G54|vmu6ZjH!ME(9X{Qn9EwQ*AT}fUh|_t z$d8X@#C71HY3=b#2?D5Q!E`=7p>meVIy z9myK6rw;z^CP+XRClQo|+fjfW9)gDSP3pHdy&`L6{?(@SZk=Io3~%BUsH9ERI^yJk zJ3(t9H#28=Q)Q*czU1bSbY{^be&E#sS-w zIIBGw7Zqag8d9wHHF_@!F<)FSiXy~PJcL$yS$09uk}5bkvwuoYulh6dMJuJKU+g#d z0BH*L7H#>V-=2gK6r}JRuhQG?T-<|_xF_!AEjPny2*fkLG^E|F0z)mQLh!ujbg%AR z{NswA|G8`lA^D&E9QG`e;x+jbu}D>toIuCSw4`5SQ~hrJ_V0lRC3f?wv4?bOvG=Bv#_%H)Jc5Fg~=r@F=W z_SSoGmjWgJxEsWOT8-=>U>}38h4C2?U?5evz&$E0%{5$d!b=EUA`|@>sOXo2Zo(}( z4%HAff#{H}NQ>|#pCZi(Qj#c?JDFLEAH;@6{h=dtyjpy0FXV}lYsI)TmKq8!P%rFa z4dxUYuHCA09~Q}t#0O67Q6K*l`Tx1-Ae&ghavFr3fb@{Od)(uA}7R?IWBzTu= zO?rEw%1p;{AG;r4Hhy+l&U|TsBoUIo-sA4aW^5HG$hmE?%W8o^tnvNW`r{Nw-TYP- z%f;IY(O;@-%)6))viYO3|E2Ywf$idT$wL8@7m&%Ikez~QXgX~jXnhz5k4#MBH@+?u z7%Zz8Cq#MiHAOxDfx5!vzV6-qYDXY;&3>|ld$7~p!L5K6a~fK3O>G&kQU7r*X0UlG zQmB^4!;2vV<+S$v7)j^Ofj+FM#)f|`-u&+2U;+!CIC0vjn z?b`)J1D5EciO`8zEUpXVQf@f`x-ah_@#k+#I!+D6dxDgS-!jT70CEsCC=G6$Gapw3 z&c?|Cwt}1p*PA&i_2HNa?&YUu&9uy5Y69_zi%-&mhllK0uGG!%0ie)X^7YW0%43|} zfr;UKY4K>aKczx0oh|7Q2>6)K!1PpKy<=|$DzAemEx66C>2)+9O;an_WYYfD1h6Gr zTUw3GR8WQ6)G3F?y1905<+Molo}aEgSU7qXg8l~mi}OR>mQLj+l>(eGTvD7;&_7E5-#?#zNHF1T{hII(Pv$>j9A5FwEMPv!cbR*iWB&8{Q+DDxzpF*OjLlIU3x z@?FJ@aFS5w(~HA{-zO z&Ul6Yk%1F4_(T2O3S8;$b|5(EtxF#HpLzCwKgb3FehE!;tP3oooNg{0D*v zUVWfotpUXTMW$0+TrHNFsp9;~irc0($Vt?q+0g$4dBUV1@t`XCv`WP=J#(I2ebT|j z!ZU1f-{s2Uo5HVg?lKuO>u(L#|Fmr2QW~X!VXtX!zc4^N&WfGrXPI9&R80H~hPmfjC1|vxG99r(Oi-!Y2qzm0HcdICE9tgq^@2~vU0wAVAJdYSao3?b zjL^1ohTWkF#e7d$|0dlk*)Ovu5ZzT7;Y3nu`G;3o70%lz;>Hrjv*=S3HF;CJRc3TAIvu7 zQiC&E&6<`}-5R?_%E^tHPPDtX6RT1-2}oF%jR}+#Egtx<$O;PO3dT`>$(Gmd2j%0cmUCwd)(kp=zUD}aDzv`&xmrJ`nsbAaD-*nqI9*z3 zg_D_u1X1^v+agtgPZDIsRrXF=X9jO#J6~4DhVu_&t5ge$`$S>YfT=@GoC#dqvq`&yQC=rykeuilHygyNy zg(ca8Y^Abe3gAUs`u7wl&%S0#XV~xr;^&uuf71W_KY2FxI@U{e?ZPq$QaY9U1~>6j z;^vHxF}93oLBakt`;8WwG9${lk@#d%htQf>rx+)+l1N_W(G`X$1^`NtL2*dCECI=N z`e>2ob>Uk-otmQH=LZC5;eh_TIU0#wl2nvpV&euafP}b(;)r; z68SnWU)tWwI-~kLb_73I>vFU={dsLVg!ETZ!eX6+0{3@`_G)&4z{{qY`+Nn!|=NU*H?9?55yY0?eIZ%n?A@wL`@%W>V7GK!B zCkQDVjZ*X=)S!C2QzESHK0m~>CU(C7yLKNPH$+&`9!2VkKIRvlr#&c@0;Kh|jy6J2 zZ6$WW_q|k*G1c~z!#phE(FbjcnPbzl5iNb<+<%&Tr4Jy1k(W<#Nu@}TM~QN{8N;P% zD00+5;XDH+(?$kc?rqtLz-!&YtupI?Mc@Cg zE}I46sqBCMG*v{Sb?eSOoQZF7L;W&V10DSeT4P^p&a2Md>foYjIbJ1VSORrFiv5tk zdw9^suB~M~7a2F0z_W8p;~%!koRCWA2rux>(3JehM&-e+LcE&3)w{;T~ z7;A)THv}a^wuC3@D7sJ5ul3X>B=*+86CB2L)t|L2m{)san8JVZ(-D6%8Zmlzit$ef zDj4z#XtNVgFx@(H3;Dd7}9E@|>y}k@LeFURfyYbRv9^caR zbh$DzD1J=1Kc)}{cl976P|3~p!^ct8GMHId925TmcGrm#xkWQhelFrq@eEx0>UWf- zYu&ka2^tl@)Zwr`RZp+7V$JP)Zf)}kC1R|kgm$_%#k~vp0%*Pwz<(@~@^>?Q@2Q|o z@sh4yeSk`t*q|Y9;kO~L_8$9CB1+;34zOc1Ti_~f7Hdncqiczt@-81m+Q#}Oi!=GpujfPmJPx?;wD)BOYg=JamHj9yPfKMVr5%%!MgGQf{0ZgL z0xb1dNL1v6O+JG3Y^+c$+G4TaAJcRQYiHc#qe2%TLM=*GnZ%3d?ZLfnJHf$UTA7$6 zGnESq@=;{*<2_mw^9#CrgP*{qHohm`Hg-2EmU(ZrDpdpOQ|9RG6Qq@XPD1UZRM6|osi2k99U)erWHD8XJizH z`{y=H4dB^ymHb=GFqsk<#?;HWq7fNT=VO$~eztYW?}(;ZuOHWuw6E?@~aek$}@@$xk_j@XrbU zKaLyNg$5`==S2Cj3h$P;bR~0j?BZ1E?1n2A#ezULPkn1c3y_%yfc4ePzyCH zSq9?A9WfZ2YVA{Nd5k@(LDfUxF-CMS$el?TI$^ z=%fbGE8E%w;0z3qV=A~Pn$3aqdu%-Qe4Yoi7tr~{a~zrlQiI;TiYF{epICHyf2b-w zd5=+a_;3Xz;rIvvhYQhhbo1g42T*{Jc=j1I{}aHxwc6!D&eFB`=(6f{rMCoRoF6{` zCr|Zu3NLSfv{c7$GDGCTAj{ZJ097*k!Rwc)j%RzOfcq&v1n*GSg1-)W>;%2F-sle1 zWB4p-6+xePlzxw~BL_S0S@piOD;aV{RJPw&#ZVD_yTk@@CBE8>7QTHr&X8%O6$bI8 z1HCvR1baXR%caoC69vI+6XQ!y&)tgp)OImboyci5kuSaSpgwPdr*a)9(?H7tvP|@~ zrB0r`@y^}*^xP0V@UfJ5;&8|&8ZA09GP2RKEYxK2r%8* za0{@N!bjULNb)FaOKlkSyq<(}Z&zN7fd_j~j8%MWe6BPQS=t%7Ws#soZF8P&*zSY(2(p}5sz zy2dz>=@x~l6IcEvjOAk45MUu#Qn@K=L#`ALauUDASN&{Ra|#()de4(jOtU$${zU2; z9c=u}uFI3BX9cY@CNV&qVgNsJWa3)IqhuNtp=&9>em#u+uBn~!Iek?z=`y+mBmnRf z3>m$Iy^vZ1j`Dd*9zy7rxi7*gG8F-8=Zs0nabe>Ee$Ykt>E3=-$HP`PpD5xK6WE|D z2%1$%NEMH!vYPsSl+9q!Nf)9QMH?IHs2F^aW-C3@-Qm=YV})C-tZ zfjvKucaUC)hVD+4cK&8tCiJvNdY9A2v7C8qHGfkxSfmZ>>xme2d%*Z^3f*)ZZFhd4 z$@mRFv*)OB#&^O*#~9A|r=_d2tUf@J7JRpTk6!}M%;fs$|G~alRUVI5!^=8RySuK_ zl>?wP=zXvU#8!z^eB6}vv+c*k?TbwGe2p%XPGNZ{E%c%@*%e619tyVsA_Jv~PpVLm zeF_DZOG0FYqOWXlQ+ZnVyovw1U76@_Puyn-`t}!U zBP!+nUn+!E8Xz14;9-kA8Fmx{AY=>7B2{;dVjPfVv~R0;`^@l9Bsfd&Fy@S1l)Z9D z?+f~vEB$7cZ9xpYesLVoUQ{O<&(iO5B4>Gs0R-@wUlz4*vW(>m3k!-uk9VAOfdj|5 zh>Z19%Cn_kj>LK2lkHYKO#b}PvPa8LXH%)&YXRm_>Fp_bqed}CgJ_2BI5t?9AsktCPhTVdPeC9DqEm-f3l|KV~v?dgkoRu;nrnT3xh zlJ4+4{Ohh@hY&vzRSaOF;|7CeM6!htjdiHgBPM5ti#kNJ66WI>*t6xPG!Cia?E@&r zrV;!3f1+Fj#t)faF%AeU{IY4AWhUF*uv_{8pb5a$!vhrDRYIOYMPaiP^*aWc$h~UCa#dB-yGAVA7HOw8GppFX zD0xvtr)(q<=5g8KEFK;9LQZ5wHM-cVYse<=_yPd6DIc}zHuuD#19xix1qE-Inqlk^ zft(z9)r`IlvD@t!dd$_&>fU{rB^PI)7!!cG$hwypT@$$>I8|5W(r0t84a=72{4L(&cwG%2EP^G4+8`U;mhEJ zhn*JJ2N2o3vglT1oFDRn&)2T1OiyD_fdHZiPkwOexXU^VeOmaon0y*vp*cJj`4}9M zw$u2#Bc3B+ns>8Bd7O%)=rkO$h+t+3a{k9Y&0$W@Qe8Jy&iumEQ>h@4qNk{x#m+n% zXfNGYht|3JoH)pk{o-+vAKmVCSUK7(4RTlr63%Y3`>b2y9VGdub)#qHxfUbQK_>O#=xi~$HV|5!ng(h`0n^C^BH+`_JbS} zqIwKsh)BS*zUQLQb*E`ISgOtGCvZ3m8+ZW&sD2C^bvM4^JA5!JhOcWsty{Lk!)>v? zC%LW!{=4UOlt4*Rz7o+*_{!+;y&o`t!5w9b0Hx5u$f*iqIBx};`?7U#SmWS=>2t7Noxuvu9G2oH%VEpon-8fvBYNoED{SYTMpH+wUh#Of(w^(YBm^QIqf z&WiS@QlV3CHdQN)_GYp)n;9$Q%iEPgEm`9$7Dcs^wV6idrf6jCYyZsJ*Wm zAtKClsZi8)T|s$Ck1yv=g1WE5sj*LZe7ud|9zNxvQ1WcbYT5$bc0s7xr^RgFlLGjn~0m3D8JwM#r3KLH{+Jy zY2xgpia7N+$y%$*Hn823z0HFRwfV>W&0Y0Ug`xk8wl;3T)ltsyh0JzC)t;=6SD((d zRCjXK7c~<$ZuS*(aH_%vRUAOq6th-y5q$!q=ZwwY1M@|ky{|i`QiWH_Z5J2a04}j~ ziEMewZ+Ks}qrEv?#}8V@i}YfHvAMp-Y2oJbd|hNHQ(kYBbE!3biOQoQ#?RG}TSy8fZS9-y)vCmqtb zu25KGVn!&&4#vyEli(e7Qzv!oyEi>~Pt(?B+!M+k@Y5wwt#N5l#dbk8D29YJ_oy69 z&-o^p(b8Cbv>YM@d^grbnC(ytjYLSKphk%5wbs~89da1VF?AuO7MHqQYx9VdGbyVa z^Qswr>lr6<7dFkHh#fXTrBrnumX8%GvWr>Fs_S^M+kXgggU8`np=y+?<7p3V4NLT4 zUc{gidT?Ax@_AfGDAOyCn9S$de3-0C4-)Pt%x9O9UJDN#8m2M&RVasF z2N&Ty>KF@5 z^j$hz_f5FYWq~fSS(D~Z#{DGd{2mAp`+rv`I|THTQ2Ff~U&6nNZy3b_~Q?CdI$QpK}rbHBZXW()M5jD5+r5~HD1a)JfG z?$w&W;`j%qBuVU*Q3rt1H(j)8P+eeD&_v9dF#KsIi;Pg zu`Q*Jy3XF@SYID~clz_6{oMFxIh>!r%Ha^n+Xg?NXu&w2@~bI05hEp$Xgd>}YZP;| zHszD5k@{vn*FjAT5qXa|w)Y66Fr@%6c}2WpK$1og zMZl@t^!Equ-eb?P7qMRpG`vxv*fLe&ug1=D5G0&PlSCb0VWB}7g$6<+7^ghtiUAD) z%6MYJezq{nM7JO(WKJnJ1x2wI2}l*g9rB$DCE6?fz@QkJop)n+W41~}(tP4R;E9&*&{fBu7$83J zes_(c_uKk(kwW|8k|M7;%SlXTQ3UUeqD85DlJQEZut1~k(xFb^io3KA&SXgk?l zcHn9NI95N#211@Qrj$R>9-^3pd}G9bVS-bx{_$`>tWVea1d-?o$7Oi0QrV1QCm2KX z^JsfCV=wxu1(0M$F=)%aidDrJ#6BZ3j0!35W9dJCyCbhex~WVjC`!j!J_ckUfsMcF z-0g2Wmy=%3n$~kA#fupY*|l8Gj(*bEs@}1vK`pJ46p0B)?^G0q8Y}8o)XyMag%+Pv zDm{Oiv(F$eH1R+H$;`}?)Z;RtXKP9tCl#n@EO_6J7ZVsB0kn74KO-zu%tc-?4!g`zb9mJ(jB0S+se#tUh(%V{L<$b4!aUJ2(7siyW*@QXX8O|C zClm0<&pcw&9+)`0B(W5yONHZ6;q*%;;~sp-J;s$o+udVP8-Sz@FqD1)EH^gC?U4$# z!SOuFMNx4*06Ar84=~+Qmp|={8y?`7ws3PxMqM^HoQ08 zI&5SzRs5biD-Z=g4Y)vO&YsU@&H%x2Qoe2%N<$+52vB|4NbJJh9qf2tWTwgWMw5M- zeB2~^WNkTaZ)pa|2+C1RrvS1ywcYD34agMhG8HuaLC`YH%EK%TF)@XouUYH+2}DV3 zE|(k?CktfpIRJOG))LnyuNUB+k^KUIEu27-U%1IwI)AI@=>6XK4hSBi} zygl-TKi&TYUw;H}6^Dl~jiK7)MKIH65+3j)S?g?z>#;3!`?Vp5QEx;i<=uY4o+5c} z9Lf(@=)hK{^yU3P-Gki^l1$>h^o_G}AQ9x(N#FcRqQ-Ru|J_0D@XdBDv~15ObM)mN zz{ND_dLuCd1T@VWZv@sH^?@sSH~|?|*{^7DQ+Tr*tdAz&bpQ~iLf}%bpiu?pBKh=R zHC#ZT2=cPe7*L7av{|2P+<^#fuD#Z4QkeEfAS-F;{%&>~I925;cQ1NR@keA~q!M;ExGQ`Irk!J#6e;ft5smyH=(|&}*cXy0+@KSy znA++$gweP`qYg0AN#~dP3-0ZlOjHJ_K7;tA9k+JWI;ET11k!!?S~u+T!}uePZZIBv zPL9R-z90{@^p)D}nBB}tn7sB*{tpkJAPilrV*tigk@VVBx!OLZq(F;A_uP8v9lj1u zWLbH6-OW^)`e?5K9v(V}>YOvFQRA}9nZ8m;;Wn~vowWe!OVUY>m$+oY<4y#ct6k&{ z<4qEgMDxx~n4WI$>-Je=-cCAhrCC*0ugj-#p71&E{dC?L`(}&qCEQ1{F^eJCCzO!K zdctWi0|kXZzm=(Bj?yEl5|uX7Etkh^gyd6MMMEhbkkoy5qe#WFU&he3g;4^G1+9(RLHb|i7=)BQtjh4SeOo`t}{aa|x5esBwL*S2hcYP-A) zLU)yM+1(SBpC)S+kQ@q1AMO%~BOMxXp1A|C+Z=_hOLiAEDtn_Da6=n=qo^ITW^{WP zdn0drpzhv-qti{>STUow@k4^~2RHE+P|dIU;5I$?ss^jskwv3u!B*4Zw0X}f;1)ai z5EOWuDj-j4|dGoDx|HQKqb;5~KX zl0QgJ*63}Xlwk*(u^ly|xo8#Uk*vQAJ1=~!oa6!Xt}v)`44)ew#mgW?S}f zYGjIfw?38o4&^wQ%sb>)SFoTSRvgk$gj$<; zmzt9&0B&06A`qz*bo)B@#_ezpfr%H0`cur8n2^vKNm&{f==xm8$%(@rqq}`@PjMeG zf)emLSWEvJbD zoTZI62{iLV%lk&VN(t0I*JiPy{vP_cy0l+6sZhJ#s{3@W+rSQxqFrnMTPrMR+hLTt z1z*+2vhpEdfB+bp(RIDG*~H0BW;7J!cEp4I!dM+yM?a+IPGV0d%_1$#eY3w3|A3|I z8u8~dcmrJ_e8@(Tt7)Z(OWL&nwz7dlgN%~C z0_RK`W%rXtsVZNq#L>bAatHmU>hOl%Uc&WzIsHbu852jJw;D_`$O!9;wx#7v_`R9Y z3s3Zlj+63DG~5sB^*D)SyY>3MHSdRLf-4xn``mYv!)Ey`^{HXrv;h^c&Q5ZiWOa5o zjrQI~{lG9V?;7KJ^~&oNvE5_6`w{?4opzB`{iUsG-2uHE$N%X18@PX|FR2J{&R6jD zKCk;7PPAQTqln76RT8rY<-gL{yFkd>TfBp>yd8(^jqCxel28w$8@)m+$U}BZI1>jN z<72MCJHQpvvz&1tX zvJXJItZHcC&|(F4=I+$P%|xEM^s~^UY%dG2qgKd-9!>Zdk@LQraBufps$wh~HJ7}{ zuTw}iznt6#z!8{mJht5-BE;Ta!9N1-2rzeJzLKkXn%ifVr4Io;1b)5k-&p-rYR02w zcf`jc?e(USORy(uKQk*IxoAvih;03!PY6{FFTR2^3$b{y<@Qor`F=C-x8eaYv85az z{tBT;lk7C&$Jj5E8jNB2^K|bPoIiAT1YFyWUF1lL90x)iJ4+3cb9kC4kr2Q@gmW>9 z$~XGObDl->oOKiZS>BOnfgkR94fq7!9nL*>!>sw{mdbuD#bZ8P0}a*DY77wVit_me zU7KF_%Nqi&iJM}TZ?z-6y*b+^PC9!+{)Yz1R*N!d%MtLb85l}#1e*oEbbJP%XlZeZ z8B_eDx#kT3%|wOZ_u#}rM_$ur*!g`6SCAut@Xguq6iUnGb0f=SPwk@%R2%;;m^6Uhc{TNvTjOb z0re=qb@YX)n*vCKzmDpY5Qccx8yt{sHR7rn^;AdGQ` zpYpYdgi$OtHE3*mGAY5okXN)UNpu#^Urpm~?cBs0i}KmCH_6g(MOB2Ymv3`|AGjD_ zpi$s>7v~9A54|!)8pUiGMsRK&3&Enc^<^QFS^I{Skgr8>JEld2NOnzPo=E0uhs$O6 zSe3+`?EBVaB=Mb6>h*Yx)EKOB-rDPUy3p?^Qsk%?eba4DtO+tugEllQSLXXqQm$+I z(^=$k1(->IQ9T5dW5<%9&yryB3$_=&GX97Nqo(d2mj{F|WB9*7C#AbmP3H(OsE=;q znZmCHZhU+zIZnQf+X(vP(uK9dMr-&V$w&c9Dr|vJ%l23|L0XmjJ@*i#Q6b^n5cO?lgndJM_?0eAYwgH!tq;L_wW8o&s`Dt{siqNHc~KbRbao!t28{ z*_E)^(GgXn2QxAf3fOQe3-H*jvxHk&08Fwrzlk9HVME2$XS{I{W%KvY*PQDUgR6wx zJM*&7PXM%&CGpoeuL9X*<{5uPBeD6);Sv3fOM5frVo!;SKFk^RkY^DT8QR9uO4{0U z4+H@o^sR4$^utaI?d@|`nY@dGpEd)qCkleS<`gu3o%%^LO3ONJ~O4v^P$#Q*x`~zk*23octm4Qi*0jg8**lH zDvXsC+#nk+Qh8CNbR}1L5nRzG=yA?UC85&x%Ye1V&z}V++z($Hp+D>yaRq>5wRZbt zphmH{BVMJ8Ea<$O_*PM`*CpK88wANw$`a-XY#Gc{KD2?0<~bYa@ageFuBh z6&DnLGd7wdEEesjd=!{zFl8sPQf2b_OMI;0e^8GG+Irw&D{T zcrLrz-#_IxP{)yVp$#o}-{DWG_mO&*3$j7>?$Cmr^o~~AJhR#I3-?D$iZU(HvI(?@ z<t{EdKi*yWo1&3Lxpf4#H^-7*qU{XW0d8*Fdyf3zRL(F4DKu`ffPp0 z#;S$$>?J}ibPkr!2$4tI2oE9Mj7WX<{(pzJqUOFRXw2)kpc{sBfap(>?z20-MC*{_ z2u}kHcXY_d((VG%QG2@YjGVd$FL}YCFti)3?_Pvnv$mU9C3cRB>_1Sr(3HJu)@18y0bx$}p| zZCIFi7k~8KI`IKu+oI~csSr9okS*a9sdx5vq6(mWO(9@TK|7L$d)o|Z;r z)}Jk0{P&wVs&;R?j%+fd;Xiy%1Jq3pdYs^t#9g@-$j_*hp+&-qFfKBpv%0%#X zPRVB+s3i1dQck=It>zvrCn{Wdh5^*mJKh#$z+t=i

    @l$h9x0UgQ&fg8hjI%bGa@pwoX5*4gi&ms9_y+@g*bKWTBHGi2NZ)wD5{7W_t zD1=fPD|Bc6Jq;rTNjMB^O3+T|MFY@3@TA?Z!o zN2q?e*@#fO2*b!q{jc|+G}_MD%qoP5R9kls$d&DYKnVvr&&`#oTw$tNBh6c8??Mzy zzTu35#2PO_{3F3$CtBc~uFLGtM0QGtik$~tOMHXv1agzslHi;uGX2NC8GV}(O@m6dV`1)C;kS2m#Nm~xd-x@bc(5lkY_SLw8<=A_n5dY7qfyti3-&&^|V4r@lX z+yGILo8;D>_QBXKuZ4`UkyVuW36QvFBt9^4bPKAYFO&5YWn%hdxova9ajovM*} z&uaE!S=i{j$}t49@oW;esWNB1B+XvOkX`0h-uTB8bX9o0kk6J0su^X;(0DY>Ks}b@ zJi&MTMK>ST&=?ti0KEdOzLV z`%rut-TVSJm{Dhu4GOu}bl52OlFl&Lf7UCr{Wiyq>aw&*lv;n@u7V4*h4AEO)JQ?~m;>td(cWUtpcsv|NGPq6!7 zR?Ll@qSPg7{2SqjjRIMVb;AQ&$sI)HE|9urlE_DU6;tS%4wn<0`>!Ok2F$B;!B7zfZ8NjER9vWpwT>R~fY67crT{+Kf0VB7m5 z&($`1eY+sro-TQ$4o2#eng8hnpw0$xE88CE+J&Y z&6|3YdK=-}mx*~novi$>+txiVwD-7771ZJh&y0-= zOpoM5z9Q5c{F;$mhA5Z!0`3PPIMVM~L2aKR^VbWivFdh&Joc5IvKU14Oz3jvnp~T% zi+XKCT~k!5%Edt`!u=7&^68<-G#~7bBbyYTDwPUhHPW$7zZLRLfL=)bJzyhdxUsc2 zRL_&gA4^WyQ32h{4w9wx++r|du`six)smU2weyAkBjDHK6{w)i}}eVnVu0^rng;m4u<_Mjk6l2uP@& zL*p@=jn@mTlk3>}92le8B=b0>-{U1bMgX&e)Q@J>*G}7I++(5}Zjisf0Rj$PQ_qR? zTrs#iqfD_6mdWgwQ`ff>bn4@joUd&ba;8G<#%15ZFqxQe?sI=jisdi&tf!{#cH!0) z546oVDPp3P!1G|(gtJVc%1=&Ig~l6|tF_;Q2yQ^@6-z>|DLg(rFK674&6LC8rt(ZU zwG}CQ&-lU3lzu(kYvGtARw&}3Rp!0F-6jX|z)Gf)w6n_OjQOrB@S8OdyFI4{<{0UHm5fVKsc`i#*k~cF>B$)p>BLhAK@G4-Q>Szy z-5jyEDsMP2tE+yTh!>e$pzizpg&D)fx0^MmzK?zz_j>v`)eG0$zu%u8P&>tv*5ZDD zN&IQV5$3f0HIAf$Pa)S|j{36E5U~^2GravfCvH740)3BZ7#Ve+NWT2IlzQRr6jZ)6 zGAWh`ESK-=<4IM>6o)L1*1txXIOhx}r<4c~`!MYnYY<`eNOTaBqog*pT_>}nKs zDFz9a9|TscZWdR4p-40^5~*T{TJJ*%N5x&YwWg)GF#qU!;quX%u~1OBym_zqF0NAV z_^gtiO!SMh-8DIJd2Ac9w9PD>Y<8tv#O5(Kl|Q05cOI>7Pthci4Cz8DdwanUsw`e> zI*gj+&mx6Ztzd%{8seZ|hkvJ+V2O&pj2j?Bk3B3TuV95{mlmhz?qVz6l(gWX8LuVK z@LRAmcT;pr5+g(Wyto^bRj<^|+Tsvyk_|Ew8_JewTUsF_moBByaAA`X$(eFpmXClP z;_R;hW1i>47@6HHs68c2DA%Vk>e@q@qH?Dma|!2_Hn-`+M>ERqFG#BF#M&2df1-}# z378jkv25%f@}u%bIgmMR3QEXKi)(a*N(^=;Lur`ZXnlsVHLxT7rvvSKR^P1c+WjV> z66v>7{=Upe0%2|y--EMe?(g468fr9Llb8;Cx+##E?K0z;6ghIM(HVG`jQS-mX164+ zA+@b%&8uH{j&Tsh9@#=nB&ng`=@D2xukzqWuqag+j$WIIsf1rLS6xK3Ledao%+*R3 zalAjowq#x-yNc!EkfMSi!|04PlAstuqG2vkm0Mk-&I>?K#OEh-T@*IxHRW?{FbuV-V4uVpgcT);Q zeznIUSIP(F`qNS|AG}acusW5p!}t<60wKi$;h`~h5m9<}>K5pwB(=jep@pAlss&3a zx|_`>lPl$W<)kh(f9vogbj>&?b13Uj$}&neGRY85T6+mIn?RFKX_;^c@mMP-834=E zVO}up=~~;H-hg=uoZJs0bKffW>#reZ^Y+=bBkNZPpmkNLi%DKnMjvkZY6#c=mEL13 zq!hLEQpa!YEs_con>)`*u!K%`OBN^i0hXeXm*zo(8r-EyO%mp7xRw+> zGxP3uj98YPA2Af4Gi2qhA_u|>2dNL?9>-BpXlT*A&PjkY0viWPDPb?iR~#q<&0*}Q z!Z$W8DKzjdVz88598uCg^7Tw>;$4c{@jmW?UWKBeWx?&@T-(fJ`OzxAO*u@|v>@a6 zsfylzf2%l|KqUn})LddATFejwfsLQkiI5<}rQHG+%Sx6g=Az86zH|-jU z+=#Zplr;yG7)DSjXTY(R%g%@qoPw{C3qlNZM{Re+p^SUDl9n^%CBDvzt*#y--9#Hp4YJ z6C3U|keAMu6RBNXJXbQNF5!5BqK=Yp^|kO{~5%YT0VptPk%#Rg|~(kDmuV zIn3!1$}Slpo@)gjbK{vzRJ;C9Ra*L^_?eRZl<9d**iwijQEdeFpGw7C^jwhTEw+^w z7||fiX9}LlkvI;-m_ktPJhrp#%fb7DN`Q%oZXKzeRs)r@vtD3)%>x3SqOk zB1VyB6xN0>Vlaai4hyv|KZ)ux!(i24mY`H^=&5->x4eUH7>qURcdo!O0NK{aA2EcC zQ2Se0o(z9g7Q^6iE_YmXq7fPnuVcs8XZZSXl&lRTYiqOyXhuSlz zyzlcjW4jYGd+#6Ap7RM2xl-J!G~SM4FRGEHok&qhZ)WvFb5RR9O}aq_p(d|tOr5lx zEcVRjbgK`F(?|FR68=rEDmAehGx2J;Bc*5!Q9>daw(+NRKEINsE*_q8$4|V(S%~9` z-E9-jvD8Z)mJU*-;k|$^6q-VOjXVtfowUH&ItHhXkg8kp?eAc>+tGKG;yAyl+qkC& z(;+iRiWEfDjaCkg>d&B4A{wDJ2PCw$prKF)Y^&~4PzNYi2BU;E#H|*#74M^Akq8>h z$v7k+l=2UcRUv1i?=|meyTjGRUPHZM$AF#9w^t`M+&Ag0I6^low;Rg3lsfxpoUVdK zM;(sN5KwMNwx!p5S`Et~0Zm^V+LVA66zVBwB(707gfR*kRw5l`oJW|RV#FFbt{^Z% zBW)+iGnk)0J$=>+M=hPr-6|HI4bryX_X6uO=^>@3e@bV((u3ji&sO!H6@x=WH!Sfx zvKEngPGt${4H%R3#YmI=gOd>cq2PMzPZ?`|jlLn$W(>&(1ig?Z}TaIY0cA4nHU)dm}Lo2;G)ZVdj5%Lgt z{GhO}eA<|!gN394|Aw>bxQ;#ZIByVlR7%U*%^yJcp$PpQpZti*| zpJS#X5vD4sN=6T5DL~;J?andaksmFEmVrnmF@v$yikRRAXN$>^1Uc$=jxUDuL`fy< zSHioC?+K$5FX^(M)BkpTN>TG(L|yZoyNr6qHDe0fIMbVxl~T5hv^=mUq&?)cSmCv< zTuHgwm^hXU_Rxb7OVH#W zGu7yk0?`9%*or%~CG?=twOfQjr>|NFHSxu=$FvEpabUsFJk9)Kp;@ zv?hW~31^85A~49G#W|Yhsu56oq*mDj55G{wqQfAe%r^E;P3f~qU}01>VCjha_T7*~n(3#tkoyzb8%IL}=cFnkQ|js`!KkNgf^rne5m(aFk?x$ZZJORA-hvfp z!csWLNlFwWjdS*%AH#&IolgS8H2q`F@mQ>kl>ePdDtm{4y#k$FW41Y&4*Lv>3yf#$ zH13}F0Iwvr;Vv0<+k3@5ib6dZY@ZPNNS2Al{)!~8`jBwKKhhOL^6xP{IbCdKPR!%}$Nf_=G#o1h4)Y3!f#@ zx;HY1gA#^7idi(%a!a~vvBWa6&-K#P3ki{NC%p~xBjt{%LbP&h+h;FBwj>22h=t>( z9b0m^Y2BmAWN`tLADj_?fPP*<>+-cFn)R1?zV8-0Vpk#%J0G#LeT|JsS|OrPX0nx& z2!8D?57)`lsFKOhUf{ya_8Z8l}5DP=?K5DpsDN(?-&G&7`dx=`fX76gq$t#?W_KND}zhltS_CTly*( zUV4bodA58i>TV3w@Y()Cmq4Dp9Tg21zVAr~U+I)r94H_wZFQi{MHXI}qT?@&ppOv2 zIwE(rDty=$DWmv3%J0MQy*=35fBNZ2Tnb~xA~oN~3W+cF*taZ5S}#xBzbRNuPfHlO z>=d5V4sz}r-rTpXP%=(&7n*+fOuDKu=psnis*^(_fa4ruM>ib9b&^lSWOA6cW|GmF zveA-^u@7&?1i*KhQW}4N*4)61PT1lvNnrn>9eFRS zCMvJBX_SnD|F#vRq^ktnPMsHyrnO&hJw_fw61yO!F&0pvi6SRqKg%o|?nyO^dFm~z zQ;OlwhX+fl=V|9jE~jej1My8ds#J`$*n>(Sj^P~~KEy(>!g3IyX<)~fU5^?WGI{)L z(7ZY|1$1*Nr$iAM_+U=v`=>D3JX|v#DWZ{5+0eK+F-V0(jr;(8&evM44taV>-m|tk zieP{O<}r!ky>?;-^14z|hj7b^yI$XrPwDga+9*rcF(^b?Ep^AOR1Opfu_@!F%h9<- z(@__1a=20q%ae2R=&YLIuM<_Imy;0>)R?neN+9uknSYIC4(N_iq5flHkEjuw>Iew6 z+{tf{Mt*!=b>V>6!5E>s;aeeVr3_pTlJFmI&b9m|4Ap?6%zuv0AFo^ZhUa-iG^mLb=FZ)ykXzpT^g3|bm>~U zQQ`E=uo7Htnd&D**Y{GiAjzQ`M*^ zcV$dieOThSq3Q<906F;eRl4Cgru;qz(JV)MN!V3fc3jIV%Q`WzrrSJU^`-iY$YO1( zbv*%lZ0g>xPZV-Y-(A=LY8~oLmHT&4%yfx=k+@_s%Z~cBfI^T5qBUvK1ob`aV$ceB zYO*yEiD9r`CU2~RZdF$gTO2aIM$cL=Yn#{_8DaN%+<9yVbpoRUtGY%$(`5?ogi59m1($UKMg2 z{&CKD>0F&?8wXGP!BfB3r>ze3WR_RYyS4+`KSnW%O<_M>}ABk*}@eq$^n8 zO`f7-IeX^$2+|f=vaB0Y66^m(I#$dllgUeaD#Zkwo`!S|q9{k>y%u5;klb0yOg0^v z#V?X5H9q-;$R|{FmCI5W)1w1#a>4}7beZ6D>WFeevJRV*>G~(v@qPDntLN_;0jIYn z)vgCWHy zdJb#GHI*;~#E%JmL9<0rmIpHOl0h!7td5JNC24LXfok@e)hGLD+2FFXW~1^-xk>9Q z#fC-n^J#qkmRvL!%3OK!Jg|AY{EYKWQ8uu87x*4l<;mAgKn7fIRytLlLe(%>^apk) zV9=^Tp630rRTo_|bx~xCu*2O=at1kcj1y}3sNjO+)s^2***D`%0!>*&MLRCiIx;-F zIa4cX3#6UrwpX@LdqO)s;pzw6-)O{FC3vG5L-t5{4O4Z6T7-1xva{@Rs+VPu1}42R zT-yrOS;&@;l3TSrwUfLfMYnci&{Dr;yds!OZkdl@s+VRI8^uaSuZot&ig2V|Xm)U4 zu*#*aQ-R7n#5fc8$%}eb>=<6c(QA0Hqnby%Pt3i*YxgFfWIl@KW-qfpz|cPCU2`S0 zqfd*Jw9GD>*}Llw*y5PbDz@}B3S}gzh_bxAQqUXYYHf9S@`hNJ<(}w-3CcXADWOor z+yLNoGs$%HSgVLPemuJBND)(enfs(fnC1nIMedkbXX*6ytc}1(16y)Nc28wX<)W>_ zkQPsKW~caU()y4zblp^A?PH~}Rfav11wk&T#he3O6E6>d_&tXeANbr0Dhf4EqODB( zI1qd*hOnsQIbW?<3}Qpy#Z~L}VpNFI5aAnk9mHe41{wwkF^u!S#!q zSk1`aOU;gomoXl+l2>AwJ0rX0^y0EsPWW+}(kj851jfiIPNOrt^D)hQLL?edSk*d7 zOBl0`$Guy$yP$N|t2sO|K4Ge~-je1!Xb!P#6D1k=s>gZW-b9L#7;@kpX@iC2&{pNR5ft=}&Zy z2$Y-yu~dAx#pcMyDb4AA8%z}#eZZKku;IgSE__$oObX+5v$P3C`)MNvXCjNzG~^oz zM{kC-f0Ls{dz4jDI?BzNYgUW&i4atK`nC1NBw=vu@xze*ApOC3sI7{uMAvbkoutISoxl`L%Zz6eTlC0 zit#M3lVNCx!#GRuP#*AYssB2h|M8ad?$=v2KuR6;i5@nRYTIt__V`eXw1W;8DE1&9 zD=;!dHpC=s2iw@D>+}0VDy0l)TOnrdcvj#0ftOPB_DqmU1PM0=m)?_>5 zk~t<(*3oFqlpTb|?R+y!mwb3R`QUh;h7F1AK{outB?R$RBS>?!b2wGdoLV-n#eEZQ zOGPFbDFZ1)CMQ)qW5qj$qJ)}{CPJWA989+ZiV(C_&qa8${lzPZwF9$#^%ay1O1@`? ztQ4hpgH}SdNCOsY%~2A;AnO^cnQP>04nPe(_ppfGTfmSF&Ssg-M?jzRO}CBLz}!iZyCJfNEIR5m}2rA}Zq6Iycx_LGMu9 ze{`k)R_Y+bMLeU)$Kalyx(^$tv&fA}0NUYaZY6gj)XgP<@t^F|ZtSm&F8(DU0$!Ma zr1@p?jxfEQBbBs0{8gf8rzeww=hX+TIoI}s+-lL(4E~1WQtpR*Jt~nz7BOFwpfA6A ztWuYATs2CWYH87WYuBa0z)XRZRFty(!S8Bk8^zK`!H0% z0ib>XA90_eszdZ6>0v(R>5FLAPY4K5IQNK8(`C(4DttJtN(UUIRv2&ns$f%UFRCj} zzkh{)u|j84uDQhit)}fM10TKrOo*f|$Drn4+~2LMy2)-(|HI#FNC3%NqL`f%^o#*? zG~ET7eWj8CnuTzzJ~!_$A>cD=F`vDeWQq%Zw)+bwB&Wrw&Vw}gXc;9u-51+MM~7ZD ziBv5&CbSz)&!^56_99@SOaET^Vab+8bkhPqM61TdV+cq_pXfNKNriXGYF4cC`R~{* zE$nQe{Iu9Di(I#EGI42@*|_I(hEhI%_HGg@U{Zmu(!8&5bnQqA7BD*eU38;X+)Img z7j%`R?FR1@$F5|2$_+@%Vj&A;Ykd=|s&fNFIOB|PNMeVP7X4u@kaW?84pWxrZxz)x z1iX5z9V#R+52i%_@|A+5CwA%y3YZKo!dl&FG={3SXOwBv<11W@m!EooJ#1=*^bt&G z!V1V#7Ua)#nvc6UbLTML3yaSkSK=h7^3Vkat0a%Zri}Npahijy=>_U-gT4sr5_=## zBXmwk4?52!6kib(i*ZTm6=kaKKy>1Hh(!xb+pOFlZOPv!d`Ru)Ns!vvs!gWly z8OCGVc$CEBRn|7%0*V1~P%kZ7S?PW(n`Nbwu26pF+vL+22Jb!(5zK^CYNf)qsJN5% zQ2fFZ%zC}y4p#`Ve*7ViTn*`E9;b=z0~>waNJTP3OC$&9qkJNk6Z#nhmo~PyJgw}M zA++rv%cUT?CE(rhgXBjL<>;(DZ{~z2jFk6yoibwd=c}nLdFWiimX6_)^trD|7X4U$ z7Rh}Y3e$y|nP2KBGTBBc!p1ZnE#=ExD7;D_EaD!|OJE?kq3t3aF7QE+N(LQWt;)Nm z7`}hNCt0mkV=!MIo8;vindFEEz0VV`7ugS-VpBl(BR?!T3e{^*(c+R=vt7m(wFtR` zfb8-a(6|xKQ^|5E{bte4+p#2#xVl~BpzMMGM{YsN&D+0&BP>E1l(@4a_#kuUwSuS4 zcs$Q%teziNvW>~X@rk0{;k!62KO!h~yb z_2rkFj~Zy8V(W_){7*#`z;%g|ZE}I9Z?+%Yd#|iANpKfdNpm=6eh&#qH;@< z*YgyM%7qzkq9oKFSEy`$TF8F*O=lWpARUL@mD`M{LmwJ-_Am(^PT;E<5HXioqXG%ghk>!iG(G$J_!5%@pvj*%=3cjy3EGhFJ0-- z{lzm$Dd8G;u8^HCO%h5P9q8LLL{ItYuAW}ztc$BoDL*$YP1VG0W>c)UsmXVN95_^q zgNCvQqfKf#UkTKws9NRjW(=H!*%+nt!_72BOPCU_@9{#epL=~`7vG(Db6&TzR_ zjd@uTeOvRnK@W{hETDWe!2W@4)F@VE2-CjcPTirynRmmAC+i!VOo*ll^t+Ek5XB>X zk5^Qi-2UxJS6-~RhUq{e;r8BG#ek*iXLS?LDg*MXL z+Dq=jSF?6~Z)4=?Ro~i(zDB@aTe&3(IJw{pJt1>C^DPEoR;;bpz1G&d_@k);GB4Lg z6GWwl0bXC7B-5ZLp1ajyYm}XsmBct9>%4Lxi*{|#mL|^>DIUfFxIrdvy^06misS zd#SvkdF3829%!REygKK)aCN|)2JrJW-*y8X4zcf$Cp|38gs%-goV=1Jx3TMS%$JSR z4$ouds~2Wpjt;#E2#T&~B=@Xc^CVLLx<9Yn7}P%QOhbgDBxU@`e}KMN;M7JCuY_)d z{O~l#M%hB~ zaoOrzwZ@l!sI!NBALG*ODR$~|i2Hwmphlz2$l_7@`zE6+TbKfc`)~USQ5s)hcJ}(A z8IkdVBTy#~Lt`*BNC)KkD&Y@hewXxkojGP;l!{(&xbC?Im)+`)?H@iOE~qY2t1E!| z8vIfB06TF_-mLQ4@2hvj-Z{$v%L4|3KIPu%1;G&>SnN20F5i=m#1PPU*9v z^X$uVa@4&a&=TBYAr5)kBP%QnK3Ho=aHlr#{EGTSe1OYzUF4EUF}sVyhA7(O_7@Bb z@O;T$l%=m?`ZUZf`i-FsuYc(*nMZI6Z)8X?tzEQ0KlKQo)E0_FH4HGB7_5v`+P%XR z&qMJ{VfhSsQs?&!jIgoy(}XF%$4A7y%CI%(IZ_*aBaAa^6JlTamg7 zJl3%>y~Gx7yL)E1I309#bHf35S>IhG@rG6Nw>}HgyZrkDL5*Ih0$sbR)f}}|;Yh{y zj)p4v06g7~T+vi*e-*dlThpKy7CzF#3j)4e1jTyBXL$+`zetwloMrw#_15Vz5_)z< z{U4Q`Ejo6vZQB`9gRwdC6VINed*(iCt;1AT&_wqReRW=UKUfyqw@kDr$5;AyN7Q?X zKgB)?x_=vIY4E9G5+eMr+a|-3ewZWrrrnS4ggu0BW>QT^*RhIp4q-zrkbbqcE5_dk z^Yv`#Gf?m)yn(pPl3v5`nY?6W&e2R0m{rY5|6#$H#_S>MOxwf1Oz5=%^tyfM=_u|I z$SGr%vdTV5PBe)70$ztsz885Mjq|c>^^y!{eN#elNM|LdaNbM=t?zIzUFZ8NE4!Q@ zsp}!_E6KHD`Xk6%$AF)9iUE@+k(0oVsUOPf?E8X!I{My`_Mxc`3ZI-&-MOK8O{VlB z>qF>Xy3Za3W>lc&lho28b?c{`pfsYkjj~BgS+Lky6!giY$G9OjOKoJ8Wer20*eG?O zu|?;{llQqw%CsNBXLtbg*MtaIKB}gjAGS0@rl#X;| z({a%N_hJM6!Hk^hJaNrU_WfF5x7o67Lyh4p(u}}_o4l!-NNQy_3yV!v@9e}2vlRHZ z|Q&ZhUpto)w4S7xB6 z+$61aWDYx$#3f@fQ%JbAwv?Rl6kavLT)~aZcui;F2z}x5mfhHd2dS2xA@Vp+mgJhi z3@Mc2u@xc2u&kR-$GlB$KNDIp#B?3z7*3 zY3~Qr6MLLCxX|SV-L6`oJiY@BZp!$?O0hDQBHN&-R<`rPUuZ{Flk<#7hsU(Dg^zF` zTm*eeyD4#$qmkuC^2MO{0nLNjjaRdAAm*dB9R-~9k$aJ50#Ro3-)%$Z&5hs5wpk`Y znR#aM%W#W=Wv*BT<2Ww7pQkMRM0Bqr*Tw18q}7|{r@Wt}UyK7Mwazg?;|$&FB(9fr z-&d|#45gp)Ihe-)aU3hR{?TCdD59P{9nnjQ|_xWk;Z^>>9xsF zDZZ0Ctaav}Vm_V-Y3vLuTR#~4(4}afcc`6ZcWO~lr8&|(t`N9bGP$u&`!Z@&d#t&6 zBNx^AIMS84A-8&_pWFv5Zedwxy)!0y)4J7zr{aXmBfZChN9(>oK9ORhq_!eoN-u6r zlhQ#S=V}#-)U(+HPjV@_PXM_C8`TG%Qr|&|dy-F<=@O%wBIG~691tD|=T&y^BRA^^ zZ*p_}LG{9{Lne`QDvh+`J4m73X%Q~newPZ-v0kSh)u1281GrTJxTqs=)pTfwvwk61 z;KQjm^vrLC?WzJ5HUJX-Gs?_?i&$If>?(FkUjGWC&C=L*)BFee^<*!-a@RdCw_-(u z18rlNEn5}&{F5d`nKYfa9pD!su0I9l?G6YPrN#rBGYxKKU%bp3GGv%fezVMF@Tp)w z9h4YyhE_=^XX&gf7QJK-+0ro3C!UCsp<#V`IO&F7~6JA5-zGUN?YZ6hAs_+g^8Xw(o{?rDloT zc_)*_X-|C92;l&qvNq}{Yg!FsB4dcTf8Vzx#IWOl_EVk5Ppc6|$nq82rIxI2YQmc{h{;_e>vrTrn(bP;`be!D!n z3b6?VBE-cIQBkmX>-8%dtxh+XCjc)441=@Q669fIdi(Uu8PX+Hbg^h9j(-X0=Nr>y zAluhWFBdY1D44sNgMER(`K#>g5W?^W-n84TfL9AhgEuz-Iq?BLLdCCkc{58lIK;&{ zCSicV?SOEDmXb2Y8bf-z3$vtSe>L-5Ss7@=uzgb!2kI`J`!5%canRxa5gQ6|RE0Py z!AF)1kNpASP-i|_$6exoyAT^e_5;cB0Smxa*gy2o&qlwUVm}VNEubwQ7y$gzVZiHO z<{aq`DH|gy@%W!;`+qwJjDY{W{$b?&%LQev{3j-ENer zt5+sAm`5#8SWA&ga!)oLEK_ja21cR7D2`X%l4$g(eQ@^5lm79%{ftQr)fXac zHB(@-bbiGBR%+&1L;XihPJpT;%d2pA&EJ+4{OFC^y@j!O(E^F_vs>I86RZ0(a^OdR z!HXe&^Ch`9V>G#n81ax(aRH}#EMpJ(x+J5D+Z0sUPDkSB`kd;Q>Txf9n{1Iyn->L% z2VsH8u>4!AxEbsCEogl3dmFq%1bgh;^Hhbvxe~eNpniP;ofS2}jjeli@XpYlJ1?nP z%F-CX{Q%KU0usuydmq-~jgJ<8%de*xUvAHj0?FwcDoo%Q@v7Ywg+uOZt`?db&$hyT zO(jQ`F!mO=sQ{!Ma4fOd;C0NHhd)4BS;6DKJmXUKS&jSpK6W1U$T!xw`fuS~$ed7# z6F)ZujrwRSTdlzI7m7Qw5V#;#(+JN!$5zZm|7kcBn+IcFQg$7YrQZW@F4My$iG zTF?tDKsd{L<1iCU*_|FOWlnU@wo?WAm*>p}(Q&>Fk;A`nrdR!IE_GBTmbdkAM6_$> zI4D^NSUBq@X7+M45cQ2;+|9dvv|Kx|Y4S)qVp^-8LAut4gu(Ad@wb!um9qnQ(anlK zSay-S7GzQNiQ5cfa^pz#X;Mn7jTR$-YQ{G<`Z^?Hj%Y}pFg!Bs=TY}LlenG6o{XHY z(v8^Tp#2h%(N>*EuEWumSX$49=FhW(#+n%S}eqWoWduwvrLkbH^Ap!V|i5`q98a)q5C|vy%lb zvhreZaKxPudvEa{c^UFw(@p9iKZ-nIi@jNU(zfF!xz=F7|9I@D8S03^V#qisCxV3Z znHa90*-SH`U#E#x#6PAuvL4sPo^!Ik1cs>fN6CyPxH!jFKthlega!QlEziCKA~Tk7;H)>C8|?gqWo2i= zfS#W8^ixEz0xg;>7X}W?Q$XG!EFzYMsJblUfQ-x|$4B=2o;wgj>TlB+W+A|0IlhN)Q^9x5Kkch`vCwy9)Uh?*!}x?{~1dNf7u1p#Nt}U?%yWjmtrItRbJo0n%pw+W)t! z|GUBe?n;f+2E#w1Fh~?$O1F(8VYcS{R=qcA*1Tl1%Leq6GS0utI)2h@m2ZOs@K?({ zv?p=|@y|rZf4Q4t0V1Wq4+h@ES_|*uC*q$U@q`XSeFs={#2+OIO$~^PfTe71-_P(VykAYtiqOQocz3qiZn5em>Py*d&NIR-H@C;|k~>zea-}C% zeKz#Asox6Pr%};;xp1>wSx%iXrgf@uzku{6f6nmfaxMQz^0ORGNtx^1enQV)+$+;h zPV2)mjZ!p%le;}B18I}P%Hzb#so@j17Y9#SsZ!7c@o9%ZL7rFeSTb-}YA+HMn0>{C z9_Cqtj8`68CyvguL# zLEl8a75;IHW3n1x#WH8BFK5aXV3dBzS0Fo%G89S4705T>XJ~{K=OHbzM?pG~wg(hX z8B;+dPL-nkfiojSk!@17z_`nN64W4KyRJ6E_>BYods;2Zg*EwJ(a9o2ltjzu@l$b3 z#oLHgW}_bmECY=~YuE$HKXbFQ zMhveUiviU0udDO-*CL^K-7=P#R|jTw2F{#w zk9^WTTIB`lui0B5RG?;G;q5jqV5bshqFS(06)~M}nyn?%oAKXK)K4?McWGbn5OZeQ zwE{k6lOc~y+A8r;_Hns8jP8J~bB4g$1i`l-b2p_!ioo8qC!+?Fp=gq7_}_t^?UuBD>`VqSd(ea;rd0n{R22n<@zR`zyj*_ zTHHrSUY;R^xC&eJ3mhFC!bdeH580gN{oGW53GMX337S=T=9C3;m~KjHN*dIVxr$`) z>Id19S`Vgt0#mW2YtjU5t;0s-lfyo)pETA#8v4HJduRCDy286CdVclbV&_1zjsqQ^ zgKsZyJp-7@?Sls;f)!C+KvM2v(3Nxr5I6b|5JSTOd3W2hKi8$VhmQ6OZ*eJyG!ztw zM)CvKqCn_sIlxwzpaZG3+O|8d6W#)7AOOc*A?h-~^4SG!wU8d~?0cSwF3A0E4Ll(5OfX8Wx28Jpm~8BLvh93MJo) z9z8s&EmZ%$;Z{wa4zcOaPiqvG$cde0Y+1U@zpGGzj44W|T_{J(%10U%_L9!Lbs z>yjJ-nJ#*c`*Mfln!CovxMWF}+q+^6N{YGzU@^++>2N)Y4?q7&v;c&^W5j;`4?r~n z*cJ4HC%zecv%~EIw4T?1`DTJU4O@K_yQg8MUK+9VhLDv8cmM@QfNu@Ok_*XVE`6zC@U_JOJ;Q;VU zEr$O@4*syX9_h6O+-d)_-|4dc{0lQc6uWt=$2)iZJB=w;48c~CvuvFw^`{nd3n2cy zRCxbPN;2be0iw70%Tv*#zfSEHS;vjoJ;H$~y6qGfaDltu%eL~PB!$-j1h>rlSiB9G zV?`;!>t%#wN*`Ogg}OG{kHI?KEVn}qu(YsxVrm?SGECBc7Pn{0r zGZujWC@G!o079O}FYP(kJ-96ZH6qIY3^*;UA1uP(8)pCyx&L@T42N0m!{6BkBLRBS zqDLN_yp5rb-b^*yHrL@S&0}2cpOXQAH(D#O;>0c${`Dw+u0w-v# zXyp=eNM7ub-$4)+F2%YA5$$KL4!jddZvhP3N%y=g`7@k#uC3G+Qtd2kExxD&DDgIp z(yF{3nENF_O5dFs{_)mG-lEZUs_PG(7LP+;j8N`B2GB`uj8}X$z%C&mp~9^d-CKRk z_qlRLXx|F1gz)`9-qh;-JyJsLbpi49_mk$BhQyj1E92Z`heE~e>~(LXsy@2s+g9#; zb-wOcE{tiD_?7Z!roDxH`YCh2ibCLQ?U8Qa#c~eiQ4h~AxZR?G-;e=~% zN`CqPF6Xu@^#PyMR^W@KjwizasP&&)B0@!3jVN)0@J`0cFN=Hy{m__qYpHgBV^G~0 zy!V`5862Im8X9+DW(!a~k%v`@s{}{Or~@|!$MBtaM;9XPW;!e*m>=UHLo;R^4ERa} zIg@3rEkttHoYlrI^XFXbMf7t9i!r=yGNuu402OxYzKA<%Gdik1rQ_;(tLPHFY!`6h z>_*S~f>Ui_1=HM`)_3#*q@phWLE1Cf%idB6Jy93u&`v6`qdRTYF6>!+8dvjO$IK|i6`VI$bNtd+Z*DOS>fT|1k$_Is;3$S zktU3taH3o8ygZ6spMdsWd52u$pGqUZ((k|0gDV}{gdOfIy`li%WdUK3BhEnf9ZW-X zlqtsQIR2@*hKZ_Ba5az6SC)td$~)g@u-4FOaavy|cB`hl+hi-mlNq3*drvtcI(6e4Wm&pnFx9 z8yr9R=iTqmcVCme%Iz-=;k7a_ek%X&ygt0-S9NSwF1x!y3)FujfzAwo6B+D9_h$(( zh*}?Y$i%U*Y68t4)Y7p9295iMtuF+qkmj3vivONFCfL4 zmR~=s^dY05j=;A5RwrIHyhqDykJ;wJnGclA(y09XbzaqyTqk(+u}eZ~7xz*0K5be8;F%jk=BQgWxOHL?@zab|W$46Sgz4Bfg{T;#Vbu+5b$krrAfD>`n) z8Kyauf+X5fX}U}F@+aJF6I0p$;m*-3-xp2tX(!!0#6Msk8e;$9i-wI9Qt3D4qhWFu z%0WM&HUmKzEoVQ(8v-*D?H-G_*Mu7(_N{M5ZUS{Rx3BSBNiAA$bEMMYKoUM*6QM>7fssQ{Z%jPPUATDMxeVXh%hQs0XX_fS# zNrAOksU~i)K}U9u7*)91kenKcwh3viZ4zFq0m(39+{*)jiRy7Hs{E!Y6u(%xL6SA@# zRQOEOS|8F<%(Wg(Gm*xtW5cBfr|hbpGd5q+I!SC`Mm0=bxGQ(As+hc$HZ3Rv<-$Y{ z|J|g~D_I!-Oh5Wz>lio*K$X(TEtZ_-V+j?#{9rN9EN``}=K zI8`oqF$fl|4dFH5xm_^NnxC3#0e)|&cN?($5V9bX3?$dM(6ngTJJ-;hC1WYvi+L6f zb9|3z5rX_yFT&^R(8R~Z6MUu}zxd+RL!9H;2jE_n_gYYn%7KJjl)Q%=GWWN`y4gl3 zHRR6cynH(wHt)T#?yw&&P^G;rt;L=e%}X8vffAmGks@fn9w!5L=RLdD?MDD2ux`05 z4fK~cK@rkIJAbe726;7;RBYNW5o;NSvI=)q+^SUN~3#Y8PQ0S6+8qPq3rXDuMmqx@9V?K>T zY1OUd?t7Qcna?Dvn#mn>sIm-39TZH9ZE(mQik zi>5GOH*iBeG%#cW0VzS^m!Wr;?0?7jSbaBY{nhZY|n^ z3s;3Ro)(ucpZ%%TQEPSQ1eloSk*$CD%Clo7;;+V0cQ?HS?Rrk?YmZM2w{3OH*66-4 zh88Bd;XAbPr`f^t-u2~E?Z1p_rFtfFt+`P`+Q#<$UR=>Zz8Nr1iyF89hJ`m zHI8!^7AE{b_p4V$Y7Rr`HqfTv)lW9_2YUb-|5d3GwBVJMh*SuNHR6w4j-6 zkLh31xbQi6@W55MkVx?;e24gKoU-9(!|It(L7l!w(I*V8wTkAm*xL7@B2JfWO*H?| zw|yS0+=V}2#Ln?-u5l?LL?(8)5h}H#2`|9_q$;1kQAemzHhq!UwUn}W3OrNeV7}7x z3Jj(9X7KJQ3#QZ% z_$1!5w;N#MB^h_RX^Pu2t3lxgs1m-!4wWY`pBUi&M*F5kn39rSY> zMYSo#=0p;X01&5x43%7^j@EJRy-MW^))QH&bBSB9@z>*LN#o05YHqx5l-zyUv#nG{#(at)=oH02cMW3c}OIy$bNF}3kccir(=*pf*qmY)sF~rorzLp;!!DH? zhtr`Me64YRCUYbTZ(l-dy-pqzA|_fRZ;-zTf%af^V$v32;DvX~E2S-C8-rjZ{fA$0 zsac^)%)fl-$xpPD{znSbLS3@lZc*%>k2a^Y$l%yadsHgR>MyH04#jNHV~U>t%8fEW z8Zr|R!DT3P-u~p#T${B$S;qy+J@T}=PB)~|wahnX71DCwrsZdeT;{z!e$o6k3l!Ew zOxMNr3xE6@01%57fD3l(F8n?&ZaNQu%&iN1g!Du$k2=7YmaG(fBiKVp7;lu9N(6Sd z6P@NQ*&UO{MgKVPBbEiZ3Z#?F>rN1$wa_Mnaynq-+^vk|Wc`j#5$_`)yY;G$|M{K$ zg3;9bSmx1pxR=$H-4D!qW#oLKu}`WlXG}_jElZ!m7)MXhh~oIWcL7ybv1PCB`0o8> z)n-@zE*u}>No)vN``=c`aeQ}+Otn7BB*{2laYD$2|8(>>J~YlNSYKTQh_ z#BT@c?q-no#*hk`(OGR7c9mgqiW4bG;Sl7r7nNPtEZGs&z}}rVLhwu{mAC8O<;Gm&@qATd#(vp-MA zVUxLVP8H-?E*mEZ#lrmUIjQ7Z*@2;?P<(8jMV6W!6Q!65kk0WxsK-hA#1pNHn;Ytx z(eNK|`Ct>K^k*x=Zve3|cXn`r&w>YeoRYQR4?kVe!Rma*pXunEP|DiaSy@|_$c8kb z9}`5C+bP*?^T(1T(l2yc>H>`F2#E3WuLxRh3CFKLk}b_LQ)bV2sxOG5iJTJZ``z{$ zrialTK9HbKQjgNNJ;3;sS-1T+0E zpe^m7T}>wKuxYo`a9rRq(Sb*L7IF~tsQJQwTqMy9Q8 zO?0>O>9ZQ>N$8h^ui{D8^QK(4SsOuWdvD*V=08fL=gGY4$y zF687n>|bfWECDp@HbdGWzy);P2H7he+KWKoFN_(US7MCG#oa4=NF}4ifeew(4HvGa z3Brv!Az5foK=C8O5Xb>i_o*y40Fow7y_boXgfer|d#n~^l{2t4HxR8H zlOLL?_C1LYq+`!UUBbbpDqA$5a!nQs6@Pg6k;y~A`)E;Yc}fTG70XPtJUZ z!RMH2RUk?e(KIr|YvfaBKc4;NTo|c%jTNaIUaanI7-OPRHuacBQ4nOalh!3$J^loT z<+hk5!=7ePCGDDuFx}S=Y=!@kUwEKAq&uVVMbm)sUba(4KB<2W^((pZ0+3PSlzwUW zZuS8GjIzYOeN((jr)8~3(1Yz&CN}2f+SYByY|*DN8~Jgd=l0mzl3Av>=W%{j$_)k| z*#cOAP~e-rWwJx>B85Qld_frxR-PFjwV0f-t0PrD@vTZngKsc3%EwYa+y-*PO!Z=h zHX`tWsrIjRjb42|)P~G0#@c*s4p(HX1-GSWa@(M}QWAbVJ-+n^_RHw|;do2aqT1nU z2utZR?dZK01Q(FQ!@%|k-dnw8^-Afm86vsWlh-&bSVDMZd(J7FsdreNSFW80J|<^> z90oom6+jfFajtcnkOutmxOLl_HzL+|8ez~v=F85=g$qNiF@mXxCURTLT_!qB;dI#3*&`Y@FaqLbV zU&>Kpi;7jdv(qCLdE`vKhNE&B$brGDE-mrRrHgdQan2h;v@s$m~O7L@5~?PW|Q-mm~0eJQV!( z#$+9jHU9U~&ky*5MJH*|-efuV^EV)H2uBY{)Th%PYlNW=uz9L@ zj_7t)N6QpV_Qo2$FIHKq1&D&GEbuO07o*Xm*#)Uo(QJ>ial#l8&2&WqfG_4wrwB+*95u@!1n^&f`a#!YUdn;?HnsMYvIh znxCS3Y1HU{VTGc>K(?sv=8WUY-L0W)&nz6c{Z{GjV{c+CYd7aRyRAq&;&SW=>p~e3 z34yFFi!ZOlUqlI&HEv%-0H|?00|W!(9}7WqaQ_i!+7*437Cp)JvfMiDBC|WsTW3tQJ^r zJSPzRzzSZ-^w+M>?U2U1x@LiVi?MB9VP^v%rfbdpMF-%l{5=2G!_c$F%*m9{4k?&d z$7t_T(MvSWM7mor%y737=qeF0DMc`(Nq%+;T1MmuV(JAvc7{0AU^Xuv7rkS>3uX1* zLuD+9-*&DCr2{x2w(>?2@2u|~`xd~$sRU`kfA=m%8d(0p81Y3YpV*ofHukZQ=nZgz z3-RNo$#T$5Tph}VVu%yk0*U!wDVD}$QtY(l`$1?^)4^k8}8SWE}x-I}1$c?T& zhzUit_s%g{cZ!>O$(N-_Dc;5D+LaZKK!^AU@<}2LYRC%Dg%;MLVLu4hq3>;-6|y_~ zGRDMap?sJX6BIuoYx)tAULL(`9(K+6H;2BqPRMoRX*k8pZ%OzZLYrx?tq zEG3qawl57LG7n`cXAv%UM|zom5fc8AzvZyYg}&0yb?`}x>{gqmgY6MM)Am<-KU{Rc z@Yv{WBsoR>aBq)k(|EKrSq#4bHKyzLcf<#(&HavXDqg`P#fUxEZ-%eUOuJo2>)zUx z?^LIhk&@;NAOQ122kzjzaP(YB3@uW|ZraU~kj1ERpqA-AY(&x_o}NCv3Y$k{Mo1N0 zpod*oX4NLta`lgY7?FZuGoI*4iZrs20IjJGCs~6SUTZta=>M~Dyta5gwAoYvPia|Q zd?-1CP>D_K-6Y)&3~!ge==w}xX2a*sWz08HM0o;oC+g7Xz`wVe9e@)OW_B+#xf<`L z+%xvqD(JaS3-d=G@s#yQWPIk1e%%iN+YER9;*QXKfIgafbr>$|3Sb5zG()7yK(nPU zt0OFp{U6u%9nD=t`C!-GD>~w3_EhBeDk)^YOj6(2!75h&oaQe0{CpS22rNW7l_XzT z`DVk}x&3=&bV?FbiM&jdC>CV{@3+Sh+AU48;uP-#Nq<^Xn^7yHoC<(Vo{709 zxlma#sbI;a|3lSRhSd=)+s@v&yGw9)cXxMpceel`8w(bKySuxS5Fog_LvRfcf`8|STTD4X&{b^x_(m8laR&Q2y9KqW9Oeb86vTt5~ZvGnPd#M4d z1|6<)Ni%%ERw=EkqAroIUJz8%`d)V&?2_a=$@y_)?yA(n)6`EBANQQ^#y)*Xxvt#X z4bOZr7r4`E*UqL@B^F(xxQWAkG(*uliiiZvKe0_DoW)supTe^w&Bo0gAB_P6A2%gk~W#f zoHxC-G%ZTqcV%EUUhBe3_uka?jRao0Q(1QEFAcVlk8rN9iCeX(jY@~1Hp-47gbM$Tx__tfiFzuQ1l+wa!1JPC{j2<0^kRc~1s|uC{Tqhs6RtdVO z=tbIWKP_M(v^>#{o1jt}kL=cQacAv`WWSXrmMVf6NI=7lU6mpp(#41lDC}#r}yba<+SZ(+CA)e7R6Off^-ZAtGxm&{9zRhWg#wZ*%vyG!WMb%X= zB{fW*f+H4tathf_EiQ$h?0sb8o{(ySI7fPqs{;sK@3YHmG)j{^O1B#GgruCYNBr;1 z+ZI^QP<#*{OzlCb8cGRF^aR#W-XXwbvaHdCvWg-gRB!e!V z8f*uFdqV11JRY?@BeJwHWzR*_STfmQ*vKSy;eXT=DP=}V(%pNsgUxWR89D<<*cR!@ zWyGG5zw_=;3?a~`WYqn^SnXbxRu-xy@ANbu(5Is}V|i7o zA88iGpBYz3y(=1x8KGDUO|gPdJu|f3{QpztPdrEqNHnE_z4N;fen$GldFAAm?N}7q-K< z#?Qe&7QYaS+CT8@{6}r>!=V9+?!tO#v!&a2tDO0{@IRVxbFz9il|t3TJ@^aId7tNp zUwd^Kmdn~8+DKd%5d}z72XxbxQ8ZPbS;=RI%x^VRm{)6Yw@B1afN!R_S_Em4=r! zzUq>l5t%acTatqmo;O&%C%%5D$-&iC_Ddh%t}oOiDVI)4pctugo{XVMB4iq-?6S)} zwW5@e*H1iJv9vrjLgQ^WIPbNxu(z%mao`S3p>Z$KuxYhjz3hHUZ7(oOf8`<5TU`fN zHB-%zBTstiOcz*DNho0gM&mM6uy8KzRQ3qJ6i-yH^1P;0+UZuBT>cyNFCBUst8WoM z6e_sd0D5c-Q+*16?3F+(9+l{54VuFn2Z_a3{K(_lu1`DBD-mJ6j z^CVSdQOu552QBhO(m>zm#+~lRkL=93;aSQ`DkAIwb&n~N65ciyCBbSWZ%XcBrd&lH zx9&O>*wqj%G#r-?ZQkW7cbuNeR^Ov5ehK`TGV+sTq+|?E)5LAf(%TsJafm>7?;Azp zx8v&Dlb6?Jp}iH%YYel&va|q^r-cd0?@D-5=BFEk-ikXuCrW#*26E5Ztu{wmjfj?R zLwV#q;Zy0m5nQ9fra{w3U)e6Am<vAX#HO;!wq9)u&D@3zbwK9oh8e zpvU?W3fqY?5@jsPOnP1ktV&KD{D9Ak6vEAExz@u6RNd5gY!B{a!Pp{8(mB)?s-f_VFwUhXHld7vSNpS z&|uIp?46o15zfV{$$E;wkxFkgNbacKf`<-^D83G!0gT!=PAut~0U9*ibKu%nIVuBxN=-%CbSw+K zr$#^AG9<@ zKG|Y(^Sax=^)V(3r}@;cP&DVcMv*JiF@;&P;@EvFiijbq)HHSamkysrG9BZb5K5b^ zvE)*sfkr`u>6!iH8|{3>B7qWGezKVvW_Y%@gq&Sg@=*rSbt;a0)N_qXrsV6w<&_f| zLG4_Hs>H3UQI;1aG6RKlW>O}P4ixd7xx`5pZX+Dh_9Jc5n!F;tRbG|*yp z?Kk)yUygKNAqs3z*3?HIF0wt=H9eagQM#U`LFnVfvT~NQ6eHBW9NaNYzSC))_*jpG z0wn;H#fOCo8jWy+Ybgx)Yz>ZOPCR8lz#66Wt1WQn8t%y-(3!F`n}SW+5+uD){v%fg zg@DtYPmWm>aKKH0j5M+wAZ8cTesw+Wv(S1A}=i$pYlq_CVl_9e7mL{D8 zRXajTl5<=dajH!c%@aOIK8=Z_!y2iqT#dIRMEW56ft}6?0#xnElj}Y05C^@~ph!G8 zp>5eX^!2JCk~aK&++4@f&hit#)sge!uRC{9*d+v2KE;EI%KBdzW{uD#ju9#Qv`&2h zMU5fNSw_YNnrXD<0ZWuj+sCjAxM8=TaM!qUs`ub8D=q?sL>uE_l`dKmTzD)|sh;hv z{AVa4xge@|%nL&m)YZIL5&EW->Xxxyea}(sP&%d>b{)M*1dF6)0zE|w|Mhlz5;skp zp>$9=`MXfT2_+WphA-CG4rrqtIOLTyUfms%V4>-DP}r zVw>FzZ$X`(%ujole82PoE-Q}JtjS5m*w25{Q+_J}w06+Ks$&=dmsd;2CD+>~Kz)Rr zjHrXPKjzh=DZ<*EdT$}<^f4>COgb=|ld4CpRKHk#9zyE0(Ul%@po7g%?R>aCnOO}h z7Yi$fx-IokwTNgOxtD?RQSj&JNnK#HaH%Skbq^UDY+#+&HNj*dpTmA{LQyXRuH`;% zON6GSu0K*2pkrCoQPji`tr%hQP+%J>usYujlt>t3vT#C4od2aR*egL)dnG*v5Dcbe z;3Pd)j?$GHbJO_-qfJya12LD4LhRsKzh~Prp)}Ij(cqp@D*-6SDm|^Tv>c7c6>(tJ z7wKh~RMw-AOr{q0Cy-Np-AN5j@njJ~UqBOyg?OoKYxt<1UtLe8Th@*u@?G51;@X$~ z04XY$R>W~+oy3g*#*`@hma+)Wl13BNg@i5aSm!cFvpEMx0di<|Ujez4d9z8C564pC zJYEcd;&6YUY8G6T-ot5JM9ie1enrm|k1n^TXB}K5v$*=edBS52#}d!op7@31$eBq-r#qO);V(4Xx%Ax^{yu)$CS`6#5tIMr`2+zY*?JJgu7L7YrF_s z&y|wCqS(*i!Q(IqnK%e#e$ILmcQWnIBv{EpMe2ag{Xw~UmgFr~nT7=2R&ALgn^mOT zIs;#hSEWdzz>?a`CS4GuN(Jaa#DSU_&tFFMB=b~MXlX$ zlx{48OCHrm^LkERQu?^R(96f7sr5b6=)5a+!5B_|YI0dlf<&x1>%-p8VlH68c#3P9 zK1V1hI{S`B05>@9l#<^GSA=|EWT{X)GOklLBY7~!btHadCe9r>C0oH(QJ<2+GDYl8 zX3jH0prbMf7bvP$GKzFjDkqC*8wk2CX4$cTF<%(32~zD3DCqWi z%Cq;plOpaV$Kj6_-YL1@A{@qKRp3VIcX`um;m}rSln~ z#RgPwDHd>J)jeqNYkR7%B2Sc<@N;ETYaM>ZS64>$a~is*+R0Q7zc!rK@+I&F}@Q~DM}@eM_|#44hiY9!jTBwC@~{a7$Eg~?V{tIWfu0XVLu|AW!5CR;FhEe zwmGin(%%;(e!?Y(6RLs}&@>rDif_Fj$^ZS={%6B}opNl;ub@PWP>OzDr2egN_yjDI zu+PvRX$njl0}p?Hd-%LiPK(W-S(aRsy-{y(I3E7wd3&xb)b>0G{W*D8`X}LKE$G6m zr@N-6`bTy3CGXm`NO<11UXBooHk`dz&O>xNX=3pdMc#O^H`Dhj-2B|_tv0EtF(Yc3 z{HQnOS{9{w?XRPq@J`<8k1R}F%7>Fxv7Qll2&BAkSBhl_Cx>dke?ql+s!K`tpq1x5#(Q*>-VH>XZ`EA^huwLcsUgg$2;n z6R=b7Aq<$Y?oBco+kM>7Hoa{ofnEmve;^PL)C*r#A>j{ff;|4yGyrT;LvP`MtGD|E zYX@5BvJ7U&c0x@YuDbxd_1{ANz?VNDj|2dEm9joTj}P>K(Dwrd5(L{R4-p0~flinm z^#7h@phJRAnRVEq4@Cf+tLl*byXgOF*#JD$L!repelN*FPoOP98#MDfWt@G%to>I* zJ^?g&gWeFKj|2p`^Ix7pi`KL8m7Mtgr%7Xp#31BbN8s^99N2+;CkK630t7#+?EIs0=?_F%)$6&7L0r!LaS4~SPE6P$Hy!Q3m!)C3a7CI&o6HuR*%sdo{sW^ULNJe zpN7~&Jo`X<(Y@!S&4xNXW9!h!_dQhQRrF!=#(Des2N z{q_DEjB;OU0>B%3Co%-C$G(pjQoH((7o@gMZTEZ6cy^+kD;rMU_pYS{7wXbSR=fU+ z|Y-D^5RCJb>+%bm8;HcMlssBPvABF|c-8 zET$r|vzCJAL&_*p--D!YuUX6SEWMV3%ja_Hq4cq7qv#mXZ=1?7tmdwlUDT1|Y&EAg zJkBS|SO3Xv0s@$oI5s5dt~G3dYo<^`V+_%?ni%q4M+!r1d;^?&*I-9fS@?kag`w47 zQ6d!FPRjoU0R46ytUe)jQ7;G95y8p;dz0lo{kQR=_hX#BXyG&QSu~L;5vYm6|NA|XYgW68< zEhAtPsTqyBZ$rcMmP_}&9&9JKH2Z!e?dm^p81LD@T5mx14!})hS=>aKJX@l7!bQG-F}7#_!3C6YW`mvTUHZiriTo* zvbG_>rmZ~y={Wm10|O!Y{r{O-sLh4i;D_chz{CE7*%3mYDELc@{6CBRf2R81Nzf_O z!v6R0pI_H_fJe>$t7roWFkb2=TL5`f$U$p>>&_^HdL_Sm+!Hupld25rb*O>Y0#o2| z@qev9%+3Yy-!{blY%%?_-sq4FdSNm^eG4Y{XO9d1KW9S%23Go)H!y()kMlUA)|84U zcO1Z^;G z(<&vH*ZU$ll{AbnkQ)`Gj{?jD-d@gc?3WruRU9)KOJq!EfkhJ_3EI}0WS<$1*&Xpn z-kWgFBpVqQg%}J!eU4RS9k2K1<9&~O+q@Dtg#v;4mT5FnkXganTBFGp)H8sAFKu~j z`^x9Y1-w>s(?&8KFo0R=QMhd2nvl=5AKja7)S4Q&po#rTTY1Rw`EM=8cB6Stb?P`R z-wXV8!6=am#D>zD8<-thq|x*yYd&TEWnv(qfvn}~s=3QwlcM`unX9IqSrNWbmrR4K z_CAa`HvNMr)Em!$p>I17Co~EAijjeO-UP%8s5UM=RQUBC)Q9)?*9+9pi=PyR>unX+ z6=+bYx>7;2Wwu?2NZd$)6@c+lJ$Mi2Zi4L;STi)8qN8p}TJ@wt?5;}ZX13tv+#WyK385h9 z1ggW^rLQeAomTLCK%b`NyB9+ra>#7{b|&b1Nz%jDho0@<(8bD&i;Yj*w{t9@HVBH{ zpZWLqvnBQSPuc~n-t!;3pTgff7hZhHPcfGvnKchrh|r*fn}2(&qjC2A&C7}WhpOzR zV5l6aJ2&^z9D4EFTZjW2K1`)w9|$491o65l;T`G>2Dv=J=@*2mE9e9LK{0;><7Y2y z{_2FH_eZkE4VSnQF@kqsh(_(j|Ey6bhGvA#2{J5>^7sCp%HDJW zi3k4^5QipEeJzEvPy=$`tBd`%$;&Mp#MDMu_Qn7wJtzEq#3<%85Sv7Ks;#P~%rrm` z$1@xq(u3=}yD3dkqiR7Q53q<)jMH&claauQk3cfUkMAB>f+CaMondsyvS+`Q{xU?t zp~6`@vG3ILAV>0_|)8E!3`&7$~o?uR@cg(-ZJoW5K z5#h&I8%&x)+ zHun=IjJKbbxyZi%`sCjU#zH$sO<(($(*+e+KC->fD(CMxRXrB{`4*wB=hC+UnbK z{XIg$Sc-(xx-!XTcs*V|H~d;&?LhNrp;@6M~~~( z%G$Ia--0F`qCr^-D1znHzKJ9NGi#>GJkvM2JnaT}2{yLU^^EB&hvQG^23YxDE7-AMXRa7Sq zD1jzk@E44bdZTuQ?4pTKeF#tJkw#SPo-xXm!PDOPH_+!DDV;p}1HD;6U$Mu23^g@0+9P&im5GMH zrJ5_1{E(cD%@5UC2bS;Lvfk(B(_Q{EFm$A~`yxL6UvKi1lfCa*QufKYqCwd>r5e}c5TO61|psNK^fZdpWMvy&)cq#Co$zgYE5bWrk zYSB?mcgTOV{O3OdJOP#NTKwn8hcCyXP18^e_uT~yM00o*R0pjwe%Q^_1O-sJ=ABmg zeVx(k^746Y%I%20#ZDNoNSe&lLv5obh~-!(yYxUHkX6;zKA2MH8wUUw_kYd({ax@^ z&|zfp%i|Ec-a>QTwI<3CfnR1^L&#kXbKk_)4svw#j>%nhj4B*%s2S&`HC5yBa~elE z6owOZvryx4-ZV0NX){ceCnp1w5B;E@_qdtsEB+uQ|K=hj>uh5H@{zNyXef|*xXcHO z?^GLBFam+9T8v6!HBROjhCAls{h#DERS>}|Ob?k}@?prHVFNQ_PNXE`t z%>m$?F&C?7H*4Cb{6Bq+_~)bc_{_Eb`#XL7!OGOOg?Z{Ii$nG?l*;&rQ7j2)kKFD_ z+hpLR9b;4v_=XcqZ(fEJL((|Nz?0kiveAnMpwqt7qWg97?uYR<)#6ImwC^*EN)Ecr zkS9T)Rq7aY7?3Jmv%0@kCO3U-G=f*4a)X1^fSobC=xo0ZoSkV*v_{#WbuP}Zhy04G zB5fJO)t;+APB+BS{2^edhOz%%<^4&h&ytk2fMzmbo5li!*~Ffuh%Z%n$`0ZfJDGf2 zNgxxw_jDuv*W1ZHowtDG&> zY01DRFMC{PKbj1N0?b(QHo}9vhG6VGJkal|-8qX_8!BFj=SwO!l%-5wzaKy9Z0R z{n~&5mXJs=W(Xyx(s7pU+3Y z(3k8pm`}T;tUWmXVgC0ilopE$a=UvWl7v}X^#@)tpf3k zIjGOb7!TNV!U#u0p&d3Znr1ST%%G`eRD|T(t~GY>2s<}e2g4@yg8+P0CP3u1mM?3& zg{kPB#o_ael%xJK2w27mZKKj>Mi2LU7S6)|?5*$QV0)ui%*&5VZt&(s5DDT<7Ny~@ z-!-ng_i5}(RshBC>+>P|1DS%GDP@1nuF98e8N#M<CrohB0s#R1845wLY|KcV zeDJDLI?^7c67ojrxKrf2Jm^9RA5hN+i{iyEs>q%$VGGWVWo`HODu-DPSK5AvzFRcuWT7=?ICdwz z#t@%rKi`?ounUuh(+=r@cru)5GFa8f(c5s`a{cdTjb;|fx=PUD9*-Kc=N4gu;j@-1 zUng6W50;?e`!$X%A>QuH98^j1cv(HcaT;Kw-42n(;ziM3YQti*yUjTosk4e&;v5%+a>$GW45uk6<=r$WRYGbsrIKvf1M|0=5LjJC3EyebDc>xj&CM z)3t?d9;o6J0$6t$7>kPyZ;|@cK1rN~6aAZ(>C=KuCfTXv*+Jr56fy!hkvML#??=p! zGFUIoEiLDZ^gFZbC>}z6^Sw!>aNdOsbLS_)-sPcW{mop6mm8!_3Z-$fA1z&NfndHz z3oAIBPHAD*#GkQR-$2&|9)w*4p&u@m7+ zxDYKD&4P9>e|6eQd3ln^@QfKc+DO`1fUL;45|#cT-vQZybwnCft8mST$QnB?ofwo~ zJBDX6Jw42p2;y`1va${aXhWmM=0vn_DSo3y=|DP=$cSBZygEsaoXzCK_x&xIoz>jMB6}WW`^<0= zyq{=9jg@URk!2p3Z?n{ob5Tg;&jURBTm1Ersf=Z(b3vAY7rQ)wBt@WKlSh7uJWF z{aX$yZj8BMkjGbufstL<&)jx>{J3n@IqQM3L0TU^LFWr+Blxe$7IQY#`mh1MAdhA< zJioVmR=StlJ`Po(t^4nH zmr1C1ldKkMeoX5?%{gVVlVh$zI8+`l{8pnf(;x0pJNP=jsGFrMH8~hir(u$a+cq|J zt&g3g__2g26(0Z9>eb1*|Gj< zA!H3CY6@QzbkX{Iljx_gw|WBnY=JyKiU9%3vN3DB>jp7{cVvZSNe926VwiJN?x8P+ z?OCTHSmHzO;p_|dlf7P3ZpF&K!7atyHFAPoCR1EX{t z!T~WTic_#xqHJHgY(P5&xm9+tj4Bou*jY7lLIUR%m*q$RJ^?b%6kYI3%wQ;NrOw#z zwHiicDj9BS&%8c00d8#lkaVD;^r%?;2^Uby*^EOPAtHt!(?$DIrc1b_f8*3?8w^$H_nCd(=1pw~aJ(a3 z$7RI;prZA9q)uxr#9SYU#@+GW#(euF&?o0mHWb2kZ75JenX96T#fBZfF6o2hsKw0S&cTNqsuMemeAHp)7qf-42Ik$qD?QO(`Fndo2w0+iEr6+_YMk{=;a)Zy7_uvmHK zRkRRK5lOKYPNu(hO*>jFmt_`kzO387v^3k%+4}!xp-e2Zx_P2hXL@OfhZxf-y{f|F zqtKA6C#NYOX>)w<7LZ^?$WIPOy&?e_`fLAlf!tJaY_jYn zESbW2GIS|v&VoA~;{2szk@%x^c0!9VA zVPokGFV4}pn%#C>$LfXHBF?8cjXDyFr**tJ?8GVYdEe;GR5URXAjh+DB;9(>f$PUM zIRk%4*Z6^UKF4aoeYd_ggGS2Z=X}OjpCv5OOuX*}IKW5)4|Bbt`nVZ zkO@$QPKwBqHkWma#t{Ow;ic`~G_>Ow@}iUxF_ z8}^SB*6CU8`F?qDbvIdIQ)Ew!|n#2s+uIEFVstt5=~IfxZ`2^< zjW21@%)#m@=Ejj$-uFCg9Hs*`;+p&>i1g#3{V35vXu;Ah87ohh6bm9Ja%Ldhv=dF8+-L&n@LfIhpaTg$qD3vxl#-ezT5k|2k!GaK*x}wTr5vu zNMz9}pWJism%dfc$5S05!ktd=ir%1_o93$^u*{>>U(ozCiYhJ-Xi(SfH;aPbJ!97T zjS&JCDpa&NGxczNCmLYh8pZ+`pcd{DUVpt-^IXYE#K-GrjaUF1zecLA)D#S0La$S& z2P=T42G`ei*B9cb`Ad@TR6Sp*TYcR|F`P0@$N6)ykVba!WsbF$Db!4rA?Xp+NR8phxH=K3uBJ zP$Ry(xZJaBW*x>wD;azt_h-t0yyX*dv3zaV$xnPvGyy))m?SSg(gtl@AWMi~IqnS=6nM>~uf!mG6l6>HLv zPJ_AX1WbO!%Y>q7F;sweB7CL;hPlsdYRkq+$$geu95A|9&6_;yGD3`2XOXAgsS&>? zKVvUO_;P4L9=afTCI80it#P>JA3mS1WJHgk>9fA-{R6Atr@<2EIvA-WYa=T0tFyVw zPTg201h#1~V?Ef2*~kLkc+`EKa;4U7mcoP-qckJc@&yr^Fc4q%wEe{dVhl+tL!>{W z#1z_JXm$#2k%v{8XvVX1m&Rl}rWP|*q?|z<&5v0No)hy*BUYSoPjohDU1`mEA}y(| z5x!-n2P!|S07A4G=;lSI88{~;w#=7-I1ArMKkX)s~fMu!Ip+H^|0wpB{DeVVsN@9qiN;zB~ z6t~*$DvHn_AeR)D6r3H$mLL{fWi7vcD3e$THdfU;*cLV7f^iIE zD)95rKdbru7U;YiLNC_x^mP3#+SkgL64-b^?~~|CZde2Z@orX#TfQcy`Nhv9T>oxZ zzA7Pow8@qT)5qDX!-@{QdI-TCWno(I9*E1#Dj(DKYpFmHT4Gku73y{oNWE|I?uxQd zy#%=-fWIr7=WfE}_PAPb@Yx~9y;2(Gb5X#xF>`C&%4v5gi+PinLIp~2?84O6!2D1c zc=>8oN~IP8_QG;>kYpj(pmCqIm+V|cAR9c;>8sP>c#*_=;*dzE;r{KmL7MB@?MyLO zqlnK23kq#o2c_p0#_8cz;kV(x#R-~ZLIL8K2mrNFw>f>}DR3XBx-$NHquD)x0MIKI zsj-0|p*J71t1(P$vTr>5=qVim4#NG0m2)N;46N4euL3?yPeT_FrNm>FfAJpu!D_U| z3MugD>$Sk`pgSD4JlyXU{`OG$pY(P zG;22^<)|L89B@s^U194)&6}kwNY9>05R+=Bw2xjQWuKi0sbuLgf^^6hS@?l_lMS!D zUA@Rpy79mkv7az(8xaIRXHG5O&S8270AM9(#|ZeSd;10ptn`CLw58;zYv!FBv#di{ z`!jE#)Sl8rT_9`#KES5UYp>W|b>0FrGBN|XI1Dzt8CfK1%oOnmwf>nA?#$;-JI=(J z7g#%j)P1}a!N%8EPFs#QECRjdV2dA-Ewhd>j|&Klv!8NZKpx+a>3bl;-&?xMd^LVl|Ry1BV+7>x` z7cI!6mcfw6_969M;)CTNdtzO?ua#03+`0{C^od;5(dglLjmwe`X>2Ik+V?E#ICwn^ zKk}oOfvk*k_beeePF!RK{Tc70M5Oq60?k}f)9=G#$vPr4jI%L17=SLwf&8$I);zu449eF!j&3#)EaE!z1X#c({a5V<%GcOOLa@zn9fF;-Ed>rV&l=f9PIe@& z9$u*0goOQTKmN991job)aSRBt5{r+%8nt(iev0X9>?Yi&2GsyNo%0)r9H*5F-?~UL zivV$sl)lqx(+c}TS#4hS%6Pr#Z8wcvWXR2C)rgOJxdcM+vasn*?ZGz-5!lIHMNgvA zjZ&mo4k$%2UzR=_{I3RrWL5Bv4I6YXJJV!MNAKO}ftB@O1&9j5DFcBrOkF;SqeRL8 z!2X~46%atLAFW&i?$wQ2=QOy?yr2EKIHNpj9MlBYSCy4_fi$)(I4Q+xGwYP;XOx*5 zV3$njJZ{6Af}X41S4EA_=d0>RLyY*E$o`dPDVZ{h0897D=T;Cc_f>srXtiq)_e2rM zUEwl4w<2)9xky>o)Z$~l`VQXN%#7a?v|*E(6}9wMYA{1d4Rid}NWtVZI{%=^`>O@9 zy2gZE#GD`4B>&-Rn%O2>nQRN~bYa{;pM%0OhkcKv#I{5J$*yby%NY=`1vJi(<`HVM z;*FxX*+8<)?O;R~&TLe%?CZ_exuDBPo!2R`u)xojIa-`3z{=O6+{f9#Z7&ZX?ULSB z7O$e9#lR;Auvk0+0`dS4XCqf>K*csrq!zkGz}LK7qwRMo1oVMhP7FWgkO4S5xjdsH zP~N`kkVZq67);pM(a=`L(>0ilaK}1~g;S)o9A0_-ltG zZ5!n8L;?`BXat2XSuol#m?m7YCD-Iyvmup2a6*6pvzs#Kt^GS8m-Lq{ zCuwe~DMogC-3sA9_#Zj@A0zgth(TN)71 zUFhsD#%30m>n)JQakU!_F7d5@ofcOM&G$v!v%SozZ%Z&VLrbsxLW()%#PngzD%<+1m}3pb z*3rguayJe%5RS+}4&-m(p2)0>lPGgA`-`$UBCknQ2G(x~HH#HTl<{&k&s6?{GMZrl z(>WY+*L)p(z)^Nvcf(!~1c8fD9a{&cCKKG!jMjk))c48|5>|npr)7#fh@{u981V!} z6tNCC$bXrM*&Rifj_hF4`4(d~DHkV&8yjGQWBB|JFOCI$s<-^}1jso)EGl*}HvcpU zpmh2q&U>o9se5AX=1!P=R+Qsa*mRR#N95MZXCqv5qT$EB&~hwtEXXayoii7cadg2x z5rokQ2RSsQ@MBt)@IXxWZ@29VrnBw}YnA0xjOo_LB~j?kKhPk}@pvSAwr$Q#4&`j> zW%kS~Abg2)cpeW-p0&!;M18mGhb3D&F~LpvSu-hpgY?FkmpA5!nEs%Cz3J_lOjWDm z6`e-clf^RZ+r~uly)qY_bm>cE=7QVE3__0!XVj|5K5@Z&J+~Ko8yNjv6*};Qz;VD* ztnQXVv%{wke5zChW~GxqcCF@M)8H$1rmNVRsU;8zS|xG=-ykWar+=c=)HGp&>%{}q zGI{J1+CICB%p_Ajz_ef7412%2_xDfwWWa|<~Rs==@7!z}B`DCs@yf`okd z>dzENKymQA=l<(xO9VbADvkN^Ow5mLG3lF(RinGu!8X7k-<514H6ZUvf=e_}yWy(0 zV)h0!aIj@Q7*VfaCx4XDFi82<1jC{2vspMU;lNCoQbc6k@8(FUVd~_yE;0fO*aqdp4^$eF!b4{8QW+FO$Nm(4yNyL#MV24LB82+=f zHP;_s4*7N*%f+LHMkgS@WYHoQWOXAFWnt;AlIiZUyMG6uX*@qcH<} zLqB|t&M-(O52pU6SsPxdr@45^3RCR!$VF8QZla~DtX_Q4fNdcmc2uk=z>2;jB3g}>w=`u}N%DK-n_f={S z1qN8-^kTn)=w5_*1=$Z2?C)*i!rPuV!{ZBoTghOf6|yHzGKKtz)H0oy&c#f4E$@=lCP-mTyo{#1C3rkW4_-3@ z-Nz|H{i^5AHB!b0msuDkkhS*%)Q52vN=?I6_`9)e6u*dV5$?9L zgCEF6V38(-sXXlU!`4E74!%#(HEt;VY)4N1@yf=L5-VnNx{$sr!<-@5^|em)FS(K+ zCjPh1&j7~?@M#{6+jk-|?4#HVYMmcjt~2lud%+TNUt6KRRdt(=^R=N)U6BBY40u&dBec?EhUxqwM(ZjU09?r!e z&O4D`-+B|sN%+EXmnW*7HG7uWlD+<9vvv}zM2fs7((G+1&XG5K!meZJKUa46TJys$ z7N$7bqNx%WW)g;6+rLe*vkd9OF$&uoa}-FH{m$WmC-ww|{cQ}FZRGYJENhy|xFb=kP_cztEZg63V@w#{+9sQhNqSX!ZPm%~`6O{j}* ziP`E#Dn`;`Vfqh3o$=Up^?Qw;uIv^U4~4r$kd=Z)*ZHt24CIeL6;V0Yf$lbPQ1QuB+WZ?|M!cXWKwA@G?}y>B!_7g(o}RLOfpgxT>OG1)Rwv zok4n22k0(&d@C7VO+wYfe(ZprDc6353Ul}ud7NWE(G~vK&I@m0At+4zeYTY-!TD7z7pceS?Ey-M`IgZ%y%EkmD=)yBwbQe=#*t-ZDTPqG!M`6Tnv-Dm6hpV%W ztD;}m{?Of>BHi67T@nJ)rKEJXgh+>UcXy|NEV?_SK{}-oka&k@@3Z%L&tJg$3@~fW z%zgjj`d$|k+~=cyX*$Of;LLx3R;Qz~bo?A8$bVGO>n970>pLDS1g?~z=@{-e-C629 zo}mYhRI;ZsZNV;9Y6q1b_oT_2Lq)p{RYi4DiZmjxR2fb%L|R|H6U)n zKzMq_AFklBG|o%lwVt4@L9~jxw~hKL&sr(%HpLJfRtFmY%lh-ZfEl2exBY^1B#-Qm zmaQky?p!O0LYq4^*x{^6l6x07C~7g0>~bm$c(~G8(%|aJ{J}4Ls9-hgst;8Ds$+A8 zrG#aTeV){)4Y*38d6LccA=9{g<+M)A;Xm4xQ_-vq+1gq~wDO$#1F!{-U-H$>dMBS( zpw*35vZ|l-bBrVTy9;^#%=HFK`n3oo!s%4)*}$Mwr_8gdv}^DqaJzu+?+yd3eSB!9 z2HGXgNop;ZZ<ym{&K8w zP( z)KcqU18d`^i?md$S%-Pat1TcN!m}|69|@ewEFNHO&`A4xFXjL(|BIV(A7-Y?Spr4Y zpMwrTjOAbYXAe;7oAqC4W@SW^`9*{69F4ng=#>iR-*$Yw0?#lhfQt->`X;L7crS(m zx8KxqIu~9LoPUEdFL=#}8x#a&LsDM>AipwB4h61;RziXLWN`cS-^aa(yQjf_|8n&I z;5GmK2E=nZY;-_oh7CL;Rw1Fk43l8_rs{;*P(t+we>j5oZ+=e%Y&3c+Sih*)2>{W? z~8IwLQ3b60MF1T8LemW%+iWa+#3Dg68 zWf0=0z()8x z#UxrMBxZbwF1uZX!KWLEj@0+|*XmGn0W{z)`{Wi43_MOLvWY;ij64Lf3~>H-q2Iy> zf^nK|2tP!EUhVHvesyMDr+^HQ$PNE){;9~tEzZ+XLHwK2aGXsv(rF@-tw5}qVQ5?t z!K<^L=e>7-K)FHh&rX-w=q;6}>Sye`bzG#6IXj@Hy~6fFGTgUqEd2GWAFA_S$1g_6 zrokW%p~fO$K@$CW5$^VNoQo0N(hc)$AE>&$KmJo^2T}*=9qh?0q&8zX_+NZ8DG;5e z<>_9Bs;oDcm7cw5U|USSiCoq6+^pKKZ)IQ9%*0G5vN;{<{Ndjgu@~Bik^Cn7;1K-P zhmt2S9!rYdyd2xKzWkmln!lR>$5a z_F9(AlbU0q!R}1_1?LEcha6WL+5WsZ3aaDDs>@_(zYd)bn@m&#?+o+(LN?M&3iXKF znP5c>gUXoJ$zeO+~r2uIr3ZRlf=$ue6mi!gNii99!gPrqnK zx3SnNEnw?>Z0{FAk!SkDevXJwFai$G5fXrqaW)(B{2JP4R|MUAUC7t<7Hfd?vkUab zJKE!ijNAJ6NOh%3J8IWR&?u`BNT+@q0d!(EUDb|vGnA0dr-x#{&kMf>$erIR9l^iC zOx@ZX(k+FZ#90vD&`?n*LobeqHavOP-o0*#rI4nB{C(FuwjK#a&eHfP9d0e?2H!_*u8hsV>0Yb;p_ zgME6m$1@=R!+(^AEN=F;)w#D&A zY+XP}!5Mt<#wi;2k?)6X2zRde~ zfqp?}{slO?!a-8(?|Ix_^Myk0Yyi?|?Ue4n&DrwudP0#3Z765`D{3H(+Lf^I8>b ziql!3SlwruEpvBr6`FShWy}Am`5+S0++<_e*8UlA2yVYf^d$wFS?ooSzrFhM{&eJN z-N392t%2H@OU3$9L+`MjR%>xb#F7YHWKMO)|4<%6)Q#v*gz82lrFZ36evu>nx5o0X ztK*4Roi5;!pUC3(apu0JD}zbHAe&L4F^5@2SC&{MjG1jngYZ-y)>gi%AZ;+|WTZ%kfw~Wy>%rF@rVr7z{_ z{ss#-Pm&&_&3xOKS_<3b5rv&DyDWFAj%T;n1k zGD1GQ={+4-)XNy9b)Gq5@^}@t=h+8m4joTr#xcqzQ)?K?(S!30>G+lMQqO*V3?PeI zSLtfD&U4iv5b-Pt7Bb?CHXTeksPRr$nq{esHrrXj?B!x-7kOfUY+v=~6YCe5z3AhC z%JZ>>7?r&qOm0*(g@zFQJs)`WoI2sNJN3=?N{lT5LG&!(6jX436Fe+AOPR3u z*&=yScQ&Z%Cz~@oy0>qUff{h{?(a}^AXhL-Cwd+xD|!+8e5La2Qm(gp2nt>YLqxA$ zU+P_rO0NC|anExk6%UCru!H~c{@J!>7OlpDeGalCzm@F0W=PU5UPSpImSOaAf`LT= zjkHHExkM*+?m{A+BL=@W*Nq-F3Gj9^1;794IQ`sWssBX=s0sPl_oBiF2Ncg$iqALp z&o}eUKd#5XtzK81B_lea$K?TikCL^Pq-leO}?knJ-=Q8##XoKIBl6E&!C z$^U>ilO?vGFDQKnWoA`{6JC9{J0runA%^4>A+=8wP7Q{Zo3 zo_i$4G->1^>oO_@kvb0PP_2QHF(iup+J__8m;t-Zq^8&djUDsoV2HtsLO+z3(4<(g+4* z!GG(4be;vVVQOd38u+{+384W;`3?ZTHO$PV0;oGC2LaT^as%6MLEXqJ#LGIeqY;s- zuKl&Iv+qVNeoO|jZlm->qZkwIyOc?%C!THuVt~lTY2`Tz2s42<3DM^dDhYQj@1Y!Y z_P}$9N4o=&l<^+&n;Kc+O{|~oyF8EW10uh>DCFp|zdHwA^8&TZ-~4Vv{9a_#q(0@G z9d5-!tMX>FxTAL^vLeF-V!6qnBf2b^6%Bgk0I0>RGP;OMmO2FQvU5I=DtTzXDr9>E zA0Tn}a68~IHuObA|Lya!iFoOQX%9_` zv5Sh=bUdvrf)~*$B7qVdByr|C2SJ41w#Z)eQh(N1tk0nxRE(bvH&%bPeJ(OD(EgBw zhla*w(m54nXNu_MF;{Llr$P_e9e><0K?O8PB+YOofQpOSnw(n8-*dY=ON) zAV~@>ZVHVk>8nqeTEf2_zj0wY?G>knPxcA@#)(lLYg;|Bd#cQW#M?yA9|i@HgGkuk z`de{SJ9v?uu*;du;6Clo5pMatOpNieNLpoA{LDEmobvq>2#C~&vL_C#NY7ay zdqDCcjKIF3O2BHs5fwI9#HrGrFTdbnh>iUEMuCQUwG9X zL9!k(9?99L$t*dUvS6}#R~>det)s5+kyUtUzmrn5qXVL3F&yOgTt%xfOD%M5t^7+@ zDxp0JpO>>{?T77cc1W{&8|AO2$%(=?r@c3taGCX|-9Ht*r2n#Ve8ljE)BV?ji5uPo z=U*>4n{vp!#Vb`w;wdKzj;noES8Xm&)$3xi)j`oA9V}zmbLOIHqsKX;^Yx>!AGt8` zd~P)pQe<7@ApiNaP{Ii5hakVlzIrf3<&_;lQn(n@2<(?YdCq>z=esEgKY z@E+IQfB*ZO22TV+HncTQGXh_@W-?Ph? zHgduE8F@36fZ)T0&SKaHq>0DmAT=V`%G{jK=;$XoSpn?C*ibVex2;(h3B^)qjEud% zg3hOLP~z~2ncd04SsRG#?jkyGBm5o){6>d~mnlC`-d>(Q{XT_knW9w)LzCjh9#1ke znzx$2_6vHiHsBM>nI9@ILbOyC;i>ssp?iU0j5W}z=_!r74>^WHgXk!6yL;}-I0!I> ze5z@bMpiD6@LE_BbUi&KP+*2PayV78bBcdKi{!Oi2BldqUOk_`n%h&g$=REcpc!m@ z4!%2LAj;47r}>GU7`OcM6+l2%f;C&DRwuEu2lAe57eP-FnU2lBR9K{hR>1+mpK3c3 z*9^Adfqb-<;h-rwz~Rf+^8?k$5`|?4@4KOdNL2Ih`H)q)P5vw&_n<)>ZXz6CH8NXP z`L5O(5O2)mu*I*kI@iGzQRxKvKVn765r~6+60V%x6wSS?g=uVyU6O^F`v6wyGk^Sm z($344nR!iI@boLI5L#K9j_{3{@^@1-6y!=s`_x_ve^ke2JkdTj{-tgb4g(AKL7WuK zdn;LD1fO;nb}08~`0-Ip8bW*FCBqW|V=>22_)og5Q<`C=ky$N4$wdCL zm?8wI6S}-}1lbz-ng*swC8QUcX|!-|pZ#T30t~H?uQUpOQjW@JW{^1lgcLB)4jt1C zwVI_ggq5?LBRNqEcsd27_-KAl>r`HBgtIyH{R+;q^|pM@r{z;bT*+f$6B6UA z!HSozU8_$aiin^cwp&AVfaB`&y)s?bm>ebG4wHu7xEs1WDV}YV2P)7F@BvbiFDL0W z+T`1UBG_%RC}zG$=z3fV%Q8ZjF1I7yAl?%jb%R(=??`|n2Rm*pCa-qzk*AJJVrfcf;@|9^mtM zWc11e0nmCt3*!%Kz)uw{9CDInjk?v!JeI50xT{!s;xsfJVmSn`Hz?)ZIff|Otb@L< zU02b`Emx^JekEeCqHd+dYDZMQvQ5bdZtbOE#aiiOGw(wJl-0ORYOf`}f60`CHK9nO zPC{cN>6$9se=o%Lk*|DW(j`clUU|eT1Wg!ayUDvyPsWI4m&k813O#R5i$9@`9i>zg7}dofuKXOC*&4MvS}6=1jVWL$s1^MuT;VJKNUB zViz#R7|qy98zHeF&cY`gNJJ~n^}XAqH3CBd5NlN&N*H>>=mm}%v&9(LM_?c{!N1Ug zFu3o_RtqMVzJ4G0#>^8u;dqgo^@X-~#=S1*`IhCVR8|jnu)w3jUU!}kzumFsX#|ugFSGUTQsNcQYjT7)rhgr;=r`eAR)JZdA=f$P_%x-$aPdm ze4hVDxU9B{{SO*fyQD#6v zWRfMvtz2909r_`1Q{?>ML|TAf{FGN+ODymmPxPL2*v>&0YT}=i ziY3A}`Nz2spJpAt$4~0=B58PL7+L35V!*3uq1-~V#>25O|np%tYiGt4Di3A;=(sakr= zsQU+4za5Z9)!G2Ut^-oM9{F4m0!_@xvanLV3TLfS@Lg5f6D^soLtNb%a{&~TQ3!jt zk&-y9$QE_0eRnf;DNQ-?4D%HwPoSBEs9L<~yANkwNxQ>;lu_H?>w{wEE>kJlWKE6| z1qW%13GDI;Kj~>J5&=XkqxX2Jh#v?8S%6U~ll_Qj8C3P(;4F_b<>Q>rMWogjuScf8 zv_fpSHW@C(xZbZ?`me@z=X!DKI-rBxQGhDE@ai&yTuyO~)Twvh)HJ%~?)Xxxf;ZgU zinAfFvphln0Qwn1w0z{*x9p2w%Sz!Wal*~^oTX?L;*wX~<3-u=2I0_5MkMv6r`|!g zzF{_FB}%u?18QDxV0Q43ewKTZ_V>C7aRrueV@9Ef7N?4zQ_-FHu%YI0dQ7kDYNRcP zJiZsi&=QhUdT&@=Y>>fj0We z^J@}X7Xi!!YncC={N2;0=w5miSpFwC)C>+?+!;FNZIe{<(es#V>ByA%IeJ4q}~dSapV%`=An_O6I;wvC`<$S`*; zU`yI82h#lLfC)M(vT#h5hl2^yhfE8RBON_jarZJ$EXALjvNm4?L(w{3ou~$P4?{UX zD{vMqJtif6r+&1{K#iJ%gA_W{*J~JeK_l_zxIGO28r7cTs5FN%o)7fwD#0yT=^U!D zB9;%d`a*?eVwO_t83X^utK>i^K>-OB8Pk)po>>&vTAs z)G)aO;b)9pwp{eMp-GqX!Zmt8U3CUN-j@nG3Y|AvW*oox7P5HyH{GjG!zNJClgo@5 zjs>}r%A0HA>C;jOztclij(2-|X`$F95hk7brpNim`@a_G!&}v9AZoQTO1#e% z$Tn{EC(+-_9n!CsED!~7DITsgO;mrzmr;OKljDiBYq>=g}5meSe z*;G(XTHi!y%wx$99MU8lOS(T_Ra@Mf3*LOIR^7+k@7vL^fX!2>gHm!47Ah2hmEWe` z!-c>;v@$xx?i|ptc_Mv)^>E;UonwST;v|oq1kN90fQI1_f=zWhc&Y-8KWwhaZ1%LtxuR3~e3 z2?o`qS{Uv=!5qUuL+~?y>H6VYC>>vU+-zijtO7UwdL$|#^}4TiGQ9f2l~k7bMd~_&tX| zTG-5&V{O_Mf5fhni#t)fKG2A0s;hS*mCj_2GoHnVfy8_J*NfJMV=I=1YI56zMiOrU zgqHHU%0=dfYRMwy0+@)#O`yF~Unaf6TX|Z3ZzercguZS}#ZaPHs!X*K52g?rqb)O8 zVuQ@K$!NO^pILAxouTapY+4z^MP_dB4-mP=uck0rElPF!_O+@H3HlphHlM8{fAQ}~ zKQ5sit#=KnrE)T1(j_BnMjr29s=3r!U(&9-L&3>~zpWvxpAJ zh^Ql#3dkuYgG;y)6^{np_5Z61QpsVdk1d*3&hO^8N7_ zHe$RzsSC+i1hXySbxBAdCt zc0j-Viq~V@cujFP!4X<$J}Z%$;>P`q2#6L6JT5eR{KO4H_9tq}x_kj{lqj%_qHu&M ze)V8fn@rng6*NS5k~6ItSipwRKmqAwT5&d~-u^BR3JH2a_#hg-Ixp|e0s~Cs1;TGx zL-yi{hbF!rV27qlDL_G1Ob^*Lpv%w}DYKaG2Ni>pW=`{8#}nZO@Jtmk`_vVImX*bC zCjOFdyk$BgdM9Z1^9RgYiO0}tpKXGzYU09$IP4M)7>XMz%lo(CN&5N;}4~!C8tI8HU{X--m4{lp9M`n zSWy>WcGFFO7H(RRZlBSE=!%orEp1TsDQfmBk?R~MzQNLmNwCW>5rnS+hmQ(+LeTnF zd|iAuG-D%&!*{TY>EA7p8{Vb>8>I4m7wJBWZ#&F&)FTaQ*xMqCe{k*28qml|zDfM> zSY{CPdax;aDN1rR|FL0|D-%wB+W8zd*dv(N+QnuPtNH4dVRA86h#vjZ9&1!bw6wH6 zv%sbXr55J+%L+NE19HH?=0tcOk}fqFV63QB_gFvPMXs(Dft-$$1Q>g~eX496Rx!X9 zudCA>?4!%HGcyzYqK8z`r%|TGlrGhS(4P7c?-%zJ$B*P@1dn$>#x3eIBJu9Vxz1ik zNb*IHj(uv#j)O;&FikY00WUtURW&kz?k{i0%OrsBb}?r{3Jc)G+hn1745lG_LPO36 zYo>@O7mG9BFwupcrkZh?v9QzQrKsg5T~Ff+9a#_A;B) z(63R$5YXUI?8ufA5@_@f^Mx&5rHp%U->H45#OT(ZMoz>zs8Dr`P7%jZ(rN&41hz^M zzC`xnSD&C99*mZxruF4LjAFPNhP%p+$E7qGnr(<|q1cV(b4D{u3tU@SCTk$h?sF$zStbvn2Neqe{Ls?Urj!>hVzzA%)WjC4 z>3GB>C8i{7gFZRa72@9XgGu$Veu|2%&0(9+5gW!Z%AX|~#bZb?k?n^zJy09I$XCCi zAvz|TJ@tq6>C3j~dW67$on#U<^iT&jnIfivI z+(Z5@v2pht7ao6{X8K841TGDSgJBZi*fZ-3FN|q=8ez(2eM365P;vvWTtF4Ic9B5c z(Ob0~llK@mz(A{=bl8{%ouSnVCuW;K|I84InUhP{e4U5P^o@*Kj&YN16YJFe3$W;avwLhwCQ_4rBo!CPYygv!ouaP-$GCT75 z5!TJI5&Uhzjp4F5-+BjB?7l7K!XC*>#HBnyJ3W-PWmd+Yhw0a#Wc__{>6?gsZb4!0 zFymnt2gQIMS9pl8Q;5>)BZq1|JYGyj0~r_1RG2;j@DtifWUPF5Sp+hRg+4lx^wSID z?8Xaink~5=q2s!=ll*?@42xE!6d4I{um-;5#IfcoK`cK4L22@wx(!;3$3+gZs-_ZD!?wj4!Nr? zCZ~=cj_YAi{y5hrbU2PwU-!U5040Q{e|KJ4Sv>UpD)|?@e6dy$fu3L_!|@>TM$@c% z#X}#052J^AdZ+>X>|NKd#mfY%`D-~_CnG<>bMO7x-jAC_lVv#o1rNrXcukucPUZ@@ zUi0J~ALl&1(D%dRMU6*Y+cxwkP)rUnE1(!Y(xWVC3z_)Nv9`=cdJ;&*Qsm0iiSShG zkC*4F#Yt}-2wT4WC0upW${>KEltDUQN1yj0x`CyPICkyuu7cp92}o%z)lE&m{;>ip z4C1bVujyK>4y3ogO8zRrp4$`@?BW%BrbRq(pbZ0`xtgoKHU)GSyy?+$Stw)6+(v(< zSSMeARQEig@!dSIAvIBF#sToRHBJA50%*g0Zcqu-di>bn1efhDhgsPPcc(7xS#Y`H z$Tab?($a9qE7U>&mA+3Kcy(0FqiCn zFdQmXzjhep=8=;T(U@b48AwqG={mnXM+A7!j)zC8Hf{$SU<8-;9cLvf7>00nnw2&s z>&9>>ICT=Z3JQFq%U?h7>Cxu}jsN5rX>wvW9YgU_$Sc)yu%2%a9AXoVg;s>=`5Xjp zJ%ZUkYf<#estgBW0;gW{Bhq?+{vNZbmgo2&Di(#$B44QLr=fFNDMQ@nHJ98Nbaa$D z`J<*S1709Y8X*c6z(DUBo^mPq@Qz-kmzf7ek^dvb-J$0*Z!z8@xZrw}QxPz1t`*`Y zkw?mEoDx4?#{&l`w&~-vKz3P!W=8?&{JOrzwRi=IW4|qtO6x5)ArPRkCU_H|{Uoc) zM{2e@)J@FRuD68fkOzH|x~--G5Ac79*OGRcJUjMt&ON)+jof7EmWVbAvm&<05^NVl z@nSw9jpY3CUo!D_3=1b>#&z__QoOKPh74lvFRjGhK?) z*KWE+lWg%dZw)nJA4+mD4XQ{&R$jnPR?EVM^)HXy_}9Vo5M+xcH`MMgP?6&q&;}n6 z(43aJ1V7CQVx^ssl$X#*kFK|Sz<35a^;K~_EG+WCIcP+e%bQMcl73)tqB&=iaqcMK z{uZG-x+XoMSC?>UUNmf}@G4~xS~UXN7@ie|Y3iq1&YeV^lU(WX*j0@=Y#;0>ie4TI z1$AB`mFJ;S;12|T4!9Na2Bdp)edyf@hfx~28RENTJ3RtUX;`B_#zrETnt?6^(T;RH{XM_co z<%(be&gDC*{&>*Hm1q?Wp*r0-k7?X;AnIFOC>)Xs;B(Rnz6&vK1QzZ(X1f7Eah023 zajnRl`Bs%v0QxM~jXPz_=ZJ#RL3u;#4`OI9PJ04u%&IG5(gbO4Z`aQ%Re3o_8vBL$ z%h0$ZM_Ss6CLSm>UVy)5z@n*Z;2j|mHI*ifw_||dGqNk~1m#nNsT5s3#dQzY74;U0 zd#cgQK#xSWGl~MkFu`TuN2s$RPc$`$(1whbwRpu(!Swr;X#PyKs!q59d}}qk?@$VP6Du(R zI(x>p2%TE~wTbOrBn`WO_RRC26tQb9gqVxO_JvYVWf{$E=|7-#X#Ui3VP}0LY*pkz z<{QXsNl6eZOXU+ON!)xo4f=@fT!SVwsxmW-Il+)H+mVDtnwh#Z)e249rTlK+IIt7_ zZY$2x&8<={4NuA`LWUVeGsmVNfC(9svnF!9lSi;?^gQ`f6pS&w1S!1acz9DTHZ-)s z+qnQ5TrMVn|G;lZx_3J`CxnnkuW198*8c57g^C*%m&2I(bZ2Vi`OcmXGg*Xy9T62( zQO=+lbIPkyoJxuJidaEgj$ZU<;|RSJv_@H39m9zG2aF}Vf_n9l2c;a%b)&HO3oY~Q zY5Zhosh6+o-yeWmaNk7p)!@6468~@w8IS78@T4RdPcilkMe{`I`FB{q?y`~`*QBvc zk`-{PsfFpFaN&lht-b%))TDSwxRHUiL7by^;Y7iDvK_DZdPpa;9o{ufZtayippD?d zWB0dx2d=@m4Uun$hV?c)Tyf-gR2MFm>fz|w5Tb+5rO(?1H9vE+Ln|phc7HG}%5Gzn zdSaLt3X-M1exTW9EWoR2=r75Q9}xyTz6ZzM@A@Tb$3EFKet zW<}PNXGBTdx{(16>oCH543mz}Q!yT*vMjyyU!aGn;8FK@#yx}t9z36Y&B)M5N;w3s~;3bb3lSm{}jYkPRk?r_bjXYRa#;x*wXd3-kaM)t;i;_H5K9*kC5ZA zU|goF+W=5;>&L(4)unO_kG{O)`uY_ro+~>L276^>?a51zmVAu`XhEGH!W*t?-=3X> zIS|+HtxA%6NGGf@k(P*+LZ9Ic z5gL*y&DNWXn{AYvIPsYu2n!b2u@bOM(k)n6P8TOc6CVB4&&}Qk%Us)edhmX5Zz)g9 zfDNy^JWTiUU#oGKt~g!bb{H;V^sr`H?`V9`Yx@$w;mqfrv=H|L+h@Dsc^8z=pbB7` zVUpwaDDj}dPMPsFR#!<2Uj+1r0?dnX!}~=S3v&~0Zabq5wcTI0ZdPlfKbs(o2Svg&RHg$On=q8?I4sb#w zT}K(eD#d@DtLJc#D@HpNFny3qXUED2a#3*J4zuzJ*!>b`mRWE7l}Gzd+hwC?H^o%( zT)m76flwkT{U^DB<}OH{shJJd9Rja^AwDuET1oR^Rirla_Nfrey#>Vo(A&M;beov= z6P1e)8A}|6xVc1L6B~S?rm}vn5P{q&PVlG=`o(S76|7uyk>Agk!`&YioVixf8R;tF z(}nBAPfBr0D0^5n26!tW?}GEN+KY7x)?Y1d6#e5BLlMbkxAG=2v@E?IdGGnnJVQld z#|plfm{l>)O}>*x%Pif(gHZ%@S4y}TU72%U2H zmq6rG;xQfRZEzfiVnSVY`1)7bYTiuqw9$=%U%!&hRG2K`>G8*m1b(yG;ZYqGuIgsP zoDIG!`C9G5pd!zUK#-=Oz+aLC5 zW-m;p>$GtKo3xDjPnnr{r{TWYA@Q!oIv!8|EqYlG3Ee019Bo;OCR`MT=_l$wtC}mPA_|pH2%Juq z;)VM2sr0zY!ZVE|P1mZXd^yMhmJPfSAE{;9W44;pH;en!DrkjX%W7z*rVDHi(&zmt zx=_nX56w;+imqx!SuG-`)%hk@Ph}xop=uz7Y^7YT1=HrHX7ZDEdkE3F1E1bIaJg%o zlb}ZG$SO54dSP7RKIZsH-L0d{5r|)sCqgu^tI(RAU5-<*ISPZJRB@6RcPTm9zt-w7 zYc5=vo%;Z?Vl3arbaQn%=iUC4rWG4fat`vzt1r5%&QfqLN($|(%vG1B_CXHh$y$Ou zlPo+Ml*Z~mDklc%Xb>~Q##bu+@HERnyuK=p9+4k1=qDtAgKSLb!DZUmzG%;&c$}ZsB`?YY&|J?+G*rh zWf#|bhA~|~K6FLUR#Mz3@XD_!u@ZJd#*p*e&phG1C))IePVAh*?S>OELRK7(g z@Ut^9w}YC8lLyYL?)kby1kSc@XNaj@P)rdk3d8MiB%t_WTce-EC)>+Rb4L??IlKN| z*fZ|TR9!tp*mFyeIf%(97v7W<>!egXSHccvDAQG9x`KgZO5#ISDc&wtsx*eiD~oP0 zJ!1Naf6Ba3cx_4TSr;B4CwLZk#VycUKmUd1{MTWU?H}R-nB+_1f>LG?%sc7CJq-iF zABqcV38$T5g&mkZ7NLu(2ik%S-(>WXT+ZqwS~-il zC|etkvyLrpAOeQ@scH4zz7s-0j&fL&W-oOAn2rnR z=6C&;X>8A8ntj04cKu?L!Q9INgz&boG-q`jdi(FKz_e==Ht%;HRX;Zem;M7cF2BTs zAx|KIIA#>!QJ;XkjIQ7&Y?u0B91J6Si5=^(VFlW=>gvgWnh#ZQ|74k=`?q;cL7kOB zZB%}Tb_LqU6Nnh}kN|s0*Qpmw61+3QftFWa8Vf*369j>PmUM#EypxwmvwyQf;Fq<8 zLXgD4`ZwnJ-w2(5qI3R*E;#}#7+r_|_YkugVB-hEzaw@$7VGLyUUJmz@xUCh|3v11 z@jHv;0GOoq?=T)MXn%0G3tXy1et8je8~WMyYLB^?*^PFrpSk$qoJ2HYFKKE}7&oQC zk`K;Iv%CTDsBG(yL2VuP0d0bKD3{WrE#u3A&^Y_CTsf|DnGS{+qdyWU*}nO-%2MGy z+-toIe_sy=DHjbtp{PvPz+=v*gMxsYb0`CvALDdZ)4>lOJzvk$TG8qk25|coCtCQT z1O{E$7#euxcs%6?K&&VwCJ{Gg;r#N?-B4uSszPl%`gv|J0~w!I2Thnkv<@I_1tTxk zpHF$2inF{JgM|)D@?S#Pa@ZsAvvg*PZM2bJfhnpXV464Yc9oekN+lGaPMT@C>~Qnh zPS15}#7-yIYmpOWps8_(7KPFuffBOOt4R|IvVfR>Kzj6UuG;T9y;$r5JU^8i{!N%u z!P}C2cT#s!UqoDS*mXxlsf3`S#sV-j&YVZy(H(+ibKwsk5yOeAuY<418v=c}$obgTDO=W}V-Q3KbLS#r6pj zRmEP#Tojl!h6rDdzf@GjE2QI0qhe%$Pi${7->G;gb7-70hdm@sqT_0=_d#8~)xC68 zyMLBVu+=vGvbmGmw`!5iwRWDlC&Ge$KCDT_ zzFL{Jdwb2LewDGBM%O_me{?@H>G)Zb34rVi(YmNs&=ZMl{d*4Xo5bxs5H{HJ7Q20# z`quIMN$x;_;Qq1o&%!B7_41{K_vzouYzrPFth3Y(>{E0Xgm< zMuT(empoH+?X(B$?|0wz=l6Ae_DS>U4=|TY$G6An_!&&*x=zG6*tgY0y^9H|n>QJ# z;CJ;_OG)@xW6OUt&famgcxtBpLJJ2_8sd7)w9*7Uq5*$by1B^T{*5p?Zrd~TCcA7w zcmQL#tiQoLZLRxWT^b#CuKVWTCaL0R^-6Dg??s`N9FA#dh(w$5nyh9&KkSJuSXZwL zZ&bng|9Lw5>(@RpfwAJ}$HP@g;-f0qIt9fpfl_q3l0*~VFt5_J$O+X%7j$B3Ja&3) z#?Z%T7ARZX145fR{{wBx3}MM;qMNNK1Q;4@dFLn2aCSU*KA-mD_&G293VHllPwq^t z_I?eM#}V0Aeh*-NSl#ws+I*CLw-X5&kl%P(%$jh|X;`Fb%{FfIVIIf1qmtKRUFKe3 z|BW!cBH2ZbiF$~t>&L03X#s(Zb8otG7H_MfH6e-A^Hkl2(wlpb=$eohaGl+H)Q9(G zUYdQ#5RO2n(ifSI7of@Wg@y#YadvYm9c6!>@Z9Gr&%TUw{~Dw6o%{rba_)fT2D;>i1?LN)uZ13H*^YCfjC68E7bq(ERwJg=jkKrj6U~o|_V^$7GB!Q$Td}FEh%T)AqkcC(Kd!%Iunqoy8Ej(z%3$N8%YZaZ*u6cuHMA@k zIhBcDSG%SiY^Vdsa-1h~L!Q6@HWc9LaR)?b%Z>Oq7fft1cK3!G>`d>-YF22$G{u2c z&QFFFpX5M*U^%=Sw8u-$wqpT)!!8D6ZP=E8;fiHq@rv`Qxyhbi0>Ibns1)X*H&_I= z1dhE1&VjSnIDp51y>iA&BdIZ__KgBI0--i!(SS2YD9ElOm9y*rf-^f@{-;y?(}Oyg zu>dj9UFSdb`=w2o>4Cv$|M2>t9$C+|p8y6zS9gNK02c*|`ls#_02clKOTVq5{F6U)*%JZn^x~RT z{xTb53N~NNxJHHNe{oVmE;DR67CTB$RCswl)>Qq5t(*@``ruhzuzPsp?1{AzmBYsy z`djzNuHsu#76O3b#Tm0Q)vxP0vK8piX2kpSRSh$m*ZJ4I4_ITMgbk~j@4XjQWT)5&;i5p+ogsZWo0c;6>ve*$ z>+cxaT~OTiZ%OiUHbqsmvsUKmXWQLpZgmyGlhS2L!J|y`PhYD$EH7&Mf8JPgbd^Dy zhI`GN-^lTx@|wQ#_&$*AbW=pgE=VbP<$hQvRx5uos(?r{JBq?bvaKbqw9@7>U1#g}@KRYU19xNEr?4!Bxvhs6~L3cl^Q#yxyF@0(1AjLGY*_;>?0Td=Ldwr{x{>6inb}P%PDR2NH0T z=498aBcgYpIA$sVwA{4Ro%X78ntH$mqPTu@XiRDA@psz;v|0S_^icF~#355kfyefaYKKw^>q956?wksL5WAbALU0uU z|KB%|OLT=`n~(L^9lszLCxyhCBUnJeXXxyAe^Vj$)mQfwdi^e?;F%^^tMwY3SlK^6 zOx!kW@BO<^_$M9&VO<5&BYYnno&7c-unh#6)HsLW$2+d7KI57L7bn=KAb&{C-B;z( z38K89+K1YX-EN7HNIhTxCm$2Ww@OI$so;Ky9<$;cCb=mE9<@Bl3lw$z)LxqGB^0|` zHK+dy1Aa&mPQD8{qM-T>oezKWXF7zuf?%W>?xO>_06*^j$mLJffKyFPybin@h(PSC z2+xwgPx$ElH+H|ZiCL!U!eSV!w&0$}rTYA6)AYYJcUu}f5Y^vQ(jR~3`vPIM$~+%I zaroNI$tjOW?gU`xc@by+vID$C>IuHMSFbw=Vm9>ar+4un{ye7oi7V^^R3qMlh_IcO z!7Hom7RVxd;O*JkRe~CnBZz~}KBa=015XkDc7 zWq3&6!PodcwwuX5`Ee%*;qkTBIp7zBep|jNaAJI)GERs{S*9+3)8WS8@fYI7yRH8m zKv`UT0bY5PXhbX{&`1;jS`=+7-!i(}o`Z>XvZf0d^cxyO*)4N+kd^+kpR^_*o4Gl( zdC2C~sMU(A!uLf~s!e#R1XBiZ;j!tFsrvUmZa^X(Lg+agF*xfpfbZY~{nDl|bkl&; zZ?L)tjGack8V~PDMo*mcFeA2~ebR6OgHuRKKw(}}Iy{lO&(Kagu?+CcI0-S`=< zIo#vd6NKbyZi2%dM+Bz-f^5ua2QhEVs50td;%{_Op zeuAhJEg6LIfMa&`UeL7%A0FE++N!2&TO{*96o@JI?d9AV!+|{cVCtGLfh6cuaCj8idW|XM*!tq82@=>y}%A4s{`T^{aCSlLPnxoS!Ed|84>fSHS;NNK(AZ&R^f0c|q(+(h zcEUZT-yK5QX0P;=kDPow?E$G`VUUEcj%T*SY?yH?DMu|jH&x`Ux&s|Dc`+a;eh4Aj zOWwqevv_{LnIluDmODUhW+!DX1}X5F3GaY%SJ;2xeHM4m^c1Dg=v1vp@fKzswfEo4 z9|)_KZ6(WLQB?E!vEnnd9zC8G+K~Pi9w!vKR!ZX0l>8eSojU(RW=IDBGx4SwiQsj} z&%tA`N&gfHLr(7wE)8-4tZh1~UlhF`AY?fR^l#JBwy<}Z7Xz`Xo(abT^g=nTsg^e} z`cD~}6wI>}Uq8A{SW1LJ_0(961Yo+Kl(Bg*7TUYy8SijqbSin6H@#Hm^eJx1a`|l- z`3QO;X(!we@>i0aA?L%z(fxB|)ExY~xwyj__^}r4SJGb~5IP*EJ@7Mlj{0P)<3Ec9G+k@rVJ9e(KHvH3tPVN=pd8LMr}ZRf|LVdulr&$Hwt(3 z3D|NIYVJ(dzt=~0Ihj_BZ+4S`i<)8yLA?-H@dLdU?W5oB6+(IJ)`AOJUmg?93c#h4 zK$5=LEIh4jvxTF<>Pnr%!(X7}pcv?oR){H13R=!<)m1X&w>p^_J1lv@)*(;{(fXib z|6p<239zb83#JZLq^a){bdE|7Y-HwAZxFP-6i?@Yi}!M%Bc0FS3LJEF6HSjLqs<+_ z_3ke#gg!h*YQydOVLt4BMN~)m!Gac>391Ahpvyb7Q+%T)`$M0j09d+P2&ja- zfIShjuXxA}UCAYuyMM1C!=^>xZxBVo2w?=gLG8BYImp0cTds@75PC%^8bH5QCZB9| z(Nl`krlT~vq*~ecy*PP-aOHaMP}2ADZJPyoB-^>YJBQ~JZ zR9wgxq1FGIhw|X?BP*M>ssqu|J!C2yzY6faH0_7C+}Q%>mVaQ$#G>}#1w<-UvklJmX3h6-5S^vT9IZ6Q@Ka(%rg0@&a9uSvhMo*;hc?3v>Z?*{L~=b53fPu#0_)ZYTPE=xs@j@=kkC8cO~iZ=+6q)gIxAJs!tvEgbh7He-Xx=F z{E2yEHDtUGl>%l{r&DeynB{R`$WTzPk*pS0bM42_itL(@!GrM`#y%N%+NZ$mx{+Nj zMP2tOg7+K(=nBk=$?o%7`gDKh((~m^SP6@U3r);7c|#dAN@G?is4$qjMhWn5B~e8F zcFvI6^vJ8UO}LA;or#la%lu zyr~!b5#}hlaP`JpdBz>=K3W1}cK2BXA7yUonP*ajC|v6udh(cs5GLQ>Cs zJEg8=>dZgT19khzm{2wCsC zpOm=64-ALSCrtS0_2VqRK`jWFLu63XW6_C2oIJ&La+mfm4<3fm;JCt;Q!RY~U8pGq zX+KG3fRAPAA=f&!XFky04<$9rb=@@(2r47qJ*_v~+NF8(ot!FFnC6!%Uy)h@YGGNa? zh|-7PQ`pCiJZufvD&+9|k%^6F9zQl)Y(g3}QKn+*{1V@TT1y+cI^ec1EfXY#|E{dp|MTS49f$leGO|+PztUq+`^`7kp!a4LoGyrBxA~0(rH_ zSjEoc=ZP7&T6@D^R(#!hVN(Ia9vfAa;%13Lp-b`0@ZB`nzsiA_KwGpbukU6_oZ$u& zKp(5mg2|Cc_GhNmhPErq(GP@~f~_tSGsyuv#&NoQ^4h6P2dcVqIAkdDiY8+jl$n|= zT0XF(Ks_;#e~oKXjWk{1!O6|1nZvvbmI?j)&RR3tYBzK(N7!bkP|v??&aEq}YrGg^ z$1fZgVXCD`ufo}(WkC4>zIfXuV;tWySTkVx-P!S+JG=Unmm)E)$c$*oPg((+U;B}I zq>xh!FGbi@kM-L4PivhGl+7l?%kjVmX$Jd=_e-*9P%`qC6W$WPzzx*r<6>*%`kom1 z(H{fA8@RI}4vjCdx$W3)lafeL*tGeueYmuMLrJCOYA~yzo(pwD!&LE@o6~O=?3P*% zL&C@qJC+{7Q6_yi8fcP|=qTV%Yem#g)VDvQc~ktTZOu-94vUn0gx-O+lA&jSYChDh z!8dQG#X?eyS(3}tSvEe!WxSG(u#7@MZgj=bM>yIDa7_!5^1tAa;TA)R7tWZVcl<}3k7$?wL8=yvn<}yy9x}csD!n?m!SlnUgl5ia z&Nf|q?jjJ!mQBd!LVUJZgV;N7;ey9|IGmE3vuRu$ctHsRwO69P0MM7Hny=Wq_ddy1 z-XUPOn-lIk(;Yb-olM^~?FWp#Tm|$TOj_+rX@GkUCc6jV`0#@b2o%2G#c%JBULSja z&g!tZdtEDd$)ZFc)_RpYuK+x*q_IWuus-L?en1Hp%V_EJ#nWL>vavbpT7pTEcz%cf zMXLJ!j@qb`s`L4rT7i3x@qnc4gt+}U9= z$A(lU+9)K&_73mAVc3@r?%$Vd;0qTgLlFxU9}ii zLxoJ0{CL>vCcw}k(Y6$#hWXG2a{^!;H%qZeyagVSTqkgU_25Ce7Vt8^N@f2j0hb3B z&8XNtg5v0R1-$$_#m%{gEW9q46;d+EoT0y{q<%EHWh5(Zm&bbnJ$_Iw>hOYbbulqC z0&03ukxGY7F8>Z=n4!>xYY!hQ^WgMyJb$mp1zWpOx2rpiylGL$t50G_($dYOVkv7^ z{HJ))D6D5;zd^J%^oi5KbK1MiEL_Q!8pIER>SHoC}o9PO<;|ly|;&fhTi)&`%SB6ej@9syRfr=huz=#BY9i zqdT)kh5m41x#;tKNv>%nRGzS0X5@x2*2xFRe3LGc{he{_D}ekp)%|soq_7g? z7)$M;%4!r_Yf>uk2oyS2y%M{g{>0?Us0&G=_VmMi?N~FMqqxxz#bJ|*=S;YhjhN3WnvhSts}CRF>CvrvixQ=6}~8`bm%GT>#R+ko(+u? zKb|L>MJ}{lP_Lt~r&`C7V?XIZ0SIcP2Su8;5g*7&=Mrkkr*pfS1cD#?Ec{RL7b|vV zt3FBrint*{q9=-VY@#T8Bzrs{{wrKZaRoY!xOL|41*{w=Yt(Z5=kpMcFqjEDo9&vs z#3Ayj+Jv|PUHEl% zaQsgkZn)m)F~p_t691ru@px0eA{wU3vu+A{ba{qemFALoC3pU{xJ-_U5?X#^zjyNw z{0V#6lR=Y?{_pTz2TEb6V0mX0g7T#T51^HifslGsGl`v5u@j&9&J+!aV5PUev)?8% z%g5>lVR7Xl&n&gZz2%+T53p`||$2`la(m4O_ zZ401}u{}d5c4fCyN?gF;YU9!DASe z4>JvI!(zmtSQppT&!kls;iLcZ&vbW0?XO=M1^igFWawkYD_F~Ih1qoDxQ%FWn(lPJ z1aok_>$&1Enyl|z#{woZ73JuScCY5ZJ;}X?G06g!wp`l8Of6c{<+8hHM@<8Ebw{Lx<(#spI{%S!^243sD{s3bGU)3#bJq~BV58^2E`0Gb?}C0 z+4G)b3Y- z0IQ{}Cdlv41mNd{i70%d)Y_2ip?8TdT6^8`D`Td|L?G+N@{8PQ#boaBvg4O$ zmdcA4ioYprkwOX7D|YN@Q7M1;`}v<&P&ag+4fZ8Q^Bf-U*hr3={G3FT0NM~WKO(Th z+f0|Ouu5O7%!Btcx^naAM&E*#;OBX%N!NhuAmK{1MeTI|{65xB)#UZwK@~4_to-U4 z43L`}kC>{SD78tPJCfqfK4lE(qOAG>fsM#wD7VB>C2VRX4*haY*yK(B1P)m9aP-aBy#TIDTsDUHHnIG7q(gjS3}38P4(IFE#Fh)-_AZ!8w8 zMbtYQNKA7g%v>i^9#ELU0zBwQT#PDd^WThRTe7YsRSRq5XP`wIgX@b(P+gTbPhFFk z+a;mZHfEnV5Th+MKdt;dHKR(sK1LTD=9g7v0zYylI-|hsw23!gnX-(*6~i*&R4BVM zTwaXf+E{V&Bn6aGBf!xsX1$ix`*4<%b&!8038sgh)Y^Mo`>11zIKOx<#AN==IdL!A zC_C>#VQo%GPP)ihjS&OABX#=bFRHtI{uOT#QJc=5IVFFS+*k9S=$$dE8-pRNNKYMf zi|J@MAM~0;-w#&P4Ml{|I#yZ6t?ivueX(RAOP|zhHd#`OBhvXI-&!kt=ljuHupnNO zEmPWhaL8m9zzo$_>X2x`-|)OzdFIi@2Jolw-%fS4UJN-LYGD1~)%cN|$t2v?#RsYDJc1fDP+y9;zY9=Q&i=T`H?}4(GV}3 zWyZacVm&WVR!0$(Ez@9yVWQ2A!6na2%BwLb|feYNvPTI?Xd7jtOoi65n5Irp|k*&b{*N&ih?tqT0SBvOEybtO(g_#*?EO(Se zLV)=lJ1o>1i7Yx#tRgWeGRPa3QNl)9cm^M}Zg;KD4DEoz27_W+iDWG0 z7o3G4q;G2-uQbe-mSm7kg32vdb8FD?_QSWmKDHRgCKF=Or2Z#7VM7+$#-B-Z{faPh zVqySYt#+Q+ziY7-^mikxDZ4-N9i*(F3=6B1F<{coZn#a~qYV5Fa2d6xlqWNu6Eh>5 zVA+sDSb95grOlm`MH0-@grLh{o=97#vp(agwmK#me|6r!-pKM%WpecH$p`GdS)2T` zgNM>s`hfN0!yoM^M?n+ph4I?HP0V+cnV*{?F|A71zH~|g23~N-Quz9-+Jrg%EfS-8 zZs3>5hNLu!Z#Gni+gLuKVwJFI#gM_u7#DXQha{eVavXZ}fM|b_4+igUmJ6tvxx}I| zY@?%+Cn>Ix3O5~cBkHMXr9=5WWop$U2Fv?O(f0`(u0>A=|HY8<|@~ zOWUuST|kCUJbAZM%0vU0F(pZ_e;<|j{R;?Gw3n`!W4d%KGZD51i(5I;mgrEy`*wW5ukLH>7iDJnA%$Md^C1yS>{JBvDkLAlVgm|jrEj|rPyqT)O-O687#L&gh}mVEUm8}yVj`-%dn3yv z&$k<#p0)4Cp8Au7N*)bA*pLq}coY^C?)@geQL@PLeo{z#IcbfO?l&tpAHe_Thua*- zcNMC(F$1cEk0ZcaQsDjiOK> z0LQc47HSz+Nt%7JjdpSkSB5Pi#e=q5j}Q@ZztEWZZtR6HD-CGRb22Am>+V4WiB`KKdNHQ3y0rk6vC^@lN0a8tEw`fbWxNH zT+bH908zTnYrw-dkT9fGIsExE)AE7&>$nExS8j#vVlI1kYNu2m^nRkA)pDLp=s+w_ zPe1CDCyOUGgGSkAIz?p)5W=JN5NrU`X!dxv=6HN^q# zevQb$N~Lhqp0x7?eL<4aiWbUYrO?admcH39O&+lOn*=Evd2l`gD>N68<80T@(;sel z1(zOyn0`lN625ZSVd~o%&0f^hK%6$MQnJq^CzHTR025G`Z=<%FKHR6Wrm0U@;S)U! z>1RiA|3bKUj03lGK!PHb`(k-Y3KK8lX%5>FfaWYrRK|9S%GBtZ$}+ss`HQ3@w@w%Z zIQG4o~OsM3s& z3}LOZ$fU}ndW4kKDZ*G&M( zn(W;WNd?lYCg{Qg(*$OyiIMb+Su%(yE1V5<7%BY9tV-Vy z6lJ2N>(=_J%Z=pR^Z@^d3(9iMbb3OW-0c-1*jmUK4=0J6wV9e3YCZ|kMT$B z`xGWWx2rke3bUp|XH~}0T|_t>3KLnBo!nl*GjTjJrYvn#W6BUstbr|QsBLIs%zJVzg_a~{lefMbZjj(a>%2Cz zV2~olw2kvV-J6MlQw%sN?b2{rXF=d$=~XUw2^Wb+=l z|Lj>NUa3dSDa@Ye9h8GRwu~fOoh@05dPB1yrzo>IuNpB<|ofGD0PzUC4`4B%jph zD9RUl!&12E7K1K0ynGx@*0af4#m-_{=2c=!m!erNvbSPu5%ozY|6w}Jp_)WC)TzeW zB#JqdRo74w2dd6t$T0a^Y-JMTySF5ky?e>b*Ho_=>og<)3vBJNWq*nA#GofTl{Qc^ zUBzj{nv<aw8Z3GF8wZ3xFUl9uxd3A%;g|yk0D<3VIV{q(_AI7 z!i)+ATq&YyrE^O&z1kPZa=jia#eRK8cK5SmM!+{Jb5VNq@=W^@zgkE^?jEs(Q-Hgj z<1f;2D04xt-*s0vLQ8X@!d>VIyI}BBsqOL4^ZwaoPhO1kfLFi{%a@D?G~{GKQz0YN zSlO<);wkzBXO(*4m5wjLsA{1;ZpQ=3RDC_>nH&H=E6w8&YqI%s^I#%6Dl-+A;W(|% z?@?dmxk1)*zrf9WbBYEuDw{0o?zc4Ou*eE9@xyOYy$hGEF+gJ-i0!-SoTxWD&5(gK zA_gE+rZ0j5EL?loz_}43Rt`zN#eU&-vvLNF4j*8>kIBDvS@ zaA7M=+HVD1;X}45M(fnstYQg8ygAX_vFQ2Eh9^EI)=}i-wDkR;A>t*q)b2$LR zGX5Hzz?*~Ei{%fpb*|>5O*|gOAIs&XoWV*)y?P1Iep(B-G}@xej>XuWswJ{w=Kc`q z0{bQ&*RTDW*`@`FtOAE?w8^2=e+qDc?g zpI``8gQjNn{c50#m+~i3*l>-feWpKhRV3Hw<}3sc!N*Q*RgIczEL7k>Im56N1q01( z$G&j%TGRP+Cl`h2*CHh!h8btqQB_rx`x(TGp}cj#i4Y3~Wn*)>@#Fw&-wZr%?0V`Z?R?Uf$$y-|UU zW^dY+2D@Hhe>q@ev!TV^@yq_M>ANVx(In;Fg*(XJN|Xelf0%Zym+` zG2jmuLEh#{;@urv51^p@qCsr8BL5`wyI*nriZ(yqqND~HQ0W7|KTDH2Gw>AtcCeXV zAq}aL)5Dku*Xda~CT?3D<%+Fp(a%qz@uy)M|8MW4CXPb2XmgUFUGZ((hto8_>EOI) zzI^IKFgX%{j;TB#2es2vS@%|T+6M_(zSfUPNu~LcXny<^qf&TI&u*X7nN$@u-}@Jx zD)N^f?Xc?UuAk1sVf#d&dxn9Im-ql%>n4fL1v>kHs}UKXuXbh?d-TLI>}ZlK7nF)2 zy5x!5_usEeHr}v3l6|Yif~techTihx{Ox|Z-s~xy9q##a zte`v0@mXpcUcM*fJ-Lx|#8@mvO&yG$aFykZCREZ3$viGW0T;}p-rXPfZfpp9I0D`s8 zYD#C7kSjg@{{Yd~bI@vdA!yC;3EpIv|zWHOs&1 z9LPCzJ8oS2#4;q+*T%T2dc0xRs+}dF5u$$b^$I~fB%GU>{Cy@mFIhX+WvNq(^VCv@ z=z++_Z4GZvJ(nou1XGJUFC5)K+<{IG9qEW9dN~=jWdqMe`e+aWgElaD_NKnP&v1Y^ zSgRk8?UP^8Pg6gKZc&>bG}vXgszEB8q=L?;e+w8K(OKVLY=#BTH1mCG8q7pcg~xA$ zE3i1JAWy8@nm-=jxwS1b5~fWcwpoMyr)A;4iFpdM5FHrY&gZmuGlfH3PBj_6EPwg#e2-~b-8x%)W49F&NHLV91a@b)!uNe>!UgqYxWGyj zar3O4GD(;`EY;O5cE_?^bl}2dD?`SK<*;D|Yd}%7ZJ;jE&jb;8zlBKnB4(WLAI)9U z)J>&y!;)8URup$_WbIi^##q|JhqPi#21;9;kb!5h7u=t>2z^5I)4@5Z>EI@a7u(&| z_xDhJ)laEYaDgMTf=dM4de(T=%6kRUEBTlCGuQ>vyUBT+HOP8jE?OrNYgct28|x$wc+YW%da3y7{koSs zE?Tvik|391;N|c22)%0V(yGr$?PY^yrFjZ6{O%<|Q*;8`yPlFxkk=~R_`4>1UsLay zdq7T65YE@YFg8-v!z34N7;jG4znej4COgcP%tq7NdXs+@&8RWBvYsXX8wQW^{-dc^ zt=e_NaxU`(OE?LpCz;ona00Q*SCJPbAz{V zzABf-Ooa56Zxw@-qOZH4K*^m51`%OFdD7>KI)z%M{3=6w=}+pLJayH_oLqe5keF^a zNQ^CUVE{(8TuyUNj{mLM`)x_f;h>p%UI}LO$0wV&U_eJ5c*QqlJp}8&c4`>| z1%b5IsKIggf6RM5SinU&*^{s5CopfH=I-IO*^O1+T^-X({W`v#_I2Z=!3oq4D#HsM zv)>LoLI)lb>Ys{Zfz@iqVd62s4vrP{POEpnU&givRNn|Zu(Ay1H+2vI=fNgXBrD4~ zr@?vZKJapBx2i;+>$Nrix27)$8|ZO#>>Nu*Y?xtnV0yFp@Aeg3ZW0`%ZP2lCynt~6iggHW8ZRLlSRP2&brh@*E} zB^iJ9p@P4tpPpS5E5XbOetwcqIR2VE>>I7sDp=0_&xm-xS6$buE@Oe6n0()Gk@&}p zZVaJ2P$tNq?}(uc7jz&04qM>ts@t?ba%`XlV!UY_A@iH1jA+5y8SK3Z8%z*LH->3= zTTAS-$MvP*2WY|r?R}UbPlb+pU@D*YNV9FYv<4$=#S}Ns#$L8~)8o%+Tc^Ir8WB~q z!(+|fYR38^Jk3)V=b%hV|G90P`%+`H*5hqm)B6)ZpTwmP3JhY-e7-Ls57_^(EqtZ^ zR{)LC@DenRT#MZK>{oyLx`Oc;bl?2}L+uD;-X@V8F+_%+_=2C8&oY-^{%(JUWDCJC zcQ2R-&U4uo%Hyrg5IX+D-aUe`{WSgOtaWAP)8m4tKA)(vJ|Jwp=v%tZ1Y=^DYR`7^ zn{T7$&6O#L8jQrl#0wl{p3Co2CkW5~P6c_ZL%brf0;%NddK^c;bjNBo-!(%JK>T2` z%YQk9lQq4uke;J}8P$)t$6bB8%xq;4PKp!Lf6)vEcrQyMa>G;W+;=r+dPnNFZ;uN8 zUFk%S_%{?%A%BTtzJL$D?*$!tJX$YE^tmSG@%(#;4=7@r)_+pS{1e~+e>TATpX#Rr z0x;576hSf|ISO$Q5+WY`05RxfhGehTZyV(_p96mXvTRRIs%Mn~nu zX%b!zeBmdSLZE}qs#Z;l#53d9AHYSw@$(s_ESb-9JZZT`_KnQZY5F0rAYkSA>-V2FJx+N}b_kTXV};LJ zF`X@HIjEsab3OYGV>A3mtAo=e5}^6{KS*Nx|J-M2OHjuRhkZr8F0jO_=y}R>|Iz>T z@n44Gh*yA!zHB;;Q~5 z0NfdW{U&^I5V6?}Tu2z--$+D|fBGTmYf|nJGGU_YgahobG*&p|Kv1X6X8jn?ZgqZu ze{0<6sPqGH;c1>}q+lrLc-p*u+t*bK!SBFP0L`(pvgzac|BRj>-ZTLUH8<1@Z|1=f5=>TU6PAjD`o@2sf%^;u zl~Qp~-6)I{*e$aB{~Ns_0@MHEa6K%nH)I31=l+)r^*_V&gcaxlgzNebYzugUJ2Wnc z%e*jG`*BDXv14Et|UM@ZB=?7ATK3m89__&oU!O#}rfnw^AyaJp3 zgq(;fW~}-5Bv(zH?jE2In0i^SI@GMYn)qCkw_A|3MlGgpg8GcFw`0=hsdzzTrWrrL zU~E5G|3lHI%xgXS#l_;`rCjAHjKJfStsEWCHd5d>*lbH_Oq|k~ff$(Z6Y%~D8&g*i!)P_Gyq@n?< zixJxCT3vZD3IUaWZZ-O~>=lK+$HarF74O-yRL@N5tTC=pSmq+K3f(oFkZkW(Lq)Fn zJJ*M-?2n>OOz*D?eRQ4ru$xg!E`wrkEYcBWzp0)g%v8x6OMhNFptef-@@mMJgyBdJ z^}+RIJ#Dh4!Xkaeedia<(TL6$gUgl0))IFuP<4*=)}xq22EW`F>vHFAR9gS;;mObv z|8M*oY+BjxlQE(qcGfB{9QIfR5em(Vqcn3zlmj)(R!j9`kPh?S*5K=J+~ei&eSNzS zN(G$wr!)&In*LhBLzX(dK*tZ;r?cjn-3anto*%ZL*>+L*ekbJbrq_1QJO5U|MPFvd zJYK-+Us#j*Nc`wA{_M&=z^_zh@(mJl{@1A{dQ*gP4i1E!oCS-I?I4VRk9H)3yoVrm z-w*x^bM8I*G%TFmKp6CU2sBg|1`+Ln!`;2~S^k}oGO61rWdZ1#(>xC$e=hWFXt!Ix ztUtM!{R5KDV)y%35LN@^W{PC(r4IK&pPQob<2+pwDq^jilyM?%(^nTjpv+VJauyBie9^F!QHTV+vSM!&VBafmyHXqm z(v|WkG+EAl5%QfJkADxZA}(h8=LH{mx&RrSnj#I?DzB zue)f@93<&c%(A2&smlh7pPp`@2UEZGvOeZN5HYsaU-DXfWk@RzPreELp^QBhCk-o@ z%k%-aBRxw_ktE<|?wl)wok(c}|2^V9&_PRNc&@kfQRfr4x{cVx8eQwVM@rp&@KFu3 zux@TX(E}<c9Cw@IPJnW*WaqT#8A-?OXF#D}c`_ejlL z&U55Z9RmUeCsz2tpI{XF zw*Othb?$LPJ2sU`O}ASRih)T&lv@e$rkuSXywBE&iWkxI69joPVaA0ydAS8ejWu&| zksCB%QS1xIMlD*@LpYRgATno(oMQ7Fn+M6sqn;XE1QRTZ+!@Q6X2z~VBv=q)e8UbB z`-dQA?Q%xSsL!SJBDS+T5MvaC-s*?pzXS7}Yh>KWEtTdnFA!nG#R{ZTe<6u>4F{?d zBEbt0YzIYflcOE)PmQmyF#cU&{Cl4H7Ym(|#*G>L97->2I{OOpH-;+F17DELUk~WT zI>B2=;ENAP!$I*Ew6uYDR>tFBh?Z)q*Y87@;XVkdfW(b6RK8gJR~uyaL#q;|K@laM zNb?A~N1UyE@*sBh{kP9iXk;LUW&}FgqK5EDUBHC0Cr|2()J#^H@aH|J{9vCt_cCS? z@TpH$a9RpxWGqJ1CLeakGj=uePwcUMZToy&jheKhNTWsWWvunvRgmxTou#J59Pfq` z1bMeVzP~>fY<#0T1SvpM0Xj3HDne2}5D-itRI90u4?8(`{wy?yc)We8-TpX0(3^N! zu0r!$f2C^~qHy+xDp-UUb4bC4Jty`1Vip{x%6 ztM^^1e*62l00bUWx^_41c&4m@LSjX$sU_8xq-p!srX_#l`a973P>Z<>0q%IXf`ja-y!U>X#p=h$YPD1P zf4<}zVr*~#^FpV|@M0i$up}woZS>nMV^9uafpm6_`S9bWoUgTE3xr+&ZZ62$2sJ%f zRYqIZLS3ZCI@U=e-!!!=P71KY#AQ<;IPQGZFsyNT3i-_W48lx2)eTDz5O3&|lV>vo z7Sr8?ZgdTVl#LJo2vBZ>Mqt3Qh_Ao-942rA*ZXmy!0Bwc`H)nF8wDsS4O5}G9 z1t59oGITAv9b!18*h^*A|r6H;~$B0p7U2 z^-J*k5uyj*I^5v{(xNqVT1wmf0{Yj7;62+MXc7_Kx(aC?f`qzUx+eIM ztXA@JIipjOBaIy8BsA( zoFv`pn^t5i7<)X|$g=sfK0^3_ z62!qlwJ+BXRXsUcH2B$e=v+(udx9Mw2nGv&sEvuokzwjP^l=zG7qxFP$0N+_YHt6^w>WGCalvw`##S}w-_{gv)_CeGsN5Bui-$G6 zBMF9iFxhX7X90^X7p7i9*V9qmLXR#-q}}9k$k08*%qt5Kg{D>8&CNOEDIndY zcx>U4S8%4@fjmm|Y&{cvl2m|87NJ5mm=5DDku)`FdIoK(qRASCf+TzRZn)0uDeYkk zMu+&i98oVdhjLqf+GuPxd|4;fYUZ~+7wHDAzqo1;~3YIYuKZ9yBV z;Q#RpF0CytPbW;B%jUTjvush22{-5!h~|U{E+a-GFCh8up#C9&(7cuvYMxY@w7MoF z@QH_0BM{>D)PV!+XJbpmxaoYm0ghkOW)VjapAP>71P%q=H#3hAoM)*VdMBNw)>dXj;YSaB#&r1l+1+7&!5 zCL0EjTZm)h^Q6jUI-Kl~-Jy13K!Gk)<0Y1jo=Uyd!x>Ujs7w-05dU_yN>>JFQ_!t#wm_q@#zyUgs& zM&~^^Z|v#0Km72>k5}mVoAJ}N;}D$KSZi|bHBbVlzB9=L-VV{;_p4yp8g;g6TPnC~hPy7`8?L^hga^Pk0y|A#n_S-_ z*&->SX~++9hwnt4i=J2c&r6QsTDVh1NOc1Z9SrAKZnpGJmCHC861GF%vVQj8eQG|d zx3~^hV6%(JM4>64eymq3!YHCRWp5kQ<-Z}i;AsedVJQ2co8!%Gb|T9BiYFr9?SujS_frIg_nI% zQ}gj{$)$(4{ebJz>S?ij;hy`;;J7L`yE2<{DmVlCU8tufAy5ZyKiRM<`l~ZGB`EBs zBT7r$Kt$9~7ueRXAPX-FyeU{L2E_NiaKNv;Yp`dz{j%q|_(?LXDv(5bYZuKP?XtGHK^+`XhxTT zElIS{{x@K2Y|wnkAy)-VUhc>PODE!JESsZa+IojkF1i0NPgKn+e(dPAj){#0*nxBP zZ|BXgf9m#%i!3n)#GBxEnFTC!MG5En+PjOC4r>XZrs-;?kinT}ha`U11LrlucF&T$X71 zJkQRWPeuYJbBmRLu09IRRGO5r?L5PpVA~36Q+jD{U}P`70^?7DTYa9F(7$iqa!Oze zCy|jPE~K-Z@3A6v<9(7l7-|+*SU8Uk9V9YbAB&%gb)q>o6NYA?=p&$BO)F4n5w1WI z!pMf>tphFJT#FRDprz2OY@@3miRejpKPnC(z76JPK?B#}j9%D3DaIf)3f|DdL&?6p z4-*!7wbIpvBBL9dRJpZS7k>Kg@CVYf$}x}RqX|-T4Z5EBtUbppTb`^bmbs~2K{LmJ zH!OefBy;})XbZnl#ZM!W%N&|v;JYrqzN7<`XAU>ed3O|}KO=aVl6iRlDl{_-K5ncgl(lFiy-LnZ zQ!V8P(U>*Yy?7E&$o8mb*N>^BM20>6i8st4^u`ZUtI`lj=j-WvEK`%uVE&CTqdayX z#G>r98C@s1n*Mb4!}g;#$!Nvk{=VP*I<;(Hrqi(zVZ|f4A!j6jVp;^Oyn#XFdRrte zt#t9magiNwh#6LXK-!z#->GZ7XHG}M6G zZC0O20Qw$AW`yJGCOutguVxrpQCy5N{4Z0*^N{)hizaVDb*|e|CFReqg1B>UU@k1m z#D7^BMC*mjiqRyq1YT;5!ejCkE5UM1*66f(A8aWEEoVDf#OkeUlR+IJ&!xg6EdoG6 z6SIV1O&X~6O%T*S7w0jMiPN@|S`yz43C@oC=glg# zjmV49Rn9YI2zw&DQt4Y+`rkAc#J5fgXwSB(a7eAHH_t7(z$i)SoUKyt1yhZwl5H-q zw&Go`Hy20P*pAX?D($_rz)1@5OEfOuFbJWVVBxsMGdkm81*T%aR$Ym7F^L6powQcg zGwvhCW}vZ~7(15$yYGx-Kd0lKvw|XPa{mazQ|(U}i)G%~3}CWk0pnaVxJoZv*#G9& z_DZJy<{2{0*Wua`54DYC^jq8`fM~O2=xOrj6UvJuib+_f9S7tI|09RN$?nqU8=jST z;k(pKYph z;VCpUNp9B?ptjl6cNf!7#pJcLz4=wo2ry?<*yz{ML*ZZ%g+|)5QEGHQFhZxAI7kKS zU;y?>1ZoCLSoqVC0~S5QUe>!6sOIm_O?9p+1UYbiE1)_${P7XOUR-Io&$l6mRsPL{ zXWK6a5bN%K;6tt3wS!=rXYj(8;8SAHY3ig(qSTG$)o5RRoTN4t-%B9h#L3|Y^>UDN z!87f601^PZ4_*WpWx9WRF!yVt?mS;a9$Z}K%-umIKt)gXy2D5=C847Cg+@^-__xo9 zaM?>V+cqUW`w83|MQt^Ai!4+NU*#Y%MVL?&I`+5Bw&d~HLwpvJ6E42Fc&&gz?*ur@E#CiltkdfSIERQ78`v-IP zf-K}pf*LJF>C>onC%58@)K!^@w(BzY{DKE`GLwSM`{}yri1{QB-)oLFCsPj|E(KDo zr>c8t+{9>a@yoI2!)UIeIxJhk6w1`X^(QwDie{Ah zjS;%Hy%NJ63^;QBQ+rv$=!)k}L1e?(71Yv|fe!z)a0@-e2R9I2hL}udKItYz|0fKl z8*1AVc$-1Tq`FCF6aFbY5KhNzj@jddjclK?66#1Dc)woN)L!M5X|4@w7Z~lkgQ}v& z1WZudHxo8$8>?-@%bW6Ijt{lsHpk0w9_0@ldWy%~0wIzVooREJ`jC;MJCl7EIJ6wc ze<5&leMC#YfswGHU*%<*y?v|tFk}luK0Ch88Z2c_fy|>e-Di$vp@|sn_)GhG0M2Hb zv7`CHRXOGen|H6?T}L^2QPhXc%&)3TgS59{@^ zp59r*(ywA%>%fs63{skG-*?9RGfZ|7SH?tjrt_NlQj`^Jro>mn`9*B3MtXRnxM_VB zM48~y4b=;qQPPxPr3sPxkYh}YsT%T+oJ*H5A&kv;2Fz>`BnJnVYkQjbT#mo&x8P;4 z<86$ba>o)OA<$;cB#|c-<0}waCR;j^m&%Ew*ASi!e1j#co57JH$4IM%A3*AHIk%e3 z{{;pD{el&0*UYWf$cK$v>+PxD(Mjf5B!@(Lm?x?4KGi~qk?SEK=vX~oT>xifFda*7 zGf04WhKy0>&y%NcQxCHBslCQCo3Jq}Pu{u&uf0fkR&q!arkviBZR!_F8m2!YK`;|x zdHEDq87LufMpKk&w23*&$77|D4iId9@EE(Yo*kHavK~gmE)LYctQ)w06XhMby5?;| z=MtC`T3gLQ>6{tYa!uj_->`&n)i10Tu~VQUH*H4p>HaO0-)3SE#zj0j^wVwpjP)Cd|BdoZ z`%Zxx>&+=B$g^hcW>YG~AJ@2}@_+d=C3h;PwM#Zm#%;cQpy+tADHn#Y=2s1?B#{VR zFPofxe>4ZzXuqZsX%T08=!XhfC=!bbF1?zGUrK=zi>V?|mi*OQq3n$ImpJB+)k#em zb*QoSBMq4KVDokAnGiyXi6}$KGcv+Nyx1^wnA+CT5_XeF`C1qJ9+n zHv|d8o@TgX;ZN~0=*xWXT`QzU5 zPei=X1Bz-U2t1;0T1*D1%z-0+p_XDTKih1VLHoHgDih}48p z{@K<&?vMISk^G~myrplhgf{(9YRs(uP_6byzro^*7;KI~CcNK-YmX03G`WC*Mb<2EaTy ztz>Nmox2~Z^8mfJI>}Qq!>sO5wwjoPi5sB(I>VCCKMFNKo`nmPs62hJ5_5p#7F`NA zY}8XX*%gb;SfbRe=?|`6WY>@5%$9%R%q^J2b9JFwBLIkx_7c${IY>^ctg^7(v-3ED zg-+GPu?74Bf!x+59AIk06!RfQf&J6gC;7fIX+jHwrlpQvH(6bX?G<1B@>c^O&Zs@f z2XS{={<}?_ktB~m7c$0B<2e9Sh}7UfuoOq!!vSsI-koJwn$Ip6R#(a+#uKspEjlUP zb`Y1|i!aMTE+PI`P&Vjc-q^^clNwmx;y4ewd@&*3t(XVO(=&)j;U~^T^10HSr(*Ob zV!`0&?G@vsXt&l5=9)}uf9E=NyiYH}>iqN8947dS7(HqD92Y}~LT_AKS)nSxZdD-- zFWR@h@XMg>xS26kA0BQAOHN+mS+EEf@Sy>`gkZ%Iu9&29 zEt>0g1c~I>+&|}tJOa(v1#^jIoycGG)S-EYAsrn7;teesiFR?Jsv-1-O50>xEStP+ zNR+7%Qg}uNn17?rS>XivyTx4ZPCFyS6i=-_Yy^L!S;I{3YS4X3cH{~?A zjCH1jV{Cm+>*7-^SASGCK0hyerc>!hUHX-X4k|sZDZ@pRD{@|6Hjh1gs zPC0p%5f0TB1SU8xpF7!pA?ulGh#f4(0G&dk;j_rmdAP8$y$l#8<`X>CwN(z-YUSZG zP$_O*dO5bqe^&pxHbhYDQ1ivV!&|oY@>6E%I5Nn+8WPCjApnp+#WFsOK4T~_E~)29 z{fcnc3?D6?FvmG?xW6KA zh%>V~xBlFe#Ba^i*?$x~6(XSv15AY|R@5H&GQ>K~MIU_fIPc36o}mbpW;mjtB1{Cp zKsy98MmqY>i6)zIzo-#>flpwahNmmo|I!AyctpXojkcK4hz>^MqM%EJ{$bmk5#N{n zRbs(GLZhrm|BJ_TyX&tL`j#F!Fhpe~2>75wpaS_1*#-VtaX`vPom%cN_{LA^yvs#m za%3vWF23vT=zn`pTVeoS$r|{x?ZR+*9Ug9t=jJl2Be*^;NhCN-;v>wfpj4=SD+nB> z=!Wp0tJ9ZY+g)j-Z1!)BmpiRXdvXCVdgiNM_{TEW170sGgHOhsiVD5k?jhWLeN#e? zjN;l#BLTFQ}iJXJ?D+4@x1Hja$CyE258iyP>%laG+Nv|0px53g5`39hE$ay-3VvMh&!8LN0 zWl^`lDmVYiCRoBrUvAN3Y%=dfV?G*P#b?fE5NQ@sLG`ytC!OV+4y;!9F5yTK6X6$* z=pOc#y~#TMhqH{J`-}~h=DBoOplu7v@ZWe{8(b;B�G`NKQwY%>%(|Vq$whmdu3V ztY@xONUP?bc08rn67AA#F|un-V@qW`$~$mQpi>k#eqAfE2en>IfCvnC*+8&~C@@f! zVgb1*xue2my-;o(Kk8EkE%LsYK^+FD2b4n%jpzmV_}OV&oG}wEYn0P`{0itcU?N*A zwc4^C;YT+&!3*xT-UxA3a1DQztyY0;89qORmX*|m19;_LED;DeZY2Si^32VpkMEiz z@XZhuwk>=_RgSf`_-aE;F5;6LM*UN+C)5zDu#^-1D0WgOI(z9;^Dp7{-%QSELPwz=oI?arhzOrez$L4C^{GJgd7&g36unt|{k~pr%hN2qAZ({k9ocX` z_YS&e?g+(S$<-FF_mqAH+!cTMfT7&U{0c2>YdKNk3f*P#Ruar@_Y~7Nt32LVw>S z<{nm<2~LD4fFa-9wm9*}g*p``1!c?j*g$=%p^RK5l;CIVTUZ<>Sy+WlcQN+5O6Nac z%512DJmmo1Oi7!XWOe_~#Qn=HE#QUW*jN^#Ps1)7yO`UYN|P3r-q^*5RoB_-URYAo zi(R+gL{aVT0CI;~w*;>d@<%>H@^yXroZVaI9rL-e}5 zFk@YvloBMe+xBFQLV`9ZQ7vbQ7TIkrhAlH(OG105PbL_qIEK-z04i%gF1;HS2Nc)v zG`eq+fbjB1)ic;!WaO_EqIgQx>oBx#+p7Mrr$zI=-U$nngQUksWS$#euX8RwcO6G- zM3`|v63=%j*RrGAN8@7JREO+K<6KN-P6RTUxMvXeRO=0YgQJCO{<7)HvG%T2)jxWC z$a>eR_}n3&dDY8yBaBDEh{{y*YY(I)%h2Q2Ux);1H30u29Outh7Y!3}>MQw(5x*{m zM1I!nvDsa4cI73-IM{#^qXFU4T_(+I4^93D_GCxCmo}`;VmRaE!=y0FJalEa83w_w zC<0L+vb`~bdO^=j7vPcusT9P|2xE@|G!A9OMf1%t|FAvHdV1&$`W!CgeJ@8D(nl;k zNZ`NA3jnbBsi4I(Dy5g6B#P;Dw!E(ZmJz7~^_hp;b$XHDE(K^yGRVdD)iGovvRiQz zH!AU8`)rw)F7%$@M9ysDTv`9pV{S*<50O}3*>3-YjY^S)6s}v0>-Xr&utm>RByMOR z{hUR=%A7CQTv3@HLFpGIkNitVUq(~x;&OfY1{rYiL*yd0&x6b>f2whHYU+ykzSS8S z*%%^IB5fnSJ~IJNVaeYSa0;ARAbNC93nDr$DU28Z6I@_<*B&C&@~g$|{xknq$t#zH zV#8J;F!z~QD*E{TJDP$@w4kr+XXS6GkTr3ie(y(hl@ES`=h)1LesR%UaAR}70c3jr zC|$z@Th6wEZw>}|akN6mbVAECjuP4aJCf$_mZJl*ew$>4d_JO;TPBR<{}m_6fY9lU z1^L9cTK|?%svr$b!=pNENh+thfl%T6JkB!AoCRrw@P(jLJSGY;WM@oi-@0l#8)uIE z^&QiQwmn0plWn6hvbN)T-xg{>*4}F05Y}-4?&`chdBD9aM zg}Mwhq7zLLk{sj&(+b!UfUap!{7eFFS>QO}S9OUM)=Y)Va^KRl({)an!G=(yFVJ^- z&F+(MCR|thB;X$N+9x%@nyo_B#K=2Zuv&R2e31;63h(MjDa&EB{bWPcS49zD9o?(* z-5&Ks>EvU&YO1AGF=gq|TrYQ$=MXs*W8LKRAfuG_8I^uLl)=gFOS`NA9U-z;{ zwRd{2HKWqe*p{%KiwxQp^*TmFh5rn+5*y zyn6jm`z-4N;w!3J)A1~se2O_1C%o90IA|}^e!->{!1iVSZPzX=Gr(g<#1Rl3U}e`| z?>nHh^E2y3fWYNGa7dl8x`SJ&)RaACG{khbG z-0kqoaWg$1maCDd$pO_R;*TI5T8C)JEXzB%<;9xDKZ+2Sv2jO=HyD{!QTSdlx<8vl zxxm0?qP?c9ayjxURiW_sHdaGWVYfwG1;ZPXNG8LU9)ht~?bf$@oue4?jxW*$T#4^U z6;5jTyii)HH!^&@1Zh%*MO+A$itKqYOLLfA;C7Sc?&H#CN1Wq?qxN` z7-Q|^%8akVU18$!fCMvZ>g0L3%ibPPMzSq9WZdOB)Uv><4aNuI0c#AU=z~;8eVo|! zHGmhR|6`C)Uo(r_eEkyWc=Js5ANguF%0A;Rn`bgE?WeH{mcbBqT3&!WsCjSN7Y+5V zY)IVyWTHDiQ^!x8<7#Qm+${P$~GG(SQ!gPbfV-O$b>jcAx2L4Q!;9N{uS~7T@NC%p!Ud zL?TpRFUF-Nf6!?vOD?#T`coP~@qL2ost2Kr^AzsTX?~Q)s*N3Y%z3JzJXqV|ggNQi zT|9)C@#ESXi`qm*|Txcq7s zjHXv zExvR2CVP|yjB%B}^zkav9|5th% z==iOue2G|4z?_OXv{qCXNX$+rT?;as@kX|bEt$B*3U?&t@b6H+-NL=cB$(pnx@}-X z73x*v<|XHg{_9KnCu2T041lM6*i7Q5Fb&X&R2I~i|I%cwSq45yGSBwgYg<4Vm z7I3X#nk+thlNzGe)yujAMVRATk=Sk*1B9+qQYFWM*shAW7fk9b&0hPsF-t^5$eK6hq z@BK&f(hZT#q$k+d?Y1&o;q8|@ZLCgG}Aff7|q ze%lW6+^*RQtI56O+syfsug)LB&P-6kKj&!<;hGdSLf94E+}a@?skV5<)v&F=XZx-C zUhtwruUV-7L4#;uiJ$?wC$uXjPmBjiqy5QGG;3WZ483hrJT)3k7vFTlB0e7<6nn%b zus?k6c6e|I-bbBzuWoLgu>kl`R>%?69?6DYG2t0g&nUz6;%~)9th^t!U1r+T?Wd(< z#6YzSM;xcdMu1s?hJEWlzNb35{_Etj@XfAr^vl3Dzko0*M^a{cuzMlRfTZt>yaXK( z=h20x{{ARse=&!~v%X8!M|8r9CyTc7hbBdPP`j!__4^Ok9J+tW$3pUV`1D(s$xxa; z;k)}p-=TL9F=Ja`J94@r_lYv)BW)|;vb6Ky$~FnfcL!Vwp6WA{C4^AnS$!%cYbhi} zse%-k)3CYj`|3lX_pEI*L%GZvAi&Gh` zSGk2ldx>S!(lZkd2rKk3;_y_k!bmR}`>i_KM8WVQ&BS9B*^GP!j=`IwGvqxMsv@ka zqos^5lW7(_#u-~UdYx2p*gi%Q{itbWB=*cgD2?ZQ>3!^&+tV{{C$p%aCfP7 zx-anH$}$4R_Gqx=MQ~f7Y{(vz9(P--B(M8ekNHFY`uEF-gAdFas6N4Wx|wuZff7wa zaH)0bnnLR(i-I071?HI2QACv!sl_EtTJc7G5=!?z=_nuc4_(LIqBAO=^-&E==RJlb zDhQ>Hh??UGDcwx718!;n@eld6BY|;2gAl~>$>!Q`zuX6dP55U<7Fg2wd2y-nb-81p2)N4#nX~-! zZ=&^zj9MKAhJ2~RvK+;lSe}=~`Fx9QdT^0dxLloXc>_4-_#GOhD+(w}a!3K4SqPFG zNy3yae%}T)+OpTwLoz3n?7{$C#0UdyqLUaMkNo*Q(S`6@<@>$mUft*lb;YXE@2dHO z?4jIdp^$zIS_1B;#d4Y=ll8d_{xt>PE7*pWskbvs$eTyYET(o|iP^W&O**x0xG+$ewlZQy-7dZG_7cWO!(OLOH z9m%%uor`v!{7He-mr-&>5(hg}SoL_7bch){1a;?^+_f@QCWX*~XtM?cIsJW}ej-j>ufKiQVJ$saTFV`30%RD824%e{HQC{~iUBpF*!D^I5*3St(5# zmJDbMJ&TKd@#Jbxyin@Ht zYFrqjV*zbMK)|+55Xv0b6fe3zPM7^-J!M=5VOCR1u%B(wZJaa3O*3}3bR)iImv7PJ z4H$fNnk9Fn7Mjb(yHxzuG%Rd3`z_v?{UWj-G;bc;s`7}p{G4QAbg+J>%m{P|3g>EB z`$}XsAO4dbPk#5lC88Exc}C8shnb^>`SrTEJZ*aIY0t<5@zA0)qyrzU->PS!L1Q5O z_bXQ-A9{;XEpD&5k?tIi`OSzIEyZ=+`DQNwO zT|?M2Z8?@8yAp1X#JReS|?fA>7^EXl69pK}S=EC})J_$8=7|a>} zgmTD?{R;t_+UKD_AoIUJ_MaTd5)5pCU;i&I)EEJPMeW}1mI)RBwl?0xdT;mZqX7l7 ztRN-CdRY_u|NN}is#k?5IPe2)>*hHKZ(5~nNRy>+#zS)7>c)DzbYsD&U@K7s67X7| zagSdPxX`Zg5lSo;zDzm+8k z`hUJO+w5e4EZ`em(l~&+?qvRfd~-MDSaLnz(y#je!0J>{4F2w?sQle9yJIpC6-WH4 z^0SpnY@%&8q)mp@eS$UUdK|0ZWkdwn{{KLA`+vDy`uMZ znAm;drj)Hpct1&^w>{-!R!O+3Wb&Q~5r-n{e`2;y?MmZsz5RV=R`yOR22KoOgEQ!Z zzqGkUCfbgKk4Gqj0%pYg{x#nw&sE@0R zBm3^{=RjHWk?+(h=D?vT@Y}TT>-Dx@?i+E52Xwtw2zUHr4f0$m`ZQKM@gKA}BAo~{ zx1v8RcVEpZu>S84<3lVbH~%-fdB1}ebQkX&a5*Bw9FExF)AzH$J27DE?AZCVWmzTg zss!}fHC%q_i0Fxq`5K}4C*SYy^)(nZ4}aWn1-GLKfTw}mt~~+pyYrxpn52=AXkJ(Dryq%EpTA5Y*&{n@-nz^4ffy>*F)__0YLnS*zlfA*X8>bK+KjLjDD z>!#K5s)~GuQ#J!*f!1&`H%U{30uVb?gefs+7{L?xy&V>K87K2h4)cKQM|@r6O-du{ z^K|a;NEJ>g&Lg#lHF!ka1flgjD?@s!pzP0k-}9ny^|A~{`m`>yFINW_heG~1=AX)hf;0OB$N@>Ss)sd0CE{u{{N^(4>(_eLdGT>8eS04qBXIk7g3 zN5_9Vyl<7gb@1$!p2~Pyir~M4!co}NzNveK?$4WstNsQ)(uW|0&8r9x+_Y08B-8zc zRDbwKGEDlfG))Aumo#vCXZZAPQoIsf7WFp@fJ$hKFn8chIpSRP5i8tQFBtL>j`X7q z5rpvnR@J@0GGUGW++MKJ`)~EHl@-*(1$H<}gTtQ=UUzNBOD(($NWTt^^FZ`ojw@62 z8sGr`93&!|fwO2%a3})Lx_z?WtH50w7&txoA8j0X<9~6I|C@q;dIiqbiar}J5d$LC z|HqK}9|aGNq;DK1+#5MZ^8cfs`ybof|8e6#h_bF+-Z;rfRD}L zv5U#tS_I}Jh_)Ni&s2cp`P&^BpaBJIpiK9#U|qb9{cCo`+DLOce+SrxScPTz6wOHW zqxP5bvp2(0`a|0}9P`=$wd;s{q~Pdjr<75x!Eid&U1v9uKB@wvHWJiBm$8Jdk z_crKGfg>`;b{%N$*vh-?hP9mw%!bKIw~mjO zXz4R-*tj54{F(c##{d}T6Y!^X9#*VS1*DrkvOKhD`Of$p@U0iDrReSI;ps^EWbCl6 z+u-}WXULt5dPwgUk8dD3?a{dR_2*U4&s^i*yOt5OX(qO#bZaeb$Nzi<3Uz5`VU?KW zLRkIc6hMWBMVQ89my+X&BOI{N1$)%W=#Mfd?NVH48wAVoeKylHnttY)TlNoR@UZ7^ zFQB(ZOPXmd9*bGe9uVYERJ`j!D%-apz|L%Lc~Bl_dMskzmO~QOvG*8P4l%atbqS1F zoPOQZ8dEfc1zE`v`FC8PWjP>oE$|VXDASG^`HH;v?3bk~wqC7Ee<6+rSO@Spdz@Wg z)@925s_G|7J*HwDCY%duRjsG?6b(NurSlK@0%sh#<2jm0rzaS1{RLD!Hy3ZmM*}%9 z5i%Wu4=?cmDwD@fjH!tc5kfXpB)rO@xfu#KqvH>2F_ zjTD%5{Kk?a_;s5|nFGQXOMBW16aaAOof8X?eV}!nU;&S$HbTXRYS+AMx&E%UXZ21icDi*Z+xefdN zOhco#o9oA#`vEa1nt9_t!56z)2Q;quzaevcn@7rUyi&R*|0eKk2BDw8BmePf12h`W z!`GOxb-bOOfm^RHx1=iknIPYwJ)a4sWm|yKk{)iJ1$~iexJS3efzHMIWykf}a1p8P1Mb2mC?fqWjOst@m|r zmLf}PnRt*DWv%)YX)|_R?=6HE=08Etf9DG##Rc;a=d2z@NcW>yYQo$L3gd4`Xs??p zg9u1tqa|b}TQ=BdL7+TMRNIo-&h#oh2Y__t6PLz0S-6~klguUF)#Kf;$oG$sGwHA+ zl(kxw89`{g+rfEC;V%}sv#TC7_%%rug`DR$QrLI1*2-sNvgH*q~{+B+$y=BVl^!F%dWGEF@ww$z@Ciz zv8&gq*%Cw#oSZ6$S=&!0dHj0>U?Et@ED-YkajaGUfu=$^;uh-@GH1Br5%K?Zl(%4mGM}@B(>P@+& z8*E*qo5D^xd%kg9K4a&dh$EBi7?rq}`T_5X_lM`Sk%jU8fnevpp7?{W>hFgDh1Lrt|cJ_tHE8ziz}zWOus` zl?)Ro)65T*d&7x@!4uNh9tSLL#51kJSSVL#L+)b0Aln*XOO*8iSI%QOW|g0KXHBGc zSZaxUhmU9H2q?ep{6bW`_E`UE1N>1+MqUvb62^bnJ_7j=}aWScG614FX7t5nja*~eoT9hF1Yw%^pv%U}aEo?Z}0bhmul z?|YyGc*jENRg{Cb(nRd%ikVqEsps8@e%?$)giALbznO<}V7q<*E-;)STG-0{2DStE zSIC9oxMCV2DPz1q?{Om=CUfG>1!&0hNRk~`k{jV-9(*!GfHD5|hL{)iShU~(xK)|l zTuWhyWCvtSWDAEKXAuR+u0J7ujLwe!BN5s6g$9t44Ywij;QciF2A11~xO+Jz%{!v| zgilP-kbc`y3;j)#^xpH$3g4|B<*!Fa5qeCm5G;#p{x*o@G8=2mB^L&&bL|n!m2vYR zHm5%>HPd|i04%^OeWvDE7!jH~e)kTH2-&@(Qca~uP%6OpF+k_UvZhWz7PtS{ zf5}}*-(Rxk=;WTmL!M&vx0uPp)X}sia|$N-4WeB#tVIX#l1!@h*h`1pqV3Pk21S1v z%DQ9ZefdyA(t2}9%07(+BpAwu|>N_+$u^7i3TvfuJp0jBBD zoN~`IgUw5(4>Td{F-IcjMM8EE3}EwG2-1(UamNs}GGHZ5)dn$iuIZ0x>{8Q23mV5Y z;kmQF%jt*T(jt#Yc`5q2GUg^OpHW@Eyz71e@H0ST;`{Ioc^uV%^S@HPd)Nw2{(ljd z($0F^$-`R3JHMOwaOOZfK!@my)P}-S3Av8AL7zdQ6Enj}58vg` zO6X?-XTRY&K<*87UDGP)-DRF7hGtiidj2xOzucHCtS;$fzqFBz{xSGty@i(i6votW zhikglpa2Ya?E*6pNxnYf#f(J|m5z|Tlq{m|t!F>w63I!m%#_7JAiF^T?yJ(1TlxZR zgKS5#h_6*D1Z$K(QHmk)uR=VCYu0eM&WAO3M^*wfhQ}`(Rtfh4BT?)NL`GYZ#)@ z#4~Gw-U-iTp4t%>K;ls**_5StzI$i#Xf+Mats`vRg*CK47UjRqSImTWxq!@l+;J^R z1j9_!3FORZ0In04zPKWZT^POuY0UEP@UE?foKW^8tv33t5@F)G5I43htH-c@2&*&_QQuV@1{y(MjS)quI9QZJIRrY5>*!wO#1ApX1s|ZcNOn zYr!c92EjVm)(bRMmHCc)eFp`!#0I!881BmT&MruQaM_v6?qrF874jmfV*VSD2%eI~YnJ9n08p&DKc)BC{0EE1T7Ew1Lrh;K2JV}w9IAYin5DgwM`!n z#@5V`O^FB#UlLBp>+~`$_1hrDl&;(mT#nZ+MxWF@A+}9ZPp=;IlYqXdxeJjg%p^Av zX@^>+RtxPll~ksW7lU4*Dz0N6f*}s-djfo#8tJ&A?y@!<(ww~&7jCSKw;>2gE6MVO z=ZehVN=_)rY9&4Vzzc)i0usy?M$Xe+)^4P=pq3twpCF`jF=upzD9$nEu_5+I!0dR}`8x*h*%p=Qx zA1y>O#?nvpY-@pV3HVaWF97}aN z=F%i7nI~w9QS{4|6>Ic>%;c6ab^n`ej=QGaDgLSD9=D>S-BIQxD1JDmFQt3U4ZB63 zvH69FhM4eAswe)D-Q|L{5Ag{kocNo(n?sbnZ-!zxf{zsVK{wVxR!YUVp6Xm+fZMB6 zOwWpmr(;ZBq2tbk)cm_H)u0!C6%EV%Gy+?KYOQ)9bBt^$niN-J?8CS^m6p0sA3tpz zU*n<&Z{>hyGGwfF7g6)Oj#79RGAD|s6sV#Fb)s6%)b_bL9WI(IJQ*jHt4J;I6Z`NFWmj(kq# z2X_s_OTDQHqrsWxnCnm~j;f!77+i)q2$$-B^!Qy7k^Q_3Wm4(^(qY@Uzyg;e(V5y* zkxg1;QqAG-b`XU$8Z;1V5@jb1+?VP)cMbQPULWjB*Rd6t$^N9i*~ti#ol35m6m*C3YONFcz(k8q(&R|a7(zH`yN(>o{}U7k zXq_?7>5o_z2=bH+!up|y)AjL%VpY`bC*+E$exdQvs+aV*n2MO66_hHlP2+(S%o0@L zR8>`TCCJWSH<|F$+L=#1d6yRtXtPZTrX9m`ZH9cM-NWmcJwkJso^862L z$%h%AbYC{8>_B+a+8hCW&PSZllfuiv3-&-ve_VUrMYET=Ih@7Q&n-DmRn+gyl$yzu zGWj~{E|c|8#)JFsSifWw1#cXjn+%luPjf+|GCH<=elV8?hVZe!4XGV2be7Ghjqz`Q zz&|+cf&pkz(`7T#gsa4;F%3M?T$G4PF=X&cvQni<8l7}N#K&LGOZ&j<&SSEUovZ-Es?0%E%$dPR}A#cU7b(!q-?WAD`VZ))xg%Jp>e&^XCE z>rN#STEQ_3*4fr2`iKVTkX%KU3`fYaNdez9WLo1S8a)!{TA4-EG?Jq@P9B+zv{TWE z8{AOX&Q0H`K$AfiTYjS3T3>6!TNu4w$^AY?XNVfv-tB4ypg zLEcG=h`R8BxD}$qvGKito5owAxIPJefm9-j0X9G@#y(Z9U3+W{pv{W@-4403a~m;D z&Q9p9Ug=Yu2JQwU!2R82tG(c_xfeLehiv&ibjj@uR$5N-+)}|*D#?x^x7MLc&& zN!U3P)ugYo?NV&gmRV+;EcuYVuv1sjby%cu3-SfDB7#I8kgj*`PrwSLXEhYdu}R65 zh^jrTbDLAq5qrD7f4O_9F%ZvKs|pK03@>N0W^e)P1sqnCnF~2*-rag?n0jS@ z++YzNKb$S^CW$RfV!ZXmMImpc06K}EvamEPWT!KYrQ(@dL_{5Tl#m#^An5*ZA(6+N zBGS{oZgs&g7+j3EM+)u(e1#VbkHnTFls|Iw)fz^2C;yY9o3e1*eSOzdHXD>ZNXvc|3*5v2f#%qgaK` za!84Tq6xbXGQmQ|SLC9R4<6Pf=;c3PPLJPV?cvz0J zEu(PD;D2lj_2T^nyxbwNW0441YMC8n-pte%>ouYV{T<>|wR4&ppM5D|`E`Pc@z24= zv~GkB{Vf6)8yKdViqQB+2$Mdby!z2yQN#JO^aA{6b_KaFsykBY=Ax|9jTX{T@|8Gfu;_S;&p3tY5cEglQso_=_^tjJ{py@jFUfLa#w;D>eEE9rzj`xWo7tCH~Aq?5+5p< z_{LHbVCzjrxXyXE(}FNf$=@6Y0m~utS^JDRv47aQ?Bj;|b@m*N?9VB=aP(OQ2sXE= zd?dk5YG8>V9Mi@Z8(N%ZSQuzb%VmGaGZc;?fJ0+n`qM6gj5U@OF z82jFXQf(F8+Bti$bo5>ws2DVptCiRC;;ZJ>`mHxVBLWuO2R+)omrZS(wA5vD!nPscG zFxlC3_`0>jab)*_(2fj6o4gBn=15uz;t(}wU*#>wMHis6H{$Fnm1;VEdQ=iakKN75 z7C5V=p9)pqPI-(y^wp72_Ud-%)?~5@LQKB+Z_eQAqX{mpT8cmJTGoW^5zfpRgX#Uj5PRvCNq7oUMlOaToql|uV=%iySuwn1f;t|x;v$ju1$BTq)JFiZiI&h zH=WX@AYIY|0upkD_x+vo>3oLGUNg+xYt5|t`d@`Yv41fRdQ+tv>{3r&lqY-_2G-9E z+IURZR7wl^-=mL$-RFWq^uTUjIZ5jf-)6XI@lv5@B=Th5#l}i{nfgJz*i3hFGJgg- zzot*V^{(K@NGDD@M+z&kQI9k@Boi3-a>i^-%^oq%!%7u6js(cLO_8j#QSYO((BTGn zwzJ9_P5uD)td4^Nl>A?ySNj%Wx*XDfV|igK$lCqDF!>|~OWP&T8g{K*X(ArGNdJYO z{}aB*WkhZpZxQP63`=eQjH!6+urXCX3>X^(c2}IabQvRO7a!W?G2B~s(PykMS|9lA zgzCI4S3${njM#ZW?|a3<#i0lN50&VOP5RFS8Y$zK5)IW}_#~HPBsBHF zT;L79cxQ7EYbiTrL1qm&J8{1g_4W}oVT;#^5IEd*`hrsAbKk@#`OBUgSb`@i1!-7c zkH14x87BvDp;-L*_J)@^T2dE0icFdKw4cx|%b1Y`4EkhIt^cc<#}aU*yv|MB*d^97 z@*tuj-}QMQ<+3y_iCsnIKWqLDDWepu+SS!KGM?qFztqC9^u7u}Ll{PSNR;(8Ltv5k zxfNNVaz!Z1vLkvs;)3au1?>csj3kESx=wUt0j}YS1T`vwxBO@a&Pu} zciz&*LP~?rdG6yGF44`01hwst+&j>jUNDo=^lItz8@ND*gyJ>H1&(&#GFhzkJ4zD& zv1Nw9{;Q%uL5db7&J8QgA{@!2x^dvXk5<~;i}{H@vU#_zZ8}3fM57r5W{g3^pT~;c37T8GzYI)ZKC7jgcdlSjrP3>MNH$1T2aZHME%Hbo(nOqx zm>39v4F+e1@M(QBgrOdBsX-F5miDPh*g#J3;78PGNOm>O;rp9)%wDV9BUBhgz3MIiR#e;b>Ojzg5#W`43m|cy&Zg zO%VeYSp-f8$As}Z@RimPtX^w7oL;%2ioo(Ixg_TtfoJI)-17f*O$6KZashx7G|+?* zv0E2YZ5f;Pk5NkySsBt&A@hn$|Ga1=y{f{M8_4}HwUU5Bxq+E!!I%dDyV6V-0$>8u z)~vfVSDLDZvjSrSli78j`)M`*&Z^oRa5Q_CZ8Nc^-3^U9`-Nb%cO%jN%?ejWGon(t z`Jqydp3SzXs)Prh_(j~2&_nwWy!b|$A0Hs{d~M{OD5O4!guG-&vG-d+k@S@WW-w3B zH=ma-(eviLI&obmlXI)Jc#t5S(dOhNNQSse72~qsz6T3=0qc29p*G@6yg=%YUu4hG z$FPqTGDffT+0$gJ3io0%oJ-#O_slUhCMQy4#Ye{!C5-mg)xN$RP(QynU?Ke@cS4%i zB5%NWT){g|)V!{jF>rumi~%DV!vxYu&^J24@JS8jj7b&wKkSiQUJnCAZ>sg?BK zR@{6q_Jklvd)~zFxCmf%a6U$=Q?M%G!}NItM1YVV>o!5{^-IfVax_3r047W|a?t)Y zUMTS`FIrmMZ))MMluSy{cSz>NLHn+@KQ+oQ8BI4yk{%^FD!vTRi@(B2etCUhJeR54 zZmkB?EbW(0>5>Prq|on-W@xvGHlGGK&e=7V%XIfTZh6Wa zBIGC&(Da^cNoHL8!xv`i{LkwA+u+_SmE!9@hA#4C5yBznlh;W;~<{F|K?=t_dIOZ#Rppi>8>`7imfNVGX1PE z2|*{SuNJ|;BrITQW8yZNn$wsubX)fr}`SLg8*Q8zB zTV6O|HZy*Njl7tdL5<4Q1^C($MB$IupjwnbMp=6bM^z^*=1{bezq)jCwnJTN|JGD% zX;>=3f;i4mXRwGqpl-?Z`Ae_R3#&0$(Z|m}6Shh9!*VP2HjI5O%?=h*9V)3#vnC)(F|$(hIdoSoSOmTe>a^&HQ}1=%~(>OZV2!6+Oyk5iI2 zaa1le$`LW_W*H z=ii@r!tQ1kvMqzYo$C=grj5u8L}TnWQi2WmumGfO1gQ!HJ=E>LsJA1em{z?N%uTE) z(>^aar~K<5ta9&ItR?lx4d{r} zOH@_6T>7zy0jq!jV-$yX{58(f9!l4y`jh<*4R%<-(<6W~*57O<2ABfT67_ zu>76f`-6YACi2uDuGI;78HG?s-=Yy*td0pM4`%B4AFjKO?*|9v<>Z`;?NWa^;tvYB zP=-90Bj6@HxHY)Pi}OVu3=X6FJS%2iYwWh-_?F+ehb?@t$4cz=;(FZAA#11w$+KI5i9A8bs|CeEc=55H=30$>%vV_sUo8Lbi;heLMDy7^&-=2_CcJ@tGl zj`hRrfrAjkv|?R`y=?Tefp;vBh$HLQY*AT#`(NH`ab$C?KUbH)M9I;((kHh-uFM%s z{cm&x{h(K;HZnj^`Nld(2zeqOPPxJ>hN_Z>1%W|GaK(>!W>Xq;Fp%@LleI@%a*t#aB(09fh7zc@vWB+vjr*Yh`TJQ1zNyXXIIVax%`yp}5d-z8bm#vlf)1u$G_x zv*BMFFS(9rnff3w41jPsz>z>>@T-fvznBYQD-Nn&P;9-NN{SaFmG>?2sVOhu*7>7P zamsp}j2Y5`H$Qi$k{_Cm$L62QsZ)pix z-D(SUV=B+N7Tcu-!HFG)7L^1ooKVKV+o}84nvdwtUuy=brNIclW-mg9ZL$42LSUAO zMF%a_B4Ncs2zTmiLnEc7bxU57BD)D?`@bA7qW#q@46Vo|KPwL_^Cg4qwQqm3>Y;BR zUVc#q5Ug+XIn$xTE*yNsF0kAvk1~u?A!%lJsd00G6nFoC#MlJeV@GlAxq(6;&=?pm z_4?jEYET!oF#lr+=3PkkNYzrY&orwb{VZwP(*>4C?Cf^T8%=7SB8+p-v_t5UB^%*^ zn_y3nt@9NI_rzUE*G zB=0u1CeL5Gbk>H&v&xO|h8}QF?(417wPcng>%lEQzbWo7!Gqh>DtZdK8XvQ~!B$82 zrGQ2W^t*e}#yBcyWhJpX&9MYzQJ$@78$K2QC<(=rSa(i48b5I3% zx&8Pns70lOlq48NYa7l9$ou8Wn1qqzrp{kx_u^MXU!uQQmGrEP5^ccfNXjCgTQpXB z|Bn$nH<$a&x=QWs{uee!9wm#Z@WCiAoKsu#P|DdDLyA~Eqy|G(VMX*YQd@`+)S>x( z%&c!)C&7ars(r6fnSZR`37<78!CSHh<-g8K z8De8o3X3->8Aalp#+1ke^amxVNZ@mJ;$+_#>tFm`T`!D3jCL8qEaF6vq*wa`L3%vaf39VPp1N?t}^S6(w*cr%RJmqiu*^E}@prkY=IJ z%aYRZCDn1av6n8yJp&Ss^>Q8AQ}%~n`58W|PuUmBR}jM_SE$W;*GhDl!&S`A8Ue|w zqPkbMVmH7=^P?z+0o89h=0T1BTv{ulTtrGwpW@U#g^n4}0#>ZGTOta;7__pl$w%QG zid4t~mK=FLfcXaM1%;v^2d2f>y_@uE?urQwoRo}2H1RCI8X@v@`qLD0!j>z3ph zGDa2hoEtbTSPn!WHB@n1+9=)L=a9$yO{roQ=ccC|hW z0&4`5^1*4*^w)qx6KXa&VDw=j`6#4A@kL!r4H(x*1`nsMv~sH{G5cD(Hv7Yhvcjr= zb>haH;`7y_-_cJUP$+@35kBs6@2;HY*Vf3LYi?ta9Ju&>DNtKKl94<);+6#^lT5~+ z^q+;iv4F;d8ZRD;I#06q7<+%qpX?Mvly(b?{#nXG^RF}YkOyL2Ca$<=uT7PTlb!us zmSj?Y^15WLHFJmh^;iZ2=2}f}l)Qd656fRG<9rzS`OJ8$=GS6xu~P9^mF}Vu%^w>w z$f1!>a~admyCnkP`cVIx3`f5>OW7|c6@2>C9YXg)0j}kle$LyGkyv=-lOo#zmu26)kDo7$t ze2iT|mH902ME++ zLYN65bt3JaIuop`1@&ej!Gw@L0HP4DnHtD?sXm!17a73z4WP~k8`Z;3m;qXMJg=Ga zsLU1Q6)g-H5jd%V<9Y{(x>y2UFJA@#AAFq{s%1{0h~=jV?`h27-$GgkM~MM@>l3S% z4~8BQtjB`B1!$lMa8`f;%fSRDNesr5N4mYrBXdF5OPw;ncSZ)y~8blKquJZ7wasNVc%@l$T;|&3W6L3 z-93~eq~;Z8d}?)6g=EO5D%1Wp?NBCWP!2wSoaB@z_cp{ls-CYlqVvD8+3UQ?dyzTw z(ZlE0e@3-`=akb#!~f(HX)A=#6Ruzo-N0hdo8FWg&NDS&b20gWt{uAIn~goG3yc@> ze(8ti!FL8jVaH?254SgO7ixOy{EaI*^@Iuke((pdp9{3w+~!Fp;iX0_e06Oreo=cm z4Yx|Y50x%lEy?CpKwI{7W>aYE-MI3{UEUddVHW3@seKY4zx7SS1?nbH@4P)IlSiE_ z(=6ee>FMGbfv%CGy69OnG!V}_X}*n3Bigq4x5Te8EQ-6pFPS=O)V-|TIdpePxjD8A zIYi){AcYP5wpPtzb*BD>Lj;;D+-9+VfNOq`#FZUFB`b+by+f+x7i1B}o33t#TQ<1N z653=#vuG=fWX9Pg?+4#-{@w0vF8xbN+#=3pz1=*BvZj*Xo_lwHIE?P}eh1k?UJQs& zle}7r#+BMXB9q(_1VVorupK>Y$b(kB8!NhaEP$%UUd3X5@7`kdubJ*2Y$B11{AaJy zq;B_-@uUKmg=iO%5sRn*-Gg+Y^yPku5nV*6SdATZTEnCjsG=x9(j27uO#Jn|5MwOx z2t6T{(m3I1C!dhKUirJte|M8N>AQ5DSr)rZYRKUq$vV6HR-cT>Kuxtxj^*1@uk4q@ zl^;HSKk}&Vm)G7PX>$Im5rpbRu6_(y4N_Rv68rPc*Q#qztMv8WPk(cl`8=zCN57mt zb_LC{Zbu61pTuBLKm9ntdyBbI$H~AV*O6Evwms5{l6#P(+?B)e-Qs~03Q*fNz zzjx-1I^iC^o0L`b_yOTx5UJ_3C%4T|YPDIfXVr8-CTB1wPN1Pv3RDTP+N4EK&UOo5 z6BI;ie}soS<*YTY2>HDK^~Gc=b!NWdqs5Ix@s&~dasx|p9>G75szkzh!Jz!L8} zYv?7GM?dQNXd+wT>mJ`iLg1$9G~%x5&*5|Tm0=z6npuem`E5f0zzq~((|^v#dR>PJ z6ma2U!T!P}KMSov5f`lds>Qs`?#tOFg~8S%U?HOb=P2qQKB>NFTg#t@kk(CTet!f4 zmqp~_pG=uMkTWMONnd2M!pRqpp+MGXDOd<+%MXwNAsGm4@k6MOG;jph3aTIkDuObB zz~h=a8L+m#v|tA32sJ>$9U%8o!F;U%+rtJAO(?tod}xprEYzL- zzWflcd@?YN8bur||M&-tdPQ%+BF+L>ru>#%Dl`h5QPR)7+_ukQ6R5Xc?u19HDLNIU>4#||wxYmEt_9Rj)0 zX&v;l*~{6)z)+-?>wI1w_+}7(*btUpvOS0k>)IcgZfp|tO9Z$~%VZ2&w_Sduetif_ zN0qAOZTPbss5KI#oqGNEH-l78iQ&Y{?%L?CtJPEh#-;ftQ2qP2wdcAP?VCXRm)O(^ zpVJ!Wh9nGQ&)kUf2y%O5tHm8f9(}UJG#VFT!i2$`fUEZfm>OGwS%N}J5hgqHJPc`X z9@Y7GcHAFQk>ee?iG6691YL5wj=0{nJgb}d^IwNmBNfHGgN;vA|K^fIZBx%55{7s3 zpHtDKYJcnUywdEd_}M0$;qo(&Utd}Lsv!j0dX7lz9wR}6HKH#hB>>v`GlntNcmQH@ z4E;@%^>L;5{j(dC*1Rz#qoTeu# z3d3(zCFO`A;kJaSA^)~9{Nni8&Eu=fYEjt;^jo_ezfl&Trr58)TN127Tujq+zSc|+ zNWb<}_Qb`JPYPa$0U?PD%b_Q}T@}hGHmz?QDHO31pqeta<9UGak=g z#3~`t+d6NSlf)M$=;{#MH1Cg#Bbdg@YlqY&diU9T*4o2ngLAlp?oCi!w>koz4U*Nq zhZOYsw+!99w1j+{Y@Nd);38m1(S0*=0fs#1K{EZ~#D+Umie@7|OLgx?FyE;#OyqjW zVbC-{ql@J->cxi4z>F>Tx~j^rB?w#>cWIt))CL)hF(5+T;KA+zMW~1bBW|~C!+#q3 z8)yFGvft%~@)X_%PM1mM$B|&Rm9f-Kd>YDaYw%~H+WBFT=^GpKoAcYI4lb|8{mL&L z&ps2UkA9npY%{>xi7Cbf^3YXd(v9kaxZU-k)nmTwr3)&)KEa=sJGO7t5><3Uf2|DX zEJTjY4=UUbG`Mh3^QL}+k8r~IrnwdtX62nfMXB&ylsTl%;3fkDxj9Zh0+y z_WkN5=&QSaLWn>Q7f#a%y1HDl;@AKvn>|OSyFp-C-Sj^7jXfcJPxB}XdKNcD{9Tz0 zm&FiGzzTQ>w+1N3ZF>cvcsGNK^f_p8yPtgeAgEzv#iox(Wy0`h$c9mTA9#EMo%{5q zj#>DEM)<;iW97(4Ot^Pmg4o@wpyftTW(acEYtGVqw&=9itwWNaYw7PJ=52h$O+08F z3oY}|Y=_jD> z$+gF4c9Xx@8Ap11kdR{=dEBxV{&YIGnx!9RrNclJtM3yRAiua;YacXqw7`KeRHLmt$(v-y?iqo=&ZTb25jmDI0Z)Ksgj61yq|| zd_3bE9CPG%&?!C*S-WH1+uh^oyab_{UGS6%kUiUe1g(`XU9Qv1qQ_-RV=3C9gF^_C zr0h`e;1qYQKqLpAP$VY5$Po?_qIi1Pv9Q$}&^1|_)VL#l4u^!_Z_$BRzznisM>o>L|W@mraPTx zlm4qJWfv7-WIl@ubCN>I_omQ?R<~hZUh*pxa)`^eEH06uyjuoMpc^lnA09!x z=n!iAyhgqOlzHGFG}6) z+@5@8zD0Rsa(Vj*;-+LqPnfRZ8OFxPAyCc}an}+tc7{OY1`M2?Q^WL-)srwa4Qk>%YI**3q0VLE+W$vE=3j)#@2)6WbW)g`z!H zJ7TU+C@L}#^fE?rJvE+Rz91BV3VNe#aD)Urb@QL~;d!-wLGBA88{c#M zXOF*;h25a~L=95}DB*7pvM;Y)xl?|s`Fg7R(^-%B*D&aRl`ZfG@nKRO%^Zk|af5l= zkmN}uU6QWd@pRYuC$cUT3WO*sLRtL`MjvAH=iJx>1cm?YO{HdXe49o8cVSPBw#2@L zgvjhx)C)oxbeHjAPhFV?JB5uoK+!ngyKFV^$Q>SDE!pAis*oF7 z#w$K?k8W{R|9LL(P#HE?HMsYS&eXIu#0P{Z|;VB{#uNCBQza`8mlAnim`R9 zXl1+U^&Mtx?uM9c`K%@>?uwZj{TJ6kDB_okb_k_Wh8ZGvP4JhkMyd}RzgsO~K1N9P;ORv+;CZoBlnjV$BO*UzK;lQ-{K z$MZ08zk*B+h-SRZLH81i;z?HA>Ydh(Sjb5~N3QK{m!I*PZ2KqDTyNJ;3mI)f zpL&UUQ1>QDeIQuH-{OI@UTOnTIrC4!|LGZ%53>%a%AvqhLh(#edPJx9a)4NiMziHk ze0kGwFl*vwl8HLz6{bo~3Jjro%kotdaK~=YvutGxWwgxlq5RPDMsX&9>$${K`vA@F z`PmJSmB-5dz1co$cLY|>${J0!ENakkp z4x{di4Wg->!MhCjvz`wu&2A7x`XK@1G)A$wP2PFI7TnvPga>0=lITjIO)qR=z^_xi_*G?ZDB9Lv_#oAEzbHtU`omN%NA z=QGIbXS*;Gi;pJU&LkNmipqmf)3mrSux9R>MO8YkZ>>d&Ijdxc#8R*kd(fLp(9<>1 zE$%Qs>|lU^b69TTzOZi?0slbFic_zK=)v9bOju@8TOIS*5ON*v*t&O8Vat>>XpjIPFe&gVSU z>9jH#V$!k~B~dzXufFMY48~xmAf%l)=7@Toia?y!Z^Kp9sJCbyqF(vhYar5{YGE`g z6H{oh9u~`G9=YE@hyqGuzkW(@MSof!T<{e13;@>u>^Zh0YfPdbrX-(bQus`5Ez)lg zn7u8S=^bpG;l_(E8aF@EpIa6Bu_yU(Wr)@N8IjTjK5$I^)GM;1%vP&6=O$h!Uz-{! zR#7)HLRWP=GGf+{xcV2!`Y8>*QOH@1q3^L8{VY7nP?|N5W$w|(*JIr zAl|@Hf&dceAx3hd zpN}>>jV22a)FYbm4Vu$H1iB$$_(E$iGs`$-iuqf0Owav{iUN zQfU1XRsII?rPe@=YMrW_jemqi(77g-730J z!lhDhP0-G-M^Df-HQDQK|Dt2a0CW(cF335;4Lx6%P!-dKmILG=lurF{g7rZfBLA3G zNSki&(&jpj{6}aH_P;I8Pwt@r*#@>;;reV(P<)&XRF=&2u|d?vA74ekDu|;QU7HnY z(C_{UHRGoj-*v7l(ug@D~@-Yel<9%vu}4crD}Oiml+8 zp1GtuS(8G7YZ0&rE=$xqnY$b$*kvd3(DiG%cI?axu|uk4l!&yLlBA8Y6yMVv-6x zx4Ew5RcmKkZhO&mxeF%6;CJN2icxf*ViWw{Iu&c2n|T1^?`?_Tozon4>{P?4@rO@N z>TfK+=WAq@Cj2mmS^g_DerGA$(sHQ>x}2I#4e~P@w;G53W-1je~$)Zr(+$# zSxiNiYo6nw(99Anm6JNZ!Z&iDSE_v>1%Pcuzaqjb{T@nhQ>9@?P%>!T=bCM_Wcgbo zk<&Z52qTYR7@5|!-FiX%1YpHNj;~YNQ&`Aqj2p;l8bQ`SdP~iU3lRI zRbt8rd7Wef6R^S;!*Zt=QX%89F%^KgBd3)_2Zv5y2xGWxkxeMW^gL+U200)@W#XzpY{K@N}e$Iw-+Z)dxU?Rt%g_2bAwR{ zxL_G1FkY$;;0ZmkVdFn{>KA~DC4*af_u^{eJ0l@r_ zJMryrk{sCh-9ki~Kl6LkQSuzyu#O>J5OkxPb_ zw|G!7&3bg1UJ*s9DcF*t$$^GD)^CG>I~^qJf7=sJ-t?6e`Tp0)p+v@FaiKx23tw%5 zr1`1u7g}09qhPce>&RMY;{_HVr9qd_veT9Qur*G7fs1#|F+6o#HA1lphC7w_$dxKIy(4HSq#=fatib zVVI_SwuXq)CnX!c8IoVE$Pj)`5|q?Ud1S)hm2CM+y`}%f3M4d~@+_LylM~-8pz~*9 zj(1ocGrkl2X3d@&j$dNP2}q~Vhci|WD|AzLF=aTPE+xD?{W%Pf3-z`J=m$M(Ba`vK z6h;+q5N%?+$e+*5nGqohOEfPwbE$p7%(Y`F=hWbLoc?%|o`;g3yYxNPJx#4xLV4?b z_PQ7=oOS31(NPmt+w?x*TQ$a}4fnZ$ea_U2>wP&m*$_TnkG{_rZ0@LeB-r2CqAuC0 zBm*bdErH~xQ;+to`BYZZre=+5{|nk~K&tVDeo0|Y8Wy0;^fOo!>)Wm%F)4xdgW!`2 zw&$(rgbm+yMzp3NQWycPP%$G?%C}LH8ZXO@aJSb|(l!G=CG^ktD@ynwA;SW;f!uNG zKVKIGpo>{%4W7LqO!(Os^OAu)$;ly7`bNiI)bPK+RLiT9rHVs~sRXG~Zv)99E_PpqW*M25l^{D6v2Pq{8QF%s1w#SFPLb@`A$P%7t0hY(*B@fc%engKw;6s#n`s*#6|GKPz2wl7Gjn^)CRQwS9tixfS~T+A8xtf%Di=Ok z!}Ro?93D839W)$<>6Amr0+JxLOgFDI;xE^&{Z!^^$D5<81KPtg7-WP58q z^hnQ3=W#`Y(^ttT6-O}18Jj0Z9XCw=93SZUkmSWKVh(O4ZKBx*?8qw|vO1b9MC~P% zA3!VNoFvzPqj7bDX4an;S?^8BfRY}YU}9byZhca%sV~&zJSgG-9Lt{uXiTuJLC(9W z-B6x|-uj7Yxyu3x!Uu~9l*6JrD^ilYK#hwBf>ikMvr+yu??KEu641Ar(AO6fS5UbB zICvNRE!8 zgXiQ9wo&hqzZE(q+UUkxGziJvM~`T+1AQnX3jdC4n4y9u$kb*?*33hY|GwoCYoPT< z*CjrOZ}reN^eM*kd60?r)z}!@lvwpZO`VT$ci&RWdX=nZ+ zyokRu#-IBV0vj6@F2!4~Y8<|}TFtZ6#Z9uN4@V1iinE1_rXn963Ve?=T8Bb`lK6q} z;?FYEC-wrLPU5hdY>Ba&9>#;;kz7-43rJLNpGj|NbqZ7wE;;I9^v@iFsI@dVg?>9B zcyAe0WUNb{lB`wTZ^-MjSl19(u-cVTQ$8{s=qF(U*hGOoD8Pm9%9nBF^xYpbOn1XA zqg0j!!zjt_U#(?n5Lp#WPf{7;EUiSy5>{Jpb84mXNvh8G?yz)~<8t@%c>yzw6gzm-U}8HE9>_OHu?)sA5viwDZ0WBj zIicjCpsnjF{51bLSy`n3^Trdh>=8gH9r23zkETR7B!m%vbd198hspNCPaOB4mI^|yr6quvszGgQe(n7}Clx`6E~E)_&$!x7}l z#WpY$;@@+e9~{vi6TF#NJ21#&sB?sEFZ0LNoa{OD`75+T0<%I2-o=M(Z@ z6}S=1?wi4;fidp?C2}2+08AnDuK!*uy%#^U^p;9c*^s!P)P{PXJSRjW_v#`O5;E?` zgctw%oJbCR2U-KS(W;Y)rG5`yt;T6%2Q2UvxP2+aEm=}7D&D2lrTsJqnQO~i45F;{ z4&07$@CKpM9CZ@T;SJw?zcj#ln1OlQ63P0HJ+K}~&lCb?Q*}TQ9;ySiMQRu^gUEgq zq6rBt0Qt}4Avh8&lkQSztz8o~RaoA*Zc^kynwU4}7Mg%*W~ppvW>&yVDXf$ivXcT= z1~*|tynQSIYbFFbhIaW!kR;NC5gFRw-Oi-mTf5;eyM%2-K8GGAcqA>m%%ca=d}ZRDqPk$a)G%_mCq0&qfpi4s!CqE zhLAQ%fw$dv^5sj!9y}U=L>I6mB=M1Or4RR$*QCaBIa~sMfz0l>a^ZGhoY# z`(AezyirmyXC<;33*4#>uz~gChe?RPcQ0GK&t>xx0p|WprQh>}tWRU1a{$7A_dU`# z^f&K@_eMwF>YBeZ7D(cA^{{I(a1Am32Vw3zw8i7+FZb>~Bhqj`A`FmorN?|-+lEK% z_UmJX2Y$_~&n_#c)Yf`MBhRZiChI1=H?v;hc`#;J4jIsT zFkc)0oESZngHDdi$0HQ1&sWp?$gOoo+V?!k`q>FYgyqfhyBJZh*B?TOgC6~HU+__= zD09>K^>7Nuo0dD8@}Fhs*^$bdWh8d&2t0^JU$m0rK{#(LsX~9G7dhrVM?nn{ep&(8 zQy2;rosrVK_WSy3&Th7o-%jt&AZEPiLJ0yF%}AE9iuRPy3@F%g*r5Bs>W1hJNr!6< zla>7-T37HTeKt={7~oa7T=M1KfCex_p}a6%Qwy^G!cUvhFN6QC&nHP)kw*6qg8=44 zkAJ#aj8=Qu1GG6*nUfJ-h+MuX*^i_By*nQh3*FiUaoF7MNT$}Y=KCPWxAYB;V`iaJk zR5NIwI)HF3VOx^RMk(*pu+x?{uaW=*1pfISl1k#_XeUQh6g3|vzd7ewZ1p8o$lhl; zL?WqlSP04f`;~3h6NtZA&S!Y$uWv@&w2r~b24;9tJ^%XjU<|`XCuPFZ`kItA_{uh$ zUW$2a5hD}96|<3ki%}W{lN>UU zGh+)2*Qa5g_ouh+_F0MPv8h76n3fLSN74;BG$7kSiBApTAnHKO^{R6nUiB`m}l>k4NvSdZa^smpUua6>@|Rv6aThbwkZ0A z16hDi=1m-dZb^4dRuewZd9()zovwtadi9w;%gpR4bTbz$VHkZuQQ1>(foeL2?q&WQ z@UU?9mO@HTQ#_}j&*7mFIAFQ{gzWsW4W0G}Dm1MT?*Z>-!)#1aW$C|Crlban6*Y75 zQ8l}p{3XI!uM`67L%Gc79+a8N{|ywkvO^hKFu{1aR_@)>`UQ!gJeb!EO8NV6r7e9- zPUjv1PdUtin|1SoH-iY&e3VSVf1D(d!56@jdPS`-BtrwyD19s zg%Y$di17|VFIg?H#Py0glwm>W{wRkfTI{~=xvVv!*c8Y^q!=qZ1b2TTh@%LsS(5yQ zREZa8%6$N9!^)SdI*lX%_Z0fhmWK@)lV#7Sx{x#5s(6gDr$vg(*x%66W(=~L=vg;T zP$*G{4)38$VV(qF24=ygF-jltM&a6MNZk01+JCpVe@Ih-tPrpx-|IZ@itH45oT zY&i)pRB!F~OuW>!FDbV=*Ai0&$c#+aeg}d77 z6rkdXzkOYgWFcl{324w)hhr0_FhYl_eFG(*GJdC@%BEw>auTV(rIVN&*%`M0*bRkz%VX41MdrwNmu4l#nHF_auwh)*+ns z7@=*nbdofmL0FbZMQ1pKgbBmee&Nv zFS8h(bhwB4@moT%j)!bwE^Rdhv6k$IT}Pf|xC9YuTf{zk%0Usf!E^fRNosL%6GIfcx<5({rH{vtRH zA1ZZuCz91~wk|KPlE5z=e{{T=@rF27{A)i5CZi!NjsE6loXX1)rz+coHpXNrqe1~t z`MMFggaRXuwf&9x3j>M2pl=pmnkKnM@ErPTbQ`x5pcUD802|Hyo18wE@rmx*U0ixj5eG0BBUgvb1F3Q^fQN9z{EOymLXSW>ZG&G|uB3e+( z3xCy3f?0tAY#nky;IhwD@IHsHB50gxvn^H!I=8x5z=Fpnl#(Sz_g{O@(B0h(-6h>!QqnneBi-VFq?9x$B}hptEy&O*r63|P zbO=aFi2826zj)6%{K0iGan0<_TF;(Y>$yL7V^F;(O}K+e>9#e5q?2HA%bRGZ?^rNm z*N|qAUiR8GRt_2a=(ckj4IqnPQFa#aTPCY|wFgsVuJ^{{DSafCo(J{c5_fP%;QbOg zmBYsjK~q6?5QqD#;p9P59xzc?0e4Q=PPCFGn(~vc*>fA2U*MI8aySk?_Rf?UU$$}S z)YK<*YG{_0md;U4Is?*0B+~gFzA(ZsEFVrhv6A-bdY~s)r#P4yPg04*gDvM6%4*XY zZc@nP2#5xIDBN-1?N@R$%y&%^u+2TZZWP7XcQC=!J~hfGei*}9H<I1QL>h6W|7kPAU2x9CH33@cAwHgT6 zHe>OLigO(k;9~98Z7(6m*I}%8m=R~Ml)t31U;{Hs-KhAJ!hDPqD9Cy+hibhhGjSC z-+AgPM(Np|Se^xY4Q@+gjzzizX6GCf2)@xXuScEep@TDYd$y}npCtnUyyt7I|7gb;Z{pk27^I=_EF(j=NxmJgN?&1D|1q>OOke#GLrS9)3(vQp{CqYDfTP2?4g)Ilbvw1 zg^%38r!4U@b#hE4YD&IBfwG;c=HPH#E=iuLz&IPBPe*aYzj()_BAE$DKlkCcEap)R zXm{#7XgytBeDZc^O_SchJ#wyoC?^-KLI@T~;ysZ0*&SRSb~7mrs}#H(t%w2^-)x_C zh*&sbk18(^K0I`ACI{4_hRu^46(Tv}ORe&@yO|_|slOXfLv1+LYM4IDYDY6-z+uw6 zHTjr2!YNy_3{;y+Y+9CoOguX(Ld#Q~tXukj3@#le#qCLF=ogyp2g97Xi>I+%q<{2C zG#USfpkwSLeZpgt=&ueRa$liup*bOj8nEyoqF%l!={i}opN_>giU5)ZGJSgyN#*?% zNN=xtF@JNZq*%^?st;N${hh^coj>`aO(Z@X5?Lq~zv=IPe7UIX)r$|fONUob`I4f% z*?mv6Mw68L1I5i#@%KV}Ie27WI!xMKq>%W-kX=3lEAgZ#JrytB!;kw} z3H)$>J&nc4w=1uRKDHcWRbe~`srp6k4LCi(-Vqz|B*wxTvb)^4EL?EGp7Z(&GVX<1hf_)=4&aLD{N-2e{n6fIu?S$%o2e{V5uF}yd;-e0P03B`XfD%R&;Z;3q%+M#%i1{(+bA3G zjYm#o{aL8@C>|Hk6>=jXfd{(OgsFao!YvzGDfe}*zooKYi#;Q=RON7s-P9V;x3N*c zl4K3jhHonMfz*|CRUKgmE5vtEB0p#GRx)TNf3KCRU37p1b)EZf>{!vFoYuVTM5vWb zAPtP4v{!$+{zgK2@c9oGcF}?LC=z&8!USjckX1^2+%z@Q3D0#MF>BB$BZs7WK|WNS zZ{^aE_S6l%FmiER7WW8Wfj`&4)NW=}eRpGGPUxi#E7-e$P6wLXD~re=KmYj=Ncu9ZF?X*6L=m|cVJ>>7pN?9j z3Xg?wQr3SZE&nFQNJfy>Cvj*~*t*lG-BKX@);A@0#9b^vG6oa z?t68V=iMBmEVkGfCLR^TN9oSri62p84e7M;?@T496M|ZBS#ce0G|Lc!bPjvZbkse)H_SPb3C^SrqSm}PTpVKI7cNx;=3DoT6YplC-Z%bRBn z!YSBxv*J%{snR9-k!OZ*1zNlCZ;>LbBiQ|6=s9U0>a!Ds;`a5*i?^&6l||Ib6>~+j zV4Gj8dnj5UTJXpxW}}G5>n?mjr6dx}?Zb3(;m;*lqbP~r!6pCV9aj-p8(Qv3maIFu z^Oez4r(ws7N?+uOpRjT?{$67RIc=*1uC`n z&ts8_feikug32)`%vnBNY00;$DA8L2LZrm^XAO$72PQG~+(Uo5lO;sR9JzSql-uk- zBy-oZ|FZ2m2o-^u@6#^t;F7(~#7Y}vEjYe;{I&Wu_3Y!EUnj%r0-ZT8uUIW1-8w<8 z-Y1#vxlDK*q5W+V%+)6^K8i7(rY2y2IwV26{M@@=z6%v#1R{2zL|dsj->>;M`^{Sh z)K>cyg>p6~c~&)hWt&!bPf|5%7$1__zZ)Q{6s@(_-A$Te&?q!@#toDK(xTx82KkzT zWQFc&7jMOI@vrS_{{=B%;ePm%|6cf}BOV-q0dWFNIoMsmC261mRBCH1#gKk{*` z_e-bHCc2#X;D$r?@M%LONyy75#N~&R2kn^dlxMXyIk72#J`5n_{Z>S0m=`2&Jcd-^ zC?m8zG^3}vX@8@rk^0$U+jl|=8675Tr%GL}yYPua3by1oZM&bI{y<-^ZH;n(c2SGj zEM_a0o6J+cWg&-|?akyA!{P^fNRzjl6o~{O{acL{$2_=#X{gGI7>*pV3Q8R1&z+he zar{yMPX|-w0BT1w$o0+}&3r(!1%^65wNW*G{!c3z`l+kr{gW>nH^Lr-1~!_3HL7_k zP}q_q?DvqC1NLpJYqcDb|DeqHVGx8@zl5jx;RGLViS@o$`Dh$bnc7?4Pl>MMciN8H zQhvAz8>#BUL^)+#F{F=A1LDSV#EltEYVetU>+v4G-AmXWHw4|9T2BrnyIyw^;8BdQHOfBFxp?ulfxDDcO8vBl_;Tl>HlW!mgiWjzGKy8O5HWnJO$v^m9D4|_+Au<8l( zh-V*`6;9l30)HCNw%Q02lDARXNRyK-5qZ+A*OCCfdHV`{bU#2eK{1A32Oc%~M_BWB zAV`n89g7n_+WsEgDx1yY^VJ58syuqBV~x+229QjK^TU8*lLKtTh%&ey+$JSU41k<; z7|IF{D0Y2bu6kg+Bc$)3n)~k6d5u73t{;e%5=e@bw1L}Mg(J*C=>X?LL0}5|Kiny} z?U3*o{Pj&V6ZjweXtIbe2IQCi3rziQiq%pJ=wQc?<~FN-0|Hk^*}EV!RZMcwBKHc}k2j8@o^Qt7V_GFO7H z!ZJuuIDV2aB-oUO6TL$QxZ(zaOL1B1m`HAZkbQz7d3ENe83EgGyZ0;?rT71Uvj2r$ zDFtw*EB%Mk`XlP{e11&i}z&f$>LJNLL@(-y(q5QUDI(QAu4OP-o{j zzKa?MkW|*bee}n4IOk9f@DPF?t#$iR5AQIdUXNDL0+U@iy05GN*^&d+ng)eD7w|kr z$HA@uGis3QG;1y2ZHCGU?y7-2S7b4zD)v` zQIjz|Ky!aF`+2UT3o}aWlc}P%?ZdDNwy_YmvpK;WwnPJEu)3WD-S$ZBxfN0(4<(eE4 z$na`eQWXdL|NcCbAX$??9VAPaIqm>$Q+K;q^iB4SRmZB@A>F4H66Qh68mm{yDnv2| zAYGvFH}qZk=YBhjf$)ADPWq6;<^;y@J$)ic_>V_S?*q)QI=~}tn*Mn{0UDZatIJIr zZF^Ybb10!gRkXlo?LvYMGJ!(TvWXuNV*4LqkXPc~FZBtt3Ijl502S9GR6~~`)}OuH z9d=HSt2-xKoBU?m@^z-EbaVepZ%J^3!c?%M`icE(T;}iKgJVxQrf*VOJ=5);+~u0d zwo^eqeoL#Zm)<-tkv_c;ow9BxRDsI-avDW@b*u{?HM^T$sy?Jy0Kur;i_iN1E)V9Z zWOv8pfNoJ}0+Q@3%9LXdXoR zhq|(k++N~o4G{C7tz<;z>nqzdg|abaTE8ycQZb_PPjJ0Qo@3m%vZsQX#_}-)>#f?4 z@4ohcZ>4MESjMzodiRH&tmnkS`Ls;B*DixhaHwdoJZWZ(@E>@r{SXv=y8(mr=Vgj5 z?!{&BVYA1;bX4fCTz;yHwz}aaJyCA|&xD8q<;YcGkatef_psHyP&03l3c)ubKiilGjdm;6v(}YTEq7|fH5!G%J`%(|%k?2iKK>k119F}}snRaQrdk)-6 zPhs=o-1c|O1aLxOR&+T(-MH40bkw)_s{`mD<6x$XRyR@xd1tG$7uSD>Q(x)9p7>8! z;UNT99Ot`ABL7XM(oL_l5h&yhNN*P@-hogRdi{Bvmb=( z?{9m<0=cK<{}<)_FOu@UR{bA0{@0W5QTzW?kpIQ+aazUOuJ0+&!vC%@VA=ftAK{J( zIWcGiYs&wWM6Lh(ZZ#-;ZRnpHbsBS8fb{QH`yn*cud@YFBE-nzRW#*M%4skc;Pv2N zX_WF}y1R;8{dU{g)bKgx0l)jxQG=s>Ld1v3@z*XsA!$MMHPPO5*Igv}3#_CA!Ee@z z_=$dwtj=;=HUS+)YC3f$TZ{e=$|EPV&8jGX<<;$A_HP0S3{O`@mMMzc1ENX=XxH77 zzYxLrPHnEGyatw-B*_8Bwo_4;eb7Z;*Ch+??55nIBl#&e-|76lFpExmwxB0-)G|OQ zbNE7qaQ!L%Z^a=|Y{c&P{yS?D6y+?q%e?Hfz?YKkCwx-d+}*NbPV-i70E(KG6r$zD zT$QTJkT|K0wq{7y4Q|1fk+(m59|Z37iE*WJ8(L@i3};5)#DQ503K0zx?d4-P2GZV( z74>&-k5y=X9jSc!V_za}yFFvbNrVB0Ds;afLFJG^^!#k{H?Vz6?OZ@`Nv`36#eUpvq5)l;Ce&aoxH{nZ(LzUex( z!s}LiKe6mmxizeOfVb_YYK-y_1*w;|o?x`oI|_e4jr!8uWgHNu0&i)|ctw<`~o zTx4s-vM5_x+O#)$y{1Ruyl*W-gPEo`f!wy%>ks1*p*!w!2kk;qN?)d{4LG1bl~-v8 zd|2?Sw`Ul)%~HNr$zO7Bt?^p5`)s?n*!_OvWlNJmz1N>s?9bW#e26t_uD>EGJ?1D? zmu@sxz4(N_1^rL`6-n@E%Po4N!W%HfQFpT~|F!MSLnZtDQ~S6j794~i8q(@F6m`>A z$Z|6(>PdXa6J&^e_c-FsmZ0AP@&ypO33Ay6B-Jv+f;)rj7_D%WG{9))$;k)_ZiCaV zt`mk_k378_`PZmH6tOtKcv-C>x38riz9U*v^GzBJ$0g{zpGb4H9MlhjLb%?igCP9~ z;*q=Hk<>&YJ;1JvR7&r?y?bPo2sH9FhR@gD+vL`R;3?l`4ERY<`9kp5GHW+ewYXhN z^9{f5O}FQPTVvMMk-tID5%~|!TkMUV&qc?^t0x*yFEl_d z)b)GbmO4Oxdkabt4SP)P;ZTt7Our#hz4{A6U8z2S<>PW7N67D9EGmJ&2KFBQ>i~6Q zMytV~N7Aq4$uFEZRD^`r;>pt2k8Nkj1o?~C_l7y;)NYWew3$y(d{0O`vuF9W%k*a6 z`*DO-JnW2def54ck54oGyB^sLp?3QxETB6LqNq<7yTbw$f^;?ReS7mG}6qp51vJqAhIdmKzTyoWHY3*=t^J~!(w^BSgp6rd7%9NhjvIOAsW zExr|Km;37puj=ZqID|Q*{KTLBid{SdM$@9i`IQbDKpUUkzP4F75{(fEHe=m6CnN)rI z?@aJtrWu#}V^G}nA57`@eYt_1yNwRJ{5wfd*kX*dx_qs~a7wY6NHD(Jd-JMyeHsK{ zcR&vK-(=%~3+1tzkK$2ChIjZy;jS#!<;Di-)%y@_<{FN{x+u^pU z_WNh=KTt|V=Sz_Lj2cVr3KV>vf+VVK`P)K5pgWKRby6psZvK$)F>8Z_1B#*$$}q93 zV|$h<(X;>Xap(XYNMd~L+2()ThmBdnEF2ix1`*O>Hih-L7*_qR^H?;QB-@q1D%>D3o_BVcVQrqx&S%J0(H?o zlt-n4=qKGksz3aKT_V(dUUX{Z0!533aMU{)X@K;$v*g=u?Z(VosblVHu z81sil#CkUR;W^|f&^`713FviW05GCUycUJ3O{?)&O8NJbwSdG~erW5$)e*O)#7|*X zLzZt6mWyvae|H1Ubd6-YLYTZ$ro-H6!41XB7rz|UAhEv6e=hC*wa%WsAZrr4+C;kr z33TW!bn4Xg`gQk13XLyC8a*JEyd^46nEghQa@w-s_on`gSH~`NHmA;I>E3ErgjmN@FT|)3)!9bs=#2mGD_UbGh#;6AIA>Iy8p?egOgkI zb;G7S(?4S+cQ{==7o25J@1=^>w(yu+V#l$&K_gLkg4pi$ zV=y|zp0-GNlLU$Ff=Olzj<|%1gl)28J^ZzJJ9igl(AYzhR3QA z9%L*lVzWB`_t#nbltn4+%U#;99r3^-m9T!n`+NEpO z9PRoVJ9pl0S!KH9mis}ND_U_YTfW*U9Y3a*3@aTqvGc;`Q9#mw{^JoHC;`r(Pqayk zOAUIEJ=FaU@Lg0HbRppkG`Qhk0H}mjtw%AC&^P`uuZsbsWWAcqn<{_02ALVn8!=k~ zC{BCH4xu3)!@1H{@p}C(@65|t8~LXo)qeE#0WnMhzL8BeYub|VBm9%XSw{LvLHv9K z7Nz&MAO$c`x}nAiBP*}p(H^^h%&gngdX{*UF~SmM>qa^h?y_iC3>{jENh5kHAA3h>%U`N zXAEETQ8CaV0}%c_@m90ST|48lrJ6cTi^CkpJU`ct5f&x4AK z@T}x%Wd5CUm?8qD-%}SF!(up=yv;LmsU!xjlK(jLohiu5)@ktjp)7b~D)Kj|ZJ~c! z4}lJ4)_Khin@9_@%Zv>5GtL2lEi}DAqfr-*v2G?nJs~qwT+Dfz75 zly^6Vh$Rc_HxBa%#WU*ac-iAZ zcjJy`ZC?#ft+6ECgUQ@jv&m~;UX_MXdu}_y(W&u4b4M`lkuEANZe}Ggm@Rg&8(ewX zu>bIL&MLTIA@lnUn5)5IIqI*cit!0}iv96aAw|rSP57b6#s!H1TlEXhzyvpa^M5V$HiM2y-KTx8#XPXyz7fQS zxD)Dr*FF(l4#Q^DttJAy`>0`AVysBn=eDx#8ZF;&Y*`674ABIT96mC7r4)S=0 zL*>VCDLK%q3x3moC5mClriwY~qCSIZE9d^l0t=Z|l<_qGk*(OpIJqNqj@w_1-shVS zX6eAEpx9`_et}ASLNKaChG8<8yT{8JNmf54_eQJsidBCkgeCUyFERsc7ndOO9!^$0 z^TnVGppzKz{%ic9O2PF z<@52NC`z zQbZFkXm%u}$t z}|=SETiMqn$3|s;R8@jhcq}}BfDSpSC=ti(TQ33G6 zE!h8fnp0rOzw_4%fMY*H6u%R2`0H?H`WA6pb#;M;-1^aP9yNhAXTaM}bLAPU=F}*r zG~T=;&dI|wmv7tK(5XdTOHvC}GYA*s<;T(!8oyj^J+Ef~vS6h{l!LS?=D&s0!|?y= zEnuEK7gpt&GW;llKKMyJ)I62v`}!TA)BLmftbq0wsyKz$s{1C+#y#klX5i6IqPiUT znS$Ga&U-LeyHsaD7s;`MkA4Zw@AWbAfa`IppGZMe&XI#7d^Z23MpI~xg(@NcnUfc8f3-*zz?yDTM-6rxvfxn`3w|7Oa6~Lq{45F&Ta>P?dM5CjUSy2H z`m@9${nTe&nr-*|XJVvexcQ&5zVQ?3XjGf^j#BVMqKnO46aYM>cyiWm%KcBcAZEcb z+tv%;j7j^??_+^>raCr67p_G>G{nT0L03f<%zed#bEvl^2dv3fHX)iR7ZXW@?Q?i~ zNxOK1a`NB*)#w+@@88KxWD{0^OTK!bTy;~c8ECg>U`Jd0f-&ZZ5ulD!5n|HUCO>^n zQy-9ZY;{;VsdXCt#$(eUna8u`GkTVYYyQV`E3wR-I+2~iGaHUsq}8J=aOGQPD#e+| zH+Jv`Aju~ej+=*_z0Xz0t7x3XtxU|6&wQT(v{KRkPx6(r;a;G)k~)TibV$C@e{$MK zil0Ub`=Cbl2j2L47JnwTXBJK7rN+EQ(0g9IRfd|^ZwZOoKM${+x$I07kz<}A4YBJzTZ)+K%{!n>X2uZ9-#y%n!QaiAH>Ii#?leTb23l~i6R6=6 z$Ap3&+zwc!b!$Dz2r}yB+)ZUUDHwB$RIU6lS97IEQU0phN4uU77#h4-i-V{ui4Ku5 zN>9Z&na0Hn_^S)_hqfKI+aPe&oK~2BE_+bi?eMRy@a3n*aNZg3r=i z%^yFxPy+YsJ~^QcjdBAA5Z{EERZIV%4J~7SU#>2!Zg7&2o)S&xJs~<$uU2H(D5@4z zl}xK#D6VgsyFFC?^7JoOXmQZ?f2}1exI>O^qR*7TnJ(_>7??PLAB~_O{Xei{V4#y zdI_~?MRbylKL7CPtl!*bXTE&i=I~PzJiecxk_dZds5p<7L@oB3zdVJ|Ubo;$-K&Xr zSRH8|{rXDI@*Ly?gU64Mww-GxIxUQXLp>AM!#gCm6MXSmw-bx{uMLDF*Fzk$3p{z~ z7n8dZw+PpU)KC%uESMw=i9=pEd*Pa7vPmTLmx~h(r(_H>ZQV|K6;Nv#PIt5{_UG`M zfESF<4S!d)JS#Gow7auVFW_lJ#ghU9uKoLlj6)iytkzQ5gzGo5kl(L5EV6rDq^*83 zztiIZ7QeC-fWbJnUg*eE8dnYUKH65M_X9x$^Hp|0tI7`H(69W5;;RXOUjEAMKC5i? zCo+r2WCZr?z%e0$fodm|OHP#5xPGBmk<(M=v^N4vQz}U@Euxau`py-UkB*B6pkJmX z6Wo|F*k>73qMM}nm;PaPjnZ4iT00B)%Dq!F!PO&5V30@iJZ4&!fJANmGt37yM*ZCR zPp`U&VFe+==4|j^We;ETx||%)A@|G5BFP|!0uBLl!m~TzHAZI!N@n0O9L8p}!uq?xnMeqr)@M7=CxfI~EBvdx z(-p-gUSFaQ9TZ5+@c)zpo#H!rz0E*Nn`9*ZmhY6z^v`1JXk>E;Tj!9RdG#-!7M_`P z)kfl-j}|Za=m05C5tBr-SK4m~VwJw-Z^iq}dqC_gI(Z_+_&rpC6ddy~?s6Kiw_Nf4 zl)V&*RWhAW_Jg7)@+A;{lI1LL>eZnHfW>a@iS&7h5NwFLd7tnvHZoJIK1ppd^!xya$R9-zA!81V3!>b8?;c|2(%gC z16$m9#YLS9xkVDRRM`^heuAhDR~@GksI}7%__e+L4YHAXbP~1#dCLw z-LyPhmGZR8uhy(u+ut;!ArT|&81PDAKv%CXnCdpPVAWUxiY9n@1i+0}93XcxMp&-z zmEls|14*lbkUbOn!EFocmoyzqa(qpsj4DwE*k01+i>(%D9adA&*piwmd(=CMq|?f* z4#-IJrp+X|NV+)!pD^!ZlR!Tl-05u!V+xJsv{~|f3GofvFpM}fTd`6aO{JzjHSbev z(*E&gmRs^1px!a~MEa#AzdP|YXjGA!QUMLEWG16foNl(Eyko}~ojuySc8(Nok4i1l z{VLw;Q=s*bxcHL%r4wXNP3V3(_X1aWv$Uo)r1d&UM%cqDghXNYAJD#iZb6MrPtwq0 z>QND_*%K1m)KaLvh~~viRfSKpgj6=gL*R;v#c&GI2gHWr#l*&q>)e#QdGsiM8L~)- zXhqaKAXsb3)qPdhPFXCZOPw8Ap+jA$IN?6f#@N@a{j0PG#f$$?vF4@%%=sSUWaY6| zBH8@L9&eGrTdszfkgm;H%f!c#-0B(rwK8GWxb%EDkxc%f)J_Gy+>VTof9>eIwD5sY z7*ZNGo_ZV&!8h*=`!26;Dg{uxpSn!WG#OuDOTKTx$DTkVDftEyE1usBtxOzRDZW%| zUshe|=wt?j)b+m@CmR(y7jtT+uroDMDl@ux7P;>^y@6_Ca?SCZU?ERl;JfZP$&57ET%tlVrSu_ zD->N*sfoaQd`I5$cg+LXhp4QDSGK36A7(Qm8FO)eJf%*oUarI`%MJM>-cyyKh@mQU z01-B%WWkH4tNu18dn;=*M#9G((_ZY>IGfsI)Kcjq z+Ea>(4Hw?{Z+h-&w_kSATA=IvZq!SmQtB1Dj)7-CQ$9EEhm??TVLM1JhH82KE}H%Z z;&1_}uWRVv+2od&*9koB0%f~T5nR3^4W5#YV#V4zhN-6geCk#nTM+DK)QAbXPWiV^ z;--h^{c=bs954NMJt(6WV345wyjWV-I2d)xH_F>e4^X(`u5W}~VB;f;#ME~NarA?x zK3gh7h6{e%6G40rDbj%t64=l4{Q?tgKDE@#CU6uFI}GA)>-pk|XuW6PYyV*e_&WFU z+F%0ZT%qrJpC66;r3lfm?r45zs}vgRIgAT-vcaI%_GLJ)Eq*LAi|q@$HR&v%GRT!E z7aVjgD=wr8a*Bnw1Mm>V>h|&thPkvW;%KGon_L53F7|7?+1lmw#^^f#^!EyDB=PJCq~GZiK4mVP+S8BvU+9E zZ-=LYv8TUBqJ`pW*t7gHw<$>}F3>L#7Bi7j)Pho3)_uKAiH5McHo!t=Tss!_p6-DG zNiMo*0IZ~rD#a|nvw_wnn=2KYM?M3`*eDx2vGz=pYkX@PrRJTDV5c8ZkZ*LkCtpH-HY+UNpyEuAZpoHC)WLRURHc3^cge zr||WxZoqn)~)tF9z1Us4dJ9E99&;UR^koYmntUSqdVQK#DXF-G#b7bn8 zdKoQkZjO0C-+`bhgLTIpOYFj5VIRuxc)acWhf7utsq`a5+1(6zoR8aXt1(OXzpB39 zA?8x9NH?v&n&Oe?*BY{UtEdr|`OX`HSm~u&wQht(m)b2iq+pu~we@>i*~h(l3dTqY z6iW6JVoUht2yTFqpqLY9iaoB_NA5Y??_%GMQ%p4KdN@qiw`OdRe?w7sV|;_!1p4Bm zBXi*Xoug_#CeDfv-v?f_A|_HA_u^urRV=MOr;aZV_qz|iA_`8Jw*ZBSJRy-Fw)sOgQPh!l|E5A(6t0*&=9tkQp>Z2-uG1NG;v`4JZ8^lQSD-16XkWiQ03;PHF#x)NCSpHl-wbqr zN@B;as*>uPvuf=fL;S@2M5~BsRU}VwHn@5zeJ2WuenQ|!YLqCpta31FFHo!aivKa8 zCe0SOW?9FTY^lcJBE;jHfpuHx|1fa6%0frjshfdaMTc0A{DD<9azOHPLHxBOise~c zoYQ#o$b*{}&0be7SaZcG`ih!5q8|FteeWG^i*>4l`yXo}b^v^VA0^LuX4urqn|Z@j zu=xcrF|mC{$+{^{(9xA6lqdKt&^zjQ)0BU-g9=HcUnWQyT!5cD22|Q3Cv4?K-_MZx zgm`dUWRsxS(*UbJ7NgMa=$jZj%s$7+Bmoo+Uyz8$iy-3HYL2gv(}PKmSJu+@DF8_$ zo?8e?lJL=V-oYr8poQr-*uOOq1H)%G6&5lTX|+1 zo9%vbPb(g1EUR3UZ3I}@gSt_b-b*;x7et`Ba>+Gn|nb=R+lV>efB`$ZALDo_sR3FCKOuoV? zaWr>s?W{y5)EYiH`8}F`c<%9c561Lhp;fg)K1rux8g)1bz zqy}rykrkZ9j27dP>Vy0p`JKAxDpyM%%_$aJqb1zBi?*MWz>O+@3p1%|*V1ZmHer48 zC2GpT6QlSUqce_>nGqWr{4U@3&PhyBhh1;RV$sB4=~z@zQ{E5FRr<0AMEWtI*n?O{ zqLE~NTvVx7To1_8YWnltqO;u;6fq`uEGoS+w+tsR5EJf%xAo*{xN|HNuGW4EqFm!7 zIjc+;^GyFvnLPb+u{hT868T6|jrlCu3i|p}jTqzjD`TQF+RNy$k2rS?sqSXTRQX@0 zn>NQ%JH{#}2dwm0saQrl=^bN*Av=WJoGflvQ|atigrN+sZOYq;W%dFlKEeCO6T-H z`4{T4LsJ9?$oEJAQ3;Dr1=shHROJ^`PEpb3ypWSKC<>KACO(czFeB$pEP>5eh~QWw zw0C!Tbmh|JXR0hB4T;Ryp_5BX`A~*&tO|1e*@!8yL7IQi7zAfi1O+U2_7%q->qe*j zaK^o|hQM)nds2S<0+;9EwNAe2Z6ncP&^Gpgac^E~pmxv|Wej$PHhJpjr#qq}Das2! zCGbU&i};$(iRDgNHJhpQu)$Xp%Dovo=MFPDUYS>g&1PuyL{Xj6e5OBt$szpmSjS$1 zc|nTbRgU<1(<&Y`E+DlB59YpzErf^Y=mP72FB1O~uiDhV?KhtsHyjI2IVGd7rMSQO z; zq`rw}n2%#4AIwptr`Ulfjnn(*44?~A`Ku>P#aKCDD6a_n1)^uG5v+J3t{@g_q~xMk z3%HsG+BGTt--@C>;-o}L;(KgNSLb0qzWPDJ)RL>pF(u!QJ`k2Av&S7=Y#1U=9}`BL zUY$J*w_iYd%g~8Gvf#zYf_e?aQdIvK==bFmEos0Z+<&mS@KF&&U}m^BvumK2jK@C=5e%{SAB&ktl#KF4<|+g@;p zn{JwgmCr0V8@X68(N|6){0TT+ zZb_t`b)p!uoAnzjS%E0iOuq4>(_RGMofxGk)isL_d)Pm!R6KUj8% zv!ui~Wk-%D&sWqmdZfzxmIGj<1l}7N6nwLRPx_p2Zc{A2h0`W75)eTZ3-X01;B)U_ z4OF_Uyx5`UhE9|nTAXzx#=MN?=a2aM5@Sz%Uyg*Rx->oTKVJxPchJ`xr;F zbcrXS+<-6pgTM?aBrhS5hyCeYZ8A+aC6MCO)L=!P_a}Cbc_C?ge9&A3ppUhN;B0Ks zO@9|7^MAdl75Ccx@7vhRB%joyN?|p_<^0RNkq)vt3QUcYteUin36%|Ta-U8)irT~; zSGW0y`;6>k5~jrV7l99XDlx@8s6NbWTCo}J3w?Z$4&r;=>$FIsIQ-;mNu42)l4e{OZ$m0s0-M(iNoE+YN%v_6nlm*Hhgf;iW{Ka@t(0kb(96W8$0)ZN zt6xEF;#0;sI-hpL>ZhuUzW%zXHRV`<;nm?RL4^$|9{kYF^b2aLWDx3#u+{yQsL$Gy z!mXIc^GVeKQZ31qY6BoL=pT{F)@d#>zjlvyweL9RqzAH?#}>o3ehi1%aB$367Wzo+ zWiXOqIlnej!0%KODM$e?I(-$`RW%^~yp**>Hi6XP^B-wEfRKi6*i_riu%CJw z=@jo``Uqx_gKBqjLx#e%cW8!#^F_td0%m2GD#G0a)zZPFl6v;Ij&_e7oz(C7qj3uv zH#A=3((!8w0qQ%0Z2_Di%#2dG6^8G znOICz`i!|H zQ9ju-)JU()H`clZK7%|n2h|SkZ=1bNt=1|ABuL*FnueB5cqCKHipA%N)2caRxl@d@ z@wL2gO%Wr9_qH5tV<96R(gThRIR(=WU*Ek1UQ%MIvWaTMd+Bi30m$mx)2TjoIV|c0 zWTHjPN6=&vyP??0_c2kq_l4fwSi9>lF72Hu#79PK#K74OBPEX*Usp|kGz#r&Csu08 zlJghZ=!64&afe`EPW6B{OlHNy`69=Q60)gz!(=tnL)}D`SBToZtHNFes z{>Uj?x;O8&Jv%y|Wj5Kwsqh_$%F(aX;CeSN*PmVE3kr4evBKd z`wVzoARMOO)+4-a7$b-F;F_~<@)7a2)3qd2?sT#lQnT^?U&br_iExWvSJA+)yL!3R zUvP>BV}9)C+v$4wTGyv>kHB`lCO(HlZNRccn<@zMl=?wl^kL#m8oWGB)==?4~!R}NgE`if( zr{~bCXVW_4oFb!=KsI%Z?ir&tBDtGEjbt}8XN0beXk!cf?kw=HO})Y&Rh%m4zx%XY zJ6eY#h=d*?HAzlOOiM~&AQ))2`g6vZ=fp%ejhK_{Q0F}H-vg>>yNdo_}^A5tO-JLWotwi#6|0b zD<_$$`vcQV{BgX}(UB|ZN<0-*f2K8u5V<0ULEjC$m>ee( zpvX;z(zH72dKgTo-Y(0!@Ps2%MNjEt4=$@!&Yv;Wl7n4-g86Nd;SlXFL!$(AT(x8m z4d}Hoc?j!<+I``4xiU;8nE&zcZ5zXdZI?Nh&4?IcWky zOD_n2ABDPpYTEhs922HvjUoN6MX}deTePH8*&p|JZGlFGEAd5yc{=7o-$wf&k?9sA z>RWEN0MHSIk6PqkinAT9vLWrRI&`T|l`lF5Ek9&1)$e~RLuzZTMvK{7>Mi8l| zQ5p1<>M#6@@a;uM%yrh$f}zJ#INI>scACN=mu9jt)ny1lple|E{BaQjWjcY3pJl^W zh4zr{39BWm4zaFq6h{$>>Jzb`*{S*0t0rNiqg^3lzn!X!+jRamr5}uT4e2(g0imcA zy6DuSNULx>PEZL5{!M=T_o(S@hx4p^cL1N~%muvAohznwWD($%!-=XuXFdR0=`6Qv$$O z=(I{eLu#GlBdEf^Cm;C19q^kg;uZzXFu@=Nxh(})fs;YH%@&9bfGmOR`#sM%DC(3$ z>GDH;Gy27%T*t)3FDWdfG%E2I-2u9@N?mi0B_l zGs|PmBz_M4Y^nnP!&m$_A)$TGS9K5o-~Zf$6f)xR)_dCYi(lLhZ{jwpN41F;7azO# z=*j@`U;U(Ypr{tN{{R@^lf=A1oD768Kv)3qRY#Z!0jmL8kP~HoLV8!&X~Fx9FG6DJ zZocf=^v!FFIpRlvIsNq14m#3(R*xL0dQbvB8$AEy;Q7DjH>R0^DOPh3O_0Dvz?u{~ zIcTs1gqvE63QP+}NY?G14dg5~{`-t~Sg#RoD9&wm>FyJ__0vGu{sc(VN!ad&~V~v-1D~3nxg3C zDz;GETX*(9g({6Kjw(PTjwMqmvq2XPR$QB9Pv?neZpSMRoji3U`zpuY2>+L;#fsNF zXH^!!MGzcPFqee9IU3Z=p59gA{4Vgr9(~%~U1<6_#oZ%bt4X7$Tup9{2B^{^UV5SC=_Tsrn6;;k~TS34X1e zLFe>IA4pL1eo9xB&PurAq;n!uGEVJLHMkBx-#+os*A;ZDCxx@BksIDZazOY4I%>Hq z)H$SoSSN~8R6W~09I!tT+9L~0<=Vw|wjgDn6n%rLLu{XwOc;xNyqF*URg@Ub?Dn9H z*JqE$^lMVNQysnom}t^v$>;cb-J9bsPnS|`K7{OXe(X5{&pkaTDRC|YddbsjEJl?T z@X@fx+vv%P)7XJC^P0hy)_9tH;cb~mMT-2QFZq@wCuA%uJZt2)G-|LyN8S=L5;>J3 zF-;!C;<=x>xG-EnlG5!DQh#VUR$jO1NZWm1(_vNepwz&XRHH22@qFa~9H|)bLOixy z6oQyr=JJ@IO;Zs}i=>AJ**6Xh8BP#{eAWJ9IqDV-jm(sHK0(b3*3@4OT|=qUK| zCuD3yE-mxBbSLljz%Kn`S41bN%y&S>{%LsyeXcy3_Ngm}I6H|)4oyqzm1aw?k2bRJ zhs=I7!p)*McYe(|{_tp)&9${-w!IvFvaW~>_L5{9eRdzWE*e6>JtScF*lcb^hs4(MucOm@@BjZJiCCd~|BxGd1k{}oDeoq0 zW!-oUpoUy2TYza;rvAkulr)e5C)FaMJy;bu{tcNMxO7cwZ1&=-lmFF+6{L}iwtw9{ z%s`Ry!UQ$R+MKTE1~G(!BORsEx}_2lWLX*Q5dl^KST264BsR??5W!_dv%s(rAb;Q0 z%iGB+{slt)w%@?-oe!DpRz2B3t@9G{YlOFiRFy@%5|+1wcH{3Amfb^T3z!5fmkp2M z7BX_z^UD2sZL&JId8V>&s(olR4wUqbE$#c8?RrP-z#%a~sT>>7aF9{Abl@va;M9&f zxJVWLN-cBHnhWeoa4M^JhO#oOH@Lk`b?HD5>(x3V+j19b$Y=v+Bn_?SShu`TWU4bV zDI*a!N!4-^%zN>V3qyE%h~sOCbJ}`T{ira&nHo5w2zh>W0o>Pa5!1e9!VY*49PDO4 z*8guYoKxtQu|6Vds4`RxUYexaJCXd>4Q5r_U6|HHj;Se>MRq;7mc1av&I;O7N#Z0-saoR(pY4YpLYzeB){(T7T_Q zBC`U(%`Epb4`_xn6?oV*r=%DY0pPY=lz^RmRjTyI$`$EDaaeYj|AwP1$P{**Fx+sC zGM=lr14qcU-)aZsVa+ZPmB5K~bLCcSA%~|TbkrtcwfxDJ58agP`RCaqtTbWd@P(T; zWx`hlzqDf2xmCQE>I$x323xXRbp`ZxLF%4xQwIC~FfB<1tuy{qc_}XWwL|%fFGIkw zQM+y|9%Lhyme!}2OT4n~n{}>|6&HpuzIcQWh7dhGQR+j0 zJtJwVD@PM0$~h{OhZrUuK`oidmOEBXfgKT{}da*0r*Z4k(AhG_r39V zV`&l@7ZEOl>SCE5`?}B78_WzWG8CV=-Bns_@L{AfY?>>W$v-}#=r$g=V`c^wS!Lv% z*nU{6fZ?2gDlEp00!zfme!X>yKjXczNnovYnDdhZs+!;B4Le+M=r`{gZ*!C2eG@Br zK~uoAaFh!i4PWuGwZbOu0=FkMI#XBz&pP%tV^dQ*#Ng15)ts>Qe zRYLF0P(tW6KUK5!~v9t3h1 z27wkWKp@#T5Qy0+sZvJ{_~NY710z=u=<-$4f3k;atKr2Z-fU zM=KZv?%)Jm4Fbu*Wr2?lFgHtXxP$!*S6R3`@97n?z-Q8FAztp&OWf?_c?~tRxs@GV zVB8WAafmRl!XMn++;T2YpUFPDcmE%^1Aoc$+PJwn$qEU1dU`@U#UPF@);HJt>X)B(sM1X9NpdId3i|%{p;Uf z^>nj+_V1Ejxc;MAKm&zH-w267goXa~Y~WTo(pgz87h4!mGU@pWB66o!{&wyk<;V$< z3jdGF{MFLavp`c7{*V*;SGOtrq5gy42n13BJ-l~E4^Fn6OdVjVvVGFE`s{nwl`l#G zpwBzJ+^L^l28f?wd(*~X@2}FBa>jmGTU)6KVntoW`;wr1QCm-&te%|a&-{nnXDNF< zzgL~?wa?03Ye27|f0b2vCHksPXx?^xwmKK=Ig#Y^EF)ctGMn#SVa#RVO6V=m@#m7y zi?@ly{+>s?yxin}KeRQu+r>Mc7!64(8-yi0m39_IOr5se#av>F} zt@$Z~tUHkyoIf#tQ`+0jyvNl;*!cL~I^{A=N+IMj)boa**!f3m?hrNQHo26{EUloW zb?IET@M0u|=|J53b%_TgrR`$$rS15(s5}oktbzj$#>qTzDZ0Y9WO$56 zx2rtV?K7X_5)6_;R5of8iN0}jXTL~#En7X3U^3Y9c+e4Io>65cc#V(Yi4q-Yus2?E9O}vxv&@*n?eka*NrTvr)y6k4b(KCblWB z1!ofJ@)B7qBvnwGmz|&xyK&qrXDg(os;!S|`?BzyV5X}6bua}(d^mVAGr$#H?*>c6tMzAUzN}6eUziA1m+;vyB%WAlRBF5RVrl zY>mTOwCD3P&GYIHu9-R{K@2pm4TPAW9c&CP_7cC?ZAZ|@Zk9jD$1om9%M*+AnlY)J zolS^beZdl}bM7EhH3(OJH=P~ClYVMuLCT-Lk#(a_-uQTX=iZhEmF6v%o?bUA^l~(QC6wS*C1>7$O=we+Q#U0ej8GBj4P$i}nsCMwRE`*)E>~dX){bWI zURXTqTi4_g0@eUBhgWe%7j3uosLB_qh?>nhD`1NVfo=E|LiNs^vk}5f;hnD$S5}_M zHK0Dv9eOKM-Qqc)qC0WVSF(RzEZG!wG`L)4;%_zdI9DTHT^*uS2sh= zVfF=Yay=MJm9#ZVfGqZ59#x2IAW%D!D(+c1z_@v!cFG@U8O1MwA0!bC{xN7?K(~js zRYywaYW2ZHZM{&C=W8&|zH1tDvT`YHAJa`sYn{L4e|0F?Ppz!ZQQXs$)L@Rteqto= zuoQJ;DJ{pr$ScqKI(~5=8ToV5_CS1PTcr@=WpYK?=*gNb60KqBhtsY)-hZa{()FR! z?$X&!jhx6_%QH{x3P96GExvkY0h&66q$x}ruJo9K`{rKJiWtbl*wb_O{wlqB>#Nf4 zt-z+z!;fX71T?WEX8la=fBO$^@{+R_^vfmAv4?e|CvGJp-sSPab0#dQ(UxbOCgd*4 z=PuKI{p+^w^MMIeO2s>rI5SUAr2BTT5H5O3*@jLdHsRK&rFG)TON!iEU@GUS*?AF6J%B~f5TY8zb3+kq3-dEQN-PSf&Xxe!o`$v z8gu8Cf=g9xnFC7_1CA3}iw?pJxZ0H2hmedy=7U2Vn;7j17np%CD6hD5y=1VB=cA5^ zTEGGKLjGxQCMju9aDO|90qqR8t)Gz(m1nt*i%@@KK-DlAng>~;_n|B;F4Y_VzMXN+ zs+a4uje$xQ1(}b6a`Oi@SXYmqN2qQ$s0K_ZDIXjyL4-XOu+Q^rXPIdynxt&J4y|b=ulI<1>_-Uo@ zj9+a*TSC8H#E@-n$b%Y~C3*a8^9jdxwhOL3gUoKBe~M3-z1EiXXlKCYU@A>U)cZnLlyi!BThzmVHNDx`)uWcfBI|*+BDjiW=m-b9 zruT^1^XTQ)>PHbzNDoXvlKMm0gOA&X?7f*hWq$2sg8WV2JILpbNX`J|iig>X;`&_5 zC$2%Ct%coY_}20}kK6htltyQ8f|$(8J~w$nYkyycxaONeL; zN-uk$^&}?aKn&|WhIg4cdvxS@h0FMxl*XLP(a$PQq-|oxL0YzAk@A3Qo>WI{`Vx!h zsARN|fEFyY_tfnK7kT4*iQC0@f+tVz_YsdYh8FX`)_l#WJsRYjNby^Y%2mxkIrE6c zRT30*f7w)-C(XNeW@Jb zR#kEa%AMk^d`$6?`t@BXWm1-*WV24L%4r<`E1Hu6Iuf*bG3ihJ;9FnDZC?c`^iFPS zBKu_1|GJDy^Am*P6Wd4qk>h=DV_0Uxu$1?f4I5Kkd)B2xZcx_eBFM zX#@NMdt~ybKMlp&(4qV}>OsSd;r_m(2Dd0Y_&%213XRHIz5%9e-Msdu?Cy54P;ZuK z*x}@);{dDWdp1UQbJovUrN8Ae1*;dFD7ZDRvUG7Bqcw<9oi`T1dcUtwho|>!J{BW9C(qq}sxW>1lBTl5j zjPO3Tu{t~ew$+C5dSm83@Yth;!?Tfk(PPZPTFAR6jCbjnFvIEW`0LG^7j4qfYWM31 zKA%{BcUpl~@!zEN;iJP3O{nCrKZ1cyv zEsCQub4g5hlDrqQJnG1(^=Jc*J~td0l{P6H7h1jM@Y|g#UCOIk#BNc4 zdHOfo;3e~fSah~tn2ckHq(am~>o1)8Uhp5}kLdyiXq}F?*%>I#nvX933Xn6O_H7%r z()+TSm5C}+1ga$II;%eBg1H4vR6LwwAR_$_`B*_-*FT6UtA+Zl@zM7m5`e6rHCV=U-QB4;r)C`nP&dU?UZ- z^R4kjf6n>-z0Z&MJ!NFTz??(H1M4+s1G<9qhN|r1yb<9GaVE{s*sE)ZjGRXt3N^*; zW914>zZnQOd52b=JjmFfqi}j{zpWqz6{{Ee@vi^_7ALQn`oVoYXYJe=oov~JUlrnD zY@$>{>X$fc!n*L7RoJcNA`B#~`3}`g(Zb|)+?^;^);>2r2bW*-=|V&CL%MUg_Lh5I zvXCKsFr>?;s>tv9j#iI!H5y5AWICmno30kCPbB{msV(lum4<9NP(r1TjDJAyU=wpR z;)Hpcvz#yQ>E$CO24WZQ1tLda*EgNDRUf!lCYA9K{n84K|CVZpgUPo$*j3x{HlIjs z6y~MT+Xk`EUn<|Y=BOpp%pS9Yb!wn1OgL0#kiUo?kQ{eXuG;EVTbcQwRXV4zQ*nt; z`x`Z`>-|-=<<8q{#VuU^a*I|I_1i<{ir(~A#cdQ0hRur(!tb43L>x+gd5qc26GWiW zP{+IBCgH9hG$h|bH~0Bb@E^0C$;KmdPTxBu!$MNz_UF^?bGY8YyB%wtj`)+s?DNY zC36We8AxpY)9b0>Nb=I+ZS{RNw)*hvbgu~jH&3y}33L5Bp66tZn{wD_$Hc9tTT|fbN z8&IQ@*YsQd`?!8%EaCE)O( ziDBethn-)ZW!8!ZBT)?ue(OQf*TUb1Q|Jr%j9~1`etCoq9i$5(Zm@Q>(FR2yy+2s>#=21Q#f z-Ziv_@KCQQhAa9XEt#(@nV(e3uX%NSPdVPo^w{cA9cyNDWS4LG6w0{1=6_PPQZa{O z6cyogfzw$PH%taIF27fU6iyry%X(}=CV>fmF~X)evtxPWdzvRF*0p%=2Xl=@tht|; z_yfy%~bzSTaVMNjPW9w)GDGR?uf!K2EYV zDZ?o*&!g=jHmY{>qeTNr0eC0ldMDIY9Odt|?yTJfw>z`z!Yj;)1hXwLQ(QAcI5lpr z@^S6G(MXn+U|p8}c;-I*+J$+fU2`9!&|LzY3gtd%)T13z_}&>RIJ4i4bY7=(gez}d z^WCYM;D~U~3sW$9qJCifqT(v6=3ZId(NlG<-8$-Jk^$OJAa)|KiD21Z0#Qs~dTzfM zm%cq42c6#CvR>iUUb_8srE+w(I-$OXx+5=t&=kD9F|??)-JxtDy^2Ufh>zBI*+{iK zph}#xShA!4({AUwTURjg_;S7hYm$Qd8Y@nE_>juj9BPm`xP87`K0Bnh8$&yz03>c* z2@7>khfI*q3tbCZ1p^{e>1f z>gc!0cKhUTGi_+}oZ0$I8|_2vPL1k!o3B4U&{4467jo;PqTs>W&zSH2U~YzGkfIKh zjn-IDtm)lrvOK|p#(@Z@*7Cx4&QmgPLRdqROq}Y^6#7?liU?0{!@KWLH=*f8^rtVi zZclH(mvYNSNkQz-STOxYGc-v>o$+ZB)-!$T8)Z8SpPIw3gD6-x_@La3He>b6 z?Uy88^~5e98ujQhQe4A0ihQy9*kHVt#&u+4GVOzpHn@+Q05S^p2*et^<#uWmQv=6+ zQ`$EsOSYNG&j80`3WJ-RqteNvgMcrY16js!2+)AmK#|lSrtm$fAH%iVxsEDY4IeaR zvj}T*VdjTcCT-ygd5R1V_=cNd2&6ZxC}jIogqmi6DGNt@p;(h;v@~@ko~#Qgw2CfKvzrGl~>> zB`O{*Xb}(R6ctNl7Qd#Ydj84=CPVoSr^9$Q60s(nP?2e7|L*9sf`qIWYu6px1Yn*X z2w+|3W!CE`aMa3RGe2-O4V!&I;L)mQZ5Zxq3h`~8cXL$f9aT;++Yxj!h#%_>Ryhl2 zE|TIwMJ-dj*cdPKHS;C@% z+s*4gZGV7Yh~%Yq?flUQdNaq5@j80PxWx#T!Zq=orw}>duQ}e%MdMY$Xk_MhS6%|t^bGY7stLF}kIyUkL~cuXEe(6xP;{+CyOyzos!!E&lrEA7TpkiO4Lt|rfBt!aI_)qIE63oKi~ zP++kDY1YZxHb2>qJpzUSN$ofZd#DuIJu<>Fb^u0qk2qR_KLZ7w`oJ6yzqRm zpv<$k5^BjKx1)SCW%h@R9no;5KXxK~aKU$0uLeIL-(gMTR!iG>_lo00(vY53cbKtz z4jO)DUu^p9JSuO`SbBKp)5!J!ufi}ce;{QsF{I9E{NVn9O)+A#`$c3Ftz=jI;1K)A z!Bw4k6y3_v>ox5&zp9#B=_PA{^TPvK?(&pwU5lT^w?ru_7`BU90>}!8O))Icb()6Z zMyxA1XgFx?G$G>}wJ6xs_1d6+I+Dc(LK|ovH>J_?agpITm_zG@Byx0AW$MQ)Uh%=% z!qr~Xg#lT^eb1ZVf8JMW(E42fdV1+ zGh}OW%}GRE2le6>n-+<}ocJZa^3Sp2_hrMq=AZg2$_Z{v^En9#FBYwFd%xQ#ty(S$ zBPHCNum|EG||BKSotQ4c4Md)t^Aif;8)?g)H4g z66V|^rd_aEi6hT$gGp&Lj84Y3(Q*IVg-@LgvW4D`1gdwZch?zjq z>OX9|jvmmZD69gRph1Eci;LjgGc8qmKT9nog{0&5-lxYA7Th!3jPxd!KP3?GgV4KI z|2yD+-lS#c0g0=hLOGAfBiKL!4FYA9l_H^6(8xF)4ks8>Y`4F4jDmsnhOBMk9#nUpAmlJ&zz zZX@5ZQK3=J>&}w!BTH?lkwmjt40a#o<1_=JkKvu;;cy&f_MZAft2so9OU?CY-?qiY zCj6Cfy3o#68iVyD0V}jKdu#!I7B7mJjBhpE+xb!ZS|G3dN2fF{ygZY~hrc(UHb7JB z6rcN7_8+cCj^F$l<*mY6Mgj%InF4Df_E=eT@G2TtbwtQN526j5O5&T>~{ z`VdAhLhp3NGTR?r)bwA-N)4$$WU!_tf z_PFA>UIJu>?bWa2+jin<(U zhPt+}gg;ZQl*kL%KD?+}k;GAsH_+b}WKE@)o(!;F9tn|CdnBT5DgFcQwo)=Q3MAQF z5ihuo=2cgiQg#7{PJeE1hF_5-!btE^4Up2Ok^K?WA8la9qQ%q`DVgUV}ppzW{0Dnn0ML~EO>|Ie&Z2wOvrmCy;(Gm z3Ca4*4lS+&#+(R(E8T#HC z{z*PqQepn8g?Lk1B&=Mxety3w7A+Y0x-X^zVu)LL(}H3f=*E-_@7KS?m$GxdtX~6) z|9;eUa)?A6Z>E%WNKK3jFS-y!>SdWXQJ496b`*!0&TX9{hyR|z7WXc|y`RbOhUDKNK`%`)>xi%uJQt$YIP4yb|0;5_pygjF z1rAM>^MBc5>?o|?BpbUSS)VN!6LPfS9c1~fH~9G7+2&HL)AfhyK8XeVmX6Z)S7KhL zXQ3@`ys&4cwY*eOpeG^f`nqh~MJ=6PzCNfc+q;2>8C^eVn8jfBHo9_>kp&|)^7Mms zIPPm&QljU!7Y!_z?#xARnWg&s{WI4)Dd3DpJl1{ zpsO|wSkv8Zqf~yTGT6cr9$Fag&;Q5YFb=*TkgMV>fBRL=M5+9pnuFCPQ(wYa#3qJg zaBT(KBG04(6MTuQ6z(?H$4Gn^CPUc<;H{9sg zkHM}2$x1LkTfIE?0rkCmJrwH>jy}d)htn4g=|5PoKn}=@u3YSr*8JWCjgXbYNIMXL z`c{zEU||?f(;;B_WuLOwZoT{|h+dS){Zz2H{@g7GWd-o(X*j*&;d=QOX>ew$Ag(E! zPDrx4&u=S@s|i8gqjCF3d%ReL$cH(nh^@-?R9rXR;xILXkTL^&p|FLA(;q(S?2;E^ zVH4)CPHZ@!>ry7Cep>gzxjVS#*SUAPAGKr`zO+eC);*fm)s(qPtpC}zrCO(Ue(mmP z!K(xiMGQOBdgtw_S4^n}I8y2l*Vqc7{^Y~#!ny{mXBmPnVFD|(ilK6X^?nO#=B!@# zUQaQs5XW7$jQx+c)1)l?XlQUvcIV9m0et+!LA#6n=(OcegtE|1f>L7t2jR<$$Wzev zZv-i6DqQKl5YCU#FC*vHv)uh*Q7Q-ayBxAruI6Xor|BCzf-;W(<%{f4Tr)9t=`WNN z5p?8WqBR943ip4Zimw;#{^AX0YQ7VbV`T_MN=r4TyVxe+Gr{j|8l*+;;vF7&#dKH| z%yV#n2e4HglR;2so1=`mAjyE*$CK3P(8)0Ky;iMWHQL{@dE#AuUj z7DXpG?&W`phozy%U;WfR<^g9=0Lr;EKYVG$k$RS1P z1vvwPwqX&Rvh-9rr%<3#G?X zWfh*5|K@6UIhpWq#J;+!7Tdev_f*Y1FWCZmYpNl)_*U@Sr8hq85KK{k}uF3JJ=)H!Iv_7kX`jvQHbp(I?w#~5dEeufk#`;)wgnMp+A8o#rqP_Yk?(E zN;s;z_0)_t(~k9;!iRMCwy|-?r3?AKXxx1MxJ6_dqiMh1CA{qU3yFJrB@IZI9ae3Z`D(|6sXJ z8M!5zz~_Cf&)t`3I88H7nJd}AX1V}E7w1+Y@9V#HA8&yq(^tue9Q`s$v7VB*q#@M4 zU^5OfWJ#{c5V-yiAoYLM;Tow91=A^}55<3`+k*f8-2dPdNf}0LTbFJ*1(n0~wBW4~-ccXFSX^{~r$o zUn5VrLC<94&PVD%mOwHvsH9&C`Zedj>Zrs)*=GGglx{FkiW`D}xA0K{&96zRVl{4U>%N5ha z=9{G2bf{aLrwiCdsKxqENr_#H(D<*We$pY&xeToz7W<{T|L3l;k+%Lr@zLzjzrr%3 zAya9gPD2E@Ip_F9SO&rR436{oa}EW4%6u{tOf#5a{E~M0|hxrB^N|$|DWrpl4(kswU3h8#Ryiii$CVOqsHw`|3-W-hurEN4A->3$>r0$hIx^5mH4i4-DzXw82Ml zseacnOY(psq$;%(l303G<3U9QyYtP&m`vN)4e4BgKMg*lMZQDwJABB9%8gbHeuvqP z9z0Gg!)y+Se<>R1oqv!&0+WX<^5tVTi*MW%=zW}>!zmBJIe!zowOQPfWrBiyxFOIr zHn;a=g|IINFcxhUAxSK64<0;dD$mzQQGR&1T2cKL+!uYYeJxsv2o(|)>e)rk7Ka2!~xi%nY|$Udtc0Juc_X2 zH3xK4m_7OTUTmG%J!D6+9fn`ghG|rFWImFfP2%~-lG0NBYaJba5v-@vM~|;=6S_QH zNTSc!tRFx>Bx>*S*;u zh9XC1ety7RiLAbhpakw`p$)*q3Y9fMn}@$fiaZmrohTEBqlIsvgx$YIFi_ReFTy}M z40&HjWG5wA4K?$vCr=p^4+}I>6n=8bXUAI`6xx1H|ss1-o7ih&FWILdz)vJ4%UK&T9UGg9i@@6e9&p75vYS_eKLFR42 zGxRB|g9xcV?W9b5qD&BjK(uR?} z3oMx9a`ZAJ8Y5vvip)2S{;U6WDHr46xmkr0y0uQ<3V6|r{s^c^NrjxwZh2DKhOd|@ zviUTsQSeMkio~2upKZl3nz_-sgWq!!$}CK1V>HHD|Dk33kHyIsrgwLqf0?)wVNSBX zK|!)o={h#qvd?q_Us^P^%V%U`Q}Ezcu~;T8IjIM}0GK_P|qn4Q*EwCDrnV zDskf9AbkSvnEn)pcc!p-t(6>87g~Mc?=e_d^lZP_2y47Sh7Kt(00t95N!0@?mz?gA zAcgw>dDOLD>)yRl7oB@@^X(m;_f`G*OOC^n_0;r7u%49CqbQD#Y`3odR#L%b#>pKw z)=C`rSBa*V#$V6ete5_~CFEW#cxX$p3 z`e%s_&j2P>zySw+1X$l`lBD?w5bg*Zk44{WlK*u(ONtmW7J5lwnQdY>cm<%&PT8(_ zYm}|rI!)`vGi^t@dMPm#Ch>WCrHTupXfrIAY1T3Uk6AIE*pC4q4Kt5HBWo6~m&l22 zpQOS1m2>91ckK_}YWe<%5j46mMrh!1(Bgr`ntB+0JZe={c+@(6uwJSY|939>9ywBB z-zKSr-L+kWc)U|PaaqWW9kZ7kQSZ3Wn@)_*_iZ9#eH?uTzoTHD!9D}elnni%NW9#c5c7# za<9Zzk590A8^sHWJYhNult||>m+aYGe6sp1&$a#x^DwYTT>+X1Gx>}w@0}R^h_C&5TUJ)T55LV> z#o6TpBt}OwqL8Xd?23y>jie)xCk9`E9i zY-+DJ$dg2%_tQ}mqdAj2*LEG9w93lL4B8&9VBv*1V0)^sP~eArol6_dqr+QpsO>qhOlMYv8fL zNNyRHwXgb8q5`54J$XjxP2SG9%nS|qa!&)N=ltXpz)9b3q2Aivz^%OHM->2E$B^?{ z{YvRrC%ybu_wZ#XGc-Zor?il3O2BL*w48QlI!q`Mu?hexJ)oU~i4%pG??cp%o07v8 z6jd|t)uzojz+yjZmtjI-KD!&;=>5g)6#xe*{rQ1^4B)c~<+WV6!_AZvd#MTE?-49w z&1U<(vCZ;#^z%#}KE_XJByiEsuX8aTf(#we+q7ok2wa3zBmpdY)~W8@9|C^i7Bgc& zheElEUGrKqY)1>Vege>;?Q+K(5hf1+ZmB}=HVH-ESD8r`TlqbH0i#m4_QLPOS~)J# zc}xYZbnHc#4?@gw7|;<|U4r@VHgHXwECR}d?<-{!3NebrVW>ybC4)x! z9}6Lf+AQT@uTH0<499l&#(cBC7QIUTG~zEnIlevy7mgNcd6yUxNq}&0wj5O^17cn| z`1A6Y51OLk4*V&;E={1;py;4m3Y+m=!fGuQN}BcZF9;T(`&Z-rU%2>_(&wC`rs()0d}fZIvX% z2<{_uc+~c3T_(Spk73Ew2Z!2TTkF__!Q#2 zl<$9JC!Vp<97gxT>m4H;N(4k3Jx;o+F#%US6TMsa@m5Lfuvnxk<_m&@)R4ru6#TUiVSiV8*(#rEd1;5t!ah4eyUA3By5p- zXLr5%K?Q)MH%i%nJN@?iQtl&bhN@>JO4n)*nyDe2KILn;tk#eY|05pF6gc07b*rc9 z$DcC8@ioPGy3cGU(%`47;}t#1jL>VDJ62B{CslpQMLnK^TP9A9cWl~mPrS0>-ohY4 zL(n9mhrwZ^VCMYgaDTWE;aXH*i*WTSVAUJcsy&3|uj>B)PWphgz6A&GWbJ~HX~UZR zEDxprc#2>O+J$rN9pCB{%jVsmi zVeV-%H2@@26i$;#!m=BmrZg@w@NquM_xZJC55(5D3>7$}A~vt~*!uvmDLeiorjwCUW=qXn&}h<|#{OA2Eux5)bxnmfRj|H!ev$K4!$ukp{e$28|I<&J-SJtO zE-%eP`I?bui7Hx3`iHx3YiZ+QKcqP(y;`ctL#V6C&$fre83;?&lcfsoCY>1(7p5YD zudFAcf>Z(FT31u7RY9G$B(}75LN~a&IfYk>*pImt;&(!#QKKanFG`H0;`jA#Eb^* zriKqvrj*rZy}B<57kxfeU-$}t&*XTqp%8QZ(RioJ5>KXDFa0Q+Nr zZJg?Pzy&@TuQT|Z-O#o3%@_t+KkhGN(3@m3F_z|jd-zKYyx3@8xbDy6Yhcjt5kQHuUGkS(-#l#{ z#>Fq}`^TM>lSAu6bjR@oz|y-PI&@uDmcCiw$R)HJzj+AAk|H^M+daR%cEi@mU8gfs zt>C)(V6iE5Cp(hP(D4El?s>aFW z-q=;FeT6C)$_HHH`xbCaa{JQ46wGjXihA7+I;WbaF{qvp%xx|PNQiSMok?p{#w}a& zU>U6giVrSrgtF%!eb%KMXL_d0UR6Kd3{ZcutF~$DzY2?sso)Iv2am2lf4cCUy4M~O z&U{5tFqEAAaQ`cRmvYJ^_NB;&!wnH({c3QgW-@C;QE`*l2k+h+L0>A~|L$}(FT_bY zJlQJ7U6mC`HhkOQ`)Z;ir?ozitYMsAVdE?m|1JdR{uP}V`S!C5Yo4(i!*fiBoO8H9 z!q!ej3THSi{l$JU=IQhI=vk)vt@N4pADhVe3qm-)i99&f_2<}~u1jh%1>X9J z%mfBAE&$reaJ7flok=WX-w$J{(}p&?#{oSeNrdKYvWj_c5U$@?o%=&6L?kSP)#usz zAgwYBgSH%=25^%?X<_mV#+-0?K1n3E5izc#%5jPA`3vbrnz?47@DK3e0k-O7#do%I zurEgmy*0hmZj0y;Usjt2`0;t{I3wK9AD~Jipe=Df?_3q^&L?Uvq}n+j}dv;)H>E@Eh5ueMJ4-qU7LSb0?w+OkO{RK z;4`QQG zBfmr=m`Sr>{UnZty+ zknPYHQs?MK?s{2{YGel)eRny#Gcc6XP@@FURJFi2h~rZ>$Lan$_KX$!`1yP6R07l-8w2NY3K4wO;}QL`%vYl(9V&)ObtUPDgiZEvYe z-Z`FP@E`Wu&NhEcKR)btu#y&%yn{zr%rwr2M7x@krmyU@VNf|Yjp1zwsD zk@BC1t}*x-T0DRu6<_Wsv-MV=`9kN&HzFO^7b5bd7n$-u zu&MD+*4{(<6CgdbYZst7%$VDeqh3k|1e6;vh1%@et#qL#`P%p)K!{dbf>4;+q%|K2 za*m4DSkCOkE-cHG51IKiDa6$Zcgd_2_f0D-w5JRka0GD|a;g8Qc~#v%1d%=iaVlru63vJ@=BX@Bg5%N_8+ZlY}7O@DsH=osd7o{B+N zu1A=*DCrUf3;zpCZ8?$-(@lJa8sa`W+AGo(R9EbN3tSa94J#m)6NXhdM1|;>4jk?ghtHLrn|= zh_lT^jPtEE%Ig_B|H@Q-LsCHfg*0x(3%fc_0AYQB~&?|`jtsbc_KC`Hz*HH95w86L{wAuG%O035yW=1 z|6+a&ea;rYqSng7N8ZFrZ*&`Ra^4J6_8z~gSZd2(gR$?*ZGTbIGV8wr(5W+pl7-Ol z2$tK|6z;*LEx!-~=;b!y^OnH2G^mZhS||-K*RuGZ2`W5m4q@Mif7)*t z$rll7!9+^Ag(rt7FhI8|JbAoXxMX`YMls!QGkF<&%L)37IIB@buSG=Cj1l=EAkA+s z>`6g#cof6bu?*u$zW{x_)$ONXfuy5-o9t5n=wC6!ukHjg#nZZ3d zQ8N78=}n@9oqkh~{dMZ$-3ya0)^(MmJ(n1iXpMY(C$(>EiZkd<2sFUZ zo%s0{^m)_Z5OFg&r!Xap7`^34khfwS6#X@-CEY8{)!&8U)b zF)VDyiUccC=_+ccf6a}(@mI?CFCuWBPg_!Pa;M~kUML~08$t)JkGuV)evG-}H9>}~ zh;o){59q64Oe?WC`+>K_KO8mf)xd!iZsif(OTUimoTG}UwW5*~rO~-9A2Yq%X5Qr+ zKn65qIkdi|eU-bFc8f63{ED`IJ7ZPFXC?9a->MLy0^lZ6Y#pI7upNt~}m@--4$N?0*{ zxW)EWG5ooN;0FPLu5G$l6CFhrSa^O;HL6gmS8`}=@40ODfR|6xzl?(Pp#XCJ0iAtG zUfJNi24CVLy7N-6}ex1pmfUBDw z1F1-l(`N!-ps?`A2ImgAxjX+>)Tu>YB2N&|*S^j}LMR*lw0J-ppaPk1@P53gPAcd> zkKlI!vvL$c#+@2!0o3>Pa!KEPQaHuIP@~cJ5b|n&{&qd-dy8Z~fJzsHn9fbkzdY@o zH>Y?*_1CN`!8TtX{pY^T-zNxfw>kM^Ewa2)MT*&)B2V$+hLSNTm)`r`?ZT|T#fC=84NQttnlNWow z=RI;QUkne9UY?Dtxt{A7{Loa*IGaZ;!7zNuS`gUA8W^QqSIdNGR6a5_7y!)&ePuNe z42-__S(>@Xex%57sIO1=dfMIp(WZ7=U&wdgJ%Y_Ag75dpiumR;*{iEBmX?=`VSy7Q zDvi*V_gHy(6J%0jRk0mR7$oWpa?GekuwIg3*lsM3m$nrIM%7M{%*@GxkF9g96yV*n z{SAfk+MM*R{++C%;H?{<1@v;r6tWwJs%+_0b$9{&dlZw!7R%p> zpf6>K(Y3_g!!xD6ohFIh*k>zgw{y^FQ>r@mt%)#*G3B5Kx!_(08WciEV3Pt}4$rgs zB?{}y)G8B!HaC_%Yp}%&f>T5P)3#1+X@W9t2tZ?7GBlH)K#jZ1 za-dj2GyAlbNU?^q1y=>P>3Tk9>E?K*u>ji+UvOBiUZ=1?*k1)2BJ4u5&R$P(UW)+0 z_OxSlA&nqIaqn8CiH3Tv_aAro_3TJ{=lfEIyo|ZjjBYOi9>Q$a;Vl|*^TXhrZ7kn4 zxKFQ{sw{(-DOMpr6Y*r$7HR4a5ap<$K2)9i7u(oP0DF$!bo*CMK&LUYM zRqJ^BTo*0|k^SDnj0L?&w|KMs&B6+HyxIMPNp~|8F*f7ii^0hffGH~s;k_L3_^j%o z^dgwck!^^=&laVsf zYTFlo4t!IME#pfjPC3YXjJVlbS*N}NXBpN`@i?znQ6*$s%qFVC!9L=w&#xJ$w zO|`Ij*Vl?vWL>=0x-jTld%#X}TQAZcb6qbjAK<+Rt0MSUTgMlpE99K>1c2T1-eWVe zo5xvQ@zEs@VSR;P7OEW*@(m&-U+JO5d0;H(z8(G?V>Y!Om0M}RVBFvpEv>QXrUtB4 zIW1m3vD^D{ns{{Q*o?E;nwQmG2}qXx%{hE=en#dK4u`XlzF41q#}c1p;)z>W&BrCR z7B!90YZ{sE*qEYf#7))8C)fMd90=Y*LZP}N&uvFDpdLoZEUi9UlnX;%11xOOL=qXb zeYI8{y9;+hsCeaLfL(*cN^w)Ym{MmWt6E6@PAR7kNdWa_;Ym?{>`&%sAEWUTr2O#@ zGk>6~MCp7eO3#$GbL>f518jG1${Rfye`wVI18yXku~ zB_`cLK6%Io6*oDAC)g&n0jh>bdkOi0#cHnDjp~-ISRxMJ2*aDHCPfZm(1MOYlNlgO zdnT$+P?@GFu|sWDCOS!_xiJ|Ivd?0-OfffR?=7kJ% z)7H;b0E*1ZzJ5Ov0NupH_Pa2>e)22M5+g;01AWcgbIzlKOQHJ@@zq$oJduz(43qPS z7f;DRV{y;jw_87!2nncK8<{>qW%NmURUmtCPqtw$z~=F_mC$LuQdIGhlM(U;V3i6& z`Hxi*oF=NQA7*v8CO%<4xAb`>5vSZ$2)0I}87P9DBX)2zOFe6L2n?PnYgJOL7ZwW4 z2vcB-?88ml(IxpjeFLPr(Ls@mTg*T@-d5J(L+k-qKJT&lwOBx*0(77zZ0TVYJz#MJ zz~VLuPu%+r&JApNRRN+NujQgfv)GKVwqD6Bdn2!Je6bt8H$P;c6(TkPok|f54~Pk2 zE-*<6NArtAeX|P&`YNnchWZ|iU$d>XDVFs+{_VPf0RL%1e`);I(3yN5a+QJ>#!)lu5FlRXb=$uDN#^b zL^`EJQo0$WyE_LE3`&tsrMri21!3rhp}U9fc(2jtzMuF0-G1Nk{zDuh?CaWVueJ7C z=XtK}aT|A;B|vb3f*oyi+iY~RyF>H$e~cw~wesI-4}b;;U=32=&s9M#4(nyQBYJvM zGIpE3?f*X8pY#XF=al^#mIT73$T^U4QUdw2;MQt4Ve81yWvk5JPxl}H2`owc~?RxUj@7zUb~h`tsal5J(?-_bSO(Z+*XS zM=u6$ob-FYmRQbtds$7&w~uKd{|TvBIR%gesi(J+>}s&W}y}d?R19piwHaF+tXqg`M<7*$aEb-B9q=6f%5CQy9YpR z-vLPD8MZ(p5DK*N01*;*Zd`W5rs-V(!~B}YlQE=6I9n;rPBlj{+|~MRIo}9xCjq2? z-JCknSY&BR{df8&A&Kme%QaVF^Sq_;kV@0j^*8CB@~Yi>h=f3eRl7Q`_LM34ERwn% zk7{;?a&cMi@c`06wM`Eb)C?^DGlhbDRtT88qS)m;xduoGrJ4yy~lWD6kOq z2obnrC$%k5z%NU2n|ex#{h`ayb0ucb<9%w?v%(b3DxTS=`q2PJa!M@Skryg{HMZIs zVl71L-!65ZNZ#KAZDxhD+cVu}r90-CoUTZdUR<@gW%@MjkFSUGS;9`4jrIpe3h4wJ zNRAjJS#$>PSl!C>3Zs)N@-vgx%#~(sQt33&hii2q)EZs6Xd`a21c{<=2QMtg)G;NOjQRJv>oX^Hw>ooPh~iuguS3-X*?1MTC^rW}Q& zd#>LV_+#rc z=6_Th09#TD;72+9io|yG2_4EKY1oT*!0w7IxiX3T1V%tpv;yEaG&s#v=A_pCnzBME z)U!#ou(3+8R{g^gp(D2N!EwM`YSBmatTB>oUq3xA@t+|SIh$X{Y6dv)lK@xU5&(xs zX0#HuQ-#!F(|EM>32YUf5&snSOR|NQEky{jv-hr`et^*=QNKJ7Wl6?cT1pq|)Pmznk$SGiVp9`+12fdx^2MR>n!K8|%Nb0VLL%+zS4czy1 z#x|7NtrTGJa8jZn+0C82UK9+L-T=ephnI-~Kq&6q-y2wM^#_{EQ%0fGX1|3C)&%E1 z*q=55^&-i3x^YZ&%zxPRDx-2g!p-C?d=5Yl%mI{2+vyrdw~LLOPQ1etpa^b+vjLU^e!t4dg^@8lVgX8jodwu*og&hQ1sV2jwC}SRo_}mSOhHtx0Wf+u zWTT!;Q_l!^3RCVM`c7Ss2TMDt%1M=ko;68%FHy}|AO$Ua5#Dp3I@$rsfm!(Pso4XqLGm^1cF3&%&1`X~ZiLcxbzaQ3Z6<0+1ZX2KY-=6yaZm9ylktE!I z`!=+b-uP`Hzzr4mVF`sCxR!XQAdxMD>!Htx?Zxr^YA#8C-2x4|3M6KPlJ*(Yr1EGrxw0hLdx58!-4oJ0+7|FF{=ZG=^5H@IvllEXID82&U zN^CB_Hmd)F)z6@*=C6MMQuu<9Y%R2~&RM|99?X4gyqrNmiG1eK_kXb9y8)e?pK7sX6`yqzgZ2?xLMn|y>7lGpUV~}P-+1hfg)a5i;bb_FH%@3RVlUzg~8Q+>fQ?&{4&lRIhwrfGNSH@qikj z>I`mxThus@-rersQpBSh1=|=Qj+^*j6TCkqP*eKRr;dK6aIy0dzJ^N;u5 zdiWdJZIxv>NA|%AhbAgmv*{paup+{=3aEJEb?82$dAuY0#AAc;uT`Jp(Tr z>lNU;kX_#IXnM0VI4=_ucgkvLHe+<|i0EeD+StWOv)f7(?XO$yttx&Osp}q3d+hRd z@A=k*g|V*vX=9;r(0y@{o=+ObWjeR$J!c+R6MFxXN?nK}59G=@Z)u4=&0C=}&Qub< zo_2JiXPbaWxYYX5N=30{0EpXAg$C8g359SA-2kDKW@SRop)D9^@I5(gaIu4WvIw7S z#c2lsJi8YjBINKiLq7r1*ZET-z6r#tziO-hMl1wd7**o&h2@s1l&u=$f8P2z%Ry&C zfkk0jWj_FIwZ}ci)BLD1*!aOcm=1hkYx-4cpWOC@w_CWIwQn1i_}DCCa4r-Y&D)UixYvv@vlv4%>u%3ro=CRXuO&*5hp2)8#{v&nx1v zB)+&0ADI0uf`oRjgMl1n$>phSNn|2-N!>pk z%#Z0z7Q1_A!aO`4_mL>=nAZITpQ(`YK1}|@2tl8VF_?o4N(^^Nx zW0Z@0=wI(LTVj0Y5#-2I=bK#hoWsht;skhw;2ZA=0h_KE-ih*b_wB->?#??d$Gl$^ zAIq^^z=M|m@J*7DG?kA<#a|p|Tz^swd9)}b-c1#tp9b(J`*#gy-y8jEUqS@YUZo>( z#HqwR?fXb*i4gTC32uwX2lgYoK&vhnU7rFk?J#+5A`laAvHgy(P}DQVaw1weAJ3>e z9`Rm@K8cKJ*hlk@ul5}6D+i_b_r9o|gizVNrwI)E>B?pB&v^A&w#9#(JUmrkm9fO* zCyNcBHxTKJg748o9Utnm>zwvU_x`g;BPr5QLK*;G=mx-II&1NyU#?2`Kz#berY?LQ zzd#$(w1(%Fnw;X)-{P6WGt7O)>e|z%-K4{?X?Jm#auUKP-7<2$<)7Ff3hFVR)-Yt0;P5m1b}gi2*q)tn71~(oU#x*y#B#|6je(YN>Tn|2 zsiz4R@9{O7I!l`S%&?c{mQ$dmvihHFTy*QhEv>`TBES>YcpXh+yQR%uN)BOKkk1_P zGo^$lb$O*q1yoB<^e5rQ?%}DH752U@9SDIj9^vEmF#*Rf$HBtrOQ;Vw#3)~thy+T0 zCSrW5^81n{nf?r|K(_|#d%-~wP#c%SBa)h)1~$CYq7Y!+!xlw6w>?|)>XMUxQk4TD zH9TH>MrxNwIij|}gN%QNAbOtnR{~kXU&^RDIX|M`jLQCRy*d_e`^Qi0MHGiEy;!c7BjNFH;w2g2G%@kysv_0d5ol^lKN}UfPJwg@lH6q-n`@I z&i9?!HOT|@QWG+f?E)Sd9TJX6;F2_Nn2SxSH=x7yvt8V*H<9dg9PHne5h)_DTMIA* zP9^PM(tH;Juk|DtM+zOha{1ZGK!l#^efmtk<7KKaKfnu5RZ6Y44_gE5%6l7KNs4AV zdGizMUZQpJvj%+68skhM0A+0psHKG4dM2HADfw~jA{MZIGQav~#d#nYBbBEqhKN0e zz8B%ILG+BaXKoW=26JgiL)KI zQXFp^tH`S*k)*mAlcb?THO>*?;lS0$G@3_+ua$3pbQ{zUk5xZD6)_`iFQ3b^7yUWK zGdCpA{_XKj>!;6-IpZ39#d?n7O~BnUTMVK?_9q`ydpvNbsYR~YAn$;{Iu2U`zPJ}f z>E0D-GxY(%19xVrNJ5mIuccVaMF@;BVzp1lmyT2K?of;lWfia2h;HwM>I!-zKKIk! zJCdS`U3{z7J|P3X6$!bgc2-)?FL~YYt1)VGq>HwwnJ`I@Z!eEcf!|!-`GJh z_b)Yut6$r>IVd;Oy9;kM-r{WW?c7b;+gvI%ZtUU_IO1jOS%`oBY;8fH#^5%y&MIrE zD)gCqNbPL7dJet))w}05i{pM^X!~+0*z=pP!)}&Y{kAEobhWMwXVgV<;k_jt--6#a zwZwM;{-;X=9iHS3XAAYNx$P)IDAuCVSO+WeR5T!pn^*O3BCzROU00o!&ujMCW0oN6 zA%ZUHX${zMuErT_c}zRL-l-Fp^a^`C28y%cAqK$3)mg;g9_jy}RsP*cDjg#Qi_ANd zvX8A1|JZl{LZ>Hf|9&oui<6M5o)!bsoH$$CwvL{4{mypPc7F0J;H$B3UvmqY@4rGvsl54ONH;6~WP(uh$iRN)}mvlo|wC znhJY`cJqLuuXX%XjUt-_wS|6=puGEPY|Cl# zjFX+B8c@k>DEdcR3y>}qoAhac)q1UpnmQ*v6!_(aZ9LQcW`}2CrE_9^!;DCTXFS#B zK6#jpuR(y+&c|-()_PSDJXY&+b!)P`-)Hh3wPU96ac{{~x&O2UTMCQKNIao64o!d^ zhL0DIdZvQupUkRPIz10R;A!%aczo>IhS&1u=dNQyZqa040+%o=rvJ|pq z0Pez1;oE|&xAJpxVcP?w(0BlvtU{6D~lbAwZPB(#20HKWa^u%_q*u_?j-?u zCKLuN$F=@29G~79>EnMigE=2y>^!cTYX)p zasR?BHVQO;wP!ZqIjW9%x!}X5ix?&B;gXO$$PPK=vlN&w?I}PvLWehnhKLGC-?Iq^ zIH!Ngl>hp;`8djJb`);LxuVjZMs>@H0)2-rIpyVD=sKn`cG;?0=zM}(-`|$k)cU-n zCpCm^cqgIuSt$o?*r&!Bp2jsZl%-);MICY9;u zJDQePUE9F|_~G&Z zG5$N>kme?J13N%k?1QZSp)Gt@#`ZM4V-i|#XJ@OFAbiytImzZN1m6|94&u3uY_3Rs z{Hd@ggJG9+0uI|~UAe@tV!q$P46Lzm7(+T4dtSh|+GYQr%PYkKaXS7$;bZo=HM-gY zg;YmG(FnUHy3+dkaIXuO7Q$nva!13w@m;k`Vb|{OCU@k#w+&%%+hT?C;!+PjUW|vA zl}4PIf$no>vt&Hh=RwfUun*8B_UE=kW5e5J&dpZU2>+;=o8s!pWI2C3mNv^Dwi!A_ zRmS+t7u7-&Y>{PIzsWf*Q*r?a!~z#ugcXJH3as|a6SXePeaBPjNW6E*BJIB&-0v?G zr0LQ!&yzcr7HHU)jLu7_Q1;RE?h~Rf3|F-)jvb~}!w>p>M|a_F=ouZ=zOg<&OiXZ#bR9*vHKV*duBt?u_viT zBUGK!zG}Fqs!J?FdJ1pNZr+&Tv7IBIs9}TR(?F=9D`f* z_@yTA)=PF?T*9^E8b{U?V=1_Hy!$bh@!S`3>bJJlH^8C}$&>1TO2U7|1}_}<{?vhY zXu-B0Ll2^SZ8ro7%N%bn>e;T3f4U*sS#0{uNNw87sG;JBw^XE8XpIDQmOB;vbNUWy zr4tH7u%H|)qPCp2u2yGnxXsNRqSU*@`%~|XD?`#nBB^Z$QCM+E13Vh-oWxcZ&Uu`)Z%X>i~?T%2cXIDjbpkS5`u`4{L(C8qh(5pp- zPWV*>kcOTh)!>}l3?Rd^x9n86ZRovbww5eY(v#mgJMxSgp3QHN+du#+Q>H7M78n+2 z{1EbWE1hDdryU-iIp)wl;T#k>)zBW z(0o?atpXN8F#EZS|FjUol5y7>cN3gk(zUZTPV-M!xahZqE)L=CvqqPn(M*Smul{kvK(=-`L7a2ljR4kx(9fr}-EMzDMhCHTTj{BJJxVj>_R-c( zDfm7Uo_o-a?X(BnWOyExLc{iWQI-N%)2^Y+HAEOgbGITb{ARotx_E%M^G%R^BEyK1 z*Vc;@1Ik)){GGnke)(sJ*tRDdPqz2xUi`HC^w&T7sNmDZ2HuO+{!p@o=c_+2C$5TD zc)W}&%Je@Ku@wd<3Ha7k1&;9NPpJpnDXdrRTt2U&VO100L2Atmn=(84duo}>67B_88-A0`VZY>? zbzo{4SHz^~diOYCEH10yczBNI7wM3{M3u|e(VP`2W7hnV9M+iq%1)_k<=AoW*`;zf zDO-IWTe-w2CujaeH6AvlVL(hkNAqhjqYO1Ytr7r|V9;fW#HWwjVkx@jwU1jXuzY1& zkTv+(paSNlHzH#Yw`QrTrjg0Q0fVoc>!mipJOB`cY3d1I42zr~)s&hd7GY>$WoV3M zMvh|jN{M$x6(;&Tx)OV(Y;@qM?3wvp0(YZ4Tkj}q97hLB{L91RAMFAfSncp`1@p;+ z`$<|`6(`Wa>Z_T$;;V$-+MQ=fq2KJ5@}Q1e2M%s~i_uDz+rzHg*B+gfo7 ziPT=3cvret_SPSG)mOLM^o8N@WH{F<7?PZX6G{Jq4!A4TM8vSvXOMipMyW0SDx*y) zVyWL6ktQ-%7F8^xZs3Ntx45HgI6c$i@2^>KN)oLfJWiA`>} zvLMENrZzcl{NtgUXOqsQ8kAMxvXZ@bZN<{i6IkGJ=FX<h9IFw}{mj z4`dfqJolS8FR_kRu$>GG52D|j9P{feGSf6PbGE-K=_QhL&*^z<>5x+tNk5=~#6_bLre}phbrSecpx#DtjjO(2V0+;wi)|YbaBg546Y=HWu+o zblloGFb+o`&iOuY8)PkhtO@YFvn(%iV@@5HoeT_E`hat6wWB&&&bCUH~HlEJ#CL8Bc7$R>WRw66KL5tL!!o3vHQny6lZ#hf5@sW zn;7|&D5~Vcqe;iU37s5{w;DCC%Kb?g3)r! zb7XMrxufvb`Iz=txh-egJJrBW40Nz}u5lilGk*gX|CpULq176)Q5VJDM5?luh4eqJ|L2b3E557f=PLwv6T7wo6U z;FIg%Gb(DK8w-i|uz;Y}NG{%++GoTJVDntv58@8p+*?!LhFiF>PjDi>A5$fljlzyU z(`<|eo>`7Pf?K%oacA;A*e$aY z_w@B)#RUb{+IYoeKg*eMhcLdr6_2_j@GbjGs(hN~?IFK?AFP)Hq2T-ckc;1a_0+F< zZs*h+C_~OV_ct~xbu|q4M{Qu!6q?^V*Z^ZdeCMdgxI%F-&>&)T#81M-n$cQ@TJMmr zEq-`ZD%xsvhl>Us`7fY9Kk)BQ-N7h_oEUE{P~Mim6ssuMEwybL#b%2UAp8F9Tr+3w z)^U<&zKPGHgODDaPf#=LoMr~D#wDwd5Kt8OaQJmi4j2{Em3>=3gaH|j?TL!5V@@d0 z4LYuPnD`FDjiw%8xrSjEoj|Q?g2<{cwH-Clt}qH0a%#n1kv&iE@w`1BItYyW71kaR zr88>wL)q^7!%l06^wsY*ma?ERkfU_Ggcaw8DE#-r(Ut-N@K9?RK$20R(rGxqxJ4)S zv-GE}t@C&ATF&l8euRDMQV0?8QT`o6KgDOlraRL+SUDpCX^3yu0lj&+TcF zhM9F(b>w;Who8=Wc^*g$fb)OHNY}OA01P#1WbToWD3+c(AHeS5#vr-LQ!drn+3%+g zN{=$X8mD_T^2wua)W6XnC7*gC@05KP%2s3785hK)5X*k8{A#P|Qf|kQseo<|Yw{$+ zusi4?YhK-1GBHQ3tD(-yvWJCv*GZagqn>ks=WLU^LR?g$T(k4FWpP1VcqtK~_l*}jN9MAJ{A=CQs!DNBy`2@oek>Ju!uq+g4Mx8WZ9gOyGj1uQcbtqq;m!y)myub>ptm6{R=?csklsk-m(w9KV1AL4 zLVsL+tqq;?2p6g}Dk|LERWnM(MR&d6%TprGmA!k=`@G}urKd+huWQ$IggTduGjt{W zetEWCY+=);nWtDu=ZFl<|0=0RReV0oG<1s}a$ft?kWiYKGEfZtnIEuzFp(7aY^ctl z&t5rUE>Zo9_q?Al)pv0#N~%m9%>b9DT@>3iA1&_+{@2+;gE&gH1P4HMnV!@SePhlvP8cYTb==@T}*YRD_GMmZcyW8wwjf!0U)uHxylg7q~=uvQz)Rej6= zH-q#kck7zn5ZNKGv*($w`?swd7G;K@UNJ)7bI^)+H`Q&$$fh@XT*`6LR3T}i(dzzg z;&e+JBdvDPFLrzOhIFx^cLicB{PFa6I!cw2ypa2q7pu%jlJA z9sd`ff(F5rK!IqN!74t0izZffG5Nu1m9VdDX z4V&I<&9XWx4mzWeb3C(^!7K1A^SD^8y=U~tJ8}ZR&Oj`1fz8fFOtu_*?Btri1KLXx zDcpbf_uuCan0UjyrHZO&z(8!2#p z=a~2_*|WmL&$V7&Q#}0>iYyG%|Dq1e(4wE+1p+jCx_#|qTxY$~7<{*3c^0s!VG6PG`m3^}U4KA*w?C)ZXoc~?VR!^_oFhjQ*7Yx?+vWZY-63{5oC*srFM9!#LJc{RGc5rrve!9 zm$la-X7aB)O<%S|xRKqszKX-WEp0FH{amBenyEu3f5z)K0~H*d_>m){PWIg)RUO_x3X!aMKTCC>G{=g z75Cdl760oP71!flMo=r8%*IV_1^)F-E44ARZct+DZ3VNNqXe_>(Of zKY3`?N} zFan?|8S)R%D^9En@kll>|F2m3@6)12gT~J7ES`>sN^)`Tb4PGawj%{kg5v5poj(^b zv&oS|32`NwW;o{6V^G(e22=VE-&2alJXU80O=lZ2^d%Lhm-Kt(Q28CSz4Tp77JEYv zzGl^K^h>ePp3HtJbxqtQcq^`BaKW75`s1^TpsyKygC*w4N(J~n(eG?hjT3no1e~SS zsak2t`};~$@YfV zj;I#{QpG~^HAGa8MZLME8^cpUE*lnDXO)l<2(Z!_qK+=9w0DUy7ArIb3YcIQ7n9{W z{ZFUQ<^=?q99H=`VpqwV)K-%{^0}csof!$-w|3;nfC0i_CPCUwp9cwAT_TUW`Hz}Q zGSZXRg^Ryb|6=k>kpXxxpPOGZYf*V912ZHj7t_&#ub=2K;*XThcpcl!F}o6pK04+# zDUFomm)Z&P`6nJTC^K0``pEQ50f@Fw!(mQm`Fe}pGg7A zU8E<#3w8P#aj*RmGGj{TBBoGAdBtP&){lPg5Vpgd@#OQTzn-`mOw`rrEQ|n=wWFn6 z;**_B%v{8cQMX+s+TgvBGp{Ll=1G}M=8Y2X>t1(18y(+h@)0@b;k%ZDC0M*uWAhDA zTE1f@&=22G4e(8#Ixl&NJ&+%;(0(pbT50e1y`;)Km*R+TMgh97NNXLspb3v{WTiddrNG+!YySGJqu`1?iPP_7}=--nfJ)eL}G!FQ0y zjPnG9{{-ccTwZ#Ad>mKth~0Jt^;4|Bc>{Sh44^ozwsGcTYk%~wy9cj4s=7wg|AgnSdpi?Wy7k=NE8ePTT>L=PWZG5OLV zy0(Lei;YWW8&)UQ;$w;eP0;A2kmGy#I1(ZMHkg|PggS5WoyrmWZ!ZoX4|0P6VaJ!G zDvg&0=TrYWnP)#9&vL5U-DS@+XjIum)G2KGU6OXqG)iyg6r|hH1h()bm<6`@xVeMQ zF8Xf>{e?bM<`Fyi8ds!}*1Uuk!tAM1+xa0Etx^1)BDz*N{oMWrO%pUW4!>`PVO(g& zj3_RvJQD}=Y0v3~oawNr+@9Q&pHuD}g?y>rqx#zgs&q+u?DaJ@GpIqQnewW-GYk3#6XFfLzB>JExugrqI%zuBm2Y+V zOu!{a;o^86%hT#SgRiUdxzgA3wI=n|2kwE~>=W?qv#xK*kgE6RibsKo(IJn*hwDij z^F)7-kNLAq=6~$yzg}niOFH#g2YNbp3`ia)acdgR#lY3pr55OAiyE;7C4{nAm&5Zl zCIp6vy0G?VaDqAIy%C=;d%lfCX*C*v)^VK-q~w%`V#E*8*cwiS7(V&g`0~( z^ekEWb4i5xF`o9fmE5AI_?AEyP%{Sp0g(UH%0M`+!1h)?hay zVB8ioTFTG^sN9;4u?q7UeU(q-5QxamW=wd-a65VO<~7%Mf`3`%zn%g4U=;LdzzGm^ z7EINr8t4Zd?Xn$GPQ00vH%YHA#c|3!M&k6at`*0pm z4DFEso82x6R>K6GJBA;i+$p`=vp8;;)W(5mbW`9)WZHeKw$zw!%ck`<96aQv5km4X_RBkT|vNjWS<%#*)F(AS~zQF#1VF;A@pmE6H zxq51T($fN-!q{FV7cI()Ij`c%hILb}t@x!(JLzOnt%i9@?d=lGnKqq`L-4RZ; zvRP0P3ut^P>*(rY(i_^^7U7_h?9$_#yvWvR4i1jmOZDC@b64uTFNg>t!y;ttf!e~C zfystdChLzvtgxaxl$Y>t|7v*X3{iaqo+sz`54#f+DOE)l`lh4t#0~ZJQ+j3abRZYI zy6?95Bpgm2YzUlqZ0LC9j-)i`VL~*!LRjp-geRtK$eH~LtTRYRQ56TP8#G56%I6JV zs$4eASy^sRGJ0=~3S}Aau-e9sHUb-`+03?elz5VC-hEZwpeagmz?x8bDsukon70py zRgC1fyT-@pkVFA7qzbs^Z#UoG3{6klOinlX*5h1LQc~hlx`QxD+FtN%e#5i!aF6D4 zt9D*Zt;=%D6E>ReZFdem=g_HqMFZftQ!qYhH(ToMhUi-<`Q%X{{v5}DItFQS)K$}- zpF`*O14YR(zB>e>M4e~jp(WTqJX?z3!gVLbJxQUS5H^l=p+2U+Kf^R`} zDQp(8+p$TwPq)mFOcgJRuu*7Q|Ndzi5Kqibd2>J?+r#6&7lmwdc zm;EYkrbTuETjAtH8}!5K0A_T3ttVs_Zi4Kwy5C^G=9p%|hLq)Ae$IY2v=3)GcQDcJ1M2A=jpR=CPQEyq7u6ub&V4)*PY7yywawXGKw_()#|Uq}nG&)di)9 zZ}3*DVcQ9Gzyr#$>;X4d}ZB!UaHwFdAUYD3+d94w@lnqptiQRjQ2q zP~*#U9TLt+Nwk`y7jSRXbkh2Q*8T!#DHKJ`gf_Sy{P0M2(WYW}G!Qz+>{$hvc7B04$=4yd{0{EvF3em;w zY4q?@K`|>OJ_Sm<^G2KN{Sq}WQ@PWiNrGtJo7H@}QgO9#`zT6CdiWj(2loiI8M94t z{@dofoycE-5@y|Xn@`KL8&y+l)>X@ON)qfnQ&>GFSxh^IBfnf|SKt&MzVY)qTp~;H zCK|unZd2n(QarsM{)mIb%>md60>YfZlNZTe6!P)UNKnuS+c??r@yY2a3)nbyHH^T; zxw`E(`=K)clG=(EwM^`>AA9Zc9Zvx4Zs%C6SF0kj2O;`(+H~4@r%9>_kV5T7jypBp zOQk)PPoy~52Z(w4vjB5V_vY6cz!4&w1jtQn{Yaqq^3`GgDY$;Rt3R&k=rO6vmku!V z+qhS45C1z+1>&D1h?5-F*Hs=UwwfuzD@FXP!XBGFnKbusyqG;0qCX_84n*LHUwCMd zqUxKsXV{ZBL22DVwHlsIC0LdTT>6Q}#`NLP7ZdNiJ;e?4+A`k?n)RyJJ+3M9u3#;k zddN1aUhX)Frad{xLe+HAT&`HM(eD{j*|33@aJo&J=~yB~N!Pr@2Drj^4iiA4)e?_` zV${&*1DTm?#2s=^J%7t-h~fvJ10U-`Z9XIC_BtU+h0qxod%Zl3*E+p%HbqNU9==1J@tm*Tn)-7QY1N&K0%mT=J5EVbZBC;$op#w5v$r zXq;O>-|5k?$kkw^-y`HiSovSCnuu>w9NKUgGliDIp3Ad1jOMEca^4YV43s;#08~hG zl+C*DA}df>p<-@PJT=M-1<9cQaT%p7|Bo(%?^u~xzN|LBU-XTlI7zTzpx7K`k25od zB&S7N%SFTDM_*Qxp}QLb7I~FB6_WXpg0hsiF42|O+jq40O+pJ34cXX+&0benVJLoK z$9y45+kgEbu%*P%Ucv-LH71+mkm)v-JF5Z{RJSRYW z@$46t$Y!5)mt4b$Z7aE*j&);m_sLSljdP+WcGX~iBy+FdVbh4y2${@{EIRYhnYD)y zK-JFx?qXUGK!x1~q-lvgfvH*cfkM+Hb#JspmRrZ>fgy~rV0&P8X;S!Sq`+7VKNA92K=T857W_8%|7~%y+3~dfx__UVeY9;ynF9EuexVjkLCG`7HCj*S0Ye9xELcl*efFcwd4fh z_TOe0DhzD7FC!$8Pjx^$Cdr_T7q8@wl6AK;Tg@-T%5e8AB+>7TRsijdxkRT8_e*@_ zIXJcb80owgr(b0Ra!y?&?oMW;MKeB=nU^(0$#wDmROuLGV(*| z4!NzkhRgfmc1M|ZVg(5(HXsh#o;?#sc5|LjOpP8A868Cg#flyWWW?lBfoW8UdnE1x zcW)ER)(9>+|C%Q)eG2Fw1U$oR!Xc3MRiY+H)Nu}A26%?IP#Y5Njwv+j%K^iRF!8c3 z*3D&?43G6L41MKj{J}wmgG(2y8@rYFW+zmb&6ltgn|u`+oMGq&~^eyEfa;<2npZI%F2@3 ztzDym6NZUJ5{2yTha*+{PRa;GD~P_)tO9iEI3!!#2_OS^e+#9tO7sB{L<;Wmkq%Ki zw4knr5(SzN$kW)q9CCe*947yW8bLQ8A~}r6FDU*3B*YsBGYCIT5rE$emytgE_-x~? zKQOFYR|}ym)K~M@Tc8>inn?elCl%r*nGEA0%F!X}V_3>fBlxd>N(FfE};FUOewi?Bxkk-87f%&BL+X>lxE1a&==eg;3 zH%&2kZs=3T-Mp!n;N4)Sl-e9eL#mZ>JMq5X9(w>(7)JR~4|yA@CC+_?N*8u-M*zb* zxijvCkEuH#n$Kn0{WT?d;79#xqBVxm0^m6Loh4MG4$(xBq-#O6hEE(#6BzF~PYM^p zM9CK$fmzmM!6AC=7|RlL$uG`;>9ntry2afLkwPkGUUw;jT|n_jzLpa(XD~j=oo=#n zQTVU%D9b@37uwR`^cIX>0QXjJ1BKx5u;!2eYbQOuOteH2%3=s z$4(<}WV#BqE4J}P`!J3wfZA?GwlAR6IVB^07Kn&e5_wuZ>s64l;*MxMostnNZa8>- z-b&;Lrt~^(+h;&+_Np1te>djO4=9K&nq62y8&X-=kf3LiUs~RzpBME)Mpy*pdW7XKtb9M$efLec}+|q`MZA*mbVb;>GjLaK5VZ=?MVOPHBoPA)6HZw?1 zdFJW!Oa)ADFSE28LNja5mil$}lBw_J_gCvoG!W7Wqh$v7*wul0iHUag)icj|1^YOr zk=q9U`d5i*{@L<^CJAceYodYtU!BoppChY;+o>S6uP6iblJy8+jPqBdApmDhG(LMq z&fwsVJI^<&Hh$>TTP;^3Z%*3n0RD{Q(?Ifu7Ud& zS(Ff=dhkFc+KY(dSj-`Q^Hdiauiz1mW>riGAU>xX3-fYKs!FpXDBk$0J(M5V*kEA~ zw0#cpRNoTlv#Hke{2IwuC_nv@R}QtSE1Ydmpt&4)SFTdRj}TT`ZC;MyIxKC^%C@9z zwPDvch3FSr{Er!8Kpe?Y9+tu=4UHUmn`-cuf@_Nnn+@0-k5r5mISs;|uOTO4X77hm zqlsep-;f#1vG>SGCMG2*WDqUT8l5)?hE$+_I0OW>%k+TcnlV^%YfpmqB8pWPDxZ2% znbS4nKF3CT{!6Au^ks9u&+79N$eqRgNWUoiA&F9ozc7=iO(N#|V|XI9V=(MazIM@;U1`J^1Oa@rCs4%P!I7_XYLVIOe{nCu4Tmtc)Z{RSamaAGtd`JlYp zeCtQrQuZz`GNAOugy;|?z#+}|WAbM!KH%bsLPs%k?)G zTMDF69C+1LRrie<*`Nn)>zp>2e&hACrj!AI0%&%vFCZfkr{ue-+9rr5h7pv^b_T6( zR8SPXb|IB_;+Y3-oi@4=bNdnQ!6#{&7Sn%e`c8zCIEbf{w_-x3f#N^iKo#2Hjob@# zeUx=~IAzZXcob_}rB!#l6_&X>;+=JaC0vc@rZSCYNXqzBSfbP z2ab&EIdp?L>bJRSfvdC)oHN45<%>W5u%jlkV{&Mna z-~UUF{+?f63fGI^0Q7mvK1j=MNPrc_X7^7c+);>A|J8RUhOkxo*O z$m}C6_cfuwI-Tb{1lwe5tf=EFAoF4|<;H1vwf}~&?V*|^j0o~Q)Rd(p2a!ZB6-k2Exo)x279N$Zkne~$Jq){N z47zP#T%!3)6!;-Ym!fjmGd+apVW^G2b)v_R#|#h*4om^6!FC%%rTUGOohL;qr+ers z{)fjZ*TXP^nrtamQ$OSIp6DHBkcqX!Z2H35OOa}-^(F@S8%t-`XrYOThgAda3*fH|3BLbU1Ao8g@=2S9-8OHTp z$UzkM)xLCI)AIje?5%^^{G%{m+_e;UXmJZr+zKsF++9m?DIVNiOY!1fin~j2Cj@tQ zcZVDH?#|rV{q4;Cmkg8SeUtM&InU=j&nwFdBRc)J;rwsV;UFC*q|8-f5~Wu9!RLa1 ztRJi){O#mu`X}z{Yd)FlA*JcgWGov)f4=bsKeW+sRYd4hhmOePBSrD*MY*CR-sucH3L@lGuE5$%t*gbs7;jz2=uA_qrIsKMxf7(kURXkeq0N1Hsi{|O?R{(DVSL{od zFu*QaeO-I=s2l8cqDJI9*rO^Ph?6RG-|{ud;a1b>5WT-f>U6M4|AyRi^4AnhFvqut zopMpchLKAJcUj14Z)Vs7q*AESuw!$@T-usm^hBxWRAK*bhqxBD>Op37$*C~wF573} zsFgM!bPaFUR?Xu%{AStr#sC@wsEvDP=^2aeJ0P$xTxs-J26Zk1^*oH%`qa|h>m_qV zHW(D!QsCa*-f?G-{of##1Flhd3Q3kAr{@>ovY z9uL3LW!Ol})WD$kMQ-0tfsUVIbRYEG<_8_kMT=FDy2Vw3L#JwV3tb=5+fSZ;k1r{l zr1C*$UdOH;v&kN0j}>%heE5A`da7Nb z-*xRSeK^~l@7fZC9_dKMQBLBw%^=qEHb?*8+K9K=w;B^%Irr}sT(}DZ8*7dLXuT@v z5}QQX68fBYl?SijLNxZz#>wM0vUS@RXz*R!%hm{iqq_ROIFa64(R%+OFV z$2Q$hK_~lqo(JJGFSi>#jI*k()>GQVjlDVvkKxWlF%&`QD^o~l1f!|JHn9kc|NB$8 zy(+<=)!savA`YXa!AD#DpR;%|BTWPqSgP_?r~4VBApe)DU(VL{ObE5+&%dO3%@O{! z_8|CMYwSQr*3Z&gEAlEuGM-L4P)ur?ZL&aKbx8K)A^x~=(e)W-&`_tu)Jna9tWhQ3 zfZPfV;3l`_W3=gjb;RYRh6cB(VSeM%(opi81Nn;y$C zI)ThIS8U=#%PXDZsA=`V46^C*hq!r9T9hNDCYhE9*tp802M|t>h?$x~V^zyg`i# zhJz9?S|Tj8h5vm~vkeiDpmZp6Wqth&nV8=|tvTvH-gz%y&QwnzW%N1o5aW=^@?zQrQl zWZ~a6l8Pg^5EvH3THsN zlZ#v;1$F&$nrfq?s0vX2p#;Hj4E*n<_kVufh@=oysHJ=Z#Gq=HG%e10wk6^LvQV=* zIeSJZ8Pj#q60J*v9(x+wW5!>FEIA02aTv?|2S@O{aavziZL8m6WmxRDcGi~faH%t> zu9&paFJL;n&;IMiMB(?s3EMK-Q4#D0-ZvwU+1Xq=X26FLvA#{Vo$<}#!iYzx zKt=j%&4c;p!rLkKoB7+RpEvy1i>qAgDo0Bdt2O5rnJ%)c&-16+uf@+wn{4>W{rf=H zIJ7^6HcKq=6)Lc6pvg~b8iL3*Xw*~-z6U4b=ii%9T5_my;5u{G@!S8SUwnJt%AcvA zlNc!Sgz_1T^aUox4wFI&itqX_PnyfjD))ce1<4kiyz$Y7c5%nIOVo>Yz5_!_bscC1 zF~T|vv7+A!h3u2nd^g_}3y6(BTjT88oc@|QE>oGEw1w+bQ{rcW`I3)E0gGqCBn&-O zVLTHhNB3H`JP=`B>(F4V)gH&|VM|O#~=9B5Ir!B(h)b>9&e8<>v#RiJp5tZGbbXjrD{PXpH};Jxc5cJ*xa z&pKaY)mjby7q_rzp23;4kvI*^DDg5Q>w)=CS4LVpi^3g8c+@{YR1yMsD(EK?76xti zzssFbD=x!FH(5mgbnLxd1T#^BjV{rXyLC=ujPW-9{geIQl9&}zuiJ<||H4cSDa$Lk zTAD*b0$?iD4Ejj(q&uo{#lkR0sr`6JAT1}>NhkNF=#@T?5)H5a#FT;GXQ*(d%6g(bsEoWf zv0$@{R5SIYSUl0-2K=)>*DB$nx*RIHr~~{ju#QE$whjqUe$!hBy1vvWb9*vdu;)+m}= z?#~Yr{~TBpG2@Ux7-y7G7guMkGm7OC-D#;JNhhZFLGAypm%i=IR@A)8tMM2zo-=?ljdq$HuTM zf7+^2%hy?2PhZ=@x_V%h%5GA!xg(@&WQ$si1&`StA43*q=!BQ`-D3~NsQt>y${;Pa zC07SGmkQJGuWWZI#!q6H;S_4E7-zN80P=A|E47yKHp|V^Ox~sFg36GUtch!HZoLmbuM1j-Z>V$9XVBBcAt{j(<>Dgp*NIdINi zVzW1e%ZVj#Xr-w*ED+MAxLji#RDR7NUv4sXE+Gx$2akQ=k#=gY>{c0$! zRmk|VPaaZr!f2^#JLU0wCMVHJ);fCQ*T=bM~`1&v7MwmgB7 z8KxIogM}hOlH2JVgMPF<)mGC?v3|5H{5(Z$VQ?hIXheUy&RM4U^pfi+WzC?t6Dgic|?5!1>XXyLX`_;R4t58*p}WNbKhN0 z2yUyS{IEt2KUjDloyWcpjS&Oyg8`8^Cb53dEb;B^<@eEUnG7gmhkzuav<2tVjO6#5 zTqg1HLvkN`yAimVqL50go?%c7O;{V5N2|%>fliZx<37p)nX1h@!g5q=DDU%-kD^wh zk}j^2KBl#1k=ORM7*gDxcu8f4tB|3;XVxrFfI0=Fh)-)JSdM&tu z@D;v56YF*vFuu(IY}zD3CzJZqC*7S)T9ebG$#U9z6!LCY`Z-DP*|DD&a`E*J%EgZg zTsA(7*xTLcQA+eO9ImcDx9`$uw{ER%K&@gL=H09Kn*~x_vR=8<`;X4=T2^J}TRfMT zp&;#yc5!3nxNP8q&t(HbR1nYO9-G6`F=g44PPydsKOR9wej4^j^&4-FV2b}CRR6~1 zhM_Nhpc5BcY5E`AVGxv~LE9gSY@U`8M7qiU6U=S?|0%jj3P;%qzwd9z;`b0$C~(vf zidJdwZhwdKH(g-yvwdQG(Z!UK4n<7R+LOE}{e-IYbK|nuEhvbX3zo_pa8?li-@?@Y z`pwpv83iNW0n*Ya<^Zz-4AEb=r5d5CasrCLZ&D#Gs zF>m+~T#s&+HzP!55V2s_v}h^*o2dSuPj%((Pv=L3j$qYK69+1cJMpF%feqOtKQ)@9 zWcMvNlRK_)XQn@yZ+sS(_ho@xI@50m_x`>>%-qx+w)9r>(04Ns&YHiS_~OViNut2d zB<$PGsLB(S46x2>7qrcHJ#XcLV~mN>v}&k1jXU4tHc(_HeHHUrYE!$j@iMtrSV0;} z-p%sD+yb&(ia=^x^|~43L&+JiDH$mvdHxURxv#YKS~nmVG!( z2STWi?7Q8Y=LOFT_A%&QY{(L?MB;{2*z`a@fEBZj##OYLLC)Z z!A9pK%elrbaBx_<8=EdVe9fG1E=IGd#(lKiv%U31wyC8fXK9#tRtZ90x+*iiYBW5N z)e){GS)@KBtg+3UqyP}Ch>Y8+>TnZ&@qfbDIysNJ5=nO~beGMVY)zQczSnya*_m%>@F@Vt7CS#-Bq~DR!^9ohVnu*nAFj8yHg zDR~baM>9eXgJdXM=?)Ek{Qwm#50f3R2x>~on`KVgZVghQhiL0rFnfY_jalNy6;Pol z#Y^Wku;Vd<1?q<`5Ce5FR8C^mrGvWag*_c;cKrSpPj?&y`vUSD{Qc3f6Bc;@daAfO zQd$zXWz-)G{xeZ7n zSXjyKVt6gPynGD7nRYODv(a*ruIapwbQ{VRCb}HXXp;x_1NzBm-~4`A3|ifaek?33 zL|J0-VsCY{wRtyyyo1~{(RTB@ZiYvOVYw4F-Ik>D1NMN`a)GU_YW5QkGujvEAyBfD(OjUMe?LF|2Q(iYk!&dO zwkislVwmU}>U3XI#m>zQz8gMFtBit08RkRvWaFLUCv;igH0(2avj8`R3bZ^TiiX?g z?v0JH*l-5eP@t{e=QmH*u~YGGFghX0Yc>{u!cNFho-}loDx2mfnA9Za4H(MwIAHW2 z7kZo@r2zplB-3~?px4ppKgnZwq!t6ax$Q*A(b|Uuj^o883|9+hxparT(AGm zqk(7o6J$~Sqvq?5qqh5^J}0Jsb5c0JXe&~C_Y*9Cmy-QL8GsSEgWsskJ4EFqa5iFT zZf42o#FD>14((J%_Konzk_ERDR2_$KMq@b#$b4k;4^77zoj~&y+dfO)G zr#P}{w(Z2F=}nU7-wWkyO!&M5O%82cdb99ef@<&DDAw2yn=g=-76--SNa5rQu5>;W z3G%j6@6+M7>$0D90%zQlcW!+Yc^St3lA|>D4x$44BW+NB>6PL!$it>~aB1)_@DsA= z9cT&4boDJkqs7f{C!C%3JZJy*UYPSmnUfx0<9k7?k@vucBjH2d!%Xk~*N@41D%q47 z7&`jmlnPP_h)#>P4cZwJ9k)wPvGbVqE2B=U*zTP;6$&UU>e`KZfE#-U{@bJY)Wv&ECb2njH_*xlbQPV|UY96gXKGGPD-lm|UDNe4wf41P~}p z*b$iw7a}yt?&k9b!RW3Y_&jAM&2RRye$p0wiJ`>qPJz$7fKqa`o#@M~(5I5{onLJX zIeT;6MKe`0F8r#_WDrYw`#pJ=_HZW4d*`z?rQfnZbpB787Jb`|PvO-b0R zOK3ClVnX?-Wn8}bl=U*PtIbrq%J>8=z(>zew|^N0!Z25v)ya>(Mw+EQHLfFu#DNcU zhJ+$$&;9lb6{ZaL)L8c4qe^5DoA6^x2W%gTE^(!H97EYht~F%srWM<>JiU&am-;Xd z%d!Gr-Lz9yKhK_(y+l2}v2H*n-pgi{>n~nG!v?*O!H4tPsftAhMfn*KZ^|JjUx((j z{@vQ1M%Cc#kU@IU`){}PvZ=ykz^GLhT!vfdEnMsqZ5}YOgSLIL-48YHM~@CI-)}*E zaec`mc=U9M`uW`bt**U!u{-r`hnVhv1f31aFZ~vltT`{jYxTht96hQmZgn4o%??ik2kqC7JWSvL+LztbnNV?`uYsN)# z#a$*bd>0p2%#0yk?`6?P-A9-W&e1m6h0HHyTY`~`v)lf<(mx{@6Pf3nXMLK#vsXO) z5;9+~S!z3B6x6>}JU!mwZ*1 zyWfrNo#3Oz@TS6VNqvB#6v-VE1wL~2O<=`*suvW&*8?Ek_Q8(Zuqa&&3ya_o& ztPDQi;fpJ7%*@Dwj5_vh`oq{;j%Qji?`%ddXcwX@oTHBsYu12F(t5PB{*1`(<{v#d zfxs+jk|_gdfX){>I+}a1y%F8FAYXCrZyGvSM{2x$U@0MpNShKz433kAt;M>&?xb>h zC+3wv4&le6n&1o-UufCfO04iINOxVZ>`D)=x*GiFemnm9G(I!q>|YG7fbw#yxcJRs zXx7^ghZV`tm}W8O^VuDh5mkk0fap_(>)yipywJqSn|pSNdEeI&hZJ5vfR67}4H}Z% zB?bVe`+ePHTAFCS5Xv*4O05IFa&$qHAUlVt4 zWtGk*v%R{E8ZT1h@MBq~H?eNK-F%a5#Iu-hY2|b($NEZTcV384cK8sJ*Wyd?#EyKT z`H@2UN&h4ubc?>XG=Z|U=;ilXaKHZ&ItB7$_U8Nj{UrB(O4_0keJZDG;n|{RsV4Am z3bbcJ+4>Hl%@El57kkjS*g3}gOuoKuwJ4ZhU9^oxXJgBlx`FPuW5Kqc+Iz>4t1Q0) zU_3U$=|`}2QwUgf)QFUF%#Qkq>GlCirWc)Sk>BFbF{<*EjftXgDe(u5B0#OYY)3VL zpA?IxmQh2hf0-jW*l9A2C>`&MaWu=Q8^Oik>*nqc?6q<@^Eq!(-gP3|*n@sZX!aas zR43B@{9A6qz(r0j{;r<9dimEeBN2!BuT6N=i?8J83KaHI+sPimJe?FS=H>`;dM?*f z2gr0X-X3kZqFRBWv-EObG!06hoa=iq?dIax;EDG}HmOAOK^Y(};FCB##>Tmq5-G6^Vd2p92 zjRVQNuO`6&BkraADO&IwhJ7!j8Q9S~wC#GvbBCQG#$@_%IE~HV zc7-CZGRFDIl}~#{5h6T7dEDq@YJIa=f5%vT(jwXaGJ}>Qnkn_O!w#FR)_SO%L^FhI zUn16q-P}>yQ+1TRWm3wGLI~@`PAaYAr1@|p+Hu~;n?bj|e>viZC^F(Skf zWX(W$JtlZ1zWkeVB|Ol>!9(VexrlZnRAfCefxIJ-JpLU)>pK5dumkEb>_hi25tq+` zRG2I0ua$O}@VXEB99ON5yIo)kgO`is;nzps*YOLHe`|Y1SpsMB_-;( z9!xmol=Y}*jR#zIN&Y+Yhdj>@4slUi+by62Oy|80FBFak^-Oke-hw3?7k%|2?Eo9B zNigt0&v{>fp4uUp;5P&BT>xUgfYRz6z2ZuRAHK^svRcC<5dMoltmu%8uhgRaJ+^Vv zT3jsG?L^Cg+Msbni$l-~xp(l_q^JDtZ>5nU&4!S52BsYVnITM;*nXg65uDhq;tw#1 z3BexUcFq>5FHxrV(`S;k;Q*m0v{sI(;#+cXVQk&HHJI1|7t7-V+fmqT)mNm?i_%qy zKl@;Mkn7L1FlrTadgRluh%*g0dNY9siGt@Id?$0p41z9FdI^E8pJjWb+z04wv-m$f zBiWJ)WS6%+`;&U(ykvj0hXr12^G%*e2iAWj-h&T|eZu9njKi+H7Qqpac~lv%G4?J`)sNtM&-q2oCv%r$=ngcao31T(BwIrI|3%72B*_ ziL?Y)8k9Pj7@f{@lzwtJ#+U-t7emM#U#}Ej6C_xZ%uOv^8RJ=Pf^%>L_^Qfs1SUTD z-kBceajIQbl(6g|cMOp^{ri@vx?^xwg4s1yr7Yi@sV4iCd|8#u~Ln4S^ zXm}>BbOMW`D4M?rE>Ex7REJb~rlif%zn}u|@_(&)edW;%}li*?=gcj20!m{~<*Fxg*9G4d<*|NYwU(NB$>U~q%JAl>DJ zdBWCe`6DRU8f}|scG&&4>op92g>E**=nfvt559+zn13cg(euXRUsGRSPG3c&mLY6a zy3jDD2?PfQ6E4B}j{VNTOm(LbwH9I8HEBM`^-a$W*Ftttx4vLidifuMp(8#wp+rY| z!ztlA6NuMCjcY{kF_{20^VuJpMTb`b<)JD}MZsJ+Ixx_+g>~Q3JPEo>)~@kav@#I8 z3tpWnijZq-9eThy8$U*y$hZDOv=YLP6}=x+WRX8^b@Z@H34WgSeb%h;d?RgL3QI=# z$%VAqBs#`4$Ds*vOK()IkEN4cwr31>_VQ;@MhyeA3^}c|yl0$HG*Xtv7EisG^1zv; zP|+vjO=AOs>8Gmt5HX{^r|IkZ|jhtmc_UzUaDQC|Ng$O4BX=WpZmX5oPNOUQ+ zUxilUX@XA!mm9#yzX857=Rd|ppSl|Ub`58DtAzNxdjc6C>+|0shS1VvW7p>t6UgLH zfG8gZH)OMl1OJM13f1W2i&O-1L>eMu_EBERaa|U{Po%R+r!HC>T>eSN3P4}akc#&e z-kos+xCdc_^5*D<1*Ig7{sQ~puJ{$CYQx|5^!UkH2sWPY$9?Tini&#JWQmN=uq$p| zF=Adu>Pa7{o|Np!Eq0u7Y<1%-1#N)+idOdY`VZ(`#RT6#^M?bhPx|AYpE|B|Yq_FY z3dWiAxP5y(_Y0E$bXEFc~j`drP zuYL0!XbgP#sPI%Q*7<<9A$^(IE)l5^TTJP@Gd5$nLF-e552R;{>X3*9d&tC|0TzU= zWxb8PNLl;G!9Sb*JKTwLxGa?F0!L!XMk>QWCr>@&8mmro_F(I}`bCD3Djm|90+t ze5=|1K9+`L^y|-cyCqW@7nSPD^aA|#D8F}0*WK_T8$>V<=tBr?CjB=?U6)6;!7p3T zJd?rV!kBr9QvuX!hu@o0Bv+SL`e3;ArZrq}{^O;`;EtuDM-eM#En*6w%<5ssnbX!= z=-JF_FK^BJYIqGJLtLs|IH7W-;oaDy=RwR(fp}E9RhhQd4u>Ih8RJ+xSdGW-ZEY|T zP&oS$sP8uJz=Gt>{)8yFz$=toMR8h--dB}LxB-N3E55Nj0ZU-B_Mm4OXC}u$VW5p% zuPe~Xj*uuBd^Q|rWId656ZtdY@%$r21AVsWq|`5h<6v5JFQH>0Y~QtR9`6{6i}QDz zfG$8^_HyI7+aTpLq0_rQIi?Ss>=YG`il=ABeF|5>3DfR5kFdbgSmMRWa=tFa%q~5G zl{@G+YUbkWBckqsh%gD9=Uuhcl6$SIxfj;5+`p2VGfs2YlZIh}Ek|UHZ4c~Bbw~8d zd#V{vk$jn66qw$ss)&@FL7_(|3~b%V8ePCzTW?qJT- zkz2sX^d53X(Iygm<5?FvFc%<7681vgY%CXO_ye`pa&KuDcCI_i;&J4*)s7WJBYKiK z#P-)@fzAohEz=g($9$Dw&HO0MdY0SnX{Eu5w83t;&U%r-?5|UnW*6aGfa81n?J$JX zMGMk|z=+OcDE$=HM5IM;PJX*I5J;QU@7a7S=0H*zeP_NWrr z0Va12K5vXj1T7;?hC~EYmg|4DgkiD}B>w0V?_~{7VCzL!Fs<-a%UxNLaRGK77A(_4 zS7(;C`L;Hz3;NES#v>`nt&40SR?^&O{^7w^-Be02Ca}IPE{y;m zS^hh?euG0xvj@?M z^mH8{jevO3SMzT^S#dpQT9-_Z64%Cd&e$9z(%3VDN35HOQCK(D1;==wS@jc3?ZliF zqDv@~NC}!=X>HX)7Zb{2#IJJe?m9lL;%@sQvcCy=#sdXWOGnv^Q9M5WNm>Q@Sc8<3 zf>tdxaDU|nIgC90I5CGsx2ai5;BAhBv6LTYQ47ftFfgr52r~E7+1o)?i)gM{s;G7HLHlM7PnSKDbeP?*p%xZ+Q4^hVmJmN}jNw4Y)&9Nr0BwQIf-V5n5;y611Zt%y z*0=BvtB@sn-AI+C3ge6U|MCKOmZ_8--Z=JmMO+nE4ZOJ)P@gyds3)j05i?|;iv^zY zGdq_O28*m|p3!9_%o|Ui3Doedmm8`vWL>3yUgN3_D&eonY{M%O#3YVNUxOOkP1>cg&MpkwzxDeAgzBk79lK&tW*y{%?gZh z`P_f@wS`RZ2{Nlvv9fy*^u?Ka;KB6nQ*BAS$R-yU%NbG8=|y}+MS=oQi+O-15DYvt z8gaudsou)s={?}T5`QfZ)I8+f<068G&cB{Q3%>ev`uUJckWyh71J(d~o9^^kMlje? zCXuz4<59e8S>@qt`)-0%LPEq6X+P?`sU#@H-1keomb&>6DHJGs`o#yZ`F>+PquNT) z8_%o;2pk!iV+08@nCb!bLgxshzuuAq8}BlWo-rm%{YdFWw5^l;c3js3yk%Y&6=nXQm zNrl&?=*M>mSKwZ$qx@g>>N?b>$6^W@MN81vFTK+Q`l~kNa{`9m3=X8087#>@(#T@Rd!3V6(3N%2s&?r ziBAA1#wW7bw6zDIaE9&1O`^OOm2=w3?v>g^E+m5}!c({z7!b3qEQ8l?LGJbK@%*IU z4$pETkH+KT$^_TcCnTV*!Pkug34>Vl?6W|hyN6xbG;3=g<`sy?xUcq1e~+vk1$+07 zwl6&aljKJXY_yo6(P!bROAL9LodhuD70S;px0bWor{z;HtlK1;+M1P4`UvK}XpTIj zm&3Pt50K?t972Ea~L2ZS&F)21xUbbj2uAEu>w0YgD(mJeJikW!@V0GkCG>3?xxrtEr^Xaq_| zK^46ZbpjUE-3R#Qy7Su{%C)fHk`v*}ev#?WGD%lYJLO-L@>DWKB_&n-J)~->gj*}o zkS@|k_fvuVyvf7Qz1Q zv0!e?3<=F+DYYumso0K;g~>l}_U*7XVJ?Q^jdC!Bsd|(;E(SBiLV7(bHue{#?Ofqtzzlu~ zsQ`6#B6S7>QM3g-ZyC#l-wh0Rc)cH1W0b;K#mLl=bGCKKqfv-(==f&x)!iIg2%lDO zf6O$#sa~);)g}0LQ!B1qp<1P=tL71wc{b{!L5Eh;T52N2YZ$t}yX&av0?xS-y@kTM zVXEurL;pk~jndk@I(7DnWJ+ju@DAa_ExuAM1YG_#GCtAA0uo*E?>E~u0{Q*F_n~wx z>)9R}EXC-G!^8VNOzR4n49Isz5TOSy=Hvwci9#__c1zK^Rw|sQMOdP%8vx&c7S2=2Uo2y{l%caMLtkTvrm= zKM84p>GkI4tA^HkLD?@6p8?uG>O=?AYuhyW2Qn86z(#A7BrzgJ>)B!VHN9p7t$DbPHSm8csX zyk~|uZTWG!UR*ZbBe>%iZ-yWK1M!ABZ0)@2X}NKzYZ{iqD22gc-*dG<>G8T!sCnS_>L2WW2d^k zexsCJ6%)?!_bY@op>RW5cd43=AK%}|Ja?#NZ>2OcCHE~IQ#H6q8EueSyD!2%^XXiM z2)<|y(i=Q%eJAmwNLKJ0rS#$|K;~vb8Q=Em1SV%=R>XIU|7Mi$lFIP)|5glOe8PL_ z{yM+T8YMi@NA+`SxUQfxjc>ip;!n14C;_}z3;^+D4YT7)WO-O~AYY`BycU>a_SSz{d>56q*8ESuIP05tAfslm5^-jA`|YZl+(M$)Ml5~ za7tT>C-H`nI-=oDRq)EhSCkTZZg1kEINsSW;$S5wHc7L4DP_yy?$ervantk_Il&xIAJeD);vjD{srS7G4b3wBt>7CGX zp5s)$n!fIPq3|m;4)wFXIWa;>xx6qD1bJIX{&N88cEPLCx__NIoJV@FG}8Ury4O<~ zoMu{V_I~)eK#~4~S3(uteYXC_YJcce=vr(&O=~DW>NBI;WHN{ArH7YRT^*?JFoJLY zkjHYq0&;&=*H8XunDw{bX?0RAiQiP#ZpJxX?WX%E!SZt)^6b#}c1gaSt2M>t2cpEY zGo>BcE7PlM3&!sQv`-NnSJQV+Bt@cync0MUxY24)4poa7B1bwyQGv@~HocnVOPEoW zjuFBMtl%V~N${+RA_dVZ*c?xf}RsV*8*THCxm6LDWvsr!6*AV}RBfyJO*=mdeQ~16h8s zO<{X~i_sH+-PuX%l}x^ETt40!?2hL$N`{{P>l2s~v}@5l7Q08E+ESsUjdX9^D&;G$ zy)InKe@C;QmA^@CA5{2@5KwQDespOK9a2>10wN@tpKu?XJv|4W_;^-A(iJ#YhF)|g zNc_+mzqQu|$XCbSkgEV|`D;Iiz4PB3(|6vk@yA~`6OVAE zr40pfbpb}fYNX2pANP_<)?a`1JCc|N6bcVU@G#>hq$Z+rqyHs6#<9T45W9L%x>~_k^3}@X+YC^-(exlm=9B=4RyfN~M=VvPW|RDO*!#;F*<) zOy}yWyCT9k#RY3n%g{B*JCQI&Oeu1^OeP~}e)a^``*X_WDU%zen!UgJC)d02@wEJT z*Vzaw@W}zAGaR2(!*C)YhsC0X0lsdUc&N9Ri7+KFi?sh81~hM>gi-)pm55>LH{;V8 z?^P|cQIeSXbK%b{tX)EMq^tUfk)rkl(T{>}f~>mxd81X|^YE~Sxk@zeKyLhzy$ym3 z?)s1Vfai{xDWfP&*-%A6jbWOi9Ik7hYKr@}`^@J*8P(rc2zJ-1D*SRFf3m_+8FnxEsDTAMtNjDu!C^0`3P)#)>3x0U+Gc5G zp8M|$hh6f7iN_4?W&cc2?{KOJ_WQAJa6{*7>#EdbX!YR-NS`)1*aW(ks7824>vM;7 z?Qa$l`;C~n5+26pOhU&ZPprR|)V^yEVWJjyP1SU~*n%h9INA(81i;~qfg})UxAsi5!0IfyxI1MQUMAqq z>3a^v<8#0_F-n3Dhu>W-mp4vM88MZ2nP7N4z8ttHkP$ z|Ae+@siY)T-&SS%L-)hq{H z*y~LyOAs%7Av)|cVs#$?Y3)5Ymqy=>yfu}dTE{iYiMWPyHeftEu8#;dPnY};E6@OZ7~1*tsGxHO+|l}%Z&DkYfks&CzTJ~Ezf>1ub34eh~jfo7Zfv=$dqF!rrm{^7*-_f4cLy^E}%$u8{n zy2^TzabJoYqPt|jQBY3I-Y~fNy7{BR#SkLFmHCxvEp8vUoeT8r*&gBrJeNqkh>kgi zI4&?=KoB1XFg}8qdgHyoJj9yK)T?I<4_*HOlH8<*49l zD69wH{mxmkur~=;f=Dgz(xcB=p2@ZL0#YVeOynQF^%;Tb-Ap8-qI*@^vmyv#1dLFC ztrIFzboaHG)wNG zR2seDp!QA9F`iKwq}XSl$O*0Zb7Y6O$D5O+CVS&dV9rPOACL4;ny8?}aed4EJOMIU z@*Y67gb~0Kut7>d5w-ZxgN_HU8`OXzJclN^>?ieyeWlp^Xo(`^wDB{c+?wUwUj`>r z%sEbfP=zAXz8gnRDukdm1eCS8Wodls3dSDC_%}cqfQaI_F6ygx&j# zp0KORB`nhaQ@nZ#4!H=t8{H<^yU`DboAmi?y=)*+Q8}Hq_hu+6Tn1NEDri{?+`eneeQNak`_8Gs=~zPwPC)>{QI4ra zyg#SQ7E&p}t&7bRtj``;wfjS|t){cVR0k=Nj&q2_CrkaJ@#YK5K!8OkRss*aVmF6$ zpwF42_l-GO-W8T)(4|8B%@%xzjb0=5)1~cz9L*a2%1bsY9X6PH4noX@o8!c0W+A(F zWJ!3*g?rRFXBj5Zzt6iqYG8OGsqKy=ihyThO2q=NY=7%RB*?as6ZrEm2~zr533L3` zACDVhhRBUUi0E@3>C+^oak|J-*<6NK^LaR%s{tKB=m4cdoaP=Z#n_xbRJ!rbnoK~K z5vEWy$riNC4{>giP8Lb3mr2J+uab@4#RYwer@?R&G#u;82-zm~#|WTZ{S!p)GVsGM z-0#+V?vX1{BkO}l)=r?unGuy&wE|PIu0=?F{|yVqwX}EF>bZn_s|Z7+Xxc943l4AJ z=?g}=9tzG}U!YiEcnm2Wh>gfq5Ie1%uzUV^&E4Qb6{JrZIAxXeERD|x%NaD<7Jt`r?bbfE;S1KyW-Ony-V<~5EB@! zAI{x;G@mR5C{~sypNw|h#VR+w{Vs`vmWd#Nge`_g=}7L91E6&O`!75`OHx)^9QNI< zao7jHL`=}8$*QY38;zy2o~ZHzL}}a{Z5%dJ9f-V;^@vNl>8sd*E8&#z3%N|534hZR_2*$I>xEC}L>LVoTo3r7UN)pd-*H7D&_PIj|bDgK}xc%zLMhRQI ze2$Bg>%xzk>irp2aJwCnfX- z_VvOI=Si9OAF5TCHo-H;RVtW9T<`RBI3&7&tBNH^X)oY(A!wRcilUrK_W!W;oncKjO}Gdm@*<$3AYDa>NN=GR zL5WDy7p0Sk^b(4phAtu^T|i1GA|)av(tALVUP5o7mq6$xp#%aae&?L)`cC=#T>EUB z-PwC~XYTRUxZ{|X8Fcy1)g1BM-_Kc+IkUb=JyWH)yFzVdXKF%SO@lJ^6)daiG&W7T z>Rw=8(`XxNx*=5xr0xi|u>R^0JAnz{cXGpdV`VFk_UAivAEp0}IlgntDqVh3dyaXp zR**HT{&szRP{rd{jg4-aG&=10J#Ckn7P#M!v~^$3HoLZRPpSPtjSTzBFC>3e@%mj! zsyEcZhj+PS^si8$$kOq3WVQApm7OeFGAUmd5Hbef{^e97@#AgqpWMX9+`S_`>+V}I z-rZY1a+GLl@7)fyda!K65}OYD^_R%mh=nVRFw0M&Zg1$g)vKpXGT#2`R;Tr3(2ZkV zy#$x@yR#g-%Dw|;8nXBv>b8DU;qk5xR--5MG5Z#C!bh3N<&KS-TbdVHjrCMF+>xOz*->!$6rJWtE_c(^PaIuTPe$+9VHgZH6%};nxq12kgibMLUJ&Ro(BY%h8|7W zZapVE*nzmtzrA)C->mhJ8_apx5e-6cG58=jIYIjc!MqjGuGJ63r2R5Ejc>aeObgge zuUThvo;>75yghiea!;Q#*qxPU=AY#sl|N#*s|yZ#1&pORV+9MjOSv;9z6Sktc4}bd zwGk4lNvC&lq^bt_izUc#smyp3UIz~rHpIZrj0NiQ`u~Koq-Exy7Tt*T(P6}hy&Uj{ zA@IjnhQ1HoJ9PIWp*_~%_o$*hhN5b9?K@a5E+3p{_4^t4GtQ#yq|qt{-Eg*=f2rd) zlH}Q-A^}mEcGG1UzU7gvBkl;Qmxs^LGy)w&EA+lR`*c zy&>UPi1SZnmenf4t_!0bqir<9JDPvbZ}D;$kS7z_(j|0KUmE)A@wPGq+55z77N<@F zmx_G46SB(SQ?SfRc-P!X*y#Fr8qINFGV>MDu>iWTqrO6AdhW$%RYv=b&bN!q@T9&i zU>0_?G}7&R>*6u+iX6DutQW@4N03&_BGNQbnM@47@AVbr(?t_IuENR~XTfFB=tce- z(%pXC6mNE_yUS)J@9G8&rG0bR7J@sqMeh*t71YpdfEqED{iX{9=|>n3 zPxzY=IFw4l>kgrz4xv^z@R5A->&`w$*aAI*t0Qs^oGRgySbg2YpufHOBiqht9s=mh z;)tlLq>JNgvq_h~Ft0yUc(JSixTvjbQEYRaTN$-!Tj=s_tLd_PND1>v14!FfSXdOq zzO=1)dp)7VKPpuv86`6H`gHh8TLCR7f0&78g9(bMcL&ftL|uXhp)L%hhk@ghe+0XH-dBOsM_ z@68&CkdLLd6k@UoU#kqpfmN_zIhd++$w5`+U)LdliN^Z{-Hq*`&sN`h_xfcsCaaVe zHm7|LNH)n5QeJYM_8(-K*L;3F4jBtw)>}PdNImN6+eol}+95|X!zvyN)2Xw$ENcbW zns5y;IIhe>?srtw*5zpMp*^78T>R39eELijnjunRg?;G8 zDQngE74tM+^*7-}?$c*Slk!W+(3}mXiDE$hSYuAiuS&Hbc^?hlwT^=QLH4>A#0MS; zUr)x$8)MvmZy!2q0Nt6w}^E;Uv7Su!G1#&%qGg<@_=30eMM5SO5Ik+d%1t*ugvIF zo)7SP%mhY4??L+2P#+T%k#|fW_IhsfeQc*NCewLP|AfgK=nvi~&Nq!2K)F9vhro|N z8*a{NR^1MFKH>NdBJ}t_Z7TLGvaApzC~731-#NnFJ7ke-uIBmT0=}uqEOr~e^qwr! zWW6nm`q9b{KsD5fe;LP5ej-)M`8-v;jwVDC#pgcw{tB40K+<50nvE|?*-5uD1KP9i zVt4W(^Mbmz>y3U(p)@b%79xs@HU6|ZwG@!7!sj1y`u&;IXBxAz=d2%-S2J61Fng10 zrJvtGjxa;JPR))C(7Di>u3jPsvM%VzQwgH9?lAzUds^<(0M(`4gCB8-#v0KTOPg^ zQ`!TZtV>=OWaSxV`Uije*>fvCr?~vy%eBQ{27q_=cv=(V+$&BNpTyZ-+Da`gbMtE1 zQKbJJcoOXTg1Ta0+=Yq0y7>-3j?~{0X1Qw4A0*jceMy5YU3yJF(vGuDu146I5IQiB#XboBE_=9`^^Wui4mtLU^eV z66Eg}B81MXT=2!US20KW9&bumq4&+;>#T`(K#=^qU!8Ac*PNSmbs~AUs(+31cE_Rm zZo^sSNuOwDp7iyXLgQ2SXp%z}83Ys@;^#;DePrc3s`9y6gCdqjk7N7VAgt_MZlswj zsnGdb;$9+LgL#Uqw>J3{5FVV&-NPc0H4`@e4a!7GFP+ z+sm$A<~8pSZ6%x^z2Ky3Ii-(dogj2_eb1nHDel)X2Xy-P=l_!543nXrkpSgm_DLlt zI_UA#iqQ>~O*c=fc9nc*pID_yCa-Sut^Dl0#gB3FY1~bj!kq zmU4AT`PU(10Q*t2e(FH4`CSXd#BRmCnq@n(@$g`-MmuAz6UrU`HPH*6uaA?M<7?yj zbz_W`!CCAQEUh++2GyUCZsQ#4{v5T%z81a*N{%Y-Jtc(=x7M|s8M(^g0WVI+%Jj2eF8SovM`jV${UJ{UWG)J2rw^?(Yah#o(0C zTx0tz0@O#lj*XOkXB9gl6(*E%ZH+G2Y6syHYyP}CMwZ!7fy5a*G->PgIn&Jr&dnB! znwvW4=sR!h7CZm59)=}U9BiH#1q*GBShQA{1aR@AnI4~VIZj3nzYfq6+ikZsZe8fe z95nq6Z~2|zDMns==CONC?x>g6#9-J7B?(*P0N{8|x=rnVRi-cD_zuE7Kcce}NGTRQ zDFao#jFlOS9?9F|9$RtK38mMKG z-FlU5+s_a3njdfr*S?iGj1E2LH!7SXb=mUr94ZzD`p*$G{D`rh5drGl4EXBq(9)1< z2@FS6|1TMr{ci7qo?^|VJHJ5IE}|@7%$740I`6vPy0a=u8&7|}#?h$&wCUHGxUyQR zVbFWNp$5H)_pA&lEs@&udP?E_ab?lD82xH1OhUC(JdL0Haq;dY_5rHr^eG3vFQq7m z7Ac|vPa}{VH1k!fnPwdlOpt@XvJFT~QO#nY1U(8W)flOFFmY5C$PTbM;~02VKgQT_ zOG^=JT`kJVVYQabj94Um5oCk=y3n|gwH35N6MPJsiDEn7=U5Lip1}H@i<;PH&$Z3_ zBqsJWP^vm-4uD%55W8zGEkJv((NTdmp@-5mjfK3sbo<&y@rKnb((v?tIq+ojM5r0cQhPpHV(J=Q9Ze*$A)|_c_|3&ju~5mWq_6vZ#wEji z)m2wbk+iWf$IG=Jp{;d#u2-rqNbU&#F^Ahm=3sX#kFthimbA>U?|1*lCw>ORK{+9p9b^yX&3440(h*5QV*t_{{({jd&e3ymIU zowK+&Q)f-b8E-c+(U*#pjrkOqalFZyTKu_M9Z`NE>4_k~!PY~(Q&~z0FA>jB-`J<< zu@9yracrsynZKZJr(yw*-)^X3b9aUgaBj22ji7cIR#Y1I5jC?r2TUXf6*&qPD$?D`$3$5%ewW{$a(MQBVy5 z;IP%+oW^_*C1I&N0F>Pzj2N_54X!mEE6ZSzSySpw!}fZ;=RKi6^}s86`3SRrW2+vB zt^8E#gOaXQ^gX>;sZT7n#EKV<8*k2;^9l`fZ8|B=x?C&rrB^-xTva7!D{;1}d|P}V z`UM@HA@p?8M)ad`mm5$5FT%iXo)8VUs;FRX6JLRHHBvcJf zNvu-Ct~B1k!w(r!pjP8pt@hv99U*nI4%PYach)z!EQi*>+^BYk{m|j`lg1L<=2{c$ z6wr@6t}{A^{>OSI2uf3n>k^}*o6_{kC=pY`?rxh3<%zW5xbO#88+Pvo7E@y_#Rh>1 z7MEWOM3^Ze3<|lN8su-)6_<Kg6o!ZI{r-@Qra(ssL>3hkNw99RAv7Dzc=!ZW-2eJbMZ?Yj%zVNO1O5N(0% z@J!cR&?*Nx>P2^XRU-@(H^l?f4s3P5a}y3B^d)8NOM$7%2TzYpS!!-+)%l`IGRh&k zzK6;FqwZwl_|ZxJHeOPdPlyMcwp2X+A*M8)P_!~p=+#-Sr+dq5G%9wo zqwBpo$9fQm$BG2=VQmvj7y&gbg*5czp7!dTz^5=L06+GbOt_ypghQMf)-*}UOns9N z6r30@4JDJa&$Qq~i*Lnqgz{RqBNUbDMSnRMC#tx$jY~tudmqO1^DDn&z>x0x1{TX- z2NSrJ8T1&plFC@`bu8Ol@NnQTUTWq2%-|NU-*3jUr1qs`Kfd=KSy)**Xy0<6)|bZw z8GHy#TL3b|78%?I`_GQA`+C0XHrOHv%)XvAMb^38tqo}{-@bAcB`qeVlo{CUw*{h4 z-`+hwSXT6PFZ?WP&JJKLxpVMAF=NG@PHBD5L-Fx%NTy=TA6{wGRU*qJRL`!dEbUYn zJyOu6scZOFb{5yB`o!WruwU7`o54YVt1f<2$OgTWj4$(m)#kgiuGNbW$CA{(#SKqm-iL(lh>AVugc}YzHADF0b9mh8_x8m8r8?H z>8OTetD+AUq7liE`jCXXN(+60zRfZLW$Ks+SvAOHRs1?;O3kRVpMHUHOY;k7uHJF_ zwYXFShs)DTTORvMiCqp~(W9(&-q-JnN$+h{Wh9`g?yk$aPd%Bo!RNMW6>@S>snnj1 zIx2k&F#>SAc+^A>3dF@AEoy5gfTG;8tFztN%1(KOy{F z{JayidCSE>ZY2k_yz*xXFJ%E(ole)#mnY4SlX^042JTHRSn2Vqo+US=ES@1Rz{dlK0ai7rj1_xGx-d) zdHg)GLc(`9U(K)UbrU}4#z;ZHS`GJ2xLXpeyah@V@Vv@JK80p4N*_2U*U?mqoN^}@ z?z~kj8k~M*&r~eRViR2qz1Q$t-~3q#Ze;CQ9+J0sW$gy`8K}Whjg1fubV@+zdi3kE zykUA@NG8yf$P>85bN&5^W`iD3OXJnIYIj0#)pgFQ*X2JVhR5;+HPb06sgX3sc%@9fcaD<8W+hjIuGHc zo`#hTwkRW~%aWI!3yBq8!33)y<8u#sovsf`Gb%Exd8+emh6A)#FL(`q%~=}ERD3_! zj8%~cge{HA<#M{*!(*q_m;x$hOo4&;S`WCjQqAMeu~rFv&QKTZ5T~E4Rrhpskm_>W zxKb@WlDJY_Jy!*fZ*qmNVhQ7>U4WALbhwPbYj<(Gv2yR_vdIIQb6q+gH1a82lTJNn(VEviAU8d zWngK|HC9BSO2E3VYS}wKhM?sf)WCS46Rn!4b&qDpM=c`|Zm{z@H(agi?&-4)|LX3d z>~hzZ*z4uMX zlzC?+cQeZ4+xbts2L$MYED#`DOA zC?PX&L`FUMk2hQS#XYpVW#5*mP{sp6Say!yqpzk9O?-jM7!|zQZJPO#w zb)(Q;wE`Q*hJBf2G$j4An99)T#kDZ~{V=P!=oN)`r&7t=o?wJ|V)EWXOnNQ zyZ3x{Dj?+qS+(H!nG-U0lZ%tEBUQucMtdXhf+;Aj zt;)+dwNwN~1aMyfSD28!Vp^cWmK2xm&oq_&+JY=+AG824n@J8YMe#-c0EAnU%|a-- z^XH;!u{`euB|;XjZ#J`Y+GOK76Ah_1ZG3)ud$);<8MO-B%r0DQ7&HUgMeSJCgLV|X zhB{RKgv^KG?7Pr~Qj!OD&f~}FZ>>n_jNE(e#JBq_-&GfL`)hxFEtbRPwr0l~kX||P zgB|qxx_+}5PN%6P$;#xfc8?P{FcZZv$39-)AO0iq;Ce!cN}g(U6^4u^F&JWqx^wlW z%JVA47W$^j+x0wEb;Cl9r1d&rU;y2G)h{;HB*0#U1E=3B-#Z)kAmT&p9ZR>O$uZf_ zm+tOKxzohGxjG1F{-%@1--Hw7UHJE*jZ~+)lw})Z^k?Cl28`^^7yf;w1H8c2nrWDUdGorkL({nN&k!S zWp?6}t+YurvCi#HEYe_`P4uUf65_#1ti#p|6tYxwjD3F+$LrMTTv%;?b9)AScCjB7 zhh9Z1Rj4KD>fbqk$Z(U1a(j`r;Q1&>hx2659FJL{18fa^oQipl911o#;XD=HXC&_` zssIA8;vTXK;dV8P3sO*>J4zMrIW4Z~waky9qtpKl{8Nl~)9_1totFxCG}%kr_GQ`? zZv*W*1s>~@3p9-P4xfV*k6(*D;w=cGuTD@SSu-gHeDTMq)&6wChkp27NSO+-k}quO zN~XelhNhu7UXG!I0{qyp`>h4vKkPlo%~y2|t!h5ZP}sO!P_|gpzF?;0e)LY%Uapb9 zoZsWEqk=LS2=5QOpC#n{94z5~UP`AX7-tMyQ($;aFz?xQI^+$_j?Ajfsc|0kgTw~CHMArruow%tD-TXMdti_x)^e2zMF~-4R z?TV_8U@}3eht6|Tpm@`$eTfc|z!iPN zsP=~UpS^VHmMQg*O~@36jg8G zi^wtBNT+BCZgBL-&70E!XG`f^@Z&Dm0w_@qQ$- zf^3mL8=tDhbUv4xl{$~1ywyq;K->P9JiUtH)MSZPJgtJ2g1xw2-j{JW>&Rv zW7#>nF@xL=vC{3UjrIaTkJ6X;Y79;ci24iJrMhryChw3?tv5gQtFm*=mzh-q;>0Nh zbH^pKS*AbmZJWc}w3TYIMFO0K(&zKi-9#G~xdbv5gj}_>$$aEARU;SSa=dS1+b2aF zrpvkszmA7Gz+~yJ+V#9-nv)N7JjJ7#>ApaE1Vk*|!VIJx6%?{%!W5H$Lq}X3*%t0G3Ot!NzhuO&szn2IQzT`e zQAlT~0SQp!H>eeum%aH6K4)l;s@_z=QqhO; zf6XwTdxdyG{1-Xj4ScoFlb$4InkOmCb5^`4O9g#ln%uh0NbUIcGEpBR` z#u!p3H?^HVuZwiC5KL&z6_%x9=4;H`lo1SRn{?OpIvrYjOx+`Q=W>~nD5c_G&)&p0 z-w%089eJ6RG8D>(AbRJ0DxhN|i`&Kh4589#s`P~3FX;cL*#G^(&GDwOC=XZ08XMjG zVAs3sivOp;GkReLvc^Adf`OtBeyyE!c@E#-*s^Y1aGDUkk#fP*yya;8imNX3IK18TrJRl^du7+9{C+BK^|bnPj% z^&NQu;Yk381&O%!NB1ip^8o}3$o6gw85GJOc> zVj*1yaXru>9dZe$jK9CU_SA!~?mne62$f0<_jb+cxuBvJ+l~h^vW`x61qJlA(UU9i z%1EJk(aLjF)8R>zz!I9^xIFqw-})%ZPANxrLn79Js zZ4AFP>HzAy028^TgG#&TlBWG@6V^^!$J{6O4~i+wf4eRgR_;NpxTal?SH-H1a8TR0 zx*eFhW6&=9X*}@a#$J=kU@C*y`Eo9ROOHdL&>9Sh_OG7 zGt2VYSTBJ>yv2T0tJ+l_C>4%gR1$X#&R2sB$15AVXSUsA(OIz?nkH4JL8fT8MB;G+ zB6I~W{Cxi5sc^+Qm#hEZ%zY-x6e3NqG8wNAQ7-PgII1I#jVjYytyTrn6X|1vTGr)t zk(nE!-X=3d7jJAH%dX|HUq^OQ;tEQ8p;sLRmkx<58^4zEwfPwp zQ6r6njfK2S^b?2pRTpK|29MZ}>Z8P*Y@ezEu1X8}4Z`7ibPg^EMPf>W5D5*t{sUE+ zuWA$GmMk~qoKF(Ith`cQu<$XYOp|om&f}q&wAsf##@(Cx)Swx=Y)}U*f#2bDX>+P& z78Nz$5%b_dDS4>YFjq%&$u7BQs;vks)o8<#H zHG@RM*@oY7Dv_NOUT# z9)#K>!!mz^a8|cbVn)Id>TMR@#5KQ7pbHU>U42fCQF;Z@X_O8+yB*5}Ca1 zQ5GtmmGEu9IVaDf(^*%~nT(x^I=}i$KFMA}k?E7$NaZaDA=g|A56Y3*oO1>W+EvTS zFHRDjK7+vsCR^?J;VH&uGwiCUcT>mH!RhP(8}s@NkDb$sLZrubi8o$hRmM{xnYcZA z9|Y~bqfh{(>?d)StbW@M!@j3PAD2h6IN!`1Ra~C)WmO}+z-|E@)9!x^(2FBy5YfkjC6CV`1)p zmFa&y{SW1o$L%|j%qL5k2u^A5LJB+_-uyIg(^txttaQ$FI?dI6Y#+4(nwAJ!_eGEf zG76=noo4v@KC>tn!&;xIt~h0A6%u=s3grsSP6XjvoIN9?M|#n&X8HT6HR4MAqD3z@ zZZdzUmQcm-I)Vc-wVAhcPh=?l_j*+1Bcl=~#K@H5q@5Sb4zWUj0?+Xu?W#7@YsscWrZlgLGi-wj@l-cJwp;+i#E}3B= zrT%RU{pNQlj%OS4J_eQ|^@1hNEAln;E8{)^UwN)Z&+DzwdW)t%Vu1@L*jbJ-l@9tc zm53k9s3aNoTsBJa;#aj{5eRrxBqxJrD>;)tt0f4g$v_Qi(`0W0XuO_)lRVv6Vu-#m zXOH^tWVJFheikcMq1fM94Ie60cQw;1TdBCJN074|(onTrDOh}<>_fjgw1TND{93S=_|moed7;106|36F z0;6;8y+b(qOUp{4sh5FyL^sThr5m+J5>~#>@jIiNVcT5pwE!W%JGZ}gPk9r*l0OHn zpUU$e2Pk--W{fSdN1M8?YniV9zza&@XaSZhjqM4LR*4{gCJLb34?7+*tO3vha8ywP z9qrK=EYP1aT%#qvtx#NcLK(g6U}ozi<&pXmmuylJ+nGe>JA)G$XvuUwLAOFS30pY3 z3`ozBNrnt}lu?9wH2l?6bngRVU7Ne5ATT2*{+*1fLraqIbm-zM zxD3yfm1#i9-n@~@7vLfk&U1jXk0Oq|lTle5U4`!D`~0wRQmQeqY%AOooX(B*A^JX2 zN&0B0%e=)Qpt@je>v+!T{x8rn53*zxNU6qDU~d_~HN2XcAj;J<*6=4Z!A%DPYq=qY zr$_EnENG$f@dJUnuEGT{q5X^!k76`kTO%ohQlZ=ypyU?6;l__s#^bAx4ID;}xfg(fYL%})~|d_n^1;? z%2T^P1s+*_C3Y6o0+J>)yV4Llt8s z_qrs1w5H5YIsE)nGC_iivZQ!=##c?fM46g!qpNA)rQ0B`kuwh71BHFophwXhMjpkG zu>McO(gn+UprkH44lD2a_X%}3|BwCQVbdvaV({)gjLU7$&Nnu6Umf<};8D(<-de(a z@wV9Q|3a2hcS91Ub+STmJR%IqO^)aGc+ru+NK=mGrZHEr-FHq_f&T?O|8KzfA80z6 z+`MS5d>T;U=5xsJ#<67YPL&zZXe?*`|3R+*vjUL^zotLTsXseapCo@4I1!bcO{QX7 zeoCEiQ!d)QLXHYh|5oe2{$4epM!Ki@sPI?1HU4)GHjSuz6s($Vn(eYtFzLK>`_Lj) z`gSD0=YNm*|G>}vsh9Pv3e-Q#>a3TM<11iP!1`8vP*j@VKPeSZpgW8y_7;W_(f-iaU0YaE#+XJcSNzDa_-&1PK!DwY@H|h0wZE9ff^#wQg zJNHGsYkNwZmsTj$e*GnFmFRGx0H%S*QcEH}Ly1iX8QzG0D&}MSg{s+H-Tq~Q4C>-x z*JlR}4_& z;E{XQ^f0{RCC(a&HpRR$OGJH(Vs0LP;Ej{yE@kyyHI1O1>xNqotn$e_M@P zXA1O?1RBEECqD8X?^e8ClTzQu-m%})Su=+rnW5DY zS*GD~=F&r7)3Ov=Iv#Ia?SopEGU~0-Bc~K7r{M#Y*G5xZ>9uZH1zdO5IH7cA>A)8{ z9lF?+VQ_YO>{|E zQ~&uSp-^EMg&~?_M%kk1W;SD?$~%}PYk8IN!>;-bi=8Q(6W)tGULh+Q{Bcrt1)S)Wah<5RuEAMbD-3? zJ^#_J;eXlz6lA+e{8;@T*m+_NbICrZ7)n;eM5K~UBjuu(KPJ|X0(>*FoD(VR$`|F; zs-8=ywBUJK()U)J8oMvHThJ5AnLqnkY>QssM}02cdYC_}ch3XkHwz<9W60AR=U)yf z6!xVFcRKEwwqIj!Erh-=w9tnSB^bu~O)rfhlLkX_nw&sRKcdkZ<&RPsV7-+3Cj zurIcRNtsEh)GJAYqQS_E)254!xmMNw4bZeQkt8DgTslnHHDR}vMN zhgzrj@GDdJ(|3Rfbe8n|)1aln25nZp*afY*0X`p66qg>$i2k9^cLTA*w_;U8fFv<@ zICJco)2!Bh30R4P^Z1)xJEL#HsunemTKk69Sfi1}@~aNeZXP9qV;J5d#rYumJU?B0 zE41U|m(ti$;JR0L*DzrB=azP$MbCUeiB0}7CFSq=&TJEfA)DN!VmNXAAV=X(Qb&(~ zAIL6?G=lUB9D!Vd980_O)ozXl?D)-P=Eo!&=(itt0yu)xM`XrcQc_uUKWatCyW>LV zyXN#&bIlnCo25uT*02%wIQ=Vh@lgAyY)^VLe)Q@(O+1^0*3>7nETmg*G+E+nd@OIn zldo;+J@voOzGbX0z z;`LiTi$I6DUb?f)<@ct#QGI;5(8_c8Yn1owH!mS9N{eU%IkYiyJ+yVLf)^yi3!)iv zrQ!5q&x3_UZt?7W0u=?GBnSBt|BgMN^z9829CZXQ91oZo--^t0zv0idpHmaG4+tu& zJHa6oRau3_PQHlM)}50!j8JFYsA)F9V93&s3&-zTdaSiU6gI;~QatZT{HT$Co6)A@ zEXe;4s7Pg=M&%9B6h$_pNN3gOk5qooQA& z{>=FBm^*EpGb8t6&U)9K&^RZr(F1qhBlkk@fN}4wy31nlT-UwBVO8>Q zq1aiA*z{}MM^2YDn<%?F`sBdC+0_6rWB_|>__qJ02JEk<^+L3xD*!DA4qQhDI#QlX z2)czql;N8U2O+9`5u=pleTter-l%vi8?mzzxNgI2u*?Z!PH zI5$*ueC+(V!Vg~LH*_VBxi-Qx#(gu;zZZQ-l=XdvmLH)5I;~Na`KBiOrjBkjoo}FK z_eJK)6xBqQ^Ad=zXYH}}8CtqW-RI^A0S#@wq)vL1?>mM09sZ~7|0F)`O-_E^; zY_$s*vn!rmv6#)nVx8~IlQ%Kv&U07)vbejhiUhO*FNVa(|E?}@&n|G+q0bh0&llFS znFH3=&6O+v7e9G?WlIr_0YVwUXJH}2mzN*6d!S2Bywm@QDG7Mkl`H(EiHu}uNQE3Ug& zge0>uyHr!LfrK2Rtkc@8`Z@~l^V|V+g1OG!V6}>Byd(eOWi0%aE8TeoKZ(V3m%uAO zkh+0j3~D;^Ya-TguVlP4u=JVEzcwx3uuR`X(VKQ{A4^%%`{Ttjqa}RXZ*yyi+3W$B zaO?Qg^!Z9e+IJ`;Q}r_Y7_=pBTjbO!eQ?7B_K6+&m(O*C`=J==yI9@ZgSq4A zxteN461Okx7bP|4g%|H!)`5Ih`TJOvyHXX!Hbn-!=eyxYc3v%KR=Nr9yejf-Q4VyT z?M-%7r()-eVupXYTt}JzfX`PTXbc0>=)WpuZCUOPdNOo>hkUdyGP@}9rIBebKZCL@ zcFIZNLC4|SRA+s2XMLj-$?y{8$3j}Pn+V!Rd(ST>ZwKpprev+<0h>n68eeB2H!KJ0 zzTbQ~7M?zMjNUm#ix&PRmETbURqb<6Ea*46TQ_z_HgwkUPN>wtmPCE=oEw>#<0#CU z-njz0&y}*0nno$fV%LA|-3Bk*hWl|v#gvKYnt&?lR1fbilOwW@BP@)1`;H6u{|&Eg zHz#_z9(mO>Va6^q-ze}LT>3jn>%jM!P7JlC;}~lX0ofLUdCY2Ri~cI}RC# zj{-Yr%-s$ za?n=7W|%=ZMOb)`R`>&Cq*3QXu z7Xfp@2`qtKf(-sA;g=*U2Fs1B@tF>)Z46$2F;q9f|iv5~nm$sg(? zuSsat;MtFx#jIDqY%SDXs0nGw(YVYhzemx+s$2cfmGC(XY@&oit<{vJXXg~&-Sb(x zc2%ULvhQDCmF>$OEdRt>+YKl6-CkX;;$%BydJ=xBabk5g@Up*Uhg7m-8Zd#T}lSjNZbcp+xWw5nM>odc51!jPhTdTjbo& z_r)&Cs$~(rqu3wr)=t>JO71Y$z3qm1h{_oSKf#ib$L<)|@}&{JZH-vMZIH&deQLn* za!$s6SLG0e@#eL_mKI0jC9B$HI$phKKc0{%c`>})>Dz;>((ugv%)Y1XHa?nmKLhbr z84C<^`}@LR_tiD6aFFT+MROLR>~#jb_*I=D(UJAQ!6np)e7vf|d1c;0Ma%z|Kk5Ry=P5Cu+_z~nx z?#wkGTfy)V@-YqfS)^TE!c&-jQE_5s%Z<{Z-LJ#5C4TRD8z2%U=ngYxEPJ}9ExG-y zTu3RgCn+vw(?hY1+I} zLwv_B>$@~_wD^N;nH~dGWej&dPL}&NKlH#1@PbWR#>Qx0AzFAagd9yw~obc%l zk6=kRJ~3F1$_gQuIZ02EXYc=M?{O&Bn!phrGs6eq-xVNlRpec|5*}Agx)1f9fHGZ< zCUjs*ZfKJ@<=qhGP2`Vd{}D|tUe@a-DPMwF=4Jg`Wj9-XJ^Y=}_Jg+YsWxj208o5j zU_ctB#^?$~zG1p5t6Apj{%*%(PIG9h=s!pZ#b%OAv*i2_p^=jc$h%1D!+lP8U%vJi z8q}8~$69^&vUy3_pWY-X9=`*UJvg|kZFeIq7E3%y?ATK%Pg+McHQM3x^Q8(ElMvGt1-t{>@zP%K+` z6DifbXnrx_F_nwE^3S=v7W7K_?(pP<^Dq>|S)7;= zUB;Q^9K!fB`aN6@?R9k5_rS@1cdnK$6_`WR65ZLls29B+ik zIjzRZFJI~~w%d^BASNIRJb-GTwR=dwY4f=&gKwLNr+!i^Dq56%;yU)(&`f7~e_fy5 zfesj$<_a2tREWY z@nvEUR@!Ia-Itpfd~PDG+ zh&u1LA`&qf+rAm25-Hlnye2El#V!7ANt`(VdfZL{n*9LIK2cb6?OyLCQ0CZ;Afi#w zXos$gIw(s$=-Mb_czNPdEXxQN!`sldy&Ws=?!k_+xdGfAIUL|1jmGlui*SXEC+I@H zhL96CQ&ln;K6PzZ9Tgq0LANzrjpK?jC?o2W;B?Lgyssnp7`e+lxrrDyU%H*1uVZq15eGrrV3`OkxQ?&XQGgpZNs zYg*%`>rxRZZaN-%zeT>u$Su&eLX99 zRkW(kxwHEx{cI;VpQ)IM$H1IgH#n%5q&d((z>k!cQfm~F8C!qh+s4HNJ9y2a8WzUI znL<6Iei`^=)*`VQc1IpMmu8ylMhw@~IJRw}?C=X-?m2K4JiI_1MXcu!>5{oWWza=z z|B3JW4wuUvVDm76v#5q>Hmcv-u@|D$NCPXk-YDV<(Joo*WhQ2Gu^H-_-Bm#Tr4L0^ zkY&b1JS7nDL{weAtl{qcDo6%b8l`9*Ej!@S!;1f>%C;7^B%?dj8P8{IyauXN z)<_=@&6$DmBYowE3j;OwW;NZ7a7eRJZ+cix}aweJ4n30%oaY#wm0zs z_yyGfkdfvk>Ed#k+=)Mxo9Q-}CP+^_tODq~tR%(@GPw0yQubF^lBc$|7jkGx)UDU$ zd#)L0C;=J*dL8No&DN{7HI_R5w3;w+k>5?ta|6@7uUuKY}k1H7NKa`Ec`ui2ChR-5)9 zBBKh0B6yq+J7No%371gs?N|MfdWVCqa3cPN;i?Tl7Ca%}ZO&}rdQt4z zMBVAcS*%~#Yz-N@h`ylIaVGRdKDk@?`jZy3>cxXNaF1PIK)V(_~9FWA)*y&XFQWC_n1djG;}N z>#R>Ff;@CXZ@%TGYQ{w3ZTES%d-^-6bnbkq^YwDq`b5KZo%>B~Sl_MrL zjU0$1m9NSVf?VxqQ#rZj7bp-CWc~AL958GhSf$KM4oo!38NXktth0S59nw*#ymTEv z;%3w`@@&r95`aO6@rZ#q`)4%K7Pq0nUs)Do(=Sp&dIBzq=5zE?RRkgztq;-GNdy4yk3+Z9C-8lo-(9pv3PBd=3;2(L}NXAAR)i zDqQ2g-Wq;+fs#yf{m2knQ9Zu+hNg~kyEn!@)N-jcz-M%n?ce&brH$ z!xs%hLJh>^HOY zbt<-1@O3zD+-a6G;?SX~2gX$b1N-UI&ZnV-ZK-9lP4#8=o-Jy?RDWbF@`*Uy86T=_ zdztXIRvva2f~SiN3wUDrc{FQE)`YKAPJHBwlQfget}Vfqh9p2)o?t6KPUHNQZm*9L z=)03yh<*1>9c00uWw$e{yQ$Ae4)CIe<)PyKL4FX)_5&$oLK5!xa+ezQi<^tQ{^)ez zdkh5U$4o2{w4^I#PtS9@1vD@}3??`+1c;P9 zmCB@86WNj0*DiO+^A`h_dV(`DVY`?5z z+QSnEK#sSiuqC#CrVfWO;L(+SP3Vuw&}Sj}V@?n8F%D2LRMk-fJu6ZfqIz0SO@%(= z>|LtELAF$qLk{;0Gf<(Nn&3L_`9r5MB?}Z_NfYQ;|EjCvN;TN}4HpPZtWEM)ri>{g z`*}a^w(n`2*QFo>Pk^p??Nc0$Ji==>&ZmMW2a3p?$hLi=%6wF1q5-x|NB&daK%2$C zHDwLJONWlBLEXe}>w9ln=*PgsoC41xV~{R*jRj3oF;{OoFv<=HCpt1Wcd zQlXQ>Hvx2ufuMCr5_kE@)2esM@>*xn|-7P4aOUIKRdwUEx8yoH!Ce`KVyNKlJ z-m@pZoE_rv3w@l{#{&jvi_ zZ=BMrhcJG8Y?)$Za^=8BFHk%*#^1yP3J!3mIIS5ZH+$Qo|#EOm;TK+>%TDbE67cY$7gEE$QSI)pjo`nify$3|7RHy)h$?)^SWa5-YrjEC zOViBmZsG{*@*6ObXZ72enSqw$z{w7;J)pVFP-SG=KC1~B8g<>lH2_UZ+J^BP7bB#aNk%L4C3NlE#178mcw1O z9=zS&V?Xae;u33-(f_U33wXWqw3fzn9kfrye3bOw^tG?OS7p=}@KIEu192D%gcUsl zF_WXb3szm%A`JoBFAW5U;;ReL^1r z8MNHesIw%VuLD3AmShHb8+Krh`e8lN;aO|nBb_!4&c(mmt*eY+EFHU7&6X%>TriL@= zdGjDn^P_u3rOwW8<7n^mY}Ra2sqT(f^jJ2fueWCq|#70rnWG zl;BQ`?|pW-$F1n}XZP%E?s>$?RI&4(J!l0N_>vnNf}Y6ew*m$o_7etKn7Io9VbVuk z)LYW$p>(pA%mO{oq8IEqZmwySS~~6UlgE4gQGcReL3XZTR6fUP)s?{IT8+syw+vD* z(l1eu=~0!mk9K~mRA$B(hy*?jb#0s#nTxskO`fIk`a$6qwZNTLJzorI0K^bHX(mUn z_-ShS2^7j!$mRld6DgA>R3~s0=-PAH+_7;`6gZz9y`Jw=SYPJ8L(9cB#8%7qmEC;y z!%0QM$Fxs(WVJBjY?F$&0pb`<53PUo_t$bALu z9~E7MaNLpo8CF4_1HEUqk=emBIyZHt>gktiCWOgtWh`?G!6hfs5ed?i$1x*dNh6uk z!X^-1W|%55NbtyIKJIMnFc~9O9A9cO{=Ub8I$fb*F&9Zgn&d5d54w$#n*i;R7WK!tDs z<*wGizlXe`jK5y40LAv*6EN*Ltg4XV-T4Cw3ex98X_9e5*>D;zIyfOvl|09GvlSq~ zH?-{nRyV!ZA0p#53}pRr@w6@^8@ayQjQJqIPD4{xA5$K~3@-%Sr9Ut90a!$-nMuJt!Lj zhx<-0&k|T_aR?uu7B2CN*r!u*KW;}bC?Ba0pE0xouYV{ zGc71zwokLpnT9S-rmMU|`nTjZjj?{r6sB5bdf!6@&A3kh%6}3N;BSTKcgDVK602SI zoAhgXALL$N-MJ+D(^PZp`bVQHQPedw>vR=GHQeR%GG8~YW^Dz%DUm~Kew;heabL9m z1Pd0ukAhq9NDXqR5iZi2vqb!-Bvu6-i`zejGK6p(Xqp{NzBd#&|I4?=#lLwE2;Qem zS$dDGy`Bm6A_#=ILX`xjxfxL+;1rZU-04Fb?kXG9^5uY9z0p z-iUo!`mQ3QCKzko|J_$o1nO@YPJ_kSzUj-U{1&UPb6E6Nhi>^sD~G_>uqgFkqyI=h zpiH9GX3E7#-;Ifnjvn>AE6^hwGFrINGt1`tXZ zh!Kz*Di?z2VRFFa1oFy@Uke+Gk3n#d9z=!nxn!aSYa&RHLC`>f^O7An! z=HbA(ksQ80WcRNKgJa(Y<(7mm{EY)Z*?CNO;{Dhmc|{Fq6#fx}@gXdi$a(QfzJWYI ztEZQgWDjJZ-QG)as*E^zXiahnuqRi`PTq5fZtZ^A^N;mPATQkmp8Iv_S{|yLqfqAs zaGE|4H3W7XkWmN)UN^c6go?;gNfoK_a`t}#Ch$dw>`Vn)$z9*_{)VzNYm2%(#tIFw zU0;1TwQv<2n}*^32u>Zh8d$rL9wlU%G*NHWfNs>%u3DIjkLDTM`oy9d0A6M2;h_@1 zM9W?;tgxsfNpNwajeJLD6yan2p0MxGnMqXYZE^K6mI;tE*Rs>pSX(A{lD?7|1^BYJ zkvv_}b>Fezo4mOsFU7X|Ckglz6$wW>l%%z9EL=g6+l0Zi-NlRxaK;M%R-*;>oSMqj zeQhOlcu}lWj7p%Q((mvaKQ;p;ROxuRRr={MmIO8RsdeFE-ezuicf zQ%+yr`()AE#s=tqsr>-_WS*7>aLS*d_ZR+_qy5tLmJ_R~xF?ZiYRW=hs z_(ec&{rFTGwLaEBnZ4ay2@)>UY>2@NOXP zGk+F(hhyvVeym^f+^tB#tQ!U8(xNXRy3&5KtHJuYixvY8+xDgX-EIYi{5k?laKM7r zEue0Q7}j=&i=XbJ86=Bk!Hk-ogzU~>F~cy9bLc(N@iVRV2ngNnY8kuRN4$wi8heEJ z-Jmu#PbjFE`(6CbG35RcTR8l>y8q>@fOPeu?yb{_ZZ?Lu7>rxuz_aLLzos@3c5L|M zz&54#%ECgRO0Dq3(n>+H$RP=rV|?@8apiA*i6XPXwx+{DUIy!;0*9`aH;P%w-PK_t zDAJbz#UMsb4pr7IKSfo)Zo$?N7LR&=Kc4{7joPV?h4G|^Vim7xQ6BDl&cZo9(8rj| zbH)&YXSiNTdkmC-FrMB7-$CgoP&(-aoDGYq23#1mFSjnzL5bEr0rY97vSrK82F-NaPti%keAAA{p{hLFKL&EzaxF1 zzU`g-ItKc)X1vguPFk{rv^CbrL2{fCt3-YLTm|Lm6#Iq6zQ7-eOuJ{Q9V86zS860#3(h|sJs##P|i$&$D^HkSBeniHa&U>caKvrf>r|%=m#or zDDDrd8@fR$U6yyD$vP2$IXCsLjjl@}dwq(c$o)Z#V=01tS+|MkM$W+@V2z|}O78}} zOm??F+*VB!k_GXMn^+`$O>)Q_@@QVI8}$rhHN7{ZZlk!T;7kC=Sx1ao*J$^SuZ>gF z)JU>7?zYQsj-d6wE?G;hP0sH`i&5O zhsH+P&4BUAbj-8eCNa)b?nT=p@`sPm)E*WFMpi)}0Dmg2!tSN>S?g=*Ho(LdxdOlt zcxOs29G7O;p!Uo8ss~xXdx7)8@}ubRioDks6Iz?YyV8_cI+jR@g=<)PmM)123o|Yzp~(MYkk&BXu5}Jh4{i!FnhG@Zo?fd!!1)w5a}I}0hvxeOAiew z{Ap$$!g0R9%rg91$zb5$Ab8cVQZMY${NY>n+Pi#r;su4ic7FeN(84r(Q@X4AhiTKl zd@rv=!5Z7fm>Z;{u;0z}_n0lCf!elT_EkhzBEvRsBPKLXS=%^kCupMFpY27J`Z%y$ z6WNJFcjwFR17G5A;nz#8{WM(XqM3M{gd)FadMh3mzO&X`xN*utBinJuS+gptjX&x| z^DD@h8KR`pKTS;fw;xC|S&Un>6lDojmQsWL9aC@_5z2)>82^3z_$tz&B;bM%^SwOG z51N%RSu|ESx|GGYfsb$zRGTK-Yu7s~jS%vKw2xYqd#iu;!X0vp+{cqOwttB?mc%qM+K&z3+dkAp zh51POtc1~iqcSKm3z@Vac}NJPj0(eQSIb?F03raXWX50fVyBgY5r$VtjCCE!-;GUQ zp5mVSthfavgjdkF-DsoVlw*>4P$~us4AFL5i9t0$4*B$@rov2mAYj$1$;tV#?}ll3 zQWQ&q!XH={(+6-Aqsni`moj^ebS)nCH;eelRQ7(yCTu~V!qSexJC!Xy$vK-}Tez#b z-erp6SN1$AfEC6&}T&th|96qyzYtrg)mW4suf3r4K&L| z6PcQoMNDW_|Fk^+6_lle%Qw+Ou1lEIV(frT{LjQ|YdC?D7B)er)|o7Ko=+nkh8oaJ<+|n@7a_^7|Zvx<|WtHP!ye#`w58ED!LoW z);>Ru;$nC1x(_{%=@jUBgjCF0$Jwyp%Q*VPxNkV8~B_1R$04j6LaG!E5c zRMBGb*M{8@YrZox%UZTlm@Qy@{hh%=J9$bUZP60oO(W&vxt8dXm0Ec&U-CruV?lhcu< zEm5^EhyVBpQHY@UXon65~lTsOifqyX4RoIHLRrlmhB%+ z#RTpg(l8MNTe@D|nAXcpsDDaFCz<3$x<5`;7REx0Ho|lo{86{E3Q?EJx@CBUjE+{} z^zNvj?Yp8$e2tWA>_U$@6y==;PItpj^n@y$+wSznH&pzw6O&Bzouvhh*=1K`G{O6f zi>L^Sx$y1+FSB;sHx2yo!R-XsuE(e~fNseP;9XO2fYFL7T^spzZLPV+(*>Df&=wzv zd=r?3`Xc7Xn=~c_=}7I(SOv}GwZF19NY-HH{)gJd-@7@-qvVUi06)}GimMw#`e7O` zvhdy^XJTz{K$>Nh_oL1j*lR%TkUSn?Br}x4a`>4~K9p&d6@Ekq8DVj|zl&qG4f{fq z3~JNUs~u4umw8#&7Gaxr$G?;@UbT$eq1P_7MSh#)@TYfO;^*#KM@#VIH)Tt2ckfl% zCM~u>ay%p28Y=pSrJ;OAI5Zl!Cc_Y0wu^ra-;g&`NTiqOX|E{saQ{9-@+N8>)#cXLN`RyOt z4KhHa-V-=>z4eTMyQlBACf5s73TMk+meFg*vfQ7MGL0g#ew1RjEvXL7gfrNM_)5JK zE|JODRr@{e)xLb`x5stEGLwoka!8%QYSr9I5%4-nf^F*%feg*31>Ws>CUCHDT}C%K z+;RTRaDEyUtT3fB@YeZJS3lOD~auAGHnr<(uyAo{wuhp0C{7uF#iz{~htlLWJgIJUZ={J0A zE40j|@H<=!$5E-E%(lc;R38#1m5BXbw#}yx)bu8>h3TZr(8Q$q+WrAo8XnEkLCQgu zR?I<3>I$dk>*$dn<~DLnrF^P$R3kX zyKz-0wYPaReEl&vBXwR z4J{5W{-ibfK8OX>g}!pU0^qhC8U`t*;{$e`erCwyzvpwDJ4L~ZI>}+YgVwtb#`_t- zP|nKFzUCSI5Lpf1sh%h(JBh(?xkOx4CZ*?LBr|8_k!8=n=VSb^-$jE)9!}m!iIe#} zP}MdcCwCD!kW%CwM(fD47YZj(w7R1ORqXhC@ISm+f6hwssku+rvt>^?zAOLk(@7F7 zWEwHz_FnoBezQk;Hd3i>R@3-*6CDKijTv(bkIWN4C|VxrA>5(eEB~Hhx7sN}eAMTa zm+R$50P92$H{Nj1GfyjPesgts0ijxhf)kn2id#5zKQY$}DU<;Ujm((0^`8RAUX;+{ zLa%tQdy}SQ)Pw0qCV`m_L$a}y9=1Lgx0jPiE z9cGTV`7FU2$maR(DKN+TkNZ63V*>K6N0n+jjmkDus~+@8!(P@Ufk8SNr*W-Q!-JU_ z_bE~6JNlAWvtudh^K!6%$E>qdBH)ixW0zCN~uCc%np2ifH0x>t+UV$CL?-LAM)`z-YZ9a6K8`50S)95J@jY$n}! zBGW5N#w33>4ff0x$De<2`kUo=5nqkZGHy?UIl7Q$#`vWNka+Vf8Fv?)^`>4XY_K34 zJF4xHK$~$OW^g*mZ7T;WreKfAZRhG5LeU`DFHw}diY==cw?;-kG18CRED137g)?*~ zy5u0TV9!UqHP}}t)upCN2Eih0jxdShaH*h8=){8q@xxC7`)kz>lgkw_4wr=JjEQWQ zHIpnEs!{WW%NP;L{)I*KK#cBYmA%u)cePuKokq`qndFG9`Ocm;=aKpd^17s|4JJp% z!J?I24>!$5I2bg2$32fiDv)B0GX|SvzTSW1L{`I`A=>7@h#ft%)CA^O;yD7O<{Y zmtni7p@=JayEZcBl;pjjT$oSH?t5+pazA`1Z#83@3%qQN{_^PLo2vZ3SumrKLyDX5 zp>^VlQT1XerDAW?F;feY&`H7`32&YQ|loq;d!lgh?7iGA?X>t}-kEwsFQ z0!ay)pVTosNz(Cid`_9q9D6u&oEz0^Y~LVK4c_UjJ56xgv_KKpT)6SqI5cqs+@%UN zQ#7mAR`||CJ3vv=D1d&%>yM7it~R2@;GG=;aPZ`_EbdCtG8P!a>)I{?vqDZM?gu-x z+DQGTv{T7khyoW|E{vgkrr`ZRn{YuHN!1nTG5JeR@pd0KjbYs&b3bv(X~KQ1iMQ+v zsH_3ShWbj*m09!f@Vs9>;p$Gzd=tmXE8od*ZkFA%YY8NiD(|W#DHxZsjk*#$a?m9VEt4&i z>6(yM*C|L;|D8yzI`-hiM|CY8McP!UmzgHLvt0R(MgHr}Eou3&SK>-c7&Nom$2$CP z_#nzu;eD-)g&cfoW++=;wmW;&p*xR?XEC&QD9siTyitO=^I+Viekq#@74ApJeQ@?&Lfg(Rq`W9aq_opcNS3MeG+ejG@yPF(%C(j7JSVm@BrG&%e()@5YEV>N6)NfOwkmB3 z&le!viFWbp8%+DMZYoz#uwKDI~Ypb@-;}=|A7AC?6gx(sN`B1P;*RWKi``< zpE_XFI9ZRtf0`M0Uu2)Es5ym>`<}HO5KoTN&TN(+NBB66a^eFN`=8n<6{#$LifkPS z>{Luhi+n-)3aOXZuv`gt26&2{q9^oL+CjTd*_}7255_Bqi`sI`=flQh?r!>aie=r8 zorGmwZ^Kh`Y&LQcK6UFLP`B$gg^+tOAH zI^pfZnFW48--9T%79{HO@r>jNFP|VU7f-*Tm4Yq&!t-^K)E1Z znLUAe^>kKpgvgS0LPi?IJTAj6Ot+@!Ym;et4x#0V9cy)Eefn|V&&s0~zK-WmcuD?5 za9W}dduv>ux^Shx6n=y$ZpmyFv`0XcyL}dby{79+g}|TJj{?LSS$q;2)&;t8b0yPE z-%I!$pMlOj+pVLgXK8sn)GP;=3K?rj0gsn18C2Xze{p52R8HWafjL|sYMVN7y@EB> zc&Haj@HGWeTEf@U#;B6o!mEd}xp{7DdT*-r?b#r&kDe-7z*wg2ZaIH$0+GgVU@;{P zbTG_q0+6g~L=Kc6^Em}<4e;;mz%@U>{93EFmSL;|E>*V3iV~ zsm`P1X(FKy;P}Sa$f~S#`tx6m!^`mX8}g)XrmYeNfF9|E&tbMVr%|}vdx%dP+nYR_ ztrwOn3qyxjBUQ7HhH&ak9fO${5a+fw1t=H>Cu3*A;yog^UCa3cC1uSfskJ-1!^8QRAOG?nwp z2p=;AkI{*FKJC!-a5G#eNrg{Ryq4xp`KI zS?@-~QHqXoNm@`#DA=0M^J(37rlPgt3LdH6RCV^)?g#bMor$0AC%S8LtI|nfFC3eg zy~kBV1-h4Yvm4`MeXy+?r6M}$f_ zEU502%ioh%c3vxf&LAFCk%DDLA_FLk&wosO&CqnPy4B*)DbWV8kw@HpW_-|`vwSwP zsh`Jdj|Ll|OUMy+hXHK9F~iQV6P4u^b}k);=U`4{cAQ7!FIl*>1VD-D{-x;PKnWEk zRf+kKxsCZ1@w;eshQ*Ly?2E5_LM6w^Zv34U&2}Adpx*TlkzF!W{ofteCUBT5XJsLo zup=3aTU5UhpuSDh>S@!}A0}k&b7=b$(%#dp{%=txeovqL>2T|5%-&BV)pR_-i28|v z%%5R*hN8eHuMFMVQu&DWYPMHut_<|i9Y<-3J=&(r6ZXL`ylcHmpz&Obz9jv>7QnbL zsZCj52{#H7J@w}UCAm=|r;S46HYHirD;!6Ba;K0C+MPZf9i=J0u!Z|?D&(_H2L@Fn z`$vpZ(DKgpd(>K>#UmeK%ymo6G=tktO?alx6CnvpVyb2LaX&6n^q z)*)_rC7mRwBv!JJqluJyx(EYo-UjQ+d6&@75boq`2^h-VWxivSb@(ojyH)&fxOc~U zVA(#)OiHV^OjpK_4qbCwzA48e5*((y z$sh4fM?C3aou=~xpkgC?LcgUoAzM6;l8ceOJU`fe$|Qyyd1611sg86(Lm3<(;T1}rSBxZmxX#%vK75Eps7eo*NRM1WZvu-=vB13)aaX()wqSM z&X#@HJ=+~%KO6$6(0W+~-c!yq9*?T)Q=3>K^W*Oyrr9=Sos6Lh6WzY6|Hj{!42YSL zK?NJ*$su9WE!4j%_vZy1<83ta>RR|PZTeL!TVrvhHHSJHG11bu)f`AF6(b%Ia^ZBu zWPuP`dpYYhoyy$JO4g09^bTQ{{^>r!aN;S}CBN(04yv8KDR&4-qI z3cB2WI{RMhF<(@nBnC?BjWZwOD%jk`_oykxE72Qh$m z&s&dlAPWCzY(31|x@tLur6`MLylQ$G_bK+8n|t^f>Fak$M^}H&isxJ#rt)p4c5Tzl z2RkR%H>#GeEtSy*%LJxggma)uV+~o>jlQXPnS_zG@or!ZME{M^fN0D$fF?3SDg8P~ z5|neMS}#lXq0!;5oU~Zh2%~_}t()>DRsk+M|11dySVz0q(zuoK6h@c6>Bg+iueUwO znf)3h&mffV(}QTfp^;b^)gGk5`~yw;Ixwg)?;eX@p1s!OOHnMd#Z#<>-z<*qXga_+ zoS(R@+&8X%mK0!}b`shfERqfLF5CpaNaG4l=T+o&7aK9m471S?1xb z(_k@QF`v6}r#s^N@jFrPGVUvu>F6!mEL13>W2BSVf5q0j6n#Gt4bLWaycxx&{|6i- zUIK{slnEj-x&5B|uXebF*$Mo~0ov5TaIf#X4u)L>b*T85+ZnW57D2uQKsA{->|}csD+W-K}<8I|~+%yBc|JFr6Cxji;ma39z)Cg|U@ce+`A61J(TiufPe><(qy=K#U_jKjPBHXmq0PRj%3 ze67jsW3DOFlq^jl+-I33gkJ|~k1l6Dkq1_^x)eTku-`2!`?%Mw@P!;xX ze&zg^#<1-__)Y8OM+Yh+)~RyI`A^aHj-_Xx*7`FPD4T*(-cwpeSM15fYX4(pL@6yn zOybM1T)X<3`VY(%B3BDxhUA}@B6|i#daM9#C&iyc?oEiJbmTf9dUl3Elg@ zMt4&G-K5^P<@R+}B zG|UM<*HVpRs|{dRbOjvt^@n)ct)H4u&iqQkAFVRt+fmjga}e*18jIjP-qkvb1XM2dUFc@q@~E6=aF+ zC)k;_gfzbG>4DQoDuvDCl;M|2xyHK*qq&(yv<3_Ue<{ibcygyj3LrJ*wQGMGCdIgH z=sLB{Hsv!>;4o_9HuXe@dDOt?$bLFIi_tR4dBh%PoyD}jUpSE;XXr9c9sA7E8*~4< z;p5i93Vpj8=kmo^{XitV(z?9%WW&2@uT^)fJkJPIODJvXj;r&KJ|GPSO7zI@-b?qPKm9{}3dAWpcmLu-d^}MExKIX@eY@7wur)xMei3LRO%7 zcXKF2&-gl0cQ0UC8VhNr;< zI1sy^;8;=SPOR~|?xES+Zqungg>~+(vRh+F9Fi+*Z+B(tHn7YnHOC#7P3Dqj7jPqw zbGkUPM5=7#-6bvhfpM=4Mb2`MXGIY(>o>#WOTCq7onGzEdb&I#(bUI6b?WGOhljoo zyHD7&+Agyj*f&=4A67tz`CusYvVmm`@*!}tEZ4KIL&Pm#1dg1ZrV|(i+Yx+tljI*Q zWr++=7DWQlC=88$D2Hoad5+Co=|Bew_}(aS`XUFlK_k#M9ibTfiTtu`iygX@GB(1v{hDteruHa< zG{)O7KoiQd@z064tI*nvglM$}1tQYp@ykN{csNrPO?4P}?+rHvup50CyxUX;IGDRX z>Tq`GesOI1IsdTlbO!G(kvXC1 zIZb7n3glTxl9#P^RZx)i_Wq!k;2b8jvO8n4IzAo#(hokd^%JJp_@iQjTkoN|FkdxG2EBhtXTbJmUbut|{+qglTeK9Of`SKBa znFOu=<_Wl5#(e72WdmjwD~*=EKe*C==sr&TZW?iiP3-UZ$V>k-TG@s$Jp5(==y-C9 z>%i1N9uTE-y`X!|?{OwDU%SzU7^Ts{FB7h+0lV2p8WU9;ysS*pvP9ItYIaE)U(}S{ zs<3Bjrs$frd$ev0Q|s$f*P<_rZo%i&hh|)+72#+N(SFVFAD6`WCn%%m<-6a1Tv&pg zvwEf+bQ~hi#fC)kzIZ|IBehB_osEG8FKj$`3~Bz~7o8pr68VL8D!9ihuHK8@NdYZ} zp6VhNO{-hGSQ0i~7i+HF{E&C4lGeU-XQR}a-s%5&bw$T>aS z2f>tWUIJ8JA`9K4tD%!SzxWHfr!-R|ykCt7@*D4I40~N4Hbi&!{#eZ=MGKOCzPNe2 z^Mq|~|3uW^{C*$#9CvKq=~}&bzmLbSv-6Ls3uW)Q9q36qyy4|Eq$St?>tJ&8E|Fzn zn^pYcJx`vv2-w`~$4o>i?VdxKiKi}Bd*ZwI3+&Drw!F&1ms<*)z7&O)u;JIS@|wP# z)|(UBT%9wju@#wS3-LK5wk&mxcs5HEx~L-a&6cCOhnh{ixGAbHYI*>^xfs@YA2J#X z?bICj5@$2u>cO-7vRHdRTEq%!#u6 zzimlacN!>L8uaECQPy$`Au#*ocVzS!DCg8P9w1-%1{>-7upWfC*}7k?+r>@JW9EHs zoUb(^I#oB?EfdWug3l6k=qkFRRfVv9@r}l8Vv>3%f*xCX+`YwHpBc`7YKEh+5uKtB zL0tJCwGCIu?FSRD6p9JbJq@IpA*meOor!^BdwtZtqNGKPnvcE(8$3j!)^3$zEmyls zo8s`6lCgpiEuDcD$%$oiE9_y=k&mt}$bxg-Vj^>)A{4s8&+Pfs=8vK0yWYDAdBPE0 z(i=u~FFqC89N7@5C@HOVHd8XlJ|C`r!tLdzsL3}ZlXc1R0OwX;tV-N?$CJF`sVYWm$sEGIyvayFG1zAr_BcDK-@G+X0qL+Pf3GPtHu{sP2ZDNU&gFN z7u+k^cEBuct;-y9UkU{TsskCF5_FicQ!e@<$MtlI_HM1NUac3QPqws(Sn_ z*FC$PKl$fm%=w>1Ox+nd!S^DPbE7ac$8#5Yh{h)!+;7bAt0mjq{TYgS%S(Zsf|w3! zHoc{esm^>U^e})NG^v11o&?Suw#;34MnQD)v((7Fx$mM=_DxKzr`5YXIr25l`qj<+ z;YB_dxoX2X6ol+NTl}in=1I?{f%))=)S4pk?u;)v%l8lY@B(kqvSf?-F5D%tF^q|5 zn?F0KJEuQMjj1GIDn)BZ10C3H$GYPwJGa9okP!^`zQ)Sf=V;Lxs;TOP$xyRs;f&{O zTFJjQP$pG9#}q>+`yBK`aY=b5idts%+?h(8zm9?))bT1VnJQY8Ux>fi*qxOsFZ_1} z-6zjuTbRZ@%IwTT4_n&+)GzP1GCI$8OE6I`4sRmevys zhX))tTLH$DaMn4OS7`0tuJ)GWi)cd;Qgj#Z5NB2Beq+d;{~>#^)?Cc=JIVxTshU~ZMS~t z8#;rox>UyJzcTOAu`e#5<|ltT_SWT_p>MI2XYr@1s$7j&9gT7SOTo{GGCyB^O0z8j zcU;LD(c;exaAsTfk9Wa0i}`F1n6uqfxO6pY$~?}n>>c>0T4&%oWLE-$z+bQvQXQX7 zRvkm0YljZ1=zh}}V89TLvndu;*nwNb!Ef(B#xt9z({}h}%2`HP6;`yK>cOnGuUQ7WuxWimZc)&RX7YH7Q#kfby|370=V>YaH22yYkH@S10l z>Y!23vb{xT(JTave&w|%KYt@)jDqm!1sXm;9>1ErfcAhbFQ_yq_EBBQJGSlU!#vd8 zke@upi*c6^`9UH+8>>5b(D`9|WeHwnW=BM-mhzAD)wEdHx{#27=242enZh`!l2#iX`@0haoQdK9FIpIHVf)I-#uv>)wme*ir$7+Xc(C%=ZXvJa7zI%XfRn_z|4PYgSKnq>4kbW>}KA2*qUtbfu2voACCdUjB%XEZ8b?ekfO-pcZ-ZlOgFWjZVX7 zCt7MxoOpF3b?w)_@W~6e?9s#HR0_^SktY)Pah^=juN8Xpga{l^DjK? zpSfPcHq#sBFV3doS}&}#ycc11MP4JtNu{qIE=sZR!#}fBvOI{czB2Rc^x``!jJ^8n%sDf1z3>J3B*bpw62iiUeN3V1d08_AU5+z6WJ zN}1lv7#JRrw2NXY!%7pu)|Y$rRB9a(D#Mm%&QE7%?omq}F6Fs;WVzxFt9y^D6O2)2 z&P>{X+)qdIeG`Wik&6_+FX|L*ikXLOy7wY;RZ}Z2Yb>$uQJ5$nyWxYaqYKz+xW$q& z$mQxMu}PP80lWSfQH7<)5!t3TpS*QZ$v_Hah5oRb(`0aXp)bNAZ-$SGA;-nQ#d?!3 zi8Oh+<0Q-}4m0%txu~b%&@u8v3|YfivU7NWL%qgdur)UQZMhkmfU1Y#&carmSU0?9 z$$C(5bdiOKW5+wJI+_B~z0kM9(Ac5^Oo z97dPuF0wCTOSGLqMtQ1xnliIFS_0l$Ufk&(E4SQr<&1RqsawWMHQeOm4q958cnchk z1)Ey_4`KHi)l~2;3LGgSA|N6l9YJ~%5J+eN=>kDSKtPJpC876@D7{D*LXqBk?3?Lwuh?R2PSD!xl+bRcJlwZYxC~214cSyanj|G<8Jt*FxdDE=-N&{2WK%g zO3bY$bzf~gH5_Fyo$xka^7Kd!Ec4nmjoMj|+pl$)4-AgfZ9@^zm3t0R(!mpbqCJ9q zzfX(T*oe6+d)0{tx$2+m-*7N@{hXwbALB9)!~!4Kk}1={x0b3;I0K9OCsacmGXnR3W#mRPOE|^r-8TE^7kz zR+ru);d7S_!ep-^r1ud)|EMp9|NP5ZFTr>Be176`sAmm1W|1n%zn^{k2uoWUzkJ$b z68V`k$m@_|?5!xDoCBD|y!tIG&nTxNgfo0~^`Bt3qf*=Es$V|1e6zSr(zPn|H2C-7 z-tz6<={F;((5EuQQK&R5DAHcojujZ%8NwzUIq8}`G1}}QnSjXqnD?Tu<@649=d$+O zj_BGQI7$0flSjNQBGu#!^o}1Vf43B(9Vp~2$mA!{;@DDgqU7UvNN%VZHkZw?&hoAZ z@F2LVq2ScIxmU@j@L=w22VRaTDldX!^6W5qw5Jb@Yx0r*Va9yqbk>Uv|B;N?<6Aq# zhVskKe@2WBaHUVf*Ws4r-_c8X57c`P9NOKT+11k1P+2-g&RkWHT~%ZP@I=jHLI(b+ zex+ln(B*>je;{D+8ldE2&Op?^^IxolyB26H)9sk!eRNs-r_*usFJkvw-mb20Y4~~8 zp0LoZBoV;h(htiyw2@%JMq?dT;Gq$Fqk6idLnffoIZw&g6f=s*MJ z--TIK@?BRF-Ah+q{|C{PA!dG>YlTyk%xO5m&WZ!es#VO74EHEuZ|S*UQ?cfUy?)T@NY!dR7Nnfe3(&u*$BwpPQ6(`O4ko}u{dWOe&cvg$IBwRr^CK6xn<-#dqE=L zITjvNe$`Y?7j|MF$8g!7Lfwv#Xfu6z7G7Q3Kiw)~=Mbe0WAa3ZzveyNrrw!a!RD+U zk8@13ndo_FuZM!F`}6rVPdB+AU(pGzL))lN3+)U`+e>Ck;!1cd#*|pALg-g9d~fUq z^3=9N={pg=A3Fyc_Vu<*TW(uE;39lALKTFfY-=~-sA$Vn4DOfnC0)fcfmAb;;uvn@ zH7?_u%Cx9_%fkDI|3C+HKMUV-jw~g}#eeCX@fqtsS!4wztkKaqRw(39 zC34?X3G^jU3&K3t;pBj0`?Tk_L3ytj3#w@p1F(a{a2x3*g&s~_`aoFzDs;8C4xlF1 zBkWZYAt+6DXe63JSr2Fjzoz{34Xigf*|PRXf@_@bdv66GgvWQl@3;q!s-Tc3XNRa~ zeTWm|XBD~+a^|nA=*k?~Wlyb9s!?>-MejYux@CG_2t)X8Be78C8x3c68oyO)-&Kpn z--3Dat3~;dE2%{`Mk%)`Ref9&{asSCKo9JWu|3=Dv3J8F`L)ows&i;-zm@464Xr8t zR@Re4KmrG73rlZ1aUaZ$=5Z{$Q#?N*Sw19z-Mk||xDbxc81;X%d=yyje@&$%o^}+@ zkFSgWE5v|2Q^qf*v4GW|6k`f|sla4}<}l}pk+^-z@rUSUehoG7PeUxz{0&XMY0#LT zd(e>>y*^>%^v*-wFPr`|BQtbWoaWN58t_4pdzOXp(n^S^^_8){RR<4v zE`=;V5(jc5U(}bc8miXGkwz8*4(e<}dA&`Qr`vdI8rSBHcRR*V!w2I!SGWSWm#zP)X}xz!+7lT=AGEY7AnTDY!% zRdN=Z(S+$bx#+svZa6tzT^wA16nqa4ZYd$e9{R9rupVvXT31 zU05X-D@5lp>N;^L8U%N|Jc}aaG|i?{M9Y@K8Cw zrzuJYh*Mx1>_c30Z6)Ai2X%*K2$glahr>5(&$*&GY_1%CeDvT9prKn*2EHTrkKQ5o zKk0s3EGE$kI^t{@d{!KkN_C<0{eU!~&+VdQ1XGgPLaLputur+8sX#!8(tHJd!YaMS zx_$n}7yG2|ZSi!*w!WFuLdu+64}osv#SN}{yHL_CRv{mgpoVf)2eIN@^2$q>_) z`E#wK{Pl03n2O4d=oFwHPYq4VK`4 zGezZ9)apE*a`UwUH+f)kWivK$rT0JVtn04Vs(ozdE9lOxl-lLxLyJ5-F1sVp|BCOp zEEI8=>Z8FPxqb}|5qwTnKzVdrf%e$jwp?T~Ly|E)m$n_Non|wOlfo)Dp7&iDz`RJU zujj)Hx~dKsidM5~`)R34Qn*%YztQB!Ie(pfg+?zMnfHZ)IHR<3yWx%<)gese6QLd4 zgcjOVS&4^9;@(QNkP@4mBw%we;^u$Ho8O1JgXN=LJ~ErI)36Thk#C>h>OdAUR9 zuuT~`b*6bSH)69M6}&**qd6r(zMOUIFO4o0T_&ZB0lVGEgDW#OCRiyomVe+$D0B2h zLs<-#!IVQ^c4JAdgzu9t%Ags=S3_?ntk0gkFq|y6+FN5Ta#i&4_o0vtS90<{ZZlRT z8+hJrVaEAUvlojFjhkfDKL6o4D25)NkJV-y-MUR^jQR?FAOT!CVz=?YRQ%Zvy2X@Zk#Z?l&Kl? z(UQ*p(FqkYBMa+vaQYaOUurZF4F#UgVfP#C2G7Jgnv7;mUL7Vq8rk?&8RbMChM}u* zSznSAz`R;^&a-K|vVrE8_usRtRoXZ#>JKVI*$L%oYy!@T0Bk&dXqDw;CDbdUce)qz z2Rlti)yu~-);9(*miAj|m*Ht5wzKMp`MHo`OG$l+Q7Jp4pPeWsiBD=~ZMD*Z>9gvh zlIBBxyX1y-n<6@;%S$Ix%hlV>B9V~Feo01QJN?0{&C82)w^Wf>~XF9JI$b*b!ykg z8lw3}Dp<&VyHAGZqP9#(nR>eK$_T$o^ZvqrP1K}u2J0>NCx6MKfLE<9auSaZO)QV5 z{4o|S<^95ut=|8+C{`IH9)k}}Gsn}rYJj{0_ZJf>p)}T6JXO9;K8qEQicvIYt$L2} zXOv&!A_STD-*4&%#@MTjXuk?9c(wf233vDDNBnPbXY%6sVrcAoFu3VSA#)b)3xS5r z-;6H=Gy}oS)B#wMKsMf9uf%q=wUDA^)Ip!j82`5oZa<3yWBV@okqk`AjCPi7{E&aFCVRU3w9c0x0fX?GnO8?-a8-xir-CmY5Yo`&9220;7|oD(~Ls(ZFYK8#Q?6(;u= z;8~{iGz(4Phrz-I7(-_GqM9)Dvs?pF-;HA;LLKAS*MJD!S1GR$gFacMGjy2eET@Lc zr-sFn-Z0%8tK8OYeYPB{aDAzH$#OUtt;Hrw0hcbo*8;h?^L>MKB_~6YIuL{3hoSw zzEry@Zcxl)nZu6ga>~C32kH{JdMGf>!Y!nz&6>ym~gmX{z)DgQ(dqV?w44gW9I{56raj_z+Ksz70CUw+g02?bXq zm?R4cEAM$Zy4(3sV|ZNkGO#z!*nL zDTUzy8BFICSG|^(CjN#?x@U@XxT30Ye4LhOz7sfJ)|q)@;=W4`q)dQclhpi3U*qsA zSvl{_&K+uN{tHW``~v`uQxOuq)R$Jwd2cLw?_9{6ki+o_%~!Fz&Ych7Lc3;oUE2P@ z>w)aanp?8^;Y9TJE{Z1G?Q6-V>ej?LZd>yL!J93XO zOHC+R<_<{22U`u>#CLh9e|tn9LUPvHCQsKN$Qt`P#f~i$|45yKTr8{N8XbOnoDJ)F zWA?ChbQx^uGtgI9#tB$4h_g@iWv5k&zjvM39@Y{F!rYgqLe)dThI8wFM%nOgITjrS zfd9%=uta)K%l%;6KyQIDu5;|%+xzTI`M3)UI5+b(u=OlW*fbMzW^-4tOeC4cg{@O^ z7Mgc2%UmHIWtl(GoqMEU*{4yXc{w7o?KUP~s*(-rTGgHB;LhRw%Hy*UhN$1ZpFSf% z&H+ibGYpLyKX-rn+^nRS!%2BP<*cCR1GyorprRig0qTBs+n@KVPN^mgh0_$ACmNQe zEs*AlIe200>~A+1`Z0fC_pL+L)In^bx?H69SjMjA<+s<779F_Q=gP->IEUrwR5eK? zkNen;6%G2-;-47sJ`7Oj93G&h31JeNAoVam!FF?>lPKTNvJM#MgC^UkT)(*&{Yo!> z`92xaq31rOb1KH1na%HeoNrkI8pZgJ9-*7h@Rkv#I~&&Jjmq-*sHpfMZ8Mr*gv>rM z-dm--IdK`ak3MQTI&0sB{31Xr)G*E8pCQ4pL>ih3oHAWymK%)1#}yd5H>+l|@@5rI z2n-2U$9(2p$rm8fFF_ag(36&oPjYwWOWUq~I*=&AMkDES_=ur;LSHWB0{6`92Jr$` zo*|h}Q<@d}Q0=`Sw3d$7J8A1}KG->}3VremjlVA?_(3;cc2{_82#$YeyCk6O9iJ+Q z5&zM;NdI0B!8Axg8z(7Stxg=56R9itzy4&%4DXLNA5R?cX0L*keu9_OMzEgvA3Ut) z@`su)&j?nj$(PujZK@O0sjk^Mr?6lK2R`S{+|L9aFPfJ7M z8vE#QvleGK#_j zBAg6Ve2}(RZq<9;v94)Za2~jo(njUExnpg0QO7n!- z2baU85!tbpqt@nolo8^d@Eo%$B42g7jhC+iY%C$A-;lk`fbJBpihi{+c&WF(WnBKs zaddk^CX}-PJ+HeLTOVgLS7G&uZFxj-tQ&MOTLV6NJO^du;6TQAeRBSD|9tJ(vpD1K z@XHjbjOozpRN*uy(T;%qDaZ23i22W{9}c@MCLBzq9arBY(YZWS+ud2}wQ#1J9w{tA z5SOZlNn;#UulsuP6)|EnMs+HB=e*#DFW0kHL6p&`IAW}0v|~XC@jTTS4}=?SbV-j% z{Q|oZTZt+CQjkW1L!T54CqJ>SxXhTHd0|ukUCJv51SE>|aW`7r_=dNQ_(n*>bNUG# z>{KAn4JT1KY@vlxs0Tx0XI_>=ozg9|q8<@4D#hp~LRx^5HJv-{kRs`B*8GgKbxP|x zLPifWt*^^20eUyeXBX^t;#S^cQZn#Cj&5EcxR&F1@r}e(F#l!N)|8TX_A;OpV6sKx zL*N`v!b{G1nkANQ|P0%e%?3^p#HZ|nC zxvA%p+jh7xKD+NTC#LOdWONwVxdpw+_5&&>-aCQ8xl2`NL^{&#qc(pqJFop^Q#yWB zAqc6_c;wzyC9g4!?B#^793Vxi<}?^dwH62Zg~dB;aS^Gl{Xf^e1!DGH7|pe2d4jd2 zCq6G0G3xH9-i6X4nGL9<;z9R=IZGE`xj8aA?pJPfr1Z%ClnX$)OamZ}hw%iPUN-Me zNF{4_s|Rbl-yW74Gd!&O;iERr5+{jFsT=GkTgq%rUq}e9E4*5)sCB|&)XOyvVBZaf zXjzs)g=sV&2f{Z`udIpWqo_Vo8Tis0wfV5U>By5c`!y6CS=qI~a>)(~H>-h~0L`jm zXq^Jd=nP=QFdx{bxI&_(ZeOZjNH0+;m}u_yw-MRT6gJf_@7LgMtA7ieuvn%;m;OkK zkF1}N2){41LQhbd>RlX|+DkwhOlgVV$iSmXyZ-nxug$!S3qh+l7eD*nz+V}z2o}2O zlEujhe_HAPSo5pyg{U``K%3T@Svua2YN68)1%rX;Y!a%J&^MK`KM6(SvR*0>yuih4 zn^4QY>*is+uA#Vr(5{3&atcdPW`!bFSWK4}s*-E%2NV;s|BSG_SvI%;T2lhEji)cex}R*3E^;4&wk(_JIV{pvmA+w-C5ruUDz27 z*}DHS(J5^BGC?q){_q|7K!&7KzrVc9R4~vlUQ0j%-mjbKl;9DL3ymd5z}N z>fU4-4`@N%ygPq+ijDy*izVYPwx*;L5it1$B-1s2FNP_Y%j$9}iBt$h$q23Rme4fI z>m&fVD$2|4YC!nJ=_68Ecks8C6FF`a8D$I zJo!zOf7x6}iq{N()9c&R9yHrEXR+N8=>9P>Bp@4_dO|-c+C-CW?NpZeJnK9CoL#Ge zVs}xm^4K(KvV6URQ1yiUc=|^BsB_oWWtc6GebL)b0^2$lJ>6w{^VO*hg5?+0vhrymturC{m+=tA6``7hL@Qw zn5lQlDvy*d_V&4vGdO1E^J2Kd_r*ITJxcKW2!n$W;43bVsEI7R=?>EDAuW$99`VqwA;hY04&h^!^goY68vD zPeSeuTfd3OLqAg)W*xQloX#iM=~7wmEBxbO{<#K~WZSvglW7rWd4vS(kdndFh)v61de{zh1lTz}$6pFc50s0K?jI?_0{*tXfwqu+t}{zNHtq_!&!)PYiT;)pZY0zVG3IaNQ0LS9R8< zT18aFFg5OIkUpS8MbGTBH`c|MHXqGc;By5iE}DeBkPKLwG~y3_(q}C z1$~OIh)HybDZ{y83gs;NW|@Se9rHiiQ}k}_!$SY8z8WJ%#~)El$#R@(T}Hiic{cy? zLPm(AH1aD+Rz_DENqG}+xc@Fi&pO(L>lAF*>2!>>TmLq`c+$VaRLUDq#K%>{ z)(B6;HzxQV5W^db{|u4pW^`xHR8vdf`A1gS=w4HQ%%4j&KUsQhcUm}Iawlc0T>YH7 z0&2L4+wK#Y&|Qs-JZX2h9PzNjSz(t$bQF^}%sM8-h=gY^ zWzm!9PG-|rO$Fuj{D04^HP(Ly_HgNq*tlT?ND(vdT+=CLc5Ixxq8TV@o_S@Dpjw8Y z7s`g11mP4@@LgyM0ZTS;bQPvGWM`y0QNAs#)BC%@p#gto+fP*I)z1P@6s=Q!Cx=Ij zCZD}hvd*x|awHF7n+K$bsh7y?y0Pc(XHwy-hI>^a$f{Y>Q6)1QpQlZz%PqespB*Bf zi&)2y-b-!`bd^Vs?02uGL=UTvJ>U%LT06#@c(!GEE)O}}h${~&hFAK`9PKl(UZ{6% zl#{YG|He1hQ6vRX$Gw8P7sCrQjPv4WJHBb|vjQ}`;MN`&i#O?}K5u+By_{aa9vi+% zx4{+{_%~<0tB`f|4*oM8=|D?6ojKm=)^Ev&!?EG`og%%&VX!(qw_0ktvTQCla@TqK zb{DEj=<}lnp>cCS^aLp~nCS0JY3w4v1tR}>W$M5v zCDM-Nt4<$WwshdNMoy#^%Q^d#Dzq?HGJR_pepSb?A_uWq+!MnP=^0-8m6^B7)z+)b zfp!R-IiXp3fFNN{6hkqE$@WGCE<$J=peU0{u=<<=lFl=ZiY7@^j+WoA*$blOct{K0 zZK1N5XjTT7vVMNQQO9^hnX-R0ajTz&VF%yevX1k*c`g*is=yRe9oei8pi(B?yQWld)Ftn ze-oV`N-9I!g2LhWHS|5wv-xFlFY#@GM0oK+b1cQm~ts7rF#sO8_3{@J}Lh6*Q zPFXx#hdIYY*2pK6QK5re<-D{Y(&5&B@I9+7MmJR`t z&lquy^Y6JmyT_70Dbkj=@8Fp(%-`e5lffF8W`d_+;q=RL!K*gK$$w&TQ{L=XW!KrL z7+j&{d$3Cbtt!0`OB5=(^YJj~IDBc`fmFsxv+XKbbki>;(#bYvp)Z7$b?lQzt^|60 zg1Q_MXWkasO+?CqXz?7H{yMpH@^H7x8>Ur^Sx+Shq*A_*pd0Se%*B!>s0OUb7%ju0l?!$DI9v-$i1hnAgZ4c6U?#}`o?5fp)!q3l@$f@ znX!lo78T!gUH%Xu^CBFTpNTqx6AfqgOOQB8J90BjgW$KPo;8ddGkZa3D|60R2#t!?Hp(wD2UTSd?Nme!04ft$~AQN`n zsE9NKeuM%}`fcqk4AUl+K<|R~5lTK6t1Dfs9p!DGs{x|3FMma&BjK;M!mnD`<#lH? z0o!kAj)~-5)2D16sn=42d{Gy}Ec7?sg5?&0vso%NLjja_#d7b`lWQ567*`11EEnRvdkO~m*-)NAb02-7_)ew!{Bs~k|b8%-5)9|Z4 z-?WYKV*!kD-?2iV*$-?UO9PpGyhgnbV7d^p%NOk|BbUO7J@k_lRWl)HHrwU zfq8-cKTa;c5GS;x(d=rOTYc%&Ak) z5|69uHxEzhQ=`gS4+7CZQ3|SRQBn*GZowHDL4kOFOqf6(V5pJ6@&x`_hn7uLPdjR-RI}z0V zT6Qve=rJ;TLJkJejA2;>u=H^)TZE+o>QSt~!e~U-A5ctqps>&8bN^@8I; zb&Feg;Yqx&AUc4*0NDuvQ4urlCb@5~F$#p|N#0@}`grSIg!kr1p#3MJCI+_(;5LyqnmKJ-$SH2RcZ9fe=D*j`lNOJp zd%lSRo`pakCj{I#Vxe%Um$B{Z%y9kiY`pJKH5xroY1;GUFOA;JZl#jwn5dVTY`0OL z2qWMY6;!}P7FSPz;fyPv{x|^kUtrAL@F8`}={(fBN*u`@^ zQnu4eF1%jY$=8o3OpKEeeAl6=0CsPIq2Ce8l8A5B$)+zzJ0Kleer1KjCmTt|Z8YRb z&NnD+xhUfDwz_h5+LLV&+dlofs^{y@Dh_}24ksL3adPfnOTf{M5AKn>^$SP@hs0Mu zQ|c`4yR|em4S@6oL>&HhSnz_v zpSC+!_gPH{M!AD2&+zwd@Q*VHcSDELDm$^ zW+}sY9ADo)jk~xsc=b`{*m6fZtlJXP?EzXl`BP*N}y&G%=fJ5J?T0CBD$5@ z(>iBd)7g+(InuGlO-YkI+>+CoPh-Q=*VD(-&eK-RXI5XX-+wy}=CcsaUeFPdI48g;`<0{+|q+CgqayYjd; zJ2gO3zMTuG9Sn-`TIa+KVW=*s8$N}}WF2Udj5W~(Y1{+PZW zctIz6@sy48ALl~wqHJTNs48}gNoY7~54bm{WUtqQ8C%&XFkEx;pg8P@TAUu|lDJPJ zLupGA_E(~hz#Qo25bh=uY(bUaQpq2Y6@uemMPs#DOk=Tm00#L2qSq)Z8)obE&$5Ho zG8|I_jrT{V#ny!hVnUTUylctd_QAAC%XHUy3$_Nkx(at_K;8H^qj@c)8AF1tlS-*n zX5CQ*u)P|Z%1bGo)OV!D@{r40I;mIeKm?E#5?>pG2xfh0I%MJhu^Wa9&2&PGBD$Y# zq^{?zzX)Yq<$~Em^I>ycz!YUm*ZW5Fz3V*s^OVtuw$#{F-+YH4@%86)W6xs3EvrWT z(9g`$oezI$M3@Sl={*3myUHfVtIw}gWMZDaSQbt*r=EFxLddZ4?L>>o;LrV^Ubs+? z9VSPe#GAiyLjqp$8upw?f{2HNB*S+l!CP%iS41dbNzuT#ko3yu(?r(!4(1;nL+a3D z_svvBwia|`7fGq25(IxqKW?J;Ww}BJk_dkka($4^^x1RQ%qytU6n2)EBZBHuSb_ zBe;5aK_)nn;MGv&SX%sqs||0{^mL~2sm5*<@Et7-W&dq|i)lyxhIW2=T$$m9!NB=cD{l!% z_2!7X;53l2y39H^I;gXKhOZ7LEm1?YI`TV2DY$u>&E29Icp3q8-*mdR_k zs-_3jc%1CdHq!qZ?38%#(UYGoH!m>`FR4I? zZg>3ErMjr)t-XG%WLT&{Yjh;Xu^2}==fIpv_L_pso-Z#)k6BONT#((^n6RD7_V!o4 zuTQyjRmOK7pAsdk=>M8w`>&0tbSdxqS!p^qw1LOz1hFK(fM;o&jx=TFWuT=asU4=K zG+(x~zT4VNczEE%6;g$%g+*CwZH6A^57#lc|FyXQFP8Q93f~_ks@W5=?R#9Np3USh z=V~TYChY9EVKQNt_426l!Dy5gr5zs#O*TT)%m?Cw0_>0Sq3EvKT=S?v!Wr5ojXYAr z!GvjG$;a`v5H~taQvC*3?G+uDs)=VGV?Sf1;%c zs;5Q9dJd&66q-b9F4uELWX#w=E>YG$l3FU&N!xf}35~(Kr)0`cy;zJ5=D0L^gp%H5 z+i)$f)t^^#T)?Gw@GBkM#A&6KLeNKfK!S-jFPIOC$}BMh99pDcbHE zH2xx&QfsVhQ}7W^{KBGh_|}3yj;8h4 zS=GY1Jn{_Z7W?_PoJR$2Od(qamuUAEQkxxvDIc>UA5uI9+h{1N$Q{UeiUN6BNH>f% z6{B~g(ofN=`x;ysq8>ANL6J#qj_yMz=@JW=5G68Xn$GP+VfAnshTPDEOrf5E2kc<2 z^}`oio2^e$u}8n>8NE#Vj1Y!^X_OB^^)1oMxr8z|**KENhio)ypJqrWf5Y-NjxlZ6 z{iH-1zHSaynDCw}gPMAcec4mocFPUU%)RySXEdsNXTjKdGdSV8XN%1{(55RMro3sy z5*9Z#9Arpo#4MXxn|IS^#P({HWt&OEfE<%_s$2J#0&=+MMyYO~-ySsI0$Ge15;C>Y zOBq-xh2opAC!5obIZ&x~nDm7rKkqJfzO{>oQS_F7U$3>k@omFr(NY}!yCoK+Q}#R9 z^s}T6J4&$T;&@JoU6^CjmrpGrMqu&6Wey4dD(Nph8f1PdB-sewdg95a@!MDj+<)-2 zJ3QP2eQ5bPgOG|z@32D^won|7=qvx;i(|{Y@o~$a;ON;q{w+jPob*mni!XBh&_iALOGgA||AZ z+o1**4w0rMU1Xr~1w2#!;p3FIxA+2Z0NK3v>G*2L&uA{awPJO$cB^NdzI-{p){W7o z_EhQUx+p3;hJ&Q3$U`ux*cdG~0iWaBpbEV2axbkIN4jfA4 zH7_C+t_NcWvM_o5Lto$kDe(carhp9;lS{9el5-PFT~Bz)1^u8 zneDR{2T+{>N+Ysc(y2jb;wbi=oF;ahBT=y2J2;{CBZIA3aKEVkAK=ZVJs z1<#P}6{j&;EaH+Uf`Yb=$HDW?uezQv*Q1foyxx?>oU{mOIp;ugA6+Z7{I>o?bwkB* zjTA3dJJxa~7cjk!pWhep=2LYezkTm(#ZkVYZFD^Ee6C_#xA1g{H)UYdZeSRZo4Zjf zUJfvLpnlY;IU`<~XP>uiSa(|o8Gyf+vI^o~SgEl#^)zjXPyVAi$Kv@L=0+0f;4>aK zy|&(#3+$#tatZS%A>-AP1B@b1&j!CtQu!@#bH#(ZYGq2cTo!|+4o#>fDy z=dNlav5O8~o8Wk2&*$5czFW^bm@TGYigg1j)f&z1M{SPgt|n`f;DwlOYXtI8n8)z8 zrDd_#IOu+=;NY5#4yJ0uphd4{4t-`cd*pq&m9vU`&MG;do4;?C$Jd6N2LIl^nd=tzjkJXO9dL>Wp{9V zLH)*SnV>a@Kj42hpIDjmhx^}Gn(^!fa^|f6Sr7l(?g!k(%6e0B*MD||BJQ5M-2s^z zo_Mw)u0gyE{Dkizgb-yoUHu*2Z*fGNH19{?E=nksulh@Q<;6yv{kzZ9Q~b&%nD&l2 zf^@wQ{qZ%29 z6B$c%x?DHrt~O0hmo*V}J6oDNN1bC!(`9w%!PzdYl!myS_E>K~N6?wB^?YvBytO4_ z*5J6Q^cpO68oxTJ1zubTf6rY{*2$G0-+35ID$qW&g>o`WM{fX8Kb2&0(JW2mhQRM& z%DW%PbpZXp`sw|}PlM=epV_wGoM8;K@yJZ2;@77Q>`K*} zkGkp3wbLb!#1@?={>fB(JWRov8K*n{+5KNPZ%cR9Y)lp?=~hhwbQl+a&`3;Ys+usc zJ%&LlCr_Ojyb4ZUu?4v3!_|)96HWN8+ySn8CZNAMZgj0$h2SKP9Ru6O+%k9>Rw7tM zPiP#($W3Gw6*`Z#9Vb>1HXd!lf_KW>Z2|cUOe4HIi`Buad31KPwr%@OYhBXj4Y@0^ zT+PtZPfQP%4uixdktNACpYJBv-oE$&T$-s#%Cb9aK5I=c7ijlN6g0?0Wmk02*)=%K z{Bp8PHFAk6zsGq;z--Rhl$W+I=FD4T85r#RB$wB96TH<{tj|^75Z8-8K>BX4F~p0&hWVbS z?~n+5a+$WmsX&i;rx>e^y{&f;qwV?BL(_TBoq})$i-EZbL`^)bUlxU=8l-xaH@h}$ zhVcaLV6D8t>fxdQrc&Xcl5);Vh9-Oqm0xKHv9#f?t)5F4pUcI1)S?3x{wizT++N+x z(h^JcowLUzTc+nYRc`|ewz?;ZOw37T7<>-Amc0O1yVb=pp-)PRGJ|s&9i8X?^u|6< z-bWI#5UwvC_@1wpxwYCHz!|uc1MD<=A1`6wnuq%?e$pZ}bU;LvW%%7*b~Yl@=dU(L zPY+kE-`4j9=L~8y(*?8JvZ>|aWS<9#%#d>Qm z?E}pmy2~#TUTaA%3!3pgS)xkP`3sd9HXFKkHq%u@YI7yU;<5DwZRvjK7cxC4!rs=M zqbr05qEx#}U^zV}iOcDk3oleZ7i8&D8I;>8!rNYS?`9;C?PcQfg~tHC+gq7M6Ys@) zeXIt~3(*K9E6-06D|a*I{s56!94V-a%XN!liK|W% zejQr?${BtF4YgDK&Vo3jy|fD*C6~1|H`Ud3bM2{jb9?g(1Ksb1o*i@1>GAxiksv;A z&gKt5$)j^x8QVc+_HGT7vov->2U)?4eERTU{>`$&Q2(QR2#Kx;aGKp=O3 zv<~T)5tp{+{}ZN2Ygq>y zgtfXfvjil2x}{=1O%>l*!Lj$CtFZ<{&2BuIoPjeO=W$%B9v^_O(c2cw@(;MJemcB| zMvI9rMo*no*S33YI0MQ$ZDzRAa`^-Bcmr&;EZmk`jW{$-ITe=K$4a8NzmoJ(iWNNxYX3h)22xKyWS9hE8uk(?% zkXGF}wox{F`4^+mUGgo|(jb3;VWJEq*k)gQZ0W!;fiQi+u`1Nf;uhJGnC-HnFilrM zK;)eEQ=p`m1AaNv9M>#FxUyd{U-*CfQG;+T!}}(=^28c=IoBXYX^L57n#U50Yt|DV;}8Y0@40kFYB! zZN-q)hNXGC_E!E7uXgY`Zlk<~*0a8e^3q3r+D5R%6A9UT;y`Pho2~rK}8*wR?%v<0tA2@E{UE=o8g=4uwedzD^tvmfE zzDHBrC-E)QBGAF}3!t~^IjeEIp8Id(P4PTIU&e_LsaHsUs^wPW3%aBsCj)kCE!`%j z6lih|pB7o_p1-(70s}n5vTF9>U@;!UB@R=esV2-OU@Sx76yHa$W#!;Z1e&VkCb5^H zbilk3dFpWO{bQ~Q-onoTn+Za|H~M!)kj<*gKXI>S>vqi#e+cgTf;{F7x!k_&FbC8+ z;=Dr(4+FCMH?V=YxZhURX@IX@&2}Fy>%TLI=MlSXou4N9!!q5l&sn=$-p9RSrju}V zWZ$m!bUNXva@a>?Gjh;EcAe{P6J{s}ZM6}z=;;YJH*+1Yfw;dw2O2TJe zwRIGcxco2XQ(z=rM*bH~cj4Cb`#lVt5GEoZACwLi1(A|)Y}7zX1cnHTNK1Dk9fC|G z22ui}V}x``_vmhru8j~FHAel$_qm>bV7qqR+d1cb@0{1Mb{Ji@V$pvkQTN0!soLht z!v>EJ7JYq^;G7HhlGPJSZx1KPgqWtAmCcS}GMpZ44R`k>w@mZgNIVuV=9E-WTw{0( zcSc9^h^ObU{MDIKkmM0#-_~rfEOGT&;%?|K`0`voabG$vtzmqz*D;=NSc7^O?Z_+V z6O#cPT+y9b8(0x?Q$1*mtE_b5HT(2e=b{R<^{^yma$@u^M#|s@KpHqRM;@JIUc}lS z@cT~4I7~D`egC{B_vk#$bfc2QTQ!t(^T4h=sNbaUWC?~(2z6_h;3M3Y*e@sDtJi-5 zXdpY(;$>u`Gux6iX7e>Z^{2@s7*|6!k8%6m+lI^FA$}^e<+em^C0%nLNFkq>?k-Ho zZYU#Yh=)B#tp|9bUu#ChW=E`f{De))dD3D3P?5dFmUj1+PudXRUv$#Il-y+m8!4a|u8`J{9c958AlD)nu z0-c9Cpnm*>qKB+qqM==4{NVZgXQJsf#-SS~OS)nIo8a%$(VLx`R2;ZK4p76*baW9Z zp>KiyB>|VYmbxH-L>jy5)w0(&qurM>>>egA7=VER_GiLf0_$~Mg_(ZE8^GQ;lD3Bx z*n78~!FywHyt6WNpD~LgPetF6G-2D_Z)0!3bqW?A|MggncyB&?e#8B=w-eYdHKyw0 z+$+6kgR@Rm^>H%YT<=@#DRUpOX%J9CY0lbZgFfc4FsDn zuz1zYMZ)Z)i< zG;lE}eZI4Nqx-;aoP<2!0fC8IWF>VBKb~qjsUoO!ivcgUHo1oxwa z@n-6~fq-)aS5>}DI2Q38bFVI|vhmOYY%Wt8?0@~ym72qEr@!lKiW*$L9J<;WDq!pGfvf8@PbL5qvq^6VK&u)IS*#r7xCg7CR`4QVb4$Hfbchgf}5cXEPgTm22kVvYwa? zBd==J^Da?pyvxQ?WsoY$s%LvD`W93#(*9}1?3hL%>0vi8NvuEAcBU-ZEpssm} z+mQEyi^mNKR%fp_8CmpW(_pAIzNM!EC0z+y(cJle%-T-7^lx#YrW?p(DZKQVA4()u z$nkkg;GA*Ot?uv3<@@J@1P5;G9&U{Q@+_CZeQn)heQtP${Tms^ExI}ZD?isZO0r(q zv{hI@x1`dvrV?`Fe6cPNe!9N68=pqaqQ?L$WcUfdoe&;Q%o4SC^>44(Wc>z@*&+|t zj4;{JgdA(a2(VO1ex$3oQCa&4){H>MxV>W%^y9k+7V;y!VaC>vZ@HqExvVK?d2oE5 ztQ7qYg2A6);Ys|@iIRca7B79uD;UaMB}6~nf}nQuxsKzTsq^-0LR6wA2ux`@DH5zd zmIrPjBMC@~1H0die`3F2w{+=uV&U(OE8fEw*Oj%@knsU1<~`f!QVGvG|Gtz+IU5cIi_WNR9pY5oOnoW*UCHZssZPWwi5AsGk1{ zx8LXgK~r@2jT@B>MgYl5ilMx|n5B&y(5=wG@%M-l3zX%_pzg_EbA|c?Ie_i&CR|wa zA8MK8XOVxoA!j`k6m5x0Nc2i(Q=rL+;vldY!EANt;mV!DSDD5K`rH%M{u<)AHuJkC zQqOJr9t-!)if@qQ)d9SOQr@n#cG5jk`_k?pF9mm^86WhHJ<<1Xm#-chO|%GmP?iDp zACN=Wh2S~DeXyOzzu2_s?o8POL?5x|L)?DlCS}tSWr1Y4%Dn~xWT9O9WZlz89|-ZN z&6M-QYg}(@L!06`&hVXooPN=!c1czaWh@%ZUHq*%`xhhK)Y}eVIVYqQ&xzvJM=tn| z_}se-E+u-D${dEetf#9NCty74igD3Z!fb1R`KlG7yP6(YB)stZX?bx5{6>V_o>n=Z zzO@>s@6)T9`fm5{4Ma9sD{OMVSg1d+3tSj}*4o(6J>apfH>ypI=$GZVKHxCi%(_Vx zd6&#d9h>YWhDQp9T5RhZyfE^knANF|HK_G*W9{kEab$y+R0SD?aB$k5qfIincklz; z#U{jmtYhW$n z9M~^^hI9%Yx*&rpk;-PHizca$!}56vx1t#$ae_td#&Invu$IG5e=IMBa>O)`=$EU` zRFNpjME9g0&_%%^mV^JBk!eR>5Ov80>Pxi#F%4iW=w81Z@;S&RKR5=tc zC{LNzB7L0ihAJ>xgno;nYEqzE<$4uLH%N~0ZX%4>WI59_?d;yBSM&MU;B5QC9N3H> zHlH?6Gp=x448zy7AJ)Y_qVWZjW1zEOcktUDs1|AI%o3t7U?<~nquGN%0+dtHyqc?ouL`G}`;QXXvQ>p}Or!lBp zu)7uspnk$aco{vzcaQh??S)CThqDXc)e|m@$Wq ztBg13&B&3GX?kr(`^J-=!c4L^jBiAxHeT%1QD!R`Mo__;(-~NC9s$YHS!1js=rg9i z?b~a4Y&hhX9SDqFDW$sfnwk#%RJZ+~6Sl%`C#M@!9naY*Y3x%Wdz2s-9}~Y<4@Ec24>snfJq{Z@ zlTBh$$jiPR8?UHuRZ`1L9oG)|+IK!#?dJEOUYu3}>O+17v3}5rSy5^{CESA+7$}}* z^`TxcIbAvZZRa*2tRpU!elhe>y&(B62&Pak@oeJSv`^MtKm*Wlr1@0qD%in2iJG`l z7TcZ34qEGFHUIk$Xqf;UQGSa+=yZX%V9SADY8v@$J zIm2N>rN;<3wM{?OiO1yk$(I{!fd6by6*^NZ!eI1J`hg6)Ek7|^e%1hTiq0BbUAFCG zJL|wHN;^OPd$vY)4R`+|eNJlMO!g=8IEEG$l*fi_VU?vP4j=3^%dny~A--@kJauzh zu41rz%OHIt-fpp$9@KbNzW@Mx=|kvkGJD()-}g6K4i#8Ma^D`};s{x6qmN=i0sy}b zjGRkrmqk~eR1#IT3M~f+4J3JVae#I=h6)oA{#=iI%~qp*pi(he-yeh-QF=q~o5Z#c zE5#<88N^25j(e z3|-=NFI<*Z^`#!KW|xK1-?B74$czBCI`*{Wj(ITfKe|IQeaq`ogXw6TlSBc`@_LXi z@>S&g{wpAk(*Z}kJ^`2R%03A_5sBxt84);YCqbD{uPENyzR?8Vb2GzKXIl&$RksY2 zzpy~o6TWpD-{B}(Q~d|*Y_Z9lW!8DEnRXj=`I(OYIL}q)(b(G+heq#VzlFb>Dw%8s zmo0Wx1ACwp5d~Yriaxhk-Df)r1))s95$5k%_v~l4*qc$&UNcnKAP&ES)(i!dX+HmK zCs!Y*;CfsPa+ek0w|xiKUl$VL_%X_jO4m}qU8vhiePQE;k-*B@lbV~%S2TE^+9x{5 zbviKE$=;sOU4Bj8ozgEK{L65f$|XefP*585u;-rrKKO^Rr??HQ_UHqL+I;+`3-&5%~ksZp7n(>bnkX7ASb$7aJBEUKev+c)6v{B9V*u z$0sE}D%UuUt!Nxul+uWk$!F`(B-xv5{O&1EyTpKWLU430MzZ2L_$7r6gZmim&EK!J4;;ogUH zk2tvh{U`?iWWlt#!38-xde@xjfJp=2l}h%fBJm7~9X41*P1S+0_vw@2lvJSgX zhk+Y2igk8>mjF@$)o1K}en;~cwgd8A5is|8`{I{1Hk9J1`?IKY;n@wHIaGkf`Qw%i z2}ZHOX+_)(1FEx&X#4;Z2%L{5+vC({B;r@t*Z5*{?}i-<)xqlGgp2hg{F_JXl=rbX=XWY<9;a-Y|a== z5yiiBTU0>xGuN}#_TY}_fA7aszxN4zMU#qiNhMpgl$p0GeHq{0TnO$H>duf)Q}xn8 z&*?E%*fE{qr$d4d+j-L-He_tG58||GlOaF_dIXnI7{!u7vH~nhws3B3tu`*Evur?p ztunbF$ScQG3XVa{+*)b9q7e`3H zvS=}Z$1H9?{|EDZP$KS1*RBp1gc(@56xlLdTkhQ_%rgF&>W`4y6;ijHQ+YGa^N_*L zD9txvWZT{#Vw^kEmv2Eq!&zSU*_hyJWF&LUTIMdQpv+wvJe@^n+pMkAeMjHTJLR2! z%p+FkG9B$2@szK+_qJi(Rl=TK9LK{>gEnB)7%{iZL)KBRS)3O^^R6fuFmWngsX{OB`MY? zsPE2xwy-G$4DIaN?Zs9~Ny%Xz;L!F4x^1L6v(IPy{bC(+!fZE_16}O1tZvKsVth*- zYI(B3jjGA61N2@Y%!PRbP^To;Lp{u7q^XQ#8yxlrC<0SPIJEI?rt6gz(|>9j7l|mlV4Nb?{5tYeZ@rU5xn^HG<0=4g-<1sI-y53G}5F# zTsA#59Bk5;6qfQu9Z=a9sDk*45`JC3-QK}*hMNSKnlCWCv}k9p%gAcpZx8hRjn2Pt zf}pvL{*}f%F6Hc^z74$G%ypPAZ=R$#WoOr~f=Q83S9E`1+j2JIWR9|x$|Bq!-@LausV$R3$f&ifd{DHFNw}A7f$*WKXEQq#c80*!K59N z76jptEc~=PkLw7cPa+Z3aEc5gZ&lZ&77R?v*;=T{a)ggT`y*yQ1-en!z<(`uGu z4lZU>IY0WZH;C-V6#Kj6wKLNa4`^j?InOpH2%F)ExU;W1SlBQ5=*z$V^ZnE*dPsAI zR-%hmCi^Zwnt%`oK7z(0*6`%irTcSl7Be0GCf*)V>&R7B^1^q&hkQDk(IW}7pi3UB zHZl9!|Kz2B$rx)FHG)V=Hi-W^S8 z{9`fal@_8z(>X626!1Ddh=Y=*nFtmV@lzq>g0H`4lB6w8wKB2a?i1g$a60F8nNr!R z+YTwL%etP}ijSpl%bkQ5j=4+0E5TC+fA|@bfnqZAOvqFbMLXel)bM|pWR6-=7QhG_ z!Uh+wv=G|c!U2o=7Bx`h#YR0X`Pw4psDeWpg@4OCgW?c=HH2aZDGc;D&fYKIdcuo0 zWYXFaIpMJ^uka1%Xj4kwN-rw^r1T62gY0||GB-u*yaM&?nU4O)|z4gFNP z)Je+S*7@}FeR-}~BfT)j&C1GWsEX?L3|+kTHDRmn9yWGej|3qw3c8kOb!u^*WIFYr z9qFWa7IASLb6b^G>v8ZjXuZn#?kH%PsDTF&+bY&nsXd=ro2s=FdhogZA)J+OrE2a} z>o~EBLg}d^I-?PVeC8EBfY<~+z{-Ovz?oUYtMXisYIsBIFQsjEGZ4mh3H%B)(70h8 z^NQRubO*cp-V3E{~Gl3Z>Z78z)p zPp#23rqmSEl99`MIuiYO0~{1-%B(<1EfHndY>Xcjxw2>h>7_k~B_cTi^tKNq_(rGj zZ}hu*eY!Rh!#?Zk+g*UtrWLC-IY7f2T&@#c8#9RViC!IK{#?j($a^V#ckX|Jfx|{w z*sXeJYDTmT!gsV^4rdONH?Z8s+c3g{ zT@1jNELK6xg+|XOsfN@tkuFaCe8ytXA@ck~H8z6G`Jt|Bj#<%)v}ibg^s=b{b`Sv= zr)y!+VUl>5jHT!PBhk1LCe3snz#33Di^twwy!yVH#%9utmQLLa6t+!27@|OHax4fh zp_iV~QuO@OO>=x!9%oxyMIzA|h>l@ncmSwAAZ)n@Ch&$q8mc6RR*{=U^|zp@{t3&! zs5@vaiXRm91=_Qrje@T{tI~?WM8LF)o()-m>xOP5=&;D`tJ}BQ(LZEp0Ghe+i&21d z)R3`-uXRKKxwmlx_kegebA(jRCL$JB6_n5?0+*C``IdD%&=- zldt}=GuiyC-Dcz6nH6wrJM{P7>c{>$)eU0zamkS`E#5}IW;O~$b;_%H9WJ_HvZ&Cc z&!lvzfoK-?7V~go%n!p(p+ogGtL1i$j(ft=^oikcs$jK1ZHXSe1p~&P!}1(!Gg5M0 z-*x)G{EC;sJ0Wb^M1*Gl%skW$AhS(P$xh_EEMJ0NWfuCCS=I;vLqop`Q;YOeuOC>H z#{jz4P&#SCmJzZ{t74u&@!q-nGm@atR1ytedH|QL$I;Zr1kEoGg%!uy>5Fuwy^EoW zHPb^W6)5U9n2Eb65_%xSTg1P)VBSvBMNYxN<%ERc+=W90lR1!)9uLl1g#Pnr)53Nk*0KHQ4DjAyjyD7pi_!Y<`W-Pr#}aJn3Vqc54<{I!eSDgf*<~J zM8f)p-gWJF+VHqL?VaMx57XHJSbBHh0N%Ui(Fyu`rk=)QF}_e>$^T~*iR{=jjfZ0C zv2PVLG@igC;|#LbDZYNa|KMJBwH1QYPth*Q*|HK`zFhQ44fGckzRb z&DY5iU-quA{RgEs|GEs#ZyyreA8+%4q?b06zY}#tDrj_K9{J%yceiE+tyn-7XE&`FrY1L#*)bf&}=b%hinjl%Q&P;8lC# zU}EgW2$_kHk|yhOu3cA6aXqN4(c|BZaqV1=z@8wW)aHdqD52aU4Y2=+I5#Cj`q9@P zC78(Rt%@Vuy`f{n1f9NBzZ@qMDYC6_#T3C~t9$HD(2kSvIb zUKE`tURrM%%_;7>(AHwgW@!j~R)-Y8o~BHfbMM)OtqH87EFziX1YVMB~s7 zhDfK?n7hYgvBr9|ivs1Y|rs&Jb{PF;2DE#8Z&MFIvSRq0TTCzr)y-s4n@7bd*ClDpK2| zTpBi+kxq?gV#Xy*xfqHKS+dZWaKAES+b3J{+`M&arv;U?@}PrMM_|1IECN&AcgL1S z0Vj9ZS)pzXU9tuLfPJ7Da8p4 zNi?MqQ!@PiK3&|mtg)s{gN~~eLZ&*nj6zmiF)zKg-lDS{+6s z>hiA7!er~y=-b4#xQ2TwLi+wQa^+KG`Us)CuXL!U#oxqaNEhZbTCIApruiA6o{Ak% zcd{QfHG1(#v>P`WN0zEdMt?-k=KFN<&osdBs% zN{}M)JP`uS(SNgk!#^@VqY{_K9+?AjPIM#e_T8Rbp^ zuQz35TIKejU@1$4m=YO+x9%AU?z~2QXc-aUGNJF?Deoi!f0ECGha%jCJZmeU9>n#< ztk2Pg1)6iXJs&WvM$W zmddg|K~V2Rg^Ea(?OJ?xxNz;IOv73m`X)5hm-9iIW{i)H9~`gZdQNBB zG}k8Y)MH`ow>jwNux-oZW2ZA7$3qnS?kR)W^GjzZTm;(hGiTLPZV4Ml>g>BQ}b#A3PU@{@7`NOgt_I}nG0_in$Xnq8-WXPNWJ{vbf3wD?-Qog zZv9@iAk=pBfp~Yc0)t+e0?i`2{Zx{K)lLQp5&#?63nKZF@3e`qkJA)ydha#XERXtA zbJ$S?ZwZcX06!s$%hId!o^k{fm(x{ulU_f8zi2j9W1}k>dLjTC4R~886fDBR-t6d2d{x>Rj6wu6<|NyW#;4XsqGKP;ua@iCMrq6 ziK5MA(Ns-`^;5?z9LDocZ5roL-}1A?oH{F-%P}7|MzLcB6UO;+%+rBCX8mhRF3dy5 z0K8(ta)tf|15|qsr_zi*>vn;_-Zz1Bsnl5v(Pe>06MroKhO5Pre=a=Ot>h@xFyS0y z%%)kIM-`8SLg|z1Nmrw&689e(q;&xBDqeZ2M2jFJX){flziw0dcB}~y{QQ3cvuS

    >TM@S~Ki7&>6b9h7q z&09Z*KU0rN-=uopklJ^l#2tAivG<~2J&^CkpabN-{ToEjoUR#d@U$+dCSu8lgRbT5 zJYWz57SRCr#czq6lg|2AO^!KbRoKZRHZ3$}Svj$OWUL1>4nRz0K}~yvv?G@x@o^O($RE~q9C)7sQO+~~Njy(vGbi@rh~;2nb2V!+yVWLp88#&cSiK8L55jPjtkGB!H(txp6t7YHlt_fW0vy(^i{0yy}+ zz->T_*`pPNT$TyVKt#N}I_`tE>)zA=?CC0hfs)Tr`>ad(I$GdZ#o+M?j}8qk)2G|GN@#} zcDc%b0J{n4v4bb<$m!Nw_`J%&{AoI?lrQ&o^C+ZS78U*;o9KCr<5)I|s#Dme zfw>z6fpqHt4Y>O7+U=2havbq{t5#zr(N!1O@yxEmkE9jj4|nM-WE?P5O<$zDq*Evs?@tF}kLiWZw>l)2SE;W$8NkzTf-sEwqp;N-=B1 zg+{JO5-ptgp-5r5!RW3L%J>B;OEDZPUW6_-%H+|lxjR)^!h7fYRI?%FO5q(Vt2c${ z`%z`92Q$65I4VW51rJ|6Uly7FtXrIL(EY`%ZOcr`F{?fmW@0pIqj%A> zfx@&kohe>e8ym10p}E_pjxy;b*)WSEajNQb>O6Wq${v5)<ea5l4qd;s>Mg{=p;s0n@m%B1KoPjiQdU z*s5z{Pli`{5BFQAF}4z1DKAE|;5zZzuGLA#7px~sS$$wy;ah{w?}32Wnh(`J-lOnM zWrED9{b)Ct_(f6E#rBFieoC@%Nm?YE1CuSWr)_ZUW|i#gPhFk%I7d3%=S9Tv0(< zqRZ#_cGBeT{`dC!3+T->VzU>D4XS8I(T6 z;nf9|xsT7BBTD1T_Jy*^sX$zS{~VQ5fEXjv%9gJQEx_ihx^nlZ9GAxURqIX77x75& z`7-C5DS1ToHWid8^fDAB3uJUcxKsLufw}c3_lNXXvX%YxO3hflW-thyZscU!!N_;4 zrzNJS1D6E*Gmp>btMTLBf-EyT7o3xqQ)FMC>InM{#_bS@THwBd?V(gMpr5Dx4L0tR zbvib^yGP_xd)22=cI5EfNJ@zt85^97Wv*iOj(DksWRdyPvAQU2j5HK+VUZV>$srqinFUmKLbmL)k8 zEHH`Jb-oXX$;cR^zAhQc`TlQP+w&MH;e?RNK-U5F0v8D)j{K?0kU^@y_I1f&3*KdW z1@&n9ebYcN2rBsaboX#YzujssbuwE%U5ayz6QcfLm8ukv@RvG^{F~Qw@(q2(oX!Be z6JFGvUJeu{-^qcyx{*@OGB2aMFX<-+BBc`5f(r-sU#L_T6J7DqVu4G>kM1Dbff?cO$8E4pJl14I(j0NT-OjAYDUB2$D*7 z=iQvM&ROfO<@yfegn9!6k79!}EvO=w|Ta(vL zFpk5@gXEU{i2-uOl|=S8r-dHTg?fU=zmB2Kh^M{;>5>Ft60M|qS{tGy24wH0F30LW z!xEpvJ$Vja9i+puV0hv~P%odCFO3ys`eg3E%{CzKYH-^8Mmxy;Ul(+qem2zC_Ypm$ zTJF<_jmER@Vyz+A>sSjC>+(l|-XOm`D2C@BH-cHl4D{?6^CoY*s^m4OipmG0FDtAv zH~Ai&6Q&%tLBj0N#>Y|%*%RDD%aTP4HUD+jPjSQnnpcWc|G0dAlrOY(xAI=dCxZ5x z2B#QMSM1krg|je@TWs6?iS43H{0-V3QKw|Ia*nACtJDr6U(8N9AXkF4p3vQI^^eBA zgG{_D%yWW6f>ljBeum#EDF5h?_)Wcc_y57fN8yOyTpmQb(nPZFN&X-$0wzOecbU~ct z6imViYw&Ji_Dj^n%xqVLer~c^; z;8ZujfS|E3{U|hPhT%5<2Kzmr9jZ}?LFH>giNm-lfN`s42s%N|pZ{T2x7HNEY}13& z6oR&*K{I#H(JE_Y!)zmOc5eNNHyGIOooc5#ODa35utqo1L4isK=%tYezrjw}C7wsn z+%#kexj0K+aMCOZBlaY%uDlP26;!`E&-R1Qu;t|lnF6@CDo_~?l<51*Cx^yMQEUw9 zng&=8itUJx);bxER+!6<<|^K=ZYK~+12H`V+NzlP=eibCRn}Nq4ev{9r+YU6s%&%Z zFS2z|v;fa1fH1^bp0Pn`WxGi>J$pc?=x8$ygu#kPXNV|ewntx2@%Y6E-d;QMkY$Rz zx4bM7xi>8`uPI^zOUA&=hm2qkX2uem@+@e7|+Z6lTnnf&VT77+(3Vf!B8w7 zcU_JF9fxjN37JkWIz`0v2hVBy<412x4Qa(Zp+_4(g+Pyz&@G@b0E&>Efl4!Bg5AvB z;b6VE{5?RL(?L3Z!54Mzh~~?EWy9lC6JPfC_(KYGBF|}nroI`p$lfXpCK^F8ln$$Umg9Cy_F!Uy=A*bx zSW39r_099P`NFuNi;A?Pze!qZeBiS3tJ6&sGQ>C#(42oFShIL}8h3+lTje*q>?;nXv1xWvYb4By;7*Q>LGCm`VATKvy>9H1g>a@hOpYWklwSk%oW9-p#`4G*2YjfqxI?{@(v(uS_M zpe6Vvv7$Fg3Jo5U2|)xVYO%vUHW#_PBn;!Cx`x`Hn=pR5n3zTf&F}4G`h~=L|J}Lyy3gz`pVd@ciVu;WBSV^y{8khKP3Oml31adAdE_Pp*e? z&U;w1(||agTj5UF?iMKWe+XT(dvhhgYewyVo>Yi)FN zG3B2xHU8+51?;YMRKk+}Hs4zh9niY0E9h~>W3!D>WD#OWP(6mp7GEY8YYu%64##7K z&T6qAA;no*KC6W{qcuT9iPxh@=qBvt%yem4p$U+%vaLIe%6~du8{wy+m=^^~O|l)&`msUfdHAHzLCny*;1rO#nVkoCHG>|^LmmaZz2=U7|Z z*@X9O)4MNxx&)06`7XbkalM2AC({q;X42?_AH!fXiv=6buQZ`eZts&%Kc2ygNJ2_m=={KmQLaZQn zjO*p~;P}1bXHUL8SYH%Q(74pyEsa0@S-H$-JWZ*0GyQddM=p>{@;@X!o%&y1#G#%|@v`-AqKPhk>d zSZeqx_fky>;q`(8!0Shw7XF5UgTa@sF5Y5-uUW|*MqwiEpYwESIGEB}iHxG+$$P;> zQi{(XeJohQR6GYnx7L;1{~XTxc?L{!QaQ;!QXt5b?1mFO>I_JH600rtM-x7;FkTZw9DQG$YmHG_GHzmz-gVrjUnjpPmc-+KZ7h0Z${ij7gi1+VoO zUP*o_V*9bs7V9&e`OklYGNCwRyUPFHs$H?wHA20HXMg7@uDJWoe=9y&Cdnk zP#iiJEofN!A4NrxI2$K8&R-tcPU|XFmIe6`ooP^os? z-#7gCF$7N6sQ&Rri~>0H>$L1m$tjMuE*@j~M{gru{c#NBt;@|+uf^T@Z=J>9PEo@B zsFf-atmjSi=X9wQb>ED8NHy;kJvAedFF$k8q|D#zaIjb1ndn_V6HiegD_xU0^)5OO$N1drT13!rbwO(!H$Ry z@d~>_S<)D}Sze#)vD%*?i7_9&Q&>X5$ChN{2q~#>6M8^-kt3ZQu6i0o3px$8!+n;9 zHQ~^Qp2Z}{)rlYR+MFir?>#~yYC;Fk8b>Q%5br1tvlxDyxhovOXa(sh`$#rQ(dLlq zboGkftwanuh%`)UYVn>uBk4R}*w*rP@u3j-9)d^)rF1;h*s%#|hKLhJaYaWOah{K_ ze*Mh}A*fn_LTw(UQ^|Tq$rgWV@sTaObEUx zHr3-*+&1Xp!@?k}2!S+IMU|G(9nbcKYg5@wT_qzDvE~S|7H`Y2-xT{~o)FQ$={Q~& zFp6xET7BogS8d`l>>WUCVVY4|9b_#b{PAY7nKbY-pJY5ACWgEZ?1pA0xDJ#bO$P9y zeLCqNubksbTkGgNgAJbE319OoMQ7-KRtb)6{kg0nzBlEJILF;VLDK4;?S zH1m{q`5W+$h)VX#3NcVCgy8hi^K4s_l)?9&vwyV)ck`TVhN2BGUG?hK?Q>;0|10fc zVL<42%?SwchSQoeQptZn5|5t9pSYbYnj>h*@ZC68Z7dJk=f4k=OOmG!@>7F_Q0?eQ z{zi1vl}U~j{N-6zRP#UZ4#v#UP+Uim&-?zO(?fZZ19SuA<8%@(Um&G!U_LYihFKOzC8OF zS?H8Izc*^%{}Guvr0cGHbeVOF#hlRT4*KmawFvfD{O&1RG4v`vbX+Bz#ZXd*c;4(P zb2xrL%Zp!dTScOb!j{rdal|~V3^|HSt`AfEaj<$^F-JH;epaa)1;Z^b9XqCT9D;Qka*L$?iDVEeT~s9<>ER)yKr-~sc;-qS1$6jpngSI?14LBwek&N z0jr!BrI;~;$@Jy(C-o+)=k0Hj79wm=`?g)|g~N&C-|=JssSoHm^X3ggD%{OeBeAWT z!~lYW^iKUW;c%j^U-3!wZ!}9_yn$4b7=`}2p$Zk5`r%8iMqaZXSae|`rZ~{26wr|= zzD-n+1{bUu@!CZKJz4mKR{yOuQqpSV(PHN%tbN?+Sjq;StJmQD(jnO!d zTXH=^J;S!~^WuMaJXt_ojcWMXK*DV}s_$>sc`$uga`C@cn(>ep|3TD^*x1%mLUTCj z%Q)BNQm33uBz{hk)k(X;3&oqWe?wy)c!D^)e@?p^Xip;r2NQA_Pf33j~-HR6-r?q_^p zJROrA89}K(zVqRl(%4po(UNw95L?Xigzv!Z2>U7zzy8Y$u~6kc@GFz087DK5eJd3q z(C9ig@xzrYpWC=zFXSz}Hl5K( ziS{EF&aJFvPb97JzM$)OzmZmfDYZQn4EZuGiH7?2a8b!e!J#`YPyljsr}N@t!~V`+ ze!y2?ywd}YU#F_Irg}@4stazT!nlH#>Jh7|di$1v^s7gx7>#@FnR`F5tWFa@KXsw) zL3F0JG^-`r&ap^CTY}a)s<>(}4qY^;+=6eMh=?CX-f6p(hFdJ@!h>8SzUBv-J~iMH zM{g0jd;WF?0}$@eHH;N-L{>Mb&<6Y8Y+p7Zp_mWTIK)xX-y0j(P?$}e#ZX|(f=Eu$ ze2N6820la%l3F_v#Z}VkuaFt1Z(e;pfz~YNWDz!hTY-=WjE`BFe7IkpJK%IvBuwSy zm%~&)(X=l;UG>rE715xQ<4?|;{O+UrV~6_k4^8%CPrt-T)dr8df%=0FQ-jKh2DxFZ zQXwrg#OV6?s@Fq_b=0u~Qm_T4d4uA!@17^Kd$PT`aJ9{+Qs%`vaVUuddPQh`-8Po= z+6ZoYYensN&>W}qVm3r@-CTKR{+k67Q@{VX#s<--hd zH6*l%QXo&_Eajb`Rr=^9)cNg1sNb%8Aaqe&fg7l>L+#7zg0mGm%^n82^|BW*sT923 zXVoaUROj;raWhZLaeC=dJDUdMLmhe*)ZIT(N5#Pj*JU!B$tdf1?HOj(Oyk^(gpBZ6 z!`wpZF52~%j#~CO$e*1QZ1|5ZWY)4(cH#t(g%H|j35nXvHC|k^Q{Qk|HVicFpw33| zKoFLpZET*9Yz?-bP?}F?bKN&Y66^}(S2zgy9dD+iBu26&DXCB5a>rrMO#JucscE2C zN*calVTn>zez|(K%k@?O2)f~zV9=stB~1kzsg`CxEPZxlGQk_nss)^%NPzoR`$UmX zDGkVv$}f0tFUpqtFgqz1Q6IuGt-jE~5VtPL{*--RA5N0|a~6^2Ial|zT+W^{D=LzM zr|HS06d6xiN2h1Jxp&ctP)spsR-da|*IENA_8R-(PSyxmZEatMloNlrQK<4a=jGRP zh`hfIHJjIM&OC=DpCw<$@*rK3>1f>;XYl6$PTVs8*lO1B=d~Zs z!m`+JX@6-)WJ(ebx>xL&)s?v^y!moC`RY@wol7j~1O^rEJNCl5%we64~ zI3o=Q6?QbDMCkfvS*33(MQk#Ux6g`TXN`oksd6_Q;NMHrc|BP?^8Ilh9p{!S&yBeu zzNpn#%mB;TmSjhq3s`j%9e}!CZECb^k5+sZwGg45F71RUyyyosjvtm{Ar!o=zIY9n zlcni9Ih*wXt?1Z#Lu};aHOp>@okZz+pah*Xt_RxxCAXue1ftDt!1Y_}o`}T$aW`IB zmglYEBKK%U^Va=J$$RGoJe+pPKCjU_kOv$xoRbudodTv8SLm(Fi9enQb)NzM)6$9lz*hL@*7TL3)#KGt;1(35IM zu-!4`e5$d$`gfBUd?HGsyHbIDmJ#FNQr@)YzZ|$n&p@sR4(K@Kwwlaz|gTB#u znIy$NMz6A9yYqx<-n-=p*byp*U}0Eye0iKek2%)palVy&Y!QH1AD+wHqC%4aS>B7d zz)AsW7`%zJa{jK8^bT)!=imh#i;&LINaXDI9{8ypO&PsuCtn3Jc2u2?H2lMaLyQzC z+pd$Bc}a&;PG3OfZ!*tQ;Fpvq02?R(}0DkzGr5!iU%M55I2?_VWs|?Hb4K1d;#_ z&2sXWatOv3=X(#a8LlIr8RWN!N&60tF7M~6yA@-uQ8eNnKKZN+7oNBpflzaAi^Aos zuCktqHz||!biUPtSW9M0+dsffK*w?KqFPmUt>N*n;Q7^}q@G^b?7nw(&pG{n(=|zT+ z3HO!4T6K-q68-8%wMc*JlVN^qSjKrU+g1>YF)USK{&g{q3UsZ-qXb<4AOg=l3nA{w{^~ z$nHm-%YG-T{?qgc`Y3j@9d`dTY@5hbenO0GUy8<3193^6&46G^tD=tYajWJAt!WoH zbdTtwKFkRWiED8MXrSe6QgeXhJ+o$;gK8R@ZxP!#kJ<6|SVSH{z4*+7HQI$^dwAX^%{ zL}T&jUqCc>G66e4SFfQkB$AnIWmE?fM@m4geEjF?m&|{O65?ynQMHVS$c;b&c~EoK zrjZRbmSp_jsr|8H1hH!t2D45{Qb&5|gUm;Tc4(iYO5C1f*OS$VxRbLrAcTF0IV5u6 zPj>3lik$gf%&HZcR&6=65rH^)Z~=UB8n@5^-vA_(NyFp~9okVUQnEHx+Ws?=VTpz< z`mT$#uB<&ogt9la$_gXZm z8>}s?0P^0~3B(O|qeDjoe&8-$4Xl*Z*h=gF@kKpE@-0JG@2yfv;}RV|JoH|TMd2fn z_o-y(^G@nGPFXgdKxKUoNFJE%A{>p^)`q2aFk+e(LUnuyoRxMkS>0K!48pM$QIg!V zJ1sZP!g!GzKiAr36MU-s26wm->)j54C8-y3k zZs81Y3AejeUc7&>Kjq-P<6e96gLwP?fW?o{!_bci62tn*-V{zJ=+d21rlm!g)#S#| zFW|cFu>5nbR%kIb={7??^kvX%*-_UX^1a!AsA9ygoGkw$W}Ea!7@1A_Dl7%V zlw2EbQ7$XFPIzz%c!5o~tf5NpKa~poI}<XwhAP;->2z@Q&Dshq`a3pBfg+h55Vih*dvNUSvV-T8kY8dkM{10$vq}4;W(X1)$ zHBL#VvDQOpE@U$qLKhChDG9a+o$h5qjB&0WC}H5Vvwg~ecGI9+XB{CqTXN-aP^dRL zE3G50DzVefh7 zlZnGXR`lT zw*NA*;bSM#NGTC;dcUbOWPPHm_@g76hkE56pA>I22N*(y?;H}bDIlK*&B7+%zrO3P z#U{Z=LMcOJz4GL;Hp~PglK`KjUbhKj3xDgLF$6!s1jiq5U5FD^5g+0D`zFK_F&Ml* zNi7Aw{oaYRJ=+;36%UAQz&ZD03QpMz$WhIqmu`?!4;78Y7JMRam|ek5{13{y>JcC) z@>`!jV+lCF)2B8mDhToEcd$bX?is{jZqmvA^QEOM8fCLqFOTHM zdm{hkw`XDjDF7CN@BZ6^ zdO>7=P5Cw3*U^!)q=#>YT1<6<^EBfZLAn8ny4(yS2vz&N$9R6|2g{4sR0N;5i0gkf z`ue|C+DdJHv^<5PNDjzs-0rvxoO2yy)*ygAd5T0sD7~twMzqqOWVi8M@>##SPlS{j z=K6b>+jL8}SbKx~Q9*aVM_nkU%nL7XGE&ka7;bq!<2&UXMx!CRwl{8hS+I^KW%n9P z%*X4ihK%U%sqjGHryQtwuCiq8%fd6#f>XcITrF^h2DFW=vyY7i0WaHf9!>S xJ;VRG>_02;pB4De3jAjU{y$uSB#bvV*nyTGE>5_zT+qOuqP&`1nT%QR{{fCG0^tAv diff --git a/docs/img/streaming-dstream-window.png b/docs/img/streaming-dstream-window.png index 276d2fee5e30e49dd0465e41baa483e5f8ee7ceb..4db3c97c291f12fd767be196b5998b10bf8b67c8 100644 GIT binary patch literal 26622 zcmeFYXIN9;(>5BUiBv^JKp-eZMT*jiG*JO*iUsK)XrzWBy#zrK6a*~vVxbs%4Lwn* ziWEs8Kp>GIoe)9`5J=9(-@iQ1d)}|-+sOqNN!WYVS~Ig|&CGr8_#5ViyoW>%fj}T$ zW1}mUAP_4F1Y&u5kOTOp`b?%Y@WqOofx!)90|W34KiESz&j%ooQGDW^2^jwNQQdf$ zUH)<1_f@a>j=ik7bIZ0-RHUqv!}v|X{AH2G-jQ?p-vhiwZsF9B7_U2dn+IS#TdYyl zoZ`vBmqmEmH>=4kwZ^yfj}1xXYd;>-d-P^Ay4a_@;#RYLXG2N_Tyr)^hw@8RugfZP zD&1JviLh#~nwOH4m!1l_E6LTbFT1W+R2XL7WT$XIq4SyHaGRs#SAsg{*do(E;4Y$w zRVw$gr8MU zadqcN((QZ77q&fl)=oy(owkr)5NY}udNYum={$I^$>IjHUlpciLk*?Qug=IMk(!f+ z9D=&A0`PK=M&owH)2Nv(VHx-_YHO#RbYifg-fBi>SpK)|@9M`FZ&AR6_y+f_XNVi9 zN4;-l8oc~M`gDq8vpt&BUuee&9VQlIE~m_%)_eERztk}P}|xW6O(EAGC^H2|w{lTIIb{ z8_%-Vy-JpgpOc^Spu~J`-eP4neR~6I`SwJ7ku%=|a>#U4n{RtMZz(QA^_pA%N%ediG_-VU@oS~^% zm$jvsP}`x4X3Y+_lnN4RZ9|QJTvs%zFzV>e!^-m-y9|nBmk5{YgIj{-IkLaB4h?Nq zX;ehpu$r?U3)X)wuTkU0Fz^^Oi8XS@WBF>Vy)U$9rys+BUxsxY_BA>QZFra0z!`A5 zVHRnc!$?2$DB)OzKR8`Wt>kB@+?FEH_1#l$=jQ%L``72GnYAsZ1jmV*~E<~_aKQ;Q8^|uHAdwQMG@be&2 zO&*J##(VYgXDfXcQjshx?sOUzZZF)D<*k~iwZ>QUZwHT&CpgoO-Wl$saz{rUKBoLa z6Ke18Q1fvSdf=y9Ecj*9i}J&3nNW^)wsgU!296tC7nRxk4_skmV-vh~<*GBs4x|h9 zfLM-LP{LrS%&^?GFSnP9%gf8_OBPZu07L)(um7om5STV1AYwwxB;u?pDlWCA87mPu zDgeVpc@G7OYM`Rh7WELb`7$PZXEvT-1uY?KscG22%3>@$pYVFe+&vmwX4$-wnl_I7 z9h6{SC=}9jOH1ucte4Y!>>Fcuc0wdb&wufB-3@m~N7|c$Q$`hh^9+4kgv2rI0aY`KY>{%Ac_ZVa;3i3mOE* z>(S%8!>==8Y4f~f#e10m$QdiTl zl8u`FlB~t0HHYf(W(_;`+t1!9%_HSDKVsU8cC5fV5c%xWDBFur$Rk%H?_BYO7eV)n+)9#u;fb!U(G3I=YAajhj%IRb->S)>7Ll zvM--(=>SFD->RM%xYDo0?sm5B6IN!E`Mvh_*zgraS+CKjxcsTGw37-*&a}0Iq+Y*) zGAQ4A?qX%wf>!$6USUbTMr)xrt8L~;l7dGpkN;CL(?hUc85y%X!FSt{-UvUF4_9m+ zm#F0Wvw^IP#S-)H_6%HH&>^WKD)P%EYeFWj+n(Z+sl1vl1D0~#LeML@h!AE;y}&((Qc4$X%vK+za*m<7Nb_Q% zPgRHS9_->MBPSVXE%V8YIn5d&_|4YU-mURk8LUgU)wjGK5%)<+e?f#Ny2nN;JlbE ze4kKR(et$j#~=GFPGoI-#*nWGHXGL5`J&-vwnSLIb&wPiFd(*mDUgENRa^?2UHFsG zZ2Stz5}|@o542HS*-RS`^m#{lZ`+0&YuO8_lYjInR2T0p8{2YC@e=%Xr{8=5Ewt02 ze|`C|l6-XV!{}fg3DNB7mRCvneX74CX4kt$7fqKu8)JR@5?Dd2YH^NYN@Y3OijCw0 zJB~1VqEC}q?L@ihiikvI8xJ^cwpYL2!Fxo@DVsUU=x1NY#GIv=t}knja3Cu=Nr#)o z1ixH|K{UouoBsK^b(ldz(bO8QKfOqw^my&i$~)a2cKzGs^T{MX?fcKs3Q}0Xc4fi# zwe1l-_!*Ih^&RhLpGwTP^!e{cs%??>eqJsuLfXB7wDD;cM!=lZ0Ta=iMSIB3|vx(?LAnZYYFa8ZpANdf#>>np(ky zWR~@=!9$6Gd^r!i>E+oni~8CvHW? zV=thZa3OmR^>;>iM8w*;sUu$kUgpfWs>E=W%49Q#*zUL4f9NNV(dWh6prK^E$_rBBg4F@wrR3V8{C?ODNRB3QK z8f&6a(d#E*wbN5W+wIeo*-6jn%8&FZ4D#JqI$?m&r{&;U{{(5RGuOAS=96Mfi5yTJ z<$2);S!w9!!&L^P#pepshXZY$KK38J#_;6c*`t<|QP@a}I~j8kQ$5*~Leopp$>3Z< zl|C{$*q_Yr?b|s-Hnv@Ze;bwfS*g1T=woS}(U_DSljW>=kaS%X`>Us8AwtraLWG<( z=(#SyA6UkFnxI9Ly5&dzyiK*-{npY1?bf?J!l?hZkn z_aW`UO2p~E`tJo3?C3GK@^xrt7L<7SwPJ0UHVBMg_m$5vZG)CR%*qWJ{eDH!5KFIn zdB_6R2b+p%Gl7CZ5iqAJ5Xfz4w- ze9mh9&kXDTqxVrQf*HY-9TWF+5~-nu5*jW+;p_pyb;^jP~PN@Hgffp+nfVt+KnC%6K=Bm zGeNiQ3J9a;2BhA$x-y7p8hf!g?uNzP%}HNYIVE_meoexzNHsC@1%feMQi8DS^aBJN zx+xgin|1JujPFfK?90lKo+{b2v$Ws-kel*?e>Nr6FA!1g#yNLP_Nd{2ljc3X)K3Fx z2H&`XFF=k=S?rLo7$0>dnJJ=89zMyH)xmJ8red=Y3vIvMj@+Bx-&lBEENNFoLcQ{6*X@2 zx5Hx7w?(~Ncv;Rg?KzO6>|N37l#0Ea z+;>S=Y$_iDGLw(_eV=jR?Tf|TwfmCEas%rePSVc$&hUO=#{pOIUynXqzrl3YJzj}en)L^B51DZR{60)?(&OxI;LPx;Z^XlM$)&6Lm?q& zUR<|1{eZZUvP(CKR?^WNY^G&~3xYrw!=l9EYZHD(elD{uJqCi4Pls;W8S-60AJot* zOA|T2M@uyswmsffJ$%Q?UJ&PS&1~A=lSPkfX?ADB*1@}3({3|B@!3;qy;MTIvl$EM zV#x1C$K5B;4*q>zPg4KnnT-gCB=P6nI?MPdC5NwLEFgh) z78NFo8*SPC0N*v)ALx?f^NeILpB^nj^n2v8KM@5(*s;_KdT@gD1ULu6IX@2CtmHva zQ!!gDMtM3OLe%}M!A<;c2jnNiQWO)IX{xLsa2)F+RLp?5Hs3)F(a7s56n-ue=sx}H z>Z+(nZQ{*#-D?L5+>aqNkqSRoYAbvTb70HW6^jOHEYTWKEn1*fd=N;hZX^$3BETt7 zFR?Tcc5IZkH$&H7wHxNSVK=roJ8>7yW6}IABM=pWMJarf&_Agc$xaOl=gb3K|L=?< z9q--UY=Jp$33#xB;{@CFf!W*X@6I;FScKGCR4J~#9xLWvES8%Msmm{1jYP7&{}Z!? z4!7bob1DOMTvepm_TDYbn`((x_KMaxNCF*^2;N-8o%@~g0AKSE^md$4+0s%%N!U^u zJ`{CdPe5@HtmC?lz+ci>sAcuP8PoX~&-}$zRMF9#Wy_zaT|!RASrAWK;h*%`I956M#f9@ktW#uT%-{b1$}R|4BgccfySNI2E}jQ3h}J-Y)}Jt4$oq4v4#*aa$dcm=jror{buIHJQp@X^?(3a~f|!lTnUzUCiwgmEry!OHPxG zWGw@Ci{@Q6>XAdFm}@~|yT3n|r`Qd@|1}Yr#=7DX{M>))dzC%pl19_wPdgarj>vl5 z%Bw#zar`2fx|+!2{%GAft@I$(AdW z81I_Fy_>OMzCXE)=)eMO1yHlpS#4O>XRVn0kf?2Il>$mlT38~$)8!` zUHu&;7rLTdj1MMEjwiDcxYr&)25{dXuRnnm^1gEjm)@iw~@9M_sC%G^@M0z+83+vpihoP7q4 z%)+3}7qk&KExA4GDftA?%K|tF;YcJCCviw8*=A|o@!UpI^2S=e^08f}EHjFYI!WiU^klI1d@V6O?&A;(4_!6QIB>~=%PA_NH%@@Gs zc6KT}aPjIt=h>*2AipDPie1)!E8Cb9moPHs!Hi(Dkh=>hRkKIh*Hq&8h5fG$BUy8V z^e^egG(=vjsS{H<7q_-mUCKrskB}8;Uz6UyCUhAkc>PSA;i;74$`60eQRj>DUZsRy zWx$IAU0z=Kpz-C3q6DsT%QSj&Iq=&|yrjOJ$49Im%arUP5^Q7{ZB!9Ad`wr2X(gk+nDrknA6w{S`{Cn!?8u3)zteXHl!jD<%#aUrd{`VuU z{}FNy^&H62^)K1#FgNX^gL65kza%Dd>d5aye|&k(OFDw0ynWH28nLrppTFr;az%3G zaw&lm6%svJdjU+jxM3y8Min&Z!WQ|CZ)MeArY7KXBPr;09(Qd>!o>|>w%^M5h zf^Zq_xS&z`EjHh14gGAexqvJXi?~RCKelZD%++pgWus`p1?ru{Gq7zJe`DC$zCw(C1x*9YAtUVkbf_FK6jR<-d1GM-1a{4&Vr*Q8c)rW{;$~o zj?ydAUVrG&B+BN@lU++|v!0rFrz9f0zftzoB%jz)*kM9P+O15L&|p2m8X^RGnZ=mkxnnK zqS9SrNOMPHBd}cWy_Ao(PbibZWHXpgPqI-LO?N-_OBjObBlp^|5r*d~<8C;}7%DhjKYT4P@9(YNcO z*}TPl!CY|Noz9+&J^XKL{#Wg6)KZSsHfdM*>uX@9&4Q_TP^_JZQ_M$>@AgM{ipo;M z%CgYuU%{y2c>7KJ9H-q+W*BkD;dF3h3#Qd7no9GD4NLjop7M`uXb4e(mS_S-8}$V% z8cq3a)c+w)jJ`m*=>)7oodODK5~>cjt_Y*NMH*~X3kP(S+zQ+sDpLs~XBqbNWiV!}wgE+PGxEW;ZDd3(dQHG?_1ksNDh z<792E)^tQ@9Z8f+%uiOtaE%=gomsQBuUKjgaGS2fgv*7zweg>)^P;!pV0RhGeSU20 z)caxm8<8t+n8Klg!`&g{!qw(^Z?t8>Y>UG4@aIogOxb-4`oU<+1!slcb95gYRadlkL@iOwp;}~MBqUsCyEjg>o%y-7 z$apOne%(vTF?vhTB@FubUyDizr=yHeY*Zmx!m1i2By{cIAyQuJUf>u8)9<9i?n_u( z>Ys{9lD)Dj6PHBp2+REx^5>QSoLu(AO&(bPtoFVca)$CZ-)LF7iwb%jyCe|~Gax6Q z{AnwJL*`&FST-BixhPBi?_C5HXQO_M><5IR11${l72% zC-C)Z1F7IkR~v1OCc_Jv_gObR5$`(>uP5x{%>{_B?p*tCd<+iN4Akg&T}&Re@{{Z?;s zMz%A~gzUBx5_7ts%7vR-_gCBDVXCknqG_rR#HVz_mz%=5Nc>4Uvn8hGC7vx39T$_F z1AMNwD)*sZV6Z*%OfMFz6*_m8Tc)I%Bm*J#w>gN z+tBurb_sF+&2XLB#tcIpcIs@Wz2>Q$X~2XX>$yF`WZQz7LSK$uDA#Z82SW;7zb;+B zDu=+Btv8ONH7CQWc;`b?Tck_48aw^lsk4D!{M)%j`VY@)k}|&ex2M4LvuEQBjnkLU zeAq##90+dR4eM_SS-T5;c@1HEt2l-W?kb}9U#&oZv|vkBmRa*oL;Ec5b8|$^X#6fO8v_deRq2X9zt@cS6X<`L zgbzq{*nIJ4@N+5>hJj^vv4mG;LZHm}lflbE@ZZ){R{fLzF&?LWOJz($E6<)c7cSt^ zD<05)r2OX?r9PDTutVJ!1+ta9K3Z%E$(U3A29y#1XVkeQf20`q7K}Ro48&kfI&Gj6 zX5(&d)SJ3y;-&2iy-|kdkr7h+1mET`P%PLnsx!N2ul+eAm83Fzot)xiLpL^i-4s{P z{h!7CD-WO!@{+3O*nkcujDINm>LkD)M{wIq>Q~K&gDR#?(18D53#4#!4l4_*b zE~QM4ixeW#zM-61M%JTg107Eu(&!k6jA>XLIJZcDce}p4dn|C}8y@#|Nt<+}DJX4V zFOlNwDqDT^uTvTegR_LmazlxxGZ=WkjpZ)h!cht2nohd}>>!|iLb zakJGxjv(0XZ*!K7+MRVrEG?m83{%qGz+zje>Z%hry9{{(&Q;XER7B*G-rTCLK<&7* za}l5SXKSuGX_^o+%MN#(Y#%~eqSHrO2@npRwI(vNgiF_B`4{6Bd1b>=POv?tLO*<9 z1XJJh39OS5q+!Tu|EX!CCo87>!bJJB*mrAzylizk>!>=lZ|1jQBhCVU5r%y7YohR< zeBKl%w>E3o=&k^G&D?WqRtnoogIn8dwYNK@)$wm zs^Nw=zF&ptG%fzV#N|ZP zRoN^>w$zNOuTMqx^zdIK&U_c98ipA9xpdEYKi{uR7K_Ayn)n4I&SS*>QP=k96rHS<<}p!uqe zg&Hp>q&gs`jy50@)Tag$Wvg>|u07k;pt z<;=XlFcFx>N}QEAIT3JK;W{6((|AKdtJw1nP*?#iG3`GQCvw`zPPHfwLi{+pFBh7; zQS&+Pd$t#6wc=pJ5sEt>jlCiWe~0kWS#!_)M8tx&MZzLdgmR6qKIYfN55Mgku}Jfs*$^Xps`9B`z>mn+5Ri7_g%zhr!4yb&C}W$|38lYFvYmW3`R5I$)W zxz@|mynP3Ug(|$?_DlyLXDTP`yQ(4*``^RcN?NxuK2#|Z!oa8}g zrorOE>OQL5l78``?Ta4glJ0x_4bbmbqPK+mf~{mucuR;jpiKmR%%;yNcV>ySx8;g` z?wPjV*|U5-y)bJElP{t_kqK2p`@g2bkyASFz;Woq7ng_oloX%y)Wj(NZ1h9{O(%tBx_)HYk@Y|bQHFzBIs4griO2t$eP89m z5>GSi#p1_Adn8=*33HT9KisMN@o-=);-K)uThR1ePgLU8;PzJTy9M`RJ{L0(8`bh4 zDJ7m(t_W>|2gWpv%Q7@<4}hXIZlvq)NZ zi`!66Qxl3Ru7sslzwWTi*a`a^i_7uRTV_BMgcq%RI-q~aY)m8Ntd+vcgp#ooc@_}Rqx%VVSWKvo6r8D`*%F<<<(cHV3ozxs zUh4C36+m~Yi|x^8lb!1d*27GfOyNKUf9H_?t{;FeerK6urOJP{+O=`C4LRw-0ge-L z5lEB0@;G_GUNlH%x4$?%xeWBr2<82UY2vJ#FXQ2x^&CdpJ!OfR0Y<)|p)&ZeU0m(M zgCs);Mx%{`aQlaLTVzf=jD^94cxhJqRfXoTnwO~=2fHi)JE_SU8PLW$sJr-~`zcG# zJp*7_XHpd}BJV_)4QcvGZ0&N;n=@bZpOUzQ&HeYl%dvxp9MK**k+Vfqtro+eUiVlq zD&&x)#$n!HJxg1+%(h(GQvG8^fl9DG24iH>sJI&cUa7oaC{5Y$(=>rvk}o~!9a?CY ztB>*>--3-d-;6M0x?m993B_u@x*XWs%Q<~p0d;#*-a~QWmmY7|IGtb%{>KgR{>lPQ7G%)%W3%|JYlI~Gj0g7SN&8DZb0s9LzqTetC2u|my!MdcuAdU|#zddqeAP;wTye6phARvLbV{$9~Fw7-!Lf^U6;y$l_pnAN`9Ajs?( zRpu0!zu8HOpYSMGwJa`x*_4@;-tlXN+u7>re%by79s-w_<-on}LsvIOX9?3!z07La zY3gPxHNVuAt4#WYjL6e1=6gXxh@kMrSu>zf1K)0m=-2WiZsqt1v?PCN2W%^Syj^ig zm?{uqNHcb^zrU8`cRczhT!pq2woWpRHtR6|{JzL`Gj%#Z3d!G>+ zX|0-3wMkEY^bl$6RQx?TJqDYvcGg}>JfS(F4y)y6Y8z9hwxlNQIu@E5@)0|4T=;42 zBoxlJ(y3j3Sl_O$UO&wF%i=HP)F{uws3J7Uh-F!i>F%`=CCqb(# zziUdD98od-d8b=L%Tmg=dopO;ZP6Ma*9W>IHM8a{yjJ3UnJh1o)S8e*Kpn`GtxMv~ zogXmyUXT2MM64&&s~ri-$1-F>D?(o$?Zotq)YtS>W!;7lY(Qr!SqBKMkn39x3*1vd zg6%ACe@0f8dcK88Xy^`BILZz404Pc(1zg**D@@d%v-HwW2lKsW+k~g7YKS`A4OI2m zr{wlVfSVez5-8SRjs><*03kRr7Eomux zo%I@o8R8gXJ=&DMVWMG z;&Zb@KeqPli*xT6Fwi!R$?a%vt2J?QfJGtA869CH~g{Am9`Pt^#IGkBCKgIoL)qs*q$id}zOn z+y@MMU5&klo|zQOopKl&y8X(v47l;H1mMPh?gj7^;0Jj9=MCVBzz=|50!;%J0FwgV z0=N9{yZ*=R9iM8jX_iwpD(EcoF5{Nd;yfF@$x*S2{3*uXyR7ct!p(xJA09;rTW1pW zAZYp_PwgCUM_QLOGNSZBnyrV}U-3~#qk}ux15}T_WE3y_NV_NJ8<-Y)WkKz44av7Z z$&TlLyK(zV(-ZmN2}K~!@}V=a-j<7Mr{Vzi8FUdp>nni6AxmyIs}z2s*-<3xs<3`% zdC=Knce|`lCvF35xx4Oih}5&MKCO+Jx(gwO|50TeBGpDJz)cT5;iX%4`oK;aKC*3z z_?Ouh3ozSYX9AE|s5!XgP3VwNrM-uVwp<-n7+Q#(G>&fI81pEI^y|cA$_K>4Mhun~ z>3v*@HR~YXtC#+>*Z_u^AWuH~WsrR{9iUOMmSofE7IZq`K(E-XiS&a$t z0r*q0)k7wq$6wCc7e-0hL-cvojtqvCr5qbuHxIm4Ja#k5%w$Y&YhExq%2%znqF#Wj zK99_7jh}c!el`^p(MGu690r}CY;M5&19eMN^mqS6F+z2B`;p8HB>vncC)eI#XH4=u z%B4$q=`p0Uz0du5*~_d5-FxFe!3hSh$hB6$bH!$k7BvTzd`V9ZnK1kt4)g?)n((oo z7%ckDKdvaA#xa?B7-l5C3?>4g3d!4HP*h$(lbCz6nZ-)eb9Mmbk_lAP{8?-cURS6X z?(A9cilhAI9vdSc6of!1*8KoM=NqNQ{AfE`Z8EKfzq;*GI!R@tbmL86c2|V%qhMXr zxt*~tDcFeDM={0^wzf-7_v=N2Wk$RwE_aST0q^hID6sgn{l{GdrtC)`y zz0J^yEo+(3%?SzwGHxg{2C};mLYqaqcey#=r;SpMRh!EF6XyilKdHDf9_yCAuLGxdhuIGvE#R$GHM>5fk!Z8g( zF9ge`JXnEQSPvY^LTdtM}5;;_K(hK^gSBb{HcHILw*!n>+J)G5?% zC>tE%@N(mHcGS{D5^8|)`^H1Kl-q1rtA$j$w=9I9Bo+!_u|()Fy%2&*o&Fn@FxUK;={Tjymb&l4w-3Jgca-sZh7)5l} zbhax_Q|>= z?v5{+a{AW2=6w>-=RyC|@9uH~dJHUa{6$cu zuG*HZ8}2>aGI|;^+%66)ET}(`aN(CS?LYhSnZCg|LCpzi)v6@Elbq=1somSgAki;uW&!g z7hib>{%aBrp(;kN(rnxf5W_IbHq%;3Wwf#7%Y3!8CWi%1Xp3XPa;?7d`C8_!1#alc zGHXlw%bghyXX`VtXCN)8VG+c!>`8OAUt!IQ;TXnPA7;;YQVn7TOV?c@wt6x7mx%V4 zBP$#3F31wYADZMGR@(g@Xv1$5kQ_Q%?g(qyww$dNKJrfj9PFlX%RNI-Dh~i979e=C zA!09L%CvsZp^k}?3n;<9F>h#Ah8ZvD?uU(LSY^ocV=K zpS%d|n(RrKUo7S9Mw2p;MkdORF`N_Y+z#g5r^}~`c+Bo0mGhuve!g69B>fTnQ#9ko zN};_FcMes-)nDqb;kQFp>@o1E9p7VJ!F8_VQQG(95(kYJprQxlJa254Dr~$UtKTa2 z(gTtwjF484a+UI*iSRXgT6_)*VFE`H5Z$R`-rOg0M8!K~qKf3}KE>w@5%B8exf#p~ zGV={gIFSpjqJ93K>}sxnqQFZfWGt^80M3~iIHbr?6ye*aVq0zfk7kZm$eJxB1ip-7 zlw^!d<01fZfvn_L>P0;BmSZLDGBaMXq9{#y;yuvCe5oj;MCh<|%y=X>tN$u6dW8cZ z9srM{46)`I%nbE!G70U=X+bf;1HoaMdyuOC3}E541=0Sh6g2AWaPT0h^>XLD(|ww6 zg=@3e!{+&dMQ2)4#Vp$(AB%p=$CRGNR^^G_Oo8_1%#mXUobLwA)YT})dBi#DIAz!6 z_#z+F{G-J)<{}4Muk{|WA9M76o^c#tu$Uhz1GKb7`!r}jtxP&TO7iNNcAJ$ZEs5P+kp)=`{*4YFAB_uvlAR|YFspj z{CpY(j=a{bv504gbhgsJCe=->zuu@!{c1Zhumnq%2jCn}x~_M$Xbz;EtH~Cii{!Zg z8Du5uuJ261nSVtdf%eH&h0(~<`k&XuuLeHQ6R6-4rd-1_<%?_$BLE&q#(gwFX7fb# zh9wV4E`4Nl<&8mM*h|63^=lhzHqUg=JOiW3lq*hOo(XstZ~uds`mc2#B>8x`8&q(7 z$pS>Udy&J_UnT*)+IwGtF2su0&sE*sqBxzJ1zsiHCm795`$K6%?lwg)-+{I!~#E6BIW- z3*MMD_9BUPag%_DwA2fVO>0%Dq4+ZBmll#fV@+sshD|1rcXjh(nE zezV<-sSvef*m0$nu%s7Ys924UMVyaJvb@8AIY?@+hrgcr3he+#-H zKoh@ED4)IPj}B6ymgUHPdNQvk*xDZUjHe|sr*l(gER$~I_Ns)|7Id+ybK3IPH;(Oy zi|ac8fAepEH#!@~`foq_OU(THEt@akR=_)f_LRc!mguZl`Q(4Ddan`lzpMLypI86) z18$$>3W&Mxu-EXTCJlSLZ>OYKP69X<|JA`5c+0v(jj>8i*1jihvH7I-bz}-&adfI@ zp>sp`M$^!BkA+XCRQKwkUI*Awx(#z51CLbmzkHhp%k>sW^UnGaY)ERFrFZ zN~VjPP+#B-S^?JOC^H)1Cs%R+lxbyeHYDafQeJ_$r1=g&Scq`#ko?ha=pEc`(+7am zli%nY9&*m1Kk_7-Tf*J)!)OWso+HfAn7<3_$`RjZJrAFWx^nI@(OFJtM`MlBji;vN zEolk27Xw^qX=q!XZySC-c;+`4=(*J@dNRS#ZYzu9 z?W&JWwishLrqu8mAPoR~Rp2A4kXhxce@Udc!s2NAVln5EG}G}oT&9(zGc0rCi++n$ zg_r14^6Dy!aZGT?wil4}bc(Pi$1zCgO!>beuD>Qsk-14|LqbE)r}tp-_$xNpJ60`% zy5SubDPzT|)^GHlJg2U%7DarpQB)`DoR2dJCSGQsM})pPx|$v64*F||KaAhaNOw^# zOaS#mGYPG)--0S!ILQ=#+}P5}?{WA{bcB9?uBcHuqgSZ3E0zP0H;Gz)`9LJAO6Cda zJxkpi>=;+9HIDDinV;ou&wWL&+V%!6{t*q_hzdcz8#63^j-3n)MGU6OT2xIf6lj$V z0+GX*tN_qVGy_G*M|c+HZ{d@j$28kYk6r}o@bqJsV}ETtPQGn{=(7~fuQDGq5<53! zzrf5NJ{8BRw`|m1e(>N3t(~1ZR(#>ez$|_FY@SGbf>(zK+Y70Lw0HJmK8S5$mI-t2 zxlM^-@Qq6jH6NB1lNs^_k+p5-Qx1i>ev=JM@5=W6%XwuW@kWq{R^5kH%B5CGvIna> zV@J{oQnx$l)(cxlfyU6hTtOsndQdBt%;FrT2u)v>tXMn+P0uoj)$9PkFGuu74z0uS zFA$vT?!yH5vi4RbKrIU9(h%H`$C zJ$|5aPf8jKJO0S*q=%)C^2}rhYKo07(>a_(Syjf9A z91sXbtsHFwB4cVu)U-myjb|&f)*gC?W2ws%76%U!;!DD47;Aq$-_b8z-f0}`=pRA} zFzyqbkhR_BcwLv=^KoIS_uP4ln8PwV*&Y?}fzc=%Ss&WQ21IWGLcFw_7GR~h7n{D- z(`LJUfLwy|Bf1m^ltsA?ukRO#m#JVn}(b~`%QtlMJ@BizaCPL0WqJSI6s z$pVn%8#DzdT;^+NBVL$!rHEc-PK&tgki`(FY-cb}5S8c}Q+h}rB3p8-2Bd#Sm@VaQTuH1<WZzc|0o_oyXx#J!E2FxCE{LSpO7#q>OU0oxNn7rQkZze~ z4ygZh#kZm{UM{nB4$d(l_W}ZACvdOOjhHf%*AHy22yjN76c4v(35wvtqhr~rJDESc zOQiwuf?YebR0i6Nw(v3D=}*y#7=CER($VW7MfNVr3X<4p+obT5iFf*x3_R<7SumzR zm>p5f`>h(nTamaDKohYR&2i0Wdd5cWH(O~X>We>)eB>kH7Fdpn2*8)cthsbt+vMbi zX+lPvwKOZ{DShjF>R-$9WlL(vTe2Ka*OV|oVV7(>sDyv_l^1y@0ZIl*AMNbvX_f_& z%9_|HlKPi+8!lRKlGK`w+xoI|?tSKP&xh8|4hEnfhrBz>GvACp_LHFyS&=30o)Mj0p$0kZeV0y0^kV#>06=oai*x zJ|$S@79H7Yn58|FHT1bOIq<9oYw5A~Y;D1` zWloMumKZb2W^KxU#*(veO14&lUZ2m;S&#IJwG*o*W~GFgACtk$ zbtiR{pKXw|WyrHpkp%0xkWI>{41=e!=~o~fTdrH7j}#2Y_)(~+;=`0Y(LepyFD~`( ztKk2s=G^0%?*IQkQIdvI2|09$bQM!X%#u`cE_LM;tI(Vma%L-X>OfM>nR03==ksAI zyHE}zhmFNV%xQ+%ftmf@bbY?p?en|ce*b*``|Y3i#{2z#z4m&)Uavh5_vd5Ga0PEz z1d-OcmAF}QW?u7aFSEu=&|7q_^)LC}6!!}rYsk(9&^`_-uW{X-`eH@OPU)O`Nb8Qu zH`d~*G2q8o4@s!#Jd8CTb3%AY)_ifIc|9gp$d`U9yeSCUtNb7{LqR?=FwOzmXYgIC zQQ@ZC^zOb6Ap@jimEROICf@11!qf?9XP-5h>~mcAut9G(7a4P-Pm>Y7`gKw5wWZw2 z(v68dgvoa?>bzk5{-hyi^0E)8?i$>2z%FZ*7xri35zp=y#CuZo9nTpW5s!stsJ|#b zpS?So#2^scQq^AZ6TN?FZ5&5@x1wf(b`h}JIlNnUz-hb|ozA~hSQ+y!w`=PnIfIsN z5z0p@7hJz&D-}`B)>$~*0;>C)8f(cb@J`{lPgv$Hy5B0j5;L~4mfnQ9{motwQ7Pf02Y=Q-Ji;j?jx z!Nw1M*jZm&vKtjzS`w`JB`4r_-|LFg2E9>a+|^ZQIRO++4LJI)X@3t|S9ZDa_8L_+ zu84o8k9eT7P){H{9^7-=@TSq6DDKvKL>huq4|CfKSHU$SIZA>tb$GTO(Nv>JFduh6 zhNy7q?CZ4wo$SMhOUxjJ+4SA)Tsey8&!%C#ypgd4avnn~ELlj=FH(#tM zt&qc?j+?dnNw;XJSTPq7l@yAn;O54&{2Xi|K7fbq{c~w?Bd207~i3{x$ zw0zMs^2}B~b5$NsZ-9ig?J$aY=_P>~<0VOc;r}T@O(bMOgJ9L>zcZGbw(Vx>QuD$W zfOtRr0}IdFP{VxmKfeozNo_hOfG-WaSt|IhISP15L4n}~kCvQ=Oa=b)pRH4ne~1h# zS&K|)KW|p4I_+?L3%A{JST@PiXIf#bdDhU%R%?37NG|Y^ba!C=5Y0Ct#BOr>m*~sRb zQ6YE)^Dki$65B-0V&Vm}E%=A@f8W7E(J=fdc3A5Rrv#`IICqu9A|q{HbVMX{UifP1 zn;xFp#?~CAr9uoFQL{zaR7U+B`OaJtwXC+9Z?n1&A3{hKX0&(dFCUBJqnHDP#jLie z!#tm)a~s|=m?FZYF9NhHZ6053F=5m0a#|5Sb&}fmH7Ohb$Ir^A66YR=Xe8drZbMEi zBmxjS6^4-SRMKIz7bW7CZ^@Pe!63I51)3JmBa?+XC=?7zw%MqK{f%uOjBhR zEtGhvy-d5srM>!c-`*r(oc6oKq%$gHa~%IU*zFp(`c|pucRz7TV^E|G3nZ;0@`ejE zXhnIm*HWfVVY&0t48?!o;3vQ=IU1)aK4tjtL5He;+|rvn!xI^gmc-X2$&ucg zzZ}g@hA=qcYcDCA^$%wQeti#jvH^O!kE@T$Ui|c3Uv#{Y3Vi-`OCESi=R9C z0>p1cvt}2%P@}642v{n`Mg6mEwibsK5x~}mP2Mda2(Jr6rPtn-Qj-LMZVrTWpD_|# z%bo`uj_8?>UUx9>nrH75TZdTp3IdvFQ`*DT8GYU%WD7#MXNzCknUVDy7`@By3$G)v z;O{q)w7kI5j?jky8aez&IB9ZCP0*=u`1-6DHg`UBXL+G1ZY{pOVe{&G}=q<8O+s)r`1)zB-Xn*@B#x2$(>bdXsK(u|Qj z`4zBEn^Zyohj_j~_o4^&NQu7%7GbgC@gT>&Q>WLA!yCL8)2kpQ_B?G))&~9b;8deH z_i~qLN?QdhRtRv=e)pm=FRJ!t3@%D&aaL;^r{vW7D?jN_hw5AV>?Ldxp%XOe9=>xS zU$DS8Q9k{rXpzA*qU**L3A%!WJ)de$zXt`EQ0`-;^3M95Z`M9xBflJ4$z&ljmyVh> zxkT{}QNla+0_9VAs3253r)Sk+E)W}KG4G4iqgh4tDSRm~lc358{NB@?-W#qzf#7V7 z&NE?q-_L~{k~TfI1;pB);~SaJf2a#GSQWPBpX`0zN6tT!?ZS?0;8m6Bz?33VGx-*Q zP9FOuJ>4v2UqHX2KF%^43Jg#4t46I~He%LJ078>+tMpK1%ww7BJn>y^$< z?e^c94%tj&Z0(a~$KvL4|2}e)Fy-r_I-rP((Hwmx65|PBT48Ivc8cwqlrd+fj(cjK zjI$C80oUIg&8I+=?TS&|D?kL$^W9(!buDdrh^Zl^JAnj({gj>3K|mA%Be`Dt6YQ`4 z-wk!7($uWO57})8suoWE1Hbwn^1fvbq2O$Q(o;o#=LJZwkC%kPfm4)cnYW8eJEU)9 zX5Q|r3wM>1?H`F5s|NfoL8F4QYEAQt?HcKNEh{d-`Uf&G0BRE%d~CaKU0axi$WTWx zFleXa41PVWfh$oz3fR#noW{*PT>#eA`!GZI$DF~ za3YTK^1K8;XqUmCTN8BoNE{^7PS#GFpFCB9EC3?*k-=XnO6rE(q@1>-uT2wtpq{c! zc*!2o$V}(RtoV7dPdbvdPQ}4xPVpEr()f}k3*#uJkR*!0b;JGyDB2XTqv&p6Uro@UjskV*@s_aG$}f!oGcZ->lc|M`& z!9>fYfZe{WLnAY|tUliOr5(OJ-&o97SCBO}0`jYc zO!pzYMLse798+*W;g#_`@OalPFP`>a_YjW3M;tG1eep0 zruk7;RvJIxs1l;&Maw^1)3Oejcw=E_t45f{abkxjFsU4FD;ofme@YeW$DUeVI*u;k z|9Vxdi^VEg#+xftou&hdKp}o!JvH=idt=0tHNtONz7J!!xNa0JTY`XE)6?tBfLT9T zltQJz5IA)$W9|ax3v5=_iSUAQBH(TOC^RA^-pOO*z2?)b+Xf^Zxh8h~D2r&E;Ah4=jgOxE ztaU$Y!hD_4n&^2!!iMJE<8*M*e&s3G-CE74b>!+S^TE=A6t~=fQ z-$g=>=L8~jrp_s=7?a{zG(Mx>a1)|T8IBp=v^+y!b1!3gG;slJ^F_Yca-q#MHPGd*%1}zw44gX|zU8)3?8z5Ka(x->kSRVov z>6PUaq8xCdF19N#Fv%nHt^}0?P?`P%K+;wnJYCa3D-5`Bj_*fCrtr8zT(%44Ua}V; ze9--Rt#kG52LP@MxL;wy((VUId55MMO|^g<1ba+P zT8Hb0d~)*(h6qdPDr?R^&Qlp706`3GT}?p$2c_F9uOY5bs$zgLyxE@mWC{j1 zfL+-bL18~8kAJQiv{2cQ3XJGXVt1^B;FcDE6@OxQhb4P}Cgpr&Y^))$XMz3)Wqw{|4)xTZmj#OwNwF03?xLkML`6 zqca#o0T4N&*;gqNDk7Y*aEA>yG|X@o1f6qQRu_l0&QzE8@8xb^i-MhV8VTz8Fkybb zUKz?kwFzY(3+-Z|+h?Xmsa5b#XJm(oR09s`dtjK3i&H2X8x-$Bxoay7;zq8Mq`chk zAe!}JPaqA*{4hyaSg~ag>u|AyR}j5(eFV2bA6kellYX<)W!eAvi8$eivupqc@~aQ4 zdGk~xWP~04&v?xM>OZ{(V^8t`HcoC}i;dfO0Quk|Owozyq(I5c4}G*NWbe>OBJ9N2 z821{Pwjwk!;u;(BMtX2_`WdjcJ6g=|wn_ObGm4ezU31!e<%I(nAl?Ox8sq?(JSuE% z$Vb$C5mt5X&DUR4#j6rTcQpkKepM!6s@jq*QteRloY-V^`S;4eh?T=-vh_yZd(}AS zt812V^HpK2M3RruixZ64z}hF7K2pHKBR4ms za4rF8(5<87X80lxyf4Z;LJZi(-Q9(j0y2Ekm4in_!~dBtVL$NpoFAVy_!eX#*?^V~K-B#L4N?xe1k%zxI1K(=sNZ-#5@g>0*zssGJo}-8KF#%=A1^PMHZzj`2gX z0JlB(CrdQ&VZSr$$uGr1bdV&pA+Py_FzmQavM^=sNj+ONw?(kk9eD?nT_;Q-kbICw zY%S@;x#PbFd)XOY`Y}FpJM<41)M&*6rgeWc4mB#x++a~;;Ar~jGG;eH^wQEdJXJB- zaGiC8vcx+dd8Ta#=t1KYOz(2@y#uM!75cFBR1P9`$L~hJaRcnzDE%|V!kt#>V+i57 zd*~DrWN?9DY;&q?@D=s)NAiEZ_qNmRcza9AAd$XP_EQyqzd(sA1R*!wOysN?EOcV1 z+jDJQ$NiswRm8>j2UT$lpYOJd<%m}Ws0cW-X(M#ECmcK)#3 zjVGE}^7zE$*-RH$@Uv?(cGWjNZx z?_W7PX=$IA-sj_9>ftUAc0IPvHf{R*Zsgw5tp56W(T#FIq<5amhXN_T8dKVrf^Dqr z#}bJ^8m(1oaqW!l?AId!dg)-0;$P{eP3ouhq9!a}J-(c2FEm$}-+wJIu<5Kr!|o2R=W(PpK!sOZx)gHRy)l_ay?aTZB)K7jfVLg!Di8!+9{8Kp;e3Bb zTf@cKJ=msQ;Ukrp(Baic<+xmuS|xc6H!o@q#kvO$wO$UW9MRG21rh>HP98{YS>T3x zqYHuXdk(-2iL`Qh z&1>D3gQLfc8@$lY3=rtZcS8?x?6dsW9esc_y9WZhfA5K?(V%+M$9)bhD8u4Ge1Of& zBYI7~&mVEQ5300#%|0+IDm?&s8gsg+y@|IcromMSehrQdu@;YBp}S(Zg4!qd{VX$7 zkK)9rgie}B@bu}bRVFgoH1k8U_sct~@Czj%P+Ubpd{+MIC_ekESGc!<=7Qn8H;XX?X2kLuf&^%y zPr*_PE0w5Z84q^ivps>O=t3X^xNBcWnV~alJN9W%BrmKS*=SPDT?=g1>PU^*kO;)e zS+>w`VYgfI>#U6*HD;MB$R0wpp|iV0Kx2i7C%6DyhCHU8ApdfZydb0AWOkxT`Cb#n zJ%+~4<#8vEFicz4m>NRM8l1lPz)C=x))(IYVD{lslVCk#Pf)Udc*kc z)~=+`&I_{kD`NQOlf4xSslU~9>)ymAfMqHmmt7l6TZaqC3sD)oc2!C4Nei~PBQ2~Mycpt;m+J16yP*^&=6pC^U|4Q;qK%t_adM3nrLkZj1Fd#6(D z5P4?T*94WoOW!EZQ&6Z4PcGj@+}1G_rv-E1Zi5+ZyrLBzZfUxI9TJ2d7U8JL)HyNF zaql)L#F4Jt6G)eRyh2j!!xS799d~*%F*>>INs4$36xDl>{zug}{jUEZ+2FY)gyawV zwLe(Gpet!Bd#^S%nxUmfwOmqy<}rVd>F*rR6l_gJa%YB$fM1Ku3}2nHksCoci53)= zz3_AWkz&GLzcO%s;-eLV;Z=Qaz`f3jk9b($i(er6?ANf!?AC{N9(#7P#Aa2jk=glU z*vm)tyK|J6@KTLMOW0u8ne3XJ=iKKs_Pf51bqV096q4GR&>idaK!y7s`Ww3E^q+Q# zfyTa97+DQN;@kWW;hBCFg8~!l1QuF72Y#G8S;%XG*NAJD!QVJUH?}e}AZ=nozGGHo zX!esXzk_(qD##T{@0#P1jorfY8iF!)qhd8Bj%;B<8tJd?lU~-UU$#0^Bv??DTv0Nt zp2KHWQ+5_Q03$OkE)+V`=;T3Bc4^l7jQ4qBmtY|ZGEGK(p0+FKFt5CtE24rEcy-k8 z5S|)sPJkY)h34@Ua8XFQ6b){%B2Ow^mI)b71_mfsI4X=pV+UfTEY@M;_UTlOID`j0 zr_t=XfoUmZap5HTw9#BmO@%!Bw!cx5oU98pE?ZXPqgn=0Y6TzL0RVv7iQt=I(2`f& zzXfve)5&iZF~azJ3{*q-c)_*}WWm};UD2-mw}4oJ3&jXS;&T6zTlc0>Bw_o_6@iUKTeRRB}? zjqqD~s_k51DAFx8Ufb(?x0q&adn890xY90m-}_?(-jTT6=hF!9RK{xU=L>XEAU8|N zX^*U3`x|esVO0&WRg0Pn-BRKWR4sRoIuG7zVP^m3C$vWX=K^6+{)CXE=ifm9j?+rM zou&Rl5%q!RApWT(i$VSplTa{Pa#6zgaH`iGcKD7e2~?c_y@tP%gqGMA_c{pbuxTgV zXOrItCz)kdu)Vt(Z$*5jcd4#yHPhwmwHVZp^+hJ?db%>HWE0=<(;B+3}US;s{ zyJmsBv(;f1X$*5MCv7ntv+j+VS53wFRzNNr^Cgy9C7g(T1MkNYv-e%6KJOp=rMHbf>d=__BOKA2b$2Xh`g)N!ffKWFJ7H% zc}yL%aIG=yEM$q~XMmr}P?U9$&x(k2q*66ik zY3hvjwA&p6zl}R=wKbPm-)_&wcndL*X`dzQT(39yemw8Rh^>cQA$JhxTT&(b(upr; zoq-?)59xD!T%+0&ajrx`Kku)>BfCFrZl&>~QbPoM_gZj1;sZ047>fSYR87{z`3At3 zlKe~cK0IJ;Z|%0pKDK~7yyo1;(fq3o&>^S!Jn19cb>naQH39tAvA5rzaMg^{A(lN% zV8q2c9-DA<&Fwg6HUk1x05ZfXp>)KYR%>4r!_u18mjQQf^Arm$AwEr tP0Wkv6{w;WE9if3)ct>T<+s;qLw-8ljS$v!)IQQ<|!engHJSO|J#;1uX=zn?X=yrDM>{hMYtwu8gop#RV0LqwWa1%q>bZ|3=yYX+ zL%S=qH1wOOo|N@q$j9dGDm==APqQ(tiU@`jQDM%(>x)&q`~)#>?n}-*Ae0lO$H(~K zHFMTcE3XmERxB^)wfU2f{@LzBt)&*%4=fr#-<@6!2*ZRofE$K$uf&Wj>XIq7z9y8E zr$LzFV+$U|r7ML(YsM8HjF-TbIim-LNVq!5&xf~{XhN|DcBA)hmCqQ#o?-K&Aq*-V zw&XM*$_qTT!Evj4MKUnv^@T+P#{4n$!I%I#S_xVrD3!#0w3kIpnO*`rVuWp*ybP-q zL$r0;4A5X~^x*NY*d;Ans%8eW=$f(^V@JpmAkv}X1(J{}#pPtjAVBvzHLvMoBVouQ z`c>^1r_9^me=!P8xtNpG@) znzu=VXpE62;DVvm&zPSjYRg&+9}Yo91bMR`9VzNq*_Ibc&6X$Yso^4cl7NPhznF42Qs^0lKl3d!^E4-(oqpJoPB9ymu>jq1#(3;8Ezv3hSjV8AGZw}IcaXC;B?9kmSU$!C&5D;M;j4xwlNUv>2a%y zi{>K{W!0k_Nf7H-wc(llCs#v}w?WC3itHzhDkCrW^mj?=FiAhT2tvgRtgIw<2t@la zS}uln+pVmeZyb)p7s70M1uAWO>bowcE2=neY}C$2kvic8PyRh9EX~f>$A7jSf zw)?F&GOIpYm6Nx1={LgotW)Bb_#m`tX{7iz`?koo(5R4N90s)fLJ7=g0(}Yz9KZ}NPzt<|oTHHXh zC0Ucv)7XO5wu2q)vmrbLP1|ECI)t=Bo?* z>X|-aEO337x|R^?E|MO~A{VkBrt-6QnHQ}gz4gNI;#p<>gVWnsQf?8E-!>d?n1Asu zr-?Xd%begir;H34_+Y>YvgJ!$Ck6V+b!IAxE1qL}_c!twOvnb45Al&h)>sAH8r!M7O&f7To>?c_Vix`BfEgx+Mo0bNkO|rr{E{EK?lm->%7@vn5WHc%Nsdz zs~FK5B9*Xf8o1)}f^);D`{#2l2+J!Vt>6=6x!@C^5d-qksOhnU-aQ9*Ybth9RNBC{ zm(z2)caMVP?&rRF1{9T5B=5<;lG1d)zmtZUN^OcjnwK+wS$r8Rxrv3wkWGZH#--viVT^T< z>s+tnoojGEZw2BwPV41lyaQ4!=oy@|NO*XFHI>e|488bLbXrq7mGe^cmx3z;b;BbC zSVE;M+JrY(ZDcjSz`M?t7)YS3e$53HDXX)MD$0=d=ZQB+w@1xSG5idP3%4gGt?}+- z;bEYGp>&Buyd#wTEw7Jc6!PY#`0l|FhWSuTb^qHr>=~^qopcbaxYbv&V(mHmdFjXT z&H!fM+g?Q-xo{{H3dzk-V3B_cgXMMFPIx}$$a`=tXra|7m`V`mEo~+ZSB3XNhoHky zMv_U-kJ(QoNRMdreYvJ7dr^+uoXv~@Q;7Df^h$$_mh6_@+dE1&xc>&7q&&8}z`R?a zllf1@*-sL?-pq9?DcDVJZArv-68Cz$y^n^zHcS<_P6H1cm03ql*;gC^YCZm$Zy^C7 zXL|&4e{J(t^jF<>+|p5vBQAzR^IJ&Zy!4iAy!TFQy!zl*Hzq#7x`xdZ#>W*0hX?g5 zhtIUo(D<3BSz5lJ&V7{PdjnJVpywHvPQTnT1*1YQ-xO=KaWVl|IP=?!*ZA#_G0vY? zlT;z0?aam?mG@s=;pXMIVj2yh{o;(AEaV^C#4rWa7)yHb4Gc0nQXU;(Uo;#adS=Trw1xi#kH$JI*wN!Eb3>e%fm*BM`>$QxhAxB zY(Cj0$ARAy5g;b!H9OW@t>+nK6@x8I0A}Sah|WC0Sh2d>F8gf9^lCcNK`tp)HIl;{p6*y)IlHIvZmn*temmtq=i^4+XHV z)wGLXdRN`Lz<+y(|MY;VZ;E?NnsWn}+lAH3HvEDn%>pB;(q3d>PBlzWw?;bxDPGdx z`1W4BRyev`8i5JJAxz9Irl-Zbvg)k>iF#*eYF|zIN9YCyRIAs9+zkp%0`^GI4ua;h z)Zf~S@ufq}@>KjeCoJ zotg(xr=1NNe?V&eG`(h_#5oT;zRk05UR~LB+IZOKQr4R*#+*Uw^CICqsdnYaDeoP7 zw{X8Q8*3PH331hj8yZLy#%h3SU9G- zwvUNbY1BnGfVmehHig0&7jbe+tOC4YxQo>!<@nx)mqAsLNNp%O+!n3}7lCsZjVJ-h zjsHRvzjo%RcVGA7YUV(3XNmDNg|*B^^;|&x;&9wH=L5sO%Smbr;PWctP$7sZIF96a zXY)%aVO9;uYwGQXmhB2V>gJnmPT_9k;dv_Z93i&8PeB#o&K^7u4lKt%0s-9T z6dDCCYOmnOJ%UlOhr(%-(h3#q|E@EJRIdG7^Y&MXtaN5&cW56YxqSP`k-@wnd7@FQ zb>_j8LwaIZURq6VzpBjQx9uCdn3*3$vflFcZB3htANv;S5M(3zY{ylKbDj!J8@(7s z{2zFgnw zQ|HmYjaud>xy!k~nYPHq1%mFOi_qJ*^o!O+)t>|oK7jiR*#^(nD_D&jorgKx0B}Hl zaaLh*hmk@~nzr=t`;5O=mh*C&5>g)+cK#Nz@Y2VIyqd;Xrass~w8hW>Do!3V8n-d3 zT5NQJ`QKIeA3SW7aB__@Odv?CTqNEsaq46*+TjDl-D+nlHvGIyVSD47KvuXI14_S* zh*FM!npg=XV%@L`IOjLX017GsQE9zG?)IIKyuu=FP0li=;$w7A89LaGdWGw+p1Y{~ zfW~_qr5n|+n7XH_J>awquU(<)ro3jQrs;JGI9>UM?}RHy_}K4!ygKo8{jDRxD)6>_ zblE0o)q0$E<1?UC^g(;|Dsx=>{Z-Qn-F+<0Sr2<1%SIhGpHk;?=c*I`lgclr-z1IF zaQim=>}D5c3rE?dQ^xz((x=vFC{5n$Wn@WJzHos*7iSTED!N~0Sqvx}G!gg#P=@!vcGmJc@?jjI=!K>2MmoI* z)H#){<{d@K!D2*t{ZG>+nbR;YUwftV5XRQW5k%Sle3%tHsk71bwGreiFipSJ|GLm$ z=QkM-%=q=R^$gqWP}p5J|@-+&#JTvH~9?mWQ7tXt?j%|=Dtesd5C6&`IzR_1AKjx{AY}tgqWowAFIX0J~pL_MN zp*L`j?Ziv5(vO}yDg(2xn5$e6V|`jjt3+s7%SDJ154||1Lo?ix%HqpOdK^U5YA>a5yxc#ZNsIv;C4sK}5c}u%j|oOW2N_m8HcO z`(l)M&J2nflUTTsY*$*w=|SnNLl6rVynRky?V?rLbP^pmSjtOf_*EOZmbiTU`-uxo zaDgix2lVtJ=-ckvfI#a$U>%iI_-denJykmdVLxK*!*`Z}e+A)Z=waof+&_Os}OTwynxJ zq&6;2w~WDDaa-OsyF98Am5r&9_;qu}pqB0I^U0}=4%NEU`PwF>LnkGL}&|CFocUFspeQ+r`XHWmd8^?I3V%M{79Mu ziaZ0R3>{ePkr+vqHQqm<#tnUPs{9sJ)qCASk!&+((Wz%|)MoQpm%M$|){@QZ49qUe z9m^4~D7jjA8zkXnxw5(6;&y&yfvL|EuDTXSlv^SDC*8xvpJ-VuIB4au-Dq)B06$aB zS(vL3WT*<^uwJtr4?lu!uxl=6H+o--u7O+#<{vjie=8`lpAZF6e$!*I7@2T>nj5*4 zt}qw{-SC&Ny5%UNH>Sak`_ZIYz=e2uC8Zy+W6L%Z?W^P| zdz8qm{TA)EU03I5Co`?ek8jw|FUM$iqf_K%O9-Fr(GU~RMb`&y&6%sTi0kTgMdX~b zh2NJ`LO;uP0bNBb!5B07_UI0dM!NU^$6Wk>!{EeymDJ5fRO@3&W`<(FihNhAt)a0} z^kiAxn2h{Xm?UV)XzO)k%kxg?#qY}u+!Wqk7Wjv*=E~a(`;$?-XD(DM89?ztGjLRH z3&D?uEjLQHcg=om1G&e)Fil z_>-U(vzS}bmw(alRhXoKFaIkb(Pv3hlCRk*mH`FgrPvL?)*>5pmznj<7a}7T1|yHm zjZf5@KPbf+uu!3U6&yEzn=(~aOB0Y4{hAFaK`ft-&IS`iH109%l!3@xc(d|+zvUpf z9Yo*i*&EhnkgnY#iDmr6-9Ci*53VA;M@~9^mTY%5*Nh2X#yt7*4;ob`E_^-_F#|xS zQ@uogLM!4HoG-sGI#K)naFpit@l1}{fYa_8hkE@d+IHzV)5Q1GYc6s^PpKo9v3~)K zp5tWGnq?2h<;#<*gl1nSWwJh{<)^>f&P)n)`=4I-BbDe?hfuc_+9<3PPA(PRR|%fk zWEqW3#8d1kS)0-0JWNf>#lE{jljpwhigXTZa#YgEJYsZZqvBoP$E^K7R;p5>Hy-O# z_J$a~%rwI0QxZAvRf`i-qR8jm-C}U58@%c6Wjk!CkNDEL|AUU>)}O!BBhyA-#xfu| zF`HXNIBm+M)!4OJS?Teup4d+1s+d7KmCk~2lZQe^DHZO5Zt}3uETyC9p#O0(;u}^! z0x62Oony@>ij`w?dRZ4Pz9Mq7zg=TnkS~*jx%Mp*N5XCJx+t1B@GsYN^4yQWND|a+ z?uvn01(V<4Y6xt=EQiVwSP?eqb(&oRSX+-p0Z2O45cF(Wz?0F|#zUc|W!Yc!6)KVR zrA3!Yh~baE3B!y8e|gMJMN|O;$;X1EGP|%}5h}eCjKxkLD;+uNQ{~g6wm93=amkq! zWyU+VP(J$Wxd(oqjxOiSs)ri3oa}ZJAT4x4dA1u>EWL-8j~PDx;YIy;m^yfuP2E*D zNzgGuB{iN2@$Y_Vp?xbZUhlJU$g=c>a9l0R0JUv#=v`@%9?j)hwmNl(*{Sfn>Eh_KfX0CWl@oL{qR02plHNE=&SFpTE;(&mZK*u|Tt32tB5(bCSfU`< zkx!c}iddVA>R~|XfWO;+itcXZIYIjY1Iuq2YjX$!>o4*!71}B!bUKo|J5aj*lFlci z+YgflwWY|)Xud2#aoF9u7e<=%YD&23^eUZyJQ{+CH#oRow<_km_-JVKVZ;PQl}<>} z;UR8QR8E@s84HU%1X?ea3BV)jGyx`LYjT;6VAg@?XCQycK$3$}m3)VFQQ;LE!HuqF zdrgG$%G2Tz`cWH{8-Aj%rj}`HhS{JvOUmUx@9+-)l79TS5KGHxA^ zdKGcznNQKwYL%jf(#Z$Yu05SEmSisx#1p(iuho^&g{u1DpGGLK@QD02vyK}+^_$#& z=c5hk?7A9VL1n5x@$vn$S?8`ZjKK!Zhl)_i%q{FMO9pSN+wI&(Wkkz$PBB}2F~2e{ zf?F-{|JHQXX!H~6i$|UWH{xQl!mSp*%OfzUWVU@;wr9LaUIsD-> zZvN*K;<0y6@+)Rkt$@@`oXphub(tPTaN!wYJ$cgpINfbAwRO$pWhUlbO1KxUM5k(l zUb|EL?E7JG&`D$0DWPGUGHLKVbPRysW?_ytVRA~wxXjjDVX~W;oRWa%W_JIjAGEkE zZwhrWMJaX09&Ft5;8e4)ShOi2%nGv|65+sRsm_a;VtTdMODBfR#{% z&shoCt>VFT%ZO%0SR$FwJzw3EZIzv##`1sJ!c_tiJ60tu}P3dFT-$mgeEsV+1+4 zKS;YmNMXBCvZ&blEJ;p+1!z6c!`~hxkPgeXlU?a4o1~9KNB5LO*?~Et(AeiVd;U!q zgt+MgXk2U8mhNzzngy*A0{;TQ{y<5A>$KGJRm#^_*(fvt` zut%JiJPn!;vx!)P?nSu2h`XwYi0t*O(%xJ#(^1%MiF8}oUMM3uw@EYH525)p!H_f} znJ)SzrN@p5i7ixp=tc*RkBsE9@*ah+X=x;!yO<35@q$?uJU1QVVpVV}cr z^KzSAK+q?p^Rs=V1*Q!+Y<4cKO6<=}NmKAXBHz}4+bN$m$+E%P)yykfmhM z_rM-bWCV5)I9OLVc1QrP40L%7Gz7LF-{$4LL(^IC9T{P@ymfYRs@1gB^Ubxg`VnZX zdz}&BhWL?v`>CRM_PcP6ameBLxOeL-Lt@G)QSXj}9+{z(*E7lY^+I&0Uf1|mUb`SU zvr9Ce)B6oPqp8#zd5r%{vHT{DNaly3u=V+X$LM?PNC%ykcBic>X|uM=(a8s=X{tID z@RrQm8CzeLSJ8 zs_G;6o2$^;?A0|c4JM;ZCWyRmu6tHS5M_3?8WaP{0?mOs_pMW&n&x#wIiA00h#jT2+WJ!VG-&+jaP4XK3-p z-E*vo__(YMb5rhnZKK^p_-*yRw>`hQJ~uG+IbQDdXMHTJN}ZfC;rKl%zkR0?Rxa2o zsET+zxNCS@0XiQ+E-PI<=L2P2d_Dp5ZwoYt$CisapHmUQk&_mq=v-hoWLQqm;L-z` zA24hwZr?+4$Rk!(Jar`6LvrAixq8_RwI3b>=(Yz5Qn#jDZ}i~-(pBd z`oB-)uzI=oQEfH*(Th-ADsLuh$XkBFeY6XVQg*-2%s#cszJ%GSkOp*g5WRoh>b{rW z>GN`yVbR3zB@i4QL!%3MnL!6))aH>cGr^?F9?xKcM6o<#`_S9Y4m2QF{>FZpo^HM@ z5hcz(({zsSXQ8fQqhoWy*e8fC>V?R!ZCYIu{E_D?2aCUbwl;g(KP@n{YD`sae1iQ|c{9k&#H4*@Bz{TrgX)9Cgn0ySG3rFn$2jY}gJ_7x@)lapuPBhvs}6MMiN zx1Jpqf3FN+f%MamBRcL67v=#567k+~3H2)HV0%Mun*<5VDo4i3qreYuSFjelc&0)0ItPM;-=W;0kIQi|z@8MKz zKOW`3Nm}k}UWqC1qepIy7isje`R?>k+1!0`4s(?w8iHtXMj8O~+{E*U|9x;C<9in1zDq-5sUoI;fIvTOhGX_}pNl8l z^G7<6=$F@fk#Ra>G@rdWft)qF4d49UZl6c9gwpF?ll3jbjyzO*29Sdc<-uJySJKQ_ zCH=)X!$MKSpj#G|kWc;|5SkE^QQ#&q8EeU0`VH0e81c0VkHvXbNHsRvo>cj=t#FO; z|7Wi4r;PRO@rw`130aVY*tC{p{wM1g+nMfV-gD-3;d}li$Vuh^@cI+ z+V5;!rzZ4$@&_h%O3Rrp&B4%m$&gx&6fSl$+Gw^-%ekha=7n&stv!sQE3oE#&7uvU zQ9iyc>w*UA2Za`SXPdsH8yl~cCo`!~ZYB-lv|H&VR{2cyPcx4~?l_E@6xIBFo#L;S zlpO2Dq^6T>O1r=%w`p~CbrRIVjwWt82d?#Y{YONsd^Jf0TbVw&9Z?Vc}azj1P$2B-I#q$f)IT%`&-xmRes81dA_lx-EV*I2P+3@uC_KMd>hUhP3c0s2PjvlRK0@(5DgmdK3FgwW1<&&?L^@A2 zCl)AT9y*yDM6?i0Ekv#+E zK|y32fNoQHjn*ajIBp(9KGsv(y-#Sci|Cyb7z#Qa*e>57u^Zs1509jR*}7k)DbAe< zMnz}Uv4;=RC|0d(DuFE+vGRSL&lT(jL5+fsudV0egKY%ZV8fx8$X^Wx?X8v)>raor z1rWB|^*i*M9)*Kd7fJ1u-q{%+V+Mv^i-Ai%id!wU{7A6%-v5#NG5mC3@B1}nOq|yB zi~UfouB;M!A2U}(teuy`QpeoL)!G!NvuGHcrWe&6l)2zkvH1|@l{Bqa&qXHjtmW=M z8uRZZPIc(fXmYvULjJhs{8=n9t2%d_tF>kViP2VTWk*J300+b+pBsCxRlI#J5DrR3}tJ`smrXpP9c1~uWBfpJv%mg%RSI4G%7Y=H* zf_G#sH)-JJP)Dd`RqOZ~CEOkEV`0}9(U@55tJQ4qWISQW48em3`@UIcA@SpQH}|_h zwN2z@!=4n2y-m9h`P^&W*ZV8DKa!arQvf5pMZ<=2@FfN`#Eo9?OQqSNa~XHv_;OUK z`F*?dpQ`}g)RYh|(XWg$WarJp2XS)~FO0aI-QT-ZvK~3bDC*YH9+&HK-p^HiE!?*8 z{Ou7iOSR#6&1vvhtuI68tJdu-B1NBcvLg5P(z8~Poi9TC8mUUypNB8JWN;*>HLdI5 zv&Zo;w`KIGNny(tf?0mcZ#nx`?MVH=lnMCDCo3gw`K9Z3oYLVPMsn9vT5cm3x7K@AahJu{@_d{5;~A#8EJ7p7}bm)VaOh+v!Kw|LFPr zaOg8rux^`b7V4lT%c1P$7bRLc2B#}H@T<>mq_V9W!5ssWPj@cY7iOWu+q8LbC=K+Z zy7+OpU;K(c#gP`P!y>~jY?t8G9P`sBEhvuRzwUlS^htH`1uf_#8bn?})Duegnb8nE zm#1*q<#R^DS?pP&K=h5Z=d5LXBTu}88=ErO-;VsA9|P97AOLHN2Y1%e%qkc?Y&*e5 zk@Qaw>KB1tn~Q*;l0M5fYT={s2-O^kW$oKN9WYu)>T+SzCE7f_H+g}i?LI4*jcgL;Lvd1LgGU3`3k)(Tx0ov0Yy6>$nLo)} zMAoHK)~`}eo8~^?C#l@juPQE^1ZU1hdgzpjbZjzJRbGmLclP3xi%;~`p0&KMe{#&P z75VL>p=kZf(omQ*+;VHm;5Mrbrv3)^Q0T_CkYG{=NxNu4uZ@`KS?&%)loxIyfgD6p{g2T>$DYedgO9psk zpo=&IcCn{_shfyf;tkqXHuU}W1EF`lo??dwXm@|@cIA30c73aQI?zLmN=Yb|s+>5F z=5YOzHxz*<-{?H?TD?Nhn7PhjCij8BnC`$&Vk6Otmfo80E1L4XIW>nTy+R^ZU~vCB zSMp)TUMlA3_ZR8t+wT#b#p5<_(nMTrgqO^J-A5V*caqCZ!?hO_6W&@D8I{%yizvRC zCY%1YgN9l%?&s%Y)=1Y98k>Ro;@s`Pjv7*e7ykiHe?sgb_L~fG&?gOzF>J?TUp8h^ zuZP<$lFAnklACBtHaar)YZ(G%NKwuH6D>XMRE>nTY>oe!P)W`ZyRXIToLO62i)zs5 zgMSUTm2Y&Yj641I!6csyMpd8q) z98a{?zEB(&E@3)$_;x~~Pxs|fsY7(1O3Nh}Z35AABm?!f2y(7)j<3~a>)FzR+xJ1N zFU&Jy5?S2A74pdFn`09S1>D@>!Cq%h_~|g4+8r_ z5fS+k>2t9NT~QaO${WOGf}slmjUFETSN=o6)-9IGP7JCyW2F~Xj`f1SXv$LUeIht0 z6eJ29etKcte6UGSvUyu_0fj2W&^Q2htrRxoU|%P4xumQ-|pY{2b{s0iS=zC!L=wx-F&%wi(T4MemV*Z#Y=+ z0TQlrx;I?Yn1uPzADm$V(8G9-uk$#4&)%=eXM)>F7m?o|`mcFlpm+9$t)I-fb^PpV9;DQrQ%%gw!#;^|+UD zU=+g@LlEz{%WcY}+-N==8FysBKb<9^BckwDPei64wD?2t0XG&ZXR`KV5Z<0wSMN}z zZAqXxtt;blSlg4j!#O3|wA!@=`5LRAYpkTKa%xjd9I%sMe-p<;Q6kdc$(z|>LK~v2 zh|W2aSf%vu!cOZL#~Tn@=HOO~)nCzsA{?NB+7a^m)Q9C*r=>S4vpWyrFDxkGm{5ZN`ilUjU-C{2`d-s$ z6U4W74!^|$wt39XB*dWXC^E3OXVT)6T^V`8p|^ zy^jVDMU-5PNOWE6@C&wIeo{8ttLeQC?O*ZUaX_-W|ClZGZYQnfOR);TL21A8wglnzgzCq}F`Gh56J z9?2#wrv&f~6w&sRK7vd_{pjf8=Z>V&bC7VKTwr^j{pXW6YyiP6Jug6kO1f+xC$feo zB7Sk``GL|GN{95cvn-TuQN$trL1eEW^I1Z29!(xqUR_@8r0|BPhtIr4NwezDTHrOG zT!}EiV(wEC99!o-7TnY3Vc6k`3k|R-f9n(8C01txRdsbv7`E}nyt*_)JE?*~^T9BR zO+Gz0blvv}4pys#NP<`j-NKwT_7bGqFM8*2&f`e1P6+QC&>G8-G#Z_{hHTGf)j!bv zhER(?i)8mP%t|a_F_S&Y7cyp4);JUMddh?S?hp;KTe)gpag}7QCYbV^58!fnmNs8eu|hG7Ia0W)gzQo( zhHg-WF4=i8K}qe{Nxq2dZOV26WwU1E!V>&rEZNgip!ddm&O};StU`x)ky?1 zYDW6AGP4@9hK`69rW89Y?+WGwBm@v|(>p7Op&ebHAo@(g*A6Jvtikg!F+56CV4J4T zHFH(`eH!s&lD54M!oT;oln-}9Q_1+sb*E;Y#r>J%Dxj&IrFga+2(ErcN-nARC)>9a zQ?ms4-vMdA`^+7IuJFUTBqa_#ob=qbSGE8+hsC?prkAN3-g7BXP)wQ=xVx@_H;#X{ zHwWsL=gqtC>eFV;!M@8V#+tXw&+Hmw7$2l)t^kWBQS3QP;2}ulk(Z*Ll@ixHYP8PD zwbK8fcsu&#)d4cMzF~@OZ`YS|fptdoX^m%h$aFrds8qAE<7Ix#bpSbCW+LfO*fWDl zTcNAX)kd;Hd4W3%x0s2<+?PU=tj9LS_LV1i-d1e=+h2ebI+Srx3Le@5SMVFQ^jEtF z5s5qVp3g+|63RY>c{DYdyX;5hD_9ih^gT_YL3%5@mmNh$oNp$rW3IT{{(fOVX6~6k zPvG&vkW2ivh(f7T&7g!uneBCo3F$W1ig!*-;SSgxs&WeU|w8x!n5&O?mU0 zYQmCqa)mjC7R43?*O9~oYgbmgXdjL=uG4YRkExLPsVoOS^P%1E6x9Xl4-K2efyxq@ zBmL&mahK06DEbuVY|_#KAhp_h%lnu`sX7{m&U}b>Idk33@^qeu{-!+p{JvQ}C(|dP zUu^dRxBg(W0LhbIGFGO^PkdwcY{ZS2tV6)HkQAsz4EaO>wUL(zq{>^CABQCS8i`0B z?KIETI^E>1>K*@F(y7e^=~m{p{a!fx6s%h>9r~&Pi%-mcI5ByhPmK1}3|LSpy-~kG zAfM-MXK@&bl)#ps@~aL3$@u4PI$>}Sr5Z?SbD#TPVcNzW5~YTd0C&n6(_0>ZO9vx% z%$X%3)217lphNb4TLtD}V<}#g-YOh!Ek>3sb_#};`{s4vXEt;b3U+c73V~Od*#hil zkGcf8D#e*bE;@u0af;15Wm!wdAVc%XNuq~yplS4$yR)DwfS%0)&|^OCX>B>x2m|H< z$z?jpmkO&yY14yVQvboJSKlPf2mG%~qorCIktYL>4lg9sgm}BV24J^3Z)Ij*d#Y>= zit>VDR?RCCW9nnBW^Wj3e-db}mNSjYY*sP5Rx`N%w4s$pr9?^}i76Xq#p!d6*0^=% zTU|n|FZ@d>YT8`x)}fqbC$QB_WLA5-?aRiM7Zog~e}q14@seZ$c(59a18`$N#pMUBGZipDT88b}mJ{(4a zFnEPDWYj39SM8PhxD7xd5fvrjxSZEqwMKv(q|CdEm@a^#T{qnMchce73SMs&4}+?p zHTg;qw8b@_2Za=-KI}GG5JX9*q=D-O!-*yw9+}Hh7wb`BWBpK#473oT5BSM^E zPF>b0^t6l`7^KRS%ZBK3-jj|*J$0x!b|-G z^@*O zSHNc+C45wCCx)L^C$q7XFt;vL)TkYockfBEESOBtENCNRX9g^<_2-~6ud4a)uY46o zDZr%gu8u9`#z&?36SzwqREv-sR~=7s_8R6d&Hyr-`LMLBH*-HeT2^tp*Ob@qSuIgS z^dTaYEbmCifM`h@8=H~&d2z=!>1;76ls=nr?@T;6XLS$-2UFi)G&>$k%8RJ*_bVDu&batG}QC5h2~RAMm+FWv}nEmk#fSj z-kQZRCjG02*4uSvoQG2&Mab01gja_!wIB&oMd0{(eTX?)}8#Imgtw6lf?BS zNer|$<_>EU2v&OGP<4?KNctmXUyWx5N+eEl-NMkz6q0(Nnz#EI!0ypvo;#OUuyWxY z=+iS_^E+5RBmfk-+#qLh>@+*1URIlUl36R~Dd*@@B_Kt99eaQlOrDUh;)+`8hbSlV>zHgeownS83N_#dx}txMIO$ zN1`Tnxa~zLenlZI|JmZx{X~3}>}Km3k<9i6-ob7vpFEax3hD$mMAum#%rOX9si#>n zf!~#l#2lMzopHiN`UomAs>E3n{9DfeYd$&(Ff4nShV#nWn%a;PmA>_hxCC7X-}Rzo zn!{Wbu(r4VD#$&-%i{SRVp57xy(s2WdjhzmJdkCQFu(P$OOh51%7Hc zYem*t79O@J`wraJ*7F4EOV-vVXhOYvffgZ)i#N<}tn2#giQ~^XiSwLItNyku@GXR2 zv8^?IWzsxcoT9q3KB>}lB0=W^Jf)5WxiQMZIoIQR!y1u&JN)CWj*>ki3NkwHmhVqT z72lVcvgo2tA>^{ZeYEECVvF}n28RPq3!RfpF!LWDbF5&;(Cly?S^q}f^rA+(R>yHH zdgbdW;ad8R z>@7l@x{h>iHA(PN6qFZ$eQRuEp7i8hZWFr24YXyM($;w0byg6Xk)SUH$IrEJxE;l5 zIURItrMZ7s*`$={p78c$%ovhSSQ-lf3QOkbT7RSYZp6<&xl^G2GJ_~TiCOVhL2Npd zMYzYvqh6Yj%>06$of9!D^2;P0KJrRwNasfwpuI%;DK?gKtfYb9HsShKOM?9Ytrd)= z`)Kf@8Oo*}uu5qP*%999-${*#36gVtg*Yl%{Q&3gF8eT4lPVEK5IvtU#0-c>X`S_t zhW!4UvAsUcu2oeS6vR7Pz;g6GUSpXP<_jy|YRoX0IBfs>Dj@5t_rIbf9)^b(SUoGu z&r=}dEWm|kuZoy)u@F!f*k$={yJ4_+4{`k`uetuU$bnC%pW(;Vr}x;ruLkqq&f`YY3nYfri#W8Z4#pj z##7^ujDMj~$8S0!L&rW%PNj1o5W4ObexpQDY;Ni5-R*q;4|HIFX)etxNha))FbeZ$ulz_NV+eD`tup7U>ye*rixX zSDeg9JAmzqawf)a-T^)V>pnhvj&+iX${UcGD&{}%Q}X=?(FoP}_Sz>SgAPhsKI+G) zxfnMVsu54+VDavvqZXFtqDQToq|5Xldd$y~VT6a*XEmB-i>2GAUV`-gzyAv$&*3sr z>`zooy!p*Ilt9bHB6``kVEL5&s5=;Pk@WjTyiRemR1eW$F>Qf zm*n9JG+YRjeinoMJ~)#H50F6Gj;(z0UJP47!q)7kBINs{e>g2Ic{-8huf%VQZ^6*L z=1&U>O9#c&tarUiaoT62pX(V`-fx@rGfV{o6AiBn>j;w%@i|ay@LN;a{!8)s71I+b zeOILzOl@-Jw${yfXJi7dJ|iBgTe2gSOpXSF_x9~>RQ_Oe3~WEvw_Li72d4x%HlL?_nn<4~#?XOsp1Yw^GKWHO-U!aA$QIb(m_{vYlWwf`vE z{`A+V-M_B-*V;lklqj0qV|w#WC;OiZP#-nfP{C~ei(b*lu|FfLOJLqRbpBwUmudsjG zEuSfG7o1%cC++tj+AAttb?LA5n4dO4qL;{_=FQlCQqq~WBj4iu{-YS!VyKJ@s>!d- z%m}J`IiLPRq1t$>R2ZZwpSBm!?%6f8Qvc#|vfmF(#7c!)x~UWE=HD3*YVT>VO?ZtT zb65|A`FGnmFg)XyZhq~l^S80h^he1+bVfR}Oa6)J&R3bn8;2WKt&b8iI!OEL|K{gH zZZr;!qRI!a-TvR*ktD>rpj@)a&62Tg$=z1O&hXpFz@WmnKJ?Q0h2d_ zryIGad9w@xjxaidiWcDo*0RXcD%i!}`V>slcX3 zY%4B(bH2|SD!-n{O*p{BqtfHnH1>vRcA9Rnjn8I$}kTaU_>q!$vz zUvsaIqb@w&tsn6o#la{OBSQwxLdIM>paL@3`#%P*Xxcd!=x3UoW~&3zwhL#Zyji&# zq-}y>%D=6PAe~{hzJO3fY79)f=KVZ2~wm6 zA~m5CN~9Bd{Vpued%w^3>x^;E8UD!_fZ5kBYp*run)5>MsnqmZsO%;3AZVmzlBO0< z4WJ*-c}%$*d1_ZFIrgnXKmk#c3oc54S!Sx>>OeJ^ItUc1d=viKl*`Vq1mjw|pN zxz+T+f1_tXfEkPwQEJMaZ7_uE3t~_r@iFlxogzj>aSZQ%Krv}9g*>_ikLEjz$=}gZ zJ|B3{$;|sPf6Q(0*H!lS00S6xZwMusbE@E0rLZL5AgNk5I!EvyumaQ|v5U4BQ`5VU zKK>ZZ$8u8zT1SzS$E7%NET#+0#QUJ_0z>!l56af`sHdK19=~8xaPmHcr`#R172(#D zW&|#_Lqj6kO4X|OX%_s<*UeE)UmhZ1^7|rAIqb71*MB6LwoxihpvOxm#dW7eZor$o z7n^x>p>O7f1FYpPN5=KCj(JT~c!tc@(As?XYG^ zGJU02M=hM+9s6O7~Yg-36g3uD`UPlA`X6 z{i^27Umuh_2laQVuDS~8l?&qN4-zF^d!N%M&{@@7d3Ey4FSW{BxH8_wnVz*TRzc+$ zQnkRGBkAX?h=x5!rR%l&JEc@zxs0eV4y>b zk=Z8*I<+4|L09}*pLl}p&{_%QtTUJAc@1;EY0ksjPBa{)E*&xJqM2THbcolB$u9U~33en#XMy4{b=`v}oYE0ZAHIW>?@~MG z*Vx$QC_}e#C>AO>g+hZ=#tI;m^e5}XlrDCL53OV~?D6T^1Qkc;($eY}yEa&t& z1syt7$1ZGQ87%XA?i(gJl7Dz>r%l|?x%UN9I*ycPCE>>b`Iep1 zPIT=N0aP$Gnyk5%&MlJp0jDE{U~a6RL*A)N)BNy7>o)l<4d4p79GTJaH`w2Lg)cGh zXw$1UO=uTpnKL2K7Mqm>=T73N{862-0~dV6IU)rZEc|34O$SZMCG2pSO5Ig!i0$vC z4AS?3wyh1ZTIUSG4)n9=vL;!;@6X}57cDCgsnK=Wl z{LFW+ArsYBa|fxSy(Bl;Es84qoiEt|#qtc0J&>I|$Rzl?1BrR?p!|~Hx@Jp)6GkK} zJPiz1Q-?4rr6@zqaAKL=>)H_dli9RYp$z#%(IqG`SU~!+6Rw5=*Zo3(MWDpTdcs<9 z0$Oazh8AHq8HLqlWC$0xnd~<*$51AFpjOFg%CgP{=B`1b1Kw7}`^(SY#6Ion>`>d2 z!(QXaxL33~lWefUvGXphSY`dZ4B4d>vg2Xbgh>hl4<9S-L(-C5@W%YLpQcx&Vl4LZ z;vAZMqD5C~t+Ig1{Su(kkorpSV(~xg5{JKPfVxo~o6wNT98Oi)hJ2%_TaW!v+qXx?*@Tfno%{pAf33{+*>z8o)9E&u7uDc{k z++;*&K!gP*6mH+4uFIV<^vGcA8oSpXp#6lVt8?`R$LNTjg7dWdO7uli{#vH*pGrvi z-?fl=!bf?KbJZosL!?U}yQ*#Lap%7FTsiVFcAAGhW;_>1@`8kUn*O^GIMHv>nsd$2 z9~=tor-sGyQn$}(-w8Z8CvN9-veVy@Wc5iH+j}VpDU0=wvphC-82HpY6_vJPf_u9? z<+ga)ttNmTrQE;t%8zbZqGY%d2y2L;X|imvwvY1A{Pz7iCtVK5h)US_pgaqBv$m>$ zKRr(3a-dn>fKSycRp4-Jw5(}91N1Mo7rKV(dvZa4Q@vZn_^UdSOg^>yQt(7^>ANfm zq6_BwVIW~}0n{5}1%)Fbyqs+iXxuC9a9PIWN<2jO{)-942wv zfySv-_naYmS1LZEewb0uY_aXeZm)-l*o6pb({X<~&_(iLHL=!RdkPK%{KgyOKPw8e z+9ctLq38yuXH6~7j^BD_#)+c)%VPtw?v*t|y0#`Ibotqea#4op zcH~tWH_{@~FnG$4UsS z(oRmdX$`bUe3CNezibR3>l4wg;}*#KioUk%pKLw`;ARlW1!PcmF~lS5Q+AuP7PDPX z`9ZZ3n=&*r2a;|o!j+PSXbd7%=G~}hs z+-tuQyKn(of4GP~?LJ!)3voGWzX>UX9+pG8I@$+A|m{H@ub4%qu$#|NJ+JCSNR zE_|YA0vs8&fpY_a1vMaC1q*TNC_3c^uPtM}$_B(w8R{wtc3$$K!veH%6xIQrcAlNR zp+uR*3%FpaD~Bsr^=0=NJu3_%)pL#9iNTzvOo{+R++#;J$&H8ItKLC++XW{mbKl z5$CiF|8+#3h;sXTo5RtFh4EbuxQm3RmAr1tYGRC8>{LwMocSklI>@(dn`|GgVS_&f z=;E|935i4KQigQ7WXLUiPC&72^*nLvZw13=K=?{?bDG^TDa{Gwe#=*Za)l9(@k^4T z??4==16%4_;?wLj$1<;#B0&=cN71uAumv2HelsX@sq9p@1L=LMS5OnRDY*v z-Ecsu`siEA$G7HRhM(>Bjn~dDyJ`QD`(2chnOYa+=+Br>(|?A*$u=-FQDJ^mh^1{5 zGL=7bk0(o+{nZWsm5B)`mt|d}Y485;y73ZW92roCZj0T#5V-zpkif^Qmw=!AzwkFQ zC7*8<#C)5*1o)rr^>(olU5q^oD3~FUiD~~TSK}@PyZ!vugZ!S_M68b@Jt|4{hhGdy zMdd{XFBgXy^pDQ?Vw4(J@dLA^?x*B1R&#DlBOQFYzyG4M_<^Zqk?(>1y_bOB?N0rw z-qR4|Nd`zQwJu6~- zJB=E9HuO-wqc&Cmxt%t&^5$b@MeJHObzQB1x^L1hYH|Vw5RWqPl*|3$F7cn9c>iY_ zGxLbXV*Hhk9{MWyJ~}c08Yq{tJl4dl?nev5)ncY%&$z^bV4k0KY@cVR?Uf>p;njYC zO(C!LFcx#6ue`4UZb# zjtoMDJzRa7qz$-om<}YUd>Is3n;1X%_g3By_ogrMHT0HRjetNoamRXdGi~k z;C}`RD2nB0WJne`yGXZ*u{XN<*VN$Cx0>GUfb7cR#xx$SzD;Kfc;u^6XJVx2J~_|4 zfq06N&C#wfHwLa~2m*yc#Xzt8|+qDG80M35P6J7kI)h3iW>tpPZc+Z!iltf{^g;7oilA@;+PY&JU;l zk+M6ajp#)^t+;HfC{`Lzbr_B*(`NyHi%`{UI~+-Zr>(bDX=4Jqtd3cd+`Ro$&x=~u zJi1q(_*qY&10-`C6;e0Ka6qds$gyY7uFL8TGq}pGoYKD>(5LJm|^+%zTE$+%QL5mOp0%={o_mIg-TKrFZ*~Gz#Ntof#jb9@D&MK@}Li*PxOoM zdj?&l%$X*2kOCVeIrki^qKR2jH?k*c1%|rqlYBCAH|^QB$3)_m*0>R8)RQV6+V~@U`HYYljo{F}YPr#vOoJ|@>#;J*P6BIU% zd9Lvja@lOkX1?hTHXPr|GYJ-NwqZrI4%36zbDi=@0=_*<#<1)iuC9uDScNhuqE;QJ z4L; zINn$}N-3~3r4M$T#$9?6v|4}~ucX{K&FX_-zygQ_^z1$koua&s%K^ZkwuX&#qd5ch zHA{l@-3*pOp4tG!9NQl3?mM8u``nr2(jl!I4*F6zu_`O%d7uD06bUQD+RrCu{kmY z@VW>*Y@W0K47s{lV(9gbS+uu^Z!i zAK+*u`YjGONnYA&?Tn3acQDcc0`*SGezumAx9n6LOB(GTa>pqpi=xr+DDA$oNU>a1 z>b0=W(8KrCC>Wqu{|Ap7-+ty_SgFP^KEpdjFUMd_RIUU&;Krk#CcljlTlp?JEr3iC zJ*#MC?dlnBW?Wjh+R6S+0Cz@IGGR4Y`#g4s?@`rj%4&|@=dGp{#wP1bUc)wXo1-~D zsUoF{@HVD_I`F#AZMr2th4U7?=ie_dRgH^20$D+-|EZ?)m^OoMw?DHZ!0K=WoZ5sR zk|9-)E68}QqP2pzjEk0m%2+ozXu|w+x5ROJgXRfwyGKy9pT|Pl&C&DlR!ds%R9!xU zOn_`ep5lhYb$q1N7SL|$DQSUdy+YZXAj$d+S5fmV%vG7Pn2Q{oD5{+c@7G?Ryc>PS zv_2B1^DnIT3nAW*p`HvS!!EVM>qr(LG3-I6!$HfuDu-qod%@cXI74iI{HlyGbWT#h zKQXS#oMS7lB2rMf&UhQDuH=kXE1P~B_-Y7*lX9GfpK+z}m@@ffB&wtz?u!Df_Eo1K z(`hJ7&{(`HLan!F#ymzrWqp4Q9(1-Qg;ldN#EO(s{c`=OQ_L*#8_xX~KCKaqzxo38 zp+N(svMn$n3@#c~$x$i>?y>UObk+0Vo!XjK`q1Y1hGvuFhqOg5HX~@?y@@g*6@6tk z$cD;fskUydp17v_dF%sky;G&G{+eS8f*@WjLhD&L*W9$ww3dV_kH0a*+2iew zYS-eK^!-WZG`3$1neW4|ybWLf=TaZPVVml|?%gOPX}C!L%Gbdc;2h0)>S20j;S#Zc z9hlkAwIr;Z@S`hq%Eu}qD)L!QS)p(nBn_l}EIF^D)N%=0S0VHB9m$W5^jtGdcWG-# zJIXz8d2K2IzJ#hrjg0o(V~(1q&8;N@l*1q6GyDsMCX*Tw0%y1JHl( zf6)b*o%j+Kr`}{$Rr6~L*hwKUBmm1dej~7{KJE1iLfiIv=Uv@B68v8&i?CO0Riyig=8``Lghb)^6zaJ+;j&}ZHUm1U;j^@_eB`bEV4;SB`~BO)zRsFF-gR? zE~VdKMw9bs;MuR`1zRWzr62ZniQnrvkmInWGfZW*tW@26O_1S-2*)L>tfeN-b?_1ZC z|18|!5Hmrv`PRFZ@y-#!?*ZP*!sjPblMqcI2^G|_F^<&Sm+pim}G09F(YR zTpO=ZwUMVljE6p86^C`710uZDjn{z(-?CpG;uNnu6_sqjz;h?D0YG1+l2f&2$!Zx{ zSo8Vm$+{!Nl#m$zQo?64{M|j_pxouR25YsaG+YuHYG*#-yb zqDM|+7N5Tlrw*q)yG&xS!bF!EpCss(@hpr=(sL?Vw(78oJyPM3Y=xipz)22Za1XR* zYNX?TB%%!KCOWV7^-A1v_iVc{%~h2%I2#^h{M>xdrc3%|lyo5L(O^^pH`Ms2?2}Aj zcaTvXjzIObO3Vs^m%H3^_Z$d5UbaFlia$t<@2Lp#ynbTyo_H$H%W_q=lf(Vh?ujUv8dF)9Fyn1jJ^C#8b-X_AS|8|8x&s=Sn9e`5oT zV$d82m)@?t7PmvOS0IE&L=8<8LFk zyAQqy7uW^m8aI)20rzNkvW$1j72cn;8?dPN4+n4f?)mk|fudkiz7}G$^Wt&gvunOh zCtajI%ola7Ya78{=6wwuW5?~W^a1hTr|ZcJ!lB_;^Q)$8eGaup1CQtD1Y$kK9n<}% zS~STW6Z_w^y2{ny#>B7^7hCEI#g{@!&Bnc5w`Y~^{RmYG$g8Bc=Idk8-7duT4pES7 zpwGh*DCxWcY)AB{r8^sq*mO@n3XHl<%2nlEGhsW`s-T{iSOQRUOsx~C?i36egVbl< zC^kSVQJ?E?S2cNK*U{UFgTiPCfr@l=LsWK_|6<*G{eskzfJ@4d+o``^ac`E|2b(K> zcKTR$JF;9bf`C->lvi0%>HHo^;_nO#FgbGnc%J`e?K)C*<#6MC@Nr^uGm4wo;Sw)Q@A??@KEjK7cMNS&-W@wL9maeFGH_458pvVeE!V)Q($fdwEb<< zaczG1*Cc)hCb3?MPyb6dF?JkJw{P>2UD!@?MMWHfmCGZ{(>HH3OB8Wv2R4iV>~i7z zK}~}CzDji(M>&GVN1t@Q$LMOlx#t(&3ShS^E-TfVO}uLy&#j zsb?5Q+zz+>fwq)X>pcRdvG~@&EIk9UN{@IQ-{dn}=b-GR%+MJC=l$v&I~8pfU3Edq ztrtVQAT-J%z(r)eKr{B+26g~}|1-hZ`DI|bqgtJz*gU8yUr<|0%Ln9~8m7s|L+s_0 zqN?tq4zx&qk->8)@(FY9?OPteu^;J%S5RsS-eKDS^@W}Pa>nUSc#$Si;DJ}&(H=saR=qT`Aic z6PoVSPQX5c4mjXxea%W!P0=LUEwOMwOqXy|DN2b<3EJwAO_`OG4dk?|CH35Bpv8~n zERZ7}*RCT0I<-zMv5@4}-K1{Mh=m8J>rSZE;F7bsvpTcBW&7{>S)ttauMwEn_`HQK zO2_p6luByTcKS;rRMo*jxq+3Ngq2Y8iOBJA`$g&VB5ewqD zoZPQ!g=?I4x^{|o`gSbX$6bq3K6+vgs=X9b`QFrJF_F>%q{4zzPK^t3R^L+gu{~tE z#l748_Rx@2q$83(CGPUBP>KpGE`Us?jWt>lzP}{uO+w zar_(PR7G4&Z^~YQ#$~JSGY7xozAwkbd?yKe+$69hldU!R1Nf&6h=rsP=%pdUJhfBb zpF4-6V!FRuA4u*DRP8=UwEO4GJ2!3Riql!ZvS8zk(|Tsrol^CvE})x1Mw&^ZX8EJ^ z7v7prbJ`qK434#%t~{3;5U~J0&Yib)-WNNo;q6W9q&}#g7yA7fvwDhI^#QOR=>lUDG#ySH6uhG6Hj; z(5eIrE^$wX*@D&)#rkPLa2I^0l&K-Ama{Mnn>5tC0@FO7B@dCw08$*YaeeB<9CU3Z zC9MkDuX^T4{WDwmlB=m$tkZ@-9pQvBJp_#7F{iu_twV83F;X%LTh|5 zZ^rV@ShQB4U9HY0Wb!Z#nFK#|tAkHpH&kSfJTcuGe~MP~Mb7mRgB#Rez1``1BPmzs z@>1gV9CWOiVGjCrgJ{chX3yjn#lF_9@dJNhB5Wep&RgGvQ+BQx!Cwc*mNdA<801xK z-CD%qCHQ=qm(0jFNvk<(ArB5@G&?u&lBt7?RNqtbf}DrP}+D) z_S{ahADvLUeP1;(ko!s!?GX#@{bGdrs{lL4Oa;g!uXskqxl>k>n&Hs zm!&`~3qZ(#)d-eu(eZK=S?Ecn-11konl!f*gtVR8m0-};vAhC@qQulcZl%N7kK_*t zs8A1hvzHs{4{im0S5SWA)nR0`PWn`DW*E;|`p>#Ne6l%nhplUzr}pJz0dqELVTl^c zKwhDgZvdFO%Q`T!Cpr1vFCXK4Jv511K0Lv9%caZSQih2Pe^v0=B&D~>qtBtw>X{=I z*>KS8Q^+>1t18O^yY$`(D&A;Wce~tzF)BrumsAcpZQ}Nxo%?eL*K2^rA1a^QNFO)mV*Dz#*(Pphd!?xCCpsLd}HeT^IpnPd2 zq~d89m|xNEP95IAzLb9Ko8(`{6m5)WN5E4}(kJ`vqer4=+x%)#no0_=i3V@%yDv|Q zYv25}#e!`HC2^*O@2*j+Ov|ovZmM*^CKq+(d|56x?!_>DdCY#aTt7YVls0io=Dq#E+AcQc^RoUWgs6N|F0NX5cDdf7QC zY+mWZBCJ}ME9#YyExVIlL9cwlIj!GUOtYmawxwuVCFKtEL(_L*d-iYHB_3`Y&u|SI zegFpDXH@fQlKgiiAwWo_zzD9c9PoX%ZE(ammq=Z87&aaA63G59zmj>nItCS+yFUHG z!AbIiYZnXdD&bYNaz=_V4up@-A3*8Z^aR zcuUTvK#UQZwU*^u>n%oScGu%0GEL;;!$5wqga>rZJ`#4L4&Qjo-{cd1EC;z5)Bx8= zf`oC?`-($mS4p{@m;Y);{-zF0HzfNL5zeV?K&KI<4qZ37Wmig3BzX+Bbs z!Kcvr>;8(P<_FghWWx~!5UDQC^r5~D!SPMI0?ST*sSJ>fSNzZ+Zxe$?O`&C{yG2(5 zl(SxoZXXK;?0J*+Qd$|~%N#ok9%jEwOsx*Xc(4l{&zzzoswRi2?{8ug3r`JzI=Del zj#I@;X>j#*kmBuor{Nfyvhff5V$4pF!gH#myyp8Jb5r^WpuDzw*)%MPICs`+#PROg zx#>}5oRwMoObWLF84*>+@ydJHvsv%HC+#Am`>s*C`unfXBSzZ8(oJvGHa7bK6xH$- zJX~j5_0kEMjcE=KxXHCVPp=JTHVfxoYfpiAoo0KjaFqMZ_eZrtk+4w~C zJ!wk4f2S#Vmci&-=}lcz1Z-ug(Mox(C!a09QY`P>Mr=!8*EnXI{EG-sw*7o3tz8H9ZSYUiQHnFj2E`T?@4YTkVWeP0Y5Inp)!F>?afQW%m> zNV?gTDS-Gwh}RCZIM-!2o_eJnkC+}M7yKa5?|bG}pZv34tMp)_y*R)rEOpD+VTyqI zjs^i<;q$Rh8C1KfxFPt}H)gQ!8d8+Q4tm#VkSOD`cP=fq|0T&m$>mkrrzvT+;7sT8 zKCc^VFQ%8@Q7j%Ug*K#7P2DfNT?IOr2;>4`NXA2v!)c|V(nVy-bfJ9P5iF8D0qSF~ zdsQZDL{%5eu3PcyEhp_fYONyhk>W*U9oLs09=d`L2Vr^`Y{{(0(MwAPVen>CQpp>U z5(bL*J9F%(9ea6{XM?m<*U1%8LO1Q0WE^mAL$uYarXDH9=9OeYjk(;-zMgWwZ-Ip&XX>Jx&0PNS zzIxk_UnFkd=NJ#>u}as4sqOSf9*4EQK|bs{MwyqHat1UCB%H=boudUJSpW~RhUsX{ z4rPICUgct?^um3|^JAodmPD~S5PlsGY12xhq^Gn!$?U=^Dlk=zEVWJFN|C0>Sc(x! zJW%0T;P%t@XMrd<+1&~}yhfGG0pTy%e6~v3L-}moEkKODLPYf@ShUpHs5)nnQL>%BjmoUeTYkKFctaMg3D1IK-ESt$XL zUxwFct*RI)0f5}tGnGW%?01^YAxU{cCOv`NMnY6YHO@#M_-tb;F|W>*`wzrHAQ1O_ zXH-y4Vd2_=<|hT8-gmoqFAnnTxruIp``&rC73(%ke?kTQK&7Ovyt0ntiQ={=z*qPG zR6%OPL0h(=w>WN?<$Cszt{fpL;Aw{7h9>MlTJ2jGyts0c-z+BI4lU((Kauib$WgZuuiyVbjsi zV!>Iu8FE^nN)^c5`IaRnp3~{;ZHBcvl$OGODLb6OC95Zk>$mSH`m0Jx zpV9PXPIIy(mU`*OpD>UE9s-vn^8Rw&e^B)AT)}tb!{{>;7tfmDdtG;?4SiL5h7lul zBd3gEC0Bul3l}C6=)ch1U-WV{;e{1U+5?M@Rj}rRCEvj~rhapSic`$DVBTpM8Eczc3)JuNu$|z463OJc#JOv1Scw!1w&5zC7*ri7h@za?*(&QB&T@8%ej;bcKYnl2C34nlXK@EY&3gJ#r9?{s-}|g)TRH@+>shZcLF!g}-d>im zjfh{3M)SnNmfvz8JXN%cv}8L1*>M5wLOmmza{)*;=JV2GHc8$`IQJdw)A@&?y{{Gd zU#YTb&MrauL*dljEs#&$Az4Qh5a`w6ZY()0z8SIyCW(KtB>m#S;&FAC0S+W zTYcb}9HbPN#+KFzsyL%Dy4C}8RH0j(T*#P3qpb*|w>!U*A0w0PC1F2t`l1k8*zZq(J(Zc^3#AM5<{vYmgSI-Eb(nP%c>xXF zrLgZOjXoKF2#voHIyjH;af-0h^1;@;dMxFcNw!)I{U+MT*jQw!nF~zH*1$QD6e?N% zMkwq>+E2&1L(;$~bw`CBCLS92s**zl!95_>4 z_zP}QSb!UAKVw*qiXr<-8_IULltokP1mKw1{c7#_pWTZW3}hZ?!!=rdZooQe};uiPKw73cY|KjvB-fl%;4)u zd({+HW*QCLrEA&J2~HzTLC=eTezEDpA*zz1H1zW?OW5INTjhbABwQ-}$aqjCf& z&^=s%!4=gQx*K*WAMg>I<|*BPr%3J{r|Y3+O+oP$MoJztxLR41zI~z;*Qr{h$Ze{% zis_XfONY==*55VzpXa#s9>0P(;Bw!KjM2ft*@zky#{}z^paQF98@sFiYAn?3+G4;3 z(rq}bwS8H)&P-gxNCl;%B9+Onibkje2JQkDMu8bT?S_tzXvzwbJf^da9!B=(fe>Ca zd53PTcOqHST$j8*gT-g-+1FDD(@CmT;~E45Ut#DjB_O`RDyFYxR29b0q+A0`bJLU3 zIZ0wh;9-z-P&KZh!ZRYys-9zu*-q2$GY01w;1Z-Mu3cxyKi|&cfrHd`C{+b8Pd7ZF zZMOUOt5Pn*zV2XuB&G>DjF4eoG3L^4Vlbx4HvvZ`uC%PgQdM5V*KZkaZ1O)TD8*lu zP@to}7qKbY4)ozS-K}k~NnW#DXY#T0@>)zR{)%a`LJPi4Bf;r$()pgQ&55q73#A9Y6IWT4FVx#|4v_aZ17@PRZ&`M z2RQN0Ph4)T3;xnlK(bI+NqqWG>So88)y!*=gyMZ?8(~!01300Jwm%nkAsT4HR$HiM zUK&qoG%ougA(t0^=qIgn#Msq&0Q0Q3EXUOY<~xU_aL^sZgA%H1?%Xf=ue#o@N5O8#{up znAZn-a;HnpRS+TJm%AUu0&YoEcI&GCC^Y^fZ-Hgf4yP4g9R3hIv!8YSl~cx#&^zl% z+thLsEnIMr#kA5vp-y+YvgQ4x*KD882RhyXAg)NUEfm?=`R?Je$6H|ugvNN@fl{xvzmWA`nT#ai)4Pc=7M2}{_8&?ZY50o=r) z3HGI0{rRAuwmDqSbf10FU7mXTGD8nl1vg)>f?-a1xpnQ&^D<{vSh`(Bit%s3FVfx} z*5EntkKJ>glcT7s##ZO>oR)V__AfFK->4V#t0Fta-9_+wfv$Pf+9(uCjb6xpC_@|T zC>#UCw)v-{bFXuZg&q+F!j6pqX+<7zIO?>xy-KDu8t)r(bCtfV@RS5kB`O(P51ne* znQgCYjycmFc4dusy`ylGR)j62%9M@`A|HsYn5kaAJ;h+oHt-^#90D+vo%pC6>5=(> zl8g6=Fp_&KsB;AF*Y}CVBnhN=q|!4y>fh7Hyj71CXo|9KUQ%uz>>2A{n2y;!OANg? zHMa5O!P=Bs?pi=sPt_-%h^li$Wk&Iy)_QFAdM`r3oTuH);c~l!TN?z#Ry%Y`PF+{2 z)1p&{;7(3+A2nuSmMGX%>vF12Ph8~yu{~dUov+lFP>Y|l*r-|Bi67~4bv^1y)NJWx z!W}ET_A66<$a)*3RMfCm<1vY=C!zo2Cq;{v*1ZSbDF*U{-?#Rd$2eq3#IAx1-vM*x zG{IT8K$AdnJ(O4beQINXHaZ0ok{yzDo&Y~Y^;TfeE#a04UnJe+KK}d}nP{|^#$Khd zlcEx#Rx@wIt@DxVj{BBgt<4}rhfQo0KQeT}d(xMaQ^cuj*GplZX@CzeRB;CBKL59gAcyU_VnqR>F3^{z0Y*Uk?oScxVUSf39x4nG_P+t z2|XR&<+QIeBdw*lNN&dE=dyHELP9_2Bs>R3-xNAYvlobrG`j(b!T=q97-?gb)t+P2 zyVKWJ8-^b?jUdM8;l|N_V8p>Zc%m}z4k2UO^^%IWZ$r-=LVw6f(4IV`1MEj2i?Z zyUA1Yf_7^WzGMps#e>1R=<;tP)!DHW+j~(!nxn2V&>^iJ1!wP(1h0Laz3rVxUP&*``@e*)^-^^7YcD4TXfV?6_H*cGHp0Ls-RY%ktQ@ zBtHbEGeArlp!4o6ntnNOk9gd9>gZih)g$^?d;vOTF~t#SGajk()~Z*kXGG)KeGgB) zOUQrKdcQt0P2;cY;67UDE8DW|Y2~I;t_xIM4hYrhk}_SC1n#(=b**Q<(iUxWa(yWEm}<7J7kra z=rsG_Dg>*^_dcuzIW;eHu5*Q@yPcN}aaxUssWiw(E?6AEkbyit<8^)gjW~u~1wQ&M zrI)}9V*b?q7vH;_h>t8!I^AYI9#i(YI*Bml6T;P8fnCVSx)YK% z;|9#jIu_RX6iDy82?uC&IwG{lZxP~mV3{R_51vt~L2F&_GrEMb(nrZ(1lg;*-ejv6 znYaw3M?-egcw5?*g81A|k`Zyy6^RM|CqOo^>ADeK@D%Hl@UiCMH$orooF&#tB@_-d zkFIquP~wRJTw!`F?@Mc$%qYtDd3)v@jZ>Y7kVT-WMe-h>?lAVcZ$~tWe3=*j>?5U* z!FF>~J1{mXAfu!rZ}-0X;-7!$bd(>;>Ief+U^1aZmeEb%i-{facCK7LK3-bA6V2@E z?j9nF&s4&6DZ%;v_YiL2LfFD@&{@+)exzpjNy%BATcH)YVDW%AW>V?)y}{8(fU&`M zvS91oD^l~iS$!-FHV;UM@ec3KM97|-6&pP*N9j1nRg?mM*8uWLO3icsS$Ez-Tn?ql;@)5zzn z98GULWtHlco}wo9{IV?@4N*|zd7MQg6eSvPM8DFK=@ITPlSxK#D@S;AMf|(A`OKyL zT);ALEL=48F6cgslcLl67CNnwuV+WmK~b$0)Nz|1F-Iv`0^?yajCVCQR%YDN>Wkl; zQWAHB0Y+2>g}61*n!vu;FGsB@-Q^~HZ3V^R(Mtwdd> z#ycc8#glSt2o8#t z&bZgrHHq;8U^0!xAvTY`j&=kSQ4qYUKvPNa9*UgGVs8SO|Mtk17HUsJ?&Mw2<1KS# zG9{jK6GD(WcrKisdnDzfsKsU3->r$6`-BC>^2Y={Aww)$eKlZ=*!iGNdfMVRJZNYx z%G4{wWn;f_ILVP_^@hSJ|mB`DrO@axWrDEw6+b3 zyga_+3Vcm&nyX($abFFVQqD{tm1Q`>3nE%lS-&o!otB!i!~%0OdM$rR(7w`SK_HpFW^CkG6om#|abX4{`7QDF!SbizoFsk{Jd36y2xX z9z1w;%(Y|C;iE^N!j9rMdFJ?p0=WHv{0tNM^}7ALpo%@E3VY>t>9~{2M+ufT<1j}- zGC9+wv-g=L^)36~IAevhJ@b)`#XZ@!TRZm5zn0g-h(tmL%?p7H!n$OdLeoo4^5u;@<_hBetjxV!p|A3&sMS2z3fMj$WQbj%F{djm9b`1=~bdD)HbciLue~6Swx1arSKcsx#)*? zqQQ85b)?CIG>-S{nrN3&u8YJmfzq&6@y$?nkJwgd*KXoap8IzNn$;?hcX^R#1+EX;lH{pTs_m6T$JV2-4(wsx) z4f16nJh#oX)UI)kiVm2!=PT(%+u8LyX7BE;vsfGMSt~h0EEN>}Rqk?wcDJ{)NCrD@ z-+3y>e~+4xu}ae7W}zYR+!o;-z^(%r?(IpM_fDP#0RqKw<8Fc6mK|p^+)mSM;}KD( znV@ZTWp$->%IUyxO1W-z^uRUr0(K<#Idx= z>37M`CAe^#?vvaeomFHbyxSw?YN+6OO(K8uyWvH{0^7c4Np?o@bDg3UM2#YHH5WB^Z(d$oV?i^R%0h%hSmmkl2r&sV0!1PAM=z-# z*+8gRs8Fz{vVv?lNuvf~DHG19hyT)ESN zO)!j2QSyW3x1?v+#LYO2#p7o$RVUVtpM;op>s_Oj%CYsaBu6U{0ua{s`R5uC^CxCo z3Z}J9Ul+=bLUy`ehzc;cs7x(J@=vGkKp5A)AmW_?yWAZbJfC_}{~wOlFO2OSXU1{M z1)UqgrLxSj?=N1gSbJh*O9g;vOlLE`SK0ghJ7sgsy*+Jg`$(^)%)UNuSF94Mf7y$; zkvk-YN@qYS~&)MXPLT1yap;(b1$UE86RsM zc>pfVL9r8Tf$To*FMw8|buvk=_4B)K;W~Y*>5&%X--Uu3ObMC>*CQ?&=a#nQbvvxw z-`XOu2#GQz@44{Pp%3H`B*If=b$tU{3jR!xkUm-3-eS!k_(E7XRsy|KB5>RCtaO*; zg%F^jRl3M&pNVUt$zD}h@NcJLg7-hbzRz*KC@L&EH&14}T59@=+?F-?3`uh$KOo^+ zeXq%Z=tXCkCwO2#tGmsr)?D|95G&htzTUf6fwAcc(B81^>L}acA@$$my?De1EyT^@ z`l7-p>GS#dWxNL+SFg`J`3(VVE6nH#eZQ>FTY4e`06~kFa&RAcUeiXnI)0B5(0t$^ zr!+0in$^k}b|U>5iMWO>YPPq@%elj>>2ha9!=u(sp4{y zw2Gc$M3XPS!T0qAVLtHk?L%11H`2$A)m2bMs7-$TrE@R-du`H`+lyhFe!2Y3MA)K@ zl(3A)3*|(?XEK=#qb=olvn!ad_|cc;P`HQ+%L;Bx{nXV6;_1|V+^m{3OU(MRuvWw8 zw2TH;TDte-zloMwmQCOdqS9s~(*j0fLKv}W1U#p3D#QgmD54~lo}t^USEsa{iMni# z_zp(cDTB@E*5Ejae`*Zsw32zmoRTq&xce&{Aq4^OtHDhQnK`4fW=D9R_Tz{R16|I-aL^y7vIsxZqtq&KQ4 zm3NZ8FE$-6747c?Q29<(e8PNJey5u5%3pcy-dyf+v8vcaMxVo4TWAk=8T^mQ)7zWm zw#ghF(Z-RCoQI0Tc6N#;%G|~(>Pl`UYwpmmYteqt=D6KB+1yh&X>ilSi*cqO;T30Y z{k+X99To7JU~6BgQ)yw9->ME~Gk!JuTG8$8)F3U+C{@B8Hay8j(sXKKb2b9j%pvi6 z3|?DT90_S#j_t;VoUXr zlbXOA6qc>82N_MDPqLt~pkM&Cau1J&u-Z z2UoA&xHXjd`(4UGmySYl+oigA9a%ufLL2 zPP61scm=Ip4~zVkk-fk&T_Eu=ndcoG7Tv^hDvMI6GjC-Y$RHd-(j1&XhEwtyxUxsH zo9yXA!;TkE?y;!Nl~Q1t|L9zA-gW(`S3k*sFdB%4rGTv z`DWQKw+wImgdZG1Mz=XC7j|}NH0z~Uxb;2;ND)YT)#*f>cy5=eY?5KvkhHqa8Ts4k zL^u!U^l@oHPS(Q9F~=l(PXmbG@G+A-;eL}{P6wt`+9wU`MzGy{TJN#b>FY+y5HCJzOHxu zf$pAZaiBxBvms@ACO5+q&g1_--&FQl+%Gtrvij>$z|GT43^0~l1Zh!=NLHwBbYb5euJEXOqP^M@>qK4cI7M- zsPDcc5Frz0hWnVlK|>vDpOg2Gv*OC*FOJ-%whq@S&F+NhGz}aLy`5T|xRzX|AUhJsv1>rNhr}Wjuo61AX2Yb_BS*Bu= zi7sfKD!$HQVf%IV3{GfTvg83&R2$)B$EEzy65K>UL9hnjOUMXJnGyp@9(9`@Ammco zTzEC#yzu-6E^EHj<<+@W)E&g_DSkiS*G_PBu(>9Iol(M&qwKvITH@f>^jhu7G)dB00nbsq^NSfk{?XG$1 zUG9r^D1i0a0oa9uIS2Dm{jlQ!yT!~c)ZK7CW9w?tSfkbEsN}XSDu3kk%xO}i@AYfn z=m8%ewme*{>EzR79a(qL3|WJW{(ZTaTNKRfsb`w_`FOwv!^7Swkx!iYpr6baul-Ur z+;8)mbEe2gOrGyUJx?@|SRSl*k2{-|#H*Q)fmY{g85xb8=-fxN_aAXzK7VzIy$eH9 z37c^zclB{z`A%vcx4<5kbp8t$@ApM(!18mncG(B10y@O)@K*go9c!SzPfCeDjd((SrJlR zyqtudoRtL8dRUrG;Mx1aY5HYu92N6s9XBma{b3*xDgCk8XwP9FJ+mP_yCq?NuwQVs zvZd@PkGj5QZBpeenkDOtF1gew49N)k&9chR*RMp;tHqTbET^z*z`9lQ9SOuLTRfGC zO9;w}^nmU}=y=@!}}v$>NJz_c_xP!oijk=edY< zCDkGgKQ*Y%Fwm|Dka5VFlH_!azagEqp(P^&V-4-tL9q_EwXm}LX^^fYys4yrF+#}* z2{@%D+uhp04Q!~6KyZo&W^^FD53Ay1O1P5jA%}^uIl)RqE=3htF_oJ2-E)gMUMt5z z>G3NR(*S5p1Br9h5|uL?il+92`D=U)m=3E%_>Y`c?n#Xea*ICsqy;21UP6rUe;{%( zM1$K6B*rqf1SSbsA0>kN14BnTE={e)L+a2mqp7<=x$9L2E&?D~bhb^bw*XK~SyI!Z zE)r%9xQD2nje6OFk<`jVQMLVVk%|{y`EHLlmIdsDZQE)_J+4N(C0w0#t`W~a=UD9N zglMgkjV-4g$)los|kSFx9AGOmd`3v<>|?1P zkN7HZs4#Dac1OBNUH@2r_0t=GO`sA}6-5I(monQ>YUZrqxjGNuh?hY8Mc;On=w8Gz z8zHPT*8+tHo)EoQc8Y;Z%CrbJa=9pPW09CEgytiEWkhR1s$n5cX81u}6LxZy~I_SI!mhbP+1V`dZR@x5swc-T3!! z3~}Pa2$2W3G_tIfHUz^p(Hk<=;yU#yaI>ei8>5|5wo)pd~QCb~2j$lVcEA!!`%e0FN@1)#= zc#~y21`d|7;mri+26#}=PDH~3<3_Kx8=B2J^l0}2+sCfb>g zn%Q#T@#czXr`mNXsB)`=kK@FP@2N{U$_oXK1W$2umwX6>nD(#%Q|ggrQOPEUl$ACS zW8mo~b)vj3?fsUXvECR5BO%CyhUs6YCuqjb(5}Y&zc?s-qUD0e3Fkx`hzk3SC&Rz& zy()*@JmBK?vWXJ_Xza3vzXn|1H)2)-+j3ES<~YVE0KGG*?aa1ayUVgRuN4KV{5i(R z5gz0?{8X?}w??s;vk-m;_zaAvBs1V6&C_ai&4%jRAeA~g#X+%xTnK-0C z`+C{SoL$YuMbYL(c&|;oIN|Lp=o$bjA-lAENY&5sLmX%PyKdIpwI&evK936B1P$s9 zjF4@#xJ=2lpPg5!n6Yg5i9mIk|J42ZuVK^wG2;7Y>woq5{$q##u^j+fwg~L>8a%zV TSpW7GU^$+3wXHex%dh_ecwyEY diff --git a/docs/img/streaming-dstream.png b/docs/img/streaming-dstream.png index 90f43b8c7138c8ae2e93cd02d7006ffa9254252d..326a3aef0fa6e5bd04e67222a972f169878b5543 100644 GIT binary patch literal 17843 zcmeIZXH-+$_b!abf^-lOqzWoUdhZ~FD!q#|BO)CH3>~6^0R#jEK|0bq(mN<1y+{cW zLWoEYgq~0n-i_z{&inh1_x*A|+;P8L$G|}L&RT2FHRoK&7M z_IQ6)X3B1SHTFWSwuuS@{7#wR3-rzClEPO?!Ip8flU1ws1F)ayefHYZIwvwr3SF)uE> zcqdcU%|bDReaHD_VzGP$&8<0&qisI4%IuE9Z#E(B$cJG6qy>R4aaIl*850celceZ8 zMBTmLce@GVqDO9&d#tbPZW{<?Sc9*&1ez(NI zZT)&DX`vOaP1@bUx7(KwZ*oH!cG`NZ@5cHI-``D14a)00>X*K;ZnAZ2E~?Sx@J+E{ zyVGzYccZ&^z=%=_EZwzP`mJIZBOSVALiPR1?fV~}`IOm32Cc_d{pb+c6x3aQ6nbMl z_wJ_VbM!>dU}O+2bae)EY1Hyz!86+>;v4>VcGNz!Q-U?7i%L;yZP`8dHxY$Bje@ej z7C61F#B9XhT~{T3u)k6h+4O$T15_TK&XSro=%c)-RCn+5=wwAnl0zGtS}@bQyAs)t zyC^s19)7tXRumsW+0ORd$mHTh-9*Dl<3y&YLNK)hHel&xmsj^Esxm~1q=xh8Jl$qE zYuXF?HygbxxA;DQo_>mXsXKSa^L^Ky=y5vl=M(j;(W6gL)BAf$5bwTrs;)5V?RN+H4*{@p57q1K?9b<8zgh2oB4WHD}% ze6zH;F7o}cVjZl*Ov~QmS^SuSl16+4HLT1mWSGnT*0@kV=~|i zEB1REVS!d~F})UB6OsIwTB|_qfd|4`m0I8XvL*$owC%^)CpYF48vI-R1+S!kZG*${ zRWg;YA6?YH%;2vQAt+PhP6WA*>%g?Y$VuuNOApcR^iK>Ex1xi-(|KvJIyQdHY9xi+ zZd|F?%^-dvcZy-C^tts(PO5Y$kn=~t{gA*JM@DQBHr*?Bx>*A97W#_1grJ}@9d$k$m}!DSxR$Fzqt6zl_vY7PHQT{ z?{`hs1#6jX>Df2iBL;UuFKoM<9PB`CZnma*NT$(jw_^@3JziLNwwJZ^%x6JozvIgD z{!>Ov5-)$5=~fFM1&}@0vh*P#VZ3(!bHOod9LNwV&f01!rv5Wn6zQky>YY718nkMr zluRPvkqfsmj{3aIk(U^1vU4&o&|?Dbad&?O-#4P<4Sb?NuWd9$C;E8gZRRJV`x}+$ zxsvIb>CL&6>HfVll*wrS3j+hpOhsr*$;=)w^Z)(#f6M~c2NhH-xp`TxI80~P{^q|R zR)61cihAIYie+Tw#cU$UBnQcTS!eph%2ZyCD@*X~S++!5QKY5J{885BrM7GZhD3_6 zYX8oYeuS@gotL2p#ek+t3fpxwpHJAXzTuTC4&N18r^Z#*KH-0|FV`qpsh>P-$=0P9 zh&Fo&9dXIo3}#Pk7*-0hG)#(~9=R+!dN#fhT}iq$m%TC5{bx}3V0OpQ6^?Pty8+?l zx&!io*2%1g6e;K(=RM2zs{=o?DMg@_#mbW$7=u>4N>g`m9GcX@+fWFH5pwzAn@e0Q zCAwYT`DwDew`rn&VmX+t;H-p@N*B9Oq_L;YDv^4xYluS&kz=WFc93m7uYOS5hw!bg zOK~QdaUZ=pK%sKeU1^J;5-@y;>k+Xb06QhyJ{Pt8tJPsS0hvmWBaM}e9m9^iA<*&yD zd3n1KFQ3QV?s@`whEWN z4siunZoW;ly9n+J5`{SO`Rw|Bm|!xAY*5p^y^E?1vE@WjtH|>5o>317)MVDz`K#L3 zHX-{H14UyX`m@5~P~n^+x^^8ycG2}wvVk4hb`QxZj*w)Ilu=1h-cs2?Dd~>lBt=Fa zRS@AbanhUM?25mQPCpfLWw-l2JxUyyAoG{3Llr4cA4LR&nlg#?O=b<3KIK3TP2TLJ zC6T0uvBrNp#@{=OAmi-uAR9$oSaVrcwnxFYK8w!GVxK+{-AkL1`||Y0 zD17B?da})$`62uv^qpwd8ozL0%t^@zv4;N79yx!-yBy8c-X2FeC)sHu%O9VF2u^0@ ziR7gUJjQ>V9zojxD-Gv-#g>7y7~$-urONoU3b&(kvwR6m0@>5F1wE0|6hOdxCBw$K zIOA_R+Mab@lh8;$_+9yWwZ#%$x^Ap&a3hxD&4XwOpRDGYBPu)c0rhZBK40D`tgIB5 zgZ#<~)+`$sEH7X)mJZ@+JMqf~2+n5X!(`cjH7$a6M%k%6ID>IWuFwQpq-j9->ny_xT9o5oH;gel%1$;p=)HE>A6>yL2$8_GGHpV$dOP9L6ZFQI{ulSd*ab zmvEHsn6ecqa)|^SSyfp>43vd2RY%kiF_(Ypb8~kGdr`ODIJU5)5~3a-Wh+TBX^2pB1R*x)n%%q^ghygK+2XN zBI>7QDNi3dolxGPYx4-6&J^N;FRy<_@EHy~8`y@FHW7<o3d=;S@(%^yjAG?Y zVnwu9DsPY_PqvZG3G%31D%3g9|)zE z(5%1Q3i(jOJ{~$j);ya&t;gE+lXbo&M-9}~9I0n>=zx*$%T=K_TRoKNJ5QHbhj?U= z(P>{dL-rc=*5PRmWj?f%-)bT(q@91V(rd*)nAr0Bc_Cpm;a&%2&Uxu3_SsXt+QSk@V?jlnMG{k#&5mZGD4W>3FTM_u%x zIia$i1;;-fj9&7Ui$P^5dzyi+O~A0TMM>-9naX4Lb}{7XSIbbn z?h`&!!kSYjO+bjSgOZ|ZJ%z{t#qy;T4% z>QvU5D(bUs7k|f~{2*edx}%v6q1RO0Q#3$Z*>1J^#JMAy`Vou#sMR;w_>8prVH3c( zL*tI-=rPufx>hUkgQ`c=pc|4Mn$U^WYV<+Qu$vlw8&kPaFYwj(gPTXDkLS#SFPHpf zSVnp;{4vB6BEy9ZINPb-nl{tH^=6VNtZulL1IJ7jd4)1XD!rj%Xd7ZXX^dI!d4tb~ zpIp5mI_Tc-wuIJe!zs_}Q;{B%SsNIGagmffk#7B6jDAd;#iN#8SZxQ^!XKxo^@!m! zv347Ck(is_O9mpD)MfghZS?T@N5r8Lwke#+APu*5zAgb5W)0y&N!B?(YN55|1I*T~ zJ-wOg`sJX2_N%dD$z04-AsJf&B~^aWh(^Yk_KQJ-BLw@oTsC`x5LhdL$0y~6W zXHm?%2HQFkq_&t$PAEG@!bM(P!SUsuTyLHrOCktVnuLi%@=h@j^oq`(uTq&Y5Vz-7 z`_?%*7(BI>xzG;LpJ~rs*!y5+>bDF3J4d9_!#3!4Sc8(wgoiybm_-G@e6GnX`XL|o z2SMq75`+aJ-=IrPO8+WTHI=CC4s+c6Fv9o4&nsxl5qsa5ylaO`Pcj{OTtjX>NokgD zHw*7s{Tk^G@olexa3$G2PEy1u)Yn*`GbAd4R>wzt~*5JVYckQdzwYhWoP8ba^25fxS>QE7V` zH6uS*56UZ+)l7S{f`;ouV`Oet;Le@6T ziCS{h23)m{8OHh>4(YJ^Au%Tn0memkf8$D~UC^uy6?zx!$P*uBsfKzEu{|y(`YeoQp51;|>iG`}`9ON~43_$sB51pUl8>8=B+> z!D{RTQ+s=ypp~K58l-^4dX37$fAr`eaD>?K40t*`W`$a-qdK_UqSb@=(G)uQynQ&p zv(V@~p&3Ir`r&D0klvA$ZU0Ps9ew)25wsRKKxL=0th?fcpVUhalyZdLSQ?EKxr4@O zf_!YH*SFZJfDOAp9L{O9pf73HQ$BK`)GD(xUOgQb#|!wNwZjy;Ghm2a=E@y~QFhvT zgP>qEWoBAs&#H=^ga%{Z7tz3`WE)0gSZY-qd^eqwMHY;ju-*Uo*_!#iMX$t{-MS1J ziZoo-3-{?5Mj-j)k<2^qj%XwA)(W`*lQ zuOo&)Ox0;M#AU#CAJW=-w1Z6i@+`-kP4m#p8=^ICNOHLpd3Y(u#W~Kf@q(Kt56jVd z1uc6gv!4okR&|~cc*turWCqnYbbPtS^5p-Dkyy9R#QQVu9hz8%Zo#dg`)o*gs?$Zd zUeBZQBIPwbF7dKZd-SPC zbMV=w{p_QL_HVi4Eh({MVd?JYhrT~qtZVt!NL)>{g)(;_F>=M}N$a%jLca#%0meg8 z1`(Zkz2?Sx-15z%zS0hx-!o>RrvT1+MhF;o4O~suU%WB&TO73lANKnp4bF%dTA$NJ z#vUiw6Zj(jxJLq&a+2XQ`HoUz2z#MEiHj(u7mMV*l@+EZwIWb0Y%_|YHsenHV4nQV zc{4Je4XZ$0!b22Bp*=ak+D0IGbT_EMrlW!1$o_@p3JHQ|S#d^m3YwZA(HTE$uzUlL z*;IY%sXWcutC5GV869rk@{jdQORmJ;8MFBq>Psx=H}Lg{X`7gWUwVE9!H+TwEnxUW zrxkqlsL??Hj3itcO#l-f#_d&b2QD)U-Bn>Xnx_(yWuH=R^z6<7LL+Yf|?@%jQr_NGD* zeWCG6T1*20CD754a>4}G${=K**YKY-7So(hTA5>QbWyF7(~QvQ!{OZ~z<4@XDIgnq z&-dGgG+19IS!ji!Tp^la8Lh`ia0k}G57&^oK58_Ap2`>x?bxptIggQVI7e;Zc>N0% z*ic@h+lxvAZvN&wvMt`kD5$SzHTCUC2xmDn_IK#KlNcooxX#k&KLl%WVraluNjl>x zsf((kCVu^RRj95O`Cj|xk~;c>m>mG4B5W6?dkrAzzbJ@1CR+q1@1H;eK5eHwL&n}&-W*|*P}AFOWkZuSM!4a@xAOvXjWAf~@w zirnd1q4dg}5DudSq575gFq^Rz8mG-<$PoZVJc$@k_%y})Vt>D_tom(i&M? zlwjZW__E&$o}kK2-;k6mq^a>Zy_Gy?T{0j+-*L;PKmC~#R`zUm>r+HXj7VN)#fRyg zW9G_Pp)ifSW?uuzj;!{!f z5VZF~p|t)ym7E;=WRgv7I9gR+0&9K#vxk%MCqo>EI=7R)$AV*11;$3Gwlfg^0zoD> z$RF?sH_Qnh6?o($10e}gBcq$IF0rK16W5-d4d-pvMm{Xi-k6ib@g!?c7b#~GO+yJ| zQ6Y|%9(~>8w!S`E-D8X2o9QQxZTB`|S1<;%`287*L)&+bb%HdHSb<^NTop)0%Umw(4`RlU@NZL2_H0$qRdM#Zyj*D5fj26tv z2Qu{AezeM^J~U0RaiKi;kv+QrAO60ezgf^CTRw;&1^kT^;5JAMA2ItCTTu~vOVc*? zEGl>IJoXuWWln!4_}$`4-U~d2n+onut?b8K=y5bRSs8NFO!Ch85wDWIH|z_0N6(>#WX8mlx}YAdB{Z;L$w~X7t_z@_aRiuX>|w zRZe%OvaGoAGaCh7ibBiMd5L*3>#~Q-J8kdURM-P z4r<1%*Fo=2Wkk`BdP4RZGc{K|Nq}qDmzJyAu3SFrkaRRAxi{746(Qg|@snMdk^|XQ zX!yYL_5FE0^nsPBDXmxqf?I`f6pfBE-W97@>3d6|`kG!gRQ`c%;w?jk;&F_D;Y!3V z(r6$Xi$B<|YHL`EA-}IqUOBMSn{F&b3g1zbH!~%W1)A=(7!BN^Ofh=J<3qCz+@DL_ zXR`CcIl7IG9%6YN=(yjldtG)kCAnw(d^dmu+Q;YMb3fxp5bkY>^0#fKQ0WIL=|jV1 zjSHiV>~TB5=ktiV6BkX z)&sr=0lUNB%ot$`hvWA{tsWQ+bP1o;U^?S_?K%?CNydrZTRw%?U?fE&B8$qSS<4r{ zZ~JX`xKMz&^$F6-0&jnx*eO*si##20JX?xTH)=hAhnG-*+B0{gfQwIdTW~D?O^4b=oyYG$GayqtD=Cib9qQ3HLJu}k#UkOH=Scyqb@>hF$DhDiM+*bY z%h#gZY@moUW#V2b9 z-fSPan{2K6{~gGf58P*A7sAl4A3@C~^W+-rxma?3?mM}A!5N=H5ysl*Gyd0Rm_nIy znbAvVn~$RcW;K536%fSexggm#1yP6Zog$I*|&s7^!h)4d!D{$G?`r^+Vf ze3plHPVm_Et&TI|yzh-iPMl|bx8o?IGfxn9Pr_t~HHmg_K%HK=cty>Z9@4-cr?9IfGX+_Qm8L#}4kYqXG4lfEYv}0bPCL|O z^$I`x9W8tfW~#!pyVOh^GH#BD?Be?EzxTn|24O14B~CEEt_wGK3_SG2&O2BvY7muJei zrpg=Un5U4Q9R#UU+Y#lnU}$N1{cZ!&Lmt`}&pGefU)e!qZZE*(*2_EnxCO;;Oowxt z(ZVKG30?<#7PbN3ff3TsO+G7qrN7{|fNwoO`VRULob{1{XdF*ky;c4c5I8`H2jBtc z45w0N3pbsqcTXd~?q-}7dSCH$yQ`~La>1`qKjRc`ZZ+%)FEKm(q)LKxnz)eXFql;A z-r_J9u%d^ObH_{;gAH?fC0|X=xK^^^H@{ShJ{$VfJ(_*dJWa}-%TAM2@91zL&Sb%e zMq&Hwm30ODPkXAss5H zW2TkY6he_9Y%D(=jG>bvXNV5s$3X1SN|thK?kgLESZmyzH=B1K=X|wMu0}pHs)P|Q zn|kReH%^H>=wrSxGGHZx40KO{viRe%{lVURS40G^QwTv;7WBl50tnB%7w!$gh^*y$ z)UGA8+P%fhg#aYc`%E7ko5>56H*^H5)ch$PN%keSuG6#;ash+FFFfP{($}6IM%0GQ z9AvzX2Bs+PWr$pTkj#$wwhU|n*=l~~jwWGnZTNwD)*H_@K-L=9jJoIor;yGJey~ldN5{9{i|1w_O zI&dw&Sfs0M;BiCgl`?8#v+-vH+z0Mohb|{}K+lLoAgtKhEl^_macd9mfVCW8oXzA8 zJR4@CcZOk=?AM#$u1lrXS*37uXQ=hRG=zR^tEmn`|Bri^^4e%ApC5EHb|_{dt94GT?x9__kP}+%@lIqpzyQcqYD0oH6;9% ziG`r`sKTBuMsieUA?C{9^|wF6$}6^MVWzfB-h0E+5gl30R|>a;?R($p*?b##Iue!_ zqKFVhdf`OhHQx;qwXaVrQFi$;Q|fpy7v!0{lofjH%yh#g%a0|mUbLBbyk4lvpwnPB zS?!Qy*>kqtMZ+Q>4+P2Wbq283QaERwDRd*WeY=w)0`fU&9WcUC9ctI+-0vhC{xO2d zhRB`OrWzw!4=S30HWgVh-g22RevY)uOB3O{DuxA@98?W!bITHz@O8v5*$Rw_=+lyK+*w)gki+tnYGt4#6{zZ2rUF?)HZqlU7!7&43p6k#B zqoPQ5wq@^(AD{%e6cZaQ+Hxn2fiUCisJG(87)ZG{ECYPgjK{}E(6={Xr+Zmvvz%9z zpJn;Y1+)u(>p)Wvyi5Q7F~Ys|ljt+irLF83vIi@a^&$F(#DV8&h_uYxMl+X}#I8Q77AVgZ8qh{s_~{@?oLyR)QyCdyWv#md^VN<0ZC619 z@|4rgg&Cdbt@e}6!Q-bwX(5UUJx)@%Dp|1BY-ojC?otUz&)trG%we3V&b~jFIky2F z8q~P)@*?%Ee}ckbJk-|pJOS0ZH>ggYC!ka-sJHWZ0^)ndbS<2Hv`Izl5w95crv)eV zk>vp%%7DAPXZrloq)VCEfL16w(p(c(?bGQS=RK9dkVvN@dxuy4k^?fE)ukTmu~+iR z-NrHLZ0L_k*h3HJpL0y1^r~gET{_i}Z!CHCn{>%_lz%T6fagI8RZcGH=qCAWWnU z+-q!FeAOvHZZe+(Xmd`tpl3Q%QB@!-K;&uN8<7v^WMG21)sAV*Zylh|h|&uQP1bl^ z0dwy3RIh3{XT4Vu>^bW&#b=1b*ztn3^I}|KAGpuY3mi$NM!Y)kMkOx8Hy!>_iSo#TeqHEhVoL@NN;SYtw?536=s6^( ze8mA9?p|>-@#ZsBlPHo!pXs$XHOMyfXz7c-(}r^asi}?1go=S*SirAWq|9ZNVGcsz z$TkWcM1@UPRE@PSD@3%DR#L5Q1m%ZjYdRg4M!VHgb;i$7fnFe$2%!zL-a0)zK200z z+OcES7<*3OX4SgjSjnkKRD0XrE9CYa%3{=cS~D==@c{xFIFKiN`PB&h7Rd4@piZu5 z0c&Z1`xdmpk@Rna=_rfWzNOd%L~u@BU6HltO7g@CD2s<-alTU>F(Q{f-hesvxXVWs z>idq{_zX+NQWrJ9E$E$$m9ep2Fe2@0j8}aTBq12rJnSag6%QP0a@H$9Ks2$|p+JB~ z3__N|0HnVdPOl${vXh`ElJl4SSbZv#9Abmxt9buOCqzIEz;1&-`8|z%emN0YQFlS> zt;d+sbEGb522GVqDa{Acu5s$0Km^E=;d14sTGUmL7&&N)Y~W0cOih<}S9iP3{p(Fz z;2GRs@4nw_-OuD8l^etlJ`SP?#C18a`>}g+BvbuOk-t9 zL}m1j*Ga7Ny^O}muuegmBbsHnkH3t`o17Jh~40rpc_JikNUa=%d&;6)7`&^k!& zvEytJ;AFuL5jEdwhvdGf>=*JXC=UU`%aX86M~^dU09am$l|?b!D%mMQrcF09NYXbl z=RA^f`jLGl&MpXBSA5dVc$?Dbwh0wwECepNVC~t4Gxv{{bO$xq-3BVIF&Sq z-yrE(e#6O&<9=(QlViTZ^lM6Ug$7v~Tb#+q4x@DZ z=(Go(G1p**(A^L*3!U=-miQCE!FB%xFdghEcVnq}@2$l~MP8^QQ{)iU0F}zx*A&|T z7^nBUz1E5GB*7Jd7QQa*S~%31zE-VGBe3a5SxwB|&mMi@YgS6JS9z(W+jPepWP9EG z=E4t}{0%p8819hRK)Z>samC*O(X#6=MA0WJbd}0Imn3(gDo0$uCHIe`tJjk(m~u9o z%B>oxp||r-n`5(^6)6h!>>EiBE9g94C? zxqjiX$cT04-TwBFlVhH&U~f#GcX57_i-e^JSPKg~KPkn?KrZ<}ot9XDH3t~j%6cm$ zj6uD-bZ7j=C3Nqy%GVw58*OT#lnoviefLK8*yA1&>uXLmCT??CVZ3&#*65zB zTtE7mXu{PQ?|S=n7->4TShI&<+l4&QCm*o4MP;~bEC{FlxQ#MbIU2FtD$iD6P2|yC z{oTGfMh5Ov4%>~eP~KmmT0!4SCUpn`h@|crd~Gc@?lD5JQl#6Kt;U8KOEwzM3F@>u zbr!%)S$U!!6A-pRf|5Em|&GrTOh`m^CwGe2+k zSw>rX1MJ4ay=K8Q?x~3j0u`Ft&1(~d5%Xc26j}NWITJyHi}-c`*@ZMxo!Hh6pCfn< zNf*;8B;0Y_PsVX1U1M;{+Gn_onIfEXi4nF+UcqeYTP-8054l!IfJ+t3@|!hnu9MTc zH#jj!%B7ib&3ZW+~UyT;Vgqs$D*0&#@ zTx#9+SH~<-_(YLBg|7HV#md?z0prjcBW?)}Hl@8bz z7`U%he=CyacGA{rI1?V<1hnd0tK57iHchFmohsf<1zJwcym@hG-R`OFAoH&>`SDT? z7yFD&IGS9hM043JooY?WylZJZ6d+5Us0&5|k?Le>kE6 zK1vlV@Y_9kN-x#-l45NOlnuXu^t)(Z8Cnr7g9#Z^<`qtT$$5{-NalGBqoTy!G?B&f z@B^>Q`}WJd%6NKk0S$R#szGqj>yH`xuolxal87c z>NQdl%V?U}f5fZOfOH>Nk`(G7E*@M383K?@Jvr;hMHU`cFksfeadH`ab%2ZVCURl) zfxHGCEZe#>9!!HjY8sz;F0C}>dIPx3N!b)QDQcJ89C`_fzWopnx$*fbux>9VvA8wV z=arBT^FB8uLE04*13`43`!owkH7me60nTY0E;pr2d7S6!R|yZ;7`O2VJebvC6@nw23-QyOla40;R`e zrcJv^3+HdCGwnW&v;62A!$qmm6wV3Sk}vxWtW{wK$wR3k3%ud6Zgm53;cJHe;CViZc<{0IjoveC6r>o5@GG5$HEo+8^+m{zGBT$ z75t$lxPh`kwliR)zds_;9~n=@0adYo)vuj(i380WDt&B>w*K>CYB#@Afh70_A+`iTZ^uB0 zpR{&E(2O;6uVfV}7m((r^`ow`xK+kS{ zzR!yp1uh#Pa*NHSW48hTYIfi*5I#`T+L~omlH8xj6T)_5_gWD?S3v^9_PZ7I zPZ||XJje&gMMq8Ui*f!IKfRt<`no#Xa?5@1@DITLL%ILstCJo~*Df*dkuJT?K7FrF zyQ?PB(U+QzHH(-6WJM?VAPD5p47h@obIOu$lIW}Z>k6%fzADq*rD+RKZT7Zworv$t zbwz4fl?B0Cxe1FMXL#k~)!SzPb@)3H1gHh71b|K+I|ax&;{Mvi^tZRC($OM$a)0P_ zEzV1Rann3krIc38n@C)^C}q!!^Er9N!2P14p4lItL4KyIh2GFg$8;}P2Y$x&#C@iA zhY=upp%Z%2IM3Fdo}BbqI@tAPwF-LaYz zeY`g$9@jDJJ$M8l(2jE)0x+{qOvX{T6@W6@JKKqa9+p;Xb8fF@X^9{8vVSBiU8;gwI!YLwA8~7u~9w9m7L}OJBk4!ChfOreO8qFjvVb1eM9h+O6un=x0sSwg}wSpM~g8Hl7ah9 zXP@lf?Vv+jWMWedl#aT$X6jtSKD*X9j&f7-Er{gJVB!>I&lw;BfNTr#T{RPW{>0{4io+>D?nsq^>WQp@ ztDhZ^zCKi}GGA0!HkXp^DoOx%mznzoJb1}H$}xtEgm31;;k=lXY{Ajwc9Y`QkvcwU(lW$9Jo={+sQu-@V%vHD_PKjiV-$I9Vrgn2TndZ zSR$Bdd=mW=;7vx(0D*@;XY=m|o(?_1b{!J{U8sDvLxavSPLka;HJ>b;9YHK zZmiU9?>jc+ubp4va%Rv1{ftMx?$XJy?tAyGp#)sloS+#qOxKg^QotLlrN4?wMDoVV zs(P5`R|1aL3Rwi@0ja3V0oCFs+;Xv`{R~@=6jJzEYV?4~m<_CzKh|DRiVP#Ap}xp! zlOc2gF^*SAL-focL+Qhrbsa#nSErH?w%NrGg>=?1ngHv4qh{ z4*SHmV8{r-^S70CFj5Uqt?Q>s@U?z2%m??i&E8|YZx+t!(ul(%rCTw%%0HQrWt-Jr zOT~qmF5AI21Y-o$MtiD1Xq!}|^=%pbF((L?R}p@0BJkDEpYJhi$^yl0H+}FMCc?=9 zO+(#;zWfoOSUl>nxcK77nB$lNrx7V!dd~m0wbI$?-bu}~;kT%fbKxR%`J?$GK5ILa zB=%q%kY6MoTEyB77e4<9j(C&h)qwKdBc)`Dha3z6Bx?t;`>jacMpyjuBnRZx7Q8eW zih>~CoJ}d^m-e!Fwt5|&nH1=1mzS`sU<;TY$-Z@X&-{q3{~v+iW=jPHbTYD4WOqL)!#XPL=91n14r zhx$0pHU@VZ(a!i?a*#_o(P{2p$Os?}@vLm!ZNqX4x9xv`E~hxc+$V%=n!|CcfKt}C zrU+!g+<8|_%SRU~(CbXO!OK_zO{1C6%eOS3>&CTi(_(AplT-uRJQu_At}wLxD%{^&c%i&~Kwjexy%8W+&xDuKG5R@)ZxNuyqvG!okI zGyL97%+zagyKw@h725$kPK)9|9AdHK*OtTPDQq@XTT$Fq#B6+g7 zm4!HEYK{rBJCfb|=v1rwC(T4Fi}&cj^>UHNllFmZ@^qQ_(w&7~=|E+duNCz(F7t9a zLQPNopXAZ=5rWd(>&aNYHuXtSY9HH5#Jz@5U2g`+)9Z{@ZdK2NhEX@bTA_Cdv9b3y ziB{S;9wgyQIwp>r4N-r61jsEGOvXAm%~x+i;5O|o56ao#KU#3H@7yeIhd78 z!m_EwHa{)%0+#b?q_VjjS<`(@Cw#pqS&P+%;fLe7yxIkUix10U$*NZd&AvrE`2XSoAk~@P2>iW6I z1joz(ZXB{u`WBqGPGx}udOGH0H|v-c!rayl%WzKNUK!3`_x!v}$@dhngnDxm641iS z3VQH78AE2Ogno#!fG=Up%m-&%<@dg-O&-PR1605CFjH5;7~AutM=r%*l3^j`4#_rC zfVg|Ihi_XA`N=-`wm!%y4xBOf$w^XdNS@) zu}BG67wqJ1ep8F%D~imiMGgL%a*=YTcv z0MNO{04qY6%Gv1Y9AmKz@>D81$~(Y!{njK%UtufC_}iC&jjAR_vWXS)LsJovB6-K1 z@ehiWn>Hi9S18NzO{r7b;1C%G`L;GHg2mzu5IcT7YxQdg6M&Q+LoN5|MrbKQ~ zx%NxM>jGh?oR-9Q8jZ}P5@Fbv_>TWHhI(!dsPTqf;J(X@*0e!?=adZO^F8QZn4zR{$5I zzdTS(lyFoQa3IhP-~9ZzTO7ol%AE7}f!4$Zl^8|VA#Br9DKy;&mqUepb4=i$Wtd&c zAreQ>4^i|yuyyB$`T#xznjo|(L9?4J2Pi+gk}=4nk>>X#K*Chmsj&vwsw$MEy1f-# z1}=B5c-BxC)f>@5DD04^32_@Elu-UTsy=dxX$O>n;@%B^tc~yV-TKXR?*YE|4Fmz8 zb+YJq?;9z{A)DNbt68PCPTy6Tj4gS5-c%?LqiA%@pw{BghPIIno*W*Ud-3PZCf836 z?EwTMBH2_r<28I_>&9Z*Rk^?lJ>8j?OMNUAMMW8=t=5I`j)p}cNi@%d7URYLh~EtC zuybNNYKzkZ2z`r2HmWyKuGub4%>F4a8g%8dX99FL0kNiUrDR3hIP9D?zhf6`B(H5_ z9o8sI&H%ffLv;G>P`<@5Z{8fv3HK38-a1Z*BYe4z<5`1z;Jyn4KUO3!zL#TgL@_7F z)&N-AM!iI@CaaBMv@5##1u2hkm;AAlm<6yJ-Y${uE0>zznw5yCk-Ch|%84YfvuImX zyCo0pM_Dbs<*(=~0T-TpcO?Mr7E+IV_s2AOM(m$#x#hzC0&C%5oK2Hi!O|L%n(x>r z;AO!6rykn^Dq(>^R%UaZJ!d;9aDr+bvsjfSGzCv(1!K=$+shp2M$qu<2jV#E9E=*6 zFgA-}Z+_TsTiw*EM>yCY{QiMn>Nj#1ax%-q23X~!%E zNlIr-P`@L=2xOOlbq>*6W`y_wGR!U|=mW9m#`;qsI~-5e3AhRqr-T3?JG*RCss5ch zs*o(b3?LaN0^e3CI2QNcetb3`!Qo-*Im!MAkWac0D4@k-Z?|m&aQ^vSpscC+w#{}~ z5P%>$R42~V%aj*hl9Pd+0%-Dq-b+X)8Be&4JtXen35?Tads@Pl>H+G#hWL`Jv1uCz zTXt_O3u4jH2=qh0v81Z(9C-1V*4bxC;w4PvM37gU8$3|w*;_&3u@y7LdP?}y#6$?DlQZHwS>^R( zJvqJ!hI_H@BjJIj&UQ5?Ihz`vu^5+z5Z#(%>4S4CjiGN^*}XW32C$FBJ)k5A@UQ!O zr~<%P3xAA-J^_nj;@EO#O3e77y=;DzcABFTC$(93)9iub->WO$r9>I8nlzVO|2bY? zQNazVhi%pkqL#+B@qVpBnr#&VOgt~B6BZb&10F)Z0HpkDy(G`%{O+vU&}bntG2Pz4kSNAAdacabS$ zi>AqKADY!Ds?x`;ICCrak2CM6H_0{5Rfe(-7D;NX`xUeC1}_2*QOG?|U71hc3v_Zg z;N+S=rX#vPO*HoJ0umbw?5P#>1-~=a#B1+D(S=e@dn3iX4TkC>`g@mSn*|G6Mpxfv z=y$UF|9TapG?=HIkUC2PXs##e?Ti_d(_445aGK&oXPRhczh~jGqkE4(=xMnOv)U;MD#B!1K8( zxw#*Jtk^(wvi3d>OoD|Cr^||ar)1`?a09IxcAc98&4Gx?=KNFJl43wBIlO$tQYy4z z1bKCwaCV&QzAA0FHPE;fCz5yir*3mOa~7g`14~8c{&1JBQ!ms@yHPK@C);hT==v6- zaQnRT1t9+tVH55VpWu0k2Z-%iATh?al44S`NXfOi3I zJnwHjZ(aPi6Y*aP?|hi^YkztW{~hK(U50;H8({AL?nL~vl=EiE|H%iQFY*7K2# z=DtN)#gn@bc+{XABJsvJ(byf_7r7I$-D3RKSVLbS2k7Xu;@+Qc$iJRHD>wfI=+WGl zeg*V*#&{2ggMe>YnL0A5eom}54LpA-0~*DEmoG%q;Pcmt^9I7ANs*_jm28I~eZFL| zC*$B~i-({5?{%=`>oVusna=J2-Kz3OSl%T=4`#&P=YdjX#ku*0+(b>n8qQ#(h{QDS zyw8?F;xF#)-t^9oZ#Ga;N5AgAKHj72;KAeXkAT24EJ-T=6)~yGc z8mgdMw+L16*Lx&H`2UO}kL>U_xWOtaFEmwD*k5?LJAhs6Z{3on`e?N5jxeNEh;j!N z(gWFFsYgclRvH*V`1nE7G##0k&NmjN#68DB^0+CqkRd&cK=vv0N%iw6wAD>f@XwIHP#)|sBh z8y^;jYt1r#6L7`bbry}Ry(4}HtSfITzB>Y!l@iaVKh!pHcB?2+ny<(-eMyEB%K%u# z4+yxnrv6G{8zyh@`^Gp-uhln!;tef#2y&#FpS|0f%P^miw+Ki35Iqc>1-8kGZV)je zET=%W4Fz55TdJyDdi27!(DLw7aCFOX;20nc39=3VPsSl@iSqzkkNA`dC zT*J^Voc|IA=&$tX9ISo?G?=@6#b#TJe!}EPQ&=T6n}sIP{we#D{O&z3S6Mp2NeKVR zQJcQ6A-$~Li^D5L2>+*-qjP(YE=S_8BQmSBg^s!O$Dc}=V`$ikY2Nxu!4-;}oq=d_ zxgny~^HEHPv$M~Y=TYopjBCGSm0MrKuk+c;YT+wa5CPzIP;}b2p?$w!qybzk@)oxz zc^>Szub7U{gXXIXVA~gA<6OKyfddM^;mxb#rPuk_C2qym#SBx+0H+U(fbZN8L5L8J zAHoL_#py)*n6_}@xGhsV5=HLHwC=U%@yrHSTHDYpMq9p}cFudoXP`F}yvDGea@BTK zBga%l+icC6T(s7{k0w1GxsPLMf9P|s)=9TMSonbOp#%jFQ4Y~epJDHZG9P+VL|*cD zkmNw$>@io7pGEe1T6BPHZZw}p)Lx8!Pyb(Wg2vpobE|XSs_%DBu9In=$jYv`KGxwG z5dWJa>uIcZe9tFqe8dt;v`n6_S?2dga){1kuClaJfHY*NNz8IuJ(6xjf^KY`U(&y+ z{Z~u@cSks9+?LW3wnNQt@xJS>xoYIiFTjrRTk~QX2!`^c^U-|7zFx?Nczz7=NlW|R z%>s=zqC^APs%0z-O8f7kPvb;|rn0nv3#X$Yf=eA=W9FHoBw=`9i3zL9(a z!u+olZ^1OE65=>X_>DN*O`dXUq$|GHNM2GdzKx=EDkg2-yurDRW}_E_FXcBE406w) zORcr|Kk>2MV87GFj5&T~_FK0}6mJnwzPm;EKbQZbhyO8x|Je-xg9QJ-A|n6u*RQK% zUBdBde!&rTTkD1eMGd;?2ob=faZ4za((rQD9WL&YHSmw?CdaQF@c~>zK?)k zo5&|M$`Y-?e883OEUXCr(F?@E6uFpUYSqzzZ%_@3U`lEC;9Seco$phJt}uTtL19Os z{HYEbESFa6bJ=ioQ2C)i0%Yke;QkGJeeJOQeTqGQ{*bU^FVrIZtneFoM~6wOu@GJ# zK+1$OSlFcWN6aVk9cb0j9jjK7Q}MagINa1mw3XHzA@*Qz+#LQI0OdF`0JjdX8*xvS zFW7f9WCc!bcf2eM5tLDn?YMdT{nsK_y|=7z0o|&1^_w6SvP!n}^5mU*PqLZtd=$!2aYd&&o@gwB6mn{F_N-TtCy`?l`9AJP_0s&~1TadJR+U19u zo8~UK&H1f}q~Z0fY!>Q|*~~}I2=_v$hG)=>BP$qonBlj74R-v*Mv2=GD6N~eH#S<4 zE!hFPIyw$KoEFW)oX=MsAr-+MM6-<2=@J5t&b^;=(}`LSynInaa7lQ6mSM?*jxsgU6k|NvqTtaVYfB1ShZw zt~hriSQ9es0Rw}jA9gK(c0kdyZLzsm(4Igm0B4)vPJhNSx-8%q;s z>v^{d@A6%y`k{V%LG?@IwRhD$hYhYn+HpPI#^tNvMe@I=k*>TJj?yagt3N=*B#K`d z!qy)6?pT=zt6P-u-#qEao|iC}TtY}{yJ}aoQ0>`>>1{@SW79|mm4ze)(v$6#$0iqA zYxyUq2_5mTmV;*Aq}0J9lgDbX6)<~1^l}%^@=%5W0HBg9gygRHD)BR;+CUIDXC}pU zcFE5>{1%A?6}3Or|7EtG%(r9pvR0^XQ)0Gjp&djU=Yd;9{%kPOy;H`5>F-o?0JnN_ z0fn{sIW2%p-=!M1f&w&pq4^|O6haN1{;(mBqOf*-aC;klI(g*c+*@o`zi4r0wkU;H zwajKg=Pr=A%%fxUd5XaRy`Ff%&~=T^dx-`A(q@y!Cj`;tw@G~Ng=x7Ls}S}{5$xEK zYPf}bIJX_8-23~DhN~8Zd;SEaQEa3z)KC~=B5=iZljsAJE5V&Y!C;eI*Ep|ch>A~p zsA##Wf`RQ`qcINv1{WzlQFdP)2~aC(odQ_IHF?@>POW%RGicIeTUIQG!uYgUc6}T_ zQLaVUTB1@vWS3Vlrg}vLq_f#i%0gNIZsmsuo?21M@0#zk-b=2t*^{cDipw%>2PsOv zxM~Ce+kTs~>weqvtIKmXGCg88E`~>k@@H*Sae~dj20WvbGdn<#_%A5$*+XXrY=+jR zk0}ZS_u4bjnpbh5an}&VPyy`it?xoGa!+|mVJh`bt&Gb5vXr81nn~-kyl^9FYjroQ zkWjq`oT8r-MP@*NGfxJGib}T7~d79){-Q6=70Sd1PUs zwB3E*EkCBR=HH4^R#EWNA}ldd;OMl~rik)nY|!H|Gv7ILYz)Mtd{u|($l<=vPP(ml!%tl9Lr^E?cp?#duX2!X`q=+{}`XV;$^AC+A7h?H&Bdw z4TCl?F#>aA5I_S~s4nQgXcu^ZYWKNDXqF`s*H1dakuWI#lI6CXXM}3vSb^yRyYA4O zndcSP-~gwM{=&xb%y&2*WepqJjc1HL$KNzb$P~H|l_M_JkM=OoSGB^z9F~TcnNu@q zt##&csj`ctRst&XbI^tJMdR(~UK~%|k1l<`5en0XdNnAFrJA~-i`vnSR5@n=n6-W7 z9czTOMrHb_K>71~G6s4E4x=}jPGYXVAWqYM-qTLL%XoW&c~q7WDx}A@-TeN6Y<{qj zORdll{48iu#W;L+Vx!3p0nTz+Xf|uXrL`^L#CEMvSD;}c&`qDOHst;e4?LsrqLB3z ziM-O;euG_-f3YoDF(dVlcg3$poFb2up_G_g%BA{5Z2FqYxXWCCJ3_}y3q8mx>Kk;k zPxP-9QPu|foX?sRQll$;DV7f!(hE(r^F}K2ir0td_KnSmWHfIV-PnKPukpzY#zWz; zLPtih{MsgyqjbZ2nZTDE{nja#(86^~`1q&e5$|9W3n+b=oYeg+b`IMd#^}~`uTQN0 z8qnU@DyJ9PalAa|+aKHxno|Ot%Mm>_*qla+hZ#w4_7Yg$LIZqNM&*m%|Nhq14Vvkm$*B~+u{xix?cRM=(m0p<{=RV+Vma+n z>RXD>dmOFpN<^hA;{J?qN^$RBA5o7tgM$ALbI-lSH zPk-%xtwm$+*0D3Cesi{@78dmBf;niDF%F5@$V>~G8*7stCk8c+F$}X#?T8SMCOoF3 z$HzRuRQ$dwCwFQ?DWy{45+d043TX9)T+B!jS6r5HZC_QF_%clJ)B9clYWuAu6Fpo% zLJGVx;`DkbUYuvUgcKv+M`4ET8=E;yDnh=07Veoq`U zs3#?={X#fShL+nm(Py_AUr1LfAF8-|nBJsmd6-CE+Y2*jxzu*Ab)zNs{55T}!nVh5($!3Wgy2~3eBY5&>< zQHqBVkDv)L!kEX>22OmMpoO;&u&!wfFStY8{qo9azva?6%|lR+}K?3%2r9Q{n3JVP^u->BXbqbcaUk)gIHQ z;rg&8_c9lYplWwN>=Eh#U4{R9aqfN5`nrPG^>SCE#@T1OR>^@$>-o?W-91R!K(Fyq zv2e;ibN`&AMIw1IAu}%7IpSoRlU_}nZSlT)Q++@?q?BGSgE@v9`!sT#|JvB5bKQ+W z#*~h9MomT=T_lg*Jexb(wYfVO+2&!{YI*ova-@Ny=GXef9Io4U9ecDuw=wa1&>0Um zEX_A5%R8&D4-TI_`?UfB4KW@j8wOQg+NT@^8*F=LrhKfUu2@#1VcNC_Vjbhp#(Jum zuh^Na2jyi`@0P`KKKc5R_^&kOmLdU^3Dx*`SIOh+)aZwfq5E_9(XqQRhM8lOEWaOA zg>sX(H2;DYXS2&crDA(;Ex51ax4uf5VF`u7}xIeNlz+P;;d=MUwYyBt5O^7a^LxNw1~IM-6;=`O>hLx*~9ImH}JB zo5!~ErNixtNZ=FGC;GF?0D1D0tFsrj&n$2BYYF%EGHp+l{i zA-cX{9%BbeR7@rdz3zQLFxV&Y5nYC7JAM1K(CpOw!!RLE^uNew2H+# zW~{JbG|#dh8uUKHtc*ds+kMjJUfzL>$K1F_$G0|R(TeP~&r3JqvkXA7}?sKnHPgq!AXtVgQ zx)7Y*pRF>qb+vw#8PaOqC&H52qZjoWP|+00o+Da+G`|I&t1cQgT)J|$2&BZbKiqi6HROR&7$wOl2p&^!?hOd{+o;OD0e&=f~vs+qk@7U|3 z=-_^IXvf3C?wr@aj?MG+Xdda8e3k1rogUX)(Qhzs9|@kgC$fA#(?+O0n)4H{)!JkF zC~cCdFSH!$#*v&s*Xgcg3Y=-5I%x~mcCGcE@09vBqgirX-nQ6Tbn><}LlbrutIRAk5{VsK zV<4_?$Xl=BduuQ3F5*)|dxJnrjtLx2a-wFZ;t7e4k<~I1g!36)ghr$8ppL}R$1EZb za&7MlrG?ESQ_xnYs=!yCrHhO9_Dw73aX$fgb1P248AGHRWRdRvWjDUvqCbzx z0PWJpyU&u+cxbC3l8383eKK?n$pf*c-82W~lt0pOA?m4YJoQ{_zNk#Np6`d^?P`1^ z;RyW3F#Aik`(jrbM^DgeF`s;Mkk!wfe-5h{IN8DNd13$ECCw(#Vr#Qb@j*89NPmnaA&?2lij@djLI_g)OlBX4RYXvO2x4)O*!|P`x z(2Cy(OL?ALdT7`ntNs>Tq@my$Zg>4b1XjhC6rqLVg0@Y0h5H0B58YH6Ule+GiZUG} z_YeWUT!gu)p&I4_&gAAaFGa!U5Kk^w1=Le`84UVP=7m#{63?NU`LLH0LBa>t|VR<##=PmE1 zX2z^Q*0mRf#usbD*yZSK(-t>BWWQP5Pz1%amxrVaapp3oYo_JdFW==CPFZFuvSvXk zbs(@h_8@$98%_#`z^+|2t;h!Vf6V{Ue2>#nKwxZv&@VLwZ!}RCa8#jE$nwb^QW?*g zSw0q47ct6rL(Rjji~OEQEmyiq`AJxDV7LVuF~6-~l9B%uJjS!}1ko!uk5`Q%1)qQ~GBxLp&zw<2t@e`Ofuij7XIj8ie;rLc1!9kI zKC$m@%GTyX*3VS!(p#NOow1rMoQ;=Z z$|7%SMv@R3|I}9$%V>&^BepLu%VR?ymUpI@IV49k9ecfRXcvJfZa=>|ge&7YlzZ!J ze{mbr>us8km}QqXF3(58*|yNP4q|^FLQ32nXkhvp81{p#F!p7G*Sg^5xNxUp8RbZY+6J2vfFlKSS9AdKZ|S zdGrc(A2d7%?^0;@8H;g*HENECUCS~fctE0Ftl$skyvnjNcXnfEUl|1e&0s80 za`4;mP7Pq{rPAheR4)9rKh)V7QL@GXTSPkwOamcHG38kQI+is%qhBW>6~Bl6m23%Z zwY6IAmrDe;V}oYR%wn%}0PVcG8=F@*yyd+(ojHk*c#?Bt=--rH^li5Y?3kzER~F~h zM@POcjKaPPaaM!Th=e#M8cxu#a}7esH$plqXjZ>8;6QLRA;j?0Kcw^ZNm zR&L7&;`(D=;4I8OR85+#lVd;Jl1HqTT9muqAir}RAlMZIw}(w?Q<2bezTW)G;;^R) z!STWpFwzR`u6n)A3{jX(X)x?P&QR;~@d%!NMg`Jez3xo%5iZ$NK`oQ289gR@8_xl! zgH0N@NdY9p!yKyEtP48MM&=2>;?)YV?pSCpwhOSTY_Y`%8On__HjyrS$&l!-U*UC?{Iaf zD45y5YL(-!2zDLLe@s!>v1!JqL-ozjBPq+$EZi(YCs62DZ>doHu+O{NOXV#3{5*<$ z0N`pVd%pk|?shFFzWmn8R_Bwce`bD;7phKN58|wO3tPgYd+0<)(93Eypa0&FfB;Xh z-==`f8C{x}u&|^x!dQ_gn_I|2aZP!)fh+iW;dMKsC@$ytF-lc{AWV&WS;+PvW^ZwSU0a^6g?SB_w zDR0NOUZwgp9cz~4ut1SHQ<`R>hCww`6#n=y`boyS{;NE`cJ)hONidY#wckd+#{qlU z{146jow5u1y^5VMOpEk2FqN7MVov?lQNujfSnzHblxJm$9eT8q%f|W~DYcl(Mu`7H zG|hhD$A+9a*fnQdC-)m{wZXsdc0r?$Y=j7-sqhyQHcbEbmI4)f30nl0!dckIzY&VG z2G8{Js?qolsZKm*iGJsF1`IpPM}2Sg7<(piU3?P*G9?vZX+~}*^ZL3|`T9O6b{J_iQEj`0>?Iiu;Vn%&SihOwt;Jz_gJuAgI&CEZ0G(O*3bWHj7ewaJ4Ph}KW;nN>7TAlsy6t1>zDmWcgSuGV|)p7F!F7*M}CIJ`1=961Iphmir4kROKCx zij-_?;%BJ1ZJ8`tx%qUURTiJ!lp_1ASe~;dqjDR%OLcs+z9^c^yi0rJlt05Ut50LC zpJP;~qdgZl7Hh?C3(WCNl=OBrAA`13OxFxezj|fo&D-{xz4VaH zO#6ek`=7~|XdrRJlGSQd%{AVoXv|x<#-~vmfh&fkmcu-VxwY5xhSS8SKcVo=+5gg*A(4G+G$Dd(^WA@WcKlCxh||n_^G~toKYBXGJ@Cu%C@%Ys z@_%`jN9Yl8%m;~k@V~c%D0Y*s+y_g0k(2zFXQ9+xjPSf;8Oi@5!~?!};)VYQ3b;lR za)~N>$(uY_PJ`Fc5ZC^Q)Uv8Ip%Dw~P6qfsUt+7TFI z6^yCh%JE-9d3V^|!ux|Z+$mzBpDmObayZDBx;rn5viyU8pU01Y(2%-=J|=bVk!L1r zvh0*w6!`f=Uu=HCD1N0wxGjXvuMQQEhvDr^@jVYAJ0;%yFNzD|qUp`ve%l&Xp(qzh zjS=mT9=*W}5MyixhPAN9{N@-ZA(*iah4_rgpQ#cvdnnfZ78{SDD@H4mOyxg87A-JCmCbW;c9Y)W&k5^S+`!kWImG>=a z^GKaVGQ07wO47>PTA9C@G*{Se&eeUcFE-Wn=QT-SjY|k1$%FW^CRy|MGlZ^R8ZYh$foOhi~UK45zY6Z_@qpbqhQ@60oH_*Dx z++ce7idM`GZZ9-G7kY`>!fWOey;Z>K+w%H9(R`vg4w>M;5WImPnw)R&#zBb(rEBcO z_%{_~)nqp0bZ(+6><-d*nWVDqaH#3Nd)vWtiQ&h!s`z07>h?OYY3TLU!(b96kBAVw z$j!(fHA8|E3RpBvU~$}$7o%k(jh-||%f&_(lGIkTZU0x(m8a72n7qz!WOYAlOm6UQ z^V?CG&ChA(Zq0AgE&Wi>bT~391D|jZ$s`z7*De_S`Wy z&_RxP_@2-YN=*HX)kJAo#Upp5^$ztzaR(irL7sHgf21wQaL5#x_rNp6*4KyWcZCdh z>vwmK@@}pivn$2PB)9i(aLYHnN5RrA0{yIfMQGg3C}8h%1O*$bzSU;=jD@%E8tM6C zym^L5EQcH!*#YC_4mH*qqei>E$phbxZI@NDV(=o!1jmcdbPkJl${ikzNh3c8WNl)b zvAeAWCEqgz=WSzz|1Q%+x=w~(A=wH5yG>jTbH0B}`DE7;RPsxTa%Js$Yt%^bL_Bxp zk<@aCxa{fd8{|cRoYKK+B-;}5w+&FdR6Nac%XLwHiy8JJqPXpw8z z;TGd{BjA(vDH0!9zaS=HSZwe9!?08j;oaINrkIA5ALLh7+DS|lcLJGqd?s}nM4wG@ zbM>n+{|VQLe~;Y;ec&z=L0o2n@l zoejKzpC{$9=-P}7$6Oh`qR7GDHNzrV8E@iby!x9~Bbb7if$d_Re|vt{P3mQxNoTpW zfU~crO{`-ZofSU(#-q$67Y5U~oI4D9aP*m(t{Id0>6rOxDGK7NP&dI{rHz88gDe zG0$B>LHw=hTzM>iNPODk4k|-&LR|fpAr%le$Sx$dvURtmc}Ig{zZoak{O2Trc9YJ9 zw|RlA_ygX!ot_;t}^_*Il zwh8mp@8+Y~;{Rvy1qUlB)4g_q>$CbWSE+QqHH6k#9yYI&+a3tWo^(1@3FN?M zGd+Zvy^LOeOdWe;#UL`AM_qR)>=xLAk0;c{q2n)<-sl2U&~gXe?Hii|i_3j+EKGps zZDaUl%86-IQFZ@~47sW*VC;S~owa1~sC|0$N9Xk=V>`!yPNsxFoWD%S{msT6YV9>R z{XJ>m$=^?tJKinL$Sqx^Htm2fWTUxwYv6@GF>sIl9F)=n4skiT6?HVu)cQH7uIHel zTc9XA$=l&vT1mOX&tq1`2ao#w{ihw!bMK3hKiVf&gIi>zjOo4z*Tz`nlmU<3bQRBN z!u;3k%biMbzqE^mZSPUWllKE#!%rDU_7XJ}#`$tMQZ07;1m~@w*GGB%{T>}Zq3^pV zo|W{l9!<+J^GV#FzQd&u7bL%)6uHy(1y`(YXfgUToY+w!%yU#=o2WJ@vIpT}qDk6c zhYQ&YULM{OpFMDY$5iz5TXC*w#$5)BCui8NtS*cI#t}%*C#pg8BkDdz5Yz7*JK(Bp zBkp`XY_Gn<@p=<@Gx!rti~Y%%1XzdMGo17*JHZttUEMQk$j0gmU`xJZHV!|8n4_-4 z#Z~>Gh1MhitSLOO7_HVz7$I{VhD0JhY_=WSz@j% z8@6lfWa0GdD^y#&LSlAJskzs7h^0DDw1*&Piws3)PeYHgmbs;R^%D21Hl^fZX|p-% zzmz2gHhan<}+<%LN_n(9>Nj)JL^^ABL#Z4 z&1~;cR4TW{=0|k@gZc4b6>Au(zLVE6Hs@oRLwU6*l&9f9SmWW5nu)>!##T6Cr+l*DM;SEKw zGmXWMZ)Lzde~&{G71(h)m+z11P(>hr)KP<1(So%On2JP#<@O2alT|40L_@_e=r?fX zz0<6w9B*fmvS?Kofu~pK1dN{39a1Q@=V^S+sVl6yc*=FMG?10rUtuO!^oeyo2sRhE znS+#$N9o1v?BbI0Bi9y6-T0fU?o_02a#o+qv6rhWy89!QgBL*ZT&KBCn}lQhU<0u(4t3{k#f>prn^W0CWDi9xw1^Ven8IKfdsGso;$MeFV(;f7aZ zh~mwK>bJAHUJSp4Rlm|;_(E(IAemKp5>$U;sEU0N<$uod%r+Tb(bnX3)GcO9o#iPu zZ5}pu;xI%(PtU*ZB7(IVldOtq^#?h`@V#w+;DnFXpJo~pTk%8o3Gn?VZH~3AFSq%U z2F1GtUMR*HUmd^)#Q~=}63NCh1KSE_m*(F_Tg@}m7->C!`1qsNigJDK{oxpoC%iuE zyy;ax3)h0tPzC)Rcdl%g47xh#M~=+}Ez%U4U3D46zJfc>@;6c@rse4-f?0oS`@zOWO{%SA_kD5IN3t|1ha@SWegoKH`(v^{AaO*O zeKwx&vfR)KQa;m--|8g^I6_4AUgg}nw$l)=nWxH2s+>GEWf8(%=$(eO`{9cza{lVo0GrU34r4y zl+?PQnuaTnP|we;ILisYRXgMkdtt$JYuYLnM^wdq&~4B1nR(VS4$C3GY{x1#0JSsE ztX{c>qyih-@t!ySL#7NPHGBgYt^b$Nn6zwM7^KwXdF*||q~%#B&c1gKTBE(lnEwe( zGgy&$YeMt) z&ET6!q&2I<_>rxNvlgO z1-vP&)L!$O>?xVh17K3jMWw!01XOd`A!DWD@Oz*|16k_b>U22OgoTLY$KBS2KkTd* zk@(H$GkCTi?%`ej#&e)$5HA$4zT?l1P5kuhM$D^wb8(G#vDLa<|FBUOcV@?E=|5=E z6!-aMJOA4Pd`6CnzT#!peqhUH!pg)jq)tu+lwyN1d~4k{lSVfo<60YD%f6uj1%Vq}#GL{`lGzqNN>c~*BVYQI{<>i1;J_W-V@|O2%<-pSdZql;h~zOo=smC? z&XrOXN1O?sPU;8G=j7&B%ByDXi^B^`<(WC;4Nfe`+5yL~*j%O>=gdSoau&Z?8~lSn znwh6!a)r&2S_;i4f6S3@&n%SKzgBYfQpz233FkphO$(!_CnsJ$n+bY~E$L$orYj|) zvMC-X6K$gmybiBxnt8SI0au9f*^oUIUvGN@GM@Zid{&eQu zf`J8VEq2?>f}8)=iR?(iafMl$=_xJ!Itm9DI_Z-r`s8aM>N|)+eTZ*##@@#?eK`GH z(>2f6$_mxpSer!kMs=HG@bK13IK?j2W-9o5MhJM@8{eDA&AtD@swl_L>=mZ^zWdX@ zSSAMj+4(mG#>%;ugzoXWm!)b}e7f>SqaD zOO$V@hG!7^EqL}f{q2sVh);@B@wZtDIT|^VRVi)$6OLQU3XoVciIV?=bb)LFmZNeC zoM_dES$tkBumG6*Et4@@Q;uN;x{`C;a6X@I-ZU1STi6V*!|FVN$0Vt|FLV(j%@J5N z9@lbw6qmYY(&CqjH#uXOTCdXtZ=~r+Y-1ZUr-`OG>~sPbD!jPIHVxg>-sDg1$Z3>1 z8c%R1yas{4(DYCo4lc1#@ap}Jr@IcemACe2Y&*=4nK01u_8E^Jd4=B|4KQ}(v2HYrSdm7kknVT=?->R{*94^j&)|rs+p%eQ0qDM zt~G%=v#07+!^)6Qx~ zzeq>|x|}Rh@lU$=CSIPtykXwyKmi)ix}Ii^C%p0$=AFvQIUx~}m)&gFlmP8Ct02_* z`R=z>CeP}Mv%#8^XXr?YP^#y71F=wiZl%uTTK96T(m23N)h z>d+mU{d#m{@Yr1SQqYc?xl6U$KN)O~N;~@2EIvQvg4i~ytCcnX$!TPX#iyXAa;p`t zwsZFBu{o*Jvv1-FrajgRll7v5L$C1Z!CrV|a_pmLfgtT^O3vw%!j^NBJ4Y$9EQ6dN(k$ zE{itgHDGxzt%6u*pOaH{+T&u`&O7_Iq$uSCMzU{ly53fU)ek$bZLFA!}2`n~CvY^-p{sAzGL9Q~mz> zS;?>+?Y)$Fs=sX`e4pZPimHCq$dH_Sr~An^WmPy5-XPg-9?IWb9eIj=5~^t8%;lD7 z)AfkL%r>FEXZ>w#rI0^ygHxoxp>>TxZ|6?&rvx%=I*X+SuG;Emc>V)HktIk{oRTSF>f&SxJkge9Kd3ve%Fe3X>4ZB zwQ-T&(rb`M22t{d)iSM)u$6Rp?bPEzq_bqR@P~9$hm5Ak&n!`_LH@hVn|+;Jc%DSd z4$zQ14NjsyBpWt!(cVyMY9{?x)(3celtYK+W25_fkE*y1vk~G1xhdi8R6Zd+%c;`& zPUc6GHt5P>0Y;2qKVJP3TboXI9h93F2til(D-^S?m6fv8L{5p0$3$my)@m)j-W||C z*x2!XLHjC|X=WN)P=md;8h%r=3L*o(obz1u>fp4r$V}ldYgUpqkQ6I`v0dWc?U{u5 zNr&*@E7OMg@_1nQoiuNwP~MplTU*CFK@+~#B0w~Zo%^`D;=h-W$8YQ2BixR6y1}?X`ob5Z6JVA zTC&;iQy!Ea$h%g$^ChUr*ODPAqD5#n!)sy@;F5zHwP14s2J~GN2%7+({Wu+l)0DKF zsXQFUC74YyM}?tU1W_NRaU;n?wmfxN-v@X3{* z;cqs>&rX>9(!Y%}SS4Km(4q_Hvlni~B3Z*33U0FTRl9Du>km0VhV95vWzb{mnZc7& z6EC>!W_;(5mu`!2tVwHbVxYC0Z9i)(#Y1V268S%!+aFmIb=$lqK4J8!|8OZ&G35D| zziqbQST}aBm2@LC&-2l3Gv@NGwW&fBzqK7~>tR{<8}h z2VLupp%ZYuVNq)x@%b!|xXC;Jl?J0zdHx)PQrF4Wv)e`30!g(m5{!TQ(H@ zw!6x5BCu4l9s20f=R?#yLd9$&;Zg}G%tc$Bb}Wes0X23R50&On(-cx@x8r0{&hU=t z;7K%ZyL?vSFl7(c}%#%OB-mKZsSbou^a zjqP-`0;;t8K328vL>?4!RLq9Av%RG^8|bLIvgM0zz2WPXmycf0(AlYqT%=9XI`m}q z8+xU!SsfAs@$J*g@L~~%4RxJf^Sq9a$sMEmn`G==j^Jp-V+F&Hx3JSMqXP{a zHVPjLXMJ;*`pKht((BmTn$Ev`Wae0^eH9puF}W&sag`*%xB+1mN5>akP`K(V(dS>` z4d(cWb9ybkLcbcT4?8t*LzEkKEQG=x!G|r`9MHkgPiYLLIo{uyIhi)~Wq6FivfBo2 zBfIwxG7g^5095X^Z$r!w3zIess5tyFm@7*6jOQ(W#`^F>P)W{xHjT1W$ND!O9}16V zEvdxqxp4Fn7!p#{7-%H`#a@#0=qICXQJ*oqw{S*_EGlw>khGs{#|F$dE^Nb7Em6tt zm%`1$!Lj6fA9N0uF4TB@()?Z6UX_?Hzm$ACa!7LIRu)JI#X^1>^Ehu#aO~)YT@AAR$@&q$b3t z^pCUsI;KP?$)X#ujP+a!y$JLuCE(ctvK#cn|9xaicFy!%9Wy8HrEzB&thfCs`o}B% z?IrGAo|mz0hIU2!P8c#+wTjK_usWJrL}#V_Y`-Sv{zdDZyLPO-l>&?0)$QXlA)JZ4 zj92{ChhCi7_gk)%zCQ@_N|O`;Z!LmGMea`W=gAzpqT}D_WNr-;hbekfZspQKDl_mx zr|!Cw;b_z5lQERVPs4PkrpeEW9~p1}NK(|kVi6S?!}WO%&Fc1ImIrqfP3gF)2)8xx z_isa=N9R!D?AbE4*BoRh8GJ<(y-G{HMnzAG0zw6dM{mzJjvv{}%ssQuRHlHBfM)L+ zCV*=`2G;*pzOY$V(fpC7Rv4^Bks9=4Djz-!W5OSyP$nnbq+@0h=fCpJ$Pj(WzUUNf zZL;l2qOG0dzp5}X|0OBjVCTYkd6=O@+G6W&7v+D{)V|V@4wO9AB#(=Zt~g#bXk!5P z7pyTen!F^^&7PGP%`}y>;MTzR`zI(o?ich(nQDKs^!vG6@w(WY2Q0=`#Rc6~Vz{A? zdA%9gy05@3uhtTCObh}j^J?s+1(BzK4vk6(=dCh$QW7~d|MN)qR|v@fhnQkZx|4*? zXoABTN6Leax!{OdQpz5hV4?$$#8uE3E6Ro^H?JTipyIPOtly5N;_LdBOOr7fMSF(+ z<>x8#bgkae6aZ|r`|y)b;go%2P}__us49^kwj;`zB#Oj~Ai|A&TkMap^oOL|W^98_b<85V5H%_D7a z2^ZpALuslmam{%IgHmH2v zC2Yyxm26g@W}zc~r`L&*%f4&%!w_$Ub7G#Fi3JnD?|w_qZCx-^QA7~{&Ykw zEWD7luO?qYT9NkiXcB8eY_0g4eEfk?5qAd@^2x(^8x^Mumhw*allY3@r~rOF1&Vp4 zXo}G|F1qXRslO0>!Z}K&)vq~eE3(PQX<{;)H&Ndtr90dIc|=y&TZmISE$+qnnh!y_ zsNjRPGoy3SK)8?Ll$!rumG5_eH>CJ@g>f^!!2&l`HAGF|%at%OHr31JhfchC7j`Fo z{td3|DtLnWUtt5>!t%O3%Kjj5Ci;*0X}V1+58?Ln*9+&>6R{M0S4Wl;CMuB@0LUWw ztH{sxz{X*yB&lg-!LVKm{FyV~8XIBTqAq%~2x9O4yUl80LNI`K zQDQFFqVXkLiT=tF{GxDJOKP-~zB(DHD4ZNe*^fPHqX&FKYcyFp_5ad_d#$_j)N%FG zT3>T3$eXf`Ha{}D<$D`e>THh6Pg_VFxBZ#%>yZe^h=`K>w%1usnTeaYv~ ze*Zs9`50}*(a$IABr@813F|{8uxxfV{v7Km>wHM-@LbXKWT^n&59VM4@ZO5sobjis8Q(E<|8e0p8L|2%{JHk822 z?(q+^YfZfb;o*|2JUdr3$|M5$;5J7r`RmVrwdpwVjRc^sK>!W(4*`jZ zQb~^gt>I#naE&m->I55m{6a>VDAmA3M)~@g$4b|oc$tld@ugeR;1uySt3*sG*W$|| zglw#YG}~1#9pJ%Iq5@wIw=hqxL62;nj~Un_dsfo`d?Ej>%$KMnhu&M;I=w>>r);K? z3y(N~Ehh%OiaqncSWe*Wl~;!Uwc>>hU05-KE&4zgh4CXn;iOyYB98!mE_LI|WNMdG z&ifNG%!FoH_&&7x&3O2kXqdjj1vgwZWWvIGWV-V%>T3a3N}EEM#SnQ~j_1=lHXDG> zr-sze8BhHTY#}Q_vfqa_YN{;@f=Cv>S9P3gH)O%Rb zO_6H9!yrfJkkrIk1CDDWqp%Y37 z+==(TYrXF|cdh&7emvhZnarMN_Uyg?zyA}*=i({R=k<%$9r9}Dzd7Z(XwuoHX8`## zzWRs#YsaSTsaUFHiz21p4nmF=dkc&_E*o!-I;NP}Kfm|Tt!n1WahbT?;3t(L2chlT z7(gSRjt3O`$dtnCQkKZ-g!$6?%R4IxVM5)2(4N>db5a^05eVNNdI+NgWb?&qOxlSo z4+se6JYEwiV4&(IH*i~4Q>v`N+ZWK>Z;}x6gj3xzOtPdB=-6-lmh?L> zez;kNUECa?ig`!4j*gFqY5?(4R&VMXc8wn_@J>};tf9C&gN_Xiy)%Mn6iEpOt7kuB zVKE05{rT`J?PjRZ2L#9k%q-G(2=(Im)>lHea$m1wSsu*9u zM=(5nKmWmh6A3~DPVAWNzA%0Ml5DJFnb*G)?gnU7C}z(!y0g313u*-Bri8G4I_W+r z@Mri;zaFIXl#{}tR)h_em9~V=I|aWuxT@m%jlZ3k0C5uin+6#7 z?vf_(z`$J4HwvRw8bNOhGZ&^?`K{ubHXD~TtwL5b7oJ$zpyaK!o|d&THc3!8Ahzaq zU9K-K9&rdy99VW*xk&H?C~S9S2A~Au zMM9Q8|HgHC0Q_Kv2{@Qd?%|<(T$y2e+g>*1#lkx;+U;l$sQ}-jIs(kvc3$w*D@uIViA?#opT9D+{J>Qu1=@HIFvg2}~;1YfFH>w|dlnb{`J%$H%~ z<(jZR2m!9!BOQn_OP_=nz!YDqiBBtDNMJu@e{VBu=Ss|F?C$DJ?KC(8Tggi?hA+*T zd{5ci!b5KXlO^l*@g)RiCoUtnHz@WL40w0Fb8$9E%#2staZGq}y#j#4JSDGZYqHgt zJ`Ecj_p3fNL5`J)k9C^vK6L!j3#kf{P(3|wGQ*{`qg7GwC$I!jBzL-WJ8+A~F15ZD z3O(27Op~>4iT=LrATeVCgPbZU|8gw!oZg-1ObQ{_mB7_Ypu*EvlgSpyuq*DedsQX&`q3E@2k{0E19BU;28~ zQKk{;Q&SKL+5&4KgXO`?L_1KK%{gpiIg0N~Ej0t*0o~ z!Kwhj68tick47QA3F#SOj+qxl?dyGETH!!nn~<@Idnn7 zQ{M-}xDO6E+;7e@LO|SBB2~>OVg8WLD{Zwl?{`z2c9@P0PTUe7)B=U_Vt(8M@@Y+g zdmb(sBG*Q36Al==>OmjCyBsAfjeXw9e4vA~;|mD&MX1nBYN?)T?+>$N#5Xgm%Z|JhhC`g%`4q^_*j@fN`Pkh1^O&K}ko4zO1^|wu%8sOs2GvaXGye-3ZWr`d2 z?O;IX6KJ$}4*0pM**w<##0zZ7$PmW*fB?IA^nl}+jyR$7zp*0AR32F|hwEjbozGtk zobS=_nC{fcl5ErD==+*#g-Guzu%z*DWN)9p=H1*jS@w-j!!j=g6hd=`(FHy-QdO+} zjkHNA@*f(FWwaf85Ym&>HL)?OOS8HD${g8k3(a7;rpAfgYLzKc>z!OwnkotIr}>2G z2_zhoq6aBDK|R9O&1;9slq@8EBv|UU4(BW-wvM<(-H+3&pxVC%|BB%8-n^^4cWo{- zi9SvU5f|uIy-N#9G=aL8Ie!@*Lu5Ey?tyWnR;;~O_{He8N~%B$_q;D)6uMn&w|YtN zR%Ew%P?&}P(*xMH1IH@YFF*!aC^mdSwQ-;lpnEl`JK?trL`~WS$VfST_$xx%Zru5X z*i1fu$3W0VQoG~5ojCZ5meIlbKM{Rv3Vz+xmHINaGIHnw3n7kN>vjg3v7+CnD4@y~ z)1f1g$vd5+`0J14Sss+vnMSq?X}2tw(w@PSTAbuPwButjX6B{8!)=;U0E!4mWN= zimw3(lv{&7L>F$Z(!@8Ee2$o(#b4`2oR)xY>j-&h*~lBemu_s^PZ?>tdBcyxMiM~W_;}&hgoe8xsgA@K zV>k;thnk8(C5?-eAvRiWR(!gkdyQzW=);w&8JdNpWScmk=_BFRZ9yC&V5MBk+Vpbj z4+nHNi$v6@-(<{rebkx>XZ6@+QMA8r)SclQHZoHuew}%8orill5Z(xh*ULv%1ajX9 z&@m`FldGw>_)98lxzpl!CLVZqgGvAvYDI04Z>j8Kp@eorH_jD2)<*W~RQKJChQ_`} zrUY`gOqhd2pmU{r&S!5uR(nxy1)uEzN#_{MQY|?x!l1j$xRBQrrGNyv!>J_lbJN4g zeSWI|y%^(5i9@yoawJ_Lj@)J_4~++qAbXce8>YQPc3q%{LaRW^EMzwLhJsX6QwV!U zBHy#sa(U$|Z$I^@NM*^2-rjQ&T-oK%`hF$4OmpMha|G=Zy7ki{GH_{a-fCaPa^ti; zsEo@vI<|}F9T&|95EI&vYZ5mYkplmn_1LjhmVC+gKB9HYaC_w-ec#h0oNI2D($MCE zjm0<#(0)%qlgWZyc?pQM@&lvBVoLcb_}z&A&zx>|5d1IGcnn{d zQZxD`$LLtr#tS=)9mLmfeTI<7ApoPz=~zN8-IZb2%a>4}W^&oJ(^(l9<%6&azVn!4 zcQ*olNoUR5d~D@APFT5iyL|>`b;RrY8t>NeD_N1M(oJS&;DQ@hS9TBck-*1w(mveu zXg($`DWw*&_igR!)MVC=d6P ziPzLrOSoiw@N{MX8DqSnOGZx#L=W%QezqoK*ix*M#iJ8|nEoQj5+L&TWsTh@vqyFG ztTII(r=dKE+viTmtX^d(JX1WfeSynVIy{6%!8mn@+IKWivv1Q>blm3?JR1S>%py#_ z8RZyersb2ruC<|LpUhK;VGPn$06%cVz?96}?XJ@z*C@eL;;H7`ockw6&HNR@;iItZ z`7-=gS47hh`Cm@EcO^N!R*pY&&U;B!BK0BGVvDW0g=8}nW$J(eIQZO#Y1yT{&aX_i zf|H*Uq8fb8HqQpQpxHDNeN9nAF`#Ms@3Q9T05d02B}TNmvUEK zcfU0^nyu=bJ~DwE@U-bpjaOn_fAxfl_lXKr6A6%T{{!%w2cGU@LULgl53I1yNv@bf z$2-4cS)ARVs2Yy>$I#hNKSk{k?SjhQwxye9D(v$o3#%9tVso{Wg%gBxV~L~YK=p0a zWNX*@!~as`h&z#Q#AD}ej_GmT8D+9^z%iXPS#6#bEug6~&9Q9>I1oN(5h#2BdGSUz zlr6&jX=_x^I>QWL11`y8|%2l9n40F*(Ez>a3@B za)N%1&d59(<_!#ZNRMtXF~ zJ48=mxbe_({|Cbdczz0Z=RENYLOq4AcmLNfnG|Cx257e~WvxR=*KUJD0Dz;_X|)^1 z4T=a8)&Ptu_&3fMN1Ryzm^iDZozjMiU5K0E^n$U6wq444h_kXLIP}0z;VgHrY^s-` zez0OYDl}TiW{t_oZjH1NmqfKNK99AQ(n;+{$sXJ#jhjSOAT?4n8B>=e7t_7l1RPq7 z5E`gTnQ%|kdI2M_7%LdI*tyrj16zOUrdpnyooF1RCr!nVCo)=#_^m>&_DiUzU#Uuz zs7t^f8`fze0ozPi&=#vI&<`w;;d(iwR}X$9gz_mF`#lcZ9oHCINiwA4$y)v~FVAgLdW1@RPQ%utUWV!h;0SJvCfb8#I z{BdCbGehY;Sht%c_Dc)8W3+2#8Cs_1Z8Ky89)93bQc+aNBr%x4&ce{9Ae8ioFK%4uD~O<>u_Un0gXFSe3bYVU>l z%QtX+6+hco#**&`-)pcdwfz z%jJMq3mu|3gfj@9uPo^ku}uGjJz%Nmnan+!^L4dL2CY0_{ApEO<_rd)HykFS+h0j! z;sMe%m9RB5<`2@dh>>L)B?ut{f*VuPsak~ra7m;g9FtKJ@Ndn+Qqc)%F@Dtdy?2KG zGFob?#r7dFc6Z)091X&^GWwqd5nYEabUA(*k2~)%UuD55wc__S=TT#IM`9Ym;b5@2 z*4)*k#AI0?fd8ZEqGfKNXQ>~t28c30d7(Axh}7k7O)I|+MrlM#RXGUNv3R%5x7G~@ zZ$6GJ*W?e{PjAd1#3#_nJQ*T!7?k_0bYdj*`6opR!p+~FC zs0H7KALlOM6$+1j5!qB8jXiJFJtKZ2J{WS?hFB-z2HnKGm~GK~#b|h4iN-+TrCIIX zL6}&^R#x>|kE(;~S_jEP-xv*4oT1mw+{pakqa1E+-hhSQa9ICSib8i&%9<+an)&0- zEg5X5@=b!AR7s`rn=+vd&4D3n%D-9#Cwcm8#HzpF4e|G4EIYc*JtrZ_mY`JuFlCpJ z@&SG3-hAC4r}x+^BIL{%{ZWQfeur?cWf07Lj>x7ymf7bdjE1dJdHMn8jzc2$3wN7e z)KI=A55z?TP;)A%krC~wzg+2G?*5*F;^G6VG5hk5lv!FxQ| zAs6|Mne1z*_KPbF2H(tF)iV*o@Y3F+!b7f?jF z`ciOO`tOZR*M|VZr~y>`?Y4%0)J(((Z!$x}Dqf^fJ3Enjd`F08%h1!8Gdt9u4lM*Q}5LA)*`nIXl_f4mLg#25` z7T|}>C3q0!^P(**a8Vq!_E<@-h>Y+9INOY0mkf`v@{0Zxd_uf9t^Z3I(oc4uZWTJQ zO=An<9qn~`gMrREP-Ab)58YWQPidO?6&fmQ(wp@5M_?1T6u=r@tU_CJ98d!s{8ij^N z`aPgujI24gwxL%GzkSkmI_6aP^|M#bvfTe)Jd_8*4bB3FN1mQ*ei8pEfil(5vs%E5 zk^^uT{(XaC6{gY7K&fkx>#v&T*#qH1fT2^F_dgWX{}ESbSO7n#%CAc%TK~&;dIKZ7 z3B$iC63ipBd6+d-#HZWBOgqb8eogzo9It;n^8dKFM@8n*z(pG>gu}3bs&#Xu;MB~l zh?7s6c@No;f|cNFxPttfC!b!=Wc3Yz6`AxfQmHm&yVCHvs@FSZ2ZU;ESOEvjN9r7W z*rY(~@R1X)^>Cw{DY0ehJCBSq!BYvy2KtE5uShaBorbXZsBCk1Iw)4>=TPt`X@g?^cE(J4)7C#bL| z*)z(wStmaxRHFR;F!!dvpC4)=*kIy7xElA&+?i^+N-ClcZ+@nZF8Az+r6@h9D(C3r zqm$ZPGckWAq$`V_8eji#xW2CEB_pJN{g1pSRQ+tJ9dnpZm~*2p$5!azkfv$NH&rj{ zU277Lh|E^ zXZMsIyWMNMjA9p&E1a(t9&VbV+$+vlgs$C8ZnCg8HdX9}lms4n*zoo6QD6PE$0M~_ zBlalSCg+tR!;rtOpTAg+|GvQf`M}?vxqm|PPe}fjZvP3%KOy;l$e!WVqRaaS>zb3q W&lP?v1paB{fR3iY&C(l>p8gkS9uJBD diff --git a/docs/img/streaming-flow.png b/docs/img/streaming-flow.png index a870cb9b1839b45da16df5560f6fefd3c1099965..bf5dd40d3559e771081de6870de5b8f40c09864e 100644 GIT binary patch literal 20425 zcmeFY2Un9p+ck<5m7)lWkrH~58Uc~si*%4)LkE#2B|t*8fYi`?mEIAM9>4;G4pOB? zApr?Zg7o$UeV%u{-}wb+t#eolRu;qD_spI>`?{_@L!_?u(_3V8WCR2Rx71XX^au!u zLHN%*H?QM=j^Dq?Bp@LA?X0M%tEQ;Ptn20B=rWp7JDZHV}qpUloT5jmuB=1RI6t@RSl zQY0@SE-yV*(ujq0RG#aPTxm&wcAJF&p+Ilg)9J5PET5M|Z_I66dP9uLONrQX6mmN~l5 zg$0jYDfSp%H?ryQW9iz4V9$KFGHu39+H`a;M@2lInC-%L*KwE{z(#D{Sl6R%e$7?0;QJee9NSM{q9%F= zBLeP({F=iOkD8j4ID#=3VZ3mw+unDk`5O$-GLcFh`Mq-6Go`&P{Ng`WSiH=IEQR6s z6fP3&t~W%qChWN@R=>@r&&(S1mPN}n^1L6Ntf@$K=m07OK7eyP&bR(bu`OZp;hIoc z{5y(H;LB(FL`0g&x|4dz4CB zov3J^9uum$#O{&sl|6c#%IoDs6GQZ;N`L;f7z zcR5YScIM6Vzwzno?T_h1yC<~*=O1`+bqWHAfQAh$wWd^gSNVMT{v_FUWqD&n!ijWa zc$cB7ox5ZFHz^sF_;-LoX7E>sz`(JWmqbwCm}U;!VTqZ$&L&*HZWO=c5SwRGOF<>e zrqE!>F$ga|^R#(iwIhQgqoZd>+DG6cTKk`gFHMisT37?wEMjt|R@UtHHs8R^r-ZcH z!TOJiV?LV0)CSZ+sYBN81yTR05usb`QW`a37pohv+mBFbcC9mZMgso+*Ni*jno$PrG zO|3U`y2O*pAsEXTR4xcM=LFa)+sgKgooxf{=qJx$9tT&HS$z2OIoMQKOI#OPuOW>U z7{fAu*kSG}U@H53$X~YyW8Xtv7yAQPG8JW=^&$G0UYB8>48 zK>AYE)SG|+Kz;Rn%_(mjuMip`YD)43{&QP7H_{ink6UiIPDs98Iwlgl^YYhw#V=bq z*SPKx2~6H$eydCP>W+Hootv+$6@P47kiNyT|A7ng{~4>(BDIFyAQJF<;&{2|S%?}Q zjvd&3;n&!{b1H>;Gn?`;Q9JW^b?UhaH+~5J|JVOF5O^pG+-k3#pPhc}>TAC33Y_uz zD6&{{c{;W^U4kSnxUXQv*Z^gfoa5NZ{$f<69-(c?T zwBCk!fqhVr|J(&uRVd-nhV+_$Aa8?%L8l51QP;PiI#icjMbPp0Y$;K$-X z-9f(hcA1eY*dwtS?!5!6M{B>&=w!k*lSbFpvgT!8-dOCTFTC65=i?R66#*j;3JNKt z*U)((c-VfHo@!WL5Q=W`8UikUjCFt6v>v`@p8QnFbl_;(;+??S&Wzql+Hqa0!$6=K z$O&`w)bRKKTWJ@Y+zd?>Z<^r_Ry;0-10aFC6KzG=L~@znyLVtpCMwn$=H(e7_I|t4-rNl z-0Jy|wwU839`Q4qP7Z9J_Qc!OYWf4BY*Nr3K5~#6^hf_!vzpgUC6x*O2IOX)(RGgr zK6WIA8Zl^w^rc%@IOjkDxA+5o1#H|Jc)&_j0Q0$sl1{7sii0YFxR|;S2HCv zVZFD;K5vxkP7IZ{;rB?*MR{$F*{bmJWAB(4| z6u3WHdSHb_FU<@tmaNnQtB1R$>;MdE$I9%{t9cvQ==70;1=_a2E(P$7Nr$6_beHXk zk?GaX5F?q|*N8GYkIX%9$x=cB=qtp-yAAW1e&Z=;H-0YiWr2^h4PqEu7}G>2Q$DsA zI>^(GX<4moN%Wty7F6z5A5Sl)s(_O_VwAv>HeP%yW=gSHMLQkbLaUlVmCYQ1|0Z+&x`dJ69VxhA*vL+!t!A98!637W`#) z#=xi8?;1#)avwQ&x1S&~Bp+U_^?I`~-7fxFmFRR`ExZu2S)xdHeE#C6?WU^-V*&3U zah63Mu3>kV*)a3_bgccrMkMP=`Qok(s}5IllCV%d7w$E>#W8VZq-j;!;Zt{fZs6s& zLkqYpw?SEqmA1x5+4cIf%J}Rxa01ZbVV-i7tAm$m>k>K5&oFeIOFD||;HN8Z+%3hT zpTQFFsk*p+XYvGq5Y^`q$4lQwP*1&7!Ht0juOwb-R-IN+l$G2tT(ke3SR0<8|jqr{V)6t6$`ZYW4R+1l+1UL<^V(KP-N=*Z! zU`@CvKigr{2(%}g8Yv*9G8eOD9*zmN$3C8mkIPr^Qp(S{e_PJ3lTIqq2o zA7(qK=1j8VN51nLUY*#GInnJFFTQ0^9%E&?COdN)V1v$0mSFC|LG$73+QAQ@EBqxt z8I=X0o1#>Mhmh2i8-;OUe5Vm`H(PX=>y83&FB`Jis!}mIe5%Ta3yy{sdz}~SK$l;U+G&}Yg>#~7vnE+l!(dqdS9ls$nNl@t9Dk@)Hz ztIhY5hPpj4c;v$^+L#5<;+c#A1<(y^?sY`6vr;2}Sp!7uk zjxc=hx^*0ERjKE%RXiVC?c*s}0<}Z6=LyM~^FCxC9N=VnfuUyM4m5%f6rbaEf-ldbbrD{L=h6u%pEA_F(7MW}wHQ5BIwu zX?@469ni}5Yn5aGWq4IrW20t3_leHiD$rtZX_iB7e?HP7CvXnWc>f z^l&1J$-cLEb$&ajYeDoVZ@~>FM^y39(8-?UPoF_xE&xkjB(m7L3#r zZLoz+P?ufb0f=1udV9_;6DCi@)6ONsXssPDP$$l(@u<`pUm2mnMf;#mi`P#EB31Yh z9`=eXn)L^6U*jFv^C)|Puo3+m%%J@D7-f@J&7mpDOLyvmHM(WaoW+wTt;Now_2{n&W+hzM6Oh8DYZjox?!ki{^MQ^2<7g;fQ-Y0N$+do7hfk(<3( z_$8nt$Mw9|kwzWvXRz48S%Z&NZyPC3Co>Ua4aPZZ2`k3BQF5sX^$T14?83S;FTQ{AZlwxD=$7mdStTn&EZA_=z zNlxhZf?RWi*VURMoH{yL@&MycqIqja8dDePpx64Mw|Ax^zxTFR1pHPp~p_IaIA0|tyk`pN*`@FdDgxW-SEGz6Cfpv z{T;lJO5oY==B_b1Bo$};)38!6Y2`7os{yU4x|i!c zIZarcvAhkH1{Sq##qeG8Hs%3a`4EJT5X7?SdcG9Bu>_gkue#eZ+X(t!cjf&I%u;v4Wa}ZV{+`a=25xJ|LTJ#)%Hww(%_8Gq9Q76bz{9;U|rM z)ao{T-sv~9oElW|5Hyg$z(w`pnS(ude;q&Qf~IcaM|>OWuU)~f>v zzu_l?q^UV&ZAiXyTIhS_o-PK@Z`=xR5{X`r1T9_^u8A!CadVn?r>8NMp^P&*XRvZQWF_K z^a=OoHXZn#xEI0O5teIqQlN%-8M9- zfOgZapnP#-HQR3yUXW1dq`nxfgH5AxVf!6I_=a@~Vy$`6=tLMt7!O$Qi@r@GExndI zEwkwm)LF8B?7=oI+og&7BJlk-9z=~egozP^GJ>;ktd_3coynz@o-7#vFhBKI=)Rau z8SR}y{JLh1F~LpGsH2*wku^@ zUF}l*dV5L(YK#_%0tB3cIt$2}BcB{T?Th|aaoTV2p3iads9A}xISe=Y#8xB2O1Y{l zr7?TQuK>=H9&%FnK;EZxpFYcI@AZTZ)d7Ug%scvlJUsyga7&BpfG#EY%MTfvljDCf zmHanHH3SnwvTmMr(rUopaCcs&S9r0LAv^#I>+ujztcKj@$N?Yw(W7J4iuei{5 zKkLBS!)L8^984F9UX1Pa`->h@8`meKWf|*&pdzdf9(0LtKv^8TN{`R#(iu*y8-^ei z0lJRRPtYWIIZxny+2dGiO++*KQBugLgp(+ePm+hV>nWH3S3nHi)m8n>tTXnd#B9rs z#>j&<gi z$q`Uol2^{QBeYRQ#*R!g+gDE>1AkTj?NLq&{gzAr6OnqvTueME)G%-6>j{37$t5jB z#+S#DcInyY)m*k+=VPQM9%ngsvdu7h5@N8c@5bcD-{7K-Q)p4`iHrQ0jAdj~#U@(& zmNJ=dgvTlh<)xJBJbZs%7t|-D2O1I?M3|9dgys1AqdzeUxCDWA27_EhI)mPf$23nj z$Cj#+t*b~O)vE$=UxLm2nyBt<+rS-J>7Ili!DJ!?2WE20ni$AAO4L;pvDneBqTH z9vYi8d6m};kVBV`8jsItXk_R*rEhI=Pp@G#;ir|`GuqqFeokb}*Tzs(;&GY#;~E$6 zzIxq2{brd%Cw;c5qy0vLqzEO?hrrv8GAHGR+5&C^u^*P4ozZpY28RzML0LlnPp9@n z_2LEDm-H72VIdo$tHOFLcLpiVHi}t5_SaJ=4g7X?^sBwhzzOuv;ap-*+W8XB{q-Sn zhCb;H9^)#{)yV1Rc z*?Cdee%?zsc>7+AmC0rB(|v~c;S@l7+0vUVOZ^|>iCj8NrW=D7noUKYh$;fepzml(pq*Yd6cBwqxmjs9v0Ba2>-X!bP(8SWej++7b4KVqS2#L4TWLT00@wB=^Ce zK!xo2WIUMmP*2m*(&1raLwWGMB!*HJ>lJoLUB72JPC=~yMBDN8Z1;&fFo`0>Zi;fy z=#hBmn3-`eo9UGJ;fxuoIoy>^zrPM4z@cAo@ZgiL!JcpO9o;>LKo@sCD!mbbyyB`~ zk1p?*OZ0G?b5L4#H^e^1eZL_M-GogaN@i@$?e{h028M*O&U4$~SdX?2t56v(UT3r; z7jWZ_fE%}8rk;*}yisE2V7HXq*n=227B0QTJ|75%Xzcz7^w9ec{X8^Z`4=0$oLd}R zrmG#7%PWG7a7oYKQ@}B9T&Kd%LIsjFh2XwJu1pplkR7!1NUc>9c=+LX2>7rzPYt}t zU9y^?<8ayfJYC95VE~wfgqzBn&d{*)(bu_PLx@TJa#ScnYM|4%#^=1~i z$Vj(Q#>RNo29ZP zC2Z;OMC8qsnq{;<6HAn;O0{ukJLzTP2u0QSyt6F+w8d{by=WBX z*A=S}tOM3E&3|i`n6Qh1q`8qTybII3*4&QGq-*+qJz!F>%9VP}u_14^GSJ-IEPLP( z=zcfUfX+>8B-EYB08gH*EAk|%VioYYBwG2CyNX!g##s~l9&5);%=h5dB)bgGmzis4 zgAeFR%YN0V;!zj{F4h-HD)ST*O)AsYTGc@qFBXiytQv`6_A_Zl>^MBOYheeEys1b} zgZ3opE0=^BQtonZOg~z4yy2a@s?oBY=onI5g{pXq&uI}=9^GS4c+NyL2vkkMazqm@ zB~<6RGEOA4F zC%Qcl?;3Za(H!TZ#`Rn3H_D=gIvbDkS)CZJ5rn?QS0p#ucu(JJr!67Hc?pUqW|!(; zovPn=BB9%sdG4}=YNZ7KsFKS`-aQqwbtiKoAz-$Q2aFM&7yWp&)|l`V--nyMsW7t> zwPAS~7%VeF@pv+5ER%jtAB@gjsgkp_n0FfvkeE3b%%&j-^^ph5QN{xZ9MoMC00e#H zWhfVV=rsae-{8u&)sWRGf}`7L61fa&b2JG7ju3k|rvk>i><0J$-}cKQ6)T|2OFHI% zqn+REi{|jr6!x?L=HVa%h;zV@xKtjjJ@k|SdTrmVmkq?61?guo%ITusrYB^!S2LUFHsIg3VkEDZ2M9qR@;(?A*kfzuuPb?__*#YCOlo9yVU zobEq&1>!Oj0JtoTER(X^I{WE#X#j|&$4o0CAO8G@1R?a=t%soJPJ~yvAgxkc*U}2R zt&N~E(w4m>-Uj+?hZz(Pn53Gdy2|>QLCi#|fNoB_?I|V)=}>HW-d8X5a=>U>8Paq- z#Hj0Fx_3F4b(yvu8-NuvK*Fd_>tf(J5QbtqN?;|8XWdRg+tNEJUF5W*)t7YCoZo4A z7)2i^(Km|8I{&^lFNSyP4+`wq1X$3@ulTA~GTy2=-!V^0R|v<1S>_6?bI)V;XsSkw z{Cb;85F_hHV9iD5joRlyO|Q$1U+)8AO}c6uJ))oTHMcZcSnbUwu3pxP%%F{@g*0fO zHTNg$D$BhO>#+k0)FTSmcJ}?I+L$>LWTymtx-P5y@Pq|a##Y8A1Ojd1{8=g)78a5v z6WiVIG{o9^L8+dbW5xRTnp;~_$$y?o90V}ZQncFTK=O%$6JK^|)C9d4TA9n6mc+$EhnAJ1=y#Khc|1rMm`a2!=1 zk7)hrj4Ts-LoR`LjVj?6Zl55w>acoE?;mvq}+5>v8>I-8CgfjR$yTgH>Wc(F;J$gKWL(yFs zqdS6X`^7B#BZdQ)f5c2KS@gM%vE=1W+7z?Rjn$Z7peKc|P*chA=EsOOOg8Jt#50!t zDYp`BxQ1AC`A*{G{56#D{%kYLV2C}n=@pM5O1_w|9MrZgU6WNnO=$p|k`hZR;%1^g zWa#%@>}ZCRqbryX-4>#Rd_uJm_3(JSVaY;Y`%t3}c<+%aq+vckZKy9U?P2*Thw&%X%h<-6g)+)tEzgii5{{;* z4h%P9^UFD(vz@u60Op_84T+zbVB>Q?=Io_`M$W?oApx!NNz-DLGL(I3hREnZ_NHbrqj-yKo$QvK<8 z{h@a1lfM?tvH(qcgHxQn~L$B><@7!d+2M-tEN{ z@71-;BJ+QIY#rtqI{E99f(qHl`~-{I1Ex`*R7%# zjH0oy6TZa=klv!@HbD4r^==7u3Kyrl1xG2G5J^vJ!f7p2(J5g-c z#G*rdX>+heJtckni;m&U1f)D!`3z`jZOcPCEx6ip>BfWo*%yru3;(yRJ2T_cFeFTy z40vUc0raF?qEC34YKH2Wn$I#vF;RIk9BfeVvi;S`7Ybks3e#;$9*Yysehl7wtR=Bv zs*`1_a@$%7^b6ZxgtKf|Ir1kp(3Q!L!UGIiG^J->xVWY&NIq{R-tUiGMfB~rI}f1`y6`8G`hY3ql4sc@{ZZR|CjJdHAM=`ehi-!4`nvXI6P?y-UnNN0VSrq0cJs|8PC zC5_Snk1}(7&J-ZGUfxyy&A8A=6?SUJvT7ysG2#53Vi#pZZ>*H&ige89$~+e=b~J-c|?&vU5g3e`I3Oio;@qZwrq0%f~nZxMTKwL!o7e4fvC zF|6u9#f5AuKg`fCj)n>n?LWOikAs%~3nSWkin-V;Nhix1m_}tf@{Q z23lexQ24PX#d;C%vVMQ~*~N75Y-wTapMgml=OM43o?}gO?=t4$!dJ3aG}I6JAXN_V z<#7dY_lG7jrhI37*|45hr7ps=MAiI$k8)C&>(CS&pz;phzQ;I{*AKTB4NoZlX>M6wXKqRFr}w)RN$={H|6X`jj3Ev1W8KvMlUIY zb)V`!wYLH@JZRY5uLuYXSMWY=HaIf5Z{tQpMe_MxNqaB)P_0OVW!Ff3D}TyxR9=1S zv%2zYhLSVsVs*FEBGa_DyNT-4=FZOvq0e0ZYIPHSqdavlk~BP;d^}$5+|P>X8Hzwj zSM|{!Q8-CDJ-qy*D$fGRz;4^paDcQmm_fns5OUz6ZeO0jlZOGWp)CX4e8)+z9E#`j zrHU^)Q?gquVTqtmXy_To(x|F?6xhv)D(|BN$Eyg2nE764weqo1v z9~f`jn-)GcE<0olAM-xi_IGC5A?BRsjiM~o!7nSRmF6^As+fki`3Ys#DN z7Pw?JTDOvnEUA^pgf~voF}_jpfGZuw?*go1*1*n|V}& z!KkxWQDqXv6OP5}TXlk+4;v6B^jG%)2%$n!T=K<`SX`GRZR%P7Y4tV&ONDiTH2-tv9SN-YwV1!Y&P~z{LQ6BWJnh;`+-gb$@)(-)xi|erUy;&_55lAvRWKW z`rWxCY%SrNuQDlle@_zQ??vd!(c6Y4&&k(x(^_NWPICed%o~P3&dE4Wa(D*j!w&s% zb;c10)k1;D%3Aem%JzkvS}JRDI!A3ApDkYQ{+Ls3%8>bVD=W) z)qX)yf5U)6(b&gy44g6nfS>KNeM7o^mPux>B#oJ_%U{od>^7OjLm|x4W<0FVF*?52m~P+Fq67HYS|+MIZs%iD12^uq#A^xGJsc~R zv<)G2kK!dP7_a@s747fj4WJ*wQ})^&Cf0a@jgm{ej7cFD_W>Jx$veb( zyW!$Dy-6O(Ehtz!;jcwXImt%4Yi`tY-zS#Aw)!+J9+LKKak1nzH{R3hi>_h~J7dP| zw9&KQpihoZucxc!^vkt;^z`J~Q#CkIbQ_8ZLxAZi9dDJu97FffW&%)Ev5 z4PU**!Y5s;b?Neejx;TshC;PFS65)vU4Of$+zT!>3yVWgit0i#zFW)cG)|v~7=?)} zE~v`C7w@X`*C}7xfwJN&-p2U++`EYcj1~&G<|?NEQHjB-QdrzHc(~hO+WGK|(UO4P z7>pi<)K~3yu&*+ilTOU#y3Omn+2S%~<%Wwube-NJKD4*wy44&*$i-xZdOt-Q<%smD zlVKKGi8$dJtyfrWy=@s*lSmU@>&tm9B-|kOIO3`%|0KbR_xo$_Y&xWfCx%$r1j!CG z#1^%lg-!cz8%ekCYRr08?*~;n1<{VO{E}vqo>#s&R?c_D-w(JO1=I#UPoG>q5n?_H=(`BI&vKPk z6ksbK;|0~GMo;?bkDQW~u4yfrAtWNwY4L`1N)L?WzQ6oWYro^W(E>YuBQKkzm+UT8 z#0FJagrDtaN3HEtxa6lIT4(XabUZ-hi(78MW-SMM2tBOHug+WcsH8Q?ul#964cC2S zdG181A**1@_T^4Mus4%HLU$Ua&%`vAwUl4%|hx_ezg}n!o6ke^NO;A`_=DZ7=Y!89iuUS zWvMS5;Xm@?X7JBsarwB&9*uZFbijpxN5=K8^VtV97j&Ps;rFs^YIvJtz@I1jIu?$B zRI|0PffFq~*BiH9&T(61?(sp!T{rf~fprNgf6FKC5Z{sEdm3g#f~*5vpz(_TBKORG z^7(qye$yJ^@#NXJuWe_2lHHV*xrNw^k2j%I>>Vqyp8La%QWnSU1^dF&Q#m|Nxq=@) zM@+p*3E|;Hli*kf8P-~vF771t1<}=Y{LI~&rRnX=-Jv{mbvh*DDj9G#xX#kwGu{^+ zo(QP#^vO#R=mU=IGcPKv^jgOky-380x8$*B9!S0iE4TadeK>;XkfuPqgwL{pVkkLRm2k2HyzJFezP)qzBrpdxev%0!kLV{K{nl(6XZ)~?%aXU_yQ|u($wG7_q6~+m z5ptwRcAX0JKQs9;w!H-w;E+W5xH4B~vOa)mGVNb5yQ)Xw=wqTNzSG1HAGu}GW(#u1 z0gT;^YMmZ6g|UWblZYocvVr@Ki%ezP+$w{!x6Q^kORJg3zGw$|`2jTuwX zvoxhCOSN>0hh6oeHw5ko(7sF3JUcf9>x z8*yYda3>vYTtb+&c6_1WWR8TTwkkm&i+9oeUvesNZ|ZusR_J+-Cur~_B^WS(KPK5;U*qCzwg5i7+VIKFp^H#hN?@@vG{zQ%oGl^nd|c@ujRO=7PL z)*NAbM3B#lE0`lBf_{6Wm~zQ&Y|^LS+<2GMOX$nq+w`sevO!7wRlRQFQ#O6=MWf5k{!?tqHh zN~qyDKH-Zy&})s%po)i&u>KVWc*D4s4Sxns;E?*? z7lz|sm>g@{wurZV^QCYF>V$=G3<*T3Fm1Pwysru&XH6g%hQBpTavk@&`ySqGX2une z;W3v7zCsX`h5xsmO=^C4bv#(LFh%*=b$a&ulyb zc=R1sIJpGR)Ae8DD-<~qbPsSeJFDyDJzFI!zVXU`4jt%^K3CE1k3*bW5??1sHU>)_ z45S>7IwABuU*WTQdh_)bC1i@k9E4&u`O3D|y#?Lv8%$g4N@jI-rsI{ zOxzP!K#d3Inyw{O@Zeh>=xR@Q{R%kgor_~x>D8j^RQoPYE9&`c#BK)&bEsx2b^n3N zAN)PRB?UYCvMs3Ha#_|>VcGRue2JC|#KVDyD-`jozmtuNl1a*T0CEaX_p)My;}^KpRJ1CC{AVoau*Lkmvze15fPT$Q@#&0x z->NrHHP4^%50VpKX9h*!7su-ge>n&ytTV)E#vJ-2csw2$AcEbYhrL`>%M5s@S$+c_ zB>paV*wk3Zb8HV}Cco=BZVC+b{DvNZNlu@GG7&-5S5IV51;7ZZ&z??LaoqF>ls-)g zq#nQj0MAL+esbsMGN-GtA~b0_5-rwH&*86K6|F1Tf0j{EiGT^2;EI9$L1ZL0&Fdc` zUobt~Gl+!C!@GHt24^;3J{F*RTGW^(7wT@e+rRjXTdr?M4R+x!sw#RanTFI2me?LK z8-ZgcaD>da#_#=mWwcHgtYLMaC`1hNj2h}WkyJXh;MsZn?XGfvG%YZ1YDlnz|1S$} zrU42$JFUDiKN{l#Es_!bdn>adepT;2c3ONuv}`BDxvU=Dc@T;S&Af|$GM@N2ytN?k zZx7%<;#rPIkmrhx@HuB(q%-3U~bt?ezq{|>Em3+yw2J`074bY5QP4E@~l z_&<7K4lcUtl00*zzE`xuLuNp%avc+O*{f^<^EIAcrNw*G)pHIrR}X=Q-@1 zQ*{zouSb-*K@-A1mB~ZD=^&Z3f|4G)j|+z4=Is@^S#<@~pIJ-+vqNW?>5B~U@@KER#G zH19PZxUQK2t=7{ESM-5g`qYMRb)-V_8m|Hp-mdP8)+1KAH4tq@!wnjFk5Iu+5ML<> z;GwvhoxPd0y%IR)mfH2p$~G_y^n23Ftsk=^H%5d+pWz>L4pALHD29tdvIqV`gVg}v z<9$S@3Z^Li&y)7AK>hn9BWwJ9;y+V@6KaG`>)%IAwcFaCY_yE>HC($orMntS6NySr zU$oVR$rIMp_uYzrN3zfU-~M`6{au&U?t!9BofhLq6Yzp?wPr%>kFZK$28GD$SI;Qy z#}ga(`o4%kjK}J5Kk5M0Al;7{8@-fS&%Y(i4!XU|9u>lG9kl*xR}`35u_P_F^eFF! zXL>`;AN0{XEKCGL4b;dS4XIdho)VewKb3&PkIut<7~U6JBJC<%hGJ)Y7RuUGs+e>? zd1jZaxKB-`_=RJ!)2E4WNl13N-Fn3SSGe-UsRr=Uz_NBJuxLeABKUoq%2e~LYLozQ zVyHDx?`amCsb{nOu>mp@l^P^g7J6L$-R4eq%*R{93foCcbyiq z40k*q!;LNVk+XTS)Zbjt#p$Tz&x#?nZQy|2x)ON6-R(;s;JbIONI$-f${N3{uT7%g z*bBd`J6Z(|LV6#cc)-O+ZBCovZ~*Ip4)va)9o>!E$}dYAmkfXDoy;k)4Lb;|ZD|Qy z{=g4gyb;R_syACTtVC!xEGhk6aGK?xXxd9 z>PLStuL%=d)&)uDNEkg6M5XUogr}6izK#1^J0FL3+dCylr6ae_pVC1zGz%Do;aL?h!@E-&NEq8WW^DB;y3m4 z$Da6I#G*EZj_<~pPq;w_BlaoznWXSjLyZ9UWVcpufTheRm)oiiw_bN9bVyHo0F&x<#+a{iVz-tC zqKrFA^*crb8M~=T?|!^JL2idnrb1R8I(2mUQ-2W2cDVn{a>eoc;PJ@T^QryLonwh< zTp*|xg81dSwn3PvMWrYdhC1}wbfBL5;Cb!N zTw!E1ML)R%b%8i3b&reY>y^T>S(QN2OrN?%aoe0?X`L2rGE-mK(SyZuVu#<2Ew4u4 zS3V^%lC^e;FU{JV+ASBAgEn>Fx|j5`T*AmFb^cSs!s3WfyCe#=eB42!S@;F)jz7A+ z(s^;uZ_GGWCf4Z&JgwRg0#E;+0OG61+dOA9)RMV|7a5}C@qnj2RSmIDIZWYvcz1O( zlyzh9ZatK3&ongHIqKEyfuyo|h(9_2|%4?@KBZr3pO@et) z31?6?&9je;~Dg#mJT6@aYCaeHAeM2p`L%$^as%sko|GvS%R zC2+{s?l*K$?DVxs#iY-`iwpZ;XLckmN>%kuF5`C7oMP%CvSIU6a^g=N){)OaBlS#T zC`#Q9q>5)d)6Od2o_o(-{M8g(K{!U4Zn*%C4%0Lal2<*Ge~S9gXbS%KXgZ>!V+P1i zxf3-DPY`5GL*8$aF+E(Q2OA8AbNOkV+)o`oHKRE0JaO(LHw;BQFyMfCCN*rn8#yG; z>|Ow?lu%0PLvG#YnpkYJ3$QR=Cx37i*bJcwsV!w}(1c!BHdF4uMEJiTV2g~786-Ei zeaj(p!&n|1!!1ba9X1H#oaObAjO7Cb>d+P{1iLu=TS-$-yw7W&VH}H-*n-{tD<8}P z_(V09Q{aUXn0R4T?fHcn3Dh&(Nfs`m6JDT@h>Fc|>$iTi?annbUTWOWf z;A*DnhZm7<3Ah4A)<{IzJ5?S~4eWPc#WYCge5b7<;y>!t7yWo+xGfO?p^ii=ODo}X zq))~Ad+*|c8(-`W3f8g;FiS(_@7U~q=QTF6u)v4yzRubl>()W3ex2xsm{aZF8u*h0 zS==VFHd^W+fhRa$7_Gcxe?+);DlL@p{VH6rsoHh zz6$<7B+bDFK}?PDau!!fZ3W_CT1#I9!e0q#0j!porZjbQ+Vknx3SJQ3%zN3HS6Wbt zXx$du-fN(HoCvoOAO z5~G+Q*WzXUWwp`9KkY84!WCpMv$ElA>yyULjFGZF*>1T{vzI{LJYp` z1V(?%eBailH@K7gBJoUfAbODNHVS;$>EEF}P`|#ydHY!FyZAT4l>))T+c~COiL1wg zl#F9tsTk$fzdAV+^E@tUm5zI_0`E>%iMp7BHT|7z7H@Vsqo|VBCgtre=oeph`90q- z*EoekPzn0JWA&@CCwBlzUEe*rlUEQ(Q^l)ECTL#ONWD4?CwE`}4wI^jP6A-=Uqy(x ziZ4$YL*$VKxo*Jw4NEb}=pI5&H9ns%q~!dpE;2h_>5qPoE1OOK2dV0Srcd6Nt?-We zhoFm{xI|g+3$wVQwUXR8h(fSJUd0UP?z<&wd;!LC6`j~4r>}<1fwu-lUCh8fX|Ilt zM~2qcYB3QNqpmN`#W)J!9$nF;u0)%dDwH@Gnz=eh)x>2ta(W zZ3exQy(61+V*YgGCg8Xk;e`$uI0T?c9@itvimnFCV#nQ~`}Uz&y~+SWJ#7Y4CW?2q zNHHYKgaiXTARcZ3{680M7CGLX5=m_P)At=25o?KxGf8t>o_lgTcW8$2)YfiR+hq|RSYBUU=)0L|Bhl&69X_llVWj($=p&66-(b1}K<$IoVAd~s z?=#OX%f5>+e%=sdD-UcsBeW5Zw-+ZXDE}EoxwuC;b|Ct3|BDRca$MWOKIc_ZPIW~y z0*4ep3Z7SZuf>z$C;Mt**;CSMekThq34pN(Tmc&%5so zE%5T2e|^^C*vSd{3^^q+VqjT#A-iU}eG53q0?Pk}4r>lB4{ebHKQ_v^I`x&?$NZmm zuC%SGEC~mrAc}(6NXHggltE!wBnkq?jzL;v>5v8lA%;Pg5O|=41VaRZh%n4ZhgJ3^ zh_Z;7Mj((FKz8WFh=4@dWr>IaO;`efK%Y2Y<`2xbxu5TOZ&jVDThFQUR>kj@S^^8oUT(a+91E&J!&I= z8{ybBtf;zf>?%kyG`M!5+6IR-o3ZKm??rmw7U3wEOR`0lwzK57OFAca@zLLK-hUEeR4@jifv>*;&+=N|q(m&er4KpZu76h4wG--s(> zi|*QN7+k6<(OkpMa&AW=fXKfYezRxO^e@vk4?E&hnxFb9npbDU%5MO$?DtGFuZivl zF+h2pRr@SXIlE!~IgJIn=103XJDb#nwPwMcUW$RuNpD7l-4ALk44eqyTn(JaR4>b# z&tNv}?H!F6pU233(d*obhuAeT!jjj%i(}QCH*iv{{eH*!NPEZ5InhuqZvM0=H~AxD zcb&t<7~vK>Y3Es2ToF?!wSYSjz;OipLcTg=M+tYq0)1~xanhpkFVsC+j3a6|JD%*y|@>BUzw=N zg>llAoLUj@21(`cHq;f+XcKP%?@c5zg$ZkHO%JkXE*x5Cw}G$vjH^kRJ^HHs^~ z^P?_bqcunF)IrcLH2Rpi_rzfj=Bm=YO*jzS)bwzil_7b18@?IlPnT!Rcs2NO7XDRz z5fA1#(NUyYb7hjjltnNLdL9nhGiD#m+AKKZJ9bmz9Y7M&LOxWx#W4`%lX&?+MT!(H zW@igmbaLBsP7PEsNAX zP?C2kd68t@5O!t_bRu{+b@uG8=7Ro0tCSjO4#%GncpW8fh+x)}xCQ(VGTlu@weD$G z%YqiM@_+8WmF9tn$U|cTK8ii4BlHdnYe-|x=-8G|>LmZY$$0nQ59VuhyTHbopG1jTBxX%!|&BP zCbgVgYV2~Cg#6L7KO3d?;#K}&>1*E+9kkI(pv-g5TLay$+ZlQ0887K(Nj|@~<`E7a zC>WT-+3K9W0Dr*Cw_+21;Hx0hEdM6P$7uAgxXW59Llfvny9WlrfRyxb%Gl?gn6V;+ ze{$Ves8)Oi zPg*`g{7Ly3Ql(C%kNBtrA$SG?VXndb0jq2zc26YvL`c)WTP)=4p82du^;AvRw4UfE zi`?Px#j={!m8WgK6X=S(Q!f)Qza12{u}X65H$$96Ya`W`UNE&mZY#K_%Ub%R%?DNs zOaeZCP5E*s4R~Pkj4w}Dn%jMN-0_{` z^~)KK?$Z|qf}vSf6>FZg41_+4^d&LP=k?$zvkg^26O|(h62(p>*w=Ke7n-JVKliIk z*wLOl;%0UWDI6cPOdt*`EYKB7Zrpu&jLz1v9#4*2RXV9Y8@YN-!@0sI5yg$T@jRxK zkw=g;C(m87G!_&UXfYCcxnPg}|NQSPkRjRWkImz@<;_>gK62IO=gZGg|G4)Trd;|S literal 31544 zcmeFZWmH>Tv;~U0y9C!33dJ3Q6iN$$7AX`f1TAg>ihFPlBj z#>l{|g_Y$;AAxrI_k$M<-*;vo^RCOw#FeuWF6_W7^a8_W2yU@~bGVex6}iX>Yy#&{ zzZ$d+Vjov%KM&mQq2%H}z&BcHh1duf^q61V49P9aZ7FP7E@&uOIW^`m8$mNGtMkWr zGtx^LJ{9Q3|EZsRi91=j1A3A+G(scNMSnTEx55@rG<1-5cn3o9bbd(qJ08ca_sfl* zEtvU=?A7q3GyEl8Cn5UEsfpn48Q^$A8V{!u=W}o#jsJMRfD(vHg*X}cc#r-EQ5%6` z+pN84leHBt6C!=Z$c^cj(e1n>Z0HK ziUk6gVB;5jdszq`*%@C4n)+{yVMoSS4YT0wv@(Q&(?9EfT`z(wvaNDFjW?qfhw9&E z599Ep*mjnVtcMYXsWkj(E5jf8BQGQIh2iw&8)vuba@G0j9Me~%81ZZc%M`>T*Vc^R zpE!mnenN_whZtV;jqmV>{;>bxKs`T;-&(*ppIW&1XWxf_DPUHmOjdfGLm75T&wQPJH))?(axe`s751LRWH`lAx-7~7wG zdG%ePzs93;0A`|MFo%81VN<{Vl+}UuTdmA&?mp4WF8MABd=f8LdHTl_!GafOZF;_j z4DulT(_3J$AoSJf+!4#oNXlJIPVGyfa~?gUn55YOEhizZqp!>#xPTEsf}# zJ?ANfnCER|#{0_`#0xx+J4(>&>0gz=X)0*Dvb^H2Ey2CGOQ(G*FaOU~SceZGv6?UM zY5eS*#3vUyVi7{HOz}m#5;844NdIQ8rlRH%anNA1xW$xKEd7Wi{n&<}v|n@k?}V?s z9g*D0+p0_Gj%ScXGS@>hjoAC&3VX(mW@WU&2UHiGEb}qPprB2OFA0PfKigNgzCK?k zNHdVHT}HF*yj)%MX&UzryEGaT&`{uzVY#oCVNtN+zGoP>J5w>m^7MCM6{3CEHi$i* zn|fnmG1C0^!FDM8^RTM`v9vW*4gInA^9i%)r>^^sgza{UW}kH>)0<5?8oEdZc3bKT zXO)d#$%wu0DU0KW(CSkQp)9|=d)B)|^{OX6-r`%g$o>*Kbh*_#v);kMZDC>a5Un)U zS~@zMz5G{hBe%s5ES60K4g0_2{}A{;1pW_!|3l#aR|o)wxc)VMJezci2b$89wS?4M zwDo*+T@`c3mGrO>&D<`423UaOG|+V}wJdx%6xd@4PZE3E@DX>^IqYX`PE(dMZ`#ri z3-=6-UgK#LA@JV2F|{SX;aHYtPjIC=`v}&dOcHz6LG(%)z|UuigK7PomC}J}H4zq5 zS_v>v5S}^kb>GG$d<}a4K9HAKIM%oZf2E!YO{_M<@9c{AxHn*&oY5^l^rTwzxzU#& zk?A*WVFH>lwgIB&TR3c`$SwUM`z?p2IE~+xt=3~~d$bo?d^RvukJsmeV+DCr4nb05Er!Gc< zIC50SHkMEFtZTZmJ*|8djx61;qO81UbxU%q!s6Z08TZj1h9#^FMV4m!-oD$Fiic>m z{XM2K_EmuCblxu_+k6~Xvh2pa@6bZrfw#=z6(n+21zPS_N{`p`_9wOzd+d;V5PR%Z z=XZtcv-MbLk!d-)qMawpcB>tc3p0r<9ySLTk;*?3X4;7a`aI-B-i{gt3hLop1vPbj zL&==!n^T(g==WWU<}IF8%&Xl$C_taspq%qovFDR! z`|5XTjmkI~t8HcHLVj|fV;8|eE55QQZ^VwbaUkYjQDeOCzHqVuMf#eWoWJW<$fa;rLrFzi0bs^l#s6sj zhIs;f(lgRLBkicm2-j-wtvVawOgMd_GM=e0)**qP9hr_=AX3#uZ!Ji)K)U2L*XyE!jPP-+sN9 ztHsEZ6MxLVPr3duccnI)l%w>Jh@*_Vu*RjJw!>E54e7^p;a;3!HL5EYE1;`e$V-1% z<1t;eXa{#YJ;0YOGAH)7z5RP*D>x!BrjPd#PcaBB zI?Zh1o8qr{oe@Jm-tQhhrdKb=#x~`@uoaaPO-fnDCgrGc@o4R+h7;=JG4w*cs3~-| zl-klfhkv_lL!-!a%K(%#^ySI67P!8?tfqonUmqtyi*nbhz5(S1h+4X_kD*^CJmy(u zuY8X-iKQ*P{bq`92L);n^A-TiEzWd<0!0gC@o4ANk#aKEk2(IZon({`ozPd@#k~q- zKgd5-xmYBh`T4qUO)$iZM@DY*=qUbUF$^L=L1zlRqHc% z{FOfpTWp982W{hOdQUG|9KAU^;;a+hZGEEO7ja;8APqqWjuP9X&}C~hGl_V;QsL{{ ze{W2fS5Q=pC4WO`qOAT>N{Vl$pphZbn*!jG``V<nj88LI)_CzwZ)q+U-xd;c>q3-PNH?E&q*grXgq z=^i_Gf7!3q;0`h4W^mq;rqe68dhTz;=*M(2dFGb=hg+W*uZ>F&RudHd;&1I`cH_x+ zft5y^po1slJ!!FR?eE)vYR8F167CWz39Srj-2k|y!%BA!L^*La#YYx`g zsS7q`=e}i;>XZk1x?S;qO5#z>4RCSaUtsUdRL==$s9}k)50wi&AoNri-I3EPz_EM3 zsGqh+cNp7waLn*&CwIgA&O;vTjqd5+An6FE z(Xgf1h&-D81+p_~=!Cp#4BEWM+Z5MRr4CB_kFb#@ku+$+$pU)yeD7Pe`Yk0=0?E1k zb{h|Y!9W$#{KR`ebJF#gA;e$W4_GYk-T?CwP%pVUPw)S-P`}t83ue{)S$~&m#(GqKXK z^6Gg(plI=Ile6aNJb+)kKZ9|;NLe(Q0 zeV$Y^uUh7fXH)lL4{#yJVy7{xHD-2Im+={puP%F5e|5e;<9vZ9djjr%mK5azF8y`O z$R$$E>>%)zeARZ&rT>@>nl7|3jM^7ojxaega|E1INs@}AnUb2Foq4)H749;;-I@6D zA^0{3Vs8aJ<2;OHW1W~REOJTAFLxpwE8IE-NG^|U;rXO-bP^Xk0|bXr_0)w#FrHvq z@dene5H!C9rZZ-(t6Yok!TEQ{h8OgKO(-&{d}x*YcNZWp_TfHbusG?x`W}DS{bOG`<*%eT%3sG$ zJF7RI_a7DtfI0F=&n&-Bq@DG+VRMBs4o_S*VTCpdf$hHnId$1PF429(FFeB;87{t-*Kf9IrKWei=J5e9jX}@tq z%QiW1AP!mz(>Z4dI3t5)_k^P?!~6ls#GB24iDuA1Z*s5ZW{zj5mD+uzKW5luK>Che z98S+211pyp8&p^@G!(>n#g`g)_WN%b*z;rFFf&*{23LIJ5O2`uaY@XYTUG#df|w+2e_`Kpt|@Eu)^Dc}|nxJ}}c4PD*V>&qA3k zuO+vkQ=$<2=Jr6Y$fchgz#8AcOedS#4B{{S;F;bCCU7;D`0#7h9v4*>{^lRCscBpMKJA>;iz!B7bV>N7KvTLYbnMEAKiC!2h z_DKgP`<1wd;GVFkjHH70Ic_EQJXQzKJboyNZ&yj9S9nkPwjXUf2o<8g;`TSHfsz$g z-WhOBNEhJtr!#WGC)Puwnw3z`-g}u#yhjq4(2e>XvHpGel_oCDNLD7B=o5Z8pA6Yw zuaj+T6}%v0_hDE#bm~qbk74& zyD&qsgIn}HL@vJ(WwKQ*vF&5RAr9x~-fz{l>btIWT}*w7xw?c-T)VL@XbKQCIdpbX z*yir2IB=$~MZ21Do(T20c1G_*c(SK&ieh$R=inB3Pv9CkYfP7V0==kI=Y+Jb)BB(J zFA{*Ch;4V<{KeLJTTC{SR4tAsVG8sRcGYRN?9E4utsr~|i~aIkM}Rd%s&Bd9JUz`0 zac0jti(N%i_e3FZViX2q10|dGQOF?HhIZRI>w}S48M^>D9Ih0nKnlxEbx(%9eV+ft z;jF*%xUbj8uc(i9lQdz8iA^7m8yM!!-8CokeNIW@!@uR1lea-S$t~~f?>|-IN*i%# z3%7uh<(Q=~{OX`ux$BKD%S>nsgDswPr|3}lpu2h=xZ2f$$GYGURtWi#n!e`j`E&x6 zOWibEk5dTz%%fl|=REm)-)TDf~pCViiA1m;H}=)l0ZT#_72fszN)>>d|?an9xHc2O97 zBpD3Imr;)>I-xvyZ#@e z`L5Phvdpgg+wr;D>s2A1`Xz<^$tN1ybWqw!&fGYi4{x0IdF#a>P(sx_&gFcYZpt5Z zE-BWoIm?a*RV|Wv01ol#TTP~k`B4(Toh))P3lB7D0(YtE=ImimVEbV}27uDbF zeH@Rx{#H5(5oCGV<%Eass>HM{QqQNa{YaD6OT3G?+6z%qc#!xPFp0j@r%8=2i5P9I z(8x&Y`2(}~bM;hDz@=KAt54F}-xaEbI>h5PY|jo;)di}$epUFHIm54l@Z{)C;6pV) zY~aAOYCmlK5Hus-E=YPD#3t%4skUZ^vBY^^qtGY)XzptOCpw3Xw9B=&P$;s-upnj)&v)Je{5vqK$L)^Em1*}Km zi59oRorbn?*ZKGL zom{ybKBf(eP8Irk{u{mPvU)L21c9Hs5zJ}WCWo_A3RY>d3si(gkST~iNaLPLM4mV< z08SLDlGBq@5-~W8Vwx_wP7rDFL~Lfc=Z1IhUnT8Ix9dp|M{{%Hj1c=jaQP11VFjl9 z(X?RaKUX|a7jZY^e0LR7ypsG?HI%u4ddjsg&H5CxLAuE*YOEaL5yJ%@3+Z5ayq`C$ zd6B807|ylz^mMmTq2#8=bY5OfAK^6V@s#z8h5!WFr$F(muvaABT!#XU5M^75Z z6~_WQ)%%U~-Vf_*;0>k%(bRhFTSi9tjxyx=S}8fRYNHokjkO)w=(X;a6bzgDzgYe( z==0)D7%O;r2u)arex*_=c0~*WbE34;S-o>LOJsDVwLSBSecg#ZhR*aXQ-s_|MFE6< zpC? zyQzumL!&NXyyGbR30#r&C;1239Nb_^B`or=!NFDEi~EMq3%2_0UKnI5Hb7u;wwY6? zlAuzITVE4zJ|*k81VAmCEW&V0WaGsdWx57&2|Jo z#sN)fkMGeVvaM-Z0&ErvAI5a|+k{so{!B`p1+`Mv$vo;&gxKw2q2B%)92Fe&eNhpT z2ztvxC7lyo;1iD=rz+`ZX?lO(!6~u%@7lfHC;SNq;F%-g>`DtiE zoHn#7QsDDH9Mg!Ro=hhATzz4WNha*DPOHSE+N+DyC0&T6i4)B<&OaS`0KJoR@cis9 zCN&Z#pTt2mvbd28WIJb?5|M~YG+vdIe)gtlrg(@`UONQWlV)TO1+WdYPZR!eqE&wE zo2Cj8dN?O26dX4%5-;@rw@u07&DS6gyo3?M6Lq#u+|>7#20yX}n4Ksx-Mq|+2wgh( z`wPFx%1DUWXA}*1I;?wl0i?z3@206-+hdAO+=~VJC0J7E*u^lXIc0yu+TLD&noORf zO~3EKh9uPfY-zjsR+{@)6w@1K7kMe*G;c{r4hBV}r1zJG!`dvI9sGl-;> zV@*0br+-qZ6iSxF*}#ZM@M}F6V|~*aAcS!H^|qO_Lq63qEekGeG(=z8o+Hvd z|C@O3LVWq-3LU(PC{8fWj$6sp52ZPo;taMMW5PY3TA@ub<%Ri~3S&jV_m+X)Ln0ak zy z-*|^}(S{?ajx`l;X#tbEo0?bpl)8V#Z8HEMn-&accK5G!m(MG~k*7O@pU zQnLXZiC2;O^5DytB7mkpx`{2K-mXld=mD|2?&s(n(=!TdaMDUt%~^7&ONX`)U3eLl z<8twuZ=+PgJ!P=s(HT>xo0*wGDBhPhwgD98S*7AA!iF)Hi1lT)G4KI+&KbLsIB9&1 z8ob2Z8V`bCpLQjKJ3wnmB z_P`rW=YT4UM{ncsuJF_{nZw^3F7Q2@ioC7+PE=UGJ7wYtDYAy^nqZ&KWg}-jZJkAJ za3^ALU%L#~I3jrt=0S+3{TR_z6z_a>Yok%k4~u{O2KnltofL%yQYWM6YyMhhe75dM zA^&FE6}`aKs4LCqf3V7`+h=wd(v1aixKrh@$5@$D0^(q!41_k=Z%72}MAE^a{o&Ot zPn1z;%wd$Q_C_;sXbL;`VEsxKkP!Sqc~~cTrhl|~d#gvBHkvILj4jZg$8?1AgUEtn zTz1;SE3Z6w_sxEK7eUFW;-nPVh9Ebigyb$^3ajdvMMd<=dO;&r`d~5en`F|&t7n`q zah7VeHvesQEH9C{(@S}d&e=U$4J;8{AhX*qb>xu38|b{w>d%oA_CtgWN_R|2EsPZz zNDntBH&d(zD+*V=&b+qzaz_)&TIac7e{^WVYZ-YME*7HZ7l>ps7#uk1ziaVaZExkEB$*ZH#7l}D{U1u8xTrEZ zPt??qyrU@xaouy!1l#Nf==L!M$y%a7A8ws5dp;MKH06Z=L!0v_Dmdl zsg=)v8Phn@3$4F5$Fl%$Vr*Zdx6kk{M0amd72)!SLhki z8%%Y`R7ai9-+yiA^RbQfwcZ?eDz82tg(>yFBrVGC?Y}l{5AmVGb1f(3xvp(HM<)_j zFJ^R8PyA7;w?2--wu+@t>3X>k3Wbd`E_u%BvompGkm~pHkls?NI@*;E$9?2s^l$lv zCVhMPC-PMOE@v&)2;wz7L>}!I*v3^q%y8?t#OibY%`+s)G(0hV+)*fHya+67M3Qx$ z*W^L7G=Z;}19^f&uZLCB9U7Z{i&P%h2o4Tau=pmMlD)Uy%tv98%+c^vpfS71EkWKI zRFMwb4804hV&j4*(Y^;ih*RkUuc2#m-&2`$sa$OUZ|2c*fTO|X z)wnwdkEaa_*o8A*D;XGDV8OqlDTM*>XU0{`_euo`akDx{4yLG?do-QNtOp${JG!d` z%l&r8$VLVYgXajR(P%{?E_6#i7d*D_E*8vm5VEb0IM>=Qj8VOha1>P?698XdOVG4l zcP&qm6OBdds4QZia3TGn&Q$aCRBVb3aY!;Kgh&z~zSW~xO|7ZECxOj_cG-6^EI4vM zru+T12p*CXyujJ)dts^_t&QV(n@{GsqUXLU36+_bK*l3&wu!b$ z5sgA=Vhdx6cq?=&7{1??#T;C2toEXB@wa&H9{Ut$d=?o@28(VBlqvL#IxzG^<8&8O zrJN|Qvsk_9=o?#yE28|6^~f5e0&*JZio~UIRYH(G9w58b_NCY={wpg3(#%uz&nu@| z(9u-fBtp#g@xR`us;To!BU zQ^WH+p@_a}Q#?doa()o(BPZZ`1Ejp(-?9+!E$8pw4-5b`Tolg~h3H)Mb)=?mgrc2L z%Q82f1hmD>R7q=A$^!ipwVlo_;8}L0fxakz5D&VQ3zX&5M&QQ}w!0O$_I)(AfN--h zGRjbjnSU0a0OTb)?$2Iv2wHTj+Ul)8SZk5pTD_!(dLvUfM|&w_9P?r=GZUYn(^CTW zU+*8%hkj<|{3h{^yXP(iPPM97fU$Z5#f&O$D?tz1m{+f!(yr#UvJZYfk&RW|uavv& zV0w$?nXD&$9|@0kYy#dt?_;^EWkajGsLVuxIpGiD@ZCk=HZK%{@1rN4dClP#Dk;gj zOSVYisr&n)&wN`9N*NU}yjWbGs)P5MCiRS)uYf zSPkl=_{lI&XTwq8o(pC38Vw@tX$dlOt>Bc5U#Hb$v3NhCZBt{>wOK=f+qvjLi zVZI4>P~9J^Z`7*AUO+w75Me7C;j~maJo0p8+zx)ukK;t_@8n$e+ZT9D2$f+{zqG}w!Z&$}I8ig}+78@{hfu28(wfr37nyO6u|U`MjuD(IKE{=2Rwy$BBp zn=OX%3{Jn!?|wf!Cr_ndeppC5URRg%Dw0sZ=0ZQd zRUo45T8~4v{Rg9rK7S`=Fk=q1FVtkkf$0Gr=1jyyA)X;abz*3}>iVB43s}QPo~;^h z2SL+cpq%pzJ4jve=O8RJTfoYz>^#Gqliy=6I;Kvncu_I(IXY2!fRqT4uuWj!2kz5^ z!gs79()b{MtUr`D0Sa6}bB`Rn^tQdAYq3h9|7`CdP;>z{{v1Ezmeq%Ia{KmK8~dv4 zhewhWgHy?UF`=;^6cqk3iLq z*B5a*M-E?bvhO)Vlb2Y^nEE;bbfIg*Y7bqsy-&4wl+Ti;*~C!A&Om@n32>j|%x+zc ze^1?3OT<1%&5$2whqyu4>X{S{k%tZX1u%r))Q4&P=ab7Zin8y01AvSlu(b^{Uc;Bd zQ*uGEYzB7D{v$A<5Wd#&TL=xbnQ)VXdF=qP7H_%Ko475(6%?lWV;wjUqSiLa$=Uu zHzcd;S?TMBeV$-J!?J;sgISqIfQFVfBh)pQVva+IH#hqL@4d0fe&Wr|hb`|Kv69&t zV80Mqctw^>>(X0w0h)p{)3M!ViC4LUeWzk68!TZI>4x@sbw0xk3$p5V%UX8MKJW=^ zRr=qE=wNX^+VDIgL_iv?wCiLJaSxan6wgYYoamE|>iZf0;E{`>edqy_+7LVE@AS$? z&xpLx7D-g4GEKKlRVf9*6DxaR{+UyD&c08JHc{Gy0; zG%%({#rSEYFG-M_K~T(R`fXX3CQ>}(R{H;bAyjMISi)-Vk787)$yhU*%;o$|cCGc68SPHdh`kViHm3;Tu5qZ7con?&(XEviw3oi0R+Ln>>(uRBS zr2@O{(G`zoG%GB)skm@hc({FFlZOAHDEJ1adZx*IAzQcASZ!nEK^am$!4Tcdrtmo0 z+=kFvZHd7F+{0gkr>J;4D~F<|ZJ>H73aYqV(;I3Bw1SAM$!d-Uu5*M8@bQ`1UfqJ24gwvu#TgQQM0yB7^?q5!Y^qoPNFkeE@l*~gQ4^;|d;)42 z2I0+fca!_;x!};>pi#`yT2mTD%-F3e`H`>xGb;oKzJY?sQI!o8NMocul94J@eaAr% zF^p;yyzQ!~75{y}m8|u*08ENr<6LVVISEGz(M^`tC%Ue~9GBI6_5I2SZS!K;AGGZ? zxl{9cDfvfoAs$sLOauR!6zZ6d#F?8G!SyL`DZdzGct?qHC6|(YR;&9>gJgPT(vi{O z*Q#P)IMneO^KTP5O@*$$!ov~2|`>}GL zw?h6fr}VeTjiq(I>N#8ctC$XUsHgZX!kVV$69mV>Chj54G!ZTg?okM2o|~uaW*;o}0pc>MHHhY7=k+_1&jvpGr#( z`(LAx!*Dl$&pJ&=C`k39=wz38-CL)C{6u!2Cqnr1iAcKV0w96bBiR8^xp-!mLt3m} zR983u3YW6!C^BapSFS6$&MA?o;0^P- zXBUSpICbDAMSW-Uluduj+!1@7JrXn4$u+HQmuJ;U(6S(bA<=N~&0>pXhJ#dn0%~o72YHFH23Pq@mk`j z>mRaw-kP~*@^QpJW5|CSUT^em1V?FCV2Jfv)zR->qlD+1pSPT&0(47`Y@T_ZBnNl= zp{cwh;Vyg|FROx3?fY=4wy%-o`5F_W%?Th{#LtetFNM}iY~#A_@A*cRs>%SXJr}xf zb>Jt#&~z0DNiOjGij8bs4kpr*ER2ZZJad8O?hv2QL27sQ0h#$Ch%H3SqMOB9JVdHqLGYINdoIi zL*yRc4=jgmsW7kl*ONC0Xq56XN1Il!u`P^YSVpbaP|0Q80oVfh4dqC(GNlIWN!GW- zKM8j!7KsK<^?+cFyKMNT5_@F2!*WiTWw>$#JPIBGiIqK?CBMNU5l68-U<6K@^A13D zxvR0FqZ6p=+|6)KnD%AHB4+zv?WdS4urSN0n&%=r!kQWWeTeG+zP4&wrqTmu1~2!( z54IeJwGd#@C|OYLvN9tJ5qWq>Oyb5*id7UYCn}@uO+beWZ9gvD&!~)15?dbkCxKVe zfpus{o7M3nwE}kzD4^WXxiB-CZ#MV^IP@OS@Y8BfE9ywqS$$8P5_kl(p8g1nDx?hA z{FC%%-C^^OH`SOgCRW$(6_*DU%V2V%{DhOF0AJgj&h-FDPGz_xQKrNhoZ5BzK_v)} zOX*X;+8forpIoQ7JwKP#_f1^ZbbnLw3^{XxUbwFP9&G>%ujU`9^~{AQ#_HsM&GUjl zOA@(MZue7lwgwJV!gSc+;X5IzVoN>Cs#V=*cCmck|M3le;MKKQ;Un6+ChernxQiSI zBiqrhvO?Wa#Y*F42wJxt?PvQVq~5M`R`vH%a4v2MPJ_{oMfhIoQ#&#_rMCj0I5 zxw7xkL;DNJJ!uZ-^=&;=f6EuDng99}Z8V$9$inkm#pvPAMRY$%zg7;a7`CcBT5rpt z)28Cb2awbV^_wK7Mv$2r|HFeah;a$3~@6FdrZOtJfW$w0-{qqy0Wx@gIp z?mpXSdr&WgvL2QD42yDo?jTx>lhWm^7Xv>?#M`njAt#%u72;g_1LIxvZ;bW5g+U!! z@4R!&uC*^WlXscVy0ySsp1R`jeM$j`Ew%H8lelB|V)Mi*%SehS_DHwr(Xd1ed0IxX zY>)PMdaCPr+Z}}stK6%0oVp}h-FcmdioL*ip+n22IRo~U-tTMMb@D^|XIA@9Gp2XI zq?@CFQla(oN1Gm%%8*`tTa8gk84(`!tT<;nEz}U@^nB&rdP?LrekaW?&N$PlWgzY< z=iu0BBr+TMSYLjDh<-GZA84gU%84Qc)D}i)stu`@yHJrFh0Vg9iw~(>t+v^qTRQ#A z91pIb%Q+{ivD_^mg`U7S4CFYX#o7Jp^;Mf7^;>O*BI-r_M=sz-UR zS7H*wi6~Fk_}?whj_UyuQ_m>Cc_}J*i`x$8YR#=24)@)SBnQmfCQwQ)S2QC{$>`Ib zfP%hR$h;m>YGAfb1@N-j0x9o^zwW;OrX8nPDN_usw9TQ9^Chy0!`AHOjPm-kAzxvq zS^3)|5kc|2?|W1N>I;?{HlYgLaqdYz#r+@EQesCRJyV&Jz9~&r#1(OL6jH+!Z*eX# zaKhMoQf1y%L-?etC}*3=PeYEjCY(;ZfNb_7v6|{_rkByi5wqh+ky!TV?JiqFW?P9g zIm%rEU-F87Tb^l`8o+Bhp(CfBQZpyo{P)~Mfo7i1o^e9M6p%0q0enW)!(T&nh7fU* zOW5PT(f)+n6lIjQ3)s)~^Bj@R7`gSg9JYz!!|T7EY=!BPN8AzRlV@w=f6;ZpdktA& z2j_BZ&VXVBB<`V57Vhd# z_ysdhW!p}%8u{mK84C+eQZ$1qeB4xm#Oo}ed)uQjT(;A z4@EQ{rM}vRMIC&gTQ!;gbHM?5b=?BID(o5>^N}8IB@ofs)N#R}T?*VRw4OED#nF<-n_9Jrfi5A#? zeGv%)tYx3rpAiRqjAMXHvU9%rt~LN@Ds7tgT#@ub8%BL}@|g>5DKjOfI=9ICCu;H4 z9f#wDGqv;*(=gIB=u^fIJgxA33_)b6H5$4&pA%AU=AJ_g+OY1KsZi;L&;YmN0S=QdJ#Jb^q)wV*+1(W2jv z5eif+wi#*l!uJnLBxzS6_T^EEi^2U%7&b>%(DrBdv@e#k<{dSiYM|fbH{@GgNXQB!kwZz@mEm(!vA&OSvJkHR z?o@*WH^_Ig>6%rVTaKPM5;pO`8@1G+)_>Yjmul`$zv45T`+RP#SD=Du8uG*0AM1KO zhNyo~fu zD544H6W5m~L$6v&Jh7LlztFz3Lu4(6=C&CYn`-%Je%Ddqfh^B|^hJ6A()V~#Y0ki) zFtu^(6P$CRb1X^1x{-$LfX`QTL*1(`m%M_7VC`Ahu$VY=9avFvi)QDk z%WL}H^i^k>$l??Ic3&LxTW{+#`JN^gul$B%e(CgA8o5mFDD&aPL-$!?f$k<%bHGW^ zbj)`1J1tof5@+0amZW?~`Ip}+i@c~2VOT0!0;;No0?OF`CPrk67>k!xTV@SiCOLMo zT!|J5TOg9+@^aZ6oJt&D&Y~{I-8rWf2<5SNNj;b0Fp#uy!r*>ja#VHHVfY_Y`p1cp z`*9RaTv{lrI!7w#*X*Ef?l)+zkr?;TrY*xjwIHD|UsR;a(D#wQ zOD>;lU_EZ>tTO$M113g^uy4q<;vv32JMc8O;IlqGAj1^yJUPqy(BFT7To@W#MNjBv zZ$}f#i)__Joge3k2qc&EL)w`D!L&pjaP6)N}@?=jsIl0(ABl`@#;iX1FKZ}2boUf8k;~rAeOwliE1nOeV+k}=VkH41C#on+Iale z8?eTZibFOk_sm0ZX)T*2-$kN$iHeP5XZZX{l>c~9VyqUX?8u&$$OR1a__)yeplb~VC5U2rOjwAXY_z}S_Y zCp|i)QTx=2Hvh<>)&iHal~&UCnn2S}rRxFV!RTL1lD6`aEc1fTzEY?&A9Y%A1^y$M zw-pn=j80tpJz4&29kT6}z7o49x5EdL7rfR5+Lv4&7uG&u)R>zcEei^-$IfTq$yekP zD#Q|a=EESM2Flpz>ky}o@)TNHif~wo?2)4hK8387mz+Z9Ca)7uo^H2s=YQTU*EIxHJz@F4BT2ma`w_xY zt0Y?(cH3mkw^Y{W58kw~vf-)MUllSgM|lemT18$(`5Qptw6^f;2WJ8Kh!=Q&R!6Cz z=>gKXUoRcW@*}%BcF4&P8M-3>vY5%o3chAK(m}UO6nSqSaU{75TkiBn;^e;KzYIny zfBGzkokU%3>Xi6=B=&oxf#FUv>FY^Sp;cTamMs!9esEXl2cAH)PAH~Wtb@ye zeUMBt8=DJ#w^sy1KJB~(tJuyRhA0aDGoxxt=~K72=A;T1d_L^uyY!V>XJDAz5BV53;u-q7jdhd|gM zXRviyV5$sb%X)x53G1ldtT>6QLrM&vq;4%WdzU<^$aP5)=2Pg%dgeOo^s2-iJBeVW3>HwA$JQELzS^W|J>v`S}MvuO>P>PGQLLh?p8?2C8 zY;D>a>L-S8>`%9f`SGwXEO;-t^nb*k-NX)!^|_l%H&sz!Hn$b1@>lLtmd zza4-2l#p_ZG_o0Yxo+bf+#rkX0soaYNzJHwF+K%u8aNUVCtmH`Y|_R$I4zrEMFP|7 zpd&Hkfw?WzXR{V0y>LFVVX9&=9>Qv<9}Ui&Aq)}wln7qj!-cl| z@sGax<28nicT`Yv#F^{KF4SO z?-~r>1u*@%R}+cIWrKi!UJNP)>vk!SV_`{G|MxC{%4J{yB_rS7l@sl`);fvG3A37r zZQ{x1N|Lq$kxgu^#3aFBpXQfS$~yuXPDMuXd`gwPHnBeS%_%;(1GPpVsW=<1fMHwO zaHDlIpCtA?oj5!%PLM8KocGt*8JKFhu9>$e7y#TaEx9;ZL>T!c5RyK3aB|ek&rF;i6 zrd%ym2XjeSt2bny)IVPf_E+u^b5IgAP{5}zy~!w4Yo@g2kG|FMRvUv!A|B|-ch;um zbIM_S97Yx)6LRakL`}1R#P>SBVRW+PVqfImEq#9ACElLi>>7R8sXN|RF*B}6E~+HT z8-4-s940$vcjhK{KRh+qOjKngEVzFW`AX;4Z0z6chjzC2qc6+X18C|$#IBoXE=Nzd zt*8$8wy$lgtXWY;lA{)ukvC#1tTR&PZ*CX#?xD!Ls=vEiXC`x1k^Eoo$e@}b-CTi~ z?z_CL0qJq}{~{q-LwKf<((liMg0fIr?^NdT(aMQRO@8U~$xq5!B3Z}f8iEsPwn{IhN@Fl+hleAPMdSceu@S*LvSSalg!(S*yFM>zsXdovzxark|!381`>>%o_6J#tQ9qcc(YcdO*4duyyt3||)!kZ{T^?s=S*k9Oyj_1Z|3G*u9;`x2?Be&yOv^8^N}czDi0YBT z{Oaw2pCDsHwCk+_RSkJ}7EC=|^r(<>L%)0+omeXPiW17YVMYn z768cA4C&Apw!fFD^Z7;Qqp|D16o7iF-v*%Q({+^!1QIj&!T$3itZi#EdOw$gS<3KN z>qR(=wAT(xZ%lW1I(n7NAI}rS2EtnO9-JM-aVOXn^&GZTADwma`^s#>F+O2j!lt6; zmp^dog82i4W%$%AV}bqqd;kAcIc(jeveGVLnJU~;^bBrQjr)M3GrOe*Qy#-4Z}4HX zqi@bWc$}#U+naWtva8Uu-Y-U+t?XLlljL7zz#njL8QtETGij@&MFwWP`rNr6*I+2w zUd@vHqTin!CPJl_c`KcsH`OQ85lCT4)7gl+ZFTc9EP+`crwDm1LG&m&p@z#WV~Ses zTa<;!mb0|P3$MOa1c?r^W5Jje9IOyQ>`8PW=3h4~)IWl6#swK+THn@r?=tMD>My(o zJ`0mMf7nH}jBS%*6=&FaJpgk_y*N8 z)H?FaR`y-9t}#kGc5LZ}tI7spZ{5Lq!E-OtWbB>D-PmiHdtSCU9EgNUc}r2sEImbx zdx4`j&8ROQj%CWpL!%65r4p)^-NAPb!X7PB4^%O4-HOJgdg_Kh#F03G9cshFs+`(h z(sbC*iWg;Wr0?r>9r9MWYV3-J?MwL@3$8emUtsm@KTw(fZZE%it^6)pSO1CRlqBTd z|9yn>=@q3l-!z7)RX6EB#2q`PBxg_qtu|H}Ru%pRO1gVMg*i>2gsVm<`M?PnUWqHK zeoPd1qC&Iyd!G3L!?dtZts1Aty=v;?wpCO{q!pgdz$Nmg!TWFI7R|!;R?Whf;)G7i zNcBCtXS`!$sw}*L6-L{tBDmn=@Ky<# z6F3WP^#Aj9Xt2Z~-EA0M0eI8Aw0|R?XL-pNF&8j@Y>!;kk&c||i!x!^C;okC+EGEy zR>&H^2%|^bLs%Opzs5x4FSrP+mZHb=VQ!pJ11oXpF;qIj=hYmTRYsJ5`vXQcmq9>) z;hdCK)uBd_$TTU9L1s6BHFx5ui_u-m!^HA6NY<%M3 zNO8u~z<@~~2{iYY7%piBv!lX3C_fB{`s>IrxtmV3wDFD!+qv_T;yQk;)bkSG`K3e+ z+rl+xu<9%FRI9s(Y3t61BZ3IEj8OLvB|uiw;Qy{AR3wU)YkK zZIt2|`3A)E8V5N{$9MVJFZ(t$NjxUM$BkI4q6Ys2**v+ z;~;!hB4Wk*gC}?gYB`R^1zP)a&+mqE?y<89HhJkwVT-fi&WTrZ5Jhm}mmL+;sX0z( z&9|`_Jy1!GpOAr`5!a5Y-#fNF^`k#K_ra|S5jzDEJW)p)mp#^?EN6qu#D#!k!Wuru zMj_!o>1>1nDVnYKj>c^=at1z}r*vrIG9_C}%zWCeuCwPLjRv^~E`3FaeeRTb zaR&6d7@Wm(_I}`s`sxtNoN(q^cG9P$`qe21hb^lc7a#>d28GXGx_sPt=!S9Ac;vmN zb+IaBm!rah29XilpeET>wV*Q1xdXe}WJH(PuRIhhA6|r_C@XKs^q<&PIHQ;iwJIsE4J6`NeHhj1Bgb$9t8B5 z)(Chv+%{?;Bcq-GiitV^jYr6<{z5F5ZW_^HT==JhAS6WoS zLv~pxgUOrF6E20H!IRJmil>Fa)=zBySd7Pvr^RvmQaXQstC!t^ zBjE7fA*%Vy5+Ccg;Z>?RDE0QsUF`y&=&7~2pG+Tw@Zu0@vvfBt4^+%#ximdgK2I$v z0aNa^_3iKDKus21Bo}%r6#4QEr8@HVEoOYEHMns%wcD|W6CvYc!|YjW@?yojr6RXq zkLyVhwEKjzw2!a8`^

    (6s9^dUuvIo7YXtTGd)E3_ElTHkN&(A?!^+cv;1AvQNH* zQ$|~h(X(2bg58E;af&ft^!;8nrX6)I_}Uu}au;~_2S=G?ABy_4-bEIM9YwVt@Voq( zE;P-Y(sudpRpU0PE(h%qV<+}XFGCP~)-P23_?o)I(B3zW9oo(8HSPpqsQ)RjioJHC z!QGCfkt+HpQQecoT90gy#0o-1>aF@t@n4RWXMxZ;EMAG8yjIq8U9K7NGims^uj3gHzjma`05)Xt)HNBTcN!fjH-Il}sw676N4Ftod zM%#*KvyU&L4L~OrB@l%DfU(R<17@-|v=QRWan6_?;sb1`h^W~6d?6J`2p0%5M*Bs@ zX9v+J09T%Nn7bG!KD~K|!Al3U83NN*{m*$+tk1C@tehkVc#^6bXocxmd9FVFhsXX| zzkO@1?;z{uG(%lJ1eH#-LW?c_w!b$TaedH(BZl3Wr=sd)){mOR#jQi+75^(H8W0Uw z2qe(vb$=Qz_r~wUH2i{Q?%EPd?SAilUDM^YQGeTNDJqFsmKcP{&AZ#wxYSO=Q~MNe za1`!c;a!--N!k4HKJw6g^{R{6Xlr&P(`K7cEHlhQ#Y1Ji^Xq9wq-mt>HpkEQgN`^% zVPM`GGtq&i0A3eI#caFnro~5d;=E^*`XYX*+w4R4-OG5!^(uYkN`CC@hpKh#<5+EG zWHR03foByuG2FDH{-mV5=`Pl%u(37N_+mmY$_}OZrdO$de}6Gg}Ay z`JuDsqWA1{AIXTx;U0F?jQdzoqpIkMK#YcALW79lLOjtUumc$0Ii+q72JCtV<<93UyC0}_&sNGJ$oNCd$m(BQi<>x=KT6* zlBHT-VioKBPLpH%_V(;*Y+bjphx_>t>b!N}iOr`6FL}+fp8na+j?Cw-rUI0+%xLuC zm|+`>=SGwSCBzqz*6R1722wRck%y;`f1{qr>>taKD!u$9}$k z&HmLmX-CiZ>E5)>!l9iG=h*q9w1TXu!bJt;p?{>bHY0*udYT>3_*c48tce%ozs4q& zmRbJ_*r9u2)t&vnxCT3eNK0oWW^3%S6f=6KEBdVtTVdVqXX1Bqd!knN=ay=tk%Y}< zYApx_`Pk->wM$L2g}FXiWClGy{nDtt02hH#kcnC#UJ_`+;604@GJQI-=9JcCd?b6IK_zt< z){n~XQ<#t9&Q6T#Z=W9-&pH@83fic$Ek)+UGt*wwBA|U#@YuXm5x2+)qJO60lfT2B znvgAF(^PO+BVwS$@NY(4$m_)L6SOA=1j1wa6;W%nbBuEl{%`AnnbmjTMii9IIY_h2J^CO$P@W+R>q4zHMd`C?Ok~XiXKdapnGvWwd?6_R zw#WpcX8>&C%;BF-)cdzIH&F;0Cd9>$tajBAz7!zZ?(m7!#Dg5?5V;KJM8aU;f3?98 zim)VMu2(NkN(pZ3!jdIJ@t#GuIbgtb|59#{bb%E31!!5goNRx>^J0FD9+Kdf3LF`y zx?WRqOSRo?(L^8ZK=S$ghrg)Ez7}g0?iwxCqoTP0Yu3aF>`Bk_K6@;!^qi{aW>xt* zjdWFNTgH|8@p^HUN?1pb`2ofJ7-dj?-+!JP+;KUHv&JtWt>q&Y7T2$8B^rxJ)$T04 zjp+wV#7=X9v_b#9w&o`=|FqrJK#d*h9(*g~9Cq&rh-kh`_wh?@xP^6yb*-QAo?^R+ zoWCZp6v_R=RD{ygV3`@*=m+mwoFhrP-ct25v0rTgk6p1aEn8)a?O}$x`wx^6@loC) z{Xl5y^I1KxWMO}vmmFvep-UU(UBcJNHxmUK*W#Rkk$ynte{ZBJrW_hHb-c ziSJGeD2~*i0=a#(uhL1LXPwG;6#u?M9Lq?_B(4jSX^2=J@|d#D(Chzn%aj4n}-ChPA3ETW!<0P7s8jbWF=)lkE?y4_WEX6oPIZ=>NkWQ9zlW&|w zAu)q3FM}jTe6-S0pKygqn{i#(cMN@H^A)nO(uY{7M|vNr$J@uHcOWK*;MaFpG}F%D zE6VyE349BFh5DH<#z=L?2Giceiv-mUSakCo!mgL0meNvv>-R)!ri1*xz1CtP-4SE* zq6ea-wh=(;iG&-*xX0CQNOE^HhqJuajs$Z z-Lu&~f)WbGBBQ--TPziR(cMJcjm}?JjG!mR6GBT$ZGTpyYEvPsH>L>f2Nqaui*Uri z9mujolM_y`Ue0;Zn&H(p8Gz&JiQovms@0P-%cOM<{lN=@)*`}uD?x_9*S?^oOS~P& zgug+$G~`>2pGz40v;x!!JgSao&AhnIkvD`n^em%He%+wRhK{ag!_H*gqnreyg6qv_ z)aCoT4Kzwz7hZ4T1e3)&m58D?#YD#KMK6%!^2QYRsmKArGl|a?6Y1oedZ6R@+3>)Z z@z~}A7nJGBlii_(`dfwb)PZ_7kXs?3jXR%Yovik)oyaES(vcHcIYM+2ROnr3sJ7+Q z=4&M;TZKP|V+5@~?fZVo7TGgGan^J? zu#)BQzpn4KiCY(oAdhOzpaN3FP(ZTajN-;DTNrRRi$IP!C%JpTS=8xXz*~QD?Ur=! zB3?C|oDoRQp8RX{dQ0KS-^+u#A|dICaLvy@#-Ad@mKwfAeIj zh;Zm25eO>W^RP4z~vRXpx{-iuei758?QS+oedbk?BR@_b#?YZ9?Na0`}IVnULI!XlJiTfO_7IQP+L8TVel$A@Z_DbFVR$7Z1uE(gA zYqrA_WkF!#;S6Nt%UPhI#UnO$wXA7zM5)larNH z1$K74GXeDD^=D&o4iN3%huC$zz#$;Git5j!Mcv`qIRpCyLVRb~wIW@zl9l0D$-r&Helng=IXdD1*cgIo`{v-q7NAJLx42ezK)%$4(JT1yI{9cp=r`0Ja{2&LcF_-m}>M3{hQs4 z`R?jOeo}M@SF_Z96q`&|c?n-p zokHK{q2?}a*X13`zT0u;ed$gJqzUHI0H#!!Hapas2O*EKMPd<>;i%fM^{%*$pN|dF z4N~4RL{e_njqYSTu82WMb9hTtwG}v2W__yLFC#=sR!mKgSn2-ZGk#C*??Yr^C z7Fq&RK9=vC+O-CK%mSP&$4Nf5cXhe=e%Zjbrm20&hga?JEkonke(C;XL?iP{q3H%c zHn%uwhkT5i>l5Bl_l?|7%w@{LlJMi!bKEHXOD4(n6<$HtaEm_n2V-$Vkvi5o8`wCi zFzyA3O5sQwM8d2It3MQHLgq5MF5-Q0ZW*<$`-5F~1{mxnTpr~k93WqU+g!v zHnSF_Gm!r{Y%!5oRiPn_iGJIxGK8i4Ua(_pTMy2EG**&W@+OsBAU@1WXQJf2Ne3Ul z2W|&70l9(Ru+#VV<%7Q-nDI{IM|WyaTX13(9dk$ zJ=&~H>fO~(c#V>6|1JwqjftEzS*mt7?%{4~+234E&itmY@g3>5yxq=zxDH2UpmCGHAaw#Yz`7?CdZK0w+Qc$;*XehuS*`*#CKa!LZ8III%EctT^mhg*k*BX57e@ zdOw7^T0W1PBeTQ0!@FZ+&DT@-J-*6nwS$Gn!QrippV7ZB2amc>)8C9_E#V3)f&yEf z`DwKk9gAvJWphGw^qfK*@79>Ve44&%1qdy_{gtQKzM|QW$r7(Nx=UJlpH{tV zOvn5L3(Xm4k$G1&y93*cXKLTN81ijumS}$v;Y79243?;4o(Zl9TIVQUKmHVK@KS{1 znlHQ!GQQD=j5m$nVWTU#{&qm>!Ev5g?|H%EwQyZ3Um{TlIm`FdAz4jt+N%BQc6 zwLfvaTejaJrY#4csT)$v`!VYJCW)0nVMmILY7{sr z^U48=p*l*!sWvfrxsMM|_9<-L|#*VRH4oTc{ z#IIJ^_CKsB8di?WGnmF7^Zv+4BU;4K(aAdY4A=W|5oNheS%oRYF|;25B>%>9GG&8^ zP;?K&I51JVAuQHkJY*&0$o~=hfpl$7E;DpdrhYlHhfM_Wq#}dw4$0TYKPDcYZ3X-O znljsXS2)`%oKz z3@V?p6)^8W9|^hUw7WsZe)9RtIGzj6P`*!3%C9?FOyBS{&R03zT1y@K(CyK(&q~C9 zYJ6JK`t#r9Z<+?S2pd700=mTpA=U^70f+AZJ1LSr6238hn=55=_02oD=*shkOu)5J z58L2!&x^KQ=91|)Hh*(G{Z-wsvpsHiF?f0N8KKmjlwWMcuMlDW0Dk6GQBHmo{2|`s zn}_DrL?_y*29=n|^QZ`;+|qXb$u`ww&R5f83%T=Pw^nO#XY7j%s0T`Do~;AxXTyKz zLd8`MtaL@%mk8*wI7M6to+v^$%kc$fLG_i?)+eiWm7(~si&C)@tk@^LT^JJ_5Kr)? zch=VMsLQ3RfjN4-^=~3LXg{;(+>^(RBwM@*pRk!BS-Z%-ggzpj9A<<9B}AWEV!cVQ z^cmxc2i+AGMV-t=XV5yK&J06ir5mn?&D}6Lj7)FySZy+NqmanY`ZV$LA~Vh#jf!%G z9~PPivfq*1rJfYC)WS;fZd)PIY3!YPx2ngK&Bz_yb(BZy$6W;`er!JL)%L$$esuzm zVzg=#*N&Mz zq6&0KFAW?Q z%a6V{?Lsa+kJ^{{;G0`byY`EYb0c$&Hpg1dz&kbpklZLD6kQNu(?6J3>OX{aQB@2gqG)%0G!XghmV7uU98^6%&z0Ys> zP;wBv>+MG^$n6_NdweFO@Gj+t447z1XY(iCY9vR!c}N(n;w?72^kSg#3k)b9-W%u` z&_3^`#XVKUgkPsIx67CW_;VZF-R{_eU{nl3iBv+ZLW;vr+N) zehz!{pTzrAz5Slg3JU_qtkXQiAHu6L=s)L!5tX~Z0VS>e;tH~Kz%IOsg?{@7G zSTxw*9IcC)G#x~$A&4jCbVaE}YWMMLq!N?krV_(1w2%pTh1?mw{Lb#$I|@d3HyK;S zByr+*k#5)Qs-OD%i7jC)sQ9M_5=#h>;?OrGetOW;FzecTl{1DR@_o5nWiRD+Vfqc6 zYr~zLMBvcp`F-ajka=_kX!dD}*8wtpI;ylhKYNrT(uMaah#JowUOCYTcdmsyyju5o)a^NNO}R6k&rFs3K*q z#j6#YK7N9?JP`|(+`ag7b_tQd#G-Q~kW^CCVByO5%Pa00lgDDI!V0I&$8to^WN_ab1(80#^zYgA+%=#yK?Y$PipUzkIM;qG>8J2wzenlQ#L0|vY zlKf+u`I&!T0NS7X3%=goKK<{qsKw;#rZG!79bC&%9DMauoHzvDL4?X&a)OAy?0s4F z{IbXXg?=0n1R@d}N2@6fu%WF9s@9LiAaGUveyH$2TJ5Th-B@&w<57q7)h?G$zm{ybgljCkw&=tus zo@sKC+UUoBbR2m{;w(XZ#MtNKxdfMrrfhX_!fEoQK5|4tC3e=FyYfG7ZN5zR!r#g9 z@024tvAUTIAo}l-(A{W1Pbdm8$Xqdn+2Vdxg z$1`w=GVW3gdroTMqA$JfM4U3COAalWSB7W03vJ8mv9wO|si;0{#v1cEfd>fG-+NA`MxG;>Zf#q=1N<64O&JsloeSQot8MXKH?_# z`~a#WP;9y-r;IKlQ=h5UWND*ALgLt)iVh&U*kyggj^W1qd7@l;w2Wx@efSe@%+C!A zRB%dir}=2o*gG}3iyHn4t2$h$5QZB&aGj0XZoW&jYl$z%&~P5w^jj+AGM$cf*GqDf zLTtmFFNBRny8Y)FS9JLT>+NfD#B;jvQ<>8n2)9LUl~mBCb863|3-W+!44 zf|)UW&ul(uF75bCr(Db1Xl+ql8-0$IZyts^BuUyIIAmZPpyDD(fB#AP@pbe0GG19> zr`&_E&c7UApV`$nM=pPIq9wknc$F-naToY3d_8&a_zpZ5IHg6>mXKAXz63HW{CyBl z&l*~e!U3r7g`2J&Rg!5nUWu{fhLuH1L}h zCX^sijK-LK+Dm@SPkXUz)^gRS9d9-C#d@>PbF-BX0pv;)_V*-dOmlP>?&|l-;JH(d z(?$^tRBa84iuqB~7OAd_!Km)C~KUOfC6Q4de;$?e8C4 zTG3i4nm7^q$(Bf6`5Dxo<+ew8DgDQ^S+1!vHnlHK4;_S`8~b3I&Sx*z#4aNvGjrwn zpP}i-g-@m#kK2x-_G2ffyccuXsJ=QULd71s&`L$j&aS(};QNpQ$Um2aDRsO#89h6o zb|(}G@P1UC@%6~zQ3h}7c*h_EQ%4Fxj)5n=E zN3}O*ixt)wG63Nd|B^wTQ?z10ZWBUOta%`K`hAwFlJDi@dl8cXx#o(=W^KbP%0*Ez z;o@qQ5J?=2sE4n|{bs(0;hVt0*^vHZsb{Pu`BUoGm{#LKNs!*XW>4Nq*HyEBxXpha zFj5Q~*N0khtq-h(Pu$wAX1e)6Gg)YCq}KM#0z#ME$JXK(7wU-sWY3!=qnh<7e(kPu z&hZw=iSfCNQ0OMGihjCA+)sV;J3rKQhBr{bOc)z29F!z>uddvU zg*hK>9?hHAI1ALZ^;YD6NOqvHt=4YaIs4{!ur!FMxi#|3v-y+&o`lUIosv@u|CMxH zCO@9rJ*(|`Yt12ZY@$bDYl74$Hd_;j}9DuD%0lNmaubd`$wC^ zLjpp=M(-ipfo{#?9EW?3Tj?skbG{lVKxx+pMn1j`q8$#Yh=i@2S%h3mT`!!J{q%dr z4@?*7G?_!RT!_1#v3*^gV~$zgXi2p7(_F%TzDNBn2jGuGf0mvrP;#;IuH#+21jQ3m z&&kGcLnECYZ3DxT7sTztLd${z4IGv_=6qVe6?^4NKk9mtU9CVi{dhMpKG5fq3Q&z) zJpa-aB>r`BpH<$QP2BBv#WF+tRf%S=ZtYYBP#aa8g2eszt;n*_HUAx>&D$RC$v_$1 zELY_wl^-tvq9P32?FD8owjfE@;U8)Ga{F7Su62m7g$Ns>N(Fsg^&4EPTUDOTyCK5q zbvJ(v_0CzgN`g~J4EcV09c@C&Lh~OJvOMDU=k^mxY<-dwHCH zCP2Ak$e{j_`(1}>k!TqggoRO3v{8qpeS!&D0MlvlqD-dYo%Ej_^bH46lzv9P&qntj&J?&y z9E(nEnI9_k_}x}cv7Y|5DK%jtfRMJRslIaK@TGwT(&aGTaQjh|enAGS2&fW5a!OiJe={iF!F7!&|hFDml(rm-Z zZ|l?4TjAi1eU7+e!?qluc#atABVAbo7A@I=#aGZszsO=5O8)oeNRWv0 z7aj&tUBpLACFD%8TBZjo(Y0pR(=`-)8F^ZYj~D10A{>zYYv?S!Gh2-~h^TPn9<45L@z))d}UEY#m>R3U>|3qwslsg+vT4RT04#$gl0u|3c&eCNxh zLjB?sL(FVH?nkjS!6p77wDWJ0EZAhc3rhXH0}+>Z5R2B;v1sOZ$BtRGmz5VP(VVfE zSPn|MD@4a%1OTtla{1oL9ov$pv@(j@6;e3fp_=ztu}QqXOX(+-^Yx@mFr(z<;__{D zNW@QhmGYbU&1qFa*(~XLiS03){>`@8g7>=jW1QwIEqJXu8>zv=$-q#J1p8&d!pkcv%0b$nK62>w|8?U8r?Vc#I86u z-EB#sy?6s2M&dl;mj}@&W~W-*lTPMU#&i4Nln+!suCRf$kaEvIC#lJ-^nerh@xiTM zeSGT~xLBbN9;IURI%5d-YEhq>uGW? z?!~>*AQ9m?s{Y2st9rs(wA#M6YBelj;)kxuu1xytq5MTxTwe5?ya%!^W(jNo<2d*0`EdT%j diff --git a/docs/img/streaming-kinesis-arch.png b/docs/img/streaming-kinesis-arch.png index bea5fa88df985ae9111d120bfa26e5769e905c2c..4fb026064dadb6278d9905dead3695ab974f581e 100644 GIT binary patch literal 86336 zcma&N1yml-vgnIjaCdii*We_;hkI~$4Q|2R-Sxu)AwY2V;KAKBxci;|-us+$pWL?| zYXLJo?K557^{c8#kdiD45+M=<1O$q_oRlgA1Qb661Y|Y>6p$kG)z26B2kELR`w`;% z1o07YgXkou;|c-s0qdV%NQkT)d?3S|wYs*Ow&G_&Q%8GNV>3q+b5>7#Cm=Tjgs`U| zaA|MuW=!sBZ|C4D=qW<^uN;EF^*_mMl;r=);$|yCsjUbimvnS7C+A`1W@V=oMIt9B z7j`kT5LA_t`A>1+PK46R&CN-WjqU5#udH9WSRGv~**F9Q1lZU)**G~_fE+BYUJh=? zo-7WoRR6Bzzv_`PcQtjfc5<_JbRhqyUSktScQ+AA%6}UA@1KA7)6LrAe_C>I{m-y~ z0kZw`gpGrho$bHt28s&*lPW0bXz%1=?&=EEFUlkQublsvwEt=6-}ymSj&6>?EVx*k z$~(B3y8wmVjQ{DKDCd7l|DSXG-|JF#u{H-r{jbs-|0(^yr~N0tFxx)^|KBF!-?RL$ zRA8D#k%Za)`<00z)$MSuK|m_Q$V*A6doKUYg>A>4!yCf;AyJZo3yTeTBvYb-E-eB2 zC|F7Z3v=WEc?1V3FJIawlzJwkn(`8`LE7?eZ1Hvd@7Z6b{fOsGJ|Q7j%d^~TA-^IK z@0;Ph1W8nZ-2_RD)I4X>pnqSZg8oVQCmCA(Uk_0Q#Q!Vb|F35dssE|y|E1jj^BK^H zG=rea?r@{+Le9Zvf^4hrI%xXf%}xjZ_$6{qcOG>wSbSzgoAE zvHM!s_W?a_e@qLz*r3e=hgHAXcHw(%$6eMnH?d3LXd&I&>5VGXD%d$Mv}? zE%@_~l3~bO*MF47Zix;wtF@<_op*k1__w*A%)h@q*PaEM4#i1_9C+OvPP4boM`F?S zUKRDUjphfuE!CU~Em`J9p9Zna?j%WmoF`Mm4djjBgm8V_jO^&qa#rU**Jh2HMy z;d00C`Hn@m!Rm5*z~|;LNSJ8$_v`EZMs3&I6I<8I_6gegdPkepbP*wsO*_yq(EI?a z+s+`$jxj#Ymh;w|dhM5vEKc*$PwA{n{`-3* zyR}yL`?IwV*(Y}=3+LC9V)sXM5gp^%d~7=PyFk|`^F)Xk*&eo|*?M0uw}T|jr8pYz z8^G}y0}VcD%sN*u*OSFMaOXwFuC~CezHEh)y*(a~HM#DOMfy7~Wa=C^Z})G#0Nuc^ zI-Dw$WNW|ZoyZj;+1~+PWt!EWGDV$wv&XZx^S7sy+Pe0$8ogiZUI!#BXiy;`d|o@h z>o$=IIJM29U!U)Xei&25M_cv#nvZ3uyb8GN%6XDCarVa*jB?w~SL?KS)C}x!I4S1v zV-Tg^esXe3XEn%hXtJElGx%X5O*E7*^!2RCennSI6u%0=9HZvDVm2@8wR7Mo#V6`P zzG`2OT=bHqwy(8u=bpkz(Y^5vMXbb-J@YG>&o@WO!d@=;`M;ne$~RY%>6C#1{8A~F z$%sqLJs8WxI+m+OkIRq7W0L{5JU<@f|1|$SIN0KPFq+2l42%qlLZTHnKCV2&Huq38Q^$K{42;H|=9Z}^;d05hPQFHw?j{ijT$Qa#er^O$t#d-nCV;Hh> zSn<8VC|ziK??0cXv|JmX@1v7R6-Zks@DE-bM)sL)p4WK zfaGF`EMSvy(|qUl8`h;Ln?Wn~e(5@jr$Qp7v^eL$syQ^@gX^ToLWj@&Kr|kf=yxSK zdOUqZhcC5XL>_wK$$SH0XGTF=vRoEpo8#}mT5#PPO{G&NB{t|SCSVT?Qh+bETdXTQ z3tyLkqA>aTcacLP7zX%6NO){gmgCjm=Clkiaa>ye>NGnQ-i2@CLXyOgX($^ZIt*V! zig{{EhNI#zXn(ckvYha}Te38^S!uF|h6cn!>CG^Q>0oyTI&1`0U@-=@$rZg*j@%dq z9XF%+;K4}4Xh)#a*wxG9^_-Se&ks6_v5Xw7z7*V&79p8!72j!tP$5Q80mM$v%i$1Z z$X8dES*g;`k)n|xKaI_( zrA(fKVmug5gl?leCXa`zHnAVp!nX#-N1-+;jYiCGB+T+VOhW>pYWaG_vCElC+wJvB zH?Rlcq{{D(1L7J}h}Fla?80Q7gW0Ga9yN(shtoVoGD;motwlmL@R^8<>) zF-kC005&dW_Y-?4hub(^w0lIsa;X z@q%SRxuce_vXXvDyQa4$Z01a4nUZKhf@WW@zAO?pLv@%bL5wHy7_>9#-wDqNPjoV% z#h@K7o(fwm+;cmbM=9R^&DJ&c`N(P66Nl1yG=-tB=cNSL(ggi&J{EkS?!0x%`f!*mw(TyGyz?p2S|^Yc^a=z#PRRN7UNk6 zv&poICPc_ai;G?jJyYV%UB z@b>0G<@}g%@Nf}PV~&@!CRgTSyn49MgEVFsHI~x0BfHiG3`H)8p7^yzuqv}WDK2Kd z+~*)VE%WJ~XvBSL1IW6c`<_z6iz>C0z2{l-G)sZRbtd1PcmtE+@>(l*zM_zye> zZpL?MA(kj8?4g957$b&IT_;_eY!+*Z{hh1AtD}g*tHrV*5LU5d3l8YvNwBX$!+F~~ zWh(KU2+Pj845Sfk>q;0(G$IMsDUJ=@O%uCD{!A<_yS)TrLzvf+b+IrrHXqq4EF3a| z#aV~Z+C#4Gd}6c-A`V=V&0x06`V_&WvDhhfP=CzM6dOUw%yyUP|WZNU%!4ma#-U64>L`zJ}nCnsYQqL|~=xk<@m^8xvz!`>F zyG9&g-b*`-kJp$OF-N7o8HB#IxopH$3)yNzKUPR>wsQ>$v!w4q9{*f3Hj2; zDTOPewaMg8D*Pn}UO5a2+W19Sc@cdSDUQ#57uY=nW=s#WR9otJw(PxkJ!aSqFPMKU zAF(QgknWS?XR1cDz~+hdpyC_J??GBT?8-DG@w-H-un+b0jYp+w_~5XM!}yXEsuT2~ z3s0cl7e&L0m#cVKIC3FWhY{9|?z&{lXuwrvy=dMyx7ScQR)wN`ZZ0e%tv3?(CYGS0 zRa)}#YT_UC92N|`cmMakLor6OxILHO?7Nj6LVj(6r89J#{@T8~NDR{16cS9wq_6~z zm%<%cNolETL_M^7LF( z&C##p53Y_>65lA8K2a8?Mk0@7fw{zW-4y+!Pa^stj0D1rSUPc*(l)la#Gj`XM!_b% zkU?bK+DE|;nBAmGrT9pLG)1%M`AejtzWkJ2qHwEn+nw7+$z!~ALC~O>S*{-cVNz`7 zQ)vo){(_V=HLmjuldv8s*AkBleGG2Q-9Lj;?6AV(c})?It$KX}!MMT8HF5AwIv)|Z z2R-FTAP0r@?9IYgQCXRg@G>XKaeo8bmX{D$S8wHDR<&Ar+!<2|I=7sI=2t4(s^Jxz z9^Kyrv!K`z!q}35StASX<#ip1c4sKwkmLe#6FDj7KME}V%tqlYp)}GTAeq90W!mWg zY44YV)IQ?`zw7m=&M?PDtDgFs zR_uoAte1};E^!NAySW*(zIjW`APY~7f+FeqaeBy&ecOM&i4pq^b~^X(R7m(Kn*7fQw+L?&g=lIr^q`ZS)Tq|5cu}fd1oW50 zsd^Adhei+{BjuKam2-{}darQevwFU5Q$r7v`ckYywK5RKch8)ywLM-B;2G|Q!yq`7 zU)VZIdwR{r;D&&Nmil6g?JA;OctentLMgOJ!lrS8z9B$4p`r=v6V)i05XfCh%xxYZ z!wn?1!Q}*MSGNVy(@K%n>KpjjqT8h$e2bEl%ZAwqcs~6VAyGKN{OAh7DA~$>-mp32)&K5 z`Lz3i3R1=MZIkSnypnOqiHaFfQ|M1DLNDfa$V(1~D}G@ zR1;oaw7a#PN#1ZRfJ2xzx+;DxQuXKq4jbk<4jb}!gAzwabK`vH0g7N8H&bL^omDS?H$15xFOU53TH=e=OIVnWhZK8RfIAHS-Ln@8Pn7Yw!$&s* z$-gZ}Wh?PVj(pP&v6BI-FwB>90t>;)6Un}#k#~|e)YtPN%JIcYy|n-pF-ttm{B-}3 zdK+T$o3CQ(rzh7+y)j(fn&B4JRVQc|ZVN^bq0%H$uL9}2Z|3~gwn z$(j`Nl>;sonO|ws^A_OINbTmUGY4p8|BZ1p8MMCZXY<+@G5FZ=byQMhg-k!jOyWBf zy|WQ(iW<(Mi{$2i#q!4QO8NtUk6Qp=l;?}Y;vSI@4WTU;P{iW758d^1oWWf zwiVUDZ&5b3dg-ucY>ArN<^I%+E5;@{r~A)>Ox0m6l_ z+)}R4*ED;UPChfy$@Ljn_U9 zR=fa`X}rudWjkH`VH%Cg5-47}@i~Y82axP~Fp+9(8`+11P49F(TY=D-CS`1XS3~?4 zLL-GTi-Xr;)pNbwYogdmPWG3>+SA+f`Sa}wod^EAT-4*XC6W`nY&>b`bv=jo&0)LO zbrJW2A0t0c<3-o=nNas66=g5dbV73xAHH~EaLcz&KqY-UuBzpAT(|Qd0F>)Ocx^HY zG-9dCXE*>A^(6310vN*#E!Wm!BIoGT17i7V?{|1lRPW~~t(o)uUp8}5jUn|`dq#A^ zA_^FA;X%;wX^a|3JyTwUsMZw$?{7}6Ws2DonOsBX$#T`X%3(b5ZVwm6H+e`nj1VgP z9fwVJ+%{g<#q>qD=77R^GG7yY09}Atn@Z_v6^=?G`g)`G)IM8lNLD(&Be&dO{S6Z7 z_6#6s!?{_PX z$cWq`PMd-J{>?mrNS=e^RCH2q^r|Jp>4yH9JhqyY*zDbuKfT*R92#%;1#?1VW$8(X<|;M%1a|M21C{cG3CKZ5e~PP^{cvIw?CllKY;O}JH*Qsvj@qiGcIQ$L+3pa*YLF;ySbDkrst*-j0mf_ zm4Y?G`VbDq2{$Xg>o!KU?MPS;!3oxjzJ7r=dD?kNi}UzQsS+tA7xa&QasoJG&OcP=qmMKQ(lqM|Zc8Z{RuIkkfB z$1nl#7W1`+j=~hbx{02EH^l|_xxQb~(^mm(KC0K`ajN?iP%m*9e#V}V^yi8mAa=j2 zD(_BMjftTTVP~DGQCaSOa~TIgp58bDi<1O{_%Mr{l4Ui+Z_T|$P3G{PP~0f>ZU5LS zYbhs}PmD29p+GY7@|n6lt)GzMhrbUx+YMNj)d!9pE((7d}+NJ)v`ozA?^g z#}NxWq`k76Lm+b?q((GDK*4uY#00H2YF`AhHnR{(9s1o&%hFpivqR|XzUE%fxfG#UQM|BIvncR5^(Cnm8>@k|b)FPtvVkcwBjF?6jR|9NX)4z2vWHYm(-FAr@e@f9j+L$v zHT<{FhY=8T5P{pJ9U;!hF=QRMbz65F2^aV(tX|_iycUqFAET29da|x|;>8GuB1-|8 z5FJzvlLsw60Fs1d9J;|PjrgismP&KDLC~+6$7U!hWe{Aze!nF!<(*25B-<-7@D~}M zy*qw`GrOKQL+AZF>cQTDQ!9I$;VeROZ8x0bWjGkLxN5YA%hurc>=8IF?1F{^vj)o` zfsCyAS&#P8g*Gu60-BW-;u|{>C9;T2w-2&&e^`@o9~`%IgVXgVeA$TRz=+Mu!N<{^ zNw0R&@U0W?`6UCAKO1T#DvvBYR6$pf=d)fJ7zyn3Evpp;K^dWa2F3MCtq|~QTG-KC zD$7%j)rJUMc=K|uEEfiRk7mnPcQ^|1BWNn$5SE)bQw4kLBxIUWfrh|Ex-oobr&Zm3 zjb;f}>V>37A>~5iw;|rK*!wlZQIcCq8MY7ox2HpnsZ{-r1*$#RD(5}*`^CF{(oUEK zA8p+NkB^EVG82(%Jo3!?sfAY*PHy!gC~#TPV-+YHiCyTOdPm0TXrTDi_A4o|m#srgYTB$5vyH17l@1 zYV&Gl@+i^zoz4I=_;$`t_D|_OhZq|pOQBu zt33(7gzX-=^58zd@$T8GBf(hsWg6EAWTYgkn%8iq3$4&;7qL4#*O*z$a41GW&_#Cih@$`H>9Ep6i zZfrWr_8~y8^?N{k+Ifk0vY1+DDl_^|G{)H|9=$Koc;!mB^$u^(GeQ+fC#XWcH@o$z z*>U;&*9JecNKw8HDdku%JPzgMb}Ak=pA#}T6JD1<)YF^Tr*#^+Wwh*XsfQ{+!t%i)^<+hF8Kh#J$6?k|)C`E@>6l6(rs zmic}%YsSgLZ{!#iL+X3v(7I7!+7$B;{6iU%jp+R1dsgZ_P>kxh?XpAujebc4pf@bYS*d77k|DVYHrNe zSt;dk9Ng!&p;`P+`o6qS>Sj0)N%HB{>-jDhpTn2yBpTgjK-WYF^l07kGiiCZHqvX( zXTBR3z6i{{?N#yajbt7T3_VO&E1oxv+n%pp4?QUgG5PCI2COI0hYft1e`|H3!Po3i zt(c*aw1Lg-@+|%Ghc)A<(^gllN-S68>(TaJRpv~2nwZcXev|u9XIg9(&I5THL_D9u zrNs<`sW+s*zWIAk@h2^ZR{vLv9$$g0-BE6^maYFvrQiD&bbu$57bYybD_(WM9qF-A zoZXRH7fBJ}o7cm`uJwne(^a(D+rvdEsa}EWbTTn7xR5H1#<3u=I8pz)cw818F4wc9 z9lA-6^C@b}rkfMqvmCB~r8@)oFf_Y-#sQ*Es8k%=$X|^Z7NU#{{WM+mq4XmXjgIL; z*I_pD*f86>{=!H*RImzuP&n$i3J9tO=&zBZK1kp?*Iq9P!b-kJZ>4J3+0df8!e7u< zr#%KtDVSc-axis@46e=40`kr5WL^j0B)(j*N}$m{Lh;VASC zethMQL8I`p^ml;$YWvA=#&K?()pvtd6Vv<7VsYl`fcH5^hn4W$<&(2jSEq*zL;gAN zC70hZdc;wOU1^zqN~{b^+q>68$lZ#_GpgT%j=kuT*w%sJlTd1ku$#lPPnDF!j%Nd{CWKri%*kprrj138%=-61r9d8zFi*m_`h5ZjZFha zwAT!5jgKkAuwwp7h1&jlU=T(0P+y}!UXi{Bi(`e0REyc@cdGVhzdTSIgKB?7Wi|K2 z#RH4RI>UB(6ved0Cp}Qk)*}Zs6AD^I9|H=wUF91b%1gEm)#E~cW_|b=|J|1pvBHm3 zVjNwy?2Lsal@e8$)BIW*`0?{R<)YP*(r185~n6DKb3G>z!DBJ2^^zEgc zJVu4O^{68=m4C{FZcup5NU;!V`tBrt`qu4ANyzP<<5GWFOxKckSwpO?D-%#y-~0+R(i1o7>ZM8X_j%?wXO z-71ec(`)!+W8BTj(k#^f(B+uB166k8V-xN$g=Gyix&^mPJodO2!wK9zm1LFv$fgYK zs~AwKVAM76WHyhkK|?N1D|?Z*hgF!XfV?mQoEuJ5$UuEY*_7HY*8CJn@TkJpze9Wx z)9=CImst9Sz4IcNv>?`+Jl=1GQ=Jp^qproUcsTAHO5Vca2-N#b$0$#N5-Ax)=#=-A zjh)p$tEU=$S3sZ8&J;yce^@O0*x6zpjy=|Jv(|r9m)-M>a$2!s z39PC52$pM|71DwST@dLJixRblUr=ri7W=TpVF~KUq0#NKa~E=3&rJ0ye0-tMSLDV@ zw6~d1ZOlde{IV)#TR4mvi_?`Y&QsIgUkeyuhS`@b&j*y{2NB;CC6 zT1To4qL+@LPKfDk=?%B}f#aCem}P`G%8-^>4&}$6N@5QZQOgr}{l*rHuOU(0v|oK9Z3bS{X^2zd6lc6k?=C|67_syxt%Ag(*c0FOIUC zL1UD!>2xvRa%N679~W0kbAIe-YGxNR^>@JX*D24N60tX7kBg}v4Hasrhts2aPP63$ zA8!8gd2caQvFW!5)24GfL-}iWxG+rrh{ExD5C1M1g)^RgaqEeURz6$p^{M4BnsDx* zYGy#ZZCvlZKjxA1L%OJbOTA?T9-mSd*-$d`=NH}R!MSC3f|!Z$ZJvgW``RvCJfV*( z`;8_ds1npI56LEvz8dA~pF;u&T?2pjbr~VGW!X^H^of^3lB3uQuy_`?;uOO{LP+5N zT;rZyfjnFp7orel@e%@>oBfM&1}A^W;Kk{$jox`k18@H69FP(e(#*UE;9F%yYLxBv zI)zcN?JQ>SwWZaqLG@r)PS<|=2&rK|yxlM-tq02zQn@@Geq6Dc-Ve`NF;keHxnnF> zQw!Rv1us$ekTe__a-~OSpcev91)ihq5=LB>Sj)~8@yH}*1RFA#u0kX*%}C#X9a~yX6N9+?t!JKV_+w7Lm-dvpotk% z7du!mBFqZ2&TcaVS1}M2A>?%LAY~j(xH#Ahk7tpfPcpluRJsD&rJ8FdfEbT)g3C;L z8!h5CG=+5gf;&F8!;UThSGtO!6En|ql3U9wK1@B~a^4A1`J#nv-s)`)G0y{*w?~nIxA9sQ z!qnT(w9{br{Ka~?I<-Rjg0mgXo~&@)f>!L{0;&GX8hmMap#jt~dus%y`$a_z{N%7$ z?JbZL`s{;(I#vpDd|!7byquGTIyWi8M}c(lt{yIy$nO{80_8unC}fNJ3QaGW+$5%g zp~ZFV`y@@zkyl+dodIBMz14#gBr;2a&3S|wc^syP&_$xnlp%#qCDe#Z;tyNtYynXV z05Re))>4r`*&c{)&j!}O`E!9Jn?upLBJA?4?`pPFJd8wrgOm)BN_~IOY9Z+!- zsXu>c3&9d2Fh=AOvxQ}^deBQzkC3^jXq!@|2{A^89;2zru0O&c~}al zyncNa*CpSl^k<6v9-n9Dd3Y&T`oWue6SrW7^xK3Pah+X4G@KQIBUZ}C_iHvYm1)2~ zD$|+U;6x0Pe87dMj4*Xp!7GahQqEka!9RrAeo!(4LkkZc&J-k&XMA4lLJwB>xw!bM zO_aDGyO)eMnLS~@t~l+zjHL;m+ObQHD0>FPrCy>!C z`|VGqioh<@4}7O#j0&sY+acO?^O^C(9EE8tO_?;0(+K30=pyjoU5kaK5?%VH&5JoO z;IoiKeN99Tdc1f2k*U+Z@UJFsd^vJL+2uD6)|&kdkbPl zUS=)0`j-Ch(Wz(qvvV`9_^CfQMsuBjO8tV~SeWqMC6B>2wM zpxH$W?u~*6tGR=Zrdl!_pR#Dc8yj z=)QUlevCp$d7w9Z)!nlTxTaIgUMTUWu{YGxi{W}htO3kQ*un)!X7-Dy>e5lzg;k+k zg^6(pG<1Y{FqV4#Hy6`jyL%PL9TYvc;y}f{E1GYYQ$1q3yL-cgste(p77XrE(|qK52~MQAPA#mZ9)v)XAj8NF={#cLcY)3|A<0c+c*z~@ShD&a2^ z3nBlEA{auQ)mP<4SH$^>IJJ(U&facOVZ_E8N@e$S{#0o1H}+)8mcf!Tmhkdnu24XQ z%-)O+XIWtsJ;~5d3vWslqp|*_d_VmJlqcw00`9c@6kIqzPXVkC61Z)19fl{zN=UO<(r@fsgpOma=E@z9|1wg0Q`gq!pzG zbZ08y|F=Xyhci@oitB(i^|AdhJoKu1r5TzghF$Dwd7AZg_vm`f-^xtON~Re))%(HU ziXD(IDJ~Qz1jag3hb=?9?FWKI-+2p^j%)?@A7OY^puxU1Iolv9awIw^b#+ilnaSoI zbu^(y$V73`#>w&Rikpz6f(wc0rS1#5e&MHHvG>&)gAV3RsY#J)UyZJ}oV0oxF;;T= zMvGUEr+{)FJ{*#v$v$Kx3t(}Q+W4C&_gMs-1q?c9ur4~2&x zE-O;hjiDLFBpBkQis94S2;ysvcF1O6DEpi3#*347QBJ(gW~oew7918uZCHjoSx*3rYA{41b+2 zE)*GT{8FPmUuzVYaz7sP(yhHZoi;7^xgnCSCjX+HQm&ehJ)h^hTGFC;?59XqIA{B7 z`5B#A^ocwXm_p=knF8-DuLSoZWcqx!>I)q3izMNm-;Ja5GFSx>bkuse{Hb3i^zeSc zaoKZ1@7Fu(Zb#U+t+0$PPnwVnpZpc>>(>J zE>)I@^781nY}vYQOb)*EjpVj^-wG;Kpk0{}ES|J^I-9nwZlO^PPC$gOPv*7++-pAY zq=08^yBAX!k_Mu$8U02B3>vQbM}7jJbN(*UsOhcMzu2l!xS3#|RbL{#ogP{}h@DC= za1lPN;;7Q44$(;r!2(chWu7L@Xi*JBs1fBz;?d=LwU%L<`YC>w|&1JV8L8;p9~bC00>JE?WECC4WJY7nDI z)*C8`fP|?PWF-Je+|C55{i<5+v}zu7f){p{+*AL9Q3JYO-M5f@Po3yQ4bC!lw@m#< zQmLrcZ|ZdH*k_im(-eu9M7pmxLru0bBr5v}J`Kayi&AlbTJT!Kf1=4T5gp zzsR9fNhfk1wZOko80NG&35-H$z)S=CXZ^>;&JqS>*z!{8k5X{Jf##1re5F`?&Ts;2 zKQ+%@dT1Am!9thpr(pj_uI?Et?$j?$E?5*p8$4BrzbFK>#0!kAbGi;XrN7!iV)jYz84|PG-9ZX?fpbt<_?KbY%kQ*E1=b?fi8ORIqIPhZ%_6Fj7*(>C{ur`@m3ZeVbR zCVKIzfd+2PR6xye!SVq@17BUa%kO#OF93hJ3k*O!P2%+J#)kDb>h?TBi< zuwA@YoG+n93_hD8!Jps7{n&-WY4vCR37BcGpcuOw6#;Q`FOl#?IloF|L{0e-G((zR z@%b<^$6ttbOCQv=p};*M^#H5WX~g35s0}hb_)GwHBWz%WC{}Gzet+dVz0rxD?r6Ou zl~CA=kRXX87Flu}hJjUI6XITPypLDZyBo{wV*XUtunAx5`c8orh4UCNn>u7Fj2f@H z49w9!CK2EH3?z<3o15xeq&n(VG z3xIIlDC_G!!g|UR@VQLQ;28CNqmZy9`t=v;yT_R*zspO^$43{Ja+a0bTQ<0UD*679 zUnTzefsDdDl}6cbl%fiuRG~s3#7XzSQA$Lz`wC9#^Yj$VdmM04rA+7v{r@Mr<75ku z%X+f7q7DHWy8`H)cb$-v%_|Ild#WLcMv=Y;FP+seH6kYFymX(8h^6!VgA004`@d)% z?GR{ZoS#=b0*SwXaK$~_ZMnf+dBW~!?icCMl;g!ikKeMBnoA7z3IdPA+|H|hrclH8 z@;kHNNpV7yF_};z;ux1_e@l>|@@jOh(@@X;;M&J;){A}{#2_buQ*9og7hTIjc9)Rp z^jAlPUAe&{2MLsk3V*~@K*)S?~F>3RxwQX>7F18?}x??MU=spd=UblK`kN)y{Hwnuy0S#KzE z*xlT`c>OG{&LG|yXI`o{###EV^-~{Ux3saxtmm{3Kl~OT*YNRKXtHn=tP%fiZQv1# z{h90fj@6JO^4sVBY#qL&Jm##TbGh+;QoFXlq>@->+bdf0)$bVGvEFGa~C)shf13SEvSuX!#GiVw0?1zxk4jD@fYBebB1? zQfy30(mJ&@Q!1}hXcsw80#_#%@Xrp-m)aC#!XV`6s_&v7OB8y%B{D z(o@out}KmCgqb3o(2<=n8Pq?9_)@Qcm@r3qq+?w~P_JZ3ll0e9v4dy4#d?>Lf46z{ z+J(Ezdh;zhklayXS0ok-BHKy4$Oo*5LbH8Ibf5agPRi#B)$8Q-D@iw;4U=7a|H$5^ z*s4R}1bubCidSz@O~&1nfMwL_k2_-X^iJI%*t4;Jpi_8a3>&>vx^-h&(x#;%9x0)* zNC+IYs1TDAr@5tpu12t?_W7VGgC5I>Zed0R5+F=fW~)@LjoA<{{mA=*Zld3$sr;4F zp@jChD&$#*juRTQ3Q{WIGT)V z!~!nV0L%Kv*#r2J07sJ;c$gL0fgOV`Ydras*;%^RPQbp|>R}=!emRnbxXW156>@X% zcb#+bdXJ+Rh^{073>aCw_Jpnyfvc-6u1A`dRG>#bp*QJQXW@u}CTfP?VQMY>s>#@0 zd`B}TfboBrg^ChHBIt6}>2m|~M#Y9TdHqNix_Xr!9+fbUD7g_emz~(t#r&G+#G!#Q zu#4XX9GyQ0FuKp8S(usc4#ks|A5Zp2Vp)!5V9+TMZ&m^!YpRc8i-b7LX9oly5VI)V zfB4t9tYe#3PR9_<({$S^=KhdV_#09x150~p1jOnE?2i$MOG_kbhH5}!xT$u!9VMkb zXFG~#^Esw%%s|=#u3UwrwJY+L@cy5;OctY3ccjnC0xNF5YM$mjVh_6!XC@|Q=#<6F zb9)WdoM;cay+XZ zC)Bvr{;@d9Au0m%a0_^f&4!GG-Tm2_=#DSH>-=lH1B9cA=8~<`*(DBQt;fk`B=19h}ItKrOUT`3bE6gy;E1Z{xp=^#mL z=i~Cj9KaS_m(|(F5!55V?Jt9jIQS5-1;te9JmPmDnk$S7X5 z@DwX#J)>V@^!-ibZ39(B3&Uk~x1}BNW?%c^nb74vi?8w+i6O>(`1$j(|t5kw%Nl-r}7*xK|Ny(!>_>u2uLE2%mD5*h^&)mxFVvW7L`ef*7j1V)Hbv23sYaY?EsH zOThOA8@84|Zpfnd*LAZi*6G9Js|SdRnZcIAOEXdohmpm_suMC7xY3RF%HjUazVNDQs# zTeOf55%#--Dii3YwN8TB>!fE5wJ)HcD(g&Sp3b^mxnLiNCbFDq@XE)YRK`cm?K)DL zE%5O2o>%e8qSC5a6SSW&@psQ1LKY(TGcM1VmidV}O~T zz`>Zecm*=4yzWs?jlV5NQo@Y701J6Iz@=@S653n1+So2P0UHOYsOREaTWdqo$NI8n z1eIVFoB+k3>80!PcS$Jfw<%tLxC=*Sn*i=`^2aRR#yS6qZP+Ok+d}PaOegXH+uLcEHiu zvFkNWOik}W6Hd) z_cL%sZ0vfbz=w@LE#E1Q`wzb$&WEK`%d{O&JF2&7x?2a3Z>Y|kMo&h?<*`SjP8enX zUc`Tq ztYqB_E;dcoDDQ!z#m%%#{pC(XcK?g6w_uAS?6z(3;O_2D0tps8xI;tZ0Rn{J!QI{6 zA-FZ}?yiC08rdHR9re*->o|y z3SEzjtEk+qvwoJe-ahB}gt-oe4@XSDgP8YEXyKi-qUL}buz8i!fle9s57VM3L9aM& z0jcN=gken26z5~FbsWZhwZ++;Aq~HQ_o=aiob`tVJ#1T<|tu45whf>s@9qg6MskdLuA^PE!N^(w?5Jc`&iXilkYr};r-g57D@Fu zt65p5S({oI71>z~i87eH3?14*<~``4c}NR2KnHh-iU4D>hLqPj$76l30|{LE-z>yd z3qH4pN^p_3y)8A+5i+HOWR6yAA(v4_6TgZEs+g+-jr_f9B9nG6QbdOG`?O}*U$mM% z-F?G;ij)P6w+CH))Ngmo(1MLexP4UY`Y~1a5Q^PiXl3Bj$fPtLK^o*=GVfE(q)Ho3 zu&1UzR2i;*No}xXHC;CMw&rV+r%`;r*k^Y25a#~j{)HLYBcQ>;* za@G{BXSs^e3^Bxc361 z?>kHkW0t?sC`%YdGS{fGgTtz-Pk_J6FKk(7he>I1ESz~G&tF#r zm2sO0O6NEqWC-SzqQaAHQ2C}+FLb$1u`g+G#4bB} z;Y(~@(PoqGj4AVim(ni*e!G>iHrES|}V<_}2>W>_=m1g&ErtTx%!{sKX5Y zw?YpVEuh}TbZ6|XkVmp1h2a6Ar%O+Rg>3G- zWBQV~N;k3vDTtdpZWzRJ5b}+}1Qbddzo0NHKGa`AIcKT01p1*wd1HGNe#zqUK8%v3 z^+^b?oW(zXS2{4a((2-}rI6f2sbZlaqWe+|#GeVf^tni_q*}u+13W6(<%H~ ze;OCY&XlA_;J~G zwLN_7C6i$M5jmp2o<3>3cBFz*HN$1~x74Jl@T@?gAc{#VmofwY5sh59Nr}WfU_{a7 z#rtw4!#mA;jp6NSu?g-GXLucNG>J0#*A&Ayq_Qkpfmdiz;1w+Mvn$kM(k9Qyy`K4N z_2lML2de+V)bhSXsT=e1u=ZB4F~ck7q{jzbR;s7TiNPnhpjvm7?`=`y^e>69@q0<`2H=Oj=}r?fC;9RU`uGA+KIhLY^wX7lE^XO(t^6%jgS)h zMb}#>o>sD22K0pVe6&&ly`RW0C7Fv>KUp-RyOnhTCcKDiEydT&5q+@tI?tb+FivwY5cpg{ zAMG3<34x*ZGJW}-f>e7r%S<3mKTxb%bnmeh$a)Z=Qhip(7?X#DbKaCz_Sv8I8mGAwLkAMZPXgZGKEhoYvrli|D6{5 zBqy!6&{ay^@{(}9l%#uH+XWp}>;Pp_%V|6yS&=s{;@JcbD6!-tY<;xC-Lv)D@yZjX z&nzEQVjI3_F){T}|48e&H5Cl+m-uo*Z#gqY;EC<^utNYe#*_&yz+X6C9^k;dY%(%5 zDGb1zbvTZewyW?<#wBj{wL5Cx;rfSyAGgz};8%{kE-q$+??C9EU#cP2GUifxMwH_- zB1w@r`nM^7p1v*=X|;BT>GKggq4-5UAkdQd89AacO#z+!J!_-uxyt5enp|~J*SpU{ zMOP#co5u0XM$65i(+%EOrBBMkexEMl`SaD+VyA)-Q4I|o5%2C)#<~;#O2gj0{Ol z!DlgkSN)d)?OYDLa@=+`YX_xen({U5eA;F~ji?BxdMjh5^L^vF-pmW8fb+@_5HR$v zcXfD)C%u$ZS-YyA_s}^9_yKJaENF;q3ln4_X>UtBD=4|lGUF|z3VW^_x1+&10RMyT zox*RssfKX3_URw*3(=987N-E%PP@U-$V}mtj3FOt*qia@+#uZPMH(SNmiOhuh&1wZr10x~UWpklH|+=CU^{~l|s^658ry$ z^ccPbYF;wt(#TH=T{aNqVTr%TFv3BZlldsXKP*=#pI>>yAGg`_UDWl#66qi9Hiz=+ z(|lZ4+R8|M&{eE>r=i4E%gU@eKC7PNY0u8r#ZP)|9$fQVQ`b5TX9ZGS5UPyriE5m^ z=ckEIpkvT2!0Rvrp4vH_eLg&_KXN2-rtTI+LrRR&?*EsixaI;`3aj(8$v}4sEM8bY z5FP=v;Vf`d|2t27@kano%Wi~a*!>aD(|eXkN>akv#G1gdmB`=j%{-$^dpc~cEBBbe zoXza_7sEy6zNc&<#{tafOq;w}>7zyNmR3jBXj`Eos#URiw6$cViys!&QW^RK5$8jf zluKyyoQZ;Tp<$8<9d`C_KC*NRkB!`xA9A*paS@n-G`R|aIqN$SGM+}+WLT6z`@5)u zakYzFikV7Xy^fD_8GTpcQX1xZTo#&KR%r>|FOKbIN2|Daz@MqNWK7BaF>|VV z1m^ePE0m~J{M|jDRGpdph(54gNXrk`Pc0Qi3h0@5fuv!r!xJg)0pNB`*D{&aO0p|} zi_{~R=LbN~Tqfhc0CN=ohw{&pe`%~&+gvQ3-8-q9BzK?^VZn30%w?Wnp~)mG_c%Hk z5nDr128Vh{2S(_^Jr-3W}rxP7l_`ZPF(Vv3> z9(>6#^p78PJY6CBL`nL~G&PA$rv}S1`%=mLcDP^KY6S3bWeXT|QZqt{;UAPHz!rJw z0tlen?Q!unmDNz?`)7^PM@C(#YNLc`pr;nf{*|IYfUWpONg#wn6 zXcycsh2d`j1v13WPm!GzZ5Bii6+vaz$76~a^IfYsc>^=G1tTyaU&+Guq~LdwQF1clTXNvG7#h~lk4)vX?K#$(E=`)!UE)G@WLNhY z$&ZAod`$kS-E3zJlyR2PBq+igmlh1ZDK;D^>F|ba^^>SOt^KpF& zM=p&e|4nF72(+$T7Xko%Vr6^Oy)_|j}V37CaI8u(YwSkt(KjpDZ;%0n0&lV z{5>C#pbkWmV5tCK1xZLyK7L{yHoBuOJaGQpPrxyjRtYI1DTB~`0yKTKfas2pof5Dp z0{u;?$e|RDJZ}vQWpi{qNXQP&t`}fVh$P_|xDLioVN#{HVG#*mTkrN4e%QvzSVgkr z_n@kQO&q>B=ilDP5W4s2?-ivUfV+qy2v$M-&?J)#OSxShkfBIrBd8VGlNX0of^S%n zmQu>iUmsF;h&sdAOMG% z^a5gSPC#Y{*e)6KD5879)E^yGe1w?a=7K7PnC(@mjw9;fS~Y9FhXRGc5S!Awv$eM_ zUs%9RaY@*I9DgvuSF-puwFjEnOmCe<8L9io*B9_f05(1>PZ(qPiAW_3RhHKU81M@E z5%%czP%KTW(@9uP4KV4^MhX$ORfRXr%OfFSsMM-0yuV!!0xVMwDSqv^>q}J{dDyPw zu@cl|-J~)3i<-4R6ntdkMw6qYxnUC{l#VQ2mO{j-H9|;^6Ggf@WD}VerdyoPgaFS2 zpb-3qSGUkMD49*IfcybV=#w}@&u`!(b;F=ZYNY`X+TM6Zx{EaSIN(s|&73RONf*!K z#bxpt`I>h*agLc7G6HMCW64l%zy3!CE?hh}7Vt{OPXHec4no`PQM_jLNg2ueBG1_* zf$CuK(8As}I-nCd>klAc(89r8@(qpGCP5H^7tdiRP)JX5T>)Z5AiI}YN1NK}0r{$u zp)SGeGOd#3U)#7!o`j2W9}{hVC{si@lE}})Jvwi5#!m`;dN8ZKUgWp zV_)t=JHXk1r%m3RuG9%<2WY=v{_M&?%m0irsRYd|BU?}n1Ll5$h{sc6B-H#-VOsyJ zlt^ewq8-meI%gI^1%+rglWj)eUhv}%khRd|~D zR=kS9!_bs8h5EfH7ol9gb9Lwa`j2^<-)US&b!O9u4F~g;YprJ;;TW}AwLdK3{E4up zyBQ?te30=CtN14X*=h{$Jde!MH5wxdJi zjYB`_JwDHPsllhvhU>$4J{!VkK}Ei=AmTvH&xsJu4Ja`79oLM45(AFGi$T*9AVWP7 zWYeP#W5;S>|2`B03{tiRX!Y*_TS{a2J1iC|Gx=wk5w_~fZZGBUH=8N&rHo?Regx(kJOK)H*W|e-1omyWI`(jq^tn)H;$H*Gi!yxM`XUpYdz(4Jsz2 zjKcbpGM!vf#IKa1bFQGRyfsW)C`-5}?J<_&4dA~~q4eh;d{e~i2!1+3H@Jl5V7JwJ)|U85Cujsuw&Qve#fvmAgxj4r*G39d z%LWj0GmhbSEZ6S(%|AwS=`0S+9ZsM7`)h-P-8?2<^K?9QdY2np^gMlIpzx*;>dmO> zlm+YiSftwM9e)&m=J$HOXIGZ}MpW>(&k;#dWIFdKF_9fV_qgg3EALC{ml_8S=EMBm zb!fb?aFqtsZ-D;>hy@&z6euOz*l$4(B_J)N!m82h;C7>@aoEleI!Z`&z~R5-np_3Wf_%v!BK-tO;V}t^w!r)jm_rH;%Lw zeCkFD_&{^cCZ6i8RMKG%V9rh1nsnL=ytH={6i5_*vlkJm0~DEk7MLUZ*$H`|MF_~2 z4Oynv_Fp+HUazbjA!_!J(^MB~fJ6qz4}aQ(jh-?_dzNP4_x`n~zZ2WhA5t|kd7UgF z-SkQK9GOz`fW2Y&N$;Tm7LAHjpfKWe_EhUuFx6aZveM+5e997SlwEim6D!+<4}&FI4vy6Uxa>yihxP%mgd+!04Th4*PI8wisU77XjM6!nknX zD1hy4__-z(E>tMub~8yZT+vTC`FDVm7Qh5Z13pBK?1NM2Rt*5J!+lK8@f5GS2ovY$ zn%J+W%J5J!JtMA>W*Z@MfhCHf96>_X-Fe;95Sw=3g+8u~war3N;>3o^$`RD*JJKR) z+>vw)&-_(tIG)~o5^@N{cII5if9k#%-`~e2lGy)H_;5Ir6|D4U=IQsg48m=`&v*Hy zg3as7#=xE(HIF9CEs$LwNFPf z1nxjte2z{j^Vj89h}-UxN{mlfA1lnCa9k!)R0%?oGSca+_QKAPV$lD8pR6J#v-XbP zfh3N1JrLh3W4$BcK6?Zr=Koa6;tLGzB6w=Q z*9NJ5Z`J#4C-@Yd%30DnM)rRyWs};-1$A(@=VvI9SqF#u^MmrgGXC#QnNoE7qhE_O zK%0DrQ=Sx#k=*hLSi}1POLe&1akuIY*Ptwtr1_-;2aiRa6+DNJn$731k12sN`4Q>S z248ZUPJdof(;R$OAftA2q*4<(xc_@QG@4vkwbYzwQWwfUfoZiXRlJYPdvjzY_NKiL zNuMUGSZWPK6=vEJA)h6K<&x=*18eBC zoe8^MV>CW%*J*MX;WDL=G!lVmIhUtzWxRz=3iAdP58Q4RS(pP0B=Ka4bc122P=5xL zY!M;XvXmewjtNSH+K5+Q0g869^qf<+Ibmv91usQ*#LoUL<9U~cpMPaxNgj#$!@DKp zWUm03>D7S-E36iYX`|Oy+{j;Rmu43(d=<6=#tH&na+DZr*iN{&N@r^^ZcWa=ErP8V z8wljmh7Ht#rxwt?)Hz>1RL1CC3^JFNrosYdd%vUxk$3KKOnNSfWbx==oE@LE;*STh zeqIQGL+zV;eZ&O+MefqT>jpds)+36L!;jbRoBmHAQMu3hPI3cZuE;@;I2#6YvH%s@ znz#MA3C}Xp9QgkN)K&7nQmrzXa0plx3SVvYRx_Sl<=`o-cW>S!JrhO;ys>(L8^ip#ys=OfY?gHQ(J2eEggHVP%MVDUzcSBZn zCgX^cT!i1NNhuS*)NIdC%~yw&lSW~%*$vD5IG6B2Ka&+sGCZL?7HYN|a`==A zeXpB{Yyh1#JwG)05njQLfO?d(5A(iwBl=X((AddLaeTW;p&x={A{2uVpzV1A-ui9W zuCg$lrDPD9-*vp)E(gHp=3BIuEy07u$+^v@;3APfWYj|CVkmDvR ze4nVRe`z1j)TZ#-Qscdc+(g-<;1sZN)5%A{4OZ$jjt`8qp~Ittq2r*UZ^B0?+u0m9 zOcjMoq{~6BpI#);vjLBJbkcj>M^5LwU-!A%S039pj7S}&K5rbEg^A_C+gz?c<>uI+fH*gMleHUhXo!O+=oR03Yfs* z^l*+TD7-K%1d2Jv?nTZ}5xXKi*g%zhI~JZGEgW(--A`Z{tx36DqM@zy!l#*}WG;p+ z4)f?+G)_GYf|ESrSV&8Yijl^u{HET}gT&}0bmFKkdl2h4D1_OtkDeV){rz6joN#Er zpkR9*5cwJdi7w(NR{B!u4l%e;a0~&VyQf+nu!g{5DMQ(OZlGiVJrG&IpcwRUn0Lu7 z{jaSbmcLyNlT_J_JEsshvDA@fuqcAdrIaVO*ngOEUNa`Tf27e1`k*?6OgTG{m0jWx0 zbM(l3cpAkSi}xoI_Jgbzg)xablwJH+%NL05(bAD*Vn+SxgUd0hd2VINrNN zZn9YAiU-KJG*?>N(%HFy>(0Ssa9^WHkt}@J!IyXJLc^8~lVG1&r=g7JkD7SAyv<#P z^D4ABSwdI|(Epb-hKH2da|TnRW4W%|C3B;BXqW$+H7&}|yx%k)fC*w6+pYrED#Y$e z3TnBvyMKQHi|v6IKxUW{-(ViY@W`>UKTC5E2$%!*K_RbaUK(2a_^H6Fy@|V_bzXYs z)0G9(&}$SYZ_MD_6d?Np%3H)9O)Jn^1C<_*f_%hkO;}Mp4p^%yFwbMrv=>Q2Lw%Wl z`eje@*D=j;l6b0+i?AexXVT8&(k_^|&1VqnqhX{IXepn+*WRY-(N@j<8z=iGPoSUh^}1B6yUG(IVC|;B@GdtHZ?Tj8>*-8f|#Y* z&bq=l+jgO#`z_{TEUgSS`hdU0HezWcDQkjoE)f|nu(DxGqJEoA7c9r`Op%|w4hU$b z1d@eY??||uObMbr-Dzk*R-V8yb_Pkn`Sqiqx6y#h127wh`!XFUuA|wY@({!LRomUc ze3_E4mk^j{*8i0A`r1{bzcFvqoHx)09es1nd-RVPrE<91wQAvAcI2*JjXSXPBD@7A z#n}ELx9>Z1$t3T!PkzU922K?z0kE<~_-;?c*CgV*o~)ANVlfZ~#F7@887C6?uOVOn z(kNS$_ttSkxcp#U!CFTV@$r+(;q?R4G{Mx^D_|;`%E*6lc2sRkEK~0I^?^8Tv2wWy zhc(AJ_r$2l{ciQxb@bWv*>*=Z#BGnmvtI*}1s zu&A8f$YV$@UlA9)JWHGP;8K?_Z#URV`pYtB>B^?}i*I%m;Sz2iM|f|~cS#T^2x#TS z1-H!Bdg-b(dnlI}PxXc4)0UJbG0ZgkXTJNCTye`6IReywfeE}Ax!So;m|Bh3^!xl0 ze&^SX?rP=D!EmF|R1QDur5cNsstPAGwcTMO9ydd}$J-M*W?=t;Cwv}*siAS%8YP-O=k}N7Vx1#Z{4(Ag_oXc!CGC6=hD{w&NHLdlVMC3>TTU5Lg zl?+a&B96gdFKx2b=Bp1Ug+EsxpYDv02nmyK54NAsNp}m!GdlY&fEmuDSN%UED2+UV zFZX17KeYD1;B8)?Ck%f~4H%vuvK%%xuM{}7nSt?r!%~Qw_>kDt(jZoZlZUVEzL zcx6^&I#ZV?9L+PEJdtpjEw$0#uz(nQD{znpLIyRl)B+(aUmQc-ZVq1nIU+d=Do8#} zGQ94vdl9yE&;rj^+)?G3vB~RtA7sc_rIP!ahZy%$qU1wqe8GSp%`pslm}Fpw1UN(+ zyKKlkT1ICb6pQ<$94P!X^N`Ptr2OZy(T=3G8ujmJhg5YR&P!w68yp{T9a0r6{vAb` z<~$Vn1q(1xCm-whoDB4R)5=n(`a>?T!)9WzRM%t@+knpMan>j6#qwq6|FEZ2?aR0G6rAs=%00o-IvA72T6hZdk zcuBoZ(QoYt=?q$1nDHOE_0J}gsl3aGz?0>nQFX$gPK=bxi$23+()v#eUxUKV#ddgd zS$~1^<#qF6qAb$$Is%D+#|VG%s^x#fv>Qy-SC5f8DEO6p&Kq_B==4FaW5lN&4GCqs z&epTZ;(Am@b4%i zhN2wm3F{?JCrBBlLI$CM$nLtmxBhT}n?OF1{DDG^T=^@tpJZTAROUPM*m*kVU=}qJ zwP-QL5lkce>>eC$`IOl8_&g|An(V)wFjw3ITmc-)RC9m6B77d$5ezk68tcdLdc7(A z5axKw^_-aUq4BW+p(Vk?ad)NQV-~?nm2xI9R?McQ^APkp`(iYa*$OTjdQxNYO5&Y7 zT6Q|No;FzTi~VcnFQ>G8Z6?Nw&+~Kj^iN@j`KgfEl82^u@Z1el=SclBsgT>@NrH7% zr+e@~>+Nf`*=Q!v$2A^Kz>oUS7z1cVI_39UoqEyN&u3Dv*s0A(p9gccGThPgPOOWS zkn>7Gh*8s*m0l;8eF;Ub3=)ef6x}=3d|sNo(Zl{dfQIPju{_c0>3z;drTDvBZORJR%G_$6njNb&xJq%5ySv&;av$d@}9Z`Nvv~Jj0*o64l~} zbMjfIRuLuIrPY}60XE_~rupTU#$lyQ^ThzFf-v+EK)?U&IBDPhL%-h%{XjDDYu*~| z{u4YS->zZlhHu5yMh^Q9J^zkQ7Riab+?D;S5Ar><%c|#{t~RBc(fjHZ@$JXqcm@vZ z8Yo=xYBoo*KydZLKCwzkkY$s=tRrWq{@l6E__Kl~|7E77+hMo_2%I$QJ;Z2vWfRGy+my1&bku$`sn#26O_`qJM! z>9#GiX9`9&IM^<&2JTd@q9gIw?)r~`Mz?4Rh8!MUF;H)Q7Ar6QZ$}h60{8!}TD^1< zRecZj1882%c(ST@&wo6bIe&PXETotT-&j{Jce)(#ZITb7pOi^PX?TfDs1L+9o^T?v z%O%ko)N;2I*#PyZMWhm}QHEbJLkX$&$sW%e?_j;tIGdDo^uDkXdLl&jiWm!VgZa>D zqG-HV8Ag(@ukifVkZ_3gq1Cxme_*+L*qKT=L?YkL7Fk>Yh`gbx5KwC;vPm&bZL`zI zHMu!wOm95#4{QY7JKn0bEA>Y_2pDg!B0>a&!e!Mm#RZJ*5nam4L{3Hzwkraknu|w=Q z8LX+vs9wL}HJ!$@ZXy?bLvH>hqXCx`mL2O6S1~@0ghOBT3eBK$dY?~5ya$~=7)i2c z`z#%AqqDS5+$pN{#rfe)w$&}tTi>*ebzv}u{|m2+M0IYaJ|8urxdw}Beix3+HE=oJ zD$LM~vzMz}2~gowsj9dB)gLR2Gu}Ly?@qBMUSuwwEh~YxSj0^3Oy)~4p+SA6oIx6_ zC=@n}lH*K()?#h`{LM3b9D_mzkHJ3F;HG=)Y4s2PPAu1E^ppCkk+WuY6)Iq zY!;li!2U#{R{$*Y1Y>^lcgO}8P|fo^JSvIOuCz?%$8^mly8h9md9Ynb7+`a6d)5B>6HZkm7w%&62ccTV z0NFBUsqs!MwaJ7|li~HB@s^hiH&n6XInlq@i=7)UhX1scbqss>F_Z4sWOy_Pg}(_1 zKmC{jLZMiI_5a@_y?L?|8i3N*$Ogs%U2@SV@6(69)4%P(y13alyv%Js1=@keeB_O6 zM68hs>hVnmzDqtYX_B`Q@vnuKGH0t>B#7u4&pr+aE6Sc)aM6xiJRflR9Xi#p64&nw z*{z^1o~}CMdb$4dN!_+)hKSy$&XujvzpK=1EC+;Rh=@Q--)g3&^;Q57WxlTGF;jSG zKHRQ3AA2IUXkxCgPw}f2Pk4$7QAOur7Xhu#XS*!9lI79RzHu7Mj~6^YSF}6L_5cXm zdZ{H{I#}5og`s=R-wHu7_vvu1yc2+3{KSSv93LJQ?Fl?x8}1_bFVy#M6DsF6?}uxJ zGDs^R4Mf5XjN!uD%nKFYNmlRA{xXn<%+MPX5qgJM`BXxRj4nh~#u2`(jC5lyNlA1u z7KmT&3_mX5xRYoh`66nV%!W}JF7*{(v6EUaS4U@PUZruD`GpvO-K#$y9V7f{lCe*p zc_OyIJ*?MhCqjOvvE7zV4DJeD=j*{wQLL%l$H3+$#w%{c9fd0RCMlEwZT&gRfTEI+ zh{^Rb2J_>HG5pV*`pMydJz&2VcHjYE)VFTGNh(9br4KvWszRN`ppq9M^pjd{a-ID_ z7I=6(ImuGlHr-U)6bI#WttgY~#(_+MafYE-sXxkT-K|Wgvl=iVO*-}%uncHma z@qPzCj`s0`^5#zRwaiCAN4<%IQ_<+W=JuflV)SUF?xF(smfbl$PhNy8sRjmt!jQbX z97E;KUsh6|oUH?SfCq-Q~vjLrC6#)dS{AZ(J(!@$K{6wg^T76qFw7WGKxFYy z5eVc-Gkn{2n+C;y?8CQ8U-gkK6Wf$?X0*dnLCM;m9;nR6kG^qUV^B2^zXKUmN%HSz z%p^XD)@O0sW4%fmUFxp=WIN|z_TAhqUzSUxvmcTBPdd#joJsLbhmO$0f7l?pA1pMuazS3SZtEoPuZbX4?ln%Agdyv4K(5 zns9>M!>O`on#^}47PZ@#KM+l*zCeq5x)P*OjXum?(~#}&N%8dcEyqCYQfZF!M6SQ_ z3YP=y=wvfPTCK@xqI@1RM~io z6i^EiVme8TT&jpHKHl!E4 zxu3WIELoV}S4uTg>QXaH0?UI{c|?cDg13zlwnAY6z?z@NnNiWCo9 z`H$8CQSKbYkQ5P*rYRN7X2k7J0ms9n-yzVU(1SAZ>oZQsRWCI25$(_K(Ph7o={5K6 zk(&%#96wQ86V+l=64<~L{E6)I(`d3W@EQ$@g2x;n8ZuSzqiI7@dw&+ic!`2UlW@q! zrz@3CS3a7e?wtnt$QLQDF)zzxpl#Kgho7q|9^qK!z?`9|bs=5BAZOLw#?U}=(o%An zwC|x3ApcbSmmP%WI3b%ss?u0Cb)|2EqdKfD!8)L+U{}PGUJI2tD5Qpk4(SoeqecuS zOktc(9l$WmQ9k#>6RYZtqm!d8T(K547(^B?r+0Irv}lF7nv$dacX?^^_xe+UJmrlg zVHvp0XL!Ayo5DhYeWny41zekx_F6t+@IFjUfFiyO`mKZz5Pi*A7!DJkYo=)}-~>UN zpo!CcJW$2^0_YL$^4!uuvRI&=Z+LvX^>FsvLmO){?0nvU;Qm)Yt^##UGYpICx`kL; zQL4BbJY@Y93a^C~FH!nE7Hk=|EU-C5a-oRuVhDOK9#`9ADL)>Ep8QAN4IZQqj@An$ zI9DK+M5@`GyDXXEhDnxP1tw8UbVf_c3(!Bq%~1)r98ssJp69BUYu6Db3HlBSJ|$Vb z&fu)*kLfcg{-WGYCXh1G+0wzr#+0q&FTZZA%b?{It*1aq%-6@_mKk|r8jYLVt7Lm- zT>cVfd?R_fJ%=f3@N&oT*HeC}1`*eXR1#hO* z2dzCZL^alKxmeTp_294yz5*1wI3hc$7+e>Su3ccK>IY}DRAY=XEe^sqDm5j+@$!Uc zF4vm+C5=|0oCCAjFolY1xxy2pqAqTW^yxH0Jx7gKT*`G-A=hds2ur9PKIEu})4`jV z=1fp>;Rr@##o3h}^qxOI_UMG74H_bzz0$Q;a_b z_9)`0HVlu+IB60GzI8Z^r}zw{39Uuv$N0TI9>vJx zpSBkI17uZl6NoZR!aA11d+1R)dC+Si>XUy%;(_x5%&rG=IH2l zpSOJK+$q~)x=d@h1i2+pGbL~KHgJx(e|CE85Tgwp{9z^=!UaiXDW&bgyChn7oCq{2 zrE6CAP-Dn@z?R|Y)A`##Gly{gVOSB1+p z+%zhwbaH)uGr6@+M2M+9z^;1j&f2fE9KxPOb zkZZ=ERskj5g%<84agyc=7#W8fe%wlB+GsgJNz&ASDmhC2TR{)dz@W4_r4XW-HvUf$ zJj&>S2K%+&M`lSY^>-na9gMgC^a=EOH9m+Izb+&)g$#e^)&AoG-2#cH^lVH~ps!BpoZ~|EJpCr=lP$_pqO_#0lFLL9nq6s6i-1M znDmjq{yyqDteVr&)wK#}uNJC28jh1&^@GyQ+=Oz(@?mC;QFC5aFnB=S5Re}G&Kgsp z$jg{Wen7Uhe2EhD5sNbG|0!H31ibtM+=dk~`!)QHKi6_F>Jo9Ln$2(93RwNynrpc& zz+5>+^p?O-7^$4ekwyXZyOcgXLFIPK1q8_fO2t(lHIHy<>7Wvpqh& z&1%kV4KqCUY%H0I_1~!0WEc6|EgmR$fqN+TEe7dey8TKo4eao4zEt(@R3Wn=TmKH9 zP?6hmHPq_778`!26gUBt0Q-vg-Gq& z>}st-)U&BE`};16{Rgu3@`snE9yx67HW!ZK3~e>ZqTl~xFNM~c!Y@vNEXWdoBB zA4>m>B>xq^M*H<{o*vtNJp)XWdtm)Ckl}F2royiAM*e0U>~gzO5T0DgOcpZU_BC*o z6vbOVo_E9IH;I~KU=v4FZ!MssSBftoCrd|c)NT82H7`UXo6M?RziFbK<34$2k#K)} zk|IKD99IXj_^OoD??5!7oCbzmZ@JX{0t5-oD`=U;FjBTE%(NgO^vHKi^fRi^pfbg! zRg^nUa$%SAZ=_)DFQG~`j7>2|w%m|*5w(Vy;QeVp4<_Wf3p4T_O>05xVl+CQ_7$T# zC?e%ivPK+eC-K`IgWH;H=@ruNfM`Ry%6|8Qu-7GOXMeimdUPIXv-!!KSuH+|`sFeV z#rNdUU6$Kkf&=qM(1rlHLO50BU2m|l@EhhDAq?jUAAl5LMTzWWNfsEte?=n? zQvMFcG-_h9L*u80nUy^=iG`fod3$CZI%~>=h~IB57dB zzI2iI!GKQhFa;aP01e}KwET1+*T+(V4)|7@Q-Q?cJG000)?CFtGUECPTee>Y-6*HG z;T$wurlopQfIm)B;ybAUBcRgxLjSg{lwI(n7yYUb4dK%t|H)Kf+<+F4@bKJg#4ygP z-;R`!_R+KZw~jYYu}0|D-LY_;>rp0WvFfk?VV7olsb@<`nTwkq6T^`HWq{}?Y-l;ypliRl2q4%F%7N~`bp zGU5p$%^`Fj&-C=3(pW`<#qiqa+uraGKItq?XS3%!{#@A~-D)%sO^Rw$YG72z@C1N^ z`kpYdx-A}wm7?r4Up(OKUwxGTIRn&N+S~Hs2RY9^4f;Vi2q>^}(&wvOnYX?t9yV3z z4aU5mLru|u3?(VM=i{}F9M3&2_*!Fh5@7#SES%W|M>L%9LfYL)l1=W=Be(h^GrFwiC2GEq7HgD#9ZJ!6*_cU1 zVN~1Qf<}CL5=J<}(z#;=fc@T}q)6BZ=`^!WS-9 z&b__gD5Shzjj2@RvMQ*^i*I#<>MT`sf0H5H03+r|q9vryc|FN=Uw<~L7G?J8d2G0P zfEVt%7?fo*eh~`ekNr+>oUG$L0XW`;+aM8*3cYlpqeik;j7`*VqTWnaj!u@}{31Is zu;2~L$KE-OX3=3l4#lcHKGaR-rCa64%OcNns&S1~GP^DW?0|A##>4hRo= zo+kL#^)C~<*FIjTwlkjaQH2(Ad_nMy^X@gYiN&V#or!47)7blgEwKwcRmlBj+AdOl z;94$q0FJ$U=~&S_JU0B|-e@jtSjv$kslW;3cbo=gFkrha+7+OpuwXerk|XcEb}$IY zV0yR$D+#VxIj{_2X^GiEx%0TPU*t3_$KpTHOrd6$cnL>M+<7(phvuYD!MQe{E=#)I zc5uoBjFyF>0teN4p^GhA3f7FX) ze0;r#(*6|v+37i94FFg-YFh9^J)M3Nq{0f&F|R#6icM-=AUfGoN1~hoovcRYE6{QOAhM{Pn+b|?fM<0GoiIoJ zSX(s+B*!`!ZOCDgHwmL|21wwP%Z}$uIlXlN{dxlbFTTzyEUqxywz#_ocMIPWL(8ea^k#eaT~r+Ei`!zt)^%G=-l_&nAME|Ml+; zr#HP+%KYUH-Jh*|clWnN{TVl%%4Q)48~LqJHAI^@1#Mj#fo_T;>X>I1Bq_2bMR`kL z@dJ9Oq|sb0MfP~KU0j1K;InMdSPe`WE$N9QdG0rLG(vDnG{UVg9QjD>$_c=l2Un$n zr?Nk;%|wmXH%5Q{2QV(qY^{5tic{t2Il+djxudg}JKsXo?cyDSA0K0Uh(fjbh*Wxh z)SM`mS7mqxPTZ)dbRw%hOas0I>Q=g+Zow7IgWia2(Mh~9i zT%{!}zn+8EgQM({Y^9)H5MwU!@K|04iWSRmWYF3S>!e_UxnD$SZ)Q~${lhaEX_#~9 zTtc&dg~4WI@hl1)G%BdOF>!Z)EaS1zlwe&^Z4iFlhd{V0eoTPfGnU6f8EABRy7iitwK2YD>ov>%6)-tvPL-hkPC z!587xNb8BKodNX)#9Zgh7n~!YBgHgQ*+en;k-AS>rJ#rExjT2-v?+Nf3H* z&+gdo(*fi*qySsMe+lRsFM#gECM}gjgs@r#FbJ}gEYPbph4#fg`r5AygM>%wlWGyh z$LYK)Bb|f7r4mwoG?5@dR2S779F#FU+y(qkiQw3b#e>TE4|af|+)ltAz@v$ft{OZv zYl@}B0z%293SD;NI^evW3}PWBb+?BFRp<*t?NA_UgzJl~B8}hhVv(v+>-T=a(F7?_ z(BcEdK~ab@o0(!wpyXoTZtUXSYz$7A96ue!;YdJ3x1xPMyjQNt|qPvaG^s58D z(J2*F_kdwP1T}OgJ-gh=+KhB(hq*2bS|JrOm&}SB;3ce2Q6~pVYUueWnd zXwf#hq6A+XyhdofH5`~o)2aEk{5i~4jFbdj#jgJzxkz!D;`Daz4Px<)2dRx zQOP;`oB3kG?+jiSYC05X1T2yI!T?F2jssi?IPpBpg2xYpxz`bJ8LoooncL-3*i8N0 z`f^pB=Ke2N(tVnxx00)*v3ws*-9U&s#HEt?h_oRYBPk!Hcj=`dyv42~Ryb$fzFhm% z+g!Y%n?i{|Lrm$!BCvHH0Oz4!dOV^H=1kp&muhi%toZ<9OwQHV-t}m@PkV}nP_~Y- z5=>L&8svkqWPPC+#~-O*qL9D=)B%7coP$b<&%A=oD{;>Qoy*ax>h%u9*6q#NnR^#S z(NC<@PmxD`aU{Z?Jz6fXTB>0YAhF;vprsAoHAb$VjJt2xBnO`$b4JaYD!Kjn_QsJz zZ*jTVD+y;4Ky5H_<>I}%=a!m3R0P64FNef$UC^HtvW^Vlnj?CHK_3=0Woz+<`x zO#WR(ByBb3%w0c5Qq-0_BG>!Lz>F4~(>-{=s2T!4GVwfqO0@%e*~ER8KU9$rLuQNm z3e3%d^-SyhwPwZ(Fi@^7-ht7+_b_^K| zEws5p>UPO$hD_c48HM{uxwX7N5?UZ-Ay3U8m2SxXAZpSA7E#3l9u@z(j#ENDPpJq7 zvKw+P!m5kiQlW!%!0h{ueqo)tk6u*MLLLa-26DxuRsh48EUKq1Y9SBM1AN9W?}dy% zkdRJDrfTHO#Cvx&*@)qpLZ5PLOadGgIIQWf>NC-{U`B?4nl$~1G<66PrdV6VTHm&q zSOzUEF>@XPHhPF#Il(Y0XW=NGs?%N{|0KVWyP;lsUOtS4q!bv-ej>L;rW*nro!~L8$U)^S zNHxPc!twEXSlmVh+$=1paDh86Hb*q1x}p7;*M@&m zx4w1njM#Y-65mT=!aStd1;N@1ubX0!Bv6Cz5DR)cB6$?25I*)1G1D)qBa?#I!BAjSOJ9G4_4t$5_p*8;}(0)%&KdBHs-kJeL5ucHFL$+-* zg^h>7O!_^ykPM&ToScrdJHB@mn)(O^hKbGZ)R+N2*EK!^v;>}LqkQOWbyCoK$ry=< zw{u0u%0e7+I0lB%W*pG@eKmPm(HXBZh#_kB+gq@#3bdmQbC+?n*~30S($u{{NG5B5 zr{79v;k1KvXzg${?4un%QFe_OKv$zvyrD;nnn!+Pgh5L2Ve1C`Ftp(#yp8rNmRAoz z&~!3ooDpJ`5QtoHEAsse69x!+ZHHFB{0O&cEh`w z@h{i)kH9@wzr{rQ*86OBh}&UBy;;oc_D{l&<@|GlOE}K$#oC)V^<;Z^6fwWkQ^CaN z&-?_hPKP?rOhNL=r3-Zi!Y1~_usbml5yetLgPeE>fTosQkh9|6ORT=jJL>1`!ofwD z6m;G7$7tf#xfit8Ay1@kGk|}AzAYx0(T9q?E;uoMHOt_!Tg=DW5bNk|ppHa;>IIN_p}_+=48rShypXS@ zzqr7Hc~LqYr5%4CYQllbJzB3da|+S#u+-d=u@3gk3dI0LN=}>U;979!{FtuSn3z2@ zo@hvlVc8oRayFKGCzv=IEDNPr4%H?vs-@0u-6A>cRSi4k9HFdOQ$0>BbxM0oiTm!f zycyg%s%W##M$#{E2d(Zg`{tW`_YnID_Yfn0CA_G*^lkjiZw#yuJK>ONzX1;AhCN*4cjACGdHksL~|E5&G;pACy?ER^l zM9{riJR6jejGc)?LPaE z{=Ww<+|fbsbD~rsqY2&d(K=K-HlipEta7<|8PZi|?jH?Xc}$bXc?pra7m=lp300eV5XTU8)#Og6jx1!4kUMtexZ9`9p1 zeT<@glfW(WrPi3b&blCX>GsBT39Wnl>i2rkWSP@(W_~d1MK~aSYASwlXM@g`#y0l! zCdJ489!dr=Ef5T(zkd(_Z;S2{m;ib@Ps*U?x(aT=beImd#sy1xnq2cFc6c-y1 zyOa(zmWfVPfp!(5LoR@UF}${oN(=mlY z6yN4+zqb3d4O}tk|E{&$v$W?&ek!vClHSjwnJv9L#z3P1zP}G+@YM9XF{AejuO;_4 z>X$DGb_NCA(_{S6f>~;~!#B+P{ME-7A&8_n>qoE+%{PSC5otWd@a{W}QhSY<=s^U+ z1HxFRU!nE|-E+0h3bzt-iyJrCl$>&O$m*GPW&U)bxdJGd&s|?E`;NX`Gjh2*P@rB- zPUN7z1+LQm7~U$Ra)^UAjBA&K!8T2Lq>2ub_#~puMI7`xnc;#jfJxoy_G%4RMO9n+ z!xJ+Iq^g?-jEqU?eSr{T+N+xpI+Tp^93a9d7c5_}ul&?*UZKN^T{vOXivRFbL89sB8 zDPM3XH>gxdfcQNMYo^4&T%>vgiqFuwIb0s(4_C5g9~MH%^WNd?%TYj4zswf6@B-Ky zrDV@Q`I+KZsTgElF-|UQyl%+Vk6Fx5bxOL^j(LN`w^piG`A{v54pT@q3+XR7w7>iq z*P;Ied{ug=%3#Qz^s#OI?Gr55G_(r_T#q)`v7uq$TJ#1VUZr`gnqyQRT(_weB}}ew z%79@Xm!|((UB{-j%{>+}6$21A)rO_X=ta479SqjMIL2KW0Ch|T2jl}Io$cKhdd1^p zNN-<{_x0DLl5u3>w?8fu!X6jrf=J%LxD3e}T$^q~ZoYOvc_@kMi$+N|5JHB7?)B6; zp6RBF+&h``2guKoKx7|F5k>RpfAKm1oF^s#?7)jN9f*Z)_yAljj3t{y!XS*GHi)^f zT>N!VtJfb>42eO>DsWzFGg>_x7MXAUDh(_Y0pFK{-M{}bxAz20-QAfrZpN?o1xGpJ z4PctP$i?B-Se|*vKrLr>mb>~CwBLmcKk$fndS#D?)VH<&{(c^R7 zZAQbnkn_##2276`X%HEPtlgd#3d~$63Q?q5$R7Tp$O6h`m7;=wlUh1a?v(y@0HrwE z6Z)~{W=b%n2~4lv;o^38+9d>=ViR`H`VEcXIoe~EuHzsR#$H;VjbGZBb2?ruF|>*V ztUaA708d0q&c3a}`5Ww^faxUEohi_61Io@^DAXD;a`HNG@ECngA=I(BSS9F8iek=Z zSy-FFSO|n8!RaQeG=sw#gQin6u{_H4sJu7~b)6qKc(8s0P_c8k<4JW%(-e5kg1Qf& z-Pc;Rd+xHt&gh5`19?v&cnAq8>S@$_Iwb>2#M2HQPezN+`aHr3#f9Ez2jV<*PNLL2 zyPqFFcE8K8iVEO<*vNxJ(TKC7mpgf&CaOya)(oZ3T0zpN3(v94P0ZiHVylI)>78{J z@T0qu#h8|BfJ)c~SpWxbNo0gVG`6D?5v<}t0KAddMSL7j{Aj3>Hi=4}Oqn(5wU><% zGYn22W=9#ppM=MOS;w2!O($=Q4uZ?++lGx-QBWRmtpQII5M{x5yfg+Y#n4cpF6k{X zQSiV#*rFGbU2TtC(ZPyXA$uq3B|qu*x+2U)$fSJN#+YrLBVnBh*i&_)e8FJc%*9 z9V4?u;e|sI=jwki1NGTd2Qr^_OiYJVK3x;+l9qB2MPo%Z?uEUDBoSUj! z437wc1%~;SF;I4+*EJ9V$1b3fhLxs{Rda@wZP|kEA`29_Z_LfwIC@^Pl- zW5bY5g<_Y6$i-0XzHTh1Dt*%kNVQL-_laro#S-y1oE7TkPM}yC#={$jXb)b{0G^g(JWu|IGM{V_Ggtq zaU;f$NPjkH&#s7CF=I{~Z6wV`RSb7L@Dytd4MS2RX@nKfau%OjlybS5ZPYobPDCd3 zYXMn8YhfaCVYy}*x@p(+!N%4tr)c8=3_Zp?jM$GaQcl`n+3pOf z#@Ky1o%E*XIoC`RC@K-1kvjb=!BFDZDTmP89?qMih-Fq3(*o(wvGwX7B8uW zFuiMe_nsBPmLZomsvmy@h{%;CVM%V8(@Uv0M6K!%)2L@m%jIN{{_PR&DInewKxUa;?MVQk!XII(EC&f;H zrss<}9?Z4%E)!0HD4b*c6+?zr;YlSi>#ZtI?jrb}Ca?@!-$q$)KsmbMe_H7O8Sd;X zHY54=Q+e$W?{D&J(mJwfayAO8Hw^&CqUr({hAGL_MyEd?fw;ArUoS)}c}JCZ@cMKs z{v83z?316H6&FT>QI}aK-UCu=cceIR;P47tlnUT_gj=K)*D$>)aEt2mDhbWQnz)ss zY*fL^eyDK#?UBar<_`KKp2sOSC~cja*zh&MDTMVp?F&Nx&~r-!LYzCJ={H^va61AmsQ#gQ-UovbXbO%=sbAPwgjbCw`&8n*3uEix$qBmCm7%`-QJ) z0ZT=7OV8ti0f|rP%DmgjX*&@LBm@5oMT6Wqb+%Obldw9N?xWeCTfW}@HHd_)VoM}FJ~b}79Kgbx=) zcss{spX{2@PQ#UR>Vz$*!tf7{z%(7b5WZnc>E|PX_Hj3t;(x1T{Ebl=DKQ{%yw3U| zQf^j|ld8NG8iAcn0kLT{iD+`7vl^HO^!vT|2#iZC2y9;}DAmn{dfT3Su#$a)%bvBb zhm&M_Ht`mR&I3|rVUuL!fD4(B-+gq4$b=pYgJsjj2sfFFKLLdg+(5e^T1bg2+SSqu<%}Tks_-SIOG>7st#STI^ngH;5c#U?OdCKhRz`%=y1g6EKRL7WhzsxqknlOXTDPuYq}>A%6e8SV zB~av#*WEQ~2)In6dIx<$9S=7ylt$~Y^Bn686$dim;lP2jW)>8D`IQ3tj}R?)kT|%l z#+K<*k~!`PAYjRY8im)5!Hr}ySlly5#Sg`!-|T}T*s0lqCMyevEG*|+6ybxVMuxZQ zYX&RnH;&yVA17vs*ce{YxqHDeh{V%d6ijv5!85-1ECCErT{0Tb7Yqkve!(w<&TdUm zs*|TQ6N$=%L&o$nvIo3&x6$z$I7pOWg+pyYxvl)mxaw>Cy<^VsU)0a>nIR9Pg?E%8 zf_{5INi4@wbbp&!q}1hIh7`ftlf>ENfIqAetB1o?FYMc4`yCYv~Q|;bLb>+sktqe z;VKK0 zsgeh3?;pGxWrayrIs?b*!14N~B87ksn&$FLsyBmvKQ;I|!ROE(Q}6rij*+1gwcgCH zPo}^@tHzVAvlkxh9uuPt4Y`@lZlx33>MSu|=aZn;0|PuBA!}U?li8aXI+meI@Z|!4 zU5oEheLsSkfq*&w0ovm=9@y5JwYE|pAXEHC|R~QH(d6cX0 zl!aWO+Ro}yQ%Eh}ubK0G1G*w!)!fNhTkDHn;l9lVa4?QFlm2 zLL5|lf^#F&h;slr1h%?nS^<1a()6ZZ0$kuU3?qe-Ja(`?R5%)UV3y6bvVjmw`$Vv# z(Lp7Y8T0SBaIy@cw?AcvxLKxOYCyTFM_-1>pPE}WcJ4uj)`*+s`S)t0D&}Z)U!3qM zmeWdNGw?Jg2f$HX>7og?#HDe1p83rBk%H+8xeDbe>vC;1#B>O1ERAylPo|bv z3%#qw6ATIx6BipEKn%nC)OnZtUR^J#+g3uljSxPF=8#E5-YG)~`xOBWS0nz z(v+9gZ&~nUXJXFM>;CT9+ld`yGKL;D7GwNTy1H&$?(%%Qg9-s|gp6yCg@BjhWmwVr z)iXd=!97HFMxGdVALXSd{BD}U z>#ci3#yCRKH=)Y8;)#b;b$4JpdzH3bOgWd8`&A><_iz?YwxsqIu;!M^9Ov-ng_#fz zY{nKVK-qdS&ij0UU{+oEpav||RBCD(VBLmUL~|v``@d<5wxt3{B+0t*%EmZ&Gho{; z4EKjY#G+u80{f?gR74ZN`U@r>vfqg*#2_LTQ&VxOX7w&TtFB}${}BN4BUye+ot{;FfRD9InD`7uP|+no3B%&0#}8M3$81jSRw66OpnqO@{Ut9VD`bY z1&)NL{R>lI4#TBh8<1O!0XA@3M<~z{GX4!7j1k?Hl?>RTfsu2gYIkYjXjBo>3`7mP z%BC|0=TbD(nBor%k7tFhx$S8F=%QaFw2-*PwkhFpc71a7$V@i7$VusCROA?#l><~{ zA?(q=gkymAyLbzHD$c{K+^-Ef`lRW2x%S@&}$KvY_C6PDwOL$_0g6v+?!HJ z-yub32o%5TH5{3wgzlswN3)!JjS&VJn}3%uCArCv%6EdImJ@zx5)a?o4HbxT;B3Xi zM@R`_tMQf&@VmXs22EE`jG(KndOYZ)1-n!GG`8et-B=0^*=!MP{+L4~=Hh}&e?d|L zuNQK{R3)DE1(p|xbEA%BHi)O)+_moWWtt!3mkR(P5P5hyt%U9`o-^8+4>Dt{00IgQ zKSeS((N+q8ww>Z63jbQ0{AJJ@Gd61w%EO>-EvmEL&KU}okvYGczb<8A0owQfq3>Hy zAcfkNT95qV8jI&u!{b^*;1yMhzV9GNJz|<*0SmIDrpB%yr-*dK2d6i5xNhsBam8Ze zCATBtP2wVgjtk0q?`OKY^my{+2dupw`h&oOfIs{qiN$phZSlh5hV z4pURKAA`yel$OY5F)ZV2WKJ$0CKa7(eyX+wPZZ9Fn+6q;OGcZsD*j$mCK3q2r;tyL ztUFOC%&$O}Z==EW#zXkjvL94au6z4GN5EMSfe2@e zn$YLsKAq?LTNC)?JCgEBy_o1Db%-ibwnE%w0sb-(DhPB@p=3+4ZsL_<5+Qiwh;~Sx z9-WPCcnY^1?m)E}r6Q^oPWlc^G+#!!&p_jyQ!>ujXP%;$FcMCpXcFL(srsUSM8{5p z_+*9?I1FD5N9LUH^^U}TVX1dp+LsXv+S^J3A^AGRzqW?fcYm5vq4SH(4NQ5k;G+a3 z3vkm%r7_GIe861|mK)ANF{+U-KoH34(Vod6jpxpF7Gor*Q%w$d;;-@6|1lbZsxqV* zgrM=D3)pv;fRryF30~67G#xGE1twmW3vhjil2fuAaWY{`^GhG#yPN%1Sg1!Kpb>Fy zs#~G;ApGc>-*Q8*5Aon@34oU}C_rFm)d(2Q;9Tr)h(rltRjo==rr+}J!yZ}R1K}ui zJ^%Kbsakt}YjruunRV#UE%9D@xY}NH-XGDfvi}#%VS9sIMyReDII)>fK;^-c85L;M zE>8xgdEn6AX23&GYAek5bbM)kPScoT5c&mfKp!j5uv18Y*)Dpk>>L=g9o{IoU#m$h z?A9kO@V06}W;u$dT!frEoj}-ck0DIQN@tR*3jGEVIt$}_r+f0T7my%NgM<<>s{2yU zzM{T)z1*F3oXpXO$ureOe`@CQ9L2d!i3wI_kz@%#Xt?_m9Rlbv=&n@TpMIU*S1JGC zs+N6~n-F$bN!uGs-WOC&v$hgpCgi-};4Nx+H~Nxa&lDoFDS=){Y$idyeZ2ydSzttp zklF6cGW~jGhdypH#(v-~WD|ypsM=_{wXH}45XaH4#~-0s79Rn{H&)m>+9-lDZK z`q_^tB|znXt3sY&vPc8Da(s(XVPu8aflRair`HR|`|n2rYmM3ZJas;`bg*6Dq%0|a zVRA`|rlW}k?Tp(Ad$!y4Y`{p)_21OZ$A#_pg(r6a2=f26B*(4HeqP=?U;8@O{+7wk z5}||PeM|Dk;>gq_NNo~GKX^$1vTU?LSvWBF-!9@sC<#}RHB2&+A99`hqOYz24n#JO z7n5&u<4k)72D+>V()JJleE92W00y?h-|4dHdRprIHv)SA5~Hxkn;L-N09Ls_&HG@q z#OD>1vF1qFo$jniaLvuu`m=y7;NC%C+|q)rwraG58uh@p)-@TccOGi(T(qDKSVZva>)Iz|5oga*(6KU#Is`CB%x(!2TzbbNoF(0e(X`%u`BzxzF( zbYma71I8XSWK7N?7Dos4*3nxg2RFbD4V-0w7QOurlrT`n5h7^3p26Ni8fc z`x78)emQ8dQ3o;?^iDKSDsok=(qX=32^#!2?{8kg@+JjDeqhi2)oIoA#fM$0Wm&HL)C~A8+X-_K_Nhs_-oym|cM*T4QTi!y5`NEZ>v~dVkdl?ei5c#9>uy0h z$>DJ`1eS`@s7DgcHCJg#B@LtLyFRV^owCD(bE0w4I*s38$9(Qe;e#JWlPNO%%Od z#rx3-U+v;91FeX~_on-WUj)Ekf#L5o<*_*cfd|S`q(K+w{l%rxWGymarZb1yWbfoj zd}j*N(j*W&S?{OcGw{CYQ=XHdf4sy<82MwXB@x8o<_suj>$ZEdr=MG!0s7ed7*x@; z@COWXBIqg@-lpYM=|nh&|H~TP zv!qt!2^q@k(G>~sfFuzn5*XVS!yMa~by|;CLq>krS(#q~|J3XS%0pz1)^y>iimIjRO@8dffD$ly4;ZpVl&0Nti<{Fe0QSq$2_gpuJ0;|1C{c0)A~Xj_?uGLx%e|`dK$G`WT2q_ zST@^cQrvap5erMNwd>vXM^-Z+%rE|8=y}vxQ@L{g%WeTTHaK-is>`Rr%{M8X)0x+3 z4~VIcMf-DNw=LNq!<#P7Q2pkHY0{7$JINF+D|cJp5x{tK7hF)I{Nl zzwZ4F-l_Q~W)9qcD6B8L0fluJJ^@+!*yIabPQ76nouteR@#=s9f^m zcN<9huZsS`@W2-#z}S@>4LWBdax6P* zK{@R2v>po3rY)kX2?J;$FACc4%_y*c2{2zhE+D7XrS3J1&X%98Dzs%NT13Tq|G7kl zrInm-RINe=JZjPf2vYD=aivX=p-RWARe6nI1eHIrB<0HGk1pPq4teTXHkaWv)P#DQ zS!u))420-`04yZ7*UWs$P+h#g+Jen@7_^0e?k>t8q>feIw9@KC5uad}{Z8_{SQ)VS_!G3=!7zwSKrC4P; zH484FKg59?WyZKo&tCoH$G5HcBgDEDav>9T4Du!v13-+rr1$B-`%@k#y^Thml*`g* za%#!za^^Rq*T*sL#7pH@_)j*K_9Qst;y26$MyXGOeH}FL6p9D#$PH(UIe@;2fiIHR zx(BV5$V~K8cq*?rKAZm>ahw@x;bU*9OK%GauQ%JjWZ7JKbwxtEIai}lSzE0O=c140 zx_eQimHOx&(I;dYip#ly>Xibz#9IBkBcTJK2VRGakWn$fo)!?;`;uA@Qe}Y zl>Bcs8$CTU)KgT`RnH-`)HgwFs3I@4KNrK@eT{l=Pm;<}M@hxV9v(0B4!Z!u-4;!T@mvI&izMdgk9{ zI}0!C!R`o@yFw8ScU%Awj`6D{X*~Xc(ZJb5-DRJLXY<91eu{da49u9%VCBb8xtMT9ub!bFbf<2f|y>)ChuEYXbry(Q(A7oDo_jsV@2ABr#~qhQ6m)Z~ zHkb#A@k{i8OvquisS8!TNKUibP|#uEGqSJ!a{bYI8$ayNt*d@uMSBKNrD4i=oCNBn zu7_@|1;uy(>AKqP_!_L{cb4I&Ih$1{puil+5qLMa0P+$rRe@c>^ebqP3UL*7+U}K2 zqMq9R2{1zeOY(O^Y(Bansa#V*4!fzCYnGIe?;N~#N5e5-3IU2+pv4CFomi<2pEDCZ z5}`sbA~%o1Wqp=_$k3ZhZ?hW6;tqhUibx4(Jet9cdMks7muxB*Pa4R33B>3E<<4GS zfQCYK+uaZ49YR*q-e7o45uTzI-45osZoN|% zzIN+g@>M6Zy?uFK#GDyroFrC%UxYE@362=LA!PP zw*%dIk_TU}94fSW!J+V6cO%WVR0jTIh}91g+oi^7M~YHf=UtrppE`DtSfihoG0`T# zetNCh?a$FmiE_?+rlz_HH!u`%Sgau{n*%Htf6GI{o+{R#IPpWO z((&bTvV^{Z&k!0XamaoHAY}N%q@)V#bRXWx-Kuf0oPR!wjB$08N#&So4+`V0954w`sMpzJ9W`n#+lbp) zpWy6FZJE)N8)=D1`DxtsO|?L(^3!jOke$>F*XlI?z+bl8-C&^6kLo%`lpLCR_2$SH z<`-)n%4FJdW9@HxtN)b!N}Pno=)12y9vIDYja>gv+wWtcj-QZ?*B-(m^htux?MS3y zEVxLcQQ=2)9g09H?A*DNm%b6&R^UNF!2Lx5kU(HkhhH3_T&yGIxdo|Dt<*`=zr13 zSFN=M#)#(5Hkrr#?x&OM%w%Pw{DfSe=PTaeN3`G^$PF@zoPH9d{{tKLi{IjW@^8;o zafP_KpDdE;Z_71*q~{+cSKRJdYwR|!6XG(Xoc~>MEdm_^Rfa-E)$C8{yfj%~X+NFW zBc&M?PduG{=k3t;{{&hw%*8N87;c8|;Gk{?zCODn-zRW%RptsWcaU%Aa_zrMaUPb) z(kSHw#i0^R8p=&?!iCE{VSY|wBGYNO>OiRO34gr`Bf$&UAjYkbzMf;2f96m<7hHZN z2M1Ch5Tdtx!Xu~r2NQJ7|`fNjY4i4ojDLX1W(87^jc zmjy|k=S5OyAHD-%(QjS>1=u`00E&kl%FhxyJgJNiNiqei{9`uHU~1sEL3#6ySc*eJ zd&fh?AxT&)1x5Gv0jpo5(=d50TXZpU`3+$1K$lR7Y$_8`I|7`> z1*(r7;%yc$+yCUy5{fCbk*OOM+{<=@mkbJ=MFzEgz&e9giU?}ozy zQ|#^BZh^IW8+EZMfLZ?pR0BT?s@*}%3BSLze4Zfr!M(-hqFt)>*;Z1!@_{$B38>vp z5-R6mjh6wfTiV`QGC#mrH!8}>gITXGr7E-X4` zWBdTQr(diqxjV~=1b3g-*)tG_|8A+2N30-j__Q@|vkK2GQL2n!qZ;Wp-D;on#aGJl zbDE);B%MyA@=kDNM9EG>giDKYEo|N9g8gb>(%eE)51DjqhH8h+_u3@OlNj0c*E-;o z;k8-%bqTOrJ#sWWMRODj>qD&nSL=K9*Wm*XQW_Ty$QASGj*4pkrG~qBult8(+UQzyvYi}q$q32TL_Ss25 zkX`oFW~xcAg()cETlx~$nC+|otuR5GT;Bw>QpQB{o;!kHd>hcfHq?0m6yx3$FyNVK zA66Pb_9JUs9NjkqMU4H~;TbA+HGZub$G1q3GdYj?#^|7?4U#ZroB3P>f(~Y*3D?>KXtx)@i2aNF1YR0u9;IGK!-ffk_T2EdB!jc56* zx2G^jH{4LZr*AOB;{l-7&pM6YzH~_AGPpzR3<<6$OsRjKVjiKm#k2C<>V_7IVEY9P z>LyDgcgzKlMFQIg7y9#38eJiaKs!BJ0ALrWat0C$-F*k7JU{nE=xV`JKdNU+&DNxm zLuU`3lDV<~&b!E{EPj8Rp-5yl&4JjQRLk${u}&QHn8UzhCdjlU#$-U^Fc7oG(roT! zLWoHQFU=iwnd_`^Dc8v*!t`f9q)|E#)Kwxh>@m+Pr|j7V7DuM7JhazRxe%+|Jj zy&+H>lmaZ^p@S9G1J4#>8@~wL!32Usld$rBFqwL9J#d~h77ROnjycxqRa9#I1YmKm z6m#bxjUzDIl*IeEh+vd*Chz|#dI{XXT9@pF3O8MUG}%;6I-Y-#TVNW=K%Doy2Grqr z7lVhTeXizx%mm-m40(QY=XTF;Unjy5)^e0}rofZOeblCXEY8Tt-HRq%-CBVD=vlli zAo52N0dKXW@`0viaWkw}G-Q+A>j`3?)N=HDdiUZNud>qv64O7w{dV>%_5CT1IzOw- zB~f}4wY39X`8v~9BFsVYnk;o`a)<++4U86(9h)`_bQvorM)+#p4UibWHm68%Jdcp?!JxYHa&#OZA(=@G8^%2pBy_u%kh2vU^`@eCn z|1@WBhS#$HmpU6F)PknF_RpYsX@;j9QkZrGBFgYT-P?rjc8F-1blgaM@$UHbKVqhR(0Vn=J3$xY4njC+T|l%+Mwl|pr1_+a97_H!4Bap19@C*Nv_MO zz-ks;qk>L3Uv=TbMQ9GEN=sS8Bo3^ROs+CKAh7k4#oR=)$~3BW<9et+CQoC?DbFh$ zxl(@uLzOpcNp*YEAH6@1$zj8#$cQ+C#>IcH2l`PHte942XwCoDY1>;~pyILB?lsQk zV4=LZbd{Y%BD#-GDv`yE=lu1+>j~d`WR>9fB3|NjoSyrJU5a*JJ zjewj5QKoedKgETM>|uJ{fZGnqNn07U2{#F(aD7Nh}XXF7r4BX`A3mLOYy>+c=zsRn| zu15CmfoVBX%B8SVVH113$MG06zi-3a?qYnj2Nt)IT~rwWt}DkaR`SWy7(14L3!5$sr?eu~X7)>1dLVgjL&wn=%=d+EH^ zGbj}vb8b8YV#j!{{T@&6w|h9$7mEUm#wpGNk<#J|$R16hb(~Em=*bfE(A#qf1Zt_qe*e;g31#*HxE^uX;9Ek-O^J zTk0~AbKVu;hY1xKZ+7Da-oyN#ta-a<;5C$TaD@Ygo1yS!V9&rf=|66v&d1@N#9z{d zk>e5x;$v*S&Fd3GKr2Sh{jk>9+Ar|7{R(5t`|XS~m^?uWu#z(*R*zfh*ZKDJ-;@|= z5C=xKY~CrF<`P9(Lsg-rG5vB__c|bQz)u;tBSA+5Hs?9pa9a=>2$nCj>=Xirc_k9A zMzCly2?VepF-@IOFP) z%V0PDwuCCjpYA=uv!3+?{q}32I1j%!4&&h0mUl{0cRknGX9a0F!EP1K1*wm$q`EOb zYo_MJW8K}rX|w|gKb-7yYYk=IzaN86zDN2%n0!@PRnhMqmEAs?zr-~stOQuBlU_@h!mhw9jMSGH5V_pwdO=s1pt6; z393bZKr0_K1$RQf=s#z=p?;zh6w81AAnay73%w`MykJ!N*+=L;&Cn6^Gj(XjD2>6B zhAu4qiKJZ+@S*^G0RmXy@+{`74#^VM|A7ZZBOkV^l7bh}J9i1^UTdm<$F^?IoGw;8 zsQ@?1>1avSkwDTib*k&v$(+YrtSqrwjHZp<8`vxDnVf;txJKX=u}Em;%tZ{9qH>%2 z)UIL2*1kM13-iHHiaw0|&D8zT^y%jZRUgF$&u)&Zi4H(gQ0!+KU?y_k_S|$aJ_Dl> znD%A|PH&=o-+lYYFM^E(kORN#%?epj2?*|1%BBo`3tl1R(lycIDv$RA_Q-U32!gjQ zYQ-`?7-atr(YZnD+1WFGae>#Ll4bHd8wylI1XbWMHNY)V(vuqLXQ}70wL6Q2-#O@w zWED2(sn_#pmA^9?Yy+C_iej^84F|eRUk#8DR%VIiP*@1?{CxT0E3kVq8-K9?j_BB0 zmJ|A1PUarc1=qKY-|VS1?JLkcVp`**;s!DDzRfDGUN9-6kLq>FM|O#?{xCQPccsIi z@oMxso!_w6|3=4f*M1F@thm7t2+N}Qu=a-Zxg}k4>vU47fIVH7xN81R$=k*PXrq19 z>K%I|KdTk!|J6&FmDOJq-^|x65rO@Y(JzB4EX+}?BK%OWva6kh1qGiG5iNq9E4Am0 z`(84Lpvy*2J#1!|f>5eP(7Z`&|0ObbS574f4OQ0{KPbyJf{jbaq`!YOe+E=T3vJx| zye(sLLN5v{iAe^7b%3o2(e?GB&C1{vS)u^JEEfZYQVA|B8zwRFxPP4lUA5pk=)XMY ztgA7~T!0D_*i+^_H5MQuj(t2zJ2+WNtz6X6Q+Lk6oPLU4HzoTo;W;f|I+YpzTj9mQ z1UpWMM!A4HeIc@0s_Xjr(Ts60QE(LB9s)z$Fis2tkP;mPwqk!P{(0-<4s`idG(F$v zh>1LKRD%BxVec3n*%v)*#Wo@M_9>OqFsEeMf8%;&x*(5>%P*IlwRU*<*kmQ$}3MAb>hUd?VRo)PWjq)BS_Cl87hc-F7k zv~=pWxlI}V)SIKil}J$+alMHG{FGJg?`D;3HUxQTL0fnF34f~FQVvunxzDzLCuc3+ zswit$HFl>mmM!>DT?#PVrl*%KEqfqpTN$EoZA|BU=m6YpsUhp|7(~zJY-Cz}FDufQ zr%f5nifSN>`&!#|BY;&Zn&1c;ft+w4ym!7JXgv=C?Tm8dv6)H507fdB8TYGH^L~1l znZ{{S3r(&4*wfb4!(n+9{n$J@0+`j<2q@oQm(5s!1a?d1vM*~!YoIo=4-hSlyl(LS z*=8$0E&pv8bl~xS58ZI8WOLe_f%aKDo%btYfFRVEY&w>TANQbbdS<+-maR!(CP`*Yqyt^!W>psN3;-%>wcSCOYy z-Q=({OGFjq-T;65|MpsP5cmJbi5I>4(9Z&A+Kc{|OZ?+KRY2!J%7X#4^;a?at=N#JVpV9hG8&QLR1Y0dX(jrg2?*flx2)w~uec^xRM&i1 zD}htd;~Jp(_)rM7t~|;PZ@wk%X77YWz^G*kckH8u232H9!{nsWG)4L;(Sh*?{;quA z)4=|)v{L`8?Jv;Qj^$kTXW4l8IH)cnoDUB`py3Uk7A^NgNu&3iF&ss~4xLvE-%ND1qP9u2h; zL-ZYz;b+b`|6!u^Z53_gOxq`su=F#81{pzGX|vpO+Z~RiLb3?`mu7|!;6q+ABZd!z zPct}FcM!=5nyQku?dx)7l{rFs$v~lY^sG=b8lg(v0YN;PZa_S!x0^eWGZBVDR|Z91 zb6i=6fopIEHd4VEP{kbz09~=fRnW7p${!6yl8B-rIhhn#1zuKZh5jf){lcuKI6xi& zL0dIp-*lKfBgD{@FFf-IUVxwtVmZApRYFw)vo7J}*J(x7%^{%P`4gf9ZS~Ynu7qgX zX2RRkL+$bSi_en%^_4Bpu2c6m@BezwC@u>He6Zcna(fZh+~f-f_=7ZwEtWIcAV^P% zAIP`e!o$Fl)+Rmoau-8v~YR7{RVV2=SOSWr|={la|Ew#Jo-`$L`bg zoC_u{JIbcNgRD4U1ui>K)uTQNm76RAG$bsUMXL;Udih`-zPE?6wlyH^_u)tsrW_QA z2x|HEc!^0ZcepzM0mYEyi@y${CmCczXl*KJJ6ISRY}$8bELz3w_$Buq4ch{UbDq4` z%BX(WzdF{|AG!>e)+awMnMqP-p-fJuy~^JHovP5a-$cG-+f0`49A}zpIz~gM0-|L@#n{C=Is$8Jr zyg{UNg{2NoYx*AUFGjkf^S~gf%zm&#`2<-9N)tRInGKa`<-YU<st^73`7nhV;?L&T)btZ4)UFlfqh$xLYgf=XSNYK`Xy{P`bTRZ@}*xc-Ar} zl*8vy-Rih3N70T4DS4%;Ae~v5ewyh_bQ#{H)GiVq2_~jXnT#>0P>vQJJdwrip!*IB zqPBHp5}_?{R)bDJij*exnCONsr`CR>v8?BDM)sjinRyuchNAI+lwls`Kj z$WD>mCu@;l65VB4+am*>QDlCgGKs$j3Lw0f>nW5`w-e~cJ+g>_-`hR2-kdJKf zXJ-`XW4eV3@qE8EXHueTT9oNttjAC!h&R!!mMJkM?w+da_8G4}OUBWO8+B8hqPN+I zM{$oMr13}tf!IcYJxta`p~6kBF{64++?Psqg8F){vR2qy+zbov=9Yc<7^#O9X$0T( zbXIzYB6Zfss_K-Mz@AGRx&paDX)t`5UpmFZvw-b}oKCtV3qtL$4mvh5HRGxJd=Gy@ z+f5e;!hm}6$pX_lU z6f4}}e z*F^l!<^SFF-!-m*|NAcT|E{MCO-=Z(ksI`0%*D;sONz_?b#+swyvjj_mWqbJ0Gm7FZ4q%8or|-I8C6uNToy}ZH#Z+pUgY9Ie^rG3zoBqWaBGkw71VDr zkST&Nx6~8SJapCnR{vrGBq}#`N}*BQY_$=m*0k6v{C5oEYUuPN=$dFP)P{E*`FepZA*%^eYg0xeY+9 zTaXXW4pi|(%y0QCQ&d@A&Q+^cTlMKOLHLtyCdq-eHc44|_y`UhC~#9G9>G5DZ+#-8 z0Fo`1n)BzOzn?@k+HbH&&w)-X^O3)1_dqaNDY*8tJYf(Lb+gTGI+dQoahsP90mgN+ zlI;OS6jZ5DquX`{igQ^idP*`GGH8+h0DDwI?pPeT?)X--vDl4P8_ap@i_1WBR;$@u z|Bu%N239;C`G;fBGC+u$gOFSyzfp zE_Vc2IfKB?HUoj+&7vI@%EBL5=_WBM_*~_`$s=Km?-~er23o(3Pf!sDQXuB zL!gw72l5YMNF0KQ{FB{__)C0V|K4YKcIS0taROp+S-2k+$Kgf}M2~bk9EmlB9%@j9 z->*}Yg^GFz5!2)w-9T2vx2LP8^1lfCyJtH+JPukQqEclKOWRCM9dqf#iP)cSwc1Ul zGud3KrDlgE*EE(%rzTOpZi8hc&&;>VJMlk z3ZlaaN4ti8j?i-CGwHQ&v{=3QkwB%E$!BoeucP7oxd8+6P#u9rN<3xBuO-YjuwwM* zl1h1njlmFD@H9|5?{zX?B%MIw>`Cy$axytA-49B0o`nfS%5J>+GoAn`42A@2zulR% z8&4ug;G_w9O5h|Y#s*+w-F$y90=d>f{97}yB6fQU}OCSd9jalAxdxlV>lf77I47+OkUT4Pv{sA zBAp%o%1TL#ii9DVX3LG%%V9MU1G&k6J|7@HzQ5ni{imMv8Z5}=Mo_TxhQLk9Lqj=Vh`Gu8EGACVAJ;Yqw9zBMK*iYDEhg2365vsDtV}Vk<_IL&!qzT0qn2=0_tp z^SF}2BcQ%8$deUyE{U}aiOI1?$*7n$HAKxfeHejSWKEjJP4U=a-55+p|n zRKG?U{%LGAmeL5vc16}7OU$4CB+MY#M~4bL5v)Y-&g+Hi#R9eZs)q37JM9^d z7+1t543T&(=m;C|v)1>uuows`+Y!FpXo3CX1WJu`1j%rk122sFeZL`3{CqtFr0Q5^ zL$1r-qx@a~jfi3Xe$a|*gxXWHN=?%68;$=$v825*yoAJ1(uL@(KS&*? z3_eL%W5VYw9rU6Z4;x1kp<+Cpt;9UceH@I7?Td#GW65f_K?3_1K;54rlXTF`|^UDSM`WEVmqSI&#U0!~{&n z07T&!rBz}oq9&MH8l3~0_m~qttrKrD$^kY_iO7YLD7Ej;q1fv!NjB~gALGG7s6Tn> zUFu?G_I9+9X6_g@e5kK{@+cYnV?>wB0MqY*<}y|K3*gl<5|!!$81~s-I6SC}ObsCl z)}S5wg<@mMU<5R8FvuXn!l?n|0VFEpZJ9HB->}XQ16ZE+4nQz97^KzMRD`d1D>Mo0 zBPpLd+&O1EZy;t6Are5lk<6dwK|IuE)fNJ8{J-0gcB6Nb+Nnxr!a~jAjNDK z_UeIi*$;!Ll?G@NJ=Eo`1 zV5mNaR+Rk0FesU#Gd3Q@$OKV||En)5c5J)`*6Zu^Lu;W)N%+#F<_ew&!ZDUA%Od>( z%|0VA>{x}tidnp7c*v$e(5&BdU7Xv5@MkRUZI=IvRd1UEf$OjFGUFj*sG|oA91Ha- zmH))1&)+PoNLcZt@d7rtzY0pT;7F+2emA4Id6hp|X?nL6z*B;kW0OaM2d^pEj$}Xu zBSWUcfEiP;0b?AQY&a>I`A{?aoo?(wXljx>LDuqeA|< zW>(ZWWVbjlTLHB)?ZE&veSv%?lc7*Lw(CS(L+?Si^?Q!F##=9rwVXMhXX;7 zzYq{GixYVkqF%&1rO?ASnxOpB*4#f|;3OG-5rFyM1CuAAnn>Ygf6@PjQd60cF!kW9 zo{+@DQ)!^QMI;1LTS?|B}N!%Z8-%WYtqgV(dDaZtD3B4$A zPjgwM3JwY=BRJA+{j`To+nZSkcI@XCI3;XwCLDD<=(jffwvZq7d6ISFST;pXkiK*Y zA_+^uV^FEA=oJ%K#gP3WQ;_e75P=bk$QwN-RT0d)bQ_s}3P|<2-YeKzbIKQdy@+dR z0HsQTq6}KG&-EvB(U&YxHM!V-{>x>*VU69VK9#)A5lc2+^#y$P`ErmRwy)D_xUEH+RG<_0t?l@q!O!QDd5X2ux`kv*zszEhR`U(gzU+1EOL|+P zYO1dqOXC0X=V~}u)nqf5uYbZ`^#gbA`a1y z+a+j6^sgUp3;I0Fl3!+`t#-5|f_J{H8GpJAy>)aDiCyVLDBQE&6UiQ8iiS{*D=u;; z*_H$6a(GxIHS(DY4;EC!O$sVc$wlIFZ`CqfD$Z8~D9h z|2vm*3wU^Lomf85wyGq5`+EDJykz?#O_1kcDA6XESyHNzXu?F8`92M3l{sXMevx-Q zEn!9U_w_QD7WDDX_|-YBg>v=nr`yUXWR7F5@aNv|zKA)2r;{uzaWUHM8ldLfN5sO$ z-NnwO^rj=ZX}Fjv`|xarjO6H7lt`I+eZY3bZd% zrXRnlJVO;-q|!5y;N}{#yjnz6)d2~jfcFVO1IK7=vF3{;5d9R2%i$bviPLKP>wz!A zQ5u0$*n$N3%L!rwCm?C_51QAOISDVp&M%eZNfM!6fDHrr_Fx>*oP>BjG#E)};DA}; z{?^Gt!4VNEiJ41NOA#cH)(P>>I?|0XOg5M^Pl_dNHzka!~kqAAnA!k34qNCynW}lmIYE7US z`rcoIM}&n&xV#U)kw=8RAAaMZ6Tsst8GJPKJs~8YD>Ohg{5)0s4#ZZu4u^pvwiapLxZkmlzAV+0Zy~Vp zBzZKUpfNS0^vr)~bwW_Q7`tHbwA3W`av(?~>*YC8)GV9p{oK9t z`~KwE8{GDsNu~~+RkJHu6Rg5A*DHiPu@{}8d+%o$ zwO9|LP!>t)bJ@6tS;o$DK~N#rAQqvM$UqXCIm3q)NEG2L`n5WGV6XAH6Q)xIOZm9w z!ulI7P~>q>yN2O<(+!Z5NkT{OZCK=05Qwd6sLofpL9wV8$#t^`B$A+~L%nw*hyXBp zM{)FZqY?uUyx25mBIn4jZ}jiivvCl5DSU9ks&VO?V$7q)uCf{MI=UrG-^EJ2P}i{- zaOY%j+%oDMoE;$3a@R+tiTKD~akOTMvOe~B=Lp=^{ z^(em&=pak!dwb$J(%bi}l!@%QLqe))x9y*^VD|$0MbKCkhK$|2Hf!KXTx+ht0!NnV zwX+&NHHWi~7;^hxb1m5uAuUp-Vj66++Vff!-q)I$d`(o_La*@49d*N_v@#k~==X5{ zO;1BHoiul=RXz?C%RY@O7x@<3;E<8(LT4*xhGH9WsYvec2P&cOazJ2#C30f^Fkg|W z441?+I~kCXU2W#)ckZGkoRv82XB3Li)GJ#}OGh~f(#!v_Ip}Vm)D^BM@i;IbKyxT#q`W{cPTtjt z4b@e9YL0uWvz*E?3BhHy;gv13sxePPxYjW&n_*h`a`&$p>g4)gjSdHV1^8XBeT{d0 zeLew}D)d=mTO}kUlNpOceX2e(-Z|?Vf^8M=2CkPyW)XY&OnxM&79||@BccSq>dQ!t zhD70Ze#vefB<$dmG+*ONDp35UyS6WopeJIN5HC(0S^$CJu?0Z1B;h9_P3aqdTeRy6 zl1Mv|k!34C+#4xpZNp2e$fn(dony0&!Ghip0psz{rthm>E#aOK|&~5p|6>O!gR0WqUiU=~`@J-2~%XNR` zJ&C8{w{O2{Hi9oHbhDg9fiFjo-A#seYR^M^DXPzAbj>B01vra}c>`Ge9tOZkLMOzT z-ex$O0FDgG6s?Ap0W}n7XqaGi<@#0p8Y20x3#l)nFR`w4Ae}ZUfLPeh*$aVoiqZ>` zd1U}H@wy;dt&vhpu?2MSxALCz4#68_H9lGEudps&mGk?0$#7pNeECn^h@ zquUELpUJBSG|O=29*_rBr0?3AqW$&*qaA2x1P2$kUM63KIwIyX?}T%TwI4{o(hqZ3y5v39ZaW~7byWM>ksoH)TRp38=%$SkqCyM7f-O3-fNY+BDZDf z=XER;T> z99YFE0R{IVI?I>t9mZt@cRLVg41xYTE*Kub?HC|*YQpvp$vb3=PeIy(clZqQ0rRX&IiDmNI~U2d^xETzEr~3I0^SN`4@!kR z1&$7mU)?w8?T@315_paCU$9L1Lac7}N`Eq3Z9hswf$IP!au zSe$$U7BtIUZ@$Andk<4+M7@%MH7i~scscHYBj7Afxm#a&MLuBwhT@V=JLGk;mAE`p zCh>_3z6_=GFbh0C`V)*sTj^eHG<)=8n6Tgt9kCOa$R|iJ<1iSb3RmRvv1(yaeNrYY z^c5$9H64jKNyZ~H_%O{zd*2EYDWRvL(AMDvs^Kv#(jBdI*J`LWr5NPEoTF6@xZwCY zZ*bOf(}QEiV6=|v=7DdwdacIP?Af;GmJr6+&<>96MN+q1990#Jd4k?ps)&#Uq;ZC1 z(Jc1QRjl8_ao8*L}C*pvML8fRv(gAgT zknwAu&^DRI!;NU`iPmt`tpOd;g_5L#pPJ89D1!dZCI9)+8fyofazQ3B)meP_G5)rc zesiYpFzC`F2q#;=P|~7H8T&NEkfE{2q$d_70e(@`R1eRnFhRdmi&SLmM&d`Wfg|Pl zyLE}|#(k%plnfcZQ*hKNy($hWnY9=Nr>G{Wv^{Ic0ey7nOm+@3wQ-(gDH5;LsTHb* z1M{j`fCN88kt+KX8W$b8D)B)S zt3)UT{0fP+tN=(aQ?n8XMk_#Uh|3u*;#0a%-w$bs=zlu!ydMX>?ay%hUBkYGxrz7l z*~$M_I^ewlz5-Z$*9eo-YNzrM=+L`ygjcYxLxLgtleF zlpF(BK-%!ihpVE-T|}=IDy_(j<1O1-%&b*fbA!01Jw&^meYO_zT8r`haR7%ui8gh_ z4!kR(wR--BP~P8dL*_xJDp)-SKDRvU7olgk=euD}lq?sLL3L&DyuuMhIFW$sNk=CK zY^O3)`S=7xuh9i`zMmGuIM^Qq2)x=e50d?e zkqkz>Y)|_EMgsSi7Jq?ztzC7M^hPNB-ou5JIj|3ce=IjOKUC&t0bH$#zXez1xNv59 zaI9h<4`$59#%BK5erd9K_O#y!Sy&Tt13;5v(ZzskI7{Pilb3!PdPTn@1JgD|>49U6zc_U)Rgi1*7ByxSqlG67n&Tm#4!zUczKL&`zFgE(2 zDVb~Lm=2ixeWAATEr_VdN%_A!Cr>~~3Hq$~cNWkpYneO2$22MR8j^kcmSWf@l}D6H zD?)!C6>5v_H=c%p76_o?fI*16iE4wdWzR4-)K@ZL;GkeDpbM=x zA0R36mq5)lcA-HRIa;U?hSgyMglltbb^)Td85cRM7~eUv*f5|4v+mC1-ESozp2?#a z#W1OgEZ!XUDqu3Y2JGybTju!v=e4}GR}}V(pzOB0Ugm1pr?~WG$X(ov2%@LSGYw0Y z3*IG%^cJ);v%WuRAsWBT65FI@^bUx#RpU+0&JStnOk#aA{+dZf3IR&YOj9jPx z$#QXFIqa}*X2-ewGPKzZ&vI0aB$Emiop!*ICBxM!PUhw>5dhvx zd>%7F714XQM@dn3)VxiYi5?33(mLKNX(l~NA6d0?gPgh;3NB3)9Omx+5&ne^=!@n| z8HH|l{hU}Z`mA`oWeiOJ``})0?;TU*FS)1khuj!z|9-Euqp6H8WmJUE53C;m5n;8G zmVlS1k7ZZusdP%*XXZd!iL%fWY=6rjJU4f$ijJ1JnR6}Gp%6*g2=wx@iZ#3^9&%7& zP@53pmPe9DK!y)uN78^{HOxOnB^OZnr2-+0+qx7QD77O4c^+M)D~jgLYn5_s;9ZO9 zRHd3z<~v;^a|*Lfb3cJNLnSCbeiqy% zqfR-c@N_ywGit{_eH>>_!_gb!V*u{CU5{#-=*Q0KmS<8y?mKW`V-Xz)U|BjY!B!@I z8L(ptb6l!n`LmGxfk_6_r-?0$5(JYEAC^(Cv)t3d9&X+Nk?8LbcKFH0r&C?Vu-0*w z&<@qI-tn3A(hD@Prii-$BLGqv-ED(mYwrc0{#gaSm1u@$hpAdT_+(Td zcI6Dp!?jJTi1~S%l=*gZnDJD|jpWk@Hn74O%y*&O2N~AHv-P;Mn_x$zN{F&uaNYlY z++O-IPKzqT(I2gIumTQTHy|)xM`C4Nw|Na3vK6+`f})KMC4czwjKfG;ISj!?oMy!6 zu>^cE;I7z1h|bo|G+zu(|JyO|;hZ9wY?-5zh8-k!mm}sz*;aPepN6YR!>%?&RULnd{HFM zH}S-7X}PilNB7i!nXenCkVCSZ2yGXzp=rTvtA)&^piO|4FsY#0fpHiTx<<5shX0E% zx$CoVr}kH##F3=GLgn!FvZvZnGX^j5X}=>zujCAOf`LWYE(JdM)$w?!QC`bIj(%SVIjsLPm6LskM2H!HHNv4>ARUUH7Of=;<-nsP3-vJMT*;Hx|6?6 z*Z5ceHPNV?-33MKhgtu{Ju$;KUkdykPu}?jB?~Q67m1ja=sBq7F}T!IP3|{At*NR} zqJCorrN}jc7K@*aZ*shM9{AJN2V+WTxXNZ0p`15F=H~XiWXST&58u=LDTPxA3E5An z6(^)naW0vT;5sL8m}JvTcf`0boY9!mdQU!sK_u)`4uGoYv!KAD?0hrQILj*MQ7p-e z4IvTcbTlTi5q9ZHp;!73U4kg&JSD4F-C&?QP1Z_3l97`sOGZ1hYY-Kb?ly=auBx5d zfBz%`!?5i0@5yJ3-7Lhc!{S<8|DmO>7PJzY8RsXoJRrRzGvvNFsAr=Rf&LxC@TX~X zLX(@;^@b{C1S=`L{Or`wdyp26katzTzHP^uVGVc)KHSE5PP;X+MsIszas4qI7&Hl@i+!vBo3QNAsIr6AJANip~*@Q!0BXl8E)r^ zR&Z5a6%I!J(kKJnZ3V9SVrwtw_6G>-D=){3q_V58PeNl=m{B5XW zR7KwVIE>SNoNSHqG!`U?AbUmLxMb(^m9Q31cEmKHXmY~3T41`N1sMO<{2j7`Gf{EL z=v_qhjBYur{c(8MaUqrDp416Y)L;l1EnEpNTj_9o=dW6iKu8D5u(rK+(CfwR-qb@@ zo_ANIG`d7vRtwoC{r5F3vsLD9&x1QpU?wCe$#mSt9-M5dYi7 zB1mot)BHM`Yj^MDTl@EOL7CXpZL#`hN(MOLd|76G5n<$Uo>Fg0FoQ%PBi*e`q8z9xUuRV#)OhB+fwr zG9C%JDD6$#0jAVp7cpi`@RjJJCEW2PC}yCZ_M{;TZr8A|%y8S(5%< zCVtgy0gNz|4M*O%sdxAZPK;B3R2qe(39MCoyvkswFh^*|lu^r~YeIokA_HcQD=x(k zgB!?UczN1g{Y28HE!3%PK!MfBtp}i!T(oBn_a> z-DcTC?Y|UtI~UZY@IwCiTV4xj|5&jFj(i~&7U%<64Xd3~qOji?7l}l<-Y;f0L;UeC zc`dnoPzcKc?hWb;AyNn+J&)6c&K3bNlDA*KcLQ;sRogn{C_hBo2YJ#NN+VsQqAQyH zn==GQWqPf(@=q`d96T5|UDta!`67(2&hueuSdI!K#a<{mZ8u&qXP=p$=&+{!&)Im) zPE?!T0ZB0m2+;_8%R!wTD4UE9MoJ4k#~O&F2)!&_`&_*B)VJT1JpVRLXIa{A{mrh3 zA4FPq`Km~>6;=sQ{5Kt7PVEc2PPu(Amor$&6zXx5cB+`m5lZgO!_#DNtLj?-mh2T( z(7YVbj&Zkjyl0JCss;nxZM99$1q_VqbMX8iTAwAImeN;8xP)vLf*G zOofFU36kDLO_D;wOz4>mfQ2`ZAa0k9CZhQWX3lk%1_qiFTtj(g#>ytlUz!V-JfbHFXyP0iSJ8{Ta&?3zQpE; zG^aG$;3z+Io1qXUa3BDb&WK3aD8bWC#fu0NVS_*neBv&ZLw2RCP~D!Qc7Q~7>Ff2U zRIcHK8-1|0gt*I=^L2?(;F@+cnX7e1nBJ(7iSs|7%lW!ZxV?doqN%0H_h1Qs&xMBB zJz4Tz~aT6@UK#5RjSFwWPd z1&58ayf7fSnIm!2nFCKjyTJH3nk@n!Mn~PoYxm$irYwt!S_6svY&&7xqPxv&=u&;f z-Ffb&SWS1J7yFX1sK3HY^YOAGAnq}3w;rJbnu9-=H z`It0<8cpIDWqQ#>iV)3}IN#{1(AXrp^^o-x5i%lNnDCf7V~6>3Y17Cs=5+xGVf}h1 zz`-WPb(`41MR#muzAz|seFpr*1(yaI`(J=MH8>6OmLQo}X39BiCII(t58@%N#Lqj_UMOx%)1mC5od7PHv*r76^KI4YM-GAd^oA|I*aPR( zug`6$T2z%}8n!9W9I9uiG^p zJlj&8_|M-xICUQG2jb&gBro+4Szk9<9yI+mHi94PeJ(Ldb&9QuR?oiMWG%!#X&5~~@>DgTuf*#z~Z@dnBHHYwu6NUy$qbJ+wxh#*~`j&2> z6T=pT@E!+hHQ3#Kp3eJW!ee4zgNO2vY0w?pfGRNXO6_0W7TSy?f^#sQVBYv*{kqI| zoV0f z9&#J&Jg{+OwyO%1y+D{IAB+zs3BS+Un- zrC6=aP<^mymIN!@@`TtOtZm3+AX6a@E6ke8)V9GM)^?(<6Sz{$3Z}m)>5S-!$pRjJDm=blM zJr&VW_C}jJuyldsKj5W1)AgsA9eTa6)%X;5BB2)D4R&o^0`3lu)gG^jLPo5J%G@4Z zM|vI^!RDRLm{N>Q%2mprSrCOA!9uq|3?U|w`dLm4)d6x4O|)-8L<#3Nyzj0=I^s#c zf@Z^tYP0i1LuK0ZDb&cnAPbui@$;Plqud<*Yyp&<&7t`*_kPkyc`)TGgxOJfbJ!bo zYxRBLR@;lE$EwUB#Luf{MzX%>YS8esdw{b;)b$G@z`@HI?C0ap?!AF$(LihaX7QJG zD0yDp8C5vJX+!TxavHbyR9yAc9+(qynNU^b3Vnc25~c+k6l&imM7oN*#{Lj-3o$~F z4y;I$*A59^bIkaVK{Wv<3HRJ=ouTr`D*6gJk|5lWcT5cwV?DC-Jfz@Iac0#R`8WnM zCIgVS=8F+bBy$%s=tvequaHAFWz({UobEGie8!2^6zkdCYRP?1Quic4WL1-HgQ`be zmG`!kiP(paP#zyTWJ%RgqBfDEjgd(arJ!1cdsvq?m>6LIc|E-i)U+qO86;EoDJ$`) zP3S6X(kF2-u{;`D^Ke?Rqs;k|rn63ClVt{9odQcE8jGxf@`&@yuhHOP-$V=1ra3Bb ztm4r_taZcoXS|~>HA@Wiu5IPx)nT1;Chg04VV)2=42E?=9RXsK-Uj2owQML-3=lu0 zub!>yDjAmizw;g3Q*rGyJUOXg+Wa^cH?h!k`Ye#MNYB<#f$aZMKORu#BAMUkbv>x` zJzGm5wR`F=Qv+w{it4FF?x=Gp42HEUT#hVvt;2BzM_&lND7VF?;qXZwf%aKn?Ii%h z70L+;p28GKjYdF=3$wFGd&6QglS`7RePGD8L6zHoOUiH)olw$rS6{#RD$Q#(VU*Z{ z8RQcQ#t=_9284_ypK*~B(^9%k9Kr90Ch(WJPV7VyskgM0*UNDroGCQzkl|NIvoj#_ zu(njXe8V0eG$T_e5_Zo3`!iw9rLhSQ;1No0-FhHEVpq|vas-eg=+6{kTWxgp7xt#g zl(h*hE67!4iIBkWx34j0{Kb%|cg;q`MRA#rvabK7CC72CK6fbXoQDsOQ0ihbyrnzo zrcmIL)yikb{Wm1h0bt#02tLm@2+wM)yZgu}<{w z=LFpzgg@%`9hA|<&|Fj#@B&yt`DzH1nXAiZZjBt&^n{NK*X%nV?yu)xQdb4qOCr!= z-IRCvn-jC{1_mx7uQm{3fht~_BJRN~;&r7JZWE=_>T=GOg-RhBBDBnQC&Yiysbh^_ zCcc%TV4hyW85fSJoAY7A@g(257$(+I8kq0KIqVOSBx2oBj6P_Yy zNnV84uhWwO7)?#IXDyh$Mbq*#%mQviTs=%`4>Z#l*5#w90$`$6G(OXZsv=ZNuoby) z31K8CkcYn8W+$=6y6v~RSW= zX`=|9!mGAiO~^k3A0;_#4|>^xdWPM59fX-aC_6qi`i=>@t|}L=wUy586W^FDm!e&> z?9%A!11l-N-wc*LeVx%pmp%$S@+8rl5E;_odTF?yICrhuWmi9z2KN0aAjlm!QHh}h z`-3q{@-X`onT3Z$QKRj^r>|4!cNkU7WX~rWqS^JJa*;$xHxv7!j|-4^+cy5+qO_;& zi)z4Y;?tuMj0%Apsl?oUj@LV@%T)p1=uOR+AFGfPu9W2|yGi3t=2{*#VU~gkCNM4e zyrbp&Yu(Iq9Y$WdjW{=%CMkqo`d{64~JSX#zfQiC9x4-+7>2V^9MBAa=% z^|8XlmxM{soZOa5wc?9e+Akit(XO`F-2b(}w6u;3$sUK@V)C+PuUe zqI;5ey_n@~e|UXrk(OpI&m4f3nNd+$G3{@~u@y^aoezUL-F<>+uTo9p)toZPj5$I{ zj^{A~B-n2Wc3BB)3UDKdHHSop2p5B6)%~R@b9UE$&5S$fDXj>H=(6Q!iZP3^!ASI_ zpF}{#9{RNDi%8ypxF|GGeKy1@-A(di9YKuW0F9l!n==DhP}Og*YzqAxh?eB~|z1Gp+9_!lyexv6pGAOm(Vp(fdzf!5Bic41Y>_ zm3yKdR&b25zMODrC32eCvWMx+i9tCDYAP(}q&%ToeVCZ7`psg{hVkOMZ zz~jiuN|F)U70H=6OKFb>_zk1hmkp@J4MJ|)O*I4}rp<_l4Av10q0-7Ny(4gmR zDlFx2Uqmp?1lI`rZC5ri$t!A?P7jXSnzaPMJ1vn}@(y)yZU?*OBgrw+39-pDpHw(- zp_45_lO$!rwNff-{NP>TXQmV>x+3vbb)(i7xSS(?Wa%YkCZY{9JUMAb9g_7q>^06p z=VO5*RrVX%og>z8Z zjboYsQz3?GUoMB=hQJue>X$l+qt=Km4K3cagFuLCgH;HJsvXi^XuK>oMvkykwo`M# z4V~SJ5K*2xt)DFELS)2=SosBy_87NUtxJ=h zMRU50?o8R@4K`c~8GxS{6%O;Y;p0J{aW!{%t4$5fSRR@!Rby+fM+5tFlUq}E&Vy{V zlhGi-F~N4G>eD)YoAvIz!|7{`bSeI}2Ho!9;7bBseG^(DSTzqvRiXm#H_oYIH~01F zR`>*tLeipnp;JG)Lw%4Sd6+GEO$-!C80OY2$lW$KCEMp)78KJ{@&v^r+&Y^886f>$1Slq z-0zrNmbt*MgUch1Xaqa>6fVc2N;<$74I1=XhSdj7g~6Sv*@;PlklcC+@t zy?b};+P(k4@p_{PtV*Y{h2mhnvfQXH7y1mgkP{iCgd|~g2*Xo&YG^LJYLOSLU51&v zHLLogWfUsO;>3>3;HVsbyCejCnfF9f4cQIlpGDz58j0aB{;sAyf~QO2Vv5Tw zuMKa}*4>y|zsDHaWkBg?e}NqmR?o##?brXG;J`B>hI}bAR=2j^mw4#kT3hz5IhF0_ zsQws~X~}S%W~lRyl)$Cw`JUsnli4Cl(qtkDb0DZHkhV@PL6rSrkPZUcipZ0+olHOu49&#q#YFSA_bnn)*d)N;+IUWKlwQYi@H+by?RZ@S2s4}p%xrcGN1ht>`D z<^S-9PfyP*oH%{{p$GPY4}9bG{q1JG)v7PeoJl3p;88`q3@SWZMw!G0dtqFbYiErD zmE7i&ZLnoiud$=6lX<`FvZpGmly>T8Jc=Ss`lQ2*+!RQ6*?4e{0TDDK3wA$Nw5gq6 z3FD);NK=1(mDD9pA?~RuddFA4^3mit`&)7L4Ximq6kpyIq~|N?>C#RyY9Yq_f0!W%PW;yWAmnsbMs42 zKk?AE?VC=XIEt4Q5`zlXv<)mr@g2pNBfn5yLIp=BUY<{M&Gx9`udpwO0`8FV6bRi; z?&Um`9nlsiqOu;9F8Pm8EHc!%p^$x0Y(^!WAGIZ%sgOb#r zG&;_b8~VbaP#C;>_!f_p(=a>wS#suG)m8<2a9vKC45PQsUWGfJ*zA*&?XLd41nr!& zgKBnIYm~J}rWNrn%4_9MJNpmafB)WX zo7SJ3oI1R?vT4)$yZ7#?SMg8%!TauPG@Em?bNziqR9AQ2zNgt}zWCB>vvZ~O8@Klj zj-sg24ckg2*wiWF&AuAz(OEH~Tn=rzvSxc!LXIVd{_^RAGjBhSt&f#VBrIghpm~Qb z9-Mv=AZC(hFyW2H(imJqA;W!0TnUYVvu-q$Uxy6;G(e}Z8!@AvCUxb!vEl$t2~=8K zw5!tfMEXL~)r#Vt;84g%#=}B}8hK>RRhUqBJ+5BUX?1Jppsq0;P7_B1QKtc}krZh6 zY#u8F21v9ou()3-sIs=*AJrC=cYdT*r{Kav+bj~Lg`&O~H5c2bUyTe~AJ}M)-)3yx zXKc7b&8-6^C?ktLa~;^U*WdlLQl2u(&`<~iBb$eQ?hkFYZhlgZUR_f_CHMVk_~Z8% z-4w{v@Fu!2vxDxBon8&i4m?EvIgRQP4OhWi74L!?s?r>myMdg77;r5_J72m{1}|FF z7@$Q1n`JWf0}-Z}^p#<(+mIUBjQlEvT$_wkrVmc!T5Sbtbc1b>1XWP*mPLC|t+qX< z-L$AQ?$`iQ^O2r1q$jG|=>qBumT}NViz+i+E>?tjSfi#d%>3*Zzy99SkKTFvO}^)z zpPajW&kZL|o_+Dqz{=mey6=t4V6`XIDR zJx2sB6<P-gEc(_&B))T$P8GDyHuFm)`o_-r?t?GjAFcUDOWTHgOBo zE=7mJvnm&aQ)#!V&6ZY9O$-c=jE;?6>X*q#ByRdfSUTsv^VgKus5ml>Rf{a zY{Mj*1+>dXbe3VKP_^**FugFWNMOeGD25W3eK3&)G~Sn@*lT zH}%ZVU;5)e`u46}TZ@HUK9`xEoLep}`HmN6Qjm(Yn*Pv00r4Lf&QCAQ!3n92@DKFr zaMFFnJX%=u^NXJ66^r?Ly?*rQiSeYp?y@@BiS=y?5c- zxJt=D^j9|C1ub8yQSvYB4^F?KOuna7XAm<>Yi+9ETYr0F?7k05>MK9}4R+^0{KJ3o zF`(*pdAqkD@HctechvF)_t=YaHSo{8=?wi0eimA8_?iML8865JK;gO`tp8HUv`XfD zCR28xRq$)?H)_kRMy=IuV3o1#jEgTkCt)QFT8UM>MLO{=3jdBrGD{b|E0(ac4Irjj zRPx*l7^{+Ccz|A17;&S5{((w@CN&MNz)0!NOP0>X2uc;ZP+k|wY6EVf-Mky+Ink6NVaBL0yVNu(F3l zJpmFYLkEWRqSNY=Ft5qJ1n6plDUiPKFo~%~td44?Rptm`5cgW%lTU<1mzUGL5LFa7-+$B!Ip zx7!ar{BU32M+SAVPE~6)T>1HwNf8zZx5M5Hs-*GV!7bkomd~pTC&6KBe#<}nyxYGu zapw=#R=(orJ5%rqG-k2CFUPzFLj9+0^j? z$Z3f*{9c1r8|Oeaeo_DiLkdi2YRP78MN|3>zDDVFkPe_tUAudrNyOYod`g*>t<%=r!GuCyHYLYb9di;=lQdf2M->FB17hx!x&x~xzfBs#LuY8>E7=5MEr5}YN{)BU{8Gb;DI0gm%rTq z-n+7#Kz*IdW%~OEFmw<0Z@E+g?Lp)}y4t4gxDNK9tycSL2C9!NytyI!#>eqzc6RQ^ zfA#bYNnwjuSIsu!8_>E0PgeEita=bw9S)5eL%w_4iC7ZIPR4(~8vPEi;K(HGqC zm{XloTU8AF8Y@%ofuEt{1*y7n_4e()3{c76Uo7jSq*7FT*uaEVz2bQe7e;C#kxJMJ zG<7i5@4A3+BuFXM@)BIuQ@Nrco6_9TrlDIInJhR3iWFJo&KJhN?Gdj75tlin@lzS2 z2u-l<1=B<22;E9v61V>H(z~sdU6(&9k66!ZUNtX_>tt~x!IRre5KZ?WYXDsr8zk@S+6y+ncQnH zzj6BHnOr`Ly*x<0w_bne^x0G5zd<}gNywO}`pJ)2liXWz#U8_#;7DqX+KV1{zSda5x_R^Fg9i_S zsUIC3y#|14WqJ8Gul(l24?gTpFmKtm^?Tp@y}R$dH=Rx`EiS$E;;&zL{+EkOOSxS3 zw%hJ_@~Nk8ylHnbnf%oL27XznbIxsS+A(XyD|ZX*FS^g7E=d@54&LAE^7!J9!;KZC zd!gdLbFklj-wfbVK zB^GRVblVgyuTWZ*%E~R`2HE#WC%|IC2CYJ5lqaZ6!i6}g2$T2O^Lgv?l> z^mN*7HNAG-%oKUDO5tk|1fx=~Hjf=X4G2TMh@uJ-iItVw>EkD#dFE&Hi}QtS0q2EI z$Z} zIG9eP%y@kyeu-X4#J?0$iF#Q#=2(eVU+8P8o&x1(*#kRnzvYMjYwg0x%JiAq!ugPl z5LLb~RMy$D5u+-7W5q>WkzTQC{EvJ?BW2Z~gW9?K#a`E{pOja$8jl1^eCjwV6Dcj@ zHL+V0nNhuevof?p8@o+Otfhh9zxa#4xa+RF_U+p@Ffecp0M*RQ^tp3q5LF;i3Y)+i zcHXe}uDkK-@bOJ-O#i_j{)>r;i4(_<-LmJlJ-6MKPJi4k2y0gh@urnZsZ^Suon5Jv zTg?X6IB>?fd|`NGWOUtV|G+>F$v&OU-)BJ!Y8^bzN(scuLTRZ~Z`7PNe2>suhn?Es z*y!lc;9#-8ADx9yeFU5dq>7btrCzVKTWyI|K+55f;dP^9Wja>6b|MAX2MiA4lmLp0 ziwm#3_~I{q@$<piltFLY9k2Dk-iw?ZD$0_erDdCm`tKv4N!+pc$>dK8p7#t<1m+>n!6B+$%J8- z(`mdFfKOcaAO;hfk+YH-fvxOO(Quf2r8u>KHeS(Xgp>y#d~oB&jdOEzKls59)>th? zrI216-JzX{Cb~P1c*@lTtx88R9op;{3ZO2Z)mG)`IF^xuJZf+KE z`2PO=$BrIdTv&iaK9>J*jfGC>@Va$3-FWl8_uYTfO*i!q40g$Rd0^=P;_iR8r%_X@%Q+y8wP&?^@pAreYoIvYBU?>0A%tZy_Mi1$Tb4nUiSaFM_|5SY9P19p{^ z15|R{m!D2y3R98A<>e7mU`KiS^l2dFr?MPZF-mn_J4|2GIST_YgQNXhZU&v|HmdFF zN^9|aWM~?)Lzo^ z2c^56)iA9xa+5N?N9*4vvi%}ignJ~fdb*zVXe6A9-|S-N@=;ER;bg#om4Aw=e(t*M|=uZnYYGcZ4} z@4oZkLytW1%_p~P*u4jn#(#zFUoDHai$u2dR|YiN&AZ*{NQ zLa~@gCU9cU{Oljzc>OgjXjo!%+1wp__dfMI-`;l9t){=zn7GrQKdCOBR!ZlL`hwzC zRksn6-aVj*^;F*2{(!N55BW1)1Js{F+W~O330w-0NDy;N-iQDp2iPR95A17kH3W3{ zkonOzRx_cl(bEhlH1JEC9gUkZ+%n*>`H*CLNIfL~Br12n97dSdB9O-xquo-@pU|gH zW2UN(Ktc!(MbsYz8AA?T!CaI+Fm~|cJ{FwZc%cv2Y#o4y*eGEt?Pe1`fZ!)pX?j6{ z#ipG}!_GWKV__LBUZ|v|l_t33^$qTSy0P!Zy@C}Nct%T?3Xgy*%mCNx=D~G19b8y< zY~D$&dqIs55ZxY)<3w!vuJCUFwdmN`m*gbTYO_>9MI)OIUgowuK9RMBQpzfEElOAj z^N7{=92MzeLE8%38LJfyCC5_R-3@tn`KHRs^1|#)d1C8c*x;L@?W; z%eE-h%|c2myR4t&0zo+kym#QgVxNm5DN$6@MKv{`rTf%OAMHh__6M^o^&0xs4el0;rKP3+_VZ_I z)yfb4;19QN+ph7ldEkThfAXWhK79C){0(5H%I1BKmDDp+qTi>3xgTi0c+FUH1qD%!^&J(I_=M&Qp=P0 zt7fN-bvIZ$zHSU}TSFiB=k%zsnxmqm7W!%7E&|BIw=v4!;lS*E}0dTmwn1t44GZjm_=1M4ur>~P>XOju!t^tT=B!G zl(WGCchaXzBqViLY@-7@7A{3;!@|`;m;^48;W>xrFtBkOzB3rFimUo8|sEGJA<^YZP72NfI<1G0!s4-6EBhQ_yT z^V$x=rU6>z($WeV1C^C_qmJ=&SFpZDGvxR$*V~#j`r}WMO_(I_2g?#WCt&)S98l8z zq?JnM6(dPbOER0zYa-EFG)W%_i!gad&8f zW_s#^+&AM_YGq~l`CtCGrNzZ>fA_n)Z`p&vlJ1Y7e-1hOkN*0<969)5fDI+=^b(1| z!687}t{ZM#zhOP<5)eZ`yCa7V9X)h-eqjz?7kJF&<>g=h>V=_^k#BwLDf|d<@%*`S zfBTca`R}j1gpvq%!1le+SJ6-DuF=(7%7} zwO9Ml_@ZO5TCKkG_FE@T9G5F3UeWpu6We#}K+S|}cKv$%ZK77Kl~-0oM_~aZJ4?qI zMBiZk72ypeSte%s(wV`rF=||4GgYfqmX}xH*tpD3tqo2pF>oi%lMnCJ%;V3=V<3$={Q$FNm=vo0YP%{M> zoMan7Y$aHe2@b`lj3;9I0typiaAA-UF*I?vn_SLij|Qa&QD z1R0?_9MBt?#I2wM))|XcEGe~+TiZ@*1Adqy5RrO^ynE(+xNokc(dfcFaZ^*Q;@od8l-J6NJnX ziKP03+WedfDF>QEn(L)2%cgFb;zUuy3oUM0;g6+JL?8xGO(neUS038<;4P~*noEnb z=O-_~SR&sCqr~yyiHYISQKX=m2_3r<+71)fqDdPhY+R65na<^g)^EV}0LK0p4?v{_ zIoHbKVs)itC0sRyo>RY?>{k(Knkp(*Ml+M#Y7$b~$2Q7p*MO1wr#f-X5tOd&MNyp- zmFcKCuQ+H4_=cU-G8>iN7}b@>s{A*ed~*Bt?JvCW{L3%C2-1%~KJv*sdHmS_`xpN= z6cbN>=X(XdK@(KxKu*yTv4fhZCH3Y}$uG@`;Nr56VW#(_`M6~&z(GZ0`@v`l}8P9|JS}gzJC2@Z8q?* zZJ*QSY21i%6_EA8x&fb;3nF|G078nbH2fKIlwn93 zbdAX|X#lJF5p@jc0on5}WZNLl-HmPg7(sAXJ=ZsuE{+)Ox|PTvMtrz`bhfr4oGPdo zdNyiGh0tq?q|<;_l(3;% zbX(PhbQbj%pyp!JE$lR^@Hc}eb4bc5q{$P`H;RSPn##sVRCFq2;=;5c-$LJI-)@~} zm+{xdT6v+gXjn!oRLY(=p0WxiNgi>MXr7rQ%fHU4Kef}*l#2=@$42a2Kc#o4K) z`RUU~4;T6ehR4@WY}-~G7(gZ&cPOQ7g#~}AQA62GAZB95fvZ>{cmuA|EqIQM|^$&s=fUpolGTm6V z9Xof9t{eN{gZHPWr|8u!EH!xY#CF1@E#y1buWyd%?!AC zW$}(RlqO0+NJybWg=bMUM=nh0e`!8@+US(uLasQHUZm|YWiy6~;ghdY6zH8s6RlZ> zT}8;f@1>Dar3Z{?@UT!@k-u13?9XNpDH16c!6H&S6E--pNXj@!1kJ!h_U+V$@*{Re zf{0pyK{2>U12q}olf)5$i2Gec6b7!}R>Evwar1CL?B-9*Eu>8`m=UHLyZ;$#FCd(4 z7)is4YtD&`6t+5&SA_sZC$<%Cj?MXIb75xk+|k4RBV+3}PK=C?=ZgK*#G-OHP-4W2 zJCz}CUzE;VZ|uV)u||&3aTpctLEJ`EUlc2oQGHgl%YrgxNUlk?$a26!bXu#(S5sA< zis&6z?0}jMiwRh-?s!vQe?NLc_{-n6esbo_Nx9j*ROps1+n#vxNr*L`{l(9@suLLUec^@YQ2#yoJHInJI(9t*m6R|D z*+#;yA6j)*X#;Smr9>N0=)KVuY8jA{SV=4p7(c>x!6I^nygE3HiWGbl`OAI~N+%ag zMoOv7(SCUl5%2F~H|8!i z#D20Qn4r~)-4XVAnGtwU7I&=!E@EBRT#!Y>`r~G`Nij+dlpOblizxJWZk;!4yXjD zBr~T6M^Bp?-E2_|goistprLE+qKxh6;yg8Kn`X+)${nc8>|i2aBP#)mWCm1yB-me8 z<+6k2TcnO-6F+J)sVt7bjo@@Vj#AV^BJ`jq^!Q-|LW@w9`)1iSVml1kR3iNbHG2$p z5ex}+Xd^=l7LD2n4GDa%@02PlxkMTvdB zxdh67J9CCubNhv*naCD{_o|xzjgl2pD!MKqp>vUA%jCtQ2Z`^oT?;~DPW904E(;s zfb`tSBe_EV(D;V2iHZJEYGPsQ#DEpb(S`WqB1(YwLP+{(mZHjRRGAUY1;qcbJD^4} z)vN&R7%Iu>E_{YAl$mxYDd-63G;7EXW5p%yG%pYL`u67gGtH!(f$I*F{gnGPCI7O>&4d>IeX6 z5rxPD2#P*?L=_}g`WoM`G*aChB{9|l+b{evH?9bx4M ztm^FK6q>&p9FkP$u3b0$$-nxa;Ux(DNS!Io*_r8|{OG@wmKMMNgWun{antn$R20BL zgOiryx5=K|O31nbBitbHJzqz!nudJ+Ha9tRL!A-C5s-kuUwLzD=v>(samMBJQj2s4yiCR zILeB))8q(2N#Xz_TIn>E+hhXdWI_cjHS>tvGx(X5z$E)8tK~W{gc`^uY+|tm!xamd z4QcT%cmkR5kb#(ogvnKxc|m6Xpo8?8#tb42Hd~FBU#HXx!x!**f{dQzlH@+6s*WF| zOyZb)_C>tTE2{1G0Y-OHyAO#b%yrNeKQXHOIG8zisLq_6o!$40!d zm}I0HYiVw3X?F7D!2?CappLH}nb^=b&~FO|BT!*wTGVGFr$YYqnD&K`h4ejf`$@z| z6$2qupwPnWsXk@wHnHjJ>ZW_u{J6sGx5_FZrKeB-XFQ(?JWa+u4PkNk?kVF#`j5Lvwc-g_vmu$u+r z2NyJ_-2z0(A|JL%k3Ig)$Deotx;=>yDCIGCRasg2hky9H>8WWJpZX|h%F8Q94j%?4 zVW$PF6pUD5u4`~$5OWj2Er4rG-=))ig9A7>xC?^U>!6pO&lhv~{OZ#~!Bkol5iB22H%SC!T+x3bl1<;{tCISnS)LGgmL=uxt zj9;2!c(j`psf@)Qpfm&PbAw96lpto3!2Z#A6Dkr{rSFDxhrpoTQb@)FKt8nJBk~CZ zG(ZuL@nkTQY;cMjfi3|h>GHIk&P{PBBWQ#R&J<{sM{c*Qj^iF!f66(dH$fKNi@;CZF(2F=J%U{6jOz;LEo zpQ$%}g-)_RovJ|x>63QAgIGPh&Y|ieVs-7502Y9&(OVzr-@d1G^1#e{FV21NN)>RW z8lh%L7h0iq?ELuW$m`!EQ+z0i=O)k2Pn}O6$?w`w+}@9CwoKW+;FUF7F=9j77^Pya zPm5b+YG|q+0>YKCyOb?o7ZZ1D@S=s~R|lfhHE2%1)(`9(ckjLhMnRBKoIZ6@n%&@I zhlbl-cDt94#vY!7g9BfC@S(?^_{O&F+lfOfuhbijZpqH~k->^li;s(!pA2hNH3|p0 z{r0av{1`YrkaeqV1EC{-^ob{+CVlSNUw}eHG#zP418;e`q%13|&!pJjhcTE^>0GZKQ60PFSL@BQxT?%lOsQPF1KldA^ysuZk3a%?~s6Ly_nSEx9f$Plzl zFBw#tOB&Tw5%xHUx4^w2V26Oxu3H9WhA7D}8Oec4)uj%jJo)#G_ zQ45M9j>rrUXOc3Z`(PCx)oox(;rl0M90ws;t^xI8bW8!%D*Xwg9f?X?q}0HU{G=fD zyr*2A)WmD4x{_K=X=P+*3S+lEFt~ls*c}hg?tdAGg#-{(yQDUi+@CClxUB6g6JP+6 zq&9EVo27aLYZ7qMdMYlL;)ho{GC=nI=Ps(Lg<7JZjO|snJ)~^7Q^}8!E2W~YKC9km zbft=_eGlB9$>m;o@z)<5cptikFbpo1wL(BaJ~BGG`{vzy@4k1}4Z8-|svK<|IF>k0 zOIF^%u=^f(0K$X=A0C*RywIpODAZJXf$C-^n}*5W&Rsiix%IZ4J9eSp^GVxGJP3pj z4?Xnomd#r}cptXu@18$@wo=B(i7z)lSSJ&SIjj7aC&TPRF+BugZ&tU`q(5E{JwF^`1qLIwqZHlv~d%Le^%cIci45q4ZtLn1E4)6 zBZ@P4_nmj(sPx1W-vq>7FF>_=?VvY8jeCLtOR5B^1|p6fi(p;rkv(X<=3=QxI0>KP zfa&eaBK!ioT;x|N2!~2S9*1v1+K_=uY^y~{iPR(_|3ExswOK}7FIiP{m@9^Mz0mVH ztF9G>$L8kN=La*X1bInNps(puy=6MYwpnffzQzh@oKqx)?!)?zr*M5taQb@NC+03t z^x>3macAAsmAtC80!}VatTwq|Fi$G__9fQz2mYU z5ER^T=bfk|W@n~mW@aFKKwW}G12P8$HKB=v6si8c0W8p686JFH(G7|hak5SQUpdmol5l&4uW2T*dYfWfJEXFA2)g(XiBZ0*l^$19)Qps`ara` zq;08XC;IykM1>Tp0Z0?^--jQ1SWdpA0ARzz02pwRUw`Nkh85&9*n`7EABz#eoHFcd zLGHhDQc#4z=wrCd<7Ya3+dyvIa;MaZmbz5;7ZcDm77&Nd#kA={ox#}VcF+3DLSFnp1 zqd{5Mf?1XCqVB7?FR+!aAWO#<;O1Nvga1_7vJ)XP^A zN+IdvDS(`|p=L}_dGtzXk;9%jAi)R}6A0T7>r#OeD#!FN zqx*=>F(S8LgacYw(S5B~RDCX7VfT|QjNZDhuzvgEiFYU8{N>`2x18E?K-?FAkY3LrWky-nTrxUyldk{eQ4<{AT9kL$ad4lK><)~$p3 zVAl;dp{o?K;{cMw43mG4BRDQxskGZ1sVbTQ=^UW=A&!q&WjLT<%2v4LlJpud zNN`N<#n8}kN-u69hYW*S6&Uh`*OQjEDQYok87b4qSV;cBAFUOcDH{wT43;!h_mN># zn{DYfQLcCp^KjCkO@>iW#*q z4_B^n3{Jrz9{(SN98?EvJ1^`$xu5vs}Y@sH{qNtpTE7s0N3+qzk;)>C;L#U#} zu5YM-C^!LXdb}`nmWCcgU5zUjA6U&l_nt)jt#q|>TCrF_vqyfcCLfoje|G6$oz?rT zOMLgs{^06EyQULoj4!%OkR~r6D+soFP^equci$l2r^H>sI}sLLv)-J!LZ7q?^rcP; zKHA`-F7tVzpx9te1BwWrREiEE4IObYBo^im22Ct#jMD%Yi31~cF}LU$i4Mk{pqIlv zVd;kdQffb-ir*Ot3(Rl2|7)svQ-YI%YyR_f#B5y06NXGYR= zG65^z5EP3tES5%#*^Ln97{EA8^Fj@uP3WbTnorxK#XOMv{8IIFU77Y&Pw-nD={t+M zAN5woF9BB~uCXF3o!_wc(cv5JTsZR15H65tXCc0q4wg{6V;0hFBC$%-jeR zePZh)D)QwpBPIF3i=C?;N?judn&RjVfM*-%AH*jMACoOxwkEHk=1mW%;8(2-k%pEGlhG z2ZNM@d7n-JPSheYNLKgS`N>w9CWsT3aqIf=^W}QAQE!J{KHs-%Y!t3ri{%PRv2}fY zH*MLDhl)MTMI_=TdTd0%w%C+ALehE?dXPCm5U|WXAPorMPX5IsZ3pFKQOAsl#E|@L zC3HqHz_Gg}y#p<|dc{ztaVg2hyDTuH>|^kpm@Y-D4PTLaQWCLz80>MFFLzLp+VH{3%Ig(@mD*wLsfys|h6?l%7@{(haK8c>2F67vK>$F)i`*uhieUfiIfNdp z$^s@CoEQSyZ6IN9DCYWdSq<^ts_xU=uP*giu$@h&3q!(KtW}$SxFL&l=Pad!@S1{3 zrY~*fGZ`60&-0o*|H&=_YCNyW4NX!mYn|{2(Ji@zwAzl7vk1f5ZF1rB3B)1@(@`Jd z1w#Qr2uFF5f$IfTH^P5$7chHa$Z7L*>2n?=NhtWy`Kfh|+R~}^(y7Obw+pHaYSCE` zbhIOqxntS~uu8f0APL#qi8Qjk<14}y6f~LqhPxjhzH#s3vHjC;KR@;UEAGmSrJ^?X zjm&`9un&qpW&Q1dC`FC4(q!vP?di_1(aR&vNAIn;mVqkZ2*j9Q)2VNh!gQ`?KwA!M z$e9@m=!Rqt(qZPyAR;fft92yT`7SN+QP}0;_;gWRi*DTs{LKX{6Rk ze=d_%#hKZ~0a{zm!WD-p#%0!g& zdYYmuark)5KdRjeSCM2P3K@f>5s1gqGX`M7!rb0+U6B_3N)J1 zVpRnjebL;pM&Z6;(}f={+tfw@ye(YHq7|5q##fLlikPROs#NBqTbP8`m{inJZ|Rts zAazOa_zLS0fsGbx%^_r8zxR=$UAK?j{p8%+&t=O;)LlEojZcuSFFU|)Nm8!LOkWS8 zu8XZq@3?k>D(oPQ#s|byEf=CSMJDs&4@A5Hh2J8z7Fg15`wn-K!b@{lG`&mN8NS;H zDyi7&&fZ4KrMgW3LU zDsAg%mm-A-kcQD;as!M-$VzM-%W_1~hF`izzDLp;o#h{tXYbPRRm9CR(s_ijazt>j zM;LW#Gs)2bcfzHL>MD;>TbmWF&g=7FRy|K1enreLf`1SzK7QsuyUG;K!Q}H z*uq^(kF5oJ$5&jBx?{o;t~T8H(D2S%MYC+Cb4aSBP!HydrP?MoU+T@T^^V@Lrh#hp zDj;LR9oGp-Mjp>jSv~}F7&Zz~6R_6d%dNIFINTY;lPU(v?}%e`m4?%7x-AyHvjGHi zHL|6&`av+kP;Rx)E-YAvl=A63!T^~%gyf*y5Sv9}b;@i+1U*~rQE~*Q^oHz)9?yz0 z=!9{)eAF?Vl^ralj??fz{ zCnp|ojvBT4!=UZ79TyMNrevBPR$Z>`vxFf!;A){{i^^)hqz>1o^!D1lqjy|OKm{lw z|0GOPQLvqXh7J~TpaOp3-Pr!!pM{*T>mn@tlTtpZT~uNj zA5Pzdn+1jgK^jsAj^z^~RtE|$s34R#O-u=+CPM+Fk^TcX#}N@6P0T~Q3Ry!n+YLZe z8v;m5H;X=|h87olW7K+6dz=@4#EwHjXsbq80Jo*^wcRrM`Z2XfUYJcQihGms_1;q8 zvCZpD7ffnB{c2`z&B35u8(#ZwRQ7!YT`PtP4qs(p%(aU z@um{@!l`}FlD_)K{t*l<+xp0)%3w~xG?zlPc;Jcd2hM1IiQ#|uyzWctl`5U-{r+kk zwg(k%3IG0EJT(%0EL^ti3K70FF@IQ@z9wFaJO`$9Hea)sWyo&Q-@ zuln)AX5Eh?24YcqN@XSb61@_(*S3PxsC;EJG;1;+nXFCaW0vOQq z`&a^iY3?G#u<$=GB0IstAkB=SK=Z*azITy%6-$s>>naDRD?q9mpe9zigAdz< hYL$Z3R|PoJIFgMA%#q@i+@0v{S_-5$mttewDUju09& zza50bNd<=+m}7@w?O_B62;)(1k%1d_5&(#iL6JKO`9cPq>Y=hA40m-{S0~oG#k2ap zA3hTy>PIna2FwNfPC+Yv<8+4Aht}#rc1A=3N(m!;@0DYQ z*a)`y^z8~2vrwcW0jv0PhUfw+zV0UCWrV!VwXr-3|TIbnSfG^-x@2O=&}a|8(v_N zj)V0ZcQ;%oX4_QLWWiLHc|P4NT~i+WfYTC{E%@nMiRpa(4{fR{Ay0VDFIlk+1DeM1 z#(H(2dZ;CnGd=>8p&r~F0_VUM9DP?BDZ;Wd|X#nBNqRH`A*BQ~P=oFt^2 zLQRRGBugM6x~#|}+bZlM>?2m9id^P6r(KcWmZPiGDIOpjz$BH-2D%09j#MXCCfcQ* zg8V3yQWim8AS1>+DkG90+4mo>Es!d`!o6z2MRl45lG{yv0%0-|nT4sQ5vG}@k)|O# z9D^zp9ZA7SztEJ*g^%lIC`w51L}yEUW?GsQns`?%R+tH;xQbt9laB}Qh>kk%;4LUD z^!BOu3HI^!8~>=n_AwhjvZ>;h#3aYG$Hd>VAGA*HPOfs++8BOx(4;=mI@0>5lBQaw zLZrr5$ZI&wW6bANI8-RjG22qwn%FMezRWXL#!i+_dgV^$7_oosy@sFEwMF{2Jlp>5 zKM^=d#>>R(!o$VWzyq=W=D^K#%pA$w&pc!wYO>Zv{0XhQuY35@kU2HuVocqVqA{AM zpt|O~lwn%1c)e1f+N`Wm+p0*haRS{SiBXSBb&A^ zscyZXKRr{#E5^|xS-oWiQ3c&5xh6&1W@HtKmD99Ex(+F1sw5ZkdHVT>{fm6Ep6Cz-qP3%Qu-37R z=+EiZ7*6SlbZB&7v>lpG>Yn>o1~Tmo%mRkK+wPh*PW)Iu_*q@fu<6--9>AArP`8fN zuGl_#k8)4?Lixf1ClKx(-ih+C}1MSlB!!yHD(Xm)^SYDDNQiBrnQVJ4&5+DgV zkT$(e1Gye2|}LK{M-#X$oGk!CVA zGMF;EkV+yyw>%n*Zi4s9QxZ~&XxPf9%g1d(n;x5HMJY_8>gWvE1gJWh`q_(^#oGSZ z%wK4r7Ya`&%2T`PShlCfz03cVjv>#Y2+=`ov~Ae1lCsGAqcZm8;BGH-&vBG+tpw!=ba_@B`;oEo}awd){on6wKGh0440>BwhXHG$8HL5ezj|zb!nRDxfp-8i8qgL zVl8Xvl&!UunlC*mJ_*K)&phZn@%we2vof^0*uCixGz$?6H3%p86@#E7X!t?du3E%l z&}419s(rWesDO%Ns<*uR<|*AM-GYU+g~;Y&P8&aYzs+Od zY!2ZBA%jrXhyPv7eMV;EA1k@Z#0kvoES@~Cz#GxuVb{@-*x|fd?t*PidowvTbu}BK z!Jt~h>sHtD)y|h!4N#?34UYj|Ku$pC`S-Wjg|F?oX+~#(Hpf3N9XgMl8{g7)tK+(0 z0uG-IUYIuS+XD9EK63D(qG6ve7`yyF4A%sX12*2K;mSW73XS=4zgfSTo$VGKyf;6jPFLmydcAa7nT`xx*-f8j2+3}gb@4ne+&dhaAMMS0*Xql<>~dI&J^}BKp67 z2tZ~w4%pqCw^7%2*H)0{HFI)cG%{w6?b z?e6Z(%f#g6<;Cd5#^~f~#l*tH!^6bP%EZdb0H$DY^Ko=H@n&#zBm1vO{?|O>7H(#) zHqP!gPL9O?<~1>O@^BX*CH?oH|NZx0>vXrV{69}}bo+0%zzj0|`-O>xk(ud#&kY{R z|F4%<%*nyo)xymUJij0Z|9>d|Z+-vgIsZkkZ0+Rk1eSuUjhT$2yM-%wu)E2>)d{lx z_vrt3ivQfR7%Iq+**{SQNvlb2SLT@)muNgQ%%pL7~5s7M+@?REa(QOHFXp6S6$ zd}*kah@i)`V6sz5J9}~cI23rBT;zXf-Fyqo%l+zRSbOPpk;8Y|>SiVAb}*LxAufwX z?M(bXT0V-4aWJnsNIA79_^3Q|zigFPEsgvB{^Dd(2amz6tVZDd_II(w<8;AwgaR~- zKv$)#w*1{65)KlM9|an?%d)%a&f|6<-+Iw@Y7R2+OiW|Y`_tmGd!X~Tw)FrH^kC?B zo~nxrP%s;cOI1@6&{#0^bKp75cH8^487L?J(f?UU{;qaBm&pH0q?&ePh$twVM#+I@ zwS#uZ@3NC>z0D&znz2p#WU=HABK$GudY5|=~#pldZtrx6oj3%XN zgQf4>>wT@-DJAOjxE+qzLmP>(J=pPxG9mb4kIV1vteO~Nv?a_-5CJ)FgK{*Z=Duj) zB>|R!m-}yAD{-cdE74-%{DnCk2(6|`!Iu<`ujkoCY?J(KwyVt!#UT!N;GM(F-bj@C zihJJu@eUJSiwvQmMWbq?Rjm38k1kHOo7BLyJZ&>sNf4KzX-YIvc6hDPN2S>|Of}v| zC5|ZYk)-Xa53$H7=xrXeTp}7PQC2oN3RFR~`M^3#kDeHWhS?XL`Hr#?^zkN5MUbc{ zm=s0u3JIsW4J=NQl`htFHSPT>1RX2$!)k zCoD`}dMqSwy2k;Tfnl_!QNZ5_2wI|YoXxOVixBI(qv>gy(rmPHq`7f_@R53_oxV+N zm)(MNbhlxC-ic;H_@v_(vgUJ`Gi{mWaDedmbeJav_to3D39KBv#w<*-<9s^;q(1qb z%A6W!3@y}c*}GlHJ-tbCj5;^Ql?cz%dQP2~If2i|(_Whahc|G})nFB!1}m{{3IAXd zUC&3v7cx=<->5AwPT!|}P>}?-LDSZ~#+X+E!=(MY+7Edbv?{vVjs{s&aykE^E3v=t zg+COs+MJZQu*8Ml?$lE2Jd?>Se%H=yR_qh4we^BUa`M|xzG@J>QA7+~k2^KDR>DM_ z_S(g(_2~W2xlEmRNvBsDK@_?^`>E?=Z_TTYw;?U3V0~|~sEy^?;5N;_vNCt-da`U< z_gvM8Hv3xfu?tv=!DY+F?g^RD|Ld$zWxeg%TA_VRE83XrTJ4a z8O8Is1mrpMJ;!Tfnc))PsQc3v6B&=4n)kG-k#%{G&WkeNa4%Vx|MjYWx~BaC&!J(2 zGEPYPN5dB&N+9Kz5Es?(%kUCe;E`y4So#Oidn#18v z_^N}xaL;kvI}t}))%bnv_qCpo* zewbmE>yj_xg(2av(iF+LWE>Kq=)5x!sR7vyUhx-SD5Q6HbtG8#C?$cvDfr5-ir*Cd zOLIJ<0Ai9zN0Qk3&LYKsQl~9I23X}gPQ?iI_rtAi{R7A-NE_|=7cFH^l3#J8$)8f6!8uo>nl=F$5YMnwNhEyYI~W*yh?2inK;Q5}JOm?RvHa zG4%jOs)O&jnGoJkhMA1`;0PS+2eBM|kd86=VfUtTW3^YZ*=>^AOV*V+*XNAmD_fg< zR<4)o%_{l*z($aeNgIpf&yZT*eq~4r=33jxKfOEf8`q)BI%a4;Ye;mQo(Q!NHcu!) zH`N)8nDCv~(E7O=Q9>eqWfb^i@%uA;=;kz8Y=N8Q_mQz!3xK6B>I{AgX+|uIn8Zl> z1gk>f>-p-h_)HEvO$k;}LK=JkX{2jv)|IqOnx128nL)d!qvbddGJ!3Jgje0NGGZs7 zus?3HB2oPHQ0T*@L*f~_JD~=^zP(CYS%r4YLp!nAL)M11M@Y4PaY^nl=Mzbh=8Q%L zi$KnSiiSNy7C>e>0Lbx&mX>Uge6ogH_6Akkcj)bS9?uMRx(Sn<$COaakOurwc(T?5 zntG1`4$#ri*If?Neq-s`vn)C$37a*$<{Jm~K){I)Sol=?JzUHKf8`m~*!93|L3T|A zdK{|X9Tmm5^%4ROlwhT)wAvwGp)PzD1)Wq*Ho%4vJvx{ik$9L3+F0tu#27EKAh^Z8 zWB7q^&+bq<^$5zpoZMI0fATsK9W(!&44JboS)C@W#hdF0>3P7<$Sjs)Ep+%{a_~R9 zk*C>Gll9LlMCsDv)#@%9)07ZjMWU6AI61C*(|Ie+3q0v##t}Ob$R|EJ?PRu-pfv`3 zcOIM4L1}O$h8InOVga&jv7ll!%I}bVeB{aS`gcEa0Ae79EdzkiJ>kgE*H>$Yv|Nqm z!cZYhgnX}}58o(Qg~^DA$fW{TkopAkd|;WjAfOn~hyb_`15%};zuNe6;`lCF`q!Z# z%h*Q6V0Gy~11VgdScrAsv`>U>!x`a>`{1;Z@E(x~xTlgvjv+Y#W_!UYNFzX{tWqXN}9q)=)U^V#?ns9p9s95qRMEZTsvUA4Ds&dWAkuc zhkpRa3b86hvB|P`;w9hk;snGgaITnQh&+H~16frgS@id5pFr|gawF;G$29-Kyh4G; z0F=qb>lCY}6XA4Wk31-dt+`$=`NP;7GsZ$Qi-q{Yyf2NSLBFM~2Db(pFiEX>a(f7L>vQT~M(BPKLOjYfwe3qi+mu_(wj?$3L=oOXQm;8~OKUtAA~r>|h`;L(~EZ()bcll|AorVv_#+ zW~;;wVZ7`=>V+e6AH@am{Q11p@C=2f$bUUBw)GWGEe)1&(mEMdb!UGV6ZF`@*5ktx z>K@u~f4W$!%U;X~yK`nz<2h*l?XnWbv^?<@cVJiCHx`e>^_+MDvG3po1;OSJnB`5l zyMJ}L&C)YS$@k9IF8OW93(Y7^oK+$#o+4Ot`9J)v?{8<9JGWOMw(t7S;h)W?=am zg@HzKEgr=_Uf@2$Pph^2VZ(2|$EF8T0GIbbKc#XF$rAACQ#n=$$Eb=ROHs_}_d28j zV{>0N0ID}k4~uZb1kxxnwe1UO5RjJmk7bQC%BNlQHo@X}Mrc)BvXjXdGh*RC!GTFi zzt&u%qe1Mu_x&ykZS+$4d{87B`ESfuORTD@%S-bEH-yGhAcQ`s4I$YsdJR`B1Yu}w z!vU)+;6+OwUBr7q5agx22%jN}Fi-}CA?}iNkRl7jmt9e9w{6{@`2rEShZVp}>;viZMUE`e1><^1VRz7kC zUM^T7a-{bp-2C|cJo}W|HECgPr*PalCHh`s6%%Y^;54~aarg$J3jip2MnIA+^Go`3 z06sbGH7?@}F@6<*79eWm8Wsn!^_}0pZV(N#-8ubD6zp7wB$pL1g?^1WK1ThOaGMB# z50yAE71z;u?;|Q#?640NtA%z(CyKGj11kKH*DF@MZc}6Ii4LZjrKx|lfofveXdJ&t z_6{|KOX4d|7K9*wf(2Z`#=Rac*Fc1JOtySu-!6ypYZ zD5Ko|PD6xV`lif19B9JcpaDeE{TGy*pKfV9K8t~b=!iwR(%2a>QbeCRM0PFwP< z)`xVw$=HPK5_}#iBf>v^)5bdgz-w~DfZ=kTk)eu&j2LrkSGI{~iz=5RDuhZYBbP+s zJt9`wBIgy3KJkq};VguzaIWe`am1Q$MSvEkpe=;TWp`ZDR(~e;{T^{dtWKd%!qw(C znvpZ5Xygub_54BIeIe>%p$adOYF<_}VJ(r0DS@dV!KYybU2oF=y@rthH?Mk)9GXbu zY5H2d1YXpzB7XO=Ochh*7_^Je*~v^$K`g0`tL-kyPm;5@We$-dEgljEA#388E4vR79qkJl-j_dD@OI^dH z(Jf8P(B&avNBP;2S?(em9cCxx#s#vduiGWEJ{2jk9hl5~X30x@w=y6~35cdL$SP`U zG!1P`!fXi&#Txo1d;vNMb-eAF;xpR6uKw!6WC&;CAQ=gWtj31{zOW?k&~8YQp`g?> z?nd%~(pA3mrde?+NNAe$yKG7o!0}ixxkO<$!j1$L)P+lzX9-jrwY7^!h*BFlJ;JNO zSA;Z);7~iet8_c>AJ0;Say|16i;*EmTMQM+sj1jcT_K_?Je#$$x7oM|E6s}5YcE>L z6ttbL_a#S0TQCmkm)V@Md`ZR5oGXa%NP-pwc#lq^Q z)D;+vycw=C<)9Eo(B4K)lr_S5 zydzz>^SfMqW070mq2cAj*2r`Zah(XgE0ERK-vC19oF5|QKI&UVqL}aD7e`hhq>I9R zo+|u`MER;-uBEQKUju=@j|vp|t588+FoOq2dwirpRr06No{M(bg*X{PdGK!e8^tq} zvxm+oGJhn*9Qx>aiWKNmBTRXEx2y<3i+KWFN?GzrK!of$f%LouBG(coo)HCSp&YTJ zu|^~a&_FBF_DV-9fzbmS;z4qbVEp?p_eq_L8+Zpzyag5BrqZY!-4i^+t^9~l(R3$Pr8@?(zCv3ev`rs1S!r4!C?YbG z>!Z=-li^g-gX}TWD7%I>(@|wK%x8eIr*|*U^(#SlOZfayIALq<-Ag?YjTFVo1|b0T zP73nPkGrSAkZisla_t0{{UbLjJQEfL8s)VT9Q#akpLl$z>7?+hAj~zgR#Z8Cpp)v4 zPZC6&GCX=r&qhmpo<$evrt0v^W7*~fLxlN?F2sOr-oLzvwMeIbsbNldD8i%=NDlE) zj@VKX^^RNau6!UEWpXL(gPz`?b zDQUHtA3LA>Qrp;#dH#Q4h;3v4ocWtoWRI6cH{f|kc3yuXgfF1*1+!7$#b~mp2l$Wf zDGluHjQy%*#NhuQ&?PZGT*X(8EC%(f&;J9a{6vr*|BqS!y9QAeC@{|UzZ$~7(;aqA z6N>OZsQJ2xc%a=o<4SAcC%ssGiv z;IXl_OXGtEr#O`(neZwVCM9$%W91p<`dkuYt1# zZ0Uv<^qd&L5MUW*H8mF?yKo5;5)OPIpi$Iwq1q`mb$gF`=XbYFrei#zo|Ul-wwxC*7{|j#ZuxCsXA8@h@G}Tw@%0b9oB)+VCb=9_ zXA`^q(v%F<&hZz@tj%m5Hv*u6$Gir-CPnt`@Qu~6g^#`hnt4)5%c=J?^jvASFa zv&lF5pL42?sOH3bDjB~CI|HEG^tD7^cI`3ERsW~2gWH}}k#qs`OP)o@?0%JUT#mS1U`%bIP)lGJb zikgN#7CcXT$rk;{Tq$Pp0?l*9l>=tXU0^6+Kiw>@d?#;V931$Q|HG6z-k(ozw!?}3 zfPr8#e1|5Lg+1J(%l1pbUIQ?Cum=wQu%Nib`zF?+=xV6(?dwDq*bj!_F1uc8s@gBw zYN{tPIYxgTnx1?~D~sCJ{PbU1t89Y8d?aGze;o!zZeukR^HNb+^AG&qq!{1w27?&9 z<&gP@A)f`yKJ}PoK%x zZ0E`nub}elW0rn78R2r;r|WwzTi^mdbzk{?yggXVDsW+*$mHyXLhpg`zqD1SZh?2} zu5PE9;z%~6n1N`EiiK)DY>m>o=n87ju$~fET@$^-|cQ~EJY>nQhahx;Rz-#^ITrt}< z08Z>Xl1mai0CNBko6<{_3sG0+A5Tt(4FR1X7>mQwH@^1|gXh@}MJK3hJt~ZLl^ExL zIjO8Qzdu{fc(KQ({+SEbmiezxykEVyf^FaTy^#pGl|5G-xNMdiO71S!`I6@#;0TDN zcYW{AXrwEXdXV{p;iT)eFESSgRJY@W-qUSav>U>#sRmU6+=H(4p-hob5x!3e;0V&l z1er&9tLJqL0*Ksb3ETT}*)23*fkQ03S%NlT9wO2e@N$o$o_`s_P7STtd3U+OgMKRaNTCz-%~N<^_C#5|-#mRvx&hE*Fc z5O?HuPEE<exP9koWRIFcaeDqI!1V^(uYx0nDSutfd(Kf6;T`2l;0ah z8TC|oQ)5DB3eUAoxDXo20?&l6YvIGSaafvZ#;0ADCSj@R>^juyoe$6^cG8I+uI*mO zHfsVvSDsE$ex7-wWb&6sC+KRbEPl?)*^)-KVeW-*AAA?dP1W`)PV> z`A$B}QzYx)pd|KoN6+{k3AOchf-i@CJJ;_b@n7}>nQLvUXZmIooI_o62}k^FZ3yP3(s=u(m@e?G%n>5Zzukz-*Ym(35NMrDbW{QFOF0Ub{ie7#H}J9)QU3u zJL~%398%5KxrZsS&mK^%gC%kFpX=$v-@nY@y6c1SUEb0~H!vWK;357CW~)c_%P6^w z56Mr_3aO9ZaYWwRrz6_AU2=kRL)PG>y9FQYXB>_#ad_+gs4_54u#PKjAd) z0f_vH^!#i=C#LH2bO8}{qzAF!!$j>c%XPb+a1S;DM;mN%;y|s9-Ns45)5n4K2=uM! zOC|-@L6;k*%qG=fNI2RF2#M_&&x;2z?^^sqtjfx0Hl>*5)@(CB&!mNxj;b0bqW&Sh z(tOv_R)L9uYl%_gWN*}L`;D%!UKcMFbpA`=Oo^JE7j4FH6E@+as+<-a7O3W1``kii%yzQb9aK!rMp|`Y zAL+Sd5-tt*M;=fA2tb5PB)B%HMT}Dp-7`p`kcZ(dNvjhX$A2>{4X65{l4`d@82GrI zNV_k2^|oSHTYZy_pNgVI2l^Irlpux@iGg9>7X(gnW!Mqkk--or%GQ!@6re192?Q%& z>;(Uh1;}^W@fifUE6jGuUYrdfcL}PHb9_0l-m;05jDF=38!sJea zM}6h6TcB5kIg+NZO8i!B4Nf%1TbIf}Z5aRBx%ofD8+}#vpUC?;~0HL z&@!pzHJxxphf#X^F~c2sG4K&RzA5oNe~8(7inuiW57N@6fcTRvNfrw_*Z94>0Lh95 z#iv~2Sdm328)!K(X|p!Z^OG(RL-SIf#i$#hvjjwfo}kD|!`uBTpM++y7QI2Ql%A&Q z)^76+8ZOxL^-e&$uCD6@SnGg==UGn=-%5eXG`Ptc0_jue8lV}esmJ=n+%~ffpZ`{Ht3^L z^z9a-(RK6zB*=xicSP^1Sd+U0Nr*nA9>GBeN#WnPseBr^oAkWH?_%3>!LG$jwcan= znIv!A$L3^jF`-E4??g7YijN|^@EGJWWKwbN&qDZ49rZnx8k#zTGC!^HmshApwQS^^ z%f!eGL@nUs0@ulTe`o1CT67ia^L)4ZkVTowz%Ah+KI^!hs3H5I7J*JgLTcLFIde?v zdoUD-xQ3~z-Rsfh?)&;GLYJ%AlGn|{?6}5Fbr7_ls>HwKjvzwt8N4A-$Il& z_CheuHQ=RRudC{Fg@=yt-;L#3tErgRGFpFPDXZ6a>4m}mgwQ##|FXj;8vordjb}l} zim;RpLJ229N0yYSPZ)@PC!m~f0kqPQEBVTUnYH}xApQh_e{>6>OVXt`6AMBVhSlC) zwZi>Qs6U5pq5lk>s``nSjNb|}gmF|;&tdS9AJ4h>7Q7k(g4hpEs{E&MaHI|DD%rr=Xdqh;Xlyrprp0@SlK~?{+!g z1E?WJqn<7exXEb^eG?U7Ui)O+)j!`8=M2n$JChF*e>(+{(ye6ENzba;P zv#0`7&RsBrZd7d~ij(4{d#m+a%hB?IS)$Kh<_$*loEl6=^#A&jre&(PazBF&W@DPL zi0RLLAtB~b4yLY0(_%EnU*_u8xb(5!oft=w2=?bL9N30vl-SpbyK-r*eb~=qU7N-G zBlcO((-VUQwQBGh}+~5K%fsKG#m>~`t@d!-XO3{5FFWKPSxXhzwHR)(vYqYfh$F23}d!%%SS`Y_Ca>; zdU==9u^sWfIrkD8o{`fK@~*P~F^KkfU7BYxErB($R1qn__;R1+)D3cB>Y$yB&7*ek z+B9rlwY-2YOskt;yf~!#HHT~5%=>))w8_}J-NQ|;)Z(Aq;yEsmr0LemIbydq{@d>M z*VP)8L6%Wc*7)Q@ypfwxj8Su)%)2JsN5)fz-Q*cKhKoxk+Q=O&SzWVwS!O}w|7CaQ z{^;Vg*h2s1xkjLIgwmow;Yy5&Do-#ml`G2*P^t~D2;g4k#(P!U2MX;f*axG2dxab z_OZZaX)QD6TaLAWJYM$c&+D)f6gX{ZL5AKHxkjoUk$Z<`NOA}b4Qp16q!13q@+?(tm?w(d|t_|-ZKE(1l)vO!V^A;QgKq-!6SRI914-x7N zBsZs7y^h#O*eLq-+cD8;fkCtdU}CgZA<;0P281}lyD`XrGu3HPf_WIFd`qseuv`6W zNm|pc#Z|{fdrzmaJ8AX8UpdXl(Pr6dlF(w=BCSCx@`B;lI^C*$tISU0Q*SaZC0rNF z((UeAnrj&vFy|(v(j4h%h!dR>8$ih+@-DxTckDJkNeg?T5fgpk|JPIHZYMNUFXLNncAD5tv ziIF(|d=1B{808gUC++XdgEJ90*71~x#b|L8#0hMLT&x%JT&<_yfm+V2Af=LSjQ!e^ z6ra#{-ubXGv1Lc1_jFsL6?sUYJUlY>>!ctmdrco<+7Lov4jYIC=ej(D1fasV`W962 z&parr0pM)l+L(TA{>`^<5dBIg!>bDL{MK@lepT zR?&J^nUtzOY6va~$yzHNHCGb4O{!|2z?(5`L@p6~X|AbiHT+5l-O>L1yWKn)Ije;q zV28-a{fp20_S|Ns@)Eh>MwV{)LFkn_Zz6lYnBVNT!Z4spP-~6UD#z2k#vU!Np4@Zt zO=)U9h zv$pKFC63?q_o?JPh1V(m+(n_i8ER3uV{zbOs<}q3cGn9G(~qVNRaT$V>R;JU5Gk5G zEI4$#f+l#7x6srjZUe%tPb!W8DoH-KFUdlH-uBC`QzTDaZ6sS)SOtPR=f0+uRatgH zTbIM1t2G9EaL#p^hO|8xfzVM zq=26hQ>~(}T3gPFO7o`YEs}8k;qn>>K1a%S$O{56%C+O~1l&!EY*3e)#W7*?s%Z0u zs~PUx0|9752OFGlKFT>85pGz{fKl}(F@0aAK zuT=;fg(%&qC^B!Xe!+|hxEqm&ON$e@0rGsJhGh2l2qD-0KoEcd!0yJNuvmI>4sHAg2$WGjq#wX9V=^gO zL})r=+H2r4Eg6l99WL8B>LIDB#Fv3$Xc#@=q%RgylTl4}Y`9J=VIp}36$6CIP(1|> zFhI4e&ByLK`}Y{GYKQ_#07{Npr*HI#mQm0ytl`IeKBQH# z-s^;}-tC?0JZ=KGK9^>SH*PZB_!iIMd4FOWpnl^;wPkZj@-u0i{!%%CcXvtiY9Q7ic!0nqwF{uKHreX&TqfP2^m*D|LT(N~$XH zowHjlsWIeJQ4Tf`l;XRYUW>pHrr^LT{cL8K6nMxk3vBTg-c%QtSH z5g&I0%EFwjE)npxpFW|cn4gxbl`75`nljn;03qgIXYYjLqbdHle$q2~We!BrZc-bJ zR+#NFBIgcMwJ}mDD96T$libpSy#M)N1AI+N$=La$dVS{YXujBm-+izVIM8@1Y9JJ6 z0Djw`9*YE04g7o)2EU)k_N`9W4{oyEaez;s&WYJT&J3@&Rvov=&~Tlgd5!vCr!-P0 zu2ZwFXCnf8cgegh!A~*uGFIX2tN%G-}hr?wiTC^?Cnm;<8a+8U# zZM)TF;?9p&tlud6KA-po24y@iurKE?uUrToe}KFAGlhSQQ&8eg&{>dKnZ~IN{zX-grhnK ztz)DZ`=4q*n~*4s5lPz;ju}X)l@{Fd{#vB3M^ayohxE)Y>r-Of6?ksr{*p0KMB2Kz z2dtHpTV<>JrKJruGIijcw<(p%yP6Gk5}Y@9l}dwh3*S)A(aKrt{O(^agT%w#&+vZo zHQ2EqW?g1)I@!8kB^0$uunB#T+BR-pCp!L|C^o}xgAr&eoBZpU73nn|>m?cDlxLT6 zcY^<23S=0lMk)X60*ciqooauYqNW2@GCbCw+jn>Tfj-mCYAhSm!0R@`y8FIVa_;lW zE^s388U^!SmF6NWNjgN`18hM#w$SC93tYamc)wpIJ`Z=b~IM zdSC=KqSqP6!EjPz4~|BNiFYDvIm}VTXVoq>1AI_C%AQ6H-HAE7D&3}rt6KD2A|sqc zg&3Mc+wqSt0TACw!6sl$`Y&7I!FZNbMfdJb*&L?V&8+p~)%KU)QnBi)7y{2n zMV_Zf))hZ*%fPkMjN@Y=^w?B6?yBsTW9jrthrX5Mw3(a^neCB>TsAYsM~}Pl@i;?s zvObiAFeqO<)ZsOY`Ibmmu-{lDMLzSLF~4aBj)abT6eN{OLkRyQt_VUuZjk?NM4_C> zSRi4GyncNGu4s0dWDb=vDkEl81>*F|!6+ok24#OyhnY;&MN~qP)88mhgxoI!mn^I3 zo@cU7G0yw8S*SEs5>JBnIVgKgEWahGpaqLg?+oo9+Fg>>Gmi%(TW*o|GDn8D zUN^mE1RK0KzGlAI2j0_zei~}X3G?pFD zmW~p7t&x=G`r1-<LLcCq0E8 zJoUw<>Y3!IpT2%zXX0~b4ugAnsmL29du%|l*GHDeLU%fS^U+%WGhS84x05^j<1`?~ z)UYlsypO|--pwG$&3v;{*XqubbG=x{k|sQqkDvi`N5noSW4fUcw&%6+p}b%>qg`_r zpvgfYu7!H(RJ+~`5`=P0luE;)YSg%dgI5znvY%!k345d?LO{^6WEu@F z0lvUCIwx0?4yYk??y4$F!oSOz;JekPdTJnbyV}ZiXr|Ni74vPD&rE}^M{~fl;ZUzh zVhOOOh{SD00e+h^*#f7K!52Fgo#?2)5Mr?DkQs=$_Au7`EBJN?qW?TQ*MD|N@tN;y z<}1L0M-9N6!G~e^1|3{NLpUq}k>`H8pwQ-ZnJa@&GZIb|NKKzgQXo8{)2ngd^C}7< zHF2oKe;rC^wfj5PKwU^^=6vO6 z`XLelGpb}rmS9&dqC$okE`!WtA61L3TWhEyK5Xl)CssvQ1yz`DO9>g}`2D zIi4wjpsmjfu27LIMqiXp^JW<(myiqShFync)_CZe1v zKKD0b8~N8qH9-i?h>4~|yF=pW7+yR3D_xC|ZwCg(7})l%$*3gklzokiR@9Z|QIF&# zGdTvA)!vPJ>R9E(V2nbl{6J)EA#S*jVwVJ+v7$Bpy97X$>up+` zycY590WM5xDw_zqwDlV|AZt}g*d^zcF-dGYHt}bqM=D*JSurPmjnKm@5n}tZE%I3dHCLBKU}~yX%B+$g zsS}=k&>;v`2VcJOO~*Hdp_*Ocn_5Lu{qa^xB~E7iW_p+KAezB;sL5Qw{qoT9JTZ$1E2K*T zi7H~wRtvLqU7{s%*sw&~D>(v=ewTFg7RLsQCs9xnYhXi;nJDN!mXF zqeR9Mxr0E@xdUz(;~0_Fs6e`G&ABaSyr8zp@^U||v}S;7B%3m}h-*fgwIQPhS`Hkq=0Jl_3jW~q3N6s1eSd5K7qRZ$ZKcgcM+WiO5RW&a;THAj+7z9bpNYlcvHRds1?AtW@T z4}RPa9iGLM|BI-zaEKz>+>-?QBWyE=bWZmVcwOeXpqUy66{wZm1Qu@|HF zok!qr9toeL*(kY@=DTe-{TNo=h#@?-J44HfX=rs2Vp|ek!~QtXtxI+4FLe*m-8@e0 z2j6MDikvo)@(=;?(Om5Z^Xgo9b_FO9%#Vr_HRCnZMM`&)DH3Y#zlttfmlU$?d34FwRFzKzrefpK=P_0hkJuI*+Kc)SJUx z#jg#_$RRODcCMl)T%FyYM)fD_#fd~hBSVW4%5Lu|tCV`RLLofwQWy(HJ_w|KL1!5* z2^qYHU=+$&W1RWhboQFa=M7+@eJ9Q5iWPnnH{XAMi{iWC@J?rY!?(6w3Y1zi&H)5| zrJ&O{3;I!!v}Nl7w|XIMhb-{0x~?`HL{~TEoIR9gmf5;r~;(#+tLrbJRWa3Ulq`x*D$;5;<0Jjk^=e-FcI-O+t{ItexPd zA=$^muUh)=4=!bROV5~VGca+ndw2;&0`_d$ojwq{n9f8 z^_dXlc{c>8(9mH9!NW6-oBr*T^8I<(4MU=Enfzff2}#ObExfU7d3aX)*J1A>F*m`a z?&qvCdJ>Xt9pr3&1rQjCjQpMfGRWT>1|MV|RHQ+sR>_|dwBeUBd~hVPJyVS6)y(QX zHOqv$B(h4CPlBsB!sF}WN!P3u*TT1IcPC8t;d_sc&7lj)VWI6y%Ic5F9?#oF>GWo= z9Uhn4&2`L)D;=4$JsyD_KDZyyy5q6cBgyth@R|4yDmP>}}wWjt~&1^~xLc8SeDB_rDDarx| zixv=cLbZ#mB{{NU^fCMer|GD^$`p^H`=>lnTpg0lg38UwNIXy0yukhW1e}$l2b)X* z8=}Q&diL+rFWwu)!_O^h(ob8%!Jhf@rK6kujPz3UsfJOR`TBFj(PZW-fkp7E_1pQB zLwhJPgCfCR!$k5Iwd}Cm9h~Rp!`Vr!qGuKobY4Qx?w9sQt619^|WDiOT!+0 z?6K(@LVU}V8Me=7JtMy`sP8u0+PDQPf%TVpaSQQp3O!UGb18EeyGz$=%(UUf+T)W7 zQ!I~n3qSn}rscIvBlK8};KxG=_5E-8*rqI9dA2rwczaV`kkyvbm=E*!n z8e`p^95%?g2epVJ><+|}5S9$@yaSZlDo(Z^T76`_=MFJWuDY*hyqo3_rDZ4EH;4l_ zu9-Hm^o80ZXnA8j&->__(fwj^jKL1KXobYy@eEN_(S{9QWPw@Y(@ebZ0Mb6^79u zvWgU|mbIp&Oxrd=StiNIXqc%Now&=ad~$JsAt&R9A?E5ht0p_A;%a}}cKOR?FEql( zC;18PAK1Xbj-Kyn(9y|P9~Udn#M?e0n^WTHtB^Mx$-6Wmcg_J4589Ul-otNJs9~jOV6(0v?K}%l1r)f4!qGW6)?1QD2aKh(~sx&Eqe1 zUu>`&WzHs%908mt5Sj>zW_t$byQ5EO&v-Bxk0~1wG3;MTsY2&RU)6fNb6Gn3LvnTZ zm87gHre?v1`yiDdM$h|Rc2wZVUeQBnlxQ8juV#X0WoU|fkxreyLlFb&E~qoDSsa}@ zx${!fy^#8|^#;%7(&Op`Hl}s+MZSemW!8Sh0l}ybba+lK+U@>gI>yqLg6cwDOV0gD zU}WdP)l>droNvB{Z%zecn|jNoY5J8%WN2GWpbb|0zi0NJ^)ua02WZ)=GjcOAn=o-f zj6NpPZipnND})uW3u_xv;?TT-kGJvOx?LM)VfxQ5G+zLA;Sua|e_khr*d7*nmzxm~ zv~ z0s`4U$($Z=nI;2ygy~9l#wLRT?I1Ez{SVbnn>vY7@y~eN9h*o!c|S*MmVt?O_1l$( z$k$)Pz$HA$Je9eg{Zv8t8NSepPb(rKlD}M|83kL$r6$@i9<@)ZhCR%IJB%RG2-GL+ z#7#l7h=z?n2;L7iaKKFYPMZo5oi>A)aeVyyqhtm4wEsub{Le<}SB4r+%H32E+qfcU z%Qcb40r2g^OfARg0H>Lbqj5%Y~y;L@1t9j7K?U z%O!SN&xo!Kqfp)r1GH&xRCT*U-;9~wbj>aTtbGobv zT|)xTDInDqqpzj=ERdj|Gzn*ZyQ09rV{9K&{aqkb>`x|JE; zt6n3biF>u;#rhq}IA9#*Nez}GdRArEZgM~MJaY&Fh?>-(2z5v z;G`e3#D#!)q!}weXAR_2M1&K*w0Oq(N%-O1!Q!ZYaHzp=uuCjba8HH!H$F?#zqfww zUHx|yiIm=m0l!g)D_}`uR#dNwaDQg={p7sRQDN?=^j@Z^Ydozo<*fT`%pR*Gtiu2i za)IA6^nai4cfV-uO!Htj9d(ggF=9kL%{J^)ZtS5T@ep0TPCwG^P|la}X> z`)GFT554=w`}x9BEgr6tbiIB^JA@o5lp{5iQ5d()FlUo8QiLA>JqzDG+Aon-`$rY# zZB$Y-*JvF7F}|rXf~!=nZXEvpo?{SDKuElk;n>xS9F3aZT2JkuUz;h^TZ03)6v(d{ zy9iE)+xWZk|E5d!4VoDDmJ`uGTm3U-mS`mFdnNBw=pFrqK&kMiD#@bde2|n)i`)AD zx~IFJ>!MAHXcQ#h!md=^eE{RMAO?%ym_Nh2G8XZ!lva1XTdzod`YNQ;e$3l?ktR-s zO`0k?t6RL;rK9Nce2*s2%b@ElGz$@$^vIjUA_j~5mW#WS^Vma6%+FOHZA3c`eMWY5I23T1JGrhM}z`*HH!igRA zk~c+q&a(a>AWcQWds-(a>#TS)!>PvHsVbhnm{Vb#B|qAPAu?~RAce`Q^5K(wmPUjs zm#04}mJX`MJCnk>xTMg5DG~pRFTFjoFDzHcVdJ-#C01=U>~u#~-T|Q0@ji0Op%nOYPsY5dqzMm4@K7r zUaI59Y}2lrY#CcBVd~Lv&9Dh+yw4w^S9*KC_wjjJioN$8S=o)) zv&~8~;vR*cX%_124C8QYj>RnS5*IZzR!~%a#0je#ZvYAWHJN#2QEG4m7Sg9zXBk)A zEae(<285^!DU}joaw?T^&S)vl9{ECWsU}O(4mWx5XyC{evjJxN-77alA>n4o^3vj{ z%C0uLGS`IA_2+bN6TgK|9+$*@4c^{B%y&vaop#6~4!=z+^LJky_evszM+P-;=QMa8 z|HaI*`NNgEP#ZhRf?MGwxUGG&IRP`q6ZFo-{V7M^m&=$_gctvI^iMYIGoJ>XuVF07 zg07~7t_9}mY>@WuaBkEfKJJ#>5_1~>57}e1@+xh#Ky#F)3+}X&oXn%GNk;d{60}^4GJR9!$kb! zw|O!~;mM)6dAn-zT#%A+ed=HG%)nWPa7lWU-s3hY(P+tOI*K60`UClVy^|YlFCAA) z1~gbPlGj|wor_GcYM1^39=9<+6r(w+dA4yo0~5Fk5xCtvds5R>4%RT-_S;AieQxEf zo1W9=;@enP=d+1M?RETcEOj~10+J10cW%w#{eu+oHgLU9+nTjRqdnvNW&7Rvar<7` z{x_@1?C$NDH?Pw)(x3OfnHP{$J+IW0vn+udnfZs{qz+2LA}vih0ZxX4@$xrhhzTtH zKtzs?IxB7L(2P5e+2Q6Ue<@wEdC|Ly3ru!SMEgI-a>8J-QWYb^i*0KTy~-&xTvgn3 zE$oqa`!e=ybpL&h}(nnKT$Q=Ci9*Eo)$B|y@CPL}_;iGOkP;yntdBwDLhixvD! z5B-V*0=(xX2{M=VQ!7kUmBbuCAxelB(8 zP*N1HblJ+isOvQ$H5%0t`*CNW>S=;v0EmY6PAm3jSogWg?OeWw_a;iHsPnS3(fBmh z?LJzc=IQ!?ZU}Kms9;(2LNB47N+8Z*iUX?RJCs5N8gt*k7q3UCEq#BI`dn&Tq$hkM zG4CDH=i>pd_VBVF9a=Y5kfm7qlHI=bo?c7PdS~g=BVUY_upm!>6lIm5Yff$WzT{$; zX)BkEawqc;&ppcPLL=foF5)}lPwU9OJS|J#iYl7Y!&S^HSHcfVDoN2UqIQ~| zajqsW6oFEQ+*9ex<-OFg$mFB%DBZPS4BGObPv^h>V<4(79Y-^G=8K>c5?KrF9^dN_ ze{;XN`0U#~T44oG0}P_ZgVDd^_a%5n=y0qW=x%Yt2weg>-if#StzbvVh}^=zvX$1n z&OwG+iE+xr>(V6QPs98<&g+LP-u&nvDr_~5(rui;>_yW^6W6Titxub6Q=S@^vzjP?NB zIWBDpIa!Is$_B4q>m=3Bph^r@Hhz5+LmOJP+=XwRt~t)Hx6OuvZDODN@%gp7G; z>b~}9PPflxrW9oAOJ5X|6E5`Uu&&A%D3q3DnpDe>wB^az|BuhcH(Rag4^KaGBIRm& zmnsjv`%d>99ae<3XBrpRt}+mIF!ukwGq6lWv2%zYu5+j8O_%MMl{F#rPGq+ZZy^Qb z8TktQZPM{BzApAcrqr$AVKB{oCm~?(vHqFA6|W|I1>B`iR!6YA^bZ?RRBZp%I8w>YFt8JEdx{Mr$(o7QrqT()emFY4&(3=q4ABy(I=~SymUx0;+`yc zW@Fqk7tr{P6;>SB{CW78e=qXh_F!x-8>rLlPUr}k{Ymw@{i}*^rl&IS!o+!6+eNnV zAtAb{r?xTHK1#FgxJ*RzLBpPZWAh!GJb$ryhENkVA|i!l=-kvdMNs;nv7ZqOkYl1Q@zje_n`te^Gce5F80kaX8uwn(ol(Ba=JbxB`(qHT|R#je*Vk$@6ZSmG@j{Be5L4@`8IZMo;FZ54y~+B z(zCwlgb5k`TvZaZ@!w>ckILHMi1?vFO~!&3U=DLZ$7?^QZ6` z2eM37U9z|wUP4-JH$~sAeM(TD6a#ZPKT`V?b&D;$e#OTn=8uUi=^nwE;ASd?dee4O z7t~52VG{l)&3Sfk=T%lk>(RGT_Hh=539cg8`46e2ioAwZQ4IQ*w_<#|Y~AljlG~q*>c^CCp-snBxAFWLpo9D`P<7cKk$RjJOuhh#%2!*IZOTLj|HV*{K28ca+ z4yrCYI4<1wHBpnj8~caLRE!3DYp$i4r|vwq|KslPEu!kLv&Cu>-p$5O@Qz``tni`k zyDOS_Rys?i{AqdZ_B4E�)RnUQbI-)c;5+s)&_`VUjS;%=tl_yXPM=usGmxwJ{gK z;~aH;kj^wX1skozpS?a#rF1C|3_)-(E-4Le_EmC7%fdi%%;t&UR4Eq4n+y8OWPfE0 zYE*%9@WsryJDRo`V>v$>2)%?kt@eosYyT)5-WSTthb^_Qq zUGDyW{}hbE;dah2Fd80sT1`k}Tl&VH>Rxn6PtwoB<3oXcn&%0}{ ztMhyNzl0eYOYWGB2tI|GyTn2@6K7SCnyZ>$)xu?sCvwCP&H9>HWeDjL3qFd1XwAvDx z$KY2{KOu#mDco1{PoIg7P>B54X+Zt<<;Xm@)kEVV9~iQn@$d^#h>X;e?qkwqc63y| z9Wlz1n8it$`BOL{8yev;H&TK#Z#y%*x=x99NSngcLYY5l>2Xr4J20F3c9I>-SJ|pn z`1apBK|@2yMj+!4f9lT|Cul;!X%Xc7QuntP#T6@q$zOSk-=}V){#OTeHrcP-bGBcv zJg|p~xObmi)PM7_8-?TvwMbjn&ne#I~5 zAB=1vY4H50@1)$CCj3O~x(BDC8&-z(8=h=!N>9Lygmwz?y%bN$4u3tamq0&IiEhoh zp_2yUscyTL(uip~onL^92^xYeYXM11XwUNIW?OGl{B?7ALsyk%w~@zjmvrpZIYQY+ zT5qV7E1q4Bm#yGp_ItPK5O<<57Up*aZ=m8)Y}Hu@#dkrcyb4{}ei6Hul!u3N2DqMU z@ZddSc9RB63`!R_qN{3;bXJK^m_`7DOSQH_p!&FBm<@8H@VobPZ>l&EAPp&N)tFHM z1&ONt%07xwi&}7MGTF1=bE%6LN-oeZL1Obth?euX^ye6$!k>Bi*wP*kd6M>n=t0Iz zeb00stQ8ITxf9&TRzGUziq*OpuMHa)?rx>nj`|Xrcuz|n0=Oe)8_(mbA0lOz=>YXj zJ0TufdXcu*fbPQ~WjpsVtMk4Fw?-;ZpBw$&bko<}cf>@NNiM{?YNVa7Wl@-?(`Z>` zZA$c-d6x{W?eW1wy})yvGwr#=sv}2=johv>6zEP=|(DCCw6i2P1xKrw(IXr`3qf-bC7FvF^)d z!yuW;WlxXf>MF!d3Cj{O)yTzXO~yBde7AGVg6 z?ZG6?;b-h`468pUI8nIof2Q`vh>eqS_Yrj!$vBZMa< z{XpC>y8ctJT-mYtX@^N&saCC2UaV}jmA4kas*46L3+4tya2%>e`GBrpXd4AU5FvA# zR2*;f@^}ISE?>(F-Q%R=o`qki3a;K-d3{NPfmccHaBGD1aP5Nm>f0*j9a#VSTLuc{06TeE~ReLQ(|x(S^3~|&hNoCFnTOC`{_jRo}-9-xYxkkp{ALAZ>(v0eG>`LZ58;X)i9x8RFp-g8FBsmtKP z>>I;my*t968!?CaY6HtaYT%vra_8g9&<}tm2UTqw2lcgS-UVW;xsII6XyK-4;wv|Z z(u@4Qm~>tEHUEoyrJ-8a{A?;RL;@pGR(7-QR!*UwSkX zvq9%)0|Ja*hGaLrqj2FKrt11fiqGM|4e%*9+#=K-9XXW_n|{k`Iws zcLqr2dvbnXaKg_67wy{Locu8HehA>uHI=pQYvzMI3|cBX-5k=I(wY@XdDczmEK#}| zm3U^;mf6>Ma@ntbMH@_hYbi|AS?Nj2VfF^Ve8mGj&Vim^c<`DDIR}6TR_uM9A}LW) zUBe+|S1QmWYbhS%HIygw42#TIJ2^WQ6?gkfGtHyOBLr0Y;(dw4?M*_ZA*XN*m@Glu zL9r1xRjDl;SSC#iGjb_9N}GiDY22nuI;G75qes&Nbhm6DQtb}mO_S%9QsxAO-^~sJ zsa%T5O!sZxuo7C6wi}ar05fsDfb~86&%z%K0HwJ*Qn{$etPY19E`{yyUVUF zJiTnD{BsBk3mb{_T)VHCXsC-|vR=+tkLpeV{!~ZgJ51UZm_-MC~4r zL=DFaJ5B8S>j{{OQjJJ{D?aYg3I(duFLl~hI?5u)gu2Gzap}UcfA(y4rwgN~gm|&P zlvf6=GzulX?eKfnslA5)2xl$BuKm!+fpk^;LK|<2kr4(!Q~3DPs~a#~A2_igA#|yF zV^%Ya;!b8OY6mI9wIe@%!t;wg`jLwa$ILEN+0r;DsmIA=1{Op*OI{CE|63`kp5A4X zVOneha>s@0wyd5PS@9Itgzp_+P9Z{ zVu>YNqb6qoFttRb`%=RpP6r0g1CEV3(3j*!)I!lS#!G+m$v=u;paO0p1eZyfKihR&Fi#N-Y~fA=KMoWW6KsiyjU0SmPx2JoqwRedYAj+-auw zK3>B_3j{sX$R`wlIDJXqy-#_PYTw>}?LP|#WJ^;H9ierPYtr5qMdf* z&DEn*^K!gjt`;1!j0Y3-=+YYU%cpr@SSw=?MNLqudsmj~bx%DyE9(z>yG4I#o*}(P z@lW_KAcN)$_^tTIPmcYIM6Qmkb-Rhv&_X$ECE|=R+lGfh3{RGzWZTvU-0eQA3qbZ) zcRO6svG1fCm6#7ps!ml^UwL?b`w9`p<3X$LijkOL$v&k+uC3)3A1gPH?1!bqWfreo z=c=<|KK_OnCk1*xXu+A}ya2Q}%EVIHxA_Zfk%Nh6ul6zDG#UcwoGLA)%TBDD9R6xR zB-9~0x4i{S2PTek`k}&3vH1(Ya@eR-sLpjiB<@qAW9!%s7Q0GV;I!tWx^DE>^8T8g z6lyZ^pw%{QKmCk~3A_mktKLb(SHcdpTt5|EH^SJEhKz&ur^OTKf7NoO6jOe9-GI|{ zh<&4RK}cM)BW}&sXHfK`$X_)>6{DmS6TYv?)9R`3_F?$bf0F)FldnKOEumGphQTB*kUFwxVev6yTd!Ym z6^i=0XM<}DAD3JEx4rOhs~sgdQY(8bg1Pm?-N05%&RMbE&P9AWjj%yVaNV;Sk)4@WMHJL!fBWhp)i&)gk$iqvc*tdh zsJrmOFvH`E+88n`FA=-k#H8QV1WX1Zg)l8Pw)GK{60j<~(m0RMzVN=fk2J#4RTNoa z|3&Y%D5m7Sl?YXQnM#YIO0ueP$BZgANz_r44Mbb8uGWm-Heb{~X(TF!!8Zr8ycHS= zDoRT=7U&9O+Mdydb6Fph{l;EoY43$i*_2X_zJ|(Alz}N2Ev-vQ8QyR~QeFOv@a|T4 zVF)>Qj7!;PC@P%(p3aDwlzyMBC+@Poyt*S@yoZ|vIgP@_YaD?BJ8@>RwR7R`2xr{IM^ zCB=E@_VEm%dv?!o22RAu_p#kR{}9}n1z(G|3xAdOH1*9W6Sskof!)I;cr*qduVZ1G z|Ixm6LyXV*Dxz@)w2wR$0xpFAVRx++1wqGg+`r$Se?ZCI7%O3EOBgNCE1{7|NT`+< zhVCe-{VL%WydZiR=h2)t|Cjt%#j4*_kRVscVsT$krrxi%zfe*xyJoW)EeB5U2Z+ri zOFM!ld`3gHZq^Ffl#e=HzV2>L3p&H`7#nGuLG*sc4rjk+Fu{ents?e1N8W@_!D&Al z%_t&xRX5)Y|FbNZ%yOl-W@*=!@8osdkvX|k;lEM@?&K1_l}OnLZfcEbbBnB{GM8E2 zMWA(p_Ha;~+A>v|5q2&L#7eR^K|0ofVR#0O$6Nw3XzjuU+C-Az`pdYL!kf2eESH4f zs&(VimJeUiZ*|(t|0H<+C!i7c<>M?k!I#HbKXFsLLEj6= zG*P=66L;dmAz5iNSiYZ(W`J#Ul;0-J4QwmpB@g7?N$TGnmr>oT7(5UZ>|5au-4=cH zxw5`jCH9Fy{*X}=@xH*fUg72Uo`}x>6>{EQ37^*}&kFAzdo7)kGzZ;ck*s6?G!|cv zE((PzVwB6eOOGC~Dtxt=XBWN`9jb32%S@348Gm?*>}XXudOEO$t#Y5Yp}He1Id9%4 zxZl#PCU^UpV!sM$m?~f-;Y#~R1~y38@n*F^db@FOWq;J@Oh!!bY!~rOy>Wz_4CO~S zZ^t*_nLi6Ii%(dcy^1dO>Rb5~4o+1PDseUF_F_YmQYDj-D`jgl$3cx_1EmhO6wm** z$m;9dW@yIRpLRYCfL#b2a9NYP7IrlKzJ>TDJQduF(=nbk!OLbb8-Ivkn{b=^e{~fX6{Kc!dBNg({^guL z-ZUod?W)}B+`}>Bn_%|w*?6;yRjV3NbHvYq)7WHGDZU4)6+4{G=r~~af#~O6_rAh? ziJN)mjva$>{&4!KUTyP3FkqiA!fQ_l9r7#=Y1YbrUnad2bZ|9g#k&Dy_35l<=yK!5 z3@|_dtkf1B8meuWb?zXl1a9V-h}uF?e#>=yQBcR%SOkWnGS5lVKIkLcFc zz4N2O@*(X-)}Vry=n+#@3g`s5vCA)%s)%37@j3enK#S+1tW~7+GdoVdB>SFk9XngY zi}r$4!Q2E;o!k#8Hj6Sm|JdX}9}-5IT=<#Ko_}O>yf$-*;Z9G)At@SC6IZXCfK~#d zr!J;-m0dvId)_Cy4GwKLa;Pib84G3ieMde;52F)ohy z2&Ms-MPz*@x+5J=V&PkK^K$LKRL(7SEdy^~fOvWEMUpNRUwR_XzBbyBX!Cyn+NK2U zKpqW%NKQ`=UH)DH$?*n--3(WK8boD!%59=TM75cMLU*8?Vx>StPdmG+PFar5Yb^b1 z3YV`EeMQ}MBP!@ltt5QA=rL0GzN8P2G5+=9OK#v1hM2?ln!4zQPujFDY4RT52{&z4 zRi!Hja{O@ktwFNmGo|3N*$GJBEDrdP%oaonghLq z2Qgs0pHTOx{L&(=6K@q?J^@jRA1sRDdUe%Dc`*n456n-#dZr@1dnlp6;$t^hnXPVg zFN3yroBm*qnX{V9*k5hFwp69bJn?KL>RT2;=-Ek^h$T?mw1t^_xtL>t8c|=ya@-7A z-nb?l@aj{cFB)NFZ0a3q8vG{gcY20&MCz5SRHyrlF z6on`7_B^^c=-;02M1aFdxkrbBl|UgGe`GfuYHfuDW`M>{<)?S0ozGEq_m8Gp9twByxZ8i%r%BTB1fi*rgp3zpc$IBS9WKHr*4qm+hgrIbDs7ESK8 zd&A^wh*x62Qo52JIyV(vJdLU#Wep<`PJD(&PSYlQeKX2`)Sz6w_Ukd{-XYSm{n|?) zOz=v#>|U>|=}c%}oQH-5aSe^t*&SrRyD?z2;Y##$P1w{Te&Iw{98kOWe&#Av&%xFR zk%KiGc{Sj~HPe+e)VwyooYwfzygE-?><_bb#r&|=N7SuIiO^uqzaLH))PCY3CCbFo zIJo?|F^859q44DOl(YJZ`TOZR7yiU0MkC59~*bo%@Dvq4TVu4)mAxz ze%$r=SN#l-q0g4IHC8SrHs=ej1&?~Yf3g?)v-~=y#-|yn*nVISO&Ou4Ouw-~%n!!E z@=6GxWwAOdEZ!rn(I>F?$_+H7gIl)-2lGC*&S`sI8suRa$3fm>-V*m3RI9;}K(;_9 z;hVd8pZ9;zuq+ninz>cp)2j{`+Ojpj~lAp7q$+f0ymq&@v2lvKMHEY4%&jY-!szM7}$z8F`+5cHm+U zWTqO~$=P0=BbiTByuul>j9syxw(+>yU+1%S45=b6KHH$xCGGgM`_X<=Vg~?_>p1@X zV&8>M+b>C*_8u5r@?EKhA?%_FQj$&p% zM9K}dqxIO^N~1SGadb8U;FeA%313r85^)B@h)%27A;6S`&GGXg<^9?W+e*9pa{qLVtT zX`uFT_Yjnk+t)~|#v4es*G#53+Nq)2?5N^Fw>`*FTNbJtYh_%|wrPJLfRT%>ZB-et z2uvelv!3VJS!r+2obx{JL>qe2{{TNxt;~r2VVLrz)IrBafYE9o(Z`8=rus#t!}dpS zz(A6jmkc@&*%WDYzsNNnO7J_BXf(RV=_n#&`}m2(yw>#Ys}zciT7J!-GPaNCV#g02 zGyu)gE8z>={CMS7tXJhNTwh#dN;esMvSm^ot(;>z>(^X_-?FrDNTL>RqI8>o9b%~u z+Xn_Cp0=f@4~$PtC1w<`cR!*J$q~IrmiQ$6L7{V2dqz3Zk-+Yx`+Dw%5X&AE=x9UX zUcR3{{sx)*r}-aaxG4t^?KD;D~u#}hCSR!1Gbw7%uw24q-__a)g3?XHN|PRV}IzgeIV3r?h58{rfpvh64B$`f+@h5&I| znL~Z`tfNJ5Ju~IL9t&d86K2Md3WxLkEUo5GM-Hdcek_rcf{`u>71gWo}cE@pA6s;GJ6e`V%k|4=JqSW@QINA%TM z!@8pjz$*OTjHQYi+m&w>#N)M=>T{Omt&5r84Hyp80OnHEPpc@#d~D!Xq2y`U%r8Zv zim?kKLH8$6>ql0Z!t@rF(Fh)3ypRzzj93oV7ndw8%idG$hOLqZo+=;d}?<92V@4`RVDcg$Nm(@ z%4U-Dy>sw~tv1-I^9%FPMGNGACTaRr5@b_Kcha6^`*`kHk?# zbYVwFbBv%-06!SLIj7K<023af$3G&u#+wMP=7BkaXmMtq_FPDcSi?g)z zeKE}sZA?=I@Uywdl5>OUp|lU|*x&r@v0@nY(p=YnJwZ2z{wU_88ljnEOREEwT2~9S zqZ@dL-rW(oxhRQ|c|y!`#}8z9VL|$!RebH~$A(OwGXN)+`@-g_|4+cMy)0jmR1Bq6 zxSqdWsC+Leh>+T5yAG_ZWw=X)qm|p=>v6vG8Gd!AesX4eNbZ~$vi#^cq`n1HnyJus zUntdfeu|c{T^yLRS>MvqtCiDYkxnL#a4(4vs7hCTC3F#oDz2wsE={Jbg~#i;hB@J!iMG)mi`O{cPP$jFO}w3tX1E7nK8JeSFH+8jug=OP z&mL#_@vckr4<3(J9v=?%TTFh%iHg2|s~pqss3--Uv=#|ui7RYC$ z7xE2M=b}{Ab;M9h4+9wHQg8=Qh&2u?2hqMj`^vt#u)4kweCmZyzS@B7uO?)-QSUo% z@rz-%!|JX-`rSy8YVk0TFBt}ixB5@xp@wt;g6F?XrlpjI4lX@BzstOjHGt?sB}%Fin&S3CVj0M4pTzT~wCI0jvHnP9A~kM!q`kHIpB9*B1M zSx0V<)NTK}$QrKT3^=ezQb_8kJgB9%8c0~1qhgTr!R(2$wysoT#+ho8au~*mOQu6|N z1e121j9p;xMgb2Q(a1Yy9(t0FwGH36@S@l9i;G$!U~GS7W|dkCRSeJ=M&skd-3WkD zH3JC;D*{=7S)Hmna|~80vvg0hPoMR=xn7dd3%y)M84Yjt1MSDf#r+0sYL+9icpBN{ z-&A6Ozb_9K3T_4Qo?DZ}ZxEb&OKO}Gj7fHpm*i|KCBI85d;Bt@-Us~%qW)luA~E+_ zMTWx@Cu{%!$Y2f9qf2YOQNhUUoB=abeglPVgv9!Wu0 zq0INiUOh=G^W{CSx^K=-wH*Y1b<38pvt{vp)OgVZb;VPm z%_uP|Rp8k?5~dn56hNhn!V@f)qwp6Lad@iO%%n)4)FZosM3nFUF+}ZyC@mU!0+93| z<+H7z^iKAhc@V7Aq#(NYo39SHDowA2-38<{fkrE`@JNnNJLY>HLFu|?{HU}~OP+7~V z)E-Qwtq$Gpv{&{MD@oV<$BrL3Rsi*^&H#SFZa)6L+1MVS;cPW*OlP?3eY=nxWLiR+ z$2prqlip5W&@BUrPp}6Xy@lTrAg`8O?>i79Kbq97*rce$n}l3koA$U`>hXr!b}Z!# zClyfI!nUNpeDDoo6V|LLnPe4c#6Jrlqnx}6N!jQ9c`zzCvNiNOoD;Et8u8II_@LUl zj(IVg-C{{R+#C9URuF8r;2BGS{PT}6`Csm&zTfSty41RhnGoYJ6%F4U0s{b1{H;4) z!CT8W@`d8Kc|lr22Bz4ipAAMqLSnG}A&^utjNse-P^=Sp(g8rQ1wLb+QrAx8ISOT}xf2(#ApvL^^;!jqxN&u}2D925GSHx)3nq!b`yy~sk!0U%c8PlF zMs7uLh+0FJ14N;uXZd-ey@ZG(M4nAI1ObWn_ZtG`kdf=4qC{oci90)TgYlPM zd@hGp*!=qy_WZ@Bz%)J1r^CG9hDELwCz{#cIWQlUf25Bt>nJ#?Am#p5Dz>PezbwyVqrnZ0DRW zOUrd2@oag@oWm6f7pC;dP5u{q&r#{}VK(W+QjQs&xlV*~ShY>l4vmt@MMt*ZI@(u9 zq!o`xJoZ0&E`PlihdrCl*(>vf2S}U`+?;5Yso(0~nT)thXFq`N%JD9Muj?IB{N!<@ z%HX8o_>Ikp&A_H>xC0mtVF^2Cc-HX?BZ}>eEiJKsO1o5x(_c60r@bwh#)F6Y!x+;0 z!4y!GM)m$=VXWW}(&$%puU_hc-CZdXg&q+Omeqkv3Vmb_VQ#yH1;YS~%&6MXld z<&$4+8WrpLr(^Ye)3`xg)pOe6S8S;sf~rbkwf!#C%(?Bb%f?~h+hZe+g) zz{vnxzUZY6Ch8Uelh$nLt$`0JHVha!@SRspaXD32A~;LIYO95DG*FadE2hkKL+-iF zH#>yP2XcOAj>!vPRsr{dRrNR68#!;uTW))aclIBBKqXAd<&O1<+#J$1MN$-!Om{8)eXJJ*Y#KSnGP#4 zW%hDw4kl|s#;+X}^}ToJ2uE66gz?KBDD#)SYeyPO<+++?7F9?2bS%@8cx0hddIxU5 z&?r@ACjSp#U;PzT*seW*w17$~DN2`szyMMzAPv&pB{g(+halaZ(m8aOba!{>kVBo# z`+na#znwpTS!?Eb*!#Y(T+R+)B1Aq8&bnz*YcuRbxm$3Y>CBu^;_O9j2%2iy8ewJ@ z9CKUGV_C^9o94IvPz!QU&y!5l!!M|aOD@$H9tO*xm;b>~MzTGnO@FV2@vTc>FDLq? z$oa3SLPG);EpvPG`_+J^e%s2gU9-U2Y=)3TYJ^^g4m+W?W~=eC(OE}ykRm5o#Y^)2 z<_^{%QlpPv(1%^ZN;t1vi5IYAD;Hci~5t6rwXqkf^As@ zZ1xow8d~ulxQ!Q#ud^rNSh|dzi}ogahfsSp_?w z)buz8hSwWGW*}|p5VNOhlmENcW-8t&bclNenYp17O zcEbj>9`l(0a`8nq#I_<W=TH>T=a>?-@x!8{9&S)1XOVHzh)fir=TxL1K?oGTUO_sP% z>L550BRXk3yQq8T2n>lO9k#O#zzZ{2JAq&np^oy=s{U%H9Q0QOgt`StT|IPaeX|8a zhs9=&Ym=hVQipbXsgo=0ipO(%%V-iUo}Hi+ZhYKpV5F9>A|X{zq<8?2R51Q zi+E`YU*YK2_^|k=q*l!na+KVmdV3rHga$4&4S4A(`CS9cprYCmahO0rzUL+k6xH6f5T#&%9`k|nuX$R0gYSTfY`wwMsDEtrX(uW6=tXjD~kdaxv zn_QW>zgk&g(jnoypZ?jmqIy~}XA?T)?^~_Jx#?uo0h<^a>A*tVC8b+0KhFWK19`?>gH*0z2%A@h4G@cz6t_tv-iu;T%aNo4KBVU-aIHwrFv7Fj0xmN6=Th-4brc$AVsc8XuL#j_~ zNG~=&o4N(v?ruI-Mx~7(m+Ae!kw%duu^RT+nX0d{EsF%-c!VHC&+P)frK&digIsA94o8zUPxWtCK z!B1}9{%XRP(Jm=(fv5X2ED=6oqV3xuH=))vvMQkxekE;`*Jf|&)!zu3AT{O96hKs1`` zG!9xS@_I3Bp_*pI-xdsT-_r?-aa|dkeHaq>1%I(>U^XWXZvtSAX--U$Yk;DaudZ$T z$=(YFnLu1G_YL_0o69%%jyL(<`uGX#WmmKMIE@=2B^&1N9qA6ej`M=ojc1gBg&T{6 z#4hV*dnR8EHuT=H}_p#sKy@_D-YDQ$Rwem!g%sMqw22;jD=Y%yR zfiR;0cn?Kj{t02uS_k_84I#6kz8`?kW5}<3{KM1Z9RMnM9U-zYzJYmxhNOWKn;}8m zWp59{u;IX2FA@k#rfcE1)dYrgDin!pJ+JZ5c4-zqB}8RZRxp*yjRi4;02j`b^7#9l znFfeS`kWe<%5n|3W~#4#1-yK#2NY|V%fNue`7)q+T@p>DcXQH)kx~G_gr|S9<>yuX zCV8Fmm~Odz>q7qB#N5~usGX1mLK)6uEuv(&f!+5`va~D@L}lqY?Kp7t`@ldxQdprL zY8cg0#{>ZLb|4yHR;dk2ZUN%N0JLKp;3cb##Z9|$xvmLxD2LtwY4tuo%2J-Ev{&- z(CegXD)>9phPx2ar_mS99KaDJ;y){ZH97%;XvPgN6m7>UYoXUfGbr?kZ^nIz*+`{QPX~5Q8A@8(U61V z=x^QQT8kLW~$H1Q0-G^1|(%@B@UEj`2J zlajSyVbKxEA~0*1L%K06P%8bvAi9LC5B9qV`IQWycqj9^ef52QCkqj8NMb@1`G>GP z+q8v&HU92}FmA0KrVct0{InBCLIAB!-_Qw2Zk;XJP~i(WcoVRzk%I&VYP?40i-bP= zHz$2s?a>av?s_~exNguBrU`_Du0Z^N^`sx;X0fsbzR!ROzgo1F!JK^q*?UtC4a(!%33iOV3dMCiPha{5yg3~A3=#b9Vfd;eUai)~;0$&JJh|7YC z?F$6e&hED*kQrHJxA*f-%Q(e4fG_<3D4ds|`+#aFeE$uHZ!)BDmIME?+2>rcgWeLe z&Ye15t>HlxHb^Bf{)^voS?>I0GXG2;wRmb&9kB;|Gdz(eQ9Gf^+_r|Ii2d~`vB0#ppyI+9i5Pp`6 z$d+1>{L!)pt_SWYm>Y%8#;7t}oT$8W`Xz61IjgUn;CDH>NE@ill8`#;xw&a2pQtHm zKlF_SP@Ts?a0392s>pjl&gvX}Q+AQB4`&h5;eJ~#(B1NkB$=$(-sz{*+E19JS)W@^ zm;m?S7*_r$xmWep02a^sw1hKM6|KbzAc%ic2y4>a7q7!n2Ys)XKVsy1o{6rQoNzba z*Y{0yTP}DG?T@s^A$@M;NVbChRc&y~g$z8-EIJ=>7-IK+k-^6BZ?HQthF>(~>{@XB zx?sQZ8j+VaPm!N15EV>x7^6p}UgA7A#~;0rLy+!h@0+gV{_?aIm_As76u9%}TU`$Q zU6+ZT%GbQ$AaQ)NlZFH4Mhnl&{V8WuF5Lq>jx=esIpE~fYb`a4BnL&uvxG5=WQ)nM zkA!9cg-pg6t|O8;!V!C-pkTs`1Mp>a$73BykioiOFoaA*fWJ@D^@wSrHm3+!2sK9Kz+EX zc9=g+Xt@t$9Skt@(3DstX*}7DM>uVo%1iJ#Wr|tgn9%eqknwN?nS;0>RwBRO8Db0E zLy})G_dGKV_Ri-e5cMGR_t8Q01z4|SbWPvNEdq6tK&I;#eI`RQ4MAz(u~Cvrf&CCJzyRA+0c{uLNg>BczsueAr zcm1ZTo{?C%a;A|f{SV^|Rfp(HGyhDNo?_q)FY+C?4;_&UfO<=-#+v%om6F9?=d&3A zVsLgDOf@`racu9+dvG2a@H1=Ky!}8K|QSufbXzl7Be%%Zd); z4?PG#2FC;rrP0af&0dSNqIK)zCpT;%*^GWeyeDxztT@i@YDK6PoaK)`Qp6lfuZm4P zs-LEtxXyLwf7zDV6tA6X$b90K#`JC}`7R)$iX^a**TO1K+I7Ob>QZ3H`*J$5@Sl>v z6U=ZxIgvHXq|t6p{CP_rpu%va2d6}x0j8KXqr5q@k)`6B;-R^G6~VkMH|RMg`&HkL za1*qP#M;Dt;(n9bD4nPAV568>FVC1-n*C^@`rkV)+Kwhi{!fhxOYMgWRI~5kKK`MD zfcE{@{#1#D$IY2B`&KU3VWVuXM!(0CuQ1d?$@4L`#P#9fAm;C~#Dq~2LH=hZ++!S~ zhuQD!_iJZHo_{xLS4ug0$(BY_+M=azd?Ai^jV#F+^F@oJnA$a*7%u z*pLnn6(N>)R4*-u5@@JZw@RXuIM@gszp#s+r)==ES^Dg`oH?K88)(yQsU=hY>Dfa|Rkz8d-Sp}znIuh7k ze!5z>UYB#Z!kU^cFDjC!o0}WhO^}v`9lfL7d&0M)=1nvbIdQ4hW*H%QrYo>yTq+1S zn>Ig`#%(xqqy;oy?<>cH_eVD2>4xGv{II+Fk?;3h@TUTD!iubhn+L*?f)7UlCHWT6 zIcj;<%=A6iXwQ}(ut8-HYuK1$zXCa&9?ys*?ei7~0?SAs zBwK3EC-#VPmb)-pu8*|d_t>?pQ>*^7G%T&p@-3;oBN15HALtkobu>E5XJF*Y*7bdm z41~!LGQd6AC$!V`d-yn11^;;t4l6oVZ~3h`nQ(u2L6n0e+KMK4V0-Xt0$6(!P(}Dm zr`_(F!@ji=f%&;E=K1vJqVroUFSFtR==WZ|?i+@}DAGH(CllwHciU0Y#)&15_5Kg! z1tZsHo}Y2I_>?`*YD{N!JYiciW?h>3^0}8-Hj#p1#B(dq4=I(RJ^aelNyp1uem-_F zm~DMj7Gns~3NPYH+u8hK-r6pgOi)-1>fH(EZQZUcD!Iyx67ExTXHODY2S3vbx;D8h z_Gm<7P6YGvcUaemmXa!0WWMAfJt;X$1#%x;2az-)G=88`p97r@qIAC3e&c(qZPR|Z z$R_RK5m4NI$YaUbF3aBg@^TIC2n=gdTpyn#T6>B0?J{Ig+AYtR=JSBDG5LAs!n%Jj zPUIlxWVZ+PAB#rasE+;u$s!eIwX?aVYXdlId zH1&3aYuN@x8NOzBR2t!cHD9vH^me56$Xsx1j;Jwa!MxqRDFHg#T=sM+r!)X{Zl@xX zqsQO#!7glB65$=fdX+Pr1(lu9?Xy6Xgqf8TWAEvZ*mQNd(f6n&#L)t9gTM81Ad%F0 z4*x3KFmKz7RzSbcS;tJ^)kbj4<2wDh1Slh_j8|u!x9%<0KFy&gUKKKc9&xIpmoG4^ z{;l9GY{lUe4Fs1$^SvNbayg?A6*KkP&6i4d34ZyY*9NpN+Psk=NL9IyQ ze~BuZi)jr8V?eU#iR4?e5K?FvP%Y}}m{VV1IVim{?=5N^AyZo3jUW2;ST3qc<@T{y zQt6{{&6oF6VnMiF=L217$o1(i>tmd*G5_ELs4{8(lzetk=%l2$-{p#(xc|JdmgfQe z`OE(j59nXae-<5X=}|17g;ulWUJFck2cBji%#sTXx>I{0sUTz=0(H30#Ih=?T`|OF z>QBk$Ks*aq&;`h)e>|Umk$rA`A6aK6RIIZ(hM~Q&DBZ1^>7IFHJhErY`Hz5n0zak= z$Ka51=cIl5?Zcum<)hg{>r}*mXFjvXQ#X6Rs6h~`xH<7qDleD3TxOc#K6@tH4SUoA z=OhRD=Js0}frsH0=N;08K!KZ8B+OrsxqfwkoAu4&%dwt_^&Q60OGUZ7=GaNSE&J&n z{uzQ(c+wEP)epg;LEMM*U2dNSpb~inOMuYqoJIKIVcctVALLg9z>SAZ+0CCH2PTYF zv5~iM-kv4=z50r;j=j3gu=d5gb&1Dv`SEU=Q)6^$2K!TxTj9~q?(m1hQy)xQ$&GJ^ z$c{Nu`DELDqNs1bD=S~r4K2>^U{swfH=0#U^KIp14~?RxmwTKe{XkG~z-z;cqc!^% zWWC+uJ%dc0c)v~@uw;s}?5h9wT7=7Pv}9=_ZmDQQli-yiKnp!HCRdte2OgQfalm?9 zF^KedUFmK{h1_!UaDez=^}V}}Vm&E?`DUC3nbY%^8^(O z73g4Lpcb4KeYhjZpDZ|+b(I!>GCijf>AWY7(K9dhJL&ueK0 zUxGMi*eK~(1-&i+p}s;acLubRC5YBx#r+@m#QwmBz}8vZKtt<57hM@BRplPH z6F`!TE{(x4RY1mjWKn)2;;-pq)1zJ5^57J?D2g2d9o;Udhah$aJK-NG!|}_+6o#b0 zm(}D9Xag}+?j+%1n0lZ7q4Qmge!IFu9o0ENcLeCp8r+u@)|>w#4MCym1fg7B9r*Fp z*1i(AYc_4SkiBG%mA9o6?hT|t+A0?aaig`|kFy^Z4GiyT+Z=w)jDH5AD$7lt8byNt z{dK3?n&nz?=?$L>s+-MnuMIMFacrcBWf31V59Yu1%Ql}4tKC+AwT;Sk!gLP^XXjHF z7DIvn2XS)2g_qZFl*vqMWkJ$&Lj>o8Pw)%GbkODmq8a? z*d+J7Ha1(8rBmLXph{2lsEmEu+=yU130=x!s-0aNHrthza`I@nRRs~PVo{w-`X%*B zAadTGxHz_E;L+Mo?9D|f7Ae{L%6ou6xj7aHn2^ixypN`G`9Z5hQe|x*6+slOCF4TW?O>v;`k9 z53G4hc#2wzXxD~-N(6P?NgmY|LU>Ee8dM6@t-Py8tJOdFh|7(ujhKqII7Mr0lPO>e z1j8&0#DAPDNh|3n^#YSC7JQBRk*Vf|{JQa>T+WYQwG>2CFydDj#PSmmzFg@)@E0&N z=rac)OhPJb-z8+g1svHGoMgLVvba#+=u()JruS_Ec*A;lAW;jhw#Nd&XT)^e3$ zUfxyesRqKbydeeR1c&9sKt7w5Q4*PZfnuKf!%+P;Cu{21dW^my<|VUItG_?6L_AQS zE?JuXQUXkjmex(q3Nh*{Nw74@kmUJPMfE;QayyavprZr?M4KcuV-z5@FP)Lqb*9u1 zI7!R;d`6wM&!2`*nDu$HDp*?8^tmPb-Mxyeum6nYNz{561s>H_EOtJ<<||KiSR8J9 zy~fA9Yva4;Y}lKz-AB}u<=l~DYE%a?_9=+i#jf~sg_Pgh)2d_d`GXyP7HrsW+lMY?pPe=U2$=>N#WWQrE6N5nO^8J;br>4mrOz{j4M1#GB z2oY2ge$S1I#$QUq5aJU@Q-o{c#_!BYTc(~uPTplSiyR&#rdg^t3NABsQm(kZ~ zF^YC_Zf1GuXS)$R9Cke)Rx5wo1n*f`7GvUd_4(Co$e`pLZ-0>Nl9xs`zz+*IcW80k zw6PYXQXP)2%@CgbvIFP`KC>mM6R| zza35ALW@p0e?!V*)$W753=lBpFub;PfJyL3+YNK(_%&Is|BTq}@d3{tvaJxIk+$>i zD}#B22b-y`Af) zN7zDT$s+D>BWbN~|Ls|Cdh=e{$klg8>Rs!ePuo6Tu9 zS?!%GUb(Nw4Sm^ubHWmZ$wgb8&iSfOl%5|pS2iy5@G)gdA^lQd!gZ}_9<29T+n2H) zao)m3RYz1@bTccsASBnf^GZlkm+Q@t2i42t3L70oFE+IW+vyTZG6JRdZ1c6k#OcO| zh;{4Zu%nRoqyso}makmnDn44d#&&H} zM`mXn$Q{wtc(2sdfMK{5>djxwidk1-emjbf85H!DT0zb+0$IV?7aLF0wXS3P;<3k!~Jqf$&)c;*vF>Vz8K4eMbj`F zOdg-LoBblT!XNSi8$X%LIv6?vO-)JF_P~~(J)gZzvNUr{_%doQCaUNsf%iRIbTJMi zTphX+AiH>-Pg&hQoJCSvy+{2dYenbS9fE&F_S%GJ{7m7fCO0&m`3oVoG~AO?)0DMS z5mi*6BVo=!N@QnsL=cvKYsBf{zIz1p(q0RDtkvBBSD+ov_mEWHk0oI5mZM11IpaN5ZB7RaVY-E zSYHTkr(8hK(3YUAJJJJ%9FKIty}F>`U4#0E_;>Xg;CMH-+^v6j$SkD;4r*&WY9OQ_ zgg+a4QYqGFKuA`Faqd)+@(3AX^53uwJiNS5{Uzfu|tZh+_kHhFE2 ziA8(Yh@Uz^Y;)Va&L3VW@yiWVZg>k*k@DZ3gIcM}df_yOJ+;qbGYK z+B6J|ctFg^k~G01*!mcMwM)~wi;i6^x&?W%MT8#|?z0XEJ8J_hT^u5Hlmy#fB3179 zu5wx~&MtT6j&%nm$HQ&6x(b=KsE@IjjB&BGFXb zH6Y1n&CcmF(6JhAjz}ODMrT-pS~K+`bNRKOV|HF4DO4EZWQ%u$fXbnLzp_h~oB_dG zK<+(jv755?+2L?D%)wz1U5F-8;~>8=RaVfDDutzg%eP;dG|Fwd)~ob>7M{+q$Cb4r z3BBJfO2Cpm&0*0HmP=kTZ)0 zLW@qmQmscxZ6LVx;QH?1jok>Z zNgbTM{+RvVgOX)F5GrqrI~afSP)eLm7ki4CAG^u-I1@U3<0X2n0K}c znb?PYB3FX&U}Q^_(6fOkEGhn7<>0h*f}0c25-?P*+nGfW*(IZvp@3O_v+4@oFUS`6 zZ{))8Sw~FK01FBuf#G8WYN)*yDy`&5k{jJOEAGB7cjQ|S{J5xeC;qYe0WbS$uk@G_ zkD6xJ2~n5$vzBWJrxlTri)F52XL-B3Oh0>?^;cfupA@N96kgRz zeFpo{Vw?Aq_VvX5SZj%Jpw4jeWaQh4^?UovB|H;&J?3I#3R)emOIZqr{SxGUj(JFZ z;A4w`WV)e^Gpc*(qa^bP#rTWYYt=YMMJaS}M32^k08&%`Qd8x z2RRn7r^W?qi)eV0kX@K{tKCWVNdYL(|0_u$9*_$o-jn$5)sRc|l<$QNTdqW!p(O$j zcshFTkoHzOm1l}x-bfn%fv#A(3#cVH1ImNuV%h6YS+}DV%(U^@V2O@V$`=H(OK(tq zT}WJ{rGVZYE!1)WqU4&4_)jRSKtJk=4=Fy~-$ti1y(|?9Y3=TD$1<&Eb5BL=px?6C z0vHgx;xIMjqcV)qiCon3c6%vWQ*XU$uQbk5xVGFtF&;=%nW{hFH$3V#B;707t;S%^ zZFU~xpSNb8stPK(#?@j}<=+l}iW7_J;>4lk4{4&m3oNg+neIW z4HapN75btRRmLL<4w6o?nP5N|Ery@q)L(*}aDVw}4zOV*zN=)4t@utTZP^_Zc#8UU?Q$c{nyLHwhl^^bL>~R zjwFTeGjKX~sgy~^#S{@D5Ggb|@EEPjyu`T87lF0?qER6py7N;ogXcpXhIE3Uua`uM zqu98g6~g0(2di70OWv_3$)}=I?$XfUrTk}g1KB-^;}=CzvI~!_pJQOKola21$IN>u zC>1BmR|)THE5gF(6ioaX5A=mm^_pLXj;=3KOF9M7&xC;Mz1dz!n;9~LNXylLLS0da zm66Ej3e6nRu~6?0dz>dM9Jv_?wk-pR-f2&M`Si-^YR-6Js!Ce2j)!{<5#-Cf<@8Fl zo=k3lMZt|)8;nzp6K_KmQ{`Qhg1$|7$v|r>l!Q5mmr>8`(S!THXZGNM)#-nK{mzK| z@wU^m*E2T*r!<&;Y(jMMq~V>YX8b}f#W-+Y>1L`i@xQAdZdr&vaIYp}4)al?ZT>)U z?X&)nr&jZ8u5tjbfe;>|Q$-D-#KT%O=^Sf+({F{TFeyY#iuw{!?=pc@MlOhy*&Mmo z0tlPE5P+rM~4%rL+hW*0Pljt-Une!8T<5yyvq~s*lSaT-V>=$ zSVWA(DWxBL@9o+_eBZ46$FIm^J|@)lG8%_4rF|2^x1`ERXj zqx`eouB`mepL$wuXWe0Y7#ym57(57I-KUC<-iarkL&V@d)}YViC`Tb87?8>%A9{o4 z#HNI!vyA5@EQ<$WlK0k6BW;x`(q#)cIsGgno{4+@dJk_9} zjXU60TGaal6}PuM!Vls$vUhMy;S^l`1~oeE zh^`ns3%;y2ahCov+!&CMq4rx9_%1Q~4o1f9u)>@!uWeW6rBpAJX`l(-sZTRoj&eDX zEXop1`?XCte#Hdk`ot8vXaht0i$NIsIK(v!0*Vs8h^Nyn5+m;vH)gEw7bEX9SO+w$ zbF`*Eb_v(PHRJi$TW3CLTxV(LHSQ$xE4Mb}cwa1E;^u9_IV9_u!gM{{iKas3;#nmh zj){t$74hOVRT4Z_Plo|+VYa;f1?eyV7|hXYx$Nau6gCI}BjLS&yipEl_*pQw*E5I*8i``L!9(wP)J_jsmmRN|C9bdKA z0l@+niQ13+>8ij=MG1k&PtnV`GGBBBGN2X<^9XwY(^k#HKPz!2Ux9!Patpaw`Sxy- zV+Z;eAA~yY8RxZYOw+6M=V}ko$%hlE%z>n%silg?&wyUgUp*1zR<-w){(nkJ+)Q-o`VDfC zD%843xNUOyDDarag$Xqc<}ps&U*VI#b5}%tDz^_NJ^_$&)nmPrV?y!71JqEv+vb~o zN!!WSsH(4`n5CK5yz+6wdD#>Qo~7oD+78GEn!fJU2m|155=+32wSwOdlwnc+6}sj(k7Lv{MY%dKDm<*yoY(Vsyd@kPxa~eH zceg7!aWnH@u<2ox?u=rPqkY~K&IPh>v^uGRLuzy^YKUX_^Y!sjB_>vLc$|lGMP*wHW zCsQ5%=Yz>sL$&M0@-jR;_)iNb(@8XGX?ns?)TxzTKnYquwdQQJ{&}KBDpa9~VuS^- z4lv$BD)uH0)-!tlewsh7w0wBhJxDe`T(O-rT}ReG4IfBNrDh$)NEjw`M~^&x?|n~Q z5%MFNMh1#9z8mU-(C-7Lbh~z;`?RC=;4$N#w>O!qSy!hlP#RaVWtIwgZ*QNa6mPYF zj1_j*kEO#VFYSFm6Xv=_OPep$si_RJ(M0}ShaPKz@F>$l^|WP4rEH1mj@o+A;Evq7 zz*S8yOYh$H@N_A#{`9RZ64T$M$Nge>7Wu6)rqdi3BXN{xwH)*%@mF6#e-N1cgl#0U-pWJ4Dxa2lkl#e#Fza~%C-uOVA1825&dJL4z;ytPMHFX%&WqWN%Mi8{H zU%gQr{z@Z16Ng`yz`dauSfEi1e<+V$QTIf63PwZ<>%6g}_UX#=j*M>0L8hz*dErVs zYxugp4k`R3#~xweTx|3nVVF9(Slt51#&q0%h6kSPVm_$8C11G+noMSVGOuy|DUG)x z*`<`9K0ld4U9`gg5HLG+)J3SpffMF39&@p4UV04dHhYuuD989l5o(R|7ro@Jewr06 zocg8p_Nx%gDCWjfk^ywY#U^SBAXg@FJ)b@XH3G`S=C-xE`F^3v=nR0Ea?g>*9FG_0 z=-(kRei{vUYvrI`V>&^N&yppq zIggJV`Ax`NO}v!em)vpPK7qtu@#4>o*((j;j`xgQdhdVVLbBV&T-}6Ai@|e{qX=l- zFoB`3jm#IL{oS%U%2_CHs-behVahXsa!4PyM0?WwR8jx;ng{D}#q+6C!xP50{q%Sv zV7tUYnB{@l4<&XJ@*}RkQIRXn2HlfNV-Hyo%%|8MbjBM)SqlIF)e4Qmrue`U%K!BeC!u| ztnSST8NTUPgyGmD@4IT6E$ap}u6OucOx-0K(0Yv-sy5Qs723W~&JUB0RGU7&Uo6J& z5b_DdjRUD59g`V0XxNp`>4gq;P=}JpAV~8=k3{#_BN>uD{!pVZ^~RV&j3rww_aXUW z7gi5drzzTXJwFpoWN^JAl$&5%8^8aC$42~b-y?Ob?2g=Rkpg2^eSFG?_z}HJ(YSvf zrU*C^{=?ENmJgD5VVCSr6()VaDN1*N!eM2DlcrYl7EQ&__#Z8STwmJwfzYK`v9vG! z?R0)OEHa9Wu|stEDN%@AaFAgo*NWRjNHMs)#PjKq&N{Z|tdX7?4Uzt_E~sm?zK>Ik zzKsqA6?Hf;B);Ips`bH=z@vIL%-Jn!s$|a-#-o{hVA;@XcK|>6`GDh|qP*g?5LQjJhk>*i zOrx{}BSCT#*bt|E)o`PBEM?V$qb`OtNIF4DeElO+xo)-K7)UykTDMEiVwk^sa<-=M zL3FkJC9#HHTI>{SBd^ZbH(JAvA;3ctPiuRu+ex(ec6Y2*H-0L+V>_^<^>#a14f>>1 zsR;$N55+AA(LL8@bIVs%q_cBx!hKi)u_29RE?al4|IQgIA08T2X^6P( z&ajkXiP4b#lNkM~jLcM_;t&vFVbpCNRJeL>ur4l~THGP|2Ln>gR3YXAHVb8fo+@+g zxH{ZwF6>w$pd}MsSOV$$tx7LK^gdBC1ZgVq2A`K=mZE!F&r90QzQ0pH4QC!E?GQ7y zyAeWMebg;I9v{AFF)i2KAtUKYQ+D_$M#uAX2nfTX^VMGyh77A)R@CoV8wM7qB&fr2 zs@gmv)TS&RqgOSc;1-4D+ZH*mXq+X4t&cl}zR@R+lIMWg+teGe+N;ENxce2eC4ki&F-H$8^Kbz1-nQO(s0%vZP^xMq#gg~p@QEUS!uWSGlMiIuTr0~gmg!f|PKQPHGv%Q(7iJwD zD2DlwNtT~U_A_>PX$nTBZ#3rd!q++u-M|}RE2O85)+5gcFi;2O*&MOxL$R92u1y^O*K)TL`^{coZRZT*=ay6; zPBFfn1a?Zd0C3F(piQkJEJ2Nl5XiV3kM)@U6MweC8wa`i7shAwaxo)Zf3YLBL*?4O zM$a?%qY_H>X8QTc?|L%z37C4HTX_RIhOfZ|{jKN&ttj|%-@BiNuyZ5$;s3~5AuI0J z!oO&}KCNaemuXG&W20JCVrkJ8I*Cu*YT8+V#3z;Oz`0~dPO}fWffR8r35$3o$QP%5 zZudY_2L1fkB9q^A$YV40Dnx&^H9-^}^XRQwi>R7zyh4O0*_f~FJ6o~^Jn~%K0Rv?{ zobewP#>AIJ<5GbITP{l;Pg$~eaeemb^&U)tp9BnDwtMUeLvAi9!{HCin8;Y2!c*8Oo|<=jQi z7^|Qx82rd+>K!tR*ckj;Wjpen-WHf1A8U2Dt&+7YCF_w}cjF*K@%49w6eb@gGA{)2 zj@#Ke|B@6^huL-#_lq$CJ4kr!RlWudAu?xmD)hrq3lL9}kSCQK@PHZRglvwmRqW4~ z&8VqZJ+g^mIlh%%27#xVanWnv{^I|oNHlXYgU`u4V#2_DJXKa&x@e6)VI`;8FW=gr z7ARRNVENx)KJowh<;yAhuTnlUoAj2&^G~?WY$dNR+n+VY+vuZ7?Cd1Iv7hol8}0on z5NIKtP=5SJ_cvh=)4Mh?Fj)67+{s$ODu+ltnSJWd22nWgG1%;{3SOwZuX^5R-Sqa? z>x%^w(!4_~e@oE{y}@M0(x8-PCFgZ0eo#RG1_jh~I)eP-k5ZOQe^|V- z^29vgyj4!1?!46wmm|P=^ zQ%5*RI}KzEbg7CQb|pmGBy5Y(lvL%Z?{dz$UoTHL5I&=c@kirM}cJ${&T?7_r$SZz6>Z{Gy%T**AEl0qsU^z{n zkfch_pIIF`xM)A4i9*Wokls1{$9fOomRWbXEzN-r1m3x)x0+2d0uP_Z4`@I%KW!BU z|It^ma>QDSiajG{-PU>&!9A` zsh;k39*nXxC45-4SXKA7$~QnW#+*&1+9ZTPmfQXLw=7uK#x+9qfI+9VM2E(;X?58?@miz&`N;x-Em`? zdv}3*wFyWR)z7UY4XkkbfJ2Z-zG+_Y^%YHoZHBHP1v21BQ?6~nnS~0axE?laju}XF|yn#pBFs8e+Hs?t8Mn+t{h16XxZw?5+&Ta!)9zl&v~$)NyAMcWDz6+L~U<@zLT2C@ZJCP zJj4o+NH{p?rH8VTF7;%73rqrnX5x90BOcu*q`0;Q=NJ{8kelUP4FK8w*iA-*OTS?a zY)jc}aAKs{T;@lqqAj7k#SVSmlVD#Pb~@{VRqb!?t*##Dq1so8 ze}f7~L4i-@z;vZVK)=J8cm!Q2eN~1_FH5@>_wmTs9i8XkT`Mb!dA#!gt{1*k2hK zi5yQ;muV5(iKw$(Tma;OOnf#OZ9^ng;w?U1|0A76PahO58b$5N6Aanlipua3D~`9wA<5r!tA#fBP6g)P{VWV)DU<;mz@SkE?xppfb}@uNV&(j0wqTN z&}yRnxI3BF!jLptF%TWTlj#M`u-88#m3GYc#o_DW7vDG1saJW9mo-Q}WU(08 z{D=YK{?@Pq&nGh(((Ck74QjnKL<7MJ3kEh*naw@tetUOLN}4M__VvENE%Z^Yy@wYl=ciU3nYwPto^$yWhYF z7RSCUzEuxd`+9P_lP=>wwq2QU=7+oAm(`A&igzbzC)xx!Q%21I=8+~D^Cu9oac?UV z{`0IWE{4Xd1D7Jvg&Mb(MuXIqB~(ME?n4^Urwfq?vLkdSv2KZ-OtR_+Bi$3%$()x$Fmvh#^&e#as>!S%kgCB8=w5}y6M2~)*#k(6k#3Dmrl?M7+28bCs zTOe6Y^|m|nJZJ8-xkcY(-Ra`5Hs79zs=-0upR$*3Rt#OCG=jaj?h<9fp#*6`!PRww z@tk@Z0YoQDSJ;xT-pdYCRzG-E@MGXheW!|z@@QDLt8EUxVZZI>8WbiS#$!l3kk(G) zxYJ}m?&|{trOiV#D{jvz7DLrm6O?+ThS{y3E%HpQ#V3B^(3*7v7h_-|ylTx8E%a=E z*)hRZJ>7i&M7uYDNxh{%%z53p>i_vKo{c@+1&<9?(c*^NjeWVt6c+Pw_~~v$4z;7t zMjF{R{cG9Mt`HpF7i>Z5x9-9^C!Ndze3x{OE z)T&|5TQ`~KQvYdMVhHSb)i68b;(@k8euhEnGkR6B;MaNTgiospL)Mf#lzQ?|t`9!) zHYs2VDw|kV>y|MR*34z6s{=Req{1&lw$=aq_rVv2_YW;1OdFh;IWmDK=u$81G*9(F zXjXfIXqAZ)IS_r@LYBcmC^N@2G93MFE0(Lh65%KUyj@G08q$UZCh+9&Y(|4;(-YRe zmVnb|54DPI5Z_mT_=n_PiIU`_%qKEOHX3dbMH8K^wbFryq@qS@xK~ltfx(-S$dx)0 zX~gH4+HCX3C1b;IN^wQ`An!aa|f zya!aIRpv_csuMNx;`hPCA8Eswn%v--%%1rcVH=Orh7z%GlqYnFKfA8k<^J8izY_R^26Zb}$*n(t?9 z83jgMn@{(Lel37hzO~Hnou0i;+x*WX_%mbn-;ww#oq^u>a)1Edn1<@b`1k0@<1~;1cbP|Gz8%dEzEac~VBEW*{HNXTSKqVXzzr znaqQy-^&sFoiSc>+<=%_fkF1cZ?eegt-#{z8Ghg(7EA5Nx7z4(xbvk)bsr^4s|=gf z2*POmrDJ6%$xNAnhf=dWaXTR#!r0>yFa;EnbGlb)RT+7!MPbQ5J=|!PyJ&Y*pNI0& z2{L4oZPlzp9g!;vH!zE@JTLdw!@)`{!*G0u1V+O-)Qi_q}oc`m=2?_W~671@y>BJJHzIPcXY2GlS&vPLk-s&69VXOA`@Me zH!&jDDK|i`;Fq@Nc5Qt8w7DFhae06J#ee9C(T{p^CixNhXkqJ-7Kf7W64`n21q+-) zCh?al!z4yzcC)QO*|H7#BIb9uG=(~S0dz_-_w-;zP`+0XFL~lH&6CK_m)=}sI&jpi zqDUHfdK|JJ7C3H7`G5Goa;uO4*qI!1bZ7uCo zZo@_z$vYDOC}cB@#;&Zn=FdWt61zagjhB;j3~Z!q#+9J^iI&%9ny^qVPx60gOLi@2 z=*h({EUU`N-sbn0l*WK$g;{;2p|f}@aIwNqnCaJp-L{LW?eaW#$wY3tm@&6Bz3fFD zs0Ro=h55TZxkL@oQ&PDyy&@JgWLNnBS!F?Y`EQ2R)UTpxr;vXs5(d3xwZ#6lfFPuR z`&SxlrFdIi{7mbw>UL;)1fm(BDs75Cc4VYPcY+C(DoP^h-gdkjz$A$hrZAIY!^e`X z#VkTX1GDvzpOJ#mB8f}9OI=<1G|sr*1i4$wV3ereBM}7 zv)99t{850$cWw+vB^sTB)M{p?dAb0*ElbdHL844_uCb8=-tN_DyB-KzYrNI|eCTBR zy0b=c`Yq?++_}3l4>byy=(8tYJ>Dr=~hUCk2&(Y#;XW zVitLqKP`qdAz<8AE)msJ0`?Q9`6uML|48Ir`EjlG^PskCU!Y@3G?!{z-Kc0mwzH3+ z3DQ*Pqo8{lC}J2|9m*(+?l)&=VC65M$uX4|`@sJnPTRni&4|PjP=;D7oT`$q>>ndBQlW768j zU2i>2T2?0HdV8q4qfbv7k`NU1$i*TeWc72q_1~BZo^0?;n9{si@HR8dQ(lD7aMJ|{ zk_~-kz7n_lPgr=x%X4EW(?33e21k&03P*K^mX}~1^!$Asf95s^=Aa3`pt9~TP9bD>H_#6 ze10%~bQi;GUNSl@d|VIztubrcC=usibv+f8=|ulyEG%+nOFg##sel*eCXZ5hR&j4J z!+`nS!$3QqnFpiHpIzJ_cdVUwFcBc3ieCMdnl?EMQ~Kx;@2@i9_Q?J9{n~7He7C$9 zVJt1Y$de*<163qw^L1oxwv1UP)5io$k<@hdK1nG}vkg;BJg&rjhlA9xq3Txt*zF40 zA&gKj#JS;_EIG65B1WCGF1diI*Vp#K+6wMQhuPb=t2lO~>0QomHp)xyqxOUc5IfB! zo-ALy+t_``v}B!(TcD;2MJD(-M${rp60kE_W#}~52!3^6YV>y@2WKio*e6h4a#dfo3MYU`yNnQ4!|@w1Je-|Dp2Eg3rjVZg6uA^RAb=8boEMFzt_r1J>; z`(KdkolQm7KmFJ|d~c>5Bkmh7%(m$-4&zi$C%a(nK+dnnAYmX&3v>S8!NI1p3nwAB zB1jPHha)sF*NF5R)*lbkwNgGU#ddrLv&H=FbcUKW>51`oBuzZq(xv+UY9+luJ3#j7 ztEO-%RpJ&>)e2XMh3i`L{XvOFW)wS)iq!A}ruJcCJ#ifh0De#|`_#Hx`-v(Yag`EQZgXRWxK51IIs&je!Xz1T2ls1==|a}HoF5=T^nZ# z`4iN8(?;1e%!9uq;D)CCNq2d>yV^NDGuv6EbB&3cv!WV&qC@sY%p^EGLz2o&d%v5A zucQto>`@2cjJ4z3MfrN?{>vWlXzP1+Ec^#yBdNFHh!{vv6m_wl;|a2B>V15M!Oia? zX3S5YOok2KKJqWnL>KP^87ZxHr14*Wb>2Hkim42iXw2v`+&wXYWL>qifx-koY7-^F z*4PHyHK>Zb=hX0>m7aC<|EGNvqKWLbQ^C-CLf!pn4iRZ=2cdxhl29s`xdaI+Z`j zPO*I8DY@-s#2ND9%-^K?Ttq|t<{r6i=(mv%ZwHR2vwNF*^K0-Q`|H)ZT5URwL{fHr zjy)Hr>W?!lg&hGdBMdLMUMwq$Eu4yXGx5JpC>YZyQPwQ=`{fn}Kw%(-8e7;bjaxeh ziBXG%^x;qCR=rV62|k>n|Fjx4BE|H`7h^i1(090(6yqNW3O>Jv;e5C0s%b>N7k1!3 zBKhRMY0#mOU$;XUeEug$Ep~xRbGn(z9vMr zqc@9tRfT0YDGdWgal<)iaC_;`Xjl~i@tc8Z&im&oHxQ7_VUX6xqc3E&iM)Grcp>i- zoU{qgf>O8_I#GND#Wa}!#LP5LG>mpw$yG+EwJRvsEI98|sT_DR3xUn42T z2FVCOjQEjk+Wj=ot>v&1tP!^BHD&$tN*La)6Al_+G2sm`)ZwtiE^{1}8qeW16|QBs zulq*^DSS@2FVzdruDL_c^InLsJhskWjyXnlMGHn?XF0$or}R+|qs3fHDyk<#fVQ+_k3lP=Jg#xAH=p24 zPz#Nt$)NmmSMBswd$u)en`-7)xloybJrG<=wY#13ZWnUQ?i-nL~-=F2#{R#!r zP?m`wnUfJC{ka|NDoC)%r9+1R0n$9J+#pgDKHI(xbEqmJ&SiMT6VLDNIoAvDhNkMjb0DdbOBH z&3HvOS&A_R#r{E=t+AlREJAKhIsdLPm_XV$>7xnJYxa0HbYe};I$hMs58y(mFCJQ>NH(%h=A`u5WVHlaa)Ygptt{YL8PF&z(nMzRNg3Uy4rdR%tujxG@l{ zYgmF@vAX>SpP3P64sDg76|al?PR&Ng{e+*KAeS5I&vi$nC{;j(TFY4dJ~ep`H*|nw zxxgW8jP}4+_8kLG$@h`plA84I0?QPpXNWP+$YkpiwhtP&iaZ*92Pb)!#dy>t)$D&T zx>{du=yNLAIC1}|55j0Mu<; zzo8@!4}F0z{`TY>X{E*8b%;GE5H>pPey&;2tU#G~X&ej%-P83*NGRt{V&cmHX@`*D_cOLJO+3n(BqtC~*LkYm6Oug*pH$wXmwt3H>TbhoW2W*#IwC zLrQU+bt6b8a<)2`-aCa_b1~ko0HcFKi6GSLZrHV> zC)aL-lbY|K29_TT~g9d*-KY+kio2=*G1Gv6xYd z*W>M3;?0=6HO+4nty7i@I`u%uus3J{i-d#Znz4fY0jZVNIUpHQ1S1}1igt#eQ;_T!rkniuPI*q5 z{0oB}S?)@I!{?np9J+5i8xpF6haOnCHuU}qAN0vbCX%x9XqZ!p%eXUaXdzx&K-f4b zlNr@SQ!`aaFFhLF;C3MXls6Wz8)~;DBLRLR7>gSDvqcS#dRQ}l5epMC9^$lfa%vmR zfX}uS0t#fIHTB&4HFdKKSIieuJ=tIK;?tLM-X>eaRr4~!Ta&(*$^s>$ z7|USa=@&MVcAa(%*>+6iGG9AwBbl3{@tjX)S=0c&(>|=gG`JK9TbN_etii%h(UVn8L=NYl+f1ar%H;oFiut zhKLEhWng95cgqd(nD+b~ZIaWM_D_)~+S|RfvXRfR?_dbDDf%ko!%5=%TKX@bx`S2X zlOk|ITbB+}*S}--{##4W1NzmwqJbs^9NzS)gpYnk2$5lhz`)v$JHS^7fA@{62z!RC z<0nT$ugT;lmqT;qC-87-AWAMNYJ11*7hV56EW-N_S}Z@Q(<_MOH@1sV=I^>mpc=>y zxanAG@dY1~7ujc_c554kYCt0Sx)SzhyIZHJ(;ZNMM}Vmh4MWqo1JPP>nd)9JK7A*3 z=Wkv#XUCMaTOx^L-qFmo^o4Y77?oM6T1+#+Og0IT{cvsH24Qz=O zfy-Qs;tlg^YT3V59a1zE{ux4kOF7(LXuHtCrHovY^<0@{kF`_VGX(asgA@Qoo?Yi= zT1}Z981I05s9e7=JB&_uhG1RsC6xsI_ex6r8M9L7sPV9@+Gf`KNa<`hYl*Z7#d*63 z$;?6iyM}x#1QLP_8A@XR7=XjD4LDrCng=AGt26bLs}(CLx`C&*j2- zyD$nR<)ba>bY^$gk_-oC)51aC6PHeN#n{z5$LG2F{8d{-=tiyKTjf4$j9e8Xw|#CGS;kBnJy|;s}+5lu(C=1sc0& z%BRNqpEP>o^~P^Ty5KBD7*ML|=;vFB8(etDCP0VadSWVsbWyJ%`5cZ1$mg*Sp>nV>)gm1cmi=#Ge} z>R`&f87@1#5*aHM!~IceCKFmB!aWfi(f2Um;h6~?5fPx9PXn?bjM<%phC359MzZ2@ zj98*EH;x0VnZksW=9xA;XHu&Zgw<4;P)Fg3&~wxC8{}>e$KK33?>on|?;wbdJwocW zzB=k)H*st%W7?-9r>Awjp8U|2oe^S=^ke?cKM{#VDaRH2)lbQGJKh^@m|=B5ur+Kt#hD*FB3xG1H09)i%-? z#UT7gv)#`m#&BokqC#6(?LG|*dJv7E9(ES}s50G~kHMf+jMU`+>gb2WkzrsOyWy~| z*YRHs<(QjJ8#G;8?dApd1z)@t?#$aho;s0%pxQieO=|6HYxD%%qA)Jks>#ef;DFOS z-`_cjZdoHgdgT;4a9Zqy4-ixlYBBn04-u{W0FYM<4s})paa!`f>|sd_6BTBQ?4ve)qy> zDsdZE#t6J^s}RHcRM%<{dYB{LU%Fg#3mn4t-8b0TAfo-IibFX@U1v*Ygl-|k+WF;T zx&iZz2MHOjDJK%`X*p6`ExkT2YL|wqSsiw*HIeZcqZ?v6Bs1;a4>KL%QogTEqj1D%Op>+afWjC+_PKw$m%n+m4<@5P*r(jW>xcW62o z{+)yHsz)hRj3bnpmB~Dm{F|X($wkNCXi|Af3IE^5DigKgq=zAVg^oL;8DDKjPh&f9 z1{g80Rvc}|lh-%R|9-YKUK1y2aom=F$l>OyOpM1(B+mP|jR7r2<1rHlBN4I^l)lq6 z9rcH~QUa!GP`a)Q)VYcUsL3Nh>7FjrgB?^G3ad+fc8&X%hg7L0frb)|F5e6_nExJZ zhv`YDP`}sNZe{*?jKBLs06fzkV-#ty#aqq)Es0R$F_m~w71w`|^)or@o?Co9B|WPP^Z_V~ZZvNn&vjtLb644{%X=U)2;a$yR1(I$%`XmTJ+_R(^SkV*ls~Va#U!s_Tb*DVDo;e>YXswEN_)9QFuUetneN zKgq{1>IovyW!iw{ZXm=;!oA)IE}2KD3L#i@nBKGq7r*ZDQqNRfol$zX8>s%8xX_=G zMo2jRh9bfd*b4P z5U~@;+TLR=6Z`(ZO9%|(9h@ShDz*19;63<(fSdH5uOd^sGJ)|GlEro)LI-1}duJJs z)>%ca2?|n!trm?7K&MP2zZP05#2{oySp20z<03zZZ5Qx8ASmb#wHVN{){;i32Kgz> zVU1Z>u^gJ`n_)j8`NnY&E8YT!RovG(P{+$SvWNqs;F_%i=ibwDs( z)t098R2E4~FDCoKb>&oUh(ij*%!7D+m(6m5-dnBHwXc6d`Wzhc3L?>8t>P#d9c}wF zLa9H#0c19By++MZzlJz8BjKvA-|ZKn!dY)v%7GxTt(|pl9U7r)2ly5CZblFpz@MeS zO6fPUlSD-*Qu$U0O5EaYG_<*N!UvQG-oGv^QZ9k0H8w{@r==`WC!`u3g$=;&d3O&5 zof=-<;r745A*B`JVD%(*ZjZ+DwBlEI^5s`}{W$^h*a4zM5o*TLNnAJ2ZYCvJGdynw z^5Kruduvp|@=Y2Hs$D8amVr8mdlD+R9GDSd+tN%Lxez&Oh)&@rwDhF8htE5XtVrzu zU}i~AUC=}Q4gP1d6JAQsW$t3Kqe?nqueiUyAT$v{oj^l-2NhGf!v2&x3Rap* zpDnx?{WGcYL2}N{m{!IuJkQBJzpEWZhVqnyefqo+(xs$`YkT-l+7kyHWGc zqY|k1wQ)eM>Bs>30bsoNls%uVv8?+ljBI>RhC!}B9J;gvKmAZD_6C+GDAbXf@v~GS zDE)ljMDEu<(2){{xiQs;?Mq-0p!%*ss*Kj|l$9|jOmYvxWcSKbRnf(kTxqZ5hg?*!&Lx2(WKlCD?QtmrmR7soT&Gb{ZR8Y+2$u zzc%QefotkL=QuwrJk2Q*5{8`V|IJjsi z7rz{YmJlT^FOHLrQ^{#0%}@O3@|WtW9vj8jIHNX)y@CVN0@qrgaw5= zx#An95-WXbMb9&Y+FW+MG4N+H3Pb0e5ixL)J=b1ERA~N`fZ*tJuyO0`Qc|*9T<{+P z!236pX28Ao&!dgat%q=?Ww%{+vVqpZ=576@ys_=~n!()L=p%#@GK zYzz4MH@?s>>0Pl}wBzL2Hn_>bXI!!hE-WGr{gU=Yk9hg@f2`ui{gxK;p%3^7js?IOzK;J9eg16#MOqtBVG-XwaGQ;4cyxQ*jT?db zFOOk!2S{fsB^-Dbt?klr@nNj*_+RM>Y0N^O%3ZQY=DaKohGl#Ol@6@=tuu`fz)!KS ziBhsRPOQVf)V+EBwt$#UUJv;HV7!Ar$e@ps3=t14-3XBIIEO+%`aTN)SRM~Vf=;QN z1~PSfRMSN!wdzrUdcFDV4jguZo7b<{S(QhN0-SZ&eNjEJRS+GWfwvil(X;;xa!?W( zqA9CTi5W+|ZSY(5W|A;o@xl^4BEt-hm~80wS~m&=Q=K`G{b~hbx{a~MUxu{#A?BIK0mWgGHM-AXLE_c;x?PZb zGI~rj*C0|A?&6V@tLT52T(i&5v}#^46gz=_bv!RwE!CiklKT7Gg=N!CM1&7?vIP0{ zO+=v|0_fRKR3rQ>13mNb;Z_l-#~J6CYBoS_iS3qV4W z$T`4NB<6!qhgRZsL(@TI~QX-+8w-&l*2g&%m^ z7SxZVd`Op7`Ega;H+x=fZfW(8qG`wQCBK8}CX*()-M-a=MSU@qG%t?5Bfgto$)N$} zrM^x*r_=8oeXhZAjrP&sT#;-CvXVtsy?3yn&G3HyjCns#OrWH^Z{^{X_u(Kdt0vp< z8?{gO%7NNX3$Kp`ft|XVdM-0~Trgjo6?>w^+B;FMJMT8d|L%+}7;;QK9^3ykyu@1A zT&A4WPF<(LxUDS}txh~n<%G^skly)|)NXro%`tqDyZS9@4pt z0u_J0V-AbKd*!gl)by1Xb`?`CD(*eCMPxYe1fcKLpp6B{kO`$mb=$5UwGc;px~Llj z|0^3*5w0jFzv?^oxuyZNLhYCnJ=@8xU$fOR=jpkQ1=7t^$Phsn1s3&W7V2p{Uvlon z))$AAr^|~|&?LW>y^D^^_V&>hFK0E<^}+&)ezDB2I98U-pA_u-%G26^85|1u*epA; zW2REVeTBZ?eQ)L5daz9dg@SkM)i|q`T~`*tOBTsY-+yk=eCp|IdKN4RxvYQL9l@O^ zXQb6nx@_US82!T*{;pcq=M(d#i5xpP2y^JTfeVyQ>wEiZm_#=6W;_+EEXH7%!F`wS zXlid;7vr3~g%M}0305zyXmT=f;|u3IyY&&ReNZG7J?Ir-I1t;PJe@mawssmrvFNqk zb(FTnMOvOuk9&#&>{Thy=8s+Kj$Y^FhN5gIOU4}E%Z(WWykEwn_!hNV>cf6BgGMV1 z{$Fj-+2}6`Ou<#9A2bf8#yL2QFhyxQF}upG|$4yuKChR#h-t7>?Q^?0I!z3 znV1@*^_fK3*ctsRdHo_8mR$R}#MNj2bKOFg>uSz|KP9I3u8(rC;JjHMyS(bN<(pFV zo(edHxJU*#)o<($YM5))4PfoxUw+q_F{t|L}_1w2=NkSdiON>f_FVK3n@4d{v-vVe;yFnTYHC~PSJDZr=#2yZfL|s#6+~H6 z*V$SGEuwOx7e-TylQ`Szw^V6rMNElLcXY;RP}OALT0dVr`ON}55H=#Oq7maLYWSP^ z`oRfoA$XebGfAvY9;{?$7NY07&Kz)s0mCENK#CFm3cOBgT0dUT+N`_%+*XGTYgZp9 zRCCnDh~7EUAg<~d70$cx1J{MMHvD6B6!>Vt%2roO?X@MEgYS2Dg&W}dqG5u7ZCiUL z$d~PB0CTLq3VPt|0=SuMY=eF)=h^HlP02WdH!p!FF@!DCwQiQ&XhqHmR|x*Z))O72 z&dlB1+mZ+xRv^27_EU2N==p$GnJ?CUd!7R+BJ^iq(8YJq=s`&-UX|x+*Vw$Df%jAy zh4Ys^O$WBV4^ePmGHcG3b0s{=r_bq5Qbrjg<$`v|r&*2}R1?xSY{@@-@FzQzzU9V`bndP%r zJJZ7a%2z|($FDjb-#OQ}JT5lPDmiMIR%HmeU;~7%Gce2rdoV-aDMCzeJb}jm41LUF zH(PShy;C#XFGvOl{NtoOB8_3KugWw$WPjk*Is*N`Z9lu#vjc-mB19i=KYBeE)Exy@ z$5j9EZIVDbZ+<)wsy##fDlfCESzwvqG$yz$`{3R`XQUJJ^js7Fn`g}3W z3Z%O){g)}Q7k9yp9V;(;PD01Qk~WbevtKJbt%4cyO?nk8-lJ43nO?wsCl{hnqKMh( z1T?Z4GJXbz4kfH+g+^DT!QH3%mfdZe+}FZg;gZS=qy}#bkZ4FK!x4Uy(E8jbYFN)& z3xDB`5*oTPSR^~lboZ>#KpO+=R$R^L{n1zTU)co`4A!qvIQVt#zkj;MDR|UX+Vo=a zfXOQkF_MKKy6>)SAwkcB`hN+GZZ^z!ZvP}IekAlEQwb`30C1FcXjBg0{09hJPksH+ z%TrZsT6$2g14JX###%otrJEh`gx7*ZyS~L#De`|2x_wyhGq4#?tP7se8WvugJi2~( zQygCC)F+}mA1(v87wJLuKK_1_S3c;unuMU#fPP2`O7a3RX+G=dG;>T@!0Cx%vA9>z z5)s#=0$e`stR+ykBe68O9#j23WW+g8;gt?LEtq>#X9RVFyoP#%@IKz?3HyhJ(s*$c zQ>Ne7u^tr)(NtbMyOw77EM^@`G{}tfASwT_?i{$znnCkB=$E2JZh@>+NV++nuk_Q+}6Cs4d^bMiAy< z-Ef?1U@w?|s^FRHbi;4)^W3T}>!116vW6{ZugBJs;~Ut^>?oDH;whhZH`(Fr-^e!@ zFq8~h=QNN6+>hx;(OHxOLNxh}j$NjD>!tQPb~WKGO0-ue-O&Ez<1c(pBnT1iqLHtK190Z!!P8*U+1&=K^83Bb#*M1rXdr6-=Hpx@d)q9xLcj zL_W6T?-Ub`RlBXw(mQB5_LO>!C9SfSH%*l_Jq~9RZaw?(9FH-gXWLe-f8jfg%3Efh zKP+pJ&&JxsY0N7{ms}+bYD8%ijuE-b-4J7iR%lJQ-2HV6cHnSl*^G&iOugSRMo@Z1 zmCk?$DM%4{uOUU)24NVzn4w~50_=*VHQ_~*Ud?pn7{3ttw+f9sU~T?Ef$ zYr_VdBPvoMj)yW?zT`lmG0?9!5ME?pj5H>^c;dBZC3M^^DJd~uXR{_f2P25&G@sM= z{O^e_IvNY>Kd{ga(9f_^GIcPe>wBDWt?3|{i&^tRXb~b!&-zl>whTR#Byjjc%Fy^` zHs$4x`Jn`ha5X+gsCZ?4uj^*kLAY)~q)u12jV!=nQ!*gN`K}=S?_U?3c10i43p)_A zu2=rvutEw%Ad2=A;T|@PQ+Nuih?{^~o9$%Gs zULXRY(SjAq28U9nGCsO(?b6t|4J*0Lfy*2Xm===gYF9mmw3M<8`# zGBWrbmj0DrBlBROxPU)TEnb-Hk;OL~AyJfjhUSBunBzxJNNam7^Yn{` z&T&MEj0-l$v`CEBiHWF|4ns<=z3nAkUKq&p(80!!RDVa3C{aY{r&-6JoKKQ>tqAvd zpXfLUSp#4nWh^2r!O2UIg$_mbto2|HZ;=K zz{?~sfQ@v)sb*P6E0M=nD~E2X^c_tTFyB*lzqgZ?yg%X7EYFD6#Yh%X`iBSEtaZQ$ zL)D&4xmFPg*I(@Bv`*6Vl-rD;@Q)6nc3ZVO{NBa#IrF6kc5SX}g-wZ^>33Z@S@77& zH~Gv9Jb)H-7fLX_P;q=Oa9m4`E9{xAo$LN=0yVb$# z6XG+Y=X#jydPFaIiv>Q;eg3a+$I4(a%NX@t1`pTltY1%p?GAbw-n`rEPFAZlZt>y8 z@(T^_I~RXdmOOPS_Fo$BkGz9+j>YZ+wy7cDaV8Y`_r<32OsJ4lrs}n&md(p(j@5K` z+AaEJ?=G1qnN6ld%lV7l?Wn@!Asx>L^tH8^IG9}y*zOlxBv0#pYqy4{^bvy{N(M-uC8&M+>>|k$%6Q0 zH&}tymE+`H%<@k$lw96@)t%V}utUnMB0|?OYm?NU1LW6Mo8*te+yf0iNL#x#C-(0`g zkB3pUTYbl26IrH|&84=`?U;gDw`VTvtcLU$Pb_^v!;NvNIqou6p? zR%eF;h!%J*tXaVE(TM-9Nh{)5O~kwKc53_A04UlGF~qldpNF>TE%VwvQx6!#E#g<; z?^nv*ewWCAbw3JgfZgpBX@T$+If354cSJr0{Vg}K6ZIG2eHXEj1cgZHJEed_cEO|# zD-*GmhjVI0zQ+*d(C}Ub+4~!t)%>Jr`@7ZrNw_I4VDyJ{lOfIJlS=kyJ;@R zlMB%=L%vsO;rN_s+P>!tbG^L=+gzoBQrv9~WcIrYph3WpU^%Djq*$NndB6+Gl>GAcuyN(@4f=cV4R78=*3S$}_%~xI@l$4|LFFm!u z>hEj>Q6UpwzQeH!K&vF6V0+$p^Z`c1i2;eSeGk8M<`adpZV)4(*_R()!yU{%X2kar z#A$>F=YQNpxksB)e`RQC>iTLwT;%vEc(I72`4|l%K0nQkR0~ zfnF~^{dgt$;oKgmOY3*D*8OE9oBVG75`hK$1t0}VBkisCZo`**M_~LnmEp{v*ib-s zWd42(o5$67VDKs;!|L_oZv-&o?<$?M8Xyozva0y1bN$EPOp!*j-8GTxdGIc(QNu1=m$LN)z^G&{m%apcsjos3E8$aP&;+6XnQj@@TdRw z?!zH9z0%wpLaTXIn6;*BNmT$&9t=nYbe*R|gO@76n;|$%lOcVwA5HOouoXzXdV;?l zqCWz;O`P2??@{R}yx7B-?*|LS4h)>Vvuz%qLStUZ_r5E?$< zyR(46ytOed>B1Hn_3O>Q#Im4e%N|`J*VkTIi*zAW^XN?b>VU7iq|!Ib1*u zN{{nWK_f;K&)tP@5pkg@r>|f1!f55*Tp#ZD{_23(TNJyRm{Iz;D%)blV?sTz4hrC< zUve7-tm0*)u7Ei8tP>ZNxG<2-4tV%rv2TcwmPZFP>Mggk--{9(mI94^f;pbvvR?BQ zMzwpnk%c2NWzqc0&hg7Y2p)k=TKa3lH{}B5La0JmHu*jFFFIzml=&86BExhFKJNsa z2Q&4QFnHlbk!lP&i}GN>_r!~JF6et+@+?Cl;8;Oy{}%&QbORjALP7`UC`*46FrX5o}QhIH7#~HB+M|6LI9Y&D5@%h$3Zyz53DuDi} z8f0-js_a_?B1`1{xDEnnp$KJ5|J^d|%?lUm)%wv0PC`afc70=nmkOMz~=b--)rh{+Y+Qc0wKMkJR~ZL4w!u#H4zQ@ZR#(FJD=Yf z#~BYJJB+ZV=~Eyy?&P`R?UeFU8^u;#Y1_@5Jr`+rYAxQ+!WFT(w9U|bImH)@#!vqVMSLzd-BXgqm~I?sZ=K^-=xcy!-O^ z1^D@-&T2BAqOrckxzNk5F{H{(kqa+`i~Ov zin?x+$V-J@3Qp_eq7cH;(#`l3pUWb)(AnOWKkrtg{qgcmXz)}*XbhDJ>z{Fh&|6#k z)%_oBz`vJ`O6j)l;w6VLGr8uw%Ldo}U;EWj$XPvikR41i=4p;b!x#KLoordF{=;e! zB2`Q%-)W!NGq)}i4}DFtXr#X@93(nVF**;w}yVJBDK`i1n6srF4Ge5X5LHFq1^qqUKrAOZ}-`72l6+#y9 z_H@a|v_0WIW0#ZO@06s*gWcp;1OJtvK%UQ%emjM0%<3FmuHLSO3d;LR2Bb~Bhccjv zJ+>22@!EZW7_W3KIlu`66ldEE47_&k=7MNCm=Rf0>{}n0D_;I-)YLj{>3~Ms&E#zMuHo>+G3icup`2d8wc=o3(ShpPOKy=sKN#1C(6iR1?z!`YNUKnA_E)R$yAZ@+S(QYc`kN>D)+D>f<#GKqctc5sP~f@Z#i|xq(&C@ zF5}C6BaU8#R73$`U6pmijwMEo9?fR8udG%U8+4sFkN1R+eH3o$D}yc0tv(Y$1e^On z_hW)LUxX}Hc$Y>Z(K6|xm6Z=ga1&mhIuB7MaetHYHs^!g3y2@)`ZA8cIm1bd!msNF z<&mm906kmxGS|=u4s21@DD{Tzwww}c z-8n*kdgrzEinjJ*{ygz1Qj(J%ci3dGx)y8+NG>UB`{;NhcV#E}9?3(!5nM47`jO}q zZJ!+^MP><1qL_uwZ=Gz1M&|0!PU?qSpPns7oI__qmyKYG$LmdHHLb>Q1IL*^y~A$9 z^Z7^k(2GWKjcL;h+AGbY7Pp!^SFN7HCb}-KB24uDnNx=xzo7J60ECXoYO?}Zj9`8* zPn7-{d3P`N59mHD3AC3UypJr;fT7oi6N_u3y|vC7iG9x zB0+;hN59(p8n(@`odJ8vDxaR`-gXKxcpPPEp7Z#4hlDUnF~U;4#FjXvvh1+bPiUxq ztQe5eZ&SEFQ4RB64^KOq7lVJr6~4D&uz%~snV@(~&UEFrLH}yQUV&}Pqu-nqtKAVr zHdCqf38g&KCP~h08QU zZ!s(3X{zTk2BV5OR=j1)-hHoKhvi7;k8gxeCf-*#O*9C}5WXTIG37Dojlw7=(Xpwl z(WhFg@dPX7RL9q-r|s5PnWy(rH7Jj>`|CxFBI1PO;#L z{1C*-&H^;OoF}CeE`dFmP3_sD>qvp$AGA0Aa8VzU_e*)G%cz>w!gYTxm@BF3fVA&KM$_-rocijJ)1xV=qW{+Ir>1cba?;VtvZz z<-{UH08!9aS1XC|%;?BvO}I%&Nz-7dJi2CZkm`}73G5N05Ebmr<(?78wdp<}+$7_+ z2l&3!pNR_H%?AaJ&R%e~5iEJLu{5JY~gdYYB8Gp_4L%#t_S~K+@1n*dVk>hek z>P(GQ9#T8Ob{grIm>T-zO5>3kO>C2P$ERd4p&_VmIJMeVysg0I$wd>(OWos{qFRPE zhE}%Wwm$Fw(DapIQT1Q6Go*A&r?gU1LxZ45N`ulJ5<^JGAl)F{9fH(LcegZ1!_eK` z_wc{>egYog%*o%`d#$zCx>DHZP1U1Jo0YdT&!v%OX#W3R0CJ4F!h7%a(}ngp7xv1= zbEShED$%*pS%}&#Yh`}a!a#Z^;L?DJv4en8Z# zHvZ+OinujzG-^CJieJ6ev51d(xr||Kq*p52)(Uk1(YK*@HBx>ZrW-|-FU;ZeTl=RX zSN<4m!~(A84plTCSrKkF!+(RX)oCdx-d7zp?r+60Wu^mAOf+D=y(n@Z`xk!r=Xpy0 zsBt~Put&C7h+^PqsS#yc*LktidEW3Gg>rml^d@Y{;r@+tuS^iKt&-KvVcl|f{TbJv z*HQ-obm!ruRR>!>Y2+x0o@@Oxc8=@I5KsX{zSWy$=?QFfE~?hbkVwQd&k2F2sj#ll zhNkL<4U1jjvf49r=c?MSo!Lssek;MpvcuYCNU*{vF%w?pwr$&rAe3Ms#KU1IJt{El z>9*`XGkW%tYfN&Lp?tOAla6$J9Qm^lAZW_J*AJE{>A(ENR{qdxkfrke9F^ri@`9D(&ZpOyTO zC3W{-soea`^1%Jj)YsRC=Wzr$r!x#yUj@3R)q3Lz&(k~Ho;;N})V=Qbr62O~jqi&8 zd}xrshLT}?x$oUThbBqU4GJH$i*zo_?e2-BdlNsd$(ZX{C8oLjDs4P-pjCLD`&bO- zM|0J8TT0)H9VP5LojD|d=QK%(ZN#qgh$)3fP@lKAf2?{B1QD}FPY0rNCC&U;bE5k=MJS%r)*$43XQk$ufu?w%V=K|$#1&OF`KAz*N#N2IWo_S?{P1W9?el3DmE8kwpWh?0ZQRlbuE05nx&e!V_teh^2 z5h71ha?UHIiBpBK`&dG8`d*Jq&hyWdg8I0OOJ$nGL?TUkOE%BKU;6C+%BMQ>H8x*% zpIiPog&TmGb+?ZuN87LjUCvG4_4tN}0Fpay3iY2~jY8;iX{YTfDJV4Gk6ho|5rLTE zlWv907yiEEr7~${zgF-{JQC1<-d*eO85TNLsT_2aag?|}Rh&1^&Ho!US;YqKRhcNK zuqS%kHTvUZl=pZT4-OF@jOSi|0%Ds>LGEmeZ{F0?S_5Ze@hkRY6!}pVfg(9!y%#W% z*-d)(cx|QWkz(3M%Kc%rp z=2|z9T@}-%l}Jf-N!0aT>`}Q3d(H1~n&u33VqD6Xgh}hZ&T#~?&|Q7o^MYAYeA>)$ zC|Xe?El)&>AJP}{*Wf(buysm#;{#Qu&(_G{KbW5v>htg8$GQ6u9S}Ah&B}+aE@>OW5xm2!cbQhN_V4-gThGmx zyGYIbhK_6aCp!^DUKo@$ZF#!t4QJ}sL*Jl~`FBXZL#G{mnL)$4<6$8wI`zA)7W8E7 z?ok#Y)8PS003+<%z5rx{;cBwG|45!ND=)ToI#m!88Qoc1Do{X*`bX+=ySG?c-}&Fw zhUAQ!+WB1T?w*PqZ=TmwE&#;-fTsDSmIpTuK!6^i*Y8vHyqqHd&`ykv8=pqDsojxL z*;x}Sf{$r1bj`jg!q#EgCld^k?$mQ6;sbGtWuQLuxmJ6lf*S{WX_t4D^#crl?pYEh zmp`W7-I1B0uUvaT^XOW~U3e2yQ@x5z^mu=*JT#Tab zO+Ng?dYwAL4h$6}HF`Hv@T_h5eZqv2(W-&Vz4*QsN7lmL@wnkqBQ>jXn4+XDko-TY zh5}9~i0A}UEf&1Ai&Tl1MeSQ~sH8~i?QF^^#3AdMmZxYeRfPvp|B}1AzuZrZA#9-N zP3Ef5gP<19g%(l3d9F@P-G+M6*pNg*Puvsdc)28c4h+@p`%GRXq{q?b(zR%+ih)0BIaxGO$KjP@DcfM%h-RZS* z>iyJ^W28`^r!n04O9vfvkBJkJJO0hvE60Zi z0V@E_2lVZr!nr;8*a;`DAYXny?0*L*m)c}vJsveDRBJSG>f9}@Wwq{h$s2|m2O_& zeNnnQ^{{Hey@j6Y@uWx-IbWcwYQ$s|*CfZbn(2Q_`cqeeVe@*Z_4#BRxs&+hk%jv5 zSq&D)P?fXQ0{Nt9;i^i&exZapwG^N%33Gb>@6HF|jAErLRrW5#)Rh>mhXr2Ow;)Eg zh8+`QA_EYnk`--lPkp{hcNFVR0y^BaVFJtji9PjjMZIjL8wj&YV{5!wLdXGKexXK! ze#Z*5i#iUlIZz7+#sYC(i@Lh@8aKJ3-#Cl{D|*=I_ybopJ+Q=u)y>_? z5j64eBG1ZWP67sJKYy|FqV|pU(sJMaHK{UfoxR$+`#7vy9TS0a+VoG_*QiaGS*0kr z7$9#(S<%ze^EB?|(~TYgaT{sj?}RRB$~$edI;2T_P~?o*#XIEwOGplYRg`&Vok&!w zo%Ws*D*74;$&|~LFLXNo!u*Pjtyt-m>sZ+Qrze36n_4xCkdTDmRwvjj=HqJ?X2Dk8 zJ2yuB<>PHK<=E49Pv>8DD7nanvX8ZkG$7+b{Mzc531LCEtC}T)PF-D5%JlT*RB2SL-R?`H(v6- z1zGL0mBy0q!`KC%ZD$(tjy()+pA;JW?Zp&Opq>S~%WiIwaWoB@Qiy4dS>g)XR?<8rsUSMWsy5?KPVN zIyW-^*Rzp7TQQdYK5xGM_9L11acz^^UA=J@GMzBH#m1WOwWbyM2o`zBxQxnnfb*}B zX=Wl+rpC}>)k6I*o$Lh<%Oh$3ZhJSy-=@@gt_MH%5i;{4_lA=aXF&Bx+d1?jiYF-Bd<<) z?3Y<5?_8Rk1`q``NJpPmx7D~V2|+;tEM}P|T7$d(KQ#b4i;KGJ;1RrjRdVFjYg7p#Qqb2-12+9j8{0tv{(3$<`~8$@4UtX}B3 zH>XJ!)bgUx>;R(uq-P(agj0nfM}3CQr|kqbOEsF7xL+EBjLwcIZ5%Z?3;v2G*^g#v z{V{CsmO^beS`QCgF-x^M048kfGQICu_7>NUC=K%GvFn+mSEOxRt`R)am7}ohLHc9V zCvZ^bW8$FMG%}52)i7UGtfJs=xCKChB0xrX86<%(%LTbBvZBaWL}sn@{FxP%D@h2O zSs<_O7b0zH4txD?LM`xKkF>OWj~z+eE+p$SGX3d!YjU&|t#DE?`8^wj;WO}XE^HEK z;DbgTIa&>IM6c4EzonVVv5zm*&RVy)WB=`zQAM~}b#+_&TJ}6qyY8GihrM7+_AB$x zO@{Z8I=O&*noa#O8K2oA(p<@m`C$u=Se0CP>#%LZQ?q;B!tqnxa6%3@93|!-puZ6( z_x@3li?cGsD4$|2oetH55Y>soY+Lc_c9&5z{|(gXzesb<{YFH%FxhtqO$R-AJR3cZ zE##&L3gmJH8o3ON2a%z+EteYP5U4?LCOBwF0LwO(PxE5#xbpGr#5UFrl|UKZ0oq;z zD(c|8^+5RR%~0qU9q(HMnQxmsaUqJBAm778zbrM|*h!?+3X)?2W&)5uhAk3z?T5tp zf`o=NP@!5ZhTY^?0q#2Y2LG~a7tkx0;UZUG@~zg^#ZO(WkN!T+9i@^M6q+xPS==XD z*q?6$A6RI^g>p(wnwo743-8WP09ksQ%H)~(@_oxf27jIX^?lK>moc$Z@ZBn@pY{|s z4xUR!%hAKV%Mml*%4b2Ly&w;qu;Guo1@jq^GrwoFd7{Q>WVQETl}v;7rY^?quRtC^ zh7h0U4HOuPIQ~gJSk+7R-C)xubgrj)0Iay2uL@H(?p^Q>O1FJsE_I!U5Ba=$js9hG8Tkj1maPe&FE_)>sx0Zu4YAI{yx)nu_4734TY(D!n z6QSMkA?w}zJBx+gug>S`QL`GUb_U=7)aIlEh}s(dp*r<3j@XmYHdbQ?yU25%d;7(c zbml_rXsc13o=+L8U1U1fiIva>Y{|MNJ`o3ayQSL#dZJI{bdRj5ZQC7%2MI#dHNNH) zHia3k!IZ8W042q#?cF|+z;b+<-0!dss7L^5q;WR$6WEkRBe(qKGnbRoCg%p%!fyTM z=PHI?v~-F%p6!Q3GPtW}T=H3BC+Cd;$!Byffcg6NiqG}LU_Dep-gisfH(ul00KltS zEC3PC7=wx>fWX;{Q|fJj`vS^-+3UEF)wsMAKv<_pmrd|zrc`h*{z9ZAuxXdL@SBOk z8W7)!`9A}Q-J#vKE;$JEbkSkk_)5s$k75!2yZmf$6G)F)seBeb54C9=pLg!E!I@Ke z))fw}1sph>+^#Dk=Gc(oJQ|UnD*!gLk(xQ{b6x-|4}METa(cu3(uMjCiujg zdLJ9|y^o!zGFJ-Mot5qCrT0hor}l@>eX+1|sso<>%=>u^^bD%jmqK;tN|Sm(a_kKz zklyUErM`43Z+SV|Qyu}-zL4!?yT6~ByC+Uk%P5N?^Sr(t2fO|+!QSm~_!pOB-FY2QoQ^P26Y-1w@T^$|pYOT(xS^J^W< z#(y?J;O}-@>H4ye%tYN`gXp-Jd0hi2VtGJ`Wh(M0+4!TfZ?P*%I37!;|CUx}cYT_F z(WxJ9ifT6U1gG%LK}$idAVI`CHq&kFyF|9CDSYSVwNyrs)80zitDN$eaSH%xb8NaO z*3OS>F1l}9#zOzK z*&zUFb&-2rj&Tt`iqbFDYD1&jn=D~q1dELntkh-&s>L{2ea87Ciln@PlKAY4;v49tWDA)FO@XN}7?fKID#NOY}!ns*^on`_aQUk->kPqO!467k;hK zdNwSnddmS6spd_m24iZxv;R%nE1Lh7YEU*fG-%R77r1fY)Ufunzez$U^+oU=v!!K< zVcPtpAdY;Dl=Eka@pJ7o)3^#K5>baW50E;62#i=kj!qSpmqb*=6*TBKvf(NUNI8#5 zVR9IQTJ+J|Y$y=CobEcC41}IE(`ge>pOCLBC##LerLLS@js9m#$LsIb7VsvC@g~CF zcp=@k976kE`)X12_Dv6f>XqzB&=+lpm#oYWuU)lRU}w|=T0{k60!<3O0`c$JuuN0+ ze#3O2#MH^dO~0*9K-Y@iOcMNI^?A9@Qa{>lAi){vpz=1|rOSfi*dA|m7xF-<5tVOw z7kUtFXYm>c3n)N+E>@T6+CV=EiA+LZO9u|Q%+4hFi(863YMOJG3S z1>A3Q?3~tgZ0jXu z0#E}DQ+XV4c>wMmlF2|LZaymZ>^WnW=*uOk0tU9x2`F zJg2ZNK)mC=)pap8ZSo5t& zt(dWS)AjYPwZ%`5;zQfjkKIvXs)bYhR@6G<-lO+zJ9V+{nR_eiCpQ*ii^lgoRdr$q zIuSDwGX$|*nQ6Bh!}n6XypYoBXS>#0Mv8M#DHqV3YnRFJoy?d3lVVtoV-lzVLymZ* z%uE#viaOR05?buf@La6OUJS0}-*?st9Tr%%7Ir`T~VeQm{x!51t^x=A9-LYygJ74!9y>(0JK&`D(p%_MU={ zk#ZPxVH3z!^Zg7Il42KK3@TLZ80ZuW=Crh1 zHUhH0E|3JN=NOI1F@MXbd`*j3#Jqpdk>vki&HS#z~6qk#S&PI{)c%{vUWQC+Ij0;#j zAQoZCI>o-?inD49XyFlJ&;7LQ*F63fZvtR2k~SJ_Vkz5^z%y6l^YL z`Pp6Z)RJs=&jhe010NUigmOEo+cK{?uAGOncuW~ZrMA592WIi6v&yFE69pp13_U)g z`~_D*pwz5BwF(W(e1n_=3K|gw{?!PbM-06iRCqfrQ}S0?KuWFjmy&(Eae`rh1 ztCtOaR8vhG;8XM^0X`CCOsywM0HW}57m{9(1#BrFrH=mPuM*U>?e!K|Uv^OAduzxLC1M&Q zqidRuyl)Z$7xFyhEWrY$M%9;$SN$jj!+eriH8h<>V&0J<4I>0 zTtM=teXa1#!CCbO0U|}Alyx7W#R4ISx%F)D?J8_sMRje~IEq&>k)sB48DFX60dvUqE`uu$;oleJzA(9*};LVz)#%WC)}l6{Ok@3bd-YnSAK) z=&T&bM8J4>SNu?Ay*jmO94XKXgiqMqAGi6?Do+KG@>wTc?oGd?F!_2+vJ#huFdj(< z3@xQK+xr8}kd{=uSO=3riph+=j~9f}^JT?59}z0IMzz-SzTI^0A=2E48~9%~n%f56 z{!DqMJ@bY4{b3^CqS(`E6clDVpJ?hN^%vzPRN;xx!X6Yf__QtdOiH#IjQ8e@!(R54 z87?6Cm4WPf0;6oTn~D0OJYY+c9Vmj|O)OGpQDDMJGaR zM}SJ`en|hxXuU70?oE{NM25#vLp40`b2P%(ao3?6aMiH5;>g$LY?}@b0CjQ8OQh1O zHa$+#i_uQ&%^Jq_e0!wB^#ih?2;kb6eE8oG`xOv~UGLa}<&_gC`aUIbr%B%Z=`USfS|p#(mFYMQ`5Y`B~d*Cne}#OHd+rL8si_`M2@?*fP`V?cu7 z1JH}H5xm_u1Tf%a;7MXC_>!5S1z3v_07(a5KJP-ffQ*)IAn++|Og#MtIDz;GtZ3k9 z_%&doh;6wDwEJkgNAx4W6X_3v7y7`L){%>!>@dsa|XK76=p4uegn&bNpGap<~~Jv>7ufwFa2@f3^nnajQ*}-HfzK zTF03@S>r%nRs7A;v3DY{o``fd0Z*_PFlQ=+A_)JKQi-=;%7E*OLMug0CU`4ySUria zLRT^>NR3Jq+T`Sb4=C339)(1)hmjCYqBXoMDwMPWq0(b)1dBy*(s@~eh#tMp?lF?qeWxRzbZH_7bEq*QQ>K|@#b6PLs74ygA#y)MijFK2KJ{F@QHMW1fgyH(`a z;@>7w!a%e3n}fimXI~i?r4w51~2f1X@dtx9k!J1N& z6p0Bg*}zOK%l!Bc9>!*Sj5QIRrT$%?P>1S>hJ%7a4yEXT>7%-zK~(O{c$?pFe>qEw zv8Dev)=P4+x(8z1A+GtxLJwUHAZDchgnkV*2SP_)Gd98sN^9p?>+f;A>Ze}gs|fXn z?2qu{@p>}hXEM~z8igUbF^Y<-6}z2upb`-4Iw>QX_o6}M#n36nXNHrxnVQ9b3*|TL zf)qRl?ZWEf2KjsAMkBkB+Y+siqZoAE_#1Yv$C^&W?MaG=|0;Uhkw?kt3cM==Z2Ov?OfxKKAPzNEJH~-v^gAp#j?axG=$+nI7>1w+3iwFk1gg); zBh`*&2CQRfW*BP12hy`JTY6)xs#j!4SVaGtQ+Ez{C#^pWj?m&%sOR>s_h3a!8JqU{ zCJkd{y@E^LfCNG4*+S+b>IiyG|uUbFEWia|q$?AMNR(3h66zA7?DonjSEbk=3&VfWKlNOHx za&Z(8SP{>NS-getmmfNkVe8}70uAFQ3M6r31gux{4nu4N6UmmiYy?JkM}Vy+6BJs# z#tP$o=o}XS8UO;0=93RbbxY0|>lp25(Sg2`en}%(@W2orCvocla30NF=Mom(`(D=8 z?+-D2PKKRPwB_IbRQ7tpC%Wp8o4z7aY{Y;PEc_DBwEScXlvQ^z5qO?(L~m%O$f=2T zlIK_>LIN?hvL_^lmM#5UR}&2KR`pl1eo790DVPJ`lOnBfgK<|kGYy#b)CnfP><}U1 z9-}uVDA<>M5GyqBd&u?Q*)%^yCR)C8RX7j|_}M~TKWUs!19_*tEOKOxgfJ;Pzm`aF z$BFy%TF6f)vtO7znApE<2qTrI#=yDM@HO9^rdn@6k?^1miXC!7Q3o?-CCfXVn2QI+ z#wc_PI&Ac375>7w#uH2KHs1)UMD>ESvb7?%BN43XV4ZSr)r0@G_l%lVfBUZ^DRIEA z;22+NGw_(7U}@>g_uACU_x3z{OQ^ z%w3vWx4nGWeGj2AG-#ZvQ(WD9^EKem!>(&L_TLAW$F2E~#xEQe>x}MQS@=+i4}9 z5HsogRihYtZuYwr0n#KUV5l959ws|OC$FWg1 z^6llOz7J*@dxbo~Hfg3$F`vr_BbAiC9?N+q79ZV7Bo5~gZR5=CGuQBY;JL>W1%2(24 zUkM&$W)bQ3jtuI1-!OoHw(aAn^=iRbgZY z%eU2*yBtXxR!GncHuZ$6_rB4Px{hu-?WEcpTOfS2L<~i|(UUP!QJY>;*|ea|gpT#} zOUd4TD4lGKZ+XL3N<|DEfr))XXE!rCZdiD053-#-2o)XiJvt={XShrn%?A<_#pN~B?}HRU?*vu*>vLt8!COplNjXQ(1R|GW#E@R3O?y(6hj9! zu72xN7j9!VTbkNLx0!I%$? zXR$bFg)!r9TF|}@2kQ7F1e;WZZ^~1(Xi6r#`GuA$2H1U&&-^Ub>=-*C9mD1RHZdU7 zZVsCZeL1dU>?^^bJeiOmtuB>aKl6d+YVeEC_H49@ZmGD5ZiKp;#@7#;zru_e6KrN} zDtSZ1hQ7=}0+JAhqYzp=u4{Kjwud=_Fhr%<@ZlCcpT7-OZVE40^%dMI)1X6@{zWRB z5<-;;xu*TdL+s2IGObmm9}Er)!Ygbs*i6)07({pf3;s%!=@Ia>A-1^7^uOk^o6sVk z&>{v@82|5M3A=_9gU-7gRt5DY+laA<>5+6(ZpQE$88IKjH==G`pZ51`-dNWVy{;J< zDl-o;<*N;4(5#cfVcDol1VV?aa@&$(RZ|*+jYEJQF$>r!6v%M7;y2UC9Evm&IH6w8 zNXy^YGNN0c7^yW zlV3_!gIXnbca|kd7gbIU#icop997uqk~-rZjrBUc*~aMA*60-a+ z9Fj)8r-hib+*E`9?KtLp;Q^7k0Y;%?x0p3rQoD)RjCeiHvo8;{)qVV<<%_XoDCf;1 zRl(GmGWr>HZo7fR=W7_Ir}e?1#@myWA{=70KL7bqkD`WHm%jDB=}c7H^iFa%SFVpLj$XtfpYlwhFq{7~(-LB8$KC-m z&;aW{ey0>7`n3DX;O6xHtsnzMngtOZ zOcjAsKk4Z%EwkqL7kwnx&{i!>HpZ`*njRj%H_?=3N<|_^D$ zST`V%SUvw8UaH*q+u+mgzY&$Rb3)?~<{M>q^=XZ{*0AMaY*Iyf!i?#+>SwBjB`PV{ z>Xe(hx5`NsIRPhRIv+^R{x}VMQP~h*%`Qb@jgNqaxb6P>!apF4f=-{_DTMN2TxApE ztH0iRJyBCwp_1K{I%nwn-AutEjHc!Pgm^qu(8Cb&dzelE%HBQ1SXD^}!Z z%xxborJOkzjH6#uiNL2S{cPgMkn`Ulwnky?XUa+0t!Ymh2@76_cI*{g7ox)t+Gxi= z%Ws$p#E%GFfuUUaz99ccO^ZTnr{Z<-4@kmQh{CCP2@_*>G?6U5H(TX`C4+G9ipjc> zhwl}}!a_q;%GN9sZ^{K@Sr4>Uv|6*No1zMB?K?Y@slAgjbDa)rW>DZ$TFo|(UW%Zg zBQDGet7fN+c(Tu6Cd@n#g&pd7XHmmK+BLP(b52qiW|Y~@ z`ebFF!6#E_H&FeU&j@G&BVnjxlX{8b*~0!0|6YA&5u=@|DB0O6t9J3zn{wMR>6%k z`l#-%|@$^`#cp_9CB%Gm&Abr`@)W!i%(7;Cy}3_GMDNGIfxiR1Jw?sVHs z-dYzKM;OzDap zipJZ>2)I_Z8X{;Nf(kj>d@tx%qJB`Ao6Wn)}L49eG(NN76CFZnO@f|-0k{LPLepAdb6R`^E*t#ebh3)$Fzd(2uDV~~!oj+*5cdKC&On2EUDhf2ItxSoSgz^uCpguVwQFyM}0&n>f84kgHQYOMv z#K?2OWmARUIS`ADH?lT4aAC-iuv2$Pn8ju;^owEc-;&EjpkJp3b((U|C+Fl^q;YW4 z>OsfLeC>r>u*=TOd(z0mRm``j{5JrC*0*>6Ed#H7*Tmj-GR5h38f_ixA;ahCR^qP| zDJ0P71soGE^eyDUd5Kq`bo|R@xy<8q2cSy|sGIY^gw!}3nfxiA`v9x2>`Aa=DZ_0m z8O{)^k26{+xqFp-e$6)UZzm%cExo+g1I69ml4J^Rz@U>vJAYO0;H=_-2g=Oz4Fww` z4TV?OjmY>@gc2yom*=8frh-sB>;nqdYeU>)0}wvu{2AuX?^o%N(AZ{tkV?`9%@;Dv zBLvP3RhppsUr%b?AnXlf{ZKMRMR;&4(l>fDllsfH`&Dm8EaSj;FS!iLf;ea$533!4#0N{})@?#tBv6=eoGzQYtg^3P=nOd1b=Sk`M6@4u5@ z-fHt5l$zVj*BffQx&Aiqg8NbV`xIquDxmP#G@H?MdoM(*Z{r3Ujj*zrB))S5=Kj-` zfGGIYqR<@yFR1j@OFX>WFYG^{_#yKFlEu}I!(7S|*^hGA80H_$y@pg5LVOXxbwdksZz!axp1rmeZEHDeyh~_tn$&W2egaPwKB<~G=o-KHp z+Vn0r1eE|3zwPftSXVD#IEJD`9~-TipDTO)Ph;jbV*0rj#zF{1Dv#Rb|_-DQNL>BSehg4iD zE2PdnO{YA?E&Ki~R6}(cs8p^r zQ+P%~^GrWLV=|ZbfS2o3=I>8=FW4*dm)}wmw7D23OiVDzCm7R3`OZRnvSEc)xoJUm zY+x*|pu)1ctfx99ik*|~Sc554@eKVHPV^Y4WQ4Fs@25R7B~^DjjXp+_AJ3CRnOx}Ep}?0VYfCvzHW+M2mR7N>0gm)Tv5xEij%;oA zU41GqTYsUS)^0v&bh#RBZFy?KOic{Ucb=0pD=@F^6FKYOq7lSTXUuUvyzsV;tVJtM5sHIo3t5)cRRsicj%F&65la~b1ANgmlAw_mE%J@F&*P=!GQei6fKovG z;T?dTu=?jHXmD0PclG{6zFfW5bnn30%}47`UNK+x81{(Eh_YTGYSCd{ShFz#x6@28 zbJ2%x1TD^?@(CL3>wpfg5giC7nZE>h&wt@}tThLS9xO+a#*4s&>iA=aC_L*gh2(bW zQHn3QFy5!{aO98yY(F~(rNHm_Ae?bT+Y32U_J7KaA_$Vk=EF4thJRlhC_&6M|FPOJ zi`XsJCI0wrYhE6aW|V6#6W$9K9gU{5L`dn^hUYMIbbqlTJBz*TV7uwzzwHR-$p5=O z0#bf^r=~Hf#QP>wRJM$@+)-EX0+%MvHnN^WeWC$P{O3hhN#{tqsPZaOGOkUCqE4O` zgKk!qbRGop@Q)y-H+U(y_po@J0AB%Q%jyv@tOPlu$ca@|Nh1l(<;N9bhOSFHeg6FJ zlK|&{y2-#jTxl&6m?kLQg+nTQ^Pqfqydx>^l;dE-!BP5&XBB)q_q#(h8B#v)f^*fx zrnLWSh4HCbaRoYKy>TcT{ulZuImJ!!Zc`{#Xv;%eZMaCYjp$EK-H84Ee?{s}2d^1P zXg{_Nn;^0>oiy6=g3g!G z7l5LYAqR*Zcq1^jEvy>(^mvad_FD`fL+wEXFn%c$Xcl0W^A_=4j-S64c2_YkVZs2$ zLQQ*szzy8buRp@mX$rD)amoYY*+9Fi_=d@O*MUm?cbnLKjGs2)LmcYZ`RU}k)j(|$ z(uLmKu-CPi0V8)e=Bt9;zY-n1jWN{y@_r{*r~ECs}thtTKTlg4Z@ zCRT5getuGfc8NdY|3UrM$~8=5_&4!gOyyh}Gevq>w-jqqe!n5?pxCeI2knnl26cu7 zjv7X`9a3ie4>=7kU4l6^o3bVQs3SgiZiE1s>>zR4Gv<8YZ&3XJN$HY9Q14x z#`bNVAwvnZYSCdRU?j|gkO&o%q}`^3@5ycacc6QkkPs*}WNjk+LF#@M(`|?m>sjNH zO)<3f_t=xA&%hunj1P8XLVnO@5nno2*YaaG%T7(u@sQ`XXumlr;^@Cdtkm(j0?Xvv zYOb!*rN3OY^S=(Q=bpSub7mY8v>Yf)bevVIdJ=G8I;0$olE`t{x2V2C9pZk5N=8|c z3hwA*N2C_3>?YiQ(tfUwk+9*R2}sikS`NOhYaeaqJPeQ2aKmmuY-ja-18Dp1@Q~}Fxq{a{Qvt=tWPJo zSPPg|1*+R$#?<{xO@lxNj($-Q!@6jq%7S%jB(0Plro&Z7KD<(Q&jqVr!&1NbhDZf; z^Zvx{B!BPz8bRsZ8OPViBc)f^Ml`-?Qku-m@5N)$pTxkZHc?xdg zgBtWF8xX=i8G?Cadd!F{^;SltW$|t$h!L{2p|KwCmvGUMn7*L-irG6@6=)w`j3bat zXyTirR!Sa*qiHPV*8Mm3(IY8b)lEpNFd%XK1Dn08;E(vF$zs7Dbpon=tZebRSnB8K zB2eYI={2Z_t3I!!(&uLEmlWL&^J#}gH~_N zsHl6>O5<%|!SE+QDW7M9)r({OFtJ>(`uNi*dK$%Iqb%Ro5;knBUYx4I5Xn-9n;;E- z$6|G7-6{N~UmkSvf^rP+mJmV)_v5>FQH^4g5ypzSpprXk>{10JV5E23?A>wSqsp0s z!QwP`13eq=e-Y^3m#M{$V2ScDX_xCr0ESH#IrVhcw4epy-E#3=7$iUM=Qa*$6}cp*!^dOdXQ0H@ep=%`A>3# za;RqMx3ri_my3)3)Ua^(YS*!FTkjSRXy{J&n4$le{>+~*E69*eP{sQ_-Bi^{EWfzd zeaI&YTh+$8uvOy?3^5BV7@UZIhUA%@YE)+m!laQQ{jx>Agh~a*_9bF)X_=fAhvl_7 z%fRB^<*0@7T1=zE)nvB|y!wu^XQA@51m;HvWRC^Q&lT!r^0y`{x4V@e%G)$=kf(Sy#v$R0jpxGbi#FCc& zZc^IR{Z4ll{^hCAsytZ`oEXqeo05kWf?xAS!K%ryJtMax=hF>0XwkXLZ>`un73%Qo6bH+`QHco%hxbaSPFVxY zg-r|T_mg8s;HW@{0DM%&_FWd1zkX)ou*@3&lP2k1t!wWUm4xEg08(2c5QZp=^i|%< z_I(6y#Ku+G5cgGg6^IegnIqsc7iCcWF;LE#F;=?6)5YZ(ui{l$r(WU!iDNWp#^exZ zLRPl zGkuCpA2&+PSzH@|3stn-=h0tY%1QgsMwMbngZ9cryGu6y0ESVxCr?Z&0NTM*$R7Ep z^+in*jNzID|H&wW56T;xm;x~cnmq+rr3Ju=_RY;c(|KetCx8>WE_HZ_mo(xldhOqW zs6_0FDv76XXJ!b9fhz(BCY*f}aEb&V>5Es9C4Nw-ex*JU1czpM=d+{IfNH%vL6G% zh+pwd0vKr-ccQ%%CV>4YCGn~nZJIb+pHuWZ30%6?SH!qPZGZwOGc%62R+AgRVMZsBieQDf7j({*0u~w~ zHL%ZRnYFkGa9Y}H$6^$C0Vs6lT3lr{AO!iV0?-6tkbIZqnCPDP=uH6z;#LGs4jc?^ zmRBjoSacI+chUf_0l7xN1W{O+F;>pC=>k%-#c7$LxaU?&fmd9LI_*B3klqQL4|&!e z-Q#WMl%K7=@-kbZni>JDDuJd-iGdJ%q2dAnOD&Lx5^wB3cF!r0&aQ+M@UzOnMv79TJ(KPA2-}45BpBywUN>t8l`iwM%nZUS%UgqICGSa@K9u+J7nF+OelS2wZ>2)|B6R`=wbD6BbkHgxg~6>-=k8Q$ zRaCoKQ>=2~a4`5dn-pN+Jea zYH_xR55T77^VQv#3?igjacequ6BkP1(TVA~=u+0HYY(le5r;*IbnVjBW}}2Gu-8dT z|Ek%SM1JQ2a0G5p9In>NwN72S*~C#l1^|{2NSZ5!&`v#ufdE)5FRg6;k=r3N!6{!UlpAEG${q5fe3YwEMDvCFzyRv}F!XpeYTdB^c$CRT#D6 z($v*VsmHu|HNZ0h)NHdCCNIWwjiTe`Enyzg{zRcCx*HhF|4w9GsPI4z-) z<2=R`X#Et(3E;pDfFr{2*bWhs?%7|4J#2V~*hxbrh$Z-`klqa_CM*fp5Zggq5%g>T zHtg_=N{jkWfeV2HaXzXwa92unR9QDmQ7HaSI>U)05Lm*jrOqM#IpXTfonlpCmtcV; z?bA`KblWSbO6@UI*=gaR%$t}wA#g^t8;^jIbR7f$lHxJ54uBSMKmazCHB(d!ZH^Z< zjYyd;GZ1w;Wmav+Rog{kdC+>}a zhiHY)OGn+$Kp#h(H3!*^n1!_+C0E1Cvi&@H}E#0S;Lg!KiGtsPiuMaESsgHBHE1^ zVhc_B-r9+ETQj%9v@3&JyTr4a;M9uyr@al#rj8Qp6X4wXO)R2n{dF9du;o)AH-Mu7 zvST=WlHooHW`ZaQet-Z8Vi-gq0GhCIp9XxuP*OJ+c)-3i4tUfD>K#~t)lQY_7%QY( zQi0-cJ5EOx^3f7%l!-Ou1}+h2J4u-(0G&FCZK|{?3Gvk&rAHb;??r&CT_@?-%${gf zb0%9tVlC2<9&+##FbQC)y|^`6aW!|E08Z_6J+sW5as}R0l7XSTp{Oi86aXA=!zO<7 zR!T7l@C=Gcf*bOn@`#&LO+Iw5bt>9asgreHdJWM88VNed6JnZf-KA@zuq+W2C9HPZ zT&t0IYO#eo*osRGu!TF%vK3Zb%qC2lZ6AO2vn|ti4V$Jte8x?ZVzNMAdu6OzAZA8~ zjy7)S=ig%A~+)*O2IUE5YLU-*c00Wz+0J0>H}BiG<3F+Kv*t*oJKL6Dq|OE`uC`NKZ9k2zu`fr@ zwV5cqVbu7T3NWb%6NIE|{2Nmsra+MtsG0bS&HZYy)eikYDi}J@digD^{c;;=b#C_} zV{RM4a|1XUoEf0oNx`=Uu7zJaCbDCC-reOW}=1zQwKer zb{gd8YMFCFgE4}vum=dDsA_>0#aUS=W!ND062Oosshxlg8wJ*&Gb2$719{jePqRpd zV?;t71s0M@n@U6e5*onU3ly;n@LZ+Grhfbd*d@O;02eioga-F^i7p!aIq4fvqaR&J zX{rTQxB;xteTU`hx>FYC?Tw=t1(!+I!>>#wT~19kMm z|GufRAwxUalxfl-QW~?T*I1Ws;#~DxJ3PMh5##m>GuJlq|ooaE9 z5Ra+eI!w}@JEC_*PXQ)@E9viWcq-eams4Rf=HSqlnzvyxLK~PpZL&=nH_N8am5@^+ z*Sb0md##e_G-2I3(azP#_)sE-cBkO6i760Mpac|{JK}SjIrv(u`$}hIX_CQ+H|z6f zZ0@@ISf`B+wyGYBmmrna9Ysz62W8Y5TAJLYeP4-IqzA)5E$)F3gh(zC6(T5so4I0b zbJPK6Q3Vl>26;p(h_S@$3E;)1Pm`EQ92tR{+UlNq2j@iIC9(ksu%xXIOwlnz+H7pqrFgw_$vyd zCZn{J;Sd&eT^bSb69n-NKn;Bw)_kFt0Oti!)&IBm=1-bkWuD*jX1=-am9_6xT1yhz zR|EnC5(1%y2D+J^wr8_{@DGk~gu@Y@e`6eu7`wycarZO~&2(d6glK_;FhUaA_r3PL zva%}oeR=))JU8D`A_+Q0t7xh6o~k$R``&x*Ip;q2EZ_62%~Yyhk!5;UBhOLgoFGcm za(FgeHxKv%(4)0gBlWg6!R2YGa%|$*8vWdV>z>J2hbj24UKozif#cCQy*)arGjZ{@ zo>+1nueBd<6}ME2O>- zKx!6?t*LRq3v`dGii6NEGp34Cq@T@>1;W5QIFrt(yoiRcMTBGRq&T17e}RO z%WdW+1Ay8NIW0x(E_HwaIOx_ib8Kp=Mt{d*T;FvzF7IuR-p;1D7!kl2&(+g`?;GN4 z=>x943d{nKG_*9vrga^0Pa6*VP#pf{>uqrydW{brjlrXD#x#n?9X*>+g*X+HC@q^h z@sG1WkS^m~&BD=Xf&-?W&Kw-08lY$v9UY^W&jVluH-$q&pZ5HuQbcur=z3mCrSE13 zrV|%E-`|b~f2Hs_ue%{AQ%Ql60v|2~<_6x1+VPig!!TJ$7AW44mOux~;MwQC8x_2a zx83l~R6Y1`Yrk+xTM@t!Sg2dGfCB3R3v30qdRje#cOsEwkM@5#3JQh$GT#l|3 z=QJ!}eT0=(wwpD+foD!@C+aYFa&u@(1zj013}^U5<%Ja%O-v9bGAQsLZ zWc4XF5G)_84XMEWgcROKP-7p~_QMN3%q(Di53xxtTxHb!CkHoxlHcO9dSfyhv z(hQ^wgeO{L<1fgmp_VZ7#D9zi9J@B}y7eD?{6$1vi=%i_KfNy&2M@u;$K48>-lm;5 zVc)ZZN|vvd|6p++iwr9RI0-56POR@3ITJXOn}xzv%$w7?h()}D*WN|EoC<_$yog1V zVu>{XNI^?Ra03@X0e%H>Rc&G*m0KG$2k!vb7+L`TE)9r@&_HHghZ7>G05Az689dTn!Hz(~YSQta&q1*;Z<3p{ zrhN_N7G|s~DQhx@xfyfD6qKU~EIeR7sP7-k(Gw!zb&S*hFd2{D< z^mMkxj!Uk_i!r<`%UjJzt`{-=8w(s$%Jn>iPovw}Q`isv!j~2Bxz|0)_mGdYo&mDa=76ysd zK71S(DHF*!$kmndXLsEeZFl`I(R}gk=kC8yzN`%3Km@IwJDM#>X)!2jBsW%-V#Lzg9(3otTr4 zVnGG4@%iN>jSdY(N2})O^N6k*V%_>}F@tW=z=^|g`tTbWgtK@3PQXz=b*Uq@L?_xQ zr>hmYt>vhP<1i7per}DY&RZ%uIOmPDWx*u{N(x*c6j(ryvSEymYYzVw zkd8#x=4iX}o|t8WS{pqPaqJbiIIJ?v?=!FbB=fWYP!`V|irJ$t!=c#^x8?-yRIn&^ z!C^(xV8D&L*1{~Uq#D;)-|%0xFgB29F0?-@12}RY6y>-uf(*M`n`2#dHl`5k%q^nX z!ooQZ*P*2Yoe;zvychI7P#o4P@f6ljSTZfP;*_tg9kXrx&Yp;^pwV#_DBH;4%{u+FJ0V_cZK``h7F1DP{-F)F&rT;>23p9a zvF%i!?l6jyoavn?Awx_k7uTEo(QLDfp@Yfw=3U8LE8`e?`i}%j)m2c91}dLpE8vc zC@JuJQ($oz-#7TbalH`aRPbrm*uNbW4%8SfzcUu#!Zc3&?-3kRA4e;z=e`%yFMO}q z^rz&Y7>`Oe44I@-MH>Aq8=tOQH6$Gs!mBiIzy9}b???H@6$2atK+eY~z-11dkmi^~ zk7gc=Dph?aM(~)6oqi+UgkAC*;pjAAevK!blpD?>HkrW-Z$ok^P&UR#jnX4oG|vCY zw*YJ;Xo1_4t1xf_p%^b1Py+}NQ#24(L5_$ork?!ThRCc1ZjshXqSV5=DuqPo)x6_; zY#wW?3g+WY002Rbet8soRrzRq-Uvr&1~9NVjjXc;@eL>?|Je~l26An(Er1+*4)@6o z(&*f$P`xO*H{=(zi1bjmDL$DUaJ8UMk+~S2A{0aS*4WtA0&oFj7;*t_P6eRRT0@T} z*-U^x6t7#l3CcNi8oYxd@ESQY5rY%0F*64d^<#sz*wEk!z!4_n=t!M9^m1}5ng!G)78FwfR8m6Fy)GK@XXaX9YQN#@sCf*=;$!~-%VXAFI61m5 z@#q?;>WFKRCRg)~=*sqPWPiF?N z;ppp9**=F5J?;Fb`dhRvhC(I}`IU)t`Ss!|e>use91#LDGkY+vW#lN+X{vt>Dx1>Nxf!{^}frq29#)exC`T~l>vj?9<;sCeZLFU@gw=Ej^)Kdj7{cX4W zd9G#i&wr1_2v8(5p^8OqA3$i`#nH5RPc&}4JQ{j7qkr>q%s%;-C@o_(OPQ)yG}%m}rMOL*gSrj^P>vG+az@Q;4xB8)Pj25|pXj{O0lI z*wE1td-~TzJ9;%u>fERp+=HBzJe670s55Lk^n=6G0MHpeI~|W7jCk|V={S1iNDQ7l z8Yhn*!km08+B~#(L@y;06U@p473(N)wgdEz#Khn_i0>d$&f*ZT}LE z{?=%)yo~^wbr%6>HnLVMZKe=i&Afbn%>DA;$NZ_+Q$(eua20M)^KJhoT5kR`m=OIR zYnuK;jb9PK5!_TEnlDI`Vwb=LMLp>c2Mz#>@EjB2JSbp+4_FeA!iR}aC?YVCuj0#9 z7sUxOCwoB%Z*oC-l`ouJxxD7&%XcTuyvC$TnZPUZIkEDcqZ7BDTYxEfet{S5wFEzI zMO8TF)66*^R7hBI)2 zw*^l2Zg&+~~VrH9S#?rw&1_g`W7tmf7ee zzEnf2Pej-Dx#c+F*C+LUc(xO#rsG#{)Z&3(yb{0u_2YmY#a8XnffwYp8+Sy{nvDe1 zY>D-JpPU$p!*9O?P;82gJFd!WzQ;urqElg+1+WJ_oEd1LIec>~0YOm?FWSuEJf!6S zXAbbS8j>817T(E%98Bn31wi8siC4U$q4#@`$lg!!^vx!mL+T;C0nd|FfeL+6CTTOj zTlvU|akR9ctghZx6_OWJ18sxm2rcH2Y{JbYo=OWGkr=R{VX5{}jIy=WwAz2Upjq5Lp=G*^=XuIx@ zVL%i>=soveKqpo<%Q8vpATHxZaa49!F)BD~_#A$0oDn;_@a59cg_1EG5Y5{lk`Ck) z(P*XA0EjS@f#DdG;D8kpv@8H9QbCuOQs~a08v}R|Or;4o$6W9cHM!@M8>AZe0_?mh zAixKho5Z0YcKwaA^+2tZ#JvkB?vBOm*cswd8OOtB0v4>F_Rmk@y%;f&;xB>JJSe1s zWs`bA?z(lLmq=O#Uqo770i)%Gq*v?r*OtKQ$Rye8YzBSl{1= z?$T_W9U|@%?ZIq|N0}vnW>Y*l*BqxOTVu~;CAPtR>g-107=XA42Woz%7DM>7*>`*{ zUO6})2TySiWeg7u;OGxI>|%ekclF2G^_y9M+R*pl83AJk)EvWVts85q&TgWu^V~J! zdlvt8HTHI~8R!-u;~^XI>aIQWv4Nr3)zT0hee5zj_+aUwhw{~smcPG(Bexo!x_rhD zL;)6p`CC)%MKtZW zDOxZ2bb6X!N!>~p76CMq_{W)h_B*kN%MkT*j1O7edR?@jZ_{%57gFbPCFOnG^Q;Kq zNVK`79q1y;z^>iCjCqw@L8pcQFe=u_hd6ax!wU)hf zni!2cK6!Iow&&`2_JubvIiE<7(}DF>Z{NBY8$J``BWI#_4bSY0V?=dg;@=5?1JMyY zQ4h`yF@tVSM`s_vRoUA4Ny|Ans|DcH7s5vRPv4HqqI2D5UV)wHAE2@Q>K~Z|CUoN* zcUEbV>$#Z;Z-WferR2CQ0>&06>;Y@Z(<|QRyZv3cy?H#*!>5D(PL?dYz{QLal=hKP zFO^yzN(!uQ3d|vTnj~0e?X{nf{d3GU>bXD13>#3afEzYl8SUGyjVj`% zs#a6DJNZC~HE99AH&eg*KV$xdAD}jZV?PF|jEUxpKNYQa{!h_-(e1^6{y+tPd^4{I z;N+|Z@g$ZL^9+_c3n0Q`qK2D-4$dsxi0SF6XhVlYkxg4`8z96$mkV%PX7an9jP-&n zpb>Z|9!iiR8UXkRe$td15L46oDf3aKtz?F_K41s=K)alh2S9TYkW}XIj0`4na<;VM znmLPe=J&hxh|rpCA6y(2Y*oC~QcJ5X8T?M>>Z-`$nHhtv2q&mfXLwBTf<1dDd)aJ{ z0XNU-CRn^}#x2tzIf-&F?f0?o_gkRTy)VkE`#c4X8|}f(ZOQS*V~nOCngD< zFkJ!2sIP<=5Bjzve$sq>YH}t<#*n8%M?sHVpO5@pLgwC8X59w+z2!RIy$etFB-Ssh+fPl))ht9zyIjT=;OcmcQOA8+#&!~u3^kM zO?&Q+Ho#5O)~i?3E68VEK8O9zhgUSC*hdA{CM=^2FsV31KP=6PtdJREDizb1ueD>Y zT&=XF*Iu31HNZltVI5W18df7m2qFhZdH;eO9C+V~!4^R&f`HWD$w?Y6Om+@AD}V_n z^@44Z-Nl93Q+A(va&u}o$&QrArAe74Xc7)4p$HZn0K_~NMm@~<6EoBg_aYafC4l22 zA$N;4?S!TvEW!xTTW`Xic6v|6Tx#;!Q@@BS>hd&ZEUt)+Yx|Gt!LJXW9j^3WmxbfO6;_}OOM`s5h3mvCd zUwt!v_4spmE1ry7ZoMw9zUnf{JsZQL3o$raf$>Egq$VWh(KAr^Le78K$QGp$mJ}>9hj&C1l*vqN2 z5sd5n`}5{I$uUsEiKgyx`DxovBqZDA{-gX5c$NKuH%)))fPs+z~Rdxk%;; z&znk_Arq$)?;ISQ!ZK-r&;e61hv;JtPK8R1i|O=lCYOIjPWsZx*UV(iw45)brh+Tr zA+^lqvT&`cGBV(%LGZ(bFIz7M2fxk7l$}j+$$)hft!vi1bf8m!G9;7q^vDpAwFgqs z*r`|!dwO-`#XCp)AXPw4d&7LpqE0ynf?fo1$(=!2$N%hN5u33PO^jm#kKdmr&A6#o z)f*@E!U-X5VX&ItV!dChsyE->A4iS=R4(2TU-;~;@#6=7fo{xf-1n6`<9Yzhks~MK z?Y#$L+t$r-%_lB{ZZiO(m*dl)zBw-0wJi=DJ`%4CoQ++4zxC#8(Wx1OlQR~d{nSl} zt|sEd@e|R%rY|`njh(qM!G%{G4v6#|PqZ}#`p>#TD z67D+^L_*W^nO36?PWD&n3&--z-k0LolRrt|-%PO8R(4k{D3!~RYidUIm=GKl`fi1z z{mDtPP-b@wFcw@G1T^xv1w77T0v*If#c#lm@0fR4;ac`%I(i1f^;lm}5~v7l?66A! z$HA3@!bQjPeC!U6IqMN8-H|X7V)bVPklQYMvkOs{oIJK zp14V=;XQbfutigQe~F1V+}*W1;j&yGRXh?`;p$YnH>C(mozRt}rd>D39DZ$L=txxY zr6bpe(b$2M!H+efdt#M&wS-ax6Xs zCvFhcqft;*SY`oOnA-(i)lJ)@edBHdYfOPk7%ZG?$GXVq+6|*;0a}G6mFBKBOmwv zgmkF33L;3%y*WQKSz1RC*NO6&ts68qCs)R&qU;*nmiqhgUO7bS&@^GeRb?SiucwwA z926|^E7R@*KSqox$}S$38=TC6F--l%TZXOfj$(F#AeA`!ljGq+kmZwuRuSHy6X2je z&UmzRc4JvZY3`i4=3ha!&mfR<^V@~*lP<(d2N6B};D-+bV7?Z6_FRn4O*aF(GmZju z&I}Sja~da202Y<8Cs86^zqS|JG{&KWN8{89!z)Z-nbw#(PBv%_BCqM`={R|EAZFo^ z_3?WQuFh+(y%}2>2%r4q4RL1RbezF)p9;}NGUp|UD~sex=)2@t069+k04#_;Y4L-@ zc}S}k;AGE$x{Q_SY4p(y8-m3iuOUfaaMFA)1<~zCplSL!{gRHQ14zxdbzjh8UkB)9 zyfT4LyWoe34KO5^PI0lM^oq!qF8c&!TBQ_F{Jd}P+wu1g{3L$<^M{z{ z2HxpE*_R;9EWXrm7<=l}iFoFjXX00nJ{o`e&%PBm-F!>-=L<%3#YU!i=PjW%RQvcK z=HbuA{NBeR4nGqC*r_0*a#+>cv1r|PLyT|vR6IXeiATQoi`;zw*`NP6?+b8TCr$6B zQzL$G+OEe0d{rA%&azpoqJhvMDtQg=($byBa0&NYJBW!6b zr`oZ`5s)k_u&Z>AmP3F!vOCuC(K9hQJ{psN8YlN=w8Cfn&aSYPiCPN};Ea+ffPtI@ zUzl&3>67mQ1$A`f-ss(!8JY8(;Nv-89a*AHw!YwpX_sj>0Z>lAWFS*hx7PRq8wBgH#o|Xfg)uMy5oQU}b1e}r?xNTq0uj1 zeJ1|;ul_P#eEzvKiYPeUa4Fct*+-IV&%EQ5?#GSVbI&~$!>ANYOisk-?)*G++xZO} z^M`&*si)bsK3d?gG++6}Xv5Fi+>z&F{=k#aV=;!h_rxW?{&nd1J{vJLLo6ud)it1*sw6@nNs+0RGE1Gnywo4%DnvH#FC_@}7v0{|Qs?_vQ@=XOl2@<5tlr7NDQ2i$ZKGd#mIT$?%AlupQ+ zkjvFEUeyF<&8^&9rL!BJApOOFjEafdU2Sm>GcgOZ(C~km zmj^mYvfEv#y-jeyN@2+=8~dPwoGgbw#aA=(A*+C?0ZStTQqE6e!Yy~oC(JgGivd}M zlhV-8788iP8loR=PcaA+_))*W%i?C>%8U{ib4_1&-20`^A#H-~kts{8=kY?lj4Y`9vuY^dCiaz=UWjl1r@xG6pLsevi~UBC zYB!s-{{FS;m{c*s_~`>bh9+=US&$T8_x1JS$Lu%t*#v`cy#8AJl~K>eHSJxgf&d zc1SrUlVigy*t`^XgY?u^08c9mzxpP6Mn%7jBPf$jJTi0|5!F<*clW_raixX4jDTd( z>j#IQN_}#nbrpug;f$)D!jd$InyrUx9L?{-EP&yl_KwZqe(vLi%DIq9;bJ1VDIS*G ze933ZRfgx`n)j!JKb}h;J90Ogn+l&E0xj>imI^^Vi6)B9&N!Zj5h3vLdM+Ee5NHNI zACLd?X~a;+Q-q|I)Tt9hG$-HZzt9$k4;+u5KlDh9jt|4p>B3jdL>xYHIL5Gu>O%bW z=+7UEy>IQKb9d#={q&ikcmnRu-u-V!?Z2v|jRUr6mAPl*yWjZ%A}W;k090qm_rLwS ze~+J>jd2>~WLG-P>QydPlql2_geBSiWqW<_YF5=F97ROHfnXm>0aK2tC4hL`C1Rs0HL|NxOINW&`mO3vmlNa+_DnWB<Eb)gZA)4KZ^%{@?)$f@uKO|VI ze;hWheJL(xO#Qy)TV6fY%kX&O_)+vV(VIapC&xS^sx{yP9?nzi#;xMSb> z_Fu<~&psRXed8N3xpims4)4b_{H2&C0CrkUA*QP0u~@V7Mgl7@5`WS0uEW0?qA@{E z!j|tJT(nd=jEzjh>#yuhQI<8z#Zg)K?Ag)S`^H=GgCBf9 zP7a*JLo8G3=PCfFFzOky=!9}Cx*=^k7;S}^z*+?x*2;*AZd)L zS6+KDdO7OY@GSQa5>>nh55;S7^4CG0CPybJX~g%Wg@c!uR=hFJNVm*&e*d@_)Kh(c z*MDEgN5!@ca0J(0c0*kAXa94I9(pT=kM4`nQwPz|KI>ez%eN7SS6_;OuNdDHOljDBJ&W`x!=$7o*u_NAm<8?&z*T=QjUk~_s|Ih)h zCzDuU8MJl~&dM}kexuE6LF1tXWhi_+prOZ+)bOKpZC$--OVN(k6A38wZ zN(`Jj9cLFW=h=6}?7>$Uk*rS(s3n{|81sCb^K*T4E+P3_UA*~ykHyT6Thf_e4(7ai zVg6YW!12+Y_tI1w6d?0u7Rw_^*1*g6w+Poow;jJRrYRJO=QR(f^T<}q1OQO>N6=$3-iPhVI3T?IswVJ&jh0p{To&6fG{FK(<<17Y9V1RtA4t|=?ivJb z)022QCe#6n%=!nhKK>NXMEP#m1#Bu}ay5qFusra$5264(7)OpBNl}&Jw<$ymeLZW^ zlIskE>&TG<0FZ^)ux3LHjhu}uFTOH57{oJh$~JG<3JZ5Kj+{6|=RF*)hy#b17^Ja4 z%|WjytCjcCP3Q)>Uz!XTU`DQi6fWrIk|pcXg{z}Nk4J9qAk6DLl@x4!kQ z^H|eIt7u+-_M5(~gvYq=zbDSwa7+VY$4?)N@e@RL2k;CXd^5(+9K)poYCJCH2<+yqv5eFo6% zCItMI;$$30Ma|ea{kx)J{iP9o7sVpciff(giQmwVh!@YRSXQovT&Yt0XYc%cetw%7vzjtqffkKDR{!6ke(@EtHZQrpguDkw*xap>w;*!f)`~f=F_uJAqmR=`L98aM2 z99)stUwaMUa4OB(&nf>~1>>ig*iaibZHlX|{6u{6Q=g8juDS{^wJyQRyUTZu7VJIy z^fTG+BZuJ71DKMF!+7n2JO3wt`e*UAKmK|QogD%MjaW+-5HD;-UM)m4+qrQYzHr842F_3; z&n3X?;kqwWy4N_*N|``VZVip5ZMnavSxar=+0|z%8fu+O-#K$Wk~`hKOi+Tc-^GSL zA9epzm6^c4UODfz8Ee93`b`%;iuy151mZ?~gwS`V0J4(<2V;2u%Q5)&b1{5mA6!68 zY8gujPTg40_vH2nvWs43!2*(c1-@q9HUEouS{%+@&GY`UM9c9zf0E0?oh(owYtAh# zRGOmWakW8z92>8S*m`X=^j{J+VxQI8Sg7D`7K;eun}pZzRe zdF2&0jzsa!!DRohXFm4@&@+eH!C~|rk6`-#6yWR&anG0U#X~&-EZKn8lm335{Ph#@ zgTMPP@$xG#rz%0t!AlvO6FPAJ&;c_*t7NTd<@D5(b@u`UTQPR=S$^>Szl+Cs9@iQN zdpBaM8!zr7HUI!MiAh93RBygH?)#&!#U)o=7fo}QM-?lrxsz{2gV6w06@ zn2G~<(j?ur05KjO`Ky=S5miD$WSEG0r5EUNMF7XQ-Hao$DxwlQuFAqHK#Gi2AhR>h zsTZ=;sTJASdF@0hAZsyYH&GuzE&0X{HR%BZ$~rkO%2!9kjw=AEFSCkboPZkUCq$TB*#YxWk5y;{nFO8Y)Ju!3UDx$@oiBrdq#Spu;G1%C0=jH+P z_7OS<@7KeUQ$RXp_9f~}eKk3W^nvW`zHHO77i|A#aB{^wOP2xa87#Avm}DJ%+Sf)y z2OJyxN##ya4$b03G%(!I#Aj^4(Ww%MmZM=EP4UJX zuf>7=2U4_RJDrb?BUbv`@BIy;mVx;4SMH08FUFcjRfYFSaz9>r;f44iV*IC{ev*EV z*u^=~xfAQaP8RrfxYp`;x<)8Ql50IBM?C>Lp5gcJi|>8+J9++(zxgKsuZ`zwa$P#G ze?N-4_s7FO`)Qm;9A)`Tg{)_dfRAo;BRjGBg_F-e!w`*)jwJ~3UZ3bRWAeZL@CPx< zCgfZH{LkakOD@d~yw_jAdhEO3{%b(f%bC}53kgTd61S&!Ry#lk6{38=MUnH5HLnqUuJ$xLs}si@qtE>B*uCA&+RkhCE`;cJ?A#!3{7shJ+ zV{*Tc1T5rOg@Lg zq;1g3XY6ZTp~A~hsG_ZzYFmZq02+jRT2EGdqI4B#&kz2iGQdw^9`nP8sr?y0Gh5io z&8X=RJffWf1~>V1nzb~PA&p+4mR$HfFZQ}ful07NagioFp7;>g#G@A#;{aLk?vxGZ z-70}iktOr2%%>H8UBB}0pk<09>j>%4_yckR4 zzN*g=Avgi@838qPLrkiJ;L-XveC|aD-|vTdA$YD|w#@2PFAQgi+uY6vjFz$&TTiH5 z?C1AMD(xrMo7W}hZlJoY*fSHi$z0{geQ#1~;Z8VOXKgUGqdAmvysi;5t2%p3ah+RT zmYRHJ47`@ihtn{5>{gmkotDu6GYnKrHxA$+BbCn_4dj3(@(*S3Aqv`iRK@f)v&*gVtsTMDplJ)Bz}vPf3Bj#(tEU;rxi zOa7jhz8}MosZ&izX2S_K-)L`L`|b{1?HfmA`@-oL`DP2x!1CQ0mE7Xrh+r5sv+Um2 zj?sWKCan8wO$;jcqN14*V}eLtkP0%<`8g!A5s@QgwhI=oXgaRP>ilaEchi2wBp8V^ zRoP5e#Jsi%LEne$1TxG zBkuGU`)nS}`)u0W51!1ZsHm#YgQ(f0)_+aclcCZ;tBFMd(2ZOB&Fed$+aYFMamC~; zS+UmQb^ovrXHItNtyrydXV_^svI1;tVmjRu171sJHIO+)9t4xD{rEVS_JbuO+$rQ8$qXv&-FbvWO87>D}BofV1DWk$Ic{=(VILkJ!@g5Uqp{~F&8IiM!Ly8eeP#p%OLI{ zf4zKH4E29<;atkmQ1*DalG%cw^pX3&5?>un`taU@<9lUAy`Nc8I6B{bUF^Rale8#7 z+K!$VphC{yu2a->mq1`2mVz-ucVA92Xj`B6i#I&q^o5D=dZlb3Am8C*%T;lPdYLY1 zGWo`P@53JZI*>0gG?KlQzteGH?W#=KpRUPvEuJLByuZXDd%?)}Q|EI|SHfA&mVggg zQjU?wxh}spUom0(<{1qrB!sQ*Y~tyJ&sJ<>UpeJ4KW;+ssLbdc)UlLNomxgH2l?F_ zigp#%Z%|cqz6YstcpGl6k!&N~!Qw9?r=WG$L_1+pV20wKF~>_VTcV;eR;|b5Jl;p3 z+#MUMhaQ(A;GiLz&s@d-6nj)kLPD?W&+H|;*we>aGo-r~(_Nx61B=swm7uoT>%E|w zZCr}<1&brxG3y=IdEBWF6yx=CMJ={iB~meXxK*_cnLu@Hv;~?mF;0N)rsgY_JFR%h@;o=ygZ47V57 z*_|gh5{Mo}>RW?jWe0`jzLPk%ZA%`0DZ=-{9M{or@e0YBVI(i-xNK&_5NtedZqID< zaV8+&6K3GRi&cPMV4w9dVNSmZ^0ut9==jfcb95rA_|bHRCIxaXSdZo@1E)=dwys&| zLBu5|1mTAHL*071dAoHY;>B^h?#~6-zkU~MdC@$e$pskdcu@)hpCdTLM=qPd_sMth>vrW(MHV6gNdg(zhA_Rs>&!==WyX zkuF)QXV$-W)%rh1HgQalI!2yCnIcs;Jj|D(lEW@$7OI!w1NiVqdqXC&JVWo3tZ&tF zOG@ZFT`oGDK6F$Dg3!0eAM!%gcreg4(%W$8Si!#H|;61(Vm4?<@Y8+P>Szt5)sJF8g1ueYewX)Dn46?r-`J zawZ=WC>3*ecFNcb@hxx@m9H3yMqU)X-E$v&mdw4^sngx|0_^hIE_eJl47@YF z?TS*LOfS)7cyM`yR!dJEYp6t1c&)-&VET6RHkWvGsU&9n=9GVA!-5b-xY4k`?9V-_ zlqD%n`LcM5>3^qOKe2zY2>0}XE#M%~@LxzHkK~*isIQMLD{YGG6d{9LS?EX)GuL}F z?zfb%83foTQir}zbHfB?DXs6f+6CU@%ZoMU^0uqB-1DZD(>=TA#G39h93zTLEpyzL zhE7VGb0{rV{n29Nf>MG}#Aw2Xe7PH@453Kd<^|ZqQ@n$$_lUWXehH3rqvN7d+(#yL zb;F9Oq02BJqyl;qRTXZy(lkQWmPVn->0VrO^FF7i4ME|{EUHn5U$Ic>;znMNd(*+O z!~*xC(1#=-4~_*u**6ERA2&n2=mq5U0{zONXk=l{7_djQqHThnyA?i9+&PhzVJ>fZ z6Xsd^KyhLQhxblj`9?_3(%}T<2(Cx_Fux%3L$l99Npw`&%Ebf9o4ZIHMM#F94Ywp5SEM47GIo=u z5-Z)mD3NBshT87J($KGa*PXd=cr(&tGUlJ$noOGp&uchPmYd19MUc5Zt%G54mU^!ff|iRab~r@ z#7^xs*YJ>v@x-uiwLL|~>we{o7OAt#KD+rt*5FI z{n5{vnStlI%5-GnPuraxr!W}B3CY~7FQMmXV&^!X zd#%epqU8TX5BwyP57$zr$c=ci@xwMS4Q|;Exq><;@~dSR^9O7k1PcR`(QF(2{IR6Q zkyv{#AO~=)e{qHXw@8nx+41&t;^EQ7x+aq;1cZpBd>?L|GKw*;q-o958q0t-JAzZ3 zLDga|kyIWImXmUF!d_L+;$>?Nd@{V%QDR>8;X4RsNK@q4-#KeR)rBDD4mYpn<@@n* zD^A0wXraxvB^YS4w%*uw@o+e2wV@cd5Yp}*sA`FLC0@~IImVQxwNxpbKQAv}S#`9v zU*CH()=)uq><|TXR1S`WBGO@aJIxOGN*tD~B7T;G1@LNesThhyv>JSlX%5X3CXQ9u z><9QFi>oK1*q;6nmPlP<7NdoG;3SwNMWc-EkJmA zppH}Tu`_o+_q;yk5ca?}jTDfgVsE&KMh zQ~2l`%q+d95X)K~p>W&xx!PVQK+SBIlf3O8*5`pr{ere?QF_I}fO9~dv&xS>K|zon z9s#|4Sh|sCaF-PNq>&rLqzAd(Cc1Ahd1@kC3}qn5J9X!(R1pX!xM&*jA?!;AMR=bi zExQ8-=IvCQ3KV|DL#2ZR-I=9j&tI@`=Wp&y&>~;1yV73lPgm}>?u6MhHcEFs_E^4s zLOPfhXv#LQ++D1uDKlvIe_z)aGtBnyeL^jRciz2>`?Kw~)c@x!kX2u|E=E@Jk3H1! z@U0@iGz<|KIWl-FV>QU9{J#9={I+#`?~&+CO_A>2q4ztttMP+Ko90}+uO$B8EB_rc zztShbvbCfidM=V`wDlqW0G)K`F;6xrQ8qO|dGdFeC`=#cK-gbK6Nw4$Gte4LNWU$%83H(-G0htr zW}bFeAB<`6aSp8oBZ}xJ@eUZMTMZ0`Yp@-Es)dHXL@;&7AO1yv+p3r@wq7oy)%|6{ zHKEV2bnUB%P3slo!+mP2RPfH(Z~rY5ULRVfiV!e2A>1UsRyv2R_g&ia;gVsqC2ptI z;-22*z_N>wKF6P?crvrE5foqu9HF?%N52%A%lN=3wbdB0ZzEgMyl#=KKf_g=WX~;N zo>o{|uEL&(26fIDWjZ=r${7u>*+XIG-H%Z$N)cu z6-2QqRo5W27T8kODep1VnL1HM((l6jA{ zw$X-uHA7rRe#e-(X$OpAS3aIG2rw054Ta&q-CeZ0urgKt{gXn5_Hj=ZEhOd~Mz$+5 zlKRr)s2@LE?rn6}ZBg;8;dRyh5!ZW0;RC@mZ@xY^?v%-a2!D4Wk6f~sJ>StJB+&GN zLVZpy(X#ZqekUcpOdP$cPwWGn^QK{B?9i%UWIit;g(qIJ~}51 ziQ>bL+WJQ-+TRFsi)#N+nS>^2MkoH=)uubS;~`E1TC9(`wp*&Rzz+;$BRRP^;-PZg zMBk@krx3TkatV};WYvIObsDg++J_o{ZcQv?mUqk#1aim^I=5OC+C`){tfw26tB4Vc=dV1(DN#^NWR@jeTL4M7DxxSqvm_eE7nX{ zSo&0zKvjF;39TH}QUatbU5+TzMiMSyyGKh36nXoXxo{;%d(3PU43t2_wO8u{(qnM2 zTr{qX`&TBVxT)jVg%2@NQap-N%I!i$vhRLwNbq4?Y`(h|c83G-um;7jhY2^^FA*S# zdA&DXM%Yv4{=QX=4nel)leZWD=sf){EO_$wZz%0I1nW`g(1ogSAuBraN)a_VZ{|jO zIoJi%!^5sWM0geWW1t=Ks%KZoPgI3g{{&8VwN5Qqo6P3f1Zzxbc#g&>5&Gd&;&p`n zo5Mf+%YsIJ;EX~_oJe(*oB23g@+m1((_#EPO=ff($GAc*BhR06;x~Z%q&g?4Go@RSEb<}0ONG?yPbwUg2kl(KS>&owpf1lG~rlBXqx580*R=vuT5_6bI@D8wamC@cdhMMh9b^)+P=TUfSNrm0@41NJQ|s~aEfH`bYB zjRqYmz=&^>zS$DX&0oEe_KCpdss74SahD3$PPR+bXOg);7)CBTjxM{IOu#tu{0Zm; zPAoT2KueXR4)-mbE`&Bz8G1yOBr{rwDPVNl4GjaW!)VK_mCt2f`cB6eG1W(XOxh%X z^g~{9mPTtGwMWYU(zMUysK}+kaRrxy%F=dt z9q9_k>j-0nJW}`hN`r(YSQXw4C7W+qSE^^mTgxtXeNyhycKF@JmdL9MfAYa)-Peu* z>SB<8i`9OF8_yZ!yX)G|Dc!?ZwxuI@!2o!S)=!4mAHIdy)o(i-SQv(T4szR#u? zD-OMT0tlADDzH`ZH@Y3Kes5LmeSDpdm5aeK(Xvu|E26&elsK21_3jQ6MS*!GIbKB5 z^L{K@Y_Y=HUuE0NSRW>AWSy4sf+xUz~L? zD%G(im~Q+edwspQ)ulcdeNYR`^a8gN%h<5K`5jYr3VT$YPG`06V?-=waMa}1dabRCZzMv zYlZkB-hv(%V!`aYGo`27yH(1Z8jm&ES$d>Rfq|G$V_bBM+rAX4c_3=FL z6pfOURbx*5=OO_1fVer|Emdq^UHF4inQ47y22(7XBC?ktt68^tT)J%r5mjE*-qJ`T(-(j;9f< z`<)KwpV`;wr=maQh9-soDS(}C3afQS96!xI1Y<3#fRpX4LfX0>_8zu{?+b%EKsrB( zzfVnm<7f4kAqXyzi2=uY6*)gt2@Dr+*|tB8Fe93*Mjd`Kkbyj1>4sv9_A-}J)8sP8 zvk`+#P|3o5$)GLJc3LJ?&FZ4aOypdt|9H8_GjoT*COILb)E($>Yb_9o+?uTv#eO54 z)AZUs5S?uNdj$$~-}}%SMnE3^{=!HhgIvw^N(ZYGqMd9ol$0>RuCSZW2F}vdjsF0f z+?gbh5W6{z+3WoPAkKU?!*aig76YhvjUM7;3I)_r}s829kY?wUk*3DlDPY_ha z7Qo4Bbu$G8kW}?AEzBsm|1V900KX7sM%+?UTvy1a^xP1KMC&2A+aCseJ_DaEhXQPv z@Z|rg(ib`d6^z6JiEh(9#N6N2xg~3*UgnT0_hg1+gn7B}1_$VJC9Z5=8M>kaBbfNH z)UI6dgiFx?s#mtUhK~ml#gK~&5`;@ZVU67S$KQha?=v>0mzDV2i`?k_h%PSDFd%@1 zRM?$(n+y}wZ1Kyu;6KA7y*%L0@6}T{cYRy0PS-;{haQdeceWRi^3O*b|0JT+`_yh; zv^FZDS zu@kItT!j#LEbFMUi{ICV)Ex(h^Prmz_M7mvf_`E%bb3YQ>pKwfcaO-xAzhl-lTA2l z1BmDmia5Tks!mM;K-**2f)Q@_lom6nLljQg#T-z%&9fpdwu{}>;wZE- z`#5amD1@qAbogIjckZ3XEDaa~lJ5l`NJP=TK-=Ek-0oLDkq0cDN3tQk!Xh4K`)r5P zifPNQWrmXbV04Uq|A4jlUz4qscoF`(Z83pR2jGaNWe%+hEPL*Xl@4;D!{^2YH>BP~ zEPM)GpMW6+6KU<*uK1uRXIgl=mJAjm*IiikCu*KU|7zTKU^N{dR{TaHxe`e9$qg4| zWe!&Zybl=mBqB$2egLPzLlATn_PNJr#BnoNSrq=MI_`Bd@C%_e3bb(lQ&?n0snhbZ z3}lp(ofL7DQ2U0JIJL|G&=&)yWHBQ{a>Y4umRW?_x`RMVoQEftFmB;gxt$ikIOG_R z#KT&EJde1yF-$*^q{Q()|DDaya53=H-w33x$mwv>-!B@YM<~96<%wDj*vj9$admVk8Fv>K>W4Rp z{mDS;>+PpAy+Kl0n~eQX4AIlo1fjbxd&R5QZ8DBs#?#VT1Y~@ywKk4uFhHXi#7y5 z5~mCrp7TB4S`W5A^nYT4;P0`gy%Ek4oP|)`HCj3w^T9PSKN&K4WY z>!~SLeq&#yU=EaQU0cbv2iwr)n4bN9yU{HW8E;DfmI-&cGH)9)7YHr%849`=2`0H8 z@LgmxSAD^tzL$$LGGwtb&G9=fL)mbg6gE=I%5~r9()0JHfT+`}`GV!4W{K-0E|d3x zuwQ@G>LEj1P7j>$lmha|V7R3gw#e}Q&L7R+py>Lm8rD3I(Qi&OE5R?{hWy~bNE>O- zszH#toG)o94d@qT4Sx^`4BfJBFK?P<+{;9;cNybDW{}53yH|kv`gJh+j@Km+G#y-< zz#CrzTQnj78AXxnIli1^=fy4}&J7^ww#3?M09xz=(3RdkmkX@YD^tW|Y$K4vD9M`I zT9sshOCM^=K2U1gG!&LrCU9bb(*EAqXi!I|5cIf7oI}IfS-pws6Ts>fuR-R%fOUy` zHbzaWBb#JKaTi zZrdw}j$X7qB!i4Qj|u#OyLx<&-jo(4Z%z!BXvQghdx)bhevB&Q67%t<1Nch*)_8{o z&xDfN3;oH&=Mlc>{(MRuFAQiwlQa10R{hN2Q9XrxC3}G-C^z90syDcQrtPt zE7_X!G)8K@URh0e>tWlRpJ(wRd)o_J#|p(J9OnHZw8=0?erAgg0Q$gCfVQ4jvy6t& zHt8hEwNDUYbn)Z%f{R2!d^}%D@=Nkr1d5~Qq=X?o_wqBRzfT;#NFAwGRufzoXMzYK z^_&rDMsLtyQ-Pu$U~oNmMDYi85DasY>{9E)G4fqIdvnX~k3*=hyH{_+lBe;FOe30@R;LKSrhd2(E9nEBxW)-*aNEkG)ALW|#Q!ecLmbwj2D!R`uLYdo!^Kl8?}30`?MW zH%Rr{H?3&RdM#XByqXP(pdCa=L&@KrP}-j)6=Ji2hY+8?Ov+VgxuKmFsn~;=j=^lw z*XUD%gRz#d)cr^KWXiboOboMpJ$SZLtKR=^Ra9c?#PU8+kW{+NKR@y72w7p~p@ z)hql;g<@;Z*fc3sX*-DcrN$?HLg$K|vG_x4O925e;aTmEUx}kUEX+6`zMXH7j0%Ac zh$~(fbU#yairHIsIP_v^|(G>~ea)7%}QH0+g8kdHSs*Y9#t5;!?WdO-bg@4xz?u z%*;39%=;t4eiwZJ3g{ldr=XXrSEJX%Uxu?-XXNKm`pI@$NB;|e+|M8&pq|T_xDSL@ zgv4mn{HWhXt=ogaT0cSTke*-#B2J?8QfXWasB($ZkN12IbsrlBqcm`J zZjdOmG+nbMm40N}Dxc1M&$Pe@*^g9utIWK2hhOd$B3;d6GDu@uMZaM%UT6Ts&~24B zaBqHRaA3>Fr_OMRvm{ANWm+ERMnUaW4TPJqaGLI6G!Y*RJ5Ze~SuiVO^3_!0UCcjA z2ze_oRBFA)lL_=@ax1a(#qDry z+1aC-1`O{&MZUjhNm`0>`>h)ruFeNzM`nT8#KD+5+d4T-)+Z|#lX5S$>R;%>IJc?_ zcLKopoPS9$oL!0)x96ag*nqtaf9+0pqhIe|{ob!HZcn<-bl?rT&b&N3bGzZ;_#d%S z{i9=dBA8uTIc>I=yq^qPc%_CKN+t5(~a6tXZp3n@}tTDP4>> z!E*R%%k;yplb<8Oh(x16z5X4dzb>DxJy1M8@yLOEaiqRq2d^!ai&dCtD?KZ`>7je( zTzLyIq5WG|urQg%Fu_3%aXr9xnjrn*3T9l9SvuE}eD)#2^lt3Vn{HS)N#CM#3MLJ7 z2O9>HW=mn1-Z*t|o{)O--dd^5!2NtQTQ4EVi}Pu{Yz>>vgpHA<#uLB?;TGd6+dNgC z_z6fle`pe8QSF>6P?NhjZ0@quU}*7^WkUT>B&bK0YLvwjbjePicdGKZzP=Epgq=`y zxk2}iWN(L+@2foAuh?)l&Gb4u;=ehJ-M+C=&b4$-LVbGajQKVVt>?QkL@j>|ac<^O>hZ-d8A=; zDRQ3r1K^V@Hi^2WW^dW3)X|lwa?Y&6$9+>z67DyT;ua(8-c==y^o*vlvb#9hE+i(u zBvf7w(Qa@MSK6<5B91nF64t1`r!^12V1t*2 zJhQ#k^3u>Jei}3hswtQcF@nz&{!x&#(k6j)*cIBMoP2i7q*6!PUKTnN zkpSb00WFnoS^yK!NX7Qser2AY|3@rTg-o#u%lVNmw)d3PhWE`E6cki?J}Yb_Q*A>R z>-o54M#f?$ewC)#o+LjzF^NDzW6aI43;w5BwGlu*6lkvjtc=3Kl*>J3_3d*kKLYCu zjK*%xP6Ay_GAu>hb7EtS%APh5>xPTgkp4A(Lnu!6i~3M%9^=A{8r(A8M+O@PMHM2Z zHtz4z)720zXXGaVO~okSXZ0WYIa@6NK1vR+VWC^vujrzMgn@pgqr7bTT{01i$2u}^po61QvRl=5(X+KtQg8I^QjX|?v0jYR zYoK%Sl(TL9!P}7DFhEh{$-5duG*ZD<2Nyr5Ct+T_51VLFyP`@m2ld4ox}t%CO4UY+ z6mvld*vnG)j@V1A5S1qu$xeDb+K(xMXu?C-{4K&~UTxq`#0PbACpW+m{S%#oL380! z1vXa!)D$rF_%QmGzmZrB*$;~Q^sI(Ykd#L0E^+UrlE4aeUQReGiwNW-m>@6?23M=r zCV&c!14060adOEU`n(^8ZH0md>IFoRz(c!Fm?n~D)ii`V3Nwjlk9=+^)! z)xfCCocu$%%9FOJ+1+NaQadfAY9isl=UI}19ZHvtlbr9IlNL}zWXjV|Sp6hr-iZw` z$3}0+mEcRvFJX}8zCpV9Ou;9=26vxX;)n3_tZ9|Ymxu8Z?NA2_bQ5hMslDRxYGT`R zy-(ory2SsqtGqqr{DHJwGHv-Fq7&Wh%yqV#Yud=xQms7;f#F4qIK3At-=nL+T)pg2 zpoM)?!KvvgYdIQ#J3R~Z5;_`DLp%rLjPM&O7h3y0*h zGAtzZYzPeu8DM^~7g&X+BX9O9B83>ByzI_haCUiYWnIj`8pL-i{JTZUnP3vL1Wrym zs!mW~$#Hw)W9k)lHMYa&=863H{cMl2eVTjX-mJz_xuyq2`oKI!U3OPb{yN{xQ*qJJ*>_-#V=+C zwU)`q&l}7Z1r?V>`|pTW=aPpzra;VR6G#2`NY8OD0v<+dH?07Nhfu%dnOznmQ3jmK znXsP_YqP|&?;r)t0CZqZOQX@s($~7l)3-IvK|e^497o8zB>FjHZ#Q9?F|_aWwNc&2OVmx zgl*~DPLaR$`-!pjCp-Htv|qE&XjzGgoK_b^03<-#pLBJE2*4d-aSZzbhbYa34heKV z?r#@7dN)Q8<#|@l`pg0cf&8KzEC|CdY6W?Tg{|&-@f2JX1Sf8GLmVo>()@uKxj-Jw z768kPGVpfh)HX;)KWCdR0RFL^W|h&!^(8w^Ac!HdZq88cvsU7kM5#JA@y?H~)LnrR zOVwkoF|%(pKlF9Je8Z$Wu_>$glhjGK-=r^sQMi>Iq9GmJ%kv4<5?OQ-M-;YM0hDsI zBM|_ay7GNqyFZYb9GcebYq*O3j!WB6c2K4->+6%MeQt&$lW(C;fvPEnWO=z1aDj}FY`b{?Q^ej~Wom_~$wH=LR1BfKNThw%x zUs#<)%Df&52dD8HF-%>gNAP{4Ef#R+R?0&%q@|E!;z(e}$4em>M>QZnxBi@_=afpt z@W}AwZiT_4OBNk>0%bj%;NAnI+U=k7fXZNG{i^|1Qu+6lBc380bTa%CTf{q0Q1f(C zQzl#HDAg!!-22!s;P^giFcu&7*tfy2;WkEMqr-PtluTjtACBcn*MEq>6LDHD87{qn zB*PR7687WQ!!sAb!I&!F2%GnOevS_>^Q)GNocPL|hnbW<&78a9U*hz(K>m3bm1VEM zQq9(7z=~fe&GvbLZ!aG=bfs-a>xfT`#+z>@!D9thlVbMZ@Z+1rra-hE|MA?gN`Ery z)OYE$3;Usq2UcMi!CM~XR$f_^+vj9m_FqDNJd7$A^5o<5NtH~DDj)r@`?09|0@cx0 zE8<)7vUZL_N{(QTqPvq;S3j*jFuDs_9MveB_>T?$R`NQE$k{@E?vEmay&+qm5Yn}f-`9EF5 z1V?P)lH3Bl)W{U)g3E9bN2OZMAIH^xHTbn*Z}$KEbO$SZq{U?-rvdir|L@v|?|3MD zGFYT$F8p=pB<24dfd8@jK^16%lnAiI?RvSMmj7s~|GsT^uv`Gvra`GglTX!uZ~IS4 zO$4I~RHo+3^{5Q{|5fV4FcYjC#5u2X>ALklZT-Inn2N;^9ks4T*~vTpQ}_QGF(}1h z08RuyS&Sb1_w)Zx$;Zl=wp+KKz02q3|NC;xQ`>xphj|;WP;0zP0(=~@Qc99lpN#|m EA1lRg!2kdN diff --git a/docs/img/structured-streaming-example-model.png b/docs/img/structured-streaming-example-model.png index af98765fd83391b444e4f00e9c8c76ca743206f3..d63498fdec4dbd8be8785f416e1b5fa497c6c031 100644 GIT binary patch literal 79409 zcmeFZcRbba{|9`e2*++5n~;*dH&GIzM6zXOMfN%znU$#ep9Ta^cLN4;TvtfXL1rxa#f+zq*%!V-UGcdkBsyfqk=sDe z<)*=yi(N@wjBX~nkAexKX`n!lkxsP72;uv>-v z7G_Gkyll$Aal85cZN}0s$ibma{KAgP!ZG5-eA6S@UiSzlyuY0{zLW7^U(zi`XG2=+ zOc}iF5vFDYL7kKP>L`4?Fv+_pm7)5-Z6P|tEG>=(R|&-T1eyfBO!N_^D>QWBeP8}I zV3u5=$j}UFEk+4X_1K|c$;ZMmu*TTLbAKCLY$M};6X<@V-&A_DoOkuzuZ{=*t|WkM zoWk@KGQ;NuorUy}htRFRo!XEN3Zc9%4BJi#T!Ll6*Dw72;wMEH#f(wYsSnNHH;`wq zEkZu4pm>FD{(Th^^gIYAOP4$X`}+nma;1yHG`F#3^6wjF<#ZYo=TD62+wqtGzBAJn zQV4zP%+On=i)t6*{`?e@!NOvdg0{Lrg3D_E?s?gNiS$c_OCI_%3fWzC z@|AH)r^LQRyXwt<*v%YCjLy#}h(l5lV2%GB{r~%aD4B~_xu9L6>N~c|Fy`Xd6Jkka zW1Rw=#H$xXEIL0XwmTyKPTJ5&`?SaAssEBd?o_c+y;RtklQ=Iqth)e`?Gzcf+GpMlxoYImUZcR`WV!v6_n!H!^KIw~>0+l;L-O$jCj$Dd zr(qr9vz3K$6g07`6u0H^Ds6)a922+4y^{*I$9Fd4x(Bi2vm)XxQsvBYLb<2St&*A~MIa+Ca* z<%Be;7~1DO(Mgad6qUwjC|_cSKwx#i2mP=GTgrvX!{@E<-G%?Q-tZMKynXU<)o&Vl zAu;yxGoqWU`;0Ij40^-5bv%sp``;6V*zP}Vj4|K3Y%HdLN5rRFX026WQ0+RH0^b{H z9R;L&bUJnrXPN9_sO)>34nL?|NEYNp`$~?)?$#UZxG)?W3>mq(NIkiCEJYJLS2^pB zEs;Xj9pCkhos$~Ds?e^y#-VBjb|_7jlCFQ)n&1(S9D%j(0Q&yi@dUH?bP%8~Ieq)blq>jk^# zwa6em-oD8eJKC}|t-I^n3RSKeHy@ws$fE2$qVXXR=4+Gq>uGC#X_oAl2JsIvc9>6o zDk+?kymXPHH^c%#S@h{{c@zKfdQOt0TV40Xm(=!5bYZ0&1Qf-$qn28(8@y2-vHn&- z579q3kqW?k`haBmGc1*O4MCqjSxv{PM1#9B`RMvv#TyjGkS47Dxe3PhN}X5?S}r50 zY~V>7UXY6<%h`BVE!Nwz{enH3)_Zp~^9ObklW#{^wH(*{z-tIx@9BRRGrzv+!5%%z z7o!}ltJ&n^yVeC}hCrScVn>L>9Cn)(%y8kdhq_9Zu6N4u$q^NJ z4}siKB^dSZqK|+m^fc^WkH;!Ev-T=!4yT_crH>UFr$Q7)$1BAp5EZ0R6ebiV5J=Se zaU6v|1)wgZP1yo3pd{dfG(Iblg?KZZdfm3Wt<9=D0|SLxA{revyVBR-H7t=e`#>)vNnDo zQEgCbX}lQ6ci^J8@H^{bvr9z4*_tD~*LwnB0XXG=u%E;Wf_@njB-~*>T6`XGep6R~ zlcwnk!#Vhi@&YGWl}b?|7%aVCjz-^_S;s{2;bMpj2EPQ>2TzT``XOvSUK-B zIPa9kv%TeWvx66#?f!?(7A55VPS+9VVcmYN(i@Y4uO9yTleEm#K$AILtv>`ebF~q>R|T$Rb9Fuqx@pJ zRQz|;pUL>X=?MI&qgOMg%yKf|UR^CC4DyK6fys^k7he*oSX#<6rks&gh%o zF4+)KOShAXy-Ot*Kuf%6N~kye{XYC~NBn!dPK3hLP!qA32^G~HDq}X2%3HTOD@Vog z{U6l&1>mGaOqoE}tRWDHNW?G9!iS6e*F#qdlrPuE*Pn?3go2yOP%Jj0L^Ndxf2}-_ zl9+z2H?8S{Azo1De4Ds&1LE=SYBAebXsG)->Y22gK|!3;)720Bto|dgKLSP1j?XRd zOCCs*(#C&eB2L)pn(1`}7mIn%rC$8?@L;fRgh@g%M1hAcnY}^!B}eCUFo*NgGTo7NTHbJd8s7Vbq=l zoDcTgS$x#2j$nCqpQo}g+-oE`lCt1Ib;jq{ZiON+`~K_)+E#?Eg(QN#xkmLerF@8e zvZxA?<_PnYwVI-9XY!l*``<qn~iqHz)fYU9JdMq8#odB6hBMne30+>XO&FK{+l( z&|Eo0zbGLLn!d<4ySdV_CFLj{DJVe2kKZJ7)qre$%{+Vw^YNLzTi25imY&W2SpQD*ycP71ad*333RBD*rY>wz z5=Kj@$LV>KZQ1j!8^Y_=(=`4P;6x&r*L#q}`~)?lw=atwYq|`Y-yPtdQ^}XV(DN`{ zBtJujZs3-2$Y%Ggo_NeH*TJlO#}T?gtDISF%n&l)l`G-*wmT8`ivoQ`1GioC{6`bC z^QJ0Jb3L*5l3eC#EVxgmS>;2WmpJsUT+zT@smm|aq5vSEk<4{E`}D^R)`kcYDwBX} z)%PnNKfl`P9n1&?*F&0okbA3Rm{1fDMf=|jzsA{`+b$Wm^&q7A5f$W7%~H+#=vaHr zF1;V|_|#fIqBT+ak2I6?a#LxK=HKcNV6o12c4P>B_sNlb&n$uES572(1-!iJPfnzG zW-6FueWz-qjV7fy7w&W9J>FjE$TgRRU)k{%;fYVpe{#b-IsE8WxUWtmN!LdDw+G*k zgJ4v;Z3i93DrIl72D7djJ@;Yq+na;!dva{+ujd-xo-imak=PFH|8R04mcO@eKRF(U zb4GxwMEuFm9mA_;f0GQX_sg^k-OBJS-kZlJ*Hg?)6S#eR*1TMKdfM=9`kib+%!BpE z3n@-9qg!LmT)rGi!7@0;mUVVp(_^BXQm87Lv+l(&$?saKIiXKm}1S@MId&Ww~=I9s6PQJOz>yV~O znlr7^EYuwFFwNMnU#s;Ac|k_PyqSKz^I=ZDVsygblnRHXKB~@~+DWd-YcFtI`xW_m zt53Nl_LHSa+qSa%FSyKBBGtoL8S&U23I&Rww&hY?9|wCci?;6` ziqMIyTf-Dh7YbEpJnCO|Cl}dd~f?*aBpN-#*3YUu&0gJ8g z6*$k&x-Rt!avHX^l&I1O3e;E_f5Z#&LDuX!T}X(eypM>?rGYheKiF$#Zh%CYM6hME z1VKfVj476NN9&NXUps=}OFdFQhu60|IHcf#u*)6esh-ZZ)7w5J zQZZbnENBnOVqWhPaqGjY_@(sqcYHOpWS%L9_~q`IAG-y_o9WvPxwVXFz8S!@9L)DP z$3Oa@?J-&_TYarLf*exQJ0JN;bT1|@TmUQ3Ti7mDsowqN{M>u;i2_55OM3=nnL{@FoLmlMtycC1n5Yvf~0IUtgyTNJE zB<4#5DvA)c#p`7PdC9=`S|?<)X&BR~L++*hpkvRpT|;%ULp=~c89{1JiszfrN^k3@ zbYAK-{Zr6Kd&P0n7$0s%Wb6`k##oGZv08WZV}=8pu$~!n*B?~}M${(Eo^X#>FVhYL zb2Q4xwm8+RF_zi^uo9t}qt{JMWc+4(EmW;rkL&VhceTyqD~G4^uKUQsSo}gdC(qdS zo55tb%(4;v=JfG7fcI?p3t@4PxIOEQat_6hX$Wc7CQAfM!ke)khAtYK$n&rfBB5_$ z<+1P!|EOd(3zncl9RtiBC*_opy#Ix%a6Zuu9-pqc<-u-s^IOgCGg%_}dqs2jj5D{I zBe3lPm;Kg6eVP;It0uc2bedb`13*4Z++^{WDG|TPLXbWbO9WJ7g!OvQeI}DOTO#p6 zIi@EYmrdG^&3tqH(UupykAggB)?y{4B3~Yi{j@Kj`i|`UQkY>-(mI$b!^IKpdfaRj zGOc51{bp2jolBPpa426j!Ly^GiqUzU_Br+XL&9V8pukQ!2Lc#SzA+yJW<@G_qG)K+ z67{%5SIHLicTv)bctPhC9Y=sz{+3pvDy5kDspukasQd#tamz;t^@T@oOG5C{^GrfM zPgmD1K&7c_l}(z})SAoE*gZv$ zUUWCcQ&&*{&b1wuX1X+01(0%n7zycoeP#!N`XSvXXrd(>^JL zW@D!EB+AK-;DtvXiIv_ToJt)=R(5`q^axwNA4av;(ZpWmwteCJ=RD#RcIGJ7sV^8t zMLR3C#8zgqx9}_wCcwJ+;mBjKZz+Yq`>0P;KcmHa!#W1zGg7atyf<^h^XGxZ@MqJJ zOKv+>@}Y}+?~Pf4bVyG6pDCK^l7Kxzi7waRLv5}rwm#=>%0!cT*cu7)tw5SKxNZ*q ztjyFb#1D1vYWUP+KF)!UT9m!cku+Rtyp_hVoxSYKF%lcTYRIQkmwkhitnAay*QbH7 zlB37|5`zki7bD@^>W=YtxA^Wh~r|mPng|FPC=eg zsi9uWkj-O`}5 zgh24kC7GTDyOKM1+R?>7s7$=Ke8`6KqrTa*boMZYE-N+YoQuLRkFywq8G4 zfA>8p1-e_t6?9%qB^w^~@irAsfX_kXpC%#)GwjqBllPl?O7`MjT!SZG`x<(wY{gV&O(&R+&Fic*EoOHzlo6DY741mBld` z*P2sb^|U>$xV@_LH0l}HKu+#YPdiTgF?v z|BZ_!XrTDYtbh72c$s`a#PTJ<9eIgI_kcBTqTWb0Bs|&9`u3UMy{N?dh zftU5fZ?|xgk(}j3dCPo#ai@5of7n$tCUrNj3)_v$$wN5kMM%(Ki#!38CFs9=xK{U?>EJ#Yp^y)MC}%uKUDaJe{d%| zfJ%VXq@jR8;KI3*;sAZ#4&}jI$I`0BS595(RzCz(dEMNvsf4f$W*G{v;ft}GXAXrX?i2H84s~OH&>KhcDXek$1t%8Q!=>2F zBGp5af$1Gbpu06IGa`i|Qx_(xxO4d*8c~>3urz;CiBU@w_W3kg@mzjbQ1tX8zhYN1 z-*BSv(z@6|!)=^l%><38XIddAJgT1O*xeRY80Zfl67~N8^aHiY!?f-n?s^XrV%|mq zWByk*MB&?8df?UTf5kLk&dTDm^}2cFjv*qvrOy?0dZvqgOAftJ~Ml$^x}y|>g^L_ zLB3bqESIej)0X7W%3_@@vB}t?R+&kpK{fT}c(dfN4?{_!W(>qs6p!F;mSaey+U|)B z!yxs8H6r&;ui6xC;ER0tXBju-Lz&r`=p*9ZHNGN?GdSq@@F)PL`B6~dimj&Ce6)qo zc?hIJq#>fiv$yuKnE#OkA8f^*DDU}s*x{z=B6i4kI`Asb04k}k#$)@4LBd(=fvgY9 zfv&PI+%rclsk?+%_Vj4ma*;E89s5G)+`E--vduD%pyPXNUM3GOu>_5w(+-)UDVs3jzrMISogklo*H?Z)4Qb)>zXl#cfHw+8ztlbWwETtW+uO2*MUKzl>UEsXsx5LZ2bgq&v zYijd;@o4r=aruss3z?3IPy5;<9s3AmY$LvS2cB!(Yu>2xema%&iIu*c&I&PY^TZ#C zTAf4E3-TOwHvqfQwL+udNs*L525Y>Ea-al$Q{7S5RsYh?2IhuTKs(J_BMvoMr3CXO zkIJ=pGIXW%d-N5pPvdDm!(ZTZhkWSA!3XR};hx}sq*9|wWJ!m{9L54d5y*>tY?m&& zsY7*;_ZgdP2!bfvsWfEd5i)+=!cP$_MjebahcC&#o3;ejcTm?e##j-we4Zc~KBLo( zv;8h9gxVC>fGFP|)@6|8Nt49$YXF}t>VCE~e_|>9w{dC(9Tnl)v9ESx+jF0H>M!if zp{|YNH%XrV$cq=40%@vC)H`>{2rsCY&Kni_1VPDcOJt=EU1!gn6*!Y*9J;gn0;RF~ zL%^z(1Nww;s9W_1uv+&GgcnGJx6$PO-H0gD*dBTtA6dV!XwnuUAXPYVPugpLy3dF+ zSrD1Vygz4eUAkzg4lVA$x)crV+6!5VFGge*)U#*Rh;HtY@fmt=eplHsJ)-G?umT$$ z7wx+MYjrKnyadD-92m`#VAy)iRw1XuVP5{QR&4$cb9>F4q$r~az3TkY&$i0hGx;BG zf+V z?3pX483AQl(!yuc1F7~&1H6D%yYpV&C5^G4>=uXoeCVfudI;rJ>qk~S*Uz1}* z{JD#$c(vq|<0AoOyLhE6w%1bR~}axys$FP^!`gy|t9VFm<8Idj`NR6-@n9 zED3yY&$$B*tsXUQ0VLLI$?Pl`#j%cg7%XyBKQZf7y-`y-n1k}+HJ1m!Y1Fh#=p+nIR+`fY1JNd8$Exnf4>EV$|1QyR(am&aC0RwNTR4$DvnHj( zQ0<2{qA?kE>2hnlL5N>~CemfX{T;Q*owDn`LsNOKpv_#FDsM>_d& z`IDO06#mMp+?8XS*dsPh^B-jV%G3FtpeCB#z%b}QVY23`2 z8UoyUD8KjWYnBex%~t-q-}B6jRZj$^*L7>Z46K4IhPsEeEb5*81}#T(2*@|(Rft!A zn_8v2`rx65V~XvX!ZRbcrSDhsZ<-zg`HAD8EePLT31(`k%K?G9>-~1sCr6F?$o5L! zyJ9SgSqUm;@phC}#}Y1Idgee(Cbwek^`?zW)}DQqMplflxjH}*aDdvxG0hv3+VU|b z%Go+(ujBnF-#NM95ZAa3wetG*Bcf4iQ#*TtD;jw|brI6QuQt&rA#9GaN;xl;4dMZL zJ%pIU9(V4JbLYHA6n2 z1+OlDd!zYN&t|=gVAJ8`ypLq(xc^MlTi%C*oH0-A7;T<+&$q!HdCrChnD=3{+W70} z=M8T~RiJ{twP#Plv3p$$P`+8mTrZP5CH$^pes{S*g2A0gtndrz%HFxdUG@V9)9YinlF%%0F6WgBpBLN2=}f{Sdl07Dx8A2)9+$rW??!GQOW2Da zH%+cBZ+}g*ciE&d(Rb0I-2LfQ3=CBqFBO-Epk?s2%Fr;J4r5F398*zx4buB_ksUvW zshb#Q!-raiH;0X-dMhn3`gKgQo%V>ACcM64kN?O4yD4xfX6r`h8L2>$AkJ!vWJFK? z6+zs)Y5TaFL}D`T8BQ@{xJ1z!2xTX^?ddExj20$1o{HNPR)#*#UAS@kbz%mWmbsCe zcZZbI{Q}N8<@gS@b{F2S@Bw`_>xHd6npQg_f%ZgUjf@{%G76!=%P0|hH=bvI#3=xl zSK^2HhPUDiAUP_A?3NlZ(paVg17KS&Uog!fYz-l(D2+|s$sd!{O(6Njp7pPB!Res@ zrTOCfkISa@9a*KR#p9Jx`Let-efYEOn<*?XA`g2w%U=Er;3{7=5>%LLhnM7gsOZ%9 zc>KJukPlo@@+ErqPwurUw_FbMaz0}1Gz*^UUgmPBI@m9>jxb8O%k4ygtgX$=nXfCx8i~`GLPu%O;DEprT%>wBHz4?{a z!cG6K)aI#|=&PaHvOXtu;v}peTBGAQ>Q!mXd7S2I>j`nQQ`R4v@0n@MrhfzYCt_*D zHJrQGd8;b|=fRK?@vYFn0^gh<+8(xp^@2*xn zLekhej2%#cd~&m3b1I0Q@VR4IhzTyJUjT{5sl_#1G&g(80%m zAw+&iNKAAsZZIO3hUWEfaTCy>ULaRXi$nfAI+G8akMtkFB>RN@Nnc1Bo!Omd4u?NT z|N0YqoW6A%_+nKFBLY(Jf|yS&>RoLuKJ-9WKGea2e>k>@i0gOR!d)SYV6z4N)rA;} zCNQ=~oHYLpfZ@2$nQ@8BXoE#QUGx3ycAHx9HsAr>oWprqk0e0>a8@xUQVOqt!S))e z2UI}BuitG*kIrv6MS%N&f&LjJ_i0z%aOxfxSNjGbpq?;r;?pcGXYGM~dR4T`(LrJO zcP$42@a`x9o&Gt61R%0AvtQS>93Ci}H}bf(0N=M5mmOEi{u3m6gbNK=3t+$DL^%6s z^>ySToV}#F(LL4a-EQSOtl2NxWvxnuz3u-tJxNpN03WCwJBdd?kmnyAFab z{oe-s-m$bu7T2c}C%&xWbnS}HcML260S|S6NO?uBg+}p96`Yql^2$BkFbA-5?c#i5 zBZs=4F$=X5<;#`1pRfv{=A=(HCBo%b_4v4VXka9VC8wYy%t3mDT0xr!V00bXSB@Y$ z{Lnjgd~0H{u`d1di~Tu0HzmY=SP6TP#Sxi`#q6D<&^wgvn@MLdu|0B1%h#=}5qDrY zHH6R2Etjag`S?sOutfBL^a9)Yh})zzN4Zb3UR8Zr{JuQyM4ZRZD!YNz$Di_eVBmh@ zu^bj(o?pG5o7c_d5Y$&MH=uK&N2uVXzn8|EcM~yxHcggCbk2QTkUQxt@@LgTWjJSr zIXMa$lMf9BmKn;xD_Zv$n;42FTOUo;m6;rX(ta&8OneDXNjB*G^bN~Tc2cTl;A8bj z>-V9F_0rLj&WrUeZ^>P=(%^9>6IDUO-&%};J|!+KBP=Vvn<=A`r+8<9;5VUuPSUx_ zJ)hg%$fSQwvd6RB7U%(>R=MnJv`NM{TM}@;=8|6pS=1fj<_J7On$Y$A&i>`aev zfcH>H0SpYs?YV zmrKBR5bvge1qpxRq4TPiZV16tddik#R;xV6K*Hkcd6?Wcg8S*TCT*bsmb%i@+h3_q zU+~AqZd@kT5HcRT+ipsN2lIQchC7>x(1S z!gnR+keS5No*zOtrVY3my~O%C#L9~f43ZLW#Y&WTW%SIDB50f@b;nZR<6*bX z&wL97#^-m5w!iyG^i=i{{6Oxk+D?b;5!(Lb^44Oxa)?`ifwkk&aqWxO;tC3rbV+pn z;S9Te`G#d0*0w7TLMW4be)BdDmwpQWefj~IOPAY!c^n_^3dV?(;6p2afq1mTxwlWd z9YgjQVI$%wzV21?b((*2e@twzbwgM>WCoBgZ5m8Lpuw@;(`|}!!)K;7l#ftkr33pF zDG=-FNcKX^@?qP_6}lK+r92={at(DyNiV-bo%PBzPgn9ZbC+6Z(Uw(le-fD2ngrC1&FmaK8-XSYV(GMg7Kbpg(0`g4IQrNVN%vAqlB1h95bDlP_?&wOVFU)t3PEv>cQNP z4+4aWnaGNjC{?swP9HE9!0;e-ZLJ=+`-{O)T~~dG@NRTy2%BT*YB?Qm;5(P-GVW+@IRCdmxrgFua@Ug#w0zOpJl!0|aGv z<#MqX$f^IEp)CWEry?0lqP$Mb8;3P&7JT~2Gz6CdOUf_rGB3*%j2^)C9&=8!UX%73 zD=Q~}a*&88pxOb%>*IxL)Gybs(Qd2^av5evugw&?UIs%u1RZn_m#d>HEva_D<@d5Iln01QhjWQmNs8 zSc_DXUDXuc>E!^Hqgv#JfNZ)M)jO7Np4fe zm+ZX9z?TP*X|&^BvEol1H&R^Y}qTJ395-sDcsV&Ve?{~ zyYvPyB$hEgB_rkD9dxJqALqUvkD+pY3lX=pgjh~3$#9+OCLQi`0`Flibn_?BTbsQ)m#->tm#T$TuoEeNUZs2J^X>^ zncV*uh1-6R@j+z;1wg^-N3&$V$Q8Vx-CcK(@ORc*a5}m%3}9=D`E`K;a5C^A8aw}T zrtMmA*@ws;04bA_ZfpAbJ#1nTTpdE}PKfCRee&h1M#%<;?uO@+H3#julGFd8SXR!$ zZ?D;*lb&zKhpgwjAe+Ms6zaQ>U%z(cInm9DwaPz;Npx1;$UHKcl0#$HH8n%>Lesj# z5`+nhg;y{Of#xg8Za@0)!!MteEj0t^%SSjwMGmey^9lMr&XPF7ts=wp3ro@UWMRoT}X&q2LX zYZZdYa+b`qd_wOjftqaAm}hU1;?iDwQd1k$#ye;fZ#XE?s?7tnXV55H6K0mI0-`_l z(N5|Hr%R)0Xi?6)*72VQ4=a6|&kC?3+ki6r)RiOYvh4f>ySG}e76ihMP`+1z+*NfL zGQPysMQMD6a<8?+T{NzC9AKcEb64+a9%CW;G8n6hf8?lAjsL|gs8+{45$6(9PTbb~ zSj>J+i1)tep^uo^VLsmW{B>envlp#2n|yUTu^>6mQORhHv71Vx52;Z@3*6PSXQL4X zg@O#l%zh0hS?56lt9rmiiWX0?#zMc@gy<%NVPbDA$D$4=d(BuR1XrVtdk_~7K$O&M zJ*Xzk4@1u&>B=<-av{a#h8(bmz{)f}lS-`1ed*min?jCck^APQFndZ657hizGbpTI`VC zk@HnwFlX@I>5DbmuH!ONS#tQLb!>0nIyX7wB3pQnr-Y?!=Hy_G zzwoWN*^^g3HHmdR9oyh`!NHv80#}fOrSQIl_Pq*|WN;H?_VcVbh>J|0hsjtN&$@hN z{kts(5d8u2zd?4vJMes%4#^1d1+xa(M5ky|1ya769_}b77lJE|->ww9f)M7PM=j0v zHv?Bgh!ERFH(aIobaH`djDNiYdu*}O+ZHIu8D*S1D;E2wJ;EQFC>bm8rUhL0Am+QK zBhUF1F%5sQsW1P-1384jl`{W%5DH}2w|59(jWH3#mIOiI4us+jFXKB-1d3uHqrZy! z+YXXwMloTNE%#!bh87f(+Xya)K5PJ`*_6aj@pt2RBo)rtL`*^PaO$kT#~fSfD6W72 zC}eu&&eMfu_>0EwL1Ljq6{R`$hJPl8G}O}zh}j!O_<5~&bZ3RJK`y$EICO4&I(HuH zNG2OA*!`W7FTnXlcgL2CYxf}9F~(`TZpC}ep=Yq#Kt~OCRrG(K|7QwBd~;lsxTnn% z55l{D9yb3Dm$IK_%CUVe)DO;8XwVouC_2%9j0zMY`bDd~kZOvf?Yh2NMa$% zv$>SCIhHsVa=(I06u(5eTo`@|KSX^GLp#wWt2>>Vx?AYXW#l0;`&ChR2GlMSNVHI^ zo#$@mXJl^Suzf%>+61osEK(M5nf&R2>D5ao%IsVVCew|+AM+|6;EcdJTUXttcx{WHY zxUCHfwNW-YT#z+oE9%_GDXHa z)b5-W53^PDK$#xO)<4SvA{zI?eXlZ?YPRBb6&afI_In(Ym$*}QnNC+U4Vl7krC9qY zpazBb5p1lD0mPrL^yKz$Ms;{DKnXXw@3g!-YOuxxS-Q(~9Yl*6ey@U)9#r z^7d11PTJ(unE+%?e?=1G&)s-rsYmpi9CZw6Uf5mwr9J^++Rw{<@6 zTRf0Gg2>!TMFu`O{m@{Nvsx6h@8m&~G?KHT(noo0qLaZhr)XLVUSu z74qngHoL)rKmFrLbd{h>nTQnKD{^5ey5Ot3PN(`+v`fpyHknAWD$YCmaKo&EJml>v z0$6g_50#A>;MGDRAk#sAmY}2umM~EyoBsBK_BI$CcjQESGxfswa!O=5TJ*yFM3U-m^pT@{-lxB~qg)6v84$ zXlq?o1*J-Fs+GQojk{95JZkMOA1GOf9-~b*o-r^8WC(YAhYTDtt)u!T9wQ?4tlhUt zcbRjsq6p!_LzT;g(h7;ifgnPx{l(vrDZ?pD^LOdB_r=w|5fSlQpqF!PVv+GaVbOAC zZ>QtpUg{ziWVBEg254skx?dGz&J3tQ(YwBN6X`oCtZ&k9oc)O^Kclu>NaQ<;88_B_ z)!_9)Qw%Jb=29B8Qxxi;YLEVy&*IH%>dYV{&~wP#ocgEVjW>L_ohtcLWSCtH#}vBQ zR#Su(*)ohZ_N>ODC2ZYA$jQII8=8?;4*{tLZOo)svCi)DV1_iVJExyZk2Gc@nZAD1 z;(T%>&eEeW32+Zx#X}Fru+V!2x~x?DyzuR=sx=9jQXy*+q@Ga#$pGbAT4~QOhx;?( zT_F8RXea;Z(%nZj9D>xil`uD7lx(ix4yBlZqKDOw$9V+#$d-Okw_r-;QQ;sb_Z>lW z7uWuC-lGgf^BRs6=f3-2h^BD>^(&tu!y3lYa&zdj>Cw1M&&EpAO9dD^MyIs{Pf6M{ zbv9SpxUWdBT>s0(UxuuTp*N8J?Q2!CZi8B}SMc;P$=<`8$%&F(2{Tsy8#pFiyw4c~ zJ5}6gLX_3eBL=pA-eY2T)^J{l6x@I8UF)e*{CN$5o_tR}3U?p=^vt8bM!ba1gu)QF z3A;>_kk9=aorM*Aof5IJ_z|8va_uZ$4VgyuJ|5308)}_HHb1d8!_?kFkY4I1J!?(4 z#C7a^7bOrl_p>wc{tZiuDjItZBC|`BNx@(uwS193sP}9}%oyB#wkso%>nAKR(<}->i7{XfaPI!h5&MQk(FVQO za^#M;i1!}fdzo5^w2Ddg8OD;~ITQr9P`lI8u1t(V&wFXCCh2hp7?fW_8lv8?1RbT- zcWl@WbRA;k;`{O0Pe<{u;}bx|q5&l*QuHKZ-qXsxg{-B^KK<0epRn91WKZVHLhy%8 z?3WdwO`i1P+0?Qo%(b7LFGNV}th{{^Mv=k9eXio|TDjz^jFkJvuUBBs$-kZMys+9k+HgZ+IqN=_7Q@yp!JPhU2(KlX@gYi+n!9H-ot$253hb*9U!nMi znb))wppSYb6vL8CkME_0@{9~)91YyiQCT<0(4&R7M0p9Bcg0Xsj3TyA@pR8S@B)gX zVGy|g6}<7g!LTs~tY*^t`L4}Bq=N*oMJqtZ6sm^k>QIRj&ypxdaE3M0Rgc<(JB8$`}Rz`QmH$>B1QjXoEDUI|`UPnt=_VB+i;0ILN;;j-SI z0H)b27sIuNAd=ym;nKzeCe0zG;M>9DgS)Mr`P#UK@O|D^8LdF8KNe2JUSdKB8H2|S%=#lfY2m>&`9!Px1H&0U;f%}^%OaHak|2QYWnRyL2F!eL#!Ns_B^u8SrwG>O=in6o^B^)){ogi%rO*__e$d1~>pzr6w6E_WUZ2b^?BaG$ z1Tg5QkNS|2cGiEfxPAoxIu0hm_HfdW@Iycdhn9i`I6*+wa-9D9$e;tn{A~4B>g1j$ z%liYCM1Wc@z9*4*4;u#I?r&zKD<%Hfa9=Xxjf3y})H8~@=atkZs$4oQufU`i52(>g z_-k3%4YpRQaN+-6;x)jxf!s&z3Kc43x}+hp6SDS1BDs0kZLWJEkNDeWg!2KMs_9+{ z3CAJZl@-{8JqM6@dvZrx7KRt|+2&_UL)f%C3N!_g-lAOuvsLzfcD{Slw{;g66k(LJ zyIXLM=)D4_|9e%+SCZZoU)KQGoHs)Tc5~_QEALKycz>qcQCr>Pswo$NbKWpH%(Okj zdV3i@y|z44fA|xe+5v@}BF}qMk33UN<4%MY>`uadmiM@E8N7U7Y`T+kkGHqH2WXfl z7pU%GP(e?88F(7g!^alH3FJE5r)0oEF**Etx#~%9iPU98E@t{=Rb3R{Ptf8hVwOd7=9N|+~UpLh1wdLMwn_pdn zt*@)i84Rw^z}4>oxKLgkE4P=q6%c_J`5A}ogC5nwP2cGGex@wfjCdr}841FVpP~0W zzg&-SuQC#GfjYi$^r6+8+A9Dh4=kwmP?F=_u`qOOhw%+=BDSEl;1r$XO(O=g@rA+q zddHo?>Q%J)d{CWKBfnD#7dI%D1dm40iup9`l1?^iuCA}{flSw`^bHeI&pkKj2Oi;% zfp}~emQt$Jkxn~3r)KY86v&>gE%u<$=pGcR%bEU`%79SnWxwX1jWfbkMvQJdgD5nJ z1U!E6{@9uxHN}c*_xc(T9_Awz_vFt(xp|jY{Jpl!=T#VFB!f@@-DY=)9tmsg+5*8e zX+827RW~xvWxo$ttGmn|i#`VdrQy2AJ6gqlYT_0JsF!a^&)sL9b~a*I9TfHL)lxD# zt(>XytTIAHVQh<;OQ^X2;HpL#xQ_jf? z=AR;LB@8wszm;$?{s!O6I z0|XM*(%DA)>cwDw@Jkt_!{0b4Qd5N2c1-4ut;mvKD~Yrlr9r35Q1 zHaS`S`5hS%S9Z1@R_>&>k5b5oW`DU7apdf~N}I{6o)(Rm&B88vvD9#&va`;XBRATM ze=Ynx4VTWXXN24r*y@3{x!enPwMVc&|Iw!`x77##@@cIg_*@MKZ)WtLF1nvBObdxA z%%l`=;jN`%oz!LF2XwlGNN38&|w%Ro0=z#h8$6sW&Tf%!!SVN#HvYeIe*e`&+u8n?XCTQV6U(_r?v-sgC$_nO?| zfb&cM&)tI}>KbK4?-Ja$yKnltjDhbDLehUQm%`2SVR1E5c84Q6XW5sKU7;xD>9d8f z_Shcf{Yq6qi+DT1iRX>CF7Pb%w01Qf(j2%XDVzTvY<+nklwJ6?(rbCMMT>ojQuZZe z7eXn@lI+<-_9Z(-D9V~4%P3?YA=_XmvSgX8lRYs8gRyU8`OZV{`+L9d`^|r4=6TL? zpZnbBI@f*O{a3vs-O>yw&e0h;f6v=#?|UE2i@3~pBffzb^IKl(@#62d zM?O!9xhM35O4p5NHQy4Ac6qQx@F);*3a{OM#=mpoYtN zMw!-)2}VO5LD~m#1`x^@%R2x4(8{v{02IwWjYqN?`F%%MAitIr?nK6!aJ3{LyJy}) zZfyGjhO>EDeK?!Fi$v84e3{!^ST$iwFgn|9!f6r!^ok)LT903Qk(L&SaP`F5#5SVYz1 z-?k(uH2pEnwjauD!4ZwQR2&Yq_f=SCZ!8wR9YvyV{*oFeW71j z5H6iW7y5<%)Dqf_S`?%pzfam^ zjRNX>55CEbR@7t-dR*3(KnOL+|A;sv*L>weVI9V+#!I4iBMG7Hx$$E_$smI+qYpEc z{sGTry6+j7U6_S1_DVnVd&?~pk*Xpm2}9gQB9`4oULX;r&M@zt?%iQSskO2b7nwy~ z`bYL;YI%8i_OU{PPq%--iUH|?icO%KMQLi0H_XEi6|#7_S7(VJgjw zpD^Ky1L1h#M$vCy7dqa{+18nSC`{Z4z6;vPsbb3Q6WL>mIn#bg+F854R*5Ft(1Ao4Kptuwa;@^!JV;;OBV%(nzv`O?@v1c=MS<8v z!Hl4dc!t%g-F~@TH=r{YLcG_v9pibXpXENuAKH?Tm7<8-cKy?>Z}fIG2qD49w)?=_D2^zoA_~t_0FN?qsA0QkaB8C zMhlc0zRy@4q$Dkz%Nz_b4-Y#S>;SE%mgn!wgwkufH2gQj9~a@X}7ekFsfbvB&;NnB$>r-@*jt-^atZ z`9H(A@_FE?(6R^7CLE_?jM{Y%Y6`x>TRydA)O!7`#7T6)t7%cI0oYT+LynD`L6~Au zx$;dAQpC>8Grx~u?5IEJnzrRat7?L^2B54JlyIt`SJ`jnlBJS)I=Yl<+PcB(F+rtgU@dGhL4>G3Y!P-D}QW^JZ#eB3cks zUocW7i9Do_Iba1(DHl;*q%Fh+<+o7#-aXea<6hyr@|5cZqm6T&E+(z1|VI4x6Osm&FO>Fut_xj1ZtSjLxW`58wA#u< z021XJfGeUl=Tgh`@F`juvsFwg!RpSG$Q@d@I3*|qVoc2VPJrzu1;vV200S2HfY9DU zEm|-q^W@awuLnGV(j|xnYVAnIV$=dblwkl?QjUU&&s6{~KUa)6;}?0xVV1LRyGnIs zMO7BBu(3C)6?uZ^Fyz)lt`?Asi~d{`RT6)q+x9g+4+tXI(;d_L7Gbh z8LP<3Q<~JEeaR_?{4&pP5(a0xwt1=&i)vk5ZnNV4+ed53(4rWQSw^O`#ErE@z{BE^ zqqx)@E3wKyswej-^2{vjbl*gOW)r>V1O)z+s_rO#FJ5@KZ;y{n`ZxCy8Quj-I z0Myz~2XrxmQ!i_ah@aOyfM{Fn9_eLBpdoVyWa#gW3*n7B8k>Roh@AnSp=sj>jvh*Wg^sF<&D*`p0{dr+TN!eSh;@rNZ*bwYLw@J$-=ZkS zU8Hp0B*ELz`J+K@QVL>?T_hjuu`+clfKtZMQ(~M%^ruaZ=2-bSKPrRIZ6I#=Xaz2- z*{wM;O($wQb>x1r;qLh)N&7nJco$!pM}tPfX%_5h#AH-X?5I`*I2fUX4Cehj1pSGU zJF1&EFQlcO0kljB!C!uRB~R_&<|}rrwUK-jZ;qX~Dpvc|m&O$CE6_)R1NmClr1)uS zY1I{*GllzLR$~!>blp2IFAd^5b0 zm78T6``5}rYZoL4HVjToVNdd;a88)IIBABjI? z4R&$5l(>!C(YveZg9J9<0J3Y4;Q-AP8w}jttw{%WnFl8lJ^{gniZzrNxulKN54()V zPnz@VWxi!15li!(Mg|ae-tDbxSvL*%SE+s6(uhpMr zZ=QlMFaJ$3dpDy<-0&yY=hDeL(<)m954xC>Ni)kjqu1Qniwra&Z|;S2I@e`otmxga z)4p~p?AY^{9I$#vS`ehU#Dx}?zZ$@whX36ikH2&$_Qvht7~v{uSUII94NBpRP@n;s z2DWe1J?+UoIUu5-U!j~2F6f}xu88RRkW0;|o>yI}IW`BQHCsS;!yMHzpYT`H!qZKv z&4fT!jyWqbF(nN731>p$Ym`98*_ftraUc<9vN|t8Ujrl}k+wcIv#?NE60MOuT7A?7HZmy%~77>IhC1TvUxOP4+cgs^NSdA5rx$j7>lWl}Ii0+USJ(D$ar$#AnI|1KVK>&Qs~9ut zy%CxJ(~Uwe`0XI)!&{hcI*!5+aE{`!Y~=Y($DN{?gfP73tCxh&jh=IH4rBfwSFI}q zcifnm!NL$+O=vjm) zp+a|ZnFng`hMaQ8?N(1%n>q%p%g*WYd)CiX3k6GZh|^I@ZQuh8uqBt&PBQK}UwHKP#{Yt}6y++rb)6wbNg@(7>fRpuzFfnT)-a?x_7X z99r#k+G4=M&yLs&GeF)E*6a#?9O$-@J5fuv=7Fz@wA1(j1MN(1_%X) z4T?NhOr8X$%cjOKMXrAfF4`UQL%m08iOPw8KiO3fpR+Jy)PS}Y=G@t@>G8V*deF!d zoO#(l(yi)@+&!N@nU5M2F_<3|?3U9NZ*~%46m%6Q5)!FIv#p(6hMqhmQ?SWYMJ0Xt zacbe^(YXkC)T>OE*0mp4j(35DCh$*Hth}@&SNPqpwFO9_(^CrcB4)XT;ggiR%3cMC z)xov&^Es{8p6?yzCk{OIc;Q;N)$8icKHz0~hZX?0AyPYN+Yido_YM+L(7W27_gdHY zNLuCX5FPi(x&7lowHyP7|KAM9O_^`H$^u$u z$zs_UYz|$W?-e-m0DG|-UtQs_P8Bc^LCaL?#^U70mILUulf;&*ru7(1iP49`5_AI1 z$sh2MalS*RNaARdL28&{GvMNv&I7^D6>3o>^EcickSt>+bRXR)G0*(+wCK2qTnBlZ z`zc|@3`2T5mJDj9TE+}4J&)?+3%}P;*78d-rnIxj!DWFaR~UjotsQcIOy)s`_`^sG zVXKOtGP&hc=wY-rpub!_N#SyYAB291@1Hx82wk^%x@7@nZ0$lQmPr;O?x(z{wQtx7 zB+Wz2(4|x#;XyBtazB7k(M_M!a=@u@vD;5>z{cHSf!uTL;Q zUm(-e$pJjcH_0VKf1LxevY};~!%cm#m}S2A!n`~FK<1rPhvP zd}%HUY!j^~66RX2kx7OQQkY$Jy%ntR7-ehlS=XV9Khcywqm;OM!|amAs)9+>)@Wu$ z)G@4t0!>(lnjn59RwZ~<`~p_Mo&kL)da_RE*Q$jz z-Y=j>Xq<0z_llihfpr8UzwZ&m^`{98{@T$tcbR6_5#Oo+FLQM8ACPKch{0+m1;ARv z*CPR)8hn8Bu$MU++h>1fAD~~5u^s)91jK?F$J3~h4v;J0CI6TP^yd?2d-M$rg5I6f zC_EnXKNB5tJT=hpf^@Zu;9*L)H*k?E(SgPW`MK{}x3-ig`Ma2AY{474^qtU1A(LqDDOEEw|&f4Y*B9X8rDC>XBMGhVxHd z1M;(sP>cw>?67vwK5sRVqg(vWvWQj*K(FRJrZ- zw9N0^o|`Xnd^S0xo-b8@B_9BYY6~jSdb1q^UW-2}lrhE+x-bn3=aPaP zx*$dDwWA_X{Z>*;1q3T9aH;iNr%h^}+Chm>h{hKyMzC56B#Yi3$b0au zsMcZZBZ_G0nWSU^vKE0{r;BJzzF9^5yS{U2&C=MMqdNEM{Rm zFT-){qHZQ;~f4|Ga}Wxe_&o{cAdQebtDbn`@;&+m9^=Nt3A6(_!2&jr-~uJwZ}`$d7N z=WRK!4E!#Q=nBFDf4j|^AKU8E#a#zwx)^6!u=)RxQxN!)_9Pm*8Ut}f<`sV+5|_j=Y! zfV7t>huUAtk#Pb0(%Ez>&qHsPU;b+p{y}Sz*>E#s2stCAUuu#P-62N2VQcUcM!Q$v zLd6%4mCjDGC^i{4nHMz75C`rNxsFAw+klzLMviz1eqahfjxIk5 z^L2hFC#(a)w`n@=MY1~|7%ZoI^&6Lq6ibRq7G(z9M%xE7+ip8Haz}Ut-T&iR zJ6tpnY5E>GWFL)?n!8JhZvr;Z(3$Z>DC_*=nw!22>lK~vI&BlTZteJeD=F^`neZic z2v-_dsRzcMMKfj5akK5N&JFrjy@%|c^TFyY-xD8&9fnup#fl%Z@=+TR(q(12&&`uB zqoevcL@2K|Yjgnf6XMk$q^OH-7DqlX4rNApRyd;>a&S>(M=igh5*;ilAikAGarR1^ z=y+fYqLtmbD5)Fn93-DNfuDaX#3tsPy1v0XqoduvoF;0b#dnTac9+blY91MLWNJE* zD-Q@u^Hc8$e+2|D5FMl>ma@JWzDbe5Iu5*d(p+<#GoY0yfX{6nxNPdEjN?;5SEjJp zIu*h|^F@r*s(K(7+69yIp6oyK*NSYV?UA_n9S9mta&VF9I5qNQ({T%u-}!yPGR=O! zJCL(+e{FtO(}2jsEWp*QfjVHl)*Oq)@Fmt?eU!;r&SKs7#C!|(XocXVDVF44yVv_+ zeq5%a%Vu<10q7cv$=N+urmEVsF(z*+H=!YgzdV`)ZN-(ycWYBpCnh+sjJRG$ZzrRj zDCw;BjC`LieFN)G&-nO2`v*c9>1WX?jdV>;frh(F)=lC>%W?$s`ek5K&CFbC?%d5v z1=Pu5XDzd_A-DFFqm2PK?m4>A|0bsDmdHa(?xWWBy6Z@tZ_wJWvT|z7N#|jW#izBh z4%dv2>b9Wd*7vLWVQ?ya*+F56t+LqHl?ijw3_vBVLO3eQb5V+M?~W%2&DCa~QWay5 z^E#PZ>M5nyKzdaf>lz-d+gYEp&_Sr*C_w%$)6~ngGfwG1wPUS)yH5e9itnJsGiT#j zOoLf6>LtJTWPwK0X@ZND*L0-$l8@+hBIA-sFRz>yHJUSG{A$(4tBkq)<yaM;me0GJ5-0`knoD3~_2(lv@vnGqRKZXbxwe4}D6sUROyO;3?0s zp2$=3zj^7mGy~&Nm-moH4-a9EswEIQUPNcfBL&H4hTPHsXEvJTX5r>pN1v0=9`Vh| zIEPi*#W&y*x)fT&??20k`lr>CrYxO-m$S{~CYG-16P9HAX(ys34mZra7dPkiyA-?) zibp!@t*vbn0*Ua3PQ4f1e0k3GIUjt9P>z>^d~Ke7!+si=^ANhG;=mnw(*~v#DG4>p zT~`gR@J;~%-=+-*;_xLVjG(cIsj1QW@8~V{IjJf=KD$jOV)E-?Aw8^%P%%j6?yjqI z!Sm_YJLJTUGcc^AQBpwzl)i+iP{-7)3qS^>ET9st9B${ZVuN%)bzMt2ddv*dp1Xc~ z9?AKfaBDDH(8(u>Pb2JJgG(?j5H*D?jQ-ISrtA5Za>|aW828s#rhk?@*e9L5x=v4u zGS3euci|GKH8?1;PVRT}9GRa^qy=pKwO7Xsxt!W&=ED`dfOyz4su-jI+fF28-g`rQ zI^rO)SnF2cfL*<{k>&;(`V%$+0&2TgKSJX_Kr{K%U-2!!?rr(LdCGx>_i6=;x-o@s z&~oW32UjXAc20^f_?Hc-6bvN9ygTt;plK!`jhtmr_ghJ=+L*7u*V@B5D>w~D@-=h{ zkZLU^bc|Qx9qyo^85U6GZ_cywNg|`8mkY4>QMmdU?H8~^>nNxgV>p2{P5dG81v#NO z7--W}eNVEq_6DF(t9vf?0@}8usXUeTmERyL+RlYfGw7&f?FEh?`Xu^f2xEf{sYQpg z=FX>R8K#LDS@^ukp=Cne21goFG{}rI{zO77Knqz#UYbYVZ)#&^C9cL#8scPX(H8{4 z;=sTGT2rP=&pzs5APBE&7oaphq53sLJP7RgyQ}iyb{x^K)-$r_nz$kuH}5n_XwKLU zueBSrN1U)OxkA7Pr>=$8sHO%4J|SEdXEdqx%z$#;;hGYc=Z}{4M&MBDUc(1*d5A_> zN>>cF$4EtLG*y@c(yU*96uBs1%E6fNWxsH5)ZlBl^I$`n&_d3jle<~W{4Sb8j#QOw zpJvX_81r_}D|Zg5f*}J${L8_8sW$f(A~+|i(hhO=4^ZDi2e%|gm&5*EwdYM?x_<*h zg)b$2L3!7+o%5CfeuoNwaJT5$b33c`I;CvV%b+9n;HCm#iHQRd+M8~vo#$mA`H-F6 zk0Sz=GUc-w!bTvrDp70|Y>q`P4Ht_2^~%4hNDoS4HWA{>&kzSFT){eaq}Zca1%0dF z2y7mU4=R5=m@RYyDi(iU8J zOkE(aE}?oSyzF7VlO^@D#Vx_}?xtJ)ox+SWUxPET#Py1>8^?V!^qb$}oOqf^U5P#5 zAeVMYdDbic9evu`u8{}4ybweyKs$S(;2hG-!vx2=9V<~9Tp4u11n+f6yNMxWoc$Xv zN>!mrJgXbA0{HNXI4|ireac1XYTHB?p#P)eunxoEDZnj*|v2EGV*uD$8xN`r#iqzAvITN1{9MiHiVfp&6Sz{Fq0YY2{M)WO%VlY<-B z=c5fUgxmc9GtOGl%TU`SAoEkvD<@-pFlPiHl@Yhvg>yXqWbM4!Hi~KHMH8iEt=;;f zJ}B3_r9rfR6&S4-gh9lBpAk@??pTEKyRKn=@QVFy?D35@04cc7jl-~qugj^lMRroX z%*tR90v=u$XEou~}~KqWHJB zfgs<7%X**K#ed3FX+J!)e>u2za)=GOS)Y9ODkzdmt0L&>_a~qG_E@>gnGjWPuF^%| zRG>3@^fq*p+*|-o{LNC$8gEensEuHe9lDfGHS`UaqrGj2tMM-`cHt8(A{>}j?&j+L zHW{MQ=9g{WW)GM5coz-~FHmb=8|U~cO+3xOUd8U=9UD)Llu}#qeYQXY!6Z`6B$h)E z26MrZq;wV>CRb>W4z;S8k&{eCDCy=Yo3%w)pb19|T~?SE0&^YVe;ZRthu#-$62}f1 zUj&Cbwa1fD4I(#CgyL{J@s&sVa<71?J^hzZ0p+(IkvqW27YtkV^95lp7F{W-6`viw z!%zi84B#MYWA1aW(cj(-AAA3VVh1^ylK$(ia`dCa8=wx>{Z<)*?(!vcy5ZMC! z))JxKqWbR1=oqg>5wQ3plK|vPteLopoMAd>3d2*QSBR;Jgvy0*16~nc^#W1@1Ce0GlwY(p zo)}3Vv}uw#B&R}0zch$$v`}Wj6NB_ILm(xMDPv!F29M+f1;u4Vd!O3kGLbR1? zeh9X*NgI3ql($?RvimkZo4;Z?-#@0n1xF#?hVw=n+NYzr@SB0Q=q^OxI4$`+g3`uQuo6Y^0>6%HeOQR6A}$9ufTa33R6E^rpc9y8au#SGM@`*%E{gMc(7YyBM7+yJ%+!=Qwwc^VI*PY%%3t zt+1!Z7GcvD>8!{~fcOmwr*ymRkUkaqP8dHGI(K*KYRu%2+};DzTy@uALw3A(Q~90# zc8gJMTj#biLowO+b(1Nt&j|Ax_Gc8-PBe)dW6yK16-7T(6#iwz5xloTi%jfq zi{mcr0bDpp%j(yz0tAY+9CN**jzKXJrQd<;J?e@e$vqBQry=+wtnnDo(y;BOQwcT& zIS$I|5Fmy*#!5kR0R^BFKzVJDYE|P#U@}0opj{pylj3YE z#-aD+s6?)`MvKnBkc^?Tc@x0)^QQ+Qe81C9&?wUQBZLDji9~&gg3+P^ z1HLRE?72@2il_q9-WdIo?B2>sh*)a zd*jE^LvT$e)@_GHPj!xfdN!K+R}FMb-P&)?caQKDsIv@kfm7rL)C?ek_w3KGRVUDh z23>3>xyS(27Pli}Iv+iP#P0mH`cV}DEiI$ss9hy_;xs8!E1-pFOB5Na$PaEW-HFi! zcU`fzmICPq`F#IB^~N-Vkg(-%Eg(@IF!OM2XJI5aSTr_;Q3I>MdOy-_xI*;G z%;U;(!^2(!PF3Ack)||S}PvX3c zgW-96Ni|$nu6oKe#L5>5Y6V+L1@Zfv!t$ zHm&+u%WJ6IkIeGFu+be8(S99iSZi}su{%?R9rjQq93om+V8dT905#W#5fL19JrEpq z^3{z%8JoX3Y{K7q*d~Qt+{6OO!0vtmwloC_*}%u-B{cgyT;XgFd|(1t&qs@Q*6={6 z{e$=ZPUy}C-qa{1)v+TNWq{t0i=oPRGe02@o@;9n=I6Y+Jr!Q}5y;^Cw&>`GF>0(R zfE!CLV)Hfk&Ie_Mi&L*VMO08rK(Brtf98;o(8Y`qbz#|k0E@<3OMU|+19ECno+8)q zOhV*JE_GJh1jc-3N zT_uY)_Nc>b^OoF7n`K>Fms6)jPs+o~$r^S(6aeM^jo$K@RMHYLW9$9rE1-c9M-43J z<5^fk1p&fJL6h#?&XVACnPDo4g-huRSopi~V7r!;?i7}1%F$J3&bci?KV`E$h=AZS z=CRMFs?yq1*3Birn=+p|0Kgo*%F8gOcf zrkv9*%T5fC-x{%C(!4QHn`P8e>ODbo-g4Vt!R}V@+aY%P_%@GQZNI7R91C-ta<4_) zq^t|ByX7PL07Gd?4nSsox$edJN=Z_BAeI3px#QNYyifk$Vp0&YpDil+^!MiPU~b*x zg6FYsdSjyTkY701#imAU`$PcU^%tqH?2J+;Fsl#fy|0&egfqI+fZ8WOwW{EVKxL2- ztaBHlgkU-rqCiQdj?XdQ-q`5sm1qfyCp}M4T1FdMbzy&j5=64VV3gHTUSbIW*})wO zGTk}((?FC7n(g-mC5|p92^=mib4JB@dQ*_Q{hr?-ZA7A1GQndp4^L%Qslf|eSqkLJ zE<&y>o=zp{rNfGA{T*=*q{7<6j~nj^BU(; z+W-VKH&dkVxKYQ*Nhn0`{;B$v$Nh1clw|Ag)Q4HVxh3|JTphUn!s6?2@m_Keqw`E-lhFICUNps9a_yDsJ4PKe%aN4kYMr3&Eu&JKV(j=D^K&dzoHZ$)#by`YeVNq+JdZKi!wa2hF@}fy$Y5-s_{0 z=3(^Bo;01WIZykp<4e1kE7-_|%%PfTz~eh~l9wC_J&_1iXc;Ur$~nt`nlNFBVT=aN zlQouSESHDXjNHQ+v(#e{AgS?vhgM`xvc1j0lO>I2W8FD1s3ky9dCb9$p7JGg1 z9mqS3MYtn{`rdMP5@)EUB+iQ_;Rmh3aVB zqER24^V*w3`pphMW)p5r`%$5fVGGJoA?rV^L!5)tRmu1r=OtlQJ+n-*ev^d zDkK}7bc;$Z`E2W8-0*h!MR2!blCAW4AL;<7boiRY53WXEuYrx5u^xm%$H$VuZmdAG z1c>lQQQNm3XiePCGtvsw)pvd27*VEn?qrrjjIX89U%p_qXFG`vrW zHz`^5FsFR%H$RLh^hz#yHjWIUa9>*8gR5^3tGxr-1{eRI@UiinwLJ&ZY%$w(vN7qX zf*Phu0a4;2zj@8{#r|m3tH~5!*Frdv$np?|O{HNzurtxftvdCb;m86T+=BwON|kYSlt%1sHPWn^cP3YcsigLC)t}e&M;*s+x$cS&Ho?6 zHlF(+`h&lm!a8=u-{lljW71(G{Ot6>5n24}Q|FvcntlO>2~Ul!I0#b6zHSq(8C6#0xYl z2j{5J3V-vG`;PK@3ersO@Ef7cMZ&BmdL_nMH+m>UKlC78{w+Fhq{zViz2t;4Rdb?v zGo!qpr=ig)3LqB2MP@P)@sIwM0fikXY;0OGH(cbHgD6Fs^$zs5nMJzgPx{-wXdrfw zjfy=|kkx3n?PpQ(2^u1wg42x@zCbhtn!tfv>bu4HLAs_1#}GPFQOGZ9(JrxxlJA_? zyXSD<>${LWE&3Pk4B!pSgM^MhK!j=@X+7Y6)Kl*=XV|<+d&N9-TJ-kAt%l3ZM8zD( z!pR|PnTO|SCwH(bl`npp#mo&aPVH)_L_e-7G*+PsKJE^U@XRn*3pfqAe7p&gfxgr; z403WDAK7)bCa>+2K6(;RuT7AeuZyu}<#$NiJ{7nn6s z=%nN4>Jo-Pujkgl2R8IBI*wIg70s6QbuVqSN13n)0VV{(0nP}0p*Y99yPUnmlA=|scjJ8wo*T%x^&p(7MS;Co zfBgP|a|^^P&A=uT=Q^~nCxx?UM3H=%Yb;i7xdV6;R{^C0R5$!Xsc4`4gis4|P8gyT zqch=d+>c_8i(s?^R|6?#>u$!GM6`$;i@ACSnoG{?s%hCZRY8s_5coA0n2s@8p7NuX zd*YKsvxc+A19ak4iIdW1bFv+6zxh^Bx|!%N1Tn#;3l`T-~!=KUoz^GG*0N&V#|7K~=`h z91nvaE4b+ahW+Nxq181Z2>D=6rWKi{Zg8V@ao@`c=P$!Pnxk=`_Z;DNPn|O;KNff6}^9<2leHVUGFB;8a3w^SWQJR83BFm;EagHGk^<&?XkcQ82?D6(el2SjsQ)9yk% z4c^btxu9@Tv6*5Mc+bAj^fgLRlC8eL>YOo{7j@>%5MWZryQ@^(v$Yx|&D>G8ii@l> z?8a12xZTxDhMVe}?y)x(-C7RGb8JXP3?fS#mrqfJ0>lJKWqWAjBRP|FWA?*|ZvLvt zoy2`3Qy5WNjg0MrF|M+EnaZ1^a%n9r&MRy7k#kQ)uV3BU!7bl5nZvykOBZfMp0Teb zbAa6&?u0!Q=DpNmS8u^qE*OmoxdNm!bPWdgX&5t>aa{P#!SF+1{*Em6-QWy}ee4Xf z@}KtZZ(UXA9D@c6SBJz_5FEnz1U7XOTJ2RN1NAXPHJxs}hBFH6vBF zZPvKrk&9Rj=OuYH0~!Ise~lE>-e>M+AE15l@7qaKQR?J$7yKGMp2;suyHB_Eg4}y8 znXo6=hgtRryhaX>VvVi=5N$|9zT@dOuz(eD~#GYm&h35c$gY^tVpxY>?Cd+;)!Fh=#Qy&ND3n~2QOk%lxc zA$ecDj@DFLkYopVCc{MmaKHE!pZ)Xef}J8t7D7}Ti2EY!VqUANUu6kpz6(}Tq$s#^0j4NOd@Rl!nvM2nSZzkHIz$7)wCJ=UPf5IzB&U8Ys73AHt1y9tfYo;g@Y>q(YlzS&#M=_%*SNKlM;|K1^oTb8V(z@)!x3s z_J99Y4i+u+QF1uacFNG-sI$_J9{=;G3$mZ}6Ho^{pw{ti7>&bk7>AAY3Du} z)C)lTCn=ck|G)kQ|NRQGxec7KxcHl)ZJe_H!A$S`FGW0{GF2jU1_~d%3Uoc*xIqa9 zYCx(J3#=*665h%Ne#?^0@+XTY?0>skK4Yoj^bn;qrjM6|RM7Pbc{%hUq4*@>cX4xW z>`xvZDcO4A76$-Nyz22s1vZ7)KX^&Dr2Ma%WH2f(QG3s=r1ROKQ32v!{?d6z_qE9* z2JO+*LqA4{yAv~enSg`!b#M3Q_+3aBy?;1oO|Orfd3#j*9|SWSR4Ea4=+9$;7N#e~ z&Ij(;fEU1LL>WZJ0m|=~E~Jcah=7Y7h!5<)!2NLxO7j4{(+Q;!LS9#nhJh7xy()~6 zgPJK5;xw@h*C_NYP#GFIjFs*D!819%=1!m|7M`I&0`}S5qm_VogUv=lChSZpn%xq; zr&QsyVBtXtN}^6X4rzw|1K8d^HmJarc&p2h@CiltT`sjzz{otZHy%h?B8g99 zUp4^;_i8wekeOQ;qi`>1=f9Lj&UUb*{4IJgrY!B)1d1LWy*$>5(o+MdAqca{U5I;mM1klieOY+xqS@( zQD!NY)_007`w9Xvm4u~+64|5#2k3b=3ok{dsB$U=Gt(-{E`SQh&?lt%PdD5IcsGEg z%l|V%dyo_yFvIXHVMU-R^WTxhQHiPn4HIhUEkNkzBQR7uuyb682)sv+js?fPkp4`S zXcQQ{VF3L7cX;Q%bYtIWyO;wlmZfYW!5I1qGy~3nG&gFH`p-&G0CR7%yE^hFh0zIm zxKsp$j^Fx&bL#-U6M>2p{ae{{yX#>QPWdR-ZVy z3PB@Gn83Xt+Kn$P>HoMhKeup3px;t07Ki(gXmsu{!3FT#b42j($8+*6ex%PqNzIJyBf4vKJ z{i&|otung?JgX**_p(62kXN@W=%(h>@u^@-DVbQuSIwhj3scPDKV>FNbZ(Ic; zdD^^dc#sj)aecErVhN}_ACDoZvIlAig1RN#-drvGXhl!~drvC~Z?k!`d+?oACn00af=wwH>I6;BJ7c%Rjak1fIetG5_Pe3h3b5Fs>GG%~8-@ z%+c;1*uajq8P4;ct{v;Ypki(iBY~2gLfTJ}9ijH`6n$cbH*mE7IJN!RUzTs#ETe%P zt*U06CUrO(t^D(rtRS0}iq%@}Mc`P=O$NPx!M72Rsp>Iz^(qmm!eW~$evNrTSG;5D z7@Yz_d7$dy{j23^{bsVw*ZtOKC5OX407nuFWfLI_)h>CMM?u*{sh+||=ot34X^Da# z;q4vFKz;qI`F^B*l2zd`NCx1|qan+dWHT$e+?=UXA4FWN+5iyOc$xp!(!d;@y|Ie#2p9RY=+j(#0GoW|9%JlQ0yIb9=?srIcv#&dQ#rmp$C zkV*A3Fc*0s4n*+%^JS`|DpX*CAzt`D`}eI|faKmnBf5@(T|Tk^D40L0JagPYPST_L zKapn&=LZTg6MiM>f^}q9|Me0fkw-;hfc+Ij38S{#_O~V)U>HtdlYbUD-5EBMqKad0 z4DfLarPmjmBkK-`goZ#MTmht%irZba?9#mk?;n+^`7mZ0lUjzJM$@WEc|QANW) z-3W=CTebusz}xwOQmaskAu6`>9V4Wv_c}C3L(w{rIsXUT&a)-f0ot}%cW8&=`tR$HTHC( zzQnZFTNX>~M`O3@220EvyjOmIo58CM*EtLn5V4MS(rDTE)m`6IC*Bg+RBKFX6=Ar- z@wFVf;qahVCb9Cg-uut#PX!IjtoK7skXFiX{|fbI_nnTD2bDHPz)R|=duGWS_Ew{Y zDnW%HV+O6%cIo#0Y2xJ8*B%lP7a&-TdA*G#Y@$T!kw7+-xnM{&CCcP7}`1F7$yeli%p-O{fwPIHtBU%09>z1*219!hgwack5= zR$F0bnqf23Elch7=J$PVnYnC+)j(`}0&A3=%$I1lVa05| z@9|D}#IVaBNPeKR1d85ID*~N$eAi~=gW4~_tmWXJ2QcQrY@bZC25fU~zunc&KpRB9 zSIND@)zY0Vi)pjcY-E_Vi{&>!Fl=IhAzOO9SAOy4`~KhO1$@>=J<#$FaH~?wZ#1`K zf8pJ2}3?vk2Fo{SAMpZuJj7In5kxZYvw^XrUfyK=V$?bW0mE%arZw?i9} zj@Gx**I9Yd!4a8^wff@HpX!l4v_Zc#58n5xXY+nCTJ2UGpkpTKo%GBth4PF5Ao#$Z8SgYV zq)pTkP_bMM-tN$pctrdnO%rwR;8Q079i{QZV1aeH-G3IB}&0}e5e zcHFk*)pPR#TU_RQZMecnPH|ccSNWA`T+omWjBQgY>SBKoXAD=>3mHfCsxdNO6W^{C zVsD{$^oTWezwUwKg&?+Ph(6?X5pD?li3KprtFHW1ceEORpbr?TbTub=QUm|yQ z_MyDEU7_5^aekYql4qv-b9+{+`NR^`u0wf4YwU?-JfgOniq1M#@h6ENn9_r!6iPAw zi$9|&n8K1megoe(asF|>prg|G9#23vSGXL!Q#JO4gW<{V6K<oue@2(16&w|Uz}UzWl@!jw$A-ju9Lklr z!p9k`G{OG{d{_K|S9aK0#p2gXd4UhxaFJ7vetcL5t}TAzAdmN#>4&>XP4PjqX_NHM zim`_5tBh5n?i~mBP~?;b){^uM{8c{`rVDc~*PW zJPqtaj|BWLIj(fLOQ4j=ukNfjTw4R{q~ZbDz1N(ft7Qv4V40>}I&1j(aSxdT$E&#KBz@>?S*@gm~ocR%5QSO@N52!W9y=8QfRo>pzUwZYu@qVMIY@FScdgt zg-r&GzXc2}wb*r&;-|K(S0fmA<#z&!@`h+nzxW27W5+NI$o=V%&^lE6lwPqbRcb=e z|F54gEW_4n{<0EF2$A`51v#em4O^`=Xu8C{jW4U=<*PqP{I0;QVfBAa|2#oT7AD7k zQI<=s%6ZJ2@kWx?-x9HrwyEFvnzykqb7hAt7MHpekm3L|Mbrd>#(StlkxvQE8SsUt z^xvo2*{_I2eZ@?HW(RFgl6!=@1arYm%RJwPv9SOJ&kjQSG_B}5A>|ne{k50>oddsE zYi=$zhr}mT#i=XJp|lFK9-lT=JA@SlIxb3YNyOf8_xG^2Yk#-5%5C`R(T$`Vzd(Hh z!rPHOC#{8&?@5A0b5y|2n-La6b(xUy%p0U@I|fA$ONH3E!l}Np3{oX&1WT4aE?13U zEVHN*kUEd2<36uQ07TK%{dRWNvTWl#9B=9e7{9@il6Nqfn5>3v-nL)h&u1??^Z3kB zdnba_k+zpxIDrmg*Uq1VUA8c75T;cTrh5*-jxL23OWZ6b%NvJ4~R!y(iMTJ$|DbKjDWj zUWE3;rM=J90tP&xO=7O|a+@H+207w~Vj)X-X=~CI*G=zbv!r%R__s-_@$YLLyg!Z{PMv*7 zuv~TedXOUWax>#X^2?s+e;d~q$`dTDuwi%|1ZRy8H^Dw>ICB}bO8I~I`tGIBP!Arq)QVO1py5pRWP*Bu>zrYL`vvQ5a}fpO9T;2sM0}*5LyU5KnQ#r z^ql8;-}m?X`0pI=-1qM6?Ci`n*Uo@8X+U;cB;(kWi`eCrZ^|d*AQ0uxIcn5 zQ~5`vxVr}mLE*`glX*1X<_cQ*U;)k5`w=UOTLO-h^>Hz)q?&T9++6;4%Eria+9}m0 zc1C!;sPcCU8b1*Lr9{xm{ch{&jn~Inq?Cpk-PzIxlQ1V zLp1NRO}_HPV+weAm^4M7yAKYm}L1QYgGv7??O%0JRL*s#p1b@#Jh zqi01@z8fxt*#I7C`>!(RvBsg)KAlD3zDN!z?0gfK z*=1(Sd`nA`#>T&j3*54I>fxt}@%vrxvwg3X+$pne`{~1n1~zETCD~lt@tv#1NsDBL zAz`kdHoxE9zaIte_b0zTX0v$r2yU_;wCpHCY5;5ICR0{hiVz>1pGtP#(SN*u<2ZEn zW>_4^>@2OUJE4Jktm8buA?g4DsF|7TkIO;ziRXQ$GlJH?iL+7b?)@OC8UMg=EO_es zzxj_2928ET-_J}%j+|ho?JTuCE6%fa*w$FlWG)Zom3C=);Dt>H?nE^GdT}BGR6zd7 zZ958F>C;I`onRC2>_m`y-zC;2Edsw)WmNoL0ybLMSvygZYTF9B~0YzoiUd zH+#S_q4*%f{k(u)hefW#GjwiNlTFYqRd4>Q0iKB`E6>#kp_U6T)Fd zSmO>wRU?KD7e0I953NS;g4}Go3rmquB^y+y7M<{ z0=(tb|5yk*=&cg5JB4o765691B&{BQwL$q+$@aI01B3A->YD1~$A^4;2Xs^yR)Cu{ zf6Qavxrth3zTO|jc^DFR!*cO%qq3~|`oUu!MUJ=4KRoN1B8m1c^%-m4)G2eGWJSG<(mVwr)$HN}Vnt*zN!i8f)=-(8F(Q z70O5Y%5JeO%ETWS2O8KkMrwBeV^Kb`Tv^h`AHT}qYztBu`vUfEV1w`+fLej{oG#aa zul&4pI-b6&%`uET77&oUjYg10d!H+Pl{h#BHu?xZ=;CX#^@-Qw%890lkn;>Pd_%|x z9{C--b2>HIxpClj`nBhl7yA`K@OiTHbbYFlUtMid>Ab^3spQFbR`LaN??7ys(e>77Q4f$H&5AJPZ{c;7o*b5~+f$&US!%H^QxQ}GfjY+Cq3Rcqt1 zvYu$)Tgrf*1dJ5pHwT@B=I_`!*NJkT;p9p6 zf@8p*S#<6k+6v9T1|kbMxWu$_Uy5 z4s`DusO3&HA=)(wgpVS!>$K48H0hM8zsT~NvZYlFl3AuT?R?p%`UO&qE zyT7KbL8bu|?_nAMqRlnPg>OYUPlxohA1jWy_nA$J{4V21`$kGXZtp+4zaLPMOU<#y z3GT73aec@V>#t?Zz-l2z+>crx2~-HIIMH;%thSWgvhK4C7>}p_v(?MuqCnBnZ5oVZ z*;(8siJt#(>7ChmFxKu)h1ET`#!z zqMS=T{qQU$ZP9y&88Z6qV+D8K(KQ*a0ea}X{&3Uc=M%gAZLG372VzfOw%_J*M7_4H zOfsV#_ckM*%0G2wMpjYcOpp0Q%ddfUlKshV2_RnX-fX4LKRctX>@0jeMbS0KwAx8t zWz$RJ9Wg~XBk8h(hF4#bO`6xu!TVQEM6TbYFQ^{#AA0apd!6tLk?vpv^PL4Hj+Ou2 zJ|G{c_RcEz1+Uvs7dXV8oM030KZRS{u=YBWRJ+*B-P?W1&d-}z~>NWFeZkO8!wyhQ% z4cyy995i6cY17}j+imN`99TM9N?7vW4}zrwAn+@Wg~I0tk2=MEoB;95u>Z;rnO)H_ z2M|soIvH=lM4J2lX!Y!WJn}6g)DS2o50WKmv9z!#3;?*tpa=TE|G2vs{rLvmI(&jn zLGIzlee0hzQyyZ ztxPE#-1h}q96VCSr|+_gKmBj8iL#gm|VcqRlNPbUjupK z@2iuqBlMpp0w#j-I-ZY*S|j$)%z5_`BVN@V&*9i_TmJ=aydsV&pqbfeZmGYdUX zn;m2m4uhKim42J-8fUyfVyY&RK`_9j9g~0azejvT@z>@%%f<3Te*_Z<(saWA8FGIu zNY&AD>+JGJl$~i+_ZvlV6_nEVA^#4RcI(1V1pGR7;Y$<9!-&z^X3V#g6a5tc2!8zE z=l)D;jOQh)4>SchPg%l&=x_2rqkh)v1V#;dbsXNyW%8NB$diAwbnkuj)9}Xx+q(Y8 zijv23JS=#Xd_AFHHLR7>=8n+_Q+q0W39pk!SLNLgx-$1LXk6+n9kF1vpg1{Bsa51_d1sgd~%$98?ppS5>y z$D9S60@~j{T4X*6EQTy?%@&$Md8ivl{vD6vVl57+{Y9&9cC@@>9b-uqWtoQZXYDP{ z?gD6RkI*kiSldd1<^kkRDgrqYNo!BQ^Uv`AYE!Z0;tJ%CRwv3mzRe2W zM*F)q_`lkR3?=pr_<4aJPBHXnu9bfPy=&2 zG9KihA87qgaqYAL$!kPP0?h`h_l{c{DhaaGv_IMeM2^nykG=l#$RzlGh%P_RSUmGE zt%n5|o;{k6kxyy+;=khETW+QLfrU2JK>FHeV1YK%>YEo@w&ryHUB^GnEK}~bjueo4 zUC5%4mV0!TV_PATrWZlcbHGFb>Ff=U7BcQT*PU2B z;J5jOQCR`-eTFrTvU35|0R*4We?PhUto1q=TJ21DUFcDHb@F%<*hhpza(AU<++~Z8 z3r+egTX-n*0v6<>f2L9a4&O;Sk1%OWhfc@ulWs+$(s(Nq)?j7DH}|iTdv_}l?2ET< zTkHeno%%!i!$)AxoZa-l`qrYyV%3Y5J zy4xTgBd$Cqlc~N~LPK?|4}#Q4#wOJwZdNIYwKc78Ed%PJ@%OUP)-o)zLd;;H? zb-mi5^d(GaZz=PS$IM^VkL6!>H|TqBHkTuXGsNTz4y8&!^%EE~EFmd>rtub@R9fevcN>Y}U|szN z9#;|ZqTO%zzLhTaE&lDvOZQ>10!Bi2SEU%mVBapVwQ0uumP{7kR2=%iwoh9Xl3rZD zV^1oqX-$5ozBNN`JYU1rzcZpg!8dlL{q_F1q}5)^Jn5%bR}fCqbUmlfV)A`xyk8_~ zY7L&co&3+N#9|gXe(Aul(WvV6gjQUDcbnDc&aJICLfSG3sIgVEUcZyY$8H~)iPfi^ z;OtUH^`5Wt`})1-)yL&Ey)By5f6wo@t(FLchE~ zgHphO;y=w83bhAEINt8tymOrO=uFwFK%g&Cye(k0xXp~?+TqUej4nue!K2J=Q0;_> z=KMLMH@83j`W9Z*j_R7b{od&}*Oo42$$NP%ts-+$gGF%9;#|8EgK0TDqowOLe_Xz* zk-Va|yf#+hex%#`+SfILjWo--1FAoFqcEZdve_03aTzFoBHx4|DNWl#Bjim>m-k;` z&)jI*kr!7HQ3^{SNFd*!JMzqrJSrGuI`c5yZ79+JE5L3&CVB<$b}WB4U&FK7DNas zJ-3Qgk`w+-`c1adtWmRt?mcO+CY0gz=-a#Mqn#$#tYiJ<6g8CnQDD@y*{B6b8Hd-Q z;=_9n+J6Qo0hcyxMReiwz8L+5=6*#DCcLHzUA|+NKI(gC6c7 zf{$ESiw_CqDmSroM2-nB(cNwYsne=fID2zO1!V8F`R6Q~!|wC9jc_#4!b(;XI?W@s zT$Z?J0?m&q7^3LAEKnz>TXtjl$7j1#_I{B*rRnl}0{nYC`1hKfZEX`}|70|1brFS@ z7(B`XWyp5ld+wTU7aA&bec9Ed?6)!JPnfmIQB5|jXA7xUw%b2d9FG!TmiIInB>k4D z`P+k`=aQD2c}&8E+81~CLkc2R$M1u})mjtGnm}ZFc1^M9uDyC!`_gl@Vyx6iPV_(! zkTDe5dV}-KK=v;71J$!?`(DuB1L?9pL^yG9+(t6*@-tg+UHb;t$X24qjK2f=r2d;8 zj|P{P{vnOAKKy&VIf98k?moUoez3?QY^7tpgX%z8l)Mh%mT4rqD2i6@4umsEgEuHQ zaKk^Pj*R^rYh{m69#4Hb^bUU#%sLTLI5j9^{w58o{+#l9?eK5wdU;rw=8tOaj^ z96>2oj+N-^l&sV*p{?hLOZ~%->^Xk!$gsh4<9m1SRiLo)hgruGMRNn;@Z1JU%}R4i ziHLwXWjbe*Iv+2KW1r;c9!;JQRt%T|-@5APNXe=9ht6ejB4=>jV?Gz%v3Yu<4YnyS5y!=N)aXMv@)=m5XJLE4-iJzK9^xsadw!lzSr&?fgv(phOU8ezS#RA@lWz_t zNUp{RS6=PhScmXT^J~qv(SF#G_-;S#HG41k7$k5+Eo>jV_RZ`dp#mu*OgboA`~Jv( zKEM~k!2&%HwQ=)-MK0)I{<87G$+IwnzpNJ}+H;4FY5Ph_F6~pygHSHgUc5SKe~sy- zz#k^(Vg~q?;5CB@S7ExKG$W*668IqfmdoX{HcWUd#TjmnqG)%9a`c6B&%4bG4$OS; zktaO+QYVLNPf=m|L&UIB2d`6qmX5IzRGlbSp}2U3a7Z;$>s@hiNHr;#g8UpEp+7d# zLykD`$*#}N*rp>q5zIu&ILCzzDmUMWO}b3@n#l&;N*fuHUC(wZz%i6OEz^x{&Il1@ zf!?Z0oW7<{Q@WTZV0=Oe8Rwo2pM}L$1eYeX&;G+l(+ZWTX}rt{5R{&ty5<7AY%V+25J8(LpRg+4x`hGSgq% z;7ZMR_*!AqDhjh&T#EPDR;P4C|fAsnyQ>}YF4=h8^c^L2fD z;9D_xRwb}F_!g$)ReJ}#PoBb8XCXFj@>!s{7Q(LC6U)KBgvMw<5_605=y1})jj5!yj3=Ja@YocBR(t7_jU)^o1b|rC>R1 zwP=70_iOtZ>v?@5Jc?j2=_k7@(ERGCv+LVx`>9I~wU)E*B8*A;A!$nKCEbNj+Ip6G zl1aZ5HdaRptc^#WCUsl25K}6-ic__{7yLe&3nw$HlA1e&9FbsufO5#KEMPGYeDz99 zQ`{w#GHYaSRg+g*^U{9Z^qcA2R3_Duc(G=Y5vW~PqZ!=SMrJ`H+w03S#(4&qW4m5) z1;M2?RC-!m^1U#WY9oDiRji6}UOCuUr^q6Uc!TL+aQYvW`DSzVT!UOCy&8Q(m*

    `Y#!dDx~xI+ySbvjdHm)*x=V~va*ecCi4`R=aDLodE&P|RXJ zkoe(`PTXizqnEf_F;@H}+ux12cpWRto@)40pc&uIr_HLL)GyyfFw9kIh^Ygwc}gmKGX=|1f7E7$x$jLrK;(iP!TU9>4wIE~q7oG<*tt3Sx^u&?$kLH5$Gls!I(91Ao3_?~Nm+n#@^_O)W(qyt)tV2tTNU-#RAA_RmZU(wxUg z@`0^6VEw}6Z>L}*SAh8oljjQ`6GHCX$1WX1+NiGPYW~cn1U7 zIjbMS(=gmafhbPI8~hTaZ87l*2(^hbuuN8HkICatr*2mYI5Mg3cH{*=f1Bvclr{Y2 z?rkk5ycj+rR#{fVsxC_}mj5kFs_;?{Miz43>Rt&O1hqz#tt=uoayJ0#6?*b5H+r`= zFE=J@G=<}|53#S-QH`F*d#V|#s=}9b1DubbQO1=`Ht74>O|d-6xt2o~nr9AwOUskk zMk_T0ms+myq!^E(3|9r3<2V`Rhc4O8-}Kuo)ocka4bC97IE2e{lO7_4iTE9KX2r)0 zW@`s1Omz$|SCqnbG!588tk%*WdpOzv2lh~HR%eT|>sn?iQTe2AsarIq0{b=pb!rBL zi;_?*y7U50pfec+nvtIC1_eiT^Ny-QUf|$4AOiw@#{ZB(RwBj-S|5fUvU+Y#tTl{| z5lYj(>FZ>Lv#T6jXxls~nuEO2pdxR(S*2U8$11-axf#k~lCA*GWZ`o#TmFi^8ASn} z{}w<6%08eWVAgDf&u25$ZY2obgT7R8Y+9ll_8KdJ-)$&Tl~+p@6a`1_;GBM5U}?8k zc_y#mltAF<=>){@|*iV>%!Q*%y^yZidy*}bn0sBq#Lzb7}kGSe)h?@syFT{=MjztWQnNrSYEC!&q}n;y z{9tKidFx{?Sv0Y?({89#ch=@4m0JwXgcrq+-ibzH!!zi%%*y7oP*jW0Szo^Y2z$y< zY4E1!%G%RKzjru_x{p2g*bc$tWk z64{PiABe8aJo1;rj(7xoDJzOC4qj>?rM#GM9ktIF2KxV9(NQg)1?u*VL$_#w6;DF0 z-Y8_tcE*D>O){lZ9-=}V~1V?8)8YRBML>M0!7TVPJ@7^9-AvzExZ~t>Ct4o0gqbV1m0fSg;cYH0e(W=BkSeR;^s%jFf`ePMaBex4bs4|K|3(8i8mI2xXFz zXH@cwl*R|(*!=XujyL>iI-tu9^!mnuP>zAEiqShz-Au29t38c2RJqgS0~ZyW7DOni zi(tO@M$?#gEr46y6D29)&14xolI#4@g{na@!n1K)X)EK?-NKdc-p)gJ@mf}k4@o6W zqSkJ)K^4vqxxZmPcv)&pK)p7o_!u}0w7$yEcS^F(J^ApyoT{gLYiy32hZVR_@6u6G zo^9LYh|4?&^h-QP2X~58XJ-z*jns0qW789GvK)SNuP%qEdZN2YlU|cTKkvYNN#lr+ zs2ly3t$rjX7?Vj0J9(-jWTK;SH?vv1$L?udB>o`2gXVm6CrpvN`zd{|#$~RoB3?wSD0bC+IkdnXmD2G} zB;dqGilZet!`Kp6G1kYpq_85(SaR~R?8*y2YgpUuVh^2;pcD zN(&jE93E*LbGI!F?@H)ZVX_qT(RGdT0Zt2!VB=@KpH{dVbW-8Uwe((qP2iROFkhwO{fz`s)9Il`*ee7GECsF z#{_;_E% zMz^?vY$f)3hA~+p!8*W?SocG4TlGBxmd$?Z$Pjftmasxw1SYkx5HZ-9CLua{wXoyo zVb2;>){i`E7^ZXAwqe2t9o#f5;%;CP%Xl9$V8l~`;kIV8T?+u zeBT@KJjG!*iUEAkK%Ex5-@Xb#W&Bsvo&^ix@P71@c|9QINq0i=qhU0ARh9Bwl)YRS z$r57v_VtMaazZ3V$Vk_EXGab5AfqOQPmPp-5`ky8K2VqzTzVMTvi8Qrt+t)qAHzQ= zFt*ZhrDxNBwX#F@$?VqP#6vW$an%-Nf(E<4T1mmCJzKUIw>0>7A5sUz6%O6$-#GIy zVC$#@0x3iEUJ>F==$B~m0+HbNU>ZtsTw_Ao?Y~fn+c%k1?O^7FgFRLaqh34NR13|3 z#;zJyi1+HQ6c3i{if5BZOYj6c20vPyh7{RtD$QU|^b>{Nf}O)-Wnv3F)7cZKY>NRm zxzLOa<*;08qkk|V3FqbcB;OTY{}8p}Mp2#uTL$a!v z;lZWa!G3dm<%rYOQ5l4lIJhVT_v&BK){H{>y67Km{Rb1t5>gAxUzI${!&U(Ha(H2~ zrC&qMP0^!$FTPC3{_LHUL%n7@!A9o=lAdh)+uSzZJ%$tuCmfZPig^m8S6D&ImFC_Xt=20j`b697aN2+CBwssI3A^`)LS0IW&m-07FPFW zSdG57^*tgWVl8lR?XK9TS~B#L+Q9}L{7@9d?jZo7N;3hxbFjB7acDP()b}oL)#H7# zx3_=_r6jSaVXC!JO*XJ9h}aUaL_-rVH-6PNZ#r`TuRnUu8jKIfmD3$M9>Oupi(cTp z|5Ntsd)KE(>@G8sd{MzaNHOvPsF+HNJnk6+TlOk3>Wt`1E4YK5O?;u*RK-X2Flou9{k|09Q z1N6dhjgDwZz2_5n+N`oAT_mNkD#C(tidjq^4aM0L}5VGFwm6tw2 zDNAL>r{XMhZg#DNl9W@&a7sHewCN=cubpI>jBhvie}uf&!WbsNCy&GdtA%HD_hAccjlMD^=Tc+;g` zBN>%14zqG;D_B+gsllGX3FG&AYgUZs&qCEzS~Ak?U!33}4EVK2U%dg7#^d#2I9r9q!J69w z_$eFQd9W=4;1^tBXRiOSq29ooZ!HbZmI@i-0hqf+RXD&awP_NP7& zN1G*%SPkAKrF5VLU4Ec55boPak0>@mRqMs%x>j>c$EKuG^-yEfd4lDzm)v#>rA7Sg zn|*g8Y0&6xojntGu=g45zrd&V)y_oe5yRz;U$ZIQjr>r|VA`uTaR{j@{PxW}LN(jr zOs7gF8gpHQT+eoW)hU`9Zu+2AEhu*O;qd15RcvHn{b2(g8f1H|a@Bgz9{g_r_eRe> znBvK+S_?%dNuVn~Z^W{LJ?eHVAWDemr%tEq0cxhCFbT!q3bUPY=K_6BZ zhS~oj!Nhmy;Eb=K z6uw%iHR@5mCIYv6doh_>%d5VZ`a$I3>^lGE;E6m;d3Cf~yriT2Bm^+wh|Ict% z3*cb9N3vy#EUJ#Kt0hlq*0Ewqy};@TT?j{VvMuu4ny|-( zg_^aTtL94JD0ke0TjpCeQtm$S&{CsB5RIt90ABtU7nJf7U8P;zU3N#u=Cc`M$3{8v zfq!YaSu8On^(}GQ5j|3+*D@j}N13li7!oiWzu(mse6|8eWp95X^@!%MhEyT_CJR() zcikvYqa-%<@sDKLCqu;9lz#lHo)PVxu{M<73evUfoM`S=X>+s&-?;C_z-Q`0e4n|? zV5M{NEGfKt^t;_?S#9|cFIv{pnTrS5jRlHm9+vQ2gOvC8zT}IqcQ9wySW%BcIcAKV z9k@G*!MDq*0D473X>7Z0Gey^M7H)jM@ zkk=`9f@(gRH>ZRRm$6XZC78C1O;p$odGFxlTEhHArfpvC6roPaSNF@_?o_sC!e`+K zSJ72Z@a@zIRRHsM;&>X`W^W)vg(G6_h5zXqbUWTRgd_gxR~vp;Sju2ir1V;?Xvlrc zy&IX&;Vme{JlR~FXzFZ;7rAfHo0Yy06%>f-?x2UvSe) z?l97Yma`v%4S)CzIM-wDtY&^cRT&JOoM|yaC|jddZ&7S>(OIhp4oN(oCU=|rE3L^7 zG(8Au93{HlAb*sM$yyN~x z?&9E#*qN!f{X-T-i$QF>mG?5n5Tg!^gq-`*Y%C&;l=cDdq2kb67|*$vGO6QPL9%}O z^1ZUFi>hpLPSHWJh0Trgod?q>GT}12(l4V87 z3g_0S(ozcXZI;Y{Cykgj#qpyK8*ytM25_3o*lVT>bswr7OcpWAJM*F%z8$cH}zkJg5xr%(EcfUb-+uvX;0s3SC;? zEn2}y8{PVH-CNnhztX`;x{MWvIv757h zv`mk5elpFY6TpesNmhsQ=o5+uE~>oBQSDdMrPaO%8J5>T9kpvG%A*9Di)1WxbsYH-48NT1tq2bBe!=C-l!8)xuUJuOc!qc_J4orM5yjvYa`S>d>Dp z-gIo9-S)d9I^96d@DU(?Ija_iL`3SRl2lsT`z-X#8j25hUSx;H3?@29y|HFXOXp@i zW6z{2b)>&=DWRY?r@kdEb!Sz@^^Tpg-Ou^~6}6@$=lYmgRla~t!&w5&3#oAIy;#zi zk}eR)MmnuoQSMjtTx#3zdXO=|et9)cFuBj;PYMTeAdc1kgmUW#W%-8_Z4SF8y{jLa`0MVF4|<& z8r6`p{~*paw`p0W2mza}zakgb3mu8nOU=^bUHH*v2sS5#QYIqPba9}iMThWmEj>H>6^Tt-IgQ>K|lA~e?oiGAMWxBz7eu?bZ^FqJGzl&{b! z_Mv$L$r=pBxdQ5Ml#2IkBuw=m8(A|QW$b;l+4hTq3~lz$?7?oKZPGvq7`F!|qW}}W zYwyZ~FcpPJxH74>r0*(Wl(5pfJQJBIS`gs53?*i>;}YHVt%>HLGZ@oaM+ACFn5!VkETLuz5!Ym*J(@{H=P-n!P` z^Mc2z{yy-j)z^V`?JQdq(;&pnfCq%cPWgBBvhWJ!rb`ZiuD?DVwW>#71pZZa=qYWm zQPF~{dZDUij!iWfU=_vE^~s5zJv9gmU;8(B8l51Y(|dBlySagfE8pHQlq25dt4+T^ zU|Ng04=Jf`MI}~Rj>VX3 z79gA4@j|Vm!x3=fax*njTw!tG+>`LD%~I@BoPoxl2MA>85bl>)uc*w2pT;Mn<2Ak#yF=On< zZ36f01oi#T6DM6pitkvfh4Dh9UiSx$7A#$-ZwZa{Z%%nQEU(3oN!ugoac&1{FAFvo zLU|}Bk)zhbjXif`H*_cH@p_oNCr$gEuiq{Qd4=i59{VrovvhIj` zPOHJL)vr4S7)@`BpYO`nHc_x)wESrfGDFPjFNU7u2JZy|n-@F3qzFbT3G|{WJu}8A zK~_c4z`KY@=7#Y}2u&poBOI;d)7ES0m@=Kd5fINXjF5L7SwKS!#1Q?;-dPxVC=dE4 z4cpwgG%h-G5K{60P$)_^ZUReC8NH!5VTMdvr))J&np|k;s4>X6WHr@RF^z3dhzr5oQi*VGfST2QFv*9gqP4 zm2i`fn|ZL-NdT5`3X~wqoz#A+aN7)ofrh^UGRP?ma9Ubof`dXiMY)QrSBd}z*t#EDQ!QHeAbx0v!|0B^gdWq=9hQ={+U{vX&8h@xp4x7yM+;oz%s~2|CnCM{h7q3OPJB2U zzjv-;ZUDWr!{_ie?wsJ#M#Yk=B?cyKJLMldzpb5vrB4B1I{3N$%pOlE9U!7-wLp$B z!P9i&;+W;mi&xk?8LI8rgnscSsi8vt(bGWXGP?vqu(7Tc7>HeqtoDQkQgDXmqU0X} z_Mupa7rG!68=$!@v{a2bgpa6AL9+4dU2=LfE>2l-Zf^K$A{vw;R_c^sIR&;)O{W~) zlyP%|$@F<2C&b(Bmgt|z%SW|+`IjQ82Y00?k>7sqA6ktD+!}<>cFB|aZ$tFG5&tV> zSJw*RC=916kN;6HyIrFPw!5Uw8xyP)<*%2J;h|6K^G&oB9Cm}=TH|yG%N>v2%XPZ4`YHoOUeK)@k4RyPSb`J9~ z*63Kjt`-&u(dQx`&*LF(o|Oo(Ewzro9A_G`vPQBCDaG~w5I>8ZRz9s7X6iYU_6Xf8 zR25TWJaL`U&@58_!+^L447g8UF8w04xk2CY&CS7f0_l-#U_)wHM<8A&%B_7K;d z$g&)>yz6uk?D0E%-NQA^F(pri(R=Vo2vbgQ#_Q0){cWpkc05^^jrdhUPkFC(mhK<2 zhyKnGCjhV}<27po>}>2hbaWRbw z1=J?0aKOdpz>?`C?jIQdu5=4b5Bb7FW5gEis zk2G zFGpgz?Q2ox3cKk;h7-agEp%_xuXpOKPzKJsrb&H=<~GQ@mW4*0-I0Yp_PZnqrFw1o zEOb2i4^t7J!c2o{uh9X%(mnIc*BClTpO&vU+%#|v&_a0QId~uWG%!!hpQ2D7Wa@k) ztk%E@4d1L~@oZ2N>pzlf$@b7#Pw5{&sdY@@{NeEA?>y|0q|J^f`6G>2HzHOn6l}H) znP2>zyIe|b@E*O0rN(}hW0!3fEU>(3i{JzkklV9FJpa%R3$=O#0(Sjt%Y@A*+Sc+L+|{G^(R;Bxz)<>U&BK2Fb|_poX}C;_6Yk z<*FekS4VD8RShuBfX$ixhp}j&3-;k@t!X}rT=e=lL?>5*?+Hi>4NJgnUJ;uTIL?;( z1>fWU?b_qV%jYtpe`zds91ikcNm@*namJV^QY5IZ`1H@N0#wR9j5}WhAQ~ozI?ya` zHF@6q=fbYMW`&x^URD0QuCMg~;_5T4RC2sDO^@aM7p=-R+6o#xzE#gbWQ97_P9_L~ z>T!Q;@qbC6U!eIpWGj-AdH*NplVy-d;bv~)LgW{I7#cgf^t7D_kA(+){hpHq2M&f6 zZ5hr=*HU0%B)zaNpdjMk*NNo>cY2{l&h1f2&)x&?JcA^-7D*5Ow!mENt5KD`9Q?tL z3mG8%7Sqp53^z^rR*CZw&OIs)AoaK;7RlzV5RM>F9q($q0j6eS_CyL(^%&ruoxPUB z0)4wsW7Zk+|6z6J-vUBW43m#`?=ZMEBJ2Iyf?B#E}JS22pcnC3n$ zr76y3nLI4Kf3!tfKE!1wGT~0T1VKcHx@8gU$O1~T<}_;nO}qF_dKFw z6uT~e8L(&y?-13(L?0_Gqgp&htwn9R1QCB|wuJ`3$DI{|_SJlmuzREudfNFknIS|D zLe+iBWXXtZ2s7mdI9LEA;B@t~PU#l-nq=76&8N`JMzi&DGm>L&UNUOCt;K9@zrik}#*9m|_IUt4MmgitCSB zT?o~>6yI+Kn^Yi|=3<;6$sE-iOzM_C!;Y&FbQ1U668eX&P_GokM_Q^NJi>R!Cy&OA zUokEWhJyNGFM;N#0646AOkUFcuv>Y8JOBc}I5j-lDLYXjH^zC{zh7mJ4^TV`O6RGO zslKyx*jOngKpGC#0>1VL+Ra6>VJdS|dF+CBaCiNkazLx>foTw4%cQ04y6(E_yPH%v zEhwyie!0Sj-X081-Jh6hlpvs<%ut+?XOa33`S`X{Efd~j?CGtepvvP_;8n=E=`d^E zQj+nppS5YQH;sKeiW#;Pjd&WNuyDcR)3|kBcN$NMpY>JIUH|RFhr!B^12o1>4>__v zyMbz9l;8Sw8>-qw@@qbatrUEp*5rsfh|W$pS?>ZkZf^Ni8mNHu-@x0OYqofr@fR5t zd&?rZuy|6`eIIB$!VK^N)z)G1@87S3V!bw@Hbc3lI4QPEOTy3=<)`mlv1&F~i3kVz zU1-BW%Mz-b5$IP(!{`xr=o>e`?=IZ+k{{YfV*`Dw`=c@%&U_Vmw)#yv^Mf1v4^yrXk4!<^&ukuI> zAQtqH^U?;|Y5@1)t+y~uv>S)~N)=(OM@skR|cP}{x4j@+j1q=s?ORw-eO#(T=OT+V` zsenIi^C0(&+Q`ZVriW}};aTVU#c`R?M{08)Gj|G&saUqA-TE}^96HLcS_va8j z1--vabfG{NDUpr9*g z+mzI`7#P*n>Pa2Ch1qwUy)qa+^6eA0RFsd3YQbG?+Ern8x47u}u<+~*s_${n#-oBq z6Lq}xxNY=%Z9~kPi}*)2$s*zGVlTk2_x_M)5EeIALpUImJ#7RqkU(=qAIf~X)Oaj; z!Le#G_k?2Z>*GA8yPw}-QM>no6bEx+qDDC=Ad9*IkOy%3w-vm2IchkL68D)So|Xrk z-p?woqthKmr+1V}5Vq6xEHe~|yG*Leh}GeG`NqE&4W8YI7HAd%yMWU1XBT*@t?v(V z*A0&+DSD`s;BOgQQ~J8cZQa_vbwj-WY(}Uk@8s~+$H##99|Q591|-k)J*qU15x_R| zy8Hlnc(>df|D@VQlFOwt;COsXS`SfqZlR;7KzjZs7Yv+h59E^_tD?(?%z09)c!sk5 ze=aimsr_Ay+v4ZUZ?2ydt(5t@E8ssrb{n zDb@*QgGoz2T~8<~0Rtr%4(H1WhAw{JUm$QTBe3^Yn z;Og!F@^GGgcMZq9zI1S(FvyMhebFL3T6TxK(N_X_lfmB-{ zqd?8^2_~SaM4G3gHA~HDZMbgaE!U?spv(mQ-RckGnwO}nm1kQxF6AU9u5O0{kNj8u zlEmHRt;0+lfA1gJcbhUOk5tu#*#ri%$3F!)GZLO=%Ar_k3f=j8d%nLT^F+XLo zbN7)|kWlTgn2kOsAH<;dn$QtVUlFMOgehHv_bfj$T&VfYb_j>NuwLQ@hL^&HY!c4Z zxy0BnGDbkxJi+sn7nK*9*~C7pT8=5z(D8MQq@Z6}4y)*SLoGjGC=lf#P`pYiP1_W~ zh(ydQDOcchhjN1au-9PUQaK_w17qz$0p@Yd7o@wjLl6HSdvE>-b@%>{Yfmbnq_RX~ zDO*!Q*0RJ{qLr)}iDbFQ&KTU4Wg^QcI};U^m~4YED$|Umj3ve{ONJPeZ3ffuJ=FcW z>wfmo^!74T<1F1^|*5Mw9%Iu58P!4{S`@#^CY0Ymzo7mEqL2M_vR{DzCMzH#fi7`5u~ z`BNTeb>qaqw#Y^`PWy+VztTr0h03qbMlN|?$cd-rgS%?w(t5A+`MtjQ?JlyJy~8(q zMcF-mFP-g^DUu$r-rNM@ReoJbyYfL~QG#d9Rpx@`hl(wd&-|ZGDa_<<_TF*$0#J?$ zAkVfy5ED3yo9&^qpwivYCv>H0lA;^A+xDU0nkvVm?S^pv2TPk@yFt-n57Hk}^#U?? zPL5w$)=pg&JbTXgBgXRRmx{C&>gZzbhNe?cT=+fcwPxEyrkIe5IJRLczi0Zm{?2~f zK;Y}EZ)bf^YFw?Kd+;P`CQ*}N;(yKsYRJG_o~%1j;|=Nk>msjF%udUz$>GN*Y(m657eIP?vZ@U3 zx$i&`5PP8>ABzhFmFLgL_n(w&xuLsSNRORc`(nhtE>Cl>EvrBu(9TI4%F^h)RCIWMt zO5E~cJR)0PoR4(`o8EYceN0zKxYi&@oT>2^W*-@8_QWVy@aZz{1?t0LZ@6Fn4S{9N zn@;rua084%#GT2n3y$b~Fq_J!dzPXZyBk1j2rvPzv-p$KnVTmX7P|S>dl_c&0TS*@ zJtWgtdAygv!I#kj4!%{W{GCl+l4A?85kum!AOfGwuF-`eyu-9rOuf@ za#_?`cNkf6J$6gxHl0X_RZn>{lq3)G3{jcFvt9N^87V_EHt0ajbG3WeaaP2|&lQQ5 z1{cQX3PvR&+R)7uJZ%Ntf7AWVE{q{Y(rgKXl?N*ge7ee?IjA`qt!l%LU|e2;tIj@lN)#)$n3Gx#MX7WoI1C=;Q6bB~6CbE_rA} zkM0<53LUzuQ37!URz!q*g7L{V)g)0 zp+OS0Xbps2OsiV_r>7Tj@p*r4Qk#nIaI7#uc#XDD%Iv}j4 zk-SixAz+>W3Xn5^6}kzAt`7+=#6=;r011-!LJ9NBa{wpK;wP-gXq)b^*2wu`VI_^W zI=`Buq#AzhdRvl6+AVM`VGCmJUx=ZF#2;{eG2@uyJP~h=X#3P6^xYy2AX&{`(?R*{ zMyU26atd*>`@-<1OBon=fqds~Y7Z%MdoUg$ zzyb$sS;Gv;nQ{bU<92>2Xl)!K^5mNQG{r*!50M z=)~;-(_hoT8g!E6rRGoZNHq<-t?;k@cw*X?*J?8wdhXJ^0XHneHjlhQhEU6PE3}8qGCDgQzLa?1P@q0~)?+ zw^Hwqc-Qj_Gw1m&j9lta)o{ttwddvsn}RtN+Tbg&dXa?F(1X9wY}QKMNA(ByVNKVp zBV-~loDuty03>F7<;UL)*6f?#UKiMq{G$)JUHsi~xLM$V{bb5_Bw&hW(ERehPyPUi zen~NhR6MNLIVd_MdyWa>cEAYv2Xg{-e}RmEYJ5!fu%t05=0FKRfq@i@@y>igx87YR z+kVd_1%PQ(WCIbFS6&d07uX(H9y{f=RW=Z95`5fz6utqaj`<^n=D)yH1t4EQ%X?&SKa>s zh)FwFy!rY45RH$MWPUO@n0YNJDty@uThkvd^En7?HqCR2E)NKozzMjA6Y2(TJSMvG z<@*0~1q&e$8EV0NVrL~a*P@nS^~orIxEpZR);?irQVkfDV2i&z0R|L$itLEekEsK% zs(dO|J6@$*%>4@v#x)HB8<>|gRt8plG~xXRc>)M+t@B5tpNcL22dTz1adX3Ju)s!7 z(2BajJH6>p$k<^(gn7o-TLg88|4$3Zb;dODMuRnBltuz?0M;r}aUu}JJRr&QBedY! z*XUqrAK3`>dDJZ@*7*QKTm`&??4Q-04F-L z0VjqU-5wbk_yyBVsMU588ACf6xYy6d?1I zt&?;SX5Ut_=7%KP*!+d1VkOnuCDxw=Dp@@0IQV+sZE&6gG`K`?7_tIiyn98;*oWI|o~MsH_`N@mzYK*e z^#LUZkb+m%$#t&zyQF41m}X9>;^0J8#r4pCS%nE6k~9Vx9eB{89C#&AyV+gZB>+SU z_(6lylN1p7`*(2T!Q)D@KbfCTcuMPBA3Hdo(fso6Z`wQi2(Stu1{#%aH@^l!SKcnn z9smFxv&%C8?tAkaxk6Q!&=T`Z2oRuk4)bIs{WY*FjLO zg{01pWC464(Y3xB8q^#8Z3g7!=~rrUE;W<5+K^k_)UTj5*|qgy;thkG!y#t|$}Q}i zAx*V2zlVk|*F-YTx=VD>sfLy!t-*V{^GcWo&id5xy zJdB&KW0&j`yRqbAAF3tsn1XYTPBv)G`MHKxNGl?oXpb2m&$?Bqu~GmC17OYS=D&`R z9^7_{%m{kexw%o7C#G#%C?58ysXg*(g4RkBHSQV3W6EA?QsogDDgul6*9~jQBjxsU=9oT%u-Ay6XteV z7X|ke56mtO`YoL=l`_{-+xrlQ(|TTKvb6k7o#1pRD9*Te&N6U{m%V5(!4W$XXc$`E z%j~YSaxt-1?xSMkn2L$M$cB;CE+=v;=BGRyGt6;}R}`Op6^gmOw1o6b)fhM)L&$qa`OiHjiN(5-g$ZiEA2Vi&c{s8a>pT znI@sLmoRziE%(nXE%9KKL*mJPiHmPpT?%TE4?;CF4@EjS156*3w~kdP;ac>*ixm|O zEK3#u8hfQ+Z~q-9JDH)!v#y6>FC}0f-`#4!X#2c35ByfQlKt=k~X+1VRO1q6;kOW?;oTbM8_>pJ_-YvH)j zA4fDRQw3LZwWa(Wl(GTHkU00JNq|B)zma4OU}*~snl?NlYx_9f)9O!5`e~7_!1%?e z=S&{<8RTzx=Y}z z8C3=S(Liy(b?Wi#qV&oqK3tQHknb(%W3PrSn;ehW*t8@I{u;xS|FFtEFox;v4Ub?E zfz}qw6NBQ*vxHJX^X}0*fA4xf@4Qs=qimk@%w_H38tkhbI=l7fa1djBZdq8PB*(`m z_8u+2*h<7|hoN4mXhMFJxc{ZBv&QRO3rjnUj!*Z37)!MqopCw@a9GcsCbcn4WH&S% z2aV0nT@3gkYkXn03NS-X_kX|A5bs+IFCd)0&tVv$1F1=Jm5En3V4Ohg@d3x*(NMg2 zSHVU|;zwlBn;#x4{Ve1X4fJbKdFu3QCI{EttATzYj8bZzZhOlyV41}TWGM~o(^FwY z7A+s;?_l%qjt9fEDqr(DRImY(*oSH*GDpTrx5m2x+*51)`}=EM>opF@{=Vty!h%oafoy(pVEn#!8>ujb#z_g;*^)QWzD}LT#m@?h=cQq87>5uY9ezH2Gt9%0#p?OTV(oDomX%Nk z*PptKe`$yr*Qy@rb#&4ss8nw`qn#_ZY5vx-()xctHzz<}>PoHwd6()95>Qk+6*t(- zPiN0hv|H6SI=U;hiK%~mc*#!_bcX4jaLmi}v)aZnM{i6NX&z&N+c{ztrDX|fL3o;{$6Vb+AD##qusEO?91sD4q4*{L@{#KyMcDh%?Elc?UNWRr zTx1QVeD`V00rC2@?Z&iflSe*TK zC7~O`o3=ASRz2=G0fLq?RSL|BV<$>JBjlX4#+=K!n+$RIFWLCt8)L><6Bfv=5Xeer zzWM`27~E}GFA3z4%UB(S@uN&g{R4^rWwhZQGC?l~H4tZMC8DPdl6>dPWUQuT zwZF3at08DkV@2nd-46{JXgF?C)CV-p!nsV{Qh{Eq?qOOU$5TpNS z^bNo92sPz~1n=1v5Jk1Tf7hpK9$Mg}ZO~L4_((bMxo6I0Dc&uAnv%9a7(YjAbTpes z64r*B{R}~eZfrG51=GeDex_ad!|=?OzXJE|{UD~VYLrt(%u(7Erh04kI&d17yDv%u zMi)U~!$ukRx^d@4U2(G}lCYFNDH7!&fJTG-Rn-H->gYp|t2r;Yh>%k&rI}o%u_lY% z%F__oi^B1bNx=!;z5T|Rz6C7NH;%pP-3kNb}i+dkZUmnrD(^$Or>h#lJYXd-wpS?G!r z?(8Wo>BG{iquFXc`uGN2$%GjWfW}SQo(`FbbYmB|mE?HP#Av_)Q# z_n$lwZBi#_ZL>rDuFQwa&y(b@SIZV%xaS^nVmkBCixm1nIg+0M~A$Co>JzI}V>17x9O9wr3!RKq#kRb<3Jo_9y zC(VU#S)?kw&*%N!kl!e;Mey*cj8@hNp7F7KVQL&aFlPcfv*gd5-r(?*^q7g+i!pae zmW7GK;%Z0P*e9_ld3%F~M;Vj#MOQVK$DbTxj<-5WR$BQuaqLK=cBZwB1pkDEvAu}d zmLW;y$|Uit-|NHLaX#z7>vlHHpHDCafP!OpnaI!^ zYOz}7H*dwsOClTu1MUwLWZn7N|Dyi zd(R$VrCf>`X`=SeH!Ncyf2viOoFCL5)EU}iqY1^y045G6*i~2UJ{q@c! zLU~j12igI=`e22BbO5cFTrn3ozN~HZamF1N$KV^*XyRmqy&N(XeF@_iA&Iw-W{XGO ztCvOakHqi2#sWWg70@*i-SIn#NB@dIB#uh&tH4@Mg{W-}H&@XkLe1rR>v+*8^?1Ln)0?J@xTKe7!tcRbr%2qktLkUtD z|AOwT+Pk)^opO^P4{3M=7{?pw8cGcoAx2X}#u&Ngrrp1j;YOan@OVFW_{zJ1#tQC5 z^-W0@MtqKQ+bJ+cS4*H$XJN}fT&<@rr^&p)L(rbHEoy|K^2yn@KlY7xpsYo&GrZ?L zcny0~l5|m2Yp+L%XSkO1W7n<*-Agcxjudafe75Y5#K><~umf{jf42hh-G@k~8H(0l zLe$cx>J%lxZ+CmNSLt8{%&!3BZMWfCL1LuQ#^)tq$NA{~r*1o&Omn}gQ++3w;w_N~ z5rpDPdUvkRdsodj#E-=pwZC2Z#dRa(#;_WtRv5@igDy(o8Afiy??$?~;!_2BPGQS5 z?L-xJWkXYE{MJjI`N{gF^)O^f9!7k(B4E;{nOW^ zUQX#w%PgV|X#lkt{PK3Hz2P^T$fO z+HVAl&%uQ~%WJio)XZSPhrB{}nKd150KP%$!oT0%n)4*2|4l3UpAmxJkQJY@W8oY z-bD2w93eU7;H*uJ#JRjZlM3R{OT*5IoN0`gw)Ep;iIC+o&{V}eA2Dk#a=&iXX5pB{ zfA)!t#F%s!YIZiWp=f#w!MFEb7ps_GTgZR>#p63<2ar?ODo7dunGLc3lo%NV zNC#KhRx^-%nD|2sHpTBSNb9DL8461jr4rxptECRqF)Q5ZZ?Is{P|y0j_44=lF7x3V z=x6Pn;L1#Gszb$Z)%idm!mpO%>yBj;`I!J_t_M8HD_ z>X3zgKz*xcfA@sWDBU*NsWNT5^?byl<*-De{l%I{BK81tDTdfdSg=#66j*tgM;yGx zZUZ>Exdk+=z>{p^0)K6;dtP{Ys??9atkz6nM@#mhHBH@e6c#=PtI0V@p0P&$pVjYp zQhudeX&8~Lrj2OD8KQi4slJK*azjhy#1Fu8MN;yM37*( ztlE1k4mGl?>D_-k7A5QiezYh*{gEqSyJz4Z#f0G~# zF}C*8e`W$E@<>4wCSm_ABV>7^$@sm4A0`np%UJ~t3yLsWieT&p)HPr}s=GDu;^PX#M zSou>cpiv(&q&8r6g_E2>tTPm;RTGC{0@Y0PbAhu0;ZBUa^ZP;S`>P|Z7vZq9jn7Ta zt~qwvlOxiZK5h84u=smK>IMG&&x|mJ!LoyKNi*-ud4-(SHDNkA|2Z)l0duS*0~31(RONcJr2Aw#`^g@on6RbU_|qqQ4pQrkF!e5#!bMwy74V7?J!6ai;_ zdO~wd#8_-D_E}t}SL4yDtiZ8U>suT31SGmKkY(X{DC+#qYyUYOZ`3oyd@dJw(77ND zaXXRYQJ61R1JTcQbW$gj+xJ`KI(BoP)s==#6>;=n3hex36 zrC}HAXWAoQ7^<&G{D-1km+_y4K3_=OdlnrHU`j9D2W~QCm!jq$oa5TpoSgFfHhG9C zdeSc?@=c{^JAKO0w;Ebd1Fhl21(Mqg)vq?hQDpz{3)j!3fBy08dl)f6N%xw~5rp`s zfBx&cwLY8|od{t*w8TgB4ucZ0E8jJ)##>&>3AyKfwYXRAVy<`AVQ9?sPmpsE9n+K? zGg9RrG_01Ii6eUQZ<74hc|HBIG2e&+`%RXR7e}9tJX79XYC<#lmBj}VU;yf!07=_e zSh0xU*?odkb7({WZ6KLcScE@%j_*k{@iV`6T-B6!pu*^KkaV*(|J=@>z!+ebXHq|} zLEB~#$F6m@i{~zPUT5!p5{0-&jN#Yjm51Aan{whC@y=RXc-YW-QPSGV&KwZA-beI1 zk|yy44#4FM^8gTKb3%pL@Rac6w|c>4|5!l->r36b23OzaA zHs~ zx?>-?IQKaefc&h13jYl*vp;U53kkl|_zL@B`$08;lK-h_l}R%7z>3ejK}a84y)tYz z(rlVQU!om@e?IAf_#!*fd^E>?HNN6f&e@(fcO?hPCpqp|?0&O-6%3owC`2V9T2C7w zU+x*@Lytf@q219!;nmXvbB7Mkq#|eoi=6~1UYHvMN&=`&#O{A5{iJS3jr|v`t4RPS zR~Oy?@eaCHB}oY0_tb|a*LyqRiy^ufGX2QKv2S$Vi{AqFL#7CnH+^hD=PoF0H7&f| zsw*(#X>8I|wEB*}L~_shd~77xbFqOj+AbgaHyvx9FEup!FK`%-Bt9Tnk` zoR4>M?hdpfULtr7zF{~(%QSz{{jkHl5sS>zM|MG9uSF7ymsRIO~2&c-ij4rZA zm1UCe{P_p6o07saQ>>*`wWZ%P?>yPzD@oHjLhf*TscacxT^ zi3rNrtL^QG-!nP!Lon^5$0LVpg%Y~5i6BwK`_@igOF?|pjLTcCJ9w)4&gR`kgA~a( zPc@VBqb}4F?mpcQ^%32j>3M^D(2#7zBl4au3x0T0q?NiekyC`3` zXU09T?!kk+uJ9>!!yrE0iKks`c1VatP8ov+1Q@W}V^EQZ2eo?eQr@tScIDVj(#c;e z4bo-@7A{PIT5g&F|H;(Gg#meg?Acq|ch@HISSnKrS{?>!j#RZq->7ZV1xZ%mNHyYt zRzKo~qu56C^+rbRYr}s6peZ*7&jM#u&%jlL_W*>o*H&X4cN$GIcuRFkhkjHD{c$owOCU2)Fb+ zip&6+R#(8#r$%9ebDG)Jh-%qt=w- zgNqo~S61{30C}*s#zd9}W_hO(`yVpnUZJkR;aRyTLpNhSK5F#VK{( zSUJmwue~rlq|qz?Q;Ddx z|AZr5fr#?;M6C0Y=3Wz|uk&Npe2evZ4KW(&i%=w`I`N+d- z{50nqkzw~aJ6j5_ORqL;STV8k`Pcp6U+36duso2DkvURwBH4+~xbqCIiUlQ>W$c}f zFm@%svyLAEz3YMX;{q`_2QCPGuT}flQO})X((4Hq6g_eLd{dWSGmVqyzscihkxTge zvy0XK7N@3dUx&|^=`5QZ?Ww*vAP_#EsbiNHoTaOUg?^Y`1FGdRhl)O2JeT8?j=$`O7Q^}F4}ERe$iTAz^C(AqMW`|{2nET)#j^?pJDM?zo#m^6ezu%U3!EY2 zMMY0_+n_)=Aq!D`K3psE^}qB=5G>=~V7fS7y+=LMK@?UFdv?(EDEt!kN~as*D; z;}PDjkhp?mvZ8h`Fww+Qq)hKyN7$9@EIPy5xyz1?+kMMFLX(DIIxEITw`hHP624XvZHZCKKdK#FkOT1s$ROpC+SPe~tdze>Q7zK1q?Q^| zgsod@tKD_8hcmthO`mdRcrb%ki)zzP+KLb!a5yt%?#Y5o|NDa2b3J}-8$X;l9m3Gw zHw>H<@=+Gjg56!m^o-5&TW#t^WgMAFL@>1ENXTP_%D8!!UN+7-8+XZuB2INSDPjYh zRDefPT^EBaOIf*_2V;WF=a`W0m6QARcn+zVy%+O4WcreGyB6P!_;?ZIa*)3rOVv&xBcZiC;loV2k$3-fc?9!U#c zlFYKz$wxzGLMlVa=OeEybuya{=cm4pOzin6>X-?t;D3B3jumXtpaw@K?qL5|?xnQm z>6X$&Hn1&8AxT=UR8tE|2>zQ^QqxGKoMWi}EuDa-Fke5AXBat=RE}M+y63U?HT^8} zB+4dskPVWfjTQzpTMFCNnzTA)+x{V&FwU8PyOcp0!V1%EJ`&cil+XZ4S%j{QxIo&_ zNi4C0;pW_4T@e6|L7o9bI5nXnVK7+L zSfM}jSY3Bm27WJFSHg9JOScC0_EE6q;qwIjeWU940f>!SZ>1si<8ibT_4uKPK)3vG z71PneEXSHecXcIROHp9OFEeawdKaFg3^q9oUP6=wKC6`;4$z!KWex}L?$&a=-(S9d z7wg1-6Yj}ncwYwXQ!{HJaIB3u3qq+eAuO zK}amGlo3+**rS!KsF8(_G~fH|)0P&|*f~>BrP_BoYF?EsI8!M&`(uYYh3A2~o~4io9Y6(s z*YH_Y9_#W%3=Wu| z=&&ND$jw9(t;L6RtKwwl_$F5DDJ8pnHZFDZbq@Ar4XzwW0jVx1*plPsm& z1CLzl2T)uPyTtK5;z?28HAItA5}Xf>SJntQfXo_i4T-wqrvOPhC^BBK0JUNIF$DhAw2$6l0S8mR zDZs}qSFrnMIQ|catU!-^8dLY)f_mzgN*)YIaepO&c!7|1%Q{vR@bRBfJ^(!l%rsgM z{zs&_!Ysl2L1Dy=?g8-rqxvyHB-5vMSjU=kxo@I{$X(<^f)gD7JN>Ko1KlPwdmF{6 zj9XNDsk&lrslJZYwN!dxI16F&IN7&pw%ROQzGA znO?7`^$+u+_?S!kda|s0$!5pe*q+i=tD=E!RR#Tci~5B!!Rs!5iYKx?di`!MFKBce9QFOp!yHg_g=Y7wM9P)(qV_s7fW0^J}l#AX5Ie4Qxa(Cz@={Q=^ z3k1*hvzR_0%_mkfEh`cb(lCN_6W$b}^(X}c+KrS_YVDIy!7?-Vj)w3Zq2A7? z%qfqY=NEZesY!aLA7J*OW6g%h%rD4|{;oY-Tsv}M3N_2iuo1zdKHHNYOr6LK#^)2+ zq)aZqWMTG;QVG%-w6^5#hnAq`oc3+mIKnO60m#KSAx?A;c|LZp2zliIK-u}^#&!bW zL$@f1f-nxvD$6*9w%-Ej_i*vz@FsSaWxrM-x&L!klOjLz>ypjxQywQ4Owp)dwkL3` zH~Lfv*T7M80JL0KUM;lYHr}+1>k2uG`U3dv)rX7xV!bN{D+J_O$9Z-H1&PuQunNOE zNtp&;CQsD*Q_A+DpVfZx@G^UeAW%uyhO4xBbq2cg}Tj_7>Y^ps&~5SyHIVlDfH2$Stp0 zx~p3`twQ7`#h`bHr%F*|OdrR^+Hok&a`RpisG|2r_YJtFCwKwBlG-)NKI8$ps>ZO~gW3>bf(k*6@ z60y{h7V3)@+8*Y~+%D8wj~ei)x6akelxkKwT3Z5|XPiG_D&gURHFHj^5=ul5pt3Dw zC*u`^X!;LFr!F$P4vkh;@1-anlOJ$%?~`p<&}=(A#+m$nM%_^OdY|mOCw-=`pQ0X= z9xg9x@Z2t3c_={wgp9-_Pg+{B!X4`sY~N4*v9(pIKo4 z{Otez{r@QN|0e|wi>4d9T^e@mOzyf0zq`4V;sp{~3X_G7B3==1^831oeHZ0o{0f~t zt{6yOtRj+a%-$yiynf%C`+_KpES!UclnjQ7Ur1(dU~Krl@=Mz918~Ee(LMMjiySPm zjaeiDQESsGvbqXCg1%ykh}I2np`vyzZ|A?m;$2_&6##wPCjq0@MZvs+d#175yr3$Hu4^nZW9BLvqVDsne;VtcQc zTc6&AZD>KMRQ(u#uriZjIe36TEfvn^c#>^1uMlIpql;1k5+pv?I|~4uZnm@B&jsnU z5c?;YNZetBNC1xHUxzFBKg zl4X5{76Kq}i*gZ>Ti^6V;Clt7`l`2lFnPVNEh+7-?RmKS$Ut@u?Ky1GG`Q$o{oHeB zR#_5pa}{nAMETPmUtN4$mc0Rb4ss4=13^4S$lH-LEq&G~P zq@9%^wpA<3;++May`T?5s`BNWDg@&m4?A$8CQB9soM_0W#Or7WrjYtrSF(b(0sJZF zjYSkynHo5*p^+)MwI`%7v9COOk)SYo65oYgN5$phiaI0U&*);D`uiDoTcobXEV7w@66ZkL}{9b@GSDo!`grcJh3az2LgSS|@)5@1vA1&CH^@r)dU&zG_f@_BI* zrX(X{W=4Cj?2bJej=UC8TncLh3=K1D&Mx#-8BBnc(G zAX1cHvS4(>H0jpg4q?PRza5*gBJAMqU%nYtU*}ElAK$^RyU~yGZd=}RDGJ1PEHKHasZlTnD4@sQhezOO) z%cwVb|AKd^nSF1eSvTz^cmG$E`?Tc2xMMSuDNW~?)r9*Kz62!$vHXV;=cLgTy0Xzz-sUf+ypckXJNB2@(=>m!L8Lc9BXPxRSiyrTq_e^nlb3DdHrl^RR?HBrHT9<(D zCRcpei`o^3=OFaiOHgoH%p+bEr#MDwW?4kNp{!_K>(&Ytd1)?P#@g|u-K{EBXt3nj zvdK>xpV3RJVAr!sq^eKNcpF8z+aI5<=FX}CtrbWsH{W%1bz|~^uFNP^P45&4?!Y_q zjz}#liBq-7Z`4X#dHn_*K>sKL?V!pFK|O)Mw4LoYEbfN+0yD5TUmgIVIp#Eg8a?R; z)(>0N<#&H7Ca?yOD({hySSmF6$j3Xf6(cSE(~M_fUjZIg*QdgK?!gdnDLMN6coijR zi}0LY$>AB>^0$=tsx)F(7ZZjuQ%G>C>7Al=oCi<_rP$VG(MILc#H@0E>5d2D>|{~v zXfgZLE&jPq%YEOc!`e2^3^9YtzVA^A?l@evLKmlu9O=EfoZ5b(Y1!&>z&^D#Or6L5+HU~_PnooV5-yr#`B_*Al zeY>89ql_Teu2t9}lnOL{m(CQ+1u43Jn_L`V?<@Reu&fqP{a?Pl=z;$}|73nbY~@ZR zp6L*@F0!gtY-3V#6=+L|cdhe%R!#a-N4~PkPhi zY0vbxm=>?P7E{EA81_NZ18S$4$ZA^)cC)tlfW4YDNzt#rOeZUnnINef5>a!#d_t%W ziU0gRBO(J@68XsNGHr89aZ*>DnGnN6-1tV|RRgAc&6lJ@JMN`4EADxip-@VFJ0@sr5)!3T!5}@8%nfD$ZRW|5$ZDV` zsjQ<*_R|uiK&dR)$2#GX1QhS2bJsFdm;#WwTa1z;V{F0&P!H^iv5MAt^6jIQyU9w_ zy>?+Uhg`Ab0dY1}O`Q4SLtE{XGDruk`Ftw!LRBCwA$I2lloI;!=k|5 z4Iav4hEGFF4Q3f5xCl$Ex*%=d3>pr&unH{X z6!{%$?Ysi0JDWgI4v45GU{F+>mONd#&S+6>%qzX0*_E=K-<@Gpk0*r64Aw-S}7Ul-QVWH zHjHqpO4*XQ-QZDvRBOi#v)FB?>o-fk2JJN0<_wdCyFGMFhgZi|23S1tkA6|B03XbM zLvcDOrwBR(pAi_e6#{c*kh%T38=#?rLH^(}<;1t}J{DJ%r)TUQ%?QE6jN>!C?Ki~U z9~)fxx@!nPxgla~L$80NB@C4NDPAA^y7)iq04_Z|Q{Z5lMEK|o+-rF z9?o6qsh?(~hkXTILi?^$Ppjn84*HldcC5DQFUrLr&+iNk1r57YMg%vAmXoa|BT%;; zjH5B^ZVDY~IxRjnJ7Le80*rr+?hh@QQSX*RMvQwKnYOW%^;<|TF1@|r2#mhr>yj&e zD{k@;*c?{pR%$`ikmQy)rRjcNhq|g0n(ZB( zAfkM~BPr|YKIp-;DRA`&NG$AHF0%hp=)yE1mG5P}(MFar&oo9cN6B_ttQF6U2FB{6jvB9o?Chwp^-`?ea1T5>NDYXZ48iSEjOw&!wUb#3Fz+5#s*-CC? z8RQOS{Sx7ht3(;nWJaGN@#Hdg#nb4(h#K;@ZQQ2+tiW{HU8$AeP2v~SwE1bkZcYIlV`eebp!1bG6ZKNEK%h1_Dz9_`(9`@Yx%>Uie=5`cp5zj z!WE|b%O>M?TWXu|IleyXR-elIAf3ww+_Lj+nNqRft5c`-JG!y{m|(<^WC1nG!G56s zq`}rTO{zQ8=DwQ3j*?0%J2Pw6bDL(`7;GK0&TN-fmMyycf%*%U8IvZd241{3Cz-2162oBwJ2VQW5J)A&Vw&_>Iy;3YQ}zhN@sC|uZ?dRjWp+tvo2 zMcj5EDbtD7s~E=tOcuV)>enXu#XVWuMdmMCgIj-`qtM+c1n$^h+Cg8-1$7B3@8(kG zUz5M_In*LR>3s2I#y)ScnX$L3r|Z;sdz?hz%W`s8`9{2|7@bU-0G)DI#|;^0%c_ni zyE?f(i5jY5n(}n{S856TG^u;e0^V7qv}~qXqA8GXW0oq`Z7(Zro#CRip3ff0&2><=M9-nno{%i%>DQ{=g;GEcEjyPe8YwvP*ou(%_s0C~K)pugpxRF&|7J zl`O=cKGSUe#K%OvK<7nebF=K-*@a?`Vu2qv??2y>>Q`gz2n-e2@*9}fm?wQclsX+c zG4w8BgwNCw}2oLZ#*RcLVXj=Mb(~9>3&2V?!e7CQ)TPk7qC8Bu^ zJ;?u@s+nRcE1R1z++$q4@Zhz5CQFx`zN$_R8ZbYd2zOnVQ8(j#z_1@IxxTR}Q3GNtpjS6Vm zsLeB?0MBqNrZ{-Lu?@NQcJ`A_bzXa*clhPoe7g3G(hVxRJ_)kA4M0)jpAcX3WX=Pgc!gdZSM{AcCB{u;x%~2h%}p8PN!K~ zMDA_RZ_!o(=oh6j%xpyDnI75Ex!bPx$vOj@={Ek44}0G4g}w)U zr_HNCCj?$ojc@nL&Z6r`q?&J6SxEmV&iU7qAr5z>gRiiSf{)I;BvwiJgY7zacWvqx z$9ZStW3HHGHFW=`B-}05T)EUeStQ%{n#;=caE=Z8B;<7n~uOFs2ys@8=yGgOu79R$fI-W3v@2+XDBtu!d00kO|)y1 znrA}}FxbKNN60fzf=Nkp|M;}0ALd78&o}m#Pbmlazx}|E1bCt?y2E}Wu|+tGqmwsC z(f7LYq_EXPIG^6$DS{6+$e>|{zn$)7Dl=(0&ztg>{?0Nq(bx|u@_x(+y(varI(&N7 z68Xb^O3eb?F=C_b0Gz$m_-L{0-Y(_`mnPL{p>K$i>%we~Bsizs?#uSU8(WvB=*Otq zlY=wh_DD}cq!vFt2Kki${B`cMsFk|HxhS+BEh2VqmC%3HzV4E{`t73pDn{U-OJ8=N zpUR9N^fJ%LK*ErwfxErPn4thOsQm3;=*pY#^uEZF{xa#0Y}TB)OLPqk5&SkIMdL%Y zF*};3Z=b1mw0EP{9KdgL%-An_@Xd z2x%48v}jA!nH6Y5hJY?ZrZuNi>_Ao?vwee9&tDD~v>pGoQ9JU`R;E*~Gue3v+^C&e z1n1kns-S0q+9T%+dzHrKI>kYIxQv^9-0-sa_T!`Yw3ta(O2M^_JtqpidoP2swG~p$ z5y!F}syH1xvX0!*);_rumX5%Rt;uw@*ZVf{=gij(sOfjc|F?)`*JIwxTsZXb`e z!|ri$+cvyi@&5AS-spx^vmZPc4)MA8=)B*tz1g1g^B3EEk3W%~1FxwV7Cg$>bai&j z4(qdxQWMY1DncqSh6bYxmf9s5x_hM;vjSxt;EQF2T-1%P6@D*uN|Na1*EOC@q!5V~EuushHKGI&Le$Yhq6DKyCsBhS6VZDQf`}fyjXFqlqIbiH z7G;dl#$XulCBOT5kN0?v-1o=-hyQVZ$OL1q>zva*_g;IgwI@_dL+J*|ZITNYF5FO2 zeyV-p!j=0ME?gSFb_Mte*R5_{;J=Hm+DdX4N(b(&1K$w8RyJ_GaDkTc{NF|E%n{%v zvKLgIK6(E3;^y4d&(FuPtve48v^lCT8bI2&E^*sEe_BCB9bsec{N>p*9*>JYPoCYs z9m-L$?5r6v{4}J3mz(@jFRMbs`x`lJm5qBct7zm78j&)cV&yqKeF`63z3*yLk88qg zB>DVC^8N_A0J_HV{=X0Z-Gl$e!GDjze^bGKGvR+T;eRvX|EHO7%nPcOP7B~7qKyx- zKEB=3?q}_|a)V>S^Et9Hc;c1`>ah~81=VbfvxMFB-nrO-8=ga!4bROS2X7g=TgwA4 zB$xd+4_`@UzbftdT1QRdirmRigpw^@Fw)Q~X)E1L(-NUB=&kM9IC#+CFgOV>$Z1kF zi5^bB@gFWn9&*oAg%m8jyy2I=B__EDnM15?!Saq@ec19y=U^Rii>}3v$A`PsqL_#0 zdg&Vw6Z*J#H&5cos!bM#{}fi_w_dA6@OYaSZi!)aiqUta%|b)(RtO9&iGkV_ogIcx z)#fZ5vDo&9bLhnXxIk{UUvYGM`w=CP6wU~+=8T(GS!aEBTd7#bLdd>B#P!DYH-;ul*XR1uT<$4S-%d4jL79}iK!<0ZEDch`7znagp_J$3g2lk+S*N2SL_o;Mwte+ljD= z`j>_|_y2m~jRN4plE;Co6%?{xet?qNr_^z})TK}cioXUf%Uxj2#P{D&03kz8QO+LZ zcq!cEkoNC^`AzjbyW)!;Z9&krF;CDi&^ znjXg=P!7=f4A&^ zq7NzzO0v0zLT3L=S-Z-2_Zj)=^TKSgRP>d(meIKl944*iBy|h@?{Wf^QkN z&2?ZgTKSh})<$s?@cOMPw=+4M+<(6H-#7p7d;Hg<^8ddJKxeqfZ_D_-OT|>ECZa!| zz3aipG&X;*@@zarO$6l52$VVCjxmB>nXerL;rrSRH47g?-?T{a6JWjJ{BMz4*m;xP z$0Hs&6v)3~ju;S#o`;<6E)b|;*Av`wtk+QJN3{u;&in zD?9SsyCOvC(!`weAa`i4|A(yZZrmD2A`$j0^2!j7o{Ceg0Mk1$TZRsqy&} zsyC^eyMA8E5`+FERmON!uN%y3Xx!}5JySVAh}(lqQ&Q_rG>mLE!Cy|7?sesL^|fzA ztCtB<7eP$!i8W4?Rx~(X?|fxtpM6-I5H@bU+x5HGyJVG)U$>|%N5Gy}r!4g0O$M=_ zy#wEkJ9+Np0m7K?}7qLU1(W^k?4!?2=sm6gIn@P90d3;RNz#41tq|ZAr)5=_7 zDmnqPSes&Rqi zDc7jR=pO7#c6g}XU-Rrg$5HDdnDQ_l^gRk^s?9RBQ#zTfO1Xx$%8RL#1eb%=`*OOP zDl^JngW6_XO}b5szfXN&6Dlqq+{$YVOLM58JMBp{H9!bFm`j`=pV{fU-7);&_27^H zw1LJfmg(zZ@2bkX zYb-Ghpd{rXcz4E^9C;bn!|esbVh2VvuU@GdvxLjCF0Y}sDG>iK@Ox`ju zFsZC=l8+9~PQ<>3Z6UxLB#B0t#L>}F`M|(H-4@z#WM;E zgP5xFvODb@r)wI}LzCF)>FIgG;ZpPi!eN(8DU7JI(yGs;GlJg2$;`dSX0E|KNix;& z;o9x~XJx!2_@*hW^xw}G&s|wcHjZXBqEOk5>qJsBHS*F$W^F-lsOZ$w#BM~>-8aXt z4QESzYYl858y+B(9;fCJUnEnA2O*0*P7FtWAkgny942QilsN|8hra4LQPL&>cwnF*yX9DW1q%w%8eMAE0+zviJA}t z7Wh0`_Rvv6Vp>c|+9z7#15n$8Rz8jnKe$Kv8M#c25A}?Y`-*Fl%<@z{QKbyl)Bty;- zpZNLnX9T8cZ=jyWTsV{^+@LHvBCJCAeCg@3htdXxH5dbTeyu$LnukhKDj9Pf_lF++ zDb;bH7btoDJo^>o#B}tuY*i9Chb#ur7*6()xm(&3 zI}qe8ZAm${9O8)UHEm}C=sIW98=b26*yC=M`#44?m6ux(a-KB7HfP?h22>E0;W9ld z*l(vH?RE6hn;j9bCweBn*#5}dqQ?OmHPpM&t=BX~l(<*wub>44)~|H0`?Q2)4dVN~_7Q?|WmE^3NhUh<}cF3&hMKSxZXGa_$wr zO1^%wUDw@Nc(8X=Wz2N_;-9+nJQM(D_snv&iO64rQmd~`Cur$vSP}(6lB&!*!h&l? z34H$fEpwS_(;UPn#3WKQWe$$*s!3DuodGVYv~e8>-C}h$hE7GOAAi{DXl2?IfRpVm%LyAYOo3Bx(Av7#jl&es<6Px7G0NX- zCyyK1dq~tAHBCH7LJ~C|R;orJWgU`YM>$ z;KyDmH+rCNf5WV&woZL5i|rUXB_?$r9PhjC-hL%ph&1YNjRs3D(TMiFmmd@^Kof42 zcjP;g6u#AAp1!4xnk=iOpo4(gn8qT*l*LrE(uqm!iyGqj2NXiDY=lH~tq!E4IJiY( zpV?73GeEN@^+9a&kFbyC_ilXn>y504)`nD51gW8{F{z-5H_hJIjiK&XHh31C!~Q`+ z%CXIKr4_B6+%G;b$<%Mjjd&H)DfpImQClwzpC41XL~yCzD|M8zHvDQe32Q`}z#2ZR zm?FMc)}W(BUm758SFUFGKp6R<6=rQ${6TD8TYqmPSrUmpY1~U}e%T%-OG09^>aE{) zpa^PLHCk1fWyxlKubB{*uob}5t{2+vScP+KpdA;1x>CINBKEwe|= zlx-3zs`%C$);cjdJEj!*69sVy?Qh>yH@!9G_4lSpTLlQMd%iUx)X-bhLpS$}YwRbJ zQswTkv1KbO-E?4<_I6WJ2Yq&mrs3@&7gUK+)^07`;d;1OV>QX?(<~g%-vdLJPS#zG zrho9tANX4%k=?Y(t#psy)s>`Bj3_jZqP_pkjTPRn=V-6+Dtx|{Q}EuSAiu_GPO_MB z9kEGK{dpAbgJZ3os`h9$$srDkV5g!x@*ebt*T!xc5PSe!U`D}bNJ>EQR&8p+^c!J~ zM8UlHcpjj-udrKyM;ivUCx<;ugrHel(_UU)5SO-rsy_$qe`C)W5>Q0Da$p(deb$&C zpcyYGVp0a;UmQXXBs|HjL$Md@ftn|8r2Rea0_>M&KFmf`|*@(=|>#q%hI_xrcVK+U?0<$f$`vWt1b8z)UXDJwF})f+{HX0MU* zui7nk=Ovh{lLlD*KQX`(c$si?(B-mCyqdpFMK1Xdi0dJaX;&TyFQSlT@w(`&PKuTt zd}D8_PYv^E+g~3|v*q&FwyU%%~=N-cMh+%RZ|)5Qu_1M{UOnp8Z~*Xf6M!SjxXq z5}GYnlze_#r>H>_!{&?v4*x_HF3#C6-Ju#7wxwiQdYQ?MD*H_j8w-5N@{=f06Dv|zF+Sx6j9;jZf zrkeZxdbd*&ahtB8&Mk|nN>tc^C`g@)dc7_>nSJOcp_87T-m%8QFA(X>Ce%dK`RTp| z%*NPEIw^}ds?lxvN$iAI{KU!)i`SVa>K@PKw~t-S_u^=_3^pN%Q^%`)L0dA+C7;#g z+G_X3eCg%x)M(n5LLbRPrTyp5$)c}8sXG(_aj5j z^j2iwti4KjxFV9Xs0IJ*V9r|5p!{w9&(yudvA+JK+0ToaWoKS1&(_DY!6`lxT6m%# zMX$u-=PVP{-Ys>;41CbSXT)&HU4TBn#S$7DW+G~?$P!58w)}J^nAuNXOPG>nnUc=C z$rMN}#2i9K`7ItJ!TYCEf!Aqg$ioc*KoLh2RzD6X}ePsJu;^r=!q)Il9&&};gG zX}*8uO1iqKz7K)o_JNh#V3Gshchsc{FCFE2p4?6Qp>ne}2X#~S0}8(D5kQ`@YijL% zF>e>#hCU|llNj-7@m+3Yp6H!%XIPFx`E$``c)76qp*H;wduUfeTTYC{9FeSZlAZ}o zm&0?-0*-f&A5i7Gav7>BEzd>2>YYxGPOOn}j5#@B9QiW35G(okbTTv7XHp~9rCOY~ z*??^|icavsnYiZh*1Wo|KR#e{>KO|eQcP9&f%#c)z2&sH3*kkrcPN=pRBoSMIEOY< zv#Z)qo)S)(J-vFzMXE~wL4ml~3;|}fGb29Yk>^r&4pjr`Ehh^HHfAoQe_|h+KwfA0 zGhG;as<{%-B2(IP42OS0J6{Dm7`gNtADew#BR<>Px31t|d8Og2d$6H(0=q#Dg=6(@%l%KUhm}=2^B?%%d6$XQNiYgQ^Ix+gQkjZlCMTY$-uRb zu(kO?Y-xPaRqRa@-=EHMkv9zttuxZE0@&|1FF9VY_cntt^zM7vd&D+Zy_e$|fO0eC zxpv=>t&Nfr>D*SzHa}0{oWB2E<1DVS`D3Roh`q#X?>Ci**+OJ@IZ;q)a_oJVWo3rg zyOJ9$t}DHXwJ1mdR%xR9wWEPCtiH0vQ7S`uIx~hlwBIRG62$fvS?cO|)qxk2AKGqAK~7wr9MH*_m6AZMBTx?_fy?_ zZHp`p_1ygJ!?Z_v=j^2ADmb#aju189*RvWG*FC*L#lIJm`A#LTNnD+< zvw4YF-k|aHDp_*)prk@x^C4dOQFwAm!u8Sy^;ai?f_BDY<`%6n;N_XV!9-8UX#fsB zDRAO8k?gV&IQh0G=}mTUVgs&TkH8b3@%Bj`{3xM%t}4-Us}VEVtFjuEaKA;$P=nF! zc_E7DA0!fE2Vz#kmz}JP*IE*|%6gefQk8zxwonN)nih!6!$W8B-49}yw)$H{*LQv+ zn0vmim#Jh2HP*L5C>LWJ%Mwcvu)a@*OmVABaDJC9;KsRFQGvwP^7z$^$I68JSox#T z>t`_wtE8bugK3iNyn29VTOfJ#{aHMu1x_eG4vCN3*O zpuyPgTefraQC_#?vg4lTPlVtqtP3DkYLcLJ21=ItAC`5+L2b#c-6NOVM@0;Fz;2l^$NNS4Ht{cuXxQhgKL~RLr(-7b^P6zBJtL^@`R<|ks3a$2NyPyZ z|1P%OS+-h5N;;3Pff~8k_EYhm`}rmslx(aWX}#n_S($fJSdHNgBr9FL1@V%0?`2ah z*YseqmOZec3kEMQv9b~osS-!PJu8j8cIa~{=?K?1IzGwmB;g5bzCx{;t=sdSeV@!m zj(*)0hM3IVN_b$ss&W>9KNJja!uY>-P7!mxsS**Q%=nIq#6505!ge9^aU;vvVJj`_ z%Z2EO3n0;_01UrzBu9P)RAV43HRSq%m=t9L(4Uc!aomX_ArtG&Y*|^s;YyYTbJ83- z%G1{(_!xa-A#ttbzWoz>wT`oO6|(WH;>ql?tSTSFIEDDO*v+xcO&)N^7eGK*5A)AY zz=u{f_PB(KX8~|cZf{OU!2l}c-yR5lupI7X(l?*kER%Kh&AiVC+I*CV+Ev`N-fzK| zFdaZBx-ga(<9)Q#7@+>v%3g*`s_18Zt<2$Ank&9!m07@b;HLg+Z@v809!9dq)JN@W zwex4(FlHCoGGW~QSfLmUR^VJKDkIi0;nYd7uE0Opr{gW zLl+cDrsA)}Uz?bwwkHZo-^-lz0V@68?|udg_j;nBsP1wKx}?@_+nMS@egYg#z+;%p zMoVUMiHQ)K)q2Tr&lOX?o!;;Il>CjbO>YDxe^+?N#+%iCjZ7d*jaLqmtT=PvzWN^&@znM#=1wr<7ceZR~&SWj8P5SQ7c`)x0LU&R53rpWcDx#bt-c z!pMf87n5)V7xtP*W(m(q994zWR;8X#dg&XL2aggDhx6W|``m-P3|+dUec7u!Q6Qrqj?fyP}W zx_i6pcl0dS8rgY|R+~zBs`*e}< zEMoKILs_zlnOW9UckGqM@nnc}L$@Kuk5yjdQRNt1%>U3SmE3kR?`?X7HWv?z)-$Nv zG+lj7U)OOVv@$!4b=50g5|m~gR$TYopFUopkV+(Wf*N`f_taPSl}orSE%oV%_=`tH zBJ=b59&>M$m^Mp9-E%c=c%2;oJpO{Hc^uT-0>UR>B>}^MlT0( zas0gG=!unlWu;?jHoyLaA3g@@Ga5$Cd$S5v9r-yldm1#f81ex{dA>GPgq?ozkZWZgXj9K`cA)zo07*uUV8b#o4)%7F73Y-DsfK& zR#3UTE9hkA#Gh)9@wdM(s#nM=ODcP>#cv;{v;-0X*Dnyb=y&@?vIv?*EBLZVVGKG= zt`}^huhi)GZ6XakDmj0u@_x5hkzNh7CY@UJ=@7$x-d_bp5z0d$@-thaGZ*QO4xP#a z_#o4Z-;^B)*5Nch+Xm)|RH5i!kA27(UF7DG7G;w66I!yl`FX1>7S_xj^z-I04~)o! z$VGIP3xXIxFZj#1;~=9luZZ<+|p3eC=87cE`J8Hq~=Q7d!EmN>6sGwSmqduF?5h zrvh9`=J@s7Qz!%LlbyT+l3T@^$%0Ye;DdUTnS9pK2MgFT%|aBZ`^Hg=&nvQ{SK5(& zmf+iNHi(yx2@bF*eYJNv*{A6vMLx7C$AtV5c_kx7gf7rJHJ@zQGmcM5xOneWChp?% zA#u}~lT1t0ZQVY-(%1?9wA@#_Upt1XexS-E*mqwy6@HTC{@hyLrd8@kvm&^FeQ?~H zz+s`!m2EyY8+ZW}di~^t)@%{?ngeK4V#?n~Ug_@ZtLn%L7fE>rK#OPDiVBJn`hnEy zlxrmo%N?f{BG84?%3O2;ke?~(Bv}_;s+-Q*s@EUznf;2p@(I02c%YdSnI?uhMdT{%-kolo~qRr`G6@3OTX&|8ePl*yha z{HiK=7R~4$``PzMzka)xOTk#meij~YwY{U1pcw3J`Y zl@|+4i0mF>4@e#wvH<$-kYsit^~s9gaTmP1!KOCp5X}td8xh)ULSH$APOwMuM7o~k zMvxz@Z;>Xo;i7at!d8tF#>^4@CNPidZZ{DT`-?BPhCu)AD$cz_l@UT=Q;)dnxs z=!pV2TO;)T_Vy+CdNGstNuV%p>LT89O=qyBbQF?qvT-to%LBI{rJF2Zii$N{LaGY# z7yn5vy|i`vvpR70vqD-UHp%zWBdy{^E%EUK^gLC2G*6?B=I==TGL>*@z3VciLhB== zG$)aE{H8sUYPr8wCJx_evU1a>%=ZArdFH)pVLXecGQf>qsjzpfUFV%9SJpCpKbCB; zkooFBW7}Kcy!6J=03ELgy*N-dFB7rqC?~RuHUW%*Ce;0>A(y`pF*%=MS}D~$$L0N@ z(}vCtQ70nBuK=do$4%0zf$CsXk^RQb?hR8PDmY(8(HIu%m+()*!pGxi2tMy`3{D zVUhjJ$Qqnu{x3a0W)UPNw^LP{F)=)*Fw)dDyxaGLR&|h&@1yfH!*wQfcL<$r@pxgB zEP67QjtiWbJ$SIS?%yk9JK^|+5Vf8D(+hVQ6F^Q#!|!i!%4klNVBgv= zJ0yHGba*+hl$S7aJM4sUN7sUNB`yh0q&S+V4j;FyfjE=!`1TLikA|D8-=$X5Z@efheD6Lw@=qw_Ewm`!CoG`ihxy?zS*L;+xmge?&1*Qt6x~$`LM`LM6PTL4LIK9 zeUSHvf7QgRX!d9*GY98qyaKQeJ<;});p!Z3!ZfJO40RY^>?A8WCtojXRxhmw(Er+u z%5Q$7drEgkl#ssI`za@zCUDfr+2(jwqh*J2|fk)W4XI8mL zGAW-B5!*4Xfs?s@?G{LBsl~AF?r*_ZLhybP83do5{EORNUgdjULaHd%KmyZixLm5@ zpJcL$c#$Mv!k<>_kf`Dr5hZzJqTR48>GaCyyJttQgHGz4d~;?TjE!a$3qBtdWm}%C zncdGX2%BwX_=m)_D}R4V>VKLNiX01d-K9}!lr~f~G~P!zKESS8SNHp^GQZiOcR31_ zw<3JCs&qQNlAc;oG!=aFtK$JQV0@my()WtcmZsf1F}(WdK!NmXnZ7pM22p%ai#C(>a`V@lx2CMVRj;Ye3?Me!Tq2gx&0(z|#xRuxt7w za-L}I9aYo|oS}(k}NkEt0Nc)uvGiyj~C-`^x&c|1^{AU7E3Zu{m3z5a33-P|IqZNl`fOySlNsHAc=o z%Px!$S=4WQS1c0~z8aw!`72a7ul08dE3W``&ons4=SXL-qd1tVkZZdQ#oFgQj#&r} ztM+BVE|x8apjSQ8PBqEFAyekaVKIiCI=vR1Wxw>Gdnux@uj4PQx+Z3`TRh>cBi{Nu z5UU%{-YkL^a8`R0qjDu=XODM$>L`JH8^YhJ95E&K`N;*SD)~@*U`g=Z??gfThtj^s z`Jy?=sDr>;c+nv$v&yO0rOt9b=Hr#S0vR-;RyWmt##D)u;rldg$0 z@XT9^`5o-7mDK70S6un)$;K&T#6*g9+rTLR(s*)}O{bDS-hjU387f$Ca+!M45`uRS z;}2A)O8ogM`?iDTt1662x>U{Hf#p_}J%hL%OEB{t|9dN|kfUTb%*ZgX2x-aP0Z3Dc ze4})`2Se=ccD$RaBrW70PhJQ|x!cjb=~9(2ePYr^!3PiIWkj^rA}b?akabaT^OA1} zVqM(RBQ%3Q-o}moybtttEak4C4yrbc5#wwmz%;j0ha|P?!yHl~tp{ouN=Ey{CPOj3 zr4~p%zY#^#r)hHE0KY;YfhT7l`BBkSe-37d@E&wIb=w{K{P_-^+qDgNT6E}UW#`PR zz8ef;BIPb^K{wJYt*mOM>fU7QP2kk`Q!~6Vn3|gm9F$mt4}&yCp8-|EXw#3jG11Z0 zFZJp&#~ZXw6tch>6%k#E>)F@fn)uyCInTmaRh|7%aMIqs&f|K+Pd zQ~c~=8dERf-O;4KTaG~lx>Qv}G(6rhc&}NLCw^rDd3|{i|&|Xm9|$@mS#86#Dhr z$;nO)H+8BMJ_W#z%3MA?}YM#lKA(&yZiz66aFs#8}(-Hyj-e zr~lFU3i*a=^*p_HFgjLK(*fBUFKYBBG3BCig#DqZ=YDGDN?&G(HgT8I?gar&&qm(+ z)6<<1W=v96TWS0Q4GG13i$_K?)lONc*AB4&JG@e14N0WwPwh3yOl_T~bl;PSReft` z%q1`9KXSPR-~D5#TWh}gWT=qn#W&e60_S8}xQTlPJ=E^jP}Et>7NVv{o0yc>KEGkI zC_@(1CW)EEZcgi1$Gl_+6Aw%Pq{4B-{1(*Vx%~4M5*^#R9VZCTtJp`CqGlD-hkVfC z9F=%d##6IZh4=>sK=n$?aWr-s%uQpAsy2qyf1U#N(TAp}&!Z)+<&K)^Et;9`8)+Va z>_Iq%h4V6H00j?7zo6YGsNHlW`9kz@vYUh>_q1Zli!zWI|^S2p<&~PE1T9h=+aFz@1Bg`3&P1FgZZeKU)66Z z+llImC_4wsvyCJEdW9j~rNb)Gn29#YtQ9F`4H2L}0Eo~sJ7}N4Y7ve5> z3;SoK|+cn3jyM$;Tw)WKvc*_#m0P zP=`5~`oTn0(rJTdlHk10vyI6?b#yShH_=8xYYrDA!6{uTuDu}J#GlD$l0?LDL>Zvg zf0y3)FoF8eY8IDvAFO0WN-Qe*tEyqU^${CzSTX?PNpDpSCy+YtlJi5&R4%HUw;NNj z#SoRF%7!^hKRfxN~U`>7`>vIJ=sUZ=W3Fy2viyyJwo;lm;u=PuR+j$7sgf9bjfCA9-Z$O}CUanr6XUH{GIw2`qH`#=k?{>7Bu!8 zsuJD8vc##DB<*fCH|Or1JlOI{Pzi_OiszbT>GyEHLyQmk-nNpkgztON=EorM8+qp} zjbe92&!jJGaNaGLwJgc)A*wRXYW=7G(-G#8+escqI3ztVSdNEp#^kZ~GSV zjfziSc4Ssb9n}$bXR~V+N|$bc5?RRw40IkAU;1&~uOQ)GVa}Cnw-VXkh3h|PxbjYv zlI$*fvTE|S-vZ__-o2m$dVu@m{3F)^*l)Gl8bAzMU9(yIVMxXHskJ<(_2g)e3Id#w zTLon+aK7d+c4Bjm%he4#l2+rx*-FuVo`7htaGi#)d(tvA*}k$KliU9w4!0SkHYvh^LcT^wInxlB=-2OI2bd@(n0&6f`B+r2PzPIPuJ#N9( z>vw4&#^B6P<7F)$WZ!#eIJh^#Ge23y+yMcbW4LP;zB@4fR59AO$t0OIL}xxtpo`f{ zIp_0c8n|D6+dU&wxfK{+9V}>r)~)Q?RV^s zEP1MUcTiyK8Y1?29$}&;aP@e4hv}+Fn^vU zOAzl!bpFI@E@g-1o=lQy2NnTqu(wj`4j2 zhP-wB#&@_}HuAL|SAPTJVHl0dD-z^;@nY|lx+PLcF!>1H7WZHh;#c~JFl!&nu2{$b zaKz&vjur&1c-!X8i+*EYtWRdz=x;i2i0|*jnGq(+-nS-`J#})EJ%<*#6KYmK60tuY z7D*N4Ap0O2Nqw!vpo-I?J%r+5XB^&M~XZA|@{K9Uu zK-^qu7(@249hLrBhTqaS5ATTomSrgnW?P_peb!~389w0a4{AGWt&O1POebSvUBbF! zq8zO<2N-kQ!(Z(#u&r{~an%;fgAl4P-H^XA`(~w<> z(c5%cU)Z@5iT`BedL#iTU^`p2^HN_OgBhz^&e?)tjW?vwZ?pyT;N`BCaFN^GBS+KU zoTF*JDVa`VS?A9Ta-Lu|Y@>DmB<^pCRg zz9&3W6o3a45DD<{poxqYqgkj4q2P*MGQLeJrBWv8*C|2&LxsU~DX!Ma>g-xL;3^4q z_Z5OZ)6&YprYY?Y5EX?3(C(fAixL$rqM63lFG@MwmABsI8^Q|Ydc5)UgJdsfRfoWJ zXpnvW`x4%rDfkRAt%ZG~HShXxO?)X&MJw2R$|@j=Cquh8v;EUIHp$Y)#YeVNUz){5 z6ZuIYWsoR6l>g(;cZ2uQ99Yz9rws?eEPQ7{s;j+)yVC%lZgtcvD(p!xKU%9g>%qyC zhQHwm{?lx1d}+ORN{1|ykHo7N1q=?bvAvi5pdcv%d|JRip9KJl$W$W_>jUTvbC8pI zgegcq|5NXXjLm~n#+?Z{g}$9QqNEEzL;E$6cLp|2EqL&@V%nFm7utTigg<#Z1UOx` zc>oCqpmpF9zln}*V`m+=i{_T256Gd|`c9^Nrdiy8M@rF*kPL@#!HQ^IKJC;k`5 z(IBsX%vEP*juysuMooqi!GG>!P4pw)%4{t}Yp=No^^n4;BQY}3P7XF*URhD-9hO`~OHM9qG3T28srS0IXabRfmsR6njWNXG`(9}0D=Y%O z(xHt!vZixeIf}9B()DJ@@6(%B_O{P0?Ey!Tc!R}`aJmLR)N#?mKUVV$pZjh< znrm4+AR2IsZxU@SQl4Wes3dE-ao7*qz}rl(RSKNv8Qqst&PCKI72xej5qG13ciQ|u zs=d3m=-k1G`c{Da?MoJxD5A{CV_I`mR;C=+x)DMh4s2#qi(_MHlZzK|*zQ1AILD_} zZUI^@_097F`zI(1BtQobTJ&%N<|u>qng7uq0E3ZhA@~ZTZ$Z=PcWZ5Z{Q`T_pL=;R zF0@miUmov5IhhD0?wPNjcjU!Y9`ki(;5T#cYj}=#`TFjmqn8(}u7CsPHoD*g{v&?l zE-k@F(R+UkWM6cN-?5pkGuypujj!1#d`kyZc^BXx@6+fPX+wZU3%Kay$L)~WvN5`!nklK$NZg*nRnP~327 z5w7{F$2xN`5sBYo0}OwJQDk>St=k-_;aKBW9hG_#=cWx%Jt|Aa$KImOq5@LuG$%m6 zzWz+J1;6vNpv%*FMBkyB_zzG7Ci@PxU^M&0q8kiS_9phs=~n^%Gacx&Dylxr0zSAE zlBusqfO`NG+U0(pFX^YisoRaavJ(gLC_8({s&euAROYV`-Ka$N5Sk)mR4J~@9DCdc zvIKS4HT`3zyQuAmHG3I-XRp|oMm`9e-J|~anwb>tM3qU&U_b=;Mb(Ydcrs$Kn zQD8($Wa3)OD>b;R9LxXjQMpx~Im#yRSYDvzb-_=QYPE@yc@{iw?Y8LAFaHmKrz#XZ z3_F$_F-PoXDv1-7Rfx45uTiH+NDboQ?S7e|wbMuUlhfI<1*E|=8)KR>#l~g#Jt*wE zTCA+Cdw4aGY<;@L6R{lg%F(6iIe=If3lE-1c z+JMM8CUFt7bmm0DBM!0vyblMHPN^*x^Oh%8KA2uWzFg$EyczALmyG&ZR_xL=wV7gs zn8<+FN<)gD1Y7pPID^1|&++@mpz)6L#$5B)uZvv0GWkvE!emKyJ9(sh&u&QPCPY#@8}4*I`plS!mf1XoZ+?iGXVn%S0 zgH?bBk9t{s&YoiROY$XtM{@$vffs;Xp11}OYN95pOhnLBo-f1^)#nqR0mKnm^1;;a_C%RSiYCFy0!(>^fC;ozS;*1wfXiYF72>k0r%vyMdo{ zdfBJQ3+$jDBH`7cNiT+f9%C6S4`^L;eH+CZYO$&1*w?)XzhhhY&m&>Y8Q8Rrq=$_j za^?8cvhioSxCsdfiPbz4UfIdXsUG*Vk!5@_@_^JTirOi)^ufL(x9jIiHJV=vgp8Rt z3TAMok7BvVkqytzOfrHL87a9>Y{`6QG71mc?>@kx19Y*Otn$tJnxsr3Dm#zn5d6E^uUOyn9n2o^NEgcx-2IdP2c!Q z3xDeB7M~~%Fp9~&B6z~~(Q{zkvLg+T&8fR&NS`#zSFbu9?0QC4TnHR`UZK%QJcbvX zToTiNOgs>bY1GkR*exo+sr&8H*Q2+tTArPHhiF{lW`6g`7;rRK+freUEv_=%_4nEs zF2FSe6w;K1OJz+4wluPe$iCuqEjE$66e$3rsn+cZ@0d>2i)5JAOMiHdtUK z0s(l>>pWF%Rw2VGBV_9`SV3fypAUQ^NnMhz~|xOGY?+>%Uo{EzwN9p&?K?qXdq&pDq=S#M5&>x zoAhY6)wt3frAzO+RI)o8AN_Xq3ZrLr1l;k1c&f0Jl#~nHmu_JwfGk0m->}>8%#TZz za!I=0>}O&u$}2$D2wKjU_;vf2R{W*xGth0^D~TGGC&tJ zP4&Buzr848P2V8oynlffY-E3;u4bxVOp_@Xs+HSA(-LlvfEa@mOdiZ#BW1X(1I#qd z0xFU7I8Nao6LDT^$bIqNdh45sjZW~2u}f|nQxN3oWKksss+1tPMUoud^h7bTt{rS% zXK^rtd5_#h8u}+ft}=4S2Y+UtJ6NZb@I9@Qs(G-Civ?_;Hp5?29_@5LmH3@AN~LO^ z^Rg*6c=ZP83DoGws}#mBc^S-oP>}ztB*DJC6?R#m?hYYtT7o4La{y-m4}8DdHvI z4@Q%1)1f$#9=tux zCpNbW%(T!jL-0u}`#B>oqbGf81`=Ko>uE#so<*8XutP(nl+ zMFa`yk`j0dM+F3F1e7l6j-f+Zx;vB<7+Qvg{~EZx?{m&`{?F^*i{}j= z7-sLi_O-6~Ub(!US+*eRC}?Y}g{G&rUQ=KZc`@77Azlxk9%~&h7*Zd1Rw0`tE^Jj?rW6RX+!6P@8S>~Z?l|@S*XL3eON8Og zwb?YFY(rTl@qK&NR}}8l-D3aE;3H>PuWG2RZXG?<#!9u#yAYvC5S{>0m*b>^MLx2q zi6X-rU-pk4*conn>|1|%5q?p7ka9k~#kjvt98>oWWa~MhlvVN>ELCzBS!MJekv0x$ ziOe(QPjmVS&)^sJ?o~OhZ1$_$+4x1Eiq}VD0c(hhY)<0-2`Z&QpKmTLsQHuThW*(8 z-g2K6xIM2n#SP2{n`8_)t;gB4=cZywfUyoULMatY%2yWt{V`7>-28BK$6@@@t=5fC zMMs5pBtB;8ouIs?Zp#KXDsG>EeKKp=J~S8@*&H5+VyDDBD||?ROUE3>a^%r?c6u5c za8G70-I|`d>i)^Y>O$EGGm333Rs$;;&{`-aq2c`DBu|$QGPwGPxFefJed3c+nk@+{ zzDXXWWvv*8pVMx(n~~gEnB4s$&7%5do1)&tX!FyQMK3ABTEHI^OxnK-;aDQUIBqGw zF{A;?5#9%Vgjs@CS}&62FwOm|ySdnJ=TL%{;X*!;rm1%u(niB4`sP?t z+;$(^X#WuZ!K7RV4O{mt4W|ZvbNcvQN!jb`pMOo77N)gwuX@;BQ-tGS1K`iPn*i;MU&Gbd12H@I3ueVG;M#k;O-BD-w=^f$u76rfkW&_Ay^ z4w$36S!&l$w42ZUiUej~`CwHx$bblrKN&nYR;O-R+wPlPob=i_R6;bT2#0OF&)o;l zc#*J~Y6YkpdZ@iyz10sqH`>y#5BsPy z_^2Ou>#Stc`yQmBww*C9Be&};HOU%*2>me>0LC1N5 z&V74!%1D)vHxUQ{3Q!}LIFn1VQIIn%E{^JBz21Euh7-0I2VT|U1;t_wx9&XE0WGIf zOok5A12<|95TSJy#xQ1$!*ZUc!O=kjk!0YbWvu1(dr2953jYb-5{?4;gyAU8v7*TZ z80P*SumUJZHGS=SSc2;3QdX#0n$gd0`Y9O+r3^GAJndzJL?cj^2_B39z!XVcWHqzDgGUn3@@fp#7$aI&+cp2P zob=WHVV`zWlhD^sUf9UcBp2va;>oO-L5@tNr|j3{Cr$J^H{Cjskt%tT(s3{r5*iTs z5LqF*M$4Pl=f|n4w*4$%eErPx@-L)nXqycU- z{;SIps>{NLI1YD2h^e~?McOEC{@~t3U_3cYU-&AsGB_lsWz^I=;eXF#t3f{-hf#y9 zHI03C2XxjZ9ay6uqP+k`p3l3+jzV{kVXpBJQ217!+kjFQD$GnwcaKovF|uv^_?IS2 zKp2S3W#Mi7#(9jP)2^R)0>T?s?fRfqy(Xgsr-S=otWd!4@(E72j|xNUnQ^qCHyi#02bz_IA|Cw0AAlP}NqyZ0QM ze=*7=RLH+ZtWQnxzp^r+j%+N71Da0>NB*x8&HL2$@W$(o@tgi+P9@`gHWf=NE9)P= zlY&&w?5G7rdsTDz7yuR@HgJRxCfV8P(`fBvUg?mnGI_JkH_bQ>{#1pqo6)j3NIw#( z)nsvjmRGLFYk>-H5Ycg=GzSy(ORyC69*IVvw&62g$ITxSa;p=b7AzRGKQP)OtX)9)(gA1hCl~#(p zY4Kh^e-5qP-nDXE`$8-Y!i^FkH9QD`pW(MQBRt97L?1spG+#UkNl0Q}cUtv<8{FfX zAAF*$@}S?$r)ncHK+o(ufTb;b($@0MGUaagUMvMy9UKPg=+Dj$sjIf*=cJQFLcT0w1VW9WJ^_CZv!!Hiv(@Zdi$R>%qB9@oI*3HN#02*|fEZ zgaBEq`+QjG+f2I4s18Mu3>ao$8B;Vwhs?1=5>3EvireSk+;}r(xL2et5$8i7;%+*W zYmV2rhM3!uM;YI7Gp(thEIvx=nR8`}-t7!%$=I8+RozN8Kv^gol9SmQNb{Go=!_*n za2beDjDV=9J&e<0TBEo2$CK>B1nh)^O*;2IBFT&9JJC29wad)aWE)=)bA_4R$L2Zl zJg3rC>-AE*AN2%GS4#*>+`EXraQ?h=&tpx>)HcrTfVo7!fyyN?Aat?4e4TQo)Fq`~ zK&#@@(9H|!wz;Q#%LJ;fA~~(3?vj7in<_M;!tLJUHq=P6G%TVN55%l{Yi-G*-D5ub zf#Wreoz{vl*FHPmH<_ADl5o9>8J&uJIhvc0k~h1T_wmVHZPt9f+__-ODgU)vMsY*;^&l;dH_Ep=^APGpf55RzvAIqo<=1?G^S z*&vNIF9M8>;F~n-jXz3o37c5n`VguLTPSCT8ur=tEhRF4c`x2CyQRCD^r92*vJ?8g z;OnOtw@+|BU_%Lri!N69w`RUiMS6W|3nq=4y0dra?< z^EbexzpWz#8q$7#M)C4Ih#=NK^X&GmC1%xp43YaxO~q12_lj)9opNM8UygM0-^Vgq zegt3zbvT-Ko)<*A#4*fQ++#PVxU%f+^Zg>|zs?TRf34Q)AEmbg;woC^h@9;*s=I|p zFwn{J{AU4VY1arN5#mx~CIvFd+#K6rz|PS+%y{jPryN!;ajGooODmrkUj?SAI44Y6 z#a*%{dJA6VVyct#^DUgEAJ^Y4LYjN`#7#M^i{h9xVLG$!7sl@U&%%2~^%t+NMQy=#1CJL8oBi{%!x7R*Wk>1k^qUPkbojz;_!-yEm zBL!y)uw1*>BDsmnZL?Sw^SkE@c%R z?)7?BMNg7w5eW3Bn)(36xD%tR)TpoSLd$mAw;al>mgh}OoI86UUizwMIf`kXOv->0 zIV(!S03?yVO->?xgg3wk2H=$vmf4An+y*i_TuJC3hU0qv=4imGNgYaZH){rrx=AFY zg8s~(35$wysd4T0Bm=W0e#$eD$RqQ-faA-}T&8Tdz$y~3qBq>cjEk%1XUJLAh~pY; z^sT3UANJ5-+_p>q2%w*hsU_gXC!i$kT3Y&Q@xs`Dsa0d z6Wh-dsI#JQ5*j)M)QpjCG}2$2qyd`P)l0zrKB{H>o5u{bTylI~P&uc>aO>(;o2 z=4q9$DQtWens`V#kq&Jbn%fLjzw`JygA$Be$1%NDW1+RAwAgaSZk1CBr!t%cWSNDa z#g0pz&S!jXR8R5*06{=~Pf^(cpVVkQONosk7pElt2&KZdvIB>qg(maH0Q#l2T*7>B zqF>n5lk}14X`SPG227XrZiHTqAUfq$at9OXho?v<06$zhIUbZe?Uk2s4L4%0TKCPhd(8K8acb>2t%rPcL7j)sB}{uU0R&qK)Z7W^OKkKs~Z!GM|dg znmP!~ni=DaQ7O?tHqT(9y%_Aw^%|n3gq4o4Kshb6Y94;bME38j1vTFHQaYJ&E8Pqa z-5UFnS(uM=Kbq+HXSn#WTdvhQ)vgYb0dTQbg?tiC7%XgbXH<7{uZQ!x|5Zjd2K7h(bD7SN9yI2@q zC|*lAYnG1+{ls4k%4>fQC)+jUDig=Ql*%ZkyL>NfhZ$`C%wd8O;L&Z8y-az70U8CB zWQ6$#sFKbPz&JLsTARhj<(@bW9WWdBPM&VfXxvvqS0ig&b9tY*SSXCteYvZHW>0a?pk+{2yP-D#D$VZ4aJqZ{Ux`Y*QfkdFCCGwT9 zZ~GW^Oj1A%G5<#j0BN4GAOCQ{%+Ts@0n>7`X@1BUxG76lm6M*5ZGZ{ENsY)kn1{Xm zTk#wY^DU1wH8pQp(w@JIZPdu9aas1f#**?Apg9bm2^zHN@`491@eaGx{4b6Zn6-5a z1bx7msD=kYJx%>yzJe#z!FCsZ*ra!C2U1&`MDaT-rW`0t^IxmoNyv9fy%jh}5 zDjm15iiec@MPJYK&sR@4zEBUXKu(|aZmdjiExn4^n|^z#oxzXAple$#9bZJmL~uJa zp_DY#_p=rRPSLYFAUyB&1j*`eG~IsAwLkW$VdsyDh+MWDB>aQh!FpM%z!7eKE~hVBKhs2NL^I zZI71zNTFbRjz)mnN%b1oHUgY*<)=CM(uapm&Vla_pzC@?3z6Hz2jUC)0!#^im?u`)?Tb+CVp99?j5j^3nP;= zK#gS760|L8jV?Dml=$#xum2r_U1|hH)Vy~><2w&G3jAX6NL*y`L}f5hBT->93K5M~ zRWxjtQ)=3@WmAfPM?VxD9p^>p1Uv{G9aWfW!U!_ZX68+?dB0^6;2aD>$X+R~yU_pT zQKJq8+x$mqg3e#6YaYCS=)B+R_yCC!DN7dyO~Ev1meK~}PtFy~O>fu) z$2JymkMC;r${2`-OgYEd4ZX7&`fUj@5Pi({)@EdP+AfIQ#bQxB^H}>QwHMD1}M6W3ovNels6yzCK>S2A7V&l2BCF!bt z2GfkoV?M9+YBP@;bL}Y{gmvsljw!sLg6Ay7q3WJ_E&gvz#DEsa6#m0lqupz}{5d0dUOAloKr$zG zTmuCD!-RH}y*VqE`j-!r3s6WcTHg_x(ObKDwwhsr5ig6;ZxU(*2=WOc3Fqj%l@ve> z?*xro4ZlLwS)p6;b>X4X^MAR)9}|tKmRyXO=6lLWX;kx;(6Al1Ll!D9VlAWbiT)Mc zV|q1u__ViVamHH!R3_+)vSasCntIjw?PwX6ecOaaS9+08{!e-;grT%fu9MwxNc79c z2Ypy~>ChKHG-g|_NVnDn%0an#2DcRhj{>s;GmuxC#577-_TsfN2HqfGCiGs-h8_@h z5l$WAAN^ey^ADK7)HLwfcqSF};+(j#ekHci!m?<_%P{oqM^)~j+cniE#~N5{WYJ)0bl;eRzTt>ts&bQ@EuEqmGb?}SYuL1J@A>rk3m@)>%t_MA+u17vv840O2C|xDU zTgW#?;Bygr&ILy?GYB0DUD2uvl;KhFgp%g}`#BSXkG8e5B3yY!I9?VvGm%4x|9LWnKPpT~Tb<6W!21pvXKbLbq=CNIdG_H$+^erSY9>?1 zkjhW}$Jq-?3`M1|b6+DP`NT3fv>?imY>hT^9TPyER#iy~c{@EvFLq z#Ld!7t|QV+udH7d(*&8>65&QY#Ede#l41@?s^lWt53QiTLG3?`u~-4xzfJ|~;kt&H z&r)X+3(b{3*YxR8g>j%9WPcqZMEv6M{CVIRI73KqweZ~+Z`!4nS`AHT|MhjRLDMne z8TtF_@fjLn!_1|iBTEw2h%=d7SJAc^zm``7nYgWRn(@Pn`$h6U5C8kO>40wy*HL#R zlB--zBj%mN9M0iHo^OSpqjUD_Jo+ za$l4C(IkxsjvKBagve3l2IaI2%gy8>8!fCO=qLhIq(Uss)SfNRmz)U%>>MxM5K%tGMm21c^&S@Lhe<*isP5SvD#!m<{hyx(y z396#}TPxx+0F17`yr9AZ7=mR%HdNX_oO`b#?YQN%RoUff4w zY?B6IWv=3SA^EQI;3eD|OLeZTTsH-FUZaO;KvfApMq^UkF?@?nQn&<@vDYL5fy>_* z))_f@dqa4{?S6vVze0eTz`nBAfbqv2CFD z_=+dPn4s zx?U+<;X9ba2c173Oq<4n;c4j9x<9OKicRD5^)lQMANR-{qKk9y``=oglHtl^tfmvE zgELV6`!?RC?-#?bW*Ai+M^l?)sI=1#Z{d%tGPPw-$k6D?agkwMIS;q|KAr2t3azCv!~hy1^kDZ;KFQ? zHaf(>j=0B#SAxVOVPq(XO1ytxz2f$jzhAZ5{*Jc4tA+7@_SGZEay=oK6*iQrr=^O?Tobfw8m2|uGjJN zc9=D)9MMH?+C6rr9M2d9`r;1Ej z6i~Z1?|&&MIuc?yN5`HzOb8SY6qUAO{vF>m%BcRj@bmrqFpE%nI1b9cQzwiC8ao?c z=#$%mHTxyJfJ#i@9~$2ymUO{u_TI70s8lRgFzj=&$u2U2Xum|?3eoC(<*v_pc5i%c939!cF%f6E7!d~+jUp4xtr#LlE_fd~?Djp{q?I|2uRt5URx$r0Jym#m z5p|n);26O`;Ebw6L(^vZ_SW2?YsCN&x~!*2QXhkyI_lOjde8DdZ-G>( z9UjnS%aj=lL1l-?)C45_0W5bDX@Z3_Fp;_o8>Kxr@H_;~V&zF15PM@YM|PAT?qD;nt9z@QTo}e0 zv7SHFvyuCap>J7IXEGf@j?|PK*JTU{h}HE-IXr%wt1BAH5B#x)!qsT-wVhdty3q>n3~ zef+C%9Z3>U9*Fjq7>|IE(82II^O>ba{Gu4Pa5wVZlt4tAO0~oqd6ZL!AaqcR^SeIx zsPU~0*eZRXtYvYJP&xc&xrU%R5m}y}mcm^+jqAEQRnq)hXL{~SQR>_ep1BoWaHJc^ zjys?_s{RO|3s#(iOp)UyfkF1FAPmm4S_9s6N-cP74 zxl8uqq_zuP|e%@1~9HH~WcUz0qWc(4tE>6bG|MSj=YMy4H z5y$j}2N4OtfW1e33o$CQo`|>;`F>af9hwKKzrxG?{rny&lnQUl1%YU-BdvIk$u}BF zJyb3du+Q3RANjeRyA04-XPTuFtEABE0zou`l>C9%1{6s6!n`z;u>J@!u?L5sUASsnX z9i+Qkk1uj`8wl$AOa|15M|W%A<$(YHJ`$CnamStU%|51h^>&x7=kErd~@EI*bO5}&BpQt!UdAo+Pg9`r& zMabQjd37pNoI8)vni&`n7!ue+a6&Jq)~z0-nx}&F{X$XSs(bC?4Lsv3f=j1>p&WF> zRDIqnAB7KFXKV4BkG_A5bFp{Vgw&s6nF)c&ECt%UE|+!~1)TFztn*Qc%vS{i&jWJ_ zA@r0VX;^JlrZXs@xY^B5QQRpE)V&zOe{M*DBLFJ#sc=H*YZUqc=VKmkm+n=u+CDn)wlAk*#ighe$hdh!<7)(-%`%Qfo3H|7G6g$k3@!lkzQ=66wi zS(2T2<6HW(dp_1-ospxSBsrmctD)LYWQZW)g`irnUDARRuwZI7+dp>=EAsXSE zs7;{-boBHo&D1w;y>AQF*o|E&GjDmZih19OLq^0 z38VtLyqH24#lu7I|2ks2Te}v?&%S#kS!#zy3DmLh9nP9BNq-6d?uYi%XWei@B)8Zj*=aMZ zwE1t6Iidj_$7$aG=1%0|ZV4A46tFKd^TYnacxZqL13p`0BHxb!*?w-)lal=C?<;LF zk>sj!q>&1meil@A|GGZK=N5D!?YX4yNG)F{d|11L8>;D(XHIds?zbp37%c5Zf364I zD@xd}np9yS&ghvp463$?#C1OaJR8P{@(pXI)A8#-83+S2p#4sG9qKnZ0V!c&`s|N| zywx?7BMPue@({|CtBnWuGEqp@Z}5(8L2|Y?I^b-es>$0w)zMKvjY9=oHLh>4)C68i z3g!ZzANa1O(DF)=Y+IKO5R&i?wC9ZV=x{_5Y3s>#&qq;fTXga}Ya zL^y%kEQ=;?eTY7QUAmc?y=0-k9|`V|esMo!2P1H1gI{yW28bET2~XoDvXbG5P9Z*& zzg6%b43sV}tnR6HSWZYx>{<8t{Ato(YYtO`IyP|EkT#S5m1%A11779T zk~>hY>u%hWGx`)272eYE*7yZJXgGWG5C>ArC&;7lq9m`Z>^Hl>L)%xn6|EkC;k>Wc zM-JJ^1z$cQ8|1meJ&dnTAVE@KS4)^IaVe!;tui74ui&_QJ*}sg@Qtt!4IAS2#>%Z@ z>Hx)Sw-R)_$)P_Wj$l~spM_#w8CxK+2 z))WgpUDROv$F~5TmeIyS9jWh6d_DuhcHF!EkT#0LKhW>8%7dB#FuNjs^i^FS@-813 zl5+){LnEbus;1^km41Q?9mce-{OMQo44jX4GNK^!G$3VZ{xSPQPk6Dh$H69nbGkg1;cn0u<~ z&;N4R9dG|YMpvNPb_!;be`CM>>69T6h#jgm+QwHae)rdodQV~)&`LMryOq~pGJ;1y zB_1^Xa%IO%%dX2}C_~_Zsn=sLU?33n=53Pg^4>X{s5frC5di}^1E<@*D4-riF_P?9 zZY#}2sGo|0JQ#+Z@mI{m21W6rqZ=kZ`0y`w1dQK}e*3{w`)s}^b=_A#&1Rf2!@TKoGIs}HS$vNwuV)H`Rqu78X08{ zHZ416oiv_^(2Wq=@uHu!v9r2D#07(d(_eAdp8)+vPt;$=lmUq14pJq{bvq{X@o3oa zu|mwJD|g%kxNWC|H%1~zv@%6kb~m)n8v>WLd`co7$)k=M-UXL{%H862ITh73f|;m} zL(zaBDyp12wCv6WekZTty}`CK)xjgYud|s8-4O{d|B86gqBiN9^A2d;4?ybkOA8R@ z3dLfnH25;Nn%x;{JyX3;8~SWl64LM}mRRm(mPo6N(R)6DEj6?6@3eaL?(<1O41*A3 zKY@99LYs`*vXJAYmW4*PAAfcLrqF=Ps^N&eCr_+^v~HrDe>%VRjw1)Yg@~Eg?;VyrCewDtwJxDy+po;g$_+w zT3(56@+J~c*tW<1wQ2yY(2)g5*EhgM4nJ5w+nk^=3K0+7dq|hgR_Y|bRjY@Y#N?g%~-v>y<_2%-ck2j zFve*!wrtDfuXqSA@a(9}?hs=CZT_Cn+Bad)GO2Hj-O}opPKYxe=5>tXd)X8krsVT| z6ybK?B^v%v1HoT+?Xz6pHS~ArCRzsIm>a6$o2=}s6=Ew}4e!VjV!aDfJNY?=k9Z}A zBqfi{UsHxXx;_)^P@Z$AzTy?D7r9S?lN(io)O{h@1y;-2DDWm*47f+$2Jz3YIJ5*A zHy$OYybMcT?!SqD<0pD$_<(}YO)cegxxVLX6cYnU89!?X5>6dXhIL8Cc7J_IMsG$p z(fbka)T^0tr#GeCA|U$WWkOQl*ekMV;DimM-zH%*2!vUO`eDWNAK1zJylp#BR9QV3 zaJT-B^M@of3QJu~nObtj{}{YMhGfweX;n8}X2yin8n(o|B|aw5IAC%2aHC>wQ{8`g zT1&-6j5*p=5#YvRVY)|#q5d*i=0`Xu`|AgCFR)7QUU9J|H6ZuLJ2z$j&Zv~=8Vsb0 z;bh~RxTbClmrVPH(-Cz&3VU_M>%9Zoc-d-QUdg?EV|bd;rU0Y~pzzU}{hEFdZf>I$ zUBiagYQ&2q88i@}_JP9||1P#GY zsEWX;g%?@NXj~%&d*{2c(S{(H zy$4p$fl^Is4C@{E2_9=&iIw)4dik=v*CZd9p*#W3e`4Gna@Xrf(^S+=_3;Z`q8C5& zxmZRKhqP%Yx&v#@i)5zE-U;7gtz`FFEGnWqL%0ADB&3by-eWZ465A6}FIia9*!u!D z2?(Mv4Cx3Us6LlxdqT4sKT zAT8ZT*_Lp$3N>d&@nueKS+L=Q#E8MD`a)4KyU`&SSb*ry$V!;%o zf(LHgynRUdG}sWJ3_0jR2>&K=Lg__-wXVB+Ro!6?r6b?M)zTLuz|wX2w92k#>;urz zL3!QNqJr;bL5TTasK@YdGwu)xkedg070gz8_%dU(mG?CZYS;?8p|6HSQSYK-{IU&- zbKbnUl@*&VAuE9O7n@hHD12Jy7yjGry41n#(u6>Vo{^Rt3@I@74inBCm=uT&2MOC^ zsMM|^R6dgaYT&hvV(fBWpF6bLPF`AK4z&2xRGyh4+Bxl!(~xIQejwL2 z`Mec<@pB`-{4#EtVXH#xU!$H0g8B^6@M*In^Ae9Us-`gbfTsLji`S zLyLxnhJ$NoC0jWY(cgL#!Qpom$rqNx+U|1v6H7%3pD$gqTwiY4T;2Zr^`wz~UIEhE zuUMy!I+n4$PYP_(EFzywCSs0>JhsZlhGZ|4PL_V-nB-=Vpa^`^YgTiTI!?&5Cz#5l zlpM(r&v=j+s9fMw^&CoBG{sAiT$6pro@UduFq`m8LtN*iqN0k5b%kM8vjnY@xJ-C` zBr`^p@QTA!5bS;DrSfvrN~X@#mS6az$)HUHlD0jAP&fGICj3_1l*v5M>6%1qc zgb*P^Za{kZfIMe$kTD{$^9U+j@m4S)BjMMPO&U$Udpe|zcmK>0AX-lfWG3{)BC})VYpH|C*ul$(p@?>WHigQaRQle zp%6h?$(r>s$+FEs+GZtsVKXTPmkDl^Rs0`1a!FiVfBRA8K=UgyvJhQ{#k10c_xK9h zE&qZaVTMq#k1>^{EcYUQswp5(egiP-an8=yvN37j%kPW;r(cE25am)&VuC-?uSLmH zhHapX&FzH6J#$fm@o>aVN|;G;lLbGs>58D~ny#4Z%Nm)Dii)a9^8TWo>w@{(uy+aN z;#OF2O&`QDAw*gx-$fx|)P&}eBI(G}6d7nqFY(cbNBvA~&X2(ykr4Eti(PGXMMcHj z(cXjuf9+yN)BI8TZQoTU`fZbU9vbLTD@m(5WB>yvs-eopJIlW=nZEngtvUkFG% z`iu+m@|YrJ>E2g^#l269|LLVu;;fmkt)QZ!vJPG!?dtW{W>)xlnQFuRV9KZ$w$i_2 zq6)P7&WS8?KB!uHve1_FXt>aL9^ah&!Hs#M9f?#Qi?wfD1Qjnia*)zo?XLG{Tho3lfrLR4bAkTpTsuQ za#a$0@R(Vb1*U~Q%M|DV0S<%Q4y~VH!+_a(Bj=R*M}s3(s&!OaHjwk2pcih>gy`gz zn#i_}1rp8@tXP7k7nU;HJiWzg=jh?G9t95NbHh1O8v=YnMrA#1s5;xv_3xV{q?-=F zCn~6{98YjK4c`nqgY2nD%gD?W`Abw)%wBZoYZNPWsZ@H}R*$=^Pkt8=Kwv}qKA27S zHS{{^d-0Ey+fB7$H*5r2#=1sT*vv^Uv;|ME-U!Kpb!Xx~u6i!Axxd+8!pFjw+b{(= z$QgS)EpTda&{A4hDIR>^17hl8UJ-3IS{6X+zU49ki&@5q?<)Sr8)H0W(i4wzA;&TZ zJyfT=zqV)QdsgEmyvFn+q&CdlJNWJ&Ahc(c2lQo;0nRHn1PJ7QC4kX z-;GmTugBFrjSTF!l1Ad?2}F#T*oB8S>UfqF1ta8>LQ$w?NzIeRjpV%T7!pIs^AkZBas6nK_u(ePR?%T25}J5>iam$U!& ziP$N3D^W^c%)Uj#wJFwlD@Hi6Zmw}&6O3$~*tkbpYeHGOWb5Agwx_P)cl?u9J`E-d z5ilm56J4WyIZ9*`LN5RL*Zt^Em2zbKm2tN~+mpFPmE_7}*~CrT>J!C-c73dwCNY`m z*&5rGn`d+Mvopl)iM$_89wdYs*%hOO1=}95$JAfz|A^H?>;ygwXkXZ|&k^v##^=cKw2p`^-9w zatyhed zxU%SZoE0&2d2?9=*8Ga)pZVqQSIKP!b3`Nt3uc~cV}QXCeSXD-Ax;Yo$D_fdP7PM$ zMJ+Q&KO$#gEYB7y8K$Hc&{v+2C0U%V5H&E=M=qSSe6jL57@yle`i5? zs>rTad>n%rjM~Y5zh2eQ$0woSP|DC0ztsv;MrC|XU#(ujx?o(+(5Ky5<*@a&csSk@^&-oc17)lszp@)pgUqj)g zHTa@=>1+reqqSvq4iO-_mc=lIpefI{HUoTCha&FG5c<+o&RLq-@2;)NxPqrsb=%`MdYKEqHJzqKU0!jU_cyMePFpV&*mi%2 zt2+t@r*fCt_`H)No8eE}a$*&5@aLCK_*z(h*1%BXbjtfqx38zVibouKZ~ljutAj-1 zYX`S`H94ohpfS~?to&Vdh~b${Q=9hL=ACUa^;Zl~!vq>ux?V*MvjCbeo^+E8ZS-(6 z&D!3G70x-2m<_W{56pGRaUF-Sd2wI;)82??5&XQ;mQboQ-`^GqGy%38MOxk8Oq`?h za`Uy#q|AJ=Baoa_L(gO{HHx+Ggdy}!#$qW z(blt_l^3KhvveGW>TJh@i9IoHYk$ML8^)9z7Id0SG;7(Nw7!|D|BRWY&_RfEn zT<=#ldtptf9DHHM*YJCv^6|c7U;Ux0n|JK3r!iKWhl@IL3qsQMk?TP=-*26Gwil$k zsVf!~8DXH;uvrs}<$1Hnc#qGiA>MV$8<_XYs*TT?X41vd=Z()P`Y;epdqNPwB9sr);B(@CKnbN^?$J%DDD`s6)yR`q+dqDS%h7H1+u%nmPr%0R&_lIUa7uAffuh@X=vARfv} zb6RKS!NAgS`1aj4p*-KNzT|=Tw%+DDc_v~_rz4|z(EL(7jty_{IGM5?X(lU~EBG08 z788cxpKshQdv`F?YEz;}d@x8F?XMc`vK(rg&z(QL9A~>2Jf-@>bF(%dGE&L&=~0u+C4r zWI|9^T+HW2Yp|?Vl{{a3ObpGgDIFcuz3*)?U75nDocct>*NK<=*@~le+^6oJ{8hbq zh#O0mtuBE{#@43Q8STfGJ`ZwOGt07=^)f5D;Cg_dSgu z0Z$`Q44?}fFzUqZ?Pkz8Yl?totx5LD_E2HG!0vY+xUwu`$Vn2+N8@y(q40f2xc|8C z8`Y1pr3lU9_EA)R0UNEjQ7bnLJW@HQlu~NJrwKGWUP@064$yq6xwI?t9HyF+NY*`8 zckDugJtm?)aoX#S>W}d^SZWPSdFOJ?v%KmWH=a!;^n5wfP0rv`SA1anjQx0I+XxX7 z@^FR+Jv7tdjT5gR_lo1|r1ZWeK%R8hTjEZwYufc|wb_=8(dL0OC9fdVG`ZLn5Ux!xE)JZPVpdQJ{dMX{x zEbD~}ueuB1zo%8m;*(Z=n_OuirLPdfH&(>!wxd0NM0Xk4J#}#!9MO!(YhOd-mx(Cv zIsT<(_Wdr9@KePk?BY#J=xDxHwb;jvrn&JY*onC17a9+%*_VD*$ic_P^T*IlK8Iuy<8ag z-1iYlyaa<##`p}$#ismNm_&ZW<1AuaDNSLwR27_y2B`rPz|x3kI*QCYDR@#+Sh|8+YIr;YS79V?n2tsioAr1|zody%Ce{G-YLVX&HHW-4nc| z>$d2guZL^;W{BksL~Se6(_5m}bvXtC>n@By`@HQ$$x?yEP^YNTqT}dIMxXQY4$T)} z3Xxi#vy=Ci^~v)5+Kw?W%H3nBN)KhE_bixa?Ln#RIk(d9?^QZN8of)$N2=O!TZ@*| z0dl``(9CG6?Sq!!(f#Y7>?xFPQ{i>mp)R|OVoXG!&FEJ$u6*OrPbEg*?_c62aPSH5 zn9+S?*&RW&aJFj&-S--}kjSWB6KH&bqTDT7S!wrdXsx{P+CX-ItQO;qp3K zV(YMsx_L$4i`b@oyDU+ly6Qma{K>OTobj@J?yb$>NtxQY*IUXwt96;W`gNZV_cQ&f zhC*@b$0qd|nUlV+F&$~-d+8}=6!k_V5FnoVx7a!jyv1$hlZmM7`M?nIMA}SBi6qyq z11zia~TiFmiM$Ht0n z!$rgSyqTC& z=&`fYoctb$BUgkdZTHCAT>aUoUs|~_{95n&82sqq#5gjqf|NB-(h6vR60_2_)GCxQ zhh}pzcOb)YAZ7%zwxpq$Hc0LQRG z*Rg`K#EcNR^_LdYS>Y$Ic7L-qcShw-_guI?duDq>fM!RSlz^~OP3#ILt@?JsHHHiupO~I;*zld@U17D~q;Id}xbTYPpKtl-OgwySJK4>l0};V4q%9ml{tc##J}pnQpcz zgD&k1sy&Tamzbd`;{#jvpxFvkEoEeFG``GNNcvc0J9ccw5V4D^e`!H+4wYcpzj~`Tf-1Af!rh@1# z^;SE7SO@bX#w_?;+Cps)ZPY6t%ii~#vYbe*T2JmBPxO4az?VPv&GUEnO^@0UxhY%C zg^$h6!BVr=5@_~I(@WF6y&c8`w}$IC^YBmHwjx(+XZZ?>BZ$2x-NQAn_|_9R zTsOs{&HUyT^R2dWiqZ{>z>K(l0sHl(Sx+u+{cZKbJNb34g*wmLQwMScOh@LRLq0WrRTgC_1r>^%YJ|zNS^D zl5%cR*D9)OVqrUE8;E+*Ws}oaPKXGJnf#=h;=6GxUI<>OKW1~hE^$(Z`mUTqt@y0a z>RuM%T+VuD9h67&tjx?#h(lsVe!Dr2X*W{X9Vt)PnV;?`mi)mj>vMnBI}i>R zRX2ZEn`YK!#L84A&ErI$-)cONS(J*LvSAmM7c?66=dL+1EpFE1BYK&c0(xuhN#0KC zbH~XCrx!Oq2}>ELl%|)IW)&;~w_KIw+dh1TD4ODfDm*?^m8n>9Xy$ITO4$c>qw;T? zIVDbzh2Uzp+$l$90#A5ta18?Xvv(5__|l^#H#2eXX$)`7`jI@{QSM%X4z2RZRBJ&& zrOFPDm@j3?sSJMVzS9I5y4t7`)3#sEd0~UN`h`)MN8-$P(^l_CT4y?fb`cQuci1DY zabTX{#f6G2tVRreoIr`C& z_&pinm(iyEyW5c@#opIp2lgs@M7p8U8X)#uqTHxg4URrR)wW<=-Nz^Swt8MvC;3zh zv@F~8)ox9?*JWYc_PKXbCLt&Gn9cgQW1z6w_&3XK#%PV=5c&0AtB{uKn?tOR-G_z_ z8w|_>Tpo)u1~^BHYO{v-Ms)1IC1VTUvwk+poyNQ_U)qc3*{e`?w#P4XYDy(^2EMS$ zUGLw-Cv%>kOpq-6ak*!D_=`|=;bU2ht+N+1DJ}=wnc&ndo7i4dzmdKwor#@$f<(AC z;=6DuRv!SYH9Dzo9!v(R{}~D^v!+FPyn>YAdD{)Fj4LYV7iM0s3qTdI>k{?W#kohj zB|Njy*~dQQCj6GBozGjrIA6bY5PHbS!4}M+ca_t-_t{wewNBPkc|(j4jaEDYea1T~ zhhARM{j0fdiVk0z>JR7v6@}$64~51ZOLi`u0^wvi6n_imni5(yyanQ(QvXRFWCznV zXc>lec8t#ymaqD+Vh-=(8jel3A5vot2j9-qoxKM&@wDT5BgVPA!y9G(KepaGp349K zAAcpI6cr9dM#(%zlsy~B-h1!8(d_eaDOCfRRI1kzZo>A60|zcL`JpU5-p#HZ)3!RXB&YntcE(OSv! zQ$Tw^$*#4j&}u}oq`uLh?Ig@3W>&sgittFN6$yPBpx*JF5}jT=-(*QBtMpF2Hhn~q zOaGSks$*t9D2tix(ndz((AJqkZYOVrSCQ@rF$|OZHKV~*()A3sE^RW^ajfT=vY$M}8{*AZz)ewh#%~t|XOzsx!Yu+1G7ccw5 zelyNp+=zc@y0PzFW(m0wP|Be*!tuEw*1$7>qFB07Gsv!6-w1s@nwF zXFf<`pjU^x#c2J41H%b=Rr=0}COa96lvl~nBl|@FQ!p~GTIQYz-%Y#y*#n>j2u&>? z&p9=d1iDqdRMN7@;us&W;R>D7{uLL)FKcYCugE0{gu1#cCLlChwYD*FWy%yd|3kB& zTz=U+2ruIm-s+~ark;~aC6EGc6(s>OSoVZo3go^_N3Eur`0i@u%G+wksaF{3YHWpd z;xj|W!Mv*ZdHX82vLU zHVr*W%O_`qv}o}%;ua8qfO&de z#(b+ZR@l9YrG@cdS_mS78D^)hD!eeNdfGXHq9O-6h4=cUxHPiSyryecRU7;gzP0R_?^76r8lT7gF$Rr)36h{mx^-H!)ySih)LA4s>4NWDF0@^;ZnJqshP zZ(Ic`iCoXqt$=C@z-hjdpO|}|(2rs=ir6at)(C-hlDWq|7iy+~fyq)>SGiOum>rb9 zfv@rOV;X+u7B4mLs%|UUrSd=D%xxsO?Vc6HAiT;En$UsLN&P&_1AMH43`G{hZ2kE> z)pM@%{`*5o{*}<5qEwr-fAHW68Z^wjst{^Hh#X<>F$5nnug2$w02=(GevU`k?B-jL zXy3jAAv8Qc`Z+KW@S}E%tg;S5=F8IGV1@@5I=T-uLIZ~EZ_)SS)f*r*dBbuWTZma< ze<-P299!1{*0jl#NXyVnyJjOGEBGmgU(+}uY)pqs)fl3BoRMu`(yBT-mtLtfl62cW zIEdld{i(W=>P!X|Oy5&Q)=Cw-?0scp&a5)`i>CMAhX`E2hI0d<5wce`J?0q6gB4IV zU@|JwoZ!tO_NJt7G7s{uN;r}4u)D`LAAQbVue9@l(gL6du#1orf%7uv!PW?WqtHh3quSLz?I^v`4rF_b_xp*0#onR)L>{-+ zC|7|yl^KN!z<{t7QsJ93`{}6P~iqU z*9->&Cdodo;7B+?*~n$1W;$`aprbkoL(%%kO#dNJ+W6--}35IpFEU*~pQe za;E2OPVt%Y&9+PagqkZT?ZI8phZwC$KX@n0LX7 z)B2qMoVT(5U%tFx)Aw#T?)~%^rM&a&|KA^Ng4hfVl$25BVqKON<=+`Mpb?soO{aV$$?YqHHL6T*AbyBCof~US zF38Wva;#hM8~ndDHK0z$?Vmj6GW^j#i6~1w6$q_QwzrBr4J~yb4-sJ7+(qgyRg%ut zvrZD%@EC&e3TeH96OT^@$}vwESLD#d7JlYK6!47M>&-hbQZh92#S{=1P+t4?%J?^g z9B_>yocN!$ZO@aW`F-Wf!(yX~lb-WM<{nlmU*=Hv&fHhQrUAw8T}@w^fSJd^4tJH! z#hCHIY|bSyvr{l>mL%awgZ+zh_Ptfye#bGZOf$(-mDCDgp*$p%91r>O0h__;zV=`C3BX zTZ;8|uK|N98Fi8I*^jOY#pk+$HTt9UrNJ0c@!e;wQ=02#UA>z^#TB89`%9i~ zbV);Aw7t-gKz$!JYu7w?D|$V1a~Kpe?ZNeXnD8r5w`1P-!};f5)%ESB6^*G^E@M`1 zJsO9IhWfU}!&clD#r+@py!(s^R;)fYz_e_7WMRez_?Noh%ZiP)WV!MMXceO{8l&LzD1KAQrBQQ~6U6!d^YLNOqu9)%(abvEtKlhk$JmzRts z>yEl}h^Yj;K_vl-{)<_)xV3c-;y9NI#WcrMWg(wfO_TT%0@f_d{b3^n=cMuuIGRN} z)-I!XS66}z_}X|JC)sKzR%v}-S|`wV}acdLumwRbome%utwrZ`s zZi%em>9{`0$mtO!@$Su0p+yc2z1un#$)X6>RanOdP#G`_marnEMfFp;UlM1F2|wsg z7pXtEu)3WHv4#)i_Yr1S-*$2c4ktN{dCrKG3q8ct%`pOhCoSlA-S=~FtBVj}`1H+V zWwZqIx{SaJVm!i~^{KCS&Btjy8|Zu;u(p>m$yuK0h;{WW={io)l%6+LZ71j7EZ;)) z2stb{)V54^8hB>)E1wY5s45Ec>J4sQ?sX$rgTtxPz6D0d8u;{6kFJ~Tb;LzZdH@|Z zLS7Bl7=MJLbE0EqNTLv=Ug&V&A2S1W)Z{iLa*Uro<5spTK7Bw$(NSHSTLsBu2fBN@o&8h%4hHS(24kG}Ouo|k)yVT=G|ca;D=W(<)&Q#zS0a_W zYr*hg8`HDrU|FDKtFNtHEvB>Zh}2EPTKq)HMi72X>2jhsX(03~9{I@IBCCPIQ2ymL z3U-&viYv4Z)0Mi83&a9e$c-wI`feJ$;q{;Njs3);Mznj2LhGLG*Qt_ueKK`rAv)SP zO)yk^#gc;%f^U*$DSXxKg@*_gpdHmI-`7Sy<&RxMOqOalMYXoX0IcMHe5)ATdZAZ$ zD6I13N1`8voa*VTb{1luWc!^|;z2h*xh1IPUK`n5E@aLT1!t&ot2@d-U|~z);y!|$ zyrUlUMNQv!c@2W4(K-q}4PX<=91%ra57(&b$Z*56#EfO1(<1Wx$_2wvc8WJ}Br z%EGC|DR|gpJoy~WS19qm;cZq@z?ScskH2o28@h@eZmv^cj3iX!Od4AMY0-1|J+Wln z0O7gbv=QrtM(x!}jqT5!S~9L|qZMW?<(Fa%)j3i+V_ua2HS6A~JECQ(MiWP8u2R&_ zRue6CXERKgwx&~M`#XudlLXPhBLG^E-1D(U`yoBmt5MU#stb$&GgY)OSw5{#>GY%M z4~0dL?L7#7SAf;22Mq5qJ)e$$m^xxlYY=BQbcw%t3|5`T)Oxo@+jZ4)<5isOL+tEk zc9yu05NERAGLI!NyjIqf&fzxnA*V1;m-|qr`3|A1({u2%$D3q<43C2(bzfOud2qit zD2g;RmMsKfg&r!HNNfHc8_#zJ8Lmg^Odc}rinZ9+>=f2KJ)(lUAGtgHfJ`B>XzGf_ zDjxxNE17Qjv@oS}DRh~$d1Tv3Symq}v-i`2`-tc%QU4>0Zg&L6&QC*)U0<+h-Jj0z=lBc_|4fiaBrk6#eg{D={;s&BlF2FwC z0y8r7goGdd+Ia7q7ZTIeS2CuvL(Xzam36w1!1uHy_%#-1!;Q3f&yY>Ori)#-vWsW% z%jdLcd?HGZ4`;_)oR(LKkoj6Ajok;!6vKu3x{g0g&wN^-(ir3nbYWg4oxN$bH%Ad?#uHc*NPUVpiC2_WMlIDXPh)Z}eU{xZL_&y@Ue^46>94yS z931YJDeh_GHmyFj2;NnP+_?ym4!7}a+j$!fK*DUFA2QC5Rog{WCFN-46blJ*DveO5 zPN*~Bo=0RX?|+&q-XAsvrnG(&O4o^}^4-&UIeHj@T=z|-3EAu|cZ(ZPvkW2*sDqfm z?jLsIYVg^V;`8Z?blk*oBjwLjl^5_VnmLCdj6?@!7VJwuf#L{ftYM?{_W}mMy-%rCdj(I zx7ESE!R?ohFzV*Bu7)D)frCzf~(ads$F!Z|yEs>`nWb&rCv{`I9yNi-E>Pl%len5~zPhA5-@AmX zLBv`5^qFa+DbI~>N6Fcq_AzZT{y>9IH9-(+mV9}k4#~OSH2UaQ!D>Y-hx!peMnCz;1xS&*Vs?%77U5_tMd(U z13@~Bfh#$?7hj9d4>!NR|GohW3(IB{S!Cx3XaZnA0q;OK`*4~>$havyX`JiuHBr5V%Kf2kz)fG(H%y9j1K^+`5kB7V*tg}avZJ0pJ4EoG9PKe2zj`|4 z2JYB7QU<%O>4i&}M&z!N*fRD5aiRZH?COqesbm07#@`NPl^-;k5iP_@qbZzUrDA-e zuj2VPA{Yfoglq~7bm=S$X*nk}o;Fl~cPyC4z`$&@u5l73=1Pc$!uM|ovY~jt*Nwg* zmGkWmFeup(E8EP?wOxxNaDMcCgA#*So-{V4sS{C+@mj;p^4XWo4+8C7sRH9i%RLtO zyUS0?-UJJr6c%@)fc7H(5w0qUTiSqh%6BYxG8Iy0>J#{-BR-6E>AKG}N?^W6 zewnEBL*L=?zXb`n+ST|vfSsW zQaSd{!L}7dE~Tl~lZf*1hUl8YJ6|d48PUV}G8^gZx^$#&UPQ8|XTKKo!kJ7dUQAFr zURuP)#(~UAdT)igMX}WqZ4RSeS05E)zf^;~V%#7QT1!lQGilaaqP(<{5mQ)oBL_h{Iiw35ZNRmm8Kq#6M%{MDZ8owy9%aZjj~ zrZ+ygl9&_s;! zGW-rZ5w3Xi&HlTEWITCMJ#*-ztNvrxYPVPCFbA;)u1wd3hpC}I;xXB$mw}wPE`qAC zK7W@=)+r2N!SCfBOB#irI-xcx$?BgnJw#2;ZnM+*v-7RbI3SNM!GQsb4TEvi9mvGi zEA$jx(L;xu08mZuc9(ut+aba4)YUBBe_5aYo-iS|sk}WZhi&sE)J#oxf+UGxr}0WP z8U&{y>c=;~1S;Ad-s4(iA9^XP^szNgOrd`YKo9T$o5Tz8oQ51 zqI~d-^!JX)0sIk;n2ivPuExTsdd4w1`7&^loo~LBWL=u4yJc%U3=HxGPVm$nKovk^ zPY}?*tCW0|iJv~|6S>7rxP|V_F(CnC#SG<=>R(+7Wq8yYyr{d>_@l3w!R>~|v7#~c zjVadr5$+*+@50#^fyi3?xtN5NP=w37czK?>2M>FLs91FHGqsO8aOSQrmb?=85HUe}Hv&aX zVjz!~E=wT$EW7Z~55I66p(JupGa&Y=mpC#*a*lxL;9v{p)u*rJ$&lbx!Kbas#Z(?C|14$hS<0rUT|BZ`mri0U4H#;2KoHh&m_I5o(-a>p zpDL?(=l1I$qkiXbd;?2M!p=1 z=6=pGPY^2!GABt&df^k|5V~+v+^9nQ{u24pFB{IFn{68YK>f?_Rw3SbvXbtS|8t>n zk`b~{?Cs&P8)^1vm3(XHom4tLbNjKnH^&)OT5bEL@@}on2&StTe3O(VulzlG3Q2Hw{@1Mo@QT+`oBbAMu#3nxD6_AwPUQxZjQBz+y4{0kqZNHQ`X zq+{WM)0ts%D{q{HydQ_^Oryyb%=We9;X{!zBZz$Y!WR&fMPIN}PTc|`{aAB^`c4IV zCG8r>eW(1)r0#j8@{|{8P5>}Kc9HV@8!!Ch{~8PWy>W&MED;C5pGLlX&II!+7(v;A z)opQ58=THelSupW?{^3hUm7{50`XVl*J<}jO}%#qwymV!NhD0Nwg~1V*c};XB+kk@ zX1kx2ofK1i8W;7^3spz!IPsUTV5p{p%pdrvYGuBXdKFY znpLHBi!;Rd>P@Kh>Q`7Bw-o8B`Ai8+rQi2&m2(f(4n~)wG^I&|>r7e4CbdKVJG>^w{JMq3m5$PF((5w%z2Y z;QLJYE%rzz=G74q>Npz5hj(^b#znVozU5S!seukVa31zmYoAg0jXq68Y*YC~tSzzQ z`Rm=xR-$)9`qx{8t}cqhn3R+;dVza+Q3pyl+bsM!4CXpcog9t@M<;sZD(&;+G!#(h zL{@VG(ehrzJt#o%D7}&9j_&JO-)2TaS+f!PFCAoSASssorW4iX%P=*vJlp(Pf#wQA zC?p`719lJQLng%qCCszTNDG@42XNYgLF3@G=#T`D&$G_WQ)F<~ zMV9=J%)}~jp=3F>d6IVxMKCuYLN{2iAqN9G9t+Q~YhlJQWdiioBkzUnnP zHTkl_DZ`=s)8`_Ct{Km(|p~Nr6CzYMd?A2S@vv(y*LmAk1m9DD`e<|jcE%7(0O^XFGatko6 z1>){vgp?hUQ=>~m3xNO+gw8TNJ^mIXSr_%r>%@LDMBE-Pc}Ou5w6b6Bm*QS;UhWOw zPyL_~1E1)l{9e+&bfeP#vAtvqrP>UyD#~8J8ZxZak^s!c(NW@~ZvDXmz$&ZzO1V>1 zvgJk1>`CaAGgckCBZGbi4$7qo$Ki36-Mw2E)fix~ifqtRUgI1;G+Vl%5NA^B^U|{_ z7D5C#O)av}B!mFrHokHuSE8hx2w)5%SyqWjkD5%Q??;+I9A2uApqOKx$EOyXDG16u zQ!lMfOKOZF2_>swFQ|CU0NuWOZM}{mvOUylTvoE~du8WiT$iUt^KFriIiI20rn0iK zdzdNY*|@-WC(00U@+)e!U)EU>q+jwJ(&iQ34N95qB;5`T-F>-2ok{mT2}dxP@4LWp z64)W5({Qq@N75o!{n}KYPgktY+B(^q@&WW|X2R3D3;t#|8oqWB`l`@xM6gnGz#qr?6_A76ce>6{NE7*gFa+AX5~a` z7Xzh#D_ieVaNH}{{!_~GPrm{nt;R}QZ3S}@w*S9XtaIJW zd>e=p@m#Y<&($VYPt)<>e6^I0j*WF?lC+5%Wd+cAJhqo(Cv?~S5`GAd@c-3Qz}7k} z#193}(BJXmbAA8o#dHLcyrx+;Cc`Wu#tJdzD@M;SZt>*`U9nE;nYLyjjqyc#-d`9y zL@Y({LH~chpj3W3IK|hEUJ99D-<>9MWj?7PLhLM|)CXwKf)}Q}PGncb@4_N?VK=S- z<5-$|9vb%RhT;e*%c($gA; zh$Q&c^{;pV4sO_nbRn}dRAYZ(_0AJ)FD9yndMq%jnKEu;#|$z`w8vk!*8<>1N!%$;uy*&%yQH$ zD)1eCzE{|tQ1GeFHLW-2OvB}K+gcrZ*P)36T@8$}Tw~&5+uZ1!m_lt8)ln!2k7Q0B zIMpI%Uw^QDne^DRM(F6rXNT>kXA@&*(+MWbm~79Lq9!1g<0uBkePVAPo6PRfNWcrxh|d8%ZosBgx0u^^_lu}< z9~`gNamVWMb%ptC$2>%FBD*0sgik+FZTn@t0Zy{m-aug9cW?7eNTIQG2zsC-0O7e& z?GP5jUaP(;j9kV@CDqYd&ZIFMYL_O#^$|z2BaaBG6_IiC#6qV}*YAq0517`b@RY2W zV0(1{9o0@g?X1#HZVvrxcQ0N0`CUc$A>vhp7$TX*{|O&z2}A7?65CKrSTD1))1t++>;O!DoK!n0dnBOl7$?PrkRdM5O@5B`046@_r9ZS_$}dU1 z(L||_i`ERE8Z2eXPo(Qo?6sy`YudU4I4LVQttpR2b|&%@p{4^5If*ANDIoLh6wqri z%qkp(H1C}J62fGHd)zsN-4m(!7Jd1AhVJrhBCHi6Lbgew6744j!|phj z@l8c#oY!dq;0!F-anjg8;J*q4cr~DE^fkrLGvxKOg`aru-1&mlC|Bc<5V0GdH9gVSn^kcrEEh3&IDI=lxF4 z!{p;7=-r9Zg-%pXD>Y5BZF_40eByzy1H>GU^nNgHZS2wX2hQ`;J=bL)fXpLzj^c&e z0X%4Nnkzpyxn^3WmrHm6ASQy=kNRx}{y?P>s4^-a?8mlgj3t2{`R%tS+Fd0?$c%T) zTbH3SQG6RTH%zY*f#*aM-YbzbtpL>*S;r)gGpf^XTEnst37&QXorUS?{(B0B+j(syFgUC8aT8I^%)QbfAA6U3fn=doUUc;3JA3IHSnI+y-QVg+R0|IBio zV|s#lODtT>7Uhn172XW*@1)(&^i zPBIdgA19i9Ks^wW7F9GvT5unBLAeuAOYuvdH($I+4H(jqm;r?|mV>~*5onrlk_0Ou zK$R`TE9jABFA>ov`)iO<1-w)ndt=bmf&uQPA47@uH(5WGTk8(%I6jVFN)B@`PjDhnBGqD@PzZPTvZbB~;ZJ*$ zn00y{$yN!=cTP7Z-L137bBKnuQMx_27X-7ji!(^d8eR&=0{1=UF!JGlFm%gp+lL2o z^b;ySHPKmtCL$XmZ2IM3zvid*n_vUcq2b0YGU~a4Z``;p^5UkRQ1(ow!M4(V>zWU_Lom_HDal>BZEL~ej&Gva1{YkR}pdTz#FNKP{s9LLQ<0;9u6 z3_V7F#GD8&?C3N*Sq-Ky+3KcVj#2V~p=lZFu+MZf(Oavm?aTS+YBia!rJQ2U34xI{K&@MSP(9(<3<9 z<1_TQdia^X`Y28QwVX#B^#bK~-{Lick}0PQJ+EgGuZ`#MNF`DE{%Xd|yB2L#UgKN> z`FoQVKdBYFBf>b4cHbLA%B1M3Sig}eblf1)SoU^vTj4TwR-2DzO;lD_UvoI#`!b{ctOyXU zkrQrAFKBTSsQF_n<}Avb6S#4KW!Ebsco38}Z1&kP#S3#Ug7JHS>K_w`wsmDZnD@2^_Wy=TI4j}U%&v;ef`J2?3ansi`7O6Hhq2pp=l zK;#+8uicn-7UEJ+IrKqZTdR}0RaPasrfqbN1|E&OM?(Z(D9g)B9cE}whZis(w;Y|o zJD8a<}S6 zRljaWxkMp@=yF) zz-h2$neN`J25$dT@Qw?osCQW+m#c!+KWp{5dT(PyJr~X>yL?Z@_;}+i9~t=0lk+Lg z={}cr_p3kLw|7{~uP#h0bqOTb`79GI#Bn36LRD22UE`*|3qFWREL>Xa$Q1M1Jnhq> znJFx&4g5oh1(uNOT^N9nBHXdX!BLmeg9iWf-D-*L#f2gT#vF`=rsS4h&0(!aHT`?LC+p%#hvcy5G_rcU zE{iVUyf>3d-z^XFKgx{?*yX-GW!Cx?T#QWP)p+0%-o7q!NPd||NO|}tv~6O-L2zJ^ zkm{T8D8?gyP*G6y@|b$3w$N|0|O z211%%JdOT?v-G%hW>w+Oam%4~kJl7=4=2@OR|(PF%YHC1kSYkuYLc;Xh=)e+Ff`w7 zPK34ZIENb~e#kG_&w_cvEJ2lqopZjh%m?EsVYVgW#s}rDNtwm2aKgAYvuJ8DR<;{4 zk}Qqk&z`x?o(-!F??dH%mqo|Syx(=(X;5~mP_`GQhH;<-@arDdmk@$>EYPeTerO3F|>8syK8a`*Yvwd-yMGy1fuDQGyb&#~cDPee=GyXsGvPSKSYrN^2W&x^&alTytgdy> zi+|KpB(1nsL{-eb)rL*`$q_YSUV6z%Q2$f;+O*o5zc@+a7mz&1|zlvY?*XfSX{JIerv!2fUw)EtF zcUjEU(?;7f8`_^?6Qc; zNMhg)Yz9_vd@X?sb$8g}Zl!dh&J1RV&<`)jPf8ITrmpf_cKPu7b0PAr|A~MvY=rm~ zrI1^1JH;8X2IGB$$H|n8vA&!H048K~i&G>waGZtYWOPy*jmg@Ny>Czoi>;U|P(tSC z89>za2HR?eOy=pEizSC-2WLGBa|9*Rj&cTIq_g?L%AJQTTFK$P;_seuQTeCirK#65 z){*l5Hv~WT-&IoiUGUe><6CY+`K; zSlJ|KAdSW97*#?_EKUyBhmkE_CFJDhCFF;T6;je0-VY0NkL5=lliZBOJu6-#jB2`&9qrw$m&wD zQ;Y6saD3j9@)wQrUEMX6lNw?0P_a|B*RQZoni)JK#2LvmR}C)m4py*#kT-cH!tAj; zl6m!0QFo7$UD1p5P9XhL3ga!7c{4>au_0A3{LC1{vJ)Z1SY^IFXy>;}00X6SzO9Gx zLEdKCov5Y1L}cENh#!-x?fu7;+-!x_=`L{SbOQcT>K!lz@T4UhRl4P1lU?1%$j+D&}K9(QK{>Jg+FN_1Ubiu*Tqtx$@?B7DO zh{~D;NDdw3_#V`Fon%M4t;OZ!etQA8rOzVCZY~kMd@Z^&+`oe*{yd9ymblj4RH?Ne6C(%rnOl4mic%%pKTO*yHu;6tBA%)-@ zmxUJdyJ+IRoCjUNCZFirfY%^Ei+j_G#b1ZIO6FEx>WB_ge&)VRWVl(Yh_rF^gOJQ+ z9q07M8j~@h?_sm~mUCR*x%`^9ib$FW#~>Q3IaUrba9h`{*!HzKAACj@dfsVhzF87o zenA|Gd0>m^xH7M?M*UTw6Kw#koSU6(WnG^nwiGH}cqMx_Ew#8$##Eg~SE-@y6PO}q z>qU|H6Dv7>5z6#?^Z0s+lcwh5lS{R?3%?alNnopH*dRpPXw)%Ve)1F1Id6F5XuL&+ ztk?PdsL@RLQVe@%)s8=RinO}6vQ#mI9^#78<$E&BvUc&FZi-luOTl?&z(li#Oz?_!O_M20isk0sI*A1X4DqIt#nt872}J^i9}W& zspbxI?DX&cdNX@Ju;OX)Yg1VfP%BtewYK1G^s2ST_}OLHllo|Un?si7udpwm1z6i- z^+);tO#C8^ukU|{1k|CYCCQ(=wb-~!dxcTC5&Sx*) zQ~KvsB#r`Z4%NDG@+^r2zqa{TT^JwPBQi0tulj0ku?G1tbDH{7EBEa8o;-C!%yvRx zvy^PWd4Rnq0dmsts@H!&^^(|Qot<|+MRA!gwjZUcqaFtO z5c{f*xL4BJo|p~~iz0SC!~W3c$(&l4;o&88qev+06BW)-lG?spRcsSU?Z4AU<>jwL z%w^9-`9;_(^?&SgwDe10_x0|iwkzfKoi%}$z(Xop_e#SemGiD-m2SLQ-?<~Nt0nlv zIl;sy(5-tru#qrxE`yc*Zt94JaWvugDA3rrs1Qs3({u%6#id@g#|(8vVL{CKubWxf zyIKe*^FW(X@t%0hn;P5fwz;;#u^ye?!440Xks6WL=GdR~s@as;WjYE`)ZD7T0aqrK&fj*~Qy+77=Zh4Vz8k(l8y&d$#?zo_PABJ-pbGzg0 zhaF%Ruy_T(qYEgGn#uem&%!%ELz&~{KW!vDV>e-d5>BS&s&%S^A3Mnh4qt0;u-=QY}>XZUV<*a?X`_~+|*)* z{zQQ*xRS40pImljX}f|WPqCqCV#QgWYa_^@dmr#^O%;Uui6}>=>h{K5G0{sjZUnSd z5zvzYh$erFGSer_kgg>=(1WQMY|@KzcOf7a8q&=wEbO6nnGeC#fH_$ zYkzp>x86FzN^FX(nmzaFQC<$=a9MJ`FPsjk?d8-*He382ZxJ>)OP2~+~*>`L;?q;A%$X6@u zwS_Z;*REB^p76`j_dUg#^F5;7 zOB91_?M|}iDFb|mJkfJK{`XyH<6K{*9sob9|E+#6Sl0hK;Xe4mU{#- z{2VAT%@)eh>a_Rmr9q+cM_noNxm}0shQC1JectMFHOM57ce;MAVbPz-Dc5Voeo*rq z8}WGzMpN0fUcb=ouNlQ<-=*r``W5h3M_^^B{1xCSu)pfLcgqUEe$Ner>K^FZ=d)J+5J4lbW^0bFi}I)<{4`*u-NI|1KI}nSLu6} zzlH>gtM+i#?TM{^qIK00joPiLFKXPmf1M4yGt3W`b#MC%~<~&nkt6M@2>TO;1t_xo-@a|KW*i zfKKTvx_R&~B0ctax#<9cuhY3+!HpsV)*`sVr#zy^*WZ5uprGtm7kdq-$KZ}jK^SBN zoC(dk@)_y~$x3$fM~`R0!7ME9rhZ2#5vSqqLO1l#=N-F50}*J|&S_}?8mVHKbtLk5 zsD(VyewJAD;c3Inq<`GS^Gpo{B5SLA9>nLG8GF3*uvf_GN18n$TjF%Ic!1I|HJP7d zdgV%-+YpoJxFYJi#IT3JU&aiUA&y2bwR!7hSUzlcU; zf<6T`V`FQZ^^yGUK&PsR? zCcX5CBz1MM#8M5I$vdTKP)aC$g+T^O(*CFGvh7;`P#2$=|q>*6&aN^0V*c zX&yNUB;a;-+a`o-b^7we(ii0J`fKZp2aLl6WRIsguHL!uhySQro#zcyQ<73yBwr*r zE)hro2+-)lx=h4Q_>rEuc?RSk+a%|4{k>{>VycyH)8CJeju3c!H;V{8|9(}eYy)^d zok7xFpWh|)l_5aVF{K>+dtIDh>RMz=2eY$fGBgLw$q8UH^*WzC=50xW>!I_(tbbDZ zY{vUr3-f_Ss+hQd`QO)@oY8-9) zX=JQ*NF+~e*hNPh)-nrl0teS)Xrh2cQN7?A?BQAIHK1i>@^YUB5Vp6pYyG>XjP5cZ zS!VylKf5sNz&BL0O>^#dEo{wC+-$pispdmRKxh}sU*Bew5%~T}eXd`BSrAtX8=z*v zFMIp6QR0MM@$jSlk)%ei!gg-&&TpYDPTaXA{G0!OZV3B#9PqTGdIn*8O@qb9clQ80 z;AO14*)d&`MzDVV&IC5#S|652{v{>VpaK0lDCV{|U|=Vu>Cb2l^-(BG zyC)+a;Y4cRjLnkZo_UHKjX+{}g)gsp%!Ox$tJq-6akn-SfLGJKnL~Sx+XdOwqlX zDXxyC;*oi@Vv^MQWDuKQ4lXWDX_YhmxkQS;^M~T9xIYzgS#0e7SI5rBz0#!0*-8`2 zxr?a!^p}SB&Dck~*-VfALxcED++)0yCv7)8&!ge}b#ovKY0;+f=l}oUzwNzD>j^M^InGX^+pxkvXTaP&PLiNJWPBMFPwkp zM;Gl!`V02!!v#bl;Z+OczxNBg5-+?8STps{KyL%KNJNW!AD2-fo}4Qfe(^S3Rp5F? z1{5y;TRb+?X+SH;5M+|j_aENHE(yC37P?%(UMXwG{b$+2n^&3}+Pb>3UoA0JZ5oTEYlHbh6w+a!b{{E zVs+Hhc2PX%qHPOE5s{8M52l$C>lkJCbGTwlDs?+uNuNhtvkE5sTa&2-eVv_468Sw7 z5|Kucu_=52O6}mlP}S8BF%i(sQzpZjNfE$FeE8c*th_>IDJzLxZSQIshwY!Ck2gB`A-|Lfsv(lTm~hdT zDSfqjqLgbK{3lk9w0~*=?0{Id#egM5FY-J8YB+JabzI5t!xrkQy>OzQ@kO8gQO(Z> z2!}I@y-6v;GhcxM#7F)6Y=kqd7_I%z9l06PkGb6ES{A| zGPrWeI8VOT?T<5=h%I7f=1KfctpO^;h|q%}gST$H_&iqB&9OlO)}8j}Y`kP5@uZ)e9Lcl}_bP zKWV+qfj@EKMdD|XsW2JwuGnrq^Vjk zQP=wDbyS8)ufMxhCgk=_$WhmLgUXB{)!#n{1pop66P!G!TxpF%*uNooKqkw0@fCP=b4yo z4y{Y!9A%OT|5pd7+yh(%RncP`IQ;p% zf5X+EA@<21i6_+WJ<(03u!uTeuT(-LT z;KFJ;Gxa@>R@`Sy8IA`*h)9DXBAqjIr=)b3ibzTbNOyM(U4nFX z4jsbK$k6be!F#>G|N53|v0O`e-}jsyXYc1cdvA^jQ~5Eav+V5bSV#2wM44UJ^O2mu zQnwNl8D-@a%)Bi-Fuu{4&-LhcQkbGfg;UMma5neB%tvB`Ig)$aJcieIsViEMQ@7cl z5;fPZG3|uPg}{dmb>2}P20C5p@-LtALgm-FrScz#jT?Hwl)7Sh%49~;!d$P8BTPnS zoKnV_+2W_&&t2QR$=&WB@rHl`}>BQZZE-&3%GEoBVZnH65)^1-%R<%06vrV?JUdknUMCr^~e}v8%iUI7Zw6l zg?;5e`t>`vbR25_@|bHmW?->&!W^XrgFWxYo5al6Ad}|Zetj*rqaW=02n z^nO$bu9Ab>4v^HO~7{;BW3n{KspA1J7XwSs5$Y zzVUZqcTcn(gJq@15M}RRcO~STx+tSdQ+k9G)!_VV1&vGDaym;z6%1z9Wi0_QGc+b!hNLWfKn{i)&s5Eqp7ZKojfT*Ppe49-Bw5L`9apsD5)|CNsl!7w7O z>GN9EGV|xp47*Pn1z3Vtn)Aw_RZ#Zt2NKyz{T1jy+Zw9%xko&AsRYpO0}^kS$9_XC z8?~X~x;Iqh5OyC*$}%`Gk?Kuq)!MLS_%mM7q>mL$qSHX9oC=(6e;xBCbNWTem#S%o zl8WIh{MGeh+SGcZV%bg}+r`l@^7ao&GV={ z0574R)GSgLVa*r!@E2F_6=BU`MnJ|vXYQ$V_$4JJ$t2b&t68;CtM7esJ0DWimxhQ3(5|xK?INckumwkD7Qg?p3xqI~%sgg;HOq0;8euVn) zVP4h7YW49()&3!HM=_ob5t`@sti;^#Gyv4&<$ef{&pg~jk-7Bh;C3`_m zdvEToGQ+n=+%`rQ*)|S9XlYq9Za(g@Zl^~>SwZf7VBnBa?f#VWe(jtuL*>fKsj=CB zmgd>IdG&U4;AYLfMr7q&q#lHFRv-SJB(7A^*IY-_{hz{EZ zFA6aUg4Xf|FiQO+Ct8%@;e`A7sk7>$KlBDo79w)Xk@XP^LS)vC=AlL7v+1 zWt-fHwtPNM)tes2ohx72E2RkG-&F@Jv=;lN_+rLaOBmwNB2^8M&!P&V2TkRERnn82 zRATL$7~8dKTmc!U$NZw_Z6Tbm-(Fq!ZKEY9J`u!H^`0+=t92B|JqcBo?MfMsvtk^%t%WS|vrtJ&j+@uix5a;myrUtjvH37V>SADSO`0 zXwk@peTK>`txV!ufD?-OOutPc-u>)QJ%@qX6ug6V%GCX5R6;!pd)}32+|kMkcir)ToW_zNbnnqr#$^;(EF(C4_MGVNR?qLLcH2 z+|qu$zYd!%X_F=lwWU*9_Z$d8(gg2;!|#X(F*UV^;?LP=+Yr;^`KN4xq7`eei|k{J z@D64&^`C9N{r>XiirH24t%ES%P(GCLJ5+9&#?Ib|W?AXWfaKgP_bs8FIQmTyybMDf~%_PhG8Po6FI%A(ZWb+vDeAw#Hv@kJb2HNo%G0@%&dX z<4}Wg|BhQeTr!p8O~B4bN{_jNK()aeL8h$g`00nz`d;xhk!_bn{(v1|p7B(K>Z*Mg z*}s7%CYPaz##Uf0RMOYLae7!SS>mF)FT=s{Q;Z>3!9?0aweHB+tdC>768_%eiqj@= zuM9R2JyaHsNaQ0C>G=Go-7iZ{KP_iM8eB+ztA^3YekInB#;KN7W&IQ}kzLvQ&Sd_o zQFQ>6 zGiwPw6(;Xj@gj)8w;8V_=SvkQqbPa5A@p*#J8phzWUTPmD*MYWY%0M!Q-d*FR8c0KSY*U-=!AzXpo?^N!4H_{d$V zZezxFEYG8RH^epFPxmr%8KzwKk-ZN0XXX1mCd7X~u_qxRr(#p~^6o0$h^LG=1(9lE z)D^_Z+wyYHD9CVp+U21rRgi_TJOO8%gH3Jt9&{GG&MKL zEhfy$BoADu2);=bx9aMz)&$sUtNW%NJ7}LvsBh}^EQ8QbA@+54>cf4i#2)X{Rs@{! zp%IS(^&854&*sw1g5j5|{noh>hF+(!f~IW1ipI<_6#Hr z+|`|F&nYbXT7#)ny_dZ-%}j4XQij2mOx9NQRyL$nHQN`&=`=$_K8O=K78Vv#0XLW8 zxT&)FiO{v|yh>w9PU#gF#48(E!=MU&cp6!HYFAX$ro)6Xv3ew(ZrAUO~#aOE@5 z$VXuhBq@laIX{-Wct`=;;Tn~aQ|f2ZS6o=WKh=KKMq*KY2eB6hWmZ-7%M!Z9^(ikU z#M5ldw#F*Xe^QIdIgj*W@GylmcGO;nd)$4?zvDfDhgbl&-4aFxC}vE=?gM47ehEwn zKOkCm**yw*^-5X0hoXjKs%h1F>24fleol@J$E5?1&yXW#22|v4knxtkTzsS3I!^PA zoM8>-^(K^oB8Tk_QXBRHVUpgw^PplfFdZ*6rDvorUgGlW+3^I?Z)BMJnvM0qsmRgr z^gEUY>Me3W6h_oCNrnFh3Npg5Z&VJj-aGtF`zCOHr=*WKdqM20<&0qJ;JMf@{{oSxVC9$Q?X1d9b0p(+}0;o$6S_U@UxMv)o!zzB~rc~vP# zYfYtDR8(i_ICJDsa^3`JM#gPRRrc#}Io{%h{l;WNcPx*y*VM4&q#sM|s1&l9sdO)C z!z0p@aH}>b?3s!V_Qp=oW6WwiY49AnZbX;!MgQ;Zh2X90riD%S!jP4=*^a=US2L$7 z%+3tzE07?njb!K(9sTNn+V@93GjwuSRT^Q3zt8t%SoN-hYPg>ja_;?f${4f* z*_RnS`>OK?*PVs7E|$xIVSQf?hw+l>mr>J_G@UuG*jywapOz%N^wRR}N1yg7R+SE? z4tQL?VjSwj?|Q^u;mqbf#a_Sff{25j^thi5bPG&x%#Rft5|i7FYTx*^?tT5B~k-?lPwPgYX9#B62fy>nngX%1M{ z%#Tsy8;rnnDSX+{uix;04L+POh&F?cDCq{YgQ!2_qx+b=G~ZC3*+i)=Ci~VocS-xm zln_5VwVWYEKUzU;|7S^0Xl7h1s7pbW$=?N16eM}nq3~=4LU^d-8R85yYUG=K>Y#45 zLs`~-YGN{vv(sb6{EDThn@2N>FF1#sD65aaAx!|&0oiNv7RaXQVCP3Sp}Y-W#-%f; zy=;{saU-KB3@lLshV=|O>7qDF)#kN7H4iwPpYBKO2MmqA=~6QH8Z50KO}-Tu)0#J+ zo#r`YveXl=NtBl(_Jn+_T#7r-9=blPVv?J(>8+sYeTV8zC;2l22@~NAuaGcT)R}2E z^XxsuyMfQ~sLZR<^JgbatX7%Au!n7<&oSZym979C#Ub`#{0nsh35#AvU2Q*FaR&g1 zkk`g=kxt7adw*s6LQF{B$9Z2L{ks-sp!9egw z0+HluGO$P%^Y_Zi$pBE-@vB@`o1U`LycG6?7wV$VRbR~WY)!K$2F~{OSJ#-&XFd|No96gg)SV8iz<;jE836yInUojlTX!($ z2Y)D@9DwK?)%ti-Cwy4dUS>Jge##UQ7h!aJ`whcgo0AX`GRKtpW)cfEx0CG_7E|Ub zs2;R*%E<)9mMxJhDMm6e4NPKq33y@)_V_~dGo#*wr~zpK*SyLeXaz8Qo@SLEf$6I- zbh^#=n`FG4gGEbiwDG1W0r)}*f`V)NC!zD3Zs~nzlgqtez^w{m<6TJzwT4E&WsUs1 zk!;2MO5l|o&_8_$#q(;9R=c;msnt6av`c>HyPu*OXyY7`YG5VGigFJ-t7ue+tybEv z97{%e)Wvvjq{ub52VOX36}kHJ#o>^{nbH5OoQpz6UcOO2p6mUib)vz5&bj>BqpUrU z1y2kgohwqmam%>%g$Jz2Tmsa&K`2QrR#TA&3-%so4&} z)TR0M2J?o7ZaSIo?(720>>hv-8yRjitTUN5{{-Tw*6NPoY64ST9jYQUoHAqiKl=Il zMm0_OPZqp;hFwh_a#Yc7Z}p5~jOZc?<;0S8m%S;4ZoU4$ezWA02i{ZSYzl;+SDkte z&Zo*MALaW7(v~K%cnJ6}PVI(IcTs^^g}m8S&~)aEw2TirZD6-XVEL|tHS*VBF(pSJ zO7_hOLjOwsx?vu6V;B;uaz$cTYnD~jp53mxb&*>mwX!?X;EHN?NK!uD`zjSbY`&YC zDfj47;e|!BpGorc)HO0MxqpVLF1_BMirmk_R?fnUYl}83Rc-?AYuYwDw{+FI@#YJg z6AXNj4gkL2)8}VpYn{(}){;J~omo_GzfnacF<6~vaulP|cH<03Bu4V50VDzkF=X|4 z-R*yS0SNFMRxQW7*V=$^1eM#cw_z2Twek7h%Cn$tF@Jd}F8x|45uO6;u!wpXqL5V) zV>@??RSUE{6_rTdK@R!*P-bDctHbX~q5g9kn{Rns(G#u!ZKa0)*aE3a1ht8xlF~2W zlWOQ!Kb!Rdn;cVSN`x0#lX6+a;Oyc@SAfQ9ni02~v=NAnXPHfqVtIW*a@H-`n^NP<;s1*)#6=1DwMzR=?Z> zg4M8_bKalq^k^@;)HnXy`u9ir5`xohUbFX1oFor>y8L5tul-?@)YHzKQxCm}7X3F; z1bFgY!rp-ic?FsNiD9_FnJfFH?ZKa=5~nKA>IczBHAO8F*Zs)_YL&vSrtU6vF(~+X ziIFH$$s@;pea8V%) zF*S$CD~QCuoB8(G>+f3*exE>*&7EFv*3+I;${* zfGp4ZuE|Ow{G~O!oA~4~olnda)X#gp`%5#U;?cFQk)C|1x!&VszhhkrYD44AkETZN zUMQod7H;ods{13JYX8M%6hZq~!*V~8s3~%s6V&aIhz3ge7IMz_TDg=<9xFZzR4{|# zhe97?M>6+Be-*5rWeMjA<%abKvDp1#3kZhdRI`-PYMWLYpnk zzW;2KPbs0j{3I`+^VFp40VW)9qynkql$x>Bg_p+MPP~owP}B-3r7F<9`0SGFgF{0V z15!M>(ajGsxt6}`-b-Bg8 zJqdr)N~?tn#s@_lxH{anQTh)O`V*B323`jsCEq#*{^ohtpE3u$K;>zMZ1Q-F%TAj$ zxB!VN`Y~Yb&uAjmGJRL*%HT7aH^XXNY!|h0x1~W84dU!XeS*JXTu(EjbgDoA07JxHndzB}dy z?FYfnI2ai0s||tKJ8#(R^5aHtf^*JB<(c; zJgS5KUW|en5=1hOef9@wuQ(gd;ymfV9mEH`9b3^x!bul4FxCF@CQsxsCl3on#l>l0 zL~e?6qQFslU&O`yU7#zp%-9v?i&@p3*4!o(WVqKl9HfkMzPi$V=& z=fW<6jKP=6F}0ul`*crU!Y$(OfRq@g?jrRf9YYllyU%6&jw_P)u8jDN{21>-vz=cJ z)>)W@C=lO4)11pKE{5!X!!+;@q?g@aus1a|`CAaS_zL4tRPx17<|moUShR2g(^&+JU=NbP=CiE~LP&2)xA-UJ+)B`~SR)1dGURso8dcaYqU4 z25;>_45pt-P$RhBS=ee?9Q;=T7dqzo8s`MrO9W_mSx{NdVIi>E%C9w6WR;xg)r#G zZSi_dt-B_VmcT$mIIe@P&fxH{+(9HH!e4?CAOI>l4vW0cpFg){U|xLmXD`O+fgtn^ z(QOwAHRc%}fU-7TY6kdgLCm<0FDbY#Z*B@zhKk(^MBKsk^*H~5)m(bL9rl>~&ypcq zZlY!zfusLUarkG`zzJ3ze*Xq;%+N@1=W^jCKs3g1+kg-BB*hS7y;+#k|DFhXZvn#= ztn-2!-pYYd+jZYy&HuB}fEuik4OP)ex%Rz= zGU)QI-<>$YFl z%3_g+Vt3;xM}ZBQ^j=R2JfIl< zK6ie02*q7IAPSgJ#dCe#(!V^+KXProdD%T13Ru1YChf`#sU9PDKeDp!>>OG5V-!aU zLskoo2n+qooo0j1K)!#Ah}ivY_hyCbK2|2-R@qMwR}5AZ(PFdF*Sp4ungw3?pdfl6 z7&huI%YPK;LjNz;2nhjfb8kpa~W%lWd_;e zZx(t=9Qy1+Y)Sy)zZ#GISLZwpt{=o7nGOez-UnPTnyp?Tg52o(?wUFR(93PUG|Oor zSOhMq<(>mtlP78$J+dWBqlP` zCqj-~DC%JLQw^Noyq8xPexCrq1`k)-=$-$lAO!-;vhGCAnGXHecqxL+u+*#U_!Oko zEWZ1?OoUDVeDYDSuWKOzh(@Kf4z^~=P5f#f790yfKIIYVn&gu_gvqfxe3*a==1{(# z&;Zn+Lw8p64^!~pLp`7mv#%16oqX7sy=8f87TK~5r2;Y6cJZK`t~ckg_MJa&3xoxC zsh&oB@c(O)KM0jdjQVz=3l7RSG>y88-RFbL4PN-kr9WkUER#WrPTxv<)e@Nq05{-S zg0Gf%Y)gn7NVb)>c;Lu>UZbY2Uei#M|}jF0avHAL~dQ#A8`$2fCf2?-dR7hLr1{QG^l z*so?j5lVf}_n%Du5G11T%X0VSwMPoy^qTQXkiz55P?k9UomMJZ+6umG@J3(k&o@cX ziFSLnN$Gemrw2kuX%FnzfxiH^${N(4p}cm3PMXFfCSlJV0gkhORHh5^6gFBJMMcA~ z;OvNR-(2!GNBELRxeUbp&rg;GvVVDgkVc$7R(SI!Socqt87&MAc`fy^?q8#MiVZBp zTwZD^yLY~`?rbmLB5S5tm6FSO#`9VZ!P#;NGEAi|tGfE6@^z{1BOoqNXD6g??mXi9 zZhl}g7tgoKRd>k2E5345DC~aabdN0c5{XKkiECaS*LukS6qW7>dOX~~&v@RfnC0Q1 z?7g1hAiBSGc*c0ZKn3)^ru}X9mP}KN5wZhi6e|j{v)Z1zNB`E6CtVT!1#7XDdXcds4k3IOPYw?UKTg$qm)NH~n)6uM97j|%1o96v_dZbRnbxc13CfqsC1R^yh zjHPM*3d%pe#RFQEqvLsjrC)j8lkGpaZ*k3)wl2YWvN8cL?A&PT6G->j?tLcsdVxcD z&VXyzJ0~W9qx~oQ1a=GS77kJcDnohPpv=Y+%FWu zcxV+LS+(A<`~0*vZU1^*nIf`7QTD+KvGzr#O+kX;4S&vl{r46efC7_A4PSB2axy#? zHt|rZ0^itOJ!z`NZsuwxrq{mnlHF{?$9j3Ex8{NJrH5b|RXqH@rI*qdK?9syJZlBg zwm-{GM274O0+eB^(IY)C%tGB6HL4E;?(91JsY(%?!C1lxHRZ?uu>@d#9Ex1qD!NZX z7$6ywx%>y|G)2rPVB1UBgAeB=VG=Ikt#=Spn?x=&lug{YdTJDz?Qi)M-Y;IKjgg7n zRJm z*{uiS-iMeNS4Gz)CAltI>sB*{Wko<+&Yb-Rfw)Zv&ddK(TXoy?vPnQ9R+l?+1tvE2|n)Kllftwi<3a)Qo8kgZVUG(p^S+?!SC1; z?P}W+m$mb={W_8P+{S-q#eKlk<^<*6a{QaHW6`}|D{SeOEuAZEmho(>d*^kh-cKa_ z_Ltk>If^!QZVy;_e#3X&ooBs>$$U!v9YJ(U3YgC09rKHC_IDsXyt@)+)wch@3W7lx z5$+5$@wfQF`!40Yp79r!RN=TRqysun|DxVLGCewXT z4EhXdVHpQ(Cl&p+eO>AXW4M9GPbK5vCn$rHgAn$wR?jPZx~jvd%kk&vQpRwP;c7H& ze``jtr4uSEk7yQ+g9Qwsua%SF4@(-hlAi{E>#fm*x--+&(?uJBd0-zP;=4i4e{EJ4hZC>tLfz&xg_ps$^Bh^HdEY~$j)cQ> z)P48fN5Vf5IKAhG7;X}&KdrkiK71a-S|7E2NItKIXg^5DlQ|3CzVTiV(~`^Iuxz@h zG@-maYf_Tlw5FsqRa$>HYqrd_?hh-WqXN$ECB|12^H?ncHS#SS!d{`_kIHA#v0zksyoB6Z5m7H= z;kih=)`PiVT|-Df^n?Q$o^M2Dst}%<=T6n{^yU$ZhKFFjZz_&?kTuS(InPW$Z%-i2 znwR8-dh94AaDMHHmH$(Liir!~dU`;gnhz4jivY9f!=u89TcfPQ4zCOe78b|H8!x%l z?C1{LNnyxRtyBzV@QqJAnj0$$VSm2yf`CEJmf9|b{n3*%`XVAvJ?t!X{cJYH;XJo! zNd>L=&Lw(Pd#P=5^QC@%bNKAR51*x%ko$WrETioD54JboR^t4=s z7Bnm-(OY3)_ov~@AMH@a6~*Q7TFYZk3nZz45Th`t<`3sPUBmOm7ylOg{-sXG5`A#; z$RNxMF0NS{!Hu{Y{Dtq4)GLYuEF63kR4rZqaPr-dc3wfkqOWSs&DI&LA?+tY&@AFA zg7brXV?2DM)JFnoEx!fDNJaieSN@gfs5xKO;*ezklwkJxddXxN^yFV72L&YelvLL2 zWM|`EWfsf?(QMKQlXSDhWHmHnlZV68gbtbCT6pGK={AW1tn=5y;^^PF88Q|=D$fu+ zTIR)5qWvTd{2)WdNl}C&#@F$CTW^P-kZN%B;6nUPLk7WE!uA1ee2HK zkAk+M6O~)$$F?d+rtBH~p6eT^q;|)|P6!cE<>lCsx8iS9i6gxBRmtN1sdfXh19vZP z{A~Z4qz5izZg#YWjM75873B&8Y6YJ}JF>dcgX@O;SBB#q4Vy?J1=vAa0Xpu!XGy;us{Rkc zEog`TWOPZON;+inZ;=Jaw`j4BfAN0p2(?~hztgIJd1Fznwi5KU_#=@g)WEmNspb6F z_~Ade<=?+W-UXri|MKa%(2rxLv^BKc>%tz?)U2yrIDQvYru zFfOAkL1(ja5&0JeJRrICzqD_q#Vq`uZRhkm`J@UeZ-re0uJUYeN|n64Ow+j~j3x7Z zY^9A=dYo$#svjY)^0lq|&%>UAQ}lrJzbKs|=orh)`kaWXQ4c$uk=|aOM$$6-mNIo@ zSBQrFMoh&<;xUuAjRT~KF{$uSc(jLxZ$y$_*TkiG(3+4wS&-2PT zU4?lGrgjUW1?+39wz77?{67*y2yOZ!=WyqRifaQJj(7Qc7~PQyhnqPLR2@AZ*DxE+ zLM{2*Z)P{lwF)z4M;$Emk)DueIYn1veqNIyv+4ND+%t z_>JE^61z6H@%XQ;kiL+6qVvt*mYsm@j>3LBets%MXUSiABSxO(ebq)qXw`N%YEodc zdbs9rbzM8cx zN(Z1JrSUxj_|r!6zJO&!ygP#Ag`Laz*JYl1>ZsaRvMCON+SKgQbj&gbp)ifn>M=YL zi#hEkmnI}0s*In_Ea5>sLoIAlNq|o%Kb@YEJRTA6eJ=A;OvLzy2&PFD7`Y|t+q1gN zG;G4DksnXInVNa{v@sh3*S5@@vu`@~jUd7&l4clZc+2!8$mdlLvpP=|%Dk#Qw-CLW zSjKiSd2g!Mvy+8zwQ@a9>XnMZCnO4~)8&Eu6B%~G<& z*PXI;$J!qWZ!9EBS&NWzHXv;~_!9LI2*k!_ddza|@zS;ZvO9*K!`;z+bQV<#Qoq-b z$v-xRmEFx=*rO+}?neGoKAD9>d9Eb3cLv}tu#@#guL}A(Z;Q_{>SEH}+niwTr>ZvX z=^?FVx&4$wkL&TCJ@m;~SH+J9$ps(w1AR>(_z-mnx_JEjQV{HYb;hr>Q3+PKcH5Ld zmVtt`~E{v9R_#{ib=}9wLLX>J%x9w?6nM2T19SQj(4L zwzjr;1qHp2tbWRropw)rw$t)TlIH#W?RYJ@M;!(W8&+Oru2^z;q1Ah7|47`Tl9tSE zyz4v1!0FC>ex7(4^F|g4i}*?X^*m(-g@sDnvI#>+73mUYx=3@v+BbHpv%(Z0WQoQ}uiD^S&;D7l+fzPzdNiQ) zLNDM)B3w%3kxkC=$d$4b1W#tKC)L^U-g4naLGl{c6S-&DYgLnO*G7vU9BavA;;3be zA|Hx(UI)m@C2TaDY>v-LDRWVt2=4G)#nM|R<^u+C#g?NWlzucye55LJ++x}0Jp53Gr*x2z3j|(yy};4Uxu`o&qzurO6srK z>hzn^Xm&ZwMUCfXSIwWB-`rCaI4he*)|sE!XVhDl)!6qHF>izh$5#xT7v}U1PCV@* z9L|Cw=ik4M`GC}%?6^ZGTDzzrb(b-6D3`5&V>+?%))vQvQSr^1chQ_HH;!d1c-O0s zZ*63=)Pk}G#`x&5%uM;Timdjm-ky+7GOp#oVGY9)k?vRsEoLuLlt7F@^sxx|VnK>P z515|I{lZDNTdEUl8hFlwt2dh7`7RTINf;dj{k~Y~sP!jae1)%H0d^toDB-yMq)pi| zI@?0sQnV0-OpoAMo-0M-gN3^`d(Mh1cp$b>UT&Y+YIAoawdGOrK@^T zrAY}16obz@HA{=USFD-GW}=syIkRs?S%lqjhaE6HZOEReo`AGK*On`&8xis9i)b3@ z{wnUCRy_Z$%!&PI+baEO=NI0FJ5lBJxyGKBD+ppIrbl9`5tSba1X?Fv>OKC|o_z}s zIWUFC#J)Y<65sK=ermF#t8N(lc;R2+-DhlDIKf}_Q*A@Dz+pQ?29+7_b|B1@ne(C{ zT)a%pxr%2oCsc*`N`>`8X@xa321A|+;=jzmOLcL`>b;q{@@T2_%`HS+xMmJ#fm_XD z>YV`7?;?KE0~IRzx5Mr`+VpV^4dUsUltD6KMm84cg;GS;mvD}L36ohZff3zQR+aZL z7Ks4@mLD7N)^k5JPG<>O?6TFQ6>|_W=*2}xV0Cf6LwQeChKN2Ne)s@8JD~)o`$hjkrB7&cg^@HESq7R&NSADfXt{g z;>gGt()7BC+*cxnLP6(7zmJ`)OS(}f9U%=Vrmch#1&A1~+%>?Qz)VmrFO6#q8nQ~l zpKTR&q_aZ8@v&!#PZuk+@jMv`TZhdVR6Mves6X1~vPl)fVqBEzw=YfaOtaV2&1p2X z`9>2F&_jt1qjE@u#ZMs$I-lt79$QRJI_3$9pgfwZguV|N9*O5$;Jj8eB_~_##~;-A)euW zL24Uedinmeui{rSYx^z@tz1D3rwy3>e&e|@x;Codqj*U`n67$W!RA0C(eRVS>rIZn z@1eav4}I$xmgvLja;zF^?bo$Gv~Et_Wbb(+Ud9@|cI+C|&=2#iB03|FK~oITdBGY) z#6^tL_5M${+lIjUFhQ6K7@x0VF>&X0h`vZ%t&upXCT~Mc6gqZNv z>v`6NQk`uU^DTbJ6$rsT7(hAc;ym%D>y}kZEEm$oc~T5H3~#{Un8k{5;W?*?w>ZAF z9&5`$$iincoorrM`LXJ>Hd;tmXSC9y4sBj7*N-6#HPOq1l~FY(-ydqdz(MTmtA|h4oMVE-wcqxV-bi2v{d$! zyuD;~mV=CCVZ+(I;*!KUHZRz{PYkQ<53(*5V4W+qwU4Q^sQijPN2KM_chKNn9C(Ke zk`23*nz@uo^oLJ{c+T^Q$Qg2q7Kpv5j~^{sHW}g-tEPZf#+R@>F}>MuM|fTofZfEV zF%IN|=}F@R!-?orx0ygM5{d_&rfa2?k-HtemFQJxhe=N4mgYx27^^y+jOKm;rLy76eToCKF;6AyaXTR{P6;RiN(e+-X^c=fPD&u z>`WZ3h@p=?5O=SaITWl&P@Bj)`OkTMxiY@V!2_*@k>-~xzrh5b${ zT8_m_g8|f_58;WVJo_SnnloGG$A-9sBHCOhXlHXgLQFaGc3 z9a~wv9kef6JNH@6?H_n}Xt__JrxL~Sg-o&ikzXHkbD@Ovjm6{jILO>b`@Abv^XnSb zZH62*gq!wt-QJtz6@O-6WVAsbY|ZE5LnrIQBKC|qma;978D9lJt0q^SUpC$ua>F9jAi^SSYENY;GNR|94KD7xuGa<;s zmUA=Z2d7J_7E2m#XE6J|v53mRSRO7Cg5~-~w0#f%c8vX(NA?pxVt31Sk;4-fE5phb zW@RW19_J3Z=azlbEVhuv#f><3HL@_O?mEko&XNW#KS;hHjbBO_~cAavS49=GcO+ zWBf^0y z;~g(f$5j`y%W2ouyXjj{_NzNtAfNvB`A)u`duq9r2qtA&_Q8$xruJ3Lmmf-1nKU%S z=X^*hbH55e|6Uo2=*nE9@5l9>gM8win-b)F zu9#|td^dNWy7P*Fkgj%T6j#R{c|pj;JA3DPJ7wdM+Su&eY}@5HJ5M~xiHcnoftIju zqb*dnZOt|a^+R6Lk*c8gEm?*eEwQ(0{6=$1&$^^ns3B7 z=AGBOO_gTG6y&X7g-n4WYKs)3Hn)sHI{q)E1#VOKYmUyz9Mw0?T=0*hTcDgLx!k`w zSO!eEO`a7JOiWf2&7=TW@;yd;8zC2^928HqPOHW>DYo4%Pb+oA!$&o_(&^$9I%SoL zu-vo1l)uqNndN>t(VupscZK!AEk89tdBgeS5T?h--)Bxu#lFvpAim*Z6R3K0tiGe0 z{C3ftm6rguJl*ni;5+OcF$&enT-*cU$lQ~$&^Yw9T1W$n9D6vt4{u&4rzBuWI#9_V ztRG!mBY5X*j8tP$OHwW6fS#AibL7DuN{OMe?|KfoMk?1~f$!q5uhBI5#Je+Q(5gFy z29MNo?Wz5!?6qtxT`QxU>)F+~v?!rz$`aGmsl?$2v--0mszdoTh~ZzuH74y0PuK1b zLl>eg?p#bbZ1m<9)4AlpF&M@clT$9b9Yi^9A7QX8jCU^8uW2Yw{YFF0O6umB~v!7sw$BbGj$oM|q(>&Slcoku~)>Bu(su^e* z3&v@mgF1LsVH@C6PN%XS`mFJ^9bDNM^ zU!R-oW~Xu@XZ~(VyLob>Zp`vZMe&{@bT{guoKmpYG})cZeleN*p+t4iTXLV=-Nv27 ziMHkBceRhzRJU`%fL;HcSsTwZRFo~h%#k)he0hk_eCA=X zAg0JY;8~jRv91MauG-~es*B7D0R5Mp=Z2#e&LzQET(~gsDKluFw6f<_zPdOx7>XCsuW?L}!H+ z#}dIux8oY1&g;wDm%I>ZRnm9dLJHpuL_?Vssxys}3mRNa)C~kWSl}n|zzwlAKuO7D zmGqm4yda&`2?g_c`Cr1P`Z}|T2%v%J+yaFGwEC82(-|H^LP0-D!xt)?GdSondbY;% z>y4Ld1YhETaABHGUPE_RWj4j*-hQmfj&1j}(Y;v>^A&0O7HI{oBsgh)!DfYCsH>uY z3OPC`!u7?(w4N%x8*D-Tp^|%%?(Rwomm)y7N(C(oibXbV3(lO6@epv*`<4)lY+#AM zxg&cUVtvc7+mN46y2r*r=aR6nbdyVZx|_}|!=)Sdxdt-7Tk~80WHi8m;Jg$K374Ec zf0PzArEOpb{@6P$Ii8-bZ>;)CTd+RcvxB`%V8mt2gflaYO_bA$anwJ97rpRY>##}6 z&G8^ikqx;S^W2aa`r+oiM-=L$zLT)!C3e=anR{0+<9>T^IXw66VSUeA$MltetLo!* z*-K`H?$PT^-TA4OAxQjaE-&e?9i+r2;sW9v1s1o0^b9_poIX({YYf+S65ol(&tI7| zn4=!|#w(~C-GTX_KVXH24u@r>vdVE^`)uCA7~ynKVzH|CEVE2+aa^!>|L#@KVG2P_ zbFs+U$#2&bQ$^}EXj)6A(@jq2D-XZvvD{K)J{8?Dh>u1`P7cFcopQ38yMky+ZK>$V zv2J^K(0ULyhR)z##uZH1!Zr;;efXraHa+Wwuf%eHw2+g-tAgfnUfrZTZTOM_d2i?T z6=d0EHj+WSXf8ohpRYA3*BV}MdSAk(xa66G`<-fZ3f}oUpedec^Y<;Ke~LYJ^^8<}8nvW_2J zm*K~zkdeYR-C7Fxbk%-fZ<;##sQTl#woBh@f=zA(+NVf#x&=+B=KEMIYJjb+mduS_|7Y7&BsCZJk|Z3g~2n8*5l7S!(> z{N#&y3!N)gPlnGj^)K>D4%&vI5T@c_pWMv1rKByZA+BRxuLmF&IP|9ICVSd<)i#LV zh~MrZLpUG*`Lut&^7eBq#f*j}ho|24SFeX@z8QJ(7S+D^Yuxu65tS|Y$fa{Uhi0XY+H3j z{$1LFq2l1mq#JDw6Cagis75${xaqyq$bKfd(as0jx4d;zG@l&v?N#X2_2o<|d_3<# zO_hcWF_;Eaa)J>TMJ4FtZc>s*DK_CIe2hJYsUK5|pLDdKCyPoopD0f-hdsr}S6H~7 zH~0R9h)b9};}uRELtNkSN*1wnteDoCj}x!g3m9vQbJ_x1f0`u*BI*@gCGTz@?;kI1+q z$AuAt<6|w0@zIA_(r-&QR^w9?45a*XQ>atvGbs^IC#QaLy)Mcej1XOJ$+yMroZ={Y z;dLQAj1F1R-ack@-92tmKh838$r;y5FH>w5n8||foEI0B);1I0xk*=BO4<=Fs4J;2 zqKHU1>~VD!S>R+h`p__-83;A1VSwsboD!GZVEf~;`kw^-KdP=WAgZlvDTt4}VIXjmo#B{cb;lEam+1zB1#Rfbc4Y}DzrT-SD#Fbu2wUM(em^ZuMgwUv_(!IV z<&s<4J;OYBX6KZ@CWgd)+!eqZT5?C#M9>naU_<1>NSiY)v#(s$m5|72Wd391A)XK? zxcOxCXUp-%TqA`;fE4&xN@uwIXgkl|Ox~jX4Bv|I`zrthgFYiHq0$ zN;TuVprz{t6fj`Op~k&JfcoKA@NE7BM6SywaQCrpnyi3>Xn?J_C~Rrj#~) z>*f`-A6li<(;lfzwfj{%qTw|UxRpQd8pqK<)4j1`YmQ$Ln5uD7BO=@5KN(x}DO;3YP)W9`;Okq>>s!N(TN;Qc z;;(s!=M~HLKkK+SNC`z64v`#Plq&|B^-)r2sN#*$wnE zYxc!Lj3+;y7KlBpp#Q?hsqh0l7$}fYNefD9H_@6v4@}5trGdN@WS)fLn<&b7h2PL( zgoEpmAehl>GI?@RGHbrhGBs~x%VJQA*_vB3oM__64jQ;Lv_bsZQ?E;6hzDi550q>S ztlI}2hiv1y1gd@=t<2ZVD7$Kx^bZx(3uTB%r=@EFcn3#yNodsooeB)!=CY>fE; zqAtipJEis*Jrq3F!3RTT90ugIKk}uikCTCBd{Me?Mtyqe5-cSk2Dh0MH%cbQx}+%W0QgCWm={~uM}kU&Ous8^uIO( zkdJ@N2k2ZDL>(+y?IX45C2Gkz`eN?Ln`B*WuK{IS@X2v zsYaoTCVw1@o%I)@2~Wa`bi8}i)Q_vFsN5%WZF#61fb$BW)H7~%K@P-S6eW>!!`kZYZh>qXf!t?cK)h7naGD<_>L%f0L%fWpL%(D?SeeRO2P zO{m?nsO~7xZ6!r2s1`bcd2fTiz-_QgI2RoA+^@x0v1#t?>W|zs-4isN-!^ow~%tXV^Q*W{L3`)aGU1ow`TyNIjkeqfJ{d z53{~&0*<1?HUfZw=By9r^#iLPSGdo5KOtYBlV=_vGHCDnRs(mPz&(S2Vy$$9sQtS} z;I?p@y_wR%+GmL#&ARTB?ZCi2)}7W<5&O&A)d#05rQd<#WX-eU70t0p%r}MUYV|ZE zX7MDwl(n}XKzfeVxV0~sF?MfrrWn7b}1$GJFUeeo9c`#+Nf8v9p8_wm_B$r2i6+%lBVx`l=-_V^{B)?noIBF zKrE9r&j$mNbCZwcro=y-hiUgFvy!S+70%XrpvslMmKiO#SI+c~84LjbBi3J$-Q76dpA#Ze$bQK0aGlUcyzqd^70BG zQdpNDu|*w+^USp*i#-gvXTi9qUZzPzy|bj@ z{$zH(0?vEQV&vUwdbWEeaPi z-^M(xeQvw`&c*3sFov@;r0QUPXH38fCH}=v*}X((%6Qfw%+H-`mkC0mR{U_tH2njOwZqMu&J$P| zw7~L%Pl{2xWKI^B&V|yH>P1pGK+?g?_?akHA78h%y=+C~<&rjJ9p}0B*c5x^KJWMP zoKJjNhVAo_#OrnKZ`yZYOt<(D1I_6ApB}5PG-@b`&yEy={hvEekU1Rvi1iz$jND2V zds0*HG+#-ffIjWCo=IZmXtb)sr}E*(9P4lpe`84Zcpv{V&1RMdkPF3hnK=g z2~E~!W_ku<5a{_&-5&ne!lr(B*Oz!4;7#l1ErmYeZ-owE-$bHyO zfsGx_Wex=a_geUD8q@#El85Bw*T{x}I&huK9}?8% zN1N4e6e|I(H3P7&o1R>ILFvwv)%i{DWh$C&U%Itcwen8y+NVkWIV1}-0(OL=#*><9 zh&4&Af+aBRAanY9Fo>VZF$%JjL!W53)|Q=DfI%85oYnqJUkCH)t@4^U$kE-YuePwE zaL%%#I;6QW#zD2)3)~Jn)SWZQt*u9Ve|s|XV+b>j&Xi4QB0sqE>5eT5JYwELnZV=z z8#CVJ6xvYARze1W83ME?9a)sJryZu0*i%n=q)7Qk5^Sepc1$d>br4O`8cf^q9SwZ* z90OQ$X(46?*j*&v@S(Wv1e-s&MS_eh7U@fY5sB5ijhDiX7Sr3FCn&=0b>q%+6nc^~d8uETH`k3+pN=a+khIC{K4}?7=>0vC6xrn3m!uE;HYy zhDUZv{UXX&42Y>}95&ZX22rg?OUx>ST#sx@O(xY*f~MFSul~5QOfoqdEr>muB_OfR zao84dlsnObdF)R<+v!lZ!DB6;T?9vVJxlUh9MhhrV=9l4sAgkh`dG2mH>YA5hm*cQ zfK%LXG2OS~F=`5y6TiOLRP!PtZS|Nuso`_~{obmUrP7I8e0eoW&-N1QDi36yf$ww1 zB6RKo8?*HaHG)+MP|C^}NLD;%gM7Df6Ij z2q%v1-h|AGB**a1d=XVg1m6Z#L)JS+0Le->&&$mwDR8?K3yEzKm9JTPq zZM}e6;??Cueu>EygGFd)RgaSTR^a?((I4#~7;OMIE?WP19tGYYVopU=4-Z>x#S*rk z-Y&N6A;_;E(B6?}VJvS|nC1yMT-&s+DIfpI5)bYTmOO<8u~lo#d$xOAkB(bS?02qt zi0^)Qb=Ydnyu)ND2OKeCsLj1_FtkuiB*ltO zO>?o=i^L4;kn7iitZrwbZ_S6k-CjngK2fa;AnKqchU5DZDge*sKj#VYyGL)L$vP-+ zHbMr~2F;4OMTE9@vntDRjUJEgUaC2u`+-P7M~x~U2NrQ;$5uwh5NoPsq%VT5?FGh5 zVC#Lesg%X14Q{nR#^)BkiELD9CV(G6+TPt-sWhf|3A7|Q4V>>lds#6_aW(WDBLo(b z{XAdAq5SM6ce1S*3CqDgfpQbvoTVwkkrtV;$f7^WPj7XSCQGBBkNOSTGp=&L)xyu~P&-$%^Xq3)^}JA?%*y^hzw_lA$8TpZtj1>@-!*VI%-dbopUeV7oY-c?F0t&cBXuPm zVpr(Q9yqp@Yw&>(vO-t_JthS;E@inx1@c19mEF-$CcqtNtkq3qb+-nU`*bALz*M%H zLfzVqlkfXlAd%{|3!EKt`lPjsqit#_34zmXp81-*!fKcUFx`xxz@oCS&}nyW=M)64 zG`I~8M6J9!=&cUnnVr}!{~j_P=^eXhsQqy65Anzxc&(%3WBSM z815%QLc(s-S8$}@HSuA+!ZSW87a?ZL6YN7mP)Td@h}o%v<`FFNxi=;me-%@i`!sXC zi)EZMTUN7dX@OUxLs$nU-_lU@FT=XXfKu8;X59&^>V5UoGtTcd5$bx~qvPYXE{7jy z{CB+;6a;r@PKS)JeA6m^Uwx&|s^R&5kpJ<`pz4m#*HTFO5D5a(jsU~C=b6(xmf zJo*I@T-`>W=)qheObYwi!UjwGgG<2z`EtawH&A(B2s){qYl-sKXn@=x(M6J^TDiFR56N= z*6y*7jLu?AJLkqKDixDFF8C__xtV88n}q$E!lF$r1(WN)-OO~lDAIEigC7NAT{QWo zWe7;uyrL8w&0wR}a>;>V>#ZSfZCj}F14F|TKw1icZ{jjmO@XmSSrfv9j)O}M+fxIU z)yI)LC0@FVz*M)!ufM3-zk`e5)1<&*E7YDf4fq)i;BP?!G$5njqwb$5zb{cLrNrP~ zYy6%E`+>uDjU8`)KZ$4IG+jGw(`vXqu+sO?n`*!hxHPBfO?%VAtt7Bj+2r-B%rtQ) z1X>X>H5IU5`sbxWESn`&0>k9nFdmwFu<;ZgSv+W=g_Gtf%=*(Kav_C5Q&@k@yM@R9MC~##PhWkXTHr1 zUb>XG5U3tS0^1}>SSgT#;}Ptqw3S#2!5iP`k^A9mfV`rR48(RnY0ZVto%6Y$=rKhO zTXT9rPOnb&04Ja~M@CpZUe?a|>$o8MXm$HdGq(t@ifelNo37*)$cgIi$v%chn6vzuPw!k&&XHMz#{SWw44q@6DrIr~b=;bn|NB$AAx%-HxZK0|vF zcwI!v6A*#$dCshMz;|xq{LV?;LNn(!*(_4%p%V+Ny2iIs&4rZT$WB0ysw3kpYJ_E- zEA0!7t=~CB1&aXj7!CHOkT@bbESl0%Kwxc|7oaJM&ZQ!9FTiL1L|ote(>vYge(#M*8!I0?eQ ziNG!XGeMYXkaznnkM)iq&+&Lpp><=!&QdQcYNRnUE)7$5LdCc|#Kbl->P5yH>GPa} zl6m5j1GUVGB&tlNUg7UWVVSbj8P90qa=h=O^LWt`Hqj<;xIOp2!md)EoOm_{-eQu_ zdliS+7J~C~;2mrg%J+He$tM(mq6cr2-_D(Z@hZ~NFVpL3<1By1mz5d&_`QO0173s7 zJeEsK8|11nrF8MSXC#)1?U#GYK6nArX(M`d<*PJ7?r>v#M+Ae?1cjSUnT)xmwF~v1 zr>g=-pK4@JK6kt`e?Hb)Q)wD|IrG96!E4UqI~~dAC})W3t3kk(WY!R&>PxPR$Zo`= zru>#s+@AmD51o^FZgxxX!SIv;HVVUp>Bi7dV8SRJ92?XlKa?X)JQ(J;k!czoZ81hsCGH{{~QNo8wlGu=!?O-*~fnQ-@BEKg{0wAv7je%aE9T>YWayL3+Nw|3u zw|Eu5<3my~9`GMo5qo}zC|){U9GehMHm;NN;L`t^jtP%4*J}fKLGZLI-Fi6c*})Pz zLc44-w;ubnQ%}-mO+!`~CQJ|JBm8pFYy%T&C6BD=u+b~6u9wTjg25xzx60D*djtoM zvLT-e9)7(OgTFB7pMVV>rbMGmz(RllXhJ~7v*|JHTqUY4Zh!MVf2{aYAl@mi&##Cm z!pdrkyfBvQ3ia5p6@%)gIn~b?V8+cXa3_@^1LQ8vg*CJRSRYRJa5{Ljq1c+)p)WZA z1I0EHc8mnAPYG8n&A`mpJMp2=ugK?^lI}@QIDZl5{{M52OSCV=-j%LK-aO4s#kg>* zRim%SUnYUY3M)wk84ZM#W~4wZ(-Q(-Y3R|pobgsPj`hNrVAwffD+RCV{iUfhjA*%! z2@v5XP&68dQD0&VP{Pxx^xl^53bHcvGb+>9Gv?EWp_V7nJ>ZK{J_3cP4%vzspa57= z#wN=oU9BDQG?89DQaCg*f?0n$cp%cTryU&73dxXxjvf=4XBt~m6|m8k;^LnbaEVSy z#XcPu;5Ym?eRG!r+rTzcQ~d_TjB0>xIeq0LzLL|f&}ip|QA4kv7rRyTjo6#JnTu~Q zGOAh>*1bL4-)H^Roe1S)-f|l8hw|xU%AUk+i<*bumH0l1B0MB{c^`ErJEKfY^IpnE zNuRWjHT<4JN4rGutTfgi+D3EX0Ev5_ta{Yh#~n=-?aa1G*o`;r2 zm-!o?Pw>Om@T&D(^-Hwy*6h1ST$xhKKbUVQZG7k+E7Fta*LIa7rX%`Ze68_pP-b1` zB*rn<*yQrt-+Z-qD!tO;ix>G&&NX@4ke}brQj1FRtc$;2KjyX^FxHrD2y5({wp|k> z`2`s`>;`ohBYGp?{2tn`{J37zY1g=&02wW1LQ_-mEG@7uOL}s_F|V|4<+7Q7W%7K` z((_s{$r?y8eO^YBt&J}$q*=2H#=F_E#^4t(>BtB4aqBdIkHhZgT2C*x;$=cpnsHzqy{|ugEjmHSu&TXS9vQhSr-$iqQVz2~7^E*Qqg_3(nmK$kT2$ z6S^E(&8x666>i5As%iA^^63lD0tPx|*6`R(1Nd+iegGAH*aY&|@(;2M!_WB1>9ho- zq)#$#IlHIdBt&Nd^f~!)&!h*4RN~1$Q-78s{h3YJ)g_JL_h)kL74RcEI{H9J?lpje z(Ec{Lxx6%{hdqvKQ^1Qj#V9TAEY2f=Egra+;qpz-jJS!-)z5Z=a zFIBCk+N}WHw0#@AN~shH_in33S;K&A`yx-oN#ty)&Yp=*Y00j&5f<*fdGh76p?cnJ z67^)%Ez6P=S!_85o>6Go4?IVMGhc4G6~PT*^C~XT#{q5)U!M%J3;N1<~CDQMF!T({*_w(pwr!x`pvkIP9?8tbp9`d~; ze%Bo67(qsLxK>fmtOL@=U7l(v(JuckPuI<~-qC@iG5&Ji#E_629YI1dIko1!9(bVi zW6iIgC%}0LGxW;7f`nB)VxFmYe;N}s%8Sm=E6pKZj@H+Ez8|h2HxPQMyVO6!6NKS2 zH5Pw*F%q~MuShO{XhC!muo}EY;WLZ`cj|45SEqDgibexpqS6)JMWM@wsHGUy8sQu+ zm*&Y!^^Elt8Kn@1L%wcqhV>^({3zsa*TcPZ1F&$1`O}Bauam`P@9v2yXS6SK}eqJ8) zN_AFz78>TbQ9<)2qs*Egx+5F49e<9Zsw`r@ zJNYHLGuY8#%qE7P{+8n|D)olJgVeG_`U1c?%O*7~eO#KdEyze?**FRe&x^q|K4*Rk zidk!uHa+>6$w#CP<*+r(PjTKbiT~}`=S~`AeQt!!V1a;9xWC=g5jbxlZ4j&XPGMaW zjISU!TJ+O>{0lJNy)5&#C}O#DF#m#mqk-2otHwGlPg4f-OSd-NTF`4Xg%A1392f7n zxEnOTUUlQ%rySrpt9wVG{T(oPX}Uj3sY2Zlan zi<3aIcu`n?U2nl@UbE>rXtH!R9l0)kL)CBlWk)`e`6dV`+?CM1G)yv;Liv=#ur(m? zWHz<3+v#V|k(K7^l=mZ~Ic+@rw5R{ir~(RrZ4Q(osjknm1p#G1LcGWe8I(SeJamBa zaklinh*nMW0SHz4zM1r(&#t&+C*pZOMGeclh=56*%eBacgYyZ!=flx)yB2*o2&>|v zVN3+kkwIoh>!fhW`n$!yzK8`5CTNeO(cqr37!mv)Je}9#*i7r`0BvmPbTjd+89bqUpxFi_Xape1n(XdGepa~ zY)&eoqe^X!jFgO@jlq#p0~27*Vq*EQJ)_K5(3d?joi{&Vk9Rf){CLQmDwV{eS`Mo<9@_SCpzq#y--w_ zJxN{5NPx~AWs?PwU*e_-CUO=4&@5HPAU@Z!SLj_vb^{B0xpkl$L7ktm+)LWhS9s2! z?IXkc6A9TE8u&PFSSCG*&tFa-wsM~|sK~yX7jm%X*)Y3XVd#Ti8T>`@6j|)9(55~j z&RL*?m?JpBqbz_}jm)8Sd*>q0s!y1m5rBoM6gHb((?B%qxuPJl%jI-_9930+dRa>1 zI1%v8vN&*QUkL@y{QuW!5Zz>y>qc+xTL!Kpkdlc!h3h-ECnF-CiKRX@LzTaiDmz3lU56(u zDSbENCrr&e+*nT3k~)I~qd11RH61NI24)IF-!L(+D*LVm0GUpF=fLnjqW!QIp2+8s ziJ{E9P8mtGd52J-<%-*6ds>uZy(bUG)$NHR{{u-zKyC~Z09Oop2+ejXc4k}z)~|Ai z5GjVH5N})t=4Xl4lg6gnRh}aTCW=~G7StYHG0lWte-GY>_tINf>x&`DbJEpK5LjMW zl%Lu=j+(DB!W3$dnFvqHs0_^AFtz+@0cuZq9wmQvhVlaGGptQZ&}R$abicl zx6kd=#bZ^fJK}_sJv1>4pQX-f29}SAGLk*8R_CmPD)6$zya!Rj^Yw+!#{uuf!)qma zbt`_0A^Et)_r0Ky2s;M@26KbRPB$n^L`1+NYUwT@3pM13*wQNlRQ^mZcHd6Fi-uBS zDi>0r%llU(R`<%idgLf@<;A|BypWG#5-jmP>Okzvu+n zUd9FdyiI=iXV?0Y!qmehsDKBp5VYBLQWK-kInOejXKH~j!*|Dir4Be6+HcL;_}5KP ze244P_m|zM2sy}Cm$1zlt!Ifl1fV)8=D-3)ubm9jY75pzVaNI%q|GWt;R`P5n@Rel z+7Ry9rfcryN(%5ndUv7X70V7qC@x_4-=Wl9o?Ke3&pPU)_a!(@cZaxGcrP3F&RgR^ zR5a5sE?hOVloz>YR-ATb&pgnQxc1j4wwMI0|pPq~%U_R{0(O{9YSK*aYrA-*sgko9&Ob z0;%&^W?p}r)|8WDDb>+Gy*?u+@U0i_uxvQa{0X@MIuOe+yiw*AZ9Fbm>!lZg()K2u z`>7GL`|&Fc2YoMdriza}rSV}F=A7Gwvo$r7rE6_1o{7AZ$M3GjD*SidMkX5tHgBx9 z$5>+z?oG;D6u^J|dZJkRuXhv`Mk$V06E=rFLxkc+3tPH^A8=0U#%b*@pNh0wSvHJj z?vNFPS)Xk4EYHa1<^SfMwo^Tgj7(rBIb0)TQT4&4M)J88OAbM19=1T?vHWu+A)CbS z5W7jFpDM$KT%{+cU1z@2KR%}}GDkz;#tEIfUq;A3(EE{T^Xnm`q~)M{N+`rgFjjB2)Lbm<|)QyVn5a(Inxm&0LQsc zFPKe^c1`l$ytMU}oaLwmAeLaiURxp6nQl=PH%c z_A@~!r`BD8Nz@Ku8~OzIzW454t$=3k;-60HRbSD)XOKw+R~|U6wo}iSqrX>lZLdIbI^`e<+Wb^^%%h-s-mkyoDJEAtEAT2ch4L_cl04 zXE(#mVoXiobhc`DwmK?tHCI6WNkplZs*%V=9|mciKK7UxW(aU!NiC3%W=B4xkKEm4 zYfSNMsK)B)-46~Mv^y-@9q7X0X(!DJDP)h_h1N#}U%Q1zz3%$kpcDETAUU1^@|XW1 zc|3p_J`4m#W1FFPlR3@K$=|pH3LGpL0lfu9iwzf7OqCVfmt-~z4e}+kudS{1-P(_L z!Z{VuTX?2F;yR5a8_nXhVxqNd8Lz(6u;bA`C9e#bL$@AhZ4|*FuUdb93LHkS?)V~U zsdN=0u%F%7*>lr~rr;FBGM6SJRP{-;8s~RjJ-D#?_{Hfao2ss^WdaEBa_PuBu4f`a zzTc}h0wr3Q7p9aoYUfREB9HXs5Q-Z14G~7oU21Inn`~l9o3JIFp>qF*5edGRpck@K1Cr8vsyL#Wl9< z@xMVq85v-}J_?+K>i}8*jyJdXQG=k~exlF+n_x;NL;<9ST^Rq*fBhF!k`&QjQe1hs z%w`GUX`26d1phPi34rkr9{gbbM^^yS0XedCK_5U*;t8K4-N%Lh^D8qPz_zlqNE!Zz z;elK>P^x;1ena=~Q~th%LLcxND?v2paQnd^f}oMpy&j=oY7P)JG+5M`kFmeUfKHM_ z`k#IO`+Lzb#EA(33uBtQH+%CwoPKD}&jtTy!iOLQSppwrEv*1t9}%?wR&gIxbfE!` zdF3%RaIW+3X1z=3v6<=H-v*c5f*$<;-9PZI1Bz@vc*2eU(#!0%fue5x1dT5ma`;Pc z1L7}JUvblX$+_hJVxoT@3V2sdh1>{lRnj}@2*>b#{?UBB#GD0##SMX({@sZe!UOoq zmBtXK#s|jaQgw~c8|!C;$t`O7j2heTngz`@Eekwy4f5V)&? z_rHP7aGu?7Ae7IWHAae-4pH_qpi>cTM(H?~qRCS|?#q`_x|cl@<|JyAwNxatEtb}L z&z-!)kSr`n)-Giw9t9S$be7N#O%#-*SzjIM&u=JFL%=o|J26F8OlBUhEJ~fk&?q2x zuVl~y#6b3B7zV)PoF+ssF$%W%J{>XvR@6JZ7E7c zNx@Cz?hHC^MOi%&K^Rx8hgBh!LlkVkq-~Oy{er>M%RHV-riB8V4sd=c~tig=gMwcN!21OuvhOA(h<5Qu$Nkc{bRG+v^ANet&UAwh`}pd9r(| z1zUvjJu{Kz_VNfm;@Nn_7*U@~(wY&KPLeJL<)cXq1(QFq=6-VXw)>B{!qdy)_aEvg z$mm^6zbKS)T)sMX+91p+;(Aw-5j7X40L}Ldh_rCq4yhtfpUvRGTx9V0A6?8^~P zb~6Uc=ke@Z*4+XZJD?3K2Bj9LA>7HNY|qZj$B14^FTv3G}ug-J%-l&wBPI{4C zSIFV|Q(305!jE+th#@l*M(A89cwr9dw3NN~RLmS>gRmQ>7#10Y`6EBRKOKw4n#Hn+ z&t>Ye4@%mfV`vNYpd+S-l+92SydY!KO0e8FfJn|T#*{9KcE|CtTm9J50yAQS*YzLL z^II_yEr-$erm6p-v1GfJyHIh}280wJ0ly&?ytOT3<12cISU9yF^CvR_f4 znEWKaQ~l%V9m!e{6NlR6p%})uZ0_RgCypug%~g%|rN%y9Au1CUqkb5!jl`HE=d@4^ zM{8!k`Gecn<7;(Ub8KJi+o z+vJr3v^2r_tQKJ>wB^woh=Zk0QaCYC&ip}%t*4Nil0vyZU)ojX6Zks052X!Ba2eg% zqPFjr(9E+8yUkhWuc-)3d_Zn2ar-r!0?2{cBAlpuaL{+KqIyY|RFdk4Zt)NI1?<0hy&$UF z$fy--?^tLx)#sg|@}@>%kyCYsB;N*7Jd?EKS=d~m)8Pg)(lQP$Ix$gRp;A`mBW6zJ ze+>A$ z9*uO3B0aJBllT85&y=Z}o@j+95;y?K%v9+nq^)h1>o=yK`MALZUnWacQqzq1{0iuV z82v0UbSJjHt@U;OJnII3ENOC{h~qnWo>0X0~6_I4e?SEsGVx(`Yp(74YoZtS>l~b*~#WiGU z9?quDxRkJNX#3z8I46g#L_S>1Jx{LUyh75AGuu@x)xAiECv*pTLp!OW&F#}cs!0g! zm_k!Q)+g!Q@w4QQCk|gk@le#xW%Ed;!O8%{`O4tE2YwM57Wi8{ed^BKV`Dc3;3O~H zouLy$o)Fbrf~u?~JL?^|F;6U%o4Wx-yv)X0W-o82 z+o-HY*oUbxGwaBAJr?$HIFQ7$I%7<9knko`cRX>kD0BcIln~qyUr4=qOgY5-kdO8A zy#M0EArsuE4*}rAkFfX*wZ?Iq_-H#dj{*PlJqj@v{e)G5PJsi*c%o=1(YUSgPj6UO zb?vQ6*;-7-J!Kd#D<@A%rDw2Om%LsS7}ED^KAWv1A}0_5ar(c!6~XJU++)!QG^=D={Vzmr;4It3__d6c?_L;VIIGTPvVLtfDO$%ERd>Cji#uo6|MNpW#gcH`Wb#UiU!9oBA^aus z8$Z#PqOhT%A;Jh*RYhb`14{3`M8ZFv&8`<{Mm9h!_{Q&3JXDMv=r!du5>a-s8M)7h zL*aIXkpSVYAQ92Tyn2#aR$unR*vjwylqRx!sCvAIbyss%@5fSK-+zj6wK2))|gHD># z59m8y6U4)DI0MZ?kV;QHp@1CsCe<*wX_}OXo1_)#A}JI)$%*PNU87P$n+BasM4gf6 z=b!9HZ?RH|Ps)I#FZBQMwU0ba!flDIbyvgb;|nX?5prmBVn}l@JEdN&bYd9X+c2rA z;wKI5JPhdRjaTUgp4{4!=kck%|NbDMGvCr`7bp?EocmWMf6(&k|EfRAf9R z?>#$}mDQ~P>xO!@+J8t$q6-Ok4FAkCYM;pFrb09cj-^aN zzYY#9z;|r4uPkd=;rl%@S)Z4VM&m{R==p038`qL}HMQqN$G%Neh*wUiwaQ-yJ0wtP z;M6AcPR81V@Xb`BsZ$XBHQuNafKt8wL8}knnmo#c$s}%ic}7&+9BN@-!256C25CVE zj1eA1Bx`S-bzR@Z(s?k#^Q_#{tw>EE!;0Wv>d@`MeIMfw3Xx#Rfbv$PDUryMS8-Yf zJ}0}<+=s?rj$({0tE4&nB@AalQ+lB1y(Zt{>96TL=(U6@!W}~!BJ!V~SIG1X^^`ka z(}aBQHHj5uDxjRS(_oWTFLI(avM9BYlwOwwq)Z%;GCYOgEgp10bSSyp<8wEJ?ViL358we%*>msmSjL|CXU%ba{cQT<20?@EEi)4lBb_?e3qsPHgi z6~Z_0CPcL0#g(2nljXM`S!haMMz33|IQkoCw-a(+>kXDA3wB(-a}i3S6Cv5U<}c(L zu6hhq!m)YZuMPoZIr<)i`r1GlA%cTES8UMW!HB;tQ0AbLpg3bTQ4E$A;&}ltB0WHm zK}OV937ertzIRObnIIw@JKeEY2y9+aoQS;aXvFsQ)H4gPo(s8a!zr3P0_Tie!TYi&|7`>PZe+iGGM@i z?`1w1>c8?alnE~hm~UGG5yb@;11SO@u7P53{O}KKgmwvvt*__;3_*F3BS`lY3?%|m zGBLJP_sQ=&9*GhIQFjCqW+eqS83LT0pqb$u;a31!6omVOsk|36l^%Eij)Mz>cNaF` zVmZDC1_Ims`_&RHfJv7;xJS?tVx9u;ek+56h42Fy14Ag3Jl=Q<_k9P z#;d<#MPUS&Wx(M8?`P+xdY$Zy?E8ejRc;th5hDkvc7!r|+mK_PZomD7`?sR1^cF}G z;r;rR%=+I$QsJeARIG0w#VOv1UPlYDzZT^66Aee{vc>6Z1ByiqA~Jsfm;&ro@&P@P z*OPyL3HN8o%}@h6BoenuLV6<6WO-!Y`7zAK_F1}!6eU3ZBoIgZeYAcoHq55JmHDgV zX4C)+rjP~O|04h}U5%F=sI7eV4tr`r-R~`$gA0h%QX9jul>DV^f~YgymkY=qk?x=E zOpm*_E5+W))TAK1C+n}GPz(_xwmhQQ3K%ACV|;{gPJgGPY9g4Csoh>OD?q}h3;}Rr zfMnpCsUqGfIbOKv9&LGYMkwFi$9rzd;0`^I*^4v<&WpzcD8m%vIwD>^JZWQC^q2e9 z^9@KBmAI{N0XqLlbOYY~Rx=nG`EOp(dyMAJ)C|W?Frp!xQ)7FAK&2d?9}Znbh$;ZT zu)sk+U9_3EDmY;LtC69K@T8zjSlyjo!$m?AkO96N4v#&qT0ngpv05J`XN*ZtJ@U4r zpy-rX_?jOcAs%N=eYfIpRzN#tPbXm<|54#1T$~+#4|%VqYljFv7NhV&T&V)c$=!7j z#mNUO)l!t=cPn7=9$nzUPw@c53TUz2W%?}&|DM~sr?3@VL1z+qw^=0MLPmU)&(@9v zqeBiygCqWU!qtNjb9%Ty!{F}5K8c0pzr0<6BplS~>XLD?-!GvE@gAesMT(jLJIuol zU8Xu?F3inEy>QIKQ=q&TDd6HWKsN?OyYJ7Y!YL9?{V*f~k&_)*3+XL_8H5oVxv`*t z_Q)s0Og6=`>>BiWq){hR|Z7VnI)eE*12 zJVA){z_}6*=j0%K&`-$(1z2JEmO?NNm0prW1U>N1d3Y1xA~&R0@X~Yqw-;;%5Zq4y z9<9O)iA!(^5c`C!0Gxe)nPbW(AT^A1x7K7|pi7z|#a~ACpQZm#@fW)b7jUER&scT@ zSLL|@mM}dHRpp45r_C}Rss#N+1w!FFiRvN&1nl4oH2A+~oIiz|e_h@Fj=9y#EYaRH z`A=9dfGz02ZiA@-e`&BB6kjhBy@bQ@*>p*oO*Uvwg-FCPXu^ZUHEq@e>`Lpf@G zu=LjkC(EO7or;ZRMx@#Ns|9!YqUJ%549|^^PZI-%f%{MPnqujG7(S!oo z?BoSRlfgtG$XKgijmSxDG^KW7T0FxC&Y&%K7_!obpNZCpN$}0y)3zC;xK(lc+6D zw20S@9PtAe#+x;V+@8nQ#^YmSbYv5CtC66{?a{pPJmM?Tq!+mGd{+xLA;0}^^yB&` z0PNF~S(P5S1{hWnQa|b+H3QrSoT!X}HPVPiji#z<2S|M^^-Jfyg`a=Yd{*77+DL=C zm;A{9yG!;h&kIdm$9=zjUP&)01G>JoG0~b{zO3RWtrVo_q7RkIj2DB%YPc zjb(c3ithwCjN+!dH(H}YSFMl_ihxi5xuU4`UI z&9ZPjE{^)u*(9Y7Cq^V-EP#6Ah%w!lrKIG_#-w-dB*uJ9@!x<=Cm$oD6LbC6h;S^3 zF!M>u%P;9MqL%u$g_A3`cd2yh^H2%H@p<8<2A;Ln@$v4f1s7EYj4XifB-p$QpB~M< z^K#deu(rmOgmBTRQU7iYe0F3RlvdT5rzleJE3w8YMfjJN?2CubqwWo;4q~i(D89yM zHpc&mz)x&m;8t$0G11Orb#gw=Z$k^~Xee=rPZZEuIB_9{W;SJ|CRQ^UXHiUA0vTSe z3r*hEcye)o6!rZ0WMZU~*}&U5RoGB%QE4?hH}*DXbZNYKSG+&_!It3 z%tl#AA(G%a$o;-NiGhJ}V))5pjn(3tq{qthHH-9ynyYRuBvmm&*KEYqYlxedP#DL{4o`HiRC`DXkx>Zuw z?Q2o)j#r)*VcOLzp!k(+i9%{WBu1W#blPe0{#yshYI;?3JieU`q5?V}%M0s<^R>uKB};8g$>Y6EZ|Oo)(|e6bG8UQ++1%K#^dafL4U2J(R0Zgeow); z52jDQ+MsxI(GE8!lj)_+>fZN(H!P!7B(LX+f9|5@9x%N#2Dwe1E=Ho$SV&;tfPP+E;Wdb?X@&34@T2h1YMDhE@!NWbvyGC(9za`*31|tJOwtj z&F>Wdo8+)tb)Ya2B zP6WJqA2_{@K||Gp`J7X8_A`1QtXHvq3kygn+;aVLD=gBmS-o~5Te=!bCusY{h17v) z=-0VDT)>6EWf%Q<(B>CQ2z_pE9GG{MFMOrRG9;VPE}qd1L#b|M-0$7fcP|^GdR(O* z#ZC$2TOloH$9u&xBP_}?a^?J=G{`i7YR;dFvW%(9`5Dyxr6td0cgXq{GpZ=m=M=J{ z#`%xd=Zb}d07B4)OC-77qUjX`5><(vq>LI=G_<(+R0GmAg{P7w9>wPf6I_u`$={DSdk_c4y<2n{k|B(&{hq7y!g2j*3|1&$ut) z3IaPvmK-HuDb5yu^cmRch?wX0mXX?)(r$oBf>K#-d?W?j%j@8F;JV1i2TP6#i~bw& z0<6gupMNrNAeTamLA}F=ABq2<-V-eFYdhm$@WUd=&t>QR+qeo28 za+V&CkjBcdVr?)l2BM$m!BuDv@Yq032B?LDL=M*)#L%MD^L}ceargokF;{u*E`mej znI;h==04K^?sy$z44qgR>T@``Q;5rnTh95LkAbBmPr9Uw$7te*@=%Z2gX)aX?7x_! zt$Ita@1s9h3YCo#Tf z_6ptL%xp3Kf^M*Y=W<^Q*gAwocsoBposQk0>T8nz$6fu<>vfMv*e_%6&+GU`0M67k zg1$WJvYB6{E3xF5GCl&@@V`;YjVzY@)-DlYWoCBIRnl1%kHls1VjOv-pFp1IIbFVf zH12Yz*bC+xxDg4Lg6fnQ35R~>Uob%rJWE>!O3Gi-0ho#vg3&jW*@`h$??{9ZSz|Xy zk}QM$fE(dY99DpczSZSy=VE03(fnRf=*THp*CRvDjoxe zDIww3b&QC+aw}^=l3l7hwBo=GX&HM)CXB#uH<&7S zTGJEm%BBTu)?RT$zF;|H#O}W%D-(UVmwia0#3Gxn!O|vlA+t;YfWrNZ?Ef@vfR<}j z;Jy*#=k7j)ZL2Cp(TX@PcyEl;5BBwy9IPHdR?`Q-q?T=BGML%w0d4hv9}Qs_P8z!1fLysadL6V#Nx~=!OD$kV2dsMC90}n4xt^V zgL#_GI!?>|-3)Y94je!;F(z3V-tbN(W}pE{h9=%0^!}j+)YOY*GvE57r9h;%3iR;R z^h1gD7A^Ht;I7fVTgTkOgd^f1#-ydo(rGd#88XPuXKjz(*}o0jJ=f|o+ZX^Y`dN9Z zt6rx5IVhO!mKHW32CUn_nFfF)EDzqIN%=)A-2wMB((Gg0FZUS0c?u4!t(F?(R6q!8 z&a*3_K2~qrTa$gHbFP*zx zRR|26LG|$aVpJED2Z->wfmK}O1G0b^MD7#BKAUm8A*1uR3D3G^Rn8CG^Jrj0M5xa> ziem#$;8=pkl~|9~&h(=CZ_f6j1d{n*J5p)XGuc`j9i;&ibse|0u4|4KzS4ZV2`|*i z!h)wKz6j{BsS=kFE0i4C+0L3~CWwrE`r z31V)DZ(%InMNcrA-n3uoX*=e@H>RO#S_0mkG90#I1Z|WQHhESGlSTr6gk#(C9IJ{L zAK1ZLzz)7sRF(sn6EI8Szh?=Aa{kv@CWjRWwh|I1Loyp#qVzR{Rora!f*4C!SA*Cw|#|TTHK$cW-Vp{F&!pm$0rc8=h2F15HYHHDu67^VJ210|s7O zEZd4=ylE#u7ZqX)pl{0}K*;F1@mld;=^vk>-*P(@@Lv~(-Ozy<0bG8U37(bcMajh@koPwKSYVq)Vnq(_WzFer~6$6oA1|J;cO zNR2Q)+fLbOyYbm5m=x4CL=7Xw)@B2M;)Fa0O#*a!k)8w^5&yFNMF4mCa`OhkKasL8 zIjuCRu?H;RwW_xfVvZPjx=N3YrXfz|dJHqm&5tWM*-D6ise+#m27AFXdKD+vJ9 z01T)BMcQqr=PeI_;k@D&{S^lM@c@vIS}o?kye>)#RlGg=`!1|ElI&_$qjmfKdc4_ztBkedp7!5JyC!K z0@b5a68J?Q05x17EO|#IGS+=98Ej1y8BWe7yxL%q{CGN>B~0ZYaX49X4Z5-1epX6i z-6Y8+w-M3PvVHpKW0MzJS`q(YDxe;#nz1Kx861Fm=rWl7B2}0(+sxEYbN*(@@=CHH z#}Hv0xz1yqPM|T(aBlJ#iH~^2;DEMBiaVSmqyb0H!O1w^Z?fK}*_0@xoq}(X1z&(HI_YS{ zy|pSrdClu4KG?+fH1RaP>$PbedU1Zr42Sx{r0% z;}s5#jW}|%*B80_?u=LK85q33%UtrZqSTdfF@|cn7gd^TUHam?a59+msiWju9Na9; z=fQNS$4b|jNbUf}a}YS>`E%=IG0d~pjhXTfFkwtu(VHtsrz@dBorT|bHCj8SZg3yi zsw&Xe0j*Uu2bykH5d&<=`WyDfJL}{?;x~_|yp>o&Uu_+GfGTPL5h*9wvoo%5z-d3c z)PISTHXuNhjv4d%V?`w#CZNe&{tyR^$dr;eOePlSfKiKs=>d0J5WoRjJ^WudWP3luoXRcyqVWG6`B7L_8uM6WwPJ2SE?;i`z`(%C z0M>x^K=6R|K!}d%s2iu}3)`+Q2@#M-$tnwDebH@X_;fTkXP_q=)zbxSA?M+DSyxwA zqrWrIAQ22SUlU_hQh>upWp}=hj5PNF^C58SMh9%u@-g>&ZruYN>j0Pweh(rItkffpNt3#g-DgT)>ejEEd{65A@xh zPVYMA$6O)4fq+^zAHNgjZxc=Mej39dWuW_3%)ZB%77-%1&|&}lXg7?A-o$4vob7%5 z;T17KPkj656;BKloSZg3Nd+7RYLCH$1(L`=Xm_cM+5Z6QEBzG|XE<3Fm?dS7HlC{3VXn?DtWOzZ*=FPC2V1 zN+PbFB3i0gMMUWF;)xhgK>#$5`+PVPyO(umy`|63OI^LEo&z2;d#FJPB9sZyv!<8u zUeE9qzzeOU#8Y^CbLqdd+h#yi;LEF<4j92@VY7nXNaq_;La0;)>fJfaxjikY5PrGvFo0c5dGmGH4>)$$K}Df8iKxof#=!{uAo+A9(uC<)qRtlN*H#IzoD z86W%7-6F9Abemm9YLy5qk8#KB9^hX98Bi^7ZSml7a>UUE%tf|bx;({f&W7SKerew} zHHh-c^~&jeAD-NcsKV`X8n|`bT@kg%2`s6APDBhoc+^M2!|E;0{pfId{jJ9hasUbC zL7Nc~czAwSG7q9fNrC92x3*S9Rm!5t0ES)q!GM1qFqLTp!ev4MvJhtDkOk zYBMW#io<;p29P5H?01cWsTO;ZW-%6Woo0L#K}SbZ95-Pjieh=u$a*?dd!7ZRT$**`(5wV||P&|gdk)4Z|6Qsjpp*c&RTdgxejE6}6 znyeb118o!WY>NDDzRKFjJ@zd2T5idh4o~>PsJmg};epn}v|jjLAR`j8EwR?7_WPB#1hn)YR@B?s)_ev!Os30uEg6mYXUp{&{SfFIb#cf zy&WI`8)URxgTT^=qBTB{a=9KDx!OE@5uEX_s}Ua`hZe;lo2c zvBdw9JVTyem%S)3?{mjC{amh;QX8xO?Rvynl`WAyf}$a|DphdWXk5$!X!o%$USvLp zBqJ7H`#gRc>X~-$!MKWGXV+B#N1NV=2{0t0c`HLJbJUBqW#G?cTM0%NGpE3rttkJV zz-1DKGbE)FwLP-o*f+mzpr;5w?_h^)?_k86uoF2u%by3`+d zxdUFg$+y^_$HHmA>T@(rPPd}ncJ`&4sqB*kkMZhkx}<_=Vs=!|#@ias8hsP)gc*9- z(Z;#3k989!t825pLvc^?z000=TzT|0F_N>QvO@jUC&ptrARoByq={jHg!v{o=)Lxk za|5g`8hpc!uwxrhi>9_fzu7ag_BP~MtxV{eEp0$DPJ3HLsZZ+oO$p?&+e@HkqAw=< zZ2zMKE)}b6xi(N0rJd0ai*NOyMg>stN!6RawRBuY^hJ239vOlQP>K46Pa;us{%A-Oi>YbD;AJi@n5x_faDruXbL6o zZH#xUq@0nrIeL?OUbo}RdT9Dr>to7tb11y0H{(TBLt;T$kETBxsoi-iSS1m z&9X5aF!Swoy>0oNYuBdtme6lGEkV0sQFlxYpfMo)mrYH0(5I@YdGA0UG z8nWGK)cf;ar0wmvp&*>ko%G|gRA3A08Lx@8$FXSUlA24<6_}JS=B`%Yre;d4I|2nC zORobTN^)BEpY&f*{lrZi3+~x`X0l6DE#5wLB!iTH z&6RU%REdx1m!p@m^JPbMt}+vK&NYnAUEGcnhQ4MWqDm|$zwHnQbOwJ;VLXfuac|DM z>`RZ_U&-FX=f2hA=ubdKb#d9YmdSJF$N}4D&w(MaT$Lo_xgJjv4!O!rM?FNaK{=YU zRwzk%;9C4?i-Z1QM(5>yhu+KaY?{GiYnP7B1i!`oi_t#NnBLH;#;Kaz zMP!}N(Kyf?7*AtR4dZ0=U+Pxd?NeB95ltk(N>Jy;g~(dP0V^R-F>xQbh>%1!qwG>V z4P<4M1o&X2HWRw6T^>(kecCtmNMfZQ&n7EaW;h#_hrr)YkbT?omfyPVuglk{8tB9s z({b7Biq>-!Z0cuAUG@$w?JZB#BmGXqIY3?pN2DHTb=uCw;NEN7;+x01U$B@;-eT5Y z)E6^ZVErAi(CvorBd-G&w}T&oo8?uf-_%r%w#5}xl?y5!2b8M&XAK{HO!7fKbXj4+ zmg3I1kID_Y9LmP;Nwc>E9G9N%FnWfa*=>9Oos;w=^D)F=hEaTXEuk@?7=lPB zTOe_oDWnR{=vEu^O-l?Y%T7{x`?-cIcd*e{jU%k>5w18_2KdcoYwUJQE+O|HUuO4~ zdKr)W*^ov04`$P$#RKF*@R#}H2x_Cgj(aMUR`RCqk=BXAEKd(u9&^Ys-VlP6!=P_Cx-@7C%* zSDV-TfY|ML%?8&*xg?i8(|FsmUAijG*iFzfao7NiC{5-S-6ibT1l#W|B17;l7Dj!#g-2bFA0m^Y zcnbG6>xV`i6Vf>~^X^SJb|$#fa4Tf&VV~^u$&=HdP4a;#G*JwaQw{&6=})CC?`lKO zZ3yvRsBkNa*bGd1$MIhHELR;n{{Aaqcuk^Fbhy~BaedYKvSx@d;&TZba{2A&k7}Kv z7+K1C4(%{9sZ|-fRoe098u4(JN5T>W%4xjN^khNW7*gTvvv8IoZXu=WXFwp+v|%R? zKe^25_vnZ1!sUgU$dgHTwHxNPl-P~8GFs1W&34N5imw&Lc>oNS(VyWCL0fTB*y(K5 zm|fKA)tF=_k$t|2Rmc3rPy#~D;fLw&QCrc#z7}CtQOe7c4*U?<*Kls`hZp;r04P^( z&BsVU>V90E$hetqFsv15r!ZyPZm74!79k_Q4mDS)H@RPU1ulQGC$^vqe~P1;cs4I* zvM;fqwE$x^~`yfZD7;|1oE_PC-qvas-I+$351y)i=&!cf8_X$ZlQp z)tV-!0m9trh12cZhP6Y|#-w-1+nmgJegQ()zOIZ>ljqx_{+YhNc&bJp&4|vUW~8wj zxdAOn>SYL0v{D8Jull|O7Xu8Jz=GK5Dch?*)wK~@q_8$DK?*|5S$FrG{RX}q1^_Ts}C6LWMJ_O=$wPA@NaNqcqfHsnpsxhgIdd-966JBnUr{( z0qj0pO}A~$7cSYA2`UmSsMnPj&Jjcsb4?{dYqkxCk0M@=aHKsHGaWP9*$Pa_w1sFe z>KaTNg~@6XsJ*|pvcM0{WNfhT2WkrDg_O5jw?oEfh1KX@nljn~s5hACSBnTA*%M4^ zuoK!mpCi{0o#ka*yF&Yn|2#hZj3u7tnkE+L5De^>v}iaTHnspTWdXv?+ZbSA($a3R z^n36V7@tHvXT#Rek-T5}X26JH=tVk59#`5WcHO|rPLTxC>$E8Sn8wZFC&%=4jTXmG z_C+@q*^6BRb{IPketbbB$Z%yItz~LxSrn`3YiJW0Bf+Vd+|u%`j6&<6qNJZJ7Go#04TayB?j62Tjzt zjYQqJYoFF^pwbn#91-bfEme)bUk1#}LONid9?W_fP#1?bGR#h5;li)-hie-|o?z6) z4;(lCxjCjZP0-}s2Q_VU=Dod@Sx6-+5xy7!{jmMHHC1J34*GmtVNA*I;c>E{d3e$h zi^6CUn{=Sp*Q$t^F2k%NE@(l52AwV>xa>N%ITv6RUS_7X^ktw^_yT9ILEnqGOig;3 zW6g#{Yf9t3{0YlLmE28GAlvCuIq8j9;b&{6Yck+0TA~66V+<4V<2W1H8A@O;phm95 z7vjf>t@cPy+lCgHfz|d^7WGGj7T2Tcz`W$!F<}KjE(x5;oqf7z-~D#y;~jxv!gzw& zVRK@D1-!HSv_>}VJ^RKm5jOJbbXK^@>8suO4Zs5yNB6|=gJ(5sHBvz%uoOo zx&=qP!d>cJzjAfWtpR{bJ6*LdIMn>zl9}=je_+x379*0wULI@|6QE*@-)?t4Z{fmj z_+VK&!*Bs(`nUj$WpxN&U@ovZ)K{Ox+TK61cJb;x|3)haLZ59ddk2qw#b&e^y))r& z%x-+~qP2~70Nx*sVbk{#1Fop>`WNPy6ncr_J|1V%|7KP#EDEf`!k8;;7?_6PUO?FnNHMpJ&kziHR16m*@$^y=A<6E{{CXbbUCuf|u`Q$f!|<`eMr&E4&L@ zFYWv!`S@FkV@05(4dz}V0`HaHhk1HO>Bd{0TxwzzV5 z7ee3x0TU2T5=~l#T({5~(`{_J>!Q=DI4s&v8ZG>64WR|t?7H!6xAXDNNoIveU&VRc zjvrs+@$T^7QG440p_zT@ir51t9)PBBzl6_y2cU5RL<~RK1e2EXb1nhS(o#}jOZlH% zt|7DOOYQ;l|ktQR0EaMaBr?8-hO` zzvuQ94?cJq!!8UiS-#X>=1L1S94RZgTb(Cpe!TMH@-=TW5bA)Kj~`4b{NL%Q3O4%T zrOUsh30QJs3|S^;(=f{*c%8Y!#V-kV`zE$Tm@Mv9fTy#_#SAfqyZ1_;t{bV_7Tpaz z+FsbNzv1BpdMRiaPlxuIi}mBQ`0(|Wn2GTBuKBLe{3m6zc6PxZm+Zz3UQ<0OPYMH$ zNEUP>1)<2MhG)xtnR^o-5W_#0SB3HZyAfUQn-dkp=fi>kQyT4ai5 za69n5tiX4HLa$ltvX*$f_w2r6P0xT_l?RG>+#37VSxHqD{n7R4Z7=az`TebvKev%( zMl)bF;`MCZ8Y9x8qdIuKHGZ@#ci;CVh|siJPV!c!7tou|;oX{V?VNH9*uT#0wEfK? zq@vp^+i4;%cfV#I-56CJb%Lk?RcqeQ;KKzZ2j=jDiNuHnkX+KsK7S+I@87_=fe@z^ zg2)Qx%RIfwEk-I{RO*QpPqDMV;9gx~085hrD6PmAFLz%S05HFs7oz{uX#W0jUpD^g zY(dy9^nPBFVppKV=dY%V8(IFUr<{Z%G2=IB{vznodu(NC(&*XVBpEj_jH-{Jz&(*? zYN#4ViO+dW@bxl9XCnZX=YzPBDf4?5e!2_)M?dxpTUthN_Sah7%zi%1oYl3~f1Yu% zv9i!#NK|}}o_>)QQh23v);;0hUe?Eljb^a9;67`(!+L`VE*pqqf-;=Yjz7^&d|Cfw zrp0hbfV)sKznjBXvaoPU$?m-%bVk_Az|=HK%Jgwa#jAfGbcLMNOc*_)lo{Y3*!YBk z;B3zZVlIXXY0KlcyR@(vO>N(E$_FutjQP`PvEoHm=V66Y=KaF7)Dxjk|80oCvXfbn zMCD;&cnh>8;;D0%dx86VK?`H`F`XTK{8O$lc952V7JCoDdT|e%(8$BwVm8M@oNLat zd8<;z-0Vi_|M8)5fEq|r0%8F^!A&&KJp;!SlU5`LqXeqiZB@i^W3Abu4;V+y)jXnw3x;rkJ6g)j?$b-u44iBBh&d+Px9R5Y$<^J*h zKY)k%?ny|C^mrIv#OuIDEd!rWhXsxZyvE5XPrGRM)Uv2I`se~k06W^G&<66D0%SMT z1=j^UDXroD)Gms14Y`jVxtlTC0}%*N;avDDK29I@yAW!~7(l4kj)J`40xBjq3Mlh9 zH966%Jkuu;VU|FF%g(No)V85l?UWg;liEFMQzTK)NyHg(bX0A!?FEZ}c$Y$xCWq3p z*sVs;V8?G%O_{SN#ix2{UP9T=>>cz>aYwB7!Pvv&eD_ap3|9A*5qMp|ru@sC{(KZ? zacwwUyMJI0zd3$%W#PUZAM4pRve5IK8x}I4$nF4($Hn8K<~T20|K#gB-b}lrz944D z12eRrC=RsPnWM2FiR6Io#2y+Bm9GMoNY66hDD!K0?y2 zMfbm;hUh2kr~vM8`O;*ki1JQOZvcdUwmfYUWU%wHBLwq5lhWgqf7_`4Wuh6$xM#{p z8IiHfy-Ji5e3fIv7rWiiE+^>ybUg0egu9fQYWJu#wa)Z+Y`ch}xDI?3VK^HuYDP=) z&eI{gVW;;W>`>lB|2VHCSZnx#$WwphE7*qG%Tcn~F=VvJOX;t90I0RIiV=gQUgd@_ z?apc1=aSN{WJo6@2l`mjqzi~%IFii=7P~AqoXa?!yV#|?(QDWWNT^VbtP6T`(jxdD z#Q*13fr+mA#!iNO)$fWP@3Y+aQOH&XXDL2wPPm%Jauf^~=aW|1VS!Nv$LcVAcPDOQ zqpUtZ&Y>)|lP21-$r~+m9YL_EKYCWUP@Z5I7C}(mBnY!KuMBLjYfz4~A9o%d$_m?2 zCe*Z$ENHOnA2-~bD5^hIARfFU@Lp;Qs(p$+VYdXw>EC&3H^8F66lt$p4DS}!6i2v zRgb-N<}W8!j_1ize|*uBLAt%Q{7hOY+B+qB&{FKJL5G9<@JYo6oZF>S8onP^2aHb| z>^81nIBhfTb8nsCmzt0@qp_(lTLa$3CU%JTg1b~bxdxK|K20s9(oDXZ6O^lWl6Vco zBI(#TqHfsc!uexJ-Sph0m?pVO(E6Cl#M`ND_b6-3fBR*bcAz!*wD_6CI(n(DJZ26H zt|cip8-5unQFt$4N{VUqE1xAwC%2+KPo;VE3OJZz`!ofrBvHgj9BuQ5$p5tBCanBL z`qu|BK_vb!xPe~nchu?1@v=dNR;NpPOUkeVHV!+LIPm6euiZm2b;{wcLJHF@8)Hkq zfM>=7la!UNo&yn)c9@YIdQa}K41+kxvtFfr7L}|^_nJowc}z(W?Bo^E|0r?{)6}3g zioz`3kS*`5_iXm?nP18KM&)-C?-wPeR$Eu^%P?AYf+a@58mS14jp~r^>o zdi7s2>(|94S$4+Ke6&9HkQ$Xe>SsAb@jks!;?s07EdF|L;_?LkP-=#mi4`qKU;o_z zylNg*BTIY+(UFecpdi*c;VRoz0d(FF)R0geA9=8``*j(+kc8$Qwo6|ekl>|7xPG=A zly3UCzs91_%xFoQcH6xE^~G=;HF4?xzyG^ns^Yma8f2 zXmA*v8hJYx(7;OXvZT2}kAej66TKBi1XCXY*sF5TDIrdMJy70ht@+yOktfpR>0?uv z6BP&g6c$qEMFo;igHlbX+|Yj9PC5Sc@CfcN?A?!WDX3UI*UtW4m0Boy9w;=E2zSk; zRWV-UT=I@2`p)?B$zr8}U2|(S{o>?q3VVQbNSsTunC1q*H=`+P{7{~ zqp!&rYqkjbBpt-gupON=z1nICsW|^@tJrG~QqhW*sNAGsEnY96OndefsFHuU21*!s zjyQ+B8E)>W*u1eouA=jGb(*gd+-#q_%w+!LoMh5C!&co^&q-oEduMoQrMu&m|9X>5 z7F}wIb3@|TltL72Ltko3I%#UjPh~t~fvwUx8nDTFtB1`gj*C2WwT2t`e+z& z>f`e!Vmbh-kIt>xoR5GWUai$fb|nT?wlY$fz6|aj4%+J6a&%a@W)gQGq3O34Kh6^n z7EUzZU|tmtN^tEC?iuR^EnU}-NGR*g@3Svo z%uFppXh^fYmr9?lc`v5fb5o=VEth`Nr7_*0sfrfaY4ck!e;VuzF5I&fMPhv4mL*%e z$9BfB^w9|*r1m);&PQ?H$zdT`>sIYvKNo)0WxlLn?!2g1KU#Mn0`IX2z!v^G+$q_W zhIKB?o#$+3OJ<*$DAP(Fl@E`xmLTFozYM$bmvud9K8S#lwFh>SAA>ZXJS#NIn8+|INGW8207m<; zRS>sMyCuYom;=F{R#aoS6Y!lj;d%lJV#yi2aU#XO(b%&wk$vQ~gbYiDAlXeh>p3!i zTrcoLp(4&1#7zD4j-$)+%EIT6=1}C?WRNp=@YY$)t~@LV?YBSLmC_E$vUAkme}13} zKZcV=AZz2?W9Q}%F6|aVa&&$eEwMViEiGE#c_lr%ho@rP+Xw0K>I5~PY&FHOCv&3f zAM-8r!mmY4hZRztBW4U>8!1`qnMK((LFpg~^Q@O3^5xB}-J6qp&9rGcfm2RDnsp_Q z_ibRkA?Y012^DiyP-W8_ronZa0W!+DslDdS-_RM7*>t%_m8B7LHs@u6a_>G;aM-2m zI+qjeH8gphi1n`@@9zkCq^2tKO_^tJ1$)59E*t@-x7!D-IO_H zeBy*-nMnsbvErjlLDd~$20c@f$tLA$<6Qc9n$C|n^;`_DJ8s9iG149UZ!_xiVTf;f z-<0wNZkr$j^?D67*{5RIf`oP7z!fX|yci4_A~&I#2;cm1?}jMX^XfVedta?>VMnrd z^?M#uji|#2yfR6v1+XyjJ$3!20^dtp_gJ0C0ZU;Ov zF;?Z8n3fylq%hiSmD;j-rZQPsIT4Rd{hJ@Dq_Ad^sXKIVm!SX@Gd3j@C_>|v_pXq`V{5@-x zquo%4GTC_^h|f;fZsmoqd&*BUX^k0c^1+a5H*84Qzt>94! zeR|k%T3eR)MtC-UWY!t^bxmfCl%09zJ}&` zY8~54C_*;6JVlwoaF&&Gm!CfysIp58l7~oPCF~15myZ^5d@fMPC?5kQ>b?7&6eR!S z=?(hbX|Z08JlpTerUyL5C9@q4l!A!?b50-T2y%;dIg0o9R--r|tM5NQ8Sy}#ibsd( zMCY!~W%cIQb(f!vUGEJSw~)~9)iv+%+qz#I?DRO*W*CzY`-knQ(8_Ql%YId#<$y8U z3)p@kW;;ZIvc!@~$@gYSubfHKOf$^}PoW#83`2iFwh5NJbU3N#W7WqRw&zmy2k)4& zKGXhTgb)btk6bdMPhz8qRuXOMkbmp|K9A>B z4(G5C@CRPYCUHxj*mmsM+uIa`CaA_-QWqV}Hxrj0(r?M@f74CvF#b90yj06X=8Iv= zk^|KHjw8?s?jrMe0j^uPyfM*VEXx!00_8j9mAO=S_i|({-cji*C4c^cX!>-}^q@6& z5M?y_`iD&N&V;i1Ndyh??h>V@mG7mxW&LpHUPA|eCbUGbR++D>9e(ne&|`y6aakxKQMtHg3J{fmj{!9@q=B`?aopIY10T z9lPvTfD`yucKX^;{4^i?S{_|K-kD@vfArk+CpTl~S#<<@pOU(1{DB$Gj}<&C2^I{S{0O&xPh|7TgA zWjV%4D}#`NGrI*ESiE8wd((K%P%iyq<6+7d@pND*B_dIh4hN3)L%s6jia!_4N(;!T z2(W8P;6?or2rV6}f>po=On6rCxByz1#*jCWZ3dj$3&U=!St5o&)uTD|nx0=KQ7BKLM4=)JncMn73ZVvKaL-dhr>oId|L3 z?gg2&Uva%^)E;AyS>jNsy0*0891SD4wvX6Qs&Q6uLjm+;Tc1w8tLS4^Oui*tNSp-3 z6AL8~H#@G_y|Ew(YYv+!%v94&bA>a>h)y2Y$ zE8}BJr~Xb{79<}7CHrwLSH%;BC_y~ER_qKl$j>kkmwd2f9iw{Sr)q8VXjVwQ;s%3W zsS|Rej|>qCja%(p+S+hGN^p;ugTCE;*zp05NZ|L&u2)a;ev91m$ok^JpBncQ6WH0lMyUj0$GHZ98aRR_6vv<>CCLdPc{bR(_` zXc=l1&@z+rRiiX7j0D?k2b&y^e`*gG{BZ6Cl4eDF zSy>NudUfK|`N85%i+R0L5U;;06DMq$WtGbxn&$bLjMOcN#SJ%$C$XjGdDEQIV0LCO zyz1A8n1FgBhwz=( ziQK_U0#atGU+d;<9gAfGcX~{AKSq13EumFa-Z6IamQ9QI3~v8#4HvlNZ9A_Elm&R! zy{jfm;B{(9e1ZkL$B;6y+G#`tCu`u%z?g(6)Hxo=sdr0vxZ2+hSWw_7cF^VX(GnS< zCijj|$0z0xF3ad)??O&$b}n{hn9SjNcytFeaCb|_H@V2kvhd(Bham@fLXP_@Dk31& ztF7Zko5apws{F48|V62vAFM0!7hNv zZtx$L0v6P~<24XF_*Yx;^CQdE!27BwhmAr?5ZUP<{7yvI`-o=Ub1b`pvO*wA7Tx@m z6Z&JQ%am~(`QxPwtijYM)-@m^@mM7SMMNy6_g));K04r|Kf21h(q|mHib=h0@};jbbI!ST@Y)b z*$0RRMe~)3(ATGKS%T_*@$PST69AW`%o}06!DEe`n>V1(ovJg@(WU%!=^gq@d!@gqVr z*{`h)XNy7WuozE;PhT=KPJeFWo-Xu>1j2!BbK3ca#2#BFsIVrD?XEpXnyc??ogCk? zcJdo(DWxHJp_9Y$@T3I((pjwiAV|j?c;PXnW$PT)SjU^{rqbZ~`P-d_c}q7}TcDH@ zSMchO(6>LJm>@3u!XsI`mGyn`KHHthuxEff9%VDia_=m$p1i4k=;rJbrfa`rDL> zQ`L9x)-`HZn{x1O>5mJ9$N4x*iMf5;<%4yuFCEOo17W=fP=Gv_u>uzy9Io=mR3~{h z$NwZ3J!yINQLmeRaxPD2s8Sv*iG*3mUt0U_NIYZ>je!M`E@Ud-B4`rh$zgkexle{CsS$X^Pn)iE}~~kKf<}N z0LM-WwX(N?h58DoS7B4>_gJ%;GP#j=W?HMdgR3>9~C_^Zr~41n;2WXvv5av z8(Jz$NE(Uh`9aG<1y%h_9Q0RM(+zGA9(wx`5gtC4_2}%~%M+8SQuzB>e|}rO4aWPP zRO__a8#+GS1>Zt6OqbOoENIoVCq|Cp-OEjEfuZchoQ-pX1@n26xruXPE#(hlN#1wfKKQxh(3J7bi^A!j56@7<4J}Nk1}bm(*qd&*`!xa=eadEjXD-QlKlboO zw&&Wpi?`PFu=c;Ca)1zw(bp5bix>^&F~0IMfs^#P-C9K21ymrR0fPPx!{tLoecDI$ zpGc2l{pU3Ao<9_09xkM{LDrh8w~$kA=^PqPeeiW>Pw=brcrAFpR_k4XXI-E?{ZZLm zc%c*E7NHEjjbI%&mZ*qX&`#>x$yHsPrTDWXzX&Dx8TkB3Wi#iOF@gH0i{s^4!V}+o z?mBu;t2AkF9;c)f!lcvdS)k4;Lj0D8D@0R$L7J=0!twK<7dJNomulJueh@E!^T#uk6 z9u0yX*}opzhm>lvS93UrDHGoMkzhe%bq#04afwbfZYZqqO*$=FCF^!BbzlUYq7D5y ze;o(}qe^0TAP;kNm~?}`j##rOh}09;pDLoC&NTZ}H1RA$5Cev z8~uHl${dVN?KfOH|FZt`1jvPo$%b9t96_U2wvb#vsNI9usXEj>?!gKp7I99!WbO8c ze3>Mdr)%g@Uq<>4J_=10x|{~nS_@@P-OBBIc3caVa0$pXXKLN;d6kXRA-Dt(CJW}3 zmicFq&{Im_xaz5%t~CH6YStM8%!=_fZwcP-f3&4UBZFa=aEeF#`qrGYr08S;N_ZjzvviwRj>>`^NXc$+iA`wmB?3*!mtphHkJIt? zSUMJ89In}d=b(QT=zQu|0O=-bN9PY*Li~9$@DAA}+=4(Ck0t-EwBJ=e;7xKG2CuK= zq%MY2*%YWcT98n*496z)&A58L$#8!tRKLpbtkU3I5B=!~d1;YTzw144foB9r%b#_D z5#9s0WmU7#a{`%Hmd29>r;G(P$=9k9IITG&RPQD>y|{J7W}6 zPb!_aJ`J=!N!IlWc(r?Lp_wv0tfMS~swn?R=vZwYp3oWGC|0E=E;7}rqRP?))pi16rV-A%5e@0D_Kp))&lL&m#3siy-4JQMo|bnwM1X>)EocC{ zw0-^bCpDdj6huYv(c7s{4Zu71q+weKv_$6eo{2tJl0h?@-X;z0Kf-^%Elh}Z3yu-G zDTtcb%Fk+_I9MyAC{vt)U*c}Tsqo~9qFXbq6tDaI{Tk(nYSjoleVL*J9)^3(I?7K6 zyCg-Iz#>Ve3Ab@TIFN%dyc%@Y%c`ZdCUS=9kEinqMP-==b?B{fPKa4`+N!3`1DTsH zuZ_}@N`s!bWUW$gFmw|leoXpb61R90I8|_!p%Ux4y)gWyAD zPP;>~2iMICYbJ~4mBuxV7LgBJnmvvBK5uAin+8i!a!6ZFRrku9?69-~^;~#nA{-)) zwbQ4&DkV`{Tl`M@6>q+8=9)|$-JCS)`lOaGBi~hRxT~_+`tn z**0Ab6IbNn-~x*AI(_5@s(7@l~r#EY8(@Mc>N{vt;oVXVTCoI>8qE@{8~%$_&OnM zOPiq&8899OiGtWsG#j1@?a;zdAiYv^1A?k)Emq{v#;MRf?J@Nm#SJRQ8FPmdo0KXg zV~R+rcXJ*`q;@!VLa;4Ha*C1k;%B+GkiM95y|1U&$hDJJ(e)IX38zOs22B|+!~55~ zrk#yhE6hE~vdy;$jDzgy{{eZw!B6(ZbG%K42%c# zQE35=rSV)HABS;ziiPT$l+{9=2nT~}7w6V@7252h#tSHul~%cGb6Yaz3RKBz zfEG`&FHgmf#ZWFyFmQdP*jE3py zvy^?=K$BV}uCw4u8&qI{&R10i!&E)Hg6+oBcbKg3Lz z0s`bjeE<5VmHD;geMPeKETMShjE2r|2j7O_0H&2AqelMN6p`@R)-J__S~yYZ#^7f3 z_p6}yh*Js4;x~!)Q#cW8jhY-ffplrlTrJBY5KWdlvdPt7xfr!CBj+m;BJ8e=>b$2h zn&4Cw*~F4(lw)HPXgJS@7eX4`i#co1Q#UBrq+@Q>-!$@yKzpVKvTD=ALcTcn#=9uz z4g%&^kg4GcQf1Yutt^wEtKs7pM_ZAhJx8Pl3}>Pzp^i=9|7-8s|Cw(8f1kVDl1g{Z z$*5>%O{EWVRw^Ne896KMj?5J1(6B_@mgqLqdCq4Vawx}gNalPNW(+wFlf#(V_kG{r z)aU*@?$3X4?}zu}dhGq)^SU2j9?efU57CGm z4GHq()aC;`E)!%!=uJ4pE!5koTcZ^B%p%>tUB&a@*x8;tgTn^DGd^6vm-X)q$T^NS z^SKAR)>SW7OzqU14mXTy=!rQ>yfnXXgwWOz31Bp5bYdXoHbrbs*`&^^Dpx~qKlJFT zn1W#_?-c+Tt>mbkb1NCIy!6qHKUjAW*MG_Ajr^7CFO$Nn98aZq`ObKc=lr_4RcNB` zAVj98XyuLiM`gnuHP|qC^So z`Dwn@O43Y!9Z{0py?f#UjTJmBDc0@78O@}@Zx%g4y(oFcBj3h9BoCf6Sc$ppZq}5u zT-&maT?}Jvp_AoEM?L%)+DEb|C%!wBvZrnwID9PP`|2t}jEFIto!Y#dsiviUUP6Z* zo)+r37w|7{3XD5JV^GK|SJ#-X&(`QyweF0DwCF-gRbHc)3cyE(Rue>>_E{7Si6fVt zZJ|avC+s@o?O4&4hSN(uyW9yMX zVXNSyyhs)|S-2PcGY`kJe$uRRz?vdT5IVv=^sA*+*|SfmsE)!i3_0?U5l-5Y*47;> z6}~YWKlJ)C7QTmjB-^zqJ56{&0Lh{HZfl6KpK9+`@O)&pmjeJq6YS3FB1wzz644ch zGxncMz8SBs={k=OBDx=T>FHju}@B2{sO&6w@hZKq?C;rp%Lk~ z&VDI_H7zH-m{IyPeFfY)w$nU8_0zOnM^Dl#yf`iK(MkZv+cNQ&+z`()b~b zR0ckr{KdVJh3iYAt`NZbToudU%`l4+_R}Dh=P+Ny<_f+^QR$2*FTeERY>y39<4K-V z2_?orDy$FtndzX59`smXBBLL8eCtnQ(;xWFCE*Ke+h2hBKvoo9a)lGiI`fY{+%0n5_%5W18wl@9H_TiKk0M&Wtx?Z*k`|! zz0-98dx`s`x_7B6-2z{U?(RmzT6WQ`Dt@MYfW;+1K^5dw|NASaChXJ133SS9tKsCK zthgXu$t(|@WHz^7`0N|e(n;tSj71<{l!q^}qJnPBR-_caINQGX5{6T%QLtHKnPh_%G?OCk-#nX+i3grV^%s(@iX81#$W38+21obNwb-k^kgZvm;fC9dw zaxJ{^aXZGOrMI(L&>T8ieAPI`{8$PW7s)8$>i6qxsX?3&n~uw2oy>Zopr_TZ3S=MJ zMb?EozcU10J|tN~x2P3DimSW7lxa>Gz3qer=2?o~GpztyPJ_4OFFl#vK=Ftl{JZ{k zLg2Lsn{;<#;pG~btQ~pU{eOHdb26`!m zPN8b>uf>UU|WT2bUevn>k(e$LQ?FSqlPK= zwN+rxdo@xUG+5~H94c^EJKOJ^;ZHm>VZqU6LKL+pewNsyo5?C>K)LC-TTdm&33$(TI8Gey+ zA8&P=S6>r30K5FgVJYn`8QK@pLT1ej)sQ+FcRL#EXC{s-8!BE*?@V}g7oOQ|v(0!7 z85WzW({0DuMGb#13)b@~nIYlMY~gke6hX>~%a-3;;pvbR0xN#RV~P*_rgmEXt{KWp zZrHSwn_uk_Yvp^}X3CltFRhgtKd<3m%+U(gH&8K9w`a*Y_bkx|+KESQLPp(h9~#_% z7rhT1W-47fSWD^BFh2XMlW6Z%oh-hp*in80#dqrCk}Z}hd{r0mI$FcO1jEZdoHi!` zGcL113Ijz zH3L6@J31y#{VDC%I|r#&vsEjq=~;GHlM-mt{RE~DrHiT$!L4vdOA?U7Ibm>KZ{OXO zcw3ZBrNOjaxXgRzcr{HE3z89UTBq)zX2dC?UaBt1F$4*^&l0RakF25$a~|ghz6(Pk z#c0`&epTcsNFm*1Q7`O@HlM~}$rW12&t~eU(@eqQxO|#513*i@DTuDfLu<6JzcK5;An#s#aIoMP)0682i+db zvWW|0aTSFnibLuCQg{brE0!!nSF}%CHs|^W?AG|(!Hu8~8*V+esA(=O?QM)P=Nz#< z^c`k)>ORwwB`M7uD~k&14g#I=Ew0NxX}~SFE_GKbP!0M}n#`Q<&5Jpm!(Ly^ygpkI z8O20SO4d95GPz;XHk&-@VpIGzks%;eEh7S%H|UfN>Ik|kLylr`8dsL`37BGaHQFk73fTypJ2lTQOP zT1e0f!*yhC*x+Ttg8i2~24AjL3@luvTLLjS5F8D(;$oQNxbs%L{{pD~zpHvcK;&7S z0s-lBu$IZpZZ_9i^iPgeHLEJ_H8Iz6UD1@MT(RWqTgfOT{>)dq%9)9#?0nf}V*CEtghpa?Z}&(vVB#M*k7Tq~e>H+yI|C+Hu!mMFXQ;>s0H}&M%M%=uFOW= zBp`#)7y*Y?n>kv3fqgLs-(rK2Us=3IK?YQ`AZHw@c86-|7#rp4cs~g>UN(!mGr95%zr?6DX>!q4n0(xq9Y? z1Xm*4OhO9MOqjc@=@)<`Ef! zXXZJTZ7Xn-%rTb$Wn26M3_S~-%U)+hx;XuQ3c-^5w4XpX9_m#$vW2+M`YHkWB7a>4 zK>JGf@i3QY?J+r)lTFlzR0JVGCIS-dgrj)HdnGVd?fs%+t;4fw&}maRG?*^D6#Zc3 zZ}dW<*y4MdzKVdq9?Y{n`;K*!dk8;5dvVHRjFmwMN(Mg6yYkz_TtnquadaYGdD8^)}PMHY;dmBELk65sb5dUKy#6>02lTLv*T=)8_shHUda^W!vdB;LF}jsWK2EVZK}VAm|9b%Oe*} z$xUZ>4FaqtXXi5je}rbr%82I*I5}zGd1GO1W3X?%s|G_4TB&O+ruxWJ{7#_Pb|yHg+}#pwsD1ZZ5ibt*7`XW&QPh#^jWX z(Hm6mp?-_cS6HxudLy{exKtYC}{>9L%bJqS+pIIo|`&Hd*kGuYX}@Hl&l|IvK$ zrtq2Nodj^u8bSIV7{3|FqVvNQ)D(pjQO0K$3IIPw0TY((XxJva8}DXoT0#CI#(KKD+59E= zRDds^VxP+{J+Zm7b!&98hg3}J65RvrKr~_)l}fLhns7;8_{$G}7)JmBo||-RS?&RT zYOq?eT>{VSc71SP@1)>u`Aa7gv^S!^J({))*bPHOoU~o*_~VZQg~ULWkH@M0sAKg+ zaX5r$fkABcAVhZEMFLxQ++Atz@ch4P|pLf3hf3A!BGXMYp diff --git a/docs/img/structured-streaming-late-data.png b/docs/img/structured-streaming-late-data.png index 2283f6782f380bcb352f294635aaf4165e1611dc..f9389c60552c086ee1e25a4c04d6bed92ac83d96 100644 GIT binary patch literal 91513 zcmeFYcT`i`*DlPlqZ9=_2ndJ-5NT4R7o~R;q(lO!5PAtUbdE}uUSoiObOi(nEp+K6 zbdU})w1AWtS_q+h!E?^<1BlT{YflB$iIL%$GxtH)vI#Kb~{iXyli;dHvBj z%?k+wL&dNExD}ss^Tvgd)#X6qz(6OrQ~!VsH0czd;h$=4Yh#PiVHd*>_zxl}PyK(N z|EGcfjRqJV^&t;ws!AO1M{aMUzE_vXX}#S!|551t-@myoIfFFTXan(T%p0=zGHvBV1U5k}ZEMIP6j$ZaW zgV2;WUA9ec<<1xn_T1msKNpnkp#6zFt_Pg-=4@-&bAbzz)f>pKY#+CB&kS&mP1cF$ z4j86~T>Ja-r&Sv_B|o97XQluk(bE6zZTF~Qg-#4Y%qHxFt#AZ0HLOG?{HYjt+Sefo z^2gtID@sMYe0NBNp7xFavkhaA-VwH>?9{y+^GLPsfm9>?6%{E|8Q3}HJsx2T^s>g# zZP)_)`IEcEywAlX8yT@j2F03?vvJ|=jJJAA-=rckB>hPO#ok5D+F-Q6+yno|D3uAUt8;9eHrMh9}E8#gQKfuD{LUJn*{cCWl8X5l?}tuw~rM7LKo2@Dr|yKFG!(G_((f@&rNieD2I+l!b+2qLg9gV~m;webQh+H=XzZU2+zy(TacF3?5Lf z_xh{Tx=moOd3E5YhIt;vxP*yX2Fz3fD_?Pu+=D&o^5-k43xo9bcNEmn>Oot)WkjtF zUbn<{$E~J-lr4|@ozN_~KHo?UqDOZ7>#2R&gg+B4F2qzlGf%A@jKVVmaECoxwNsY8 zb@~2V>m=XM60zs6V@?DC?hX0jomlB=h>)kI{$0RpE3r_^??1ZU> zPkH7%a5BJsQ(uHhq(@i9aXe~zF793=h=}Rxk*?IP{_e&8{of#Ku<2UG=EUSMNCIz! zaVDxY@q$@d^I9NTS7z`|T_Sz4ve;dRsrh{DJZzI=L1FV$g}WKY<^8c`_m<^{!mhwK zYJK6{#?#MO_=>;EB_3u-^>URSLk8o@U6l8XZtQtCKCChrA78I=;oPA5+MDJQ|W z=!7yYTZkx*5);_e<379P4}x9@6EEh!J)0_SN>lHBDea{959)2EdrpcrTzPax+~{N) zkLjyiqlE)i1mo}sEeHo<6MZPZV7kvO8>yxx+}jp47`ce*LO_PM zWL=O7-?#aCURemiffZwitx66d_s5raAGNyUHFSvE{y%3~y+8hOY*Thp`l$TlO!A`l zi{#_lNx`H|LxSX5NA2SGAXMX%EsbRi$Y{*V2nhQpCuqbYtXn(g!UkRq$+r57uyT4a zBmD2qCLLsZPO`~9ysVn8vaip=s|djvPx8m&x8EYHxxmO}YzMBr+~poM>_g-GS#}l< z8wWUP>2t7rX@Z}po$K|;%X7hl!`Wr`1SFNiVcXmOz58i()Nj<~kLsKB2T69phFh}| zYW|yf3$7zk9$m%9LxV#-%Z4OLlUrWqMic)!1H()IBnV0CbwCJl!9FB|hx{u@E^B!} zlVJLmENMR86XyXGW)RX$31`Y;?5c~T;Tz|RUCe1S*+VahCIudi(T&P>APxdz#N9c5 z2obRlZJS6jYZ>>w?>-rxUdQugNsh?Z5O)@4<2U7avv%~OI~;1;^s00_FnLV4*l;r8 zdC}=gQBG;`&pMjFt@`-f#sx;9L-8iPZ_bxj>JChU*l{k4uq!ih(^o$0`_=0cEm?86 zxY5D}#z{>7+$OFQ$WHypz|WlBB2e!RW^_G7JL}cJ0^HE%RrRP^tKC9g>j|hNDLP!J zFsV>GF5>W3YQ6Hj8GA8Q3EWgDD@}HieK^>s-os<@wk=eZHAtU9$A9t-XUBkE2gVwS z(=)=W7fg74F8$AH)DxR4Vi3NN5Z3Ggo{|H4r^rruLrJ6?HRW}#V2W|y+Q|;B%6756 zxNn@?vu)WyK31v@HqThTQIQ~;vW)$XYbVGTi(?Ipz<d*d6#;Jg>oH?l4=XyQlwbEGg-UAkJvwci1u z%yTO^sV7s`sF4G1F_EA1?^64NQ&!C?PolGz%m}W{mpKiUBLu2+fIUAI^hzjcEv}S% zf4o6>##Y3a84sp^Pb6RDrJv{=kwSAhaDj`^ZN-!*@izvx<`?w}4B**Z=u)N_U?vB1 z_obm0G$$prgzb9pFvD#NHnE;n@no<|@`n;!X}kE;xO8NY$<%}`c<#0Hr?_9x!mes7 zN+c)ALyPi#33h^&@X9*uhCJGKvEKO~vM>~R`18U;5DvVPW_kBNZhEs$v| zOV~EQHLw!9fb2IYsWwNhfs$~VY*PTBqN2V9Zd{AP_t%<9Sv1Ubs=X|$;t_7?7mv}D)a}R7s zOlkm)e#a}tvVh-~71sSa`9^J>aqhU@(!U~PDl6Ah+Ih)4I7o|ian!BPnxuvlR@CUz zv6h#tMVc-=i99)k^2~e?A{dn{#V`dYRbOt*Epkvq=9TW0{I7!emSwdcn9f9uU*T&G zR`gWbzJk8ljXUDjUy{@qh931e3zXgJLZ0cWNUq@^Lux#xmZN+i?&{3789NIjaOw}z zN6m2FS!W{cN&VXwe(V~H2ZCGfT$}f4=RPagAUEqb;x9ZEP(T+6W>q|07}v_=&^*NMOiW9hPInQy!`S`53NoBp0e`5rGL2g>YGnN{fxJ~!@AQrhfW6hyc zQh4Zfg$Gz#!tvNEl> zpVn)>F7!!{w*74CV&Ma8uP=w?PAgcip%||pCZ%4)s@IT|BjfAO~>9GjZXz$AYl)Htb z7hCoU>l^&)9|BI(q#41#R@Xy4*Yx!vNV_F#g1S2N6tD(h+|!18^*OT?o8l|W6rFk^6 zE5Do&F|=IC^v0^aH%_SX-mUPQ8Rfa9 zxaM20e2T13d7SYKLVTDwr=tH>ujT!vcgaP&EvtQyQfhRz8Xt;RvDw`@GO>Woyx}bj z*I1m--jZOC82=@VM)2a9YO@SCn!?zc)PsVn+AX(4i>ofbB~Qr!ce@fYf5z_ASI0Zk zq?qDeD#ikj&Grf2P)paZx#9-KMrRf*Z?E8}z8te+6iCf2y??R zy(d7Q?NFPw2Ry~S1?8tEH7WKS7Cv8)-NNZzIITYstev^aGzx|n-*YZRQesn`8dfy@ zKc^zc9EcJhGrzMIxOwAlOhVV>$m9EHgfQ(JS>8kDV}asw1^5^JiS=sF*f)0Qb%>12 zQW9N#uJUgC3xykpA->&B8P~!03oBD83+>{b+vgI`4CS>S9mFw<8{2Hc-#h)M#xt9) zC}*nXX7SI!1qpWD-jeic9B7A6G5d9K42$9-x#0BD<{p(icSYovQtj%0!<35Z+2ja+ zTkONb`u5!_a(cy zKO>E{@sid9Ft+=G|K9JfD*>rSW(@qc)2UureVf}kM`Xds%S#xX+E3Cy0Px>S|Brk> z&84jOS)jeMMsOw>TMq*m!?gPq|WxbABVQBsNj zT=Uf9JIhPpjFwWi%RPSD@)9d+L^^;yexBpYG7ix&%Qsbs@FF8OwaG8WjeA^N-C%FhQwt4sWVhioI65W@Cs z_t+sPJaES$@SiCH2dYabU|Y*ub_8uDHZ>3+JCyrTZo%F&`O@eNc166B&Sz5H`@WBbqvysuk*`mSAPO0DAIjDo?unGiwm-c0gd zX0q3x-4d6L!b74=aDOy_DYQXHlm}PRdU)?vy2<)KG{z^MJhxwbvKMJLWt!izS-x}N ziWIY=m|(BmJTlC7t;0Qp?Y8mH4Lk7NXE$AinV{^ERSZn%>gF#z)Uf1?%Nns1`haP0 zFh=JyN8(h?^Dp#&>+-Jhs&#*w^{OjSw!NZ3oUX~PPz1{$Z136FPY2LT<3@3m_4;Y& z_G7d4yX$9~;}-}rVhkp z?=1UNg0xmi+->G;ps89F&j=ZLdVacKo~|Y4nZ*!op0OKn1I4`87!B0FcGY3kb4L5A zvasSNSqHfuS_Vy&FKmaVR^>NHs=R(Hd-`U7{6J8AsqvJ%Lf=ACM(aG`;?_Q0o2yIO00e}GPZZ#h9K~3#xKM13Nun9Te zIfW64JUP0dMQ$?0sdCUAS3k5|s=4PIze+O2O6iQ(dd;5U{yrMMxUL5k8An^CSF_x5 zkG3X!l3vI_BLq%;Lw59Vf3RTL(LZp5<0@(TVQKFd17uXknIEhthm|O*Glu#F%+7O1 z6`8Rm@a-u-*pK;_1&T*daLKC_`!=NbE{S_!G|W2EklTV%g%%-{O-8aSHomD$LVKFk zd)7yO47S7lIye>FW>=H(O_4*t~0_U~zr-NSv+UZ;Ve)sVGF0TZ`Li8&7WSY!a%)UsPX+*o^- z>Cit5U^r{t?MIxIHjHhN*!KJ2MC2s;(2a4Hw>nRlHz&2rM7AI|`lprE$K}U1zlf;C z)we^v-Kk;GC0wG}K^oti68@#KkjdgaO+0Q%x3Fu@Pb>lB<9Np~lb*mSsDOh-Fw576LynL+9 zbw`rzPZzkrj!MX}UppP(4+}!*8uj8efHixnsXgyh8#m*6gw7`pgDO`5g)l$+{N$&IMpjrrYGfyT&gIm6{sMFuhN;uN)lb?DPzb zl(;oeNIxt6Y(ZMs@~dG6L{ptDfqGB5=)VHtY0OmWUP}7`MJ~@)RhCAx={G7ZY>}nw z1EJaEujY5MJR<8i{o1;-JsK;4BW?omL1d3y+V_XM~vXp;s;oRd@mKAG5XKb zk_X7?CP{qwTRJzzh#IT;WpT< zob9%R!>QCh@lSK(aNesYdaToX72H?3zSx?W)5F5&wr6>EiSSIw=XTO<6Sz-#M)DM4 zxL0pb9!UK1etDG+?CqR^3C`HJqbB)!K#ymo25$^93c;VMTPq#M7X@v-O;f25!bKk1L71~ZjO}VZM2TfgKennQpfeY_;b*DQ z_T^|xnl$(JJ!65%{pO;dSoat<4R$v6L$Zzz3gD|if!ZB`!D;PXw}LqYOn}5sS8-#U zTX-C*t)b(+DDb5C;kqRgPI^JJm3#E(|MYhOZrZheFBf{*p^ZR8>BAzE~ zWg$%IVX_pRaZz+enbnVgl97<8I~8;?u&8~x&tK>Hx9j;{r7DgKcB${(iQ~1NdZ#q5 zIBb5pAT^kfM^T&UZ?(1lH?^_C0e+WdA9A$2iZ%=kB&4FNL!vg;zj!AvJw7*Me-rOL zTX-ByM-@6TAx(HPgs}Qsl&wNGjS_SABSdQX zk;|Kmw#~L=D=XPZY2`~V#$&zjQu_pF;4}=VVLD%ahh0aLZ@I$XUiFuFd{|@nEa`4_ zTs2678#KE&uFi0HIfv2_6s@+J1J6e{{K8bKj@b` zrr`z0KVHo;IT9-*qA>10%wo7q#B2BM`C%T6uHGlR6_r#dE%0ZlC%r^hXAWI*+1z9R zTOwX%J`j^jFuzF6#C*=?eabx#!Q<%2bK&>80 zy=WRS`xIFpY#t_!P`ZRGkxt|M()}5v znu?j)i`DfW_jkGC1qy;Q_X&O&24~OYV=28H3ND(nE!*dkiqo#t?Dev^2DEuo#t?(@ zz6Aiy9-9f33Ep$Et!2yY3gNl2u}Tfg8$2&Ope{>G&7_;LDknr)eJEbk@TWG2&E6Mr zVBsILdL*18P&l(|+Fk4WB>K3B3%ysJq=B+_I9sIOF_BmH4*%92)yQKO4or1E=W~m6 zRi9wRNKIs4*!`H#HN9lDF#YKnbf9+L@H1XQ>ai&gI&M7kc4ScM%=sn0$lQAYU3lvm z?Io|Whh*Q(h8l~7$ld^b*`AQ7c1z?*=(AXYF0wD})j+rH4cHoAxfCew8Y$2`k_d3B zDIAplEJNQQ#jz<)ncvQj^FZ&9Dy)$Uri~`n>1Eoo?i?oJ~!$CR-o1eqrWNmPB-!?q&;V5m#X~*{ z?3A@*Wb$AQ7y}&FaM*^%v+^5e;mYocj%vwxDPXwU$c3l@+Z<)Lo@Un}(L@ZiV&V2Y z-yK;JyU?Pi6hA^)g|}%Ud$96}1*>D#GYlOK(|+WBLqsJTUy{|v<9D?#>krK>2hSs- zRDu-M?&&vI49GL^VHYS1cn8U@kPn-K%a@(#F^!B{kixO9xj_f(Yj%Cspk5fq0^Ha7 zFFn|C*s^}A7n9}@7aGL6(Fo{z6$~T>k&d&r>xE~Vb5*B6>elYZ@qPa-Olj`L_?W0V#G34bn1;p2@h)d64Wc~R@_G^C8k?DgKk zxkYoy+qZZlV3@{*CMLJitIi=+=jCr3*T-}@?#XHge4>8eo7#*BXkC6iRU^NIX{>6& z(Ct5}nHl=8#n!`bYSC5+)cZcu~N-;O{@8{opJH<{GD zybzkQ)B}Fgt?AM3<1PN>y+sXq(^+>rPBe{ITy3c#i~FoAG%k|oA`{qyEum*mS(Loe z@vl3JMi1Ek*mp<0V4crTfZ|kYIOXrMI4W1@-7TFNA#ov>$?J^d!wa=?fz4he;C!d* zD+@0k(-x{{Hs?l9!G?AMk|K=dY~qL zb@qPD6y=%wx1mG9l1;wG=l}U_b#s+6>GV6OUL~%aTsQH^>wc6Dt-QpW9g_G-9RudD+ZgwBR%_U(>feSS8& za#mJZ@9YUsGo203PFc6pJ~$g)pk|^>jpUaaa_ww*c1m>NQp(xr?HN9r3Xs=s4Luv4 zonm%-r*$@Z(t43Hho($!tkT(#a^|mS)RW(%yOd(-v4rJdB7~E2@*?HbW3}0jzen*D zba6Mm%vKW>PdRx;*xcoEdHQ>FCU1QIM8%opXO~e%H~yYho>Z*t!rdKb;)EvUn!mLuLUF_v-(d(lD(<7yx7wJ1B{v->@<6V+XD|3uHYS(K*;MtZ*Or z^nk3xAXL7nQ48D}-7`9y?A=Eb6-F5l;LiG8L(xy^KuP{u$8Oqdo$1I|mLIA{!zhos zo)o932yBV*4}tI$f7tBxWQroee{4O%dqQ6US)ZrLFaT z_WBL$SScS%kE4Xk>76mv525pr&wQC80$YP)7tmlF!X?dlu07+Pr0t>KX1;CuBag0v z4Y6X!R`5W(tIqu8!GzyBdb|%Bo=s+(P>jd`goW`aWCk8>eMJjq%5-N0Y`j`oDHeKoZAne?fSlhU7}$7MBG?Fbc`+PLOO7Dt|w0 z%0u3)Cj_1Bp+9rqH|K`XTDkY(L*DWneB2CJU2L(MzhA%?WtS+|klW1qy|Ns5jQ$Jd zU!{|>Zfx#4@`Riw9XD?uRKFl6*5JY>rd5j2wgQ;HZz6ST4qYI^{`F9}=ex94&W_U5 z^d1dr7~ei@rIwBu66gx2DgJ7dB-oE~t#41cJ82tdF2Sr;; z49M`j;NQU$Z!}SnqJ!%5uS1X)2OJT%`Q$SM^tdl1il@P|5x975f%Qb$Ph{;# zc99&ImL~1@MD%kt5qQ>lF6ubx$H&#KZ0S{bCE9OHQp;2Wf{nCXpXyG+NCl5Ym z=1b_PKX^G$;nf6!&|Hl0!ZjbDn42d`zw=;`?n>DtTw`IsA|lJ{D2fNS1q>W!v~+W9&&$^|CWRIbW^SL z&XQGM?93xEa*yueu1I4qjCFcP9(Q~x-PPvjSgA8h&xao^A6Q-q>O3jzx&d2?FfWW{ z=9|#QCZ3-jRjb*9f$E*NpK4>nmpvL+zgo8(_pgMYU-&}oeyZ9!i*pzL=N@HitU?3%KN0 z*H-;-2}064H{>}#^Z4uG-}CeD#~m}KS10VIFa-8qQO`-<<~t8Zw|>fwjoa+)*=JOCf;D% z^N6mK^P=vET)d;`tK181B?)?X7gOd1zTHq#Kso`O}#A_IW>$dL; z7p|q2d%E6~u$ry_pJUvvUi!7+dOg}l6x^J@N% zN6fEG!+aFP7#W2Mdq0e)%|Zq>Og&Og|$Fz39H z8$&v*&l;5CJUSA}y1W8~yKmXES~AUf8-2u~8HvXeNwR20A&+Bc#PNhmA{}k@GkWK> z9(flSqLl~qc3oEv!ck!ymPZ@t>IQ>l5ItklA8r3ITJ>+X@T=aLZ@^8viayWSml?Uh z?16fIP=8eyb-}MrKvI>lG_UilK*%v2#=YSLhXz&Htb&;nD4u-C7%S6$whVtj(~%C% zZ6}ND&~^r|TA^8f`|veA)GBm*9?&5c#@#5$7F)b<-8zy76xMTlGHv&T0k%LwJT^I7 zm}34*JxdjSPQ5m<5vX}{Nei3Ol=Q{zLItqSrk~gPhesZ5l6iJzSyNjm3`Kj-&a@}7 zjJy8i`mj~R_xCT(Q%_VRLV$|jL{(kQMhq8(c6Qcyr5bM;ARs`_M&GK-=!mfNbfcu^ zxk{7JU9I`+BVoL2A4~oBRBoVTEWne!9ZO5!+ zOgzmZW>J$*tk<}m!!=RS(?yXx{IgLcP@i4$i?1SRm-IXVxfm-l%4Imvi%W)hY)Q4_VA zA&JYpaXd>6gADI^4rJJ^2XZmO*())b%ZjjWDU6I95M`l}CO%5CT3G8>X}xGP|59X9 zW?DK}tvAEK!|%F9N1oxiV3?x0Ch&Rq?fHigIkMSh1IN4UlpZSKDOyIEW!NKZqFjxp zteyb6qsTjHB{>`tW#NB;m57t`vjwfMw*RU-DuW>y6|f`1@emUBlky3xmJYknNXWo@ zH&%N3^nPmCchNf~9vOMOhlf|_M?$nwC5PpfFF!~yvj`XnyKJ`PpmkEn@iBfC=#00A zYt={SM95{c9q-c_AVoc8Rctq*cH4!3GOwLNY zq5M8j@oW01zg0l9{^ARZ@N|dguilQtbcUrCQaPfRa%5T>OtRoxA!tDp%iBcA>t%hl zUdT0>;Ul;0$~>NjyC}Jx1M?MYW|`~-f~Gh&hdjE!U(85xV81x@?YZG`vU#WU#Dyrj z;6IP%hDB|l!pw#B3rLy1+sf}btr&$U&b2J3R82C6T4U1Ww$)G087ep4W6 z{T45?r(ZN@$85Mk0me`D<^6JAJd}YyTcpnSE*AN-5TB8y?B8td*9r!fJ(iA+8Zndy zwSs$KuF}ovcXvptFI+X_B+n_whRRjuijcEs&M7zd9EGuNYHQHn3%jX(7t3(_wNo`v z0{hT`m!ijY2N82&&&4#U^iSpS>NVwWHM#WCZ8^R zewRVy+5>&=vn`Otx-Z_s!DB^@eHzfG0DZz*1FuY1uPaycytP>m52!SWSDd4f7#2b) zOquk<$+^LCYgT^J$`M~^eWxL)*_-%#`kW5Jh@tiw3(*sz2_lOdd+s0n^oi$^*|_(} z^?5Z00avZJ%UyOoDKDdEdPJKoy_!hmn8H>%!m?2-_~crgG&(k6z5D%6oJ!PkuK%Sl z8rh@2zT$-gWV?KW4+Wk3rq|kZcT*4oXVIEfio;n%zmk zQxTrejO0EmK0CIWJL2T(y_#hqT>k1j@};nTY+Ek4ZyZNqQ#9;_Qy_T9 z#eh5u`jq27PhODcm9LcQU5o+|N*NIo0E2t2Q6cE0?aCkDZ-qHU@M5=fbPoMUq~Y+J z>oc2=JRXPg-ma>N-5-&*$mEg#IdqJe^d};82d^&r7Ybkv#Qmtet|UPEPG5?j0ZWw( zYFKl9+{gJ(@bcG6nexP`YQ#~Sx(k)L98Jf$lA$c>ie6fzljIf& zUaUzbf6{*T9InQot_k8DwpdV$WSp~lqoapv_kW^l?NxI7B$fgwig|h;z6fq~8i7NA zBK~>Odm&tXw9W&%c9Y>RSN#y|4cr_4GGpK6C2I&rVxw>fshYg5$6yb>ljP^ns_Fkr zWnIvaaL{nP!L`yLrXTGeT-THIK_+hC@P3;qC412v{dBHqqzTRLx;NrS?0LvC0UGg3 z3LKY)zV)a(;h5WEqQ+$-UGb_Ihno z`xblv476kv#;yzk(}>lwaXP1{@VD9&T|XOiBG0fp{Hf!hR(zFejE2ck7TJFOrfRP< z9~Nf1mFnAJy4qwJmvaxRl*c=Ml!*tJ^&U39ln`J}9}kJ5+xvbvbj$C9N*KU2HCBw9t3JJUP|>LX1WBV=tOWO6h?3~mS0G#wTTd|S zPkdX`f2f+v6|6>~(8XIs8-nBuq39fXD`Lra(u*Luwfhx6Qet1>?1x` z%Qsv4m+x2*YRr)z75b(rUD5UJLcTQ53xfPqk6ZJ3Mn!A;a-BDM%D)M&zcKy@MY}P` z(0Q;%!T@UZc>Uzb;;Tej%B&ivj-kuxE|sCaO^c1JdjfEN zedSF5PU-?8uW;(1+o&~Kun&h{NhK}~FJA-Sr(rA7W|X`YH^{NLt?TTIbnl#;XXRGU z+0@6u*rEG4S$Lj!WyvF7kB`wQjp5OQ<|5byxmqby{Zvs;>cFqp*;yjkdVPd7oX*79 z4e_IVJS3W3S$@3O@YwU-Z=UP-A>pZ=d!F5ot`&Vu;|k80H2HX)SuOFM}5^ zp?Y4yJyF~(*oK=yD7bk|OcDLCsO5IK@5=}ww%!K^^{QLvxzSe@9=1sq;~UE41qw1e ziU5YO5{li|3n{_kZ?ut}jghb94c*`R@L)nzZ)7>*T^}geOl-3gonav+Gizj@L7!^v z-wKYAJ7PSddr>f&Z@VJ^FN7r0FcoVGA_}=dtsQIJo?G^BJZ#o^Yz}o{veYcP&!v|I zyF@0VxW)f$V9~H!gSjdMHgrzE~?vycDyD?~e{VkkGS^Slc)h6A@< zuRUv^O1{J(6w+QNeSBUya?iK3eBd?4X9d zys67!qt9bEGmZ#7(+qvi@I5l6Wt% zoj0s~M0Jj(U5$v8=-U;_9%6gj<2EObad0HH@BCKMW##wzOVdVvJ8J2?Dj5F6^#UXJ zIU_JRLx3VmH$R+zH(#eWtBV)rU!>ANjREr6xgTuq-6zZj?yZc*+ zc9F9GmsM%$;}wY;Fp_#$>vwQv7o}ZV=3l3CguWT0_}&B`=dE&c{2m;E!uq>BVjJ_@ zz&Gq3d|8)^!+1dhuv#w9AOT8YHXW9Y80iw&$gm1(#=aJFnP9WxHMPDy6?u6BT!8Ve zB^9D8^yoi4n6o`Ag>%}QMJvB#76#9{>9&Jcg#&`hP zcz3KL8`~OclG;zU$S;JBXM@8#%c)b#LFuOJ02XfFk_Yv!*j}%Beu95hu#sMf2tn7?zLPKyqj1PDfk??L7h64ZQ&dT&i z|FZVmW#oKdBb^U|{(cpMviuHG>c^9s(C5u-e87F$jHc;H~UK zAOj5a#ZF`BRwnod-ByevQ(g{Bzi3iipxWsQJS)cNj zP^aSEb-3L%-YM2WlQWJku?&S@g}JI_#~Z=6fh&vO+&3Qs#b zEFZ*Z|5tM(bKux>9?qLqEaYO8`3tm%v%}#D@2?Gdb`HfsG33CLx+x0MN%{#^Zq5Ix zv~Ds-!8hMG4k2q&hN8%NI9W+nY`Jx`|5Jfkr_R@CyRWtIfL5%{I9E*?_eIzAy(l%? zaU!LDZRJ)I==YzAo-eS!`^vOAleq>RYt!f5e-4KNm<2WFRRJH7W@y$3CNYdcZ*IRj zVXk4*O`&=!#voLubG`=GqZP@-{USu5qN-hc%YAfKMXO}fJ}#8t^6pg3CGK4vC+!|g zv#;h{n{8EDPDs@L><~e0&>gn6cyfm&oX52DG74PKlc$_0Wt=is^U~LG%E2`+nA;`6 zZnddd_k&lgJ0r87)0TIfs~xEdn#9(uDIWE;8CQz|Cf#+_yd(IGe<^VD)mYnHjfEQk z6kR_!SWkUtE?iKVLDLpRT}ZoSTYOnrL2)g_Tr!2QLiRj-eJELT*2uX6FOizZc2^e0oY0EpnR)|2l~77lI}FVL z6S-91DToBG%sK|Sc;zq;Lv}t;rzYsKz_SLV2Ks!rT`EdW!}St*a+`V20IOTOpRe7^ zVP0WM-w6Y@nf1;Wi=J$Sdi3{FhcZjFzpm0p+4UvO3v~^FsHq-tVMQOmj-7$9_poju zVV&jj&bPBRB!bkD#p&NfL>SmCC>Xz*b}bMaTCeQ)(k?EJN7IRaI1o)O-KlMrT;s6B zQ)g4(lcO=-}JgZ2+`}RLkDMe zP&0Xj&Mb0C-83`fKb+;W;01|Kd3fJJE8>NMU)({j0N>2Z3MSr%G0qP9%g5q>Rotpo z*L|u8ow~S_F*+%B-p(Rjg~EUbuR6ZS z^2?ZVj|v-T+7jNmzN+x^S5!87?I}c9-nV%QI%wXnHzO%&n;8&2^(<^`%C*eA|G#&p zL~&2pm>mkAwOv{k@j5V}ysG|1y+Y^dsq6f@&B}Sn4;-=5lg#XV#&5xWbp8Ef=ofHO znHK#p!4e!S{TuDKa!293;JZRQbySb{^iX%0aS}KUod_%c%RFDRrWw`3zfN;UF$y_c zrmWSeW~cqnaux9w7pY-S+yv@jw?5#x=bbsWT*$+KUX^fWGwX|fPdg6j)Z5D&!{@*= zo#js2(5ZdG#(l{z-^4gj~(juyy=RH_dzq>d-AGnYywNiGt_NC6B@bmpSrGx3rPzx{*dxL)8 z()&)-oY@CVY5ljO<1Is*<+8wH%i4cbiMCp}Ut!m}FK&76@tzKj42xkU|2qvS>;!G`n~j8c=N@mY3R;NtdOlz(`4>eMDtHk{yZ-k-n^eiT5; z2Q`tJ)iS`jpKT)?68>%`)x4G1r?hx}jKcbLO)@Pr9}laMDf`d*ekrR7zU9oncBCMiS{!#X7GeijzmaLy9L^i zoEFM)(2tk2aO&KwHr%+N{pE#C>xqBbT8+8O$=Y@aP~P$Bo^T=e`BQT9A?(y`lR+@3 zLG-C!n`G`k+go1z($p=+ucR-ybq$A5609UPC#klKvQ@$TNwmc^%5GcgX&t4m<)gV{ zxy_CpR25-qo04M}_*~~!9zdHH@;x?azO+Lpuw#{7TC9<97+>7>!TAe=lK022%EaR}LSC zb`NZl@xbGcDnPhtO|SlysHYa@oj?(0&vk(54fRgH_I`)(}9 zMarHQa+q^Z;_TKy3D|(^c+_$UBjVWM>d7`36#~vy^E_+fO7-o|>qJQst|^rxml_Ii~&JS7VafdAfaLTY$)G_1I$dr>nXFlt2k zX;DnT{(EV=W_fh*IOpU~^o8YYkMFA>VdDHqCkeWh3r*vlp0690pqqu#+vbpa0U!&4 zyq%B&Iaw=MU6)69;}pKOeVDWoe@qk|^g@M)XtVh7z+w+)`vS8OaChvQ&6shF6_2T! z_~fjP0u~}xJSS+Qc`{=&r+9?1PNg}(pZ?^Va6j|Yx^J6U1IiwNG>o7!3`HZfbYMvNLYLu#gk#EO&%?;ZU<&-44^{Xd`F z*L~jSIL_l7mk_~H8od$SZpCV+w({y2Y;z1``tF(11+?XuiFSp}FN)cmx%B;TjH=&m z-HK~#@lVw#B*C1d3-p9N8cJ?^TNH3JAe$>$zgV=Rgr^xg!6pneIW2CzyNn|Z)geb` zD!l%btC0weeAPC8Z)Q{}42sG6QGsA)XV$uaEEbXYfP?D~+XV5DtK(lXbMa~VvygZH zzKyOKDkK#5wN&%}ycf>%bbq?B3n$z5 z*_ULtdgh=qO|Ew`H9)3%Px%G8ts;*HTJHCtNB6M1AaTp633yDP;tur);*H z*%}?N^zQP`6`2FmOL`j|qrwB0>ubs~(b&jCIqJ_Xy19p6yn6*Muayk2k>4?)n0rK7 zl)+|blOi<2TsX)U5KMy!$gG^nwb6Ll!ICMdjvhq9eSk2-7C3+cKRxKVDd-JrZ_ zG1-1_Q@!`i(yT5(?JxyxH0dX1w!6uYKFaBg5=cA;)D#Zh=?Czp{!W>7>c>H*;g*(j z9zi3Sys{*X@2fx3+|cWhn^IR|1daD3s>1Ed+iDiGZHnR$|0Y(yGqRZEsdEpmu=(+JdBYEB)&CmxfIeu?i#nXn>{?@)iy8n8 zFzemAa21*{=?KWkBc7XZ3nZYI;&$bB`B*M#EtHBr#+ugg0%+xnYc1q`l2Xy!FB1)O zwAUQU#e6}BcS>3`0h|v3Imaec7X>>4n zZqkL#JuqWvzEfpeL+jABi72{fq4)Ows6>)iw^L=0|Q?$WT!s ziW#Lt?@0sTu^N~u+|QIolOviMErIJwk`)HiVXYSy!=&)C%n7PWDx-w0-$E9+A!Zy2 zjn-=SuC9F|=J`HPd`4`oxVCq@x7NK|WKfKr!JvUopR~OoJXfta{5=M`_`(n9JCcT> zCMRVUIv)Br>}_I#)%i`ESNH4V*CGo~#PM}hh3i*Mi(Tr^ugwZ&`{>qxU6|)no=W%sVJk|b0yUK$ zQYJ8{@&DFTR-WIy_Zw5&li5JDM)ptKUw^t0zIsOIl5-Hdmf9X^BC1jh=oPw<@gaX! zb>myIj3Ulyz|B+q`@Tm!g-(iWe#3C3R~#6@(4ozR%XQIr}v+Ii&`$%)<@w2 zL)@zW6=hm{hgK#msqa+UxZKy8)6zA})pXLEpf~_${ZSD{TsQg~g4+w4`>s6;vVHw% z@qTIF%w z(&ziE!f6edfE4Q;?wjQ%uL*xZ8?y2fJlv}$Hy6Grm>|`A5MzQ7Bw8Ic#SWl{2QOVi zMvG&pE%Oq~Uv{by9P|INnmoXz}TFMEYstjFo_(s?{ag^2G;T!=X`Oe2;1zD)Pk}U%VoH z1klAYC@nMKPt7}(X#4)mCLhNO+ksk$KR+$wMq1Rn$-RY>->W}ZwftW>rLX+deh`r* zuorEY{t^b?u-pld*Q&v`2JE}I>_CBfijx2M6?QGkr&)Aflb3@=t1^C;Sxfa-LMSF| zHp2E0W|(C3pbRqzXK0u10u?D2Gx4Q0S}zX;;>ZqPuP~AuEr5XjA^QsV_o4^yCu0iK z=_KWNC`QKJNdqssqnq>0GF$*Nwd7Li{C+h*7(Tav_QoE2%=hW0nV{R*YPcFEIiA^*vXTd!@ZFJAM zQfemWo?}4Brh>$}$1eCuih|F0*OgO(gQf!CEU;Idnp4mV%ct9#QEXF=GL99XLHChR zZPW+zcaA)T(Zr>#9|P@Cd}BaqgZj76b5&YB_SH=_I7%g|T6W=u-JeQ=h|=hi_na&I z$=Li9o3pDotK?xG7?{MRMS<@9lA5Wo828netq)J3SOUCKj(CrNOurTHi^rYrnE`ti zwgwF*J&3*9-g@|BU6%jatRpUdXT#Yx(}`n*Q|qH7KP^klNsPS#JRm*#AeuK4WNRE3 z&eZ_PX+L!)={ft8R-4%Nt5Bso1SyLDGSv51q~g4hnY4QzhH%~?&9BNeg1cpc71uy$ z{6jPZHE2{ZZ~Cv=S3zbur0v;b_54;(ec&HMF@dy z3)$6U`l4>luyO)B(4bjfiiu5AkZaMb0(0V^f`-XK61BPg5^D#T^N#x`0Qe1IZ3@&@ zmS&ot+xDe@nF58DN{>tt@40MPi%>RkaW91KDLuT|g)dd=7J5_Qf>^jydj?rYTIF9q z0$2m)hA=QNFMef{Wh#D>8(2T5Q^yJ2y0AD2nW`vos&Iw_OeeF}M}?hD<$e|F>OoM} zBj87ofa^a$t`v@`KX$8Uu$h!L{sHF_aO-Kjy3$PpfoRF?h8oF#HmSR4n$o<(zd|>^AdmPcp>1)58Sx1Gv+TL&fw1(_Mv z9@hQvU#i#R|tzX5nYb2bKk4eSkfZ(g>uz}BzQ5f{e&V=f4RD-AC~y5?Ogj;g!^ zJUf@THSXqs#e^}M>TmFx+K}k4=?&BqW6puAtS`BYReXHlI-sUMLNS8YrMkxB3ObmR zsvmjwj%5dGbv)q~ophbVHpWigo~7A;nI+N>zu+EC%h-FU>zCVlTkbU zGLl2{165q|o5%Pge2VR)le4zWk}sY-aaTmr5$OzTYby24>TEPFhdJv&6jpC#Z{2(35@fo<)=GWG~tuBS5_kBaDSp2zh zXRO2%CQrJj%);BAn~5P$Z00CH zwIE^=bLCE4-TD#mr&mfryDnglSlriJNFPxkD{%Xuzk3!W$Lri5!3Inv+J56vLLLMy zVR1dj@aZ1i)%Q=fS~dmiDdr3_7BA>&W$KL(7Gc{h7g z3Nu!m4p2D$!~+PMIM>7!2`kUhoAtQ$Ee$B=filk=_`i4w`Qn->Mc_EcPj_iWuR}52 z=5&T*G1Xq$%<>1R$>cwVBgjA5z82!LzcYWO61J$k40x00L93~qb_Mej^pTpR&&jBK z-&t~1DFaY@1hPgk#gV?1Yp;G1sOjQfT6}z_59gMpIG!|yq>Fu%>R<3WNu93nYf1Z( zDs^uzEb!WibD54kv)m$S z)t=7FcUgt)TYR+U#q~W_>0>_{4*u19_K|#C4=|?~fbp<#_;pK1&$$4>4 zHhT{kTJU?(!xSdP^aoa1@cskAvnnYzMniF$m^3^#KVH)Uq{`}A?KC`t^p*XR@io$` zLo~p`Dh!}8>HJ*(WIvG_z*H57-cQtn0k`(9S6W<&6PlbtUhB05lWKeF-{uzO0_haC zx;QDz=#%5<8Gyb=Wt#J!mCii@pX=#YYvnE8XAe&W1`!569|Z#E(NI3qNPaTEn7BbS z?NyA_pD&m9CdBmRzco1bFK?*;-|BEU!OtU}|Fn`XA%CNAZwnYsIKGH?S7~!m;lJcO z?UYuhw3%M1jhh!2=LM>VYUv{zOqDjdLg}Nl?QzTsnp&zoJKH`N6O;V)XXx8L$Dt~) zmFMnlepwYAr&?$6_)+qkcUk29Ya+=RK4a7N_kU)OZ$)UOBy>Afk`1&?J-_q*a)Hn3 z)V=3c4pepGWZMZ>*=`V)=krG-F+f=%&MNe}sD2%Y!TSwx`dse6-$uc{2DhF}% zT@!$Zi#scAzDbKaA(-ioe?G)P@wM3tdt_CV>*yeOMdCit@#|Y^*0kTFZq{Jy*18bQ z;hEw1l@VAgStY-IA1}Kxc)H&^%=5IFHtLeOzasyNRG;Fe#E@kNS8KDh$IFl*Mny+f z;nRxOQbt^-(9NH%9PF=g(_hb+zB$bGH1!Yr17?#&+U^ zf)`4>%VCWo4!?rOx{aU=MVyY#b(Fu!m~Y$MkLeqFosD~r86BOO(HdFWo-tVr20p^8 z^fkCG%`!$|*=?wj8ve@z&Zb^BZ$$nScm8vUDBidbsu2R;f8Ey{W_%pA`~{_BX={F0>s!(%V!-&k8yo*NnY(Zi$ZdkL4t z^|xVf!Jzx*Y0KW>fKAt0yWSrEQXuqt2bU9uWB9*SC?oEgt7FAGE6x>+JSgBZodv?~ zwzvY?r}t|p=4-2nR;6=|Pgu@b%6{*tnSE)~#^6R4oEj8rU$(QDmOuz@6-nP12@h%| z?+u*rIMuS7a0pvw!Iixg%;CC>N9gXi^A7J@^oRRh$X2XTTxO>BkNR&7*gAj|c@H%C zz1d^#;I@Huu3q#sYU&!c_?#nD>AdOO6FNha+G*NG7MT>rYz}8{#|l71rzS}x-Bz0y zEf8%zus;DL#!UO@+U(KnKduzl(cDa^xh7rjXLf(nPG5)bGNe*4K;`?0iKJv@p>~l0 zfJ%;a31IpIUl8oT^_m0ozU{}OZWyF(c)q*tpjNVMlkzAC(^IFJNyN#OPT#e~o@13W zW(+qT5mFI!G<3KE0%i^5UiDW2y})Q6twnK#2D*r%&Pf+H8RVRO-e5%A?PI+E4YyQQ z38P)EPgh(Unu~{K_=i?%yP^mDHSqh3@{Me;FTi4nh)|97_6G&lds4%S48foCq)!U* zUe&wEaL3UI71N+@1xbuD)n9u%6sUxIM5D>h-Sz+dm+-Xwyt%hh!bz04ou4 zj<etzCXen7i=the>a^5(vwp=0~~`sIOkbrr6e z#yMH#E?Y+SI&}d`4#DD-4&MfIQw(8E5B7$ADKO(io)1tGAy$#BTnMBTrIaZjdyDD7 zRrGuHI;;ondBJK^Eq^*79&Y&G<{|=ZzfFzbBD+nW2@XL1@DxZEysE20*2z#_I`1$# z#;Vfx^gaglaj_fbeg_q2N~8}bdKR6Vv3P~nJ7*tpkNk!9c(~jpejaQXf%aM97IX|Z z839F%0<#ptTtj0Muw3V5TsWtW0|&?N=j~OZ?~=`3iM5;Rr|rOy<2&KFUm7 za3-h8&;iVqy@^saA%4b)i0x~CaFRGP{_O;@cEBe_cJJp#7Wv4>pr(eTgT0otQc08| z9_TyNY|Zx_^jI4AxKw=(5Lg8QdmpmIXM#*5|0y%OqnMiqCGPBA!BDYT+c&;10w6xc zsfzl6Y5@{O{Vfv+qb!>P|~cdFFv5=!rwe^aBBDX195}pQs0S5xBF(^(f+F zJ*KimRt!wLfiPamW-{h4ri|S&awLQhr zP=rl5vDv4$3}~YT=d!3hO2a>%17=^{!=9hsFBzpwr1=4mR8|T8cN>=V?140HW6?1!nZ*-H}GJ#PbfEfK;q`xjq+^T!)5;U+@e$sCgC`(NjN1anq4LaI=|PM=ToPskc76F(VE~o`JFKpZ5h7Q zufFR`{5=aCSv=lN%?b@NR>9%>F5`{3jG4T<8=Rkq4$Uj!;%@&QubyWmKIB|_eM4axsm*o>QwNrae zOU~ovgB^ca9KbCW9ArxQJsY!l2d+~!%yhe*De?8_^c{jm8ev6#xQusx341T@ zUdqxPq^>(#tEK5AaE6Hb19a|Bua7G$8EsM;03A**JYPgfYV z*=t5M8JceJ_x%+P`iPKfC4N@04N2^e-JsS|&v0if2Cd?T2Ec-n&WI!gyi2wKCt?+XXns24KK& zL)H9GBVTWtb0zqw$~A_qMKU`5oW*F`4yLY5hizHQt^}-6e5gdT(f<4U#?WO>;e_KA z&&@Ft>Ie{C?e&5YIQc!|g=2?WS*I8rLonQc^E?X76(b`2A`k1x@3>8av)JN;~l&FkBG=9}y}SLx}BxD?Sw z^T)iV60$8jIhC%s1$eXWPT;_Tg6HpjYO;dQq)j|i*pK;pZ+Z;0((bo;zf}zRgwu7^ z_Wp~CBrsw0*f?RPRf2OFjQl3_Y4zFX^f)G3Yv2;M&6=kgD>=%6tZo9m5RMxib;9gI zEZ=Z7oWKYg?dyZPu{Ti-x{;j{h`;+i8Kkw-;++nNWdP8OmPQ2Fssri-z&v-_^Q1JD za}zjp!D%fwdlow(#?=K@`=Wb7Z7kC;`*_M0Y_oC1?tW8I8~NzqiIAFc7?9op`lIa4 zyIJk!7S5V>uOZ5H^I2eD#lwRcm)w~a!g1+3Y%zc(R-d&%nl^qii+U%&`Q2~g`l~8n zTpH5j{Ud!OExkbNYFfSb@_i-4_dn71x0WPm8QPKDyJ*-@F=;8@-7aH#Fh%~EYvXI? z|E~8jrO^8)v(e+oRR-W)zqi^AF?`9!_Hp4-@&_e-^rk*S^U`x>IqVe?22Ix z5dfg=*+>Z@?e~^TSASsgtTJCb6ZkYZ!Soi2G>Ie^bIGA|n3F$M?D-R6llqaJ)_OcV zoGTX~2{`-yZa&jN_y1CMNflrhd08~F9Qj4@WA5l1S;Ji9%&d(5%~CvJ&8^+fiJpU- zvN@pg)UrO=__-8|)5q%>7=A_f9Xrp-o?Y@4YI{dXgH_12UWcjq2FC_R;pgzE(QLLi zkFP1JhhMlRS@aW)j;t2Sa5Bp}$+_#?|4ygJMz59ESN}e(W1z2$IbwVECrPdm?#iEi z;Y)yy;>?RpXOzo=j>mET_dS`i-|UrtrVTb6K=rG!=B&c+3i6%_hM$;$f@EJdv6X`Z3H_ty6?F{g8TJ$Wb`)!$oVIF zEop+QO9(AG<#-Wb47o^6`;F*1Bq3KgckU&rg)}Yc2r!hxzT(r3uHxXT9{1%xt?7=p zFXYAR@D=hn8HKA=%tK5TpOrJlk9Me+l? z@`!YKgHV;!Rcy2ZTM%!3GylK6Cc(LXHalt%su-odXshOff@kndY+ew74@!jo+iK!I z*PE0n#m@!TY@ccIV%6u$adgXSOsxi@N>T1?|I+8J0md`aX#Chc&uP0@@**DRk&(C7_TJ(?eH6Ro2`=h(hN55CwW=zkaG zr2XU;(q>Uzn#oZx6S&Z>!OH6o&Y5TTqYc-Tw5*IZ8G4;8CslVuZ3iql8ky)ue&rkO zb*&pMr?oa67O4+6?|eLdmzdH}ixn;PVin$L)-PUsEJ=Vbn8QQdl+keT_<#2em0jTS8Tcdu zs4{!t{^h2#PfIIpe#L{j?jc0&?Mz0bH;k=2xNnxIBCiQDBl#D|!}3uuCbhMCnSQiO z9|g{7e}(?Ie@;{K8Myg?ul9zkE|}C&3lx76t_>5g{hSRzAQvUfY^0TD3Eymt)a}*u zyBW37N&LHxJzy1XE9w38{j+q&JEP_9ENtFi#sszAk&cS1>N)cl!3ka!LXIEXPs#bP zAmO~boh>|Z-X9W~G`Q4Ri_&Ky`?-v5a=V7YOAC${>}ABqCQA1Xma~L^Pw*P=xxqfB zhmHPgMq}#3$A6+PG71)u_@-z4$DH$3Ck9>fSy6f{w?G01r}9Y`)sUTb&fcN#0ANr# zct(ZED&Y;F0_Ee+@3pXtMS;{))e(8s&PM&6El;`$SLw=}-MH9AXacCB^W&OA>a1(a zS4Uh=ECRR0$WC}{9BlTP7M}kl+Lz5vH?N%793A$qMTy?KF_Hv@pL{k zJ0?955tJ5}pYoES^pPMI-oy3Y0)=gZomz7eP4)?eWNuXJhf-dhV|`~dB~m?)KRL?K z>KPUw*eR6LTmk`sE0VS#=tq!-F(kQ=tAWyO>_>Y5F;5~3Co*A15^g>vt8aP|J$Nqf zTIt~@3A#7w=MRoedkim?OBfse+-NT-)Faco`h2rco2fUBAy|!^e;`!byrN=n%@2@! z+}1`XG;-!$z{vsjCL>C8^Sa~5C4}k+laf5=t1s{}=Z&_OuGYlX|8>PrprOS4gp>3~ zd^82?;DW^XzdB?*$Y)^v@<%J$DywAKc?aVXy$@> zgd*aly$GtKOKSNkLV58HVZcJ}Q)? z+txuBp}hUG9r}A>92fQP{Cm*0WQwmR+TN5^4K{CTM3yc==aMUSqWk1OhamtqVc%qkS8&Rb|XXoF3C>mK2p@ls|1(D*tdf`(IL6IkeA6 zyW&w;(d}+RmH$d zszaV&2%8nt=#7bY2quT)erooOCj8Jm`kEjHh8xoFL#BE2-=dYyZtg4_kntRNeEQtE zIPr|!BMfQAKgUO(UZt=zExL4}-?M@tc9_rn3QuE@psqI7W2_iDd7$UVbrYr2RbNQ1 zEa?>he>@J6!{bd?x6hb=B#Zz18g2>OTRZ~NQI5dMl4g2BTeYHM;gSh-&L~mbV@@)J ze&NsZ0GdPeuO`L=I&1x|42ck2dW8v~uwqunJj(>*7O3i^#O)1Ox~duA-p30Tca||K z55*Z3+Gw>;|5^7(W3e->uWF%PCQD->IsH77>Wc70zJUE+LLFN;6N6NNC!?B~8Ea8- zpwR8OziOoTAAlY3f)Rzzn5#F+Ct@;Q8ePZ0>3U1B*gXL8GGBjYlzni6GT|55_vO*y zDi8)vF~MH55dH$%W|9|0QoCRAFUfm5Ta+S4e~C~0r}s!`HgUZE)j06$eDszoy0fOl z5M(5#3mKg(4?mf9i{OO6 zyfN*vd2^j%IG4j9l)lX$jNp7_GGd01*m?Lr3fs{MC@c{T?4#$kyTH(2YI546!7;;B z?+@v*{h`)!o|dIvRqscP19ECU_~y0Rk=Iu{8lNc?<)!4Sex5;!7rdnFBJ#EHGZmT$ z0-HtT0`iG%WLFcc1jZk`=e^%GD)Kg&YbTBN$~ZFQwN?b+EG8-+StjoJORF={AMsn@ zc8}jUxaQ(tcqhhDpWm6P8|xV{*|5M7bBe*KL_!HK1|R@9J4X%5M?q3T=B7$Oh9HUy z(52seI+nfPDxM(;ch`$b{2Te~@9Q%>1811V$_cp9(l>AZb0KbmgjFSp`#T=G*`y*b zViYK#nHgKRShLW{@D(1m6X4A|u;CAM_;^0?{;N&DWMu2%|7ff5|7a^E0U|=d5G*QE zjMKRoJ{!9~Afms?oQP6{JKg>F8ZUPOnkNw0%RF@!TP?`Gz}s|hFTGw_P*v+{5_67W z?+HSXQrBcsV86k8t>q0!Efw$`5B&niYf6}9Lz)2pu~8^Ess%aVClE0y4A01eM;YT;&bOq6muAxsZD;t<8aqP;Wn@sqO zdT!9MB*W8*sP;hPdR71WJlbOQ*8|8$s74=#lZ#=jC}{7emKr>xj~1&+S4=02!(G(Q zM*VQi)SG-wNO*U#>pPnaVN{x@>JjTzvL}mqbG z{N=E+SFDRBM%%hqAS=FFgKig(JWG>Be?5p9YRLbI3;*XPm|WO5Jt0t6cK*MB4lPY* zRqUk~zoE7b90`uy)xAo_X7k|G$#zR?5Clj@0|7$U*qlvz;oL#|wo$991Mpf(Kz*_MD8c|=(m z9CBa{7?|D?t@g|4o$6fnQVq^l0i01M`dY&j$skdL%GhS6=?%?lK2?B2vTe zS+oKIBkuA}*9Z~pBF@s`o`li9ieD21V&e9UVOaGr`D-dq5)s~7Rr?nTyk4b-AMHY7 zWllHLOx|SM2E%H-JhFc8zrMx9;4<|h)-&J4T%^b;P^j(A>J2dBnloni1;lQwJpL}h z2F;UC=}y~#`6*!>`r;TE@G$QL|E5%xf6gsE6C^7bJuS%2F`t?CK)OuaD=Ts%@XLL5 zhL=Y1vN(tita>lo+X@`i_r<-`K!a8Agj&wLHaK~!Ovo{W9`TKlJaIz+?y7+$dZQcu=IqnZ(XG@BYrplH&g=oiVJkTnY8*|_N} z2tcW^z;!k1tCyMCjcijIw*jiidLPLqYhs|Fewl$49kz`l@O_1k&Y%AG29oJg(_l-h z=CrZ8Tl(7?c#r_ktYCjF$CFbG>8e|v#v;Om{l5ZNZmtM|%l`6OSZb7107qxL!QMXVHroja<`yR>E@`#xE`(TbEF^`DJtHm<-s0>xb41EmT5?c%)acw1=8W48o` zVK0Q~xb2?HB<}CzBmR<uao+XTxT8;pI?tQS0j~EHIH3{1(*3O0=+Mf( zVlTX6`YfZMqn-K%20bhHY}XEezx62O_YuLl|BJT6ZnAw+=> zT4*07K;LD^5d0iLawSa*^rfeb);!=_P3z$Ws-teV*Pt_MM~cE6y3Sji04P0efO||z zb_K})Gbl4h8sPDgvtn@YoDyhr&$q!)OQv86(?WIB+DjJq#Vdzl^#x`lu;Ezv1A%Tz zb$4FiCP30H9N$JMi*=47a*th~Tre`=y0S=611HG8GBI$C;H8%{F*Z9E8>yR<=nna3 z2udDbXOsr)Z>nw%L#(1l^lpY{Mrzik>_0KbUV8*@`5L-~o)UM)NgSCcPV#J63B|An zJKbvYcjjY8vH_WdDS{6NPMEmIDp=7Y=C{g^`mj`=Ayk~%fHvL$k@LQ$8=ybv)dT-N z*s$PL8-AbJ?NBCI0rp)KuoQYxI5F*qv$)H_sXg;L>57c|#tao9An;Xi`t1nTmc|Lz zV*0mx3tW5a-*l$Dvms1dOE|(P58f)Z@VmUUM|HbwRV49emV4mBD&48?SFGWWCuT)Z1%PmPU+3H+-n91u zW2`VjU|frhH&wKp*PR~yOOpO}X?0dzd1(Y@87)%S2Mtt!e%2)orRE-Guy2g%qtrsf z=V=j4_vr!}cKhe|qBY&K68gH@0|f!_`$+Y;I5eZ0b# zQ%hBDTF)vXFD^E&@R?4}y*r|o2r$UG@xugba*YicISQx${iO=5DBx5U$UTb4U!sZB zI*I<;DDkYsEp@@=1-aL`$La4Nyw$><+`tQE_@|nZrgX!c3X{{Hn1sUkh;yP@s8V^E zx}R5M`Ek%QPtFD#?i!8QA_QarL+#+*1t*LF8j5&!8}V0rbiiBA#GvUj7LRxcm;S;+ zPP}#UP)vlXhI1Wv%OPPf`S*t@MDSH@SnZip@`-p7;uH0SUb27oWP{C}Cufw!K)Qo! z2$0V!hf|Uv0=U@91fWa;GISo{pT_|eLE<+t zLx|F-p9!#9@5q(sJnXwC>d7-2oMsGaSve1Jy0TTd+ZWC0TK*5JI*3AF_dn_rH9NL{ zP0U+KW0Tue*vb+Wmo{E)qaf?l5yyBw8uY@aOsF^-q%n;c%>qf$xpd8>1tmnCF05=N zwVdqpK-;ml7AN+lO=f~2_J`x}YpjLK(LEXU!@Ip4?t?7@W~H!ad)H`Agz283HeJT2 zaUifyy!y&iJJWN%D{sgcNfrQktL3H0sO%dSY((tdogPz0VQV^>s(Vi66AG;)6R*NCZ`vBi=+J2VMi8ivBw&S^ zj-oT6_64o(!QBE+(JfAr z##b&%xlM^^G?&~#3{Z_@sXoAf8a(>eiDz)c%TGW_MLSlt%L*`wcmX;hh_u-u&6C~z z6u`B*Q{DM_*C_*Ee@3F?7dnj7Z!^Sl(^1mH<=JB)grRh)|035**?{D8w+h-K5-6;Jyb;A>MyQRNAcJSren%gUg4zBuC$Ha zTxX-hkbY%jx4dy$pxi`DxFx4iy8)v=T+BNn0_1`T;-?q}pKzPPa81a*!b5fjs0%F3 zIPytH9Mr2^H;rGSQe7b(8P)+S-m-fs=L*{{Xcix5m3==DlK27&jpzU=#{GRfdnv*M zi}QCDVQyK_WVEzb>BO);;alVC*S@~&X&lNA#0&?M5i&wx=P9IcygdAb$v=Gu*_6g| z3r^}r9>hIv62Zb6PUk;RNIOC<;h?5D1)j~r8`l@QwgBkE_$*DD_u-V$BPvGn4j_Zr z2$0{vuZ)18mj>A3MZ6;S7!E5R12O*7I{{`DQ4@?4W#s{t?X$CHX_O?r;9ZYze;4b# zM3Pk--h2fo#g6@Uq3T3@BDP3EqO52tD##D~H47J+0p$up2%ya%4H`Yw0B1IEqx}bl zfT@@U;%b8485CXd`)YZk{qH7S^ungFW@l=7Dx|IK!=9Tn+TUPadq|nsGE;w3UwMj> zXYI8b_e>Uy<1e;ZGD>4p>54N;u7mB zZ|XB&^DHOsd6!Mc58H;WsZs)#%AN zEZyvsa|+un=``gf);61Q)1@Ylkzi$&6SKu9!t0w%GIJ%0?8HB2YT3e@N>Cwp3OB3k zR3Z|W<%JVYdfQzZ$uI>1AOq+V-E49)pi=5M(h-5%ATdi11%Qbgb{{^O(SdjbOjt@teuCH$hm zB)ZjX(dpa)JkPRJL?txidKcmekeC?c(7nWM`qR%hnM-vvF zw5TuPcP3Ye3h`^zG*XVX(U8KFhy^d4z1DCs| zh*0?)KAu;rfj60vL%h1Pl|qh1jipsU;kM?*P4oRJ1BBoc2a(JB)`+>y9S6sZ)!{d9 z(JW7I4p-Wj+Qm>4Kcn(dUjC6HFUG35nj}8S71tBzuU-Pw;DMz3A8wJ_H$~F# zni)a%<;Hjhfu&U?#cSJs5A4iU>|Y7G!{)cPSKD-Z>eI$m109bcS`+Q_M)~uSG`6Cw z#yO@)=RZ+?vk|F64uc`9Kv55jn3p$U-A71VA@reGfss1Hk)<04&5o}Y_Azi7bw#=cLz@5Q2` z{w&&9>=VAzDGgg$_Z9wT@|K-Mc-doYPxGRSKS{mBzo~)v#a+<9q`@f(P z1|ywB+|eNQ>4ed)8c;JGt6ZtrypTw$bblqcs%;-#r`r*V?AAW=PvohGC@8GD88ug&B zVEur1K2T7 znqr}|83@rGWmdkH$#T5;_;n2JCW_QaNKE}WvxVA_Ez6bj>Q$vQ9wR%+5{yvWqnO>R zMXWGd$>uBjjFL@!C;}-$PkEnIJRa6y!#caNEjBTJIS4b_dh~+Q&!nh9$f!6;>e}KRsd7?!inu=B zUsH0>q>yq+Y$F6Td(N-3zXoH(@PXQNR+G%p`YFZhiA+FQA$y;`FhoD4C3%1kCw zG6yo4=_K3uJK-eBoZUlw@>UOZ!Z%3`)l^EXY1Q&Ap1g8pW7CIKh*$f)TW*Gr2h`le zIQU$=Y8P!|VyLF%U*Xq}OBjV7q~`w$Ub6}D`W5nazZRPr5h!qWdHl>LyQT?q5rX6O z1tj1njh1HwK6AkIT_XN%TDWygJCv2u za3FQbGEVhLMDL*Y`HehzpvX1HYqU`$q`=M_xjLp2n%KEPKVdPFXk*E41SDynevMrYN zc3!*?hHX$B057W*%Ri))@b8s#1fa<8jmIRjA%pFu+=RCVFBLzlN_ta1nH(j^FP3@+ z)am~W)G^0Z9&K8SiLd+F6hRvZJYb1XDbD$sAElzQ^wy&uq!Cb~*>SVH72uj|%2h+Q zV18M8Y33Yq58o30=Uw)HL_e zA|U#hN*uTD{;6>0cHBdoV>0~>Djr(gi<`fC3j&kY%NrVWSQWp=m0Td}l{WNVyNlkR zChnqm%lZzm-omFgr-g`=$$PE!yOgcac33Spxv}saK-2;iUU|R3>6(&ztd4X@>Bnl` z*+RN{RV_Wfr03!HWBXQb6xcX8c75_xfryirNp^rYm7+q1A@KkbcgQeBkl=3t=iL)b z)1n>*a*pU3n}27#xp8s&+u(~@gRIWGeH5;EEoyUK1F9SpVesevxRKPEt7N@}mAR2) zR$sdLPa4-zw}Ormu=vuUI}w5K$*CVhUF@&VtXF4%z`~V6gtOpOP2?z zR?v97ekCjxB6KS~)U#PBY4Bra??<{NX&DBd1qt4lKn-Is)SSn?KyBnm4BcfKu4f6)`EdH_24nn$;Ewv1#JA%azuFl5H|#Z3ub<4pnt#LS*q@@4Um&&+lVIt{{j60_;wvyA>dhXAnS9)F|V z#Up$%LmZ&ZlY`Ui#Z8E0wr0BL{+)AACPjiCO7gzWLb!YYfj{bdF?38O$$F~S87NeT z6gJSl)2Z>_?|?Hd8A0Vle5=e`zKU2x;KCke@}>{)%rPkjby1~w5&$gy-Vy!in5U*q8>?Zet3*N40xz4Uuq;~{owfNs1duDkg-G&f@L1QS)o&Xp)ew*kH^Tcv-yGOEG&cKU zmIvPDiiZ95H}sHK+YJaymscU6x2mvOpC1^=(2jTtwf)u<1VS}T;y&mDne10;RGH>s zM3$eG=Mu8gY|xogh>gZ&%UggvFT1H0J2!c~R!wPlir`Y1M$KVqFuoWYluuFro2Q%&Zl1eBXZ9t&-W!Q87KW0gL_4R`Kp*$&>I^30-bADSA6V>^E|WAvLXH z_o^Clt0G8T6dHA40^n+K4I=qgoW@gNJH>9MEa2fJ7jy0j#D^2t9pof4HCJ{b{XOr4 z{UfdMXG>e`UJ_HVPh@g_RX_&+Ki0lH9Lo3mUs6(0QAt9|lFBHujx{?Wdl+O-*6h2q zNwS2DUAF9GAI2D=EMs4@j(rC$Y*{hF_e*4#TA*q)Y=x)QF=dF z3P$#G^V;p6K)i_p)tTG3Hdr8nfw+6`yojMq-&Y=zlFp}R-OBs|zoA5yYSr`I2;E<4 z4;B}0K2tck#XT@uA)p>D+=@D~d|?;h0fVv;>;p9xdS)|lkT~D`k-B&{iZh))SgV-P zoTFP=nEPRNRpA|X<5;9czMP^Oi?rWW#zoSqY5?`1^tJLVyLjroEGVC?|FB~9HFaPG zm+6*?Le1^MgAcK*$zpyDtprSJ2$Cs; zCv9oH(%!I0GF8(zRw5(PB5_n|_PLkQ;jCJqL{CMl?UwR`XBD0j znJ03lLnt zrl>ddNE@OajjUVl6;k#h>MOiRWxKPY*nVtqTMRf(G^Fr~DnMfAnk=JO?%Q9T3PRt^ z!wP&QNT(UTwp(UgGs;wejtmwrj&cjO2^#ccJDQ7qrxd;@tb7C+6gLVJ&G$b>Ln%2b z51LRZ)D;eVBL^$bl_FmeFJBv+sfx*3e1#fdTs*(^gbW zNCcl6c2FeozT;>m4@s_?7Z4R;k15QYUfV795Hj%IttAs7c`Q|p@5Ihi0OvxZ>ES)G zl3nlW34n7>XI(q4a?RNM^?PG(L+s@yf1%9ghsJ73b>?5VU4uwb;KJ&QEx7t(d{=RZ zi27ok8GGs6M{$0Sf|upsSI%!JDb|tx5$WRih_cr5?eh z19I+=LPP_pXyibzyXLi~h~F+FZ@8QK@>%xM)t%%^m=S%G__F+3AJ(|5^Y_nj-o^#u zi*8DgM$4APCb2asX_kl(!;aa^f{|BQRb08og4EozD=Ybp> z(^wviKbK-GGVqvJVQ(ti2go6Uyog@_9&G{?gn1NLREiPJ(bjQ3DWp(y_|4#aNon`g z%gPC+vVQuPK3I%)ia`qBym+h-&_5N_52x?&M{~fD&Ch(il~wFS|OSv8qTDjD=g`CwhQ(m zZgy@R*of?NciS;YWK)dR)kU0=87gM z;v}y9BRI`~@0WQ2k7Cr7n?VuYcETAmuD|n`Ow)IF81yP!Q!MNh6|eO)HEe;P@{D12 zPs@mrcl@gj)m`Bq2;C8k`<0C2WDD0_SLN+TMPH#%wm#vz6?7G+5p&~-u9qC#9wMWyKZD%c&#r|E04gY=#*!t7pBmi z9@fAMxFSQ#h_b~dlQp_e!$>fb?L(KVo)mtDR`vrecRYuYQ?SxW=dFxsva2ctH?@O+ z0Hgj&8xW{yB^OO>XMxak`oA93DQHvNdC zHk?g4iA%b6w`bddmtfKFADN5hQkQ7ov%UFYrJzE}G_K+F7IqtHPix?K`P0oOPO@)4 zd1_yt*w5;`Qz4a9#i*Z5|kU8e*@ir zt?JnQ#B*rJVY#wi@1fYVp3YIl@PwO+AO+YJWHJ6p*t!ulGpzVxCdQ<7_|@&nehbD) zmGV~`ZhAU8w8g8#7P0-2rN22N>P_YGPViw0t@Uoi9l~ExNUJMn8b9 z)d25rF14flz^NjZu~D4a;utOT78^eLhsZSQGrs1@+MFQO>_gxjd*RkCz*-79bipEaZq!0G zcnaAxFVwGOGc|Tkvilt=14%KVHOuE{;=wS$Fd2jkUym%+JS47sT=!8C=V3eQTgt{} z1do|+H!J??OG@5lj(Brw1Dw4p@!2e+UBUS6J_TVy+whp17&K?0?is50W~a5FYvwME zz~Q%7c!@QOx?{HryrqN!z=)-pzN;n|JkXSqK8R%;^A&^^6Os%s0VwVY}r(f1~o)TI@Uj@lN| zLgJoh${hLBJ&;_;mi(*TMC&M-5$LaUfASdF(=Ry+h%1*GHw#?hC}^m~mK!|e9?=B4 zwHGS=Br5GV*la5=4ifPusmoVjP4esh|LFpscMs-w#S<)#LZdK3+ z2ML-Jd4tv?_iRBRBzn4ywK(6~0_nN7?S3U0rDU4U_9*G&K*_1JGZQDYl)+i}9WU)L z$~2*lMn`Y5@J|+~y+95@xQ-_eoFH;=mHp&ZMNpnjp_)vRI*N-5UXu5V?y#2X`=2$N zwtDPEdgNHExQ2(Slpfy_6~@x!MEsLmiAKU5UYn?syB&S6oTcb4mRsZ((J7SwU=>q# zDc+`#C@j#jY;4&MYeMfd;WVH%)9qJ@TXe;EuC0 zJ{P2EG01t(I<+}cs61CO#=(5hEuvFcF7n>)3_0U15ZfO2tYZg>{z@0gCNtL=TFQQ& z?C=Vu|D|kI?TNLFy8WDu4OCrO1^o~tT^}H#bk-!Q^}vu`&>Wd{y>4h-v(oG0Ur0J# zDtcgq@}<gw^@?+eaPlJ=`>C?S|pUE763tSUKuN&2kEx}hc?msw|vA8Cb``WHIQw0{%w<-FoZc0X(KFDk%dnc~Db2Lrf^I_rH zg?F0fC+-w3A~i3mzWr0=1&TS1_Hn3GD4IVOL$W^VAf<{$o#55h3pfBm0CE18ajMK$*P9W& zb=SJc$=;?`alke{uX?)+d)^tzVYC<=Wgl|9S*uLS+ig)96+II{dy0OBc-f+5OTT~E zKWikbU~Bx1yEMpum+B*40mqR7mhI{?1O!|eW5xW3)d7Ejp?@*zZ_+M`+!Xh z)*tC8b+Pa3w`ODAonTjks2d1X*@$XT8}`YO=ZijjPqjN<1rH4w%E6g#Ju7_mAmA~7 z_z&gDYeDX8I?Ao^td0dTo(mw?T~j9I3o5IHYnS@vZxF2f+Hp}=5vs4{ZvtTjRwgUDw5a%F zLn5t}VrSwNZD;34AVcI{-34~u5h-B%Df1Db3qBOxKZHKfqTG((UclSHG(%Brib*A? zg^9J3U_FlauK;vP2_%+wQ7EhA3_rA694up%e%F^F6HYyC@|hC@@=oa%?n2J&h{`qx z;Iuq$fOafF;HcActw-IMf&2DnC88J`KC0WF#JC_JvBT!Ww!c(G#=Jqw=Ut)^#`Fv; z2URAyKhlIhPfFga(w0>bF0nN2Rna{u3TMo;uwBjfYVq<~8pJh2GPQPO_0q28~Z!~wH zJnW5zj}LAqwyZg`M6d9}YSgToI!%J7=Tb!O=^vg|%Wj0xa(E9$mo*i@iY-<w!()e;M5&A35Dt> z)m2!W?v85kXR|IY1nE}8LF@*8jOKbf&et&^x5?78k_?%6(;xYQd>FvXusU{_0tvq& z)8^_ZzN|ny!(||HY_Kbm`&%)r0MZe1TdB`|lq`K=ZJb$l`^xoeFMb@=VBHX3OB=eJ*wP9P3s zlkO(7H$OtA2zp#R14A1_d7J%fXj;<+fbK{jZV!-`465AkeRgz~TQv&L!+uuX1HLWLWb*NjFIVrKVG_W?-rQSS~Dl~|6$aJvMf=|z z9AO?S{XfUe^m}J8{ui~4IwBnZgLflX>)*DW{LR*h|GjEd@K{!7#FEtEjy+^45@S+b zoA}S|X56uV4v=YYqetvtgA2D1-~Cy?Y|4FI&0y!nIUBuL@5cs87s&Rnhdgha%(7Z> z#|>t(mA@;ut*{?;`=oWmF|p;mR&gzIgO6bZ21fi^t5}Q5V+MH-Qsapjr8w+swa&XHq80p&m_HAg3M|p3n=- zJbvV=m)P*lM^;KqRYpcE!R6H(uY?VwcLdE72S6(Hzvk|kqITxZJ-i*Sk(=X{rOp|T zr)S9~uM_5CP7>w@R;t%k)81herzSx=h^p{ykBrmG98>={I(e1HAq-2I44V6!X6gIml7%%<)D{e+t9>?_X!L zrO2|_vX$juXIsG)N7(LiJrF}KPK8N~eTX;;Wa4kR_WU;O#KPCNDHzSCWZ@6RyydPD z^6p)$kd*AXpjDtxOz9nGATX5ByVPd zO_JF-h@1h&-`{;}o-LHjJPk&p@PbYAO=FgouJTKhQ%WcQxnA^0?aW&5RWi0rg9`i1 z@AIQ;kPM2y+M^iu{`a#*uM!r%D4-0)iPMpZT&#a(%R_1zFRyU0HnofznMokv>YLpC z&vxpQdG+UJHt9g>e9%QDuD}q9zu){#7hgT>qkvFlKRlujZctC0N=cg2q@f@c3ZNj|CiS2@CAFs* z4n!Ln*-d}G;q9<@^re;9QJJ$J-(}XI1RTvqa0!=xjTW8~4T@#Q?giE8Iymv|)DGm2 zWge2P*i>()29+f8@+TWJH{yqICiB7U+S3EWy*>Ne2aMhRR@?jABM0;XIaXi_3i+2< ztRL@vHYF~&9{1mRH9$~Qz1YjyEJq~Bqg&DYO$+YQF;yi5il?$*Jm$*Mk4?Uv8p?L& zq?5@o{k7to=$thrX!mr)oke&VDVb(9VWC%q-t#B&KB$|?gJxaOeAuyROOX3k%zV&z zXYq7hq=&l)u^T@B> z{~#LeT^IR_*bxM3Y`a`(eFmPCFQWNp4Gu|2MGq?8 z9uoUc(QFc3!3=<}c}VKR7D}Lg>mj*GKY-FbT){tggT?q=y5a8);40al&bFiy3nyOu zuGIm4BmI|j_1|v!|Nm}Df-+)*rLWOvv@nq9=QZQt$f3ewRU8Sw*-1|&kq;SsIDU|rEpF%Bq>t?sQ04St%vxOs`7Se%UEiFQ`L={oN zWy090;ETy%Y#B9e-(O)q6=CCavEFy5K$)sUN>IF4V&eX&;%IC>C>vg=jZ#8r$&M=i z9c#o+w`Tk6@;75-Vn63mCU8bV&qL>iyMgh_1&Gcck;g%~;#Qvewrs2NdzlsD{5A1u z!jTU!J!-sjupdK}Sjo;iVXRn?+r0CeG3_qr+)#1TwDFZzeZC$H|A{)p7p^@u`UCk7FiD24ke(g9Lj)~dmHuu(F_}IAuuP_Qp_Thud2mAKr7m{maUz(?H1Q zHhwrqnU9FQDe7jtj07KeU(Tyw@vlD|WKtG}aCCDV(EeiIHw1yvb)<};jzcf0iA8oO z75(CHoI3XN%?aL#3iZzhD-hL0Mw`KBn({)_Ng|>C=@vaR;<1vFu9( zUFLZA^!YD1Bhqv5Y{1YW?GfgYtc4$~%OYi?(_A;Pr_`d((0?iJR};& zKd$UNq?ajFuX)9z>B#JOJq<5GFc;k%QbptW3DdN&5+zmthB~2hwf1mMRHQjAs$)To zJqF$swD4-wluAAKWs!BZrJ@16WxS;U<3X^Ry{I4h^juHD)Jw|>F*p8Ud+f*+_wJ(4 zM7Op!yf>aBB;?o(#&@lcvVAAst|m4cI8>g7x#~jcR&t!MIuc6*7IN1qtm*+}>I7i9 zIZ;F?Q;n4g}`H-Z&*-WC&ULcrjFw}zTQm`wEppHkfxd^MC#*z&|nHQ zt9fU`!v|d%s;iWuX6o3fadVHS9Kx;}7f&^^s!m~*9V9Bc#Qg==7&CW`{N{)%<)i7= zbY7%Y{P1_iv5s97Zw1bdxox$zzv@q+{>)h_+MaFp65Z1r!Be&SLdC0WzA7#~8?j8Y zFJDp+sEh#22IS*n`=YkH*GI?ZXq!HOse5Ruh)C~(-#=sv;A>t zx^3U=AUDulU2l@PFWa3f^R5w>g+76ymAI3G1dCRqRV=%*}#5yiSY^7M2T3qKd zT?=ZnH#@+N`96oLXZT-XdHC+RLdmU9kG{Ce-hQrB7I0&84EdwTtZ~en z=k3WGC^NDnTqoaN_+Tp4g~s8laBdw3zE-M`%1SQ2<_AN2eo1q_;~YIsybOprxUdx( zMFl$nK8P`$UV=bttp!hIn(Wa?pMtWXuRl^PoZW>IooHa)iIfzOGN|Lkke=s2`yFz) z(>D5PZ-8>lfGRmjPn<kvMNx^ zA=@sIWyd4U*lWWs>ngmUL>N<}|6QsopX@za#&o+Ec)px+Dn7?92YwEkjI*gE4?V^_T_v!akTI@zw^%C(KdnQ z3IVvG_p`y{{|_T`+Lc{iE_yj?%r)x)Zmxpu>zg~kPC_zG0yPzsD3 zey9B?IlSc*4B@7OFXO}3W{m6?Ue+F`{ZuH7JKlv*yJ7+Ze8uZLsmDi||(>Ke?sj52e3~{v7$_^X)2W>*n<@hQ8Z0HGdMjQHe zq`B*O8&+|X)04x8t{@yy`K>s+M*)}*@+|i4mA@WfF=vv0eTHrTjg@RJhZ3y*Ufy!o zkqRez)iFD-_O*?bg)wxU8rHIWkfTlkQ8$EAk*o~KaoV5sWtGOQ_m012LhU$M5LfKAs{Gd__C- zaFb*QoEjKzsTq570WrUOWXL)eZ!q1=l(20#`@kK;R#8u=Ftni&eR2?S8dfKM<^=5K zb<)O#79^UqPH3ne=u-1J25k>1j*}D+Iejwv-U9c*j_c5X%fe9eZGCZ`OkdIUAv()- zf%sUWvZeMx^hY0q2Z2JKd!ukL-bd($H(h_m%YYEw9y$8iBKIN}nkoh+682p6N5W#| zFdoT1!&78eVQY3=%!xAKRDiPVaR|)e)`1tJ&UotjRawvWQ%av|#&LN_D4z%k}!cIQ>OXJ`$*?0fgU~vrc&RcDCy)mCFQ9jwfvvk-?SrDOUANY(PNmNZ!HMfw{PfC-$3SpI&++QTyE&4rFgv zzJRdET}ZQgu|$%HP{%IgM}-JHZ^iX5Z5Q*~#F@67j062j?CV;)Y-;Q{>7w11J--aQ zc`ub92dnmUNYW&BhZ#M*H&bd-|AxOjAAQN8-%Hbd@RvviEdq4Rq(x}C0V+k~?4kmK z`(vzE;taQbd_#>rwTe4_W4mpc(H>o{&xxhu5t|e*kPf!yoRDL5ctRtw_^@K?XX>qX zwLwp>u?#DuluF><*4=pf8pU?H+kt;56=ek|6=`DyG*RWQG5`D8jE8>|QtwbL`(k(V zC2Zn`+xFbd=ALdK`pK-XFb2Cx4l!=|m~Th!z>`jzu;Q+To6AySUZarp=B4$z_znb< zzh9v2B@sNe`~7a8K-srzH1HemYBQ?ARn-@>4RZP@=kfa=9bPjbqMXhF#){Ja`(Lkfq-6&(_&nG|haXwe7f7FnBo+1dK_nG^ z0tTeR|9Ta?ft>!IJN|hXc<}$c16-8_GLR%2{{0Tp-wr>d7yQ?d0#5zM)x+!0YhS1S zUx!5YKkfjpI-DWtt*pc|OYr7{6sCKtKJQwx?OEL4c+(h)SL^0`FT1~&D9JP0sV&Ov z|7Ecmkuu6Mm2*jUyn7EdU0&n7I|$>;dib=qDQ(nw@q1UJ?ye2Wd$LkHIR(j?q`Q3p zc@O2&9_NkDP3^Fm8y^uyCZWKWz}6SAFpqk=rlC=J*9Au{$#wTM1sXd-1pS%T$RzPx`E$w9M5j0f5_oSECII zd*kXInwu&CKfS#D$Z03!(J#?t@0(FxE5YtL?YYg#Xv5XQii0oAcpIjJg>f*apyZkl zlLuL&fzPx-DFn$%3JCwKBaz6H%XKqTe|Px`w6yJ68NSSadoOTomp?6(D|l*e9H$7> z#+TZR*((rbUH=yTmnSofVqB2}=TP&+cmDavzS2cCZ1nD`U(M4a&WLF`M0!lU z#j3kAaYQqvHw^7(>}uq@g?abRA0;xKU(;_P;j6yZST#9U7*qe-#`w*2juGiT~Txzs&US{C)Vh6R^Y0@W*QXvxmS={Jnwx^Xi|s{I6F5 zfBzmK>%ZMg8a!z_hj(c7k5?EAdM{Oy^dD&ovR4rEWjcj`=%mm2fMB$~%TV*|OG9@7 zj64nd-`Ofp9EE$Hnu}@f;fQtcYu%$y_$YB(kS;S+5L;lx(7g672Uu8(<+tU=*M{N? z;|!n1xdW872K30`Yo}p;kM=8x zmfPkf`r^4n+qq0*|2x?B&{!5-NRT=5d^ zz2la>;`@}d&qV&pQ~lB>CL%37g4>HuY-08E&_ zzC{6Q|8h^2icQJ4XgifmNaDy=$+}cx>(UvkbBNyV+SfSl0RwXQPjdKum=zW59Sb6F zd(CY9+h|2>?3!ot7~%ubt$m|6@t2dhkI48!KYtqa-9M7AmAEd^;+-E5 zATFUvjtJ`z!}wg2TBM|HS~TN}109QQ?=@_V(z7(r!3K67&oQ-G(Jpd})+7hzqsn7h z2Z9`|Q1F2@b*;NYTupvyPdUlq;rOA1J1x}c$raW^2W$Ss>1PT>X`%SY@_Ostyfz=-&E#%0)Lo1(#@d%(mRW9AR(j#G)lVO7}8`z%*r|Y&oqVH5f47F zifFjjE}B6V$05e2tLksp%==V)#u(Zzh-|>o zJJGQ^g-Loo3$OdysZeRANh7Q2xP{#-ew&pK#BugFzU&lOy4F2>Z7lASyz^^Ux@x-2 z=e+iNx0A)#tRb$AT8o3wGKgc>mT7;qK@0=8L0qgs>2FbUk^y0UYj1W1b>{S1Sy-;c zjh9*-hHPlEq==1Rqt2P&t|OOphABp^&fC!eI~Lv4kpQh{D&Q2asAH6-w4#P>7+|W# zVuI2}J!yTI*6D*=k!gYP$C7;>MK~ka2AVUI%-hjFMM>JYprUN|&=_`idVguBfcRu} zH>c5U*sLNiAo#de)!44=zDO-0lsS>dsxIo8a7Qd%-xc8WkXW4&!PftPN`TLG-amO( zbboP>bA(`NS5k&X3ec?MhaL%Rgk0`ui%7w3s6Jnz+ee>sA!g-1Tt4rDRxi~Q55R1( zbnqFAZKdmNPx9fhqrzX8MkQj&;o8a<<1HLemeHcy&xx}rDoa8KcKam1PSb%V+UF;8 ziEY!ZSt!vT@6NkqQtkKa>9a0!cvy!-qi>s+ZAtmqyTc7~BcoQfv5duIr;WcR$OrA? z%)QSovge%W+JCpF`!wSxy0P*ohezygLzH?oT2QjR(myU41>_|jIehQqYUQw9U%N|` z!>PrGVw?#s)t)+j*=^?$BX{TB+);MJ4mO+1KfaRe>?=3xwdc>?&yF5=F59@|fHw55 z$afiD``|d7;kiC*kJ{SrjW`9P5ug5adG07l{tgTexue}D8SE|)H$I*hE3(o;Wjp}CA2!~2i#<9{093eh#5_zbHaJWhP!{qU2ikP%7cK=mGI z(1sTyx}Vv#Rh|6;;+1P6yCYX}EqhPH&#I($74>{+kKQ|cYVH2ZPYM=m;i0oX^ldjM5x1Ow3!n5XOH3ZTWI$Sz*fw=BvJg7nTctQ+b`kJ4*Yceiwrsa@>@k_ zjf`L)%?D6(jz$ih5LBTAV3gs<%F+H4;)U4MHVdN0GTBBC$%oS(o@PWOIBj#`DZU3W zMFMzd3*}zPH{zRwi5Aj17Thf#d#azWK*%VnJCYO9%Lyj?}MfNqNwZ$~}ed&%DD zgwTd6E5IuXODG^SEC@tmOYzv1*xc-)pX$8yQR3n%b-f-Zevaskc-ShXFbtV-Jow5U>avqyg`$y7g!WWx0uI@j9fsmLz;YZ~G)ioaUC(+MF@lp3 zfzQ1xek#u*=tA=sUv<-@a`1k4{%n%9*un!@RgL(Bftyl(!j?B%Fu%Mp#F1Pp8~C|v zAV=CK!%KXW3F3eeJW7DXxX@j}P5?oVc*z^}a!f1|&xqf$AaYo;9;HvTH3;0U z)ZuvBHz46N-TAOwd0Yc}2yi%~+BhA%pqky(u^EkN0WLYU7RfZ;@kaz<@lGgV7T@u) z=Cpbd4)K~*4#s%*#%fJ7;zD;(PJjm&34Z~is71oDOL4*#^^HE+c<%rTeqMjv8S3C` z>o@SSXC(j54__nB;10lUG_V(?nw@)ceX3GN;XL@NzEdh}PdZb+=1^bQ93XLW-ftbd* z#N|e7rRux3`b5KI0E$VE= z+ArjGyMIn}o6YVc$E#V`2b2vM4d+ETyvfwB(Ob8im4+Tj6_LYl95OZ&!iUG*XCaZH z=Q2NUuDJ1|b(&YgJ#A`K?r*=%-PLnk>f)-{3i0_q`qQLC%t^mS>nQxwp=>YH?5wqT zb2>3=M{m2XhbuYx)+(`lkl-$2YNksWB(C59e~6v-ek5ZMnf^NBR8CdMxy z#!s1^OOZgzw5RbtC!8*AH5?co z#F}HbR_t?B1IkMv;IJh488POBYU6k8s#K&=Tvu6r5D;9D=}u;;zrr8esTb(}{-7E* z3+R)fmL^4~DN;WTV+*+!(WvMs`k)1j;qQhw@d1We3n&2AxU-u9GY(f!u%enK^v&3M9Gd?8W$sv5^tygRB2o6@U$$>J%H}*C9fJj^n8~!<)_*%=I10TgyHj z!GuvIRwG+Rj`6hk9OHj+gTXRS>&^M#MG+trY638vYZ>ojm`>W1bzF6OozPR&zAH`B zBDB#bFMv2*<@324dEJzYL(5ovw=KdZvsU`u>mRC>$04`aga&+DTOBJ*a{%yb%o5M- z-jQM8FxV+rZBdRb*gm*bSXsyTc{$=#_n3*sJnJ`c@vR3AH|`RhG@KAy`^U$aBy-Kj zWCWL#SN(1OcxEy`nh8axg+I|Pi4$9=0%~5zWjl*`nofSgWZq<2l!3mE_y-@q17zHi zxPfEDO~Bfe zJC{dtF#*JnEUW<2kjGRs@YF1&)&>N(n>oim7)e;ITI;fPKT*U&TjQF)?re7x`#ou6 z?A24&-iy3vUbMGqKP81gemhn*K99o%?^eMWBjy&Hw6MNH80<8D%|8ql-EnbqetNn2 z<5Qs=(+VI3-wdjHQbO!r9)qkBtt^|FqJZI1*}urkvQ%wGn2PK)EpN@7sVR5v;Kc+| z-v#Ec;hP&amY)^B?1i*Kb((5tU2?;nF;X6WB0PQ%~XVxv?9 zci;Lk3Xi7ag0RulRX$v&pGDe`>{cziNFbw*1gc&ZGw{o%p+(H&p0L$lo=@*(fSz$5 z{wBKBUDC&l#3eyTgHp;|_Al+pl^o|ABE~9v(*x={%RsFNc;-KxwnY+nP8mKn`eA}k z#+^zYg!=sY5NH8gAygPS&q*oz*k(VV&98sI%!PeEU{eH)ZF_aQm`06LusV|05@kHT zMquPY*2^t~D`MGRy{dU*5J3zvr4>?zLx*%Se_d-btt@#EHn9%1Z?-0C#MQGW|FM@8 z#@A4%eY18tKV2V9WCZ(#rDory&CB|Y`_3eOSfgD%?Z9o`OoZ}14eRW223`luNGMaQ zpzvv_Scmb{mHN~kRaFC#urTJS`wY>#{{5A{Cw_+#G@jSa#IxwegzmWC+Y5zqEI;}l zt2BVz&DrNl?qStv7uwmVj^?qXd|YEk=K{P{6>(?8e0uj8iT#J;!Z4RqoWaVKnEe>@ z-KFEa?XIRVd}@AFww#e4zOTIR5~gsf%+0tI5KIG2pU}z?K1V_v5M%vGIJ*My$I;Ku zEqU#CMwG!8TG{>~`R09BxpKvr9`DHe;{!oqOx`{c?x6|45+KRVuoIOv{b0dO4ps0i zM?H@Ekf)6Zoz4_Q4!-)MNxrEyhe9&DyX&{_824C1*sv_~=E`ytT^1wKI-v~bj>x((%P@|tP@WsyL zG*@)?Z@A{h{qfZLjQKmNquy?X0wW6HUdnz&J)y*D)w9bDgC3C~Z9me72VO#q^Jr=T zSOFBo6QCgA0c&kCdly}DR3(?sip^D*XdZ_+w46P~ZBYIIdVo>POn5XkJR%~GF*(w> z4bd83%ki;snMRyr80ON3k`PZ2GBO+(h8?9BdqWs7Mt=t(ru8rL{QSa>UAQmDM+8TF zd|L?jnp9mIQ^rtyqjNrIUxQe|AM;%HEWK7(GTtt{2OKr3Eaz99H6G)l*5e66n} z%)9=BV&@*}bYkox{-UqFt40iJzBjTurSkK8>KPRKp|yAOwquuSJ~PS-fxXc#@YS=r zR8xNCe&^~6-P~BGYVq}*E9$QDFWiz`ab4UqXSt-{JaUweAKB5B9=-Ffyct+%4k|W* z<{i@BaTjuveSFdiTWF^6(+wNtN< z-HZBxZG2|3e+!dkQAn$1_!YzxmBhQ>@&;_h+&zjKpf5M#(!YM7(T%QF-F3wg^oB+K zP~4W|8d_-o6z`JeZc&DS3ygJQ_+BNQPVxZqoVaKn zJa%A#%>CN0hVGPzh!x9DBs1-bNGFt4JYs#6nsRfb-sjmszGYWzW|jNP`Ob&fcCW(k ziF8uYXOQoJExF044LwE;^9yISno4<5)$R6?%|?qGa-2 z^vB;W?MtBSPh@kV=Nptcc?MX8CM5(T6`7_$?bArn7W@NeVn)`2mr9#H@A0Dj1Xz$+ z)mDV@^v2ypnBX(?kV#s{f}kVRbzxf7HkV0f;mxa)RX&%r-?QigkGz_D%KL zVD{IjU6>$wTk7&v114haS|BvDE9*RJlXsk_D#<8qBQ(~^*oT8~2YPB2aipT$)0A-l z?$C_uw4`sY_|8YId9g3t^fiJ6J@GhpQB9fYaS--f-=q%(F|FUF`b9}~KuP?y>)fq6 zy&w4seo9R|<)%z$Tv=QH?C@6R=Y5F>wKMg3re&JpzdZta_x7*s!K2|^t3rfO_(wW< z-O+rKGC#;%e6`gxgH3F3om)b|iS}bV3kQYe7*&ShC!b+R9b$sbAJpL`DcYlG!$&Qh zDqZwG=}ykj=vjsedPky0fvp{#hc|1PN=<41c9Ym_A#no zU*$_iaYmg;)s5|Yrb|eBvdHXZktJjp!)<5rhLzMHV+V<-6xVw{y2Gbjh~M3dW*vjp zNOo3rZN^zb$jItPYgKAr`wX9Q9inIm(3e>D0 z=K#!5x8j4{3BJU)ey?Fxgfc%qy9lBjzSD>`;3piQRkHI@HEcl82miq;&tCw3 zuf+lnlF`1@B*er#V$P#;;#s$^8S)`8o(~~)ZMhjb5RckY${gp)OpbUK>y2kIFOPP} zroZJFR18ALP{))O@z7Q~M8diB7b=QepTygy zyr(JTv)DHO6K@jMK~II=bsu%M>VGIV=Lt|Y5?Vjm$IH;IULW<1J_+s?+fe{S`E1X5 zXMpUI@O% z+IfWa+nW%N_}XKKuzEgAv_HOGn=fDLXvuUDW!qB*@`^=-u8*}i3jdTY(zD)$hS4zk zkVc0<{X2aT9rPQBh%UyU9(!-4Bi8=A8iNh9-!5xO{DY;Td~e-#*XW`$s^8%?bB2p^ z2Q|)|pRsxzetti@C`WL4?Ijt`RSEuGM9cO5_;@QLkD;y2-vNcJp|c2t^J5f%EW3&` zyy-Ee+l01Of&8`|Ubce$JQK_dvrv+kH}CCz=x-ecGia^3_4Z#gcr?DsU47roqDp|g3;9WH@1xZaFN;pJA%(Ubh z5oGZUzP8ZV`>T)J6xsS}rB!A9TVKKw-Rw@-+_ABXhp6shm?TR39O7^>gUbI7{ygH9 z-oM86CjtnvFaCxw4g(ut$yze5f7zLyKInw(T3(c$(bAM)nN5-&2obI7M{RD^_zd_i z4}=Qk+zxebZ7fVb2`d=SQTX3tm6Gves^<@#x z%6x+A4HVt`)V2IuW5TUsnFU^3Z}nHbrjk>fg!*mdHxykHL!N1GFd$AZ3#81Jm7GcT z&e1VatMke~D!0i^G-JAYAg}an8P=ptYHdJ@ex5@}t~lz|R?4bJb2UE@J& zvwGB?#G)09lp_u3r4GVL(=_mc4vq@Kt8Wgm4920ES=G3&Tzeq)Sh?kp|7&1-Z0>rl zxO5SZ+{fM3P@;!p4(m7O*_`>EA>kX9dve@9uR-WhNoB{>rO%U}uzn6iA9)!2>wWX- zBpCG98bua*CbA0>3!M&Qb)FQEdgQ3{!WaBh?$ak*tKADC&r_YaVqNxVwnV*@eN-T#>?yc`=8v#Ggrco5USL zDTpN!F_w1%NjX;_i8$nMG>j_r?{B2Y9cWbfPo(baxqpjTlaYP8{5RIeF7)>|Qs|6~ zOiTJNXHdxh{q69=^QV8qbMW!Mzk%el7lW#cH?m!=-zf~e*gQt|iQ<2KB?aHZ7KLR= zO_}bCr95P0?El2?{(Zy^`Rsi+q*J~G)QaehossDKa(eg$kQ2{o5BG;w;Y)9qyCD67 zF5ndJH4n zjN!sz?%m)(yVEJy*yZ&q>oTcZU@u7K_YJ$boPk%}&Mt_ml3%=#?deu|porq`LqwNF zVkO5lvTHn-=1jA3yuH{A9V@-rR$A*|QwPvXjpDq{c<~dWD4$0IW;P+X<6eHhgEfT3 z(R#>&-bZX8!EcPqvZOygZrRP&!EMmjst2KH*0YkH)&^4YevM_i*}8&E5bOTx*uj!| zW_)rWVZr($B%ZNl%|!d|5J!`Enk^$aye>fy#Hi?X4%0L^1C$W~%Dsmf8j?3rcJf;p z;?bkBPqPg7m-YKq{f-6{W*EKyDdZQuo&I1b^~W#>Q6G7I0WspX(q>Nyk9HF>^lFpk zf_wXSX_f4ZA!Eh&Lhlq2>2jHq%RJ)#4|ne!)nwNHi!v4*3s_JQX;u&w5D<`r;)sfL z1rbon2m(?$vDcy znANB#I)V*IFdADgz(^tWCzn_u^%)mF>;M8dcOWa;uJ}tFbuFWes^)qFMkDxCwfJJ+ zp;x#!sevjWqgWTH6t3@Ae>Mjz*68I8-y0p8L@21aFyc|7mgt;RP*&h;J$%NLJv~f_ zNKm*_S|AfZsFt>?pf=RW&MsERMda)N0;+idFVdd!%R!Vq<@+_jhq=**fhJ2fe;9)X zQ|^PCe(?93fU>Q`_E2%}2bUv1lGO(cPOrDg0hnv!ttG@@GNZbMs47*xQXSKZc3aIJ zP2dav>bPj-c{rpK0{3l5j}DaI&l)GVsM(1%39ah`_W@WMh$qe0Rh*yCsGSEYL^(_i z>kS;g3t~yQj=*Zr1HoZ!zL(>OZ;&&BzmEwiVo$pE)c#Hf-S^l{hv2aA$3Y`J9LE6v z#kfBdZ}GIg2~u$C{#GqFogYK;0U|LS*1_gn6fY24A`dUED?Td`GdyPJL9+GUE=2FA zsdhJMZ&XNJ{_TWqsnuD%?$Wh$d8Qas!?K4mzJg25NKO##qJ7rQLRovaiirJ3B(hc7b zW7lo{?MyXj$vt@5JTVS-(P2_(bFgj(u7vc>=1Xs#c$ezY8cQI|;z{?@Bs-99P9DqV zRV7B0U9s|)e+e_1*qcqHFMAbs`2AKUaKcsMi|unys=)%bCBB0RoUhpQY zy11%E-KvnNT-{ZT8N3YBS3p}WencDF{!Jf(YdR3o`UA2~ANF{kdbPS)D*Isi{ReLl zZhx$K)S&w|#k)Asg2bBBDMh3;^$G24%vAqek#WXhI}=NIzg@S@niRHvS}a2m zcYd}D8RR`^#u)fCT7(h-9--)hERWP{bnp9gRI2 zjrcPIoPpYpQAY48Pg$^5;bq!NRSvBjCQN*4Wa{~mIJgQucl1J^Ax7D@msu#O{%mjg zmRnk27uE8$3ORbXTzPqkk8ycgGytYD!U|jxzRNq=UNS+u!Q@c{y3%*wd0VUJACXk; z%w7FnR6Q&O@=^`f7;87A%i!Y5qv-vp&&V(rkGq?wk`brLk@5<@8J5n=%FS3kGgoml zcmJef{B4Q3AOmsjPR-kE7nZA$jI2sRW@Ip-V15pXB|I-y6Yr7&O$k8DP?*J`8JBF0 zjCm~w!qZY!XBpE_Iw*cW`~D+mqoU@W??$m;fhVYovLMtlCBc-8yzyX z{t7$`Nw((qSbU&eBDK5{nbkasP#~1+gvB*C16>fdm&y}p^vSX6_Dt7HlQyaSi^3pHW#%o&wcaL4CR^|kI z56va64_=2VsPRVjR<*=}W8bXcz1VhGyRxV`!fs`=L~2zl!|sn5OWtUikKG7j!5sdGr_%}l7rt6m{E4iaUXs)%1D>L zvn&F4AC2^h8(xGdkOQKh6UGz+sOp~qIMe~-*&va~3g7`Oy>LK#A8nV7sQD2IchVc$ zqXwkd=JaGTj7Zp?B|N#x89F2F8v0(dFQmog$HWhQi165SCU#xXYytNG^o6rX&0Fbq zSrWaQqP#F$W2Bbmvot+g>Y<;Fdv0;Z?rh2sW$_Km`XuG(Y`q~nfp?i{i11ynh2PQr z;H#VYYXk?Bt5)zWNmT+-WJLVNGJm%V{tFe4oWaj%pzbu_V zQBogXYRxZ|erE>__~j4e19%YBBXPXwxqGWTLH%30?xvR-N|0HwehoCtt0|@ekY}n! zWX=zICN)^t;bt}#@b*9j47LJ+;VNyAXkH**K-eG8W3n%Fd%H`C+H6x`Cyuocw!12b z=#s=No#ZuY)kVYK)351>*a9&PDh7yWu_f*pbOm6T*w0p1^H`XBw+m={$+&JGz*KG^PL0 z_YQr3%V2HC9@vy^nXhgeH7p(Fp0j&#|CV5F=ZS7)TBO~b6YahRp6@VSO3o!WTh0Ji z;q9W${Pfwb?Jt<91>QB`&q&!JPqbEJw7@oVRU_RL`7hNc`tR!%{9u+fw8phOxb zN0qhCs&nkJ@JE!~U#!l-OVuo0YG;0`?F{+)bhrOo;a8G(u#vJ zpIGkw{!NpE`jg_HcR>b&Rjgcm{gnuNwAbMzP&L__ z=`k(74<6(bg_mpad*C;3Cf7~6hJ)q$=+QorLEh%fAW=CwIYvEuU3Gd9K_nqBM!c*% zn$6uqt97f^?Bska98A+2y)Ks6GA$!NE<^5jsLBt^nAjQsG`8>0dsV{|!(T4-PBS(& zW=Tc~I}^v#s0!L6>ubq@Gp>uSjZP)Tp(;yACpZ>*AC&JlLB?hWle6K}=9O@vn zKkM!W+K$Zk$^vVJxSpL#{$iAR*dggGn#^jHy)5y*$^GxXk zsBVdhwu0&wM?6-J2si?h&%=YY`zC9A`E)qkbL4>&)6*nIjp?Rw3&x(X!?(pLO7F0~ z$<0lV65loh9e?tS*Wb6^sGQzCsW%Gei)wbOUNlE;zh2ogv!B#!IhRgQ*bdS8?ztAR8oA-M8K*Sq_lK`&m2;=t&R zm7esQZw>J-DnRw`yzTnX^lqhdHg*P(0L&f|fYDu_6+??*>G}Qkm6S@x538sBw?t(p zjLNKktPol>;T_$7km1RooYG%hnTI22ypnOwf#~ag_4T>i%_48S2J#Cu7F4oeA@o|f z5t@YPb)j>&KMjtd9C-Ko31uY8`JIf4Gf!T0n*Yk@(B|N6bSA|&LFkmEh%?r^O8QAJ=DO?sOuJFNtgA zN845;@Sz5Z%gkxPLS?3>*C&6;=k@FB9kgs2!R%c?^^}XUz?in@th)+0QIC*Kd8E>( zrjGeo1Ju)@_gTz*1zfeNg8M|F!*XZ1e4t)ka~1P9VV#oE=DwC#AH1>qR&&+or>~N> z^lX%{Z!*-or;1t`&OeIv@ik4h`HEhlo2u5H{vi;0z~NhAFyvH{NFz4B!ZrgNg%aIA zGq^>)wa>V;{$xBmvOc5m1E9*l?hNRozdyK5x=eZlD*sC2b+pUX2o~pxVS9l$sENr6 ziOP0nFKCd=za!VOwI=KoS=H+T9@sD}I{@bTYvYNY3f2z9C+}9L)!dJ_@t1QtwI>51 zB;E0iYBA{JZF}OlYc}JmzY|kSzNy%$X}g-M()$7ry}xJ7gP(<^?#Ao=mB;rgE6^gu z#n)>g%sDg?L*Hsv8*}CkX$!@o0TJyoun*IdrsFOCx{E7w4?RacRyRo{^B%+XDOf#b z_`qkF!PhM zSE>y4TR6vEJh1+?x5Ubj2fR7uE6^*QDi2b!xL*_(F=DN^5KPcTqBpz$-Ot^BH+wge z4?JwZX%OF=A+(7@UAU5&Ke~|Jw$y{^nwb48EZ8}ICd;T+3eafwa zUG&0M&t{txneJxJ@K}U;&&R zOIN;6=-2N+sMsWZ%T6;-)yPT+VYcfMcZ?tL)E)pg-UKB>3$9T%ND!H46 z@GfK;1jPZPmD=;pMuQ-)>bp6sqhQ1qlq~nkXfNwTEtPeI>;+XkP$H>v^c{+*TaRSl zKO{?;{*Yx9UROU-7w#>7BlHJd6{vitki{%Wjq0CSGbt-9`8u+Iss`r)f?`pbEdu?_z&7oOE?_5}a_W zaj^`;y$v}FOX)Q?j04EcPNOWZ$KLa%m-vn8u4lwZRZAZI7d z_x6mk%Da)Z;23jt2jZiV^Pd{_12^1RwQeu9hA!hlA6C!_hCY*9>BJ-U8L$EP_)t(t z`kt2aPBj0O(8(O@_?mVwu2z~?%|plhB(DfM-Y>f7Aex5fPY=`{gvx_s|FyqeYQI}k z2|Cm1!);Fc{xlcfms{4cXdtdhAHP?-^H?IXTJ#85ji$rb8Y$7^2W3wmU0)GoQ`1FEN z)?mBY0Zzp*-8te!S%1Q3%sL<~VEG~&hOx2?f44v+1)1oxv3jck_ZvkgV_HMADv`C# z>p(si*ZqMRS8%nq&a|Yowc?~VGX?y%WJapt@q;Rs)0nws%UibsLvC@#Nl|K)R%6L` z-|bHkA6Av0Uh-%xbXF8E>mOo{&OivbXb`7%qI!uDe*gmLMJa^9d;)6e&2WU*=T^(k zU1lNsZml2f#{~eU4NpiTfOs7R9&$mZ_mgLoC zitorZmbT%kWHK6HcF)#9QyWr@-@wfGp~caP_*BWp`jZX**9HuHfTF;NDF8`UohR$}CjcQw0?=#>A|IWUs`DJ#X9(OQ z-ZhuJwGLE5F3Jg<#9X}%q}OvMSH;~M*f*>K0Kn=(Ao)j|!3pxNJzi|{=Hc#7K%M@k zcHP#wBO|rru>nTwMQr7ji6cpX26vr;hT@|Cm&gUUWw3Bas6I(M@!yy z`-}(x(JX%A%%*z51O<|_VbtGv{L%B#OBcUh3lx(#oxm7S?GJ<0jJ8{2^(XI)D`J@7 ztYS={naV&;Osl!)_~X%-cD;i61SHO$c3r~zUud&%`PnswqnzJ|Q)O@ENX3>xtXPt7 z9!m-AmR^dBKuBVv=C9gU06Fl+TWZ|PqhsNvb9o%1$CO1;v=$3*DbM5f+^G5Q_c46# z5>9T90JKb0rUaav!H9OyH~f+M8<#^Ukdb*>1P7P(n^8#wwpxH?=XJ$tH&#}&pY({r z7rn!X+c+mvdJ(bTui3M&&f3%)f}+jlH$d^TOHRS#?yl>khIh-;{IZ_jyt>t;(f1mn z?{F!+FFXkd#bo3itehaC`lV7KE7KSLK(%evGr zFLEY0O6{PuH~Ixq3)l|K)E{Rj+cqaBO5&Ux*DVLkZKCdxiwmsBopJt5PIjh$D}!MU zubRSz^#n$cbsuu`IBfBR=_E_Nr1kKxAOPq85GdZI{$XV)xCQXe!Tk;}$KtS=fxv=@FI9 zT>CWv3flX3L9|#{)W|X{!3;P);*%=8D6uc9%ApqvYO%8-ilicnkv_; zzhbop-UtuUdl8Aqcge#pbewZLj4BJOt?mzhg~AI{Tjp-$phLh{nc~d~Ac^$@(f1z| z*0PK2U`^%_M^yV zev2nFQJkfXfxl_{zVek@qv#g?Zek_e#lP_k)+4{KHIVD#)-#XmZDvX}o$<|&B&Y$B zY95ST-eYl|t(euhZO}~IuUNhvB;fdaH0_alfWSO3|-)p;?eMs6C*=WZR6k{{jX_KPO zAeYi|Os`>iST16jf(o#rxfo+~76WikUmFHSm)&@Km0_|r9=G^|>N`r7VRk^$@IrXX zN~vy4Frf&~mg8Wmo7gi&i9qAw0-4C<94X84IH=0l7iYB%0@jxyX6y~UAXq!UAp_W z?vXEIl3Cb@2x{N0iSrU1jVZTmxeVl7k%eG(xMl?XjQq6;`RrU_7v5cJA~?)z)F-B9 zjjeb*BMQ6RTTN+al~y-C=Ej=_Zz2J_wxjZ#y+G-C4FAaD9R&n;g`2+N`J*TNo}rrZh=Y(eCK2tqo1Lh^sWqaeQVCug6dFdffTEkm4O( z*@Y_bOrvSio7HxLxR)LLV9h}#_Sn;e$%9&dA@9ipZy1N$BZsE(Jdg-kmTJqg`X{== zi^fZ=Ps*LuD?soeLk(-89r_#f2g1OGX-Ozps$=#&&GoW)>^aB-4WX0kOvT!IgEGCq zJWsAl3gGDnwl*mDNRCjq472e8It5VKHaM$iFQ>0Mz#p!O%_Vj?ESD6aBga2HX77vB zHhM^Rud_APn+)BB@6nkMuIdvtu+$2)Su2RBIo)~TlhM|PhMp-NPHy^-BQw#@ zetu<9>Ta5-79!Cm4PxD#l5LSyt(iUr-L%^#c)r<`IaX>mHoc&5a&lfRD(To=)(D1h5gGQDxn4Ly9Y9YFJ|EOaRhqt>``9}lCo*AbJa!mr%U>ZQ11XjO#H{RH# zF#7qRn-0?1u>FV@;$*}MR=bw>SL>= z+eF>5>#E(zuC=@U`d+3V{F5$BB~ol;`kh(0$W8p_w!-)-_<*PZ`vl#vKi8k`-fQc% z6A<{zW;-W+{z@%DZI0)ka@QA6(RLHR#%ZiIhHE2S9K%3Ue@pYIx#N^!K02K4n8sjU zSYC0@4q5T|?Lqy?Bwk+Of=t4Dv@Sc9?7?{M=eUrZNk52af)=?oWJv(rI5g%k+}^X( z+J#rpYE)XQe{XM=i;G8xj7IFIPpx-Lbt40>lGj%R$!T*Wxu@^^k1u(I3CA8MR&yI~ zY$;0$-g!S1lJh!Mj2v8l3VWfapt8P37<1R%>#_{u#CYeQh;Q+oYv9|R6~Gw8i)sm! zmy&YOzQAI-K{>oZQv;I&l6L_ztXrd7uZb>umgdbH5Akfx3*RqK_>;yf0z*bhO|%BB;1;&VStr;m*67Bn$)w6tWO zCnyxZ{(_N!H2{?O{Da$rKEdX&>AT(Gr-71X%0n`by4|m3$+wl)qif4sy^sO*hcBF7 zn8pq8@DF~o6N9LoJ+0t=G$(9&+zr3dTfLHJShb(Pg}##$e%RR$7GViEkg?dS3~v?9Xjl+7Jl@{Ig1iqh{}TH+e7`%#1YFts!d5Vj#U0$*B$;z z9GA-azO6H4=&0)->m{Ni5i17@ugPy8{VQ0-CoHJ5LO!#N8ajB+e10Wm+*}`)ZZaej zgq!8_@T_~Q3*16Lt3RuEEKs$7eNXdBVf;Q^O!RA^&dp!~8M!E>8W%{IW)ydyRoBR) zEPg-UI{(e}ZVI(IqxeI%(!Dl)10sAi^c0~hc(o{++5k%tcXn_F& zHPk=b&n|c9Ir2OXu`$VnB=G;5J({uaui|_yM9SV@E!l*C<_(qpTOPjK9>@b|wv?&~ zw;sw6?)IAO%3m_2bv}{y$G;+x*WlybOkJX*1IVMvkNJ3`_Q%&G-X)9BId@TQkuVE5 zZnh#0xUcohVHF>f@6)uZ`DBVbhlT>-NY{v54w6m|7o(@ByGLxc*HZwcJJ-E4>S^Is_~{5;0awR2+Z-kt(-obB3Xwzp*{ zg(t=*w8dL3(3%(QSj*_M@`y4HlHuq#`Z$a1I0c)`Gqffh9xAmbeKA+4)dQ>jCA0N> zs>OHK>W{Vy#63Oc+VgdK)cObMkD6riyu1L+O8oIRksyz*CT~_cwK>1-kBQ&$J8vKC zrNzU>Lko7a4KnO_pPrd2<^-Pge`P*Dp$unI2~&ftxjb;na3ZWe3^%HQiq@Jbv6V##6gXYh4&eisPS*=1qdnwB`nS-n3Uz-oDSw-)o*&pU=& zKiZm)b@;1f4%`eVH$l-{Tk0{>zN=MZZ@HO$Jh`7H=y6m9src(0B0kD{qAWRC``&pVQ82s8D|yqJ*yd|0&Ghw#w^ zrDQs%`@L$A_5+W57k|SU<+U!V>&@R~aJ{-E0q~jA{X5{U5+fi*!Tc)#Wxkeu;5E~# zfpDwXf@}bnx;^h8hO-erJOP&FDZn`x82P%w^DjpB(MOx_`ZO_E^FqYk?oHF%dpKL9 zCVOl`hbx$`5INDG*Tg@v0LA%1nfT#Cs#&ihY35EhE$v59K4+?!w*SvNpO;b`_muWB zmaVe=%w~S;K>k8|{)zh>5k`j%p2&W0V5vk&5$yF{^u{k=U0kyLzEbXchZCwUtZt6~g6t`q>GS@k5Qv`^yRM(L(IVO0Qhu~fX@inL zr^#bg`87KW^Q}-V-f>GG3Q7zn#IlX1`st(NT=s% zW*^^rWT=)*PF#OL&G4Jn-@LwaS`SP4)eJy)_GD*@yvK_%6+miVQP>Cqm60gtr1PPzNlXrykoW*PRwG-G!2gJ*wB zRsShkjp3bRNt1di)_EBa>ykw;o{Yb);dFg>4+j^qkZ|TGKxek2rYrJo0pIbn3u`Yg znXmpP=&J0X);@i-a&_28-JX7MK>l%2zQJmWrN|GJ%@2yN{!+&N^Kv)6NiTgWAo2uU zOM9wVJmyc0JWHk$rTT)`an(_+-_1h0>7DGN|G1jzs!-O_mU)<3`xgU*yN}9UL~e%~ ztzT1eByATL_HR`FpEStz%})4x#%Tn6itDEM5M;$s)%omxAEL|GNuj+wT7}0I1kuZb zq6SR<0q7np`kh<#Lpg4G^Ys~EO##Cc+@mR~fXXv>AfymyG?B)V30FMd+q+T%&aBcD zf3Mvrysf=f=EK#5=C z{Ek#`#869O5cZu^;-m#M)sEwjK;jT5bk5UqavfJT>9>>>>%@Be4v_D6HQ_knXnq@0 zy*ezah=&&}WF44r{(*(PX`5{>reYMIhj*VroEpfY8j%n=fb&A{aBYg+Q=Vd1MpPpp zd83xHqE%cax+)HKO<&$y9w74PTF;}3e5-&K#koZ#i=)`cAX%=CD7}wcc z8&A3ND%xB+?zVWdEKK>E52+$QIuRtCZSX^?ah`3D8VB0wYCpPsYJ#$?>iFde;@T?eE5Fbwi2-X`_=0`UVZ6%c<}bpLOX zj~uUoYH5~cras&+EPc88d}ybQ-t18!u9#ipCw$Z@r~{+l#S?<9^)rE#eZo7!vo#w? z=z)~okB|_ZqpDu^T!KTbv7eom2>Cp&`V>S@?YkABCX&g0b-$K;3-0_q;-ai}KZo*F@TYVte0 z!^-rM0N1;8y^n`I%9~7=QWs}QTldF$tH{t{AgU{{<>7&~+x!I!uRtW${gFPvEhK7- zoeR|#F=!r?rH9D`FtV>tP-9sjpKI$GTjP~Ux>}#0ffIw+uC;1HlTLmw1jG)9OZld` zNly_`4vF+?4B)EdKGlCNB5=g*)7Esybo58|;{_@B^ml0pJI9j955YRPSihDKsqFBP zGue!zQ&|e!QsMMhYsSxz=%ZN25AJCTzneXCD|qas3hlMKa%E!XjJYwfhe1jjly>+4_hDSHn;1YX!4wT+*wyhMO_&^Zo=^$LU`y#6K5Gu8VXKv%XcN zW>a2CEtPCkto#A-8<*CWU#!bNrz)=N%hgg?+0ePGFpKiGb8r54FI>@_zzTKUIe|U# zzx}*`As@a^+lor4;OL)bhF9~xCKJ1}sR2^`wM7e7f#BT35yVlMQl6%QRqRtRD9Y?HIL>b3E#m++iu?619vtPC3Z+>hgbeHE>}K z&t&swtV+yTW=CXv>A`=2QC3bLezRgUaVy()YcF`?h8?1FazMnqDWBbUdEl?->mDk3 zd7z^Oh@j$nB+146;#$B#u(fMW${S6b%%03xQlg+GK6ispcmg=0=RrX+zW!5xvIe92 z;4!1pi8dk6{W=3?JGUL^96Wdx2-&w_^tOrq>0GFVp)G@04S zov(G;_2~r2^V~3o*aKld$Qo_kB{E&CL>rRR&7HfG1SJO}d$wA(%4el{KkLbx%e1W$ zCV_nG{*7o&ka`PghHBZ_=tt)MB2G#2F1GVig`vx|yMca5j|mP)WYA2rMka`&%(SS% z8g}uNSqhqJfH8{? z+nK)uOp5GU9CRqDiQvZ76OS^;6G)M)w)3o>b*H+kVkx}(uOp6k)YgR z2BcQ@aLj*m7VK~Nm$SejA_A@Yfv$W8nRWB%`Yk&H55`z4_+{vc)ioK2fZA^6zT4~# z?AMhZxIE@HD~?66mNMZO7B*t1w3_sYhh0igj?(C(w>c`A=(BVjr8phV14X7TKQBfA z%CEF!3aA6+_-xn3;S$@yN0cMG3ZOh@0+eO^(l9iUNb&uS#$@4|2lsXPUi`@b2LJ{9 z721LFnUGcaTC5)dh#NyHpGG9FwR|cwaHayOnQZf&^BBtXTHA43>BQVuDR3ST5hVp( zcf&xSB=)y~z-;xu8VHQC-la^~zA81$37jhFr;Os3ouQP6?dcs50~hNBMfD1t`*E6G z+2fon*(|a=<$PG)`5y1MTO@qy){whjV9n_08AE65wHD!K5KA37Q(+uZGeFsEq!+9W zV&(yF0sTD6>DUy{qj07h7` z^fIywLtNjP({UyuaUSkOh_u=w6%2I#fM1#^Km&z!YwhYIIHqY^nx6h*wcaTVo`21^ z!~di1!w?>a(vNZ&l|fvqDSt}pOhF_XN&c?9wDJ&OjOp-Sef;7(K!|(~uZ4nV%~S{4 z@(8MG!}cXWWhpDw4kFH>cyc4P{N;rC?3ibN2;ct~{!UbCJn9|MeE6Z=SaP8-= z{UpnP0=DA4ekJp@h<>F_xXNOJgK(&~if3{UqSZV{QqFPue96eIJ3ha)P^v0C1Wu?1 zeKi$N@{m%Eh8BlHgY96X+!_T^P>{?R>_^n9q=~l**iCd|XdzwGc+YoNYevPSyaC>p z=r2;!u3|T@@=E>1WDYmwDUk+Jlx9?Z-P8jYkgwb4*Q%S@d@#SA*DG4t5}%pu66Geq z3317MZMARmNo{B%Bo!~-;AvWx07wfiTX!R$O1HZ`TEs8n9@_kLE$`#W&fGF2)&)6f zCM}+V%v)xMVXEtexy9WCVRi3wKU$uEhELwj_2J`7Zvr_zRTpN%Yjp@tW!C)!!)il4zq>KK7Imo1ejqt#e3NN3asaSBG-Neyk$T)rwZj5lx6Py&d+(GZ@aUG)yz3-NO!$|{TBS` zO`tne84q`O(kvTG&03FF8VjOsgdQb`+N8=+yW6m@EWgUr1eO2jUsfnl6jT^#OClQ7Pr87$TF06NjbN!L#(I(xN?P7 zKH>x8Jjh%F1sLo^YY#s<(Jis4yNDYWK$%P^SU z8o(as|Cx52>HYT03mvD`%WPWkL+3n&jv=oy3o2zR-yb1RP$!(s`^SC!oBliJ zxHdd=HaRytHE5Ae+vK%!U!xlN+(s}YBT|?K^3%J~A&Za>ns9(#aGeg?2A+5`J``(D)% zNpe%F@xKy!M<-wkDo=!W@A6zL_^4DXJN5XHtN$bTiTKsPrc!T1UTtG2Y3J{S`bWMA zkYbk<1Lby~8v`4CDt>ypwPrXkrgi-C7(T7Hm5rNCuACF>z#y|O%fNY9h~#tYzk=)# z;i1gVB}qA6ya(vB75yE0sDQUG6g@*(6%$_wjyfXw9|b>?jF`y8+CO(dRFK}Y13&o+ zGn(u7y9c5}m$mZPqq?pU+7F{Y;a#fOw%gPFE7Z?A+(!rWC71Oy=_Dxingx-t60I){ zc5EcDuP+%7T>edR9phow0Zm>=u9JGhlc?M|78_b`{$A@JT}wMKL$9+Of-*Rh48ouj9Y zK%+P35M(*%^;jRhR0qnNibPh=^O#n4?4t3k*GV;zWmkjF<)CX34;_>-M+q;yceyej zeXrW7Ex(b$zEQK)0^{L#-bhg0RQ0PiD-^j@m$ui?;}Cs|+xCSze&>w^HIMwx_~!hH z`;Z$`9v1CfLla{=Km8-3@2+i-Y|Vrld9Bed_0O(;SB?Wo>Q>fW5OV@zU&|nAeO$;f zUE&?0u*KQwJ73kW)=VIGr3UpV*@qs6zw@_iucVJkr1hh+=zXt0x8-AmGd6Qu=z?PQ zZ$Be=w)r0;U>#2FC`ZKvy#%W(*O zrkd41N@hRc%>=n_&UQDd`HF7SYIMx1@v2o-cr3I`r?wq!zeg6!Tz+j%cA3B0DPy>i z1ww+-{j}@;D8OgFC#VXuWs|X#4&fbBtK!@?LVB5heB{n-Ez-ygun>RPr68z0G*Ze; z|r+xGPl%oM*%h8sj^0W%W(AJ+bQ5T7`46 zPO4nN7Y|l#iVd{xP*kFv(_kso-^}NwZ~uE!YzUoE$@W8cg?G?82c;{8FZJN$>2Dhu zkrI^wl}C-gW*1|1BcHv2s^iHB45Q}V) zdg?7_|7x}uE}dC@=Hx6~?VhL~x{GtrgEzjxa^naOpw-IS&?x8CX9$7db>vUr7T|wb z$9`@f4gi*ZLu>c{tvv3R&-~vU=U_Bn2Q=}9!tdYqn&4k@Uh;Nj1N_Qc{{pcOK7Q=k z*Qx?R-{<~%Q@^GF|FX3jR?@VmTWa039mGw#wzz8A> z#QEbo=T7wJEUn=`u>dx}Om(%$DfAQjrP&N7{FQ_**ng$7E$Oq-cN*5iq`5HZ&AZ;) z|L-K5Z?r?b&}W20gHFVG3av=-)f?PgmCqvu^|2i8mjx03A};wW1%?c(ENVP!T|Hm$ zOcjGxuhP?Fc-B}gydM4P&HMJ<_0uSe&P<};6#=1KREuNNpllw=s^HoB+y{=8d#D}a zRi#wX?ge7On_GiSb#M3*|1B9jem8UEpV8nBltI~25|Dsy%syO4z|k)oV#$r|8W-2_ z(_g*d=SvT?KW zg$dp*)2-Ml1y~R;#XJAn4uN-z0aYU4&=sj-5i*d%!>RE7(r9L|Vy5~0^8ZOedAuhp zF68%6VYheo-`_kQFYm8qq=67n`P@}z)ipS7C$J|v9nZmmZDy0N`rlvp%E4O$GvBOt zR|+rR?lleK=n^x`7-2SPpjKp9hpSCrC|MvxtF0klyd=8J$a2w%uuy4w85n*}jkAbvfKBi0fv)%!TycGO`|FwP+w(cnN<)|| zj-dT`zB@Bf(79H1#6#OyN@i02PV4hDS%r>32OiY2S34lwk_ZxC8B6k zQ-xd;l+3JTyFq2IKj>G(K*;un^<-ByXY{Z)qHpa-pg<_@YpK4u%X(z7ZB6{lVo~(f zWWZC#1q?(vlJ50h1+HwD0e3)f>bB1%CH-3_v7e8$_gLwt5l>1YiBTy(qQDDF4}${H z3{@2!R<5z7*0b||^)hY11OtKLjTcArDuw!b8eUhT2P>uw;WRJK!pe!C(mj}qro=WW zP2*29#>I!Tz=ltvT|D{A3&GAq3KKttbPCp4LAZ(=K&N|xUg;OF0XeCG znC@&({YkKIEq$?J0i=6VJjI`_EhS?NRD)KY;uC8*OSd@6K#&9>FY>*E`GYQ%bPvt7 zFvmXObnLQ7RkCrv&xV6{)Kd` zkVPDyz%>HVtbbZfU)8NPmAW785jq?YqLUJ#`pz#;m!x1jR^?)Jhq1D=RZbtIS}(&u z*w#&Y1?E=3IByots#ErFy)qY9gFX$TtC+lA4S-lh>8WMb}vni*5V5P(VkxgdZiBJu6@&bFx`Cg zshx-%5MJJV`}H++*Pti>hq3%?mjZ@evpmt<9C4_a4l*@+k;ooId`{Z#alb-e>#>2H zG=0^%6CRW3&5*tLX9qhH^qJR8bv58M{N-|VtFlOsDwVG;YqGvo!L7P06r_MN!+o=^ z#`-}61&by>A@Lo~u7>20{uS2HhqEI3|9Mgb_Q&aTc_vuWEzD*9*ZW{PuY%u1Rb zTPk6p6*}e(&i<7H1YLMIKOvYf!;DWvBPrh@UEfROwaJx}y?ZgF!oK~oR{HL$)2)!tFD|ASS!DxTUMgt*5=8UiUh&lf-c2H z0zki`xkUmbs zw{jl#!S7lGP0IhH8~?5WRMb3NkiLZy_!40->qB;B&1I5r(fby!y3-@O8vErRTmYd_ zww-H%#s_r3Q3R4U_&}NU`;Y&5`P+CQ|I!{1AJ`0*ubju4c4+@h!DnOUQsW_q+o`o{ zgUb%eZ0Lju==1_Xr$W?=2aUoGPOCcY)xI7-9ROhQJO_$8XhdZjV=RouAJNt|AQhne zH6l=;(Dr>udWUnRh2gJ5iJ)gHb(*&=2`~7-%6#ixyg#l6_G5eR7&t zh$rTYju3joiYEh7-54nCk@fR`eyu#0ZUTKU$b|;Hx85&j3l|p_6ej9cX59e!L_Rmsox{>vvlz!HuHMFoc4CGjj3Q*N3=}`eJ@#`l z7{0hEK3g7UF7YfUqmwT8uexmWKb$+@=o08#0#(P)cU?@C67J&IB$f@jXwDUND1&M? zq#~%27MtQIdTVJimW!px#(p(B1-XiT9L zqvHb>cEwaDeeG8_Tkr{ww(PrUc6~cW!vf#A54lnnxg5hWPe+^zqDZ~(WVr7N9ARd; zBYzfA7aye`N1~c0Q&hsbE*1gNjef67?3Bmvb|A?K`gw3$8>Xp$W|k~g+^TP7G2p(J z$wf1VUZ_FA^U!IF7h0S)Ude^M)#P?ddmCNgaUbmj-c%oE2fv_`*X(P&2^dq42NNuM zQL-`J!6`jIU+*9}p1`@2?HTZpRd5Krgwl_aVe-cF{_6#~Fd6*f!}_h*#p>g(js8I! ziE-fVLUxa+d1uszrj)zf4sS8{uBMho4i+K(dfg^sKRCXm_Lzjs^_W{|8>$f^+*7W~ zVBYq4lNb~DQu#=Q>lP8jnD!`^I;WVoSXymbuMpOQVTI0N|o zLy?0u_upZz^xkU^=slliE!pAA-!-k}kL^IlBJzZTbkypKA~%Cm19<)*m6>gdFYIk# z()MlK+r=%VCFZa(Mhx$VR_hsEe~IIB(%DKg-M)!koh??Tqeo5W9jj82=T{9^Q_gq= z-~tJ<5rkUHMqJ8SkBYPKGo2pAb#fyWgWJ{8PN@YQRul2pNo!D;zg9xuIlL%f5JsQL zEtTBoaJOysu$H+rffH32vDRZF-SObawdKj_$!Qxu)ifgimu0lit{JwNLF>39Zh2L* z=%XsiqCr*UB|ctUh}NHxdH>=73p;@p<&1zNYTUH+;^OyDdiMUxyL;()64kC}O#;<> zJl*=mN;QG)Lp@&_H76QrQ>SIJ=B;vY`=6EFts|nqEU;)H_yNW=+$yys5Ibl0)q}m@ zE4k0Yset)(i;9Ze6z%#lN(zg|9b2`S@aOXqTwFB!+1Z~&a*)fH(CU{a6vkcYIo`iF zP;0EhS%t`(U(=2|Yfkg{)s(2Gu3dtc*(Cl}%=m4_#L=`2)3H;Ad1nSQX1;%X9z%*t zilbk!5%nHNwN-P|ubbzff6UOL4f@B62vqV&0`6sASf2!&$}RUZ-pCGi;;fLz7z3>$XZ+N zs(1QmqhH{x;NlFsMU!3;*fe=)?MsLzeNOKPIdny-r^VR)W+u@=J^yH>zVDQnKTS(R zFmKG?^M#7E0*F0|QL|pMu=j6pCl53EtW+6ozf(dc7t%C59JDvyeL`A0ruZs(+6mJdrTQSvV-m^QG{m$qemDe7-s4t zHf#9ZKl|UK(&N&jfw`z`GQD*Agq#NbeR-KX^%zYUP3B=%Khf3=w zkN1~1D6GEemx@=*nkYIc{r2NrgGXJ1tV=p6l-FoKLAEqHFDONdua56Xr^0ZB|G zA%(q)WSVyG{i+0Sxi=N(+La}b;rg#WP9Y?Wp$$RJCMiy5&6%KJoF zAv`+~&Rf!gwJk8xX>9o5p;9V4-ILIOA~nda#o4sX71e0m>(OWV)(`h zV9|u7%A8TLs0sT}uoe~3It42dTA#%3VqG#@h$ z?Ec&5Y1%)h?Y+_Am3&rhH{z*eN%7CsdbK!Jwig%zLU^M^@iKBOGGC+P=R8xz6G}5H<{Qqk2 z+vAyj-~T(jJ4Y&|Vx$vARLWs<=-tUFs-+^QN^(BioQD)0-iC<6tQ?Y(vkhZY%4FHZ zGB(?27-kmBhB^FRiawvu_xJt&@q0Xee}4DR9*@1a?)$p#`?_A&ecjjfyd2Ab9WX{M6IovzTqmb$O~t7q_N6p@6t*>v8)+8N_?Kdso~@fOxZi&UotfYU?;QE zXY2~kuT0Y&DrD*zF2|tu_8;6io$re6g!fr;kDa?l5o#gXf)y;Fi=UI! zkaSut@>&Nf_q5YgNU?CE-6@>5>>=M$?q{ol-h+zsU!;|G^p!sWFv5c67oCp>7r*!^ zc07C$U~>(!+Zm|SwRZq-3~OxrQYxb+tlw{WH%-bJ2Ux;~pPismD=S=-M}0p}EeJtW zJE@EomH}wP$_Xd_5o)kGPbbhx5i_4DWk4X~rh`1tZ>vd29Ciu1%kh+nJRkR?jKS2p zArI{3M_t4}sh;V1LXq&rJNl;V`E#0ID#z9X+ib+TK^M?) z*5pTC*_>ZjD$t5&osfR9ss589>4o)&?nC3^3(j@6{<@mSR%?86{j(V!>yiJGTGN-& z5g$|Jp@Nt@$j}+cJ6+z z8P5Xmn8%x7i{N(6LN>7*|D}T1Lr&feyHew2@-C5^e-2dDIUAuys?8dxK2R3_zHJ>Y zAa+8W5CY0onT$|c4|IAJA|wW@ZSH6lu={iyo6s$7hh5?>ze(nK#NP)hZ-%Xf27*)i zEUgj0?!=dM$TaT!c9N#=x_e`tIjN)5q zzXL`}vINGBLeB9AA*gRgjopTVbP#JN^6lR zZW+-vkW`KDuwEN&{t|hDEz~IFR5BtC53I`6IDF*E__~`^i2Gb1@8vlx1EH>TItiEMqX(k zyZ$PDrz*+vpJES)8caSiiCk}6nXJQ@%WKCLsEaZ77kEJuZssI&-}ubeRqO52J9`m< z^YrC&lxI3$T9x5R;L!Xi_!By9CXRZjV18&E8Z&eLykg&ELwH0rp8YojKB2LpYv2e2 zQImka~Z$%iAt-cO}xbn`-=~r2=iRY2K%iH;H?+&w_?Lnb!$?jp8BSbrIU<#p(Rp zKZzbVH(x7TmJh*mji?9{FCrb6y;}OTzH}NaI0a^bwSyto#oIPr&z5PdRCWaWBmr5E zZ#_wkwMTs1l@p3(>AxW}fVu3J1*h1aADwJd-smRvYjz70R4{4ve9=LN zEEzB07-b#%GxTo9pI@7zySw~@zW5=MjC8V3{P{V&_4AN(_HZx2w&*&jJ?_aW7?0>D`L{W*CAw$HW89k8}1FE9P}|MC0chPr;;2`8kM z-y@EKrN{+g6?7qOob*@TXK zY9wbiTgcx(5kE6q5}sgz0h|yv-!H-qJDW^Bdf%F$jR`3tULE#YnsenNM1})69-_N{ zJbuz_lU?q+;s+rh&ViSWlh!_a3pV9mHDUaG3cs#4*Zmi<%MNo?J(Y_xD;sSoi=Q_| zDG^AgM*BLg5D4o88xcdTdBVdjgB$3X#~N6iebk<}KkLteKiQI)8zqhA;4gB-vcM7^ zNT%$|u{zwTSPWg4=|4Ur4<$6ibvB0Df$dVg#H7pm_Qn{<8Nqn#3*|=>9!?qDP|yC> zf1~LYP-PM~Ir1z)8t#@5<4jI`^*l8RB2RC^%r)ilZ`UP->KNB>^V;#H`ttx1Zq6^p z4T)F1|1fTHnlRy`BJGpT)9#a>J)D;lx@Jhm0W#Owj=Ze7{Rq$tqNU$58WqU&_aN>D zjTmCZD5}$Pn%jQj4<8Wg5Xs+%muYmf*ioLL&vsJ7YOndta8&jqJ#sua_tLA~f8#&l z@2d-e)Ag}-%)V~jlWEB}Zs9%>`%AU7U`@lqGXBzO+x%I;DOy_8?M+6-rh*$u&Ps_e zyjf?XU|hISuTet;bE$JW85jNpruf8}V5y6YY(@vT*l|X^g@`BxN5}9r>8QwVV?aROVYb&wJzXzU~|crRc%w#|Sa*8{!rdK!Tioe#HU1E*PAC1Q=umkk)CRKV;h zwfou>acA?pmPI%2#p@A;jq@qUSxEcgtpS;vp*m*oQo(vm|1#7nXz>A~Q5D}L{@hu? z$t0tEBoOcV4BT-zjCX5-ql0VsgRM^EGn>k~n*-UHvr=J($HfiW#tc5s;iEzb7H`nU`c-QM-ge*!N{(!#i2} zf7t}d$Zum%;bj9Cg8h6(_rWs2?m5uRDl=Jmn}xAM0K%sp8n`vuWrTLyp%F5N=psk< zJ?k(Nf=9p2-j+w0uS~Lz1=35K6S(8;myuDLVh4a@6ARp`qYtaSS*bw}`Z}_t=F)>Dc8NH{Ox$cm*#h4xAna+E$c^Aeh?j}%0 z+ca%ERO%kgaf0m1R8O0J(dxD9$E;MIkS$FUX}V!E?iQ)>A`b8f0|8yd&RyL9VJ`W} zw}`dNvKvvda8@$*r(jp$$65&3{mVAx1VSOGx&R{p(kSNdKrH30Tzkrb9(iTnaBR@% z!X)qa)%%QZ#31OF7(bLEPWr7?leW~JLULqRqilgQ*fdO<_ya|zZ~}}Tf9G%Qn7)5? zy;qi?^r>yeZ(yTkCruydk@KONDn=Thq78XuGdoKJ-tf~j*iRf^fB{|Y*+b%Q7Jhh0 zK8O^;nXxR`6EDc~6>=dn(gkdkY*1$Io@i4gX6i;K%w=W9@dro1^L&&p{f_ZK)!ZqY zc?&f6yG|Ke1z@|?>}8A40ORiwuxJWZ*BvG9+*f&M?C6j$(?n5ndahf&w^S17FZtMi z+!L_64Pm)-xH2$Sc$Ge$O7Up%k0U|GyNL*f1*SBHd6jSq<0dl&pCcvSku+-qnewiR zT-v>785gF&tVk2)mVs#?*Y7e-g5D=dzFC@b)XqD-C_e- z`i83Vko1`nH~4X4%i6-UAD!EvMqszm^#_3yS27TDLO6J&);niX^$Gj~bzk`-M^V=o zM2#j#LUhRP47>-|Z)0&|_+K{1WNn?ivQ#JkV7_q(^iH)^4+{NmGu~lW=N^LjZHT-uA zzO&6Q^wfFqbS38+*jAiEn75zLs?F37O41j{+jDP7nIDUgC&&QpLdQcxMyr6KWvg7d zWlXX&f_^>~rB;~>DA$=*@_arhKTF-{REH!(fn<17p+S}CwvQpP06M1=!3RSOGmdm< zRzO{a;R)}$|E#xJEUNuGp{{bLyDRXVJJAm&>A5Fab%^Zyrw2OiM?5 zLB=g+n77^3DhDnP~@a7+D)yMj-s*7+ohw^)hKM?gAxgSAJs zcMoTf@VAYKgB~U@xdfokx3_I-i0+hLSR6610KG5J{rSLbi^uGWNTgYIi%=g^=Ke?R z4z^W?##6s9M(R*EtBJYWoBZdKje=S#T7c$rFq z5lVQq!7fGm(kqgyZIi!o6Z&0WFBouP;p1Yp+&SfmXmp5LwrhJ}-wDBfB1^0Ewk?{Z z%{^eG;Z=@kn?qOdB`>Vj)G845+~+M7`jJWJoK-mCGlAY8Lr&m z=!Pt)4~Y}mqU~sc=NZ3DVd$fs)cXJU6i-x8Z-8J;{Vki+e(6fmyC+J#+cp^BtOW$` z@v-~iU4Tb0j|@?6bS-EZIYt5i4$T^5ZM?H*V+r0o9ZCB99v@(ZQ(;|(5YS8P&c zVC2}u@wMYPhf^&U-Pl;@cqic&>kQ98tIi0j=9Mw$j;&N>NOpBh-vnksndmgDTYB>@ zyMNi35U&aPU`#^C5o1#A2K#!;;*sEMkEs|-S-!Rmh6@iM0cA}?( z>B`a;;n^h%yl9XXzo>U0Cj4_+xmN741F4KhuO9T(W=AZ(=9g{bcofXHZanC9&^foU zO*Sd>+Cs;ZU!NDZWU6wFcgyfYw&g~Mljs%OD;>SU>K31#8h=7PQUE@F7C_j{HSXEM z59!B7_*9YrY|8xWt?)|z#Nv&>+OL{O+WQ6AwZAlmHdmbqbC~jCe}?P5EBE-D@eM;; z%+K0=rXMJmbV~3;ulJAu$-jB5SAS`I&ANtQh;I~J$Za$^@phHhHHO3D?4o3zp6+R5 z1HGfYyv!CxKPy=r?_?8DtX=*7_mhR$|CcAE|LIA{#{wX~`Qc@t>^vFb6D0zEzz%uV{Z|7IgsX!*J5S>W-7WLL-)X9{6@)_-|Vk zbH1y1>wT#_WJ*mtUZj-^H>0!j7r^|jfQu*2kG=N0FA_(^cYuPd(L^vAPq=Sr%n7U( zK@aaW>^MSEBSf3eRC+n=@p?gwsHN`4)iH9V%OsXpGXCmD6x2B^E`swupk8tVkp~oC zGvRO2RVw{ry}~nZGW)LUG{7rz0zyBO&6jVR4?$WnWE^q$R|j_@uf&B7yQkUrOm4C> z%d=?El{SK|TEWc4o0U`83dMWziCow5QdY_jYq3niiATa_0Gl?j`ZfIp063aAMi~`A zjU>7Warb;#+js_x9{I&PKug+=q#D{7IBY&~*Iw8gY6nybBw$QM@9#|)Es@*&Ha|Ol)n7d0P z{?L#$V<5QjQr5Lgt28cEM6=G8Q8ngYU7jUZSmYHYP3K3pk8%aa>EUYA=jR=nezvs&9eHjB&2}$7H4gPVu4yrjX&CB&2!onlz zrJ=A?XV{KnBL?j!`j}h$Yvf*3aGWR%S`1-x=gy3PUqvPV0B5 zQr^A>dN|W>Jg^sK0{|%{?vVkt^-k|($hcehKZ|>4METQoNp~ta`o-TwHb{`s^Bjog z?b2oGe4kgYe45ZtsbCPah_ph9gznARf#wbRxity@H3v+`CS=gH)7n}i<8>Qmz}eu% zaZ4C4^6l^3P2k(#Nvk`Qz`IuJxTICbwtrI4Ko|gRR8&a*e>X()M#fi$YZQZcp|lvp zX}q&^u~QV2Js8DvbD558&b7z^BLlyK=x-GwUp@yjivj|o9gVGsyZS@6UFq2Zm)Cbq z)9WOPev-GN7|z!f1wsEd5&$OOZ-o;)E?1Y^r{LNf{Qk3lLFGyXk_B=e$vo3CG8fpH zZ@;2Q{%p^dvl-_?6ow}Gla_%m?73Yl2uM}riDBWPH}(5R#A>w8#4HI48f0AwU2gcxavp`RyoIi{o2hkVLs*I4h))zAW-~e{&mXa9U{saw1L!Kqt*Gv4*8-2cbMy$Kr_Ob| zd+VLxU6)rV+8V21qk6(s#5a@y++-QAUHyK!$pGA{8jf7JH1jt=b%~LH76*TB5#cR# zB4*&+_z!@sh(a&U-OM^A`muL_PXQ$N4p61Je(d3^30s~TpN%G$p?3G4tc{sDUZAmj zSQp1ij4B?xgkufqRb0@BHa|)~-pnykNt5PpxBMZ^KjL64OWZXPa%Fw=W3-rOiu;X7R~2m)_tw5w`YXTGu)wg=hT2 zp(B($hYge0^E)TN1Fe>SuSmOvaZ4d4SyDmtoA}>L@&>pxuj3yJ%{Rw{*o;B2vfG5Hqxu+z3zJerrLO&Wu-`B%Kxr=GaahP%WDp{OG zjy^Ild!$ub0z2bZcnN(C=ydox40WR6Z63}Nbjwkw0-X?6A^xPFG}!>^Yg>RG5seB< zTgaA^z4fQzU8grt5fsKOD5IeU0d6#Z`J3u+EAsmiRR9V&TwQCVOkfJy{muGK5$C>( z*nF0PFAnRQ3i(y=y09pTjZ+)7E8^BTF=vz8^2Fzq;4IqaV_vtLSZ=Icw)H{OHq);T~MNQHpXm22xX zhK@wn4FNR^2Yo!unT;s0(^un@R7p3lIWxXU*=b3Kk#aWJph2SxJmoe z3+tW3RDxZ;A4L)@r{ehjb>ZIPQ@ELu6_vfNID*>B!tyLmi{2Re*l*D+l7!)nVGgVG z25zm931T>lG+SMsFJtVWkDq9(@0_Rl2DOBVZNT7gFL9 z$@Tu=@F!`f|73OPZBX;Q zlrw@+1cEaINS(+BFk)~)g_{+y^Q}PiOSMQ<2s;N{tXwvNSnGk9JRvk z-$kWZwQ-Ym(5udlMx;(=lDAJL^(NF7!n$Z_Y*sfMtlXr6lzUJSI1|)-0Yk!Te!JlA zQgAwVHBeW*te_$--G?Tz8a6C&%EKfLtHLnUt*?!UeTkCyl=@h%&}1!6WIAsxfpn@00a$lcM$}+i6*K!s|@{$AzhG0^MZw! zhShJqmdzF#JQ5B+yyw-O&h>a)CWT3!eoNg_P|SDHW=srSK*~|iz*v7YTy+PZ+F{w_ zkxLG&_4!I`3CIACJcr{IRbZ7I=F?KYs0p2?;BLkC`vBm}>BkQo<^jC&GhJ7p`FhXe zVh&-oQ{)r{*3z+-{CZZ0jCb~ytu&uorCt)7F&?GOVATf>8IX2<}inHl)IEf zT1zoppkLn*$XYGeFy*>VPfcjhW0u9StUHT0BtSynb5=<1%g>+n3f~6R0ZEZ>-Oeql z$g+zWL(GF<_0DPFSv1Kg&+z5WCD;3;mo|B9)VE7D4i?r0Q_B9g2k>{C`v+WB+ZE?g zU7+(x!zZ;C-#yEs=$^I(?A$pG*b52_lakH}{=3EPxbo6)Q5=Q7;DFK@TlWW6E(MLI zq>990)ro8l^V>MiDl+{KKR4o;d8;7~gSR|q4^324HPOzv;VK(+kDm(FkM&EL76^_bfdZ>4o@hhN5j`}$fmmwE5sv)he|hBoBFrK?!&A%)^p>UGlu_95ro zIhD%GJ1dnyrB7+2wx;$fC$0^%MYaFXe4xp>mMQK{jQXkVcVW3K46R+t)%~xFMN9Oo z3fCwUH@u3Bf@ueI1fFE?xKTF%jWjuYeyqFio>!LrIlgfZRF7X2RHE;Wear2Q%o{MT zQ=2HH`W9^B>lIQolZHRqzMo3XciK`EA=ge_`@(LX`Fr!cwvGT5m)|jof>~DbX0Etj zkD93In#ckuzlWkt;*;F3{vy7Uk4I6lqL~@XVLPI{nff*MAfW#xMRN6Makx> zlt`T5&_txDJKXHeL&`*G=fqFj0a%E+F5AEL{IDZbY-aC9d4Je#^!i)ER`tcAs*~&5 zO3p$;X2?n}Zjv&f#IA|ZX*-_^djxO2J7h)LxOb4$hO|zh|3zsYHr2hE0q~zwK2jYN zMb#DwXzksV9a2DU?5{OF*%ihd1BXZpOGl*PT^sVWv5^Ee7lC*(UHyTMtk0`u%=(2% zQ(l^0x<{F^BI3;54@-n{P;?tW#QtAIaP^^fJ1^Yy-nX66716krb3~7kf7-K`R}C4^P#l>+*W~KIrT9#s58Pl)fnfg3d^OU zfQAeocS0e09{Bj8pNzrkz>nVU*a#e_MLs)uzrfqkQJX>6e$5GR%wY8E{HzX-z=eCD zNgndzHrJOX{AAlay z3q3Gt2OOPd_S&Sjlgo$J5$B9-$I!56MVS32S4? zY8#av(Jlthv(^F;RI99Dk-BTkm}q=<yV}RR6wg~q0kaP-dFh=IX1SGBZsab1%zoA5U1&(Xa zod!D)Z3BR#ZTv(qkv?R|_&YB3QIKpxX)z2E)?H%(f_lA*a1e}s)svfne;gj0{ao}` zIFPyL8+i=;wZcify+D9E0Sa$v^W@$k&iB#B9#)+=FNvdq4LcrEJlgg)9II3C8C~ZU zhHBvMmVkGuRLn;55t|PCA4Y9E%iI2VQKkN8j;#wzQpM zDJ=m2JMCSW;9HAhca#>3-CZo!2t5np!xfFI(LO_lX~jSije|XkgV-C4oh zIM!$1c&ZWV#QRR0b+C&c86>Wj%z#93z~ed*e<9Rz<|gIkWj-Npm#UYm5ut%E!Uwdm z-v?wHuq(FH`;|wwLWnQ1RZrl#Qg1kZG)0s2$hOCh4XD2z~}4 zk-Keuz!fNLJ{tonZkWm4NFoY4T79}ZTFUqh3dO?nko!gCvO?*paTlDI5vUXZ)uT&4 z2NzFdw3hSB0@@XXS;+a9B^ss0+*2xH54Plc<*{eHpXH%`+D@*y-APtP(rn(KUbZ}< zjt*tIVoO!OQ?AT}M|>HJ)ttt0V20br!%g+&vNzgYrE_MxKPzl^<&Ik6$SahH7uRuZ z%WOxqH(eO(b`VwPukiGFV9#AR-yd-U&iCXWmFuhO?2K04BnWQeU!4<$({LW(+t!u> zXeet`Y^qco@_MVMbQ%zKl%=Jp{t<--c;f^3J^iekJ7))`n`IM-kQ9t%IZIJArE<&@ z0!RWd>JjF88IwArVJ_C^2qCkT_MZ?94$=^e}} zc5=jd7rUi;0dV~=6M59*(-2Li=CYaXZDO-`g_|$E_6QA<{q5Vw;f06 z-5go;o%@#cZOlgnwkC|qHMgSzK`PrZU~>r9Sc+sypk(ZAa|O+&0`$O?Gk}sc`TRmo zJ}0uNo-|T+ZJTdcb@=KJ2zCR);!P#TrxP10s-wO*$e2015TmiboINex@yoQl7#Yf~ zn~oQRxMk6L9RU28PqLJuXCw4ze?&S=vxI`qygC2($~2(s4N@0YW6vGfNZMFJ`bWCc zn;w$!Zm-BvWm2aDw&xZo{VZ3`c-l_ok?`zUpykEdPHiH6;?~a8f)wH5OEc1` za$4#ESo?t^N{q*p1XxLe^F)zUU6|MP<=8Zm%kB}e!3a&3+76$OC{vdM;vnBn?0>Ul zDh_?PMci-&Dz!h4!|_ltebaj?ur{aF*!jBv!qcY5UR1EU@RIVrGvcB$X>>LmT; z8+j}r8dGa%><&^KKfd`FD{ak1T52sqE@Tee0R7ytSzgJKU!#C|7qo0JQG=Zug1F0s z*SxedOxpGIN4fOTSEu`PGqAZaOje9rR^%7MNV9s{*P|Aab6uld`flQgw+YTV@Q#fS zQ;NuIhr;8>!w-I{-)~6$I<*af|MclJB_9BGeQPiJs7{J-O%sNR?~?PfgCK?WFZinr zjDGXdH5j_equj9U#>G!3x)>KSx<^z@okm*;)p?ddwR)$w2(ntG_wkQ-rZ_W!tB}ilA;{mg*b`K`m5&LcjLkM- z@X|ujHJ29s!+t(`Ct|cJE14{iX@X8=u~Tk>(^uSc_?MU32XEk9%HJ}@;t+%b5O~Yh zcQiZd`%tMJ1?;H;LvnicdG84ur_7(Hi}u&hY_u}3qK=akZaKzCpRguGF-Q|yG941d zEAsrF;L~2i+H$(ARv(Ktbw<}ghXuu7DyZE3_^m>*R<7{}CndeP*lQ1i$fKwqo&H40 zRX=S2eQ7G|ro(sI ziwuak7z+WokQe+uid>raa_WMiSZP4y4|s7Cc;m5jI=w&8R;)L)GP8J7(Gndw1mKeFQ*+6#qM9M}u5#2=ko;VA8$A{e|)l2NS6U?wQT=M%W4@xOVP z^!1lp(|Ly)O|n**^*#GE5I+R7Qn~5)q@iX5pIZrwb?)%1g#-gwy*HP~*Llw~ZM3Kz zK(J8K?edJVArx_Mc7c_1o&7$dYQg`-hU*#$q#aM=7dILsW#zT_Z1_)*?fY16OVv~Hn4c8aPOx~g>F&l1I zCvM%uSsfQxbQVYoR~RgO9&|HV0lOIvtp3y%AA#*kPp#(qvvKcjooQZx(Pg;lRYcf@ zD;9dR=@$oG8!Wl7Yo8}KI5|VfhYhLQ4s=QPS}~$IvZ&;M7chNxg_QCfs&`jLVL5Q= zw!hf}z|i?^e>c%LkLS!v_()m^0ZDzc;2mioDYpb0>~9#i7zx{jwyTLRzvG&Dz9Jdg0OGrL zAG#-0Z-$FK@CLDQ0d__s!05vLibo8#-pB}od|jn62HD9fUd6QySNw+`V6DT#KW@`k z+H}#!|HEdH=x&zMf(u7@RR@Y}sNFw20(7qX9>nC!-(s8#u>gA0{bJ(B-w;;i=`(R9 z&ktgx~s2?{dB~ z-8~SEu?G;Te~xZPv4;}W`3~#2j98J@l%FEefTAdtKA815?iO%lICs-WGflV`a96(2 zD(bfCAJJ_bldIDAOyh0yXPkrmi;-Sc7YmiQ@T%22B?hdWuxbwMHJ%qx2Q(51>vRr$ zkb^PA6r8?zxQurja5ire4E<`!{W;;DwsD0IgGxlA>RM?(;l&SO*nm`6Q5lIp6dcqv zbL9;_Az}z+uB6um>79L4#+-e$sVkTLX49^kk17$Hv;g?W@UMIXH&;+~p(4#IK}lgg znk0teB)Se#Q=?d3?ez_0Z17rx-3c5#fLVX-^0+8r(~wX`B=Voy2q;S2RQk_OMA~!< z_k=Le#ljcpWB!o6E!3vTOa9v*+f^SIuTXq?pJ6wY7{x0#vlj0{^MKA}mwXugKZ`Q! zj;YhGxmQwG5HlnnFBlhEdKU*lUE|U$E9_KS{BgPt7xRc?-chvI4abIb2GRbtNP*|~ z1~sOuBMGuxlqcLUGg`Rwk=o__90YFuXu;{n)KLU;xNyjd=F{LJ?{CJ=e`e36AZbZT z9aFmil@e5;GkZ^DjRb=}PUADb0~fUa7~C;n0oNqoe3}VQe;|Aoc;2Kie_HKTJ3VUy zki&_MVa{QpYwrE9SsNBs4R|%-3+{E!i0F5KJ18wbw$}$dw;~Odd7^>%ZBw;ZPSvX= zER)IHfbFjN4-k)o_)$Fk_rjqlgH|pW=w;)&hPD;nMS!q!TjU@P`Gv8TT+^0Pibu)U zi|+q$2z##*X6NtQI}!{E*;kFecsq)PvCLv=qvJlsA&sqxft1aaKARmMy}1(U}Hq6U4$ zO67t}wN8POL<$Ws{;;iv3^c@HS-=AhtT8L&N0#AGa03D~42b9z*dZ#Pn46qzqykHg z02xDA%#)7$)p3^&>$1N$RJE+@oyi=^-Bhnxf}Wp5odYMdPv*8I_Jo8fqFf!Xfp2haH_EmFwqfC%MIblGTOFBuVWwwH`@ zVu0h3cPmCplD^Q^W#QA44Bz3F-}FqOBR*2ifnb3H?3Nh-Zqkm#OtzYT&O=d&2h=4s zK(qr>%7NXP*ZYJ4PNdUb(@4FFDa&--Y@||lmZ|K<2Hlp^0-!Iv@mbVZF z`l<1L%&nsx;#L&U(VYjSIq1v)7<4YJgR%^2$>1Xyy7repR(e@@yH5{KX9k8(ZqNME zuxV9ZL-ioGqOx5KCCU%HGTn)NDlL^46g?>85ALl*kZ@<$VZvl^dJe&`=RgyK7eVzu zQ`{ZyIX(Oy?#2mbdlQ=R7->Ck75jXTl?4$XP1V*P*@wBd`+jQd0yU1!-U~PdnUvrE zR9YE(-wJ489ALaM%E*&(wz>{lMLsi?4zJ*172BU9<)9r>y~3j9Ni6%I$~tK;yVW$= zUn>N-TZ-d#G0UslamrUsA_jvSKCSYLj$k($SJjz$F4fE|+w*r1RG-#~G17~JO(3P~ zVw_<27;isnFGaq0Z~05(Q%;b>VB$3%x1=a`9hgE^+3a1sg|U>#A7uiRnox8ZBKiIj zZ%y3j>Hp?wC5BJgSmj_|K*^ok`%U=+|H`eU3aVu?>i`VauB*Tm7{MI0qh~hViCx~l zhvl$;sc2!@vZUR<2a949e6o@v&83ot)xYVwJ>L!m^$tQK`MkL@w19DmF#&f`dXT z4vmWrl`fUUEQ<7hzy4PO|0{w2cL}`MH*3_FX;ZUZSM+A#zfI3to+&zY_fLD#Sn literal 138226 zcmeFZcR1T`-#^@{R#B81t=XkE9YPy|+Iz*GEw%TE5kyr>QPi$Y39*veTdJzs*du}% zMa_u4WBh!t>wDezb6@TK`|mlP<9Lqi4{`VqpYuH5>-BoQ-=9QiYbw!Du~MBocaBCy zSzh9 zkvXR#|5VTK{K^DHAfv)*$A%>XREh8QZ3TyqZzD&N&hh)SqiF*ChdL&inBFz-VXWOo z0+jBS=e;D0R2CsqrEtjQK3#M6oc0`T9G_d&70&j5GC8rjveFf>7t`6*{qDy3^W;}$ z&i&`*KWFeCKlo1={3jLsClmfB6aJkE-|FAQujIwqUa$rU>koF$uCP~%gR5cX!%?I= z1`h`AXg?v}7c61N$JdI4F=&R{nNS}$cGU5f6uicdh|JM@cpFH zf84ema%8CwuQCdQ)0h4Ccb+FNe*KxnR%qyPW%s2@G)W-Y zz<#g9O@oiw>$;;sWvFoq#MX;W+?3ht5}bX#z*zCoT((oqx^|_)(;woOKq5O1{{CN~ zSp!s@osj+=0Ut^mLqi{qDvt@A?W2mWO;riY^%ElkgPL(@FSx(gy38)8DLV~|w619m zG_*_WrgDzmdLV>%mM}*gwKUjb)Jn_W-W|oy3tY4t z4#sw0$25cN(}SK3e<~j(jSXv)ux?gRhrw5ayxA1cbPz{VH|P{osugCOoVBZ0TC!*o zX2&OW96f4o{6K8AaW=F~x$=T&5Jm&z=9`oof-hXCWSTNBB1U&1`*x`VO04wYfG z&CwJ5Uv)0-H3{Olc@|H;i{VudCX4*y-#M~mSx0kbr#bsf_}earZ_$+x!v$@2q$^W< zn%}HWtqqN?8*xpbrU&LeOrS_Vwc(9=nG9`r@&n|Sm~GO`8Po0!LoacOSo-!(@!*fQ z5%{4-@f5<2a?%ns$-JxlO>B4C@;$Vc5=Y_U#2VDBXtQ_k$C5C1slAVA6Qorvg0b!ZLB+mv`;MTBeNu3}5&WatOdt^l&F-VHQ&v+G#k)mJgg_daP866Uib~mF`d` zLSB$bb&~mS7&TvPqDx=akRkpS2IoRN>D|%fXcZbb7<+$IX@eVJ#)<3@i%gAlE85ts zVP8{PNbk(kq{gkE)XZ!}2J?5Twky5LV|X_j_!`$1pH=2nR5_A=3edMJ+&NGPt$o;lqpv7 z)eptmXh4EblFi*wmuUHRHmECi(`J|H=K9tAPo}6Kfyf6=9|I;K&==8^(kcE7HkCif#3qBu>5jNeo%Qpc@AS!D+1_F6ENTbtOz&Ol!UT?4V~)TV22> zhWyCxeo^bEjgcVi$>!!;f@C`taqBh@hVapQ^<-biqkbkr$~d*gj?a8DW%)H_r8KgE zcP%#(J}C!Vpmhe{aMR}`Y+t`VM?W$LDqd-{KXq*HWGW0`3pyzzZSfL&PgMuK-mDr- z+8_&Kj=?&oZ-TOk*Pt!B7LBy8GuSIBm+$HZD{bWTd^jAF*4XHu2@pdMwA4)|Jh`~n zb9oLtAJCC>kA9B#C?gHo@hJwVm5Rz1u+vQ3b0Awzc;EZb(3b4q>&icq?_KSig^9E& z1Dfz}ZI^`U+8d7t97)>kZ=>fh)&lSY!&qbHTZr;)X1FoZA-Lx4`_ivSnG+z)C?fL~ zTjK6y#$^uSTR+sAhBiC`62A}G$vG@+Jjov^$lygrNysdXSDPbfQO)B{=$s1CPQFD! zwwYRVl?xoJ;7_Oj&W}#PBLO1Z)apF$+U?f#MGcs%W zssmEe-0o*8e2$(7n;?+tBq0+I+c@$z*@**-9wmLEuIan&J6UIIn39O#58oU`EGkkb zJz3nZo5F7l^SC!5-3FvPF<06{^fhlbW0F3Q-T5CC_8%v~$DcDiu3tPB8`(lL)URhM zXXo03tI~3!9;ToSP_}5@R*J6TRGRt}5$MCaX39udmXr(hVZrB%0yReEZcBcA65yX% z_(?E!SD@}&ivrk~xzo2`Wo-W*uZ^|;(413cw*EVXND2~@(wwxB$NtlcgA3ir8&VTb zRSTu!N95|D!&t^Zj8}qH)QH!Pv@8vxkslpR=!UIp<#Il(bQ_m~dx2e;s2h?apjzJ` zYj;1f4|;E8^BMfOZuz42`RCr0$#6twwRdmHxfbg$!LYD&?<^ z$unt&tSp2>T|Q5H>ExX<03Hi#3qHsM@8>lgw3(f7m9%=B%LGyL9UK5fbu6IhXw(6O$hJu7lowYGWlpmaa$ zgGCa;u0YX+j{@i5Z(L)X^1+ij!N_LZ#KxR;2?VDS1~-@iYJApz6x7zVA7|;E9HNG< zE=?|#<;@C*Z~77aAdX2U@dMWq>tJv$T|ou6W`3`;jgD{39T*o3((9V%C>9&OKfLw3 zJ(KirHa_boTHLUm(EG-LOu-vc4mrvS!w3bhhLl7?8FCfxLSeI zdVMEBvUNdycnhPQLbI&gulJmdNbPWZl(rijNLCc-Iv(&tMDm}Xt7Xy2j4)9~>2ih_AoRF#D5|Wf91f;W+ zy)L0h((2pYaq9kdX-#W4;NiYmT$Z8y6AbkDHulaZk;%imvz3C{Vw@$lFQuq2|ArmU zTwv~2j^fXGM{*zV1DC>YQ-**v>$MhJGQ-R=|@=F|Lj_`6n%lc+d#;`~`@sAPxjl zMVSLpZS8=sgp-v#Ke?Oci`Gp$dP+TL%+|Nqu2uVIJ{eh-1$z@t&?e(Ib?R%4y|TSG zRw2?FXz_s+;fYgU$Ge@rP|&T%WxK>&T&RAwskcRKjfRf$Za~GNJY14*>!(v1(Ba zNidR7soBA^U0GVSS%E&hd}i<lNM)wIRf*b z7gL4Tu8Cr9>MZnsdYO7?Q*Wwlh4pwg~wjDtpKaYDC`5T>!?X&;MaU|L6>64TeQ|b%DK| zTJK7-zu2wa^q~%|jTYb9Tz9t-L~VT~(06Ynl?g9zzf^RR!f3I+KEaT(f)=Mgh6=+(G4rPoQcB9net$b#lpuoy8 z1AJlTzIJh{U&z(&_VUu^1~q?FI_;W6J8cNTb=){)q|W0}eU_RjCUR?S(vJ!&>qz>@eZj5Ud~PdjWE7Gcg& zJ^9ATPkPKPc)G@OqpzO9BkMAD-6siZuG)DMa~y$C``CFHfoZ`d+p;b`cR~wX@18N_ z@Ri=~#2uH_X$o`(&!!;UO;ZV}E2)H9WX8#BCn*H$^L zY3A0m-W4m)8O1!bMCU|y4Q;&$9&XO@x;s-Th!~M9TO~WpxcNUdhQG0(@=vYm`9mMs zUj*Dxq-z|}7khoYJ+n8O>bgTs?2ORZNx}?^s#kDb1!8*7g=kr3IWJu_OfV46+;l9A zNivs6Q+b^?qkApRPtc6$~THWIs(by{3~Qp8|1g zo!KCGi%b*{q$CBp5OF(Wg^pr6tiD=-kfv0zgYd_MQL}|Qizl%^3_2pMQUp<`ujz!+ zW(&H(7WL?u#~r6D+_jF~@KpXm5@rw1UYU!Tm7Q3)a6t#9)fez7w(%5F6@L(x-^>%# z%M7y~=)6_p>F>s3P~NrkL%q+=*w1lSPwD)bele?G*z;Qrde3+sAPUQQ*Nfr1Zlk-x z9@KSj!NJ1??f_>VRdLKWAOGT<%4*l5KdwaDU32jXrA;$k`JBT-kyh5X!DA2Z1=?sjx~J^i6h6H(&b5;7*HO^PjkzFXN-t-D~Bj}vam+YdKsq9EsDA#TC=D>2hR z|9kw|DU7GOlLA=!?eHu*U31r7Q3(}?`PZG`IRTbf`3xR24l;aRvvA0^O^~|0IfMD| zR+rc!kj^mE2MfXF@SNsoM!iuLH`wcNRi*aZ+mbFYYH20uQ^rcL$z4xl2uk^dUIZpv*RhOVr~B`5=F*O zn3}%t zA1;Vm%|-VG!BBJnlHrnFNk2plee5iEE@X%~r;WQBB~mT-@3`?F!ePimDtrF&-Lx1H zxlG>9k3VDwW2?0fdsNe}X+Q^VzLar1gPrjvxP-AkuU+O9`H=rZi%g-CD&avsxXseA zpbkZMl&Z8{-e@=+ZYTX`8v4(FrtLyr(cRPgpwqBvN(i%R=;6|<*RYmoyL2W@9uIt? zIZv)@DU(_Crwa7XwY!rot3Q#WDGIGdN6*-|*GG0T;T`0o(q)|mh+Jt-K8C@S=$*8) z!^3Ky>7b5&{Y4M|yq>T3-rFR@UK(=&5I)`Djwnt7f4IZ4dulPxnv^Q^R!B+r4;1pZ zbnw4g>HojK;1hoDvfHm*8Nz3wQOpwlay72ft84Pz9co&Ib0LgZm9;{E+82K-w0}vM z@%P@_4jY!s4QR@I6Z=v+NMTD;!vAJ9R+B$qt=I&Hi9SabL63<0;QamK9~ADKNUHpx z=fh3OqXX)d(fz~I0+kkr+-tO|Q;(-swA!agY28t~`vU_->MnW(EttJ2fbD2_x{v_z zgE9%N_eA`|99QwtZMe1N0rjE<@OIlu+c#S;QThC~4cl}Knla)~8x^uI;lj%8HF3uL!3iVUE(Y7Mp2L_l2T zCX=iY?~A@|C_&T>;hAN12IUDrOGo>$vIXoDPaDb1J8j*-;=H+_x%yC`>HM-J4eJMh zu77oBOsA+RyyL?w%=l9qPSk!JZRr3wl>1QXE#*U%2#EP|T`8p5smM^HmVraqCRg)U zvyQPeSB9E#C-eR2y?q%e$=g49eoB4*W717B%PA-j&|PQ~+F;%SSB^bljriD9z%P}* z2(gn%BCw z#+J2p8lr(6I=~Jq-*=qi%g^&SrF7BkCyEDCC>%>eP(OLn+$FeEp`^KzW)fYolI&_I z=2Cx$v$Kr=SYJTv_G&cd=skDP&orcl zoCO2;ZkIlh=aNF$P=)`XnOxg5^WW7qz9i5)yS>i^Dv(wF9`v5->QL@$nejbE1McHM zcR+NW^XhtC?B2ua;LC}(bGR;D{ZpqFV=6OfSM8p8B|E)3a(;YS@NMG*K-$)8wLIDw z{mADMreqyO8OTS4{xzwWQslXYZh1!PilkfJV?NDknlb>EmAQtK3P+msTXq1@4h`;I zB!)Qrnr~3lE`%N8iqp|&G&}jSh=jU=tIFB45bP@TbXLt=LB(nJqKGyFt#+2{1@?m% zxe`7t$aSBQn}kd$F(;E`^{F97$a4Nl!BSCLI6L~mFUYuvb@RO+4dNe*-cDMp90$HF zxRU=g=i%l1U*HeTE@FO-U)xrv8rjpg_^t@ zh1G7RMpowJ?)qvwXSj&*eVPLC|J@5IyPaKPXcba2c(1Z-N82ht5(+li(+{fF!j7`* z#U@luw<$aM)zOO~h&xsk^-X!Ab_h=C1Goo%D92Jw=wLQ!)d0>mU&Bq1 zdb3r?=t(<>ziXAJb}PHF|`RX8e@(*lUX%tJps z&ybvkD!Q?!KPPKm7eklfaEcJcc zEqbO?_2@B>UN%Hcf3k1jMY?&xv6q)DP;K1)0@gtQek!(oJsUU0cZ!t4M~M}A9Pt96 znL@LJhv`MU79DzK#AiUemd$31Iqytvr=-0n%X*$pZ7YQ?bahHDTxFt-^;>T)5m9B{ z=H3}jC3I}82!wE^RsZF8cgHDg8n{BK%Qhjy)y_qStGPr}39L=87PM2B5(f3-P!J9h zT=HH?Ze8f}+DS07bMfVHrk9%rc^5E5Q)zy%NzN+U%fcMOH&Z|Ql;O$A#$r2E1e(sS(k`X}lJR4!a*YCkQg`=s zGEmzyi5@vqQ%E3OIrZEx_#QsHEQSpbDP7jIx>nN90&~DrX!uW-&NA~)g&uwg;!5mr z2Ayi+h#Y`>bcUmo;2XdfX)ae(ui?ORNsAL8l(2hZAWmj0T|tMcrAUr^ zmg2o}b`9%7uR3~xz1ORBuriC%*=?W(Q-F{k>AbCBeo>yf&0|fQKymvi1`t;vK};4$ zbb{BLq1TfhGIOA2y)J89mc6%^EtP<`{O2w?@3XTVg#~KRZJ8 z+tI;Qjh3?vQE_>3EAsd3aMpNsJ`Z?jf!u%9Gs|kAMh@7Q%89N&gshBm`xwO0xOP)A zLAdnq<5Az2L+Z@&Q-ddJp?idCT*$|;Vue!l~Pjv~%*Ulolx_do*{GB|-DUH%i@6G!T<}rSSnCoYHu(DNy>$R#=opu3far{U7 z@T|Joo%=?a0MyR#N$E_c(3t+GtBpPW=q#yum$$RQ7#_Iv9n9guRSLfF+uCOWfZ*-l zyy$EqK(Fd0tEpkp71!2-7$)=NiY!ZT!9rZ@nUSZ`&5*Oo zd-rNW>F*UT|!%oGcaw2CIkZhDD>88#m504n?J;tn)nT(USBo& zV6~ol)MQVd#(b|Roh|Jy4E5Xh5(+It6>Q!imBiR!a`GA=``sBNXBSm+Ub(qzw@Ts$_n_xslf-?d4(K zGi1|K?g#R^L3_ewQZFvL^F89A_`{@2Ii)>k**jZX3PQ@39pb^BGN=bT%M0j>HPqx< zI(PzgW-~=Hr325fNYw|70KbAj5<11Agp02xXL$c2GES9tCW1Xtu`U#5 z=MQ~Js(tsz#PZ_&@l1=qxrR7g@h5TUEvXpFy}MdMf2*}nziquAIw9s~Y7^Xo9yRBm zoi5fom9oizs@$VC2Vs8_fnZf?B-qx&;CV+(*DiJg_kEk9ouCvZ-HoHmms>HDto`f~uKhaCzqz=vKA6Mu zM@pKpiUd6$E8E_Wh=FAC;#pj`0^DiV4}d8t-^{7e@{}XsPx)UhOGtnGJsE9^y^?tPHoO zjS$e#Z@LH9e=PAdn+~2CetcQ)-dPY8-B0`jb5vG14;XJyoTe?ti!D1J@GvbS;%-xv zgtnF8H{}aPXE%maI#9Z^E7=$=$>}W&zx}hdLQ#dNJc1E!e--eV@WA=|1Cj;qbF|}) z>tZ@&J_y>h>8^qoZ`8{Q0ItK0E7<94*Zrq043rU(dwH(!(4JJl9Zlvm-)6e|ggmD) z*~~kr*Rq_ByQtM7|FN}U#mFb@`9;&-fk>nw{sWzG+j;$sH2GrN+tOUOGsA-}>xrEY zvi&1;kLSy|^a1AhmaF93$wS{7#PnlJol(6&Z8>i@>=G#HS2KHM)HR*k`pzGgeRDQl zuL?_RXjP>QRCj^L+#UEm8^iXi3^>dR`w@Mfr4Q0A>$;=AQ?iM!C{73K< z_e|Q|nMvOGs96;s1}Ux?UfQz)gd~HCT>CvapJ@PmIbqth$Z*?h)Lda{4o|55(MBk} z^ixL8>-CEt6Yy_CAV^rOULlId>G)mg`#5$RP$Q5vtO%Co-)-9QO`MG=ts`-Aa__~o z_taOWPdYR%xT1IIFT9)ouw_)Xm98$7UCpy=;dE08dpOYQA!GCf#ff3zM)(0^T>R`(N@SCr1c7^ zaz?P0xT7;259wpwRz(%r~S)IyEUrGpo-_9$My=UGk9qhHx|{Fl->B5ICOhAchmk z+zl%5OPhL2&0#6muhBnj;ox~3BQh=qj@_7~h^XVb!}#DyV~n_|0Nci>N`=&yOq<+M z(sWNj{lY!_DQne$=GA8FFK{oslKrp5-V>H8moQdsuD;v%i9x&RNaM=+2Dj=pd-U-G zXSJ^Ym9Ne8a&b{S8uJTs8;>@qLNEf_d7I49@qg&l7oBx&XO#4>;clKHS?znfhoIb! z03>`eWukhb?2Ts;Cl?q3#yja%FX5;(tzW)ko3 zZR?O=+If4gKKlU7rjU}FxLt5n+1Iz{up9E&2TCL<-AThD-1FBx`dSM>*$%1V*)W^5 z3Fhx8NiN9h0Y52ybXj!t#qQfCMZ!@%v?VGlyEG`JxU`qBfpni6I0;DfdC9t1nbdor zx<|S#MEHm!5$a60p~wPG38O7b4Z0Mi+p8C_+^eJqW(V=j<~G4kH`p=@b7xGE*9sE7 zCK200zg8y$Vwa%JRYT5AF{UV7cNHpSg+d?}7}-d^kmOL~@p`hhfoG~;LrQfC)pvrb z4nh_H2*r|gL*<72v0RMdiwE4T9UYrzGJcT8OtZKiT9tHgRcmw{@^)|E2+8g6S36zg zk802l#zkwf0|7u)$>2{907g@H_6&wMK)B7i4CIbZR3BO3%{!vp9``x7D_36g9xrjc zk(HgN(3+ZD4`wNq8?k zU64>OitH(zO|Hw^Mu@G&w_=}k)J5{%k?&FtPak-Tyb2?l}$A&su0(ZyZ-wrAH z@j;%4x^siU$tAcQy_{e@+#rO;GE%Ri#a7zRDe)a`pw1FZ-)u_uj;Z$%YkPRh@M5-L z^rTMO-1Mf(*PtBFZ$Z*w0+toEo$KKluV{-v*j9H9hlNhj!DoVO2y3Z$(nNy=%a3tL z%~1&s8T!C5wr%*$(sx4PUdq5~v8YHrV*AV@s=elRd5x>$KemGJ06bOj75Z|K0Cnw_oqv*CSLpTi(BIB>@;lnxQn4L%YwMU2Dfe zwAiM`tHGFU?tupAx^*nB;a2%C9UX@W1n17kw!R2P>$uIkfgDr7m$6*p5g6QTk=t;R zQPh7aB#7yFKMrU-bfYRAgO$ZSdj~vk+o5~!$q*Ki>4BV<Pf)#h>YNw~_jS*>p@g zQausBZ|6_6;y>4Z!%qJ>lK(f4J@uM`ct^t^J=+ zhqqA%7Bi8XN09`gh>_1>r7W$y1&e#)sjx6qM7ZEFM7XJh@#f2}^6}zHCA(=uh*=1Rmejs)-sJn*W zi8=Wojq^eiJT7etiiBt^KTc1(@i=`C=l1hr?B9JcZA$Wo&Kf#=H5`t~ir0+dlok^p zkGgF+-I6C?uAStHCC%+5!Vag0sl+3QQ%P{;#0%%ik1h@zi&y?%A8C-eX>!(&|Cn@} zDb$ihpc+{>`mFGM%+XP51`>08;+DxA&ZU(Vg-_o9%Dl?*B1||Nr%p zLVi)MKo(PXm)@==_>y`4_okKkLih|Po?FuavB^OaK=`tP%~hG3)n{8$DI=)3{L@9w z{^&;K!kEalld5VbuTC=*z{%5P8+A=c+}B+1tYT#7WFpdd#y)xE&CSi}dtEF(^OAp@ zPtom9&zC$+5AfMFDpR0Y%7a5#6?8xpUuR__Ihy~g+^L|qM*k51Y&J3&TmjQNlQeYY zaugl)LQVe8hcY(IBZzN14`z8rJKE6IcHu$xDcRzE$ofUmsoN+^bgOFkDiPromV8U!qW6U0Ml#Mc=-?O?(Jz(<{fm!;d?F(-3)U!b!5p@-U~R)97LJXu z-B;MP@m%@CBaL+DUAf-Tnb?*JY_O)t*i_YC*6_xvQ<*E?C9)=I_v;*h_>JV!Dv>=} z5NwzVhZ~1x0!>+he9NXFqygI6rCLd=ElWSVTJ4@&wyEMV6wWC}_LnP#e5Jnsg?ppm z*S{XkxkV<@d=O#&;MY^OgNf_-I)^-$0j@;Y^Nskq;k7zy=W^Js{RySskr5HgaxN+V z9^2@HzWL+I@0#exook;M@Mb5czCG0z9hfGMY~!8)IhR#b_E<62X^8UC$mZ5oRE)~d zlcL2nl&)OkEN$wuy)Fik=?Smaf*6 zl}%JHn0C(6R%3(Jt35ih+)6}exOI&-w*1mbzXU9dn>Q6zyQKbc7Wq_+r~Y=jS_leR#-g3L#`RQ3dgGAMul+OgW9i!S{mCBufrT+9XIaJ>J3;$+vW4W3L9AU6p4^qsd{lL%Xd7DX zWdGlDQCOLrACry7nJ%@vj)!{!Yxn~Fk|3vM&vALvljG}7UeiUxC*Ps~H@UmjaI0xs zE43jhV-=|23S(bso6Ob5$-Vub(P~O3_ED2ut~3EF&m;)pvNs?EzecvbvC;7c?nk0f zILRrNoJ)5*Fri>FGjOTfz$@xoa(_tN%8uiC^U#OWN$5a!Yv$CJvshlR?5@Wb?kX&3b4@>7L!vi>k1z?AO_xfKuf-Qok z@M}OPvP>aYxe{sCk{jX!=ZcG7tz( zwResFU?P-mY#tP>NqvYk$v3!9&B}pR$lUfgVr}_9E^>*!(sXYH2hI7 z(PuU|5qAE2_%=k9SG&;1h1%sP8*xw(eEm^t&yxfDwUjrm4WH;*_Qx(cs8XQaw{JBQ z5re3zT%Dt_K8`hCO?03@TTT5I%g7VMaa-wkS@GS6J4tQRu!)Vy z)mk_JoL--@0>S7m$*l_0CQa6}XFj}6c9En!Uo#FLj?`R{W~W-I^>;fX z7y$RYQep6$MI7z0*)mGX-_(7>FXmrd;4Amlu~CNa&eTC`?zem=Uwyu}=8@WB+U>t+ z>;(H2QnTZ5p~)p(bdG3M=CwF%=1{S=+CI?8(tXJF%qts;H3b)RUo2$Cqh<4I7C>7K zfWAGrK_R<~7pH&A$Vu_ORUYiI=o|*`(obQ@EO26Qh+Gq|HoX}~5cRkdGG67}nmcUJ zT~;~VIy&7dY4^p_i zbxmfQ^U}pd;8zh>&OpzXQ#jkfz&|(wb^#V0eHwVn>J_*jSbD{@j9WcPi}H zlHmx}@x$ZY^_KR5EwX<#9+V3{rxjH;t+2fo^pPe_=FS6E#=l3GX-a-!^DkdrdsA?c zw#=f?r{&HA*2Rvy!oN;-YWJ@1PZa>_cMA&`)Jlxo8YJUw91eRDQC#2k=H3@_&A*e_ zl%5;|R0F>F6CR!PF4y3{892UfBq)zo4Fp4L0}rBcdp${qFk@d*P0h|yN16nzJr+IE zv@lsQrCImAK@t{KExG&NO0r}p2wOHftH@oy<7;A%W=j&?D|%jIkY{dE1TKXw5KAri zq#yDpJ3##7Y$=Rk-Ten@N*oC_?OE4~)vAkl$okJFc^IM@9R3Lc5VmCLV%3h%3_2h3 z>of-+6Scg+VB&DIQ@QT3Ves4Pfxc4?>cZPzP8Fg9N7#^R(@5@N>RL|qu#=nSIhPJx zf(mRdYjUk-v^wy2Zc$TA5Fs_={%s2Fk-rMmNwOTAt<$Tm=?gdLCSiEUY4w5ysn2xQ zb_xU~UyD3A;1-{>E946P@FuxK!!-GL6tpSb*97w zc#0Mt^aN>E%aNC=-Ai4Q!ZVp*VTxAG?OS5IqejE(Ms@PJAYm6v_u2#1iW?1hmGYQ^ieLSU;X4~2> zD42JspFR7eyl^F>q(U<0$fxR9%%LG@edbt)iSHA_$$sX_ zaf2Ib!l&ZLpslGShwziB*qs9U0K!tEtk!h1&jM=pQG@+gap}>-8(u40IL;kVPnnX} z;YK)~NC&b*NqUr=wMLX!_9mPCLc{R&c`*G3P4spK{K;oLlQ4{s{(y}hasB+>^ZCPS zb3vnVgcT;P*v_RivR+yFVo0Ds3$H2vPy{eB>Mxp?1nxGt68yv99jlw^_C$wCc2J;h zZ?i$3|AYzu(qT_ScRdf21kOj?d^y^=sMfr7yUexF)a&aTR*hpc`V!D;_zYdDEu7Sj zN00X{TAb-}(J(|j|MQ+ZB2aw?6xaESKvOb^*Bt}LTGu7~^g~kX0#!`o$*Ye&$)7#C zMKj$Kj-jKQZoxHZz{N7Q9}JfTI9gV_neEW!JaTakmmUD}-O%_AU8`x{5sQ$q)0q+!>DFYEdKHGWOvkU9S6led@YjSij;pZNRVNVR1X%JbPM zqLz(M#&<4+1=&ZX2-kMXp4jgQyi9+^uf-u6Cu?D0(XXdJJ<#si;J0tDO zIA3YaJ-YD4lnC6^fqZ`t^bFo*+Pk`ZNK4bb^SDm)Q7!1ckny@y)*bC>vIDVB0l6^N zkum3%LV+eT{pNE~;f~Hxmt;k;A92yK#mm*XN0n!zuQ(Yvjlm`laq;HQJL^T4k`Uqn zz8cC6r{c##;ON$5bU$}s#m#DFSuNY}DAMYKxnc<+twDTE<8ENdM6p7I?l4_yR`Fu^ z{pl7oGpq~(4y*rJz>r>Ay=oS&`Ha7ufv4yd+b}^1%qBWl)>3EuC|W&2tS$x--D&WF z!02(u=t#CwF5nE8mE6-1sA(*Ky{`6YP_I5wv;RI=Q^k72;>tva6IaM~RSEIuZsV_0 z(qM!`^sdoN!Yhr-dTb#x!AS<`j$oa?iNyM~5Bs*s%m5Wbr=)^VWTy2JVEmmR6B{gb zz0f=WpSQbILKmKSPq6N_5FlyjWBatk6*IdiQ{_oGHH#Icabyh|o5>DfYkos@0{JwC3ZNtpvcywD<>olLD_qWvuyITDj zW9g!XwQO;N+Jha$(xp36y(g`gULSd!M8l3pWirL%6dxXBhPqrni;#x0y`F>u=^xA9 zf+=8wsDu}cw2no&4)*S4hbfX87e0?ms{n@PWMw`p$#4jE?>veId8e^N1MA}$qeZRiy5AHkBSMAxa=l7;rfCOT3W&8)7C)EFGnI3y2q z3kmI^(yGx;*26+IR~Ry&SW0xq;=w6#*BXunyoY zJ9r80lli>i^p$v$->PCsU0D22&0LZwsH2F5rrgEU%?1>(@{{9{N^n)SSG&t@UzuU@ z_fVRDaYMjG#Shhba>FMkM~>((w8@#~8n=T25P3Db_edPB(R*VWPF~wocwR2l_am06 zax`a9zoQFwya5!f)z2yfpJQ88rt~g-T|npkb;z+E?lJ@o+sY0aRC_p^taL`Q?mI5X zoqZaD6l|DvZ)G(lf73N;!&f?lHtr}|3grBz`pOD*K8;?>nUTjnYAKG5^H|gz?GT#z z7KZH|Cr0S7}kEel<~jaAu9pW5@qTA>C|J28)Us3bI~ zQQ3jctz6Ol8MO`xio>5^Ag?@bAL1nZ>#Z}+@Vv~#J}AwE+pv(2@I);mKSSJmv?cTS zkn{`Dvs^HOVRvr<`R_q|j&vv#A?l?Oz(vE?;A&>I(&rRHx`$h+Yj-&3fveS#L&EjbQut5(>Ed4v5Km{2Y$6rbbZcx6(63h{7fG?3hC?ys{x@&g-AgDI z{Ir}BCblzPY<2+?BXdCM9Wa-A#AGgZ`VM{6)0N6^^o;oH1|<32e)-ZS7i_|6t{3f6 z51VFio78c;9A!B=Ft|x@#yje_O*9BOHQ;Ck(p*m0H31U?yqRv=DD};>&&wo)w9%lNk+k6QHT(i>RQKX4by7qrh4ZYNga?It~V{}pA;S%t*M4Pc)OhuOiFf> z8ILoa20EBpExB(D5X)A4H5wHLOIcIasV{0YZKqJ0&>k4)HKu97lg~-B1wE zlzU20=O3={#QqQx{@GGqS64klrXU`%%-bI6GOB<6GCldg%FoLIqe-{P2!-|1^P6Sg zhFt`kmu57XVFt*2oL||zy41{2rwfmU2ZWSEZ3WCuSN-`xE&v~pZAy_}aw-!N{N$>A; zcnT57sLx>UY%W62+4YUXu|A82!te`NTMXynSS)KSr2l6T!+9xJMJIoLyw zP_^4U$#V&1{c`7`I92Y`y`EveZ1d8M$;Pv1Xk8rutix^yYu2%Q8qpFb)5_sIFQ($@ zr|?`0W`$uV-h|!p=HXB7RW4PjcJ_yZLUdA}nVs_6E7lb8$w3oq*<#_`ZD@TvowVw{ z?=Mnn27_>JlQ%Y3DMp<}q0sG?{K(Y+qws;A=PrFi~E!-s?mJVOlPk(kNzx877{FGt2y;hmxQnn<(>4C-waF=5BoB&nl5SB(wFY1q(D?T99 zYHbF;nTXrOOxmwnhi{NZngOL7 zX%&zT=|;MS?nY$jM!G?|r6h-L7?1`5=^nbfd*&Ixzvq8l`_;UebJmXaiM7{hBJ(V| z^Stsk1t81ns3ehlJ8RS`opedQ-( z?bJrATOZtgSWc_>buaMylZjVhF0_f3*dDk|^k1q_R($)9N=cPuwc5>f#&m#R{qo3f z!tXBNt>bfLN@wJNr6_*~A@k%zRi6wr@F`^AZxes>?b_8FX&U ze7q?<(iC_I?rB=15^(_Mw^oiKK(b}ouM;QYVRH{7bM!oyT0X)ZyD?eHH+mm+Ah$wi zheF5M$?`~MY;5yNf}LnXysY1RFP4i_ifsj_wQb)rNp6OQt9h*u7~^33Fv%^fB0 zi}qz^9n9;B$l!W1;wPQsJBRB$@0ZThf-@dY2hfV?9I#M1wc-MPHw~Uyq9*ScPY8W< z+L`-0JS}&!{-xPm=;-nh5UR1#h6_k5B`^4(q5)~*e-t+; zq;7xaK7LhEaMpZ+ZcLGFIU?%(t;e=$y-L-wk)#>Elds`8Dv4V|_V^+8XhHvG*`lJt z)^h$)$zWMstV#b_zKLTFHt%<*YG+PQ-OP}O$VtW`0;Bs11eK?e2Tt^V04u#!ogTsqN& z8l6f4YH1k{MFiD$7bNTylUn zykbpRR>yc_%Km{RPxreoaIc&X{S)fvBg$-_-0w%-2a5TYa}Jf3ElbBg*^jR^d5`J) zNKKpHD~H~8bEFyE{UJK3aF$(8vSf>@tE!TD&3C!iLjR~gr?scwVmlIryZ--7Z3ZQi zt&!O#*W;x2afMKyTP|!Yws-RCTCEDr>_X1n)iPxs3$(0bGd)rZ3TC?t9bi|^UO56m z-$F~i+V34dLp8KZZ?p^AJt>pL^(hgdNtz2^9rM@o($crG+Qth>zhwG=oT4bs?Mj6I zs|i`QV)u;2s3iS4(y)T0K5K7>yMOarQ=bt!AK4DA^xeH2ad@*%rc(bNU6HcpB3*W{ zVFB_3JkIpP{+A!bUY)iFW}}p1OFRtShqp`fQ%7@LzZNek#@5_Y^j1%;5H9yjBM5tS zVrm9SDC|LUMw$N8M!c<2siNPQk70pn++3j^{aKYWx4ovT|AAD>7{?p2jrcMn>yRV% z3v8#c%(a`N&wS;5AUlhD_V;AoE6)KWi<&RxHTHwIYO2dSGQV3AKm5}vR@*!pb!Bo| z68WpC9C^4ro_>dFN{(($AM^ikGzstKnB%q%}gxpT= zXV^>PaXs1daXo9^sisQAQ{PaPw&4JBXaAn5H+dMhV_*{8KWWzk zs55yXU#@6#6BF2ni;z)Eu0M<#*fxduqidHsqLO^qkfpErtdR^8dh~F9N!by5M#wl~ zZZDT~RQeTKSHCCOP$}7O+T9}3MN4v^WSpJYxCfO171MToZk~07;Hmh)Ll7%;gV48H zI5u6c?2HT4WNX=Y)0^mud1RG2CTz53&GWl(I>J@J(w2Gg*I9kVwMb2_e}hIvD6A^x zqo+O6LHnL}6?|I!h+@u$*nh0yZL_G^Rj$vpJK1f-?Cd)ykC?D%pXs(%wNC*e4^e!@ z%i7G82cRV>z369bA`uj62)(no0L9ODurW8U?E(*5r1fYC=sN= z+^YOS&nY149Ww|6Y3d-O7wN1@Jj%i~>HvDwT)O{@&~*bc#U z`!lU>>z1kvQoWVR8Ex_>I}(G*9|+$_6R{YUpM9fbR91KpJ+fZ!@|l~yQ$f5Ru!{M9 zev4~)PM@zte&cRz9k+TQGi)KnG)d;q9_+@WsB(d{UVI5j8(cjHdaxDOPZq%Pz%sOT4Mt zkf2$I>FTzRU(Q%L&$rSsp*+@!KhWKY$kD{%`aB)`bKYxjXMs+Huh8o>1bKRX| z=S`R3V`i_P?Qn zE8*9lyt%aRuqo?#acGa*R4p!dHB;*dI8r#&TfqHAmR9dnbnh4L?Vx4Vz1e z;ab<&r=_x-ozdig05ysVew7bP>-v4-jfwXNpMc}A6}+*UdP-3PdTkhDoYs}yP#OR7;0|+BtPM210VN=aQ6qVS0|a# zDpw-@oh22m7uC!sIq|ak$pbi*?p6AS?=wtap;I_`*qiRk;H>I2Pf1pjbc^o%roFw1 zH*chxaW^U*?diOmdOBgM(EE0kmsw!%6*~G*hOl=tGMhrbw4z!ar9quu8nzm``Dj6? z9kv-I<8%8d#vQ!6sg`gD@)b8l0*OrM9gh3qHmI5h>O*Q@x8_7p`YkE}fDpFY#0ztU z^xFIx-m4yU&P-%zce^P1-Mr#?#tPsKL^%nY5vvsWnk^Ziq~6UHCB z`LZ(IrS&Ir$;(S$HLCCijjaW1R9VfS2lZiO7dxP_w5gDg;;Q+oOoAHNHQnHEvVZLd z+sAm|EOW0!zwg=c2>I!A);c#fN?cD&&e|peFOJImvy9D#ofiPS7hLHjTH2TsT@(Zc z=bxN{_ne9I)SgxUy5G(4_MyX9hGu8`*VhH@^iiGP?X({0@{3Qt>>LZCjF64!f7b3?gka=TynRqIv-cIzyiI47C6BwGCkje=N%uo3wY{<5 zQ~e2OA;Z1r?cidZzGkCZ@8c(pqsN#Tvx+m{k5i|Vu8}vZ6E7;kf#Qc)9oM_ew|^?A zJpn5%^7OPkjzz}0mAoWq*`P*8aO)xDY5%SLY+XhgqFqXw3WQ+r#ocu0hexp?-S~kz+lm_(sb<=z}RITA}V-_;o6pwD6-%Y2X_tuO1<|H~m z2%U?yH1H_f=b%b;-=!=&t@Kavh)(HAdF1p=>&I#n;!(ar$1T@8X50IO>H~T5)2YV% z5b5X@J~ifO`)($hF8~xfr^!Bd-vfW=ks20+d#?d*UpC-oR)wcl8Q17?{C6<7`Ri3x zNSAC|!*>R(qGCgf9EZJU)WSGL`_{c+hTO z5=hSnp8O_>s{XvfYZOqjp!GrZ190eXTql59bEWM3LPG>rv@fI9687i^SQS_?9^*Xo zAx@@T8G)Q~_cV6k>OAXSSoN>tAAxalXM0HBApm(h;%KV-?m4n6I121E(iZ+Q3DWJT zZjqS;`B!SZ#ud5QRF!Gr*|SS)1IG`*Q69>@TUYMSNWuH>~c0J{%CsSz*|o+WFY$ zICm1bb<&M&`YILY%STzN1CdOQ?ayVJplq_dHi+b0fzh} zO9iZFF`qr8H`0xr7u@YE;;fX?Qsvh_X(H77gHwJkVBbRV0RUa`n!uf)$C*3g+_Jq?HPgM7s+|+Yh6<@T~ZXP;G}V;rqQw{zX0_E`*v^#9_a@FkJoES{ z@yD#mEkk+fwzC9u8uQ^`OAT2EYUwT%y#OK9`73^xVZSmtIX}jP#7RNe|(N zt}Ql=cJbjBvsdYIH20X-C!jVPD>@{?bCrA%?I2tCw9PHEBAq*xI;*WzLe#W7u_uLU zO83vT=^U&sMv;8MYw+U$QA|!Eqv!b7$KPcB!k(0s6cTT;n&p#;$Q8Ji{-T;G$cvzn z2c(7F9O;@h@R27cX;fYCh!%A5ID028mlFe5%rNhB7Fh-vEN`srZ=$f+<;AK1+=|qU z!lf?HR%R3X7X2oZ;&+p}V?V!6BA)UGhRy4=N76%~+{}#fba8W_1>v7Qz}=gR)wOan zg9Q=H9~u9lrhP2j3Yv}S9)1H>nLE4r=ma1-mFyyEW$2Mc#Q@N`nXPFog$9!+1W3#c ze{x7NKsY&(jb8aPa~fS5gPm}wATwQsr<$fiPxbRPj_8<7 z7xshnw#X3bw7c(Hb(iq034w)Dv&wfmAIRA2*$G=qC?jK?- zw*In#WdZoD$VH<`J^q|x2|^4C839{UHh<*^yAq`Ul8{$OJ|sP|NLta<4v}ztsmqMPO|!S@dq$)H z^RCD99f=Bk8CCCo!tqaBFEi!y@MEVs>l2;J5rg`3A6-u4j#8?m4CGL8CLk@N9@6Q& ze$Z*W{{jWRLmzIt#sP1=M3wz#r zMp)7Xt<#G4FnW#yII8rK4@BL;0VvNRK`)$3nn6E>Tx+m6-y7c2R^(ssT&@n?(Hxb> zJtc}sXeXMGyGj=gi}DD*0lCSDOVb!9VVOR*e<>)b9PSGm>FL3+mGpFysYwty^x*N1|S^1Li==);5(J7xAEIw!4 z`bItf$m*0f$%JX0-KC7T6Y_!LPjAu5Y;<=f4kWVehhV41hR5gAwgXpz0#lKLguN)G zt?oV<*_}Zr76tjOo`u>>b1|ks)!X+$S*fX<&H>l0&-c>jFR`s2*H<4etB=%CRic-1z+Bbz| zG}zgw(67SRO%WFpkXo$651v_LqM7#Xk+MMO6Z)>#%(p_8_Bx8tZFikUIPZ888QNys zGtZxhqX&)IrQcBbZVzN)ps#g*Cc`>5l3~@lGx{ZD4|MEIJD7H%DV5V}<3@LRHPq&6 zLtIx)*ox)#`m%nARqCF5XN{? zd@FltnO*G<1Rb@zyO1xT)VVtlaf>KBsC^)GSD#Ko&q}!?8rJUFxk);zmpQ;F@(tgY zA5tp#-2DeL_NxnVib8ir28WdjeQmgp7y8(QG16wrnbnZ{; zSmsb{FITEO7bJF;1+hcnRl5C`1MT0~o2ourIynWX+n7K32+tw)75t;2gbN597w#-{c<>`dCC!u51~R_fWi&aG zBCpW5bYih9YRm1yB#ne-Ol~K0^!HF(&gd>W+vRl=s+g_>#f`DXkb_4z;P`n%iC;HBr_PoDRAl!>~^`E}_0$u>_R{ zoXBk-9~Mg1Xsc=dH%~eWpEl1t6`L}78PGnT@S;Umn?xSXN*^!iR3)peAL`?s-CvwF z?&V2-qH>J)N0M=Q{Guw7$&E@8B-wJNNibP5jjhBWjvS;E7_3N>N4X?}+=}8h8&g8$ z%13)=eA9bLdr7wqP6W#if}*7Z}7pBacH#(q1J{sgdn>~J2e(H=&b0uKRz zhg$`Z@%4XOYMtTXwT0e=?&)`do~nKMOmAw) zej(bPT&_+TTkK1}swChNUrD76dDP3B2w!(SBbpffK6u*Db++muertLoq1y6}O>yb_ zJx8&jz>oAFba{W^GFJZ2p+!`x$#E_sb4Wrx$sKN?*QnjLsQn$8~Qf+e)gPbnM9sI)f_gmH(C1rkUUJxC+GD@fsl}P^9cRbk!=mWp^_R9 zpS`@gA^p0om`H)ax#7h%fLfjdYsTiH-2*!H-RYjfOBU%cHD4pac&p!3JEWjI%jc^Z znftsf+WglPs?)_WZ&p!584p|x#y$mZlUf)?s%LKNv#4LEx7V5r)?u+ddg0dHl`?6X z&>;m74ME!tvWFu*^*E-YRyWUX7peXaSsY+mz+hRRFS>nPX7xTKoC<5M2l0Q0Ulg4^ zbzwKxaspcox1zlH5>YB6rKQ$#v1F)BJXdkrwbDA>JiqpgTps)e^IRD|$hW!k&z8cF z%aWqxWiYXqK$h`Q#HgWB(c>bcu0*?pz|`j!g%XjM2;PQiq2?6Bh{3OK*kTUnBGa5} z+ICjDM6mkZSa4pkQN?go=v|Q+LcFqoZYLoTLfl`hp6!9OwN^*BtHlSIAU}qLWN%=S zj`KKdLOp{U0-@#d(We{mrO%^Pg~r5EBmBo2@Oxa>N9uxv1WgAevwqM z`7*y{m$Ab(uUP6{wGzb}YJk<@e4@{nY&_?1X_p>FT$P!}Zi)&!HS%?GZap4Iocl^1 z@M4&m<7?Js@DX5xb}Jy+753NlCgJL8vpIXG5zjf4jL2$ifCE~-on~oisR%Fd7=h&Z z0K<&konZd{=UhRRy^%>BuJkA*Ef~G5soN>{IG;PNQt8NM>$R(YwfsEo`MK0eh22Tn z|M7ou1R(E{*NwGI?9Z*XlTotv?(j=Q{by|?ap>myo>rk7WL?F^xWZ^?s`fl9;lUeX zo)^V=7*)Z0*`~mwhf0c~mGC%}j>___Jd{390*2)E#ufJ?5vC?)(4VS%y*-QQP9w?h zX11D)pK3Jz{T)rvZ`>Ag8r&i{@Z@D%Vl#R4Nw*$zd4?Vn zu%Cukz$EPe;O>jFeAf}XM-}X0-rv^MOwvOd2?EB}-s!l14FCCT=N%DW>oV+mFlFRy z372Pmzq8G6`7|z)TI1Ri$PH)dV}9u}nf%ylckWfUAD(h~*$4bu@`aHaVq-iFd#W7- z$f0JbT~sh>v!uv4^c-+Cc$b>^JZCG?5OLn|+#={E**0t;Y=zXMTdWH_QY`G}oOb!rzhKnJOMc=!~*tpbYBQko{{2!kV zqSD)`eY9oP#3bxyA7yzE9GfZ6|03tl;{-QV*7H-hDFV3g`~5vq__3=@J_>tvkX8oA zFcu(Aci@6)joLi_-yJ3!qn|8h{KtDfoue5mFE(A8n(Grzf*t-nRL2eL|E*CoWS@e4 zywHV9Mf~`nlH@=@U&b2*_d`zWwtQd3hM;DWL#I>FjvQu__EE!72y@-(qXtzC=r?@@k1QIuNnMphE{b_wRKdULfI#p~+pE$CM>h5qKODOmi&3%R@A%QF;GWOKFQSdv?ySB#bjzW!(n) z+`TziEsV}yRCY#PEqy_cmmAf{Y~gx+-pQ)c`FLFT(PY8Uv$TgPe#OtP%cK#xdLRt? z{41v8O3*ew|4KT!4e)e^epRW(Nx9s_u33p39mSww>|@L+JlH8zYt!;|bm-k;eN@{! zL~)?(&NJItz4p(Twmt7jzsy?`&?IFc(`eN^?f`2sY^qaHNFJff;O(*3tYjSA^Qera zt+A&}CL1+sLx?{T8@G8lI>&xCzi4ns^`YW5Qr7$4{?ie?*Mri2smU;@f#P3k)Vqwf zfSnwuad=1_&heI;v$5Sq0YY|;oe0D7@BRT3#CSkSRnWf$$n@rV+9yP!1ZRosx@4&y z$t4-=sp z3<#-Y6LFHrjXzfOCQi}h_nwrr3|QDjd_EyIy$#t1@L*~&0k-tEqxFCs`DRCfN4hoC zzaAJ_0Mg^U%KKgzzK|ei1UL{8T^Klgs3n|hIX5V?w2dG9N6i1j{E;Vq3tcrUp(HJ| z`thCDE(XIHk_MUVAb~!bbgjJH*FT(CTJH^2PCx*IGkjHkf@%Tx&1aPp;IVt9T`nbj z!(tbSQf!7B-As$U?QQC#$MeO2ycAQ?JafOYWJqII{rwCHQv z=TvNtSaMgVJgUoax6aU;Y`4(Sj8#GJYW#J^Lv4rI`LkhVyqGj=8X3@uXZi6;r)nIf zYO-xI)y00U{eMi0;J}6Aulg&+*pGQE$uM5YRQTHRX_TeO8F%HkrTg!XMutrXJtC@S z@7PGX=;^7_X_6;*ZF5voUb%b&-ODA#P*K#4(;)!w&=wj@ zRN8$o!u`07pc$AC8h?<`-UV@)%)4#Z4zV~nI7Im6(R;{f*IH87l@^_XuQ3gn&(IRZ@D!7LA4M&ZNIgt{5(lhOQE{FuW-Qh!zc}>?f;+k`v}> zj?zVnFPC=93|GOo&|u-WdN06Vl#4^B>fFc2Vb8XSzXA8BQeCeeIJSAVf67Z2za4r> zh^MJ|-!Jv=6i!ZJSoCq%ca+XvDp=}pJS6pW`2IgNDk3mai!r0U$)HNM!GT@|YWFGt z<1;(rgF8iXs;M!8%b)#o zxta@6eXu$v#@4l!1`8@A=bWKd{Y8_0(4+xhfvzj`c&%I+cFM8bq};9CQV)E!j;YY{nIEI>>f~l zrKXo!gQX@rZB9~~egp85$asvBykWC6A;AhXs^_wo^{Yhsb?R~LB;j&Z2q1*9T1O{R zq%e7*<}mty@|#>YITAM|)%DrO&GGbuDw0t-ngxt=T$+XPHzzm%WG-_WK3e0TXXovm zA@YP}`a799X7ZdL^C={(+MbHT?`8yXEpmfWdT zG&(LRe|$a1ijYe1)|k|s79V3V{O5063(7UgAwHUbunr6_zPhD(jB`hZK3V zy3%dDW#PsuiQ(2E){Yp7xo;Opb&q&jXph8Id;7u`BC_ELpc@sYI<}MF;gLgI36SCC z>fhSd)S7*V;g#mkfL83fTyqgV1Tu6&KHQ}HHY!t*bIvr@M@6J_`e{-8oRGgBF8y)$ zB7FPz)r~9FJ(t*H$lc6|{o6yVaow39a*H7?zR_`GkauX= zZ;bTaJTpN8aB+B-=BN}htuzgLkxXw-g7`W9-&vftC<*3W5J0+Ten!dHI9{0I!P>%j z%2RVHg-|1Ljmxzg?;eWo0TC9|`ti%IGlD*A(@4 zT?!}l&{^w9o0O5Yq-iS41+5?q`2V5o_MjD_2n`ef>%q1WV1Sw`^DXFR_|?#(LVg!n z^D8~$LOBUeM|IP65eoqS5-l@*g-bnVtI6S)^XRk*xWwY%kYR!W4LtIhzHZ4dv#3I= zWM?P2b~XdekmYB6u0f(!_XBn-&#=z2f)5O9>7296W^Peigl{adnVxMT=~!!W-*#Ik zQB`)1xlVJNV~8zL*+NIXN4dUKGxGzx3_asZiQx+co3!ElhZVHtD!5^IK*<%nS zs3-l^k!lB;tx>CYj_(7+dArI(^hfw zC>>rt^S5YKvvnUCt~1>NPnAguWx#$j*sdyYh5QdQeDOxs3s`qYPv*NtV~(I?1OD?t zSkuJOke8U@b)wFwqh#`i;O1?pMyv*z0kWd7c1_=%kfa@t?e@!msbLgBjVqf?Sw&s>4K$#xuJ40w`D-%Q*h<(1d1Q1n#|%2;e@=ruO7@^S!_!%( zC*|*i2&*cE@^M!LoE5{DTf1ro_(-pYFv417rkYN!p{y{!Il;!aD5(gh^FOusMGzWq zYCA;q>)1yJ_0axbC?E2SBXIFA^p#pyBaUE35x5DxbV0rdIrmcS{MG*yi2pEAn1hZ( z0u;w-D+dP3QzNt;U%BG*t+_PX%(;z4vMtB)giqyDK->zf9$8d}er zE7kZ!&K$w*==b{h6(+eDc7_GKM5`wgb^4cuM-UoU_+W_=z8l-K4P|H*Zu|I@hx z`&j=E0yE(MLQsgqk(JvDOCzc1IWx)UR1=kxR2duQQ;mJ8UA?f^%BNRH!3&$Lh*rW2@ltI~r z;f#A2BqK{Fgp-;g$sZFD5I8L`5>Pq#$$0{V7xso-mPlLflUW@?o+L{|NHJB2MWa*u z>#rFYuyUse9T=?}VO$6M%unf49wSalN^tezV%?30K$OosXU>vhl|E|kdTJBz9M}Rm zdMEugpUQCwR5JY^^vhpY=TPBBm;Xl(b{I?^XB&Y6u^Stxm=;cp3j=@Tob95cR9=hK z4o`&Ap#Um%mnIe?aa`o)r>L|iZ+Q1#YRw0+LG`v*Bzm`3SgkiD_8dV{YwkJl9f`M{ z_b`bYpfQ^9M>)>+OC~S#N8))-@Un={-LSyR_^p0vITn1Se%`gd$c;e^F*mTo z5g>cW~dEXnn^COM2ihEpkt-ThBnqyi26#!l@{9B_o-Y;ItI|*TQTBJ{E|}Jp{Cf#l3@)XKGI>21XXtumh&LRi zWxu*q%x3zCN5&K{;tOVfJmgkuCeC!L?8O(ezH&COSRslh7QB6l7frT>`zxk$!9I!& zj3KoIw7cEod_3%Fw_I3{#-8=KV_Y}}@u9^(b%`DknzvYZvs-^|q)!kGcy6R?y6WeF z3wD_K%oC}$ch#uBrrV`EVE^_OGnu@vVJf%9=iE{F3@@K^X1a|84)(sk1m#)J*=+JucsSu*Mc8k!V(B~S`QAO^^%EY&kL0Xu z%?4f9^=gX=C}dkC;`LRRF3mSl-(|;6Gj#3CxF=Z=7Cs{(U~a~_ML}*BpFL|doX$1v0*~Q~0&

    @WQhvuP^-_s2(LV_2!Du-XkK@q%Z$lncIvi9$}oWjOq3ds`glm4D9h1Ah=iM zC2{NNNFi2KRlH57HwWs@G9_+kgWw5pAb;u~R5$wv8GW6zWa`26jWs**U~d$rIaakr zK)lVPQ%07nnuTi;!}KB|{DVF`?ZHR*E9Wpk9FnXp1uOn`aQ%3JLFMvmU5~T{W$lgp z$ch+kA_4mD!GrjX8Z5uz0W~q<0OJ!hR19^A`Ui-be_xyC<;D9slg~rGOI}o6R?!8P zQTDcwdJ%%-x^(*I5ywD7J3D$vHl(7?qadSlqyt)M=lrnH1a*E`LVW3^A$Ej|5M-@y ziNuzRFIAv;n;p&L_2P}0z?I%Fa{bRMnP7_*>n&^?OWr#&L#M8lD;G#c0W#E4;1(88 z9_O#@iKWga70Om3{IbELjjTXrn6N27Q{zW>Evs=zLuH~MNVtxz?vr@h-Ym(O=qbYX#7mF{TZz#_z)GUN zGXJ!ZJC6X3y8|H7#UuUq#+u}^k~5Ty?WKOIRT&n|Wtyt2&=7ly?y1U+CG?<&m4zJ@OI>X2>sG7(4QZAs(Rl zyV%C02-7}mk?l?o30ZIs+tkaFU4p1Lx12p4qWR%(!TSyHjo83gcxEzUpa4DgS5Hv~JW*Z8w&w-^vh|@`8KR2*lp?01#qw|+A*giFMXtsZ zLWy$;T4a=~em#j`ovfCNd}Fj{<2hGOu0O>)ajjzDtKm=1PMsd}^5GIP=QNVnB+x_I zC{G$;w6ps9-SF^FrXw;GX^W-0E;~X?5bre7O|HT4G|gPWO_<5@23Kd37J;|->+SQj ztV=g>$Mp=5 zNoi`>cZD9pc|^}w*a4DzzGrhz4l!=VGc*OpPdeg;cWy)E$HTX~aPHAnsqJ3V0S82- zjF0)-Kd(7DaOS`#Vnq2d(=P-Co{tl8_mM|GT1DbTK6pTBV`q7GpQpYGv&`TxkeDw* z2x!jhb=DaPb0eH8hat~HVe#G{<16rLUaEl+jbM;1pN;7y9ge6?7tDn5>8Q8b zVpez-zPYH^Im=63ABQ{MswdguhB+h2>N{o4;!{^E1V##u(YWAg#pA9T8ZCSta1c&^ ze{oQ3FmvEQ>!i%N#hsI9jBCwUWk4Ubt&cCBn_QSg!#ZUfSMy;Gx28PW)Og45KTLT6 ztE!Gy-Nx7TTqB>o2we7rCAm1-;N@0ZS<^)(LVsB#Od88_+|=+CUpYG;J!F?2GUi>? zn!&R!QRt`og_JiimdtG6TGR=jOdHV*r6!t+sUuvkySZJFJ<$gDKvG(I&LL3n;e5&f zmmXh?_8bA)zwIHj5Wj9D*;;pP{mD>2F|0a8Joll-5t?g!mU_>R$9g2A(}(w#1^xd% zU*0N*m&)Quax(swRXyJ(EcC+I&fDgPy&HD{{{~f5rD) z(<=0&rY%`c`oZ>#$b7-)*G@7QB?VU5{N(N_^ENSgdYq6QM9XT%w7Q4=S>U8<1NPH< z$$<<-^7m9VNpKuK`6MbUQS@$RmGrpW`w40eNXi@-(?u^tAGE$q3(~)1`=(eLqs zcbh8&qwh1$;o~Ix>(JLcQv_hsPkI*u=bR^9GjI~)#XS0#$cH?v36Gda__1#v0%EBS zW2n0iS68^s(R;V@wIa=Dtv`0>f2m!(L|S}yWZ<}au(eaI(k@y+)gB!99t9lhFLTH( z`Nu|rD#u*XTUBok6ZCYH0n{?#M-dQQ0;~34yf97IP5F5Dt99O_4|)kB|JUCm@wvU+ zxG1hcWalftQXv2o8-E6i5!jM{JR1#-OZytnu1H~6`)t*Rjs7}DrVETv7ErbPKRs90 z>%iAMyVhfjOo_`XJ^ghGJa8@IMKXcA`u}zjEdwU<$BoXqJ(X(fF>shT^JFN3OCbsq zU-9E!3!@+d&HIDsRBUMP^wu3+i%+gL(x5As z?0@60yf{-l?z*+z(w;Hm@84`l1!;Yt@}XnzVY2)U38R!{U#qy{!a*f{)Q<H_4Z`vhCw2T?3)qqUAvV{$)J zE`nXHh?=TSp$7kF_JjZQ5k})$?t&HI@b+)&7&j;pDCj1Vu83c`;vq2?Jnp1MUX~GE zo4P5{Sru}&z(O!VXKgb#$ee(dfgva-<_SXQN^Mex(8Fb(i;I40{`HC_-qPx%>z@!D zl6jcOV=W2b=&-37&$T(Jm1ybH+H=1OG+?}zr%%wvF8`eiphzUh9b$n~bx1l&B}?FQ z>nbP&iJQjW_K1Akjy;P|mtF~|ZpvlhN2GK55dBgK6=1uS^R!scXHX3C4>NYz3io}w zJ&2P_7Ma%P_0S=vKj!P-uS)@1)s!s33~jy~2&tzgJm8tpBN1I5w?Onj{cPyW`70N` z=S4I^JFALVmlp-~=bv1b8J#q*`g!a`*fhxVmdF+Sv8nlH8vesw6qQB%PDujQdVWOZ(8q zAu+}#64ntTG0y0oZ0NY4XW@O2G%dk`(IG_qd7P`Xv2;SRQ_crOxAK^B&uFH<p&6c9}&Ea4tS9yRY1t$47lTEi#U#kz`zKBxhpt9%g@en2@b&&wjpzXjq& z%X`Uv#^rT9wm-MGorbjRh>cGvbQ*Wn+wGaNV`oQ73KM%^(u`7_%!^))vhAwS2yv|#PccYXmok}1ks7A&bMFjU z4RJHpwWfz`Vh`;bH*jdQjwN5k!&JLYK_nrvolV=hzlyWU&~0v@aO6flzN8~e)%(lb zw6$&J9lI|}TWYM=MI=k|)RDOn0TEX&W}#(;OBKIegfW>XDwe7C`nMQ{4a}Xt+n40s zIqx%KgkyXg8&dVqt~<@E)Y8cCh5@2<{iE%HtZ8$s-jTRrPUpJatyS#G)LKr^4c{@WH*HwF`DQl1eDK<@ zn|CXcv%&`mR(xl|bqvE(ENzh%37xP|a&J$`= zBBtK2jwT~xscXEZLz8OKjtel|?%whntkBdy*6pR8C8(}RO`eYYq2CvbS^9*S&0yzKyf#TO zYM^}9MI@0RhUv*(ap*^Tlvrcm!cG?z*YteYm>S!bWjqMGs}@i}a?2Ip=+#6myF1nV zxQpYjd%W`)xS*?RfNdiik+w0ipzm%byN-{9(=LM=mvB|QFg;}PGJyQV#Aq+laV`7s za^2SK*zr+>p=yxZMC+1d#^Qf}P_*75FoA>8NuK7&O zF3vwRz}F?-SnU)C2qaxO>2jES^~epaDcLk#sDr`|wu4qXIYfF#@}|k&%^a8LaTl-4 zO%viUgZP!kn;0Q*ma!;NdEJq(q9)9dl{$mgQKBlfW?R;giCpR_4whHm5T*6P&igpy z+8wK@#tb;BscqSHU|wBH-Pd1RNZQr9%Oj9mIoMtyOQDLo|2SfFbn%GDUYc&gftDat z(|i7K?xsr~j$%Dzvb=}xr@F8!W8A_zdF(NoJlJn#ssoDOK;^G&#Id=LSL%=kE41vK zL<&LLTAMPTZ5c+uNf*p*V{BG*P+mIVWqOA@&{exut|A$z@R<&98|wJ`QpbPkoh)!{ z#`e-4UJ6Kq)@>VVgrZzARh#F3UCTCi2%7`e@i9O29+2{8`!?+)&h2i;nA7_`0|o!TsXT zK&h-=74Hh8cWo@*OQA@rTUme_O3sA@qBk&U6Kt;@D3k9SRW;)wPD7sI)S3UNnW{3; zmmnrSYqSryJyTpoX%3EMsUES3WmXfB$IRrcsNVMcyR$x(Ils8PN^3fF>$NV=tP@09 zbXqDR(_eXD8bFVIUprzTMGxulQ3UQ{=E4@4E-@HMHC~4wP2ECcwemU9c|dMpB){G2 zqn$jsJJP0>`uUOXdL<)y@*!KpPx0H`an95w{4hd=@hxwn#~mz*YMe-s1D8+sY8gL8 z0@=4WaHKeNMXof~N1;E#v9gPLo3v&r6DA zh_#C=P#nw`C@@|-^meVsPAJYe`o(Nj=ek&gBCC;9gNMP`J9fxV1WF=n&PjjI?=2|{ z@A5(y);Fod)`0uZQ$5s3uW#m;8VoX%%h(3tWn$e(A@7Cg^4voBA(z$ta=LkE3 zSy}X!ei9@V*g}5oKYd*?IcktU4(~qEd+D)_e(`X^sB;`w=QP1FLtLS-jE)h& z-PZ?=?5aVxtM;CXIa_RY1n70-u@l9K0f>> z+xs^Wqb{`nf2ufe$=?ZF?S76#O)7usFLMBF`)?^G2-L;?D+Q$9#OH=9dKTuqCn3J0 zFMM#T<8)k(2|EKuSSc~{4)_@iLRXDb-{nNpUY3uSkk{zvkBZ;uc6c3hSKt0a-+qhr zLij4GwMA!@{oQ&U{CX7`r@Y3*V8`?K_oZHc_FQc4gK1;qC5l%^z-=sA zZgop6d${}Ulwth%kkp72k=mC9G&lg?3`Xv^cjVO#e$m#*k9#);C;z&)`r-QjN8NiyHJP<<-IzkkX7Mehaln_IJ03q<~fX>W)|KIh# z&zJXE>&cf{<2cE6?Q-tDkMnmPd$hI#elGMw*z^b7dWnL~<;lb~iKa5c&M(>4jun!t zZn$-7IR!Khg5K5(^bs=+DWH$2t>m2Sz)7!se**-(VgU{N16!>DilP5euUS6|K1cri zyW_V0;Q#oaq+^b5!hbbdQBs@I^VW0RT{sPn&UFOZLPUSpBG2yffwoGfRdu7>!j;32J_{_~e-fhgHmFR_0AcX5x0Vz5jmMA6U8hgY=@(#*kw@ z|0=3B#ME&>Ae3pK=-R2Nc<0Lr7i;fDVz@A-ZF6yPB3$!^0?%Qm&ZAGgB7*V1agGI!5r z1P~m}vag;@3t?r^zP!8`gdBRTqV@3elm-(~Q~-8)@^(3~KQleIlGvv!gu;e@iv!#4 za$Zf>EgGT^Nf+L%iR1 z4k^DMa4~*s4)V7qHeTVs*+zhiD8}0+Y4lE6w(+!0ex*inN>v~>+R8f+vDx}QjmuQv zJt0u@CM{_pE_9|w@%Q#{+qU~UtfQ_IcQAG3-0)h(R6DSPARgfPOB*w z24~;DpAKyQzP#Q&b?$tjQT|&ItxQ09e4etD0&*@TL|f@m>_L_LXVXg8V z3Rk)8Dsg!F?f1(` znV3=bj4i?tJi^qsU>+76Js-O9W&H4@WW~aun{G&*7KxNfA=5{?5ArB<18}!yx`Z`r z(_oW1$^fBzrkkR8`=2rN?3yk}7oGhv;4fuoqtxkWX{5X-QM7Y zA2y!P)g4t4y9nf&4v=gU_+eO+bL-jnxfxye8 zlom9G4y#!_H(rqb_ZT=`Fvx}s&_EEl53Kj7crMkf0en?F(Ru^KnhtvC$ew8F(qsk5 zbVO_nG+A-j=2cqKn%|4fBUK)~O93Fa86Vy}<1_$C@?q=JFJcFd9&c-OeO?1A?0J;P zX$b?sulm~CPwzUAl+g7nX#ffc`t(BK%g7p8;+H}IRH&^(c~Juj7xzmWkG=)6NGSP*8SC9Y zei`g_;j#{;w^z*oSiNMx1!)7%u`MJ788k1;ruO`p;JlsT#pM?!_N(k-+XX(;g8R?? zjJ~-OY&-rs86$aE9@E^C{i0)KNcL1z&l5iQTZ(f{)u{8(Z}2L8l$`}$^=Jc`#j(QA zqND`3KIwh0$%{2=L%wG^(mKk2ya`ANX_1_i3=BqtDAM~*(Q!D4^5O%u?4Mqb198Si zQAg2W=kCJ?Qk%oZa+*@>*I110`t4@3v?)_FAjKuA=S-J0iK=FC;pA&aEBNQHT<&a0 zF#ZL|*zc2el&QQBU{!0?;cc3+&A(e#8w~blTq)3zD9bXtc}r`%jwPkF{3Qjj{}<$C z<6i>sw!7!wfvvlk;DO_E&F}|r`nHj5Q-ar(JFND|!E|q%nHk6R0pQ*gDg?5=0NObh zGu=BBnD04)OU02ptQNAvJeTkt<~fs$P^tE5X(Bqf8Jb% zaa*TM7P9NWETz;CG<3afFeOK^&ub^CJ^r&`&!5@?bAIxo&`Ku*4ac6dtoj3H-?H|i zXq!4LEB>YTCb7I38m7K{)HLH*yt6OE0%-j@17HDtCGPaohx^GVMx}FZf@znnPeVh; zs~`E6Wt~ztJztxd+q02KvlXO!*xX5OIgh8ben76?v?}Cen^1x0zQ!L4@dGP#YQ0~9 z^lPUtbVv|{?18LSJL%}3*XH1tfw0t!2yqw6F zRQmAmuj+O+iATk$-Osvk!H9jeHu0~v#rT2o2a=rDPpd4wES5VE*$j+>7B~0!l`5Ox z%TebYT=+dfF$#H5t8cO(;k;<<5Qs*_>|O%)wt6Y^-lptVg^(Ma$*F7_G4Ll z-oSV5?f3$Q0G;8q{S~?Ip0#vy*wcQOGfmWyPIcWO1$-<&yR2K?Tl*E7LeeL?}b3=qo)UUk6=`sOMHm(eYD%j3= z0;*cG?$O&eHm-0GB|BNOk5WcQcMv48vkKNx>Wf0zKiT~(eq|D_rtK0@kpC+DLG^|H z!kMo*dW6BAdIgcHqg=*}wb{sN(n*tZ0tjY^V@V}ABE6R+@~p;hImPQls5ql?RNjA8Qv zMiP|O%C=tN^1mA7CI67#=d;x8@DV+b%ncAJkpC)#R~^!?b{NwT300tKsNfskc?pI@ zgl9kVz?X^4wNO{QIJ(Fix|cRwIW}CzD6RVUTTy!-luSndG4(_QSKQUS&P@FClUx=A zGNyo)DbB0yXQG@4%`*KR!fd$xRfmltk$KQ$<_4c0!gH)vWZl?vhXNxJce{3@bDh4Q z1H0OX^w?ni(zL$ zInCjodkf6`tgu^AwV2-Y^7c~>)t(Phw38s>gd#YaGJ`VD8OC1w9UDQ-p+@%DQn>#k$liRx@pEowSr*NhkWI5>Ph`)2sWGRNakceo^y z{(3j4IjhRSH=jrG;gC7jpQWM~fv2w6il`gUbxo%wDfQNiEMXO3>N*7lNLfQil^4~f zC&N3(XYsaq9@(*?Bf>_Epqp~VB}4fB?E^P3wDKJzY(q~@H|h4!acTy}*ZcG?h-jLw{Y2ajMyGb=-k%)tsEGu~v@`xu?kwK;K(f8V^~Tkv0Y|y_ z%`4n6Y>bj&x(8C}1Ai?e4%JuaLYm1Ry#GyG+`KLmH-xK$;Zo%>m8KEB`6IN`C`o#X z>d4~zChdVYx@cGFGP3cz8Hs7?UP7aHxk(DWq&_0~%;?H&#!Nq+oP$S|9=@m<=_@*C z<8RxtVYa#+#0cuB|1)RgW`E{K6)jl+{G>TY#fJyI<#X!vA^VWpwJi`B)-@J@t{)?(>MVKj-cx_zL`{F2hF z`a@;gVs&_x4LWi}2#;_dAkHO|f^{Z!|9l;S4vBVXYgC?$z4WY_?IiTdV6KnG(8hhA z{jxe9X6z)LIC9LmXv?xJTSt+vshrI-tr|~eigx8-`n)LJ*{-mR^g$M$WpQULkKNYJ zDAF)aArhN9R-Go?$Z37MC$Kg3#MASv!#LiD^+p#UqBd=k*XtV3CBAc0_eMi@qh<*T zO#y&L4Lf#K@R=+ut@x;J2Ib9QLBZHn&11TIo(w%X)8sCL9dY4G1o12=quFPoWvvLl zBs}uh(c)B%q`>qKK8Izg?w1Jh^>B7)tP?Kad5WX`Zf6;wt8UI#D!SM~$TLapD#_#{ z4t7HKLM-l40*Ys#wqxXd!C>wc^uTa6&psnH>x3g`Fn#d2s0cA38+&rhVx`~LrNfMx zYS~kcd1srGuM{*CUh@mlQd2h~D9yp1(Lxjc0@~2UUmKkp&xRFF_DX+HB@DnV8HqO= zXA|ScX&L*avEqu}N$q&l{`m>4sHRP(}oE9BlMTQR* z$@m=N6tD2`q~|(@mrKmT*_pCODq z*Ev~lv+%;g_f4Xan;l-2CVad5;XV9qQ5%^=#9dtAv^@}5q=VMKyrW%PukEDw5_U;@qeAXv;^{M`)S>3+-x}AihOR9UZHgDo;A0hx4 zsZ>A!snbXpq3Sk5QQ%JX@}IKg_|}k44HTGw$C4C*_!lLlL2U8Td09a6MGLWi1Oqi~u|+n1dpYW% z(vuj;3ZWVsdwlGB!nr}|^1i=LZ}t2ePV0Jt$d!Q&q~OJ+#8Yy+TEq1H-fKh^lMHeS z3g7;#;{kvAuOi$FnLp84b4PEXO=ndJtBFf&908NdYrGs#⋘BHmrl1_3GQF0K0AU|s6|{*U!`BD+Y-bGJ+@-1Bz9_`JES>udFAd5ZJ-n@jY?M`2g` zL68B@GP$gjWsl^;fGSd$ma^eT52R2f5%S3Dncmi1RRHmJ-wh3kfVG!mFnGW88S49* z-D?H!0y5(l7LPVj!v|Oq-x@^xzv-Dxo%O9yjmJDR@SbZAe$|3~LhjIq0(cfuy3kgE zYK^AU{J~IA#>M_Ic1k4lMhmC4lj=iv8lpVa2RqZE6_X}6cTsu7Yj1G%TApEJz)R0( zSN>29i;=Dns$})rX~MbAf)Xnj5#FOVWQm;8`w$(091VfHOjx$!IhAB6q|uHxJ%%W9~w`3+hg4bUmT7UXVio z^~3sBY6;nIc=XE){R3rbx=2UI8**$&9zBFZdJVHOdT2eUl{`rb=Gg6A_CQDnS~TMS zvp-8m_~9pBncDDqqoqXKgtEb-@m8ZrP=~T34ED^?mGdZal~^zzWhY3OkF7ZdmHD*U-a*iaH>qQ|G~xdmgTlw} zjLD;}TBi?YXK!QwqF?{=yUC^JrKaF;Uvo z^ZxP(j89vFelX)zTco9zjvMwPTGR-m+mCPTch&`ja`eo_nE8XIK5WR-y%1hDWX-Gs z+U!Lmhr2%5Ki_4H z6SZ|Ym~kj2Ph>h%iU?Z)t?gm9Gr0Z0dy-YoqZ-edXG)QVmIF#<^b(MS&BCb~X@DFy zvvI7mhWXx$n!8#Qet23@1FrUMFI8zk_g=AQhTl6TrsoQuO4d3N>8ue zqXr;_PNso8r^S^R$mKfW6&e_{o2|BOTgC;De{8rkT>6*qRDV&S2~J~f!qF!b6m;*N zsaYb_%(5Xf2D19(Vm4tRICc_nNGSzn)!uhJgGU?F*^pXkwckI}zjV>hk9RIx^y==n z{X$pw&IU*e=RZQwEs$!C_SQ5|3-EhUt{`18R(;>b>ZoP*=U#BX3v;1V)F*V~7pMeO zM|KQkpVX--Ot`U^GJQ;b@tRcf3`mLgSPghP!wA7#A?;FToRXWskqZvw<1)+p+h zElg^HP_=#n<(13F&6cIBUNF1`5qu(Bk0jR}CjQ(q9Autd?k?+#T<-%dtGPg%3 z#15tUeoZnczq0}mirLB%6D2qxhD_61aAe;vdxkI}1h?^lK*w)a?I%lV1_eti!o89H zJ<)|W|2)Jy9g{leHCog(P_Daw?3ixE7JS3bmHF0XgcTJTfeS%yz9+l`IaBTFYvSz? zlRMlx<@$;D+n9#KM2~DgYWxfX0j(HZ`6<<5b*8XZLzIMIQFQ>`s~B+EOce){AY@uul$Z)_tS0$CczkdnP8ly97CU)YZL#^Q zHfbt(){pdp`=1ez4q%sP#e?SyE3b@7r>y0f?7D=U%h}g11HH0LPRPZS{c7HQ(sGm6 zUhgo(0sZqR?el`@e(7N5i>j#^&)Rp>q)(A*<^rpy%CixhW|ZX0;TP*)~x06bS^TNAX}uwp_p@LgHtnz1E7ZBDseCU9C_;e~Lx z0FCF8Y<4Cqdw+qA3yK5rre<`0_hEszQ5wX)cWKa?ja{3d7xOb3LsC#%SV5-g{#JUn zUAG3#AE5Yc^uREC24za zcttbsbY>KV!Wx%>n(P&OaDfH;kxNQsR4)hE1W1WGttGOM6QHQ}asIcW+Q#$TuFEQU zVhM0i$}RI59G%7JEktBDUs3Haee};9v5I<-_)3U&B0S{&l*uAD(sO0@s6KAXvhwQ% z$d51uAtM!dQPHFOy``)RU|{``W#`RmxaFCPV)sVee)s6zYsH%i+AFcUw_1(7q6<(N zJlTNn_)3wqxgPUyf$#9pn9w|zbSStL`^3K>j~pN;f!{~6eqXF21vcCgL|&|#e~Sa-Vv2;HX8Ggs}?DF%ezs~?fMqwLyS z(UzqyLP=M*za9C23#w}>yvY-lg^wgC1N*gqp%uf{Es__=VL#LjdhY4X02H?5LXS79e!P^q#xflpYeaa`87zMWnfHKjsuB_%F@sP`3VOf(n!sE`L8 zLvZ-&>&bgpg&XYcFl&Ku-&X1DOYnXUV11s#_(mAdKCQ5VqjrSo9*mwf*b)GQ`3=t| znz1Fj!ifn|@3Pd~XI+8pDp782QTU6^D6I+2ZSXI*Gd+iod;s%O5B8jVcztH!(!L|b zlJNZwC^Du;%Bswdz2B(nmocuY7>OQFg9# zm=*i`3G$A^G$6*7!!q_Pf>p2Qhq#n{LDiI8HshR_l1@=$@F%SS(<);EgB|X0+7q^AN5+e(ZCT&kf7)ML^utHCHJX)2+GubRj%KSr=|*>3p-O37C|%h zv3lGeSz7L#GIsn^CdNi$XPnjhQ5*-;QpeXyjw*7E`2flaRsQbbLt$IDj;`SHm zq-9(FUqe>hjyK#8zVl+t;BMkFBHSjF5`oX$|H@&A=J%fFA7#%Po9_wHnpU29gFAqD z8j1a0<17fxwayq6Y(1nrNocFrZq*4H!$m^M}2tZ#4S;|vYqASG6ego;4j z{oXyGr=I?&ueq$p4*UvlH*iYqX;F!R!uWvpI20$|?_T(ddIBRhe5Ye#VGKcT78dtL zikQy@Fzl!nK@gjh^NeoUioKO=rdIlCEETk>y{dM(9FN{?1!|%X)-iT&kiQHPh=LSq z-1zQoUvbmTaC1ceOFP)M_B?rcHhYkIN@4$*C{=FtZ%uw_iGT5ZLc|Xy*q6WIg;W*| z6)ahRg`Ze1NaJ@KL*?Nu2}H7~(=gafU4!R8#kEQuIE!Dk-e)_1`>|ga>=s`YMf2}b z!#Yp>63;%=o({*0h^U|ScK@&~Dq22haZBJ`iwOWbF&9(Bk>fbcHQHi}f zZ{pQ&S~*_oGz~%a7ByD`)HiO`l6o0KbRpsnek^0xS0>0A{ihZ_YS;JhjcPwLBV0@5 z=02rD)$onzsHt^Fhrk0gqWo|@kMdZ(WjA=WQ*)TlLgUuf_G9iG&&jw{s6)oH@=>2S zarIZ#fVIiw?R_SYN?!LM`mQP!A_x5Pg}x?~HNK_pM0Xfe*`6gH30c{%&7nC_G5hGv zyN+C4JEh_=mgc6L#UBkcw~3WgZk*Ov^#A&KJWK&NVF9Z&^S+1U!Xlw(i*7&lpUWPU z72W5W<0f+EQwK8YfE>;xvajM;85m=biK&_~)4(>%8yCqXNsWBmQT1AwHzFZw_b9V+#D=4!k4aT{I3dejqv36) zUSg!OF>MK~2EjL9Q@SFqM2aKc{Qy$SuDw@deMt=n`jpU>s_E+0>1=242%;M2>ic_z zH4t&D5SyiO(TMMxI?#nZ7uh3Kl0h+Ptb;j)*~-TuD8S9*p!YjOD-ntx*^ut;=<+iH zMXkrh>G}7qo_)58ysvw_Al~5+NGAo)R@_V%bNQbWhlcW+%^%#}Ds!L$p>v@CHBMV^ zCG)=xh@@OOtq8A#{r-sxf^p+<<&*e;RK3rv%Gvs>40!pDnbhoSR$gNCGQpuJsq3Dv z1=E1D#X4EAApSnVh-cuxhxJQ4cj6%>a)W29zLICs;>t5_T5~|mk=iy*DG&;K~gp?$RP$$Ofq8*7;p8kiTsN{Ssh`##P8ecQ@F zeZTlRIB*c>pM|ICA1KiXh7{NOjNGlf`beUpd(z!GgE=S1;qHWxrX4IJ&$~5xk6E_V z0$8IhCjB?YxT)RWU27CmtTUIC+dXPK>dPPQE*LJvkGvcq!w^ zsqlrz;?{MU7_bXKCEf-ZINUjE0pzk6e@f;xs}pa@z1NZliZj3x^{-%aLT%#U6Mj}9 zeoJstpI&8)aqY*=ZQ;!nh}clzL1WJ%$4BfK!<|^BT;%6#))J-z{cza1qM}x27Dw+s zND?rb`KJbtHIo1L*}%5>K<|zNLq#Uq|C|GFf$3}Ub4gXramkO_5Rj?+AN*eS`IG#> zozUjWAjE{QPg?%Aeiz^Q((J#fmOU)PxpIw={IoP#3szos2p6KlVv69|%+)5DMQ_o_ z?o_N_56W-EoZeUb#1m)i8}ieB_l&SzX!54WbH-!&`cRd>`%o@jp(Bj`SJ9L8NvNH0 za-Xz6Ys+d3^pVW7ppNXf=yJNGVxRQ5XH|#JtnSs~^%PkbQp(&LVSyj{<^9(v*bVid zuT7MjA?q#0QHysk?pzps{lL*KJ9c?vg=Z-2gF#Ta8q9wm`+UEd2IBSZgWMSbz$R(^ zdoP7ucMMq!`6zj+<$nNmCqP*lY{a)5$UK#W_#WUQVbDQQ2ZWz(C9##2)^@PTu4mCd zz+5922t92$i?N7)>L3lDAoTkGOXjJV9*}Ub%SMF#C#Ux>nMVmYQEtrVhdQrYtHB3^ zTq3|J{_n%*3I8o6r_mM>3Y46J%Ju2rN)Dhr)L4#zp&Gw9yzYNg6Od(NA$FcD9~0a1 z#RY{LOb_t<@46hoKjxR3pqv_^;``N^Yts5kZ7S?bZUM%vgCYXl4B^j#YVmhpJQV^0 z#dO>$zw$-w)D}aB zp?x`Z$orXKl4G~E*Vh#Td*NX2PW^#ocmJfFNwK>kW-pTye&k!A4@4?V%NX-To!nfZs%0L8hw9VkivY zx@My2(RZC?o%0TVK42=ZNrR?YHpTKj3nAE)pJ(&~!0{W7;nd$`5a0$z6U+Pbw^pPX zn{dC)U(q%f3^oa9?@gngmCqU33&&G?Pb<|{>whn|Dj4Wdc$5&mSw_)}Qu4J$nR+Gb zS3&bA@X@(36UY4>&uyppN*ll*r-+WiLNW8wYkJ-?HO=e$=P!tIS6VgSn}_C#|n@B673grI5|g_g$=lmR{2rvXMEF; zU+S=LJ$e-jC}jZSWz;gKX52;s077RAkf4wiQzUUv=HD?S4mgHn%w+SyW^;jUr}?Rq z=4OS`7i0p1ZOa@wS)?1M z)`S!iVe_H|X6MWNb`BT(gSy+FWp)eb@^7clDBHY{LTIs^I{%(iClW!7-6Ze6yt-)G ze3t~GM&atec()U?G2NT75_mo-X}C109UioSSRY!MjbHJ-oZq!y>_T(#M0+?eefkPa ze_`zXw$UfUPj4<@F|3d$X!E*dy*V_m7ot|9Uzndtz3rr;VL5+sc*Q`4cEd$rBy^)a zS?e)yynTF-=R9;9(Tc?Kc>)$(ju!htxe9QJVXgGwbN$M%f)xsw*GETysf-1%Kl&jw z(p8!k-=|=-PdW)YdiA2~e5sD|$TeV+#D_P4H(Ed~hrM)wWz5!>Y`YNQ#!4^M{HYBz z47R`H`P(=DxmN)b;pzW>e`3I^@iQ5qFcF8-x}l-!NMMQbF|!05S>U?2Z>+6HqR;g8 z_aZftVpbs`i(NUu9OeyrcH?w_)~}b?W8V!tN8DA32CTA??E+q`J1tqCyGOu7gF=tr zp;{2^36HQWnzlYq0p7-{5&8A6C(@0^1J3&Z*oH%XyngS_4h?F{_H1^|ZYBG*yIt=( z@H}1QsVg$XEPwF#q*^|M&mn#LFO&9=Wgg zv;KY+|8I*T(ub5OUK|>%Bg;l|#gj?BZ`3WDgKgXXxIR*^BLYhoJMuABWbT0M$U7GH zCCMR?d2j@Tm~MHVu1bIoQxP{rtrTE9E>LTbSK@RhSWjYH87^?!^&$LtT;x&7;F zqQUI$3`1Y{(oy_-?jS50>Jvng|47V^4d)j*j^-s2()ieJx+muQ>1 z?vpkuB$!@8m$>4z>q8K&aHwQ2^T5_yz3rMtDVbD0xW3^$1_cfn4(MudRxz*pb-sCv zsLNBaLxlqJ4zQlmPkNG=HtXATkl6kMiw|vq@tcfdUonml16x*M#-6gHwklD1l%%|= zn`gX>!Ua66TEaHYQY*p}w3af_Ls@ouYoF$xlu~TDy9zuVVr*&^+n>J+WLE-kpg#)^ zgyQz=Asku4T*hGCT$*B0fzN)?i~FQSQRWVGGCl%gdt}_7XF|2ruROeIA0jMns7GzT zp8VJPL>2`WWtga7@Gf<%D(%-;`T4rcC$=R^Vu6B`-xVd$(odPxLa0lNeG zG<-lTKW1AhdHF;P(oyvF)NP4 z&h1?d^&%pkic~Z+Os1zjv7pe0BUbbs5h~uy!*f|5am(S>U06Js*XHkr$@uA=8@^l5 zuQaW^*>aATgCbu37v??_q0b?cX#UQpHd?Svy}tgG2)cwILo4tYJI+xnMxP3Tj}uQ3 zacY7Dakk@d-%&@7!lTL>pFGWsS#X>iq+B~PO$cu_fG6rdHHkwFB4@;});XJEUv#dU z$YCn2wje=*eXCT#xzDs3&e3B@6*72wJNaCH;lp0DxN#25i`YBaN)(r0(*f}^BSUrj zkQkosKDj)r&Ou4MgLL87Iejf1%H$nsF!EZ#4$WB`-qF2eK5XzKaa;vHU47lmC?*dl ziVs-aiE%G_=+zgp$G6%k#5kj}_QTS(=!^|xvTyV~Vs z)tX4a>Bv*+OrelDQEdLTGEO4lDpK)GM??%0qkY3XV) zJxwH;Xuj0WHp$~0MSRSbWpR5f;}48P4ozql2Du$Ig4T*O>$}*$|EP7kZFy+N8$^bb z43MKETSIs>hgDZhD;-CjV#_a~rMGB5$QJF#y=|K^@?49?DAzeI&k5`$zh;az_(=o4gO?N#2v64K=dt2ZmuM8BZRm-6% zKG-^OO>{BRX})ng-8mpwlILjRmEpUURrQ3UL>A=bvzC!t9Hcd=4@w*ss}_CviN3h- zavy{=beqkcjSVplmx;tH&vt1i0Jhc#k&f{mBy*BLEV-L9addK{^FXPJ(g=9U(U;kI z&_hc214MOwn}~?-Cv_Td7HP=z(RoaGmoLIrG8*^TX9V+-dhn2OTa((JQ*M`(dRh2q zXpd-{T5pq2hRPe8ozmDsaa*tGBJDYG>L%#q*}I_1NRkS_uiHJ-14T_1Kk|8Q*w40$ zdn2oyx#6h)`LnlLE5=Xq`{GgJvgY3{`09@g%C~Rs{R**9^Jihm?CS&K0cWau&Z>}j zK2Y?xZF{zAS-DeuH}ysMc!&X`COq0A4WNDLv$etf<*%J7{mh9T5f+dvQkK#cD=lDpR!ywKBqWaMQ>z9H%82{(xmFpbiC0Ws)MUP$9)+)j!t_8B# zPw!IJU|=mQ;yqNu90yG^zDSeXDDveh*Jer@~; z)>`@ShKs}NaFvCES}1-=x*o%Z6e^c! zWf7xJsxKr$HX*rb|Cw@q8t}Xda9deEZ7ifD=$`ED21mjd_1A8Uj!<>}1HRmO_gUGK z8ZnJyx6lvBs$u1dVYR#ZRy^COsRw=v?+n*h`1!|93Cv2q8(UeEvFq_?` z*BPMXn@Psy_}IPaU5ZMYR44H`kx=#jp10G5R$r!6j) zw2}YZ0xT7U|5lWS;P)CIAq$)5+DRI&zEf(M+PW2I9zoE`W$T!784t8mDF zz0H7eu*rf-=v_{$iKF!)vxo;F9f~amwFYcue4v_rXwF#D+5_Jvap|+n&QE2dpi>_C zJ+vI|U9MW%z`w4X$@2vs&D$;z1MBK}<2Sxn_b9J3#h~jr>sx4>c*bKmu711(R#WK# zaR>+0pWyh(tLX?L??_AR@PmDlzJ^2*N1hdrURFOvfw$5b9$h@Lar+v$@dNOKYAjm6 z^xi^`SGnb|^9===DS-eMT+ISDtDMnyHWE5|7kh4~3_WiR+5aZ4*NnGG)QzVJFn1#$ zHl(Y0Y?yOTIu0Bpm(K5h9n0KsM--5D6Avw$>YT3Vo`co)qd(E^7{`4-pO!ONo>6^| zO~be{P2tjzR}!n%?=A04U+wE-{UZ?D-m{>%!G4T~s7_6DJ;~;7eUD&NoIEO`-YYr6*#WNq?^%h%u#jgby~@XHtduKsyTbjy2m-nUW~hC zSAc!U^{-b3G;T`I%aTv4kRLCR@MM8MLPPhuF?2IxB!OFun>)79b7M4~=Ce)z8M_@R zw?FKs#!G5{{1Rf;AhuUW@if?e-)$9aSq+X;f!!6FQa?-8AmUt$&Cu{<@p6=15=oH} z&}X+)tFmlPDuNA%4TR;kQf^MU*72lKU5<`zxH|lM=fDokg-t%IpvC6yez4**&ZO$YW7Q?9QQ;x z5s3SV>a*fCvDTjH_ri5e0krzI7XrK;$|E%Pr5qmez*d_!ZX&XPDa~b7Fl@h|Md%*B z!8{PB_;tOQu6hSL-%xdmRwdA8<`2CEolAWt*@P<4?;*|m-K`!7#DxdzTRZgj%U;6z zohf?GY!HQBYLlhWSjb zS&OtLtiPyE^5J+J?^C_@7P^rY%^W#1I>M(Tz6d+2!a{znWp-Y&sHF+32R!u} zUFO@SIT&!*+MFYyIRD+Cx`aP^Sx+jA%|v|TB70HHUntXC{hheoTo#QxRLBYHy;Co~ z((dD@0*bmut~;IALAST`Ep3#1XR6AKYIwzX(5-Rr+lJKZ+}l*_@Fg0$Y1uJ%*CgCW z9*YFx0u64oVRtULN2I!F2rvRKHVW7lM^fZ5^1)#glRAcfKAB{NdDI#rmt)i;kl!|2 zX(2N@pWM&L>lK+lI~c{bgn!-y8o%59M8`OP08a}FRZbXBgbuSUS+|jF1ZLjV?2N_&Mp~HM4lec^}+G+XwGXW>zD|i+q3K|XI>Vvl^0>0no z=XO2;)ePk#urT7Hn@rtW^87}V$tVA7&^F|&`G_-}sIfcCrJ@Md} zwUEWaQy-4W-JFTT!B&^i75R2f{h#7cs<9bgrnIu+rR_q?+LmIU^4Vv{!%oLlCy2hb zE9vUMm$a+JZ4|sWC}sGjh}&vXuUfUlj=XqY8ms!K*5)2@He=8!N9eLuLe3Dae?)pb z^_QVpV;pbab9ry!h01_!6}UHZZu^c*4_i|EGUng!_3rtYf_Ig zt84TEce&_{p<^Z}j+Z3r#-p{@QhCST3 zJWdHPCe^%a-S4?hw9MdV{eo9Ker?U}-RXX{pOtE}wfgB$K^%W?;xb z{hLobQDG#C&e0lT{puQm#c?@!NT|(GnH@JzwPa!(-&L# zSnx$OB@@e~lQ4uDSDA{nVL#n8nG}%0T>r3R=CGyL%9Rz}`tZwSRT5X?q@+rF#$B>V z3&A10PuKLPXpi#=hpA+(My}@g46N>ogrH6gg`Gj;X(b=eIRe!~BXm|u$aI&L!NWQV z*P;#3$Dt-;Fqw%f*x_7uT!9=%%;D7((QulUkd0Ro&wI!chSn~AMjv!45dJQxDoi{zR-{*+PCRAYE?$g{WBDtao09=_61cBhx4WU^|& z=j;H=OV!1Wb_aZXE)qGazeB}Aa4*41(uFRPX1hj58^P_NUwc@KtNqa08tzupy zkk$%I+uwlE>gb(PF9+I!R!07)V70^S7w{IUK4Ed?+cTc$I5a$r0$Wwf62YzzNH(oa zcRE1q>->G&o!$a;p(U}m-(heh!y1|b4n`C)oweUX@2}lrihueTM^Yxp$iw?S2fs)y z*X|)n#WOTlO0;@Xo$504Q%T=cM+9=$gg^O+B|KfMGEq^fFD4$`Hq}Xdt~yeC3z=Fd zfu!r{eO|Vx`YX1;Xa0p%ahc7qKO9%v5*l{>1M^3Gc1WxT-NCbL2KS|@H1{3RMv(aY zlf!wtSXH}Db1!Pu`SX~Mc*Q8Qh3cWEQmz44x{TZq5A1v`RHoTnd>-@ZnXskp7uu=0 zadf@<(Ni>?A_qebm*hZ$m6=Y7|Isx%gSvtfU-R%PeR!z&8Ojlr_9TlmI-Ul{fRj_c z>GwaP6wytuPY&dn3-;*p&vDNu2JS=@0x|E>-i62|>^O+KFor(SKq#nXaUG6n&^^u% zYIU-hjp}YIcc(Mm=nmDIwIi~^$BBEPdEp@WPdslouTwSZQ=LjLW_5`zmwsnMce2*U znMQA)Xy>4YPv6yhMs8Hv1WTR%K7SuAu+=31(p8IBvF(v)mGTH~Fo%~+`VtGjB!J4x zmf&mbwNYCFrT8;H=AvtlOUzp_h0#L4;8Qy9X6J&cFTP`NT0oWeOUn#Uf*;)}bO<+x z_77d=k0O!Rtd8AzZVf%mSKw32^Z^VbCAd<$Si0 zfi)=wfF=@x<1&gyi_25#>);jQY6?OR76}v&W7rZ1T_A(#&B^0iFL5?f>|f7MON=b7 zm3KyKkk198!|%0*yt(pBHpCPZI%F>MY5vm1Pef(MqYk*nQu~27GhQ=Skk(-|mv}M> zYIt>r4*O&M=bx+bm6?xNHuT;<3z6W4Lmjk?t7JtW!O!(k8q`6t3?Ui2Kcvwq zzP7DQ&n$;H=Ux=$OJ3 zn?dS#LhRYgKiX>b^-48#H5MN2GR6xT9%;KfJA49MKvm6PB$k7>V9wG59y(vVgez z_CU;@m5%k=wfo445RNoj>t82o#mn2WaUhy|jtv;>-Cjs{C z7pQ1`r2WLzz8oQ4_t;mV(RP2IlJD7=qnbVQwM(k7X4au1=lg0?_P-si2n)P=pW6V# z4_Rz%-#-rFbtZyfMmx~zgtK#qb;Ha*fOGV@s9@b!Is%H-20e$aR45s~Sa~XEG!#J# zYzB0cuwtsc#ebwq?>#n$YXLSlkO#Aib+9+_&5*KRxY25;O1h;-1$~PS%366h4uvt; zeyVT{Q6LAD1J%^0StIC+xcA)#3tT-j(a_F{-&@>{eyw8Nc6JSA2WTyH5@i&c2OVw` znt1xq$JR2#ls-Y_J`$TA#4Rrj^L6@Iu-LHF~P zRU81!h|Q>H!RvfkaH^!8ag#F*c$cR5w2)1G0${ptp6lD*(ZZs+v44_Cf_vDGysD-5yD^YnTZ)DrUMVL|B<(Z^6Zw_qZPomA%}%JpK~8;Q*im zUwUnuv}xaz8W8ah-oiBc?|na4Ktih(p61puEd0hSWtPC@4|*V3SoIw-)8s}hQh>p- zdW%J>DOY!555u^MDJ_((`)#6&32^Dn^l4vA*#hN@_H2ex1GyQ2im4Iy^u~+bmZ=;1 zHZ8PvrUdIH{(~_vl)7sdHO}gs?nzj5aQhP`_)V&WBbudR1f<9j6AkT#z)@^J1pHUM zu9f5sVP~gYj{PB_*c5ibQ_8JUHO^LIjzg|UDsnh-jEWBtshv+*jO^Iw9SjSx=ka8i ze^Lk264AmDC}QCKR%eodjNFH&3X;$FHS@*Oe>5$~`FP^tqn_o`&KS11Z}Fzf$VGUV z&dIyvAp}y}<;bhAP)_xNJcB1+2piT6f;E>W`$O=i4@{nn>DG6R=3k8~-aIKe9&yx+ z3vlF8Y+Bi$x%VV0ltmw~oQ(kJ`igExP07$xX779l#Qq44CM^a3Bdnl_+oTjbbBbAP zb^i7<3>{X}GF}ljW&1w|aNXeD8l$f2z7_bOg(gP$@$4FIhPkHYe8ol_o{-LkSL5P5d}5Ys)aYvK~{3f6?2qIwcQ zV4AL0Tl>W*#ftgDg#@uxvfv=N^qII*)|kVWmRU=b+CdqkLDr$KvqJ8d-Cy*>7Q54t z&aZ;HGsq@L9{|Y-xcazl!yd4kq39RO#y?%|@R}pYG-HU)^+mW3D5&2lcls zkBPT%hP$1=tlW?5Cr13^F|ji;ZN)I4nUrUrIi6xt?7Jis0dOwez7(9rCp&$Wl&^tg z@4V2S1TeC)gJxKag7Kt&M<49xv(=YGK^G zJN<-N?y)sX>Xwx_0Lu>#G~>v)!%olk-GFXeFlf$txw^3iExzkb975tJ^&nhuY377k z9sBK@0#yOnT^mGmeloGwfN4!KAGn^}xc7oxXV;SvMsn~xA1d@jmL&Rd5F!9)o}lEo zFU!O1B}8{BY|eX2S+#eO`0adJlH8mWd=!#T(tOG%*WlMIu1S8E=yTSgPgz2&jdD%8 z;BftQgM~LFrC_f6Rx4S2l3F+kPMQHjT>5>--hWQ9DJT{Fyehk8nTDP>M6=$Sd9|tU z-SxyI`r>kZ8Anx-y%f4(M^?ho`Y*BQ-!%(V#{g!rtna|_0vYFRJh=(#UIPSytyNP7 z9skk!r-c#!x{xjxwcxwuz&*A((6Ya-yp+txm--va=xPH7Wsh1&0}--Kd$XO;+T52-)p5d< zj{t8TW4?J=jt)5p;GNU}{z<($=R7$5^K za#!c`XL=i*Spx7e_~U)z?R(~AC!6-9Swukql-9uchf{%CROr^VbN#IO;vdtVoi>LD z=z|Ft@F{o z|MS>x8HDwj`yj}2OTE!f(<0qA0xhoLpyU3wrKC#!sxHewW=dA0?|p58LiaYyNg};Y z97n6l{d#+rL&5^3Ub_m?{*dUVSOAo(R~+Y?45+!-Nw=q32oZPy6TpC8vw2q`L{SmH zW^ZXw~t|5)F9YMbu;V)O0!*%XO3mZU=LBlT6cSFV&2l2YL7 zIp3*RwXfTxpKRhul+0CKBHe3hFK-2vBvSq$~I06ao2XlelRi)^&c? zDk2rNQr#%FWC6@v=LBH!A4rn_oq{DYPJiY4*UMgK1ozaWad0#^U6huuSMz&;8b>_f z$@5BI|AxJOex%;A-dTiC!Emq9+G|t&uu10|f=|Nx)hab*9^&gz7w_7HG>kg4Gozj- z6Cqab+1nH(2mP~4&X^r;&~hyM%xiXtu9nR&J9`968W@JyD_{*me6~)mi-umV;Y5-m zK@r{S`q9S*+*dw`+uMtyP6AZ*iZ@aHvmkOMbNG0@A#DDwC4%Wm1-G6YIthv>*MbP* zpxCde1F@xz{uk`RsCumwByfW76xwU?YZL`EWbB58B{g@{f-K zX1uH51qF)~8<3DxS{?;Y-Dpz6D0MSAeIV}JhfrNJeJx&5X}GWY{kt}I#(>4eu;#7c z2P)10&5Ei_Z4PP*NrdSEhvd)820wPbI{p9p>O3s+(MfS%-Z0O1fvr?Mgu=)hLPdHy z;7n+PSga;5%RM{-5w+iJvS8K{fC_#$2BOgK+k_g0j&_z5JlKiBUn5os1ku;3=#SA+ zA+T9yv_kN=w>#n0ikEmdV}P{nJGQUxn~1-?+}bbm9(ND_rs$O$nXHG;`LEXZ2)(AG@(G~zgW0p`e z?0MFm^}`@jc_6StWlyTz9Mf@fBS@FI*_6`X+xlo zKsgMzI5C`@g^z$q+J_0CtR4~e+1@!NWIOK3+;K0!NNx&?>#mVO=CTSoYLehDx1{s% zDdkTUugP~}4W(FvoS<5n=g}CM!Ynz-CpynC;Zek*=j!p_XzO>HJL=3m^!}O7|Lfnm zlSd7_eVibfyDv!#_5@tbd>VR*_77&$ShdVmGb>%wbeHRDMihWgB5A3>e}CZ<7l^CV z>Yh}+w!Z_PE9&m&nXNTV2=zeF3MtG;_TJW|3p@Ays5zXH^@Xj$D}-xYxGP(yBcSo8 z{t<;r5!${HJz$|1=WLR-lE8ADYz>m!t_XphOr0^U_v4mFnzF*IE3$ZEpbJX{k+1q| zzo0HSP4&hoGN=CJWe58BH5Kpyrt4Z#zc$Al=->Fv*-Z_|;|MP(bu43kH-#5-)u!^F z2W5F!xe)RfG#1f}`URl(jSEFqL7s@+8e+^~ndj4#zV&S(?K*ULwbLJWU`5w3Li6A2 z+g7(*RX&txVSOp+*VCXKXB|$)17-X{=ePZJ2+D7u^0}jTN(g3v7|6M?lkX?0T5?zx zyh1bU&G}Kj3)dt9KAdv}fCYx0Fuerke$TbqC|}RtWeQ_rcF9AaDTNtti7WQlHqR_2 zijth=ch=akL1Y_WOX;S6I-+wJFe*R^d!r1SUl9g!1l!k;Vp=iqlU=C9Ll%B2#6B8Q z2w}QVGZN5n?zT?)h#{ZtlTVpMFPAe0zwdvTC3JuMa^2->CS=yxdd_-5jtW&rT|gc& zajLxVz+R}3D$>32yyk~fMPj7Kp`D+Yu51 zbh_nO#8yE7uTW@~rH1xUf2NW{mGN3&LKM@0v@HO~zJ#_+EV!~a%?Fy>zq^Ookb?~F zq_4#30wmt7Ol2%qnKw97c@X>UcB(if&wWaSO|e^8K2jx|qv_*of>Y@o(NiH5JXbGI z%d8v2J^(4AM#^d7r(_>AH`uMUgM6;~Cr9pLzE|;VRAAmx1fOi|wCN(%QDgruQ?^Js_uGV}00Po0{@)s`I>U{qSWTAl=X=&lC z>WkZh!uKx|H0#G>$dD9`!sPk`I=gArV(HCv0V5GxC zBpfgfW^~TCbJ?!)1sNm7YGu|ti31r3d1G+b)h(duXM(smpmS4@L;J9cne9ds6YI*3F4+A$3e3m>>&NdboXC{;Qs3^U1VFL>Xo+X?AiIkWJuM+r_Kwf4zh`!6q7dK z^oiZBZo`%Q{i5TOlsmB&g2c_G((nWh@!Di)=n&*UNV9?D^hQ#uSAIso>qA@av^4+P zh?QmbeB}Y06=En*A$1ilIG>h|Bq_;PtyWU{k5%^N(;M9-KHAmCjlkC2C0BHwaVtvK zm<@wr*RjoyM;Ng{MCCzKuf7GZSZ1r(u`G91N5BmVcY5Wl=wP`QlG6?}uDrG^-`@+( z{p&pUYjaD~Jw>uPK2JDEBtF>Yf7fG5K5CMI=zxZT@0eYooM;Cu~%%CXeGoNkw zN9MvA$6PWr7o+Ke9PzP`W|h4y@g{Z(-3d3SF<%_a-(N)SIyNS;Wbkag_w zRUcT?;%%T{E05})vPCGv2mBzNrki<$IZ=yY%2Wcnw$FHR*E}H?r|oj9B>@L}6_7-) zIzBQ=ElvKKnnqZP=nh!l7W1o! z@)82lief?Bf=R;T9>dVEr=A6v8u%rp$+N4Yp^yy#dwP6B2WT}aYC`hJSk19WSCeiH z!Hll<>G)x);zta+!*@-_?`}CBj~=uabOiE3fsVSJ&J*I?$dvC(*CRBL(;_>>xto76 z?oXh*4LoWuDU_+jUbb890ls?!O>IA$@3zG66{daB0gc{*EI|;{;^TKm(WfcoSV!h zoG#rA)(uAFtkoRAy-#eSaq*~o@{zh` z!0h*~a5~6TTnkG=U@id=md!Jw_)(H!sP93PH1jbB*%~fCXbuICxcbu5%{r6w z^aCTEbZB^s6f3rHIG|LE{BENIP!j%7Bh_}@s;eb$&@X<6`1IzI?jfDL(L4ZszE3>` zY2DP^5`38vAD}CyYjH)ieI{i5>8 z5Pt8!W;*BGl6Pji_|_Br5g&00Tcu>afeDbp`4g;Vj7rN3yDyeQUNOV;7@NAEIhxVa z@I99AS?p51S8Jnse@mZU@}j9K%ro8A&Lo6iQZ$TJ>d>a}cw5XaXNow&e2GlglnaA& z_q6bk7hbmx=_!D8E0rv)i>Y=m$PUlEst|Au43j?R(`S#>W zG2*A@oqpjS1M^P3OY`=ej1Mqh;#u9{Hs@ll&=sxr>V1FgD?&Q`8Jaw=jNAy( zeP<9?OK}i+R|ZT;o8X!!>@;>Xg;+K!AbG=ZWE7B~xT<@hqUf$%+`rSBI;t8=QK})= zvc6xzQ9+pU=ezqxN?PUXMk3IrPd5rjFZn*ajAyD!M$r!iHZ}&9S)RX5ewL_I{_WD& z>ymrlP!>=}XIOy+^D;;Oc~wT=xJMODdy~&uTrcfMj_%PchcASTzgUT%6P+%+R99m@ z2OyJm8^8TAn|Lc->`i{$Bu~LOsMzsfj-45d_lEyj+BF9`-I}i0HMW*kjalW3`n00) zt2329_c=L8isN)+%zQM*LwVR2%zhy&^L@0kX*LL&(V0a7G{N&F+tS_?00Y*ZoAD3t zE1y3>Z5EzYTB~HLr#felg>+b_EGtGh3TN&o|9EH!US=#e;89qhsC)C|BmVkDp*) z(U^kW)I_e?Zx8(S8r%Ln;e+KTk`0uVfcrH6!5yD`R>DYv8n^rtW0$LDpfQMD6mP81 z9{YC-g%CUobnLxtQ}yUI&YB{FBSVa*GoNpjm$M6uTd-J)8Mpk>l3YNZkrM`BX83C- zGU+GS9C>lBs+SgqJhmATNBu9?u<|s3SN3QX82$;1RC(`!<6dMTO8adtDgCu(8?$w~ z;`QD%_JUf|pFP#NIqpuKbg{Rl?20rog3EhO#QX{R{mT!)y|SUq1l-%v%|fu@m`hBI zmVV>AREad2q>`?K)C1J{KBi0;Q$mLIx&(lSS2BYBtWO#jnCmWofMtfi;xzGGGsjnx znDhWv*K$r{3KOFazY8>V-L7J7EcP**Hbqj=F@)rG-^&$;u`NKP_3dAqCnTj^`H{4o zp;pGO4#04LMzIfkXEyeSRwA_}>-eC5=eUpTVtRYg>ZNC(h%d#Q5j`GA0GboA^n2v8 zpQ_pwsNgZuHE~>VLhle`>N(%jkmO15Qn1jJ8U+D_$pI}1vfpP3o(?99LMDglt6g2~yFx&W@Je*=9_Aqx9CKL)bkQ$^| z;TOQQ6Rf}s|K+5jN8;Iqf_bn%35&mr-vyrk6u;%U&W*DCwsF-=TcCyTpjpl0L!IwK z+avE5qlus`$|@HIMQs(SpPQ~W(r*+EWV``J z{te9xJL4!)AL z;`ASz`vEq`7U-3U(!*```J|yWJ4OQ`N~et88Ui{DOr71d&u!2#oLks8AiL!3I$91W z>%ppTll{_@RyXcb`LUTPlOaq%nZ09)hS(HNeHnUrCrbQrj|xt~RB^0$*5&xq?Ga}X zbu0TC>E2sULxXy{p5qp3i0h>IyO-)-$9Nb4;bFDkr;cx%-nxCY<#7efwCppFOo!H`? z$emB)J6|AF-gDt^)_dxNg7>z$>xTO%!(7sU59Ok313B&^F>r57o*sj(@*hW;Ezv*~ zE&802m7O5wTIu5`<}W#}*lF4vN!(z!XRFoO%ZMgev^Oq@ao^_20D`)^RVnS@_v-ei zjFQE^`UI)GP3qW=QI_KZ6K00Hp$@MBrw>%|WjBTg`>OTp$5jfc${SSiKjWpQIn7@b zvEj|)uxp<+%1x-5cx>JsK5GF?r17=HeJD%|CM(@K!{L3Y%5N()-ruzIj-qJSY*-}vC` zmD!oL3@9NfY*D)qVB)K$!KKg?mgczAqpQn6t+dPTqW3@RXBY&OwZ%3F?KlOCtIKgY zNXt*Q4V)08`Y7^vIEuX+vtVZ$kX>*k@R>E2-$0(++Lv|8W9j;zx$*u!TBIGPxQkLQ(T61hNU-BMPmwT0#IH?^bin6LC&HoH7*a)F9M~IO%_KbwpJ^T zBEF|AMsTZW{!MCwK}wZFKDevEJ2`S}&UH*xAY;u%l|H1#o(uY0j9#hrBs?Zi=mufQ z9s9UhJZSrg{p^@ckUfVsEj&M+!5J)P{kqU1pn3_db7cq@K()1W0(uhCMybBG@9EL` z69Gkb9p}60pr5GIgorl*82dpeJBwchE`lEHu0QL#klw&$tMswAd4aoji3g zvbU?Wc9vy5A zE}`BQ(0!pZa%UhsRt?tJ6TwPmjL`M6akqw6pXm|G)5M6k5J>c>CjfL4Jp5O#a_S6B zju)4n?%tAyb#vxTN@;oadX$9KhDN}iQ%!*CNp+!Je?GK{0ZKb^r`ahwZtxIP2#^wP zad9eu2*6AoO}7&*B?@vX6mWl4GkQ>mbjne5nzYZ;M%Nf$C#`TUBMc*h>B@8Bc8_Qx+Zg7;e*=m_8PL>3LI%DnFuzmWXQFdVs9&qXqP%2e%4H zO6J|Qb^w*L;54Ho3O?ov5giPN=f?!DF0I{2DL>2}U2=g}0Iv-@g|DAr!;u;@^Ou_rVm|rB*4w3#3>8$rk?p+mm_)|9JgyN4 zcqKP~$)629BwM=`B6&MYr)s@KjmC)VHc$)^F$W6Tnz_4LhI2>8@Top+Gi8zXl3+u` z1m^n4AeI=VS|}}Y>4KnRhL>A5Jiohcd&0>ayNmb@&2~2G`DR~=lE)^Myt7* zQR>vM{k!yHNOg2gPV1#J+LQ-+4d)T@_kjBcsc6yin4Fek3&e*b(`P#?=eH^D(4 zGuj(6rA=*K{Rrw!3{m)r5nHi1C=UQkmy|Q*Pcopd`DC`rp9Qmir%Qo>P|5Pkl}?RO z1kHW(sqeZb|Dbs}18LwbGoQqx%c}h%S*HIrXw-cl3)3RuZvc}X5QFkw3e2t5?=c1F z@+UH{Kv(r+XfzK)+L@fWSKEmt^qe)UEW#J+~Zqd#m1iLf)lCg{H90@frp( z@8_HXW1}MVN$yTj0k5r0hOBem|L2o$)_`4mv0J9WL#QQv{R`GeFi-cvYzS}3fJ>!y z!q<+x%sRrYi<$pvSMh}4d7XEEx-t=42Ff+|VQhI&n-31Ag6`_8@|Jx0r>DSgd;ag= i|FyvXpB8xX=odV9b_nCxD@@#;q3&(NTjiRL&;AdCUyR8B literal 132875 zcmeFZRaBeZ7B&hL+Tv{~R)Pn&mg3&x?rz0hgFA&1iWA)3t++#h;vU>BcyafWezO1n zT`kJkIj5nkQ2v5dxwUAfPg6}A)O!Y>g*kQDi(;)eKp>4lf_6#k3jm_sF6 z&%x!l1N7R)cE$l{6kmh=UcRgn$;6oReL)+7Mo-Ub9P*pr_()f5uFsavss1M8Y(6k| z*m{CP()a3#`-$3a`A(LJ<*3Dz^zeDi9o8fFZZSntCOh7h_qGI9mJoP_F1dO9_lt~Z!Y-iuyB_WFsC`Y{1UUSUN{{2>7{dybc zu8i8!RPE=tCv-(+zbakIcfI&imeqG(p8Z?Dp|V1){(N-kvMO?g`CYp%hP`~z`1&cg zXY7`YPke`ovi7H1B}F{?-c2Vo=e5OzORlHW+3v6G%@Kh*fZGk>P4d4_fCXjdvgpKh zpV7qD4i`Ogstj33Rk0QnoyH0``Ptuv0)EYfDSanhLJHQ>OBE8New6vbcjG;<&`M;a zS3mF(IX#PB?jinDRQz;N1L(J?Fhi0P(t2ChGUr5|i&?nM1@J36k>DvZ{Vj|)b4!z}jY2f=?i;2QLPfuVUyvsnM-aRjg7YyA=fPxwD)_un)4 z|HjtRC{gy6DrgIz{>$Sikt7-j{5uT)&$IZ~;1n+8W82Ku2_DG#QT|^q#Fgf!BRJp; z{0ql>;=~Q`oHhE$z_9sSkOC~f`UT1^fd{Iq_|vaPG28wXoNv1NXsc(rcF`C znz6gL*US-xjr8BE%-;NTHlFuQc6^|niKhLtn!i;MQ1Lw^(Rl{*H?%%?d+!|RxNc!I=a)1~t)opWs;dd5~Os|yj1KH5(@dEi|x?)XTmOMK1S z_sl8M5aNB;-^-xX%GAnl&e#15`1IAZ1VmjHP~Fn*uuoE^Q9rDi9N_YQ0C@}&p~HdY zFn8YRw5?rLhX(3jHwFieef3Wj#ilT+WF0>No|E29lVI%ejP)%Zvfu)=m5 zWs%c&ikhRNjKVLEd;>1>!s&`f*&AMp>Y|O)6T4Bv2G4z`8L;%Lm$Z)=+|x^~NN%4h-Ol1)w2JmQmKx9o7#D3JjvVLs@Ca znHz@5?{))0j{pH5X4SLDL_Sotf9e|9ml>q25+v=$BAu&OMQF_pdNiC@)fQ^{W-Spi zA4}f?G|JQ_zGZ$fF*h7ejfmF~9^hlCT4@)TiyqoGmu$RS5uUGoB3Np9$e>Cwo41*C zjoeZcNKbR$oQcDU1z{#KzDX{q^1 zB1w3=gpRUrY53Zt5U6pBks?ztfqC#}s<6}1U1lpa{i!iWF zQb=SuoQRXo=h^x9)`GVapU%+U^_~fCji#i42}fkjZj#%*2ZMk!PWc0iqzTYpbLfTk z9bU0170={T8%2hRrt?v%1@rhCLBIvoH;fNK@>!nPB+4c}*9<%JvOGQ5$Hpg@Ksy!| zle+`o;R1RelbC}PI{d;;cbzRpqn!jvX0t|T=!o4>yK~gz9{n8JXOh&Lh-l<#&%L>7wQxldMsEh_nm_I)Wd+p z#)%(5-DBLt2QjbrN6+{eFnLdfgeY!bl`F)v`>^}jW-JP=FzSr-yx2KfX`8gQI{cPeZ%}}} z^H9$VjW<~_-v=G^*zECIqY-nvFRzjV`=JeI2=DkCI1e;#h#MP#e0(&*{m$-ou@Hjf zYf0Z$?>zB|#f*5Uzb~F*bSFxzGd77p)@3VrqVH2{p>%V;$HE5YztWBzO{PbiDb(q3 ziZwZg=@<5@%;4Tf3%vHj^k^pbzjuyu->hPrp{`%hHEvdUe_|6=jlQ2ynH38kb97aeII zSX}$1&l^29S+}0rL)-W9Ex<-bCa5piqDYVTfO!~~?NH14H7M@J?*R)Uws1UxsC ze_)SLwvdiaN2v(Osd?s7ERGp71VdJ}tNp89+Vi#r)thyd?->#?6rieW(~A$cn-W_A z4x1_0S2sakZNT7yPQWxfxs&0I%a&O*X^T z1-9FqdnUZdGNN5_!_H&($=|2=9HHJEzENbkIR5j?oQ$sra}Ql5;6O=C#mG(u3P2^$0h0F1{04v&N7T-8YMiP?4 z^MkPQx339BDQUcA+eb69HS;h&rY3l{?oewRn;db@kFRK7LPw>a^0Ak9OP5GkATJA5 zsQRZ4{CHsY`l=CHgOJj0?U%+{a1h3NARZa?4@wC`7ZG}530c`dj|>3m!J@B<051@HDPis|1D+Mqte+u$sJD2n-hOD z>lrwr(;#K#`;Z~3`s$f*cN}=VSUZPRx5+U;DaJnfq{4)t5?65Kz0*6Yxeb#{*YP@j zaI67<&&rM4{W}9lHOy&yGX$Pqy)6{YL|4{Y6QDJ7@zh2Mg z&~N@0CAE|k!^HcsDLucOGDff2)JgZ?;2y`6Nx`FLpIX~#Ke1c;soBqTDVJN2mOs}y zhAG+k6a?*iRLQqBLxW{F%F=D_6rD}}u*KYThCW$o)bkLp8iYpMOVY={T0^R$226^0 zU=o<~Z}`-vT0MKh*Aj9q-O&BR-UxM4<-i>D{;qL;X=s6Em|jc*%e5aOJ!Q+&S!lRN z`}UZ%sv=1EQL_vlR#>$$3BS^(NiKqH{#YyBi zb+kB(#4}FamXj&;o2dNX;^0Uh9^NL@IaUkDr`(mY_jby^%q{UtE@#*VZ6Z5FC$vxO zYP~Syo6A92tpCRQ8Riu*!9<)|`7NnLmh<*xR$)<*F6hv>JSe9&{w3cX!;hAMDd%T) z+FvOX*Hvw*ozv|*Al?%mMB+^mGFIqTk-n*5=)I-cEw&+f*dWc<3E#uUheoZ3SI}y7mFwQ15XF1ZS(Zq3In&-UJn4X;^Vzg>Q@HbacwXupeHWtkG0oOuiF&Lh^<}=ZR&rMc;)gRuA8?ba1>FgRl zpMIwgylY+E*OaB`o{cwgXBKZ<{tapHKer|9#Xj{Ca8am&4FnzB?G8sUg%P6-VxktR z3`-n%s63q8)2NPU(O}wqHF?#CfrMHeWf~Np9sVy?PQ<0%TVs_IEk31Jg)zB~dYjhE z?_(Y=dI}w^5X$Mx6Mw2LzSDH#k<67qB8T)^aHs(1DEPmeZhlC$m+a@pOrzmXpz8fdDIKH5D?h~riPI-H!%;Oq{S=heXOMt3K&yu;-_TlPD!FZ983S@r5_MIi?FcV>>9 z?ofp{pPMAJxN*SrF~c~hds0iE*w-KOunOpvox5>xSeNIAGcPh|m2miZD9O8|-+GtjkzCnzB(Pnupm)Tcy1e#% z#?!^Zrc!ML&=xyP6MA~C<@|Kof5}Vg6ppsFabtl2L{{<`F{Y8+M%k3cFTgQl(R5NH zG%Ru0Mm%2{`Y_ol3<|xhbjg8qqfKmDah5HG)5kM?0?VgRx-R&)e;x)bi} zSB9~s&PO?bXsJ2YLOph>x7jwTJ0|mVm1BCV`v>sup47k|`Q_SCvh-gAxMF{vZOrXB z`htsD>Cyc$aE|XDXq?U6NwhJjs?kMs1@!4JN5Y}&tMTo}&ZB5?UR97mOK0;@7+a|rHEt0E&4G-YmKA*LL8g`)j5FYFNS z*GvJiHYc<^;f4D=yPePqi=O0EO1rGP<7OsN{gGlTg718tp-L#bEnHAyu--1saj)J1}0x^bo zmAd~91Ae2Wr6Yq%ppK&ey>yTGrqmg=h=_<|)uvJLkg$#V&tz=($xMcp=W^sF> z9WXQO<682zIh)V>mK4EV$Bh}Wu7gxgmjdciTbl9!$eMpmA z(E8zus@9I;bmSsBKGTRD5Cb27-$Rl>AlO8j_=X2sg_l37#!9MxLV;rOw5F^^r31Td zm+G#nF}WhoK}~3W;#b*XC|y?#oh&w1y;#awQvHTRVv+2Z@ygx>5rb~)cgFkq7k?wbA3|hXhfE>Z?w?_Y zpGc?%HGhcD+fXds@Lmu~wrN&FJ6U=ex}Irxa_qi7+uj>~OF~jqU$Pjce{$t=_+diI zu90b^$n~d_4`mnmTl%Pk80NvDbndTUEf8qq!wF`rc%<;o&^LzuY1l|oTCUmmygX0} zb;!fO1uw>-0P15cq>L>boKh1!po-ab4bCv5J!c#`2EZXNQ>NTvvuOQx7@*|MRdBR%=-XX7 zN=eeUDQIleq&)dp+*(Wd;@ur&UwYxP(d-F0p!gr61{D?}CYBvn#|g?gWmNW8QHw~j z9^~mcwO3eLAr)5CDKHU~nIPgAm~}TDdiXXlb2PBQ2)5*mCc5(3lv2zvh>-{ygHY%P zd1Aej$Fx+>YvPmf%g=?SCVL-j7MxSunBLO?=h1(_D8a^HS=$hs6z2&Ngn)J z!2|ikPnLKw8Z4V48VOfoXFu!&<~Nh^Po385KZ=dRvFn62))EM?meupl<=#&W!;Nmk0AELYD6;+xQes|(Du9B&mA6&)3iw#j=u=8g-w z-Xi~2L$J_pelS<4bb1xl>uItWXC4vQ?67MOmGMC%v~ zUr6kwh;HLgUlb!Zg33icyt?}Wi-oG=$XYh@A&i4hMn)UvH%&Hgxwc=JXsV^#ZJu-= z4D8;QI7k@1wr%v(E#K zf_K;amMZN*%RX)~ZxG|~vA@+NZiCA{^>zs;vfGB;+0z;P62Kz=N$Hp)L^gru1dFfO ziV_F3jnts+P8?f%B}W4&xvB`b@_>4<)m^@AzzlbyVC702j z4b=7WYKbY6pF!bt7f80hWM+%&<2=Y0gP?B6XSA0BiGZ4hg?$E^p~#Go;=!Sle^n+%KFH zNEr!uB+RpwRGepjkCeZTp}OFAxkd8kTa)%^6@7Nd|8aixgHp}=2yjWmBA3BT@r&zr z^1+3;>@x$JZJd|b6iu@MX5J&hDjqp!wJbSWPLV7$!ZWWoAEIclnh90WQ)kqWKFJ2J zcLmdz15c~GkZiQmNP7FZ{AXWW(e8CxJl=2gv#aZ!o|j0`b24NQ#`~7YvrCzaa6i^- znQT5t%aullQN)znv+=brwc}>ht9`r_Ja5 znBF}rhy&5*Z5%4O+ZQZ@(-X=1TeMvv7M?vIhi3~fBzU7^#kr5Ly`^)C?3ChFD{N(7MgStxCCwMor}^0obW3FaA?Ek7h_;|( z&J>k|buVN>d9iZ2I!DH>hR&0~95ivJ2oJypw-%_Sn|wbUS9^MRmiq?)De1G&Fu!>A z*AY4!XBNFJldA3xEf#*`Fe>d_I@SX52d&d>>a*D2ZJK|CVrn8rdF%lNgXy=sOcVyP z(FUCZv5dum?05Zj+*M{dGkeJ-hc{R497^3)RI$dPzc=zhAM@dK8A#6Wy~U(MU84A> z=yH_`nWe9k+ElCWwx$yU>M#qrkQrYGx4DNKz5zEoZ!~K04XChj!HTGQt9osQberdm z(moc;(X~b-EFa=t-;Sw7gTIO)`-qA+n$;#GVEBBVe-QvfBq#i;L4v1M?Z8RQsg#gJE`8_z4kgPOFh#Hrm^T5_Dpl&b+R>E~X zE^9eB{&mAOA^>o&K~uIGmCka(?co@^$u*)F+@(>ex)erCI!GR6foxc=Pwp(jRRjm2 zp3&<``Ag=7AXEUCM=}jBlTDSP6tp1lL8jcQs!9XZ-w7|OF)|{ftuQPh1F8%Li$jL0 zfgswf+%zGK(EwchNJX~^IRfz!b42zlso*9zu@q|u7S#y8!CP+CDyp1thsOjYsIWi? zn2sS)-&UP~`S8$o(UV7x@ZHQYo<_1EVqEdd$f_Ue55JF_=>(p;1j4yJ+%PL3#VAJc z$G|(T9KX3cbAY|xm*?V0<0C zRSH}|lprKT@wYDvm7?Rzma@b4Z3-V9KWc8(5S~5~1*sVkBBzQ8QQHc*{l@U-@D{)= zT?{exrc4XL#3y{AzhzIaT0-f(&l&VQGEEf4EyF)2f8rwv%?gwPHNN~PJl_!Uz#K?g zA#qW6ewlf~xAeJmncTP>ag@S&xM$S&RN)J)3yb~6G{05{s4`3FXFQT+y8QdS^A91d zz0a7Aor}!!EMbZ@xlHo@MxTV_g@RsgM9kJW-1)R_Hre&aq>1AW#wlzOOzK9=)V*1C z*X`RhbKZ$|-^&sIAW%`J;gE`fZ$EhaE)!0h{P@vk>5Itu)s4x#H&(7Mv{_tzaSjpo zS!TS=N}6^_a4N{7ijZ^fye6qgq{4jRaHM<=WLkWgQ#nHQKGPL{zU}J$l+6P8dFw*B zXLBsYinYMoA$g|5S=T_d$Lau6d4kWfM13#gXE3i(67l47 zRr}}u*L^QT_`G8$S-95r4?pQWviD7Kfv`Y+U<@cdspmB{m|- z*1MN~7(2#Gg5=rdpQfhx{Xg&SgWr5hT!NlrTB~vnjs`Q4uNos&y4sz1h005AuKj`P zJevDssYBFc!u+K-7rr%jD@Ia|CQDoVaE-BOQRcEjBWUDMOH4q3wUY`BIaNE;E)ok! zp4aQ=AV%c*|Iz<4?`^p{KhG50lBa49TC!NliZR`WK#T_Ljo}VuzNmgUO_DZR_alm4 zp!lpWm7*~Cbuc=*sz+U6PL4zzRSCq};iN}h0DOZ$+>#xxkC`cbrw)bJ@t7fWx9#>9 zu{7#f^j+9gLuowF-|gS9mc2Yne4d@U@bTVQAyAka3w;ybMXAki zTs-;~gIe?+Qz`~eNPA7NmM9L+H~YHCjnFr@&3|TTXWxWVZIAz^?2lLHu5#265cp<# zlpv_A#VNcDeT)D&%G@lrZM*dt6vl3Rt7;7l3nxDt1Z;5i8(2ioT_rnCuF*moVgSMd zD*@k+WFOv7KHrTP3(a{4PToN4eG{<{kPP+OTH@LR^=*5`3*X8R9dQmbD5k0lwGD~s zH|MgA#0V!0AdlkUhu& z+0l3-D&40$YsO66>U;L_7E@PAuitCe-#5I6JAY+3QVM`aSS?Bd zZKFkGKP{n{>PZ`Bk&Am`Nxsaj$*r+n=MNHwkQlc43CDLk1UfoxT_;moyas1*^EgU; z-e3iL={@6ii&ei(GeYP^$1)OqqejVn3*Q@tg=JV&?tC*q$!G1_Pxy^yPspVDmFqFT z&l4_~6AX|SaJA4U1J_)jW68euyzyr0cSsQy@RgjDNa{U71s8z^U1;T~U*S5XPk<|7 z$qX!wn=yhWf7$$G2>uC3dgz7C7u7pryjaG4~84r>hgftT(yjPn9}^M<3Dcc zgD0#;srN->`dCcC=+0SXzD)vKYJc2h&@SCyrbx>h(PSq<3PDJ-#Vz#z}!@pGl?GA6(qKNiRXA>5MGP)Gg zj3~j4Pvtu9DIS0qT=|ZD;YaG+FfIsJG;gLZcPH1hW7xVxBs5W8DJDmBy?>gfEx{z& zJg#8`N_r~t8QALzc#JN1)Mhaa}hy)9B;m|y8+~IIY}e5bB~NeHiRSi z0GnI`_ONw=;S)UYxk9vakd(F_!LJX28|o2S0EkmjA9*{c3HKw+jq`-&>Afb*RpEAO zboClygeESBb)&{08EXoNatRE7Ni&gkr*1NdBmELG1Yw7Dpb8TOv~*?4?zv#ah;VieFKgR>p;LUwHm z+$jf1Pjx;xQm&{ZJq_wl=76LJ4jB?YMkoXOm2yN`ESGXg_}d!N>Ygr$b@^L@rcaN9 zNae4;QV-vJT1Zs4QqO{?2Y4JQ)_SU3P}*l(_7Q4r)G|1ZVk=N?MT4{!d|_{EZRz!i zN?Equ)8pu6JrrsOHh22-4r;6}ka*f%QD&S4nDp!8NCZ5J`FF09Rwgv%ifuGbahC)d z{p-rF;}28kC6a$7nF@pRMK3MFncJNnKmW2_;EK;V>l|hkBksVVWHo zDw1y?TM+>la9|OGUU=<4Q?-}2_X500LWqV8kP@=}oDEL~Y@xs^c^h|`BOQn(t8Q?7 z1R=tP+>j!Btj8T0qQ%7u#G|-($U{!B;qg^^AWyW$5%Rm5QaR zHZl?ylz8=hO;9otkGiGjsX&ix%eM1Fr^~5+@9Ft zNQ#I6nQlSE2TuzpkKu%TE_mSt*Bs%Gh4p$9Cd)*U)+!dKcz?(>eJs-_TU*;nxK1-) zb@!#pV4mZEOGwb3C6ikBg%aS2_h4_Yn14Avpf@g^8s(JGvrW9Yr~{ugB2Ds-fxo)} zb38JMUW0M4+-+?l`LKiT#id56TEE?!?j`yUm;ujDTC|!ip;K_kG zc$7CgzsC>2;@hL9dMn+)*E3=!2XWoGm{UsoM zokaJjubR=K!dEfa!Cd^?8Ot<;Iwx9=93C)J_6jo>;d%LX6Vx|+%S%1No-pJ6Ydoe) zr&ILJD@gEIBNEK@cJq0=H(2;lF*;@2QAeEdtPvW?+eh+DpNVZ-Rov(`1FM|>EouG(AoAcG(HEFPvsUp-zU{N04KR<+ZslYQ> zL=M4uSrj%4zs8fhB6A3P6|sb=Qt=;ueu&Ot>Q(x44uVnJWcK=AI-c~4*;WwvP-sSy zQGZx_R-!`NDjg#Zm%Y~65IjpzauM`F|oZi0Fpm{|~EWOu&1lyw?l{5B=E*LR6 zy*=r=Q;cWS^T6F3D<+pT4*@EjZjEHQr~EE}JJgGfRiMtz%JIO#j1o@eDJhd8ucY<< z%ld7Ug-bworWL2fM8+A0S~Z(|&0@3XUq0^notOJaFiTwn{@~Z{lDFE!a`NsP4w?B|>;WVe4Jj3}mC^JF1O5ji)V47Fv@$`t_uPGm3Akkt}^Ok!K!F zlUSVs8f30ojLFm8oZxKq^!4aI#nlG|vM*PE#MlS%K(zVqKNobcmHY9Ewp1fh0HUz) zBnnp+=kmueTkqhRRG3?U5hEC|@_Nv*{9F%yCq8fkB|m&OA_oJDzj8yXes2Xxvv#V` z4KoOCU4Jv80XMRyaqjexe}6BTnwtw)O(Sts`SE4j)t$(wA8+_@;YQR7tCBt5dfd7> zKwkzHWlRwggZ@hpJ122rCQi*Gw!(NW3e%PwM`iK!^ovtjstzr23V!F!eBzT^REPTa z7OC8g?+)}S=zYr7kqK0Q_zbA1zh5oVhx!bK-)Em-S<9k_(}$_g7g+u;KiShMDHQmZ zpNx?c=QQ$W>44MvDM8O}T*x(AxJAJEkd?hqHykc4g-@#oy}NU|^*Xqnmfd3j6-gYC z4|3HKAU-rp%7d?&|#cCs}l3Yao$&`|$fiiN@N2rq5U)*bnPEsU+(M6qI+ zptB*s!5ik}VY@FZTSih}Ket75TXudck-OpoW_^e=desXCKue5X_GSPez=ok{Jde+Li)dcl_0tfP7pvmoLwGk2F^2thj(T+Qz?V36zn*w= z{aEEL^QyfZ%GEsT8g93KloF?Z!>K!C6jQN|H@W-5aE26}S`Zf~t@jAckbeA~$yMry zv7IT~@Fn;S_yO>-aQ$R$lXDiSjQjdnO=L~{EWUzU}q2W;Jtx+1E+#eTu6=g zk-^?#-FN7^ET5fp!E|`+3={Vn>-B^toBKJ!=0ln%_RjKyxC$6dD=oD?Z!{kLq!(v-%yEi;yC<}5q5EKjFe?e05#DvFy0HA2oRJ9TRgL2*QjneP3L0pVuCzt}RmMbt@Ex2-ePiOs zs0$E+p4mQY`KGt@Y=%iATXrE_Nu z((j8n=9Pl}>-4ov+WS@OHYMPvH5t*itidQlK~PKo32rZ21RCJTKG(IVj|SH3QP_e@ z6K~fYABGzHt$u%KJSc}6477kMxohr)2DVD@g#%`bK!nram3!T zK4pz}YI7lc=S+`t^(ojdaN+oqwM=^iU!*Dg9D0}%MPynn^b-*OpLMhyW zI>L6y6e5#F&JGx1=;CbH>X)~5N6+o&wpaKoBlJU#?AG%05#H5jZuf|o?db#St};pr z3huen52{-RIRd1-eZ`c6vJ-MP@5c8qc`*;`O*fbi#x#RC8l0~LaOq-K#^{^%)#5xi zfAczp{2JAmg9CKc3x<+DIj&|*p5){9f0i3r~~y$`&X#CdAr7c=`d z)?4AJR!1E;VxDBXhs^yfc5+^tG1hC5`W06ZhbPDHkoFq5LU>$R$>%!T8A&$jqs5CH zCF)kypTS2dRLDY_`!AJmrU98~FNLbQmDsJ#)c3~!0y<7})JnSGTFt=VMYnV4tJTE?LPhn}oj62snXlQkIgq%M?hRVu~9SvED`PJPH zVmE)2JRb?Fdu()s(chFfY8WHq357PXi5x+0pVCs8BM3$Vl7{qS6wB~s5gs$l| zJ-^|jPHR$U*98vtpNMlvdp#&JNlk3wY9*jImb2a{;u8zMqH$tbg|Cj~5APY<#e8_$ zaAU3E7O1^ar|CUkc@zu{$TNWiN`(eF$7cPKI|;lJMcXd$qTXtkbXf(W1KpJih4hl| zCaQo|koNFZgSEv)otTPMDtNYijPNi#rsj`>4JK_)a{90=?6-Xuk>3{shBG+dT97j( z%M*eliwZL99~F-gx1o#hVMc}9av-1>u3u{WnRmI&5$^sN?PscV8Q4pn*1QJo`evpA zBph4dO-mWjMJxn@fpw**Z8>SBVcS$Ij`zdl0idUj*B$;y{!XtI&ba2rKcRM|xg8&4 zGVAR=WGFYW;n4hHjH?Q8jrWd{hsz;RoKUNm-KZ_3yQfF#>Z(`a?FxbJWfxnkK}DGV zNa7uOT*XRUIv?hgEyed2?T&ecqdz2pT}qPCB66djHi|wS6svFw@vsmvUy%wp%m#*@ zAKtk1&?a9T7$y&A@yZXrCb29v=*2ew`bxqTak&0NlFG6| zv|U34Cqi9TJdiOaH}u{HN(%0{zO2)tM8&@mi{gxXG1PhI+fx~tgART)%4k7AL9$?a z<$b_fubxcJqTR{zcFlJ?^=k=;62hsA8dE<4VU?r#@L)RAOX zRLK;a7gPpt^|4b`TRHTQlwPpf^dw(@JL#~r^xVEbMi@j3zN703AznBoDqCra{8eW$ zpxHJt0 zPvJ8GKm$p*HIo9|*MfNe`&j@6vYPk9z$TwFX4O^H4d+l9kM_OoIbc_Cnt-opQvI*q zUayLQB`cpX{xIP-=8moqN#0!7zQ5ve7RE_M>jwv-YcrckfG!jI(L73n(mpet0x>E` zI&*w%RbI&W;{`h1`5Y)O0vaF&p^}td*s4%Ej;?jOTvy#}%7VM_d=y0`Rdd&C4-nh( zmJB`*1k9bKZ_N75lDGr05>v0tpUeAw!=ik3w?Z7!)Hga*GzCkhA$3k%3~Ik1gO3SjLx(AG?wNkKlbs%Ri!f5?}tGoL2BqeD#% zy09dhGsK|9VT^3^`IgO`ss>?#mdsZxIRmv>JGKvh%knwwVc=FMOw6Suog65a*qtOM zV5+o9)EVD{{ol=M!N;+*L)YD2{gBBlL`jkCGv=g9%-u9_iK0QA;IQXqP6(3(} z1c3Kbbo43BFRw<>6$vqyQ!I%9Tl_YMFNWvr=b)SKkqKB3IJqrri^vMmnhy}-!#*L% z*>%?#SlsmIpCoYp6Se%JCA5_b5s{X$?^Gv^WX0{TwiZ1vHRaIOX-=M&Z=2RE*Nv%X zG>Y%zQyb?SC6v-aD|A&Baq|1)|5~`?0f>+YsowVnN7oNnf+0wvPx21OxZ2%UoA>14B3KH-nC6(wKjt#C{Z^va8F5^Td zqwa8ZjMeK@VF+9P8=0aqBGP5^oAepL_Ny~ZfD#K_kN>!YQOegjXlx3_<_f!b1IdMo zMa%)@x#1+o@{cW-9>#qlUE#In%zm+-vNxgWjgXXkv$|$`Un$N~xxRj-prcDL52l|-_dVclr+j=*S*}~gO;fH*eAAvSqvy&# zoM$weQiL|?8`V!W=0)>ovh=&44%=`!O56IppD}JS_(c42;Q@GvXR?@alDl&LI+i3$8Q_Z6f5Y14<=>N>>T` zD-(q0bwY(9G*FYwXJ9ue-A#~8vP4tIV7|S|`s~1XEaeL@P;(h}oemGUaj9);jKX1y z3c85Cz-aaJ?r5uJBt9KkDNtTfdC>SXM!l-9L6J!kye>yM8s!CEl58p@{2nsH^Uek0 zkr`ucHv2oXk4)!B5f@Gm_pc!ZR~}cShUWFggRpBji*j|%;Z2YqE0-a{39d3g2VGi zQH@Sn$+!?cslL3oe~otb;s!^)EC}9sK`Q`5E#l+rFY&Me1E|L#@??L;_T=Kzac0Wn zy}r)&JR-Zb;nm&}t6utL759W29aQ)fzDBmD`Jx}$K3}fO^#=L3?y6aN=6QA$(HS&_ z5*?i1r~7jP07>M^)imGQ2I@Tp8e#PwN!Bg(9_epzB{lE?`#ps1W==Z|0FEtbJVbrO zN|J0EJaj)@mg`u-=a+80Cj~8XT8Ae{nr%^C$GWo#p0mZaERlszQh@G9;)s6085bm8 zyyeStFt8~*rW|&4+Bksni#8%xt)s=74-8C%RmU4;36ZHHRboHoG>G5(y}GhIp;abV z?UgD?yK6hcla$yUH_IHKSoA#(p0BZ%G7|I3t(P51# z;o%{h^E$zNgUO3Bt$P34b4V)PU+X@Il3mefg5#RB*@*c^PimvA`!g8*1)jyuB7f_? zmB>mOH$cPSe)A$KG=VQ>E0qeq9irn<(E9uox9@gpld5Htq@^m2Y6JqIA~Fh+91C3X zo$&o-o{8$u-XdpW*jk#LNMOn_QGS@~ZSweSarP<7h!(!6QO#h-k9rjk|@* zEew;&pR!fIlrDaHcz#sA9xL=FT!aMc##+izXY-Xj=prt6Tm@?vRXHGs4cJ2T^+wV4jmOWZ^CR;Ia(b}`TP1Q&Abyl zCXo%oXI8Yf2B=;#$S*I`HIJ{GU$B>bt$p>L@zvG&hi?A0ZF(Ku{+vV|vCO}wHDwUp z8))|KivrhPgEqNH}E8+)|l0kdIl6d*(KVUX8t!kbmi;B7r(w75f;w%;1Sb znb$kEbR^I|SK3S#S7CSh_@<7wt~{lZr(d)K&xm1|(RW}!4=fGUV>WfE?wYbpc_f)N zct%_=HJ&gMuP!c%2RiA0x2{*%N8akBXPJyk1c`L5h))*I?QVkzxt~zz9F1*{=x6(4 z6KDAxU8;B#_SqX)&DiBv@`88ZmfOV1#q{J_xT{4+%UrT3weNBpC^DJDV+zJWSbI8s zz&r5X3SPjJ-UwekdS(tOo=ug~@uem=k6WVXw@Z3I$54i_O_iwS?{nIYhaTG}bDF(# z-oJ0^c)(-Skx@4FD)Q=g-bi8eyd?c=HZ)FouFa%cObVMKfR~2ZspdA>GW4E9IIaV?T;y%hc#pS85@GU;V zG%1Q_t`Hf|wNhmwQO7-7nAT=|!A~Y^!6$V3xDuoL5`0q1tCr;a%sGyQZitn7N*d#d z(52TwGNf)K?J0(}-Ei$Q3ax9o&On#?a0**43dl?`osad_{d2TC0dJj^XxHq#E{D(I{nD5oinjqKgvv>IP+KAY_Bi&CgMOC9iJ3^Mnk12je1FY?#8IN z!OIV!H9w+pP|4QTMM$ZW|9m0?SsAih>Xe#=vo(No6N%jF*|Wt-W^jYPvbC3lxv=V` z7PMrTTC_EW=hQ)c0rrK1udk!4*L)|_$3qyNOwrXAT@L-B`xhRba%r5 zL)gnTniLk0_`6$%d zMTb1k{)qC?$>rr>i}enaV7+zVSkCE@?pa_+fZgmRHKm|E5oG#3fY4zF?Q%uj&^CzLk2D`jhz%6oRj#9XWOvNoDxrk`u=&e05TPSP8tQDK*U z5Vyq{Xqr|YQ&^|Yn0#guK!6j^wYxGoITAnv&2Z{&C|DfZC^tNw%`lnY@hDI7X0upL zl3mVfzsDA@-!NI>dZSH`W@Nx~YGJRi$en40Tsd;>2M>V}cG{@)cURRZ&kse(*`r?s zSTv0HTOYy=x`{cze*NmctoLHMJ1u<4o5#lKoC66}-y~O5Q)~KySXJ9aX-33-m3Xyu zvEk!$5Ti^^GdQWCDznG?2YqpQM5qonp~gx{UsJD#>yK7_!i zZ=h$A_m5>axzDF>S?v-TqKn_dS`5kVHqWNulmC{9V~y92?-hB`jHs|(bsRz_&i?j& z0ZTQny`G~d<2aBT9wViokg3_7%AFPH{;{&FSY_M&^aGL+`7({--ebW?Z`7932y@hk zuv<5AhxKY_oJO72v!>-_{tqDma?gVAsStA}TO6nd+aRlL$TSN_V(X!f%49tWV=x+F z7s|`#Jt~I^DXpd6gb@Gp69U_ztNY=^oRL~a4vYIBkQRO_m}V`DY0UibBDa{~rA-zg z=o=$AR^AR!fAP7o zm+?4zI%jWPaOqi;1_RW}tE*HT zr+C_6#p>ducb)Vnp-#jT{-iQh3kIibkJ99kk1UnnL^T+jDvV0O;rL5~GAZz#7tziL zsf@_juQRIlPc!p2Ny~P=_vrTSjTrF>;ynOip)omU^K07C|5kArt7OLIUppJpOZwxy zZlHbLF(+N9^N2#5r=e##)RpT6Sx7ko_DUg<0lzcqak_oq$^#MiK+NS*#iB? zy~r4ZQ~Nw$>(QL7T2cOhj^osnkX`T;(t9%{lm(y$hoIOvRQb|1}-0PBt`hPtA z1d5^{mdz9h2}}-iku(>38_PrPS>>y5!lj%L;A#7tHFd|kATsAca~d{ss`E9VZN~SU!{y0M}j%2>Edg&*EO?R2z}LU)8(! zlnmemfFaMiS?{X<^6IW{c2#1AHI@0% z`npE+TN(Hc(L=r%)AWRQ9jf**-Q;PU(aW#!R(9Sm@Sje0 z-)gtPU-cs3wLrs~G|H3$yY0;S;iopzbm=R9R_3%=jTpdv?@S;}MB;=A3VZZ9p*NQ# z=k!54u=H8iaNko)RLoY|ZGJ>O-zcjJ*Zo;J|44Um=SNq$08)m81$UWnMw_xifdYI@6YqlX-aTxpM9oe>k*CDF@U1yT?SI8>E}16<${K zS0nv>X>u$+u5tpnCW9=fv#egK*Q~MfX}bf7%W_;cDSVG}z}94-$7lClwCQLmKIF4I z31{2ubMXcmt`bs};tyvmrIAJYkW_y$RW^$sUQ7VN&iL&blzFz!`Q!6oZQ3(|>A^1~ zz(SV?Ty&CS1;3Z&ZE$DtX_HF3s?at=&*O^g2?-hBIMQ^DEbNif9-bXi9C=Sb*V1UF z3szH-SI7Hy)48o4^u&+JcukgiYhIfR|5kU#5&Zg;#8FReWBh?iF!wj+CG$0q&vW3W zSx&EZnXUm)T5tWD&&hKV!ZxUrwy34}l7j5*eS0lO=Ea#(!n5>k&=a>@XSp$JFn#Dt z_hNy3mxtx%JWy&N>FjS|-3IMAa*muiKsu zFfr?ieGMyQ@t1=~l91Hj`Kzf|THM!Gol~)2eIVe~-ri=%&CfMcfBU=hW~^aD-k!TO zw6e=iJ135}BHQ|__HQII9H!rtZXe?_ZSY;&qsybTI(+H8S$A>bUGT%r>BIYsb=krU zF2ySmHZv%Kxa|*|Mw-DLadk#2YIK=-9B*Gb6}pZscCTS3bPa+MX1_xc7I=Zas}_j= zw3YaV+bSVK$TL+_|DvSV#?h|Jlv$^|GACz|jMn zD!!*r-y80ZhqIzJGY4VUOeozn3Tw52&W zeh}5<=2?636Y*JTs+3nqsdOiyX{3F2tGao7v=_Y19a)xiSTwqScWR1jtlDxm)-fg7 zZXz~)qJAQBs!$a-iwxB*nmM zAYNS~pFD~zRY&fSmuE0Y_UDAt@Hn+kfnx8+tPY;d{SgorgnYoN$Z9y@+P4V*o)u9B3piOy~*1i*Ak`C0OwyoteK)4Eg? zo%KYRSx&!BvDv-)Ag%nvcUeeDTqB!Z_lq)*llv-<9z9-;vr=-2{$bQlZhicE3H~}6 zrQzX{Zaa;Nw=d7VuasrAS$4~CX6cfajz^S(H%b$@8mey>vGtlOQP{s=Vn1s z7)%l;24Yqa8B6%Yn;k5*a34h@^{)1@-f2C#4A?t2~Y={DV?8qrK=9Gn;85 zut9^Ky0T28M|GZGvICgUPN7tQvT7|;|ATFOFYj8ka_7+N6!=i8O}B7dj|d;}EB8bO z?U;D2{*>ov8ld%0*`md@Zrii5DEaD|4%0;oiUsa1#i$8x{dcMR9di(+rh)W+#6izR z?rBdB^M;KLWHR55S$mPIK=Ukb-b}lOgD)k(c)m>jH@B6l13TJ|BO*gY9W4g^ia*KJ ze?3p3*DIRzCaFZ=hm@3T+6 z8ad?Kvnu|U%XJ3+mnJX|VXvV^8%d$sP1?z(-CJZF+8^XJi%RXe%2s_ZsV^%Jphdbl z$unXndwBt~hDRecM$W|RlDRXANRQm77R4jU$5b|hE4)1~?gekKG(jS_krF`Xgt0u~ z@_T<7)FTI)b)`F8MjmlxANEQiDeKTC zUQo}E_qbCTH}T=z<2+3fD8b-Nb{MG$BAwqaF#Bc{YiGOsm53=;(>d<4 zux3X(O|*tT)}p7%I6X`5jysEXl8@RGfVhYG)R?zd7xS*Vj{+FyJJoU}kTBzue|OBY z8b`ATq2eX*TzM$W9X%i_E|9yMgL^BG7r{^3|)KemL{op^UQh$x%###bSj*LY+Q(%xf_ z;VwRy+kz7sc=|a5H#LP9{V~mVU~H*P^bPApnz=mMn~fyau*D^u^(>BG=jCJRts}%? z&ktiUvAhr*p!uiCloN0zlG+o;VR0m%`jbSrVo!~qX2PCUwR`uoE2PStuoAhq+CdU_ zX9P%n3#54iV!9X6HX^1;taR~b#hK~Xs+*8*bPDLFDRhNSqepF!DE zA$J#WXnxZxs(tQZ)V$_!?eXJF`W}lWetX@K^SMecQ*j|^!sy|fw^EK8oOM}Mxek#u zvX$J|yWcGBl2`tkc|a{u+*~vsmBNQ2Evus4?j8Y$A|tQ!PhS;J!75{D4=TMBFGx*- zA{ir$hH%@F$ICNzz1kg6q*q>p*sp1NcWk2P#zB$sdEcm!jVL5F$gRRw3_nOR%Dum5zCBqz{wzw&#BVEyc6`*GzE11%<#aI< z?BQkeu|Jxq*qAu?%?|&D1RXnw(FE_K0Fn4DENmiiWBH(%#KW|RZu09$dnRzYjR$0w z{JQr%+V*&Id^^lMpCbovmCbUr$GMxFq`YqR_s_e2&kN_6l~Rd%hS;CsIOX9XAql9X zxUb_>8=@n#%D(nXKHW2CBR(aBTg_mjijL8j&NN%l-Kq~3Jr&DrJPvJyA-1OB6WQdu zx+XF6jGfm0}tcb%Hnzgl@sust>juC$Go zpJy)yPYouShbPSp9gGhWxD84RLlqfMo~JkD4Y#z{Gp+CYr$w#UzQju#8u;o)MzBp6 zD1L1B&T;RK}xbD@wG-ZU5^<051!yCF#L%xP4Guh`oCxbgM@?1xS*Qt7$ z%L4kYW%oE#o&k@EP70L0r;K#jAUxhZa_K2M+DJWcX9TZUY1uumfy>+H@6zZsJM^1* zcd6_=doKa#B21c&VR0d>GbQqA;4RE=h;7~YM4@=#hrUx;0UH&rso3ecp2=h=q*McU z!!MHj9CWYVkHXEjB`5@(ul-6I$ZK?0?&L~xTzjpPZHC!7!3QonCqlf1SIeY0W>Rkt z8PkDQ9+m#Iph)iFf%N0RHXT!~xS}Tuu>m{yt)JP1NGmb#-BqZNT8&BDuSI?8`K18Y zNuJqfm`<>K&E2#p%(hr#4wJbL$PhT+cdl04grn`h#272O6;y&3tMR2pE6!;fe08bG zdt75xO0U^if*6}-+HO0dhkR%m2gmlq3xvHv`M#G7o2UTaR-v`%oDy}bH2dwdpy^Up z#gUU?wlckd+Uvpbg>fUDiGku8F~8qkl0$U`9aQ){@89XtX}yFQ=74yInvOT|#_qSs z)`9pic3km1)4Z2csm`s;0nxD0lQyw!xceKgt5(^vyt`3zn!m}c&vApG6~$%Z-PO2_ zPbrPD{WS`@`Cnxqj7{7E(UYv{Ech$X`)Sff`}iW$)3*0IV>_C&ZCp%~EwigP1fWuI zWX_c}UTl24UMY0iar(Mda(n@9r1u)V>{J5ToB%g8T3Olciq1NTUZT*Y`-@$l^3+ws z`FWhNW2#zZIx_@y$K0tPus^Td|$G3wbI~4CC8MTTd)stDr1MJhbsdX?k$9X!q_J-teqsyFiP2Ta<&^C#gyWh`m8PBqSWJmIc^ zm!d&^#mQo;2z#0D>+Q|iE9`@oLgn^gnYt@w|BIY>7FAtf5>Z1xbfiMgba_P#Yx(@? zp@-Wbn*v6SKXBdQeEzEHpD*9-Cz_3NO`LcdP1X-=da{&&J2+CSFctP+~*FE-eDD+?zh85QpqHla3) zT63esR=7C^m>#ENL=VO8lr)lQcJMZ>>#8UlJ`X+nny-nG3KljWh4IJU6&hd}GDfZ9GuV^6 zNe#sgXbKe0oKOCi_X@`};z6g!R??-zl*{MM6UzqC4H@8b+3YF{;0ast%sY3^`IVAi z?#4YwefkT)&VaY934Hq{qOc(^&SQY6vD4UFQg=T|2^8CQ~Z~O8=*f2V}NsUfv7nY znKxORB#)(LTwgbPEb5QyMw+Xwe}7z$@A`hyx3k}eaO>!g>br!dlhm-8ZHt@&1@nV@#!%(~9B!2W5m7MZ2; zIvEJ}t(37vWrLXw;G2=MK8b}UJ{-FQBz99W*~9@%E^sc#^UQO**^tCvYL7soS|u-pB)Eb}1gtWe7nf&}>sOX*3=LpIpV=|SbcL-|Q+GFB zXJ8PrBG6xRVdd@95XGKn^B^sz`Op%K5^41BxraJ7LA}*~W=uTmT}n*Ry9Qc$Pdsob z1MS(RBs+6RAAcxr%oEi9S~RD$3;j<&bkU4hU5|^s;M=7~S^KECvHJv-qkWcRFg@RO zB{}#L)3fpV`&8po)rgdpyMyNuoS-SqK_Lwqey(!tr#}*GlNe?Lx}jclylS%k*uIRq z_M3CfoygMTGrad&ZuG`^M(*}VUWfJY{%}{9fKvUVa6Z5al0nq+l=;H(P48AC7ky{X zTkeb6MJDM(AwE1l-@HeuIoygN574sIQQG!vx=85HVchDHQ6JP%)2=Sz+LZk52HjxN zy^`{Sv&RwhI7rJ$;&<<})mZFcFSpZ?bXlzw%Yv~aF^z=o!{ehqy z%%SW`r0Bb+Th>vy|4l##l7j(_ih1d@UxF($jDws$m$A34X8DXGd)JOx;y$OOL`5Lr zW$VAG&@mZULLHrR+Ly1v=;e;Knnk(_l|R9G!;LY9zc_udQDOTR%>tskAW>I{ z`(G&KKD|mHh-t{#6EVX-35NvN=Ld0IrS>GB_Y(Xle&(^hH!Ny}fM5D!HJ)xEou>E} zdKyFj;Z!s*aqpChl!vCZ_o z-CoUxst>@2-zv67MgWby^vuO&E>>Ggf7qOf8?m_~j}FmI94`-j6i&8IOA zUGZNoKN(IN;5)&CLOIE#J)dv>QA#vhti?uqUx|(R0iwq_)fSb)k6fUT} zNdp#ALeG|!zyerp_er<_@qjant8KU#ANdy=Ze^-+3~m|*$SgUZBUI)g9QEc@vo(V| z4aU!DW(E4q3Z61N!n`{c_7&UQ1&U?x=4-%NC_vPuibf-A@ycuXO*KB3r&4v++(XiJ zM*C8tv}JDxFG;LwuVottFnM1fumoz7g5OU?U2NbFHF*5Ey{YrMxs4kGy=c&OX++2w z{5X^No*Z0<@O&R~m&~rB!y3KVsSh)A{Y0!R9|rlTd=ju}jdk<>&#{O(-^i*FVt(;0 zkSwk{pOJ_r>UM3tljf|jomxbLP-s60EQNARvSB@B4|ro7r}hw|OJk5yD5SLB+|IrC zSlzO=j6w&s1_0zooy^Ky8?8QgK$aVvl@aV;aTI5fQCQS;kVX`eI`-#0Z+=@i;Mm8n z*V4PI#8zDp^IUNvkOB+&$ET2+w2v;jL>%!q#r@NadFj<=&qi=V%ryJwjq3YvF#O~d3?%ywMBcmWuPnA zG_S^6(kQ>rsTAs#CVk${ll!++bIdKB!w}Cq^2RyI09mLa)Z1@dhR7=j#u$+?7P7Y3 zN6OnII-S2#FIi4}U0iPrrq}5oQL1-Tgdy`qF=F2W_4jvm+IKZ)b1BV2^Hb+MK8ji{ zNZtxsBWSE@rLG)X`b)jhT`r_NHK#kXRkRVDO&c+VSn-!wg~`}fnItTGDGtq!9AP|D zJo6FYsibX&dH*oYi{b_CcfYPPSRP5G2+}rD2YM!7cdrLxAd?hNqL^aHH4?f!oDK&L zAp{CZ*gk_qNC11#K+QM)PoDNWloT*sR^z;s>Yh}-v}ZKm&6FFqeYlG-emr)C{kUek z3pQ3Qp|@4cYBz21W?a$W#t?ALC(j>+1vKa{hYtyDBBdS)MZNt@uJy9oBUm$Y&e&S`jr4pH+anythCP zN!P4X&|=iiJ7cGdC~Naxws^AF+1Z_!R@e>1p1%z^B8v{sRIub~ebB{e={ zl%v7}QW`pn{ol%kIcUHq`Q%TW*Q#Fgu2<7OBBa*Q2`1_C#468b1hti&k^Xezq$=bV zXo#pFm0Z}!Xvd+L3w5QXJJRY0%PvQLCJy?I@6Ks+{{4>AMJ^Y@^emue9ENtb;O!=_ zl+3X4==JSto)SUXUsr5fHrVyM-K0){DCq#I0^cVT_yZ5XYi6?>gBed8&WX!LYf5(& z66eKIHwQ`#J%sA;c+L3Kc6g$*=Ad3!>?$emTl%iW7cP_TJ<`&dfB%&<|o`OJN_Fo54Z1_JOe(4 zOIir5#eO=y?@o2&#{8f(j)?pkY1;nq7+o; z%I*OP#^VB)1tNsBw3E5S*zy|!ByQ@ncyX)VDZzNNYn@dK3%Tc0?w5-*_oSUKw0$fN zKau#(Hn%q}cwL=AAkeOBH0OWNh1+rq<&1YN_Eh4=Og`hY+j-=#w;ho&`6@0$I%7$u zB7NOEfNYq`Y%fWPomM!I@1Z!|C}@k&(eRmp5jAu~K)t$hk;B;!??h~8#Tbr*zxM$6M zY=`Yx$5AB0Bv0$+-PsAEox5MX82hK5d~qN7#d#pPDxN7@Uq4amO?baUb*R~q6;(Q( zZ7|rFD{SQQhZE2Rku=cVQM`+Q8u18HDr^|d-U1=W=4dz($GILir+P~2 z>DDaFX^EU&->hFX;;S3*V?3<{2^11t5SP@D$>PP`ZDf!l3f9-*LDJJX2cyh_(URre3qeb}|k3nozAzb}iZk zE4#%RH=lsEp9YHS;0S>!t=8oKeBf;wtT@(My>Alntrn0-GAOOoEh@CbG~mUm>Bi48 zNb1?&*vC7YLtBa`Ih^HYOjwSaogw_nMJ`4I{^3;^Hvxi+2Y{#MU2%Fh=Lpj<9*fEI zd(tb`J9V#m;>2=nY`Qe%#j2I+6K=)c^UUkEi?`y%n2!nI0A(35W9O>Fmw~bjq08g} zXK6VNKwnZ=(~`R)5#!e5vVV09PG4Qm>@Y_y2ykDd+dHNCd~6gm+Y$WRU^yMvR@NPXMu9%H9|LF~(%+ouEw=syM2;HG_X)g!yiz2|Z-`oAJAz3i*yj+Qg|O{c{zNSh z9FpAygmDXGLMq^F1M!*bjG)|bl@YPMPsSM+}(;N*RI?_2{20$zfGYUHm9EjCbMv*@epm% zNrHpNcO5VuWGgZ-hfQdEjMC4XzKUN66qE4 zmQZXn$^pb8eSxWmsgWLv#X`^~&k1g)^#KR$s^EWy@_z+G`kafcMT*}DHvBn;@$3pz z*3@wsZHj2~oxAQ@oN1D(6<&JZZ+W=yE_T3FqK72wseGt{Nao!L{{yy>_aKr|xt2AX{ zDgfYAOK&oB9Cn84Da0dS@jF{PJ7U~~^!DT5&4ju83F@QzqLNsxxGDVs6+ESQ1YYSQ z`{~K}eYdd4gO=6xL4DLM&}00n8eW>hGw`)J;>1IU@WQOoZd*nc&JZ$L{(D-^?u4-! zGGI${xdb=o_~*C3H~_z$LWiu~c(DYh@ea=1>39wL*U_as9Ec&(Vv_zgLBi780p0D! zU6;uIX9O6d4zz%9CZYW!uGA$acgNC7xH@iv%G=D!>iw$3-KzEM6#R%kwTolvrzTT- zqF$;#sUz0XA)AR8;XUj3QH(8&cK@1^M2=W_qy(rddZ8Nwi6;2MQOz~s0QUG3($Qjc z{V@SW<)@TpJ+a2x8_Ja4qla3LK72?693h$;OL)a7N5fqqK_#G}Xxxv}>i!535Gi8= zH|qxk(E<~!L#~nQiNv)LalH2(-#}jVrr9`oKvgT21Nx>S{mUS#@hLgp|D@o`3P>S; zis!~ORNjG#!O0K)glv!WhjiR;=9i~!g+z^Ba5kF_j`n9B&(9h~&%U#!1|l&o&{47* z>58)#lv59d=%!ock(pr$#RIl{5d(Y6|G`4bZTfF9!%q)maGC#L_JV8Eq3?7nxze($#AE800WM{Sr5 zYMA1-D{EoIobgO;Cy7ez;lDK^fIRS_r};-~tTfsNwUi z8$JDEQgMu%Eh;2pjdicHlQ>TGX(KYSNRkc?k{qGvM2M8*S-ALuG- zrD})GrgPU_qn4SyqwTZEkP{#=(gRUDC>o8;%^R%?ym~Y8dAbQ8bF3|<<9c((!kypx zWNgM;SIv|84gJ+>8(p7p4}SIvxF7;ZSe#A=<2K{pCU4vpU%yp9DW5RhNTnjz&(C)n zVX_X}7UqU5=UVpd2Q1aYDB^2B-v&PSJE2t=TCXX@WSD1OwT!q)@9b2bVq7{pHgFm- zJohnhE#3FOV{`?|Cti zZRyE7ZT6J#1t{#jO^k=KAm)adCg!;IXp$^%Mo`~qkHVQnZ+7GfW5ypG*-|( zTTZ@XT|&I{I!$HqisOJf`!=8S7BRIXsyA5Zi^?(m5 ztm{AGKC;P(WjNQ6vUB4+?^@s76#jswq{(0KpM4P=aG;!C(1Ak@EK1Qwzrjm8(_e@hywSc9sq3xP{N-# z$NKnYF3>O4?dXjQt)I%pVA0N)TN{!3)8IsdN&fndgUf~*m|FvJ*&(JDBrOi44mYqP zUTWT+6)Imjpa>re@_!a6?h3+Gj~=Gw*Od--VFUU8Gh;JXp+Bbbx$0;>LQ4TCxn^U+ z+JX2a1oNvsmHg*60)Vmp)K$Bwio~%1p1@^xTR`i>dh*KtRp$c;Ka4=0-(Uh_-z(VN zzKEn53H@ZzMRXU4%!-+nrTav@S6+<2g)ew#8XIvW+JN)}FDP<6PoBL?x4^g;GIe?a ziSik{=@l@%j}nhgA)aeSQSWd+3Zj<&d?SY-46`Br=!PN_&s4#Yqh1yE8=*TL|Mg>< z=QstelQATU@3H5%+0kQ*3X+(p$Sc>eBf-~HHdHh6x~)4@)7ky+0`u*=?%X`>|9U3k z;U5iC63Ex1h&+$k!-29QNPiZIgh zHFH;An^jB_+pYeWat4ZX;&`w|q-B}+HbDCPH-ecjX;&*8CV>>)8n+5VKd75t>w6&_ zc&Il=sJt7WF;CQ$Nk2PZdwX`#Zxw4tdZJI_GF9;|P{wNFAaQ3u7@YFLHI}x7EFGDwyb?Rx}(j z4>Gu{4RJaf~ zoo=Z13a@&b^Gzo}x2;W?x32VlGj!g0b6@D6ZF~Z#x7!7qRH{8nKIPfyrN&f?^e5jh3<${5qih(b9^nB)p-eikdnuOv3(6tJ+!wu(uq2s zOQT^iou66b=fis#m^1cDX_(fTJ^N&VMBB6<;LoD+v^5(DmpL+@+6ZWA`R7J}N9yxrj)M zI@aP%FQjq@ap``@X12S)hStbsWDmehW3^mP-XE9&YpMbnplW@<&4==7Yz}4N>gnNV zQ5l7jx{Ho%oNm=FZGbV`n5r81!nASHARD;)JI(Uup}$z>K60~X|Hnx}K`VDMn$2d7 z^+v0BRyC+sVZRMFyK6=fzWW-vdVQmhn#L!OUn}{`k7GhJzI%I8roex zB0JqGIt>XkbkXtRXkyf%zwFk}EA!slhEY(W?)a+b>NC$*fHtZ$P^G($#BO+%bwy8z=rn}~hhMx;`_T$F6@Nh9B8jP@5?0cT7=9hS>+2Nwy zn085SgNf3(89?cx)CVu)`-*7xJNqaEwK&aYcnYL#x#CuKi(82#=udaBv^RVmhl)jK ztOWR$ED7DK`264OWW360_wgRrk4w?A!7$x7J`H%0C+hy!n}E9{o>|#{`7v2gHy4R& zRCoPjKB;&Lhg1+tT9Q5(3C%729SlS>NRZnJTv?;Yf;QsB=eTqtK0tWmlHMOUz`NER zTZcxyN`e!`yPG~&%W=HN3N~>MZ4RjT`Dv*D690-Je3qwfXX9k4#uC$N!-!QK2>B*E z+TNcql8OG<%H0oh+*EbEZn?KJn zAFv8aAg?uh(v{Tk^4@jKO%rX9enVQ zpVVXfX$xU}onL5^+9pp!!A0?1B*?f;90Ji${E(*3=s#n3$pJ^pEE$4yyb> z8%;aYEwrI}Ti`WN;I*w7D#fq%|M|onfa}!$i2^Y!JU~@u9*&&gMm8iKj{HJ)9brl& zaXL{T6Irh-2MBEdX4-Lpq^f}h;aXmw6@E~H@ENTWHQ63MyDmHUkCra{P270U=2jq&HWZI{g@^gn}f9oO{v)g~*jOjQkB$k$VjiYb1Wa zzmEd5R#VD|-13mBxEPM{S@-KELhbPZdPEGRS*INoLzFStSM`jeK4Ei!Jis@A zi8v`$KgP+Zp?8$7`vJ(b9IvW|FU=e|;KbAE5aVJklN6)75YV`*wEGx!Ysok@%D!rD9o7RA7}~N%zCZz*x{w$#gaAKeb3px1lq~<(>v|mBcBP{dwO7hnkai>o$LJ694<*7 z?mn0&6Fr4!| z_sYw%!fC^n3GR`@iX|huZ)U2TgQ_HI>+N>%`6s;ezi(K@oA3to)gG?druoQdBlTM< zOV?{mLT$ba2wi|-<)cgTT)m&E+qJq!n6`-by zLxUKSG{LF*GW085Jff)G7t3P7SE_>n(&wh`sPWhVA;ZQnuDs&1ImEi;hy997XL{H@ zF!F<;Hh82Ss>d}=KXU$8%pIgEJYY%bWO~=!yHROW{ZG^&PlE+e8NcRRl@JQq)LdAm z!kM1Ob%#|}7nWX69SJ|gt%;WGJJN~WbQ@JzTz*m=9gwn41;SwhCdfd@EO$&ZMBLaD zNIyE2?Y{pGJs0vanA#0(zw3cKQRXvq5g7!(@E!DImIrke&+Ha|k*)MU?AROBn8`PV z-Qq<7f9pt=eWhn*x2R7Yp}edUr$A0pc6QLlK+><_6^g!0WTW8HILnhu=y?)1AJ1~> zG`hJzGp|Ltciw|F=oKceg>4GAkY=0|@|fkx^`5cQwfQJ_$Vk+&XwP)b$&mU%z>s-^ z^>5c00ar398&uD}_v(8dY~#SU#S!RSmf^$O9g=z)rsC!eI3M?;Go)HWh>CHv%Q==Q; z8z<~7Hm3c{T$Wi$;>BA!Q|>>+MtAa9+-ENOgY%!ge9Ej6c%glgDDfyUG39%P)4Nyf zZe%lMntfxw8LD3IEcOoG8=j{2P_r@@Jjp#Eh;t`*Yiu-FW{ES?P%vdIZ~V@jN2{*M zseTsKdX%nv1&iq;HTPpXe2rbf_-AI^^Lw+sXgjJD%6C*!#|qw7ahk!-c2DJf zm(-pj{$Rs1t)PsKRvX~`Nx}tD8@sdmReRE9R@8!p@Sk6ZExL089YOZQbCd*jM(Qq8 zWKOFuV@;ie0K0|_s`fYcFwC?!uKjwV^6I$&xK;Q7vv9 zT^M$jZCzynE;S}*i24Fwz}@8<^$}$ij{Gqo(1fZ8JNqL%qho6JpX#Q%6EAWW*jy3YhBym{R@8sT^z>QMwA6aj48u8TUJ9)!(# zKW=)ku74hWiC*f@g{E5N<(T3{iL-WP>u2OV$c>e24BZx#QGB-tRK-~oe{k?`c<<-f zk=mq|MQL|6`ZNT-X{zb=jfxT$jTR&p`oL9RYVZ0=bliJAquDVks%-I9E-yTk|0N{K z#)>&kRAVN`H1GZ^byT0`V+XvV^r1Fuob=Qc(njN-}jFb)ih0Mi5pcW(<|0nzSc0LHVLox9-Jbu zHX?Itq_mY)nD}8|!0&ZGNFP77GD>s=j=thF^v)G_%$qfPW1K0@8C7{3*}vo!-L~K$T#;C&wjB(F7oV&g>=}2vhaF6uhj6d z#Fiy#RGQ^$=SaQJEfGg}K_D`KX~-0gccdqJgXPvyopfMn86e2~Ni6YW*Q`cg+_dFJ zx}Ao?f#ux6d;b^esHsBNFFmTP>zp6*-+Q`>OL48&t#=W}^=c>t1cTofS=F9<`Q-$< za+IINJ0z@ETwa$Jm0rt}8h1J^KFQ&tWQ%k6D!??M@n4ddNrNnD6N(eRI~0Y>@?w2H;JY)(!_gt|M4ct}bNO@}k6CZ@h`Zsr;dH}tM$3+u z?VI$rtg|WnkMneI`Wr<-Yqm(qfq?Q!`l*`4RcRt{-d z_TaK@_UK@f)iNcE{LsXs8sEr~Ft^Q4kR(xu525e5&q<9gBM?680;> zfjeUex~Pda9kD2H3_+(z>zT6nSDUSE!w$E@D{f>pRZkmQb5;|Ov0=z=g*-6D8kY{i0+b0Ae8dOY`4`2!rcD+S{mqS5qz9hBxfs%V@m)YJv6b1z)cn6)hjSMKJiH@H#H-c&6%R9tGd7&yUEgCc+tk_LlCy$}#|cuv0fb;V>0ycn z#1Jaz4E!G6N?SQfcYmo5^JHZDnG za+4(`x_ezpBj5p5x0*prWA7EI@kjaKKzm>a<}0bElIp6@22Eb<@1JFnuO0go>w)c( zNw*taz1yY*tzShmnOayd7eDnIvF1w0kfP$OxiK#YzNkB+3lrP7ge9@-(SZ{lRC^-( zqf?c(UJD@%G>WE{z5$OL>yv1GjDK~EI>d0_EoN957*{6;rm9uM;D&B*v{(iY!h0xD zQM&e_Md31ZBa8MbYHw)DK7_G29XF16uRpLL8~DJ4$PVJ@UGTq9X8<3-g@f(?Zar%< zk={bPG;m>wEqOQU`~aBYf++n(E%BbNdN-3P^!=0MHmK)EP!HqZ%!%|d6C{7#scyl6CLJAjFFLy<= zyQqS7x_pBY`qtvx^Fo)tARPBhVKiH6FO6W0B^4UW{&9CT6{wgVyA_7X>Q&Ma)dKyD zhVX4tJo?cwM4K&(_=5VStrm0~H^?|W{9JWQJxt0Q^$;k5 zzOzJk|5(e+dyiX`&Vz^VQ`wxtfjohpd#{7hf?BZw8cUA)w)`>97K$yZkiW%;9oL*SdfkO@ zfBR?oGy` z!qyxNWc5lWO4Q&hf1|zFBtuI?dyQV4aW*n#)*>9saRSIwB6dHl{hSEGEc__r_Y}2n zHFru}E5hn%HH&&jukQDiGa(pXIwZ+v-!ts{9UoOt%j#5gPv1X}7N%9F)1j$bP|*@5 ziwod$xwwzrNC>1F{nFBW!5cZ^l8O@~j{Qn>;12OeR=jgS$@+q&`IQzskCVB_Qd0zz z4~|c)X8rxfgDRbn(Ia%$>@2|T7)a?rKK0;Xi)i`IK|;*r=Qw;>I`*!Ck1VSNczFLq z-djgS)qM}+FoF)T^hhIv=7j+WJ?Gb@I8P?Wn8rPJx8|f)sD_t z0uu;36F+H1C#MDE-w#xN|9rG$EVs$YdB5%yAJOh@cX+2{nld)EC5y5HGO1eVc45eF zOUU*hkkxp*<5hFl7l>>2l5G$2pf79ksuMh+3GMQgk-!(KyH2-W@R6H{e0af; zWod@HL&JV2vO>Hg`s*6tig&oU^S}t@UZ}CW0X(6C*Z8~y}K_x7X_2U-S*LN(23gqrvdq( z{jVMkTpo@JLN4(5ao7s`w_>W|C2jGRT<;@k0p*CRtX=J!TSqUaH6ZU6WjQq%qq!0p z^fe1J#0sqQOrpO%0B5rf%y?P%gimT6Xv}+tspT0BC%Twg2e(T3(?wKDBrb0N)Ol_( zT3IxiqFEibcb`!`V(jI+*i?isEVJH6MK`uwMG>uUvP0mhuej%iYNXbjhw}-mfJKbd z=ezh{pPNH@KW61ZzMcYhqFP{6wmvNY$Ws!6-}Z+OMXIjcFIoy=pVXw+D|uM#+3YGc z(p{#4Z{OmCr{vexKL<`FI|qTwm=@JlqXGOto$q&chsWVb*;pFZYX6K<>do7=$VL1 z?mEC|pe98?=!8Gg1|IXF3$Q0-gCuZ5a4jLh>FXHGu0EeBh$tN^ZS;to4u14Qp_ID2 zL{CnO1#|j(nv1>9N~qUu-o7oCp?Ok@zedjisSCGrNTe&R+*&&AA zwjAXF$ruf5`*S&K;KDo5;2gY z5VLCkD7K+@MjdPI{>_N;N(Kmc)*m7%GciQF7o&KYOZZ~Ca7UUYm+v7VGrQ* z0R3V9r9Wg?gR&w7V4nDxXaWxe^9$&JUG-+>1f!s-ct^(jnQLe?s{!C>Svz@m_d>JR zNifbX?WH9$cF+6UPdoQ~Np3BKSn;dI_qR<6+jg@diP^8j31c-ZQt%2AY&Z>izDZOH zSMK;<(MQ&|9%B&o4U<(2dv$x%R^<8*dKVB?v@e4u1^aC0WKYGGHr!Ht+AORm2ZxS% zSj^ysQC1qVu$gKTjq_oRj=BQ*oMfp&>*#`CCYK#{&V%Ef_nQLzx*XPCTELgomH>73bnG8n7j+iXf`3Vew)0cr5U}y*L}2!ZoK$ z0}Z{acaG6+A?h#hroZrqc1_5RAECNYM-=}_+RfnnIEp{8Q&;?WpowtpG;@xnwRl;nR{ zwn&T@&$XSD)g;g~l)im)r0uKig5G0faTyBw;yW{a@uS9(R$`e|uz8j*B>^#AV{S;v z^G3NfRJcwnM{cPS5MyX{!bM$|SVumSNkE%byBUOX1SgZLAo-3F`|1iOLUgI!H&KIupON!jlkZK&r6-Qmx{% zY!1LcK>pS^AwH_!Sb*{kY%)~mOG)j~v8uG^)8=bgUX6U)z0kr2p@OEj)Q5jh0V+6F;LAk0pWJc+O ztV;Kr?f|6#fcXLjnyhm@sm*@{?D~VqzAcx%+`BXRpi}>;=+KCW#Aqc)(%thGz}R14 z2hYU|;^;qW;c<9-_LBIvwfW}Gsl8q=|Nj@moS^i8uo%C82m2Q{6Q&HSyFBdpB zpw2ON2f_!PDgFFfybA=&rQlSZuFyQ5M5i{8i%)h8{Ng*kXkePUsm?$0-p7Jrv z3|2Zx0mQt}MfPZfx`RXRA+5w!=oKOlPMHNQpu#UKrZ@i#y6k6&j9H<3`9>2 zc-0zBPZCVx1tqa~=1$5ro3b;_G?8?sdMl_XQ4-hHEi~#Js@*<6f@h4|a@ZZR3vKKi zL8kN1WtMJ0mdy_CJyR($Da5Whm$eh-pYLT~J$ZH^mX^^WhUdnDy|Ezag+W2MxePy4?Oy0_l%81ZQNHqu3$5FSAyh2jC(|vWbCJ# z9`Dfo2hjyd`KXK7kD%4dNzNjMi>rENQvR5axA>DS8sE((tnCP@D9zCd=_ZSo`1a_= zS|?iPx)@9^Qdc{tG;kyQ3B+YVicdFnEBz5VeDcK(y@>mAwdkB8q+3PXOS@D&!){8N zjUTh#p1DT<3dF--Z5VRZ>GlNd%l^EOfN*}O@KqZov?cTjmfGc>sdS(9o1i2jX($Gg%>I`}dMK=9vMM)Px{>6y19HD)0o z!r&?3FyM9zQ7<1c9IZvwgC$Q$WpsYmN59!p6Hz8FMbmti)`Q*rkf*uT&_=DL z7qJ=Z94%sImULVS1D_zvN5GVGw>h*TlMi}=)-p=&jdyz^ZmC2R6y}q9>x`R}gpY;A zk!fj;(-MvL71F|0#rj5K1Z&ha`vth57j)Yn!JEaJJtXgzFIk$NRvi-*@A<^4#*(RD zW=Hq9=oZ^?bbR(!cTwiZaxzrq=nyRNA_2*rBIEeEtUKzYbsB5NojY6yM}L|TlVX9M z;maKGQ>ynQA`iv%YaW8rNo>Pl1IL9(sV8pm`b&%}Dush}Mgvrqx03Usz$ba^jaDpD zLYkFUM)KA^PKAb>O3-Qe>(dGW2192&q%QhH-;cTdm7-$69sF8Zj!>tAz8nv@*YVXD zxV%|pRfH@0Q)tmc3^GR&4mnGZ(eQdZx3iLDhnItUbQfW)Q!a|^h?;r?GL4VN`AKyh z^M=Q?5EnEB&Y#&XI@bobI>XAG!u#F_B&qi$MQ`qwX>v2pnl|yCTlqe%!G%3C=JJBA zG=DSEq*X%~9_qodx80#O1DnwDRY$oYjUBf9ELv9xS9{G`q~%#_e}S-w5sML@@eMB3 zW%z|y|KQ@%jXBEmkx)YpTE#>1{kOLh9mBh30}N@mJCH`O%h&F+^3D&(N{{nwIa``U zSJDgtcbb*DP<(Bd-uPQl6^ZBPEt2!S0AkxIl`3skrezbZ&Z0URn{XXw)ujv= zr?EEx!yG9ZH_mz+;eD945giqDecY}2XbDIcN;A)3hW7aT3FOf%YBi?6=^mSkubzr$ z6?hp36o&&mtN2Wqs8Y4W?Y6le1PE7^h0J7G;zrO9mApUn!fI8-5((Q(Y`>Z34nCEi z@NY35FK0etRS8O$`fyG=M~GE zUC6_)g|)3No5isxS6rQO!P}2NUnj+bi-E(!z%~U4P(p<((j(6MRtqa%qMN8XMB5KC;5=3|`k>k7qGGHM~m&Zzz&CY(NLeo01 zdz8Cdz8fKvyxtsVX<{{xLbvmlvA&WNQwamU@mBjbvuY^`lRkQM#h~@Yd!vfW}!%ER=O{LC$rR@OSke(G#TQ=m{L%a#*2K?J0?VHmN zreEz3{;YrhT>k@c$foLcsDO(1yS0!G7<6%lBx(!n4u3wWH=YR~mH3l3wLbxx%aG@C^W7_D z9ibi%vlQ2Y1mdl6gNK@03h>UBMenma zKRBQuDwRGMbSX-_=OWhUWTG+~)CPpy#*I!29+3-p8M((|(7Ug89XIuyHQQBBGnQ-6 z7k7ICEWAX=YGWAFuYsx z5o2|a0ElMACe$uTgKKgeEz6j4T$PO!CTykNHhWar*A6kgK=>*|@D+q=xNDdS0a_sr zWEt)FW^CK;Q<0O5E>Cqstper5e3Ljm^**&V4zbXR1RK(P^}+D`y!$vb)%}WOf$x<8rc56TpT^l1US6AWmdj-e} z2;UNs(H|Cqafy3)j@yy@na|HGr+34P^Ifjk*TS>`LkbXX4#eo2t!Gd0ao4~LV5SI8 zb#&S6#{9jg{-W%V4chU-dMyM`qt4`k*ie<9-J0g zJ5Lx8%@>8%yYju}{p9#tTtge1V&^0f@>^gGd! z^X02(L+6p|PKr+C{^{sFV#3k|36I~MnDFid7A8(@96^Gh2}^_|Ttn>)=JLDk`hhVr zu+@SC_E@F-#++C6vN>J$A-4&Jb4{92c;At6&2U9o52{&Jzkm0HwybDYU4HrjV1z2m zJQ;EgD}|mmG@~V?SXbn*gQp>$Kz#gA>kZ7Z&tb!M`v#qR?ixbY746&w%r3YTRvMRu zS=HtDTJiHSkrW8M4jTu8JlQW^gkUHUg-rJ00XaYqhuq~f&m@+>#87dq!m~u}cNgMA zouoIOO$Q0|)%vdb$R|N3MD^+K#(6qyC`Yp{T(NR=4DKB|PW7y=y@+O|bgj`?h!49+ z`*{=&22P0ILDDRjzWE7=4DMn^EKoZ=5bfr6$RmOi-#Ui5q6RTz=u{4Qnw?sNSG{kc-z&ww|#(0J1W}obaD)@23oGYv=G#xbzzqgVw$eA ze@<;N-U`Hg`H3r{S$AJ~DC4YQo|mF)kR8w|Z-Rhm#-}~yzDz+ow{<4Qh>+>%o%y)D zeeYUvbNXWIgV!RfC@Z%2SV0f6W^~!J=*n~_5)bwZl4Br9PE0Ke8_($L5GjZ)gCBH-1n`DbJRh8 zh45%VegnhOv%R9)q@H>BRGvDwM1JF$Cvw;70XY+n@Vr4*1TNI)gP)z9i@-P-Gsfe+ zE;31>bc!M)n)&r5+lHX70asTlV;czRW>}o?K04`PlI^JIUvIcFmZ~m8R&I)n2GAW- zt1a-SMnIi5T3pV+U|YRf(4-k&lPw{iFYWThCw8lKvki^dBV3<=Gy)kQYceJI5Ej5& z^*sD`Im!F>@JaZH#uO1J&exG=LJMnJxfe)CnS@cN_YSe^wOkULv<942X+iBw#zLC* zS^W=I>P@&qRnz3CHb9K6Tvwvf&Tg>v>kOAy?Bd@UkCv+v=>2=H{QFeWxdX%rF!yr@ zR@&;n0EP&0XOX;$hAz>kaih9+=79X+B-Nhu${-}XH0LZ~Qt`K=g#kqxIFmW)RkUtK zlx(cq^`$0CkR;n#PTyZgH%RY#4aEJtQEIK+nvZB`<`M7`VZ4i$$ynCdJL(;~iFcEv zT56W2s=P1VGCP$Hz2^d(hp^yYzMu^p>T|iwrP_i}Qbtt6bLdT;P$fzfp?c6IvhJ=ra^|q&9GnlEfHuEt$Ohd4_;%*r`DdA}D zsoPv?T*)CUFQFZfENb*oXC1khwT&{Br#QD9kOT#AsP2#Vr z~$6Hy_D;jO3vHL4d-7w zZAj~KZF!mYoUG->IWG2r(9gP%7T+L4JSPFD&nJ+^w-eCKbKDm{h3uzTZCWsZV)C&m zmnm^+do{QFo)2Hmc=ZQJAP|#$Y(5DW)O{DdKF?Yn|8SkJMv0JPFa-2V=s6h8wc5ew zJF?P{jN*KnZm<0PL#bDVm876gn4v7}I55dkO^WU-o*P6gpoUue4?rZ6nbX9ZPe7D6 zIJQ-doweFy(V>um6pNt*rwItLBID=5K>Hy3Ubpq!;AYeQ*QWHs>}EeSs{6-|^BhZF zJj4Ur85BTY#a0GG`IAK1tgb9T<9jUW=c_?qo(v}R4@H;%y!0HR1|*2-Q<4BmX6ulK z0kbW~>3U(qSVXgG-1w?Ckr%eeDJUAc}t zxsC`Mq<(IAp%JU`=hyeG&)t?|Yi^WsjbdI^I+;R~L+dZQ<+Q@;aaoA-P-tI-IJN{8 zkMH@O*c)TSC+l5N!L|vMqOhnOdpMHOGGpvdI`-_emr3Q`y%@|@%sy!wIcN*9%c=W2gAs_aXPwv=Zmp+ei0RK22@Y zY?PD1<(7*w%&5iMiN)@$Q~KlnZ6e(bQOJ#8a$+|%tJcsU?u zR6e(+PpR##PF&QAW!LH3oQd4}lzPJEtbpv+LBr9u#RoQ=FNpEeR7216 zgo;+FurOBkv!_RI=M>DItObxtGH5`6HuC10q0ZFe?stjzpYOV!w+SY2>V9bUC9eKb+dChN@((~c#t zs!7y#7?nAD7<&XueRx@K!j;{kBLKrWuMX)c{{Rnc^&xX1KgNn&a3o~C4XSXzLW*@9Ugt&_+u?N|f#+`<{CNr`UL(;Rug z%`T1xOWrC8^PIcVmSxE{$beOgAdZVSX66yN$!5wQEH~1A+IadTCEHEO8%28oE(LFo zVJu-}){f+ZhtBn9X23@

    ?DMrPNil?mG+b@}n(hk-4DUzS`HxkKVY!NgZF_kggTp z>)~3)++t$&i5R^gxlqUzS%FT%#v^D;6(cVBs2LMMp`%m*tfdGp1F5FrlNG+R^{#wU zUgXJ~laOvOnt+xvNCgP@CO^XDxDzSj9}At5C-_McYolSSa#Hvn`0SZ`tR~4`^X<&$ zAlV2^ERa8je7?nTY_q+P8cFV@L^GmW5K;6#=`H*gsj@f;tG6kTZ88wBrkQ6$>r>q! zBs9AHF224c5}Or*c%Q{@&OkhLgAFORV&QNBwL ziXuw@TtGbCA-=MCI?V9MM=RzmG1ezNKqBqZw@1WJZ$Yfq?YKI|of&{ zfbrZ(eB0x6i(;Cyvcjb=`o&m(hhQO@A$P%x^W!v2(wY98k!HIKSObgCCQv3LG%rx5 zz(6znV{7JUx5PUikr+zj+F&um!>MbEBO6ubGSS*>m2>(h_3vsU82qj(VXHg znP7;%$v!KHS64G0LaV7FvhpyRHNZV{&(YM`Tdi^YG~dVjic@z)cg5cje)X_3ZZO*o zC?8fq*BTu6AYIjzAN{T;;cZ35U-=?38pg<)C$ilT(L%h7GwpDZClNVub}djl;Z#d_ z*)E^Z$ZRd!sNh~x^O~BDMfgeyYPh*g%ks|q)8WMpr-)Ba1y7u~Pfu~+l{A%GZhEHG zXZ=xD^9E=`MDxNtA~m2X%WML+(3i~V)CyYBL^8JDEM7U6lLGV%2mxj+`I3fu1IpQ9 zBh@+4P|E$IHIH_)_tlMCue)&MI@)_La6gl?b6jVO0Qvp*UR3ZbSX}Ln>DiI=w7(PT zcj|rlz2i(SGenp)y|wLjNn&P@g$JfR9Ed=^%Osx3_*y6|GJpWk{X^-if91kFENVq2 zE#scuv-|bHPka^ao>Hp*L%<-oKnUdO1AOrgR^&_OA5C)bXrMGiG%*p&ZvcMBr-O-W zTXPB6(C3uufzl+%4w0Z`xDp|&#Cuh(Y=HAW{uK}P`vt$6;DhQjYHdBs9gO8nO&RI+ zUyJA9UR(1_@ZBDzj9kb8$r7z#{)hkW^yhv5JbQ5uaGrCC!LF^Czmg-aO|K}ZVYZV% zH7Te@A^CzT%2^EC-#&Guz#Oa`rhX-xB3obg1E=^Usy-kXIo?@(TFdaz?K&O!`ad$r ztk@y-qU)MRz#?6*)PMiM21+!Tz9chz@^`@NzD#WX3iSG4!>PN$@!zM8_DTP~4p|_F zE>s<<>;DA}NKIk_rKF?)0dl71LwLU-6Q}#D3;e4k0LTJynT_W#INn}L1}K8^2eN0# z%<4mt$6o(Dy1vpO`&rFEahuBwn_i!gT=-XI2V^G(WOWG(h~_^t{s)nNuigS!kP(h&VFWe#@RzT7dZ7C|JP-2lefKS-c z7dJKTGHLB-mFQ0z{Gf6Ic0N9U>AjKc0;ILF-vn)u&Hl=h18|2ai{wtW znU6YXea~}O-i`mF&TsPK9g}#V#_eyGB*h-D<@|+dIN(j;1iNEL+`e_uQQ1^ z0+!<^jPt}AcGpD%|4`aBSv>+D>k_iZ$HY9Od<;TH1OA}^-?N3W4Pv5}eQzxK>y`BvB#z!mP23NHNf zuQy1BAmf7e?7txWq48^CyY;j$ss4Z%Tm%69ZCen8tYZ6bsQ+~fM63W>YB8?bOdrNcPtM^t}C5q|^NB_6*tCxHBW;D6%;c;g#n;w1kO7Eo^CKi>aCxph4H zWw_`68{Oms^pEM>Kg9lb5u}mlJ~V~j#fJXtrN3^KYmvQN;X_=1di`21oOhx%7+XW{~#54_5SgE zpAh1F{c|8z1YTxCE{%6NKUZ*lK5~OG5u-B7h!#HIs*~84KJZfPn3N014-rI;R)%Gt zO7Tl>;qwBX(>FE|_`geFf{9aFyJA_$EFsB-u@CANCZ0!x<=ilaaZOuI7it!1moudZ zUe}>!QuilP65eqQU!*F0+?jc@w-O;&lQwnkHCpM`o5TYLzO4#F%Kc1+R(H#oL(*~} z|C<~wA)Xi8)1Uk|eWtIXI~~khoDw3YUmpi_kwpk?*OGK*u40LEY4jNYw*HT~HZ31E zVMRz?hd9Jfs?JN4G~`0|!n>lTrsg#JgQ5rXWOFiu!jmWMcrjF0(Ye$OJI9L=V)rL) zbnpNi1lI$W2^Dxi8-VCP{%~XgiEAv9ONndYB^Bntl?Bv|mPK~6XfIy2Jky-rG_6e4 zfm`S8zLlyuH?AZ(zOEROt|XqN?^C?E^yTy!B04%+x@W|=bKrq?+5D_5?0k|zuj;Zs zH8s^LdNSPkzgEm&A7Brift)sfNh}Ne(S12_Hg(0oLUTCYLul=7uP^V>^V=3&d(Ari zMevq6lHAlN*aEV=W_kj8jJ?Gw;_3ypkZ%sbYqGE(4gmd7>1t{o29M2-7v*i)B?~w% z5B+IBfX1)HS|uXu3vkwCfP5?-{-j#;F zM4`58LG#GTk}J-?AyPapj=ty1f!H9o6vZ9Va$<&JA7SWz13=^u)SYXXUzs5ly)(ntLnElJUra$0ZQNzw0c|e zhw+|bg6QViQG4un;;DB1QNVt`e&Q%#w?>iGe2MAtCvo_Tw|v4cbg3+M-%YZXaNA>p z=ukkS;Gc$USd(#VN6H1=D^?SIhcC>#(^C6L73CCVWj}9{;=SZYN5u!TgKCvHTiAT{=8s!$(2={R5MuvfJT#6i39@y1$O^Jeep25sIUr0lyh=SxOVf-Mc1H zdf5>~E(4*9ldMXo$`XV~7~i0y&g8vqf0*?F0Hux&7`DzYs0g_J{lf=?4CiR}HAfe^ zPv?dR@UEZFOZcUXw3HJqDMh@r%j>kTu&{HNdn|lC9uac9;g?yZzyO!{eqY*YEx+CP zGa3pAmEqybf%F%t)w{V-kjF~e#M&lXReX=(Lx(Vf+tzsIf?Vp0FY>jjD^-6KI(yfrq)8$c)3c9{Uq5^F~fnf-x0>VmO|5@`lr2bTMG;N!40~Cr?q-f%;uhHsV z5zTLs88gXLqd$DWVMXM}-*!L_6q$xGFUlhp!FGBRxS<5u*Zz1*v-@`>zyOAE?BR)D zsRshnJP80+i_%|fU$DpJuzRk_lXgT75Ru3=6hh+sjKuE}XyQf>;D?Vh_|_l7KtaXA z|G9feXI7(aY(0R6kK%tWG5_fnt$vqW5Ua};PFh~Y7tLri33QQs@5h?08}jw{q{I^l-7GyiY-SEDOJ-|6Y?W;YgzvGVQnt_!Kg`*P9Xym& zDS*AH!r>Ukn`Hiiz@6Sad|9EQBcvi2wUq!z4>bGAL^;{Lb0m#IQ|&=^Y$g)-&2XwZ z<+jW83axYU)f-?Qj!J%8n|AtLI5dpZi&lXx_rVL|=`QQ6OYhBMSiT*OHZyySE&K?^ zvM@VH-m=AjH3UvkYQ)vVHGGktNYs>Xhb=1sq;PZO6+EE_d zj#IRoN(~nEy=D!wtfPt@TfBNuJu1arR@6#_nt_C=XjPR~PiQ6e0>TWOKs-*vL^Hib zVw0F6>TzPIQSK+KUNcQ!Tdd_aQfuZkj$SPM-5wM4Oqvd|=@E(xtR0ePh=U2srfnFH z-||jXuX(W>b{Vul&674Yhbtq^i7vX%$BG%l^U z%)DsENbpKaD&|}nbqd>KN>&toS9`Aasi|&i*SpuU5X;N+X*?0tYw-!f)@YMwLi?-y z*KfgX6La0$XhU7An8~x!>Cdzc+p43YxW3XJwyipm2Q8!APqBc$G+|TPDZx>3w(e z1TaJ;|6{YNXqZB`%wlueJY)Oa{1@PT5k`?jHik48N*vFd22LTZ?*!1SVb=N~WK42V z1X0T|5vre)THAkA#(YebLC*?N&RG3?k`$OSz9N*8P$#0@`FiPH{CcI6@9>H1!h2Q2G0*fJzRt(WZ$exf#TQ)G!jKu}Im! z9ntby!F^x+#92#hyPA0$aFo(%hBrfO4+1@tM8S=Jc>ARj5fZK9bU!exmuS?Tx7ue9 zZZ@QTCr863;&)$r*gGQ_M|)0^Yu^yAc%Le3O;1YAKVS0B;8B3N>p)|Qkw<8;H*j`T z?4>iVxoTH>JIm{_rC!NL65bvsU^b6)#2L4pA}gd+lIO8hl6RA2YT1gaV3R4e%ID2k zhtP(o?a3{KIZ?#$NH1+|#JiNS2I=`8O4L->hSYIr%Pdcm_vozxCw=sBCaM5GBeb?V zr|tD*)s3zrw`Dk@(9a?F)j1XPg+ezszpd(5vbVPrS36Y_sj(07Q5w2hW%Y)8s`JtD zW9WeDqwim= z4~flx|5_ga+&bY2;SzGw{(t`7umVK)*=2Rm;eUqiUyxP;4A+G8Y0SR_=Kl;bJ`8Q% z9%DK~y5kqu{twXp4JHa&&s{(u=rE|@{s!~MBhalQ+Dx0|tBmY6h8EBGganynKHH2R z*)JZM=C4yixKU)y%2aGL^UGAW$o7HpgL}%Dp}dkBV%m!@Zc>iIf-osZeF7_d9+m_s zFO#s@@GaIWUo;o4-?DnpW1Ka0sQfHzs#JLl@2fL@SL1*S{=)$AnA?Hr*ng*QMI<99 zmsRX*m&F)znSLAIpGo6Qn=2Y|Fk<5*D+}@7-{#llKjMR^R)O2w+6J~$JXgoqQdTFQ z1r*pveXc(drf>hWd!D{f&C41YQ{f+yU@cG2 z9|HtF0wyf8UegM>KD4j&5_I5EGq4*cHfb;7c4Tp|TMc}?NOA4_Nh<99b;tr$2*?~|Jrzapn6gs!|&ErgoX#LzO8JEC1WvDN~R}&K-6;@uATenWv~)Rs-@7( zfD8(E)$mSB(U9-4;vEvsa(-|7a8l;t6Ru{17SDzdp>;_UI z!T#vNfV<-zzmhJ@c7bQSfGDrP`Al=_K@8_o7yrR~5_Va;aG#X19qZT@K5Lh+o-aM8{lMu)0m^B`SG+;GCd*@iQtsWRIvrP+^N4kKYo_+TE8 z*(9+BWd@WMyTq(q#m1AV%wd0Rs9kP-gqo!byAeHs#euUCpW{{0VrW3b>5C!^af0%|3b*6N?xW zM{{oL1Lvd^0~Z*lh*gIgTRS)`Aa(VExca@F5<9k!ir%{7ZrIgNF21G?Pn_l5slmZ7 z(go7vq9aviatf)m+&v0-_vs>yFv!cVnF}0h9R_e6&-^k0mwmDItDnzDU{9qf_WJmH zQ(?Al^a@wDxq9(x(Q2#RO81A0ZR!W8`CM{#vc&SX4LpZ!Y!D{*dwue@*Ga4j42Hw^ zr+%tbS?|2cD<(;1=?>KqST(q%Y0Ewp$A`9Gf;CfUR>AjK$~i`|-9@&=zI6RUP?_3w z70|$r@8K^A-7fft=|(;=9)8hMQc9i zOXORXOVu_OpKBJYt>i*l#ek3HpoLCc(~ORX#E@7d8?t4rS_w$9tfID+cX7rIiv z$oUQibT(|HT$CU%{Vf#5$?+p|+SpQCWSrN>T{gBZW)T^jrgCZrNuyP6I(+;k_8%P) zq&a|XgKPV2K zWxC!N$>TvS1p<1hG=krXJuAF=-q?vbgm}74s7+cIGhtKg^7kG;i-YYyUo^v+-F-9k zu4aQS=F|_Cq|`d??C+^D6oyNydq-i)k8R_Z2 zk0&>Ss~l-@1Td-nu8?!M3| z;L2+h<0muksjFPcS5H*c)q$RR;3RM@lAT(d_D@C&$7^@yMs|ZWc%h?-pQNf4NiMRa z!v}Lh;`3FPfbTsI!YO%&duV-!4zX%?q3_k3G@VoLe5uTwKn3cLFUX6J=H^dG`nzG1x)xu;eO-pKL{hW$Y!TnOeZsRL zqZJ={&JJn7>RdN;OT^$&e!Sp7z1MzXPh3PczFsc)QmG6@wErS&<{{ zBVRE9b@(F*_ir7St_ZAOA0r-dK=Ihsr76X>rJ-`6nT*9S2D64-9X^NGVd3TNZ ziXp@R#w}phoxg6*5=jp+)Q})*suFKI>6WTy1ytK&wI?l~W3^s-9#IL~2SYVWqo0pG zP4g}8wLLLgjwjaQXOs43@)NJc|2h6^Ek!QNY6SZCM;7CzYr94v7RiQ`%=dYCVm)T3 zBMfPH^c`!-JD)pK`?k<)BFLuVg+Q0TSFlv8Eag*q%7RbnI>PlafnC$XsGB6wyPT z9WVo|S;gq2%o%jNNQ&iB+alk}_Z${lFJfO4UHN1kJep&NE#a>^LG&%`7k~If^AXvn zVRUk&$RK5`bTYk{rnkT@dF-mVmnLSCRVpCX#>UX_mXx~$`!U)pM}vGHxW++%!hJq# z^=VsY-l1muCavD?4e^6|e^KxcSEQZlh(PM<_o+iXN*vJIan_GQVH^l6MA$%FsOp2) zNR_BJMI?fdh-kK=%ZGN)*Dyb);a>iF!*Xx;b1pzsjjvnZ6rK#a7kVZ+-OZTS%(%np z7^QMbRHRj^U6|bD_Rgfr4&gJ){rI(bKGDN3*K2WM%)uKFW;tElldCbkQD9xix!+3$ zUz%X;9g|w0p5n^uNAkal*eDAtycZ#OE{}mjit*9ygBAJK&<)9hvl3{UBReST8E}q? z(@9a_pDVn%D5%_In~n#V4FOg5#NBC-@ApAPnISSRPO4v6@Qo^N2e)(uoOiiJ z#9s1WoDK^ca34?_%5{^nyXHQFg%F|RN`Cay;e9eHmA`)LMx_pNjZAqfu3AG%Tg21C zQe=oae4m;;r(T#Cyn0E87_V3z?@7!#_`Wd|xJS0SlIWOnsI~8FRQ85uZA|Ov31+9G z{gnit5RJ9xM!)mC@$3F`xz@A|&=GX5^p*(kT=jukk+o1x`+`g9dcW&q-xolt5Yp0p zF>ZLyigWV!0O*XDdsq{u=Aq*HYfc(A>q-2tZcO%~0J62$M}2sxdBCS%8dx$?k8?^%NA7$?1;C z+|`Q7+M0HW6mIogyol-x^1N~DAgih2GG`3OF8ij!_qFr7y{Rwa@8Wapl15%87Ek#H zoT*#|(Ppvn%|3b(LNdDBqsz~eZ20i5dIu-D&EphzIq{_=zb2e>yQnU=Z?PuMo|N!> z(Tn5mnyGH=w1^8as9Gs$im3D3lnL{-!MxMqu$ro2Hn$U2AM;HnkF`3j)eDuZYBIXE zL*Awm4DXXgw9}nri^*nkq~3B9dtL5m?ljV@q5Wj{hhqfG)^B+) z5CpE_i&S3g@vIe(b|8y)81-gdWGTO_8IGnQf*%4tTl|L+e!Y#E%GaiSqyi?M4#rOP z*|WYJZ2nSx?$bN>N(5)B-1@8Ak9h*9VPTRj<%`?p)K{BBAK3yiNk&E;!qi7yd#$P$ zKO*<%1K#`W)VYaz9jf|HDjsDE4@b@1M&tSNP2;)~ht0jikD?w)ug?2_nR?U;NE{E= z{wTW2ti#QjtfoPON=2MUT%}a*B4SFv4NRZ%KkwUTk%@d@b@lnViJMfjCs*S#$;Ejy zyv}^V+%}58lgIzluweK?_1vvG(tYuBEOmZZt(CfRgk!}o`$K+8f5z^|i64<7dVsq3 z#{`wDH7AJ!sp{2CvwYKIJlWY&Z0eTC;SjfquG8W>9zL5i+`ubjewUv9U}WYyaH=ku z+<}Z1MiQKD#+~dwwe;Xr&=r#u+8F)8^Zq@!&ur4eW;jGB+3jPl?PV+LJM*JPp4{oz z8325Pj1OOWbPy1{@NDPVDX9<0N1Cwm(`i;-&|V)XCAPe9V^`Yl1#M17_nLi3jgm4d z>TWdh%-&%kCeoqPOMwt*wrHi)x{$Yo#z^4tl1n*oJkh9+Td99nK(>~QQ`#_YRpTpK zKm8hYJBt5?|4*csXMt~dmE*VRCP9`i-29uHBTVLrL*Qb*%J9v%8|tHc`u6z5jsSaM zD6EX)qhir2Zu{O~PEt}Nepl*|ygYa37vQ^NU{{d9ud}6{>7@X0wgiL$F|+Ptz06^) z7EIy?;^VyeD{_o_x|6%6Pk~DByF~sUHv`s>3^d?f50Hbq6jpv}g02TkB9%m1!1VTC zj-7vwOn=Xq*B5Mny5ivP|LCvr=YPHU83gd(OXWv2|7(lCmNNT0fK3Yvo9Z7=$^T}7 z1606y`<)<)-QUNN|E(SH+z$(|)9D+}3s*?2>Rdf^Uw2nJbxx%&O3O2%!%T2gk=PyyG1)IzM@B=p9R3=cO#Qhi`E!(4Ke|{ z?GUGGRrCG4`)?n^KMWC8gcZz^I@rKu9w&&mYQifc;Vw!U-@xacKRVG@*3tUzr3`S| zh?T#^O_?q*6bCPrslBbNZrMJ5ZGD~=`$*Sg8Sz@0a#5tuVLi}~HQ;;aqQ;!*m15rN zH*IzSd>lX%o9NtDPi=7|)FX-r=}k-$!`(cTKdXCrR6}eLQstC4#5Gh|({J`g#L@*4 zjPVF`ym1Hn7PSG=9R&E@L_jV=B-QT1rTiYx>{gx8SYgEe*q(xc8_vXC-VsPo#kyn{ zUl>8j_S?hboEs;(uB*X3V0{@;g}V!c;ReTi3=Ni*lg8#@;reK%&z(6J$_8f!ZlHca z3pbcB*XA1`i7=X>%`|8gu?)tgO0wYu7E=D^d!RV;^-|K&p;=cYce5gN4-F1z97WnW zz~uMrd3VrQ=eS}qAGfRI=|+(E#nTe^Gp6LiU450xqtFW+w+!PDL%w9V1Me@y8fu0W zuWA7!HWkQa+0trWcdVDEUM(ZxZix#2P1F`}k=VWvHGf0OdW@9>PdE2&VW@IvOk9+# zBTQi+$XA=(h-;}*3HAv(vs zIocBtu56=6KZb`NAwRjLD&H6f4J;{__DnoC^t##21v67K_iou2=#~<&GMnxpW&@i= z@4EfcMk;hy)b`WrPGik(O{MgfRe^W3)sai>NL}OesI=&H=da@ovvEdQDC=^aqJ^es zlS4g&?88>&#?7y-Kaf{{)2L2R9yD|Nin*^8J`$+rOB>aKC)|IK6@2$Mt^Qx8vBq0q z2?66j=79Tu&0+F5ptiv7%L=af&Aa`VzX&WF|Ns7@M~DIMP;p0?xm^dzQbk!|Q=;`7 zLdRt20HEN5P0knc56%m_G(`czs8txq>@T{}>;6m}CZyeKH!_=<3vOx}nydN6pdf^q za$k6AYH9)oG1W!IjE9PBKsH|aGC)p3;&Dz+PN6IF>S-h?ue8-vaZW6)!h3K1;ehX$ zf0Fm_QO{vNKPwgxKCShoZVB7Ktlczw;>zN^&rb%S=(I<`238u-Je>}jZDO#_+g3&M z!R!$PCbZ)aCiU+{+ISR(z(ykZ6R_UA*PHhG@am&wEAH%oThgp?mD}LM!mG5?OE1!& zbV1A8&My7jr5IL0!PEu64R}kP7W#P)D?CrH@rLpB`WojU-Sg4?d9QDe;IFnsMb@s- zec#XsqL>5=moGP zoj>?J(iH-LADVX*v5s*s`iVa)E8bMWrd=sdc&|K?DZaVE+rqQrfGGqp04;`g@b}tO z>Zv1#n7Y&283w5j_^ndE^n`)&Z%e|LvYRPR-Y4YU3$_?PT_{!gEb?&5JTxpUi^XkGDrTQh z4+Yc~ed#*X$DYZnuTrbHbbD{9s^ANh!Y15 zg}yI(k~$@-_tZc*rRRm^sfFr)ort#Tz|dR>PR{S8SEQK+Q=6P?7XE1O-a4<0#%E85 zM)t1!@-)H2d1o)xMbo*t1m~8`tdw0&bun?o2oZp!xqu66{`7-dwITQMse3tX(^((U z_-A=a!kddX&WGQ`)8}t)#*kH-l;yVPvsjz%`gUgSLD3}iS@b_xsubq*f(NuuD;{(1_bgGW+=Uiv8j0GVFv<{WCa4~4XXGd_` zb3;%iPfIXK@R+vD1JJf<&bI9BaWCXzERu;$O@>bGmb6bk*}Tr5YHWM$pNjt2WoOEw zL+B@DQX<_8ApSMliRk;Pr;<1|Z_-hXL|bLy77m8{iyxW(YyVnBh1;1d&AW>{bK8MH z+9ijV7r~0&-aesr_zHOn>5~hgC2CDc$!ROuTGf^_?Jgc+YG49@i-|L?R)F1$>8VZEQ{)AB^RccLi;voR{KCYF967$hqQ!)gm1AzC0M+H-5)fjjjGH1tn*D$*V;|km zW_&M~C!=A`6%XBWTPo@D7%gValq$q#`_kTS=8F#^I>D82ZQ~oiPtxgre9`O7Njo|; zhaDeqPJZ|ZZhmNs9NTkSSkEQf#hHd0MEbkza2x%qv($2ogJ2L)-{bSbD@x?qf}uxW z$PKuFJ))l`^ytstm5zOpoPgb6e<#q%oqsUduIJ;aT<5r!ua>H%)mU7j`}MV98OAqL zVSJAZN>6n?mz=YL%mcMLarut`=TOQZGjVh~l0a>fd(RWDR-)g_&eO*q7G+v>b#fq= zBb|`xVBE#KLV~#bS(C>Pt7QIbQYbFtcF1IDtlNTLatwCtW_*IgXdX@%|5)zPsFQM9 zQKdNcf!`0Ir28H3tNrAI&FF{VFre;fjHj&om!P<{+PORpxPQzmg^=nph z*?$e(nc`MAHK*1znuRBErfjjjImD@HDfLp0r+@QaBJ$dQe@XMbn*G<-LB3#AL_H+@ zYK)&`FY5rRWt9?FVR#h<0IkmPdbDm>clkyyAv5{3!h8S``_xSbT)rGeLpI9|3vU4{gn`<2dpOpde5?;up3OSe5 z2PZ{q%9jq}PC26BGHiNo3xKI@R+rL?DmxQ<5Qi{`UOIsWGMc>ZTqNw~%0HCaO^kZ% zdk!C6fva2x_<)+51i~i#45A|*UTX7_J$E2G(Frt+Zexq2<)!Fgj-&XSQnaK}1=huR zOKX#%!f+z4(Xk=HoY`lW%*?G2Z_oF@E;*Vu?`6?yykDtA;;CAwmU~Uwm>}eCS49~b zK_*;!OT%roFix<+ldqe{MSU#W0e_9bllk@tnV;zC?sL_G&|gW5HhK!cf7ED41Czc| z0;`5S>`{aWIV+G~#w1iJ;r%Y>g6IP5TGC$TDWslVJ?O61h7l?4`-Jg(8eDR2Ob(fa z=_Tdll7QyIrtlfmdgg@NW!pQL8&50S-oSj#vGcmyu94FzGFWYUNvEQQ1tQ;(U*r5k;|Vw0|`4ykXz5^ zlJm-U<|$i+zC>b$>E?KEtezLcAeh~3Fltjuabk*5V=^rvCqF*jRQ7tSoeMdVm%9qodZ$|5qCSTP`H z<#tPN#hu-zScD*uqE{Z|45LxNq|-LLI%ruLFM#(W1YYhsdJgJwQ&wKIAUhC62CHn$B{1+UZh z4-#p@aeR8{%Ju1_M4#I_@BA-i)0HNw+o$Zl>L)U1thnce`*Q~T^B&*P5RO8DT*5<~ z@bu3-Lc-?pT%{BA_H!dVJ=+=2m!cFCjN)8mJ#0VC&Zg4a&laf>ShkMl2FWDQm*f=u z)}R$jJA^DQw6&Yldjb`lSCb8k7Xz!EJ6lBq{Ur8kM-k00>zx{M@ojOt(^{9{t|&0{)Z7Bxcr zu-eW)3{O&sv4^1HwR4)IqcXOrsM9g}?rFj+7T>QpQx9=j59bHq zHaT_6dtUYCZJpNYx#erFqOv-ze-b@o33u`8s^g#};^jn^2}k$9nQCGW_IiW1wSQqd*(cD(wv8G>kOsW-V zz01t3AVnN6(GCuMO8-(U7%gwSN_OMwozuovj=CO2cd=OdyT{M5T)B{jnboL)czSr4 z5Ppt*`lkOvcZ3z%qA^#r-j80xZYDAOAYcx=r@p<cT4q8{{BcTVQqUwzTeOb2#NxA4yWRiX&zj zjjTW22J(>vcbBik{GK0c>}KUFio+>kg`nKc&As_nXy;QRBAMW6sB)<{uMEEyv5m0b ze#&}EOb#*N5XB3a`=v)XD4O?(1``3=mr6n+=$sz7&w2j1|8@HI7xY_`Ia;dSj*${P z-20kO+Y@CQV2+!2c-1~Lr29Zh-e@0<9g8ispi}Dz)EP&c2+cNiUw&>-x2@+3Zs>J} zD-El=1!~5DUK6{%DnN6WIk>0+qV zS|3Q2WK=I6-3xm)n8cf03Zzy5!hTd3i14p!*dROxU|wf!skv$wd>N@}s76zu!aJ_U z=cV(zK|(3aC2|d1&T=_co5uX7e}KR@4}we1XVtA zowNzDUw+w&cKsQqO>?ubO(RVL$m=#WW1Rh8+58J2oA-ug^FvRm|A?`BuwgSXoS1G`|Oy~(32O=fW4d?#peDz(NMy8Y2f(0nn2m`S7E%&O!#VbMPY0D%l%vu zJ5XTF2(cgd{)o^PkQZr<|3ic|)bN%?3;;N_9^)4*vpv-I;Ndro!J*Tll9*gTf-r7f z&LbfZ*l2Tic5OE{{nv{5+^5L-GYcqyX3US&p$eX@W> zS1+x7aGiQ@xB$ZR>(OF7R)Q5?o47j+W0R63rY1k;+3N032%+LdWI3cqxNP%20mMa@j@JY~!HRnTtRT|X z%mlR^9CPMLmN2;FAIsG$7p<2RKuX+3ni>~}@@t#yN^cI$RyH$8F-F5}Y)6mgMz-V& zmFjRwc)jDmr8+Oy%IFT%Y`oju>yJzMH$Ma@D3=tva;eqwK*Owx6+JvI#TG{9N9F7164Z$X_3OD{Ata}eD4UkY(J?FhzjC9|9o`&{tsD3 zZNlj_o28CHD0n7DIDMdmJ?l8Nr%T`L3V(H)pl>l)R_RgDRZNYq+XBbduwR>lYSGsC zD5)gQ+bE0FDDR@k?ql{wFuH^4RD?d!DZ>tHcEp}-Mp2*cN>#0q8_|@VG^M_;s&+1u zcGm-m*1^Y^rJbQ_m%oa6=gark0F-1VuZ}b-O8x`EI-m@W^Y;e)0BIH6`hAOgXq{}2 zUk5_4u*5hu24t7HDTo-v-*eI}cWOkzShMvmM7{M#4h|0We5KJaRbPo~3J%f4SyJcv z^rg>Jvkz6`oVDqgdN{Plbt**Y(c-ACK=Lrkn=n$7Q3?(KjXCm&%+e zt`XPIleHlq(=8{8Or3}cSYAsy4E45bU56jkAwc!5fI@StGlO+xAeC>FU%-nd zFDeH7>VRddtN`7DJ35mxymm?(8cRFG~|vnMRk2Rp=eSlc&QM`$e`rleu=aUW&9QQy2jOiu&c0mLtT5TUP1}R>*FYt6ffa(% z!jjefuT27k9i_>*kix7S$>cHGD9+xlTDVQb!h;kFBgtYWrY~kmJ;E9#1_*^3STXSR zNrm3x9%|22aJ)t67L&M)Gu>KkTbWDu%Q@Y-W81Yq)GSE<01({m*0{pZ=gEWtDb}Ie zZpy0qCDk59sX)j~YJ3Q1!cnWXd?u%I{rs$8?R4t0pKC9;KqX5}ZU9OVs-&bDiCo&e zZi&t7u}md*VVybT2$zV&PxQL}vSs&QoLtZ^PVUQLt>ER_-|LVV`L`4!MU+1j(oazP zljQf-JNUS7UdZvR`h!*(BO@bmgLX#hONh}(zaGOKmPP}JWTI3TVg@?9Y@NKK3IbNL z0DDS*uL*l1hZ8)6k2rvni-*zt-wC%>tcRKuvJ2aKRX@1PIGg#FIrmDep=E%Z^C!qG z42Mq0^;wf0q$DjJYWwi|ZeEhCS@``G#%@t^#Ot55kR;q!b@Xl}$#l_H{D9)RZ(S!D zkN&ob0oV4BmRyGSW^omHd?VV|S-qCU`Iwpe&AU+bVW5GQMfX`M$5vRcO0Kg-(#=3Z zk}=J9^p}rdw-lv-e6o9tbo<4bz-4dY(&sR^3snvb;sPvh6$Ph(E{Y z=}YELU{F=oo?IVrTlm~1M;NRTIjjS_w{p^lnrVkS2Q*_x!s_OO@g7baZRZEpW zY)_VmLDZOYzJBG|;Us1Tgo6(P$bnC`j6!@c5$G=t6FrRth*tgvG62B-0YZp$9Fu6I z;31vjuZ0WYQ|oDoRtyr{&yfZsMPJo9VVW>Z7KHNuxbT0)`aj-azJU9*g#Dt54*xd} zElXvP$5T<4;C3-bdXAJ5;4cc7$4WB6F~&YhH&>d){-K-oTu_ke%6I!n!voF6Fw!aI z-^1b8g=zQ$6%H;w0>`f?cz}S0nZHxOZn5`zh%P12A3Gpu9QA4Gl#=pHPSjol^S4Pw zi_aN9eUSgSIKYbc-M$y6A7;b*`2c?Z+uDa<&1Ja+ZT?WCmx)BAW_Y&x*2x8~cS}ku zSNJ|vuIT)}P){Ue9shh|3~cA|UQke*2uiAxz>JM`g|It5>V$h%yR&FAOHfc2se$<; zw)J9d=rWPrxq^3hBuKUW(rWEXV?bxUVXc{oU37SoS>!7;q#!Z?S^8$zKVtlQcmKtk zJa~u!u$rAe#MElZ*)xF5R~D{dtCtF!MHHm`9#q~L1Zp^Lp0MW5610+&miYlAa95K= zHfL~7OKEfUREA8#iOW+b@e!*idVqUHK6>!k^Phh|;5$w^bayAj8eKTq1xYBjEb zPaOzx^L$G?jGdSz3EeX$O8(B0A+QYVZT5^WSQPp-A9m#Jua%EOuGv z>UMdo@Zs$m9mBVoCkaSHtiCi)WJg}OYnntcw>d2VCtypjUGL11ljg6*hG)iMDFSRrhx*N=Su_YZ|#Qh*LDViF!i z3uYs~a@;|^jEdUVLE>uu9Qox3r+lf!qnG82=4}7izBD3QKl=_1pB#X+N_EXKVc*k; zZw0&;Z-Q}COJ)SB04FecB%rd0k^HO_PK`V_SW>?#7QPwumPlIxEa|u<0NdX0x z5H)7J!u31b0DqK+9yU7|k!VNrbP(k<@v~|7P*Y3@8aR#V=m%L_YO8^eUGv4)JTC8M7PqM5}R`m*?b9=pA$}ckBsvl)(#2 zW5gbBP2=<^{-`y883On|eUr}ELc9j&DnsP z$lMbkxbh4YT3(51cFD&{O*V`DaY``~ONLD`8|2Ax$xy+sy zUJZVvAztMqvZxs06K9iUlmF8Tfcukmwl9JY9D{MbKAe$Qfd7cY`#rRqgHRn-iEc4x zL;iPfPv7?8Yn7dvV<81`J!GQCommN<+#m^%)0yyLI@aviVvBmzmMW5RO+WbeB0zqF zH0R@o2Uxo*XqdHYf1rFH|1dUQ@55+=8`=(~=mRpw9B#=L`5pI~K0sdXHE{6y-%U{+ zP>Sa?T<;eM_(PBFGFm3B?ei_u|94_H|6_(PatQh(IEik88ulM8jE+tCBp4&__MW0l zqB0npH%(@VlVi@ncAULKv7KYEe`DJ8fnz+$am{vAKKhe{D0~_Y>_3|>Fc2nNWxkQ; zlQz;LM&kLREABA^ZcZppp#GfM{cYJVkBsJ3P@gfpPTk*p?F716#x46>?K-Du;B+_l zXn??Nv(<27Uz7~t=+VQU zcH*nY=a(}1?aX^F{2nU7)dir>ixd^WF1UwA%F{7dz`@I6O(+4H91>$`^>E5T4tsSx;psOJWM zSU(iVPu4GWfLmX7#CmxQ{vJqmIl+4%M8bZEN8hM(sp#&GI!=pAxFadX<{eM9ooD6o z+^K=!5XIhbW~J5BpWb`_@?caJI7~&_^1#ZuR%L%``ruto-rIUeOjo+!mtr$$j}cXm zY?1_%vOG#yG4gZMV#7n+WnR&lnwjLcGL6#@EEfL)>#`h5U@%q~DS=0smRp&-DXet@ zt7czo>l`C&Yv|oJMvJc->bulo6GXZ&6|tlfUA}fvikY`=sNew6cT!mS6`Mx04ShkN zKv=0qc_pk!^Jgjc$Ae4ex{;f|7T5$|4O$%>k1{hB$Tw-J)cTSDI%p<95~7q-7(aJH zOdDUQGD^zOo5oVn|EaJ!i@3;#5P0Mu#!6_j4dQ8iE!n9@F`mz5`^HnO=*xv{9>vG! zeqh&wIk8A#cEugXtXy;sGYD8tyTG^IYW~GdoyohJkEuFh^Q4Wl%yJ7?;M$hOyG`_3 zcOd`B$9nOH6Ep?=^o?fe7<3O8Yi7{wN)0>77`NWLPBlm@{Y;Gk-yG)yIA6^C z4iEW?3&)zL8z-!q_2oT`Aq^U-7T{>6ij)}-@EB7BEhQcc*?oUH z=$(O{I_UJ_-)`hp5fLzUgZmxDM1lUS-0|K1xng2PPJ!ourB>svDW*ElV4Yi7GFy6& z?~5b*yZ>}u6~#jv50|yaEB!FGAy4q8$e+YZ3Cm|U@GnT#&Mia(u8EgQN=nW?$>1bo z`JpAYqrHUuDWBVFxl&3>%GAyh?NdC}0yWKs=9Fk2;2=FpgYn-KdB^}Aj4g`~=XnXf ztNFNodx`Umo+0u9Ti~wC*(i}1eWJda`$0Gcv{yh{#(XS;Gu-DM$FcGA_hJ3oD^_et z*Dbaj3uw$5l$d3?b^qRkESL{wZxj)sB*#Dl+T8)QY82whk!6P(nP!rG639OmA>SP z9$8}KWt}J|^*V9vxA?Ye^__;CA*O6H?6`F+h3ym(p9q;2znnDQ^pf$uXDB|Tg%mMm zX|BfO)^RaZ#+hw(Q9Mb2GKp#QBguV@vac)c|Nf5c_m)6Th1_$*RCWx12IL=a+TK2z zzZyhqt;Lz(}R%QvNzVLkIH&P%TBthS33zNQ_!lzxt8 z@78406(N&TBQ7P{<`KKaak35dR@b(cBZYNV96y&a5ics@1UtljOroU)h63AQ^SDSQ)~e3m;ad$(2I< zqdeLQvwI{L!kOCW&EoLHAzN3i_c(G-XXSwDR3ptq6SCZ=0wQxUyZj(NsfJm@`|h%s ze6X~OiM68GK*~7!2U!?TyH>?6$Rk-C>;aLZ>rZUos_#Q*Tm|~+{LDt9XkM&bYtFD4 zERdErxXOddHuol$3S*-U-B?6(3J8o+(u*!LJ~z5`*r6P@i)rfhb23eVI|JG~Msy}HnLG#n$SJLMYy3Me-ihQ>8KU3@5E}Ndy0}as~cDhRmdqUInKSjiwi!A ztnhAi4C%xwT?t%aaw5UvOGhTIm82zAf+8d3HB?A8bY>qX@<{9(L_7F~cB)6MEu@)B z9cEOek9Anogqk8j#F*|L>!{zwhzb2pL6M%r&2rs~9YhiW{&dd49X7+#d3x1)s#M;! z=EE|pdLHA!R6)s{zPQm$)l*%FBk@q;A?huM;B|u}PV;R?26mJJV6F>)0Vn}eB~H$P z-9NstXVvuppjwZaKAoI6$C!Qt@QyaGoADf&p;0c?cATwGnTNzST3Bg%X>^^>b0w#C zvv-4ZZ15iNzR8mcOC5%nP7EGmAO#UVRQ(0iPpu%%CSLMqcX#c61q^&jFJERlE#lg( zh3(B%I_kF`~F)^|o%uv}8J1riuYP-&(Gjo`QQwj@-lB*zH82=LBc=1j>r3VP#5?cA^PGv}A`Zu~YCo5%5uxNYez+FrWpi<|hDtz$0@n6`hqTbkH z0*Y?v%f(n`3i6d~I>+C6&Jk8w$FrE!*PH2Tv747GUxRaPzjP_s>k^nflKT^?Qg8_@ z)WgP2k4I8H(bCZ zj@r?r(hPaJMZDeL?`4#K%b6(5bXFFOUBj-0AtSO}n(zMT+4ls@C%gQcmEKS?8Spcw zpiiGRWoLUJAyQE}c)AJHRueVU-ON*?RAoF3rU!Tku$%co1y~Mfu~LF51H|XHxCKp4 zOfip8bpzK3!AnfbA&0ozj&HLeq|YVZ3Vcr>z|d^@kyt0Rj^S`7YZy)4f3!as#E9Ci z7@83LP)h4!qN92nc#tOPfDeW$HK9O0m>RZY0VWPu zhaUm|?;K3XOyBid_MfvY>JgJxWNY}~Qf541+%qZYq)5&timoi)xRmZ-iD>v>N(PYx z)=xk&Wr6x~ANlDr1t}={@Yy8WW42feY?(*6m3zxrEN|ZO?A;)LExpNjFYp^~!GuFl zR#vtEYSMhbzI-y{Md_$>_o_^$c77speK2jKCc11>x8D83`h8;*)ja3F8y`chO$y2w z%bH`F?_S#KRt5tgM6`9t{QURVuwUGH>vk{&HIbI*B zkbDHRxX2guDCK*c$^g6aR4m^V2aB5{g2n*-|W46G$dGIYrYt2g4{Jg zRI3PdEtm&Akrs{6WGModxw0_lYCog^Qm1c8{!k3uc0wAy^aG@uNH40E!HR+=f$Smr zoKhckc^d1Rm2boDyQ^+_P%2aWa z%u~ytCzK$l-v6Y6^>qnB7lbcvx3z&_f4OUuN56>4h_CTa#1f0EV{SA~b zCU!d1dR1-@ZxLTw-<$1&X~nD%^O4ApXj`BB4S!OVlH84KkIb6J;%TxLR!WDy@qFER z^k5#1z9Z5{u|mnar_45*rShc;2$<#Ae;23yFt;Y~P_ep0UY|F3S#b@I&|dlwpB~Lt zoz$A1wBVVd>YXdTPgsh};b{SSBv^KO)wn-SM5puMF1b%ms~jHnIU)sd z2Na9zwI{=SV!(MKn1J5)dwYbL(m`aHn%~}VR=i)c)tBb8u;6pRraoN8$FyjpaD$)M zumxQPit8E2LRpb`aoj(uc2(T8Q*^UyU~3*x#I@-38VWPEyR6ajn85$eWg2N-+%_b_ z8ea@rj*X65ETlZ*NL$Q~C&wsPx7I5p@w=ahta5L|hDnyIE#?BSS*R^#KAC@Dx&LFo zWR@T;htwbD$)_z342D@aRFDPpSl^wmDq2phf@~}b{A#T8z$Mzl`55{x^z_Gys>0e2 zeI*cn`Y%fizVsqn*yyoXZ_gc=PqVX*1x4ORT27US1M1t1j-XeBXWPflkUbKYFms@} zgrMLLzwygKZRfEy!_^Jr`^en>wBlr6-J{HlJUFv-x`BFEnQ_MPCD? z{EzI`X5anPRl3AVk&0#p0lcIuBKBZ2Z4g5dFs)s>Rr2&6WbuT6m~Wzi>>7t$mF^cjC9g`z zqx_JkkL;#~*)e;fMers+^&UNi;Iq#BMck4o4i1cI;y5dJM}_c+f^pcrFfgaCEPs69 z@$;dYw8H&k*} zOBqvhDN%mIGe;P|{E|?FKpn4Nen~akgkmg{z)ecTNLbeIOhg3<*}E`HSa+sZp}P(w zACd)(4wog)X#f>jlQaO@S}3SGUiGx}Tw@;p{8D>jo#%8Yy+MTkzE|h*ZeNjUjH^NW zK8e?~l13DRj>W1%Sbt4+F6~(MDP>ixo}6A*e#mAk7h6J3x}1Rg8*WJEW6gl`8V?w; zd*p`q@(IFYxdR@Gx7KA}3+GkcFykZGeFwg-qjxyS|JoMGLWB}kLGnQRfxnXFJS7Ny zzU+&a5JYnLzB8gg&F7Mubz55$yxxTQbPux9vR>6Gk`cfl+MQgR@qA6O0fIF#PVLc- z>UMV;=8evL?)56}f)11AJ!XCdntcGApK zHLT!%9bAXLxvphq>y;oire%8vCdxIrsl|)b6eVK9WdzUkwQ$JB%@Uf0_tPB#)Pum6 zb3S$a*ReT_GJG<6h}}3?iMlHa<$W$|3bWDwvI4fgf^!fM>q@Q^FZ=+YkU&EALMN-R zdFgH7=TE>>r_qd;gF$IhVO@puhuZpf1#B&-ViK6ZNkYHLUE$aBTq-v5)@aX`Gfl*~ z*x>UlPZ?NB&~`I;oR3E35o@^VNC1qK3=Zqg$D2l2 zwu_4YD$pM>An#HKoJlyaY;)G*mO=w@>P?Zh<4ZBon(mSNWb4^ z-P~mlvt7P~eBmwA%}5qBhz@5-U}As!7HfZ1ny5k+#Jgn%4}AC~)ceok^)GPyUvC0_ zkOK^?`$u-qL@zk6x{z~?!V0uQ2rI81x|BmD>Pdj*ErOJ`ISs-x4p!z0#9%`o$|%pg?dK?5CtWVCL-X;(wDsndmW=q zKVdYb!~mDGHZK3)Sq~s|0N$cFm@HMD>IcmZnL+dbjw$Ga-bEbcy%9m@(UY)^U7QHV z#}55-D*_OKsNlP0D)$I#Hg@Ur#WrF_n+s7|7nJT)V~@9;v+D=*yf(Npc3@xwczLw; z%0c=9WrybrwG6Mmv`0z}cn4xY&O*rl;}kI=5O1O58Ov)DHxp&DYye{={d|Y z^x$o#Q9ru~WO)?x>7ksKF6Btm=V2$JJfX2`zF-B-n)27zgylE=0K$0@f+NXQMbZ#)vWfl)cf?I#ou@MKW-cI48gm}hwRM$!2<+mB-xd( zi<%vp&m*3Bgd7maJ9RWM32Nw>5(~wJ2sFT8l#Z@Va z%hpVD6+0mzc-cPMbn1P%d(D!51P)>^Et4xQK|;#gA9w&H_-O&;@vd{if@aOyDB7xZ zNp?xBtc^~eQAb^mKj3Gc55pz?;{~|4?Y{LCkLcp{ptb<#=uRR~e5*g- z{Mq&xNb~-W*C!ExF~uRzMEd7P|Hm*t2>3w*TlFG^fmHFoM)s_pB ze_pMinBB=;CzS4-w0`hu_hCmfloBpa59vqRy0$geYE3Xij*~QV`4PmkH^3b5jFPvf z$&cONy@m|uEC7N50xJP0Bun{Oj`zE9qO}uSl8>@bwd)b33XnO>X3f%U=C%m}oG_6j zVPU+IpEdM9t~w1t-b+G-MKvMx8QmU1xD%!#P5H_NAG5;NR}_^jS+TX7jJW53+ZZ}$ znM3pgeiVPB?iq%>9$!p}rYpZ`B_cEqGg^xLt-MF(*ZNn?4En2tL*{_mge~CYBZpa< zWzGscSK;Sl0bx=AP&K+1Q=v0!|IhI(2dZ`%jVJtk=>O00qyY-o{_GDG|Hp>>>s#MoasWf#-A#`FE9PKV z>YoU#{(r^N^!?2q|7Xno{0fo`Y-HBTv&R0%$o%JjK>&l_DVs$1Zw6A1D`>L>7mLs050T7f>GeYk zgk^K?@AyfF4}yi?6@APQUZVs$H~723DJ{fC!TrZ*e}4Hr_`o;Yj>fOZR0`;h1E8Gq zyLs}dnQ0iOnSH7ZV?)OqikC-g98HyGgQF|(>Eo+K^s&yz^%ByTO%3N6-^5YI)=84- zXlUL8EyN4#hVD-2_;qQ#FL_~%Hu!jB(2Ct4B8^WrIy!prqLHTuu*TB`Tc$oG3w(1OiwFBLYyYsu-dR3^D^*EkUT}_@DN2P0+ zEnGrOgghNc2dUH1sOk*~;6~Z{Zg5bR18!S%^xYvyi>jW&zwmh@)TP>=4d42fO!mxz5pVFVYv)}kbjL2sm zN{udxW3M&V6@0Ln#`|(u!T$HlKNe>7t>D!rhKV|}Km$pm>vG8?N_xV1w=cr$7iQy8 zp3;5K#lgdq)ZK561yG(8UZ*7K1o}Nf(>h}Sop5D(I?Zzh!(_CYEYbOA?_6Ga+{V=o zsh4VqS4;t5nU@Aom(XFpfC#m^CUMVXnDCokvVTcFV*8dOli>Xlb6%IO+bj#0j%0rY~ejz)zT{HAsy_k(lAQotzep3!Z5WJlUBv>i_0>G2#Im zn;t&|Bh-$V4Y8Ep#@&*=b_zjouxQI2TcmAMQzeW4V6980{(8TQOe}YF$-cdVRr%lt zEG|B%<42T7T#`@7GQ19&Bey3C6|(e5#4#2ZQ@2%$Bp1Eo>prGt#a0y0b(4kX;f8CE zo(|Z3K40)nw{2K^QeWD8S`}>eiuYT(-;EUDp3QqcB%lz6dnXakpyJy1L=S`poVgh8 zk9GZEHj%S1t%XsOzAN%&o2#^4bJ`91M7~bg{(-A-(`nc2_-9+n`j#Dd=aU|#+Cf`r zW4w(OMxkoX6Tz8#2TzMlh)SiSjpsSD$E&Dkr1fPDxPF`8pVv2yrOMCJd*uU%!T>-F zY=vJ~iCjpQ`HswD%LtKV>Fmp#&(j$^ZZ{O&e?{~zqQ}va9_6Y{Eqvv1PtoUhnuO&c zD`;&q2rodBq-0f3ekL`o7{A@SebUvMOKVG%=7)ANbVAKSo!lSu0AtV_BO?mcB0XlA z-nD$dzuPztp|?@M(*((WvvBH`_3bEHQmCMg-F$gz(6Dp6trM@x1|O zX0qLNrZ_m!~hX1PMA(yG?YZ?mAJ|)5zKOOX*)OyInM*REy3rf z_X{zCXDnU`O^}88h53WS%Qe@$y?!@WZj*%pG?$Hlj%uPim2ZUHHnOwl);YcDex2u& zj#s{u9NJpM{vzbqFOGa$8;)Iv`0n)TNQ`fKwh!bjdi@jhb5t4CMjQ1vl;aEIaC(?Ig~0aqB_n#lZa`@{zUlRyW(8wQ`#y@WpORBVTfVJu6ad+gD5&6pTTeAVLp zkJKa0W)-x7wDZ6tFHPiDz&1M>24 zz#;6phN4=gG*HSb=ym#C&I46RHb*2E)XyR-{~nD*cB~~igDG$9v*9hrA<}~Y5gLP&TYtBWJw4Z( zCP)6Ri|C2PR4cWz3=gOEk*k573sl5M*65HFUUqbMIkkWmo8>CKS_iqL^NrLT8KuPp zePs}E=6C{dRyhJ}4?06A&;BF`Sa|*O5TtWKRq0`c`boPVJzIQLM1t03O4eT76&s7- z!DXt}GA8y1PiPKkDpbNji(Sh+iiFz6%>ukzGWLJ%$uvM8SKB-8AASNz9R3w%L$>Nh z$*wl$-^|Enn;jSf-(MaV-E&sT6sGj}do472JPQsZ)Xsj5Rqwt;;o?7?%4alIMvf- z6OgZO;k|uIjwymL=OO%k-lM^o#II~F;xAv5HMhK~mtYL9g2gb`L0t6`r?Wf2h4^qG zwY2p)Fk4$CfI|bU>FgiC5m4A$g+I>ZfzRnVk_@okW+@{2ij@pk_nc+h_uYzGeF#Pz zUD3SNLa{>miRR}y?R?sw`+$D}lK1_*vnjO9EUi{wUiQEdFz)j5inI3Nf>Tn8`yG|w z`GlqeUTHCE7G^K^L|0Yp_NAnoS!FRh?LHw=V6ZXe0`^{Vg59t|f<39RIP~=MgL#Dp zkIOH{l93~qPHi>YUnDi6=tGdc9s~1fY%ZeqsNs!Py>YiZ;qg9xBq4Kz05r=KgLUhx z@4ROtzhGvFEIFnpOuyA6BpG(A(3%`L*leDyggCqInV*hQ}I{&29vbh@O2_O-OhBjc0bs z^bDiYVzR&L2|+h?$w!@`RF3@$hdgUai}3Qllq2#i&h{5%*7_1$7_Zl461MW{ghxIe z_$d;z0IqZ@O6G-b#8ZqOWw$egi;A#wGOR(czpF{nSP5XVB2|0K#)ii z$)jyz|N8NVj~WI1GBG?_X9xTu6NTy%1IjI;K0sgReA|>-I6*^WSKW{0e_bcd9GN(o zX+G)C-&?hF79L&uYkRr?en^jSXfVmKrWmjXRgtU!TD4Sv+5^N!exIowHFvi@Y(uu9 z-jLpd{^x;udDVWUSb8AD zPJ?F(=<#b6TorfHf>YN~WvG4n7r32MYZNEi@w;E9;;Eo^u{yyu5;feI7~0~ z*nX|P(z0UJ%A?}q#CZ}6oYi|>g1W;3E?;cZieQz@FGnz1d z1|l&*o=w-})-07h3y+SDIoW?mFE&GSd(LVlEMs-oJXPfqZLF}9)jc&KXq7!w8PCX*$~I3vZNMB(YV{PQ{-O4_?L%!0z9Q{!OTC$W38t1S#$+Am+ixisF3dY%WmjWUrX_}XSa2xxu&ee3$+ zf&Aj>mzKp;ARSdd6XXXx*zskGLi6wa&U(#&3I=p#vs4yG$tPLA75x(yJwf8RPp{Im z3s!mkdH{jvu6^%)?R{MvE(|w8Iv}5_3ajG7MtR`I z5O#u969n(a3i<*zs$k{B2F&mxg94Ep-!%({w{Qxe0>$&O@q~V{fl(P zrG$ACLStprTyn4lNA9il4@ta*ffSw$;PIFg88_8JdW5S0?cq_)@(arFZI8M2p6i87 zx7r=_${gk zznICoC%>WrHo}0=P?aQ>yDk-xBLI1CooP33%JHZy<0ha(%CF&>dL>X|&Q>=c65r+i zA^y(z7{NDNR|XZy(Kt;SFGP-V29XEfDmV)fU%WZ><4AIed@RiEz+{m3L1#_+?GXNN za}?-sX=<8Z$1m;D(@9plRXOO>Qm5*D+j5XvxnHHoNx6E@y=oB;Cy_Z>YJ$h;>^Eg4 zUMdp&Kt(xOSHie|R#a=)4oA1CRMrcmk*vL>z8&=-C-pqyTgu#rN{~a}*~O3@HgQb@ zQB@+{O5UQMr7%zc?pdm;H*lHV^o)Jhg=+n#s;BBrRX$ZMBsG!-9tsMyGny|aFz$2$ zjWL#xfH7VheyHGGr|%4C=W6~md#kg9q`il%^YjB(BprfK7SX)g7wM34@BmCH{{dyb z1%4A`8Y{gkU1z0N(@fUE-jwb+e}kyCUISJTv)+E86hdc_Va4t?>jYz)u23x>DbL~V zl=kSVG`5VH^JRk$Fbur5WdqMlM!BBf_X^sRn7%a(jZvcOjZKM%s#4Z^v!&X(&@Yna zw7I^e>s69hC51K1B!Y{U3m!p#FB(BWcz6~A)%r7lC&qa>7%q)rIa$)9DKI!mwK5P& zJ}*#bS!8t4D{>*0TqSI9OQY0)uf5DYIUW@A=yn1Kx>y%-$;B~RK8Tu&sK_$B#pEuf z>sZ-#WT~O9VUaW_Ea-J9xH(-SAdlj*yZjKu1;e<73X-i4xxOZr?Y{!v&Opa&j9R&c znJdUxCtZzdm8;ifg=?slW3`AwAvFQISb{ej#+4Y^6EbU4RJU@+PKXMG9jnfV4k4vH z(Th0+tt)t7&|aUO)?U9M35%w4Orl%)GDM1ZmHy6hNwLrl#1(p)^NkVvmDl2IZz^9NYk(DjSt$(|bW>g%lB51`>#_g4A+?cuiWv zx^DwoN~dco$X3v5Y8c2mQDuv(Bsm2s)ouMvC>KU|^imOd1{jjHn&$UG@++x%ka_#> zjpH$2n$A~Ej9@Xow~h|%F8Q?)=C(vfqHc@J?LA!urQi2WW{x5H(9UVxI<%~qt ztS)yu1Ee?-1C1QQL=;MM_x;FDKk4`jen`(%j0gn$jt!FwbR zsqUXm-#?wL!Czg#XLtcfOy#)%`Qy)0{35b-7pI9tst8rQO81kZf!;OvU74%aw<6gd ziD8vtH3=nV19MGM-}@RSt~OupjkEfp5ft1&K}_R2J58_2)yyz&*(0uZ1B`of{6!6h zL(Y^3{%Fm@Vhyj|DgMwWOQ{K5kqWlj%Qf<<83e$w%~)}*6Wv?x8kA5Bcr|%Ca zz(%UP2q(Vzm#X9WC48L`+!{tVHS`Y*DRnP8)!I*^9Kz^&?vr+f>cs2btoX1Dzfi!5 zDm`y7Yd)m{V|0-e?QdWJn)MysW?8ZIzE4PtYlBM~B(!YG+q@6;SqF{6yV(lK0*)%ec8dX@73t+)L@}m@i|pL*KDNQ|I6F{g@-oKd zGVPNiHTk0a6(?EPBLVhRO>9?Q(>hLMwA@bKtLSM*mBJK%v2)9#rCLU%&lCeK7;&a2%uV|FwBJf$eLXFs+*}~H8#~VBh@nzoHtRbVAn1H zj{W>vYg3UWaR!X<@LjAVcUXmj+mz^(w?0{T-)8s?A2D|MV6U^2>(x<^zy%@JwnBwU zmfS^9<&OuZN1Wy|cxGJb3iyKJcV%BI8wGd@+EjNFvTQd1-BMq)6vd1^oAOEEwot$q z?u%C|sf{)2zIb-jNrz0bI(HGVCuI=Fn z`Xb*+c!|G5YE*hCpF7vOA<~(_lopbN`3lJcPXWDFh0!<_Wg-T^%u7QC?UCa1_Aipf z-Lpp%&y_F&VhgrZ#00M%_P`z^9boPpS{T_GDFdZn;H)5Mm<)y)m7b-ci=J7sxt)<| zEHin1#CD;UuE%$#hpk+Oc?NiT?IEN>O8J`pG#QqznVC6F5sCXmynd8(cBHdkR|vkQ zbet~=KqZE5kfhFImA55yt!>gK*KKhz4BM~2t}c5j7XtWyC_VKGA8J|UR;`qsw6L%r~{;7W+|iY5O0$%ii^-u*V7Oo2ZE5#zM*b}lR=IglKgZaDUhh|-4pF_~ zH5Ep?D7L7Qnh zR1RAEAs*B{qSX6XZ6677fq;|?g)~I4n`xOpezlOYLi}WXk=d8-s%do|!tTA^B`Fa6 zhDHP95m|v`fkI(1z3bekiT%bNat5f zk0L5+l*bF`NgvpWuY5ekTO&1-0jkku?towq$eqSXu`FztJ90pzD_=CniVV{dO3W)| zBk2_m0|^^MspGKHkARR>X(n)cONanCm+^PiN;ROcb>y{7kGLd@pkUIMLVa?e@7@hm zu^Zr^IpG^9E9kbi(Y&_|zDeGO8Efy%hdfX->|wa=hljBjtpELfZbErOAXDEi_)clXAS+j1{@EVr`HusxP}`$#HAeg6s~JjX%r*9_Ei=;c zItB(Z@^;fQpNwEjc!!d49vzlZ_JdHi!lK_p{x$Ib{78D;^^_FodiqR?TA=LvGgWm} zO&Bk0BZR%2si%D%q^(9*J1VRiLQYcCY-$K)Vjd;e2z(C}g4%A#6ElRrR2n=Tp977= zoV5qt`+k%_nAytLe-jpw@AAde02v23Ua-2+?Ui>^oe3?caab0+@%!$vT$9;CfwPaug3I z%2{@`)?0{Jt&83v3`n_b8bLaL#bUoAvj6;ud;L<>k7;kIq4wH!(ib!rK58wBeFo~p zp+8GG?-TgVIkIEhP)_0Fvu0A6`tnj>T3-0&Jup2TiD zt*WW2IxJVnr;Dj`W@E`?$K?yzk1XCqe#Dl3V3TsvQ%uNT-srVS`o}*X`|Cu}ZEA5$ zE}u`6m1PhSL}lSTM72m$gA}T;ksiaeWTZJ%N#e*s6?p~|eKKh-8KAi!J}>tno>?72 zRj7(E$SG8VlR%fC&mnaPjd-XbH1I-ua`rv6QRhi6ra5*lVdq$iJjKnw>gxXZ`zQni zsPWzaW*sf+XjSI^{6Z1=9F)taUM@?ER(Jwcoifo}i%V&*&l~Mgr#fDJ>)pi}@P=@^ zJ6orGBxKhmh&aes0*1HIq*hNP)ieQLfV;w1-(c<0Y^$7k7q&t&9$Dth(2aG=c0uxuvD5w35=;uNG_lBhYa2vqQ zVb^ocf&vwmwm8efPray)bj`59_{!@$7@`xZ*R5MIuJ|^M#n`Yy5)YCHX@}(|A289P zP3SolgfU6=cm?QF3n9j9mK8h(%oDSA6zwGAUp{i93Ct*H?q3RwSAh+u+Ce?@?lRv3 z8|nVshSVn5euPW57L)B@%r{36VukYpE@ch%gcL?T9c{}5nbXGn zgN{hX+|9`#%-DsVbrt6BGf_6e#(>6G$5ZWWFY=OW-Q%q2+m5z$s6RYKm0>KaqjKAr z*TU(gh$uv5NfDrvxD==3w=oql>?N~Y9sfqBp{OMCRcJcdG@F|G4tCoO99(pJEvz@%yeJK|-=D&Z>*O^I?1|{WvF+OGl82lTbz6pG2t{aq!MB2QQ?!%$dEEQW zvevUIDrZf5q-eJyOm~! z0<}>lLze@p4uac7Uux77jMj6-KuLNFYV}%*DCCoFX(JL42ECXFnCp0(A(#GLy$qgX zfxy9iy-=eiT|8Yr^5g2Lg|z}BM76F;MIQsi0g8`MKnrDU zR;|3qCotY4D=*6sMNVXRIJii7#Bc7WLjPOX4SXUECku_Q0q)U^N=Co;N2~Uy<8af!EMe#~orOU}di0t^05BunJl5HvXbe5l5hYm=jLvnatkZ2R}c94*_ll4TBvhG#{(t$m=^tp^3bT^IK zc^nPi1$GHOXSqCLZa&%i?f? zQHe_}(j7VDD1w-W8mZ&6zF`Anm>v~)>U0}Y>a}0tbRgD>{8UU~LwgSQy6*chC&^*d zNfk6de2hpeEodf%_#dF=5`wpYDmEv`YCHg;I;}-%O$Y{3b;mEdL>gNQfwzoL))TJC ztvTi{*w*iWr||~ZTmjUs*0bko7Nf+lrqeb0;1)PZ_cO05?m8cg_gJ+kWYRnd_ov

    ~Cd{YKOrhHKU| zN%MoAA@&mHrl0T}DRNTHxSl%n1AKeOHA}T>WHK*YE0LXuK1|hyRj@lq&q3cGku>HI z7S|kkrn%*n2lxrm>zym^BHHW67Gq1(~ir@E-^whoq zX^_++Gek#`fn%Wix(5wdpq}5aSE6S_F>IFY@P6q|k}Z8MMlS9;uQ?tVLo<0yIgDTl z@GILvk-qR`&!EJ@M%Z0Yp+%Xng~$@K(oM)N-ii+#1+aeUE;XtwJWhiCu)z@h_7xnMS>f6}m~oP^kX(X&U|d z@S9f3$Dl0DR;HH7c(yOVTC36jWUcpG=beEdcbZcmkaXsAfVfS$Sdo>lK}_ zaF?1Bp)-ZkMt6-@p<1^o+Q&O^jN5m0uL_HS(Tv;L{96R*SD_n0{dKBjw&(fqHOm(e z6e2EfWK>6F7Ge|_EO&6jy7W#GO1w+L>3U*3;X80!z{iy0K!93{Y;Io%9>4y~TMFF~Eck3J%>X9|rX5i!V|s z?K!qGTMZ#k(?~7gqpZ0jj87j%L`FvCX~_?FY5q-FfS~*616^@KF!J{#CmUjx$*Vjq*->FcT@cjnvMMU9iIP1NSD(p;^{p5Lkl&+1-JuHsh<r9Is6KSpK*4Hj2uEBT1s zjJA(u)BN=C;{AXQ(yyIrZFv}jo{p}+vlll#o8o1$`@$o~af6KG#GU<)J>NSFU1!9t zFCmaPmd;`3k?+ghiNyy$FglX>5G<1d^f#P*Kt@ zLY%UWyz<*F^om7jS*DZ48u*Qn9D2$3KG9PXIA3qXYT@Uhzc%9E@exRNw!w;0hW2nR z*f$xudp>q+s8~XO?V?f1(bRUSM@CctGRZSVa}e3@XUtQQ4g@xY8{Nn~SVp+7*`m?Lubct`d}K_faZ z3iPzIf9x|iqw!ahFE6Y3+n1|W$MWq+`Yw{kUx^#|X%1|RAzmgP=6xTjs2lUwuC6grxc2VX+0NSt;W={geG!Zjfulke*;$RxtMBq>0OjxGsmTxMWm zk@F;1piih2_Q$a6hE?(koH-oH(> zC#rgxMVcAcmr<_$(=7))A(h=e<7i$G9^rPry=lts#{e$<#TT2dz-REAr?lt6iFyvBkTOh0c!hGm~b0RIsoe zGx79WIQ(>rV!$hIm(CwrC!eJa>-m%;;wfQXn)mVi=AKfkTh+kWRnIut*m(*)E!dua zyt_QnexuE%BZ`M7(5|B&`BjbN&9;1vadV+2FD6idb|2V|kxxC_{#6Q_Nd$BV{<^b# z81qG?n7K^-h*jAeGpIx5ajst{|K>}+&;O{xvEQ0>VS2THo9=X=rL($WM`i7*E4SEl zb0(S~Rdg(^NXhrxLH5%Pm|6wlf$O@x70sSdWZV7RTk z#a0yzG!{k;@b16PS(OP4?;Le(-d~VXb$1G)F<2TH-fvB7UwuW|PIAY9igTFz>x|rdTaPxi2*CP$XHdbK*IrfH3$7`C z1`~EX41rV*WK&?FQv+>zvFjC{$NXV?-bQ(Y95{k7M^iUP-WM5m8jZBws}>)fpsF?` z$0d>q%3W$`Y;1h&l%ipOx#6PDH~%7{p)gEyVleW+t8c~)*>@!3k&8Q2X1zl|DdO4) zLz-c373`#MsTv@x4z%qT`(CbH?(0{|Gr1UJuk|gVoa!|isJn9{Tblt!LBgguCYTZI z1Y?mXs*Xb%yeTpb{x(ic;VUXi_-5s28PivXzsi{I&LkJ@&kyz<71))UsP8v=ub9kP zgTH^D)0Zt+do#U%WMs*1hy(-bFFCq{7(q1Z^~=kw%FZK10um>6 zOOQr2n{Dj|s*fB1nclSB{#}3d_h_Fz7dhv=uPYgfOt8kw7T+p_x z=GAA_u9Q2$SscU=5c1qw5*{iw#fKb8`cYfXSczc_)(^gD)G!G%)%t+xtx`Pp?jlatVDxYNq2(iM<7E>!-k zY!izY=%uVMqZD^sZz5a1tyZ29*ZA=W{WM|;X6MQx zV2o;L^s%Zp0Q@EZ@Y1AX0q1!2kr?Vf-3Y*aV?`m|_ngb|@;Q`~vtP(eysgMRy`w9!GZax&OdYffFqdxUCnfP9+G>th5kwSH|>; zek9~3eLQW7yyKX#!%3U9LlnWNbpMEh(Ib!E!ckE_vO#osu1I%El57`AO;-tMtJtqK zn`)_T5wY>bCL(u>4Dk{<3nm67STIwmbsSJrs;Pc#?Q7rFZh1SFEBEpmlz^#|)b-%nCJa+;pw_o3ludJZE+mF$*!ax9$L5Y$X6d2#gb-R*e18qVJt zbxIrA^aQkjTknCJ>^!2~jdAJtwsA_TtAEIqQGG4maOmY&4mxf7qz{N4n4HOTdwcHW z#KyQ6wTkx7OIUVJw^65oB4`qY%GO8U6?g@Ms{@LkpeW7&4#4SGrHh@d>a_2a9kG~U zlzb^hQErx$hjOKL$m z8V#(KVlBIX?4KO_Z1zTPBbvWjR1Ka16$vS>W-mCrVtidR5x2TV_&(e9wH>|7syYvxzPr6W1iN>B z{(efbHgfBHG7IvO?)wB2p+t(a!@2@RyOh)tJk8cD1E_)_pgQys& zfQR|3E@Qt`1u)-EU9qWhpUhw9uX684;nOo}L*7i%joWIN@N{Re3ZC?}SdlS_Y{;ubn@{cRBN z-7`ps`ppMpQx!e|4PSPr$zv`1i#U}5bNc<@^*bdovA0C5Xe{|PoPL0-EWu^=x-8>j zd*5px(#l|ZS$)$U@J`jsH&8eno2!a#E{rCGsY<0qDB@baC4T{wr`(m!LV;U3y0=!> z!(9DPew-fdt)k z%vU#zsug4Z=`2@^>-2=UaPA)TmktmUM|}_1Cg~Tyq|b}P)ybc#eZ`l;=WvfjyZ9{- zeMoehm8Fx9=*Q5>MN4;w<=P)N%&dOC%jn4}l2$i|k2zi4Q@88hRwd-gKPQiLT%0;nzOugOi^*$|E2IqCS| zX*$2Y<94uQjPa`Gox@}GQVY|0!*6n>77_a!#BBZPIm83aXOHTmjGIXg8o5NNqp|$_ zU+MR&{&~mr(e!P%sxG*2Khw6Hq#3@F`iXL$yMhvDQusZCI$ENK^WKAKN}Y~8TUH~5 z+{S&VEi>(md-rzr7@UPp?@Wti!i(VX+`@ffgZc{!Zq*G&OP|Jyw zuZ?EJ#LJvH&ZuF)A%UMAZKiqo3XZ4+a1bve^dxXeQhy?+oEkDSh#u+doX1@CrL;5S zA#V5drO#A`i*;h*f&d$t1xWz0CaT9uFfmC?4Fo_>lH}lhB8E?xqYT!-W;e6W@epT9O)iO+`nu39nUuCp{Ad zWY@nF*5k(Zt=kp<7MuPn?xVheExvuw6d*OzQOq`la z@!&t4qTgd>K!4Sko#b2RUz(z|8fCul=#zxb#m>!#gR`LwBsa~}J2A{!URo18i2|-C zqXBW~e_wWjN9f1oC&o>U8=9<6%Q8wsoOsEpBKZao$+SYLh;wd}Y98%vDc za~rFQDb0LQanAJ^ocX41q*di6Q53PsRRxQqjI(DCwCV<+Um0*+m?kdpp!-+9-oLj_ zA3*bxne}kC^WYNN;c_-eM_}`5by9 zN3#Rof)}A$kH$9oCSRB)0oJXFrTY_~-FblGmw~7ESEy&XeSp*#S=9!tY00O<6HMZwf^zO|K-zJ6abwR^nM5WFW>vaIr|836M|tFKeZ138+!kL z2dy3rm@?-Zc+mg*(*ODaQ82(LX2GNQ1F8GZnM0@ZX$gkCSa%!8Fr4Un42<6VTAzdL z!dvY>|J#2)mp_{Ynqw9NXUM*Gw4_Y@DXds2kSc-agyE`TQk;Ehad>&f0qPOHAqYR`bvz`1|jtGPyh08eJs`b?7IA=5v! zI`bWXlVwj79kwN|{q4gVbw;3g$!fYf!4b6lKG6L7wwW?@dp`Z6{yx?yCiozqTjdW^$3$1oSRztgaN(Hl+a);^KYj?ZkXnMyKXeu3S00E|=#wJE9oEdy`pgZL|?KD+!UvREx#aCmyJh z+vFFMSO5&!h6QYOb}@XH|86U9pu3AvXX@b|xMk#Tm0UZJ2W!I_o9boj-^|gx&gZ0D zv)7*ke3~4*%xzvI$oOr}{UR2B@?n~DP$?#0SO2CDTc% zXAo}q@)yejkqu5yPtGsOM6##QEFB7qYwGYW4%bnuc;j1i9o`3P`a39u6!f>ltUjoW zwbk+)f?-(<4>q6CNL9;oF%5tAWUH0~s7Q8WnQZhPGE1^5AG1=N6C^|LXKdgPE$pNS zMy*s^3ELFdZ;i%O+Pgw3dTA|vH2kB5ygFio9`=WA3OMXN0?H7md9iiyV`n#B7ylMz z`s4k@g6hBFC1&^9d2EaeZXC=NyRYJW(W)U+$%iJhYLttzPkgedC~{mKiUPnpCg1g< ztY8oay`v8$>rGA$4O&2QYitOFvSaEO%79n+C=rFNPL>&#_fgyHDxOK@YdR;G!gAh^ zbo5GvN##qrm6Gk~L z%8-tDbWP}9JB09@8m}1fyYGHrcYA0K1KGd6ptoQAL1^4^A~Z?KouZ61v-nmcq!7OZ zV9|}0-R&?EbZ04hNjYEV6`)<$asp9&GE^mh6A|I zYhIH30av0A!F4;D0DI1azje?5c_}qW&z@~(GIx3k?Nwjc>bY9Rj4`*jmmS*m2gg1c z848UEqVE}79F=1gOt^XTD&0kAKSnfIBceREbO^`;a* z?_c--U(=2C3bknEn|hPvKr@@LL#13?>FQ0KdmjPK0#o0Ou$D2~oq35!dr6Dgk3Pk& zOJ;`J_l;MLubw3GsKka2Z!UpqpmSSAPuDBW_<(NGrrpPaKw~7$I>)KiOTnJ6pBAe; zeUB_F2BAcB)pBF(#ZJ9i>Iss9Rfc1chC#kP_u^a&ocoD@7RjahPkMmeEaoi=BQT*! z_<7a+ms^Cu09_%KUq0MvT_LSC^s}0S8L64359-bD{PB&efd(!`V0S8^-D8^4VqU|<{pp-@Gt)Z09@t@zRUInwBYd8xH5Wa?6G7XWJfP(aO20PrfLgF z$e;zld{l;gP$F)N)Xm#6mza?|fvItt$^hCG84In7FQ!f%OncZwc7^6r7Nl56Qh}HC zk*`0o_N})fG5cStHUxgxD*ktk!7Z|PR}XZbPnvz|(lc@LQdfK8<@2>^>FA_Rsp@bO zH)%Wr#(C4V{3tXAR3cYSkWJC@#I(!dJ4NB$7th?o%7KVbbdwr#@o7s&XZ zjuh~WPJuEOa^BwR`oi9xGxTad%|6y=z+7kLYqIH9xZUcvP;$rqYjSI<*Qr0hlSkXV zT8nmDe%Hn9&B+~6SU3@;JPdSsrGOx{pqN-$n+&pW6`qO`J~N2is%2tdFdE)fsg#er zF#1+^^;EOQR)xoN^1IXZ-L^@=m#Yf@92BE>$2iy^GM+td_;WD64>iH7>l-s?OH%)T zSm-C%Kp)Uo`3!D@-HAJ#Cy8vhe0<-$emv#Mo(f>qDmO8=M^q)g@{hZ}cIGZ3D_vf) z{hpUO#pRUGY&08XIPdic>~JVYY3Oj;k<%DSQR%)~#G>&}B2eiYFUo63?NmKN9D7BM<>B!oYp;xoJK>C373jjcP*CWq!wZzZLbfpUabRrrov_c$HzR@_m1Gu-JZ>2*`VeYd%2yM!`ZkzFn8hOLX32QV5S5 zh`D17T{xA`(XUb%ia9zX6|#y(!1q^IQv3831!a#ot72&t&$ub0g+&~8IrB8i-r~%> z9A&jWNEps}o^`lCk6?Q#2V9%QqBat?+8rKeI%hrC9#oZJEw4=$zQ;={4K#GI&>NR~ z_soRC4*$T*f$i%YB?@Id=TTKcdRb{nQ?O_k8VIF9R4LZ2klgpWC{RgPsqD-M#54Aj z2hE)6amYVezREwP$$4WhWBEZl+z{<#p*3u0K=H1_`a?ibarM!L1sfnK@HNq4D}2vnc|V@h;k2xyNUzKoOxL%iS!L3*QL%q9v|8-g zl&TEe?NjWn2sfmt zeHLOjZ;3}p+*l0gd3HcP#L@g2XemsKa}OLIDzFLkPIo$0oluQOQK@}f{Vh3h1C!(0 z^8;?*ppXz{qId7#PY4@A`JuKJ4d*kagWtN}QLy&pzq@~3l=Nt}-RlNgjz>20pJvEe zOb29K8SW6qH5P^X0TW%2y3cpdy^B97;V5;eSLdvL@2-VV&>;XS5b`?i-*4H|67#=B zVmrPIhaz4PZ3`$@%9&=F5}*J#h5L1rxE}|MYUux0AqCj!2Dw>lt0Z2rYRm1tn+B!~ zP%@l*&gkwAsc|xr$nf^=-HW$ZMx~7qsB>+IjTjSfE`y0a&;jY?Mw9mW3En)`_1t185+qRmv})v?QHH@vmF1t1)-90 z*(Cq1o>b@Bw+QWiCIT!{)S`{i*E60nH7OhN$^1^~XKFK25<%0?M!ssM)wmt@^C;D3 zpq}gvmJQabQgA#(!@vV#oF69gP@O-Zs2eC=gl}rn3hT`L*VRIAY!T-kV66dE{2Qpx zBtO~=e%l|(EaB8Yl}os2vf%eP)$WL8%P;a9jdPmL9Ct9oga-Fh03)VBf6~-~MVJ1o z{OT{f0U(ush{0I3JWtGHg`LnB&!Ig$Y0I3b2`;hmx_tV zx=33-iY_c=CFO&;M(dxfdB7qnAF*Ni-SH80+3VY56DE0O`SbqJAOSehZo z0BMOl6FrsQWz{iFCh<7d=s1fi@ueGCZI|t3f5m1!$zbL&l^2_!ND-Y2-QB>-O#@Yk zoE1hwwji_&_p(0jCyu(pT)muB#5TO7lZ9NYrYgdRm^%;*4mlYZyzU)wjl#&8+jHZC z^zOC^G0%w~JM1j)#q?j=#1@wE#vcd|er$dC`-}o*P35OF9f>0nf1+c{gx`B4JbF$e zRANh_?)I3qod$4)9uR=dYfcPM!AZpII;_5#{qRh9Alwbg{+Q`DffR9;6l<&KdplB% z?o|T%EJFA^C?|1f-5mJ_$*n4kybt4*GQjW$WEu^goaw^ctK2xuvA`HrDpAgbPmuT^ z)AzXLTe^WU1_#F2X-0sCz(Mrs_jm@&KG=elSNnm%w?JO;bZadmY#P#RK!;PlBj(r% zdW7gf(8`m*3hwo6BZ@OiCCc5qBhNC(*R@sAfJMVXQo|PWI$R@5qUj9r4FdHy@0dRX z|DI+_p`6%7kEDZ}=J$ZIa6#+FLT+YDGWKaIqqWAK0YD zBT>dbFmmz0DMC`jQXzt|yVWszl)8Kgi&MP=)48umtcA2mH>niwyZJ`NZ*#N?Kiy1X zX57XWNGv>|>_}n^*1Qk&ekMi*P@__dDCBLN)0!8NN|Diro+Y054RMIQN*!4E==kK9 z%FEU=u*Csqf*R%I=IuWW0)WhCRIIVj3cTe4G%D8WGjRiQ>CuiR0JfOH>}WsE8}BY$ zLWTfbS%GoL?d;CV)~8~2tV$9Jmos4iOVMYfvHX!a4TyYrzL(qoeN`k3OEFY#y&lC?~WIix1 z)|xI?OgBqGf{(FR`r&1eFjyeky8_9gphCa+mr?uxEntuFHjY&1cRTY%VcY7=Bt-)i zBd#c7OgG=&CBE2W7q|>aYN8Fr4x-OV_-oczQ$wC`Cn^aU86eN(e5WUyJnht2Zlx7e zb+g>f0NgTkdY|YiDxqTO+9zm04=j4>D^kC|L|A@9x#+Au;T3E1ADvreh^=go8Q=4^ z)1bMy0Sh3!BI(*3m(3 zz%bg*p@MAEEux4i&%B!q>eYM49<_TJZyMB4Gu)=PjA*cz0zfkL=E}y(=CyWN1HN(G z+t~~E<6KkvU0oK0Bfhw(vBfVT!ntH|`kz*`rc+hx8VQ!IPD!~Q+pZ7klHVg+i@+YQ`XCz~9*X6IlSHO-ujqO9kvVlna-l z#($kdX^sxYcpM55R-i>o>%j;aW{2%Hq{3T=ZV-#KVB2>|xU}*qz=Ul_nggSghli&^ z9;6?oBJ}WRvDOk&N$Q<$Arphk`n9>^lJC zQi9n98D@-RN%6_6zps{5U`vmZ4viJA%9L5(bX|Rl)Ru*6pO4s*?oPfBDq6a4ZRaF06FQ>Hms*?*Z#(38lMJywqbaWkO4W zuxIr&sK3*?(`U?wpMG>jJcuEF98Yh3UyQMu)MFh-@ujk}0{FE6_Z*r01batq7~_7p z#|?n0%z53wG2tRi^*?=ObW9&CE6M6a`3O3%{lmFFW_lkq%2=!cuyIL&p~l0PDuDcn z1&mia#pt4YH`Zl9TXKy_o35^F&6?UK$%oU34!hLVQ2$BoV%8?6mq=NijUjzuN5Rn2 z($<1Lz-i!$Z{H;|mp1e6y3g3G7U{aR86I*NP8w3v(Eeqc!}^G!1E?Uq_O^VyuN75A z9m$#gSw@QEBXo{$U3|~`z<@xxmfxjChK#FRkV@KB=+@w^Vn`gwZlBkj z7mV?4iToH3;s(8ZqIlXVEtCQ;ef`?CFZlSb6`zPG_d-RtNr_Ld+^!n;YtOI@!-zwPsF!M3{^uyRv}ZCJ-k?v*fe5MHJi~Tj{7d%hNz^pt zZZJcxOA?(u39Va;nf91BtvP&J6$KX-HptfM2w72O=^RwFq2<(X=Y7koG<2bU+-EfJ z7vZ6?NZA)%;=ROtJ%KHwBuVfe!h0%p^Zb9=A%YjzP)GKbnBhO54r)A~T8aT-71Kvb zXiWU#)%7cu5K79%VI}Bi3qdf(?StA+wV_S75M%(65ksxcFvc$A$*Km1#5zA?F82G* zNJUqAud^YYO`uhPZrt@0yAEWr_e~8;>rrLyyspuhVi<5&T%+@yOoIwRo`Qs!dwi7k zAIV_>*J)xuQ`@Cf(UlS!AO`~#>*{Zwd(;yr2@i%-g0$mFFZ40=5xh4W&$?cF1h4-E zB#G8v>!<2e;@tgbEx0DCIiE6yk0E`>3sn@D+1+CuCUZXKa6mUo9b#QzRQRr7x2Sn> z8_mY34fj6d3LPv(=28XD1~Gg^EWT?j0JHeOq7aG=vL5Z#_>KmDo8vF_BWpJJ5>7Z2t>dV-p^6<)7hcSdwWFDg++rwQc-Sl(=@wU{dP$HOrA5r zwCiDW@ZRh=>)N<{?(%!_exNUki2Xs~@~Nk%4#Ti<+}BCgrPNC{yl9GSIQ#++2IYk= zMXACc!0q53AQ?mgoo_oAR0`T7hY_1!aJwK3@KQ4YYHCgQ0C~vLEN)^n%LxBBqLm;G zJyD$$BrnE@a3>kio1gatu76Fea`+accxhRrvp~ z_Z4ncrCr~GAYD?@DIL-chZ2!gln&|cMmhwf1?dLKLw9$#Af1Qi(A|8SnP=vmnRlM| zFZj6j#kJ4o0`7g^YyDQQm2JJ&d{}jrr9HflZ8;~DuOf&)n#}e6j7lceM+iI)-UJ%O zP^m4JhYHsa6ivCafKsUv`ZA3n7Lf_8$5v|TajgToMyCKoarDziE*fj^zsJFV-FU8A zWx*x%^B&0_Me?^E_30GkONrk9)Q~WIeCJ~lnQY%$7``n1$&qkH;c5`6-CSoyjX|r_ z+^3x=qOXdtnxtmSWUXP;?5$aWD&xq&tId)4G+QnTXaiu2tTqP76{@}7pr1 z${ifG$o%8G%PHb9;<@-dt$PJdJzwF zMmree+qxIn2aO(`J3uL>GfvR&i9^PX5h_e-l4+i=f{j??_bJp443i$~GP3~`6swS;HZzWf+|Vi2kH7SJgy7hm1e;s>d607sOr*^rj)O zR+S<#^Kf(7K0_Mn#sDgQ6f1KuD~&u{6i0O1!N*}rW}JO#FM|C0m-3Z^Z^X!Qj|`ha z0R!*^uuOPq43QYT5^_h1T6UmNXyTZNvMw>*0Y}6QFB@BwlhvFg`wL;`4TaOr@d%}o z&Cgr0_(191}=66PF)AeFyA&(k)hR|yLT z76(*G=qbF}ADH|%mFo#Ch17B!+2(iz#|HhLysT_yw!u+cPE`m#%X?t{Se`*Xgwt|= z>O*>@>?!4Q*Xf}&gk3qkshG>gGp z62X^*P1e4-T_#9~s(~OH=m&VNKlRoBcwhdgsd)n90b3pumsKGtm++V~mZy~hjJD2t zmT-Ororibu%B?m%x9Ce1BC0Z7`s$f-lVmz|TuIIW{Pi~~G(aE3#0JvZ$oDll(f&@i zf6%a407yatZlP9J1jIr#Z#`4_onPctFgSrDNwVq13^Dq}^$(H;ipeTo8|&b|AG^iQ6%BNuH8MTzfLjlwjY`YP$ix1%pavU~$j z-tP<6;UZ{f(V*3OEu{mVo$NkF^jO`JW=Z3bs!4US-+JghK5%1ZROk^@^yVE zh@@eK1z61YG3BXkro;sIFM94DmK-3t0AEp&M)kW3?TFc`E1*#6 zhWC2m4Cu>%k%#YonBndvjum~z-NX3l%*w2VOw)}l8Hp-H` z7EB?qIPEW`Oah94GOteXMuDQlxnyQoK_EU&x_9&njqKzL%R|*}x>}t>)i(vl>No8` zraR~9UfV>rE0x>99>(5r|9d-E*U6AcAb8F1WU3$;f}7!5;aM#cfemQ2qmR>WOinq} z|4V>ZR2pQ&8tSIcBs5ooVK@*wR2;_a(pOR=;xe#0LkoG+m&CgF{{4=m0lCO;r9NOT z5*8o@YS_PvrmPg$wsaKlqbMH4boCD+BocV^FrJ~wDRJ#+*HhfNFmJ8pa zGo>ws6Jga7^cpk8f`rz&Zv&a!kt3#rYHmAmd?(|bURQU>DQIZ-Zm4JReNY88?pdnO-bgRgYx!@a2#Fn1}<&W6k9}Db8Ksi)X!cdAZVodX&c!ww-(h;h|5_+5|?WY}l`1t!-Fixy+SozRKz5 zCNQDVJ{FR=8N42>q~|ahJapdrmGKlOQ0M*oZ{>nBz^L9lrK$7cuj0f2Rz*JM8om+0 z8~XP!z0aqYHYvvJf=d;wjxJj;Zp&>G^a$D}7RB>?^q)UC@z*|Jek@RD8^Sf{7Qgqs z(i$$NS8qfNb~gmp_5KTi#h`(m)L47JEOtT}WyQPDSrW;E^fyj9Bm#EH{mOK!oO4m) zQro19*l)OOTR9tY>F-`yaywd$haImCiRa2EXC^wDjjC|a*6m%s8|VTOOF^F=CoYxV zU{w9b^7|i{|J@Lbq{PH0l43pGl3Ca9ryt;RtvF(bW&pKGm1_}(_2_XwP~yg$EY)2q zkgdK0Lp-w#Rl~vtEwFkfH_#qx*KMMlKIDDy0f1?9V#RJ_iKEi%07hPAagq$Ekx1WO z8q9XxSw<}WqZdGk(K{Cj-V}b9k)Jr z<|$anXy*g!#SYpu%(_1J46p;5ZABfRDL50*ad$!Zsy zZjVp~cUzR+%a? zKK@OK;00mVQfTf4ZFm-w+D|WwmRkqpD)Y%G7OlozTYEz8dn7J zSP~EAy9d76VyI-HyE7)1tYTT{}CCiONvT^%_SsrsuMqftW zTjz`#`o;d$Hv!Cn$BtakUpH?EQ?sr(Z8`75%BMle5gDit`JU$JT@`gk0c9A#j)p&J zC|56EE9{a{nS_HLmlaIK6`CO`VupjNT6i!vnvm*2aZx5?A~HoAR(6}C`-g$hJKt#WIq3QrAWsj`6zr4PnyFkrF*zcccS)s z;X|1ns#cTR_vaAo?R8Ps_xbEEx5Hql94aENOsG8`&QN!Lo?|hn){-x=)_wQH%l)Yk zVod+J(u6~~SUFf-#D?`zd+c1vbgb;X=Vo)?Grl>?aYKWeRJh(vr`&Wyzqi4I$l;KB zy~zJp(~I<%qmc@375lp1--OW(rak;1g?czFx8{SH!B^lt@QX=ourFAi$+o#xzlyhv ze_0?RqN1Yn>b=h|f5*@t#;F_QjEq^j%DUUqLfL!DdI@9AHnA=2QoEmU^-~;Td!=h> zHH(hHLF1c<=Ccno0vIm&`ITXtls`q`aB)z`$q}7k1{i|8;pdDGnu_*!W(SJf3(VDs z&Wmf$CAEkc@ZeC+wOhsB;V6E<$$nFJ@i|pyn2U!xn!QifXv6hNh?;LxXwOTY-F`%x zebl{o;5Ee|Qb`a_dwDy%yij>Jd%*tD@Nu@(GYrd5AJOc)?{!9^!5IM{_f(Wz0_U&96KL|Rr`7C4L)ukhUC(P_T`lLEUsWZz8jUW} zmN4$fz(6$}^(@lQ-EM1bJ>Xnzw+VIqp=Eo>dCf?bnOQ;6Qy$?dv&FoRwf%T~=53HA z@aLtLJkM<|t{U{Eo4VB*%#{PEO@(c)9OLS*8=I|#La@FNKH@JL@p$3=N)<(a0g&z zxS7m^;%Us8)N;XUABxrZRc2>j@$qd4M|^GLLt&cP&pZ2+2re^7G2r&ua1o=)@4_>h zX_T}#A78w^r1xNS=C-~0s;~#1tFj+P-#BQyy=gB5M`i6-5b~I6Lg(_vzB+0&8(H0< zP1b0;pTC4=U5Fr6Eq|*3Nq+T4j6sEAu8omZ$buaMGJoE9z(_Hi$A>VTB zwbcByLVbc`qO$dlE|dg@R@l4F#Nm{z&zm}83+DVja^iGmb_j&L1x#5vL_BIEYHfKEF0h*@ui$XIX=zuW7~ z+z99<4b)CwAaf$ZJ?J<>x~K5>Y%^E|*%xJw0g!2Oc02r-5P zrrBw?AnCy}#pBp@vSqko!p(9XI;o-~+Yapzt|s-^vNeCu!l{J z6H>!cF3W&WSei|6DE>n20Odzw!U+465}X24ufLp3jvkFphSv^HZx)6(+xXWxFG;l` zV|sySRp%XOJm@X5%0uUJI908@`(=H(1MZP+kNgkx)X?flAQ2VT2z5nlu$MTf_#3q* z?Y5Ojon1u2YKJdQan_ts*wUW3qOcp*vJ`J#36C7^n=Ru)I^-;CR-b zVkm48qYT#{3tB32)5z);G6EX+`<<<2Wr0IR8U_h%p+b|DCAn&w+BUI{t3l>Pu|?{! zmN&m%VPV2p!=g3$;WGUF&B$Scv9^Mf)y1uA?bZ&;n7nJXwt~z9@6^KeOBT{FdmT=> z@OVqY|<@qT|Jqo{NQ!e-hDv}?3PL|ZI#!Vmp;-wlN= znVn;=O5dqt(V4>*sc84{-%gA7*N2%U11!M_iB-WEOWCITR!Cj zo(Mm5!K7u{&3#8)u5IWg(Iq?K@xA<74pyDGAIeR;Bm^%|+^Y)d)Nvz0l=OQ*nhtyj z42B{7>}l1p0t4>T0Ko-$KVBbjV1!oQzgsBk1_0*wM~~k@ZAhq~;2W_fkGC6z9W)J) z%f$RNEnV%v)0LFtg?^IvWep_Bwd?Wb7{6@c_DlEf!m~?jjco zRc_FFy_kEwZ;96q+Suha--WM?r9}9{WhM4FUsy?hW>d38^!CH*Ij|;9JPEp*BhyT1 z`dNl~x>Sa%V!o{X>T({m*&M#tFw^Q}LZK<@30rwRK6NF@3zv`LWT!7RRfAPXwF z=jYHvU+PG6-bYZ=;2oWMh8NE#6)x`YF0oTfu%z9OdI@UQ2uv0fn z-Y8O`88~c#6~^85_>FlNE+KwB149Wa(!|95hq>}ELMK-#%1>_@jSKVD&R=bu=q&0N zDR%J)8coG2pD<^f73K8LmU;$-OQf?SnY8_2ztJkf`+evCv;UPyn*OlY;vVFS0c2h6 zvXm#LUvMd|2}6>Gvz@K2B|(hNz`2E=z&TAHF0`A z%PlF4g5A$JWO2N`RCyh|cSi`R8we=m$siwSzxswTkbb_7O$P9eAuhM&_o{CVt!a3J zHVE6QaK|)<9o9>0DN^|cugpo8zNxu=!97?nM))y>{0sOg?dYoqrWn7!5bAG4J3(bb z7cL+8OjK#7%U>WKspdPy{mOlz!Z*iVn?F{bW{-fg!mAT&TbRr#GwCPNe$Qhfuth>= zxjMb7r06cBrK}VD-TEPCOkvnF!OstG*qivKOIYXFFd@q{SVB@a;f)V`Uv|!hQenZ@ zO@|cAB}cD9$MIJf^ph{g!Vz7b=M@)M-(U2hRoOP238E{=9~n3IPm$hjJrf*PvfP_r zeN(vo^>(Yjm|2s3iSTTOZ#pe z@>LSitz|65=$)|Z?pGUO({_QLur4{{pT?F+a09iLZ~fMF^9H{wQs8M!@AutebT0*8 zr8}`|JiHuhF+GjW=#(5mSHzo1;7_!C&LpR8Y9SMBn0-F=;p1IU!z@&-hg+!VoOvkG zrHPIt2t*AlQ+TEP({R%Amt0Xd+SRh&mHNkz-#_|&3}Gp>FsQN^9C+A3v24-Yg83d2qZU!FE+i$GCW zq{|$tg|_eB>n1ofd{6rW#)tvc=!0flVcXxtdnB??Q51A_bba&?@udxAb5h_#6XL28 z)8Lfl4do}Q-lk=-mfG!jZpK*7f!e*d-(Fe=+z~)BL*j}8S_PUYj}bNeEAhnx#TO`8 zWwJC&?2>h*FRUj#A%)kAUuqRsj0-6s$Xm}%uIA$J-s1`jn#R72^q!qT_tQCSbp&3hwG5S@P;}@{{6-#c3}>H_Upx9Yijl#?a_}5njs0 zR!xh^= z#q|gAVfenQ*#r}&fuMcj9(e}>6Ynzm^a+gtB|)| zK?>SJsAD-|DdhLDgAbx)Vz&KZ9bAl?$j~;)L75Ec(TGu|Jd_Vlwcw7B@L9g%KH51} zLgk(1n0G(e|Ku@u#6t{XK(GC)*z-oj2>}V<3{LUe9?y-qWW7iOsXjTnpys6@2813; zv@@PZdugb~0kv9r%V)7$+l7ntABq$fk~lw6)N$QNAe7qlYVr4B;|gnIxaw#S$N-!P z(uZNZFNVqID#QcPqy&PMJ|DRTk)V0a`09Ha#jB`4n-M!HODo-i9k=7D%*Rs~AMgro zJ95g7>Ay?FHM;nJ)T)@^H%&4J$A+m4J5+5lC+eelU9p})Rh8_ahfQe$#h&Q^mLDS? z4#Wdm?q4|%1ou_aTY*opE;PEu{pE)bAxSJ|d}>D+fY{cGHsC}1o{ntkuD*IFhM)M! zRri`lUgW3`uizUK>Hx1GLzE%0{9CI3s9lCK>7p~$vS`@9i-`Z@yALeZsIQC}r5i3_WgD_bReyG&&c zv~7nk{CRtIQgpH8v7}yWqubUWs-*bbB6H&%g9YReH99A!epbEFj+%nnZyreR> z_EIKRT_%p@%;H`_moP&w;$7WRzp?Aliv}&sq_Z{g}TgEEwzMb*hnlm=fT0zs#ml=UYP3H>(WeCE zoBMdHA(p?r{^ZPuRp*b*m$xX;BA%t{OCk>TN5XxR1ZaOhX~4IClWc$CwfD56wbHBW zG{0!&<4x|2JB5_xmkMrb!h2v>I)X6GLdQF=(Sn(B3-gTzhq0&5l$M_}u|<>rRCk>vP;NXK)29TkwwzPdthJw;X45a)L$EwR zpDY7xiU4c?8v-z%aOwOLsD8(-w<)bsF5jo;`Ru`A+>52!O5waaHs%jcFDmLwmYRSi zKfN!O4vA%EI+vaOpj0$jc#)_anrQ~Em0^XJY)oxWC0DaM1x zy5Xd^&r`7O=3NDg`(ZdRY^A1RuzpDkL#I9Q{z-4F7rvSwty31d9>q#+|=ZKuSZi* zB!cD#Rm1Pq&5M9TLOF@slD%R18yr!KV0r(>u}n#+JpX!$a^ZJ%*Yi2G%~VsNtLf;; zHEAWY)dnZ1S(W+Hu$%rZ(b?_QgvQX$L5c-m=-BH)i-oDrn|gY!p9+avx;oC)<6_TwveEVA zh@niLuR^;ZRnkKIFVB_ z?Jp=SRu}LYAyc&encZh+Xj2wYGOup<%MklyfREg}YQ`2jE6Xy_uSr>W7r zQ9ij35q%#hFiX=cD7K&3QP5CZaz6>4ig;;W}V#b zCC?~8q6c|DV&=Sr!#}*Re_~an8Y~JPQ=XQ^c9|yz^k6eB$xP@fyE9VDsXEnj$8^Ll z)pdu!yw^%xaP=!zJX}(#t3ebRbfRMF8ZQlWRFS}8q>e3;WO6p2ZY+e~tZXqa!>R#@ z$S&(@cLU4C&lUBT!%(O4SY_6#=G5ydE~`P_CV6mps1rb?^MJhtp~=+$FNEqN`&_ls zB+7AGG*E+Cqq?%VPS<6*FzcI1=i<%2rO@?Z)|n?~%GEvh1rPM7C$;o^(Yb{BYo7qH zdlExbAr|r?PzPutV@cJULbWbe8VY$Hjr;dYbf(aP@zaIfs-00I@P4|2Lhh7sm(C3RI;?oI=lT4nn|lQuFom z{Au|Q)}3*P2zIwpWxr!t*{>!oO1I#J((0?B~-?@a=J9D z{gpmvU6-{8?UWV7*Ni_;sadSPuN<1BskItV8DR!g4)+OoGbzC93PA*%43K(Fw7)qS z{=`xr3(VRJkqb3*aJf;+{Y9_-{xZ*S1!;96dMpJL4`tU({ckwM3CE z)$uxUbiATz!v68QmQywjSJu1&=p=sVLZh?IOXN@mkD4l{IaB9Cjokgo(*q;bRc$i! zDxIb#V?00S7yh{k_r{h7mv?xpy^^uE`Z~oPxN5*Zm9<%L(h_7FuT)cldN*N!Q5d`B zV{DZ|uk&TG=$v6ct}lYB6#iY-Nszn3(h9M=00dwb&#{?xt!+|eT^G)gmv?h}bYZ&W zu77l){q0rr>)mk$kqS4<(Qz|i*6?#RJmv^+g_?R(XT8H^G?At2i=5hAXN9O-5CsGI&twQDHuYdRh{!}H&PvFc7 z)YU$onP1(_mlSo$c$GXJLy%~Cy_>04!GY&pzq)New`0$GtrGAx2i|UxuTJDLOW3Ai z(G;|`~HwfCZEqc4|cq-AnC} zBAqX2O3@PJ&mYg~ISXINzirQ7k1DA37}(ZwLF|CPJ^j~T1H4P>Ho^Wf!~cFje?7iZ zL7y)J2=o&Fg@ZtFu!s{ML=^S8@-Cn=TpuR|l&e0nGLhp;hIrG$g#KWdr*F2jG>{uA zdDyOEl$9~ltGYm?C$T^UN&@}!ApZZ&|KEZC zzXSi+f!3r}u zg^ayoIX=SG3)9bL{z2U(+yXgQpl4IoVvGUYBL9cYK>o}}DALFx)ZtxpV+7ok_(M>l z;(Y?SVxH|c2YmsG4(ANJhXuS89q9<-by`?%pD&dEh~54L1}Tuha4kj@9w)Y|{ueqY zeIV12x9N(j)Z4GCUw={hHq7aFwSC+FDL3!!x+qAYHbzRM0QcXqRS^+(2<2T+KbCt_^KBrZSOq_JQdk<{=g#0(q|bi15o-TV~|p z)o8zaAw@u&b8ic+;%1-5F*9IqcM8@Q8Cq~3nR~TLH*_61B=B~r;bgGJ>)3F&SYduu zOnU?}`EL8)m-t9TOo)d$lNA}(8D?8jCTyoPr?kBTyuq#%oY)Y!qmfZc6TwQhHRELe zGZ0dwV$~Sfg^>8Lnz@v;%6A3{eZX?#3Z<(fN7QuOHT9~`8jw5 z9nhdXLuYjqiVjvccOci~i3`~#uceI(8{zxsf!*OYO2nQ{vu2wba$ zCll!4=X{1Z9-s(eR~+eM!bB}ka-@DobYBSbz!b#=nEJ^Fl89=u`PafFHadO`72>KvG|`trXjFHIis7S(8#oPrIN~S zBq(8xwk!`GcpSw&;DOA@tki6&Ids&t6(>==kyk+1{HSw=ZCWe`m7a zI9C*|*5(Ml6G?}4)xTlxHTkl*$wy~@4|n3~)U{2)7hSZBhGcv`s^!-FLdK4^3mj2Rl3J$#?DP z`h7$CzTACI<2#%V2dU1V#Lff|NS)fNeU0Egf(w0>llP^oka)%a_3m|ur=n}`gQ634!?4$qL5@qd`|=`qFe435ZIe5F zQGh{V94zg(=tucDHSmCo11Yp(Ay|(TK@PFUR2%@0GqpG}icM^0RGw;jCp@txS?`|M zXvj&%MltV(hacmhs|!1NZTin~g00!Kud6L}?Jn`wO^{bcb)u`(R!!WZi-Jp+1~M!} z!M)k|n#V%cYD?+1Yg05VcG^2wbwcfn-pbydKHR?ZPRy|Po9LZvP`j^EynZk6a}Q4T z(%cDXhv@sD2^ePZaYR0O&Ui5zH@`>$)p!f;5+M4+yd|X>*VtjiRw}deg&qsd%JBy!K6vOBbQ2n8)|z zJ{ztI(?(vZ_R@|X9)NxN0PXiAlH?k2sUSy}9&d0dTxU~hBV!Y1Sj}59 z7_*i?ge3?E7TUqLv-HMYZd!l>nN3)`1Q#VrkTq`Rjio0M)!WybZFOj!_4e-piQ$v;=9VlE=G1r#;SCL}F5;Fw2kOhmfEaM$_<{6s z`f2eqU$oetRB7p2uq7uijdf(xS7TVu*CW=$@}u&PPgYH*qVYX&ck7BW%XFW>pymI0EkjKf_q_!Rcc3BDd?Pfp<^h~R!4 zImlRUkUp%>iCnSE`zghi7?WzHr|Uh&0?Iyv*75y7GC1|LrwlkwZ2+&B(y) zIpGCf)D~&i*Z=ceaF5>Sl%|p}L^!O`O(RT-uHTJVoRC%toOv$2m9zlaabnNLkd%9E z<#pWnj8}`LT}k){-srwo{?j?w>3V9(_sB{r=U5NLZj7P!FdqFy+M0^X(Gp#+%2Dl$ z5mIVwp~LWTNffCUnyS-SZg_Tt5m)MEwB*A)N7h$D8DeSxk;1K3@-oA9g0)7Ff0Z!M z!osAwuw&(M!j{aSZJ0~96VfJ|+EaL{22?HDtRbvGNs(AYr*lWgN8d!azz2g8;oQhi zFfv*|OX8XirWls!N7DE#;6s@iBTOI)K(qrJ=5vqQCqd2vbB_qYA9V@3Der&TNL?_s69uljD_k<7`Va5R~|M6bbXaF-_+FMJ0tZJz%$b4Vi) z$>=*p>^xPx+?MuxJElZFiYfLD7-lvj;=HqGm+C&HLJxwXwu@}5LHe`WdVQ(*D+HIx=Q4nR8 z01^S&gWo=RifIsqbQtk8oNCeXOvD5wkAu2U7FJh8293nSitzdD?@k?qu-4T)%m<@+ zPO&c109NbWRWyaVT?>ih;NSyI`9@P+{>m?m^R6#`2oF1m-k>aFI%gO&{p9wo2X&S6 z-r?pHH%8gL!kNg`J%0N$SM>~;kk%G@_I&x!8d>=2qshQU<4{d=3x!|V=#!w;(3212 zj{yrl%^z&IOOVVcoYm2%`MoJ(vwAU}XPCJA}4zpQ5oSj>8UT98AF{+#?JV_u(k` zn$N^8UFLzRNSwTye%}bgfb$1W5GmZ39;r(T`No=0^Q{ zy54ER^I`63VOSwVOH=~%>8cmbXbxoQu%@_(O?PM2D-bwWF=XOhl0Lu6@NuFpF?pZiLuKpqHpa;fb+IiBvNu zD%!oIVlaD!`JlXR%t%Y9;$tEr-eh*M^U$+|y>8&la`Pn0_?nm4?@)5o!K9$6g|ND^ zDgz;1CmnvYVYFscbIYQa30;?G)b1CDN^&2s3Q;B?PLEZssDeTmELd*>_BOEz)+AYk z$ZO!&7ch{fYr+5|KIs|{X+2fA=3=0oT5tP zU5lL_?m@rfm6)}&!3mi;c3Q=c8+%G860H%f-9gI;BPXMuL}&$)QD{8Uoz9Phrb*$d z0zFGR8o6(%4q>(qFh#Z&x>=t<#TqXa?4|g_!^m0$Z`TkvuNt0q1{Z{nI&A50eGJku zT75BB3*RZI;(OV3Xt~7^g#Cll^OK{WY6;yW%y5!;6NTrM7Y0O21bm`L7grScruxD@ zNQcnZf|-{y&`A7NkH{cwd6WPWr|bC+g?&-90=M$VUT?LHpqw`x;$pjsQ#!y@5nI$E6vFFGVyBSr^G0_q59E~tF;U22pu9K zO0>CND11+E9OejF_(|z{ha(^3ucHmC-h9N`=d({CWPY65*4lwAfy4fg7b9BQ6)svj zXppmmmq8v|hPx^)yw{saX-k(uZs^oA2Upd7Ls3(MFqjD^ZUbc_P;wCtVShWVaTSm| zN*=BQZY59W$Up>n%~7Jx9wMiOpD+)J^o~<>ZBG2LF~wA+NCcFjnB=#3oX^U<}AQuEVNGHyy^JZabg_<8$E_6JnQju}nXEQN9$xNOceLi4~9KYlt ze%RQ9IT9cgncEast9S%*w6*pY!v@gp+$IX$)CIK!ekZb4NxH2bAxVw)ZBoSQA(4r1 z9~d;}K0F8ev23erZB9f@SCe6?BGb8+0Wjng>)?1-IPf9VbgYS<-O|+Z!$NB|ym&Xs zry{Lb8&#jH_G(g8sN4DHjudV>lRm9FHzTz=Ap}L&fde%xR1_DM-;XHwjDWl`tSN92slsY~c{+_h;#U*5S|9RK<)BjA7gY~FrtMag5Pb)Fn} z2Q9KpTO}6O+A9?x*41uU#j1sa_{YU11Sh2jQ>{7&r1TL?voG0>YAZ?wIkT4)IG4OpQRSI)=B`Fb&nb2(=^(=Fy>&kQTG`>$eJ&NWTjKu=WD&R` zaUQB)C!S9JxbHJi@<_`yU(H0LZs-rXY_?M-36?3*IS>iOxd%$GrF0m6wDl|si)Q-j z*(z?3MVh~ppRynC{`4XT1xMR0|M{G_E_k9RsUd8magD3fGLFjow%So@jn^uqwuC(A zGvW?>(UUQLkg+HVMK^|DxKq$(@!YpT)--$aK2L>mIpUFBTJ4*<_}-DP9qGl2*g?)k zd+DGdda%;S6{g5j>x4u0&kP+e?eRm-6RO`kFLSq8y zy|gXg@-$&_C}m{bT-oRSl+xl)$V(-7?kxI(cBJ-ANAQYfH5uhq5bw!GwnpS03c~d_ z@#0)~7g|o*%?*zY$KW>bV0>j<%t-!kkJ9uTx-leX$KLa zpdhl0naCvw)_CIW&*kCUrqJl*wWACb17F>gJ-gzbb9NKbN-i7&>2Q1{ydK!O!7|Lr z;XtbKHNuLVutszncLne2fZI?MdwueaBxvHNpGimW`d5c4yDS2A1NWAG!SPl%?bRlc z=!!OA%$nYR>#%*=dG{>LZ%{8Z?en**gd?mo;y3k6(H2S^-m5NN*t6{zlQY)91Cb&T%(j6I^lSola zaodfdj;krBDC8IINpTPF5Dh%>KTA?F$?XeEPCmovcmPjjRY`5%8Fw?L?Wn@J%9S?A zD7ZDo-;QhBP$Va3j4nH&>g?JfpF=`l z8l-JT+vIF>iZWWd3zqBfNz` z@qJoP-3xm8e~<0G^=*Bd_cHC(@>*kD6^NWR^x!y1@QH4+%~gdIv7fy63ok?r_d(ft ze=VDYaQAlE%Zgem@>r1=1Ysxi_U)4xg+`rM_@=;a)lRoIX3du^lH9wWO)&MEf7^HR z=NL(sq?SLvL&uq|PGXbxAc_A!RqmLHi1IZxJ{y@{l8j)3g=7I%=& zxw>k-;TJH{tN?!&!;9*6(H%2o1u{=LzV0Cy+YuyDy{il5F~b=)I0C=WaCeCNN zaQ7Wus`eV1RI(W@;E6{9yX)Fh0;Md$$)+3MyGkT3X&LqONXJGk3e=iKcsr1nQ$1KpIb`ff8*pGsc{%*8t-#T(-Ypbi^c zPkBphT+i{ejz|zy*9|vumeA|=^Eh$=Vn=^9_m5hp{|P>W2rx|)pIItmOOTVnmQb*z zy_xQ$iuQ4uei6YdpJ-QRB+v^L-0?dkBWIDv=`flNxMp^-iTc5JN_?8Jmo#A{ZqPZk zYQwhE5R`ykbLT46IF~B*g!ahUMv6C_peOj(k<*Lu(OVifK9~W?sF94m$hj9%@~j~D zYZ|bJseql`u&Jq?bL?uBHD0La#aD-bBLCGn*i%{+!V8Ktx)=1PFz?F>t(<5Bt99!TDf z#!_e(RU{;e4PQ7`j0VFPaLf>(*kWa&tQl&m5}+_wQ|WX$!K~H1=?EZUcYAhR?&-A5 zs2#=E&f6xV;XISG|G-%{g$t?B@0OrVb1}BzASNbBw;F~cy~)84%-n%bXqIU zRE-AqyX>dN1s9-jOu+xb*Qyt0Q?fG!8O?vu_R{uw3$oh+mh#rZjyLR3rwBgpH|}Vj z1LX)acUbQbT+I)rNJ9b1+{g%i6fEiP>!GdpiHZtaMpC?K8!u3f(CxGj=CPXxL(Wgj zhwL7bq-|N`?IFBi+uKq8$@_;MuP<_h^AB`gpGW3x7yAV2wYN}-x$^Okd)iE=@21EC zT@MIoOhe-u8RNvBjqeE#P21t=w)g8cCODCp9EA38o9s53-mnjOoHtKxFvHnpeCFt7 z`s8}W1}2ywfkeCSpOA@G#X5M>ES1hZ5>C!S4ns`a2)}UD1Q|prQ;vc#P58RZ$$G{( zqe-K5=-NqItuE&SK6stsevbV0E9 z+IrtRy>TwjhOa5Bbz5yD#t;_lW$%0x=NW+DZ~qO<(JB42Mypw2<&rQ6!M=mCGoX8L* zThDV02{5A2_NMn?_Q}Az7~>B=?}o*Gnql9Z(gpDl{lE6UDk{!p>z3dW2m}l665QQM zARRmqT!Xtqu*M~LXrS?iKnMg2?h+ul1=rwCV~y+W&q_=Za}_20<&SE)zhr)+t?8=J3sEFKj=ICr@$z`cplm1E$YeZh8{>!jAi(a-(R#hI&Qakp5p;R~<}GADow+jx@zUI>)xI@ly6 z!4<$b8Vbd!y>>$G98C983a!H(e|TKckJoc^Elevc-SsVIm%?PoU~*=GN#D#=qFa>0 z;5wgO-wDt@x<;uX$+ObQF;hyomO*!?*!h0$y@Z5XcaaIXC0hliI!)`5%i|U8Dfn^0 zV#vB>vTyi9&G*#dYIrP*KHj^#^Fcv z?aI>r3YH0pLi2-3d${915^2Ed)WT{Y7TqP?E9b>Cp_1`+;hH+_6zmbR(x)OKNkBH6ue8H9+7;C0)E>6+4`Ff zK{@EkVZdxku7&arkBmAJw95F1b9f5(0*enn7P z-o(ULOrI8o(6tHrF3sh^>DL$hh8s7y0_2a-q`>T0pU)+Uv8lq}Hct~>5v{B!Yeh`$ zKgvGO*54HPu*6KUfvhVZ5f zc>J&fgHL6s`!^_-oAAXWm4aMLwIaDCHyV+5P@k8}FAaF3hzA}a_@>;y#ErqyuP5r! zrOxuRA<~=tD$$RTCT52LnZYG?ZIHUuO`GqSgi;7qKyP@+x`&?8=1Yz;IgcNuZMVjhdf;z zpMa-|;$k2*e;R{UbsJd@vCTdSM;b~Dzf$0bi2bq^!zK`&UM2twADUgAV^6KT#2L{3 z#E7-)o&S%d7U2_dK9atQqa@K>YGp+VvYZcgOiXC5Gvv>3g6k06X$U5T3Tv4<8%KkQ zkhDEF79ZJQpN3r!L_u!D(3M>tg}vQ8mx>_bqp{g9T?Szodd^N+JYH_)s}C>f`tGzA zL~@-u5M^&23qIKE3iV?kC$>^n#z5~LsmEmktDHW~)$bxJ`-qesmow};o5q?lrNZ&h z0p06#VbWuT`%GX_Y4q}Wi3;5P9E0re%OHG)t{uA5pZYGm0D4|CjPLK^cA$SHZBF}# zu$3FVIF`>++9%{izgJdswtRLR^_{(sXq7m@)P?vG?v}%r-1Lz}-bYGB{7rIR^N-arqhWbE8z3Vho%{*;KCPd}F&-sj zmGM~ooTj2oEcQ)llNe8@binx2X@54e_92gkY{2~&;Lw}7X)jlr-eCb*lEsU`DaosJMhYOG>4r=s`5H?B0{8Fhx}wnc)80B6M`@F9O9M_SiiTslL|(@H(j^z5Ca zh*mQLXW=0GNm7Bh$fRwS%qO%}=qsDOO7Qq|&^b4Id!%IKUiw=0rg7U79>=J~ESevL zwSw`xZ&^xd70t; zrQUhC{7C^KsMA_@lnvI)slbM0(l7mUIr=iCeE8yxjJ>q@YxShnqK&MSW-DbF;X!yZ zA>UKt$zdJ4X$(qWjtRwu2*0w>^^9S}Qf5z6`nCeI)mIdbY8A-gBP5Q7hoKi=H_gdt zoG?4R#WvQL@XuJYhZ(2c>i)Ad2IP+Wg; zKIG#{zWVE=ZJbtw2X>FH6Ux5Zpse$pjcG?%0|V=v*36x|SYg{}AGr+&I>jN+V799? z5qb#aMmV9||X zdo@w0y=2~ttF*@-eH;69ow%v1{v}h4P|SYt>Y;mtfA~To9he{YoSTpI_{yR0-~cN} zf6*1MIpfGNR0mfQEwk%@11sA!BQW-s>C13vDx0>Zwp>ZL378={U)kDDo_1{RjTw28 zt?uT_aqpD;F7a-~EBacsxp=;QUs(U+e){X@UO#1uQoWB;LegA=tT;LwO>?LHmjuvyF}Utbp6G>_An?Euo&1ZKY`=JW|B+Of(sw#jFr2 z&ZabZ#|>R#nT-(X8_~0{($I@=Zdk6P4|8$S&RacVY%?FPg`O!xS=523P4Jwu6vD87 zmp@uFj`;2y=;fV_!T>zKoNe2vWstCUPCr*A!g7uD?8mTFQ>#Gk0Q3nas5enT(h`XRttU22qmz=u~3CdgZ=7b&aFgqu({H zc(ez=+9gj-$yHCCyO?h!5QBFSaW}N$?dddj997_-Nu=%VT^rNZSSbjp#@ zk!p>y(5k2}_M|+1)9=!M@?);@p%c%sUIf(B7P0*-IC>#G6P}%hiw;((P<>h2GtXaz zG(YlEDxekmWFMdnT5J&(zmHf8sRLAQP<_ogzgYIi~7By3~BDZ$)-4lPY#ZVg`RBDI2;OeZpCy)|MMzsbm4c?;Mh(uC2Mn>6{#%x7Wvt zeI$cL11g(wDSj9nRQJ_fnDyJy*`ul;`};YUl6RC&37J!;lmH)UJDg6*Lndjl^M|gj z?8JI10ZnYZH}k6^O7qDu8oe+?Cg!UD??m>E<>7wbjbl_HONPIwO%Z_AR;o@{RHh>-aRG1J0!& zp=9NC8bWvS;uwxIxutdO6Sn4Asv1h`iZyH-GU?_VM#H25ye{X?gN!}@4g7DDVX*x8 z%e$VcCK{N){+hK({RRu;PW?)fR*LtUYphpNs+)F-0CG|)vVtWE@X|c!uKag>o%Tk&{jmGf!Upz4b4uc;XrVb3E2ERmM}D&>e+Jbd@VjGa4M54)Zzsd#hfLj zz4wjY6usA8q!PhjagR`$ptokr&5#IAFw1TFS0UwPu+(FGh`)@w7by7pT3oXK+9yOZ z?8|kI{uwH5*`d<~hKcCg5KEeZpEjg`+wjX?9A-=y5xlhUW1#8@DMsqS^mK{k&7F7; z&hmMiHSQQ^fKX|jaf0o#IZEfF;0uE+LB(=|R@q^xn=B7OH?qXC(Dk&UjDpQbEkl!? z_rt;)uPsJROlEO4EFQCEnqRDg-IIp|;L?r13O#Mo*}P|$Xeh0|eV7Wy3O=Cm5gXdIASg-DJzO1q&--`@*;6mCdax(fMQpNS=mbaeoj42#WXkn%CJ)(^qJEV z-T+}=xJKXPDN2a7tm=2b7q{ePu+gNB$Xxa<4^vy?li$?xvo45<94pT-1@B_yN~if2 z`NoSW6ZYsn4;H2UQDeTpT&XbNRsJ6lt7;(fTT3Cx3uiKjYIZ8Lg09~tJX2?*&9eXP zm2eDb*SFzBYm0h|B?ABG9$q(kqO}9>DK&qV|1}|s)F7un+cWzcZYxtO3KY>E?XnrV zBGUOCRVGf#l{$Z&b`UTE97&6x<9OaSi%aZI83qir8p~$QX~Bhf^hOeX_Fv61)mbLU zrvy}<^p`!>4qIl2JjAt12^1;7{41cw4MVuUSX}@QebQ*pY}JdH({sQ!YvcC)`0Q95_3>tt=)9ov1Tuw@XT zqHKq7^o}Iiot_Vm<@wLPQvyGy#0ESgp_K?gm38zHS_uCl=@Y^mp(65HmP~q4rt59T zm>MpzX}*l%86o0jhzwD&>TUY4a59Rr$xb|;Dod)*r(_^3% zqP|^W&_$@o=TrHht`V%*5WwbUPlHt>d;%Z7YW(gAqbi5&@Juf&C<|siwpr>Vq4Tx& z!_6M1ClyZ?7O(O(D%yR}ThKt!5c*yjmN>%Ec7#$6oA^Y)PDn)oo2>cHs>Z`)ee&HU zbqJ?#JIO$Lx`oVsRXCj;Vy6Tr3aiHbhHGpRqWmPcbu#esHDdp(ZPy8W_haqY0NyOAe(r@M5j+6u0}OEpzOk z9IfZ%u!c%u^!E}AjFkzmt3b69`5&*&Fdt~_!q_Ez%_@?Z6l?BDn0Wt4*5}T zg70*#=HmIG4_iOwP>yip-cR%XN*um%fP?$nT4BW>w4ObV;dFe1x`k}TLNH0x?u=7q zBZVljaO@WlC+f|DNEy`b3n4?6Y)KeUP{K8Q5!KH#D5Cs)!$e19SK0$jDmUjTF66}y ziXUb@#6w2auxm<+A=A>3@ED-Kb*HpS-rskbz}KIBGitoet4P3x_!ar&Wx283@t;Y9 z9@GbSIW?jue4Sds;+P8$o3RIjqpQCv!((waG}#*()gRKrh%UOLm%=cPNuY2|p*Gv= z-*ZhA39SAe(4CW>-S~zt4C^VpC=wmlu+;w97tww=6JfQefaBVskVc%#?=h7|yv$2|UVbIG?aGF=sW8gcu>t1m=j%>C z=5YO~vxVVk!v5gL>;d#>X`h-PL>Q(-RUSUrTiHeGDJ! z8|~y`Sv@PUPO_BP;A9zVGD!Avo|k4x%5D0Iu;w0&0p6wPSRu_^v%6^LHR8`aE@EaL z%OA?QNCf}71b;5GTaFS%X@xqT1DaMO8u;NMV4BKS`UWZC7WT0u?#nG6{3S9lES56d zBlr%JaZ)GwO%XWAY1w#j(iw`2RG-6Eo%xcf=aDR%K8tZ$K?0pt5==$rxY zBapC~_HmLrdyb@RsJ5L+Ba`ex9Y~th<38UjmXOqQlr48dOPLL8!IOp41a3sYlY)wW z?IdO&#GB<)YxBjvQL8044cHIlqBJ)kh9qx!q~z9M|;M9X8NZ~v8rlt(N}qF(Kr1zD-?H^kl}AFUVbcN^Vrw{ z9zhq#vRXZkZ*H+VU(52{G5!c1Q6DLdAH(_j)fN|`x`f!zsp_+(8-d05NlJMYaX?~( zY(#j(Dncb|{BAh6L?ZmSu~+7o@<+T>ZMSro0jmcr zSr5e#oCh*c{IHyl%<;E03a~oMA5W|}nWyCqCkdT$*~fUtrn&(fSF32OlP}lgDRtm2 zzZRJ#f7AZIXAgfJ(o>Lywhi!QJcW;`9=9*1G7&pQQ-o1;i8~_d&tW^?&{Uyc5v;r! zVjUrk2RkF)`)-LwsI)4EFcQnObji0a^vab-{z&wxgI#|n=z zR#Sb#22>6(Np{d+u_+n+aVGtf!2bTFfD19hY>`y=Y4(C|g*lqlI)Wtr;3r#e8BrOe zoY&bdwO`nBU0$S_n`#7LQD-a}vfG#+pE!n}kjsl74}5ykJGNsqmRzuz0?>-oD`)#= zsGj-10rRg=4K5D;fB&HlK=5}{V1$QJ@ZM-SBxKTqIgtKbfK4cRBK?CquC26@frMhe zbxjf0Db*NAA4RZD{cr#DA2*7n-+IzYlHtW$LtCWzJk4tqElK`0z^MGoA3{b#WI!3Ql_Stl#fB{smys7$Q4{1m8#(@ z=OH%Kh@mvuuoA9}*OC7)jAyh?51K9s`d(QDzjzv)S?S{G;iQ!7<0hT(btHDg`#T8y zERwHV4uEh7f0+r-cYI9BOdUjNld>G^xPGoxCa6fFlxJZiExd=X$RcK#F3P`Mbd=i0 zw^F@QQpI|fori8`K9#)rgNWfZ%qqzMYac8Fk*bwb&>i;iByUFZ!-xsX3S^XlNTW%w znd6vVlD1+D2UN;~^f%#2u-7KmNT_YUw)r4P6ZzAqBS*Bz8m#6=JyIq4Z z5dMx3yJ}tJNV%oQiSv|Kq@*OOQKoqQSe>D%f=IrVxXt1$->GJaW=VA)d{EM>d3SR! z`F_QB%X3R-3t)v;g~f;7Kunw7Dzk2^)+HxgC8$W&Ne{nxK#(Ry)MZYzqiVI!!byfK zAk9A+{Ns@h1<(Q9uCX`tb6)XQ zXmd{h{u2@vQUPWB32@t+fuvw|n*Ft!H+2J zO0tV`HeJ?^a8`zbNu_;B;kOt5@T2jRfkZtG+$VnC&zUX%lkb@Af5sHhJ9TJ(x>O;1 z9sLAZ_biDxE^SBnQpZ#z1;(~up$q~@K^mQn{l!z3JbetSn?&2;WMHFad3rXJ{>xTQ zcmuM(Qx9%nBDz%&55t!}fidJsS?K5NVH)L0G!i4NZ{Ol(R8ty7E<2xFT1lK>uT>E` zf8pp{6g|4)_PQGpJD^_2r!11KiX9=}uZun{^9c2#s3;MYx?xX=#f3;+KJg}8DXOn9 zTTn}N&+b)Cp;|M)T@0EQ*x_gXoGP)5FDxniSg=FF$hwBR^A3U>6Z#|I#t!S*!ypl9 z_3XIp@{uyt_*CzU7;Mmwi~8MkO-aj3->0Y`59Y+%fuowoCmbgNomz{j?~cDs$C>&v zEN-YeM&PF(Nc$#4ZQ814ecNNwD4Tpc;d*EqLg4*%>20 zuzm9D#Yey9B}>X-3`0(>;J5ZbjFrt8@DcAI=TQG5Yjo>+HC%Q}(F+STwMR4?dM^6e zqeIK21oQUctt#2$yH-XgDEcibil2jA32{X+Z|mLcLrPKU8@o#aQmi9Uxr`dVXAK;1 z(AY9Z5_*6A7AOA^C_%h(MPx44^DE!}9~5kZ;pCEt#hx+#&@@Bn)BP6L(BaI&S?gp6 z1oUcP*E&uJ)j25Z$YeYLgM1_D+_=*p!)slTco|J-l&N{HpW{3godjb1sa{Mbjdzc4 zQ~t%O8~=R9c3fCToJWg;hYIcdU<+%z;mHvs=#a~FaJ<9OjyheJ7dUj~BF$a`96G*y zLsw|9WG4Bs*UWwGV5*zV=P@7p2j@{itH8X;?{|*ucHJ`j9x+qAPCU1hQ{7!wgp%YCmUt3Y!G^R-HDcI*n8 zI+MQW3@Zzo>`Q4wr3;)qTmP*AN`PVWvkleW|~W&g(oc5R|bo zbnW(xBT%Tmig+Aj<q=3{n$1x}zB%tm-L8pF=s6SbQT8#5u5wOzLP_5G(|BhcFA5|zoZbtsyUz=i#b?GZ(nnw>$Z=vITqZ3%y`-JPB-b<(W3w1Nr!NG2c>ifN1wFaC2h|yhMVvVo--lwK zmSf}Ik@4a5)gEB?JH}55#6z6RujM16IQCKca?njPhXdPn<%Nz(4}(;K=y%L31a7^> z5#J)Ui}f79;^$|Q{bfx`o|0dtatqrbf+49Fmg!g~Ve$L^|zY{#AIk| z)VGw%z_llA3-GLL#V$c~ZZ{W7>=&U)z^7K+G`BdW9(cCU(G%eH+`!3%w5E)hz^us@ zeCLBK+!bMYIUw2>gyol-Pew3#!G|{HL@N@TDq%vYp6!mvBTRX8a*Mf=*{vVlZ_WyT zUhIhhYj|#<{KzyHuvnJns-9p7T$RFyx(dM6IY0_6=Wtqi<_Nz(=z5`8dE1_`T8wqz zVQCN>IgW3>9KbYZ|0RcKG-5) z{|6*r;?+^~UZVMyhN3FxIq`HTDmF#=wXiUQooN#!j(wn7ZikGi{3loZHt32?`sEpoh#$(uKzLW^;i)Tk6r~A7Q3sFYh&M25 z!QLRXCB!rqtfapT42RGJse~TR9@Nn9err-{tpJ|aVZnr2@J&t@7U+gpdwk2Z7g>3p zO~)LXc$wyWJd-Gs=ag;Qcjk^dif$y-Md|B)_$YSKUB*l!7BWmX;=n_J`{DLK<<>n2 zfv6+}!JnxSzfre|@>mCh?ZxgFiE;+BH}|GD%MBj&oyxcNa`vfHc$~enmyl963LS{S zr?27%+ehB{i;f52_q9GNz+j>U2Bip4(baNA3FwEUn-HI9_aG|y7ZZ+3M#D2{^%_H9 z`qU~prW=ZtixqfJV&ucZ5t*%abQ0IE;3&_GPO(8c1MaKdXlRW1j(R~h+xXXoNxts5 z#$v8+Ko%L>Fq`HfAewr0?fTP*1-$(CL6?kvsl7{M%Y`}G`6~=30{?KoTo}=$3O;<7 zyVj>~e5CXXmZ}%60aN{{g1eIABukb4Q53*Dl_uUg6r8%Pdz#MYc|sOfaf-uXB2q;% zH!W2h@Eh0Xfi2;&fkOLDMB#Yky(ud2xWn@3il`8__B-YNFG@|4gTFr_X>!%3jOI|x zgGgF51tE@o^{waLThUl+D^ZR)wsT1)+L~~Qa7Iqz^ud5sk7^<71eDdz0rCjYyxN|Y zkYLya@JePnIwJJRJwIK(LZA2|kfR&M5Ya0aFx;!lQRfoG1DL~2b#8*!Bp06#ysd*S z&i%B}HcBJqr#EvmxyqsFeKHjDP(|09y8f3}LR<63UIsK0)jUyXG|`}SF~|y=2+TFf zdS+N5`^(u!=oTedUu@a(`c8G^d?^?zRyVR;lefD*;7^Xz?i=?OAJ`<}O!HAZvEM_M zUXt!$c%1;(lXm4%-~|D6eZ;NDdhKP4INU7(Yq_S__J}$MHtna|qwmv9@agx6-CCUI z$0EIA3^b8$f0gXD)4MQ*PE-2nE znPec?-`+zPFut(8_Cs6p4zQw*z~f>I8a4pEqQi7hTUCGej+F4+N^#^FYk5L}vqF*_ zW_2IELkPVbYcek0xeH40yNU>`$6)3tEL!TXgv12F6)B65DW#3U^!hR0TFd<=pH}tL zM&2>MhU=3lUcUp;Pdy}v1!&X z_54dRw0ttARyWNP0M|*P5Xgp;!2> zY0<1FfJi4f2fO21bohMuu;vOVzFny{SERhMkRY;e&N@@+U40RVpWaCuG@t~oo4Xax&0Jj?J4;5 z9LAHi*f9H9J^MWC5e14yH5yi*T&u}uVch5&U?KQu5 zI~zJezyL=3a4&496N6cIfAcWHsZ6hl2l}3vd)#j>tA&(LPc7sbd>9>Ha3~t4jR%~4 z=;Ns}aWa}S6^lC`Uh>#D@*TCzt-0K`=s4v!0rfc)I)$(8<%#TlVn^f08ec4altK%2 zNl#x#ql&dR6!UUg8G*5TH5(mon2ifJ8Tdn>q&702!2AfWf?uc@Bjam*F^R`ZJU`>Ey29i1zR-8|UD@)os<(EEl z07I+dU#er-RSoxxi4M#)s<()os!caYiK?n+PIs7>#EvP9#?LYEFTFzI^!H3r8_wXVx1N0=u{4{Eqg?jSXJp zz6Fb8Mk+hamkPnWs9BQrUcmXbs;{45R&dlbO58iMWW~=*)Sst>Ql@S};U)Zr1E(qD59Mu($<8g14gmY|MWeOO^9a%SftDI5RNadn=^Lo|gs zFIBBboMh6Fdihrr(voBu%aL!d5DZ&cgjb>}NN(&K2e#kGhv5FpcNF*75TJZ+#6K6x9e&C9#gS`+Z8 zzeti{DtShenDx1!O6}7}eMOrLoAhPV7Ru+u{MoAOIlY+{Juutloa3k_368?XPPqlY za=rX1;{m3G011W4}yoKcyMb?g7Zu8mUGpEb##t7y@3u znH>J1jrGT_2tWvQXEAOw&!p6DoX}b7952QJe=U;n9NOKXw?Wf7KDm08i8hfVEVfj^ zFy;a9$ScEFa0QoE1v5gaUxHC<+~>?2*>Z>M5XrV-ue4gV>c35f8~SI>6+>%yORGy= zN^w>>V|0KeabtwmQ{}yW2|MwU4255l?-lpqi(l+cwsCo<6Ji9mzj0@o(9&q8zO-^# zG&|3`t0~4g(_&y^l6-N;b?L>UU@ghXU*LN7u-T$9)R7wjiH*BfoHp(JF&?%0DB`q&h2JLWwN$tG$RG|&( zpE%HkO{}hH4JSR$m3G8kE@=ho9@Lf-ZPsG<*nu(NO!zPGyAe+p`n=}HC{_rJvn&Iq zj88!93%`OJSSm}WDPG&OeJriAN7MN{6b#yV)?8r7%!^gE5tsG`+kQbkUt4Zs23cTq z^P`Aq(Etx~g?sOjXdS<_t z8)2eTY5bt=oZ%GMWK>~veBG|ATg({O((x12QLW4d`g(VlO|w^p$}K+h6TK&efTTgA>NXyKL(OJaS=I{~*p;)_-gS|a z%>tO|luAcMG#l2=rv7nAZMCn0@)IZx^?su+jMMn)!z>rvNg}2&I;@x!su7ctgT}oY zhrUiM8zFIU0)Bu_+#ks3Oty8KCR`cJS$b({o`KfhA+eSW-*EIgBaO+4XIeJO**lI_ z&BoQzpH4WOidOY0C9wBigPi@x%qHjin$!5noKne-1XvBnFYVJ~yruu7xq!ABM6yCU z&pGk)$hLc|jFeBO_cim9g+uRT6G%fyv_7lniip}8Tc{sr=e@jsv`|u}d0%-e2>_a( z7R}~N9(LtdPi*rSjwdy`Km2T3+%hYd+S*nDG`#u&tE_CFoq~h!ih#8&aPDfSIH_+v z)ywSFW+s@?pg*VZ?OPW$my?jt!put z#T~aIi_1_AY;X@Z1{)xW-Hwb@4u)|)DZiF{^khnqFxHBaDy!Q?G{;MPSGFW~Uo$m- zydKtL5!iAsk(N(ER=(df>`3-~Tmt_$&T+TcjN3l)CPnUQXYVW43ltVfu})xkyHva? ze30sw(D>6KQE*z)`z7xj7Vied4UklsZA8+tQ5n33=C|vZvP@o{B%5x;q|bYjb=g}cZ~x?j(JTpYeU*;Rt-nkZ z^L;|hxWVI7ox(DX-AeNGq>1Y-3_$b!PH8l0cEKV$s84K;-+C^0!(Re_STXj62Vkc? zv_AphKbF@Qxz3+@J(Gu;+mPI<@C_+}N2Fv|X2+gR^`Kv3xO}d79_wFG(JzE-(7`=` zMCoLMDTan+CliR&PMEDQHjk5@HECy{_*fLce*QkM#iDrFr7~*hp7Ci4+_1ZO3vfS~q(C%MNc@Hlx05-g&wXlfRVi?hv#_jKl%7P3GLvM2 zMnAZAGp>k_>vl)X-{MtVb<9pi*-*9GI!Ee-QSSoi%Kyoabj6=Gb9cB3Uz=ZB#wR_= z81R>RMbSF(R7~Fxj8PqDr2N@%YSfZ!*7dE?=A2iHf5X*4k>Tx7(W}gz^XB~-Uw?hj z#J%+^q?prc9ry65m)+Z1y!TX~a7NHe>;#~=*PBtC4RS&29E&@#xruU&xz|2^`FjvC zVff5p{jYm%`p_zzGUsAd`rK>H^y{rHn#8^2UTwBFyp>5+${vQz*|C2i_|u0zKvc&X zpK!!{fT-6hvb7#4#8s$#2Uy?JxPYw90Nn3tk+{5O^)BB`p1fZR1o+u2xZ44r*6SQE ztx*8nw@Ta#V<#ITJ?iH3Qk^!%TD;y`7=mXo=<=uX8WQf8)(Qaz*hUm`?)sl)Pu};M zCg{=gm+ZT!5wWTU2lk(NrGljlz8eGwOw9P$yh5Vnm(oMx5`3|9YmoUSoN?m&W%1fV z?}phcVRpj()bY)wo;8zRe|pG^mdx=cAEs9DIKrA50GrE$9VVgN{gI}TX`=L;=@a9D zqK(vgdT$nB;oTWiZ}M8g+S6Xngk_hvyg7>fH;K!briSM;VUFfx?y-e|)z|dyEzN8? zN~xRImi$m%XvL-(%CO>R*zL*M=-SGfe6Mjm3ug;x{JK85aJVa}mu-E3O{0}KR^^V? zOZ8Q&{Lo7Y@LuMPx64%C56ecT>f0;wVvU_!?Sou8&>*0s2zU=4pN-ZO^wE@$4`UOf zmqCTCi%i9=i;JdH*;)*M;{3RzRa#l8<^A|(-rhmhxDT*(Eqbl6m+&1=x)fxyc1G9u z>E93!KM-)`wq;i6yEkaaVN*uyLR)dzTw!X2a*B4b6v3)RJ%rIP(TWebwySzhQ za(;SvRn}XzgHRd>w(kEBJ2p7_MUXys*63P2_=!JcUWi1?@I8sUVbPWQ>zk^NaRTE4 z_rlh3U)4-Wx%;n64$f7k1l&pl>xvA9i9H8oz)VJfmgI4t>T0s{i`((bN;CeCsOyHw z;|MbaPz_OuMDjE@_FBbnXSe5I17gz8Q}T@#!iKVgb^GSGrBBnNKuS|e+ne3WI)iw< z-3gPn7f*aSVOd!x zwRp3GwMqbX%^vf?>5fxr(NYnpDr09Tj5JzAkfa4h1FB)=cQSxL8))yApc@|PUuFfy zg*U$UZmg~L+MwoEUU-;i9YCN00^Jz3Lk+vw%F0 zQm1+zVV^K__EDv!@rUXzGxh*D9bd_iBcpgXwUxAvLwhe$qH~`eMv`EdZR6@ zAPAGA0+4k^oP7Uap!-x&i^*Te@4ka zuhl=>{l8EC|JR1Pya+4^2&k%nXVMyf;`F~?{=0qt76<&7AO8R4hyPN+f2rXAmI^dk a5B(#)6uWJ;Z$3hR{{fyWKP#0n4*EY*=#0kz diff --git a/docs/img/triplet.png b/docs/img/triplet.png index 8b82a09bed29fecb45eb63ad12d48342ba47895d..5d38ccebd3f21d8dc95ef81aff09e5dc02e3ae2d 100644 GIT binary patch literal 19255 zcmX_nWmr_*`!(I&-7VeS-6$a4CEeX2E#2KMDcy~l_3?WGTkI(PD-VehC%$zfO z?{oKBD_UJu4i$+A2?7EFRY6|*BLoDD2JpKB0u=D`IPMD%1Oz1G7bz)q1t}>Cbyp{= zFZPxY5DZD?CMKND?k=vDSSBXp(;pa-T)jRFe(w=%1#TGMO^T%QP_9CBWDn zf`IXP@~J{0*xlfPlf?+o#kasK7dGloGf>xrDt(7&)KIi_;e>`RflSOZH2MgEW1S0a z5ldqkwni*|QJhF&P4O2YaXJId)EKfch2nyq!V_~2Tl@h(@!6d{CmPxwEu00~ETdFA z_`=e8hOLtMr}8X=20JG^ySlUTgqorj!gpp?g=rS;>nWufHl}elMP>yR$m)o8bQBm$ zBnirLQ=2beiWG|b5TVqC#dQjx2lgfzJmDz6LQIs&SgO-er*N2x4#o*!p`g4e5fbRU zQ4A~l5)$B@u(MyQ$wXBVa<*@>1hv+P?>jnXVE%5IJnJ}mg z2?3$dq984y>Aij#0H2AywDGE#wq53xNY)ulK_G|_ZWP9iND4)Tff0^iqn-$>1jCI4 zmuM0lI?z;EW5e8d7GrD0iRooz(CP0R;Unh z3Je4o$#TkAM@UL4VY1noJRu?XlSEpAqsa_%0oT$s`?tTp`%dAo5MciMa1>VsN?1MlHtzM;7sUD({3WrK0d^=7( zL2?aTXV_>a)8Trw%Aw!p7@raD$n)PRb3*QoVAHE^x=yl`$R#-L^ahkErWL1c4Z8iX zp3QHznqKemdwc?ekdcv3uP_Hgpnd}%`)=6fIUI%Qu-;Oo(`eR$sa0>(BO35rxnv8| z3oO!OQ4s?~X+ff%>o$V>xj`d#pQOr_li~QwY6GQ^>q5Co-^;C1=iYz&m4_{uh1Y=F zkINJEP+avn`T3!K_zb;ySt)~co`vN={MT_beg2#c_vQKzC2wiqgdG+uHFm`c0v-rm z$RuTVBk+xOn>|;5yzace7KD$4H2MJ=oNLlULo%f*DS*YL{E~ALJZRU!kA}N|_fM-T5E}s}{5$1X@8t{hlgNo+ST{)uH#`R^B(%sspdvml3pC`MUuDbFoS0D{2tbrI3In30nS4ozCGS7xuafg+#`a(C=XX`hcQ3JL)71~#``-V~TVa^Kz&}*PW*iOcs2vL_lGUg? zYEd09roo|?jW#DX#%%+bkU!T$=tIpB&ILrG{`~&PZ+{LF4%IMK0(DDd;{ls1#);=g zm+&mCEO*@Mv_I_Fy28o51H9gPDByT1>kj8X2a7_!Yv1Vf_yBcCdE=UDG;SJ&REDkT z3=M~ZuvQ^iA3|i}OHu|A^!8MtUBhDB>mT)i@@8M@oA~0s%UhwEhqN-|alU*R&$kVD zY9hXn=Y@&+KNqMHVEkAq&(JuBlQs08Cu0y-R-OVYN7xwnx^!?e9$@ROgHFgvLsQpc zI`nlh5`>5awI)M)u z6QmiuWB%U`;B??tz8mKB$%S{LQnhIsLECZ>@i|WQ!66jSwMOKtD!`zys8G9=xr{s0o-n42iJ2KnN4e955 zH`8mex!HMt%K*N^tT2m}ZxBO7v6N;$qvA-keoD1)>D!X6rPRxK67?Pbe;e6_#^quE zc<>ux4=NXmL*KqXGc#J6Ux><`tRhm23Zj5%Q3EWu*zR&Lr%2QD+2dlBBaX^bLH@1# zepw#~TFHArP>j5%;p?8W;0vK{uM^}Ik5_eqVl#SuzOVX>{$H@#3*Pl#Z~5|a$!N8}W5U78v-IG6k; zSZ+>pbL1fWAczE`ft={2W67IH6GRo7Y)%WZ|D1;MvR^f|{z)PZ3zi_OHH1toP`GO_ z2KPg`61fb9d{NS`y^U`rBcEgq_keGT;il_jo6B+PRmhE`fW%-m<6^|=G!rv2;d5pz zCh5vL5XEb=L54}07>46}ewdS@{AWs#I(>m6`sDMAXriC9I^vBQOd!R0(Zi^GJ+q>L zFxyNIgt)rR%}tc<@xj5tXl(kX^YHMGbaux3!j{PFC%a!Y0lOUz>g=!VG#EGUzc}Wq zihcV&7maou3~ESH`Qq|lz6WE|`o}8GNIIJ=3_U7orlKQ495l++S1GWkzp7YuX z;4^2#tu6pkAovxx7sx;R!r`Tb>VL|5Hl~A;;C_d; zEmM*dchzSP)PYLj7EErqV ztjDrS^y8B>8S+nAF@MS0TLRx(g*D{tR=$&Z5hCNWmi})eoXEqwYY#0zUIGG<^+adj zA}L3Y@6cCN(;=Bz?jC;cYYJiwAY=uDq=7dPgx2ocoOX;Pr)mCKEM9_cvsHXl0`8%a zm9Q?HpAGTHjJ0}iw`iXVb{0BU!vB(JDpkMu4}}!E%}^a6!vm#A`mj@@$GLW^f?Ef?F9b46wFhtUm4;+Y%D_;d%-~R z=#;^Nq8#n+?ydmROn(Tpkk{4l)+kh~*KecdH56=0cZG6&X@*IiF3&puC1l1SAZtx~ z;@kKqOHjXIwmm8l?tda8G@a64X zIt+bA_FO@hZV{qLVdC$tvtsYLJ&7kvQs@zhMB=n@B9Heg#tA02+U~)LB}`s_A1+VO zRrpv;y@kmzh?V)3)rc`y$xK-#eTs(m3Ug!`?3}x{rfTXOHd<)~ccJvzAB@n+MRUm< z^{#=O(_Asv<>fRHF#8d4cx2Tpi^mSng)G!YTxk@w@NB6DIn}%KCD7p}iL~557NKV8 zr|9h{u@m6%<4AWra*Ecw{UN&hF4H_KBw~Ri_l54n5eGooLLlHuv+#&4m%(ZnCriDy zh4mB7#C>%19VihMpI5SY9f(z0|j&0DJF&NM6r1?cQDn=r&ebmfIf(J zKON^liS8}C$et33v{J7n1qZEE;UtluQAyPt#i2JbY1jT4BM81mnLp;JFFNuIFjB?r)eUbaYhEt!VfOR*2 z*)B0qvq$rU69Pd|WFH6m7crha6Y1moj};?xrJ-V(SfB_W-UGs+DKx@aUVC4`Z?6*V z5tkBA{iea(+L9vkv2UAf=tSI6`3MqpFF^9Q=_c+iyP7M4Rn7TO*;36~X$5qW1LW{BWLj$>!EgBJl%O+^QZ`Q9;<+Ra%v@5ph&|B}FYvR`c| zQY#SM^CC8q(&;-L#Zu(hrI>;fUYAS%#>Qj$6K5H_Jp7p>)u8_+m;71?I>q%e?Ij7cOy1KEc2 zL(rl64_hsD(!I1FyJ9w(v86EY6eneWH{E9EFHs?J4jZtKb{LpdbpkMpvAtmyD;zgk z<9{$62TM{U(suTQe?+vI%nJJQc(tW8{WwT6X|#v6?nZLMrc+RK4OBp4a1MWW!_XU? z1|A2C)isAMTunjo2&(02Kl{<)CP}3H=*^ zd?yW_CEl`-JcYJsvrm*FXMIQ9>YPM!mvZdgw3bXx|7&y-pENO&n7O^z1rhKDq)^B9 zy|4KqepDav^nE7bmqR-}&L!K5HJLE?B~b7qN+L>w3aE>rYqo6ZcJ6zsu;Eqov3fE%qFH*VmbU2<)R24UVvDQ^sf4!sOm@A&VFd#K67P1aGiUvco zAm1BOiw_~9DGH2Q9k=wn9qC75MlNUtOf1XvRTwNYIn3eCC%}_zed7Uw3C86{i*Ci@ z$8I9m{m>UjKi2HCJV84?pgh3%1&V`pv4CE}1SnDy6^8nR(st~Cc5)LoI9kEmww_vl z)ltTwMbKj1rLYH52;J2Yh;RGsURMUZ!6r>OS({x@w1TWf9baDz9JPBq_nzwjw~=`X zBZv4FCFpTx@)@ZaKWrcv#Z`At7|HP4kZD5Xrh%M<`eo(k}AOV|sD~w-heLSH~K}N?+w)$4D>f@x%`-}nC)zZ z5D;3dd)kmFQH0dvk^abnTon3ECIe9J0x-BR5$Vg+m>FaEMU=SI9Z|5Xd;Y2(f38i} zKj0f{k=lNFu(A!oj!;In)YT7ylqXx=ilk6{h@hv@WP9he-)wi;tjgwd%BOrJ8J&)^ z{<{0~C$3P7BWI~o2BA6J_k1DG^V=_84(;>p<`Wu2)dzkn`RhU86nMVe8KWSx-|OaI zm*hNIck1;EjYZsE2NA(^Mq<9|30I{0OLe#&znd*5({cQ}fabh`%ZxEK0vv9MfZPMi zD)&u;e0^s4Ff|Y~KqBFHe?}_?dTL9&w`pw0ju!565Pbh-Gb8;Wzh8<)O0N(oCmSJ& zD48&JD`g|qe+Hh+Q7WVxU$|n#F&^d0eM$8wOp}-a1A&LvVVxWHa!SJQPdeN$!43&z zv}EX!FBhWu(U7KS!^ee#fk$bdlcx|U>%y;Lgl-2TkS#km=4KjqX&*jJOyVdJf;1Gk zxKlk{a^?0Tnoz#?t-?3YAvRGl>UnuLv>#fE_{TTWIA|qDDwgf1?Hh0ujx7=6sldd$ zh&I>n*2x9={Iy9IyUK;Pk96uW&+{Fe>*cA@+H?Ann3(Z*qfzlolwD~Z5rQk3)?GmC ziUZFzHM?HM;=M=?pOaNR3R2_YvG72B-5e-JEOzJZgE^mR2!lM~I`!N0dA+}R&Eabj zg;lK1;YzpG-(Kj*g#60=Qc*|Q1UtADr&XGLq&PvVec@prX;Yxi zv45fHpgiyRL&Zm+qvWbg08^AF83*&NMck7w>*V(N{(P-SHlB#qF6-9NYip!_rueO4 zM0;6S&XM~|?NE~WZ6whTI(-zDI%m{6Wn|g|VN2^?Gp)DEoQyo8t|`nIwiaI9*cr@r z3(&t-pKCAw@%K(a5PYZ6mf2dZhWvWBi`gSOKACFrbuf>izh|H2`!kDD1q$&TYBcp* z2Nu{J6IS~uA#e`sw_d1EKL2053dYRqc|96Wb>Gvn3>p_345QhDo5>9r&_GzdK|bRL z^^wH#`K!sv##IuHOGSUO9Hydr?c$I-!Pp`yoFt1n%5Cl{UW>9e@m&4{=HN#(W3p)G z@Y)I~oOs|tbW6?9w)T|e-=B&O*?iKPA9*gaiRq2{%w{sBgj$ih<;5Kzl46Jrq%5WsdQ&NRB3{ug_38d~w;1W!#de0AQf;7Z(Lvkb0VREv0YF8@kHi*Cf?y>pV-~k?UG8TaTbpG1*1?@B=+p|8r^m1P4 za|ne}cLn=XpBe2+5ei%~^VnCg@5IWg^cSsYH#F3-zJGdtV&cjOY`<^d z8@K13e7RkP`f-e8r(r6o4k}N5YKu8=ZBIn`ES2a{jE9iBrUoa@5-A3tt1t4cXx5U zA{X72t!l>~uI&d4@L9O!x%)f|Jnm*Zo{~I;WMWqF7Ca@~%Bm z&ARzNIfaZFl3dq%b%?-56kGBPJpq(CJ$1TjvDQi*9c&v)=wOYW1u}t3G?N)K7kNJ(LMAR*l?9 z>sz`2AF^hz$X*KCT=?G!MvTkn?6Ze#_P>t2e{ubSj>sSOp@O136L34e-qAt0YZOw* z1asT1xNQyawHH`=t>f8?2>87wRn?%W8U@Y{mwBn_b$b1bx%g7^JEwV|<(hZ$XP2>A zpZ^H%Q4}~zK&O{_%yx>GLO0UTq;=k6y0Hr!i*|84X={E-+5MDXoUS0FQZ|$by7i{M zQbacVX3&fX;x-VqYhBAe|7MI7tag<7vXT}5W{LJW{LRuRL8$&@%CYef8u#7a^7UcE zX=Gal{rR$i3c5eIgu#2FnW2=Vekz0InvMjws5FjM48(3#SPiNd_gZ%A04`D&w+ z;8+sFgk-hJW~cm#p#Zi~w~S0EC>e_?zISV#$H}0eU==Y^XC_RK@nH>Uj*aJf;&piH zm-Sp_QadEoR*PkjTX!w>;NACy2`79w-A{LZ*x~zO9;)d1UMkyKpy+)$B8I9W|DhnO zRuu-1-c)~jx{g5aWcVA)Uyt@`k0GKTHCk9Y z+PXhnBg*fqnec)CvfqtSqYx)N`5b@YHjLUECpdS+B zGNloT1uN>lChc^YMi=0fqmGG6v>g{#0(=F%}vF@O=u}EwG z0DIL7>e6Jeau#PqTJul7c!agK1|ms;Tj8;9q~>inq<$Xx+p(f zrl2V~6HkD+@yGzA0V8rVo5wB`8Id`W02t1?knQEt?}^A43DJl9F=*_Qc4ov)`5+m# zniU*5{pjf17Z0XVGG}^d;#VF){evq5Vh)sz7m0hs;g1_>&y)=lVSB&zPFs>l>!^HP z>kat3x(7_1Q5*TVW@{Bj$4|uGpAu*;gLU#NjK^wPu)8GOHHx~Ki(~OVYZU|pbd8t` z(vJGf>E;Gtu>O)1P`h!|>>Y%%{w6RUjtdu3w#cqzlD2cgtHkxclPrPfdo%z_WtxL8 z{yz4e3n09p34c~Ru7$Nt&OGvEX8TTlw`1jrec_3B;D`)@Uhg-ff%$r_(M$?{>Dn*+ zW)qx7{h0YQq8V&v4J>hUr0FvViCcNfv>TtTx&8JhzYhhIrXP~02~hsjj=hU_Ull5k zT{aOzQ_9p#PQQzFAK)B!#b}DUNWcE6BZ;j#`S@x^)y%T8cRTFelvN2E!(#-#y`Bko zN}Joq$U(V-nVOy+*@%5X7v@&iK?&Oey7WP1yq|%~hpGH({xc<RA93 zF#!F9B@8jEamJ$>|S5vCu=%VSvy3Mq}0+U3eb3S% z0##t|L*ZDZ)wbx!*QQwq%hdjhxN$_J)nlI645k~nWZ|#l5&p0ObX*D4uw%^wtf}C_ z$}ojGtdj|B3~vSpA5Vv3S=rjp==Pi8a}Xi9^rHzNjvwbmSw4|g!9)JeJDN5u6#c;{ z2fvC=3rMn_mgWUq5)#=#r>l+BkBl@P5ig<#!rKfkHg!Sl`mk23e^5=aD+Op~QvOsE zxfNMN=o8fmRvT#7waFnejUEY0;(xkP3ev+fz|)atDYi#1;K7f^`m{UY8==-w46)~; zElYKuOK+Je;AXYXpD>$Ow+}0BIj1b?HmrI*#Whh_&)wiu<$z>)(ou+o*o{D(LcqZFbjcyQs> zwqXaZXFU!5mS~!eHF4h(KSX~xi{}9O=5>>eBE>IZqJ;fL>d6<56wt3xKildp?K4qq z5jTURgebMM;UzDzrmA4J{Z)$SLROS3SHx*C7BOJ*lLdu%R8kRosHeI|_0uMq?puJE zQ)CQYc+AOJR1_pyyg#t{4AVbZ;yw#BW7v1ttzhX z`S!LXU&_P+kgpOc-|#Uk2{<*#N;p5&N+V)Q|E9cxD|!oSJN;fGDLF}3 zb6-ZvLL*kg+%XGI#BTXiji@^E_%WaEx4f)OcDJogj}E)wFJz5Gr~CE=MBC&SV)f#DB7(a6yE1u*taNus7>*+D`LDPP3E?UzPO&hWuYs zasOIx`RnxD%oFp8d6ryd`W{eotq{#uTHkHfh0!se3!3;JXd6y`Mrc*s?D6wr1PSl= zRQqgX{2`C;_j%F9j~Ka(?UG_aAbzL9c(LszF|rK)nBw4dq~yoU9G0N@}b^P@yokkjIX59>@XB%7eVk%b}-;BCnV5Sf-f zw6fK<_>SXZGD?TsNDxfTA`#dEVMM=<86lZ=PECh%W~e4Hwy?z%Zwo1}?oUfnyJq>H z3uZdJ9Ci>QoliJXymdRCy!NKc80p8$7bCw*9-z>)QFaW8GDiCb3XY?E*57^ToA0)B z?3$!h@UXGg!I96tik(D=BL|1p@DN>xK1L!) zVIy=hPR(}ZJIm96&jbf0_t&g*E5r@ ztzgH1(y-i+LH`!F2sMGUN(M~DFt4;GF3QeU>d{x)KXxMmEfarS2$dfsvOCPo5ZAYa zRsRLD#QbYyJt|T;?%lXjyK&HDdTZuA}=huxPCKvkWsV8V&|Bm7YtlfG>R3o&Q-02kk}$ zfsw<74S>AW^{YLdA2)L+O>sjP8w}vm^}W9db-`(7cIRQ}97HX;k+l%wdM5$MDcj#g zeWxDUQ1CEDke#v*nM=JSz_e+H-t-4g58Kil2oGA79tx-phr9jHcpC0190nqeaD>?O zwiSGAjIdj!%~)j_{l-cv$o2QR_B3e+q5W4ynZ!J9i>MhR^o(hkbVTWK;6wOLvguG36;1=>RI~mE2#MMR(PxBC zBkn}ss}7J!6T0+;NUc<6=lcj#9QRbr7_K5C$=&AC{_bj zRR$-)MCE6czuuI)enm)|f(=trp5J3vGDTL4w(_iTEiGVH ztUOV#8JR|M;gb=Mmm2`M7t>29nF>nu-LsFnAqW=pR`xo>b6DL*63Ek~tg{Dg=o*Ck zuKA_-pCJFM`o>*#e#6s9(Fn2D8z`Ey>#|6i{>bKTB+iT(1AtjW0Oh*MoXlXOpP~Mp zb9JBo0pxUFv~c=s#wXYWON&wEk=`ciJA%&NZO;vd(>-AstCA3%xJ%dYOpl(Z)p_B6 zdlg97sK%~wm;`)*Q@mpaK!trDF4iK+0|{i863@1S-d`(wBy5dbWjI~(qjF%E52?O4 z0{000K;J$u{l~Dtp%v6olCTT_5K8-+=t0|aI>`bISkVB1Z=dxy0I}kTlYqI+1P75g zIQ1IMU>`TsU5o9kfp8@bG&R!qxQJd*DCCG!b~!p^)9-SDv@v=J#OauKV-wgzWG-Env zveuyV>H`)FoQuH07D2;Y_%j2Mgu_2Nc`ZE2w2sLivvAzDws;?W0E#(ICsL$>>K_;C)TI{@uIJvfYex^Q`=^)J2em1WI5)zIz9-1Zs!%}0g z)dkXktSk_5uE{@DDSPXr;>dhKToxBbR;znN={LxtYS41@XxAUTq!5m8Y8|NmMShv;1F=s=i&GF3moH&PMm?PaZR9k5KSDmo=D$;J+Tf4XhwU8823eNrXHueWy6?HwxR(J@HMOxq|&2kYK8jqVi-FQAd|_p@Ft zeKw9)@o^88v4hGRm`};h3jomFvV8FMv~|Rv4@kLza(V-0L;ovneAj+B^zGG z=v?~59>(qE-54UR9u$~%{JMijpMsOzjvA}x>YsuY#;CGUC~S8QclKr&M%&Vs}f~-;A_bi zpzS@rOHuaLK$^jx$fftq^7#1AV94^ZWKFTNnPzNQ5-})r&{`J&IGtw89sqmO5z}7? zfBN9)tJ2|F_^q~&;-i#u#}+<=;EcZ=CMXHrVQw=~*`&^BX$)4<79M>E05`BSZ@Xx2GERSKJdILgs&L~=wFem*W0JfzrP=R6mE-Hz-j*#1pyHrye{mT;;Qby8j zl9WW5P0ffLF)d5csdv}M7Rby>h)b<& zEdzX7Wy?fybg@h6(2i*-c5*>C1h#wyW3};4q19X)Yn1zX(^mL?B3OsQ@3*Np*B1$_ zlnB}C$ijlnE#L4;B)yPZM^T8l8L~|0>-72FT{^j+H@Wp4XPV6&3&P`7=5giP&3`ly zZdf0EwVXQPJkbWosU6^A88rh^YD&~xdevt^vBU#J6&^Vu`=;2c;4-mzWY&$)63)N> zSj@-jupw{NGOr6M24?7=>>Lq)l(&zvpp;1O{sT44dpI(0-K6tKb$*jf$%)Cm+Cz&NBF1L8rT$d6Yc@( z3TFqZWb&6IVKVK&+_E3f$-QuN_Cu4&rFfcv$zRx!O~UZ{yHP7LrNDp?5ww=93dXB) zD>gqMC2W*<>pXWq!AA7b^qJVUh;u-IKD0(21Zxuz^V+w2-xL>0?1J-vQ3OEvni30_ z0#=Xx(99H&gVC%<0*YQ{Xy{L~o0F!XL`qvebKZzQyEb8ZTu4P4t@B7M;F6+ufe!hVA+8{I_+NDFm}*PsD~pb84Pa$}cDXUF^2#$?LM0;FM6 zP{4jPeKH~5ilG;&kY+p+F8!TThu<5I^}TsvQeybwzj5o8A@-HAfX&`GS{>2yt{8GS zyR_jMFagW-nTf)rrr8w^`Ag#W<*?0#CgTj{NLV+U`h}*>*Y@2-Agw)+Ay}C>!=8y;s3$ z8sWn{>$mJGq|5g-3uuYPTvV86Xdf#ScWQ_!mJvFeTzt`;i^3-1vX>2m%oZ9J^ks#A zI4m}A$mu%~-w)SO17k=r&&fG>ZKF18jrY1{Ul#He%b$hID;^Gu!Z0sP2SL2G4A5awH8YuiiJ#Se`yhYRDlYnFf}9>9jy&fY|etf5Fi8U))M{|oDfI|E;6>^ z`||YL;?`+0iCS*;M%^a~GZ(%ldFM#Ogh;7dlMZnPx`x0IhfhyRVpE`P{AUDy309@V zPZJjPlQ!^^Bm+|R7M-yx{4{YN145oiIF1OmMQEObA`Vm7+OfM&_rO$X{aj9gb=wV= zT&R6E7d%6NSuUi*WZ?Vj5=9aPEB#D$rBqy;3DE-3!hazipsy^?4~SF=!F`FVY~+LjyU;qh|cNEjUk_*}@I z_|>Lt6g`IzB`)sy8n+}j%XX5=VKWi{8e#K7(W8_Za(h|?DmPYCl((n;&*7I19T=pP zxK(afy=#EzRMLs}bzPoLY89#x;*d$+ObP*>xB{RO4St2hfl__VeE55J3jNNf$z_(q z4G?K8Tz8Ga;K&371m+CN4BfgcC-FictEBnWly8{Ka?k=NdEPM;J>^5k8(S#DLn&kZ zhy>96O?Zm+)?PjsF;Q2WyCdAznr4$lFm0UOJyFx9Gw4xlYnt#>Q%#1Z5;?R||0Fsxgi71-w`y|Q5v}?PW9AZ7ay{vX0a+X`e>TATs#r(+pfk~Y0)@2HoqSLFG~^Ai z3A%qoW^H{jEUwrQf7QAXZ2%~u7NEg1hBfosr+;L9o2K)XdN%B(D$Oda7Rc2vMSc35 z88P{#7nzm)Umqxb2I|Q_We;LxMpB15`Nl3|YfexHHWx<~eZu-tEvwG}eW-0c*=VEn zGb*-JV1@RfVS>3_GW~6lt({VQG+BVK#eR)5i9otH-ms5REg8obQt6w_MH2zv@`NL` zE&BY*Lq2)Z$rfne?&T_P@IJBeHP)Hv|V z`3G@)FidZ;O_eg|XK!7TlwnXGm?8V@F16XS1D(ZfF-zIV!Y)AuU|f zEtF3xxrb@JZ@}Ys*@rPwYv~CcVd~YBuQN2D7=~DbOBZQj|7H6_wgeOHcA|x67n%nC zM%D8>A(xf(qQ>x)VF@)Wx6H$;i+FD}WatJk_scW_p}3ZP&!x1sW+s+DX#fw?(uO5?3R_35>FLxcfS1f}QL6EbY`O~gGG(+} z10t5LybPU6b~pw-*h6}sbdPvSIFtRRhdpP&?glVkHRd*DjBh{eR+=z9UjDc!2qcIz zpyK%rmS9ej4^PcimLM_?x=oQFWBQC&1I%tgmwo9t?w4qd$a~Do@x}=QfGv*YSZOA? ziJ{v?V5zCgE&Y=jm|FRiBS=F0Da$ZQK9%-Y+5yNr{nzo1fkd7(iQk5er(da-u*Q6X z+jatTzI^9jr}j}b0n{Ncr6&M`E%zrI!YC9aVW}m#|23o;{4P7j{T|U=<_SRQ`Sa`? z89CPf**JjKL9H_?n3QE>2SUehZc%CdllbT;pLCMNE8RH%5JS9CCq7lkXIvwRO@w;g z=5l_H%-J$zrW>Tvu1fJLC?bdp(U>ioBjx_M%1Um3O(nrb0 z$eE);s=Y>4zs{N+zDakpHCRcIdoQT)-JdPR9#tj7D2f*$2k%{*Cy0zzh!6YxWS|f> zKXQZ^WWUWr<}eA|;?itzfB%H&)IYJ9h+Xk@=gaa}FBE2p;Lxl-rLW^GK2L0seQ(cp zZ!b?5;|O^FWKfp=1_NPfEZ_S9p-UP_1qtRo|p&aH?WqGQEO6=0u3R z0Myw_n?7Bkrul5X$l{!&-t~RI{13|u4>N1;yAus|?2br6PuSf{gBopsmXA}|XD?oB z)T8&{(6+MC8{lWln92HUNHNZ%oWzUfMmA)_bg}Ua*}=$E;r08R;q@1EW>hMHj%a9Z zHU{$}Y_h}6FFKIwL$`dg2p0IZ1Njt zVA_%E_WIvv|ERZJto1ps(f@J2*ecalBlKv$Q{|Ua)75IfP^17}_Bb=tdQy~#*|eI@ z?g3gEZmd!$<^!EAG3gmbI}92m`7fMZvEG)IX?9ChE0ter@ah2W+D@6FoJ=HZ&$acr zXWk84e0FQCZjEC^OjZzy*f4hHZ$$KAhwj?9+cyhnfRkxXidVjh@&`quynwaU1+n=myFEZWQrG%W;Lcaou)zYuIWdJ$8 ziqDI6q+!k8UlGwMOu0+4m$+i*q+W4Jb?Z4!b2_WfZ1U>{dHX>KeSC$H(2ZGsf0F0R zlhR{>x5r9A`n}LqPM7<_wF}p}*A-+h&QCuZ=Tr%bkQq1#VZ2YC+q6pQ4Pa656xSWa zN2L16nIrUo-6o%y6&)sO2l(ifSdw>3XM4Mhz|(h?8{}c8mZv*`L%0Y4mk6kHf1K{ z)=Or4#T>TJ%%&n`Wp$yUwrlgKDm|Ih;_c}^4T|?l&QhUXnyW*>>{eJi08zG-O!|sWu8^O4P!QcTU%Hyv`Xm^pAG6=^0H?Uhe zfGb&=Jv_}}e2ov%L@z($ob#lUv_33n`D@5$dz))H>DXZ&ySLYskn3|&)u!5ex6>}9 zIyuQ|5q>4~b~_W6P`?p~G5?5pVC`|S43GJG-JDoa)6!r3HD8jj1_O;80+&fM+}&Y( zT){xr<*!JBC3yg5DJ+Vvyx{y@+(EwFjKhYXd)n0%A@VW`5ijJ^glk9-E1kF;!3P+F=0bpPHbCWeaWi;dcMT^ zd*51412MfgMZQ7!2GbP0N^GfD0$HNB z;hlQh&2G;vIH<|dC~&GqKVx#57xLe5J}WzkG7h=$KLl{^7b$?ct#C|29@5BrFM`8- zG!-5FX_(7z0FCUG;RG%Qis^AqfKnQd0p_0)h4P2E012hr?s)Get!$E^_#>4-p*hU6 zYy%%uzSZfzFzyGfDji*-m0f~L?Pf9;GTnhkPFUCoTK967{A!*vi|D;k#HZG@z9u~u z;Hh#a5064VgOCn|4Ulc6l7brR5h9EdCPVfyJQ>0a!+&s48``wWtA!_WiuZW#_CwyC z9P*_{pPUB`q4Hg9wEYHX)kQh!>?ZOM=pqOkW?m2C^-F;4u88XZm?$dp)rAk5x8W&R zvOGw3;wv27jFE$K%7J#CmL<2srRIChV-Kf?rRO^bvc6YGt8AX^uRBncW|=AYn<0E0 zIfvas@rJ6JZesy|!e;yL-oJmoiR!F##(+^s`rJ>in)#+y9!WUszHD55)~;imkcUpQ z-R}MTyQq0|$9>Fo2+Yq9SO|iSY9q(Zvc6OFzVO;N_9Mv<>VCf#)&VNMlea9`bj{08 z2rzr%3W&K$b1C=|Y%QZwyCA>kw&&5J$I7>!s@aCMt@<1k zsD+N5)o%f^Mq73|k=xV&Tdz~;-6bZZI$x^Xn9FtBqZ*gm>Qs|x=;B^3{Yn(uRe4{y z3bHSUx54FLG;Uv*vczm8AAgaUJg>!mhvB|XimvXA&sT!&2xX){8aS_^le1Mn7+Jls z;v`lG3-*8#V08E}m2t+B-%Pf8uD*Hgr z6yV=dxfO9{fcAIOP0qA!7kvakgYg7A0yC1{ooM>MMWa4WX=MB-2T!DA&zbfY+xd!0 zT)Z9$XwFP2g2ppcT5XpU8&-k8i!4Id2CJ9O++E0QB6R}50BP&A=zl8DP(Y)#RI8iv zyBex&^q+nb5J(ZI<_k|eM>sp2C%%W&@BpIYQy4RuO*dmemPM00cFrBW)Cw6-^n^^s z5w`F@jUOe+^~1?h&F4l(fp{WHSm}RvnbRngibQ+Cj0jg_@*92!(5Mr1Rp1RI_GV8G zNA`IUly23$P$l9l8G(mFSa>f&8(ii8oltm`IQkCm2kr3HkAMQ|@wB4gD=9J{Kho(J zvIT%8N^0BjL`s`83cihy0H~OWClD>5WySnq`=s)d&taV=9qjydv6rgvY|vjVf2s5{ zjPeEvmESz&_x^y7CG-DJl}HJ<%k__$l^6hkhK8n*|3pK8gga^k(8cEfYNwg9y>zr} zmdg&&=aUj(WLLx`(O$(3EF#Wky@l~{6O#{X(x+I0zMP&Be%ayqILE2{KkXZgq`e7! z%y-`xUz8y0S=%KFJYQg<_WlGau=oZDPIiDv7Z4xdO(y@rYCB9}(0Z#ZM&0~;e0&Ni z?(U4@d{Nsyeq*cRfZ%yN>T0Wtt3vW7S8hPA(X{555K{QVn)m>T+nzL=H5R<=w}qATOzqm^ENQ`2b7x60JKN)sD6ztDaTo}RJ~-|SPtH51WG>ALMs$4xV-Z>uz4IqrU@#Uu=i*j;YkW|ejH|m z1cEOY(BM{5HP+N>R(<@xD$YF~%KVGtgK^0=*mS`dx3z{`W@{~@9ZYU*ty@XP#&p4u zTw9lNnIgJaQkLJe8J7*e5n)gf2BFwpCfCw1l}j$mXc%KE8oxt&?Qj2@=Z|^4p67d> z@AsVZoO9lv)O69z^S6Is^Tt!UfxS8;k$Q0MAq{to%zWeLSP>YeS<{s#+e=EqtL9@B z(Q`X+O8 z6oB<`4M7DEYs6Yj9rOcW6rsufm+hNU^$V4<*5Pz9baU*6Cjjm#l+^e1k93vJRVF7P4x?*8o1@gWB}*_6iIv!hobHR{YZSNzYjZsy+cCu#bb*H75phXOAk zWP?3>`4m&NM?cGdF?NNzW>jQ9{!F??<3Mpd4vSry-JdCV*YMt? z^xzJB#uoCx4ihip3}W%Si#R&}jQMz6Ictdn9iG+~Ik^h!u=O3;pH(DxdV7}nM1tD^RL#_S&Q#u8FZaC5_6C9uJiY5aQg3!P2pd_y0EMMSfhIH|JK8Rx^DG zlg-J=HJ%})uvpQIo<{{&3(uZSjC)yO3@X);5zp^WbA0K zb<`}W?!BLU4L9m&+YQ(0KTEV)d9nJxJD^Re(;XJskCOje-kHl1cX)9t-ygk(8;lC< zS;F!s8;slMan&JKlF8$>_Vo^(y(smX!sSRej)9DM0H@KWNT0=kWjYZSyKRFuH;0XZ zK*u$4ee(7+orS4``u%FGugw)R9)R$`<08`)v!}Zdc~UXNb&j`CMTs5_gpQ&61zjZP zIykOh=HfhCEk(m_&^OFXGev%I(o+u4-7SBwg#~EKivXezrQ9N<<#<$>_POkdGL8s? zk6W}8`U)2yi*?Lvis_{nwYGOng?GLFfXcR0zjgg(&xUD*#AIWgbWr0c^{#*5h;~I1 zwYTqOu0n;N5KX4c!>>9>weVj*$}k`BEB1Q00F`V7O{-4TD&9658`gWl^^$q=H~?MgxtPl`EAgzOu57;-ob%_2s8^8 z1cK?MEh=l3N7^y!IZ#5GLSGFd%--)_!6w|eq;NRkukQiwgpaEWPHXGc#&|-0dx~y- zzVT4efuLF=uiEZO0XU6M;ag#?WUU3rM#j_oVDFtb@G4w?l)LOX97$Fo}XNUD+K00(Yc~ut|_c8tM-x9jg zj{JVI2;xz})+(FFe`b{)t{^X$#H0HPbT&CQ5L)h{u9BYd-v;aW(! zRgb9f@t4&V$R=P&;7B2OC84#Jokj`k?-)!Dk^!kQgu5OUWN#3SY4dsxkG>`h084?Z z0|@H75!u3U06ig(+E^YqN!MEv=i>`{@cwK%6mgG~9x+ZDF%242vW)O3`)L5`&HuW= z7O2;~8v_v>+hqoL#(6kQlvQ@zP~h{>J&|t`vZ9-G={zHDJZ0rJxBu77e{OUA;Z1z_ z$sgp49da&pafDOdr_X{H5@B{lH{Z0EYByHJJNKRMY_V#k7|N7zA~kz!-Dx`qPG62w zvXYOm-n6kspfi6+;VVuz4~91T-3^d;q=Q;`2RZNA?3s z9Ar<9?q_3w7LJK4&!>{K#@NAr6n|U>!e3lcS}vIf0y?AHm>W3P`{N=hsl8~n+VZ{UF9;yR;3ceLrFA@-RBwak7QJhk&{zr z?X}mMbFR57LPt1F_&S{C=P=h}J0}NnZU?qoeQocNClLMf}l)}p_ z#BS175F`4KNAE#nc@y{1s1T&!;NTF$N$&-!A=_iSzvkDCA*MU_-}8dr^9Fv+qyN?p ze-moM29UrClN1?QSX%yp!7Tg+8VMEdmI)g_>kQIgtz>^rUeN%IAdG{QmNN(l z8uz~+(6~NzP!JFy5NR=CRS(c}Jvd#Ag~cz8-_Ic8{qU@+4QzLs9?^q^veNr^ z_>LS9Td=x~9w*L^{HG==Iv>5VC{cvtg?=ZHT+b;A8r>aEE;rfY-QM2T_<#9jaXS|+ zQi+O+I=&pF=DS~QhR4PZ{TBX-jESjEo6ALlj0_D80Dysk1_lTT5flH<+rYbEWbt=s zTsSasad9a{#o+Rl_ug^DTr+q0l%h`cSFENt;y#9aworMoH zG>Ob!d-2hFI3fX0@JhY;I2gLlD#az(Y^{kT-{d;V70=Ksf*=$$Q{oik( zWMpN3u;y;GyOZ+rwomaauba>0iMPBQX4*E9D8T+N&kGvZ(;66noR@|7)H)M$+Y<{z@wn!3)%oObyFaSkN}-rT z2WJ}`p-68r0Q0{^!$E+93|{dq6$*17j+ zxiN^}7?gB6^SPBNU9RQx^~~|2ap@Zq(_j5sNJn{SW!wQ<)NE70@;n6x2j*(YI{q(@ zgRx}&1^AT~$Ncrn1X9U)yqQ0uP%H08v$Bk8(-Fb{Z_0xP3iS&IU>z)08^l#q(AjKu z;COm^hPviJQ1Mxs>OV~JUWvsK^23+wsFERHQ-I-@>;bo1axQ`&!{gPKy{0q0c7xE@ z=f{CX$+_1b*;Izm1-)Dh7!n!ZJo;59yufL}FI|qg85uq_RMhJ$|1TZ&QkQmPN;pCR zA8gT3cu?LyYT<=t=aL~BL;#roOi?y?z;ZF6HD|SUV_C{cQ?tZ%bN7e*6DBNQt~BYM1WyG%a;~iMHWF9HAGMLKm)}z)OB4@tD`TQ zor=a`z3zi!qzZeeGtU%^{o7w=1B*Z;fS;b8KK5rB?ms`lGd0+`#+AX>Yib9yS#{L; z5(x{-cSNNgP&iHS)*s_3PDzrn_ajkW=c3ge9lDRHx8sc6hF6=N%k}0+k&%(-YV>ar z;Zafj_a}2dRG`|^2gL&eQc(<3A&7jg1K)Muf$^wc!?o6^g#bNt+nfuL=oA|?hg8H(;@o#vY}l*hG+kVX6@BcRZTkQTg*nG`3!wdUv98O)4aBbde$-}`IkWS zaE!o52>bU4NbbV-e-r>I40<%O@;L_%Px8%B;2f$%646 zG9*&}xz&0Ek(Zn8iT+g~C_v5SjOS4{C1bSbTGDoLu3sOpe47&aRp{BuceH&Cpec^v zDH21#06JoX?Ccl_{hp07akcP0BOxf>b!!-j7Ljitxmik0Vyew^eKA#vD%;5Lt==yxMwoTn<%S2-K_Z+8ho^PX2 zfCfYhZ3FYhX@5q;z!$^+^QE{-1~X+~cz7>ClCYONRX4Bwa1z|@@_i?aXly!!-uYXs zTchbHszU><|2Gn`o8oqo*F4dH1qQ7_ecE__T(ftTs<{r<*dp?OcMJ{-JF8lF>P?~7xn6gk z-s%4OsIIa+q-^y+ZyqGl*mBLrfI@%;%_)v1ey!Pe zpSD8V zEcjs^xoYa_SPqoVA^sCwoWg)E8XiiMNJ1!wQ|e6QRvL6~UID1qB{`suM#vXIVY%jQ zas_;^H{7?VMNzy11ID3igp5z;OFgy&p?+ihu5nLzthk{Cg++=#vP9y)kAK&_ z25U@t<~~%~d7pUB0A}ivyHJBY8fXd&%>Z-%a135Cep`3-=1+2y)Ug9|zIs>*F~KFj zdm38WTCWGIr#DKO|3U>V!60wAFGqr3rL3Swuv~4VZ<^TbR!OoC3+`r*FT)9hg0IJ< z4VH5fwx7o{`Ts!GPz>IJ^EVP?2TV}J{>ho#=jOsfV953S7(x*V%J?tbPk@_w4EK|d za-WzG_T%-ex=!@izUxJT`yjCsIqwvSGNY^bQCN(N?2q#*ilaLrz87y|OVm(M?(2?W z_(AFH*%-z-&rNG~Xi+Y(#Ku?XVEDpG{+u?bMaXX$p1^u(4Re-s_SqH}6O+td=k<7A z>Xy(S?hN?zuW#{dY{8I-mD&%8dn2N3BM-eRt5Q)=NJx9tF!tT1JG&g99r{JfY~U;* zfq{eHJavER(U15xKe|=wqyKjq5Jfgrfq3-KTL3egyW-sPQHh=CPaF!BI7f=dLe=`6 zhb)PU%j-tBSgBK-h3XwBlnpGZkyyVQ!5|UyCyRLF_#e}37*x!S1Unsoh;|RyXObGn zlNjp0CQwKvN`wWH?j9PkM3P=bv>W!=z-*w(kp1DI;IV(>w{v#oX9~@zip5PU}TE#AIA7tl}K& za2P_UH^}+epYp5{vG_2ER=_@Nhy?mb*{5uWZnaQ`0R%#?AJ{oxI7yQ>|;*&J|e1GvcDNLP@%wV(prij0?C2B$}o*zm%0DCjWtBCz{ zwN?Eujsb|E947w?!@M(J*lL4iA_?5ARiS_!6)wB`rf2ag9HLmCvbW#Yhx>!BFBuR8 z3MgUPwgmS8q3QfPFfbZ(mlQNlrr$^sWiy%hQy>UITdY^XTHDsSooiS6h4|@?xxakI z^9wGz*TbLx3&sl4{YEJrq@4bh9AV zV*MxYxrPTltX)p1Sac_pB)WNm=DUo|po}h8ubj0+_4)dEO5?E2XjhALia`d&{I>4@ z^Qe|r9A>F$(0s1N~U`oBb&fxHD;8*9u^!tlEKlT zVX;`D)dpQX=4pQ(f;@;<63N3aB80awq%wWFmNFT3uE;?_;O>Gn`#|gS7n^L%oQ3HM zJcvwYj z0nc00EI!Y=8~E?h1M~pR-$e*xjvKB^Mvvz!b_)xCVZ|jRc7aQ&)INF0heoMTf#u9e zBBpGF1>-L)RKDPNI#2!7uGU5{O=+JnUQ99gyUBZ1N=_e#(O??89RO9fVJ}nen|IOu z#vb|WzhV^)a;Fu?cN-2X0c`R&o8DJ{Jns%qTZeM6*({Kk9ZN(cWAS@|sH+k`@`F1( z;KBa4>)E2A5h*W)fJV6di>5;!@>`Ldyu2D>XOo~i@*{3=o^d{kVI$pO|81?w@OqWs zkvF3#<{vReA>A#d%m=a9oY3c5*;@3wTARJSG=8>qGXZZ(ty3*+?J~2OvgA@q@qd)a z0S449W00JL$ zZeXf2z{%r3q{&m{=jH9cO_VESVUhu>bQT^oIy(AqAY?CN)l%R2x)~;#)&k}6C$(=q zzb%Ek^ZFANB(7yry3O!x`AMU312Ff%({{njp)dWH-E06U@q zD;OnDaOOJ+#k2d%FFD0m-E^z#ebI>OD@8u*z@qCkDS~2@=SmvY1<_t7^!3|dN(NT zwq4r^1qPb7Pl8nb3Qd?}m(N?BoKWRyG3dgygi&16Y?scZ;#&b1d)G@Z%{fP@??;s-L=lJ zZ@}1&K>Iwdm>a6icXaTDh1d8pX5IotgOc)uN7$P(@^bgjJ)*CgRC(QUD8Mtp)RnaH zEkV-|F6H@Nz+~%G4*waGzlTQ){%rih3U@*Nzh(dJNYZoqlFjR`{=&7$H!(E@R6$%! z*xolxohdR<#|p1n_7GRHdSlAO5+K8Sa`?UX-=-NL(gr>_w+0xyo<*<_efol7c!ui_ zb%)-uM9@tf&>OMCg^Y|y7+UvH8;AZe6&tuSI)V*8xC=;1l}GG-)e7+>)=N~_#qAD- z{Cx`bLp8%_H8LV^+o$6h@OKuI)2;6AiX1z}ZlK`6wS)qcS|mh7zy#i}MKhIZq!G($ z?(PPrvC8eNArjB=4pI&L5^{Mz4ePA#6a_y_>lrwZd_EqQeh|@cu!bkoSOnaA?gc;1=$JZ-qI(#*3-l6u@ta?Su1 zkcTX(f1(ZoQQ*m3@!lYofhyE3Fq`*)ds8EB`T$`0ZTsVCE0x5MxM0%N1t=ps?8R^z0~H?@^p2KEJFMlnoE_7w?@=Yv#D@4_K;BbW7Aqdz z2#0|s3zs)pBs5dHrc0$kQW?#jD(X|TUjd53d~YRqvXew5&*JfJ6qnM0K&=I!CV*?wLg8^ixd1+MwOI}#|zQPJP7Q|HZg zcl)oi0wnK&E!hj+`0l_X+yqjmh#+ODhMI2)Kb<0c4y5*J#f(3*Zj3eq8r;a73>gv< zy&|$|WPr5L{AXwcCY5Q9bnDYTT9t6z@bOf&LAU;sZhWFverEg`S_aiU9QwXLbO%uP zkdl%4&f8=;N8w<>DXH4!3?uSNro@3(7wO-lgw-*Sa+OzClu;MLjg5QB<$)14uifTS zx=ybTVK|&t;YKr}2N=6q3`ZXL(WWDS;)!v$9rR5(8v(=To{VQOUO9~#;ze*mOe~vw z5W*Z~TP-n+&`ZiZw%?40jITFQQ6Qp*35#%<2lqfq%Vy|qjJEoCLU?&*^`9^Sq<8hP zzjhmxp-q4M2-J|$GLa(j6Wny)@~u0WEKJoe%X0IcM)AR<*9OFjIRHDQ9{>AA8ozhb zWfs@}l}ic1Y;HZ*Y@167CZ3oi?dhV`LC^vxpXzL|p2lQ;t9jZy9OY7>9#H^LD<+%S zmq{k$q5MH#Z;?pGQ!Q_NVih4CMPAs13HM?gU${XxX;#Vs<3^^hRtTlEc`QnifY4pY zli&m%$U@C8(cMqI)zdEx3}ww*4Rwc3N*7JwHG5$b`=U zafgto@b+cwAl_LnkX}2z4*XM#_U8eb;q{=m_Q${N0p`e%Te_pNY3ZpOGxYqBcllxAl zK^e<*?q9+-t-eYXRRTbHrR}rM?AOeI+gK0)%r63jJ9U8{1b?NMQ0>B?^3b5QyH9_S zry9dvMId%b7Kx8T<@IgriyW}K9bO?7va#9Kj8UOssN_RK~}w?;&WRE(Ki zI@8ICtnSXmYpsc1lE!fnbG+=wS^s*&a>kZQbkbfFV}NTI?lrUJ?~>`F+xPf-#t4YS zO)^X?g>;re^|gZD?MX9)@=UcA98&d-gHiBul0>&Kx{8AmiBE|Nk#Y>WyH23e*|J0_ z5Dj6OBUNyb*S^GRHgU}vF6RG_%Tl6jyAyGvY2;eQitT#SqLPt5k_f8PGdgKZH4vcDwaK7|K$u4L02 z1YvlG@)}S!B`;m`t^BNRZsnEYvET~`>Bh@O3l>6yg21%c)3HgDl9ILuBGBqQVKe*w zVr|(yJ?o7iy6ZGrL1@SPn*5{5o<2No7IJQy{4RTvb-P;m_h;0(?psy+B^Zp~ytWh` z0H`8_Hmi-VsD{gX@VK4^0~LIO3vig)-YKg}e}5||@jguejk~dKy}$@lle)a^fNU}f zBSr=Hf!HioM;ha*3s@v%a!9sxBC+@dt0iQKr2+YvdsSaF{p%vyp>On?FPHg_yI@Ex zRETd1@6h2DLIRLN;W5QkbRI44L0jR5hhH-_bTcJ2Ew&u@sas&IgOSi@lC+4kvLRW4 zaiAc!Z2PP-fcV>&mwl0;?Fu>;mEjhDmI&e47c1;pNaMuf^9F8N@ZPU6CuPlNhS1nL zyHLe`4jsPQ2%K6QL!I$4g!tU*0ulV^AvLrbk8*uA9i#Y8*H!J!oO?pNjhm}sS5hFh z3goQwI8#?@6r)2qM;5;+VR%ErLP@};2ZJtF9zZvk5Z^W>gbK$|^UH@$LZU@Kh8-G0 zCp8m-%7w&F(*m!I+0>$LBk+zGRlfG;{PVwk2 z4pmAv){$DQ_B)p3wUm^H2sfL{G~wI}>2LzMl(DtNMod#3ix3|N!rLVwuNyp2bDnt- z+L6+EAQ77RCTZ1sOA@AHW(*np;T(U*edS@3s#E*ECnW+D$UrUUM*ncH%V1^w{3P3ZZ2Cep@>YoaH9kJP^tBdM$?KN{!7%3f*!T#Pu7CfK)wg;eNO1OR zbaWa)ij~v2LZ6*4s!sg9tt2La@T)l@LUA{LDMw-=8otrnvB(s@BqK~}dJxho`w{hB z_k6ilbgJ*2kkKKrF`)AsgPyYqwlFpeH-hFEB3Gc+c=ymqI=HS?gWF;LNR})j{*TDs z0p7NysafP{)of*Nf|4S!J`BK!e<0tBp)?cPir*L`Jj=t32HHG7z)M`P|qecka>or5HP@! zL_(k?Sgk|XplKj#RUwD%@5LW<5n~p-|I-R3u$!mB2;H27IY|B2**u%+lPb z6KN*Ku|Ioc3i=B$w4bxAoX}XuV2jhCn+Vs60Ov}|1a`DXr`C!B=sk(hi+sk$MOocg z=V^b}aG4+2CAw`XsbH&*4z_0!)hvvw6AMTlB`*ik#a=%{EKov6<2V^5cg@a@0M}Koo)1!KJyMaYM5##s zRpg7gU7ks`LwChK9K?P zPajz@F(A|TIw99W3tk9K*DQGfPAYI;mDPsCV{c3^kj_S!$6S9*pAKL2i#RH;saq^H zv6oVCrY`)_;XwMOmIw~c5KsA4kt3(CWJ+;6jk~b#^j%h+LtUOhzz?-M(aK5BYIL0f z&W+0{R+&Og@Z&y_E=zf64mjz2$-a;cgn+B+dftUIHOuJtskJ1RyiMm>7^110b*$wC z1lC+C*G%K2XO`M1z?&ktUOA)Xd3h?TJ+1E!W*sSGyni?J{2Aihv#Xd|Z^* z8Y!2pW>J&l@m}%nPT8OO5<`ZZ$ip}zYT)?D50f-H?)333$M9N+Iv zcL#BXXhzkEwsYK1nyTb1S{U3-!ds4RO%k)geM5kGRfk+myQkWkpxM2e&Z$)IPNh^- z+t#$i@#EE|$-KuGaKcYErznjrQLr!5;>9Hc+=lA5L#U=lw$IaWfr`X zogg8JeA;|Y>2lbo;qW$#(JNx?exFtHo2Dm$MUG`=MN8nb5^$vkmt7Q$A}1w1Ky3T0 z#??ukP>GsMbLh-?92T27Qxc;Mg)>E`WG<oN+~6S?#H@T^?is=}Tt zT5C|5{lW`JQB`-{ji2wyE++w1E(jmOFBaZTh?4xCPEgR+GY|1jHJ8aPZpj|*FHPuH{#SR zMqwrBXwKd6?|qlR2?ZP$7u(fQu0KQbbVgWAf~H9@1Yt*Zl&P$KcElxYMMf_aVt=+p|1ISq#cH;KgX^(lT*YCxEPuvo^i;Of;K3 z)dhl;OkZA#s$lb<1~lZ~U+eYT{o1~(3tiXz>RC8WE4-I2;LTfp?J5hZDVU=Qt}C@w zZYD>*P_U6o+NBzWZe`!)DPWaLoxw)>&^e$m$dD}>pxM3=x;|ZQw5_=Srxj&HM=Nj^2(>i2G9wd#WY}Rb!+fl z1ZQSY&D%Jj6Us+On6a$v>{IORWk;kPmyNrWnJFHAr($sWmD81jeuiqLcXh&|<7kan-O9~?1`%IJJKk_;xb#DtTMm?_K(3Ry`uHL%S8q1%9?p$PX zadP5qx{z}VzJ`(T^_Iba35 zaasa`iET)ylp!_q_oF4))8)Nle_7QTPu=M_mMWZyt>q+G#`+W#CMBU?{Mop_#Hc;a zU%pvPVCCUJN!+D083>Td>|WuCEG=F+LV#Oc!9tt^iUB`}*w6Q5fX=QPlD{4vs@9S# z=i{U1s$JlW)*`nB@cU%X9R{+sPuJ4;@15}7g#00-g&0%<%7L>g<_@>_TD!G+*0r8< zAi+&GK}{sW7DTJ~1glL#1LD2Mi+?b@#DjPH)Xgeakh=K7@28*I)LXHft#jerXx4yc zF2*J^-GyYy2OmW}x;jUCZL3)UETDU=&NA+Tsyc;bswnGV#CFkrBTX6vPB|PkrD8ak zoK(-$1-6EdA;0L-W1aKw7m?c^XkF)apBj_I01R67kx!+D#D<5nrM*bns(1nz3JMA@ zpkiljF{#}EW*>yh1-2krYj3f7gtvX#ntAFKzPDBX6NyDd0wvLcEllg#nUfTe4#7)V4 z@~sX3q@TygN+`jdR!u2vR+)cNb!^k#dp^XSk#EFex!t@)oN73Kv zD}x`sX3=mq8nOYX1Zbj(Ef#f7O@C7go$2Hf4;mQMJPiqkvay2v^4Y-9&#o6_3g;oi z%%h9;{4T7zdEL6$Q7$SVsERC3HKMBwa9EB1GB|v5!*7g{JDDQ4m=mQHibx58sAUsK zc49^5e1_TA(i#jt?xJow(>SY1)a>+XJOyObJBkr1T24dBy7ZC925)rgVAq-_7R*nIV2HVmXJ$N>h zCdfMK-5?fcheU20GyR3s#9^E`E5$l)-)+*t=j5GtzD1F^lW=gTV|8w4yKg$SS%1q1 z8th1ZzcjHq5GLmrQ=6JfeJ@lfep>LZkS zx;b&)*^Q$T^iC`kyJyl>kDJxE&WiS7;6v8|SZ59Hf3sasfX_z#0Xk+^hN=~1HownA zVpPyaze2PBC(_%M8}<<^uT4uQ!_-=BrCj>Jm$bCwXct+Y>RY=(SY{qnd>Q{!;21vD zBGV~gp|)C8qN`$?c*Qm>KrAN(5i0ojteP(p{N?^961;HOh=bb4^!MLxWw*zmJ`YrZYv33dBC^(lE)Kj7r^9(PG_(B&F1Tgv=U5~5Q%#OQaE1ti9 zRRV37vFN=SG<24x#SK8CRiZ|kRu3v_-=B!gF{^hQ(rNvw6^U2^WGjj5i{%w-%>yWor_n+|JJ&F-D-h`wxc+BRcWe3?vu1k>-P z6lbp~zle*tZ)pxs2%Oc3gk*97jrDMKw@j8L;%j7`b?B`+tjzWj_$*H24jHLn^R?xJ zdfh3&_D2@o~6;gpvW7t+H4nl zl3(?!nN`i-c=`|8bL%yrlp=NSx>nF(f4~x-Ukj)oSve(l{?zbk_$)_1sk-^lHOsAA0Q_2j+9D_z(2ZG*~PD zNKoNF$LtpHDen$YcIk^m`OEg&m+*X@hYYFN%k_zCy+5H${>>Mk8a~|0yYu4hksYGq z%1_F`^(!(ac;R5dwCY%}z?$Oz6^YH`8dn^g8eO3ZxEskJG8sA*(RvsHTWd!J)%3{?~`v zK~m;rj+}Aq3ajo>?XENjeCT-~M&QHo01li2hp35C+8XokNzS(dOC#zle&`?DbC+Cfds5jAIJ6Nv>4X4~b7=Y2dF`tEVF7Q@ELr z%SvWS3QPS;i~M*ry1X&k9@Aw?Elg8wB@BKj6dW#*jUjDU?n|O@8mQsU>&{c4CCDaH z>db(|^hE01HgX5JH9XFq6W{)`rr|5xrJLsjJ|6TPEW0_Z)cv?Mt0r`uR74rc3Jas~ zLx^(6f%+h`Etn%PsquJ}@>O!o2&S|IP!$chn09gzHX3l7lHzR=^ixg^UBe{aEDDB^ zOclw?aWC>QD`EUDup1k`7uNRY6All@RY2`{S@`?ICF?ypR}#psVQ#9Mz>wYF00(N*2GW_yjY1j_2uz|4rem~pEijF^fJ zZq2QHA_6=#m_W*Gu>>U*6=F*9pMM zx&GRv6p0+NAf=>G>_-07gw#ok3lL#Pz)4owVe3vP9$(+&<5F%p($EOjmiX>F;jKOK z+jy>$d;-YXRJEIQs^J}hc=*35A8#G|Y*PMcE~Ym%vc`vE+HL~9 z+Y7A0!0UkTs(p(a<$j!*vEf$_r^&Er+UC+sUU3uT^xPsBy?zIF2Mh~hV5y3y!V!_F zZ}&&OhI(fSLrkBHV^E%&bU$LHqp`p=l~XcS=>~(eA(2$npid4pa%R2H z+FqLNV8;>z;C#eG1w)B0jZdx_ws7AZ<4M2~6Q{-}M{S)qY}?SRNE`Ty)og&Y{dw!y zdQM9y=01;p5a~!JlEXwStT!cdOopl9Z=urdvcPg3G2Gm4J~Jbp#hQAnQ&s~;QZ&W) zX$?lD&^@kAda>zs2#!i>Y>_I8v!YvmYvehTBpzzkqk>lxsqP#AsE*5@F~n6}VGdZ@ z6?~!kj7hNH&(C&TzNSPhER&!1#F^kp`HF+u*-SwDpwjlt@%22vlSx%g8wYQQDZ6cK zSC3;41Un?aDZxl3+{oOf$hyX%>GL(`>2Oi5PdYZE1()|Qrs1V)CK(o5vit0K(6Ic& z$qUm;U$&?~UL6eGmVPI#g$qv$=N)u(aDv>x==oeY1AgJzrfbmC@w<8`0%#-!Z-l?( zl(gjeyMP(c45;gq&4&(-E+FqaI$)fp_x1e`Hk)5`ld%kE{|M48vV32n(5eW}O5(FN z#sa=m5=Lc&B+xU?uCXCaSm^E@)4=108tQfxF9nVC6p~q`vp?nVwY)-w z7RLAaa+GUS$1(b=y6tmXE0?Ot>acF|g9{8_(SfThK_q9(ypRI#EkL1#B4r9KD&bAE z>wI1BWpr)C_Abfw%k5(IM+vD@W=v!vBB6}4`-^?_#2~Q-X>g*DOzoq3QKd0VcDSx&xmN}w;=5E>p6*eseoezqO zZUQ`Y5^A$DGdpXM!%yxhYVGB+xk=|mE=MKQ{1Imo*oT6pY{zg45?ICBBfc+YLL#=A zt}?_BBX8%>7jaR{P;5(4ZR@nicFa=bS7?pPUNP|i)Xd1N`7jjmN8aQ@sO#RSIOB7C z=WdaVrv7L0OlEzZoLBy;bm^R)fr1}~7X$jM<#vVR?wv8nbDFYRDts(+xdWBTDaY6g z$p=(ZTVa)W>cpbZkdd@iwJr0{95a4ZC&l{>96j#J|%AUtQqzqE+hR6lVy7^JYMtBloZuuaTU zfV-=Lo7JT6q0PDxG3f1EI(E67afMU;(+t2NeoR#j%XUDHyo2`_lcR-4r zh{RChL0;{CgXH>qp9XAl1VPe=uJ6;sldS`7dQ}Hd!ObL2uX6 zIFy?$EXqlJud=`2+2)m}!+7{le_(I%kEp9xXWbzv=;K6|%)z7e64?&Qm*%rl`zvc2y**GmQZu=OIVCn#R|c1;Ic@ zkG{TC#_nH3_a&7uK0rDId7Wu>A0R9z@;+zwG->pBG@GQI06NhUAZR)w#}q8J2BXy4 zxW7pPwd>v)a&(fh+y!)&nEB)ByT3o(7gl7X?kPtQ06cL$Nz{7_=JPSQN?p^t!&Dr; zDZYt3#dt~uw6(_zy*E%KHPkTn=}qTi0+uK=bup29=$DH=5aDqmh9}DCYFr-G01nY| z1}y4Ahy7(sc7pXAnH3oNB^>iMDET@p6FCFL!EaU%3gld`5h*|`SRTI%vtH~3H+xzA zjAi(V{!f!*5WZ~Fz7+0)(tRT`z}aW50h#rY4}+WbWYxlmvf``O?pbwrNY*x@DMzhh5s2+1NTYg$fEc;GM7Y4Rv*b)2D{1@jh_xkyY7r*5lWjwIz zVm3*OGpDMS$YpX4KQ(z?fXRohS zL-0l8kG!afB%mYF9WTmoULPgDT5GSsx7lMgg;@ms*8);{mV`kfpjEJRyz5(Fu)3|@ z+T{0&^l}iQpQHV4){Ng*4KdtpFVdYyyrZ-V4QPdC%p%Clm8ypDmQviweGMP=BT5~@HsoFl0(^ro&r$^hwN3pJ_y4U$s)O;UM(si)}`YM3NY+DA2 zadoV3CF0N-fW{sOPgKM3(I*^=A9< zY+zyj-@uT+4Brn|?Ufa#I)2=MF+1FS&tuj^XncB?>%5B~5M>B;ob zm7*Qq%KR-Qr==W(+z_}sWi?+<^xJq4d-#VMKEmO~yjQjy8;E{&8pvYE?{JJj81ROzw&Nb77gu-bP-JI=nG592wb>SN&IRbS=|0K=ItGT5xp|kBqW$@? zGFc`LH%73d+;iIIEawpVJvq(WEB95fz81`9YV%wZNS)iVSFYnt%1fn+=#t z9w4TDf^dmXh z&*z+ep;&I0Bzx&?XkhwOTm2cJld7Y3a8fOv(R}&I9gxrLefO_DA|m>-+i&u%j20x= zGK=;+X!CNh)gdTr*aFH-Ig@+@u<7~Ij!vu)+X<0o0e*1u+pNVyMw7O@r>X~=wbzyl z%zEd(oZ6Lnv?2ts&F_QSargz=uUNsLXsN&Vhy1q(Da{NKVLVO8CdF7vUCD;i zsL`5(z_SISs9j3u{aVa<)AJVibaw~FOhFxa;CWtR2kw{6h zib@oM_(Xe1;m0Fu!DpsiRXxrA)VoE0V|ZG&4x`y7Mj_3jg={|PxMS+<9zY7$>kf7md9*VZbr3f zsv@TkUA_CZ;ehgYQzceNTO16I)YpeAMs5OneZIV%;G$jsnAebMNHINPP`v&6rmht` zQ%Mj;%Z1reQ7*sZO_<=l#o z)vPGzh5iHAc{En9=orFuuWH7&`lNk_=2Ba3q{>fDmH!mka;-^5BU_*<2M5zPB*Tr| zh)dni5OI6#Ic93Sh=Z}?8cZ|owik+MWGB)6*B+Ulaw{$>Eg2c0Lgoc{(i5S1(~C9~ zAs<@x-VsB!8@i#h#aaF*&d5eHwsI$WZwTcFbl>sOs4fL*D9z|#NtLfO(+wyL1zY)k zvbg5mE?KmUdX_c$ZmD+#wvC!|@@qkvhW9L^?8@IuQD*N8B{GbQ-V}s#W>$vQp)VN* zR7khJse61n9HJl8hDu{RN%*YYm(a>hb5Kj~h2g^kJPk-e-Z4W##{jMRnGGrEa~JJ1 z;)j1n@%W$CO`vgs4i!*d!v}M1phT*pZ1vCDBL@^%2v$cEDHj^XBj`tu=OPenZR3p< zH>x?c)sn#fTncut3TS^R*f!89ywHOA4GdfUq)AHUW{hUfa(8~vX9v-rUY$ne{aJ-Y zB6H%#S3RL(s+%}NT@d?}sM2EJJAQRhFo>|%-MlzyP0-GYWL2;dAe@!>g(Xa_`OaukoPL8pFf)}!OC&wQ6#p%BEec9R=IkLjEugxx8nqa&A(n^|e0&1TUb)GJMY zbej)lZ#^Z~^6cw#fgDG?-T(pNw)yv60Ewm(1T2KKTbbZ3KYz~rSjK$%6Gr%pvno?# z9(bfU5_ldj5XUjc<7PaHlS9ie7nW{gKMr$k!@y-m{C4x$FXB~K^0&Ns)OOyp7`fW^ zCz;%um(YMi(tbgHjEz-KP(+nfc&xawy}GbK%;a6_A~a}=DqRys7qoS{W3j-;39jrR zYfbUBt61xCTD-NJSntjTv(LL6e4Knw+n}P!vTD88@NA1fi=$A89@KX1X_M>6FIk3} z>_en?pclD<_vre!50rjNH%YMWt10i!wFtAF0!`6l=8hW~;4(L*s{7MR7kEf^RmH9t zFQ`ehTEBDMxThin86V|h)wM~H*~JQyK% zIuDMY@`*2jA|bM%r5G-KJn(9D07ozU$F)?a)zF$-|EH~YjLx&`+P1?+jh!@3V>hBgtjJC3vv|TYZ?$X{})|0^&2q9($-`(N}3f2n$b)Fcj<|4 zq6w*15r`8p(8e~GFH<397p`Pm>03Oi_$Qp&`|!EaoGKTZAboXC?&6b5I|~#8g8q)V zfdEXU&+G_QuOLha3H={%71LQ4aiiBN8wEuByiYZMlPkqT z_F!{k;#u+H{x+Tkkuu0Bi*(^ydw-90qkkHKf$o{4{INgt-i*_Qc(LB^Nh*A_H$Hhn zWBAHAF*4Ev&$cRO;-!h{)Yc{rPvj~6!yauCBKg% zmI6K`S!Q@w8at*#N5WLbIP2>AT(GJ}uE<^wcHkIe!9t&g6q$@a3-wq0rXV5D7+#nN z!O&adv&~Ew7?ZsQevhCl?Ha7K7})C8@pKTG$6Ghw3CA2qdyMz}+XuXv8l&&7Q=!kP z>FE)G&w|xUM|Mhw32~<%+xrTQd6U%C$N4rQjW7lc54yh(5rj_i>noyjaGnm zeMP{B<*ypjC7@;+YN}&jrZj#=DTb`&GSK`EBha;V$mPIo=}cs-;T@hb5gT8fMBoi) ztigfXpxyLX=J=k349q87VEJz91Q$xGbyR+)O{*LaYfZgQXE?Y~)rlexEpyFbf8J^wJDTrEVA+@RQ&oJrepTHw%+5^V6MLA`iI8J zeF*qhXmbw$O0aiaS}7_c6UqJO$%QUll(EK{ucXGfZ)zkZ0CU|%2&32SLyu<2WXEH0={xHZZ94qa$kS|+MZBA zU||Rd)v?&DzB~x^?6du}>*W2s?r|->)KtbKQ*C_+?N4MSkir?v(G$3Hrk#IW-R3VU z-z(Z+{xgFMOdBdybr2Qv2XRLp7+;yTZ?a;4RwX!SXp`XaihZ&9nqiAfUu5oa_Agn5(_VS3mdkn1J{%>rdS@uE zg58ffhK9%|y%+?z;Bi{j94LOdtaS@)R;xw$I?ij3rqDzfa|qAu$avByoj6axTp*wl z@umo(`-@k(4`H|)_SbKfWjOxjs<33l)C6Z{{Q!`Q2qs;#|4JF$LGq0i{6U@Um!^AL zinuO)lu+yLy?gr!zDBrytFH>oj{l4&CpfvJfjjz-o1YjM+PTT@S=>;PYBq;j`_d6& ztHR(>+wBjHXD_E_l$dG(!Yog^_UCi1_}`ClZW)Hq^v52_UlsQ;2k*C)>_!rB^x^9V z;fzKPuqAYm{?i{R8XZr-PF{4>BUv9TI&wPwyow{wHu9r<+iF+uwS;c$v*l6>{o1ic zn?ZgOcr*cYz~Yxq-C!Q2S`h!UEBBj2UjSr)0}Kz&6P*J6;7DKsOCL}KZL*){ibW;N ziM<+m9Q@!D=UaAdsI&6-bC>27CC;gW#MwW8vwLWtXrPfUpL_}!!? z{+G32@waOdn-M#)pk;af3PmXgI_su>Ufh00azc)7i-3zokf-S7VTEJTJQu_-lf!3c zhXc(dR%dvG_<5chKga7a4Zm5vgS#_k!FX7Y7Ix5Y#!~sTpaf@5GLv~XS}h#_a%Qfl zyIWAr0<;Lz;wx;GFw8T42M(@EStl%c!&mBg;$JyG81~5Md!5Kz#$|Hr8)}xl7LHTc z<>5eSb;{Q6fl0ynVZ*}^pYuO)9u-5v%hV0ht>%q`f1PMa`nnwf?I#J+V!eL;@iW!a zaiol@dSz?sOk2s&p;&81J)s`Z+N_A|KY9^K-B2MhjwM1QTetE#f)}w#=`VR?iGO*a zVt;g12xV_uFzE66hLht|KfiV0{2Xa8WYS{$a7bCm$iTEi@LT!eCKasp_EAT>ewp`} zZg$64CAfR}u2pR=x(L?fxV&Zgzz>o_6zbt36bTYE#w9DT7!vz3sTn~L%{f}q{Lfth zPG~3>2~Fvr=i-K6g&jUw6|X;7KU;@G2GAnMA&reh@k0`(N%Zp zE)Amrc?2X-JQW$_5MS#32-q_yKIBFC3zSqg;J&~3^r(xA&I#k-GJ?74E!SlPAx@5z zK?29up>KZ+YbSpzK}@;D5Bf?V|CW@#g5I!5X8-J{tB)JE>%MD$2Q^BgdxtS26H_s z3v=XecsUG1*CyeD3#?-drmJx-C%I#!ophb@c17yKdm9-n2%;k-> z6yVfLVqz#eI$m@Y$>(;)RG@7i%wTgiziQ9ze)C*Litrh*J<`b~p9xN4+7pM4{0?$- zgyG9@z(z$C_05tkkXtASK?JxXoXci+&zu9ZBJJjMlKVc_hL&p_g9`9^7DV)#0YT#} zc5e)SCmzHg=$(}EB!1XR1+0BDhVF%xjB^E9F5iXQ&jut6<1R;ljZ=ZpGxQ=B!zlg8 z3dU%LYQf>>T$7`*lq1Rqt85Y_6}qu) zKxKb(xfgLNu^|MmQNL?h`@tV0*$xq3>@o!Q3Xa&7d$eZ8q`LBuwBbcA*y!Y5@2TeJ zXU``^2k%2|?m9hOwWM66pnk_C`F4fHPM*{zvs`?)zfBBPGbtso&SuO-xzD+AJlQ@4 zX{8d8#6=O&hY<(e>b$*k?Z1RNk_Q{w0v9SNGyPlemKjr(S51Cq2g3=MZR4xALU_{t zol0r`d8XFqiGuM-x@Vd43C6E{^*l{c*E?k>jO*4%Wv-w+d_Y0nlL51bj8=GSnUbyq z@d=}Jc^_n?>Jl@7fwk`<=!itv%Cc30@9lwPduPXy0MXL@XYB;air4E?w)_3Lo;Um# zfgt5Oj&6h?U85kO7U#eH0ODbHZWtgeS)$kYC+0eGzg+a#J?sGEEQb|g;5*22Mzc-V zjAa`hiMZlpbX^Y8N(}ap`bmFsKCe}{e*R@aJWI$14#VhXiBO0OckEjoLiQt?*>EHc z%YT%ppPChU;r?#~9cH6gs@zk0bg~Yd0SK4QG|bjroswU(p_2?zg$Fl=p#}HMzA-6D zAggEd52Q#TA{*)fZDT#x>+^k(qqy{KPAUw>1(icC?(Us(uGIo z)@19hw6dM*J`hvk>iW8CqB=&Y*&f}Pq7nmTfINcL<_(DxL^kfzT0)8sF3dG=|LZQ+ z%Al*EXkbPZHkVUb%*CvxHbArBRuj4zp}$>z3=+o;XpAbabC@8%-h$JF%vKH~gSHZd z)2w5$5JWft9~bL$X9=|9W3i?(B#9yF z(YYh3sTdX&)SS{(GupP7S;VTu`L_1j)ztY^P<Ek$3FgP|h^Ojii9GsApe=Z$pl9CK-2oE5uaT?e<>H{>{ z+QbrN%b0u4qV&ii9vR}KdbUk4Ofa#BMx<*1SqR0m)nZQC56f(!LYr=Z8upJLe{vnu z{1unNc`5aC-IwQ%w%FH|yxE6CjeAwI9<(?Lqy63QOT+Hq)eV`t#e&08K=;5-?0>PX18*uEL zf0m{d0y7OsLakr_nE!9vr`Wo3R=4+#NjkcC9ytSU6h_Q9Tv`F(p8YJXZCeJ?B=0rc zm)6>*?{x1v%sW+eMyZ&JC!(!&W*x78#YFTQ;`6o~^(13pSu80f#4M**mgRU=trjHk z>xnMi^0dp`10@J6`F~F8coXC(Hef4N=BJdrNUdS1{H;tKXo$C?6Oq>-o$ggR>DUgX zPNP07Y$_=&&Hws6L44HO!j(cde@MQ)_XR9D;0fIz)AKGFA#*{;eeC45wspp({dnDj zlda8|SEnbpz{-9Uv*jCu27k1kuW=ck0b+(O{f- zpEjfq0U@fiFjM26WI*3n1qq4j_zphRBQqi#>L8uWn z@zJXi(;r+Vq&8-Hos2WG1uT_Xv^mEgEUfdgO4Dza!`yq6FzAz=^RhBqH4v(;K-uM44o3fNKA}VZUXdw9lwP9xN;kl6-0!ko zLKAC)d(;$}33)X@B2NllZ~X&mb@QIAcl>ijZ8)69COW?KbPuGC@a?My+vEz(& zn71yCxU8n-mUq&|YGz9W#H@OA%p$8t;#b4Afb!u2t`#lQbaPBfo)QhkKY~D;cSF@6fm3$!haGitaxv!Qhbr4@ zZYL^WFkWZ-a@Fhk48`n^`FWg|RncpOYYG`HVu;jZz8Jo%@YBtVEc|OZb_gR}Dg{Mg zC=6V>i`TAnP3sJ_w;N&$Q-0x6MSOZj(Vc3$EZxG^%i+4q4!pKzp6jWHKBk?Uaxig> z2d@aSKYNp1-if<%b{$joIl7@HB3$vGneyqlCtru1ozlt>(fHqaEsXU~dXmcCY6%2s zXNZjo&dSwu!=htBttx9Q2Np2$e~7`WmOGY@p{xoC;A+bAgVr%~stST>yZZ5*SBdaA z9mIWSCSO0a6fxM*=NV8C5aPQR)00lZM(|LeVS%uo;^N;(9gQ$xaN+Wq(gW3gD+vgu zw4TyXmt)4oAEA?tyE%>S+# zdcqUQybOgc1_+<#_+A57p^=TjEGwMO=Vca^758qQ>Er|EC>-kv51!@_@((T{Qtq>i zHB~u9MT7ct$Ii*^$&WXu7i}?(Oh&rIdk01fDS!HD2A$N{;Q9vhq|Q(cM@m!uUZ8;0 zvFj~}R+!PZ2pP*005(v3k;!p{Z>T9jZ+NO2Ma66`4v)?)e3v79ZEg}nnIOGB)11aNpo5VkuJCs3@<>+5QEjKR0K%ek+TUEjxPnT*VzE*R;g|_(a$bXI&$vK>vEIv6BgCQ(E%OimaP@W1 zH5#=yBh(~ukC|~OP`!Gs!Tk!ru57&ubyVqz8j=mAfmv9$&>7lC2N5{H09%IfcGxyo z);op{Xr05@viRwik$W1k-+cS+J1=utQan`%G5H+#LddjNR7Y_gN@VyiH*BaDNsMKO0{~|@US4QdY)!VB&I{>1F5|}La5_!e63PI`uvS`VoH_=M z8t(C_##0^+R;Of?%GaLts`s@Myrn0y8||LuUe0%tVHUT(J^@5*65~m1F5Lqh+kLam z(h>?7X6WSoi|Vamk%=l+wbpcl4;0{T^rj(475|TC5)kE5n3OQ(+3+}VR#&M79?(>6F6`?SOJug0uhujrO zIG6a!iQD`F*qp+mE~nzfFQ&+_lv-P6c8h~(Sc_jQSDg7>lD^}&jRx?%PV5YGg~y?3 zEG&8{<<=E}1bOofc~KUh^Fy#A3TWDyqv7D3I`qOuRhLxsPHhW*;qywlD9AvXKf~h~ zyg^gs+KY>eW3-s3VS#cl@B)yo{N`TA9M)vw!QfB)M6|ThmVwWm2xb!9;})?}P?#M=E^_aBxwCQS~;zUcd=gHj2b*AVKW@An+@ zBw;OrpBmF-2#U~S&u9LfmCu1eoNdU zuLaU4GHXW62nn&Ja%tvYPA9sYFJV5V&%<{8X4ovX%JDP#ehuf5&` ziFkprpUB0dN#7A38^x*5?4H8t+oQorZCXy*<2IUJ-pKjz^~8$4>?d{DzwjI`R+DgO zIc`H9n<8yD>k@Uopgle*Joj#BRkF+FV+=df3ItOMSdQEZGb^Z0boG!t7>-a%j{k(U)`47deTat)W% zUpVHJ4EvB-2*Fms& zK`0rg0z5mE9AOSxVao2Zb-ZXPWRUT%M6mC~?5FI2Y*?HU_A=-lpoPTp{T$}7KFGp$*1W?zW!HR2Ry~rBLv=F?>4)P zx+p!Guo8Dgz_?UWf(ysS2NFZ%VSuotFcclvy%!E#HS;3Coc{t&XQps`p zH`#tFy_DYX{*}p?jZa7z#@o4H1C%g(Fj1O3<-9)R;4OZM!buo%N*$E-IZw%z*!LUV zP&VoJ6FQMqv0Ca6FTi!SZ*IS9n9KBHxh9)>(Ks?ShuzoYR+mx^d21e`PB?~`m>9iv z-w^V5_q)?eSU$O?16t-2gi3L3XASZOJMruc#Z%9=mLDVqZ&qET=Dd*W#%GPMkkvc@t$ z%O2dfCJnMAMemqQlM(ypPISxkwlziH_w_-`#!~{5C6xuvZS>T;#j6;DA5lj;R1pMx zI&CH|Z+hg7A9S9WvEXKNq%5R7l$E`2L0hQy+cynd&olKD9ziOab{Sn&DD zpTEwC0c=7exe`!D;jH>um+rV?22dv~(loYbL^Ln!^Cn2=a@c^xEqLZUV7N`W35ofr zpgSVM7h6Ahh<^bMCVgu5vRrGPJ9tIGD}sU;wxiJ>4+;4$NpAAPDQ7)?_dwiCLSq9l zBvy5-;sg}TId!vYcis=piB;2i7T;NuDRHMr^i)gcnehiG>@y_tE)N7Rfb`Zth0wr@ zBf)dYRd)Kh(&6pB=DF|wwrCjTn#&Vc`Mw=)qLy8Va6uCMju@evIvi-EO1@*>FfXpFKrSN^m*hOBY{dyjHSb2AgkrI> zsb;Zfkj!*`I(;Ii^9iNnnN=*^U+k}cxt!Ysc{kdH{L!XMBG8Z)dx0&RjtyszFQ^mk zP271LK^7;WJZAjt1uF~7dT_~dwpa}gRLgD4Y(L(m3MsIe^;xzy}80RQ6Qq_smAd z>PdOg4JNm_RS=N{e#QoKI?|IC7)$vDL3FsdXXhW+A%nt1eA6Aadr+W!81O^z^ojeD zCOIvWf`f{WEq@@?cDGUJ;p{>H?WG12^7-24uB?w#`zQ)2heABKZG(ux=~QirF?*U3 zR)B29mTI7Vot%`^r`xz>?cp#tia&d1)-cAj7doDxGv!7kiN C&|Ema7tXl8wd3q z+ux?pvuLrbNIvZcSrXnR9=D4e8-Bx*F#!&m02JCES_472Ccnnfs$Y~_`s0m7VljfDrJ#%wRtssV216eGi2MrIySSB;pwVbW z`~K;V7jRi`f()>@O=7O1JkXFKfVmI9Wo(R+s!gD^{w9eAgMgMVz0fAmBWf`Fs)!^C z%5)s2_Hc7MKM`H+KnJ4=>o``E>!^c-Hp_s#X-ySip~CfX(&J@%T!oCZ#%uTzkh)B* zo-VGeChybwFcT4H?D^QLdzC~fQ9QfnV6i9(t*=~__tj&Z2H0qI{X7N7}`>n>AThG<0IzF(~50c zwsKklyO6gurJWin-QRgU9^#&~$dWxVl52C-#Yc*zGrv;VZ#dZ_u$d4X^L*PfyQ_3p2VuM6i_) z^RIjE-S^1>#z6lcqL_39|LT#~zL&s>Y-oz=fwwmLCvXE}=Rk7e_K>7x(G_X5l-mvv z=cobaflzAGM_zia730E9U9ZRdW$Y`@uW)j6s9p(rW86%RbxYJtl>EY&^TJ1dVcqm1 zxUM2?>8&N}#DCC5kZpC?g~`4Ym9a~wP@pXZ)yYZzkd7C8(NXgedVG$SeSy_&6lv`w z%8_0cEYMFgTt7^)r~tLNq!utjEUFn|VL;gPvi z+CJ6m2V_ytQQ7go$cOnc7UL7*z8Pd7*6AC$~jb8Fe)?V?=YI-CjW z3+YtW9>^aF!>Hn{;IVJ9a<3-8xtmIfv+eti6tDXI(3}Cr^{T*F$MEjn-jE*}LP$B% zwE&HjS+G(fQeSd41dD~hg!Fd^Rx>J#R-MiXc(d*r*7?jifa+{Ap)@=(nmi0^Qc{*) z$jF|7YzSCO$ca z1TH9%o+hcHn)Szqp#90A@4eBPY*Jon1?LSlHrkgIMGyrd*2Nwfi3lb75MBO)@i3O0 zm>4z`Jckb5mm2Y}9Z1)$oE99KCmpC`F*_VpC?fiAfysl|PH;3(qR+7mq(uIs;6tVF zrkHE5(8h-dYeOCCM#_%LnfAvgyag5u>+k&$2^j%B8IJ@fX*YL~Yi{M(fxs+=(=~qR z7-vw@d?`^n4H#`2h~jP;0f=5NnEh(<-L)zPHn!QcKt=O0Rc=?85@E76Y8J(5jnPPz z`5aZ5P8$wDp!4R~tg2!l*&mxomM0JAj|lgq`*=zO2?liXoixTRD-zUL(3&jwbAU zV!V_v-6)3RB)#X~e)P2Ez9&q0UHH)=E0VKF@#57(1r-SZuQu8dL>=V(J%{5d?=0S` zw!VbeEnT=zT~BX4c&dJ?xwq<>`qdKcY)U^YDdTdcKr;~WJHVlW)(@$aIUkm+Zftu`~`)ypfeNecD*~l*vy-4B}#I{zFIHJ;$LSq22T~dl* z@;B3K?0YZ#@9UUpA8f{-iwr9v-F5ysIm7^JynsV(P)K4J^WoxUEWv$siag^87&vW21xyzj;s1B)Zy!{nl!q5J03C@HxEYqr6@-xosv zo^LI#p*mJl)w~X{@1JHat|%7GVzCC(giMbA!*6j85#Ucu%DkGw7L$+wdm#OJS$KfB zL>tP#4#C;Me!1Q&CJiG@hdUL5iHYg-{E?(tHk0?xtA`cK&kn~THBcR!ey54|UIVI8 z&zo1z%x8Xu!iE(X&b{V^&0u`bM5bgyM)D0r-9&{>D8IR#{te$D*ft~JgTP1!`~wuZ zh~5_p-=z~;^_nb(EvydEjW?9E&3o=oX1xO}f-0siM&;{NT`qgCO6YfHuyKc|dS{Cs0zfxIPKOc)4805h4 zIEPQTzC<@QN7_21a&mHdjq4TZ-#rbU9c3-i-V%Hp zr3}Gs^$!~0%@f={k6t<_T|eMen$iCASitjb@_@}w8cMooXr!#ch?ro~sOi=HcKJq$ z>YE_CS&(Yg4G20+_kDJjsi~=@OQMwF=hzAGDxbSkIK_4bqV5B-(`-N6<)k4;A!NXV zp&2_SnLub*#Jo|i26F72KjfC$qFzLOI?PV9N*mvC{<5F}A75Ad7x9I(3Ubj{Td3{M$A-z!_Bq72_Yw&zWKggreFgKy^r!G@K5BL?HZnJ=YuRw!~2sl z(=;(I)RJpj83ifoKMMs|F6{0QML{)|E3AsgbLyymBu{{@(f0YQk%X6bJrBVHAnWRd z{wky;Cx`M_rbh+UB?9bQ03uK%-Uo?24C~u~7&*y|Hvs%$05ts{MBRensKNoj&cky0 z^y%&_44=mh3slQ{#v}%{Z)O9we!y1Lp47f?*xIrXr)C6gm~9B-+g5W;2SNw?{PW>2hrs%M1%T`gfxc(fps|`-(nwQZ zU!UG_=VsThi1sz0+!>v@fX8M1^U`4Nsz}QB$%p;Ax1F z0K=*e0Jl6L0Kz_aw=V$h6+i47`h7sbkx)^8ZS@2h0%(6x&HVP|s8K2a%E|_KjTld# z7WI4>XOTQe!IL=WDhBKgd9Ilwhh5z2TX>8mg9y<#{#-Jkn?l|5%bZY)k~q&MlR8=I zz$;Cl?Oi+y^VyTll9l?@&e4qaYk9$1;{*Op*Hr%|Ctt+j?d@RsuRtM{Wq&ZfZ^lL) zJDSRd7=}R$W{s8}fq-5BVrz>#t@X`30LLXK#z>dcr-~rjY`Vae!+>`b{j&aOidqSpq{}m>^cV+YBA=u(XlBm zDylkwJitsRPs8yi$`V)sSHNzWO&N|$O@#x3x9K)VUVz6~GBz*ix+nxSoQyNLC$szGGLQZvV z{aG>`b{j(Di44mUQ+ayR@iY!E*S~qnTT3(NPWNZ^e7kvs2K$Ey!?AgYe zreXm^t==ts@n_X@*SAyqZ~y^SHvzDDsgbZKHuB(wasRbEdLWW&0DLP}txXaXB7QGm zl@v}vtBCB9J8)GB*k(~sQZg39yp)THeJ05-4iJk7`cc{pof`vO*@e{AG0QpoH2@&& z+Ier`&P-@It&bCl1#UpPjle6BayTtH*=z#tSWtl=mXZtbCleGGhaTrR`0|Zj4RVl? zzUccBz={47#{GTX7!WIX2<+veH|>=vsO*LZDqg4Wv5I2_8=Hv`Q1B5B~H zSmsU6PE~S<3JVibP>6L6?W$r;{`bO#R*FZ@t;zY)MN~dP6~2w?YEtj*NC$RHBxkp+ z<|%VO+{BVFXeBXK9I9iN;{sWWV@U%Fiz}VMjZGsWW{AHI6h%QmO3Bi2#J8av(4F?G z_BvJi=X{;RyPXwjV-z<+dc)fqqJO?Z+v2!TakYW2LBHpdjV%_5V$w6)k3pnd?lC5V z|KvCS-|6q0AVkQ?x!k_k+T&h*F=cVt2uCjjv&Uo#;Hk6;jT1U$q@Q#AAZpDPVxyF0J93~<#p z!f5B(Zv)9_-|VW(HP3GFuH7m_)b92_(|GODDISmSX(67<`J4h%maIm%+>>c3|C`GI ze_Vir@x5GttGRT_Gjw?lI%K7xt{_j%Pm?wC5a+BA5mRAgZ^fA6VfaQ*LQ~s)9i`Hm z9A?-2Q&C*pFg%eVd^<#yR#qY`()88ELZm4^5hFW~fgqfjl5Vgs~?sdu1(~L2WqtxQ$#Ah%%$O~n+%85Kc*IEh3b*NZ4<-09e1pH>l2-Oa^GLb z_s3FIpC7Hnz`(%D_zZ5Z1-t-+vR(;LHT)}V86l}PTH%q&qy?vQIX{g;e%pHiI!8UL zP4-=fX35YGIH1NvRZ=ec`q>koBkWKoc%Nx@g)ft%guyxRKshPZicQ8+Snl*g#{mnl zlPuJpIKV!Kp@lfwy(OIVC@IO$KT^VUffsbkGDiyAII^;)1}p&bagfIi8tVS9=y}Y~ z`^J@LS)N<855Ts&?!O`xA=p?Z$V?C4^XWHWhjXdbo1;e(@a0!dDydFvjMH>|WE!~T zUA?`%1;-k%!Ti6VZ8AP0Jz(GmncHk0c4#iQepu7lAGo|So5GaC}O+<$N3 z4U{aQ0LtSJ6AXYf;|GY`^nr%!;P7ybBO#8xt$=_)a8wjjIw>V3<>7i8mx7|=RHZH- z-~rjrD1FjKY%>pC^#|%;6bQW+fb7};IE5BFy!ilfIs^u|S($uXEpW1m%gC_CqVTxg zfWQR-9%@k8+1Yes?y7x*`P65@{5{ZV?&nP|z({SEcuZ(0?9AZHcH~c1N~Nf zPN&nuDc=F-^T3}5G_>Hk`d9WETwm9`Y~u!=IChQoAX-< zN%FY2D|2`!qU!dhO^ArnZjLp1M3#m~+{~~39 zPc@+=IJLyXK!m<0X|p%NtQQ*V+sME2Nw0=jRqXETP6Cm+@hqGB3rAP>x8a z|EtdWC**!Zjz-;+a&Q7A?-Ldfg6R; zygX7lhG3%-ifNH9x)M%B^mNGypg{1lf5j>K_rm_q-~2PE(Wv=H$)#cdJgfO?wpfu7 zNEf)TN584Hx-eqNZ~vqTYc=SDt!3}1u>5Zc`M>eEh#vS}vx!OL_Z(m}zApeS+7Q?s zK;w%JpjjQ+8_xRt|GdXcpEQcMckE->H&P@_%#Q<3ALv39+*|@4JK=!iOU{P@Zg4O= zZQ;X%%V%Ii0}X3JU}MR;N9v&ct2p`RTcF5sMzuqY#quZ8X?N~rxSv;7MsZRE1O$ja z`^)?;Dx&cB_g@?cLclY(#>C?b1?OTTCVB(|?iCt!D|DFp> z1izddG?=FC)9mK%5S5a0e0EM5)>n}Md97%;_Kw+QC$z+0Tx=yM?&hg-hDSv70a5;=`m3y~6czbrUHfON9t9v~ik#oqs96o1dhyHhI{m&C~fr{k(L7i%c zR)atw;0*_w3?w8Z;DF62yUivDU{ef~`#!)8my(J~PDu$YC@83kxoq^5N=SIPu!93L zg-U8tQZSIj5dc0Uj8FG}z5AJoK{5ZIAL{}U>>XcHs diff --git a/docs/img/vertex-cut.png b/docs/img/vertex-cut.png deleted file mode 100644 index 0a508dcee99e5ad61f72a34fca24eadf18d41715..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12246 zcmW-nWk3{f7sUx#xy2#6eZ($Z=Q($X|) z?k?7Lj#dZ=OvDzZrrfR`ZthkDrlw=ltjrkh-Wt)-5DnAbzP|pxX__h1Dbu`6L&F_X zyq!UWSH5??l{DnUUphzn&9(JHUOpV^H##EyaP4vM8lY`mMNJz=HsA_#waHN=kOhg5NhDIXx5IX#4E&M3LU zP2;6tk1}Qq`t=w(Q9N}8AwYSHjEsyjn)X+=l)6yWq)o|@f0rp8g)&Ek z49AfSYO0E%4DIm7QH%Kz-CzAiW>n1&7dh`rTdfvZ+ui4ouO@DP#cs~jwxKY3T-@GF6#}CMZ8XSIP*Tw2yvOQ>eP@1c^@q@2O2e)BJ$eJhrevP4r0$0 zNu%~`i@R)fWqQI%&deovIy!H4dRp>CyxmEwb!y28b^3So^z<$UFgg0Pb&ZB?KF^P| zIjz60(yhn23WK#c=+YeIt87^SC;sDA1BEV>s}6p%7!?$?nn;U5UydPUyzPvq5Ev+w z47&}y@BX(nnU(X=hWSM`=$=rmy+}H0q(-jS6&E=hwmI1Hhb<;68=-WN_1k=X@AHHS zn^8xa$;wt=ribw-;myGBWMY7CO(tyl)SVO+MoDFh=5R~D61Ul zty}_cPt4;^P1DRzy}P@+DBA7%>HaxXV~8Gwm7G=ardi$x+!;)g#H~`2Yiq(=351}b>JX=t?KyW-v!kV5yY%I zeQPZa?KgrRn~Ji(c}cJhxtZ*?d&CR_?+(=3-4BK`ZxmwguaE!sr0yz>CQ!ynDBt2U zQz5J7i)N9$ZPRSVwh>6CGQInA(BQf&6~^BC_y<#FzEq=BF_+ykiUUsI`)7t7E95r8 zP@&XaDpl3%<>?>K8=`%Y*+0~EcB{m?hW!l6*+Z@(QktPLDG9aYzIY<0$%C6Aw|$;K<0)kLo)SbGiY<`&3MS zRopA(Hs{S-|9kc58t51*J{kNfv33iL%I0$z88#b^B_c0-?`kTBSKZ5Mv+ad-N#g9og=q{C7vh4q)!|S+zsn z+W`VK;;;l=T~U>s#l#G24_vO`45J#y)kZ~ZCi1btNbJ9OtcN*e{T!qm9@xql0&DFJ zITyj0(iA>WXISL2Zi;DWE|Lgzuae~eM2Kx|PR8 zC>BqzgQKO^>NSmcZEL-7CP5MozbfJ|a5WgT*q3&paoo(KOtrVSLm-e81=2xfyNn?8UxW*(96TN0ewbBs68h5VMPFeZXJ zc|uKM&6I!Xc~Vj08X-L0z1xiuD3u5} zYyY(av=i(wW@DPDhEO*ucxI)m4t{wicm39vzsgpoZWu`tT=&g(3%LVvtk+YST+p$0 zdtLH`d9*X9!R_apHaeG|citQtg5bz)_@6ER^W+i=sKE0S4iQw72Ju6wbQI2|;J)!7 zgt231$-spV)OK+Vb`|P}7tOGAomFSuj8C5a+YWLmt8kfk6$wkE{(xf=cvtFz$AMtf zXc_bUdS5Z$l4&mV;)lv^nv~dR663NZLHiED7=&4Htt3%s*`VV7M(mX?q2UvN| z*P^KLU;4oA-tHnmZo0pXJsl}|>r~!lY2(p5qK13zq2zPnkAVeUnA#te>G|Cyzkfx5 zTjkp8vnYO+j?9e8(Q|7~A6ujN(?NTGol6Q`>VH=C# za}N<2BpUO5zWWt|MDM)bI-LZGzPo&EXEHxNl6`SRB)?2H*-d~)UXqf6M?cSj%O5@V zP&SA-;(4Fx)Hir1pv_j3ual>{*JH;j!DYiaJ|Q&kbq?&^IN)?mVIgK7*`<$}8<7-j zi#x+4CcJ7t4%QsZ_IFTsrb3mXEFCXu z?x{2U1tWx>NBxm!jg`cIkEk=3$PJrj5U*&dtE9eW!}(HA74lA%lFn-%><=1Y&;8>WO*f^D`bxBztib-!ysmq zNLUOL(Z!rsC{280iw6eD0 zQf@yl4Ld@jXf|HO!eDLR>up21Mi7j66(Y!HQwQa!16~1=Lf8~xxY)_cQp)dcB1yeS zF_SIqN+PQJ?#!kN}qA#C;oY}wc zAdO1A{bx+nFZ*s|G|XzLMla#oC_NU54+@Ra{tona;@i8ASW-W#wC^gv4ZYiB-EF2* zkx)@e3w*kdkt?llvHuZTUEiZ`M|z6z72GT6C9ZW%0r?DSZq?chPhOfCI9XIOJoBxg z&fV%kS2SFpXytdrCgsnHyJ(Js^#%n2-(LHH3!kz~U8wud1Z(hFvla+OIeH6hr-={a<2tt0cOdsfKK4qaB{6ZH|8T+Av1bB zNhxsull#+ZprUTWR<iZqS#xv8S;z5)J-5Bc?H(o|%l0~X)iGZ& zFI1b53ar1A&K^UfmzUPHCgR?alnZnATJ|uo`IGqbV^12T-UTAmzipvXvz)Xe%PRsW z`KnH{Ji(nHds zc486gM%qAtBA2kb7f>QKvT?8Lu&RI?XBNv$cxi~G{Nm3m^5@cHc2@>JdzU>%tE2Eq ze6UveU6{cHg9IRKXwzhTn}f~y>=#`{v$qwamlr7E+zEu}2pqkS-1HPX z$!Siw;o=RKsr12Ia|nZoBWi?GEZS8>-e5_Ru^PQ5Yb;7O z?yF9nAx8qf3A1g@DKFxM{87{kvVhUrQ!!**r=3BS))qiie{42xvc{G!_zPM35u?&a z%we2cG^iA5gGNdF{eBC_~%ah8EHf!4%w4FpOmW8wK2W;XpMcWkl)bwH>>F%m^G6(2aHyG1`0R%) z22#F$9sMnZ$P&XYBqWr}?m?b}!&7PNNz%l_-h&3|v?+H3{}h;6Yptdf@p`|LTIFmPd z+9e(96Ua)s8OJA#;zQpZDbLi1EDM1t-~cnkWItF{#8eLp=ud~pzbzEQ!l zyro^Qnyme}!0u<2aav|Lp#!a%3&^{SN_e_*#CI5ClW zbPH_1+xznDFA+vLdeo*5`z_XK>hMtY*u+9eB7OPVsaD6Vx=Wj_vle_MC(BuAb4ACL zoGPh*t0qlepZ5tv_RTdv)RIgnFQMW`LWWo|#*-}}Fs3=t%^cZO#at9Y&LX|D`KMn?~I81m1N{eN!#tcu6{cf=sOL+akDvA(w6bD z0MAZDUTqMx-kaaNWoyhCPU~{dh8^tk+fP?TUxdP}C?;)_#f~^##WZ<3j+s!^B*SgW zRVDRR+5z*inf9}iZ*sY%5$v`? z-uJqFn|5;em>9+@?~x)T)JQt z(*6)GwW6tbYY!{=OA(q>2IZgDjdXHW(&X90EoP9yv<_VWqRfNKXs5^HKlG3DnMnGr z#wPjem6vcz&{l5ypNknsk@cP^@RfaNjYsSTx5W#$Q%@`oP#fHEosi9ceR;QlNwA8N zswt*FnUBSP5LiuS#2)Ch%&v*ak>)mTyU~)d$!{lir^eI4Xe+Q>@t&>A^vV)qMb~n& z#?dL1?We8C)1jLwuv&+Ozts@i)E-5pY*}yA**Z@3+uYa`AQ>c|+UClz@OyUp0&zm} z#5~v<4QTSEWci@1GLTP&f@}kru+15AqqviA6(wFh)*datEhK)GpqZTD>Pnm)7}0j_ z%7UM)otwv;K}z-+ZkE(fK*@wgV3G_}X;+(`uC+7(BC_!I2S6Hv`+xgB+Q%xo1Bz67 zp-g%A&m>0_IG)_jo8vIF>LhZq0bRw9)c`Rn66Td8K6^(z3ENmyH7 z$*-+(j?jH8CIX}`$9w%|vw0o~-yKUmkC(myA}Q@ZTI3--@P`^yXA=g(Hs22TmsfkEdHohLW53hk z@p~SNY)xW#CQoD)m_+sVM&jeSVYaDxuk(zx08;a`3QREZ!o$kIbwY$;oRLl&-_^V3 zmjy`$eJ*Gyc&xEHujZAoA7!BmHC?&6Zz918j(~*nfnR*g&G^|HdRrgY^K_WwLW)nB zcfO7!svlL?en=kK>n=;NASBD#0PJ3oMR`~oP&UzrgRdt4O-<8{O-=ESwmxNDDZl&p z`Cr{g2bg}{2iy3tELUl>@_#=3xb@t5-b10ovDQcuqGS_pVY=ND@P+QZc9jCWntvlR`m#`GB%;&ms0R%sy3~`Dl(Z+LQ8`B!*c(l>IhaW2aba2YafO>GH)4;dKB zd-Ydi$i$kB*&3sg3&#fk!(sqIvW#Od^5OSaRkCSn@|OZLa$ysr)FE;R3puVd&_yqj zMUZBS`SVXsPq)aXPAWdmz^%AzxbA_8WJ)Dpv~nB|1=%F%iBCEQ)qoe@Vrzz&c>Sa= z1X-O1V>WgYo+x&cc^n+&m9dnk{}GFEB^;eF<$kxjPKy@mVvF#8I+7sW1F51N7nNnW zz;!ex{k`5ik8wLd2+5eu_ewJ%&-_9aFlCc{QD;S09@BfMxB&f_KBEtGWR@mSW%tqDPCYpEp%M=m)#`Oj|G70 zW)hd?^cn46*Zqlr^uR?QAS#~bGFe)T=<5w;VOpgI^rr-3dp-VB)H`&*5WelyEL#W_|fr z@-n6zTgx?isg&Yh1nn6^^H%}L`OYp>Cz07fkO$x;@9db)Q8Eyj4Ow;1=Lyu zGN-x~WR0oKZjE(cjs2Lhy!NihhhK`3aakC7adra7wIX2(EAZwO00`#Q5V)S_mWSEo zR^L>X=eyT&4?9Q^x@Pj&B>YrED)hkl&htkkcj%_%6HOfNl=gGtN&BAD+wdpI-D8o} zWJbcmUYuV9&pqWqZYdubq(n+;8zEsq=JmF6zUV3qdI9V=35M*y?RJx05iL@rGe%ad zBA!AR7g*T*$b*3A9ud{XBQ!Z-4kVU5L~(SjbNw7SAI{UUO*-mwu||fcS@!YUU%d?% zcNh^n<_nV$bBAU?r34|b*rE9Airzr^3K1@b3y<9O_AsOm2<)bh+b_?k0Nf-@4C!M3 z)X;Ys7_*wo;^haHBm5}g)V#-8tyP4TJ$6E!o%mUfWDzWkAJr9g%ba`*bk%|fbW5cM zst>1k9Q@id^$}wtknNO0g?oUiJ%U7{mg`+}B>C;u z?4tUArzP5#2$8-H;07zteD;Y~X#iz0R5Tpc737x1oZP=s<{lr-p(gBT{PT9D%rrwb z^+*rl`ZLDzFB64NcVnUJMDz3{bT7`$Q`ZIryyG$=_PzZxyH7{e^`>jhc0J)@_;b_& zSL(%dty^u(h!?Z=2y(9cy=nvJB2TxVbOzuJ>SoLJe@fmt|JY3ZkNma4X)2@g{ld5? zD4cu7F1|B3xa;NSWK~o2s2Yis21=&yRT8B)%Y={#lB5D*fjN?vBkT0Uu<#OX@;#UI zjwG#-=&r&sY*C$yOI~76)><&fI64<9uUro6Ra|P(1;<|mJF;IVhIACKTVFtp(J0dq zhHGbKOtyOH(Wo|Gc~tK!tq>x{CwxGl6rpFazU$6}S#b&bF{U@N*|tLo#0#zHP&6+L zq)2o{)fD{zj<)7N%6qGisPqj;Wl^X8~59s_1Tkb2%4^Wina|B#);8o+ArYF1!by25Z z4&z*^v?`GR$ltoxAl^h#IBP8$917y$)=09Gy8UDE@@ncgDL3dsumr0^sXBECA9BY~ z8%WG@HdhpeVVW5k63PQ`!VwSO53jE%l+i|!ti{v40k)~@Bll0XCv+L7LFFGlGV8z^ z2*HPnl3jL?tejAG?A@tzQtAS4ay)Aa2JDxJOC*n9fp~yr+i%+^Qs^}7UaJNIA|4U_ zjK!-l1jGLkOG{D5-B^0bNBTUy*=8CmhA@fKRmnrHy`17Xn@yqXc~W{!)Mxu;2DA$0=aTFGiV-I z#-g%;Zu=g*%ne2%2wmvL2As*vfr=fJu#(@0vg*uuWXur#VALj{qV-xF?RlTy`MWG< zK;GOZq;w;JQiNq1E||kej3r+4OWe0GNlOJQGT~j2%~dY%FH*|3ypJe%f6+UXL+$`! zA|ge|2q&+_`I7;u8ymPM7=d5SSCY|_Ba9ywd1{beR9;9VbF=E!WA(Rnf+DR^2j!dyAWEz=6~ZWgv?77=e&+?LsEtTG_lkz*u_IOjrVs z(P9}oQNZXDbl*qo1Po6{*gikjiiBBaN1CFFN4XwycTpKa>$GR`(s8%{zgAg7#^Hy{ zz0pw=K?dl8E?~-;+Uo5oFwRY-7Zucf?<@bQWbUO;VL(aml5bQ2@h__i0FO#jVi`G= z)2;2@WMw|(`=5RLNLbAa(aBHyb>8(?#qbtxWe6@u^4ZD1hQkJbsLb(G55HPw76!>f zg5X6_?_riYr&3^Qjd3by>i4?RYWNjiJHSYbt2=c{81*_lDs9Ej_-LVV^(MXt#F0@e zY}j?Sy%LvyM`(=S^SQ+hSO)x5r3tkKh)Q_B?HjCVZZftn8(!=^?dbN>c?H_=#E!S!}PO!!F+wQ_OUY*hI*^@g8alRZv$~BNo z`VY|N7M7}#>)T!_Y+w>>vqzrDz%D(&Q2yuucUY|GzqlgvKdx;f1-hj6?`j1sJZaqu zwSH4kQ7Urq7s$oH9ET`0z?)VoWA-)=X7H6s7^*qun1W~P``XFn4b+|d6owo1dRLi% zwS-j9Gi6vGDDF>SFql*vsvKTI_51#AW|(XNO6e(&p}AHnN|Trf%APi^-Ug1mp(1k>9{ z_>(ahkn4^jKj6I)$K1dv=56IxXr%%@uAy>BzNtK@INxW`isQx z&9{Y9f$cAZQj{MH(Qv;zHfg3mk5~cLxHs9ZS56a4V^r(s+c1R&Z+I<2mY&+f`u@*% z&|)0x>FnX^Wx6j+89#b{QCTm7)TQOrZ+T$CW4p9Gp_8mu0Hu6yB6t>&UQ`8Ul#z;L| zgpzGiiW)c%14nqXwX3NtP3Djq?L{bTRO2p?29HdxZ?Z#q|IBlaun0HP#%n`-HH~NC zTRURb)he+kToi5>vb^~DSau#7m_RZ@H;yU^V)n7QZYPFhWxkBNWN%6TZuA+s{i`u? z?WcL)wIl{RBE?#XS-1wwq4tkF+XFDnv5}3)IMnXUAu=>=-29!D_f58yezTC-bw-6(z-&ezYI^I?5#Yc;<|>+2;Wb}eifK2JSQW$U>yV# zaQdFfkIDMCfY7y*CeK1^Q){#Kz#L1_=FJ{6uGV3b%O=!u)5=N(@q7zjp!@QE$|t9u zY}4Q|>^-nLlbNuEd39P{u%%KbuyW+~sa2nN{50T7&lY-goxvYDg}K_bDSXwVepJ8S ztu_A`=-b`RKt0N35X}qe|74t@S~gts*hYfFm@t?X&>3<-9dYuPoPp(R#otMOkCcV7 zQl6*dt!=Y0dd!KJ*vKcB=^PF^v3UrE0M7lt^K)!Y=B+Agk3^6}iP;oDL{U|kcceN! zZl45ph|Y?B@jBwXt2?5)pGg1jzhPt(@)s||(bf0AAZF>`$lE5)6yTtaiXqF9Ohtb= zm}puW=nOp!kGBP@M$_3~jx9CHNwKE6amQQHtz!i&0o2UV#~ZkFtBpzEke3{1hoeQ- zn3r@x%WH8WimeR?us8m>bL>g6!GP%qwgnl?x*51p1QR5LWUVFZA3)(sM{k3?9WtSZJ zNYdCd5dQgYoxsxW-&(aC$=f=Yu_y;ry_Ct%M6NgAxN=fWy!3!Npb`3P`T_7VVhZ#p zYmAjDsY#sV3FFQG{Aq@>~e`z z5de>%QHW{2Cpzv1i2NKd&}(vS6HA7Pm^Kip3P!I0ZYpUcsX35mvWfOo3IaTRQ=ny9 z+(Pjp$DKDj$gX!JECmGzw>mGA1P<+5ztJ|Z=`OiLg$$hVwoqocgjv@ctuX5uKYS?Y z27IEEY==6tu_-W_IzNm!D0&qLdrb4YY&l}q6$ZSLJWg7Dek1-R(@aYvlBfAVlE@)x z*Dtr$usuaNi~C)TifEwy>-Tb#rV%J2~ zYDJ;RB$(UQe=f1akAGB?7J(0trg7uMU@aSYQ*HnWfCd29 zzd?4N2f{u+!$2gViTG)exdKD&SVaNFBFiV>Xyx169W1JP`T6rChY+IR>*M99yT88} zjS99|*P~PwxI)0a}gv~oAz(6pM-%!6@x|~zth@c z19=+uuY588Ld|lO3jj1sI4joPBfl#cf=`8_V3INq;r-2j+yuDk36M?UDpcA$0fzgz z45US7;S{DI;49K-BzBhJ1pzl?()eqHAY96%fcvXU$J)A|^5A2MjP&#aA)_-uViCjk z1V4sYUr`4=CZ{uNj{NPXCw)$qqjJj)zGT^$Z>M@2{{Snc9%enf?{w+88_`5pnv*uP z^XG*#H$W0bh_LcFSO)ja)uuDK2GhAb8D5x{zb}a2C!~+j)q1QuwIQ}kW#on1klD+3g#q;3PVkO&wW)2uLwZc)IM^rGN-{7lqTcbty?*O@ir1*PJp9kFZ73)6JQs* z>mE(tB)IPjULI{`TbrlwaFs>O03>A{yT<<3P+lFy*#9dnsoGRHDwR*LbGkCXuTU4M znJ`VbgHaM-ziR|er>pYkrZicii+DOPiXW!3co3)6N^e$UCSc-QG4^TIZ(Skc=0j)) zNpG+V+duzO{6UKbR%Ok=v|AN|Nx1UOa(HBk2MEtq4i}b6a0yc-1t)%ZUJ>9;X*Xtb z^Hi-6x8T$AJtdsI{AM-?lJE!3?s6g{uyMg}9p^L(N%P)_;T1imYz00z*%g)*Xk#=U z^F^8~b0jZCuEr`b?w0Qh_)PEct3l4%1~CAAzn4HXg*RcJ1^v4fhHOy~@+jL>8+hd7 zJD+pQTx-<%y488}h%t$9fSeR8mt*y*!^3U>r&`B!cOXJTLYUhdV?Ul8BkDy&Qt`=} zwKPnu(VKK?z`S&$ST?SPNVmt)l=KPcNW1I>X`?=`g7@ERfC(l9>-o+AxQZ+~_ZAwb zahxVgQYkT7Dar{^`I$kT`u;l#)Sv8gJe;;VsL}3J_q)aJJKzwx_Xh?&SIjW7dLaLE zt;%2f1laCffb9-LHT*BUph{(JMQ;J-Fj4_ummR15_^^Pm@*ci?NmShlIPJSV%sO(~ z(~MT>3o%9O_FgAU^-ITm9gt_C_ErYpPkFaFm>tGe#PJ!`fwcxhEMJP@J`VZ*ui!Ym zfWqhx>&7=%LQ+__*Xi1~`wEX9R&{N$pq5l6KWgL(H%0j5`0|bx`fjyEg$wQ_EvK^#>qEh>GiSLb4Jj{}rgdNU=k=ACH`c zOzZ|g!1M{hex5Uf-H6=z`)*{(u^SX=t*n%~?EiI&j~YfV;n#7$Ip8WWkBnKx`|0VY zS^9=LsuONrA~LJW&XdLskQdWsmq4If4`vn+_*%4p-dni9CFp;1Vz)y>L|wJr z0RAaRyGs74GV$e&-orm2ScVi6S5-Dj9rOlBRMxTh%MK&2T1B~JL~h6s zS6@}{(Z2`KZ%0bgs4R(5xQ&X6@T#`Sk88j#XIA*SRBO-#P=iC#6#Zu|7X-o6OvN{XwS-^)e)V(PPQCr8t(Ht8;B$*#z;nhNgZc$ zg^~;s^|>#d-nC!6AT&#?{D?R4aPAR7+tp^A3U0wO%s; z78APO=3-+T$YQ0o`|ab+^TRo+bK!Sb(R7YL)7fUHQjrqC|5r~|n>OK*DNZHyP)tRj z14r&6g22ZumNqFP@f81$@nPxER4wb~klq5%Sq~V2NPi2Ylwz*3easgXn)Bbcp34PC z;AJuAxptzKDd&K`KcW2Be!IH5!fy*dF9}BP#In}yL}_O8J7r%`81<2ZqH%}>{|@4Q z2nX^%2ry>#;?Fqyyv0D=I+q=os}-*l6uXuoRFa3Gp-2+Y*HmVffKvJ5n(XrDW%xBxrFf?(P~01b3IVP~6?U6faVwxXaJI@80*m zKVxK!BqL{>z2vO5_MCG^s;RukLMKCqgM-6TkeAVbgF_mFgF|FOMR@ropPRQ24i28h zT3T97L0Xzx&DjBDZD$Dw$DC+xV#4d_>f~&RXJRrq#>Rr~?4c165u;(!(b?J6IYvEd zGHQ~YVPLRMg0tQOhvaqZRfb8j{(}!i4#!uI*n+57#HcgXKursw@ENX7UD3vg7ZI@l zJ|WxCNCS=l3`Mkvrnd}UB9}PKPoM@<-=ijsrDK~K!`CHKA2Lz9<4)pBToEVSxpHSk zBHCexa3Gqc7itF_S~`w%m9h^5Cz#c_dAYgO9DzfsiXTv`**O))IJ8ejmBzVP2e}m4 z6_nx2L%+SoM4~~Lq$xJFvbN4s$nQWyP!o~Rg&}rrj4->S(1gJa70FtvG14XTn2L4B z2;pI3J}Hrs=sr>R0zZ?I5bbeupQy@4l#%jsqwWE?53>;AdTP`h2N_oMpDaj`Op{5Kx^Sr{O3%ga&qzg*+IvnTk7nw{j!;V)AzzE zdriA(ysnJg46HH6B^jU$ULSR4KxqUzYi>|I>R>z82poVNag>{_hnJdFfgU&Y+3QdH z_0Cw!8_FC&}$Io0M}W^5{BWOg&=i4mbk;cNZP&CRyDwMhCq zR&~#j{eS)aaT0bOoIq7SH1uE9(-Iw?+WVEZoiwW>0Xi0ZWpg(s9bVbYk%Zn z*VR3{*@_q7VbTdqnQTPzMA1+F60R`HBKH>AUm_8B*JU!nnbeKrFyScSp&VM|) zAN8U9?|N`+^G*0%K_HNY1=1iJ9(c1va2|%&3HiF__s(u5loa6{2^3u;$*8Mj0tozt z^bMi1l+X4}FUBif{|l*_g`O1MI*zfRFn6oJOrTS3ev8)eR(Iv%k|3Z3C!V)#192x9AAF9R=fz_o_No zihmAtELJHACPg@z5EsOsdXM8C+6%Eg>4u-DLhFXSgb&8V$Y{cVpq|bD7tyPNqC)PgUEl_QYW@eJzG$9RV=Uwys6Z`3I$hs$@sgBs=zUr%a z2cnNO8Z%TIzzlMe;Q-q-Dn~{GgpJYqdfwXnH$Eqlt&Rbg;B!dF{m(z^$T)|7j@xpA z_?)7`!o&TLC!`;NcYU3AN(TNYV10I?8?JuH-;cIzlpy{IV#cK%G!nL{>(38Eb3wc+ zxrGM@Wg~rxax|=kMny*>9t%Vhi_sT}7dp!dYF5m3Pm`cG?7xDdWuyfId>?HC?gt*< z9Z~gVAw$tremWv;0spQWwCYea=I)O^j9>7GG@enxSJ&6~q~M@mz5Ni+4Y-4e9l&Ee zU<}kya|=vtn{ImF1*v_)knj(i=>4ZCAmRTLR~p>E9P0RtIsS0u;E&O~j~ETuKrL5j z1}>!QIoO8?g}PKvI_0@6Z7PWCJ|pLf|6?5kvlF9Vl=N=cEU>pv8jrv3ML*EIJdd#IU+@1M96)ETq z*o@7B=YUJ{?@O#MjXm!HqBP#VP8^l~IQ#KVFID*^xM*VkiK)Wv*k&kVWryn(Sa=8E zdlYgwgrehsY}0+GB;)^2evbrg0$50GW3fVCt%%B&Dornto6`3aXcia6-j0 z+(`-vWB)NLyd(d7T776qLo;=sDf@dZwT*?I-qrX#f~&F=`HT3A$w*C4(=mwK3X_Ja zC@0HQ(1F>r!b)->82}3NI<KL0;_{dO#)e%+?0lNR%~}KvlK-O^D(?FMIcu=L&QMeA zb5vG}qM<}34$0TDG8U~FE>6|oiaiP>M&&uJB7KRcLkdjVgYxH}g4kEpnKYhvepaAD z^-8Wo2^qQsCH#uHfK4oM=~&$YmgS5^G94K{@6PVTnt~j?p~EW{ZmZS)_WkduprdcfX6DAn?j0% zQF@M3OIe#Wkv`=nBrlR2v!pmeUX`s;T#rheSAGqxEj)rV#M&t`jvzQJ@; zUG0?KN6=%fqr}Fj<#*+W$I`Ey(gz$(e#OQ%9=g}RIPvb}$$WNb7;UQm;Pu>~5bFg$ zn*G(){x#TS8z3u&T;e=b+mJ_&&KR20TdYXH!c;W0-B9Am2H$YJB)~|?Ci50HWE+~1 z>7MKZ%9f|3IGnh6SS`v~=)J~x)hd|$^EH`&Mj+i&k+I;@babFtQt?_PQ-Hy&$!7_& z`_Gc_2UHUOd&(mKePYNg_#)dm8kEn;O@L&~x8yA!?9XSRHXOq%jlbWpx`S`ZZ&Fs? z;P0}CEv01Pb+{QP$D!6adlj;|*s^8sSwi;)MDN6%Ccgx8L4q=hygFF}`1|1Yb*LReg^ac=uJR-v)QQlFx}FjKx<~ z9W)%a_aTLIDvLK-f8gd3l(pVe-SCz{EKS>2@1mHps7iQ1Q1hh!56ysAFCUvCnGcAX z7#n&DX+#uimYsoJC>^wa*hU{;+{}wFBz+}6TdX<%HO6i?bgV8Xz^C#VVe-Eb_7R*( zEy?|C%Uoag<0fab-1DrXZYg^n(69eB>^$rUQ<=pwoPhU;@h7gw+j!~LWy_WRXW#92 z*t7w7Jb`-_DfS=#I5{^fxUtO9Uo=-M7_nK@Q|D-yzVb@LXA9|uT+e3OZ4!Psj<#g$ zRCKLZ1^*HF&UY`^b{%gu_Nl$eD#pW~(ffgN)zk489~!$a&veKp+>#v}DEmh+MNiM| zBZ%ppQp1+19*wKs+vabzto&kM>u1D7jtX1cuU#fA(^$POVOk%KBi1qvD}q4Vcj$GU zc*;iCIanV({;akI8z7cxC1$v2B!4$Qh#SS&x`@RA_YHuPG<6KKFqB?v%`bT8dsrCb zbWbpTu$8=Or_@%+({}z+JX_9$C`F|Jo4BASsrgEg)RsC+aKPpC-;}|KnrQS#zp`+U zFjf0}m=jdIOsUl(kMDbIaezX0Y`17QSvCy;wu~!o`zhsr!ys#UPW0Wdm>gX(=+*f& zR;^VojG@6?dL?(1D_5X?0KDVG3+4Q=Tn5;m4cP;lW5NdDzAqANI@8g8{Ue3n@@q5ET|N5`{gH}i)XO!D;^nHic^x$%wI)N9Kt;nN<7&gbl)XFq z+Hg-XWAc%2EX#Etr^{!W=b85!v52maYOaRWyO6`n&}6{>t@B6>u>EdU_P(E?c2!y~ zPl>TUpsnZ(=~QM5#ldpgUl~XETIrFCq-;ifM-9_CBXJ&a08m9Wi}%cYgEMwV1!)G4xJUB&v9Uqi?~w>yoCarWmZ=_!Il3FQ1i#~Yx= zDM1mX!)%IzWa#KP4>oMy6B_zyr7o1-No?lz1#&1r8WtRiDavkN>PF4V@QwbD323i8 zMPEfcnH^8e{r#s3MSEVz>~OK&@K}~0+B{J&eM23Vq=+0>@g8~(UG<59BynXMp9>#?kr8cAZYW!#?97}lh0Nv?L)aK-+KX4 z>O1CdSKzy#kn~KGM?~g-3*k%UpP89C!Ib7ZH_Ieg-C{U34M8KT^LNc;kiH#!E6Pq) zv}~)CVA#&n0V{#!N!6z(6ryu@wI_B#Sek@sC>OPx5@-64o%wgmqwTof8)znqSfWK1rP-E)+1-AZm~^Gn=VgW2M{h$vJ7s zU~{ehqiqM*+oH7Z6&d+6f0o87TWd+qtQskLPh#nmAy7*UhCZ);{`lGf$=G~KjvxAD z5;Ml=g6Exqh%?>Yk8aM4aUH&KKYF6oy^iZ~x?c&ivt!PUVtiH=@U#PGQW#-G$|hoT zkdK%(ynoDnE@kIWIr#~?Aee_GPA1gG)~5R|C&qyAMSSdLG6atX^aaAgLgc_?KPml7 zR26v4e_cKNyG2D}7(JN9U1zrtcLlqf_G-6Uf%7budNUs`5O9@5vFUeXJh z#Fa9@9LHoj(${ff+;w&)WvIGvTS{-(xVO1h-!s*8XE~ZBtNYq7%QlP*9g;xB9>-L* zCLr^F8EU)-sKX~LBxDsWE1>O`a@%K8zx$JIdUe$`Iw94ZJ`vDP+Iv&R>2fP6GjnsA z`XoY7zs{m6$_I6k^Vszx|H{Zr&_|`YJwb1>MGQWM!T6Z{@V0i7nP)4k^Vr(9P|Mp# zqVL`mvVN~BuA1t9;%r9GuAJ^O>BLEG-9Ns2I&=wCNE3vyfN*3sqO4nT7|DU>uT#qS z8G6e?)B#;QVRjIz&vnZuI069>`H zt!0Xgahwa_kpb-xWkZ4CldgKq8as^8OuZ@XST!mh`!z>i_6^Qo&7{nuD+*KXzh@dWOeDt z7j5CkNiSkLLD=`$C-x3KSDcqoW;M&U&_&JE7 z7c)*!M4F2=^aw6(*$mQ19Bt|9tU*AOPehldj^W-u51YlZ9&G*gwv3Iz|1v#QDwjpk zAEO*X?({%UtHab|t4wJAKc0-rdI2<@298U(Y~rX6|K#A_#2xFlZXN<6^cH z$Y+~0*eN!}63P&Yj4XS5D>K#_AS^E4k(-U2oRH$R|`b|7v-V>X$ zh!3O7mTZyiMYb>rm`Pj9s1HYcId$2kt^q-Ub>Vpi7-chK)am6%?hrRzO#Cj z9}l#{dsikW0<%7EZny-WLhMRMq2r|CG|r}>?QegjW^0*m$EBhSQoo(6#rk4nz4hd| z>D~PcnC9^gZY&fKez6k~Wou2Bk~HC_VhmI)_H2R9*vJ*!zh2pO_0g}PEZecIlkk; zk)2GTxAyeUL}CAuCYu_qU&D)Wp2}Xuq%x{`J0qq-Z5k5lLF;En0qrFADx`$1wRw?7d^T8CyPw zClnV5H{gDRVR*m=Lo|r=A&0V62-Gt zhPB)`H^30|?oB1Z#~6O}Uii)EnI323#uiEE8z5r}=gPNTq9mkSp!m%Dm`8g8D@~=o zXG85m&?`C`Y6#+8ern+>X@MQp`e4lWhg4hf(arHW^xBVQo(;;3uLNg6zoa@H=KYGy zHS`E5hZ{;fwl?}Zr#+^(0iy8J)o12X7T<@=eWLv$7JZtiE<9SKb~kI%)FURgJF34u zLUL9LDmq#?$NrnZljo3pSWt2^`<!c%rqY48slv7wwkga@Gt3%jr@Tqc?Pak3}W8 zMIZmV%s}3_TFrE02G&9`=U4ASSLA0`z*nBf@-A&TI9-X+%86^0y&0Uk*iDwbG&I#A zE}rA>&&bup$6AtD!>57KW6RS*Sl4 zdqkBrg}1Bhwk~CtRbYzrL6_{iv9l}jVw>TWIu__KFcTkn3 z<#Jw{-|@ae_yy0~y5Ooy=LkFHywi_mndSE;h8_U&DpRH0Q@$#6U zq>15~4+EMl_;-@Rn!X_TYM7F{40tlexQjP=O3Gr&jWFyHc>8JSD7cM(7f%j;P1&1e zxdr^P`0f|~F~fpFn(QC0O7-MqHQI}8*9Va3Su6qWh5X7J@y}jCQi^8W9PHJ7!co$c zpBlt9)$2VV1@naq0$zq*V*VKilY7493cK69!7_n9AtiaBb7-$7sW$s#D1Ul%#QJw=YUAiJ z#a0~xhjZ9(J3M?I*f4fD^ZRA=skG^NB~Pj0)i#Awk$&JFHXV6&vH@P9eN1ee$qz-e zTYdr}us`Fpe|?91Z|pk{rxdeRz31uotsmD9nf!4Hz4<9t8qpf=x$Vco9jZR)F~zFr zHAh}?#dB%ZT~NKFP!_VMs@?}rmavNt-9eVlAk4KV^~%U8!FPo!+H`&y5^+Lz2*&0=2)eAwxKKa-sF zD=3;a3fG0s&%wULrqnp?J6-0d4`?Meu|%_8V1Cgb>Gs1liTFRIwhFC;E8V9Pz&^!w zVMuiwlg-^S`P%0GSeJ-onYo~iQQu1AxszCL)2_rFsfPXMkT)O^O{<-?-vaNJhI3CA zQ+&L{o!HKz3_y$STIFr^8Jnpt|ALT`L*SN20xi5U#2!{4B?^0&22Z>1kL{eYbHAosoR(s2 zA{gp%vUZao%g(^i^#JiJj?Z|6=nq`};FRiM!aRxNort0U%YU!=!X=x!`QA?xxP1J((J* z(VqT<&XCIdZGo3hy(6rm{yp*ohz)PDs*=@=qbIK~{c+aenlV&BOta>H^U4b%cnu38 zDp0h?yX?(=Tb43BKCaR%6{vDL%A2KCktQI@#Q&Nik0?p5BN$r+HTPAmbtHoS#eXxX#pcm)}d;TiqH|^4=}RnSF^OvQO%T!J1FTR54rv2 z*V*GAy?W9KAFX=H+ZU{qLy{wX8jx>BeYaKAiJ&Cqs9V<*G4jJzq!<*wOxX_*_R3F) z%f389*$!`tp)Yi>C;I77y$5bM!-tdK;x;2c;uAHoACBkT7dWWPya{&DSGAK`g_qe( zYrv*HBt73+8n-%^=mTM{dX;lS^Zex6Wtkoi`y1$7Q)e4M%3sd)WdU5J^(|JUvev(Q znRJ~CI|j~Q{fjO72x&?f`a5a# zp@z0ehX!R@d9f)u2d{HlFb~jq%mP>pkylu=4C6&bh;mQU8Daasw+TKuJkEb7!_+&l z>$HxL71R^lJlgjid^=)jx>mv~kcO!jy5~gKWKSa^TfLWpJB^MUQu;2n=w0c_BB7iI zZK)8Hn_my8dk@f=i$RYSZ)z}qB|Ja`=FJr?%5Q2&ZNWH`i_mfQqIoy@mFvv3McDse z-c51>j4~O8e)vKbyrxa`Z1CLZ6u*0&O0Ooi9U3b~ZvL2L=~~LoklBwJIAtt+3KE5o zti}tFAB(KA%j$&v;S-A6lsktM>)TF1udK|u+w#8p+ge4SqE+DF*UZtt6zmcwuhMrw z{R490p*%WGC|LQoK>oAV-jN#G1vu<+En<5@HzqW`)yH2R-cKg~G7)%S#oqrcii`3u zGGd+TQ62NgO*c)TiyNv|;p9f`LuE(E4G_00!6ySeZOU!2`Wm?pN3MOQ_0FXQ!6XVH zvdEEFY3h4Ka>DoWGG0uE?I8{X7Uo<>v{^p~mrA`yab z?u|boGO06mV#!SYjC57^+KIolmj~dbZiL z8uRx~`2!c4Ul$o+MSf~RlJjTjgLi(4#V3|oC_qL%6-+)0#){QJwfYnf=&EUqP_Zy0 z$F|W-o~xOQz*C!wj1c`Ou`vK~{gPCBGtP6ROS*JeWZt277p6w~&K|SYE zA-{6s@pZn-%bVgXkPx4D!6MwyTBd~1K@~h?M{@aC2|~!@lmjj@Kp%Y-`NMfmMn60@ z59kTbEQbk`XqWz4p?f_Jq>I-gzZvLYWC%i2l%e3$NfvY=p6gPhL2nrC8s*&Mx?64I z^*?UAHu{&d>h6O0eaT0-f1$&b=vUc|1W#(7%&TM}b?*DRQc^wqjFf4%9PqQoF9m+* zql**S+{|9Z);7lOdWHN#{(WAjsb`dW@7M3N@HAtSZG8IPkMGG#A0cu(EpkXFQu$lg z3(R41p(!M+GHP!{@+P}Wo=fw&#l#kvK}eoX#j@>*-y~8-yKmhz>=+H*Z81l0W(Jd0 zb-er#2nWM~gtl!IfX~DDS5)xty7fr z?YNV44<{0Kxa@@j6O`U%rjUTQ^K`Q>@~Qvf64f-ch-B|7>^D6^YW0Lmx&I}n%@5Fv zP{+I@(c1jY@t}gfs@cb=BUfkR2onq3q&;dvuhPOb^qV_2ehncbI#!fduLtA`ht??& z_j}1(I;41P>WnaE7dsLoX&URygXRNw09d0IImY2oaKsBKoqr^rCljW+-(N7u z)@C^D3DT})ex|4QIVX+I@TRCj12g=vD5Q;)ip(8;di3s!{oyxbr4gI9$UfXcI!P-fZAf^BZo@NY;u#5RD+>(SZ|C1>=4C37}6JCcxgv zoSS1<>tg^~g5SQzn?cfaeL}|=@dDth58B1Ur?0}1l^o9@feruUZ}P;8LLyq*ze<}8 z8OlOgzsx>@lX%L5(405DOQy)n@8WC}7Ulp{pq74;ITWa`{*Z{*zqWN(nE9k0Cnn9; zd~lQ*{+@|oIIQ>SEQUXcBp#`s9|Nbqn?5T-#uOKg5WUZc%n#u@!m(0YJE@UJcvP$i ztShrjx`edbQV^?dp`wns`>jY^fMtH%K}5vVe`7r)VnWFTzVBchI~oBDN)!Y%uVPY1 zBl6^srdCgJk_XlJs_rk)$gK_sU1*@}bW)vC{6H$H~W2N1=&8=(PT{Ej9 zB8xR@%<{_XNFM$E-EgA0G{32MKYKs4ar+bDjssKA&B!of`O?!M#QbGKIxx&av-&>) z#jH$mPi8fvI{FlR*-f*%^aw5K6aNwa2;7dYf6cFbTAdPK7X%0kll{%-_kmkBNJekS zqZysc?r7cS8+U{Eqc9l~NdPb1r%u(j;<-1WlAD!aDuPgBPJD5|IOQoC9gI$0X-#@o zp`517Wuas*r?Rp$>`KKG0T^b_ECkMvqD++j)l^4EaFmaqwF(V*>W}gqXa*uw`q##b?*RHXfh}DK zugDwDH*{qFT!Y2;%Lc+ZT_V4V5F!ETLlvfd#a zP#HJ{v=ZJd;5#oWQ>q0(6UOudUBc0~hSVB`h^&889d2&7dL_DIb8ScS~ z4z}#d344lqs_;C#5J-;_N|CLs!is~o@I|PdKYQ;3ri&~!(cT?9#QbsSS4_fQTP!VM zGS#X)y}aF9PWcxG<(v;#DZ}Lt7dIsJ@1(T*8Ne)KK@Xcy+0tdvR8N(JS~F$rR2-D@{mME-Mzmah88Gn5v|X4o&4LyXA$R zpp9=dk(GDB>(>spJYt?cLTzqlJD5&Kf6o zi1Wu#Vv0}Lm2@0rZ^tjkDwr#VXkIsT#`)v)!K1VLPG#V|*=z|Zc`H0mrQC2*NcN&k zIHaAGFMnf{aFJ;70IH&>ED2cY8>_2k%&4Q(4P$e!epZu3XM0qzlj`1yts@t3h^c=n zl>M`ysQYX}<a!kB@*?Mo9Wo1Nb1t@*{;ly!M}>cQ23LYWN~_*N<1)kNy`YPB zs0qj{weM&G6-&L`*?rXjB%sezi@*&|?BlT9Rjg|tufMf0B)1O%&^1_`JqWgM#Oat- zMj}VkI~Bi$ZEB%6MLZt%1Z-5FSefMW-0X?mfS$T$$n5L zJbVx9F9oC($(W?O`Y-!g8lSQu2Hq#UlPp90*$dg8YShdG*Z2O7Nq9M092)+vLhArx zdSCi%&xRZ%)yylWX?2~N1E4>uUNI#F1qB%ySqK3$zGTUNs;B}1{BeR38qHIEx+rb( z2A!RY)A4c!Iek;22Fo`FW?j|JAHtfQ1^AvE| zjE)+4#d9dagLkA7ZY*Men00{Hge=HI-ZL|Bz+oN@rqHpcnwxgf!-h%R>;odn*T~kVsZ@!9IyG#| zC}J)R##FZ z8_O>mFf!u#DVf%ru0d;Yav+F7{ubErVlDymS54i>w0i;%MOF=RYPu^jubWYVM9h3y zr;Jr)9lCb>xF*RpWY1-7mr9%#|KBJKd{1~QFj9jH$^ z+MziIN+S{3=@(bssr&126`K)q#_?**W9vfm+7#nRhX#!)8UeB(TJf0-+TD&g+D8!B z;>ue#T{F;VKcQpj@{{kkO(#ZhV;Q^wCys-*0yaXNTr_etl2h?)&L$x}AxR65vDGyh zg165uD}U1Q7brLDi|kl_MLxb?j))1!M%(>;se3Ns%YYsOio zd{R5k1UPC5wy#BT0>KCnL$7XStX5%lg3BrY=?J>wtBb$9Zg`6J^enQOSuW(5e zZt|>!3M`cb7C|q>0}^_dj>#vpPn&-{s+@HY?Toi&raL~txeoj><({;FbSwXtb}{|CZ z6bLQ?JmHPn;=RPs)}K*lBGOcL<0<7*9an37t~B-@!xHrI5eo|yjAv@ybe5EIv*;+y zp*Y)Lh(~&@)YS!+Rp2XAX)JOB#OysPCqG)F|M|5aOf!T2MtN{=D7E?>k~sYBtq zFQg*)VeSq#MV7M2>PGn1)5I?cu^k;5)mB%De5`gb649pbE%q`Ia^fqY>o;UEL3!SF zLu7s?rgwg?d_$HxFMtcYNYrLi5Yv2^c+9xKfN%ED}lCdBLgX!sMR@cGL;H9)*fKl+Y`-*Yljbi8$u zn9Alc-L>C%)30H=gbV2b)TFRI(8j}|n#Y$^SB*I*lr>fP(|}jdp8QpWg2lJ5Y9Q*w zyuMOF3e~EjkbdIj`!QZ&SBt8W4lMF>+wC5d9!D6v086HrJbeHb(Ve{Tiy}6Or6tF# z?blmKrql~z9(pV)NNgYwX4*V%g026^!h`wxHTG*&q^c*geyVc_=O!@4QG;6kb?jbB zBLBVwltp4-iY8kNJgpOTB6$fv4uusXJgYsGRwpC}yk+rND;lB=9&d>N>`O?F;M;fA zjNAk4-;@6ZJq8hi6^EnB8PCV14=Gk%NTH;-vV(V*2Xx(fx2ya*Ct1|)KE*DNV{>oE zDdcp(9<=jGCT?c_e;5kWx$aepxSuq(0l`u2>l*;H$1y@uZ(9@&=mAlGxYP8Bt`kTm zs=Irx!%18=>Nq_<-|yQ!powAc9w$dlaQNqKGu2dnU2~Mlizx3!nH|UA69YyXr(6l0 z=bYb7Kplf3ejJwWv13SaJT-zPe4quOK8&7b%PM(#B0*c(3JP1K0ml;l>e&Q3Ujf;R zzqwm6jzW!LmZ-(29?IO~uDH!pB zT}o7S;@Txa-`=~Me-2~9Iy-R44Up7IgS5JedNm)-kzM>j-xhz<(j5QC89a%yQ+w3o zmuD+AcTvRq<`(#3zS$qG8jAj&Ma#u(M>pYGbt4_7|pq7#!p)_WOGDrXY!~FnjrH8 zTs1D%^Zj6EHWia4=u%YRDBnO3g<`)VPg*t{S>U14|G^q+6%wY-J`SBp2#jKO9K`e8 z)bN9{I5+;q$ylDgZ1KFKvHPU zo$&mEq9R$w9$#|FnreTnh63Ix1^pzqRsO(w?yFA<uCEl+$CUj7S+s6TNmUaMlZ z`H-aycXl&&mK^1amY0$!YUf?uo+GTprT=V05E|Nt(JUFIk%a?KF1TyZ%;|WpI4ee4b z{t~`bLpjeG@(AHal${%m?({jZ)``Z1EWP1jJmol7#!OR7ERz!O_^~%38g1+!OlwXj ztrrGz%kb*5(P-)^+b_h|4VA>LogZN=6n*+Ml-~*2|Fi*oUsaNNI)6x~q*HCs?)*#f zAn?}^YmR8hJabthkCPKm;^CG1=$@UgO`)>ET*hGP{M$D)Z1^nYnkiozr?o&T0hT;Y ziKSmO1UCK8$}lUZyh+fMAf@w_O3HGSL9wZ9**v;Z^pYtUgV! zD-#;wC-|gJ-8#Gh$aG13i1<93-fA@=I|b_ZaqSR}ja>lcD7uR@Qgm`-7bFaQn>Z`Q z1%;k#;-&W*IHcJP;_u&%y_`A{{VYGx^j}LmxaXsMx zoGGE`Qo-b2oTOT{?rCM%j$>$ImH7AMJ6x5l#Fz-2n&;{lM2H8*5<}ra+qVOv$H|HE z*&Yz=(+6A%MUEUz>oe~Dcpqh%plX469r7zuh&K;IZ(G*MomiJN*`TQYTQ!GN04BWr z`De>Y!(y!)T4J+)VVs6~Rw2$L4`sYuV>6pLpe5+>czmaq`psMy_^oCtnK{5cafr7q z=gp5_L$bp{3-8IXUQT_{Kq0a%5^QyO@yOA$j>UzQr6|t+J5h$0tqV=B`+KZ4m5^774hQGX)D@r$xi z{n5`@ET0=>*TX}k#Pqgzx*^ulMm6t``qrCCqanYI9db?);GlGb^I2mv&~dF*YQRx- zKN;0G_>qgz%J&^#TmVFG-DdzUqqwpCB7oiOss12jCGJy5QeA$AXG5#1lVmM;&bTg7 ze_sU<5ghTQ1&gXc8r{ikRXA2)q+Tj4axv09+p&-~kWf#DcZ=&qxHo&DE*cA1vd<74 z$~s?OddwxnNxuQI+AHWzpU8T5=nWrfN1&i`vEk}uFvMN<6~rQAq;+lq zhfpo$!b?aEuCsnEUXZ&G@W1cQR!i)m1kZoG9z7!4KALY((Um%4B(&GLRVeHRqj31j zoI-w7EZr3Zn`S8;`j{X)F?M`H8J+}hVmfV(rK#asq7jfe-u0coP>$mlOpET9`r|)d z7mNzzd9`i1+d~7|+C>^8h>bXw9ny?p?XBF$D6h)#-zLi{=NJ_yyB{y0J3STJR&V#O z1^-2T$%)ZJA|JGMKx6BFI~Zqk6z-$CuUDQ(F(w1WYu&t(8&cM*L-cjGMJ@W8%eSWd zu39h!U5BwA(iSmuHIhy2^L=r#{BV^of(N~yJ7$R{39k}y5#K#wj5^FecOMA+@Cb7} ze?d8F0V@LEaUAAXB=1-C1qNSa61~+XtR&jqq^t7Gpgzc0mJ6mNZ$?)iSTO;Is5i<- zJZzM4$Z+nn_fO1vir)(yw6?wcHIdO;g@hmKLG8rCrm^4bA%g;Lh8|Ku06G8*JWIG0BabJvoLCrnN<6ow>mNuh0o zZ32grPk&Dw_o{dbc01lvu$9 z1Wy)O=sxb!0n~zs!jX%fv4|E97lTyfQEkxP#VEXsTSQP0wMR?@l|^=(z6uO7CTR8? zAa`)rZGF8z86EnHm`qGdPhnKAe7w~xFX6q4RWJCc;sM?(fv_`x_I^U3C z7{Uw!H#Z`~jeTr18$@$s^Gm9Iw}DDXwy+;%K}}S$@^UjVuGYI#)nT6FY3I8n01M6G5ARkel1} zoYY^e@6j?;qOezOwUk?9lIqXI1%odS&tl2m%j690e@&_sMuB8RvSqUBqxvGcnkhlrGP;MzFUeam*aUuYljEDxnMaaX0xDX z!R%wQNi2SP;XGd!;|*=*ZoR&o)1u(R>UfHi@9P3&;g(om8!wa%ZlE^?dYT)-230)! zVI}z9JN&K7)On}Zj{hG=JsqWydq{uKApCGna2@=%Ft3wPAy{M1AHEk}pZfxN-aqg~ zF}D|t<64XZL|})zVs0us!aPZ~I}o)MRVsYaN?S;9Y?S=jCMW?_IeY~5kV07?|Mwn5 z?68+<4;=v^bb1Q+@r2;IiI7*VmD2K(oiwHT39k!+!F7W_@85L-k)lPhix%Q6R?XvA z5t~f>Y~x>?qvOxHxOL@Fsw>%7%epfLW0CXs_0>nMz_;b5net(b@MT)Djm>?8unk{o zkoiyiPwc*T3slfkNBpHFF z7HoXE*+9atyZ={V-&<9R+XgZ*kq|#VFZt$RY4F-357&u2aLzc$2ZS^8Uf2Jn&N7Z; zl8E8*1dRKIE~`I{z6>cRN|=iU%ON4XZr@RG`M~Z~V2#2$wK0{DWh?>*kbAzO z)q^PhziRBstx@VPFLc_QgO^_2?6^l)*#V{v^=z2#f*9!?4@c7>3{)=~9dV9BGk=ds z*MWP!q(=*45G^iGSlF5V+Av7c7G2*gl65yrHFLfOEvB6zUo4saqeOX;$>5FBO8#|B zn5gQhD;i)(>C+K-YqWhs_9grE#J)$>`h;!wR0jhGdR|)$k*Et%9h->)a|(k}S}qxV zuXgwRADfMV+If|<#mGGslKs^PsV!|xCHn_ai1@#+tZcS6lgxi!i9OJfs$;j>*#AGKzB(+b{rj4d4iN+q1f&rG0qGbd1f)eoKuTgr zNog3mQ^M2egk!4~99?hP^5?*JF zphkDs@!XVZ16bOaYf4-GG%~&kXB!+f7B48Jmf4h#DJpC#Og+)D4_g1_JU{=C8ro*d zh5eVjW^a~x-5>7c6?;C}Cv`8KNP9_w_8a`vcgyl?AK{F?#Qkq@V~Kx69}FeoW5@PTX@Z!g@+2*I57p8|j>>8g@- z;By+lG}q&fjbRBfSGmeTyhdwIUxoBJ|(FI7QO^BFV!VgF@=6?M5eazeTELBnzW)`PAINAVFH7+v=N0i^#W z_S4y=QayYa`SMHgA(Ni!R~Nw^$MRxg4BH!rulOGyB!HLFi7p;hYQtTA5(aX(BFqY2 zemXxC4(;xq2%e=`QwqK$xPH*{Io^7w^0aUubJ?!32#HP$iF{3Fms$%!dE*E&-y{7= zJqbbIMggKGWdFe>m4r^Im#(ny1QV$6#4@NJ4zzHl6V}|Pc))jcru(e_`Ec~b*6yzJ zS0RVIsDsomoQM%HyTQbn0NG8pIMbkK&Kpi02!3+0YfiGq&Yv`Q(i-B%7g-uot%Eaq zwOdSX2}UL6G*RC(y6zBJluPbfELl(p6`+3cH=f?fQ9Sr2Lt zOzb@n%^g?lI1CTPbQ=xW_)+BBU$5Njl7T}3{G$PB1^!-yRIy_zNrjl+V|c0TETqBO zM0xI({qSS5Xw~mACKi2rea@^v682@KbwSuJe(b^z{{#q6L#2D*ISIr6__nWt_+poy z-~D#xRxkqq!C78#Hh+Rgae34#jK7EXW;*`ozy<5fno=HoYX6o}>bqqfP-?pDv~znV z{zg7zz_Ry9*IVU_W{~Qo8?>AXHz*vf4+t>oae1g9*ov@fZrk_-pS$K zpi$Zq8~?ykFuC>bvz^p6oD+nKeL!VS{f~0}?_>N5Wcip4u#4O9Gq0kQ^({V^tO{n9 z$<~&{Vh)ZdsT5dr<=oC;)ROhXlN2Hk+5P)q5h}s734p({f~lU%q5tda;1xHa#MqQ* zg*_@f8$x4C@mH+pxrhXrE*`-z6#YrcIl+-YP6><8OtEDnqePrD*fhhHJTRF^+o22` zF=a~;8@76urGn>iK{J*fB?!aF5cX5S!xQCecH|To{MLPigPCD*Z+DNrQD0Z@0ex5_ z?tUq<tq_lazE$T#MrAU-p4R>m$lu zwEG1SqBCt3D+?Jr5#-GE+GVxDhl?qfg%|i4I~~Uf-ZYt#Zqo8b=lFL6&Hv%vgZQ3w zp|`&8{xoKA;+dP_d)fW;>(IP*4%t!4mkI@kLYO&j+U3_AP|6T_4odhLs2AT^;(<3* z8;5}Ios4#fvQzryv(+mlXcX#?Syl)7NuzX`t&5K_0yF=A@q+DI=}xZ)K(UELVXi}# zBQL~me0@VV?*28{IoyTU&jC(J-yjpM3uZ07PF?d`mMc(EmW_83Sh= zRY#zZBs~7Ph^Gyf+V83>4bKMcb9E7}*oDXstew}KlcL|duN29^?tQuvhO|TyGP*#nmtgGhV9;MpkYlsI?^rQrMd|Y5PSPX` zz^12CEeHCi$1x#we79u)wFzPhVik>2uvV&8n{K1DgricPrD$4H;P*hsp7dwm8<9#4 z+DLt+I>6I)C$)k1&gIqI(x_1{h(`oW4O?J>uyFKPsVb&I2--xj|CnGIj`ptA*8fP+l*GqO)Zs&yt zhq4)}$`Rn(m!1xW2;QHU;^mWMUflnZ& zh&bLuKIauq&j=TA7WMN`u(dCzr&=-eDM#;@uVV%bLSvseGO2$j zQBsJKj$spz@mmU5Ea`s)731A8`V+7QWz2?&-Y(Da%JuT=CZ%~@?z@=Tx7?%rXh*8C zSS!+p#FK>l!y67c5d+a;dCqNR(TU)H?jf-(FVkfUZ{y;p9$v6W z`utRXXT}_H_11erCz4424BS;=W))lfqogbLsPhDh4EBLQOkCUm-k`l-W1K#LBonn- zFh%)73`<^|XQ*FxX9H7f) zKnaUJ?}7utF$cB2k&-+zBQTM(x*6M;ca!1$^B!VdqS_$d;HN-@-dCYEfF%p4T>)Xo z;wWiUPiULlgJ^?Dc-o9`H!wPjVEc7^wx9)b0rcf*GOimN?m}k4z8Ct*UvVAis*)KQ z%dKRzihj?pN0P*)DHj6Zbn2LM9S1>eRZ&Gb!8a?n%BAq3qfNI|b^Pm0?&QiY5w)(1z zVl6${H;>`=jIdnMpgma@xXpy>6ahT=on65ZO5&dMLQ5lrzdQuAqO{INu!MDGk=}HB zF>;Eq&xY(x7Dn|(*`JPo=`3a#pzoypw5T({9rZ*&SbB=DuzKEmx??uaklG|h-D}iV zcGv&!!vhqmexTYnQj{AhTN){{1{(9oJ4-G8Xgw0t2^~%;KKUo&D2hSuD#>?u$IdLf z#e#XSJB+Msin6{Q`225Zh8@dn7@k2@oeE?7IlBMm#@^s8SNI065jGG14X65WcwfL+ zLSQjxVf7@oiARg~bl^nZ&``%e4+v#AY#`yZaehl%;kKfb_PBMzkBZa&AE_yJ;57XU zW4_~1oMY?QDr5?v86yyxP+NPJ)fc3#G#L%I?)sF|JgSW&Z%03+CpNWM@!`$9{j|rK z5KZvV*~9sGvhok`779U=2JUdo>g!cs=h3ljWt%@fD;S*>aA1!dnu6XIbB+^sP`*H>l9V1-?7xqQhrNEnLxb^AA?0u_csG=3d<1wi@RSJ`N$;kdE5~2}$(--l| z)bBw8f!7H&jy6u2Rw^KYd;dBEWj>PAi&8L;jKe=z(S6DL<_S5SJb;M+4vu~}4v>)o zuki@3_yv-CgV{=0W7>-0NIS8X9T6E>Y7h~-x@%wMIvV6P|GIPAJ;M6MW^OuxDkKIk z-u0M$*-?Tlhtt;$lIt82k-qes##_-`Z>e+AXiihAtI-5iI}4hm`+FDM4-mm}OAE1M zky0Z~=NFoPeltTsV+8NN$dua0$u;1)gc&r1i*y~hm&w?56Y&`iYo??^VoYNNk?Scqz!q+{@frY{jdSJ+wWxb)Ir z2@6oJ1K6F9o1TEw8MFYd=o9jv>s_7Vu?MX@}54KIs%vhkm={_6YjBp{YJXAcJ#Y!8o zJbOM?4vE!q{r8My5M8+?c^OZ93{EO;x$H7!=lBtHRNb0DcC1?~hDWwqWS0P(9-Vg} zZ6JtT4b-ii_2qJ*+eSo1MMXrIe+eqD6jzYbp)xs)pt9@@q{c}&LG5V0)OGL6Or9H93>Vq&ogtcmc@+8oIIaPdlrWT&IBSx77an~ z$d{isJK_}uezaDpt;U9!t}%8MP7LB2w)B6%6bq|UCGNr+;82j*9T1Nf zU!{*BKS{zxi?_$lvQl*s8+UzwD6h00fmi6cizX!W6lnVxs_)S4d9tiASiOuSpFHcx z6Kp#~k$F5`d1d!XqL?ZztZjUe|JHepZ$J$%@&|%f zuDD``LLJ2nEsUgchnWNuw(3_D`T~UhA)P(EYN1E<M4_xk zE=#5;9ZZPn3Zkiz5k3Y!5vcFvbLqW*DAifWHQUzcr#4(0{7<`FJarLEG zCBghogiJI{=3MmkD(_)mj1Rx2^IqIHPcPomwzXAZh{5DPmAb9nTjO&Y*q6&E(sS%n zE`%mJesANmrTBG!7VC4l7C+%B`u?Jm2*efoT)#8P4!PYfD{q7`Vp{)F>(N^0;-nEB z$J<#XMmBQS_Kam|(EB}TaI#|gQD7Ip-y+!1+eXK-fptD@8qrT`+dfE~H0s!;h z{1xtl{J9zsdBn!sIV6AJ#=QC>`aK~v=M)X|%!4>PMJ1y&tGIbZdZWa|>`ziD8h?-~ z=Z9mBm8V!dIO%RbYpZPTzf!(Tf2`k@WH-CF_x!f$P~s`hz#xB0$J|>1*l{7l$;UuT zrbDHU!70AgbWfyjelY5!+5>V^4?;cL6NFw(pb$$Zpj(R$1FZ$9$VIGj=$r;Io#@!` zEI7U4bz0vu0q5_-bxpQsF1?(%Cy#(UFb@^7(U|v4EPDZh`~czaVX_lO?U@U_cc9|eO>5OlZeSzn^*gi=A-8Jp#8 zn_j;`bLr*6T9P>qBPaBNygA@uMJqzMi%xMIu^w{1M9NaXh>0C5EHjWm(698(?&JjfF8+fo_k=g^>$ zlQ~}tCH?@RJU1va3H^ousDdp&M(+~n7Qxt48!3;YX00l%9F=|sURJ1mF2uy)^H+2x z8gHc#!EzuMO-iY8@_#lq^~)VDLKU%$?GKT;*ulu#Dn)YH1$mEWJbsx!Bi;L0H`jLy zez~)_=#c*;S7tMb*eCggf{?agp!ann6%MZVzzWS3Lu7=CwTi8qgF$)c?LmNsZm(Hn zhN)KRZLbj07+&f^Y({yze^?-4KWFn)|LY|bg$Y7jb5}Q&6%-7uk8_==LG)2CR}ead z0t~JsJkFex25;fjUr3;Tn0IhMw+iyzHx2w5wopVAq$o_x{kE{@gzN56O`X%FG{q)z z;=C_c22xQaZZWa;$vWYs?VkVN(Qab!wHsH(iD z=@iTYXOLX$k}xVNZX1`MB{TK~E!i2r=|k#+K~6*m9p&21t}LP`(+dut?x=l#yoXoGlaftyYI^bLF=b6jgq;wq0N*F0foh1FGoRY|{hMSo5fqOMpa zNLLln58M0OSO+ww8j7J$biW$1yK=7j zvf^UxxC`!~g?c~1e3iCjUqM9sD28bUv;uas&voVKg?&&Y%zfbB0#Ketf#J_x4he)z zM+i&F37dW3i23-gQ;!@T%c*>A5G5rZ^YAA@>~>sJr(K=;PeN`}O;CjRYIz2t!O?Le zN)^F=Ys<~L|JfRbbM;cik1emKu6<9-NdZC@~iQJok``DIKPq4G9$$I4F0GNoypP38Cxwe zMgu-Jq%paMBXQ)Lfx6wYnk<|tHSX$(W9{qh1*i==Iy%puJyS^&`5qsCvHdd|i3A)I zOL@@iY*P17?|i9YY@@qzQXCQ7wdf^LPUNF+9E@+r@?7=Gf2iRVzGt9Me?OOcGfmOn zBi6`7w|8XRxwb!&-F&0w;!~V#Dt%AFx#kCw3%x1OFwyXo$PR0ZKwz4H+49BN3qIuu znOd2tLPa6#wZJ>`Q-|z#&mD6&Ic%k6;Ej7Cfmih5<6^)T$>b6T{Bn0O$lzxE=e~x9 z2J$%lVL52bRBVm;bV~P%2zD5X1SIelVr3q z-|jki=7LE#`!S*TNu_-_BeTn|`~7y>-wCHNOu7Nx@r>C&M$#4*?Co;*g*i-kwLg7Y zt!_H!9GBkKTN)B_ofo^N?XdX)3!4N=qa=Bqq>=HG$5Lu!f%hD4nNM)RKV(>9RBTMF z9mW=O6Nsy#^&3{2n*eq0>G<7Ue#943nGmdX80w}dE|25mL`Lbonf;>8WXzPenGkj3 z?Vj2X5QE)}FMIt7O}C&2iGznXmxz(ZT|6-eV1O`^zD2B@U*lO$$-b>gQAKQ*N*`=1 zNlj-zAsWYD``!Hev;aKxlO=K=EVQJ@14r?gGD_+lM-d}*fKQe~dcJ90OEdIWt_Gg7ya@+^q&47a{49Or5hS!dU$*BAsvoi25| z-)wM0+~8*xR|EDQhQ6=8mwj=3a03_V*Vk)Li9Ql?tDtUmFqOVb%RUiFi)vJNl=>jn znl$_vF`iz;I-mbW=e64&^@TSR^;{rWc|lCL+H}sU+Iw&9>Dvc(Ji%z`$- zW+C++1IK{Nx$XYwEi!uLYjS_a?fR1=`A7Axm_u}hP*yMJ-(XzU0u15VoE;bqs}z6% z=OGfGDccyj&+-8vm?^f{QruBEb0~4-hkre)VKXOuO!3a?=QK#gE`ifU(0d0v^IILo z1_|VruV&t+df{T$s-#70h@%?w5(5r7aLn*ehh7&Qrg9d@iGl*J1l4O|bD%_vWXnvA z8xW2llu7XVy(=8gHIANNlDiB6c(?<)r&to7kE&!R4Nb{`|**vz=Txi8yhbf`Pf8tJWmQWHOS0T{j{oYyP({=uu3ReWIo;9>z@VZE+ zhyp-&FeR!iZt=w?kbAl{Tl}s!kUOA#Kk^zc4|nFn={x0kX*ty;=ehl-qw$p_ApB!r z*Yj$MO@oi3qqPI9LZ7)ZU?*3`K*tt!)IC>l5dHPa1bLPR!jxEXMH(U&CQ(ZMc;452 zDL5@P=qYn3p)sO)1_1uL9|Fqp<(5h@M2L?V;Cll~9B43Z{#14+Ml^yf<{Al#Z!XP4 zSH}b*;WYstB#m2}T1fjmMy!l`x-Y6JDJbA)le;ijEVa#*0W#}}8^P7fZ>;kyqzTGj zUu@wuioJQge`5c86Kig5o8+FPnXwg03$--0Vg|XaU%JMqD_>C0aNDG+8J7E8yV1NN zR&#chLyfW4IYHM+_ddyYbnk%5hrNAo^o%zn)WHm+JK#KMZ7YizPo0abxowV^zja6`=`0&5FE{@(SIz1I z^uuIHspQ40Rx3)o*z)%APk-1%V;|MmxEC9Gq%0p5I_^kGUWQMtFRMpdu4)dQ)b)li!?3z`NGEGjfmlIX+lrO zn#C{#W^}D_ldQzZXgK3Pvky+)krfoS&6y&58L0h~qtR3!n9kJP385PzR!W>!2k_bP zrqRS`bvML)D4e&>r&rmW-*Tk15Bh}Ai_h6Ypctf|(a<*$%yuro`VG1_p1k>u-IAgb zG~FdoIhtHF=gPac>tCJu4`jXJOtV_4#Yd@1@%ejR1I-_Q|W zoD^m<;+JizTmfEw+FM={WQcARRn_Q3IbShlk;wsl%IbwvK9InGZ3*FSE(96=Cx-r; zs`tr7xhYYDM?t`)L}{n5=@8yw#{(nE*@(QpyY#3SS?iW#ULrNzbBO#SW#x@CL??u5DW|G1=z!hnjJg&4!Tcf^X;M9k&4ArZ3OTG+aEy*a0^k1#(LYBv%L$|7=WNlZjGhg&(qS zGo~Yp(w0RC`Y6~)ISwhi7*p|j#!SMDYTh2)ds6mpk6=#aaZ0WwTcdRR;qz;K`Qcv9 zYR|Ljo>BW6#uH~X<-2T1lz9Esc)A|YlbM~Sp?oBrWfe*Vd2Iu%95_=2+0krx;lI5; zRz0L0HRwF;sv6=b5ecWby;_WqrB6hSM>c%q+g~^?*WeeP+AQF%#1a<|Op97jiJ$wCF+;Yv>XhAMdto-PX(APAd?zI~}ZKg#F7LYg> z3-iIJX5PenYL|k#!dWTvSb|I?vMk;!V)NNjgLDb0g=~?xsn*R$7fED-$Z>D|FH8WC z{UDC~w>QkU-rn2NbRxPsBx+<5_IdCOo5#LAaRw~XBR)c)?(|iqtLq>V>L%eA+ToV& zD{nN@D0FXvcwDabX(R17Z$MY}@-YN5Wa7G$Y_qQ`9LTF=%?nc_vbwfJxIaK*;ai&- zB$hD>%;xyI#NK#4A;r zEcj%3my5T0hMF}68yj1VOL>q>S=FAU!vCBw^9@PbK2uq}<>IFjh-j5@1lb`1E(j&j zzVC3epYhw~fUJx2XwIg!s>V@NH3VWe5!DX&_yMsnwTh|ho9#0|I+wWZill>_Za7|w z)wfO!wZll?IsIAtEo`uDy9+&9h>6ML;Bps3_D3ptxZDnC2pYz9_Gu3sMffJ#0S}0K z&qMv#Og4KkPG|RCA+-7|!SAlA3oHd0u$@G9q7zC|536OVNyN_9oCbUiAQS8=I=0BS ziCkB~58?7?GjUU0G`<4PM*>eRIk_&+%zJ7?o--{67%m#!=sa)WZhq-Bh!M`rWH+9c z0F_FL3i0(Iy;_ASK^s+*eQi^{aIaLVw~M8+m;{#8o#1L{GB(}h$wnW7?5eA@;iQ)% zv!4!itF^E#dOrd8CDV_aul|M*=>2av!%G*^=SMO@Zo~pUYUA31_AIxpt*XMVu7~|k z=j)#yN6Cvm=q6pKrYX0t*LSAnn6*_ z@D?Ihwrwv2$Jh4^4@TFHU7Eq}BFOvr+dE#7_wmq_XkIwo zhVpsI*w@hT%Ftt(X0j!B{DS0Tt=-udpkeZD4Go0EoKrtZ6D_rR&0=Ibkb>HGJvqvfH+r`sNXEihDsO*;h30xXPD5!tw`Y*Hl*;8H z(b$xi7aS_mcVlXg1C}DVnw89EKk8K$nm)0vQ>qx6xDD4eAkwxw`I#|qlUgHy%|c&> zk>-f8w>HW9hBhGC^VP)sMZz}x0jy~Ic&oA4CIf(s!%3nWPKyo$xEiig%eLs#LB#ef zlu(4Q!T|xIK!ZVB>(b;_SZFL|4dpIT`sLGy0_n|f834OC&;C-@T@u(rvhlB;4bWn) z%6!4eHu~Jq|0MZ61HWvJ^XjNreGGkAK6-;Mipi(Hf$t$(c<3QRW9`$+c}EiS?-!e- zZPSHr4xjf{iK3A6yvlw(;WFrnf+Ani&^+QVD=^J86L2%A^XUn z;Z@9!Exu#}k?Fxo#Yc`^lkp-RO*@!$SBBkE-@Zg!KQ<<6&DZmN^(dz+z^%_lI>ntP zMOBHKXN`M4@BCvS)D?k3WNhCPP6EM4bgfd{k8Y{bn#xjq!MAV?B4q0L3IT%pf>h{q zB3C=QE8`-iDD47%1y#?-FwW7+!M>plOeOlZR9J0}Z?Md4U`^cySDe2qZJz5Oa(wv4 z#`x5(R9SG@{Lq&@Fci4a!RbZTPgiTk9}CeuF=Ty5W^Q?s?7Y68al1!3oTbE)lJK>8 z+IyXm?ygIsY@Gbm0rZuce+JkwmAJ+f>$;Gc@jia4sv*{XrFMXsM2@oLy5#C@)yhh8 z5nhqDoMvPiqp=oxkUHtIjeqmpDh~%zrq_oUjaQMoqBXx&Un2wgw)W`|NL9vT_V%_f zlL@b{#PJM#;BqrktZYG)mX^j=k5?`m7b+fZ>0-yCWImmWr|a9&c9SHStS>RD034yV zKXmfXEOOBpf}ejBcDSWUpu7KwY||iBU~et`jPaeP*Me*gWVfk}z?B!Wrn@wfxTt@2 z8XN?kpuE1J(tJG%Fr{N_t9A)WI=dFARajo(=~BhTjJK5Bq_*l31v>7S<`ul~oF&c|L);j3B zpL8=u`+OEPSR(nxVy`i)tMkC1aPjPszp&y|iH)}-$_`3u zpS`Cz=a>j+WW6(j!ZN)@Zn?4G&$zi0X7e#ZUFU47z2JNqn#2(g7R#F-sxB=zmOK$& zpuT)rQC)p`ae<*g>^LM*ADSmbmVOBg-r^h|{-lf*Zna9tq&$UaQ<0WsZbXS5>`_(x z`K+D&L`-$+bkDJ4W^B?iEeh(744gGY1i~%$N4ps%M9UxJXB@s zK%Y8Ircy_tz1qc#zF>HL;Z(_39^)STSrVVCZrj92Q<;H-oOtfCb(ufOMqrplbtlA3 zkk0U;_ll04^{sBD^RMP+zb!u?S?E4*HG}~|PPN@F{RKLX8k_O}= zt3DZLOMZM9@)NTu3Op$IG_^qAW44Mr~p#v9>}TZ4(X#pW`SK644X)aG2jDIomb=cL?VZY9gFZQ`WdnE2W9 zMJl^mcB6jfvHFj;N)o43p?m1$J^8{aaXMtZ&CT&s71DN7K%0a0^eZ{8Hvt_y-lpVL zHbY3MYSeDSf>@RvOFvO_(u)$$R`A97dB?tm=YGxTFTJmE1{7tuCKCvCDS_0OCwtFW z8FNE+#xx0m6=9SiU6sqMcg3AIZccs8776B%ne|yl&Ftcbi$`fXrwIz2c4`{Zt8GVD zsg`N(enNT+Q^zkY$&%-d_sP`56{|PM)jyiQSJcfA0c+9h+GJR}8+MQK3Go=F4o0%t z?TuYOu3TZD*`aF4|KfN%L#^B>hU|J)Ayc*)CE@xblRUX8B!xAA&t^cmpvLljm@wBk zly&p@-{!sD-*GO`a^_ono-(wzyZfLs>4@A15j=%%GFg4`iYv~vFWyaggvv`(`}?RE ztO$7kMS0iE_?2t6G1`uy`)*nUgswi)!$JNryK!y~D4MH|(uz0uY*VZBs+XdUf{V8< zzI1Vvn=p^SGTj}+{WG@^fhRjvF$x<;H|nQ+0Mpc);otFAz;|`DKTy40>x!;Vnl}a3 zlsug7%pXa~12^&w7*?VAKw%yrAEs|2r=mY2JYMt(CblFM6%{Q9McT0(3U(-`;$2{Q zx}mi)r0z_ok3|k5p(S37Cie117XDUEp4ptZz*Mn;B!%^&qM)l3sgyBO3#RNb0ebFq zQmbLIVh}c{wohQp$Fuq@MJP2sak1izw3Dq=Qu4dGaV@B5K^q_pnZH{vQ}CU+)2&`2 z)NnIcw9xwxX5@7=^T*+`ZzZ$m2Ik*BdUdQFLmAq^?h;5~yW5@zWL1ehq+nXXvh2lw z!!wBM(|f5&>FI;&g!@A>o-E7#W2TWzgD+>-7@8H)g2&-~suJb*>U#V8&*IZh-i_j! zn?_apY+j#kWkFHJ-sbWUJcp0|n}o%T^V)f~ev13c&1{k_os@#Af`WRMpQZToUZ7U1 zEV5t}le+Jdeiv|h3Dtc%;;xj1R75%I#Civtk~!4c&zeA`t;mdw4IrT2bx{3ZthcPcp7Mjct}YX*zcDDknMn$ zwciu~mSe$rggIWZ#us@}obJQXTUion7eBz6tEF5HrVZgo9X2){h{U?G_B)hqIMd3r$w0B9FUsIJ90%p=;NA{^FdE z&{FYu`Nz%8ZFu0Psu-U<+Le`(x6sqn9c76)w?$QDI|$dH6XcfNjzG`N>y~uT+8P!J?$@<# zyw~ZHd280Vk0onvSPc#UvR^3NDOfXZl@= zJ(?-o&siZ8jyvb41fV?vbT!~AB9EXy)e1e>zDXVjzm z`5@`#Et-H%gXz~5v&lbd=nGM24bDy!pX@lRu5aJ24t9t@*d!cv&mSJcsVg`6YK2vc zqK(|9i^XgW2XpMq9NHe{!L>%N`NU}uX<;IW+<#_ z(C05qJ=`*TOt5vHccK}5;nGmoxVa-lUQxB)(!B{5{4P$TdD8x~*O#Yq1wW9Fdr?NZ zH-d8;v+K)Gt*n8e17b`x60$ap)LOB$=e?T-H;t#Emf8 z^xH=e=uigLW3Wp2$kK~K`6#`fiz0+$d+*^}@A3#F`MUJ;9AiY0XKvoJy@TzHk-A|ns+Q=J0(au@}9wzc?)44J}{pst6~_M()5{As<**z zvz0>GRo*jWx>c6W2yKmP*XKuZ=f|qABxgG**U&>IZM+|}-o6v7D_+l<*y}{&E}+w0 zI)2-*d7d^5zOLi%hz5<$81K@h^G2YbyZ95P@(sN(4(gd> zC5pf6|MLP<_OQH+VveVx%7_k3nI+J_AvCq|Ew+ccv(yDg6uW*)Vb-zXq zjLw1Giu1h4gYCEH7}LF)%N%fSIYCF#=4gG6X9@zj6Paj+bF2_UL&{n zabLKj1oUfz;PM(>$h9$1#@ai`njuiUna*yN@6H8*dj!gXQp|S*^`;jekmMQx7PLBj z8R}eeS1*Z%I$bG2f|!oqKYDQ2a4|5v8>!=3BK*xEo4-WJFnLLn@V;SM^Y76$UBQ-a z(c#Ad&c>n4ZtSw9u}0Rze%9HZouaD2KomTmTWTaQq%|joGv-^Q)?hC}|G1RYR;9Hk zscv636)Vm~TAm+hc~r4FHPlc=VNSRiwMlQ$MR950pmo)7>fN9m=~HJv&|QGia8N@%qnnBnlsgybAtlb!^A-fNWI zHtFs@_QQWktQNP)(3blLX>R9?$e!n#>ZJmJ@>0U+!d+? z;dP|kH$l!$L8*?8FFld<#h?wr`tRB3Pj*$&Yh(TtDVsS{J9%!niT?yibE=3=pfE!D zT*eKKU6BT*D}0S2NI6Q{40hil(+?1>^91_4p&7tD1!S=;=xpN!OYjE3v%{Yy&00UV zl>qtQ5`YiYSGU!P53>$^{xCEm7pC4Y-u?W$790g29$aBXbeKb$-!B+m*mn1j>7zB` z;ZXn933V-3m5{iFtIVA|)|(?AIG7q|?@^npG+Lac0nw4^-A)HN=b+cTd9w1!J#FP( z5=Ji?*8Z5%K1im0YNpIIoZEiW(`(hCi>O`R(UC7~gW&RBZ(QihB<4^zeF?`1Vj-A1 zQt4Hyg%&RfWx(@K*L_P!W)sv5Il8XKQO$5I@xBU%zD_Z$YYCQ6U{%G+qx-|k$c<$~ zvpI%wD~wBEe9X_oYn-U%IssA$Bw#@l8!GKICJejYgHkBb<4Lwr+nUH%CEPPnQ<1+kT)wfxQ?# zRy~muS8W@~h~yh##}JAup_e$#ut3lbp*SYP0Sy-bj={Xm%Nm3oTh?cAvqol|LpJ(Wp;hj+JS3M>ZYu<)&ENNjE%6w z*}@p?kAxp42h#->*t*^(bqREWC@4`^xm&bLA#cRSvw2lP6$haPsHH=^hP3Y+imAeJ zQFga)qU~73#5Ig%{hqT=p7AfgIa>2qObY}U>= zshbodu$v-6xE-VwKe6Zi1Sb_l+PpFDUX9nhl)YznE1rrv7^L%7>i*dluvx0s%AEh% z?)z9{OG?96kz{rf{LG6m@KDt<(HKP-hoT-K20=px;{+_eHQ#?L(?|$ zmM?Ee4ri_JEZqR3$&0u!VF+@&22cp!HChE5&L~XDDExpioftka@PHeL#tnjKz8?=k z>rB)hl9OK%g9q8l8y?``(ryp;aJPB?qHj1}jmY~xw))Yi5^O)HSb zQTN;m`#vszUZy`AW04xQVDCFUWye;6Bk{wAVGOmn2w^Ab24u1Q^K?7@)-}{+mw-az4`(1xMv}7vrAy6ris6+Zm3y*Ls6SzI1Uu<1bVuRMC-v^jR`s;)3EXroqti*uc(xAQBJfn0EHwgXD1(5lmR3Au_Z_Ac)Ps>Fy5 ztK91JD3v8b0IBU-D#+MHSLO0C-t1FrKVy62&iONKO?6Ap{!z47{-E2PlEHn$r*t4v zv4})R(!qenGR(Fy8Kt5xRxfM28q$Bod(2lw#DDkk=la;V@h-IUjM#OE_t>|eYXzOh z`|GC*{5lCF=z516gEug~i)nMJZ(!_}DssyV%+mkZ%}d?gVcubB=p?nFOfvE=wD@21 zmHYod&r92dd=@>QL-w8bc<_~_hv`^=e6WIrznJ2L>o^rcAm_00B%@doIoYpSRtf2o z8pw*H7~wp?^2Z~6F!bpl7VrV=XF2uN^BASb;xIr5`)>*cdC`IXTQE}3&vi{Ot7^4B zD|9wqVjGJg(47_>?+{bDHfi>4hkS@V<>H7TTKpnWBUPKO+UNzlJv}?=g#Po-%l*B* zZ)}f}LLD0QF@H?ue>*8lJxknp@3elL`2d6cWr8=VOV4Z5h-^K=ZOAhgP&@EbN_?T< z9WH1!1djUHa6J@)g16usBo&iar}J`hD;n&$#2k(2f^wgDxnKp|izKx2%ui;*f6a9= zy}I`aX3%9Iv=(hiRwMbL*s6PmK@V zQlH0kIUgOhGQ9;lkWzZUD<}+j<$%jfK#P$QgGN)`QoU1jUF91VF0k z-;cF>L$wp$6S1D1+>X00tqfGc(s8>14Jnj(I*8AMF@$<)5hD#mrr?~rK2aRoOcRH#cXez=LwY`9G@fR^xrn#J^;9BYGmBoIvwtP?0W(- zBu*exk|O7a;B;ZfP6DC042bkvS`M$r@?r^_x^2pgc+iPk-ApQ|%TaB$o+yBoX8pnNcr zE;y!RppqZv5#U=HLtSmk*8mKgb&P9re1%UT>dvP2qhSL(`(q|&}deu<4N7(FVK z85r!3Sx2;=o%Qb(9!Xn$ey#m&kD#&Xc;FDrdiJ_5diCB3ZdT|>(p0N&pY%1>~qH<478qnEEerIzCsl}PBgRHs>P?kJeEMD7FK-d zAGbx~eO69~+y*b9R#YKJKQpAfrbg+45ZkO+gPzuUOOsoHZy>7#`78!%08~muWy;IM z`1c}QY`MSi2Kf-)NauNY17ayrWIJjD=8JMBuWZ?iypQ5aZii5qn$a}Et3;`_PNvzB zb{875ED3IY{6>S4tkCQIjCFDH!2qgH>2)niiP^>1e zsU-e<;O{h8wVtLXpWK^1^0kZ@#P4e9pZ*#MWsDOH<&~Y9^i2lPs>;991Gulneh;ie z)dnn#0lLQjW9lrUs%oPxeCU!!knWI{?hcVuxzAJ$FFzM~??pvv)&m;r zw;b7GWV%MOW95cJCwnN5@%v&NmV66FszRHbjBab&atSzh=gPbLGZL`VYhDQLHTT+2b-%z-=VaU8 zO8bnCU~kshjoH@5dlWgW_Zt+SOaDV0U2Qz}IK4V1g!;6QVBNJIymlJ-oT6PV6cnVb zk@)tnRzxhnouuM?4;tyy^Xj8uyiDW;OBXm1fy20~!XdA9tz_;&f%wtvZj~9bHB9*M z$r!1J<82x#2R+oGovsAF!N?l0752_2UAOxU2aUvF!F>F{J~7P`Rv_(~r^Bdy!lWx` zu(19Ih*VivKa6vbZ1Kq-GLb={AmTI^=2u;5JLLX;=8@xumXIn+l88&$t+NBE>bkA?XGLFQZurLe);%np#LG`D+8e^joDU9o({d)k z4=!?1@}oF`Z29Gk!kZ2SazLkBw(b2B{>ItRWM%zE#ppqe*W(v#Q7 zxC-XGs?K_BX)QIQGR&_0y6PCE8#?W3WBWsl*jh7=_l72Gp)}QR-{L5+=L4fsFe5ud+byFw?DT`31p)XWV z|FiZ~<0U>r?^_NA^D*9TYd$TlMG(XF{@*Xz2ExuCT@yv)x*n#HDmaFOKVAJvGN9@L zW&om${!l-g$-XP#7e13sm%B`{@!HpHe<_`|r;F)#H2a-@l=K)q@#gazod#Fp zVs^=XLdjbPS8S_qUg?GYV=ZMot-0?ixPJOpcL+ckR9U7XrhJoa#vi^4`E%ZpAMAgC zJDrJYGpBhrBRv|cjh0>!JZgwpIUc-!>_-uzlkHutjsXB|gogo7raO$H(Ll3kh(Ju;^rK@~IzI&U}$P z6#X(!j!9ue;^wpDey;X(!Sfrdv}mU&AR1w%SIWxBB$o%W+2=(HU-z>4NPF_dWmGy- zEkv6MF@J05q>mWqM&nvsf;@@1D{yu=c9>(tdea4SlIx!3D%6&!YUxj~2^hF1_5cx? z23=qcUf0N$4C1I8Hxpd6$eedxD+w2(>0YCESIMo5T-bgC1HTYloBrQXhX7&`%IY!o z7Mi?PHuiV3Yq-yU!v3vJzRno_@~wXIrrY4PrXys`E_(BOjd@)G^s%BcWIlZ2^z>*y-dE91S7bP*~>-jQ|6vl^m|%Yq##K@mcUVzu~s?Q~k-u zD;M_#s@5A6wS)#CsV>JQ+pO!WdoPaHKR^Id%L~qBT5naDbilt@F+eGTgoqe00Z#Nw zjhln9x&`DFI-m~F0wD}LzRj}%`IRpyX!KwG#?uUg-bLWe}sW~m`t}P}`B2kYC z*TGQOxxLQ&y$n`7FzS(_f;fN0uyvd{t3VJ`7-%9E>Zsq^-TbA`y8bYl&LpABm6YGq zX)Y0>%y+XIHc*6!ux|413kPIZv4+vu>|q4;B4s$K%i6bE!BRE#i6DK7TSRqiT>TkwG;kwejG^tt=q_+e@xp`(ut=AB%$cbS|xj$yQ?%0e1oV?39Wu#KANnJfBn0#md(CAYh1 z*p9IKJFES}=|a^qQ_dWg6q6JSzi+HMgDygNw6?ak44ZV>U&S0+Q{TgL80QwwdbU^D7(w=%Co~W-R&Ry=6>e6;Nx0$Ducfh1pZVg*CSw|?cz%`!DHjU z;nELeMdDcZiUt3C=S-p>g6%Tz6S*z1#L>2iUoopu;ePl^Hq(Pvg+47aJuFZ(d~h}V zUV;8r5B`d+8~%R8fSki`jrn+tgsRPwnC$j*o=%RhilvtsbyvzshdI$IAx}F}Q*3-5 z^)vrgH><=qTJN&7Eyybz<^!HtqV~6v33T(zpRvuUrd+sGRl^;!TdmDdGU>SF}}~>2JcfINHaeKZ>Ky&VR36NOzk!` z{L+B)c=E~VeofdR;|~C+*c{fbj=heul^`{G|(kiA~>T9PY$U-MrU&r7u#o5W8dtB7w*zB@Xndc64 zD$tFj)o@J%%7Pz_QEf&^$b7&muZb<-k~V4S&EZh*_(F{Way6Z-@6Hf+1 z96fy4w>KWiH@;@4P>?!vkI0F98d*BUvjJ(G3ylk`74_0dB*J;~<@3@=FB#8$kmpb@ zkMoHPF;CqGrO8@*QJ&-VUuoUn!!)U51-|3;aaWVLhg$*hX!vO-g6u+=>On$A$Vxc1c+<^JBAk|mupUq8v z-+@Ho9HZ4zFqW^)l?2*)D*5pvE=!A+2NP-ej9%p8yP)WE%H?8NpR3E-ITx@z0gThe z zM1y(Xh>*bVv9r)fOBjm!<<1P3kJGg@nqy`$(flCQYOU(va|1LDsBj(>Y>)02l`W1E zt*Qz1N}8miJ}{`BFx(fL(+FyuY{${K<~gZ+;YG?S!soY=GwEGV@&zN&zTWki^!80Z zA0kJhQBac!9dBEPF~IKO&0FAay6Ia?OGj1r0^%(}H*UcCWs~zWr%C@2dLJ4~QB6x! z{u;&4ge{x`*sXT@^4wqAf2N3IbIyt!qKS~o*O;qb{9fH)QKVj)EerZkUV107^1T#$ViR% zMuLO0D}$R(hGv*KTvkXYy>CY_&#i_<`7H`pz3r0BZ#++XNtbWR;ok0IZiHO2&YQ?` zlsMjIukV0XB*NyqHcPm6q}63F9PlYGK3L2{&XC_wfDyrcYoEc5vv9ly`w15$L?ejO zdW4ki?dDdm%cmNlygRu~^@n&0!*}7IH4_+3$VQW}*;$anKb7!^T@G8*!J4mfao(py zk8Qi4o~N!ce>)}N(_!FGZ{*7CB~FV=3SpLbSbtnocE>1zjLW82rA(bbDGh3DJmJe; zn!(;I@M!X1cBT~aaw+^%DD%UK`P?PLaK91gVki#0#s$|(**lkA`qu3MuDx7H#d!*{ zF|2hC*=$CMiqIn+0vj%di%9}{!ocuItLkOW(5I6(s-89AoKz2S^87=x!*(^|SzT*R zCDE^o^9JEQtPtaROw?z=8BHB1YvSN==+HEZHtBw#`)*|3*4|r5E0(W`>0*k_@oxbb zH^(+kYMemEi}$zJvgJB@EoAR+I#bmLReQJfo0~_M(`{zAi?ZIHPOc7anpubxgbLCj zOL(uU)IRAb+8-9Y^JZKUgHoF&+_?Md1-(YZa3uPI*nAEk6qun5pW^CFY*cH%V*o<{ zm&!X_6yD6tm+5>Fgi4ev)Z(d-)KWx639UAi3CfB1|pVFt_WGSc(i>$Smg5HZGJhTA<87n;yMgr= z>ih);f?az!Y&x4fM(LKK5Yy|+{t`NRt$0>u+g*FZuJmLOK8^*O*;;nOAnq!Jp@(oW zGyLW*>SeMgg~W+m`-hQ{I!|#&7>&wE8Vh}~3uE_EoTi=9$Z&3|2MqTfU-NuTBS*B= zui^39qvwk77!e8D`5SUB{gL7cn7n{7QzuVO3fBAIdR51PQDaiLIP7oUnfJQ*jGckS zk@se{eoWo;-TKQzzu&ndCf*o-0RJUW6Ali^c&JFOFELxjR4a&Fuf3;_DELlk{7lcf zqC&VS^MQS_cOQe{&X{KYH|&x-CO1qfN<0-P&X@auPg|&xPdu&1XG>+Mmr9 zM0EHc-okXGtm9y;6_u2fTwhO5PhXd$|6)ur3#fi@X<~NR`PIvXH>y|P2mHe>^tQd` z_0N6#_a=8ZJMk>><^Bd^AU!1)Jp2CQ6UYWj6{(>lHBTg)tNreo zTs+HsG(8rX1lH!N?XLBV>Gi61iWXfxQaGF5W)z+kn@bjt3eO##PZbzbF_VX2mFFAC zi6FsvQBdrpzG_IVXp-KXEzV%0J6lHR1Y6mOBjwPc)fDQt7BL>K?d8f5IgLQZ%Dm1M zWL&k?Wxb+NoW&kFu^Fq`px>>>v)%5$`Yailwtn5qgsZr_IWl~O&J=6Xa#R53bbn0G zk?!a&(ixsRxB(goCjTksCTjR(wl8c?gXTX0A%V&++K>*cA7_TjgGp)mk7&KN0S7Zv z1?M@{XcyrdK7G;E8!q5MbBWymf0zVH#Dx-6ZP*`X+K= zO7z+hzw@)-2_F{+T0x)V10Apl@V*aKSAhcxK-3aix^<2<+7U~>GVw}JjkXxndibo( zRSzXgZIe=4xt&NvFm{PMFksBY6gIs-qdSOAA<&0H&6lf$R?ndelhmfr3rt+Y^#Da8R zo9+wJ=BV9t7J;=dpUd};iBIPA5liIzV4D6mj7E*$Afv2P7Lyn~PXJUN(-6JmtMr>^Cp3y$Ys-fEKn+>4}QgFV{P%m z{)G5@Pxq0+hJIzCyRHe+_n{n7qyjq0X@(fhl9XxR^s7;;iryTWspx4?ir`MevA^Lz z*=f!7+x*111oQ0s3~5KGce>u6=!h!PDGga~C4b&8Ek7F)(WHcX_vW5t^KKH`8N4`# z``VXQyKM5QeD@25{jS1iD@R7%pK%+)_hW zInL^?#kV-Bk@I_qJh~rrelx!_%~X&Z2{@DM?JgF~)M6_VQks*}mYq8q2t3*cz=5X@ zr;X^9m|Zh>w!ZIKg!e$+f1Q3%vVH)ldvR?0i&G9xPEXXQ;a$Gf8X8Z5{SNy(6MU`c{Yi4WiG|$z5>`1Eo|DPxaN9 z64a$InY{e?5Uwvgr>^St;lNPE+E3Z`s56oBhTS`A9}XQ>MAc{mF6CZ|j5Kh=gnuIj zz|q~pOwIIG*jIr5BlkofH%9E1Wy0LLDgumQ%T0ytJ2Xx#Dkx}FNd42+upEwr!lZqe zZ!t0L=031Dfu#!Qp?r|8ZP&Tg-REay9LzUl?Y~@h8uQ1HmuSz@Rrzc%HM9JogrveR zGn+*g0v9WEmb!S(Bc>=eR6D{*A&MrShB`V+%2qJ&H&pLUZ)kLeaIz0{+#fe|jnCWt zj#$<}UifFaqx0ivZh4C;_kf?#^n>p1a3D8>$4MG%N9<^#++Ee$m&%)Lxc z2CK;SkSF-_pVktdHaLP{`81f2*xHnyZ@YyaGQ7ChW(TbzBQ+A-pI7E}4OUWacPpcP zjL(f@WTnADj&ME7s6F@wq>c&Qq5i%AQlgyoJ3V!MncD-3uKq%DQm14TM4e%h zowub9a=w|b@!31xk>Qw^`E>m6huY^EY5{BB*B+l3>ve81v9XK;e+$4$p}q_K3%$J# zTg0#E(cs&K>9F`Q4u_HeIP;e=&33bBb(;?nwK3VI212{oU7E)L*YF_;?wBS|_`gwO zAm@Ue4TA^DJCB3>FyR!exp?cYCX37_PrWz6U916shq3Y?T5E(CRiJ};cU{3NN&4Vy z4F0)O|JM5(Ic!RI?%8H=G{xkrO)d(HcgLu4j}aqll5%1WoHCy*+cj~T) zAGxe@yo7+_{u3Z{!~j0cs3>JG!EC<#oR4@xZAhgQv|elHe~ho1{~dsy!LD|SYYZur zpDu^l&ku|q4$@3~CHQzVnSAOgx*%7_g^gvWP^a+fsEiQ-4d;D^c!|6YYzAb50DkoK zM5$yKv2zL0ihdlP4_)^itSX?C-&4ntz+L(auq@pLQO?6q@Y-jb%qYN_Yt77~wF=kM zJ_M3O#{uKvu0bj;Ccwtq?H7Id@Z&?8rWPEtwa<{>*ACbLLVLQ39-C*`cf=d>Us;G^%j)o?C^G- zk)F?M_%$aqoDE!y8ejrAN%_2gbfplJ*+l9GTK&oc@+Tg?RZfxQzBT@^;YDLO9v0g+NBzXbg-x-5XLbg!)<`Bt*KHt>MH?{ zL6v3qVSipl8Pl#FG3kKk=B45$;{S+QB46}sTt!O1?#id7_z-y?_1lHP-UM7Cd1$taO*6rSz(k`y zI~UXWosn|1loF-{qrnUWiEmwY+Zd{Tp|#)?;y`091h&ZQX{fxg1fJegR~1{=w@YBK zjmL3L84%l0ks#Ayyb8(DnS}y;ziM`a;EPD4^;^x~ zcUMKB`AX}UqDmWae?8zvd_I6l@iuO+oQ20ckBl8w|3iqG%OjajX8y=E3*aLnne55u z6~~d-&e#VTqNL>WXoW|H2Ss_Iv|rqgi49&H9ObIcafQeOEF6)W2H$I2p3!u%Fda{5 zGw%ylfXmeC?g*Ht5OJo$-jCVqlxN_BTtAEiY9KZu^o&meKDS6+p$VyF#elD(>{CoQ zp1%$s@@+7tR#|{F(>**Bal$~7uEhrR{=EV&3Qabh;rI=WKc|0g;7T;8;LrrlugXYp zbM?6qYyU0l#Tn#vN!0&~@iiIb$)RC0U-U{_3mR&hh&7krR9G(0jnQQ3??hcEwYt3@ zJs`>9$|~jI<2HVd4>Ie`{!IZl6E!`wJ-;F$x5>iJoN zC3MSvD*=5F4PAJ=tx|t#vhZmCgWFBCWa#$nc(G1bdwfR<{IrzO&s|VTeSC;iMIbR2 zI%Qtg79O~9CvhrY`4Pzk109{xZDFTf7p8@ky_g+Oas7uv zfO*4vj+q6(_;6#3x6;zjtSq39i+bUls>@<@EblQaI55f8l_IVkc8i-a`TFQ> zfO*Z@7lq1`(4v)wM0;X@M|7Qpm-W2`x6=!WEQ@{+nS--j<_zZ)hbvg0+X+EIOI&q+ zbXw*dq_P7b;?P`30VHBX!lVQN|(>M;!cNqtF@a}@7VqZtcr~3GO zYsF{N;xyA8*m-GPUZdu2^IARb$72IHduN2<%q&17eG_VxS;TGH|IS&o$jXw@bz95? z*KZBZklN?2Q0L3kir|G>jHvyXIrx*Y;c{+)4(Xb^jQ&2JOJZRPj^-U7Ln^aVnK7CFCWk9dwiL-0lQT&Pg#FpSW>`?Q?XV3=#N&~) z%vbpgx{7Hq|8sgFH@~9i+GUs3yQqCjh3Pc%bzqyM$JA#xk}mk?JS=<`TlsrxMG;)v zw}^-+bSH^$0Lm+MfF@;C?>_sfz<4nRxw?6*fmisk?Ntf_+!#>MurTQU0-eG4tmh|9 z;w;Y>wF!i&9%h|sqhzNw783Z!uKo!}^ry|Y8ZMnS@W?&Qlq~P;&GO%xD^(US3XY-{ zNWhO#^rKL)&PO4~f&mfS=pg@NsOU2~tMgH5J-{Ek-$dMCN>50*DOC1BEpxnMdu7`) z66rM2fQwr|qcnlR@b72F3e@0;Q-&1L!JTK5rd0^z#BlNj!w$pEVi+qiRN%gip+CM2 zB6@pGWOdJGq+@}S4^jx8LJ8t`MMp(-9E$P9s9^PiP)pe()EYk?#T;bVQFZ6vN_@iv zxz9f}+R82hzC=;`q4m+liHVKy1=w~SOrdj--ezw7D$(OSryrPd0GBZ?LDxj(6(gqoXK%_-DG*ZI=Blxr;%VXT9L2C}t)>W= z_CC+*F8!h*Qp>3yGF5&WgsLoxisUOx9{r8)fo#OW2J%a(J0wLYi#YxNk z89lP`cDN>ShZ4~{K5pb+{c)DN3-|36E=h)Z=6{qdeK&ElrXvmW+537Fi|U?Fb=Nr1 zd+N>KQZFPUy&~5T%=){eN5)qClbSrX1liNgYZ*QG+q&IK2wEUK7BVM&?7o;(JVG0U zrsrEb>`W>a?c(hkE_dm5-RhM1()}e+QU3WFK;%jzLCB-O03Bo|dZLbX?kT5n^i`{kpuih_>1X8b|$ielvBs1Jn#<7MEksUN7^oYh!L=LDR_92%=>087MIRx>+wA=Yf1Ek-G z(*;gq73-ikya;X06^eK&#H$AhE&Wn?@4 zjS&YL3SBkavMFUBvm;y`QVIZI5r4f*(MvOXp%;Bj06v;b#6QLy%;b|ki#n((M@3|k zk?*qPcQgQ-gN=JVIQ^w)(W09zUKl-eBD1%WQ&o1VmM(LJn|a6So9T16+oBKKjkTEb zw8gE_avJhxMXmMF_Pg1qyDaY5*Py1`b}7Yn+86cS?MnStRGgJW2{PTEO}{sGAj+am zlx|12$C3|NI(7E}Emb+ntGpj9(r1#(5BfSIH$uJv+g}PbsmM~KEl8Wp#t=VFex-=t z@OSw?;hd1y{C33uq^JMs3n6kF3V5~%9eriF=-CNh?@Li|3KU{`CFbyPCbTW|%vD*Z zZcJx&H+zNDouHJRKYhG*anh|9Nb<)JhRCwl_j~zI26WkM766b+mIQuCeOqJu)68pv zK}@99Dsd|S$!H>FJ3QM5jvl1^)b^gsP;e5I#hfPfQ)Cpr8_mWuxw1KmCl=SSkEtW= zU5aHkIXFUb2PLd*Ahg!DPdMD?a;oho{YFB3`#8 zq0t}^4Pc2Np9p$dSQ+}V#s@o&26|^iIZrGnP_6{mKIo_L0V>!rS%;AE5#}Qvf~<*X z*K;+Wm@e~RwqnPyy=oP!aM!t*4cU<3>$1FH+qG%Hd{8`nvVp4UT;acL{G?CT^Dx#*7h)G-us()3|I2R1XKuaF*{X{K9(tArXo(o7u8 z9G`|Rit{``JYO&-_*7*#6)t$Vnvo+Dinr(-pF|%U+3xL$@`p+vS4Px-Mj~B1UK0Bg zLGE_jop;AzXAQ0RaNmWjuCAJ|TDaRmmwKVwy~Y>WVi=MjZaVm!ygT*oS_9VY0yX8wz4Xt*t(3 zlSH^x0xbwFFz{XJ9~4_2zie8^c(4Vfa~Y(wh~7^qj~T{9;wa6*r`|xj{L2aKyj#yd z=O@!t(Pr2p385%Why$*s7lo;9A^oxH)qpvukh<AVx7x zG69X6ot+g?=--w7dbEx=G$NK@Fu^OcADGFzx!z{svbNj@U9pSS?Rh2i?r z9!sydN(p7|gs>^>>nlaN+(pXhb^U4>DSoXvQK21+2{MUuSuhd&FNH~Nq$yPWaQ@y? zJ!eV|l2hhoHQ6zXYCX-*OX!^(os#d8`-Z#oHOs%79*+Uj`?-{ql=II&vfiD4b3a{f zd^~N&>_Ve-bni8MNohJm-o`*6*GK+?Mi~WamGn%0*%$)3-F$OGq>_cAe}F!EKD6wB-Xk;Q8%Q~ZUwTJ@Ez#xm&e?}GdCkJGszg8hin=xZb&w-4y;~4Cz1?>>uwU3lW?32Kk~gi z$@gy=@?pOU#cz$ZCO>v1E)&%6(zJNn=w8PD)(>?%6;4h~N%6IT%pMGPn`al7EdQjY zAC-KyPk5@n0onQE*jUWfG0H$|I1jPyl^R&TKs5PDD3$uUF90cq$c=@5YI(g#klBs; z_vW(=vh3apbIj&eteDN=43JxUPoN6QVZBL^Yx$u1;3h6V|K{ zMq!i9-@VQ~D}k8Ak+*fNhkMo4tO|Lcmksq{+8vF7qpE+@_q~c;P|;im8t{<}KW!5Q(&sle<0Sf!>O*YSe{VwISx zXZZ0brtWb@{5?R#%LOK!O9XYN!-M=uafTSeNqYN!rGZtfa2@Okc8Fyd0^>zg zM(;mCD47{|8)ChUX(W1o`Be(}pdNLcX61{?H&8Rc2^ev+!E9UGkR#ThENFm?Bqv7$ zWk-8#+#Tii?^4(v-P*1)ZjR;xk!(CX_~Ado^)I@udLZR3Oi$w}d>n{yJYN_0gJOw( zU+BIE)Qmf;kaDG^EHttK`cX9)Q&d6yBOSy}>?T?C*vQoY|C#Q-g73teH+Q^Qmf$3B zpvR>ZPfuY?yIpc^_*tZsWFs4WEN{HVGwE{62)|-UjW2&4(*&mhGa6r)&S)p3t(E3g zgbJd}R)n=D6LdSD=QpCjBojtMX!Rq?rf%-QeuQO#DVLyMpdZXu7E4ChBU)z!Sr1t! z%#6U<-;DaASp*CyTtJ(95Inv%3o@`viDV8r6luGz>Y4sZ({i%>s`11(}L6l}6g1H#u|GPsq})q03oY-f3Vz(l5^738>$hl^X4 zH?+xrPL|e&(Qy_WM0(B-HLeq5&lwcp*PA{HA;I z;iY-NPX-9xx8pJL6g$y`sPeXNF2`kjC- zd{fmKwK{PMGZb@?VJr|B=5NA5`c0IWufOXDJkjr%YV}-}a(*65t}W)n;Lz>=m6v$Ma*fr2k|Pau`T!CAA@Zh9b>_ z@jorA>H_RI>7GNqOj=Iev!4D9;nzFjHP@Nf5dh6Ybs_QB9@}Sptb6o*HyG6htqMPJN9(Uc^G~FxOOxGvTg{SoB-H7f7|Dj|*(~t~_jt zY9kfxR@IkJCIt7lJFy4SRVb(>qh=myI4e!UEPh*OKC763=J}a6fU~nnVu4!ix#*j8 zk3qxXC`xR@$9Xl=4Jv$Lst`nM*QMM=zY`4K@3kYJYT)|80UAAXB2NVlUtMjWO-s;l zuC-R~7nhZw{I9nubYefx-v^3!e@CPNl6uq4Iu#bZ0+M5d7wd)LS369sA~auWHoAlZ`cK{ej#!%XHvUMThSojElN|_3=0W^MKrAE^>moB5ioeeWl3tfY zYe25EhGm8Q8rLIT3o-*bc|m(TzVBUg2BKJZ)ADz+WzQ$%r!X`RfDuUOx(paa$%-~V z61*Zc3s9d%``T<>3A&n^m}QB$*DyS~|NYPV*qQ1M6kT9njnTjpjijw9ju?tlHqCe`>`eo#ux*4K^r+pPQs1g z`i^r{DiO&R5)<;#De1!J%nOKn+Km|QEUWu7^sNIvtL`CnY`6cOXLHa=*FbVhYj)hk%b5GzJ<9pAuQ&bO^{MZC{w62(4l$yd@ma8SkudDdbYUtyJaQqk zDa|o_`z*9>1FE4IX;YAAiuFmI)1*Qj+vTbwxbFo`k?|LuA$xz0w#yKOmYh_Q3C#lvCxqQURBWC7eM;(j(k}QH{LqDqkzUoY8#t1J zba^EHCvP>JuGoI4*mDj#W!O;tZ&jElOQOvLa$exXxS1&UR;p+I$AIRQ^8 z_jcvST-EmiH6UJU8MJBdB4Kc zI1cYS0ixrFe+*d2vfSbw+N{mqr!S*ck;+yasR1@JY4pU4Tw>rvjlbbcOUfD6AZK_& zm8h(y7*!P*2Z0WRc!vH$A{AWsK3Fh{7dDL4+w@|jTb$8St3x%1CdzJ@)$MT>+}Y>yvk$4j%UVpRHCc(of;yJLoDmm zj6bTdDi9DPACcz+TDHa(3choAze!m|F!=n(3#slC<4h7|0VH)l*~r zuc-#UM^bUS+svU~bbII(kQ+h$Oj{+2yP;6=uwEGNTZN%!rLK*-!b~H%4N; z9sx}(!G+B)R0{joks%s08Cfq1Ms9ktGQtYSGrzlk{Fq)-*E>nL0F$=>=wD*`!;PyQ z^vn0+=_uG=jgHjJ&G&94;YYa)TQ(5S>8ii3!XGw6<}WkOR^P^W1T;*UXRVph+3%)* zdyYAO!BNXaCY#JMc9Dw;^5K{kQeC5Z;cMc zAEc|%?QnmV@{^82BF9I%6xN8bBDaDV*E7&3kdf+FKXau^%tuK_{dc%cC8BJuj-=Ko z*VOxvBR$AF_3UuD0*_ilFNB8W(Kd%{a!MU#QyMDo|*VuB z>Fb+7E8kE!ZMx@Em&8YhY$(KX6c^$2UVCp+lfPWjgVCOs%^@!dY|p!p*R`y+n@F#= ztI`Ddw&$_tHsVUIYjQWAyYjVX*ZcbRhfwg#Syj2uO}`;@vtGn!Z%_2%B6{31R#Wso ztnbYmKE4tOHTACy>j;%kqIRStqgt_SyPro|#DK;0t{$a9mvPodrjdOTivk`)Vj82p z<_=zreg6^&QiAWvk_s}jz>@~~hOOXKZexHI14w0q&*&qANp<~%HXPzF+1b ztDnYi+!>9y!C@0wBn4>}%QpRqMWl5OkSD>9>5Xjl82<|CNuc*9bn^gD24pM~oi*1O ziurzz^G2t2bUdmOgnw-E%W6Y-@4A-M%RFO}Awg&#qoP$O>ttpOIobpEKAa_V$#YYr-!zCB{g-yd5%X3^5NOax5XQOa=b zYWj-dp!NO?fTPEyka_*~{SgmOfPASVpr%NsBTKkq4achEm;LWNQ_#!Oe2(n_ zN6mW$`^2wCIG+1+*n)`MSeKz67Zho1U9$t}p-UO^?K?H*4#@$}Xt&+0UvH2IG=wWH zaqC+ArG=OB5th7~yo_*e3+osoMnHmsA=W7U-7xQF&@7C;SDpmd&x|nMYf3if$(056 zfnGYLH45A2dA$jFW{c!7;?BkBZW~swr67>Av6@=VlDdJFc4*|8?@c28kW|WAHK!QD z+duS|Ka-6NAvd?mFQ>$YhL2O^v`-tngs)FW2Qzn#8B1F|5bE2KH=?hXt_u@Sr)X9F z{&Ex-1EfEB-TOmFRy>!~f5!^J2%?Q#5^wuNty8^l^d zRjB;mp;rge+`~)Ybj{$fe^Jg+arcjsgtNfpP7hErKQz7~@n@GUKf+?BG_KRU-n-z$ zBa-a&T_$-Qgo-$}-OG2Z3ATUwmg)NBmzt5OGC25U%a){56xxaD`^f3W`np!W@)~$1 zd^?l{es~6ddAZz!d)4uH0%FJ_nzBS$6t5wpC2eebFJ5pb0tj|EsG7zfr7HB zfCrCUC1@ikwyto$zXDp_?&_&MZ-1g*zk=6NJl_Y|E)gn>`W1>kjF9qvCjFIPpr#5Lf?InqC|q43f2mP zXcZo15H5I@itb-{~33!xYIf7s)*`Lm7cgCe*yYC`8 zvbo&?taIveND(y@F-n>Yc;La5dbXSr$*8L4uY5K)$CTNR3F5FOZD?@CSqh$SyCO5SW`)0EDt@&bmtq5q7l62y0lJNwQqX+-AJrQMq~R+rK9b z0xu8@%z-OkOA`gXuZ~t48W=wT;qUZzPeby+4zs<-BTN0$3$Y>Oa;_ad+X9nXepW=K zZDXTIF?zONbmqQmH;-kE8|))qXDdHc$8!Gb;LqN=(Mok%5s!$Yh^y^IAf~;SJA2jU zf;rLcNm3Fw2F6%LiWMXZQZbeloc|B^RgtxBhy`;QxyAiv6t9bu=MlyB98)xZ55+(t z1M9Skpq#jW&BxOzAU-qbta&R+aH%Oo5%&yRXr?Bqzejf~`d*PO%^wOcpjI1 zp34T@G+Z~Dnyt09ypR~5qv}V6YpH-}xOI|>5nUX^xJjD_HS7F7z6;jI&vCL4{H#WZ zs!p}r5Oul6{MPVj4J2Y$Xno~N9Ny@BCneZt7Wogz`n- zuSRT?M$sX7GxpxVYOEqDrkrfCyTeL$&-uE;Q~vb?5DpDkZB*;mZ1F-|?>OWr73{HF z?#gK2w6tK=!%$FlNKY+vh2OYn>X;_!nikL*)YSe;W&V%rx` z_}-jo7^_<4!@}qxMpq7=I5*XgkduZFV18*M8{0)RVD(Y^+bJV)ZZ1khU%vh~>i*wQ z`qI}bTU`5R;nuP+-%mwNPx$*|!M~{upc%lVq%%0AtPI97mIdpO~+>^<)mII)nuoW!^OETcr1)s z0{)w0|COop>m;JlAZAIUH322xG0dZr(HXZoVar9ufMIbIhIX!@B1MJ zP#NsVcrd-RG%^U;3gSD@LwKQGXSV20p85Xpuxh%L*c>#=;Bo7GGx5pR*n~eBevuN! z0(#!ap!g(g|F5`ap>l~@|D(E(V7_rE{x_?CZd{G72nRtfP*4?lk9Jg3BUv^g({Y<2 z$=ZPF?cP033H75M;h0MV-tYV_WMO99LadwZ#1{(&uXol--@Xc=`OZ?1>2n8G(4wMC zUWP_~{9M*I(+y25^zIYim22N*dww1TE7Kf*pLcK7)hfJ4`0b>IMMJ>qjC3Q?hU@v~ zV&p2FS#A+-@GC7d&o!ryJ~ZqsKWF<-~Yrp z<_FrkSUDLFisp-yvVbgtwj10jD!Dcu>Pc?W;k`ZY*(FGA^`$FG8z$q=;$M9Vm z@%63H=TcV#zHBpLQwq3ef#Ip~8<`t}09OMQ=cH>Aqn}|Z6#61a5BTYU$B}b(0ba^Y6iSK(aDy&6E-X`z}b`qoZ}83@cR^EWwDq0#$PD=++(EzN?Au;w9-+AqHxld4=%0Qq*f_hJH^{7oL!{wvl!^3_) z(8tk@$v(%!qZE-(7+f(OJ@7xo_Ys{4>PqzLi7rH&UCL|lGS{5`T}~&-<98{!A9_NF z&dDj_(DBi|M}kwJTsGzt09`Iy;9WNV^l+?X{bV#BLN7D=;D1dZ?^a5JEXF0tug$e- zaaiT8)=aLABc1?WjOP+t0Lm?_{gwfrg<(J6&>Osnm*wIuKE?D9*$Mg0A$&SVt^y^v zO!klcSdo%?=^7$CGAsm%YdDjB&=z^4wR2GtHrz&Lx8{VwtkJniyVq9f?02`p%}?Qc z*A#h?PEhxn9ot3z;>Lb@~+4XpSfNUwHB&})2J*TUkk1(ij8m{7BIUzQE?@FPw|3`~GRu|b+nPx%$HOMtE zw9s-OZRsc)$IZPst;$B%p>l|AHnp48$~peJqqUjMihN_?ouK{xgPYBg@ruEf6zit- z0`JPC4dT4+)N%rkR>cqUy#ER>7-Lzh`6xTxVBTTW;mSoTuVBQtu3y(@CwY&(@4Hy> zvvr>rygwoRZR2?>+{7e;Q*v=1>TK=LJNNZ}`r8e)LZVt~Xp^b5FfLm}Hopw4p9WX` zXj@Us#-|!e%wg;V?QxNgs-7>b?9X`V^6$ZPa((&>{9xP6vdH(YB2aI|G`OkJGM<-X zwy5QbH~S68=N=sMGu*7T$ARh>4yjA$^FPcv4`p6SQn=U8b!UlQ$0lz=2S<`G9E=g3 z0Hqy_%7J{{ych~A)sWy-(xeZ7L|e}jtBi%3c-QZ8QIdyU2>)Av!z+kG=fmDd*|EZN zd^IL`^2vtPrHDUG%l$< zT@HKak%Yq~ya3;3LKvu>z?i$ERkr=kZD3SCGdHq4Pzu-Gi)Lg_#B(!pA@k?93ALU} z?^VJfihEC^y^8Jf1=mbndb2p|B0L9_Op7>TqAvHer8Ezy2gqxPHC5d_{CqO@lYSkf zGWXV79hBYhG@KSMcYf1>lV~M6)^Fb!c{9?)vtOF&W$M5YMG_EyqRsit!S#GK!Gk8; zO%w#5P5oZVQv;AEezg#wT*(N^;ZYHr%sxl@UU0OQjD(m#TU95xj}a(ixaRa+D%^MI ziwyZGo29aIs75^+CMO}{P=fE}?cVXG$nqvRUcYZFso1%%&u&5t(D0;rb1W1|zueS( z6XStn@ZI6={3AnGYta70k14T0U&beSG!_1uyzYv#Ba3Yy?w;vdRIkL==Om&}3&bc~ z4sk#ubo~zHIlNZ%-w)%)qUCo}gPW6-R{xe==rpFp7U6Lob-lKQb6n)a`-ZK<5Ms z(Y^&^U76?JM|)#qn?C#f>N3wB28x|+0(;!N%VUwyR3*bV<5c_vH~8yqj?j$t@x~u6q5`(O*188rX-%?nOvC*nQeU=7#q%DpRCZ0tHuNYf7x;@HQ`et zX;o-}G=J{rN=B8$r8HR|l8gOTc>NcfwfhRWxW;$zWc$-(rI9#P6 z1lvf*-6^iooaWAoEW)trV+A(%S^MkDb#~Mg+0N=}QuoxUxis@jJ>XV_t%bE8;&_a> zbN@};X9E!H1k$x4bp%Cmh4w+3$MJHHuGnbM5SLwvv|nHk>y@zTA2y7PswE`9T3};J zFHUjyMS>JiZcdKwZJ^W}uHYP0oFzOc_Z}dWKY4I`@OT@vh(+h-o1;@zSr}Lp*T)2_ z0@hxyD6f{QyxRW_7ff6C=4fn|T4<1JQU+8SK;MhcYHAemgD;pI0$jf_nUo z6-U{JG9e;x!`jhT-0p9Pyl7D+zP88sdEB!7kgZ5Od}#kM8gz$A^aNn96+BUqZxdCM zDm{i7C`%eHd2i@4)H*kIQw&W#GN(aeo%tEieBdA{`ZX_)+1x05S!9pO`X)~;+7a}K zcjc--cid#Vv4j(B7_vQ}f5Wz_Goc6Azg(4ijx zFh%SW2X?*+f)|^!8S0HA>erVhA1Ba7lafqbqK$O#tNe^C)G6fny{%42&k1^VX_ZuG zZ7%!;SNZ(CQ(smy@|cmqBQeW_R1O59ZmPUL!H$%kYea6%3S|yOpfva?^P6LRQ~qn4 zZCc}Wexgi_xlbl$KFyH5vgas*t>h#BG|ALoE%nw+%H}TQfpG;ShcSgf8TL?Ws#9T^Ss!*c98NJ#ktBgP9mJ@(=H&66{pD)7OG!0(ErevNHA6<^wK&@5$7fb zMYI0a&U-N|>pHz3bE_hI2V+hF7{Dj8%)WYk(Z6ePr-uUdg{v`ai&AX2^yq9_-PuH1 z<}?ESso3S9zQ;EJgw5GRDt*l7um7SyTSU+Q)uA~F&@90B+b-R{fjDbRHemdXFOMJjTnQSZJ+Xh!;wk}g^phAp(DJRECmzUw%5|JH$a&*@0 zWXmFagZaQWST^Qs8*69FGK7XdO_jorbyla^6! z2@4Zrui2CvQ4!f&Mn*vjzV?#@th#(f$GwI6%_sH}bfZZr!^!W4>XZVSeAKM?P#0%2KN-?M6j>wIHR{1DI)DGjnLLyT1NM(w zTVE>F9??W*S;BYVy7EXEnt#FWEg#!EQTDFoxi3u)OXSxRY`Sdt2J%ERz=C@cPe$EjvzA`%TJ|wzV*M_bhVeK{%=o`@h$I3!#l1B z-!{!TzMU*OhwuN;p-{60*+2o*nu4(2u&twGRLl4K97dJ{)iBpCE;NqkIUzvzV#1m2 zz_HkQcgSL>Tc~CFrwLxoT${3jKQ)7dqA7KiDI!AV0it5bvsnY}gXGQSExH5`5KC^c zi~tfpP{jo^UUvw4wkW@b0=VgF(s+^&=LAQ07x-ji=#)T?p3-EjE`UZ3=ES7wCb74x-ZLw1EnQuaU4fyWW@$JAH zt;09(nK~TuURHl}d1YJ8!hrAimh}u=vKpnVTbo3jE-(Vld&1ig!O%R-7I*2MC8CPP zpc?W4AXWJgM3(|u(W>MKvSX<=I<=b{A$&=}U0&~Z6D*WGn={s5yuLyBY2O1$i|I?P z*yR%xE^^U=08{RNYM@HsgeWUZE{$;j`Nm^ssw}FQs=Ltj?w-3dcD{0YahmpE2DqNduSghKX+DF zJe0qq%2HGHF0E*)y|k?-eUe))7`LCR>ncOQuWDI8;$6%SV zR9+3}O28O0J|0<*Dnd>0Va0a}Q5T4fGVtHSW0qODQT1P4eDGW;*4Xb}eN?dJ#u)zy zm`uetqm<94*{PoK0Z8Cf&vhO-V2~Z!8k&Z0}w8!L`%P&&0otVo>3#!fw_;*)Grp5 zXHRdzIDiOe5$QBZ19FnT8dc3()gxcP69%hZMc;y_xNXo!ivYLp2gc2C$kiQ)?cbof zeLMX=vGbedjU2zxU%2T>hIazl=Y}a;>_0%e#ngxvVEIe*!DVFR#LH!T1#D@nj66f9 z2Qab$n8wOf(r<)EkIE~Y4{A5HLPtn@IXGg@r(?yuMUeJFzicmyf?`04iW_|(4SU%utZG@i>ybNM$U=Cw4)KRwejd)9=bTyL)F4O28? zvKjWpz8~wn*v^0SpoiLe7$)OxkSfM!5q5_ixyV5~O4)u_42xZrdjyzQHZrHYVdO5t z+r;%s<9u!O4!x=S$Lk=LVd$B*G1gX+d9~wYt>n~-R@+a!G@>pPJ;!qx`2eY?N|L9q zanlL{U@LG?x!u%niD8fffw0R{|DzS_qHpTii*z9qK$eY+vAwXc;FY?Km1DyofQ@ik zcD$jXjVnxjoab4w;v#{7&&FAtD|>ioTq|YTyFS~;Fu{T22DBWTHG2g?GEh_C`uX4}QpB`yM zlMqTslf#);cPOc;v)XD1g&QZv+kOXtm4jMe=!emVqXq>ndRm`B3nbj$XYt_{Og zi`Kdr)-mZ3JmnyTvi_(Na6r#teof-5{sUDX5;6&R>4e$qqd%wK{KRHw|F@oX!(;){ z{HlD>=zt#EAd&~EcP-FKb2_aHZ8c!|>PYurLJ}!Ix7CUQW~YCEl-c_EGY-1^@308n z&9vwSwX1v~)blS)_Ke&ZyJCwFl1?)R8I9XicUk8AC^#k>BYUW_!*qWpmZ}Z0^vABa zDX;Ur(JK08W++92e#?J4*iN?eWnFLb2GOieH!do4EgEzUC(HB$4hda=+${&yztF-Ug!0tjl=8Fe8B7c zd~ylfO52mCnM^B{BUd(aP(YDWKF>}1x-s;3#V@gtE%ErE#KUsl6H-yo(a(UFeB;J#fVSVx=AUiUTe*+l9 z;*0vZ+w*1R@dkwB_m7ua?xVW@o3qCjPpOXctx?`?$S{D>k|VcVmpi;sirD4U%sZjU zRdU@Lr5*HLZ1q6Ug5J7UE((DC%^!sP*9{*TQs?)D*+*~SrOxkf(~HD<^ZXHBg089; zwkG(F>Y}r+i8>509Az1N!3+X z$@@|h8!hf#mFB7H5!gWatQSwjcUJlVLktuytMVp8KvQGVv!;jz{OAvj(@Bx4vi&?* zX;bp4Xv&OKVr2A(CFhlKDPk*f)QTVD&SV+Fg}HQ%-Hk+{QY#h1k$7q=6{}Tw%;^&; z8^_%twOS0YJRR_0Mo`0yCP5X~M|BZU?+IVPOPpy?ExM8CCaSNB*!@OG^Sb}jzn!>1LBR0I z8TOazyETt>*{;{=q|{}N#m2GRZ;PWx9WPDge51SI%v-mjy@61=Z2Af2PEo7JA3kTl zn-!fvQw^B9S75S{H&r=_jV2}Vv%KJ>+iN^>p6N>Oyn?YJA}A^*P|n!AgJ$4zzZ-*; zH13emRM~jKv74pv(4+1LuL@A$1(m{ZncpL1UwZpA>kGaA%c#F$EPw|aELlekHDg=- z+pGa{u|xkZX6TxuOL_%5zX53qF#Z>or^&CIuLQ+;Vsy&+-pR)l4MqKM`Ec=*c}05q zG!RfLBkkwyUhH;1cIfX-^~pLkqhTqxDx3f+S#eoj=eoNG3{$ zUNs~=fAxN8V;ppr@6q4X~4b1X+BS5cjHW1gx z$7bs;0ZM)sr($CDBL9_IhxmqSjDbMVd8(?EXGm~bT5nX?jXvJ^l>-p-N)MeyrwztZV+7nnT3wc87>WHa_q*IE zp}=fGQI~(NH!D`~kULQ>Cp!A)cqXgP{7yC%c*t2}zk@D_htbfc1x2pPbi{_9GoL!&PsBX)Q_alX&f^O7%6%)jhur06d@U2G$4fvkA@WM+2r{|| z%?$)-uYcRgxiHhc0mZX&3tZ5SOq@8m1)}(ZiVapQyZsu|=8gn_`8~Z+gG4qSB6l#w z77i^Il&bTKiy+Z7Q3+4a`vh7^1XWC>yX`!vQH|iw`%m%FB;3|{<|#h|blWNDRW3-k z#3fIxGlf#a*NYw-9JxWs5}GlZZxHZJ-`4b8?N1D;L=~GOi$w}~{DYl31dMjpQ2WCo zmk+;zG-agVlI;XHp!PakRhAn8z2(TsqyzH<#l!Y5zQJEbU$@Gux*?d!gBD}iDV#1` z^U7KiQ_XZ=(C5hHms^7khi&R&BWn9F^B@U%*}BR4-i3j4S!`QJ;|hykmf~%rU5>Gy zJfIH$h#u=}_B-kZE87jF=}$LhsaDr=vIW4O7|d@8 z;n&yVS6!Ag0Rth~8k@bAAmFB6T_UgfyFLhJ!y0RMQ`$_{|3?Je_0c~Hoc$2URkCKo ziEL3F-Y$WDSU53Iw6Zg=`X`ZDa@iLzWVri}1?-ojy6Yb4%<*8*n*NgWs=4Ubt(l`= zSl$-**EpX(818+%@tJMQQ^+bH_4*Ma5)Jo)>%g}(bZWv>o6|)FyPwu8`(wevC&9L9hMx3|!tCo8_yz;;1=Pm{}n6;{S@9w-|DaAg%p&p*s+z^fH(y*`Yqo z{$N=>OdKVfN-`smCHDoh%lAn{#sr!zK+t@!Q$I65&!OLR0uK-Qi(SHt}OcCS}AZ!NXl@MB&6)x0!S!o_))_0;7P`$*4H{d}P zvK&!!v&ZYmNB776hC>1!mUpfumA?X(s|wD4MobjH;~bQy=F2LL&tfaR9@%7GSZk^U^Q2rBmpP_5$gu(KFxskh|Dr9ZpN70iK%X-y zZ4-FqLSe}5ki=_FSB{L59VsaO25de2zd}zgpuG|~CB-F?Q_hX+S#mH+n zcuxK$a8cmx7*Bmu(l_HQI93B)5@ZcVDZro zs=hL~pCXJ{Dy{nK|9;}x0uzd~BQ`VlLDQCuE$`qdZ6z{;uXQ3Ke0}}*h<{W7uB;7L zkSo;?{54!FK+Ykz#}#;dIE=9BxZAr0T*mtF1Jzat3P}}-`@M8e%E+y zgj?8^*3p>OahcUxjPoq^`tw`H^Tl*mOK#i^j#9kx95avGQLkd4X=v&=jOTaA*G`Yj zh(B-3H^v&Qwj+VNdUV?@bE+k=^UYkLVgm;&Da_`vPK-JKORwMy9Ghjy|5ZJsOYWPX zjMjN=HSE6)meC$(RHll40H@(kmH5}YA;~{Isx}n5Zrq47%9t&@e*zXcK7uUnvz~)o+aNVx(>$ z@i2y>h5m=4>T`eEx9BhCQ{G}uWo8y=n=)``J}q^1_4gv$%F4LN($!rL(HnB`6L$L@ zX|F}0{*m0pBPqKm``A?mz0U=)z*IAEY+T5Y$~!_sj!ev7*x$76U?00LWPnL1HZw`v zol>#jZHiCH0Nu0Q1lS#9)NyLyT{8=$4lVP2SX#|{zEPYxKm81iA?GvTu}ZPN{N~5< z?nNccPra2_v&Kdy<|#a$YUBdaNRhW_HpRo&kh;bSdR8P~G%krhK7Xu#AD-S~meV4< zL7?*Dd4_LHGwdkspVTJ!)rWMo+n0E%M!mqEKLQ2nl)PWqeFvAAn{~>z;zR znuJ3*Klqb@nX01o4zO#w=TmfWZ$PkqHB|M?DVmlaaQ8#t4SlzSrh)#41tRLMe1~0w zpYMG41a`o@y${NxRZA^5CSce-s@m)r2F}f%sN0Qp2GerU)ydfDn&;K~i%*q;)lI)H zy%!<+nkl(Fv24Y*e`r6lEQu9|&t&ls@WElElA@%oF<4S`W^cV3=3)+;s_k+lCa{p9 znAZN*%t_X1FZi<82nAleedh5W(ev0?e zsW?&UVpk_uwhP|;d|?`ElhthublA7TtPoq0{?-$Kmm z<{d=1%*L)?pQeqC{cmsx&v~wF?x*2(SNWZ2#$%TQCY0j1D3hPdvKM3SohkV@rZ^3! z%3pY0HvjnNn+#P0{HDU<$%5Nu;Seqbin@As>8@i541D&D+Gc6JZU2__`C{b!UqV_W zeuOX(9&qpa(4@w~m@l`dS!Pk+yC{%}pZ9v5cUbYnCil;l!)UP0A8ry(H9bfto_D$fcFoTioyfyd2#2Sr2&39Gq+wUb%f)7F-v z4wa{X$q+_H_Y)_zGDSF^VQPjNsH?nk5SfZUE zk&w428i&+Puk)*;XH{r#a)@e(t;v3M$+|qgy-oqtx6W{9`)fbddEfmk2J zq&^6ssGzWLBswbtcd6YM)9Qf`4>`FhGwkp6P!5!&bU(@d`Joc=>7xPxBwu>N%-+g8 z?Vq)TjOGF-hx&!vb;3?Ud^=jExf^Z%qKlWZrDXKt9qG!Sl5AZdtM@R?P2TMRXcOgMla%s1A#WB2HQg?uJsn zUra0**$>{Q#RbJ~=9K(gSeb68>N&sMW zj5X}`qhNuh`b)oJKs#c-TL7#7Yy#%{=bCLc2al4Jx5flDgL)hytm-QAm6ca2xbraa zvXp$!gH}ABZVC8C@#K8V4#GEq42PWAUR-U#;t@;B@C%loo|rO7Qt4&WYJ(_|H*$o# zvguh$N=o?puVjXWhX83+Ipct{!o)#s9>sIBx``&jYTYFE!aT5IyRYmv#> zxKr}pyN>y4YS6f?m1ayfKD|uM4L%n(M{msl4;$Cb{=?YV7;q3=w@AFnQ9ru{lCISx zcJ((~}E}eXw){X>aVqaZlXi+L#Ve)XYK}04nYGITtgp z)whwZQUYQaUadph18II5RH>fo^4SJG1b}=q%*g*o?bcSmgJos&t5}`%H(tx=5Ej*( zVA0XiZ}ZR(DW4{guWP;6V0^n#9%FyD{$(MMPW%ru>PAl+E5;r-ShCbWt4~9-e8KoT zpvSPUet*?lC^CzvO^YO<8=~5X118%AM`1V;8cAGRse4~sB!8ilf$-*Lc;3xZ_x~g` zyP!&_&Cp4N`x&4%QZMLNvmCHOfg`rB(!8KKH#)KDai*kAVA|)g_IZ9YCf|~SzN2Ey zZ4f4uyI(r2MREPdnCBNSbB!OjKgIlrLj#oI6VOYP)FwnQ84=idp=i$ttPaf(2UVKu znn~S9e?3YnqV&7YWImpUH~GO4_e(|ba1{@mX9B7sZw+uBUV<&JmED*2PrM&yfnBYu z?Q7|~DGm`8@KkpxJpfz%<9j+<4!4P))SaP~oj6SV`W_;jg@SYi-5$gRfd~&jWZuM4 zi+*%MN!>IM@>T6U52_%-eEr!3y*6-m{9gKUvJvsci5@K!k2)qxLfK)BGRNRe_V7JW zC#{&L`PF*^izeaNlxD(t&%sdjWoJRA5*}iq7G2T+TAklA5t$( z5MX$_CR}3Ffh21SY2N7ViQdGEFC~ra7QjniGndI1q&Dk@qQ?YIkz(3&<(8hFBOowm z9Fvn{Q5Y@#RMw)UzN|ybVc*m@b$A2VjWYEIv5+s=hT!O_NMfOvvcpcV_I*1qfSuaW74K0L#cwm;Icp z;-#(jjVHPuf&eQKn-GTAKF%?sd7k&jPH>B4#277IIiy}Qqqsq0eJ)elC(W==DcwR z!7TGjLDtEc++#>~zy9cy(erFOb;Qf|%ad5hg|lDZ6jAA}!Z>LRAKWHK^LhDxtXJ{- zM2xCW<=WfN!8^}pMWzfx{5Tp3Y9lhN$qub^{W`OEn<|@V0Uh}Oq4(*r?qm_9$uxqw z$TB!rD|PcMa-=F(|MiH#p>mwZI~vii>I{x4tOg9-+7f&zMEKIol%F*st=cf8({>B$ zpNVcBVXcR0gh7#0F4C5iznt5B$D0<4gc-m{=Li;D*;dS=DZOhswPK?aD?Pj5EyL~D zWpXH?vTwKL_oQx!FJ-<0S&;3ZD#ER}%_k*k1Lv0-{B0ri$Jr(Vf?v@-y#VeH^>2_% zTH1nQ({N`H>s+#}od0^04UR9-=2;myXl3Li=21WmP}81~UHn zSLn;AHqr5tSXDzGo#&+6Bu zHBT*waJ-koO`}KaalJa)Z`)zM@Mp6Q3f84eVR(JrR1u1pvW;i=oc4;^)B?VbuBr;+QjCmB=YIm% z;V^FFb!?tdao6)q7SS$PU9^5y)!wDDAbVRcF&_ZuLa&B!CMt!alj`{zcjq;9iLfj; zVFNa-EM#poQ6fHW5te}$DmKZPhdFr)+2Pmnv3mvL7e@zkYYgtq$z*zJjFg+egRJcsg?Fm61n}R9o)l$SZIN9343bAle^CM#lTUTFGgUefk8pnUM5) zLn0ipeF0%a=bZr835s_^I>J}!w?I8$vi|YRUS-xY0&(`P>%k!NmzF)DvksT-!C3nt z$o;L~?IQRucy`e5FHQyH^$hyokv@OB6nY$^+3=dF{n^CewsF?iWK*`zklfV|v-bMq zCV$+Wn5$H!(mq>_uy-Og#Gu2yz!l3WeLqKs*H{o10JXFct)p;QDEY8OlQ)4cEHp6A z2K5!&`6qR~s#z>|%Qfsy4q^Y6z4I`9LF@^VmkYJK^Jm#-IDfL}_xGxME(FRl*ACTp zwLa1~t1U!A`8cVLNg^=g)5{+ZsaIyzbGI0udv1@1Utw|gTSh}F(m7aixG_`W;WI1P z-r4e#(W%eMLiOuegybxrW|M;iy|j);HmC@^M&A5Y|pZ7ZQ?Y?> zV3grmxW6!0hGzQNKD0eA_GA8==Mz97g#PR6`7pQ%l5x;w zG^{-171wysf1`gJJ^=dh!`a?Rgxc~n5Y938hl68GS8Pe){e6t0mscku_F8ZjJPGff zdk_+#fC_==!jH1{x|A)`hE%zq?>Wsc%~&AO zi{kp95FCm=`4jvB8m&!b*tU0Y=>{$5r8a8b-YEI~{#c z;Wyi(htT;Gr)Sqk``lk>5YlJdyOTTqnw!6gjmrjU)oz9@Gk*VIUjZO*r5tDPDHLER0^ef{`K}NM zrQc^vu9}lxC6@G%*Lx?<;pM3hoTvP>E!(#qaJJs%dAyx&s1m2m{+M*liUM~|^s%T@scXTxEQbtddN$F$A8Vt$F$hiNM zlZtHp_YZnzmuI`NP7E1sCkzA5?=_oXl=cqkqVA}nhA@8lN|i+QuHitte` zF=2Y~l<70xN^L|E;2_q1lol{Gbm?AmEi=bNk!yvgh&2V`FWc0&YkmjU zanHHScV4y&?yfh3A<^2lS?!*EGrFU>Ubm=|0MVzFXrVhFDtGLG0nRcN^m175EXHE2 zN_8(iJV+zzHV&4@>_z)CQvEitzErW-0U;09arARRB*PJqa)(7r>ld7-PslTppnI?RP`UWAE23M_@{Zw)%{)yS!` zTD^u6;ThdA?x@W~(-Z0IB)VfDRyLj?P@y9u`D#DCd`YA0hUPgRO$f7TgA4 zkW@4wBX>=uYK29BrM^E%#@?Q)l24zO0^zm?+@$225tU`fiPvFjL_&)BTcFLmxQ$hq z{W$Q-iWT@;$WFqR1A|>mPdrN`ZD~rEE^4Zh6p5rTCFAegQ#-SY0^oHZ86I3h+vfEc zj3suGWd_GUK29Xm2K@Pp3=yz-`D9?XJCa&?d*|ft9%jETr1Xca71z!S8XJGLV+9l! zQj+nyZi}TTqafxBX!1UmIlU-Med*laHsdH)Ut&nLX^mAJ*=hUrhl`p6*jHR-jB`GW z2*tlZj|(E3opP3XM*$+=v%qhC*o}o#f=1pCJPeXszZVKj;hieT_ids*%wCo_8x70+ z`iR;w=mvJ#dGg6Ax(EW4E0?h*Iy0CX6t1SaB@zk`docI0~v zm7x`(S#h3vCK8@UG}hv7-l)YCLZpb7;4Aj|_}7on6cRz_Pdb!&e_dN{-o8~3Pvyri z`VegHL`kEwp;IHm>J!^PVqLw_8R5wd;u0E^mK&&FEO_>5c!?PUj#c zJnL!kF2x>8?=CiPEkEFHa2)Fx0gj>cfm^EN%Twe`8RVzK6i!&V-rVbq|_va_7Q#rOTN$#^QyjUGvMQQFBvX2>oC)lE8;FCpB(sVX#a zEiMue+j$=h11cFAiB|G0YgtW;9wR`;CT6Z6611s1nT@GRULP2m2riW3g)cFxY{MNf z)_c;8ybpWOsgVw`ewD^sR!)kcqF8zlvE7b0Ww15q_u8lzc+@GeF;FLE)BeII}gIkTCP-Pd!X-mA#pwTmxf?3aj5%!fr^1OY5@zNjDV|ba~EKOd6dMzg__=dYjIe$$u3~Y=QM=$k;az- z9(O=_|7rqMX1W3yC9R_zVEfCaGMJyvf<8&DxLG)JuK9%BVyRM7bJ`$3KlF^5d2>z$ zJ|plCf1Wc*JX%5-aWV5X6Go;+3Xu5xt;bh_u6u{+TOUCKZItg=(tTL@1#7+tXLFt3 zYw0Tv_a=@x^ny~5Ah1Wg@0DC)+FEPZrO`>Ram0Z%M3eAyjEB9!37azqk;!MDhUV|rK}2? z2NeHWvsL+p+>cgdVwk+_9O!{rismZ$)s;#UoQU8xC3KndU-)|f=Kby$1Cr)sMBlHi zY(*nLY)w8t;|ah)FgOc@pJ&y4ocTvjtUm2hqtQ+v7Qr@9GU8PD;GMXZJSAYzyBwS% z9V%s2MqbfAID8cQRPUx;0MKp=EB^BC@-6z6LKW==mj&#PGj+A%2t&U|J`twSA2acy z#|&YXI~v_SVejp^;HWHarG`*GM!h;Uc#r(scTom&5uP(Id9MW4`sGzqsr8nJQE#=i z!H-UEH`mx|AFteIwaGFCVNbUw^%J7E_XP0R7!-vVAA13yvTkHM9=9(4Lb3*8TinoN z6Zz#J?N@fg*40Bf??G^r{_gO&fs${h z-aWfBE06uxM-fsv0iGqJr3YUmnIin{UY}wK0xk7k1{!>WnE=sMUmQ{8aI3VYB5bxs zS}kclBcf$4QdSmsk?P-IJS}&R9O?+x&=piaz%Xxzv(1MA{YJ+KoTAei=G;L678l7S z@rnb}Sf~Z{=L1&??1IBx}7W1l+`{yS5+Za8!Z=0$j zy+o4-#+$G^f)+pPiCdX9Fq9C{D_+$^pB2W7*fY^wU3I;T^6}G36v#tAqW-ZUc-tGa zY4QM(5dD8dokc)gUDK`c;1WWx;O>DA?(QDkg1fs*aCe6$xVw|!?(Po3-GVjT^M3!} zP8sxsefF-ZwVqX~T-RjPoYpl@8Lx4LrRbPb=IxSdM~F@qrc7(o5@CDhR^vMzsc-3M$gp7KxyysbuNa_i6NGP<6$O@S^;`r#`XU& z+CxtACr$w-ku0deBCElSn;Ea(6}Okd*##F167_-A6rbY7!K@`8=q(WL%|aX-9+JS8 zo#>eVe*0K!OD(Ld>I_GSE3hzzh%x-mA;FyQW8_RP$JUn;b-l##&MW5Q1%e4>b9pM1GfDg&$x<^_OJ$s8?S-7*9Hs8;apl z582>9T8Kc~ijxnsTdc9WehHZ~I-Hhv=Y!L7mY+5d7Q_`cruq*29;2fl>0Py|h$0L?3>*Ika?_ZQ?-t zwOzBKMfM%R#gVCRoWP2-rG%eIrgxloBdf0^9NI$dT~c@pWps8>iYdvS2-zu$QmfHz z715CeuhKDiy+$T~$;R^r{O&G~A6h;@@RxFmI^LH2Uls2*{UJS<{I7}eIZ+<-!O;YA z-}W)%SB%0QXga>La)5h_1#je#s3nEIm!s4KEl11K{X?0$LzP=;KHLYsb~3Nmu2)DG zP>*rS(mey6K-RsW(UxlfJ>Chn4c;EP=+f4mCW(lRQlQUr8`SOE@1Yznv$2As)znB z`pED(kLQ6&y+#7=+YL?ZBXgE6w{iYq-mhdFZ9+w5z4M~VrlOX%NmB2k=tJg-AHETE zlMATgaaz+nWdJT^R4W3YH{C7Mrv1|H*2@t2>6e33eQF{=E}S0RWf2=OMKNkOhEv~v z37eeO?RuuFf+T66t3UiE=ZM~`(Qz;Y2`o&sNqI80zXBE&5yKRjg&oOi z^)qZn*+x>zKjV0!9P=6_lAhIgV}Ub%Yo|=F;3)FGsjjPr!ts;ev!NpUogXV^M-U-% z8KtoEDL!WGw^CePg|-@AcKQ-F>6L9hTFwtnMt!Jv=5^s$h0qiUHuwojpD@dy{qH9M z0nE?Gornu92=Ojqx$RON;@u887xw?isx#Yad5GkAc2p2&E4Y13l=vC12%56*Dz9R` zR)ovV-2XFVOq&2j~|%vn-7|;(R{^MH8Mg11&{xyD;7@N}BzEkqkM7Ty7`a z90OdBdNZJGY*uzb`X~N#P_b0PnjrLrJN$qj;zxe^Mn+>f+#vl!IpXcFYoQo&JnZYu zS-zK=x~Z&dQH6J6^uB9IfiI7$<80e%o_h`4@$5 zay=zmITX1R^*i2Gd?JU48KVQay(FyKgsVFZXL6;gpf=BV54hv1?T|tvJ&##=9sDRGkpQo-n7Id-tU_KV?#XOl8TmC>8^UQ z&`?eg*!`3^|GGGwgc335r!Zk^hTm~aYyAlBk5+d--I_b!(t8GFqI1#9Xa%Jdh`l?O zjB7`pD3Lin&5Ww_XBtrl!B_GGK<>YFUPwsnq&&JrKN)>5cD+|eCFB-2f%8F$Zy4R} zlcy&B7xS*4-^G7u9!XokAw}VKEkEQ>5NT;Q7m&*~c3;P?fQf-L9-7`)8l1g7{0#$K zdhQ(uEJq_;?9Iq5Lb=gk!jjpP0s1}%f9=sGS@;gye0@H^8Mx*~;RR}WOnhq9lFrf` zdE?Q~A6~JsqF& zHH!r{_P2rR)tQ+#+d}ldx~zu>&?SAGWk{ev7D)P~J7{vUBAA%5@?FUhCD`0Z0%yJC z4f5jt^^R*RBQ7|G-uh@(xXEsVXWeWgwa;pFZWrq@{s+eon@q6QFkh}4gqXWC>I*o! zD<|Ux`T5f{S|j{;MyfB7$5ix=vKd&lP(5f?jB>}^z=!G?Czsov{Uia#>Z8h7?tf3! zB{M`C)6x^WcCiEu5xuK^zf?5%!iy7gBgjf;8j)eHX=MjpfJW5&=7AY)A|#)T2s00t zJ$N_jIF&Bw`=FRWN9=odotM6DGy3N$r z2lY#-{ECBNo5=&|3~SiuIfTYSMUv;Pc40R%)oKK^&Qv$eUzRHbQ<>5Hij!Z zMqaI#+!1Y^hUB)!!>C~JHETx>vzB`R7waEGWLC;P8B_m`(;3)i7?9&(hgtnXvDo>k z1XZc8%o;vQ&5xHv)sVYnIBW+mL_>dgdqrVcF+;=7q`EF@u+FoWQgJZxGSWtRR4Xm4 z>ei-i8I>|Ki<0sun>^#e+8t-c5fx~VKl6tG*25PJrAUj z49MmS6DHPR8~Z1|j$BjJ*!#kTfaR%;_&bJ0PoGbcz7J2hs`aOUgDFVa1nB4A=^L!b z`&6z}LuW09`B*mVGMZeBooAY3JWn;us$QWZPU%q0X{NG}xFU0}8;YqXa}kg~Nq&zy zFk{py)M{e%@5?mGeAbssTGm2ABI4!yF)Ai*#jQRtPS@50tH&IamggQIC)5=E>g9gf zs!(a6@<#;hu_9R5ZY}w#t9|5=4h3qPJ9~v5b8T=U@|Ex;ZGG{DYQ6fgb=shAz6(o4 zxSHKv9t8sWm8;ntD1?5VlAL{Oc&$Azxw3cmulpwkKG%92f9!(1kiT-arNQ$*mzOoo zcORLFy5~N@%5}QunwS10K?%z*CT5}Z$h8iL!{at8ug9Os?kt{2hHCwp16NwuFj;eT z;sVjWM%*i)=OS`cBi#+^TvD1Y)ZI-2g$?ihrGEB^w=$ovN$6Da&yf9T`PtgT&Z(^aE-$Z?=p9v?k!r)4J z_x#pSjRX*BC=O=A;@I1m>M8ywLxKZSm1^@98g}MCN4de|V_xF^2;Xw5L{o!^;V5HO zQxaUcWC)>8kP{x&S4-*IQ`41+hzlvXoEPVJZL?5KDVF{lI+ey9?{#OqCD9C)$*9&( z<&R_h@y4!lTchr_49FAet>7f;|7pL`-bArr3c33=WX$hcrT1>Qv-SRJl}G==m~P0I z=1>ZdX~4>jKoINviVccjSx+4Q>5iOqmzdFo?okb*xD&d*gugUNHR!i6gAyDvOwTz& zQG}_2kR>`ktJYl<-}Vcu$&;8meo^^A_5Z5jqBly1#*g=4JGh|({IsL=1`GbpPX9uQ z1oae!&kb#5zNFKB^cgK7@I(Dhf^5qLjMWsEAA<{}+~V_}}uh6Z(RFUccTp z!a*VLBhhL&hX2}Z==(fMA4*{&(jKX=tnqS$$Vl-6 z>?VzQpOV-bV1BUBvJsJ0+LcfoFzQ0ioLO8vjCx-`aJjt*Xkcum&jljwIA)r-dTJ}! z>K~`{)?4Et1h1q{8|W;wnwZzl1r_ui@VX~N>+aVtiDuxCV6|U&7u~g8L2!vLrtgu zOZMoA#kD@XsQXdHCDT!xibS634q2489_Em}oQ zJ>4E2EA|**d1veS5WyCN7hax-w^m1JSD68^P@k6x%s+oW(MkB^Wc%w!+U0o8=4ewD z5Z_{r{4&OEm-n;h59NIcR|Bi6H$Pniv$oa&cBbPv#q+UwF76xcmg~Qb+(7!6>OiyB zrsb43{*AZhd2nLJ1b&Ey|H}@|ptyRRm)KKi*-ztJ){oO+Y|Bh485#UbZ>Z+=;n$6? zW2LU267p`5Wn6D#!7U~Yj`DeQpft-!yKXz+?ZQ2=7xbYPJmva(!_#Ubu^y(Nl zU2q}H`>(^F{|DaU@CDT5cX@C;t97E+gUR9zp~c8x<;Z{B;58~_(Bv0wNepKoXzBzk zV)B9e;Vc@Gxqb#vcfr*uj_ZtjASU=tA(wzmsd=#{t?39_^)D&ur?{D@u`_>7 zu6m3GeuhQ+?T(aGhJOW8I5?wHZJ}a3U%!`fGQIwUy1ZN0hV%ZZ#Uzr++-rJ2WrT~( zxxser7&$ftm(+*o_W7HIhEFlAPlB>M6;+YA_@ECp159wD6J-@E`=L*CVx{a8!0>7s zvM2dy_|6cB>-+^5vnleMA=klJ5b%37W{^R~0kt8U+n9~6c!J&rg`;us|AI^4a_$f9 zH1+!Lk?{8#-~Hd4k8}<$FSi0wH4s2ut0i?v<2D1~z2}?n=OjrW>w4Fv+cA5jP#O2D zjGXJD{YOoNrLz51nG-%Wr_N7ZI&??Wmzf9m=7Gt1wTTTUy`Mrf%l!^s+4CK5XGq?% z^&K+QBQ{&~rQR!{5E} zH8Dhbx8<##_kEJ6lU8`kQDw^Qif~|MU zk+>6szW-9N4Ay70--5`zV$_?B_ zz*}n!C-gQkXpPyYUv2@WTY|RKA>x`tqc}os+k5Dmt1)-KoOtD;8NG!xu5!DblaE>3 zAha7$#a$JS!@(*x!LHla!vdLFWzV&BM{c8I_P^^DqDx|f+6pU7c~QyPD0$!N8n=;Z zxHJuZ4~_`iD`l=wah}V+=i2$N9f2j)j z#Fw0hgGCx(*T6e+e8t+{F{m%e_3^DiNq*AIB}M{Ib6frFfyCbAv^*HgS#?0k+V$_q zpR1G?FpkGL`kDDWh=c4Z3azcTdyz%H6p7V$S>k4N5upe$mRrHi!+A(emBFipc;!kh z1xH~tBjaaXJt`k!(;8jZ$GpF|4ciO8?#T*zr-bO?mF*h|gOY{~r_V_w(7ss ze-sAWOvV%ASI@^CNq>M~rWlgjXxf6rM}ecfYqK;8)BJs;zDtPdauf^3i$`lAe0noP zDGiEjnTjAJRP^z1aBPU4>TiVUD*d@JIFVTiyHBi!6j~mhkWo4#<5NW$1qA&#=I>>1%C`WaU*Vyl^?M_5)6A>>>VI4SSdizI*j zq1af3<1_t_&;2f={IEJ-x8nEo>uAG%iYU=>-&?|`Ige)ETt!MVSN~ii>BCb3RUCz? zIO`|q6XB&e^H835U*~D)`_7e8Pp^%hq0|}*{1x$Dcs!nXCTU*c5n{AcM;|a;E=jT0 zf$uPT)}anvvJ6?$SBcVQX~7iDQuX2H{})Z)!J|+XBR7l|EyYU|9{|U#F@-pt6{h(3 zb4Lv3u#-Z+1HILL)6wNV^p7>?#n$f$&#}pZWiFnW zyF;jRHQ+MlDTLWl8(K1-;=nf?FHtQU2HA7K;J(TP9@+ihIi|jG;zjxmeJew|I?m8m zIioK`^n-&F3o~_>D1G|?C$KGof(GWR!dDTqaV?Ic7i9pnlfgxoqir=@Wd1OqABmrBS0c9eg{uKz5^C-)ARv>e8EDLpvuq)&S0g(ZEC^ zc0{?Q1!3n|Sm^=C$7@t9l~|Lrj+%kf-LtPj8M2UlGO8dv$$- zl7{<(aWm4puY{^up!TtrpyX6Axi5w|958z&uq5WC!4~~n2*G7R`E2=sTA3WR6BcD1 z$!sL|Il<%DtU~dN%S?aF1hs&|J)FQCmqls~Eamvrr@e8NNRX;bk82mO)iH+ws&L-f zVKS3B1|I%0GAcT&-QncS6m(FX$!(9bdhD#Okn_jqbss)yNTE+r>uleVTNamKn5|l` z`D>cBDYAkI6CBr^?c+3S(t$#Dv&U*S4(PBZlimNm65iTOH6!Buslt3*hue6QzOwVb z(L6@^#a35W0apWf!U1sZ_B4Ut8OZ*O@=fk9u^$7L|8a9S>e#W_Cs$#x$O)g;iXe7= z0&3x9^*7y~{-@8CUs6-ln2r(uLmV!V}Kgv)F~eWgM0_bUpR zy#&rk#rw_=ZvZ)3fHvZqbJA$?O=|%$I}NlgI>i$2xYPNOe#fc(LMo?mpRa@&L>b79r55!ukCp(A`!DX{1yM-t2xObPh2#NWHkAm2;MzBRZdiTV0qzxANTj$js_OQ&d0pYRs=7PR}U)n^oxBr(sxKE z2$;(%wu5o@dK`q2FbeI3xO}>z(MF8@u?hp-f+t1#6_#R7#F)*IM}?JjC?P>3Y$+yc zE(0D_cF;uIb2l%F+RldKddE=n!?4|+SqRBaJGI19M%6dLTaM4{(AiQm^8MzQXdUz59XIJ0cTv>?!Czz-j;&F#?}0SbH8}8$QLvMHuVJc1hwGSl`J?D z^{vl4Kpr5l*()}y>EJm7Fl7&Q)u~S=qWQ8Yqms$)$WHvh1M9;Y9lAKIy7y5>>Ym$o z5yGl}*toIC!F!qMDp~7dxHES^#G29U-dkAmw`9(tMp_~` z@~DM!`0aK7tHQB*HGS+sc$MT~+3BmxpfC({M}jZNs3X!xwZhA2js&01zTpuCLtrI* zF8N*S4-azmJ6W-=xktxk2!Oz?m38jS(!B@HKJ)rGBH)oYfnv?J#Mz#(aKOyMWcw~N zq9oizJe>uVEq!bnnM^LH2<2uZ{ZUG)QJWUAX}}sDWwe&+04w18!jpNAAxx{$HtWSI zqp$4}@D!y#Ud8JRVw1d-aWXVjM?izMe6x_4IAov3D?z24FGGsM1XWyh1@HkkL)w_q zt5Y@YKdFOkFFRi@mNp+l8gxpdBo zrtAUj_Fwj8;|JPNR3`J&b9)k(-21-!V&N4UsCMT&Sq`Ht2}sRkcALnYm#qaCh8^hf zA{_?4_bCJG4(&Sxg?jHjn7!v_lR8yWRm1c8J>A`ifmg7Dp*{yl@_qxhnaSchz(kAf zOjT)R6cCO3+h7t-47!`(gY-cEQVO!ZgZWhA8nslNK!2{R>JNNMMNV<6xz9dd&y|3& z6MhI7p2)SXh7&i^{O=JH@eh#uO}(8@);D0C-}x}eg3%G`jZhk7bhG5Q|G!u*;amPUxImBemauXJ93bzXZySW1)@FLx{9rA2 zfIx)nG|nM@xyCFFA~vyIi!is3s-bYz-F>$0$afAA!64U5GPB8VJz~m}|4ckGZKmGi z$ruDf8t7C|w!&W4sAi@{Wd`;ANW{4QRa8}XT@;;2)NXTxpEn1iX2r>8W;Ut8%v(NV zpVuz=z0{EWY9#92f@bTw;7_r_bvo#>e>P>3(4AP|$g!XOs?fbr(ZR_@C6K;?K5HJ@ zN-R+K_St=00GJjh&w+N$LVSuX^srTZ%u1*Q?nFaG?dl9yiA}S;}IN0Uu$*ePvGtx5#>py(neFk;f z(yhqS*bPLfVCLIL_#ThWYk{7!&R@rvmp@X>g>LGSus-~2Wy{WB_b>i6zP*ZXvhf2L zAe^ZyYCF1*Eu%Y_?gBDndA%;v>)xMz=*Nj*xjbiLLOOlj3j zn>W}gwxCU#ETlAFxJ5~>+svo|0gxw9(hFHlRz#$U;1y)>&nAht`0lu}=@tM+87g`u z{)OmEDonz?w6oAG)NfOo|CPB3*&bBU%9=H=YD>n__-_R)G1WZH zxG14EDZWg})eMy|D7oHi;3uxD_#VT#B@xX^_>Ru!_k1V0w!x)(DV~z1G0h`HE`aSA z%D8&fEVrYGm7}A-^xuc^+9qP7HP~cUZSO-QMOXC^j;`90zPuet0!u}6rdw9{DQ5kn zn1a6C%y(EmIEn5KmHKa!#x$Zz9}%3kOQ-zIFJUJIUeG;>xwai0z%PM+XwWZQfvsG& z?}14#aURFKU+%em$h%jVhX0zBc5p^iphnA;;7E>vP$pq7$bTl^WH{Y(ZVjot(EP=# z1WTVxq^A9=bd|sJF~gPf!SqTpQjo&Jr$nPHY~H6A&gEf~h8C`;BBfom@uFJ+Hxet-^;g9Q*@{*=TAwpzkeziC7x}A~!%iv@ADuYdIfSIP9TUxC!T#A75L7jt? zZETJojtk`VDs`V-hTl+9n`$$jJE~nbzO%|5q<6g6QxN*B@a2*)^E1H*g?!WnE$jvM zk&Gs3xq0^G7{v-G7mSDr__x>du=fuk*C=Lm)AJhHe~iNisj1AdVqSsyDlBnd`XHC8 z&c@_(#_N!bPjRClB{wcJmD9lt+QBi^!5-7^VjoD)Ye_?1=?OtM%wLsKOU8{Nf5nzA{E-4psjVAAoJ0|QQ%7T!gB4tTAMv~A(AdmS zD^7jkvIJ?W>*FR`ifp~0tu1P|34IHHGuBV5@S6$@twB8`37XM2Qg)pYr!6G@`!hpg z2(N2`T@(3xU3w5YJNprH1LP$8(W9J)7=PfA^#+RaKeT&1LdX;14Xw&skq&u9k3-r< zKE4QSmK$l*p&WsCc-z587AV0=LWolgc|2Gf1R3$TACn>qdG`+zeE2@0-5~PIobNJ~Ff?)QOp6F1v&-eCUG6t!rxd6Pwq!YN-I(r1dn5KoOZFZHV*shb`L^opfS@ z^DWo=0KJ5o0gfrj0+$|?!qX|bD@Qh zBPMRIqXE@nJv=%iiOo#mV0Gj~>sg)hs9T%Dl2@aa7l~`me)h3jO5P+Ny;LQ|S0Tk& zfa#wIJ~i%5<>@p$u0?x-CfV++Y!KMWoH+4 zr0Td~Qv{PRX{$Om431H2-Ws?p_RE!5T0<$b9k>VJE2HzgRg?nr7D=e+*57uk0aNkNP>^aooJfy!I5 zD415&*m?dqg@B2kD}hU{QABrM{6BRI*TFu}(fUsO8wl2_acKd6p)7Jec{qso_ zhbHgl)2F)EKy1gC*nxBYaxMp5$7viE^0#!2d>x@g4KWoV9}$PqEBi1$pUXcY(RWeq ze!lY*1U*xEXNe7yGfh@@C*~i#S|G=mR4CnrxTG$xx>CBo{KcB1%+FqnbS}S@BYksy zKn_q*_fGx8(s<^ZK)aqeC?~I{E%J+=k7-&J>)DUCtC0% zW&0#;AHVo6PD9+Rkmi_;{ZPqv%!7DnXQ=BBAk3bcNA8;8PoAMNCpy_u=@|wNJC^Yt zyup0O)D`B`l(kw=C$+h;^0LdJ@1WLKL-oPK0uvJCNSF5p7e9vNc_}5j|ImRZqh20x z45f!ywkh{-?TW$eQWvJc7?S6I>`!aXPaus4wM4jRzXP7exGeT+SQ8qqq`&4j*d%~= zX!+&jBm#qj&x1PqL%fLLho{^PHpd??5rn5FERk98J(Jls4-e-wf~x97q>=cVI!Y_i zPAuKxw5*W@O@Z6{7 z_2HFM9cCKaBzFHN5Blz2MqB)ke9b(|lrh{Hj0giu@e;mQD~!@u6q0}%ovWJ5wIOQEi# z09{E++x%1(E8Lx-(OphGqS-j-y2Y^JmW7m)hEz3-XTZmG@%xr}UJW^BOmTu6411*S zHwI;X^OO;hJjKXdMqA=zrj~Y{A<`vA8``Px-?Z~*74<6~-Mhr#{ohB;&wAkA-)64r zdaue;wNJMp0(Mt;B}ZDSAJdrSHfnjS51W6Bz98$xegeBN{kyKb`&Wj1SA&jTiH25< zj#Q3}T#9^Eadk__>s!7I5$ma=X#4PBcCnnV-9o<;_M|*V1CPY;E&9vHw?38WT&<_H z?FkeL8(w=bvAMq?iHx?s>4tEKnQ zWrWWC{Mz_Q65ZnAOm6G9fal-DWlvBaHGwXN$LwSJ`OA0K6_-p2KFPhW4=oIkbD&H6 zdr{mSQY2t%?VUdQ4Q&>GwO_?OgQi9b#$22th|)Q#hSU-pNRP$Iqn9HJUPLR0kN$)- z+8qERzJm81kNp0J1u{uDJyKPr1+YeX%~!u-#{ubuSA-}J)Z#tTFKIswj>?L0iGP*L zofV_Yi=_U9E52^&U{l?p`1%J3wmrzQ*S@i}a#&!wYt3IxEU(LbK50Z5W%RsygCJo= zQpp8GV$~u~K!@RtQ%qMsA{YtuiK=W9i^URWvKfkv_9r~M?MmS+l^+Q&W}fynks}d5 z@jFn5t9DBTtcgT2;oBWjff@0fv7v}(8Up%7B@=s-)|zckyF02C-4FCC-{x{$HOA4a zioN8MZ{z(^OuN)1J(aBJ!wWivXs%XlJk2J~jRR5|~ zO&)Q@GAzoySn+nJW-|D~mRlXj;U@NqBse&=f0BgCG!1kX;KL+mErlN=6JnRc+epTU zsd&76jw2!y7w1$o8@A z51a*rV|L=HY)2AaL4k?FrJ55u_k^IjXv*h5b2ZYQbw(Nn*0)_=!pb92+_+9-p@8LIe23l2|*@aF!?QapJ*T!|09r{NNU?j=EkD{5|` zFC zf)=SK7~3O%PYhFTVEwCox%A%#bm9IfsSo<@qFTug6X|uQdUF6y(#pCx@9@{gRm&@R z>jA!3ACgj!Ke5vnwimpDvIK3CU!Op5L^P6-GT~!s)NI(fS()-Q&IhR{@xfOF^s#*u zp8X7*veSJvhyVP$8OIc^zJUVyFXWdDvhEry zZI=R5WUxPnV&Gss^|7oQG(x%avVA*P@v+!!>v`VkClGBN`CiuI-onm8rwZ@QoFD%e zNM!g+z1kM%Y}4+iTIi$9M_!~F7A_5BNqk)c+Rw(^e^!gk@mf~$r}_(uenzH438`-x z;aH;u!^kq>Ah6s&c{6${LLu7bb>aW#raiCStQ>zG*Jr=p>3#-Br|JsM`R$s;WKbDr zBMCIbL;vE)UV$O_T5u&(S!`L7ztl$t_Rc;fG_1iui|9b0^1H zG6s=vPAjI-@f|=^_duGyI2o;wk+?P$9ZWFe+^E@`<8r9{Er|D61D}zdY)5wFn8mjS z*s#$y(NugnV8OsOEhUJeRf}2BaCb-dBP&We;=c?k8b`26wfyI^Osu{);C4@B87$6G zq9|$1ApfHEUmcGULTQq^4wD#B93W`D^nen9#NmgP0__)`fS$`iRe))^j@Edi1mvFk zcRbHI3AD`KFdK8Pgu^9+aFdK2Lfca3acN1@p9>oSs+lFR-i zgb*kcA}7GEZV}%JnsB-X^aJKaqhtKf>Q*188l~xAEysWI^)-ba-gMX!-T9TgxnN0r zeBf-2O|D>aF{he5FMLA{`pDha1ow1@3c*GLCfoOBD*&V*r)|%X6_>TJWP2iyO@dzO zuUN^6(vBn}LRF=X0dT3 zS2dSjV@YZLx*H^afo#>XtdaHPyPAVdDV;Z_~+!x+rE71>0IYJMcP4Hotda(gI&zr(Ci>j z_@@QY(BG1uYKRC5ljQ}%4HI{5cDEwCW_oNTs@WL{2{Sodpfu}TzeZT6s(C=^VBW0gGG#8j#tf3#DOd&I2DV;m=Y**g*erZ9h@}Z0jT- zus@cDIy)wo{=q@je&Z9j@5c`N<>x9T*u1pIQKAMl#fB~=A+mC5y{)L74)oJKvXW@U ztwv(YYS#oTy?^ABLj&YtAnz#!G4B)S_hLDjMO~?hj!HVRzJj!6V0gev0c&mYX)}*K z&FB1KOsTvFW};6bIw{1FMN6t`j` z=1XsCN}D+ASv|#qV7^SeJ>+#kw&f&9$Sy6?tUnEzH>>2$mJfw}BoCo^nj2`9a+qMN zW-{h)dqeOw(H&9(H@~URgD)}SN3^9BXTS!6s&Swwy2)VKzTcUn^t_d@l)%?{=8H`d z`|Dl0?;Zpkt?N2P0aeWpQp4tvi5E~b7X8O)R9YK_vn+hrvZ3oh+T;}9?fBoZG@bK3 zSnp2A!y2hJ0I470RwEo118ELy*qNYoN${?E^M!=U0Y0PKuItfvsH#&e9v_Fsxo(dV zXry>UUDF0^pM2ixSiylt$lolL(Znp^=Q*zB@V2_jOM5o9{b zIX39+{aXAVUy-L^+Te%wpF@k^53>wCRGkmTkADkS&x4uf{w858Tj^=tuRMpM$FvR) zOC{dj9jukSCe^8+EQU%?R2_DMex_B40QVn=EbuCWN00ThbG|eFbHZR~HoIeSQKy1? zL^8`j1sW-|QptE{xJ7q2xARX4ba{xO-%)GxY<(%u&l|yGoyCHExr7SYwyERH={#pv z<^SO%w0BGiT-7nyXNl@n`mNxA4H_;+~5@@TkuraABXS zAo}`BXX9&;QI_HP4Kvt_@MlE_$wFBoO2h9znoUdfs>Bjm(*Vm1B5|e<8@l|C@$=x! ze&$+w;s);KaZ8RTHVGkVFV2fI$o7m93-0<_J*-1@TIJ1c*BZi7grSEMKg*xR*WieD zDWg8|_#JVQ8-Doz>0lD5gqR2(+^@WDSntXTcCza2o)S;N#n4Fw`sglSi{Cs(VDFgh zq{gb^0us@%)a;x6NWp zWkLu*B4Mek9v>zHs24Fgd|Y;8r@P9sI_Kg&rft3c$sA#W{!indw&mexZWIPTvJE29 ziZ@y4Z-%n}^>*KlZB2MDL}D#3hr8E;DzDttIS`IAUN-2={ef~_H8Q;!P7#rbW7jDy zxlY0PUW+~`r&AAwz_`-HWW7*d=m{dHiAf-Mf7Jek$Rs47uQi@+S=Xz|p)iq4gr==}^Mw zTRjGA$^9=bE;i7^fYle>CU1{4iX?o*s2fHqqfl%O!!U6zh?D!ddnZ zH|&;sety`)d4K8J@_eL%F(Kdk6cV*8!_c{6zDv)j|Jf6MyK`8M6Q6 zewz?h_s45|S1jowzXmgCz<#vJ{Bh$!zU@@qwE7Xcl7dfryDz;Ads`GXKHvpL35l2= zwUfoMOcK2ZhtwhAy2Jr$M%vi84t;2x4%kHr(`r_&o7iILdL27q>g|?%zM`O0|7)W? zVR!sqvlnY8vpd743_;d-$158W;zJD+s(M=kW%YwMk2*G%@YuSNN~UkTTZ*YBifl7<5NhQ7-pzTe}se~b4vi^!Tp*$aMh=K+4o380{82pX7G>!&oO7sk!TXu|J^a z`zuCgvFCvCwSJj~hxdtp6UVNw;HWbZ325Bb6tS#cJ9z~t4Imn}4{u>3Jmd;-O5lZ& zt2<<@nH&V7VeEb)7&r2NqnZQ}YVZdSF z#E00*br+)@m48~!d?gC^xnC#U$EDt9`XG{M@W}Th?Bn)#4qJ5v3X0hdR?jZ~xNmj+ zG#yROoB!xwTne~&)ii9q6+jR1upKm<2)TMssO=s7MV}E7Gdv&7{9+Y>Zlf)wNzBdhoEg~GlrQjF-L|)WY>T^I zhI@W+?~n|uUL7UHn9!S%6(ZpO8cS?GXr3)BR0Bl!?X7fm+_fCO$~oIrBxhL% zq*2D|gr}j#>4a}R-*M_L^-n&@5nI7{{1CGGMnG(yidZ$zc|Fs2-{KEgpzPF<%_EI? zQCc~|5IoDBPL(}A`{ZTWzwT$qD(_#m9p?7+xuJY6uc;Y|{`k}$lkU^Y!10wQ%tZV^ zHz2p$o*Z(^p!{CLE$a@O07k-Rr_DjWnPbp&on^r~B<96|7jiuk8-`9JC6yS?rRc;I zQu_0Sj42jM<%0Lbzncn^ACUcb84*~7J{ zpHent#=3&@g?~kgh)wNU0b9C!`Q}?4_u5fRyu7@2CxE<&q_qbx5BM^XU1TLex|qsC zNm`>{Z_g!pgcZ8|Kr3S-oA%cn`BV*2T&W}9c1elyaEYp#jtI0ks!iY0w%Gme;Z!to`b zoVTfAw4D|iTOd(FDAZ_NAo4QQBD;Cuw1y2_LH9%9Oy^b5@Zj30L zFJ?^NV5(s>T>P#A6ML0mp3IG#;KKBL0n{@cvi>iC*=i5yy%trpMQ71c{w6lJ_0Cf@ zZR;=Qn|@LIZx7~<+;3|EiKvtuprY}!JbcQqVV&)afE}9NN;rYSQ3r|j?~vQ!o;n1K z$*@q8GLip6q>F{pfDMc88`$!>mDe2|)I?PLw-ZLPd*KfmT7o-xq7km>AJ?`>*Nc?N zkSB+rEE6yl#gy`(FcPGx!^bkyQZKQPqbQwx515^xAzl)e`kGhJ;N4HQqy)E1%!ehS zO-&;LB^Jn3w@zhbr^i>I_h)b27n=_@Ws}S@j}=Qd(wch4h1L!;x>J#E80i*}4nYA)MFx-& zkWMKn>8_z0hVJf~;e2_1|La`m#X09CzCmWsUVHC*-Ru68S3SVY{OIC^d=5?K!v}x; z`UQ}&u7KbhqXs)+JFWN1 zp0bdy-K->7R%;og;9g~wp|Ur0sX;3eiDb&?qb%HFmEw~>;FGQj3sMIX?Z0ysjmw;J z;xH793Nqd$%EP+XQB6?e5$WB2&DX0j*X*^_lY9fa^_d}55hgVL=~bJ98S0$+8)%!MPG2t2s&4yp6RIN`5+jvGEtR!{GM0%- zoEsG^uuksmPW+4f?-`z64r?q*8Lz+}5hFv&A4Z=H4Bu)>8ftwaF4AZsY*Ohm;g`he zaEPW6{7vaXSG>KqN!Q{G?E%65ea5C0bRb{K8^YgPT@9NQ(9!Z8Uj53GpKTe<_ldGs zG=~@hA#f6w|B@bZZZ7OCh*2$bgKuf7V!)uY+~xQ;Fkp*HWv&i#l_nO4Y?Dpbn$ zRT%>?sn7iY9O(&+7X5a$Qg)JR&E&X~wHv+27&ET!EVei4k;MFhIy_;fgwOqp-AZ4l z7JHeRk5{ao{G6_gvTTHDNZ?f&3Kui4Wwb0Uemrmc8oDuB4vWnM3-~8yllnlTyp^pt z0UyZ<0>j|lXAnUJHF_%rcl}8Iy4^?mY%s6<%Wdt2NeXSx@*d9e<}W?>t1-OC;!~6Q zU8#mmzWgp%K>7!dLFMd4HQKLdjAYl4iMJdLp<5h`e|orqG68wBvxhRvBds^)lehvm z7+|e3co|?zGCWacj>*1m#I#rUflgA{I1%UWtT6z*=BY3ZGpwl&l+uekni=2c?S|ZZ zgZ;jNq}j$eJur<#ex-9sgL%Eb^O1Srs$>}{?c~BJn8*~{R&d)~*a5Z~xfiC<@?eB| zKR89`%M{E@`OYT>`C$O8o6$!*tB>cVGtW+bU=UW0Uoi`r&%F;MIP5aj_X^CyxjWy{ zy4$`PTw}Rb2ikV+_v*)xUfBALx;}P+>uM|ixis|#;|-)Nkz&)0WCNWm z-LWYvB8iZmh?@Vyl;qXYfUKnZUfEr?^FkfT+u(|s?&)wvrU;6F7X=ExW4rUGWjKk$ zM@C|iCTb%rrTAWW`6%**HPfwbXq;&LERMskTY1m|R{0ata8`td6V}qIREPX9da{d% zhh-Ug%0l**9e(k%pUjVy5z3-n%UgBGZ6XCZ3hEL#K~G2zc64Qdc(CkL)2o|gZsfleDo?WGx%8D zY@*zg>9P=Vw^9NO;eNxD$DxW3g@~cfAv4f1?WuQC4bgHH#z7I{E)aK}Y8Wl-|IHKXqI(Kz%JQnJKFCCoVp zv$1YPP7AV^0faL5Wxm_=6_Ec_0}DP_6<36lk*ym$kHQ4&hEl;`NJ3^Phh9hR-BHk{ zp)9ERV#-)YM@K)HEnxOB7SH4X04}CA`sSSVDPvS-e<1YIwu55J%NwzT``Mj?xo3Z6 zIMbt;VLCVnC2}a`3hUG>=`EnVu4iOY5SB>@yClcGBWqY8&-gs1gxwYC$efb@sFZ>Wh=yU}V# z%JRpiJEIJDW}gLtz-Zu2%bmpUYNzjjR>z}0`2S!UC+z3q;SS6Hn|&D#suz>_Pq~3{ zW@zm7qkt7({@~Am{CQt?V2<(9D>fMiY&TA|GIMzpf(-G5kb{|wChrP(%?!s%*dKR_ z9;7f9UD%(aRoG+)0CO-?4`__bQe5VR1CAf$2&WNlj2hEo)NA3=S^i1NIe z$qi?;XzZ5i4K?e}-A|HK7;Hwowl6a%j^mg6#*9mBdsMWdx!oz{zFqrwPihhkCQbMtpE0_V1qgD_!Pudod_5`9ix)YL)39&A6dH4N9hkT}Z+!tMCuH_PhF4H1K$%{)k7+bG75S7Q|u3_l;IaX3baqpM0Nwg=Ek*zbOAV0JZ-8BpG=<7B>JamO zp#%l2YK=R!IX)@UiVU5^_|a>i5yC|7D7i`>m&;`yocxO_#rU&=z;Q2~fAsS5a(N3f z?fD>JSF`QDpF}ZT)^z8!cy6Pdw{C*6fcY#!%A-P?W~u7U`m$pGMPdq^FW9`pQ2-HY z7kXInSvjL9Y15%d>xMH_=!%Zu=`kWTiXn(R&-Z*(2@K2p3QvwBhzEg(ofaD|E~Zsf zjQMh*0PtyZ(S>a%q7My2Hi~uI>h1SmZ$Z`5=ZURcqt%iK=OM7_tqiH>x-K#ycZ~X> z@3#%Wc47?3qX%ztL0mQeKcL1bUIQi~156n^e3x68t&HKTgE%}cLV-*8!1Jj(D<=wo zEYZP`bR*I+ClFC*5s>9=r;DHMpX8oX_bT>)g_4~47UT23`C>B6)r8t}Jg|ktH+YBqHYF$nI>n$A=t1;DYPBh=dFP*~6X}`i#0%XP6OvZ#wEHL6F?$fl!A8(x&u@+xra<$SAG4UJc*{? z*a{mM_DQmq6t(2X z^CrJD{Zb>$^>CRHV2T{wpk1ELx8o`RFl|a9&ZLP28hU1l?4xmZI=}ZhF#Yk1K6hy4 zBX)Vonk-+ol&5az2tcwzt366zv9fQ8r7XqFPb#te81G$gQfnRKXWuZ`I`dL~HYJW~ z8oND3clWPbGwtEfRjT3c&_A_Qn+CTyHRxvJhhDxW=BY_b5MpyE{Rg+PkmFPdAf`QR z3b*(H`47Gw&G)Q@)h8uKqK=f8M|aX0L!i+2eWLu+fCg=6a~T^!8Y8fg+PJ(V3KnDL!fE% z9O7349 zV;i1u56F|7Z$YPmaoF8JA^>v~D5$><*opjPN_*~h-G%{yznLVtogQ?nJ&bC;EBU(QO(P5c1}A&2w$a zcj)t2&&WF2(Nx`eBL>h`brWXW_vBFXF>kAovLu-RS#lj3g>ItBeTz~e*(APr6JfO;Ill_GwtEmlrUFT#sO3|IygkU>B z%GZJY>^2Gt7cX-KDBphu*XI1fOyx*SrJbj2B0*VLIFJdnIM_GKV&G^qdzH(*^XVIG z-F-7w<#9{PDH2?)-KcYKfY{%=I{~-#;8xLqKp<;m@^d<~h#tS6)8D4bRNNdlAM<`N zho^|t_~_$#$ADk`<6~9!JplK5R##P1`d{OmNzlX?*wRuwO?f4IUa}vnkSA$slG#fo z_4n$}gy(^a<9O((&wLn87rjTo^HMMd{e#SZZiscmfo=^mKKsqFS-h!-3J=zPoCB$KbYEP? z>IVbjkP&`T=5!1<3StOZ(a37Qis6>-AKhHa3kw?0fi%Cs_kwg}2Fh_rtfxtiUsW%* zjYsR-fgegLzXRi-eu1$UFy;~#N^yG>=i>hQ)C6}Hj(RROF&MAJK!>@?3;P_x`p5%u zN}Q)HF>}rxmumnM{wKfdP=tL^K1t&SiBe7a{A-mmJCUA1XhlG*jS;RZJ<*4O8r^ znvv1igIEA3pxd`ZDCF%J{`O?qqun!%fK)g=X`WKJI74jsk{6BJ4G2859v_xrjwP0g zoDF>}>7X|7t_yQG>@N-y#Cs$Ev&YvV?G1cOxQHP`IIh7Twp@AkNbIon7Rxq+T`(qVz*EiVz-9r}U?1upd3UCNp zZ^QMj)mQSvf@0nCsP|v8Q+%7@a>TkspTl5NoyuTcu=Aic5Lsq|Y72f+Y|Y?D5j99! zuQ0ANGpU~vr0og)8KTAx%R46=YON3XLGq=`y>L>+kZsK;k%x0k=3R?Z%2+A-?N)kw zLQ;>odl~IwM4*Q~g^7fKb?j?VG_veBT5cwxuD&cOvl2T{{1N50fC*qANhsNaMr3=R z{tF>S^&K-ne0hE>z(Q1x^~yfrW2wNMtkIG!ijg`c@%EAq;_KcJpt~c-nMKBRrMqQb zZfl6X^tU1Im-;;FFP9}!Ms4^I)m`Q?*CD!fMv;L2Fd`7 zl!s>n6t+;~NacC!H*NkIq^GNUE&|HVMo)cW^N!wdW=%-@$#bC^;iqh>gC*9e(C5*z zVM01$f{d8j*yPNlBBokBax+*)IK5i;6fweKI10hvS}hHf6ZPfLY|nU#Yuo}|?S3q@ zAN0~~EKB;`rf(j;lrL&|iM1Eu!X+Tucur|$2zpW;I`Gzk2gN&l4ik+ib~yJvpl;CW zVTm#We(hhq8)p0ou0!U4U#KJVvi4fH z12USCJ$opOsL5g0c&QdN%h<0>C#Pip`!b^6;JZT+!NzKv`69s-MelX=uzbgB1$@?{! z;oamrbHjYXgj~-p-Fn5!(@!t0JI=FAvg-t5Hjyd|TpOSc(Tu{hSJA!eqrf=GJrw`$ zw}*3)#dPfFTTa$EfuB%&=EHWE19%q>G7mvczu6v%&Kf|8dVC-}33_U%m z7?Rq7+WgiBI?TeUv<+8U&>uyJz_w)TfYA^499K9*}Mz{-1s4b?{%e+bV`^Be! z#p|ukSW=!q2(5#Wt99ODry$I|w9a@sJ3wI)dipt6QfmBUA2>Nsw*BMnWx9TW&r_VmL`sjV_UTs!uudD_d#uXFbkM017f(4 z@pf$Nztg&D*>e2JvyeX>WCIA{LnaZ=bM42cF+N!KYw85(Db3U8`wAkDqIe5XuFead z14_c zlET#^oyI+1@hL59xsmet9diz2p5Py0$%+f@`%0ijBJSuakO9;CvAVDCxD9h-k!pn) za4B7w1tW3`9Y_6<~sP($Az6Fb%Zp0y=%1Mf5zsbPb6$J-%G%JHbYW znYCK(z4guK|K_^SBRzl}r2Le@_L2)|Zr2){2D_W!Z# zDhF7aQY^+6OOD!WtQgdm8;=*oR*R`k@|n}uQ1QLUwk;}&Xl*+c!(bS#XLC(dOAa{i zAuHi)O8Ba@|3BQ+1*lt2!AdwVh=eP%46X8cAApNP{T3!0D~%(*;3V_(hL+J)oYsIw!7PNfcd zZ2Dsks$+@v$KkuTKH!CtTz;bf}I}3C%}y4tU!k?0)Y;L5Q4Ft$pzwkUbNK5Y4b71FOyI5ejLz zqtKbWhyR&Qi1>)gRn+F%;sY<>Xjs&Dbw5*y1MJJ9DVjIxXYN)1$%jM(A)**%atkHq z@`qo@vNEf-FttHpWQ255BJK9tfYDJzY`^rhf3=U3CIsLWWG!%Z&`DBqxry#*A0*4@=SuY;k85k5P9e^+7pNAf=QazgEKK zmX`@>o(eh1T#ulV$hCZ>MZ3Ww6+BNbc>X3bX;FDBD28yQ11o}n=IQP$V%F7j6hZ?5 zcLuQ;+sy^U-sQeHWx%~xcCgoLUtK@drJCw?tUwPt|3|m=8QpOcins5Frq8WUON--A zTfZD5XoxaW|IUCHo@R|&$QcM2JTPHZQoV*NH1b-ZXtu@Re0atMyghS%Y7S=v`c6(s zMg{zoT$Ko))M20ewKEpuQsk{+c)UZm2&jwZ8mFN2B;}yZn~-f;pWw;oe+vS|wd%Oc zHU#ahLarOn=mrVbs@`=626mQn^)GOOt`9>_qy%mIM{^yo4$e9QwJ>e36*kwthmx|E zm;CPfZspl=hnj9G5@p!|k{iNB+{edFFN=1K)b;;iiT0GZ19js^>onR_t zfjzOrey~$a){kFlYc~dh2XD@a!r^z5J<|RstVEJt(~CYZhBoJGW&wkM|F zij2MoOPbEJaiG~Keym3LeFCYajpIQ5xW#HH1tAh>z(f@=vihuYEdgGOSepOK0RQ{( zZUu8-is)&@3PZ4{*m6sdqJd7c1y-BmD#jIcRUEg$N4KucMdP?-(<71Xzw@Hozl>jY zqbQ<<*^vqGh8#|XQvz*w?DHsF##V1!!;3a|F-A$5b4jIeANospEgXDD4@8Jw(wbFX zUN3y2*UR^dXIPW=cpqgB0g@c@d*a1Koo`R-V~Ds-Tl^8t2mX>L%!_`#!B91}!##>L zil@b0oNB|)L}0P#$M62#CP;j_Jx$7mJtf%#oXeNGap-z9)&f3;gnc2hmI4OP$Np=2 zU3K0M5-ty#lrifL*L=I^->BmadgonBV=oQF>RiEBMe*R1s-iG70 z^gKI5ga{{myLUYP-p0)2K`{!v9HSKYJG-=P4_OHO#b-~&HeKXo)NG{ z-tr;^mkB$xTFisJdL`$E-fhr8@CXUw1Y}qW-HqnP-Y$HWoP|(_He3U-E&+Xh}%fZ>N3-QI$h$e0$p8Hd2)? zwb(4SQmOuOB*KG&( z$nu+`531xjZTo=F=wYSvrhlEKDxHVQoCkzl%_qEeQZX7qo{PDeCJz+IQD2L=B$@kcyq}zz!XSq^q)( z*1^wufR}W@xd~$NBM!9h?sgxYSz}dQ?T@O@7jnwoaiBMGq$#DBDQz=L+tTHEnQ}?O zmH(Ovct`8uxr9u{s+3}O%iI@f$rD~fl0)rT1CmTNS!qS7@o2-2C0Jyn!NS=J}%ecy&R)jQSKZa>M@b;!sHUC zP12+c9SPj`WGBTqud!boY^H}n(0=(aY9aUJ@?7ISp)8?)D1+}b+YBKUI6r))l?J*R zM^yh7h#rn-%U*FT?(=)VRbj*UB7en+vx5u=mSbK8|qFHj!SV!o3O?CkhQx)PiOd z0d9#b36Zj=hn23rJO(w%sP>)KmzrVY`>Wqky0M0Nw4XGYovo^GBsi{zu93*RPX7Qk z`m=Ie@a0SadCpmmVB*VT6RP16-BNWx60~wF_VzyJE19!JNK6RkCr)aVG25Y?L2d zuJmnL%9>sNBS+5sySt=_QZz`0%l^|>8EwNZ?vbJ&Gl0c#mV2UlmupjrPebmPoi1j3 zCVF;=&{uYJuTD{mvQ1jFF$sL zM{0=qGMyGgp@nhQedKf(q_?zks_;X7`FV99hB_Lb`>Q99%MQVR3ZOk=&?*fI;TVMN zJ8;>kElj3gfJHnT7uVF6YS51PAkf+W6yK{8i2llnSa?r~w+j?WH`SIGV5g7ksic0* zZ0HnjL!C(Y-}C-_N9Dj&i4B+rJ5MAHm106b8u3jM71c&x+L~+j0u(+GhGc!J@494( zCKQ=(s8h7*NzI?&#_{4tPhxgcl6)D(>U_U)DvJ_zJtf^h9BdU>p^+^UUbS%l1|?&j z=sNItC7&ty<7W2P6GqUP>u=O~gR;l_@Nsji_wK6QEmIlmmf2Sg>8N)VE{*BERsgFK zexEB|yD$0g!8fAVHCo9cQp+ID)rNjwE20^v8g;=q1fy#wjH2tDIRyn8!TL^-qd5DG zpzpww3%@4!CLi0Snu)MTFRoafu_X474NI26Y8SBeW~o$06x`T)Qn&OI2-WHsOff$* z)X#FOLjJHZhppQf=NB9Ydc2_5wwsfAf^C{pyMKkX;wtgoZOhcBbir!^+T3HIfLj*s zYUQQMy~(&bNfqLR%B!--)g*AB(Qk7p_86j$j(Rx&p2KS{+MIJ}D{CVC5qSF-_y^|& zC!NH&O)v{UIR4+oLm0aVKz8_h`IY^my&uP=FV}kDnS?(DSGtB)$fjSo<@I7Go%+{J z4Z05mvIlbRzS!x&Jz`!t({YlMQNY3$d>u<1r09to$0Y&W;HwxNvftKNXfl$hwhlk3 zYf2cot{jOF*pxeKCq(;EDHZZ+=9B9IUCP1g_lUZ-jXffC7F@&*(5}NRgCs>63>nis zPUMY&4`~Tlw-#)Ho(EtD>5h)7QP`Vtp=vrCV#F?6R30) zk8wlUC{hJlX)xTF4VX)Lco4Kpyu@}^l$HGDsIz&Spw6+*JHSJOOxe|fQ@PoJB3+DL zzrZi(Z$a%OOzs9x_Sj|s*SZ9D8sbdb=h3?Ao!1JK8Tjhbhd-OH-fvF@F74IU|2sQ8 zlIOiW>4Bmpf4Nd#N-Ijycw(W8We!_+a!VCH{-nX#^o5r4Rmamt)0h7Y9_mWeA}{P>?1yj>tC05xP}r+@U|pm> zrXcOy@>-6L&jRRaKf{t`=@)QIqr+GY1?iE6LB}c4NT zdcvYwt{*i`Q+SzIXjJHbjJ-B`P?sHkC8x~CZ315HB1PTQp;$nfoYB%_iBs=;$GF;4p5wiI8BoFE_H_A`J4M6aC|Bt*;Z8JZ4!mXGwSVwibrkq z_OG^j7$lSd<>CRkVUbw6ba-X_4bnEpn)%!?Brz2y#Lw@J70(>okLq1p42LIf>`PZ- zna|BiAxv00$0_3~riSe~yWDX5N8_z{o^<*P#&L{SsX||e=ZKG0^6;OfZp==j%1Nro zWn*O%=VcnK&?Igh{)i4UNdMp`ad!`sFTdMzeZU$7`GwaaC#;yB zg=5`bz06?8_EA}35`C(^eEO`Rovzbku830E(iZglS2lb8@8{}?EwEYFgOnEl&lf5h ze6i3)lTtq42(WQ{puRxlLm*>`wC4T?MCtorAo&{xd+FM#PlmlOLM^dy%}*)Rp#FYR z_~f}<@Ly$}f3u`UuKED+Jyu; zB9&d31^w&vS0Q>s8Frc|$~efE#OEw8q!VXU7T#~FjO!CwjARpGs?DIhgM16bJPAVO zSHL-yXEt_Al?C4(`X4GxQJc7Z7$Yov_K2fp=tU*jR0ba&h%Gbl!X1`R&qrVOlNn9sv=RT|zAMsbo;Xb58&R0!} zTdbi#F80G~@g4};J6$#5YCxjUA!+E6^_gAWbL05py|#=hlVjwM+3L|bL){XvHGQ?< zc|>O?ttD}@CjGDh2O4LW@x6nbE#HY|aOGc@+o>UPE#$!afau+QVdu{`T)o?gEzKEf znQR)+0b=x9)ZcPX-g?-R4!?WYFmJiL9yf1!IAK5@oju~L(3@!_!K1hTlmc?TGv93d zeS%RjZbya<#C*s_fDZUvK1`RHw$CWA1FDPsC+eo>UsSj@nz{CQvGKf%w4lhT5um@G zr!KA7|09Vs+);I%q_5!zIgv#cgI$62(dd&;{Lqgi*HaG9x_av96i>yYJPrsBB|FIg?c4Aw-BEYM&(PA}RLmkDeyRd%?QE~EsJnup$ zl3?@9XgVI_gyYlK+*SB|P(h~&s)dMb?~pM_48HaN$U8?c*dVnMiUew6C0r^WGqT$%&E^+19>Z z2cU*jdJ_gofg%0W-Ent1r*i3m2l)j?QnbVs++9+8`|^J`wcGI4;SN+TfsrOvLZ{&F zmmO$09}XU-A3pD&HNqdLMKCUcw5PFCFlg+?^y7qs08>3BFUlPCP0HtBtYSYgE}Dzb z7M2Y#fs_wCkjOzQi#3L8yXvuT_SL`9~#+9 z8%_JQ%RqVy-lK|cZ1U~pthE=bc&H}N@fEy2Ayel1b?-P|{V@Dz*bT5^I;yA$QT$sW z?Z9&}!yf-aT=b;P;<*_o<7$fJA;ZH-6$4^hiH@ElUJ|*!Y`GS0?cp)tQklghC_Ebe z7|QmL*$(b+hv>)lt)sk@*Bk3pYq~8d)*4&Nd7bV9HL2VFO*Zb2mS|f6frNaF8z8H#-SL@Olw%Ba&-^19gpZKf?PIwKsA+=14*-MHk~?*yrczI%H^;DDsO%xW3hhZ$HUK-8w_$*H$S zdomXquFm<(J44<(znAq3*i=|I%cBy!h&)*twEwf<&R}r~%*TNuZxfJwcWFpM*qgmY2 zw-tukL_HpJv9cCzt^-#$p8VRH^R4>+nTfQEd947+_vd@A&D=%w9QPLez+Yj0vmf)o zW76Eg%hdRFx4H;fy8#*kM(tXnEU~Pm6;sTaSEBkF2z?xZK^Of@^U1DTy@FYk0%}t8 z?)x3-fLr0w&yW@;)+hFr2HO7PEoukln~d1%>oke+@E3`;z-yU+^xEsaP`hnnii)l! zo@-iwq7t2ST#>X(1Zw>1p8^!x!jenEfi|weVL4PF0IRYNW1?J2xwk3RXhgY6#`t#M z)f#bt1Ez$uLQdNc&beDgy)A>7d{X}C%HS3HUBL05%~DGhEyA>*K`zr-L>vCO@SQ0Z~iNs4NeenV{O{OWu>hk&~_sRs{=wTc(;hc}115 z9ZofOxm;|G1+Rbxtv=Pr0cIq6FE?%&bF*kRCd1it;lJkayBgd*z+o$@f3s_QfBVZv=!M=3pz z<^|nEPz#UX58iNwiixXT2S5i@AO1%P(G6D+F$sKYF#P(J8tNDeS3N0?s)hVd6HScS zW+;9dD+>=^0g^DaWnn5X?8Dn~+D#{PgHI#)w)9_`Rs3|N4!8vW$*(s zUD#Ezx;6tY+~fiVZvlq8$6z=7vxft7bjk2B%eTr~7h7(2XkCzou`z+N-O3AhyA;vj zt}&4^&1(mbL&s97sWI~-Eg){nW-9#K*^?ox3BadEvMQ(U%GGe(i37P_gjSN#;1h!a~s(##{*& z-~&+JdRry@A1ds4msDq*+>qJ)Bj5*~sRtREKR;IoV`Jtxt~dS~a+2;=o|#$yFT(3n z7+Fmi!J6ldDQH@FmbE2}@!x)uVu`91%Y7lI6e0-G2oOPu(_dkihu@~}ys-T{JR@DZ z)idNFQgvX7**K<1`84ZSC3E?{zP#+H_)(BWdRCbE+=(6)(*s#Y&Dm}8@LLwMY9KSF z8@~VlNh17S|506q89=;_Z) zkCko6YqaCD0`y)qY*@@1fo_-XOklfET?F7-EWDdKP#*=Sw7 zYB>>&6Uf+6F^X?~EtVG&_h;#69r{~(xJxVbAq;F0KfsvHZVldP}G#q z*$us&hqznbNVzR|s=5xFSKcU{dqz1+OZ#tp&P_K0=X%4pm=C)23AtCF zsqbi|!69>fa)d|S#5QBSHhQ|%5(Lq4!}R%)6Kv);-klR|@AxmT>|~z_@@MJJ*Fy73 zGe87?Sc2XxQLlTq&4V}$F36ddv{|y6S2r;ijM(zu{Qi|zloSZ3j}OmJl?oJ-ONZXy zFuYo>!euJV-?&Yu4Bx9>uclc3&qYdoyT(7+kbV~z*>qkteR2(xez?z+^yiBxv6`u< zsEC(FrgC9lk91T~I`xhSkrsb=^Kb|Rf^4Yuk%%_Ox;$Kkq{Gz% zJHP+v1rZ~rt{w)0Nd?~laB8b&mIUh=>6|*u5fZ_Kpo^s_CcXO&1~`B%zg9?0eLqPp z^Pe;(@`c;Y@>U=W8C(ELjQC<#oBRd z$O6jQDBQ)}?oGD>q)sQgzv9^kfqlSqF@usI>fa`|30)kegXTm1=m!mgLtA5tHE9h_@72zBW`nuU=J6NtZv}Y^==iM?(f$jN(BT*7U(t1JoT4iA% zf`Xel8lX6Rpu<(ozIda;k@j=PRpAcZNZZRDk4QY|Rv?iiGY|>(H*MF%DkR~j`EK;% zS3Q0+XZX8cf8!W>rl5i1tqxc{KbYvKC?cH0u z;?}{~pVPSuoOV!-ySY#>(7R{3X^SToMm4HPubZxrJwV`wr%FR!P<84=Fce(m>}L17 zsFsFRinfGM`vY~%jMMzGmB@VDxUc5ANrIL1a@!`dV&ky!2#ZqI8(MZfv?U!yNZ0Uy zygQM9(K9qOB4k6EtZYgtH=s1uKMgJZJaF_(nQ%JVEoa4B{c6j{c9(I&TrZkKb%m_FfV%!RBAJr!Y8G7w^YRG+E1MmYuR$My5OTPF*UVj{-$#+jYx3L~KpfsNcCi zIc1^JSjj|AXY5RAF^c*cRt|qKF!lY;5>wmjf2q0d{GlC57!Km5dz1RTHAIxI!5wS{{p-QDy2 zXfac{=_2!jLQen}OQ zR@qkTJNdi5&3JUX_)8k~ox+e?qG7=DPQ`+?nF<5qgg2Yv)^FfLHc=g2QcQv4ABIQ$ zX6E&Gc!9Qgz)!mrnrxi>wftmBw8`fu8>8&tB+v5VCoa0;=a(mIGq*4FC2|U=waGwu z?mJt7FYpq|0@iJ4DQkg2!7ZNIIQgMhVsztR0RICITCmDPB?PCpI$53 zeC(JDjGNCy-4wpBCrGjSVLJEpyUFRgo^*YOixhC3wU2fksm$iae9NtS`}eE~OUPb) z+UU{;BLjCn8vBdAwE*Wv&MuLy0EPWq9FN-M(J-;UYEoh{QiV%l`s((zxNlH)G9{Ac;%d=<6_x;p!p8ne7~VMvfU`t{ zk@ryyN#q$1U6|3q@Z3`t;Iy-}V)T3ULf z%LncWyF_35RXA%TAaL=QefUlKsph`Ydu?X>8v_3~q?y%anAaTaTSjmCsax2%mD9gA zYJQm(x#l02K4L`MWCRFIt@DIu$2p6)yvo2<@-l%DOk0sdrJ;}H}Wkm zJ{Wn?9dErC|1P#<;0>v2phlt}NfkaSf5Vy-JNBYgYz$~TO=dJTMEOO`0-$<0Ui`hP zGn^F}LDSTA(^!fTjLIpZc9IE28b4e73M7?niSGl66Q7}ZzY`!q=dG&}#cF;j# z>x_TZmoIp5U>A#S&z!EucbKCL)!`cZ}U_^#N#zhm!GQ8k4hFHklcmGJTWH9aIJ zBF!oCySRX^jZT%B&O4mN%VxECUB6^=d zpP^O{2#e(`6lmE5xdl)gz<)0|ZaZPwT(&=co0Gq!hNY+4ytcI)>=Bo)S+=3K`|!87M}Hlq#?ffsqf#bq%++4>Gm(95?6W-lP zT-~dw7<$)N-e`6M^ zp~VoP=2pbGUaGB|<>`hL)+~ygtXKr(k@NPvU`vuLa5ia1GM3%%7MEv=dgj%LOI=f) zwPD-nKvh5~p@x&h(KWfI2=0}q8e483SQMw#8McY6^dX+P@N#LUmFTn0nh)cwuU?VQ z*2YWS)|tzfHQ;Dh8O9-tZ?6Y4*-$hYw`Jz!^D_yeIZ=JEJ1;tuv0m$6E%~&7E4qc= zWPaT-0Z`~}WH2NX_cqr({!;-GJcuubEnsaZ+*R&$3d-@zm}QM)eW_D*Ou1>;YKP6P zdZHkcy;+Nngh^c`y>D$gJB-5lrwGm%6IO1}+@}O9+L>?_*fbFz(bal57h7Bj!8KmC zdwvRhQlEmuquXbi45#qQ3Hv-tX%TrHyUUs>|GkA__W1;@s7r0$wxWxn-fIN_6O2xy z?@{W-fNpHqhvbJ4hJl#Bc2HAZvZCi@$PY4gKHICBm!mJ=KAW7D1Jz82Mx6DhE8Nb< z!tk{U1)o{S+6Pyt5bW2Q?l?9mp1Wlfe)|qpA+f__zt@}CKK$VUM-WNetS9>xOdZ}W z;~|0L?M1O$yr+dC$qV>kr{9y^sc(M-r8hPmR5y)PPoOEw#!uoC3B2@7C$164?7rrY z;4H_#K_(_9zq(=YAYK(cBK>5{k+hD<_WQwCjcpM9^!%ev@AbhGpB&@}Hm6Nq!dTjm z_V1GxPQ;Sp9;;~XV#qCXW;^o@eY~avluNKu$dhAjbE0RQAD4uKmX1OktBcJ!+P~zy z0TYYjiaa@d7n}$Hd>zlhhClijXkIp%yp7tP)oU}1(&{dq@jVa4rY8lT1%;={5ZrsI zLlKR#!T#~--&+-QeHG&x*}vwfU!nG51PyRCNIBeRi0@y7V|@KAPuJ=u6%+N!w|gqA zpKw*qaTd-6HJp9nW+#Q3aDeXmjAtGnCx6Q9ZaTQRY91AsV0>M}Qh9`va8OM$7`Cj`#d6KHc(}U7Qgo=j7zpYIF%-a*K7fAy;{~|4f>=E3g z1k(-}jda`AjCo!;n0Cf8eJ}quw~j7Yg0uGbR&=hj;~t>t#wTUL z>$vqMOo}P@=k86!$D}E6qy2Jmv+eF+JUzAgl`ZJUkXHdlx0a%zCnQz` zwId*9+bDIGhnvNo;Qx!KuZ(Jg>zc(~N^vM&+)9Ic3$(aPaVNODL$Tt;O0Yn2m*5n4 zFYZ#@-67=WdDpk@|L`N6oPGA5nLUH_cAiM!vnfV(C$7sL`A6i>m!{~QL+bR~F;4uB zjdke%6o1vYM*BFraY}vi;6{ZHOF}yoAhku=Lo-o7_yv6> z*P#XsfL>OO86_~Ssj@(MZ4NA)1`6$r>+Lfxm7rILwzT_yPIMi=vW{1NzS@M<+Q|Mj zz*MTZiedYlw%0@KC)0V=>Af%DLY{l;SV-C+7z~}M|MmeQ#6b{#YxG9!ux1K^*E*Pk zqlw-}^tCWYY1SazKG;oORbrbqD<~Mbunl z1o6kU^(Weyhrj9ztMYeV@3NMtFCs2^cz))!KUG(wtvm+12oZk|!L7>KR8$V4931yt z=!^V*2}1K$3V$O~)q!-ZpjY0D>FU;=|U6BKVODvjNFR?i2 z^j~32;i98ysZGZpx}LL#8%s|kdF1^5{W5V#R0j}zcH?DfXEKKIdtM6 z#-s{cAcaq6-CVO^D;xePU*=na5xs;>jv61DS9rAJEv4uCpW$MOA@i&Gr}zlLkxCQo zm;Dmx(U6n%=f6t~-Q(P19~$4Y72?q*`>n(4&&utf7fOAQ|K4um1J=gS=U+4ODd2#6 zflku@4WQ|HU8^Wh_e*en>I&yXfeWP4f3f&y+tNtGO*PZ=x1$J(mI8YlEN6{UA5w78 z<*vxdD*!kZaTCfOHeF$T_5sWq3u#D$n)?P)(UJ(PaKZoi5O;x4^Q-fZ6=A?g@Z{^0 zmhD;p$#iCm1&FZywQ&Hn#(VkC+67W{L1sd{`KD~v|7t&IEA$&e{#=fDR1WI(-L-MY zuOz>49;bnCQKVl+2wIbBQkqg+P@}g4)hHVyC$dQemsy?r?p@zozoggnEBtOeRk(=O zES&$)nEbt&6E1%pa--N*$x798UK5~4Z@5&`#O&>@!V$5ZW%+X^!|wH+-3p&29PB_L zU^j7IyI7s{@A%uOMDli%N91dlVX!*uqX6-df?64*xc5pKtz^ML!YAP$1+&EKFd*3I zfXOr0F9o{Pi)7)r=p%pQbdncIW7v63WE+iS6Un3<`X?!*{CPmdX^N(o`hbV$nPSf@ zl*sbzmHew%a^u;(-vB%}vwHoRMLB?ABjv{P_5{)yzP!u_wu$ICQ1q|h@GVH)8e3`E z9mn`0c0#gF9@zm)JjBM4V877)5FiT+oR`7t^gVB~@%bg$wO1!Q6#vhe^)L)d^&9W6 zgHV^ghAZJ^i3YH`28M_n&(2m#gd&I(-~+wSk%tI~L)hU@0Nl{QwO554YVwVQgoG`1 zL<6yw*Rk<4&M*s46u*=gdwb1gvjAQ5>qVqQ{qhD74v_tISP7&#z?@~C25*?8>l&^W z$XCWM`Eht(NmC0)#8Od<09##e|Ic%tOVqRZps&{Ko{*8+QMhS)dHd^i!n@g%^`9L! zhSOL5Y5*qmIEoQ4c5In5uIy+V(uq8Td}Yg=&(c7jX>k{&45>Xx%$hgF%gqfUP;Q2? z2De?}Zh!Iptlgf0hHn?^EJBzwsi>e}E|?sEEsZ>BD+tIoMI(f7F%W%n35>0g$jWio z5Ko-t3x_@$LKdbr^l}wf`o}lQ+isW-%A<0t!z$RjFLyhB>H#PFf`U%pKjqN~`|>|^ zg(&B{6o^f{s%Cy@dK7?~%jb9>75^@#1XvE6;)m;4J4)KSC_ef0$rUBa5M(YT-^GEA zsw5&p%4#sL@p5_dXC}Va-{1@joe*^TZ%;kbc&i%FUS<;PWPTo00YfWGf&{h{;gIQi zB|6@c5B;?JY6VeaEpLkQ-KD5H-%Sx)@$1U@fcrq+MzKxeSV?JhysM$9%*`fg&JPpW z5<`<#qUbKmU&$0e#^+;~B8B(8An%2X&2Z#gxb5+@uNv8hSU7Sz-sIO83~IL)8=Vfj zimf@FXx61v_P@apPyVSO+K+-;%Q*o0M@&&whRcmpP)N*=fCq)6SDUuo-QAv1fJ&(T`UQ`g-YVJVz1{p8WP(itDsf`C&UnL>lEzgMhoH=>HUbk1&gLIX<{w&{)7e)VqpU0d+we z!&7Eo`#|3<0H+!p*`O%zWO+!DR5#IawXC0gxz|*@paI}DY*EMW-k1NLx3a9-7!c)= z*c)o8KZ8{z>2QG0W6r+LVP$W}`8-&H`=BV&2l|#KS%N?NeLFQZ6_wfkF@%i8X+1sc ziAxGKmx*jcgf~B#d&&wKZ7x{djk~mV0J9WLn%X%5*M*6Sx)L6wJciF^7jUVfn&q?Q zE=Z3?-aQeDwiNjfObxfWT#_&(9D#)~rGzvD?q_@Ho!(yGcfzRf{iY>28P6#=*Jn6l zQ^)BKf3b`^16)^(OnTtK5c2J!Tx`i)hjf-77vy7->{3GwTV3cr{S11LT_bf%1~N%b7M0R?}&w`D68Td_~dY+wX~EIo6K z%(x7CynbGRIqrLx{aKOmq(j+jRG?~|(x45p@{Xw13?ow?u60%hX0EsUvM;B{4;7=d zp0b6>IaWa>ajJ{`<##dEN410C;De6vl5>*BXuJ^ z#Si)WdyW<~W5V|U8t1sjdstR%{-4CIA4^zRAOGV9iJ|w8q$`|YR!_!ovrTO*gSqIH z3gWh6b;t6)TbqA>NoqoKi(h)`Dv&9w3hqhYoh$?md8)`zopCN!W;yc}D zOKkQ5920JTk~LCPmS%tJ`G&L8bak*`Rwf)hj+&&-gpNgnOeq1txMhWTz-g~I*7`7F zC-K#uZT**4EIh(Mx0SwcZVnV^U?>LtOf=$ec$=!c_y~8Z=(Z3`lI^xBzg6otd$G@% zJc(}B9X0C}2B32fG7}mN(o(8M;N9g*j2cS+VWCHsB4i`@ z(M<8M`tB{n>lq?xEC@9+ZNZngNn|0-M z)IVFasIqt2*uF*fvDtk>o)5vrJuG4=ss3GdJ0Wd28$Gc-wr}JuBy?<`_qv-k z%@mC8R`qB1om1qg(>s9}fpvyQMVrk^)(()aI|C2o?3(*^<~!P04#O*aEmgNaj_gC) z))-$gc1o+lLd#J1aiJczb5Y%YkgzWncqwqz{+o<8ra5fcS`wjg_Jz{r5cmvz%7?T2 zkjBPFbG<^Rb0?>;MoA>bThn?Xk=hRnis=!2b3X z9_|}4sQ*whbweNV#WAy`V~4LLtrLTWV}a!3pz*xXt;!4%u<`vat->{1GsSrnk~1G9 zXpxITP9o@8;1Yv<+{Wbol;EzIAYnWT>vfail6wkv3tRYfFG%W?YV7V}t*+!hKUa{2 zrez9%S;Hqjm7u&=GFgnSfd`ep262j|y){DQ={`e-wH`1S#EZ(r#Xv_V{n(HR)k?hip!Qbt@W9b$!_S5#*jlq{GPGfDXu}RXLM%xakrZ@C)WZ|Q-j^%T z=7qkGh5@dvB9@l2Ek$BPuh-54AP}eZRi9_8yBx2xcJ9Z2U%^CH!jIc4-Wl!nO{U? z6RLucDZQ)~+B&x4C&d{p(`qB(RTK7!lhPmfc)+9w#puFq-Nl@90fqg{-N*5_i9W$g zQ|ImSBIDH3pJ%<+eFk42h9y5vO*I7tabq;ThAL`8{c)Az*F_L0^`hlD*veW^##>hM zzUgj`ja;_~H7L%5)uh<@E5vd@6DBfo8LX(kEfqM=^M@~`Fl)mVpe%_$2AE^fso5*6 zoSD$qP|2t*m_*eEm5TPfdU|`QT8|WR{YqbpD=8ySuc? zr`m?)v56U~d-Zd8?^WBu;~Gdo!$IW2h(NA&n-4rcgyrv+%#O*Hcscs&*3h;Lmk$g& z#F(`=l5e@o?BKF+M;o_@J!aL654o_bGZ^N;-wGLhtzFJ=4~KWu*Nd;{u4rssRET2A zSuLN31jQZ>o*Zlle)tC4wwy^D9SY4#<8N}6fwiz znRI8k9rNE&O#shjh*#a!=;uy`-KDR3agy<=-y~ZAnWyE z2G&Bte{)z&->6sK&H5=r#N5>*6^P5>`erZ@XN3VL4l;r|)%YrKn|Z4!K9jV;(L>tr zLV%{&-=U2|7G6a#5CA`D7}U3>=msTyaRhn6nN)joEJ^t1EOX;O#VvsqDMFf`3u^vW zaR1F>cnY&S(VQU~ZnL`sN<&Bg$x{5**lUP?*v$O0g{rjg{b|On;O=5O!wA8+V#XnU zcAk$=s$Jw;vHlS=kFFU|oon>uI}v)fSyd##R1GFtVpYR~Muy=Z#jZp^%@*LCPXX^w zgT^tj(nOl*gi^d^^=6At^WU7NH7CNR4tvp=5@^air{*p6gw1SaSeyy;AVvYKjzh<+c?DRQ#s+pn+C>3i^joCkLxHqt-oz+-+JSI&o2 z1&&y9NNTI8P(CNiNVDU$E;wmBiEI2mT>Zm$r?KGwCw*;u(ZBO?JEtOuAoXx3y7f0I zq4$sIh`(Zbuv&tS8UtVxZ0!tGb{`f(o+R=bPGjK1GU zw9T-oyW;O0Od|1jjS@VN7t3yco!$t2)l@ERwR6w4()jYKHZ~;9bcoE5lJ8HFvHS%< zYX270nfSE94;7;VM$3+y77>jQG>Y~-dp92w=>=Mif_-RE9X!eK45U^m4oW3D9(722 zBgB8=R;IBp)>nR`LG(6Q@7h#+4Y+*Hi{RwEOLcqNxPBaSgj$vV)RFs>UxQqtt-kV> zpxX0=)O|nL=bAf6<1it0&KM0bzHM<8+RDV}!I8YQzO1LSmTnX#FD{NdV;qb!<5S~0Gn=U^nb!X6IX5eoV&x$+vDhH_i2Kc#!(>}+4zYSmsl6e1 z8o2A4$~TMx6!wafVwWOJ-@(h!GVG%(h}T-cIX3qnlY1!t`$tz5aFDt~6LNV&Dc!*2 zsE8!+ID8J^sV@#!j#6aVYE_f4JP4?KEE=%HALf=-H&A!dbLAkp?AC67CuiGKM zmL0lAwp(n%8BH~1K87Z-@BxnyeGGT9^Fd@}s@BCgI_8?N&BTf zt3x<)1rr@3NKaZQ^g?l{Y=AQe?jq(nkUt;cno|=!p#2N4tWv)eCwpeE0u6n%uj8Yx zKM2wMNZ^=F+|8xA`}Z0H{g+1UbA#IL`7Czz$Xue7{(1*Z`}M#pcc_QQ0oaxvWuJv6vTa8C1obQoVeE0PQq^@aT__Y*{*@-k%FaSr+6+_D zD(wtqHMn~#dOha9LmzKIF8NVPf*>o(N+weaYWOKvsNX{z0JCOdR{dRvNJvfw_g|ll zorUoqpV#^q8EhEzdcSwA`k`p^ubI!x{fw@tVCRE|HqCDn@x zJ%2B=+8CC|5CAJx4-u3lLNVK7ljS?&x*ZNKX*AG0gKA-_EWNEwG3%>itFB}+4zyO zmf4{$M1LFP3%Q)VQtRlr6B?^i*tkok@;Hs^Nz2TPVGr2mqiDynA`_6Q{yMUUD10z* z3m`qO$PJ(nksCgJ1W$$W_T?x;H`8G>Pya zJ8Swnyn%LcrSP>$1;#0t(^`d@jXuPI=l%xUBj#uDc$rPs578_i840o1KEQ-tQ4bOS z0}9(W%BrdAY7G-6`s}n)_$P2m_-5PP(@&)WC8B;no{V+ppP^Q|&5pj1*}lGRe$~)N z;^O^>N-v5D_tD^%{q^VFNA#cPSXGcY?Msky56GQ5Hz(tvrVH3pbEhbtEU8oy{ZqFD z84LoB07EfRPK&Cn)k&z@w&OQmE)#;>87E$JcNZpIQQ4c|s;GD4ilBDiv+2q}b}7=z zBP zZFi*c&XEe@`|mV+13wZcIb8U_Ei;m1RFKb^3>AhwX=x}V2}1hO;?|Ag`dCL7%RF^+ z55gCdx86`)aM%1vkH*K-DMQxXJf7BKFd-MLorhOJ!>1(|jVJCjr$|D{%IN2@vN&wZ zu7`ZSIucc8uYbrkDFvOVA7d0{=#O@NlN5zS(yQk|!p~%54;hDMloC*paGl6_4MUMh zM^FhZbvlly(wrsED1EHT@5WhJ<3T6mFx(qsQ3VC)7^^B-$PwVi$Ft?|kFA{ibDbUB z#PjDDVK$pAkdq+1Cwh5_sTTZ^V~cTfdz-J0_4J5NtK0tiexHBNlXtc}0m8q5dh8mL zfCNlC@67&C z?*jfQnV3e2yY6T6PP}V!x^-34?N3YLFdwzN>9*WEf*4oorSX{ko#&QlIxJm+025BV zd`7?^vMYy|C8`y;)8t_a1ezi<=rKpi^Fel#eH;J8NPNa*>g?Z&de))2J@RFO8x!7S z#lKk2(c3tCj{I$U;frJMKd&FMJsKQxc@cXrB2X0wb4tL%Uz-Pp&yD<&&5-P>i9<1O zK0pxzK3U9l+8KMSVBIxkWr61PH;ZtD zrle*xoSr3>>c{HT1;X<$$MW?Ve!HC-4#w?0KQ^WuytJot&SBqh|7KO{l6^oa+M$je zFI#EUhwiPwJ~U+h6Vq44E!OxvbyM>i#)@=#GX^Z*>3d(Aw207&r zJj73zU7)9SoQaCxRN|+And~`O^l`t_{Kw>5Jk2GFzVv_>VmXv8bqyNX zN_P49o#D_H064VRbZR${Sh{OdHbC8GXA!v z!ag5_oEtQ#%}60~3hg^20Jd_$4u9yhK^?#Ro8yNrsRYYI<)l?exVyjkuZe^h zh+X;&u13$_KTKW1Q*P3-{1TNVR=93Em7e_KH{hPkR;j%DAISTmCPDn8YfETp-af&u z(eeI#Ce6t`r|wWiozExdgc${PTR06&soO(zT=~4!J7fuyCpEt}gtIjV{b2zAqQU2J zy>AjTL_n+uMXGQ>LgeYNi;;hk2-=v)c|dxpqq77Ulqa(Oosi~Vd$-ekPb*WXKVKI zu+DR9yCe5sunG>_ZHg4_rml0!kw&s5C0bk-;xge&Y*khJ;4Qi}N3P8x%wObl?!H3j zbMO4nYBpy7j}a(_FzQf&DA}hk1o&V60}|j%paT*)q+0#`6+JVoWm0YGY@U6@Rjcn{ ziE_%h3mXr1%7-nX%ly6%9Lf`rq`IehF>B7BFGa=cnBFKP2ZkYYs#PLi-OVcWgkcHY17733>r?6S}@?+ zCd-Bfe33lL+gpDM#V zJ%=6yJ=S0M*{5RORD@7oBHrRviY;$FoCaNw3s}PmWg7p@?yuC|7iuZ@Viy-sd~qbW zDQu*X?-nZ-FrG%Y-MKM+_F0}{N@Fx zMr&?|r4!3j%0^zfiSCqo_Zmy|07rD8rsMoem%z-`uh_S7VH%U)RBE@WtAai$=?CIk z3bQL@B!9ADIuHnFJ?q#Hd}Z(-t!zRJMS}da7w$u!gFf{dMegXB%cRYxX1+x^2kASt z1=vXP-Xx5#W|oTlSeeo`?5>pi@3QGWC6|~yBr-rWJ*RY=5&DK~n`CTADi%Y4eeaq7 zI(uX7=onk(P?W+d;Va*diUZNY&XNZLb{XhHLrN1^wR;8-k232l-3Wh(*0{5zVH@Ls zZcX>w+t~#e3*lXXTUeeiG|uZI-!lPAi9jU4;glVC+u7wbkGwE>;%S&{+eoP-5_tiV zncJ`5Elz@z|7XN&#B0K&Y#b^o8SdVp%l@di(LAtCs-Ky`mHm6ypZ$K|fEGQI%gBbM z$`uuSSn(s_morO^A8C?}dq1_Iz%Kqvp=MY=G44Aa!;3pur%DGB z$(*)&Rd6Ep6nIndfOb^kb+b1CoA-u--$Oq|m~AdG%S}~nx`3?{AyJM%{=Wy#E9uqZ z0$E_oXSfjjOL{k?=3&cWYMEerKXV*2>|zt?;}YF}FPmTU28Z#%Y$-2>g%=8O7NbcO z4*KE4*vQ`MmU*qsQ`>pVxlLir$P2 z6t?NI$5iU?3#%$?sei5em#gK!H@$imz*zlG%mnv8G&teHs4mc|$tQShQ!!&wQG?2R z_dclgUj$|DpWaJQUQtd?CFfUU*tgFJK^U-tm#kDI$`HDjD25LHLK zR0j+>!&BSF~{m4u8S&ed-H%Vj+2zd~EHfXbyN;_f2H$Jv~gOd$j(!rJ+oIYxcQIG7(i}7hpy?SMMJ^&g=D< zhl#}iICl^~_JjDb@inxl%;d2J`{@i3%=Y*Kx%q61Rc>sRF z-mKLz9v_5$aC}vY~j0XJQY=EESG@tx9hD*uMOW}lmh}&CNRfQ)A z5EhE;1zGaiv#cd7No|g?V_^ZQUj`*QrA4%~wBq+OiNQM7KQ2vcioi!H0Xp60Q8Pd1 zMOf5p#q@}CRGZmL1>wm{qDD;sZS2`59$F+p0C4Og8k8Jo`Bq=K>>_&%{sag)h7UzR z$pl>c1`_KX+EPTev78S}c9Kdgpl3w#B~YRO>CdLpf4JVWInBi5s8-XOIJA$G1R!Lh z4=G8y4JR=+uf{$e1C~T33vsDFP>?5C1tFooD=~b+lgKhkYWqAon3)5Wmy+O^MCa4VZ4J#MeheGe%!? zC;&^jVvz`+9Bs4N1lk^R}Mp_P}LSmOk;Zn8|TSo>X zBQ#RD3h%o(6NF2~Ly!{NO1r_fy!xnX{;gQyCBxylDD;qJoL|=H5mj-#Yvmm>K7=_v z3NE7V#6V-FMjs2Xwy|BSd-X<{cbY`te(L@T>&_2BFIb7hz^7Oql#2X18D!2u-MhXH z(@(m)Cz{xSCA(OOUFq9nG!X!!)64j0(nV*|dO*+JstpJ=bXya^0riF6h>nkMT^4L> ze&w_x5=c+6_Aj;eB8YC9BL=Rn8ESuYk;$XRb#NS(h$k>b!&qjxpz26>6>6Mjfx`$OkB>@kt z3=GIX*zq-@h#BwCAK@8*`C5GM zI->XCQcYIHALAxLu?nHl{}kUzA0RSwdbb)={rbUvWPa$6k?^Zu+T_mbFuc|ai_Z|O z9?%+v;?pG}U*#9<&dM+zI=a!e#AwxYe;r8YJo#wnX%Bm}b^fd^<2cM?XzBL&TAt=o z#A3Pge$|l#_*R^-lMgtPI8NxyWK7d#LT0qPyW0iAuIK_uS{M3pBUf6Iz6(7*p?STD zA9X1LppDHgu>)ptTVyPVv+SCG9wP$vc!Dt}a1zGf3ld5QG?gF+22B0~wETL{|5luN z6X0d>j%k%hNu4JUBE(p1aWs&cd6%0F-(;3%r<8 z>h@h}=ni`W70Cd$DtbOiX51~p!XI>kGBEDwmcj?$>|$K}u%!vSl3=Nrvvn|B3L6$& z864ewN`a86d98UYFeHef%9DuMz%e~EoJhbW(VHx}fpqo<#qU&Z|K`W?W0r;aCze*= zJOCP$ysc%A!T=GGuY0n~(Wuq2$s~@DV=?h(5dFtF1pPZYLce8QN(0ZYj~qy=yT^)2 z#O({`8=)gXKKD`_I%KKZnAv$kEKG0<9a4xV@iqq~$nqDa^@cQ~3Oe_lUs7=%qivpe`+N9wz zu;`Pz>Lc(H$&%&K^y8+J257WPs2M$WM}kGLDD|XGr6jza4n!$+|F|H{UX4US(Z;r2OQK)u8NQkw|8WzuGw zi02H)S%tDo#p>8gch;Y&{U82Co(2}`{{YxYRhBU|wqFiV*7$v=cT;zz==!sp%2&!h zVaGO6E}!Wdt@6)A(Vfe=$ZW?1TEOEA=21KZ6!iglSy4l=3W#RUpR73!rP|*P#w({R zSXtxk9!^@r{}`7OHI?-02btc5n&EtwjxWi9n4FP-%aiNsStu&Defta(F(k}NA}G7u zg^A3GZE{qi6XnAMmuOXaho*rS@vs%$H`HSP|2){nfFBFpoS8uDXGyJi@ZNXmML>UA zgi6M5e60Ffrb7I_Hp=l&$+I0#A?liM>FbbKrjCnTp2V*#KNjTR{g39j#+tcbb*`z$g9JxQ?`)lzlKzybi+u5~_VG%! zlSJJubK$S*ppvS*7&L31FbWx#m#018dHX&*Zu5Eb5hxB@8#TTUxNTc7zJ4+{#_;TT zg0jP2*A&hr?#Cz-B%W@>ygF}x-QN`u_2R1QyA;(Fq1GTS#d4U+{1mVMVt)#=-xihh z7f;?4#{UNpD#5%p@h0fh4F#|-xyx(O$Iqhx>2n37+9gDl1NNQHB32SJ(PNcXI$_7e z$$Gz8M1SYCSFl^(tsp-t>L~hSJScW-R7Aqv6*_g1_~fW*b1{<8qY?mgPXVX!3MQ?K z`Za%M1a>(I`(1i+Vz09`*7*MI?3!+mPn!zhgf7Vq`r3-IGDj%#R`V^Jm)DMXkUo9Nmu!iwr-bQBER ztt{mSQkMmbP!ahFy~{}6%&=LE&I zwM$uqbyWi}1!M++>emDogOhzu^WpwQ&7-Wp+~$q(APr#f4d@ zOC**fk2&5Le(9Eb$n7EY2$0<~jNj#V_kTV{2cv%3lPO+EObvZlX|{P4A-qv5O+Cyo z*KM6a06p?KXx}CMU$rSZV#}eB85SZ(<-6|<_gis1Wj4lOwhn}20s+v*jJH>3OPjyi zeC?w5?>qM&#(M%oeVB)kEy`IBF}}RTa7Mjq2%R3|6v`rEa7{O<%0*a8;@4ih?tm2||CKd>Djd zeoO5kll3P8qqljBxk$>rdV+erm$hF{?w?8r#yZ$N&@I-DU256 zvzEnOff`2h=qoOyjsqYuzDv3xCMH>l9vl{AI%c^M&Eb9WrU-UT#=t)n{g z1{odCoo+%jqYvjZ_`S%J_1(#mpvTLNCDA)Jam;bVNM}-nvG4{Rgj0lcLPB@%7E64Z zZ@ix0rkejLvc5TE-xlt-*ByIic>`};O2e_)# zzbJWuk6@9}p8%rPcS6P^GF$=`ZX~K^@HMnsJ{NoX4xTdeke9Y0$qt$N6LW|Ba?*+X z?A_&g-;R$F|8hzLcEX%Q9D<|0le5`qxY#i?O_IXW|FIrtQ#4Y_8Ue)d2;R1dU)V)U zId~hLp?70C&##3Ekq(lXB80a&tFRnOczP(H@}ofiquj~|;)ZXi^N;c}Re_)Pva0{Qc}ndIB7;Ynl*MDOpCd-FX%CMz(oCjXo^uAJucwo!ffrry(5 zEmd6CEo2+2C; zco9VeA$;&GPN8>Mb7)$N7#s=lP?C1BI##2~bZk^9>9%Upbw6Q95OsZ~@}h6B5B)S&sZp+dZ4f2Y*$>P3B3xaD8e~?;5K1IyW(NfzF~P-Lq_+O9k!XHvqTA28ebm*CX6Wjfg!V zX)XUEjBhXh;Qst=*HjeI|10>DyavrUXBz@VS{v*Co)D#=k?PEx+-?m9ZsE)lQL^L~ zFMapd<;{9@Lz3Q;D3K=tiTJ^pQxJ3m4j5DrdjuO*{Z;LbndES+vOOV6m9qDVx}TD- zeKP0`xx0$6Gx3`(EKVZk3(lYkdTNLn7~aSU$UurT|Eg*K&5uQVuR{1di_4gH990kA z+!uLF?CkOmeEC(WpDJV}ee{Og!HRYFZA)jIDIXmFf)fX*;xle9R&75fdAmVKh&kwb z>}Bn_d|Bbg9A8GgB;xP7I$d`VP!QH@IC*4rV>T6WoD%Qe9iCccEol(az)@%Fbo3!{ z@#Lh4_?M%Yqbs+I3l#b*3Vx8DuJ1g@vx}86Y+xR~+SQO)S6Zu0dbE(b-y9Syh8PKV zSs7>nLkHRUA#9xI2%B$sBVV7;0D>Cgv*lfWxf6c0tcTe&ayInHOVhG_6G}9n|BbnO zFgO=g?R{L-jPBj}$Ah#PB0*4v>G*~&-?a7-&J4W^b-8GRFUM8JVL7c69-h*aItecK z#Md^?r)hkpyp1kEFb=E?Y$dUSIT-t$@*Om~YADvZ(){_E$NyJ^M)cZ+iT?srbe!Sb zJBcaiHZ_6ei}fd&(gJ12drY*evG-*ERdZU_ z3T{v2yd(zu(V4K~vy$k^Um+-XjdGsvpNjv9Q|>lk)aMd}F2do3WW~rPd&VLKxX{n4 z5Or_5s%6f(>spw)|2`lyuwT4?szLIqsp*#I?!+Drz`sM8ILZ5gkL_O+`7oZbhqyHe z44veq1k1f4kQI`|Mu_b@Y0Z^ooVKOCJpciK_Jg4Lg7l+futiP2*WYp#v0EY}ss$&7 zkepn?JOJ;$ftFAfC`&s+M|Urg<|-1V`e8K&j&R6#yL3d&{S&v1geLOuME1e0@FnAg z#FG;MdH4vTZ*1U(G((u0+89nwPJThxOuE4da>jRj$3=e^Mpdi?Z7theTwHFFWWWeG z$nCwCWVZV78@&l2J>~sYBNKZ<-B|zrizmK5)`_fL*g|}*Q#!Hm4IA8rue`j<7Z*G~ z_`QiT@V1;LLiv*g6VB7gR#8?R_A*7+_T6zVj1lTr`s(fc0qI{0ZrKOOg`%Q1^lz=3 z(?{in6PuNU-eQ}wY9CFQ(_Ol#Waz`#*w&sr)}aR`6~Kl+F^QE^9$nuPPW3m~EXV(& zb++=;g{J=2+aZ9;71>59vRoL}TIQuPh=loO%@%04$;eUbasc4iDEwLxur9SBN$?|A8 zztwwU$#{CSD=EzUIl2_y7h)+*XV*2ElW-Zxa%MjE-*?%UmVJ-zA%rXBlJ);0b1oPb zs-F{4qqC9<1dyQeq2sIKJ}Qemh@8IBLxb#Yp7Vb1WGs|yu5M+U*;hx8^#28TXmfuO zYXK^~F%gOc+`c!)W1zBM3Q1IRm_se8XI^_lqI0phtzvFON!Q^CJ9ALNl?w@k!Wq+N zA0Kq-5eoYsV}JY~uVz#v;tuR!w^M4Yg&@?gsf@+B{4xYyYZw5N)nOooyGQocba$Fb zReI9;A%{uF(T+G){7eKM@V=(dCG!vb;qm%$(BND7I@}WZ{Ca6v{F?B@b(cHC`P%k1 z{C2p$sT{1lU6XA=62+p@ov>|@A3||CGq6m!`fY|j;;<(!%$v7gc7Bsyykje~*nmn> ziH%Zq&9XpYc%CXgH8p(D=JokhmGlBcG{KQ0A|(` zl=wR-hOJ~mJ(|8BuShC09Jh$KmMmcrkf+;3$82Dj$~`f7IQ z22#_0f)iq)d|i|+zi#~WuW^-hmj@mck>9KZb&?+m0S21ZJR^g;s@7Qhhokjy4nyTF z5LyRl$Q*Y8oQk&1-dpCtc{=#)KmcAkp!47R;%JIFa|?JBgrr1%_(4P`VXI{4c(@e% z0{%V2{@HV1>~hx4nh*#*A*U3+OVf7@aT-disaUhw{R@b^(sc;m`yvQ=dh6O%v~#x6 z^0gr{B4c)SX%lhjd&fA{L!9mALPm&jxy=;LQ$)&VI`qQ~ipIc+E~lOE9SSrrVyPky zB|5Wp!Ln>CoDBu?rsJtezF6>Nard+<1W`JKytnV{;J=3L-3CdQWMK7|>et7~AkP;S zNZ}#T7_O~dZqa6Zj;(EjUEj|?@?d`+TG2d~2>D8Ptd z5YY#bY!d0gG$v0R4ZV`dY6edO1HoHqhh3ur*hHVY;H=O+xQ{tv+;q*2jzB8ln%}p>9x~$6?cBeCs^I? z-MXcohKY>;J$2Qr<|p4@g7#{_4OS(xRQi*g8yjdn=p;TluNGx>T*3|SEq-lrpWnHm zPE;6g!~N3MUW2u?MqlLjCT5lI5zSH(sB?JCKK#wM30E5x+wn<9dHCG^ zwY+jLLV`RS&RX{M(fP&=N}^oLV_YT`LABCQ+1Tv6F#+E1C4(g;f`2CtJ6*gI`hl%( zTfIF`p!Oc8w5F=zJawe#^rWP%_HQ!R8D%H$WPAEfo}&7GlV2}<_vzJdDl9A<{KQ1v z-n8zEeJ|*`6?zf&*#beGC1@Q+41xd;4qT_7zR%1jU2WfTYPN;%bXr zFS^WFdXuGw`vLXoSxVj5OTLfgIL<_Y0Z>F2f4!k)xVS*i7>B3(!Oi-KO)4&5`$gLhenHJpoZ!l5yT z-3ZLRwvyW}15*-lQzXT6@xHFw-7T0#zw>$m&q*bSwy;+|2_FQ>!Hy+i*1Bs92}>l1 za}tv0n=5~A3+Vo?(51UPQ+0MzWaj*Zwa+@uYa6_e2$FUoI@hDEOWZl+@ zW&zwcf`4D;?c8NkkfIgn|?DGlt@ zm{Jkd+=)$!$b`s2tZA&B3Px5U&D3pY;D;qoqN|#3t~mLe82ZfJxZvPm0EcZ_vg_ZEt%~wYpZiJ2W1XYEmD8je%9@zS-&?#4KfBcR;GQt3zjd zJ? z7zL=@A$*sqvA?2jcuEPjGeYM%lNqC_&^hR|4}e>YMllk)@+p-Cay_*5V1eoU+}m4W ze$Q~$^lOV)(E)YxoL2>0WgOAvC~E(Fdw+L{8=FE-p`4*djvI~em6~4kRbl@0@bdLs zCJ@pHH?4?|F=JbAz;R!ta@I2)N7Xu%1Nln)0eYPbuhnoK!uguv7D0&BVNXDL8lqfM z(#j{+H?_j6gVEt`EN!;^Xjnk)4}vp8bBUy_OkbV_%VFU@!ACY(a@F&aqQrh}8yp)` z6fQM|&s8K^P1YmI#pQR578IZJ-LxH+p_@_(ceFz<`@oI=$JAMWMfpW*oD>NGB?T#w zZjh2jLb|&Igpuyjp#|x#p@s(O?hfhh?oP=e=e~UJx@+A(0DhQr-gEZZ`*}W(b)@P^ zm_|PRA-e$*F_zXrWuTS*80WZm_D=_MB?a)&B@A3O)JS_IHxaX1%;7aQXr{VN!RgE#Q?}HIlqUaaY4i1cHPSN1&FQoUN3V95s)7WBF>(>9&Rv6E89VsSfw=v{UdV8;&>~zY$u0q|WPZRq zcn0Za73^ND82rFpp6h!vu6j?~`{$SFtJ&`b9vH&EEB4J^|GOovrD?l55#h_1=yfpP z?&NooBbs5+>x)|&QBBDOpU8)d>er~GM@mEtF9W8D8-LG7w$p$^kn16@72>E50$WL& z-*`;Ei0@OR0$V^`p||J<^jN3U_q0=eNANbW#rNCVmZ((O{F2f&1PW`r6$|0(8SH2{ zY{E{5eYqAR@y3{&0bp|+1@OxYgyU=nT7iGQs0o5&b~D!G8Dn8xKd@0b#6>01$Z|bs z>WUdY-{WP_(hn1mno?xY3HbT>{XoIa%~8%{d#Zob3=EJc!mB*wJg@j-$aRTQh`(oq zq?sT-nCP28(35Ybez)}FSIYZ@H|eERaqK34!@9DtTMZ zpL+|LMt1i`X?H(%n@wkNml>;#6@7t*LPsB4LOMQYtU*`qDCs%rIAw5pDlwJ(hFMGP zKKEi==F3_3I+9b;0psI2ireZ(UscK_ElrFzvI?h*`Iv?I!8p|F%>KCA!_D&HK3R!{9~EVwDlV~2Nbb&1~SJ~&UZMDZ%rT*Tl#rhj6iO< zbF=??9(|_kz>R-S43iq2~goS``HR zpX;JqYkufWh1|f7d_GJbHoQYIqY9?&E8=+)a&<{_fS>i~JhIK+_`&3#x_|#!W$ps$ zqBZvyqX_Aa?BJ8h#c6f?L>ZQy~5E% zRgcVtSK;C@3B903&Bk>u1Hpgmx8R+>uSjuTzmFY8mPp5sg$;EXM5O)oN(Y)4UZ35` z>4&#cA1QG<&_)ah!L!RZCfA5oT8(e9ja$Nv;CHjI!4ysq%SS}KmnBgP*HS1z7qLs zLp)MK!=h^uf&yY~*M_rbVa9vYZyL&!3IW^lBYxaH0rZ?ztp@GJ$tH@Fyam6`v{F-Y z=Fqt9m)V|pQQAnJX4#{b=8K3^jl6DYSRWU)R<(&zF|@vIbxKNaAB-YFKWm68R3$++ zxq8kK`1x0@#aLttI~HhaaO&yoDW&&bIJk?j-0n($3!(?~MRvkpBjzdNl?V7@PF0XVs_MQ@gu*Gl*8^&dFc{K=3JUCVTulN6uJ)@I~HNZudCxNCtG~oQakE-xAQ+WOdapj z#jrKXGctJ99C6yAgdgaVLxu|?6nTA)!(6R;axM;^per-DIVH8lUCOdJIn&l?Ea1qpmV=e2r=QbU%^#}xezOL;D z-=BMa*R%R;aHDj6FVdV{*v$Cc3%l7XD{P**32*zfrGLUcl0)kPj&;AS_0r_uNqaiW zRx-y=-H(1+jjLbfx|R3ix;4*=ZeR$Zv*}WqsU9suKGZkzuBpF{X}tVs%)Ks(nMY4A zXl3*Ds2A$#xf#yO@5dzvlIIypw@nfXXF=`scGshc} zl88lI#vex%x!LAcz=x0Vq;7W05OlOt2@jVaE5ZdGDGqz=MQt-<8$#ZRMPfTgSDdZ~ z`5;?Te-@iI-=RRZ?Ox8^Ax8<5@6T+f z`=HYR1}Ky~i5k=$zR1=3WQ?_TSyHz`^6uS{WmGW#JSg4c1=;;yqo-C|D^B{2 zpzb$IE}9(<_9c6)-|6#28U)(N@JEIQ2S2M3tGm<18vh-(iaHNvYqgJ9Wi}g~b`c;N zDeCg7cGn*|sW1^vpH)mG*>L$0@HE5r57bN8zYVcb#Z8%IiW@|q+VX3zGFZgv%9DN8 zkqX_x`Ah<{#fAXc<0`t%lS(|4RD_Z7#k^O|1Q#r@C^lJn7qE-8DdruTs~Zg1gp-o>SC{;AVwlNy~Jw zX8WoDJ;LE?CrRn)@mPU(dF#@9>ogPb$E#TpgRblPHUYPv05>* zA-WbK2*{;4T`JH!$)&{6fZ}ODlxvsDopgj~|1=Ee`P;>(dUe{1zKd!743|ne&N0K7 ztRyb}#UkfU!C>)KcIx+ndS328X#H#0s;a(&I=Q&0bQn~cFb1}hVyBqOx1JSXoWN=j zqSgDV?T+A!x&-t+vy+x8gS!E}f6BC`!3)mblHh%%#<(*eUf?W+hA%dZqhse?s*DbM zcLr6aaJl$FV;Uwen{FegtZjaZT3G2|f@}5uzq?3i#Xx{^nKykR!WW9MHmh*F>*cwe z3%C8L{lbzz|H)Hpo~-;P($0$c&_O1l7T;YqNW@L^OYFVjb_Y~#^8mFk!cNj*-Tn-1 zvlm1X6#hjO)-Dzi3fU5NB*50iHxwYa45n`&b2@&OQJ+#PSdh1O5>z7}w-(uFN0y!2@N~s%H2Jrc%$Z=jhV;Nz#=-^eb+|<(~$wk zkjxuB6+oG)aK7fo<}_3Px0BMI7!FL`D}LaQ{2CS2@&nC8wx7Hz3$htUwLO8SpFeNg zMZUW)Ku@4OTr-%Mu=1_dI70kiNG^VnM@7Z~Sg}7CYq%rlD7j!O>kI}~*6X*a^IJX%JOn#DxB0%u%pSn87LnF@aP$g<*Yt*M&_E3o2$w2I{8CMR~L*SK-z-xTJ&Tge}`CPx=rzDz)$&Tmxkc1jkL%PoE$%$&*N z9yP^a#NyWky&qu^&di3$7pO!miIDR+Oj6@fEZsrM%VpZxIV$^s2|_Q((W^=xOp7Bz z?4O|>Jp6{Y{F?a%{d$m zSSWwJ-|21A9a{xDZ)xDX@QaORrIh4|{2Pt>w>M7;(hDj&JaiyASrVo^I7tt_}bCM!L8)Y_}P<^rB0Z(#oEnrJc3w5Ip^KD zgS`>Tj()w#Vo*g@6y$oj&HLew^tAb5fC50i*NwKo22KqvzN7dX@@PK1Lb~mB`pSt=G2!&;X3wc53Dwsz$Q}D6FkG+MDk8h5gP(R` zQMUc|bICwR&c~tQbyM);tTa^%E+x|fVrQ7*wEx${KR8?Q>EfMwVM;^6ai?}SL%}KJ z5Ew3O*ilBg@B8HtO#T8)8_spxnkUF-yu~FWR@}(bZMhSl?!RbhaxGl{!^Gh=tP%+g z`#k*#$b*T^yaMc;)~l^#O211Sl7JwXr^`ttbg>_+$T;4c+=TX+59E$5Nb^X6z6w3l zVhDs2?UB6{u4Qe}XXORa#1}4io;>#PnvO|%HXBlWXu1+1Qfle_G-A;W2)?0O0c~(^ zBg!7uPh06fPu}u7=165%Tu~67*{yd5z2g&Ezm$1hwHZRkKa_YqtZ9k)By>nQiRGulL&*K2hI zQ9So4j0chKDAjbS3fB#>0Gk5R2=gB+g1g~QeqA<4-lM&c<=qrB-1^7%l=9>rmqZ4) z!}BWp3`u)7eLZSCmo=V0I>Bf17&d_)CcCJ;_7~Fd^4abk=5or~C~kY;FTC!0Zs3pR zdv3_RSfoNuJDpcWJjY^bE@J4!x+&t5_Mfsi#PX59y+6cBa37v|M2XT-_K3);^yT`a zAus5;@z(~A0tEyN9nK)c;ri|6&jsCUsLcKF8=omr*H5y1Lw`MU`s%K;?yesSHC=j8 zagEjpOu4MO@IvZN$E~r3y9Z*6YP*53pNoJ*FQpm%p7RkPX=r8$!ewe=f?IAW+q;h5 ziJTG@#SW0x+u)C8<-<5(jiB2{i}2q(icnDg3Lb`H*ZzWgq=a>x%#3nd+&Ht15=j^i;SNEg-A z`zn%y+WCX=Oy01GyWehl1V`kNq#p`l9=ln=WZU-fL2z#;7Y5H=O8%xoYb8*>)0)qM!Go)72=8o#(-Pq7WX)Bt5@^>8-NQ=5WU(EyHQS;@#@xCYOh* zgAdR+;_98BXuqfDQ%sw7u|?Zm>hr9&FDaeG8H?YK$rz4l(OM(^Q)H=g0G-2>LaxVvS*4N(b^pPtu*q1-x~=Iv;E9j5txUF&pl6~5R<7^ zt$t&TsnXPO`2cWbuIC{;X{dRhPbw8MXW)a+i%sP0jNrV~9>~rm(Na?q-vs>p%|FDp~9=rok{DBAeFV)1Cw=COQ3K`^9L(nMnfdsNRfJ;yi7giaJbHN+4CG=WrMhqbI= z(mDmOJ>V7HkY~CWev^uh@vN2br#YC9Stk3rig4-Nyv(Eky042?CO&Y zoOSbZ_cuOvMG%u|_YMneS=G^I>q#kUpUoJFG{SwQ!#2!pn_sKbfvutiub=Kud*5U~ z*qzp2@RmKU+pJyZA~yY-M5ymvd@G_UktJ-{N+gzv-}zC$J6EQ6AxDG=ir%I})C&sK z(5{p#hNux$ka0VE#*y6VDG<4h(=2Hb9t16D&+N@l-g#ed<0gs=l=bPdo_xwMax3-3 z2K5}dk_wqM{HMt*g4gder9rp``}wNuQf{m+XqSvp%Ea0 zcZg2$3A0f#YB(bOC>}0gO$~i>nh#Q%6#|kJ(HY2J5!_#<2-BIploS{L=HE8@?jz;T zF}DG+)g9~y9=kE#Lb25Xi}A56^2?IsPeEm^!*4H39LL-WQu*Bx6hE-!U_EmD=-1W9 zZ2zn#wk}WNm^5otU6wip*ZKPo@^Hl^G+1ft|99n`So^hL^15x+hq zh531K7e9vaNehs04;#?6PrRB7CguH44V(>=A3aMkyFlQsG@}mZg23d@AOOT6k&LyJ zj{Og*a)$=f!lH_@KUQ_gSv87m|3ums)f1(@QZ)WrYE3lzdJAd7v!TWkh9^=}saxg) z_;k4&{0ZH!EggIp@SEHNsd>pCN93tnjg4{uxd4dFM8l!rb?a{|a1mqR3vHJZujv`E zjEP6%JdXqsrE6+Yjx!-b$dEj(r=bbhwI4gxrL?0D+*$12u%sAFY;BiuH;PqO!gwVY8Cx7o?l;D!6x?Tr(|Nb_zhT6}I|&a)4S`7z!s@w5d_h4UBUiFBzjlGfX? zbe2Bkc>;*gw}G3xFw_@>4+{Oz+F3<4ZPIXD;LIMWs)9J!w@3UbP%T& zN~5Zbr&@W#IBa7$p_!_t^>KS&3Bbw>u?k|~M`v0LDN{(Sz55FK6fTHIP=k=V{ttCA z+i~^%&`9O&ePY({&(IH8C726D(sC@DC0K*cxxAio3ls6y#;P zbK*~Bm=@Ct^R@qNnH)*v$E&{N>hj?aoqC62Opam{J|3_2L)XBoHlr}N{Q8Yp5^+ds zx+0X%{;E5?h)%eZQ%GpFc>8%jcH04-E|lY>uCG$!2*lg`=n_+g%9vWCOde(|vYxT> zKbbXL_!L`tG2W`-h4{N}qe^Wlk5n(3y)4(S;rH6!aaTP@MOhwDP;W4|;#|WkSN>Yh z>8?c%oYgt1 zD3#czD_--97Q!sX)B$Wq<4(T(GP*v}o)Aq_P>^z&Ygfz^Lzkb)wIPEk`Y*_Q|R%u;va+=%HU_*xRynh zS4hAo8^&0HWaN-k6&KQ*?IWRbY}T%5z#TLBjO;@@0N_X+i5dp3Jaw=Se52J%f!)^K zvIJjs;=-x^Aa=-D)ppxz$Rv%wXCbv9F*@?;_VBuum*EJU&;DvuSo{W`46F7myH%wPs zUyy)Ydq5k!E1yA_$7262EMvXcH010{-IrNMVNLMA3cy zV8B>F7WZ0~QZ=L{i~#{9M4mCUzc@T`HXD^M!kj3rpFs#sM70`6Uhh|TIf|%jBu)ts z?-+Iv^i_`IofVhsaVz$Z8r*qLt5TIPhaOZjnb_vNk5R_5w$xEz=N}ppCi@X{oyR}V z$BX29RkOA*Ctd~|jh_TDAecmoW8@aM$nc|0s7r4(s*!3;L9Mu@@g#*jP zFC|GukD}*`m26gI)cy(pcph={O?r6k^Dcv(#7B&IFWYvvQV6=A!CPDbCa(-DUr6(R z-AS0VCium3$1oBCoAfjDyI^d#^u-AB>6_3@*Ge=+9f5xHKr@+)ev?Pfr~0;@%gU+1 znmXR+BTzC$Cw%ZxTEV`FzI6pW)d&Z9W0x$(Z-am_L^e_%3RPf&uek#Yr6DD5(U7w= zJ@X+7G;>n_U7F>@C1I_p?b6%TbFUPMjd1ea(!jDhMh z+Wd{LOhqDoXHK7Gsv^e$3aHV2qdg5+{xOlE&5Pk=fJMS1?9_CGi^HcA+EfrFen+`| zfL@^EiHiTO#|8)t{eAXmM7?+%>?I;@PaT3d z{wQskQoq#)?U#nx-xU;kUm-cXcGC$eu&q5bvR;AC2tIu*9NoR^W>0#ZZy8|Q{0Tqn zg-=;Ff?)8R1U>U76s!-9Y&48F$TlZ3W#aiA=!70gof~_G;btflZhwh>)5T?os~1x7 z6k&(aD-Hp5^UZl_XK`}MHg^BKWFE0X9le}Je}&LQjPuM-P+!5l<~}BN@*8H!S!I}t5LjW=ad+1zjUR0`@%`t3 zh&U7h&0`a}Bi2WZe&pjApK2a!}r*rXb69OeIp5xP!aad`gy zol4>F$39=B4L!^U8QGA&!Zr*y>5So_hpsQ^pL=Zb;;CGei3#HKcT~|(C;#}<&-WxW z;eX)>v5Urf(QoS7UC?1HMN9ZMCy|<&OQD8>yegyF$K2P&p~10LmgnH~EBuBK@HOx< z1&uQA@@F*KZutLewaxfSm-pg3h4uTSoO-`Z@&1*f%PmP#!~(vaAIm>ByTh3-aV`L5 zO2P$#!#_x#hflYsL{EJyHVKK!t?yp_@ZcJcm?UG!Xy*lhhW3eQ=g-&#;Fs_1lpV1( zCMZb73^w%mdd>G0{Xkth2;wi5bJFa-8#4Xf7H`HY8EoSDChc)i?s9|{fhORTYRUdY7my{c<24X$!9 zSIAH15mt5-=q6Or0$wUzBcUlo+WL8!KCrAz0v>}uVAg+=`npgL_zkK{t07t|}MiB_Dt0K(+85{w*N z;C7e9Z8}vRJ_9(oBl%XnQ^oj2Y9q+*OVX3|vVSMzV-#!gt2cwN<#lSIK10~p_KxLh zZDp?%BT=V`G|WKpJ#?HkzbB{Ba2wHi-uUUW;fhM5?2K1-j3+XwxwU18TCWbOvA|NK^0`mGC4tgLeB^=jW@8 zFKRA21|I13FMVSDq;iLY*45)u1KR5}4V_<@xpG{OZq%!;%6AN&6B2MfsvFGsSQgo2 zcW4N7Ku|2t-9m;Zg0c5T1y&wwx<_l;6e_MU8BFks(lbmK$LjXp71}Yh%TReZr)A{? zpG;RdCuZ3x#VSe#V|4!`ZVO0|{+h*o*Lz^E?|st)x(m?Iwn#iXCT=BOta=H})J<`S2C&r(+5_z@~e-l4WBYy33CO~q*?kfO`qwRN#hpVR{)fbUT zZjmdhtCYqM2chOJnTdh9tKHT?M0<7$aL~2*fBU2~B{ixI+oQbiBr_WhE-4h^ zRsWqWob|dgDol#9TdC%ddUK*574UklEp>jvXy3D3w6GW~Yt>YKwL8kRFrdkjwi5WJ z^0Qc;<{gYcy|1iQwqsBii47u0biryVRo9Ycz5nsCN%-WrP|8mwbhyT1x+XPKWFHU1 zX`)5qlcrv9M2q+hh-qKR9IZvyBjDt=C`WkMxU;`ux#5oOs1wp-Zl8w-9OmZ`6B8z- zjbeWH#pBKN*|t@YHb;h#O?v#FyyKF}sWq=ixTT-nT@iaC--enEeYiZhgljTaMl`M- zao~psPpe6L3|>{8f})bc;$N!T{DDSA=$wpv*ug1CRu;QFmu&XUAAm<>b}2TXRGB`2 zrY0e|XYfM#*Y{}NAc9wK<`Q_-(0}c{x&2j5unYXg^m==DL@wMx56$}OPHz=5#Xgyr zbxq&7Eds68kAP~W)^Ls2Dh6%uM;Z)9v$slmD}SDo)D0r9{lx?`A@IW6;OOiln8?wm zvXkzO?QFqnF~8Q;uOf7IN_*m|K9zdJv=}ScbfKNBbKcQLVwNFGJZOu$NS?zTlAq%w0&|*Pe;M!V^RGr{jGDJ+7WE+r^clfjp*5cXn56>kWZKbyz~L^? z%bBWT`wyM<`Vs1}P5?CWS}UbS$b|m(vsaCl*)?bwI(FnfwvC&fBXj`n#$KaYiSL~x z+s?=2)NeQYBQSn)#1t@CZ#EEK=BMiJM+w?TBK*$aMPFmibBYr8A>IF$lZatK+#lB- z8&@6ksUWPc#iaXuoF(IrDx+6qgIVIistp`O7@fB&UX+Sf!giVy)=cDm*zG5ySr)}~1f%=IbCo2BBKpRkwZ-)*a z2hDYm=fV?LBb^wAw$5+g)%eG5Ib^=tfhCgOUEU6}w0Rwst~SHkJv}KrVWUcdPb*wX zo@Yy3@SDymiWb-s9b3y;S)$Ryt{fW}*69Ryf6{peKb%x*z3nzF6M7hPx>>&OOLJTC zx)|116RzS-a*cZ~BumD<51Ltko9GdLsS4&~s(RxjT({ypDO9(7cb4WR=x}48wCZ$I z1^jjA<4wzD`=k5J8BEaWR7h#XlZd?K-mdL<1Q^i#B|T%lz8$U0K4)B{+QX~t@^qGk z->A3FpIjh}|AWkoF^s&?vv>y@7c=Puj7l%pC55ED0n;L^z(3UjNhyv*Ym-YypyI+ z?GA2GDV6K|Z-gXWhM5 zIe_HyJYn z<3hP6&ky3&0tM1*)9dbW#??QlUQTkF%q)6!3*{D`S2M#di~ct!SS2{C6>N^6uF|u> zf|rG1OTpAv0(G;2)+HEA-{$H|osS2ywnNW}hDOpj_^p=urZNqsq%P_KQmWYbP-AL{ z6>db-f1PWcM~WX?GNXSX!ZjsBqy)6$l6&m-_go_GY$omSU-k|KT02SW1O1yCt&`Ok zpnbw&Hp5o!HcOBFWCF88&G<4SwF+aFfpec}Gb^nPO7fP@Tue;Dk?A2ww1B7THNb+W zevwjrYq}6NUpd@VNq`o2ge-}WU-G!B1`vorj+*$O8DON#UvGlwHXo8!-0Ap!mZ0lv z)@=M`K26u09U;%FilYX^wcCFXfb=ub-QkkmM!CxT(zWtS%WgMFcZaJ+w?qY;r%@k3 zCkXE^jhr!IB=rzXAI+zpjk$Ddd_n>3?Cf+7qxgrw6?fnQ0>4Mq9W;Z?URM~|lx~jI zG#-7WUYs($G-{*n4Jd-3kkseOh#!Z%xW5YLWxb#M-K{$e*~vCKZgZ3Fi_U?m@rPUl zpzg|VfiolGs`1Os5KptC?ejkpeOEFcB;CB8vR>4!20L7Lb8KZ*)=Fq7Z^(+tA6Qo_^%6=2PdSqO#{mD}3P2ra7%ie8|&T(fca9WOcxsgYiyDMq zaa&cn#}nNN3eAy7{3Qb=bLA1f$%UqlkGZMmMOa1u2{bpaX9N3$2tjvJ2dq^u@$wF~ z2VVsvGen`2pXZ35)S?{}OL0Y~)!ogiej+%MZc;T7YK)h-9nt7;lA)gA6u>Uj#Ykr8$c z^l5#Z!b`m&8>0<+Jw{za)q!T39oauITHuH%Y~1;5WRL-R7~BdZ+-evSa`E{Wanh4& z*b>qjXm-AjX#$>{sg4bYw1YRuAjW~xEe)6d5V-2J!}EK-d{u2q8NP-@LF^FsXu}ru zQ@90%gi4Gr3zWU_`+QdNTDw2TINjD;&@|)u8sC#>^j-diV?@q}Y3X0_XvH0|DLi1x z3sIZ-SRF9pXdHktseu1{13E9Jw`~x|ezwN*Y|vYElf0ubgnF}_;HQi}=I)Qeh6DQn zmTJN%$RxvWwELTgKNsnzoM8V$DQ{LBT+ISDHoD+Hg5!3zi!k|{Ge~K>>BuLoiB(&& zB@(C%C*eKMM>Y3{3f1k zd!{vRKJ9zZnta-6T-N*~!De%SN*n@|`uDf{FtM;)^AmX#%TDxi_a4>y*@RM>^>0Ok z`}Ukin?YVzjwmVn3CyC}c_i!KCc#_keDjoIDF~1TJn6&T(SeJ%p^@V<1Plw4m01@r zS+bc;pUv>3hw$Y!hLd*4k0O1XyX%W{Ooc+B`2p`xlO}g2s# zIxZ8{xR`rZG09m7yrHR6Gspbk6vCI~pVOxE3_;iu^Qvlo7RUI8Aoj1z(+aC4^VPFn zo)UOfEge=Yf~&wMgRk|sm`_GtQ>e>$*X`Q@&W%w8OMBB%3%*IlrdSr1cXjg#W(x9R zWGbXnnN|&-;h=Q4{*80b=Ggk1)R9)fD#6r0(A~v)2ZPN#7k~g?`p-ze)?P95RQ)RM zAOEIwhlwJ{+z>6t25EJglrkK zwwiWIs%c+_?NsBO_$C*gPQ_CD;dFR|yob#4lA=g}rDu)|qg0E;cV zMxt!OLfoZW?9)4Fu%IYws zip}_yTLzgQS13rqN_(@-yF`S+LZ>5v$sV`TZF@o&-cvJ3q`{&iI`}ZqbHPM3%*C9n z)j-S%eOG2!TW;R4x zDG}*)G8EIm6}me_sfq*%^>59+Fppwj{7M|g1eB|Dl`zS9do=sVekjc#_ze|HkC zu+zkg;Z8CFT46^)(y2Z}4z`H7>m2pGn7&J*v7dp0IexdKwRv9CtyI;N>QTLLbt zg=SYwiT~#vLGaBTmetsJoWx$=-9#`V|)Ig7`-gdLb!qEJOyveRtQ;x6%#Kc-BjRd3N6wY{zGVpoe;Cswz?I{>c0hb!M^V(1ma zaBg^{{!9dbKIF3vCr*=7f@W{ITGwmR=q-4ard%Z^)?EKP3J)GlJyEN3HeRj_kncZc zb}89XBn~J#oim6k5QZ4KEBfZb%gOzpA2i{ko)7)I_m}D6e<59mTFg6xhTeHOu5*>? z!hLZ}?iAsXdfvACOK;sX-Y1~_WsC%%(uoFCe3N%iMr3L@rnv>hji3i}9i~EKig2#M zMtgHzV>}xN*UIO!KJ4J^&l}-@_p~NgJ&PRP23F4ZG=~7{6e#*yXjY4zG7~@k8?Wm8 z2O&?hn`smj71vjp?N2~jZ{;lfy!j4WiIak#q2SnOYqmD_89J)_X|D2fv8M`Im_uUd zC9hUK=H^?YLC?F3`{8;!NMvfgy0@C}B+Vgc1Y_#XmZkQ)G^ZKsX*St=b2XCb*W(?9 z*FvAYbyVa|+aZ&*Gpy;oiOG_{uTyR`3>v%zKI?AP$EAPNZ-v!7LqbNYKY{w$z5%yH zY#JJcSC-ql{B1$z^EQAft^-m<`F|G(I`i-+l^S+UMnV`8Frgz%Hn!(v~A4OR6~p3=oY}cD9U#*w;&psI<=~+ zjkP(UuNbMzYArz)oq!740=oZf;Je{{YNXW%@6=v)p<>~4`2WF3N>jw_6iM`XXQMzW zp|+e?+DMYFzFucJd**N>;sy8GWTSXEx$%DLO7Zr-c~la)vu=+XciOo<#R_N9 z)H0NOd`Nb+d3E-t{S7=Tl|+g!s;L_9ib!C#K4Q+L36PfHUDA2_jfk;{ZT0N(94X7E zwB0U>2|o%yt?He6KHQ76-dnELJw1%TA0Od&H}D@Y*glh%t&HN3ki^06WQ*+Ae7q|+ zJSab#wt!1j1kdTG`IscnVk*`C1m^(3ykybFk!{b9Z>Sd&LBQe|vl-9U2#S02`K`F# zrva~P%slx5Lq2JJ^Q!Hm161n%3~^-z)uMfs0fkzm8{U3KdoNIV{?A zp09JvOYhrhk*iOs^VB%%|H5Pp{y`2V=X13j85e90;56$R?3$LAhk%M(?C2g)xe^px z#pGF?jEL9B6mw{haU>eBwCDk4P1Nr)s09^HRC?sZbKO`9j|ejaRP>aq#q-ZCDEuyh z4edcea%y|}q5@!Sc?^f`YURyK+19TIF50y{Y5{npHJIH=<8F%=R0+}qL_o*9fT;Ty z@3N@rjt%SFEry6lmh}`KGe4rshJE2dfFP@`xKAqhVK?6bBJ-wW9<{>_$gbD@wp;d0 zz_;&(r7Bejiqosk26&u{8{fgSZ^shZjBG6}3rkC{S0Rznh4glKb#je$)VIC2CQG!V z5#QvD1s?d_nT|{kk96G*+rAqt0c@l7bv`u#hcM%7=dL3QYk*@_v>)#TFeD&SvSMVL zuH6x8SviUEs|`)K0JKOz$v*G-a6olI%EmbLJ{vdm=kk7F9sKzvQcyYIgKw2LZW>z9 zU&sgH$%7|om4b*%_6u`D`<-05#HC;^)LZisfNGMXZ{3oRuOgM63zUt+B~Te2oZ!)| z`(cb3?(x&ZDBx5ief03EB5(PRSS)`lZ|bcg+)ER(O7D(dSaP8IRboXN>g6_Ag0<#x z@FzbI>usgOrc{Hz*WMVFU{Y!)tO~ZD5A_j3LPnMjC%M`PWs6a>ZM@xSd)y427W?yE zK}|u7ngd%I3vVH_Z5f=$T|f2ukpa7^kwtHn(_`c=(eRdfQj&N1@}%kX?(R-XLc&L} zmlQ`th!^WjwbP27ipqGt(hTbTbZg+YEIIPyEVH*cP(f~pKc}=&lk!{G@ziSNYVjkO zmI9Qu+)hj;Wohz+>KeEhvo#h-`~Pfgpy0*J$OPV}`=ZxNRR_U*PM!4rr*Rmo3_#La zzcoMPW<2xLWdJSHJ~x8f7}z}dMBnw6n`RjFJnA;0WLTgrdtC(p7F!MdxX^C;kTt&0 z+2=Xzj7wboIhrU63E?@s7wH{auQF`1*Bf1vAN24cPGvAj2b5QU)Z{1wC%Hr9SJ-lKxm z8S=__zt7z8@G19D1ZW#z+LqFOuh#(Wro&z58|+`m&-s{mJg%kM#gr%jf&{Ia$xa zB!)nTLc6fUEDZTqpO?%tM)!MeF>^AA~6ckY5 zUu?EC&RXjOzGv3#pEQQeFM#@BGk0i&&v%S2g3G42y47S0cfqa}m&f4~On!#;KdJNl zC!O39%-L@6rCnLmNyBQk!%mQx*(7a((hU_5QZUi!Kuv|F5@YGiP&``nIp^!fcWe-? zidPlDw5e-_t6{QjKDP0Q`->#0;zg?pEGk>=JKGJ4-V4+bTQd?^v*WfAs+jfJc zWNLL^9RG9wv9>Vl0yf~U=Is_ahfVIGG|!)Fwj zxVr$mC?)ho0A7n%uX*9Yq)I8oL3ziBEPG>2JdjnO;!GC!} zKsUeQuPuC`PC@{ke<3CLJmFYAh@@V`!;0OpeY$M>Yy>-|ux;EokL-j%y3nrskaNid zmVl?w82ojJnmgPsa<((Iv+b*uRZ)sV9*uD92=WJWTauLcT2-d^yXQmt9C$Nx__f1R zPe3_xma=^TA+If!0g!<$r80lw!FfPc(sYS){0cb7LPq5UB%-8{mStyTTY{OpYwsch3X1(Mgdi*hKBzc>_~+V1(XfzxNA(Hrxg* zJeO@%3i`;<6}kE|=rG9`D2u{!bL@wdRWVQqfNvy;e7CO44yqo71CQ>6s00LiXp3A3 z;_~egJ>R^Gg(3(V+NvxjnJu0Po=@>N(3PjN;C!Pa1|=5^5cYk=H@~_h{74Eh*~}<^ zpwV*2=N3{A+ zASQYF+~!4mPZh(z2+B;`4%gTe;8>>50(9E9$mwmR=<8fP2(_kUi$dRbLx)@cXzCv( z=CuWYt6HoKVr?vbv#ikwpB(tTtx5CweBNoE6V(&27KNTQBpIn^VzL3yF(QhZFK)bZ z3nMLx7Wv7FJARtnUHp3kuO}wl-jr7akE1%>e_i&e>7hRpOCNFRaWPQ2-2j+QVBrmh!cF!J=$!tX zjq%1&Dd&HPddsLfnyzaXcMb0D7T7oh2oi!P5G-i0jk~)O+#P~LaCdj#5Zv7*xZBsc zpZAP2s{e24?pjr~=Dg+(9~fj({jZ{D=81YrHx3f1!5fe3;$vXKNV0$EG;(IIY)#I|tver)T}`LFu>m#=+MS3a!O7~JL|6*hlEHqzde_j=(zW6d2r z$W5<{1MUqkrLYOw$z&wk!q9cgr616H*xT1Xq2hgQfx0X|HP!ZVy9@=x-p*5I1?{4( zeSE=?6)s!)Rs9dD(@%$pq#M4$56;nFo1^_7=k!4qkl8s$(~_l2<_cErdempOvTLbu z{ATKDXrK6Dqq}vbew>%Hr|R>Zf{ggmWveM4=utq0KbJ7E!z*M*e?+z0yV)D^=Wo)g z8(~@O$h0s?)F9~mMA70zl+J2p!E9TwCzUYMBl9?`*hb+GZ*NV+=VioFLDOiF_a&w( zM%|B@NA=4NbAgO%2*!_yBWm!&)f8B11XVQAkIa&x+ZF^iK|h#b3j|I=o;P*jW3WY) z#hsgIOro!7v)HbH_|g5nX`%cm=cZ))157gs@6SX>^&eXJMfN3Y8kkzI({kH%{sVu` zQ#gD%l-ui~cLQo+?bW`x0+PA|W)yboG{0_@isz22}K* zm6rS>B%$T!j2Ay2YcaiB9~Rn)@SVt)g+FLdA4RSag&~_P4gE%M3BN_y=6h`#R!3fy zSj_v=eKdc!eW1;^J7~j4uQUUGSz^)xGs2?=m6#h3L?E2sE_Stf5 z=(Kfv&kUiznJZ6+fBS3|6WE3V4`9nT-{rEppA@cL8zc2{YWP>?(m zaaUIV*oqG}kl!8_a6TdY^?Qbu2jTcqhi65YPmh0EfzseMBW0cJtx4l4mzP?J)a;F* z)W=yfJ)@>%qPt}6*B7D=TZNG?BR+U>BKJ>YA7N_{y^2V$t^Rh|s0MU=#eDo^NV{IM z+t?PpZhA-t^gPum*miR2Kd$F0JZwOx1|V)Q@7T-G1fOWsI(W1}r%~oG){PVbk>kLW zdq@4kX-ak2g6iuWdx(zBA~6IDUzf_6;++OCqe7(3vNyigT=^MLupC{C5nX2*OuA^c zftj7#Cgw7k#{}sny=yOPEKrsR+LvF~)5OOQr0FfHW4K;L*|nQC1h;j^Y?5ko`D=lG zK8rczvKX`DXG)@hNhuF~WqDy0;p~;%oE{K6skKdT*X5+p6pG1uCWAS|-l>V;A@{$f zO3hleH@y$dq1eT1UtFLLfBZ%I={t-PyNyVFs&kz#s?VyAjF}k-U1#7sYYk}l_a}g! z2UCs_b_KrL3?DW0a2@_M!Y=QtS3@1~VA^2hvxVJw*bjXLC!v(FLsXbQrI@16?}Ouo zJn%&70EoN@kGAN;cTMN73I*CnRmwj8AZSWI(8XW6wQbUwo!89M!BoQO&+n03n6&k+ zV~7_|L2}hYl0~78jIAf2zcieqV+)?of(ECA%g*v<)xb8LlO%Q>Xgq(VvELZ0u_%V$ z-*!+6q?@-6W!7#La64X6jm4p7KqrwbQ|-wAdhg(CiA6^g&@mc__M8;Yx+c;wk`2!@ zjfno4M7XPt=kxQ=A3_Y{3nz^~Zj!FMR_Ss=XOZ0)R4%J4WJo$${xy--S*{uQRn#0g zWgO_4x5aGb4}-^dA?U@&?LH#^xWXnJW)yF#8R4b zxsJbS&tB}?4Cn0vcTTx;1*ObOCr|%joV3;aDcw6j#lsTSl>3O#>T~)BL>Hh{l zKMqdsPhA;kW}^Pfs9%(Em5%2$fy1NpV6e7Qhaer=iuBzO@_keH9%wh+k5}y4ny^KJ zehgs6n{-NV$ak1Ywc*4PNyPxCqZ@1pOFAFyC~qYe1nth{CjUJc4)Gh@(ckC$m__0T zQU^3f(eV8Z8M1ZKZg;LY+Jgs@Sv4;mlq`juPqMof@{|UQcy`q4lu_m#HS#)jJPVPW z@yztA7;Ve(zOR3Pu!$=k00rcShK#TcUbSBF|%M%*2jv zjcqzYK`7JCsU2~ekLXh7>OkYeSa_3{p@NWsQ%fTUS?gb?3Q;Tk#-c+A!*9iqqEplr zoi=BWzcmWQ;V|#2O#7yGh342bIu~6Q8;Y--6*Gu=oIS^eVv&18XlpZy663fXDbII) zea)jDR6qlkjKWs01Dda;1lGDWTvIkW&$+&mfxmjn`4|gvHTe8S_&k0s^!+=PJ=1wK zoD}y3bq`A$lcPnkX;BpRz}CBZZN;g6Eq&p&`Bm@vl(2}s&9G_rnNWnkKWTsH+d1uk zTXa`I=%D$rW2bMkMH0vA-$q{N;b7XPL7TM#)X-Aj0I`}v;N*1~7msZ9)+ggucz57k zn#Fs<^Ld!3VA^yi5TtGloW$*j-y4-bYOs{e${P|SOT176K$MoY$>>b%h%2LhV>~-c zN={{w5^TwnXUPos0xSya0QWBlu&QV*6_$fBaa3Z5ch&zc+bqLkpUxIL1H+nZ5yR&GxA(;~g)Go_^-HlwCy0K%xE- z=lh!nr@kmH2R0d>?N8^mJ=bH5EpW09O1DZvDz^yz306q+Fjf^9kWYmBBju^DT1KOM zMZIde4NO*t0||7;@%R#R{T!OL;Kh;OyDZetFTF1>z0b*K8Xsj+I^PsYH9M&77e{n{ z*ShNTZ29&Tki=0r(iD&K6_g{5X%Hm@$(130tw0@L#w|vTVvCkv$0W-R1(ZCz?b@gmp8ps-6N0^Dxw$!u8}4TaWLV;fR@jrWBc}5dVS^mUX4ZFPGD35hR`%Eh$)5 z=fL=W+~~IIkAA#|2sehQA!>8_*lh6yVN5H8CV_ROX~1T*gsz(<`K-&K(bWp=R~g-P zd=LXJTjfGpDA@Ui7rvs&SbZ}pe@UkXcKE2)tOeB`PWTEBzw^#GmZ$2g!Sv<(Sj+%u zdn8$)0flkh2*gFq-;s(;FW5MXp7Lp_>*xfivIB`Aib7=t2P2Aw`S}5C&>XS%MQro7 zs+!(z`4jKBpC)`>rwf#lc$Oz~B+M?Y92EtI8ZFFGtiq0zWX%!DOB|Wn^Z-@SXX6%l z-P!@*t~Ny{E&YINN2zxh)iQ3Woe;hgTEXBtzO}5}@*y?zi}7^>03;(7B_GW(sVkp`!xOrH&rrvq`J{Bs4LCpJCyUx|&TqASD|oiCq~VHLHxN z8|mcwv65e!for;Euq(K8uB~hnT~~+#6#KP#Z%L@ zqwsZKDxwE^GO2}=Id~fkBSkumUZy=HJPUm%aLckZ>j?Pd9?MC@AgU}TEu&-6#@948 zUM$bml7f2Ahz7ELzdXHYUt}^*x$-xT$38lc%Sr_pLA6hYy)$rA!=xS<>aDi|mD>i*^E#zTgnVJ$SMAy2Y9(H+?rp zn%#7lHO^XAAM;YZ8|tR4YyY~=JGm9+Pg^eP{e1no>fc4EFvZ=!|0^Pi!p{qB3|Z;> z`wXEtbMY+}s%+B6PsdbM22@=kB)zD-!TUD)lA}rAY4`i=suF&~`U$ZcaM)3QM+yaH zGreas6`(Hvp+3-kP}a*ivi*J}zJ7nok$}lqcKN?T?{R~eOM{6^s>bQ$kFEHtuxdwe ztSCkr*@4BC5xAwl@@|Cck9&WoCi{{d`I0d!iYY3KD~*T^sWilTH`%h2v)of=E7U~{ zqM$Kv(D4|@I|u{jeyySDRjW?ymzO;AAK51|s=X1xni6PC^8({bgW$r0v~Z>le})@N z%?ZGv3L(f7BoaI7+FYm`B44QsYfmkYr z!Y4W(AGJ(DEG)<1i3O}mKi`4DEKiXSZ`>LmDXGza@UZCxftKqSNM`QDB@(C02O2PI;uyjZ#nU=U$nIJEL3YMm--Rni?! zWTIYTB{6FB&G@7GrXiPr>;xDL69x@%2c3H*A^e}qc92+SW3&+$hAEXO2l#s%7ElPs z3;nnAVVH4H3A^4!b|D2r4Z@5MFh==H!uJGz%#f{RpQI|i{*HgT{VzZKO}9k!Ml4Y` zkK*^$@bwD%fWRlAU-S{45kA%kAzg~XtXPyvNU0-!P(klDqhe5WFbR0{RHk{1X@q=l zFEsj@mHPHG2Ld7niP#S&16>gvOMM)5an`OJU>&Ock+{Zx@!P=+pj%oev)>v6+L&@i z5PxlUC|LS?cq{+tY~bAcXrG9qD+OInPh+E{?>XuDqF(GFAP>$gGnB0S|6dA&dTNW> za$6xR1|SlMgb{q9J>el%N6&1g=l}W`i}4LiDurUD-Ff>xMFaw2+&bKObQJ2u{9k`! zV+mF^X{LTYS=7(GT_qJ449?6i!cI&>u8P6bJkal;c5JDX7TWCC^hds~y zH?7O`U2!7p#}t*dG+o<0!1~kz1ZGUuNN>u+{e{L}wcFQ076_3M&JpwJz+!&qbUDIq zJZJ_|vs6sy|1`P^)tPkv}=Xb|E)J(q53_U~VbohlP{Ov^`}cQ|4HCeqwi>U={3 z)1l1?-|?nrVS%t%8@NOm;?oj<{A5b*$T2cncd3n}Q?+W0le-@;U zY|fAvxX1aKe58^8_HbJ2(_^1~FCbj5&YASf=hclCZYfa>Ru=IR zPYe^PqNx(f%Q`>Z)>ieR$b%Dz`qklp$#&<+=rYhO#5(Ec%v;>+eD&*&8MQdH;_Eil zxeThC?(fVv763-oB=|g{dD5K!vBK+eVX4H&(fXYffB#})S{zJza-}$KE%_q^pi7+Pqp#Sl zJlVCJnGBWnrm|+J_#Mgz_tSYU(+jpiS8x}p!kjIdzX;#Nbezf_Hv5)b)Y9W#cU*tk zk!P=UgTr>AxrUo%k;L>P%C?fgzkGIq*6u(;F5;H_nEjf|y4T&xlUQ&C@@qTXA(2BZ zE(C7q8{=(JBnhff$|OAS zGu98>wDk-0?;~gP?0$PlQBi26W*K3BvXLFIxxSg#dpn6;1|k`=*Z5$)a0>_?Qj_je zh+qeRmgP8qJSOV}`NY`R3P6UeF&>?(&WcMEi0k0|_>RZ$Tp={Pm_vSC;7D#5hb5^ZAm(X%z zd&TUl>}-r@qkBnpkeI@}Yw@h5_T2k{P8K$tP)CW1p;1u~JC$kcW*&Ig&Q-TjuQ zOBWu2R?s*!ltP;-tG;YoS1=2CrTUt7r7mfNpZO2X-ij(%WFc++fG4$B@Qs2ZCdNsq zfrr3!;_#9z*@SEsVy3D%JW_x=1LVyXmGKZP1~4zT2u}Bf;2-TK=nb=4zMfB?Sw@ao zS_v+hnhCj=4FA=|wI-}~%@nym-#&x~@6r)K&7^8=M8^{YEJ@D31ENS+ILo*x-VEeY zYA&vK;dN1YjWY4{v0iAJl84y>XnF<8%`AS8=^9a4ZJZF#52U zzfS4>H{JxNMJ0T!yiEnxj*fe|&Wm7Lj{rKO>wa%4{B0!gw+oDbFUK^0^_vC8-Wve? z97~9EC^iC)1zg~+TUW$SrX-K8!3p>9=g*PqzR3Ze=Q6N$_pa&p13sj%VG)@$^jB*{>BqdbzMALI(0057XTN znalGY(BN-4BrCu?m>UnRt1Dh1-nH_moX6<)44bU0UIq^)Hsc_3Qzft2!|&;$0b z&ktgW9UUFZjW&bgEWQc)>UKu`VR%&n?>ja>g!{sc@mog?^&#D&Gt01aq$w=2*L`U~ zwNSUC_~`jZ$UbcKXhQ_$Xmn8-^2BH=UFJoDUAo=X9QXRaF<<(sX-pR0TY4^Xw_cGj zx_hrTD2iC-<}0tkTXCCG@1qm%Z(^aKsx|&aYob231-mX_JE-p9Xo(8G-l|Z%gE>E) z`2;fKT>Gl{Cx>q@9q%$#o}(F<__4PA7`5@j_-s#IjgNl3St0&{k{OI!F#Kgq>{NQq_k0{Q` z;clgAeSudQsy&c@4s@0LM45xGO^M?TxVlG(D~$0ol}3NlJb08eSPSNjSWTHjVKDuv zp`#;i$nQKY>Ghs5hfE1EHfP48G+ST5aQ_na7S4IKm3lmaJ8-^q{%o{|7Z}H`(Tv)N zOl!QX%hC@lW_Zb%j9?Qhp`F5^cOTZc@nkY?dYq6w+xlOBj43hQ*#T~f(nb!lmdHoo?R_*SkWuk0QCr;nd#yX4- zN?+O;QwEd9ykr-cewPn>v?GQGy%|VJA^3Z0BzKue#7a9CRqoUmC=?_n4MXEnq|T;| z$LV>p5-sxdm*dR)eu-ku`?X+R*LyeG=lN>=_3^dn?aBK(U(*7nVR%OhY_@~Eu=r>D zQK;N?E7D3;yOFLVJyvZ@Q?==a^n@NSQ=Nr<|h z*X~aPX;nX+$)e6m`Sk^J2+g<3pBGmq-KOhM>F3zOEOt{<=%K!DliE`KVhyDhf(|Up zzQUd7L~t*OkjR~l>2YI z8C*{=T4~h*&-1?3xQvVf13sP@eVcmBD`HY4307giMXCXPak+sI{^=TSgYN~>&=^~1 zmFajzwCK~{4n%T%ySSE*_e7=MZ?6*pZQxI|iP*j)M<*gL#^vm;3$e9j`!&CpnLq-Is*#QdASX zis*Opq}R6q3KepKcai#uU&V0ECQ0$zbOmF3pzUDwv|0{XzCpECX)vvx^~?jqtn)H_ zxdJ+;x9%FWV0uMdT%0)DnSd#mWAGk8{DDF~=U~sl{qhCmBtE>7O0+hnOy~Qvd71(5 ziKAkr4LbYci|5Io>lu}ilW8}B4dKuXi0ZFzMlT6zp6_|xZ5D3B&@H-knLP*WfAe^p zAoa$@bMXCYt;X?5CUq2-z=BD zc@>5c5HsHQ=g#uHL{`Lb4UX`}*=i3svj$onUeY`dPmxM4HjA{}9k;K2S12@?aghprp2FRdK6?V+QD){#U;U za(&GIB)TmZr^t6wN}1sI8o@v|JG;=G+i(Cpu>*EB#$6462(U|t%55)W(i}{3s{5!p zn3AXi$W%!hr)Nbvy|LR3-P!);ZD^B?i{a1IVJ{|XU-YX>*mQ35d66#o($P;K4>4H{ zS9TzvppH*HC(2zDh|UA~WA5#JU_raBfZik^BJ+V~KF$F%GW`fMYA&LULjPcwL>fJ^#R(oI8I+Np72k-juU~$Zg9=NSh_yO7d zta9fwQGD$n8ec6heQTJ>5&dH0?fLXYi zK4J?qGc%GNBOI8SAu*B!=Wn_gsKM{tG!dDYsDPS5Hahq~?ar>16mQdN(OF7-=D01m zXvy?{leG`oF-5WPKS}6!Gy#4VHa4B2>F%X2ar&PTzSz;C@O(S3)d>IWV`J_MO7D+> z2)2ZE`At6J(kgcUF=FQ`2K>zY&Zq%o z;7jZ^#XpMH697eaOWwaLlkcK?Okt)W2~a}MzR=frGiMQ{cLL`C8qGU3HwNJm)BW&UKipJJZnW{xIvEbqTR z9)#1qjVH5mxS{BKxm$l*h@U7hSCVir7D(YMGoHdQO(^e7i1FYSEQ}dQ#UbIV5Ni6t z#zgfrUs<73oxfsZeUWhp527zEikP?|iU<9uvtRk#&u_DPAGTN#XCuT)5VRv#U;lHSjVxN#ue~tHn?aM@bj>*AW$bQo&K>Pf?;HDtz z(L1<3!EJOmUtf@)u-|)*NfPH6aB_r+q9MNFH+o7-qxx z%-3&GS`l-guzzGj7wnk-h78ee4cG+w(7iSLm{9+ao_&Zj0db$hhc@NCITq2|hC^|M zbe5wyaejSy+;jMc%xXX~ zWZFX257_gat^V-&R7<4QKJ_L9G5WR3_tG1m!dLDzASAHF99stqea%`moMj9)tP5$6 zD)e*I9LGb<1Sn{ktaAt>xu>8K;?`bLaOqd&futEl1{JRL8mAVKxM`tQv-PlTo#dqJ zkSrNKYP+Qe*C}) zn=u?dabipAszO;?LDuHtY&YE)K>vKC&evl4TljK8Wt4B=4|%gyMGC*|VP7N(y$E6F z;s>;>&{X~{?$5kRKPT#bP3(x#%TD?V?5c0RrN&%M1OEP-)5i4%xlEN;wA91pgCx

    m2bE8d_H$ijkzzq}w#rBOJ3j%x!;_0%`*kayu6Oj*bWBVaRLIGt?%(~& zng?}ZClrQ`n?|KLRfZS>SxxUChiZD*0Res-8}l~#i@AY?{s-4A70vZmG|;muqs&tGnBBzF;L!$35< zzbHSqpkc5d7Wy^alBtT|-(LLR@Th-ny-FiEE$d5n+j74d!6K@8r3CXA9SNv;8o zNztGm%pM|;Z>6)OS;Ypz!a|#;#br%lNWHkH#UFq|!+x9>eR7$ah@G-1I=S&n^e2@> zutpkp^OYhBSdm!0)Bdm{QBG#~bS8ZyjPC!=7AiDqqRVMWI8Bd}vzD)o=;-NN5W)Ua zMsuqt$2Jx*^x33NYgw;JF&=2bY{(zz{uI@LYyq<4&wVUNAJy0o8dgLI6jCImoN2}P zdRbr4CU^LlSgpypILps{T5vaaH!FWOeFUCxB!Go&GI5mei7O|&mfZ$bc_MPz3h3gr!xAyhDge zK!Avw{@t5?Ec3HN7bpmDf0nkc8_oe)>2mi)QEdHo3HBp-yZRUUZ2-R zPu2(9Nle4=(%ZXoAFnzm zYLwE)iALkaRr$=^m%GV&OF8oKkSw%clS^uM&mW<@9HT*A19MLtYsZ}_2??y=(cSBh zEx1UXB6qQ+@{oF1FbsM#D%c*?=61e*VWk?da&601wqa*^aM#|$ARv7DOdk0@ls{x* zCzxZ4Efzw4=E728}QC1G?)V&hpCo`ac!QWq+Ue9bY1CH+@M z`(0o5xD3hVOWqMR9vF0OTPQ_|BTq*SgmQL2nL1`v$S1BuH9vY*?`e0Wbh5uSWw7nG zMgM=B{a?jCCT7x(Yt8kX=(LNNM&8-NwEp5L*_@oSI+~iuqsY_YRZE;U3pAXmiruMd zsX(70U+uyp*C}EtKZ!d(Ri0M+0r1`&Z@-=*!1=8mv@(8{fZzB!vXI}YVA4|i*g1cm znq+Gdvf9<^i;%ihQYspi^27N9NwW12CT>j;R9;ZbdD0E$g6fuej0~-eR}CVANJA(f zJMjn2m!nehih>`sfP{fVb3iC98+YXDDvK^<>hs!6Hgb@fEymHFuD>?{sOJyy!6o!j z#ab;@&ciUu%0iZ!g8%`rTJU+hW)s$`KlP`K9SOve6K?6wVFNF0+WToKD^`}THf0ZV z__1e09{ezVxn5*D8*~9JI~}qnUklBq*FnmU6Mot?9JWAbiXlOPv*|kZo+;`$2man* z)7`$Alz05^LjEk3*)-oCmKLIb8E0(==fx!C1a$F0@B_`brUCm_M&|52U1~G~UGbu7 z*2_O4AYt`OY|f3700yyd}x7Aw>7)EYtpC& zhF&Q1vs09{O@}w4?ej$;4I~Fv-C>!yGi7G`k2zW|QcvTWPLhvTE#0XNFgDI=Tg3PK z6P?6!DzFH{esduvfAR{-eH{dOir$un4DL3@?%NDM@L$*gRwGs1_)xCK8lA6E^N(Ba3~{&!THz&-b3}!f1#by=eejLcm3O`BCr$?Gq!X)}IfMKBR$g zk^^kack)e?KfgoYEA>x`P&(?ekY|f7VX`U9x}0O<7r4 zTbs0SDjBih!CF~9n@EuwXWjx3()_9|cl0)ePeRor)otf*3rjL@`1if^`i{Z@zlhDa zQ|2siYX83xf4#0*uOIS5@pJ0g+`Ib-+cQ=3kK5=YbXvxKLO6zo+rICyA-;{AteA29 zv2aE`Y(bJst)j=#K!LETg_;23M;Vc44iT0H`b-t&poVsFU zJ_zYscfLSQtAo-;N{uQc<==R)b@&~gk4X1NfT%Vde$bsOrlO~-NR}BLWF^g=A21Lk z?5H_2j#LiIaWL#pH7uX~v#gMIrFyLfgP!h+2xp>2?;Zgewg_Nz>+B@HID116 z@x0yrLi&8x*$EFBBjc+ve`Tf7>`Fp4QLQaV^v97whK8&h@@(WybiC!iWlwaF-ae?) zh`fzFg;!G1Q0>qZOsV&ZyqryT*u0XJ1P2B-k%5=my3o#v6A=&El?-YyM^Pqz=<5gS zbNQ3Jzsl?fJ5P~lO5uQ3T$46#P%?m{6M2=4X8mp$PdpM5RMH6PaL|`?w2cD!N%~)K za8?FPyl5`jY6pmXY(e6Fi;iuThFk9fsFmj|m9*M;9BvCwib?DlD%BNs{FG#9V2cRi z|Knpt06|M@qgrp~a+bmm0d3ny-)!_D27o+NJ2XvyCJP7C!JL_86ldG*Q7sY0<#H2c z6c^c=IPUpLXSv?4d7J?`=kABAL7=DGQ;%b-Mvm2(I!~3Ol5(O_|C%p=+N8tN;mBXL zm4J0Yep~*m#0?~6{1>e-LntqftvJ)|pG$NQ9QO_;gXn|j!3rLB_*{R*LR2HT|@-EOIib9mUuMi5ng%m}jjxJL;Or!)pumj78MEhZr;0qSVEo^$w8W~}#A^Xhkhdvb=t`vt zRV5=~bkC5h-Aa(ct#zfW(MQ$58O8a^8x-GBXN0~9P-2yqi z^<3HfL9v6zs`_EeHKX5aU;ftf#YTPBf&Wz!0FSxlRi6y7#71v2XWw8lLf6yEG=`c@ z+(aN_MVxI40q8})(bNIc=;U#-YhTI(8_)Oe?^!-7$)x=8iF~r%yCBcFZ?b+iLCz|s zMpHEoiQ58c#RIkijMvQxifl6BFAdDoxtRJ_Q1wmw>zi|0HeS!g1ka866-e@Wu#N#1>@8tx7)uqd-p1r+Ks%ZyJ*?b z;peQ>E0!y=X#<&=%Kdj)8uG_A8*n}Z1Zk`UgPdMpP6A$lep#=X!~pA0RZ@EI+UB}- z)?tJld69V0pPTYQ$TkafIXO8SY~8wP%4r({7kK2Lw&M%xz0Tw6324RaFNEBtSiU$r zteUTIwXFB3R8=@OU`i$U_Td7BrF`HTzeRVOKS$|r#kFy3Q~y!ed)qHWdZzVE*CmB- zqp$JKm1H|Ik`MHNc}^O5C@C_X@)zsMs~-JLqxUH`;Qjb)Fw3x;^xyWgZV~c^7k*E7 z>jye?#_=@gW4rn$4!}D`cKEh71nx;2tPS3`?pC7<0hPn8&6G(xk)-sx_ID$ad(Py2 z8lBhO?ivnAEsCFXzGj7-6o+E^t+OJ~{y+`QjHFsxel`)hY`Pz-nF9`xK3oQz!_b{~ z-3Hb&`&&LD9;}|9`KI^Ouelz-@k;%M&2wBfkdxRRmB3*Z-;xcuOMp8W(rjf<+ZYQ= zjgK^br=?hOo;e2S>IAHVpuhuB*9dVdxH+r%rGo`7v}(b0A4@0~LIV4etiqXA-=%hn zu%68=|Gi&5^WJCAaGQzNG@_qT^A$9Ew_AGFY9yU;Srn0rx@fHaUffgIBy|@z}(bA;ISN65qYfK8>Zr zz8EUJG2D1}Y68x5nDbA@@5h?)l0_a*gfL0pr#mzL76XW_VhV~rjhphL7pWD$?wcU@ z^4^{%yoolqJLGQLDQ2aU827Zxvt|ZdklPEcCXroM1F~W|4R69uCzn4B3pM8nOFCvZ zty6bW`E#f8B`7Ne%S}ws{3-(Gri*pR1uBE_aZ9O!ehK2K14+|>KQg)q^NG`XXDD*B z#ESFreCJ5#@aAhx@G4Qv3zn&W^yV5U9loBhe3Z;ucn? z8j|~K&&OI&zx+VPL4b77*dr=uRh2(J3;pNV-2KxNtP+eWiY@)+q+%`~nhT{&hVf$y z!zVN|91aU{o1QFu;*C_vGnyiO8Qp8hHHH>GK)VOTzTFO}=8H8AIKnxFC2Wmb z_Foe4<-)EnhEA1iZeV2!O4o5mHFN*eKWs*90jWkTR3jwv$nERDzIDSp90h{eU7x$BJ_^6$URsiPJ&{CT#eFR*e3)H&+ zichSLCKpmti>iW{Ul2hon|*+5l&fzV<`{09(8kRu4hnL8bHh`>jhz)*i3Kmos4$9J ziCri2S+T9j4q0E3P>kTPyx&!b!)l8D-?gO`-i&@`)uzaKF)wT5Iv!V0BplP?sOD+n z-!KfrzO@k#B~27g=={ziBsw!?`fx?m$_=mla?9cD`DI#!ysaELZa1NuqDDbWqBv8} z0zrcKhd8 z{Md>IzFp}XY^Xb?Telpcf+7F;g}?UXd)S_Q#-*vPJkzHQAiyrL$qSKjTm-sBrJ~hN zIQVYe=k3GY2i&IC_kyLIRA+%@=i(;A1*~2y&b}A6?>&==`BgSQG#PFEjzi7^!rWsm z%~)}yISZWPL}plijG3M%*$GHbJMGuXvV#vfU@HQk0(~hyA!h`C6SNyKYz1MCeU}X~ zHDJRh*d9wF%<4cJL91Kz6QHJsVR&`vN|}U%#p3#lr0rr&x7i-rgaD4S6idODb~x`z z6Y&yk1J9Rf(sZSks+Ge)v}=rz5k6>c^wmIj|4CmM+Ii%0@7=Aa(klJGnQF1guJTKp z;;=`eL)48C%(R^d=;QL(pa;IZl8DRTpqxPlx!%3&FzZ~dt;sHrxCwv(Ih{E3{5_+W zz5REo-Z8t-Sbm7_BJybjZOysrE#BS7lOBDw6>+W0@*Wv#bFUKVZll}2R{~R%io^U< z8nsCs@RwtQ$HVX080K=FS^NhUwQMt-3_A1yJf^;uhYFWAMv5#~( zZ4b(0N&RinGZ0t6ya=Zim5mvtfBc?;iW@?TlUOZEb(UXt;NNzQQi1WUkkuw2hRk;& zVk>;(yProuZa9VKeJG$-B(q^w-yBN{hT`~Zc4JldGXJzM`W6a`?I94hs(IuN%Fq;v z|G~pT)TwFY)1~73u|Ml650O;G`=Z&2Kb~a7RbQ;A2OD+lX>UB;kAZ+)+f>#JlEMkA zr>dp~lf0Bn(K&5&9P@vZR4HFK%k{o)s{lY-RUS<0Ycf)hXVaPz%KSc;up@#JMbg-t zw0DGK&0MrqBkr^LCi2n{J!qrlVwPx3aP>QgWoi6QLWhB=wcSOy>M(bMaQ1JV{W5M;AJ%p@%KKxi2rm&^1VC)*g59iAD1qv z!=U|5#}-gCLb1xG;8SwvbH7M=+I7x~>ES-k(xgE(&Qcm$*q6$V1}P`+hu!3FScfI6 z=h8utNm5&+GFEH22m_M~I@^fWTvR_$ZOEGjzx(ISlpJ{h6H{z0{6B~?X2lpN`P){9kp7}oYuuUS>s`!F32f&vhTu0dRnzVz1%=9XS z@|^vtlpsG)Z%@xgWe|9biC^WvrIW(a$(pwve>}X@>IMZTc6Yo%VaZ{0Q$}GS8RU0y z+V&9Vfh106XQPe2QJic+9$%s?m0TFw&E3`EdSxZexc4p(c?{vuM~P>XaUdJ#{4p>f z7IOS{j0ARuZR^Hj9_1*kIGnB3s)iDX}njzfjHX6nZi70H*$X_p5ZejT{^( zru3dvHkq_(Y%`Bowf#Grwvf4)r)iP??<9C@ZJgVu6va0DOFg%rv@Ax8^CI1)lzllm z)kIkWkpJxaN+^k3aa~K!Wf_eVnPR;7T_*%PH*-P~Ft4T}iOWa?{-sYRh!gcC1-Md3 zO(;+RhcAxA-KnzXnZpp$4QI8QVmP_3)baC^)l}_PTdR{uzP0-$i%QM6Gy_``y|;lm zUeGT&HU()CG?|>&mxni~mzPs5O@yi__agN4YMNs6D8i$lVn>34h zM3}%mbm@>FGsFzo9!?c>#SbAHrJCH&^G_@-8ZCCO5c*Nup_Yj<6}hYv*# zEZ=h)G*?U`1=#7Mt^p#5%e6_yaH~aa=BEW4M{j4DQLrAy#j_@|r53fSfMWtZ+4OSk zdY*Uf-;DbM9{%_noxm@Q4p~GZ@vxX^j9iYC9oXbWP>R{yMeIo878E|u)v!xE!!VwK z5=Az8wE>lt$ysBzGb7F+=p?}^!I65nN!qYJ-)L9;_9iLbN5hO2sAJ<1B|BKolamF3 zLfr9OF=yN`Y3bUpXqXg?D7A<~7n;p*A*3X1HgJ>UlB6{Kh0f>a8Vu0qU0Jr7@O*@c zZu(ah{+(@i$O$3qm9qezk1g7y)0ej0tJ&M{Y-8Hv>;z26<-(u4`YIEdoHekKtdIM?7eqE*H77?*4?n zWF>HnfZvT4@?ECFY@n-Q!F29&pAHM1wG3W?=@?<2L+lnG@m+%hpo;iyJzh)*ELr^T zm%>0E0Jt6?1)g4LXCTwGbsGi50sG_>`=79>Xw)SqP&i1EfvkD4hffWXheLv-@At_5 zRFxM~F-6&ucj0?E41}< zu?#sclIa&7BVXs=DV`Ha>-)8I?I6mmhmqeHcbqOmP`(Q(X0`X8CK%)3cW+aqar(nz zu^IRHzuGw$rDF`u)y&E;b|F_vwSSlM#efHsc?%m`cdt}cjHmmUskLW`}y(l==MsE)Q7|ju} zrq(5}ni4=}j`GalS<90MB6NTfEyoXiIYZ@a8&Ob0nv#A*?k258X+2S!W#^=rjh>qw z>StyrRUy2BVW+5O4avK~p zY+Qd>45@ZSewIT{PaEAc-gW4gq@!#+MGh`+4PP$=#n2>|>_5+{ef!pxLtqv*7g8*4 zF#5*>YC=D_IDM-bIzD~NmVSkHY*T)#r2Wky{SwL!mKf6YI}%~?dLWVnSOlLsRcR){>rVFP}u&N{OEC^m`VUnh7N)z zFOXpfg>qyh>&`&(@byMr`D&tnKLbgwWVNRBy#4KoMF)3aPrI1Zt%j_1N@|K zI9VRlQp*{t?sVz)XZe4{HK98fEmG;qk((LvxG)g@vSVnIkqfyyGDq@GBlu(q9JHSJ ztuy*OGtVFUFju}6`#gsw09?0`JgJB6`AffbBech;I@r+b)CPgi_-#eFuLTJhjvb`h znOon`GyOXXwB2sXnbK-~J93a!<@o5=AJRSkwp}UqQMrp(8YFZ8Dzk%yKPu`}J3PY1 zHZ^;eX@7;?m1Vy6WeRA^tn0Y&|39MMGOEok>ej~H-Jvb+5ZqmZyE_4jQ{3H(6c19| z3dOa!yF+oe;_i0xyzh6;H%9*CM+S`Seediw*PPdyGZsm1H@jNuCM7rt+MVeY07DoN zDJ6WkTev1?D~*c1;QuXbW)B>*Z7oG&Oyw#raUgkENbf8(8N}H5s-IpFQ&XYwtCT){M}-d;B1oEeJ~LFl{}MWnt+xxaOSB70f#zuCnA`-E|{$+gWk zNP?CIoiqi_D=!ExZ8{9f)HFYS_%`DA42>cW^}6qeq}-oE^>Uxr3#g7bsNPk_F1L-C zb*RMca$j-ll09D70dWVIFlWJ~Df-szG7)k}w$LUaVEx{3>yLYEQ7Uo`mfixyBVhYV$yyMtlk_Dc^rvdU&dYh{t7B}9>8z}fF{E@1ASil zAb1O?(5pFEP|1n@-ajqa$ZX=*?^%%M%qs*MATQ?YO#3v3SR5VBg-I?DHiVW;)}8E} z!aacy2G3JFh`gh4b(~%Y{l7Ymd=><2SrXkuA_yVe@K6#1e$>;l{54ZR{cPOzWzktl z%?EpJD-iL!M_8tvDcbyB>fBEMwYoEfiZN!pNro<%h>6?IrP;F0?*o41vd^0#oAzvd z;LqJzB<%Y-?50WA@sMlEmkulK5Piphe&AmGXuhlL3?HBuC3}Pu>>O87;Rvkmw8Wfj z_{V#49G$Q=FKNz)n{`;3JF@%?lPy04s`In9qBE<|HJnGzfUAMy#5ywAO&RJ<`r0go zAD`QbL6Mpue7|(HJLG%W1*PU&-TBPu=sf-J$-v_%=x>cZEtQM)h*ct(th-QmxsFFn zDM%o1KkC*WDcZ10ql1xtF3@cy=`NZnJ=`(7jYOBj{B=*bP=*GLGRgcqHNQOZF}|p| z?(IJgLH`n`oLZZ=kRU6jFEusPE6^0HcvUos%<^s!4l8!z_Y+_|rL_WVqYy)b#5}z= z?{cy~97R{wXVeY92a7d~8uh975lB{74Bcy-bu%df9Bx$iCN@btkRFtsO2)ei&t2Z; zrsa1qW&c<2yX{`m03(xELANdp$lN(gJyQ-ltQre2H#-~hP!|UHzq+SO_DFL9M1eTD z1nQ8Bq<#u8MDrw;uiCb2dWQjCHgCL?zIhv30*?ohQAC@6>(rVnqHMV+do#rXCLUl- zDvu=pcfp)`z*Lbe-(dEwjAj_wY@-#bPmJnu-rk?^`E;e1X?6s@!snJJ^JZ57xFVzo zI2;>d03KW88QHs#4jp!X?9?qV+qPq~GKnvtwBiCVdh zap0T#BjA3R68ikA4^uP@Bk<*NOsi*EoTC_WeSsh#Mq^m5XPu*hM{kXwu16HBOOEu# zMy0JN)_!QhJ~oH|<4CVCQ&={$F??`^&I*G1rB8MUAvx{_o`9?4bw!8?sII^E<=!~9&u)ZC4!N`DBHX$f103!*sbiN=Hs zLFhmSQS5Ai#s1MxR0OHij#_Nn$_W{?Kv)if7aAGc5GUS{#=$`#^zHVdNnl(>*K?$& zj*|uyMtJE$=wI56U(?!oq64Ga-?l2g?~ILP(%J>%EJ*e~ z^|M@OJHiUh6$I@b{q&8{FTT2Y{0Kd14o1}hkw{>!pql0h9YIf0~15{H4ADC z`YZnY4p_#I7IBw<99au`%Vo6cJ8a5>eTH3A;2uPYiZVb&Mt1qwtmrXqx#)Zm)5fF# zh?>e$O#5mMLO+s*P7Iz|rZ=PB9{Mab@+C7nhqy$zT~e9N!$v&T2<)O*1txd?~*x_oF-o1UZadC)diq&tqi zK4I$gE@&u7IA|0GC#xy&8icP*7%&adp>|2r@%|{XtU~6t(x$V%K{(WSqqHi78&byeg2m0fqw!-8Nwv zL*orPc~4Vr4?pAt3dsefWg>^cSKKM0Unis`RFp+dS~bT8_c2*aRpGO{v)y`og|nh2 zb|?CZ1__x#lki53a6Zl!78%}B-a zk%_kDrU2Rx6G~DEt}6-IDB7PKE%%U{Ki(W32*S<5G8q^g?5!qI3E2~1`u~K}_+8*+ zOfO`DCX=V^F(vb8X_5c>NTqpgSsov;Px@BA5PpnUWnxKKGS#C;JB?g4l3i6mFPnh|*I zRV6`}Qu7xP<#Gjt=zCEp7aAz|@fA!|q~f#(#55h=>X)xAs@ ze7L(P2~c|LL;QZU%hT9FnS8j>px9Ul4I3;;FpjqcWX6CM!QJe2CrByz|5vwaA7BgN z8DM&wqyOX@R9mM!g!{`A2QUZ*;JU^Oz0cp=>s)E!7^oK*n*2dfWy|ux$fcqcJOUk#WgRK>9lXZ_k{sHhrUDhD?oJP~{P19lKgjHiY$F z=1?>7tkPWwD^(n5fY51XX?Pw@mN}kyC9F1FkNhQc$m|_AV7n%|q45}%W z*;0ETzl|ff1!BLfB3tFZJg_A@mAAdIHqh%ZUdSUQ`OGGWRf^&MV`C~u7IMk5TN}MrA`Eq&CX&!E^n^r_X8MO&+1NIl97vrHAPg`i1|<^{6Tk3QVY)Zy zhDNA*o*X+nJKJO+`_%f1e$D3#hT~Gmd)wpy#{W)GH;TzqT1>1SFf$pXO#ib_$CB~O zdcrVdVv9G+*qO)J`|&9~z$q7=GS8{tqLaI!%fRJekDnA`^HGP4G2@%lMmwr}SFv2= z7FQN0x_bQGO7`3_=JeOTGHvDp(N!zfrt_Zf)Bn3~NCsvMp!Lqia&8f)Q*0rc3l!Nw+sxKzloAGhoQc`8Q-9 zI-2M4ZB}m*fwA%YuPWY;=d_Oi(SV~8u9|Yld|PNFEN3Kr$^ zCM>-xD}@-=;KMK|H<&wUdDoCH4GoYKdmf2K*Q0C6J7ZO>2NWT)d65fks|}r#Y?N=R zq5Mg*xYcb3k{>D*=J_`bDd@a|kOhvwf}lK9UE%_^7; zv}(i~FVcUTbalFD1|xq_&P^XKIU}hN2KLKuCG-ZA^_#} zJtVr3U9K=l<&Od%P|x17&71WzkXG5{-@@Xi2O6lUl^8UskCU1Bg8OQH1o_lcRlSFW z6XJ{L%r3+%$%BdImUR%s%gv`tqM9RZN{veUjN=<|8L%N!#mykDw6-zl{wy85 z{o}{KtR9W?*9qa4HyvYd8WE8VJtx5Th4;5|Lb$qj2qfu)WjPVe5|tJ}J&DC48sZIi z%ea7S1%K^)Q8Iv^O zB~4PQ48-n>`9-w7TV$it5t&Bf@{o4rsS$-lOY~a`H{ocM+Om zz@=$@mm6@3R7+h^uzjW9X+zWbk>yY5zT5THR$FB6x)Du_~F@OI; z`~$@L&aNFD%dQ>EM|zK2J3!ezz!ilR2;{_7Cr*-gX?wC={dpOf%6;4WV_nuO)x$!7DIG`Nob`zeJ_REX}w#=};nrwB9wj3-Jj zI!S#C>m|{SgBZ>zW1Zc1Xo-WpL&^NG$h{q4q8C?9QHp)^FBXGfgIui??(#w=DW^0t z>Lq+$m{B}c0M~C8_I&8(BVu~DhadLKl<%=RA08*ao3R^^bsB;;wJ{SQmb2-}y_~m6 zH&MgR9#YKn#K6fPTc1C?Vn*PG!%#TLhb>cfr=xV)=48BtzOiVpMYerTTQqId^_BJUBBLLB&+;NHNch z@(fE)6dm){osl}970{}yWsdN4JF<3b^are%gQN3is+}z6a%cM-arTvUi2*Y^U-I0T zGu1kyV7r4-$Ui@>ZXSPa5P%QM;@H-qO^26^47sLC?5m))=J zf1bpfQ-WZod*TQz7)$g6Sl0gd(f%e;X6nRa5`Ji!Lpia&toKXZyWc<8KqPl;JGYdJ63ZQq7`I9JGX6X|{fzK79JzAJnirx?FkCPB&wTHVqGBJLT zwd#-Xq@!dNk&(~i-il3yddyfslp@$#$ZS%&d{N0pdEb?7|0xFpp|2^Grd8i#LUAlP zm#KGvNtccA($O~z6`tYdeG%xF&u$RhJZyIK7ryg@1B7ebjZkNT;Bo?0b%2u#h89s* zrN8TMPj1+2)mQ-NDKip4;$3!Le@ZrC;0-)0pe(R6lpCMKp%sdAR|x*Mc8W|_hO4J@ ztQ`F-;08;N;!MOXqGD7bTZq)Rcw3>+{s<%K5DkB@ZOn z0j@L0q)+Yb5EEc54?A2dYc+n4{O(Gr@B|Ix8a2lMafG14RC=kL-?z3|#~`ob_hvh- zf$W_^NfiPZ%4y32pQ~to46XXV0)zSHEK6h*xo@r6=tg{$hZcoi|KY~hF$Uk>N3#An zI@a78AN`cKA1c1PJ1*?K`%x4BtqD8?{xMr+Yxw9aC6Y`X0{G-%Z2VJ2mGiup?ueOC z=bssmb$a(Fdsyt^PqXqf`x%Dx({@|@2)gi5@Z&_&V;%`Do`=;h?+F3o&e%8`3UI66 zJUk-Gw~yvZEi5d8{qm4}4#tV)TD~34ioPAGJpN1`-8#o=+UuhHV=qk9`rt*SBQ{Z(@0!r<=iK^liCggX<2GUr5@Z?Z^qdlYSj%$ij3%OFmxH^88%fX1 zBq1h7mgVo(qx8G%nF_eC7d)R934b{cyiD(U8`@bw^q^Goc#sz*OZ25~d9?g?cT_YO zHGTsHmLJf8;xzNjoHzs4Upt|e8EoQk)E^99#1A+i<sjMpLlHO)?MqCX@c56{LI1`irN#Oqt>yp6_s z`ws6ZvXAeiombS`%I;=61Z*_^ol?rt?{S7M@cIUPd0X!YqVk#%mertV-pfVBRc#>4 z{uLojUnTugN?gLNFKV%N#x9g)`h>2`Wgl58V)7zkiIwCcJKO+&YN|eyx+E0CDw4D0 z6@v0x&udx*Wj@YPd?=vq?{0PdLC59J81+&K817Jh)91h!g3SKyt>^*SrX4SGocerT zoTLh^o!BT|Z^+3Fp(5dwMusV8y200UBa!Uy&Cix})YMy9-!XG)op=m(K_m%Im-R?R zkg*~Hn_Dbqakk$952wqp*+=gEbnL|9I@ca>Y1r`ktAUsi#B$3{5GPHA=D_Mta|KzNhP#m?5c`g4Jzx(DkdOSDfhD5-7PIqv4uhMvtzQe3e~X1RHuky*weIm zs8HiGTS<}u{D{id(9SSRr?<*$KJ0eO2p10rPLB`1V;@(!@LT)u?-vQ6j<#FKRfR?G z1bSb09vWrEm)YSX8rZF<>M+zdHoXGgqn*x(VoOq8O85$micMunUbXx8&|27#u?Pb( zR{i65ZBzV@SHlKmLv$>;LZ#F|`?K0TWr;cmcu(52$;1SEZDXJ?lPaEbA5e%0|Qn^ z*Z8sbMa=w(jcbx6_D7AAbcvss*^57a{WSTynL(R5Bf9N|BbX-FmXovndHc;8Z8&-| z=E73oMiX9OCcOrAU`Pbq*6BZNNr#V~3&Ya-#6~mOPy*Qlw6X(WJ`6aRwlCFGqb7mh zD^1v&W|SnoOMm%zMFZ~UbE?wyl;p5NgRi&)ecpt#7$!a#U~@{L_)j0?i=9g<>Dui0 zBL7WX-+wCIY(SXoQTVVwN2Q=iazuyr))>jnQ`@zz8w__P*&fmDNjq{0m?U zD=I6f=fc?<|FzxD&<~b!^7um_qRdyh!7P$WlnsTDD8#~YrYpwfXlatcxQf>~J5~e( zFG^3oGS0|XL#)I?4rFa5$l63y#e?DHiH@VQ)oIx184%0DW4)F~TvEZ65y@78>Z&{+-4!Pds!Q*e@L=H9$tLH z;4@z5Ia0^T{z--Vk-*phx&G+(K8a%_jk-fCqJri@ zcbfk3{`vm-c?DNwQ$%5o(u*0+(qW9vnmc_$(^-ZP$!96gc(sK!h+g`&dT6aZ-KoK^ zUF8vO2PmGPzNA!(iK_B>UHn`g*%%rkTcy0wkl;^ zs0c>BR^jW2Lt!lZLgX6)A#{jRq7 zh=7z%SgKO3*1w<8oZ(g3u~BlL2!qTdXwJmTkGrK(P3(VT5=~C4CuT#*<;@3qV<7J3 z7d63Jb&Eo!mBHF0tBCTU9?B@T zB(l&P%=wKQp7{&obgkHFX>z87xVvik*}m8 zYKY#AU$lM0XZvJ~W3a)2Q)*Vy_8Q2u4V3w~PXlur3(X!XTL_hkB_sWwi~Bb&Q^6ni z%WM1^^f=n$oPgw&U@2(_>7^Y2wcqGCWHJ39Iffvw2M;Qm~CUj z>|=|D)6-@SaD-oT(v#W_ZiL4me%!ASt!97Z9$+O z38`^6zbbUn+VG~mNnY}ZZN!H-G`aMt?>f5LZBW?Zy|oX6RM{8Wp zg2$IlsrPIKbj}yHPywN#ui&JW$IqqDGBQu53NtiVA^{w!m(><`_8TKQ4|6z96ThOf9LaKG8*38+DYfQ9#~z8gF(5;ti#)Z2gY{z~~`k}9_1L523WyuE5 zf-TkqQ`)mIo(^+Obq=6Q)_@K!)}z!4K6pVqxF97hC@eyQix6=fcF#?yolKyX9=jg9 zjZgWjD^Iq(W5f+9R>q}2ioY@yQ$w0Q4>d!iTP7(n*S2+c>(R(yO7A{z+#8=jL7Ij1 zx(iG+OJbY`UF_f`o)K`lCmyPgZf?yw`_$ah1LSqU-%DCJ2Q_&V&jRvkJoVBgUab|& zfWErxDgRBy?(#WLkR+df{3HowbRUy~%>QnRIZ3qm8&R#Zw)bH!GR7DLP@syN2!^1G zM2AliMx%ScmMLFMkvGboBO+QFLhw&Kp;dZMen76?iO8^%Mr|Gr8-YGY+wFTHv`Zd1 z5A}aL%}I8S=M^cBb%OyPi`5H0+>(B_1&sk9?m?&!D1^z(&wO-*_b(SrYM%F)Xtx#n zQg3O)Yo!Se)?PHUc0UXn4Cl(z*nawXXx+KodcajU z;hBGoOBrXzz|v@^ca~`L-*_>um+7(w9`BCGFf#?0i|ByoJ_MLh^KYIRYYdLYO0Zkt z^A_F`$%t^HeF9H#IT5c+bR*vaDFT7$=#`U#Q=Y&J?Xy7gu&`^R%zLc zAuUCJtgQ1ix!J#W9;GP4n@B(;$2A+l)4O6gUmi@0*l{<`yjp4zNFcTAK)_(&dJByhPpvi zA-HPBaFd~f5nP=$#!Sm~^&Q-BEZ!hhe)L9(>v|f?76J2a)kD(>L9c9--h^n7B%@&e zbcCY9Z8nWUzI#@Z+f)XA)CUBe5`$dRFXLRSDf|=sc%we7b>d9TV5lg6)suR_FMa&# zpPLH!kn*zqcHi0`B7)|QMzl5%oL4@Vl`|NQ*$es^4-8HH@@Yw46*0-VU-=Qp+P+7N zgrNfX-YfahIrx{xL`F)E^L4DbH+0g(tlZa8-vupooHy+vcjx{%iTZ_fY?a#pJEY%Vk zKU%Wc^YuWR(t^#($pTdb+d$5I-fGj4d6)DD@H|z{Hmw!r zNaJL9c2R^t%v-+HlUAe-BHwl@i1wHJ-An3B)%mQM@OxU*6xSffTOC;6NdBnC%qeVg zn$^<|?Jn$&9S6RtQF9rs*(GO)^QSe4p^{6s5iRNHd6Q98?5_-U{kfWc6ksIOa~Hwo zEWyJqb;~uPd2LE{6M^ub4kN$H_pZY4F0{B;cLdE+sZnuJVrr44J75d4cnbyX<%D_{`@>u2B>Glxcrm|7i|(JP1QgG4KQ_l}cHa zM&}xn*A(Fd$d_B0C~38??kskF^uDH>=)bgWo<81;OFF&YEoD{bK2C|gZD#I2{AIQ$ z^S`+Zt^RO-$qnp!ZZw&CSys`0e|Q57?e^lNbc2tq7K;`N;rPXhlXE=ko1G9MPZq%$ z@4Q=ptIB0Cy1YjSU|Fi&r`I*QacUgah8MNHSgR1cABya~j!o>)?X($i|50MC74p_o zbF1TezQYp0fdXZbc)!dwIi9PYy(}DuN?G}zBy-iG2g>q#_9);Tv`^^6GFDr_a*Yd=yNljI5RjD^NX^fFE0_!V^rMmdYa+w% z-Qc1m=v&6)(z(CCrPQ=#Q4LtTO!NsOI!=Ckzq~~9dx$F4W1Zvz)8c@5n(mnSsd6sT z*aH=pW>c$>zQf8Lou6Yoh|)JZtTves#8`ck2*N$x+4>X8F(!{X0{du@Y0d!+?ipPI$f*_DUo#K-1tgNq8%CTHw5^ycf(t;5RtQO zofI}N6F3t)#)&m?=@Q(J;wCmy;}NoBfI$!2h*yH?dUx9vmYLX1w+zzdd9BUvr!|#k3f&>FnJ2(t??aT0q@t=0A`bexh!rY zSs4yZqa3vpxim{ja5gEU0?KZLQU$k92kbtt;5Qpm{Yx~stLFl4rS+m@w# zKxPU~U`hnrUNaFLn|8SnB9^92Wmo+t)gGSaPH_S5=uq9H+zC{1p^1c5)K=6}?pLPhQ_VtD6vcYlwt6 zIQ8@>H>;C?DJkDRd)xrz_jCs+4Olo_Nm^%jG3;)vOwGMTX5c3m?6r=OECKk!;nRER z$qOU>vDl`01Q_fk47Z)`7&Xi?O|h}V8GF^$)kT#q%LouxTLGyjsYRWqb1bf0!70zKHRqj@wdfX7e@207SV>SI40BGbH?(-lAvL1QH z5#u7-Qam71D?j3S!yG61%N8)y7B_t=Y{Q;HSbW@~BwEVl2kk!*kS98)Z_IXU8U-C~ zCbz|M>X(tI0`Mw6mdP%0#?3-x7mpgNt~Eb@+n;TndH>CL^3|r_p5Tv-ZC;ZvVjLFb zZqcjS`)X5ZiGV|#U7vcuODsxNN}BV{f;vCL3o3RW^{%0Wp_{Ym1a`vCSr%s+^R$G< z!|vrpl2VjXq3`?&ukxlkw>-bg8|4X=-$9E{y>$>TnQX9R1XG@r9-F<7f1hI4)2&-_ zEo1&m^uG(OQJnS4x^X|y#_=5TXw>cB`J@0eoSFx&3*j7}dX3i0?F_$?CaDMp^4C!F zC>X}A90@!ro2ftKaYITDsFi8-gM)o?WR<-WuPx!?3WmeCV3D8OUn6~4SVi{ub=Xug zUWJIsyg>aY0_|%)Eu-x(W!oSK@J4Y4CsXcon1I z^Kwp=J^1w{e7%G4w>Q3EBVK{Lu1zc@m2EB+S;37ESBZP%?qIF^Y4RUbbr+Y#KClED zxwmsk+6wtYh|IgKK*w!>O}3?f2xS;M!jhwNN$R`xZ=V#1q7K0DjdS!C!=Dhbqtgj8 z(WuIV+Z~4N2_QrE@Q$A-@7)h6~Hau`IVF;C0jXpRx{Pz%eUZJv`H)hBH#nL zxt*G1JMEYo+L^YN|*zY2KSLZm`q06Q28oa=*6 zc8QFMhSsmz1492oxlzO(rmM-{>PH*Fw*$Jrz5Q?Ar=&J*@)Lp-^BUP>Q|v`x8d-L| z4Z1V!H0>i$*SgB2i-HO_AZW|?9x~HqRj8Mvat?)b8y(5%%ghIM6en_J=C++cW-&e_ zc)4#WbD^oCp-##*8{J=XI=>S2r)=mQ)a^X!e4jd6gcCgiq$%dH34_-i=9;87md_{5 zqLvZ`gX;eh;N#^&`fq9I8$mD-db`)R&O;e-!yA8chr!9z2n(4W+ z4{VXyPyFmL0C~1|Z@}p{b|chDU`JP15q~V2)RT-^Q=eagM%Co!t*2EZ_ctlIhkyML za3>mY_}HdRjnDmM?)!3IeK?&DO1qx$P0cz_G7lii;B2L0IQ$zmA%~S(PqBolitigGE|1=r|J*a*p6ZYC!@yw$MP~1w4<=spTGgEMjU87 zC5u*urvH8rFird+2^B!?rP)V7kZu`r8Pq*~^8M-QX<1oWclVq@f~u-pU$m>LE;uYE zY@;NA7O~Nfq!eS*rPHky+??}k%DvKvTSGY;t>E=wHCTyu-m1s)T$CUJZX+uQmiTEt^A zfQDvqLQ8`98UXC4DPA-$I+UEtOo+t0xc#0ZcBTix&>d8&5E44AYI=JGzA@ycB~mHb zf0I>E_muI0@PJI>@mhJA>IEfdf25Q}M~C=0yhZQENS;d@@LIDP6ggOS;*N&(L61nM zbB>j#5r-3}Gy@zdyfoO2pZM{91L&Q^gg=!l z<#BQ7Q}{e`w1t4XUd9Y$LoDApw1{S@jyc-vD`d)P@O`|147m*Um%u?t2POB&8M5Zt z(P}X*co_|_b|IxG$jJQO*Lxn-^uV!TRzD2^PNb9JgmkYf%ups_^cjAc?}v+$L<4W$Yu zG6|J!Ku!>5$1Xk`k8N|y`{m3th%nPRyL_dq&2@DU!iJ?mJtPr)#7+Vb_){A+R@x06 ztOyI%knWz7X04_&b|o1hGfhlAUVh?5Jpw7dYz9p=sJ#5Q^@#d8F)k5)PW>X2XE;xo zL?~W%W^TEhLpO%_?%EuF#?UeQzbs!+-j9jk8xd#z4f&L#EfBQL@|(h9Ki52)-{^me z1lj;=*hQ&{_d9F7?r1hTa09M^`QuLrV^Gxncs{sLoT z?;2Uzg^}KjNU#9}yTDShJ}f*=Tneipc_6inZkipETdCIIHx#@Lq(W&l@&g;xS`^&n zSxZ86tX&li2w!68Ct9}cdLf>d25t~QX>@#3EyAy5a#GKdYhX`~&y7q}jI6~x3i%|X zF1~&&_o)_(5R_S!pvQ-lds`Wp;I*VWRa4+J`J3Z!6$Xkw zu{!TR3xAX>{7N=o#3 zQpB40Q28F^uiZuNK2Y9UyOczh(h(^CC1*Q7I+OfzU@}~*JOV^OLjMjc>+ggu=66?7 z%vw47k9kuB)zOFfN1>w~PM@57dz68-ypn7InT&3Q4a1fo ztxJhCwxGixI89h#%+W5y`QU@P3IN;&;*whxc?y&LHgc!|TImDLGi98`Z1*L8uE%9IGYXlxGq#&j|$LCND4PV^-w3T<$Mjjj{qlzT8}Kb?4!<# z1&gld8bO9-^b`beHAb63+gGM=0LRSmY!RLb0uAabfHm+dg750=f5l*YDent7QC!+( zYr0V;D0yeB+ZCdosmN%P34!@`~i@B^t zLI3Y`@GoHynKL^D{D`VoMblY>aV#oof%Br`_=K%`a|>}*DE}1rcXoITmYSx`?rDkS zeOCOl8(Xb0!}+U|JsDuL263d-oVOG^xYh><`stpJv33=iGb`{o3ms*zg7t;0LfD&q zqjR1TMg3Z>+EX zp0N?L-$`Ynrw;;U&d<+J|J$1o;osx&@QUzZS9PlZabZ}eav668NT+65{_ z4M7Z~M{V-<96UmK+H&hg)~@3?k}u5dRz`c0B8%es*D>1CN_*qmLlFxxQm!tZZ;xP! z8Y~w@z?@dWfSbh|V>rQ1&FdbEvhk?$I(+0ZlpdhL9+bPL80*!J_NCnUokl2cC*!j! zVbIvJux9xY<=n|-K%4@_@c`D)_lUz9flPFki9fC6=NdlzM?$}jKNUm6U)^O5_cKVh z{hACOf$Sh5$POBX9Lzv|UXLrKy^Tf%2_d!hg4o8sO|`-M4rloOszxcM7CE0v6_#{4 zEsM{8v6DNt$FEKSaW%QN!n%s}SRKnk=5qt!CW(&3zme?_#La$7dr>Eo(}R3>{bAORfF@xQ`h*#(h3Ja(49cf`{-6P`o>Q=oiivTucA0LyjU_Ej!_-hZE<6=!d@uK>GmCIMG$FmIlSBoo8wmzefSD zS0JooGL-Tf{NpDiOjx1uCHb1%M?Zd~Wv}6w!8w+>VJTl(36|8IcO9V!GXbkC;T&ihyw!yJ(%9kmX zeSu&QAHT*9XI+vn0PXQ;>~(WwS^6YreL{$$DbHzQp752PrH$bR^|L^KSsxf>@^~1? zHQ6Mr!S=}LBn!U%vQUbY-P6hsA|#w#Z;dU7G<_Gk3A)n3DXcTYcC@#_glM{?huS}A~Aq>ERm<4iMUN6vY>(}KZ` zz^V0w$x{W=H(jr7b(z9&)aEFTsaM?Z0Jf&rkBuxC@Tl^+@j4&0?iB5fu>0d+2i(@HI4I6w?onx2)5{cC z47wLcNSXM_(IzxSb}!&@`%jIMlCqLHEG%qiW_LI3hq%8x2V}IkUIE7#4dBU7Ny+jR zm~4~m7p}*>QV*KiF}Hl&F&E3?xEk?Qw)$bPeoE}SH0PTGBJnVoy2o-7lM%%s;r&FF zTv$0Qm3xS4ik7`~dVtTUVHe@x)xkY8S@Kz(#9WCby$rKwO|m#!EVVDcVRKqF6>CB| zF|rCF&ZVQ{o&zn*c!;k}QOkdAGJ6}(CZbmhPT=bssk{DdbeO(w^K(H~ve*%ke4@z_m(n9q^?B)RQ);pa9h$3?T;YICPVqeP=}K9KJk`}V(mbSCDG zu}>b^rvUP2Sqo@f8ylc%xkqbOeOmf}b5Dsm$1t%7UkT94t4O}maJx^!h}D$^dPD-; z>*+Q4Zt6ZYam2zBkN2f0Ojdm&m^Gsl0S*YKLowP;fVyp$Q+d_*VWZ&*`{q`ssLSKw z@O|L+#K^BkoB}`+&ejuY=y!rVH-}2V*75*B4adXNV@7-Jma6=xaSLp4=HAZO5x%6J z7VqtI;;QJ{_)%Z<@dpuN^jC8kCxZE0=dC!0n()hx6#bB`3c_BR-=(<6Sa^q9Bj0&9 z4SZh6J%Pl-e%D!3mFq5}FAZq#6%y*1pEbum)k7 z2ta$oq z37Q+cq^+_VL87}%a(V^AL;ebwdI>obmOnZ{WP|Gw(ay<kETGcwS;RTBfYukKXcd9;UWsLj-%JuVk8%*q(58Y z3%IhEvq(S6Ar+BF4~S$5L~;XuZLxVR|0QC~IpL7$6q)&B(NFnC)07)==fQ4{udOqxsSd1r)$a<^`LgaFngaku6i3Df|Nr zdNco(08dCMl-jtqa56L0$pKGwLs~6=dnV_}{ zhM9`Ee(C3Se3-~>KlP|%XKovlqEz1gA19%Thz1e@d(8s1WAG(~*)5FGmP`;&^fu`~ zB{X*^W%@Z8DJiH#v0?(A?BL*F=x?7%gTI-3qQ@g5uvxAKjF0A=7TP^z}fI44b z4IpqY0m9;wW0J*%BCgSj3|D;6;nphTeH+4Rs!t_yG}`7@TFkUa)$-kwDkIv8ebsidf%_a%sMfMy6D zc5M+=tjRH1kd+aVmoeOz=aVW-^@E0XlZ+pFt(@}83h!87#WwRSeUr;G(TzE54&ryF$te%#WL94 z`*C2S6Xo%&Sd8j33`OSt4^`h7UH9|t9iy?6#*NL!Hco6?jg!V!qsBZj8ndx&+qP{! z=l93G_gUYy@+#-eo|!$f_XnH2J>2zL=~qlPsBZ*ens0%jH$9|k8 zR59ZakO%$!dV&kFfQIs45lwRvm<7Cz4ZV$W%Rsnj>i%Jfx;X?`#V-5d&295C9_eZa zJk%ogOL;!u2M0zvAOusR6%}7$^*hj=T+lsRE>B$YeTtqsm~)F5RO%_UPlP0%!N{-; z-m`KyO%(jxSN>{21euVm3guVZJC(k_k>b1QbWLtRc={>`lHyu8p*4AIjki0>EVjI` zQ8S3Hf8B7)=|yz0exm#3c=>^GOxO6-cC|A}iEZS&f8q7{^!VuNK|#O;3}Es2R`ZR# z0-D`id7DCvrJI>VtW4;+8AU^ZGs&&n_zk?UF}6FAd#@AY?-i+-uffL?5Cj;WZf{SQ zXTzBcE2os#YIX2L{rt8@DaD|u)1-oEm8THN8Aqb9724Lumxz1v$CT6QxR2jX7Ja6$ zz@_O}mO_v_a1hooyhwTpG@g>rSct91XlFWs#c=(}$ZZqkdko59XOh4+7{&aMqHNG4 zwaQ9Hsje2d9_D@@3E92ZKv}v|ZDHS?aZJa;R1mR{C4Bfx`|{A6Z|^Y~FUI$b_%Ge! zg^rX4K5XYwR<@#7s;M1dkYE9lfDOdm+@)UXV&bFfTElgxJtHX82 zP=w`I6B^&!q3iN8IzIpSDBri_%ACu{dddf93$_mEr!JQ`#(!S(#kMOn0|{I=1!gOs06u(EJ&8~-b+{%l zjPfL&@wmgmbEWwhT#OK0EDbDPDUri*7}XS6`GNA^e|{q}gYO})NuX7?@pYEk11 zT{%q|CWmXg6Zu=`8zjz4-4egs1%ai)oUoZ=AjQA7?Y9Z`aJFRwR?n|uB{bNWN7ul* z?0BXy+dp=TB)9WuP8m!wi^x$Nf`fhh?GTo?KfLvsw$_e=JwH>^Oh_plTAj8`53)>X z?1=s%sGmLeZf=AKoUcEwaSBZ6HstK6Saj%ic$3l~v2lmRuWNQxTMm2k3)C8OmE8;k z-{*!L)_7A=WAd5REKFpM19&IAAJ^TQPhWzed&G1@*{t+bgoFMZAQX~+=1CxEj*mse z)y;JO36!l)KuoAJD$WQizWOq@>Gus%W)a^t^W(Mi(C_y6kp5l*0peDYsr?JXa%L82 z9v1&kTKrQ&UzYCy*DkmTEHjmfJ9NkUy-hrJ!k3Itqk@S%ajHQeevcTFB<% z@o~4pADKZbYQU#Q1J#D>nkn^X1eeM`=^3=IgD1)F&>X@JfrtQbvKuDZQ0t+MQo z>J?o8;m|`kiX5|a-zq<8&~UpRv+(}fZdUba(T#7<&@-^=tl`(>9yz(A0FDruaX5lA zPN(&U;TXwQ`p^Tw?aQdi6}knGg3h3!As#7j>mI3EW$rL2%;@@vNnYliAor%^Z8df* zR4oOWB@iH*^F*wg9Qjp`l-yW5KKrn6>vhwN7h$M%pzUV`Ja2KZNOg&}cnnl*`oNIw zC2fA*K7YQ_jL}Ega)Hx#e$3Shz8shN`Sw~}pFpmn44&P{uetFd1&x!DTv{^!G{YiU z0yi^p_R3jdplOX(N)zW!Er6-Oy3x6D0#1mmY@0{wOVI5-O6T2~Ly{?rwH^QoF~C;;5p+){RKf3mf7KrSUZsX}1>(QE&$?S^*^uH%JAK!ACsVe3+1U z-H4?(e8{beH1g6V-x`9Hy0WQKDq!g8pPqu^q2(zM1m}|fbuDl=r_$UCeE4sTg^ksQ zhrDm;u~`Wu&nwa;o%GdD_SKsAHpQ7XI?8P|$8C+-cuFlnw8eox#LkX} zD$Bt5gXKH7R5NcA>v*x2g6vDfAAusjG29HM(o6iKyE2CSgU~hFW7WY zB#gTuMzML@cLHP|i4;I5XfJ!0Q1>Uj>IF_2lxzQ$=q_%QCU|ZZuW%~Is9*^Kr%Kz^ zb{J^7@LD2+ZZej4$uOwX&l&KhGGg&{3(#Y`gG4To{{FMa3!*?P7Wfnv`Cx9(!{ z{h^qtV!J>(y~`XCMa{14(0AwD{i6%A0MGM)ij84nr6FdM4KHrz&Ef1<`RnOODH)CKfXhN5J*LjCg_~Xtc|Ut24X^PH>X?y+9ym=1rl)4 z94$Cp%Qz7-q~FGkzjsS)MNz;b;(DqXtBMHA{V4W_8HDUn1Q$RItGW%RF1o9n#Pr|h zgBv?0>t5*-AcjAZv6a3}#%#DtFn)(Hf#3a-kI;JtP{v4$NOtWXWrP?s-iGvC*VOHD z9&}m$`(qLm$c8INKF|`X8fS;cprpX{gbqzS14!KqLmCD}hwYZ4ZOgZm#GeWEKKwdZ z2)Sm#Y|-uOx{|>%sE2vO7bW_D|4u2C+uoE z1;56OicdWNgD2~t<(FnYg|*t<3kOAJuD#E->=#legMAJ_A_%zb$|e@2u5^j?xgID> z2YP_NEzCP^(t3cS2;>CJZ`7j>^f74$Kn4xidJ2R0Gu#V=l9~vv=R8hZ*CuIk>33>I6@J^qb`&k9Gaky`V9J z9l!83aV0PmH*p3alQXiV`uzhdZG$Mzh3L%8#@E1DSa9LHvMlW1fjD6<00}^yL*gtv z$!~4f`}`DzFVkkNyH~>^Z%+_)T8tY%QqJgoe5J^zkRi33&v%N4%=QN*MeDSw@DFnt zuWS9`-yArnBHx?}n_08GC=%T^ljCd~ zNakLH%4R%C?ZL2c0_4lX0T*KQcM44V20y;**(i6}|7bElv z(zL`jizS(PKoI7$9?tqY)0*X?rdm$PcX!Zsy(MUU*&^a!1!8KSDcQu{$vA|Up(pJ3 zhPt{s7fB~|LIzTdYe&cYSbEbh)(8u%&J1hNX7Q5XU|(DdHGan;Vn@zYCmAR(5`|5+ zqGqnU@cf+ysaTSvjwE`1Vx-<~4WB_$)pj{UaDMAOMqoYf;z+XTwLpx{hbYaxA?+k^ zC*Vid^`@_5C+QHNZpKR`FFE>x3bzVO9=yv&{MY|BscAQ#@38L z+y%sq4MQTr1;S)_kazef;|L6K8KqDa{|&(UA_I#r(HXP`AffmWhd~cvfDh1me@M0=y`S z;V5xu7|31_wx+#_1_y@M(@gh`pmYN$7%Y5ITy?E#p zkh^#cF)4%xM$URgm0m&?@cfi=Xho}%?3L@3;m+E!B3Q{tUjO|YhikCXKne23(9qn|lSh|74t!Wy;-A`wkhp)w zV2(~hZ7svAkiqJ88Sd~_`0{7ibK%fo(B`5TgGSN0J#AR2hI%|SbWrZMoB!rdCZ-8; z$F9<{GM4kvkr5t8N*IjpceE(2Ildc&L@iUs%5-DjT!wn;xbMrxZqIC9EyBFsgCTX3 z&KFHLyKqZbUmTJ}EpC2@Z11P8Wr4`dB*vC4l;jl*)M`jqc&SUcu~S>bD2w$ja6#S4 zT)3%I{$HlrpsNu_x$xs0kynyfn%lc7r-~C*e=EXxsb$N>gU;fip($Hd;pB1Z*_jQW z0iPKwM$O?9Fc_p~4%q!h?`^{)@hl`Cj{=XUYGZ)+Gi}4M&_1fBlI=JxY+A<5GsAnJ zf%^gs+qFVECA%5n{biQnGq)If$A*KA4I{Ruv~#uPm=0n&DY2L50%2I7|tcZ3num8=Ksq{U-H3Qz7 zHGLhONIar%(qiJ`6@u}#J*~_oKe<-pk4tjKLi1CBf597RY-nwkn>|^D%COhWpQXOq z4uE85LSm^K0o%cNL_%cee;@e3Ox*{TL4tc+m0uP+L(O63(L-kw)L(QS6gj-0aWKBT z8a2RHF)KFz31dp5NauSVfAo_%u8wq@ULoT~LyL2-aqo~Rh(B^$PTDA!&VZLf3{uWP zQ}}exnPm^V;nY>#JGG-UZxbD^%bxc<=*iX$%S!o(5@_tpG zd8Su6VKkaT_TrW|Ls|`^SbRqx_S;o{RrK0Un(4Y|Qh?vGiP4%@Oq9ax!82PiC*76Q z;W%{_7x+&pwzuR>*?zYHm7%4jm6XjC86Ev_mv8>yfH2%0psU^6Q^IWEvr$sbcxs{q zJ{OVWe7^^78{%g+^-{ORAuLijec(}2ut+n=OaUoeQJNe zKRA!ca;f?%Y<|O|8ZVjJZtV_%n6vRspzd_{ohm^{rS}P27T$eCZq2I62lSkI1xG*H z4J}_*TJWSONgFEgCm?BkiS&_d`5|Xa%Ba6%Ov}i)V@xD)#;(*9-H42mjIqyI?A%^) zC9=RsPq#JQNNHD9Ma!ay&^TuwJM`alM-V^Bo+{oA1P3mgoDgGSwNIo+P=TKc(5W48 zHA$f&eG{{dvHwCBHh@O%hh0G>t!T`H#7naW-^(Y{hz-tcc`5u;+UyA`T^`_N5Kq_}-o=AE-KT9~r2%@Y-dQ&VWp{R997`Ow)YgI+AP_ ze)w(s9jVRsh}}2C+*8R=8C#%(;HD@!#(zxrIEnZxHqCgl2u+K+mX3i&m5I|?+9biC zTk!&%Ne{!{$#&Hp{U*xXsQ&MlxjfN+_E*_4)oLkmW zBu%mw=%<%UL7ax&=#YQLN^GRpfGghW9<_OD(K}-H=_#PumC^h+*HI20ur&)KY=fh# zii(H?rr1vv=|i9Tj4F>xT7f+@H?R-Q8(5D1U>Dn#SC&=07{Nb!C zFya;2L!%dD86GWv$$-gNLG%%{t!&C=03s;|R8AgD7ApBF3s&q5{KD9G4L^5J7#`1j)E$kn$hi!Qd%1;$oI`Ds_`duUM;3Bp@sjo%hy)xgC_lS@AE4p zce?|t(8h@r5nptJ!F4JacvDJjffGxEOqf}_*K5eWL0(`VjM#E(rNlILD-l>eulcZ| zP8@RRl_Al_F2@$5KA=CtcpgOJz#jFhr!6JhgDmG_w|F)zb{5V3ZJkZC^WS7-vI{Wj z3^O%HftJPX4+#xz%?9+eISc}|x2Sy)X$-%PL^41FLKylA0t-7iTi-}j-C*Xom(O|Q zFtM?J6@^>zXVHgS8uNkXE(;_SPXzjlV~+p+ji`DnfCavUc$J}ky|4Y$#J%?@d)njw zlRq906BD3I9Mot^mgYd3@qRk~^>pMLxYV2~W}~DMwkHm&Tl5_V8oMz>f&*wUHBTOg z5yJ+D+~VQ>KG(hYvu%o>l*$U>E&MMr=Lj!`3i%#wGK4!(x|8w$T=E1c7GFPm(5h-v zmvJyLyToDJz@cG7ZEobd&m0t_d|HryXa{7{{}G0oX60j#&ax3XS+lt@Q8HwbBe1=b zM}mhBnV~tLo>J;dpqTY+?FYgj5ThnN74}#{4HBWF*9=mDKi0q3vFd@GE~uYHof&(o zmSn2hR02pfyLx(H@wpR%m$s3mULPUFi05>2^XJUyM&XBo$(E-hZP{f=!}^(PKUs@S zMjh?2o%Gt*CKKN?^prl}A2|zn@bu_)%$>k`=XpD7LuA!1bH9#4jo{DQq ziD3t!-9f{!n;qqH)AGIo4Zb6S zLMG0r&x(>wQm#)yL4k#Z6`(hmu2a{vL;Ra zuB6sPptOUt6=J18!?&muvR017X{IXgX^@iiSvlA1pqb!-*tY<`=d1LMIvs^y=_()9 zYacEhW|#@bNlqh7V3Ww^K=KTD0x$H&Pzg5tAG=`8N0IcN?l@zmua{S{vv=(qCL8cJML z1`H{TI@H9{H1&?4^oeJbZ{gWKMs&@937H!kf6F3iA`7NATd9h*j`0Ya=#&OM^!fWp!M7~ox6pleXVnXvuB zUJq=zlf3z@102*CVl*L)uf;^hvKu=T8mq%%x;zatCL3n9){P>)w@^~Hg|;A=JL_s< zh3!h_TR4SiX#PCohIryGan;-|)!ScnXiAKaj4soivOqPI8&f$(?=4{vDV4p1Zj|Vc z4XgDJOsxHiZHK#?3q>99DC!kyX?sgq*C2w!(X-<-m?fn&5?lP{DIU+b5mQKt_UQrh z4r|AlS%%a4Nw@XzcmsD{6&k_DO(}(%cWD`A3*F`N*iWZn>8lMS=~v+u=fw}P|GtcV z)@}njv@l8Iwm4w;(BB4yCZnp;z?eZEYRkKL)eEAL9C3|L8pL6i0?FWyD8A3kX4Bo&f_sl`1}1_2Ea9grJ0Ou?<)ErIt@} zNsqD08OF$d1;GFZF`G?8iDNLr0_)>nv^;H;#e`Bf(K^XsMc&&mdLeI=^C!P2M(Z0o z6gHUA?)I@15|&;^7FJS_tTGC!7*OH+BoA{G$@|Xy73F6ry=nU$Jk=DdQsl$`7DMpx4Vr1q38`D@`|xMi-MWNWd3KPhXj#;(-#G(Nl?!r+K)MR z@D{>RjPkAzb?D7zKpvDMdSD%!0{8vU51>>Sb1WUs#PU~nhr${3|0+(HdcjDPu0mM5 zLeHNgmuwQw2&J4!Kw9*^IodT8L>C-7)j|%TbgHDkm#P!dn7?!yW)_(?y1~t*zrVi= z3yAvRpVU14Mx{)M1$v(Zg7C$%`bl=}I3r22?Iqa=gzve;Zo$4>D|+@s_D5#OFtfrk zfw)X{NRK5|uI#1*p?13MXN^^FJJXUx9U+1~_ua6*&o{DV$=p=pQ2I6+$d9EoKgAl& ziC}AXq8$4g#C^$ZtnjesQ=(vXv>88q4$j9~+zq^?dJ zYrk0y2roc4_e@9ZsWxrU`%vet!xj-r=R1*rzW&D1XTD_2wicMFu?R5+24t-C!JVP@ zkSDj&IBONSW$1@_bNE>A&=^r#m(aA~Ku&1*YY4=7i3;YJg#Fq5-R2Hj^8}e$fdvOC zcRwFRb5}o|U%g~YndBRC%vO-Tz!U3Mc!q#a8ol>cD>#-NJ!-5FrOg5X4DL(N*-d4(H*{c9;G#e%In1}zu}8ua}MV&lG?1VLS``M#brbv*5K ze!gGcpDtc*b`rOLVbfrKhD9M-YjWHL&CB9989>BlTKBr-2f3NjeSaCd&oWp4RAIU` zA~j#cejdMy@?26t!MHJ0>ab~k6C-V?s#sZcW)k9!Bq#Pf$~6Evj1&9Z#igg?8!tUQ zKN~eEukd(1(CajuP5OP@swnXHtNVi7z?U10%o4u&CEATkJYVOu?k1}wnwKDoh6O*c z+}>mS&mBk#8re)AQ8pm427z|UWyyd}3l9Yio{_r0d$=4UFT_jL$Nmie{(A#PB{)m+ zQ#1&35<&rLz+T8eDD?^paU;eoYRuAf8Eb;TZ98Z`E=-3cN3Ll`PF|k!M;$nL09eLg z%&U+G?yb0jLRBwORw!Q^GSR|6P)^Y9+L%Tt2!jf#W7=zhMe$;xC^<0G<5gk$%+X)r)Ke&zuJ67L zjg6Cex8hb&7AKOPOR*`}p6dPdM^DM3Z;mk+RITPw-&#z(7@-Qoo zBH$osMDzJHD7Z5Wu@3*-kqMGz5ig+#JZ*ty-BJ{Fem^BHXdWi>`BC9WAG%s6^--3D zrkPbN#~X2po6Da51Tu&E584wQ$~HKkzMp0din`z=lx;kX02tLlTq>fG2z-(@Dvl!{ z;=4>vmejBvoZy7q8jpAQrEZ6*Rr>5V!A1yAe4l90lI+re>7$EljcLR6n9`8l5>cMX z*sE{#zy_o>fkyB2v7u_cf^CU4f;snKqGFfk_(1ftQ^21R9yu7wHYN18#>BxxV3^n8 zCQJp9u(&k61b|*;OSQx#I~`$&f*!s+Yhw=uwljlDfRb#ht?7X+)Y6F|=z9l)QAGu3 z?J*!u=OZ7Z=c;XE`{&0i?xcFB&l4QIc3l{A#?%`k3RlEKsAqb(jR_CBGLsHz=L6$Y z%;uC`-Kl?p(yQm;%}`ebnV4^>hL?`RXbQW`qMGbw^Q+y@evm`?eY@AA>(8H>i)Byb zJ6<=p*`FC*Pkx`b_EA_KO?}ko3BzBWCB05~12;87nfeh#G6L`ixzL{fPvLdjlDMWt zQmEdiiy;_c>Wi{bo|8ku#DqcuTk9QbW%U!n2JAz+5ToHA;}&8CY4Z`&iAvuc-W1TD zyjy8xhHqY|8_JfUni)Q_gj19U+4&>-P#vDix3W$$iNA=A}Z*mFNm6fWg8wVBVnTWyR0JAoQWME4SgUp z4!vfMdKuN8YfDRudWE+5)WPP*-7BRy(Y0C`ETI+`IpQMTko}D=sqIZ;?>4s!bCTe# zF2WMZ5kCd()v`s_z`5u$vA}d*2}Q%g)OVA70&n?GpWZnGbWdV@-;-}5|9(ZX3m+f!0(VPkz7kLg27`t;j2XU!5laCYn3MY2Vy`l!G!>~R zT8-8n?b3d8JjaU;#cW7q=O0Qhq4Oc+B}&~@0e(d?f0m#338;Yfr>sql%z9r0ZWqiO zSlb!I9+(HjxuFvbyFAF&B}LwQ(AhbaHMF)YJGgFG&huO+^-TAJk0-bH?@90kRS?Eg zc9*O%k>J_(qxMeKFP@*}L{A^l;+SQnGLwBW{o2Bl|SslsXl |7+7E{jhvfh-Z ziTVP$&{*PjxnkKHxTV<5vxp}y6_3_CPV4&~SiE4<`5?U;>RgOi6dfnG3a$ycglS5H z8t3O>Vb^C!YvR60%I$frUbaYVfq!1zsgtJX^U^j(jh0VcwIEZy45|*_G?3HlPw323L>KnPsd;dn|MrM^@W^_ zqF758kGd9*_}-4D&>+koW)H#|-FVh_c7Bd?ZuW&_dGGw~`Q4Pefi1h=SvdhaQ69Y< zGCko5$=cjxj>DA@NqgNO1n1o?846qVSPIK-rlDi1p$bf~PI$52>8CemiPRbsx!F$( zrb9Z`NWXp9A_pPVkZ$`JUdqz|X9sxFCC$@^s5w3Pc7%NGC;yZZ3Ldo)T%$*>TF%?r zt_gs9V7b0jfYW)xOq`W$HAG!`M?#{@^ntx>#Hr)+&24y4EbNnf6$1ud{mVt?u0^)(n%LfI`M7v3zp5BPeVJ{K*(HlF-L&izHmnb^-6AB6*sM~? z3)_@-J{>%7_w}Ho+iC;4S&ls#J}Xj3Ry#YjKwC*FraDt0r-{W96x6}^PU_>eu0$A@ z;JW;>|AaeW&jKD#K$a~4bSt0PCtn?sutgROKPfe(I{Qn=RhaBX4doa@yW1ETh zL?ODCd>wVlei>MF44O*H9NAPC(`b3UXuSkYrOE^H0k~n65fVGucJh;BL#A%29L>%n zCr!~PH;}x`t1`=4NO))P1Y$B#T*Mhp%)Vhj)S6r1JBwfFrco2 z$~;=HNv)Qt>OqEzHHzI z-UyS(o;qMO3K$T6ysXR^Zg*iX7mE8PyxZS{Is>4-H40x(vZzrLHGf1zhL*z|0n?!A zuVI^?di97=op&>`I86E(I|c!#i&bRw%C71{b(E%)o|5msKi>FH34>8X6?4Bh1%(}2hAqVOMV&1+ z_iH{NBAVx{(w(_&k36Ds18>kqfVS*_imzh+0kk@UwFFX?{zFks}i z3LCs@YUR)_OMn1E5|*04y5;U)`aZVeKFr>T`fH>^9WK>*F3;a8=GgukC{dqJko>^M z8R(AS*jC_YR2_6a*;T;sY>ox~x)|?r!O<@B{g@@*E!5T1D>}u&_O;x0J<6>T(9|OV zH{%zCgTQ~7r> z&X0Wkg9r661(NE_v~B8_v#2|Tp3hl)3!5CS%#t>J9SQ^dOxEBWyM3&xTY2K?6akB{j2N?WuPiH=Agh=X zCcMhD42DvlWTHdao;e7)G8&%Zw-zZ>j$Jhn6oiF?+);P4s_+Idd37er815He`%MBI z?-J3I+n?lrMNAWiTeJG)7JBT-y1E`DvaLuK<{iH78M2z$_E{Fz+0*^io%H;g>yi0CC z$>F7>0#OSxh;q5TS9E2OZ^vUiI=3nBa3IFu?O>7}hCrM^+xh&_h+M@3H-Xr<_z|(@ z_ZzbJH`@>(Cpt+MCue-VOk*|tS7cEU)&0!-qroCi&fc<(gfA(uI`V)`M~k@1scJNc zO^a)!yC8AE-u|}v1s#gyeMF-JB45kvXnuQpoA~1@^k`}9DYm=vmr;v}+=zl*#?}}J zJ-z}e4p1^$L>Y{M*vn5-30xGhAMvbsSyW+OWz!ctGP$zW)qmD;$?#L^C|X$->{Fjl zG(9tzoKKx=_0#`*EI3o%p5U8xcO(Vo+C!*9t5A$Dj8PP4oic`bGxtRZ(e3jq)wMK| z%V=d3hri+9ilpC&fc~Z(A<43Fqk%3iJ>KL7Zl%Rc^k-3gGo_>Rmz%bhm9KVxFc!8C zn^1eWmUB1C#Sw_1oIvD&k-A5q9CuTYlx0FRxArv3f3)cNq#d0xEnbGU)Lb7jFu9CQdS?gtDIH# zxq0{Ik~WPvHvz$Velqn;^yLiH6-=D94BVA8?D;IbMJ4LvGRCGEZFslVCR87WdX8J& zQSX89pv=;M+k07oLUwbvvn3-zO4=|AggD>3okzy{d9dDu)xBo4?~!b18^4w;YKR>! z0w+N3>JY3neu9uGt+i_Z70ZKU~(z;rkYh^->ETm+8zSq{eQOz*sP?nVxNUbZOHz*|YF_aXJj!a1`1 zc;uR0>j||RhU8ZywH}M<8nTgIE|Oo%m0wy(t4>N(H+}etzxuasJiO&1ai~)+v#pHc z+%SL6KrKU8M@8D$%84vfv*pSt8`K5zrgShDMhA({hq??&>R(o^K@F_Ud|{}rYPx)% z*n#Sqx8WH5wsIw8QX%#s&fQkSd0i$5CMB3qC{5QCX*D$Pf>77}ayMx`W~%*slwi~0 zo#$trMKyx;&3Ihbk`5@js0TFfGL95()>xzr_j_GExm>79w=Grp-rh9DCz{5(PrKU})#>0Sht;4lo34zgxMqczZ?vzk zfRwCV_{BXvcer&ACuLP1lOO2%JVV$nFI;<2?re|`SCJfRt8<$qRRa4-e{FJ+c^F@) z?bkJnnlpnFST9cg&Ih3&+8T}Ly_sEsNME(FYE zATdTw7r{;}Fk6DuxEH_TaB+&$7F*SDR?7rl3^{^PP*55h_0LaET23olvwRMY1?nM8 z{8#_x-^nSt3EQJ)0F#oSGoL+)Pd};($;|P7u@=&EGS*M4 zi&EM~Q2ydXWtXfAPeI*ZoJmUUM1HKF(ekNpZ^)<4XQ3>fSieGuv$Grt?I~W{X~uerV@(8-FHw{gJ^%r0*d4ZI z7I=5*bt^}0>oZRXC!Suxme}^gh;KI4T0B4XS!VP&XI6&}rZ*`6ksb_6zNXg1)LJWc zR)vGg5{ddK_UQOeR+lA}F}=uy=6gaRUxG_zWGfj|)vC^=Mk*^L59~vcLk=?nHFdS> zwZbb<^I$HW;`A%!GXV098E~)6)`dSb=GS?xQAZgkb%tuOlYqSJdw-kq^8I^IW;K=A z*&yYEs}$wP-yp*K`47pVDvN^~@d!vS9g!fEdupT=}- z&V#OndlkL5+3EKu%-_~Y|PraHDIZOu?Q!PDe{c+Q8L7*wm*B zedJ~(XC;peM0ke)lvB}C@Q9Of(NKvT=(nD7O+sA>SK9m6ZFjS5@m=loTi0FglU*3` zHJCQ5osL_uvJFi}Td-C+%73obnJ*^Sdo@%DHviCKMS_^cnPl4d+x*bvX6^1^p?y;j z=)H@Rq)$O>v=AkeS`%Dbma?qEn1Xq4Zt-I$Pwj(>?%}fgHxLk)`^~Sp{>9UcP0qd{ zd{;7Y({Y~|JYabif?}yhFCCba=Al_w!nap>BRGvsI87nlOg&ovp4p;Gaw?->9TVQS zgRJiNrO~EqRJJm;(a|Nl#=ROpvirxb+esR2rm`L4=g|)hwmMbjT%hNqQ1Z#R6T>ER zRjIOS9l1?GkQFxyheaf~{h9j5DFZioP8pA{vkQed1>P1rZ{s9p%@sH)jmg{&yWA4Q zniZ&;wS98{kr%S|iZb$&UNDL*$A`UGzG^8@Q4*DX^hkODO|F4ZB6ZA~dC*;Fo*KRV z4I@6QoH-?;p~!NnIgnpV#7fTyvWg$Rk&%&ZNA%w^)v5QSVaA`@6;S7 zU#o8EECvfEr%a|sV*N)~#$0K_`Y6LTi#;~HUiVd41~f2DJAG-Q%_uR{Q#B~_9%qHN z^Kq@#fh`gx?A}d)BL^*>=e5H6!>#j`aLm9+%(JBZN3qU#87-@CW73fge>e*RCdon5 zVSSEX3@t_|W%L@h!ZzAK{Smy0L$1N+1KYJ{ynj$y=rQ5uMf6xgsSnw-m6N=&xg$2J zwl#3L6rM@1*KtnY&s7ccx2?~P=P0Vn=EDmeO2+gv~4R7(AA`)1a%9E#|>S5*cosBo5FGA5l z<_QxmUv`FOwG}_3UY*71{4LVrwgSq;zCYXjK5=n4)wO)(U1m16j@&6oNuw{WrMLXG zHTnUq+VM}TJ}4U;-o)Io-$^$p^3<_dC)Pz}peBj)E~c9eQaL_prQ?J13Kf4bU_|rKXJu`j%pU&Ctw^8?t3(-<5s_Jhvm^y!g5D*tP zH-*HEFIB)fL?`tlF>pD09}pxIq#$@dzLESzh)YVIw>0-6@Bk$p?2`wR`zdNv|61DK z72EfI$UfVM5_Xu0vQ-x3JHld-;@rm1@g>a-NSAWgsDYAEge#UwsY2 z5N$`J%0x1Q#~0emA=5yXVO$=sFgRfDgcEa<0}sH2Q492Rz#r?y%6sf*rEA%mN8B-u zJ65;zY&ScLW!+e>3^%(XjnG~f<^tez-un>BN^U7L+1MwX<-qc%@9utzIp58dXd}Q# zChbume}a>0(=B;yK6k%<7TsqU;6()qgg&2~8@!AF25 zmtybtO)*ZL1IVbgR@?9U_6N?&4`yNFz?fqM+np!0{cOJH*Y7UlYS2zrPM*3_dsf*A z8?Pvr%PrQoJXddYDU7=8n^qUE_C|7t_njZtOn&a?F`Zsmk-Sy!FDacL`=gzn&qact z$1#2{3B?qA|jkl03wfP_#UhSW8Jh zbXXGsGEmrexY`=MChOv*-1#eRS^QcoX} z+&xkn$%b1ccw5%a-!h{T@~L~`8WnA#gzKu`*GA{lzf)Prs_2)PONq9Xw?KtT%Hik` zOsT@w<h2X638)&Y>QCw+h>DSp9Wq|q0fzsPaB)AuWa zUMR!IC+iZWs6!`%ofUFQ2e2t1HKQZZ+`<86J1|l z3kSV6RCJaL4-N7RDG1A@rn++WyelZEc+}EI?XacrSTGlUyZY(BsiL-3aw~nk#g>x- z8yK7@!5u?$OU+ifRQ!T5xZm1f#yHPYF|KzVZrV*STogI6fNh|tfj-$-hxjvv)dwMt z+Ueeg!QnF7_YoiNHB#Kx*{#^Ck&ub#>XuT#wFxG;1^nH$4JMpBW_`xsCI?UwvC zwzIK*#Cem0nRH6OsoxIW!F=MTAGF8POtNe@Le*Z~G|rcFPg)8M?{6~pBf;RJ2!WdM znKL*Oqxvn(n`YA2ElF599CH^N(>sni@5NM3;}f>UWGH-vB@E5pm3F9(2`_X1q&ytP zPX~KhH6_okzoMW7_w<@=;d{8cnf8zpxiy9z^X^`xxNx?n-QpN!Gtb%vjqYP=nSg%? z?Oc;WKV;CX+F|69w3=Kf`Z~m$^rb!uKLwhoK?&| zMdz=Mg?~G>L*s68Q6=@JBPG4mjJt?J6NoQ_T1}F>T*Xod%NZTt#B*VgZf^LXp;T(;i%NP)oxCbRx-M(QDvm0flCM8f* zgI%VtPV;~d=Uj-+$ykg)rS%AeS|mQmp?a&8Os)BS-Zyq&Lz5F9Q!rO~gxxf$td_(t z?vpDwke=$Y54I<^Hjb(@%Fbrzh-fg{S0L~m24vH(HX_u8?o0{%KdP>>E6T2I(K}x#I;c=N z+g#exeMq|oZF-YSS}(LU=`X|YgayrAtcn*Klp7o@e$FS5)(uSx@=Q}VS$%tVyb%68 zbvV0JF)df)qk5jzGO-I5>}fE+rwzZw0Gc-cg`PQy9Z>&jgG=Jr~y zmke##2s@xZ(XVUVDa_9WBxMDt@|=b7xG82-+iW0*WfYUvnv8W;ik40+%gwExd3#y-6!vPnl+_tyi(0S8Fl+a#(hF>EXgdF;jfSmuWSAUr}qs12mB@<+gsu z(!KzAZojVEyXQuMjBwGHb!P+dgEw0Q1^lR{@hA^_PZ!I3f*9v;0|s zCDs=dr9(tqTH%=6r(oiR#9sg`S5Yw_f^BOdxJlVpV)19L&;9!G2wd_F#93AgvpKF{ z1H~5Zn5Zx@80s1&`ZW6N#GUZ*>@DB=Zq~Y4OxR9#mZ&+l$?baNTT91|Jn%}Fo ziW~KknZ_*BQtX142%pX;FcM5bBtEG?q-28W>HU1Y58gZ>Cp%=$emh}6$dQy|)X2%j z>?%rS?NCLWGV^*a+i~F@5L9J^bwZJWH3S4<`7x2(It0zM4|jFT# zK6_LZoSlLTMh~FHB8Rs)!a~tL93hMB#RLithM>fVFPqRcv(^DP+Sls{w38vQZ8}GW zU&-RA2*&?>*MaAKGKo%=X-msY(?lv&aFZd|J#b-Z^-YdA2!uWDIkzQ8tOhk!S!9jR zl9_O{Wvf0XU10E++-z++ZPy|H=0Rz+w*$Az<^B4OAhi$;ZpoN)3&z@9%6vPE!;&j= zxknuYVloKuvfC~zE4zvUaeTH7`g<mLD@rvra~r5uq8BE6wZ* zq@E6uB*n~`jwgJ#HXcv`@XXp0`0Exw`yT}`sC7t6gVostl=ime%BpVLjH3fOlu5Yqz~0gulu9N7b1+B07PZ`R0b3@MO{WhElL~R%e+xMX?=jReezxM!|h1JqgUE=xo$=>I~~%c$tfH7 zDf4ykrgF4i=mY=Qx5G?Co0X*>SfKG~EZ*Ii9jX1F(S4m=Uo}G&3{$-VXiCnOL#=s7m3(dkNXSW!Mi6078Ygc z((IYbt48$Ci^`b8WeuMK!!cq*YQtKdP)vej^{Qz*FBXPcHPO88fPlq&IWZGK%F z_Rl*QjRwCr?KbF4_Tpz>`X<}%N&c`ooC0b=ynUi?$*asa13H0c6@I=Nc8}{H2h(42Ek5=gY`(Qn!2cNg);SsAH`a*iMw_!f|bf!?H?7SChQ^ z;pzF(wu@25^Gg>2B{Di8mwU=3)MvcowoA59_q_>*`(EfaJgtjp>1LlwkaR15|jZJ={VQSbb|H~7yAnzE8iFq-cy*BG9X01K2Y3U2mc#HEuJq2*u0%10)Spu zi#nUs1eb82H#ub}*lUYOvgyc*XSq&wAb=iX^PvM78Tn=?$sbq&fV-WE50)(p%Aw~} z+B*tGwCS6mVyB)s?h}1FLoWza7f5ZroZy{e*8Q$djR>AZ!j#74lhmq+GY&K~i3&}Y z^q9iE1+N2jJ!OVyDy%=dSlZid&fYz)slT5lguElX5`BKkm!^8tiz-aK()!%#V4L{! z^_(rTUwgy*IMndhW^UxSAzX25I$tKt>+{!w`70HCs@8kCN4G)JrcO#H_J`b@KcbhP zpRNhsE{ajFvhnu>DynwcG#1ch8>XhhP;Xt&p6<<38%j1uM3y|G848;d8Lr3Buak?0 zANE(4E`l5s9=0=xy|Qz%vk#~sIxX%*syC0tZ33#QFK~?T|2Afg<@ZA5v)k0a|6M`r z>$Y6?TyuXhn{{R2=OCT%hkZ~Rqi`66KDn-+G$zKAkjpj=?pI`L(=xSr?oQ6!)^Ii3lHjm2C2f|v9oTXfBv^&(VP&V)^@3q z|E2AXpULOGUI$k_iq6f?yy2i?UA}7+>?1V)Qv*SrgD8%2a^$nIsrDRX*5~9|N@LD| zWXUvN6^vvt&R5v?OXzxyT-jX~XS5NL*oUuuVyk`y+YLnwLDQtCyQ&%0%o6xw0&*UN zkg|D->B&Y22HU>{&BSr9Vp9uMuJ|kNnV6#e(K8zctdBq(W0ZLhMz~209vwm52XF+k2rlhDE<0sWl zFvmJy!5H@focO_-3SdnWhI2l`;^R3-}DFIL8KPpx!N8!CI9;=_l&oNv=a(ca2@Xkgl z%|1`S)j8=3+?aA6=e!nTR4pGq1v^&>$%9AkgPQ&fu$PcDmPu7U-)P6H3{J!zw;GL@=JwF!*f zr|0=5l=1jRzv*g06NTO$nfuZc)CdKoc#}u1e&sApU0pIe=N5Uq7`L;_gBi<=>(uqM zSpQCKw?u`<$>tISfFP*2;H~<`$E$hWv+uVJKfp?tw*|VX%Tm6w z2P4{an2$4h)d=M;OwEv=Zx2(EK(>4uXWAbmhna&IlvA|>V5Jf@?bQJ{(1bNucXv0? zrdP)|SjdlNvqx3yy`sOch6G?@3NQa`+Ivg_V-Ck>(we8A?sU*$8VyB!B+w6dq{DXC zmeVKxC>_&E-{qu&2DafE*X{C;u0(>-Z8S{Y?%%eUb#q%O>JD7)_&^S;sRNT=il><0 ze`xZHnNw@|0p#)V>l7}knMCgrB)+LL)M_(r0R05Dn57ny2ikC8LBju`^A&RR0dkAj z?>wDc>UDFY-5X$>Jy4?JMAg#&AL?2_I417T*GVoT=)Lz4J3pc8+)6Bu&khDfL z%>ZokQjYWh6mX-~p28|DZrruib)e*&sOb`zW3&)AzS2;IoTyr4TcIxn-ccTygI`=f zPw}>K6y|J-K0nMipz%MDDN!!|{XBYi=+etN83>E-H>&2P?)pzzJSeA?T1X-qE0xq| zi-fWcUV)uE{gvAsD_?M$W-4zs$kF4hgK8{wN}^wQ3{5;G&wU;kN*5*@5_+nUPWODM zEEhe!na*%M=Ruq+v#f;0#S#gA z;c+jg71~07#+&Y$YP8k2E?I$2<%y6T92f}32R57dLhuU5O+t2%=;IM@sq*ckRXfxv zr=>fdHC{axQ-t8jM4is+r<56zt%-CzN}`bIlx$=dGX--TnQ&4D(RoHSZv;KS5J*=Q|sLI|elEtmf2Xj-f5 z4#rLNr~qX(?7h6vuYjMlUe$WeInvJ;G7N~M3rzovB?u+-W`x4=(V9-PT2J4$j^;;| z=~iM~dz`FvV0J=5CQunrmWy$Vv@Ubz$iAG9RWZD6A{ak;;1?wk=T9CovwN6wI~4e9 zhn9Q_;FzeUgk&!PrWf`S?l-NYg5drs;1S9-w_C0x@CR^;s zJF5G4OiWejuZ_=RL?Jn2-Li8^?~;`scgU`-^!kA4#QYvv>K_xEojsYs{p*nYylofd zJi;4R_+>h;Tf+rx4)JH_MrZGbH^$7gUm{R`2d=9J0A$i=3Q-?qY=ID!4B>5=%M_w~ zYB`fRlm6c82lplr1|Fk3hb$NpyRxT zu`jdPA(LnxZt8APjcTi%A+=E}E+lW*C5A0o^^Ykl!~X>;ntd`+;2o8J>f}d(6cy_w zYdMtWq(^}M2~~!-o}c!QuY5r`a0k5j%cxyjS`)ONLHXC?d{r}UN7ciyBL1SJ@|VpD*AVQLuKx5{C-jHw19G|GN#V$IOrTwF2sSV{%3n=!^$W9 zK#ug-(GENvBEj^F_ZwM1ljwBqvat)+b1Hz~(Cp z3WUN`7N|3=v&QTvR~)-1fY`#YWu@aUp%-qX0oBrg^5}&bMOlzcL08@S4?UZ#>R+q$ zbC*d1l_zMmgAXC=ok2s24#w$j zska}ffEU-kDJqd*7j&*lf+&p{9$4vpbW}5exB7q=F8btqy$DTpVLUe63mTYNQPhl? zUDU#*KtxV8qa`S>Iy9E4$97Fi6Pld0v!mT|OTt6+TR|`9GROM~0v)pJ4VzMlnP79p zZmNd?R=(#8($y{y{@TZHLPW?m2RwRXde}jr!S^oylql-m0=ik-99N;B8hy~c17c4Padzp6G zSZu0|ar8;0j}>Iho-=qE{qRefcY{=0w=nSmyswoSznYx+g9siU9!dyg2oAF+qZ~LB zQ`1adN{fRQN*p_q)6?Z!e4o9LU*qkRm3>xJP1Vtj$zqQAzC{<{Adf8|_1Nrv#&KAs zhsrSf(>#(5*>D=o-A!1jZ2O?h0TB+q+b}d!#5l2X5|`DMac>c-$|B^C7&rqTOc8&H z3oK@aQrFqZ(~jy+qGmcRz}z`FZAst&_P%F zx#M$-qO(bMJ+{Y+!U2$s93?z{=J0BOap3w3#NKiLROJPZ6Xgw=(+0`%oym=d&FZ9F z`n5Nyqv>{QqVV-%YQyoykef7*i*v`qY>@jcMV@C!m0(lol32gB?LG<2i_Ygru)$&Z z7u<{2VX0~Ne5ng)=3cIC&7&T27dao~0OpMVPvHY1h@x4?X?Dl2m!8hU>B5@7(&E=1 z-nwP6PgHHSgw!$k;65@Aak!?G_~d^bE1x_7es?YCZ;)cYV2z%<_zBQSn!bevXye6cW>coOZ>1ixk_&%p?uixHtCOSv$f3&3&QgUIT$E>{6HRHkmuy_zZ`6+4Ih zsnQ%~N|(kd!rwdz#!fth5>M&0Zqi=Aq@cQ*aVms9Xrd1M5!N+)BN_h{}<}6C2 zM&GG7lQ-kRGB%zX{i%aT?~oyR%-Ua}8NV5a_*7L#7S+ZU>5r{g56#Am#jYKkf-XPo z7F=cUK4JM~CF>9K5xn2)00!H;LYp|5_1zmNA-q}KmNtz;54JxPBNqo8J;?Atb-4N) zZbd*exR}WoXm>`h6lw1~M4nH_#R7W>o~_BIK;)dsdf@)MJHCqWndGPRD)$$Xb)z3E zuq=~)V*4GYx@D1bzz)rRUYB+Jl=#V)#;`2)u&vC*vHF0v1f-#T%pC&qj2W4iV?)lZNomUX;Jb0V>>=_0AV zdFN!H>o=;Wu1jo zRc*uIQn#z~it=z;H~0bb1|B5#8k;TgYUr--q()Y&mfdc8t9m5Xm_KYF<_Dl3NjlU% zR>3@YUWJo-KF`0ZTf0Wyx{Kv)AnAk{JO%EiG3(Ms&E!%+fE3j$cD|}Nqe=f%iwAVh z`)@M3B;qc$tAx?Pq?W(dpb5dpz99-i#xOHAAZy!})@2p>)n|KiA!eu za5{O9fPQJQh&psu)6=x1;BzV5Z^hg)qi83iJvvW+T*9W<58A+d zds7+9nw+oJAi4TSQnQgc>v4~TW7Xbf^OYj$#Ps3bcJ-}%;UMu8ab69hy~lTZt;RNG zyN1#jqd$w&-{(ph%~%JeY8F#eS54|{maT!$hDb+amkHA$G^gOEsi`!087)YO!NML8t?S$sKrV|X@V{V<8IW=yg;Ui@f1dT)@tRo zG}4|_3EN?pV4vS%T4ST5ehj#@01Om+p5!EGd`VLAC?xGdU zTGVS!I;6->6{!g<&-1tk4=t;Wd}H%TXu*9xpfxZ7n@9(AEQug)og-PVP9c?HLL_#z zP>Nops;sW?drqA4mKl^6<#Y8{a_#n4elI?dJxQ8nX_76F$lnjK`KYAFj_9P}F}qw^ zUS39OETHnR)-Z~}RAk>Vr0xop8~im8zf}UNo8RSWNm6NbPv|>~YD=!dHsM3Y>Uv)> zDZ3^^qMP=9h3uZqH6#}&GxDh^GTEQSbR9hnsQwEP@aTaKCMso6lzVA*ybAb1!KWD@ zXb^(c-@_N%jH-oaHMI`*)UR~MaVoE}vskgWtu8nFvz-&$(FHcHl@D$>esnRXKxnw; zR8XfVvgs7=IwXI5+eOcSqek1@-*jBRp5<}mz4mmU4kS!0d+!u4yWM1MQ+jO#l?#WT z%}P^xZvm9|}Vd_g>Ah=l%7v*H#o@wA$WoyRSPI4tYopc<*_66;`?e&1fhz|jpD0q@1<=Fy}a0uhIcj}f*jas@UwPrtfT zgQuf456$|Txz^E7s8MI7`b|d@pYyt)thDHKgAm2Y-U~b0^w!69F!-*9uj*!l-tL`j z;`>F$m27MlqO{AulYior(&H|tQu!U2BTPCUXd4GnnL3?m;Uh(}MJ$a|JwM6prpn}s zi5kMTEh6rc2N{ykxp_(F{Ju;~Y^aX%fjm0M8C^U6p#CS`9xGRZ$87>|--}cr%~k#F zp*ekPb|Eb_0U*%j>O=YTg%#Txrg=R0xEaUfdc}_+Ile6za{WF{f&|) z4Wc_0U$G=C!yVo8m+nPiDS9dDqJd;5hB`o7w2_u2!_*Z zq(XXoFK-^qv8Rh(QE(K0>`Ri=H&ih}srWG|ggVj0K`up&)HoVDBe*-(j3LB#m#jp_ zC(QNF_hWV6imF%`In(~U2Xe;|c2SWwP2|6RZCNQ!Eo{?S`P%B6-MF3uHp3T6piD4V zi|I-y$6F(A0(&nVAyZspnB{P{jAMHXdb2Aptuzu$JO}fQ>s*>#-hh+!rdI*~Ao%I? zU-rS~saan>34o<8p@>Wno}VI+kXy&-=cURp<5YAM?Sh!w<;CoILP?upDI#q-iN-*AT6qV{*Xv2a5AIb3h|~TBINIe7tJsZ(iDLwhCW?yC z zAyk{a5QU#$1pw~kG&8^<4~=hkQ{|a39-bs}DcVTn<_>gylSizY+zmYLZO$}a^W*x0 zY#KIQW#|@)#P6$fbIp6H=yc`?(mgiR?bth+hvka;6$#xUZOr} zX$P%+lH;?!WTPuRFy3S<()0XVO9^kem5FTQS&-eNQbmx0<$TEpI|p?SPlF{UvIY!5EP& z|3|2?JAyGzgHlm&fgLXTaTQ!pG^I6)0NG-4STCH>4h^YWK^9n;MhCxUKLm{Pk6-ok z_A8n`QTZU$9md3GeD%yLN?HYx)w{mu);6Z&Ca%4t_)9yLvc`$U*tgbp6$s$Y z^EL9j;!HCzFtDA{eLm69*N}8@mn-a zSeHw+x~pK~eV8^402{HyG5~L$r_()4t(xv9yDgSFOVb8BWYJVL-vbr_c0c>hdM36H z}rRsnlnXBA}~jJs`5ue|Bp!&H<(#E^h|fpF)-}dc;bRT*B4^revg%{LZe070sjz&vHT`lXIGf8O1@)JW(rfmx6>tMX zsu|rWXxVL>;;1tOA1JVDM}vnb&<2ud)EQI?M-jW;l*B{)bE!J>b5!ei?9Nxg02q^q zb74X;j~9)L`|Ica2%oU#PlUJa^2THJ3TfNf9jMsE7`{-(?8m*5brxnjDw^weq;)ay z(aA2rH_e+JnDAd z3k(|$xE&M2JwkwT+Q-B)PA^uclN=X^?UocN54&Ohy(N`+?3P&zlpSZ@g^}IFJwCAq zlCS5UR{B7z=QsBWkb)1v9Aapj0F=)mFB)z{(7ssL_IN0G-28rQxau7pqNOdKml8t4 z!B10dcqGZRk--g^OV&1Tv^kI=$d0txLHg}^pUg;a74NOjjSn06%BF{zts%==I4VC= zawv(XR!(&*idN}2fc&8AJDVlv;+hyzYX)O<9`uj|N+p%tQ~kTh&j+LM`y+ipK7^wJ zp$bg2<%xs@X7MeqPv~kTyL1uUTtz6gzjf%74A#>2RB%R%c^XLuI-lBT&eP>n&EO)S zE=r`U%)~OMDde%|v(1bJgKLJHZv6ra++U}93W%|2DOD5!!$v^-uDMyN`a3n7gpT=k zOW(U~~tX`YW3xf@5f0`AuCr!*Cz=H)w-Auiwa$Aj6%WAyk-0FIvyy%c&2-X#_I;U6XU2 zRNnG8pl0)aX>cuNv1A-%{EQ{zq%EUW{$=a!lAwRfaKeyVlP!E6)iKYYc03m_Y3wG` ztd}hCSAn+8Uz!Yf=3fK|uCSs1(sNJQpuI5Re?!QGAaw9KKBy4k1btyDM^)GRcC-Ix zw3_v0o9yw^HjZRX;ebM?1~QT#Qe{*Hn062%PYE~jIqt|>pA?{yJcmwrmWBPB&6YCp z%)bN(z{;TOjpO`7q(<(IL#f}_76T39tom-*CR#*A!9lci|#s^yR?%GpX*^s zLeR*1n;b2TEZ!Z(f5H`&!RLA1=Y=C+qw0ivtK_Wc63}b_tRi_NI{oq9i5EILo+IVa zx3^Kz(*s*i(`ztUEJ)1lqvI+VmVp6hUR{Ch;>>{Dv)t26V^$~~1TsHFvk-esCI$3Y2R_P3 zdvV+Q5pPfMDeMwAQsRcW$YF_xOCn128S0+?-vTB!S6N-br7aU0xRNM_AARY|TQu97 zh3^llP7nkR)iBDJ*t}1%7uKho2H$FWVQwN~^gyHClFpRJ#pWTMlz*<~8q5PnzBB#K zRp*#Jg}iFM{!pThX4Dsa|50t%5yf7b#yr0+SKemR^F((;i}9H*aS3jBXHz>4L?E=L z@k4mteS6>7#9DCQ$sd-YN_X$kuh_+8zm}CbKdlwc^bt)=Ec)FOx326KVeJqLz4PO#65uOO(1qFVq~kVdG31duLM%C0mv`@NmCK`6SzcFh7TZT)O0<3QaT zj6*kB0SPYdezn7=)B_9YofwU!dDZz%j}kWYPc~eq@f7O&J~9v#LiYmF6Lg0=juAVB zlu3vvRW09NxS4{v>q882c5!(>Lu7)03uvcw+9?CIItj`9hMYF(RD?uy^(!}m?*-%k zNV+7d6J)aIf&$`}J?YxbW_7Y(9dB$GID5_it$1_*X<^G*WciT#@q{)qkIL zw|e;PtG{=&Jy2To--z&TUmDA|P$R?8IZU__i2?mQSm4bWKuV(B_ytQyP}Hc-F(QIA z2C9e}Ff+M2?6PKOs~`%3UsbU2m!B@X;~<&a^=#~@=?3pzbH$$iNe`^aIKe;Mq#b%A>D=FkdF1DSVbQq3f7 zj9HK-82t|Y;uTEAwi`bP*M)Jx>WLbiQrPxW4*Bnoxs^-ld7=Q7p~M69R;<`0{P0Wh z9Y*O3L=)61_9K1ujw_gMQQ%e9+hcKBoTnMQ+##|@qc^b2w6n+STgkn^V|n~2#O4(I#3nNkZw z-tx%#YDVXCQge^{;H1Tr96ZaYs|5nLHFo(heg{H5ZvKbgG;^Y&A6`&;?S+rGE9jUq z1RQ>-C_@Z!L&mChbPE{UDy7Oo@=)nA#J^GS-H7>Ny3!~JU?-!}RDb)RWjaBIp{lxG z@##Fxs0lS6N%5j*)m6ECZ8NvyqoI8Y>Sk93KW@SJS%Jews+#rv2BKokp0o;EDLJ;= z90^m?DIktboHO0=bH>(NAT#&itM_XqIpW-1LIKOJ9j z2;}DU&7`p&ePB|{#ov%q7osTzm!@j=s;qfML-f}h^fo6?ST6?HUR6E7y@POo0{H_Z zsN;Q1h>LrD27PEJtIRBS;mJHljy!USQx5BGy)w0N_`7+P{EO0)prF3#W#!Zg?9kh| zuPoLdly=YQ#m1zF7B_}y+!O!Dw-p5vVV5`B*}EB|)GabTv&@{{nzIg$>1fs#)Wj*O zDCJaDkH!*V?;IvTj4t?uqq}V8ClMAFrsjXU7l4Q2dW=f$|D75wtd?;Tu}|$jk&~+w zjRBn*YpzNvK{0_yGmbdSp=uqDK#$~*7np%$3dR!t^?EJk^%nc<+ZAExISI7$%gG+mp^ebUA@>*FL>uNL5#$oqgvB9`x!l{6N;%61_~o4 z)&{QB%I48PR^oHE0D+H@=QZWJ`C$w=Q^)*=AOj$(g5jliEF#D&o=3`U6UW~`TVQ32 z@!#l+nMl=`Gc0n>0>FL;DSRbZHOlv=WrBmhPnE`>$NM13K)ip;>@N09&ZM7ZG1mEc zt=X==8V4(R((YREnrB~p7OM&#Wqn$Ju0izo5(OI%j3mhmlfyTVS3XsifM|+VtV!fV z%EnM1sj>;5v-_LMhiELC(XSV%PNE+LUsZ(;FA&VrtLip9K~<9X4~GcCK<{1fOyS87 zuf3LZz27QUc;yxP&6P)i)amGHW`S%nW4_oCU;08EEB*y#kCvP*d(BU(65kx$i2J&l zH=<+Hn$<4=#J{`0Hpgbn)fK&DNFNr`i-$t)sVedUwu=$_Z9mB(w1Nd{tFFvFK8)CC8smBr z63M&+86a)7TvG{gtF;@zz#Or+TPZecW1;{`?ptgZpnMxv8b8)*avSfgX7Z7!VM@$9 zFC+goWN%;hbm$ApkoLuvuvQxDm1D|Q>w^O`di3*h`Ll601%g{;8-EL9gCrxD49%I& zR@E$18W2ZM)hztw-^>(CEEB5}UTbP~Q7JVJ>4{LI29Fxkg$)C1? zY@K7lfg+R|u4V^A7o|63Uc5p)hFYl&!qu#GPX^Bg`sF&J`RQV*vzjI<*r!s48Db*i z6MD4nRlL}6F25it2ApAmIU6}{F~Susi7*uR^f6byJV%d>)ezgTp%h+)JC!jRE}YOOC9AO(O#hXe| zwF4?Wf|fA1jPk%wwM7p=f}IP}_JB2K&+x%BSHjaoy^Pa#qdbtDc)K}_CTRXoNh0cY z_S|8AOsW(J;@{gZ5+l2$TuvT%s?$t0leKY{MZQA#EpA4 zYK=4b+vPIqM9M!S)BZaJ5%Ox&LFAK#sk+OA_UMFZ9O~5LfWjbfdD2k{I`JmFtL9dZ z4}DDqQQ`77X;h^{g)G(cEzlkgpC8HTEGNEUpK4E-Rpscre-c`n3B^7P_#r4N_lVBI#$Oa3aZf zMgf6cfuFg8f-s8tCkj!>0iprbop^!PI$u+H;+rZ$61t$wh>ISr{;yCE$*P4P@B0uE zl9QXOxZMiyTrmR_M!ZbXRi#t#5e~L;YD;tS=!ccxffGsgrlGd9Vjz>^9fUv732wr7Lw=liN{Ke~7_9{cS_k+qM^_ z6MW^8H;yGA(-l;dV>Q|0$FH7PB>CSCWc0DJn@e+90$Lx&m9wGI{LAG&r{m%RUoYb> z+bH&j@)Qi)Dd+gAa7*;0z4~o*Z*SvP!6j<+uZv{&Ombx;b{K|P<2_eASqzHj{(Z*p zHY%EHXo$NVkdK2J=@iFm#jh}6zB{z+J3)q?3x@Uhsr~HG3>-Y(DT}eaas@Q_LHL3@ zhmA|!=qZzSz+A`-5d-j8(IBb!1g(O-v>XG1@prQG%Ee23tZ193qNBcwy& z!W2i_h;m=FFyv){mnq$D>x#+l{c`@!BT4!_oS}SC-7X(^QHUXY)QsO+V9equMt05H zXv>mtapGzZ=y-b|U+jyQ)r=KG(06Wcf!dKlqB($2*$$wyOYP!c^Acrhcu?;U>HH~B z&2v|bLt+$}xPxqlAhJni0$iwwje86=M(hP6gFyQERLvlz=Zg2j84wqM!*&!qTKt-^ z$@#DmNU`-b6VLAKA}}?_KhU{2+b z1133)NA>f6`w;caMy;~%$u_T83c`|Aai{DKpqJU?+zM9D^DRpU7bpkfLHq4ctN0}U zwu>B+e0<0f=eV|YPFDHxYFZlOZOu8MeMPLVOn{tvWI6AAH+r-5?u@qT@RVMAH|o8> zR8cqH^RWkDKPXFAdMD3DB*|#4mC7)zsB8A!{6S){+Pr!!0+@H)>=ndoKJkqM$~?JT zUW<~Yexo^2Mxt=!KCIV2I476~{k<7VZVbUxykv;5m;TN!74gFF!>J)4=@H1wHbNS^u2a3_N<<}}|F<`XJ?|Nu zl$ZYUpkkb?QpV%ogC%UlfCKcNn}nqU$^>Thyhu!S6iY44w#x(B=8}7$h(df-kql05 zzl)tu$n}!4>nd+jIm^rFYnYkPZfN*n*P-`QRNl|Gnmc-k|4}4-y$ufAfG^k^sNRnK zt8{&6K_KC2%TfTyqVmLwV+}axiNs8!2MCEd-?sIGGvq7YDUT(j$^xNNaZ;>GeiOYf z@3dH6tYfshXkY1~Iqk_*)&Kgk$GJdAXsS=m_yrR+I0{eTs{i4dsy^c#Plo$m;d|-G z7dSYxGnJWXK9BplN(y_m^E%Jx#}W$UNRAsHGj}C+tG!EWI&Qa2g<7#X1Gn;Tdmf8Y4zl-)G7lz;$n|!W<%U zF^Rqjt^9`zWXRqE(4-{WUsl^ZCHmeEA#(ys+xJH;b$r6^8h6i%t$;r?7gNMR0O{)M zOo=9{NJV*rmeTwCtGZVH8opQ@I}Uu)$-RvE9%X>U)MIjO$%*tHkQ=9D-=zg$o_cHY z6%xh)3`rCx<8rTgatr0P%5vc`tzXDKs>s{)pDa9jp_spzSIA12ghk*Ny88B+0<>_h z4#*YB?bzqAOj-%8j*o>e8Kpksh<*04P}fz$B+o^HBA6rP8g<|FqEcu!Js5UZ(?}qd z89zKcQ2)qvPSV%c2O<`iKcCFdQ9jSnQMFJJ?68ELNxX;|gY6VG2daGH@n!w`MvRS* z7kBBuIN<|UWI;J2klNse-Icx6^k%k}RTvqfC7SjNjMB0`|A?HJi_Z(3?+Q~z8jmx4 z?z@D!HoQ+d&`DUe=wthtAA_u{t(R)8$h&aD!UPJvYlPlG=Cop1*J^0KzY;Xs~K~`pfS1BSiT3tD3mF>Au$M z1sC8H0Ah`orx~^T9<9mXlf*jWf!5vrIW~~Ff9hA0vsIXsa!$({_`^?76+nwBz2PI! zNz@y{Zn>5aJ7K9osAe8*9W08w5_5|7;hHD1}4aA}G znJ3X+mwlrRBBE|Z2*-c_TS$;t?St?fm58MQQdG&k<7$+SC!!u25(4Yf!?vhyowc@+ zk&%;A4F^StwRryDzkkJ7fJZvfmcNl?%gG`nf#rssvmHB&d6aW5tC8d)U^)yFIl^?7=Lq>q}9`z9yf z?y(6wTm2ArKq+cKfPOMt)#FSV;9T(~k?lJ%lPn&b!bLj@%aWhSRm&-kG@9hmTHNBJ1#sy_(K4JwKLhf5%bOC(#5c7=b&x;akX``-h}s_v z4c4D7J5Mebr{^CMeI{dW$APCu^Tfi;Zi^z57~)6cJjwLro~O1rn{qy-5707SVU!fV z;R7>9K>q*SKQRZfN6&{b<_LEG&0;=+PrY7;?P4W4f@UCBTxWIieq>=4x~UBwJB_TiJH8X@Umw8X!(1FULn~lrPd-BU1$TCl zIdqI#3Ek}Pw@qIuM8!&J<(Z;Bb7r~n=pF9J!5{$v|7 zyq&f4V_2no^ zepz2Axn!oj`L|v&d87?vj@IB653zRM$ExqRE>j~E7Jd&sd`gpynr3m=Ai!88iCAhl=2_D>nyThD$zyCQ`T+9`F?Y+9Y zs=6vA@M3_E{NP0bmy>PZD0KPH&P)kp-LQJW;LBs9^Yh9aY%uVJ<%G-Sd`m&S@kCt~s3GPZ=HzOkDc|RY1nQWgq&ct}%<~DD?j10WjGs#Bl zO^Ubk?8|yp@5^fJ*6W35&pU44(O!aZ%t!dk)U3(NTy-xPno z9nDjgQWUR;{VDYjO6QsHJqbrc2hU|irUM^u@Jc88CK|^HF8vp%9x$2)_K`h@aI)(R zBM?I+zqIZbw|I`Yi~}P3{}fp|8z9@uK*3Rvgi8?y;NKpti2sc1y(qFIyhzIS)Z^C6 z5iT`inV5NisA;Yzuf8444i^MPzg$Gq!JXG_z3V0H)*o${)w+gufG8|YbW=%OCmM8U zjh~+t(Mv!NQ@{uO3>Oi3U4&w$k>L_Uk%fJzo_3c~3a7-_twiKaSSjeYP)uTe#1Mp1 zjdVDKMhYIZV#0bVb~@!DXb-YjfphfzXcD|_dOVXT`3G(Uarjlrl1sM3=)emu3H8tKz+5!;o(3%q5HaGX zia#yev=a@b04>I-W!i|{aEzlty8h)fKMMBdm~u;=3*Y7nZmoH z6i%TzOURYG0fVF6J&0hk*kIg_*Cbe;`&pjqya0S|;H;w-r1oBo+QUdgUo}oq`*R`* zapun*Z$|6#zO#!y_wX2{n*IH{GyF^!#F^;i;NOuQYOPIW?O+p?zaSvA;s(C{pvpoK zyv5DGV0m~lt+NkkjIhIKV7$Hv+^ok2c32z!-ko|lj2;2a=yG>=lz9{s>t@oH<@M@o zqLRnke?|+NxF-839Ok>cK>`_|ZR%s2r=1KWd0x(skGA>6-uE zOJ`LY1`V!-;6h1z%cL?3;I<_VNUML?CY2>i`r}CYf0B&1cPqzCOs1K~M8cWKX-5{v z#aBoeXosq*OyJT`GX-{?GE6Bmm5(FyKVU;eWpNU1-?rvvPeS5 zkRngQ%#Oc?K3h*W6Gl`LX#XvE6WZK0T71_CU*x->eNAGZf-3+IsiiYGRqMh`A$00v z@+ZcYYmZy|>bysrn{sIv@_uCd73|{{ooVaoh|G!R+AeTuEH~Q6!OsMMWW=u|z1WTX zYVx03WbeFA^Z{m->mhjLg~twERfF30i{L9_gtEW#sMaU@4QyfX*)K@IxyL;CA(V?F z5C&#^a*_#oYy1g1D=$dcg;x+pzSP1G_x7e8ws(21kYK!Dux{QI=n z?1`p^;*jvPg4!>XK25a^Sr`2~A0Maw?mb_o0ysqI5ML2x7Z1L-?oJVG?VP@lZ=eF5 z4gZmL&ijV$+v|?_6btX&f*^Ud5Mu-~A+JP#LC9i&ib_0LzWgyWY4dMu{BHzin2ux@ z<>F*Q2Hwh=-X$$>=1pB?1y>H=oqS80E4aH0_-nq1Hc$y7@R!jD7s`FY_yO07Lwof} z>B}T%QGZ||1_joc&^qpF*2zZgi$tH|giRuhL|Q7M)>rkVF+}FblxCfPVEsDMywe+_ zSYN}ox3&Yy2>~oqKWR}b6oW-=U8u7f)>*q$MX2F9+XH=$5<>OimI<^ZeclB$*)h?TIX)K!RL&$#6e*WRg=5#U#T z6HvVyU8VA9D!IB&;7t*@bC?y!a1v4kwl@B+%2H$8=Xio0X=j?L&>xvf3TAW|YG(cJ zb^lhHzy1T;4nb1EIW2PJ|4plC_K{79G*#urK$tcU9R?>ER7%7nwHDsR)D0q8 zSCu9pG_U(2o_avgPrmi(tUVQAUvd8kS_oIgk%2}CFb?DoOSein6PAdD7yJJGtEd=b zPd4cV%uID>dF`=A+9Gc0`?+ebTdFp4G1n+2Z<-av;0!HCJ7g9&gzIttK4E>3I@ggG zV&2W!Ef!xYI~X@&4DHi2f=Z6Sk7JK=hDNMPPEIA*58BXDrGw2`HQASBDAE`cAKykf zhbRIzgcK@#bYO9fM0MetkzKs>jXa`xIpGe&py?J(pap|HzD{teC;xEVZ=G;Z*2Xvy z6USZogpkkwc?>1)XA32f*^Nc-^#LxusDZ59Yx+{d*MJEQRax{OymNTRSoD~WRf==c zYTm*o*@$01bt8%@9m{y#?RYTP(|;ksn_;?DBa9Ef;%EJpo@6WysDf>soc)j74V$H@ zVnKkNg&;j8Zlrm8K8A>zUz;8-)ow6zYGiD5ePKbpPyf({j-EO1AZGHClbFtM=@1dm zm3-gI%9U7P-Pe*tDa4L126eR`6}LH*<(zApU{yAV?nSU>c@SjXCQF79H)(J~dbD@l z#*cG_#(xWQuv+qySrB8Zk3j;1YvW5ei2aH-Ob2%6teUNKqoZ$&*`Ey%TDLaz{&fZX zRfKD*))lhuFh~FlIXHyfJT_xiGPJ8X)=68$Q*5o)KcTPs6n9X;O(_+C`Q-_+*whA5x`nP$0LuI5bqLbjD`&^}2!6>`fZ3vVkG@=eCe zi{!|kPT%asIhCFro)Jl6WD!#Tw29Iv)i!iQMAh|?XF*(pi(4{M6{l9S6BD7aAa1Yf zfznHTN-n{&_JvPmgx8fy9Ng+7P|Y*o>$5@ zr~Wxh!ABeS2jaNAPx2ijR%fk+cKETM5Ilv_9it&1_J&H*VAy!Get@Q5sYgTjjHShZ zi?y&YY6Y;A=d3p1!g|6965P`M_kL|&iL~dzyKb1<{$e9r_M=?v(pl)aEjMTi67NPr zb&F zm=2(GVvRu$QE*cl;RkZlj+v#WUek*FtRF$=G_O7=3v1Ci^x(lY5iA=9K|}`flfVdw z?M(|)&N!gY6iMIEuzC=*AvoVF{sz$MTdm%+r|AtF4R7lp!Ix* zN8=)H3x<>A!mOQtWtz6wTTE&Jw?0icGr*Hs#WoPva>3`}Q$pX$7h=GXq@SDk=7WG6 z#il@dQ2XIDEqFM+5ijONJZzLP>OZ{f(F%A|*1qTM=xIQzPp zQ`=O`yyaab2%R{ztMorl+hXJ#dG17Ibb<{Le6u<`tnzNO4O4C7PgG+6qyJB=$Om;u zWX4>|BcMY&qoTT3T}sD;6`qu~l`OjTI|o31K20wHLClBOBjU1XGmDk}^Fc>z`V%Yj z6gXp2^;nK!6$h&hb7?I`0jTcgj@ybgiTMX{6)Ht|X-RRuX(xIlWynhSK#i#@sma-0 zrRr9Kl_@vT2%K_0CeyeLjgS6$x96p8I=i^7xqaRJ5%9+!6O~uy=TlXoP2)6Q(x!Bb z5)8WlX{Qm= zvk^AMY$y`heldJyabErcc>taE*WT=g{7`-E?Pd~8E&3THbJC( zfm;8=1FC#=6t3eNB+Ymi5A^ucBcjskoim~w1FnfhGf#LE#buF4!ye`dQL7tO%6w3> z&T&_mu5%m^!xFWZlE&am-8ZFYw@)D-LbrUHTr3eSBO9Jz;RU3PfUBimFRakrTVtgL$kLzk1lc)}l+qv*z z7ZSs;qlheQeivLah$b1)s^tPsajP`t7;bdYeES3Ci6HoM?U8NJ!>RLr39;wf@-lfF zSP&A2_qI&-{IO>u8E`QtTmQNb>9hiujc7h@xFEZnt}B;iyCVa^^eHy9)Q*Qo}Q1U5Ch0wy5Nd)5QROI%jM!2f)K z+)l4r26auwTAtQ=1^&xM3kB~^*5IKYP30{La1>TD8*Ha$;~HiWhKe;{`rnI(qSX6! z&X@iiAomCT-%AvU4HIpN7jRSmeMZ7j9^%Y=|8wt^v=Ds0B$6_heXvXDB&P0Yi^ZSH z_OpCASjLw@d8|J^iz!Q?ynr{gbW&r`4zb0_s;miExy$Y?+^Y=wO-V9)xr4;x)p-#EKGOQC_y!pNB zF;#JIX6f*eSM%%Aq7C#+*E0mO3k3~>FeZSFK252;t17g733h3^aV|)=Y+shyt)366OZ{DApFyZK zXf8lS-A{={8r%9X`5RpL7EdoK%)e9KZGA<8Z!-aWe^1biX5R_7*;?X3az3DccYgFc zbcbAYTAd!XM1U+u?RKlx2H)1|RvY%VNuSYsi;b=IRkh^vUQc1H!=lH{rsE7YS--1R z49OYKiI+PZ&}mk3t^ve0zT$>oZ|}T7=&(fSMEl&oZDLRCxyOiBuyx;AFSp$fo3V+m zl+8eI@6x|rUZp@eYb9qB!YVNuL$)Ayfmsc+mJXHP)bGi2^UH=ltpIZIKZ7bh8$PEs zDZdyCQlDm;cNFh$R0H&qa7+v`BX(+x(vCaiT}v4*Hy}M*0x^Gz(Nlf?6A0qZci(`> zmyjSCwnMeRA7c>Nn@glYWcgrH__ja`#vOrd%mD1@Mc6zbJgnxL`WmOHe3N$t=v&&N{@ z85lY=zJ6O<|!#PI}Ny2~>i7hz~Lh z@XjdO86fU-8zw`Oz%|D=B^Vw`J8f-kCzVQokv3DapYZ(Jp%47o6|4 zSgTUcWzqqp`3%`8A}M_lU#*S0(jmS@Tx@1l{|>NANg#J}t2{N%?$?aJJa(G8`Mx=| zhD>i8*)kvfx7rjkM2fT!WQP1vc9+;BP>60vN!wV+yo)N#Uf<*HpT(;VTP}FE_kBl} zt;e+bMqz%>#22@g8GqLIG4-gnv=OZq?e{~3BGc3)He5wh?N12nBte9;4#8{ z*N>t-_`Zior1VqQ5SHe{3^EF2ib4nXsVd12u`cj0;O9*<{(!R)_sy+2wq@@BL85kh z$Ki>_sA$7j@GnI-0K;;{9dPx{b?zXU3gS3}Ymj5_A79vi(pNN;R-XPRN1@tj3K+39 z#gD^}o~zU~)2e03^bO~egdZeTanpd#qHQJpq9AZje?nN*WTClySR=)=gYqWrcfug@n#qYmkxX|T%6X>D@s7gOh-B{LPAWhG z;j_L4uS=P5CDqU0CvQ@=Z=jaR&jf}qkQGN!UuWpQBWXRD&g~3?>FX>`?f=qUJEzk* zhSdhArjFS|ap+us%TG}TgT@PiI2+XtXzBqZV9~S`PNwr5!qUfr`IGg=lWg{ z#vyEaAI%*5U)&BXd}f@dBw(u~J>8=0WtrGgn3P;TWX$urGn+epKfG1lg~40^(xG7v zUw3)cy&KTQ5xtGrvw^akVei=k-(GrsfAQxs{8?%Ku$ltO=yjW74uYZ-5xln zha$N z7>I}|RE^#I$#9Kz6QkymKOG1=zOfT7-JpAHg`$c55m1myp&H!cQ6k|Ad~JHjaQ5iv zM>|p`(%sN3>8(0np50g{%u|NR&x|#od4}k7*M<`}M<{!>_b+$+nylBwG~D((TH{;! zc$aIrGPbTadfQtw;wH9aVm{*;)=nd|tQ;*FeBY@0xXxsJk~21!gJ2nk-i##rgq9yapU zr;5V;N@n;%Zteb25wD{-Cx}>(3xJICHl#wB(PrwgR%xY=G_%oA;xemZI+pzII~QYQ zgyMNQwca3Lzn7&|RuX*+99AKlyd)5?)*HBs*8AP&;6SfpTRAR)gz&Sp1qH*1DWSo? zchy`o$P@K?WmdKk_6I(k7-MLBz|#eB;{eD$eP&+j88`8T@8s5Ui2(iu#rydj6G)Wv zWXiu_+M~bvGOaKIRM-R6J^@d*X~nrd+k-*^Np1|J1N?JV9q+ks245 zH|ynZUihrbmTbAg@^_1O|J+W+k6IX#HU%nwo-hyF>TUi3u?+PM*k>d@ck6&Z;~Y!? z)1}4=OfmWUbiR1I7BIqk2%!A2DfEX)IP=+UknMm4{^wcUWTDs(K~zzXpy)?_kXs2Y zRAbQRvZJu`Sj=sN@G#ls{7QGy3r!oJIgLM}Gc2Ei0+wi|RRU2lZ1=dVF8S;s=X=Dr z4t)hEI*>B9_-^fQW-k9V6|lxPWHAdH*sBVlxcx=mtAcbZW_uAc)z)-Sp-cm-R^Vc~ zQnXba9Gy_ArJH?~ZFQVMIc5GL?gD|?For>h=#Vq`W^(w=%Gx}$HfQp`4+F^01pq+W zfIXB;OWye>Z}cSt6d*om-ex+T>YcObO|{UmcqzNHt|Y5wEFnjorO$WN?VDCG-(lJba4EfRhk7tOHkbMZ}7>gpITFxhojBi7fp;KYL zEQEdC29QVpYrLrzc^K44zXD5yb&E~U`n{DIj{{!dy5Hr_WxeI&4yHeJ}9S`750fd%f^uN`oWZC~lsvoe2XW$<1Tt8wC3S;P0S4 zdp}EuOYbw#GaSqTPFaGuW+SF~%roTHSb=RHKenqb3`M|y#vV#nEu{_X+>|uW7_(FI zL6cA*DBs?H+Mum^KNa4FG-@bLacdY$ySg!n8?FtnIImgjTvio?#W34iBzNEYk|XJ4 zW)M$910!h{(~s}FVy6V67|%5{qJuF*@IUitrhTr0U!^VXkDPneWW6khUbL-8^5~WN zQs8;X)&=jJjdB>?=mzt#&*&(y-asWN&q{>2XSPKOu=3w4c!@>I4{+^9B1zVJ{afHC zl{s?sJDjZkq~E7yFE`|wh_TsetY4X+>abez;~bC#bM3sKW~kWXk?gtx+IuT1f?Vp) z!kZ2``$2D+miodxMCdsa{5krTR9B^W;?HAS80GruWTDNqS1p^3b_LR(622D%t-2+N zymF|^Ye)=mtn)H>;)$gOq%f#$-;F|Qtz%XFb8G;3# z@J(79NWspUUxT9qeh%-C(UEu6rdG|B4g%?Fum7|~f{>+nj(($u^g$`lfKG$@ zdJ24PT`^P3JD^{;|0N_XRzL)YpNn+hmqw0lT8Jc7PPFed)$UWmAJ@YmXxK0u^0y$| z?J4txU_J=H?pB+VUVJDMyAWdgwAG9fn&Q-WEA}84O2k{&7PAZBHrpzv{oq;2A3{bQ4k&|U zGg`mX9G-Q$D7_|Q8)8dFSN*o%^ZKFK||8O187pXpV?OMDjBGg~RW4>UKF zxmcY~&2oUZ%b}ohpZdKeb<_1~ZE`4#YI^p+4}(@Tf}FT^OpTVP&~wvO5avh9zy^m= z2qI#_AKEZ0Dz&(!pzp_9Ykt<9c(mtyOnpM`!F$B$x8J@Xd(l40FSw)$7oPBrUqZ#Uy0OrUox>JXN9Hrfx?;a(`xit9U$C-aBtsxbI7SKQcvrzu9Dlh z*J*{@?oXLbs4sjWTp%85oSgAMv}@3CDXcRuN3bPgP!<8pm~iaAnQO4q`ZB9b$1+)H zMs?vtHxYR37VGBJR+=H@C8X62w9+PC1IJvjsE;ycy?RMYOk^0k)0; zOV6HfwaDNZy-A$r0%+q{Y+R8ZwL6d#7CbmVPaYen?*D9A1fKFe;Ix`eKlR6rt&-lH zl8j2!oZj4aaF**mr0Fs&^fJVp=e)gm2X(~G?!Q@s@CNWv^k&)K5?=oWp0C^ZcKc5o zZ3WbdwxBxH8m*^zWSz!2IDfvOX8&~{<>Y46$~dNc7kRyD;IBy~?4|`h>XrsnNEW9t zR~*=P4}A;MbnFYXoC%*;Op!(oKl!Z`Zjw6t5h!w7ELYj0XdOZ-Ay@XlYLos}Lw74? zjnF1=x?BMpj79E8BRqmgwoQlfY5QN?&RXgHoyT!Lv);0EE47HG47`ONhKr?wBmx6& zpu`_~O9e%gM<`W1KZnO{lb%a+kW>U0YB~$Ee!Er&%gzfet%~6#&C^;>|9pb<$LLiZNx7)9~V94?Q z`!l~6^?oHb%6!p{Q88hp820APNFXm(9`OXrm^oRLo~9Miy!tvc;N>uT_T&B9 z*STl)@gQN-f3GTl{QYB37qv7_pxL4YFkjTpTa?yMN7=@?V~F4P9YCVl)?E(@v1Nh8 zIjlo6gh!t)+>ASIPBQ!7d!H|X`CZ`6?7HsvI$e?XH{$v|Gq(x}<99w9Ne`)P{MY3|pGJx3+M471KkCgqQtL71Mekz-SL;Ez|~Ir5R7 z8kKE4MNR72RDemfN~q&s87iMn^#G*M%IvGWnrZdaVwn)!FZKR^;yMU*oO)|ijU})w zov2g&ga13^OZjL0ndZa2)sn2J*9$z^3G|m|gXCA)orJPO_lAxJ&U2y-EA=s{?8NDU z!1?MZu5hqrgo||>8tTaJ`VgTmg4r~CtkYPDYJuzEYf9N7KTHQ>0q|-Wc+S>5WoK#w zz6mvBip`KS;Z!;fY#T7A;i&rgu)(-Zd8rGMcj;t;zC7)QSCO+-{Th zd9i$fmZ3a+#F|8}-_iehR?Aw|^~?NUez!p<@NWI|Uv$o*E5^1t>~-U?8>^>FZ9N%3ubBPA;coXEQR^ z<&(xRp)%?HBUeDav(7P}{p!>B()Pc4scA5ps4OGQ+p>P!xey)#zJC2?qAVl= z)~*kxO<)5)M^lpnCQsdxifQK8wYe7|Z;sK)>t~%@JkJ8Ed(OD2=C%N-n?WkzzbF+v zz-||)Az*xmOfnFxZqlmwhcL10+_L9?N6qovK&A@K$(7Hh=zSEoe7BcW=#ODj*oBfJ zk#Iymn^9(5JJTf-bG>ICq&v!uWs4e?cJeYcW~Wu1{4DeP|xx1Aq5g0GU~ru zE}%AKRRune0<#hk%jS@xCmpLN#7;!C7MbNxtkiJgtFv`xmOZf>#}G}S_?PU# zO9Wua%IzY#!$_g=D3wUbQCbaoo`hf}+fSWWe%~+JD@mH0($?}=_Z#y}wwLUE*S@uj z=w9(0F)tVKkj?^Os}H19+~P{x0cj6~=@a)A9qUNTiah7mnXwr}*-47%ji^cY#bdA;GOL4 z2Tc&X>fc`ZtKql)Rw*EcrE6!OUqvE=-#A|8yb!XILnjTxrC77tq z_|dRqG^swn*s@;y8F|Km9*P4g^r^^mYw>xu?c&QXqtkf{za$o%BoI_UP~aKGUq(@M z-^oxpvDTlVrgfo2F?XAf`+Es7AAdZmjnIaHU^w4P}J9y{56a)78v%#b(GU&W~qF93<5EtFS3@Dk;lkxN!{?5{D zMr-L7)xp2k!>qEtWHwXI<*=1!$hkG=Ham@e$WB3gx%PMqDW9S0NvX&?DCP@vm%bNo zh#KqMT@$@r#F%kgeTW6z0#d^1zr9G&k7G%EsZ8+4w&T%v#ewd}U<+MYpVHCiG~iGU zqfScB;HX=oT{e8&q~-3&R~&L)?0g2u`=O?piYRx=x32G$cx@5HIww-=hu@uH65u@; zf!@;6Vr>NTA?|U$c0VI{Ss&W9`beAqZs1~>LWR~Dyw2^p#d=*HvC9q(AIz@Hk~1=K zwCTQ}46!D6b8&G|RE*VkX8bWyp;@-+NiHxp&9}wvGd6orf&CJ59`^PU(DZj0nEW(V zn+DgSWvf1Jf$~sNVvR<+HF7ZOhq=qbrd{?*vcmXs^aO!L?5TuNlO;}W_r(7~B7qGa ze!N{pgfj$|0B}SiTUK4IDv}tI$Y$(6o7A0Ng$^NA`{c5oI`C?qFlzchSj9%Hh;sU$`XFcM4eFczg?vZ?7cKzaJ3$}Ltf>g zlW!rZc$~rl*(a3EzdGvD3 z%5f4e_DV_=e3((nZF4jEybmUIJ#eWjwqenMJC{9Avlq%tSaF2szU%@a#p%PD%&G}a z^a>+)cWGaz;P(#n>3>BNS1I`s>~J}9iEw3{y1|>oRWE;Xx*DlgiVcfpCYBb%^%Cmz zV(|iDk_ly^Tf>nYR>IVh&04xffGwh;T1DG?+-`pDG38nue$+LMa)1pZ4$4>?_1i4j zmX%e79@s%oJx6(XOgh}WydKh!`G-{4C-r*7->KVbH7EKr6ZaITb>W6MVRl~>e-Ko< zB7ANgQ`=2t$Yuo#vaFI{(a_+`wEs1(AJ8YT(r=2fb^O|RH4A&4c;U-uJ@TAhCY#5g zV^XbPFaCp&`H(CrpYTriv~dSQOVby8VdCXl;$z*W5Mq)F{I(TvO|Ey_1}E|{%@IIy zVF@pWW>uKF{NL!VCowEx6V5*kp) zY8VEfaX2!n45sp)=fetlQbUp-|Z_AJC0$^yy^0a$%cjvdCBh+HR-KBjr|>%>(z6!tJ@)PCK5)UjD>h}&lp|ED zYoO68ZvxWC1(RXR_ICjlkxTv&9yDRj?CvT_8{}Z|qo^t5e%zAojZy~kohyEtS~|*M zP^U)Hy4l-9HM_R;e1K}k@$y=KN9z4+bhV!|{E{HCL&P8m0=gf|4lkDTl^ zMT@o&m4>+r;>CVjQY)7EYQZkIoy58KhOLzlGurSGxvffk*v>cP&zBek-XRT$^mrm! z@o;#Qc@)kTGWtd0x>^ssMXCF@2aU?JCtIva0OA#2Ux0U1U6-r9Y@qRsDY!r&=l$jjI-=xjP*KJ+vH^K?#q;Q9pAuiO>djRm|xB|LW; zo;5*PJiWr`p?+*Dw*pRdCBWh5=7SK@w8Hj#-Xz2^G!ab{-D5Mfnj#0ICD3kh@O$q_ z^hNihF_zU|cO%Gt3gp=NAwiOXBIVOR`n}h&^pH?yqzo-Tjo?&&=H>eJOm4)ExOFl% z*o?wob0<1xGc_ynie}m=*o?CM#qCfjS@`%cJIVK-og;!|wqq&>uTek**L4hReuc^M zP~$z>qV@%)}zk|DZD;WkiRQF-4UVmvgnk4rIQF(AS|Ux;K>Br&$13^ zGNjh_uZ-Xhz<;PoIJ~=<9o#iYrb;^-rVCt*3{H$eYx)!v7)4w+S}i5R9VJ_Q0=+mi z1UG_=fNV{>?TOwO9gX~jZ)b`K0y{u7C=O8gxJWq1xCD5wrBdFMDsB9k)nLo? zY5PxDpa#9iz!kKv9p_iu9p$F|O97FTm+2S{UR&QrF$e6t?D+tm22iJS#*9_dO;f5N z6sXofC+h4lSTqn;N;ii@0Yuo=V`(;-mys`R1Zk!^3{zY?_$&Bn(X51agkJ}B^JCyX z?qAthwaZ%SrS#&j_Y$|}Zx-0C3GN7M4RWO{E+dF*w>s>0@COX_4eh`#QX>?jktcFW z^rG+kH2xeJp++$Od=~Y0)s+gUIur9SvQN=9$^k1ELR76)e`IG{Op~u~PV86Ov*kb+ z#iZe_|GMPyz z#vljrANTD7bQrGKwlRc^64_u<=^E0$Wf=`hu59*|v3jvdcmu!PpVABWw+_$Q^O zX-@Yfi?IZ5+4cM|Z3HTg*RMv_Nnf5objYLpw)4doJ**$^z26QLa<2P(x5}bjBGHrx zjCnQpoxuuSb}jEmKqp@(^ejq*XIAHP3#Fa92d}?Kv`}1(j71^M{MIG6*X>-@d z%c&YOl|wcocNFTMZ^NbITTOnYwT_lCEF}wLa9z-^G7}G+!y*fROM~ux+`t1^(2D9QC&~qN{sA?Ny}$}GY`8sc zU4AZWyK?{QUc#$svnBW+9$Gs&`>E*H??_sniQ85Ye<@h!`Ev1?K1nZQM$W?cBSx(8 z4a4kJ1-f&R7e1<9JXdSPYY;9F#Wkt6ouN+-f}o={8KOH-hljf-m|Ov6*X7K(G{g%E ze!|yk(z!wN)RD}>tO=?9Q}&u)IvCZp97!#e#$34qYDw?w>cToYc$N z;N-%jZNT47j zFCHxBg?i@A_AUGuMwV0O(8iQxKkI0vXC(yKt20X=5EckPApZ?r3Ej9DZg%26(&`n^C1J{$U<`o;Q}n_sy!AYC!qTt=nx>1atiD$9POcGF5ZhatzM zBZh2j*-GWk-Ty{E77Eet%HyzOLPn`~Z%wMvGJhfHDE3hxJHwrxnZ{W3+fx&*osi}j zN379r$h*@xJseiUS(88RQmTyi!z|p*hSM^686B4@$Cgc}gjyX7;*f+(7uxU=8E?D{!FkZPJ7{b|9Pirsj4?&B;c zb*rVM?Hz)iQKb+~Sz&LHf-<;8B{1Tb6c{b%f2TQR^B-SCXJ6z!ZM&8_+)g`fcD3tW z4sNDwbEPR*3Fh0vDZ0eDINj^h-KN!biG?&R)tP`Zctjxn9fPatV`Wq<5|ef(gOrjt z#Dj@|tNi4y5-kkMh4}IP(QvPJC%CYH?O;SmIJl2-6+wP}fI=7QDEe#Mfq|VxTL<-X zOF+{cN9M=mA9L+1Y^t{#8mGAj%p-H-QA{YW7N+ge7fwB?{IBEW+hV4}E)L>R7Ihw|-pSCl;F z$A@UZ>xa1&5$wp+#Q)|*qa^YoB#VDvg$aFM+T+RnpMrx11=$%=K9lQZRQ2(CmD;q1 z3PpYhrf$XfmHrOcoql1 z{3zX_2ut4}r!EhU?g)vCav!X&om^o1ms@k`m(}_GJ%NdsJDV@wMhLL(rPu8aia|)%>Qzb) z-BTLoCz#|Y6@coR4XFOy44zAqdHc3bPl)>5Z&@nSG5CHJFL=@C-;l1;Z^wza=2&%T zRI5Pi?&_cZAW{4lBhaf4A?LV2_#+^0k(BmyLdNUW|6K~ZpGrQh%cUCg*qUfgl%YJd zs6X$Z_%<+ppWY>;TJRo-f<~rvikhk(kB0|&m+%(dTgH>@NA|JrQfg>hyGH|tET#g;4n3GWmP<`xBeNgDzaO`6v#mGibYf8?{ z%YE|20497c&sBxM#6%>n2pq5KtzZJtVvKKbS8K#4J;W^gnW#vrJv&SW3Fw7gkJ7If zh*b&gqHsJEDBPk9ZBjVUx>Klzw)wnY-6t5~eQVRPdRMvG=mhqZc7_m#a7Kai`yPKe zuKVMwN$Kw0a~2P{m#twP3lV8CgpfOuVHD@<5%^Nye1sxHl)A_o*bevDH&J}7cl#J| z`0!Kn6J}E(`ItA^@c9*cd`yIHa!7nX;}5g$(r(MP>$m)Fl|7Hj{#|idPZ8K1d|*_3 z%}gn4ej|0uPF`0W_EWe@Kit+k%KFS+82#bjLs#{g2lG_0R_no^B()X_{ zuzs}D5|6I?kx>i)=k*W{g*TN0nyA)GEM8^iJ(6rR*0X&{QnFaaW^Xk6DQahac#H8( zT=)21=)jXo)RG1%4LK&ejJ}-l{@qt!N+vh&^Tfsn|0|Sw8ZCsW`*OB2ua6_O1Wr*E z@+qVsi;kse|KDs-_(mY@rNMq1f;P2#^04>OhoqEnT{{&u5Y(jysx)X{DF9<+eSfzO z_YTtY1Gq#jIk(a`^WmPEiQ)QDQ^gQ(Ya9HG`z^@=aU9w)2^v16{EJ$7Pb2FEB?*vi zYZ9TIhh@?%>J~e+ARamk?i-3nT9L#R-hH1*r=cDs8Ha*P^j0u%`tQI9W>Q7-#|?Bf z_?v(L5h*gPOwOVWFS5bHz{VdJAx8($_NU>Vc~+%Zji`7PR%e5d_ZfNbwuJR`)_n$=h$%qcXd0LYr(3OE z9rot1<516$BX4M93qJpqJ^@Q(R+MSKmFf5>w_^Ea6#MX3;@Zd(L_cz?p}iLJ&qBKN zcCkTnzOefS2gZyqpGlj`T+K_5X+qTwZtiXx3r!6N?UR%?9Ld{LH=ndFN5IpisYu=2 zMAv_h40YCakO2F4SZJOcNK}E}QE)unpV88?$U+=0% zjy}m6G0MZ6yemMXI=D-p1c-tdg#I*5 zH2gh1_sNtm*13jzCgWu)+@7U{@W4c|6XW)QZnn&&wWzzCjj)}{cL5`q3L}c zsp2n{D0osNBmbEIA^V|iooDgs#3Tky=fg%QQtCMp$NA>!M&bc|h36);S9G_T+6W|K zdH{AVV*sc6S+nheOHsyjo=XWM`X{2%+5f0)$gEIZa+-ye9jZ^}K$&sBJECLf-`rLw zvxKU+b^PSn;yjqIa@iH)4}ak9{%iT6l|GQ^9=3WqzWudUFs$@e5_5xm?=&l)Kre~R zwU+yzhPEA3ezGFj4VQQAVsWTo^-c9eGVDL-2Abk`t*mo)yJ-^YHDAIhmFPLNPVktSj# zCAY`(`F5q0e06Q8Fz)YIuE}uS@}@xB8&I<4RSpH+To75X#Pf$G<=24Pq-INxn;36f;`F1o zYPN4~wa9`#yex(73e-dSq~9_T`dNemvKci$?oVRd;Ls?aZSaD#Foo~1cBNyJL&R)} z#FzDEU#HDmOS0dsC66-CdYIKYwZCo6WT?KA8%2Kv-W7xQCRB^)jUssu99wGds5_t$r6YbH6+8)3Y%h3@R`Lv#Rb! zEoEWNI2Rf?7|0HO_X^yYD6LRINCu-`{lDhL8qx|ZTI@0>{K@bZaPW%Xv!wd7F=iE0 zbg@N0H{;{?c?+S=A#T$o9TTJfsrk+7CTz~|BE=)^!j2>H8sn?3FJzw;aRvnD&>YL8B6)@nb3{&WlKFf=sq=Vdv`L@sviP`e+p zz%bDKcl6DLs4%N_o z{znuE7qibm`O3J(Ug?hn5I4Z*Z7ga|ayUDVjHuVt=VDWtZWp%yi6d(L!puIzg!ML4 z_^>g)xchA{q1W=kOx|A><0P<$=9;Lxk9Y152(r2P1v3%5ml;`5W^^fOZIgz0GIcN0 z)Oj`jZU2%y3H0Y5o^8^vtx`<{th&&r?Lkn3NXu%&l>)6w&?wjH!?%S%R301FR9fmm zUl+Y9exY7?ey#bLPKqrq#e{qkYXs-oL3%_P&2$KFnI{Jdb0~S)*VD!Gx~p zg7)b~4U$|j)WLah{TuYsjm0;IOSgxmk=Bn6o~pjI(L<027ey1&?9v0i^n;QHYd+|T zoHv6gL*WUy^~r-8*VF$eLkzBY#=717-@n^^M^YBPbT)o^t;cODCHl)DR%=~dYYK-_O`nUj8 z``abp#WWOi*LEeT!QOUvaqE<7bSaS)4&!HBLlY$F?~bgJ4Tt3-`kAE-$*RxB>EpD^ zf}W@BQCdg)17fpQC!T_fkVm`;7+>b!3rrPAs zt%{;e1)+T;Z~nkZNFwA9d%C9a5mVLk(4Q)(Y!9H`;rxnXsGAiv)kZ=05BH-nK$HW~ z&JX$geGq9{r)ke4wQHtkzDDl-EGt3PTK(VDaB)YZNA$Eg&P8~2D2W+Hkx1G3fJX&l zoiC;nXb{naeYLUALQe&hR#i_L{SXOOIU3Bk{LQ$`VZZ2Yg?M0lUkQ4<3;O}`zkfXP z$;_b*GL%!f3yZU*fc+5;!2cXtTQRL-M8nhP&hVfIWuDvzN6A~1 z-cSd$=<*FHkVP9a2u>h{;cs)5&g-P22%)RQ_k?xuyd>-${K}QYvK##&ZvO`{Bosrs zIrH6xhrfH5HV~`?WR*{HWf8PonJQ0+#CXu#(#6u+-Ct6Di^-*;-b~T8CD56g61qVc z9w#fLq=YQ>E_!PpXDgqqYQ;}~#xuwugB!tzMJkFWr;$Lc?t=CT{QCQYYdewcA6ny^ z53OGwLLPfMA3Bc+nQ>AU#opfvWTe;JGsq%vDEsV!GNNQT4au&QeBB?llkU>q@$e_g zW-j1&9-puJQMP8dFE_FuXD|jPuyQE!rJyof8lw|Q3{4Ax!t4n!V4QyRjo)CiGpVO$ z&c0I+@ox#De`TAVg5S-PF~hNb6M14J9ntG2it*llQ!El?`D<~620m|t_OZLuew>c! z(ARkyZS1%#k?Jrsiz->Q$uxxQ#fbb+_CgEx;$qhhJvqi93k9X|^>&KSAg_ZDFvIg7 z;N2^tPasCx`h@|aK5uKe+&}ni*(knsA@}=GY96K4t^_QS;?0wAmfNl9(1w4!=iooy zlRcRynO%+6h3s2YoRqAFd{T3K`jbsV3byrj5z9oJK^HFZFDn8?`PfUAxuDE(O*oS} zvadEz@usD(8tfFUBFZWdig=Q}fC#qg_PdP3X@Uu|@(!9lhq?bIDjx$4O_CPaePwcb zl1~>P(Pkf0xx3m5{_NxNk`Rf0&SI?P>>x`1>kZ=jN>X4PRS8B+nwmwvf(hnF#aN1d z(v0l0sq!sC1~>kG(gjA+TWZl>65`3(Dfpn@=8_)%=uilH`Zr&{)6c3d@U*L{j{E$7 zX#@qu}rq^ja#I)t#8t`#k^2UhNVG zZ!7|Yj)d~JJ55aR9nA_LIHgmZp?H=B|E~Y{9~(&c>Odk)IFR`p$eAOhIo9x<7@6TQ z0|jKX0QC;q-Uyw_{76Qll2dGl{;MPK1~T!R5JE|>)02%UsBE|MsSG$U<9w+ajJNyD z_JDa3V;jW5a8832D7|W-dpnobFqWLAc0W2q>46+mN`Tw>4$Sn|;MBl|OYUkf)!>iQ zN#%QpN-C=wm4GSsh`N8Ul|Ytntx$^+`8f1%w{asEbFFM5oc*V?kP}(?Ezbw5N8VLU z#Zl>2|11Ok;x7f&`T^gECV$vh@#mIu6N8`X^%Bn_Ou3>b=1TqZhg>TI-Vku;uO^qX zJMTF!`5pdx!O6a?%A8ya3Wzq2rFc@4%&mj8PDTu(5O{KnI4OZoDRSS@zc03};?Q<` ztkl-^P2o84wwUJF@L}i9X{t<%-G!KzZZV>LF%;b5`P8mpBmT;H89>3Ikq6bq*`12S zgeeqoJ*>_{^FFWa9R7tUmuQ;g`fm$=GZp+TwxQOv^cz64Q~bty0C<4URV>7@;S3cF zpf`O-+_I{vz!-D(FhLSNW#U~VyI-2rVu3SZ1V?dVd(}!fY!^3`l)h0R>)}s^i{wh( zRXIC#%D-U^WydBkBBR>;?6`+Quhp3|v8XbdJf z$ZTJo-j6h^l|m2d2L_cWLcfX(!FzMi(SKD?YW)G}3tcmtR6T9FW7>aOk>0h>%3|`yri%UpzifznPfp2{#E3I)bL#D0PJ`~=dhaTpr zYx_F}wS4j~f8UdvRFyG0nJu&r6jc~hAl+kbQ%^MXBg(u$|G<1>Jtb=oOLwyhw~UQ{ z;z&&4q0Lf7p4m2anvpu=r)#r&?N)nqUm>{jt~pH}cYWWNCG+zR!-1arzS<{i2b#Sn zkmrNknbND{4hYe=zUSkmtCu6~(RN`G9`>2L*SAEL9G0IQlD?C6yb3S(X3+p7DKf+5 zU`Pp#1U4n_B84YFmc;ECUi>4-x4s*q^G~R_GQ#KI@S!``q{iulKzYdF_RpFXG4^Y< zXVIjeqv)K+7#Y8i5Lsxth%2&5$Zi~^X286Jz1eZ8x3ohAUykAED)G`7A>gnWNad8x z5Cu8)rR50=mJ7wqBHr^!g}{r!`SF(FYPZD^`DqZlvCv#MciG*?OWq4M1$2Cw%oxwt zn47%}w#qbOjEYF(#RxivNZ=Hn+gj0HA^WpfBIsD#|FU1MebbIAc&E1=I}QkFX3#A zm{PUd;tuN=Lt#HHTpNCm(J4z}SaYBIqfi=B&x2O!Fuw1w(e9ui+dhA@8*gjrS1tIw z1P#jwq%V`|52bxEr)v#gc1fc)KRgwX|M~CqQMOhkFK2NXJrvJF+N}~izgVmn5xJlu zs~=O9m?EI}AD<-%!AaI}J={>Zul%3N5&Hj5cR(ZN%|Jyqb}zVm~>23V{z zn6JF+pWp+Psh>BOMn*?fneb4=Gd8S!1kVQT_Xix`Q_WCzVY&Ou*M|kY{T)fv9>p#X z_OIU*K&WNV&%+x)l}y4LEQq) zjFmAK^qDlVJZ?mj{kNTpaR|La_Ky}5-}{ssekzh4h}w`kSSo}4i+<^}7|fdyT$+{&Z?EJR@=mR%*0-*^@Pa# zm<1=1Bu}KW;Kn^bvYdczf*`J^)wjMKe_$0)O;r`kE$R4Y89Dd^aJ2?M@dki`aVg(G zVTgy;W5O}h6S|nSL#O&X_90}eKC5n!h4t!Q1)o7v8{MzWn~K1 z#nZ=qW1jWw{hA}?F7%07tWW=NwQPW{^-qdxZ_$AdTGBz{R-5^fE0TquOz9D|Jx=Vx zE*NvZJn7kxmZ>|&TRCRX!De+7cpdDNaG)TcCP42nSa4^7yIrglmd%SjCR<wT}3tjjn5XLXPgL0D(}E?z}#NTVp^Jzl?F*{34#)aI70uCR8qPJW9j;-f|~ zo`hUT6os(P3;)c*LTZ2fBoRCe<`x&{vXWtjp{qzDI}CX8P+UZfhmhUk#92tjwyJW5 z{!zh_-M|+L2p+IbL@7Jp)#YNIYh{vb1(;M0)f>S`5koAT9vBg<^$5hE!Q3uBwqI*M z5TOLnbleKv_vh0r))?P-Zz609>!$AgV|2oaZcP5}A7A(O`+NOqRpT*yxjXavG8Sa) zp<;v;GQX53Wi9JSxBwQs-f+3+7#wS?k`1G%q&DqMv)V_EDNX#*gjQ5jU~v+c{Ztk` zb?w|M1ZRc|Jq&Hi`VHvXt{gAY_bOQl5Rp{4ma4yl$p5=e6gL$ZFct{axVDJF$3u; zCZY%2`kguP!V=aBL^`z=KVk5Kq~ENtV`HxD?nho-P6LR?Os(F-J3KH(AjbWE59{Jmh&ZaIo%v zHqd0?cYn5EAnp};&GCAYO!#s&Sl$g8sV`QyThZ#U!CkP!@t)H*sMlMW368GdtK+aQ zTFEhZ=zMx2t?s_O(B+{d*4)~f-V6WL)BP2HU+U|9 zjtJwMyL(F)maPuNeCh0|-C~^}<7zMaB4*dv8Wd|P72<4HL4~~2(pg@;+VWin(Q)&; zj|NQgy@vS9znE6+ofp!Ck_{~6kfTlq9pAI60fn3gxS6n=R2GY+aHi@X5w>Hu{@kr# z8iaNCExbMxnZp9ixqdYpfZR<$jTa54VSE14BHa*1Cj!~ID+k^CL58a5NI|bh68NON zY?i&bev!j=(iwmhw2&Vp?GO4+ju;B8Q12#___CQZ?qINrVpicYMlulpH#x-%iQNq& zfl)=H9PkIgI3kJup=|R_vqgNG7D{`}EIhgV!-NLj-vTTdOtVfYJ@Q+Huclt-?ybu6 zWfokX@$4nvt4cw)1piR+JI&9IPMnnSRvQar9{5a9ELwC9OSA zrF=1)WTd0(2cYstC=4J+-9o4Ac&uI?FNrig-e(B`)e{XrB?980elQ6nJ+;BBwFTm+ zFK;um28H+R$^9aFahm?>i%#^0P)#ZmHoYhPi|+QI+09fA1N@wS+Gb8170;~CK|ZE1 zRqO4ut|q6^s8C3;W(=iZ3Z)yRwdlu}E%I_HT7-Je^pq(%WRVPtGnIIPfEopbek%XO z{yM<&CU8%>!#CD%m-1UC@NV2triU7MMFHU6p{U|cA$Nlfjfh}#oNTB+U~PCK>68YD z4N0=o2m?+DM-PA3X&8y=&GnI9WaZt#U8Z)2IG}T@@WjJR^5>j_#?ic^^_2-C)uXNT zAylA9IeVRJGDD!yaVISz+Dc1#`AyY^nC7%TU?R9OxQ|}y%N2%A^J{ASL#D~M zY7fu9m^L03X(~HGH=yFFtM^9K&^;r(*ndlbcV$}f-pu6;r;5*^D$@-G_c26vyyxW3 z7G;uNK-n(vnNImWAm5PSdwD?k)3W?49>z?%ZR_tb7kD}USu^SU7#4pf6oGoQ=NT4B zk}t$%lVJ!EN~jJi+dMMYNqHZPEaaH5n8o<>?a6nFn?|O7C`1L!e)=nD+oi}OG5Gdb z@yHLR#g8-S!k;rvR@kf_I0AB>=5NyR_tW1;f!YFt#|MpZW+MujQuWzHr@+Stutt=? zmyY^uw)N?a3gRV1;J=e#BW$ArP;zAoiJ^*9P^FdcD<@b0PQ32$hm1q-WJ(NZAEv0Y z@KnXT_i9aZ*GVK$jzvME(XLQgRp5`oA_>FYO|37q^Bvj?1v zAs9$2Hop~sbIVZLQ~-*W(6$t4vpbKgN`JPr5X;>EeYDdk41J%%U<4N!2XP_xv&`r5 zN|~Cif33Tl=6cdpvypLHK${#Goh53@E@(>4e=@wqQVK!WnG(G!u)m`c5j?zbg{g^{ zwU;u33xkWP+vau6@zeQ5&~s?fJylX086>7%t4}TCCS?EGkvD`K#omWmIc~7ZO5Z0Yu1^XuSXhFRpJ(3#`z7 zG~>(d=B^Fz<(?%xf?r>Dg(Q9N&31DrhtTgWt~awkzJ0*9+4ZMHdVRVnc`<+mGbm+# zZa1n`a@zJR9-#kfXVe!{Ap<{oO~z~Z`L#Oa<>9O(Ks1IJ9rhuX915bNj`k&2e8{Gn z>e}#v&!hO@JS@RqZ)g7PTTGAgH$6fgsBbL z%YRMp1)^58ig=vP-g%5-%9AM7Pg?CqvK@(w`X!S2(h0`!Pw=MD>s2XOP`ik5pGv+q zsB#9RiD3^{>&272A&j}}{$9QxsYi>|EkG`c2%5XV4&riu z#vr0LYS$=4kATk%#KcoX+S^VZAGcaQE9>cLm)vH1wfkw}%{JL>tURn9xA`T$=1%0w zG=`F`QJgluyZEKi1)m0{Yp$<U2&n8ai@)A)z9T~#c=sV4ro_6PIIF-%lJuo|<`n~2nopdRg|Qt?bc2Q_ z&FxZ&vW2-s+Vb1KUd(}CLyXf?;lK5k#79ba1VE<#o6MPwUUVLg41-FzFW;MGOj|PgIu`4s%D} z$R-u?`9Ya)i2#Z3pNi~}IrsO5smQ0Kxx#dd(k7szB-p*!QZKTr#@{`5?GSn&vhuMF zISThA1O<4_mTyPszjye}hNn<^kYKq3I@yzQ4}~myi&hF>R8DT?e(rzeVkZ6?w^@xM zieqm+ChWGxtcv`ZFUg;zu5#A}azrHR$|mZ?!^r|Hz9NsiJ9+UJBS>~znOP(b4NSi& zL3QBBFFb*%80eudZ8E-APzW6B=x4n|s{iH&0Mhm(9)_o*%vcilE)wB!1P+^ur!(LB zaM@B4l5FkI!JD9k7*Ph=qa+^ zn=ky?!kZELo=5ceJLB5*W?Se;P3+Z^{u|c4$)hH{U<&0OvdS3`5!~-VJAh~vsx;u8 z$vAcaVVI0WJLq*e!$*0P1-M41tB`|Dm-N|7Q##QDrfgJ+{Vk+d_J)G<&J!htlmh8i zc#~~S_i-ao1nBJZE8XF>`!WDXz(|_v5HYRxZ+?#~ip9>2EY5PrYqxd!8G`YwFmzXI z{7(>W?qB$HpZ*#V(gWDs!PF z+OXeG{x4-!;ZaX!h`5v?$7v@2BNhP56yUI=%ossd5P`=M89$fQ61~dvWPIC7d&2@l zU8Pa{wN#2C$9#5bwK)k!nvt=1vS(bHL9pNP&(G&M#mwTh&Cw(R2Oh>rG&1x+c~&0k z!Vz(4lov5K2bo5A@SrkTCZC<>pH&NBCUGjR*I(Utcoyo>Ji6(Id`tXTE{S0}K z)VV!7;VcYrSCQjQ0%S*O@VWrItM7Y**xo`MW$5BS`CREZ<1h9bW%kOLpe? zLmXIBcA+p^`a3aW?N#B!b{buI2>_bzy4REp1RsPJ!XpChZ{rA)2u*`q*6xnu*vLc2lHPT`!5sB1b_&2Xo$ml(_bv6M6MM`z>TBWt;#;i0)1R1ZP)5g~_uT zjuighjy_N}bMF8%J`fID>idX1nZe28cAK=iq}JkVxlZ1#L(BfKjAd7KGcIn`F?WwWVb%&|CL;7D3_k za{Lnp*vvjZm<`#1;gjRO@T8>T%f^z#CFCm+7bV50=WkD^bmpP><)XH~XBzk6UFN28 zHfFm%e^=>wisJ3@CpTK?xIq`rxp2d%&e7<5O|zXgN$p1AjZ^sLpIIBJ)-jWcVo%V<%YUUN|wvvGV8Vn6C>;C)0( zAU50$8$|^!afkmXBflT%G+4;nF+Awt1R=5MA2MO31!>HWs7H9rY0`ldT&Y#S1`6t0 z`Z9%&4&K&Bvmv>5?1o6ppi<#V(8Pyy+wqP8;4mLF(w>1bgQ2{Yq38mlc%13Soe**R z?q}(Q`baURH~f=!u3=f z#}Wl{XCc~c?|7(@MQYrO0Idn?h&-Yx40Jeg%DQDCa>0-ke45K==z%Q#>g`_BTLd=jP@n zxAl~UzoVauzw{-`Kj61U>R^-T$+TC})M*|$?VLibS8Ly7ecqFa$dNnAS22zh`iV@T zuO#D6~Lynd%WCrBW%e^mS^6@(~f z{f4)BA%@Z`Iopq&Q};f|R)@{I^Zr~WtR+-N?i<|b$Y2IX_)wH2n*7_nUw(XJekzUL zX5u;j9bg+@@n`@(S|l+V7P6`E(wDbM#S&FxedIe4^<+9EQ0e+gNr@`54%Uv`bU`}-4XO@fnieb5=E73a@fpICej$eYljj{y5I)x4YX#|AH#kNi;N;9{TJQ8?y6%z*ah% zBtU{R4ucbDrhskMEB@68%1%9r2Zc00LbpJLPi8yY+x%VHU^(T&dWtaQQvb~Zlvm~E zNojZf4g0>g=j~T90(A`@maKwRmYj@&m0i4oXL%o-ly0x4M2#ukQr|DtQrIgXO1wD> zk^eOW8LjQ(W{4jBA*8dSOp!Tq197H5iMHvx$s>6G5X%$@k|aT%RLBD_e2yE*pZG#@ zm*(9{g#o3K{uKr*Ds~DKKn>%9^Bfs6%nW?CwCT85qW%i%VWG(pEDX>p@ETT(fL}&# z{7^exXUN)TJ5_HJEe?LBJ;8jrcnZoDve-SR@$&qA{TrDor25`s+miw_0#nfz>X!Wi z;ot(awqar!O&@k^4zgT)oX2!>Txl{G13n23nJ)F*)r#Ijc23@JkLPRi%9B+^!PfYV zL6bOV6K;CD#c-C3BItrr0Z;w)1_(N~Xvh26gaI|(=A3d4hziuQb~z9O`s5jhr0Y$o54>X!n`Gmauj%cH18(+h2Z)iclPs6@zHk^W*`1jbzn~>ee>QgQcMr(jHJmx5lOsCcPXlM zwOG#DRG8RS06{2bBrJYTQDl6l;Y!eiU){WvTnK-BU`_LjOxTlW zBuvmvB4C;nQIrFZm0BRcBk7Yh1A%ex<#<@TeJ_~hBc>v$sF2T-{rpVuw~7U;z83vr z*|V`7h0#ZP6>g9^(P&sm@`Yd_j7Sxm0M?CW=fgETG9?bqbc7$8B%7)G~OiQncEwzhgV#&Ku5bi~ctWbt*wu*F}r!A!nRhyuCZ=M%fS}&6^)j z=wMz=V>kwj*&IsgTOJBJ#s2s_sjgOQDKiHBsxS3td$fLTCtb-xC7+O!ymRR&T3iEI zH@&cSIv(c;vvH3skH%}4OHDlUe|S^JhDg2wI=!EFBC=S z#gW#JA89bcWTp^)Xb-j<40)xvFbr9=g07mQZ4b5d=;`Tm5VF0W;_v)@;&7Q=<6e*w zBwsYf-!E9EvCb))c0;y;?O`b7f8}`28cvv(@&=uR7B~v)5oMG;%YsQ?ta?j&U~0M0 zihvt!Y)|&xdxS%YL_f=1!6Ry%c>@+@v9!Fh*|=HzCgU~@J-=s{93DYF-eU>btsk8r z$rU!?6tnRk0kq1b!c6qnSb<6>+_VAf)#}BYIEya7v)D4;X01Ejo2?u-%ze45V*y?U z?^_d<8QEmk#gCR99*@VPDKB83&=qHOUxF7jMlC>zH9PZOy}CB}BAgLp^%FJP9m1<} zuJCiOlX4F^FgfdHQiMc}*rV+}6|~9wdK_mnTNPd(Rv-cI zfv8_aE&68j?Y+yboqhxyBS-rM+~AWz-Mu%^T-7{&BVSY~LxFe@LMD_dI~&EzSe7!y zCNEYsJ6!-1d}Wy<-fI~{lELE`XNG4(+v2;MlN1lH&0+bZd`3FslPH6sK=z)=j1Crp zy(2@{axRrQ{)R;o4YAe6h0M8n=b^P>UrwSnYdm~X&7C&$yeZo!N`A4`rQEwA;s__n z4tU~sFbk(du@4oWuxhY$$9_Rmo!W=)$JtldCE7;Y&lCk?nCrAveJQ})?m*h)k!mi~ zTVwYA`l7xxw>VK!sr)+$4!klW($2|J!v&hQ3gHz1y6<533tbFxNHg4o(9ht0#~`4; zqxL6?hxq5i6t1y!3EfPIA#WnIgSU3{AtWi-dlZy4e&cc3QY2pI6PmC>?8$UI`vVeZ z2MCs;nX^^5SAcXUF0+Cy98SrxYf-yG#Mj#ADV<(aO>m+l%k?MAy=r_J7vr0z(PjAH0QtHK<*W1vK`eOcaoF)IcW|mr1&o6mf z;rqiRoQF#x&5?q*wT+p(Ym}GwW+!FW17=JMU@L`m=SqfRh-YDL{7J`Z7G+)T z;H7=yam=I9RCGbjUvBU=^N{e-K)6ziUj6yO8fqTGODlsBPR{j7z+3F8W&E50tQpYNgUzQ}wMs`_|t(tKRE8l4oOHNilte=51ce1A{M$n2!wAox>(EW^2aa1&*PYxuFFXEZ%>1uhp zUQ4Tf5q!@7`=;~dI4*WWjeIQth4h0q&Wcoi1|-{=aIZ<_S#oMZy@2R9uR#u^BLYeKot5d5~HxU!G|aZuSf{MgwSy;PLgh3)P_W3~ig#?(_H z=g;FNmkbPbM2XiQ{d8=X1|b0~M|~z0obQz(TMFBJk=2?^CA1aRvs{H6>v4U$aJ=u{ zX7SN~wYEkNC*k7v?9chJ1^?TqolVGl{45$s`FNv?iCm%PRt*S#!H_y4jtlvBs&?(OUfZLPpah~wLbBd`Q1t6 zButey{95f>QI`kH#8Bo|mrs@od$wzxsQ@Re-~_8k>($qf2z{Rj$V3}~WR(O;<aN1Fhq#XeuDaKthxgfeXCAwvgk*1eeC@>=Qr&u}r2d@lx+q0!=m1x|3y}eX}01!#!Z4Po)PdE&P(HnnHRg@%i zp68G^1$G5LE6iAvm$rB?dkr_7DGH>{ws$6x&-ZX?Lra{FzLi=LUWGXJwiDOH$^*p# zH#e4HG3`GpIujtj*Xx>rPcNCx#L{ccm5ct~+4|bSFMArkYAJmI{yh+f0fn|O;ywlN zmZln7GVTKBi`viPEJd_AmB-brFJAs)XPK1$p1xvgw!@E9`5h;>>Y)a-nF7bb0zk_M zq(xUiokj5}JgK?196x#%+btC~%+||E`+ODyBy}Due8p_KL?D$D()^6~r7VZP-gCG7 zm>0a+fQRZ5f}t8;^s_{=i8ufEq=8E**SkI#$Rr8FZ*x^3TMT`21w?Z_)M;{|cbRa; z{+I~geh~Yh`&+J3(tRwo+vp^^*XWUG9t846qD*07R3mB9pcyEurn(F$ku#aN>WHH^G=f86jn@%g&ob*^{N0j_Ksv&_yc_s-CK zIJK8J?{G?P8Y6+uth+3Z+r5VPe@_p@w1W*r8IEwSFyt05Nnwa*lcM@!(YS)gB|6@g zo?tB&ROMfSh0@HUyp@dY!E<+EW&$!|4y;7vo|&G005YV<-to0n=*AH>P+nUreq_?x zX64!anLNc)zQbn)19r9ZE^4{j-d|QOA~!I?(4R&cZ};%oe{_pZqL8i9y#E+OO7Y&8 z?$_`oOjjhc<>F{Vp+kqW{K12Bk0$Mjg_WFQDJDt~z2f-cZ^`oBn7ZE5kSF*vxz|%T zheB=@!No87XPNuK)(-#13D&;g(85|V*mk7Z)K*7}17J4w_zowfATA1GGG@{{2380ssLZ9?)Q*Svi-fO@K%up0{Bdai@mWb$>n=R1-Yg^7{j zs@Hi3Rk_L{+y`+f+h7J{Ld11^osN&Htc9aJcD#yZm!=+{$`2kq%{mK+mwube?>&6+ ze3}&vzVTyQ4uW1$AZoHCPB(q%393Z9Zc8d^$;cZ0CvWGjQ)FC*EShKNbQtLyTL_g~ zjbQGAO13}R9z5fh6*LMnjZEXDgETgFDwgIeQ6ngy3Z%LnHL5T&*CTO0f=}K z&$nd5IsjpE*4#_ns#Nxky(LQ|QX$I57b@0Se%QBnBae19aS}pS!Xd#IpZ%V17Aiir zz6QLN6@Q$eSuIoju1wfZwKO$XMt96rPU~l7gL;#tj@$4x4{jDAI1CLIQHSUpK2-Nt zdfG)_ce$7+g43I3IM+Igo(z`Pj@|NCeM9-D2#}F{1|GVJQihB}AZ5(=JZ8o^+5jTh zksQ_>Zsu6+DvOf$>E10uux(^h=#uCrmO-Rgr+;qH{s1DfEb#j|%NsqnHJ3G)g$WW- zx?Ue7OGr0TGcPUrFvMfLQE5bj9FXCLQWLAs|lucq;xbQ6X308#i ze_&2%UT<(hA(6e6#HFXLP4+ya(VD7F9Ha?4l`^+aWEc>Wp92~T#M`n?)c)nA9W0lI zak`VT6e)I>sz;hp8Gw!vEyjwMt6xluZ^x1IZdn4By@r4h>V-h(JK7JSzl%!yTNy3i z6f3L$O1r3HuGZB@G}rYZZQYKOnUgx^k;-pVD|^xKdoB3IjsWS{BYy7n`avKUhxl*k z#_amsh%&Nvw81C+b>TR$-Jx(U-{alAUsLvm#bQ``0QjBD4jQI@#bD!-Ms9EK)H(Xs z(G(|QBG00Hcpm$+<-7a0Tho&#T`ioS4*4Rz-j3<>!&ocy0v`R zbk@Hfz?qwqD{xn|t(3(cnWCbMOrfhy!S4pSPNhE+%#SX(B zr}HGkRjM(qD3vwHQdnR_f}1gt4`Q1)4S}!Dnv50EBwF}Fg4rKNYhbLvfXTfCr#CZt zbB5-QUOtPJE*n^1Ak)h1^B2Za1kiglk29+kZQ9=Y8?CLa{R=Ejw{3S=FZkBoI>N(- zOLfI2yPl4Kq6=79lDnd2{q*+yLWgGtQ3P%pkwa znsA$lvT~C0HhFRI>*ILM)4N$?g80tO4O~qQDp*6c24clL#wa=cR+v)LQ4Gowv};jP z((htDMM=B2MO&WHo8`Z6YLZXB!?R$y06ln)a~I6Yi?YdRvf7JR19D#)sZGC~X};?G zOfA_h{Qs|(Z^D;GHdq%=ZcIB`&vc;}%u>Y+QC6#+WuUad>e*XKMj8;bx4Dl^4wWl~ z1b^`$lOD`zm>a{NVXSo;h@vD0Wtkk5jUmg}+L8$N4B0KVym%@20=z{!J5gWFUU!y3~J}pZ2WbMOYfLW55M}Q@77#j?w z{PA_xj#oPf)MmFWDab0G@1;c8A55{<=&%%3>-Y3D;J5DgTDhR_bGZcMzCkAz^q+R0 z`TQP=HZsEdzVf|;jd%PRGA;*`bbHYsP^s+g{|4`20@`D7N(*qQ*1p#b_*5#9MBp;F z!*8HYW$LgJ9iJAqEs1!zT~b3W%e$>0F-`vtSW)ko`hTrPOkC5IsRSe>ojQ8<}br z!P%1b>2Y8PVgjto0WaNj&|dUz2@G#duXyH>rlbnc0@7)&zqEhD_|Avte8F=R4f&ZzrX=29=%&TBEMf;`8B1**JW}(dRd{~D%gkK~Clb3FPDnxZ4z(G}#j@RcHE zqm@6gDGv)nW(@!Meu6u%2wrHW^X=J!VhCZ0H!OekocUe7Tlo|DQQ6f)S6MpJr$+B0>EkbJZPt~{3w{aD8J$#IV_ybPr(+;2%GF(-I1aDz^CQi1TjpG0r^*WX5B`JfJM!W9x$V!_MHiebwS zPZ4-d+qtHEA*$`c(gNvd#u$%ws;+#hV5HX=!`%|~PtVm7LAtfNnXJ6=a+^P23w~)H zi>aG9fdaLJ20WU=UTK!c?!TC^^Itj3>2gwvNcnIs7%b*z0gC}NBPGdJC-Qr?{aSE{ zQ!dyaiGBYfz3Xed`%fL4p@*S4Nad|cri!2|kL(#s*{$RMF4)>yH+BjFgm;FPpF6<9 zJ&^L$tELLusmX{+QndMs;ji4Hrhl#h&w@Yt#5+*AkLn#qQg(?SH~EAjMnD7dKjO zzwyJn?=+o@D!&yCygVZ%=4ndy!kr0Bm3N<3{70{HtIXC4Z%9H6% zzk*&3s)aU^VVwZc`Ri@p!Eu~qclm;)R>$G>A0R~u0;DJX^`{zA_WnRAVbt|ohH=@NY8RP znP2lb$mEA!O+EE_%w;geW};@Vg+&)Am>qhkF2q0Qs`V=B_^uz|raLr%QG>$}?BUq; zcRb-l$`hKi+@tHVUdqgbmm-@J7@8bME&kH(t(m&)5Cwnb+W0|7R$dzTm+f$ARV#M3 zrtgy+0=lTzQ=Hcyn$*{6c@9Jb1g&G6khWJqPPkSIQk&nL>Id;#{S^P{ST?+LvA);VGH)J z=Moz8f0i|N+6x^|2I~~9Hb+zk!)khxYOi34vKxCi2 zcnEDae7p6|-ug3|t>c-r^^?Jq?Rs6Zz>3q1Mwi$ick&NV~w8Qq7Hv`5|GU2 zxObY+ea*aXJ4{aw-$^1Sy=Ogq=ftAlh5EKj!+Od8onnh=I4BJBU6Emc zdpt@GIT#rFytI6+t{^(Bs3m}TV&vnqnW+wENb)Nz2KtBg!V~0~P4M0M2IW+gNq8NE zvr)aKc+E}&6I{n-$m(4r?a#-Gh8`5Yh=6b+^3?rIK*qAE8uwC=jFnC@In$%Utg}j# zO}ukPE?aex zSG|5W*1qF=b$9!t{Tgtg35xMuKDipzTgQhEznD7nu#(s~g=BM?WzEWk%~%(jEZ-X+ z-t}I|jST_{t$^%fNT7K_&ZoUhGH!Tf7YSbXJ^db$aZ|G^$r^pA5*9*|It7 z>@C4%dL})gV1?i$y_PbJaxg}%MvDu4)Xs9|O0#eJW_gLM%}i%!2zw5o4_x2uM8BY7 zO-j)!S6!)n`Dw|KV`)GQ`ZU;$>;Ql}W6X0Y(HYMyC?!J|;a)*HA6QAOqGv*xr zd=ybrdN}90- zCvS#7oi_Iqji{Syf^nOwjgH~Eu&v0ykxG`d@?aLrXAyd5RB>IyauolP2O#mqP(FZI zCj+KVs=F{GxM^wO10hh)(A!3vmD}GklokdWZW{upO?Dl(71FCYqjs2Nat>9~QvTy% zgGRgJ14&%r6egTgQui*Lxp=PA1KA9QmA6hP3nzvuXBiM>*L7jKOX==zhHe27a1bPJLs)mMYznBk&2PW|?`r*-%MWCrKd z<6}pHbiV7+r{{C^W+I58nRn5jjsAcA?K1`!5j%7&S`iaJ4 zhXU|&n-lDp4@#~{zL|-t@L351v#)lNvM1-T6l(MLEP9NIgACMXb3>2&_~@_%`yJC2 z-znwiBPtcs4>d?!LGt{|ci%wxvis%)R)=Y*&wo9h4GJwu7~?hV{r2L_Oya3fyDVJa z#kX#_FX+y5lPNp83m&m^;oBmCi*;p|v}MGN19Y{D9#QdL{As*YKR(5~)zLBXF#n_E zr@w6=smOd2Do9`NIMS0lfDuVhWJ}O_L)a_WA5Gry^W~-qvx6K-b*Lnt|E(( zC}v6a>ly!f5ldTo29fVhvMJ+zr(K&$(fLYJq&>MQwczsN=#}pdTS(XVX%wKGZ+3vD zzKdaytt|W@)TD9U4jlJP=DI2_dm#quP_K7#o&JaY-41gmp+4(iW-{pj+ivu6s;z33 zCeV!T5ab4&?rwcKa#803GNp56PiqGXQ-*l&r`k-*^`{KERJ~3}UfL$aenETmF?1%x zolucMBHfqeVa?AWSrgt1aaL?P(tDO*YL9X%GGfMv-X{VJR^+hwiPGX@Eq7mb_!PP_S!)$OZ2q_X74`fEX>L+l&Z4$EkC~$#GQd$H^>Cbwr(_3| zM(4fEC$zM4*g_;;?eKb%u+_&$NpG9^UHH&?PE1jS7V~@w^o3+HUlySeN+el3d9Ywu|+q+hOo4?MpX|pKK?!d#=hFT22oW+I??U2wnQ_o60dIs*xK|8YNgi zQ#*=%W}^Y0P7Gwe&kKo-y zP}qgmF@(_#Ej-ZDv^UiHZIuaV@Hl5G%IMe?0pVm$#7fpmrhO@}!-fyAsInQ#;8#Be zXh&Ll7f~KiJ08DgUtz$lQ(dzR(@rAqR?nnNdf|Gl*u1uiY?|pe-mS_r|pXxTGy_c z&TAd$h3iJn-kvwaX~>3avE!#HB}Ml-6+6@YQvWviU4g<@_cqt#S;9&6aNN$AK3DrQ z;n=v{!?^$^(+G(e8r2}Z@3Z}E#Y)uU26M+%FG&7KryY(w$7EwwHXVM+Ox&H#$Gtd+ zVkk-+0Z9+~I!stkuIaVA4i7Y`$1GW^H2j==6w!R{9$FJM&-MD^=C#Y__cpyiALW`> zaMiD&g^+V7x{PFv`PF&E)vv!Z6{bG8{BCq=EdlF8lj0+_MvperhSJk*GC0;ZwAS4C z9S_UAz*H*x6bhu209H?($ZcT1_fGySz9kM?;n)IA_o=%sM8EVp?towt$lnTMtUs%2 z-%#Zw+$YA`bxvGPB-}0DHXD^~@52o?8^-nmWs+Qz+;@eP4Bz_y!s^ztweG@AidRDjFmZ>@kJ&l5oJw!K;cSP-BlF z_?8MUqb?RPUD9X^ck~LM7>rYi`N#OX$ly4jtim6ni#EmwW-DvU`ZGooBE5?EQoWz^ z_`@XBF$kfMIR>SgfHso&ePW7|=HZvHSA`kYeD5_Tb<=G%JQVL-oiw}>y0T@W*u;yD zJ%4PVIkGa>zq$L`i*RMYrXDccvjr3}8Vsn?w5x)-+G8zaY|>_l4>+F*F*a+J$SOtK z@=SaZAKX|3M`|RegsS|f+|giJ#kI^&H@R;7x`nqSp6&y&H#2jABrSCK`v0UIpqLQgCht`t?Oz8S z)O5Xq_$4u_9U%W#n;i<-&>4Q;K4$pmz9vjyUIe#PX28GY@b5|ddf?mEK24)30s?xN za=zynZ$D$Xfpt39?l9Fl$VOc(AW~bzR9NKUP!IQc|G+crVOz4I^>*<`21(rM;%8nC zsaO#GC-093g0t0SpGO^zgh7)tz+NU!u;pIiU_ip^sFd@hiU8f52~YAHmbdb*N7B;4 zX1>ju=ag{KW8LC?A*~*SBc>g|Ykal)3dS&T=dXfzYmPn0wz8y+EA+t@k&B>*GKyWk zZn_9H;j%cmDu>s}yV6@63w-oVvx#IV=sW}jbHRhe$));q(@I*L77_-4(lF3XVkTuQx~}!oNgMwZ z`ca>Xrg>UwRRSPkC>|BKCCp29jWx^q?hBF2K3bxW45bQnvo(MPzH*F0s>P-6O^)4u zJX0kYNebgXyYQS4Uf^}SRsto4 zk>OTjRKoP>ahZGvgOIa_kWN#{mv%(_ zNMxwGY`sI;KyuO=pBj(@yvnvp%|F2N!aLNG9otTd2rA-Z?0$QCCmv6Tvy3%{mX#f> zq~g(G`HmX-5GgBYg-JFEE>9cRHe6~KBO<_7eWjVoV6UUVybc4hs!yRhIyxQDfpVQqNkHclt%G6S zyBN=kf!wO?BqQP-_bn*#u)HhCAmUBc`@orNmXX?t9@JGR9>nxRKvnS2-#(uPKp|M} z==moR-)t>7Y#}OilsUsHo)HaOU*E@)w)k@?{YU(I+#hYVwq8KaO~pbc$BPAOKey+lnKIt13F2?g z^h56=w+T2ijT+6V4ixqNUcKK_ng>`MMl8411!v6%l%X7-FW};vm=d+~eQU0NKL7F* zR>!e!@A*cQ&>veC>sPZ`>|0Yc<&fM1DH_ePPSPvN5S~ZMunK)SnJf;44i=`nat%5a zJQv2m&VN|eBMP4V66SVFYVe&s4_3i06(4`V?YnMhO&NAWMsP)~D!e)AJ zZQ59W0r@%H+GX=m`9rI4YO(yJaedwAqGZb~L7I41fe$j+_>K>z3~-pB~ejFa7iBw*|$>WoH4a zxcn}`k0znr_WsQVu93~>KOQx0OAdmg4ZkF-i1L(lt*Q$xH$lG?7OTCB76JaE1wI(&Z#A_qIeBHT z%SrKE_;=pt_ojJU;(qWx6;`jmv(z!bU+YI3Im9@g=w~Pmh?X-Oz$Xb0P(ex1R=Ma8 zGsKK*u-hY?42so1P%#O=r=gZR5J0tjX;nz7oN6u-IX)tEYAqG{!Q46$e)U4#O8qcS zEQ;N3c}}fQ7&lfp?pQ~na)lFahl@UBe|}kVJU=r>2^RU9|C<;tH8H{Z@kTCr@r*oM z@_SlDm9+;nRC(m9Ntd&-7?Fc$F@|5o&K*24b>s4F80EP(i2$?-Lzk;DCy>_jF!B>% zyLnpY+A!>{PN5yyWR|3%!s4#eT{;%Bq3Ad;rQJ3)AUK%>pxEZZ#=cKQ)TEr#Q(m96 zJbqt5iP8XScfu$l!)1E!I~?B{%T;j8XuNIUb=ef+r-hlZq*@2F6YrCj`PF0(c1&p7)9bsra7HS=)mHEL>O7x;DCrgKyccKc+ z@*+R{rAQQ*>;p;<*1lMVJ4<|TJURp|{7*i={#%hL5+k*~ovF$D<~{oSMC{Di&sPr4 zk2%SnA4*X`NJRx}+H#{8&pP<}l# ziEZ8J;?x?yao}hJ5SM}5{T%8>5yGxN8YV!y`X_^z7xHg21Da6xIP?b~j8z#7$p<{0zqvqCqqY>*9Y?Iy0 z1nO4^jTQn{Z_dQD-Z=b8=+_2H%xKDyen{ZxW;Vfp+i9glms?A6GKKNTy!DkheO)vc zk&@kw7eQ(<Qisb;i@(Ef@!~ zt(u{uK<@M@Nor(D9^{79(#f39GIdAS8*AiCcL9&TUw>taZp87k{|+D(mkFGiKmh%1 z15>aGg9w*;qd@qE!Wz5c@@SA9fG+z5lZXa+$&zrv2I3p{z8IRnt(HO*-%EH0`9PW= z;#yk&68W=a_n)k5T48b}>mofE5mJD5I-C9Uqq#b2>cgWYJvS6i_mCTF>C0XGO4K$y zMua<)u)2gv0q`hZoHdzp_($$rGcYPI^Cqf@w37VgSYD~;{VaZ-I1GCgTr zj-IPLyW5Jse=a@`@OAynt&$_PW{)*Ny3Xp&XprlWGlx%zK%QAq#M;mqeW~-{dLNcR z%qn>8c$ zt@+azAmn0%I)M7n>i9RTJg?(8JGF5wggoN4_Z{Bre@cP>E=fje*I&w`;WH@m9g_97 z`Hs2e?$juMpvo7s>!8NM);(MLy=T_hh8Ku2eRw3+;C9>!+qruXh?63l6)GD3%%CGiz!buaTzk3PQi$-LJ=CQ_X(>THQ*qnec4=}A2b zMXD4`W(CP)0<&3@cjMi@4@HqBog&3h>-Fcf^z<}FDZ63p{U-ylk4gjZc$LFgBB>p7 zZk5Y2_YJ6$>fCq9XdXuNe?$`S$lXwnUDshs#Vhs{iU=>sy0BrA9oOx^53a(4-4b;UBUh5)n}`eC{P`$WK%o-Fz-h55{V$U2IYHjtsM z;qq*waRhc6KJ7LbmQJeDS5i+KCIncRJ)oy{FkMb_&p5FO< z_ng5fW^@Wr2~NOiEBW57gUeP*r!o3I`_Q~k1iBv_@Z>Bbwu7Ye!@?I2qf+u)da7yj zGH&NAvy-;m-bH6h<0L!^ldW!R43XHW_F1WoTifZX40qlphXT4|qx>1abB1@x^Ee@8 zio&4b!HA>?ZYHI5Fp!x*mJOZidqP0MEF-d0>leCdNfwQf^^F|x1EzE5vHq2*m0Pz% zhQR+zSlw%6I9%8uc<1)G235jXU2V&2);Hq>4EzfNv@~ZKp>du z6N+o>gCSwtbsF&Z6{49ujwK{26~xN<+2|F_l6CdByi-x@{87EPg-8N z-@CjpbSX@B?{_T7g$Vm8RdAP^Y(AdJ0Z(LTMxe z(8StxN&O13ye-X>m}rEqOD5d4Zp)K8FPK6nGm=Z+#*n=hFy$FqY+OSm>_#mKd8Cp$ zya_UCXT*BsblR;@to;%CSOnNW`0h>Gt152M$r;9?HM>z*LUDVJg%vK*@c3kJzlFw< zl8Jkn-d+=qQ!Rk!zuis#eTj`M&6jL^~y^4DE(en!R1JR0{ZXap}i6 zI{MG4D2)0awT8cSPW%C1SC;SkM{+WrLCa`~Bz*KW9YGp(nB{2iQ41cQVT+`ApqG<= zGli*=cTox(J5Z|}vv!`GYfxHH@9DZC5^w8i11e2AppP_U_&UU^ zj4k)K{BKfJgh1iGBJvoKoD`76#+Fp`1B|XAY6*?cCrY)nOtu&A zZcC(Om}9WA1Za)_e0h$Jgc-~PQ7jjnyDDNhb1I&egRjN3@UXG^;9^_%+k=0!hLyDP zzbUIC!z34BFz*V>fo?n!t6Z#9#+5vd^|{p*-{RM${FaXU_`T3@z`OwG>68X9)9oNA=p1W+hgh(Fa1JF!lL4jh7JGmww6M zIz))q3ox4d*$;i>`L)`;Hzf@;$bb_ZuwzFEPcC}b6s`5SsZWHd*oLM?bfWakcOo?? z(DqP$!@iaKDm?kR^K1xzLX9iTV7wh@pLB)e%6b>+Vt8yY=KwG1{vJ82ycKLAeFs@Y zAS}ix;9ILkk#f{buy+lpF?HA`MFu>KXkHeMOYd7F2t^%0ih46wCKt1zWNWiiC%V!U z=r<(M1{_I)FNi#TPMb76$L7Dc6`8r|!v#^pODOXD)6Ubq>y7GjYlPPAQDNPG-jt6Y zFLcM+qKj5``mKm&OZ@V=DLq?m{p)q0cH)XDo-#^y{iOHp`D(t=eI@N_`~g$)(PIzt zEF$h4HxsnhhA)oTEQ)+jTr$(S{m}I=si4{e><9m!p?X_9d4Gg|3bV^@G^7l)6`f^pSPIJYtab zN*k8vg`UKW-yGO)HTr8puI4UL*ef)opExM}0y_3qfmM{aB<| z%jr@rmtF1IzUN7yRNkjC3%vPKHitB^ne0wr3To@v*CnkXKU;Njt_}GB;ubXsAyvwT zOG_~22g=habb#Xb&aoJgGG&$hS^?)^(UFeONk`G!{tPV-TZ8=?llYVUufFV-cidT2 zmBvb|i;9QIYroNo8o&z!34Bxsh!7R5o`h=0pU>Y(+dkCz%y6DARrlBwBA>$sv?`yzR;U8v(NkFQX^1jc*S~DZrjW$;qtp_{Q3ss9|GfGCr1+6x~21Xc#`+*>gOf?H< zjPL0IawNb(mJaUjGVNnsg-?Cui^lx6MMu`{cCLkm`U{S_0UX$q|tUVDDV zZ(C1BMn6tFF+?2BULNjvJV0bk#VS-$>KQMKf7f7&)#G^>cJ(FZhOq&Qo`LzE#h!E< zd4yO_D5}y_ z#!6Vg(4+Y!ZPfOs&1RyW%W@4c9hVR~Gr<=I8QmnjlOjpp+uM`Lu*y3r+`BDK8WLuX z3%~stWJ7qiieCPyVj$=HYaG|6I1XhRsL zQk3~C@*JOV90elEUvlhhr61iVC3UQ=RB4m>x+%k-JQ5} z2Q+HbTqWT1s?LbRKcw;D;8i9&?yu!Bw9V!TY7)(zgg@B}BQS%^xwK1L< zhLI1&I1h#i7og9jP}FMe)qwu`9N@=FvUpRj0P11(ov5W&`(xgntqz{tP{9!DIjlU6 zznKAlj=~t#Jx?_5j+IUqG6^b)OEa0w{d9NXDTq+}S@B4!2Rl7y>XW$t8uKTBX<^oYloq2tRB(HL7-mI>BO?Yak5=82yNju%TH~+ zC8J8>TIqBpIz2`&o5+nu0%mNvxwg2QKQOJ4wX6Q)S`13)$waJ}zTQ#2o@ujk-w^RK zeD3n&TP{V#Q8e+kjAC^`4M6gRK4r2S6jAcY& zm!&|(E7B6g0$cf33{Os!$cnTAAtSAk6s6Rh%(%lzmsU1;n~e>f%%TLx9L{d}{A%Wt}rAFjnfY&s{COjVy8UvrIPO%( z=+_M3;z)>VRyI&+wv9`haSdO@(W2g4V2g zRiM>XEZg31G-N;A{uzp!ukF|;@bk|$Eri5RP_8=uxs1>Adfs9_p-U%0 zcr>PK?vRv!kK?9{K8IQf%%pRbkdjOy5Y#1E9z(n72xzjWNM6Y-p)FMXG%ElCrs#;bg{Rl`1F^X{}0 zJ7)mL*09#%u2#(cgucn*L68fsvSRucv0VCy_{V$&#uQMw5PvW+rGpcC+un8i&Y>Np ziEn-O6ASeuG?g+ZHXN0VM_ft=E-~yZ78U*`Qi53V(9+3=&r2=PU@jr>s3uQZm#Q;C zt;DgPghk~NrM^@Vy9bOUKLBk(i%NbU^oOCdR_y`-sALS{SDL(;9NzA2fB{xfp)nY9 z>w3-IQ&cEV7you@BUT$r5av2!2c(xyWamEKyFKdv+EJEP0EyGCy`AWS(9~LYX0iX4 z1QHk*ZI<6f`jp0i{2q>ka>e1Ocv`-H|2}^QUV@^HYpIFU@LZd_^(1eBIKn~Aex~co z%_nRs=dyowbWJ&{X1z1MN)|UhDmtiT@>cmW!7R&!&S-&S2A`T=SXt~W_tRkT=%&&4 zTR3x5Aa|8Doc(8Go6@OfWm|BZE!*sSaw6sXnZf7HcLRWi`?d>{8+N-9n=c=o21pwX z)j^OE3IH}Z3S@8Z|EgGa9^^!@hfxngVz}O4Hpv&AZw$wEzCMrq=Z;AvVB{OZ15eN? z6HR)FsDCtX77rL!9&c8t#XU}g(DAoNGMkM%1OFs2p%kJUT7)nA^M00A(){BKmmrV^XUYgcnt=qZs!Asn zyR4OvqMHY+>vP0d3}KON40~yk@J}tvOyd|~?!se>23Y1$e-5Wr$$O46-mTQdS5?%+ zSjyv-xWG?|RFBSft^{L82_Jg+B$66ih6HSxdp-9BPo%CbR z>`s3Z`l+78yu0upZ{Y4y7(Av6P%?(Ff|aZ=L%Fe!iH+BsL@oUPkUf=$+mm{M;poRJ z8Oe_Mn8+%{>_wRncu?Rtcu8gx0*k-Xfcd0;Iv2q2O5T+X0zOWHDDhbpX&MB{$w6IY zv(FQaay^Jbn{@BpRtKNG=beRXzKuR7{dRrS*Go|eZRVy}g-_BG9)K(Xb73IFO($_I zk?;jRAjNd$#B>r+irb$6fa0o*>E?iMGwFYf~DE8ntDs zaPS-AF3tosRiQxuA%^S#q&gfoH#nM>&81z(iHb%LT`$Sl-Yg5MrKpBt;knzn(stn9 zjV+{l)0*D)c)d`j*RVcp3!6Vh1HiO$Mbq;RD;>{f=rYwZVTkw;`=O5zIt?XDSOX)UW zE{zIL5l3a!oj6JtCa1m_O4`)!8i=okj=qXO!NK?SDLgYa@IILMruZ!>~d3MGi~Vl&Oyw&WaTKDLgm{Zzz4eFSB?)Q}%!PaFoJ25}NGKHKJDd#tW&#Jd`&{ zrUB^>8reSz<_@NkPxxrBu|DI=LAxN3)hE}JR@RYOUV8;U3Vg`gwEb6{xxVhP9|uxD zBx>9`k>am`HaJrOEFhh|@!e1v91(({Axjv_utyaO5vFY#Tmu%%tA?m{8)m+N=wHl2 zkxSLazURTYN(dY-U3(nho;w(ew0`BVEnSG88cDR3qhUupN{8A@NYlT;usasJkn=I# z@X$m$@HA}<0%};u>t>#|P9Pm;J~H_m_VbMG=H#^>_I|X1)XcK$MJI%giprbmH^rh& zB!THs;FBG2q?ECTp?1G#mQ=5B=9V&s8;#S(ITV^YZZZh~GL;%sz~Wau^a_UlQ|PIw z(pcV3IeWq`cXgOE^YIgVvo(|fYCaiOzzYpvMP=Knj|k&SMt4BxbmmC+0>!1#^_m0o zNzgddJYYS_yzOUwo80 ze`#`*F48ZgbRs6NU9WYAeiJ-gTw#CW=lyZnff)4Iv{5gqkPQDz5o>igr9JcxY>-x` zbf97zL?#$sF0ducG&W}0Y?IC{b(T@CA&|oEkM(9m{PxDn)qCuuMZ$O`t#<5jbr_xr;_Q#+M-bBcqtfZyVi*wv^vTn+^h?iiTaDo0{f1?9~+ zJ~lC?2IptxUfvxv7Hnj#WNI%gjhY#*;)UPeV_S;FGBT{dOut{@JRTfSr*-B8DAjp%f{As4p27$J9^?7BiAQCKh@cmvfJBum$;g9ga{D@m*-W@w^Ts zY7zD7edv#PnJ57I$qH}eeO|t` zNaJ$v>gUb$+9{Xo)x?COFzAO_yQvANw|^G=XcRAzZ56m;`+dqE{sBA4O_UpmAnO0X z9JxMZpPlNl6%2{kXyq+*9;cAJ0PCMV+xV&IHzzkO5}d0OLp)zpTTHj$gmqB#2H1+A zXSN>m?&F|k)V>S*84%o-ADfq-&*>~e79A3hKd~}oYZy?MGJWnUeK8uyS@d-uI3#+5}gaa!iyVdNtIuh3whKbDi_C(GoR6^_U+ z0!Tmg6hv4EtzrKxg0vgU6>n6)FG6+3voNl3|CQ2k@9o|$X9GLIr-~nI&jxldzm(Cc zYd_9tEqq9-+Ds2n>jJ5IWY_78!XU%qcz3UsT&YBl@_uTer$oZH81Bi^XYuc-0+@?? zxiO4yfA3_OB{fXS+!(M%cYp1j*cZaz6l&h1NO)kghp~!&Ew|~}X0Ev|KiJ{z+6blg z750TSTGY)?OdvpX@d|{WZl!$1vx%MR^xqKL)Wh!oL%Z#7fH4IqALNCFR+hMCGPfE# z`)GQ(cszoPO-LH?;tCenR0(r)(*?jfKwN8P7$sgr#%_26bZdBZu=LR31oHPmJE{Q>tVKfRa6 zRtm zU|D$vd3rcqD7HxWx5a4FZ((k{kM9eAsyBYH_FD5aNf0es+o=6`IwxObI`>@hwLh0e zW?UOj;BtpsCnme2qvM#I$3ye_=)GwT0Pe^x9Jtgg%Hb;+`reLu2Tn#EIPP);VoPwB z0cAnhUmY{leOZE>l*L{8tG&(+(CQD@QoFAM4hQ!qwJ?q;{5L*m1ahy*3Z&kOPMV1& z^Gd~C2U!U%#t~Ge(=zt)+&DjiT80JLVh!M;qhBfMK2sn6+KR~8PlT0)y`!Q5b9^dB$~pdpy*3oPyoc$Dv37vg&>W~l zSgTrz-lSw}hkLozt_yyacPu)(9cEHmBmI19Z6VGEnt_CD_fYs;1w(8~y)O48Bk&eiQpMl9J2c zFeAC=qKbCdA~b_%Q8HX?w)qmbu!@)Z>le_IgIrUZ)E+KFWOq7dH|7^) ztnukIy+Se;6Sh~J?GNV#u3D(KGyCy*8ot@2_+wcbo8M?Z#H}-i7ydZ?y~%rfdQHLN zn)3z#VM=mC)5C;K{4IfT@*PbIYlf8#|C1xPd2DFuW`paao}BMKD#G(9OMwWd%B2$a zT=BV+@77l*fxuw(kebEz#*J2;=b=tSPZqV`Qm_smI?k)-{gosbT))C)cOl!%mVRTs$1Z?E$;#@r2EKIW`K%4g-1F(MyDosKV9LDTQO;UASnL4QhMzp^w1CI3S>oXrVVIS~jI!VlVtzKoxpuDVfPC6-fe6zFK z{2<)ReIR4Xo zWm3dMl^AMhQehxQDb@NP=vV##C|;_JvFMJ|1~qmR-uB?hEtVPa&oLPzQk1C4;%G2F zy?|?jUW!&=gK$u9_=QK%UtyLh--MkXTIE#_|>4Ln*ZF@KH)w}d) zq?5ZTR_1bwt3R{4pj4kSZt>Q^AKP^1?uV87AY`3p`iOK*`V{2e#+35gtvAb&s7Ks3 z-j1%%x2QiDemit~YcC<;XqJHEje^i!Prnm<;>WMp&(d1g7hUKF-8s>gvJC+~mZpuB z$;8p8#|ItVXx(Td5LP`#66^T^(9-^uJ?AQ^PG!=V34lzN)PN@ig~+PdI6F!CCiGQk zMB;IAwamzNh>J+(CS!Zq!HU{>!d{c}R&U;C*rZm;$L&oP=g}B)x(@_*%jDeIBj^@i zM}-5+iTAP*oVy-nK9=avsf|!J+&C@NBjHKV!)U4v1S4s5 zTQKFKTH#Ue-P~}mR1rm}#!ppRQ|h(X!Ah}-t3K{aPR(y8LfzPHNhD4mN=v#6Za>*? z5`N#*vB)bgIL$4*Tq%tN#`mAwbW(WP*as?WfAAsp z27;6VB_Byh>mv8~MQ?`yc>=KDD+whNa#zHUySF!bd~4_wehe;kPSJub6*;O0m;Fwb z8Wx@3>*~%M{d?#LAd)67z=4T#=dV%>2+dAUoHw|{8gvDPG|coCQf5c4M3Kb#AD_mu z$LN3@D2``}vwb7PKELfP*Z=+6)cfqjoon+Q9)3#0V!>B~5Kwn_cK~QjF4~=s+!EG; za?l()n&=pw>}GU4o7rUZkVdD+ycKFB(5LiaiuTpb6%3?8BgZLqjVAL1$48Dy&pM(p zpzNzR(~i6aX*id8BNyF=i{3aD&}=5rAaygf;c122XTTQIS5|;)axx+1BSnWTUAnS7 zUBbaOG4yxNY+bwo@|KD%Ow5^%9<5H1JEg~QZtc(ycKPAXLT{#kzJ1Q^_@jbOs?GQP z37R8pc3yhKjR#vphH58-mNVC(W+VVBVf(SY?Xf`W@!_Ue5lQT#6aT2e9lIKx#v2aI zQAS>{AA~c!<*Pj2Zt$`Z;FlTR@T#E`_2%4?PK?w3jw#$v`+`5x5>vp`M*e4+?YsBE zq-HtbHo4$wh$MMJ9Nr*FGF${NFYiw{t#0q4 z%8zpJfxoI~&c8Jn_L=}!=5zHmYDdW2EVd5qu<` zMzCe=U&xL!(BeAn#@$dhH7HvTzFTq+;?(1{;TLd9(uL#pITFwQp}|I){l`+zqxO#{ z(CsCaw-2AQpH$;-fY#Q{9Zzi!uE}7)+j+73orM@DFw4cU{E=R_XMW zluYNY&*NAAT>&ZWdJ4Sq$VDk1Aff+??ji4}CW0E9N}=D(Yr4d|yt4IN=y1D3kqi%8 zm7j;r4MF8R*Sjw{@?@v&ely&i?hvK1<#VKpeRCrMi$Ec?q8`uD+r7+lqd4sBKfClp zlP@}{y5|*VCpMH+QwA!8HFWyf!m|~WNYR@XrvL2IM&9J37LfC|hifJG&2^dG$&em3L zlT)}RYV6TJEsC9Jq8C-gkeRmx4f~T@3Y> z3x%tHs1fAtpVtmn*S^uoKbjbli*eX5Cte6=l?nS{4#1qCXY)0T zB+j+~H)?*29rDLmuojx&Zy7$#@0f=``qdKfK?PTZnoZ0k>G>{z`U3&7znN|!`OcFU zI^L{ zVH|FX4#`(qiPNb0;x}7PYEFl8hvVuPHz}^dHudW)KuKS+Ko57#GGn>SwySuY77skKj14xnS zg^402yrkhHo&YQ>9lP7EA$ut0*&2#)jI`t2?FU<5$wxUR%mQNDH?_L2Z##3ync?X$Usj^Ya%M~{Py&xR0)c( z>p7uXb>%W@;NEzmoxIxoi{_f8`YLM#=UYF=s;G3c+w4u+*(Byx1K_&sh~QflNwT}xsg1GeUpqALb4J|TjV61{zj$Vwjjt3jEg|Jb2D{by zi$}&~6*J+h9U&!QAl{D!U@YP{=o51kwSk=lPfsNK+U z!)82a$P7-lZGz9{C}ds4)Z5ami-^?_XBU%+i!M9_c8RRmTH~-{N8i;Xp*^{>GY&%jH_`Z-%-|i-p$6x=0g>A$ookjI1bH)ga zvzSTRMs4D|bD;`K=sT|(0RR6Z*Zwa3y&1f~=w{GVA?jvtBO~gulcC6l=TfuxOlAXd zA7!WItoCY+N$5SqK@GODY8q>+$5AbqlZ=hY%!Zdt>Qt$8d6ge}4##H=8wmOh;%6s<*lQ;= zn7t1ejLDN|j<`AET5PQ`-nm}1OXq=4Y55Xhy`4C~+l}NEt)SQ5$hFSn4@WC9xDh0u z!_c4@z$GV)`GsE@UitxD-PT<*dE4wnFv)Pa_rD#kDCFrZahvW&yVLJ^+OX{mcpvrd~ z+)*T*vfj{(kRCvR9*M*@%2v>H^ehK{rP@ zgc<)jn5l6**#e@c~p7cmC%+#Zmu^{2xC^@)9gMiaHE}War;m>u-rn0Iy6^9 zq?2zIVwXhI?WRI}`1|`ouOn0BC{m`PU7Gj&085-y_8k03LCx!q3*`aIB=V0xu7@MU zOHZAjb|zy7@vKM4gz!lHl~l(Wf(cOi!AMfr<1WIW>7~^CH$p1jJQ;s3pk0CG)$5?b zbGS2are5gZPa*MawV0e+NRK6HmHZ)-krolru@yzG(x+*bda$e+n1@y!_iH zN)kU6CVUmDmcM*{+HRfGS{)sz*UkBZp02mc$NIk(z#`f$r`)SH@Yf)nNVULS1#^Me z)%l4QXN9e41(k=T!=PJhllBcb{eDhhETuOah81w!3zklQnYCk+qFBjN=;Fkgv3(qy zXc^K2Cm9I^_@+M2eg(EG?-2o_vwzVQCJ4STf8O4~7OwhJx4@dRMvS3XcCX9s!?MA{ zc#jWP4Np0f@E^Uyz0U#y=Mw5&y%yYh-vpwrfB>;TYL0wr&Vt#wW{a0qz%h7pPie3C zSi_5N0zrBrWl~eO)Fi&V2YhZmi94zZL1+nHVAm51B4%2Vr)iq0X*REQ-?_LS9?O?- zwd(1#eTCjDtHe-JGc>ri#rIEN69mQrSKlhd%G#RR0e&0CB>kflOe)#JUd^>5lsbxvaMe#G^ej@XiRofi|U0>9)=Ijt6~wN8Inbdgw-!nc6g!sQXVVOxC|mzt!{pU!qG(V8qU&l}E88(iHl!YFp~z*JYHF ztH+Z^czvc`?7NHt3M^2A2XutQpxZT$}UrJdkYXp@r+}3+?Ug^M6Vmumak{2RChjc=gEjh%r~-!r4ICYnHQ+prw$dZaT0zcxJnKA(#55=_aphMV)LzbO=bem?1dHE+{ z%VTdalOGUj3NXHbcxbFGuCH@r_5f$DUHY#|#N&SVPyTeWI`FYQx$WKgPImgyOXi`{ z-3ND0nSVYv3C;_bulHQLOq?{bN}jq4+R9Q=M_wVn(HzP2rDibVH0lV=K=^X3$;aMn#i+Z2^+%B^O@jc>XXV_j!8S#Nf*wG;@iWcgP5cm7vQB`;7Tx zP&$Ia`^Vs$crIk0pr)^5)8C>9@h+=ETWL&19tC*F9nrgb|>M)Rk z`t^}`b9SJ|iRc<1<_m=8-h9k}PFuYbZXefeU$}Pb#E?w^C7EXrps}hqA=>u&+P?tQ z2zS)cviX*_Ss%NRxbp90+Iy_VS#>7ZCSbsGBU;|vtnVwG#$N*yJfea=tawsUbq*W* z$6EA*g{ev6(Luwi2K%kd0}6fTgiQ~oD4qGfcd53Np&12AiMSu#e|i|bw5*k7-6>r> zS-pA>Z}L|4LPDGl1QH7>Mrt-Qllbt>p}Z0Xm?JsvWL5oWOv2v@FKl>^g`I1wM`B=u zoCWbM!^ipd9V9}MJGck!4ccb0CyN?O0YZGDqe(J;v75a-Y(V)mwY0SMzB~yj!>o>$ z`SYT{r)m0*;vTs3HbKq_2G*s<@5=0ZT)w0Fq>ey+)Uu}w<;4x)d|k|$CN3}+uMBb- zZy7(W;Zj81*e=JO?Ok-SChJjEZHj7iQobK~dY$_AAR6--;1i5AO_J{g6#c1M8;6YM zOjy1eF(g;huXlX+&3dD7YC*1^Y{^j+Et6`kj9Mi5;Zx)o!ALq(@Gq;WA38KH+hX@$ zbW+mAlgtk4V}pUz^6Z-bmCwdXnA>m_g<>@gHkB}zj8IX?{Brp6KbRK~a$#8}4i+!7 zE0G8NR>WB$Zg3hwyjh|3trWI4V?E#&cXL|^dlpwr$~Pj<#;e?`6RD48^?6L7+!U%a zttIYhqQ1ddm3l0THviY-$t=Hc_4-?3Ppb9dr<=^`aQu=U^{V=(V48ZK8SpKcZ4(~J ziO2Y)26mNv?G-WpR2a1KT9zm9@uP9f7OeVJX=%Mf*+N_5RjtG3y|Vkeec(B~YQm&f zgMb>l*3dvRh zs@?}5-sVsf`^ zBm4YtL%C>tKK1KGY3w5+Z5kyv*Cw2h(6369F4yuW6az0F-de_!_fP|kkxX@;Be`z^ zYB7WwNr3O7O@k5b?a%t>EJp0ckOF$#z=VwbEfZvp1C>MJ(ucj#Sjji%Z>j<|LO*W* zWpSUOxehQ>#_mNGAV3_2R9G6ySzv~z^tN?%G%LlVOp=J@CyvK)`GG{HBG=MGS7?JC zlH3lGL`(!&yFjOK;SBaA`~h3%h{9$e+XLDzn{GRO&6JIK=_xtyhW8Ri zk&jAYMW$k08w{SMY;x@YcIbdofgv^8bgYw<@Y-5Z8;xblA1$S*{V)?q^6U{%0v$)Z zTyuMSr;d$E^m4$hPw~IUdb?Ccfp>g) zH~vm)9V7IgC0F`|$jOo33M7+yyaKDcPp|!cm|S7sby$inNgz$CxpPc=*i%>A9*pfs z8u;R!+WV1w1g}pfo)S`Lt2_8C>>>V4jJE|ZIKc`6bhp3WxeDvT)k%GQzg?)smeZd! zyIBdOJxZ&P>F1u{=>Jhb2Q7C#Z#1e__PHx^!F6WD_VBEIz80tdWL6G(kheR3S$s1z zxuT7ZzTK9?zb9Be4?wo}0HIC_iz;g|PiYs*kkO*e-mcrSxz1Z^JQqqCkk$ne=+>me zIFv74zsCU5KtV!9r+n&%@8a1$t#w_=0Fuc-Owwkm1{xJG@|Pmwwn!&{x%1M>#MO1J zMemm@htr@*E-$N}^}p?qbU+JRrXqViF?x?y`8B{-1FP*S(L6q$U8EDAcgy8j>6Vor z=C*!Y5FPoo_AKlE9Y@rgQqIsLNYQATy5*A!xLZ^xIt0eo{~ucP8tmBt#>-+*Fe^OP zM>1|UkeWWdZfelf&mi7EN>ts`v|<-htD(WkH&OqS|4P*V`L&xuE!rrvUSlngzJ7ki z1c=5fM~~7}wzxXM+8`f9O>38;p5KZGvl92j8%jRs5DFmiR>OwW4PPe#_Xiw$FhXZA zIP_TM-0RHK#rG~(<*KEbd9iRq6w#t6;$-ZrXQ?8lkmA!1c^X9FjTpp5fC|LZ1=jne z?(*#ekhRpV`y?dRb*m&KyLbx-bdg`)xXdMNS5uK|pJ3@}(=#)}5cq0x?QUiwE#&P` zo^Fjl(Iz~;sp@3kqV)RCcW-7YW%$!rJd#P?h9^+_xWYkqn20JePni7kr7BVd?UVhz zUU3#+QtBm0+}H#clJ<5vViNS;IjP;w@4CptrZkPVOxSbslPS4h*JadQX&S}aF}a|r z=JYg2B@Q(l7QGo#wKIQbE<JDI9_vS<#CLsaYUw(dD4be)0-HEfqbZ3x3t+{1UAmL3VC#wKyaH( z^qY(Bo-v7|zZK~Iio4&K0p+7Y75w$zwV^K^AizYlB4yQ!V9VXcpNPuc>E~~l^^f{H zL4vl}9u6I)INS+#5fwFjau0P?V zoq{Syo?9{o#(>hva4Na{nWVBZ)AS|psqFMiBiJ2zB`M?<080c2O%wffp%klpKDTTG z4JNO%bQ{dnfZ&rZ?It}SMSEF=sUaY2-Og`VUm!PT>{2)x-?NeKnsGSWh|uZex0m4y z7Wf2jw1QrS!|A_Z6?4TE|5oQT@y zNujRPY-Q9;e*m#!kITW~Axj7swE1}PjE<&wbm_CP^#E4Lb`J{eEE--(-lYkj{dvoJ zt9Po+?kHcI-GfW|@gJtTiZKtpEP+6f#MHO`uek3dl5IPZqqyjcS+X=LeSK!)Mz&RJ zsfQ@|YN9t`!&I8`Uz6vxPgIhh{M6}NF53!+9v<3MW<4QorVUp=_q_IsFS`%z@-UjvbQ|KRGP#>RsXFVc!>^K2yKX)k85!?mZjb=) z!w*VIvE-*LoK;tF1O7)FsO(iFTWzUyd{WfO7RGpKQ7xOm=TVg#G^S`3p8ISR_h7KN z>(ct;Oq6t?&w2R5z*%4?(u0YR7i3R#Ke7mHy@6E80Islfj9JXS7L=a>BT;Hzb2b?{iWTC8_) zjUGp<3Sl@$9Oor`L*?^)yB~u@31&CU7~_k7+3J$}bfBY)N5%jD71ClZd>*wS9gfhP znJ0{<`gmHdBTWmxgMr)ZmrCSPt@dR6jZKQAQ$_f>^3Q?D)A}m|kCu2Ybs8-??;w~x zV{Auh*1GM9FU3}I{h-r`e~r(vbQ9Lhm^-hPBgTjZe9i>7ByxuAWGHtp%wY{>mS0}S zh`@uB>xV$tHj;#P>Zf>z4M(!i*A?W~hdTAEu5RfEnaml6k$BsR|NERHYNN*7l(-PQnV;UHlz*h)=AL%#yI_4pK0gy5t-NNP*} z?Hw(ynmQf5>bherQ6chTr8SKe56h+t%Lc0-#m_yTBRwKlYhkk}ps|GvhT^liCWIu< zfduz^Q=@_Cjp?xz!#3S%E7;6KHM3P!19}2X(aGyj&{vg5dTu0yeL~{r77a< zKU1UDug7L4#_Oe=fw7)28#-Q+L77oaKW{EHL_ z`?V$bw}E9Fj+$#&v;U(dk_nWlx3b0DCOHNGCIb(LRF2@+Vc$urJ+LM+G`MFErJip z`?WE%m@Rik4hJx$LU){E^&MEU24B9CZga7LDYzztX1L{%z2y;J+(%$q6PdN4#R#S7 zIT&T2jDkWq;z}8D)LDx8S~!*5Hm2YxJ}$q}+Q&Pti`_Ac)`k&iKCsN|?7Is=z9g+F zQq5{4nJ6rFlZecvlfN(ETrRvcr+zxzP|`26D>yp#)%hF~&4ch`e0lL(>4bzMW^*oCzT$$SK%W#R5qjD+p%yfiy!2Xa5JAMIQOP$I85Erl-@C6f|=(y>je-fkzW*du_kNjZB4yaR6I z1Wy?ibA*4}4&VP|A!`Vbd}k;&##VWyrQaC9w32bv^nlS;5l_l%w#Fu!aTR%YdwUrR zS{hoeZL1-h+0A*~9d&fuNw%)DKJEn(BXj&Q-scRW@jfG<`a{en%mZt}1Ru<^;_)Oj zUZ*j5W-tI00qB`SMVd_avj~);+IN&>!kLu!){k4h#pkl2kJthePrZ#NmpKM?OD zvaIVJy~#?uq+xXnNVd{--E9?a?*Kd(@k{WZ?uSg@vBlVDmtcEN%R5+NK@38_tI3wX z*x}Mc^LvEb57uXNFR2R&SH3^-&!G9x`V;YU0F#Gwed9xyP%1~Z)AiDvR|*_ zfZ>ruBPbS~?fWOiMFYa6W_LGXFi5iNp3)#xFcrWeeXH5QjDTXjPDH1fU2A?4thDW1 z#LHv=SV66JBl}fY_~B;lw~=3SoVbFX(8?i!&lkDc{Sb*wv`u($SM8WfLyh_6I zzS)bFmz*0VAq~9)=L26j#&}Wl1t(Ujpi}z@4cbL7IM+bH9BEP`m^&<<)f)CVPfNAB z!S`82l>DSxj-9DKimX)0p9G-vJ)B=cEjQdq(n?B7*11xj%|!4o_AB98iKRo@8RKYg zx0jsSf1_vdn|6f)+`qDtROp6}tGDx4e9_E{cZ|+?Cx{{9&Q6 zym`TE%jW`>@Y?U@iddY*U;wkX!=w3|3oWY$06&|H%jJ})fJc45figJO z!$pBdg67d{0ndyM6ui!pYfyr?E!pS0s3$`b-O@9k^?>ZfsGF6~;L8bRIV_FDb#yOJ z3-aDHIFkS1NH)|vQc-kz4>9KD!0{`on5a=RA zUPmE=R|Fm>vN6cm4A|DDU=|Dp^B^7Re|;<{-HSVZ+3A1rKXAEK$Oz$@Dvle;6cMpo z^mZ6yQ=(^q9xB_EX!KdTfHt|@B&4{<{TfWhQterC8NgW#S+_tiHcQCl&@O_LfSw$- z#4;Lzn~b+m2r=$LjDS7~<8wvnNU&`JP`qWS){Z*kQPaNOxn;X zZSDDhq>b}AW0o&m56%CAojJ^9A^_7KH`lv+t)g1M5mk4LjbyD2pAX-(RXBXUu?zY% zHPVg>7&C7jvWeOujB38=fHeA)z3kdy?m!9qkRiL@X72cJSx5R6Z=>V~ak7nQWT3&h=5nSKu0K*7U*Bth=e$`Rp$FGiWM- z88rn|XP=)^1iSi$43SR!{?j~Ob9ZzN`0VDtM zxjx5Q*?8#Y3V}4A#$49UMnYVPZM3dU^dOTD`TiA>Xda!fDM6&NMWq<1Q9aeqbGx%v zRp*weufZuws2WI>ZpwSIi+41{g7MqTs|02LC=FeAhB>#c-1Nq;`IG}%i_4cy^Yr}o z&NKz+2LZRC4A`J~t#CVgSZ(a!Hy}E8J_tDF_bE}d{8>@AFkJPY%^&KRNSc}M=XT*; z=k7-~jV?s=j%e|-7%rhvFxlO8WYm%ud?EsgIqNtzy4Cegj?(paAl*vPjS)dZ-hT|(lB zoY3fk+KEQs>o0S#4ZQ`w0UXq#$Lvd}UUuh(%qvo)3|S@8IM=%7CHc^V-U6Thg-4_{ zKmX(X)b#ATr&tQXI&&b!kaLJTwcNw(ABy7IsiT&>DAOR~d*tlq0y4rB4(Ebm8t(H) z>}HVWN9mAz`u#O#(*gP9bZeTMB8}tr=k=WP;RnD8Ct*L)R;YXCNOPIzSU11_|5)S$)!#AfS1WqAu}W_wCE>+ z%p`wNiad)0#uti$-kwBa4unr94JgvSpE1nigUS`qmy^X-tZ7$xs1^8y$4FtF6-ZQq<_?dgt?Pw!*&yP|f=N2LOnIj!&D*6Q19`is=A% zIS~awr=7IBQi!`yOluOJzn((Qqs9pCM!HTzY~FQ23<27hJajgg$dW#!bXp`RBqLQ0 zd;Y6#)Br7%<0Jb!cwhXj9`KFASoUx|TW+28kSdkR_75_y(`E+R00@_9!qm~k0}l*Q zKaM(-g`>8^)J%qU_A?Vagu22Ci`u)!SFTt5i8c- zlmS1`5&~*5tE0xIx47aL6*@^Ag!8LO2+H3PPS$)$G$zQY4Xup!kW6ZvP7rp^&Z~6O zFMA+L&?$=-sAhOP4bOWj74~Gl(36_HmWqFq#BS?+{)0MG?F)C|9x+NXk8fn%zrScD zO~}^y@ki%Gtwq+Nhl6wh>J@aRX-NM&U=rN}_Dn#HydUv=?M)k%LgbJnq;5KZ2rOFR zlp+iKBma(|NkXy=2k?+UQVYSe?AS6i28<_Q`oYDLZxmWG=dv}POM8c z13N{`SP{D-kis;-H5>q1GT|H5T}ExDMrnHNCY#NoevnA)?Z~3kTdEvD?VVhYdisK$ z6=hEg(A`$t=!=|~L%ANVdC&Lfyo$iOI-}!d7l`BYe=HB zw=!*njlOwpbAw?K=e|}IDjtR_pd+acBcOMzL$fv7P)>4b!tLC zj$3*Va$XZ26TRp`b!y{We+USQMCBR%GR&0o{xj+AFSrx$1Gr6iddI(RrQ=8u8VW5p z*RkWRu;(^Sg3&BGATPK6Yu#U@Jrl#Ho43H+A-~Yhu5@aWSuEv4H@J3px*JxYa5zAM zUWb2sY->ZH{K_)?@f9SvlD3`&iqLOAtq^edn;06}ZnFe3`W;k|fh4AmH{c`{u2> z&U5Solk3ZNK~b_9+f0LP$bM$^{Q@i?B=C{swixw~!`r;zG;6|n z6(UUessfYO#E?8~__E&ag2yiQ<#(ZRjf{hrau_~bzk4rm4{*K^GR9JtDy|y8u5TBr zE9S%F*!Yn%lSw?8M22vRE)nT%7#`@82OVYMgczC$mBPHO z8Yg0-@1~f(7U_&YwJkrC`@WS-8X--+0H=)Qs#ytPqHYmny)RL86lou~p{)oj=u~=} zhX8849JqOp;#bNPs9Cb{Fmiq|a^T=!6e3(>anu3L@lF*B*@Wj5%?K=^MXkhTKwMw) z%;Uty$3I3kRbJ@);T?6a|NaU~gC0XDuM9!2OsB}Z8?_iTf98y&@uj3r7SBlEH?vHhfK_Cu3dRtE&+uJD3(&f#3HVN8Uqf#Wy)FSTu1xBC{HCLOE1I5&j_zXS)A!VkT#I+%JA(<^X*b?Qq7Bu+9~#o&LDwV zYp7s%WiSWkA^&vO6-IDg-Kv9A0zkP>@Fc_FyN9MkHVidRm=`(b7^ zjyd*gX(eK8k7(sPqex7v$I~H9AXP+veJft*cF8N(+d&`Og0J9FK4d|-2hr2Zg>H4i z)$r;0shF-`tI62Qw5MvZM={pxnOzLz8cfm`9~|iUUB~Gl^Zhn_w05v5iFC3SukNu% z*gaEdOsJOWDu;ho37#>$Gv|zN49Sh0*^&j`q!rEEzn?y1U?~su!?9`4{|b7)&J`7O zu<~7vBFvQArp+=cCqrE2bUv%n$7fOD!XDEc-9R(Bniy(Pp_}Dc&%*|LZPe_bWN6JW zO7H~-nSJlmw&?uZ{u^j-pfVGY>jLQcdUEYj2nUx7nn${>d_KvcicbXGYW;~`C0t}S z9bwf)+tzF|%)=G;>(`z(dGXF#7~yO9RIv z38G&W-Bvmg2)kCm<=dAioDBH8fSTLplKyq6K}5QEt}|==-f-)dC*j8a!=&p}>8IPB zh(CMd`I2u2d6$ZR`?Y%>zo`haxj7m)^m+a9``L+Q=(2*DA?CO2Nd&JRLEU6kH^qw_ z<-;c4^6xA)G&ZDq{dKk)8(nu0qCWWm{Tj*R7&8Os{d1(7Xek_pPpGLqjUmr^=k)MU>^QJhNv;k3ngA&UNe%lhCj~SXVBh`hM`ZOiq}7Md)8r=t^qaCosi6! zF*-)M*ACKnUeU{q5)b@c%0&GQtXWUFvTs{dYQm!awmyvdw0}mVEX;4cyVUz6KCTBg z%`o7u0nBjl`d}4Zm6EAd#7mqmSQG70Q%3`jr2OPqn!4vmCO#=aR1b~XmjD9<)A zRl%YZ;u&D*)ODH+x5Yhie^cA-T=6!_4C`SS+HR;kdH0t%T>iJCzY8B0$|b@I0I{xG z&DI^@;JH#aiL@}7daiQ25Fr=4tauEeeBR=|JGy2Q`9kcj{L@Rtr6)1-)~$U1voL>_ zSKGgPb?+nj+CAjwIoxllVF-i8v?^(;P7@_-NYXW&)}ICQdF8t&&b>5)WRTk?y^nf8 zh3#sFSzV!(0J*klU6Vv?R->HfS8x6xmBMVv7B&C1wGJd$p|a8lau^<}eKkBe$1?J%hR^(}0E;OBSrZH{Rng-l%$T4iF{wDmh5&poJyO!|4cOluL!5g&pdi22n@qo&igJ9;!1aIo}hz*c3N zCFumlTk84FP*=J_zrh3E@qy!6h%m6*)~IdCvt8)>?q=FeN+-=+y{Gc@@dU2QR^0m*?p%_q*cTZcUfY7@U|EE!3@Oj2;*7}f?is(|R ztF-t@(}ib?0i9119g)-E&cUSr-t+y{ado5Z@2ZY<%DvyDC(ZuPop879lc_b6?wOmR ziYZGd%O54fU!Ko1l3x|FGoNLwBs-CaqCD^1|7BS%$@K|B(JKJCcap3tSp*lMS3n9p zBiDmUs41i15{5JJ)+uvF+#E<{?eI*OkeH=N=$CQNX|g$7a^Bjbjt?NCC6Z9LAEzX| z6P$1&tND06Wv>2Z-J_04a5irt?Vr7_8HORW-I$?e%SGnR4# z6;>_y8Wmsqu&=qrnen2FQA@(^3q3DE{ zKl>)a)@xJj2A?-`2UyB!!Vp6z_%_brCgL%{V?t`V?s+tzWlQ0EaYh~Cen?$iT|<{A z(^L@*4sQ5~kRy^l6#YcZ600jSvl>C6n*x2*z--p0wc30IV=*b!gUN-(0C+Y=*vDBE6ryP=A*hZP^C_BDTg81&O{Z+<*( zESAikov0eJ7zseSNyMA@<012dLG!k zrRvM!AH-O#+!>}LZD|lP%D_R6=F^Lzn|H}Z-f-}t9uQF->DkRcXQ@Wb|M2=8h1#4H zp&c_34uta7vI2%7&k%|@gfEnMhROb$b)}t!jer3iTeF9vR8c^M+*9s4*c1<j6Y11#ti6yGpTu)+qO$tM|&sQFAkcjEr>W5}o zX5grkXF)$8G56XHbZ4jhI`&l3M-aLV&qs}qv|8&M$0ho3#AT5_5bqpvb}FY#Ui8f7 zVSz^`W`{yR0@vOrrLqn?mo7TiR$S82SI{H0=&jREbKuLcS<0wcoOf$fR)sIvWFdAZ z4ns7hfx}?bZ~_yldw0wv(mH!oa37nzvwa=rWF&5Z@Jq8?@w{EOj4m^Kv6n0Oube4# zOw@V^n@>AEQh(jAb^^Y8fy)Cwmw|VW#byv=LRp?k1$F)!&M8@dsQM&~@62ESp0UQ} z8+JSSL&2x3`CcmyX0>L;E|qEe94AGxn)5^`*UN$uVz1zz3!5|;|kZKrkP zKg*>Lk;r6$!f<4(R2jcQVi?YGXrCVzir*%aapmZd=*@bYK1gbUv%ZMx9;uQUK#o3-^{e&*8dtDhnu^A z_Z?yEM-2K*e{|-)MnxcsDImN!l?r`pU|W+2PloM;%Cv=>5%F`hQicbN+MS4%QG@Mc zJoz*ZXuoD$StRGz5=ev0ot>MNU&1*3U;#!r+W~l^Q*>`4YXlrIdJQ^-GiQ!uY)V z;T+Z1RfU)OOv=E#=!KUuSslzL?c}lx;R5G{?)>jSB8x!MA17%xU-N;vHXGMK%K@Zf90F)kAIe%8beF`2BswoFT0Fou1qSZ9|P zvQZ1etW&lSm#w<`rgV_k&5bU16LL-$l$R%|m6Vq6z0DwOkPRo$xBaCvYELif%ezO{ zV0M1RqU&2?+CXgZfm_?c>rE~#vqp4*GnbWfHt<*qfU)4T>-GwX4IO5x`hgV3!Kb|u z#cz+^m-CA|Lgihj37NG2u#CEUddl38rNP9nP=l15vH3!qNmijMc~-Ks69)~~nxgA4 zPRy;eon70h>ae6VeYJTNm)+p=KN_zXbegI;*v2fB=+LS7X|M-Z8o_8m@lHeM#qu1d zEK_N85pmv%nCXjIsxMz20C%ock3EV=P?-3pJFOP-#SQ@+ML#%BB3@0pp=wOT+b#N; z#SR9Rz+BV73yw(G^z#vn$BLSuMULAWjux~e*o#vD?q}(9m6y+0h~9lUj!NQxd=x!# zsg`R413|p5s^_16$yA4A2Vfi&(cL?i%bj*iq4Juv;Ibu24c*0Z)QqzSU_CMkdy%^o z(e0y50WFM!PvcGQ;5^r_m=NB=A8h8uZ!9Kdzop@!IetW2X_?PKpGRNQ`2-SY_r|j| zT_QqJ$BU@IBLC7Q&4=tBMi)N#j(Qy^!9#n{@1LFo zGHx5*-sZl)vnp9tq2*f2K=#gSYirBP%lq(7HH6ED&GQdeW%l|M&cL)GiBbB&#ro5Y z$Y}?ncTwM`ISV<#cxq6FW9wDgB%cml7vD?*)+p%`4#x}JTxX$Atw5V_Z~D5@#odS^ z@dy1(c|FA%qB;jt&q?hW4tpk9WUGs9m)#uy0`e2fNL}hSC!~&I;M|3Nw>^x$LsBi% z{e~?6yr4mH`CYSy%eJYo&g6{r-N%~T*0oFuq!zad{_@;2yJR(zYeK;NtFTato-f&q zrFJ>`&3ytEvS3-`*7Z`+pPi7xJSOGhw`Gxi&cQZDMkhai1UX=OL#yc|!UHRhDO;3B zhSoaTn;y5m&(5WOC0^bb%2<)!(2~#L&%Yi3aa(ROlgxi4V;hu&#C6@9Jk4Vv3iegE>JOO zN)X}3JdOk-r_|gfKK^$LWEQ7)xwIJCHlmMY6YwvL+iabX`(s|FbW}NPuu-y!hvfIQ z!H#weKM>JVE%^Kbt}}*aOBu>s@F zR(ZTz6Aeqx&(Bwrd(Y2L`uSyV7)Rd?;RXTwcVUQ|jLl#%3f!kJakD*Mg|GdrN+4=e zG+TYr7dFEQ97}I_tIC%-mO;8z87lgYVJB>##GH3K&#g(w@-VQuTx47+Rh^MroU2$U z`Y#E>QhNlt^)E&R|F-zwRa?>VPkUX^P(B}*pdi_`63Jl6jkjR4^Avez7lgAZCXcmu&f~g@MWr~{R`BoKuPbAQ zmxFJCCUUQi2U{^3hxaWwo8Bz_96v_75`7L3L8tfze@AvpN%7)cVp@c8fn5o|`F%Er zEELU%7%GjA1}<;}>$Wg0^Rs(-G*RD4=-*QrF~L4W|9gt*bOt(AmAB1uM{T-mpEbzI zjih#_*)v6ayS1%Iz+j3L4P!q5fH1K{Wn#^No=lz=xcO)C}2juk9T`FqVqD& zR)$B~kL}|+nUJuloQVSC`ZIpB?sIo z%_M4;e1^^6JQhP6Cm?2Pp3BE4$U+2T9GnL;upQ$IE#wZ%NBiD*7`H?j_;@Dzd~1Hm zr_r;yJY9SuyD`GM*0SG6w$;k(MI26zjSru&o)rQsr5pc%T6tluY#TDIY&?4hT zE5e;I_N|=UMYM$Kr1C!O;+z=9^vM((1&AXW)y~Lf%TuRVg`zIU1PM~{N+nkquoHT< z`dnB|Bt2?TWS=PhJ-Qa{SEb)Ie(^3BSirQ@d|iTBhrYUy&Q>h`pG8v|=a!Ig|y9XGeS z7lo-c!elvMhqu$d%Btdg$c)BsK=`G(VN%+N$(!ZBA!Oz;{+n>}FX`WP?0hvz;4%BS zrtlW-Q#W%xN91Gf%-^XaD~u~COJh2c5ON{f%WnGDpNFp5Y#>K%N!ni?jhDtuJTk;lF&qQ5A2&Vdi%ibCsqzm6@dU1FbowHH{7} zaIvKe4t}Ppm>lT4#-b|^)-!@`a;I~=P;tcaxa_8ffoJg5oSoS}KAcS5{7e?^q+iGN z=xS<8zc1Uxao@xBv+^)dQS*Lg#59}ykgZ`SQ2LQ{riLXSx#rz*$bttfdXodtXS!=A z?vo>!9nW@-y>`3A8clis@iCs^YgIvBV=uTnExVOCQct0+c%Ou!Fb}-0okIwvnZec8Xq2sfa$QI1w2e8%M9&X|(FOA$;Vf z>mu`U_KH(?Q1!ZoU63-Np~hzGA-MVUJkbSWIT^L^E$a6F=HULl>^nSiwhFbd$9_X2 z-e_|`(q`Dz(GK$N9<~gEAb+?&hBDy0-8~GCQ4i!(1|Ky?sqk^J{WpebofU}&)O`3y zKVaG8Ek)?AWS7tJ{v>A@OXn^W*iOZR?*hAmy_nd207j;6h_B_?_QQyoSeKJ_G}pWD zL0I^x$-#jQ(1Uj2WJ+542rg4VaR+IU@B17{2O~Wi4qBPi#Bl@Vi z#7pCP2bh$0j!pWAc4sUkQu?>B=5$dzkHWGf{06&O;HI;GD9#%bp--n!wlSIa|IB&^ zA|MaJP}(IR9b<%u*1)WN!zOB@eKbmUmCJK({J59QYFtmI6~5W*$V@4GwOSvL*jl(N za%+5Bt-}g)u9JbiKL7GIpRb0=E~6=f5z$Jr*$pmyjJx=V1>I@*6=SUHWB(!BrVI@Y z&87HHRGhahhZW5yjfbOj?JUZWfJoY23hJU1$-l12)6sWU`}Q%On_Rv7@8;+PFMODg z7xJ&`SA2(4(3?AWIj3E|X5JSR?hKnUb53_QqCEGWj<02NA4iN8DQyG3eYrx~6!b2S z?6H)HFSKXuaeT=y@9=vd{zBdT>>+eorj|-H61GgnJE@KsrE?N=p;K6oTv9FoLbYk! z7dUcfJt2+rWVIoOG7}v$mJ->;?!wM++itL-bPS2+b2Hn4r{UAs5RRY5%CKt-F_DH5@;IFRDoB2QeGPIf_$7i<(XkX<{r=RakrrPlxe)KtExbjr*^R>{ zpooJs;3pr}J9M{h&(3GaL9NJ$mF})<&2pFwxfy{9KT#i~6gE{?q2y9Oh*2f^$oQz3 zn2H#5En&9^cLYhAW>F9+Gl`e_>S@sh7bB55BcF3-1PwksY1oQu^}C7trJ1ORc2Hl2-YfQdt;}kROUEGDmgPWAxq2_#YnjI~gpGH_-Q0IQ zP7SkDEIFLn>IFR1W4s!06^2<}5*|1uW+vx!0Vn@x)tey_TA6^B9FyUO98atc4s%}j zfQbU@&AnSCphF6tQAcvU0%|rSFd8V;e=v@;$~j!g4?7z3G0hC;7KMY5?amKgC0~2s zJ%~eRzxx&=pWF`P$LH+uO8WJDpMOh-r%LxJ zTquz+>OjmX{3grN3mx^+n#{GLyPGV=luMqMCJ`NKBs_P+7ijyc1qQ#d@H(r=-!2rT zC?4;_)&C?9yT<#0euY> zHAx|*V6#*IQlQWFuo+9~hxBkmY#Ir_V<>-}k5|uJd59C>NGZ3mC14 z^h=;TH|G8@AKd;V^0Ej0YzPTIGhYwv*}4L>j75eUM-Y2|tNg+pZMp+<>LsG*&ay}G z6rG?XboN%omb<5|J|>_EVLci(|-rmXo&h+q}|KUVI`EdN6Wu z%`yCHk-=|j?lD549F<2m%YhD2>H}Uum&44{^AQ2`*5jx(8y%u7Asj8K`W>V$ZxRb9 z-X)YvV4j%TiMb zNxT8YvCEgBHZgF&-PQ)*s|#j}_wH;zx48Bh^|BYa?gTJSHumW$A)HZZs3*;2aYva` zuP^3#;_FL1vL`!U{@mhJ>T_GVvP=csPXBwD#SgSUFLFU?38DjuCa!U8cM>wUFc#`m z^{TUUnhjx*WJ1oX%s^}-Z`CfF&`2pXN4n=KRMiH`3dfB5}CM;E;u zmjsAd-F@F{hVcy!)myLArGt2#(L}=F-X`FwzIR?33`feqy-5l6AP$Giqq(y-%;YUg zjI3aeazUAwG*W?+BN4*%4i$#`&IiZdEVN2UQpP$`%0JvrEBd4OJ;8foG$1cYBzEG=b5$(iJ0|^$=WPGV>bU&~mA?MY*jFp{=C!hE9L=cFCB-7wP1) z#U0wPa;*JKLVL*D63Fu}{pYji>~W61kgGBtz5kHNnZ0YGzvE-Ku;5)etp+)iu^+{b zKVyC)ou#6AEOfN1x9bM@gD4R3J8~L-bpY3}&F!9ZQGu{V{6S{rLcA<+p|Fdi50klj z*iMGFRtWRrb})-}nLaR@zZa^s?d$E`$HF8eCZ?4h0W829Y&$hS5p<%*Ca+`g)QhNx zziP|&QK0`6`~=5sI@2C*K;au<+sjJo=q6!5%MLg-bzla{qS2uCO2q|m6E zsv6(kqtwm5BnQUd#Z#?i}=Y;;D{^C4Y8QgGTj;!oRB(6RF{8^1iyoY8Nx244Qjd~2q%9Z z!#9_Owc3$?pbz&d8LJ768>wlyVL35wsk5<|3{Pdcm$E!cSe*iKYBPW6ha{mIs`ySK zRoNHkxzL}HD|oU9pSpa}_1C`hSiBASW)HfT+UrL8vulJeT!on<{s+l^T}H3wDcGh) z!anf+UQ+Jrte>*KRC0;x;&Oi~*Z&-y_2&E{Fv?Tkm}n27rL~^!=#w(@wBGU?!m?;| z>(Lzac3t_mRJ+t2vXB>#Sye-PnYOHyM&;^i-uSv@1=AG7UAiKr)ptWPdJuBD|6W{t zGHK@(gF_mw71npgmTX>H`yMwth8n8?vbys)c6(MMFO;#N;8Mph`0q1do{8r0Y5ZQZ z*Jxn$(17!D$t-QbV9LY5q+}B_(=`1Z4|#mNa4E{^1Al~$>XfHUbE_fxl7d7ovja2W z>5)ds`EgVs-+rQxwE9R2FMQ61r0Y$^&Gk?)$Jh^@mVf*|rmp%S>MvS?bO;E7DBYb) zw}f;Hill(7NOyO)fW!jQ-Hmj2igY*9EWNO>@8kEqAKrhkaql^E=giD;p2TXnT#>L~%g3LP4GlEJk5J8vd?qr!sictQ z=IQv~%gL_DU+Y+2|AM~v=_v^fPP~RP9c=}OITU3RH9)D4yQ{dxGA$K);<-*0pJ-1`-sCXRf z=@+i9u7FSX-Tklw@hz~UT?CBM{i4RrdM{o_1C z?x>72N@($?Oq}|tqHXWP99?vSh2?B?K8mX_j2M`0WC^F73F_>Yth?i%TjMydb zS(6i;ClBeOJ-v`H>MAU{K?0iS#IAi|<{ie3hpXreMl-qymldYee+i}bCntt$;%3%f;oE_b)K0;M#r{t>p_ z8_5(cb-dj`{U8v_uv8FRPY_d(rqz_Emq$ZbD|dcrp1DyNkNCwRy{gy8zL>PQ4*5L~ zd8-dRhmp+c`#x8P^WJ?Y<*wqT*VFXlP03W|dY-sfbC@dLuW)#zN4fuu{iMDd{yXe+ zFsAcvE6Sd9?pQ+@h!zLl1pX7Bsf>7Ui^j4k7+Y&xBF{fPPND=qJO;5iJYiH#HBXzK7V}Ov!rD|}v^wqo&?~fHU7_-L8AxVka8^ zL1{vn>oIN;#<>=t-B`dA~5Sr@bA5~r_Xp+lc{7?IT$3C}Qvk^v?FUx22*MwYBoHz{CHgJ!7X)qd~i6 zj2Tn66?B|MOgox%trGlHvd$g`f*NmA?N6rtlLs_SF$>`;Fn<>38O^^@>Fw*z-^kh; zgA3Vxo7{=SnEoA3Bl0Blr#l{fw4tei?S6a&h}ar@P05YI9QXxaSN1NxCl}K^@S|#9 z_ipfM^!0+!rDWE+&pyR3PRic?s8`#BdB372CMIUfKA==-SuLv@KY~WSP*iL9{_?Sj z1Zx+b1x(wc>h$cPtNBkcYrO33B!v$zGKG1teb9?47+~@i{-vDFDE%%yG2n`T5^Dy- zi6@(pyKT8p?yT*JR;PE(`%)DjKMv;=XEEC>KwyGcSPlR)Z2pTVKM(k3!`>h5$ELmD z#iV|b`67=KPTcMljy?Z(F=!Fn5g?CGKN9XCsNloT z-KkKPK#|`tU`NNs{N#*`7Z5Z=E=d|=Qrp&MkcOdv41U8X!N`eFa$v;RzoM7W4q3PM z^_4ghJjy;dO;IzwkR<ZXlYCLSY5_fvmPMm$tyz5c_TV45}bNv;r)XL@7w@ zdPaL-U?4s|UR+$W!*dJBX}ku43*=kM$MI@7!dK9GEzLr2 zI})aPiQ~a!MD{M{Qjq3CcyE4&AH)om@#7aJu)KtkYtM(T2_IncSr1r(fgbRu%4i%8 zMerL!ykt2*ace6}3~^6^V|t<6e1aeSoG(to`v}cKz?s^9KpDkc4u4GPv)Fwo{onUB|= zkw^)3z|;J~%^N!+Cra(gI)))%2I44e*ms9!rm*zo4GW>**cbHx)8bd@G~8^^#OiL+ z%Kz`&s~h{c8$RC)JMAoL;+Xx{rAMJv>%Mi4?(e1n(8b$fZ{3>JRHDz$REaoB@e|G% zZURrsaoIE4ZOJG;j0~b9Dxt>#I=#ZtW98`lm#ZHdV8Y@1wW#!LXhaW zy?>U+iE6!`g}hq$aS&A8llyh2dJ!38UY06-d~^>GbfnvR>|TYD+5STItp>8PCx8So zyxyf0V_(>s*O~b3iW58|+il%tu=5FeC=&G=YDA`u(wHmgKX&2%in{V<1<3%1vlUa9 z7CCVh@&odM=`##IN=e6!HJWZs+><3uR3)9*82z04Lw~~*-i>)rBqn&nXI;PU- zizGo(Zoh|9x=D_xL?1zvhG*?`)OEbS8-z#Q)HR2Gm;QB9jYv@lT*U1%Wc?TqXR8BG zM>M)d3R;%Md)JKy_`g1(*VpXa#8ix_6wH8a)v0gUnf3LI5pZ)fN#9;7&hSKM=uy0{ zWqJ3+dcTu3y8J$?=_nL2$G*-ozrOWrZR~&ls&)N(Lv{PT;uFXclW&c-<%{<|x((Mc zeu~}}$GT&5NNxN)IA!8Z>I9vOV*d)%rZLOJ&XbQ%HCRMQ=-lVku05mKWWH}59xEd` ztZ(9d0)DrNVC^13J(f19{R^!j=RGu}SIG9X=Io%aUhg(;%(GPOANbOtk4;%^K~w2K zn>(8jwJqu*tOmv-wqAdqyo|k|kgBKa#^D*epMvZvSB=Oj{mthI95D{1syF_*4XxCq zn3v`@p138qIzHfLEDjRdNh0IpUUl56Ihp9{Y-d^u#`jYP%SNe)lGartNT`$O-JwDoX zdOvOg?7OsUh0^DZFKXc&Tb5;ntB2a+zfab-G-QKmIj^*=4v~nyvqVVfg!!#5gF>9C zrAhg;D@PfY#?xje=}-7ehgtsY-+5+|K59i9G4Hwa%=4d5?wjxBSAL-anrzG$A2uD-MH(6NkyWI z10NRa_5wtBY`2HDH z@}MwE0MVrdK|CvS#0wKbx>;29bRql*lNzdE6zT<)0e{6D(b{W)La!$91By8%XPnsb zcciPe`S=Oc))7oydc^)8=M+0v;k*WO<)!xbXv2F)g6lpP6W0--^}FS|a+TVzZ1U0G zwUvsf42wUOt3EYu#ckelMPek|pdh^0Jh1~A-H<;LM@=>s{G)-gCYSG)Ksvwzc5R(k zvBpHvfPQK6EKFV6XT(H!jfWS0&O@1nLHYWw=m8^*j^4u`JBpQ!G-EN}&BOnWh-$>% zqVn^W$@F;@;zs4Si)yhrXg^Ivjc*>4bL7oC3u-MmO6a&b(!@4D55{QEW@N-MaTiD@ zqSF6e|8G3B7dP1%8u}*m_5R{tS%`y|H;cB44R>qQW+QB4ERFnA!qsH?7(GP|sp7uN zjoIG16vmI9OeFjiOA?un4)iP7FBN4eKSix1eR4>rH>d+WI(i+#_GG(`^K>-5qGVbw zP||tZKJ!(o*t0}AutqNFkk;V5Q=`?<&Hv6J@OB{%h1wg{L%Le%g~vzeJb6jrg9C0p zw!EH#%XerJ1wY~?_>gv;9XLK^twV=`tvEr227@@1G&(JBT3MR1;`85l=3v8|?h8m) zHTPO}x+i3nzv3^S=BRs-_#9tD{4`r?@8MN^uENk*F9M)7lQ6Y_s%7-}8j6-W z3_&1$MlDKey$qHR71r}4E%-aydxhadzSMqCeeD+q7?+IpzvD-S9t%Dr2}sEUuV7Eq z3u}c-)C5^&tfPv>4r=lsx&RIbk_mAgsxWU!Bx$5ie_y9;Ot0d6ZqvN1qD_@E>t>|o zE~Q2M`tmV4K)`ERiy9jf)f8Dt>^&v&TbEa;-I}kp7`{u4r^mC>v^>+?W+F%G-ZEJT zrdc3rsCn6xCnM!{HHLpgMuBMQh0J7P5lHK!$CxXB3QYoVPbD}*_@QW$!BkU8Oixiz z@3Yaih0?Fhp+pdZl=z~vbfOvl*ei}|j+b~FPJIAE@cMjHc=Dkr847n2kK2zF+IRSY zkfoALjPq+BfLOn+`%TQ1R4Da*GKAWH-F$*Vc(TKU!_J6800r>_@pdA!WF2Oq(+@r! z=2szCdGmxk$p{FAvMVEk6jlmedVO!Jihg{ajhRX?M4*74jp?Did4px(oXv;ZD1L4=JzL?uz~zXap8~{!M0KbpuUyhZ^JE~mGYrqpd)*P1(2m#F zd=llbNVeN1*yG@q+M8(~)*i36utIg@$-TC%5P1td_WvW7^Qttjs-6+wSxMwIZIDn< zV=r9UD%P(geCNxMXo_1~;ZD|JtL4abJA2w=uh58JZd}&zU?=51GByPo385C-tj^Dd zfK~v{W27+|c^67kW*hw18nd=k+AFgZ9T-zknqz9fjjT%`w+QFl7mi(G8fl6N8-;$&FlN#XflgI?;3X{!i7q@})#nF4PcFWDKO@DOW==k~u!z})iY?95 z+gkU|acmP>g7q}Gg1?VJKV75n5;0RkaCjy4>IFL>`Fm@c?WBrDKuzyanyHY4*_er$ ztaMFl){>8qmP9_01r-@ZrfnI!U4tfzxH?&(mF7O|1?iCUmN^J|Tb;e%dVk$Fj#pf# zZckcA(fuv@I0t^-6tU{O_vFTi?P1;IAEQ~FSykH45csZzKRIz1skWz2n;Trybs)33 z@Q}+(k})!VVorgIYNI8SrpnRv8-?2WkP&8s5F^8F_~_x*$YfGdj>V4Tk)+57($!w> z9nq^<@^R8hpI10P8;<54{>t3`?sPHOk^DV4xwGBb^mW;NOGP*8o2A7ugxK55qPXJ)6 zH!>$TcklSADQWkQ=N{x|F#4+YuUup+a}1{-b1e^lM#Xud zFNb~*{8qNqV?%V~Yp@@&Pi+5N%V@ZhZwRU{D6>8~g_nmyrZh#toIRd?EGhU6(Zncr z@)M5G3li!jV#K?Fq0Y)}5bSc-`;(1;4p`_tN?(|SP>0Nh{klWWM9g6x5dKqT3uz-xAjf?e(=P#Kj4)uL z9{i722A@EbF2@@Wz=lNeP>6K*n_L9}0yB5PB5wZimAu8US0;K(;T+fACf5PsQoH?b584@ZhllEEkDC~hK9qg@=eh0 z?}e)*`70j*$)>7Gi|S^}?B<9@^MV3MBQAf94UwI8^55=p)%K*_qT1dC=cl@U2Ft-KL9b>{eWpEJe!-9D*LNm@!{Iz6a0s=sFBeVRP;gh(;5N)Ghm7gkepDW4z{ z@M{*6BBGVhp2%cj@&DHdU&7UM@5T(xaOb{5NO#eKa6zB72~m1&2PS^g#T4g~MnqJM zng<>KAfqu?U=EkTfsYD3gkygio<>F_^fB`~j&6Jnw>t2>`M?47ldvh_yhLJ|_lljD zJ>YeW?uE#Rd!}qQO7#ESPh_;L;wdGw*>!#kyh5)e7tMmN)ZE9+LQ(LrEGtgakzt2T zb|RU{EAtwH)B!(Ytl3mlft$Acz9vh)J&72DR%TR}a~zcJ#=eZ7fqR=@7n_oMw|_7Q z83D@Zhidg4r`dQ!VL{yhGWe-!P+=1N62q=tjPrtO^=>{33Sxm#fjjkxp%7q8q~l!F z2r*BxSg`bdEVq3 z^JM;KckyGmU@PQrL$^QZV9UacGYLOS==boj1}@f5wpPDtqK-3`SLsB`x@oUc(=)hO z4(*E1tcuS*WJ@%jY}*iiGhcqUR*0=7u~pP=h_`04D17wyzF^RjBwF5Bl1I1@xL2eUHsoi^O!#%BFPs@JcQFD z6XfX9e~TZ4MBb?IG;(#*yV5Zwk||Gx5xl;mdi5mSskfnWC^#CR@{~){sTXnBJ8i4j zx^(P@GLesSF{nUKC!g`l-7#_5&!#EA{hi_JkIXWar>l+b${98r#r-#?QOR7W3!{R9 zwkyrge<$rnVm}-nnPV;_G_FLNOD+qX3+Nn!|0>yxmmI6~dFWOctG0v53nFqx--DP4K8ozDI~LA?b|wT(y}$S`CY3x76;)4WhgVxGXQo;TNi zF;E)kkBC*7H;7I#GqV84O0de#$z_Fz+@7B1#@Tf*WSVfJx1(jL(~8+_hW*oeSNj5V`4Oe3hAk}2 zL~kj4-e0V16BLgk=j{vqu^Kg{81;L(ZCo`lX};^{y$#Y?0KM+i>lVA<|9vy{pKsQsHG@VPPj%gW0$_65^zfKtN;cXCm^^YFhS06TU> zIp;VWW8bKBQ{(Um$&zzwas8EPr1ISe{b8PnLwZMr@#GQ2op+CNsrP3L*K($*gDU$~uJj)#=vsh-cENrR-EdtQ47? znpA0^Rpa_4W?Y0mp<>-4NUT0av`Jvewq(7d+4kt&4ngTsb)on5MZ#_rHoFS3msG@$ zNMhwQ!HV_$+9UJqb!0)N$raqFh@fAsXR-{hfBmm01ID%c7?Awu$(MLPO!S4fFl;f=ea{3CDLz+^*~10XlnC`v*KuF*9YyvmMV#D74}vfxSptg zTlu?%NhX(xka7FsbfNLg(x@e4!XKS8FNH46vnPk8j2o}TH_y-r7Hm4(ZH@*)pO;z9 zu~&Vp`DK34LyPn0#i(VEsb2k8G-4&J$Hl#&Vhh{bZgOGw**=cw+hTzPla+z$;1lU#Fp9H>jD}5M3=EsmPVx5csQP-Qs zXcK*%RR|%XpyYnY4R&HW9TU#?e#+PLLh0#aviafLA>I;bOhzGuB>x>ItchqwA))q?E{ zXE%A#!HrBW{#8QNl~(4hK8a5-uAdKp-`;Ul+0RF7@PexUZoUi((u2*LWxQt3LP$}+ z%KGJvD=mB~e`?GvE+gE>v?^Dve@&jofB{*oR%<&9x~vXGl*6?d!pqZSFmPYPx3hbX~5xQo3@PG4%N z{R&fmJf%aK*hvQL^4kCZ^Xq2_Pm!_$v%i2a{0Wzk6yiS3hSR}|sCu5X?(G9M-=P!OE>JN%?D_=5x z>E0;O@=6c9{GtGopN4q5#^EHb;n?N<4^j4Bn%grY9n@E}JTXKkT!fnYe1nOtXFuK>WUxG!nW5k{NcbG?k7a&)=5L zXul{e;(mH9jL~kbS{H9s$TH5QeuV49!Q#C3GnTL~_%HS++XA*2%C6aCO1CF|W<+=4 zRD|uMxt;MVA`Hj=6&Tve!3ei#B`y(F-geeyItM3bNpUusAycWiwlz0e)&OBR4XspA zx8wo)lVZ`yM-(kwxK*0p?2XF!+@wL_dwHxCOUo_@d^bX#(tZE>m9rwk{lsHxxk>`D zH|po}SOCrit8*CDZ&bG?Qa9Id(8xEsn7fLwie^44dU9708D+JeG$Mm5VfTLopKgxz zt0XXzne9YKhCcqCShoB37de&+XkmrE=o8RN=;rV~&KKY$3}^^e`1=X{X?6!#u4nNY z7)zQuo*I79x6GAHf^CI3K9=}#C@OE#X483VDdiro+OQ{&!JtQRYrYQ+`^Mfk!-G^F z8wX63Zy7KBI<7AO+eX`GXf)>j?1OS(*w;w@&`s#BanJ?OQ~~*v>)}iZJOZELMrBVu zUX^!Pd51YsMGKb2z%B*z4P|-p9^zXnV4?omL_1C1C0E(2@wR*VyD~?F4E?%SXk;#0 z27LE^{yAay7)o(DHZy0aGj{q2WV9npneAF}A$LaZjy{Oh|{8z%|Cwi@(^KINq-fiG7;67x2y>G^mGh*xzs|+ z(7NvND3_Bq`EN|dCFu?Lb|;4=1i*)8B|I6gSzot6gi`+k9PgQ6SvaVh#sn$!i;WG3 zy@et#KKQ}`Ez6u6ohv&})Np!)f-YFy{UK~oml7wUTY&an@VpB2{Hl~h-rG2x655wJ;1_m&Gr9mpjn#BVOdVY;5QxopAwB{WpBH0?lN2=?8XCX1 zSD}iUBxgaZHRoN;PD;u!3)kvDC`p_?nkh2KyY9>U$Pj-6jBeH6_r4h=x=&%ET(cH_ zFjwUZ5&$`OAryp(<+*_2Pz2&lns~#chRnF^AY=<0pZOK9b{Bg)f3MX^PYJg}nW&m= z3v^&o^3H2xVz;AD+6?hwutEV7Hqgb`yp}5x z9k+_=ci&WrS2j+(bqND;sN5{t021?<#ue;;Mee~P-LX$H^om1xY>Le;EH(t;N~!9^bW~@WJM3UbN~0arK+SZN>En5HO#z zVM}LCKbsF#>K_5TGp15nRy4DxL`I})Gy~yd|m@;{N0d&g%s6`du)1vi#Qyt>6*_KI(Sk9i#c^HYw{@P;6L-_4?fB7q z7i4NDpu?DqH!Ejo7+r=or#IkJCr0`tdgtO?^%=PU)S_2#<7xejMS@~%`3PzV;LjLlU>-vW!In`$ zF74g@4Dy4^Dytv$;;)cdH<1e^8?Wt%@n*rqy&H%7P(f;#`)Qd%Q#XAvjex@kvEp|A4PSP4 z<&S`yomytE4XWVSD2I_>zQ$t;f4o9mu~b$_1fqBOtehXxaUG|0KF-aFd_-M6}V@rcAY ziLXFQEa>FWj5twNU7=N$7E+3gGN0VNWsx1>Asj3XK=52h1{_0rJ}N|)j%fRNIPeBq zkT`s+>CpA;dTwDI zoo}%CJMz3)(yy`|`Eu&<_HOx;68YuVC&d^w@~@U#@4!S<+`B8l1Ha5&P? z^71l1g^2Us7z9wOxH&s-9AIK&e`@!<1_sm~%QguF&di^53p<^6@9|ue?Bh+4!lk|s zG2iW9v#fIciKE7E+ip&M1?x%#0E~)to7C#C@LT;t#KX3zZKn>e9k8@gK`c~Kgp(7m z4jqsr48~DO`rPvw`QN>?u&@vj7KWsyZB-13|ITW=O+&;ov~xJzrT<0nyFOVO5*gsR zsBUz;1?ztzCV9S_o;0h>ObjGOab*b4RL0@eSdICPVy#IthXa49uA=f+oaK=hw|{gG zSZF84@s=`(e=)4unY9hJaEB2CVLe@t)~52aMQc@kY5fIP9H8eyh!vN6m*2l#IlzAW zo-}t|<_|JUPL~y_RDmvjKLsaJ{1Q%Y#fiV=l!0%<2E!L5Sz&$`r&ilQ`kogpKQpt| z3?bF)dTl}9x_sY4t%ua_tI-JG4Hb`Ccg3N1OhJC;_CGG1VDwS(cOBU*!0W3A0hH`d z^TGN;4MuOW6zvO5@KCx9Fh3156z5a3YOY5pvhJFiwZWp#2L3{seLb#n9_oS`3Bjd! z@CeJiJ!9un?C~Ms{kN0ekBIVnjo{B_1V|TDFN-t-P2@@o|Di36Uh%$io+(HIMH?|9 zPj?WDB1Y2hx*J-Gx#c#WDB$`u%URrI8=FhTBS$gNBe}5{T%5%c47$9$fC*^}3}wYN z1T)9LwDNf;Dp}4w<#yXbKppKTbNb;`Plj%g)YLBXNUp$-XFZaLzFt#HOIQ~QJ~z_! zJ0Ul|!e$Qgz3CU%<5qgl6Py$@>w6V)Hp`$Udyb z9=pV;B>}Acoh$x_bhNY`PxnV;w$zhXL~#w)5XAfLDgmK5cN!taIPl;a;tN@&~BYr$=oj$hwBQHMgwS{Z$uI@)sBVHTe zER|h+VtKPa-qlHfAi@I2qRB*dB?k)cCjC_AkYlpSmGDA5r;u!$+LS*2fV73D+@3SD zXRJ?eUlD-rGYA3haFah^JN)wH%k-U!o@LxK_dE=^m43=(2Q-w-S>RKq$5wm`{d~^E zOwDE0Yy=sFy0^U$5oOR!m)cMCgIs(l#PkFL=!V`)br_r@2tdn13-Qi5JQnT!?<2XP zq@iJ9#;vY4GlGGw3TkS;4<}6}k`?{sXX?6gNVeUHBz-EjvEQqu=n=gmu_?oOy^>9A zNsU*Bgm(uKVaK5CyjDA{Chz2J_0J ze@SiYgvTp{XDc{|o7qP=ne3Hsa$zAj`)(CYhX$RI|4ruMe3a%(_I;n#x^y)U?S^>j&F`egt>t#f#J*fd zDZMT8;>_y1;@SiBjDtOhg23_nND#_Zoe8e$@YH;=b1^$drZzOfhKgI`S7Wq(UGU2|-h69wjO zwGG}sOeKLu?!-6QT*Yte%_M;|$w)Nu(y{(UpJOy=U&@QH9L>+?P}ATP*y66HpfI&D zO0_rj!I**h=r_h$l#_9k74j=Ao>1-4I#^~M}YJh zpIj)LrNlJY#9f1H*G|&+LCS%eQ&9GI;XVB54VaaIBCP%ZG|>htBonP|zu9>h{(b6g zyyVDMdW-Y3?1$%tx2L1eQk~dgVumuU>mEBV`o!_ax^lB|$40}oX5HzL7kJqf&lD$j zZLn_?Kcwk$B2SEc?EEFRM44Iv8ntqx{H)H8U_CVq4>qTL7h1Y|R<`U}4bXWn^vf{C zWKBsMYDlcK=w|vGX&q9!nx%pFkm1ucDHvUeh_rZOm)bs>okWT1loHFWrUKqz?!?Mn z!mIUp*!w7Hf8YpKpz+74&cjdid7xGhBHrm!`1gvIi+R7!#TO2YCh!zEG<`B?ItW+3=DuB!&wbV20} z8bYKW@*$!pC)nf);IdlU;49kuhqRMQ%z-v^sXu{I^y4Is9BNSD+$gZ+thxY={4S0r zF>cA}p{(K$0>Vui_4SlKv5g9p71SrukxS_}Zx^K3_Wh*w| z{dNB`1X;5G2V8_9LxfD&n!GRQWM(hyR2+B!5-lK9WF@JuCzZZYwPmE7xdS_NQMVtu zQrPnpKS@mMn^pV6dc&9wetTFWuFbcnDTvXY>J)MuKyS0rU$`*2=I7&0^;t^wm+`=_ ziUTHB)_%ulhkNhIo7K92V6-W2y<^C?XLe~`mrDA%&@5DMf-Kd{Zmvo3~_0x{C-O%ZR^*`!x^FiU=_;n6=&jSaeRSarbXrP z)HpTzz$OyS&~#ZD;AtLUvZ8MROn%x)uNTq2K)5tLzqS_S*7{vb^UuIchz7q}@FP!!#`iYIbHV+=< zkfTv-c&)5eh=)5#Me~bgeR2s8SkyO)eh9wVi$r~5StC1b+jgZtc#eey%Tgo3jbzGn ze->!ibbkAEBa4j7HH1(Q((d)G8*wQ_jcWk7`&TK~j6o+6C?{CiZyPCgYUVX~ z7qZ^b7wq3O_1`uPp!}w&!dwNWgq<_>U1P;x7JGo;kObY>Us>jHmS2{2|Bb)II($oB z+gm_mlOC`iO_)%3=-L|k@uQv3-lPnF+(GnX*pb@ry)5kdWakvkOP7v|wLC(9qO(@0 zw35UZYGs+Hj)Ksd282R?)upII&GF=p{;rIf{M>g@PJGPSoNbnDK+H5z`W1 zSWZ~=jmh@1nfGTsuZ21<;Kmbka(O!xdBO$Q&?@$nP(HihLocdU_$Hb-)WIUMrHjmK z<7et&t7g@^QAo=WvItvFG?1^dXM-RSDYme*o5I!05Vj?vN>O%GqwLh%M0J7F+f(1M zW>vp#XNW(ufrB37Y5c!9BMZq{p9lJ}IAl<=bZ1%16%4Ugfk4 zz~oUYWoPKme!eGV){hYldKPiFUqqKgSRX!_s_TZGaGZa!-)kdLuG=fX0i5}j*$sSF zpY@x~eyQp_Eb1=h&9^CM6T7Pd~&c4;41ddm_SfWE)l6M#{6> zmHL_4bfA5$vRec}Aa$LvlRP1(?zREB0oABz%hB?&qKap)>nd=uve?9pZb+x+(k3w` z*cWRKO&q#HLW|ir9AqdqI5hbY7oOGrEAHum4=|tDPfz_K>{w-!LQd_lvu#>GK#$xJ zlchL}V0avWn0glnUyw(i82Dy-nz9D-2E19U^M$`hv~4Ip`bU2BvM=V#%QMl`eN9yt z?_>ecizv^D)-x6R8SvaumXU^NZYSf6+ls3|4kt<;Xo@uv5=z4 zh=_>83I)m0C!)|ETB&{%depyyk8*WxI3mc3zCgG7H#Wm2*V6skjCzV2M2)n>JuZM<7ZMTASQU-tQZ=XpiMu;#Cw3^2MIPn# z$2U~a+GR^yKSJQvT@aA(okH{kQ=YMFEk_x=rX}x3@U?UV9GMqmd7DdcFryrMHT_J+ z!6FFuFO36$O<#9!{=rq~gB8gHozNfG-ELC^&hvQx4petH`qDk)oyW+;bo8C|^*3Qi z#3&Q&QS}tBU}d6D*EVn!rKjtK%mXHYn|Iuk0T(OyJ+O5TsAOAPh{kV|^h29i8UAgS z`KYQ9l-t49>dL8J3~^<4o|R8T z9MFm; z0uERwRR8?1=1Q$u|M4hC4d&NpVTViuWWqvnk0#0SL~QnhLEx#RIib5-p9@Zw^}up5 z1lfE+SsBi?+V_xR&@zsO)At&b5;c`#GcQK5@?x@l=a56%FPe?$8Lv;;5;R8%!{p_p zYbl_l!Wnw?5ie`ld46iQo4EcwIKD^2h16ECxK8B$v)U=*TTC#BzG4RE23YTboDIkB zj2<*puDUn0mR!F_ zQc-kB=;~Tv(6%NcLegrf-V*UA2n`n2H70-77Wk4C#hirYtqb8>ift8aQo@(lWF`?s zlwRka0}zG3ZZ$MY3Ll8((1ZbLDIj~JqB{X1RzHwtV0A8*lD4*~DqoTgrNots6*`Tm zSm=;WmJ4KRnu+Djxxf24tgKtC{B1E=4>H^cDvJi|rjD`lLkz!)gc)(poBHNl zE-j&k$%uLFX~?&Id(!zkczC#*mf%-AFS5F=Lf4MHn+7`(8ZoYbR)uZL-@z`-|MRel0GEWk5YQIEp-Z(|+E2hZLDc4u@2M}0*UO@obVC)Vp*QuQ$tPkBFNtg@ zc)Ojcte2~|U+rR0@owJ6d+(R3_mvuFMhPF3tftw8o>uO8!U^II*4k%INbxB7`0<<& zNfPh@i7Bxv5NPgW4xbO{%iY^5=AIC2?}7S69;S$|-iZRCGo)&>Y8yr2k`WQ49wPeN z=~{-iohKwh|&}xXA7^*weMxb?g4vfPnvd(x*BS$KR5mwGRtFn^7 zeibT7i~ihT(tgf>*-QHL4DJ#m_l889kT^|NzsOyr!-=_2lns7*;bi$`3-|KTYQ2mv zmIqH@GlmJS&z?IT}nt zHrW?}##nc$x_@z>k{OsP5c@^Xm+aftak{o)Y!Zf%ZRDw4%oiz(z;92)SC$7n zraO$wMHV}L1q^#Mg)Q-?fY#?visvpLS%Vq*Zq?i!)#Y7;8jq*Vh;o(BO$Tl}blOVt z@h&K;chBege$3Cn>)f%9>r2nJUz&x1CSiph?5Jwp`e)YAM+z53Z*sino!$A@A)%>R zo%be@Yh19LWsT=JfY7#tJbN*5!9C#nnYsL3YZde2 zkQtAAmq6&rWnWPL6a>_*#Gy<}wja!R*z_p8d1eL`Q=5Q`4l3#MX>U9oqcWo+rkm!D z+%(#bm2l%ZsIqh>G4}mysdkhVL)AN|bry=lM5~NzqnDkf!{XG=*VV6`jlTQe>c`dQ zDk{lDeVR_Onj|y+dX{z}9L;b)6!~8ck+EQ9@pV&PvMEkRL_eOVJ|6aVZj)$fXoROuaa9V= zRi3qE^@@oXdG_zcrRrXWCk{(aof_9gWP#TmM?~Ut^uNlLRyz2;1jzw$k>v%9!h7#L zc!#uYp4pEZ@ku&C6qnLMhSNzHmN=B{o0KjL(l?uE1ixw<=E_*L97RNCuN4*t%q?u< zwY}P^igh;5>u{^tFXck)zLPwB34S`8tYeY|;`nY}p3YEys1ifW{2DdBG?O}C!93%7 zM=&I!-|8#A)465?in|m9HZ9pM!Mwg{Jl$JOmS2*|Wuo)Ycc`a%cQ&z{nmAC8ti(YN z9;SyzEOz?8vH#J+DfD@+u%g}uCYt@B^WQX`pAqqN?~fG+5>4z!ItvgOvlsy1E&oN7 z_l-`9yA%o#qT5TU#v)cDTKXZ|d{74uT40zTZm^iFb#iiwN`yDxsTBOeL_!f} znGlric$x+cY1O%%8MXI7UU3@uQ=OUNug7^}ZxC&$x@=w1Slo~Zx>V9vGc7dQE|=yO z6y|9-D}P86Jo(flIZ)Fi@XUwzo!T{FKS~>N&n~LX!1@)CY5`ZDB>C{+19zimH`l8- zdFYn@=~9kfcwiC|5+Ktk>sfO^5lx1oV@q+aousESAOXzIr19y6RJGsL%gP;@=sU=^ z8&%bwR@GBbJg1)dC2{dyXXxo!RwX&sy4VVt!LBFwR7npHQS$axWs+~q^r z!6Ze|6ee5!w0aN4i^Be4G7{wQ!ty4CnnndfaOO9gSmN5|Zd#9RTF0)r0!2}FqLUOR zkt!u;W7UVZS$a=ozdd}1l5?wVkeav)PA9HKKJK_x3ba)TTU2w{z7%w9(4KZ=qs!=t zFwDCM!?kOJqEl?CTqjF52;Yg@j13q*@Ob%^lOi!$dW%X-Hh%HSBaWD%Q93 zGJ1O~z$egk{ZpqlO`E65K{D+%fmUltBdSpai5vHUy)W!bSF!lBKj2)uaby!ae#1fy z5|Wagj22{8ljOo`@8qImG&16>%VSxMS=@hiU;lMh{dI({t?Iq;^pd6&F%U!&?fs2JJ(-Fp%BA)eH5H~L4xjLl&X7~F z>3-_w3ewT#Ar969HF6X z4a*<8$s?N>9C!*MoL(mE|DMKY@5$$<-a6O3yU6NGjfl^)(~`?9k_SdB|CaLd%jUZw zhf-~Ex2e>6zRuA5@FC30($d0ckkolB9nuJhuav_|K1W(YJZ!%eMKZ~*&}}QzI=@J( zWp?RA1vgrKob7m+)2C{EsNARgh1iv+^n9$5U-RCoam)1FYVQ<--6Y0!LKpI?#O}|d z@Wt;^9!`BWVwdeYpzod{UbH4ZZx|dN4O{)eq-!uuJp2^=d2aN53yU%6or#YD=nCTp=Wsh&2+l1ukV@o%gV_2hpxkq;TRP^wX$*XdgdD!fEc%{A9 zFJtZY`G9HfatG7U^?P>2bN>{yob=x|iaU*wU-btJ=bFWPiP2}biQm3DqRTG9Z|slU zH)h2^CZy9JAMv}tMF8_lkD38Jmr2rG(qi>hFMM>4P)>C2P#$zpC?C2&s31Db318w| zs1Vz(J;`qir~qLBvV)g*|B#1dD3nOep-|cTSFPDI6o$|fFmIGueYc^sYC-Lcm zc>5v+<+vv$Cre|Qoj?>z0|bdklQB1`sxX@$CYG7{`hpj)>^KE@StWGDTN7NqPF_(C zM&L(+E=}7vc!o$9$X*w{9qyC&Xzo}5HK)*%_^XtUZjO!iH#&C1_^L(-5|_5UeReSH z5)Kr_^gP2s@dSTkMNi#thUF(_<*2D~4irNFmNRo5Alafm)$Fs$unB%Yrbi=T1qkjm z?gR)f!CeE5ySuvtr_m5xg9Qr`+!~tT?h@SH=6>F9X4b4Vf1$hXb58Brb#4BX1Wx;T zP)3DXs7~5Wil_14K(TN74sEzVu415X_tA5p-OB;!XZ|a>kgrXG9dsiq)rK2nbpZR_ z`RX&}nSYDqAeMFDQO87(_apSUZsPCGj2qR8@``!bP9igc&CM^V{GS+Nn&d8xEN7~9 zYbOS<6FclbUFjMKLWg)B@HhxW>Cz{Eir`eK`}5V%7g)M_RKMJ533CLcqAk&sajjAq z^3Vz)jCp)*+raa^Y{$yDBWK3o*CzI;qx#WB8!;jw{JPxX%GAPC_2caV6WLVGCbtDO z3-+orD%^G>pOOB{U(}nmz<5)L8Mk_McU(6mpKsP2Ow8{7pdqT?^;YIoFn4b$lGrEv zD@Wx6^Dnx0=YfdJ+u=^RGf5)QWacJ#BS zuI@?0tFu}0A17D)`V?TI&LwmR-ewW=OlX3Q;D@-aKGaCSqmCcQXWUz&Yjd^WNEE6S z0+RD9@#AWhK*^@EFt`7K*oL6|P6Yf)jz@ek#n*N+Ha6C6T9L@pf(%>t(?LIog3&{r z#pTlu-wYjHdHgm|7nxY*|NolsziN*Y6RR~YY&56com_k`QR{*CXRQ-IwVdiiSrQNs zC`>=h_cueR7=IC{7(cg^j8DmUc|dac{ikFsgGr2pN=#JU3|4ePYbVTM&!Z{k`n3af zm#af_(Aej$`~I|j=Vj`4fzI_#-lMbC88TC!r8N7tNnbDLNKn|Y?YZ3p_3q@&#TqGI z3m>BX@~1K9jeO4>_(Qez@#ylzjSrc$t(WitxsXo zYg9e1J5EmYDEvPo>x2lA13RUE`v*9gM2YDraPTMWSMUT!!`DY6E~s2~qfYipuJH0m z@Jc+Tg7DlU>gWq^m}${C2u<_(i&<9)OAAx-$YB2WtEjPlrlQibly`@lF~kR)O9aR$ z{C?DVf!7>yXdSvV_0RjUp4YA%YXO&I`69PRi!jlctoz@~ANp$h*86bm>z7t{4}v6? zw)W!T+xht6LP;dwBmSoULmb-PJ4J{{(Mu891s&m9Tb3$e1{KO5(gH|Om0ch67lhYx zw-a&gi@D)DfG|ddFqg(Z`|`Czf%J*b{zd2fz?|ZBLjn_4^QPc^QlMk->ciLr(?AN^ zw)Of>%(pMhX?rd>F})kbav!>=amo@I6VnMz3gza%?;uUT`W+Te@tC4fzW+WDoMaZ? z*GCg?_%cKxOmzgw@j$-$?qw=o0jvb$CdYoyTP@G3)f^DB_V{DvZuWz9>e;~mBVOGE z6WVY5x!69)UW13BwJgOEBgLY==yckG_OHWspL~(8v%5YPo17IYmVSrsIwiiSOGkD99&}EKBxzVbfu^&40An@_v(l zr_q}~b<_QDmHt4U{rLXik@VnylgEoe?s|I~#}pau0~>T=NP10E%QD%DOv&D+=Eq=E zJQOiNzz2{DMt=PRTuh4{{lc@3vzYPqueuGEf1kL-`#%*Qx!J+u%n7piwYp6CgvzQV z&Cc(xI<&8q7$ZNQye?_)KK=~FH{ zqhNXtoCt-D|L}$M6%#VifV~4Ln^rhx6O|%oL|U>HUh<{v1-;hT?7Qz{otEnzWeRU` zi4S{WGZA^^SOSLRe(l0IZjIBFZY}e-PIZ+3Op>twi5Kt{KO={=DB%Q%Xl!HjFv`e3 zIk6q1lK)t`}+7a(W4_xid{hmr_!Vt5JiJ*p~xEiGmKLtkR@ zPjf_@UGUxEyvl@S9NW&%lyiI{;5_R_7>umrpE4qSYCiqIKP5z~R=6d`;uMyxmiVf0 z&+8KkB7K*mtBT*d4PS9hlK!EfE%SX9@ysp5?3lm_Z+t|vQTv8S&qmTZoP1$EK>hk682H=_PT1zlE_?5VSkZlk zP^wFi2@%J~hZiEEnU7QGqecN6*)Y5Y{(Kf2+4i~C`<HiTl_iz>Y!n6N8gX8LrA z1Y6Jf9V@XpLhM;UwJF<)?MC=rf??bL+8ych?A$Q^id+F5Py+K!!Og=pa9=$heem3E z`+Hp7H)>1PW0;cnlAf^Qzc8*A39nx4j1FET-Xhf{UjO_Yl59xe=F#{(HMdyr{VJ-C zzc`0S1M#GT?6pc=y{T}f<9p%`0|9V# z6Uo>Vfm1dK?cB8zeisx{>-q!vu3barY%jcxLw?WpO>X5Ik-=Y>-N?jl!4$fb0d;I- z^hP#ONA^)Bf^qH3@1n;a7f8wM;{Ou7(yXHvh*~^sBz?<~9tvDo4m`vjk~(whS>DGy z)5mJqZvBB!jb+*X6_*d2RCINP`~8cWh$a5;S~ck2Ht^|;3c~e5nhPykv(wki8SM6oT4cTe>m{+ zk|=s%3m@78zb+%=Am>Dn43}2)KuZEef}QB||{C(O)VTKHOb#qaQlhd_h>ZMEcIu$CDd zaWIJQsr=&YIrx#Sj^Fi(PSl6KNJJf=Zo)N~dGJYD0za&cJl(kjK$R{;S)ceR5omL~ zO3ZE)$NZrK4bLd#(v+?xRU?sJYaw%LroeY{MZmG@Al~*iGSYy76-BCDO`jET8$U*? z@+@gx^4gAXTlQX70;Ks$)s1@O!I4neJO#Y>L^AKbDNRR;oP9+a>f;>UeHM4I+S#)e zuQj$&sLeEJ*-6)5^}GKmqk!1Wdl9WxKJt}J<1BloUZg=p3usUA45iuv%S1eW4XgP4 z$OwFSGUntrIcwF>6`h!dv*p(l_#sr@Z}2bk%S2L`PG%_6ENiO{@n=kl>c27dYE+Vo zSe!zUspN$Z2GNE|*3SFzA#dT(3dQ}El2U6_Lv~WE+4 z`weMs%4}hWT-7Cal$+I2MxnEQrLnL{$_8!jB(pKH$P; zj{ir)XFGai{imp(tJ#L4Vsn0t#H{XJZ{-F2R1@a?Bd$I3S6Gp%`YW)a%b-WXmRxiT zJTRuFU(2%LFE6!EDOn|3xweZxj=OI<@4g-j!SPj&hA*-0o*W$7Jawt<=e4sCffJVc zB6HSQR>l56miq>!Ds0u-CE2Ki*O6QfKFkj14ksV)?v~*4PXkF|D$2?OgjK{Z9(ltk zz?2sLJ-i16JNAK`GcB6y^@WcubD3Nf^KAXDVN+(RSPqUa9s8r2^ukUS6K!Tg6Z#{2 zDq-%3vKRsB!tvfhtvM*C|Kr(YPm%lKzz^vXG$^vLBZoHI@=%fht&o1Y!BGU0PhmEqeZ;9hmIcig(qD?KMT~;WCed9rifi#-DJpp3Kp z4MxO{FRh7TGkE%CqY3`~Cy?xqP3LD<>r)mlgQHI>b0}ZPelN@&`;iMTAtp#dNH6c! z_U3_ z&Fm8pR??4`zgi`I@#uOvPdqpi%mSPz?R292UAoe~bOg3QB!ODl8!cN~TTUy5WNW`& zj{tGea@C3Vv8nMxRi;nO_gWhS@qY@as#k;@NVM-xQks*zk#a%BBD_{96gY?p=CJ(wn(w21b2~t!HE*{i zNAs8}QIcwQGxAP%&b2te9ratU0Ke<1Yu&~aK;4TVG&wAG@}i%@E3IuiX@ z#U%ss4J28cU2*rC%Ctro4Or84yHe64NyysK>bD?y#j=S*HeOM+Bi~M5f*Wy!8L5M@ zgT<52Go{o`MeD(i804FOb!v*=z7WE8pzIZP5`}HEji;j1#5u!p%wHcQe3CF6qhc>u6Gg#Un#gLct3-6s#6A@6o==$E)Y$Ox{n zNPfDb4{o}s<2wS}b{r1hID_8G$Riqizm&fraFAhnN2&@$#O#zPMcLIy{QPV*FRuqbW9u$>L5U& z3QP_WE2oO+6TX`?*I0+X+29*(06XlVhGHgj{pE~`7yLpZ1**Wr!a5Kj>v4gJ+&&%u zRLLm|+Ov5#rWB>|K?i%3cmA7C^Xj0w5POdJgOD!JVH%R9w=r7cZUE8Sw0 zMLs=~kX;Xs*`lD;IVt=N+}Gjc@SRcOxhkX8+SLm8(TJ7b9%OEwn}QM;2uILnZV}ag zuJuehxL|p%Kh1p~R8JJ$tqen+36!R3W`a%3&CQ*|?bAVU>(0i{teh@<{p*UyOCXhx ztOD64AtWaSH}8PC@6nTI9AMQFG~;8>EurUcFzwAU)B5uHyXEojH8}+vYGIFxWr-mR zdoUO4S2^$!(`pGc;0`~Jbx1mm;-i@a#Rq7t(?Z0-+)p5+gP7!W++9qQ(5;-xFQ{dF zWTaLky-5x6QzVm`!b^qhfKm7Fny7#pl^5WZ-+p#w{tyH@SMKxLUH&Am&0^iSlyEh~ z;gUBD>kcBjHW?Txj+s=~B|$Ch6%sK#%OudroUB~$06dwj-OX5)iD~z_<>)7lR0&fl zW&la3z#!&8JEOYZjtPrA_>~Aeu=?y^&+3`ksN%S(QW+D-yJq*TWZ?C@wj7%Fy0Y?v z^WJ#KK-y_{^#nla#j%fT)Pj8~RhjQwN$y(ThMg5nw7T4b_~+q7w$QsF1(C7O`QDqR zN<9$;CG5YRxEAWnh@iDWA{uq?ez?vbj7o$8n+PR9lG^imc3LmQ1dkuY%p98t`swi{ zYFxFn$YR7`uv;D1XjW7o%98;4CCC}Pj{7w;azAf7#W8Uy*Gfb_2SRpk?0y6Rk9KUM z^5t}k$pPcqh}6fr)jF;iOBEE8%Au@VFvj2 zL7hMvCq!NQktl;LpN;gRF?uZzM{l~XbZM?~YP33SnrFeNd8WjsgK(IQ z+16$dNp38QFjuIV;`*AOjqM*dabjQ3yS*re(J=+@@2GVoUu(8JSYMonnk%IH~ zt1azWMZdXM*9=x{ci1j^Cq_kpzbJjb(f5)*^fJrvi8bkmccDnTy8b9RO*cfWdXbCz zRV}AH>3nVTb8dTvw+}F+ih_xegB~Zk5w}7>nlrGfYfEnWiosYqHVrvmOF1X8M6V1Q zBQz@4Ze`@`Gx*4hIuU0v=)BYtOM?nGG5?qarVHv5A}9H|$4_#10sN z-30Jr1wS!A$YJ$rt@9pEJO830c^-vJ(_FPWX`V}yCyuU8;c6rT{_d$780<)9h z0+6=x`{WwhwU!>fl>pjU@1}%)mpoBk@mSJZC#v^A_7hvjNpAilyK)})9a$JHtulS~ zXLC?4%I@V4Yl0@Bvjn^-Jz!7xX> zUCQgBJl!o3MG-=QKxqecgMzs)#J0t>v8JmMEm%3_t6uvg@tcZ-qk&sI8`S%Ce+k^H z*Igpu=xdhGtuQ8SKEEJd@HYf^L#i=x{VfruVX5&UDV|zk*S`e`K7ahFgu(s4qB`F$ z#6B9o4nxND-o*`W5;6sLJIx;9m788wGFto^Eb_K=tVzxkU!N^Cr+GGZvRB3v4`gHs zTmxR!;Qk zbGMoCZtO{cUAkegNqwO`DRs+_rKuc>_SX{tQ1IJxp-17dEK%scRfo}Us0&vvp^`TN z#E&8g9!}#Yb!)5k&E!Ta9~jPc29>WG=Kh`pRc=vX6YsKMpSVqiHUqaO+IVB;THcA@ zkL6cCd$eC*EY6FKpHaybUdd@<_B%DSL%llV9in$@jd!=sgAf`l%f?6%L+lp8BiH)D zj~GIE&0JgFq-BE?z8*&&^Q~*%bMSmJm@l}fpa)au66~W(?JVktENZ&`yCjoO8F%8G zdpDZ%zA*pDQ~k%O#M5?8>Dj#DMOnFg8qVdub;bQ8^KTmk;s>0MZY~aCC_7@)coNBD zCHDLPOu1MgOntC%uvKW)YLpZ+WOOXiU50cUQ^%sH>-3gaLk&e zrufx=p4VE9^C@Nw$_9yh@d6)YqUdf#@i?P=S0Knrx+y7+HH9MM65bc3G zFfalJ=@xeq7}P{_yF@!TiY0o45e#o>gtaK{^D74QCD!F@j`gpj zZw=29VBbYN7c+f&07x)``|9P8B9xgYgq{=47g~b!Sjvm++xwR92{F1e#_vlP4x)Kf zM4mi)UcdF6WBR5d)ReMJ`6IKgzc={|*`dVjJ&D zZ*%j%a4abJwUJ!UPxB@x!7vAs*gKh zz!tZ^k(-re)h{Zuo!<+De!NN_ikkC-mUfDC?{DYs*PpximT(UUt{wsOuX+YQkUz!4 z!xhkAu~UBzabYU)sn{;F&vvxO`mgEl%Q zXwN&=S|kK8H4E=w^Y{|y5)XR1?-2;#JWD4!vY?rMh(D^bJmqEZKr9n>`WWK2z<+~k zz-k0*Y;JZ;H_XaJ?yU!~;`H<~DBwvc=d(dmrkQS)dY&`_*c0$I^B_i+u-FJzE>|gM zbc(%J+UMvtj9dhT9={k0St)4!moloJ5SFlUDl_@P-Pux3E;<6~u{bThzJFw7gzlm* zkS7X+8${mEt~N9XLCY@XPZ33SK|K8p5En?nn`pwkvY*`V&Nlq*HQ$vJH_S!--AkoN z+5hceV7A0Ce$=$$H$Qe>n~lFDNN%Pz;lk|pYUuv$WPo=7E9`y7V=E-5XwHLi7> zjT?)KBWCouzoTTQWVw*=qhyWmX6KM_rt=F527O_P9B*hmOpY=dJbu55{j45 zdI=VrgunPAUA?;#Nac#motEBKzcrYs?v8GZ3K&?*iHf0)v#}31vU(2Eo0s@v(1nZ^ zr6lgjRwrPz8Q(fd;NHe`XHt-O=#%m$4D|XiO|znVB4YxpibV8Fo&I8zT0a^0q`9tG zF%~B@{scVr?^}og08_{eg8Vzud=h%3c3c7(CrfvZBiS*=ghUayHKg|(qwxX%3iwk} z*$84mu#B%S`LBtk6TEi#=1&?Sph4@+&`s#_I~6ZXH5rEkZdu}^zplO|(~eD6q}u&9 zQIOLX=sYMGblURFyX`!KL*{d-O*oq2sMmi==}YmQUDWFta2Gtiq%^-g?=R`xpKLXE zKK>hHf4zTzQFZ_wht2(_$P(Z11Xj~Zoi+PX%*PyAeH-$T!u!?8G&{XX)e?Dkdg*^^mcXeQWKutDJX*3amD18whKDSG&Oor8CX-sM=||TGvM*ALIvrsv8WVL8d}X7(c9--~x=Qo0iu=SN z!wV*IieNd3_Zec75&6*o+0FEezW&nwRB6Gc_}{;*;~Lg5r14btk`eZ+9(K51W-=xz zvzU4pOZbv$j?NKrtTTQlv<1IVIz^4P;J2^$o8)Dgs@(9T5AXUBZhy4jRWG`bc08Hp z?ruSSEH|pSA8_*TA5Ycq@hRn>ESJo}gZM0Iafm&e1h>7sRTZ)>yH;P8ka{3SyciyP z0D#&k1SGu!-!|zgu9cPXy4Kn73ckB|C25h}@0w%7)~?1Q%ok9`rv%`qxc}v?0P&Gq z0P^acM3-(1Xd%@RuWLcAOQe+|(EF$xa1d!onubZ}y`}Lx?YMRLeVy)=ktTOiIn^;& z;{8+US}*0ORj_-72^wHMK?efJJd2@H5uX~}LyK10O!=0Q)9n9C2e@@CVnyQz`ERtZ z=f)P2Wzc$8DPJ3*qD=Z%Px;RRE}kJvCdu_dOI*3_I@dkr=R{avW;fwC32mP7?IlpF zOpw^k=V*{KnL*KbjzH)YHaRlYN2vy@M-DLGIjAlMQbY9H5?T}L580?S_6DAnydX)%oJGI&*O6+ zznJKq;VhMcrb=yIZ0C}yKIOCOct&5WZLbmSxz+GgSh7kd{?wW-+w3x1k-56Cyt$c= z9%hk#AN%xJDRQ>Emhr2yvhr}b)itUpXt$bn$B02RKcBANr~|W2enSa zl+{RVV*YzgK@LmI^ECa*LpGhLJy1US8Om$D>(JGRv(6GA)<{pb$t>ZE`inoyc+olJ z?x#TRXB8j_yAh0aPWu6)9d&q%BwP(jKY}R8WspAhcq`Qn0dk-uHLH$Sg;g-GR|n3z zXRUB=p!R_;l${J|h_zz+P2~fSQiYX4U_0V0&JosUwH+h4{=b}_6WZwT2kwz2P;9ev zGwflEiys6F9rrZ(cId6A!Q#2*f7_rUnsd^?I|>tIpQjV^QV*zAa#$Z=2|V;_*8Czw zgqza}HM>o}7Rj!*{%>dA!PJ9`wSn5&*DIT;Y{D-f;=cXe-VURYZ9U@b)+91jHoBxl zwuIAsADzc8vOUrf=AtRQH#D11 zJS)dF79a?=Y{|XfEk9*=kx>*2UL^nzJj4&Zmd|@9gVrAB>dRlv)aFDD4e)3B<^@dY zSbq|2idFd?clBYCMihS;p)|c$RTpFesmhWJTSi7k{wiI@SO5vI#xT9N zpm`{8RxsUbx;){iQ@{~1S=pV1NeTRV15aw=Id-#F%f5E#rTfRCNjz!sP8DfaL;D4v zRy2h)MIO+}{(i`VC!;7GYJ@+Xs$t>Nk)}f*_uIKI734B3h{&sa+aDCXua7jod^Y$i ziS|jR7v&2uzve#g-cSTk5M9|nO;rG?Uyz?uDX1L4MB&OPJl*X2K=1hOqt<6C_MFg& z*txPF6T5bh9azv;fQ+z0CQBK2Xu?R@u?6l@kZ=l_mbi3zBlqcQ%ja~hhT{W=Y#!ib zDR`7t-RQa3CHl;vw~M-vD&x>wY-{E@I}m&%u7es(yDh?s)&!3ezJHzTx%xLSC)({( zOUT-6I4VYmwIAJQw60nR^?2$pu>2zMOcfJN@g3d_PvDGi*+bmVjy`0$cw{`FwG#qZ zp8j#KP2rwaVsBXcA=RX_R>S7mD$6QlUvd_bPTZ33#!wWTN`ULAZ3V~h3nPqk#CU7a z0QN-lawC@iDk;N1)$D{!d|pkKu8Y+iEUL*g8Jdjh~X{r zE9L!FB!sZ_WZLJ=(w4I4Z_yraD}Hs;kHW07NUs|_OM#oW_YZrZ-`RzW$NS-(Ru$Ug zuT;jXy6hs=^7!!>?{l9}^?RbH?ApD-VEP~Q-vV-oVAGuA8vA^Q!#!ffbQqnCG@pT0 z>f4lM&QFi`a;4uwS_CdW`TvWK32NdlwJP1gN8oM09k_fOeG|E!sT%&N#VIsS%l(wA zp3oKgZ)eT;koGHVy7EpD%O(V11ibrJaRd<#l#Y5(^S^@}d=v4`Q)0xzUiG{Ud&e=* z6|kCeoG>!X@H0~P%{r8V2MA_VqUi%4IBiM-<3++}OUCI`7$Q>6Fp)~h4RkafH`8)4 zW3P40MGz9}=KR-{KQ%K`cG;sOK$qBYF6U@Icc+Pd(A+aYUpt-I52ez~li*GVKd05r zbk3)8{JQeLkJUo!qdmQ&dLD1>G0N($5A!-Sr~fjT;}HM2jmz+MyXADuJreGA$2;g| zJ6XwTCT)K*wGo5i>YDh~WhM={5VXXzw9;@&jJ0B`o~-GBTw=VtDr1{9}+X( zK?7f5+eY>q|apLwNtSgado2Ih~IU@!Y- zG&x%)g%7u2S_(_$h+J|HBN7ZO?*06IKBax!z&=_mG#4l3vbD&vh^w8k)-+ih7 z$>_D97WogctylSS^40rX2-|PZQ&EqjcUPoYc3KLQ!khsO6Zzj~UA==IY=k3phWPMA z<1F+|wFUQ0e>f2UqZKwh3{6#ViLGSv^QDSd$!l%5Tw7!8)h;H-Lt5J)6haW_F=q~!bQmP_}7M~rq=9pA-UN@cVhlC63_&jcZ; zu~XJD{f55HPBuB|IEy{i@^wDhc=0xsJh;=;;&}J+6VH4gT!EXD;FFqt!)S8`#B=C$1&a!})c5-cL ztI0Ot7MZ?X8LaNsBY+B*X$OY5rXU+G-~U`%MPo1PBxD3XNdRXT2Zq0woWStIEW!Z!ScM6WyVxLPCNG zee8nxLcZUrzWvoGo8@}Hsf-#vuP1!5-l5kIVYSu1veTREux9@>R^dIL>}<-fdZkUIyN4{N|rp z_5zNM!#2YDfQACM)jj|^&HO4^jk||OoD-9)Z>$-2pfzFleZiKU-<9PYGTjk*dHH4Y zH-lf?S*sd!_~`3%)O`9PDvExT#kzsg(5i4^6ZhVZIab>guU*UD_)=^yucW4+(E(}` zs!1NT`rMDNLlCJVW||>Yk^93yiVxCVx|p)-ifq^uAMocsBDEKEP_be4H<}6t?b9p= zl)r32`HXzp1U&}3sU6=P)6P1EfaVTx{x|$$LeM6# z`LRmjgc~&F8t8omguVA67B@SgCEf#R$^Jlls3q+fx^TvT^TmFs_@Q<*euMbn)3^rm z@#w8xToWzRh}qQ5O!~1_dh+Z@{g@4-{SF%Hgw~1IehCY}A7JI#ANG*6Zg%DA?s#jO zLE3XJ^#>7B;M91gFKmqORyKhFvGU_(50YhJWxXZ=TY|xbkM~`EvpJ22I?-#zZK_49 z-xy<7iK_d70}gCbc@fme4BB zKX0adfj;6U7?59nknPET8;o)Xsh=%*9Bg)zIV&Sps) zx^7rX^5V4eeDbo5jv4^&2HCv(#y%UKG#M^%GE>(S___oFcpMwr(I2>V;+3_KZaC}o zTX1t1K!E6+2pEFFu<0dXahnBDt0(cQ-XBOQ6GAYE6z^Cuke>5>U0t0IMr1`rg>3o} zY^^-LLFC)1J3@Hiq2y1vU7iS<{@sO&51HKA4C4l0C!1Vw^Z##qqZZ)JVVR!|S+MS= zpjkiyiiDjfxb%b*WoL8p8`UNq!{6)G2M#|BlXqd-_R`Z$sHa_z-Kdm(LQ;;%LgoXQ zZj}4AjRM|tbzG^f2;-@MN#672ep(uU()1dtgdIwtjo#8C| zYVT~Oo6!mMrnD*oXa0t`Zb+OF+egZm0IFO(gwkrMi}z_=LqKp;>~&ySH2;f;^>SRe73wpZ+R=W_JCkxM6;tUXe1pf=b7m^4_- z^`%1_l)=xR{3k{3ILsX$b;{Wlv(QHCGS9CwbM!D�g$S^viyTp85uQyFBkv(!;WY zWs+6ycL(}MiPC-=8MQiSUPtg~l4_YFXZNQQ-DB*Hk{h)26?F$(pJRHSiA~_i;qz1u zgqd+{>uj&F$AYfwYlcssq7x&0=tLXBi1W*xx?Dbma-qMyCNP(`QEYV86Y`9sME*rb zH*J{bbJbmudCXd#aE2|cPd*J$;g|bQ?tTegXOovkC$9EXf4Wf=InUOFA_9c6)H?8p zhGPN%w5p*)3$(S(GLI=GYNge)E zP7OMon7G4K=Og_*FfN}|$%zhjJFIU0k?vxiYzQs^9n2UyV49bZTR6mQD%vlA_Cs!2 z^72PB*b!~0Nu?w`I6fbT$L)Mg&V$r~wf<`xFzdwPWqa)uwDhQzweJ?#vR{_f6RmK- zye!RDr8myw@@!HFvmTw{xl7wvs%G|Jtr((lfYaL>%>@lay5S2SUSn5@Xz`R=>RHmD zVm=-U^lB7-cc(oPsChN3EB18Qr0+P3GyX*0{SRnuNi*01G*ezF(^Cz11Oo{y0>Jp* zowCkLQG=$A;JG%qF<8+dmIXT#?2!q_kSAerm-GKn=~PQJg~kcr^C~KP)I#vz|LNVY z5KngJU4S<#PZaa(N*CTc?Y!0+r0gsZuQNLy0qpJmrV{j%$>W5?S2LQ}+^9Qu3QFZleie_@2=tBYB2=G3y zPCq#3Y4Dv#!jrL;1GLxFCVINh9aVg<#XK{(0?Hiire&shVN&~m6xL@CtVC}Ec)=zx zkwco+li5pOce<=sPF@YBcoogA(5lb9?o|Z;6K8h;U#`lNTm=<6Yg7+uco>^Oi_?;Q zv*}17=Wr(|^beI2)(a}63iXdWv8|-4Lp6^F{i5fKx=*^X4+u$6qa3 zZ$-`=YT#bM&e1&yY^93h_ycW~RRg4IIBTyqm?q+%M;f6onJaj*JC`uAIhCCK_4S}m zV%Sp&>kPcdVuNO|c2>{6)FjT&JBY_I#I5`9y9NHYfxMroD60LVD>e(|gLY+sLVrx)wkilx6_CJl#24@t)aqgC#3%p362tBANT%f^sU1G4_T3pdHIc zEY0fY?>3$*9tIl)>E>Wss0ad$#R5GIwu2Y@dOeVm)6V$8+R>$s=6b*H<)(3NbyY<( z-o%1OZ&sDT_0H@cu6IgE?r&$jl%xC_{!iJ_K$RMkbl>##8I`@2b|cF<{1(vbo5IL6 zruesOXNEYhr|3;`)}IJO6HL-^Rc1|bbHD;m_Vd5ZCV=AK(Fi`-L|0&#g>B!DIqVQC zT5_l%tuH_-f;w6qGUaQHc`!Q0ARw@!lWgDSa*;>0lC5oW`OwJS;4DF;h=9YNfpxMc zY%VMbS-GgPPet)ou{1YOHBjOIA8V1T;Jn*u;5nW6uNqsLLaOG9TkGDqKGyVL{cjI3gnFN$>ex|3bhY5CpwzX|yuxCFdj<9I`Z%Q}}zrZ%M zYb>`F{p%1O8)_v)KGHpm&A-2geY~Etozgqj<71bFkg4_H*n|OownML5LtRynG)WI;!APq?^ zn-SyJ%dwa!SezOF7j2+aeZF5b;h&=-_qVueqI6dyQiw;dR4*}+=XUxnKn=Rx^ChDX z)@N9!7>GyxKhKZ^n;u!v|HM(p%dy3U1s+D0E0OXbkcCV1R+@V;C+5sik`p`!Vs{w3 z=p+Au(+BKWUgzD6aPSO1?#Wfn?~_k86Pv2pc`U@Knvyl-PRbH=9T;I&J$Va|gG?~* zPp9tXHe}a>u`nyPrEz-`eM<}F&~18%dCOQ9vg=tMORZiDdtx`_Z2=-HE_afUSe@`4 z{PV?QR zy*GBfoOzs+BLuVQ>(DDq#n+kHX=q!KA68w`RRXfebWo^{f!Jq}>iIz-P0J&)E-8&nUyDia8+-&bITS1WUR9(a(^7>@WoVb|2`|ANQ1OFAM96Oxk)9_Ph#Y#PLHsF~dHw2P=kYcxnP-1u-Bl)il6`sC zEUg-|QY@p@OHtnU)^H=`v)u|AtbeJqr5%uozxP8vY@Ie^`%1gVI~>#R2UsSR+VOQm zh32Zm&oW_}?klxWsa(gA0O*V{9T?yF|*s%?DNB}HH2eZ{ohBj;& zD>4KSk~CetX?^|c`P2DB*_<&ZPhE)_@dtDUr)f&N7NvOKxzwk}is&^vr7O#A_Ul8a=X;`f#t?$XJq>URKW%P%GG4%T_!4E{EV^#R`@ z1`gs>Mne)L+x_*;YujZu(3S$m4#{yxEo?xMyM+6DCZHJjL58^-5*;9a4C#e3W59%d zT&SYg^UM?o+Z@uLc7}-d_V!*QVG$A&6xSJT8w;-905Y6sBSw@SboM#s|u#znnRiil3DSv+tR}a5Y(34id zET8b<)(%=V+AkELSEdP4OvsGB;qX{BtS=7c z%wq?vVKb0jz*?S+z17>}4Xk`vo#Ozr58Isrgys~BRjaU zgoyLg!&1O+)ZhfbO&GteLqn8e!hdogJg#>sg}iw-!3;|QS2gUcc&=h85F2%EC4_&& z-K`s>2>ogHM+WpR*57}Yd@RFLihdN zg{yP+IPwt-SDvQb{{z6`@3D+NS3F|pf*6`0mJU)#s`Oa){v6S#*?i5nMvyo0zBGgr5*8)9AJZom|0xeRzBfg)|#^H}OMDXcq!h z5)K!b_#2R>Mq*?7VdJ#fl(BjAVW4OX)+rk}FmO+DpR+{pH2OqC7_Akn3er(H)`A05fSu@UUrB-k(*_wb(%M{RE1upy*`^2n5g5qZNA|UB?Pak6`@Kr5^9P9aw zGIJC(?4pP?QI`;~H+^#6Z~|;7(?bynIH!c{l#^f>(a^zj=%qj=W+5 zsr(eX)!HmJ;kZ$oUo#HB1ahsLbjsL{)YM0@lCnk>AI9^oiYvbGn^7(A&l#s6eA%7q zMA8nDM7>z<2D6_ZW79`P{l-9<&Mm(h)F*C+z3K*(4p!Hkz`cZTVQuingdq30}%OaB8346Xtvl$IX$W zHztTl-ze>j;5Zoj%|2JnRAdlKhtphx)Ym(bL` zYVct3;`-=yLFjfuDuJO;wv`XV2vR`9M+50@M=g+d(DsY3qGRt}Y?wGIHbozQ%}}%u ziLD!* zS_LZM4z&kQ*RwWoHE3N-K?U)fcz~g0K zOYy9hpw`t_U{nc1SeuXjq|ZD&GWI=l2AxyA?_sA@-X-d#0Z!fu_v-2dKCM9XaF8@0e_ZRnNV{=Cuzh>y$pkx~h0 zy%?su`xZwOBNjbP?^6Sfe?`WLG^E#oKjjxNDWw#$G4M$Vrww zy6!pWB@1(A|DKbS!y+3J^rn#&qj)#b<-M`h{|ps;3C+#-5sV~9%V(LIZUcJ2kY7vy1*jpt1i;q<&W z;r-(ZTk^T!@2XaPCKw!o%wPd)hhR|GZowC-A*CYximav7DEtgkJ6|4$kpckE`3G0w zvbKk^$<1^3S_OW8#x!S=DF_k6&&^=^+W>4a&1$lAyA+UtNmCa6%CP*d@0B5gQ0hxX zo<8ZiBTif9@yCR3#ra2noD*V1y=~hmCYjz7m&fy#W1$jE4d>9iUlz{_xw2G7_FB-c zTMN&Qn2+NIJ9n1&oh)%@MsHbIwt@g|^6D*$Z3z3RTwN%;Vhoz^pMYzfU0Uc5H@wYz zf)ia{Z}&UcQffXkp_Y#&UrM9$qq$SV$y#AlB7Q>CL%)TFCt+Kza;@)=9^48d-SH#j zR-6l)ZU4-3Fu-e}T?|rd_*)b|4-u{ko&I5xIf`x}E0|}<6Qr>G3=Mn##TE}SAXBS` z0@WPW=f>N63}&*8`0u<`zIc@BgAO|F_h^a_OBDuJuyjP7E`n=k50eqEcsLStoQR;| zYUAkd;|Aq_gAlxCK-yy%bRTjy?e3)9z=75>ec93v zGhqTaBhohK$$kt^pUT&g$j92Z|6;|ZF<@SkN;#=F=>rLCpS1Hs-4TbY#S`vHp}6uT z)rYXO#5KI=@61+@^@2~!*gBu0H=OFA>j}8mC)V7V!6|V{-tQ8>$;o)707nM7l0j7Y zXt}Uupwe1yXT$6HK+A0jD zVgbB{0MS%pf@t!CpmIE7`6!KCXv!XuOG>@%0{1P@YrDAW6WlJWimIBt>+f~~^g^`B zat)+%-;P;qfZEdWIGw=3eeLJ7n}eSZ&kt9g0%M~e1e}(FztJ-Vb+qa!XJTsSP_Vlq zM`a=w&(Gb5N1+Hcwn}9B9zSjhAJMGCifVRH9KZ=2AR8S#)=pfSrkz@ULFzrM=UKM2 zPF;p&foiL8=keIc=02X_xT&EI$aL z=xGM8f5SJ;uvn+F-s%6u_N%INxE=i<0T2_KlN-?2c_Ob+H&Iq?oZLB2_tVY3@?ad8 zKhIc5URl)g$ySJB(nH+faMc+7vrQezE8(fBQlt~xVp4L|C&nQ7x2!$tfqj*L%?O7c zdc@Pu4LJ@)tzRRZCoc_i_uE;2+OtHO9EltefzrQA;qUlvbyk&L{W1WTIK7XKV*EM8 zGqB3K>aFHG2cq-8F?Jq3o18L|+ z*c;+>k}D>#3ypk(lW9_xc-c$|4^m4>Vif%+={+%1Z*YqOHJ#kFymUxJSp>nK4wj z0@5Qtj-x|Sp0~f4(b^Nbb@#6{jkxd~dYp?vg^|`Eoh)FcpgC)bopd@9aMD zK=8N-KQ!Ghcn*7VT_2igHqou8a^aE88*NO#8jyd_ZcWT5}@efWX>R}(=PG4tuKR-*6ek3p!Ha#7Y>*&(I4yvh~n_7n6in^4mG zb%Y{JZv%%lu?*Xt)%VRI6fUo#xdR!sO~%YSa#`gsnDay5z?F0NIK51yWfUBj{Ty>6 zf0?nMfr#$=9Ib2q&{yuX9+~fYs+FLiQ;!+xI_CM1MOC#EFaONVY_A%TO?Pn&q;gxL zcO#N7Shof#D|Ccgc_Pr<(!8gFk-Eblv+8x&kp?!(Rw2y7!?_$Jm2gw zhW_N(#dvi*RX8y2Nr~&XYL`T6#~_4X6Dj2#g}%0u>Mbzn%!#_JNn(|I==^2Boo9%0 z_+n4lWuj1)Nu(FlEo2|(dpR7zvS3r;NK|47hE(&YeXvjMhO@%J%&V&6Q#O`o_H4Ev zUqNIcpG+0t0Is;C*tIhJT*fmE(KdIzdyiTww)jy^E7{X|asd)fiR^BgNa$-)e%oV0 zj50jzpF}^AjO()`G``W>WAk9^YV%{r;caIq5BX{>3*UQg*l7S}fNSF1ECLF111g#q zU1?*Cmi&$55(>fN*5o!hZ2J;_us2LHJA2_|oz*eSH{=J4BuND;h7yCNI%a}FI?48f zmx>6+NKDAb{~TvaOCzzhEy)#^5)q%!Me@~Z z5lfNDkN}Mrw8%cel?aT0k&l#+N!z#c+WPw++BwV3V}3IN+2X4L4b|+qL&_?Ypm#&NsPdn;|s^JIn1I?N`8anK1V{w3Z8yqoRrBJ?>xR zDvcEXA~6v!&V?Z$(phU>83S%x0Gd=xDk%TZr*v*@! zGmQmSugKZTYumsC3!<9SAh<}ld2Uxv>{OD8)Zc_|b_fH;SRlN?3O9!h2-)U}_*6ww zUwnIy)(h_nqpduPkA5~*V6BL3tyke8K}ZZ6GCcv7_jYl*Ts7aF>ICp6*t6QYtxtW)wqVg z`+1+l_UG{Zxz2nym`!2b%oP%S*5!f*j(>-ZGX$3BR%clCjj2ZSXGTIFqZs%#5bl~n zauWVg;F^k=(e~T3#@C1KxI}wTga@h5(8a{@_vPys!8~fH3F7_7dCSI@RT~axLOaaW znk{EmNr?4dcWAiN-*}!GzHZ4(0`<6ogQM{`_c8V7GDdDJB_UYGynN(nhg(0q{4r^@ zK&pZkJGM|>@~}C?^|^w%MG`n<0p^kKjhg6vH9QXJIWMWd4JIFE6rrBlMuAO6tv~tx zJ_oU+h740G5Z9CpCq)ZW(H-U9Vi(;zNS*x7Nm!tq_GU4|!4N zpjy{@LF6>bRh0qfP0h#J)(`!9dH$Em*N}PR0fGZ)pnAMx*K7X=P%it{r4ePLr+3R$ zh<~fc{P_kJTomhlM$_&PxeDaL+@TUZH0k&uD;QdgC}Z{4B0aUMKS+_mCa{5j71P}e ztn0;;TR{JvMBoQf$3ghc_=9*hdfOVsz5v;TYzAOghu944Z*x@UR9UvOSS30HSU`lY zWt@ALStZv{+(V&qqYw|qN=UJ`h>O=hkkF>XP1@1Xksma%2!7!m_{I}*c61p%b#ozV zelqlJUrfgtmjdkqTShBQ=Vv4RK&Wy;&NgUh%_*zZ)?Enyz;cjY6~P@YSJd0(Y`x31 z0CA3;m9-LhAxx#pm+7HwlbNSm{j%2Y0Gkc@GVVaY62Y@y)09#TYzJ zyMrfpuG0E%9Lq!^BC}UgYPE^$+NAb6L@l@yHNnw{A;$$~2gp^3F5#7!p<~f_zpbv% znh`@B<75|h?i+1kpm;#IP8VYI;3AUx?*;f}l111Be7{t`tBBryLSY1Kq(x+3ByChk)Ry9~$DB=Dw51p^ zvZa4b*$$Yr)sZm_Y>z7dL}O48^Udr3gE$dBSx29me4Ov} z1LNh4>5M$M<7+STGeSea$S?A3M((8~Auh3ge+1bCp(^IdNW1=Lsf4nQ-t1`=sS{=n z7(sz}A|Q}!A&Fy%mYlcKptxp$>8#VgSi#Y;NbdXc9H+LJL0@+WcZ{7ai9Z%SixL=H z?8Q8sHqvlmurU3tSh&TMn@zOXU{s5E)$0Pvg(nti74@2L=kd12hMWQ<-dWWQ24zu@ zV_}kWdqOMXBngjTWTkhqdR|^$K&wAvf0obrtX?%4-@_pwATlEY5wZ28)RwNv@SV`Z z7U|#r{Bxk;BI5}m|7g-^gSpMZ7pT)l|K55_MTIrU|KS}4JU8{?=zAX6=VVyiVz+{r z;V_iTncs-W{G=Ky=E1gTLejn8f)ne!;BG8<5Nkz$7(&DihBTWc-Jjrzwh5x)%n5l8 zKhh;OAjG>hLyYMOd^dMA4wo#oPeY>CJ7{2jckQ;fuwgbzKii%sMnEN%1TLf-LTw-I zESom76Z>LRY2K3m#vZmY{}9y{_OceD;=B-1UB8v9le2?+#$HHEa#Gx#EiY#d#B%M< zZ_z&iyX&hlGycJC+>FAa?2i}93`R;$OIahc4LgZ2zwkfW?p9&a6M&_qOw1scO@GHI z8YHPi@Ck$x>J?ZKS)gV&ruhS#xJ|}B(#^&jp!I7t2GR`=QD(&O|bK=JOHvPb?Bl0Rk=MQw{EIHI>P-z!= zj!XW6k{uI|;mP{H`6<8N0SR`;;t0VGA0A@>BMeg(3m$u*Hi~qnZk65u2gz1^*aIV( zY4{3Y!?mg_r-X7$bAn8}htwdtZO)^b8s7BVi?ZUDYv^nWspV71s&HqwMVjI^HOBsW z2aK5Jq?r1@eZA1@lcQMq3c^vMDL5MsQ;@`8m`lFRc81CavZ21fmlZvTf2bOZ8 z!FgKuGflNHIMGik_F}W-v2{7^Ydqb)j8JuGaO5m^y1Ql1YtuT|8gtLJt%X8m0k;Po zp~~f-UKomtb+HEUq)}IJ_?UwxgL4w|z7P=0LxJ!yyn6tN=#Z`w0(I*>?m>r7?Q`m1 za1aoN?yDoX8JA@WPWtESI2Q@c{JGeq7iTU$p!et}VJdtGJYR9PY;l&j7=1x63&cHc z;+uCQ9Hv#Uw50pw_aw_OAImxH@F7gDRbN+X{-roVX|5ON0%kmUQ&z<9)q-7^%9((e z7#SUnGWn4@U)i8sQILb5f}DJn#Ad8xI+CsXIC2cAc~4VkXt^|$)!Rt;>O=ksB5DpP z=CTu?Jy@#F46%?!n!(qAako1XxHt!eBB`ShphLwLq#uVmgFThH;Mv0!p%f2pR=jL- zUYe+qF9W*|8BLQ37-=*QT#2sl{9r81!;)D)L_d;`a^t|^BGxQ19EJp$Hb>{Fb7K0( zu`o!fAvpAxM?X;Rw&cG4Mi$(V>t_yp*ds34_%$cgckK4^c+$N&ap4*~Sc+8S$MEa9g=LVGEr+6f7K4-W`cxTVJG+=Au5-!(q_q3V;(~$7 zP(T6W3<{+n18Nz`Y2*;fXER^>w%Z*`g}rHdaLLj-p(8@~kUrV7@C1p(YcouT+&`wD z$fL2HKUPZ;Twm#RAFu{6pUBCq>3Y5eN1TJQ8LR$d{tyj_eP3`8#~)btz||d5(Ax)i&l5J>-*e6_GjgRo=-JHv-z$PJ z@CE8!j~sx9D=%P^VmiSVLkU9_LtP|~+r)RrP-RVhvOU=KJO!?`IHJafA~B;apY=*E z>VUQDqRymB0gY|UXSjF}!)D+ftn`d6fMmk*+gBohMLzS%k~pL6UOH(+5Wd3t`}gl! zC1^(Io!$jZ(%+~?X3gk_ZTmA}^ssYwo>DV2mnbbl7!*duMk@HPo9zaVWej0q zJ5A|OS)ZY{n!uPJhsLQVtaK@H{Yc#@O759Q_2W~1d50hKgU;`WsRGW;2=@dGf*v_T zc{8$Gt4$c>s@d4>*fc&d8M#1VQc8!x{VgEpGo0hi7;Pk4%ke{%m&G;eSLwlsWd@OWdS zHKP@w__CK35_3yog~Qhnq*VJxHa}7%hGyF%=NDkNcv?-c#$zRd4AR?}v=tu0ReWWf zJ0{+lVP_TG1#9rwH?{#-dIT025AI)EEpR^rT7RoRgkK*Qdyo(i;)hyd8YZQBYI|Nq z{K87Fq3##@1z#2gAH*2Z)rBn#`-(%;W-t z-y*A_k^@H}=e>QRh}ZSYGX0K@W!p!ZlOVhv2ZW0M$Lzhs z{1;VoukI`L?ewQVvUe_AZ=|T2aiNO?yFcJkq2h_)j^{Oo?i>i##n*d0d&LUq7xW|a%MuCdaLXdlOT<;!jKmg@0-T_hxnWS zjJ{9)w{r=dQ_qgcW8I2_7?8{eEG4N3l=vC%9P_GHF&HL<^R*wOJT22k=GO+}**Siv z-Av7Q$tFjSJ=lDnN_X0Tkq{B|+CF!sza2W!(iukhxb==nkw?tV;1x=7op__Mq&ica zlrp73GX9wGR_gE;rx!>Pe!%L_MZ?VwzButLT9W0l1H%GonHN)W&2|$lrL4qMks7g4 zx(a8^Y&v$)>Mc@jqaIIH{ChBW6wjc#zZ->wpZ9)ZTkeCAGwCC;1WB}a zFg02?+MFp7>y3{gkNgTWPZDu*Cp#lGRU5A(1@-4z1tvQm5h3`aFljnNd9sw+JtIhk zJZ+pn62<}aN`*!wojppE;AB=aQS8m(W-98vnz~U<{A~NMcBVh|M#Yvr8PsenA@0Rp zA*|aoT4G;szXMX9sDcW(fy-W2mCam`Ti|E?dwy%Yem4;q2gKBG1j{tf_;|Lw!dE$soD?Qu9&tP70 z*Ca0^wy?Yxhz%9&Ez%xAumbUc9^vb5@y1Z-z}7!jW5`W!Ws$9{ue~soB&O%spxbRR zsZXxkeX!p&A1|0`2{oaI#-I#nmO)tr5uF7d7dFtu9;HKb2RqZ723vp1^k%e}v;(%5 zWl|HtiTCA>>Emqh7PPSkK;5s=ZwQjxye}}s>?UpJd7n?Ev0X=Aph`|L!AaSf_CQD2 zD6$=I^KE#fR$fA`YWh2DfJ7(@3WXl6BR2zt>t?hhRB$|q+N}noUyV}UtH^{WC_Fro zi?~=wKK1Dm6Ftc9p`Gc^)zkCU%-K>sd}Hsn4g-uC5)xA2^?&Z3Zxj*9-i_)j8DNfh}aek7z2*eEx3ISzWtUSUZmBVj4|JR({o& z1_OOU$;$$hLR-o~fGMw1AB;{2e#P zbeGAYTj|O%T1Vw=KjD)@NwvT|v|U>mQ}4s*>&O2fcuH^gE`0TUb~PovUkrL3vzSzp zjl&&{x78T)u~9r^w5B(Nei;tpTt%gvolKJ%r3rf3-}v6M(p5X&T-$%)eu2eM)0SL+ z;#9grPorZX!WA+SCb|fw>&-mq+fv?MBYv{4o5PoNV@<4w&K_%sf5F0B}M0 z(4C1xeLs<-VaOnQ$8IlwgbDgY?Dm+jDJJ5K9Bsg`FQT=&Oq@1er!SpwOt(zwtmTjH z7i5@;87xN?Q(+cmf(w(cDJ^w4xY-$b0joVtKX*hL(~r*dFJU$LK-%68uwk1#C!ftCrK5sARZ%E4?&IyyRz@LroX z$hW``%F9LQ&$95?E3VVyJM%FP)i11`K#0g@@@q-%lX6p{s0^c< zr$j#mrNrDE&iv?kF$tBRd{q{Mbj8ytJCmI1b_`PQ@MkJST;~k6my<377h80)Ez>)w zcd0Oo`BkFRDp(AgN-P?U(bUv5bG;wk_Fy{qAEMGAl`09x1;}CHz%%A4ocAcR-Dv(R zv#wZlAVua5eIC{zdP*unciBKQO!BgwDj|!QXg7+*6oWRN@!sXfPkt96d~_<_a&{ji z82+ueh{A=RKcz=Rnw|RzMR{2o#Maw6ynd&U{Q6Y|DB%^M6i=J2U}njU&+03~PoYm< zyD*V7IVj*2>EVYr{WwVdk&Jrs`N9e3CcJou`|3NWx^mlV4-g^kSVlU=U7yg(ouhXC z5CsZMNJ5|k$U&$L|(YVhFkBk(EGUn0aGgX+A7ZPNezOX;JT65Ua005&A zQ2IzQc*o0S(P#MlU%&HGZYdx-!+61$KnB)$)8U!T7(`7+he6sYaU}T3tX%89h9^WG zMu67QPwL>Y6V^1t=3(ArDS#83MkACG0XDScX&ko_P*-Fsaj0q63RtqS^U;(`-%6;T zyo>u1^nt%3XPKy*kDZ;Jmmc3ie1G~OV&$53gFR>)9w^f+bB^~z$-Rz#M%3-PH@ke1 zgaZ@7A+PpwaFZ!=$T+qhE(Q#h+~zK!1)` zj1ss)Vk%WxJ?5q9ioHjl5-zA~rW;(WPycQ{76QO(5_c<}kHH4srU>1l#D#|=bUH9k z?~;35L&KeFUEs`N{6h=L@$vB~DAq_CGEr&QE4I%H)br*lW9mcd74dYeaSb$;(d*M*g+XpJa(Ohy8foEO|CEi)AoLqhQkdadIh+2} z-<%254KWZ3Gxk_2k4n>bE0&QQj{;xg4b~&}7$cD&iqO6izYT{SoKccbE`n2x2 zA1KErNOd#nAinW^k=5S>bS87ZawkQ$QgArP&aa}PrphdcOfazikzXy8ee82E>r zs*;wq@xK>~O|x*WOVSG`f>ePVzpe>?B%dPX|Fsi`@O6lI5Ap~+@n`5QaaW$i_}yeARg#y`vl<0ciefV z4r1ScGQ85ISGFLHn2RdVj-gONuN4;7zWOA8c1EL%Md1vYgK{ssC;_e=ZB`gs(QeY) zsRrC8F{yUow5qL<=xH8(EU2n;cpph??$ zBypQ@ae;PpI0}atdiMOY8!_b7>i#B^bb1c6VcwSs!v2`#@G!jI(2@?hHL;67sP9{< zO>V?r^o!&^iRqEzgS?nrbWRd0sD}=0EJMf~?;H{rt%uLTk%EpBsm6d(;?gu47XyxS zlJqggF+p&G=XA+vZZsf&`#hcW2ABGLy3;d2d%K#ne*Z8LH2Jud59MurvsEa5FJ52I zsVFR!6IY2zQa>skZ4rOrPk=}^uJi~jAs;+W*^&10Q8F5ThAc3NP}4J2lf zOtrE^4FV4k5ze?KR-+Xu(E#W@Mw|ude5e8QdTHiRn7;IHtP6|Y644x$O-5B*!1{2; zJ5#uRIX=Mp>>T!9kY6M75IEL|fI~X*BeFzDFynmRMCOsJF~f#xCATWxRb;(0$w!^c zwqGMlNf;M1)|VQ5T8@OVV)P)Ub2??nSrB9mnF~9MXb$dPnQoqUe4c8u7`4>&)<)F3 z>@4)nLvPoH1bvcumA$G{_z#oGUVI_0(^%T*R&yYkM( zB~-IgoCcOaO?yO?_}1KJ>>AoXO>_+NfLLv7{Rg7Tcrc%vCa{j`qgd!KNa~rnGx|x+>~Z-1@2F*x0kSLZc$syI%r!KQCDCZDoJ)gc3O{?W zBk@C{zkYEQ*?7P&G>1R1X?A}wB`8ZK6(Tqz1#5b$3T?+}uGm99WskhEPTZH{)~Dgs_+ynL&V6xkBGn z&9&*h6$OW!v%HnZwPH#r8?)3(x#pD#>6H^eYw+YZ<>f zhdaGntVUKJgZ$CQC)1`lB70#hjR{srYYs6Of9$%Cl;biCxh>X^6%UV2Zm*%f;z)@cdLdC`s7DK~}J8Nz)W)ygeHZr4q1M60904zBzQhQfee9Hhp= zDquBvM;BjQ({WQLZw%UF=HcfSuzf@nRWs+jSYrNmhxwbfZqA)NhCsnti z1~Gh|-L6LT_l*=t8Xq@fe=n<-n<5Zm@-C1NED+LB&whAY7411FuXIYru3!;Y$e|m3 zWEJ;z5}8NZUBjS1!MFIvVx`0|JW&@LuV(@mipP8F1_qHVHW3k#5^-@f+lC$bEZ@dJ zXBM$Ea)ejK3DX}EL7kd1c=tkg7t}SdRjT#zdi(WK&VXJ#rRjx z*e6X9&JK2A>(yQYEFFQuuD>M-Z|SJ<1NXXsA;*4{Q)~aWwh3}Vlxxw~ zF0Vg9`&^A#CN6-5X$}o*&|UU3V)4M%U!RQWrGP~rRxNaf@vQY&Fb419{h`0?mK0fJ z^J1->Zn^HsxQJ2)i^gc@)#m-L)JtGbA`$--HvFBiHzHk5dagOnk~o4eckq1x2sO7g zI4H18lbIk^EaXxy0t`5s*Dr(s4F^H=fKUP@7~ z&5}1E!x&Q^=POkLg)BS#a62=yN?3c%tJz?i%8*)Qf8w#GFphp4`#uxGq4+k-=w}Tm;Bj*l1F_y;D;aCzk)8D7^CrS0k z`i3%DI#cvJ#3|nb8z#aDHW``}x4(C0&H5%uZf1gB$2N-TXdD;67{wXNo<#G1Nst2l zieJnj5hLpBG(5NuCK(Vpe`(B35`t+0XZG6RQc&mRG#SwH|oW?NJ(_o!=f_4O} z1IFTTRNoa`r(@!yb^(oZhCC=HzwX#53}cU_sO4>sR%PCH+Yzmmy< zo9%u&`+eN~8h-oRkyZ)twcDAWPZ^7<*un=Fd3g~6HF?!UK`zZ!MW;TgrIA2TY_~f2 zj_+-C4UoG8mCp7apTD;dN9O7U)}cO2P}rj+D`-3Xt*Qx`%|Ex9S7&eRh?K>Z0N6EXJB>nz;Ziiaj1b$48{7mVYpD2!&HM|d({NQHdXoC6X zb#hE0OtNKcMz+PJU*IA4ypbU+4mM@n^Ece(Sm5+Gv^T{aOS zwYY9i44vyhD`lI+&SeZX6(7Kb@$!)d0%~kS@q@F@u>XY2(bt2K2(+y<&|t~Nu{frE z5YYcDbMah(3}wthHg_0}G=$?Cz%^l)_Bl|4fASpR#$2HGD5XQ%neDcDxkH*m*WWjr zmv5Gm03ul1%6WFdX1Z&#JKpC-{KPb>-waOqzx8gWT*wYcT$B?97kdl-ppSk(cep@pEk&oP z!MB&n^wa|UA?;ZT-||E>!N2eFJ?hQb$w(*Cx_Ne|jq-`k%=zX8k7$T$)d`Jl1%c;1 zffe9+jm716e{4XUOEG&Oi_CBZ@Oudd-N1VUHFBS=+oo>$@{I*o;1{7|G-U zYaO;2Rz56B7oO=e*a@yUf!FvM3h{d?zWAG?#GVMlffgW1mJR7jzy<9r z%KYG+MiyhUcpq#zTwZqy~iFNn$Oxf2#{$1-Oq3ps8d=d8pk>?#Z4cYEI>t}s| z&MVZX?U<^3d>aglM9m%B*P$^VTN-g}I#J}@3+PrQ^R(@eC8q2@kjL+<^=PqHI$F5t{~OZW>-uU>@g z0a1;SX%siLO>eaqvlAp#!25h}49A`ESHIZrCXcBczRkq9`W`q(Vy#SnpS_|yqh#4H z{C{O*`5B^jHR)N9DVgpRp9dK?a@wn({j>K|c2sM&X>-{s z(GYp1o4*v#*jSYE`1E9zGkh`i%%_URE~4k0%B1oqI*eyFU;|HZCe^={PTO(C!pHV)> zRD6IGMulB#GY1g%%3tlpMkIV#`YxQg^ofUK%JgIq@8BdZ+wgCUFaD|%(^tcU&y87i z6DfH7RH?R!l&V=|PPqy&>Mh-Cx;i1d#R;f>_>T(rrVJmB0&P9trqn454lgvIgQ_M$ zVo*9q{|N8i@P6g(PRU3cQ?#5eZ?nY>SRgGynfdT`y(hj=t^LgX>*u=fq^bZ~u>5-1 zSnaBy-<{JP-_GMs^-inT2LIhG_3ZZN_oxg0SLdK>56b~>h%2dYCDDeP8`Yg34-1^(F=IiC(my&_%P?D{IWK$I_Jm zshq1klmj_DHdHY_kbk{`AmVIhvK$g^&1t;TCM{zWzo*-3VY`!U1xwn}Zr_)nVMe{& znl*fE=p1@cmge*Gp8s_qC?I;q4WclNYgK%oRCTM8<-C0bP#xFzBe?+~(o;P&`SSn{ zQr6ojBvHGr*R47FOZ<(r{+&|HgJ6YWHj$DSk@KCv+=A4AD`xVLq8wRTuICAFvZ<>( z2A297F2R%!1Ye0IFDV9To{DbZH97<|y@z?rS5)2(L z6#6vNgo)$h)DBIwpl!TkBqFjuJ6HzCN0%k97p^V;2K1W!vbW0m+Mp)0Ytu%@P_qJq@I z{0c5KWOw25RHr}7m=g1C!y07a8;(g zEcS)3>F@m3-=3a&5BmknB~k3BJoTNdS&B?_(vh3Ls*x?(ji0%8MR&7kz29BlgKce$ z>EzvWoIl5Su{R$>kr@spa|l)#$RX)QZr?ZEPvBzrq3fGOY~_RD#F2{-$|SD9_usd5 zkLke3ZBtXfgTg=SB!8XK!uJUz6_FV4ce4aug$R|FEs^-MIXg(s7$w`hS)X8g9xux9 zT%adws<4v@!g>i94Jq&>Aqqyu9ALh4w2@0$|;9PTDyaT)3~ZA z3mu$yBfcIml#eHg^{a$7du#gh7KBBVSdKg@g?bPCzb+qkF&VFac-W{0D#>VS;!gjY zMUr^(GSDp;01y%3x>#jjZ#7p`i9-ek34cfCNWZMpz}`x6J`Mjh<+dz4S1HBnH6V2<#!;D!LszE~ z0d>o%*>y!qi^z55Jpqp@%d~`5-Vbr~1a(>d`fEiF2n`s2kBLKGVp%7z;@{r6A>KAJ z%41d)5xTpn&p%($UWHE$Yw!6L`n@X0qt}$^@)0ca<8EIx&yczY*wz%C441bpM4TOI zJ?>1QkvJr>(0E~e(lW~*$ndqJp`nOV4baPea>=IR*cH|9hM|h zoO~Dg6grv^lr}U%gf}M5^U_CwT3zO09RRqw|E1hdn(_h`4&)C+K&?FaNX$am8vO@| zkoVzTV|W`;k@&OGXi~w<4?Nj!%sXJa< z$7k~yTboF*bG%c`Rd)-WQ$Rf4AH8gV&4`DYTo-72_v)PqwY?uY1Y^(2t~5K^<$Wc4XWb+x@-! zbJgM!mT>~otKXq@nOqvQ`BGH=!W?a_wlsfuueu~A?-ZP$pV)KuI8rEx=)M6Xz5ORa zVjgyO%)oW0?sbYWWL#-Nrrh3I0ra2dLv4_BvbK}qGZ$YMmIyO6NW9)YqvZb|*KMkq zqh8@?aE06TzL&{&KuDPT?s-WTfwZAwydUm*@;zdg4^DdAWY%|bKcB>j8`hxuf5gSP zHr*2n1p6`-EfXpUjVOM)v>CALFK~44T;!}z41L^sn~(ZMa@hD;v#n+9uItMA8UM_$ zVKA&=bmvk6|MF^1M}y^m!>v5&`D988ZjhJ9;=%Ua4T$KU(z!&wjD-ecDGjwb)k zrU$Tbr{MbRTWzZO?vosJbiT5YMXSs%?$P?q3)hapY+89{v<3y&C^K zuZWVXPtC_Q*RsfzUi*kNxeTxMw0)IM)A8Le3W;SP3Pq z{Wx(voGp!mQeB(ue)dP8f6$&ZsCyiRNv*RQ!^_g3BUc&?L&jDa;~i`D|5F&1e!#|a z4s~={*q_umzMcf)02L#MolAR>QUUHlE?4b|~? z_PZNUI*nljNZ=gGC8h~1-&4(amolveL1nd z_^`$J1!+5q;EGT?gIZb0^N2t@#EFLXo?c~uxBjrAQ|2DU1FbUV%B*9FZJ(nAl?fC> zkKw&^e$FNRk-z{I?j)EaYdgMpb394%HF_0stFuIN5)Vf*Jeh1X8F=8T_1|3>4arWM z(;al7j}9Nm>)n@+J0rXzq=Qj{)92UTa(XNxf^~-*^WG{Pfs7=B9GTzw|2b725<`Lu zMvw0piDu-ch^fg?pvirbT>_}%_`ka&h&4o1rjDko#H%Ox7>#f6-xL>(Gm|c^^!`@+ z6z(Z-bh`u8ErS4~oU9Vr!29d(8%v86NVt#g{MXgrw!dS#49Xb&54oW3(0Qi#u@)eO z#b5__Lw%hpee%0X7XFZXv!#7bw4o|E+H@|6kMaNAyCW?%8o(TU+*9lW8ZJz|jT{#b z5L_ffLG90PN#l-KO9OdK15v@lwpoVU!PXDrYn>3B(zLK73P*qzwh(w^HNX7#3}Us^ z>d)Q`3eRaQfVRKVXA2Hi8S5&{up>4uXkY%Q0us`EAu_lu%>OmoWgPVgx+q*7kQTv4 zlngi9JM4&}lL9fFe_N-2Q#glI9|4{G_;cr|gdCNWwAGZ9?rD3u+c~<}o;&xD!N$^3)WgTq%l5XVWe<|? z(KRoBo$&Bz9n1C~KYso|-W#wSu*^y|HC?=WYq8_p#Xx*u@jVs>a5Lh^mD`PFG@>-G zWADZBHGV0&ShPudi#0!wkylJunC0B`^cwf2C!PqICv~bilBum!or2e{dR>kLP<_Xb z+~4NA=Xd)9`C_Nw`JnSfXc|?BY$~#o%9{7-l;}M_HK$8G=9kw=A$>Um2hq1LabCD^ z;Zk=ZDNiqSv2QtkY|i@9P|GSQCxn#K89#RCi*Y#dNz>hn_pd#@Uts0n==k+2WBvoG zNa}D;_=TTK{SW;v-~agSVPj3v^*)<7qIQ<|I#p6dR6^eUggv@&p&#%eRZZwZ$9eY4 zmrskYh)5`1zDxzV{1M`PuA@@heVe$cEKUZddycyN%v5esb^Lty@>FT)cSxJk`6mPyR12Iknlm-(CG5-|^-3{aY2@ zj;}lD;_yBgm^W-xaJ3Qz;Nm9@Kt9QQXuEJe`27=Sax}KO3pTWx`n|PS>g1m((ysi$ zP%R~#-nf}zm)3Ae?XSbQ-g+m60@Dk%9>gOvd0Ab*MSt2?xkDv57AL>2dayHFkNk2= z3q3oa9r~Q};(?l%^Ce|PgD55;gv@&8dv-a$nh9|wbEHXHfwP>-Jk&)p!<5+0b70KYrefY`#$vwJvO0)`NgDcC+);+xtXPKYt z=v44T+i@$#3Yjhufv2{5@;U>JfT?57xg<`-T;Q1`eTV4h4g7>@C>w5--#?D@`pH)J3oQulKV}Sl7 z4G+WzO|b`b4u9+H8kJFMzg?Os{yT)|*L;%Fj)37ZP@=vcx_qS@m#)xAOa#LuAR+&`uio$7tuckv*dQv741hZss}=}HPIhiR#7$nD99v~HqT3Kyo>GF zPcWigT&LUQbC9>>%Ujh4oD9mh;av2yB<~c6Tsq1;hjAg5sW{-pr=oiUPa_^AL-QVJ zEhoIO&S>X2V&4?HE>!in#nbdddTqaPDIoq|~tMGN!PTr(~7KOM)^dU#%S>PsQW+cei33FWNJ<;NxvSUbe$UbVaL+T3Qj3hr8QLh2Gl1#aH z0~>jGf4Ox;h!;;JqX*JMWfhh4;d-dv&tl|JP$zD~_eMJCu)zDup_=?Ex5Z{9uD@+UsMZWtV2-cu8Uwh1Nxp=nV~hoM~#C zZQ3XQ8PA`{iwoK{Ts-Y^A8B_H?6J!}Et_t1@yQ!VsQDy*xUPZ=J&BAt8kQ#?7UZFe zj~{t6l?0+;)IF;S#{^@>7Tg!bUdQ)wUXOuvq3Ep^VCld z*0&tP*RUAbJXh(S3Nm3e+|j52aI~_;Cw!(25IS@K7BRukc`WFwy)JUKlEzWL9xzT_toH= zIrTe<%=$zJ^MO1Osjun&##b4TzWwAiEN7dKteD#zKPU!_bgsm+l!A-1Ih0Z=B?` zHK}wpKCI=9_fsvN2FDkTV|*>Y^JbKJV~$qKwO0crYxnVDT9PH+Bl0^YEU5aF)Cctc z(pqthMod?BEG=2}Fx4SDC~b7^%kq@Y@zY_Kz&|7jxBv)2R}wh0bvM1xoOne@GEeR3 zp#R0OWcIEuF7$}kZ{TSFWmh=~FHFAf-)0@*3SLhlL)=$36eZy0D(%3kT#f(#@%Qqs zHCa8s3evv(x8TH;kqYL?8R|jBrH7l2d~^M4dSPqq*TBu%xIJs7^f#b(7oUGEWZ?j_ z?q>TiDIzW;>sZ6^Sg7j0=68SdY1IGT<#f;ikzTIv|1!=j)Z9g#tJW7KNJ2lVS#Z_y1|oy*#mTlk||jB#>f-F>z?eTpNmP;ZCf` z!tE0e%1J;*4qDy6Tk3}Y{Zts?YKAGxwSp*Eqw7ZjwL*UJLK_G3+5bEg{G0hh(ZWAB z4);iof-t&_>7ydzm9GDJoG0X6eXj41;l}Rc1x^WS?p3 zSFX(4_}x53cNeB&$Xa~T_uBVaJ|25lE@R;^E$Db>zTHvsyMz~3cV%mqNG^OJC)f@t zLRn?E#4a`@T+6m1RS&^nl&ySoEEv85{wq)bT)QaBv|{DS&Nw-jn)pa-!BuBJFXZ68 zY@>rSK=Ef(>Str0m`o%Po1dvia^Mt2+#eTRa^%xv0iDOw%YAmVKrLWE+vEarZ-QO* zco33E=EaY`@HycA-)}Q6LSVd_Dx&<5$xHb33 zW%u)}EAe1FraT++nLy@UgR3K?^VI5$RkFu0F!!zg^&zMl`7~jFh}(v3gv+Fy6SR2@ z7dmSlueQ)>Kg%OMqM=~JgTJvs@y?Z=kU#HDO}XFoD6=>UvJMZ&>h*s>%@X$U|9l80 zoo_z3fE_u_VUO!x8O*Hcg@OJ zCxJ|HI0@kd#ZX**D+*S z7J0sLHRh=e&lH__VX`TaA5$=sUG-rkPIM<>xltzlyAHN}<>2Ga0V zk~)17<4w<&ol$z_U!gy{tfuP_K#07qiq$B8sPf`pJBFL0@@vk&z1{` zVFY&Q#tJ<4)vWZg{Jo?y33%l-*zU>E?I_8yjgzsfnoe!)?r;!5vS|MJ^WrV3E^mI- zt{1o}5F-LWiy8m`hPO@x4u>Q`B7mK`Li1k%>!Q-Tb4=rT z|GI&})eT3W%kwQW#DxC3Bdbx~C9HjaCbx-8u9*G)n0ky_OEO;CA!|3G>LHFiC(_Lg zc^@yb!6)u4U#pflqzKQ3G|cH33IqNvUW;m1x}~fVA@2TmI%^xqpG}ZTQ7xeNq@NEn z)qktsu<@%FcKF1T<1^2=0Ls-RkSO+1mN9T7nLxQ9i2F!Zp)oj%H`QnF`_b)LI+-eY znf9T-zd7n(dHIv)FXAit{w;s_uVvXGgtBy_YL<4-fEzb2lPg*&ne$clr%xYmZ2qZk z2r~fo@O*4D?q2GP`9{hxKZrTHh~!ZTYGG4qRCydL=jhCKrtoX&bvLkhei{s9gEXd$>Y8#UE^QzIU^*+9N+c_4q+0eNV}#$dYpt8z^7B z7{oR&10&BG2y`|KnZD)R4Y*y+dk`mQRsP@}sAS8ecJBok9b&N(%L|LSN^UlUl@|hS zC%%NbA25`x)Cvz!yH%`5SkP3}1}o1c)^w#yJu-Ro>Qt#M zKeFNfT9Y5$ovve0)OsU*J*F`{4S@1zFqzRVq`-Wn=4}wO4Z}agBF#py&$JT}hapA? zxE76`uBvtML=;9-?tii9y$*z=JbcvVlWg3zZQk*rVn<^gOa#{ zXToD!;M8#@o0xSBaeZs(_;`mLP?k}1D|kkgfB#uvSusONJs6J3|pI+!`Ma?28~GIV3~qB z(FKz>61a6eyR|*`!mDaGvze;^+{4FCar2s> zH^C{5!r1;%V~so1L{bMiQNCTkQ-eSRjg606>T_$G2hz4#NHDfW&D-<*lvnCBeg3m8 zlt%&xtQw{mGH!!>>L=yip;>uO@MZmJWI!O!u1t)zaci=&zOuHmEt(h5NSl(VCjO;@V13Z;qgdC_5$%73 z%Re5bd`V4s~|QGnvuuC|(Jh{I#Q8ahg(>eIL!1u}8g0gd!mjvLB=ZzP@NeZB^o z``~1Os5>eUlKia8({eOi<>21dXG%$@Ox?@Eu0$n^BBS^>*48;|jnm3y$J|At(eL(x z2E_e3^=&Ht;Zd>Nb#)selk$fgNev@+;Vld=y!sq+i{V2H(LmzBi;e;#$u}YLe>=^H zx8{1EhZ-9mD_&nWh^w#FM>YL+d$Muq>yfRz$ih)yZc&XvMWWRR(_CwH(WIiR{`Der zZ6`%ZReP=qaYQ|MG^zoT-k6uya&bpqVMmn=KC<&H_O4P>4xG-+NnoX5ARYA*J*X)B zy@V=k6ylVvZ_J4ET2ADKgAc_jR?D2N(=d(K>hm} zAtN99U?FsmM7Z+f3dXJ`r5rp>AG7!f87qu28lhv?yk76PC223Wi@HD#%Z|z5Fb`|W zm_I(pXc{>kSFTy6vd+^o_!L)@U|gP69)#IK)z)zoLY7Ax>alP~NOk)>JN06Be~R^n z5`9(subHp#X`m-JGE9hge_X;@aRReKz^$dYl6=Vn<>lJNGFNA#jd5Vf{qa)bZ+SB5 z^tl$ZtJf=b-GDRhwwhu2of?gRBLO{gYn49fJZpI6uznGGWBr(yqJo5|&D3orU2J{~ zg*n>(s&=PvYdK16!$1Vb)IE~`0o#F}bzfwd>UF;Xq(=(5ii21M=(8a-NhwpGMQ;{Q zVKgbkBKW=e-i`kX*uIS4|Dk-lm@C!YQ^IoWIqt&K(ap*i^O1jU-;@~~;W`3f&fblh zdtKGqPHUT@_z|VP?;kYi#r(gj35g{0z03jJM$l-^7`I|=ZBWnFxc%~e5&y{qP_h<| znE$isUqqpQZQ9;;xVLF<+oY(aGd9svwM)?=w_F=JI)nkOCJv)Jp5*hKaR-D)Pu-;n zJEn#lhMilu^EI2`P-OKGaW%pHSM;CAYI}=Q6?wa$?H-~p?u1-%))neM9ei=u9;)~l zLLgIh{aIMQb+FS5XD&@Qk}59AF0+95JQ|hXIxL2RN||BBGAi>rCeE&QsmQwx7+vx{=1H5Ax;lQ(XSR0liUi__Vgug?9I~1f#;1O^mcR9bROc3^UDlW$j z5*cj@jkTj(5!Q7ZMg!%3_4Sq6(gT?V&~s5+u~~BscYMVS(nGsy%}r^9yL=hIqs|qO(hH4ZO2!&=Qg=)2q^6`ptI%4E&#qfX3fD$tN+YI<0Z-$Ds z{i%>Ft#t3yA1xH0acKJu?u?ZdzFB-Dd!1r5YvXMf>ie=dof%5-@>Vvr066(mU?lq5?OiXzEe&Dp|zR|HnmwL#S^SeFkH>XaSe*b>Ubs?(og zrm7<*DNf{E^g{CmZcH0Z zp!u6H#rN2x>bNM${mv)ps|TNYH=ZWw`ZQ`Ljs~v0SpIa|o*R>qq7XvlfZSl0OI}Qj zp!d*Ato&mLlWo6rh+DTZ8olWwe{kJY+UsqpkZkX5lZnTPZHY$j0!OO?t$At!aZdcu zgIA;=>}bG}yK`A!FP)Odpv?L*O-J0b8ea#}iYlG1i;bK4;T?`L@+Ii(o;CREFrCui zuM_W~uB$4(xw=|?b8N1NNML~Bu07(9xLDiD_{pv^Sth|z`Q^{KnAe-D2ExlHY9$*} z!wb{el{^115na0^bU7uo;&}yZ58&|iY_vz%!v)hv&$c#&cK;A|elH+N1EdiwsKltB zM5xs|88}Il>4*7}NBF!M6~)?<$Av_UgA-WRk}9tXb}G3<&_8PT#w)nA$F2p7UCWvT z60~dt+gf)j)K5*{X#l`Fb<_yypX=LSf%mRs`yb32QxB zwVvCVWI=g3Oi-rh2AfEDy6FvvioH@PkX)Gvt<_x^wf?*U)%^^lXZJT%Gy?2tQ)3slb!R zafx@`6bLnlHqpamC*LZ_mLn?!SGfQbcw; zmALd?BtX+A6~h4J6SV_8DIoET2;57+gI3I{jd+6B)N@H38B$EjD!XBn%4sU^1)t7E ztOdfeRqkt1;Dt@7IBn&z^i>9%xawmMdJVq{A`NsO*#1Io`mDncN3MqUa;AE(lTUnY z=x<8@$g4upNjsD9sR2ipiO=D@YO%fvZbr@Lhe%kNmI0){$y4f#*=D;H4mnxNKOZVB zwTBeskZ*u_veZkX-F&#}B#H}`&+uXmN|>YRMRg;DURnP}Y@UE_71M1(9r*N&Kc1P- z%zlwsNTqSSsav{_JKPc5x1Y~LJUw3P?%zYH2Lpj;sg0iqL1$A#lauzVLrrQi?|1v- zQkL3Nj-b#}5%2qlho@EW%F3Z0xNidC=4QZ98&dD>kH5+~w{AITUb*xA$4hh5!(U95 zO&3Q68%F1-Nres2j##d`r|`WZ6(bIVs8v8)WQYk5wrN)6+7PYyU{-y z-W**xnF;^-$F^e3_;|9=V@Qg}GIqMEm>(184y- zOLtsZ)`3?^wt2Op0Y*m=t?(TQ!zP#P9LfWi${UTj1j7DN-s~V;e`AcHh zV=Z(Ugg(1{)Yd$(`$Ew}AV-*w)DsE!pB7MWYOQQx8n2F{Mh5%k68bk=BZohsjGE0q zTuEQq{P{CKR)gfPnIZe?1N)uEkj1|)i7{FItQmHg!wNH6NX!xXO?;pZX#qB%&B2iT zXbd4G8!^W>s2+pKc?qePg^8&w6VD`&yzT`MYA6pf|_e;8Qo#@Z5#Y zc4GcN*~El~(%>l>rY&GnVPGD{1;wjJ1I%J~Ve{t_>?AXH>Z(AcvF zu*91Bvud@yMO}+FiX$jXG|Z5eEU}dpD#iHNPmC+oJH7+$SRGSY6$UMXxIyWSDCvTf z*}6D#?hw$EOS1$89clrrxbLsZV|Cod!{V!3!2I)JT!&Vksj|&^AwJiKiP-&(jgG0d z)*j{La{LkWYrd0dpLs;Gn{gS#AEyKOO`9%D zz0UjyZEgPQJn1G4R+X`nEc>Ov;FW*J<8Z@&Ipz;ncw-NPo&hNJEelRHF3^Cd0u#AGF#KkAoX=CN~po7hEnt65E zsIlx8Rju8zsWoxjvlQJFL1He-lhZRHWps;<<4@pur)0%jK(?=EoxxQLFEm>P8jp1n zEp{qAN`ovP@$`>-A4u61H|g-|i1HAgt5QlbT6tE`fWSGa-5(f(ita;tt&aQiFHK2) zX2K4uuMG{?cBdA;9UxDh$6#0ZyA5nMngUZTp@>Hm5*pcj)HRK20z_fboZ?vRvtwoq z>be>;Zkt9i`={PeXGV2gj#CAo#L!Q^y;(7R1J&X4w4Bd7aHP@m8a%enDw{9-{?*3y zdhMB-23{yqK-$5jQ=eree2}nSRj*Z9-ffbRzBhD(^_$x~kIPKWRcC|B9~fuMug+bF z$o(d3FYqD#-N`s0sW$tlJaEV6?qS5($LX<*^_8?WzN(-@r!v!~yT`~P4@6YKQfCLx zQm=3EoJ$+~8=8L8vGHNZO_zlN{gH;jthq z<+zw~Ac5p;?lf3*2DuA6DqhzyjwnBQ;$^pxCuj!kzA_H>5;;I}2u%xRjKy^~O8X#U+ovKSB-7#|7vCm6;YE1|ak=_w$7p7T5n=!H(_LFLhE zkD)b`@^CAP7E(RCl`er%@#@B}a1$fMO&ycjGrs)QD)r^YgFQ+w({zHc&ZDP{>PMXe z+K!aEOVlAd^>Rpwm7#Dc)hJpqHi|kf29sucV^V4{nWcZp#3oD{(Slhx;IadG8(5ke z$+343{UU85KlkUew)r468!GLtvkVl>37rKn^8k1aY&@!XlkJ*!BLFM7u*L-Rv0uBr z?fm7;m@PsE%l0dDMyHhWH1s*2Z)?4$r?u8tb9%GYcCDDER<5g7RJN_wljirJu9m`e z>($?C4rQgCpAG!F_{ME^h7?oIP_@#HBjjk;O~}xqNg;0ruXxxt@qyz(>qAF7kpOCN$b6~T1B=q4%|0kwi`I(sJI*GW30gh zQ`lwehik@lcL>cTAIV^5e9sNGfhG0hq%YCOw*MFyxb(1n+1pUT{TWFNgnr~5E$^){ zsFIg8@G-#YLL~^~FJTX=R$ItdH^99-j&_zfbmsSgJqev&Ouyl6RgBjXF(^n?{Fn*3 zv|vc`WCM?Mp_x!A%byL}!mEZ6OItMVQ56+gk-VtX&{MOyV@J68TI&3_+wXY*5qwNz zD+s*M;!;0IRXv;>U+^gRy{RlJyDl$hvZW$TwvS}!x@NdJDq+{3;d^a9LI2OOUfoGN zr}wARP}ET$r?mf`rXA?;oXPvj!nl5aPLED0Vc5~siCFGV_4rr|!Jz1_IF+U@renyB zC3FzrLq<#9n9TO1JGF!!&nf(R?qy%jrar+VUhVpvyeo#)*tsG7-G;z&e^j4q78~U} z)OE>QF|^Cs&ictFALsO3O8TtyfHXOPWY?n*kVjC*CN!p%rmhc4!5{*qK(TE2cUYzn z5Y@)rZ8ciP?Qm)KZz_E8yy>6MFGtdkqZV^s|FItu!l3Zo_qC1uqr%wEEB{A#dm5Q$l-Oq z*xeSC{=!)jOXTYl@|&!TX(oHliEpZIY4Yc2J3e3GyH-^zlV?=B{biEZw0*jz45WP9vqtfupPd%KX!kDKr5F|Ud>u36)b z3_QrQ*!0GocjTsrMe8mv@&mdp9YlugZuk4AVZ^8!FPiwL(|I!0B4%0!2L1Qq)f%dT z>HT`a>Rt~UpzAwBDgX-$jQ_?gG3ZR1(x*T%$>P_R>MbFze`Ox1-swj^xlc1-8dd5? z4G)>^f7ids;<-L_7EPA|Sbr>WTvM_%M8_NVoT)Z-zUqF^gZ!#8d2ffOJuR(SQC2b= z9y1A+zLqI zhL?sKq!q?r1$HsMjl!$E>UZxhujXyaSW%&0J1UsO_RtD;T=GizZe+il4m!Ik6(c>M z-XUMM^0d0q$ryhy?Z=rVbE4-Wdjtr>Gx%FS)8vE_w@dv-HCCJcv9XSio_M(uay3@A zVvUpJVBv>@E!8YcYf;H9(O*EOz^6l|Mrs7S91$PuKL~UeclWhk7;Oi*kQIp5;ltvR zfn|D6Nz-QPoDFPdetH0S&lz=fJ$ zec)p#$6LE5^@EGNsaInw{JZ-{tKh#bk^7K^W!HWAV6pTAm)5<2BL{?wyziBp?=Wrz zwdc$qorXTxv6SumScPE#WB~t6H*2Q0f1%TAZ&zS}&A#KY>Wasq)r z%-QG@TCdQVUiiCvx<>Hgk(=l1kRc1nj!Y>nrH+F$#D1^Rwq4&$A!80AS!P@wU*SKK za}Z-YWT@$=EaArDFP&bW4{ym^=j6St{6}H$&-LGa+jF*WO!eZM5*S2|m{ORj9=tm^ zM(-bF4glg;k{cU*Cchjquew1wSYK$s=_`Nyirlzs#yL6NGe7y*vpbRAruu+jnw@CG zX>_bEQ+x>9&KS&d^1F#N#7=>PZH&1GIPB&=CZ*)IY)FiC`CG=lc8p&AG`37o4_aE8 zSQK!XUDFajk@Bm$zW_Vi0+c63V+OD)Pi!7to7g0*oTQif7-}sL995l~oe$kRc@1(l zrdGdeOmx{K?ePD$bjF2$lns0@8aaO;SeGVq-{6jyTaA#+#oP)aGkWE^hA@aK)h3%1 zFtg9=vH5gI@7e6PC0qXh0yB99tACkemcy38Ld3w{JcgiR@k75FE10p3^{D~x&m}&Y z8LR}_KnaK4gB!T`y`;MeHZLt!1wNu1CWqk?7;L&8%>-rc~V#J*lr0~DCW3NR+uG&HO<)NI>;qW7qjI0 z-a{xwrQYX?v&DqjZt#6FH^WE{!8TU;X(#BUwb8ZNFWDxw^9fW$~=z^GP0;N>TD zqd5eKJ%&=s4}ANf<8S9Fc|PbXSCpiqak5@kCB}1(Ntj`DK=XiU@wB^A!XVqHn-djN zA)Y3|5ti0|nqF%*7essa&510V>|OwPx08yQyk;)n_h48;&A=S0=&4c@#Tb83eYCK= z{{vLu#R7eH7C^;!-)oJtid1Ji4K~$X$#cV$u+)!1>7-)VEkhfZBc2K>1lu63F(+mL z`x7)2Y&-6lv|^~%KCJZuql?=9n3{gb;zKX}_?n{TV-FaXh8Tg@na_4A&$^<&Id=4U zq~gVz5Bhbf`&EAG7(y9smCTUKh~!emsz;_n;jUp`b02sl+K;$0z@o9aQ+Tp%XNvgh z`_GxNG6L&`AN6wl5ItLfAJG=aQox@sol|EgSM`z2zZcqv4h=#&`umXF5q5%%g>XeY zURB(&IZ93%VcPgK!$G$1L-=}u*h|=?UyfQS@Ue!qsGws|qKisd#@Nv?(CC|1qT-3;p(?ngEmm=xlDo=3b^qC!2vi=O>3&sse@ zNYlR+E2qKcmqwJyYaDSpW#27M@aDPE;SUH_3aTgPIryZ$R|PxP!F#j<%wW1JpyeCY zCDf-MEC#Ye=+$YT+T+A-Z+LZA_-7Tey=)18@l`pMG0yip;494H)cQ{D!&4(#raw9B ze2D#@m!tB1cw}cc7A62|kFZ?bbz|X=iaVZvVRDIQb-vTYE3`~$4UNvXeWcjOy>RD~ zjC)}?^6ANZGwrW2JTB{N;9ggBXr*)(GituMekqCD&*aLKV+B{u_W@154N&@T>*pDE z=uJud#%tp(%ZdlRMzA+omYfjv1Vx?0KyL*v(4=k;lA#?MtMZ2FP=I_0kQi5Z-LNYP zUg`_RZlcWu&$71mnj3XJM)D;liZ@L-cR($9C>PDmWZU=;ikLl{i9 za84}r@?YWi?Nw(fMi(;ymo_lESURJU&rF-&c*m=*#b5}rIT=2_*MUsvr;p879s7-^ z+NZ7n>cjKRNq;)-rboir3R&^4d{E@o_BfBDT!@~I+& z_Yqy=i&YEl_L2-reMVSn(^s=BLl4|bsScie7X4TNg?VL`hiX>4^jr-5D@1a=P1UMR z=}NvvmmMR2I1?6Ca%z!kiSZ95yu2>&7vf#saE-Uu6`}(htp%r;hpE&VthCunb?ikyT;9t!Gh*|r zg*tn#id8u@HEtqmj8-n^*!31TEM*y!wphd~ox=lJdj%FG%uCNsYR0dZ8o$XN3{2g5 zub87<9~43G7r7EehX{yl?X;^VC382l{hD8FX(0vp#Cb@)HWWMJuGxC0(z(KCNLUe) ze2^PQVftf4y;Sxf_pGQM2wwev7BZjkNU407P$t z^qL#Zbg?(a+=XxMw?|gFbxasi`Z{G7Ho1qU-PGgc&T}JB;~3H6;F95mZ>FSp^PO20 ze{3C@XWSin*%6A?S4}t>dGO%en{w3JZRHrR>cbSa_j1XC&qML9em0uiu?j6rH-$l$ zNIhszw$ta8qpsMiVV(JEOn69YrEF`Dh@C ze)1ENY?t~vUwH9o_F43Q7z!}L+_`)^LvcGyTvU@&mUrfJ0rLy$v8&_%{sx1iP~%G* zGwWO&FQox`lc?@;#0OS4wnEhjMBH`xZ-Ec^?Ug^F@%$y%9A{Lvcb3a0)fzf8xg}FG z0;}guMc9ticI`xXV`#q3D}=3>yhixxa~8k@nH8%K6cgDqn|N>Uu+7KvVp?x(ftoJn z#`^e6T7gG=h!5coRF5Rf`d=S2rX$G>E`zVk1h-OBNg1?1=)tNbR1@XwJwIz19kc-QgS` zea=~(`lg(>PSw8i^eyKAbCV-nx+O0ELDZJyN>dO0S z>ju~T-d$c>^yPJfY*=*Zv~)jgB(+*Wc!h{`0AK328K7h@YOGL#?K=UR3xMTv`adK4DhILwd5nNh(Zk_7lVLbjb~8WnkbL_*T}qKmD&2jO6|H-i5&3J1?14% z%CfQ!7hF%&@Z>|4YGsfK2y@s?Te=1r5bwAfaAd`Q}9$z!e(mCa3XnlA*Q(m z_&7X}a1@G|gw!!)r+%9GsrfV^orjL3W*E+Ig4;R1V}f2^R&MQDL8B8@9aT1uD{*se ze$3Fuy|cZ=9Q!4XX5A1)74U7ioHtjjBYcsiL7|+rqcfwuwn`qQ`)TgI@-pUjvc`OIz#ZZOL3kp ziVwx`;K1!G2NLyC;agFPXQu&#pb5>**+Z+5i#eQr$i&*v+vxt2?v)@_=G7TV2&e=n}LA(iP4D16@(4qEI@ z6@du~vJImYKEKRNueW)4jMZQJJT|%`4U0QHD^JE)u=2P_VW#=hOaMiN8oMr7z*bJxo(-(Rq*t_95=WX_w zH{wQFcPkTH)U;pwZ}~OnH4U8bMY~6Vem|$h0Kdv6W1lBADxH1!iTaVl;J?&q?p9mk z{OwVYyAG{u&Y^3|BBSnbWei7!Hg3^<4{pPo>(519w=z9r;(^Ya7A$!bOpc`XKxWPt zWh)H!4iKIBQkG)M!!o*BrqPLGuE>IgLTJR+V3YKW@O!-4LcPe;Tv&vRXU5N^TVWl0 zY%9(Xf47d({_GzLqHTviqE&dOS#;@i*oe0u#~7YK3gnG>3%4GM`|s$g?sSHW1N3V+ zeUGNGx$GOe^N?4K4J_3lQ8;m?UmK#uTmcwnB6ou?tl5ca){P1_dR}pIzw0RVTMRcH zP3@L>M?pOAd5t z-h+fX320Nk9J;x#x&<%q%)Fyo%$c8=xH2Q{e*2@D-Tvd!A9H67=L)Z%as{Oi>D!SH zGzvy&A)O)eL~Ls+(?(m}G0{1uTgUshBL3UKA#-gSMmfrPvveL}V0TxRs{5^YFTYv0 zyT6U2WR2+!H&Qj;3#^O@z`q}ikuDuEW5`4sScF}Y7tB+e6t+nPt>1#N8Khzb^6{RVG9Hq>IXiL2i&g* z``l~uYH{wsHCPQW3&Rp3O&p8mvlxq{B!<%bMBHE`1`YXw;193v0KY>I(?T0;G?Wjh zh3Hu9gWrc<_BKJV=xMUVQ711%j^5l&WP$PA=~CsKHCU{RitVoX!?bBUscjHtl$2jo z5qyp*`yQ>Tj*Z*dFvHdW!T0)p8C>Kd4W$e15v3Up(t1?eQanvSmne8$0}c&55xVkM zm%T&(l?LS^x!!vKp*-6iJM#y;<1}JMPc{tHHXaUqIxKKxK;3=6@V#OAeAJi|(q!^! zgAM+fKL$|!BujFMi_Pl^^B`RGl5Rm}-1iH>*q?!3p|voyN-tp1ChS7%7q zhbUKy>Al&_pBuZ!DXAVV0FB}n#)KC)lz=$lkQG?V9Nv#Pd^SZPFv=UocS0 ztSoL+h33@pTY-5&h}B;DofBhz8!A3lhPGtkIrN$lr6(sW)oR56AB9NK>- z0f5vUtRs=QF*P@x@|hRJ@02e+F2EwPz;mu*dxWD0Ow1Wm{_Gz28Pn;XI#%=QE#YA- z-EQ4W?M`!K5{n(^2|S|Ss&_^JD;yK)Ge#36>&c(YcPuRZB}vjFWctOC62tf1Y-ZPE>Ef12%+&zD6AKSs1iu zf)UR$#8x844lxH+Syr6}#;=Bf*lzd)OT``B`XFLz>HYS{w9#S2bYV;&r8CTbKkHYB zrRt^fcD04#s-EoTy5U+oR%f^QD@kW%pyvrBW?ohXu0vB%l}8?^g?DCtius6Vgx^9~ z+xA#gWawmO_KYyAK`3?h3B7qB*|+7pH8Uen7Vw&(LkA)*lPoexI%T{bDLK$r>efvU zGmd5G&qk0k_^jdIH&w8Nt;*C?3t!-{luSFp?d-?ue|`%f%2}7y&NitCLJFKLJQ?(A zi;+mlIg|0XVh4H;`>T49L*AFhBEY~asxE-Kf4!b{c^g2= z^%&@l{aKOJP(ZOP7bjnVY>T7Fu5CPOpj(e9;aDLpgo$g0o4{>J^qa>B^GP9c6V~7P z9-4FGD!lxaN-(=pfMZ^A9J+7}UP&Pp;47CPf0K%9RIMI;$qjXMip1p--oeI_UmrwR zH92EN)%lL$haxT7JE+|+qH~86qSfF0cqjKK=bJ9ke6xI-MV1qK*)IawRWfXJw?tkX!38x4mE9Y?e+6&Th)3C7Zd6k)qjdj z_9MPWO&tv<1yC>B939g-dCt2&+&xWbj*FjW={V@?_b?jg?@0Pug7)-2|EcNMm$GF$ zaru+i!I+}>TotQVNPfMTILhn+BOspP3W2!JQ`P zfc~geeRQaES-Tm=0qm~7lJRyevVOOICF910h{sKOoS^iP8au&a#cI&9Q-U{Z*Qh?s z`~*L&SCGy&`$$DvmHx-Kx{=qv*17W4WhB*g>KZT3OTTf`PJ!Td9n2?}qM2)_oR$HK zK|CyI^NqQ!f~D0{h=|Mb^Jo?cI(pu|kB7|vi?R2NYN~7BeXZ~)AS%+EN^hZeNK^!* zcLC`{dhea6h)7pJI*IgNLhl_SHS}JDP(!atfWXf4zI&hX-)Ec;XUuOaBP%Ow-Rqun zUh}$tvp`J$r*d)X4Qbhh@c|8qK14AJdT#5vI`=6;hv>KC=`-E;>K$8mfiaoZIaa(QfE6t>S*oC6Xp}?rL4_$^;4~rfaP|F3&@;2osZt%RkER4&@WcJA z&K}f+xwmu{0wKTjUD~CdNVE;hcs+)FYHC(RtyGW+y-xqYuzBo?LOPyc)HS}KJUyD)Cpl1gp-VF28Ycp1d)_#`9u$R{h)Lq%$$Ne=4Wgu}?-2(w_7L{rqqXU% z;muj;>n<62|KNF)6|Kpp7>gj z?^?7$Yvn)0@}Lz8NNEyEMUEs+4qU zF5n)1(9_e~D(#QiO-+Qr#$5y;zgmHb+pQ?y(a!gtGmz)jE^}Ye@A6LA%fXt~cKNsK z=dnvxL89!#MGkE%g!R-%(KV^B%w0L$9F;lgl+Wt4Rm6N@R0v3jb%W6|r=os(9v zo#jxmaj>zZYaJLlS6A0?w!OY@LACi*&~OJsHa`RJ4K=u6a~=M1VX#HB%0r)Etn4N+gE{g6heCGn)-M?ifcn>{$QK3rxj@pR>vwpYfZQ z%)SCJ`rI`IU3(5TY+Db0(2P~^fC(@5b6fwtCq$cAlkCFo}ha_xn5Sn056Xk`VlkX`8d!WN&-_@8p*W=ucoMmjk#^>B$ ziLqU;8Oi*IdRzkzM`iJ+mG3`1&Q)CYIGyu7n;+fFd!v-1o=^n6=K#eJKsOz)T= z`PbSxo($F|0S~wpnC@OnzDg)bOQ`uL7b+wf4C*yQY}U^Y&;LnVJ2+fz_VVao?O72a z{^I6e%5{a2FO1rA@7qOAU*h_{RXCRafqBQJ%?j-Nfe`D1%WzDIVvbWTuH#8W3OClu z9ouIZ^!UJ>7O#Y(*%?ROtd7*(=@`bpr|6bVH%i~%$BG-vOHvd3Mv)ZX2FPxG+6?xV z-fgXGVC=&1i(>?$Hww25%#yCnI@R= zxh-*`UZ9{Gw&5I7gH%|*8i1v} z4m{yZpOKx)6HbG5lIIzTXyV#QJz(!KebzzFt$VR<)5F~w_6ljR_V=itn{6=!lS?dW zM2w2_n^!#hO8y`j52bYs$jKP!JgzqI=YY^b|9_bA|5Djqy1MBtOgi?vzA!Ur@IJD+ zdL)&c_Hmm-SSeC2$;Ukqz^ycnpU*_RChofWGA7zIGOXP;VYxqZ84Lihg`ZwMlxq%$afYWc;#K{U^fY>U_)X(BcSgwOUsYT%|OXt>|*H`eyzNu7R#I$cyD~TCO`2 zw%^G~kIxC-3!Va=w+5-41qXtlThff8RB2+-Re4#+$!bV!rkQwf65=E!m@O>*d+~lS zO%enmJ5}oNrb_QwcDFHpEL)`sdmbEVl~3RPOSBnC1PPAfBM%CgaL z{s+=`Dm~S>9r#xBqa!iYhjA)HmeN6WnR7pCcyY+OvY{=xda>v3s+uFu_az1*uoK^h z{S%W5n9T*$d~XJ$($+#kol^XF?XAfNL!575|L&lZ?Klh~G-H@v$bI+bPr#ZRrY zBQ+&^RFGttkI`)7xo+x#ODKA%PQOTOKzZocJz6*Bk92DJIqR;rTlQgu>pw~vZS;2u zOVcVb%=dYVdwEEBR`Z`lPSi~w)$#@mF34(rxZkQH*gZ3M_k>DV&;?NDIi&iecL5i3 zS!#{Lar_SBdSy*csAFgJN}mt5H@GG@wh|wz{TnJj%8t!!IUL?8uCPp7km;C>Of%P- z$P|{^ZG%OB{^noem>Pu|Y;9&gZP$1E?%goL;z+dAQnP$=-vwJo zTP*gXuwZ}I;9*+iqIcHQak`zu&K;wnJi}AP`_oQWOQ#!Vx%BgomJXbn8;_ludCsII zR*(1rOJvs#Tja==h&}`(1!5$s%xrB?(N>x6I zshlRbG#I30;6VQ`w;*u6&v-*0cEArOxEph~2g@G9RQnpZ2!>|2Rkes?lvEG1NC9Bg*;KP8yA5Yh$0vh*;lfBY-cUtZ`B|39-v& z+JU`K=d>@^d`k>p#@)D1+)SzLv$mi!iM0fqZB~7B0!}ElktPyk!oj-Wa#KIIGb&O+ z4WeHgD2Grq-vf=GRt^k4+MCOjM@UMLQCsP20LUm&!Jga-^zzF9Vh)k$6~9qc7qm`8 z0t!~Y{My3S205Zmsn)+fPRL!)PwLh9O(@4H0jZA$6E_t%HZhzTUfxQ-Z+%ki-6vT; zB7;lyGHS$0F1XjsQ}Re(yNYYnwT3J1dGdD+owS8tZ@gK}*nc1@mUJ5E(};YtxrFsp zO4nKl^Wr>=7LAm|Z43>#-uT$e!CVm*$^)9fR)%tU%)4JwyCaG&k0hz(%uf-D)61LY z0r7827AuF&>=?4_KH|SZJaEFbJ7W>{gA9;+ON(eR))cURAh9m&H7sg;2$6%@E}@m~ zOX7@oze=fR=9g`o@VV!9xEFts!|Ry(B>dI?kj}9q&)elGw_8n(aglbR5HgUAcv z2;22L9q&-AG)m(_lIQS~GfR{>yl*P-Cb8G&T+8#aVW?(14%*YkEIad$y%y$FNJNjc zlHO+a61jVK1Nt>-8Tg0HMKKAy>~OY;0Q)+-)jZ! zFgfCX7n_OKO3a!X1Km+<=Rly;x5I904HWJ=IPm-@E5^!VeDMDTM#&>o3g_nZls?_fxJPQJ#yvI9I~fy{8q=Q|cnB z2{ahwne!PC7yr1M`*Wjex^`+bKqXZrYA=PI=>nfNpi;7UDttqctFiobJa+%r1_!D& z=!E;~Dv?KEz%*C*9)3vfaaQFq6e}c7fK~kGu~%Duz-FRgfx1== zm)u_lSH(Od9M>h@9l@1hANSABx=GRm4(pvIQB}@kK{_{7y!9^$ddB-u(4=W-S5}x$ z{fD8DP+iIfw2f&FeEwXopx8|@LyCYzldZ&2(Y{k%X*&0Woi)Oy!5%%gut*YfW5Q`5 z32*OMl;&7w`7MX~{k*tHWVkU>i3<6@Em*amZc%H$As~f(07YM^j^mOo>++A-$j?*w z@>_lqT(vMhb`P!`?M(Ehyc9s zOrvhWECtjO5egzZRM)y=WSn{ov5rFsAPn-MWJ|czs$n&#WI6virb!XSI~Z&(Uz;$t zUPQjV*@BTWpXt#{{^dPTrX7_g#aI&yiWR?cEyX>^>*`*gq*Npq!CVMAT||SrHcq#W znt7jrrqJ$6(b3dmK{B;1Q5?JF3SlXcsMb(9>5t6hphtYOFby`BpZIkMg4d#O@SFMgwKG8)`noTN5*381Fq1T_-7g|GdJZ0zDv=d8fX%8XWW7b%V*o1LP) z$z=-ar4}M2Z!@tDB5AhRbwgL9t7NuHFkp%ovo<(pQSbhq3<_E2Q>D?wPDp(yo6 z4s7c(3g(o)Ox{qod~6K~(FKrIdhlc^&0H851hC|kmTQ>!e-v3zc@cfJl?@J8rL#IW zZ=2X$V%iMYk)l~ncCH)J6yAP0XcpIC$0DhO-&l#(if7!_m6bP2vQ&=E*0L)5R8o){ ziAXAGUlyN2?WX|{b|Dv;RTvZ2*Xmfb_z(-fsxZBL(&9NW=A&^q68J9K7D0c8-S>fn zmX0zTb*oLQnig#I033Lh$cg9g#k@M$($B8;LbW|`X{J(A1=yjO!z1T7ZjKZu%CGms zByuaKK0Hc0E5#PV29dgt zYz;**D=G<=1?vm%u&>w0H>Qud{dhO1bW*K$me5Dg-c(h;tg(c@60Q|@5{e}=vtg=g zd61TxTCo_&|CQ>05B8iWK0|s|mHAs&LvwgJ8wyxOU(TU>Xsx~o;2izoc1k2?(uenn z0Yyp;1?+4asSl?ZD2B!r2EJ<0d&Ec=Y6-s=gP@&C$Y+fl7Jsi<7q$;EgknI_Srmq1 z0Y`n4G-=mBD@Q^JX-2noR8DZW;)l%Zs9$`+?#(gS?h5|Zc@L&eBM5{M60pryVgT=K zB?zboGX^&TWw1c83$KCZlh|eEOEy9^HPpW_TJS5z$h~zI!K2R$@^+Bkz}#z79E^f5 z^SxOI5j024#O@wdO?EitV_iNGCrV@w!<>yVV2L^j%mZOk)r1*7_%8zi%#YH z8N4x^UF2<`;r^Yx*pAQf2fdXLN5Kq*^KZ`crw^GLF6jaTM`#B9v7&YG#K*C4CjpIe ztKIpjwiB2ua;cs?7StqmT_?BebA|G|nw|}Dt(v`euhBf;&wh`;EbWUQPugyM8;@o4 zw_p5awVUupU?1-DjTCdpUc771zVg-I;=7^Yh$W_XtRazSqBV(XxEC))1guB6>i4EM zm^>yT4~Cr#YS#4Y&v#={CH;oJXHF+4TBWf9)}Wl~(FtYYO}XEVgOb@u z?yV@&_k#jd3T7Q#g`;9FMD6)v``x3i8C{iQ2Rp?bZ(WcveAQ8m>DTz~Pr5J~y+yQG9<_(8EI&Qb_L*|C!`alG>{z-H+r7bo_4PC+O=oZQJ)9AN+dhM#b(TM8KT8$&J2p(dw@CK;^@s=YiKWqeXE`ymtYiO! z;mNTUOjUMK7@c-x0J)mgMAe9*OJ@J*_H&{GNm~U67RF189_iVTwZA;k8N0!xKn>15 zGFng0EITPcYa~u=eR^g}j+O-lRbN1n63q6 z)L8bCd?YULt_ru${bfMuc%}c(mU2u7;{ekgBEn+fkLYBe1x;R>7B91SyW_l5Pvx8x zixrRJ_UjB|8J2!MkZhpxZeRlg^;3<@c#dB*O3i<&jd!oCN67d>+to`9&m=x#x|=y_ zb5)oTotKbLJW;jirF=JV&(S#-LwN7&8T%);rkFeX&HI7Ig5p8Nj(^*AREOy+JPYIb zg92ZyorTcnBz(s9mR7l;*7LJUca!!5Z9*~zE?Y{WNelLhY^?uHXy=S~4mMZ0jS_KG zCUdd1IiYLC_|W`v{v_0!RXYMq=P670v=DrhFG+p=frio8o8uv>uA~9C7e_O7Gw~dg zLPHKMe+bCPaqrIY#01=hx6?|p=#NUO%(1momFo-Scm-*Tq(!FcZr`}s$3`Vk^%F_I zxr0(u3|ljEtk4|Xc`PdM4ZhUoMiBUPv&kUppodDDdI<1yLl;@T7iEDoZ>$ ztbn-q$~%MW17n~5VGzdw zWWLf1$DmTLgpdcXsOB2*4fdxrI)){Lp~~eFP8Q< zJ-S(~#{zZOb%Q03Vh+yfP?FMH&h6P79Z=Xd->I0}YO1ZrxGvzQgwjv4nJwS^`aSc# z@q=hw?s{@?4{N%3BWcT^hFsMU^MmqD^@J~_v)+u8E^Y!}a&T``YA`4*8m&1YW{nK& z;{N<0!6W-5{JVG5me2Jedl|R%;~_MM|Io%}zyQ?n>nsa@H@??-DpU^ckI=sQ=wAN? z5h94NpfC+6r43_3gCHzn1Hkt!=S}mjGj^4G+jCs;tsy%AX==HPo2G_1-ivq1pIz+z zNy^I%A`sNT?>yE+qP;3dAv~?o{ARw~F)ln!*1SUhMf4p$dGm@t;q;E$36Nn9G2I8p zjwuj0U%xhXW~sTHO5lD2n+NlCwLMm?R|F)5nFbKDe4R~@>=vg!s377zzUZRWR%-?k ztVw`0zMzMM2Tp1RK!kRKmn{jHr~#4kvzOEVzazC2pNxxFZY`0w)uu3-W8Id|BjCGo z`B4P=B1k)Y8CT#oQ}32uK|2%+6Z!!oZd^!Ld!SC^Ly1P1#GE-yXWFGnzE0xlX*_>h zzMtU~O&y589A1?8AW~b>z2Fi|p;tqu?!4b5?)-=2Q>oEt&#pnsRmrTT&?T_OM=xa> zzX7$2wR~&V(99RA6#E&57z`87vR|YtahS7)a|8Cj?zde0E*AEJXZuX46@hbUz3Dy? zPS$P0t}pyn3{t!)u)qJMQfV1sL{Z+6TDPR&aT%K#fPrqYC-tOw51R0}b~aj{%3z-- zyiLMeWSU?uInzwn?TwbTP+y0tjeq?CFT?jb5UBqlkO_$Vt; z47m5G^LwATMVEegFKEiSS8SG55IoZ0;d8p5!XzGJw`vP@Q^Fw%l4e9sZ=-C(UPa#F|Hl*kvgu`y5cTSNNA`I*3-@9CqRmQmIm3671LA#{ zJCHrbpNR!nMhcE?&tVyfx-j}?frY4d_b%#N$<&H1p5rAicCL6vA*4z?XrA}^r#!808np1C5%t+2sFPaZdw91ze*`f>sSobKU6za|U zv7))9BRTA?)lX#o<}L|vL~a5{mJp&4T`^lxh18pZ34YYl6g?tZB%A5vNmb)k_68(2 zA_hi�J}|VrMqo?1Hii@RMf^FFwp9@aWPKv_#dWz1|>~f_^sE4j2+DIvi#9p%TyZ z$TotlDJSbMjlcrG>-=7Y;$rn%@6>_2`tx1K<-xA7hCBPi#Ae-_Z9s`47PW0x!89ZaLTG%5L=cOi1j1uBEp zGsi8OZu8?bPPkly#zwBLm;XF{Z2KYN;3gk8i`v4E12WVq0?pp4@lWK44XWcFnJ%4c z9)B~`KFILTbnHCa7}(&D5L@9VQ3AaFKcnb>4O~TE*;_u6(ejs(x={N)S^pBNTlA7& z*K}0{?F;x*-%cgHnI9DXBhe0e!Dw_7ewHQ<4r@4gXL;Q_vY|m7tDUsw;8oguEpc;$ zT&im~)MD6yTp54VdjA3t@%{CE(l3Aq<_7Vh_XLPWZgHF6p?NNThyC^o1-Iyzt9QT0 z_8?khqF$H%eo!3#c#+NY^SyJ0D-PegME59f-)H{x{ZnN>`&Cn8zBT$ZMbcw<|4u-n zv#gHjq@J0+9#Zm@fTTBrtx}x*nAHH{_1`dY8c)IBOgAkkitOs>Z-53djoc`iI7qtL z>iUsNGya819K1!%Nv79Yw${p6lKsfLYOfGxRSVcVdBCQHXYP4*;|Kr2ciAqXTGKKH z?iY)jJFn&kedIYxwqayS6ZGTFvKXr6U`!0%)+w22E>K%QXkA%I7Lo|1y4b6B1+D^b zX8sl@7r15t{g-6>5w~Y3ClbrVcfC1#j zyF~lG@Y%4c%vt+Co)S8Gu`-|(G24;HxJ^vE30sf~} zYcPcgcFKf_5}M>|SM`XdpvXFs`Yt5MiezMOX3?4Ez>#rOQ{UFjHjCQkbVvrm zf8H$&oakL*Cr!gwZEnV&A^$c*LHdb|N8M=mQ_ylZRW3h19^AWq(k#qo6y75D%mS)8&0HgNhT1=TxkX6tXT_A+;*;cA=~ef z5PW#~ib@Q6J`}G5F#u=i2K|)k=UcQ{@Hi?aW!S@!r`0tAvcc^8P3;JVaS42Vurx<) z>mq+p4$x?MpM&vSlo2l>(`)&#`V}-uoGDV!x49qrUFRmt7;A-p1(|j6|H6lPLgwHd zNcffp&FAtXTtpX`H_z?D4euRAIOc+|*_p?CaRncX>Umg&=+3$Y)uer1h?i+rkeKt4 z*#7{e_xwp;LtD&|-6DOgCbTeR)lO)?nXF#{27OD*0*2bbX$@Ysl^Rv8v5f7G7Vn`#`l#tHsW7m0`h5 zhQ=n{JmP*%$I*?7Y`G|<`X^^HT~Apm^|Vi|jB-qFzoR&zML_{Y;j&a3YB!Rt0UdU& zuw9RFb(W)E5*X@gAX2c>mDhCUl=b1{B2PMh3=cPK!R6l@cKuxATh-a#H7CTWrLkE< zuwOzlzZR(-cCkCtAkdfKMHA+h!j7R=JK!nxt{3PNkyy&*L&@VX1mQjNF!NUtAj3FW zjHTBFbX|XE5dx>fJd!Zmj6iG(p0#fB_qaS9Vi`RjeJCSQ9m4bp|FNH@QO8NbIm;+< zAr!!3)}2kcPBZs{GcjzoGAigr{|qlmkafLvZfw)frXP5iyvsodoa{cocRts&pft|S zPj(B&6-?u63s&a`&rt$o0~xzuXrXLy`b{kmB@kVWblR1*g-Ef>F5JgN?MDY5sK2Py zgG_ZyO|~J(?&fa`NO?MkP)zuIy2y@ks_GD=MddSy)Dw{jWK@Tv=aP9i6_t8P< z#`UxJ7W2hC@Y7IGmgUfgg045Bsl_j7gzf*CBaSd#s1ZlrtKXSqoDq)`Mo#>x1%F8TdEsP1 zK@ALjjV|J*)5m0NQQnKlTZv`^kLWfx6rkHpoADQBdRh*w)_S+B*ua{#c#>ve8ya=% zfSPv`23Ptej~QftEa?`8Jv`{ysG&4C9kGorG~EuhShOh=pXTobqx`esVTfyo4^F$Y zG#^BaERt5IQ2{%UO{-Q79){=$Vl(+sq>393GT1Z!tnD}+F@-y#TI7vCUqA{iC5c3o zKVY)x?=@hJZ=NU|kin@k53^|5YxiEp^==bLABm2&t-CBs=ZLAg%h^11yBNL}bVgaE z<$#C#t&v2_@l9g&hzmzb5^XNY?=O4(1e2LNu`by#J88Ggl3#C# zxC5+7rI^W@<+?EXO=!ttY!0^Mz*_qT9s!G^8;iTDy4tKydHQt?wHtp?#e4lD0hh%9 z^H3$CE$AG(biUQ1@r1M8kB`n_|8El|jkI0d(~O8GMYgraWuLcXXNyaI_C`<|2>1vo zAdNyKS)_ShQmv`+w*jO3gnlMC{CKpfljqP) zkUg5xuY>f|$b*|*YyVK+xZ5Fmwp>Yp(=4Fet9^gI*|%T@rNK_$1e&mKBARzDV&2R?i_AuT!Cno?|$ zTxU(L;GGB=Ey?g3mR|qCZLa<{sYAF_q!8tFWW1oM8RbQZA@DKI^aiH(8vh&nL!^gF z_s7w`Db-kMqR4?ZB$t>=C7lel_?fIx*b5Drn!gDYqwntfcRr}jkuCv}xpo%fwp^7U z3oX7JWyzO1#-_$arccBkvB~+PYE5lyt8#|8qCMTX5dn9#Ew4C!D+QB0J8h)P@F(w_ zX#<6<4k&~y`I?DY{WCmwd~bxp%|L)egGN0rMu3I-|=u zLX`QvHz!vZD1CVOI#kU@rO54yI;x&tC)g~4i?XM_>Qvg-b;Pe|y4&S+>t5KY9sdax zRx#PyIB`3O|GnRv7k*Q0bk?wmgA8Nab>8{IbOE)9q3eAEk3QT=1#Os}*C9UEAxsXCGd%obKhutkIhFf46bmYn$VcP1FL_2pA>8ofBkt1ZAK4 z&MCpe=~N0P{H8AF3KodV+>KQW6_4FKe&!rd)?XRrwk)$zwv?VSQN#`3tI4hRGM78eRSvh8WloKJUE1Q1L042S<<*`MtVL1iXrveKN0*f2e*tf|j==lYFLU03VQ_J zo9mjX1C!;Y)+w7$6d;~@ls>kXLaudx(F!%~D7i(>XPUi<`rDdeU?SCpt#_t-SQd0= z{}&ZIqw@}7-27va_OHCCPQ*9rqql6xpT{qu1;(}t=$$*!9}@g;o;>2O4o?EVgI*LU zrEE<4-PVy(l(@Q9NiIPxkeccfsROw6jKI2&2P{J->dtEBx5h!>xa;^C&tb)3k zqXMQmlj~wj17(@B8~H7%u?0U!_iwlbjofg6K!$z@_w&6M{sF*mR0}B zaKf4!itQ}m$YF$cR#WXy@q>$Msj~S74${OY2e1l<;X&Gg8~`#SL22n=I(S^+Y%GjU zr-#U6T$8%FU}j#kMJl5Jx5jy+{t88*_8Fa^z)?8gIFS-kl*1O}j+)M3C_aF#KSn6) z9)E~7#;|wH1ZYE(bAs17@#P$`QijXf5%-RYM<=KwJX4NoZi{+b*5Be(Q@_meB+GOA zZguYkI8mmn$~hrO8-zgKGJLydAYq;N?BUL1X7Qw2;9kCc)DUVm!O>9hxCDWCqP0yQ z+gDq{FUX3-i~W-77IzK?5=FNWt4uVUNxQT~tEGvpM2jzJdJ207VxJ@qHZU&lxU0_? zf>Jm{=iBO!obcI4bFrEu$Dr+y)9A3uGt}W~ZxZ+qe0I3k zhA_6^Ne#7R4ynS}v7M-CTd6 z(em;u+NK zvw8S;GdWLe#+$8emzo=nRaT&0jg1ysH9G1&+tnKGXLgB-@(5$n=GlZBeO2yW?z6|J zigR?c-N-h~mHMJ4#_YQ-saL>1X0N=|N&G1FXa1S|S=vQ*m#Bb;JizkEh}f;FaT4w| zAhso(cQ0V-SC+v#rs%xEc477DAdg9i98#;yhz02ZDz~y(xSkP;-|>R<%&&|`*$;WV zk?4Z^Z_-5_EE^&f%lv)Z>t2iR_)XbIDz3z)?Ty*r`84lF>q=;UQb}hb`At~`b6**R zVdspEl^mIndD@D{3I~s`Pq#8IY6r8p;@#{#$~9k6z9_%nTC5%I)7vldvzm-<(Kq^GvK_{ zgNJFG-_Jad%-=&2`WRPjRt!Nv>71Vlo@<#%ncW0qZy%izm85h_v1ytM`m9zXv< z8Y$o>Tf}_H<8cnC1=rxwhn&s*jU?g*_@^aR$+Ja|# z1rhGKUzJ?zR~!rCIn*<`vej+ayg!)@$x=zCwM-6svCIpZC`Pmm*c$7T*G zq3YBuz?{Q74(>RQMOe|v20uy_aVkPepPZafI(_R!Kh!o_?v35qbW54L@c;BIEt|T{ z?d3Rj*Vi$CpkS23UV1L-O|NRKm-D2`TiawbS~-U{dO2rY6d29C_r*+5o_2qwC4sBU zMxp!CF0nnytjV{lj&}EgN_a9;Ek^c=%*Xu6t#2IT#JD&Ch~}dfURec)Z`T+=Z|90N zWS8tDuIx<~RA-2}afb7^B%h%Bu!?U%{u8x*LHhaTC4FdM`+|aj0OgIB2Nvr`O(x%` z3zqTNu>3j&*Y4dm`AqGb5I0~a{5X z@X=(yuCV*LX>HM-6^)cb%TvlsY?z6c{>J8=3qlF8$@V^n!u@2>EI?(Lqqb*f)73MGFcRtCLk`%U<;YOIz%N=3md03wsU6=^hc=$?JyRY+v382qIZ&Ek=U>!RLS{*i^>!Wlo z;L5_do9n!_<3hb4uF8*|4(cBPH-R&_u$tWdB{lrm#`3M?7G&JyBBIzUp2KoGARJu| zTsN^2^A{w_3;0F*nj!0ZVQCkrE+?fO?=V{DWK>OtGZq$?;2Np07dG!zj#BcaqQ&GW zc?`eFzio&9Runqq@q6YZ^HL8khVE}R4$Q5h4ZHy!vUwP+bRKf-7^P=RZ`Zn zFJjkM{r8xerk%ageeVM7>}5!$8_bu(`;%G8n8e$s3pbWI@}h+nj9MxOoG+-PjK<6#&%9-G*%j)k z!zT%l8>3WV zz*#4$fb$;xY;K26nS3vo!TkqOdit#`SrNcRp};dQ=gR>BJ(G4(LlaR7czNU*y#A|6Ae_9%!vLHCnM>m&%y|y>S$>0;tkV#n_tZJudY)Ck18lpLjrjB(l|FrJh*J zNq~>DqIzX?SjaEz%nCSGbU?7jkC~Tg(zpyC-vIjab!|^p@J-lci)#nA17axN+J5!s z*?QXrm-1GY=p<2~Vhf|{y%&DMwSDoUd8Mb(7hj;y+c^Hx>zh1C;Dk-LeJu?@WQOW* zkyLj{5P_AYZo3?MJguk;EU8GlFGVCuEicDjM9rA+`?RiD0BlQ)XCG<@>9l7X81^px z!KT>Qdp<=xj9va44_wLr&qXN%gL1&RN=NSMDRKOR6uH)$OZ(z(@nzw#W7RyNWx7A} zGv{Kh6Kp`nAjyM4VK&k}V&4fc-GFVZUzT1#9ctZbpyCu_)(aAEEB~co=OIXB>PH4R zSQ>5`|GFx66u_l(xzQTZVr1Iufs~0&df&3qw$eJ(Lm0=2PXDf0EToGKLT$sx>wyGOJjaU0Vt!8+7wNwbC+a^E1~b zEThhYAd;jEQ_d@KBWe-FkAm3=p__0O*Sr^dn&xE~cAmmPH&q@w#Xjjffk^(?8_0d7 zdT5Zu75&05Kn9%@?P(acHkf~`F-HNjZ$|&M`GROvMO{5O$pGg8H^SW?Inpor$O{j@ zeM;7LpI?bjwc_AJppf5TVKor{>MHi3k%v<60f~)!ch}!8QEinI*YgJD5WtTtNC>7a zGWWu=5Q@$`tXt`1Z>KhU`@)xzqnIZX+hJZpN5N5Sr5vEf`x05LPqzRbtnRB@zeJ6 z6DnT7rPv}!P>dYfyXPBPlJD0g0m`fY-h6+{68}wQjQ;BHLsB5Wufk`-9#x6XRM8s~ z9cvL@f1rL@6Cbalk&XZ06G})a4KTwT^*Jepscg8N^@-YiJm(dsV;1NACBv^P|FkCzSTp_Oh*HP&9JU)ENS|i3O)@I%VLZfc-ON&(KQN-ue1o*-K zc<;i7DPwz*3%S6;lPp;F^nhN{lV$z)q?e4?WZ&H1aL`v#4UG;fP#T-+HZGv(i;33f zN*o(n1;@Dr71pKpqb!;_5d=Xsm3!%K?9}+(-$;4}K1=|aZ`3o{rK`~F z@(nP(o_?W21Nhw!8#5dqaqDI=I4#bU5MFc2{9GBe{5LNiG)%i9fSfQNXn2uf{+1r* z?^|}KRF@${kj{K@-1hc&0MGWDWRddaQ!$W3Y$tyCMgn(l)2v56=<@Zws4suEcp97Q^!lY7RGe zgqr$Q$rB@8$Y4c9*71`z0>RQp&lQS@Pi*EM<(7r^cIxeY2z{N8IV9*G;k3CvtEo%W z*DGkMGv2m7@ zS4jD^%EREO1CLfN;9ID%U!C3q)`ssNptBdpcpAM(w*yVFvcnj{^i3$j7jTp-kYzVDK}3JB?GVmFwt zTCA|QYHz$lgP7m-^)p@>ix481*reb^u0 z9uLt7L0tG-DNPiI#7yKVwEU-CJ%Rkp5{E!jLm?x|O=$ZC4mIL-a=RJ+k@Q%8k{e)_ z&XXtdea$K0JeF&_@=;$hhw{+=^c#nGdztyAzecwVD+X}p%79{!mn|~IziC5UA|rWs z*>t58^-g*g72&AWtt}INLDUNh)Ry7`zf*YP-R^++bG>QsUI_z@9T^FGX}cfapy_gwp6}7EXns?dbY1$Jm@kpf6Oe!!xh~SQDPL8G!;y!*h*!aX8xqsg?_!tlBOG@mfDq?Pc`}T`X{5>a}ukQ5eXkLk08J z5A!6SrAO5DJ1bgp25L^U7v}ht-A)wt`tO8tmGp2mj4QO0gHm+|-Pgl`CU=;)p!60% zGLHvry#1`9Df3fy3} zTjAw;-!C6`huweV@cHURTOhMM&+Ag)@!Hgmsv4`9MF9o3{a1=~B9e&A{dqEPdtxRuR5BFEv`2dZn+1E>7HG6oO=!%dxh5FB5hWO@RijUOU zR$n}%%>HNxS{O$x6s4#GD!s19!%NGyS=Z;dC9HS(nT1s!ygH_O&a9(a)5uxz0S&*e z*=zAZxOvfcb*}txuTZFv_X}Wi7?j>LJlDr8!~%W1mz6WV82ii8p}OI42YZgW_^odt zU65A(a#bT#PQg@q{aO0#)xeg)p{Vgy4FIbg<}otZ6pA_(mAP*)C#LZRgf>nY>-L;gaQRUH|BfT(d^OYrMrjzn1p>OrCV zB-0wuO;=ft{|2W7lTbT}4k5|(*!N2Qm8aApql@m0RL=V#8{K0y9KMlS8b5jROoGUc z=je0`&(twU9L9ciiQW3V$H{}^QW(!1Wf8|z(L157D6J z77s)qWx(BMh?N~l6wY#a!dMVEs{*7#YMUG7-G*}89877bw_Cd4UV>Qo6@a*#tf?`5t~VN{o|K;0zLvqchjg znE^el2(d`BTw6WEuFl19LS-LL*ZJPoh~b%V8^xZvT!;M8aTAqnU7Q!zu{wWuw(g|> z=l(}4P15{XqGMhpN7wxIG_+GRdjmbZauQ7Y!Pb64z*iwWTD!sTK^gt-EPwj<^Cjxq z%bEF8BX0OAQPjnx`&sm3mf`o3DMU764as zNkMHomj8>d_YP}ni@rq_#0H3fNGB@2cN7Q&rAkLYdX4lB(n~}Xq)YEzdXwHkT0-v~ zq=nu?ClGks<2~W7=u=Wx6T`CD%NoV(;)JZ-w^O>XoeM8n;Iw994ir{h2!fzh%gnBtuSinf|0##o^haFYWRQa|9Jt(_mIVT56>h+}_ zBCUd%W{X3ltq3v*rbQ~%q@BFTPv-2FCIgC#pbQ%3`Q~FEyrg0lX#A(Df=^!M=NzdU zmK%{07)p~b&eO3T1Dj-vtXjiftcg> zgx=^yBfhLO4Iu-Qow}tp8PJqhVG?Km)%9U}WR4-X{xL~~yG#*TZ{P5Ro?RaD)ouK>3Mf^&rKcx9 z+WrWS45#2!^D)E^3W(V#5iGQO!`eys6p3o#45b)N`0&Wlu~EsE<50(0>}L3Ev0Z_4 zW0}Y|gi;uHKj)jlao_PPp^@Ctt1;ca@5$7+79 z-~v(dnUkG&HM7W0vClZm2KKx8Tqy`G?UCk~69jQK5ZL9^%Su^v3&u6xy+1t`ctPqz z-&u;*WEd;x^e?R_z)^>USe_TTLLP7t-`1uZJ z4(X5c>ocXYSXwH!@(>5X<$`YR-u#$474_(+*QEZY zV#H3K_F^SxO#BA<)Pi)X^91j|GkpB;?jmAAXi!@2(yg0cgLGs!=jHJ$+(CtSUyy;9 zkigBVpnR%Tki*@Nw;w8L9A(Ui7kqyU*_$g69<<)i>;3f4OYHk?eHrd-n(ljvOH}Mk zn;#NLWBZOCRN6=e%eAdbG8_d$*B0gaLstWr&(i3M5MywH1v}sEH0^K9DgOBo{)KP( za?M}wU8WVPlS`v}%k7IK@0;(MDeJY{emY?jxbhyE5~~=-F2= z?@rB}hjtt~pM-Pgz8&KGI;ynzY&6)h_OOP0N51F@c&S<*i3!OxQ8;wSdAifqnul}Y zOYQcSK=q3SuAPKu6+EWekgYyglXJ&HFM&6dy-g1j>EL0aHeEpvy~--HpJa z^2F)M=%ZclIF;X025px0P2kV2Bg67r?~n5FociSthbwYizKM3$GJQ;3Ge=r%@YRHu zaf>xYSn&$G#CBE&`wrT25-&SGk26qcwuf`9Gkc^rH%~`!(r7*R*EIUJS+qDlv%u{~ ze?luSeLa-OMm3|#6qG|qPpimd0;)FoK(h)ez+oOu2D?RL7{$hOeSarr)+_^x>~Mx# z{PqbE^vgAr-?RomsK;QzdR4CEPKP4-e4WGCdQc&#VnRta-tH;6QpYoC?FCmhe1=!F zO5zu8$|{5HW5p`%eX66%B6F%Q)jtL`8;#^`*i;1D@6UK|T#m`laa)(yzmO-f|A4u3 z>@Rr&FMOQ>Eh(o68x|ooEMbvi*Q|MiH$<#Ts@Sd|-XJ-(o_TrBSXN=qwTJ1v)u)!T z{XXwa#3wQ?kGpUss;~OHP3nzr245*IWSO3aFBHS8tsZhKgq9iZ$(QEd;^nB0osKD8 z(#VoWnd`IbnXENr#3!$%I2=V;abKCLS6+sCOiasD$gLWX4DY-ja(ZkLYW>LdBo)3|;>9-l+=?gEX!?q)ZrcGP_S z{OX4j?>K9zK(ALSrjF;ve<~x;JL|ivtoU=MqQC3F1>jv)t{5vy(K^d0day-#sDMjR z@f0CUyN1u%F%gYreUh^!e!dskZkspNMjvv+W*GfaW+102d1HeXN~D=cP5{9^ zVzlPL4(KDw+m!mP!q^kP5O#DY_Uo&_3)$5us_@Z{Xfz$CvqG9+#P{>anv?BOQAD@e z8g0re3q|Id`HbBom5Y*#xe2(gy3q-3mi-*Q@dM%mSDz25MiU;)R*P>~2L~#X$3ak_B8M8H>>BM^0QMSYnVx8hj z1h>oBRU_q6Pck=#Q*W3cVcE%t*-kkvxW^_3jTz!GZcbfj_n3>AMr!e#g_ypR-F*&> zaO|s_{41nr&XdBmrt?CVXd{_!50dwUJ}0~$!9{^Xlev0IrXEB!JgK8XCqo)ZF$F|( zR{K<9agUxGung+LYVK>}caa5lnT8wNj6?YZL~jIV47ep9$f^Ij$YV?C~l1;J;?q@-wHIqDhqCThD`4rpE9(nsY%)La-~` z&obA06MI+Nb=lC?L|2G`(DerPuQ12~I3XA^s#Kkihy>$#AO1wEadym|ijR|;nft@t zH$;zJ*gzh{E=_08Pu*cGJr-UM2P~&`pO|MFtv87h(9S{vFCp_^MJ`3fV?4|`rqeQZ zMlWj0+}E#}w0y`^U7G?@E+1|AIyzDy<;*H2^zVzEEc0Ej^X>b4#(~;8&Rp^|Hx&$W z7vmR};tE9!c;{tO!8Skg_41zUrOpNJ2@@q73tjbz-4PUQ}>yBb= z>=O;tj8NUi55+0Qf=sPI7CSxnS=A{4Po_jSGg2CEXb&~}1A%Uo*XP3qDZ^w@D~)hv zAHxC;co`0&|5AcnyeD2sS9f3Cju^K+O#kZr-q~T!_1);ePbpQFd+H4?}cY2?b==R zss9&vQa%Q3V63DvHAxR$uiOgw3KcF`Z&o*)i7He#DpsZ~-oTgTzXY>!==BmwlOQ2%Fg!G?WaC;oT( z6>H|dXAHl4%QUf?SiAS@+(oJ`F{plu-YDlq1XuBE`#{=ek(d9zQX>Jsz@ z{lDB*B-IX8>;c}zFgj;Fn$ug0*Q9p>sd_5h!xWQj<9t=jWi*a0)c~k@ar{oT-W0(G z*R{)hz)Q2%`)Y7%?zeR8qG*Ys5$o>i&=J)%IP;u(!1J5X+D!%1JrPs8t7F4eL`Na~ z(|8a2s%lJkcobu$_FS3$xex83$-FJY7jH(E3nsjL+6!|`UTQu2WLY;>Reo^7%=h*B z5Q9UN2o)873YG53)aKWjMjPUy+zJSYK>+n3*D(nM?%a5E^L~3iQ%8RYhj-?=aJ5Yr zf)V;jOQ4DQY{wJL1J)ceXBX)m;^BplT;m>VenxL{T#-CU=5S2Six1wjxKwLlsv z0-&*|3VC5;o0Vj_`FBUI08!8zCwS?c-QoRSj~hL#=yQaqjHn(Apn8b!o?TN{9Jk*H z7c`Xj#dQ5ZxMT=0e*sQq2(gr56SZ$yrFGC1k|Le3Q(gd@|1%>@h6ykJzi`r#dY-Ax+u3)EO+5sTM=vn8p0ovA;&BX`n4I@)YYwAy!Fqm<`SYofb@6-SP7)-i?c|1Xm+cMnzvR%FNzEH0LKm_W zhitDG0}?)04aD8c z>Twb^T*{;Dx099RhKrOF-4JolqBLv`0x$P;9-bWb9a9p}FO#Z)SYcni$uYbvwxC3x`?aKJF6tT@)2I zigEeQiw8XH%1PLE9G&sc=}lb^43lAwCq8lR!00A_vDYtA>^I$N5LLcm^5?sr_n!&a zJUT6Wyz{+p5HrVGx_8@=}|UBv>{{_*EP*AMu^Gwau#JV9ws z)hY*ZS=X||`c!k!Be8C9ldWFuj~!=Y7pLeG6U{p2D;Jh5Ii1n#PF%C_a@}_nxd={E zsKi^>K50RYBs5;N2vAtduO+ncxVlT-_?c+aDdhMT@}xDzFg4WU=XvI(gI;NMG+593 z{xP;@zxz~9h~Y{0FPh&CDaAYcPV?VJcCs<6zRtSdSU!OLsNZ;_iiQir2-?pNa7eOK>|`+3ih<=pcR z>gx%t&yofgtF?Y%mM?2}`|+oI7QvO4FYGS!I%3BMp@*x`#?RzjE=y`31M8U94lI77 zy0HDmo$$v;6LWL}c_){YI*01g5V3r1I^Y~;!sr_q3nKO~v8o6&QdoIy=QE4GL_Mh6x`c4OV430_El zxNYF0vVYQWxNS~B3}N1zX`;OM=n6L`W{X>DiyOSf`Fg4fiC?67OcrF8#2VvE=;q)7 z+vdof32J%JJff%G-_xU9?Lf&g+)WXXGkoWbd%BeSO0v0c6Y8k&XK-YhI}M%)W!(I` zFOi!hZcC*}-1Tw?x>dJgp(e5-kpz5!(1jP66U@=h{UDQ6nMh4=zC0L8m~Fbm2;a}o zOMahQm#O7kzGfY!c~^jg1y#yJxmCg=Z0zpx>#tKl=-svScId?A%!yX%W;A`}!o6`h ziPd!Xr2V5hA1ek!p}Uanx^bkS)P)*|J{nsXwD3(CSQe_mw)OUTxCTUNP1c5ZvDZ5f zb%&Lc9SVEmeTMZjw@pi-(#P9FBseK8zI&2iH$J{*&5QfE>7kXLN(Kx2nxf_k6%WMp zzwxT_^UxG}?VzZsH1sU(o{Ea)!^eK57dg!e3VAqe|BS<|xpqx6fhSK&Y39x~#XWOy ztcA%bXE)E~9f!J=+Q4^mJzq&2bI`GM8bv%vd(dUqWj@n34*-PDR*y|ubSVv&YNk-d zEd+(-svGuTdppdK3M=b_e{;G;=uL;tlR&BWIx$%0m>up730DIbwzkC*8+!`TcG;89 zk1sIE<>vhG6HyJIEHRT7!HfFr5i+xSoHILOe5{-nwnf(wIqC8amtFinvR7-rQ`QL$DdN`OXk^Tpo6r8zv>}e!xndfZN*v zDaj6xJi{DbS$QpY-!(p>QRl;^*fg#;ZiK~c$np+62#@)Mc#}g_lNm}$Z`|uBTv)T~ zlP@cQ!=mqj?>UZ(z0<-Cq>XbBsh54u9jvK`@U@=^gM zfau21j-!FG4cvQotY%Gri&AGk33q+5aikxTjGgl&ZN&mo=HM9ru5^XCDl3fS&$1;N z&Q#j5Uw*57vcfQtgm}{F!%*)5lq;OzQqVUCsLgrB^TQ`9*lZ;+1K4t^6+_G!ny zEbWu9)KG=rcQZd6a^p#ODX$K3STvioYDr1*CG&;C=ZAN{0f?Ee03l z^TWd4Cr1c#UCR%>qC=tx1#r+^sC815{o~Ma7-%W2;6D2!*pq-KePnV-Pr;SKR=%97 zw?i7<)sW_Tj{!z+PmE^g(oV0}s<~9yI%r}G>s?z8+^OHa%MQkkW^s4#w6qrHB2p;E z$`Bx;3{+a~aswyHp9sFs0sG%0m0N~aAP0}n;~v>7X7{1AB7Fw|8PRP7pDio0u+Owz z&*Ul9mlK;*zb>lve>N)nV;=h-q#6$AEq|H|R6C&v z#mj^(r#bltS+D(IOWF7OPXr}ve)xE5&$$6LTEtRROS$BxTuRaE#hvfWI4>^nu#v`e z|E#HS!W=0BGOk`V);o%l{h^_gMjL+TGTxDi0kbQOG;a3e0ktQ71V3*UpP#_bO}p%d z$AmL>zbRO?d@Yz;GmqdL&~-XgnSxbbgY#@bvrVq0%_%!W8RBB?aP#?0oWviRr!|y6 zyxxHt%Jwf((2J>^pbWROds9^gO>q4x|A#de&21+f3U6Qfc;dm*j(Vbn==NE25C@IC zI8B>nb!M{nLb>u33F<&i%zR zvswd%E|6|vrugkb6e@c>j|5KdO;JHqR|HN=USRl&QS_poy+HR_^kw|Jw3TdIn}d-4 zF`96ssU^DbW%(<%f_sZaVd{4e4?w|Zi!9DIW)*9#ybr=+URH72Zu^`;*9RKLb?C-* zSgo*s>Kl%=xR;QQE`R=qt6h(GkQu~`^SvwSdqG33FfRc#h5 z=5Q|5`eJ3pz)8O#WAi7n6Vo#<^}ceQ7tN^9KMmzhAQj6MM0akAFT=&sg3@Ue^Q<|N zPrwLS7Sbp9evo=q>6sk1C?8Fr9n>E9=@ZhUYFqEw@Sv1+ff5J@sO&V_QK;{nhiR>y zps8}b`tUdY7Ik#N39AI9iW5VGwPPv-A|bh~3&X3Y*)IBAh}v05N;jJ{+L~A=3W@i5 z5Q`}%rRag(r%UfNdwO%b$iSrJ6*~)yOg(>O`UQ)?X;Qd)DyNlO8D&_?N?Q41oYzp$ zi~NO4+m3KE|F9~^n-L@^^2yDvD3K zVlFX-tb-t0kK)8OwSpBhpc8jH&KF6CUFnu7rFKZw9Q`)+5}8*S(P)QYB$3#RTA&P{ zE2P;eq=|-D_q>2*^r8WncrJ}OhAh&Iia82=hETDfbzc|=OB2#uW5wxPYHv7LXd%nO zpL6jVl--vu%*H3qpygQQ4LpM?wD;at&ZR-3!kmQax000rF23w6him zOqdR|S`nggSj1@(xqkhM#fENw*1;hpZ^iqogp{!ipii_Wp5~qna-JN2zQ15Q5fRNT z1K0RQ-!*nS-pLkpwM0;3UU5P%&Sm$Tk>~~epr!=Zjs<^fZ_YEWgK5`qEg`U?GhgH8k?`=b;>(6=>J@Q z%!j_nb)O_zyzYwB|G7n^L@lt$Rlw@oN(*7j533CL@M|6k8MtzB>~_V|my(DJSj(v$ z7P0)tCRfer*F27*bc2NdIje3u9gDlc^N0wtNTo4Zm=bWpzhIO!kW+dK1Bu9AGm5IP zVnhO;yGwoN0TL0a7=g&M-B+dt-$g>McE5X7(Jtyfj`6ya-j};VI|l6pCW?wbl~a3E zV{h+RMe9ft3)_>?V8ND435?905#~K-P<6+g@Z07bWGR)?2~@Rz++nL{L#`p7RffEM zu!-luWMm2{=#v>q(Ef;9BUAib&7}(fdBzCAm%vngLr?qOSVr`r1d#hOVKm(-xOx3q z6CD_+{xT>2_#b=nyqDzPaap=GgvIJ=BY`W^z4ZqTelz#523+-6?PZck0c+}vy%?w3 zXloR=oTzcSTUm(KAz|!@G=P91g6QETaqH=quj&PJIrZJV_=rF{w&}o~2&ZO}eGQ!V zzV~J+Fw_cdHe{5Sn(`mBqW=)w!?S>-t(lsXDNuG^L*@gW|B9B}pIHlZ0elBry3zb1T)F7(yt=nppGJ%KxN%bP-N+KRp*z;I82$Fe5i>{ z?I3~}`km!$-{9s6hlXA8kM|#Ym4|mkt49@PRDT+8R4_kiAyg2Ruq0-v3=87>D;4l`dFYCk0h|r-0I4_3#>T4n zZ%+D0jpF+_`Qle1&ah=BoL+%QvxYC7F`DLROst4e7ko+u?E=|Ov7c~7?WEBRDI@D^ z;Gh~GpB>UTX>OkZY|avh)=SWCX|4Kd7VTP5YG~WfpoFc^9!8)=jHnOzZ%tYJH+(MD z{G#2@SuQLQt&DX5B7eKBuP!(@v__0N)#ulqSEaaeq2fhj1KiJ72I8yDu>w`NF%QAJn24a$s!&%p7J|1=X2_D5jxHB8 zKQD>E-O+yB&ejK43KK>$ZKktavd@}Bmm%Iq;cutBsATlupFOq#P$t86xh{)MxtR0+5iszCoK*_I^|$+o3gx%%YZtW4_3SgR#EpXQlc%R)IL-ZYu8?7|)Z zWvHD$Pc;{kOb3KBOk|sZIe_sYtO$m@dU|!Gd67lc1Fs7mS@*GsDh88i&`cp?|3TLZiTn)89uW97P%+e93+9;NhGhfyOFdLj;3d{&? zoujlKkOrSz4E5ySxbx3cT;x&ail_k!G)UPu)EJIegc}765)`j{c zY%FGNxipm3d#lIVbWK`eJ@sMcM_nhQXm|G_vo#gxDyN|(%DCISFVl5JCaw23K6kws zA~pV2xokI#8xG;YiVH~!57HS&5_;Hnj_=-r*Hvlgu(C!+jou-8GN$NUhMl~txhA5qduUfZ-U3UwK6NNU_+0zt@hpub z-rN_{;0u-sA#V7mul=vG162!oUiWfPxeJ(+L+@%Da&vN#G}y@vz)UOKPCGpBfqg1G z?!1Co5q&0co%HVa3z9)zRd?`@C<2x8K{{5TYv| zvRbXAcW*M3O9={IpMg29T_Cj&T(4|J_tZ#DLlc^?;vfgqpn#zXJ54(=@oMTI^zt&U z@Din@qp9*g#Nvgf8U*X{&42CsTtjYWSWZ(6wr

    ;*@b0AzJfmL~Or|0E$3T@AVNUTXaw{znl zOvp*1h5G8Um5;i?dJPQJL1Wgr-`qGJ% z;J`&{D6h_a%JXRyg|gHq5g&hQpkw9Vn(X9QnFdxxG>-f3b~C zr8_s6eqHBqbH=OyQAQNwmGi2I6KDwr;!5ClB6kXAflpq}|F~yTr9XA<-IV#^-cMkC zIs*{`NBd1pkErr`AuII9q8UAr=qw9*XzW$QX@;ezpnu`=i?J&A8ePiwy5JESGOs`L zV_<`yvJKSu1Hq#K;v;36xA#_I22=6_7J3+|Prm8k7il!pn!`TmYx*b$N2Ze{Pi+rteQM+Uz5hL=^)> zKi!EZsxJVWV?rh*>3N?8mq*44og@H_SIwdbuf`4rJJ%Sm-A=n8$Kx&Hqvs!D$LE=t zBN9KhdIvL@$2f@OK!a=N<6Cn9Jwi!>Dr4|h8RwzDVjXd=;f zJX+DwR}mDPs&hTqo@Ji<`f5Q@`|{lHHhbgg(|Xq8kJYJv&HrvFI}_VwcmrGwlqid` z(w}^hGzADz@r`)-?1s0WvW?VO48S|1QYDv9Ovyi+{NMn_E%i)V#{K1*{sQjD2(4l2 zLFmW0D_X8Fz%tT?c~HE~`Be81#$WjR|Cv31OYPk}q%h3jEnry$X&#prMKVDmUCeOY=-HC5q^C0TM1S) zGg{PiKFyeVqr`PHb6Q)a>+OjcX#ebh*ju{U(IOLZM44UBg4WWavr)WknMh9=DJ>#X9>kgn*&tIPUBX4!vNiE{rL({Q>)BV>t zV@c~QZMy>2d1CGtUh*0z+0Ddji_JPEQo%oXi&(sB-`n}Ms zDau{mk``D#ETH1+w+ng#p&o86Ox+1Jpb?K$BDd{PboDgtIj&Xwzf(UT+eRk`ywPQi%2hJm%z`;VDzZl?G&>-mv?t z1@H&B&1D3$+?Zt*wb8Z?93`ZsvwF-011szzq+&zgTCIlF=b-i8wb`W(W!d3{JDJW0 z`yR8f*00FaYFh$c?z9bjXT%X@nde!6s1(-xY4Q1bZ!h4Ff|u8wZ_st${{ z7Ex3i;ucJ}+V+YumNOA~85}fCHlHO@;{8F8OwR|FL>kuGbMJ29QjbGt_`0#($gj=& z02C@pt&{7-#`_FrMGs^1mKAX+VIf+jZOLbL%bf__j{$}?e>i2uXHjsd?Dv^m@)_VV z%fk0IJQssZr6KFK692W7N9^so^k^>;TwWe3z%I1-TUr6YIy_CFDj3AA|OnP2-u zd-GMF?zQjINi0+#Q;LzW=K){}eOuy6bB#}hh~J^Vb#8nGi8q=)jmdJaEcYHrtG|lk zzSlVX=3Dq}ACAP88nM@nvu?vLZ-*yI6;>5M)!h zmo;9|xp#0&)?piJRX|{>vsT@

    @Nj5t-jS!mTPEY+Sc{*Rb{5BiSq}*M1$FtXd2- z`gU_0P|P*+iPMZtmD9DRW4SmkvNDhZF6JQO)qB#G?pr2-PDH8PIxJ3X?mGMCyK*2p zkf77WSP`SS~LO}dqDWkGJztK2}fH62}6uv ztXF(V)HoY`)qZgC6`~@^ij?h4 zX@Lv*+)7y0J8{ujL`$!`=d8wG>=Az@0bOZLsPi$NL!L%LL6aZD9&&}4Zf&p z`^Nw7|4n_LbaWd!`KhO*Egkw9UA#u5p=BZG=&3QmlAvJ;IA8HA5lU~PV0e0HcU*UZ zp!>v|@#iMVsyUk>h6q@1s7o${J4SF@-~gdD%jLaDRFhu`wk zn^I%JN%^R74Bta37p&&X;D!7VnyPnT)bebioEa@3SW($;F-{uHKz4kHxWT%R5a1;! zu%6fwj?d!!QTXfX@Z>p4uH%VU)$&Whz(Yu<`2#0?>PxjL>B8`|e{P}OK zO%c$-pxJFvHx-Ke*?C%2)iPW|73}F%Q(K}FY~|@=X?2YZ8?7H3oV=~V@vX**sBWL6 zO^saaCf7+?J?k4oDIVQ?h?)5m8AUjYc9~6s$1R_)6BP3;PO>*u?^2m{?TZ3YAh8Q? zNV|N!xkA}brEfmEr>yY1g1dTO9dTO9|~?OGJ3UA@6>2)15py_xmqk0&;Gn^7w^O?lqHFDCx} zf;W^=)!P&N|9kwWw+0_>Z&1`E``~;xY;`$^h>`R|uvFwsE3cxFfv3bZbAioey1k5=cRv>4n&nIDqWv2z~J=VY zsC&7SwhV~rHY3+wmOvh*t*2NNZ}$qufeGS2ul?9?()(|O3y6ip+}8du9Prdmz8z~( zG;=KSxc#Xbn=0Dj)De|;?*RVOeS)@b1$h9-n-y<670FlmffIuv91Gw$d1rJZ)>l9s z!cTtEI}B>bf|NsC*6PBII;N#hS}--MXq;PQW7z9~p3H)ea<0!QO&7O!P0t&;O?adu%Qe(NySC@I$_iKfibG$|v zS)LBmC@_=7TFe(lt3*WUVC`e1=%?IxCX^(tw$c(6aFm*HZa9bsug9LG0UU-hx&sa$ z*}ycrq{$R!0sfRUsK^cT8&xIekAFF6e?(IM5lT({@agH~asK9VDCvkiv&e%+9o=nO ze+71FB>>B>iUcM0e3TRYzbT@zm2*ORxN(=>?eoH zZ~EqiMox<-oETc4n^adB5^v5rWeAMzp#iIV(rd`obH(>QD{E`UgTmV?+O{PeqranU zM{#)bl^RhCRAqAr?oDr?^!Wg1uwG&_OGh93hE5UQ>HiLpXTaxdE_`r)SA+~POCW>$ zao^wzIVFoSD%_PjN44V1!mJqQBAw{lZh}|~_OO`z_ah=6?e!AQaVCQf+$Pk;MFMPQ zmS+|E0?J{&7P@Q>b-}@cs-A#gkCANRR&pA?1AkbnP3-%s(p32`-f1ohHqxhN0XV^F9<1Ui~DC%MaaW^^m`EtCn2$ z$vNF2ReclX1*SpttI?h-az9n2T+ld$+rK^$HH_YB(P*5%Wp#HEZN;UEY_aIlt{#Wk)9*5acusRR1-{M1JQ#Y4S76Mkhx zj&cAr9{3c~xK#P!Mv2V&0LcDG^$_ST_~$T0eaCTUYuQ9dG`w96E!NmUE2{1^_`lT` zzsHa-;yecWWpLVqSpc-l8$eI?;siRP9*uH&R|jYEcSb>hmN}G6xc4)C0Cqv=m2s$` zzyJ9bO43x!oJs}2T(mQQ1W)g`II_rGQOhE^Xwe_-eDVvLFn~q?$G=~2RX8HqD^lnN}`j@DQF-FLaN!cmIfLcA)(}P`A@)# zS@YX_btN`x59B7Z9x9gr)}ewOFA1dwZlNB zR(&e{bc%&o=WYr>}Z_7U- zNJ~E8C;3?Gjsad22q) zOE)B%EBZQGaemaRrCWdF1^CBl*+jS`?Nt=W{Yh+ktCSGo=xhnLG$F&SYIcU3gBV+WhFATosLkTI@kCq?}%+Z_E6GG zZEqPXusQU8-N~!~!MWq@|H}zgKF%UwcT32rIIn7t!4(1k(2i*E*{qte%=`q>JD(PS z)s!Sc!-VV7!9$F*>yY^M`d*c=QNdI7i%#534ia<9428+pGo>b5zNcd&q0&GxDEYnj z&zH(v!DO?p_cB=FV3Ia^VEov~L=;3HyF) zjZwNws(`k`GSJHLIUpipLfO?$#WL zMk5b)VXh&?t;SS2Uq~l4uXftoO13ktg!+3oqLlpH|`d3qC6FlNR_Q$4Bis zs}vNI7&zZqdbCM>&j@H)>7I38#r$Ic3B6%?({|K<$A7|uZR@bB$g%NjAU3Md8coy_ z#o6tI*UhNkN1m|1fMzp74qM4^#cn&?h1Jh-Uxqca8 zj~O)zw~v~iUXxy!{U$D7OY}1eB^q4j6zDI1p*j?N{|O<}3uTG)Z(@YL0;IQf&jD?W z$BGAUdi1o>1TLKc!39UXnx=3K$dGn2FJvl~psMGjs*FHmb=XozS_r$}Q0%Jxo=Mw* zIB(NKi{}H^WbY>wYxA>s=PZX?*N*Y$O(11vXA7{F#=@6MW!2BIslFeU;rCOYQF#>1 z?ZFg|Mj{}wq-O3n*cr;44ln145+L50VE66))eT>r9`1#f9y$D&UK(mzgLwzf$oc}K00kQ;^2Ba`Kg+!ca#5WCKvM~Lo8tSaQ#N~33XGK~ehX`$?85gDP+Q{lMUsn^QV-!uvdc)s4)u-<6ZDO?i9v(10j|7*GzSdw>w09)p5 z-(SSyZ+tw!BhVO+L%?axOxYWuC_{c&rDjlkNEP?pg%Exk5&m-ml^4e~vGC5js%Mno z&5R`3th8vGHX=*EO%o<}qLO-Ag>yg2;$=oB{Z#r7kPvI&+M6vb3Gu$SNQe7SDuhv% z_1ET;fF8nntJ4Xcv)(AES?{TEjDT+&IG{f9_?c>&&HMZ&v|Z`G-t%1D`Wjuvv)<>V z^9;l=r=M)6K?AcexjKhDo5`;Q_^@sEmSB=&yqlTQ)ksh>OWKxRvjx#i{Gt-fpnC3- zqK2Go!+t|&%SxV#gA{Y5aX5~tmPn6cqes6u4VQ*&?s-s_wEUc&vJ0!fW!S((&aKLXY9y0KAHe7xP$Ke!<$*lYc1^63pQ$!B-3ooCz^!5{~DX=#m`uY zPoyJX?1ej!TnBFwN#8Hhvq$4Qe{m}+J-1i!s+GC4SJ7fTk$df30e>)Bv{E1^medZ(E zv;jJOp-fdN2uiBVatfa+Ome2J{w-O(VB|V&Rr>owKnCU8b#r!FD|+i_GuyhcE<_5K zt_ST+YJS^|*sFg15v=Mv*&J^^etBhbds;e?F!N;(kpO}+a02t(+lbBBZ;97^S(J*G z;mdR25wi8K^s}#jyufl8!HNV}{6_TB-SEohsnqz~k!Ht7VRGN*A9JoxR8&t;IRn;3 z$Qe8cWqlt-<;2$ZgH|kY8#(pKBDC6c^I6Jcp*P?O^6-25BUGFi2ENd;#Fcm`Ap`a8 z5nT|%7XK!R^4{8$B-7XL9#;Nw8vou5O*lrAOS ztHX(u6?LLv`0*1gyR7S(wr`I8-2`1xOE1D)O}rAU-UORd!TFcto!^+JqCFIyxHw(L z#N%^I>+rXB!-d{TyzHN^Kjd*g)w!GzKR}`y9tkSuD7m&&c9(TUnL;eEU4-Rw6oi8V z&|>8ljv=r)tw7B{m6y*buING$&+p7%s3JBr5w{d9hWvz(bBrZiCVESnRA~>B7$7eU zT1ceZSI5L_)X63|>R6s1N!$;698SNCDnzvidTU*TBZ}9ND}^N#PK$FTKNh}Ni1JO6 zB7~TU)3ifI=iJJ`@E7>ikMBtYUxrTgQ6lb2^j0t`s4u}z!J$VL0_$n~iSNbJNe20P zgXCXx{_AM}*lb4kj=e)auJxN&t+-~tDmA*I*7!K{ZtZ?m`J^F8fU|9>Z z();-*cCohSGMU)Z%A%*UX2>Vk8q{_jrHNN&oh|Ictd=;=M2(BM?ju*m)a6pG;@T7t zX9v84?`!8;lb@cKxDr@@{G@xs@1=fdA$T*B8s@*VuB$mGllYqg^TfMh3ZBOI$q#>Y zk7|Sw(2R_n=LaJ{1@Iw2XsTt4nXM@Q{ui@cZ9wro+%n>*O|AumjnpC314&C~cK-!)8m7_o<81BNFew zuJzA0WoQ$4i}TgwQ&=IYVrN;K(}$c^S9}V`)&8|XsOaGV#h0OtVY3JNAKo#1`M(%@ zudt@R@M};-1*MA=DG5bDMXK}`1pyT>RyssFNbfZSrHc@HuOdb1Ql&+DC-l&J4=tf4 zgkCmIS`%5cVu}3Q-3nHq4x91EXp!9OcQFX1@&6 zr@D6`@dNZ}j_TP^+t?a9wu{!_oHm0vCfX)Bn~sjO(`80qA5Jc>Yr>fMd{<+)=b?Kr z`j!rN3bxq#2#Ki^QHB6yac|`;E;m4c5#6WNihkc8`W6F8##|Btv8lbLtLKdtYaZvL zn^b?7(2YXwa+Iou$3V}zn$R@Uc?Mpm{v_e5v`|ovOtR+bRh@Q7ns+qNOg6+9^9}QP z0t}+EjlY+SRaaK|u6qtyL(KS=)hM^$u_u-=xR&u|#B&Gp0MaYnwB z>Bs&>1E7ex!VZoto&P*c-NyUwY?HRYdxqu^pGqUs_jK8lH9Ug-N!BO{990x(N^eR2 zicKl0#dTVZZ!Pd>KRFiEs3u^7QQO{!K1Dvcu*t4xm3}5^F%( zWeS_kiu@P)y=9J`6&~@jBYOJ=7@z$jDbJ*bjLeij_n? zk^l84s7dpMid{USCN52@tBO7wnG9G?g~^E|_?UlRWK<{bl*R9FAgcRsRRj+>c}+~1($up zdieRgHF=^aQf#Am94XFKi1w0vlM^PPNXK(=<)t8PcSjZeAw{?jEn-y*R=5}}uA3ha zy)A1DTZZYhu<5q-c_*PX=YX4hiYBG$R_;_WqJ#BmcQd|KoQ4OXIwnA;>Bt9N&r8;A<+_0mcb{A$l9we+!wG zk+i;j!-M!99geeDI^s~Z=iVHjmdlrakx>$@QoC1BwEJd7_4O7lZ$Tmg>**Rfd6lB% z3mQ#?R2M4BOB7sn)~XCola1>D z+#EDprbpRs>~21Tj-6rDF#WcECQX^ib7qGG{@^SIIFf#gkG9FP`C1wXVu)BF-$)7{ z+euB5KZG^?f^lAPa877bR_W)b`fYlUtmYa#q{g&3l@a67qR>+>2KfKPqV?-pC+~_m zUlPBBdk6j6ZZoCk8LL1qW4g!75u|vgvP{72S+>ik1S>c(-QKOk$5-}4ng53CG?e?O z#tH|9p}Aa@C9MTrdvI5jHxgo?`VF5G;W!7a3+|1Q@7<9&yFD+1szvilS#>`0PpOGV zz@u5c5YkJ%Rqt-h?;6nCj4vH{m+$Iq@Z?;9#HK!(!B3L3kOuc(jxka_MY_sytldKu z*6|+0w3U0(z?^aq%c>JU%?rBv0cQ@rLvBx&h8IuheU!FL7u0ThP`m?u=8_-59t-73 zJmdofve7+zA*2tvzdUu^3e0{!eEjdq!sOfWbETJg`Ewzp@SskO)q#wq6z@A=pbT=~ zER1~wb_=w!Z9X*hXAxT_|BXP2!HRobt@hIW`7{!L#x&X4uWb?)S>A3&yg00|Cm5{} zp14*Laa$$-@Z}+sg&M5e^UYN>3psT(vz`=CE0inFrpw=Fb$%P$)v=8o-}-erV-;`m zXE8eH82+aDhF7L()atx-#8o?nHmN4LQp_Eecj3~1bQ7@5mw4%wZ{@x8# zz~DAV{n0Op?Rh@S)*tz!f4x|%6z8+z-n%D?O+r>m=rn=le{>iqqdQZUrpg>9QkXHI zag_B|C3ey<>}8k;^w?;O=dY)yqPJI60Mpl3Kt)aMz(2v2%J>-rBFbX!vjyF1tfe|< zdJ4UK57QY&mNTqLzcN%y^IXn~<)>rYW93nEzn6l+0Bj6PyYpi#3$d_VD%@Ceg}GZ< zxxlfl9;6(7#C)bo@3YdC3sb;a1MVXB!BK(c{>iX z9l8jd8W-IF{m2z!)Je9QtTMYClj`g&`t3c@ieGHGl>f8*&*W6z8P$p%TGPkwa=f<~ z`1$vg{m)x4jsW23_q}XD>&ndYUqOLY&SxeKw=>fBA{1Z^)J2jaTijp+kj=8}nkDDh znqgbECB^o*_6uK@?|V_W(OvK=kMLx&0B}Y}2-MyCbzD28Q^QwLc{|wlvVi=1Nx&d{u6v&iwyZbV| z%^gbjsHSMM_$N5-ErHsRKLXy8CK9!qh4yCUbf0-S0h6h}7Jj_Q&Wh9im(%<8Z}k*2 zu~CnkZt}-DGfs2|nD}-8CZohg4Khq9V%wm}sWrQ7e>FRR|NQp$!jF@J(o&N5!e&eq zzAq-~V!KLZv+sSmcAQK6W)EpzUR}BFvcYv}h+~`nb~RWUc?~(M$eyL(6X1E2CvdHf}mq9V?+#FYjJUL+|Ca(wmv_Bofj4nhSyV?|<-@E|brMJL&7gqoe-0jF_DPC|bBz)^6Tk353RP=AFQwzP zkKa@O}R)xS55V1c%#wu=g+Bt;S@jCPvYSW5f(YMF}K%Bl#m8ZkpZ)J5Vs~%KKf0;`p?^dE7Yz}gc@I3;eRK1(t$qdReNDp;E#SCQz z7UFTV+J7blF4u`>zrIhtg(#NgdlB8+hqc!~gtmqDq%r1-1 zPGFkNNA`zr6?&M;1GtebxLhNbGjBOFc$CGN1Io}`dcYowKLN^ah#Q8-C*63DT)gVm zq?{u(4k`8@0HWa+k&$*8L=P#E{5l$I5voQ5n z!_F4~GXXgiK#!whju}Fs6I=9_)TB$GP!K3ZOd$OIM~u-Rj{fN0ijd_PO=oD7#J|o+ z*8V3>T8VyH7jHy+P>gfNwf)jT12=_UH7<>8o%e=6{A=VA@IY+-mMMn?&7;`b5fdzIg`o z{ro$XN}?1@ka6;3S!*~D1#PSGxI7rH4BAEXf11{U-CZgx?CR$N!*UP}a^UDONF;jv zSDr)Ga#>aFT-)E)Bc~TtzQefZ>`;Eu3)L4{455Va8!zNWRqm}c#jG*YyT{%H^hph=b^dyJp$|-MYWM~Oya6wY`^f``jo*HvK>c?;dX)>VItTH(Q%~m zziO8LZi3Hc=gk^JNL>7bubd@H12=L#j(qurc^HL5>5DY0LRJQ0X?95!imFI4n zZ$3iX;C!S3h{MIwy>KZrrrLAaUB00PvQcoF11nAZOROc|M=M=UhhYUhMJVN1fU!bf z#07ogGkn#VX=Eh-Lc%qt3K6~^DV=mbkgGIFn4k>~be3&+ckwW#99P~?9`@{SN?Kf? zrC!a~_TXhizo-m9W1ko+Gwu=gU`MJ5=PYvfTh2;458!G)9?mGEk5i)e^n*_3U ztu>dHlML<0!9(W+I@kW9y7hH0E6w(mTdvqI*`vVK?N5*iEly)qhO($&47mt0&;_Q( z_J-5Qf?3Lk|3fh-R-2{B70SF?6Z5o@%A7P1FaU!oM_dSniV`4lxN<6YC{vq!@9)g| z+9gIGRr2v+5(ZsPNe~oHYImu8hTia~e7478Sgc_fIQBa~z}Bh`lK1w&hNqOhUk*++ zW|4G3dE@=rsHJAU?Y2a!6f8ngS0kk> zUd?Vo`w4sC;2(a?q%BDKV)AD9KuiF>KShvB_5-Y}KJr3rMMnNpg4_X5bZ*eqRu@99 zNbbFNIQ8lKDFwWKOeMUDm~*x^$kS`3OY}Bd@V%Op>#VshF!E|>$Cb+xWgl`AiEA+( zQ!#yYyd+{k7OsAWkDK%l;*1Rg$?B z@euv8C{ot66kDk@uBsQ+qNm}lr4(!m`N~QZ%qB}@J*xtbRb$t$@AWIS#)~6RI*V;QTnFa0=%QFbN*Be@H znPyhhesW=dZSX+VWYn2_8n#+{SEnAa^ajIO`hMC*VKD=05_QkK`o8Hr|HQ9;1zvwJ zo0n%lg{ZP`aMYbPd*ih|`m1^3L3UjvJFOkdM#ZFRh9vgdpKM7^CToPaWm8RG!9Chl zT?r0$_O(`$)+B9J>Yo;3iIVg{gU7^Xa_+CH&$&Uel4iH$S>9>dS-56J2B{f86U)h# zPE`hleo+kQ^MxuFT15dVIacG6q)nD#v7{fwNSqToP=vu~o2t?(*iLr=A3q)kDc4G} zOA&sWFTTs*Zxke_(*x%8pTTx*8g$2NR}yOmPY~u|p4gDfx}?Ue0F8w-Sz||U@2?rs zX5qx&1sC50u>I65Lf@qiSthMy3rC&FR;r#2i0wbdfE_xoj=)GM9WLkZB_2au5J?Agc+k7`dac>4H; zSm~E!#a+rM`6LnJ&$7zxqLYR_3*NA@COO+&e@VMr)A`9n7l*?ffG-m{>3HUEv-_lm z;!>=pnh@{sD3zp?aJ-s<7d1O;|FImc1k^LAN!q2Qd8RI1AiW%=ru?KnjK&1fI3Z4o ziL`q+E@js^i7%YbpbY+5*s3!vnP)~5{_ z3TKI!EIMiu<06@_fZC?sJppPx)k^=6QKVCXR%jALJNd|fEOm{&js_?{XAyRc!@RfZF9o*E4b)-{-lhe~(o+H-LO*d*{U8^y|^+QGR+F-?hq+q$4;as3o%08?4zC z$m#JGWMdk+rwVkS#z-0%kjcuDY+ow7bx~Y{6%MXfo6oWl^*8v^R@OF`#^+_bxXZul z0KUmXohap*n>FIFg*P$zG(M_ipP`pJoTY#B#O?cMQz^U+hkF;nXSL%r2sEydAvOsl z&umUJomp>3;&v`&kAo_|o9?PU2x_U;6SXz?4Yx_*L&7ups9Cn33{*d^o2r1@D|iPg zT~ve5h92CeQaI^=UUr%5e9ztFSLPV-etW8Dd^KtVZ10vutVgH-JjXUZZH=b>F;dO) z|51F1P;@31@n%b{0sjDzMh^~Nr}CF06pXvj^URDOkPJ-wpXuM;lfrti!`du#U zg=G}aLz@%>l8(^}CZ~%&=CiWx8~TO~vyU@o6-iyguYb21KA>=2@Vyt#&VJOry1orR z`2vg*n>-L<`?p-Ci#_f>eJWw^gLQ#DpjNCE1&AiWapuq^4Ze~!uYmGj#BNLxYE>3F zaPKJ5O6KALb(W-J6bsphE3fU*&=W%F@H`$*F7CZE#QwMrG)bS{ep8`GRo;ONg|--$ zmEA&-|AEz?as#5&1dLjT!_Rz(>Mg~9pi$-KnQf_+5`vC!J{PSpaAh&Q;VA&>kTj5i z_;A2&C+g=|7rWaqh#$>McTB*!MShf1j{L*OueA{>Xv1aTaPy={UBF{+;$8D+OUSPP zO-UN|LD1NZKiTjK+WNZKf;$Z@F%bP7=uA{hO&+XcDk-1gB4C4XU0%9&NqAQ+q|OgR_}twRezPeve3XquRY zqaEP%ebGY$gGjRb&qkBd_$iif6!Fp^9o>Vua1bW5=d~~G8vAK7SXj8emMr_`(%B4B z0OZ*tipK8xfGG}K%c;d|`CXFVyZ;=~3gikNrcfNV_9xQDQ?=lFJ6_r_#F1IdaycQa z(~WoogxV#V=IAJDESll%Q4NKpR6r*IyrNqIBNE<}kWxg-5(o3GY$j`>-N(x<0Og>W zqs>N0Y_~c%Sy*OYC`e{&@5U_4l1~@|>Y?ZB(@i^*NaxGSfK2V>8y~~F7j(DUaSQOP zCM)BNsxxtx5XFUWhpHM$#5NcVx`6~GmCN_QGR}NKbZCh(7eKdiu*6o?)CDeRwH{bq z(MjM86C-0q)%ag4uyeB->sQrDjR8 zfD_*VLg?wocsnD7sxQEF%l>1dlkxPbwZD~aQU|<^_D^1%E|{7_LqCMg;WpiGu(G~^ zw~vag%gUFE#^gS`MT6&s?#}=p&0VSQc*5cp-p1_F^DKoi8*R6)3muP5(fwjEYgbY| zNSA<$rbuN!E*&JAAMH*x*Tvb+ipE@IXBT;84kpqUfD=`{_C*aHr`n~5+Kd<}EzIYa z>eFy4f7xoi$7s}-b4gQ`#Lc2JivZPl&LWT{;u$Hc<%h?MsX=+3eQ4#NaA&-BSrJHD z7NtMVe{H>+f{5t`_D$6-#=SfMC-eff;_i)sV)d)6`r2+#Ta5frHn~C%3CV-6;A7 zIjLmicgH%;Uj@&2CzReC6;#>z`FK0{CzH-Pcp23GaUS=zSLfl6;L=P|M}_?JB_Vd6 z&oL)f?#<2}bC0Bpj58i9hu?}+3%a`!u0Qo?CDlzBY#cnGrW4Rs8CXsQTE8N+TRKK) zpR65kYZ9ybPZu#vpe!^{av_;lUbo~?@HY((mKT7&OUgJ2)0V#N?&E_?tx z*Hsm*+R*E{og#&qwb#9>TJI*H;kZ2Cu(anal+}Z9O24O1Z00e(Kj**DVHpMfm}KJy z#Yc}jPh(O2ZjYZzcJX3bXVW`mmc)sAhy#u68k;7m5z-fEV}G)Sy`U)kLJu%LST&oS zDbVON@xaf1Ors)`LRr`KOD~`fUl9&q2hG4cQBma#T-x z?A?G8^SyVj%UTM)UUbrWIT6_U;MUs9$>Z~~;IQtiM!y&PiO3^Tk~Qh_XVc-_6Kr2U zX@vIAy{j5~HifSJRI4SZ3nHeBsjDY*^KS?L^7q3Twr>VRe z2Q{gg2xksOrwCE)pZJbPL*UN^nb{O>?W>9b*gG>gBjj1&%)3RmQ2EL3s%K5NU;IO8q?sTopn0k#%c90- z{uFBqeq@GCE4sSgm-CUK9FCXHiZ9yFRAzi9nEz7nerClZ*lp6ivNV*)MrK9LQ8t^7 zH5EFFVdJWGGe$OetKI4zcUj@)fK$S3_J7iqbnj&PaXgZKakM4L7Ae^T+hJ_af&GRior!OVY(k<8r{uh;)8y^3)9Uu2#vso$f#+a_=( zrZ<>nDQB9EjYAguJ1U7`BMPkC_NF^pbuu~X=5a#0MpUeI_t02+O;XP@1-fe5Cv#6t zo8$xD8GbV_|JD-!nu^FEbK{A!68K0jnp~`XEH9FnxD1jmh%-B^chG4y{!n`_ zgd%0zEV~oQA0@UBTwJk;e&6{T(ZWg&?<>zIF2Z0x=1SepvKpP@bs;{tm@R6?4V=bK zhDO7HP2DEL5wr&sbn;r!iumi()})}HRgT}>TAVG~eh@vHhhZgKW<-t6aw-4IS@TBP z^}(}(UQK-9QfBBtys-3+^`>JM!Utem`gdl=*9K^+cBxK5hSnA@0Iz8vb(tX(oRHsQ zb3WHjCPi3pZ`p;%llc~kF1#7h;9^^bEOP#L!%2eg-Gb6#q&i7W=Osf#S(%C()fFM# zK7=*?lAapq>xyUg;buXTnT0~q0$;C-C2=zfmV`TTYXo4IQe;0FfNUqX7iC#4ih>kf zK)5Tt{h0Bq{GY?hE17#?vVg9lvzvaIgB$f}h7F^qreds*%<@0Gdh^0l z7YG=yLQ(^ejd^W-|U%J4epcHD?UE>wDU0)5^4`+{93`?4|mb-)QgU1 z*{b8V>Qm0%w}TC@S9!HJ+`@d*D_n0;oar68?UcsjJtbz?L=I9SjB5OG9(f1-bXYL@ z9+t$ip3?CWuGhrI96Gzw72G%~b??|v^T%(EHNIf8rQ3_%mmmM`m&4*_9h*nuHuH7m z^=b#^2SXKR0$dl)EDoS_3Gv{|Z=Am8Bcsm{k8XzPhLD^SIKKIZvfuQ2KT64%{#~qx zTVj=t3$8$gTQzgl1gt40ceU`sk~mOK0smq3?8eQ&SI1$!gzouYQ)?~DPB3y6ugkl) zT@q`ieMehPHaK=C3c(;bVn?iNpDiSwhn&>uBeZzebe^f845D^vxBm?Xzh*K8ATSQvmcf&3zKo18};EO z;+DA&mhu0_G<;Qunk-iVwsOmPUK>0H!a30YsmJ74iOLplLaq1r6tNaZheB4PMNS%M zx3!0)k(Bm=EhW@!KngjVtTbPMP&v2v;b~J(&!qOpW4Oo_cxLm|iZtu6lOMg-}F2oQBG_bRrYnZm+(2P3h#3V$_L;ZQg<8yuj7!~j`NL%oE5)96h>)k|B)Px<@k8`9swrMopE zgLt{hB6h~gaLgTH#YwRqe(LBHV|{Y0OkjC!d9SVw9^Pfn6_o{c;u8VsD=Tfixz|I7)cQM%<%rKGS!t-lD&FOB40YAY zXx5tn)YIMn({tXcmcSG>4)F+MVHn|(|J47_JFZfI{poJ1>q#PF3}q>?Qo*-Ay9fe% zp6?DhMoGl4-1|5V!~M-qA(gqJ=1!cdo&kFjd=r^{&!)Fks_?oV_nww&i z5AHH}#cZmZKsr>PQu<8kj%vQT;}15+M!@78ZqtFg zB5ZGqu@F&hx$cxtkWo;V(7WU=Q-1rN>WUo=MtEA!?Xl*MkESjntC*F52Oge<#nyCG zV{85V)FjkTLh@#9OEmVgcnNe;RQhQa)Gt~qb^5;Hwyp&If`_yIWtfERg3k9|WTvdf z?|*F})O+;HGK=n+N`MG?=>>0dIs8_hWpQb39FJ<~mRs0`Cb*gJSEFOjl4F&RZ{6rX z>XVcGe2j$WuAa|&qV8Jj#eOZT`;sr=`epm6xS;<_{7&8Ta}SWA0bz<$NY8OVV3^6^ zSU*)FSQwKnA6NO5Y9&b3(C(8{#e-1qCf}c(>GE2@1%qkWIMs^Sa;A&xSQM(yL1xhg z8QlG5s$$M(Gt+WwI5(-kHHZ?;^|kq@1{`pT-s#~p0R|KyV^E4Mm#4E>!V$8+24!Sh zUN+w_9i;}5#oV3tTcpVO^O!|w2WAZ=?!Tm@O|;v;Cyb18iix@@-BW-*5u0yX9k@xo zurjr=uFQ~_CNs$JLDFB8-78gzxTE2_8B5jD^@Y8)6kIP91dMwh_5Lbb>P$&awdjM# zD^q)f!A=lLyGroAnyw5ZNYZ)8tAz(~IT7NzAp_$syhNq;2E~_7O+3{cx{Y@ty6D_l zxz_2-3a<0#!Dl`Da&n*WoBv;Oy6`KNJ0t&S#(y1!^%UBPWcRfZJ{*=e5z=o|XEG$+ z+kgC+;MH4PM-Uwz{8Lu*Y@h>Pg(`&xD2sRjF zh&299wtjA?nK~jZD*LjlV0yJYfY*>_$C34AefWw`Mn$whb?VB_M_tvda$br({}QPF zX*bq~5YYN!D7#WI6!|9v%1P1Pp?aahsW@0ujZq~^UYf!lXLm!+%SYS_KT(@=f*9<2 z5f5!CobuD$QKmx{D|7{&yzauKwXb|tU&D{shgDZUW=LSXZSkt5HP}LQ9&%W#=@2*) zzDjK{HM-Jk6zYF#;Wo}m2PX06l84~0ol>*EawNYmrn)Prl}aY7(w*opyPpiiS%>40 zD@!B2?+vo&LZU|)uJ+UCVpm@CY-KBWMH{)DtouSqeyMWi4Hz!xe^*`GZVC1TT6ql@y8vP_oI-6TAD-1a>>-)FZ3;@ zlgr^%G2U&AINeAR+m9bve9)z5id6XI|n@;-4TD-_wBz&dolh*a7&V`*+_1G!4#x^@S^Ex$ zi*01j^dgST?(U?lk(*22G;Cx(Mp&a>yACX|`w)|$(pBXb|8b;4p{`dhy}=??)e%RU|r zrd>PEhnQ|aTsu8LWKsUeJyjZ-RxYEf;nMu{T6Ia6au;jjw)+)TW|i`wP`~OTk{J;- zhL_0!%Uqz36H+5B+QN=@32=0vuah2f*xtBZ0VVwux}5QhiZhBq7EeX)s@M0G1)KJ! z&#;GSXxf1*bM1HapW|=LI1m|I(Ihf{f%Pf2xNjX_N$?pbqO~jAV6=YEwNSEms+s(fH+0M zVY_SBJ?oO;(0kwKoZD$F*Lq@IQN~`q8bIiM#O0pTNm)UbE6y^xr%b`|A%ZktZV|gs zO~GL!>VGp!Dw~bzv|TqILreuMeLQ!nJgltA+2MHi<)#gpIgZI)bl=ho45b{l_(W;Z zti(((NxIyK9Nhw*e|ZmbIaQT8v}A9GgEDxn13Z3ypMuHEip`qCmDWyYoluPf%1QmA z2qX3HY;TW3?-_Lib;4*l1Xzunzsy5J6_7EygnzRrB0AFbFrfXIaxtq$(QGW*A8cHVUPk ze=~#j`ru;vV;jtUcn;~IFw&L^c-k%%5uCyRBea)WB28y|5c>`Ja-f|*2lKa9*QmQW z?^|bhepQkb{Z8wTJX-w5iFvks0gK56jO9RYRZp2lPk+#ky5)C;+zxs;DoF}}7Xxs( zIH{R&teaCQ5BE6zm4+tb$R1n*pCnfZNhB#;HFKTRd?wJ7+~S;);fWoQu*BX^(&G5i zzn(2)EabP}RI^~ntgCO zK@_aR&cIFy)&j>4tz%*$ZudD%siU9xtRD`(LVJGAQmPGvM*d0+xTf=-H85bD_qMo$l9GjdLsltE%Y-n6N zS>W$loTh!%E`oG{u*{2^^SI_-tIi?zDbRmYVKFv>NwDvut}%b}ad9T;;^=5C5V>c= zAu@GbnLKl|abEDHjH`D+L3tq0YmOsA-N|)vjA%!&5srD$B>E&x|S*FGlfR^B%f?_Z!gF z0bRYeg_%~!e9%fv^BU#iW$0xf-|p?B_=2EnNfIV1yhW+8XX}i^a%NY3R<$(BhD(V1k>J!wx8P~>(A;UE{a;v&CBKq|AWgVNZBP%28m%j7A3wIC~Z}tn|el{Ry z!`%wKBKD^ojz%TT>v z49*XDzw%q2tdoE;CPEimnMrU(ci?-Jz;cFM)o$BTJ>OZKb%lN=^U1-6_e|sKXRLW< zM7Mdqj_XB62u@)?B{$64+OGt!s(CZV8L|vo_r=7Xerx`tfd4o*H9O1n4LMO)8%T5O z;mX7^ab;3Ow7p-%T27>3IIXeh_Juc3*AbtEoWNmyCw>HMFfhyBU7c?MSw=)gD1aK+ zs_0(*rE;6&q%+%n&7*a1%p&2nPvjH#XNAdpk{3nPVja!_oWvCK^KskkgJrsA zWOxn9%$I}zS>v_0F0CKa8q_oJ(SBWyQGNR2Y15a+`ebL+ff;1R^ajvv&Ei3NF8$}+ z^qmO7>z=YN<^$<;-xlB68nJ5eeSx?&=HB*r*cQN$7aQpEWsGj6Y%$9C9MVU56bBGx z)#!TT^Dk894-=Sb*O`HyLg`{)Y;D#$(~%w*e4I9%<%4&bCb^M=v+|D7lg?7@FkL5M zLsFkRA2VBjV%5qwwy5iD6my-3;6~yPqOrfu$IJ3<+@fQoNB&DhBCe}r+B5evtE~a_ zWGdyzwdhIbkanQF;qjbmk6nfVx4Rryz8o+*>{o<&`j2lxtg_wV>%WR;0{*+Ts~B>c zoI}0^^{X~-AOM>m+@y1Fj_5@i*p@BCyD|Az0vbaVb3RVm;EleT93~w%R%X=po#}zP zf3Mkdy;>&CWRfO-{n)AQsOge>*5mUPiC$HXWj`{M&UbO3GrdQty3L=UwtuygZue-wvP$o4l zUcLsv2rl|$7V#^hU|6X zjpj6m3B8A~aAMC5V!gtC9>0WFtJ&=1Ajm`umb1)vY~cF`jSADy%trC}jkJBRlkz|K z^1F>u=9L%MqOYD7 zJfKTYi;_`MEt-W>|C43gr+elhe4l)T+J)rOEn%p;zTTvorC^(Y8J>|cF;4wF7kiIo z7LR34JO=H?PHxL`cKe!n9 zOSPgv;d>7WqUBdR=)_GG?Lu4#tKM~wNw)V&!3Mr_Mh<4aOY18cO026heh3B~DsnH@gSqT|;gFO*T&|LDVU)Id zl^5$ZZ58u6YSe%g$!hLes2PS=NKV?x1W~g-9*_FPv_69{;!)VG2x`Lb6uYj)YPi-< zT*G|AqfGp?E4JMhmArbE6=1agyB5I9Zs4OTtD)hjC(Wx`B;m0rWkc9r57MJk4$SN! z9)vtjX8K)fOinJTLvDiq_p0{Cp!Uu@6vL$r2_Y2bRh4Y`tKrVUrm`KqUz5V7hlkM?J z^LN%o3gZ=N5SB1EF0mP9US^lLzQu>gw0U*JX0%(Z1ibeZ4%0vyH=lNRWndVxvQWc z=SW$L^&*q)3JzMay-vj^XEiEYp)`hyS%xhWHxUUOkcdpPtN#PkOoscyZ{l()m0W1Z ze!$7Z?N!Vw<~GTAZz79yKg0uGVWp>0QdwVH|5D6ufj3neY;l;gFj{3ONE=oBte6V| zH;8#)-~6p@&V`|V`$PDqNhZ0^lD+wDjt^jE+)w+4?E()*A;kk4M48V@gkYZ=QrVKj z`e<|L3252w9BEd2jre4vKyx*oz(-dky7z+H*)I6u$X7u1W2j5Cu#BF$!{g}foM|fU z>R&&lTAfR!@e|4G9jkOB{zhyOpbBNI^-L0L=EQC8)t- z6H&>wCp%_Y`ygU{WclZ4l1-%prz)V~v9dBm-#e=u&%{0%`(9|57? zE~&HKN&UzF?qf(qs1=)fTGVrsr_~-w{Tb6$>k0A7qDat1bzbR>gY%HVJi5yaBN80e zTh@MBEt@cRTKSOiP6@BSi+|R#2A;m5kvE)DSFq6osZA{R;rKS_oG;`Nx3^S#-*aU1K;YyT#6{L z9al#TW}IGql^J-*(Tzz%epVo!khAQGzUH|q`vNt`_-8@&ncaHEoy@#FRJL_K#1B%qmd!`3S zRWXH?P*eF4iD!t}as@L!s%*8g#%;$|)m@BP(~?&Wv0c0`RV%G99LfMT@|{LeUGh)3 z&d)kaYE%#&e37G;5ztnedi>B)Y~xAfBE7k;e@q4L)}?Y@H3!3A|4lNviH*>h+N>sr zD(FWmo$ATF;c8wLQ72Gc-RC9r`Ya~4K=FOz-P!l2rrNnI_9kr$0rJT7BOroK!TSx+ z$GY!vs#5veh8=6k@oO75*v(&|R`ueuZWb>{=84_RRBUb~H=y|-nw82X1VAPXKe+p} z;4XthSGBF{+}n%(Sx?l*fAuUmZW{whOwr0Oh=jV?wLl7ulpdLkNg&h1X|VZataf2O)QM&RqG^ zjf|OzPw3cl5>rTxju&AWCnvG@yo~>5l>It@?^rEk8a(IS`zc1>XIK5x83zgS^S%2j zoK4cS*ty6%Zn2BCe)YM(qHckcpMK8&S4R1G73r`f!)?@YG>iKB=GWHR7NYIrg_eH* z?7tPg!imM7F1x{g2+31(+|vW;Mc*f0U!v2xg<9AiY7RV7&QS{HRIZMdX#(v(j zif`4GH-GzGY@^V=25;C0gwv;QleI@3v^NHq8o<=Giu#SnCt%KVkvt~l#P+V*;Wk-5 z>;E!R$n<)^p(HA&cEvza{~#^i9XcSs{B5e8{^#(E46>AO0?VQdAqfeRd|b*41I8U? zo!-1RZ5KCbYPIYe;Gc}5yg43xps4T^}vs(my-#nw`cEH(z0gM6(57fBV?b-r1hdpD4$nVwY2;-VNg^L z5T)j_8Obj!1#Z2Zq`zdKnUkK?YLk!P_rC=zS)5IrD)h{u8?HbduE$3w+{1^b7MHg) zh{z0VZLt{I^Uy1w=Qw0Q&kU`53ep`!6!w0g4$)gcOW_5Pp#kkXi#?>lkRZJP)|(nnoNncd;KgNa&JVTPsTv2hMIlq@NkL-YAuz`jIoeA zMq;om|LtH#yY=kZf6vgR9t|{HG|Q_u;G_U2*X`% zI5Moe_v45Y)7-l7P;wQe?1vP|r$ZpvN3O0aYP9+f(HgBA1$HI?;WqoGuG^1@Rn*~}FJ^~un z*_RiGMHKA3aFaE86Za8-(v_2P(-}XBJeT!S*!LSKB%4mjLFp0*c`>#rQak8MJc)Mu zVUm-{8bqy`5s5X))vBMg!rjFNpZ{L*+SJW_xllJ1F#M7a>=x4$=$*@0fa_?P=%j}X zZ!zs&dF>Zo1)}f54$~Uwm;<}*9Ec?@=qNy!oDhpkiArj9(?5-2%_!NN^#0&nk!axP z=y4ES@xNL!?ChrD@u2amoKa#O&!~MK0#$p0S6g*aw@Im9Djg5MlSBPo*2R7N-^|_( z+#_?DqiBvJN+eHv`{7BIZ#;2y{X4Ed-!D0Ov|Lx$S@vH0-8vE-ds)LS{+anLT(^eO z{bb?>>;}T!8>w_QH7{R~^Lo*_P+?zo)-u_64JbqWc<8{OxJc8Y2mP=NTK!ZxZo|MU zvX2P*SIW3^wd-jlx*r?lk7K{l(SNS=6B^vZ?DEv_txQf!hC@Dk&R*aEAECQzn!e>y z$}e;px0x*p!S(5WXlR-BoOQ4UNnV{rU?xO+k8QKJj;1n13Ate#1PvN#6V!Z6egzzyO;uyfp&k8`1ZIx2xq*VMT6H<;_O=$DYM= z#sLxb*_;4|_P??DR-Gi=hb3GnX5?q3rg4$3w?UnT*%Zi%Gx+v?eg}F_==_lQ$fvsF zN-$9J(->@N(4Uv`a4Ot% zPowGDtKW0kQ(cv{P z_P)n%qZh4Vx>sp`RQSskzJba$kB7GVgI`f`8=N})t=$2*NQ3_Q%fMukt(YI5T>jsv$XMKB5GDVwBVi)*7uF3~~3t=Nh{h(}o|4hlzP0_(^YdaeF z#|hPest&E)CaP%%U2;UgoX|o&(uL&~Pbb8ef9C_@+x@yU8`N|mPEs)3AS9N81FkHM zfUmCOJEE|qyUCi+lag;VSk``&_R^s(w4_7gbrZWY=I$E-44K#b9RmzT_X;*I)>vL) zv-|aOXBv5)LU!7H&*MRNtMi$rA*M?*%hnLkP8U9RJR2jXcN%(Hr(Sk ziomAB$Z%wI5q~B6iE~ZFu>$VDfg5ISAwj29(~Xp?>k|4U*mcmCykGkfa>T`*g5^Kj z$0R(O{H~}A2I8WDGWB(IW0{ib?OrSI`7D2<<^5?yx2h{DOv=nXiRd8h#*h5WJ3{yQ zvjMEDmyyBw;zi=lCnyU|tLfjri$Q{|x9}R4biJr&2emJnt3C->8>1Vv;t#GF@4nnS zG1!j6)h5pS(AjF;A*EG$dG-2v)1#2+d3uc`*3p#UoKGV%c2DFtjJr2mV4ha1oh5#+ zgSdQS&h5gP7S~+#bG!3pjcZs{RH862Ijz*U#1292Hh*pz5w)%acx-|~orsRo!V|>J zXtc&)^Qyul&S#LC%IhTY@|xOwI{NumlD`@wTJg#OaX9GurW?PQB-D7;$RM~C?ZhYc zCTP$Lu|0z<{uMb5XK)X`eG&cOMR0?jQKTcP+9$L&s@mjj_Bi8zc?6>$7Ugni=@(94 z9%m;HB+;I;@@Yup9O*%!s7#5X#~+?i<6Dd^IbCWdIlu$%)jcnZ*J5Y1JfbHu8A^V| z6v~3eYXze$o6P-5rg)GpVT%g>J>()+=s)5e*^2Hj+Bb(ir~c6&DYfkeTfv6^&TVWN z;m}gWW**~X&7~VK?LJMSFr=R@0GoB#HBPIFSGA~gjtVW0(78sBvr@n>^!JFjs+G?> z-2`LKfZKzwW&18$Zm`D!`{Z9vY!)h9Gje-v2E>Tb2uQMo3Xd0eI!ilN`J;KDS=3&9 znRo|PUAw6Y5toh%G_{tv+le$Q@#~CwkFHU1pO2-b=xdDO0oLfgTZ?b@5`Igj69qn-7+;b zc}tE*AL$aFMp}!C?&$b%){KChReGs0UQC9nUu%C-jS)WweHGinA)gSq72~IIk59jY z-uI6z;Pm6skPxm!S%qeHrEAe?<+l9%EY;551%=1;kq(kYAI+Gz4LefSMK{bq33RfE);_dlg%tAv(=9=3V`V_E+#_9cjr z)yI;F^YL+?MSF+;#PKxtc;3#+G$fp7!e$rpQeDZw|I#QPjdyws=*e|gPM1Zu6=<#avHr0)t7UmTgZO;ZgPLPc^&^-h^?8}~lh#_YnSg)d+2M&INSq?9Vz zf_Nj_N&65YM2`o;*$*}3FbH{pPXTXqkO&SIPHoG`HSZ^U>J{NGG0z$6_qK}qnRv7R zc-A-fA1N|@m)e)YLaQMlAry(hIVoGidE<6eTsM)p@_${1rv38eoht=eC(lt+!`%wG z5M+|lBU9w7Gpntytq!8dkKuydj-NPFMGeVEQ(085_7tb#qEj=KpA&^SV=ZA_`I;@a zv=$jew7G`;s#Dqe#D^0(N$w}J3SxK0+G1=$oCCk!kJ18ME23M4u})T@n$@8KpJlvh z(zpy;D*ZNz_-FpfHJb`Kof$M86_;2{WW6`BS0@w%^}f;`C;%dCA0E`kCil}%*(H5$ zRLy4FX*0D-iR^PwTKn{GoA8B3$+_xvOwNB3-b9IQEJeDM(!JcPD;63peENxZ;ZOeV zl&U9L!s07U`I*vPs%|- z(aC`>)hS-&vW$DN7Jpz*n`oMU9qTY6Qi0p=>FP)eJy{&3)XPh1TQ#ES-T+IXGF`SL?M_Jts&N?)IV6K?E9_dFWBzTIHqKAu?11&Ey(IIk z^lw8j)>CTj{+FP!L`y)!44}i1fw1oADHZ;2m7S)JPg*xHi!t7Yv)RD9wAqtJAna~^ zQHjsU`i6cEl37mHMmq*7w`dpEoiC|0mo{X(FI(Ur*6r3$xrLWmW~gSw2@b@YIr9$? znuh+y{M$QGhBId5kldCBEcd#j6<+3F8xz?kV4cBU4<(>v+)sClGH{=ib6s68Ppm+7 zoTpOgB5RPOS*od5O~WfzeEZms-Xk$FW7`}$v-yt*a#h&+HPn64!}7 zPlk^=EzMWkK5bH z@>N*;itygxhIZbMxeg4(S=tyQ@UFbi2Qt8I250^|f+_6O1DULuTm>{871x=F>J|6t z>1rL};fs=!3S&2{_+j&TW0v8DlRRRkS)2%tBSwfM^5vp!j4z$dDH4quQ?#|EAqnS0 zd7OMg&>szRmm1vRnfsSN21O7Gk0##ZyUpr&a-MfTBgh8^*eI?cZj}6JpduJSWWhEH zcw=6@mpvklAzNPPPD%_8+(UdYbY2Ctk*(5c?wz#o^=Hmu{J>X*P4KSI%f4h6MuNwH zmg`!7q>3lnYNSHOXjNJvpuLdPYitjekYBs!1^dwq_umFU>O)0rtNcoV|0^iG`>XsM zhj6|R}?zHWVujj+BdJYeu0JUb%5ASGUoDJyCoJy;xpYmG?;m^{bTKm zxZDw)%LL}zQ<(92o=`&v)rp7Tmp(ZG9W0%+7ORhUN#8)Qk4bBwT=j-C*bxiujkn#Y zdw|FF8EdCyvrF*&*S2G)Us4TzRsKRcttRaPBbj*>i`5%#m$#|6J6KcYZWM=CDW=tH zYoQ;7SC#o6V+5%Ro-Tw+OsQ^-lo@2#s|0U7rYTR(U@A%1i!fyT@J; zbjQs+7Zrdp^zSv_+L_tYzb>sV%FPVP7V>Vx%q~#T%&C@PYd08e1t32kF3aZ#pCCVC zwm22hxraa?32~_Z{5AmO#>m&~mpWEY*C_Am;U_=O2YWl3WV}T@Yxa%5pvtXCKacr# z%{+_`JmNne!($zX88hZ^Jtj6fmYiBn-!v;uUsLRKnSYNf38ZFjSk$tJpaWiA8$t^p z{)$|*g6gUZ?P{*ERes)qt?s?TJAVp1>>W{LmT7$oyN!+t10A-3U0@(6$G~+?dv~s- zA;t~t^KD{#0;gO;n+~^d|J>#0Nn^Y3N45d>Ot7~4kE0LTiu*&ls7nfo+FmrBrRvjm z@LZ`;`g~eYC3fA5uGgUeQ%TYSgJpp7$#zGVpLcDB`z;25UewItv*4tz?hS5!7=*k@EnNID8LAmEVgRzIECBPxS z(mmI_v6o@gLi!i8JwQX3A{sZiVsmz$@gP2&`%Z12RVY*7jS@Lm{N1YTCbEZX6&;7!4m7h~nJr$SE}kMn(|aI3O)Irbj_Gxff>rxP(1_n9n=Yy| z$6^9Q!7jg2F`qiq&qs-FYKqAI45715qviCtFu&>e+NC8>~u?*I(KsteR8&C_zoL40G(m;pd~7@ zJEV~hZ@zu-%|XB6_W`YE&zNVvR120TaiB}r9<`ht27`o$jtzzNKiHD7FUy*7SF##N zJ77?73>y#-4)b)Qy0m|QH*j#wTgxB(?ejJ{Hd4<~ro766n^~|vOmQnvF4J{1ej-|L z4Cg>z;5C?jKZxGx>t`9dFB;yNWK9~p7dDOu9Mp_Yw1ehm`D9HSksc?(nWu6rk{tW@ zgf&+vSZV9Uo>$YQ@Vb0KJ#cWwV?3szAH-M@o$<~|EtL(7nrxrD3e;bVy*wZ1fgPJ$ z4;+f$@QST9YE@jo6jZ1}KWj@1)z-~}J&~T{W6twW^X#8Q84~kIL3k^IDq|%&yI*4} z02m%{|3T)_B*5FB65cZDtu)jm+yY$!NVL~;h60T{6hXh7F0cv=MxwX(sCFja0%UI~ zvG@7*AEMgJ;Qh@uEj09a$Kl)f6?RjlrW%d%M{O>E*wCJb)#gla1iX<`+deKn# zrBqdH-0c%)DBg~{3;1%eEs`94x=vg5nl=KZNYaJWzbEIs?SF{${v`z!N$vwo1%#SE z;f8aUHJFx?vI|&b@bp9$I%O+|S>Nif`E3o%$eSDhF7A7m?AB;0oE#Db1vwK2Gr)FzSm0DH-`6df(wk)QA4i zGdm;28Bu6(33KXd^F;t0x*awWWi9sORAW!0L_P^E_~RT(na~RXOJ67*lgu+6XwWqH zdgIcpKlx}9&@jE9VM_LG+Aw5LmDOLdUqr9@m{tP-^lI_DzXuIwe91FNJP}+zUHL0u z-HN#iofa=nJg{2%PhJdNe_XpuCH_p<7Fi<|{~v;SOVG6IY03pMl&{2P*FW$gZ5^I3 znMBmisoVM?bw&;U$KARCuS}kLmQ37#lkQ^jZo7!@`3P10_= zE>&G;@$b%2VT6j49Q^l*n8Wx^&}T}gf_J5__Bwr5ys_OHXE4*7u$1jDy?DVZ<5hT9 z_tSj@w%$}eSM1NMGqMI_yn7s~`u=S!UyXesHH1)Il1s>5K?{{(Rt7pXC zc@Hp0kId1&YOA1hVdz&WtLXmdhi(5QX-RLm3lROjKwPK&=gci)#qWdi8khb@6J=0a z4%ol!N5tUimqMEH{&VHdB!u)sdDTjZ;f`hh_9n(_7bdx7Ji9_GL?~Wh26H`Y{uR`c zdDeVtH4PeKD=59LY=zHHTcDgGUx-5GZ5yw0tB>>N=-|V~$Z9E}ZI-6(xipK%Q%?;5 zqT6ayqMVomm&i%4T8CUxI6zk;|kBBN;tJY85&Pe)Qt|-+2OA`jpssghn&(s8Lm-Wj!^NJztxi7e1 z0<6MAJO80V*9}V>@Z00;q~AM8VFG6=YK1l*8>tjpCvZ;#bg>Csj%V&488Hc5)=ymv za6U&;xRWi36vs_K@yX)l{kMdK4nChSIM3l9<7yJp@btv_PnRK{Uv4b~gp|C@{u_m+ zU-i&x?YV7s;6L9EU{VSW?IhKg*&tlFm^K-ExC#MPp|?zo5`FD8+a3 z$1IC)R}ZMlD!$|q);QiJ@ca7&O2GDL9uSM6A3b*!MjHUNNq(DG;-7pwl%)ynR!z0wFl{VFwRpp2)-G;-eBpuL^B;lKvZ&J^p#gE-hV@ zO@ zhOSrn@}H1rd&r-M)0Y3PZNH7l%8V=7a`Oa^$xB2ECxlF9>!VcCc-H(uZij#6b&EYw z48p2)`qX|?A{vyrXWB*&(S9P9)w@;UQr2a4di`T(y7u>Ty|D_J4i#}~T8$O{_}vX( zgbm*HgU{5bTsi>5$2A`vB}H6!KB|8|Uu{_AzdvxRV{%5C6nThb#j%V`C$!j-zw4ym zRl)D&J7T~O{hVTLGim6b;!AJXvDr5zIXOAF{`(#7Jfj#GX6* z`x`GEgYxz0v3N`w<^eiP9PVtiSj@n8A}tME$Zjtb@b$^fGtQupW3M2w#wyZO8?+j7 z@ZAtfP#a~$0=|%{)~J0I)>(0rT-`>h1v;|@* zAarpJE(yCE5Ggb{Gza+Vn^?BCaSsL|4kd^(atQvUuSg_o^PLi|WZzl#<~8JMofs=E z-7rrO;!if^Bts^c_Z#DAI5L(dRdoWb7>iP*mW2D{=gjqsmus5{n})UEvs}%=YvO4w0a zL>|aQ&9tl>&1K^6{rCszEba=s=iad4CieXob~Iu-#~V)HxOx!Y+w)8R5#!LJWJP@u zs|>4IF8lF^pI3O}Y(d|mHYs=S?e=Qbq+)qt_Tk;3#7S#yz7;bht)NFuq;Ac(WPJ28 zLZq(Ct{@!q?b+upqn@;iXf1Hs$-j$BKmB5Nzq+bPV2G8Xd0^}J{W8AWs!)r)+&c|y zW1OCO&ZIo2gx&KhH_ti}IrQGB9cw#G5{aMi1ePJc@1#JZ#VeA;Z#)n;&jG$F!c^8n2MSK{ z6|N|U8E%V5kU#UZp7%u{v6K$rY+V&4jSS+@GEO?84Y*#nGcw0KIO@FLS?D_hwk0CxJB6w@DvWd&=I%7OUGWO0P!2=21=kZso9iHtE7NI z6K>s8_}9a%1QJ&)^_N?gqBrQAT+K+j`zB-J?F82Ed24faqJtCZq`1z?QU>YPtjGxg zS*{7|_IHFa{$#~j(}wqep5^`?g%LUbUg&yZ_>^W63fq=_>AAegTxp6cqE4H6N)EAn zQYSk8e-Cb<-07a*5_~@F``L#7-z<^DqpVvA)?&{KvhU?)Kg_4MwgBS=! zyfI{HKsHqpQQg{TSIEu})r}eSc-48u{Q)hW1NIcX#P2CC6WB}lh|#3JU@d*N74PZx z#N)_p^IqvofRVrv>$jfbH4$R8M+0xZ7+4R0W=%?MT@&YgX6acpzCYipX@;H`XF0a+ z&)Vr&qabbE7?pn;=cGDMx}S6x2QV`Jep~p~Y~{VmXEI*-nzt;lKZU&V0)yONm=u*W zKh%T`hYy9rA}n}QKfEv(_`=%<3f~IBf6n!m@wk#;gaE!Tu4%m9>~WqU&h8ef{C?f7 z`Dmu7M$^Mn#X}H~S^8*0f$5IZ=Q9t<^g1yOUsjA`sjI*^q8K797*Vx1N0zehWY=dJ zD%!sno=p~+bj|dm@v%BOKUo|FTYkY4&^$VhT+lLqNWWfnvNhG zqCa%a^C_C$7OzQI{s#Cg6osAnGtjIb{ zI9?IEJbuM*eN`~*d9JgN@3@^XeYxF_HtFdW7&G?zyX3bx^*VHsPpcZR^y{SugMz$t zRE2mQcI^wwr#GQwu*Aqxqqa}q=H}y?zaDc0XzQa~_s@6l$Nk<@rqoFWt@1K#am^^Q z|KKNiDj&!m%d&M6G3L3k8Hb&^DYfzBf>|5aO7QFVOMngN^;84;NpI$Ea5A`{@`Uzv50!l5spXKn}Vlry|2I$~k5@l}T19sEiO{IFMbzQFsBI+cfNaZ9ny*?*zGRu#WJ=vaL?tmATb)KiFU(htv4Dd**o3SsFjm9 zQv{VSbc3ZJx5ed&Xi@(dBVyZWv^W9QT)LcATHdJoN$405>Y+w@U*3XrlX5o*=pGlT z?(0fu$uQp#I}PTLsaV6GJUrSKdHh89bMN-S4YRAfzUrQkcIlkGzNKy*A|}l zW$%6-Gi2E1dmvgt9t7LM(<9zeu z=|DWWA1TNjsQ?;zs{bmu>BZ8LfiWxfnm)(zI#}Nh=%psaW&F;AJyCHrRkPaLT%sZ) zT>T9GP4ahcp*Dc+Y*qWyBk`XpjAf;VTk(&*6B&+Q9cxl&7C8_s)HKWL;H#uXT&dAo za0pN{wQLie4hju&Hyf1*L27TUQ~b4H4Gn8=3qTJi1X{sIsIpmP{TmUXCC7$u!gBW4tG-S@jL(sjAnwPR^ywUxCjm(XxY&3hCJ z!v)=#=w|xpP&>wptWySWhxKJ3ZJy#a7eArtj+6K(O>6cVZ=~59;WUDe!nu`ZjJOFu zon3)0V3zGBRrt5x${5PFDuF3*MsJ+#5tCE5WmU7$!s56_u+~;)^1!e*{*zB>or$&lrb3S)mOXf~bXrO6dxDWw8_hm* zx31737G;e6$?|}Y6XD)7lkch61c;zKlb9RrubXTp(IWgDg=nD#Dd4ch*mX+O4W;$) z&KEL^4NY~6x!NW^Cs-ZyWS(U}1flcQ_l=)i&ELN=Sp}!I`99*)rq!fm)~);M;@)XX zCV#z(R6`tvEeZ*yxDDry-R#QRg_>Ms%N%ADc_dgjnV#RkgjQMfA6}jX91HwoBrK(4 zXXmU@ta`C3TwdF1a2L%U$CrRt5k2K;lQxRBnIsIOxjSGX`i zXNhPGMp0w0FgMB1Ps4T&RzHxBnfs9N6Ymx+m1Dq`vR*y@>%*5uVKtjIqyVu<&jGTy zBc0}E4zuur+`T#qi|%D7i;*sh!;P=ykaTuR5ov4aaGuV1_|T_3g{pGKZhM{PzXVu+ zkkU4^s2j{IKMZOCDF`d?iny6BK<8=4n>X2A;6mLzcmWg3WxjQ!d7c#uq7c7=2Z~c^w zt`KKXr=tjoIyMByFe|M+8!n&0Ccob$)Lyi!8QTKmQ%$E$Zf`A{sRzFO^<@i}Lq(qn zNjd!8_wT5XBfKP5eC&$M3I`=k*L3<~5zjXlMpSJ%%;HAq?kt8WJB6KY7b2_x9?~rp zR-XIvkL#$jmMNRbbX23mQdL7TX0%7if)!X{RPzWfbTfNHW-QvZPf|?Vi!m{fLuZ8>A8| zOc5;t*jL%?uYa$W&NG64SxNI?YFEmUi>0b3GekY@5#n|v{n7AAY|OYsI+>H5=uuEn z9UMAKyo^aV%^&UlmiQ{Q#CQZ)KK>o|YOw!x>Q?E>(kH<3Hy3R{QX{L=3Dr}sL^qNAur2eP0m?d zaL_Zr!aj$xmr!Y8;S81&*JM7!GcOS-ukY?i3|bi8;V*WLE$o~3K25xpzP8JN(F_=@ zw4@*tz=BaV{`r*pg2U&dcU#NR*n=2olp$ z)A}KNorb4;&nL!V?FX-m{Up(JGowIvXff(ZQQ;Z;E~xlfoR#h1Yg?ie^^p(moB=dL z-1bnunIrq|W9=jB;3V!is?{fMqd4cozx&S`avIN{9D?sz^VaWW9Xs9JT*x@o2{@4&@z z?^ix7CXfmxwpeofV3+Yew@ zFX+SF&Wxl_likQ0hgjs{w$kq^K4xQL#}%jXOW^41V2`dedYMoA-z>Zni4^gZIeUe* zc|ita^|=4|d#+}q595@c^#(!$NxlgI1xRum{XbjSlo~4olGL?jcjH<{b^d z{NGt0ZOgeSae}HUQq+=1X(AJPuXC9lez!Jx^*o3;JqOkk`fLT68ij@K4iKxGTQbl( zPrVS5TQiW@@=F&E1}tMsx^(ey_jJ4Ngcem*#}J9w;;}g?YY*(lWI)MejE*p0)2UJz zyy?OZrj}T}O8!P)5y$C8ZFYUIO6%PMpBq?7nWowQH9s8x#a~hPh5@>Y-#^rva%DMo zC_HdiV8q6x2V{@c3^eTy8SFqIoSMw`?i^-ytJ<{`eY?n$vz7|+IX)fPJ<~r(L(fbF zSHGK#;(xmhkMV_H-{q|VY;fQP&q1+~kbzXel3n6?_=gLedBy~?lF5lu&OLf+ZX6cL zf7B&wM*0_j1gp?;tfqXiod$WMOyTjm6DQy0uLMW+8jxk--pzL%b8W>M{5La zv<8PMiIS2@_{*-BqO#-fr{s1;IM1j1qx`!=1BbNoXAk7e7pW_ju_Mn>msV*L)UYQ0 zytH<6qNc?+C&^ah`A5X!S)7tQgDDqRBG9k9Y_BrxH-YT+OFLWdY$O$SRI#30SPUqE zy}bW4J4j!xPrtH?h96^W&)-acCvQYFpjZLo2bDkXz%R&(o5SaM=2z#ye&de^k)&vah=}WHS+waVS|=2_uZ-Jg8|AI0d0eZ zKJE`E8s+Rv7x;aG9nOlV`)iEOqOwEdMoIF9O96xUqL_#m5(g;7%)^Mwce4cjWc7Lj zXn2auz^8Q#E2SCkQnKk_blPE#9I!ST!PzfWc)e$t6B@dD+L#2f^GYsjixIXwoPvZX zHMj~r=BelUk;8rqMV+Qa1CB{UB(}ff-_VGK`Uq<#RjLZDnl1NUyNE6`M>wWb_$xz1 zlixk->LWb16|3P|?iBs{HhfqS@rRs8|MyM!;kIkN9Bw8<=6@F+e7f{vSm?VRw|X2) zQ~+XyC>#w%R?ga!nsuT0X$7&+gm7cg9+uj( z%J?7Zb8$7UG>KBfo-BX2Z;B|A_bUtEC4Gmgn)^8&_F*jIPZ;c;iohyv2$T9-cyJP4 z-aYX`?n-KXoxiGt4FzlmbWyF&9qj>;RD^YJH8ty|eals1^7n#)1G-l{=hM@evp{F;C=DK_m6yY zOnuc8A)eNKG&XBI!_UXW*~xF6fnS@v5xHW47oWjQ?6_OJg*_d9!jXm|4f*DUy=Bzv zF!4S|)w*jP^2iPvtyY!?AQX$Jc<5gh4n9r6H{-D;sUfo;(39%q$o4-DJ}h{QmQL5#)5J_4R~{zn1qNHuDk@r~21}51u2Df|D5gQ1)uQ zbQ7;4x7(KR`41SK@PXYM354yo^x~@E-EE)k^CI^XofJO5QVqgll1s?%@*;1wH6Q!T zkvpY-_;M@0XPN9t_JGbz>|fyL6DTWgK$znsiaK&U@c<&CAhK1##LqcLjVSlbk1&^i3iuwj9yvR0lmAvYCWACe6W{iOcY9McC|> zv{Gtg#6AOXE2uQBA{9|90b>qfA}byTqZS)E9?JX zh&|j_(J?IHy3{M)L&|}tpFfgF5hF5(!H2v!YFiMc1sSZgwe8fItZ~ERx#mF}Rr%Vs z_8aIOyV|7#cS2Dp!AJYQHVdq~^UXMOY;Nvn4lqa8u^oI1XDYenbOC)JNYX95Jrl1b zuYAGhh2@4TzCNav5~+7r_)aN)*v=1e+fAn$4@Xt;$dNKG?^_~K-}Pz92K?Qh+X@1q zZgDp3?Cg#!UGw~;{e;cT>cSxA(oiars?lgz9;1{Ae$KHe0(+u$bI9!`FRMPgEKNWi zql`hOQUm@2yyB?zY`d@i_F>`bTO!cNDk z|M`ioICHrfxJ8L|wB1>6O12y0Wl$L2a5vr`7!zTiB+c#!Rui4qhIzjSoMv4>D$_{_ z#Xo+4AhxiZ;P!Qx>#%X3*0iLyx6A$7L5``%Q8AKQ%aNYV`iB%W6f-4@-?c!Gra>IOBBw(KQR1Gqd#XiZVyKZP#{AlP^ZKf3#!H%+Mh_9>+a* z;{M7Gvu3&!rAyxEh&qo?@-kM{W&2^qln;h97vm9%1gy!Nl9j4{r;nQJ+fmPgRHL0 zz}tOpl5gv<2&&lz_PS3Lg$0jQLHpFru2(3Sm^3csyoRHbqcYunp!pGPNa`%zOXDfVc^d~Et2~)DS~VvzB*mN$cQ`ha5WD# zcLUxUQ2j}t=-dOs*s3vpq88pHe`Ld)A>wB2?q1QD{hhji`EHQe-(}vr0g`ShQ>rZG z@$aVD1+Lrd26s=n>e1f)iRakl(K=nw1Y49$kV9(}mrCnn< zSe2r9t(sk?H*4!|;y&_TlirfuGQ^M&59bsQ+8(ovR(lw=$xAw~uPS2E{vrU>RYbfN zsl;O>h&Sz)9}H6LA8blO>SVT^-?<)6S|4t!ZsvvF)QS38V3cSkzkVaqAbCNegj^eW z7S}>q@9JOkA3x09|qNlA)tfBp6jvEYX)!G z*JnLz{d}n}NikM)NsXm`Ncxxr*1Q$W?}+u$kFj=GkD!**4D&98`~%vp+x%>ugMsVL z?ru-`)t=Je<7BvQjzYVyq`1wo_cGt&=W)I(`ST;tB_hTpl!n#X=z{FQ`O?qZ=8P8F zZOboi;R^rpcplr$0%B|`%C5kOO`S28Wq#!;XlP}R0Q9pqR*N^0Pl5Tj$a*5YV_w{W z-tDcwKzQ7==KV4Ae6TA-zk92^T!fLV^N?5!N$kQf`KgP zx3B9pQ%XD^hY|$+<|R-RN;jQVL+0)ATNS|6`g&B^BqjpSoX}t9`IQ%4*8+qW)b4um zTwNKEh~b0>o-zIXAe;p7Caq5d>@r3?@Ioq^pnvHspV3`8aRH%>4xGS&m-)%YI@l6P zt-OW|5ZV6EYD;{HqPg#NOU3)7hKb8X7`V5Bc1#nlLsx%(cJm|6w&#~SU%q+;9qjiB za#UH*1NYxUkSEG9pa2ql;s7QV7kbhyY*KEhj%XIDfigu;OK&Ta+1{N&);AZt>J{8E zFC{xRQUxuv8?oLk{$)K_>&-ZAjWlZizMtZV)H8Ow>E0sn!+p{nDES>``gBh&Q@AFQ zFEt@Ts0X8lxso9=p`X$sz08&DYOdUH`NI%-%xD_J>m3N;;I0`cm^Z&!*Dr&rL=^T6_)d)ZpIE@M6oA9sb=hPDp}pxs-NPrVwlX z!Ne6IjP6`rO@a}JmTz(=AkGvlmLUS_3U!gW{?0C2p&*0v>`dJ5>o`Q%@`DL&Wp_YC zy%$y|Oh^9Lyi)8CyYHu9sfTj7O-d_bhV1*xLu=3_Z$Ksjrljk%>Q?IM#8z^|Ea|RE z9n{eWSM zhpD{!_bgJpt=={5ss-$$&Bs9dW#Miy?;9Lbv$GM!;U3{n^64a+vdC*PQ~gKOX6RYN zd)!lU<*5$66;wYZpY4a+4yoCrUjImhl2W7W9 z-+AzTAG$6p-U9*}=eI2(`uJATT};F#FNarAgzCj=rsg$GlFi&!9o$)>w-b>-%et1YE%A@>H2d6e^|u z&TT$(F>esjq0^V?m%<@+=BL~25E)jlz0{%y(@tQwr)`mPYili0_`{PfIE>1J4nlz3 z=g3m5KGcVUE_tt0`k+-x{+M);Ve^kvc9cmCLFgd)W4)kNFA=|Q&%HyswUbmKb%-lV z$1ny>yugC+#8N=R$U?k3uj(kKKfo{xqX?j#4nc-6|C` zSffNdsU^AyC!*Tn-OU--Ry7KGIJins=juw%d^2fvlBCC~*LbKQA0UygDfIBsVfVP+7&OdRL?j=|~kXn)w)5^f#4t$C`g{@nI%y zJod}tx8br{J=F~FCS`Hy^ZxxzX0Xl$v#6uSG*TqfuGA%hOwsqODd~Emt@!jfQLh(t zxw`l&?dXoJOZ&CpKUJ^D*$n21-+q}Gz}pmcf$o#i?vVGA(V#r_gzw^nHoy2?1EdlT zKJ3w%-)gO2Ul`wCtIZ5WF#Q=UyDkitP^K%>xr}YSy8U~5nU|z;t@4Xa-9ssTXNBSX z&|l=Lk#+23+R{@p-v}xE(EBWo-zKMcRLn9x@mO2A`<~c5mEaI7lk8{WHU>1z&4JNS zD49Q~hSYlRpQd|?+kMx(G5bh`)m%G$L1Hwzi@DFOcM1VjbKp#=9-Cl^ z_ORIMz4NTN)3WIYWI>CnL!!-oXOcj;JN@AoJu_pn!*cW|1S>F12CmsDHD1C$eoKLw zkySFEV0P17TxQe}_H))kB%17A{~1jv^zWq@#IJ z_gf!)g5X=_-S}s2hkypleLX-uFGNSa8S%As0J!zyp`Z+bn%O2+ApCxE_>$J$X_W6^ zj>G%?8DcntZ=ZtS;#<8omq$fS5svbo%TSg0!2VO#90kI9$-Ebizl$wKcol{4&I(8B;8mqr|Z%oiKX9Uv{5R9wkMm z+}z-78^`XQY80_LmV%>yBgNLd!4v^Vp&+k@S}%C?ie6a3=4ZU5`Zv2ROWhJ1@X%Vo+16$)M9>32o9;tsMqujE9thPVeYM3Y#1E8YK4 zOCG@Y?!iO<%)?Un;b16_F=&dk9tU_qpk7`tlPc548?`OX=-xOSTo!k^Rrn`F}{%h;OlxukuW znuqZ;gg0VjB4kr=G2HsPlJYcJ9L zG=oXRp=vW8f7c0!1oL@o*vSnCxq%sy)(RgtEPQ#KJhN+rFRtUF8#BYU8gcZJ>M*q_ ze@Ig?&9A+3cWzUtF_Tsz7m4b#%(BiQnP8n-VpEfd@QkW=kiZToC?R&KKiDA`Dmhy% z(DOOWef2`iW43%f3xZfRsc@TfoEqfHxGtWa7poHEP{A<OY1Xyn6LYDbN2)t`6(k zC-g7civgQ1xuG}cwY;v{)>LV9KrJK5>hq^Rj(HG zkz@evs0~|ZUea?D>st9gqaM|kpmZ2qYPdt+!W>+B@vu8Hfh+ZG3>(lEVYk z&!Y)Y^2m|5YG#915eHWffGShW*BSz(EPqJl65kevx}aDovwI<#Jei!pt<^j*D-=kp z?-j$KjEm{`0^DsL68hJ3|K2(2?GND@<)&}NC zKohD6%Oy{kf?Pnt9}C`@q#sTJ%pH_kA?x64K?Xg%%LU*C^_8>|4GEsdFMWB|`yZ`~ z7DOnW!Dsl1)?yF)6b1FOtbnd;SLlN2+XG2(iuo@0c~@*vi-a5 zMPPhOhi5B|a3f*}7eA?-gftJRbuD3Y%0*nS53`|+nWE$m zEa}_`%mhY|5ncWEnAY`CNyVSs(@m;XJH;j|eHlDXGdIn?7 zn8+utUs6vdzy2#+p2+T|vi#>;?eL*eus^g>VmGw0C!;}vkrekr0{i3BK)_pL|Ed71 zB$Zrv^`WDnDO7TJS|6@dSK zVBbu$YSCTUK~w8NC7ed-QLZ!_X@#(Vl2!I@T4D!F!MA=zYnP`m(71ta)zNu|Q7N`C zwLrA+{Pzt-@{mPZUQugj>B{8mjLft$j9A9}TF@s{q1#^0yigf`8b|rB31m#v*XCQv zZlMbx7Ztt{TLjX#?e-skoYcX|MHi&PruR7h%5u(~w$jpgB*N;4DxStK&`q$|GKw7jt z@UCi@gt!aP$tG0Nxpe7hTH_c5>c~h8n*4Zwf#)aoVmIO+I+d>#)ss%OC~j;R(_HXKm1Tuty(3!jNIK8P+*_x(vq^o9B!pq zos<-n$|ij!5()itv;T--`qe!BBD%%f)S#=V&)MGI;DbC(gzl`7UnRtthJ_x*J@hCl zv3<3cNk`qoUkjN(~$uA^-OcF<+klle%*7mXZr*o%i z2YgEoUURFA^DpfV-sfnUC^KbVfvEN2(hp?$@{go1Uh%SlPzCcr0}WW5y- z39-dIpxWtp@kX&F(&`Dz=--$=E>C~xat{b-5B1pDQO6BN6od^%!X<*ba(uwDn+|BH zSI!-O%9c_r#m_CTbsY+w$Y>QJ#j+6J;SJMcti zh=O)fBB#_qwx><%dtQl;+tB}u-&!cOwVAK5su)`}o$>2eq6xE=a>qrVk&|Z5YZE!< z&zT`d=+RMW!7DF1pF@BG@_%9v#-e6q^pQ9oHU1c*RzI3_adF!@;=fCOZ-FNHKe1Nl zLZ}-dq!00CYU3{@+6PhbeBYzpLP#6Jz{C@KrdeN4ve-At^R(oRQ{sf4BmwQ= z|6FY)jdl_Qde9%#iTwo63Sqet!g-qOR#w5^BX@PLMn3Y$BW*NkwpeaB$-m`s#_jU= zX1r?o!ojD?SY4quf5z4FuX85JA*mO?TI>zzg|y3VyGwQ7e)`5yiSFZu*B*;t9bJr} zTL|eJMm1|!ODz|Sr~&!5QSxl4JRBabEK}QSKKLLJ2h5rrfg=P#+)p$MZ$}LnG)PyP zRY--dHp2uO{2TT)`Wdog;%vRXk+o@4tqj{~ggpD~v-YYf_IK{ER5VIXxOAe-!k%`z zI<2i)zxn2CiHkZ6nR2uI^yJju1|lEI4afIwvk9{f(<^BBEwyvv-vaVK;u*0; zm@Qtzm=EzpAFb|UTuYOF4&3o}WiF--@6x|u`ak^HbpHnd-HA}AUn(9UI`cxD@fzj< z&zH!nsr8#nk3NG07~=WlVtKyy@|BZMF4)}q&7}`@4)I*_#QFK!%U2%ri_e=|zq#~r zdj);&Le<}>cuASp0>ORgYuqFL9RtE_?85j6@r0JL0EW0Hh`u7Dw)t9mG7h$r-+Od;+G{$un{4AOa#(7!3%=>yl|u-NsGI4D5B*FO6Jl?NVp zz+h{Cmi_BCU1s9a&F6ojS$mU>`Rq9|#j4JgpZ)&X){0{Fsn^)O_O%JK($X?1(*v(Q zmh0phCeX_C190V{(N{4BDc5TW|Id_!fQJDS?XXrrnkYhy%2Qz%vfr0?6_J zL)_Ei4jev#OpvFntimSGoQ#Sm@;cmYGX%M@^-!6NQ62Hoy@wxu*!1Ygp8-3h))%&X zg;q*~jk-TMCe3ypv8z<+NBe9bcvvRAA&BLwaS5D#r{F^AhA zWdoBRurOhU*>%+SFysZEj<3>zuMK*xE^zX5^--RFa8EPH+AjLjCf0yi1V*z1_H_Tv zvQ}t$o-8(~$*0D3zHyUjz-eycEH~cfMAx2kGMIO>udXD`bO2(dlxFw-k7UX7n-dwxd1 zJe+jtZzeH2s4dKP9@374j*!N41~vbvap@oA0(Ye;oaJc8>cH+e%()%{M@2%#jn+fQ z!{x#MoiXLRdCFIwPv_>=Z!Uel_VbnJYcF4UYx&3ZTiagDFW-5-_VSgtmVaEowe7|H z3hCU&4i(6a8*ad09^@pRt301B&8^>D`qug;Uwiq=yYz3|0?oBwbLkszy)%{Kx~`K2 z5LqE!U+aPGYxKY?e^_lbX!{{Dc*qVmX|_rCk#1dDDZl-m?t_9f<$9SiD*n!*E6Vv> zJwd*8=|n4@d)5u|LL0pNa?d^YNJT}3>^XKH>8X2#*`bpz*l-_vYGVSXXC<}<>LpyA zh5IhVj9Yx+p)Raw!1Jevz~cjTj0SbBegL_(5TPv?L3wKNhMt&TD)a+*J$v?&%AQpU ztDKA;Lf&3pb}Iv&>Q$+A#_hMi+3wzZ@6BoAty54huRYK0e%fZG*}wkvFMAtm?|sJW z8)w+7-0$@Ug%IWn)SC9sCe*y5`TBlX?yKYkEuS=lfYuS+%pJPqvJaQqGQpg!b)ten zgZrgP>$e6>%j?LK_uqfl0s}rkh?p-7cYfhBk!EQr@_4{s-glta4r)An2ZtXmPa*!f z{|*{%l~eVNO7oI`!~sETZ98Zu`eU+TRabmq;6R=YAi5gN9Ws0-Ut=o;S*J z@C)kzS0=^>yP3Jv_WFmN)7f5|H6M8Yz>?Oivnb>M#sps@abRdm|G&TFT{hDpaO{Ow z%5lR|e~+kn;RZRvf3CVi1Zt;7*Q{AB>vh9b`kgTGdk6X7P@tX)G@vzkKD5@kUv` z`qZ_M^=~Xb7$>FkzV`B!x0Zig zzqRef{PLaWYcF4UYx&3ZTiagDuOrX%jU(A)b0ZJ#q1>StI5%;oeG?N92OT_7*60V< z?z-!)Y}K2bSM$Q}W!gMPe))x;m6p6+dB~T~ld;BPwp@Ex@YYw8rAQGz_0&`H#v5Rv%gIS{`R-Owd-xl5?dKta<^Q94*?+UwNu(+2$$LX-#K&USfzsxJHl+h4>{!H zl&|lH<@tEEe9}yNUVMYJ;-h8x+UqpsufO?0nyjU1XB~gCV!3`6cZ++#B{Y9v?gyRS z%>gN{G?oWi-82wo9)_FeQ6nw31ZaoVRXpM6c^e<_N|9PE1c8=_}1tUWnfc{b7@o{75 z3zLHXvGGx*WUM;h`XJ9DeR%T};v_{Ld4{~mKOYS1ggEP|q@F)yOyMk#VaOxINtbVu zigbh_j}UKaIq7o2kY|XKKGZqHSsufXM~IWIz4b?%j1%Db5@`WLob~+N5>K~j2p=KQ zSHvyT=QEEq^e4|7gQNuvan^~iRmf)^X+jwC2=S1nIe5q;#K|Mf3vrg`gJGQzXFci) z7~(9CVaOxIW4flEhde@@^^q6)pSWc<;Rk$JC&XFR7xT1J!Eo#d@oY+%J?%jQ&jPTC zd-edtq(gU|gu!uy|DyY^e)X$tY?_!?D}TAd-?xvwRHja7$&{HkHTW30e&xCk+Chg* zk|j%)$o==<4|Z*vn()?L?UElNK>0%Om%JMOqcw$oLSd+fQl z*~DUBZDj9OXlPq_BgFM^v7U`nmORu!B6Me2e^bjkvKHsdglo}?gR57s%vYh)lutfv z*OaWaugm=f_KMi*Rm-*WX~#^g@5IIG2uz0ADhbW9POE1QhTcliYnM)=M<9}33*5ph zKuz6rvc~8!X=5pt0B!mbMH(=$=)lo{8F)Qd$7bo1Gmj|nRb2)jQ2d>w@zrL@AKAct zv8@k-RPIs>#1~pmax5F;?J+*b@@F9BolC^`Yn25`VVT|EpJaVE>(XwoC{rld6zNxpN{YKluks@4*Q-SOSK(m`4=(7r z`VU_z(!|&zL_7dzED>jU=deQ4q-?IPv;iSo2`5XK=lWG)cdX{>Pu;c^*w*^9uh2em zoW$p(BV;)t^3!orc0L%}8X6!kYjT5_l|z5hVa&oL(w36Vwjb6mBX`rUOOCRYW^cXq zmJA<0JS)4#GV2%LC8xN>Kyuy>zbJznBW|>;NIrS~%_6f;SE`xvs|U`POC}9yv_`f< zd@=KuTW*nl{RhbS2?q)G|H79up?7zErZYQY?8fj7`vLldXFp7SxqL04ePK_d4TxFE zApC$b`h&QqXWd-zMCvxk()V7IdADCHr_b;~pxINddR$H$)67?Nvj}_B zO*hFpeM9HCPk-L_qQf3;SY2lmhnVP88+g-XshuZ_6$u+5Z`Gt3ZXO{f;l_=veHXeB zNw;Q^ZbtsKV21p|r~WFb5ZIc6?IlXV;0RdR^gFTf0GO*RPWi zkPCwV2BSO)_0bUwqM9CVm~a*2vCEU&cm@hyAi@i!tLc6bx8UfYiG}9e@>(r<=bc6J z&XbQxMP-G~;_Gr={YwOBG5gOtVf=%u8`M%V$1=p_i9)`!Qa^uc@})G-#y=%D+K*GF zO>)zBm$~JY5uav?!HUUM5Q#}w{DgV1XDxcq*Vc^Hp<32->I6yq>E;ovOuX$pzYAVR7MT0l#zogrMO5Z&~(s@?=^T% z2qT1eY!t?8oZlJFsTUz+4)Nyb06XBiP+Qj^iIBcZEb^2P~(rV&EeKX z+gU7t+mGU+ZczpU& z95bw$_0!4AMtw%EpK`kGX*}aczm>Bloh!|qHoNxPYo(^9M!s~?sTx%1~32`9U7|%VY(vbuqyQ ze_BfM!)bRdv;FFHGH};Xqir=7UbTJx`R8S?z4mfdTJ>AwYR{i8JC*xA>l)>!Zx79# zBi}r3OrumyDa3~juDId~`v}6($9`IM^{edEhtL@|H3PvU?!-7Nd^*B%{L zx#PB5P#l?w6abbKQtwK{UZ6t?9}PDh}G*p2Dm0#um_V;h-eq@K6dTgjk9J4 zt%EpbAIYpAS&O_4X115TPf!NXLk>66``VW0yQsgCymYxGzv8#Xz=^sOqB(ZhSm!0Jk= z?pZ3OCHmTjPMn0vu3t$!`+lL(ZT(jX>18J&YQ{O~eJ}kC)Q*?yFQ}hRk$#p*pTF;e zXYLf&k11pMV)|IyLe{@dXK6NS9IULVl@C|0mqj0}mN(w7wWF+fh3vC)PuX`wm41n> z*kTQTU<+d|woP&#ImzlEuM;Ldbnt!)x-c)7I-~`}f&NynA8UH`z4h|a+iT_RWt;p+ z=s5Jy2g8zL?CF#0HMw|q<#$dCxQWp^d0X&2xz3cREbXon)D<#zcu(n9Rgzj=qNRv4 z9w4?pWVO<9pOpZ{UXrm5A!)kd8i)fAM;@Au^c>?9@=NrU#rl$ZIYjr0`t7a1l$&q9 zS-$g~@5u1s!<%lE`ZwfHGu;b$XZ-q18PTfQ1vFbFpS|!Zoix)wpwGKeo>_9XOb*^g z$tT!KeRJl@E3cGZx@rEg$DL^Vnc?e_SY>A8Ga0}xdYH`=c2gRpdYu$&&`EoktnmZ_ z4&-X$v@?{rds;x3GAURKZzgRPy^ruR_zJ$77sLw$dCiah91}@cZH9f@N=i!QvnQV} z-@9mzL?us{jRcRSU`sldsP!RRjC1k3stD_z?$0V!6Fj$x2;tD_CHPOcTPrJ_deCAG4p7mlq;Yv@5quiw*QGywZuU~sF(zbtmddnFFgEgb~EXo>wq; z=Ja5^Isz@>;H()L^ieEl%HyuFlsE3{v5st;_KM55HT@fOl6r+sP=By|oxJt_DtYbg z74nS!(E`V=gUjXM(YuIq~GR`9>S!~k58N{7CBWu6&Bfz+ zMvu~7{blmE;0X@WdEMJ%Y-YT@aTF+hVolj`XLh-+ow)6h-Mv4$)KaSds zb@GpYFPB%|tJS|ucay{S*-<8p9&D>zsI7Z|b(7BNdG3sTg3xyiE%g0MgPk|bs6pKE zO_7lteT8i%^hq9aTyJv_r=WKbrFMc2<^`f$gqifHi+ZS>nJ7yluaczWu1><=7zGSF zx&D27mC3|$JLyE~PVho@V>y?;ph=SAY3IHG8{T8W zL1)XPSf7ir|1|deI`r_P}KqTT8r$(Dz4l>8n$WwUK*<5MYZ~IwbjjYc3eS=~9?&9MOMOgqfk&Y1K zzTpZTfx8x%%%Dd;`a8>wnoP6s%k0!v-*;){7|Cz8(LDB9Yc6Fk%1M|}zf1qXrf``4 zPr1j18GT~KIO!Y-F=02I9%hr8G<%`R>+5I>Sf4)p7QBKce96g#u#Bf z-mX0u(>NJC2N9kWuTqFmL-9h%;d7%91q1WWK_t(gZe=e0zAJ9*F z>#Mt1!GVN$!9m;*pHF;ShSZ%pqR*kf;WfmQYZgg%l{f?|thqxbeFp73BC65={z%$&c7C-lCjC4JTUf@{axLtV7x1)^Mp zC^Ov4<2s(!Oxiat$xmkcV4Wf+O6V9tc28sNd6|$4Lgt|rvLY-n{qZ*LsA2u}=qC#n zFPD3te9KmME?!nEpWLsn_OmjZ0J2FSRVN&qX{Rs`DtFdo{m9&h_zJWL@U*_Behzh~ zJ$O%qeX?LTti&qO37681W%AXt&X;db`?7rHD_@cEzyz5l>Tg|HO=+)esx4p#ECAX#W`e99JY0(2dPT~YzAale z)k?T)dv_hbFoD)h`%-uK1J_FBie86Aci`R^-EE6bjseF+8SpS|>e*X14jCc!JB*ZG z{RT)6l>sZpFqubR#t{6`7TdvdFX)T(3z}Flh=QGW-c>$z{O9D(Km10%@r`fDFMs*V zu-SsgZ@>L^dsTS&h>>#4CqH8!tiS}Q`O?08>Lx!;5kIE6fz^!xp@Otum`5Caf_;Q} zxnQd%&CF$ipw-oVbG6fX@?jCZkjt=1a@nnuWX9#s%GJ~NmFaT|O9garY~EPNiGwp` z#GYoBG-ljaN%Sn}_3oxu2b5$OU`+=Z)JgE$K%`}U)ZoLOL0^>v3Qp~-k~&}kB4zNi zT>SOZy&8be_u})!aZ1{c3p)8ZZeXeV)>cKgi%HU&_uonIZ58RO7sgcZ2c1CU(*oKc zwAWZ=9!}!~awNhSPdb<<2*&HAp5@tDs;#8`h_@$8Kt6sZKd9Lgy?%+n$Y&l7UpHW{i_}Yhq56GtX#gTc4TFUm5m~`5Dtr@nhq(g^%i#kd*CB)5{Yr`{#?Py=MP<$f>TIQJnNec*iJ?Jf*69=>u^Vdw$ zpge=WFGrcBs6EPVPzfj-AQE}AebNy^SMqQbl#*>|Mh3raTz^cEeen}}$lsq{WGl~Z zdt{k>_MraKuebYaIA0^7UCd_DPhh^Nq_9t^6Ja)KSq6v$uVX&GV5L0z>Nh%U;rL&2lMTwM>d@R!XtH4O6^+ZHu0l8j5uPxXLQ2tLP$-skVy=uTrJK#E@=xpTsQ3Kj%Yda#O)uAHM9a15pp1Oe)g)_@hLr zuF9JFr88vyb59HQ1v~uk!)5B!sRiY)T*AEZ#vAhGFMnBA$5qI*Z+=(Gu#cJgF!}mv zFrVOfcwhQ4J+xD@PlDH}XaF1WQ$4m;xi$pNR&Dj4;?c>38g>8N8>9*mmkI`QZk zcBicnhX3^q`t{5W8)W19^&-#Rr_$;_&9@YZ$(xXD5^u4Ku(G!qeNa$u z&m?Hu@pf`BK20XH3p&)!#m~!zLHD?wc9ZgoN;jBd(@af`cB~8;P(hDG&q!u4Y*iw2 zr&ESan!)?efBLQ5{<~Y6QKPL$bwh2x$_yNLI`ClqEwEYYwo+C3V6#qaPKvm6(U6LI zy{$>|&}6>yRB&bYKe)LFaRF$!wo7VjfDHj=W>{{cKYsjoG_Ex#muJT;BS??`eEQ`p z@6w;frAK4?ZUz508XO&5NV5<^oB-zvGkl$ClWzL^$HzXh3A2O743SCWhAFKQr^R=m z56k@Zgx5XN+jRncEhuN6af#|vEJE-}qYs(}N;<+Y&qyIR+pt2VWS$Y|4`nmdampz# z>kyJRd5|}tCd!_ZUsRuZ5OzlWyXyoiCR)q&!w?TW|Bn3WvE}lGLwmRnMd$+&ZsfTd zhGq;QO&R8O2Hg3YKKByCksOYE7WBtt8TuV3%z9Ur$|*Y8h>vS9F=hxeVj+qh@C7X& zl!%A*!@fYVT%@Ypv8XYBEhi0R%;4|KQDzB!fIKwAipU=H5zt;)slMd{w``)<~{ODsiXlG~D^k2AWTS?Rt5C>A4eimH% zLudrt{?Z()gaOhpZ10E5r3GJ)XA^S#ie$L z%AqqqT7*?(-P|OlK376EtcJs6T~B>uuS8#$#RQSPN@|l~Zt{$NcJ9+Yq5t^wU|*3` zUU;r8!)h$WHcF%$alVdmicjq9d?kz4m+&yoEJ#6Y}qn7`skzW%ctj_e~}Cy zzKgCHtdt6U7Kcyq3M8NtoTi)6xSSIR3o9`AlqVN9G27`R<9fbmibli7vR;z=D47H0U+QTK!V#=V)VV*m~FDP7clVGU}B{5frl#G!q{1MCS zS0E=Tpq-{)h+S^#S8Hn(N8@tS8Q4V~eAs6}pQt!F3L$kjj1;w+vbI`OMhpH zt?hs3tIqe!n@+|jAPzhv-2ePaSyrPf%yu1Q6J{3KDEDgO8o>IA=HXbMWVR4Np8-EW zB;vjewhd^-`eS)^mgR@CHCA7%v2Vp3&`Xk7*13oPk6LfkRwRp&KU67U%~y9$BzfFkyDuF=M1x zg+8nzmV9hy=*lw2TNEqyFt2$AQJFaTMfE8bVFrC90O<%(9O5Q7+pt0{rHtf7*-_Wy zg!$S}Y(pO8?e+Ul)&J1FhDx#SIeFhRi{;Nxu9TAx*Ht?Bx@r^tz)e%qJ)IdcjsR0Q zHPTq5vSH8?M=>TDw{*AH0_`36^w-%Y`x>MEX~VugxmhQ_u*p9z zfb~VX>zGc`AP<|+V-K^2hGJ<@zE~xOMIg3Iw_PlG#dId3Kf3GW7bbeJA`Fvcn54tx zPq9t_V}%=DJH!_uOUrftHJva+88*GQ{npffc3mI@YvPG-wOxJ9$|jRAd8Q=rCE)f$ zc93b`zC^CL?7MQ@amUF+4?QG%?X?%U6kMvQsj&$&tTe;dG7moNDAhw(3!1Iemr37t zlj5n^hNcK+KqNXCL$S*P3~~C#(Kqe4m6K*(NOOx55F78k^x|_(Ykg}{Y4G%FVtaFE zO%ogwFT7pmUUs5>_|~OSCtfI5O`Iju<~5N~JCcnUIZFCeV<~&+gSKc8ek4!uBMz7? z<)%k)x_v9^q;x}>Y|w!sNeEsYXb`l?sxx$c1h*dqjI*8tB4wku4&dpDvr`yv)MvT! zptQ!%w?%psB8&2r>9Ww`-qrfmjw&6%a$ShYF_AKA1ueB@YQOgI-gw8PIkMor|CMng z++UijM(!!9OC?Z~-H|Ik15nX}~9 z@<096xgf6vyf)wg9>*(-SIf)q*2>`iJ!SF%+CTj@Bh)UwzJz6lqmI9(;M{tR=DD8X zV%YLnk$uoZ%0wB_9`nebGT=PDryxUfD1)A|F8QHOnBx5gDI4V_Ew7JuT~z+OKB7F0 zH`@k8nI33^{Nlcxj6nsizbOM(~F*>b0##hE}iAudepB2PLc~ zpxt5!Ud&7Qc2Wd(f-YRm$-cI1<$C$YbF1v58Ydq$+9u2_Hk6(*je4w~s1N(LRr0<} zVleU@+orEAmZ49s_9>f(S?g=vT=l#3SKGAE{~z0Ph^$_}$xTrI`y)AO?`n%X+oMYT z!Tyxt6D^~EbGfzCKf)U8{wO$wFkiXKF&O%Uls#q^8+TYhlK zH{~Zk`H9r(n^{*~byZ^1h)su`{`ki~mJ2VuQ2O@mFVipjZ`p3UAv*C@qN}!g>6=0o zhGXRwRstt|K9Clxv~+K+qB6+Y-NKL>KL`K z-4IrI*$76vgQv!H@^Jw@cdviy1f|M?__WC}y&hl^4X=a35BL}Z%8ND6_B4Z>HeLmg zvWEkTwu@MIua{zSsX?k#|GI`csn->8`|f*yoc)apZPS}@`z!s!K&dhX`)V=&D(nIZ_SoQ z?zTrNBGrGqnS%_(!<#uqua*0n>?4Rp)-fHT{_k${L5(;-0yoBlWkf%0jS$5l9+4+y zq#^I7wHM1zUApwAJ=3jEzZ1sx(_p{=g+MY#wU_@>(#7#m;O7wBhbV4x?Iu`X%Y5Xu z4SKWLO^!cwSKSvR^)MCcnPKZl)i`$JNhy$r{1QLGUnujwON)K)N0fW9Q$1PBC=<#v z`1!oB%@DUb!B5yeWdJ>J*lGQ`b^5x*8hdpFA2Nx*#D(i7Ux@V&gbM&jjxeI>vZof-HbZ;<00u2o@g*T`)o}XbRKmgL=3@3_rI`4 zALO>k35V<={q^;_Y{}v;=!Y3J@PYEOnAbw<8=6Pc(Ts`zK#SyIZ1>Q2N!uzf`4gt` zqYTP6=4WjMKXG3$c~LK*3DHCb${iL0XWM`{)#E>Tg8HcOJIQ;?YV>QzYh{=1*2}It z=qeq(e-9JmkPu}d9@2I?oPh)w;--kp#0S>w`O2@vK`#4=k?9FmHuY%0gpE$*=wwc@ zKBx8WQ!U@QWTssHql@L5Yp#(GKm5=(XOG8!0utNM>-};tdm0AA*17-SmXX>gd zU44js%=Fm}I@m-Np51gjhH<#(b*#VZb4sFiz;$f19+NnTh{$Y5YwYplb0k3Cz2e`X#J+swi$kh)nVC9(pgMG0#7h=@*=~C!qbt}v4I%?87 zAHl4sN1ixGn;ds=uEoeS{krMoAns#&sO0SS9o!7;+E=bSXV2lim-+R+DvnO>|&=FxgbMQ8uivl=b>PH{S7X&_M<>^Y(C( zw1~D{Ta2Z+Ehim#JLy128rPy5R9x@L&o7=-125#&VS<;sJ^0YZs72iAn7=;iv^<6U2k74PSlayll6PCqu!S{e&3j&c za@;WcaIMHd`S@4P5Scf-h1?3e+U}#rCIa%Fm=Faa9{R`1G(MQ;XnkJ@D%k}bg_c~o zi>o*`>{~tK7;=tLVYhVodXr zSc}T-1{TqRd`+NBzp1sxL15+f6XK~{U&I?j&rfsN21!$Kir93~NA4Nzqf*e(NRK?w z@_6aR=XD>O2lbDi&&$HsUzLS#ye6wxuVRx4#0~$@p*zZmk-Nz*y7GPh2_KV>9elX- z*K4LFuYb~JWG|yB0R0ntsX&QMAY3Fb?1MamS<2~~(Ers{<%0e3{`uTH^6bJjGJcmT zdrcD$MWiDHOB2A5ZQ#cbowfo)ZHEWk1@EkrB`fP?w;lV+=$-nfY^my|#j>|^nr#A; zzy~O!FXBllkcRwXU)ls7i2OtzR*WoO^rpQ1_99uU6T|D)uC=kDQdc=(g1EYGU)g1s z-R!~KWYV^|&4I}QUXTaSrx0fzVMZ)zK}b*CtSwdwU~Bcy+CJjjC-#G+Z$$Qv?P)bT zP=EBNPaU|6%>K)B^3RuQWY{3xm`W4w*-?g+VbhelR^C~D81tdL%mYjvjzV)RFV-J) z%)`Kgy}po#g#vh$5d9SsI+!RLtP>U&&$wK!{Mn^)+ikbWi!Z)tdx!15`|hC_TNQuz z-FNLH1&=)Phz#9fnEbzsek406EGa3|ZKC>^?`|>%nQl);xA?)ltKEsi&Un2OuVyN06J|h)FbSmX zU?L9kVS){|N3zK_$m%;G{0LNdyJY|XKmbWZK~&rz$~_L~k|H)S>UCATTt5VWS4*)^ z8e{;2Ll6JBKGXKm6=(k~C!BDCeDj;%6s*+aSDwjjYr^0C?sxLluYOh5>bN^a_Yyzx zb6=F6RaH`^^7rc9+rMgCs*`8hpRrm{RpITLO0T&jCJ{! z1hk*JD8#kZHrm_xMNKrhNq;usVxfE+diIeXLwD8+Ml{?RvAOb%)vwKT#H5+b`m4M$^Eesf_W~Yu!kIEvX3LyRg*$!DUi*v> zbZ~xh{TA8j{&)S<^@@4M;rjt}#|t?~X(UW90a61n!s3lL>4)=1wn{$UHm2^()O zC4-5hKk4*c`aA#7i!JjHZ3~!g-<1DDn!>~}{%1?PdBV+bt-f;c+@cNks@TW&7~*bz z{askb^#l3f$-6Y-z8$tkh_*u+(kI=A2Jo@4gbFPD8zPB`%k z`bU@{)&_Vb53inaKKr(p{>~bi_NOlZk@CrBb_8u6|4$e_Se|%!i9ElkR`wjKdj(+? zgg(e^^T|wu5kvPe(-jz)pnmd=wf3-sl|is_63?__V0*-O68llP`AX|I#aG6IKGOS9 zqz{(7XEFcqL-XXxCmyj0GsJVggon@3dyJKX4mn&7K4hZof51WJPqcyfro1MXZxcMJ zD1wjv8R#a-XWDjoU)T6<+s<$-nu6>CO z520ynSL+``zVqG7;m!}YUNVe};fLmQJ6!`#J0v&KqAC4-P zr9aTUM>n@$8pbzF>ecHX2=*O+fL#3jE9AOsej-2r`OoFS2OpH#vuEoooclW7Tj>|8 z7Qg-NZ_BN>-fFMWeobE=J>;+>ZL+pfU&+LmL@{xOiCkMTh>1nm114xhe`lXq31{Ot zx4Gvno;0(DRB$Z$gaQUNbML+Pw+@6G(rqO@{@Mp3lZQEV3gFeT{SP?A-g`EI*lCiE zq`B%3-_KGw^0Fj$a&}tW;2wa<@48*WHV1R%RJN|Eg({t{wbTl9%gUY6elo`!;>2=iqAgEE+8od*wT#k%3^|Si(FdUG|L4j!{X@x%FFYqW zF8krdx-ZFOIq#bn*-bk1cCWTj;EvHh)jHZ&gZM;WC;m}mG0#XyOPJvwa!Q9K`p~t{ z$nE5@m)?^%KiD8+D=TajgFQrX980g#>CQ6Z*lWPPhUbBW@2``U>o&{Yy9^Ypz(X1D zrHChd@{IFz0U?@8>MCxr?;*-Cd9q%=-u{=lcgXEJIsAxD5{LAlX-ZnVw#FveFwu79 zPygFqUHi075TEq3y8#>u_RT(CI&+E!IGp@T)Irvy`0wN($nTJ!$OhHG& z+)Vw^F59%K*t42G8DW)GiGJ}BE3dGhS9hH#s?$CU zTb_FQnX=pNd&n&}{lZo$-E`AUGHcc>IpT;Tj8A+1JoVI5*5<jcJgW)u_ z(uUUy}wHgmyz z_K)i|n_eZSOyEVF{y$h_tE1AaDYN99x+`UbAC*R({5`q!^4T(Den&~v_U%qM@KEh4 z8h_|BOaY#+e_E`ep0Cp+{|8OQ{B5PB#snY|_;K|m&g1w%+=DWSf5XyqP>;zjY>2r; zM+!rzlt~%*HXfe|Z6L3;fqrPLv}Y@xY7d!KWIr+gF8whQHsAj#i*`N?574+4;l}Ot z_w{q}s{hy#1MM1N@$G|~xIeX+%80E{eXC&XTrRr><1e~aWRm;XWxz)D*)|{&Uf9 z+U}SU18maljSn`;ZbLUF_iwPnwww?Hc=Zu|?bY`-nyUMb+Aevxw1-UOs?EX&PIiF= z8i6M?F1dePtRw2dvq4v*ESDSq_gcAe_O)4GGtSK^e+gD){@+d4+c8cj&8D6A9XaX~ z$I2FeuV}B0DjRlva-^>9p+DMi{58Jy-kFKx1kY2pMGhR`p%LJO0j((6>~i7_*Q3{x>(vgAd$e`(qt& zzyWg6MHk7j#~y3rdt2HXE1_o2oGE|*``?W%7LJ_q#jnbMf!hgQp~i;#Rr*S1mA=x~ zLnl};>5@FN;Tod{b@FN_ddK>M7RjzhP~M&Dq7Fad#f*$_(i&z`Z&wA5m195|V*W|{ z))wme2Jz6FU1vl?s&! zUk>f9uZ|+#;8)0tNi$q0@%pNn!yCcb*69rSLjBuH-fVnYHE9Ny`Y1_3%{pw-A2LK*k5JNnpqiSR^kFvAV+BqT*i7>)_DaV}@ zT^hY{utCRe3ROTr+$>}EA2Y^9IuTVrbfl~su!HR@(PPa=QnGrv6t7z&#p`RNxOSa% z*Oh6y4$U!ZqhIW)RFvQt5xmkU!7}A6|d9!_^S6M=u(Eq#Zo`eGjZ7&}# zeLtJH?VLADz5uU4P``EKsdDF;<9s`V<-)(+FY|^RAsyJ|+EnMmCmn76Cezp-^a~;A zLcG2GpY;2*cxJD6@ozGv6mVD?JqvL`OMfs;m9Sc2#ebLn-q&gV%=CZk$If<+K7|MR zr8Ocw%Zp|Ee)`#X=!|jN>&^+5WD*6jtpAd+6uAE5_-VNRYrffogES#B!QUt271#GX zK@zVFuxhF5?m~pjY)oc5oM<4m8eDdQ5%9l<* zOTWf;sZ>|@HM`;Jnr6Rt*54T`70+p!wxjeUf`lQuuj|Kk`a0FsAIg%kLq#zayhz0Tsp-&C z{Y?>$V_s+D4>_#x|Ficc0Cp7B+9%0m-ysA@!j7!6!vpjO4V!=8?!|I1NLC1~&<7X@aqxP^l~Mir^s^!ThXrDZJvR!r6Rd)UL*ZUV(xk_-K{oZSP?{lDy z!o2`_;k>{=tZt0gYOzYNANHby|M9O61{2q1nz(cxC{`=vZ!5no;kRqjEFU;!7&f@a zUi;fOzIu8~ZTFD!lWm_J#+JOkHqD+XQ(Ye0eb4=BI}SKdpCL23O`JJ&80Nv_JgGb- zs`#{{@`RJsEXtfF5E|jDp6GB8_w|6f@3?R$t2& zauop5e4sIO=BD}ctj~fu)(`Q%ix%K3CV0gYs|0|6{sJ7rqybKdaTOWXy!tPFOVR0; zU{GCxj<^`d5-w34I3)Ce_Wf%mn7tkV{`4gi{zAd`r#S=8y+=6X&1Jb9gvqmHB4k*W z{3_2O<9TR_o(9Qg2Tl-MmVIb<=d}&?XaBwWiWT!()s!qo(>rJT=mi{nuGzggyM! ze0yd3Jp95Rzx?ywh(T?7VkJ1yZamc2o_}qge_&~Yaic>zw*!?Q%F_f(F^Psgp+3n9 z9*40CTQOdlpjMmovj;GNb;`*fu_qpXwA5{CtFzVn(1Z8eK6~zLM}F|bcKVrL^%WC! z_U#1uT`u`16KPyww4wfxTijPBT^3`P*XJn((G_HN9xoEgq+JqKqq%@Vbu;>@(QQ#OI=g3l{oobNsN?dh5N#o}2ot-S)d*+5Pw4?JJ4C_O-9sNNnQ%&Ue1k z_sZIM3fs$tBH?4{}C`9*wqw!TtPPD|P|NXt6`9}eI|u zUXGE} zCT`2kHrvc-%k|b<54s-_tNU;r<0k&J<v=5a2@1C>=0~)#MN(uhYZF6yYh8R@I^aPhB(?Tzcb#!r*=)6J&o#GYF=H1D*1! z!V@G;9;aQI&h4u_;ws);IcM3IB&TPGP&5cVv_wF735t%OXD6Q!s@i5+c!JD(_j?XmVUuQt z-gU^j|6~7{`z_lfeH38OI=k8RGaj~cj@r`BxPctDH*9HVUw)B&WZ$87?&{lFY+v18 znl0DgcKe<1&i-2N3$lUqE=VeQyg+Wwfhx_auAmpU(f)be7X5oA^#5M@<(LCRx2Zzt9F zIyrtpBqG8ktict%7WgF^Z?a!{;dwjmzYnv&-2bPbvx3k3(9A*m?`Ef;^;P@uM^E%$ zk{hwq5%SZMYgAm5sBN{g?Y|oQjNy&hi;eFt$QNt;$1zr(lVp5(M(lAsVJugfrGAzm zrLEUJGjYgXiW4F}x&&Nt#FOuqA=h5dv|K;Thm|UqNBUKcFwb&>D6%xYHkE^)wtk$%M zA2W$p<`yl&*BSAFwxAiMxS#hT}73Jj12e_gwJsG)Co@lv{d; z##@rVa=ms{;gkqcckbe@Lu4agc}}1C#7*VYrMK@2B}UK}!RV8u6Ae`Roa4LkqC99Z zH{mna5{x zK67ge2kJr#R+dNp*7DgFymn2Ru^|~)P)suLF-51mqVh_86wW}|qcJl03! z?({xu%vo-%0$Va<2?ji@FvCp^SDSIr<*z|pPS3%Y6KMRrukVM(-i$-D?P^C*vJvJX zPb==754Y(3&{_F@G$-x&brDXKv2$~^8CRMSkH4n)>!d!R7GIN5n}Tdlalt&#`5|AF zLHsFBh)~CWrJQsrS#$L3>O0l)bJ)JJK9&bF?eHz^^y}x^yRZGWO*-V%<(s&kc%)r&^2hC_&*J|Q>HwmJaKb;v|qsq^My z2fX))D3VVgRcF6$V1M}xnN?i=qdYQTc6hD|k2CSUb&7xD7SqF6xrNpAZZJ&=z2GYfbrW7Qb? z>mK~w&rnDh=;_*BR6uT4Uw+vLpE8oKf8s{>h(q_Ysn7k}X?B+ruk&Y}ev19)-&5?& zFMZAB;x*atM>;_M&|gTK)Lnn?!)q(rD(t6tYTs3jcvyq96bsl^`;QtHzXHvegL@s^ z_wYVLZOewi@ycpPXc*IxNo9XZ&hy*|gL&zkQmnrie`%gRcTlAQ_5c!);Z=Jr*2 zaUpWc0apf01Ez0Uk3P(CISxKM;TZ`N8~m~+uc3TiBAhd4Hpa9C{_5R;0rN58@($Yu zUuY!F*I#?ZuRkxnIMrTy=|!6hT&`?dfPD`6T*yu82V?X1F=JQ9D}du|{Die_)iJAK zeB?7)ENI{}8smJej}vBWm(OTE;lgJ&Jiqy55S}mH3E@wA^`EjcP;TkLGhE;(@pMIg z`Fr6zKKTV&X%dVL<&%zR(u1J4N58JNQ>0lzjUrk{pLtF)os>urwgCemK^g$eMzmbX z7Hv~zmj(3u5f!sO)Ynj4fB04xbvYkzbaAiKdGqG@wTbrR0+s=oSbO*W@3X!3IRIn% zLVJGdbNKS`bN0duQ?VcFt2Q5YoHuWtKd%pjJ!q2=BSzZlYmBos#*McLYpv_{qW{oV z^dGJwqrLfBtwZ<|?b;t-J!KxB&pAd`?*~d{v|fJdrK3YTC(V+T+_GUf8S&5~jxrrCbWKm8U5uBdMJ6KFVVp z2YZmq1X#!lnhZ{hyyf+o@1?0e%2dBK_UFKM$On$G?_PLrqOt=OEkb?K|2g>tqw)T`UrR z6+7VI!`&{jm9rpQ=k`+`5LSf!>0{zizFGRya`gWm=KsC&w`~6|?mivT9DSMxuRpx@ z%v*rh{BdhNZ~!j!KAytM1ySN%5*^SX(>T~4i@uQj+OH?bug5@^23{6B?F^DI^HKxV zn_gCqwi1!bO|_S?I%#`|>kMUFv}mzC^6+2n@I&^sH?Y}%cMBR{xIz0XuS~NGzk7LX zg#bo*Lm#2-^5^^IwD`7RVpW7(P-7KYfycR7XeUo;6@6?k02M&xI%6SVH zyANt&GYRV;=Qq#qX>sbDd-xD?_1E0_aTmk=1Kl37i|aeO3Ew72o(^#p3&=XTZkz_N zs6!sF!N(^ZaS&e1yw_Ew^6sEF{us>3yqBlFWRpI0fc3+FT<+9i1dM~<@% z_x+p=S#ABYJ^B$lZ1SfOL%3g)*{pf7)k{86SypJryY^_L7$jWIWd5= zT8{ci;U0o3xgw4EGsOAa%hh|FxSBT)D+lrXJq)kDF2wWr0@!dN7PWAD2ma7C5+BIp zzUmxbxv-@suvs>NKiZCT$(!EdDprzVr5h*e*fuB4_zcd8w3sa8L}}C`Ie0{&FUpdd z@Mc5qWy|k~9efJyEiX|>;dW1uWc0v4ZrlXhZO?u2j`PjSOTA<34d2SHoHE6@wR|=B z`?gJkZIxA4!KbkGfSzG3nr7FYahM%)>56+XlY_?l-v7Z+I9a3O6|y)Ti$Rc`h&GAN z6!t0dh=?-B&1lrnCQcrRoO!NDtGev(Z30!aHF?FsmNyHWDD&B5ZZg@#FKc1|UVWQ2a>gF%BWxI! zQ-0S&GXyy-Bu$YW*RMn3(>_c{!Bv=W2pH6kAPFvy{SU(g?f1TOp5KrL zN;~g{hwW4E*@8ciL}(gq+wOh1A7AQ$ z8&p(M9(Nj4%2V}G|Hj{++Wsd~8vTOVGjAOa3 zTZ;Sm?p13$ld2Ljw1l5`#3|1_ZATrpf47`43u(&p4fye6IDVZz@9W?7AFOn(TWil_v6DR0&aEO2l9KF#6aO2O7~>QdpOF-32Mv}dKl2r z6Vy=YjG5Ez&;$3h$N%y7ZmBNzclzL&U--1mnLEc${N!ihVX3dD@xpKGAd5ptk^BCi z9(ux_`uB@1$fi+3kmz~5T3uy=%f?@@WRcx<|6{i9Q#aYtMe!BXD#fl!&wKqp_WWOd z>Jw)*GOGK65x%0yuy9fMV57cPjth>u(w^v3JoM^dtd63%R2c0aij$v~*C*8@E&~@H z;OfU`r6&AffPUQ{*U%++R`R+&QGz%y`Mk!lm3rqS44?P7`8*!-oR?@L+9qDug3n(u z$>Pssm^|@K{?jB2Ctn6(vV~vA1JK;1Ik z50hs6;lphK+j2!A@%e)s?Y9)q?NKkD$;k!%gNCPFSdVcoPT>Su6IUqm6-=xeHxoG_*RV_!I@?DXUQc=_Z9frcS$!8Y7L&~Vs=u%)n!N6MC}Qiw2+AZz4? zJ>kUDy3eE;R;i3zXPuUXs==6NkKcZceRlUrZF+-iIF=cX#3UDwVVH>@jAJ0327^lz z2L(VLi#VX*4vL%gas&K%n6aOO+4`ASwmJjH>vQJX{4|^U{=<*85hF*(zylsN;8XYd z6@{$RXJrGX=;Whh|*8# ztbSo*d@xSnG3QBf(r`SNGjP-f`6qp@Pj!ei`JfE!9)5=2nVC@x_#{`* zV0`)ICq1tZ+LEU_@D>)HMX9%eWE z@@lWQB2T9tu%(@P0LGH9o@~2ql6{neOf?O%vEw%Iqg6aEdfgtm>o@kpbB?eJ?`Tzx z0;wOA%Li=sW9nG_HeW?pAXg&Cee5${N468}uJ#xEb^i1w@&8`=Lj%p*UmEIJP~$G4 z#zgjIk8{Z9Fd0KX0B_eg-baS|tY}on(mG9$FzY%@r8RNosU~q5kM`I);{g#x(n}nN zcnDd1F>Yr0Aj4N+4nJfcd-avrNI4L@_xZ{n|G>Z6e9|XR^P6nyHm>2Vmp|@nppta0 zb7h4JU3ls)>>0WUdm>LZ2?VJ>X};s^A659 z^|0tLyq?xl-(w&0d$MFJu3jU3Za#hV2M*}A6K2vGd1W{cALCec)iL(|!;gVH@dK^P zwG8>^&2RKmzbJPd^wDqyPnb#C#q;q4Y0|0Z(VeosuKWLmJ|>2*$Of)&qUZHRxJSTT z*z5w(9Phk{<8XNeiA$ezoh2G9V+QdN6nE%3IHAnPWNexbo#5Bn5gT6HW?=$VN3czsZJ;EI&w*!ey(57Z9)E2 zvks7dw1#uC1u*h>U4<;8U6W=m4{c6CQ`?FYZikKExIfapf4n1Wvg@KV=C%D|vuNH- zn>KZd{pHr{?SkXau*uD^OWD`Pik1z>tZM6y9%XBd8ewY=A8xA+8|q(UY^GY9vu9yh zLoY4xn-)`$sw3i*}h-+u}WzUj`o>3`xhA@1i4-NAc@}tk(}3iE+ml%R%4I zPB7YV3DIucpopHJ&y16<)5bXw6MrR5#)wBp?>PA(ZM5lhNjmrq7a-el$Yx(Orn z@-(!oDaS=ZbiBS_Q4%i%VwnUyP9jW{MJef{at%5J;Y9wYPW+JX6&AI6&iR|?pKV)h zy`Am8$3Ctju4sr0e8?bKmnFY!Qb;XiWK{Jn+tt_uIwW4BAUr{}CHxlcpA1R-st-15 zuczlpM(=BKaZYH`CKY4W>n`=J)Op-is4Lx*0O@}66USns>Ob`uRgwSCp7JqUW6km5 z=jnlQneuA@Nm7m&UHRd#6Q$=Wp*gc&Z`i_!Ap>ma;Mh#OAwmadzB+x5Em)NQnw)oT zIvCOwUF9s$UrPVUOL|&dT1=uingD6o& z;wg{O6n97hsr|$+v#DdQ$m@^iq9sc(q17k$-dw!639B|2pbG5~D+^^$sh4tdYa_No z98Q{XT;qotxG;sbHTY8+R6uTbbOzoO|!R4F>AXBtIHPR!Fn+v9=hkxo@GPu zOWN?kGi=dQkJ>9Z6#B0+)`q-g3mdV;jy81NjoeA;@NNr-a;#d&ju97+pcMFFL(gFb zLxO>y`diA@GFPI}pPdF19%}8+`V4})a1lm62TPAe8fPpBO?2aS)Y~Xgidar7zZixd zBzJ-C5e~S{Z)#KV#7-cCp_0EOUkyG7|AY$QG$M&}+vJaGC9i1NhXleRs4-+HDkxbc zf3A~S+Ch)H4xChq?8r;4i7?=X8*OTP?|Y!#^6TqTVK)GiFSX-wMA*`fJZ?AJXoK}^ zt+8Y9UVh{*STNIOOq*uU;Oo7A!^b=(XKz)HLZ94Y!Z_Os{deQl#$d8cUmB>DjVlq^ zpV@c&19HIUFC*~^FE^})9`UQoQvTx2U(&G&>In2*_S;4Hj59K82u`{9Ojq=P)yCQ2 zwKudO>u+YmHrmpfu#W(P><9PkHuev`=r7?TOh^39;s3qzmt%v*=)(R~PPY&2r3`}f zU;1u3vV3B30K3zVjIXzxmQoD8a(={fs;dnm2C;Yta z;4|eUc^2Xq;EOIe&+eT3`zmS=N%ygnKlb1D+uQ$aYhrtkzDqIQN*N6#sHn>^l*dHX zes%f=PEKEO!D@YPAUoC5YuWy)Uie_Lo$`Awq~_`O0Vy9tA!qVo`m=mm&=D&qmN5vb z@Ayg;(B_8}F2jC!zrMK}-J4WoXp{_nn4NUOF?P#s@bmE#e47xixX6R#cBsME5&GDO z(W~dEof@A%cUH5LX2UUQw#F*MI<@_r_{`b!YbMRqKZ!$kqneW26{CjlmyAsLVNUN& z6K7ZS?ihVCB~ovt79-?6xFy|ajO0WYmFzUk98xv>j!|vC# z3-H)3(|&J_lux7}tyN1b*Q~kfDR*rv+U9q}h*e(~vZ{fGGXBWfEYd5xwKN4eGP81C)p4xA$KWD8&PwkpCBlE5>L9&(v z5ZvkiO|<{oX&3wRy?1vg<6TLNo%f*4H?>2y+Ps)3`-r?Xyt*=FMPi|Nl#wdwcW=Es25C)k+1 z4zXdFMB^sz+*gPLkcU)X_TX(D+Y5*&{g8;vW~kzOVPt!`Yz-LbKu}&}cz)SlRUdrw zk{v713enXL3sBBYQBzg{y++0#b;}@_5~vF193n{pY z-$wg?;?rl~YgE^VPBS*TNAzL4vX%P`ZaaQW@57ud)9S*0=&#;?S6S5tk6y(FVZ+ja zSS>nm=umuGJI%n+#ppWC_^Ugzbet)L6X2Zmorj**M*j*Q{j<$bl#@)_2V80X($WXj! zzfwV}lMHmr*7pBi`9lNJou{Pxq-W_}u*>W(LyHY># zWEa7_Zp2k0SlfrwrP3U$4VUe!}L0G|6(ex!W-1O4ZHO?+p7ppCDZH z*X9`Sarx4;U4_*@a4IoDWoKKSO-TJ3Vn0jn?$BSAygEJ99@g&o0-Ge@Yg zp}e+{SK2msCZ(Vku;egeJlFE;q*HF-h*mHtSqT}UHgObc=>lU`!1{>O-5u;Mubc=8vbr$9*$iW5Oi}WrzkrXFnD9J^dq%k``fUkYx7`4_~o=#)Mg>f8x5CSd;nAZqB8XjFW7q3XYEoiC!@1 zgi8Fv50F*CcQR~}Z89jHH=oB{0fDHxNxfbjJuq1CK{)8;E?{<;>3*HMEC47Qa2T$^QJ+hv7cbMc+9+MtKvL=x-33}yspK$fKE})bZZhz-y zB|1r=rAYEwK^h@$=H;8o4(c03l>e7x%9zRueAFi?UHc117cX821VZ*D=8Fe575V%{ zpT`-WKeDJ%<+HU>ruWavkAAkoau3m_h303)7rL+xGwfc%qqC6s}c&=~d zv3ezlwqU9I(0kY#l?r>dWdCR&)vmPm=1A%uxhk!z5VmMemDCyieG#4^7GRax=T1El zd+jW$lIc7czwxykVC3`(4Q@k&~=9sIzdCv7g3XwzaHSgq|bvZkMckoYF`bg zebpyMK9S6I$3>k)jI^QFW=+WChaW`Wf5-~rE;;8* z)QaNbF|OYU86YI-eEFSM7Y4EbiVs5ha>-T8v35CI+9%A$Z1@&iXNR3^F!nCvnB+c* z*DI@`C`{j(#vD~R@?aMv#0O*gS~O@t=4DM7+VdK-drV$ z9RQrinl?O9IraK~D7?4-yJPsc*9GHW#S2%GBs*!ko-~w5W$EgbKyfWEu0!hVE#afQ zX40n8z9(^Nh8#ELz6ghy>i!*{69i*(J!{Uw~7KQ{5T zu7%j+_a~Qq&;I_%L$w5YV2bw^=Y9DM+iUOr{UaDy*MZ4}Qa3DbpFj6NrL=`*4t{c+ zYQ6Rh165i*J{^6IPipZrR4S*8(bFO&SPZ2Fk|r6#M*~k!fK-y=$%g^h9w~<}mx^Sr zcnO?nAM4Z#K2%;?i_8T5#_|-*CcKvLOX`wau)Xrv>!)msG0B(9uRLKEym&_FvdiU9 z%=oLgPzv{(i%@K&Bn-gtF_njGVUyf4ESIdED}Ul7J9O`g85{Z|mzh zraFS4XNnQR#2n$tpZTJ#zQ*`Awy@&ShYyO^v}V0L&1U29hib4TZ4wv4j;M7a|G~MK*YT)LE#AoV(mHS(Bt4s5X3?RK@?CIL~xy?wmq*P>$Q<0%;SL| zUI;zjD}SK}*?$CHMgH7bUoSM;lOh|fwyLu`e#@=#`sHx%Kg{rc#J*HHROXcup7Lt? z9(;{{#`pRK^SJ!6rViTjeI62Ie8}ikZR{qSd2{opKI1DzoDSP&Zm<6rdg$%{-DmuB zhh6DU(6Fbs-aUJ%^%);nl{keC?8s+fE6OI`onT@6ULh|9(af@Amem2L61`(%u9#jgn7KVa^Nh+ zCt%;0`X74+zbrrSe^=Pkf4r+$`8FSuXZ+<^8(`KBetD;m`q(Z*YTuJMrKH=G@xs1H zxP23X;zfDFB}s<-W|eWWAehIQha<=Z%;R2%I1GY_>gK7K^vd#x8Vw5e%I_*EE{IGe zoyy>zq%Berjez(;ha6?Q?YUo6??Gqv-@jdVtvz}Bt@Z}?0`yNM!-srgD@}Uwmux@M z#{Ry+KjZjy+v9|JoMkxZi-LK)x%z0t=`2AXer?Lr_Sh{q+dprQjl20P`ho@ckN{x3 zl0+MneS&chf=KlGfAs0z{tr+00kiapaF%?`%i~cR9i8#i;oZe>N=YA(87J)+kP}q> zl>oY|E~NK22uOj^-6BsD;CY_ou$Z`BpK(MyugHiz7BvX!#)J`Pn?#KEgBasMFZhXZ zixca-#xKBF@!MN&vL~N-tU(JsJL`KFeZ%JCOT_%-eM~CE{ZwfW>dh6F1XVorRpYPj zyU?Ye+l_d%AE47o@udHt9e$9<3;TF;tOGJ0Cmpg!&kJH z4vwa|(#35+sF2-g(q(x=s6l|8y${+~r;k0IP%*Dx2!;GM*q_adHX(!Bj+;`?Ds|=v zwR!9yUMCHhl+W|TAIUNvVT(L&Nm%}wvzF^KA@3*sF zi4S>jKf)zxPnMw=H)f;%#)B~X^jDhz6)6d zJ=(mW>YaoPjV^^ODJd{k$;sL(5Q_>yH|nqj6r_6tPhqIwME_Mk`Cleix-jLLf7u`Z zFxht7eIHwjH$r@?oJ{^g7rK{#e*Ai#6j?WvLw(9ClvU^&IkJVIXXOY*EswO-{x$bF zwWQuWDd|>*NY`17Y0^=Conw_H_<@QOsF(fVyK%oKIxB(Z=HZLmKlVM!YuV2nj=Dd-1Fa8TI@Yc#Y&H zy^t?jlFY23c(44_M%Z;_vp*#Zzo6gcN0hYvy!?}9^fUK0oP6L4$1CFbV3vx=-YKVj z+3vsZZu{#4_qLte6;MM1#6Wy$Z;c5PY}`8Q+DuGd`6Ly-XgTAB=l!U%)^ET->py6q z5prUyA2tT>gLi@ZV54vUiUjs8iq$RHmjpMyyeU|WS1b8?C0Cj(!i7{1)j4hMcQknD z5F3iUnTC%XWkZGy_k9rNVA(puB22V3ajz4uoM9UraPZFotWod*3`H8qcvl&D!q=cf zh`?-RDWeVcuWv84otpDO5cGO-yEM#pJkl%o{leBzBc6iu^9JoMuQws7uTTQCO` z-f7aC6X1Cmh>7))V^_CPm~`iO@WkK#3Y~%m=y+eH`{JOV=bNF*?*+++$H|k2UjL^~ zTzS-o)FS9QDo?%rpZ)@E=_Z^0#CYycu0-T9NVU!AGmyH+xsWNb<7tQ?Sj%NDSdz6O zY|u;kDQK2dR7;R0c|0)7oR%IcO3I2wU#EGvB84lkp2Lds`~K8?y>|Bi06+jqL_t(V zUnW1`H2e8~Z)?v_zSvH>V6vF%RQSuEUv0bWy1RcMrJv!;;Qgq-aK0D=)A;9HD4xpQ zr<=D|y)%n8Aw%vH>Lbn5Q!PPK$m2|V0AKW_>Y?JE1|>(%xAFv~h(aH38`Kc=Xi@5< zSeJn-Di`95nYaD^*FADyRTpBJr#^P=Pp`0#o%9*&hrNi4AKaH&r?=3b43f(bsJFUi zl;6O=KCQD=dCTvJA@g`Y?C8?FaknTuf-q<-`^yxY&RB%JuA5I(AHA ze;$BIx1qy^*}%brG2uPTh728QoQ#XG)W;0Q*%$mZq{aMHbNuN>A|cm|8JlF z*BPLV`d{hBzdD6lHd_J2NCB4YRi5gb@@IY^g?Vd2NamnNu6+ zn+$@wAF9RbnJGQ|3WgMvH`h#K-BJ&&Ae(`@@r-&|{AYN+IroCu?yz6$` zeXsrC3;40yJ9>-_5$`(DdrnIl&4XikQkDPkr6^@weXWYu6Vg>#n&Y`vM)oDhk8UUE z^{5hcMP}#%)9e5%4JTbXkr+2qlvgH2TS1SgCBEtklAqw0OcE8i;xTfTc@`bPUisgI z_AlLQnf>1V%v~(2(2RGrbJ7gX?2{XU4O&T^)g7Kp0d@TNwd}I1er1OrJP{AAa~f3Y zep$X63>R()3{iOV79E+NRUM3nV6OgZ9JnHdE5Fcnp$pcfN^m>i1AwnOLyD`={Ik4N zCEG}Y30on0w1x-*c`0*JX|lBjOTvijB9Rb+f}i75X`#`;lvj05Qkx{O9QLO`>5U51 zPqK0_WT>`ZVb5`LWq3w;qRiB@^gum&Ueae^<^ok3sJ71vIeM@B_TdvwvFDz9)~>kx zyQSj0d5rs*F}Od#hW(oQ*|3pZt%cSwepN^zZkDmM;Ru~*cL}+ZwHXOPKrSy zG4%R>=%K;c`+9Hx?}G8Kf%3aqR$Fd_;(Dv=t~!`H9-&{Vipvwz^_Pmkiw>)E3B3+a zMetDvsmAREnrO1Ui1UXrf0Y|!)SZ)KKg5D$XGJDnVuy0VjA8gL_Jy-2+T=4PN4gWw z^ReQdE9u^~_j`&@-AfOCpTxeN^rx{sY$>Kn{V7LKc!IP;#2ed#W3_Bf8j2^gFei}? zSex=1sk{WbS1DvdBZ`+;-8;rHHy?M!zw|0dDX3W<)Qupb2vu#dG_JF}Ccl9V^Y6N& zi+mj|uZRian{WDs?Y`%}uz&fCz~_U|6hLoipaJ?x=}VDnfJPj4;0&todFD!XQXkC4 z^6?58<#@$JhM|H4^FB>1=}WX*EC(I$=ZT);d6QY|HDgGl_8eC{B2jHWr&Zu;&_)KU z(gQLQz4GVAkYW<7>bGawzw`?EZJATXsq_22Vd~s#eXvTVo zZ?*LfcF7NaX~!P5pTCmCs@*y8m#%nS(z_j4tq4?%KDFgOPD5t@k~9)am-3g=fFWrB(C(0O1=_}Bt#k$TiQcRc?J8dOT41J6!$#F z6A@KlJLE;}DW3TAeC0K$?#+lK%hj%+mx-rDKofJ0UTr^-U_FmU-Qq?KNnf=F=s|5K zd~e_52|@9qoDr6#j9&TO{%3yaLVNX1_tl>1=3gY$b(r>nJY{$dZw@g+cR8E{qk`+d?0+O8+$0b1 z3mH+?>;F^&m4`rGX8Q7k>o*}N?yy0H-u|EbkGffzLhcxioh5F%@<|QF3umcOh%STW zrLp$fU4z^(k@FM?%?h9{31NU%Z-0*?Iru_fV-L7h@e5$m7+{UT=@up|l(rI>-L4(NFkTYcRVH4tW;e zg3WPP+DF%1^&Jh$q5H5^(B+TLoH*sy1zi$W&;;>)_^{Uk{9wm5=u7c~8v_=q>(<}X zy{D=M!e7*51*=$0%Qpq}pxg+zr{zf&jg5pc{>7*!y$DjiyiOx|sVCv6T)Jagalt&# z<2)g#P0AmJ!eyfCe{#gA|V$%ktq0WmNBBxPUZIv;fCQ=z zQ|Ey~4t*u(O+MYAKKzZV*=mCOf8@v)W9&&-z5ZXQtJa##uFa{p@3PI=+y7fP{uKu3 zVG-Rp5f**iWk~%sUFSTj)|KEm$nZR#7%F`|GK;giX#u@TdF|9yiO>W4y~dY>TZ|up zfivoJB?=}I9)9?*_=#HQRB2{K)^hj0Tf_2uodAzn`6A8*H$VuO#Klb?Tw!+TRW( zj2Ik`d-LZmvMv}kGEN6u(`wsV|5YboZ)bd=%&>xg-c(!g{Jpkl_S_7wtx@$ucnYHV z(AgSF9i`XSwAFDqlsWG!F{(%w`Aeju5BZf23Syq_Z^%zxrxAbN3hSi8pcbPnvqVeR z)GCdZ+N;*D;#ZBDO&xesypmEj!bYJb`n~ek`c74cD}()WnPl%gPw}d+D8CV0?nyHa zQa(_Dsw+29cPGv!$?7_Sr34l(Ju#6w%3xqTH>dz_D73s7D2EQ zhhkh8NkJMU2?H}-2vpw{NOD0h_$7{`d$wd?PPMc2%AfR^^p~_2!1WcXr49!U9AH1V z;wJmVafjON*n6fs2ilcEmsr9wlyO~v1>;Ms5SA0w3=}+@^sdsY^B*}IA*Rr&von? z?#?H71WJh``me?T?_=^A7oqG|Mc?<1$EvCC6EBN4?2D;?617?$9vj>QDfnfMk9J6& z`=&-f4Ub1nmih+(up)gSCKCSnQ6uQPwe}+5Wd0xS-SjKzFnQv^syn+Z)v>HuXInyP=_x&2nLel z^3+oOYPs5>P1)~wIGsWO&_rB5>V-Y{R3l%A3Tfoq6fBFgoXLd8`OzgfaYZG!N_gnO zKeIyThZQ_Q{FnRhv9;G-4`0t7fNMV$(!SR-$BbLWcH3$V+hX03HiSDxAk3NaPdokQ zj@`R8C|Ax+o7k`&kF=rdtnE@%N|BsoOJ4trz4rTW<3kVir7h=_{-Z8hz*Jd2*fOo2 zf}($T`Q1=4C}4l?!Hdnj3GVT|sY$Ck4>*kRVg)#?1HekL{&ST^(Q5t|03) zisyMXLq-}aB{%UHFYqh;IbOn1TQ&I1%!1?=WKkZcPCX>ZRImJ{j=GKg!%E9M%Vz|a zh0GCo`2szJzu0lx0YSvQ(FDbSU~>)af8b&E!z+JfL$L9Ew*baKIlm2fXyyvBy2F3< z3Ci<^s4p2kn>c!D(#VJQNOAZ=JA9g+YsjQj6!WR zr@?p!*gtPT11zD(A&Vg8avl@R<1Vx079`;+zj+>uHQ@EiU!x1EsbQ$-bI`!ScFASe z+rfu^upxSP$>gimJQim#kE^^2`SB0t5$18>;Ax<7|U}EE6p8N_<@E)4c=W;-&V?(@#l+PK3wVNA_JO=uf-H4mfp^-SkL&se8~S z2iPSCbYjzq^xU~~?1dMmx_=?PYkE$ z-l9{U=?Y&rtDPYJkv;D+r3qq>ckYQ3#Po4a7`iSB4Yj`A5vz=_{de2QzHr>mcG`P4 zwjDPpC(N7*wxH;W&~LqiZFsX2X2C(<(c9aYgFcV%; zOY?Z!T3_+l-lxEZ`U^QyUX?rVSHcyfUZ}GKeSAcjV3h{b(ib5Q(iIHdM_*Rm=+VrJ zzrbzT9-^PODVjNyyv^w#=gD=zu>}N#-z)!{!~VI?C{7<^XsxfQ9^rD2p6+~fLA33z z7}vt+HH-&BbV5g?i#l=v9fw??{O!B$VORZPvaP!6YRu_&f%_`(VR$Jn;DU_D19d=; z14oc6(-bFcbmTa`k_|Kq*arFFis};P@n-0SNrBi- z`zP|nz^{Rg`MGT3mC)nfbof$T+hK=*7Vs z+wgX>JjV@UJbRF=JxTt)xC#5pH-2Cr`sgVVqkBPIEa60caMGD^O@8CfJSLr)pT$)k zAMxNsHg05b5J!-4rMsolwbbZ-C;}p zspKD4}km3M6D1`p%Xy$!XDRc3h)>2%i((NOB{;jU#Yu0N#d#B z3$h@Ox2f$Fh|GS~T;Iys5)SixY)NBIo)4{f6+-^8Oi*}&L=5bJdEANcq;ZWfk0%7_ zn=p^7ex8@dRbE500sR6mMZ`N{9v2>EWgz4qaReLVM@`o1mA}+Q#Bk1SlE)K5Zqq!j z`gvX+S9weH3%nGO4V7P$&JCEyg}e;7+>>Sn&AdUz^}s}kgH9+I4jFmWX;rUVZoR$T zeCtED!%n*tM&=3AX#JK-b6c5!N*8{efEf>2X(w;6grq6s98X#LRz9qLac>A$is;zh-@h3J2r6c5Na_d+Wm;$D_u9(TMPmvtM| zugPOu;6-2dq|&PXm*+8_`+Oc}S)(9NC6wO_%I^hPzuJDU|M&L)WsZN1oUWVX<-WWu zmUzi)gsXl*$b&e7jqsP%2Fvm~5f7Si5s5auho|;fr-q00inx!t7;k6J>ghX(XKxwVoq3)y;|+|HxX)q0HCNZ2;`)p1s(zWy z4^|aU9v4oY$9RqIJHr0DZvZWYd<@UyD(^|KsxR`cyiyLzUhnI)0dY%PgpRnE-Bf(3 zyDLI>m>_=P`RDLkTRcOA%%QI~$&cUm6c*#_vrjzsto`P(Im+rXxcHf??bSQKVlV#u zllJ>3kr#=Ry5jlp=1YK*SjY`B-eL<%snuK{gH6+~ft*NwH z6#g7}{D2LA@QK&1%Q*A6x>Bo4Gd>bOjoNn!QO9|9gd~&OHpUYOTFTQ!@;9r$n1_gb$ua| zkb#b*Jf=ASl7r^q9l6^2}2;HK195;$?#D}UIgv+PfM#Mj&w*?C_&)n0k| zC0eQ51wLdk6#YgX4F0{0eucpOgu?v--!t7IJ=WqWzU~$}5whB3x=-?N!XjX7@qjGi zvN?z3gs&f|PY|is|EZkN0|5pe>Z-T@CkLi-d@!bxV%%r4)B3-+(mhYQjta_01jV)Y zH&+|4z2Q?VQ$0eZbIAJ`$MH^?@KYY~+&58I?N#aJv|5D^JCX*+1OC7^XLk9}$@(1~ zCZ6|c+i`eMef*c-w=1&-&iVdb_P(#|hMaMB^f{;583&x3m3Dk|)~s0|8V^-V@fFRQ zaWKr|wqwi5T#jENHnNBESgVXTXRoRswAudbo0Xup1mzxl2^wfyy=5SSq zP;~fU#yGzc+xcM`6U1LS)ZVvFlf?w_sdh#uelm9U>{-QwCLcJtcVGB$Yc+rssQX@+aop`X8UY5q_spk25-?bwqR=Wq~<;KFB`e$R?Q={qr@+Ps4u@F zDc&dKup0Nm9&(agf-Xxr@HLh~<$_VEF)nkdMB*l8vMu#2DbQeD0jtHgM2j8#-*b4Ie(j)?R0QTYrO%ZNrT=wg1|A zS6g$<@oc7`U**U5{seE@1Jv&WfiF9gALWsb%aX%toasEYQ-1O*9!a*b{-D#;rk5u? zW;y@Vc`9F#@^kX+ACLYGdrRJD|N7_S_RLdH+6&MB2V?UrAD5A$uD8jtuEVkc#qnkr67BK%~!5KzFKmFl4^d$COq{p9I;K2r5p9gP?$l6W0p> z6;JgWBv~t!R{L$WHV}bH(n${1%|mPK9}^0}pW>QiaPlZB$aC_+c`i6abDj~+`s4&E zIAoUMLBGUJ^;I=XvQ|p=%Aa&EJ1*jlEe6!P@BE$p{F=+Cs%{y$s*Isx-f&gq%Nt_S zxOP&%Kq3JmiU`w+H{&2xbWV>kFHMu7K7M`Eq?t2;o}C~u74P-`P;hVmhgZNqR`~cw z-|qpTxQ;?$dAy!=mPc*@nPk0$%x=lJcTdJshC`-Pq3RcPiwxq38VMysMkmKaSlm<+ zGVmJjd8tKTm5w;a3y!<<=f_Kuj@;p=E$uVMXvP0i_WdJjKa=_JxnHuo|7SPbWwhBk z`yOp4+qrf@hx1SpxP%N*KOoa7yM!rI7ONwl zGTLH)lFS4Bp?dRP<0Cn8H;t3uT#o94NzS%(95O@q$@la6#McT3P8@Z`tC`(pf|!#D zu@ywzAy5zD$hj7m6<=b%`Az)?Lb}VFnmOkK)|!wf+Z3lCaeP!?Q6DP(sh*t51m$z} znB708Ufvi-md$zX2*#(%q(!ghIt7aQXc0*ne!0T0Z>h_AUA`s7k(j!quad9FAA7{E z|HV~y=j2=L$;ba*!_wfvgE6VHs;xF=73^U?!RF4LYcpoduxFq7r#H&b_MSU`9*__%b(?(S6|SmN|)quVt{*+G1T%N+$wh|M-&={(2-Ld zd7kvMuv220pwvjzmv+QOl&d4TQvk7Wjuh{k_@Q>gIitSfiQD8fx${JOImZ%>{B)-M zNuO%tgh5gt;>u%baOHWCOoc2+@;N@`34~txJJ$Z}3^U$%-Ol>lCuD&d_`bs?+20?2 z02k6u{CT#f#l{KtUk2KrUK`M}JSSp2vy8av5r~E>v9OFc%JcG6Q1tg|dum(sQJ#2n z`UPjKN54l8>-B$i=idJB;~B?9c}v8T*TH2FV_6rCf5AofJ?A$=x*-y*#*;Cm`bk2M!|H&Iu=< zV|$DZ($C+1b%P19$@Yya9<*If+|~?V$@|FVN7@CG^0jCk%NAv=%Fj;Vd8sqh=@)6XpY7;2CpQ&9#0H4_DJ^h?C5-a6i}9sUPKr|#n%u3`-%64ix*t9guxcU9qG|ss;;oHv zvReKl-zhWdsWkFQ<<(<%?;P;)Q;{; z2^V6U5l)o-_=gwSV}F0hD-0Pj#P;5MZ`*CR-E6z#nxxc(!9$ zZ@u;G;P)PA?|8>MZ1c@Gw~aR1$cAB6S);IU;X-@*>8I`S#~-(Q?zzWpdEcMz#KhYj z_LVQ5V(;92U;FS!KW*FX@U|LVbejBNGJG0WXc$BOO(L?AxjU0<|lft~gFPuNQ@*000gcKe;}z4)r; zq~lJuJ8t`pU4H2WczyH5)KJ#~zI)F=dGaZ->dG<=q#9W3FI=IK(g|Hzpc?vO8+mAn z9xcIr&hk+BGBcA~7ckHB;^3#KBVX_Je`*Q=nvqUC74~Y6|LYp&)ImG?P;UP+UhQA1 z+vCSS3f}`lelwKE>se2BL_skYR*HsjmDgIi2X@A@yn>jZ9#IR zILmud$o0cGgJ3BK^KlT+5P>ET*|aeZzVgFYGzZ}ou1*czXo6T109*k#5b`~ zdmmbUF>~S5HvQ4Z+su#me|ZjFD)T21tM@;Z^h`*3c)08dHkL%A97@?2d2rkjOxemJ zji4EjX?D$ArQR-o;7#pvWfGqOetOma+9emAZ_iKtxApJe-wrzHAUpc#qix@P_q727 zTEE6|_0?B59=qDQN5b3H7s`OZyc}|1`7=pA%gWr-K z3xjrslvSwWSKD{;dDSXI75Uj-s3OMOsn7iz+hUz86`H z&#gJp2RK2>wbNo?eVTK1TniRa6dC{_PrFp1PLWSqEy#b)8$FkHvN zwAnae9#=UArNAX5eYPJkkB3uGbJRawAmXk|S5oAJjWYeBW4WFIX&<_50TT_mF)6qW zg_G4Kji6mZpgF!^d+blng1hhu(qt&)M&pItjI%5-3do$`u~&ZTfk$qW!tU~eq{HUy zPkrBd%Z>J%o3C|E=ci%Vqu{@f{}e8w18e|}_Z;{E`_*j^*%!{fz{XElCr9m?IF^xf zDY<|7GQs!?CLTg@bB-rbKbCT{X~YRV4jBbqVZ3Re;t2CN^E^L;iB1hY)QL8tzOR5| z+bNI)2stUn>GOiU{!dTr?f>!v>SBeBe{8&)2aab9g3Yo^v^3@kdRDsPP~Wa|bVR|T zgR*m`!`mSaX*Fm&&x?3|lT6wS0fQxPt)$2CfOI)V;NV^`oe~bX-1grfR zvx|hUwDYe?54*#+wbRa<*eUFg=1}AAAnqiFD68ux$xl6mH+E8kT}~DgW(R)G26Kxe4~uN(-RD{D&xR^nHmvD4 z^da^i`jz`u)OB4ZGSQo<&G=)s@)An72kIv-JLDwf!5~;C1a(85&s^5`Ce4{oK#nyLaE(w}kPH=E0S7Mm;$1gfV^ycBK5W z0r_>hiShgj&N%My!B;@W$(tbK!BarSX#+t$5(u)o+9LFLMuYSyYYAJHe=&&j2e`TO z=G*r!`kKA#9b4Ehe)eOZ?7I5ut8L1ZDR%C;=UQ7P%*bb>FlNjc`}oH{Zh!vspY7gz z@3sB+-{0>0(_MD>!MoWhCr+|y(_X~(3*lFkS#k@K=5op3h#AttA47Of8TZTd-EUZ^A)FC`JXc4R<9k4&8fRq;qGBoRS991sg$@ISEa$CH#^GD^~4am4-NKlksYE zAvd{ZO$Nk!<##=}HkQNw{E8X<=A{>($Lh}$Nv1AL`q0Vv)o466ha6@@ut6kWuNgde zh#m34kNR;Fmi}IL>9@M2})LXIta7K67pEv?*wM6K{UUW4>Tpo;ul@ zonvbUljFy)?eb$Fw55GR7oo3gEJc~;0Wv5~7=e$-EQ^fl*zWdq8mSyQm(5vUaYBZ+ z+TSaslk6!vd09`3Qy;l*C|mP^y2BwyPq=O?_6HOZ!#}74NE?k9IjVVb?Ja531o7wX zieupGX@d9$_^V{OF+p;k$0=uBP`^~4i%MxUWTJL_1Vzsa7-zPz z0I%PD^Q&L5|Gng!-sZ8#9&2Cy>Q{{uBb^mSj~;Djo_VHE%$;-2Irgn@eaj{t{%$)M zZ?c^AmGAf~oAIp7bwIgIvv@n@_rZxY3fwVB^y!mbqKY(NPHDr)nMZDnWvw$la4>!{(R`{(f;z6zt{;UoPgJQuk^js&i&>OY|p(9u)g5S{TKV!wO{^Hg1k>CE=a#-Jnutg z&DvPICe3O|MT&NzPbK>l@xsr-tAH$zX=Lb=u z8vn9GS+j#*(>%ow5gV+~E+C^`M8 zU)*L_=>xY9UPsiu!i2Tgt!T!Vj;z2nCC6l0pDs-!Uz`+?o_qc?9ugr4V7xrFb+yvY+G~vaH6qJ4ieH8wsI3a?gBd#B8IoUFG>c8!iCme}YW`FV( zNta)Kx$V63&W_&|C$2a?|NQgqgCE4UFOw$u%Ba8L8RFX)Uu|#M^sR7Ps?G)7pqy>fmk<|RFH%SAl& zj~lPM#?JWc3H~M1M}tGOq!81S(kD}JYck4IDupO zrNE8H(x@GI3EQ!vB#f*9p~P3I*($4yv19m-^M_Bi=kZSNwc@Fp*ACH;XHe^_?6M5klKMmt0U^ z3evqG^Yb`eOgwoRQjr%lm?GB!iBamlOa+dYb+P?JR>~lm6%Tz#R>9ygIOn-n{I@55eSrSdQI z3|xS8Bc^!KUs0Ji8%22odL?Z(AU`I|dtYh~x#A$Oi6@_Dj^g5vd7vb+m3XiJQ(x4Y z^h=#fhqV@<9vL!Krc`N>uD<@Q`0>wG(qm4#P7o^Y2K5diJBBZWrIiN}8{YKFbHY~w zJL;=uq)-mWiSa?|3ePJ%sv&*)X4E4Aua9xG5BB6(XPxy$sKelC-#XIL{{J@b_1CR6 zYd{uaccN8B568L5_W1beW)~dNVf>KL+H0>@$nE1Q6&cDjTvWm?L{uIj!!q>AVvfmS zkA&}(FTpMozk?f+EE>0pe&*v8N2~aYW;J$EN9BEuMy%>2P;@-h zNh&Q?`QV*{G)yLv!AtZ ze)F4l*n211C71uw_sMb{c|~5~rGwB-{xQRDgT`xl*O-#Y(H|N7;%*IsJ} zAAGPXbt&XCAt&tkfq_E~IRqb)_zqUT{nIYIZ53shd@ap(b9OB|~=YAzk{LON7Q z56n5f4|FgRgW?X6t{P9eE2JbJ$-qtkWSA8v0MCqJv8Y43o*(6mt4wi~vn1l86t-9v zevob*e%QSnXgbpV($vyuJTpM3uS`17;RK4s^B`Lkj6%K5mDea6PD zHWvHsjIhCYHIuJoHepXQ-uUtePpr2;SE22&)2`SQa#uU$b6>GJ`1JbU9=y*Ud-NfD z0!xRV!6&Jw{%4B4jCbE#3jDT8a-RaT89O)LZ*6d|XpS)%t zEGDrV;LC?tUv*o18(M9TwAA;*Cee4pL)B80u)X@AGS%+&|K9#DA5w1%NZ+Y?6#Y?o z;aja6{{y#spYv;J*~ipRsH2EW3JnW>Jw>Mch|oiIEJzjnwF)Aj+~dS!($6LIJY_Ob z*B}a&hhotZec6d}@n&;l{IbqHBl+dBjtQsPM|K+*q^8|&JFdQat6#Oe@PXItM66J2 z+TdMw)sbdLT-h-kk;+@Q+{T)4eW1*_`dVy3$O+azq}19*swZUVc>9Z?pzm$SR}!S3 z(!MK4kQ{=Lin6(U^5&#$w`}D3YDzrVadjCt8gIht^bI!Lq{Ao8y5j`#MjLG!EApWu z*8?3v4%tel7&va6o%pWRZQz0>#a>|p2ge?TBr#;&4eSd?jQ6sp0sZW?fBefXzIR3| zskZ@b;M;BVJ|D3``q*Kry2QrW=y!j;u()UU8)$uIKVYxk_M_PMsTIw5e@$a^)tmC_ z>{G?~-^2Tt_7n4a)bwmgSv77{W%Ut`igLLfF}j~u%DN)cg+-#j`j6J|Lw*hMuMga7 zADMKJ&6xg%zZ!Y+$tQb_6?x)@^WXT!H*AL;cJR&WCmp`8U2ySLHgS)A{euML&3~w; z{0K4ZaD4S7bfUbJuE_5QMILa1ObP2VG9*OV5Zg`-<~CxBWW$S@U`Tuxuu{LEmGH<8 zuW4e-=`oMA7&3b9H&{5puk(KStWWzFN!ML>UHjederIoa%Uc3}1)jNY8uvRp?6AY^ zrkifE55E6h_JbeaV53H_g6GHnP$BQFz@T(O@tACl{KS>c0P{YAA`jJaqb3T%<&Z$tfv=sbk?K{zmBlc#&5;3If__2+9TeqSu?S5;j1=h z&TLySe*u)Z7|Q_%`MyiTFuOT?_(;sgF7%aZvmd$F7T$54uXR7mA{$tgm9(v3g1q`Zhn+a)(FrEVgpDo?e&pjP5% z95`Xo70la=mSu;4YPC6hhP9J4RtYniiV*573D z10CCJyOTBHRZJfJaqz~Eznqp20c ztFJfj_*eD3?v-Z$(q5v1-JSzID$wRX=FZ(tjL+CRbKQn!Ld7@WbtEgwS;S^dpRp`}W zr*iT`yYirK^1+Uff$b^o89nIaz6$;&8}}HBIAxO+LLjOlSFP+sh)QK4kosW4bn9)l zw>$9$NymlmH$luV4Q{&m7U}v9Z21GuaproIKPNTbDj}bw<5#QbjtZ5g(ZuZ`s%VPF2C5e+;U6%?Qef; zYpjvBJy4|;2>E8xb=O^IAO7%%jj#QF^w@*#va5b=!|>yY%ZxqF3Y|Eg(slFlOC0qT z^&jRdFOTQts#}Hanl#IeQaBDABxlBjrNtV{RpqBHUFVoB(oYUncqZFMp0dHmP;5-z z+?S^~*znh_V(sw2*aCMU9CrXF*Ej)?JF4OVDQ1*QA7#VAK(4b8wr%vPlWJ^d5qL&_ zu%)=USi&1Hq>Dc`>+&Z95O`j+UgeLuk|pAS7u?t9ROPK3%(J=K(JT-M)v1EW(%;Q;fl*Hvit72qt1L|S085|IQn>O$Qi54xYCT9|Mwp-(0?)R zg8{G9a9#CgGphFI1lg*qj>T))t3{p6p7cE)3o&6P|HrJI<1N9jWgM5Pfp}`8TE?`> z2wM}X^jfO#d?v+k@{N6$eVkc5qJBWO-*W$wmPM3A5Lb4lFVZUugut9UUW--5ugcdl ze?hOc=eE-q?+Ft0%3qqT!|dN$3$7SE-TdVK$T)*w!5MYZ;6a4LMbuB2tnd0uz7+^w zH!$4U5KZNQOya4S^_^&>E9ZnIY_CSwWrmMkxVz>!>ia_C2MMm+-G0ZnchIESk@oTT zZW0{Le9$gB^@a*}EhMj;WH+64m>rIp@(Fw=Io!a za-eIHpm@k}dO}=$3rZfrLWV-O&>7-Ix#C$lCll%8`~hzJ|9V?Nq9Y>Rae{c$&9}r> z7Tg0C`@cf=LN{#`c){$ns=k%==FIJ=m8z8n7tO0=ME9x7RK}8ed=)hdlfN`quD;B2 z#RaKf#ha_&-SYoGdtU;eM^Wrw*=+8F5Rfy35J19}K#)rjfdB#?AQ8Xy!mTBtb3Q$@FLn9)c@@9D@vD1M65c@9aI z^Y=`tEaJ*r@JpRS;n+ds+3FpXj}=6`VZDv4Wwmw8UU?3aQ+T?I*|fnhDknWXAWqK8 z^XjBl5S2&r6b)mQSB_q6VlPl&_< zts=}r`+8m^3Ritq=@l=?@*Ge0aeg%&bKkFj%D+rSu%FFj+gW72yVT3#cPGjgY`?IM}V7*UlHkeFwZndd=Qr5BG#4-UaE&(FRYQ+e1o zDAL;28lO0->^kKk$@!HerKm1VrTuoJ|C5XqM!1*h($yXmFGy7!Px85~R&Dmt6&`_VrAeFSO&gjO*Xa?}LYY@RJXMc>QE>nC!-OC>p^~i3C~OmYs`y=nKHF zHV^3)gbN}*ACL)8P`o4~`6`Yaqa4&Q%H)r)ukE_qUUuWxuBphN^1BZA`{HRfDZRn< z;192~8C}m>P#$*eUG||X_r}g-S+j{u*&c0@SbAovL9e2j6I*yk@XWkRI9o0`{W?T|F`>{vu z^lKko63KE&YkmCl_T1ykuqgVGdkfvBGVZBUEd7WbyuI*n>KBlee8sT z?T0_UFSch%{YY2!7eAUU@)R)@gm$IHNLT`$R?7?Pl8|{AynqCGE?$w|-lwbg9?X*k zu@$7BVE;m%{{DA&+vT7Bq>UXr)?dl&ixXzjg#P`FZ+ydEdg&$m=}&)Z*M8yCcFGwS zy5GwtqOF0_Tadn{8K-n7s&haOy=;RTB%$L_NRYKDJW@kjkLy4z8jXQ4 z$3?EZUFR1$$(29m^Nl!DL_~rt{$PW>;uspzdiy;H#m}3Kd z!mN4ND0_X%RBPR2iYwvH;F@&Bp6&IQ9cYXA4@lgDU^-nAa2_Batj8~pb3D;@#51q+ z92b4gBgd6qh18WHRkzEyQvb5P+ThzQ_-a)%USmH0%;Rm*!uDH3zi+>TY|>WS`OVEp ztT5xNmHYqz_uKJ_Gdwk=f%Ce?NsVtYt!woaH5RSE#Jv)oWjLq!4cKx=Te$I7*uSH} z=CbXw-C|j06Wc7?8Q$^t_D1Nx<_tDt4*;GA47@ePDQ`>S0SG&_VU?;N@iycwA^4!o zh4``WU>lwX!?a?ed%>i4THCmdJMFM@pU?4*^k=;6Dw-UI1y}k%?Z6*xCFpj8I|%}b zAec^<1Y~Mjd6KiUxC6EI9QFsRaoncUC$;>-iFzG z5O%ZEKb&~9JZje;-JK~j6o1upyE}aVdCjRu*x5ahNLhD(_j`TC^5B|%F(Xx$mhZZq zkR*GNig*#!ZVB=8v;;*b$UMDW|0vK!Veu?SWeIv>KRx8Ok_yr*VA7pqUyg?n1c~Q; z86ULyDl=Rc8!_gluMY%1b&q{W+T+k$6U6Z?&zKwXcAUjP-ogf ze$&%3R07emLNO1MH%UF2dyc6L#)d zC-}Wfye5tTxBCKY;X2pXKWqX$X&ahNFqzQfg!6>BBE4T@T>5!l;Dq8M$b3#GnCKXn z=-oGg%U?lR>#wd4BJ1eGdZgaGcboOXvv%gkkMh3cmRoMYhjrE`t3DmLy7=asZ?=s# z+Q`0g?PYe)ukY}CC%=5?GH zJr5BC;mVW8<#A?{d-War+Q9WUu_4%(tr4`m0Bfa9q3crWU)VFR1LDp{aK)|vH@p7+q!<3kPO25l zw7-1Kf7kskDS++M$2Q~pV zbHhw@2L76J(a%6A#-&MbLjE}6RzSyjp=tvUOw1w6Lhzdp{&P6d&UTD4=c7!n4s9MX z+-9%8xy^dVe%3H?8*jf{o=n*_5UR0>i@*a#S+@OOVb8n{h_7t_ZiTSJo4Nl>X6WNU z?T9ePbFtBs;@S`2CSKI76QPbf3g{m_5c0=UL7wtFK`#So1tcBIOVG>2n4y5cJuN}e zaa`j48u}6+o8X%Np7;0c%U6@nKF@X@!?Mku{lyJ-RiNEC&a_LuabFtTu4hO6=dm?B zXvvY7=zGhE;kNe3k+$CGRc!1kqa0s-_%IuRRmyo-8y|An276@kTEg*61Nu(d#qp>~?kLWqn3nM?D3;Z0p~BBPpo+a2F1;duX^yMAup zx#=sm<(6C8)mL9#u;~xTdFP#HyYIfc{o?ljwjX}~CTqc~iG29a#V6D;>`<$}>y@I4 zc1Jg293g(*_Xsj6w_U0i+OPUkeIZMc$8j(hPcOp@VTuY&csea)p-le6p@2jfAO7Jo z@IC}v>RfR4$-Y|fv!DH}ZNL5YLDvtCqehLgAN=44{%YnqXB=;@zWkC;0!Cf(I>EtF z=2+ddVb?S0Z#BA>>bOC$FnAUoF;Hd%Bco5>T*Sb_nE(5HgTfaS>B{l%j&3 zr2kpk8Hslg zyiBBtgt95lv&c6EWqa9)@rvu9jGAci5oBVH*AU}Gyt`VS6cKd(Ri@>+bQSqZ{g+q$ zmvzt=v3i4_d|(uPQzp5$1DwmDJgMSMPJ=l4dnMT?J-~pF0mC})4+uSYH1J4vioheC zGAR$~9S1>?pOGmqX}y0#pRvQ+ceZisZ-B3ZKS7G643q7W6W*2bJZm>hzjP_{*OK3L z_GR|_1Fo~NFFuBkENy0A-(y!>G;)MblnqZ`!K%$sd&UP9^Kp|iYu-E?w#M4lF!Q(8 zI&OlsHsXU$L-A$L!gk0fT##lY9_9oN#N!mfvNF7=^AxAN;#V+B5xr9XrP!3FLmfP^ ze;st~Bh2yoTFA%=>NO{ak%Rm9VWu?p@dM8_*ILIn_`dhYh1lMlApZ0>@3*OE z?SN9p+c7^l-Yz}xt6h+wKFR*m4)rm7v}s)bX%_wJXIUAmewdb^t-P%#v(z+m^q`>i z{u$68fX%=Bm;7dPshF@Q1%$Q-r zhQ+?#74klN$~JV(0x=o%!2tLKVeTG+Em zGnD4_nqb|+lf%jJyowP=$WYC%MlA)a4P7px{K2I>Ci6L8##ZT_AL0p%kl|h~Dul}e z`E@ET4POOIRr%T7fmnIggxAx!DS8WLa2KyK2FqC2!-}#tn~PN@dKv&9SM zTI<}|HtXrf?f?GrI~$C_Z6g}9erv3;mQC4iC+uN>:}tr;8n53)9t;XgbNx6MDEk3q?b3w@Ie6>V{-h zd$nt*%twb!`K$Gp6&0w;Nl;}IGF0=w;q{-%p>P9dj67$>hfWgYj#C@6l(&T;ztZ9P6Luvty$y!$-8bENnOY zK)^zLakCAwHm!+v%avi&vw4v<&gZ1}Tx)DvU;{zZ(6Yz|qK!A;E4l+PK|TO&d;q_o zi63zWV19k#8TUJU319S_^(wyXxoDBSzF?ufJb!_`gqy!tnp*6@6V9-;$8G3e_Vib) zAR{Ns24ljTKYq2BuVqkXzLG*6D6{ud@ICcc-hNP?+7IQ(L$Y@F`#&%vkAnZ3%>Ucj zccuPipGQYi|3|zJ2VO_=Yl+%V4NQ%NbzYCQ7_jO_p1#1=;gD#;Q{Scmv_skk$%^A? zaotq@qNRl((bmZ}&Y496ZQ^W{yaDou0HEP3i=1Q{hKG3vANv37bC+G9XZMd;pVFm4_3JP z*EjV~*9i8(T&>%JEdW~3_l{UF-&UDB7q7K1vc>=Wlg;Dr4r~Uz`dT*TZ9CiQ@7Ti{ zG1l>V0}nzEv@2~_*cY)yB58<)t!j!e^#jX&C!e&XE+?HYA zR1wMcZVDQrs2?I_1kX6OZNxCv-}r!k@j7|XAxB#GC(U|yg7{8MF#q|aAK5me%r<)8 z$#%Sb6@N<~_yK^er)+OS)5Ax9TLilMipmwyS(e^J^>e^U*dH>mpY-Z!KXR@h9x9|rE-^{G#N%Fa0B47>7*3+?=iuk_Wb9KW+)VUW#ATdA>?&{w9>W(ciZ zt%iI21ele+w>kj=}`>}iNxzC3@;RmSE zBrdDf{R8syer*x%Lq2oSCwwmBOJDku>(Cz$r=NbhPlA5$TVJvL4?Ns9+iWuGaER+V zaA14=lY57+CxxTEL?0ns_9ltq2>Dx5=+UHE=~`tGdEp)#H5;{w2*?Cypg424@G03p*hHFLSta9HQ>gY=dHN0 zW@p5SEWe57so3}x@F`}<(vWIqH@Z)l>e{Z9ZowBe!a*u|V7Tjc^-7t=?oO0Q+w_BvOLiRSi zUdmT0`K$3a`6E2m<8kg6Ph5PtzZ)mh7c7{E33E)yjh=7czxV6*{BuuN&1E-y?TdEK zg*QsW-x$zNPQ@7753k8%@@)QV0R2D$zpvT6S6;R`FFtRt zJ^i%3h{>~8|9H1eIR0!KGXdll{d(&0{Jsl^i1D?BG8JFlutT;~x z1Fugq@=yzvM##Xt9+&tDFVYjAp!O^)%)xc7*o(Gi2$*a~ThY!H+5xyhBsrl6w<+ui zKaVj5#^JbbPS|Loz5SiL+I_#dE9jQy*h>?{Ws$epl|OvIzJ1grtb*Lk-h1j~`|4Fo zKkIzZp+{RIKc2*!YkoY5{+#rO0Z`nE=rN@gyNkJdCDRCZm@$+OW>NhdW&4P>LPc_Y zm!xah2YH!YZ7evEjkm`(jA`d!^5kL|zXt&VT6zoBg-@Y~DkEwMEZ9QEPONvX9~ep&`S$ zU|>Y?K*%2kac%(f2F-I`wFR@DON>h}0E=RVZ7|VBZZ^e6ZMnS-!UO%h9&6=;fUHRG z8(44>Hq~d;G3VeGDp4Fhx4xA~hN$e*-`krmK!_K~G zI=`Bui7^g9INQvDF`p{?ph4%5iF_Hy$uTxqzv1Sj1c44ugnMEy&DIt@Gezx??=>}$~e z2?YnQ$A0828#!tfta8L&W?W^)jsG#|(~3yfJK4vbFhz7`zn<1V2b^e@n8f1*RNP#0 zLXWGuI9bMaN!|D)rUj6%S@QzJy~(&~CTq&yl_!K9_)~Dd!Ug0y0!xxUQ z`ItE4#}7Vu(9!nIuU~2Nuu>uq⪳efyi&+8JkGiaioxE3{Rv3iU}dK1D}fws*E= z2EQo|XD<6J#CPUzEB!y}jdozDbRVTH6G6THUp^WfYL7}w>^rRw&z<>DXZd^T|I54n zYpv6l&b%cTT@R~3ZCNmv)~kqvFK=KxM5rCKBF@v53MfZll+sf6cAunol9G31R;G|o z^-p%~>oUeQo&$-4urht|V2n4oDjiS3k2~oTcIK(aVf7h0z|7~!YgvZ(Z$8l{%r<`4 zZnoCM&8lj-Y&9$HD#FMjf=V8U>kKE~_*;9^&AtAf{_CIY-+%nQ&AR_iTVvPvp`%A% ziT5>VH(%XMy=gdxb~hDF3p)n^k6Ee9q5jJfDWfKj(u!gxBGmKGNa?%BFgx|3`B4|C z4g^cd!`=}elZ;IBYb)}Ux-*RvMvMI+F&}>E;nLB^e9Zp*hu`_NyreHn=n(6L3F0d6 z8AngH8An%RyMF%8UG}gk+w6b|K|c5%iZK+&_VIw6T&Q17AJlc$D>Nm&h(09S2rv9C zHS6Mh1+fP8S&~k9dQKP35q*yOMA`#5+bi1`SNd`3qcJA;ht^l%K@+CU7RSADtxee0s48rN%O_XeEqw7=d+Cnv+k8xxG4P@L zs5RHNRn}U|M&NbNVR&f6A1BOe0=ss1&=i|+UZWS`&65SQU$X_TzG`!4y=b#vc;4ne zGSh#ve)>IIW!qhCjXe&q2HXemZ8B7LF~o>Il5)DB;s3x@&vD)7invxh&|hdbT>l01 zD}(K?Y7(ERnyz-(TD$O6E~mi_nK`3xN?x!39mWWD(@3YO=UCh zPhm68ued#B-K@@J&oXx{)}u)?vho75igQ6w(Gs$@xbP*$MYTt24W1mLCVz?}b*7tV zPX1W%6R9AHYVjzoL;jGRt8A!OapP)OJ=PGfdGThK>HZ}>z-HWd^Xr$qk>=7?IxZ&? zcym4!j?Wuop6|HRR6HH4-HYM*=by2A@A;*DVA|m*i%-%9yH0LXHWvJ}s?1-xgmdtm zlX*uouS6vH|RSlj5 z?@+!(Pd{K_UipNdCuS@4r!^wf+9C|pk$stilSq8xbRITeJ4msi^TYvn9K1sw^ zgE@(&iCp@De4(0;=c;SLJkIf6s(*S#Odew*4{e4xbzvLhMTM_X`xkG4^PKILPlgzz zGi@Y0`6i-}p1!6@GoN$=Pp&u{wh*s;;?<&I!$;U*M}5@3{-rCxr*@~b-{1FZJpK8O z9d+y}-iCeB4E+h)HrsM;JC*AKWfdr2U+MpDnU(DyW{^+NB61bMJTCAhw}1K|eNSyz zc#S`jQ6CR^Tau`X6LiC+MCqHy6?kfFyg?%!A%oilX|fHHlI;>xGRSUHJU`-!{Nj_K zcuSUQdCniK3Hd-E#sOI5f}3p=Y#VO0i5+8yF3E06b_h9(>{2%~jg#jvZ&y z4n7hO^saZ(etAGW55t49qmKEgPX=;a!iRgD_$}Iu+oxY@xBP^rjS%WRpzwHiI69Ns zl^|0~KaI1#e`I^Awy9hXrI-Ck>TTRk$yn-daLRG7e*mBbTMpp?FgO3_2JHRiYnh$Z z*kOkqW*1&~q22m}Z`sM8INw$oH5%7W+{>f=`D>8uf09{v{b_+6_|?g|56`scZ@vz% zS~lBi<2SIiHr>L;U`HXypO?$27mplpV_~k zc+^fj@kCo^opmHdzlP_Yd#>%j|NizJyry~4XRdKwI7cJ>S&x1VNoLQi@Ab_(*N=3X zICDlKCXeDGRo`4anl$tJVHVH_;s2wGE3nzN*Cm9aoJE zo;wE@#i91T{SUFf{OLYhh*u8saO)3lvV9LY)K(p{y01DRZ|?Wu5M^uPj+1JIEKH|d zhywvCA9N{>`P`Nr*e@@ecs>2ywpsqXKGkLk8CdVmqXYUXf6Ax0V1O9rg$^EsXZ#`| zC8rbe@PK^1UH{HIEC0=+KlO}Pja%(=pP7!0r5}@(3%J=9Tid(ev%jz08q5hZtfb^V zHLO2%NChzv)wSwtdDOpF9H{M3FSZZ1(bl$@G?Sh&Xh7Rdy3-f^)l?29fHM&0IBA%| zNjLIrY8qj~Fk!|?Gp=acV$$2}0|y>qx7>WA=g$w{`Q~-D&9?8bcfM;+U!7M>hNEv` z@K!)u=9w~Fb`+%DT{b`ul$m&vJ5+Nn-2Ro<|XuyfEugP2pQVFa^|t6<=D#Elc=><^OHP!8qZh=rxiGq zBObQl3&H4z*cX5BLr2)(ANZ^N-vfV1nU)_II%KfTdiWv#u~hcMd=ftdHwuICFrg7o zYu3DAfi=yaXN%^{v4wMHdmk$;hT`@1h4bh8 zD&Z!q`sWykyu7c+1dH2V_2NP(Cm>CqcXqmAN)tEBr~X3>NhMzrln*Gqzd*?^d3m1y z!tiwAh+@cyoj z=(X{@!j#vV!lz&=PMGnpb#bfx;r`$F)*{Cpcbu)d>Z*0P_F~kWC+%1FK5A=Beuqt3 zH{P!3MeYSFPDCGb%rQQ3_NzPo$M)U-(6~mBA{HpY5nu=S68nF*1t!W~xaI4(XK1j^ zaj!Faopn9k+as=3)ljXApv%Sy?J}f@+;`*wWVCd<%Q_?=$W3-~PsC{p>$& z-Ge^j18&{NcpbvW2^p&8Qg*+NqAq`Y+kZL@C(V{+m^151d*s1?+JjF$kNY5&gcZTK z-miw$cpI!c&c=>i(?$*L(KJ&b{=WO}>l1)K|Jjf1ljncVA3FHv)oR)_OWC{&JCQ-u zFY7pder@soqrNSRLhB$s(#9(>6-;3mDssmFa|yzEQ$|)o2C`8c&)or+kl!;XBOybQ zg($8&!82@A;3BD}dmbJTqU?C8OM2?a0h+_bVY=3VF>B46XSV@pnuqeWuT~q1CD%i- z%51^#xqdrUwvsDQp3GDOSCyTBuVs!HG0In$jlkxXzRC=*6zJ7r zR({{9f8JEGzLFnuV!4$Y@MFRYPh%orzyQ6Yj>}G!nOM)vOO0qF2Kg=_Z&bBHjW=-I zCv61w^jV0H0POSLX*TPH=kV$7pLxFgaM9Txu^a#M=l0f#TgJ^ER>5)No55vGkmq@M zJeThgdZ@x6iZmQUuYgKY9M6TP;u%%6UE(g!3hw{Xk@_;I?;@@=nYpw2U|!aYc#xh< z96G;he-S>`-}*nO(x(IMo!3>-UOJQ-^`x7wG>KBJ%V>M7&eZlQ1YRC$zo_ekqlUfI z&eBOFTH?9%Y^L5N{8)lYA!Q*=|%Dp7E=EK1-y=qmV&o8r;QH4?Mx@^M}aa+Zb6El z<2;wHO4Hj@f{F*FziH8ghZX;Q>vtJ#M;vj4!)1B+^NmN_?x);ow%>Pb!H*8LWjAeR z_TdkI*eA{Y8+&~1^_~x6g*NY3xI(G)p%~xe-W^|VUik70)`k!Hjah%Zt+MtyKC#Fq z7#{_wV-=+(S8_VVk(}}Fu#s1FMaUpOCR}UN&F#tG-*1bad(!VG8XE8x3imtYJ%74L z2)ay+Q@QjF{Jnxz+4tW4bK7#uEtlDc9G<-YzwD|@FR`m`>wFK#-3~w14m$7vd;e~4 zw{=H#f69pyU0gYM#T8dzCEZVL>Yne#LpZF?<)7MdvIXr*U(Z89gUD(fkWTfH=c$Kx zUxO$G!g^@|19iNRS4e`S4}n7*Mn_zn3wOTGj~9+op(u_Ql@vSzc#5dEuzV?b;bk0Xz)#~BM z<4(1=;H$6xDgj<6Eceh0A9dZ5O{BK;7JQ@hkJTBzFcWnKCv2qSBG7lSF=)6kFFB}F z#1Td(8f`jms2ItAKj0*APQVRByB*vTtJt_FNpnk+9e(sj?dhi;x5poS$Z2($gPY3f zrygnFz2$Cv-4kDx8^jyCNDccWzaoFim)Z;|BxjO7MWh6$kG$g7j=(i@E@bEW#Ur0u;I;B0U^vYZE>!bA_-q2@{v~k!uCQaDp<9f>a zku!(*^m)ITfal}U7mksx8w?G;m@ghML^}IS#`*Q1tT5T))tn|>80CEh5rovlOPCs4 za*gmC3>zdK<@bCAtoW9v#)@;USmo=QT&=P?_FI{L?tj?D7kt9z%zoXemdAnD53U5_ zs`=6Q$;vJ0A6u}BtQD`ma0NbJTc=7K7cuw)11QZ0lI%yhA^ITb6R*O6fW$Nk;G*=T zpARK{O*uc`J>$eagfF6P!Nxax?PtSmsOW5d6^(IKZT)F6b`NB5_V*-tSe%hHA*`*gx$0`RdbI1(b zkpO)lev2))w!@D)5$!d$weS^YTrG(kW8N4tNN@Q_9}v6Gi4ZnQeKCBGEm(HTg>~QY z{)62J`;ja)$Fu0Zj!I8K%A4c4{8TyAX8WZ2p)_&6Ey0^v_R06&{R^xN8f!c4v{RnF z_wj{)wktVd#&FwV_TxtnuxT4DyJ@ot6DHUWJM3V8`onK+{=7N<5C*Oi6KAkRVITBI zZCItng#!$){`*P$*WGv8xNWw_O6ge8;tBLMH)IFS1K!0e8O*@ToZ@&6+{X-zLl8da z#TV_d`+kGhE1R*GIc!>hJ(@Ig(Lz4S>Ab!F zN~K7JT>c!-g^A25Mtpgi;}wSO9Es$hR~hU`JWD22P3I!#PZn;sF?8ZpWEUNu4kGxm z^lOhJ|Hj5xUe1dkCl|Pf(qIf~4?W@|cI9W!b2=S30KD?@bM5+@{>NW8;H)w?0c7P- zb(cR8SHyK)Ng(;0c3J9AIVhBZa6!aN7LrJILWLYxTmi*LkO?_nVaT#%6Owu|<9LG5 z>-FzUopl@}o1m9jrdP!E1j$xXWtjK~GAYL^40}U=t&-$M_@6%i*w`ICK4^l$_FL43-O&8T_9mUlFLvi_+Af(E5*V+`KLAh4i2FWVh1S`2YBU0Vg7 zkR7-gcYIWUtJm<=HxA%YXWYy1l+!_3o<#wEAGnNMced~>Opx0nV#bnhP~ncEA@wg*@+@7 zb^Uv;zI)KMo-oIAiK%(GRT?WKhx6+S@MId#QV%cV(0a96$*JkMa#*@?Eq^BY4Gc*< z0IA8ZcqJ!(X;O}pOq}pvym&CmZNLQlNQcyOgAF&fv(LNCK6CL|_$bMo3ZrFn+QyF} zQ74pN^rt9_cq95-PK5FGjcTYz@B73yMIW4}w_+T{K)bQ+yZzk88Y>afPPBaBMW5>N zG`7-76?GC=ZT+xb$*Zhi>O#s0!Ues=n)*ximC7sS3Y*X_41&U`Z+{DU9Z?_YBq)7K z8`RcKo7xv&}%R>)7V6VJ9%Z@zq$fE7$d3zIo@RsXsM#k`x zYwxlTT)DTuX_rO!D!RuWd)WQ=-*12T{cmiqefIaK=dv9=fH?PZ=Gc8v(?Tzsc6x(`Gz+09S#k>JEIcHaFA)~WMAEtE z243g~rW&{hJrj|d7>JLcl0udsQylYGrK?7S@t7|JioDrCiP2(0G>M|}%0v<|lo_}L zv=xIze#L`NlQ?tyw(Z|(TW+nGwpFML z&*fm9ZM+Q~H&@}&7TlbhG;hJzc254p1$OzxXSj_EqXMD5zkc0k?Szk=4O^xZWLoL} znSo{O)S)HdbwG>J#v{poEtG4jZ z6a(N);uJVb7sGbGj3oIhGsH@$(21*XIH|yXXehqky=cfHy#BqJop<5q>~oi$hZ~}o zQW?t*_@qTz2_~Ig&eVsd=MaU4h54x$=f_fnaED}2lTJd0bdv~-HgUTsF4b}8^;OoKh~YFsMF+dt^GWiuns$ z24s@=yY&C$1BxSL;DaR`VOUo$R9@pKf2i_Ot%*$}{xA!v+&J zwlmM>R|!YswW2hJLVM*H%JqYa9qHc()p^533R}rxDt%{?PS=5&em_fl!X!rsYw){;;psB+k6*vv&76d)w4C zYNC7fgnE7EGoQg%cYkZUzk6S_GfY5bd;04t?E8~r@`+*0Ti#-$Rvlw6J^!4&`qY#D zV?4j=JYu!gY$)0>e_Vyuh;0N0VefLjdFB&1nCM!}Rn_3t2FQIno4M~XzMi@8_1FCi zn)6?IrFb~c>-VawuWoCsGuEDX_(5w%U+5na^J|Fu1@@1u%jjpr^Xc~yIMKEk_i=yz z<9+_+(`~ldCZ+ToJo?|~*v9)^S}?l&wtw0u_T6-e6P-iHtZUm$o#wwYFMHH}eC=|)7s;|&_ z74~S-EUuAJfvU(TKygvoF?dkXQY#dyHe?@ZkThL@11YD`x%o5B=p=E%?;*i?g1y zPhW6~Dyaq@h1W9ITyq`&G9)*x<&Ue&_;fTnn2L|4PMu8pQ~kRxa8GyD|5W6FTUh?zjXDdZSzS}Y?s~N>$WVP zGnZ7VCy%LZJ}cWl>jgF``{P=CW&a2JBpcCxXeYHvo-@?gE~2s)9NANJ4Xxt(6I5Gr zn6+u@(9Sk`YDtI?O|KlJMn{nn5=BgyMCYMvo**q5{blfLH*o-l5f}N3RcEPum%#Bz z=lFtg=!*TOd-38SUbh^yx3o6~u3MXWcx6vo{V{G_njIxNAl%B26AEb-gnc7(BC9P}KqA$5WA-RCi z-6wSTIwJbiM;`iz>%RT=+b>faJo}3q>?$u|H~Z;7FSXk@+Y#q?+NW>4-=?0m!!j#- zyY04f-v4^|A3llIh6OFu-^cfSEz>8rIKhRj1kk52&^<=sLpJR9PO|`e zbXw#zPoI-f6YmU=m&Mzz!yrZ*BB@{CGc^!InNKE(Q!9Tb$qLzPA zR;i$~^}L|vppMI~x%uE=I5#&=pRnOX+ilPH+b?ggc`?}cKDv3v^}ed7$j>Ki(wULM zbDeX%BHtUSzl)n>E6pQe6LfO|mw_2gTx7yKO7}u|NMWV^UG{%}G6AzSV_R;~Zd#*rl zPAoDs`|{y7+jE}-?B7p5W_SJaCmyc@>)?~89c|yc?Ot1d{D$Z+@c0YNc*Edn~lp{uK=RCPICsvmhlEF6eQL$Diw+ zV8oYkJZGyUWx$jNapmAljWR)zD%!K-BwQ09@~W*Z&i{p5KeXtRLCHr-U*G^1edSC@Xj;POo#M$@p zRZRJ>*J>(@=`8=>Pdw`RCQh6fyn1|m);@R8(TNKtI-j}GrmowwX``NsB;B5P{9nGZ zuH=WSKPxAN7!l|!>a#rSKIZPy?d+`FqBVjGaPRUH%yFlymkuJQ5BF>3$lFkUJC!-S-ouKEa(0gf!i_$;=#N!k}dg6pR zPC8y7VV8mU+6`a7;FsSxK&I{ZROv8$7+}Ii?Js7tXTN5ro_LVWnKK(KAu}$ZW$@BN z5EL9KeN%th#pO#d$K7W2(m~{OJUEg;H^NH&-3-ZqVWK`z{~=VLKdPVec7?X(1Jnd1 zJUz@OTNsbOf)??_Bc1LUIAP$dhseWx;v;&N1i4Uy_wDryyPn-Ro}-bH-lh~*TkvqKu5P-Q8!@&XX1zOYsBJ= zv`ojULR??@;|C}B5Rcc@p?E!td)sh)vf;*?*aeq-!FJwt&yv*}gpC+6!cIQz96S7o z6VU!fd7B(I0$-GzL zGK?-j8xrc*+KTg*=*ws<-`g1DaY4*lIgVb)AB^+5y6S3Uuno&vw&ptP*xF;q+B)lv zvvtRfv$X-&T6aBaCq{+KW7)DIy?RcU$ed6;0ZITBHg&6j=syfp*ACzdXGE{gDgks*8IpuM6itD$wCHtFM_Re@6OKQV#U^33&RYF4ms6BmeKC z`a`S;l$&^xzo+e=AMjw{=i>*W?K7~WrmeV)`jj&+whKOaV!ICmJo3=r?ZW9N+UKwP zPP#FOjX%BLbNNERmF?dRT2vnq_3>`(%0X%BKikpwa{WjBgiD1eD|r7`v3FlORl5^Z zeqHz=^KJf^Q$ZzC%f8`st8VtOaVPZIHQQ@bj0O+JXIXY6JBmiEne)s@9K|f^vzzW3l+2+A?23*IbPe|Xx!3o3tee@_vH{+btdR}P&;}Zci_Ol zIax+OXV%8STmcXN$N1!=kDX;ZzGGLL@y##WbJ+XF8^97gvbW8|NxNwWA7yJ}f|{#K zM&ag~d&6>toNxS(wo2Tsk$kr!{kNU`R(tT#r|ijRUwp%SgZtNRH+d5qx9*1a-0yD0 z#P9zCz{?nb&~fZ^?Zf4TN9J;SHEX>*b%F@aF0~>9dTh`pA@6IxWWl1^gy6X z93#A6LnNVxb_9r398l1`@cc7&=Nmin7hWhVo`g;|?r#MT`iJE#}SKAX`nAu|Twmz_h=IEpKuX?7D-!u}DX_C!5Y*V0L_)Fk<~n*w-Y5+EE5mJwwZ$CMWN=|g-z#cd+k zEcgK)uA1lS3%)8vHj^e#v5POg#(wqlpV-g-_g1{Y(SG}hN+CTyoiuqH+xLKjZIg*x z_!}nNZ;h+seD5+q;+%l;6?0UXZ3O>*=s^FWcr)eLgZB;^^$oS=2p)fE8vHfxK88RDAVDBITA(%0~s6th7o&{o+*Z zN-$Lowep|hYn91u%ofBj0IwG|`)h`Lg>hMhg%AFZ9i4UtxcpNG)NBU-t`p8TyYxU5 zeU;tti&O1C_8z}X#E^b-JNcUQ$l^6gx;(GZF(E@(h7yldx?+mMb=KPpAp_eEm$DJ^ zw8Bq)CLsfl?8oNLdEL*ubC9mx$y?-k_D}Ymw90J1uN-3Ii-b28k#6(n&5tn!3`TF3 zb<0knlry(+j+Yc&$9gu$K!U+;_#5VL0p(v^@Q_{YSgpL?vdjTq@1XJL!wtAAK}F8-TLPY4By9Z#f9=)U;m>0;r_d8>VhY0tvwbS*MHPT_^Umu`0K=cIyxAy{qT#h z^hf#~L)ZNO%~bzk7*3Y?&FYY_6DQI5RAdnb&66g-%?^cqzx|D?BQZN(ch$uCVtc>-RSqP9 zywR7V6yWmH9H)f+A*gLhcEakf(8DK?b32Dk;;~2S718ZDj*GV0_U-oWefGol3vb1W z{K5WOH1{pz>M~zxhE4yuElVRN`Fw>LMCatT`y^CeTKyxR>_|2VcbHVkOzCw4-zQ|C zZxah2j~M6a?StnOD&xc%H7hc5CAvfMZe8b04h*^XEG0$dM!SEWL=g*d13MmiS$4hwohbnasxV@3Ze5 zZg$uV%$-fY*-qa3EL*o1OBXA?F1<7%R+?oio4c5~+E;bfHoC}msmPXA*LuufeQKtq zIWzmfE^n_;mh%gCi@k&@#tAXM8-w?K;11cP%s7bz(R?q!X<;C$7ku0X*0c`fdBwN07*naR4I;?@@ixt8}TM& zAc4mv34|Bvpz%1TZht_U%3z{!hm5GCSM`(Lgs~q3_c+6a6Bk-uJh$0)C)<3>DfW-Q z|ApL&@Vk3|W%u3lOMBOD`*=RygnOuzS5KZo24*0E>-Wa%@AA7LD3_lt{b8m4g%8qr zOI!cwT6~4mV~;;%U%uuu?d;Eqo@24nX~c+8{@Mfg=c4cVAT`;Zs~)@K|6bUetp3^N z*}=E6&12BarNmrqy64{e+mqOT;O@JAR#OLj!gS8*$JqC7z1PO$6ZN#af8j?|EB(Ll zd75Nt{lDl7*(Wk|=8s*lw`8ctPu-kgr{y`5B^!KF_g zt2YEl2vgf@V#6uLgrte{g7> z39+H}{e6+N$7 z;py$^l~?I3v!`LHb*)q2i>+)po;q>RH+=XgU+u`jAO{=8fNy2}hxf{t zEA8Ju3V;SbpbZX>CnIeCd{vJ&L;-EM`D=iRmM`CE8Jd7*p6x)}f*F}w=M*YOFo!6~W zdYPm2fHJ16pQu;W4|A&?3s%A0UxD;=ocounTe+{YHrg~{RP?B!W6+Jc4iqdjmnCO?9NjrjR7jWyT8M&Ij< zwQ=ijfc)dV9RJ7{+5#uU*si#GlOHGGb(^n_a?^jdMSd8FuVvC0?3YS?dZvG2T-ra` zME1%dEgH5<<%yOMp+^(+%%9pJVHrxCm~>`(dwQix%G$P7n@R?Vk;Rp%tU*zA`1l&4n1IOENT)9|2>;X(zM)?rn) z`b+XroQlg~RUW6up(r23;^D07>yPcsJow}pIw&q3=OC7!pq{YdCN}lm@3&vxUi&%t z#~*#jZk}!q@u@)eEf|p-^ZNw+t>0Z9#$^8z8!vBz&c=D%} z?Z3ADv)yy8$Kpmzm|?XC1Hbac&Uxg>(f08(@nHb$D>nxlvF72ChyKql{NxFC`IR?e zK!eFj85p+qj=iuy48OAduk8QcsQy29+CGSD5JboyZzcpKE}If>E4=SYkZF0G&U0AB zqa%!19-~t%JI9zDfeiRl$nh$D@MFgs^xct<-A04^1`te~F)YSJfKNWb7K8C|65jCh zsVzTQ--L~X37c`Vzzw2#-QYG7wA(7Hg#C0K;Ut7VNJ%RdnmEaTVz|22Z`zVi3U^2W z7{3&KmMqAdl%{050ob#R{EB{&dL=svvZ&J1JVUu2Ln!*PKDuMyysWY`m0GVnFM5SK zEU9f(6{U@HJD2v8%>w2$h_bKPxuoz<&j9V~fp?@^uAt6`K^8-eFV8}6z zuTsGNdz?_?q}bNmY;Rj_vz^m3PvNJ)-B(edRwr@%PJKBc#<3A0SCch>KcVkmiU~5_ z&~Xn~J|yFWgzp8*hmQDTXs3_+Wu)*ZT|1<&>R)Ch)`k3u0d-W!@rpb_Qqs1Uk7brE zAh0r~J$^R-*wiE4#4<2bpZQ8rDG}7;c zkDr748@%F|?rZ4wv>hj4n(*+bksrBeehG1i$MvtB@4&Q-Pfgh;oN*Qt&$3BPTT*0w z$h}5cU4#mwu!8%zkDi5xUa=3NZ;QhML8zMb$vb4A+!TfnwjTQD19r#%-qOo;`Le|7 z6HxUyd^i>q6xQfmv@W%Y&e?EBfNloX9DnXD2|9?1m!l!7a5~~Od3avQ�Op4-srf zALRujoi_PYHS+8Lmo<^LDlbt>S__8NG&w;<}PQBv8?kOE6M1ei%?wHw_Mnv%e7> zfUyquIn$&T$0D2n;~pVHx%v#RJTzf4!69B{;ba-pc}^buF+?0z%41a;c`X<>(VAz! zWb-iLHXkI7n2g{$w}GH1ha=Wc_^$;_TDV#lscoai zSmRsy1+Y9GIdUQ6&Iz2{6%B%NX12|jOxg~!J|B?t_c}J>T(jUET=3(xha=q zQynQycGh`+fwFt9n{<>8vTbL!6P+OQ=Xlk&oEU>=_#_r_J{%l604osy`D!G$nc#;4 z_?j~PnX4xG!2o_3BmA6u<+1(5%1X+|n^7P4Fa@t2JC3UM2{S(U#snPqDC0vg#&Knv zkD1V(_%NJ#c+TLjEt5a~{5;_u`HtuWnJ34q`bXm;@6s2E=Xr9xN+0}~wtozy9^FYL*_KKk(gaO-n(x+i18w z`RJoof0r=4zC3!&2#EoU0zz9t?5#exFf zf49xXM{yp03HjhfJ|R`G2t>}6p!e6j7NzSWqvid~fFz^^lQCzl|Sv1|8&jG+ft5K^XKQ?pEMH})w5tbj{%|Lm`6CFLXIm=uu9Kz6h|829r38p zyJI8HA@d6-9YYm%g@2x}N>6@@BYUhA<6yP14HFW$iRO~-VT*>_8u+T-{?m@M?|$n` zq+G;yHhj2o{BTV}`WuDsYb zpY%3+$4} zFl2Ame{SnOhFAM6p69=+xNM8;F9{i{u2+rGO{iexk)zVgRer%+^i|jyy$e-ysCDjW z)8*jZ`JufNQg1#O6izl8IPz6Xtt#WB85NJMBQR~1O`1u6PDF5(4p*ZMn?Kx|vDx{H z_-M?qXZ~fwUwF!fE}Uz8nsWt zmNpD~ZpK6jq+%ku2BV^H%k?J(?d@@15@TwE(g+z`tOPxx5>o)SKuN!?zsTdlIC+&{ z@JO5>E9LciNia|EPdN1m+m}-7T(C?DtTWqH>8rM6e#MC+ZOOnD@eRI;gy&ol*65R1 zEp07W4aTp?mMbd5&*Q6`+%|!MlWNt#)tK~AzWT{_!<$&fkvI2JV;tWcp$z_-DCrTn zLZl3(Z!8P_qnxsg@ZgA*tq2*~+b?20F{MXL$xkvfzUsHEXT>qSEW6BK#zpx-Ri@|s zoke!&Tz>HpjLectvI&ZHNB*hwz6zz6!70TU&SR`t}&!Yvec65$1q(%xFF-};YA$e z;T!>1Qt@FH7ctbIG>aUu&nH%4FIs5pkDuVhKJ?H-D!$urgYC9MUe@0XoB7O_Y@1E> zx@Nao_DV+Sw(i*RZpa#cPIXl+i0wZAF^|SSq*cE_*jeblN20#cs=%rT)p~?>z|(eJ zlu2F?Df z8$wB!AGK1{-wIy;%UgdsJeTn<#w#~hf9?W%>|YP&uI2bfZ{6JXde1bhD#N}T+@GU? zuVwNThuDLIL`4%&-SwoshzIz-sp=mS=DunV1Nw%6ZN8VpAiUz#IB1bg-E&_IS{}6D z-u-j06CHRlpMJ^_HsgnP7FrTP`~_4 zf=c;?ER?C>2RsR+F_0jI)x;UeIRU`eOnfkjn|@pGMhJ*FyMNQT;VN096E5dR&co|23-DWL4R{jXfC;xH1@6Dr zhHFpDV61!_f(bd?I5dx3)!J5F(*|H-ZaDX3!`1=3iO_4OuC8q0^%EsXZ$atrs*t2o zf%Wa5Nn5dhCA-_Iv-;1t)kP^oR|f8NGrvKR?mx6m78EC#B{|yex{p}dGid44WsK@j zkTf}-^JAGzF>UeeTR+df9R3I+vpMQG12#?+#(!p`W$w}h&T z&?#&b{VHjkAAL9J%<@!5$)m)SR?_FSO}bi(ROJtfj`deb?hBAh?TV0fqrmk386F7ZOW?AS@F-5BKO}c;JBtdTr9|$sb*BY5#xQ?eN2`D^qd9ZI1nJ#%qiWZgaAm*Ab4#8QAVQSMFf+w_E8ujQRNGrs9LZv6>ilVp!}1ythmeqHTeFz+AQ z@u@^~f6@$$)Hns5H*1c4S=4K^Rj2jT_Wsn#T@Ce?50Zme6Ee6VoJKLE$?znP-)MRU zq^CHZ*5yxrOk^NZLmwB44vj5ZLS{|8iuyz04cI`JVKHBQL;DBq!gGURVRx26Bc;%n^;caM;=al1e|hRppWsEH4X+^l z;r_erJ2zb?7bzewAbkCA1UCL3HF~uB;&9xYa6niL{OEAWz2tM<_5bCmKb2Hl?+fjp z18moy`zde`jz2!x;>C~`?PE_o)1H0m347x4M~eC(!Vd$S{*i<6VSr!b0}WVVfjC|4 z`&<9_X0CtM<%*&|b?G0$ua*$iTa8~ciL3uAb%;SQ+hl(IEYpFp@)#1-6<3fveV>tZ zv1i*jGeo)>*R??AoqG!2W!Bvz`Uihow5BhfelYGAK6Ze1u zAJXtQ$c8x7gc${?$)C>s1_Mvwhv3z;CcMhYNdIt&&hLpFhKIy{MY<}%d=OFj-66I!)uu`g`!1B;vHgS2nRKA2o zj*9(@x?r6!I6vTpx)EQr9h@$G`N2B)gK-b*pM2yW8Rr6BQd&XO2NB9G-&fzE*ZY{J zbcxP7iMV7B))~!9;QrlUn{PSA$?v@LPTPC$y_K~G@JYM=z^fwv@jtLTmf}^*2j2N% zn~W7|GcUN>?m2m??L4LjVx+9pi(#`Zx3_upgx{`bFa{q@&RLUaR++0qW$50hqizxK9MuCxb^|F%u) z-EHQ&Ar$06T@J^GR>n`*IQr%exTwCEoYx`oJWbi_#VIX^<&=9K+eMsJ=6!2|#951i z#Jz$ObUZay?`$)iX?4>XNjkvfvjuc9StbKvou4P;pd=!SBa)ZtILjeF9{3}%pwr_d z!Jg{xuUg^4>l=#VH8`$3+xG46vQ0PN8im$|d+)x(?)}X#{ld-;QWZ}b94E~FTyDro z@p&N3aYaxy;)IUlAm|7)F(NTw89^VVt*n2VC)$DhUXZ4eEeRb@wl3`%wyl>=zFyY} zs@?^QdiTyV>Z9Ee1HXCm=GaANpX7dm(rUs%A3o968@GW^lEp+Bo|NK3>H|Uwxw8Iy z-T%FVgkAl~G}=7(FQDy*VzTn2Q!lVlc-^TQ9(nlxq7G4?tp6ggei#Vr{1`=_@J=)N z332`TDn$Ah;vDDGcBZqB;g5eTFY)F%@szT%|4+7Hn^(IRq^a=l2SUf49w&fd#}d{} zXC(6g^2Z?PC*;GB_*J1NqKEac`#9z>9@=Q4{1{J8h;z#a|6DC?go>uAeIO5Vp`jMWv z_zNd2Uqg%6(PHun*9D)%;zSm3Od@d7iv1T~Ip?P8{PER(zVb<&ulMtuub2$N>o9`> z8%N+F;iys8IBK*FL|@m4S1kvj?;DipNz0$}z(XD9=MeFL{MuWr(8mOb-}HqsdsBbf zAh((09usw6ulhPGzi=AxkuhBycY9Im%i$r5I3;g;Gg@Kag06y zalEPZynrl2Wr;tFcAOADr%5o!oeuHsATkG!=u3KA|KMGg>$Y<-!=7Oz;7K9U6PKL1 z4t-O93LJrLs$P3mdVhqS_BC~sVpTGvDvcBPE^RY$bL?9MAOF>_ewE^T24>xJvt624 z+<4kvOSL!Iqz|2Jn5#j!%|8GAKPN6d1DZ6OGqkZTdAMl7h3EBQV z(BCl`g^s3x#4#ZDIETb5KhJ|WLjU1`cmW;fwg-$OO@=cLKL&Pdl&wZS$tlQdq2s(p z#_DT6XySG*3_Y*QG4`R;FLz#iMe~%a?@#y}2cCQGIpgb@n{TnTKdAF|O=IEb?ww$A z^+M3R{S#l(ugKH^(xa7T@kipyd?wP(l54w(=PIEbSkz%gP#$4zMA3%K*JHoS8A zb?lY(+6&KA9Fb35rr7T9-VdAqkH8JiFnlH*H!s}8mao6yRRacZ7o1v`Q%eEDELzEV zoD)2zr3Z;LwV*BX%(%f9+v{FZLe$I|s4UQ4Q2@#?_#zd4yRMN@I{Xfx@CgdY~OnY(EkAQ5HiF^3yKLljBmKgEMBF-_y&EGPsAaf6KUKN z)*m?f#6@uHj{cHEbs@<39EXw0I*&veIpT1h+lR?j|L}892wK@(FeUXnKpXLw>mO!e zT&)dd&zzPD=lOZ=?Ib~-(_0kh=gEJ>FOMT$5HpjXhK~=#2H*@G=BuPRF%GIDLB{8} z=ma~}4~CJ>%p+WKN!KKPkM(cIK!PTyas@rQ(nj+ZD>^~a=XjscA2LY}%9j5quhNM3 zr$D(o3QVi&Md})V0*&|Wsm&7S*E2WUVk`UOAOB!aJ@u508#k__BDxU&s(tT5d1=m>ul&i+8iz@YT)RPQ2b8IP&@)t#Fz*Z=T(L`|ZY8P{-oO`yEa)anCSb zQ}_Ug<8SWk>8noXV&6lEPJiKf`?gc+HiG4yLE9-_2 zpM1`M^_#LtiY6b=#|KOn;Xd(g+wWvIef=uC_10VMsH2XmQ$W|F#_W8YePzGtb|O}r z-8p?nyLy)w?c+PF(RBvBOi7&>wtm}=m{8>SfNOulBGq;wJCQzb`@$LDozUY+vv!3_ z(geL?ffF*&m#CvdqQHG2L!u{5;T1sTd!jG$ik`eGdGh=PF8T^B2X!GsNGP0;K@)U( zy^GO`bAswY-bXR22W zB2~I}M}Kng>IUnuE^QD+sXoKr;RjJ-qzfkYqB~`up;KclJnV#iq-P=nKz0 zZq4w8=2xD#)>mg)+k&}LzgOV2k!I5lJJ1&5!tmnUdG_?{v+Y^HckFvOzTnC)uE0m| z)M5air1Ay?23H4^s>^+z!q7|P6Lp0PHwr@1GSABTulPTW1RG*e@q_v>i7CGn zEaHK{OVJLyH|ZokyxubirV|G`s2%W}a*$SX3a`npj0xsxF{)r68(h1>FVCOrO!gh= zuXfG6#0gmzC$|V0#~&xlIH?t{y2R@)q}A)0t^&>_Lsed;bM=nO(d1dNj}+3h>AGIs zblK81r^-)qsyy<~@viDmzRIgIJfgx+yaiJxk=8)c@F#xNGHQtX$%*uqZWLQ_RTUI3 z*p8Mqhkan6$SRx=p=!5+Hl>pusEg|=()J7P`K9*xp{&wDNADj8tKiG^JGwkfM&n~ks zB{t`Q@7pPNMyAWp{*ZO&?aG~xIn^d((rhMnMZ5OLXW8}BcvsxLaLX;X*!=nPZNCrh zBgiaY}qVl_WoT#)wg0qKQ)^KaLm? z83ci5=;_Y?_pP&d&~;qLn5WsCzsZyT1#ZFHOTR38(F`&21W|re z3Rv<4k5!o=4WUk$4*p7BeY&^(*Zkf3GZobl?Xqv*AP6T!spUDh@dHj&FW3 z?fUN5@x%5fC5vtMp4hZ0jcvf97FL!=v8p^Wft^0s@xz6TVcgPSyULI#z8xI9(;(O$ z0%*5tVTG9wGQ}K>O+|14Pv4o9|Pt;7%+X?jv@a`19zg$U@>lnxxk!2 zE=(gACh^x6=EJ)2+taa!#_fMU{8~!*%wSP<&58F*>4A?*YquQ@xslw|YZ>{^=Bx2< z<_P-3973nUF^u2b9NwghiJxEMAlerJcO5?FyeFj}{@^?5%m3#eteFm+FMsFN&rctD z*Po|}o32ii|8b@5AX7czDE>M%w(UsEC7)BO9712bI4~Uo+7~wds%8tjfZQE2gNJ+1 zB44NX;DhHSOyPChlRG%4+>s`K^lkg7-?i&DrioL2KCM6f;c49o_YoOi^u6sT(}6ty z)4l^X|7qL`C}n;TFF#p*PIvrNa~*kE1Z&(Xivusn&;yP!f0i;sjxLx>S(0z;=rdZ^ z>L8c=N4zc&*-kW5BlNY466qzH&P!P7S9d7A1lhF2dCvZMK~Fj_W_erx9A0wbZy+4S zcM}%P@E3d*&Mho_XmPh2IMD=UuYR=1PlPPwV-87<@G{@J;U%q(wM+J%7bD*}n`0?k zdI~N^UfI5$8&>W2d2*@#ly3zsYF)-UN#x;2F|+GtG!U(c_{zYfw;;7H@!AUd*y_ax z$OV}Bn7e0Yc;~>{^oU2Elm6^IZ%-fm;0M!h{nl^gu~^Gn_kRkN!`OGn=@3nY)yj2YDe1 zzTi1?gm+P}@OKq=OM^%UI|QAz{04I_;E`K|R=V)BCi4g0{acU{Tn@iWSr{ATEFoiQ zbUX6v(fRV?8qObn);Z}v{``CCqaXcfdil#=9-K=AM?E(EEFM2X?n*ipO-#z#k12DyO;KCwC{Z9JLxN5`ARzD{tr!u zZaK{36}B^Ob&;P)*F0lmA;yCRBtllVUFduf7BAEs0S#RRYi>&x@JKhmtYiX3;H^C; zG;i6#A8l1nt`g6g813sXC-&?Pl?Jtj?Q@L{{I>+Odp8gcPr`{yo6 zJHPfh679Jiho5w>v~k-JY5ie`rFEOP*y4yZFxzn=O*P3k3DLn3+=W6CMi|C$@bic6 zuG_G9hn43$Z@Mwve#1}F_Rn0EZoB-FwCRygPKP|~u`y|I;$Y*D39z^Jv$cw8Oc_Bq z7~-F8hvWFi2Ag~lgB}0&rF8pe{|dKT?y_5IHy?dW+IYkfX~SV#)4DBN(l~E@i%Fm( z(g;fXi?M?P-HIj+3K+S-U~)l~xv=w=o70Y)ZbB|xpC-TY`E>I){xOZ8c4j*4{Ac0q z6xg1I!dD-!#vhJ|;b2ZFaE*IGt1k`~kskTaHp93Lb{;;Sm|us#f^pFbw>DwkAbjqx zzA{~Z-H+3cix&f&eAt%ssz*L7t@*+wX%7T&F4=U%z0-!nw^}mRm5V+?4LXvj%n|zxm;U{9#XeVH!E?2y}{DJ<&HWkxd6+u9s`& zemk({e={@=W^0f5n?KAgE7$(g)HMR4bN-) zNz+9ehQ|3r4owgH#dFd>{=?sjsU&(`aN2 z@vR+y9P(L)-wlicirT78$jiDE{aXw2 zcfRwT>7_4ysXa!)`d;E;^8@~C+I{u0>E*{i#@>kf`TzTQ>12E^eEUb!i=OtVbg$#K zrH$yHWoFNvY5T3Wrk{NGTj~G&!~dIJezD%6%G&GFF1{UT2`?0Q$2;CZ0qLCcpJHzm z9ph$_V)2vyp>G>MHZhU!2+NNN+{?qV7R^jQbV$`A9kR}>W9hJiG<+I!$ki?j)}c}| ztyTC*3NSTU1YJ4MB$WKcm8kR-w6@~KN!OP@VXZT=HraO_}OIo?b<|<#<6) z!$V)iT`>DcLdky@$6q!}*V)WtU`S0nc|NhVERxH?Ted>#S za#@5}>o(-bX2nOoI2hwkm-MrresZ62u^L;%vFmJZc+Pg5@p%R>!}HDF{M#={J8)a~ zCVWX;eEvCj_T2h(!b5){Z9VxE+kJR<#vXX|X${y;2|rsG~V+c1N0YZW((~JoacjRe`!JO zf7jkj$Oq#In@E&RnPCMW)LD89)^^DZ)~mE|5X;z?@g=|HHf?mJqwZ9qP@|AqT zb9y==?`JqM|3v^42gO`%1=*;?O~?v4D2D|5sZC!fw*|coW-HFG+#$>le*8W0sn1GZ zy!^8Cwzs`)35#Yk=^x(v)==ja>17uj*H5(t>&H)h8lI>1JbRwj8{YQu^y0HGN_)F( z_Pf9PyVl2(e(BkMcLgp)xxhrbJ!4p1;Z_GS=QZ59Qx|3)=XqsdU9URA#gtl}=dQQy z@l9!B6-z?4v3zDc@!7CpQ#$+HC#K8(`mYjiX?*_kpT8JY7UUT}?(Fp5`CHRd{`*_f z`7if3qrUGqpOxMhkFQIopPo*?;tIQsvzwsPpZ%8f_Sd{1J#-0oXI_2v)n>;%k3G?z z*~VP;yv6kf>@{C7Ui3rEtL{hFL}P_epKdxXhUWYF7Q@+tM~V>|j#vxkoLL}uR7KeB zLs_Kvl5g#C=+?HZ6GKTeJmdpdQKA!;xawYs zqL|c>)qV3-LIwod4{yYUdfN}T$TQ)DbzxqzSbhRp?gcb%$}$GeTN@8Xvn*f@v(2F4 z(T8+ljp4?JLnV)GNb6Okv*gvbdGn@y&5zEle?t4PLE#6rB;<$s3Yt}wzN_}bD|;D# zxU^O37%8SLrWJakTi*$Vm-C3_7ZOa+F&K+Ka`5_7Z(-!3#6!+}RQkp@zL7ru@sIDF zH)gI%pSqY&zDnuBkG?3Kw5%;Tl(I2Bxm%fxPz4X#_ z@B5sdPCMhlu+iV@%dG?N0#^0KZF zyL8n}ccgE9`1R?rr!8&oA)m3w-_lQc#&gX!ULV+V>A2Vn+vx+wl0Na6hNpNQDwgA+ z(0Z@YWv6w&9?R8G=XZ9FHYNQWMMGIXbr6Ccsng`+)R}|MH^r-OK&WF}MBXn)I(9{y;kXgcGr2@t$^z z<~SCrI?8^?A?En*+qPToY`+$_eE;Of_QwJ0GKuHu4P##6OgGEF3<27gTRA5Ix4G5L zH=H?UH1Y#qDf2(n?I??nzc;1bE5ZH_Lstkv-^q5dsxL<57=KVPmjxplVU>Ru)+TzK z*rwqo+HmX48N?^pw?FT|6T-TnWXU8!c*r*%rxLB@OD{p$svnaze`ASSLpAxW}bUxI<}CW%yL=M4C_UT@>@;m^MH3B8;7j z7ROw0apTHMUh)!K5PZ%He%)SBGs@?L<99dzn4{-5H{*~CPwqQDLwxeV=cchQ|83fV z3%FtY9va042)?q*vQq=whdO#~6Y6U|&M{mH%X1Bds0q+n_-qCXUX##^+f`=Q%B;ADH+?Yc85?JL0JHxF}-`RPhPA+S5NJ&hv$ecm>JZ!lQA~eO9TfTrmooz=o5trhJcigR4(96U?1u zg7ot*>|yLANMNHsJ zY68!tupB}jaXzt^Nic_S!C=qJ+sho{JivwaY2*)I2EZ3eFpu`&8Hu}ZyDja;oWccn z7S45t;Msg!B;Iky?KTIF1LwsE-|)kHl3%hPterKtgX#rIEb)f3J0@Piy1m>kX!YGi zIaQ2;RfByO|L89I>81Zds+wwj+RF^V%A8K|VifG#pi{&kQFMW6xmYG4FKW%{f*jM1 z`vq3#*sfzwJ4tJbP8`)2kPcHQY?emr)io$Xw#Ep^{}zO#&pT)ubo8L!LcWX7Ge|98 zdI^>`vuVxWSfUouf%-wW-o^>!l15I^S?K5B;I%*e)&g>^pj_5s8tfD1K$$x}XZ6t; z_Qn+2V&uHbYt6OTATf%?4!-qx6fd4T;>cst6Q1&4(j|ZS=jk_o<2TZK-@7lD!CETZ zWhWP3d~v$+$}7`jaEsz8c-uU8CHYQgEVOWE5A0LuCI2~YlIdzsiD z{m~z#XFTH>=|K;AkoC1M4;wdcP5j&yWe0w*zUW0SLR^N@b6@y!Tb~%lZyDZYHY|J5EaDjoc(RbDq@hX*HWd=S)cV>eI>!17 zfx@L%zy9`C9{C$Ld*4PgA|L$D1&8T0g@qC(?GWfk9DUEU?WkkYF1)>P2X=Pt#2YPl z<4tsX?zp4wKA&^o+8Z>H{9HieId6U7eY~6j?eq8}Q?rKgy@@hCD8s@m4`D?##^0VsgPipW_ar_)A3iSjzRbKjI692uB{| zYSS+s{pzKI;9$kSv7c)b%mq%EmJ9eoV&Pjqjym>u+l6x{79DnS@d0^p`}Nn^M|QO! z0>OndP8Mr1Suhv4rJ9gI@~y$K%@+$8dUGb}y$b)bv;i+3Sce7M4OqBkPVkH5_FHeZ z$z>Xg9hT{|fnN}7`NiT_;3v-BzYkVFV=Ml{(jBCyzD$JL0F8Jb&uN$z_Z3F%gz%R3Fe{@w!nA-MWx8i!_c46USwxi1Hdm{LF02*+3jt^xWfVaCDH`$aNwqoCqC_0(pUfG zOX+>@dtZ7qp27UgXFjv5=s_fY@rz$fZ+g?4(&4y^=b69qVtX(?pAXF4SNsiZF94w8 zVLxXl{^KId5Eibuz{W?U%pG=Q+V!Aw(;c_{ERFs2>NIid^=WJe{zyE(Hu2X{S@;|+ zf0h=6;yG24?jB1;ssi3#wOCC8xOH-f9k3S-7{{@ zB3~h(M9G8aWb=38D1JMvfjt{H9h!di*M2Mg-Wy+Gi)P>a<~P$J_`SDM%7qtRn11-f zAKD`BDfhmw#}c-2ahGpFrBC=_;nYnR6SB}}25~6Ii;eq&MxL^s=L?l(FPe2K+9YP9g6h*j>ht(YZjoXEIGrVNStf__WO_kwaWq=Q9~F=Wr310n76W zaORIb3?gMSM~66X*nGk{#pja|d|{0uzxV+gimYM2Xak_RcDY4HNOM~0>OE%g#eS zXJ6;{7KXErkk?3(L{>C-hkho{C3=527N;HjwLPJi^y|xSoiqn(R&;mJ+wb0hQ{Y$T zf$~t0o>P95G|~sXlzQj!(M4%3)bcvh^CR*%#;A=4v|C-E_PJ)7OTYT7znboMzx&x|(7ki*wb$ALAFrP;#A4AQxPym}nXzYM z;`b1renp$jS1^XYF!zLEws3~u(u5=UQ}8=xZ3)#k{+Ozu$pl;f{-#wowN3TwEZ9qo&wXzC#X>!QxH7G42{Og5K5{OIk0tENjusg2to^h1E$`T7(546IRBO zO}J$As}G-8@~9ipcy77{R2B(V@6;6G2X8%WWU<(l|La+eq(##lChHF zU)mobw|I+gkQMmUvEyRGI2KgL@n%bwyCoOcF$0Vb7wLwEI`e_mWeQomNGk?iiKA-4 z(l5UK>ea0vJ1g;hu5ZYvT$X)rKehKa&+#u~+|9Kfy8*dywF7@)$)!sU(QscB<$@@s z_)n9&YChrI(Rx0il#_F)0~}O-#3I-)zWwSIYeC8_@s$^Un;tB$Xv?75Pu*5R(RZbC zu$*`Gz36rzQ*7cA1hsZP&shYSXvSmI29J6d*ZfE!uV~Stkp+V3YCMs_!b6^CwlbO$ zgoj@Fi(PtBC~#G?bILL$qP;7`lb{#%;#L1OT(v*^>F&$95Pnjuv%3`t5R{MZ_K3HI zz;WemNFI1VeOqHId_Y=RvfZGYjjFGp>I_mq&0h*-H9nS(L1#PZV$^8_;qF*O$C`iHoz`Qo*|$_-GdX7CiQZlhX@c{Oa`XKYB|# z@4WNUm%sewbnLOm_UY;FtKN=Bg`9itx#@-*Zb;95-pkU@-S7VReZ3L8#x|H;T!`Xg zn7s%9YRV_(2Q0D;VSeBrhB;4cg#G-ZF(0bWKSJAcskw=Pe|W}DF#bN;#p_>uSV(7S zt>G8@E4>5BZ`V7J}$|_PHJ}0pn(uah%(N8u%>UF@M>vHSf%Kuxai< zEQfGw8|@jvbHBLwwr>5VG&4Gtp7zY=rypMRU3)RYOJDlZ^rwINr<5?T<&sM-Nf%vo z5f(kSq?f(&4Yp%$9sDwZjY0I&TKI{-nYo)zbEX#630chjfoFmt9w&_<E~)V)X&VYgCSQ}gd_eccCkhHxPbMdPl9`J0$sq4ev~xr$>yl_Aiq z5Bb)zz!i5v6Q?#sH$vG|U5eI*HHWxOi)F>X?1$3$v0aC(aPj8v?)g9l>Ba_iQ5_*m z&A+$xE~PoP8(77h`$_FQK$$tkWZrX4H^Lv?m^ z4lXK7=bI?f$jqGNMV&*Qdv*Ck-zZT-j6~-UNqi{#F~=M;aDKWCkGeVk{PWXyzVn@Q?gdXv=Rf}GcDL0S{_s4E9b-e> zdBw#i-pUs_SK99e@s2HAbI;?Y2;3EB3u1m3*UZR_-2ucMeY``A*SM^o1x~+m(af%U z%N(XqoSXXz(Kp|4$v*sz(7s^CKZx_gyTRP|@n%R{;Nv5BtP39-#5FdA6QXeXK?i9q z;>h0(9CLj60afB3(_i+gH>Tfz%d6}eb-YFM?QefOn+;sK?6S+!v!3-VTOfPIYu=i+ z;W^-T#~f_o9dulHv^!S1*$-7Cf$PZ-@mUBRfFXWVEEt)BIjMajrWDqV` zH56H}sbUMpM`@3Y@a(+!vTH7dDY7k5cCjW`1&cTZOBUm0Mfu@rtU|kHtZcMt zXp}^HQ_lw!QWtxBA!{PYgkIYGO^E|eaHpTVI;;m8CMwjYYuw-fE zR9H9Wl2)*bELc=!bfLPtHU3#cAXHL{3X+70*7ITE*xfC;kVg%3&M$m(I))80MUWINqMrQ)T+!Uan{;3E5jO7Yw={Pl>p(*m)-u($&-FW8(i zs5ul?0QeVN?=9w)`PqK?p|0ooLUjj9{1>uup}PHLf48^{8ci2Lg49pA&2Rc1<}k9a zzV>+PxTuEXc>$^fwC_`%#up~*{5uvzD`F;8oqs3uoVV{Ymi{ahA|@a@!8zrM#%g@n zz@<6CE)Yd9e)3Q%irNzLG~_k?k;V$E4x}Nh+V8$_KfpaWHCK|%Q}~mvTXg)b6%G*( zo|gHjY6hfWX~+YNdER?&%cidChVRUo#^3}VH6BPdzXs%aC&gNT9s&dC)S8H9%^@-J z-|J9GC+D$3QKVU|#nh`^i0KpYb`GNejRw}SXcSfD2F2r_xOwKqA&;x}`+P{h$diq~ zYKnOboZ-g+I;D)dXq*bNmj@+58HI+ z$MG8l`MVxFtGKxH*vI{nJ&*fuaC_gw9`>;G_kaKQ>C{tCZA9G7*!9<6Z#&HR;QvRQ zbzXYrvtNX}cK8glQCu@)XA*W@jbZ`No`FT5X)BWHt;F#;m(Mg?JK{UL_&6gjit+i* z+*!tFe$y{}gpU2;8awj1h(_n~ZJnNb*oXoi7G*52f$e$&*Yd;M83i5qxE20UoDc2y zTSB>LhL`YRQLTAi7Z=lb?M$7mS%%huR1e5**{A<@;M-8+mQO6makm}qe&uieE*8#S zZEw=N<(6A)0gW*pl=9&Ze>gqwdC#-O!(V&Vo6<=q-v`gxTaVj?*V!T(cm8qn0DYQE zWq-Io`=5OJPVzYTxY5Fy>V14PGftLYAxmB~qn`XIF?w}X^04m?{+duXV55af)m|5Y z2BTds-B?|BSBdNBd2jjI0fXVChNOx{6u2qiw78@ONATEykI`cS)i$8Dhk)oWzP8ai zJisbW7;#Fyg0-78qtyrY#OCT*c!;dojZZwR6rTBOLft`!#EMY0AGr93y4fuAInU%l zK{6IN17o&1ZGYrK4o>LY4Nx1oob|#9yOqBDwSc-oH=!l1YCmZ4ug$5{?Q~$#KodQ; zlk*Foe}!UUF8G{6W){H;zngMMa|*w_r5PbVdt*PX>02s+2U7g|+D?YM$^J#@HULbz z5c2OjA2%6~>}0-L&Q{~a)mH|?)p5bCaY@&+2=cpCJYGS;WC$5{8Z?k~YqK60oFCd# zUes$aQ4IUXx%%(y4@_D$1)9IyCBxKc`X@^spoXK)^2r#jx@3@T>R)yXcIp#-t=jLt zlv@!G<)FJ=6%1MAkYW~57<7pm7;-t1A&-wx0^al}cqW9+m4Y>EP zYQH;YU&TM1+yZ4jszNlKuM{Y~1*JF7MHBQKBRzbq`B;qta{Y(9pmEN=C-CfB z$m1f>(|`E|wiD`-kH6m*&OY*ykE9DOxS*1-|4+a2m9M0yKmF;6&oVpbu}{WrnHS-O z24nUhe=bb%*=)S!(q4RJL(Lb*x@pRx%pG_BMp5_xw$Jj~8;dlyvut(@b+aa)=UgbW zYjJ#E+Z|hwZ;NPo((qo;W5k5Fe{#u%JHdR%6c?1}U&o4ug!*I)U@biXqmY&+z*;K%2+bN3&A2XV2HWg+`rdwRYoO#dhv8-gU4 zxcIukB`=y$8RQf+Z(z3$T0CT^`DI8z0*+~sql>ojMMvg1ij4-w51eo@MSOy6Q{uIZ zzV_4&a6TWmXHrGVmTiU<1j<$-bP-dtXIau$ZL=>bdENRi-;O6ag57l3Tl@R+%NYSp zM;t_^E=KF74C_#7Jkdb-LN=75#3#tQ5-)kJj+N|FOv0o8nqR^)KPCs}1KC5AH>n8Z zS3Empf>5z%Sy?SGNCw*rmmYQdECB6562H)pZaq59+J|hT2&cdrdJ~5Wn_`9c8SpG+N2Oj7`h_K_M z$=R%6g4*m8u+udA!45Y4aPOFO(FZT0bE0bk&tJ%p8C_)Q9N;|)_0^AmvQW0sOo%tK z{&IT?D>|tACm*YGsrXmzhiBju^Fwx0M)pbkO1n>`X-%5k!)NKPN#|YgOKBq>T>p^|{&9NT z;~tk@^P1PBx4!kQwxer*dyb1!fB1)gm|pwZ*J7b+-k#ZY!4sZ=#UWo@nE>SVGGAK7 zKRR;}%6AqunzrqiFWe7WJR2J3%dJLiF^t!>bH&T8s6&318F2b+7H^&>WRZOOj()Va z)ni8=caPbkn0B7AKJ+7>FPQlPUL&KC=kb!h4!uYAghCnk$DzR686gmhPvdE37B82Z znN5doJ}kZJwHK#9d-vPZXFl_pbjBHHq>p{hH zonQw)E7-RfgqQl%LKiw$J3;c6c->ZY0qrf7@Oee{Rw7bl=C1`d=LLILd6K~v!Uc`i zi1J}l3M}ecsCWcxK`mySh)=K&zrrL)_7dlLSyU&;x)OK83L^C&L#Yo=7hTj&kPS<` z7Nk1CKK#<1P-^n8M)UZK6yJ<%%@IV-DH18@m39n1$IF;EwNY0PBu9`{Wn1BbR$sI? zeXVgJK`0?VGOB1-qO4{StZ}O>4&oE+;Ae&UWuK`;WvWiF55MdXBzuY1 z;#DWu!OsfzEe7GGKDE$=&X?M5A(?kk9WRdRiy^ysLVtGQR0fVo%?l<$>Li?yrLQmL zoRXh%y}2P;m?EiQ;~|^^)(%!KdEc6OKt>XD+)Wed-e*OpkihqtXjr@PhRFzyJIDZPyuh>izK_|8aW58{Uw1 z?AVbW^w3{S&wb%5(v~e-ZCBaGO@|tsfbLxU%Wo7l%jgYSNRM=}pVcVx7j0*e&jpPb z2l14&ZupWH&8Upj6=Vmkz~Nj$^bnGxUDTG4rR|1xA4{r-V3Qdg2zM9u@37X7I+Uf> zS7EZmjgaY~-HaRWyomVq{hr_JzAI%oA+Rq^3{|yhXK&P`L9FD5MG)xT>Ts~fzqH?T zfIg%bz06{zpZPF|xj+r`7eMGM&{!af!_J&4CT_4CY~#O-KRvWB?57;_e@q-)FdO6? zvPq6gN8XqDZ{*@o&o7<%JZk>>(hT=}F* zR@qI+QqN7^E_5@7LQkGcS8tweU9~^_xs>?3(+{{~J+^3|t|bU_mMPzvlQteAxrFC> z3cx3-Gu{Yd7brm>pUE&H4j|Zq)Z*luhG5N_LhR}SHJuRtH2=C=oq`H;Z7>KoNRFyO zM|@E)8rg*tvQ*9YKK?GIJeK)rUBvf8$+J>$&FO*~e?9fjap}o{5JXG~;G1DQ-z(ui zUqr<_9e8^jcUK*E!oAXOzTuteum9Ia(ibkjEIsBik4aB{@{`l={Lb&B2R`tDy*pTD z&G)|dy>#)#7pK4Yi@z{kAA07a(^G%>dFcq;a>&~qxf_YMWKv&3d)Ph~%y7%4|-B#kKL~M$xnVVz53O! zPQUhRzn1v2fV-kFhd%L%PuMQSZ-4vS*u8j2dd~ANN{@KddG_39KEs*0#t&Zrz}ODO z3l!u(+P0wmR6P7h_mQ{!u$t%f_%WHRT%as@(X5wDC5WKMoypU?wCZT6H}VF;gGs!C zeb`Ngc)Mx9d@~urB%E+*#ap`lj_XAL(nFB)OB@l5M2L2*1gbG%;|w%H7CpWs?-kRE z9AT6O1EG>n77ijSU{pVXUa@ya#a)p76L0BJiEP!;HgDuy!h=b?f_>OchIp%Py$+Qo zBbbB}E~UtB!V;%=lL>z_AEc8Y`zrA|Ru&ToP;BmsTv&>|b(_*=*+g2)wyII9&H?}i z1%Q>J0Tr*Hs(P?nPvW(c>Og;*j9?Njc@=x#(|u~cDFglR#mdQD+qP|&*#PHI{^E|f zEB?LW(4vRUtMQLiqjTgFpz@WVst8#s_DnOR%VA&Vf0r)qR+27?2ur*}DLJrJ0b=pP z+b*c|9U6R1u9Kjs2$nS9Rj_wvc5FtpPAe zW?RuhuAN3dCZxbkMtM-%LS7!QC12nar6xo*h*XVxLe)w)# z@sBtipg_4s$l`!=itq+z7gRZ$=|YfuYw#SbHMmWaa{1QDwR3!My}x~N4Q?wO$BwgI zSajjepI>>wuctE~ad!H|$NnsR`qQ7b&w1yamtOe77p5mX;R*KaFNt0%2$;sHK? ziwio`i?{AQ{aG(e_dV@_wxF_M<0e}qEj>&czXA{ z-<__y>MC2*eB`6gPfz-#U$Na~W8)L{u>bWNHrayN`t=*I*oRweu>eYYXp8w?_SUel zztoOwY+rs@Z$1h};etW#3YNWSMm<>s{j51sVcBBTioik6oFrtaVM#NF>Y#oFjlbZj zy34xs;xC#m3Y8qfZa9VH-CTVn+5`w^((UNfx3r(Q=U7w#&^FR!V-3NQ>xkDyeivkYiI@GWPBh6X`9)hSR9nHi zo%$lQQV~|V{W5D%p}r4o%)r_7x8v^&Ow4h~>5~N!ruc3Lh=^oW?2v5AjRr{MoEP8X_$DdNY@ET3mX@aZ~E*R`9 z0!U}V5^pKbRO_1GAS$21;^V-_q}uDc)z6j0f}|EW`)E=dU3xSLoik#sHZ~EnO~|nJ zz@5F3r7rZPW#gI6$h&~Pl|H6K>i~vT`{5CHcwc`iZW+><3QiV7qQ$11+wvbD5i{kf zF|MG4^cLji2gm6H`9Uy9bq=yxv@W)Dtk72@cs3YuzJ-!Ispjz9F5cKCkNx`VsE0yEi?|6VWa+1Ghdi_i8e_#58ZeHdcuOaOhi9DOjwi&IkmWJw zJ}Lde!yl8r^pF2LUHN;-em$&b{ENY1hii5G3ji)PfX{NCVqMzQ zok*u%qKPlMw$sJPZ}FpnWcHG;egyR+lEGq-6tvnnCbUg{R^h&|KC+;o6$+Jpm)s2D zO_Lc;cN~*zmhV3zB);^8x)3e%?4|vZBX}^zUokXaB4D;tXaIA;U+3nLX1o+LLDpz2 z5wbAr|}lw=O(Q@xP0~FHKO@=_WnG(g%euoNPf_b z=Jm8kZOAV?w|6Pcv9CPI@Fq4c(WX%c{o&uL{dA;!wfOkUkJ+IiL$U}JQ{pSFl=(-T z_GXxkq3@c!RjztwS&egAu13D+whJNFz+^iY+luCTaO|kRjFY8nQQKh9g4XH+b$Cm$ z0c0^ZiH38MZ8||U}yLazSciwqt+P-~z`oRx=kiPrf z@22aozur{lVdJJl(phI;fW@4%({U%>%Wl2o0t-Lu@Z2&knr*-iGcK}>jjuDP2me!f zduY7IY+;{uuz>wt>y20*Dgm7Em{WvHDT;$&KYcs;Kts(_tOJfH8swrG^MTJsX5lTY zynS_gY6=UD!)Xt85RM`i^ym3ln0)kO9-scdE54Tg<;(w=uDs&w>096Wmc7}MyeFP` zqQ&y~wIZS9%GO+`=KBg#21|B;I#9CJAep<_UkNGS zP}!1gJxPDsNMT{0$!OcEwt{3?xHMlyt0q3q})b+U?M<|JC`AKF25gukr7R+0CPaMbi%n!V5@W_RtP6#;p}L% zFkZOoovS>uVdFe&k3O4O7Ih|DS;)2x-foSm&_mGeK?@~2>J1M*{_s(slx#&Zc%lpJ z_)$SIW>csjT=uQnzr6S>ZXO6f))(+)j+T5Y0WWQFoc?0MH94(Z_1toQ(zcG=t8%x$ zU(|=7N#(%`^^gn_gi9Zb4V!9wm_o)#cC$lmcrHA*C$TItyQe(K@J6flS98xY<4=Fq zMX?f8PSkvZd`{i@k?uaeluJssP7BKX(tK{ouI~Ug5gy+Myy(JuMOrh*H$HOw7!x)r zap46!l}4vbkJ%Y4p5Y}`TtqwJq*KyS#~hcQ^7QAXYp?lH`rdc1NI&@AmFb4-uSx&( zPydv@_{A?$NY}z$Pg@S#YPVgUe9F(Id;i?&c~1mR(S>9cMMuK}O)c(VEQ!EebZLITz? zi^f#c8t~{Ed;1Q3D~9S6_Eq|TK{QHUix~1vqg?Q`1yOWhyU!5h;dl<*$eyt@Ju_vC zXAeH}QR)52M6>S|VlnI5XTExO^WZ-edb% zC%(x7i?{BG1DppM`|QEMt{D?kfn5n9E$D)*X30Cvz^n^d(j9N_{Br$VsBo#mHm%|e zy^SfALqiVdj{fljT*Z2YOhI!JPt=de=xx8`Ih zpgGa&4QH;gkXd=hQ=Hpr*xnY1Gs@@1o!V1IQLMf3IMHMQ6?@fP-+VQG9I`q>1ldi2 z?~C}G&RHgnVJB|E-EKcs8PHPyxX=#_L8(o5GEZ0p{lq*EV))vK@;t-P9)A{>Xkx6I zDB~iI_2t9Gd@E@*#>N(SmAh7tGs5nL70hL4 z=op)Lpq9^be8M+&ZgU_AF&8rKiaG0rE56H( zzsEXj;wpr2&Hu{LHrHUE*;uOVpvJO=(DE{9vWB5Bp9eI02Z7XPOjwR%&QhOTF5`KGjc*N(Jj za(9}>^R$Ms7&bAs4m*w3r7eeVOQU#x6*X83`RqrFWqd}NEwXSo8QO5C*?8Bm4aS`i` z2Rt;LcE*G3MG)*`*UlYj`>i+I4z)ddc3Hf+P&SUw#*K&Kt)^SRg;@~`Ud#vH=F6B- zE^oUeA7ech3!mI&X3sApZ{(SF$>{=VDfUxWx`lbGF`|4XEK7Y})|L3O7R}gaofsvJ z9o6+_LMze{b)$u95qzFSZIf6Pp;Lay2w?1xN62Ch49N4n)wgs$+t>Xo4ua&FLlSyX z!iP2hY@d|_T9|IH`elD}#uOr2z^p3-NSLIw>#3986+kiezJR=CsO^i~rCs<-4tbj7 zql|9$i`K2)EIh#DFZ=0l^CR6Cg**G^?0oQYf%>u-KnhX!j){IX{_qPgpxs%DwZ)mf z16efsQn>2r(e^hT<%q@>C^!{}T8oMIh-$n9z~_{~%pv`ez;uT#@`?GNnoqJX4xss8 zF}D~N_UEV`5c8i~Gss7cQ9+M~4=($05d)2RT8^_Aem2fop&ua~wyKVeSt@8hq>=A= z$-sUvg2|u{a~m7p8rpoYOe=l>tX1%e7mSbZ<1qTsFMteegrfw30cTivYXu? ztlHnpf#I9*U%&Xv&*tnnD7Jzmmw3rL$T)MPyf)OB6O``F_aJ>IDxl`X_#UWT+BdiD zdj)c0Y%!Tv7pUn3_i^Q3%Q1^tB(3l~*4eI{3^)(qM|QAhxmkW7dG7IUt79k=)Vn(V zRp)sgla0#Pns1QLsk`*A_3T9_Ul~q~E#`!UScF<2&x<;{#cvqjFIaHlZI4`JA)Lka z=m-|IW~QfY!DY|xUADL~F*Rw=u$rEpg3kCOFLdYambsKlfymuYocp-YqD35AG(&$} zkl{kzD4we}f<+hBbJ2x6l6Y$-3vsi@<xYP}ZqVChe{LEhnOndQ8>zRK{{#9h#D% z9NUNe2BS_io?1LjMpz&`m7*tEC;pPrn;ngO24jfI1C>o%mrwjSXELm%01F@#dKn+?7CVwo?1`r@bW zG~;3#`y3`@ED?X|;_dd4{cfrI5;%RTPgLzSSk|Ihx89Thf)*%5s`0B|5@>zZ0*~xd z;5%fpt)h)e%9Tv=3UA~GO@h?67X!11C7`iIA5L!TYrNSSKP!*2yM&hfCakSI_li(D zVYk9bov&xj7anAjJ#0{4D438&mL3jCXzY+jU9vn7DL^WC!0z=WSE|)%lZlre#sSNXC-bw3PI8LJ?j`v z@t7Kv8HTCOBMv&QFN~&zh~R61(4|F-lct~Ofk4+R6?>x*Y#irlsS zC`0`PCe5rpXlh^Qel4Hv)W$?w2f)-pR@{;LyS!#!wI3c~eDWZv9T;_0+?5Y?e8g5t zQEUZSSK=k_pyD=`;3J#c19sE3ewO= z>TO376TmKz+zhIli*yg-KCvH0XpS@e(YNLhKrP&|$m~g7LHOE9BlB z;xC)ZZYkGpujB$4j33i{rn z%UU$!{AMu>$j+-cH9;mLOT|DLMh@s_)v>69PkGIe$BA&sFLML8_O7wUgOd6Hq$ zde|Rz1~Ad!JEo(k55d}P-C|dIdSB#%$%7-LLk+34{3K|4f-&5=8h`jmv9loSq=$p# zM?q^72htdW3`OUUtuAv)9k06U{gp%7X=dumZo(1|Wo5TunAip~j3OX|ybF!rg1s|e z7ty}mxyL{>=_yE)OrKz2-$qjdoe;8A43Z=0Ox3Y4XwdAhG##i@eK!jBQjWei|KkU? zpm>|vFyf3jDpqwYn#gw#c6||OseyyIJKEb1eljQlgsVk6nW&d^qW;=xV-V+T<$s)E zZKpl#XVrctrMAO%2NA35x1g?RN+7>-fvh^mV;Jy#~*AvI3?uhE5#pBlF904aqDhjdSjTpggp? zUTx?GIx5`3>Bvsy9-J7BL>A18%vaLZVmH_t+}1abMUkO7+;WEn6>deG8J|t-aAy^L zK7}1-oHVC-j$1gnL&^Su#(EVlte9NNhDr;;Z#usF=wj@5{^-Ek;t;3u6(NgN2efna5A&Ki#_FOm@z5uKPoPRP zw(lyR@>t12BxLYZw)V20boaOWa~O+fvvY_eV#7CN&LC!dUfnF<40fN*&Jsu55NG<* z*AC{&$6?;YT^Wg`^OnN^rQT;R0db4t?no>>m4%qx@MbVx^&bsDGXQa z7>v6*7tljvDU_#*^~YGuFlSxJesY)@0eRkGb{_%ZhQ^OlQIJ9lT&fXTsR;BZC(rEH zyDwJd_{uFZ-EB_k!ce%vZ7gjieizRt%pqNfn(LvEAbr8_M#a?3p~8w03Bn0c z#7k(Uh9rsxv})c#SV{3W(_ovz%=2t(VRNRTdVkp;;W3p4^`LPss4*`nz3E@^2(nIo zC-igoE_qn$T+%~#%LmblruJ%M0uVS}Br2$0=W?yys+a)I?PH6l9(*4g7;TXY(p~zQ zT+=o9A*Sh#_PIpyOE*FmHuXO2)c7g0H^1w$YJZRTL?9zv?$qk27*yneKb78sy%bii zt}f8VxV;M(Xl<~cWr3bLsc^p1 zoF%Bbl?r8t;KJjMS7e;G5ubpxTXVa{q!uU)lEL^L^3*aTXu`?F2h8)9%6YsnV3-SA z=!lDA+`+`(JKQzKg%|3`P6_!>h2Mc}W3~Wt@y2#4*}~aclfy+8(rkf^hOkf^&@yFB zY_0}_>Fa&5*}k&h`ZsyN)E~Pc&AV&xs25*`CZsu#tsJAm6%P|j`Dy~h>O?RBsEq#1 z$HwmrJ?-ZYQnG--o(VU+7Jrn-Ew_v%;yQ#_dJKmxu8gIXI+l=Oi(es&3uKHf7d$O) zkmt8w`VO=la|(l=rnVqR`^0`z)a4t>POPH>5{;Z{>0}K$56fCK^H3B)gnR{2fo7ND zGu#{sS`h6Nhd8wXAey&jr7jmq5gwKUn5;Mzp5>!Ct8~sF`od8(pvdgShEM!fT_2=u z(92WVEl8HUXhxxyxzR#+1g!1`+JI!*N0so`>?RlLOFrv3>4`_UCPEwd5Y|kx!ovrWRk(yM%km+ZRXX1<@uxl1fgS4ul!aSb z`PY6^NbzTd@VhY=T&;*isAzr*oN8JLmTsqJGx=$El#<=#(KkUPP|V3L{6L6*n4fc- z=X+y6d71Yah9vIfvN?qn${Pw7emCb3lAPMOwv4sLfZ{F~Jc>En2w#mqz2Whv0|#yX zi{a}^4)C$t#gJ%iwY3-|=)Na!<9k`{(1sgo%i-1y7tMdzK(0*!8m>)y@d>gQ;_4M= zpcvIu_r)px*bs+g`MFOIK7X)*$wHOnQx+wv2loqQi0b&N{f&RkT@ewFjePDkjc*%2 ziqXCfYbb!U@XOT4mOYN02<`%OW>fXZab<^Ra`mn8%LiwQ|2R*PYkp zum|Xt#}Cd-SBI?iDk@-S0G{=PEQ^tEMu;D6+yZOvHdBY5_Nb4BACv|7SM66!=rQ4h zi!I8f#qtj_Up+pkt9(IdI1*CO>cb$pO!*o^AZi=Hwi>6*Zck-F{I{5Y7b~4ktoebm54y|ITlf#VBj**XW%wiled7 zTy_hRTvX>~LTdT)h5Mmt+e==*_WFBv{z+cx%lV5QB(!4R`@rM-zm_k(GpyuEPX?0n zlG#IX_}JnX8gdsQ3**IIX4F7;r!khq$?rTKOX|VJH^!CP4)IxaU^2SJy66wQWqXRc z%j|bGmwjB;qL~~;qv;SUHH6zdOTEIJXO4DHS8+F1(I`&|BFKIe0I&DHBW@&l_B!S` zE2Wq4Rs%7IEYxLtnkvI&Yb(&f&o1usV8w4VFef^9s$WaD?cq%={_^QsU*@cIR>6a0q++J}yHe_oG%qMprtK+qy- z<2u-RTu8MS7Q9I`pwd!5Ywk9IZgn^iqSX4RM+sQ9f06NbciK_06!f_*9A5CP1eCh7 z5DM?q+imlD_$%62?f~}@$X4=!_(>suBe5GUT0K@s7mX6cC&*rmCyTeiMXFI&7pToc z@KF;N_nn{n_c9_B)#gj!@-L(3o^*!0I{rC~3n0q91z1*0m245D;_^LViOZ;(#>kKs zA#{X63xd@FS3AmS;~e+Rf(sk^q6&5{Au+Ia0vHdWXXC0!ciz%RTpzL#w;TiB%$1I_ zPb(Fa>LQaT^A@;h%eCKhSm2<31j*VBFKIDoOseLu{bE0rV63q_%;P_g=h$%vA>-%? z!B}EbU@krfFeV-^TQmc%1-QI8S{PWM*%u{kx%Nvr`HZl{m2=`-3|#i2nG6&(2P<2t z0W0X^V30!zb185&C1i1ahdeiN;8%ZjJ$cH~kH4;$T)_@|AY8h!AMp{k@S<`SZxvA; z$-PV4`t}*!*uE%pP0YC=Qn))5(@t(f27jIj65os>WepzUNL~z#P*8dj+Q->EhQv{* zmrBWRw6+Q^#Ax2iMsl(rrF5sd;Oev4O-NJ7|TEkJp1^{G0XrApR!8 zL*T;+IB6_*_x$Httem4Rk_?6q2^{qd&u7)56RniriSy#iV-w>8LPTRdbm(@vF04{)?p*xq=w$cQ{d`b7lmx=v|tK*^p|N;IaC`FZGAfz3Rdl3Ui>|( z<;mSin>hGD%oa5E3L$k|Ha0mvvKy#bbS7WKZ7N+-Ho^$7<&Ta*w z{3oxCT^}E8QTONd&Sw&>1N0E`e57D}2B97+(w(qr4=lG|wcq0v4zd#*;9Y?0kf*tf(3^TtT^~d) zVqA6aJCmC3Q0?u>U+Ny}So0Fk6M*E0`WkbkjJj{0Qq#xiNb@YmY3TF(jgxNYZ{&KcLx{FG=7*Hl1UN59X7GkVp&;V_s#RV z|C+z9SDP%WoBdJQ&PjKOXu16&k)3Fp-ik2SG#}_ z&n9_l$AQmXdT(zZp~ zEG(1@`+NryzZ!g1CCD}xuzRL3pNJKy3<5Z;Pv2E_D~dLEt(&U|1D766dG_E$1jV)@G(hFmeYd=eEc>R)yA&TD1k-agFc)1Ge-Fx>n+rl zkj0LulRP3$vubf<$U4a&N8P{Ihm5GlAA*gK{4HB~P9lyD?WBc^L(t_n{B{4U_U~Q% z-Q(eR=0W6%AZtjDn;oMS~*NR|$#dfd#ZvAJ2s>FYH-ep!V5@Uf!<@^o& zYUoB~2hd!Z`PjcHP_RlgNU~e7;4)OY6SC|(`(0i(Gjwuaz~)@hwm(|+1pC{5Eq*D- zs5!TS$X~*Jdx)#u^0i{IKke`7y`@(yebEfwpz}O%4!fx#rwP`9rvJ)3=P*?=QJ(jq zWO8Pc6_HX~1&(cU6oGciZ)jvKZQ!{VMb`%*`?JA_%9?*6`)5vhP`U&60k6O6>uGj+ z(nNDv!XB๶lnZk9VY%OYqA}wIIU4o~%{jyb^&6Y3dNA3opN4eBCi!XA#92g&* z(Q&6o0eZ?%{?XwjGmhf5_w0`hWjo zn%{MM+Koz94Po(vIiOsiyX*tU?dx1%CbV+_IbnIh-DTK0gtwdQIN|h^e(ub8jsU+l zT+f_S&`UW_`-BsAD+hi28B7%UVQ=!E(XuVhj2h$&jZLIG9{$uc_HUn0^S4}^IHMRH zi&aAq>~jkv%n$4CnjB)X@KK<1=*02|=QCI=W2a1mT}PgjX77J?8b|-skA;vQ)*Fm` zVYSi7gyI=W3!K5{TiPFjKsGemt8JW0h(jDZ;i`^_9Y?|`OV5F35TxN;~Q`LgxD{+q;sU^RUJyKape&K?3y)`Hf z{QjnFhpZ4@_E+{n&L*2Wa>*q0=65^1PxdA~1xp%n*)8aNtM>QymwSyz=3grpqe}Bc z!?|)XEpQ3al4hRt99Hgl)@s&n89U?v06+jqL_t*d+-owKEjjNtl+^`lp9@wO=sk~g z#o+UwvWj`boI_qk;t4w4s#n&5=4u!Sk_7uWpLDa2rt`yjsc7Of5W_WSAAB)^iI=cf zS+EY3l48W)5muJ2&+Ee0-q(G1efWF`W8k|TJcAR zG(GLF;oXTvi0cnOE{)vqA8Gxq*QRy1-;{>%>ejlf!R)|=6HfYT2+@J#hz3uoke~;tS`VEKp?StUsD}C4S z2e1i-Aj(odpjz)HpOR371G0$Mpp)9iT3*P`{ItPnE>L(=&ra}({wUmZBEq7&AgWGK zEXEt`En3xn$@bf3K`M!lJrogsJ>vNEvkhC){0 ^*3LW*57tx8kyRa%530(#uET6 z2OOd~XpAfiaf1}yHy7v*<^x+J7vdIC6A*_vyrE{-rmbn`mZQ_;ktd{yjhoYkv2iTc zL>}0pA8kMbe5`jt$`HD{pPf5%Ta01Jx)%;&~X!XnGInPmUD?e7&h7gC`YKP@K6b&(HDIO*whD&1d^?u zCtg~vR$UNPC&(7v@Ju73KFD0_D;gmS>jM&W*h6AjYI)J$O0&N-X<+d(mW(M%dBIdB z0x#Tr$;qMd7BrqyghYCML+e&gTD!C*P?z$D4pW|a5UG!*?2W~t_nj{>JMr7gXPPU07#LD<8pD=7U$f^G+W zz)#zI@{AyPOT6*2a+Mstb6af%JLQqm6pwimdza1DWq+kkK3YWSmn-CBc8TBRE+oI< zbS+(vwmz%&_lj?ejtNycC`g|+^Q?P#MI33Ej1x3l{A3~f+8N|>j8@3Wvb#8gY-I-5 zEUHsKlvT>N_BarBE4=0u59@`Czn1SKs`r?!`;cCOwSLC0&g#a}Ct#QA?3-TlYkO)L ztM)fBr*CS1EhqjO7lb8V$6zt%OJ6jj1vxvKuI$T$Kf0o_GqD@qu9##BlBvY&MyxaM zMul)f7F~rlze=2Djhtn{qxu$Hh&goOro|yVA#W{SxjMujx?^K$@*XFqom-E!+p30l z-JW!3TQcQzn&ZBD)3qsCQ7uAHFH=*BGE*&Bu84?#Sp^ z+5^Aey7`E7GhW@hX3x$vw&T_`vSWL?`G?<5YXG-x*pN0)j9U)%ardIDXR&}WJvx!5 z$JV8(@eOHeVpE#husKbyJH!`bpyMd=bYx`I@_-8+`HeN~Wi|eb@^u-1HdxmD7e42k zXt?0XImPA^%qdeFv3Ry_Q@Zt@_f2bNr_#tBx7gy@=pM{1$SE$y4o~gDg4vwy3R{%& zeVFEu6VoHiA$~Tb$%&0=68ST;=`d`A^Vdp`j3R%aA73}lS57mBcrmc(qC691!xFCr zu}*V7>x8S%{W<@m`+XXJ+QYJf?01C+qn?N+^a<+gO6rUG9Eqo z9Cp@w;3&oAS#H=_$VZjYN<9kP`|&-;*5(2Hb6|3OtPB{*Kzi$>;E z`xSHdMjel~e>x-Lehs%9Xys#6h*t6qJnpV&Ooqbsf*@^RzO&T1>0`t5)$3cLtQB)p z9u(f>E?bqQ&9S;bjd6@0jEV^tjP%v$UXy1X^4LU!csm|VB%I&lP|g+JMiJ?l`$D?r zrhOlO_1r+&E?D}b;W~IRZR)bW_tOmstzEl$h%KP@Gunfl^G83enN#+7fT*uMDKQxW zQ67@FZeQy|v7(V*b4i9K4`5FE};4Xthlb|36o+%cIoWJ#SP8r0z$TDR)0?Lu4DC(D3-CnXJpMnjK4vZg%IL&>F z-|7o}YI$|NDd`99G{aBDG(0*2vGZxe2JE)L4^8eo+p~K&UfDX6X7Eygxp6F9;D_cO zJm+i++ji~7Znx?5?VG-yKJ$gkT*2b_cYpoybkS>mFHK+pWn%sMG=?9-W23%MV>``g zXK8CM{}eKbD(-^x@4*uP+I}t^(Ed>@lyT?a#Kd}g74I+?J*Fm+53^Xv8nXqg@7%sK z{r5{Qv9}`1rJJybb;!Dj^u)(MJsr9A2t30IGO)-wgz+%UMOi=&4en&K-DH5o=TJ9? zXVvj`%ej%UG&79;L9;lZfxQkGud?Cd#sv6>v6x4Ee0(B}PpmUN$HuYSjD2u-8v3xd zZlI4R_S>Z>?t%w%{LMYf$bZf!9Q%w9@$rcX%NYiMahaXLe1cu8bF-l1S!cU2N9@4u z;JkgC^9di^xpo>0@iVxQb8;6RT!);S+@0p9rjUQT(k$RNzxt)LXV(sU=GYAQ`CKx- zA!q%eho$qL{OmLYx${^Eo5f<;JaUA2U?23)Jmw27#+fJhaA9~1`8kRY=RW5BFcx9C zAfNLG{*Ye`GLJ~>kx#TAjm;0rt1_v2W<{f1=*M5EFsk9G9{9>p8u75f;p8lf&OZG5H84?5^p%WW8xL8 z+p8-zoNpzA}+Z z2G#YHg@#sF3Mg>36Qt}CCo9j@M0m+l;%vWA5wBput1rx9?z$LTi!RsL&Yf{Al1;AP zV~b|f_@T;YRoUVh7RhF&XE2HRk6el3k6kOuZk({YE?pWMP2-zixu~`di)?dPj2XfY z;pwUA^neFHB3=2ducm8%@+0Hb;fim5C0%>Xf20RI=n-jRoiCbkmxva6dM1vP5R|}P zu=an=M>*oD>+Bqr))$#n3S4>!lBL8YTTNIen#7g-qVQ-nw&`Mrm;(B68V5dk4Y@1?!<1xzx?q3(^@d2*gLQz@czGeZrXU#{Zhj1ZgW_a zm_w`}EUm?21lJ9?09F@LxS+=33on2%S#Sp#cK~r=bOiDkZ`)ml1rRQHFbBBkL7MF} zqZ}7y^0IIH*ze3#03Y&}GIC{6*Ha!ET3soiz|~HWvP+z-JXaIpB~OX7{X#{&f(!Ao zuI|6)ryTKAb<7`Lg)ml}Q^v7?wHAw^jMK!#9$OTf=0X_e5#|yX<+%{c@z0B9T~M)( zbB@*T@p)*^op+$kbUJj+MEd9l-<2joZo~w!QlBdMk zexV{>!G(BPSNC7@Q;vA*Iy*Xyj!n`JK}HxHr7LrR=ww z^2de7gJ4?;{w(z$y@ZP@@S|~tZ-uKh$FF2kp72J#I0c=n?7OUK0@pp5;uS3A7+&dX z^7z-B$fr*IIwZ6Sl|1WLG?Th&zkJ$u96TQ1$E26@Vu{yn4z6=$u|xmC3vUKli?yo@ z)ZriIkhT{04_q{Vc=pkpBj{E^^ z3otO-l4lwA7hN>_X|?CIoBtG>R>MV}u&rdWiEueYvU|abe_zC3n$wSKu(CzJ&LiF< zLs;k-r>pNyg%SSA7@Y`t!df08lTs%X5Z~<*Jb{Oi`OfH1jZbLeg>c)iPf^noTyLW3W zt;0b${`a;0T%ckMI7UYRtss_M9Aj?H&11*n3Q3S<~CiW70i7heyN;HZEyAZ*OJ=WNQEk>hQqGBwup(w^!FcG3+qJkwTmSDwB z6ctn~pmcb^Ki}+r=g!@GFZVs(d+;v2g>z@l?Ck7L**)j%+p|Ykd}-`q9Kxji+&Qge z#g)6rKKmaoga3BBijA_P0xykxTGm=;L;2;e_P0q?kXz`gS)DS|$<{Dw#>p~_by#VJ zNi)O|a7@T*^xJrnt*b}er16tqZLGb9fU+nfl}~ypnDEzU|2GYm)cCKVdlokv#y%vD zhv+`Qts5~WG+(d0#z$x;$&cL&C?D?yz)M=fnkUQ4^mwr{3!{*T$P_VA!HQ8 zi_ySakjO{~28RS; zEa(5CZHh#W3x2F9#b2jMGp`gIo_6M}mu3vw6j+6$UD`6{Maw0pabUnrZxjHzYzSq= zvUbY6n)0hWlN{R{YKb%m1c8kL>Z*sq(JnkdViW#W&BE(Rt>(6|2U)Bj!-Fm+$pCQ< zeoUfaB?uC3BIe^2E2{{EfPu&I^_=yXqqQDeRfL9UvQ=uzGfP(~&Cypqr~dH0bXs-= z`RR85C$GMAy=7EfLDMc8+$ABnLx5nx-6goY4=%xi2AAME5Zv8@ySux4g1ZE_ftk7U zzTZ9fth>%Sf9BV&?w-A$uBxtns_MS#=K6TQ{%Yd7j3N{&q4&g*fqzA;CqY8 zneR18mY*@UEe59Y_mU}{JkT&Gm?t+0H~3RW8WjA%(^P&O^3NP*N8_;-wnF zw9qu~0#GbZE-wkV!XyAzXFr1y2#@_V8i7I596o)-lYu)doEwd{U%G;ifZ0`C(uLvX z#8E0p6#8yYo$Xz?VM3m2g`;mdGqt6U@KWMyNsTf{HbQ#wAn))3iDl^MC;#mW=_2un zNb&dCU=H@Ccw8G%6JZK|x8CMB@(CKH^9lJHVi|$ZW=B;6j6>X1dw}D4krzeWMJ?u^FC%M7=$F<6cYTln@O~iuB z^3b_=l_Lig?Bc~G1JL1n-6GdvH8-9Cqa3evQU7S8)vIjN^n)LH2*8At&bKj>f}u0c zD&)a4PCvPK^@#d*@P98Z3aY(^6iRRz(!UQL5bL0?+q)USS4OBUoDj4M1QuZ5Q8O3O z(uNwb6LBAvLMR9WLKN>xnkG(ZF-O z0wkRu#^72>l1hjGCASJ@eY7RV`E}RI0Dyl1F@Q)k6+`K0zRxZ|r%6wrHUd+0>@W|1 zH95mR$aV>Y zE;=~lJ>5urEe@y!6LtQ@ANLA0Y#5XV-n{#lsttB`@gG7{ENppo!b8VD{b6TRAt5cm z<^H6FmsPIZA1A5BDq8%D-B35vjIF5UQt0GcqbJA!AhcfeT5EN(6Kp_}M(_XBjM?>| z>U!3mX86~unM+GD>|1Rh0iHozMSfv9y1~_D`zMgtvY5M!k~=J?6&g%hBMRsN-QHz$ z(S)anbY8jyu`Igi+f`@X{V*y`YMRFaaf}D!yt5RI@P8OHANZ&8FaFf(I7Fp z^RPzb(|V48KG%CP#-riThHNK%CUPL~{T!6ZU4YHMcs9_k?l$K|qb!jGT}GX^>}P5Q z2KrVa`2IW~JD-3AI1T*t{8zPWwsHl(MsQ>;*GJH-C3a*b_H@xQ zQ)P$3`y#_NPGv8EBf#i0Ss;TdMCVKAwPm|YpPJzVYT9OH8m~mD7rZ*lke`D!`NPvq zct-&V)vw(IN)?R#IFDoeNF;S+cZNEnF5Y1=EcAWWUB*X_$7Wpml};rW401ZoCiJSr z*pxN5k2eeY zmZtOBw$0w8JTW_2>BLZq0);RM;wjP?o+Vk(+hKiJAfTPEB=?@aEK@gD(ja%~yT&ZY zX1VkCiGk!Zu+74dW{yS0z5pwNO7i1_7pwGgB>w!nmss>Y{hTSPP%Yqo7X{uqU8pT^ z)k2#d{kOQNHcjsGIHqLsE&t3pN4<$;ZdLJGJ zZd5MHeifI6%I1J?E>1DmiNR+r9q-#zraU$~BZS4?IQpeCRB^U`#E{K30^e+>J#H>l zD`{6vR)VATj9DLSvbfNaEkjNFFyruoIq0j(VHOMj6k0Ck7nKtFYnnMUw8-6T zzbXOBcCUFCuFGk79zi0wMt}bJjgCwuuZZ0xffY9;d|aR!?^s478_&}g-DDQgvcYv- zUYd%qLUfNZMcpLa5|Ml9_W;hT;HKqQV_)WVsD%8Pf~`VCRpy!Vp8`MjoOlwKU229g zSL_uOgh>pFyt1U{&rwk|XDc8plUUMdiw~GdPYk&M_4lvfao)sK9)-lGk=RuG*BoX- z4AMK$5b(Ju&;|UqS2%7Q!3Nx;Gbs z)q)^g1{FP1`16mS5BSM)J+ZzgS~uv$;tmN5=kjOtd?8ijVIx(f%{?tTriW7l5_8#% zDbmW0DYA*uvUOj_`aMI4{_dHz@Ra;)TH`fSqhjrNZ)02qASJQnO@Q8IQ8+lAy*z5pmS8O)zYo`o zys2nm2QYHnX@AQb`cXn}OZ@s6@vRyIkRAwYp=5a>dv$tn{VFIgtWv6+WsWzr){~zT z;8l^Mg3H$W=eA+Q;Cy`J+<}VJp(&%P;?<0X|JYrA_Cq_W zi;1`#2GJsXz=z<+QPUFQ*kOfctaUq&sJ5Tc=Q(24?A7`CNNCiI+r_;bvW^Q0kSB_3 z|A9sOm{q%lsSyF~%sD1ic*dxt1`F0`+V`biW~16FVqSQ_BI-?RwJ^H%uNz5BxcDs8 zIG1MRw)Pv11!7Eb!#qNKiDn)7ax47o#|%fqHs&cMFAg+%>IAp?RbOp!k+qeQr}D}G z8}@{c0s~umBG^LePVJBnuB25fr2C|`?P_0oJrkx@!BJHoWH?DE&fjr)SS6`**wM#2 zA}ob^CWg92PfUcIXs}pY$_)xq7hk7D(}Y2@wLh<%kw;VEw+d}s@r6p$Ea#l_n|o~opVD~l$k zu9Rk+LR``!1ncM0D4X8mRrg4PdVB6its#QPPU9Vn+2cVD6-hUsx=gpahJKIvI@bFX zmAaPss0JO-8J#Gm+3VV=B?-lcfcyRC!xOyOl&qTe&ssYHl$5u+>$XYN?wA7dgET!` zF`iW6{Ne=^x|HhT#Xr3;4lek3lcsqD?L0M9s6r~GyXgP>iu3Us<~QDPxqYe^DYlBU zB25whh#OT{r9^cI*3gw?-WQlaQ~GGRsR#IFTrASOwB1^rxZCEL>f4s@KfT4R5o-Ny zEYgQAm2r-@dU6%eSA_^luQPgqU#-Mkh@JrYTvNac_j#+l9jeccC-gs!;Q>V)e8~xt zwBvc@YwZT}-pa^#i~|Rit$IVSlm4OH|H9WX5@5zA&nV~HEnMi;0a?+gx$ zj{4Ojn+M8Tx=m+&5ga;Eth9>3KHwS8)mV9MtJHn!#<{*egG2*wYcL$*-Uxj^G|nF zR*c>Q+G5Pv{hJ1mm-X=EEd$7cT9pwT^YQ@#dnU7OmBCJ9i}3}Okrp_flP-Gm=Lv5| z_{RltN4_xovfTovK@mK5I<8!{43|&{PKMl2_Yrm3QEv^4lbs8Cq_nUF^@johgX-}V zLt6$rda4PuU+{F&csG7}l%Pc&byte-i)P;Lxq4K;or2_0Kdu#qspLx3{PQaYofcrEWt4?a1OmKOoDn^-m{p&bh0y7Kn9qXd@(S%S*O6J( z&>*9iB)ztRHvh=l_i)O_e{8F;sQuW?^U#J~P?HOWD?>LuPE2bp&AB}M4jc5cKR$>d zGi-bgj(r&U2t^A2ESs-_zZZzf7~}djh^fN}r8RL_E@(kxo?`b7f21C%gKsc+lH2sj z$~ou*P%_{^FI(VM?|W~5G;D=ujjwD%1}1{3tN1ZUXk-NFsbF%KlR<4mPrF{T(VN4e2~yygp8>KAcg<9aEI@ z&k!bmj=8ONTEgEn71b54^2}vWh1y5{@!7|}o!P8qmAjBU^uOLRbX}&LO+2sZ630IC zZX)ylL)-u&vCm>rV%gDK0u3_P33n%LROXAQ-(}t|&Jat#3_)c$T@$^et;6V@J6Ryn zCz6KNo>7k+y`L$3WUdpRw3&WqU-bcbsdrg1y1u;~-~q>^p8ir7$zadk#+p$Fh(hlz zh~G}%fzR&Wf#-B=)*D}`UDwnZHorA-_fY5h;C(>{cNZ>;M! zlg=Ua+r9V56O<{|g4X|F_Hw8@@wc&p>Y_W_g92Zh8Cp*rY8yT3ri?*!hdOz!=`302 zD~vh~<^U!tzNk#5lm+js>E5HYhAhS=e%O;JQ@|0FZ(Uy^ck!va_-eLmtc~PU`6tdee3VqN(h}~49)reQn_lH+j+e@(4SkozfkCDWDc}q@Yze*2 z!2#|}DhwF|iEbJV%Qz?iqf%^ZL0TyDv{QdN>%3Ue&2LwQ@7HRIrHD_?r!**p4=HYt z!qCvUweerpvIZY9+t?Qg+iVaY6LgS}e~EE0e2a}10+tJA74ke|=@q@;^GOV6yq78IZRURK!(6dms6t>0!&14&Ku?+B(^m$q zZb4ZJ>chHkzf%wWmKh>8ugm%u_}mNJN?A4W&dueFlyAP9Ibr=FCaj^O|1f&9OQ?Sx zO3zQ!Cb9E^W_d?Et_;x)=`)!(9kPHrHQfLPdfn~mB+@A?Du=B?Z39UvyIel8jL!0- zNZ)!OD&)Us+gZh2h%Hlcs51`L33tJ_I;{qFae(OB?*}wVEgPoJ+{L`Q@0>%N5>Y%F z_987*Qns@L3jW2k#je9U^i(hLG^QOTS<{?L1cH)1vfgZI_e#!e4ACZU3CkV44x-<^V+?7ML!~qFtE>}W1l;cR`;QGiO;q#UCjp^P z5~+Z^v8SZoq~5=7F~Z-k96OLE`dt_7jwd@v9UST>eb#a{Ky`Y>6a?M`+gy$O?WFL z@}K_MtNPBguSFLvLKifKj-5p+Rn0uu7;MAla>dlg#Y-Q*##GQv;Np-R=cTNQRX493 z+KEe|y7PCg7S3%O9&I|)|E7$XoMaO%dctPg7_tFL^uzycPBhj5X!8EC#rWG`MkutR z)4M>=*qfuND~U^P0tFE|e%0TTRLzv|CJDlAR+k;%Zooj6Y7R{SCW@2IX%sh>iPwxr zu2)4QIGVj!DwclcA_%XF{7@R=-IH2w=&>}h#$pegFa|h?LghHzYc9U-Zf~?q}c7xWo}Y zzT`oB)=y8as%Ib4xY-KZR#=Fw3c zKHMXLPMoUxkIA+*J@?jt8vTYPu#>-%%&xXX1O};z7zv}~zhg5VVLQ~SR>2Sop>*ivP*`0)P9L^ZlrpQvZ1HN=zAfI+@ zing?t`djHTx3(ah?K&n$2>Gv>ILDZSl8^%?XzsazX%i@_Cgv9QyU?y%elx4kWc|K2 zOWYGSO--==fV;lmN;0LZ13pLoh~U)Akv-ca9bq<&#TH4oTx^67B`GKf)7QPOM({OG z8TZFUQ`NHLYxrId>&SPh6Pz3_UzyDYQSO0#o!(<0|Dc-CC-i)aLR)O$Wf{vr>}Z0K z*q3FBO0kytp>yE8x%UJeY{NGMrE-3i(;J}1NBNXnj@bZAB(wO8d#fO3D4xy7EJ^FP z#BwjNZWLpuh0xgD9e5B6b1o( zJr?fT9WG9mDv}(Ums1=1kx9o4?5pX>AF4k;f4D=s-q$kr;0<{5y4&x(>Zz<;hnk`I z)g5h|Pj0sAe`gk>~pSZ9*_|#EWF6 zQG$;I=_Q{Sv8IA3cRpy&M;yO(Co1umCCag6fc1LhD=10ufm?BXI%|LROJ?)@Mi}qV z2L}U(_<4D*)Xx!~sE2O1eNy@qv$o>d;*pgp5ZvErr2ef7{)E@)I?TrEt_eK>4-+x8 z_bJo*!PJngfPG4!*6uoqvX3i=B-+lJV;0-40sn zr1S$W8qyqY7V|cHtn@AyH5jPtHe_pgNqEiF%tGuhp)&csjAxfXUaO7c*Prf^7FCLG ze5bvA&wN6aakw_D?)Xf5y(y3W;&01WLyKQlv@4$3`?QWZ=1~%LA08^XV0jA8E%Ii6 z`Yior1-PJ$e>Fm3VVb8dn73+&{<*K8)6Y~Kv6W=H$KS%C+4^#+t6U&CbWUp>guKmW>OBzFNo z9V3ugW^z>`U?`PumtKZK0Y@8L)M(3H)5lP>1X<1G*9pPHoqp%i0D+#~9s<9opz}MB zWK~B$+`sxOiffy{e3#Vig;3*`iNcbRd0`Pwe{<%we;>b8L6Q+}C{$CdDa2F%5eSit={TIV-UucKty<+(Ui)`cr(8M)*l*v zg*wWMlx&1n_n-#PoTZV#*UQ!0)lLY@)3#fqN)Xue ziCO5XtMQ`y306d8ns57$ZO_HeYgIAu(%bV@Ak;7%leo~g4Dg8$@T30i+ib{k+VP33 zHgD@*8(U#bklaJ1SP8sOS2|A>=$DwDOjCk9vnH)!%KHW%NxZ4g@b_L-%Y57W`bEO( zE=lCn-!1V-?74e9K4YJRW~fCx^D;W!(sBz=%qlD;ZXc+H=jvtA-h1ifF)%qfMkn~d z9xfF7S>OIoH#F|R@O+}=_ixV@LCX?toz?#1xhfJGy|1oM)w-KP6U!Zf(Sgkt0l7GY z119t~@6VhcSm~h_RGDSGLx6S4t|x2bJ&?P#3=MNtTQu39S82Odm=!Gsj60NFg|)88 zV<>%Z(dymJ33#4^x1xUQ<$q}HIJFlC$|C4ZF=3zFr01wQEcC8`w=dd)o+GAj(!}3N z%9~ZzT2@M~FL|&lxhX2|YyB!~dcw(Gv+klzK1X0ve8z!1=wy_=4zG#Ip8I_sBX|wo+k~9HQSwl0})OoWlQL^-4Ds@ARft`YaVK_ zimhM0!%^VANp!$lkczNo9-_Fjd^VC0-ax81M6DA9LI3j@Tb;|C7x${?L)p}b*;UjF znh;>F{4q}rP<@k(KV4RK>UzX^OqucCV1QjN0>IPXN}cL#)I?(Ct;N!($m@Rc;kewc zA#YFML?T4twmqbvio^>m)?>cvP)w_k=t1YA-?rbFq$uoOvK5yyzDh5-!L|N8V)~4| zx?Uhgtc6H3H;c=^+iqRcC3vPU;5A$QfWsLv_IlOzMa1_pxhyZhU?h%|Vb#R9TDirP zN5fIW5G*xO)A^4n4G_@ocioS2yBEVYQ9nvy?<{QOzr zfgMIBjueZv2Kv+&x5G^AteMLBSR!epOOD(fG4knX-j}}BA=)-Y8fzt|5fVbIR_L({ zA4jeD+;dm@_s z(D~%1x8PnE!}9327vgWP(SCWVQwiw)A3s9s88`L!FU{gajTSY(7zsY^aeS<5+b!^7 z`6#|KYmAUiyf;Jmc^Xiigc?Zhm1N|7sxMF~6T17*Zh`{I`{Vf}c3Ii&7dDMqa>>&5 zcq6FC1qMGPeUn`cUtFRta6^_Y3J_7lqteI$Qx38G?Vb1V;zoab=$YEBu-9;&Yc-OO zT8-ZFx}+1>E%!(Ajpa!V_1wC(@qZ4ya(r3LyK0V`y&0X`H~|K{twjzBu#^q8Moot(H?JPB51j>xzf#fxQW-; zC8B*V$3e{pm25(afL3T_+@dU{f7AWtrtXUI4`V#zJh4sZ-tL{Y=e?N2v&!1kEax75 z6(PvG^@{Uc*RA2e6UBPCb??9!FuXX83~#9U^cv<_w^uMs5dV z)ewAZ$rAkv$0|yLZB>p-tx)`yOYmKq;`zQ-)rTq)l~^tGfVPQ~1tStuIW&9}uW7J# znCKy*0nQ~TC&3jpW;~=O0pq`FsCIFGOzJ0g=u*~P>vVebb&PMWf~SzRXaKxCgG!fJBNRumxilFxE?9(ZRS|4s_QmE+1VPo z7ovWkzqcyrT%{x=nA(}0k6UCDfqG@=A2c3ydJ3uWZdlqIe)}V%``k1?j$Xw%SlL9x zzr&UC4I)kYo$5Wj>OPX=9fXhKZ73OpVYv{b$$K6myhBfjnHj`Y&q5VX4?VuX{oAbf zD)!by6-Z5qXB-slwsc7mC6+zKs7sG|1Ad4 z7F_)iOguPO%?JsJu-||p{IG%|8*@-2KvDD@Snjw-FsWG*M&iqmFzOlL&${E@BQ&{a z&Z703IY!~~ZxWlHtoDU8_2JMfecIXJx^6i?h*q3wGI;)O#}Id7lMRtLy_O27c=eCx zLfD`TZt{nztx50AdC}{X-5?(#)ldpP(Yg6}RFN6K^!NODe!3{+e zFcog7zpa#YH!ld`2=wiG!Ot7A9yqBU@o%e~eyYnN;(Q`~AA4I7)>FkdhWTVC#6|)H zq__+2x*h4f9UgK%oUH9NWEp*C>rD+5715yg7aLwuxDpBx-1Wxd|4@)qSG3d!Wt@3Z z9jsm`%@f9u?hSnxD^PrDM@(o}Klfy^I=$oTzCA)e)v|6)>V24^YuOhBhs9}jw7FfP zNZ9_bpiDZYt-JLfTp^U|E2K0j6Yqp}D8;a>@y*5#O^V0#pJ?)2SjtKDvk( z!@o?}HtF>VwnJcd?Wn{f2-0{ix4t9|7NS%S^i%d26V$_x{VM#|_2F4HFWuQ*OSwK^ zu-=T|Ig9sI_tU`;o@&(-TH{>MpR3w{2WDR+Lgv|x=IUSepd4C;r_-9+h$Efc)fh-S z(>Ye#ZCRB`zp?q-_J1 z`5}K7m^*`C8knv(8d@D7EX}Zb^ZWa&m3z@_e2#SC0qF=s8#rF8`QBBkJRGgP+$YG2t445ov7+C96;c6teM;a@!e&AA41ClGVVh(aWo z1qh1RKbW#8p6oApjmZ)n?y-%w5kp=>5Drzr0X9|TwH#%&J!EX7WG!W;_NYpayZ_*! zO2@VlnJ3PI1D;AfxZl%*6X>azSTfP(_CFGmg>Y$ocvG?c2p*@pK0y7UEMTWdJF>>d zSDt8Bw8Z0%cW1`DqXc=_SBDZ8ydIEKaoNn6dKQJ&_Plk~gr3*DZUof2P}C-I%n%ks zm+Tfv<3w#S{+L?mhf>Jql7o^YzQ!_00vCHR7V-iZLOJ}P_x!h+8s>0rp&OW%&*H(O zAw>f-aD5)4!`QL9eaB~&Zp$RKZq~E&DU*D1d4M0wV!r~Yb&XuYVo7)uTDE-7iwD9* zhDQf$cdcDUbh7nbXT0YlU}Mv3YfMe(m15rdD+q|2M(32e6T#+Tmwu*Z>8rk{voh z@qZW{41w40VyjGd6;`%i!y%5SP&fL(i&qQFKIJsRHT)!V*}Q8ZuwZG;7yD%ygEK^= z$E1Pt#%op0x5{zzUQBYl9T2ZDmQ6ML#g;)QtoL@0%hhE&iSy<2>KdL-c8=?x{zpN6 z_k0x!NtCk-ms|0`9iE4@n@Xx%Df;j9dsDSIP|e2$>{D3~_9GSRT6p}sd#^EVr`LVV zp67sD7nBVIH=3Z>IP@OKHsN8Frq3oKB8iR=IE2Y@^PIAtV1B;zw|@^F57-Jur12<(MY1O4JUd%WtY_g6$G<-M-=B8(##y z_2GSEViS{_**MI)aN%SxhbziMMew~Q+$4=wSK}Y#ao8iwo1Bk;Xvmy@f>1}0(G>HZEH&{=52_ zb7fkSu0@jfy_4{p`CT^_?}lW{whJHeLmq?w;vx74V6`^&4=oDWg6w*;)R``M_1{v- z!&96?4d-9Tdcq}iw<4KGx6Mj#%Xc`(c4E3|S91IW3 zwL3=ne3~zG5u(CsynnnD1?rAnczaBz#zYx&v~f4yZwx;>l0fao{hY02CKI@O=kj8H zLv55{zB2Dn`g^B2)Hos#at^|;L5z>8IZ2$KX3U|~(z-lkt_dFkI%#DQngC&ML7O&v zDe=qi@OrbUY;t_kx4zF{3FF_tW~zL?U&TkT$z7e*F|=5q;2p0CpX-c-xT8;nNvH3n zo(6d3pyD2?HNb(d&P=|khkl7SJ@2Fjnm%U*!S5~YLy#vC^YZ_W6g5g(@)mE?O#+CBGenkArQQ;FLR^N|`OGm*=3CvfhKV!R`Z8O{c3Y`7*(^Q$z26VjIq1jEv%mE9h1pN6`w^FuR7DxrcDgx=m3h~tO}uuI*6nM%El&o2oa0Z%~tQ7Os+q0m*u9kvfjzT=QbCNJai zkU1!A&sfhVM<=sBICPk#U%1a4Wg=m8|KMwRgdaa|#PRLjWJC>|JkKGD^Ow^n#|uRE z9Z(`%5o>lARdw4xQ6TGeQ_!({4G#FdpPu~BWtFvkOcb48WaYK!=3>>{e_a->IKQG4 z&Lz4zN^JTWF1iXo3A5}|KacIZ+5Gv)Gb=TY(r@ z?G&dyn%2De-vQnj_)VeyFT2{YU-$-7@j3bUMInpd?)BD zoR$X=eJ1}!0EDVHsxHM{DpPxn!>027pQ!)<&}_uZy-ZJ7JkGRxRUgEf|2i>fx&@Ts zE_N-)-AZNb`=Z~`1scBAr6K1oC*91FM5O|bjvsqGdVLY=_47aNEslq7jr1*QW!~|XaT}?kDf8tfo10Q`@oS<|AHcp`jFa<6pQ7ek{y2B zqcrJEuf-8K&hv;3SUa{)E$8=wMG+6hW<@LZWz)$DCQ4CN-?>PG?Zg_Z=S>M;^CU#D zXlO7Q5=jm-eDT>1w^)c2IGz=P$|7ogD~5OT|A=5IAAqeqL;P)3nc*bAw-0xXyoMc9 zlmSdgDLI)g2mw!iPD6Nt^W;4cT@@(0=4{8O4YDqB_MKwH)o^}xqd-)yUX;=*tr~Y+ zi(7M~l^^&|#*65@;B+KY`t+o^{rOODDW-7goMsGCq=Nzc(X;WZYR z2T`<6QK8r}dV8f}I-;&hUW3rwSFA48sFC2dq&faqEdbmG30Z+=B*@rz*F1kjJZmuM zp>dUibTX9KU24ES)9az7`OBQZ(~`+8=!DQ=5_$9X>z#dduYC1(o}h9jzCF6rPQp78 z6%*Av{<}#D60EuXl8F~VVUy^*z-7dHFGfi6zw>MU7vbssDK?bu845P-FSXd`dcU<} zuFtgfPM2mLzZke2E*Bd{> z^}bC#I!chs2PoHEj`l6~4;gsiCoqyZgs64HjVj|7)~U~*yw&RvfcQhj%*cF&ZssPjfBK|~*28Bv#+*gEU&$Az$@qA^Ng1kKW9Q32Nd{sIpla`MSV~1Q zvSRS;1yOy&F4)ykEhk;4XO;xg&SHbd^J&_k4ZK7IB$K|Jk=Pzcw)_)WK~%V(??;E; zrZTuLV~U`!CnCPpk3V`bUUMW>EEIcT75)!kG4)On#k_z&aJQ$Ulo=F6wDy&OQYSiB zN!D8EbNdcm|Ahw?1OB`J4?O1zPaa;`uKz#CO8j}`9h8N0FQmWFT{6&3)EdtPZS274 zg)F!at-emlXnBE!!Ih*e-)g6v%rW?-WKoDc>P00N(Bv5qyNzyP$2Bd13FQB5rHutU z9B~J8VgC)Sm!bexycUH;h)8@QWo>cycECWLd~Qc! z6CiK`gI0wr40UOHwakB4cU>kPxPe@~WcUA_0uV8(c;hR0VBsyKTVk&LiGIp~O%!Z9 zoH&}tT%V9DH+hJ_O|F`)7B&Vp# zQ|Egv$?Q-9e4F53A;E-?lEgxVWJxxBwKw#FAF41;R8xVjo;H<|ANkfp%W=Fz)p>2J za9BBwYI(L{MCh9Ky{Ies>MEL2OE&xqCS*tF68bJYZkICM&A!oq~9AID zRydkqu8lR1<2A#3H5&{c&$O?x!}EucPn(p9%-|E?)KvmE;P$bUWQe8-Yf<_UlaYw9 zyG>xHe+66vG!UX1jztuE1BwA|E#`YT?sfsa*zK56`-^gz=7wc{K*GgxoMQxh9?q;N zzeoNeWnJ@s`g=GdbhT8ArJCxddPT=&t+co{^!v}wq$^0u_j-=L)OKrppwa|}>BG#z zw7#_fQ-SQm|Iya|#K6Yv|6rU^x=f-Wre#pGCo}g|P-Nyd_vk+wttWO=ADi3V@}eLJ z+~}}SrA(SNrf;30(hxXnF_ziC-e^6cylPSo_?w=joNblSEbW&ititiXWoQ1Q`#!EL zLEAD0KdfcB*OtwW_$NmvKJflg^1Sp+=(YzgTcsBdeL{R}>YEU|ivkU|j_)JEE8_YQ z1C###kF*xF$*95&P&PV#^*G(QT~rLtg}uB#slPYU0W$&9k90Z_IZP z---Z)C0JBe*$SRS_SwOM$C&NMxxNl1#KSz+ktyGCMD7zrHch-9=R4}((pFl_`OHdx zya$JjA=LC3F1#c4VL{Su`@zR{el`NMv(vZQ4(cNiSSIK+;V@_ zH^fF(*a;v4hP5lCY5nU-Dy=Imcu_Za0PEqC_B7H}<13zP!%X=;t4T^3=6Eix(*sQ= zlLS>W51sCU!WzjigW-E`gCe9@#6!qt{9ws_qWS-|-x<`Irth6u_lD0^5BC+Y4gSqA z?d98{lc1z8W_gaCD!1FcDtRsux4~>V3jn+HwmLBilAdo@Q)^%>3VfD3W96Q1S|9+51}a6-_B?y0g4Sd9$~YN7m!OO+}eYjxwK?F8*72 z?`Oz>1R5Te?CC1!&2E_I<;^#1-yHVJ2Kh*9LF!fQEWVddPI$LPqBL5j-m1-eozAhKOKy z?hq^9+^mjvjVccNDmX_&|D+=u9)JYFdv*d_OYy?TZHTrgLmzK=N_1(8Yt}|*zxUbF zOD-}mOjYf`VotH=nBR2h)7n17BBKtCU$>1%3RBGPtIO-H$DVUMvUdi~u}b7azM!E| zQnVXB`DYd-eOl8+sm(L5D*5}9bxF=zYHm<2TY?DA`#pTCbU8yRA@%jw-9Y<6_`tQ_ zNSxYE^cOh2;?t;7u_)W&@^--U!4lEWyQ20p7Xr8Q5m#dtQV^YX5TV_RGMKGdH&HU1 zcI8F}#o^CLe!j!QBGAF`(4o!!-jRwvZtNzHA*-oz3Q(3!kogVOGgLJ z_qFvp8fV+Ry$)SBBe!z^=jE3OLd)C@JRhV^Swqg;BFGW*&i7mhTkhHuoQO&Grsg@f z9Ra)oiVAmZQ5laGsTq6A8j*Rk*1$x*5k%r^W8Z?`g2vW&Y~(R^nFy429iGNM6xYz* zp#Y2a^r*MY3C(UP^0^EjJc5b@4@2^D255dwjAWqU-(trW?N6-t1#9uC3wPI=P3mE% zBVsvafRLr$w>^6LnPfDdD?H8@LD<|q3DSL50A5*GKcE;yFLcig@{dg=?)-Vja>!TcD z0-|h&bD-Z2z-^LZq5fK6h1-CW*$JY+dC^(9tzP0cF#YlNLmhxBd%6gx#$DS%*uZZFJ;xGXKLj^ z*9cswe~`g7%iDEQ1JV_H^6RoY)$&bcXeg6J5!-}bm} zSI%osL_F3YL)^^PNizAap2iC;2QU4ttNSGz7v91u0jdbtX?P3ZP?r7OpSPSnBpgOy7_Oh&@hPQysnRcfKmg66n2%Ix95J2dwx3C9_hYOQ_-c{U z1a1e zUqXr=wO%VrS+p4-`>f^Z$%8#9hjbp@#>{o z6xWFK?ppXLyQb8QWwkhPCZGa5?U&9{}3bPp>T?A#p626mOQlo9|F<{A@s^}am=Ooam zH3iYI=8JGGM@c;`${~7Wwh(bz3P%CTK>CqDNPBvlq+-3-5yhOmrdC->TpTU~1p}LH z+hab|EbK0!7pK8habq@?c=52l0LCpDR8dQ^z|k>&K&S1q9{%8{&m>mQ=Ra-F?x0}6 zanparRQv_P#^l9F6OPNK#s83Vr>$4bq1!_SZO8Pz?;mJTe1}xYRNLm1f5{aq65;qc zUzwyR;GWiOL#8FifBuEbs`<|KO>) z<{YBife+kj{ri0o3$$A{VUQs`edFLU$=W*BvguS=IX#ooN$I0vP_Z|Cztz|-()ex4 zw!DuZ$PRrK-p>m$W7kK>{tasRSavm;%_f>Cu&tDEcGaU5e%55Weh16B+IMEI`YlUn z*9huZ@QI5nUUp?R=$s>H0oG4{LF}hc-OO7=iZS?Y^=3mkiSPs&HmP=;B88@LtzipY zQ2B%I6X6h$$BLjtpZ?+q{~3`#*_O;xJP|v?Nv8wa40TXHz$!!Nd_@hy*Zov>DTPUb z!}S4v!hDAdl>$bGG~d;eaQ$XMZ#R-RraEcEiB*5rV4kS;Ztde_xiefTq1~V4y%gmt-mGoc+>5|54o*<12b38ZZkf^*eVefl;D5>A!QkRccc}FHI;gRjN+lwRoSf@m=D? zIv=ZZm>z#>8QCQBn_cN;JrKcGc)ywWA|_Is4R#rw63G9xC;QsYgxW9osf){JJnjcp z((j|ormA>8{h&`+r8UYBAg_ZTq?Ks$?z>eU75t@{VkcaIU2>@P$^QfHKoP%O$U%lC z>wAi``=tDqANjyvE+0HTrTn2n<-m=i-WY4&dsA1ajQ{KtU#iadF1=5e?SIs^Ku+7X zN}KVWd@yRb+&kjUg8YUcJn`5=(xcaL@p`K@n5|10JZV8{zr-K(wLuise9}x(TAqF< zGuA@|>Z zzr6ULr{xzrcawSg2dx-_dHO>!+>^o<2Gtf=3^C`si8Afgr)0*Mmt@|w?=$_eT}g-K zmXoSwmzCDKT}f+QbkeGAmA#3fZszAc?cyL_kGe%5)d?e7vHUX-yF%BBg-(LIv;~)QM zuV*GjE9&%;OD>T?g9fQi*GaeTzqUQt)m8YPCF)OT8IFN{1a#mWlQID{MCv?ghBk5h zv^v>HLQ^Zp!GHur;@=7Ykr2@HwIegIvj_0_5EbeR7g8(0dLj(vgB;{j79digd=T}B z<+DyhLY5iJN7ir_in-#MoUFJZIgZApzg@ue%?JDi7n{;!cD}6+0C-(<-B3ijnHv!?hDBSBODkMX5| zW0u7e#w}zulJO65J6|V$o6m1y9|dTkuNsFHXRUR)`ds@kK);Jll9_sg;L}^LkCJQk zRn6c1;U>M1>zISrlOao$Gxh9ORc++@hqeZnK)+=82n1%B0rEMPgz=9vyjX*L&IN$4M(oDV&2eJdML zlnoC^`p>WYGh3GObumRfZ+-dSwIA`wqgeGz8ZGN{h-37>EVfls+Sin(T{PBhU%NT- zxUM>^@6=dg9V>f=J@%mVI^s8Gi22R+#vGX0#|NR28{ZF_=>13B+Mn}vZTugUjJlh= z1;k5IpC8&ApRwxRB0lh^laNIv!#2?mKA-m@pz(Tg&mGqq%lYS@?^!DrKkvNrY|`wm zTd$Jscl@QkTGz_{XbXjCJJ`We3^hF0&BcCT&kUAnFaHZO_R_k2d+EHD zZENGBwT-?jsK`|g6k8_gKeQchJ^k|^_sTl!tRsgUa!A(JN=3SK=^{rTeYC9}d}`R= zWv~4XwdaTC^Yjn!aphp4BMPS1vAIEykM0r4;R zJ-`^FMk9ZAf=oSZkc|*H;)js2LSu=e-Gkn#tjlsS4J4}}CydqV};Ddv1lrP4Q)3F49^@Fuww{Q?QBIsuA z0GAa1Bd##8LjONqda6wNb|Tfy;5IvSl^u5cmD({(l!g7D+L)cO{2r5yW;87kEEtRO z1W=Er_%|b%9oX*)(T|Y`fj9O45J0d)LyUhI3pwG7m6NS?vK_A)SLrGgyyCsukJc{v zVSpin?~+Fzx)Za`_E7+vd`3IrI3}I|>S4JN((i=yJ0V0g`9B!2q5MxbRSM*_8<2Rh zwg;Ag&0_V0@OoYHeBsBX&av&Q&%J?H@16V>HQpTf*;ea@5>3=H>fxJ-! zz62zWH4B1fFx9nK+!v)OFMNk$efH_c^3exl>fKt6Db}AGK1?UVb;EMRy6SCq zfz9e&>$(ZF_UIq+2=NC!;(&H=90c_JL^F%}JEcPoXi**z;3Oc=XrLE6z^iFNG0D8Z zSy{hmJCfIdv;s|{t!O(BZGUR~5k{{mlXd^dXPz7?n{2X)bnDiwKujePHr;en)8&oP zFUlulKL~wG^@ZN49dKWX4fOHH-$}#nl4+xcNeBHJ=f-<=mu-3$cb22$0j zV=Rgi!MKDFYI|@-2;3&K2({jj-%Q%8>{U13M1HbcH`(rpKC;e^y2?!VZJhj%n`O$F z7p*P!p*gk{4nTD!68vuye%k)Whu$xZhKpfy84BT-* z&LmnX{8&oMqt-w}tvHYf!&&}ss3p`K^syEyIgU#iL^-w{(EfoB{~|wXswDkkD+CW- znL_7kp|4uCZiAz(ed+Olp2y{zOV4I$XT0g!3vB6UxUoU-=Jc*4{o%ck0D~V1p$8J7 z;l_i9{%Uzl->-g7U(HNYr)8J7k115?$}$|7Ou({xEZYrBcXRQW#xJS_Ewl!bo8mvX z0XC!E2!pN6N=gd0Z0i5CLos{7R{1Fgd5pLa>$&j^ylMQ~g-8FogNa{!7yuJy7%y;O z(tV4qca**NKRmyhc!t$xZ@!Kjg7BJ6Q~U?Rzz#Snss9&sa3dq~CJ2ytsU|Yc0S$N1 z9D{=FE{gJ|%lbofp3%UAvXMWY;>;JfbN^GWejD{nnChZ8j9-8Gg?v2reJ`SZ@hNZ0 zz8yPDM?E@sR@k_A%( znqhGQ`$Pt_zv7ww6){>yi2MqW%Gp0K*{}U9jQ@iSYei1b5`q=jL-ispRnh#0{CEH{ zT1J4c!b2cp{pA;*O-0o2c_WDRH(z^6X3Uyl5A8Nyg=noeJv+8-seQPmw5bP)97{Q_ z!fn7a_bCEAM~FCRnk+9u@)71`8h4Stsm#`2QAiJ#ji<4c(lIVCVTn2smQ~Li#hXKM z##y!Q&HUuB!M2y#kw+d`R%A^XX6lOMapT4n9An3hm9gU{$RziNn3~G0Eg$uHeCU1l z%*vR}i!<0ohh@L4luSigCF$4?e&S(yLq7pqts4>JGq5(Jj@fi$ z3U&y-T5|kvolKR&^4AIa%1Hk!&+#V7@FR{pTRN|>vb1gALE5+Ps1s)G>`Q~#`w0`1 zw$UrO^TSc8N-!FJn&Ln7y{7&j#-$J%z(aH>#63jO2GVv?@k`Vr) zj=%b0+@?Z6tcigT>TdEs?0^GSAv6muCjW;B$#ZR^JK;kG;er}yai%|nhTy~>5MT^J zlE2ym5A$XL@>z}0O0XDzDp*B*ADGsX)0GFTK3n=loaS2K-fhEpz_ z00-2=@*?y&^9!IB3x$9{RHz-WR{jsgv)`J;XU5!!5gW&~eKoljVaP|ap+EY^Xc0#q zt_}F(%52?N_tjC)g4A%S5ybk;nKR^VeI0pjTy18nVd|*y3SBzOA>B5TOHbTEe&1&s zJ1*FJg@u<&|DeMF2h;;7q2c;N7!b`j0vHvI_=5cJ`PoM}79!7RkfVo*i_mbZ1@-u| zNUU%GouE6Rw@p#iV3#=eLI}RF0!qhIjW>J%;K|1a+gk*^di5$QW?_i$4C*awuDNEx zvF^I-%DQW=CY|*UFl+f zU^!#I&86c?Ys%I;b(3Dbj*_E}!qH259lWn}+i83G$-1lAAKKvGTZcX`6Q##~%)hcKG~IL`fg(MY0!~wD6N=>bNi-q4T2tFnSUP#9=E7BaAU>G`85&0^Tg< z2YFbg4B3E&n=JhS83#w|67?IKW5<3$frfU6w&x()tG7SOkkzL(SiPu9lQk#8fG;`v z(}#Rj6RVYI zJLu#X_ASGv|M)s7x><9LqrgRoGva^&Z;F4~f&HGassC$`Seyjd7{)(~Z5>DX5fQ|E zOWo%T@qgs;zmt_$)xXju>6?jP$OUKimN|NZGBlq@IgknH^@QonXCvYm#~<8OvE6CDH#Y2xY4x3d^&Ed z#AeQJ676Hg5xEh941GGf`-&U7r)khAyz#DPQ@ z&T_xCiO*cA#*J~z0Q5Lpo%Tbs4SWP!MIV8OW9p0<)8rlfP^+<+J*NG2J!hiNrg1?hwo+d;J?XdqhHa9H219M2Y28q&jx=$9567U;nCpPO)l~y zE#^cOVrAyZD=&_)(}wtvL-p6A<0TI&|>XPm^6Fv-kbC;v4^LUp@^-d`YO;Jlzj5#w=D-V%)@@XD6KY2mPYv z$%P*Vcx{wiec9=9_667Ln1L5Y!o@4QPBg`T91|LA|BuaU`Y-QA3V&AJRYR-Y#p~2uvbWQdw_ADeB{{| zJ^tnsGIr8A(q%c3mYr9#Ni*I?*JopJbBqbX^*7wq{tA!1SK%8=ggBw`qfK!?V)t*j zLA306FG!EDY=i~=Q95ZTZ}*XP@IR}CET4GO?rD{T!^X5TA$208K4wonflGi#*JL>J z@yvh+OUT0ttor^!EZb9KxQJqX+{Yh=*N!yi=j$I^=QoeRT;qo7D_K1@mJR$QS&hrq zfDwHKz5^?v$C1zEaPuqpqdMjMZ~Ys*ltDa}k095jC_D1A7@ETPB%a13=u29H&OX^0 zjS?Yv=FSf*tVX{w(pK(t?b_8Bt3Q0-ljQzKju-!Vnd+I7xRZ9>ngYEZdgvi}S)be1Tvy+m((OfB%#YI_Y8+c6qpQtk>zfd>X3nz8 zs@k=)RbHQsepM&VUX^y8mzR#Ktg8E*ttjo6UqRX|vrO1-M}rx*%;JisCq$IGIf3d~ zb7ZD|DR=5O-^kRjzmo4KOpw{rr&*ot^=*wAidXCN<}CfAvhIOme}J}yEST#~p8e22 z@Jx#Ou@W15BEI_ab0(|4eL3OzTV=JKPB7W_vu|G+cG2;&^Hyu*`IQWKHTRv@UXaHg zxK}Q^ZHSTY{p;KE=X1IivfI^jTzwr;U^RC@>KchCb+9Vq|mMH(LLDc^koId;X zBe}Kz#Z*6o&>OF1;`Owat@Il$`YI+S4fy$C#Gu8Y1!gp|(wgEwUU1mq)ih20-vT4V zRx&^}#_I+g{k!pUcodo=rB~|Iy_H{n@wu$K!N#UPo*j^A zOF$%;jx?NOlF|GX?FnELoF(`(H_#m>6h9%=)&H4%3XIU-xF=Z^bxJ)fKBH4eqaxNv zqMu+7?>ol3@hT%G&afiniwU3E13hId9=K>?{WF~$#-zB}#o`b@hSXebER>0pW^?rw zvycAwmHbbqA-ZkUuI4hzaI&A}{1@-zZ2UrDbTYI%bT;_i&Y~*RFIPT{ZCP<}`x`;x z1|h#lin5UtySOpo-ki@RghfhM3<)}*UlG=%;bwa^3hYap!${~~cy(~ho3EIdop;{3 zCgN((^MeRJFM}%uQ_HI5WYeyP%B^qgBfFikqa55{XU^U{S1uTQpxk=UI%KOCNWHL{ z4*E97WV8;X?pOZFS+pYHO@1-%pw>lfV61rcRzD)4%)H^7zwQw<2k+lWVPX zHCxNLZ*?j%k^gLO2-+r06D~G%*zwN%Eez>4?!pDNP z>UxkJXJ0W^9=-8md9u@XqJLwnQ`%*hU1ZFd8)fWAV`Qt&J30HPAz%x;$mE-D7CM7_1B&W_Vwm8L)aP6$1)hYA90 z_UxH*Y2Rb?FQ3!%Y=D#9_B=#3&(~|)fE5HNU&2hPCdteuyy19BsomXve&!u&y}5SlH5 zmaqX=Paw{}S?rU;XxE~M^=A5&+Hb6XrI@HUg8TyFtC-Jgk<5CzOWAC!g2IkOa%Tt{ES*I-Gf0PLQAi$i~Q5Xm) zhf|g(?pVNv1X(zUv#%MU0)MeNwd=YwQu2i>#b)S79P3-qA8%OMgo(w7PF5>z zzOKHr1Y_y{j-5J5C*2ziV-Z$?&73}6SEJ3)W2Qb6&X#H4Ov~bHK`pZ_Omsm-2i;$- zjjoWwfr&YMNDB!*!$g}^i}omvNjwlN%(X4-N1-0Xv@YhbGWwjVV+O{`AHJU?yZ&;Q zz)}ytGv6ENCd_)v)1TZfUDxo_pXDp5T5cUV=)ysAP_a`F_7;tS-J z)kn*E*=70JmRKQx(;u>Mv7R?yH~QAqB5kVX>oG$&{ca_D>VxkK&pjcN^}_Dc1A}gq z?gt$uT~=R9-Gu#&kQ?P!nEt8%@s}99mI?ZxJFbzpby;YdR$XmPIq=XEYyT+$LuyXZxr}dCK?tflZUS)Nw8+{@BSVr0LOr`mL zRhD0cb4)@*tpDi`pr$7O*UKU*0PR|m|{oAc|48j7mGoh7jK>xFz*t*a}CFYH{ZtV4ZjTuM#sRLoT z_QDo6RMS~>9DEckBh~sqkFS(ZTkMm=I%T#J^r(b)Gs=b7!lXC;pscTg+TsbunmHrNzzR#1t^dNxG9|}egNbvr3m^VL zhd?{Aegn8qxW|S+=PCck{|jH!-e7|bJWCz$337Fhqhk48WZ3w-q-&RjpEjyPL$adO z4ck-9(+N(DG1+(nJ4H_W+j&+h_|u){q3~1i8B6D7=bbFP_6Q8zxS+BKA(<5eBBhN< z^TSd)<&7Gm$=0{6bo>dT?8M4j5EqxPs}n;!18UM%>MCDtqnWj^Jt;rh3!1?o!U8&? znXb?oX-U-|53Trz2_L*_g;mj*4BEfvadOvRE~hyR&elzOZ@vCvx%jGq=6?{E6#pY` zpvy##2-L=RYJg^?w`jw8MnK zy3)r#_6dv|m@vlJfxm#Y(msLLG_Sn)H}ch&pXc=>d>G)uGY^wL4|qb_wbk1djKN_s zN#JW5|1*0vn(JS@zGn@qU<7?G>vxJ|su4$~CWaxQl~H=kwtUjSPrOt&ME9KT(xew^HT#zhU<6g)hn`omB9JW|qM7)IbLz$|$@BL)QW ztc-K8#X>AYG9k#oDUx`p{G_^~r@U_}n%PK+XfxYQSgNUJPt7#jsi894McndXKzGcT zHno(v`lC0JSjUF!_TY_&Z#-kr4)v#L-6X~K05JYdU`o);#W_J;Oi9Kjp*_1XAd3~K zAR2W6w8Ol^-ld9|f8Z3wG05S-sw%LkqwvPi`s=T|7&WevAFaQO);~gzHubv7j7qJ& z_S*Vd-xzs(=>0bNWzV&02kbSC6?gcf%q)G2sM)k11mAr}|HG@5xk!hxkP3qxIW6^% z!?r;(C_91{q0IfdB5PnyL}8O9vU(or~Z=Un5B`E)v=k#oEhLzqQhRsrNAHE-!pr6Q7!yeMTiH?;me!8RC0;^8!wM1iZwxHh$lYE+y zaB=zHI-fccv;5PiPLWIc_K`XI7m76PbHI_Z&U%|l`}Q4maI3OcGh69Ca4lQ5)>Svy zpwP#LmZsi>jZ}XQFz~}t8UL}*HtAm*|Ld5c;{?WMd;kM~T|+`&01cDLC!BGqT>1Oo z6ujd6)@v_H|9-!fvwwestukxYJd_PvnP>!tLDizMY5ZqY^1guABr^|89-Ic)z}G)i zQAx?)baITt-`8+pNG8m5a+j`*77<>fG8oQ&D=3WLAQXY?JMsmXj9p%ILjVFd_el=45A>2pGh8KuYS zET1&+6K6hXBfhw;3*#cfq zb7iB9v@_`W_mjSps|TE6+zazVT^FWouFTi+z=e2W7OAhCSI_&*1-Z5VrEb`sf)j?Y zz5S6v#qqw*V%~|*{NZRsabJjjI__np) z|E#gr2C_%@L-p%{y37}cPBPj)6xfRb@5*z+tP=HK+T*`T{}}2?f34y8$Gsx3+RXZf zzP5=;v{hGILk>Ug9J%e!7yGVc_>hMN$i|y(D|_vKct7I8X}^59K)azC5@Lz*L4i8w*fMkNC(7$mER4Ts@9R?FVA09p4Epxz~k=myU%3e3V6V1 z$d@yNuBra8LZC4x-(!ej`pDW1HnJ$1oMtSBI9B>6D;o)vO@F2ka!AX=bDaz1OpCx1 zZr&}U!hiSxVlWy46P~a;P7TKilgA?s{k6d-hf!A}iS@QR5srz%Mw`rev31U0>?4F} z`#}dg^K0sV%3-+-MuAyZQsIWjHj&O4q?{!MLNEW2@9~m4(qDk?m^EvL#asuSELvP? z%3BZ12_r)7$MxP-mT7>7;Pum5B|HP_qKO!4S?s0+#4_RyuvkjVgN-!svn*p4d38;- z*0orL6aFiI{eSkZ15m5tSP%Wu1t|(BC>AV$iv8KZ60m{YC>qg3&}XcPK8>I;#*%0( zF~$;Oo-Hvk$s;C~n20e^A&ExO#D=}2g3>$h`)2n$|NiIxKexlZc=z7(&z#-a*`3+h zJ$v?SiL=Z`-}GhcUy5c8_{mOvRxUK@{Wz8ZF4rx8;35b&Rs5=7D7Y%}bP>#|=*ov> zC5@sc&JxccaF-%N;Vvl4&bHrqL{1TAFTU_kdvNmISSP;} zQ&G9jIvO-}B7HXU^7kC0Ov4)YW%lgTPuRV8Twh66PVGHm!vFYbnH@WIaou!)ZrZhN zkNw{;c!uesS;R+`U z;?6S?MYeudsjNvaCcry)StrJ7D6q`t&Yq3U$&0T=T(Y9it-W@#bFcY7oAu&kY?a<{ zuIoDMI`dS1?<~VN^fH2C z>U7!_udzeP;xHDRbjiF+zr>{*>5k-~{l|bQ4pwS-*EnpQee6sJ)C+ES-*{Z<5>&nP z{Kh;h@611A398%$DVt;@^b0>3BtriPiL2guos5j+mxj~!_!KOD5|V;E(i{Z8rt+KC ztrRbne}?E(Jk7Cf+xpu1u0$?YF?;O&XYRA(hK!4m&bM>N3@pK* z?p`&O&z~z8!1n#$QizEZ?nj8rXTwHFJb#haX(A8dqDy%YuMbyKSxcok`<>|)$INWZ zo|~2;StEX7@8nk4n+|)vVVr?>>}^8=-4C{2&Z+LF{X+SbizENK4Vp1eYCa6(+3 zT!blJPX6LNF+&Lx$KfUD!3?Jtpz6J#2auiw(Rn6-6Z$2bFvZJ+BbksMpyQwuBtPMV zxIDQCQ{3sCR(vAgqC7D}2@}WRCFsEnrx&2)7R=-(L3Ex8;GPcr8I+}&;tVShp61zc z&^Q;&8sYGHl*^)C^t9=+y1ZKJz)x9+*CJ!YGa7;~2W?03^_cHCC2+n|H(|7JbjU;}6U zHXPtekRB*yszc%|7s0fwNL$LKG$~$+mVPNsiio#bH z9D0Hg20TtLkFBS7E`SrJxVL~=s|p5`)-U0Nh{xPV=s0c#86=!A#Tn*~Jn5V_^8hN~ zgnkJpbQ+`?VaOxn!o&Rvd~~1Z=Xv!s7$?Xr*i_~BQ%0~fvkksPwDDPI71nFLzP9fH zM^Upy?vt7C+ugVSAPh7$QTg%x!1Mp3YcBLtWr_sE&1PF{VCCG@vqy0FuP!CGkcj}kojOGq(G6+0xue9~79*~x07GZH- z7*_ep^Yi#?nf({^&@6x62FjH`?LS;R3c3yIM?171s4G~0;)CyiUpe_av?KkX)MEEB z+K`E7kF~`ZK=auK2o@xz>^#NkCx1bPdwvC@t<%qK9(LMn|8=PUuHeS|qfU}wxtw~F zOPc@z@p87rFuL;ed^w{F9K5{;P8je4bh6w}N|V|F(T)HBKmbWZK~&;7vh21`zX2KE zAP5G>`j%llW%9K)ZuHL9z0DYV^nHz-?QU1!eR7!?wfA19+qyoJJI^8LTG}p@oyv#w zDzkjtG)(m5M?92Xh@mv9M_+aMJ69EMNMfz|^=AR3QQJzv91y2KY$SPB(J$XSM* z#owT=k=T0Q4FmTlqSV%OP>x(r?~hr zZ**0){eh}!{f0WlcS~A^N`x{<>f(AwDyKxSpto}#f2Duu=SR)CIH9<~+>}W+Y`_59 zWRp#7lL4F97DI;FsADJCEmM%$YxBVZ5aTXaaKP59l;^dh@ z;B?#ysT=aLZjv|VuSYNBIbOD^%7Nub|!Rs`pOB-M>L(miUoW|P8V0uPD97P+OIs$6MbGHuhy& zE2FOY0Hdw&)@{b=rQE1c4+I|LtgiC+87;`q@?~t)mfO_7K4SlO@2y30@rL_roG{^h zbmBVu!3aDYp^b0)A5P44J9HUBRH~0Q$cQRJ^NC-e9|dsPq5Q?;+}iv2^riJ}qdDLq1K1>)1%bk8ZvTTpvb~|LuCeD|Qf7?})k}q7p%NG8lvSA*H7ZC7;e_Q} z8YWy&j{!nD`peHfaz6?yMK7L#2T%NJX&Y@^U&qSlGLqa@{R%VL_8#>Ed+)h(tqcR1_6(PrL{;Ia(qA<1HfDoij7h)7lCTUB_TAMew&gbS7`&pZ#k)Cpk&dq<*m~a$ zHga9I;zuX`QC~HbuU(%$cKVlkTe~GIEDmSmV^v|MNQVIf?ZRXGdRXhWt+2FuhTS-I zPPI~D8F@J&Q-X@fyq0qc6I5J6I!1LrzY8|v{*h;8`Y=Gv-UXdM1955)Vd_?8iz!{x z!zHL7B}nng5-F6>aVZP1Q|GSs;fEhqj#O_xLu~ZeA=xmD*uFP?x=q35?v`=31+KAI zK4>?dJF*h3;?x#WKDH$jFTKfE1+~H|2v#sIiS$`zfIO*p<-E_#UVGBpvz2iWYgK8w zjX9s7VgAaV8ILv}>@#o?F7>BvXQ1GRg6?+Cx30#@E}Wi;Nf|yNgN$&+D&Yf0o(>o> z&E+_#Zu+x-x7%*Kte(rV`2AjaTvJeX+8H+My5JA4d^xz{bD7k$ir&(Dj-<-lI?Xg= z8PW+7S0hDO>M97wh%s^oqd`Q&2;RXj6Bc}lyA?)bAx@)7jn(7AkC`LxnN093oIwoF z9J$a%j^OP$Auc8@=s0c#fg&ixFD1K&A--T!k)QJM<(CbP9|nM#BdxL5StsmKH2R2> zD0Y$K3-Ha?UF1_Jb-IQ0L75w~{QP1tcg|<_y-Q9jl1Vp*;}C>iz54ip2wXSL69K6U zpH%}LxojYoym{Ez%2})Zu*CWJubt~8`Jc(Z`U&<(yerrq0@dyd(k9(nBP8z4bA-g1Il&aCpZh0} zLHD)ES7W+VXlt0yl z^y7@Eb2;IJF`i(l_^Hl`hiA7SJp!Jkhx=9Gd45i-yhWIAFX=769BHGy4SJ$7)c;v^ z>l!G7>O`6PNv-~1iO-GAx6pKyHLQjPw)H_<3@vF=HIcgWx?OC&juqCeTL{NkZi#G_u&5c^(OI<>$QGAZ&b*#Hs$XdGa@b_)41ll zXZfB=%A(-lZFjNVh97_p&Aa$4GwPtN?_<^)oBxMpnUu|K0|~w+vtJUXS5-Nvc>9V zI<(|0Ta|7xUL_wXE=I4^l!1Zz7gh0M-f8zM2%{|Q@NYF_dwcTnKUNjJ!NcGE1T#ek zhS@Cj{jxI{dmnoFK?=^W^AC8?K_%zL312}HB)DP-GC0L~pE{MhAj48Tr7ek*c~Zbx zR!Zf)B!i~rDG3QvJdq=M=R@<)?M%-gx#~OY^RYY;5y7kpS7DEaV=(^-F2*8d-A?Eard4D!a87|fBw16oH^6B z9=d~7V5ztF_3$pqfhn4W4|-wXi~h@3g!)5Wa!)e4pf6=0)g;2$p0ixkt_vsQByKSe z@hSm9$i1;UcU}j5<@Uav3EM#K5$r8OtVCY+NR#`mF?|kr_T6Bh51?^?u&!>v4m*Ga z`{Z}o^!IMG?FQBN=djx4P_l9MG5g{^UW+O1*CA-~uuL@fuEU^N zth_OoG8mSp8ce2;&zv$!=q?ZGDLCWr`h)}`MMWJ`g6ng|Hp3l*&}|MBrLng7F7PTFeotYpL~zj8PS_15n@zy@x%wQV|RsBv(MS0}vux=ZY-RQc1- zzZ~Ez!ef8_o&E6-zo?WHU!;%w>Nj0>4%~Uih68217(d4F7aQQwx{G8cNmDL==D)7> zrCI*aP|N=F_DCIPe$+Q?OKqM9C4Rim3{3`C*`tVG#yi%Dx|U~6+dqBTW4auCXMR|p z4#dF#Z@u}7{rkDUyIA_+pxU?2`m)_}&*Rn;`=7CXt>jlV5eN~?LR$EM7~7TS|Mir; z`qFbYmBFpEJ!`V9%M}w$Z>r)&kF2%FOm6YdceAtqGikr$S=-!U^eeP-QK

      nn zKor<#qmfUUst!zLm!Q>&7ily&B8Zxc@(!~rsE@6OZtvyKI=p6GL)p67mV<`{nd>lM zM%~PL{WUx1hSwTaO*z~4#N>w?nSda}Q{3s3AC^~47}{FM&p|V8F25Xb>B4qS3nlwZGqy#v zOG3JUuW-ow5FTI%FZl7klB3eqUNTU4Qh$&!EUCu9d*~x`@2)VAj)*M($dGLzZGj(x zS>9M=ioO8{KsU6fpMKi?xeBVbuJcCQ+Hgdkg6seO_^O>SXuI07s@HharcHBx8)9!f zI?B!qs&E(v7d4LfM}Or^oZ{5i z<>w5u_P(Jx)|x{H9JDKPiNUGd1xuwbKk^VxNLR@|=!dJ>{;3@}=~@*u(@t~!r+RYD zQI+<^I3CH7%IXdF80?ZHd3((#D7)7~iu9b##TS3J9sIJl40|zf2FZ!1UT9Zfe>0w( znYx)DzH1kK^LV@VCl7dshrIm5Oe$O~{~0g8V2?fW8?-GCV*qZN=P%_|Q=Iu_`n;Zc z{GlxR#+zYrR~!vb|q)MmmO1IcjhZ?w>?MVOFOqg2uo=AQczvTqDYHVc*Y^`k&jSS zLMm%P(x><;;?pK&mueeDu4ljQn*#I~O34mL8FVrK3aYLw)D52r>|4>!xyR3tZFjO~ zpMI*kEEmga z`A>1mlF9`!K^vYUgmNASeJKvZaKM)C(MKP(lTJEGB`9%P4JGTg46GzFu-SY1?4 zIhZm9{NT16vV-s6#sl1ViaYzC)B^|2IB}viW^^rH8nzmsKey|l@o`8h7$J0tz4O?h zIV4j!+-8pPPyo)Co(z-u@}Bx;dr3$qoG{#DrWwkmYuD~RXvXI|^T{$TF4sW6oF{oA z@ADV5#j1(Lwk1|&{OO@z+u#5GciVN>U7csGe!6XKM~%jy8D6O;pK*;nfAXC+xYSg^ zS~0I(B)+RZ_Sj=SFvvp^sHapf#ri2uqd2U%pG`_Hx8f)z@A%=qPEHw7#+fj2H#iy{ zxI0dW%fli}L)&e&Kp#MG&zQ45IAp~hkFK|(Kbu+r>E0LQu3zrJi9D0J?9&U zn2em_dOo7u#1(&Sl%G>a*@@*05srlOW{{_4>-*WxMHg@zjcErdn?58(=7t69faMHBANXv$Y zOTj*${b`<+?YQecw!&Xed~l+H&D#A>QK$d+<}2*T zQCr%v2M@Jd(dKZ58S8aJxz}I)x7~f))i(avJ?+3fHt@FX?KfWWG`~pozoGP7AADK< zDhtBYR*U$38pkLdJ!mel8FV*f=;N!B16CBkITYma2p>m zjMnb2$>N6@a zaKt_q%YXO1MkQ4BjILo@@6xrKZ98mNU&)~5nbK)JOWGjp@^yiU$j=i@D`1LOH#cIY zN>(X>@{D9p>GME!POyCJ#u=h~6Q=m8a$DTSp%8pVBR|5lvcCN`_Sy4){_~%GM)s=s zG;~mu(W3x=8CZfW%iWE5LwbW|;etFL0*|&<`F!LD6RUUhVnhsq@TR z0#lrMrz{mNM+GK}a9L%L*^yfRbh9JRxiZGS%g(qy%7h-G6tHgdbz?pm9g+ z?8~Bx0#@@4_guT_+KCuMyU0d->2REYd4X-R`Cy-6#_6pO{Ps@!+mzo|$`xPaj~REi zH(+hsU_Ui9WbDLqKR0f?7%FTq36H#%m;V>t{+s2mef!5YneD9q;z7sG3BRg)hXh~r zxSt^RWZ=O79Qew+-~9Y0W2fi9Lr(B!`qdYw+s|&k+#Y@SzA%uMkFc8VKmO=_A3)E0~HEK-HeZmwkm1z}|AZb$Ejh7xYVmjfGh~Dxl5lpX~ zW50(^4D+}L7X7d+A7XQC-N1#r9n8VirLXP0 z>)tl{u+g^Dz;0OzbD#g2?R-X<=`GKWZQEF{jvZ~?jvcHMW{h>l^RuE&*lf9V{NS_< zv%GmN!gF};;w88i+o$sv*m|1{#>?D*Y%2r~qIkzP_T7|*pmZOLBQsTztJ_8>m!Mf$ zm-2J^DhjxY;EX@%D2K}uapi}8vQzmr1;{)r@8x8p9MZk(3W_GAqt2y2_eg$xKOgkB zcHDKib-Jai&BI675F;U~D_Re>UY9KFG2#I4WBTke)(hGJ7|KTK!h3bZRXGYW%@p^# z2fyl;zidG~R%IVbOEkPsUB2yaDmqPN5U#qF);EUHUw$dBcqtSQ`uR*94oYJZ#I`%^ zhQ8EqjkB`$-h1yf%4)=y*(=XYkLlIVOtY6?e%YRR<{7LA_%A=ajuRf*;9ZXIbK2^F z`+~Jt-pW7Xa?2riFsQIGKkjH;i7Sj(B%TosV+M??NV3d$kFxTLKuG76Bl-m&)EzwM zk&o8MV=H`I=HXdx`M1Hs- zJ^eEJV%*mO!wd>+ z5qsWqrpW$dj<+ZO`k>AEY*s-OVBANju{bUd{nLZL^y|O&I>3&@3wpP1J?)2AomVo8 z4DXIPY&WzqX?;D?meLiXLG{ydg_ZdK-Hrrv+d=q64K#;tIi;R6P_X}MG zl6ZgD5#kXMvp#;6>)!n;jJ<7+GaWXUmb1aK}Xts_uQJY zsv}-~TMr2>vzH#a(T*BbdK-`0WQKjp0}nYCa<=t>Ghd1Y`SWd5ZFo0S9F~>p2h*rd zTb7oua6$4&@r*eWp<`f@pwKeG9)&;Dp)TDk(DkdECJw7p_tFU(j>l&S?Ty!-dmrqB zW_R3i$C?{7d;bBO$b$grh7PEg{mgh`C*AHN4)prP_qTA0JC{I*<*C$PD;ODBIG#>u062KgB~_p(I~&aiVw4y*wW zkZUsNS~P;y%R!*3^PpK;5lbb^ z!6(r$T$vP9<`APD^B5@k5lyWZub!f`MzlWWddKw zIoi*!5MfGau#rmkmt(TlmVbO8G|SKVmZ6Qy6}{gojF3+4kf7_2ZPXXA{bPrK8?Ukh zvwX!eKy(i9;$Y~fZ~*?DH(#}XPcJ?gV9An2cKM|7uFJ;!!{wppj6cW+&(6L0N4EJE z+j@3Y?t)FV{2R0GSCuzC#~rS+*BUyv*MSeO@yg=awTKzK75q*#7WKUXZp#o z=v#*s9ihILjI@S4sc~K^yL@@6Eo1glyb^uFgS3^xk%;>xMV#`Jw;*|>IPd9bi@J0# zyOfj!j!A@Bg!Ih$STZ{CpON<1)3;mqVdFgM^G0oA4`Ew| zTQ3=F13On^2x-}Dd;ams_Pw*l*#RwY^uiC4`dt%&6)MN=wVVS=P7!lNOj0vF-YRI@lyd6@;1 zU*wVU&ggULQHiNCg%|Otf*;dKU7A+vPh4Swsh{JGP=4*>gd2Uvo%f0-nEE;12<^XFel@bC%a7%jy0mPI{86?(W3@H-#SY*KygV<(!2sO1 z?5khD$S%9^-k#(rDs#Vezu*^@x#FFZz4T)`|}f)s`2_@N3oZ-+5XN>6|09o=F)doTkB zA91(@gtNRi#tD=Aj6Br-_`t7{)M|%L+Pkk!szKD0_nwIFL5)I56be`$FT&2^L zK^a`S6(m`TSBfUygmfg9U%Zt!VL6wkQBD`-;x;9J$|hI9LY`Kd@`AxyHWx-R~-D-QgRb+pI;)497kNUidJI8+;&L zXdk`zicR^|4K{AlU0FJ#uS36SaNz??7EnXvr$7CvefZ&rcEFgi)*b_A+$W6fNejsB zD;`#1wS2bvvQ4B}I0k@r9tNvik0tZ7YU^ZZ$}hj#O<59uLjLjjCdg#{c?ZZoG|QW_ z(N<#MoiYyHZdYt|vaxZN-NhGQZ2kN9XOvpIZrhz;3*X$(&KR(_-2yICt{rEa;5zxr zyKVeod)cP_ds*lBUc7A4T$?>>mc4@$>;LwrKiC==DeiQj zN&Qk>;W>y~=SN7#_<}{e7*^^^(@61Bv}8=>MVR6R-cdeKg+Hb%zmPwtWm=`uOZ`&3 zTz;ZIvqaa} z1k0>6*lSUqX@bEF3$TbF-l!l{K#W7YO#EotZsIAPvCi{jfc!F|1c`a%(luYchG%cf z)|lzdx`S&B1{1nO`(UsG9e6(9c;V$EgWTtZ{}HCR z(~&lJwL*K)wNF3WV7u+D*QT4hk`}%BAAiZ?UM@ZZ0O`0B~BGYHd*VREXAgu+{K7ANZ! z23`pFz!FC8<9x#nH`u42eyXT-fSk1qH~l;t&>jcZv~M4-E`r@{z!uobA{#IpVpmR{ zZugzOQ;1iGU(PbS;)*MLF#1ae9_0fpoUQ5uEpZ8^^pGag0?!*dZl%G~Gx>Ze!0@~Y ze(~Ak&nvw5@|otBaKaE*u)s%ri}~@Rv70#d&1T++htdiUm{R%m%#x1nB+C(lS8PuX zJmfh4_4`_RreH}f)!cR5r+h<=^+Wh5(!EOLdOwBuvGGuScrGI^uX60 zAr;G;2xIQDWCCI8o8lp~NG7D|6cMI)Q644`cwa6)`_Cj%Um86a#|gt9!}(W8 zhq%%Zu5eF>v@w7OX>dd1iuN77zI4SboC&8-^g)Kegv?flqf*S?oc5$m{okM1zaO~Y z-u%Z@`*_AHK0|EjqJ@6im>>87yZ4QYBTR9H6E49L&ABr_vYD^G?0jGN!(^Lw$4~9O z>HqSymS8C-Uy8WT$qFb~x}n^q^p%wn0?XpyEIv`i&P(Nyhoq%n#xEB?U;du)5Jf2B zdP)w7v&SgEjHn49tWUIL!C=+UP0(OL0@WY-1hXlGr5MygUVi8WUU0du84sOmkJ*eJ zI(4?=Pda-I*m;`4y5k#RcIbW>0OU;33iL5K(~K|l{xV2QU|U7=2qseZ1nY$ZG81h! zgXF8SjOFXiEO5dUce<3eT%IJegeBs88Vo|1aNHI-*C%`kd3k*+M_d%M@!Tly z%#q|Ft~P@(y%;8!SE(zCeJ~6@_yK1aEb54f0{f3X)=wxr|NQe6wbro1x*Js^?hqS2 z_I$g2@>H9-^f^2C;K6lXmdQk~yz)wW`|Y=J@Y|u*ecj%^EdXa>QC>72nX$;kT!*xW z_ML2T&;Hyc#O5x;$_H*Ake+rKar*IEj(yJriRUkciQZ4lO!)C6PWml|-{%--V}6$O z->Rs5DNnxq*tWVKZ65Ee95CZYG?pdbSGh#9Pv8D_>#eu?K?vkgch}8Eo_>ujoi)QI z-*kzM9+GgJK0RAnE8pQ`Pqypse$ZyjoMX@3InhQA?xUb;LoUTX`|Pv*{rlj<#)Wrr z$jEJ7C^Kben-Iz|=tR0wB58eBy%8jha6!c@1?#pvGoP~(DFhM11C_ywBIun2_!5Tj z*l~!I9v(XrPLTg0BO1CGIznFZ5FYpkfBBIaaDNDd@bElfgUT>cdjb)XI(80Xegjqc z-+X-KN-)ol^ztLu#w!2EAHHW7o^`ZM|JRe{g|D}e^X<*0msx*U>r3~Hw=IrMr+e00 zA{iroMY#K?SKEwNUa)UWydI<4JS{V94oP!h?ek4~{$ zCQqpei)z!}`IGP4vrqrczVm~h+j{G7=+73_r>4^WS6kRMNXQP5x786cE8Z~{j1OUU zAG{0UHW<4DPQw_5dtkU@y$=!d&*tLLVxHs^;uawL7$J`yD7G17o|u$Fq8T3B9(I&h z@E{Ew2!I(^+z5Ef!Q0vBBTlg2-uu(YboGJW{rcJ5&px-$-hIcfH074~`rQVD1KhKa zr-6l~L2<)n43cp+CigU3j?*(W$d<;Z5;}G6>`QDu{@{I|WyWRPzC;zzWbWVrF)j=2Zz?6Y|G>xC4 zdKI&r%9WN$CXosP9{I8$jpF-Ks+AZx!%I@@I8Z!jt8K7M6L$W~)oZ^h8vag78ex3{U+sPx1m=)j#K# z`%ze-E*2*s{|rXvvLUj|+oHi4BQT(90&`OGeTL1vWK@jw_m4{N7(L!v&`65wq4%N zjy~aZ`@wh5u?Z6<*wm?0Bfk*8F2Ak24YY&DPxNc4XtgK8{m?^aJ!~wwFhN zA_gPO$yWJOXOZ%qd}cdjS4NqK#lnQ#i6#h5KMGGr8B#@Lnux-Yih_!RhBFB1Jlr|> zEusg`3J<=D9q#hKtJ32aQT|u4R{6}Yj`Eokd0jQc{h7Zy$p_5Hd(fe$*;uTqzN;Y; z%JH1`w}J+AEt+jJ z{_`*UsZ%SMB~D#Zhe25Z(kuF@Bc*aETQ-4B#x^T7 z@`odcN%+0vOvCRghv(y`^ps%*I@_(WA0P)3INPcMGjX^V;DMMa@ZLLb*x#R=5;?CP z(6w84>)CsKAJAHenPQ8$q5tAVwiGkMmM*~;cziKlJP*rtq|jWp__D$!LT%9DZ^MHF zu#Z(cT)>J@~ouaSU4V!>{vGN zQ+DfG{>+n{I`6#K%X}UeZBu}>$NbAkkz7#->Qg!u3~~no>AFyoC|eNn<&ovj*;w3r zX)$Im9(?F=_R5RX?cdM;vni-Z8US2r$DMR0_8jW(^7vk6_%O?Xi_jVbT?QOI3mSD} zaxBA`AHjTGQI-W&)`WDG=p$i4R(g;)1bE#;U~${4P*8$YJjp+!V;ltXa(`tY5V>E| z`k_7wHX(hb^{=w5h~tFPXV67Z0qYiKC}B2d8w^6UYunzIRxGw-$DL{4yW|WTJ9e!7 z>tFw}&iG)_R2-iR#~gEv&qO|9!hc&Q9z?@|X-z0WhuyUI8B}4wOf;E+$3kbSZVL`Ec@F%zxRZls-0zE% zKepU@dn{?(&!+zEk9O&$m)fLBlQMSog>>%L(|9$6d*X>Fe2MM4I4I?yLyq^=D84V9 zmm^Qkqz<6hSe7c%keA}SG=KRzl7Dukw0XQvgJvO1G7)B;j=rHVc}HW^i#6|s6NdZZ zg3refG*0S2Rq+cZt2~7C3Vv1Rr;w5`$r9u%mYFnPEcu`Q`(NzbQ%9FPZBiPmUwE>O zKQKAowqmfIb<=3OYeN0L#FoqRo3FiSryRGJU3=R@w$Z>XGO}O?y!~R8{}aoqTGZT{T3t93wO`I05R%`@ew;9yrpyC@?wOvyvWIjf8qA(hLH zFGGVoKuS66EyI&`x%7pOW$F9MBpQ`&1+$>cBb1@o%Udu{PdQZPAe6%U?Kfv+Ni=R`r5!c)6x->)MqsuXmQ`bLxB~`-In%5)2F+9lLOV}e z;a}r&URKBwi&A;3sys{8t?HIPr4MPTuT7ue7qH}pgvqVr$gl=*^$d_dVN-IdTaqJ{ zi*oQjALgaR0`h{3(1qrJ8S5K%2|QJ%|_PCehctkd0R=(eS9@bQHE@ntK(O39LyaR@Aw zc`4%R`=JzS?@(ND;k74}Txi;2X(CLGT!0}o;Nf8qIHZpS;f3&kd5tc?FXVu~rCpvM zniOzh9>QP9qw-xyD_|jZ!B@E`z%P>gkNxpCb|zkG=g*r{TGI{O}!{aP02(pXq;xaoT32eP zg}sQ`0^^U_t!@KmuIs|j+~;pDbcpZ(%{oUV@S!8d*!Z(9u$%9B$ew>~rp|cEm{s%| zY~yc!)Motoy3DUWkOlMS+F4)Q-yVJFK4+KeB;folv^sfvVx-_#IyZTK_4WKKom%7a zRVPW1xhs0fh>G(LX{f~BInAVyVhM41A1p%R^cEyeCm8OEe?30D2o+C|@%T=Y!N`by znG^0r!Pl(=OxU6J8CrNL;L8CwT%Lddg)IjU%c3_l;)7Ut(OAyCULqvUK|Dd?6jU#E zRJ#wJ!H@f!mD=Mh??*XQj;X?_OFBZogcC9VPSEijed9^OK?Z!W)d5r7{)2K+O3Ey`1u1`HbIKE;U--!& z5&B2yxcfw!7?#qztner&Kg*x=gnm&^#0hbEScJso(zahXVTy~kDZqT)DkqETT_%$& zB`PRtG3Y5O5PqD2)d~YOzPu01-wrzLE4InzgYDjX@AX+q8Lvi+{Oo5xv#YMU%GT?% zp&fnPX+8+Xnb^J`H3pzCj|^qcVUN%c7YynNVJ$f84l}h_M?EfJtM;93e(#O!lYu+h z+gtBruWUEkW(*l=Z*IQ3y}R*__Q3{&?W6Sv+06AevyXdWpsnY|fSdUJN9%84?{7HN z-X6GsV3(f}EYDN_x`Pe>O+f?v1~J|U0v zDb6o_ly_oEMUZ~QF7PW*7V}$M<$nUFzFm0MQNH}6SRV~}H`>lWW^m;My6tWkUNYRK zOqxF&SaK{6Dn$U?pteQn;{ z&y~n(htsD_HeP0z$FxV=m3Ny|tv2aY%$M{jRtM-Lcf^)=U(doxWu`)G8S zeq+0jI2h?SrRU#jFRE%M=$#}ykOj-VDESL~G;4p+i+va2*~_1rd_k5}3A~gaGtG_R z$(oQ(cuX#MFial8BR#rR{@8KoWbnYhZ;0zJ-SC8m&&)v_cKA;@<08B3r`OuFCm;2& z!gGtX_3M^nmV$qUjrlAmM_p1~8TTp-U*C}szQW5%?}F|hb>}!pJuF5Mp3zqoE?$CQ zn#E1v6_Z+z=2`{C6S?cKLuk0~`CAl;0{AjCn3j>X=5U3^AKCmd+d z7B9=d;6@jV(yLS?)Sg{y?ynoUq$`uYNwv{9qWRF zaXO-oRrpeW%uK?7i_c7Q<6^$m&%zQ*e~yF?Pn@yEI@=ZlSDc}?c*#OrhG*LnY%{Q8 z<#OQIxSvZgS9bCD;ib^~lDIU}KLACQ$VOX%A1jB?WCPCO@vtNoe(@s!2lXm&$O&iD z67oK@3t_pa_g%y=Gj3;FC|vmS*}u{c18D2%@nEf`XkRwlavS^dSI)4zZoAs{-FIJm z>Zzx!PoJ7^${$5*P{1uFjz0Qm`|Dr-YQuKg%MKWGoX;%oh<1}(P;hXOeIA-S`vEfh zFvde`mbFV!QBTY@z2r?8NmY*oNm|Y&uLI>s%K7J04|H4}z-maCxdinv=;!DVUD-Q*K<&hoYwNHNTzw@m#k6g_IW@f`LIj8LL zw0mGrh#?p>n_eh{2FEzVj(fS?{PVwA-+mh>>8;)6p9uPewEt%L3mT~J_V0uD-oi|) zQEPO-tRP*AO}6uni`X1hWMqoV&v`@fGKx8r*6QEz)Iva>Z22diB7(?8nK zVY?+6QQH!ro;~52M>0JNMVIvhoQIyY_5J)S^gNQRHbPXLAi@++v&#MgyDKel5htW` zUE^h4x~^pVRJliR^1uUr>iXn@15x^$BVfPV`zL^O!U^eCa_%Y zVdjeu2NGygJZ*5?$=~!Z@Av=aPWaV+Nl8q!ELaKND{(ME-nni=Rz1k&LVhI|F!)oZ zC}@V$uh?lbWtW+@@45htq)N+UiDepw0MTYNF^L8*f8`^lM?jm@po+X@r@C(nFqMPy z_)8u#E+`o?-OCz5utz>Fr$3#0|Q;_vmG3op*)Ze8Z)_ z&zaI|43INhjz0br+xtsn{AHN4N%+!788}$c60=FXGEjbHSO{RIvluq}<99{1#-Q>f z=owL-d84u{SH4w+iZJ$FXp9EGZRRg#4n#g{Bs$o8_5)NU#jkr z#L4)R#^Ki&_lrCc+`46J*d#8_RDyl>z+O_PoOOv^cg49je*Abpt@xBvPKl{C9`Kvr z{KmfgoJv4>Q`>zK6p-SkLLl^r>IsT#`+k7iYJn zuoc45;CT^wF@d!p5^+AXdC5~4*T9D}^2D=|^79$VS-HFIHOe^o<9ENl-Ir-T^2j5$ z(MB7QRikk{5u5wbO`bg225q$+mX?0Q2M<}6lqoD>$H6~QZb1ql`3dR5ucKZ*#8^)0 zvnqsAhwMXyXR-Xs!8#0@F$cK36eDCRIYX=zj7Y4a>WDH*>J&yuM>_fOUU<21(xk2u zJr#pLDu_%Z5!I7myK}0)_wM~i6@&K(Xa<;X6Xz0^y3e)M0KRya?cZfG7Ai7tuB976E@1R zxeUG0WIJ})Wc$laQ|(9NcPbo(-fMTear*6c z*N}1bmS=97{tdLXKfdd6+hnsa!?w{r|H9+5-ehkGM*d7s+Go8`2zuL%gh`@~XVS8H z-+=j2-vC_^?_3emdD{>n@6}FzT9GWvfl@Xpzs929s`9});dMm1j8JkA@=Dtg zk~mo|j{C-E5NGVPZb3W4fif%Ga}gt+9ZP)>YQz4U*w?>#x!rKp1$OGGr`m@ferOk8 ze6godReo1T_)Rz62F8iP>WGmx`6Ik?V2m!KEr3JQtx(^x4p zXKA(NNxgW6a;DjG%qrs^ZyXTg0R#LnvYp#VZBT z-Kwo)@!^m86a>y}(vQ$_gb9+4zJkP=aX}Y?ckm-6uRtFl@!$jlt%pX1kVZV6-bwNN zlNI^^06+jqL_t(TnBvhxd2tz-GbHLbVaPxRnNFaOG$rVn#M&bN=bz2CbH?v)b3R{v z)4{1@cPD?x_UWU}!ZQ2QZP(b8%dfX-ID9L(3On$u+jX;A;k$3mu=DT0dKwN31n2|L@c3w~#d5(b?IpJ0z) zGCc9AJIW=h-BJ!7@m9PE{X%dCNeUNiBI4OiVBjZ7 zp`*+f+X`%y%a;h>XdD|1b93R%w-{;{Uiu^34|^)HlV0C$1@|xW3ojKJB%ZtQV&Q&- zxVZnBpyQwy3~|Xf@LcIHCB^bfU(^Ees7@y$UR}qybmEtXbua(OKLG#D*I&KxCL~P) zMXx)wL4vV8%H)euuzv8OE(HTl5BjP99&ibtmJJEH2N++1xpyC548x06M_-OL=I|5j z>*riz>-Ola=xYRSztf&}!6iQoFY%~{i~*v~8+u)#LUpOG$evHaA9+f~#9DkU%so?!s1O`8hL)WY)=`~qLr zV?I+cnB!Mx>)4?)@Xpq$Q&)s_@v9?n4v^Eoea8^qzC8zjI=Nokdmh;yfNvbw^8quo zZ5%Z7GNJu?IWhMk&xE}ESrMU^z01Jy77n&?AP)I&7507%hYNJT;XEDez%j?$5yzft zGiT29fwQ0d=X5Byj@ekW}+iiWoY`Y!zuv5>x#JY4{7wu%wd0U)j+6Fe+x@9~8 zHP&nCA(g*aUqLR`hxDRvT1QlO2!)r^NF91k;o-&v*!UC@axst|Asa|QR@NwFRrH}2 zi${p3IOLZ$n7nrnrp18h5)}TCmqQYOPJXyk-jzRZL}H!;HI0xjE`fiZUU^#^<-hd& z6R79cXyw)Utw0q?8ohUmvbTJbi3i#&)ZHXgjpCBgGL(;(zMldkY<_Al4|-+KK-oBqrbp~^Io-|3f+wy*Vn+U^{` zU6$kB&%DY;Pa0Nl`5a)+QjqxsMn4iDx9zLRvJk}l~LkV z7R~1`Pnb_%yan^|^YlTl!4AdS`=YoJq^*6M$F1X6>xb-UI{$WFh4z*iuxy43}8pY_V z)p1X*lTJFxUVQOI>%Y-v_LY;*vp)R>WT#|uA2ZIl=4@-u^rAsizE#RkBZBi;OrTx4 zGR!Gzxq{09F|!rR9JwTuJb=VO7;H%ZJd^|dsPlr}>9316={TVFRtrlonKulQA$cv| zWIjmiy!}BG_CjS2Jqd>6k2s+_0*-C)-}7(Y-YiG{+hI^H3=*yM$sWV^8)NHq?_u}c zey#6Q$EBLrUw^&tTNjwEo?p)1Y7Xy_}+Icu#%GsmuuFQe1weU>yd{+-{f?+K)pf;0edV_5?LR<&Zvl zKm-9op5<3L+>IP@Tmc?C8omsr5+sj;3(14O^aVtIq{^qyhkpywM8kK(q+d)YPrtUx z|MQ>y!2UMn{(>S`CpP+aJ199a^O-xZw<$6GyT;vM-x_x zs_Lbn`?)`USS49Ev%W&8%dx^1>UGxE1zJ`0*kuSj5JZ=S5qF;yhJHoX*8pGd@08=8 zWlo&b&M3p^r2yq@fwmY-Xx(OozW{LQ2wxt!#loV63$R?JopDp$6R_9WfrpH>DS!N( z{d3x1Z0?-TVh(EuR^TNIuM|i>FGqN(kj~kc@_trSv~cNxTA`K#40(~nTpr^tDpow{ zCtVAYjpI^Jh?)H=ETzv>EQVC52!MNf^nlU4$27o<`XY7nfo! z#S$bg-#ql_lWdO>`(yCz*Y?EY5BmT>J}>ok&lCIh+y4mLVVAuyz|_eHA3Ao@l1yw! zj+rH`x%>-xqzqo>G71VN0f|E&)ST}RVaDj6iNB3DKF zq^%SuVIVmKT`*-T#w}O=WaIe+z49wrvos^))c%C0POLy z5+?PA+2Z>HA)V?6A)SXu$onLmdG=Qiju)bBoxr1zfGSF;GN@GeBkmQ& z`3Eedg>*cr2rJ)uGESldJ+FoIoWUycUx^$Z?#CZTz+Et81_6Z>Q<+m-*$HtA`Jt>Y zzVJ`G@#_D{NLDv;-T9*}n~c8bdAsI}^qjg|?B)mF%CgjLr?Ga)knDD~BbS_aynQ_L zy>jN@`6-kJZoRA`RJjReJfi){bR@;6-hfBMOX_U&_z^Lo)# ze~!5L7nZFOhY;HxdAbd60AX&r<^p?h`rn;%R^7dB!YlJwll8v=gsx|n=`%0IOD3ac zm(X!2Sq@@#IBJLMGgvZ;A))A1RgUB!1 zzD$El8*uS}0A0NA)w{19a^#72@fEk&$){atyX>|vmf)-(In)er78p;^T()>g22-42 z;R@l)acV5D749D(@fFc8K>8j)i30)ini1bU)@wNS$9T;fmi z*m)?=0r$Z>WNY<$cSC)fu42iBWrnbt5YCz^2jB>SI>uEPw@;TTvz z9V@!HqzHSL`BF^0#ZsvF0_`$OZowMJ?>Y>MaD7U!=oNGYPtO0OAED0$MOuQ)4*3WM zo-9-Phq5K#d^rZc;<)5d>Z-&)rrA*PGjDXVL*kM}u&AviK@)J4r#cs9B%M0|qP)vs zipOx45yQQ#K<(-HU3#M;($1N9tqt6CYvU&Tn{K+Pop|Dj_R1@- zgf#2<_uY5jwKLB=)B5-C?*nFiH`v&|e(n{h*C+W0B0p544QEovCC?aC_fsn&4+p6- zxhX4UuR?z6fHfwA3C`FW2n(F>iv-*cFbq6$kcWeh5dsH=5B3xC&s|c-=;P(ayd`*M zukZ}z%Jf1x31<0JzL8&skrCx*Uc4Uw9dVhXFLgxw*rUgKJ_vfmv8P~vwv~4J>8IPk zfdlPFKl+g^T9j@^0$%mxxR2UyyY1$uc)t4T4BLJ90e0@iKeWvUZQ~z0ls^uX@shn$ zjvC7!^DG@!7p_XwI?ObSB|^ZKj}VTtw1_(lUbu~cxObs(Pe{iw`9+T+ek7A&IK3MM z`8#zCB%_!H{XIemSK5N~O>rddJ^>O1A0^7$lJX!xj>)b?@^gyhm6J|3ZMZ8jR~sC1 zr5(Lfw=B8Wese>m6Y8#=c!z!S;E5K_v+HBWf9HIgH2U%^&gw*X8p(B+pKX_3d#@*3 z+FqLNUuB}Y-dz7f1NKF0|E{~@Z0y(dsm!5i!n?-Vug^WqP8b}A_Pgz1M`4-xlyq#r z9wcKMe8r^kcKdIhM#m@4M1k4(6Dycc;aRuZpMQlmtsX?JcNXesrHth#n1-QK5EUCC z15-cVyW%6{eI<{41Scx~QMu7KPhN?Canvg5!Ox#RFco$-nLWV2&fJ<931G_xvOo^Z(YwQ_Z#zeJugR2PR*v%?8%&z5R(HT913AXiV<)^&dPF^ z@??5xnju=TpGQiG;F0U=74aZ_ng;K4(JOo8BQR2FM?Lwo5P1+JWr~-M*?4`KcRIop zFPY;&8dT^6(4G6K@aUP~LL+1yrt=poE=k5qA@}m*M$w!}xopW&xASEfgk3Oyo-M=- zOV0Y*chuoH#qv;WX8x|d{GVs-)t8>P*Jr#~?OB$53GTgqKOCyDHTDP`Vw+;q@s5}Y z!9hFDj$qy5OtOxhI(wbt-bZ{{4KqJ6c!pUXYRg>!>4CD=Kz@(Asvq*AiXwlnOTbm9 z7&XP6HhA(E@etYL^9N3Nyscap0$H_xW4@F>Pr&_@fD(-y{EE~;V0S7yYJ^qpE)?qeZ)9;&HByF;yJL#8A@TWEG&Cu;?$;K18*Io|fXd&jH6h_a1FSx82nqdGJ>@^>2Ul8DJM)c%dD2)KPZa zamV@IXGQIn=cW2Tuf6t~-F^4nUY5k^wjR2ZjT(InK4NT&b~Efv$vw>2cCsz?nYkF` z3XL?FY?J;QoyCA7s*DWKC<4V~J(f!GdOo zC_s|rD}OK`GA{^T#G2k|urF$brxyA;Q~pGkOFzp?#xyca6`iCVa}Rtyh-nTXx&AC633OKFRL->v%hN_dwZy|KsgUyWFnL;;mlf z(T9Fz)Bbjn?YQf{A>k%m{w}~5*#4X4hXxv`{GXossQu{=zX)}tiT_^n^CvK9HZn@l z*>)X;Q8OHqUEglT%hT^d`yAt?`x4|>{Wn8C;e&QvlFvZW)C#*`M;v+~R;Oqo)aA5@o&iE3-M(p-FtceEE}kbiSm90Pvu;;#Fu+^fB!#b^FH=jW9*@) zI%kx;Sfug(6k!cr(Lh`5lQBJal3S3TDV`Nfnr?~*HrcP@3RcCBtXM%;!7}5$79q5+ z{ELSC1T%4|8`o0;8JPOV>GT*SDoAz_*SI_nO_{vc}O>glc zq^qRg;GO+eMPF5TK7o9l&!=Bgew75ua$ba_Z9Q#~_j>uy+Kr+Zjr&IONESjm`U|fl z0d(z6q|1-c7$<80qd;80gJb)-lK#LQk|!DYEG-Q5!hU&bp`Tue4=!ynh{p7IlH@QP zcrkd}o$ZBZ|Bfwv{@|OrKla#T_SLU`)ppon2OB zz6ZrH;(3>+ck6u|;VAlBuDg)AUPJCoJ87RV ze~cNJ`RE-O3M=i-yXd#}wMWOB&q-@N$j0A3)~+3QS83wa;QNE`eBJJx{5%?PG(?HH zG^O%)HRbHp4chBMGEgw9)g}ME8fxO|Nn-ltkK0?9jkNwT zxdA(j09OBlW@LZs4=34(QAb)2E}QeV1(lD`o$y#83DWk5Gjq+n$;e>RY{Z3S~;2p<(3=Ulb+{BU!h=@ps4}ad- zj~@VqE5GR1dQlS6d48gwdx%#|ki9G)L;zd?;TPmFMY{JYAD7J^b3d7_-ed z`;4>67{)Kr%YEipm^~IclGG{X#sCCp8JGN*cU;E@&E|m~TZQ9GEW7Q>4CN1afXX=h zC;(kvk-@jiDUxuz%-^^f`!3=ldGL z060+QOF=mh2L7DE$(MK5D?hCdbxQ+hKA?tsmZ>kzB-5B*XPW*$7fNU|w< zQ69Asf|679PyU3dtLpiJ5f>_q>vrAw;3CR{vy*7^K5zzm4zr9gYZ=c74sftuEn2j| zxJMQT&t`uz)1LYJP?j!V)RS~V{*qfT^-I%tf0RpF zKUg>N^+R#W`76BkgJyil^Kv7JkX%zg-d7b)WzY%I5XghDgsapM0-gMvTzn!_7$F@M z?8*~=_hy=Xfy=@X<6r<%&wgyiQQI`ikvks?+*9DDb88<<(-O#8*9lgj5|$^@JI z{4v=QsX>RGWuq}@)-bQF<+GuXciwu{?!EI`8++0>3pUTdCRF}G7S)9>p8Yq=?*<#& z!ul=$`|i2H-hSidpyIXS_Yd}uwJ4RHMtEn!^SZ3Wne$x=!Q*H&& z5^*r^2g6*rJX^HWg2>8c@C}yA*nCf+AK`Myf};f-;uc-8mbPHvcGhNNoS;oPBNNXb zaim#Fe#Ui|>4s^9=!EVRmyiy$E_00IzDOgZ7p^>nSXsDdhiv38#94<i)CpJJJ`BdddWRDd-vJE zHrixMKTN|1>`=#8e>ot=0dSw0#95tq_HYoJ11)^6g#j}Tv;+Z^e?q!olQoxL@e4W* z3#QU@?@;C>uGi0QMKva%5#^H(W8>m(8{PtURmxeEMM&b%w&%71w7qfNqd zzvT)HMzwE|?Qhy{36@7!+XfqMlFi7ay)b>+2+NMA zoT>~t1CC|uZ6;*rjNZD+A2dN-1U(Y!QW&AyE^nt}d8FYfo()l?p4G!yT)4bM1Q}B} z!{tih(T{+cOi9uk@=sb*C!+!q*8bqgJQNV1!t`)qMEN;Ij(?UG)09ilLPn$KS|a}k z@4jIVO};}CuK_sL&N_5ZmhRkVersn;$!<#`<8FA^j=uSDeBFk_DR$>yIou|nS4E_f zoa)E(=eJ*NM;!ZgpVD8j6Y@?CNO*(<&GuicqEuhG`srG0uKxuMK_Nx$S zjVi3c%D+^ijli$aF$x{e(7;G*f-$3Pf4vQczaYFw#U&e}Rx?fLu9AlWVz4tHkLZtb z(-M&26A5Ap>gK5Hp{$fm&m(}7B9Li{=kgbBktSX+JZ1O5%rN!>fv;$d**VKv`x3TR z95ln=nJ>*`zVSz#@wlv$vvoLdMm@K}pv0_=huN(4HnEN~-?YwiKd}yr=2$DxgvB+? zSnb_%EY(=lshcfWXMJ0?!60kXxtn4Bj?3z$LzyMFAd}0>??R{L5pmMdF|NlUA#QQi zZ}~f0|B-#BkO6uTD^NCmjCoeR1u2(sLOH40-y2-t(FZu=co~4$ePeNWT;Y+G_z2RQ zIHAAb19EoP3d}xpo*2}kJ!92ouxU`XYQ5ZAFK_EF`dqfjnHO&795JHJQwG)*%16F< z$!mkk!k2Sjdda~9+;cxAzL3(7g9#i+LPj*>4&N>L;<;b#BnJ8^W zXd+HBxNS;j$h*pgFugLdVsWt$3Sv`BMWvFFJ0kv3XDu;{ljX!e@?hO)+b+yH_GR;| z53qmY{@egvOYBLt8TS6$WV5ZjOv@p7C zwc(&O2VoE{y?{-oi%K>X#jTC}t`AL~i4XK(W3qdyGUChxoRft^yx2$qwdTzH~pOFYM+On=_XW>{|2zqf~gwkWiLY&(1O zSl_y1+Xmu6HY6((@*IR8)i1V*OiKfwEUW)#?@ItQtBP~0*V$(nn1y{21#uxRG5Xx$ zX%vk@lBaP&#h9n41c^jm(BOg#QKO@dKv%)a#g z>Z_{n+;h+WcmLhfJu`nd|C#x3)vZ&fPHpGhdr#lR?#jj8iW*1Ek%%jxJ@$)7G7(Gd zmolbnKh?7QQpYe)Kqt+3%&{QG&xp-8JHBWZ!pAg{9ygC9aX=I}x2BvCt;#o^C7uyR zVH7+i%?itcTs{nr=E~;sDYIzr3Y86@m^GkofGAD%wHn_v^3M(X&->?(=wA-vbMBeF zy9zz_lPAfO(xdL%<&saF*t+Y_zF+?1oyW;b5ArmA>Ir`>N9v;4wXIw36pr-|-7gn^ z^|SJ_*Ssyu&rbex%p|9CXoJuh|8PS1!5pF+%tdsP;O6F3RZ(h_${e7A8zT^#Sf68`Z_P5k0XU(Z4ni_v=Rr<}cC@7c7<0g-cCG zEY2;`M?vsXb$fr5o_s@hoSfUst4+Jgk0v|c&~bz<97r}oNYN%p`dNw+ucbq8=k5b|T8GO{(F7%Pph%wsW4lGp-V}-H8*YR*i87}PXh5mq+ zdstZWy2P&+|NMi6X{q9zFTqxGa&lR_RO%~<@c16LfeLp)KKr`(^-v+9 z;mY9dvt0KN&1l|h?qlrW&LBGu3@x<9XWYFxsCOyhdss^E(A|7)n={ z9`p4d7A8mg^^ah>kToB%!XLZzPF(y^Y;Z{L$WQJGz#X&9xQ4q z?6tcDakrqpuS{upHy-XBw7Y>do_GWvv9tviP6xwPKgdMUU4I!(#4$pC~o!WNC z7tK;X^+?e`LJVFii6)>kH?Z$G4ouiXA>z=bs|>F+Szv}t;|`pYM8hJUGT}srQi%gr zr5R3|YXB+DOTC_U6V3+vam(yw|MvOj=Is*dQF8Jto|<|ZzWq{p^?7M`0_0jb<6nLt zFFoUE=UMe^`S2-6$}wl}u4OZkFMsZD<+wjN(Jn%u)NF`d`IP}+NHniY5LCz43nh4}YqCwQFK+=pIV%SO4TNj{d36LP_Y+hye(Ux;< zS>Mv0XpgfaoKM+fEqb?&sipgNj-`+b39c1%@JBoG7iXMccMo*Su)YTccj}A`$DIeb zn9{!@4~?#n0p;pPTjDU3oy2D6wT4r@3BqUU1soXPT3^AT*hJF%QLUet}&mB{T(UGaP#$#Lh>VAz-}f>7+H@nyW) z((*;y`ar1VfRp@-wYc*b`DF_wJ^BX`{f7k=EP~*@fx~@6w(z1eo!zBu-NFuZ0R=;T zx-%&j#4x|(k2yH5V37r5(C+HgF^GjUw6^vTgo3q}Bow&r&<4~Scg8Rc`ys7CJmi}$ zrWIJURJf@EN{}$Vf%-THuNGGNUVUqE(G~`cSvOO#CwfadsVFRh!mqmIueEwS)&9Hn zoE>_NXdF<7IAF1BBo@^C4s)G3?a^-JW2GkA^M^L*3o-P^*!A6ydK|~&n200hQQ(|U zP0CE$AFAbJg8BinU`&yx+4v$K;0`aE!BM_YtV6*Tj6(rbK$+Z=nNyu!)su}dSB`kz z++^EdfxJKeix6}OF72B%v?o64dbG(qq%GlK+G-FTSbq+&YmM2)2%r&(dZc7@!oCPI z&V=U8cZU5J{>$0+LcrAFE=G<#{WLi`eKK=czI*ojC#}J?@BV=N@T8;VS^h-;dEQB9 zh@AbhNw=SAMm+2NAD4bvUi^nARHE|$TA+7o2;xob(P|giPa6$F1U%KE9E6y!Mlge>Gj~WUF^f ze6x#-{KP0fVX}nRr01O(ywFC^H8U|V;5QBs;^9l2k$@i%#c;6uZp*ESQ?3w~WYyYJ zIHVCLEcD|=cRDGDE}4SXENtk}6IgVNJ98|bXrp$MP9ih;pyBQuJVG-(f7l+wS%`O> zEzozP=}9;KDm|$0w?T)vgPrlPOC=fv1u}j7ZZW@*;E6QuBETI8_D&n^3y*c+gxlV2 zgA;K4g<5sDcxL*+J+16FXN3L6ANcNyKXjPQg<}dZ$D7$?uGR`b6s7@FqanGC0SWB< z<+j-ZC27d0uAXQcS3G#94|6f)Ch{j0#j&dwjaC%L1*H9h()K?f>{TYP1ELTIOof$7 z*p>EM|1A`xo`f*O+z~nxPcACzTiXMplP7k*{m~U;i(TX-_I+^%_9bg8#o8R>*n;k~ zr5Hnu@P;DZPZf`~#0|5#w%PnZ{tLm?K#kRW4Nz*0D>b2H@QP9*$619zI? zk1BR&k{%E2kv{zSB`vhDU)X`wF6|2QxcBBkg5!eUQS5hi8Xt7gU#(Ok zjqLt`3C=eY_M6SAU(6to?fyx=Vtt{H%{R*L5GoQ6J18o4tJsVq->tN*ZHoR{+3$Yw zj|J(GcOI11>d8S(^0?Kaf`ai8Ff5#>wWjh_pe?f&w8PMb+h--KaejS zFO$4eZ9J*dqhEaCXXV9z^m-H0?(r|jYg5bTaQ=7f_Y+C@Vce3}k!_OmAI8F5E%Wq! z9W2{#yH3vI&Q65iyy#qc<0&7~MFjs``0hUb?Lj|H3cX>$GKC)da16v4wCT{!Io^in^mF7=^q)~7ab`3^$>s0nSw z@AYnf&I`F#a;OYldT{ecWV-Do!dNK7r{YKSqX0Osz&;9a6!?RWbihvc4>YWF1dhcC zINu)EQM_9p)A7eT@bY!M%f|mwjmLL*cY^7wysmd8Z%6HizNNp7zzVYmJl-eBFa!P5 zoHG8T8}A(2z)rI@vkChR`Hp&Xkb-dT?pmNk_XreY>?2S00yzq8;C>FHd1a3Umlx5> zS>!?_X2stI_FJl&A#OL&-2Pg=24A8fj&z>2^VaaA6$KV*wk!E}k^O1za;@gfmUaRpdb_LG)L=qE|7wnb>aMTTu`p))O1X*QYwO?|6(8un+}e&|EpY_U=o(z?2_K&HoejpKM<)24@^58_6> zaQsHWAJ)`Kx_*Ymvba+hxrqJvw_*K*a@7?V$x$zOx%KDApIv|aWP!TRcvs^U~NLU>1uuYxrEEh&t-yfJU~4I0$3kOex;+4tS`laKewN{>p8CJ z7$OoteGfj7sShjk>&Z19TN}~$MfLaN{WboV>9H}~X`tt7`WI;woOGiz+@a>S;yez2 zjCOlzzrAxsY5v!0+yUp00pTPU7f<#VYrQx_y|syp3fe&k**j{#Hf@L}XnBF!9wVm` z2SkBxPILQ6_u9F$g`I9|ld)Br38!U#p-(_|Uqu5DPy*7Amg=H@WWoTpyED zB!lm2e2^^K4*yhQK<_r!hdBF(T1K-Aed__r5=pjH{EfN5{6M25k0L-|g+lK?n+u|U zb~#o1+i8KGX1a}LwL63rJ79lEQNMXUaDTQ#*fX9@bvhFX&eQ+d>%*8cNGfE!B^Ml1 zCLvun4g}f94Vq$6L}ls!-X8yh9y+$ioE0X$t-BCd^ylvY42fFjcb>UkkgrWmM}vx* z6x%qZ;i)of;aZ|;xjPf~LkUv>^7YnM!|fJV5VT}Ce873h>eMFcY^;8Vh0{PN>QhIl zLpakIu}QMhQwOENa1;2G1MBh6eQdgdRgbZjAG-d9<2@|&$1;_SfYs!MY^5Pu_CXfK z?Rc(LC|syQz5Q`4e1baqBI!Q^_Cvj@(C~_qFfsxXPnC6K8jEIXG>j?}p)S8+FK>fN zZI0Y9STR?|fggpiu|L$qO7el*Wdb&rj~Xu_-gE$^Ih!6v%sMj?c9Ho;2DQ{e^BbyjlFInc1T@!WX=dU|qBP zWL-N$i-#(h+XlR5PxuFMhOZ0)4V*qJ?F1irP)D5n-jDZP)uV5hWZj_;-%Ytgon2nV zA8g};kjr>uH%ziNrM}5l>J!Wz``yK5JWI!y{w(>+__ICZ%l+97Vb4T1RrP*dYW5=6 z^k*p-5Oh9nncwZ11|;gk$!Yv*-=v{^DbM-P!3mXsJAA8SHb59lAYcbM!$R&_M1=Dw>UP@?^r(ruCJK0;?}tzK!9G<3Hw;|G=4zr>kqc_k7T>Xe)R0R zp?BzrACI}r9q{0s!b$+&Y1~d@(JbUgP}#XTLaUevbQJX{9SbRHLssf7N`3|<^;ZNc z6J%&B{<5BO385QIY}CTZp-H(V{(tl6TU zlb?DS99$$%dfroIm1XoQdBtC!sf%Xsp1PnJYVo~qe^vhMzY&o7mZ5&%1 zMW`z#VjYa6dE0uu_fl;!E+^tI>p6}I=koZ6mv*N8-}{e$tpYK}u+v}tjBK%gyZrLU zyY4a6AOG;1vVGfDS+Hte<%( z6$gHlfHo^(K5?q~9(D9(!h^q*SNh&AR@A|~XhAdY!#g0({-_6`$=61qrMK0lr>c|{?rEg6g)jD6EwUi-E=_`Wqf=@)8wKqbZ|Q=dq9U=w5EJO z6x(-X_u3Er!anG0ICWJ0On2ys;%;zM+t6=>Rxt}V(BJINbmu-HtC9uaszn|BRr-5(+i&sH1+er-Fx3}dDK@>J zKMVp5Pz&wwJ%nyI*dLukm;47^xNt^Y>2K;g9-PLaS=@SA&tvTeNsLr**YQ?-Kolr$ zw`?kGR-6rq<+TQM6T60x<{admoIKOd1cZL^Og3*hG__G5;*bJn1MMmJVFq-q%$Ys* zU-SRIFIYI+UFFR8y+Z8${ieX}a@G+~le4Fx%$b+pCuhVL1y&vXS~*?bEoV(b-gKJY zaqEq;=8jurpZyQ9Txmiw^H@)zn%Czmjl8blUnwfsm zeY^bh$Ct}9j(SO@0}ogk1!#FZovGuWod6Ymwc{>NgaFH5sjr7rUNl=9eJI+=>jLOnZ&Y5sKal3ma;xJbLZR2=j{@3hdvJs~2bAmm=sea{s!Jt|m zQV4twe4HFL9jrQaka(?5+nLO0-ylEQat!){9&~Vd&Yz7z9TT)0YMEZ;oDh9_y=|2* zU_zX*Gwnx@<(%O5RO-;T?QXx_K?ZyD$C$%D;()1S`0lWudSE={KQ5S2SB@9*uw64a zjYTs}UUf*`89IQEvb;4Or@JE?`70UVYgKQuQnEMlqdqUWj-XFQekNWM149;*g{QK~ z11*xFQQqva|0h5ER_>}@jGrlQcvkvkrmVT*U#7KS2Ib%Vf%E00qfd^7xP#;kUwxyT z^|Izyv&hE9^ZI3pe(7aVf1fPSuiwqnM@#1Ems@)E#J>kGAl4Y;ujPm}7EbYLWkCFf z^wE<6{cRr_maY0#yRCymvT{9 z1Yo{CVm4o2W zn?PRP-eRYssWa+GvmU8S$Ro|5ZY(E|PjRf##)$?VG{Y-b*xclUhDsDx1T?zhsouG2 z+llJz9VWS4biqD=*Ce41W5FS+h^IcNOWrWU+s*#mK-k8(5*Fsqy4`$k0iOR#V;Dn1 z4{iim0rCQAoAQH5K(u3iKsMzNYHiRwA+Yr`d2mNA1$2Vy%TASGvQ+`bhPnV+3H|bz zMvVhgP|=Lva91ahS)J;FKvigVazw{|^RKyAeL??~5b;N`Il!>S%|vyL-0eZ(n1VLB z{=}Qrlh&Bip5JWSs@{ESo;PPS{;UDz5Sq+PcB}rU zTHmoB&YxZJXP%X?j6Lc>n@pL;qM3_l!V)$pQ}?qE9CbYTA*C5kS}6~7((<;6AF}ly zE<93cG0@l+{FDudQub%GY-(Uz!m_=wR(aM|_W$C)e_%H5j?F1weUO?BuSE-G+5GwG+3KXm z3?Re0u(od7HhEz4R*~QSr)<4ym2~g5pY-f~h;%PoseS70t@>;LAPaK``@{*$d;*=c zru?1!?~MO8nE!tEUsp1{=N8ltmz;8RHFJ%P{Z`I7wz@xdh0ZMp?XidKw`_^5Uc6XV z>i+Tt3uKQ4^JU5Wd9qL!=lS@-jyh&g7wfkUKPa2eeV24ES|(kKmrK`@J#~IrC0%>% zD?Kar*SGQFMd^AZu8<31O-d{IXhN3)pLO;_=dc0#=LNQuKh~R_+?mCYj((yQ6037P)0HO9!8rADfs1 zKIe7&*(9g28E7oT37uZmV?SeWI9_P2kAalj*22X!-BLDToxil%^{=lb~!}>4ecL@1kRW7OFBn?Gpc5z-C})8jEIFmcq0PuMxUN`DEOz-f6*G4t^x9 zXSXw87g!dY9B&Aewg-*bOxz&lWa@H(EnNt3&;)GNzL|Zq!Tv2X-83<6bN;&Lra2(s_C@+pq5f z>eGwazWEEJr+=RG=*4W0-T?yGg_jL${O#*4$OHV4Q~I;#xpmQN1b2MsU1p=hLo%%R zus$+1ynVY2>VnzAEn8&e;w{EMeD%Lqz3T;T&mm8c-p4&#diQ>W(Q$`KZBk}mfGFrX zpzRkp@)PZ={O;sG$1Ge$yz&o-`b+Lrf9b!6+xXwv_Fwn2|AhPJSUE+0{ZFqFyoC|@ zgt_4QFaC}Ergz&NrBA^oIpB%r%$;_~Tes z`S^sp^`m#0jp#fxs`Jgr(4Y(t4$4x@H5=}ekq7T;=uU-6*UAH>`=G~4-{DV_Ze2t} zy4zzZide2sGynbA-`I7#QQ$w$kDLPtsVk;h{A^kXh&pQ*u;LBJ8QrO9n@Wx=G2c~H zydRs;QHirBV-bBf6+`s^cy-t5)WEPx&r& zF`L1r4Gqr)G*BXOvE$=}cilZX%u8BSXPSw$N5Tk5CnAP#C- zfuX>|;9}Q#vTaNG!AtRGw{|!e?57QY{AI40l_>L;5OxuVL9{tHj_F8#rgb%7R)m>*9_|-+Mq7d4{Y9~bIk_Xwtk&#dGG<*bpPElvi3F^`LD~Q`>>}< z|MOliUAkz8DJ?xFr+Umx{LTEf+|ciUIo?cJPH}|b$?@V)4Cg6>a>R)iDyp=v|MhF=>-m%~P0Y6wN0#L@c(cf(7<1VisFK4P2 z?T_X507oID_Q*&4ESCbJ&U_cJ;;piec}!?LRdvH|5gO~%RLTY9I6Ea|9U`x$fijQCuS zKW&Ab{E-HTLLU%1wfQzJT{N$r1QnT%nQXj~!n&C4pcL|&%2fkAvc2GMslgQcH$B-0 zgcAp3$#GLwD?&Y+Pe=p*6#H)%`)|JKT3Wk{;jwbks~(^F+d{nPjl^>jE8q%l6sN zD8qWU$eLgLT<*N)zhvavD@2}rjEw2sXTAC^HG4@S@~&AvhPndEp*mUe^oGdH7ZeFHCUCC)5F;*Tx@q({VC8?`1HW=x zyP!Z0JZlh`;|5WsbkbrM9Ar;;NYfh13Q8d#GE(ClkN&N`wwYL7$Nrrk|0*1aX~Z=_ zrNrDzSn;<%Mx6n3yqV^WmRH^YKT;SxDV2-!9LVR`Hh!%q4g18Y2WXj_8+IuR&{!Mf zL#FBz3K0in$?;b5*o@Hnq#%VI`==ZK%9)5o9uK?Y8a|GksW@i|rpaM>O*lzgI2H(P zzSpD$M8UC%#{ldTpL~6lw`gBQ(6gNW2DDoAoyDM4f7U45u)1cX zPvz9TpP@cVw}hu&$nVA*oL`(vrhTeL*){#Youv#SVAO4BNo z2s8#!1iWkzwF<28F391LA;0@$(`MPWalLG_JIx-@J3#KYyUNHvG_Xy2bTT zS;x$E^sA*tb5hE=*sTHZQ%_b@29Yv8WLrQKa{+Yn-9+OqWfPK4SjcOa{HJ;iu+X<- zzk0}gl6k6@3$c^MA77503+32T&XM<@{swvcDqYl$@n8qsq1)u|UUrx~YQ-ze%zJ-* zy{xR5ed}qv{nM| zxE|3*IJa-ysCS#Km#q&zD4W*aD;w{-SH|@HXDB1vHp?h*og=e*TCJ|ODf>0GCK&&< z#%;C(wUL(rd9qAMZM8Z)kBbIGz+*rha59B_gss|`Yn91+gAR?0HlUG%atRyTJ16UD zl|yL53#7_EP|5EXP2j5PNdsT2LOF!3_#5?~+x92F=omv|YFj`nCP(p*XF01la~E{dzz;UG4EBUmzSUY~ zbNP^w8s|6#8biLJZq4O&?6-)jOX)!P3Ap>KzK5Er$G?gOq`BM*Duw|#8;h!)k2RRU zaR*4dO7kXaLpP-H1DPonm7hqnKAYB45@FT0NeJ?AH1yX>-^$La57O9s4qM3sZ)^Lq zzB=}sd+R>SJ?_BleI#*Rlo#;MOQ7^a+35S7mOFXm%&%8br`yJv2J9Yq-$Cpzb}to z)%bq0fqQ-|rycWb`G@O8zW!Q~etlQj5`7eN{X-AR+M8~$AA7-=?q9UX-=B#0nJI5? zygw1|GP5~H?=rQ&d@C zL;4QI!L3{DeTwYYpbA^0cc9JFJK-L>f2|B`-zHt!2Jceq?)LYnbz_cm<{gfItn4*A(^xbMWh^m~o-z4gA_8s$G*VV^pBin*#sSPJK8&3Y5D9-y`miXc7p%Bwou?lhn5P%b9FSY}{X;0jy0~U}h9u)pbJKIsSUi^^ zCyk=@o?l*18u!>Xkd-JhND&lK6|^WVK|Zm;^o`;v z9ivP23zW;3G%l2R>tVU!TmK-BdC@6J7=|n!t=?z<17x2A4zM|9n=W7taK5ocbe(%P z`FKwaXAr@{9OfKbAk(jN_Uk+|U+13%x>$$BxJ~QV+oIX%s4l$euWM8nU&69O&Y6U$&+(GK+{e5Oex3^ur;J=5lZ|tQNuH>IzdRRl`E?!HSE<`% z_Su#YZF0PojE?;i$KL{IPOSzY;*Uaz+?+~ISmSos;JhSq88FA2jknGMt>?;k3*Md; zKIR!-(zrhr$hL$S*^XDa2-{_j@PA4ZjX6L$Q+$q>-ORD zIYK>dbA{^#TP&tA_IwOdXT*WAAcQ~b^OZRm)!}HDuRPihN(pmXj+4*iL&p$^V+s2N ze`BF;gy6{WdJD$SZ4Xv=gE2PmuaZYysGq5c>W=;Hzv5WLxDtY?8BUt{cQ?o13<*7% zFVK%a)&o-SoYoAV@N%MAgNlaisuHlG1FPtskk!CJLh>;-a1!TZCS|=^qDLLUY{dHLQzcOr3f9wL1JnU(B6n4=39Z+{vMLk0H;1d(jL)D&Zicz|+qM8|wW| zG+ijuJ43p9;x4hsHB8oJ8Pi2N$U(uKYAE&vPQ8N!cbB0T{Odx0OmZ;Mh)({)8=dj5 zKCpZ=ZxC;<`ESGe2SU5u40@N#L5D7%o)X*F$#p;aru^OeUoHRi{YFukLojDx$oA@8 zTuYbkq4SLXvP=aJ-e-u%BZqN!D()V$MJ$C{$2o@c5qhy11b4au*RhZJ28&?0gBEwf zVu2nDcIg8m#XMm$ac6AIHR#_K=~ad&5V8nA-On-#a}auW=v7VUBiB$F+PA=Oa4(^| zS}ju=e4T8ZqzP!_vfv}i%`Pg!C;9we!#Rfsqzn@_iH+q1`ptNAJR=(l`_Mn_B%bS3 zwwY+ZWB>HxUkltsj-G{ed2ZNz)g;$*vzUW~sLS!3w;A3lw{v|=w#^0ASZyXVr}-@C zGo!V)aa7lFD*EQKDKCd5Uh65ax4v&Nmxy9zqCtRN&ZiSXq+i zb9Kxh7O>bgZ(-adF}V~F(SS6dibJ9BGjQ{DDCu%Cc`f-}HjiIre-=Mzk;%92m4;IG zS7@sog~7$HwTBHRjS?vZ`^2MROMARclGliu=>_t;DvfA}0JSslB9a}MxY z&mr=VHXhueMKe8rcO?pCtdn|Nl~fS0i$Yl7O!^oHmM%;VuzCX36Nxm`m!;@k+xwvo zuqw^)pxGTNipQNQpyDDTYHPtTdB&g^84U5{PxU}jeI4Ib_HWwokg2oFHb62Ar=xjL4AcN~de$&Q53+*BOxjik_Vr$Ib(-61F(3wc6;?gdnn5&NC|?~H%X zi9BDL;X&Jh`ESGe2P4~d2N>GnPKSrL%eGA$W$k@;$!~snt^D9S7s*+lop_eQ*dEm{ zSdKe4#6ZT4M`RQbz(X*Z$iyBqCK`svug_{qmOeVWq}|13bDKc)mi;BKSCshUvw z!R7k-9Aq)3ggMRdX^vIS41_Kp+va7uxmrr?*zd81A5f+me`Ls9EtVT8`R} z`pxTXFzR)y`ULY%HfLu|tWF*65Dke3KgJsPIp&Bb8I(&HGN=n8;%!h7IP~$|s^2W; zUKOzfojRd?;FGmys{E<1uxCf@r+E$bt6ZaneB;Y;s|B?^>_?zH=iDOVSp=&Xf`xH( zcDGgWu*TxI)9q*9>Q@^#@Ok!&Y6GB8rc7zkj6!?$TzeRI=co%oBSeShNTtz{<`kt< z9d~q9UnB!P`LsF>40K2XGjQr^x@qrR(m;FCT(--izN)w@8wybNxbkM9 z{R8^0pX!dP%C|ceyY}63aM!zcY8Mo4@Bh*F{#ep6o~R@qu_gbmhIlU;$|&BCrUS#@ zo2CTCnWwpmIo`>CpF7|O6i$Sl@$XsEa=zLH^WT8J*Kc=~i{JJ%>3Z9)>|Ku>Y2m8s z7dLq6p+WCpbsDl25%R2c#HN8D5Ra+B7)gVKDa(t_#>Qe%O?&6u0?3a6G#rnqq^DCl zwwA}gp|~TruQyo)HQTD#+@Kz(jTQiC7`9Pa)Z?E=hmf^7WE}<6qgvmq;!SSIcieqO zob_eZ%wQ-r7`PFRmud+%Ys zLjm+3Y^-eZ+R1HcKcUNw4kjUjW8TNL>X6R^P1;YDzZ3!aD1;?$l~dTzZbFks2TwxI zRPC86f2jiPCNzD+CnfJ>_Gvev$(w@x`6Q3yKz^P$+rOR(pXBz?pNJt#?l)wm4a8#6 z$>{00XC4l>#W=G?55*0l3UtpDaC?7#eD&7Zu@(fDPz&cB$Pb=}KpIYuqPM_!)Iek6 zw2Qh#3nPODUX+3#`9hnmTI)$Oe#l`P5GV_DEi+}mOEbG?wf*Mj9f<$#v_JaJ#|v;2 z(sG=2#xTxiN{eO`*w@pgk9y#dl*YlE4`@?HH^vtV5C`1RQfo^wgbzEFBj7qyWWvbmiYEBTyB zyb3QI=h5+Y!~rK$CfPm)ZMQvsg~-(Zwd#qnG(N_f5X1~8FJY^EO>J`-m4i%E@4^{c z$#W03SN?O%n;mK!ZNPrVe)B@*2e@R4@yB}uA1(!tKaOC5z9{W7*YHj=#1Q{1*8?)| z3C#qKdqO2Pi$9M4DD(6$XnDO6+hha1Xd9rPzB+WXct>f53 z-qe2X+0PKzu`lMV{8*La$x)b8{GQcsus-IhfOvM!A$A5>Y;aCOqQYHnHgAiO5 zZt{%cG~5IL+6ezpr($t~z->3*Ah-VJdb#5O=0F+0wj1THJ8qWEoAe9cTQ!?>ne?uYFU(nO zW;6D8w*9UAS^CTD3M*VcSOhLF!~v_4<8=u$X+P&_`|w>HpYXok8?OIvx#QLwHI_HY z+I#Mlty?!)Eb;0Q#BssG#jBA+m>)ji)Iw4uTzn&LnAggVX8YACbLv{@pO+m$Wx&jsBA+> zH&RADF~~&-ozOpZBhH|ckI)he{1ze%>d;4F^U_M6*=K)$|9nese7$il7Ciol_Xag9 zd6Yc|F)0VNOtmufF|2)ft90ZH1{-sy6~FbveDT~v{^dWHO+{bc~( zed2)4%C4CI_FS=A?$<{gA1=NjJX7QtZr@wPPLgw5ksPHH|ManTHSnqdy&ueprf zN9B0|^~=XcZQi%xg1H>J!e*8X=a3=WHA*yp6`?bQh0U!4EhjVPMhEEFrfD-dKVa=UwyaRok zjjc4ss5|V#54k{Sb?^m}DSTpe*o}RwLOWw_gsAXUFK3|aT6?WZ?Msv>4C95J>?8PW7j!Me0Z~c_LdX8hG3F)PATN17M2LzUuejSE zgWmagYo>9t`J*9a;Dct3gH~ju^2W7tw89=FI7A~{ONngJ53F7J-LK1!ul$Bw^Iuo$ zm&F_3pAG#M=*O3~>&MyrOW|l=l_mOK*xz~bv*fAI{C)YKFMPS|vFD03bgChQ^A5z( z$B550(1BJFXT8g3E%tJ}>fJH74pw=Yto>}^?QwS1yuJGi8aPKQ{W)FiO8am5?XTqW zZ(b-r{mB*btLy%sjNq$TMX`_H>1&O)=?Acym4hGgXnE4_Jy)Ln{Nvtro{Qm z__!~|i{FmepX-y4iF^!!r`34M<+NNhgE0L3CB**1^F{NzWc}7*eZ+56dh|UlNDCUu zV-TTrmSrY+nM!3MdZICn%$(4-N-WAi1^XeiI_+;f`Ietmp6aX0R4)E!1230f4J$W- zb>qpW9)zZRo7vx^FOq)eht8D`o&H)|?GFdf8GySh{O%71ItxI}%6hgn4$A7_h7#{%_MSgZAqm&wBOvkCLB| z7@-X|`H9hv=~vF*#q3<{rg0T>a-$2-041Qc*A2;Kcg|dGZIV}JPWEU>8pTou_K7Eb zN&-424Gqe|ItnQGi`^uO?4`J5C>m<9_PUAPmhEZlthL|GM$B-o;pZD;dTxy{)}2=F zN6WSb9Akj^<>N0Yp3GDS7|!5>Bg4r@i0T~2eyy-=z*f1E`eHi>%W$CRkEA$H@P()hJD{LF^;ckD;S@22WA5bs=XJMh zgSAMZ%4%W`y$;v%Ui_NAy0%A4(!haM_IhT$n+ywIzx?@+t4b@H3v{6=oP@kaUi&wnnL zUw*k<^}S2%_sPF`yZn!59V5rR;*Ijur~kfnZ0iyDfEwn~u!tC7RVp0VOxw>s(O-G3 z<$5?3_0W?(cK(WnefMYuU0+(K^21gh8%*8|+K+dChJxYJm)#jk;gvvu@=jN4?b8we6>{i`~5w=c;(Y(T`*mAN}Y@ za^;m*%DN_j<@iHKN&k$nd18+Pl>Kx@HhgMabxtB{M4%n>J<)=K2++z*^ zf;0b7S||Vgu*5M^j_1-pj2B_a-x>ez&z(8{J^FD^kuO|yy;*q27JkMJn4`|;wHcF}C^zp`uva<)YB+gchV;ZaAuz}?qt@WAMo>E&C zo@vpjQS1IniL@m2)#@xWTl5A~t7C&po?Z4sdlvZ4{ijqjgE{7Ql5aM_e7r$#<|$x> z*DdmM=;~U!UW+l{b=CX@XDJt)rlp}zj6En}rJ(3V!?Tx1T)sxzA0kci(-teC=yrlXK5KSAKZKzuWK8kNaIY z;ly{!Gmmv3m%p|=@eGIxgkke}xodrn4VNVEQP{?h+0Igc0dh=<9Z z8T%32?YemRuV4OqIsXf1%R>*|XCK8o?zrRRHLrP%yx;{ds4aA6W+fCi<1AaY%zj54 zaYUkHG41NBua+-;=}YpJuY5($`@bKR3;yMk@}fU{oxJvqeS@TonX%v1 z;4ujhKiJEKGjNbbzNCB;WQP{b_E>~$9o9v&t-5HoFiT7H(3q%92CwzNR|8Y2RbCrz zibpnidbm`yMU++SH=p`Zx$-;zD*yTYOXQj#e|Og3eKs;OEW?A_8wNr-=`%0K%Y|mqMTA)_^)xjA zQA+vawXfZ2rj6|%UZ`sf8+YrMHIqwb4$*@zZdphE*@A40;c_lOJ!$BiIAGHP zIj9SH7zg0wcxY%}@Ka{_=mX;ph<`g0|1H}F%+{s)F=YFTul^wo_VLHQ+X3c&0$%&Y zBm=)%6l}vCXs-k=Z0TO^FVtFW&?aK!;X<6PHiRn9eV`fM7*mgL#RGlG)!2U-J8*#J zQnG$ipNeGic!7~l`pHMwvA^c8nTWqe#gc$|=)1|`mY<{-w+$SG@PcC;Yx6DIA&~u!8vjI|XLh6it{5=0j zCG_0#L?f+&{!)HDj|mc9tz>z~AZUV)+O1{!cS8MR-2H3U$lrbVRJr0?|7tB>^r9EZ zJKybu!Jos?` zkec>`cSQ4;SanOAK_O09aRHl*{eEoD+c(3T(a;0wgrHT$-JFg2sG|GP%HomenAO0~ zmX24m6d3cqh&$w5idbHC#U*m~-~5H#d-v_O=yk>!XPDnttXL7f?Kr@K*wdf>bo+hc z6Q7VTfBDN6%Zo4kC;9dzUy|3o>7DY*6W%U;ddHgObErwX2|=E^{gvLd7Zi>gLI^gT zomI^mut_bNK^RVsVLafTOZrsMfNa>jRrX))ol&9EFe*ajd@%c^Q-%_ za%I6k#rdzlf1W(&`F||W`Tak(Ipo&gUN6^SZuzCpEw}0Z@bK*YFbU?xk)g&90nmSh z%viu3DP`W4GRiz>{b=D^n`^WZFqg^WBiiLU<~&xX@n5gE;G-;CsI7FLKO?r;hUXpB zu?E9ZH9>8Nxg;RWaGyHFzRx_Eo1zd22)}VY0*CWE#C}V8(FVLI(ADHYITRIW_ORa+ zDX(`M5RYKj#x8>mIIoV!5BY8bB(&6*Vgo{I$A0+D3*2HpiG9+s%RD3=EVCNq=fc4A z9}Q#5q(PJu>>9uco_l`mv9^|&ibZmBTmV`rp%PP_;*0}hVC`!>(3f1`_pt(qQqF_m z0L>*E1K7YAMUhM%ETI%*ARi$bX8dLUJ}|Xq$9{L8x{LE;bAK8&cs&=fgPdf^anfc3 zgx;Lj0a0>Vg@-lDU(vj6*#`S%&XqZj#TEM_oXWfGqH}e@>@W2t@SEhB&wQqQ@{^yG zCq3y&(c#R4-}#;2k@L?#U(P)9OgZ(`Q{~c2FO}CF_au44N$-(Yop7psGytCj$HG|* z*_zE}pZbb^z&>$6tB)}~{+KtmY{U7&V}MXVRD@&_@?F&|BULtKx<>gNLoLbg@j?sl zAAqmhKwA1XdOURh%f=4+|IN=1l z6A_DN?|8>MBg(OE5D|8F8;o55^SB|&_mMOYv+?hq$Og>}+keQRkJGOsJyzsp zCz`uQbW!JSolEYx^;dGIF3#U`*R49g=-hJ8?e;MNz8k}1(}r6NK7B-?RpDJ|d@Qzs zpJNsf1#(H#UCdLIVfDqv|4QnAEw!p>&%0Vdvy9dapnaex@Kgt668V4sx{b!Uda2)G zhCif1Km6LMU{2-yPTl<61MbBvYaTIIo4a7Wl4wPmuY<3gd&_#v)bO7}_k{N_o>x%o zZJdL+YXDtP2q8O7XjW4{S0W!FWIOKiTso}qf;yu`Kg3hLS;9WR>o_HJ?Du0c{D3m) z_-ErJ&qv58^A)t&QaHu{%O7n(ZdsmX(Ce6W4&9^3wn$c*ZlA-|dHWjp1CMS8(oT{5>#|0RJke~tw^y#vXeeln1kz@Jt zU%FTPmb`qsC~IiGl!{Nlf_l6QUZ|LRv8SLv#m zcF!b$A%?p_KG)I@P?IQnOOT)3OJDY9*87auA9hq@LT3hbIDfgB z{_Xa(VWn{eZ`&(4TAyZn*IIy zS>Q(;_IUf5YjSs~ukm%12k&2F&q3o^=nW68l@043(svm>Xpdvz6T@3JZ_rN-Z_vL? zY?STWw#wkZHvg+dgkdHDE)++H@lLb)Bva~+DCQ?Nr4GztLew|ID;hBN0&e1fUW_+# zRlo6Rxrk3SBWEqwDpqU2T|$WL!{920~~2yKmtK zxI=wub9nd{RG?mq9dKz0Xs%QSl3UIHJ$mdo@Dcbt^<>sYN9vZBLj0r+QlU{LLCJ=eIClINz?ZzZep zbVaK~+h2xt7kc7{LTEb{P{pX$B=Z0MbsO{rt|BXz_Sj!1xoBqc+GvCG7T8Ldl(S2j#yte80ZkAgGEMbB#Cw0`{YX-u`E{H!fd^i3Y@6DL1x za2jeWJgO+(r&_z{WBbY56hg~mSYP>a-Jas&BlO2j6aSC(DU}#=a>aVDeL_R z?z65jfQJYm^E}3q>7cTx3&uX@Y_K={(ZQslu}cL%|A5(y-|NwBbZAX=6T7sPW4~kn z&W(R97Z5Nlj6ET9%xh=cHWG^omh#o@ppCT3(lN5r#}39Ae4-f}qR^(nC+bMNlM2^V z4?Vb6PJ8RI_Hy{AJ?&|7!37rxUi7~^%CW~DYj?Qe-D%(W#y8|muX?8Z?Kzjq{s%qE z?6GqawKpH*u$z8>&Xcu2*=NUjb%;Swi8I^0KMs1A@BJ9Jvbldlq8*4W2u6T*76a1~ z(p_Uf6vhS+#gF%BW4oZ}f42}6lNU0uk0)!#^bfp<=l8$!Z}NfDPf&bV-v9ph+heux z*{&}5kO1ydJm#2VY!U6F?|X~f@|)}B&rkcPh(6G05ev0;%zn2Bel34HYU`vH%^(~K z=f9PU$K-)6dbip7jk52Gyfct>n->sK)K7E(OUh{25uz@Bcs%Td4B z257O=fj*)Ow`kY9_BN55e<0iStFB$zraxlHSYprsHwfJ$EpcX7o@!|2$E%`>AquZ&=kT_H{y zx_fXJwL)FY>ruQXqxH{UpmlgfL6`oqXi$1D{ix1QgEF9Z%k^>m1>D_p4dxx#HGp`) zEqbfF`w5=IAc$WvU$n~qt=k9nBLLfEpQU5w8}!rOh57;JcwK8VgLxc2Gj`IHy>yj1 zG&@O6C#?BNbrKS}t3v9zt2$9F?JMbl{vyxQNwYHAF(n!h_u~iJMt#JjmA)l^8b=-b zJvk7U8I8Y-MIs?)7Qcp=X?_GPxMu~R&V!MJJrVqBw4Z8>L%L-J$6+;?MZiRk(?EIf6R+<#!AW?)r_KmBQYQG7JqEdXEP#ABN8d*A!)_tw`PB_BKIGI`{qk2HPE zEY+RnnO}Cm{}%sQJ4>eXh$BaN22JzbW0?NnLO;~#o8B>NA6Y2VQruCy6i97rpeg)QthHo z09*jZuIX>IP@;MNm4>vfV}E1(@xG(JzJ6J}bXl7G5eJLEc^ZEAiwr=Nyx#r-qKjjA zgljOU)%I*LaM0nEnv4~t$47cZ4XOMGtO{M*!i8-roLdm(Lo zd}0C0?_9$o|KQMe*`jyNZanOnvh3I2mrXj?V8r+6T-2++v^hte)}7^D5||M5XWk78 zalpyVHEfR-&~rp*IG=O6^fNuxe6xM=O4+*iVKN_KwZWi&b+q}>-vKg}_}_Kk{iff6 zdv(jad6>WZ?3{pio6*n4w3|1u%j`vT4A+@DhvxrENyZw`N^#a{5!0$FQ-}pL2HlXH zkRYI~O~@fWWFaIduN%#>*ix`9rVPdqm!x!Zj&k9Uq&dLzg2vmr)^YT|%eEYB3@ z*sum1kH!!UQto3AQ3$Z&xMo$l0YtG}bm+T&4CL4^+gqGMCiodM&UdZ!E%`h4!-@Js z*ieW^d|LUloSR$_g;1bLjyb;?XS}0^W6!x}*Oqxlh~qhPtYQ9ukCE?iGNF-*oR4`Z z$;2II%6I>Lcgow|@O-&PUlRYG_q<2m``-5|wLJ=Sg+BZ2vyZ&_&2N@d{_F+vsn35` z4teBb%^z5^V2(}I{>Dj&`Q=I4Fej)>EN|%=Oms|StE+MlEH=lea;6su;I$Y50c&g%yH&AXdFM@V*Kr-7tJ67Lb+(R za#_DDTQDg1KCn&(^==@0M6*dhu3&yrKr;bdCURhWi5kvI4tHOBjNr1ya-1^QJ6|?0ULl*8?=Ra{9x98LuabFs_lF(8wOtqPXwjcddXSGnVFi5v zhql^>ZP~x=1kG{gE+7iWRuUI9tHcIumG~z9M>Tb6g5y`_)VEl{Fr9sH@RK%F3WFi64)|kBB@>jD(;QU@`S?=i zCVf%-X>U2!7R)~Q!4KMFd`(2}b`tJ>n?HZPoS-k6KkY3ql~14hJz2SWKYJ?4>{7o> z#eO%)eUXpb95+96o1z*I(;?$!a^{6H~>|PQc*ZkJEz9qQx?22!FRhH>fCMTc% zaXsJG8@BZN9>&nd$=FY8I41J(K^l076S7@1IH^T5b_9rv1+3oUFU*JRH6%a3Ys~I8 zJM@5kOny0-J;^j`5opgEBn4=?b&#Obf~Y%i40y~z#$TT-O8K5(Rie2(Y+2ZMm6(vh z#H?_yZ`D71LYvV8@m!wU>hdbQW54^$nte zDlSxqdgsZo{s#LN$Uxsh8JM?3wl7>JBf1aW@bwV9BMy(4`hpq8l;==m?o4dI>x;ZX zE_r@=-sSm>oXjn2#m}}buap1oKhHJb_4%hW{vR&$pN|KOB>zH_?rTzvEm|gP_d8PV z+50ilHMmXYZ(V0|&HRB)(m%M>=9|9Z?K;m4r6XWI8mEJg>U=V!i}8bf_$`p_{flM$ zyv4F@!5%WWV5#)<>Rn)ZC-0J8JZjpb>grshbCu^Z?5W0je!Suw0^PvN`S+oXn`Dzd znz`@N5m~sP-_8LrhuJ(f9Yc0v+kCv{IDFxLh|bl{hW8Gse~gyqR`9}SRU5Z-0gS-RHieH~I9XA?9PTwf&T5V=*6-T9Ky=d z4s4`>-`dB#(X2gbfL440LGXh>?HEv%s0*KXz0s{VUoRihSYlxji&E{lU=~K0j`+~S zv!DHJ`TF_i$f1WFDKGisKh;-^=+hgxzGs<){bVg6$BP}Cv=VV3t>i8DC$(tCMmRaf zVj0SU1qlQi2?ui4U6#6nB36bc=8vN9S5F&EClvj031<+#r(y?D1Jx=^>64W}p-wbIvPZy)` z!b&Wj^(tXl-)n?-6^)JUA;V+)$e`9CHkd>Z2SD&!DE0BHE-aquihP&8@5t{4Q9kAO zNGU(^2<>#>0w2|W#=7ygLSIZq6miFxLFMblg{tn)SAM^fOc$H=R@44@^JU?}C5HFu zBbwc+zb*DzPxZX$#DDzwhogN!Cbtm@h=Gf^aR7sifRl|lAWBisW`v#b@9y(K1Gl>4 z2;ssUFa23Lo&i9M@k_hpKlR2q#r%SCJAeM7I4i$eaCOXqNTc6ua8pdW8qjxgDp6g`w1KRQ<8_V!XrEp zUQ_F)+wN9Ikw>oTvPU!V2LRl8js>}PT{z9&pp`tgJJTP^ zW%?u>G?b40;h$O*-8bB_Dj;G|4>}oVA2d_cE zJ6`p?%%X zp$yrLC*Pp6;BhiF$AYH{R{imA-);JC-}jvQYT2gm_Pyw$i{!~qesV6QE%9Z`mdRz8 zT_#U>!V~0EAAY+$>M>80!;g5PUsJ1HcD^}*{bp1i>%vOLln~W9Ua;48C%tG!flwAF z$v9bFyr54G*khC2wr)V~yzc=yVDD9klZ{0{l=9%<)TwMkU?k#{ z`gi6(^IT&%TllxJK`mg7@oX-S`GrL(n` zAR)Bfn9eanxGN0%RtRIaDL`I=4-oSY)W-=RL}LEHT!Vd`hZJ%jb0HSYOh1J^m=AT% zLA~h`I!xC7ja#)3BD zVk{UR`i*ev@n_2lLIw(9iCg6?+0X{S9M9E{zgJp4aPpbmF+OsB;?Ao%jf+exnxvNd zo~-5ADPz}kf!Z!u-T$8ZijR}~(6hXNHr*3{49(a_STVmucoWV!$K`eG-)Zq@Aek-- zIgSY}1`J5XHycsr8KD0nuXAHaA}o$4Xs1UE8-=&rbe;V3r{81m3j5+0zgQcmxyW|J z@sWb7ufAIEQu>TM{W&j^r#$`ncFvQgmkI1g+R&F6v_8gTP{Y^^Rn8!UKn{53lsPmL zSk_Y(VJrT!KMz=?Fyv6BFcx>ze)jBl$!)j%T249T6v05hMggeb}qKw=8F!?-gzh1e0{g~Wz$6d1Dsuf!2 z2~_1xoCv}|EoI)A-C154Q?~IuP#Hn92~Qi96CWpE(DTWVIpn>2&XbmPlcD4!tYu=x zN3>o~y9udFrJv_`RbTmp5_8zGAM?QY@kcJ=f)RI80LO*w`~{2b>vp(s^}n1rBevQey&ALZGwTV)~=m6X`y{Q94S$)M0ivCO@svAqk ze)ZWD;-3Opxst~yLe%7V#o4|Xe1ISHCTymy`6^YY-1X2oe;A^-qD07*na zRC%a!_1Xt_#N#XDpZMU(diVUWoO|xMvU26hBxyDvcna<-U-^nW@rh5guSEXS1wYYq zmc@G4fo zkoQ~BvIb!BgLA-Wqb%s#&O`Nysq24nwOoARXXP=Ed5nDc!yl&LSp$zb<`_BYq?7Dx zt>>Tnae4I{PSdBm^vMi7TBi2f^v>fmtb&VJQXEN`891dyv%E{(X@=jDg}t)>lC5&* z#xA+#?tA2s2ke^$bk1Ml^}y3E!iulLazB)l1t*Sbl8A$AOqxz7&6AUD&;#+RJ!nohF4B&8WhKDfsQ~4=kkV&lb(J&KA(H zPzD-cEU2Zs%# zXjTet22wjdN&sm7uon(4kvwf|{f|X7n z)%=2aMi=5yk9p7*>akc4JmwnIp&wu9_dZEue_CBD53FA=cdva=mdzWM16TM50JvBN zIi^poMSJYPPTDr1jxgAahRj$XA`Td>vepgkCf9r$@>45ARM+S|s#SZ_ z1TdB8EA8h9ZJ>V9deZhxmA_-Z=fb4pUj+;K$ebjsYUc*VR5h5Tuvy1_LR|O&=LL}A zx}WNS>yC{dlTgQllmNzn@kayVri^ldT$5*ns9;^l05z)fp+%ZjyQ~}*;*2L51AeC@ z*sm8qi30y%A{gDeSwJ?ZQUSa1KadMa!3~+ht-jKjj6BEz(Sqg z<}Y1(BS+W5rP94<59waEN_tl4M+%njE!}$lh*)_%@stvJ;p9NvdalP1x>zBv?jWva z1M*$ZJ$3)nA34=x`kBvs#@?qLjb;`2zz04cU;XM=o_(5bDZM@G{(W&POzEuq8WwbWEnXA%8Y`)IzM)w&2rzCF}dNkJ7vGsD`bJb z>x2rMf{_&9r(%>>q=lsmY2crVG2kx+;Iz}_d2%(%^Mv(=t~Pyd@>*;fOba;IjGf! z&WvM8XZ&~OznPr>ZA@zpVzCI1Sz~dmU*{T}SYiIa9c2CzwBUE~~w^w3iUJqYzSC)RTwM>lh7A zCIzIUC|BjcdFfqrh<}QxN{o7g=m6@l3)^)9T6w|PB~!*5YBUHa<(rVi+=Lymhg!~K zd8K&eA1W6zg0Irw8k^1Nig(&{?5EGm_*+0ybbYVBWen=n_TQu4gyU))W2H)b=9ad| zQautg@$)&kl=m>wNQ%q{LsOe^MZj#WV3CBI$YfAz)B zo@IBEfBfSgZz5#&kly>=_sV(aohM)Y;wRFmY`tdZcHP47grKc4FQZkra3NB4>Yr00Mmr1z0e zlHUCf)BMB@7UVAGB-4-M8ayCyCh**b-Z%#x)`h$O{QhNfPBvJvjl!f^ta<6c{@Cv7~49;Z~$ zAx_efcsOas;@Of#^W`zC^-G#-y5!m$Z;_`y{)kGZ@>>ZAqf~1%4s^?d3~Hs)m<&;` zyX3<`%5ca*p)B?j^1+9KHDYMsYsFxXFJg_uPqBoMZ?b})K`}c_cKr)mN;0ZLpdI_o z7n%Q3%(Q}WB}BA#H2#>V^eWHa-J=i%aq82H#*tVg!=f541bbB<)Y)GV5TXb;<7$nu zf!?Ug^hXYn&mPxQjqz1FV~<;M@lG?evGJl4A=_IvfNtEbAJCbw-$YW^u&0y%>Uwod zXZ%}sSU%#Qio9%=FOfDJ4xYEnDQ$qf&+{LdZ1Pe@Ln^ZyXy5^D+$hfR!@MU%+Z`SM z=-CuUUt!2j%YSsGKP-r20Uif&%q2bAHD1_{TG*KE)W;-Ie7;c|IX@{49)c|J!k@68 zi|fEigWt&uX!|<18IHLx?6#9PqpD1$VZkiPS9~h*zv+&~9?y=P};^4L_xvMMbo6&SJ|tK$McdpX6<%M+ z1Jy`gt2px^_>G?77z6ABb;P4T(?Zz|M!nNQ26n6sDw`D6g}jda>LdCD?joN12V~TZ zQ<$&gRJSb|dfWUJFwJ{04^wfD!<^Ua;1x@_*bOfw-)SB2kk?tDbzgKAs3T7_cQ7Aj zvzh171Y73l1T3?rDnCyEUN1_qaYO&mXK6Uu;tuYTzxEZk$cELgUO^ywcG`}L8~9=&VK3WVl!mF^F3->xFI%HZbB zvTeh9*}8t6Y`%Z3{_d8c`|psUYrZXA%T`L?Q(i26k9(G_vDhC&v!g{m$D*Gxy;IJ+ z@pvM}=o&YM4CzAOS3dVqt?!aE&N##Mn#J2!zxvg3=9y>8w=X$QUitdB%Rc)bqMp$k zW%N`Ven4T4*T!`@6lTB&lUg*ZDG{Aw(F~7g;v^csLsyN;9S;r4y$`OFoAqm&kJx{o zrs`SX^OxtQ17&mT{In1!8kJ+1Jm>&a%hA-EQcRTddnIU@{0ub-89!{z<)?PhzSO^w zKNP9;k(6`U9sA7>?iD!7qF6a8!x(qM(%;rnMF4`wZ@<-Wr+4dlJh7ODg81=H48uS4uNN&Z12iK*P2qP1>;Uw_Gc|&V%DN#Kb#RB>BN7h z>VSE^P&dE_*Kd>?Z@p9I_m0Tp_w&2U=jll!emM5?FSagcc5LZqKt>4esJSGWf(rO0 zIO#_~Emor&-*YabS#7foIUPE073tjfCSUs@4f|Z|1oYzd)PQ}n7hF&qE%Y@bfoJV2 ztwUrDbnKtm_`7o%J?tUAIpml%;w;M{dFmZhLd1S!Z&U zrQ~m{ub;Fu|NC(Q2HOV zyfntQo-Ymc?%iBO1CBOxrO*~Y;+1VU4ydh`v;nQsj~|w^O2#$M*S_@kb~6kXn&v_| z@x&A5th3INi*?cLl_#94=T3MRyWiigdA&`xbK-dM5Ad7$s ziaS*lv=&+wcYTUhx`++t<6I>{Wv z2@uFa67oO4bLQM{zWaUIzd#c2B=?@VGiT1soO5RG-1E%>K2&Y`l*<41J6?LZjk0*A zQy)IXu}8rm%IV$%C~?mE3Hu=wbm2GpP_uM4s-0IeeZkUV*}{JH0`>d^Y~N0<<+=|xG0uwlE|~R^9~KQT+pr5AzFrwy@gx=|m&IJ}~VA4{7ni2M7mx&~uZY1o92jY~f6H zCZn)C1&ME#zCL^_yX9|r7@bJ}nG=!zr%eCrYcJt=(_@2SeHK0S-bRLJFpM8=FJA@y z=Lz=GZ-7@;f6-~Ec9Qp>@YLPFN5eso`2pc9-e10nzRltV$=&?4fqX*w8$F9RxVy57 z7Jl>zQ>vjO2UJqGygbUy&6W4P8O?xe_C05vd5P+H1yz_LwewVQ+N>p`&AUm zb0;sM1V8_vz)aCc_5!tohcEizWGQsnZ}c@zbp`~0W?@2*pERi}j|9g2mbXBTLmf>A zfGYKoGR4v$2XpXAd0l|uOPIrvdaz$0%Ez9l{>lgNtRDFr-(nnB8EMj3sOc!e+p>dZ z!xaUMdO%R~`WG5p}ix0L4LmQT>@g&5*zF(Yu+f4O;)`p|n}Aug_LL zzcQkl_HF`nPWND~p~{L1D&(bnYga9&^Upt@`uFc|C@lg@Hm{=0^c%qW_om#eq~?i6l?S$osZ*yC_BMNG?n^ZGq|?|4rCUtdHz5cL&sf*0 zdE8UY8|at4@O$b|RzgEg98Y}?8o_Gl+O^C?ODsZ+quOkhW+GlV1Z4B}+mE{L*N+Z3 z>S$W`-aGWg@(-zO?$eY#;Zo{=vE3eB-52hIw5zl+0UP_+04D}NKFHypfB&Oog-!IE z>dK2Z(w94GSy~!LG@Ux<<;wUl1tN>1>_1GIGKFrq;Rc%XkH6Ao|9%@Ace%$DSvssx z9~zs5i*7{a2dM2{jghszZv1@1{yMd?w6v67c;N*(z7@iN!?FxCG#JpZugQoJBjlH7 z#OE-_4)0$9T!ANXk&4WCZHPggv;x9PBE;QuRi>6d@QB`nR0QcAzq^MoSH zFS%s}%cYz@{9>}_uzbV%5{~z>5^p5`y)S?C1t!0+Ji_t{%d@fgVS3^8k^I}K{M|du zjQIBJSTgxbpX2ahG|_*3VL^Z3YJ;LboWSotkv?$dcTn`6{l5NJ@%#O~Wgk)b?n)Zc zZ5Q><%aV6J-uD=2;zIsbhJbb(@K`+oZNOy3mS^Eb5W;)}qx9u~`XXIWZzl^5kOUSN zwF$$nn!;qEwmwK$yO;fEilrHkI8 z!;c!LNxb?)h7EBhdV@&g`|F|+r-}9gfKjZtxR~z0|9;xKb*oqqhYlS|ef##6S!SzO zucpO|7t>pBy%iYEFpw_ib-UTwyJ;JrCQ-OGhu&VYoQ`M1S!M>GAmK`69lhlx;U;`8 zsnnU3%<7H~0Jzo>gwu!h5BdqkH-)v;6#~HtPGDhp4CgOl-YuLT@GVTxtn9A9CNhJ9 z55%bcj$Oej6rrLO|E zBMdwX6XYO6qbmp!l`LM$0u+alK#yazEQ+D{`mv5A)^23Hf&k?2y=w)Z!!)(lAMsW{ zO&2rnfJE3?T>(J{9~Z$aNPcc>^;e!)EsZk@g)dxBsnK zP-ZK)L93JDv=wM|sY^b79j%@S3#Y+E7c3*hrt#PV>)m-TQ+HlI_>FITBjxO<+PsMF z`2GG6X; zjr#qgM~|kFBS+HwcV6Rsc2P!V7GWLH9@cSd->`YWRr0jADx8AJu4vsF+PHKv^&N4r z>|Go1)w(IyC7~gs^-6>0yWqL@13C$9#gm0bYg?_`E}>aIcLtyP$Snm=8JD#BE(%BWcWIg7VUOw`uX<*#2@@pW}w@6r9hE>y{1y~#5lhIQv*K{8riZ()j_ zAJ7(h`J{@?*}q}F;$C_xbXzjh_eX@E41YyUAJY4z5?J-cuFYAC%P7(GMNOgh zL^4;a2ugFTws8IufAO4f+;lG6LvP);+N@6nh>uGqi5vna&4huZ@r8lPPB1Le?ROWH zbaJPZ1d-=puGh9trP#*rilO)y;OQffVtZpiQ66M5Fzl3+Z+d36;~uMPr|X7!Eb6vx zdg5vIp>;c-yuQSm+GCWxh!_3+@=ltGWs!xcjjtYWnh7V|)10_;n^N&>fZkkWC~Mdr-9MkhA#CA|&JH*GX57fq z{dh)V;^6OWg>TpNl*4K^X)CPZ9+(bsJ2%yJPFRB}NdBP>QslS-`Nd<>d z>Kg{1mSYm!3Q=12I5}Jhx8`$?p-!U1DF|m%^!+_f*3B+LlC)FC`cOIt52EgTLj!AE zXu&Ae*&`BH`g5(!DGfKRF=uq&AJ>gP-} zpsx9r6-mx=LNxh&w0sa1AvaD*$c(U=5J>3ysLL)I`gHHc&d&b*VsECdpg}Rk3OQr7 z@6lk~fu{Bz@S|p{-%%3{Nwet0qH)NUPaqb#>=y{474~8Du=H6|Og~o5r0d=-v$1V< zOMmwobyaGi^P0p1%?*WVzH@`EfB4ixb&}&7y@W47Nn8_~P>W|u>)R$ROYSpjh@o~R zx!_=$Nd&wsR<^43JdN2H1qftA)?Yrv&9_}hTgy!#w4ou79E(kyogKwb@JDMgyz8xN z)^hY5#A9j2<&$>8>%h6b`d=_=kHOL(uWpYDD-C?V)Bx&+M7<6^)#` zjY@oDZsn$F{KWD&#T>=-?lbjK}wlUWEkFbZ>eVaMDp*9EU%An{D_yLb(PPLV2u}z`U>E z#^-8JB`5HAY^jI?Q!4shm4=(o*juRC z!3&!qr@IM_6M$e6I0D5OZPKNW*6K$l0q<4ih)%$_+TxXCv+df2*gX6;Ux_8(O%nII zS+Ufy9=Y>EE;r#ZK^B200nPKq_1>6#Ekp0HK31dZm{NX?A`N_}`v4$0;YNFa-C0oU zigR=-Ma22fUm{Ck9p%rq^A$-kC?+DA!cKfsQkM}+bd&gn1VYrnkLCqj(ImCv;4a3wc>kce6Ar5raWZh4J zeM+lJj8Cu!GTPAEYK1Y%X%^hFcSi6A=O>G-{7x>LpB>e^tyhI3u^ zVC6(nAjyeNrecL3coA69_SCm*@w6uKzA<_KNA?`KyP`C~(YyI~DQ2ee3BxQcK(;n#=xH_t*;q?S-7j^u$PP2x zy?zKk4*DjlFG#5j^hU;fpO8W1#~D?G$?nh0338-PPCmI2wOpPWS$M{9e2{2or-=(E z3f)xL8ocQ4@|sW;;?4><^x8hJD>|jmhO)wjJ{v8dsztOIJpH)GXj3#Sjx`cIqkhWZ z2#hm`PwyYjH3xIYr4|bM#zbJLX`bP`^A98YFuw^{H&Iw#B)K0x1GnI;ZmjPqT}%;H z&3=}WRlUXJ*7qy9Sqh)I&{{Af3knLOr~7w&yLOe+p0N7g+(qH*8u+3MfJ z=LX`S@#bm$iR@a2*G*QC%r{k4{-Ml(^2zgrcn{4-W6`SymYv1Wv|h|KRGSMH1@G7F zql{)|pX9D2yFJl5iT;sTrfZx5c$x_P@O(yF@6_TX{cUDqY3t4IgxI;}q{W1t63QB0 zY~OJyS$<)flu^J&&@?1(9PjTV?)`Yj7#v?w4zE#guH* z^_jk1U&pxbPk}=G51QEA{}j#%Fkq67 zl zW5lJC>c$MGE1+Y|EQ>Qi>ztz(DQ#OfVhsGd^8U#H-Gyr~BXu&a_@SAu`!=QcN!|ae z@?W;7uC>>qP)7E*f-6*aTnb=P+ASmGK%x>oddY|xk zP`kIY$fi|eXn~!sJqLK%dDZyo!9qn(>x;PQj zRF|qAmxU*03kP2YWj^wbRo3^>y5G-d?W>s0au%ha{tLmQ@Q1Je?$-l1cQaNJN7}C0 zMUN7|s*Z-jY^6qYd+@bel`{MVbWv%vI%GECr$_w6z4M9!=a*1A}IG96T^q z@DGYn6$-2`IMc8K@Some2BNW`=pmiuXCQ?SR|cx8C&iTfas?)F8l;gIakby>nw|tm zB>!TEUMt~4%)7lHwZ=%M6Xyra zgpX><-^sO@B<;?U)HNm%9k+jGeFz_u4K*}?$$|j-&Jg8;e-8^B*PZs&>~vxp8X9 zaKN}+-}|%h&b@dQymtL8eZ1i9eBbf&k{)47$7uil!3*7XlV2vI>0Iu~#ZDQljhzIE-xe^Yw8vo&t-WmiR{oL-jIj8#qc_10~JdI=KK$H&lJcq)3ij3=&S0TUM`#-LP)N zMbeDgyU4%qF02eivhn4Hjzm5YK{bZnlR}at-cw@~i$7Rj!>uphp2qPqP6Yl_pCYYmGt5Ga--ctZYFTE0pX z%uehYB!oN@BLY|fk3^Pvx{U*J)IiW%hG8OWrA0g^Ga5LIr-SE+MYa}|gFdUfu5I;X zRX9G6#i5)Acz3_8+838$b)9Vz%KlEl+4+-6sn@h2*-$O`LLB)^_HImx%+v2uS*eoqGQ za^OmAssWuIc+Ah73k%d&3MxK8_rN+D7eaQ=cCB0++3PEyV*%nLd0&hVE&SdJRGM50 ze@uP4AjxPVBCymX2PQ209vRhInJ7{0lYR@O1kMD3t9?J?r13^t0KyQj_xFv+7OmxIR&?X6alh$)+iwEKtgZ33!>XWnbCn z_*cXZ9_L@bYjK~$LOxM1=XFc!@F*vbh3bIhA;9_i`a=E5N}qKrbj12!BD|w4H#bP~ z@!Cu>(j=Ah@A3L-pD`8OZsF$($l(I9(ovfE@M=G-3crfoEU3!qNrvm0QWIzU<`)8N2K&yeH`gd z1LoY()E1PWSW%s9m{4GR;H&k5<&PMPLp}EiP;Bf1k9X)`@q7JK zYyi`8Digk#wXQ(>_lS*uf7w!+Z_5Fz$J39|u3ez>hZhvKZiR_h7Kpeorm8bI}C$Hsjj&7_on!mGQHR;@HQEbsHl3GHl3 z@F6iVp?l2L3*t_dA99G5HRj)Ezt7A$(D&^uHwB8_&k5n6)3TD16n`p`FTWcIQ&m=4 z&v=D0l#_st?dD%4nL9*Yonemaj* z5^zU8PE}dON%7wMy&zXk9_u@KnC7J}H|a@EjZ`;Qb81EHEOF)N=3lm;2)7dHit%+0 zCVoMTA!?3K>PJa&)*SlhGworknlf68-lr$$2=p6ktBo4ASaExHPHZlCFByT8sgs&OU%QwW{*MJW*S_d7u zWb)Fv*E;mC8#bPPzj$!*Xs4z=a=98mTplikq~ILEu3BRi_9Ui?6fe3JpiJJ@*%{!0 z4gG}kQ*QT7a`6(GeOk7*+%ulPagxXUK5KbmyMA7}LRs{x(Es|xVWz&`ik$#iru6z5b^G?Ze4J-y&an8VFpg8AaraSBNX@(8h zAfg?f9xn3%S4vwmsA+|bSP#{(54YyE4Y#2?@;+6Mz4-8pzUGW6O$KITZ@b9(8HrZM zA&e>~xEhFm*u=iWh{=h*t@*q7Y!qTHYqlvq+qG_p8G7v`P(6pCF4Jxg`f43rI*;RAsQ^d2bb9Cx%rKQsX682;m^V7AeR`-r-bmyUC_VJVzCZ%A2`$*6$N>mgn3)PzO0A(Gou7L)i@IMp~R?)E-R(zFjMw>P38-}Zx~@_Z`DK;h$o)ykEKM$ zY9Zp8(9ces5@RR^#Aa3j!(6;jB(U<@AjFu;t@vKQz4-C#eO|F@>e~m~@>bdBqXX~N zB0D3sJFSP16RF#`dj)SE)Jy|Gll?z#K@-Brw-4Z^T;*GT%l8TwPsoc@y(sGoD{31K z>S5$>i=^?f`)^Y}`ExM-{)$4Bxrj19ms|fBaI-Nhyu8BTeW#}{{IKk?kN*5qvJpdm zS{5-NLUPVmxW6v!RR+&~i*?d4c)t%PB$wd^Ss4&UZ_c}0l`5GJ5(M$P%nb~F*eMb= z`ru}im|sql=TBrj*0ei;dw#Yep=73##d>Sh)pnoXTTEV>N_ImP*{7C;cM{ zW3(ipO>Ta3WhAJEaO}$WF96o>!xJzb7=gTD;Ek^?{P$JW>3xImH7FA{dOP^a{feh5 z6~u&Vb1B`9KPvfQP!bk*h`6e%5A0C2>9j$Nl86&jJf&tfj!gz;GJ!we<>E=FEWtv$ z3$}7z9pQhCnqv4|*+-gS{c1NS$@%?br#AjNMsPNhQhD&f%~A@&F(7|R7B9ubNm7Jg zmengGfXA>c^s4rw*S|=aOl8&{HfjSX1X~uwjonn>mOk6nA>=4SGVb7A8bME>nln$s zh~pq-Ftnw5Bd0j_KA9igq;y*_PFXUh%@|K_$npi3cc$i9M|_LBZxi#^9$TDfl|daR zGk;cm)Q}+lVwDaF?@XNU)cd&6mh_+T!#Z+7lVrR>bfWI89oFGLd<%ebRtS)OPdiS}(d<>MZ@(~=~!1$M?fsBFUWY!VtgTV7HVh5ki z++EUms#2BGxRl_pysYPpq;c6}oM1g1+gD1kcNrJ< z=Gjau{)Tbr#e9_@C(}RVj>ZXEn3rf&m5p&c4hZNJlQap-K(YZvGVXkac z62JD0FR%#$y6v;iDGM8TP;ePSh`v^gm=L9aACNSLr8n5O({rgVH|}jIL9|N6Z6_GX zJupqiHJnl-i)fe!Z$MdnyM#x4X814SCfYEeVs)&y+cK{5${m08Qy{@C z1J(LiRXZ^e>(L8Hwt@Cf(rDW}#ULk<&wNmNeGx;9ay6w^;f6}3k>X%n{9d}zG<$)3 zeL5o{h#Hr5Jodi2cEgtdBM%*m{D7IuZwOn4$qi(?H5Q&KKp!aH^(U<34A}v<*kOt` zz47BLT^ZM_)cziM{p{`ex8@$Q$sIL9XHRQSQS?ZMiS zdQ`Z7t23E>fMF$Y^z`?JYU&8$HLtRfX<4~PTrOqN*booF>h1U_CqHE4_U_k75G={ zzC8f@f3NPk@PMhwJ>+a-U*#T@JoUN$z@ytgT-12JCw&o1 zIo#ufxYx@kzEn!@EUnCU{?oVP^ZueBX}a#;^e4~clY;$_ zOWs*SL|A3({+|2H!;^wptfYLa!}Tk9TaFc;ef4a{X>i@PZd9U$f2GomgTh|>3m3iw z%|4Ato?o~cdOwg&Uj#KDKj^&1dQmDo|2^5{wW+r!ph4u=pLr^9s%gaZyIL@hl z4;lWt$%<$*QEmhMjM>(%zI1ny<pJfo%{#x>?m4mq_LP>DmSiQ9R>gE5hM=g|Dr=)R-&d_!Efe4{h;PSAn&t+o>bj?gwKE5kgwm=i$nEbd~G_qNuE zV3hNIZaDR)BRpLrZtu&f|p~vu%yd0euo=4~^U}UyR#Zc=vZZ+GbZ9CbaJ{ZlY>kn5Tce;>VZyM`yHo!r_N* z_lY8&gvR{Yifb99BM!n`IV|zeQ&*@=pMun2A5Kjcsfy+VL~7Uzejgnm2llgbc&RJL zj5Kxar28feE2H1VcbE8VljwBsv|_-7hq=Pd{0mn`>bw`CpZT+wyMM4LkVz`Ei7p)R z;(}&(&x9iF%TAqBP?4P)31E+VI#earU^Z?AwoD9Zrfzi6&^Vb2vf2YKqs`!G;!h|5)ykRSxFe{ zh9LA2=n}lHuk%5%b$ysJR4I0*wyetY-OE->*SisM!Ml9yWvqE?wXyYXUR24Np5JRr zV?Who3Uims=iBZeG=douw<%oc*QC0GKckai&+}qK77zZ^RZNN=8>d<&SQKcfDf{QWe9 zfa09_qc&pg4G~fROr7?yjboRw!(TL~&pCZVhGCFMa(r%?j`aUxI(JW836hKMiA6 zsjjie6Mvlf_z|64emQ1MK_bw5#eQNsyJgIAssL*>o&(O=Wr366*6q%)%Lwi!>&iX%T)YW zHrzFcakT#bJg}F9Z)vXCorBuyx>sp!D3?oM5uMkVPKXU1}PdoFnnT~cz&KjVO~ zT?Hguin-L$5$=G7KqX+Lx z3eBP#2Y8?S{K~0scwkWZT$d+~ka7u0xM0Q|a7Y0EN(l+7f6f?>WGGB!t-m3-HED%@ z-m3ksZ)1~YLeYH8E%PhJRE|+jp{^WU$~Kv9=^2f;)i!subp7ivi4aJ8jnxI>3~rAI zCfLzHbswO?+7WW@qo?D6ZlLQA@jYW#-N?Y;vmH%!G^t2?qjVT(w9_Io`_=Pf`7A>G zUwQJ~3~JFLSo`+H3$k2-Wdh_k19cipp}z$#ceQo^+*|QRc1s|wBd}VR;?vGGa@#C_ z2%44**-OkdQ?jJ&I|#}AWj`9t4=q60ds-pJ?oP&FWLME9@9pdmh?;_u(y7qsm`{!x zY?IO8@nk)R=5tTfmV0FQKuRZE8~cXm=Wpp2uJ&E0@#4ngG0^uRrx#<|>&+)UA36S} zofA8`;0adloHo+<1FPbw%uFN=_0BWt*gt)1^qEbR4!sNL;>W@L^N}X{>1Mw6#_e_| zM*Z4X>dH|7(SU}pGYUrBY5qQq@H)97zBQZvIDOSHL6DziZK-L<={5BIQMuZF{VER}Xu9GYRdnhUzBI99RL{tI$RL*I*qW?WSfOo|iC zUl#|$s--RVb9|!DQs9i9kVWZ;NmwByqH4>~KsuEGJjv-(@*eSQ?N!6pE;7s1$Pov> zmg{k+BRUbm0Z2h>Bdn@YQc}jU32P)V{Ry&8l@-1ROj;4D_h_;HDKDWYl&A8Kkgl?8 zq)Y0*^R%daz*G=0dqBsCxQw9aP?%cU_&%_-nbb2VT373a<7zMA_^RcWtUh>YBhkj1 z|G@IUQ@8ltg*GWz=MW)`D3;X=qu`z6bAJcLc4q~CGK-Hvy$Qyqdw91!llFFxZ6k;{WzkJeD*x^ieJwB6 zph*ukj?2u$hLqpZqA$hHJy>(@oQ0wES-tE@D8FoI5O2DqBVBhCi8?Qc8V!&-+R@pY zvGH$5(&?y@+$)d5+a8`H0gUfGkkv8;w@!!?tZ!3_`Jhek8%jmX{5j6=Iy1{k3Os=c zgiC!9sJVvDm@=}bBA)}b&1D2rBs@1PGPe8vT5B?46|)N!cJwRO*%};ReT$sDkm|w| ze8ieAMYC(VQDWMQ)H){=Ul3IvNb6S+P+P7Z2V4>*m3_IY{@c<5A*82*A#rZEyD;m$ z&n$JAfJRL%Pxao*fs^M<8AnO{aRjlE*0%EW3Y=;yfk;&b)L&`#C=ZuQQ{uZHQm3rL z6&>OUI*gftDMpt^D>>6uMm)Z844nJ+pg*UZgFUctRIzA-M|#@&ijT#)piN|75SZB8 zv;jm9u=0FH>#xH1{ka&jjsNvL|MN!1dS4qKDU^jvEHsl5AE`+p+z8fMqMY@eVc~V| zr^HL9H0_Tfe7QZt-n89y1eOl_)PAYIP>#vuW@@9dl9flK$3e|a(9Dd$Hae18~nNLUK$9z3MAg8 zDBTR9iCmWrnoEKZw?PBBAI^?cA`gL-F!l~HbS>?7wFR3X;TImf z*+9y6&}Pdij=gz6UR46GG&YrxDPYe?E`#5S@L_`mYQwC4x^qu`_?^RYX~^Aj;^&p2 zQk|1%VY)zOs^8`Q+#BPKVl&nv1udc9=6^W3hmaxa`Jk%k^{vu@kGNffjp27Rf4M9v z_R>Vk!!Ybn6Gdp~bMKYts@y|^6dVE~Mp0b*hxbN=zK$d$53vAOC8{OejM-Wh*BzAmPPfPj0Ek`Jl_BL3rZMu3NY;$JyGNX*xpC zeuHGfs&>NXf%%H+<;EiKWr#e^o!d{AhtQQ37~7Df3+(jY8-%##>k(+KtPDm z{$lvQ3-yyeFc>iFfILh^y>^y-8_@m};V+Z!fyJkNj$5;`_NHUp?0;Jx>pGh1E3;ZW zdP1%12sUxOv;yI@=n*ZZp9*e+996L7tRCewj zKcx0PTttZ*ZV1}^6^eTf(n`QMpM8Zz%f*vm(yrfcR@VoU&(AJodS`vs7k|FKJrfMw2JC}MLuv=^-`)$FHa`#hnZ_U z>IP|5^N9oh;-&7Q)B#|nBHowlBcpF4dIofQTw3ay$c8ip{R|9{d;Mz(7CzzZvscl$ zan~XjM>ouF^@zJtgw3)t6waUR;(1o%eiWIxonhu97-gwWYKAt#QH6LJL2K;!!jjEo zR1x@NLex}V0L=bJf#&KMFde;wFp+pV#=-s;&ii53^0t!KFAhI`4B5*UpGs_;NHhQ9 z%UAgw@vp4ktTIe+g=<6AmNu*p2rQC7O$bCiI?tIT>|8tJVhekxhtg=VPqyxb&%fR} zoAC9H4eX*x-eLWWo99e|NPXE@QQdP27Kf-+Ef~R)*0gZs0fA!Dl(nS|0~r8uFtht` zxuBxF+-Z{v(P!Y^-wxaN0P$yAnEz+cdP!Js zML6qA;&Hx``L7Y?>o|(AR$TIH-EP+s!QuX06DmzGa}q{fILXEi(Aq zHSZU=x{uc>v0ePYdWA90Bm5%&fdwsPA>`EF%_?A0!(J@)MxDgSn~TN?*I zi>j4K8vp(6jJ#=;>>IjxT0TLXoLeQs`#-}ePVB8x)4_h!Y=P@xCvs;qk|a#ywYjR? zGT4j*OPo&dLb#;z-MI|=7|Uj?Z5qBwK01I`yAZbFVwBo&u?2lfuS6uoG!47n2pj~b z?gZ;3;&7Z*N;C*3w;>W7GenwjKPgzhIuaeL(wgMt=4rdH1){0kOuw6oyyENn*cXfF zklbAi=9v@bj}eTzqr^1}C^l0z5v9xG<2nw>q`=UCuY6Lr%3vrl(Jq(FT>d+KCC?n6 zc7NROo6buH9q38VbqbQfHU9^)Y&hX36e%wKkRQ_Vs#q)MVQ0=NrvQ4tN>n?8V4Tft zOwe#YN|xE4Pfyw5av!b#cjR#O+?V5Z1?5m{F;ubkj@g>RqlI|MT$;3W?V1`Qt zy+0bSK9^Lq72tmZNnt=23&xlGOG!pXoMN;lg|cr#;hDw|j*cq%eC&L?|GdU>;veHj z&m1Q#>Z$`<1t>(o$~f;^%eWT%BA;!9|F_?9YhR8DkVpu-Ir-YgKvON90!PUsf@)XT zBZ6y^E2dC9fTNIj!B-hKs7%?|E)AUhnA<4B7A)wJc|0`n=v*}O0QE- zB*gFjjjAk{;m*(bKZ;e2e~C*S8zaeIiXc3)8zgV+s6LrUf#ngr7ncz+$sAQ-$<*B> z0o3ZrJ{~BFUugE-GQu-Z5I=>?iFyC2Y)CX%m8|NJX=pU>#Kk>o_Sa|n5;)wR^80Lt z6D}4 z3fD`^ySs22AR>Fy*Zg0HqhEobGrYU#4Sl(0s@6VMYeiJGzc#1Q_YFoRaW{3e6UmbO zK6pI(*R0dahn3F0;yV)cumWKfsy9{oS3rr=9Cd4F$B&aT;(KLZY<3oP&dc_SCWe^Q zt>twlFJ)!?k+I;w@FBh~3X11>@|d7fY#)CtB7Io zS-o-)!fj(IIdbl4~LTJ+f0 zX8;;zjyZql_hx_Kc{ot>dDg3MKL>#~y|``dGZ+?6=AFzd~c<1G_` zJs_R2iCV}$H}LT+N5FcUVXEl31r##KDhM61$%{qJn76N9w>IqQW7HL}r1 zkTNt+TWH}K*8I-Lttq3l+-_3IA00;B)MCN^Zj#G-4Pr?H*vY0+QlNa1 zS?f~osbJ^PZzr$}^t$`(G?WCnj6`!Z$+#U;Y5X3m;>No93yWR(ce5=p1Cv~{yoEMV|62$)e9ILJ!0-fmw`8YFcQ1Unu)Mt| z%3tAKZuRDI|ax*3kum1J-wR+V$kdw)YZJn3)D4E&66!Y)$j1@LXIqa&JQzf zQlJ=Vew6;_8%hEQn?#KD)8@OwW{a?Om-SYrL0gMA7Cj%Y zdOynp$*XR9mMtMkCh7lrW2j3)t|;68J(xs*Y*a@I6jCRC$R~Dhhi_&P>$`={WYw?o z$&UVSk@|1zhzj^LjC>idV~P%;_A|K;?r?z+cI3`rAv)gxrXgz{a3SPr4nZ-K<|)aB zwytol;5@G%U^_YdN4&I9#ECZEdP(W-hm5;pL2#U=qChXZ^h(WvgCrNKjQgc1%zeh5^y=1jdULVN@b;Y?|6l9OYw^7Vsxe zbQj00YV~O%lfXUll|`gBfu*u@3%u2`Y6`)So|`NcG@rfZJ~wIi88kvKZo21%O7OL9 zVj4rXFLxqV0yjsMc@LlQ%F_?MO3goR%jL1D6rJ~y`|A}5+l1|9>!i2Oizl1}raLeH z^=Ji~o$taslx^4VK15rx%1*cAo~tU6eDgR8ocLkq5m6T5)S5=u`x@0bUv|8hNx>Xc zgw?$wr*xz8d#~_F$Oc+2c7#x0PAm^`z*DpmNfcma47(WT=j5QKe}oZcNpaal%Y7u`23z8PH!uO$8~Wc) zk{}i@jhtHF@F0%SzPW@W^2{QsHaOJEnS|A)8*s~{aX~iXOZntBFWg7+(tujqYDnMiV5dsV$U=!W;zcvCg0o$Rf^qkG1CxDr;JeeNr zXEgM=_^`N0WCK|Mu+To|`bEcztl`Mdpl!yzm;HZNDiZ7$uY*X6qd$Ur;|$RCF%-y% zM9^iTyLThPUl`L!cN)(z^XKGMyV6TRO)sr4h_-<|r?c@U++ z0;&B>#SkROA{KX9&0s0)6c{9fA^JhzKm!X6R`vNr<#XhTVF{zD^*mLnIUgiW!{d;O zg~vfI7jE&H-D;5J{HvgOwlKHLPj0?Z>8GT$1x|I zfp5B4ZQW+7CVyvay;YWl&X+8PP7?CW0dusE20ps6b8!_oeRF%Qvb-j*qpdqw+bvC4 z+m{(@M1W1>NwQEdO4$i2&SUI@%H)n=lDv`OJy_$d>SBKA0W0 z4rj{vFAf%g(o=en%6baONbUZ|#EdN_aCL?p-z?OjrF=40-0smQ-dNHy|NJ`VlX&z^^~ z(=YXH2zpIE0a$w1LdWU}A?8v)(^xv;g<8#4ZHEFT?na{Zxf~vvVl>H>2GWf!O$9b@ z^Xed)*ariGgjnAjTpXA#ksmQ^8UDC9u&6!$k+J^%qursKi0sk~BjNAAkSiGn90%6L z-L)&WX`Jz*?r3h~_hWeP`l|cIEjdgIUtIQxuHW07XXbDx_z$O4OXK?n6%2R`fEBG2 zo0TdP)`mZQs!txM-cL9~F5bwciSK6TfbVf8v;3-l@3QMK2;YPRS}nXwnR@ljig_n8 zy%hLebIe8`iLc!yTlC$8MK!MTbv-=hQ~Dwf|7%G|c32V!Kc!%W0yD;CrYTw^!0j~% z;y+|gO7$f;z5cI$JYK-er&GY>u5x<4|LjRppUKA$i$mz!y`-_91qcYqbxK#GttL(w ziyqO#$9P#3zkA)p=dL^l2Dv|6%%E3vI9|UaQX??Bo0rB`3!rgy?qsuyk|w@O&x_sW z7+3jV#HNU;l~fh$D}7QCgMsQoXTlkQF%%nh0p1nJKqU#QNw>5XpPQRI6}KLvXXoT( z1dceHTF!zF=YQJm4Dz)7Kt;#G`J$Nb=H6xVed~XBW&ig^UctE?OBv6@xd%=Ia9;#Y zNt=(rbwF=%>lsSlyd9K^Sx4!4?JjJY_@i~a2g?OZ+Gs-bRupLYQiv5xCqEKE+!N@E z5O+$g*-f;lH}m~!{%v3(uoaQwm?;9G*SQ|3+76Kyp84&E_1VpGBp(0q1d;U*>e!Y+ z$Y$`KpJYrCyNf>Md&scJ4Zsfvb~x#2Iz|MDg)F0XV)}WazI0>ww+2e1!?;$QD!@;) ztG}=C(mhKbHO6;zC9P=AxTR6PB&u z9~GLW&1npqZC@Yl)JOil+LxiVgXOz=u}aaXP_>654!zyiO#E%mgWkw)mHSrV-3My(P5kto30y_Pi>wI#NYf|ehlivKjiR%R{h za0>`pM0Us30ljU&)V7zO*2mgL+>v8y=RZ#pcZO%4WVM%J3l50GR!n`z0G`gjFAkqK zPpRpy5vPF7u2|7XmBND|*$&QtKxcUSgq6E9(vBaLuF8>5O zfL{ksG+(+LTLj4nuoh0f!I@(;@`1iy{o7$vL$Ke3*UEC1+NnxyMUkR#=LJ)6h+ zkBaT^6 z*BtT%4R%;tuT^qB1y;b;5cRXNq*EaG=Cm=|HsWCC)$;ClVl)%%*Pl7FjL>edK%K93 zUdHS%^?{52ikF>s5iL(%lP#OjTw1AjiinF=1u@U?Wxmkq67lQtW|@qa^cGl6$Qj?) z=&9eyp8`gxZCOOEd~GRE^*Vk7+tQw_F0~p9qoh$}i7D?fo`$@7z0_g}M}?a1+VrGsz1?+T(a0{oz5k&M=+^$mNN`TmKrf`)~4Mw6jWRSDjjZ$>4v+S24f*P1| zD$Q~YwtUm}Sjdlz$6__j*4@B{IJ!aJPMxNCF{J+?G%ci*6s7)I}u+T_CaED;Q z-95qG-8Hy7B)Ge~ySrQC?(WdId!y6$Gc{B7PQ4%c4|MH)?RBkFjy0$2B*~lgZAa^pz`_l{ZzTG^^u_3Z?bxQ7Iuks-F|b*|8zxByf+=G zym=i+yd9?hZ_u(Pi718WV)^-Q`eo|!G~iG zb4@>_v8^DzbUNI-%Sz_U!(?s)WFIL#9si6XBUe@myX_sEKdJbTS6AMn{_PUra`7r* zu^)sF$`+hYGTUW@;x*DRz1-}yK)c1)b?}QtA)`b@T*_ ze2BsK|9rb2zQA8G8TiwCDG%p_U$Qq1wj>hRF}j3$hk*SXjGqu-x{9#cb~*eI;E^La zv;{c}>Y3FA**<1z>usNsm*kczW47lvs`AC&x0s@|x%hnFB%C$@A5;)z`8>AdTC;WG z`C3@17^IgcoaJNI_SInXpAs#G7)r!E8e)ybXLd)_e^E=gmsc>*8k&V>9j+^ETJJ)O zqViv^F*;L+0jsq6f~wPw#izV$`WtD{N~0ye_O7^`HVGVY6X7F(EsDpBmGlL*ls@4L z2_NbP0epk`L8i(RAvV;e&hA{5+Hd5QMY^mO#)m!d8mh{Eng@fjBx%R2w^7hXWD5jgFxi*K8ZlWLipRb>;7GFOT9(j_!(uXlLc8ajF zy5m-S3lKsyJnI+5FIWLyCjQJ?S?M&>OS^f~I{v4(C5WZ#Mnb3E67&3Uj+ZS#eA8;y^z`M@>)Au3a3V1zdE`9@?W z_u*`|zb8mSrXwR^4%m{AxRE-T<-~o7)b{tF`|ao3Rj@z-qy<_Gz2W8s!sw)8A*iLj z?XcBja2OvSP15k+j;8&M0i!t`Y8Pn;2(^XTPTa}$6t@yUK1anhW`(hhC8?-x(SP!b zwHl5gF=Pc=)f4i0{0M=vsBv<9c{sltq0F8p3C90+Zk^OqNcDeBJpUb$;^gp*@8ay7 zlEePjaXUee1&SHV+cdFF$jerCK{FMB7Ti+)%-iiz^_V47VV9_z-@4zfm?{SYB2U%G zG)C9y7`DXyH$v>@&h+N?RrML`N4+18zhr^`7V=&o36Y*0R2Q*n{6INn&6nC4XGBz!zQMe{m|L?S|A? z&NfsIm3npFS3?>cn$Z(&@|%-v@=(Lee*E4Xrl)2~>lKWYTFVvU<@VLTbE1mld9A91 zBSsf7Jk@-(bP(yV4-CI;}ZBv4JvmJii{PVk?Unuqy zUUv66lL(?{225~dk%XCdvaZ*GK^mt$=|m=v8p@N@c%^f|*=DC!60r{iYgPBSkZUBn zeLmuLNi!~MxQ8A=Yvuh74^DEtl4hFeDOazq8*ho=UT!Efk9jZNEt|c~(Ve-KE3p`^HRR%6Cd$mA28_NU8DmvxHMC}{NCcv_S?=I96HhbC36;Kw>yjP5px z21jy64eu|{jdy}Zfwz%gbC$6T6LKfiF2ak8N}DUq5P5tBMwyR2&Y{b2oMF3LS(Q72 zq}mE$P&oho*Cm#_qG@~C`ADc^*9%*RwUJ;UP?SNh)2f-m7o2L!EB^JJD=TNx4w2z2 za+7C^jJlhVfq#UlxmAlk1ro0 zK%A)c|7YrxfU}%{F_Ob^U8!Is^y(v-hBLgF@q_+~b;i~g&Fv$a(%@u7HSJ`QEE3!IGOSc~;3MJbCh^=rH> z%3i$iR}$X_q+dZK;6N||gX0f#3ou!vc-yBO++P#H6~sb0O#ykBaV% zFiRx_hJpXXN=A}qD=iX;*IK30;Q-%;#CQ33V<?t1-N{SC~8ufpFf9zJ0m5s{oP4cziAEB6XL%}z`5R@myn`l*C6fBHV1-|wSA;+1?`IJ- z`tdKoh;CZ;tQ{Ok$;*hchAn2?46lKPJ~4GPWUe<{f1kkUeqi99$5qGJw>TX}^-x8jp7JeN5mcoYjx4V`7(5Ab02_q%55`cZ*6Gj0K81^V zn#a5nw5qrLxHI-cy@A5y8G(Rt;u}d`HdztyZ^>k|91y)TmoCW{VXPyR1f43R7EMz1 zNRps-1^)urzh6f)TQU0gB3x92SM9E3@f<~RZ020=4b4e0bpBXlkuaudNlZ!_jT=wa z^GSsxVojre-;8#n*27}w&83G#N0d&;UjppYnNwfxm;RK@Aa)PwTl~-^bipN$c;vnX zbdB6 z9V%us^(k5SEP$;vRaZYZ3cvB5Dvol z{yoiB3=$@u?7JEVT2j)182Zb2MJpU-7jKMx;Jw|c*d@QlmkdY8zA^`bs*hQH?|la@ zNCj)?aBht{pv(;L2PJ9?^+@*YaPWc^Z0BtCd=1xElKB&)wG_2j<^3HOlY`#eY-mtA zf?2R1_B5JW3`4w$|5e>b;pqH`q)xlj;w=wFrelQI?H7h4evWIGZF^rso&q{%Pm6b+ zkAhmGq5shl$i$*PheBAU)p~LeAa_)S8=lXw!ri%Vw~nb*@9-xWV%gZUekn5hpG8Kg zxQ;rd{uc;S+K&w=>krF+16K?fCzWsGdStCiVsT85G3s#Z(#-r$q**c%kuVJ(nkN{I zq7_{)k+G|-no=eWfnOxg&ZBtUif{nN0G{hJ##D_#7mR-yx{n#?JGpuSoWRbm(?*XU0X7}iqMFS%1aAlchW zjmB14UPnMx7&5m$j66vfaVQ^Ws>^l=idD@{qw#FBsfqK>CDNm$lhb6J^Gq6%P(pdOmi_i$_7{h2m zoa5YMrIYADf(LG%?Vhhr3ll6y-Is4nka{z32X2tw`wbQ@tGT?OsjbK8=;%?JWmWFS zbM=RzzZr`6;mMjsKg4as6P#>Rw<(n5!1xL8L4UZv1sYOngJDgdE(mTHQH2v#qA$5*o&&wx|Q7pEq#qh{x2h}UFy**?|A=rgfh>^@99C9)bCud zdt&}+(p}&<4W|KTay9T7=IEw32AMOg*e7Di4s37*#7PLAp4f%*?nZvCdupguZVl5f z=6g0qRGb5AA>d3)mua<;CG1Jj&+`(eoJ~Kd@>ZsQ*K?f z!5J|ECG z)MC~AdrhBY=H<5L#Y~ucceMnXtwrZ;|B{M>!SemHcl1#z6>bQ1G!D@qUIppgrRUk{ zfTwtEZyuTNsjtzE2`lJ^sD367T_%%TC24ji-N&{KV&jOTZ{7dK{dPBJaag6O%l{0+ z{jzJh@vV!NsUc^a@hO*ZeNOUP-gbd{>5-ze`x=@ZiS{EtRp08g1{9S84@7>z7a^>B z#Kt?=4mO~CcQ6>|Ib(u@AvAR|sAf)8IXF5}lzxYn#@H|_v^2ViEV@}_yl#J1=XQ^_ z!_JYh`38fIcV5szMKn4xCgAySA*Qx|Wy!tUQa$E#8%*d?GR_I?FYY|nbq8T^5r2cL ze}@O;mfN(YqwM1NI}m?jP*c%05NLBsbvE;CT2S+izrT>o8csaGV@}s=7sZ>%#B zh3$!`he73)H;pz3HMK^OV%@Q`6C~#_)W8~-EDw{{Dhmo$!@!y)@@0z@brmn> z=T4i=o<+(ijsHfKiHEZv5eYvjk@>zLk7A|E7ERc;a18hmo@wW}3u0JYxVuAP_2?sa zI;|!wUI|xKkeK_mIGxA(wK^lK+xgRJ|L12T0|$TSjR2{m)Kj)MmhzK7B+g1JX0_`@ zHXwF3k5!^l>PB1XRy96Hnp<~PzOIHs$3N1gr0f4(CI0OFjiduC2EyY5Dv_Uk0wh}S z3Je7)VCJWD@HnxZ@c3`yYc!dgIDyUovS#49kkRX+s?ab501~CH$huxJ&lRURRH7_9 zl{epc%cJ!MdG^e(pIQ5-z<5@W8v>EG9ULeZc{8G#iu)DWAqd|5d_-uPF-u=#UTu_f zJr#Lp8kxiK_g#SyV541}!bDJ;s<>CsMYcOW3E&V6`Vlp#}v%T>0g`jaO^+xBfZ;T_Q|+6v(Q9QLXW;{>k#S*Mg%E>8|O72@Vt>hzfmG( z!wsmxlVGUhN#%0?<&xH zR^QE!WCmwo89Q;Lqt8d{s63`xhf&@ji7>)K-7w^iF$V{ESy4X*-3Z?_nCX(ucE7st z)7_h@>Kx>xnXCzVAon?##s!iOw7SD}h6YEbWne&Z zzTO(2+H)G9VXveR+jr%`x zXRRyo0BvtF6iN05O&HskV503%m3vPfKTtP%>_h|z&xDeEcsNB{fVhOOB?u_EQs57NOOnAt}@}+v}`~(F%DGK^{bqX zNDz$hyJI90Z2v;1nN$RfWC`hT=e?W9!v)c}Qtk7QFS#IpNT@gdz$+6@{}Pq1Pu>3b z2)DgaI}}fCwizMapFH1ZR^`TRZvgSAW6E zeXdxZ^?UK|Mz;tHc`jP~O~+G&AQu`g?~j+QVt-VA_0Xd2F4GBI4~P}XgqlRPymy-g zqO=v~M`wS>&K*xFtRGu$@k=$d$`>@LWk@+LTfiA$_HB7DqUmAM1^WmP?i?^8Nj|Xc zrRvy;60t09$Dj;K_Y|;l{tIaTl)7biYKa~fykEeveXB2W^BuTOxX}!NM-jxbTZTy& zKMpaH&Gi>0{2#{*L@q9OdRQ?V^Ks@_w}Z)qyr@>R6Za+T8{wgfoEvwK-5;IUT0$1n zF(O9|jts?baQIX~VzFcq5mqtZHR^ih9-NKV2N%2hal-3)RGCo~IkZ}5Y*D`BW6|cL zQJALfa3AGBI#VFI|B(n&eYBMB#K>l;U3&$ zoQ!q(D97A7rH=%q9+SWM;son{-)6&J>ll>~tlK4&{oZA+Onr6kYk|`D@{)>{5LMb& z0goFsthorT48uu48gE#2yKYgjZJK)Y2ub#2*N(t(n8uvNWMj1APU6t?m8hNXp5=$g zALjNbdR#LxV7tR5xqO#9LE=72fv)v<({8=lo@ewNq$jNJ*H4>Q=f`QeVws}+AV2|a zyqF3X!yIki)R*MxjO6S*%gdCJ7Af<=`RwVy!|@`cZ{LT%TC!5@uq*wBfj{8jE+jX% zQJY@BQ;qWe)VgsYIH;v`ovPMM`;+De<~Zt#4`bHS~Qj5gr9hUm%#K!%zb%4Cb%-x)Ju)z73j%SNA@$2faZ53%xW$#N~XS z+zrN^2ow`Ux}!|O3UUK65i3mbzcfQ-YZlwLaAez2O8(?7yLUZV)h9H5I2Bv+%11T1 z*hHson;efqUgg086n(U`boV=t9*O@Zsw{xtT)39;P8B5d$W^AGEMV?_@D6oqpjYIG zva*12wzg)i)wZVlwHEDSsB4B<2KP*=?T*ua99{?5bTFlK9y{+-r-eU-B9SVH$IDX= zGBg(E+;h%y31=<-d&)cJxGjb%bU3$g|)w8u|E3g>rp z@2wA7^c`bApSxkEA@`>HZRJ#xiin#bpAb{qYo}Efievot@|f&B*7KifLsC11D#;uZ z17qgpvRe+KtuL)-!A{et@cHjJISW`~(EI)&+3=l7+5SvbaGbCZP9fc%&Q8D{;z8s| zhx)mAHe#iC)*Af}6F2DkPO`vAh`dcUVXd4fGSw|sd#UqLtnFbr98qLUgF`OwOvSg5 z05vn+JmwbRp8JBdEvje^!0!7tWhs!WCtFd7uZw$+Ifv9e3zm&1dPPB&RqY61Ek!O_ z9Ujbkr<`lO#GJ7nV%->yUsYs_S}i-YEhLFgjQ*u$z9{xMivx_GFI2(=)mPXPc=Y{h zGkv~1>ck?-*YatKY)kvsXs2jGuDz7zD{(e9&d3r>h5A< zU+=9B+ZkNn;Btd9dWtSlHscWX?HB$sC4NuKClOkIXDm{yeT2BsnJvnC`or*pW zUT!6NMP^I0Ki!E!a9|4Y6JHZf!C;?154x6FgcAZj;z3{&{~ZONBA?#Sz`LDe1jawrq9TMBA`S|jNs6GkHz=k>Q>$nNNj%X}~co9`~et-yu zok5FVEoiuQE2ppUa@LT&g;t|Zyz4erZmCM=xCV$4AIhe#iTDCVG3|6vIINx;WRkor z1oSdRG+!QJ6CR(lv-f&ym(JRlvk9`GykGo^p18`-BX zKZj*B&5w}oK)zclja@n~Ywezo7dA1nU4Czs&t8N>R6IDh!je!hPo2z+HN?wk<(x>_XEyUOa)R(61xNK06d zUFSz37M)?8&@d9MNRm!3JB-Fq?ND-~!=F7Ezq_~Nc7W$MD1<|4DSWJCBxoB5UvT(OQ8E~Ff)ZYP^>Emz(d^)C!MYQTVXC6hgAXXKr z^I@I9DS?j;U9^!pZ6@CHux`KnMj_d3Mx5rfKGS9UVe88E&Cw?jo^P{pugWq$3=4QAup3DI!b~p5<6vNSY z<&zcoWz^^<@U*6k4u+Z~*Lh7-qbBPZ+2Jr!iTSIn;AqiMS5bLAKcBvAK6;On{O{AI z%VK=f`sA^Pp^JpJ%_bjCz2c3#-x{t;f;HVUMKU+tnAYD?-&W47YrF40Xh*T6&X?PC z?{f(kW{|YLCoer%{??Dar%Vln`-Y4Uk8Gztpg-L?kd1GCqK|Hb>#rH+FIxl_q79<< zDL9}yB5n@$s$3lHrx|FUX@b_K;QMa6F^ZMn0viStoSI5CL)})_zt?mJle`j#xU1YJ zM33AA!8#;Ian&*U_{A&B0&Z}m-{sX}-&vPdrpZKHHO2K*qII*Sb1E}!bYA2=W8m&8 z!95GC)t4bjW2!uN334|IIE#m63abwYAUg}{7u?Aa)KWC1hTg2!$(Xe;XicL1FcjIl zmRUPm3P+e7R-V*M;QVq6%VhtAw^dRVfu&Y~%?*NB^nNpbS1ZW8x-^~$gh&pktC4v{ zE<=tj^QJ|*&pNBV69(5OaNv(I>jE@A5a_0IuF-Y@51b|}U`_!d4cK5IqvVc8*KTR< zJ=Ucv&SOzxLfcX{uLXGPgYFV13Dvl~D5Tp2DKdF5 z4ik`-ic_kb{QaZM0umF|j&O9^`T$X^K62C0zwl>s>gfojRXbCgv2N<=(q$u5moeyU z81pCA9=%;-U{enf)wh^Z!u86Rh}{8uUd=h&y-&GN&(wKzwGS0_b!dlUL@GKn_6v)P z73FPFgqHDN^JNopmb|sAUdN(G*KDnmmvjT^nmed?ww6h?n)z*vJp-JqvOHFQE!&4> zgihg@Wd$c1JGHeD37a!y)r)p~5#GOs|F){!{dixbApV&^Lys-9#xQ)>x&K(?_Pt%SytiC@eG&y?8cX|IJB52t2_-kAoT} zChdVBXNN;FoJ@58A*r9UM3JWaN!^T9+knmi&1nzi>NJ{gs<1Wj{?sNqaDGjBRYqZ=kT5XXt=?CkL^ON zzSKo@%02);u2vSjTJCUku`PlS(WSJ5+dASS;~8@|ZK*xWpw5ufYXnv&IsOGg00+%k z6|C`Zeu-}*_1D&nJeCK)3_KSlikcH;)RnhH`+$b8R03eXa8Ti)Z+fOVe<@|HV+!}q zim;QQUEI(}{lkmbKFme#Qqzd<%X1U26eu`rL8zz6wkhA+`{Rd6V|z2#mS;mZsd;Y6 zR6%1jEm+f5zI#!ogIGOH=$qUCOurDeins9A0sge(@RhEDDlJeFMQShh6PMdl2!KE_ zaj~fH!DJHGe?Ia+q#9+a@^yw=e{52gbAMX4^(0~v#b(JW7`>=6@-PiGHqC;}Nkswv zD#ca*!^Yk0lHS6aE00qwy6NO6nU2Bd;*pTp;?WSL!Jl+X@avU{ZDD{Y3~o@%Tih1B zQ$xaz75(OsL@+9tm=MgG8sS^dG)FffZPnt%(Ta>)mDtP-$8uDA9&A1B8q{#fJ4Y>l z{xWs5XU!=qVC$uqd@a|`B+F-CO_7=D#EiL@PxH9lunP-EpboXLx>^BlSbGXR968>b zN}{Q1RdDB@b&`L$T%o>Xzfclen`u&Ag7b2>soqcWs6sK7FQ)05n=eJgYfCh#A=m}2 z%SyHPA&w@RHJE+HuTRJfuNa3bh>w1?z1SQhbvTJ(pC4lHllrWfOlxj>#Ypf~lXqA) z{9(*Olq}{eb3YKKH7sI{b<5+Fj;d2-id(82nSu8*BC1JEZ-e$+rdUM6#GxWzh5gbz z;g%vI1lOxW+jlE}OV_T*JUei5v|km6u`b zDe%$Q(=uqV27dvTR&HWg`kQPC+&8TZd1uN*a$L7+R(*eCc&@(RJ%SPf*d@QGL3U00 zwX}>z2^s9>i**IothBPj`Y=}I!q=a?25O}{o*blig0+4g$>U-dh`Ed~T%~8SuoN?A(T_@)@pn$tQdov0yk!O(Qj~;Up0j_rK&i{z?sCdahjp zczng{dXJgUmdU6q#{W6<8MbYSGe@TX7@MA|$A@$~o5uuqdkJ1pin7`9%0_K|J*=7psn9v>x*l-^6rb$!Zq>-2{r?0cDakTo^2$}(@8-ejM z?HkvrJbbfc@?ZiBqRAPtWT1shg=SMntSm*46U~y1HeMsg5A^H+ zFYl7|0mQHCEU>Z1a}|}}$V3IGlF_$LuI0f@gyKWH{Mk_>ex$`@lR^_3%L?mNE%Quk zsHj)P{+0Fh1Wgz7mn9@my;1x`l#$obYM*Cw-{!LjXXPL+?oZwV0Zwr>DUJI=_cM$QS;1+XIReSF2eur$oqsW7C zYhWF7F1u-}+JzQ3=ZIO7E0Wnz-xOVXEf(J_Oa^AJN~gX&n2#2j(Cjxq>QJiP)tp(c zMR!*o7D>D*w%SX2Su3_%={_>tY^Z#Y;<*Sj2Bq20GTxek_bclQ=%Y|Z^xSr}Zk`U9 zTX$F-jV0;Ew;E)+K0bVO-a*Avd&)2gZ~NLBjAlXe?EI8ln)eY6y#dQEv{QTynj_V3 zxLsYG|ECM!Vd)nIYG5$VvlU&uh2y(uQKU?><4f1X4MTYsQW{=f{I@}!Wnzx4cZpQC zM%m*Qv3%a^D-L5G@r4XNsf!jHhnQr()duU#iszZ|*`FrNw=B0LA}$@uf68Ay?nxiH zJrlc3@I$7RlB;fn1TIR!gj^T@;Zu0K+1-MJ@ z?;clN3XxUr+FcKJ>3~4~72&Z#*?74v)^k3`DAN9zB9MympUs@_wcRx-W~Vx;qXVqU;MDB^>?arEKq$op=z#}}` zU`&d5JN@He+YIt0UHhaCzzfJI+6Kxvon1Xx0#PF+RL($x2w1M zwWn-V0(~;RG+CQk?G3cR?9RBAV<1eyIMYMC_!l{a;10-);sD-ZoFFzaZjw8&6eX@g z{*@}B4d;)Gs++)m>z$T(@84Ncm4hHr0Gj|Z?nh>oGUm2HdjZM4b0HUAGL(aMUu=?$ zP}}xsiq@}dI+$M@mM8W*3;DG$CxMJ+ptMcQ;V=#ekM>ygYbLKj%mg&oP)z3+oQK!$9*)w9gs2${5ZG1-N~66wY+%q0 z3Vk0Wkx;v;4h9i9~A} z{8k@Q6~6!ue0m#T`jXh681P(WQB%B%spU1}Vs7L$(|*ab2p6G`eoC9qY4}+I=SewAu=FR=EAC9PcJ|+G-rO-LBZ9U&v(29W!5Z>rChRH>lC8y=FFf^NCy5{qY!DVS`%5^4|xX=P>@Y(#S z@#1@NB^&pLtMHtI`Gp>(0&Aq)vS}2io&v71P;AZhRoOHf1kUUfKJen_6FAj6bpZ!{ z)OsEB)x(=h8Hc}Kket*CRlR9(PB3T04@!i;?&P-DI5!JbrM-~2KlrS!-vFv&e<--dKH$LPK2@KSxSaLymdCBwZ+efy4AET=Aa$#Ta#Q7K!Jk z4^U-h>#%Nx+4Tm_-T0lgkkx&$bGGhQ6MSNJIO1)|{O)G5s45_qwDrQ(`UdxkOk)`!~M%OfW^1DIgOO`n-}esAn_o!NojlH?e+)gjH)qfvOHXdO*(<)TsIWpkZ5H=Ov^<4Tsx^i5AJAZF1 z-%E$AXU4*g%!Lq#%30dTMYH^sh0 zF}vW27%@wfG{E&TR2xvNZRSYUM=ePkTj$+bGLX^c9$WgY(H@j2s@}6p$gMl(?~hbW`bb_gr{eZeLi=tTM+#d zaNT6>!oCsIa?L45=8aO3k{JiEUm@TNu|2nqH~MXJOh{n1;&;vj0zsc?t7N`xgU;g2 z^fr;BDUzXgd$i0Vtp;9Vnnd1LcqnXIPR{rfrgREk8^S+hHIh{%Z*^~`OC%8O~g2sWv2Kj9*;Slqt@SZds64SG%l*dxyCW8)M>O_342buVCtNKxV;y00Wx*l zqxLtAW{-sm48RBFuf!eij;WUfrcy*3#A~iLiH$V)axDvjn}jTKDBFE-x~q0jPD}JQ z^4#G|HVYmpU9OkVtt>4Pj%!A)kX;e-DAm>5za=lkICZUQiKM6vUdMY zJ>JpM-?#)_>m~Vk8UsF z)w6p|oxcsyt;V}&?zhkTmDeF-dc88c)dsy5M^$2A70&1=C5IG^&j|KV9(v6g2l4ef zF#8du9!}$apT+@!;|`y@Mv1*bhum?QL-jIQ>$V{<^EkP6utK6spqs)Wg^gC51W$K` z-t#)YNB6E2*^(&H({yR91^1mvQaJD5FBnQm7<`d_NG96R>R&98J=d?hY2=~nGN9;G zZKMPOn{JFB_Rg{3rU=$y3o2(PAV{jj9)&_9i%e3WY`?m{?H<~Q4n@?Re;B^Ht1Fr9 z4#oQ3@0fW{Dms)0<5z*#8Efm~^`XfWjoe7s0FsGICkkVNx;M}&wfTxrT|ZGD)# zGLy4d{1o!_0a~C^wlQH3s4;O}^bn4C=wKM?|GMy;(0kE;T^7{EMFNW{*J-AGen>*u zozZU~tn&BPdcD6@oxbygY$J9;AaimjI6HEb>+lLrBcgku^NSOT(Yn|6@*bzD2?qV- zkxHpe>?EfnGd)jANgbxc?w6HPm_y==rsdEneSsO&cEDbMF>o#3#hCZ+Pnc`osNF|> zU&{LYuLBy%NzKoDjDyieqqj$PBmXS3SFik@Le-3X|1?|*_!nS2@D!~HW76yI&?Oen zJ4rb2kgR-MkksC0?eb#t2}g=RsAm82%AI4U+y#8cw==S%Fl|~L;qTNWRAlIyu8@aP z(;Jx^ToM2T?23Paqmv9c_x|(`GjX1ddIv>ws_X^uPDPmMFYJDUqEXV>Ggn5&2@2#? zxu|iQ2d)tv^Ex`;^7w6Gyb8hc-dz4L=<)o4`a|A**YCx3Ec^-vm%{h6LeK)^(@BGk z3>oTRu#j;{B(IZw5_$EHyaGjeM0kr~wRhfmPIJ6*)oh?E`#k3`&JUA8aHtpOItAj! zDmfBwMmGEYZ@^>{Stp$V1-~+F6Tv z$4X@gXAS*Bvn~*fc2bah(_M4#Tw!{aFh_$r93$C0O4~I{JO5855fyNl22KdKvv8t6 z^t@@+xw#u$vRl>O_h$;#-cSk6&|a!?pI(KhmqHEP*Y<1e$_GR z=vftf!Q>a3U(Z(DWw^!@i@xz1=$SM*Ht+rp+y2QTYD*}|c$9nLKdD%38rmq0?~Z0X z2CN44Tdpm@;}cQkD^l4BL|*oyWnTBjsm_N$hmxJ2tsU>|D*;kygl>jSyqsh9&I6gz zBdZ%rCl8Cim-GvFgF5X7VV}*tRBed73%82e)NQ)}OH;5`roXgYySAHz`U>4go_r#< zS*Q-P<#lyvK!d^q=7(3MI6w0(%V%6@$$Z;rYmY`})XO>SazCsqvO6lYb7YT#1}s5a z7oC&Mh1~9T8_=z1@71F*a+@Iy$@R*!RNXJRT82&dG+dW8bQ2&WlT~NeZ5FMrr#=!K zSgr5rqb~;GIs!U=J_pO{9S0b#uBtVE8k$r^BQry|;cr$KXh|t0{#&~noBuU}k0Jwn z>m9@hz^`ULl8dgO}N5E;D1rGpa49gQc73Yjg#UMzV$ZZMcX zbbSO<1SakJj3w`!S5UJiY&(hNd>#$JGv#Ju#hcM0S+(Tm2vOJAia~hw@>SUJ_&&qd z@Ph$ER~CGW^uhL5c1mjs?(l0Ne}=LtMbc5l6<+KLWqFPl2`lOxT?#s%+Nf>|m_ZyX%Vu_|9V3lioFFl(lmfbm%pz4Upnf$%`po3OQh zx=Z}65(R!C-!<3SkJTF=Mmm*0#7bFO5X+HxrXBNGpdP&Z>Ju|0sOhDaMtisypfHU2 zXn_NghmreP!Akj@GA*kW5;P&yl@k9_euU5jD`F+#V5La61y0gx;t2saY{`49RdgfKfJ>z zs<{0cRBp_jK(!BV?GfWmnFKHkL=0U6A0atVaVt5a^4Ako&%%2u8&j&IgqZ=)UucIW zP`oc;W?2M$9?EIy=^%&|*V)rRA&NkQM^mj`*@bQNU8L}4waVW?mLS=z&2d6`gk;El z$aSQw{c=w6mawxixPI`4V!itlNXcY#7fvIxGcLUA9f>p4Musa@+Do@Gf=86Uz9Ij>Jt~Lm6H3JU+_v+_OS<=8CJGc& z2HVleKHgZ1@?!&sA=;MR(E2-n* zH@DMD$K^Edjl@o>FP#H8O7VVsht8!aY`VGXSDHJMsp2qm1lGaa_n%#Cu99qR!O00FYmskEiU*{#fPM1)aq3J@c2b=j4lCsTKx?v9hH zPik4!a_3T(%2$$TdT86r(X^NqM8Q9@nqwb4iChJ|assB*#%eeArs=HT6yOCve|S49 z5a1fvpQ+Ay3Bx5BvHqOuhvXQ+S4VG&*$H#!J11dJ$oot9qt(2gaCC$>)fQ2*#_&@N zN@S}Mp4x%qXa286wudPN$VWL&Tj!JiLOresFoLdlw8vhgUxdPC<>3-%*Vb2Ht>=v? z{00zl)O*GXE-9LYex$u#>cfJpqy#DE@vC>TJZ z^HNx7*@fTM#^PeM;nH# z73etlo@g{0eOqiQ)3rVns5hiQ0jrVfljU8HEClpSuHTX&iPp@+fU)LJm zNK(1E%4&uj{OKIQgJ%1arQp%UA6u=t6h=K9^41l;+!=Io-bOw|lP={kZHV_J9`Wn! zk5bI`3w+F#FCvG}1BRyudcW6-FVjyap-!-R3+(>8YrvF2M{!_VtsB_OL=fUX*gu0P zuGz1Q6!7-GF2HH*v0028bF z`MZ4FkTs96oGuRc4!+TH=NQbLyckpP1L_aQ<8LRksDPuy@IK$IkFApYZM#U!^j}_9 z8jEQr(eO#N9Y?WCarw1>1}#Y9Bo!vBAjf{w{$AKzPOWTUvfTeag~(YK_T zc)2pCuF*a$fsNVt_VxKfKIPYI$O%?xPvHqGQ(*L6Bt)tp}gD7DNEl{*ArMMS&cPLO?3&p*7(cmtn zxCD18#ogTr5C~S>2@u=~8g%*Xe*5k2-|uPNPXEGe~{Fbu>Lrcf^kbQ8hnq}ddODL!Lw?588@zQ z>~cahtFFr->3>6TLhecF122b#WzROBry19sd&SP9^yon8QDLU`6ldq#Jt?j zzg$gkY3UrTAS}`+v+=OlIUmZ^;tM1Aq5siGvB1N#a`|~ZTNm58cfV7ASl!@;N;kwX z!3EHBQb*yCq-%VM=a@x*RYM*N#4hwO%54HvyFq)y^WIEP#B7R6<5=j%bGz(Fi{cM+ zSIgsc!J;a3DXp#!w40@L4@#1S$rl;JG6shYZRbYpr{{gL1mQDl&xuXayowuH@?jbE z)d?fIPHL1`S#~J1gLj2Zi~K{ULMTUXBK zTeUV;xmP)s?xSX7{=b%-T9qL4#_8GS2^rRulRZd3DM_*Rzu@Gn1(*KZV*|k61&_LT zGL}qBVjZFLzFX+|2XhfI-wwx{ib9R+^87xhNw%R!B)dkJ8jGupg4^~3vwPkB$Cshy z0j7mLw;p#qcC)+dlmb7 z4E!2&shri&g0q>umR{U&XB$u%JX41T6ZAP5&iUHEZj%91h{pxM1`#SIc6DG)~7=kw~{`qq%+-RGV%OX-n5W zBnf*Q)6LZy#SIcqa3NdIq*w?zU|$OJCd(1r<%>JZAi3I1nB`(T1tpV|(wI`{qU! z=~Co1wQKEsE3tNyV!&nfAg%(l=I@+p6A;w0}(IX{}al@TCN7^G{&&%ot@)0CR z1b&7_EiM^KLG@1a8o*5k|IiTU!(=lu!)M=%0jPC_{~Csu)A(m!M4<)PsAlz*>B_+F zz?AfolmAd)ND=>baT`mOL`pS6w$%jPeYdTw{`&@Z+2C-9E*ZAj_PD0)cGU(q!1$FG z)`n6P0 zVtn`^2@?jYs2r$(2kZ_ENf`Z~czhbF z-Mg=Qn~a298-sxJZ?56Joc5pSqw!xc?H4OEaaq3Ve`~F`0=oSgmlnFI&?>=zSifd< z*rk?~-hMZuAHMviI(e6`B9Y*EeGpwj8O$q$%uGZWOX?|`aebUD8GBfB$4acQODMc7 zLd|Ell55N^95T*W7hBYl-YqzeHk&(3beiH1TE-c!|UcIGPZXkp20M zfbQ{@w)03QB+KP#zEgEA06D!R4>BXn3ZI zQ(5?3-UGdc)10CoE^)x1t+|HPkL2J7+ouRuHJ?IVA4m}>eBLgd9zbkScl!2*pH;Zt zHI37$YK%)N6l>Bf!Snl6f1ivOnXE}y)bY28ho?7Q&xpiAOS!_$bmhuzqR-e_N;~&F zJ`K-4yUQi;St+sQ2GoD1L)P3G)^dM<3{cnQf(JBm6iqgN}w(jDc~Q%i>@Ux08F=+o@yJGP?T7u(tm zo4PgF!Z0V|nyP5j^=oj0Next^BviIoqy$r}2qlZtv*;$e-R0Oaoy#b;iVhC^Dh%gV z#lemvSz#`q*!p~Ps1V}5U5nWVyV)>vzSSn5?UBL2Eb3Ar(DC_UM1VA)<>7Y3pSd5t zZ1H1M(u}Qz@TwyIt%f)QcU9VyPco_M=)v^1kSFe(V_E!2bvBj8eWOCnx2|X0Cpn)0 z^uxM#f|24JFSR43n}@HZDWR&Yzac5ZMHbxK=M!X!nVSk@KocSNyX%w74v9NH6Z<34 z8Rhem(w6f|MX^qVE&_oEO=*qN>U-6$=g{TZ$nvlz<8?My`q-p`>5!~xd_KtUYU{2= z;nZy_kN%lhj^`f%_HPs#5*9bvFsV4UXmxB$kV|5O33YkC7Z&nki4pxPAv#wW5gE^q zf;|X;C09%iwMRBlt6l1eeRB9pU-uGI6f)AF4q0*p#a_b1+dG@?&v76@U zY716whko)WY|*PIbN`)$OICKSzS2K%oIbGdL;H7dh?{0i?u^?$u!?Ld2oq@u@V=*Z z?$ZNd_;GqzW&yqmmk=!WfEJDN2pmb4Enr8ChDQJhiKRGO`aNU=#a%*qOk}Tr0~?=v zKa6J*Fn54_Y<<1GAZV9sx^jkw!qfHEP~=TaK$9s!!y6i6TAfhI>tq$BQ1n`QX>E;9 z1n|L*52M@)7IWa8krZuz_EWwhC{C`-*5`oM19ovHJUYWakEwnO*%Vtm>Vc;pn=~Q* zTyNl8cD7L4x#b}XLe<@+3cG)bZwtOF=ukfymKH=DlvCo)1sDigzDCbUgUy=Vp5mmK zaS*e4oQcFZ2IsNlb%#>N$BLMPa*lfg&Z$0W1EzC{S%N(@}9)g(V8~mfVI7;R3IMG9W)` zbDl^Z-@**SUN zM{GRgP6T(w$~rD!^u{j#f3k{OWLCy(HeI zmm|LJX{b$<;y0sLO%%RGjIP4Z#8;>>o$`?*3h9 z-6a1ahWTW#*)s#wPck)1b!cHHja;xnArCx=-ohkWu zrRl!B*Z!fA)4@$osI<=BeO}DxXZb#9wt?edvxGCjpt8|;rQpPzW~Q2su%qZ;*uD~p zwv$@flAeV&cpnaLI)=}kbUHuvpL5lv=pWfwprUcxsXWq52Kafe7TfhCP^o@#>Xm{I z@r3jbbpHzl_!36#+s7$YQaUxEnVDqEa#Pg zxL?ify}rj2`b|(@pXvtyn8XdDdK!L_m*RwqaJ&!4&PYA$SF=))ThQaasV26IC?e`8&oB$wC!DcVO)0c~E4u`FIW z%KYINx=&&+K-Ycwv8k7M&M|hAT66v!4AC{Q>9YjX^Gdk+Vsq!hmtp;9N#4V>NcGTyg1x zaUuP|98M#dTp=x)Xff$fYzju+7SceR8Y*6?D~(|8e`$Pn{c;0DqV8GI^PXLUk93flk}GZ|A`XLOgBs~ zy}o?I;(!3`)rfK_FtK;B)l(SdONhc^@d!wiR1b2wh|FKiG=?RM3j`Ulxqx?kdz+hK+z?na4!2Oi)!>(` z0g$_iw?M18!fuQRaW1MMiAtkSm$1IDwL8BY!ax@-Cbn0NKnW%eJ;di4H;Drr+7h2` zwC5s%S@&*m()i?_Ds$h%%^v@mbMF%E9=J07>*c_Lw*M~1VNQ~_(6_Me6?R4!_kM7n z9)a^{IqjEmR(ZPNb7t^K{E%C*WD4knDqZ&wHzwdyqx9op+Z!P6-s8|Op zab6Hs?PlhDK)zJYt*LIX0-F4LELdkg!Q94ydq?ptjq=>#uSNQJs7+wgNz|kMIc%Yc zADZCye4f?Oa_shC?fiZ4<8hLFvarQg4U;|iQnc3a0;JCk!{OHJUmRV0vFJ5j*}i@_ zs-GSmstX>>wsQTGj?_^bocnU5x2CxDwk|XH3{s6~?<46()_!pp=2QCdsuYs;C$yWv zH)BE0A9o!>Bz3b_d)-MN#M}L=Msn&E$kTIM)yv!KGK{9p9P}(WLNAs~zEgex|( zW&?j3=w0+?3A4q9h$z{xx;f&(_VXbd_#5{Y1%dF2^z<&IjZ9R)-l?{48W+kAv?p(Q z>o=XZ6AtEZ9{3dyO^WW)T~`H8=C2f<8-BbI&(mLu9&DoT`b=QfNCxUSwNLP|nrW@5 z<=#T87CG1~-|gGDdR+h!k8azHg3qY`=l@8{KXOK7oM7c~mGXhD#Pd$r85D9cfRHKM z-b|03`%7y(wIv5=QYG(te6jIKXI|H;VIhdwVRAFLxSt(FRp&=xwcy1mS2?xX=iRc2 zs?{FpbRi9CXX{cmB5ZL;8P|fR{C6JLh|$q1wFLZ}-xpGEqPnBjCu5^&3$fL{5~q3s z^w@Dkx6jN3PR{9fmcKPw2!(+xQgd683=>B$GxA}!z*aLRMMy>PD`z)%x00up7+~99Hf$~x(%|TF zG0~Y8aPb)8n*pxjO5smGJz9IEItF`JtY1RT24uA@inQ!(3`NDgA8oZu4G2i=R1-m$ zQctX7=70?Mj2|L}nQ6yyrF>^^JVT9NIV5eluO$3fDTrXraQ$m221B*E{RfVIkjM3M zU+f6M9Xk7F<|@p4U!JpAw9GvTWTe+uV0e~iNIrbqkb56ztbB2;oJlKbQ;1X_ALreF^H!><%PPQFKy^1ve?eg~VDAev z2$#(uSwq)|+A8TZnH%FNoO>8|y`rAZ%i-b^V!-iP!~Yq~3g5};bS@~=Jl?(>f%rTL z(9@9?GT;wZX2H@P*P64Ds1I28{(zJVIiZuB=t5<Zg?~P+@D+>r>dY1r+S6 zcNpXteWs&H`9$lz!g887YAO;B-5*_~nSK)4JX|dT)m@Ec*O^W$!bh-4wv*>;+pw(C za1xs>e0vqS+B)PRkk!Ro*MxICa>Ri9n#TGKn$)G}hv)%sCUA$H&#`l|{0O~^!3S{9 z_^`@IgO-7;3cEY+c++}h?aq$ns;r@Kh31;QL%jC2_HbT)zIiHE>QYs60V?xo1wzx~ zIMoA~&BWf}{zXj3IC<7KZSb}6|781Q?l;SiCo1EvQq3AfW&~Nk7118sBq#6VB&sbq zYwT>4ckg%*MoiKer$~>?$?9HVcGGwJo@ou&aj^953#zkV=Yo(twvofMx(UOQ=iT+q z2m3U+cEyCu=Z1J;>?wt|S|iZ$j0BFe9&88MC~#`!XJ~ zgf?Qev!{;ip13uh`dr&Or&Qt?JSBPc@zQ4!_*FvPGT8WFx@h&;8nFd=`w-U#H|4@o0*Y6~Cu6Ot2a5j*X zz2{WA?xI6AhD3a*!D-OGfiUeo7>K$GCEdIhql!K4n zH4f{)4-%T83%-3@W{;+n{CU|lnO;7+^z%8PVaYhlX#2WAp8tM2;GP**W;AtPh==Ct zDqa%M@I7EAFI8$^=8-w%co+-g9b}3O#798iAdrp0VvAOgn$pT@mz6sWkkN_ty02;K zbSBIlyw01N;WThRIqm_kepJ;<-geRp*5dRp*?8l>injAc>o0crseN7y<Qb&g`*Q7e9kzW9zEx%s@{;YmX=(2OKFDc>AiKS=khg_( zWi?Lz`2*}yVv*DIoHf0`;!*5C_$f02YC&8fPvlyvo12$Q$t)=r@i0ZNaWK3?!T%a# zCX_jIU({TzS%2;mX<}5=28+OV>S~IIhMHC4shej!fY@$U4(yGFP!>p>RRF@GD}ip**(>jaa%(Q=hFfObTb_A})yirgUs= z;4<~R*k=zTe!-NSXWOeJ5(}G0#W^Ht3H&Z zF1j30sp5?bgT2(>W7Ze?D@k9@%?rP!HxNVNnd_P7(O}{mjSoruI#+3adv zr7RkM4iyQ(sM;z-FF#H2_9s?*{_NCfky!EeQg)r&n1fqYWvgx5iYOePbSI~JEz0763>Ol#PmvH1n2!b4ABIRbZ{#-$KW-4D zWLHa#Cn3}D?XVg%Sos($BKkAUL2yAXvX=q_(m1JjU@_jYPwmsRUNnAHHG1U5wR@FZ z=dFR~XTG23{cF@NP-eb5bDGC` zm|kWIyMNg$dZkgE{?1OMcyZIoT|CVPceQ-Saa9)EuzDAoA-@@c`jo}<4m-{+ZY}1l z-=$@T17n8 zPe;OF5085<03jutJjc(rdf3xq&)Nl?H`uyjF5qhUX~u3IWx$ApIQNeG71BM2D$$ZS zK7VQwFlBLVyoU+#;NEc~f{OVHR66`O%|}yyV>$HL@mWO<7>XS`Sn$jzTH+JK8v;H7 zlB_ak+Jt;F&or@!zhw7+B){%%IhyHgbW9dzI9Vf;s822gSkP)`;=|^?kJDm3U4fs= zziZ77wKa+(m2ra9>17F#(H4uJPy7WtPwYIT=(2E*@(&m^RJnO2Sp}5^TeD*;TGKzf zs&s1y`KBvl2^mEz2Gu1dV{5K_KeIXZYm-`9#T;)lia5T^O9_KR7 zxvu+N-PmMpj|rwxi#ph;BsDVj6XU>uAC0iPqM~eVgR7=l?N{0z&h4ow1FEq%9gBPy z?M4xFz^^D?wt-N$>|4{_SN{T$Kl^Ukq#nmPUtLl31LiT6-mDXhAa7Y1~^LV z&nw0Cp-tTqO3gfVmggsMpC96Bg4e(2Tyi)StcXykgsI3ZR?6f(3DH1PiGHtrie99W z%##p^-quV;F#O=jd2!W%R(M;abbl$T(MtF#u9^QDf2eEFl1Q{eLAc{7C_;v1a2iJ! zIgDt9=i54J1&bJv!I*~EVp8~}Nk$eiTTQ79Gb{6NGw>bOban$-I1eYa9qBXB{6Kt_ z8D_m1Pn70&j8`?xT_*Y-Z65_BU1sDLztn7Yj?%cEeYKm`!fxnVJo>m0ZT;qB7j@y? zNAUCzPf&&pT$D#K7F zV0HaRUW(h}?YofZVT06Wsg!J=`VVEf^%&(>mRBVc5%uIqxTk?($pWK~pAi3MNw(W+ z5yQQZpY+D`xZ+T15Anxp_-ZolaKfxPN0B%fKTDT?cvN4EB>>S|%2j57mk;I{YP7W$vG#(fw%gO1-<-?cTIn-8Lnu zg{je;1*9An6GsF*TMV&BT+mC~x`wc>Mow)jV>i4jX#u@_%h_x~wKCiB(k_;5jcr+d z{vjG;V`>l9U=++4N4FWILyPkpEr@9IAX!4I{WtY7P*Ywti|v4na;H>|IEP`^#q2Yq zOX_+iI>C?E%C8C!EY{#RyD74^a^(NvUs%zo&Z=w%qWvy7cq*?~w~`r4c?Lk{lJt@O z)RI=oLe_RlQ-$VsYvqes%Q}1jw+i#^h>3NoRyAKq3LaK(Q$Nr1*Y@kO8Llt;(zPg; z2ZsF^o4DUS`%zf;G=Tx|PDi%nRbgZPpC~xOwRKR(oKXq=5G0&Dw2w30zuegAxrx-@ zl9hmHCtK0ch|N6`B~;cQdUVct75Bg1X=6>ngM2)b8WcyehHz^hE7y>g78d_3@_Zrs z_$||y@F;fwd)F0r@%@?r!<8gb1s^j@FYr{SaUj{!(CKwUW1}_6ZFzT2RQA6$kOT6&SJL(;S-EZjUYcY#C;mIz)6 zfe!Z;MPFd9Ys1VssiehZ+6B}?DW6vKKHLT4sGFU4)pWswxz@1b<{FrkAdP!|r|HEh zV(WvtxXVH=B|2=ZJc7EYlErc@`hg(O+4;#XE*5os+D2o1LeO~k%_~;cmR7=MJm);j z-M!KBJ|8P9W;iahu0OO5ct1GPuj3OwAFu*_@%E^!3)*2ff_|Bvpp#Oy0r+pWpo|%J zbUpn;WA!IF6F}3a9dsYP)>Tu>!{HJdzReSl*cX9EwKTDEv!qty)~xDNaL)ergKnZ? z)040ush90qfQD#TKM$x3cnCoVt?13OvJ)uW<%IsF4che0}-8X#UwfV^OMvlKNTC>p#LQvlDlte-{5Yv0gqCGZ_ zOJRj=VQQ)&-*G6|@*kS|>;^Kgj}tn>^mxh%Z$YX=%%Xp+XaJH^EywPs8(nB!1-u*PcU05O~3tRjt zPG#fgVD>nvEKZ9uaeqzjw+S@E z!Z{M`T5tLB{PgHNfUj-htW6_+zc?|~AR!VfeVb2q{_L{+esNBRG1b^LC%TLb!yxmz zHNxP5H-^wyz_qTI#`v5QV>DKHl`EU8l&c?T9Zx64zHd7axmMb?K9*o#IYy{YmRuB! z<}^L?ZN<_Ho%wa_&=OdC*wO#WY%kB_gms$+3>zOQ8fmmrO`3LE5JziZENvsw%NXy- zD&l5uj3t>yZF%L-eAMJKK`tn=ug>Tz?r&b+cO1N(e89ufm#)44DV#g!n+BG63WZOM z#$l!3%w%NWJuZ3Qt7Ue`@4PYRVcq05<;X;&SexuMbzXyj$1C{tvg3$jz`Z^Z@6RNa zPXZD(!x9)5{R0ag3t<({4=i9p{i8hC`NujD88E{f;^ZzTHlev4eYzJo_&GlYkk zZRv{BRiOhM+2D{Osb{mOJT65cobL--dSi+ED2Y&wW;4G19ic<5gU}3vJ?q%il=oNn zj_W#K3Y<+3jwj!05E@%9<2;l{HFW)70Qtj2GGLas(fiOAUu*FETKm!D4OusZXHhyP zqaobyVve=W>Xba!WAjxu+$!6vD7x2z9E<20vZ&n<$IbjV9U79wSDaZq{n{UI#)G+3{9-$b{bYA~2}a*17GcUHbkHFp3fH zBjlH*dKWXC;M3pX)kafC_!*qa<`|aPVPpxXM@}DSlD^Ohcv=V)?xkU|g3?XVMz$Me z*hXEtt3;lcs9M`{f>E0rc)rJedeHFxs0~`2MGMlYVpEmXrzkUx-#qxH9;WIjdBq?w z>>aj5;ta+vZt3X0V@~~Mc6jGmWC#$4kn<;-e>ia=IRrXzEek zcva49_;!5$Z~l0;_Cj|-2e9 zxnV-URj?3{q8iu|IgW;cn0l>^GkVVh>@lZXl=$&NF{Lgo_=7dPj%ii}i6xG%t@(ju z#$GjpIFo&jGWeYgf4*Dbx43Ug)fov4QL!U&34;gIt6dpb=1pY11HEXZaF;nr+i*QzJow zK5BUD4i#$9b;w=XC3TO#01?L*TurgNx9h%5Ha_lem6CY1^I=6R^m;4`2|_rv@tc1y61?%}yaS9^hbKjQ(5@N=KyRR>p&l~#~GmlvHr5qnF|0oQm42FJI8 zv8%aJ@~vFoxzFh$`*wWlmD#G=iDoj()CX?fv}iAGMkXyqn%@ts5lz*utk}1$dw3wb z68x5*1`^8Q^XRWdvsZ5ch(UIuQH$qrD;V%|Bb&CC#hlPD*F>O&k_UNRNud{4F!FDx z#`Ws^qU{fo%ldx^#>e(w(`=p03CbJ}Z$dnlxQ49ibjWZvCkY$HR85 zes$w_qLs0KL%l@5Ou1YpIF#l*k^sINEosoT*Z$devJLCFo0CWvDp0_;Zh-&zETO^^_a936 zAB6DIMA-4X!zt*CK!%yFlSWb$k(hTy2@`JWuPDpyyNUZ!t#h;){EckAEC zWwzpoRYlF`kyW(-ea%|#_O$$Pp*R9@t67|KevR_anK9k7>*1K-NuIC!HWyx=`>19F zg;QhR2$z`Qh{U4_=f-@M5$u98b+i-A8oNTJ^Ld0 z#-&b+P$gy*Vf`q=byVmTUrWlYbRjPhyE}g?w@v6M!5d6XyfkBuZu*%DKN9z%A{xUQ z9hJL)PpSp}CWL{#gD`N}I|~-E%-iS(gGXUhe8~#IL}qI}lKPJ{tzqQnWrjKsq>N5IjSLX-MrWhQ1mJ-37!@=ef zb-)GjTDuWvzso((=>xcB)c!Dw>`NfJpytY!R3|!0%F}MIIZHF}Q^`9L(P3D=CXmxi zZ~sD>$|3qTV~h4BoBaO~5+Fug!-+yAzG1@1d$LIxfly&b4XUs}=j8}K|M)9K>aV4~ zmXMH5%Vuj%Z1YSSqfvG~(8j4hHp&Ari=PzK@zG?M*pc_JKW!wY*>l~6Xiz`)eQa}! z`g+zy5u{+4V3=f?e}v-6th3{6h1>b{vcNrY%Tz@^!LrzJ7mVjQ!_$E{c)n%+4OFsB zMP<5OVzcFXU7@&T50ygb`$JNn1hNIqFVpPO3fD`ed;MdkFxe(RUE{YM>>?9i5*{zC z43?rc1N+r8jemO4^~sa7$I(tHZtidsYX{!L4-Tv5E&`#Iab~LG8c#cpr-}R?cJamj z=QLgx)Aby=-AdHb9Z*h(?kxB1ux)i;rt-2`&OtLVh4mVDG%b`AAzAlVINx>=qUH%Z zs{7Q|dPJm!Bd2xQY=r!s#L><6bA|Tlk&|&M6A+?kn7%HWJksSaqpXD9$!2$xTbX$i zIH`qqL*LM3W@E}q?~~Obyju8_B1lBGyL!k^CeBtoY0WoK$m)6r!W85TrBCr%?4F1wOwKaYlbht*g8|7~;nFQJYl^Fh$ zEWF$*^yNWLHT?qgvy9e1T>kD_*`25B|1{bG=nN`jJ(jQ#*P8161%!N+@=F=ij$S8a zqY5UO72GpsO^*v=_!Oi&tucG5yZI(>vkY=tV(fE|`{Yt6#-jmQdCkkXa(|3grx8_GZTlcJaD!-c{z+lZQ~l{IfvVfN%2&XsBu6f=p#?-LqUwT5 zSYQyH=}T86F>;6l{nV(l{FzMAEDOhE*RDxuVK*C+eK&nLp-Ev7U@h^WxI>)wq8QlR z=B*8T5eNzV%rYVs*A`oLDZ?wsAkHL}>bm@ZK)GM`h}I}v%DEo8K4&1JtO~HbL3LT= zAGPWzcIwNrq{18gzQ<#R)+qIjXqQ5Nvlj0n8^BC3rSO7A{jYfW|MEr)s@-TDv3N^w zCkayfE1}$ncVNo>ezYG-n8BN7-!0wx&t0|;zZl56Im8Xt5*xoI;@acHrXZQp&Eg>E ziRl+`KUjZ_uFuDrZY7j$oo!$r+PsNl*E?B@uU(itla3lJloFrA_hl>m+hx@`ZETNa-3zf7Bgq#@<0%-Q<3>v$_jFU?SUv*>8npcp4{tj(WQ*v7^K@(u zP}t(QUGX=$t6W94d_o;2Gy2~0EqlEm7oahEpVch#99_;Bwa_JMNx$Twd@R?xnbDo1 zSbc}iMhzl@=o=@HRd-Y0-y9`9`goLSjFHw?%p;STgJfgl;M zQ7_9;dXE2HcKoksc{l&X_sr}RmBu%kn`7Z$1l9DfRx>-K0f{VSib~hJo^kcJ@_fI~ z1;D@iYip?xb-e|jzJ|gN-m{RyDF&rZl5pr^r5DN;3z;W40L6H(n8#bFWwmonsi6@( zDrd`j`fA$6LIb0vGP@F9(#crmWN%8(RhvETUKH~ZZmwrL7nY1mkH}e;PJ=<8w5QZ4 zVY91aO1tv2UB~)(;Sb89s*N} zyfQw~n^QV!$v}B|XK`4ng&#b-l6ibp9sZ#}rb*JdF87arKv~o86tuC=k9>v?>NTzV zqf>$NsVc1e{%kM1fdn3JeYzeln|-^k2F_vdZ3ys!BmkHS+yx&-?e6M#b>F_ACYPHC zfIsZ?9PVqyv4AJKbM;ZBWtxSAd*`DWIm~^84$6om%sx#tLHlmxGu8hcG`APCUwkV} zU}%!DjFGF3?gC-VZh_9KOAyjSs zG#oYuc{WzF`;(sc9ISm8=L>_2v8{Tw!Tj}q{ieSiy%h|P+`Ij2F9!r%lT|Tk$bofC zIkPOEw3G@6%rZ-=PQG>)lupG7Qm|OY4Hq$oC3HW;hBA`@gG1y#r>341Py_YE-~V^k zJ`TcGYKz+&{)njeaoN=UWGw-MO2+p@jZnBrkI`wv(|*F@c>g)fRb%N*#)&Hmr&A%0 zu_x&0O`Eu<+6`G9s{S6mJRvq>mJr5N{A+{FHB#ma39u=XXJ=61`Hrp~$ z1<27%xR6Hhbo!;IGh;lVDR!2c)h@sfz26DYtSa~U4}9*AWo#gFr7+d zBF`Y2VDc8-Rc(4QiTTA&($J9q2T?t~4KdEN;Z>1hld!X_(=tYv!43L zmZj)eo?V_`7eatNY{=s{R3w-#yd6L3(yUMiiau;mB zn8z5ZuAj1aKAvfIoypL6XE?Wt=r{VJ>W^erVJ%OaSEVa@0)-m+`+MLlHclVHs#%Ka ziSy-0?s4EGXIi$A!jrqeHIaf-6OBy>yq+ak!jI@PE}RtN;2dJ}k+!P4#k2LyWm}(2 zhCg<~waATN^LFu;cQHC;<7{;^behDQiv7RmB)Br%|NA+>OK05NOoYI{%6AG~itWJw zQ80@aFiIbDA{np`zMB2-9np8D2rZB45fH1q#x)q_yS*gc|C2HB zW&&eqvDCkkNICg_(>y?xmDSUW4^rwPb{!%ms(S}rp?rTnV=@_MD0c3uzI&cuSDJ1V zO&+dCRM~5BUsBFrkxSGMd>2U3PqQW}>^(YtXlnB93`dfYaVyEc)0^b{n1ajj&7>ouTrnoZB=x(bN=!W+htT z=HB}(&h0E|JsWbMyB)1PD;f0|KB%D8JBgHtPetLU%vZFkX@q3)YwfpcOP7L4rf86c=?SAO+ zOEO#Meyj+8MA3G6J$QTO8~)i>umjcM*9@Vs$*S3Jq1U#|V+&2`j>oyKw^f%5O+DpH zn1CkXDkGY)9~B%f%j~U@(^}3#i?4nm(EeWNkbou6S=)^Z~!TL{p4cwud$h&c>i=4C2(XL1VIr6Q=TRyot))(LC zW)BhQ8CIPV;jT}UiNKu4TcQO=gG{H#j^eseZXwjg?xVl2cB~r2FI6G^(!6ZYf%L(i z%N)2aaR2wMR&_5&3?S6bDqKF&;JuWkG?K3ly%Lp>%~!Jl5!MZ z1*2FuOCE2A@`4B#3bRPK`49g1Kj{RC_Z?0{-+8bgGm~N#_wM=zjT0e4B0Wq`Ob#Y& zLzuGzMs-f-)-(wU&DAyu!S|#l zjrCeaz5EXcQ=<`u z=c5lC<2Q+-YL2GMF&7gZ3JnCIpfOf;L*00)ge45@%s46s66KHNKZs7>6YVOMaRHp9tq<^$*#G=w5`aiy(pi! zuRQ^A`QJ7(X(HaDP%BTqfxVO||1PB$NPHs+Ac?)~^-`jPXNI(;yI)2ES_ zA0#*nlrSa^TX1MztVgFiV5>)R~y%q17rVE14z6=ZPOns6P~+eM*?{N5hI6hHWS zS)PN@zQV+F6D5YeA3!stor_wmgElU~JkT`mPFkV@MwyVZ9q>{U;f9RM1lQ<=a=F-& zuYNE{?lJM^c$|DM;S1MLS-ZsrbRGVr+S2}x^9_x7BD>BW(6)qiYtBl6NRC)HeIWeI z8WVPhxjER}8~B~+;9TD7xgD*rp~kQblC7jQPKn7$KAZd3SsgsYF8(6Yw=Wvk`O>AG zDwxSn{xgA-=I68%qk&G&DOIH%Vh#<5RI;*E&bz}jciU;t>>Z| zZGMWr8x;%0vh)Z}(mX~TatBjLmPG=LXtYp5g9IXX@j#y89(9AIscQU#0ckR8=?*q@ zQ5$;x=-6ONz$z4}SaoXC=hb)xi?wb4tHc)RR=m`!^!!B()PK4V+W#u@`TyDj3msLxpxusL2~kX#NCRbN#1|{=c3MiQD|Ctn5BCrgl~pvBph#cj zkz_y}FYW&Ts{dsa!h#nDs5+m()LRFRcjb&W@tyXC@bO+LuChSLE{Prj1GQ9wvS}4iAP;PtF|jZ{=f5r;kwGZ3c`KoHw?1PPM?T@cxfOxw5&7c^aC46N|7%@BZDzFb}BnJ3s39W-(@izIvV&$|Gk~LV+q7b&<`5Q@vg$ zDS$(Q8>u9>DBc&G!%M!my!6UOCzEfeA^KesEvqdwlwdaGbs>!2Wa0ZTgp7FUeUtGX#s{%J5c?3`?twoxk9 z6u-h{DIox2319RHH3;VEU5b~0*zgs|eL7`>_FHysFxc5AZF)4z0j3YtXF3!yfaF-2 zh4_jle}i64e<5IV-e(}8I+jvNjXgM@61!w!?1Q+V+Gd_ZSXNDpm&Nx(q3_SVZ*v63 z>TGD`75@*i`GS3$^O&G8ssti7LZtGUaQbZczOOBGYRh1el%W(^ufG|d-s$zHM*dul zH*&l>`mI(D!~=QDZ}XmO>yP6$*1=c(5gQxX)UB2B{9;57RL3urKw>L1b|@-^%i1*% z#GO?_Bh*89+J6u{RmWxL20PBQKlnb)Jz0YhRPHqckdd<@ft2_UV~W#>9|kGwSW!(d zfrPUyMEE01MNB%8aIl5#47GD?h;nGQw=@Lb@p{QE7q&h$+^GRKq~AS!h128~rtK;U z`w>sbscsX8M1P1J0B5$}*!Y@zagzps8Xjx0>$-Z$8G`p&f74Nz+Q`RaF*>9S79ra> z*e60TDj34UzThv(Sc@N!^d9}oF5-rNn;YN2D3{+SX)3B8ZAR9{UT&!8&jwZKvm+s; z)P?d$Q%X>1gNdMxCb$poAlxXKjn_ktZW8MAIVgzf0mXUa74jtLl-%?t1|k-Bfa*mi zQ@;q>zS~6_ex6%+3sY60p{pp4@>oQP;F6EgET>iGtW zI=^`3Btv!FWh7C>Fmy6Lgmb?0(M&p%Ff2;Ow^*06TZsRn-dXK9o;*(wYyg ziw}M|n#IS+A=(ZWi5Yy`e;aXFtnb=jljXA%G8>g`A6EEyCdMezAscyfioe+XzT7Kn zd+%TU_CiV2#us4?+nt3N`((e&Ap2Cm#ie?>>oX3KV5*-@4IJWtjF2U{{|6H5`7@7% zCe9Z}-5`M5w;P61rB@MHD&YFv{l{NkFG;c{yA-AzjMI_Bb?!{QO|yRM$f0PX-!I<5 zTd9SHd=_U7uJ57*sH7@7iu8%=U`N}3Wgp{0C8wHOX2CoZq*X4i`0jt`vZS@brJD1T zt({^4YKp`wRq~5R*?8_`X~flmvE*Ojv7fuzhuKK2+Vp-`)gxDMF@Gy@bowhe>-i>L zHUF&`u=t@Y)bq{iLU-eFNk^kqZjvdplf7UQe~_$gKsw*)V-M60z$LX5M*$HodkE(t zt4bhqJe4pOmb&_LjD-B}SCPt#piV9vsd8^xBK3#ONsOv2cwph9Z@Y zU%pc8>k0ksy?c+Gun)*Jz4Z0lP_8wd#A*+|^kEZ!9k~UJARZ>hi2o!mxOHlM8kc>4 zeaN@>KDp&KNW}w+YV#7#3q^>CW{P7fJC`-JDq}$r2x-bUG9=CJut%@hI}obY|DftC zgW3wat&w8ION$k!NO5I=PWIurV|fBxXyhNyXief}os&q^jrZ;l&2&(d!2Z{dHl&|k6e8ZM|+^8Ju3WV|I%uklO9_u94d zM%w&?nCfooD`D4vjwcdFH67m(Q4@tE_IKX(?!DDlpPmLumhP?jB^|(gAGVNZ>heZD zRwv)3XIT7tr$GE3FZ|zo8O|Pq|q))(9PQz55N5yFM0a+gB zpCcKnc}j+b!PHf`q4Vq1#roq$>lG*W)8!vXr)RmUtFyC)vvc-kmp9~TA}9J+;t{gWr4c0n`9q+{3oXV&h<90>RPb)Y4J$^X`di# zKqv6yyV+y(UEf3=k>O0}_xMhRNMJO-Ej`e=0f}jVWxE4{FX~cPTQtg?Z${{E%VI7H zzJtGw$CCRy|0XiKMM?e4Pkj~|5!jNow!%CvjCK&5Xl=S{2ciGbh;siE6-P6L^n_l4 z3%_qcuT_49sDoBm+TcyH2p>_2ibNO&k-`FhRXkKR|mQK-Q9ryRGQ!&LXSUb;O_tmV4?a^E?)sWMmVV+ z`A^1-aD*jRo2p*I+9iP2c2_dDBmkEFj4*ccUI@N@`~JCNpl?6aVGEbZHG zsS4u~g7e;^6-*+Qhiq~X2s|0(g-GZE=8WP$Y)Tbw9-X^fSfm z$}nXGO191vJa^Ls+!CbyCKA%|9Mv?n@N9;th9aDR&G0w8^QA z8gjM7i{~@M^yW4(kAY^i;>gbX6EQpEEdS-{o-k2Kylfet33)-PeZ^4kmDF~=_z*{u z_(ULichi|>IK;!Vz-Q8SuH}-=`Tng{if0SY1(^PYyr4Icpu?Zn!RR<|EKudW{a{TN zk^S&5^gj_Y_2xPOU}B~38!2vi2>j$4zNf$KcM)pY`_sa$`FG1E z325)*w|3o_AaQ2=>)X<-YK1RLR+R9yDA!Dv>>6datm*uB^}7Rgt8B+SES0r&(*azb zYwQJGGO1TNQs9Y_k%tZ}*09NvP=~gk8`JcE&{5YcA%M5H6BbBUup8+A)7Ca>qD-Gv z$>Vr5P>nSyAIGCnwZ+MD;D#41xK@TkpnsorlnNVDmbj)Bi%j{Q@e1${e+hRg54#!n zgS*-1OnqSwy&$MJC(G2_b$M&PKd}emDx(CJ37A#1K$C(|NR2*z&U+9n8Afi@#(0F9 zRZ}~5e)o~NbEk^)6S2P3&f9&#Z_#p9BFdvfZLnM~fhS7X8|0}9?Mk1eyog3SS6bH| z3E`8}V`Ra^aiqZWgzOpMp;P(Xuyw5);-&?!>u6e)th>?R?^Isrx=4E~K5`&Z(2?5_ z|1}17QjP#d?9s;HKZ1?;{trg3Xwn<+cmnubNisGf9kmIvMxuq@u6ZDeTHYcBW1<*m z&|=vpmdMoXlsT|a02Bx*r>68g+I zZ*#s$tS3n{bIgSIAyo+}JUC43=qgNJZ)9t>-2>IOLd^)A(BGTVl>>cun;OByeB~UB zM2|D!;P@#O_R~nPct&ZZyUnakSp;eW)NzSv5C>E?uysncdB((bkyJM~MGHxXoiSA@ z&BPKuSxVVWpISPYHKd|&{cjL(+8|Zty!E3WX|y0EFo%TZ@eDV zLzF~NpOWS!cXkpbsB&&NNJtr^O--k8eIzX6L;v*HdVPN)>AXee(X&F=z~eu0j764^ zVqP)*9tr&xwCg7u7wAGF%Vd{KfuwdzgM$x*%r6D$QT3T5*ND1AtPZ>QGvb8BuvH-# z^HYhr%jn$LX{Hm{oS));H)a?{K9v&y`Ix_$r}9;eK`RVg(|WHgKT0f+REG!3wZ0j{ zarX~JbdC`k4j^%HZsOq|wZlGJbPSmkdqGBy-hk?S>uJF4=(U_+&cLLaG9*RHz^6x> zdy!iu#R??53|;}=WOgoD&w^r8<{`|uQZ0f2iT*_X+|fLqprfLkJFrBk zgeB2Ua`87BytW@wol*pn2m@#eL7!;fkdGq`49^P5Q&3}BVLgg`Wf3#V$e#oilWScC zm53J5&1hon{Y5Q83~4d9+<3uDbvqa?XanT&RCUSxPKHT(czR4Uf1~#jO+YcD%JM%< zQyyDC0C5!1dZX9N)X$KTWc`14@gHy`bfR!Pg^QEP187hzW3EpIe2TIH=hbUv`0Q58 z2p|@wc$gM;ZWUkA?D$S@IR-Yr&deU{@-;k&ykD*u!kxKBp!v+MjuAQ3&jaDW5iq6U z0fW*wW&^oC84_?gmbCjZkwPqpmN^(@PB;*zC6T_uK{ap9zZS698T0Hy zxZJ3c5Q&(=i1d0h<3+Hst0@xzt}Q~`3+xCwkVY4iy@DUM^<1AW-#Yq%K_iIX|3)2a zMr;5viU#e1EAk7)7Y}ri7&Mv=oTBZwBZ7kxwqteY7d6LJgBfQ$v~7LC-k)Nm$+B`XP%h8C2 zZ~Fvx7-Kl9ZKl~1@ZB;fm)$AZ1BceKNB7(jW5B_Ho;y=?+qmnTDnIm8n4oY=o1cmn zgHdL%z`T+Dj``zz&*cWycnmqOFJ-zGiu*-qg?5Kg4<^MgI~ZryOU4(2qxW-NZ~!Em zH?wH|hb;Vm)4(Ci+B1qoJurb?P=j0~v}(1qvBAO$DOu@&OUM0~WwYk9OX?Pd6o5bv z#~}kLPW~A5nLbFemVv_i)BbhLm%60?N#h9c1?6eI#+4F`zRc;nCBmB=GC2p-$pIVs72nHtMTR`+jkp1u9TrHl~ z$qheknz*;&?<#Ck=d}Nt824J|2=(u-VRa+3WxP4w>xwdo?wO_<;d#oB*$BQtI34FE zJ&f}D3^K}?m0+8pzx}4lylCo_62E*x<%#bxf(ZFSQCQy+v~|oYgnsfw4-#Wn;$Q8l z4HIL_uKOl7goz*$F+)q?4=h5nBmg(C?2Km5qdA*VDt+%j?@X*r2p3Vb0AO@Ze>}nN zvTn@ox(c@daXeD5RzAp(EdQ_;Lly6jR`puhqVjD;Tf_B0u;@|xiMKKSb0gKP$2LxR zn*)7K>>?c%@>~c6Gdp6MN5EzuWk$*gn}wVOs`_!~p%;B)$ObH5g(SRKxpjE@YqdWnD0jdF+yMJfAFfs58Xe?AC3uw4CbnjmvG9i_Tz=SaRrcxmJp;f zfa?LPd8CP4w=^Nl{let{>eGXrgMsQN#Q2#TIH$D`b|4oaW2=bkE!qw}9+O9qHyUzm} zx@2#k^z>)S0qAb>&XsesgWt3GCnVPZuuG7D+H{BC6R{_*zXc?}%Hcsqe;!xsYWghq z^-&boAm%E-==j?p;pRTDi7~1vBv{AG(tf-gIwo3^&5`Zj-6#JGf9Q+ z5oS(NH7Wwd(r?LQ1xh3O&t(t>ZQ~377n__wGP6WfCgcDN9;~SR>U>AcVAVtQHr|cJ zc*ll2<9=>^y&M^6l*(*vs^VH~bCS;bC<04R&KCLhkR~jrlVDeXcu0e1>}M~4`a+6O z0X@LBT+T{@W+J3XA(CltxzczVoR6@jB+ncK zt4O(ju(+x9X&BU6?(QsC%35re+*77oxc;JD8UtxkGLg>zg+}Xz*{=U&NnfG)gW=Eh zJ#L!=dTcTmfs`HF*M9oT453Fx9ztF`QqohiW|&4@o4o{|t7@FEQ!k*pwS9CWb^d-a z{q(~r6G;CKB})GM2mtvh**0bhXrNn8)5Ny8Dd4J5NCDY~q`3;L zm-KvqPMBil-EVK{ch;sL?c(7G%`vm|$wL0R_1*V`>FXcgfJ(2vl=W z&HHZU*W1$9F$e&Xas#Y8v=>hTj9162U{l7ng%IkBn0R8m^g#%mlQ>uhq}Pb22`7ys zOqc(^xA?DEQ(M(a|Q1JE)QV2oiHO2d84NGMU}TfLb_+VjJrzBFGL>iL&R zmVo$eKv0*e@|b9H0`mX`&6M*JV&=e@m>lXsFm(9gdftnyBA^gxOVj%9E5pL-oaEL1 zpD2!>AWCB51n0JTo53~bI=gwo_)5DLee>nY#H2QA?*&({+Z1BLsg`v7p`h*AlB0Z5 zIYVAq$Ol7ip9-m^k6!+^6Q=iVl9WyLNlApBdIW$c9#G@x>4U1auz*~aR^1N+z0RKx zE4ED&DI%kt;6HB^=g~0Y=(aUQ2#$X<(cv}W7T~+u4&Ij8VBk7>GVVXUf8rCcth2~I zP`Yim8SG}Y??Ff~{VAgPPpT7xwcWm0_KN{UQa)@Gq8>%Z{4>=TtJF@VPUC2yI&P^* z$ja81H#@*S*Si(Aem`xaRD!ul$FyTBZ>Ut&E>P0U1=LFeyy*r z@!j5qiu|_Bif2kvj{uE@@OJ)I7sadrMc51h78qE9JdCY2j(Ggtv4SBfIM`*(PhgLQ zJ4v(EUs1dYAqh83LSI|e$fF8tX84er4JTo6Fquhwm;hAxGz=L80U%RL*)9dxduKCU z)T$7zEiKA9{G_~?`7`>-_@dxp#pAth#h0@60?(z76>1W`e$V#AZ_l>wF+owWd_!>F zu|l)z{J&|(*9XEdG`U(SvJInLEKKv2Tt5sVdQv~uIR;16B>CrG@ir7$RqY=KaNN+c z0d2%{fA7kihLy2V%^?qRIO%b!8`!(fv19{LpU8QPh*`Ti*ssa;VGp+eW=y^lwJFrBHA3W>+o zeg8vM&ywk|x>nWXFfAVY6tOjofCk5<2hBw?y&o<96odD-u)E?CNGNA8Ucu-xl z{)f40rVZvI!BC@oS;lt>Z^G%)viuXipiJ6 zHkKvvM@8l&%WE`u#~1YTMas2asua(Boa)t;2q*BQ7Ypuf5Coqga(chI*~)Lc(nYmHGWcs*`Q`q{n@#sXPh~S{!Pm zq|Qb^z5CX3NIn{pc96S-z*I1!<1Z6&mBy1o3tVsgx%7l5FWFBBzFyg|bDHX$d{kZO zyRwbibN{Y_0f7G75%E|PZ?vXf2}Z1~I} zc%gh{02$L%K2X1<$6ug~xo}#;iImV(h+TMI9&Rzskf#O%^woS>zCPaL%3HWLclD-(y5cHzHnjp0 zKbooQ7eg=&_PC}fyoYjMpL#-{PT1Ge>J#em>4AD$wQl!=4w#nWM#3iB_(R|LU5VS) z>Y8=wZCm8*Z9FK~?^MFaP$;Y--kxgv>;Ga8akJp~_fOyeQM$5_VwnSuLl>R{csZ3i8uUpF>aqC{i(p%g5`WbP^OO%alF1reMTYGi9TWWA^AL=prf z$nLQeS6>3?;gWfO3R;j<_Di%u#A(m7Q zR@FgF;}S8=uBk5z0=9d6r3~|5t?eu|v3o#jVH)5#%-a^pFo9 z+T8eRR2j^fodInM;*)dbxcm2y3uo&#XJ4fzgxa?XS^TnotgnA&jm^6@ zUKX>k%&)Z`F#St1%@*70iKAzFW=hHz3ow^RYx*i_i5W1n*o@7hw>uQbr`dg|P)e~N z0|Uz`nAyg!z7cXH(5?&1oqYRZQF~57w_Djb3iPhro2s0&2i>&*_6uEm z;yB~OMcR#@D(ONTCgb*=SJ0wS%g7OcZ7#J+@#87kUY0FFWV^vLl5CFn_NUb%=Kvw- zqJ34;`c)VIuAmRp&HcjKkhx^q{qAVhWcJA){ZVLvR;>JIWu1rDX=(96oA}l*N{*h( z`gt|cRWZl=8_xx#hokvdmjsye{NsGHm1<&Aba7$xl@>`ES@MQOAB_4NlVnHczcZ&# zx;#qFN`&eE2V}J<0;N<9{@NU^e!s)|e{K9B|ABmLN36WZDcIjfg8KA?LM|Y-6K=@C z8A(p+usrnXdF_zJZxj!8Q%Y<;!M-p4!lOI1DINQhvDcJrNdBnqf};`CUg!YO zv;(hR37Rw3eh$%jsKCmbnbrC}!ZEy1zF?Q-{Ft7+%=?E2h5a4BA^W+9#;x05MDJXr zew~us6_ZCKQm!}smrta$1#@-__)cuAmGZ8s94)B)u;^4g>H&9g>%a<6_CZ=*TCo4% z`XDAV!|i=@-5M)TyG*UnJ$8fWIfr#kQ-}=F!@OnaH3Q@aBj$lTW`Tmz$8ta{Np^$Z zGcht@&5Ytqv`({F?yQH)P6sUud|X63b}+qoraPm0)@zq0esy6jck8)9#o+E)eAL(b zO76CX&5e2@m0M2ix)*m8Fig%MNA8?((a{)$@0I(q?Yyk-Ft5Ttny6{;Ir;TL=lR#{ z;XLJYO@|VZ!45%D^;q_aihc))jikbjsZn@~8h@vM_nQ|LZ0pDoHB^ZYkI+$}&#Lpc zUaHuaC*&x1q(7Rf#A1gPBLhWUn)J`&43PHoP35*1X6}rumUB0u^2YE_9bhaLE z5D&cjKWdLnugwu8IRf9aqPY?NZ{h;ZW*zc z@cP5W=S!2nb8IV;9i5Osx%~~=f|r}}LXZ9YIpO=SXGDJt>96Mg5fzURPX1k|R~r4F z+D$6T*bEYFs$NU4v0YAx0#U$=R5d2GAF)kvghfxSR!B!sPd}n|SZI;72wGSegQ~V% zI>oK(K2h=TGQb_+QF^8YQwPC+!Dfqw4+o)ppeG?Vc4do94jdmp0k$B-+rUxbx&BdM zhMNgo;$))nkMO3LVIiDVe00uKUT=Yjg$C2SGmC&4BFkUQO`L4X`zlRg{l=u!mi`t- z%Z&(YZ`Mf(<*tp^r97o|X}^ew?*@=m^gk$Bc9nD>j*GsZ4{k%uBgb~JO&Eglstdd4 z{#a~mbN+B@STll@-`YsLG7O>DFH<-_)=ajMRCWJVixPA0eY$aXaig^I3Yu3-#`Qjt zb^y)FF@R;Qt)V-9gM407;;&wx6_ob6&bHlSL4gwP(eWj*Br^M z-}>^r?f}Hhzd=ap{)AOKcPW~Ts#o&!*#O1>H%rPPYJFCVZ5$fDTD{;pk2yZ%$-b+r3w-z0Va*>kgWDqMd$22B%4g*X7;u7>Pv& zb-nrIm<7+@^3{5aanJ2%gJ&lfmoW`ZeY)*e*yCaIvH@HYUF7Bscr96sxqUMAvqqxb zH_VtnGy}JN8MKStK1VcGzI#GeE0BbS81Wg_I*1QShn2}rJ~lj)V#%5`YwH%E==69ld2ktd}H}uf_8y0;^)<1H9~9S8Ybr%5H&lMk{#P&Oq`4+7KzLX zxc!_+aTf#J|1XTGvO9Nn9u9&{+!YIlZ{d-GhZlJ=QSn`xEwhcGrG*xYG#R1B3ei5P z02Pv{aoBjop_veH98t=AOa;9)962Pn%A|Bt>a>Wx&NHmglOWeN#KJBZ>}SF`Xj$>S z2|c3!5Y;;#BxndN(<^H4`SpzJa~*iQI_>k5kVOlxe)aNfx%1LFU^=-ALG_0Gj81LKR11on}y z;ulKZ=R7qjS?mPXQt53&D^w-Tw;YIt`V3%bBf}VC?BNKu1qoHzOl4r~W0CGOx}k^LHcMyL>F})MzqJpotgHB+fcuxSwqIZk zC*1-u9_oIuF+riR3C5BC)9x26OxlzdT6l`4>Si}jyY#k+UzG3c;g&g3oUMAn`LoY@ zIz@7KCsgqecf9|A3ZoPp%*@acjvyz8s*Dt#!E3i zAX>p4KjeT6VYI=^Vtem1U5AHNi*Y$|$T#qO_A5(h*X}3qjqB_8EeB}rXbcHU<}ET! zf?k2KEH38l8B#nejRU@NbFvodvE;XfBsVwZoNmQ8Ozr|%V6%8(s4)X z{s%j(S4)Q&13m#{3GDB%a{KI4I5+PWO>fr#l~XcQOj3s`7KTE137&FbK*}55lT^YZ z){yulj-+C(u3AFMF;R8>K9LzTa39}L@b3YCzAZCUk6Q_!JwNpIbUWpM>Wf27Cf~0M zVKeGbII;+hJ;8}%F}D%~QkOJ=xHT~-(Gnvf?2WJ}?Nl$x;je0^I5+-xK402AyscDZ zY~d+nm8C%vTiP6MEKDwNgoHeGFiq1)fpVkBt4Umz{zHE@OkFG=lMLPP6|#IdpAB40iD zX{&?{BK_Bq{NF#m;6R(TUh}=a%8lZ1=A*S*ST;R~B;*cqJ?1n6k+|;3pw7KtvGmJfNzxuQurCA@; z@MnGMei@W;*s&%Art@X8Iy`)v&55iwT&qig@e;W3>z{&bCthF*hz+!hN^X$tt=+SrJ%svG7xOxu3V$R?jxe>AlqVj;Csr!tc?8bz{+ca#o|t~2w3;H`zD?@ zLIr5Jny=RwdejKJHY16W={gRnUcnxO%c9%K{UQx?4BR`CVjCpAXU6oBLhb<}0`9|L zStZKjm};dPX1E*pcyVtDj!EiD5Gj00R8-rKY3oOW4=8?8&foq?c89N?WqU|sZ52yS zIfg@wx)FLUnGU=8bzXTEt0J3fCO!wGS5(Q8TVodqa=5=NWK+p{r_43Qm~s%%74W~>~XGL zcbN`1gG4c>GT5ps)#Xi>jI06doj^*lvrRu20imA%40-|u&6wM6NS@VDJsybZAzED_c zwJ2bw&8s8eqlSVeXB35U9Bg(1bcSv7fC~VW-*?N|GSzunL%?EydCVRJWg*nRIm!BLiEY8Oer=Yo?sgKg9{_61x*i;6BvkP-p>OMWfSAL2LL!{gfJ5m08@<_7FBc2crjepK6G)Y?WSmjsS3`_%Z5>3vYJH~3Xo#~ zBe_uz&50f0(gHAKLjqsFIxYC)S_B>0TO91cfd~wX|eh=^Lfc`Pi_orOu z`SD}<)OGBoN4dgKS?c91gDiPxaa)|Ky94Ta+h}iSDeU(rdPPf@BhUvu3gR+QUfrb= zE&8AZ2Ag#!u5uTT3uX(fuf0DvV_nRNYp;I)qH&`q&BJu;DOq(PQ7JKr8@99795{7^ zm*V`_w7(M_ys%^386!E&KAbJGgV`f2-0t+a)IGvIEy|w#ym35q5Rf`Ac@Rb$UDM`u zrn}7qU%b<_ar?YI5&&*jQfbsdz_h3#J8!^#871bpL^LbyidNP5X~=?By|yA@FA(rY-x`RVEYl`rmD?|Qz&X7 z4VPCx`G)Ak`-h{8R1(I94;As9Ihff9ju{x^L&pl33cABvKDYprwEBRR_H6+ApQxTF_6ju*-$C1N+xdRW?q$04 zTB^N3=$i~PL>q4W{;)7@z!bD)D7>U}H5OXnIfy*>+Q(XF2DDWdKIX7md%RHRwmV|$ z{Dpdq2tT6kd;7pFF`&!G8ouKBu z>(#5ucFvJ5n7#X^wIS^=WrSnxJCr*aGoIXKL)IN2 zLfnY;j=P;G1`^sC zL)^yiOs?pp>tWUmPqT;@gQThlJ*%h*9nxiEmj*B;9?w(oQN8c)|FD zm<1NcN@(=GL%zd4^M2>cyqFE%V!CNp2Wh_+cO6|(b2Qr*Vr;~{_^RCfC|y09oGW-- z+FFcJICkV$*p6^j+`erublFXJx;Zy3Ew!(ECYi?G^Tw#VczkfLi~nfTm83jA zo9`*{x^oi7or%mT@n(Q~>Tu*$m08@%Q&BMkuhnE@bmz_AaavZR40vynK|v_>QK)pz zY!MildatM25*@ zl*PkfIcEEeY!8CyBtsU%3~@I0F4b|$`N(_glSnlkM#AiT88MJ95F8i6Z?-o()%1Yl zzTNGm-sb6X(A@o;Ig!rKK5UL<63?L4l>DB|WyR<=H^y7`vTOHuDxdRTa)hh4bKA2k zn|+AOxs?N<1*tRe?`dJ7fP+OLUh|LS;dkVBdd?COu8shA6A#0za%nYV zzJ6}bq8fXQKG0> zIm}A<^EL=|3P@Sp+Ek0ZX1QshJLJAfH0i!Zwen2Ri68!#T{oa^-#va14D(s%2M-|YKFl+XA zyqFAU2sD9dNzKHe%HRA|y@$txK{%74m}zzR?I4u=J9~t|mpwOq?iX!CfH&dV_G%rm zxBm-m;4o18#WUL_TIVEZHY!*}^v>75<{d1t(O2u&^69Hn3iZ9mg5+l8%f-Y@bTN#| zJL2`z%lK8?QQ#JXb3Kl{OJ;5M?Uwpd&(OQiqzA+KRgRBJa#`iCdTHJJklw^7g*bwX zQJA;TK3RUi4dVHD@W)s@G}Dpf>EvC4)Sp=YhMTDFgwnKpmB(WDK!Ao6W%rkJIg!Da z7=aY7jyaHPwa7e>z@BM~$BHj>(+yWn2yhfNFB!i=N@VuFgup1xE>|eHDaV^EQOOk$ zh5^R6c*X#~W5A}8Ce0#5g9wNpV2qoHNgWw!7htdR%_ErjSYb9qB70B3oq8I_z%4DS z%2I>CT$|9Jhlu}r5b;acp>u8PRSKok=F%nZ|H1JoGveKSQCXV-g_77Qa_Z$Y5Yhdd{l`lKu|2w}vb#QE zen}e_gk^`%8Z-3H@jbOO&U9hBE^2|qv+Mg^+W~0$U}C_`Xo?$b;DgRT#jeImec1j6 z`&k73sSPK7Ol{KlrzzJiTGtIgs2=og{9qyeRT4-1Ktf|R+o&7+B{#pmZhq;$*A9)| zW}-OjSAhin2i5e_?>y&jhtEv|0y z&sA-!q%648^u_mefk}4_-+nU=It;v@{?J`2i8!Q*D22`SV+fjv_MKdaThc5z$4cqt?HC}&aT7eXUqpkJ$b72>p zE+2y!X|ZwlCHnnbcDE3Z4yZ3fUJKR$e?A=^jw|YSb}9=G9Xes(73>=6l}ySXU;vwm z)Y#STODf`wyx$tvxO8AGpGA~_WImGpA;`1I`uJh2x0ZP?j35MHl;zMb?Jc!>xBqSY zI=-e!I>Sl_vjy=PMThMx?&o)xAPK>oe0VKKxoOoHI6@gRI~VFM7jg_&q743xZ6AOH z*3f;TeSQbelNBDYCQ^)x=s*N~foF3NT>+?mB8~ zEd#L3?&!VrEmtI(Wm$|R5aVE91g8}h4b}GIP`vvES-a7rG6QZVo8}{B5-}^PtCJ(x zyR>Q?uj7w>WTyGsrGDRsi9KMb-g&*~(}7_$U*C)2c-CiQUM)r1TV+Ns3w0zczeZqZ6|U3uIkEDJ-P~Uwzlp5J62rYA z=Atyxfpv()#tS{YDGN|aQt^qrD`@SQNa$nFRgp9X@7?Y>)kx^oN(f@i-yR>T#POR_ zjt6!e*m_AQQzVrbMsf$(`VQX0^Vq;0*_X|zPp*^yQlhPG5D=z+(ISkJ6`cE_GqnS& z4XgZxVDKTQtdrlVJiZgtDTJ?o?>|Pttn@yAG@9t9Qv2KS1Erx>BvR?gFGYmqTQ_ks z-Uqq=KL3vt{QS=UI=}GTPb}!%#9cyaOIB4s^>c8xMf%(Seu`rhq{dq)Q)h=hf1aEy zT-c?Rn8RFwn>;4OQ6l0N)zl+!U?E~eEGP21?4E3qetQ=LJAYGVaiLN^m;ojaADk55YIpH_ZS=^WkgzEEEMqYUCKP{T5pvb!M zr>0$~)`W2T7B8!x*#s;E$Th>I2BAM^?vMH&{~*$>(qReUt(pPwmUBI)vHA*AJ}{e&-;Ed89x?&-nk9YN z*FW_Ou@T#r%;_6O!S~R_`wUH{vJwzY#SbYEY~r%%$W@3h5Sb~nB8THG;?q6Fjs@L} z%@HM!$2f;B1)yLaaA$*+nIas*L_jmrhk=w(c(BVL9%uOSK`SwLmJm4@K_wF91=C;4v~ker^yBX$~LJsV=y z{1&f3ZrGJ<%%dO+!zd$XE9s`$Sv?&~`(3(JcvS`wDqWhYX)65|bLhzLmhhgvZ4jG# zGAoRk5I%U4WEh?r>a5Qqdi1PV>bu4;FTRbW{c@E7GxWsfcG`gTOFrko%7iP9Js%z) ztRD-6>k?BaO`8nAIA0Ho6_+*1J`=tyEYLqTD>~bBsAlMgxM+rO(_d2dtyNp4(hx*H zl{ksGXF7GeRXc4!G)lc*iVJ*hMLzc2WYlx0weFtVW6U3@DM^dgR&gLyZrxV!RCRv; z-TqUEumfOzBOs#b`*2)+{mrsspyNk*7Qt5IrRbnrR- z+4&LUBqlJ!@aT%a2nhKMgGtng?;-{x<}u%daeYZ1ZD9qjn;^ZFBK|-;PDytR6c-6= zDig*K;vbWECM{j5)BD}7sj%NB?2Kqz1X32)Lmj#>yFeZ6b=}bfu}b}Y*9Ya1|CZ%1 zRKGc(BcTS*KzMLdIHJC3X#P9JtKJU$R=RoHus&+_d3Bk27OCGO*4uZU44B>`4;jA7 zf0SsBC3q(BW{xhNJh1?^8#@(HmUAp{F9buru|J&A&LX>WEdQr>%L8?Z)!`scZY-Q3 zOW_(?zE!AsgWBGGe^wzoo{8s@68)_a?m5TVR*kkn#~p?C^T_^O zwSA6h4_UR<-X|+!euHB#74`XV+wU9p);$B9ce;KfPo zWo3Gy$swtgVVC-n$aPq)O$MKgIR%I`72Jkgpz2QD@=SWANSJTfG3x$)*1+zg<8Pw3 ztiAcPRzRi`ds8OCKSbe#ck>Sy54GM>oEOn`1lNbtc=oSF zuhEshD+KZKki4ZT*09EE9Xj$hl}77+Ss;9`x<`{uqL}FUyUf*SoF;P3=DnpT>o~~G zeu2^4r)P~^=Lco0g+L}w4AmxdUFpGJa|+*1*uis$o@z(0Ru}Bnum00taK8UySK;4A zCYKK5u0m(oA)5UFaqN;f+QI9CA@b(qdA|S3mQv?#o*L&|#+Oa6Yhn~O8#?c2H266_4-axLp$rr9^{Dp~ zLUdtYvItxy_}vYeRhX-qQQl$o|1Q4FmxI?CVp=XMG0mJY#LKyzM4aru;GrpF0M~DVX)Ud2f`e3fDqQG3n*ko==;tZ3YnW_nc_+g#-OsXw}^4z>RF$ znp81BOq#r*!45xkxZpp%-ImPawgZ}A^F-o3@>Mo}xnQo4u5AAl+SSSEO(IJM$IfCYRu;t-zfo=dJL zoHfJlORndoU?SZl$7xai37w{=WARsaU7qI$Uqcpqkt6iuk6UaSZ9sp%+<=pqU+a?O z{i2V@>88PKDON$EI)eKe&kAP~+H*qAX$#HKCP@KRR27(V@15CxUl}!e7Dr>RFIUB} zGQ&5d9gZKgp4rD1@^4dnW;q2YulpMMd%1Cu+{a^5XGNeK@wuHqHKz-78-G_qA!G1S z`MzKWcz4?oLDKGGBnkHcGUbyW^g2{Lp&a%&eS7&`O>xc3A?Ds?tU38R-!eUq#z)jO z)P2k7mQD__5U^{Oz&)Ko{|!W#PjT4RaG6`2Phq}qzrd=RpM<|s-PiYqA2;zm;73jk zH%B%v-RG&M1Z2wFKOnoElecuCWI_J@A^jbGScqlc{^;-_>3MAYl4M#-AI-W($TaIE zn&bL^0Lef$zx{V?{kyHk-J3Scs{0<4^-nx4ojQSy19ofR7lNko6QsJn!FGw^*Mlm3 z{v2so_^QAS+2540$OS*qp2>Gk`dVh~Sg~7#5EJ2n-X8r$p<5@%`bhGNE2M4N^1Py} zb-VM#J?6{w1NWE7`|KsvRn^K>AN%3C4eBWLVPu4_(}&0?^M6{R5Yl#lE@bWB*~0ZV zvBk3fZ~Oay3|6pZf6c>>$>t^cqMKD~w2yGRCPI6oFR9VKrdEGRY}9@SXNbA}Mi-~_ z7s{}&!EMHPXydnm92Rl(7fb7Jw9gs9j#%w)(6@E0N_j*;^EXYKDih|> zOd#-xL*3fDrE&EwYM~*0iNdR;d9UN8THi=&LjaI#ld`NXl)BB2nnh|S?;#EIj*wbC#-#$2$0%>ou;|M$qxm;D7X~yTJS5tohDoK6}b`*m*U|Rmt@|BPN6*Ub~S>Gzv z`a@3Lj76sIP|sFdpzeC%PH9?uyVP%dNEfKzEQ1sFNhZ$)KH@Bue8+V}`h=j25VkQw z>P8iEe#n?y9~1XApoQCRZrdt<(1oJ?_S;WRI6)WxcY&~I(IR=vTizmn{No>Gv5w95 z-eaL10#Cbzwy-v&I%A&q^fiC5U0}_+K(_nAhsyZbv$YT6_{nC1MoAN)wKCB5gsv$o z1&REPwF#Rk^fP^*eQj*F{@G__*#q~=lb2s@v@>3_NbjfG-q~lc01qJreOrMd|WBb*Ih*b zrymdpEERUhq8SU~uSH#Lx?pzuw`{?zdG8aXasDw$y=$GSpz0HTPJw5JRcm{8n8K2u zUE5oljKE!bMH3qG@{YxLLhz19a#nURdL>WtA}g*vRA37xPJYBfm%@Pn<6JuOIGu4# z{FSPIf8T(<&bVE+Y;BiLU0CYY1*Wd99_iM_rY_xqfma+OuBS&ApmgGbH@U#Ca?qVE zgNbA{7P#y&Nj-4Uw#B+%oF=z-%8V^rwAzPlF>K=8dD1dvsx(fRpt~`~OMOeL?ZQBP z#G$a^vG!ek7kWfKhp;2Vq_?lLQ@ZtL)V+DLbZ*=rZTic?x<{X|Ps^NHvj6)|ms;H| zQdR8+hN15m=gj}r3~GC`gMzI6Cw-3jN7nzjdr7hW+?p@^$G6i`_JX$sH_l@HKMz{% z)!nL3{N{JEMF(UEIQX@7+H@TZO_ql7XMckG2?AbNG|<;8&C9QohPG#P@-|M7 zlz4~K=_H7Q(!BOo>FLs)?Yo~W z!?pD`PNAQL5OrWS)cT?iF&$~;6hb}de_y6YEn z0G3afl&1(M(}YM1@sM_`<8l2$Ue-16LjB#+0?f@S0ggo$o2T^m%9U4NFMaxh+*xOx z1r|Fuv@esJ?pz{M_CH(}%`6-j?a0EL5p+tAHR5Kb}s1)xI)8 zU%v3bWtYmb+wYc1`|hoqDpXFJ+qzf(Ad7D@9%A9lg+Ux6ggNIxpZxaP>+O-ucfRwT zK6M8H?W32i+%!67HLbGy%n75*lR`az{(L$5=%eMf+isI(%U8(kS<}@%RZ^u-FPg2j z6LI~BH*8>Eq=z4{59^0GVM_5y)DBrRBR9JV>aXFQf4flh=WqR`=tyZ=aI89JP}NcI zh)xR&XY9PfMF;RTM^vw%L&8HChjcg$#)N<_Sd5}BhP-WY&c}xI1vdll{4`!HkCl&t ze1WE$;!Z~w&p417A$7J(Zwren5zpi@kJ8kF2blpxn_YmDdp*uQQUv$M`U5rZ$Y-faN zG+`V)oAhuCaQ5z@3u5}{lzzrdpJ~}Zw`IkP^6bqw+B5Cz9(z)sV?R);hJzbEe8L3H z`FiI6-oj{W)@aR~X6LXyJIk|H&^NfgC2RlAe(=lCURnRA9%DY`{b_{#KiVe-oX`LD z7F#gGFNX7vI7(Xdmr46Yk{tkDQDIu2<7!3IT=6Bp19nD>Vz5iCK^N!n2?N?mr!Jl? z{o7r#b=3-4{QE!2erIUkSfj7p9(Iqlpx{vddVTD0c>i}u+as4t#~t66ren^Ps`2_* zlnye>F~A16m$UyQ&igwdbOQ~}u4-L+g+3m(OwV9lAma{vw^ZpHz7a@ESc*y}l%%d5KUy(_{)Ra7A+0z1q^mP1g5QCz`Td{mK7ab@r~4c`4?K9uS@MRD-z;*{ucZ5$)3xSyUU>W4-!84K zt#Y;M_xVqL#0+D$&_;@l6==r@Sa5L9x?j&A>(}{ejXoU&+wzAsOANm>4S7R8 zz%34RV6S|W&W1Q3LXh62n>|ntj`6@Fq7{1VJjPy5!W;{iG5S`1sQV|MUM$Z%_q;up zIZlu4*pcDsD?TFoy!+Boviz=W8vdAU|FW6L8E2d!x88cI)#bl_=)L)J(4K@rYOYP_ z7|%BhK(-&^ObfCWpxeD@hC=8}eYzd6>(SrH@TTXqQ+u_v?DG#MAbc_uh6@6a#wN3< zY`=))?mLL^vIY)?m{N;OT99`_&wyonuQPjDoa4`qCNZ8CY;hC-qChsX^Le2Y0-}NX zBQRgH5zA7BGx_VZOo(*fOvcg2S4^;rAAnx)0?^l|RjYT7Jy#7_Nj;OQIwB;#hzmS)}6+B9*Z z9TJN($KaoKV?eXkzJYxV{FgZIC%*EKg5<&sh_VFt74|oX>Bml8>@-umV5iyE)hliH z8Pa-nXK$Ad0vjvzAZ6`lPzH43fB`YMH#f{aK+lwEla8mZmyUZblB#3Q(Suj(?7moT zi)XyA6H?c_@rYImCh!>KHoS0^Etu8mqj#;kd#DnR&!f^(f~z}>7hEE2XK}uUh$rdF z$KpIchdUoI4#o!#;`v4NDI#c8C>wDHQ4GsKUgCfZg?OM(iSix4fIib-cd$3gxP#A> zHr;V1s~(ZY=XE#Cf_JF#UEe`JB#iwL@x}a&bv9OEF+x#=kxxBMDi z^skZe2ftrxCoNPTb

      a zGF13|5A9z|ey>mQsImoBhay$upQXA?+m`pKUjH=S`Yr0cbh99OLFO#N02OShnI{SO!UVLoehv%G( zHa*tb7v(e?r!t#xkVJ7Nt4mf-e#=uOzv(qz0Ze|mzhFt`7Y4x+JyTi3j2-AK zt_Ho?m;A_wfpp@@bKp#w!FzV#5B%e?u=n971g}Nk4F1_?p+jEYjeI3{j?zY=* zVcTuD4JVy+Qn>KK3&Xe__6cX5czD#hCE^$DL>n;(S`y+28JL`qXSmWUB1##^S3Jp2xbm0C^TKKoFatw`Mi$FZ zw3T5*IOwDXKglg<_h!D<##V>+TVSlnm_R-oL^fy>XdWC z<4^vl+^p==)!e^=D^!Q#hM(=w`jPC$gE!2z`5Qws0I+HmAOZUqh6s(U>Hjs=m9>A@ zT05+3sW~jW5>w<>#L0(w#b0!Ku9l0YPS9f@}&_ld(;*opnsd#^b;{aD8Wo z&OV)!0BpDXaa9?=d^P}WlVJ)bAh^?!%gY8O?Fc7iG~&su*5Kc%Dvr!O)p()=j4e3n z&Ofa*V-oGGfv??ABd%~`4I<{L7V6oH(l95XLW+~GcraZ!VXctWHR>X*f2}-?7H@5f zddHvD-!PZCLiVV7iwCasvhG~Vxwefr1Q$JZP=TXJPEZZD)s?hN#8h15R8X(s07WSy;c#yF9y%Q#WT2{ulkF@ES^ zOp>8}@FO*BORTF+@b#tqsI+ua)+gf?N7+OtC{4%eMe%dL6`f$pgSdDTDot?~foRJc z3lJ`TqE&uUr>%{2%qn`7VZ4+0G+k+evacZRDO@!D0jkb|tg~csAB7gIKgsQ9a#@@a z+i9-fXq1x9p&T*}d?lV`z47KdVTZjBv(4eTDLh}5Tw;kOBCpvCIN7rP`s;_i_S!4l zcH3>?vpek@ZpMe%mi^e0CWXsis#6c;H$8||>?3qqfKgu=JGhG!tda1t+&b?>e#)&jlHw9nIu=xXHQ3`xGa10THny$Eo$(f@ z(kPIGVVrP6p4u^ol@SRMPzkO^QB?tvCqXPGgI&p=WH(fGtjaHzF|Wz5fyHzTtZ%?m z2Y#Bi96rCzreV=V7PS4r_~eaG+iWkh7GE7=`iut8bSy;Bi89t*v0ou;Jdq#N%g=!h z|9Epj+Tp18)M>+PP{d6rxdA2L|4swk7`VhXFiLa0G z#>Lt)6s9)Kd<+HE#gneKf9FsZ?)CGyoE0z33}~vKgV{yzysn ziwk$?@rv`aI8l(%^Ty6!TVgu-r1avOL#HuP97J2IyvlUtbZBECppsX7D6&1Q!hT9E zoH)bvmp(fb?TGuCOvCkU8YU1pnJ^VA$#`v_hRHLoGUMtT#_M{|e8l;njCuHAtbOT^ zg!YN}-2BvOVKT~Z;mS6^L71!!5ARe3XKk&Y0Uc#Pd{s)|Y7v4rLRMD;|}`x>Tbtrk8K@Id15h{qs`IW-w4*=N!+e z8XIGqQXed+$qzwEpmWz1^(pI!!lBs{XquYh6}qPm@n;iM!0~tmzTSd~Gkj$k6TQ>g z+HGazRQM=;e$wPgwhFBs<5z!M0uO)TujYm*m)EGQjwT=L;w!~?ZPNaLoOI(vatqql z&{j;C4f07dPMmR)j4M1DtlxB#`B69VwF2uLL`%q!$qJ02oNkfHL3TE|5Z|1j zPL$KfXsUbl8!062=WbhbYVBub_KuE7kam!yfv@99iAi9&(bw~##B(pc61Lyt(D1(( zUJeHycwo5r;)`w1zCISUQ}*-EKfk@EdHU(6+a6q-?flj7t1B)FpIBvi^Jg@4OkDGo zO_zsWZn}s*?x&D4s>A3K`7>QG4xyCW+Xy1nhMYGkrS}qUY?AVrTnI|5k>BKCfs&_? zIk#P`Zz>P{L3^0}@XG%SN1b|}efjd{n{N)|#*Hg6n*FEz(7>aQJ{q>(dh77Rt8NHy zzWq-4=>?~uPaOhn`dHuP?xy@vl(fxceHQejc9W*plfK83X0$(lG_RlvE0~;(5+SQx z#F?jRFsDVHXn5w6Kt><=Sz{Li#pyhOs+-C)en?k+NSfm1i$^A0&LBExlneP)26-{e z?ed#^&^ZUsT<-qeDyz)+mAl>Wx@;1`o!h83dR*jeA4+iZZ^IBblf*n>6c~5!$D>WAb2fn1=TG-s&rc z0k6LuhT^MRGbc_Cwx%aea=EOy=79tJ|(PL9I6!=+BnRJ`zJ8;Yu$|=3)YiZxHb2 zkvJ#Id~ZL$9?{>^{(-fSofywCpl`u})njNc{^kTH&G0Z7<3%6ma*P}I>_wrfQ>p{< zEz4KbNo9Ih8nj_WQBE3dGuPncnxNqI~+-H_j8E78~A zGc3_D8JKDrCrwG$S%eZC$~J@+O7s=!kxZaeU$bMBpPabO!WGxw5{|^19g8foNci3F zeiv3-Z8hbd<8ZOX77O>>b5HpE=Ra>N;ntAJZVNFu#OB77|sf@aC6cul}_MPWx0$bJ!DLCuBkAJ6)cyh zvXw`4g6toxh#G#$ixY!jJTZ{tpvf2(P?KAA@Ju9}yX2>Cd_X-jTyx{?;eXG+l>UflKfX!yd%r{>cI5^IZTaD)~`FZ(|Z#yIUSQ~gw~#7$HmR~`Qz(tL+~2fl&Mp3gEcd>W8cz= zSXn+9D@X@Tnj8jU@@x>&22Pn427q=TCMgEshXTyytP=5HB5^_kb-|2An8DXCF&Q@# z?SIC|wlEN{aSXzw`7nHy&>qC0FW^lt|4eDp31Ku?S$-zCUq=Gd7Y^~%_*k4*N@$Jb z+*q;fy7tcsLRJ{xbc&(`S!#-VEgJtwpLAdKald3J!&DNPNoGsah?jLQzc+~Va@$Gi zMZ;BRd^MTZ46Y30WbpLC{;DP)#7yU;8EzzPr5XI3Ux5c__Yt7P%)@9{U>vt{qL%zP zVK#{Sr^N|pTUCbgNNU$8#G7w2YYgJY)4r5n<#h%t%P+>T?w!>))wjw?M9uP&2SYu6 z;vrb(-yQplhamIHCj>1mt+0q29I{QRU8Pl}T#6T7kd?%A&J=nh^2X#T`H|(|?A`J2 zhaLAm+$PNU^~z(9JvOIlAZ)+=_QQQztL?A0<32}(yKefaeQlk6Ko+_qKQobDI^wxC z5ytTZ!O|n-xo8O)%K6a>JVa>x5F>t;C>ov!7fj=T*Da4MJTE7Re*OEq!(n(xz2Jfi zhI{Y5H!QpCvSip$VbrKm;kUp2ZP;?lEyGQ>{Q-MNjt&=`b~MDW-V}9BbkP<0qjc8) zfS%C)j30PKc5FLpH)=<<6w~8LGnSS51nbYB0tI8N10IplXk%sF5fPvHnK%Bt!52M? zc~vgisT@HGuKbKw9O+GYD2$MSdwy7)eb$c=_%SKgrwmm#Aw#|TL@Eb7B42-(e>$Er z^OcQ>7#Lsp!>e-HBF0Z}@4WpMwk)256HYeSr@%Rg!C)iPG3iVCmaSyJGC@6km-F}3 z{`3hS9Qj@qys;QLVual&Y_QH+VY7|b3wPdqUqN<$W$asLTo^7s^LX=7y(%DIG2*$v z{<49YuTFb(1*tLNtSNuLZ~r~9KieUHT-h>ZYJ2$lx6TOvdG>jCkL0)*Zo0p5;9h}! z2w(jgHF9|1#2If=hTwHFI*_d-h%QB*41&GY{>%No&YS%|b!}!ay*KhfKF2zm?_uD` zQ0)mL)4T0Ea0-nl$akiFMvHzP2L~3%lv+WGbRQNkLI&!YSafk>__r7)BNQirL?>wc zj7O}GWkoFK&JalFuWGvNlnqGoI{*Mc07*naR9|0#rj9sy=5ao0ruwlOd}zj%W}IB( zYnnr_@}`}euH(Z1oKRz4hg`+uZND9D-$cN{HuTRb3eKn$%OU|MBZ#$dLc%5@e8o3c znGSDjvpo)nWAaR2CS-r$wxd)o7opX+iu}gh=yIDIJ<69K@LV{drAK{}F9W3TG63zo z8}=tp@fCC)UMFMNPY#{QBAKYILg1;J+oFZKW}RAbMxg8wT0dNGf*Kh$(KQ3(`xY>q zj>%RA`Y$)&W8J+!;I+w_n>_GY)=#$9>P$GA#c{heSryM46S@q zhsF7+P(lXQL~&V!5}1ZWHYY-hW_20oA^@i^GtLQwE8X&VRy&sISpmx%`6Z-|k}cD+ z@{4VW32Yx3fzjg->9eHB-;#6ca2Rzt5=`i+{-ORIMu9Ez*(TJrxT6QLXR;DWY%xJ41w1GKp%rBgfp`31U zmSP9?Ji6vs5@iD7E3=8@{AoW1-lr3P{NK-nePPQsTtDx)X ztx#A^5OuOxL8jPxd_hi3nPDR|!$zDm#tBz~2yq!RgJ;n4k_Bj4RxB@r%*UQILf_yv z#<6Vt(m2DxOAv_pw4me=^m?hD$cI0s35rgTDRZm*d;u(OPzWyfkT?}8^ackSh zi~|e4_F;osdRjlTzYoSSc!wM=3m}&DbKkQ4_u4tkH)>>2aGYIp<8Anw%Rk^k?k97r z{oVWLUH@;&?;7Y%|Hpout5z<*_NH*-uYX_BcDN7Bw~pA~Cd$~x*mu~?iH&)zv$sCK z+5hvp!v2&hj5r~$M6>Do$C4X8=*B^?UVXjKh8As1Kz%x%<>U2>A&w;RN9Um* z*_mf^N081^nEDbe#GrJ$Zou~l(`$%dRb02knJebZApP6j)?vw&t(?k(+g9aqg&6Op z2{{=%5R+;AfPlYR$=5GA&hzV!`8?9?JVMImKO5M>2YsJF-p1M=<>H^@i?YsX`Kdep zQk@c~4h4yu9Rw|p$4RfL&OI-BAmh$Ixi-jl`hsU4#uRq-Aru-6)ZG*Zhw-@QMoUVM; zV2@Xt88aB0ohySN9m5O>=m|(CPG?g{(6q8lmSD!C3Vp^W@~fAhT{!DTX#69d)mLRi zsB#4@Pqn&78N^3$ZkC@qv9D=iS^N0$AB2mltTLN_^n76#tTG#g*D1KK8CRJN!9bb= zYuNxM$U;5)8DT;{yX|kQ%`k8$9FEsd_-e@15z}mtbm+djg>PT*!%P+-hvAdV_ulsN zFdCC;gYcwr5C-{Ky-AeF)9CVBb({V#)8@9D2iw0XLB88f`T3b^z6SHq<4=YY&$>9* zXM!F0l`n?XR$Sip5#)sQQ1lyZXmk8>Cfgs|N*0>^zp0{L|Iydizsw#DipXaOQ#`%C z)2sNSOAAsQ4O`@PbUDQ_-GbT!PlOAmaYmez@Kn5N@)w^7t!}^x85lh)^;KR%^CLjp zIsuTOv--8*)y^S9uzH8DO!7L9S7Z2bfGOB$o$Z|apta*QO|It9UQt|mM!hqRFy-&| zVH;;1ysdMRf)iz&)Uf@cFk!)sfjPOsiMf^*Ot^7ef%JOyH3A-_Q=qB*N>Uup)B0ID zbq|8r^)RZyFXNq*k;OT+9kX#z64&2R^B=Qm9RJr5NbOIM0a8;$8d zv5)l=I9H!zU$tTRF}szI`$IxwM_Yhi*9XJFTXr>mY5izZh6p{cw;V7mhgMh_Kyu+j*XTa9n)x#lz2k{&SnO-S@B)!`-)B;cuPbe%w}zb|OE;v#uzr zsGhhfE8(^B_-Lb6YA5lmkeo}<$W=wD6pr`{W{zqajRrm_TIcjoimN>Hd!Edt{-4U6;!XLZ255JN#Qq$J&^O=v;KQ)@p(ogd+==qzW*dJx zY`y8mc9U#v%vO{5eoLgHuB+)g_4yPRXZ3YW|L>jtUpDX&iL~69kp58qv8M8qtkyP| ziD&VW?{pp)UD-q?UV}VSiPAZ2Fq|;O!Itsj)m8o#1+aQ#q)>8pOX{S)SzZ!~k07F0 zFNV(QXRCBD17WMcFp0&9?qPUP#;@HD8#dI&0=7j1wrODeFid*2VSwHkf^pqC_iPrD*V$b+xUl69?6u;UckGLV;`@r`k!FYrJl$r(pzcmkOx<)L~C7I=|Ql2`l?)s&w~h=ICe zAe0^jtLd{42L)5dS)6h5gM2cH>endc>Fh`0B(q>9aMZD!+R{zm&QpC5AATHl%6T?< zcIA~liG0R?n3#n3R&Xlf5I&ihcbWGopF+ZGC%UfQ{nRKZVoH2ymC1E?CKxg z62C@-+}rHXLk|uA`QHoS2S2;s8am}-=uCd;AgZt~EltZj4m;OGk5`(>X;_FG$_tXw zQh`zd&Eb-l(fxHv8ato8Rh;KTJPOIup#{g?p-d^POgu%R(Ho4>Hb@~e1mT2sZh+2g zrv(bexFhDX&yrR0%}esrSv#+JP5<})hI(L_m)n0&DB0gN{BeJpa-w zrpNTKP{YwO@J5__1j&uD9FwHtB1sx=Td$n2oi?+Y3anuUFLv;TLM&Zg^M zynK3@4SH0YRrPdED4c0jnW-(}nJeL3?nS!nqy~Y85Q3SGvIy`dzuhe9_CJY^1NE;E%}W%nXw9ysdx)2UQqFn zGU~u%?3!YXiu|mMesZ7;X(*p(X>KytS*p93U2H4~afA#^PRKJ{=@k*B4CWi;6Uz~Q zt{`Jb$>n>NCl-+5F~4joXkOVqODadTv^0*oU`XSL^T$fCeB4HsN?bvX9x`(|$|XysQbKeMMk!gHl-l7_wx zErn_Lz|Ew$-VGDqekV-E8&~aTq)Vvbt};=0IR3*S;2f#2}9 z&^C5-7>;}8wgtw77Ca1+J<>X)c$%JdFd$I8V48<}igrxzz?U-5v+?((mtKnTG%nWt zFns&l-wrq2a6>rv(ktyPwj~!|)NCVB%JM6o_ER8D`ePdXKr~E~y_KgH?D3=-N}yMH zpJ~n%i-?Zp35c`aFG7+?j#`iemKF(w$I}XawqYj?n}vJx`%f&7%m zUdzNrzif!?gx>#Pf|r*Z_ug%XFbbRH>x*}MDr`@Ltz6+@op6r~^5KBlLO+t9RW7!lTvQVS z2Z`h6OCl&P6XmZWtOXm;{!RJ2Z-4m_w9GylRHktI1zo~Hx3B%j~*39 zV3X1jZNqFeNh<~uc7x1UwTzZd82w0)@w1`-Yudk>{ndcYI1PKyK!{M+{S;@#>0cNj z6x>VksjJyI0SUVPGoRu}UlDjMHUd%-^{2q9hZSfEmh6W_t;Qd*o;#t5sLCc(Nb$M@ zL1Pf>2nLc>?8@br%d>hxM$TQ~XjSl!AaDW=50%IW&_PlEBrfyk*d>cIzQ)+%(g82? zvTyMEk}qRCr18e}iVd77uo_j#7;&TT0T zS*A#kDTa(Zj(PE85X_J)B2DxKJ-e(*Cad>9taJMTQp-(L?WpL}w-`s%C0rC0nSeD%v;2n&s!-&QGWY|4*&u-}lL zq!U8<3@6Om@dfC&pMEa9`^@v<{TE*eQ>)zjT3+1W4a7Y!C(HQJfp)Cen`z_Uj=V!5 zp#tpFw&46>^im%Q3oO4(7_-#keveqTo9Ihz0ENU(#O=&FGqBG5IL~kXaIY=s*nRii z!-gAfC@%dDa(}aP&N;`vjCvj>&c1u%5qOA(mP>s@2il&YELdt+*;7yxW};y{aiy1e zrRP1LG^4@zV~C)DE3Rzl^3{zzL}Ls}{EVK=+#xEU)xkX9bBh-tLpA=MVn-^8GzKHR z;+%|plJcwZldP8Cb7eXy*QYE;J$?-&KlSp~JMUp5?_WkCbFjqXi^O-w@%jWeR7J*kxR zk1uK7A0B$_Nl9SvX?z{j7cjPB6g6dAcA86%kY~|^yixHO& z=|EO;i=Uvfq_P&|0fFRES%RcfKJqcVx8x_oaxnQ3DVU^F9?2w_B=<{{dZsLd`cV+q zzgvYiNbMk7866KXcpl}AdB|S`Wmxi6=2u;e6z2uV6UBK-;h5IZNxf{L~k~=1+0YdK^}p znVg&~Q%KP4;$Z5+*F&*_?8STk5ng@hi7*{|buzSI8-X!PE)_=LmDORR$An??jShp` zhKHee*gFtoR}0W%p{?V*0~2rUlP86#6DEYo@4pu&zc)U7h%bFkc>B#T>CJb-s}DXJ zTF1;6mRN7C@R7Av!9yoE3tS3tWA1No$&^^ zKgus*?Is@f;r=SL0MFZ}O~Va;dpP22dxZVJc?#a2pDJ>Lm;B_K@cFGj6INSkd7BX* zMEl}rIsrw~|MLNWDXangzhKbvNwxNu(v;-s&;Fl^mEl$7r%$ub;P16J-^M;>$J@%x zEKfwX-C|tW0-Ln*gBfiYbc{gX!M%{!MmQZAhy)@)VsQjc2T7L?#fA1e$slBoC$KIEXWb1vD$WDSveZQ}^nxOrPb0>@Jw3 zQ>W6aV3OP~QTetYg-G#gLX~w$eeyLvdLrMP;r-qBFTdU%U>ZOoW3W{D+1<+=F5oDbf3ExhyWv*FEW zo(j+0{>Sj<6HkX#zPL>of;~NXk10ORo5vafEV%^&ZF%nl4`C6=Kq=e)$?1 z`cE_$;T(6|arR}@%dY;vaQZQa*p>~02PbXYtYFry5KKBGOn()Gh!AbHG{)&s{{OT0 zB>;96Roa!LJL&8M2!u841Q1k20TqTt*;G(uQ2_;I00ji)SHulb$0(zKiZh@iE`ST8 z$Rf(PfWi#2$i9OtK?qAAAqz>TyOaFicTRox)~okkzwUGg`ZceT`_8#lr%s)%>fT#j zw{E@iUh=)OxtzcmHMI`TetJY9P7sg`mIw_;nV{1}UW{goXz&#zAJ!Kno*ybBO!1tq zIFCFDm?*EH;go}UHQHic+J@RpA?xy-5J$7>&spr*&paD`^2^K3Jn7FS8*LDFee(_& zg!ox{tPsIqWc!UsZ^0(kKlT3z{B?%T%h-~wb!&Iz^Pq`_h3-`_-AK^uCAqU zKlXP@;y`gP=y4Kbsulk*d14FoskG14>XW1sO^Vl|wY*4Az(oHD8V(r*Yw<|ikc=s0 zU7i!-Xx9D@;i*LRQAdC3$*<*o^iyYq+0V|QMJv)~8*dQ)^@xM4|F94Ffw2QrzHFX$ z$6>aMFm0=rEX+%3QoJ6$Nrie!=4+fpf0@pfN17T(+6M9bU|r@#e#6WE&})b%JxCCy zcq+EM%9|zPx~U>f#SN7Cnwjx1V&Gx`)Nn#1^L2#8ZAcf4b(vT2V zPB@|A;3Y^oO!f$+haj6`(@casSSBbsLDChnwe)ven6Y3N8IU$29#bv41d1XFi@y8!;aIk(l=gkhYX3gryF*^*FhxwZ6nl&9gSC};}cr+`2j(W@?DcGy=m!G080!HmYiuYFCNFy1~4Fdi#q z-o4+RVVfD7RY`v5y$^!oX&&+XARp${ zODa=6$x`q~Mau&u8>16MMk9LF6~7#kAM=_ZSq$-^0`lq~#WuV`2j~R7?Dd8S4TqY7 zb|YrGg8?BA8dwmxvn+4QS!Q{4=!?3sT(pEKFKdVVDSU(}uJRn3?4T!1@q$jW$M%f| zctPLN-|a?Y^PrssQ`^DDv}x+I)R$II^hQ}eyyFzbtm|;X;VXuwR-ti!B;)K>t0*w>@*hP+rbsnH8pw(qH@&FkRlok+p+CL0OTY_v? zIHBR>Zx6~Yhv|U4disDm%16xU1Y7#MKV%2@joWKw{ck+T7D;_exMfwLew{Aw#d;)o-XTB`=V?MFw2J@?%+Z1=HC z2W?WNo}?fA;0LYf7nfZfx-r?pkJ(Xg>MuLclC%T&;9c~>ywLgL0(%W}>gF?8Wb~ZC zs&$YT^WByuUeKZu!wFM7*28dR<)cQmhfQ{VYZ!&+=XsAmXwtw*S>C*>&FSGkDJUnc)(MJ6^_6@fztmm2j<&VJ|Y=<6tC>HNL7p}kQ zHj7#RxRCpj+-Qe-iG4yzxWB2g8e|i~O93rUtxIr0(5zhz9ap}W6G+^=n-gaI(iJyCtab*oqGJSehITIJ;baK|~YHs2(S#Nf>3ruEa%*5BohGFo2*Ui3-fi4N*3 ztSG+=)-*y=cL@5wrN24ST)lNrn_txZ`)#4cp;&P#TAbpZ0tJc}cPLPxxVxlKq&O6J zcXzi0clRQ}9fC`eoAjWlHepBmPb z7WGv5123MOq;wXG?|fH;#1ixDZCbX}&0q2+=B_s*<*#EqvHW$ZuhIYS$9Pe11cjAZ z&A`kvcb11goo-odep<{5i_o7%T81nm*d@~9A4h*;$}L{HQ+Eu4Z5~wt=50QdH&(Al zvYZ4oomq#+iMWDli+a}B$O~%^$}fR8uVB5>JHPE42*$(&lnaX=M%?3HW^0lOWT@s= zhQ1VlmtxOiGYyFXM+z$tFF;fG&VbhY4c{d*UN7&gmXHS3_Qgk|0NsZ(dbH5r0%xRR zNKnOD^BSjJ*X`o=Dxzzk`TcS4qU1L!Dk5N}1!X!FDLvfATxZ?P>b!CYblP!_KG(cD zDDY51#c`A`ManA{BMv?!nqqIM!*|Ds-(|bfFudEM$Bu=a0l_?-Oriw$(}v0etOc^o z&xa$V9&abndcG1rYHpz|uO0yOuV2JXIgEg6=oL$)mEUa{jwfH$eVc-EWV1;$xaT zGyaKna>1Zm(H?li&)okXjV~2{AZRay=m`&Ei89)=* z`>yZQ+i+;wpIrB|Jhpy`TxTNcanm9Z@!*X}&#gKrW`gV)Y1FFR^~ncai@|VSht{7@ z1@BEK_}p$&T8aOH5ih@)O_(WOJx2}NG~mK16u`IPzFB6Hl^`=&FS{nz4fSe#=4f?lM{n5My{d<>f2IxEh}f&JR@P6M zyH}e1Rcld#cnvujgfU>-G)e&UX2Y{v{Iq`V&J>$z=`qR3;bm&XPp z*&d~{3VG4ntFs+7e|Xiyh55_ahBWDHO8bY~IssW7a;~_XPZ@LSReY)kEbN2f!!@iP zaW$R!+0fL$AA2#{_f_o4EaK1k`9`fzC0-bRt49BnG}qSmOZc;pnDiyXabDO=O0Hf> zKcl?oFlJiOlweg%M=LN+w^Pr_zp2iU%YXWC&FW@RDO&d`3E#&<*0JptNWPHL^=2Vd zsC*M$zYX(M#sBW3ocn*ISnjvhM;D9H9;NkM(4>Rl=m2qt9~ZLo6Y47dKhPvcjj^n< zljMS-*{#$PRGPSco5#}%+>Ua)Ja|`v~`Vh@;Y_BH5 z@PIr@zfgxEOg;yUFCKf>bbs=tnoMk{m1c5bBj`=EKMGu!qkw-ZvPynwyBI*@W#7hX z%+h%f2n^nQIiD)>r@sqba-j0?0ek3j%*LRZRJW58ugbdINxZGJW6~W6B!h;0PalG$ zat%XdS&E1`hufPj^}X|}wt<7fCIEZzZ}?Xwa;TQ_@x;;u?h5eYa*4tL0}t?oDok`# zY;b{j0NYL3Hka#~5~)&VcX>PcmsXX)&;>)p`J#uV%1P3E^A}LXKpKd_ZMb9dq*vK^ zQ}fU2J43Xufc93>;i(+r6U4uB4V1lg)s(v#@DzA1;uPR;A52tBggbR&s@s`%5Ohm^H@Z zSlO#Ev4b>?{s;bFIbJq@pt^X@NuwYN>#4aPysbx}887tbaU47~-;ICMtkFjQgza}3 z`(F`^KWb<=fhC7Gli?0Hf=)Rvj>!ckFN6P1Q@5v-jieNd8A>>CcFl6Jw2%_`!V9V_)zP3ct6W_7*u-mIRlDFR;H#T&hOJ^|kA z9x*?C=|tT_N#<9~KkZz~KMhJW5-IXZAM(5X8fib+b_FF?UKh%?jupS-ZXFHqp~g6% z(SBsS?U)qiHN0Q(2T^P2=v5SxFi*fQ_Se7;%fO6lwq_&<8pVMk;N7bPOQBS41M`XL zz^X6VL&h5o7u|EX4l}#~(fpv+g{#|f`2;C6+yLj*60azDHsyWDbK)x?4oCl3A!}|!&9%i*ApzyytJ1sQj9$dQ|R`g*mHC znr%?_-;c8^7_`#k;I^NUPc@0~xDn$fQ1sJM&$V>I*sf}Tgxle(=Fv)=1dF>SC2Ue9 zsCUGeL`e0FLY7pr3Sef}G;zy7(Ab!=Drd4rm8Sort+9U3Xli}F; z0lO%dA_xv_8UiXJBDxK4OfKfNlkXfMowWXs1_vP&pM*0@T2i6e{cpEez^ND zW`=VAO;PzBUk3>meQN7pE3M)c;FBrRFu5%L0*9w<1DjVd4wgb|;D^PvLnWSu7Rrh> zzBGRQgF44qOskPo9hQQhnO40gnwBtZEBQG(8=WL|kyeXeSm6AF9K6VVP2hU@){qx? zcQvBbrgC`d9cMZzz?m*a-jtxbeC3lzrkd02_&L_)X_fP&tn?qd$T9|z9GWvgnIDh4 zwWp5PL%iO$vgr9Au9?^oeaiK@SE|%ig+h)2*BSP+rxFh?1-DE5|H=T44nMf+Ei2iI z{6kxDlL&cgxU749c!wF{$hEUkOW(-`VfZk{zYi*LkHFwnV{FGD=JUu`RHY02LEmUL zoF~TY32L0sd|#3HQtBWcU{g>^*YzfgmL=wld9RFJHuWQ$=JYc0`ma)2kSw&GWA;)v zv}@d<4YqL^*0Ubx!uL`o>Oj-H=7VLo5v@}>xb=#Z&~kk&3a_tG$EJZuF7TRQS1%}1 zdpZJ(vD3yUz03-Av_j}84DOv##;*3-3a3r@@`^3pymc1)fUvGi*uI7T&c}Dt_YQ{y zcHCl3>{|j)FnPJZJfX>=TD??({hYV@J8q*mJ*H_>$mp!k^I6 z64&uQjA@Wm;Z~~?Grv~dDVulk+E1-4>RouxZtk+;nwfVW+pK}{fa+kL_VewJ4>_60 zZgE&)JdtX>ZN6|%W@!=7+I5-f+E}JC?TE*BZ=X`-AU*Ycp~ER2quYAbVp#6D*{6Ee z4ko(_c{AJo0D|u0)H*%i($@~$v+aI$DvQPi-4)FSg!41KZ2+EU!FsDB@E;4fE(?OB za`c&KqsCS}eF<$_3&|41{zrqMV3?f}cEAIQH~J)@=BPsHYC)WHO`UdIi0_EB512J! zZe=>nl<{U=e4@d69$)@cd3%Ne5@t4{t`@&tcUqJ4#9qULz;()A+kST^jg`w|zK4;o z{MMoOy+kwUMz|yW{1PhFjR7#`H6Joo33I93Wktw%gSwNQnbvcVShuAzdI;aOxzFeJ0)C5rE%Q^(slp;8Q*Z?PFm%6A#Zb4hg~* z6vy^ch^)7$=>p&ccpe?91Ws1h_M2QD?}wxq;;F1Q6?Xz@ z!BDKjFWF(6p|_HA5r{z)3+r#>QK7e)$mx2F#3yXmFD$zJinSp{i~!d0JE@N&A346 zs*&|d3Xk|^vZe@dit2j5g5cpk>BY0&)XpsShDz~%9?co$grQcrl@e#LsJ(yIW6*B2 zSm1d|Oxhs}(?7dk6m;=1N0TF9Eoy}x_hp)>q#Py4Z_d)(X^j8B3;k4s+AF9!%R){j zi)leWfdkiCl(X54N`-F12YOVy9ZCV&Q8J%X_#H9Tadfaf|3CnWmDr}CScd=60G(&@ zfjO>o_83ao3Ce}m+nI3@;bV$Z{!#`3Mi_; zr@Lhy=w!7{EG2ceqV1d>BC>L}`_*WZuu!k5l>=pX;dL<|bE%se)7lp^E4zwp{Sgk~ z&wzJ-AvV$I`|b>jZZUJL?m1vW@95c!Hm%?gXx6D}6~xy`TT$Q*_^%fmWiLY$_+%te ztxieCnt+(NeXzFOA(39=4&)ZOal856YZTM-#ci8^f*!;y0W8L`W9gZjdWo$)s1^;4 z7SP2L{6~uY$#y@hnMb0FYA;Zl4L*S;gI0t>YHLlSgl%NeqmA%Mv zZC*$E%uneJApu2VmG=}}w=da|;evOsTI<%qm0;~O_wN?RKTlDX7pZmh1~JS@Wa`dQ zz9u2`uK)~?9X=c64 ziJcu_p8_#Fx5}KINP=RdKx0uUQ_~*Y^w*8?NBJzWBsEP)zYFjFi@@xWLri;H8VG+5 zC@6#N|63_vzrGgB7L=Yv;>8qAlU~&IrctPVd;&kcz5Kbhh{~b#`|{`-)m|dGmghQx zv`CJqN#oOlkqz5r3;7v7+!uvtzl$)id&<-~W zrch6qT-LupqBm&BtkJgiTUrB8o5h)?(wl1%uf`Wv%vd2g^geXVDtp(kF8t*cFT`#x z(Fk;)y2fcJdR^@_qjnI+MHS$&?DoBZz@5>=^Pl@!n+F0f`^OrlbKY-Abn-U)8~aaW z38pzd|7<4)4F@!dGc45x02zembz{oU){aueHYQrL9bt)9S|Hb{DV!227>loO(QGmx&s0M{RD`uB=7YxscW>|HPFLU=!(@FjMC_KQAco!p{JSngmvU=Lc@*UzYJs;sO8{ zC*6=@91jT4*P5zuCLqE{imACuY-|`JENcJN@vKyRx5gt17@ed4MDjz;VT7v(3-|UR z#UUuf7-QBT?dE;rMi5vAXl%%*;*n5zoZQLeDAa*O&uKKDclGXzx7aOTIe_p0NQxy{tR z`QiOnCoVyN2?sT3wB{E^7iGzJCZho^N8a{V1Oe5lZ$GYn zdeSr=O_;`tg`}e&sY=6;??WSRDrc{6%;dVq8xI4J_Fs#?ho>HL&&>wh+tjByQ)8m5=+`|6CqBQ=npU#29&F z8ny?%q=7?QoUYIAQ-zMEHafb5?Hadz?g9ihXB>ti@PQNM*n^215A@P6Wf^;FHTr05 zynx~q1E2s~8lyC(7L$KxLfd(!&<+C~^V>nQpkEc24lrojS;zs$ z0`V_G2~cXe{N6sNR(0)@FX9QAE(S)+w-x#`7>`j7B&DtY{*};g%^oK=h6rH{b9o_= z!ZrOOC>bn9+7To(!~s#Q`*#IzDFUomO;MoTsoabqy>LDQR_9u#T=fQEDtc z&$S|Eqwk-YrIEeVya8serpE2cHJ1HydsoTWIf$QZIv*TX5{WS? zSOACOX??vKg#Sgr^6cin21pYA!>s9u1;hW#q^(`z13dLzBD!xl>TAB{z*-#90yIrw z?u)OhfV#g3agT?q9vp8Gd3^9XPuKb=o_~nTg6*B0XCE1{lU;fmA#S^#vqe+LgLCWf zV?|o#=lyQWGu`NLV-07HZ#fCoRS)-Aqu(wN(ED$}*6hUhcwHFjcz41azcu;1^_;6k zaV*hEerN|Fbz~aEz!^;P`1q$-=$i(YCdSm36epfWHSZ)p;op_8WcbYIGGN>rPhRSU zmbsHI_VO#$gNgqZh6)YZpNisgin(2fnSu(xUdq<}Y&^;;zVz`gi_85o&y||ck}~A; z+vS?%xG*5{ua~cIkJQqa1G<_}u}Q{93-P0IyKUsP@N1bPN5PZamA(;AqboxTZ4t4` zPa)7}rq3}j@XhB}Rk)L+HGZMy5Mz@Hh6%XF1L!{MdISLr{04)3m-AXGnZnx6-;ruP znRl8|MvzN%nYX{UcPfN1Az<>2I6CS0v&F}2-cvh#IM?X5j7j)BdpF*~9#EUK!X5P-{tNl?6)=r*2I1{}{ zF#$GkVUKY+8D2fxg8$tEPBEE!E>=%W?DmF9&*g%D?V`)!<>eOz{8tybJ){!%O^#t$lfoz-U>#K9 z`_cf5(jh4}xOle<@G?`rmK7fH(=7s7gkqKT@N7-+IBbMo)i^_n2Dm~B0M0(TH-8aH zvt3&~j;nC~?OfPqsQB2W`|=DmV%;k#0pyV-jg;i*ZFG-8p|QYXq5i3zFlgaY#wzrV z060?(*%S5$G@A^Oa^YE5#^00&HG5kZc!>-fkJ z(w}ztY^h*fc_Z&JnD(T|A)~ceL8VTCLBdC$oOyj${!o!kTd@eT-D9+u-{|a8Cm146 z(iutN-SXzp`ztmgHQU+63NL3~MuS^F+bD87#-7g3mv$<2H`ZV3*B5)M40$!y5!qu# zpzJi0X{>I3?75-kqqK~(p+r2!E>THBbh#|N*Ugg?&?Db-=rb0#!o#fIZXyt=PNyBV z7a0Q1Yto<=PXK48n>~8TWG92)O})A8MIGQj3=+=D4c8Vy55uo!HCCYHP6oEdv%eFa zq#{v5Mm{64E8`kJ_G%*}2uPlZ|Bn2p0dzkDx-NyfS{kqAu6k_u&T(4{3_HSDRoa}w z)@Wy@qpaZO1V1X-R`@Xnj&yUwc`Hy3%2mr$#D#L@c_WBmR0A@y=_4-hYOh(hKK`kY zt&HeQ&q-K-O{7ma@&}W#rWYJK5~YEtfwx^?2~D8^sPB)v0>8jT7ijA1T?I zZ%g|;k>4Yd{+wl{S$Jb)0-H+Dr2J_x^L`NG&ZqL;&9q6`cdgXvW*aRu!=XcT+`CWE zHZdM}-C+yrd%hsPTiM~T9E~pplh(y>=8JIidTPDgEFgXt0jZ~J1Q2*Su*XpWLKDV) zIVlxkSf$gZbgZ__-m1nDdX5CVl-qgTYwg|%@^E#hV2L(}6+>*Ooi*rQ@K9w#JN8qv zhZy1aZA)k$e*NmTS10=8*8{4vmMY$`=*uvTaR4Zeba)K1ojLCjJ4p-9;gr2wQvAi`%>YZ~LP znPW>4u(4Yc@9Q!4*&jsHNMh%30(^jyagj-HNg}nBFpgKC^MI?1N!|;lnKo4W``WAt^Qn zCNDRy)q$v#Va6@SNUt5h2LFe=SVMpdeCdgw@$ zb!b`HXy4&j(PeT6*m-Tf@_hzj**%&Qie0XLb?}M>CjUq2DVe;ZH6v#U<23}%ok_Mw zY;J+iKzp7>M2i>p?q<*ArYDlw+y&IL_LTb{o)VUs6XzXKuTR1bZTi|j7)AFneftB~%^bxr21xePM=8mt{d zV?nsHrzbp7-UD>-^lsgaK=fJJT8;R<@w*T34K33sS=PB|b&04blTDi|1!%nR_w=m7 zrbYtynn6)n@y|le!bc398bFBL22J-LF01_`L>9cN_8PYa$5thcPO)9NLIk^h%qg<5 zzT1_;MD$_^gPjBwS;?rM<TRYPi%722{{fM!6f#>@&Ulh7+Rhy63 z&CD?%2zQ>^6LGqpOXaxPdeq8&5-%rpmcnqQB<2R8f&arv5-)FwoLi41gVcwk@Po;zm3P( z*?2tkLbKYdokHmRhXD#>DRwcwW<6)O1}ik#Z6vjhfv8*&7;E9!r#>ifqkrPf7MRQv zr6;V)Xnu63O=rz{z&Y*T^zm*6zQ@NhhI4N%!wNA5JJna1Kfmc1@n9`6k}2r)ar*Dm zNSu=-i~a}h24QEZuY82E5+pyw3&M%r#w2x%iMU+D>-yJXAN6E6y0PgC)NtQ8@5ZIJ z%c9ssO-FbatRv;OPzy?K?t+arH|vK*AbkEt0Vs6hkFG5h4K|@)4s6Rz!2_KRQ~DVq zgj!Q!SJHnH*GvLPC_UgMlYRp(!X`3Ed}ep_$x-csm0=zjHn!1xqOz^J^zZ96k@a1J zE9jsnz~SM%CoAGfmlFqt__=>1^^JS@kL`00ov0x879bn^8cin^TUJwSw!Y4`Bi8$;*+*(k#^Ub0w*?z-8KH(l=*&Fu!C0{$(`1ugL# zFu^jBXk7l;Jhi6JOm8@i-A^Ewko+Q+JbwgfSaH9vQwA!b{(?=j_dw-ALP$y^pl=>k@^(M_WBMdYk79# ziRE@7r1BLMRujCL1wXGvw4rDP6t=M|lZk6BjDP@ppflsO*$5 zX^DyJ)?}v8yXo@P7lx3A^Ni2ZU0H{ESVCJHl8Q-YbDhF4^m3Nm6TKrr9G72icy@Ah+NV%mVHgPwz8xkt?Gwep>KNB5uYVEx~Z@;Dh!Jkh< zBVT@I7&7k;BCrjN$E(C`6K0bpo?Gunqkx=#n>RI9HI~!}G-(9gf1$KyI*c84!72aE z*>*&K7)VvvwvA%`WW47q`lg815j!g6AMHH%e;5hhnaPf1&b@Z~j%DAOs%!l%t8TKs zc~GMeBT*UF_>BRaQVGy1e*c#)zpk4o#?IXas!yT*u_AkHZWd6-8?9$Cz5Z*$)p((W zw*8+8({y85iRXLDMk0BOPaW022Ww#Lt~(fiTSnD8hc%3vcTnqkMliCgOcn@lSevZ% zl8?ElSr}+$h#pxwlHqQ-&%t;7lGl}u2rPOzrtd$lK7NJIFtb=|Y;i|HjJabu=J;8q zDOSZl*kJgG-!SWwd2AJu&2`;xP$1!tf?%{K-jn)k>2(C)G~i*|v`d1UlH`3T1y?(z zZ|*8X3cFpz-MK>hHP5|c^^hH0%7XB-|&-Y`?nPT_`nU?`RN?~2a@;XH|R>yd&hS)CeN}=jC8Q;uzjzCUeEKsVW7}Uh zo(@1s<#T<-clnN|?7DnTAd0z4p&^_td}7kDm`E_=%UWE11OSwAL%kgO>kmC+S;djw z)=O6J!e`@|k8#FK3+YC{)$puX3ZI;7SmeL2BNw~!gms&XYtlo9p9F>*&1KErF zpu!axj*mH4!Qs4>gF7btBJWy_9>!kv*wpVISA@h;dw)&U+rcWzd4cqVQB*%!r*b>k zk#5s8l^tSLJ#am9?^G=yRu0y28_}MxzsCT(MX1=+K9#|*rUFoWnZ!mcS!aD7peOQ( zr&k7AgGm;y8&a2}8fdl{Mvld{>1YQac@={)JZsWnc7NF-X)cDO{%%+a+hb$);hHtRZ+ifJ* zI?_6WLjZ@+3mEYRdp*OKh-pOV4jKy@s|5x6o^u`_V;JQ&C0UNu&SHnoccFMM>!KMD zEGmtAX!zD4M}@L`wA&s`3Ct?JG`V$QLdGn68AgecOBg%Su|dovKQ77OMFaL;G7X~R z%XT}L_o5dR>sr4zw&EU&xQ==_1k!Ldo|?HVDHP-=WLR`=o-9JvSt=ROWZt^}QXN@f zIaTynk%Te*a!<6nWfU@vklUv>qt4roGl&!MI`H_|9n|#TYjj{C^anLdLO|)66hZJr z3ziJw;}f^omk&06cl~w81w(;Tl9`sK@Ixs;RA10?q2r4429L9yPwaaAn$s5g9C&|9 zxR_d>UPV1~k_K;2y3k-S!qDja9{(gZKp*HQ{ZnY1WBC+rB}W~e#-+mNGV2U#bv6}? zt=j=ncXVRZ*mQSuC&UdQghey2E)GJfs~vn9V|?>wRQ;Q3)~bmSGGcwTmV{)>PXa3F z&blcv(;fE}B?!UUoV%@P)S*9!mn$x?V*oI^QuniQXZK~eIVcgS;s5gTCKfob^VbD= zBW$JNpB4N>6|rMN4~ujLrp6$C=K=B2rQdC*j`YxCVR8+sUtI26VlN21M=$>xcU-50 zl&j|xGElAID>;u9? zRI*vkJuXnty3*|WeF=+02#fu$1z#`5vlN#eIu4p@Uw$;I+KR%QH0KWfp8XNtPH}T? zl{d9U5Xd0_F|Nx{{XkSLGJ(2K<6>0E=KjH$3DSspJJc*f53}75_M_jwU30Kr`)TN^ zd@F7|QgE@Hc>*M}l!emxzouBd@3L9Giz5WRk_CD{EHbv-7FhbvA|qNBF+KZJ{xO@q zS?b5w0o(k_OikzUrn|&2g%)Bgd-`;Wmymthc{^qDqEt-lEEcDiEeTmMZOlyFZO9Ps zVT>a*TtEK8Jzx*T+Ituxc6|A>*dM-QH$b+xJK8`X(oJshma0X%&fHMW3+!Q#@i5!g zt(9K$W24i1`)!bf{ z)YAdHu!bX6B4zW9`OByGyPKK^L^{Gd_?EmG(IB|vSuMSw{DBFsiqP>libIzU#1=#m z%;C8asU!O7*I@HV2Q@IE4X3| zi6k_VB|74cDeNQp#_IT{Rd)j!(@yU&1_Ni<-a~`%-T%u9r@qSXe)y9e6kn|JZ0Wl* zGn>=Qy_PZZ^RsHZe2Z5U>L{TyQX^0)Zw(?+y)C0K_@;iB*Ih_hNf!B0>1g4tX@qFX zvB?4>pIBg&wPKLa$dN{xx}LeR<{#1zxCPP+EHSdyvQBNPf;JfUE6C8|JJIhpPg+VzGl{A@$^u+ zxE8HbT)lc#j0`NB-Ew+rJ1(zySHlLvU9*2<hjzx_cI^`Z$372TAi8PVbU zlp5W~W@{Ky(1TeS>yH`(+mh6pm=$F9KUaGGJ62DP{hz=epD9|z|F#)+XrtdmXJ9iy zP~^#Jd}>Ee+YSyw3sfZjb&n@puY1O}1j9C(BVq0_z`Pxki@|L!qq`;=!(OucC}4z! zs>Db+RF-=|ct9(bC&G&JV$4}3EfxKlF6T$mOZ4|qUgYQYk<3>*zGm#a_|UJ8F{8#( zia{}a_yjc~nzFh(we7=8J*)wh(%t;WJX03#sgV4G2=js+lWyTu0|dI!iHDMwj4N<~TJKz*2*serGs zB=rZpbi;2(U*d|JyU9%>#&8y!uX+#BWGAQc9?;1FJmL2BMof}`ikD~xCb|E!Tl5)s zNw}LcnpIbkvRvMVjMJ|jGXidM!7P(vE%Z1l6tai}>XD%tg-q{J61NDZhpGZDMjgah z(;qFd?x;!g=o<;We*f|{@LF98PfP6M09QbK!{O2+sDlfG+2};8+j;easl`V1`W`)V z*jd!TJ-DsMztH6DpdXo?E$Yd67ZIRA8ZAgW7!7{=Y(BI_Z|_C_+BYbV#)P+kHa~~~UNN+!^>8KA ztGIP^P6X)zYs9@03vW5ppzcpYoony>>|7r&(CEL-&FxBOUjtmh7_)|Pp|HTDV<$1K z;6t%s9_-qee8sN2R4y|0m43DfMQc5|TT~7Gg(*`7jYyiKk(wHogQ4pUxV$!qNQbIf z(NO;X`K8Y82Xq05oCx&HcJxs`#)msectB>#V*t6pvlr!^lSV(! zeNDLOgB%>VjVgMVRq=5ZSx_wyDp_j*!Wfl}dHN@XQMYQ*!qHE}HNQa>rmnEH>y*M` z5@^cASHM)?q^Q{m+<5ep!+V52YkUSfvmP=symJqv0Qp&2#Kk3lEd+BY;&Ee#5nJ@wg~xIx_&dJTcJ$tWk_Hy?7s&m4Ijv4{}PF z4&<$|_0MV&Zi-q@4c)E_7s#5J%aUxmkj!W_P-EJ;eKG6&+&$Af+hozSOBPXzYWy`ye!!&U8 z9Kd@Wzxg$XC&&}{aP2fCj%W6b!^D5s8hYLRki(Z4k4Qxg7c8HG-q4&t*V~f(v)UDb zeTf2lGZfjQUpm$3xQczmG|2w&Vh}iL0uC~+6RLy%e}5IO`4*z|W|tTZVNcAeEtKuP z5weX3?5Zo5uTFkBm6zLzY+oM2_T}V`88Y_8V+iERg%7 z)F;Tg%O+D(T4)B_A%cLe0a6?N9SuADd6DanGtTbmbL|dG4hf={YD9Ezf5cu!ikYa=_4M@$~S>v|To)~XwL9WVzF*-l{tx-2IZoP8d)!@x*0 z#QWxTHxYs1oEYEvPtEW8-v@qn%=zM)K^8-;&yFn_wY*EuRF64~!Cg%rIqfu!K~9xj zg@wL8N~d&ULolJ8R~OyNQ$<>0SNm}YRmF}clYc}Wf#C^=iK=mG4)eeFYf%1$6t-_t z3VaG!)U!(CeC~o(HCxs9=$|-tCpk}Wv+Q&jPw(!8FF?2RjFaeqpA#(6iOm^wos{48 za39Wob|3W*{i8$EiJHV_`<*dUn!?6fC`BQnc-P`8#6wSN-8NjMHa$C$vw$Mq-q$`w z8F}ATboi0qfA(2SRazdg$JWX~WCAjX(Q|xZJUp`652istA`EwXwHO@N+~wF|H;j4u z)`Pf_MhjOOs|iUNGQiQJy6tZPB#FjB)hQ>VNcoI{qvR@`>SRY0aCJ&MeV}VXD6kpc zZX1Hvv)R(EbaH*$n$Y89d3dUC1B|YfOsZRq07kay!wXy1hx*q&8U6NPIipIQPhTx$ z$J1)4MlXU`xQ>xhcEzrN2FEAYkd<4(P4aH-os;UQDOGA;uVt^N_S6^qhK#Bh-1DYs zWA=nQtW-H33scrTHjC0lqn++8wgR99!ei}|>SIu1j{LMrea?0T;B3ZEzE1w*=0_3O z`ZaM4OW~l{{ny;{D4ekVu5RfW41J?a^wlOJtvAuicN!mT88nS$$yJSBAN|fl3C9cl zxkS#HhlS-+0iszcDC#kg)@C>5%X^$w!cwZOfyc%K0Xmtc*qqh)@Duw!)(<3%adu%R zAM2Z13!4RKgLF3oVXWs>oB3RERL)XXXJRi((HKlobmat6qL&(<>BeKHoyJ(&G?d>j zsr|#!Qu9Nts?KShmw>gIR;wL$iR6p3NwF|GNgb;r8n`2l1NY_&Zk z%#*b`O=mKqteuEn|EL|Eo~H$oOy99{44?>tEq-g)2^28rOYfk&_-UtWyYoT!W}pTx zRyp3dveW~2pF_YNa{nDZc*LHy*)GjN)9q)hsHo#hV^oO<^~6Zr)^iGOMJ)Ik#F_ih zyBshmt`%|2%?7a_HQ|+UWL;s;X;-$ne%Iz%O~tDxrT@d@Z-qo0TfH9GiT~>}3|sOC{-$-gE?_UasY-Cv5b4 z6!38QPN&W)+xLUt!!EDg6qxQEJ@nk)1FNI!Hu`m2_QfQ4EW$}um z4XlD1kR}a$+Rj#X(ckGY2=yQ}Tj~)L2%YuzHV@Z5-P<3rSfm*yZIQqq^;;n1>VW^gGJN3Xc=Amm22VOQZg>fgkcrCIl$&lzPUvMAxoB_Euii4Y zXNtU=_g_2|fekHddC6UKVh|*&;B2(8+c4VTs`ep1=B@vCt{Zg#ZKeBk^7s-U5|~-W zg=99F#G20XCc|>7C4(jG>hWs5p`{Xo*YX_{NDY95@b~ zc%Na`kxF()ZrIaPxN;qPnUUtHS21MhXSb|r@Vm(61}ldYSNLqrcxRO+)WGWAF@3-L z`5en{!R_hk1PR|;rM@reX2k6wwW+xaY*=F;fAh=VhbVjKd*XDoRKS<-WE;c!F~6ss zyS0yKP-rho-^HtTw0MlGMmZ}K{aTE0b$G|-{ zle#5kJntpryUy{0!{+$qsVLY7IIq=gm|HoO-7hkWa=?3`tS5E70&--VhHNjtH)s(@ zrxo*6?V_nvUgbOLB=IR?MlK$P8|l`=PKS|a|VtbC0Vbgkn)+t6y^km|(#E$?^GIKhc2GjSg2$RgH67MXX*k zQQw&U(va{8=BSy-a@hyIFswgM)0-a6R!`4$jr$$`Ps67IHbpAGBh%VPcODk-f`^7E zsWUh(a8}+~+=>N4y~(l0cT8ze`eZsNNKPh^o{%5%%uM6mMA=yyus1~8@5cmK}5v zEcN(hk}Di^i5t%UCDh?JTztsgpLXh)zuHE1?t3lkfn-~_MEcDR0_)xeGD3gQ@liGg zEph`1-~pg0uTmvFvM$%5+nLqxiZLX&Rl!MFpygL2qN9)^PMX_#vCl^wU$a3NXMemo zVE5^rNI9?ew&*?NtJ9v^mORFXwDF(m1Dw$1E1_w^rjS(M=#f{_4?q1~W9SHnXc9?~W$rUk zH5Zf~(vAI&c-_7zM<;Lux6t5L8hEzjA_;CR&&}kI#XTM;Uq1G+kJ+SApDoiiBb*O8 zD8u<5n9=8Iy%A2?@#9gaJHtNf_HP#M_~VbaXW!#KbdW7h7{d+op(htDP;csQuVPw< zrC}Hom_WpH*_ds&u*s_hcxGFSe$r!;SL|>&gcIW6sxm?1%WajJL*gtG^tyZ~BQJY^ zxBKvJ;0tm3657KJbr}1a&5GAV)k)bjal#a*4*clWC`=-8^_6`T0MG85Z@gjHXZN=R zt~UG5cfMo#4fJRF{$DeH+MzVuA4>5wudG#x^)Cu5>*9I%!dxM)-`GO9erK1kr-v&&sEYtha^fTtZaZt#C*{hNVb$xaEpM&$rk$6=TKIw)ZIEglW zy>-G72friy@Mo7*c%=(IKKTp(9=!fh=#hWbO6N0hCae{r=Zz(zo~ z@^pWYPPjwML1tOb+Le9B`$^hwSh1%l&8lYv& z7t1P5f?aUI1>rA$`Ac~74%>yDw%^vDY5B|rJ@pU40c|+&0P+CJ@tzYfqc>b9bm6{s z+5L}&rAwBEr7QzY8=jp<;TUPh3V$jF&*dm9`+lX7R7U7-7!j7Cjvph#GE8<^KH6Be z`lQf-3A?t{CQ=-q3}PVN)rn&x|8UJ`cfO`c{K!Mz5&rL%JHzRxpB~=xp7(@} zH{Q6q;t@*+u+CyB+GIX zxEs!)l@@tCtLD$Vln3P`50|%^zVhNpo~DB%kr!z^V?_=9nJ4|(t!xz$@|Mh<9~M0~ z7x2ZfWZr_XbpFE7wPBW^(x%Tm0n#t4G=}BhMgpol4(T`lHhR{ng*-UnTyhuCyH|%m~L|(u}dSbmZtRTYn$-E61N^8~+F5|5r|bdUKd0-a_!U z1AwC*p(kMutqj5CBV#o^pRY+-JZFNkO!|#g+5Uw z_6>_8C?)9n>5Pz(M?1qVMUZ-~721z%B$$`ewk5UXAT&z)5x9?^C7m#rCiS)BlE-AT zSV;I$-n7v~cIW3hTl@G?p0}y9$gidUfc;Oqn!jmMH){!1MxtSm6xW0)UZ_@FoRCw| zFYrnq7FY0(_KaPA)M+{XFweykH{N=8IOB{n!fw0m7Ixir*MikhNAAD>{&3=nC)yi4 z#~*blR+)`~UgPY!wjC?6xQAOD1`1=?MAHhaGJ_r3+m>Pbk?*q!89wGqg{h z5XMcyeKl6Qam$lYpr>rcNsI-_G!~H`pVN7N%?8`fLa4rCg|cg;tp^4)X-4_{;^B~ulKsTqLH~o-D==Eb>6lXB>XWO*fk{1_*Irl#j z=05t*u;8g@!!oWQ%Mw?7j2?>tV9iNbJ%pvEwu%U|ugD+4ceT-Rtn;8AS0VXN!f146>Y3^)d!>2g z)AGN^3hiL_h_R%2^Y~>P`muE3qVU}PkA`{hc_;RgT7lQ(KCW_c)M|Wpuc>bZs!rXs65ng;^HYUibzlO_kp&jGK*wt1K z?Ho6-ytxg3wc+?-%hYX&v%fxU6LAL^MV@EOM zuu*L1vD3&(_Qx9bH~xVd$?5}_FVqj!CrDAoH)~YN%N);hti)Y@<(9B_eri{+r9a2U z`u=I;e_ajB=SFxaKC}q3p9wGh)Et=IheOGzQq;rskxqi9Q5wF5aZnP&lFy7h>@5OJ zNUVm*n@@f0i13+zJ2$-VeeVmm-F91;Hf`FFH_pO^3&TMN9c24oop$0;Vclt%h{Cfm z^)ekfX+@o7gQ!0h;|ep|6O8-w!Uo)Ogs*f4JPY+;QmdP9_AKrUBXEB`5^rjZ!V0tz zcw1v6p4mp=57A;-%SbC2Yv8#YvMk4h+%in$EguWsSiCR-=bX6YD~FuWvQ6$KFJ$69 zoTb!b(tB$HZl;M?EUK!3j z^USc`I%|a!jyl-4iQ+^e;}nP4MK+-#v{QPX+9oP$_kHvQI*!eg#15pfpfA_n0Zp1w z4#_2$=0!Ox02Nh-H(H!XU?C_Lro!NyjqCe@-BbTOkG zj(dISEoeOWQzxbRQ#L-BF2ZEVBY(aT552Rj79V2At-gAgu+db5V<)eMl|hq3$ApPC zX;Q(OVz6-+r^F>t+F(URKj~SDrKc|~3QHHv$E4khVeyOe!b{K335)P$&Be2y3s2v0 zdziezI$@*TcMMaoWR(qZ=36UC)Qljxrnt9K{^sH|@?WW#E7D+Rz~^heaz0=1&u7B?S+m3Af4M38d?)yv7fSh@J{Y|CXH>k}{684+k0AqgTZYN9 zrQy+QZwk-c@qkT=)1DoZCWSRO-7JiovIZu|ri8ILA2V^1O}JKE5C%AIa1ws$OE3Aj z!LeiB-0;%e7s8@B&xI#%y3-D>WSqXs4q=_wZW+f6AGcfMKYCFdubVgi8Er#9Db@)6 z@T(K;US6jVmvLMZq-NelIH~1{HI(P&J{Gc0Uy|3?_}RFIWU0r$7Qb98(KK$2S0?IL z$|qf=otHZu`H8lke0iHW|GYduZ^XYAKjtYlsxLahoZn#UpSHvJQ_As*aau%82&IQX zH>SlHRmpvm+rE*y^0Qzg`o6T6@)9oC-}<>c;v?7>emRfyTzmx6vRZgw+Dmx}7wm8S zTn6zG>Ha~`uU4+3^b;)fl2HUvKtFJ@$o8y8 zAJrbW+`4DGt+ogse&4&o4=%ne?7jEi;kxUt3zH{LF4zq8HLSNV*iSSC9Dvo;1t+2Dur4^rbt}8!vIcK|nASiV2U7k9A`l7#)=nz4Jv_NB&+` zD(o>tda13p1({#sxjbr125h8HfAzOj9BAJ*=SK&dhm|4s{NgIuY-E_W^=rZ!8*dsW zVMd=TKs-?q%Iga0FJ-Y%*_E&)FRM!BN_J=mOF1zqH*U4nFhMfCV9Py@7C!fEnETkH z;n@f84|iU8dD!yZZx2(q+}JugUoT|hAmBFls*UWG+nWL^j~o;`!^!_1i#|b0|2L1% zFEgL7HDhCVhJUH?z*65R{pEh+dCUK+Zv10ELHNMN%R}d)CE<=AUm7~GI(7ApHV#ua z+aj#C-ukwhjDGL|Asq-fW_q73ibW=vk_#uSQO+B!jtO|h7q1XcUT?ifMAXDXQD(gg|Iv}MLE+}4-jQPr|#ZG1Q=*3sC5k1=w zt+(g;*BUlhj^a<>EP43xjxtp^+N&*debp{&33IMn;zy-#==i#mS$$LquX`|lsF zy6UPh0r!1FE%7UthaGlU`17CtZ2OoUf7Bs1;lkH4`Gt2*K-q+uy_!i~kqBraK;F-C z(uu3QY+?(Uqp>QC+Z1u)tq0G;vM%{X~B0k4E|5|ba`lPuy>K=450Q3v-kW5HBHyUwcz6HVBT2NQo&mQD^| zJ@upElrzo=pZ@fx!|GU+^O28ygj|MPy6djH!hZYh7dQ#}g-;wEw%HPIK0%KO@FgeA z_`!-%ZJ30l`etL--xSGXKGRPAVm5L0pNy>6(*aJJaX@nZf;>xcb0YExEm;P&^-~wz z6=06)3VL2*6pI?mq_Px*5mA|7POrA2y0LLO zi+q=dr|-BQpQT+Ewt43}!{l|>GbdWxaT&QcQz@SYOqL0+aDGW%s#Yt{g!L8@6Icd2 z^yF#l*ulxXTYqs;c>30R!j#Q6K*#2Qt+u^>fAxQL`QL8Lq zcR^;Rc)j?4b@|`By&235!p>u7u$9IF@1U}spxd+Tyc{!F=lO>pw+XZLw|_&JzQc~i zIA9-yaMt)u>-hyjlQlteE7n-gd7IxBl{H;hEd+3!Cn_Gaj(<3b|d_S6=^8 zr+W5kj`4qG{Xf+7FG6jIA&hhHTO7AM_9>#`-^vr2eVpPrZKJf6fI|gL$PIPc^c5oS z*6MZ6v*_x1b#TwyBZ*oSFVc9*>6IIcBVB!ZaTctWKbNPaKW!^}$sbaNg505&8iO8O|RRAe>TLrg%XRqnPAOf~0kOt9N;MZ@(#j;ez5t z$kd2`BYN=_Y(%d%t#SK_25RYFScAww{BQLUfOgNpKqS8ibKDA>=ttv)AVp8{+N!Yc zD9%_RQUNlTrr}2U+}Dc-Ur*!;t78s(PgsC$WVq67_uY37zyJO3!iwF4BCeXAPZ&ViZ{LnItB}5D5p&Z zp^Uh{PKwy&q>xf7Z=Pr8u7mp#Q=pi5<76D2%N1;EPhBINe#*aw&wlND;pn4}4s++u z9dawquD|~J@J`Hy@HN|$|MmT0hu3c9lZu>JWL!cgeas;ZWjBZGUj}w!JJ~Bj_mAHj zW;7{lCM#D(iT^Y%$@akc=myNEw>vpzl-n@0^SgC_Syy_HYlD+U7^xj zuyBMjP;7nH?xeL;okqL|6ur^d>Zw)m+ksz!VLW%JxCA=uO~IHYfOUq z(57L|spyL82l>L!kyGigYI?RK9wZk|=;sFFnad>_FVEYrl+gLt*T=b6-Z(B>?jH%E zt>$CzmWLmGB0Pj+?z|Uq?Oz-gVcg-(D_l3xlkN#q*9zlTkNXng^-1n$);?|= z*hN7K{$Q32gm`1ZWhG=Hx$<}}JSpebIF~M35IX12_dU(x-b6jvV4SG~o6s{cpt`W} z;j*z9H1P0iy!om&6)Nn56Y0!K!c)1G$8ybJ4rl+%=bSieMxSF@G9RY!!DaWgd~W`w z4{@~oPiw#yt1kZ8|L6zpJHh70;G*~(UZ3W{rQhS%SksPiSn0(*%{Xq1#)KzdiL^9` z<36Y4VLWOX|5AkD!N(;$Xb2cM zu-S0#|B4A(MY#OXtol=zVU*ans*@fXeQ@(DxAI6W+{<-YtGv{KCSjeRHf)lM^86u= zm(1863dECiHpVhuyx98;@uKgLlVy;);4A(tlMY_)^!jW0T)gUi^|d#I>;86YxbDW= z!V^zDV|@DZ!`E8gh=-P4cX~tEpN?EGqd!_VunWTkz<#jGtwA-AB`Ve7?*V2ZQC`^JhK^eIv?o^DG%1I{Htu&izdySmlCv8w8 zR^p0n=DoOJLAd3Pd%^?QPnZ+Lvz~l9EX0@OIYHco`yP8O2ouDsV^5t;r>`G2-(H7AMMP1ejRfZ_%kMbb9*l%PuiX|id~mC%7}8x$RtS2uSjedI6xb&%V@$N?J{P} zvar$m>xQp??!@rrvwmPcy1s7ak6P}+w@yPIo z?Pk~n8DAHs4><|QRe4NiPpW7q5qgQ~U*@5ZWH3X+%6C<7-c|AzHJAR4j|PRNw;r)?3d7I0P^apo1N7GC6$ z0V&l$B~CuPNnN&V1>TMJcce+N@rw{^-Mk(GobkJ%;aCrMEA7^Z9Hsar5F!B#BxHc8(+cWs*qxpTG0|G z%EpD&rmY($r43sJuFe>EId2D>&nJdS$VbfFYOY`35L1_N}7g+h#OpFA0~R?b*1& zF$3-L>zVutE5D$*923BKY6tHr_?X3{DO18ke5v#ChaL!v=68io#1QvSv?bQ)5WY+v zwt5xx|8mTuwWu&tgw3wMD>Gz&Iri$BPMk^5&-|CyR=Y`!Ff9}AH00;LtEM++f;{>* zySYf)AkVzM z{_-~)=bg5HzufQThFk8mu5-~S!X45cmF4Qw zu`61f$Um~4yvR1SvW#zfJn)al!zEYzAzbsnw}jj8xgYoAqEyYsa5BDn#|h$n_jpTq z&%V3a#^H)1wrQb4@r@Ij2{n_xWu-UXKg?uqrQU;WCy#<=OG zo7QZ4Rb)Q%%roJXQ%(s#``OQILXVr=@3rerw&&Gk?6=0Zpln|-JP&I_eCv~XUX3y1 zdb|G79JGYlfh6`^%vEQ6vkR#W8|pK_%He`%#-GjcMi?fVmSa9(!Z`eq0F^lz%JGnYZTQlcz7$SA`DB}MK8mX_qSFW290 zSix{PzJAALuY3?@VtxlVEF8CfSQ{H_^I^3U52l@XKwZj*RD9*KyUUN47B8yQDazSj zFbV_Wm<~+9;X#+dNqh_@%f@12jpTe}PFZ|QG#MAo11v7r8FY;2Kr+7pCGJb5^>;xwC_saRaEAYMj!AM;F(_f!! zB?2|6@VRU&op~;tkcoCyo@K?a!MS z#*rnA8~h8*nrlzPv9`r6#|w@fK5lfzv7M`iCNW6^8N~w8GGt)-MQjT@JIrA}(b=~;oqz&^FNF`n?&-mG76(8($Jrz#A zjT0YmTz+p;xI|F9g494V8+TNrF1)wkr2EB}{w|#L{fon6PdwGQ@qSV~_4IS$oB#RK zaMt-3h6DH8Gko&c!?8MNQ;Z4z>WMuh(6-SgePKUP{pCyDaMMomIolU5Z>$>sqoFcn z`l5Ez(0DwuANWDR6`NAR9$wKA&Gr$hEQO?Ve(sHGdh-^ll-?>Y4(#tEaEZ+;&x`1xhwj(ZSJdklY$O<3?)BG;M+ z+;8u+%Z@vw|32wdf4>=zX)@WuMAkOjY}3{JWM$lBt3p}bcHD8t`8%(6wN;<2F1wpO za>zTv7IW`vkMS`f#QWIvvVC8fgbucS++-iDShC_+&ddl{D2G^yrMN)j5AhLA5~?7 z4>5R65G%LP;T11y_R~v$q%6>W~GtI;6A@-H7)sa1*-9R(!3p_ZuU8(j@T~1xmP$pi2Dg%;g`a$L5BPbru zY#clB5iIykTb~ne!;U`ETd>f-r!{N;)I{(RP036ARR&bK@;DaWuUvct#RE8=BtC-h zDDj$SEni(u-AscER@(|y`ac`V#}LL2EA;+skoOa?JU!WC-CoyO*i8t3dE8iOCr)ly zxqk!K7>7al_>78aHy>Fm7q-#Le*!9vNH=6!X z`#5JEB4DoX9u9xV*Md=peecxW-jM!R81!H(8}ule@>$q}1gvcnB@yFJV>{P17fj{^ zC43d@%j}7Y6F&RZbi7Rt|KsXwCB~#e-c5X}J+j93@CWagUVrrS(gXInw?!6{?%d=> zf51G7UE$eenU>hpdEF_DUKDLz=SN2__;uqkX{U2KeI)RfpEKyQoAwjb-mg1t)<@N8 zncZBYI!_3Mig5rEJV%K}K<{H^*o3t=|m0dIPC^Gk(JKSMw zd#U^j>@ogdr<1<)o%GIkzB7IBgC9&s9C3ti_q^vlC(W*xUV5oNy2pgsjrIzmC7W)N zUihqkPLDe90siE|!iDzOpwnc5YikCb(_gI z??Se-f4>aR$Mn@Or)L-*c9gHM_bgsdqP`1H`elw)+Vb%sss|l-rl&o}b9wMBbSwPn zzBD@5-wv2mTw{+q@|KnrD{S)9j`{I@xyf_j1NTYyxyP>Q^ItnHo&Mu<{i6!I@4mZF zf*yM4q4vn@5^H$kiS|xE{d8ZM$j2l|_uTz1>3L6kqOUGnw2;L+b8WJ0ep+DnFN^Gp zr%Zs(#gAqibi!J02ZN*OFKM%>bl5^;6ZXtm5Gt>w>WPhi(u@Ywh#*Y}2a;-G>T=Nu zMy<|$SvH6tHA{Nog2UABY`ybHd;|yatIOeLc^&|^qnH${kT}88(c*;X1>Rwds_1Cs z6EHll%uQ@-bV?9RjdCnJTB3u}s^4@QDY->!L9aHc-thASuj|1|Gi6i3zB7+1oy~Wb z!-FIkN1gk!Y!JVim+A@*Q@^wI&Li;=9K?@ul0gPVY=`ej)6VAzExiSY(H~mbLAdJI zkT&Wh8uACV2Te&YTyU8B@aUJsM{p3oLFM8j*rQz^CE!EKFb@Yl2^Oo)h8s6v!75TF z&D+jME&@ncj)g~q6COACg^u?*5d6WCMp|EuqJM`!JH_f2yX0Bo;y+A%>h(+FBRGiP zpmOmM?A2~rHi#efOM2mg!_@C3gYv{j$5R}7yqSzK?$SqevDGa&p-I2e_*`EEM-2=Z zDQ$oiy561irEjHgeDBBUxHrDkepP;~`-Jz+gq>q-kj}^^d?NbG=D{y~A$y1Zt^LAZ zYtR3k9Wz?5_lnm#e>xn?*1$H#o8awGLAinymUsuT9d!F9`mx%fALzPSKkWYCC+DQs z*yQk~fBtLHOKQ%Od(5LwZ2w}bCXTfU;_bHG(i`x1ih8K!zMY#bbuRH*s$ujG3o4#F zVbt$G7R5M%1X(`sbQ;f)MtGEW%hTDnVj{rmG+q^+`DX=7ol86yH=w({ZvS`cA5}-| zPi^cp&CE>14C`ZQ_6DBtR2da}#5ss^uGeCw!X84h>&3w5+M|!WV4scGgdFss{nGyX z?rD$q|2%!|v>*G#8Ixyw?zv|=?6AYqK?fa__St72|B~ad#zl_b{N^|5%rnnSU;gr! z)7fXA?Nxc7+OwYWSbrpX$zoeo#)qZs{&zE*L}OBDQ-cd_vISY06ha3ky3l>>`de)a zRj(Hz>-;)m%nJR7Wr!%vbb@pxlYm_$&A?@XS(&**r+_+a9~FcLe+}MU;Cx zBFc2qpC7#a@kX=9lEu81A-)W{+@4_Ll_?ACX_2FzduV#XBMwMk`OcZ?$M#NBCO~=D z*ue)M>|eCx%c(oqQ*dKRdGkzlg@1xh8&CjjqTkY*-l8%_{7DDo$`vTUg-EI&77MJiFQqfK0eQ5 zdBSsCM2taO$lHNCpK^!f5ektjJTnji4rCjY&ppdW`zJUEJ&N zX$5Z|U+OD*p90Auc{}CnRjF+i38H>LsJX}{4@fF;j#Zlh7X1Y&GiuKZ=ou}!A9m!m zIR!?0(t&K@nbDZ}6|CVl$X|_bk)v1YlxHA6BX4{zze^Rnk?~c5y_y*{Z-9+@)RoyN z8oSS1hOUbr2FGFaFUW&YDNM^cV*_+kT|sJGQrHm!sDV`mTroUBG z4CQ)UU`*v0a&eIrM6v_(67#G^0Y=jwKABXI!xGQM{lN70f1zusbBXs;Jm*ig{sWqu zR>g4>#YJ{&&;qrd$t5kLCoyWtPj(p^pk8~^!EwZLIZ0FUgGHIf+P-7~M(XT4(N^g5 zKj)mE4u9pF)8DVWrfhPZ<2Twx^o#%XIA5Xg-Z#I>UM#)1kzwuWuO`e=*(Z@x+Y39c zUH?mxnFuvO0aQOwUahZp9W%GCC-6(;<-X+=TD&_px3__HKT8c zg+1aN#xXu*!h+Sp%%AbdBOjsSyuzcJ(66$47-$~-&cf16iiIwZGzdl`Z$tCIHi!V;+op)Zk z?z(u52#<;0dyl)N1Ma_fy8qty^v5rGPZ=L4V4?+?fc{9Py>5svII|kgzjEtACcPqN zWe4OMtbgbEyenp%M?1ENci7PiF_bECXn%|VI$t|(H0`Khg+-$427P39@z3Q=m8w0L z1OC{t;`NQ0%Ps+&yEF3QCTgXhH*UL3ue!tQ4_mWQ{)wz2DB;q&l|I~<4Mc!yaP!-tRmOuRw)ADiV=^*_uwZ2Y^)tb!mM znK0uM%>z^2kUOwwd2Njw%+gqj_cp|+=NcoArCO|++5bb=P}nqj(3(%bGsYPD^~QAMGY?5``?r_c4gWk7pBurrfh8Wp4k2L~2c`Zu zU3*13_gg2}xqgWA{906Y;ig-r`yYIit>oe9mKfOr%6I4^0nB?FC#YqNNrS{o8T-Re z1aR&MpEY%im*b7ozmg#2t@lQXjdcYa6GAjZuHuChD2!mA&fn(l03LLzdSG zwf)}DuL<&)R`Cm}cAu%dr$ugDPnaNnXw`EYQHng0LG(^Qo5qZZEmq?$oHgGkLV1jJ zrQJVZw`I$gr`y#dhG%Ba+Q7vIS%k%c^F-C@HMRoZ2DkGaT zx#PB5rR}%d+9ynSERP8=9vx%?h1F$D*fD9wgqd%iXZiLxCKGyGH^w{HEbcm$aG83> zpw$^he;AbG4ac-ruQ;`t(8O2dv&~! z-^*{kZCU!;KdwqwUwea3v@JJZ@*~}Riv=E!+F~=803V_klb+x{`N?~i znJlw9;a7WqnfaVDv=rT!!(wMhmnQ0c<>6NXrNTT6$<8RFX1% z=R z_!n$M>kwK80jaKw9q6eWdAT-ZO=IXkjmH11LZ{QFnf}sY3PPR31qX`QX?i0k*r-K- zDfPm09)L-$l0?;jRjhXLA6P&0_K7p&SJNAR;-3p+9-}$-_!I04mY;PAayN8*D($ZL z$JOaE&p0}re)5OY?swbCAFJWrhdq;QJBt2clc-&f6&g1>Y$2ahMYtf$953bRh37ng zWDF5{?;HNTi>G65dDAgke`_Ws%~lPL!nJb-I8USeDvNb)*uMOoGtv>SeygvNn2Pp0 z%ia<6*k`;no%->2+vLF3{;s8Xge*%tivHt!{leO@g0E_~_*EYgWH1W6cVCdN9U|H! z`9IU>t2Id%J$mikJEe7c{SCy0B!&eY2S=CbH}izdZTDqnW%@jMdEZ zZ8D3=liOF=SK4jG5))<1?0s4AV8V>^(c+yKrCsi{y~hdpv_~M&KJqa+#-x|GV-q^` z4<>YclEo%tnB-w)6370CCM(N~o|Q?SoDZv$LVllQK_`MOC)!dHF4#~1kTqlso3u>Y zx^NAB0_wPh;RD$?@|DMt@POoWCbivH9@iTq^w*z>$oS#SJ`!Lnz8E{PH}+hyl2v^3 z?R{q}-FJL2fQil9?RbUx6`w-SsxT(R;!)jb3rV^5@G;GBe8oz!_&ZvBN~E z@t}O6y-SVBS9~~PkATnd_Z~B8Cwn;RyYcAX)*c;`&4I%E)tJzvnJeL@G6g+gtQn$+ zkugV8x1v^Tc5Z`>093Gn=0fjO@Qq@J;Fd}}7_|%st}JWXqRre~h8+70XNDYmV=j{) z)#CM_H|a*vpMdtDC$N1Bz62v1y0yR0w3Z2Jvp88SYLZVKm6JDGA|T`w=!CmWJu23G zB|Fe@(=}wslpFnTM(vD$9MsqgpJ!7|7oUedCUgno;@<_CDLahIrZ9|J{S7yzgP-^6bjB$kN!#CX8w=^MPXHMS4Mx|$={LE4 zWge@~7*z06Kg;yJ+|}gK{Dl1v6HB39;4@_3`fNz$a+h(Nx()CxNLjxZ9(m?Tr{~>D z-~Hj)>G`kx5Bs>4m`+Wjo`2yb>5%8WCVlsFAM!5%&Ke1w!*+r9*FSVZ`#pBrdYvhF z%WW&tpRT;IZ4Uw1Fts44;h4YaN*V8PN$DQpf-rL&rb@l=oClyUmn5nN%zXV?qs*V7 zIC|>duGP6Qu4`S=^fH6$w|zONr*Dx*uXW>(#lB8#IzP4r~-!%{4{CCqzr?2XoV>PniPwFy>7S$Mj}ENTOZMFa za5g5lc$AaL&J`=>`}^y7^cLTem>BgR=FG>Tiv$XJnaJcL5?W!V)nyLNkF>{1LHRHq z#SEXenqiAc(7$9jN%({&&B~xrAaFw$VG_Xr4I@~~;fk$i6 zLT=AVBG2io-9Y|EuMqI6w1W=0J=Y^UlkzUdnmo})mZwX9=|p93!FOv`t+dIqBBL$p zz-oI`;x(drvsN=?DVnM)ba0n!t-q?`@S`zB4wuCUBonk7{omjJ*r{=d*=oA@+#DQo zXTPI^>w*rsFKq1p^t81c|MG(%_O}z;>Xj?J?a_AXqpjcYA^wY6!hs~RL342siU@)# z)%;c5#D&!jnuwiY+wJ08jbGDm{ExA-)96Q10(@q^LN{`a@(xkta* zZ^+fHoP3JzetXd$?Ge(~r&B-nuC$mqcO^|&=?Wxu=^xqb7ub88)>T@uG9D#s+CwfM z$4pScoP`dp5ActaG6G#Y07S3>D6 zI9C1PANq%_k=@%gND>}&nf*&(SFEnTq}P;Ioi2HdSkmt!7prW zE6&VkOn@@E$;6heD>J{@DnWZRoJmhs_41hb?Usjs@ipTY^duk4j3srMbkocHeFD_- zG{hV{+J^r;1->w);#aYAc{FDdaLeQS2l0cZq!%uj>vM*u ze{f;@r@vm2UihzXZ&<@(x}*#2MZ^F4whyKc{Ksqj#;9{8j%gvi1&6WUXv&L^;2?g3 z%Ed=;w6=3uM?6+}LsXyABpS1U{6!M1L|3*T#U<|K5$7K3_G1xWf_K@PgaEHi`)M)hW|Y{5bN#yLUl zY_rVtJ;i7v$Baax-$-ALIq0rD)E3^aymfFmi3u*7wX*k|*+j`|v!%^nF;NneV#z1U z;LB=ICd)YCF#$u0`&ald@fB@@^Qhh&drXgs9rX==WY6j7D|W?a?Z!r*2_EpQ(^&Nz ztNy4TZL}?OicliVf+lim{>VP8JPg}5#5fBq>-ofWvmipGK`IDbo|Cd8f~?cp7Dq%d99s@ z)zBjHq;s@0X8nUzwp;ikcm$q{#{?!eY39mh4NY+pdmfQgu0o;=CMWE>Q_X-;&kWbl z;{tCba=IeTvOQ15a!AY1gQ?(`rS=^04hXAg8%PR$N_iqbcw|mStK6WEdd{HFZZ)&E z)T2G~s@ls9HVn|76=^^c?Whfog-0nlok?=FFnCGgux`n}#Eq9#blG6wTg!=-INIoc zS+n&AwyD!}@wvxTj!pT$jjxUV55I;T)@J;N&DHbcC{~);NoHN67`~(F<2zSx2>XfRW3e)0|hX#^WCx_ zRZ4sa7E^|#zTVUI2~c=B7@~U57t!0qM{5JS&cH9ZlM2sf>#4g|EiviBBny)yHZ$fi zGHdP}{edmH|9^CqRE}A^*~L^q{k&wV}T^<8mXrHgdyu(YEliJZg6DCpKx;&Cka= zvSkJgfx*Hcw$Cf2fKF#t>sa|DCuHc#Sbd<=8$VFX@ndhvmj~t+ozC!vKJbtss$@z( zP}@L~2c_ZVKoZ&%?KRHzDjhrV7VOrac1#D_%QC`%WskAa7+s+W4z>Q|TkW~3Y(0#2O3D@7 z=>N%!e;)&FY!Jlej1}=^t*Jh4MEu7%YbRsIKO6hI#@jN)T)(j2%o+bpAL|^`|NN&O zKeeHsKe@)Rakt(qBZQC8Y;mc?AVyg`n5iV7$}$_z?1lHrgIM{3b$jG1uLr%K^Q{{( zS;h^^iC_3eI`_QaK{;&`>b~x+A50H^z}{)=%{RAqqu52a{rUBn>?A1L07(Pv6+{E} z;tg5Nzph%hM|oLaxS;CWF=K#_0~Uk2Ayn~UYiXNaxbmzHW2PPzI{sr{NUYeJ zt_j~CeaySletX|D?RdxSY!wFa5a&bJNqP%T6-ALpG6^DE>C<6krLO9TzT_udP_#f& zUB505svtGmW~djxqI^1`I)@=M_*jzeX$7fU;_&0RA77QJPEF6d$H>#fmiUygeL^ba z7f)o-gpK%<7ZW!jmuOX9w-0@pYI@ZZgto-T%CDxc+bd-eK3aQq{XzWeaeuuItBiIEP}dKlb)bK@U{fwfHWlnIAogl)*3c;{ zL&Z@J)rc)S`O23~y$0#@RkWeJhNwlV_B`0Uu^comV^QDf?H6Dh1FpLEI{RAY$D4R> znNBYZm9D({y7Z0@o|xYGrlU^_$XC|BRRKH7ehuG*Q(ne@u@5LnvZWaVa=a zao2Z!^w9Vv{TitLC7U8_iOa3PUdd=V$E{wqCjIq_EA1l}Cr#^ECzfu$<+gOpJO4X< z^8NqMR&nIXVOy0kr4!#0lJN7}Ae|&T!F_FCIaWkLkIYuRn^}@b7)E2@fAf zV4^muBitc{MIMJvP)IDg4A+(ZAt%!G*U!b3$U3Ox?b6T7r8AX;qcaJDwWB}&Fee)O z46P;Je>Q3|(k3<_wCb`+Q_pqEN5Tbb{XC!cRa>}0PZWS>f`%?iJ7kHhaqTBOX{HNC z!B*%{qz+WsT=eb?Q!-#Ssuf~fQgoiS917KGHEylr^mI@)ka@JUH6|nXmKSPt=$F)t z;2`~7L+97Y(^s0g&gkAwgRRNW?mNsvlr>#8?3HDtOd3H>pp=0sys>XpPUYYkjg@%h zbnd0MVB30cdqRh(qL(k_FPF~>8~HM>a_px%tt72QC`0C!p0+}kmcGO&lhs0Z@eq}4 ztK$-iwcu#_*LYu^97Pe$KW9`w_>|0DfOi+^;QIjQqp&(IY84!_xFRZ>+SJ(yy?O-n_v+a&gLF{@#)a;!l-jfpp&=5MARwSd%^LLw~<6 zmw$Q85XqscF!WNgP0bM2U zPhQ}MPUGxCr?T5{(rAXvss0Ya{1jT{qzhUhA*KRvp;PNGV6OiZ-Yj~RH)^V zMZ!&Tan3Oi9Q;^?xq|Bugd6wn(NS#WwhtR)ELNDUUb8Cw_V<5G-~9ehW|aNj@u5$r zhum*3e^K@vn>ZuQV#S%E#8@5VdVG8l#Tso&J;gH#a! z^GuoLspO??zwgNH4V15Hx4!n;sDIdVCi`CzRfWQY4kt8eM(i?E8o?K{r^4k3x?;q5 zbVcDg?t2Pfmeu^J;2o!{X= zF6BxoTHN1_gCFuc;zCd9m<84eiE6@uBvVN57viu#KT3;X&6>a#lcj%fk9}$Tnm-iN z*Z+-&UD4^{bF+9j#^BnD|J;Csnd|@V@uSpRct8E=3;041disL*Uq4{irl~M)w0)Hr zD&r?6Vq0ljte^g(Xn8g{LNZjmatL0GhYe0%*e}NHs6*TEFJ*ATyo>y{ydIy%JN|dK z5jiz~a;Jj+&Gl_5_%y}wnPNZ?RTMQ5DfHdQ38*;97xK{Cja!^vBbFla+FmH$s-KBr z~mk6F)MFMEkF0w)6%P7^sMwxci5(R-%VK__B9*j z%r5)7J>^9ssw%i2ysw9cqu&?)EUM5%_>I2bhkk(nts-oyq&z_hS&_|1RAPaHaGc|hKS#E-fwR;)_zKjHI1FoX8z{`!J+_Aky)58CfOY4wQS-xudc^krGq3uH6y0;9!h@DL285?({1w4ezc!Et?Yf$$m`B;_&t*jgwIK8m3 zaPhF9C|7<7*UB^>pW&J5oxkBO6AgG!wfvpl^7_%Mo}gp}lAM2ro}soRFPR0ddV*c` zEBP8*h{s0#r=0)Y?~}Zn(4<+!T!fecyUYWxFGR>IQ&-2xokTYVb$MA=c-eN{UR{5z zxmtDYdVq9ISX+NjG_uY{wuSJxlJzb-G^0k8R~Z9&nfp7a0)-r8=u z_#BB_`CA{Ax{jv5Y$ltOP-Uu5JIWJ(<(2hHTy@D4Es!K$q8V$sGbwg8Kk*e*o2pW` zr+mShUrk?@gY)Mphmn6hldNhS5dQ{Z57Ce*9LS+WJU0W$sHtE5;2M5G1oYDe z!%{zvN7JA3UI`vbgX)j9Tzmyip?_Oq9oue=W2ui&eKRf$OOgMw#k$)2psSFRp)JRJ;fG=-bw&dSO@l(aR8rRc5R0 z<3YFIz9OCS_3t(PH#kkg{@zbYuYcSgV`FsN<>|QNKRdQMqfn#Um%n{RddYL1Y<{rE zd`9Nj>&xxMJx z(&>!hM>hEZNRkDBr0^Af!DqIj(pFRd<%++jZ_Th(X0EL5KK!XKrTzB0N1AJ29BjWX z7vm51nmWnO-}cwtm%@jQoYDDw3?lxESqyIT(Hao zukAeyf9ffTuORuoaOKtQiI3o*a`=cwb!w=#$QLeH=7H;?P$zz4j!}A!174QL0et9k zz{FCoafWgeAj&iDd6rY_G4!|rpIjf-@F_QyIZ>wTmq)&A*-tKLMoY4*Ahf-3<<;%g zya$!TM>KU^9jh&lg$tH>;Nk^Tneseiz)pMx#Y5GUFDTw>PjoX1VoEn>*fbx+MYh>l zx!#A2GycRc%+vM{`@GUH=6mVfFu z&S+jEGO2c)0u#h%{Le4aHP_#m7S5k%acdL9`7zC~gW_mXwH`OKh-3AekPWx_*px-Z zm-N%dzc@4doQ5O6C>Je^Jat0FM#Hb7?<$Yr$-D$Qd3LrVn&}DSOoC71FSO_!adafe zD|^88>YkDZ^%?oFYdZqwE*Z>i@3k|uvh0&)e6C`^`B@$Db5|P+x!l{5U)6p=U$+;s zDVDOl9rTT8hD7ZWZqz?p6gOI$k^T?WhS-;5gYe$fCNyd0orEAI(Cv>@cbv%deMW;; z9j7hH81kwwk|n%!Y}uY&Kzipf=!uRB?gMnl1CypBgEJU7;~pKYtNkJPmGb0Hul8J~ zEE>4y+i{e)1*`03zM9F6-`BimBWquKxgHWa>XduTR`S!AINOIoI}O}PWa~GM&P(KnRC-9~}4T}E_)&IEzY3vzK z{5N9X;7AHNEL{c$=D9_NY zc$;qH{|t$L_eoC&PVG#MQvq}A1M>AkDSa+&dq^JJEQp;f&VMIUF zBq%2{@eZp$UUqrfW7l2$IuC=%JlNIFKGs||cx5jqR320rZ{87SjFi_)X1P%i-*Bxt ze*gDsWdqCe_(f+S0o{vcU*%$Rf6uPvt+Y75-L93s^sOJP6=O|>31TL|PyOci)61U! zG!O7O*%rJwd%S7ZCcCG3+uk*eEZ#iLnq&QU#Z77D4Hu^6f4iu$^LTkjpL4G{3-!Z* zF(x9ppYIJiZmqW3$kPFxQ|14b`!G^=@*AecOrlZG0Sa&W7*LUKa{I!w z&aOLo8}&B>;gj$Ld-)La%t-&E*d(VxS3;vU=SH`{j2YqxWZ`~sVYS^UM(CZ%j%!DC zH2?uAL%NU0x@WRehw2S`^A|0F(J3#?n*pV0E5}u|=d!dsS{b_eja7fO!%c9^_;cE! ztuTP_)_zg0i~m-ZkPo`Jl$5+<<5%At!%GuvNEBNPeBiY`$Ev@v=uDxu)DKHop0sH( z<#a-q$!$=w6-~Fi>(c-7VRo%gd`t_Un^B3P*aDYzH~PPGJXsg<&&iD&36ifmN#JAa z=XF0?YvV@ldhiET|K!U^;9W_X8G0SViyoY-D5n!rEp#b4i^j^P*!W*c>G(9$zf1lp zo)J+PF;YgA9H(WJz zUZY6cnm=}Qnq1J1n3Pzm2&zN(t=fcK?Fix&kBux(Z~s%Xe~R-*mZv}2v?$RC z4;1dPu8ZD63)xW61Ki57H;*F!@v^_BOaJ`0qQ>N*Q)Pnq`#(81z4Up9HHvARX)=aH zMs|K~+HBAL)4WCVd(=4C@@`4D{^}#?#*5DHQ9d9G-<9K56YG`_o_(vy==C*0JwS^M ztJqz}IEi}#x&i2H8srSku4|y&DHk_`C-q1bfyOSytylv+RrECEW49U>onXa4vWvfy z6?H4}(A9`UAI)#nzn?#4O2OJ))6xG5rOAX7nl$syU=|i^1e4VX*qkX?2X@`@yvNF0 z)HPf)%6%W#q_?2z zi)PN^g-&^!EkR#kocL%C3}Z5-z;lE3q*iXG0bHDjhD_&E@Y6+~4^~aB7QsIJ!l!{| zmEp_%lfkpo?8S?`jJk;9B8^l3td21;4BG)o!hs~(QXYpH*RKoH$^ZJ*aAea(i4Uev zH=nyZ*jZ})XZ~U<85}ncqB-`nPX<2cj z&m>HYUiBl*ITVZG&gl4cTl6-zQCH_$;o=9a<6g(ATJwyiKvI3J;ssW?$|lPg2mX4+ zmFfTfvUVmCLIT^{0HQv-=JD#kzG;)RXzy3s znfdMM`im~=OVzW$d=mZ2{OYT`GnIE4`cE9Ye+^YUd#}RhA|KNK-p{R$>Ip{MHd>kt zr0Dv@lpi*7SINJ>kH9sVoyN2*<#~=)&9bPB$=~QWoQ?+82*L z5)`tDuV7un^|Q<}1e6D@bG+366fg_{@Yv|3Zk`Y=tT~K8}m4m5#p#IR(hCwD#Hr?Yc>9*^arsaRX#wSwd z*ah^6m5*>UY+Ru$`P1%dF1BSCD=(8)%{eR!GTz9Uk9{EXc4);O8MG<4RdzaEWxta9 zP4BFH(%y1OT70j&+Rb^;NiOP?@(tAA>ThWNM_+C3^WAq&w_SJBwDGx5v;+{LoLutoSPV4UlS_f`307ia%sZLO#*{IH%Uh?m=kFMeqgXAil* zozv|{^bz~;1u+`YT*x^jy6XCi)2)~NGTnaFA8ew1eww@E!_tyH?{D`NkrwRz%rx(! zH>KN$94vkK}%l0G|2xozv$~RUcyu3VnBL)jFpYGd>rW4dgPBL-{>{hUny5h zn!lQ=T65Oc&hTwk>ovvLyzCh5#T^Ds{#sntK2)&rpfn^iOhIpvv3;>i(Id$xy z4eTg~N3TSGcKRIYf$QQZ_(U5);Qb%|9rVEi{$vMzuhw!~S;pS<#sLP-DWi7eS>zfr zNPoVL4V9pi)xH{doY(<(oVm}Mm3AVKlWZ5ZFMTH*n9}e6aM?OB)%urJ@{wIBVf>~AjD z7_C>!J=Kmm&o}4##sqA&79;Gv)CmFFG^XI13wlZXuy$fN@5<8o2wn>^hZ(dr&a`Cw%X_JKu zO>RE>&ab?oq1#4ET2q7c-w^%J>N4VK#9qdV&*#tU?ekgY^I7Be`EY(NshCJzJ+Jk7 z!snYWw$Gl=O^X(9()!$dkI#9;PHDXHzajdcu0j{_Bs{uLcYiS=^qi)ZkgZn-7B zv7 znFTIXaH^00$nLR_1qMiGR1@wxlL38(08hCtxKP1$rGE%AwfDMw7VIxA)@vtY@~O?q zOPoH)c~AYa=BIwHI`~<6(5kM^oppUY9cAJDx<2&$pho-j>$UZcr*65{Zw}|FKmO_R zbf0_N&Ge2HIHfJ+k1`L52m330DSyasg77Hu!Te~mts@*5ywpqMcqZ*9bAlQ>V>6zX zz~8stwk-YSihs-`bM@8bQj2pw3c#eey%5D?xG!;^f>l@l!ryY*v-qa(r)A3@m-M=` z9GX2zduPU2S@#8vb7g-i>nmR{>PH`iF6wI{E^3N4%A@b7uLecuzNC+nrtufjNxDHE z$=8Ayp~SH;$L%Q9nS(9W&&#?E4P#3D)$T z+EUt?vr+%)?SI9=MB#)c&AbCy1PUUI0#eY!&S^qazmR}V1d(0rDNwLx-o}+SNS&p< za(wRC3R-$m0ZC=klD*+?g~|)+&b`X|(TlfWtv`*ENBucA3Bg3fh&^hs+HNqJNaLii zV)ISYDx1K#-AA7cz0PJHn)i z#G5dF`*1$Mx0_#EG0+f|j%SDVlIB3k2fTdqIHd~P$T zY1Zxb-VJ*+Zx)+fNV576qqBm@ZR(JQh!=bB4V%#|w{4kC#GZS>Md=!wl)GWsvUHtI zv|McyFxT92YkK+f4@>*(vWv+S6S@4dIgd%Q!sk9==KhYjjTlCw=*P&541OG7qyMuV zCKUf>RJX19kY52_|Mm~3i!c34=G=nKZ8hjq9`~@c%KH8-=DRgEVMV{4Z^yGOUPf#J zZ5Au=GNYk6F?rI2*eQ#US@y%jGNl{3sW zggbv@(eTfBX$sK-2hk%9Nzi3=o;g3`{rK1VczedJk zU)fsawY|eg;S2EI1+dplYFtru*k=y3AZHa!Sz!-BqI)_q8HW8#!i8EmaPbQH5 z_V<7IWbsgpr#aD|F2BOB_n9oTi$#mcu*0OOHL_>2-)x)Iee2p2R85T}x z(u|NA6quc17h)-oBjJK@3_M&Tye)6*P(KVGQ2dk^Y$m?W5RThV~ewVQAF%(ENcP3-Y1 za9&W#WXgP-sGDUp@ExoE*i^PC@iuSf`CmRa({=fLeh;5}=QiUI1DyM~fMNoS1dgqs z@nirj{17V;{5>ExNy2+?Zn^bVi>H<8_UpDu-#q76QK#5{`qb~F{r1`;ZEC9=7v+_N ztTbEP!{@|H>2ug@^nYuA?9uvyqc|>MKRkQWire^)I9OwIrmNHEzxus&$~VuTdPicC z>P64^=d{y5-ND9>xxS)n!2+8&r$6H_PsY>ZhKbZT=_{VRjkfU)-W=G_;5uo{V62#D zlV8LEZR3B99po`lHkZebEv}gu-o#d)5l_g(7~*vrxq8{B)t~l5H06<_yucZO!sbX4 zWy<4tqVYMX{-AQ1R!}kmNy-;2{oj^PJ^fwy8M2E$VEw}47u?5de3NltuQaHy=J%A3Y-lkExq|xpEzM61v_9H`AKk)KT3OIx0-%Xd2Q>ueqCN4 zL$3s`V?mW|2oRlR?^<7#v8s|EjIX-(hMHu?9$$O?O@7^v?ki{6YH@p`e_Ulxj>-$} z_r%oP^sT;fY>#Flf}QZx-vm9|`eO;@!+x+WZOR!)2R8yH)a4ftj&+<*S{=@R;To!U>jSQ^O7q@!| zoT(Z+L|td$yDt4KS9uT!A67o1cFnctkJW;D@j@{2jP<9OEQBDU%3m{3jG!rkupm9>)DPItU{LxTl zNVs6{tnAPo{n4yFY9j&lnS2k9p|* z)8_VQ9f8Y@E0096T8vdfw1<8KFahH?z)Y0c%`bsN5c^HGl|>z^XLWt0nek@D3gz=` zRa)Hi!iUCXC9s?Z>L0XT-8r~;ZGG|S1F1(_qt$O@fi@GbU;lG~=yRn$i7B7Y%Rc98 zXv=+~G<>c{aG2PjQ?Je!kKihEJ78bSoi&Zc>Vdcl?#sFUi8QG+PD2D(s!k2J^3-|-h15DC(IVO z-{$#B!8kcsTQ=9~zg6c87j8m44~S9Xzu}8_Ynu?eI@$OHJ1fQ*KNvf#kVm$G9J3;I zgh?|SgE(Gjk9)_8)tE4g^CfLW%I<06s1c+OLOs=iL^x2X1J4B0(f`t?gi}rbQnsmv zdwdr{wc}$RCv%yf8!aNczsT0+K}rmH-^c+b95_tMq1eW{^Rc>_Ul;U7GiRgmb_#fl ztr%YG$<8BFA>G3JnM`-aE>5OydmGajWAp_*@*Ouws{0D{Wy2^B+k2j6tZn~?M6}!{ zk1513*&>02{K}b0P<@i~1vro-T;=3(-OKgPI*D@{oC1kQQPI1!!(@pyXZJUzE$+Nk z!*2O+zhGat9$)I^uaeUs{|+iId4UgVuco&O9a*xcU`OFJJ1A{8oIcp1y>4$T6T@FK z-1wMklQzwGXmvb~fzy)ke&w1}A%00$UVU9U%NBY3u&V2Uc1ef zEKYaZ>7UZ>cfGT{*YDoxA@{pa+Hw2s7}kB9r;f%8$4w$#uW~HB)s(z82^X~LQ6583 z=r;lV#aFQA?~>%Uz`N|3+xCewiCxeuOR%fHX`?*RG_qKRaKTzWuPYg>|M;JsW4RAK z|H41o7mR}Na>*u z*eC69hi&ZLXfbQgK^*oy_gN|b+$YN!g zJvM4|Ovb4$G|ty{vJuA{{on0>&8%ZzQt5M6;V^5zz&0)rb)m>f-`{U%gBE!!a~Za# zv-=~vFtZF!R^g583$JVZY()Z-7fi(7F#G28w1Xd!F0zR;e&s3Cx6e2`J@JtbN(b0i zo95XB2=B35V5^ppm&fdw=m3|`%eu6?(f@51pRD-D|BL}Q-h4}X{;S`XmhpIYmG-~y zz0$#tdWgS!Zi!8n&u68XZF8;9Ydj-16jPdnC4O2wn36{-{ptR;4Dx2%WEfqZJjMma zk$~|iX2gh%8LTF=1LB%-g)xJ1hE-<44{i}Nj#N0M9r0ubw4247_Tzt8dFJRZA5KL` zy!nMuhW%uuuXukU&WOGU(TGIH!)wYGBo|yh^&l;SRXC7jL9BPC`e82D`&0eUxHaS% z59w^rM7RC%XeNESnJjYVYiYOX>M+ydv#ktB?n!%YXju^w}S-ov$1+UaekZE3f(U z34gneGDGPxE@Y!jHjLwtSpJc(`ufib;Ia1nA2*Xoh3q~ay3AY`NJ>geoa3pT=wmv6 zNK)n1`CR?-H)w(qe>VxK7d(m>}lb4BNC55FU{eR)1voe@)x$y<5X)&Gl!d z>+Qn;V@~D|A9v_`CZ~5eV^a66=tkb5ya{~SjQKTyaIZ=%S0#5B}$uzP;~bU+}q&hd=nf>0j)<%PjC&U~?7jKgJOn z&8ftdCpgHSuu+V9o$et(tfFl*P}yFIH-tTq2~V0yT|wF?@hXn2v@-`mWsI_J!^6lO zFAY%Oq7y86fOobQyp%_Ioy13w@)92_z1nI;jvU?}#$?K@S*z`)(q5u%7lLy~;-WYv zPvU0RCr*3r7r$LM?;jC3>`4z#+ikt2uN303x|o2A_oKyync2aDlgBU7g_~GT zcD`y1>zhbSif9!H(CNv`bTq6~l^ub4TpW z?Us1lhGSM1`D!3DD3fUNFVsb z7fW)-k3Igx^pKP9ljhrC=94cr=|Sq{b1%p{`CX9g65r_mum^UC)M=4A{_%ChfxU~) ze#v>+G4Jd7!tWjKuyuOn5l>H>+md$twZz_s$K&4f>8`)*Fcl$j zO{QXva##uhi}r#w#jUw)Cvb6f2VCclRU5 zp4-AM|5AP6Vh|7G04yOGQA>73B3w59awrBn8v8`^O>3-36NEh57C(T^XI{d%<* zv&qhz0u#in+V)9dJ7;4@dV2Ks@hiJvpSPwhcDqxR`w_-$?r2wZ}-%s>fZ`k_6gyF z=nh`?p|)lxHlax~6qHLMJUW?kq2LPw^*TM_5f4h+Y`M8FedOI^ zeBp9rghwfPrr44RY*SxhlWgR;@#S$U+uUAEq{W2~k7ts{Nr)h_{|IPUaqLW;tz(1i zCA|ep{b2yV@R74KTzPRk);$u+=hPJrRBExdE$KC={g8M3`nwz$)#SmRwvNjs^9fCS zZAYvyLXK5-QXo-B?*(Wl%_83kiXHmOt{;aU~%C~oUKd~W90c5;V|l`UT}w#uz-E_hjbiis`vkyqW?HtvtFv`A_!RG zy@VkaNrF8m!OC)Qi*-dbzKxMgIc-mZFYPamWk*EW2}q2X$?u&#(Qo z(tbVeT$$Xn&AAKH(yO~y!S|#cN9H2?!g#Ge){g6!-u9MnI_=x`Ko`U8t0YWtZTP=w z%g4xQN%E;|y1^zsbF89|bP|+KXv~Jk1S#{rnd@_IYBu{Zd7+)-)=gUf6U2+`9iaMw zRCdk7#W+l0D?uLly0ppG_Eq$pR{j0-bj=SxYuA|*OA)!@LpUKTzIH!yoY@PiOz|+9 z|8vz2&{dxpqaA}J;gZWUi(MmEma{0D5omec<9Hb~;5ruUr8TysJlnLv`_@^L{&e|2 z(lPHjA$=`hr|z}aUg?MS~+=395cwK26`UeX|xi?U7^{BLCMP(iJ z1*uofd?e$T^GP5Sb8f1KX@-jAg}|MegKq{y?M{p|Fd=R7AJc;JD}96(iV z)WgM#7pF%&;t{?bcieG48G6!5C#8>n{Nw2zAO1}G(21v}mp=C?=~YKQ-530H#oJ`* zpQY3oz+pXJDKC?b3MMpZ7MFFB`xgnSw{=VxViu z9z{8hGgU{M%7af=`$nm}p5+7NqmGisCT=!K{{At8ICRdM*>T~pYL30|cSZWhXTIt* z3dE%juKhP%4HnSV(O>DXALVK~|Tz?cwyj-#D13zIE{@~d2GE>CRvC5JKm4ibh>I?@cZGDmNc!U^e7v0H8_X= zqtA1^(f^Y({%PeGe&wS1-19C>|Ng#DVB)TH)N`Jk_PEg#)Qr22c-DB)Y0_HAY_O?GCY%NF7X+q0}&dTUp zPU}oEo(J{tRq&=5KPr!7s~o<~^vZKcaYh8|NgK)L{n}*Na*HLZy7u7G-=rrk*(d#Q z?X@rz=dHKey!BC2Lzc==z5`nF8*0OKW9@lf5vA`^fE0->}WkVFo$Z@btvGN$Y=tm><4N{jmck8Ecxg%^_*4gAPt} z<9Ao9eoMOLXYWckU3S@6Syl8tT3ViaD?>z%vCaKvx#~C36EllK5ve3tk9D2gGr!zM za6=AJkt19jbDJ4RF2T;y>2>6`wlPBx>9K~OaL*XaPUDcEGi*JmF$>7+LOyunm(qJa z`uTL*vgK+2{r69AdefWIqaXd~v9vglmWdZ8ZrFbM)1RidzV)r?yWjn8+W(LjrWZZ) zpVRA)dX{}eV6pRI!3XDC4+Y6i%$53#)}rVH#iL&x@m5?@PkCNPwi9%Q{kV1KOAbM= zUh=A9zB+8qWo3Qip|P2J4T>(g>@Vrn$Np#f(a(Nuk4-K}FMs*V(`#S*+QbU8$xOTN zzI%H6+uxp!Ip&!3v5$Q$z4M*#Oz-%w&!jJW;|J;8Z#pVH@*($&g*GI7zEUph8Z8mKWT=nwyA>(BELJO(ZdGQ{DL4#ChGzi$w%Ko(?F9TMIZel;w z`o|p>%(GX%+A*tB>G`jDyDycFR@yyF)Y(`)IM3dLIM*ga!C3+3OPq-wKZ!?R%$gmN z<<~$EI3c^s=??aolRi4!1sU-@~aC5~#}SBJL{2r|!QT0Qg-pdhM$J_ux2XvPxatjwYGn z7?``TTPNb)boG9bOJp{nr7%g*a2$3D$Z=!lOm75H2}rRq=(KHH#R)2U8g#k6VR*#G zA={mH8169%=RMw?J*7Xnm(Jr8gcl3D8L|!B7MKHDhk3nRK8aSWz4Q!Zoi11F z95MMJo$FIpA{}n;+2x~Rcf8{r(~0)@J}b-Cc6#6gADF)Lo$n+b^)$UB=lHfAv4Qude8I!_GtL+jtApLh#GP~vrkuJ%k&<<+!)EN1{oo&IPQ%?e9z zL9bFCI1>{w)|v7&+ZXJeWlWf{fZ_>HctZN%2S1p0+G(dCTub|f_B7nlM<1PD@PZel zV~;&Hz4yKEO;3H%o6?cbcw&0%YmW4fhY`PVkBj^h(7({3yHB)*!}KXpH@`zDW5Q5A z0@~)nWLV#lxs(Oe^y|XneC>13}2o&~eYnG~mG_9Fs3nEe~0H$?LTX zc9~lXR0g!I9u zWkT>AJHH;Su*q5`Y+0e?tJ-oryPSnTBzFTJn!=s6J<=r zxDYW!Tq(8$J>IMilX2~M(efa_-0+R!cUb?HF;U{G?{T#O_#fS8g#OJyu>KLZr->mr zNFobgPe3*<8w9}#!v-4eWf_*Z(|NvKM`TFf2z~*AV_T6&Im!E2mUEp_P7Sgd@Ime^SSA)yHLy12dhV(-nD< z-|c@R=}3xAAX~%quj;pyUwFJ31TX00A6J@S!{Ocz{m zfv<+TtFYt(N83(B}5W#y@H1 ze#lMYx#)6hFBdeoD|v0YlvE8AitJ$d>9wi zyvTixe=(8AGOxY9{c?qPKJ&?swj1Y7Y@%#_Gf5UJO+=C5KzXI>)X_gA_W+X;jZ*S8 zu*4gJ8ri`0=a&~u;2(!zH6ri7vKL76F07Y7|InWA#o<}=-(K`b|FyTYvzvCHJ!=Py z^?1YT|FSQSWHL#4{Krp8r`v@|otP*)=IH18d(3zNIA5RPT`&06C(7iPx}x%dCEn0& z?EhhkjpH8|DXlG<0O>?}#!x58bUSj-+s;lY9UBMawVA?!gSV*LR<|0#55$uFFy26L z=mpEVYRez7hK>tL```U8cMb+Ke9sru_z|hJgKQwEvJQw{%uY(>^+JrhR&0jG*1a|| z$!`mV0|#$Wx2*4i@G8 zx2x=abXEGyS58Y$JL(wwu-MA<=}&*!-!sNnFxN+#Z}+|Lde^(sx4-@Ev~bZT=_PM? zkH5Trm44*!DO9O*iT5P7%+Nnzi4RLp8y=Mgd$m{cEb-u9$1A)d4_ezi&NEcZ3kHjh zJ)zE|;te<5nht%*G3m2kJw5Ha@4o50^Uh0$9CAnp?Yca;``z!J&N=6tbmWmort|HK ztB?4XSELIs`IFxdX%bX=3yw#B^^sn>mA39xUgnKHnwFF`?tW`tm}8|13JMe~poaS> zn#+av9jvHN60C9Y&itC7Ea{y=?9`4RviUC&L)I@?IVwJagZPb0e>%sibj@|wr;mT( zo3#XK`|Y+)k9x>{{!XKLw!GR`8rjSL?b^b&QZK72IRyvF+tpf?AwGhG_>D_{+ARH9 z;^@b5si*R&S0%|QI7r^Ade@=&2oB=6p7r-%{wyYDTUX_U?YuO8)vA^0c6%Sz_S>pDeLh7OLR=ZJGz z!1k(1tp)euOqkpDA42S8zvK4Xrfuw9UsqpyLo_f$_j}!A*SHbn0*8bh9=G!b9>WH3JWxb^A?28!*y$*succ2N3wiO6j~>+H8LB zyWcHcaPj}mc+1^qkGpvkd8B4LG=V*o_E)4EzV__2R!0PNuv(rXKW;*;KcIuB0lmTK z8vl#l>qcM7O!@+~BG{L(C}?FXGN%Pv+A=hjE^|+%Gb#+$c$jMC$gM3Vp=ZT<(0|>l-SRFx_bT(s|)^+4nFwcbn?k3`+Lmlme=($?^65dM?ac)@7b$f^{VukXT2(Y z{S)s<_px)o?A#@j^cL)?e_Km^q|IqDt_LaC^fKyJV;wd0bEdB4|vc5YTZP zt6-oWNf0#Sz!G=5@$AGK$fmY|Y$QSOir!_gyhfgkJKYfc#-P6kw$ZcdWQA=X_~e(; z(wlEZsE%~FO>`{g(JcE6J1b6jw-KvNdag99^_jx@yOO2POT5DZV>zh&xq<2Lztox? z<59a2e|&f0f`#_D!vc?=LmqQLy2q|NcNz5j3;$^GcdExBs~-JD_3n?g7ylcY{}~$i zC1vHx)#cQ!t{qO6m%vfE@2S5Bvne=&&%4_=XXmhNgW0wmPqvUx zWb+crc45U%8<%v#2hppo@cH=sFE^Frb$Ov);-a0da5P=am7ai(TVM+k^<4?mv@-`=X+BJ|?O=;!- zAQBZlG+J!?a+?ROC>ws6-ucY9osB)52uw%sCytv9GVE42tP;1PHhP&U9WP**%jn$9 zL9^O^Jm=c<^R^<12{T>{fA-mDr>QbwCOvoEbyuG_V1qludBb49AcJ4OPblI$l5&bXq~x6Qnrf zZKSk$R@}A)oo(JmhGpAM+k?JC2IL?Kg6H-f8t(FBSmG+DN>j(sQigHr&yDnzS6`Dp z^~G;H_i}e?iFKJv5diS#{b&T zKl&(gYU}~3-wV>RbOG8{_W2Tbx{{W2Up+vumr^-0o5;6I-i!xEU9e&K^=hNcYi~Qj>;l*d zJTO*clZh) zm0RaX*zM1f7JLwWjCMH=+y>F}V+0`HC=1w|U#G=1p(IFYiFZ;@{Uh})V2SsY++5=N zW*EBD$66c4^_CIjYcD2R&OG~<>1A(zf7)V;Ez(L{>1c!Z=PY3W~-tTML3RGf2$m}@Hi{*Y#)z5!xfYG0lf7|;Le8C84@so4UPlq0QsDHhZg%ndUJ@c8*Odt8k zM{EWC_35cEKE_^8W$R|M&STcUkAkKBOS}(xNA?6K&0GazC}=1FMaqHY+In8r+G*XJ zMx=UzR1^;EwUso6ceQ7AJWq8y={>ICY{h{S#G-M8QZo7^l>tjUlXuBux(=y7oz3RU z>?V3udhZFR_@{hpAy{R`OWpZ2HIHh|vkM7c_-^OfER7-4RR7Smv?+MKK_wbPyXj7WQIbVB^A*Nf%ZrE?yiTkhsfEJEZ44^|26kq~*7-NH2f$@oCxe<-P(co-Y@P z{KD~Q{un;~gR^*ve+>tHqQ@n=2N zejMY65Bt%BaopZf`QZKb^mm)8jY)^~`;jKSfzcWArQo^qY^>uo6mv#Qf44$U1D7^0 zakWs}9~$z2B~G6FBIpE`-$3vxm2H-|$efoBGDI6vXw~Amc$$E)VBCDfjd;Hs5j2z? z1ZyR|uAQ8Sb6+8EE3cN=*SsIF_dRRU8Fl>d2kq;t#`Pn(ziP$opjetp%F1`c_!GIu zN-v!wvgmd(8n$C$*t9c-=d#503aFnY?>yDyQwY-`*$AnBaTGzkk5q%Y3PSSc_Mm_#>1<=r23C zo?{H!E^^E8Ja%lfFX*z^fuI$HAInKMiVKOTO9Ga-XSTG|iF-ABV$D|EtyrC|dXbONMjyNK{4pFH?zH^&+kIZd^&(ma=pT0b z|LmOyfM!LN?`v{To*_rcQ5*zBP(<1XKnj!w@Iu_y3(!=id76_jUL5bk9up(AE9jQ@85WsgtU1-MV$_*18Kl z0(<-QCjP4LX=apzsemLBN1{lJct}~lXA8E6!0mM2iC)kEznLGj0#a{{@9!Vun4}SE zXZ3TAQ8wZsLtvEofm7y{_z?#Zf|sbDG^ZCg)Bm{dK`n{>2^lS6$8EPvFW7sACOw$= zh_BwoyeBl`N3r0MdZSLI=69Y4qx^8~A$aGEc<7MxR(aaF*dI1?-Kx@UVbfYAl{zjfssF zTKN(a9o%kMTJ}xQ!Wz^S@u1g)?etYfL~?a)>_#3FM+|zIR2ZkPWgooXUN#_ZD=-iV z<0qtD51NslO6&JZZOW7>Y1XVO(#<#bin>w=*zNEG_f4bu$~Uj7Avem2A5^+imSxFa z`s>bK6c3w}@hVNUEuQNbDEm`agy0Vtp^Qh`P=L)F6L^la;6J29rNqO0+#AZ=!VgUH z+|NP67Y@?|Dk~*cTUbjI#OPrC^`@r1X6)n>g(LJZjKo40t4-!rBpj>-giIL^_JIWQ zB8~HiSNTF{v40bRLPdzW0+Ln63)Cta{XoEIdlkYEh{O$lT+@IACKDnpaO$=Pj(!cD zdPz?A9z*sYhU|XI_8*TG~xG=boFJQ)>bqV zv;2}6C&_|$Cx3K8=ntIGqxJumRuG7bvf+uevZ4yiCFIO(Geay2xL?-SD>GTa zZxg?&EYqVt(hQ0p1mzy%_qY5t{l8CqC2g|FCh4-vF4M!dv6eg75AS>5`_gZJ`&;|i z!)LVb_FGOpQTL$w#Bn$J>?D847;OU>amG-UZIy2J&bo|ponSvE&Eybb-keI5vm4c3 zlPeEKD&{bx{@g)w!JZt$xCWt{Ks{DIbC4aUR;lMSztjOVwGOd%@S!V$BdY!)s1B(!O>$o)u zhM@ed_Q|@7hG9x@+cfoI=j(v|*QEUR6|zy%=ZewxMrVqot2?t34!Pd-QA2)zg=Y7+B>`|NJ7Ur*MmQdX69vBE4*mYW>| z*%;0oN8QnWnhb_;F0RS#D4x(nZ%Q~2@&^rLzgc0Ee?!X5hP)*lZvWX~QKUjse{K2| zPNadZKR@lO{i$qlhiF#;;Wpv{T>@lp_c6vOrLXGwtx`zK6aFf~Bw7?@{7l>+gd;9! zA@mMZXs7SvBI@ieFmc2yT+6k=*H}%WFIzq)op{W_>A!#WE5*wP5#3SfWBTZw(>Yi5 zK2Z_+yzM_S7gTc9W6 zbM-`d?xMx%k$LmeHal#UHq}D|UQHh%3f$NEaI zSNr#1*Jj;A>AbuVi5rsM_zw>ch%GFhZ81X1v`Nwxni!VPu=?c%d+(msn>Hmq{=`$O zN0V!5f*84vI(R?3nH{C;{3v~VX%rJ@Hc70>Vsq5qG`e>4+uKTeUsl;#tVTRwoBfs0 zp9d73v=RMkKyb=zE&AJlCQFR@+L%Vm`dK~=&9%;v=4QD{qwUhpmgx6|uYW&X zcjGPgD&J|Roz_NiFee;+^wH_8v(8HAop)aP$mcFj|L<+Dmu$L*n6!0kD&8EurQtR-t+99j!pkJ+Gry$GX##v&R1G@-+lM=;m>?c z_n`Zx9k<(3;|AH1!9dne=pp0}jk?xDyFCjW?I>VR*!`F^lj$T>n_gds&{w_@DGAgGE{{7i7y}_c z4G;bq^xi(}p7fm`ULBQJaK5hz(bU)Tmn2;TzQxo~hga4iVW;u$|% zPg$N%F|(&*x6~2kl_;8C0l@J6bWqzL9YtGbx$&ub684yG0GeU9owiMJ(T1R`#oLlzfQ^DblO399(_`<7UuyYT|Jroksf^-c`^73 zNatc@>2L^5Sf*JV1UjfdjJS7JL960NXpu&7YvP#%r_qNp;j}^nM|;&?6Hr@hx=}jh z1*>?{gE1!P@t469;Kkbvd{u3ymj?>gMkz{v)*-anzZZHoPa%4F zz5e4TqEK-5zy3O`1lp&IMk-_GbO7JU#^844?HwGedM#3*gj^T z{p@EAcc>jc_`wgR?|tuk>B5Vzus(w^H4|s7FA2#*?ttoqUAKY+2ep!|9kL8h1*}g4 zLHb^%vHTKkvcFW@*(}#$7khaN`J$W1t9ovtb)J)s# zs&nJZtJ9e$*nL%I*vg7d^sgj)n{-;sbf5NxOTVA~^0&Lw>8GEb4m|Kcs0^vV*IGaG zna`x-jyo>B<0BWOZ(sQDraSImdy?OT%z7doIs$sRSY_r&m?N9BSN-&3(hN@U8EVVW zxyxxvv~vgH2<8fK=Pu!#x)&OvbqF|8&Nl%S51~nqGzw6?*Kfs*p7WIJ&-t00-P98| z=c5R`XXq4^5Ir&pO>G+MDiP!dKz`JLOnmX<+|ORB+5Yw?#;b8|yCwzq>iztyhH0@W0ppAzl34pQMwIdy%bf4F4H!|K1mv9#B}b z{kdCP8?3)%vn>7Vd0$GuyXh~biY-2E%B1v;H@-4W)at)U+Pjd~ue!#LGo7a_h1Hg4JqtT8!dmqJJ%C_gn*IP3JYS6>IG(xJ zdg;tQlskOK?bECF+C5F}+9n;Pt#3wXj52Ax_0#xi(^J>v$!YAwNxEs6kVey&X$->+ z#Rw+U#4KNFTcLetmauoGev22(PmAWxO^fC{mF7S3c$)wCV`=v6C(`Vj|C(k^nV7aY z;5ljIU3C*A8W7+sgBH+RIrizqf1#YvXF$-)czf}Ius>GM_HM7h{Ih2sIusfNNll*P zJYcYh?EH~VTa0|WAr(@uBNb4KuVSICKa2MH-L*Zltj~iD%(U^cKW9db8xI zx%1NJKmYNx!3G-)CG}Lic4_>~WXq999+}?vu`i}?Uhr>z&C$eEY=B?NPr)`HttM_= z$xMtre(ytR?t_n}`HwznzeP{Y)1=wTzt_#=+4yOb(}eY>rHLC%OH=ePX!52T=s}RK zg)$2(Y;E;=2YzsGL&`R{wxoEpDKD?TvA3YGfP523ZUjFDS7&kZ$xKYMoPse^tno)~dSxhkH z*=rg!H`La!h0%1xoQZchUvP_D!L_C4JS*>Uxsv0XW92fWSa~H@<%u)t1W}Un5`9aH zs_49PAQC)~L^Un|XWf(!>L3__3@N_y@|(d_HyswsVsHA(U(j%k(i2NpF=)STasUb_2Pbur7{nwWK)azepk^`v+ zm>{3xm4HP{x|Fjbz3mMzOK<(a`L-#j@t6>~zK+{Pf$ECjjUmcKsp8f`2f1j=}mARQz5)WUU(W`qo$AI0akEB##DSdx4;$t^_Go(Io_o@RfBAE|?Z2Rk(^94r2RSY90+L+Dn?=Lc4Y|xOGj7NnFm%%-HQ3L& zzL}b{x{;^+1^>twp^Qgb4ZS?qL~e4&j^^V<qkr|7rtASTH6sCk$&)o0EIoBgIHPy6h#i^kB}T4n??Pug-}002M$Nkl$;#=*OfPq-8*}VTPn| z^UQy~iFPcIJQ#CA29s+2$CEa&8g`aiUnS?HyxA-(#9BW&O>+E$qP>M}Ja zm(A)I_7IdG8IdCNU@|xKxh~s(ZOc!GVUu6Fkc`pH{Wx{ri?vVAGk4fJ%{+Yn^t~Vb z!b}o}r{>H{@A&YA>C*H6)%k47+Z;p;foxcH5_ow%;j@?iy!0VgPTUgPO{5PQ(MY^FAU?Tu*zO z>BeN?+&SsNTW?8EYClMBG+Y#V3i`)AN`^Uyy!u^>yj5v+nIP z*}+r*5l=nom^580Pkeu3jqS5vwENB=;T)z5+tgf@?N6zVQ!5kgsV>xgtWzD4)~Qs` z1{R_0Q@oF$?{u6puP8F8qtT|k?`Xg259UM0IJeG=_8B+OWJFw$FHEwZ_UFF#UHb^Y zd*1V&N>4S7p%;7ad*5r5W}p1hH`67be4kc8^C7f6?NP{&WcGSwhK$u8xcP5s)-Qga zmM&Oi6Ejmc-7HPrbn`TM<4w|}4K}pNvW^;Q^L=4O8uaDI3!i*4&6)jhn)BcT>52R9 zNsr!fxBVt=I6duj#PieSP5AJRCWLG?R7~H@&Pd<7x9Gy~@N265AA*Cn@Mz z%H}WnA|qgWr@!(R&#@={54Zo;i)HAu_D8h3X8QlI-%z~%8SjJqQ2M&H9Xcxo({~H+ zr1P`ZlOe#{R0Wk(U08{?l;ONlzAYb|GBohp>cbBMhjJcA44#zoHC?)f2^t%)a(L?G zN$EZR=ah7&KIX!zfgo4bxO(E~E~R^Ze}jHE>V{&2G;za?G|{zQ>e7?=uE|r<81-j7 zfhUx9;T(HJ8t1&Oxp>|@y`niMEqL;YG+(RC<~;JSLGlTU^dxDjU?1(uOWqckFf}8wNxX zZb&l)D+w?O+m(#gN_h7E8b5AvTCBZ+&V1cV(_239sWt59-O03S>AUZ`y(R$n^A+V? z>|d&ViX}60TOSqfwQA&7b1CdtG8lHvuyJb4xTAOGtAqVp$T!<&lSMX?(gLkxFCHE@w=|QDcy7J&9W;nE+a-A*_p zt@n&gHErf^se}(#<5c6oYlbvox%h@9KIHbDA6=84)FOuuzx%z3S5?=A@UC~g%O=gf z@~t0f57)gk`H9c7WsPgVtzzozRoqISxomIUOG&VwlV)PmIL%O$jR&1pyfkJ*!*{~a z)-f_u8VU4hh9p?gbBDHs?A+3ppMoPM{LJe$5(PbrW_(8!c2yw9xzkke#^a615{x>H zBNL1}(u{9(A`L9#JwxdAB){@lVa8r&yhq*)IK%6-A-FwBW-~$kcty*c1bOs>>me&Y z0;8{|tNquu{N_B`uu^|X*AMeE&D?V{HgSAcdW#08@A&WqaM=>x^^ptpC97wp%{JP= zeQ_kd=qP>ie~PT*_NRT*PM2z__PeFGKi&5C7F}|(fj&3?_S0Wsli>`UnKT=tD*~@y z@svIHJPt+6b?ge~INbh+X#bD#hpPWagGYuz9O6a-SjIcE+|ts7;;pOd+}a9x8*I&w zv^L}p^r$cD06*eEqX`2C=z$5NcvXI^R^jZVkGY(B($VSjms}AjYYc9-)z)b)PtYHI zG|io@S6Lci3bHdCIeJ<_@~`S#Gg z_h{U-+*U;J0CD*!eoD9kw~3=wJ_1^+nmYzwu;E25ObRy+PvJT`;!idTzp3u2ekl}re18v1=cK#ud#rC%#!N|DZ8p7cWxByW_0&_- z_19mYzWc*#(ix{5uN$54jk?b=#mI;3x}sMy@4fyO+mCa`u`kgztIKuOGD%kZP*1j~ zSgO&zDl&?+@!Hy8+vmR^ZKTP$n=b!$y6abeOw&%rtYxqGqj(>XnUG$7gxQC!61TQvNiNOROElU z+kY^~A3n?E%yNAgfR#_G3?`cc1%(bkVtQw>@?6 zWjbiW%KHCcklz{3ea&iW|2&OaqGj)w{r6S&AqA_mIPhed3A0T%THn5oIZ3PiM(f64 zf+j6#?<4gJtl24dh;9AZaQm-(uCnJ4=>Lbl{})a?NTcPoIrJTUUO;D^#-BTY7FoiP za^o|cNNYi~ld?V%nth>{DC>!MhKwO%Wb6cdFcV`X8;5}FeZt6*Ocssy54ou9I&5;zgPeTcR7YrD@s1 zg=z8pc_wFR6JA4Mg$*A!7&mT``atd7s_W5MO|p&GL>V_OOeoBsH`l~jvP7%VwAzf- zM}GYY8x_)q-U54c*W)I+GhbsRf)u5<0D(!}W}-u8x*(q?+C zd7}1do0wOXGa(qOVAejT-Eh!i+X4r{Xj?!+#DN_{*gatP_V?mJCm^Y1yjh9oq3g1* z?=IdfJ=D(2_U7?o@*NfHM>_HcZp1lfb<($g@V_SZt6uf00TutYZ@ww*^2$s4Nb^(a zu@!Gl12JuO;)y4wGtM|8efx+1n@&4FA2v?-y)w6=nxjo(Z_H!dB>imBeVybJ!BFA<2MLV zpV4)~Mo!UL_uijw{L}5}u)_{Zn{K+P(OZ?ntrwr1_B!+GJ{3LaEeR-aV(*=B*hGO%Bh zW-w@dV}?{%rUe6~AQ2B}_GiMZmZyxiHFRi3qYwHmFz5sf+Idp0Szp;`pkw^Y6D$5& zh!YI3OLNFPZRR3clDv;KnA}MBrDlDk0(lm9S3Q-`ckn!E*0gZ@)X8_3xcO z-F@oIj!Mtob%!*bPg1L`cWFPM(bA{CrjeakpzQO?c<8u-;r9WD{ok)G!|mT)1iuKM zASxf_dJ;b6J;BOc|0<6gQjw4|pfQh5OfTxgxVF-PRiTJiI62cgx*Fe$F<}seBOVDx zK$$=S>5;~1#tUdDQI-=zeeT16753`da=oU&1k#dm+H`p7*z~4Xza+5-C0{~bQ(?I# z#g;GgS0tJ68mDWBhmg_bZu7b9e?5+C^zZ?-#EpnFt8;&q#&r^U0fv5?l1E;~4O_b+ zUH_3*6%D$XEKCG>i|ZuY%R$&20ue807?~g-Qn~Jcp&LpMr4W5cKzI+C0=hiKTN^C` zqLVTYoskA~{c%lXPc!!V9Ir77y86GTos^!KGdKPD_F2Lk`~e?1^VAa)ANT4S$AmZX zXtJ1VzMebA!$+4nST^gnesy*p$T!(^6jan>s!8X zfCS|)S+pQMblYv|zMKDG*Enp#f|?N{ma0$4iy4HVgc5aF#AlS6}xB zJ3s#T;|;FrVa&MAtD*g%bXu${m&i!kc2J|pXF4|}lt>fOnSsB%ynmo|e3%11o- zY43cTUajz`%qwW(ud4jKOHSkb!OyQtzrW=USZfJ~AGlB2Tu-9L>q)CEOIL&SNi$y8 zlJG-Z{t))*YX89|e=NtQ{~fE&w@ZG<0FKuP&V1d8>Am`LW;2}qq0goL_S!A2r)9FE z)L^U~;+oN9RG``3jD|NQi>|BoXA+SM-;)~9zwYfHPV*NmY^pEqyXP~tD&zp&TyhiO zlV(hu*$OizGSzokUxsd@!PD*w{SMXsFO;#a+CR*-uKIt-7j_(su++V^rF? zgIXKds&NG*pFsd+fOhV{+Mh)uk48y|hMDnANs&fwhp|mgv*M7c(Y@ZX{zks_4(caH zX_XdV7aTt>zvifc^D?~-{`S*fX=K;b#F8sKH;YTsHrOHw^XTis-#D{u7HbOv`2XCmyj0ZSW&p%-Mkkj8OLV5f2!?7q~+K!aKjF{@}for;PJD zYP`B`YRxtRhx<@gZr-2=;|mrpN{1YB$ZAkSUU4+@P3aq#pO)6f4qFEwd~jmY>{q|L zDQ&ZbUVAk6@wa2}S#*IOmT0RaRvu-Y|6qbsmntk^<4}2NBjSWe zE8~%82;Cg|H)$r0D!WAmncY1?OIHsb>^No}`_k zrH5<^-qoeA=&8Yu9jA+`1`igbu+lL?CzfXn>EP@sDIyNnw*1BpOaxm`Gy4x_`E8{c z%a%2XGf|uEj$JZJn=MaBe2n7Y=kJ-Wy7u?xVsUstPi@|N-j~uR|Md*{DEri~xn_MK z0?*rL?f-*We&aU_7ySTk+L!hpUvlB%%Tn*R{rhLq9e3W7sn%eVjn+$N>nS;#cu&^g zdGh4(Hd)ps-7{I~%g4cAxB9eLWC<@5Hk`Sr zm5JoIY(?MT16GhSB`ctK6v)W|0xclreC!kDRnQ$KCkVlK_7d=|;bFzU$t6rysf=7c zQhf=7M-8v(!_^nFD(7sy`aON>MiR@`S>SUS4LVF~zl>H8TETY$mGhImaWX`I0g5@TatXJoHXUfcZeJ5Edg z{;B^=xBm4ms12$RBEIoe$E9Nrd!apuWQ;#W6MlTi!rmm&hYtKEJPTgPAO09ZEtbq{ zD*tiyuEfB@EF=-ZZ z{Z^ALFF)oGbHy>*CoxuT*;`-eLt}RK7$;2DM{G9Pa?AAS!w;n=G{N=gtUK*znNs37 zZAZX{{(R(yj|7a-hWjJ*Mwq=yss4(GNGo)`Vq1cx3l^j$d20f`oVjEnnyAA>E!mo` z$+)TOZ;l1#S*=GoSaaxt@~IXVa58=b9Sm;l{ki!BCp#bB>? z8{L&Q+j*b#n)mIKUViFZ)A?`OE4{|flM-n{>2wZQ#|F&X%!FMTQ9 z_P4vz-ZOSdqq?}~%x#~}$#l|ML9`8E84oPd2mvF#6YSrlnd%9{8MI10E%VS;C#WEfz|GzHzf73q)(|L>u0>WF}M|odW^Q-ePOwJ07fHX=gpBpE1tw=s^jy*h67?}WQ z(SD5=C2-keV#Y#b=6=*Vfk*?-31(;sHtDuPICLA@;!I!L^=%x+Xz;$2SM4=%_Rs3O zx7=*w^u_=BzMjtd4S`oUu*Vg;POQ-6T{E=FEqz3qbgHVat3Ro^(Za-M5U_s9N0#(2*s=0oByd=&Z-5B#A3gYSSL zL%>>wY|}7pak?@Sl=j7MjZ1jzV%cb|%#e?>!h$cye&DUIOCP`JvUI(^L?0ei*Sq#& z@OZ{+PDqFD{~TTCnark*@&OP3s8{f6@`taA-EhT8(*|ahMt?QH>uKXB<#(+rrZd%G zHteakW+=2d1&=$lgp3zW+=>e~lAGi!gOWk80vTFD>mks}RBO9R2-xnMCHCNq!+W%` zfN&bJ0L!1}c|QFP8n*r?>yODI{{H@t`%K{7ci-LM>N@mq!eQ09+HJSp?0))hcipFJ zlqBW`ymT_fO>e3qR#&nW0J_>><4w{A8*QRJG#93MdewB{eC^k%Nw7tl_6OuOnrNFaablXF2{S$jVnHkaDqCj4u<5Mw(KffC_6E?;8P||u%+mBW39n3!Hk+|d9mO#pkaBI;p+eoIlEVWSF`+0dsK{e z7eQ8dFJvwy&DzaKL8QiL);BDF_wCg>6rPmmCSB3s%&3i-)&8qhk@y~%*?vuXR7ARq3?cjDTU?A8#Q#AP zjC?GiQ+VU6W~OKEuyvX;c|v02|4Ea^r}6saJQJ0p^~9M_b>2fre&bu#wl?Ng_r8aW zr-A*SR`S_1hH8AMch3Q;-Y5?0aN- zYWDq=@7A6fSWQVGL|zb#cz5{C@~!8rp4oV-UDDXD37%nnlxRg?XKhyc&c586JuB8S zJH6pYy9N@tzl=0bu!gM+m$z!;McZonX!xg!{fEg?Z4MCWe?eQe=tl zMOm$R=iT>fU)}L(ORcP07XzzOx7~KzH0z!R^MfOR}Xoa&J|;w3nwWeDHCcJMO&ONbI!JP8F%O6B(WMdCe!%&1rs`@mlVK zu1<%%_uFa7=TGSOUUebc-!%csT*`{E=v+3GLC_7lCzS2GY%|15{exTPbKXd`Rs4dA!w{@B`DK`Xj@2VvYVM1UNQ1n75qiuz~4vs;SO3|04aY8WOG88cS*0+0k z#}|EY2@%OKXgPMSOD7fobV7ZQf0;f9i{tY(y*vMLzt>qDUvl^h({z2lHTEy#on|H= zLVl~)4$hJBCtfT5M9b2G<{*pA2^S``iAxcs>5q zymac>=jvuP@4r*%;lStZp&Que*=MBDsXuYnGI8P9zK-Z0|JMbQu&(ug>$?5TXjO0Q zXXSC7lfM*hu*JF7&=+MG*OY)2yVm%0W}~hQ1-u2;kZTl!=eSmRj`fyKD%WNGA1P++ z8sB-TuWi9s)#o^s>ew5;@rP`=1ne_mwnC#iZp4|qA3N43(s)&K+h=T+&VBEj(hsh@ zHht@VuGXvfKG{%J*w2W)o@v~KPJo>P90!j0*J-E?QVJJIo4v0)Ak{MBv$x?-2nyIGta0UdK1V#TJN-9p+eXg{B8!L^9pveue&gF^cYM{avfb4mdgQAn%iA_dhh-UgsO?`svA}JM6GS`qAJ1 zrq>s5wb#4HjUA_PI7%5|6YTUKbLTHi3!j{uY!@~rzj#O^lyFz*`^1$`U~xX0NwK`g zBInJvfrnBHaE0nX|F+PLM1>|j(%4t+Pq*KhCg@>0AJp;r01AkDE(cG4fe2_kwQ7cFzM3+x@1!zDob8 zU+AP0ukCH3#v%{j2lj+ViGEF*l}y4cA%hBfYJJ4WrC>fz6{Fb^jDp3Tk~>J<;9WRq zd3H0XI9-nUh%inczX?-XEqOX5ngVp`g^Y(bA?Z=gGZLA{-*@4hn z7u!D_w>|CAMfQyMemvcE?}Me-7T!LiIq5H`@iZ+7m^J=$R@`&fThiFrKjUj2pAl5&MyW{ zapO>?fon8qi~1v|RF%eg-1GrGuqt_A6~ zr;1h$uHzD$?Lc%NK-4q*S$>@y`uv_-fJwNaG9Ofj-WYa)M@z7u7cpB|< zv?iWLap$RV39Cp*RIl^PBXID}HX9oLjY!TFqg{ZMICW zf5kCryRA0YxT(utk)EOtD)CVzzA8IWX|z=)&yX?t8`$bi{t83o-Pv-9Mw>F4=ny=D zROM-Y+C27#Z&aWW1r_PBEm@V=D~AUDs@lKepf~yJnyY&+IAM$yAHC|;SA+LK1hEsp z=%>1q!R(p&re#|5#2B0idYXTs5uMDPw;)ZQK7gM&%ykbY5b~Y)rVsSU)+UoCr7xa1 zQ#xWy&)VriaXM@3_mv-M5M}~g2@Wkmx#q8Eoh+wUw+=6Dg);s_k3ODGd*k~Je^m}4 z>#CF;*fP`8DIfWYCe2=JWIy+v-=!1YzJI@}2-)V$ov#Oj=q~@EUVPit+)_u`UZTxJ zS_|Szob+qb4Ep#&2{YgLjHl}uX(TlIcEbr=J8^$5=A`no3JU5Gto+)vnbcZ5BY(FPWEz6JipOD zwc$IzMSUt9X_-O`h`frwVflNsKTCiKJb}ShOzQAdl-D#DEm@ql7_(ve=T{z+K5@}E zTlCDN(U~7OKV9{WPn!RZ5got4yXUXV_Gi$6{OtX9{(pWa{otzWT6Dt+8~(ok%-5v# z)xIa|bxgiaIzf|X>>0;?X8fuUjYtX2_R{Xp!~P#Ssq|MwB5%0;yHb3hfM0p9>-zC) z0vMsEfZ>XcofR96IiC(z(7d?_nF9vjjB#z8%e6s?kst9&Tp@wyhYTVOVnXzMP6(kj zhdd27o&+OAU3Tsj8lA{*fl+r9r?N;BBDCyP6Q$XHNqXSM!?)$?!}+iXuYfP;T9n3W zu*t_qPJ89d^s*Npl76WE>Kd&e=EE-i3HJ3&E$v6fczO;!_zGKa&P0LVCb(uJ4kTnN zWN1k_M|pM=<7r@e2IW+mNmA7vI*mBcf|Ji>vaM9kYzBqZY=2u`03)!ix`NMumetS| zX+t5DGQe+~2MihFylETKBMsc%e1u7-6&jgOU4^{>o@(Sh&M;~aLQ**s%qI$}diN+TnE?}~3lm_$sY+_q7 zgm3m${zlU`jY;VaE6K9o>%7^ejU#PEBVpl{+sk2HgsRM{7{$(P&d>Xjcx zx)0%t-D3>ioX}f9#-PNJss1_tc!9F26U(=2kPIgLMOvZ6_K&d|qjxUtq-PgDFFZw8*!_ zZVvsLG|Tn1kcLsxgW-r*xH+FbmeB0Ht+Q6)a71u=aw3ET!yz%8ytRDP1+6Nc>o>lF zCqrl)B5x(}dy=0$7UnNnXgAi4TGD#@!WFM(vXL(rf4ivK!~;C%%A18Oi@c#Bzp>xY zXV?lzSeNZTwB(mXHw*RX<%(Hy3UQ@lQvkU)704SeNi5tI}W^*h4uwcOSj%}mu(CUZeIecz?ShLfRHJQH{@)b_N5B# z8^e?T9MGkRAy=sj)1gq5Z2AOxXl1Oa&(&@J%Fto&U~3(Kr7puCJBfvk&HmU<2!R{% zN;N`sd${o;ZVM{gaum)mh?FYyhgE=eWKm}-p%{h-a&_XLa zZcFi&i+gz?22ak8^1ZaxZMeBW#G`!Jzb?>PVbBe7>k34!?qq9CuQXYSZst}pFn&DT zSg_Dn@3N=aY7JAytyc2A5%1F1E&CF92*W;|eGMJQbaK>kOy5_UeZ}vqtbL{5H~i`8 zpqHJj8M8}Ly7}64r@o!AVISu&%lf!yVsE%bT0!5p(B3Ah@@4lC@hY=5(yvLgvZHKm zuyrNThXyPgu3^y&!esP@5zmZMF3Nx(aiE8Y1cY22w@#JQ&{xDWN|Sz24cM%&lo=f4 zj?g%*TfqZ`A`bKr zk${6s{;+$%HvZRf`-j{Chl>0$_~B_OPe$?c$=aKPJvsP9_1UMrJiYG!ek3hu?2B{$ zSH6>u(5BOSJ!?mM(#pgTjc=&QU$rhZD|nCg7HvHs?V6j<2OfDey-xd>5pRYg58gMu z=zzWLYni-4GI^q&itA<;U9zIjznGcZFCf|k(IMXdh1~-V75@(#Hu(*^NBiil{|~tX zHtAKi9%|RW>Vg`Y8ho%hCW-TaO^z8i#DLfGc!i9yghAROlU5ErU&oAM=Zf2e8EFd8 zDq%C;^aXq*(hN7h!WNTo&GJ3bXJAR!tFAjO*XHT;xLh%La+xKKOX3H+13%G(-eLEYe#)ujKF?l zrd#X7NU-EuM_Dhi3JusPgbhNz(k}G3b*S|lxB&*9-~hT*;*#3gKc!6pLKTu$%{Do3g{s8<0&LoF>7bo%U5ZmPv6fE zKj(*kGjYWx=kpdUO!M_ys@0W;9q?THiskG_pR_%LZqdF$x8L~>TV+X^-oixD*lP2Q z(=&J6ChfMg3wDatPq0_eV*4McWz;D_kY!f#3e#`quqhD3#Mr+~^l>NJx-?>=V zj_5IT-buDPAKNuBN~J4q+bhXV)_3-w0aJ-zOx;v77BmhSLL4;qSEod)}Hn6EE<4yLg6^Isr# zG-;;Y;?zav?m?gLGW-78=*z-hk|@)9F<1Car6Waw%BEO(tRVHNROylb*Y3>5>r#wuff4{@vLx1s*X+l#fG6w?u?Rej@FF?@l7{gKfUIq zN2CibW{1Z*Flltw2S1yBdhtioNWFp>uX_eYU+vq@Pwg{7vxH6j)$9N0P;Gme_Az_k zC++64b3u{{7wB2rt@+@1Yew&KWl7ccr$iiRy@l2wYMD0wDzTHwHgR3 z-v~e_=Nk;dQHJw~1F47h?~wCmeMVadz$%~mD5LulM`Wj3L}Yi@jj3e=?|9rEFqNm@a}X6ifXXI~bx3Uv@3163v%cz7~T`=m{OKo0@u zFHXz!`aVJhPZghDEK8x2rr^+C$q*B5PTTuU+e4<%Wmb0qficpvb*Cl@E=eT&JkrKL$NhnLpeqP4AqaQT| zY?=!iWmxU<2P3K!ox4Bb7j^#$Ki&Cf$?IT@!*?l5$N{YLoO5(tr>R{2V-X^1!cXzx zIdilGXtjnfq(XaD9U(Up#<_# zSokdGt_X*m6I%FdG8nuWbXuU4A~}^f26q8l>C<6x5$B2Sr@#8$^u$x`4foZ@zbH+a zJi%6&ae?Lu8BeLIFX%;828WjXP4${1gpN&~u%+uSv=|VSGVYE<3VLu|4`%xhb@{pJ zV7o013$+SzoLyWeO;}=Ici}5lzxd5f>Cbwdrwn{3;QX(AH@)@sCz=n(AQL;S3-*ux zgMm1!M?d=c%hFZX{=QU!#d*s7p8xYIdj)FZ#4c;&JRKRYm89&qLK`QP{u{m?ko+<} z?El04pL@jG?f+VR5-%N`!LR>*qeQ&?RTp)xNBWhxNW;yA?UR+SHQ<}p^I*WbGt%4? zDiK-<2$dpEIdk7Stttwtg*vY;>uJsU>OH)Cg^cs-H$_HD#F?B=nn+;69{I-U^)<#9 ztO8=tNgu}QF+TFb0}tZ#X-oC_dh$pE*H{-RAm=;;T%y-8x!D>qMU!QkG@B#*F3HOd zQ>slYxO9wxh&v>j35DWH;>LLU0SS2-W05EmMo*h0UYM-ciORd3Tt@U@A?L*@`t%&v zrz6$J)oy>&gX-&Cy)n|{q2_LI$Nbt}^T>Jmt8%jJhOc;(YnR)Mr($6-_v^}sVlp&O zP(#O7SIR$l1(Gp0uSqgV#>AKH+oZ{1K88UaUz%eT8sl=#u?hQ*@sNvi{>+b5JwC<_ zmenEXlV2l!oUU{Kx*dB4YL8OwBS`z|(u#7%?_)>%hm@kc=^^9}sKoxt?@kpuqn3yR zU5038oHy%h_8pcl8bAWFM;Xq8R?a3x|4}EkdwVw>S^-Z3jbL_Pjq=xgxvarE5L-kr zKH_8u!_q!=yj7sEC9sY=4*Tpr^2j5t1qS?VMw&Tuo8GBDH7jki#nj&M`iOyi=;q5G zdxzTSSex{Am*v%N{g$oJg!zKBbne2G?t9#PoCiky5dzY*>_L(@Ee#z!2UvwNUL|&p z(BIIVx@gT!YgAw&Z<=*eK%M?BzFec2R{gWQ;CfY_j6;J6G_ZS*BopU=W z;%-?J1evm-!|?(_!x@GaK?%jA3V5!FOt0a4)27`d%DCc3IsmW&IG~qX_K{a_msp}?^u$(Aa@5Aq`y4UqxeLytV3zW87N<3)<&7MpIEjyP~1+bEn(*ZJbrXnjVTRcBoM2?HU& zv1zm#))J3l1FK6X;Xs@>_3yaiNqY4_1lvSH#IwBwq73+9_kf<~Jdkg6R)hV6y*6%# zHl6j)FHfhx`(w88uG`O#=jrRPM;)?n+GYE#(nwx0 z)?E)Be%L+W%G&?&r{<(vZl9$Ix>@P(ci*4xzW<^0z(bE(+hrn}w#!9>r@9PUC#gf- zKu?o5Tz^`6hT85nTW*?m-Er&mEVbXwH`y@UA^W=gl1@S)I?k#;5yxfwgWu{Rw_=$# z(7s8NX8+|vH;!-9q~X@8o7b%;@>-PIwr%{cb{%q~&o$Big}}r9-{`G|+kfSYE9?I> zmFVpoE4r7P< z>jECn^qsf5KJTl*&jqZk;vMjvSC`LLoY{?|&POk|n@HORLW6hqK0_e-QYOae>+Fk| zq8WW;OqwlPyj1uL(!#DWX|5*BCeEId=4EAY2_L0Sf27WtKpSB<27>Tt%mB02#J|y2 zzsUHP39=PTdT2>_OgNRm8EeBl zw3hJ5PoJ<<*D(5l#f$S6E4qHI&|4&U2{_>`5eE{4rF``-fzexmwtP{t_8JsbpX@q8 za}X?|VFDh0>3iTLo7^V)m?2n7|D|+ZTikLp-(vrcJ6iXf7Q$K$OXn}j^-oC~P41t6 zg=|}JAD8=Q8&gTItU7NQPZd%}TOp}#BVkofE6}@*_KSP#bIjPrMNvu32nMzr=E@(4 zgx8QKA%#@G0)XT~i4m2Z=Cr7R{2Z*~qYh)n6N%G_uu=KX^hxz!{`ThdJ8f{wvg?%z zi{*TLg7=ZfeDAUAe*fo|V%(Io|HppM*-e|G@16GDb5~oQT_ZVzG(;N1Qa$1Nw7#mx z-U`j|>X*DolMud#nO$^svDL7!Y8dDu9HJ__67mPGXbmp6in1FKNS*OvW*m0|haG^=X=7!(0VG1;S6GkNmCL>h3b%{NM~Jno;;SHAh9 zP`N=SvEKS`pG`mb@`qK9l@8&9W)a~yN8PpmC$v=j>TCaye*Wtl6YYoNcnEMwmj+Z0%@Z!myoA2W2=o&5dn|AHUWwwhxW&zh;V$!Ma= z7;gX4R#s5hJz&%RrZycyeXdZIf4cU6;b?;pV#BzR0P-tuF2ZKChiDT4O`I@tK(jiI z;t3_~3ct=apR!?0_N}rn1s|7j8dlnxJd|VTV?JpKp5LfGCl(;r6TV#HQx72~=_Af~ zd-UjK>dQylIE6tv;}*U?h^}DL`aD5z6X?(<8kl4lqwxwWc^=zlMVhn8dMRnIsIhYv zrZEc^rxjYsQ-zp3s1jB_vP`@!*Gk>x(HSB?n<5jBB<((B`4G zewru)GSSS<+gQ=YFZnTXhpe^i_|58-KeOxgNCNaJFSGNAx7<(~f0ixA;1ve_+<50S zpCd<$Jch3V-Qs)M?%LMsus^K0mSHP1Mw*xiI|Otohwd@@i-1T{#yd*p26V$HZaT`4 zg&}{^{(iG&%}P7$utUe<{Z4!;r3E6gHYUy3=Ce4f`jqt1ktVVjRRhU_YnB75{5pxKGI9xD_gL+z-?`cqO=2yKWb;xOc{kvPz zH-GrQ>H9zZl_tHOY-MJE&X^G3MxNitzHpiLIhc@+IrRDZLeBx|(Eaw*1=B}}$hAZ> z)%WP;{o?Qbw5|5-w%RNm`hvY|<<125PS9lBC??G`eJr&o)Vk2mcC{0O`5pijjgaiBg= zf1s^M>(2jeP$IUuxU!$mKfm*%)x>(q+0NTPBfa4jN9hyu;}T!jVx<{3qoehd8_`Ef zcZBHo0(x5A<~fhPaJc=u^R)G6*{53iC)#{YXu~(XqgfAa{l#$m@7VrZ^lSZpXmD^t zYqMezSjJnprYq9VTq&R@SsK{{_Z5enbBVOHKxjbdIS$x4u0+xZ^abSuMUt(IndtXx!ofIPwZn}!-e?7R=W(a>3*T9*pzeh(DaecA6pAsUEb=n!d~w+H%`4cD7)58Ef|ArY%u3G_YC zCxnfcFzGMe=j($$JnY(ZqxBN<1|NpJRuA{5pI&;kb1?Iev_n7V&G1pec6yZ_JD6%i zy+FUca$h%0*A@CGe>tlSfYVc0bsrYpFiJowig;BqEf*$$j^Obyyh$vS3{3}y@91qu z^pi9Ju6@h$2CLlM5x1LE zYgeuELnQ1*yR*L2@Y=|OShr!a!HYDDa#GOK=|I`v=*oW4<~ffzuv30z>zv=o{#MR} zfS~v_A2D+6hmh699F~7A+rQmFgC`|?GC1PA{F!emleSllR{~iux~XnlH~T7xUI^OI zih3gsj8MlSrG_>DqTP>if_Bz9d1vHWCH=QHKvhurlfSc_S?b-Hb6;YcsQ(68R6JylvbHMS@?D=n&&aB1 zCf`oq$$G_mtA$$Rq3RYP8yOr>51v9ufE!{ci(k;-Lvp2weDMVZL+s#bjf4loIO0dmW)kH zbbYaR($JA~#ls`?X%CS^1>aPvL5RlnJ)kmFtaR=ihaLgv&-%A>k}b{7?0}hA!D?@z zX)z7kx?m^%3>Pewqs~#IbbS>)`ZU6fowrLrzUH^-rkid`&wlo^Ep2Tc9{J^$()(|A z4o*IHUwuEbe*queNZamgw($l=2;0~%>&A4X9M=Yl^#D$UhzE>xV5hV)f7K_=s9!&8 ztO2dYIxU~-rme7Zj+5s3G+Q}k_5_PdxNyKW`YIG1#O%kPOrQPQ<>|spuFwqe;P}Mq z0}nr%K61g=(;9?2qfnKWuL^-LOp7ssaM5&~}zTzx3#X($}xJvc&-N<}XZd z`S;JJZ+_-qZGdSj*0Mufd*dzXKQ8!&y+YkmXf2+w!j0dK+iaQ6KJ8^D6O&%?#W+@( z{ov~B)A@Scx*5>T+cjaf#ikpjDN`oeQ=-Y*T$oSLkJE*ir$@ApXcLrg`mg!&O7?%9 z$~So$_WuF&e^ZBCasi9gad#%@gQH;dGZbjj z2c-8V39_2sf*0X)lfezhVyz(KReio9Ja4{Mnz0gBPv{qGbs0Ar zOrTLW`y%PygqlQCHy)Wbc?8m^%O)ihiIqH@^Ljfs4%B0N&T7RO{W3o*XEvk`=i4c^ z9wV=;FJuNY;;YO4mEFRAt?UiyGi*=>x23nMGOs7yhJBzD@vs%Tvv{rsnsKc({OnVt zMTi)z8NT_36TszHSSKtx@+&K+bD;SFJm{CJGpEu$tEWK_S~;PVFX{%N}fU?abUB*3%sEKo3e!Og?t@lK>7U4DqJXQ;cpd>3xx31HlN}Sslf>eH>8Nxe1yhut_t~WCd%@0i{{c zt=CF*ZKY2evuIS>YsQW?X@<;eebQ{{-_toSe7`fAW~MWa-#N!u4cv0eE$QKhA5MoK z@LYc@Pxn-&67mw?rp2?|)s-XnrqF*1#e&`LZQTc06 zWR;m_yvMRHf|`^*3r_>~%QjB}aslKn8m#G8SwKR(+6BxBEe+Niyz;XC>?woMbbN2X z#nUxr(3M}(>&YG!Bbg!NPlt*>HRh-in|eVHZ=pQrr|c|ydi)~tx&l?~sI0Eg7i{$2MF3Ed9+e{QhoCng_I5Pmt_ z|5g1LtrSgpYbD;^U$2q=58Dj#P!5df-=dETShJ5t8N`wjP%$Nk`_;I0Eb_cuJAPmD zYswz$t^Ou{^SoKU$xkgS@`sD{B!74vvN|3&pz7*$K3WrH+Pru?U-4RID{IG&(T#{+ z)x<}^=e5o7O?ZoV<;NibNLz}qayZ^Wp&wDO`357!H???clDx}Az)Lt{hgrpdTW1ii5p05o{TQnUPr)O25hAZDCnV~GHX-6`OMVl=IkBVhR5~VXVj5?+j%RZrM#xwZ zC?b)!*5ogES(l>SwV$$0)p3^`TbaFNP|GvJ(nc9OU}C*JqP}(S89SvbfBpOPyWjmT z?Y;Nj>x|m&yYIgA%U}L7J!|J}($pyvbWPEHi{AX;zMBU#*7mJE>zp){l7qF+ST&Yc zaIpUy?qVYo2-G`_n%CTxNz$gy-+|Oq=+{Ue*!O(L`sH z>nn9dJoH68ei^j|l;v77^2w@Cn$daMPX`Q{8D_$abc0rjGfUI-K%tC8LVE!#@3`a> zgTAMHhIQ^w8|bJz_ju+G)hpfZ$XTz;Q`+H~NuP0!3 zp7M{^O$-jRQ18?84CKvvP4nyzU0^E=aJ2~k`iZZm=gin8eg5m;PnUe}XOTSUArpY7 zzx(6q%bNWD;Qu`}z4P2JB=%lw1~wQyS(ApWh+x$y%c^PDtc>91jxTSy{;=Ds_kY9w z@A?~V|LB3RmivEaVo)AwZ!u5;mT_x~T(=a^q+S1_S;nO==YeCPL%ucT$Zx=~P(ZTF zcmv&dC@!VdWi_+emT&Z02sRkB2a!t44a$sCw4D&Go z4cPUi@TFb)>ZTz3NG)Y!W&3(KtyO;--1EmjA!H5tGGV8!{8(v5|7ojqmBVYG)NMCA z@Rol@Q0WzeBn*K3J*n5~q{?nWlnq#-1>Yqkzx2@(f3U|(zC$K>Cf*uc8=mYFujJ#C zq;=<;kr(8e_c%2j45p4}R&mgF}9JFGExQGH+$#&GJqCt#SD~FpZBA z0^$P^?|?oili)Xa4%i!`i0z^?Rv^&OyzgW$%J>TG;Ro(xlV%rReDOLpX?E$QmzvIw zIA9-5nDH=MtCa=0fARgtT`M$DCaOkhLXBW6qqTav-vQ-EgMe7XfrMthr`mnPsFD7L zieK;iA(7cH<2Yc|_Huipg}J(oHkfAf3)gG%^S0Y=OFQq}ztvqWRjusFoWG~@-?V)? z^O7j=uJozX_KcHNg7`bW$=iARKX-RqsNwsvq8IKy3Fy@A={|3XW`APQetK#@}T z)hEp;4RdDrfQ>Z6>FwYGhM?VH-!bdnbmj*>oqnM&VGi|h(DU{%2j`}mC#lwWHDelk zBlN6`aEfjkoMmH=&{9*sjq>JTk)|@B7pETYf!-Rje@Ig^mj!HgIL3Pq{8qx{@=bw< z<*!wY(!0?oW@nb4&r^?+4VWYX`B^({llXGvcYgef+GL8c!REmyz13dZX~`HOC%?Vv zw)CP`pKaH-Cd<>OPEPN9(<`*fqRW0vn%TZPdfLOFWJJDcZI<(M9F7ps&$>UFL6n{S zci8`>3h5%QkfxsD_8;z1`&IZ?!+%z}|LfjE2TP6*e&WUmxtp>(r1v_TfRP9Eph(bA0ux3|_}IjeojX=t-t*#y9{yWy>_pDoJlM-LqB_S1$0>SQUx`Rf z&H8)ex8gnbUyzM5Mi2go=)itXqqO2o^ILkF!fTzpimA=~wVDBLC`%e4__M-fW&vzZ zDB@NRnu4AWhV7<7QPu^p#~FOsLuMg|zSp??;*Wp`gwRd!V}cd-uQ0_syvC2Rp_8z8 z_yBoBA(VWFTxD9wfFD@Mb`IN<7Fpfpm$-&g9HMMMvdVaYYEk$Idyx?Fuw64BoQii& zj2UV`5I~L>ZH27XGLGu_Iu^xAI%^!LA zDd|Ho1n<#r1oVE_SN8Jydlk&!IwMIUBQ~-#hG*h|Nu+(By-Qke+LUz3C6}ZRfB3^` z;>3wn@d1fU84xOcB7#1?_{A^U{q~_R*wgkhV;@43+paBHAys3QVxrLsuFxx+rYzH+ z`wGV>1v5=sbi`MAFP-soIm%mzI{hRkHbC$YxS0{|FYGnb{BVi+gn!Zp2HA7@qKhs{ z=bd+6gHRt&`((F;=}$lYYC7|X)6>;eH7BR*A3i5-Rh_R|+&fLu zy_T;w_i;jfkA-D&2t8G&MPA^n0_^9cnQCmj?&S`{GW^O#g5`p}zA?|r6gC-Opw%qv zLSVnMrP}l3UtT@aIxBa0@7Q~nFLak-zDWB;o6^#IT);aK>&;$awhRIGKsnQ*R!a-o zz0s>K-N82n%Xko4Sr}zA9G2e|7kXF&_UEF5_PLQ6qvgZQy?61MW_42Y^-AI?#~+ri z)t6=2->(epb2F4fqYpMwxT$-`=`T+k>We(Ag5axjd|lLDC(@)D*X8<4S`FV_C1o^d zgxv$K0s9ZP|Dp$bn)ZLIFwX=G;*RhU+*(8bblcLbL== zXbzqk$T@yZqlKPmkQ3y*JayFgWt`!mDC*d`K9a3y5QMtLYkFnTZffhHw=*F%^6~Q$tzw?Q% zT;Xm%*(meV+5SQ=)r)}ZKnR%wb|(@#scdZ5EZ1E%X3f{J@P#mKs4O5f0>|r8LP0Ou z36K!w0^&o4B9iJrhHsS5iwhz%5p4R}!Y3zE;A`s*9|Djj;W z+mZ$8v4`(XH~jKP>C2yZN4lb1*EI9o^p}g@o_6l%YnkwU&N=58`^O!=zb!Umd#iHA zd2N%rPY!uYof8k;M}C_PBBKNJ$9NqwRxU}*nV~aO+>Dx`#YHHf({zDcK*jB%lcB|h zmErsa3)364uh40FJ!)NS7LcH zt0l*?%4$MaV&nVurcX_8J@t4q-@1O_OS#X{#+#G$)yyf{NR`h+^I;FZ&cjk~Ce^HM zSBl8~94)H!>9VZ{ewkL{M)~3PFHNCS+JW^x1yt1X0t#=q|8t$7hf)`nKSjUdkYo4# zzhw^jgL@A8;NPJE&oDKiDwCJ9a)=QgjKO@b2Z8>3fQ2NvrH>L%b56pzwFMs*Vbj>x_r04CiYub6c zXV}U?8!y;BR;VRxX$s4}?jrcL!mrWZm(m96A6T5POz7vNS;?e3oZLb6a`Zo(f=k>Iy^8%`C1`2$GF~C{Ei%yduls`G_W$(j|N7>8YZ`lHZZ6=7 zVF^!_M`%fPK#!}dwKVdb`+#8VKk@?RzAk7CMfp)p*nzm&S^%jDAzQ#Q%@BszA>;tI|6CYkH~FQ4fT+XFogs1B zWI$`5j45fS`49TUqaMevCsCp-(Bi5-V6@19a0Z=#9(UT*=s$An62`kyTpF*Jxbc|L zh(~DRo6LTIi18NC_*B_|mE1}jmftndtNpX$qb~wlnGDN#*l4K0Qugo%cqbkKoH~#m zc>z7i{a5db=)ndPN>e9KNHY)HFFp9+gVoBOn$DVx9h7El*}vsUyqb9P%{Qk*_S-XU zvB?JhP)3t1sK(l3sG?GjG87-JhyNq=!GOhz&*iI{OPA;)HuZ10-k9MW$ZtgfEv|jg z0xnnFa5MCBPHVZ*2s%q_>zX@p=|$gO%4b~!~^DH9YH@Q z&4Q_b9qMrF+J(WbE?i5aOy}R9Zoe}f@`|^m+y4HK4g}Y&r28J4oen?c9qE_9V@+cR zeb^{f>A)DV1~gSrhLFqlg0Fe)j7{StPFL&DXoPKmrR}T~hemyt9%&U)9Vuy)w2e;Y zxuk-{5Y~D|2&U z67X7=e*TqZhh>~Re&)6rmho=$<->-Tec5O#Zu@*?J4G6P9Js(vzmf;56XxM9LJY9; zpwZ|l9<;d969@7~*?<;@C(;CsuN=2J6gPSqmho=$McweIEC2jn&d(zb@yZmNz*FP-B06k!tMtrCQ_%|73)_`Vi1i;iqU{W?FW&_5=Y{0Ch zYPf5~ec0j?wTfFk8CqOO0S#Z1uN5Rt`G6%Y$EwOg!+t>;e_HyF|M(AkwXu~(-_M?P+Qn^r1%KVctNuTGUjkrRQJr1PbkEAnundbV zG9sIRAR>X_ZZx7H1`!pBvbg6*O*H(tM)ODg(WvO}MvNvd5sk(Lg9{P?MMOYE6l7mz z6&MB>X6@;j|NG84U){R*z1OdMx_jpJ%&VDu _Yr_NS&@2jqRZ#rV9^?x?=fd?K) z?|8>M@Y48M>2g)O*19}HIE)41S-M>l65^!SwK&P=PYAoIu; z18MoXX=zCF8^$mc!kprVcA;q}*;+o*iCtxcJZ$G#K-+m1ZK?Y(j#Xf@6n{usihkCh zzb$s$I?djDXZ(hsjFr7R-MiT}6FS)IaB=jwOn?6Lr=$b$1ysKNhJNIG zqs>P2A17QMyx?S999ZvTL*s=t#Fc~#s=OU6T+*((a~UozW>CsV=i9{*pb1r{!O=ob z4cRU+5eZG{OGo-T_+teW!mZefB^8 zn-17)(j~vSHofYtr=^D;e$;=@QS*qmuH&VQ(Gpj@mSrWb`X(%CN<7o`<9{;ZU;b{w zP1XNXWpL{zh>!>Up)w8$CeBAN2_yrUnLGtnc4vCwt0j0N~$>k@bV>$a4?CT%)G~3fL z{2yh8RXB_7px=?q#*Vg}&4(?ey8_@)J!#w1zO|D z-m%hE;_e8>F_S~$&4Y4C29t^8lwFB_XEOf>t&I=oMZMXckpI|L&~UE6u}ty>ZhZh( z-;#^vDl73^5AIgazSxtk@D==vGU+WS9wjXMFqx<;WXP%K^>DcT0So@b$+XnO3h}osK{LczgeLyFPlk?DemI zeOj_)Njm<;&$9(UF0gPhjDO7LLJMPG`WxifNS} z;Y*eePaDJ{+Vu1&em^Y-iQUNmEP@gKU(kT{bdaSua>jypZ6?64*{;r~xAx+1;k zb??EK_Lj5hW|dYCtx2!^yVKIwzW0+b3$19z$*`S4N*c%2j%~et1&LKWiI+YseO2{U z@~C`A+On^x@k ze`rc8>tIvWRrHQ!hSttH3CwZP%ZHVGRm^&qYD?JO|Kd@~;%zHl75|Ed$!P3xa`vvP zWu*tJj7ThNP<2l2OuQ@@8g~-WwpIUid7a*QmTe1#jo(z$ z-)+;|=ds=lzoJT#!iq_#w(!Xem|x5nOF2> z@nc;O^S%U`m28C0uj+&Sbxrk9cOHHH!i3LCuG^`MUE!)XyTFdJ35@1HZWyBxlY(Z?qKxI{0zr1i@fXl!0+FDN z7XEnRzfM1*qyQlf;}=kT1j)z$J0x!U1uS(nIZd>X&nP1U)naj)?#n7 zE5jh>U7#&#q`DU39x9OBf)q_SVTot45kI!mGDF0h#hNcX@Ck`oSd8G8-)CW`(z6ad zFg^QEpOh}STnF`1!Kl#b@-S2)k9q@$R(@S3Pj5K@J%ru93&G2^(`y9ii z57>a42WSV{4YuJ^B3hW4gT=$aUA9lF4p@{{VCUD$9RLYI_P)1DL)d*bj0LGRSkziQ zFql>j&cw&hW~JpbW~WDSES)h2M+I57Y$guStiZ8qaE7%TMnA(?$QznJJFVPh+q8WD z$EJ1rFG@4;^8Q&SD|W>3-JpDGMKMg@F_{kmVm{Cu;qR%#f*E`{6FlZn?^(0bDX)7u z-ZecZoqY1i>6&Y0@?YQkW_MtV#DHnJ7Dj@n&`qPhQ z6F~QErKFd;&4|%-Wa z<@d?<|EZ^aK7IQ~7kapCrstb)zdIfM#`mRjzVacvNOd-Y#?biv(~p0*VHxu!uDBO$ zwO4=slTF4JubruTrKa7@kjf~d;w1m@;*Auqlldk!aNeAfzPQn1UN>EYmU!1qG;@_C8tTFBJb-~BUL{&%;*o#9ZpLGdV< zeKmy;#$`DcMx~45Fl@3e#*RVi&NyeVR;FrL1Qj;UoPaJfmO;<9Ha@JhigDv)A1(8;O^J7EU(%F#C;G`Q zFJ&n4(e_)mDe+G2OPUhzL_gW(r3@uL+J4J6C0@2a?ToLd&;RES%A(B{zw+9f)63t) z#sAf#p8qD(Sf$TOe6)R*ZA!dT`;w-_JJC;ec_~AQkG9{kO^J7EU(%F#C;G`QFJ&n4 z(e_)mDe=*^-`I4zDIzQn%2nM`g_QP46Dx2N%!Bu0g23&aA_0l>o>-7LlVFLn%*y%0 zn{>Rh#$omh=(deAlT6YHR=mZJk6;n69`SPAc9)Wwfr<);GF4-v{*{^$_y^{xryrHmuR zDipWaG6>w_GGK|T-d<8>wqnfc5ln~nXd>za*{0;-IK~b|QmoS_#LYL+7P!V1X^9hh zyNW0Iw;UYfEYD+|V~3va>%U?A4@c2gnuH46S^EPRCx9!rJ-FHYskH@j+j%n2%k4dW^zoX|L zw?i6!?83Ber)^Tg3;w5VJvYq+UoMi(!Q$CmybGD{AKMb|U*mg`=V6B%`{4)02C+Eg zA?!Y*-#Q)AiwkY81Ng(2)cR#}I7o*f&lQ}qyQOjLG;7aVD}92bDsf&m zm?vbqfAPciSkvN%9RXWYC0_ZuKmOfz z8VmjTuZ)#4W=h;`*1PBC&5>@BT(^*0KYE!h?wag(FNz46{LaOp=Y8(fwTYKdd)hDqhPn}C zs}j!xHxE`4kjfKZ;7{2MvWTKvSrAyyY~T_!hqh`mOB0q%qNK0?g!*4zDdUi3%qx|+ z%BBLO+?+cEDYJ0G5+AKT{+_RCwot>D$gkylx`u|*Uc2v--t*?;(tA(;ayt6xqtkc3 z^PTkk=Rbe6Y&RI)<(FTcjyU3ow0!xpbm|*ko(?`>FWWIUcMcc1usaDmuedPidbWzi z#Wbe1+$Dycc4-bC)wIR48EGZ{0M5-ULqqtICf-Z724B=1#xaCZf)%>~80}G2dW?;dLi;1*TGs!lDOyzxQIVLnl%J$9iRMs`|)}EUb zBjzbw{cwSK_J6GBKhe1&5da9#SD*TkbjRKIkJ!uRSjpgi<=_2Z3wF1itl$A5dp4nL z?N)cYtBkb5t9maLcjw1ky=_Hb@eoZ3Tlv@NOaGU6$)k=-7D?FG-!0+(soScuwPST3 zl9vAj9rv89>-?Cj=<4(pkKbZf)mz{H*{W$D%NF?dkA9v${gt!Kjn>;f8&lVIk{0=9Mz6(p$<8k!rf%eQ^KO+>>_pj_2wM*+!Ux;?gme@xMcYD5%+iPcmiGEf?IU_?uT?p=c-_dE}Al%rnmv%btMeo_lV3*0Y{vFaCe?2`^5E zAH2W4EZ%jPfkhU61OPj>u-ZXAse4m@DMA}?fs1Ws;r(OWrN#HW+55q;(`+u@5oWvb zV%OOmmSJ}o-z*&uv4Xpg84r2qA*7p13T*V)g1{`2(or$0U2bI(1U*lqCgHEY(SlTJD* zedHq_NxSU0U3%|ZUS$hreDObb3=UEkyi3j(9-%F4Qzq$8C|vcTHG9&gje~TeQJYS1 z{5#FcxttwGPlAnV)O)xR{`2R)kuLb100Xwx^B})f>q!MYg zsDZxzFbB;kBMA?C){MoB**_q_X#Kx*`SSFtw|!`{eg`4VH&S}f$G({U^q_sy!TaxN zLyWZL{AX>={&Z4sqMziGKv|>eCeD1B`tc8IDCTJx;k0YJBwL4pM;rgyAzl2{s7ABu z`e));{}eKWYcx=?AgM|`T4k`o{GWU5U4%DIoqk+U>dKBZIu9m6&(+RqAZU*(GQ-EZ zl$z~}d`obltv&uIhNLAPZH%7T$kVw6y*=hc9drLxbeS~97H!3zg(a?jG9*FtMYNgJ zR>`7yc(*GmbwrT4rl;;@J4|YL{Ph3zTYz1D5Gi?_%4ajV{Yt!sp#D0$B9E-uu zco%rC2SZJcqG|Dm%i`6;8c{}`IPv{jeEg01;o;e~fWxmr zKJ6*{rT6{a@#(bBd_BGLm9I>E6!JqK`j9PTkkiIU)bYa~{&0HFd)@=Nr(uWL%lzF} zke`oJa(5XY>Eok{dT&@=PT3;HEGoIsg!Q#FeJyRlcV0OU_BDjvWrW;yHjJHkJeb#E zG0P56#-!MzEiD1zf)a0*iEf1k)BQW06d|sMuaSG*1L41MG#YCB1O=$1jEQ zHThiN8ypzMqqcLf2nU_e$KSv1n6v=9x4wVgCF#jeezH9V$wkMFneM#v&h&~`ydqt6 z(M4%*>~4J5|2i)1yyHS!M4Z3n?0CeMyUgex+C_Tj86(?*c4#K~NYH&n|2s9g5U;p3 zIaGOgaQzY;MFH5Spy8Cp6k_)tl!6pb_=UgvUHaf>zP5gyZ$_>D{lCsh|MkP4`RE9{ zmiBdvnm5e4K46ijZ#bawPta}eWeg7gfg~^EKd`-}mXn7vJ|&XA2DB<)BH*xQUJRFbe=)0;n zJ}ec#z)4{T8xju&P5$DP=_w283CS~X$0kmHO4aiTHnW?~fVu${%<_k=vfXGiJ4iu1 zlGzq`rUzpybLaRC7|;HXt-LJSlZ?s9E8MY6o7?lInZ&$9G86~~TYx0MbRK=2$Rd_D0+DLD~toKybKlBvfm}e{smbhr90+h#14>(zK zS*cIWZKhN+7!(u%tpRXC!x5W;Ii5K-b&x6XNEVmC^H>0%B37N2eMNh#1Kxq&=27Yv z|JaL{5koTB0vay4*#{3M!4g+}1C~6bx1eOMLGtYDPb+vTY@wn}2y%=uX{}3(`q=T=RkZ?@xdBXMdL7``-7aRjW46JLUPv?|}y%XbWaf zKV<*(zP~-*KIGuG<3bs~A__Y}57QKSNpHbU`qK^=$zmg7^RuY2dm?YU`I_0z}}c+3C!OuF~J2hG-`q@6n^yRf3x0)o~K z+_9FkI7J*SuqW<4L=Rz#xUf98+{o>LB zx8L>0^v(}|!3P1HPQNP-ECvu;8Csk%+>FZDlrdl8Wgj(MZ3#=fMpR`i){*t-27)$L zmfIs|e4N51+q~nS)-s>bZtHRV+Z6m?MNn^J<(MI^gHl_Z1KJ$S7`IS&m@7rZ1~v`? zYD@^Cfs;Bkoc*yKpdEBP@hW-(XPL_;=yF$OEcYbHCV`8;>3}jqldFK%C+SgUyb!}7 zdtd*!_b}7ZM`0p>Y1GeDKZX| zJ)rau%vxkyz%s-uYXu&ZR>vaIfw7f&XK~d;xBicOyexQtgk9Wf%O9vcqKzi4^-i3FGEte>h3WxBXH9o$#_uhM_bIv&@JsD4NeC%Cs zNc-*mxU^tP$O&1uhOArSW%_)d8NWu!FJf{>*J$*w?0{Wi+;zsCO*r^|F&_M?rP&3% z=iwb&^RT0BKKkN8d;t&0U$iJEx^YpDC->9$`(Sy zM&0zs05IE;j*0K+qm2klTw|py>!$yDFPe!lrz0KZFH@7h+ezP@U++YckPt)F{WtsfL5VA6s7kI7@TJx0!UidEP0U`{E z)qPGLOoGIl^4M?}Ung*WCxy(q+rjsUN~3P(O{@0$=&=Ptq^1 zxW>jpKmHr@NipX88Lv4=y4DG|Ld(QscYG|&4$x;v+!_Zg zan`Xi${h2yRwjO|Cr%hdf<{-A84XSu2BG>f`l5{GS=RRbnJq{L+L~RWTWfE8)xP9O z9F^7&VTrSj1ySZ0%D{)oy_!oWlqWX8jvB=$$r|*|6KyT7LN4N2J~T1;tZ_?w(-l1X`om1L(}wWBT+zWv-lKOk3EFo@9#@{)i(M&+slTzKES)l-y>)y!4h6kHRB#udoHG z(@r}rEn2k59+h0Wbg46$;4^nYopHt)X^%bjNPJQJ3@l{5`Gmhp?>_mswB5E_`8&tr ztCRdHC0`iNU0ifMb+#Q`)UA-Ko&G*&`7kFfZNj$YVLQXHJIodsZDEZIUU)gZl{vKc zk#PqZjxFrq7fO9WjEiB$4{c2zoNS>C{SfAayd}aC&jfWO+~+eF=uKxdC7glXaoh#R z_pAAWBHqmgeE+?6OCNjp8`59?*%RVBBih3~=8y)OSpR?E(_c&X-v6N4zJD6M1^(`zKASH7 z*2it~uea*QzhahlY0f_!_x(9)Np2jR;ndr zlHZz9t-NTA7^jijb5X!&3+FG~$48s)_);hn!TamlUSb3sS(*m1WxKVI-hymY;v={3 zDnD6?1zGp3{uItr^JZN!s`{w_DKify!4mf})f*X3nDwnF{BIfH!DsAahi%Dw4E$>J z(bmOerjeTbS*w7{Y6hxlakO< zv{ROx*P)CwTxYEARD(b3S%YLKoKt-YXnU(T2Ak$d=-|UY+m_25kpW_}!)`6Y8&inHg z`k~O*-=}J~qWfdA{Lc?loN?yam0SAQs4ZZ&``QoE$@qGWCTNf6e*E{xKgaD9oBu`~ z`5TcCCgoV8X$dv%tgJ;Ww|`me*SzXlN=7OpUF)@nUabCrRb9p8+vZ$dzm|F5r5wT? zs&9ubfy%6HoxFveXgjv|7Wv$bzOR2fe~#pT?vg^P9FT9+b^oOpb;7B2F7NG1+{as? zbQ`r8x};DD3-8>jTMM-mWTz!Qa{KnN?fxdGEUZ}e>;kO>l+Y$UPMf;^(z6w6{#T5) zQuJ^M-6@v}qDZZnE&qZzTwIyR5rZpO%VS2`o30du255qCzd!+|9!+o{ACu=67U zb7tdfhRbK9)vH$9;>+-4$Lea?=Zci!3Fm&Kh+2OoTJ+H=o6 z(=2>3vQy%tm)wQKg|Z7SxFDT(-g)VkTW&FOKBo8L=RY;Q2s@&-*=oM^yCuFtIS6_9 z*dyP2#YYhN;(sm_4PYml=?Xmv$E-iv&?*`wCPZT`!+r}#839v(8lKCHA^ad>`awLS z1+6bG!W~w!kb6BAWNrxPANP5n5o|B&W%x9*3_uV(`vdb=& z9u%gL9(?dYd*9u~7hjytKmYu6;e{8bWy_Ws)n1R=IUV)9!_%KUX&>9|#&^W|*F@th zrPyIce=^plPs7eV$Zpp~w5Bb_qJJqC$AyWKe4#%clX();?^=ciX5^<0h^Z%&{u^|WDVfdD5d zAl6}ZnK&Vnk7Jantp>3McF?Fgh9a`wek}ec5A`j1%85tyxs1Jp>^b12YhQnwE8GEJ zxI4V8IGdd($p7#E)Yt6GF$McRItB~;^S^u}y&5myLQeD&pA&0Ib24&L@@-ZUWR8y;PvGZ7sm?aMos*Kt3rrM7fFykMJ zRJK06)mFEi-T3wOpM3x8Iv_~DmiYSkd84!`VxRK!(7jt-9*w4YsgJ4RJ`|CzGE%mW z-H{e3*}nRTC7iHIr7Xf7s_%-rE)U1LB|&BiylZ=JozLCq`}()@XJ`M*8STcBt^tbC z?1U+dNKZl?K6^o_#(2#7+o&Q{#60NA-(K92WPGFj02VOT@Fn>8?Z_QKYq492JGzF@ zjJt!5#_q2p|Lo9o_4T)xmo%x$< zT`2O!Kh1OGY5L2D$n#G&+^sbYe#C#+Z@RS^92iQ&uqk&EatGMsciS277CRarFk6-` z{`EEKlHXjLe*Npq)1{YQYC8$Zd(NCWY0;uZc-P&0^CfLhKk~TiuDi^iq*LOqvgbYZ z!1PQ!DYw%O3(bdI+~nezzb}q{^p6#Ar=UH`3jc9|Pcv^*A5@?oJ%oYU>976?6$ixW z61EN3WZd|XV1%^Zi)IwigAXpYo+&+&lqj+){>Z<6(@6TK^|!!BKmV4%V!Ebhj zm&<`F+3@3$nPx8|1mJq$zRlj1s9+;Vz|Ls}Ubj&$cR4Uv+*ebtqj0Sz6+`TXj!@5@)_h z>PwJSC0>vksye||dUi&M)TwZnQo8n<{?vo_C%Q)w^m1raGcrr3K$6UY5LtRTl#kl$ zGO{OW6EFsm#&5<)X4}5AHl?G9l{U>f7LAo(U;mLt^2GW-#}K?%;vJpYLlqoLJSewd zJMnzXMGbG3VzWess8ah(OFE{2d^D3N7csDkf&X~y5eqG=2ZwC|Yt5Q9>Cu@pvDh-4 zmMveIR;^xxol7&)Q=Yh2I(YxR5_gW>aqk1^nj3FVci#@?~-=hZX5eL=e~PBHa&Ky?a|K+EQIm>YW^a4KC;K%WZWIaNBOwu z!m{anBom7+}7a+M%HVd|2xpGxn zxq2uqU$GK?8%`_m4mrO6>@S~lSbE{J4#DE)s&w5gccvR}y))enKi+-c;&j&?x2K!o z$5m*@G=K$F`jH;2-X8<6Gr$Rch9{zz2u0eIfI75Eyz%&VACLyf116DPOwSOi$9Yf=|p4Y z(UzhZ(O`ykjr zEy}T_`MrJD)XnuP>si99k7|VLiqx&XxEa>zIvx)eh`$g*~#~n)us+6y&P)Pw3lDCf}|?(JP0cj zRs*vQOa4A_9u*grwIK&}H~NvIo!1BQ$k&O$#kVO-QwB*X==24Z^cG}Qi5H|q)vsV# z)*!X4=mcBorzg{6SJ2$mlT5U!mrRu2#9 zVrlU>f*z1H39GE;9K)szg@LdDeO9)h~N~=F^Y=80@r2fBd@mt+M`ruM$p%PeEz2~nFw_#Pl&|WWZX`*B)J7U%P%9;DOjp-XN_=KqFa5P zTWgN@qZ$v!AB!pXFc5QL|@$e+*V-DXVWE8ZNJ_S+|*6W~mfRD;-xb?T(Q?BKtpoUgS3pZfJ4%qdVb zP&+}%M#$9ckEJ#7#4FR+pVD%Eou3+Or$5W4!JbT9m|TZK(jAl0=Jj^ z^WzdI+f0p?$_GEX!Q4%9vj5jhKF!NqP$tAx+YxjMKd~= z7ZLL^xv~Js6+vKyU<`~~@3=Red(jn6&_9pc0$1N~JDwT8Iz0`~dYheL`^w(L;g0Z! zPn0O|I6+0=gh=K?R64W*E6@I99hh;jF_BJ#F$xT1dUUSI!06dF%9vMM@iVrRHRPk* zPETr2QiVAVQj=AMXxALLrT!R)AOKPD&-fJ8Uv?JU5dQzdH-2PWsofj>^LQ=rFZdw9 zYhHGQvteA0A^zoe!42X6UK;<>Td=P`CqSPw8BjidIhNps;i=RarG`)7*} zh)Kw!{BcoTBwY_)1!*XUEEDd=(>Nf^sX%!HZ5*OZcx255R;I?fP!Tf4wNi4KFTL2R zNu@Fth*!m*k{8!%@}o9vq*S1;T}9v5-z_?s{x>(!kvdoq?l)Aiz|0g%99uR%0xISN zvnmmPZfTEU)>&)^G+GptKS+fej0P$*`Ub;B)=8r8Oix5*O##|5u!eqWR?}@ZHpz#n zN#LdV+^lYPPlJ^D3GeG4CT!TJ#ojb9lEokK$Y-cJ5^WXY4zII?5Ylo1jf-VmDC5G5?_}aGB`oydpZ$)Dj+h_7 z0vr<;S-3F91sFYY$z5e!Oko|%d60&QGBLHxHQCv&@I>e@8?nDFptoIb<6eqoxl7g; zbD~v20F^P%RFkxXWxEAAy}=JIYoMt$|P7hx8}! zp9m=%eNBbci^Z0BaW^6s=g8L{w~U1`?i@5f!q9x{6B8$iOPViz`+vqPSiix@7Wm?~&P`7}c)!ZfGB}#SB{}hy zm<0w|Dm;i3l!Wb4bKo}VoAuT;P7=Xm#AC&~uTKOyhA5{K;OsIp+w6}k+$v*z?noW7 zsD^J^`Zw*}RsV7Ne--{(bS6GS+)oo{fot%7xJ$3RF8#@Yd&4{NV}pKTvf%9;{~Ox> z&Hgm@2wmLzc($R3lZN0#kN>7Txg&Kb6Pk3Cqd)%hobL|HHet5u&oEmCO?!(z6KpC- z3^|5%b^TCKIb_5aSJJkclSavg2%6?gtMFiJwgoXElko>G9!*F__T{r+D7^X_n%Y+U zVgwj}mRcW{D>Bg$#USgx{xXo;kv3#<+sVeFslza(6`GsErZpurL8`5L(V zY5YlN%p@&iVU+TM&gTxN@9Xatoh<*$Q|4ouT#giEkzyfyI+cNRArxLIZOU5AqtYRZ z8Zp^+%oRo3H;V7`yuheDQ21b{NvMW;DK6f0YyvtTJ*v@`M>HXIs~C@uZ#wq zJSjX^vwaXSz=IZGHC`||v~qPC!ixyktio45@RGu{T>a*A)tsyWIe^KIlPr;GwrY)) zY_4YG8TbJlgZNDHAhw=P$4Yl}4ai35g#8gko>?OZAtqaY8^GvP1DPVp4iu$1o+w*s zKgt{8lN@z5@u81XR&R@r{Y zSXmgbvLlEh$#;S*E^*OqGLQzrVPhkp>n7_sG`n%G8A0)~j-g588)O^d!M0`WkVNqh zT)YIyNQ=^BC%nHv9R|(5@x0ru@)t|PB{W}^2?I~M|Gii9aiAH{?@SzL!gJ+RK^g}3 zSay9Xom@{2hVplW=9jX}>;aEqIRs~Ulm29x^ATS`jy3Y)!pFMUZ8f+Sk8R@j7+cJo z;fp2A+k3D8x#@ws-3jepf<17oWnR9h!aRdUiz-}bp&tD5qt~(4Wh1miAiEIF8!Wg8 zB!3sbWt;WJZ}j?;r))volWfkMaEDEKO+NNXn<%M?<*6U*TtBC)=#3wBh<)I&#@E?7q9b~?HjQo8k zo5c%oTXaRgT(I+K0T0L^Uf$FA^_MN!FO$a_ZCLG^R5{B;XT%*&>%C|OO{pQxO(dZA z5;nBVWxu^4-H9(~^h)}tA6=W?^yK+z#hUe(Xkg%x^rd5;klud5`tulX%jH-^`tgOA zrx*YE;pXTZYzPj+BS?hjdURa+#|EBx&x4BobdN^fh=XSc?L)*f-Tdz(m@S;2czezx zXDyzUe&pT<)1&u4l$I}EnwBqFnpQosJgr)~0t;isgcALa2TaEn;+gYjr&;sora9Zp zOIt2nkha`@t2B4p`B_5A$bQ^zldHdc;Qe{;x(In0&z`(i)^zze(6fJqPmuq6>0M?A zy#CDe-|v5J8pQMU<4FUv2htUve|>tv+rKv+Hsfl0*8g0bKK8EH`6Q0W^9u+4*<1eK z+~dEmzt7!yenHgx{9j%L#x)csx3Mlggg8u(3|Uwuzl*8HVsoxVW4}hk1=OyTapvSz7gkp#UXkUNYZu0z3 z&ayk>P-9Avb;NVLIQ5i0OAm(;gAx0dD`K?~iq$wN?qkSDwo=#zR~u4RRt+r%bWSbw zs8Jb-gcCBE9|AJ3zNM4=BzZ_vrY1iMXuSLSo0(*bY;TmxW=?<{Z8aYMlV`Lyo^dM6 zO1z}$d0d7I=+_3uTC#;Vl-NQS z=(#J51b|#{p-a}S^G7OKEEknK$c)_Dp)CmXB~Rm{4w;bH!UnymS?0A0_UM zQD~j8R)|C6U6hio%1G_63BnwO#X+T!_-CI*KcuCuYU6f7A0dW@1ED2#Ue==`*MrVT zgI;_E$)m)xGT=u!$kxvAgZ(}MA8vxeCm*ccFc#QIlhs9?d|mod@yIetr2K}^-wl9n z8!5{m zaWR;X#%G^*X*&A(PqjfJhjYLX&jW`+!9f$jL9bXgB;cTNdX}nf71;a{Z!F?33Fg*< zf%EI^|56_IWjo8jWEFOh-GBX^Y4I(0r$_EyoYuvQB10_f4x5EVt!>iGxm#kF*c=?# zB{qncK;l{J0l;b4F~&)lE7n`!uMak;!NYf;t;LQsyugqzqQtRg^(yRCTakvAFH5VI zJ(^ZxQEvGIkE920xkufl0qjKE`mx)m?e^L=ZN1wL`T2UrG3Wlgh}AIQSoCLSK1o#* zBaLfiHUCuhbo;TLi`sRm#>vPaXx0@KV0g#?wQOX_To*>Q69N75cMp zTj)fk_>)dDsZEW?BO5y5#E{{}Bw9Ja?1tGH%RRQBgZk6x-SejX?a9EP;EeNq*9RM1N*e)b}1 zdOa4=D;|C*Eqn06v~2PHY3aT9+9Dno@@CGT zlXidhfoU5op7BdA>#%Lr^&$v~#;AWKGi52PWdj!_;lKuaqjl841~w9!0E7&xzu{3kli#j4NH7*tyO zrG(d0FC)9Sym)~!K@em%nv4}1l-I}RBAYSepK&{N(BNN57Fa_dc}!4!>)O=UzwVFh z_lOg-s*Hz>n}00+R|Hg^YT8^rsVJrpCiMt$SX>E4)azSjr^GI{PgsyRbrmLln4Juw64* z43H)`ikGgZ+=av^hwQyoY~0|FY82U(*jRhDxD|YZKV`F8KyQE+5;BUsllJxG&m=hU z`nx>Rzbk*&25PxPJeMP0mJr-n`t$Kivs*TtWld_vpUF+%;P<}k?1WqlRhcuunXH13 zbe#;p)!|#^yl^Y#cA9bn~G1tToG2g|G|e7zx3FvY3Ws;OY;wSW3P(WRW813 z>ZhYEsLOT+&J3EG%g+J#lzcsPy}NDPO@z6 z?<`B{rl0;c{qgGCu&Zmgw9^w0Obd3}8H;`jL-oUHfp{KIj*}V70vbu}N{o7A;dG9( zfifC)$d%__mwVoxiyvUHjj^O#A-TpQdeh+c6#y^^29q zdTFoj`qL&7Id>+JAPeg%``7uk_|s?`{G=avw$Q6AcRYIk|Fj1dma^DQWX)CSRf7kl zuQto4Q2v5TewW_+=Hq(S|8B|%P#Zjo&CMhJ`})%Y?fvhz>Fu+wfr6$@Ote;JlQ~K` z0pqYLK`X9b3la6Wl=y8udD*p+Wo#d*KtaGl^7cx#)wRv%YW#=@nOd&BViwT)FLvAT z22^G2p($TCeVX!hqi^!>>+i8JYX9>`L?g-ff~+p_26@kB%h;9tvWr6>1dJhzBg8c6 zM8olCNsyT+BFfnGKA0n}GS3Hr7orH{mIS?R)t?rWY@v>n?4Z9uo$c3RV0S=O@8+$} z4Ts2eg7S448!9s<^1C1l)fnhl{ck)woD_4l{yWh2S}|pp{H1IvQ@!WIfBU7?^=qTW0YwcUivRtt#7uV43>OO5Ep+} zn#$OJW6p?H6#ySs(7Bl~p)*T1u>Oj#jxkS5*W{(Z#T;ewZhZZ7KrkAL_dB@T=sWki zyqKXal0p8$|FQ|)VUe~0^k+}@1RDAxqzsayoo{k{vSFK~)rvM%BNtKE+B3Xv!Ro?; z3(zH3Tstz8e(ks|@XOy^n-1LPae3mg7z9^ABJ1cI6%a3O4pc1YgO*|`^y_%d=T?P5 zUhvDjSr+x&QMLwoes*}t?SHgIvfZEhj5KS@c{beEVYghn{}QP%$%8a*k@lX@C~P1e)02k|1I~Xc{?qP&m4mswv(;~$+al6PGpNh|7;Ty z(9nfHS}sI%mGlU5H6t{CQeinvya>Iz28~`ctRXHL+tGu!v)&?If-gMwY+5mtruydz ze)IdA(~1?V@pfaQ?J|7hC=(oWe;lB1lz)JB+9c zJuO5mwp?k&ne3n+>x4{J4_x(LW>tPN^SLdp4n`FZ<|%_MGIxqi*Le`rif>9oHr)E?HILZAg2zh+#?_Dr_leiPr=NozwbU zp2v_Fm@}e1aPjdmQ`v|^%>Clrg)0V{GLfDw9D17)N0qzDhj`{)@oUISUM8G3Xo&9} z7}G!-z+%Ys3n>~5oONISP0{}v=Y%D$5Gl)qtF8&xd(kX2l7$6rARs6KFZuPz-RGp56M;ruW@yE|wvt@M`|T7?=K4>1X7{DDWG)Sv4iW(Rf| zxYkIr5GBuOYkiR|^TFQBY8fl46GTx4BhWYa%m3uZ*y`Cd*t3$2TCgF+*Ok}ZY>#Nx z{MQ5ik0EDa*9^a!tQz-%MvR~e9j0ph_x1NMZcGmm+vR*R>TusEgbYi~X$nz63B-Ed zD7D;Fi!h+wOaa$TRg+%hBtl>8Gc5aHVY7Xn;|ufJHBTH=E@(_tS=XT|t*~Z)-Q|gn z`udv@C&K^P8w?T!ieRpsdPpr!AwKivp=(BefmUNGd>q+sW04_3+vQNvq(;cPn3qaF zWLq17k;x4{hoNta>2~eC!zxB8r^-klla*j(EB@jM$Ryc2(iiiu(Kqz*uE(ao_-KLJ z@?5O8-7?HUu9(|AH+9b8cKFX_rW(Rqv_PHo6eNw^n+3Es!An6Vo|TCzFOuH9p)xPg zCII7SKXTZ)Gp2m(UvdhUOdcDO3LHRKf1Hm>vSYvsC>wRCkRBv6dQy8*U(Bf|LQ85F zu&+OC1Y1mL|I4@T;R!j9Z_!La))Xbm`RHIjO|AuKV+(9`)0`;i3zd;Zc+3-dQVE~g z00G`V_$dGChy=ZThiWLvvSuo( zy4thpfTn{qYlXeZS}tU_hNDpK%#|u=+aMZ}L8D&s-gd`*Y3Z^Rr0Azfw7}KZ-<}RT zaBuWLM=;5|;lPdL|A`p?rcJ2RkzX?a`})Id8k9tV(|8+r{CD(MmWST${S;<1^JURB z)FeJ*6bc)ypcS!e-Pi4l7%DU zk+2aeSxt@!jVVE^Y&2k<$EOE-GZroE(a>u%`;3UU@j+>`eRe^N6*eSIPC^BZAlKG( zs7foW*yw|73I;DVcYGEXjk1!E4W&fa9xxU%{X zyAt2!x3L5Qm&YOLC5Iq;Q?IP2Fn#^aiDr(e=zkY{V$b7SG@~qyI4i}w=B7WC**-oA z7P#TIyV7bbNDa`}jT6 z@S?}1N3dx1=pzrO<&Q44onGAS)h_YT$>~_o;s*k_u*GNN`K+`KGUhI}b=Vokzbp-} z8L}^D+E+AJt;RwQtC z`q^OhC@tkLahJy$8vEwhb=rD+Q^%?M`ZLyZ9E1tOm!4}l%(6HUGL6FjB%@zDCG&AH z-*uwv#d3*F$b)gOvMg^51}g`ydg-0xS%XIOu$dMm$xC>DfjSo8VMwz{7*u$+Ps4r| zf{ItOCw)=O4GdA*UZE;H`^CKAF;7zbv-!#f0sUT=hrCR*XfNDgyZ&=O6p~_$&Hq_2 zYrW~6v=Ahy@zYjQn6P2awq%vF!|Ar^0Jw*kw{dqEZYX(+8#|#x)fsU97IVjgh=gPv z4ScMZF;}OzCxOXM{i`h@lE;KY<{Ruo4Dx5KrxI6L1%t5(iPFCQ?mO9^hAM3t`CQbK z{-=;_QZrZ@vDKzUE^5EhlB-8T(5o2-G5v>5`%u-1XzIY3A^);_ov70 zvZHfrINv4-l;z1nIO&+=JHmFU!LNmW#CkHt*wZMVU(t@X?W zuazrSrd2Cfq@h)-uo$>1tr;5fgO5t0jyuKp1(wkyF04(PHa!jE6UYOzXQzR|K`buL zuw&-TS!w2+S!u=$>|!gv2ErE|x{L{nd~4GhJW4vuuYAr($v(kLmC7eIN7TQKl-=yz zrb(@+TYruE_4WU)cl@!|Z9g+P7P$4!dyBz2iar|t?KU3Q*Z*0PG4|IYPi7Rz6p8;V zc3=PM8bPCE{}Ji^q#qT8JUF+jtdgEi<}EnYbLe#}VRERpx<1(0LaiWw9F#v2n*2xy zl1yLmaa)>v_>m#`n*AAnF$t;U4obejUBUe=aYQZ znWA@V%6M(YwxqVjX=XOS0%abT?9YSSA-hMVjgKIr*R8A^>ruw$Ow!3R4SxLvY8b?9 z!jo*}HnjpP%-9C;f6-f8wC2yY6jQgbt{birwZRa2HRgY#Q5i{u*RBtF2w^&lF;tt_ z{94w|a>g#$vR>D&S_=;!Z?2cl>v7sf1GN*Ryry@^+oF$it?tYZlO!`=6H=(1lt(H6MC}teGZ;-N>MkZevBy zFJOq7<5jeCA&!ZB#hZ}HCOL4cZrt$41T560#&-y+4--&OYwEDXRoo%|vpw6i$vlI{ zm_Un#u?j!c~D6CQRJ5o2x=Czn)Z&TNYJ@Hg|XE13U(uB1{(s z$);F{f@4>vm=GatY9;?IZT)YZJZXnSbS{gPSNQ!yu6(=(AYNl`U}+N zC)r0nV?NJ=G$OM&XhC_Y*B9tx>Z9l#6ZPFxQ&TsM1wq$422~i#>VV5EJ_b3KV(df^ z*m^e^pxoEKgO8Y33?gu%eMFG1LafB|L0}Yq;NlobYl~!2#{Qyzk+Z0lD5ueo2~gnD zfqAR8GUk|^PSEI4R*CO;Uw^k#%ilSsfd9?Sj0vRXnDElEpU1surT`OU)N;4ZA0K#V z?9%jGtd|AuePD@C7HSo8{AHhw>h zd<_2n0(~``HEp&jz3m<~X2gA;>zLn`BZps^O7g>@gWX z;9|gHE3;ahf=;kO-z4WEjg_ex*_60=cj(WGL9c4IRhdb|4EY6t=md>kG}a*KGakb9 z^>>Sytz_h|Rn2~3FFCP6FIR|aLi|t3*bl)3U1-O-Xcp4AOn#Xzolqc|1p7^G`)@bN|-e3VpaEgrY* zCQGwka^*P+ymj$#zD;TQHw~4sJ6}vQ1(7W;K5WmQ4{Wh+jW4qCeP(Y%})khO;RR*>f)W4`Hs@zP)K8vL6PD*qF<;nu?5sfx2P zKt33Fba7O9Vr;1oD{(cQuL>kH@_qf?f<5PdcW2L@Ik%Vdd#4{4RG<@W+1?aw2Co8H zu^vGlSu~03jg3j;&^WnoeGHPkJ~)g|6nR6ABMz86ntU(?fA7~9=8PWvoQDrGZ&@so zp;FgR>B?(fF5^xf_U4RRVouR{4tLRqPAzRBwFHg?vRHM3>el z?^`JnA(O--5qXyMtZ#`Gwy?HrsrJR#L%s9#`rJWeH*D4R7y8~1(l;He+{+CpN-1YP2P6(ef=v7HtpNaUKMpK)}{TV z`9FN9xM>xP^Ml6~VKs-fqr^Me+ZT_ev|+jU!Sl$n#_;tzw!noKN!=4D2@ggBhW6jfo z-Nf^2l8+(Ls*HJEc?g+24D;0}#vSM)9$7XO=YRP1MwL&2uCAjX%nYu|>Zu zd1zUB<(!!~W~Z$e%uBOo4niD%{fFcfBp<_rUrVeM_Mv3GqrVGm zwlf`MH(G;r;SW==nyaL@VDITKxjlpwMuO6Evw|{-AeC>2M;T*6!-S-U7eg|)E$PWJ z@)kYh!Ny8aXIbHbEG%)g38>#{23cnDiUwr|Bec)FOPY3gl=bxwpNBoBy#E{VRA%jd zg&_qcg_rduKCVSG4=P?T*;zgMR^x)xPaDPp_uRiE@t6u}=)i*@CSnG=YZx&$u}bhp zfhKIicBw(0d5OI3q=!c0*K99=)?#-V(}NE!O;_J=d%LdtWqk`wm<9Ot+Z>b(LJLqv zCz66$uq?|2iexr*yqb@B5vc0(F1DDjTgOQI`P{m~Y5epR4;P<`K|T%2j8kSL=`C26 zWr9jS>Dt#n><0tNCT^~<)p-4H%r=b^Lyt#mY*%H}v{}!%)sJ_tXOODQN|5t1<}8@W zB!b%eq<^fkEq|vCUxLton0o`^`r+F8L%h-}dT7W4Y4JxRwtw*9N7J`|^o#VPpZzLb ze8shC`N~z^q|5oz-(Fv%ShULy>Dh-Jn2tH(Y3cAM?_(ePnHC?~u?2@7(!ZO%%9x;% zTo+@ZO6F58l@nj$MDY z+5%4`xWtshbr3f$qcKIjWj37t?APZB@84ys>>FZ9JkYvreSSkR8mNp~G{=JT%gs$! zm66{14@fZC{?CD6eQx?E`d0Iq`&l+FbZ-sHq7tYaM#!wivh%fq=p&rhN^eA79B2+J zZ;!M30NNC%Ip4Mt6pf5w@Dtv`utDFfS35z;*4N)GvPt+~zMVigu0=B+Q!%J~7R-ZU z$R59&KuITo)!?ANaDEb4_gRi$0nT0AAx6j~*vu%T1g*YtD%%PdBx%`JxS-V+{A|u- zCVc znK!Y^AL1%AauP-#t{)#gyYx5LrB8kJY%G>tUVD8)v=}qE?an`@FMjJ>JGcw&gcmn8R2P74}v5A35Dty6AZxt7>~ zmoy$}35`?5zWJ`WPT3@CPZ31Cfg$~T{mplz|NOy)>6>^m@*!8oqF0+_Djbe`ONfJ&pj+{iQSj{u9U8ME?YzwWh>?B;?9ZI-{ua^7tt^0 z3M*6aTLDT4nHYzKyuzzpKFxZzL33_KvlL@VqiaL!FEKU~sCkvJ!W+ulAIjjQ_=8!a z*0T(XoN{n@34O@7LK|cegNS8{8I~cHJOa;inpg%QqeKmyrf4iwkc_1RTD8udEDMo> ze@UP5jiA58DWQ)>&Jt1(_=xuG4gIqVZLWY!nlq?euB2BfuW9Tj;Q=>@{?WZeBrNe9 z%7vKS&~YuAdGD}*N#T~TK!&9Trt2L&-9*@U^|SUl5u%?sCIOaQ`pLA-VShWOHhtoT z1rRU8SX5=e7T0A9BbF(s8BrXA9$6J$jLRD1CcQN*7?5u_{$#io?=b799=E{Yw&|!3 ze>fdJYgU>yR#8{pn*QzWpGdzOYr9FJ=i^Qadb^MyP<0{IeCjt8J{EL4N^zR>93{FK znYbD(f-J1zk%>XaoW>DJ$xAXY?{b1dgOt>UvlUVAzmiwCMMbB%t@u^2sZT3E^kF)( zx4!;Od(k=@#Q(J*a$ih^bM=G5tzc2V_NAhOAWw%+(5i7K}A@W2lQ2A(9Q7 zvOnQoP1jdKT8sM0Mj|e_d`$v(lbv?PH`329|9zck0&{oW^FaFGXa6&O;!9_x*S+kB z^wyJ(ip4X$?tnT`ZpqAk&E8T9`3cs)n}562cN3Q?j5U?$}b! z)_=txSN(92m-+fV$~y4R230+qad9HE6cUhtv?{j#i<9ca&#Fpd;8qu9!fW&eKcI?3 zL5?%wb;?aFN9UJwYrdW!J_{Q(rd#(7?J2X%pS49NRnlfoY_WzM#~t}EU$HXXbo)Ix z?oKyjcjv=P@%Z!dl~z7$<_ueGp1V=UBch)ao0EkW0Ha`pE))O8_# zA3pGS19#=ZpLgL82utz02EMj|bKyKZ#y%I1vv0Tcg0$!3c20Xd_AzPEF5BB~a|%v> zQa9~9mmY$XqJLOSaY0z(Q9ULwu0=B%u{pV#Uoog=@Q2_gm}W1SzX@u!;Z<6PKgrdn z0GB~!mP2&gMJ=M-CX<-=SkGb~Xx>(DT*!ay(uy2mqzXepgTH9 z8#CpFw$6{OdC_;s@#VDn=F-SPLuiyPx#+z)>LqfWy|t*z&ua)U$>75__Qo8cmTT0T(hqco6&#YLrI-POm z57QUE@#A#D(a%lqdDAP>*7LBt67R#cgv;+1Tm+az{YR8uw(Vrl=3E#4$gxS$3OSZ|M&NDkH6MLZ$}4mTJ^XL~k(mbGvv{+1R;Fp%yfz8I(o@9!|B(8t}Nh8PQ0u zaU`8fPxN9^2+sXLyK5p&sx^($fH>Mw7bVa)GSHDlc{{2m3+QmE#J2(&AXHk0k zA^WB09{!~CG&}}vAIhTm#TeQGbS6MnfO43ECY_P>>yLa({_LCfC5%+! zRa-BW?WBP%jUNOw=GyTfwG$ET?=?&uSE)2=|c4@VCR#WvkK;e|A~A@UpAT zmw&i>Y`b$8U=Doh!B4>B?E9s^e9oVwUGa#!-JM#eCII@x9W3&bra#r1z%;IvE#9m% zh0Pt042{+EY2}i1`(5}-;ApKdIGFCe;3o%Y{rQOMO8I4#mvfyydQa{i;s zMoh-#TT)bmn=R_?S<)%_O$;Vc+%WKQT^@61vhGj|%@!NtB_P(?6bp8{y9HZ6ce=N{g2~lHTwS zpTRWfw!dXwpoB5LF~;Ho)rBHWy}w-QRR=oX*Z^ZJFn0^^cT zE&Ww3tt)oNBB>lt3zmmRo z_Rrc-PhQawz7YR~Z~i!a_uPxqKfU!;>G-3bdjnJB-LSy`W8&U$_|7 zdh>or+pd~oN^HSK^C^?%y^`*oN}*MzdWR(iaNV~VWF4%fGG_T~t@#9ySsc}`BUZ6* zhgMGlP_c~t-HZ2Xf9X5trEmP;=joO^?qwxDqBcDpdg!6);DZlNd+)t>+H0@9(ze@f zo94}%m*&iwlUA%)k(Mr9nwBhCl5V{5#&rGl*QYD5yfR&U@x|#k*W8jm`Q@|IPTOyj zUiQMLr4wKBytF%ZE>T9mrwn=H55|&>kg4U~Qo2r{{<4YpM?08!);};n3Ztl0r3*-# z`4I5Da6(IJ>ebG;*1^QM_ zmx~rHO7rK>H(w55lN9~P-F4{~zxYM^;ZHBOXP7@{ zU_HfmpShxwp}*`{9~bS%kQ?1&S$I1*+U25lZLJL5BP7b9<)qtey=BKXy;8CCn*T_P z4xajtS{`=L6VkH|-QUgHb)Yo~n%-y(bPZBMCXZ}~R?wNB2kI7S7>_%y#LLcC;N`+I zZ}ATn*gMT^((oV6&Kqa3V^@qT?v%klrjQ^*B37mf($sLzP!4Zjk5!M`l0WEhRcD_} zyFO;SwA0Req`7!k1>avWz`q(of0KrGuteUnSrpZP>O^0wM5fukNk7K=7IN^T{$rH) z+MDj|?M1USx2JFY+d1jcnPYt~;f$14-P|*eW^!Bl;9$Aa%;!?Ye1yJ-n1v4fsf)iw zPd%nRoZxT90aEkEp*dfBw2?=ZN?f6%an&6DO@0*@$9#;rj4V+Y3R-()2(2EHh-VZS zs08fm-wXbCr}XYwH%X<0D$B5}WciDTQ`{D zz+8uQUB(3qHq|6O%`Y%?*xs@@MHa4TcU^-~hS1dGRxaCr zO(&diLVDTDUY4HmjAx`Rw;bJCz;m7x3(V+c<;s<|aQ5wQe>;8s>tDxC<^N8f`j79Y zmpuRQ^ww7&m7cKsuF!=s%SZhwCFOLalDCxI^&YSO`XjRSr+#yXK8C7Eib2aHNJ8S? zK)f814jL5q_W3@~%27NJUFqC&&rN5Zd1gBQ{PWXAmtT`U@Y!#sH@)gF z(n&`@507->H<>Ah`O)k{h$LH#Lw{%FXT?WZwm$ec7tJ`aF$q>8?6~b#Ri;<5P2@uW z3%8!<3p!?JdZZedt)^EUAyb3iMiwvbQE#&h$uw9R^v0w8NjHUCr0Hp3+A#j=;9*P7 zYhyQ;q;L+*i9bY$8O|UHIIyG4vOFQxd3ag-vo6A}3op#<&mmg`KJ1HT>}vV|7Rzum zZ_u{db7FH^p1UXZp~-;M($8gRZQrC<+n7jN8n_OX6-L8fG;QPw|JzMoOJ6-yuwrrg z>>J;hF6beV)<={bY@RIVPc7m`3|hRXj6H~!e)otip$8}GJV;RmdJ8qDui`{`#exxG zh@meu(84j>avW!N?6bItMJtn%a{vM@tGv2rB;lWr4By3Z@sQkh^`XSi$>kaV=2K@fOxac9OlTtaI0E}5bMf53kX z?=Ab_XTE`V_V{ zV{I4Nc+55n?<(dZ8jp{E{Nw4Yv(8GVopzeNR^VIbT%2C>(&wf3{LRbLg86g&{ekF= zyOwKn1QlcoI3fCjo$k5F-R#Fca*95YZ?PwBj|oiUjB^=r&1r6Lw>#Ojv~TzWSrFkA zT^he~j_jg8rc0B57|1m@-H}fD(7)TePAMZ7JWo00lyt};hftJprJZ-)Ih}OUNp{?M z=bh;@pZSa(@BY|-q|bi!hv`HA`}p*t=NyXZWUW2XRmI!L@i=PzLwfNYmojKPjzu$w zK!@X>JKI7sbhMi`3=H0)ZS;Y_gabDFAWam{*B942cH6=3!K7ojUlL_&dR4T*^N?FNG zx==PIzNB&v-eETnJI(lE3qDuRHhjq|A(Pq=GIiqFsoolmaGGfTqviK_yu`7m(_l)Q z!rKLR+j+Zu5v}Z<12%_m71Hg-jF+)}GyW=`j*ZIyF|S(-h+=(*NKO8u;s>cn$JA_Z zg)OJP{xS)zMr-t(|8+wqL8mlCR5IpaWU8f`*rrOVlj<$>*05wCoE}7y69vn%Oh8&E z&|jUaMQV&9{q@z_25l^2yIjx(m#;N=OzeS&9!|%cd|LY5^|u-KUOVwc(XW5cr|_P& zCFxDCJklQJp$WNrfJwTJMSuCk+mF+qCPj4`zyBi+Dwcgr7SNao*vJ}{4KLA0=&!{e zI5~EQ@HE3Ir~Nw~5BrHIdItUhU+FyJ)&F1%XM8-#KKujeYP&WX3OBN8*&|ER(k1uX z_0ts7G@+1U1C^0Lc&B<}Qk=TbPupU8+Iokb@v4&9-pKkxqo$zB7pyZnn!2D7sUUf@ z^S|k7l{iTU=A}3BeliRAHLhH=alDE{-ls9S@r26<{(Fuel3AC2am6+1b;sY}sdKs_&WGh>gcv)8Ox^7QzT#T8LNx@tETK|Aef8qDB27(9$$)mvQ zvcRjDC)t+QT_*mKVKsIJp7yzK;S1W|w|jB4Z=YLMGI_2Z(c~5;(`qX>hkak?SwLP*)IqkOsA#Z4qtZW?mPlWvBF^)wu z7lDc~2@0Qy_dxMkR}>IX5fI!_1Qdk;g6y&r79kM!?8{`5 zS(1Owy>+|m^?PsLteH00uBt|Bb^CPnAveo-mR>eLKDSO5PKFyA zwq-Wb)f&S-XKk%WalS%J2hbb;sNu)gKE6HFMz2V5O&(`KmT&?dcPhF0>|nrB($RQ) zZP;&(OENidR$E&wRYeBPyyjvstM6iX#nMaZyA=Iw`7D01{LLnhE5_2s^j_4!v`xmr zTisvt31{s?45k$ePRn7T^2N$(dTtX*n^Tc0ofn6s5yw?DmJK2f;@fHd>sxBAvBRH2 z{O8cmlyF9kpCnsv0Hg95<=Rax$(pjQdf>ZSGxV)EaD#2g7ni6ptia~`hadl)vK=rZ zj}((?|Jo0(l7{+PISNZPI2+vs>W=)0O;|x1%;@-HPft>@Va9yM%5Hl5y!LIVL9GM5i)SFh0Fba z@U!dX+ZX=^^akt9vafyZYdxM>maV(k`2hzUAn$q4dvx~M8E2d!N1pUO`P>l)$Qhsi zD4x}7f;+Hp2(m`}_f&pWgLOMfU+KBqvAz&5=CgKBYC2T{LBHgbfxi)X$YQ#;>yBag zHHc^V!Ao~6zt=ba^#yVama*Ua$TKp1`gHm6kAE!t?YCb(**-`aK76>GdFGkAjQ9BC zkC&TozFGGCudm2uXCE(n?uuog&_=~zo21ec`TKzNo(-CrX^OM8_2!c|4w@BLPp{-~ zM$q_C!$Pwo$=s&rot`yV=H_;U`T4mGw-mjOgy0R*T8s?0wXKz=>eZO3HAq^xZwY6b zv1idu9pP<}mT^|~q%;1X@}I49qx`{M>)rlYOAO?5IwTIg4me~|hWmaex=} z>&*3{{k7V+@DJ~`^nOd*0`#2dHkc!*-tNhwd$SDw+I8(}J`R|j`0byT^?;e1yS24V zj>1>F|M-*BWw-6O#9pRYv4BBtl{Mp=vHVLHyb_e-l==s*X>O9HWlN-SfIR(|tcDnu_R0a9Yr;AuPy)i_l8)FJ*1+hRFcuG88Op$788sPd%TY2BpA!21KDwqthggV5K9PdHDw_u`R99w`@J ze6j9Dr-<=m4$DaT;>Emw4@K(5ek$Zrbij6kt1 zT+fBe^~y{$cTTG!t$fy2Q|0e>m*3YW3#(#arf!3WZY~>48V1C03s54DyBlbmLOLNs zHs4ayU+Y-nhHvq-KI@3RV~8ngW7Y=UF4YxP(o%sN9R{TN63B!R%h5(2qIp2yqJv?7 zn{HCL1VCrwVtwAoJgz)gzSbM^i_#0zS2@*~Gt4+xMjaO#G|TEImY)JRcZ9tWvh{h} zOYM=?$@;SOch>&XCy&PljkVH*H>4Xw-)UeL+OONrYH?3-NE@c{()KrOyB%`A)AHsK zOfQs{rh0DHU!0=DEI&rUOYtnd74S0o&}(Ck#~m6CFO#j$=vyB|9FJ|3Kh|Gr8N>wqxej)%tmOO&kC}Dh~8;?CC#}>A(1vY_mD~M5t!4Z=B8-Z3xBW z_se~rJzTfpAtKK_AnR$SCex$q(=5v##{#*qw2;tc`#U8s|5k)S2Wm9L-FR*Ybq45# zHX~}VowXg}(`Ia5TA>sMOXge)&ZeNio=c4KYGHsa?=%O$~IeYGxM+%S!E3l zNogYab6z<0v!r~%7TdzxN@6R@!I0NjJR`s!*xxD$b%YalBZo<&O^`S2NPHh;cYwi6 zIbzc&3$>Z%lMy3E7`cA>ojG%++<*W5 za?CNu$TinoBL{ryoASGhzM?DK*sLg3;t$GW*W@P!AM9murH9|3tvnaE>l3sZ`Y<1K z^g*}my{lb(NZ2Tr!jElCew?mD<ZVn6#96%BIY5PxMncY9GJX?m;)vBzjedIByRtm^(8R$_gnVYm7NPs>u$NMC8#V~Bel(13=aPTUtvA(r@tF; zn&ihhj{fom^7kvx!m?z1nH__}xt^PqHWiUtn6=rMG@*}UE1O&8`PZ^vA$2=A1QXha zfxFO`u|50kJz4a4daKW}3*qS?e7Xnx)5c-Tw9JJvxVPS~B$5j}0FXsngkpm8wvD&ZiM1#}8X=d3Sn1Yu@Z zwpUkKfR~gifUgfMq1AX?Au6&4W(|$?+J)Jmo`W{)bbMCCB=2MgBd-#(&A4xzA1p_@ z9y*erI96wJn;zNltU-kS8utWBj!@m&ggfs@k+Z&V zq`nN~eFOJDr0iMwO<%5MZzplz;&D@;sxz^U;$-BHMf8`qaav!+D}8l{S3q-ckRE^D9+>5Ex3%_p8waReYyAq@ zw!B|)NQRhheIm~chY60u<-{#GK6&y5I$(D6(MQW=mtCg$R-!z9$?x~R_dQ)+c-C2G z$>E>>j{M=rr^&FP^~x*rT_%SrFMf*2&wTbdx%jEyYksy3u3OK#SO)Qky3j z^f-N|y{%2w@dhYB#yR`UA>-{h9BN$gkw)B5)}hmJXta$U%Nnh=1g;JBI<$J*wf=S3 zgN;FT;81m$ubZyQPnsRs$Mo(tv&xwkjHrM=hL_SJO{eQHmln%(&Hj7sHVafYhQ89k z-q3HhedgK>H6wv(`V++sFW2U7cZY`KzS4F3dmSCY4cY%YlH2p%vhw#;qv%Jv`k;sU z3GMK>wKKlOWZSa0nY>2i9-c>NanfPX2Asr;=R01gABf5ap%OxRfKHQ;FL`)#eAbVy zm1QedX>D5V0R6fAH@C=>&%FpapkFjJ%ItDGjO9<`u4@yVeXFJb>a|aG6?7Z4ME!d3 zxau6~RJFwk8G6J1N~q|M^IQ~R0A)by%B{ngbx$KK^k8fXtW4|lMl&_q1#*O8VJ@#1azc|fl#E=fg z^G*lK!4Tx3t?b|Iru8Ag>tLRyVfOp$c%gR0N#EDm++12)jss>UoXD6lW8`mt`K-;>9}%G5XRz6*f%t>NX_D|e_;{UPS9^WWG z?QH#*S$sqIwVO`1GRC|&G+#yv6vP`F)A|rRmFVhLMJj)Z!^r zT?hSwvi`+s8dn4LzWuEqAMj@CkpOII41{)goRA@ZfRguUt#|=?)9v@mjdLDQtbzV> z>UlqtrsigyL8mJ(AW**irmOOh)!LTvlJaH!Bl|LIr?e^KZeQXv?!n`pPnj)VNPgU} zck2a(*H#!h&xLc#GnIXZ_a6>RacaFAo`>|$q1lmXcAc@llYMPF`d!X4``usflFNU4 zi)_FB_HymD*J4GJZBJ9GzW&VT7a;fFe}B2@jtAvOSKT0MT3awPl{2yNT-XWusSEQ( zN?~!=Pfww~1WI_Y{>^6oo6mck&~RBJecyw(AG4YD6DhxOdCJS91^s37n$_|@U;B|P zSiDRwxZnaE0CkGx3a3n&A~)W6qf}Q{%jeFxSYCW}0s8(nJg!4G@X^>dZm(E=#&dO- zE2+1A@2{BaX}OsBXFVD;qZ4xdu0vnT>yXtQxcAQawQfx56%8D?7j>K4u+8wWNVs@0 z-_zo_=d1OI8~*s9&bU6XTdUN#Gt4}O6yIXBx!c{Lmx*D;h#}eyti)`tmUc7(bkJ)F zajXk*LLQtQ=Fn%0#nAH{Kc=BH*~qjG)|1XMa~kn%G{)Od-x@rCG=+Yua^w(Yox1SQ zQ~9&yU6X_whbf1Pdr+44rwd|u;aiSH4X%a zZg+>I+I9Q0l+m={zHa=F7p7b;_Vo*|*ZGP#z+)%~-q1eeSTf!21C7c-lx($P#Yr?%Ln`0Phc!4%{cD_tt1v-I3R> zLVhK3NUM0>7EwHJ>S|I&xmJy zI5f^I4Ot;Dmw|l;01nFj!(ZMg|9tQ%`QQgXDBN&=pac%q@~fKs!sl_{yjZuP;Os$e zMNvwA)2!>GcO~x8pqZMC#-Rv+Jk+QGGQR6}TgaqwBYT(Yjnt&Bwp!l1$4ff zH*^lECU+acIG(Cd4Tw`hv!g?8cuaK^E#xPK@mYuaLAE|#OlxP17n#r~EtnyZYCfsoDsBf$a(XO}VEW|?824P`Y>sBttdhoFttmdspZ z<7>qLSqn(7Le_HduxwDJG+}SIw#va$0Y0ln4Nd%6Mem{fDUCGdwjzwvT^7<0^JxQzh3r8Lmi1t` zsa=@vX^i)V{Yhs-3;vKtx$F_IxP;Uv54KK+8pnyl>~(-X&WAtV8~)*zd*u1~uNSXi zfaOpJ7hU}qS-GlN+IT+;E!Y7)F|+IfnX-Wehu*t7EX1;lVx_xgf6}voIDgGHCr3t}7vp zkOy%cl7`i0kZ}tREthG;! z3xatV^NWcVIOTRjioK5h8A`52J?r#veUK*b;HPavpwE$YGI(@@48wkL)%bE|B?i#= zF#!990qUl1NL3+XnU4}LV?!24QeqQ4Fz_h8BCgBTFp8DDW`JJovt zUWkMKv}mDtlIEIshz#v#1A^lvr@U&j7SG6K&hrS95ThclIU&vC33iN|5E%Fo>dSTJ zu#{A;6Nki);~bzgePh53Pf7IkHMr*g-<2EIa|B&t!Q$m|)%CZ@(I0w`-fPm=2Un!> z=Q?~T&Gv16GC-}@P4Eb4!T9CiG=pFB{aR$#*%aSPIb@kQ9^0RJ+9ZkKZ3}U)L$i@i zR+9RdxG@9`P-#JN2{e5(Rn-c`wd1$U;M0;L4)ao_N?jdM99Ft%hH0bc%5YiAOn5NpYvK)t(K-G zO|oW1v$U>SBQ4EX2D*m(ZQ}(O4i21g*4tn_KMuyS*2>x{sj9D$8Xm*yrGCUvslu}` zDYH&(M&6d3Ay#S}Xe^Bfew5$%dc`n+xbVUY z<>s4jmNS2Rt-R-*+sl{{jnFN1w@xZB=_bAAnEa+u%4mA(ZE3qdXcinuoeV~DJ7*n_ zj`qn9?7eZ&td}i!)M5K5Mb$rIss1gVxJwBH?*tqw(WJ1`itHO~?f6Q*sQuv$bO3c; z&;T;OfKkzknOx{Z+9wW^<}pKb3Fe@t7O7a>hNE37TH0lBD`$&Ej3{G0{~^V-RfF|t zuc?&w+6q})hX>92Dm26j9Vn{+kAv~WOb+<+3!3iJE~}=gTtPoY<%i@!1cy$8kilsa zGH5>HNq(U1`X){sH+(Cnc?k`kA(TLHS`M{Lr1lzIXg+q}9@rn?M$IalpP}2!RrBUa zYj+sJ=a!d0dbw=(zOQ$ubdSr-*ld!_-rl|}T<-QS<&T=bUj4saUVHgTk!MBjC}N4n zpM6qpxbY@A1^Hk4=2>#u!PBLRj55%-y-n~mG)#v?2~AO>Wg~ynEApMwz9eVQQMUAF z+g-L6S^A_r4qe>vyDRlL>!KgXiEkg5ir}*Lw)|-gDMJG|X6?}XdQ(=MR!Fo``Gv~F zY1u)TEWKj+xFv`7;m8;OTDD@P{0$E|eG&FP0Fgj$zw!$*dvaHnVQ`)O}*N zoO_i}gQpS32eWA+9<+=@L=jK>FUI+>(vAV@b1(h94)}1E*+35bqUQD2U$0{iA3bnS z;a6uVu??B!%1=3S`W&3|D$6v&riII8;R|od@>dqi$_2}$ZFRSvJaW`oGqg^IjvFq+ zCXbTQ)5b|nL#=XQR?6qcZAZvp{2H{BC>2QeKJqL+;_#U4yYIeo_~D1AC<81uY0@M) z#HNKJK>46Uz~YJBpR15X?cU(KV8pCDu^NpjMU^RT(D9cKI@FB{O7^hWjZ|!gJ0{wz&ech3-;&&BguxA zm3#xhfiWE1J(OEfTaR&CQLt+2(%+%J?aqIIr`ywPEZ0!A!J}d zPYb7>Ovn(^PH;o{^GKD4j=_?nw^iG*=f4!^ZBaq84=GqQTo*lw5YX~APZvVTfq z0N9_*#y(gmUMMXUGK#sc_rU+F1G%%VxJbURLnEearwq0=HObt^|1Cc{{t~$k$>)FL zYcjv(T)FInlM^PqPh>)0_J5?mddWgrSwC7v*YFIXoAR%m_h>R;cG`L0kYnDlxs0wy ze}>Io7tWh2=bw7ET!g{1uN`-yG+zBvIeMn;$LO*;Bnxr8+w#W-C($wtv7BU&=kXDD zw^a`WU^3yaeP~wLwbMqUjE3p_PmgiYnJ*OOa`@Onljr{4> zdvOiz`3=1Bll%LTQ_hiNXYoe9v!P9{c@l$W*LN1bG~V@pzgNC~+(+@igKO|wT*C*O z4OlWe8zpP6eCn=ChepNmO3~}7{kAFPpZZoX6Mf>@}{SjBIi^tmUPU_TaM@Lt7gL%rKkt z<~#l^PdxjQ9C_rCvh~(@DK~(_xN+lj53h61IY+L({&x8ko?|K@?x0HZN!6pu7upzr zF8#xjCS4AWXU{jEd0Ae4c%C#bNx#%vJG4=TO_(Be!$;s4F4gt*Qe9V%*<`htS%zm@ zJnIhPGbe6L*FvY$h@mU}OAkhn%s_Di zE?7Cyz}Lyg98wm8W)D4$MP0LIkW~!G7+4>_4fqPg5DXgCV-ThqBi>x{$3Yrf`iG@~ zxG`wMU{RY+MTgU4K&h#v1^d;sXkN_Ym;U=;-J6TL&(l900qI3MQ*U$TD~5-7g0E%y zYA->sco9$Zx(FHaoKNkKH^lI3F45#@KSM32tF~dF3Ta$>P1&m|IcNs6V2UuqTHvm& z0WU~oejL2BXT6Lk^+$2yb3lxN`DBrh19qG`%cYlWG!=jx)Z-vp4F=7~CkM}TRvgL? zQX1rpM{XcJZG0q-P*{G=u>3$GR%aZ~qZi~!?Bgs1SQ1}rjef``%No0#vj6m{hA;Q>|^c4((T$8l%%yh&=iq+pq2i0mwn$W?ZyZqW$uS z!?@AIUmH%$WF%0Y|G~M~rq~WNxUczX1`;PGoktIuo@Vp{0i4v- z)FO|4Y1;lP8;8n{?|7G7_RtQp*Xf^?&tih)b!T5BAKmj?vhNu5J>BsC?#kb`;0f7# zzi-PSS6(V#-qr`PREiv#J0m|jM-+UY-1(1>$qpl|4&*SeVR7#0t*PJAe?8>JE!A`5GL9cZZwD!j+{VBf z{ffbIhHsGPhRwF*!BSYS?WH1e%Re9X%wC6UTAM*L44SQ2B1;6@6?9Zf^Rse)FAkWo zs>RDz%7aflCwuO)EqtB#2lRc@NLIha^i>=`wK?m`((^Pz2DiJz?%6-kLf=onByau& zbj#syYVkC!ykp7zNnn%b@!{N4Vw1Xa1Jty*4w~UU_%D6<#msUTApYIo?#9P~ro;Sp z{WLw5moIQ&IUZbQn<*TeZ->N6<1r29SljEp7Zjw)5c?pnvvzKID z>qAT~k7xO}Hpkw|+n!-0{-uhs``4e~y}4Wm!p3enLq=^nO&TXm!T?$B?CU(eqF+}V zg~2&&`Xkuy2bv@%Lt1k)2GEwiGG7)x|C}s+X}&CbX`zhUda`V`$JR0klRUJ2>2<_FpfY6$L~(n z&3MbU2pfeiYFZ^t>)K@Jk)vh7^RG+8m|;>g1P_iFkQ$`U0vH->Qjhyk>_497NI5>` zCk}}e$Gwn2x4T2bj_^?zW#@Ww4v>+*R*b$^V*rtFHu#p@H+kjYnI35Qo<_(A1g0Bp zt2g;&gK^!)@qoZVGyQ{(IJ!(;I2fi7{BRj22g`H-6LIdjrZdj)p#?&n6-WIh4KS%$ zBQCc?CKmF4FHSnlcX})r+sx@WRC-CLfHSvUu&3f7-dTJ#rPCmBhR9s$Nu0P&PX5#( z{XS@R{RwZAK_?i?z5P29o$_X*(0=jeXhLIyj7R$&jkY_ozD@_rx?3Ly)RwPaBMZ@X z7h)FFW?O6`3!iya8b&qBkkO4gU|Z;Br?d84U-mC1zw_ykczy2w&dea&*p+P}r+#fO zjdJ8E-;}@H_zk&1UXsV(SS|aG#lRKax*PuQRc4{r>yY^_|Kqp(RUXIrC-|JtIu&%U z{Wom>ZdvjWUOYJOfu7cj__}pNc;z4LT_gdQ?A(?r*?7srTK+paT#KGzh107@fjR;4nMk9urOruYh(-V3+};^wLTB4 zACi3<-`1oew-I@7QUkzQJF|rvj)^ZAf)`dq!0EJY`c~r>pkM z^4!((Zg=_|>*MwcA%&)SwjqmbL$3@7fpEZ!CSlP>*$f#sa0rR4s4 z9$_}nH?7YH4L&R>_m1VggP#u!=zWGmavjG_-XIHA!Xwv2ddf2R*JrZluxb2#k3K8= z@4vsyoM|hW0)?J|wiWUsmNj-MR5i-fi6gpHq!2Pc3c$V1F1ze9x$B{)WRIP<*4bw2 zQSb`}s>OB3K8N@0b8o_c*@ERVX@?zU@{YSmC0?4lH<}N-LP^1BSS+RGG)?cQg%vUI z0>wCBEyfDQH*O>2x86oLfcBq%-Y#!EIbSNUWze+Uw$jNaT7>;8BM1)j;>h4WW^GuB zb={43Def1(@CAjv^|N}(tMbP3uFN*7l5tZf`m47dv2pCN$I92g{&o2cwrn`=i1!Jm zFUVRx!v{OMeG+KqMV%Ij+l0p*(pHQoHtT@X9t@gUM^v&hw=m$zpE?wSX5Z<6oq?6L z#psbTd&*?lVd8k%eAGxCWQiph#Fkq;igO$X0xsU~L;IevU0?v%cq z7ed!vfQcNQq3oJWT8wyP*2{UEkRhAzX`0zb_C%dr5`5rZZ<8D5JQ!*k*q|2EsCS(_ zNp`_8Yy3DJAj?X`>DDzv8l(yXYL(S!zqL332UlQ2QM7GsWArol8`~}~Sp0fxJ051) zZnX`g9oMgFk`XJLWOIH24inG*GY5NeF~A>(jKW7JHW?+OHXEz^8_*YNfBKxZ$uhCO zG2BW2a%4jH;{bcb|DAq@Hj{JO=dj80{=Kl@*BtEc@Yn*8FxxBPDT|Nug4Z*zr?dFL zW2mZ*mbUeR{Ccq_9rp>X1wYQ;h%M#6_QGCob3|TmYK1QBLou!sM^EsP;Cbq6IPbjt z3AM70_)QRD9M5H%3gkl>gF{nmtZI)N9`g;?ecDR9G3<=LM#=Gc^J_S4J+@5Y1hgPmQzS2j1PQUB+H#<^Pwl{Bc4w*hsgWAGv(t&LBVg8x! z(zZ}YZsKr{C4N=%?gyVLgwp$I{Ww7U?2E6->x-7i#IYkWV95Imu8PPG@^oJ0K|^Lp z_1Mj&vSB#xS8(-cT`q0d19QzQb3+f*d0Gi3ogYW}4^Sr@YyaSVg!^flyP?G5-pTq# z@2POA*I%<5PocRbkvs#;1H0glZh^d>()LnL`yX*5PVWGhCu6^cv zJi%v#UB4%IJiQ#Sm~h$W%>56L`>wn~7X9aSnX=2~;47?%(!LcWK!Y|c6qwEdKTbsX z)2;W*=+UF)eeZi;A=GZCJ^9NceT`s&5fv>3<21+x5&*JROiuj}zVW>AgYYJyDO zdAii3VUfWn4i#$ppu9$m}>C+yC^>eMoM) z^WVv=zYM7XaoUK%Yqs6}x7u7daK=!DcG@^$f;5aCBlROkY5T3g1*{_ED-xZ++ba`1 z?slVpc;BZ5GC#;`#h`cdvZdC37cG<(3*M9!Z@z(cyh8U)s2e^6dm2nb8=k1X$Ts7) z4qk|A`_KL=cHHMP0djkrP>Ed!iO-c#oa@h_R-%;jr?C8<&q8rQn!wHtT3=z<`M6Ba z#SfSX?>Nuc)n)BW|A&>l8frYEXXL|mf~d3akT_Aiu>8f^KfS{O`Ewp!#*mc*JkKFB z<9KoTy`~&;1K@vd5`zhNj<7&wQZ>O9&HLRZ zMDfg^zZ8|3YER;%ttJ;g_ssCUJt~2utkKVO75VlLs{yp`#OFXabr2&vK!05h!Z@BQ zPHqcu%VbPxmZM&9wBvs8Ud(`5U!gw-i21RDgZ9}~<)&}Ce!3!i<+P(@*!J(h%X;5P zGcJ87PMT!ZLl?`kxsSvdor=3{wQayW&YwZ{j~24_Wa~+Dyj*d7rH=wBJ;P1gDk(Zm zQuesPI8UDgLZ`eT3{d;SzQb6rriWZzu!r+r&h=_#ILP7?7TV=6w>>CBh76GpeBc9~ z-7BzlgX~PkIcbKLu6dM^x^ISU{E_{Vk7?1tO!mswj*fo`};3@W7Vc~C*Y z4jT7@C7|0^>6F0jnMUadZbgTzFpg&|`~5E;CA)w0s~8xuH>M@{E30e{teo<$cgQ<2 zXvQ};W4G8+Mo*uPO#wI2b6QVZ;j4vizcm zJHCn#aM@3w7`0MNB%%C158$}LvuoXLBWM)*2TIg zB!5l?v5uIL)mLZogZ$fIw%KPsvY%XZ&CNMg43v{;SUP?7`}P$MoN*J+aa+yQc2qxN zBt*#7J7vmgzszQcp_`8B9w%hTl0$kW_B9+j7Bk9lc;H5%%W!dC^!$I&Zl957Z+%?e z#31-q`)y~R$Od8Vxfjo@>@WK(&niED5zftd;psJ*9g=-Tl;y@|y1NS|_vKzMC9<@NTl@NZwGP2JJjh z7R%wExczqd^Snwq=mYPNZHA&>SoOO6`q#I}AMSohmg8OM9{4iTF$ZiT!GpgkS8ZXGHKz zdQqO2=|THf(yIJi3nW#RS%m#L+loVD&v_>dao{f-3YQ7}DrsjT2?)Zu>~J4=fif zJw)nvIthaY-Q^J_Z0~B>sTw$Xd*34I9o0|bV|K%jhvJB1a5>RZu!lN@%;WGp5LE-NO!HlG|ERj#UVWmGOFK#i1SOx&|C5+^iYG>VKwmAb3;me8g!*{DQa%}3MEpY z_yMO7eu;eN%~?3@vmcb-{p}ukec{ryjsX(;?C!frBbHolx%a+0;GML^WN_Q1qz)P+o_iJ-&{t$4HUw2WM>7=c9DLGl4B7T-r|J**vUB5Je-7I4wi`&1^z@dRlAM4WGj#es z4vCv;npz8b2yhveAamx#IV&4C002M$NklX(tv}S^mYE(Kl%k;^B644AWw15dhU7n8B9*8kUjR;L(%%`r$21T0o~nq z-(Bvy>z}d=A8Q!e(7^z2(g~Qp+}?%sLEAAuv0)*kkk-9HGaYm=$I7Ek%P~0An{uRu z&YuG~r~c2O@*8}K@8#DQ8SVi29oJwFIyUl)ujZj~QOUlHFi#8`0wcyQyT=t0Iqt}K zPStrD&pqzO&*l>+8+1vR$m$i%vTAh;28i&*O3Y9j6n$YXTO{Y)>k3fYC*s~tiR&mi z<^06$?y!LUsgLAE7~(dlm!vqQ9GEyf={ikfc$|=-NWQOOFP85%GT}&z$_{e(f&417 zRuJTO{yg;cXosG)Fa5^TSkmK4Oi6X^r-c1U%uxyw_Ie#=_*1{9 zEK$Cv6{MF<^m3XGU1jLw&P3+Vn=6<9_&oXE94ox%mp&zjPN=|?9z4-Fv=*a)8F@`2GSq8oOn4IvDALxMDS?8T0 zAKZSZJbn8u@_}!2=H+ej`shn!{@?cye*3qwra~UL`a=1@#ZSp;KRjQK-*G739Ab7g zY}qifp=a{D%w)^sfb;LY-v;M)-d(nCWM(&l8#j`l2G8sNIUt9yZcz7bA-cg?pBJ$n zA2KC1afpVB@Dz``aza)(RWwJ5N_lCLjpLE&m{qXp#`wGj^6=%EOPA|3NiDmf|14a* zLT3T2tKowL_BG>+E!f8H3(Nqe^y#{ENSzgmr)j-!|5A2J?sG~ZmRY{@?XVbop0CXW zzHFAG9lWKPj}wo6&Lv7OzWUY~9G?H$LY137z7{^t$A_+kS7rG>eu$4IJT9$Ge5_4F z`xA3z_54LL?!fnl`|nXwiAS22rWB{MF}I^bEtfW&q}NRBi#)Tr)zFdsi2?I#aF~Z5 ze8=n0%g<@qPRkyOlQQTqJZ=~R_m}sEl*R889R|r&p)n3|>0|AfUBx!`$lQ5~+Sdau z{dKp%C7KM6JUvgBVB)bD@?%>by^b($TORPG6!%bM;DFhrO{dAYNt?(rY=^W0TdS<% zfEPCG=cP1!RJlxZ_Je>x<@&-W+~)rm6$0t3|qo9ju;_BG5LjK z3yU#G#|FTqqE*=Uo4j#L0^BvjtJ4m_?Mb>kTH9LHmQOtQlI*a<4lwnW>^L}K^sYqKFB|H z5ksm^r)^rY`oE@ybB~*vfs1rcBZ|j8KNMSoea;qxpEsX@7$bn>{NP&Jz7beyikO!)yLzsr;1 zzdz*v;C{2^&)ENfGtLHFz>UJ(TW;1jMPo4KdgRnibpy^kKX89>sBMq^U68m`+boqI z$Z!Ci`O*A9}i#__bh6~%?z#xb_D!vf7>7SS5^|ebf!OX<_kcl{SUIYJwqFp|rDMaGUCCJp!sYXfO!<@9rhTB3uWqV0dNT>axeX~B~| zxq}RM5>D4)rhJ~?Yq_|?M?!oW=;-yIxE0mf|FRGE`vWCWra#W3Le)jeXjV14^p9>5 zW_sF#J#So1A+Az~>>Su0y_3k&%eGZ7=a4)(4!QEU(PWA;gLL9yl`y`_yS_r--hP*s zVwHXbiE^`J|eOYZ-w?d_7bftytOa5U+l5JK&j=o;N{Kx#kV_iO&7o{lEf0}J-7F#?2o*^ zV?pa9(H8ONA&oKLfz z0WX5;F{Fhf)Dob!+luFxfWw0T+*rgZUc{PgM(+> zo46LU(YUOzvZ@jvh^moV%$Tais+VfSbFNcvKWJX&)UU_4YF6->o zub;-1mO{vm_xK*W%#^PjdzhU4liw-YK>8Vp4gWdn{Q8rR>%sla z8pqj30ES@Hm>&VC#k5U+X^%RiPD~d;%A7{YypW(5NjB7g&vZ{C7S^7x!B;d_FIy(7 zu#cHe?_BVPtO*+}vVo2mGgekEUxv~0HfhG!H>+!^bRV$_bPl@te%4+vqT0}V4QIz$ z2rL%^-L=2^K{29nI#mN+l3QctOA{0;bE>}9(ae=Reg}yB?jnh=gaqD}FgT=M*N6N+ z{MymjAK(S-t78{iWnuvQRB>N1OuJr;P57Jcxl88Z;1_QB5swCZ#DmkXwf!PZb=ga8 zm+iQ{wFOhN@ga|8OJvpJMY1wX&t8i+L=4r~Yi$(vPvG?I7F>v1anSGAR-+GrnJX&j z?PhO>Zg+?4%l^tRxykk?(3htvPLckE~ClTCYrW|*`-tuwm>$uIB z+9acL>{fEbyV)Nb-^`$XmEXqU@-Cb|jzfpTv|}MgZn*I36Xbx&HekjamL0yuF7l1d zVV59lB7dNe{5x-i z42k?+!0RefUJK!89j-4LD(t4u)5{$u_b|9F!+3#RLOyKRsY5+;6r7l9;6H|j1jL9; zh~U@yv<^dz>3JS=jGjTBYxw|q&fxe8LCDy#4S!cPuctvXJP5pE?DX63%2j+*54ouT z`Y!rYdNB8DjZ8!}DpVR3^zH*@B+MkXhrl>J#!X$IN7O*$l@xSizD3pEw; zy@ZJY9Pxg_<)bb@WGe@XS2h`*EwSw#u@-!JNBn&%g2}bOR?= zBL?wYwKy%-9ttVK+l0eHIG$E=(2N!PVapP6Q6G%KAbh?U7G!fwVW6sMUIV?8!6C|M z9PpH*50Zc4OPaUe|D@pzkl*2$K`?34O{EEgRm+zwk(C%g<7VDG;!uGH^lE&Kq^hg%PF?H6siLfde) z&f>Hw*zkTXRi}3*Z8uEEUW2yFHVoK;Jqx&RK^x-PP=hy44Wn#&_7Hrra}{Qda?nio zGr%A@--hwRz^~`vos|xry|w+{9aH42pZ`d{u-!0y(--$o`o*B6KTGkc`=af?VuWnF z7ZBzUABPtU%>;>Z;0dS8(KEsnv_QXP|Hpz1nf8o=O-=R1t~q_+Js=co8>^ zq|jb&B(F>vj;UCjKIu2;4z)4~nM_~`}L=+Z-8`+OOx^BWsS z`%Cb|l_@X0VoTZiwa^VUxNHj+EyFdU4cA<}#HJn9461k)-CqnB(u3>KA??q2u{IRb zC^=~6$Rx;v51s8bBcy6IHmhr0g8?AiG?QU^34%IHfSQo`QO0mFFj^II#o3>gcYWfF z0kyxGty4zh!UJ_39<0YrngrwFs~fAZ$seX|wqV+33m%YL@zA^mv%;D&u-216U2*L* z>*H*cYOI~E!4lWHx*?e2i8uY>%A%bk_#kA!bG+Te@q|h7!88rD9V7HOM6=El?CSQ7 z?H3Ak+x}W!D_3u~)6B2S<}k~jmIVLCJqsfi$zu-5G)*+>Q8bI6rJs}w@M{hEW-Xuq5d z796VELUE}yr)G2GP`2M1Ov|oCzfp@NdzBS7Yf0N|7t*)$AS@{KCw#y`RLjJbpOXR+X=gJ-TKP7iQ^Bgww zkC$~u7fX4`#p#G_HUOt zKe=3VhrylyTfV&K=s>D(?Y|Zq;j~rat#sIfFGv3=`~4oHz8DDOcn)Jg(LDH&!%ztF zMQz8;6jYmMIya$1tvIbo+b;?wwhVD?`zcxy3#@~uz#s5k4&D&fE5Bqt5n0`+Gwg)~yUN`6xy$~v9 zoF*ftj!G!4dF)!2xB#J+DK2FGuKQ1B50t*CCsT*5yrw z*^GNRzFiMN{U|hY=aX4EygKp#fUz!%$>4g<5hv&Xf>U^vUnfhqaGyXpoN`zM1NX&-(6G#au0q81-wNoMm~O5Fw8q@ZF}&r%jtMi~Ns& zzmKQg3hTp``E&5P#cG!exol0%oPBOD7uI3Vc0D0ACgy~o_n?1f5yj2osCmUaBLlm9 zHKK9bYtiv81G6nFWN;O4+{l4Z4!}<{)e$u;-2`M9GqO>x|Ix|vo@2fxiW4H)9S+xlHxFL;IL%zwJY)66jj@ljEQyOWWWiXOZa!gMb$3 z49q8FSsb{j2+w2c1RmGSU>DY<#ZZ*>Sp!Y$RXKt~8K68`MS&mBGagLXKp)s%%wZmZ zjQvRkm)al^;>x4fL4HlQDg$wW;4&!fg7i=>RGh{4va#O14i?!cKedqRpJ%zj^Gw5^ z{J^{9)dh>?+n4+y7p;uXXm7PSJ&$bzfrHWwSXMn0hXs1Dd8h6Th$WOZRodGwY4A-r zvCZ50LP5^}`F3a!KEP3hnV&klGthF+WI6g8X=@q8_5K*`ISp0^%NWo0FycO}cyBrX zhty^==nuN!xzLt%@jewyviUc2v4a?sD7Lg~qJ-8c4-VW7{D(#!2H z-FJG@|6ylaS=WY_LiH2$C7RBfhJfUaN4W&kRqpkpL*7DP5E`|idK%93T2J2y#-`Og zjmU9*JkQ#W=bo;Sfa7r`OO!-~j6@0EX@mmxb?a-;tW9plmt|kXfPc}j=0)uFS2VZB z84dUVjfU92Tu%;NRi@^W_E35O*W%$MvU%(aZtI@f-vsgB4AXf`IbQk0*uUd=%(E%x0xgCKfn&v6I4X*x*IBIoGN2>+A1l&Zpp2( z1Rny2_djC`ylz!E}el8umOgrFtox0gs)Y<9q#1#M8O& z2l?XR!Q*-MVZAsKOqQWMp=>@rY}-Tolc%IU#$!I2iI9_TN%rfw0Xm*TE$cX><9Gq) z%d=7buG;_0pFUI;ELtu<{p~+u%amyx-rB{sOJAzOzJ=>5*mn6QIQ0?gih=GFHIvYG z*^#5KC@{sF_=Vc8?QviBd7gye$KF0Y&mr;L=i|7i4MhJ>WH*8AH0Z(Q zb{Natw&ZE~#J*?S7bdZx|5@i9CWq}jO{NUvES46zy6G7C?CrE8teO}W6gE%2J-+hF1 zTu%Y!%VXju<%t(LHu2kA?B!N#754u_K505#XRREd$_C9jq^%enCOk5k2kq>TnI0!Z zARVSDw4{RZV)>y+0;n_PjXFl~cv{{a50G^ml0p1V7?XVNAxZ0#$chK$yc15q_hI`l zOc)(xRXw;tzz?IYChAdblHRxE%&(I(xJ$YodLmfu+rA|uwwjWVShwsRS#Zo7Rmr<=!E^_=L(M#GL)*%ebAv6$}+8q-gy{Cx&KlgJ%S!w zAD))vyMAMV_(DzlNI=GkqY!ZfmhIDU{lDbNSND_gSOQ%tG>)4jBWlz8nNs+DkxzZ_ zew@(#jrF+O6EaYHjGGpjk{dj)7=AWi?UGBP%V&LW%PZw8UwuZ34xC}dPIKM%@`W>x zlBs=eM*6KEv8r6-mKN(IYty(?j2EYSyi?-PVfZ$g17?h`9Wp`7w>K{Zt67J_O$yW0 zjjVGcGRrY!nRRE}dE4o7^>_bQKJvv2%6^Zeb?dNiQGo|xm)kos>cG=fc1{`QzfT=T zV_c`57!A%W>c~oHLK3%p>+leFU52M_U;4vz&OFn=4p32zrX?aN+X1jlNOPPVS0kN> zN3~;w?_~)YvNF5f9p>R@>MxZ~nT#K;E@;T~Z1tbo#Ayd&*NGr&hzMH`bP+AHf?)T&Uo+natDdL>nJ&6{|Q!>G>F1< z2KANRwSRk)Z8y+XJI?AT(_iY+{eqAi<}%G;%(k@~UV&3zQbiJp3iS}jJ^{)P7*XlCifTx&qEfpEAK-UK(;A{$ds`7UK3>;H=$=V@IgW zGzRs+BimM7uL&f_LC1xdcUyZ~rH^=;;=8RKD!oFKLYgsU%oRjS38`;_H~sP#$`N{S zzwq;c0M!<3JCKPwbe%d(WT0#Fpj}*F%=9=RZA*P9-ddS7c6gFa)IJUUJV3n31gzqr z{Hh;9=z;NWg=x~b-zhR=^pK=<`)hxbg|}T*^Z`FXEEXXzvF&It1n;+e2D7C_e8={+ z%+%&Q&*Yb)=WWDcUVTi+1i!{QR9_AJXHd)dl=?=iQXet3#mrJRmoH8)7!O2hbxE+u z%9VY+((6*0J?YYdtYj0nJNF)W5k8pLSt9g7U7iVDYjp1kbl<2M77 z*?Wg)TyV~9C5mk)hkXTuX1)Anm8Pa9^f&9U zJq2ENQ_J2bksI%uGMsroLFnz?(}>D8MPEaulPVFz zH#?^G?1}x!jr)SprG82*CuN~n9t1dI4vO77FGemeXHtH=b!mKUV)I&-%bm-%dMsa+>GQ5Msmje+cM?!pjGhHTLIX z@~?SK=FFkSX37oUJw&D_D+RnQI8ox*g_W@!U8=HKf#D?itlta)J6W92Dc=3$VmW%c zy=XFeDLx8>It)9TU%3BUDOZ2&kUE1?zDCvY6n)fMJ*mF*Y-lqPBV z*Lkx1g%=8;cRkH{b4UttJVPbXV={FtRuoY*vuUWX4A?6vFG>aUe}w&F??hS;vOwFlnCwc{D#wKtZ?6VJ^Tm(}xo zA8setKYEpH($TEJs!_6aFAtbe4wtPC+f&d+dIwAW1RW(X(hv0(AEd)`*I^7jPMkua zQ-fwC&d!Dhzf8t`cvjU;lEzif>JrVtHSs=PFl4;oH|t>;F5KDN{Cn&)LvFhCtMY-* zpD%B&kBel6pU#|e8DwWWTbGM;VKy}qA%iZbM7+>)PDQ{^jse{X0(9CAStgF_x#I$o zR+b0i#KC!U7?&|T^3Ga9BhEP^2J(7f|1cR(zY<7lT8|24^(AQ*F_|Lg&&rqOKbs%O zB*5i%m@RK|ZRHph2OcX4>!0;tksPOOjpirm;RQ79KWakdmC_w zrp7k?mh=BaL!FnuqE0s10_C0^FXQXe_M1{_l9!{8AII(Aaj0uEPgjzP7(^mLz?pv(`QvU*ygv{e3P8EpODKk-?$z z-&ua0NWVkMfF1+DRY-Z_!&yuQC5ej%o>aF=0*Qd?{;;>w#OfB##tx2pnKNt-K_SFUN^ro5j>^qFC}V7l9i z5C<*kaMZ9N^5TmxW~lV?=^>w!YyWsGb`9*G@X|{!=>YtvKlE;0HCI)MYX?}NuJ{bj zuhy=@2K;TKJS>K9wOY@>&FengJ8 zi4Weh^4xXqal%p|c@I$vk;G{@9;|S6{S+1CnwlC&bIjJjtqYou{a#E)LM0*vFHpIXH%6jQKxior73`nqF(r>i`;69=8EY#K){ zh--ZT#&v7i0{k%D@feR~j^oU;;M@>;NrQ1tifyUH;NM$O{?k5lnEc?&pTq+-Mght$ zz#^f|I%GSF+gBX-Hn#rSEATPe|!w6`dOyjCbvGC z&RAfZU%lWza>lV=l(RLzR$7{u%OAggiCi^rwNhXy3kfyQC2da)5lu_me_IQ0yzpjh z(F$n>q-EE&u9T*SPg|5fGzwGch}sD>c_{lB_Q4!W2d|BD>` zt#8SZo7d?lwd%TFkz$YBh<4!lsKde@sWt&j!b0g*E?*zDf}9~q57ep2{x59+Tl zckWynKYEzd)>dI4suEwZuNKa*`Ot4cjZ8%22@uzJvMsEM* z8FJz|zmQ+ucy}mV9)KJ3x3{*aK6JKN5YFSt%rO@x-l*?poAY0r_jD$``-1liYCrLvqTePmn)< z?89=@){XM~zaNsbfB%pd2E1F|y`Nog&O6`;R54guJeL<|+QOC2vpoGfvdGl!XX4}` zx#MT&%h<~CBtfbBlkL$Bc_$#|Fy~e*Iq8~{@%Zm3a&W) zbh+a6il;5!{Acs$hSwK^ny<;CmCwsS2P9E|h=KFjdIn#(r07wj|h zUSsYC87M1dR3AgT=9~6>`vuX1o>lKmqF$Ni`prVm^KvEddv5>4@?gpOE@P9F2X3G} zcz9bNEE?jxIG*&`qy-xf=H7i+-4b5X?*bp#(8HFSPSou!xYfmA9MpsQZ zP!$y#-0@=`zl7Y`;I?L)(st-J%)(M_ixqkqt(Rm>Zzw>_Uzdj?~9*~J+Uup zG@7rx@`~=+w%ryJF}PHvFZ64%?3;c@7voGX&NQQ*R=4mz&-Tso<(Wv10?$~=Qhi@? zg~Xy>BVYBh4vErX03xsN9LVD`%~}kI@mjC_qP7S9Q9F{3FM;P0&I#E2lgrs@#|<^O zY*WHgcI}5~4=PyoURrW1Lv+_=D5^%Xv|>~`<-^|Tu*2a^GM6`1*A11X`f1Y8^qjOe zFO!P8!fz|Ga*!QO3mgTWRaPd&_#9ci^vwT}ckVn>zI5I%rFnJo_!3I@4`4^bkz)Qa zI8?i_;RT)ZZIQ~u0v_VHrxk%VWz!n8qa<}9VG!QRksM(XBKqzS4i(4!!66f)xOS{C zO40#)+zqApX#r1msC7g?DnHPjLzdLEfaB#H=HVm5(&9SPc*1)~(u)FU6MEPvf2xZ_ zEm2aE28H#e{db);UH*B^new@BUM7FJ{oihm{)K2CW!-k8_8Yea{{k+R*O5QJ75pC) zY&39#{ls@W(f)V&=>N!{zUm5?9A`| zmmGt^CK=QwcsZ|ObX~YydS``Hk2(w_6^xjOv5)s8INt6ZG9!w2)c*Mu z*wz(qNK3^~88m#FR99CggJuc+GK=_ij4+PpvPli&P(3)bBu!&Bg4&qbt}hItYFpI7 zRKw8=a)=rTg?YlZtwxRrvaA!#=Bwqkppino^oEK`44Sp$!%Vob-v6CD$Tc_o)ABdK z-?wcuO~#BIiiZZgu*Gs3%CAdQXam=s>)&$Ub@aK+t+ju}U_Ou%cbG zzs?$RTIi_*pAMO$?LE2%73ELAbC0X7!?~yP6bEbY)*1U2(eld2KBWkOKSj;$`}2HK@dw5ad5?AB*R(jz<2e#R z&rGM@2zhXFhrWzJeCB486#dRS@9fh-vmBn*;|b+rm@#dV3~wAFRT#(MU=-UK=85Tm z*;>rhuf!&Rt8hu?uo-owv%i1=jp5h4(cECO0fNf1nt86}HoIL#S!@5UZV`G1Q!UxRt zL7*-9VqrSuzdk;wKK?Lw4J4x_sf=zmR`E)W_2f zm5UNToRZnzmgnnP-Z4>78y2Ak0!Lzk#|ato@{vJH7SRnGHj8Ql*YbwzjzDsZClK*G zdjuN9VbJ^lHNg^-sIx=pS)Ru8tbOz9i+CtRAgdCsLlNz#Bz>Sc4K3l!1%~tZ$>494 zpBD%71JxgYg8wm^QKx2kn&qcH#`BTr_ z4*>)%(VsnR^#2<9@L4~X!_Tx1NZUFbknNU1eK$Z$KnnA;{nyA|A3aJIA9R?!zGStu zR#eM~VMAp^eTA0W;jACZ+ZVUXmmA^8&BgnNsJNl0SX3pB8N^ZDQ^k9S*Ol9>wk+)$LoGi&(D9OAjyHA z_@1|Krw=yVFOxUOsd6cqE83vhFuf6Gs;iLLim&aUllZKU_=!D2JhY`Oy-wNR7*T(7 zpLEEqI37?33~=5F<9MzBv}} z$eEZ!TN4!3evA;y_`&qJ-;d)_!2ubK+ndL81c7cjutJ%2AV}lH;J_J|KTlKi+itr} zPCohMoT3}T$*s5Es$y`NcXd?-W@A-gDQ1;kQ>YWpGUHzDy1hkpi*2XDC7D=NM#r?f zD0N!(Q+2&?O2Uz6>tG~lCM(Xg=JhMH7LsnM|lh$RgNPA1O8jehx(Om#K z0BO=8(m2!f2x)Xn9*Pqn77V zP!F~=#tGwi(R>VtDITD%cg9!$0%Uz2C)9%%6vjhPeNxZVCvD(%aL9DhcNoW+R_M@K zZKP+1+DB)+nXUTD{;bD*fh?0Z4<0AX^&ywr;YRtXGukac#$%hu@j|xOvZm**+W)^l z@Gg1i_vgsRF#z7Lfqs;NUf{ws$JmC0EC|Nrd0 z2b^VBapzy1^W;&O8A+oo6hR`$lE`2{CW&B!kuk6~_OjprUIYKN*X%kjUV{B`+7G+7 z2{vGZk-f9Uumlb@|Qaax_e;QPSLGR!`d3;^`u=Lco^+pPfqUyrMG^FeKV;aJedCPkYzJT^yp9XY{2jW~aO6p=a(RT!dUX)P$QP%=`z0-R@FADOlpogcN z5HFWa56?r+g}w4G;&>zM$ieA>)fP2iX%ByB)BMJjNUTU<@Za0rW0ybWaj@~IGYpE; zO=sP%I6-OFrC)Sko`+e^*i+smm_qR+bpF-k?S45PcDw_6(3Bljex^J_%n6^T@9_>i z|Ld6)19V_%uZLUA@AJ{#j-RS7NWdxovXBvJ$^fk-^qg@>>r7ei*4*RrCqFvNIeB}o zRt}$}bESHFRo1nllCmChNCl8ki^Zfik9sg~eo{xg63tgv=FfkxUb!qdj=&R;0FsZ_ zLm)pDqY7frCg}6BCbxiim-ZZd(SzV8wNqtVhcu8$GEw)^TdHI3CK;v=F0A8BifNQ) z>sPPPjg`mPzy9mL+P-~{?1$~l!eIl*?6aT!tZ#ij_q2^Rq#bO-1O2vQXu#b#8+65R zzm4d9SU+~SVN^oBj3gza>^^Vc?ASE#2e$i>o`(_i{8GPOLANvSz)jG9VNc;mo4j&* zv>Rc?F!bq{>2)ak%3*$k`oT-Tc;Z~PUVG97r-8yg^O?^$d^A12@P#kv4Bu(HNXkz> zr1-})F{qZvW!9(xa+Lf_Q`QOc1LOG;`BcC%Ufuq%R+=$L_Mg5qZy$cFZQS**Ho5(F z8#siBdBcpQHR(!R<3%q*DBB8<=q-2MM@Gr#QLjKGAAHMuwt5rRJ`kSwF#o2HD z$;a%g-@Ijk+&?nji@ENyMBI(U?qypqKqkvx*3oYkr0w5L0h27V>W- zMm?atY;M!tvG&MGxobVWTF?bwJ-d~3h0`4rEjuiPe^ryIRfT%A zSYGn%C;0?0ozR|8%{<&Q7 z9wyS`J?sGkXM|k}5Zd_#?Z7ZnxrEjIs~4n8KH(iO2c?{0aDM6&&bGh)#J|`lKJf{= z^2#gS{n7AX#ns(+-)%qf)C-f|qEq?{{d^@WjLYyWf9@#RvZJhSLdpS&=c3I4`St}` zU^3yG9LoH|zg=EBEqd^g2nD(;(FMKi;_2~7M}D)~77=Gas!v4$zh>32edHq_vG>0B zz5Zb{#aQ~!4}IuE&crh=J||@uUsYTljecGN^)N20TqjDIC-)?~s6Xxe=aW`rR5V|R z!X3o1C4(=`nB8m6wTV5~neDt=x3iq20kzJ7sR~Qtr)&(Hi%<3!o?H$bXZ=z7feY>G z@BPXv?9D%Wx&7IPKX3o*@4jGr_HonMQ8Eyr!LsiumP}$;-Kmw>6$^m&x_KXb-2}i*22pH}qtlJ$Yj+tnWD|^NfZ`8JL-4lG3d8{Ib3Me-;ILJW?g-warh3GZI8 zB)NcqK>O0^N2erT)RdjRMjziER#{TTj87zxaV5{XymQ_} z*=&)ou#i*p#Sa=PSi|q`ISscTVE$M5WC_{=290Hy#cqdYr(2Q?7`YO#!9%JWNo=9y zq--Z;CJ;lOsEGSnb|{DYv;wPF3gTD2;4*vLyZ+Y09L~p~D2NZ0j*Z8xF(!e1+K$yD z9yz~Hl60AP0gd+LP!l=~@yWO5yi55R9eF5a18dE?bv|Z~lrBB`kilXUlx83Jzz2?u((Do1Aol+E zzu(1h`BNX~?MR>cl3s1kr2nKJP4Y@I^!ledh+(eo$E8r^15(@{KxD+9O8D z9~k!VtHT3Unnfk#%Q7aEW<$dx_Q1#!?ZjPQvhl6ov%wQC^h|A#n~F>-I>f{$+|paP z0v9q76S#i$i2c|9_!9f&pLwqR<;VZU&rQ1h-Uk;#amfC&(mv1nR_9z%aw!g;1GXg8 z%zCa!-aUHI$kdQ^J#%wcO0`L-OJU$r;g`jOP*rhEkmtNn5=O@{7lCjKyW?xkFIe|v z3;%FSC~F8(Xn7GUo(<@Z`+2lrgH>oq=$u}+7xa|rw8WQZxOuX|v&_FFQOL8(vckFY zM)t_CkoSS+|D_k6VV7R`7Q6ntx7(k8ldWHNJYNpWLp4Ce&T2&i1bgF{ z2@ie(wj}3F$~6Z_b?NZe-8oGj!!&Ab!xm*Vi`)5ec?aR-N6^|JH^{EadKF(`0;i{| zxXb0g5L^f+3oOJOFv@x^Mpa!1|AutiOFY5_^gP(lqejkA$^XgiQkZF#*|x#+tbgTk zHZ(NUv^6vOtNoDuhqZtKa7X2IWqZUY zi}=KCGVT0x8Xhj<0x|8vC!gvWv;_7=KD*^Z{KqsLgDLWFC0n0sGs({aZsZec{ z_&jZ8M``w+_q^v(qcr>TKmT)^)JpW{TzY|T&GdFM*9XLbxI^;(8u_J{vd%>P4>!Vw z9T@(&w=)M_Bm*fc&f+&Ve46bZz1ddm-J;vIwp!2dF)5doVsx4TJ@e)21;YJP&_?k} z6VVvh5D$CAZ(%}+R%uW2ufFDT`<0)4o?jI3vCn|%&0qNo(rdi=w0StvQec@XBhcA5HzI@%y`g&3;%sEdF6?mOmq;NW7HtJJA=!^Ol zutBqWC{+~C|Ek%Env)y?4-k`{#aO1kTBb-pr4xt);@YquX!*2iHcE&4Ot4E9t0(MCWH^^`GDqDTxNmYc2 zy?gDh>%L(R-27d;@1}cf!}%w;f+6yf@)#?YQY7*oTaq#O%duqGqkTP4`5$`zQ_hrX zl%otQMwgfc`9~fEgoU>J!E5jvuw7VXS(+eH&yuYCW=ZAoFTUd0_A9^t{wmGGA+7^e z5Fd`TN*s^Tt)S1m_`I}|MDk!F2;UT0mmftp72h)2BWKU~M?OPekCG)LH_0BNIAJB93X84=;ZTU#Ey%GLTp zBGVo^4pwxs)xXjdY4UT1@p_F`%FI)zz@_HIxmYHstw=&Q5(PS zChIyyXP$JY?VI&P>y01fs9f1FM`?DqRKO~Gyk@C*93iJs4x9+-$rqhzPr2wUd(Ydh zv}c$Z| zps=a#0sTFpO-+IvTF4AH58WWAA}ix*_&C-an#kEFv+eEia{ZTBvOpS=a$7xr5O3ru z0S=}A^Z17`{}~jHZTXgnnzaX4UuwfE*LZ(FI5_B+0mV@b&e$S+q$$h4{?{9pf_U1}2&WwM|4Uas&+G4iwq~+olu57j zg1&vu<>mp~qIg6c=@VRO;Q8IW{gNz;%2Ds0k$)n>-FQTq4EKOlfF))L`A5A82peqCo8KTCd=o?t1m{dl>Q=A%lIfmV zyX0|a`q_aW{pd&S?Qeg(op;`O^P@bXK2NvvPdM0tJOvs@_GYD^7V^#=<#jL`GqHsV=;0hRpvrD&pJ?Y%j z?1|@|D!Fvnln(B{<<9$b73;lr^X*&guKTvzJ@;?72Y2kThj#7t3k4>prX4TmC-kY% zty!S~pbpeKZtY6jaLj7EV8t4a($L|6`KbS>ahcClbgO*uOhN`#MzjkrMnX!Saqtw( zIfI38!aYMzPu^90g>R~wS%%28VLX_L)79|uz0xg zpF9tG0%xKns7?N!;miC}=-Opfi(5^4M=bx>j11dLpY=rjo*)`JY{%}scB8IQ<|^ge z?%ry5ZrNt{X)6+Sch??m<QX>rU|%l(NwO{r$uZ=UrHE9M<}-~3D^!-iys9qn`o3ELg`xnG9xQ-8tA-fBMJm`x@We_u!6{)=_+p@7ig%U-MC!+#Ik{IOu#5p*?UO<$|t`n55!aO z6R^%m)QN!POfnfpzGdi#VJn|~MU2{@*Sz?tc7u*}xbMNkw;j1o08}tApkK9g*vYdU|3HGG(PqVA9zS=(g;Sbv@U-`;J*daVmdg1S> z?ET&$U^w`wT;lNCzx~@jpSwcyL000X74|7V8AiZ&OPz4Xlpf<_uQ!7HVcZR2AEY^G zr5S@|Ob|FSGGgQ7xIV0uR%-oEPj!o&S=A_+y!yPvIgY7=RBTR$?CPM~b zPZ_}z-G5|DJ$#v#f>kn&isG59^LUE9wEzsDQbgO-kaZ=kw?6MDSRVF@EXwlU0w;#Li_}#fD46h zrf%W4pgUlS5vjF%YsMMz0^-|b&%7=*{#Qu>cG-nz`mdSe`QNC&6Pj#|PfSUboTb2f06y?|M{UKd_JU;-}E@`F=yNO*qH6ns<1IBz4knCzyF#7 z9qfw)*55y1E2MlI(GR8V?YG=)eBd0Hl8rfrPlmJ-x5EXj1VW^U2zv;yp0>M_Daqr= z=YJxsEQ6jVWadIqhjo${=R15+w-R+g_=tK}wh<^Lk+=ITpr8aB;ZP`nuTO2xbe}Ga zn4Y!Ox{3SkKmTHT^E*FiGxX;L*4t*DiZT>n${$iaQc$s!<}^Ri z!Rj&)&i3hof-}}x|GHss3;g^w;RQ3%)%B-huXmEeEy_Szaou5k{aR_Je55U5D?8h5 z-~8q`+jF1$TwA^R(6+k<{YMkN=}m7kwr9TKwa?c%_BtO``Pir98@RNAe1m^4`$1IB z2V@qR9A86)-2dy*=I zb5o|gQD6cG&>LH?uFfJ|Z{=2oo6o*b{Du7Mcu5}vOq$~aSq2$jMrYJOL1lz~&xGOZl6%0%90y<$=!6TTEb$tS`$Qb{zEsHdkd2aexg617l_Tgaq= zqhV`HQ^ME?RM=X>2meVviJFvh5v{TYEkMjd^i>hFWuf>B`PcEa@{?$)hl}^t@XP$C z^1m4Jzfnt}rbBgtn#3TRueyOE-1ew&m7%f(g0QUAy?58bsa z!~O<#ob$xMk?|J?i8D6goFo(qzy0U`;A4jdz0SrFdyvkswq=g!=dG1!nVOu?;k{G7 zf{b=|hD($fKPhm@7kv68P6ruxX%a^n>*>?@QRNLkJzSon`}R8h#6@|6^!d=hWFiM~ zNFZo%H(oFh7=iQ}vUU9UgA5Z8YhkZNm#?RgVGbOD^xB#pjw#(_dvra?laOiTZ@`LE zl}&N+RP{ij#{xc$)wKSRJJV~$<7(%zy8 z;NW0?r7UC17(A*vy9^WGutXW-w^&T32SEM=X<_#~YeQ?!%cxrilmB3Al!d^pgac=G zYL|4Bzkmewv@;BQMzQEY$w!^oIxn-@ZRN(#5~96Las-AU4sv7V4oV2Nz4hz&3qSb` z`{wuVFccP-Tylw>b=FxCWzj`kwFj<)dksGhM(KCPsK-%^etH?Myx zXl!%rQ+=G=w959a?6tWEN3Cn8Zgn1Gqgj@zaPX0)lQ}vv6jFy(g-Bv@>8efx=Okl# ztXaWacs&pw!={INY&z|EX2kt`O4bouG;hjkFB{+0COi+vP-f#spmKz(|~3 zHm`dAW%hqQezm>!wXe0SuDU8kTp~}l*K%X#1!tXP&%X3L-&)(R?X#?`=R;v?FJVT4 zpdLNSOHza>WEpZ|$ECTWLn;$RLNUncSqwJA-NlZ?}y+%TC|@S+fVe zC&ig=gc&(Tym;=5ABqpz8Jx8zELW$nV(^E0p~*owa%o;2n+3&oGT4hW`Zd%_)5CI95cvlV~I zZG>LtKT1!lyz2QGl3slI-=|J+?AC3zQaq23X2I%6J_THr$e(XXIY1y~G*eg7-$~hD&SsrIY`OpJrLK^7Erz z|EDLkCGMVY`rzTtmCvw_q2nrr8T~x{J^eoY)?zm?7!Eq)Jz%&8toO%00%a7HdG#hh zfg7xc3<5TJHUc&xG|7z?vQe`d_6l3X3kacworw-KGvUU0dRi-Sq?qFO_;XLOU;3Hn z*?T|mX{AxN2p~B!W_D^)U(Z#)ig68Jy9aeM;GCbkR&mxuKp7s77!v_!r>C0m#Wqdi zxbQrDLo9`O!n=a0rK0|qbTsUBI$WrSIsf4=(g{APPtFgXGNnDaERAQh2|&*~DM!N% zF(P)P?Gg{M2gJRwhuA%yVkE_xR$+8&+^tDH+pSQRKI@Y6{QT1oT=oB!gpgJa57=+N z=_mc1CC(k{*ZD(C=zVLPCV{2AkO%xDy2WCkrOcbiGtXY=4MNL6a)YlZ_Z}qjI-rM% z$jxaaS5HCKoI(hvB0%op)x6 zXMGyCP{+DE=_`?8;vaF8AJ7TI$}i5#>g}GkX|32A)xCz3Yt)D72$;@&Q`R*u)!Br8 zlUg4(EnJ067m^JMvYB43YSWoztSXxs&@H*cy}r7Qwjiaek4E}<#CdaFtfcas5@ymN z({g&XDwcRs2h2)3+DL-(k>RIwxx;lg-R8aYH)b~y~%TYmO( zBM*H=t`vej;?Y-`A&#`p!DZui7{^FXl>O(jKEAkxd&E%U6mhFeN+|Nae{30LD2vjGVmI#+ZcNwcEyLI&56o%Q9;~J0nDtvICY=8I<+-)@LsKxjHAp2^avg=sKX`fz=di@qb`*=Q%sNE zf}t*#8r?>n$oO!K%}~$ zde-#IMVvCtpS&xGSDNG{=s|13%-yrXWkfZ9A!}5?9Ha!Ep27ezb$kcnPXZcxX|Syc zj|SZscw}q(b1ysJ-QV;6PwPzA!*O$wqo4QThg9bqk}8=fv~(xVBM?&;6AtBIH|!2` zgm4o!$B~?sYn~8-PWF&l6{xU;9}wf&L)z)}i06NdTfX;K4gS)|RQ~){nYlcY{tnknZMNI<9uaTL4mF+H95VMkEbZo*t_PMWBWlKR8 zomnA;**o6&azALDN#C&M6Kokvt6-P}isYQ{xfuBCVk!v*i9xE%IcN+RGXh}4H0%eH z|0#aN4Jg+vEB6n3z(%49mfC&NjLC9%&sRRff3d>4{DKc)_)TeMikX|9Ol=)@{X&9N zx3u+HZfcyl3;?CXpsxs?v8O%ZG4|ab+-;w{`g(iW%U))m{NyM7qZv3Ct9kwFUvFRh z;ur1G$DLtUJo||@JfK^crTFS&Yo_?~i$ydSz(jMCpp;D-Y@@8|=+k(7Sn~q4FDRdo z3CcK*1ejT&Bb#(u|BO1MX~~a%R?EAsgH_LRpbzZGU6kHf!j}ZD{#nRNo{y}ihd19w zOT9+E*&v2=ghg^dHghxRY0{!D_SQ+73aS3mmKef&g4~SkG=L&_>nAh}4O^>e`oB zQ+HK2vC|~ex}^G>`10~f=LlYf2}4PCKRsxbsJr-5XTcJ6x4t}kR%uTUt6f>G4q@$( zb*hs~eE}^1YQIP$rI6|jI1%%aZ2wXUo=`?ynQEBTmA z%FBiv8xaEm;TrZM)&5QIdq^L+z9r?wtlc$qiR~Lc-PUSr9DO-k_AL}_3DiS?jDI3nXm(8>iCoEex8x-MS7$W=>@*@ zVQ*Graw!qInEY@Xn!aKyFE2j-boQ$0!Y|_}%WDq4B!KsY1*N>l59g}P=C+J?Z~L0~MxLeciI|WkIjh}NM4xbK656A) zhI%x)oK`u1r4H3+g82SV{A(rl6mxMr*RNS&|Lu)0vlEVA>&iY~EhpN0deQ_MHd#69 z1qzaJi1NhyqA?IfTblATva3#>=Wh;!pR)pcIpD*~f5op51k#Mq6Uc7v6ak5o+J5gl zWoLzo8_|Q?qrp%wqU{L#bmx%tCsN8x?h}jDwqydry4wZ4?8&=JH`TII$!{i9AM$fA zdxmYwMtJnkGd=g zUz18|2a|%5@e|b?x{^EFVxf`Sfmvy$zFVu%P?-7UK3W9RsmTX(0PrhnO2_TVwNZw% zK0$;McUF1^c%M@lSgQ*Sf8!Ugu;2LOzw-+L$j@gy;~6zd7su=U_up^Ne)h9{wczVt z{tT^}KiS7KwDs`CLZDa-gfvS2lihPdI;RO`*tG4N9dyQ`9RQLKQD5*3J8&U$Sd?Zc z%NS5|DZsRDu%l4nwnx8PI(zN3yx z@dD!Kj1s0z`0SW~KbsSXqSy`}g_;sd0|FeBwtBozTU1oD=GN+K?y6bar@402X!oMpyfV<{W&)5So7 z$U@ah16^;Xu$Q88-^L?_PAwp=UNkcFP!Ttv2XdeEPy|<(=0w8oP$4{L6gScVjk^Vk zz?)%&X8_qr!=i-+3|V3V)UUI|800_UoKx-HZ~Ym2&-?$ceebp{%{T-1VSuxY{SF9E zjkDToMPLY`aF|gVVS<04+Y9Jn+Q}=2!M~oYyG0ycY9`1Dc<}ijy^4AwWmI!ntR=Pt zU(1OH_V~ey7kLQ|+3|DP$>Q#%o9r0(&yaT?ifa-`e@=NvxgULdUynY1>HTF|S+Z{R zi2do`{IiYe;;zMFIE?()|HJd`#0~4cZ)dv^iishufB~i-AGnijpu6P1ERxv2`GW@t za#T#@Pb5bAC2i!T8_!hPDuJ+A5W6rq& zS){0$%D2(~`hR$V{ntPFd;8!AKWJmR=;cEn`jCH2^9YOJ0q?)`OTT1)`lo+tn~qy! zzwrxK*cx3F!ntIOrw28bhA))u!6#(HaTFt$ryP8~tQ)C2I#`+3rPWQnHaywqE6OHa zw}App{T#|OeZZO4%4$UGE6x<3{x(eDCtdnv$}V=&LOJHje-uG-qYXe=hC&zR8Ol9Y zx)Dx4n0lxOl1O{1^B16YHhk#tK_t`^qkW++W)6Zg8rM%I%O5tM?Ufq zd-0239ONzv_`@Im&=qE^H2bON{g}P*=@;puqkjLb5abd7^2PfAQJiF)WuEzWVaac9 zBAtJ_cW%&S9(4sqqwaVcfP+kFPcH4BhpaRMEySQSixp>V>8*++Th|l~7<2jIn^oXJel%d2TI*0DkY;UR>9wCKqP9R4V zDAUh&ni>kdFV|GTrtnYet16miBFrcgXIUvC`;_9)fvYxI2QO1nplR7=hgPHck4mL~ zR@HrdQaz^b%3JkzPU+664y%G{YY7wE zR3GUSDff8MVfq#g)m6?p@uk`NLewRH!lepB8W`kA;y@XvIfqFu*)3jb;j6f1CKdLu z2J|!nSNW0yM7V%u5RcZJe#K5j{&M|)q}o50u8-aSU7Ozfpl*ulw_67;(@E|p*=qIQ ztT5xp3bQ!JtSkjf!CsCnkTJiQTtL6>!b)uDYouywOU;NP9k3ptr7WvVduh*QJM8&2 zS6N}p4nF9WJ*Av#H3UU^9$)VfANO!0T=@oXv=?(~LwePAOw-<|Z5YzWw27%sJ8tbt zZ4G;wed6=qu#bQK>qlg}rr%18Q0jtmT8l?ba2}%}7FS|=oycvi4;H+g6@Oac+iXHw zNFzxoK~VutA@?Ryk-Tk|Ta2+vO2L_8Cm8P~tOp3(&NA_Z&~fsR344B3fTU8K?MOQ^ z$e$SZ1avN(;3`%^wgjK#VVF6VdC2(KnB^SU5%@eNVg@8E_7sJc1}firr%z|QRKGro z5r!MKPtaDL_i9V>?e{)d#F;0dBj^mhS3K(p_EVStm?rPKAyc%nf+VdXNfYFlOxMW> zzJXznFx6#2;cyfoV=3rC_5nTllrfP(m0!@_5*{28#}PC?B3{5M|07Nv57ppwtHIrs#c%P%%oM^D9AWJ6u(fG zF}?^x_myO-GAKuAU_Q>69@>z#E6nmdojJR^X+DtJ1;#9?-3*-jh0cIN2ojA{yEP7g z2ey6o>dey%&)#fr`?(j||NN_evX{Q}rS?aE^hfr_H@X9Z zi|wa>;*zxD&)ZBX7bT}qI<}>$;2(c>PhfV=txEGK*{T2K&jTFfnYQPOqUs`Ph@4p)9j7EJzQ!tIq%;W!i(1sOl~Z&MKm^ELEn zW!cA_q(P#QaE8vS>_7~IQg>@q{ze@6w}i)yc>1+9;2}W5hb#!3V2ZF<^FL9?vtVwS ze~F_kFA~op&3`%|&M2d<_Rne6mM>KvP<_=FCMmC|w^M4Lrt}qty2~PTe)k70DNEU}qG{a`(^Ez@#BIq7+;dI_ZAxMObh=HawLxVgrg+kvv zAf0d0zDch244Z=gg4=yKu^*)eF8ll9KgegjWn81xyUP7C%yL#`MYpIwmY?4}@e~0N zRAFMD#rhszO6bJ4f-FUP5$%$1pYhf8Q?9ibFdJ1~hGJ;eScyfFb9`5dv$H0WOZ&fZ z<_HpB$mpiq*}_1K+IZg){v;(JbYs*;~-w|VJ8X~G%u@m#O_wq4qC!<~d}^R<*>8JR2? zmXlXo3OF$GT23@{jN}Rz=Lve<#eMFP4YEXuSSG%PUoE^vMp5t6K7V+x-L#Sl3nG8R z-`@p0SzZhidZV9(h-mcGbMtII^#HTJPkwqwEk(XB69&ysT~|KM#zSxtPECN(i~ods zUGG5qCxHWOzw?Yof}D|YT4D9AY8<>-v!t_5&hetb^9J+1z=^=ybtn zP?=Ek-*5Tf6#TM6e%#?BBy>~bO}h0(*+%h*!ktdc->%YaeH5DcJL3LMp|51}yI0Bq z*NwlxkXBnk5Ar0%`I79yFpjjC88@llE^8!HdK@rLv5)|cQe3zvGF)u655h+FZkfBv z0ibxHRNIRE=RutmYT+6Ud)Yxp8FF^wRT&nVMI+^dbXweibX^f>gCwPFR5qH~)!Lj? zH*qrIp^LlFK)4JpJD7lU7;>s(h*5o1w8sko1Y}iMnB`{Rb6wi=_&u$cN-4x60=(cT z^=S2cCUQ>)G;;MRk^BcIs>Vpq6yf_W=(clP7&MP#__9S@(m=7 z+_S%$UMk^1PSLoL_E|if83yEd8L~EvWTq;QLe?}L`>nHo!8HR(PS%Qs%cZQ1mai=I zcpCM#l6wc3djmOT6jf{-uQ@e}sM<&CnMOuPKeS!{UuGAOFg!|-Uu&BPeOCnwC+ZOi zeq4Vo#Vh#pzx#kDp_aPNWW!xcl*PJ`JP~f_TAAQaT*3sVkKQL+^LrAX;nM~$ubCET z=;^;HbV-HL0qElmzj@8s740RBq*l+9qS$asuT0~|jSa{qXz1Fg0g`i3 z`~E`4Lhyr3`S6@>`Ibgw&jV8hyk!`b&)!W7d3mU-Fc{71)Q18%*my|%m-8>?tgKah zEBOY%texSK`tK*SUGEQ!%~#fy$7%+zp+4^$g0jwkjrpjto8@0K;w#{zV{||Ido@wK z5R?Z5^8929#RyFTS!yFXCxrKHhLn8;MyFv&uL$;22MvxCfJ+mS@%z*m#16%$RzFpa z4Cp^C%AEa6C({w%n%IU%iLeJ*T(sH_&^$yxpniGOP-Xy;;pTKATrz&%P^~2~@=bg^e(IZix&}4pY2X5dV@FMv zB1sJN9yh5Bc02f=!x8E0<)YONqZbNregAw{>=gW1A#9nO?mlJNXms*Y3p*c7k01G_hMJ`^DU-*qFW+H!(46_C-pol7vfp#K0S=8 zH#`_WMTZ$y2Wt+B`6_k_YELKb9!X*{))4zHD_VY49*-1n?*jDg6mAQW9)w9hfs5IJKanVJ|y8*uOwwuu< zdoCm&dtltxBk4IM3FmRmCPeFW9ab98^2#tjLa54`~= z3!~;r$$D!Rh^KW7z}aupA(oxK6Ckq{KACVFWB9XwvH?(2U?Z0M^X3zxmFZBHVs-(k zP^qfvGn~A^K3^CF4w&HTx)vb{7RBSXoj;0;O@TWx_{QcjoCC{=h~q0IGsSy}koe{L z&tci%Ck%wnj*SaO>NUkHyoI!lnB}-3Gx(C?H?`~rk@R23Tiq~jUc2eSsqX&(2GbxS zbnvnI_P1xrv?Hu-ccbt>XbQ^+fn>bk1-M)Bku$J9`w{&{N`xq;=jGGqf7}@y^{kMM zMP^^k2Yr=uPN@NvJnB7=5+~@gMVqc>oL{?Qsau|$^M6h1B>@4R{|vHJi;|vi>|u!N zO**Ggs(5~ugc(m_Zr5DMyP*4>Gd|#I(if@oKjQl?DyfVZQPR(;8z&eFp;cuafogAe z#Dc1HDaFi}%+$LXRRI%qt7MGesdlS6*JFKK?oD_9eMl!9FTl4y**q7Nhr45fzHwD& zRn0VDbH%lMdEPPFE)iP~hWJA%dFZ?$AOH-J>mfv)Zjff?$goKE7tLyU$9;*=r${Wr zG&lZV)Fn6DC8C{_D9JnGr4H%}J<}A^z32FP+0_6M*6_>Udt07v0@d9tvB$&^221q>MdIlB1-rfVKaGHNY_;_S(-(0Kf z7sX%aqp?wpG?Vo51CEM3 zxe?IBf%wrqzoeSNeB^O1)rw;(%SXROwPm)r7{j?IqItsb{h9zK)PKxMU8ZHr)SJY) z!ieZssxMGpa1Knf`8yy{@!gd|8&W1U)cz2(&dcTw8Wlztx-U3Z0_8LWM*cI9u9i8& z6OT8-0x`HZkDnhZFTVWQ%ejw(BbI8SwasFUe{)!+T{y}qy|URiYguJ8{6%!UO{1Cz zUIL(TQp)doKAz>8KU(Fa>~8I0^A4Z{BpUKYe%@f;maTJ_m;dt6-+jEsT>RqX*ZoED zX{5DKS=VD)T03d@afJJ>)e4l5jhcqf47&73(ONHeyB@7A>ey+Pt*S@oX+ELPj9C(O z!S&3A!){?hb8F60Q}*FlD*F%_YVcEzGDsfL;JdqVJw84F&IsIkczp@@Z2=$GVZbN9 z&nKW-t!GLcer)pW?oCByO+!ofL$&L_>X!DzBMraf5n#kg4&@M=lT*%-oL^bJ&nD5> zDxSy&7;-O@?MvA!s=D7|NevQV*I}dk)Th}W0kmCWFsN4TlK?9ZoNx|t)_&33g%9eelk_xF>Jt(w<;m&ZWt&R+fJ ze(d$DkISp|wv$|)Lf(tV|Fp?~k3GP;*Y+Ke;P^k-lU(DT>!g``R+=&1Wbd=m$>g+C zzIkB2sWB=GcR9$T#jVZ}3z{>;}MBlp+Ub#(^>Ioq~gt zMd(+`WM%^3<2TrTSEvWR$+sXL>@+W#JgrS`acWxZgg?~qZMSO>ltq}tL&>WuC^eXh zS-vb;gFRB~A|0VzL8zXrg`>C@avnQ?y&U-JdTB@spnjYl49<3QrP*ea;S00Z=v6=_>EV9~C;uF#Z_$ zwIdZ}i!%-;EScvXK}!|kW@QZ@;ubmQn^VTGkHj9%^5_!}tj+ZDe*lI0T#72EK*R(I`tq^;&W**43zENf*j8 z-_j7jT(a}D8h)zINw;C4cm7~GU}OHdc%d$@_KU*H6MB}5)=;iRXg23cv=xjs17WK#n1 z*t`WQ?LAi05Z%BlfB2gol&%2Y!^PxWQVAZgzLn|i>6kibnVp||WI11=*09i*$&349 zhMbKuTdjX=(FrWN&WMHeV>4auE-L!fWrQr)nnC@$FhXdEFH z%S0r~AoEvJ1u`Fdc}#XUCtDTSGg`NA#|1W@(ULtTFqZcUObCM{K_vYz`itUn5m|Mzvl0ZLakg zd9(c)7X#xW7#6u{6+JdBa_uO?4#{`K6vZ0ctQ74u&t>JBr+Lw)mTuIAnchmgP48om8zZ>-a2b zJZ+v#{3~ZPnpIDQ~(H=pk7k;8ZjhmjTR` zFyoXZCKfF;>g!fYadfiKj3h&^LD4BW1wtbK!Tv}rO%Jad^~yDz7(4m_jV$6Gmt^J` z{9J3AB?e_*3_rLVP4Cs>l3+664#8gmV>C9KFh&Flp8f494f=EJm(yqVk*5Cf)?mh? zyoI?{jug_PwWM*Jt2>pw{US}T!(j65oYU8?Qx+}PAm~%|Q(37*-dLHe`u?8Ct_ht; zU05*?J<%bZ08!`FL%ZRmm|qFRVmonU(7+W5{4}opY)n!Pwlt1Rw2?kZcL+r>w;0jx zXnA99ZgC4z=R2)}c|T2hnUg?9Iv3|eJoTQy=Cx(dkn+uZEc z;yA2xRS`37r8e`RzP{DyuIO~5z|Iwd*h0G6xeVi#e-o7IPt5RjvJnpoGN8ODj`vRmOj~F3O6>pnCG0xac6|t`R3}+wrCtDa&5{-} zTcsj)sJLg5$xdFzeGi3&(&Q4bQ#uAiPn#HcF&(R+0iv@=gfNNt7x(Jy~_ed($&=EfyVNh%J_j+zVg3z`4mUNJ`KL)YI<73CEvXs zFK&3mwHbYf#hR4>9<6NfV)y<6j~-nIdZ5=!#JL>uIcn@a>*pVXg3L*i(k#F||IVkF>~raq*Vyxt)l*;V zeMCo;s1xgV&aF^yq3a!MUvd;Jom_7-Qan|R_^@OqCeZGTqeg5#<%aY+JkQl|Fqz%0{=3PqW#pdxs9EcU5lZ zz_Q$q5gFdKlRHUiQRPaz>Kv~Z^j|#NQJ%kts$t&c186Vj-dY)`Kf_7as9V{U6i2P7 zi|WD%sRvjcu8AIby4}BS;i$joCzES3yHYOiamBYB@=_X5rzL zzmJ+8c)2SWHUCoMK%INXm>cW*!6{iP0$af@@#S_dQ9ANVIO$1~I4|*)%Ah#PxX4X9 zKj{DJ*|jY0xX)Q1@;6gwz^YROyl*~Bm&>o2qKLpCQ*feqw0j_prv8#0j4LOp**gaN z_2AY9qC@s8KJOV1YQ}gQ!KNAH(#>v6d~OgJSkukMO`d8+To2MK+Ys?7JLMQk`N9jB zS}!s-W+If}zB9SL{z|qGg2L&Q2#OFOF}%hwLEn9M!oOg4?T|Y&zY?IC8gIvPHDsA$ zC^yUZO(ns541qdBWi*5{@NJX#v3iEaL|onE8T5#{F^YW*0V?a^QN4`to6#L0cQX}9 z?p88v(2TgCjz13;Bj=8=#~-sKOU0%N3A0-^dmhC*;lRLtjvU5Zeicj+I)+W{9 zF6%l0!Cm;WbdBPcf>>DEYkVg8_^`%{U?z3#z*f;Enj=bpWZb;ha-!2uI)Wj>LdxxW z-zWFEIM*fOgxD^Pz=ERdthY~;7502 zDw;+_W7En2K-N>Gqs9Fmn1kOLE0%|K#C(o*#CGvTih1*zC32|_-h=L`B06!KxF2(M zd~$QnJ)7ci-WSX0I&OpKCZH$!>ZOi!T|Ut33*66qg(R%nCV$sVU>h8Q^o;YA-emDB@`emvF`bE#>=ShS zcWgLr7@2kMp7^id<=XHD8UQBM*mOq(T6iqpV%S&n@vKeGZ51PmE&sGYaJYdi+c@~E zcH1G?I?_!|ak|pul+|@Wcy$fvKx=*8%XXT+>D=t}Xfz-a(w`qLnjv9qu~{=dJorK_ z+pOq1#cX+;9}yTXibb}hKZaZ`I~k~bp*!?q{Hs4<5wT5*&jeX zlDNi0ImtM4tp3wEMQPpvylhLd5z{vl$M^s7lZYZW=q~`ZL@Nh|mhtD1=uxQq1;Ke3 z;n7-pKk$@$geY5JfrrvUm6y6#&qP=S*h9}zMT4f*uzzOvivtoZQb^bUF6S@~SfAdq$0ile7Y1Qbd*jd5 zjf!?b@A)Inx#u%w;Hj61al$#Ht@xse17}GJbBe8$mBu8c_ZTD8EzKSaC;+hne=%&T zAaUN=KxbirB%I!>~KN+mgF zwdlrIHH9Q|+|0v~??QsSYH7~OynJ5;?Z1C0K3+JUE+7}RZ6!WiSbxPO4>=Kup5{ar z%eOf;ZOZZO8>6d~sIV&Aje(Ojw(;Kelyl-0bmFIqGTK!1WgT?s@~Y@b9b!~idanqy znR$8E9}Tb|fcpV-&(k!4eHIYx3xnaG+ow7cc=y;AWk9tfo*~r2L9?sH0 zj*^qoTlRDfY4DKHTklnqs7Y9xc15ZEfZlO!H`QSZ4OahqDT<&o%ghOdPUYV?xtSb@wB0&J!U!jt4N1&^p@=c;jaE6zk zFXb=Op>0E*iZ|U|3((?lT zrHFX@TQtdSw!rMG`F=3XGd4ZDY~g;YPI$K@schcA~Ol)h1c=Tr6Qh|;Um4yfCq4X{dy zvoSj<3f+k;=PL%9%2c;x70w6-hLf72-d5d-TnalKl5#M#?Wc!dMdwtMxY#VIwavwO z%CDu@c83AJgl(e#okfqcT%gSG(#;sp#x_KERoAdBoU=~zizIMxb6Xy;j;s>B?35_2 zZ{7z#$X_*ltZgmxa#Fsxq+*?G8xa@!49e0R%zuQVHf3KoK%`PQ2%-I1S$TZ}Bhf-? z`Vmk>2Uboo=|{N5u{2jJ(kx-q;8kKLp-?$g7?W-CP>eGBjCvWT zlD!kXHdg1QKkHLb9|eu`z5f~OmA#!1D4H`+W{Ztuir)l``|NeByR{vZDKB}-=j|z3 zA@_5iBJk{3FNHm;g_s!>?!!Xc%(mXQ3guSQCTEgP2U(9 z$K)&_BwML#%9eu~j1z68Wnaq_!6%S;gUc5?_Tf8E_qYc0?)yeuWr&kp1Qq1?8jJ%Y zyj~M}s8`Us5_*(Ab`j9)?EJVuR0N38F|N=6h%f+(PD7fJG_P5@n_)0Ar`u?aQJm`c zQm#Fn?h+JK218uZpLbrMDUuey{Q^9nezV@&M@1zC^Tm*Ep(5HE#k0`?CBJTBQ zYKz{}ox-@!$5@q%)^7WZEx$$=k#qt9+1g{vSQyl+Hri=~C(1p@OkVz_bMRnG|`vw?)l#+FSe_ZJ@oB%+ExZ) zD(NASTXY|-2?7eEZ7=Xs;{K^4zg)JC0&c?rdy3nLjvM9UD0I_`#Y1z*hbBDjl($sB zDTotoj{Sa*botILeNQ5D^&5fZsM^Bt#xReT;4a`x1AfDiMt3Q5wzd!~lx}3{BMuLY z5nQ*o$HaAYK5QS5WqTCXNz!nzIeYHvz0CqRqv1wk3kY0&%9W&Or56e#hQ1jE{E}m)gBe9<(3ADt^#CUbm>(L-d}h z5E>i-gt4N2raVmp*JWP{!fob)p{F)FS+a^xS7iAOW^p|lv=)bY=9t1&^o*yPZ#sxL zat!nRadsQ_A5qq_!Y@7BwTsO8V(ZMYr#@WEbTi{pdc4N(+Ivi^Md;;ie*y#Qp!)C} z?o<#RUVSX58aOjvo@-T58l)7v@zP$g6ofgM;s{Tn+%DF78mK-cIf<1t)z~XHw61Ur ztPi?IR38%C@g7>Lyh?JOzl9sbk7$cBDfwS$8EXtw?85 zzDG_*vdpmV$(tePDv}q&dQmTFmP`2<-$Dzf$b#tSBy&^mZ*#|0wVH_pKA{5{L#(bf z$K*jKXg;vws~9}|Is;{$f6kSM6ZiY|GC|moTLFG>)}U5l1hbTO?1xyUH&2aqm!h(bAra1vQ~sTKj;WE#{0hCCnPSrH z$kP6Jx5e-LYPG5g^J}8r)p=@$*;-s{MFPUoILXvEduA1G!!rUQjlWoAQ*)pVM-m$Sc1FSnnrSCwC1U)?DBTyc=84Sg z3tmODnIzP#paS%1RQre%p<*HtO%Ns((}$2LaP2m;!J?no;)~|RLfHolZEzX>pQS{E zh!#H0?;i1#?#_ChFBk(%Ex>^aw0c5KED=RkqSR$LfA;Vn*&wVC!c7h(buR*nkO(D2k4wz?YBrAtu*t;325l*+ z6YWIpWpQA0y2s(A>wD7-5x#4Wn$8DZJZCKlVQ`i$Zms5KB%9~p=|0K*GOp+;KXYzg zxOZqbG}O+7+2=C%`FlC5PUQQ+tm|If=oAz<>9LrPOBe>%FnrSx<-l0<+hhoA9F9`f z`+YdJzrRP)3ywK)y67|-t^FTpWOWW*LD$S0C7Gm#7zX_h8BJ$@3=?|*YXrdWS{?Dw z>tgsO$wgg8dreX=q?SoZ+=CIg*Mni$D2tCVKxK=}cehbJ(Q`@Eqp~sRDk<#K@=(J~ z0vQ`!aYz`5$juI)bsGw+*^7BqH8G@9#xt z3K@;KY4|*)2SdvZ6bPZN3s5TELpy^a@9RzI=MWsIh4V{)IT`M(r{`?LvB%*b^3*7X zW%VIEihCqdS?-wMmIdzn!LNT8?@+1@o0~z6?Y4VcgzB)mlIOK}_T6xs;e7XQMDqC| z6$*+8B(qK)Lhv~%#=DZQRy+=Zr%UYPtwjE@rJO8h?Q(0{GYz&V&2oH(u5Ao{;udX7`Mp$Q9YF{J%eY*2K2rxblDs1pYh0nC>OeBE0eTwB<& zPf_*_1}g%^H9sp;t|PyR}BXvJ~&%)Qj?V;>^+5-mkVI7lxxOBeuUxoUwBJdR!Ob3h!|gE;BDR_Oni zkWwk?egsGqK@PRg(Cc@6sqG1L?a)ZOmxh4s!Z$W_i@>=+>U%@gQ z5!9MhVeoI#VXNuvx}bpoG^yD}nhOR940LL``QEHXn`bxyuJ~q0gtvkjtIDui>fZ_1 zTn-5Vd1NYUtM*8l`ZcwW_;HWl&B8f2T9Lw@LC_Ymq+qk|RT z@Cu&e0#PHq1tNF2JWH#uy7b}BuXM_$bBrs~@fk73v}`QuHp@*lHCMjgdrq4(KGbq->)jSVwZ57 zn)t0*7rCuJ(~hbT&eI=f1kGq3D#=C0f9^7sZ6y0XBEw;y3-S@b1Qz8(hWGJrnj#8- z^Uw#=Wzb%*e~cL@L}0dgN~Qg&tH@UK-uh5765=!|ax(h&nw(rHnr~lrk$~VU2A69e zEcL9slXELS&N9={#KM$pdPW9eYZ3X~CVkh%_u~r*A#GIZtDc796YaDy_^&ei zpM_us%Lhl%n~y5;i7N<@JnvFXZR-~{GU-)}d!ojWLzg*9CmqH!`*zr;8RURa=c-WC zSPK*wW6<{~19rUe;BVe*3c8h}hJj&<42D3f6O|Ed7L7BDXCUzP9mV4qQ5oY0Ic0dS zem9Uw?nSS|DuB#`N}SXdMGjhtteU*aX8dllpF}C@}SpF%@0d2A86Tm|{A_7-a_gyop}|j!jTddQVx+a*vPO9~s@j z=)*d?>~x8BZlzblgH6laISF{W#c4?_XtlHIQwTiOyLw3Ul=O(=f&VdK83MtQmLI`` zg|hv}B>KQHs-0ttJ0EuX%ysShNw;SlzpU50oZT^JnnV|r5{--R&pf5BqxL-Ja%{vp*lb#5@;$4|O)(zzM! zPJEomCAI9_R-aEBK)037i=a*N4N!C z8xGwr4=T`Iiteqev3bPui%Afbm}12w59A6iPeA^VHx7&WzxV>b^+mK{|!cQ9IFD~4sWV~Vt|4>CPQ@{^@9EmU;Pg@VPABlsk`{o{{*J6jM*S{S& zi4f5}&afe=ku6*@knI{9rOrYE8?DtlaGV{#y{r^rTH&allV<(7L*%>P^8P{6eV(6e zw#cpyrwpK>J8L!pg-9B8Z4!}xCh=Z91br&a>Nnn*dP$}kyhuA^9B-|Ej7`IpD6A)Q zhiSDIpV*OjQ59)1T(3_5fGg2by~kM`K)}Qo37hZ&ISlA?0yoc%Qg_n+%LZn!|FdI} zkprKTSUy8?Gkhsxq~G~>9XZ%uTZ-s=u)UxX-&+C*QR5z zSKAnHFf&(h_fOdw;4qG7LN`o;(7qlm!S@qE1$|~;j`Th0lVzIuu*G`~O~eQG?l3ruWqh*h z)X!n|>YdT@c1^WcxGhvMo5v3}4hxVmm=;9bhra&5%b}kN=|*&&$qeXh3YUN>n;r)% z-Bn)R{pucsMgi$O&ftec&kgK>gf?0f5$_p1+_{|uPPcu2=9BjUNvz#Zt>6;R{;a8ZoRPFwE2Qb9 z#4HYWXjddr>I-prKQeEryFD*815^gh&jZ{bWx$lV>KDY4v?ea}Xx?{r9fzU}g}MWz z_`&z>@?sI&s+vLQxKVpLpK}DM4WTVyw3Cf`_?n{DySKX7aezT$=Ml`a*$*w$mN2{W z07JDEqZb$ml)d)Ec|q2y<2Uu*K(_sL!mr&o_tj$gN8aHle^e@1?J2AjO3EvMAfm6u zQS=e2!+VR}s51*d(XJm*xSou}USMES|5gWnyq$4ykx97VI(FXim*t9Qsh*YHYA1b_ z9Ny`%{rf{SIFo@-)GyR{$&#%J79=H(exiV1u%0|(o(UW%jbC!h`}5~(3RLV8kDbsm}`FeLpV#M^C;FQ;aZTFzkqept|_BFr`h&*Z_S!lv&%h# z`)jc&WK7_V2_J?;EK zicND5aB6o!u2%A;o1sCf8AWyW3I#9s9cLcT7>B<95=w<~;IUGf!QT7(BNJJ>IpylG`E#Z(;xJ=%B3*Vpv1cjvHSn3)-5jq6DRyb8%qt$MGcuHTAmlxU^` zNnpOj4ZQ(O7)EJ(|66eUklB?G1_!=Q03f@u_n8mT>JR{c!*oTa0K>u|)4(U-LDog% z2GmyvtYrg4qFce?4A330e;29(pjCFx*Ia*N$1*P1pTU zxP&a3AF{Sz{RWc($}6@Gt2>0WOeDY<4PxKH38MI)Bxq`^Uoa_&0AC?nue^ksy0s~5 zV5Yfz(df6^k*2&oR0x*T?Z`KEztyCE*)j62jENnw{5onDDXM-JR5GNOwb2FH{Qiy$ zf4fcL<_nZgb5Atp!5A2$nP-q%94ONPOMw!Nhs{GdvLS-d%}Qr58JK?l6aX@v!NVuQ zyLXy=zLnE5X@D#CX15GGd_kk0*9Kpvjsgy!U9S!l%&_LPgSo6ydb!M{nLhRA$~`jj zz5+*7XJlLQ<6~BX7-128+2qnc^<*W}m6~jhG3SJC z2xPDzYhj>(-p$zkrp$j)ASTHqQ&l!8(I+u6b@4zan~c$4cDO$Z}-! zpO)mf*=OFA#epe?k$H36x$k=_J$v^@A7mH8ezS1S-~H2juUxxK!Gl&__FJA$K-J^- zsJlCkC$3m5Ye-qEa?V9u@E_(6jz8Q(cj~?qDjW+o4{9_}Kabdwd{`6{=cTfw`MSzK zN>wv-7(G$G?L9MV|K+q{uz(M1P)QRJr!rT^R^ymM4Z-kEry2U=>JWan75gXuHJMkm0pQ1 zu`(t7!}HjHBTMA6CKSUW)la5zDeU~b!mr)DJ*u(sMpEY76CJ(Ch@_@+gt>j_8QZ!f z*pEZMyIDmatTC8|0}N7_jd~RZ09y^j+Rk{+G^rJ4v=)qdKDnW~lnA;SI2{Z6qX8;u zZH@mNIpFM1yRiRs!u1pxK8d+g zCST{mn9awv2^}zU=kw$T-MpnZ-1P&m%|U8YeSRN!smrxP-yRqjYF8=ShQ>Cdv>?`PlkLT~7)_}dZFOwqD*hryl8^Ub@^^2JP zmgr3ztpV)n$%5miszzK~t8VjQc%h^Va?PRCW9$33w*?odH73d|YD4P>=H}XDgXX-Z z_n{Z9MUP6Q;byiIG&#y;4iiaPmGz!R^t{06D@a1beSVH@i#$P}^mf50zm&I?B^COMf?mk)^g2g19Pgt!Q5}ms%bcdgq(2zd!fZ?(_gMaG; zjMFOh$G#=YB~Q15R65UUvbhdQuSV_+-U6)z-O+vz+gf~S$H-A`xeO|^(LJ6gS?;f~ zw$b8=aSOF8QUVl_%pSQcIJ+h`4x@LA@L8_l77E56GYY=g0$#Uz&jSjVA=V_Xn%?bB zMl{_{OulLtljqhqD>gF{20G>5h_mVF;IW02vILcx^c@)fNi$If?osFDLT3gOc=nb_ zlZ&c8sx}DmXvP$UHRX~(V00IBTtm?PA1i*)<=?_n!CJ7_dlQlyYfZ*|l`P(YDW>&p zNw8mmi@MwrHzSgEwz`O)1f8f@``*OYU;mWX=WyF=MV`Ee%h6G=8u7pEMXCzU;>^M! zi0JjQ7d@SVHYhb^-W9TQOre3#Yne+I1y1aJ-ooX99Zsmd;c>S;PHdu(2ka-~-dlYN zN+kWQtNl2*VmDP(_lhjixz6g!N0h&E6Fp^4A_di*8e2U;cM1sGqptIhu$Z*VT4v2= z)<|DK(7v?n&f7-iUQ}Pn^}}PzJ?2Hp^B0T31mj5y>9Xxv(1`~*_2|xkWkVTyYMu1- z_Xl|8If zP>pL&vbr@zSRgrC=31dLO7nYA)6BS`h6TAO8)`SL51u?)ijZZg4Ueqe*LeVJs)Ge> zL2SC7wj!mwuaw;v?^!`%t`oXs{#QPNy08Ye`mil6wWWq@w;?5mCUCOW z=u>?2o%0Wx7~F>iJ_@&VmMV#HXYJs~PwZ&V@l=YbI|tVTQ&UriVFT_yZ{E+^tq%iH zb?uF)`hss!*iQ_o9X$Mp-uA57q^#Jz0CALHm7ar?ZFMEp8PJ2A@x zEdvkN`;*$4WtIPKRvHgf%${38IV=Vvrnx6iYo#xOK|Em%JeIE1e0FUAyY61Ydw*oX zV`;Bz8q!6vp2GyjubL|Q^1MntKRs$c+uVtHYKL-c|3%N!Ap6G_2E%uSf`SLeHpx4J4H}1}NWV(*qo6gdpkrW=Rh^8o}P% z-zV1Pe`16B6K3=<$&?+_bAiBXg2y`nMh zu38iranbr11}W9aIwY_79%r5Jj@_LvK+|@KIqCedmfjDEgy!SwMFdIij`_rX7w~pO zx%njBzOqBCpt4!Qnv33PvLULKf?mN52twKYz`MdK*Yb_7bz-6_@SGDg^ER-(wkEQ+ zOFK|x{AAoPr{}fgpff(^{NE|!i4p2nH0~1}|M&Yy%bmsw49YC17d&PP^yumhYwL{{ z%9o4?dCRq{GLMs-5|-HHz$=+21(cF}WRB>u_2O7mU28s&J2#;H?~(U_B&c|Z+YIYs z5}qSVvCBK1&-?Wsrw7usp8z&FYpn^wnM}8Rc~i-v0hd2v1dE3^bmuLVo_m9nwKd(# zs--i@X3X=u+cV$8LMcBch*<+>lrbHDY-W0xcG@%nUa;6m2-Ax^N z_ker*dMI*!&n55p1mzk-B;okLck|>MUh3B0f(Ni#TL-N>O3-Em-e}v0eh+RyipEMV zt=dbneD0{yTdzNXX18A6*)QXpC2~*5)M2k8?2KY}Jh!%qLFTrzP6l6*JKF!hFrM!O zQ-@qVt@c((GsQ8O1rmGm8H%{2Tt&X@*H88k&K=_O zK(gJ|nyid_exgZH3ztxRHUH5yDs8WntWLC^{3i_%KmrRIL^fNf>CGJfWuJJ~{vzb> z#2`q@2nb@45Zyy%ybEkN4!c#H7Y6$LM8N;eYl_ct_~OHXxza{Wp?4RO0z&z56a;5V z1~0IlPh*Kye(sb>`)&kLDk|i}Q8p=tSW?K+tDnuqeg>@&rSEoL-*Ju@{yR)^z2h$N zOkkAj{EsNcF^#;mGljeZ*;TJ}-=uY_v*qSb@)z3;r+3Z#Ajsqp-mqKkPHPhy{b7mF z#9MCq8#Da}<^s-UNx!24^0gluy>R+Q;eL+0DqkLh&RO}4?nR|zYBRO(g9?y7vn(}@qOqfR!1dghCw_x{K!mI&^jx!gaQ!dQ}79 zf~eBcf}5V>e^{!}J}`Q$KQbPr94UX0`;31ijhBED{pQ_m-_j2L@7Sn<00C~nXxR8? z!Qxqa{2gi52e(|vMbRSX)aNP#939^RIrLPi@EG6!%@DPy)YyI{%rEJI?-&y zVo8Q2@9u=z-&ND<<#N;v&R82e*X@R!VQ%XpWcqTU-q&NlUJ5H~?Gf@m8PF6FGVDb5 zIUkB!!SlCBB~!%o(74b;ZxT?D~x8_KxI!?|4rmv^4E8J?^t4r2)_T*)wRbn zq4x1M>u?+@r$V{KD%NE#PGZey-5)?8!GPIc<@InSTZ^Uw2qf0ys?_xt1f==q_V!Z~7j57o6{duin{QqY5* zYCA>y094eF2n^q8QOo^g?E6Mnwv7RKSY0M|(i(fyCPIhCVv!r*F8Lcu&0Fe4$rL|I zG9gdiW&BZETAgSd@v9X%bbVC_F}U-=T{+atoIg{h#n&6TJR568ZO>bEeORj#lEH^h z;)dNXMyYA7c6vAnv(^ABhx1t|MLwe?wB znIXN8&-yn!+s5a#f9BpGDmU^W$_y>XW|Q zCSD5iFg%o&{<^eePcHTD^PsP(xmijm4LF@ya&|IS*E-Ha46WueZ9(Xafs#xS!M%^2 zX3+w^()ktcf^~(#0aoC{ACIY{^t5Obi0tX3BF832RUckCk2~~|wrxk8+2ee_>H+@^ z@%^AA2iC>CN+gDF<5g$7@JwRyIF}$Mz_zOgU*jq&%7B#OfvGJHS+8&^FB-3MqP(6S zkfgf1Nw|-CDXlD~uNo6Y=7jX!_YAkL_k;7THmNMu`LA1ovc+)D%GPMo-WHM?F;=h$7xwpMle`d7ky`n1a3j>K8i9|DvOzl~^S=?bW(fULjwz<{v zLIcO}boy#!NSRRpLDp)%Jix?bNObc-O7f#@7mjvUa7WI_(H3Sk@3|2Yt`M~RqX;BV@*w9o*5LN3 zg*=R?bCB-S5cnpfiiN8w<>Sy6FY$uQ)$9SwSHWFiGO3%jrP=z(FA)k7V*tq7g&ZD4O{LV_Zj_EY^ z=E&w#Lo*{HF-{c~eaIB!|k1)--xy=|NtnIMOSD zlBl|UsqG!n#UEc4n3Q+BbhP0ejbq%wIUhHZwT47Gv2FqD;8xEFmQU2rD1+KRf4X0y*(08S67!SO0e-rJOlTSx`Int zF!)+q>PSlcUNknNEPqZJbcppbtT$*6atWeiVR2>(N{VO(@M(#+(A(zHF10sqb7 zFZF9_NEWDmfrfGhIwxul+e*8ZFF~ekyFMy*_3+2+G3w{ApXrleG;+^2?o_lp?2~ z8z1>xMzYCdtW~_@E(F|Q|7%p;vb79}PK1>|oe0BOB&6x!T~XI(*?(ed{U4c<=O2ys z#HHO-*!+1IthJEv-r-2lAzLpvunsz{DtW9^t<%Ks2WjLtUNYB~^Kxj#o+qmqCMbPke+L^&Pd;chdrLjzb587=8#%Xd_NyBOSo^Z)ufF<3g50z$FzC3j#I81R&6aJ* zSROdRxm8e-Tn}WQ8g-`YX>L8##4PCwOnb`H8n)+eqtZ!qX~2eJw{)Rg{{LIhSoW2XxBdchSrrBV From d5683a703c99bc8df020c1e0767883eef197dc2a Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 5 Jul 2016 00:46:42 +0800 Subject: [PATCH 0886/1470] [SPARK-16329][SQL][BACKPORT-2.0] Star Expansion over Table Containing No Column #### What changes were proposed in this pull request? Star expansion over a table containing zero column does not work since 1.6. However, it works in Spark 1.5.1. This PR is to fix the issue in the master branch. For example, ```scala val rddNoCols = sqlContext.sparkContext.parallelize(1 to 10).map(_ => Row.empty) val dfNoCols = sqlContext.createDataFrame(rddNoCols, StructType(Seq.empty)) dfNoCols.registerTempTable("temp_table_no_cols") sqlContext.sql("select * from temp_table_no_cols").show ``` Without the fix, users will get the following the exception: ``` java.lang.IllegalArgumentException: requirement failed at scala.Predef$.require(Predef.scala:221) at org.apache.spark.sql.catalyst.analysis.UnresolvedStar.expand(unresolved.scala:199) ``` #### How was this patch tested? Tests are added Author: gatorsmile Closes #14040 from gatorsmile/starExpansionEmptyTable. --- .../sql/catalyst/analysis/unresolved.scala | 15 ++++----- .../org/apache/spark/sql/SQLQuerySuite.scala | 31 +++++++++++++++++++ 2 files changed, 37 insertions(+), 9 deletions(-) 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 b883546135f07..609089a302c88 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 @@ -215,23 +215,20 @@ abstract class Star extends LeafExpression with NamedExpression { case class UnresolvedStar(target: Option[Seq[String]]) extends Star with Unevaluable { override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { + // If there is no table specified, use all input attributes. + if (target.isEmpty) return input.output - // First try to expand assuming it is table.*. - val expandedAttributes: Seq[Attribute] = target match { - // If there is no table specified, use all input attributes. - case None => input.output - // If there is a table, pick out attributes that are part of this table. - case Some(t) => if (t.size == 1) { - input.output.filter(_.qualifier.exists(resolver(_, t.head))) + val expandedAttributes = + if (target.get.size == 1) { + // If there is a table, pick out attributes that are part of this table. + input.output.filter(_.qualifier.exists(resolver(_, target.get.head))) } else { List() } - } if (expandedAttributes.nonEmpty) return expandedAttributes // Try to resolve it as a struct expansion. If there is a conflict and both are possible, // (i.e. [name].* is both a table and a struct), the struct path can always be qualified. - require(target.isDefined) val attribute = input.resolve(target.get, resolver) if (attribute.isDefined) { // This target resolved to an attribute in child. It must be a struct. Expand it. 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 084ba9b78ec50..dca9e5e503c72 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 @@ -2115,6 +2115,37 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("Star Expansion - table with zero column") { + withTempTable("temp_table_no_cols") { + val rddNoCols = sparkContext.parallelize(1 to 10).map(_ => Row.empty) + val dfNoCols = spark.createDataFrame(rddNoCols, StructType(Seq.empty)) + dfNoCols.createTempView("temp_table_no_cols") + + // ResolvedStar + checkAnswer( + dfNoCols, + dfNoCols.select(dfNoCols.col("*"))) + + // UnresolvedStar + checkAnswer( + dfNoCols, + sql("SELECT * FROM temp_table_no_cols")) + checkAnswer( + dfNoCols, + dfNoCols.select($"*")) + + var e = intercept[AnalysisException] { + sql("SELECT a.* FROM temp_table_no_cols a") + }.getMessage + assert(e.contains("cannot resolve 'a.*' give input columns ''")) + + e = intercept[AnalysisException] { + dfNoCols.select($"b.*") + }.getMessage + assert(e.contains("cannot resolve 'b.*' give input columns ''")) + } + } + test("Common subexpression elimination") { // TODO: support subexpression elimination in whole stage codegen withSQLConf("spark.sql.codegen.wholeStage" -> "false") { From cc100ab54c7d6e7afeeab5439b21e70eb835a505 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Mon, 4 Jul 2016 21:16:17 +0100 Subject: [PATCH 0887/1470] [SPARK-16353][BUILD][DOC] Missing javadoc options for java unidoc Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-16353 ## What changes were proposed in this pull request? The javadoc options for the java unidoc generation are ignored when generating the java unidoc. For example, the generated `index.html` has the wrong HTML page title. This can be seen at http://spark.apache.org/docs/latest/api/java/index.html. I changed the relevant setting scope from `doc` to `(JavaUnidoc, unidoc)`. ## How was this patch tested? I ran `docs/jekyll build` and verified that the java unidoc `index.html` has the correct HTML page title. Author: Michael Allman Closes #14031 from mallman/spark-16353. (cherry picked from commit 7dbffcdd6dc76b8e8d6a9cd6eeb24323a6b740c3) Signed-off-by: Sean Owen --- project/SparkBuild.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8e3dcc2f38875..6018b22bae600 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -723,8 +723,7 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hadoop"))) }, - // Javadoc options: create a window title, and group key packages on index page - javacOptions in doc := Seq( + javacOptions in (JavaUnidoc, unidoc) := Seq( "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", "-public", "-noqualifier", "java.lang" From 0754ccb2be79e90bc746de54b01aa6af55f3291f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 5 Jul 2016 11:36:05 -0700 Subject: [PATCH 0888/1470] [SPARK-16311][SQL] Metadata refresh should work on temporary views ## What changes were proposed in this pull request? This patch fixes the bug that the refresh command does not work on temporary views. This patch is based on https://github.com/apache/spark/pull/13989, but removes the public Dataset.refresh() API as well as improved test coverage. Note that I actually think the public refresh() API is very useful. We can in the future implement it by also invalidating the lazy vals in QueryExecution (or alternatively just create a new QueryExecution). ## How was this patch tested? Re-enabled a previously ignored test, and added a new test suite for Hive testing behavior of temporary views against MetastoreRelation. Author: Reynold Xin Author: petermaxlee Closes #14009 from rxin/SPARK-16311. (cherry picked from commit 16a2a7d714f945b06978e3bd20a58ea32f0621ac) Signed-off-by: Reynold Xin --- .../sql/catalyst/catalog/SessionCatalog.scala | 16 ++--- .../catalyst/plans/logical/LogicalPlan.scala | 5 ++ .../spark/sql/execution/command/ddl.scala | 2 +- .../spark/sql/execution/command/tables.scala | 4 +- .../datasources/LogicalRelation.scala | 5 ++ .../spark/sql/internal/SessionState.scala | 4 +- .../apache/spark/sql/MetadataCacheSuite.scala | 8 +-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 -- .../spark/sql/hive/HiveSessionCatalog.scala | 5 +- .../sql/hive/HiveMetadataCacheSuite.scala | 62 +++++++++++++++++++ .../sql/hive/MetastoreDataSourcesSuite.scala | 16 ++--- .../apache/spark/sql/hive/parquetSuites.scala | 6 +- 12 files changed, 101 insertions(+), 36 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 8c620d36e5679..e1d49912c311f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -462,17 +462,17 @@ class SessionCatalog( } } - // TODO: It's strange that we have both refresh and invalidate here. - /** * Refresh the cache entry for a metastore table, if any. */ - def refreshTable(name: TableIdentifier): Unit = { /* no-op */ } - - /** - * Invalidate the cache entry for a metastore table, if any. - */ - def invalidateTable(name: TableIdentifier): Unit = { /* no-op */ } + def refreshTable(name: TableIdentifier): Unit = { + // Go through temporary tables and invalidate them. + // If the database is defined, this is definitely not a temp table. + // If the database is not defined, there is a good chance this is a temp table. + if (name.database.isEmpty) { + tempTables.get(name.table).foreach(_.refresh()) + } + } /** * Drop all existing temporary tables. 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 4984f235b412c..d0b2b5d7b2df6 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 @@ -265,6 +265,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { s"Reference '$name' is ambiguous, could be: $referenceNames.") } } + + /** + * Refreshes (or invalidates) any metadata/data cached in the plan recursively. + */ + def refresh(): Unit = children.foreach(_.refresh()) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index fc00912bf9f59..226f61ef404ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -206,7 +206,7 @@ case class DropTableCommand( } catch { case NonFatal(e) => log.warn(e.toString, e) } - catalog.invalidateTable(tableName) + catalog.refreshTable(tableName) catalog.dropTable(tableName, ifExists) } Seq.empty[Row] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 30dc7e81e9eeb..14836044cabee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -172,7 +172,7 @@ case class AlterTableRenameCommand( } // Invalidate the table last, otherwise uncaching the table would load the logical plan // back into the hive metastore cache - catalog.invalidateTable(oldName) + catalog.refreshTable(oldName) catalog.renameTable(oldName, newName) if (wasCached) { sparkSession.catalog.cacheTable(newName.unquotedString) @@ -373,7 +373,7 @@ case class TruncateTableCommand( } // After deleting the data, invalidate the table to make sure we don't keep around a stale // file relation in the metastore cache. - spark.sessionState.invalidateTable(tableName.unquotedString) + spark.sessionState.refreshTable(tableName.unquotedString) // Also try to drop the contents of the table from the columnar cache try { spark.sharedState.cacheManager.uncacheQuery(spark.table(tableName.quotedString)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 39c8606fd14b5..90711f2b1dde4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -85,5 +85,10 @@ case class LogicalRelation( expectedOutputAttributes, metastoreTableIdentifier).asInstanceOf[this.type] + override def refresh(): Unit = relation match { + case fs: HadoopFsRelation => fs.refresh() + case _ => // Do nothing. + } + override def simpleString: String = s"Relation[${Utils.truncatedString(output, ",")}] $relation" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 5f5cf5c6d30c4..01cc13f9df885 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -166,8 +166,8 @@ private[sql] class SessionState(sparkSession: SparkSession) { def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan) - def invalidateTable(tableName: String): Unit = { - catalog.invalidateTable(sqlParser.parseTableIdentifier(tableName)) + def refreshTable(tableName: String): Unit = { + catalog.refreshTable(sqlParser.parseTableIdentifier(tableName)) } def addJar(path: String): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala index d872f4baa6218..3f8cc8164d040 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -59,8 +59,8 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { } } - ignore("SPARK-16337 temporary view refresh") { - withTempPath { (location: File) => + test("SPARK-16337 temporary view refresh") { + withTempTable("view_refresh") { withTempPath { (location: File) => // Create a Parquet directory spark.range(start = 0, end = 100, step = 1, numPartitions = 3) .write.parquet(location.getAbsolutePath) @@ -77,12 +77,12 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { sql("select count(*) from view_refresh").first() } assert(e.getMessage.contains("FileNotFoundException")) - assert(e.getMessage.contains("refresh()")) + assert(e.getMessage.contains("REFRESH")) // Refresh and we should be able to read it again. spark.catalog.refreshTable("view_refresh") val newCount = sql("select count(*) from view_refresh").first().getLong(0) assert(newCount > 0 && newCount < 100) - } + }} } } 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 2e0b5d59b5783..d6c532563ff9c 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 @@ -147,10 +147,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log // it is better at here to invalidate the cache to avoid confusing waring logs from the // cache loader (e.g. cannot find data source provider, which is only defined for // data source table.). - invalidateTable(tableIdent) - } - - def invalidateTable(tableIdent: TableIdentifier): Unit = { cachedDataSourceTables.invalidate(getQualifiedTableName(tableIdent)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 2589b9d4a0284..ea818b5ebca75 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -90,13 +90,10 @@ private[sql] class HiveSessionCatalog( val CreateTables: Rule[LogicalPlan] = metastoreCatalog.CreateTables override def refreshTable(name: TableIdentifier): Unit = { + super.refreshTable(name) metastoreCatalog.refreshTable(name) } - override def invalidateTable(name: TableIdentifier): Unit = { - metastoreCatalog.invalidateTable(name) - } - def invalidateCache(): Unit = { metastoreCatalog.cachedDataSourceTables.invalidateAll() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala new file mode 100644 index 0000000000000..5714d06f0fe7a --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +/** + * Test suite to handle metadata cache related. + */ +class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + test("SPARK-16337 temporary view refresh") { + withTempTable("view_refresh") { + withTable("view_table") { + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.saveAsTable("view_table") + + // Read the table in + spark.table("view_table").filter("id > -1").createOrReplaceTempView("view_refresh") + assert(sql("select count(*) from view_refresh").first().getLong(0) == 100) + + // Delete a file using the Hadoop file system interface since the path returned by + // inputFiles is not recognizable by Java IO. + val p = new Path(spark.table("view_table").inputFiles.head) + assert(p.getFileSystem(hiveContext.sessionState.newHadoopConf()).delete(p, false)) + + // Read it again and now we should see a FileNotFoundException + val e = intercept[SparkException] { + sql("select count(*) from view_refresh").first() + } + assert(e.getMessage.contains("FileNotFoundException")) + assert(e.getMessage.contains("REFRESH")) + + // Refresh and we should be able to read it again. + spark.catalog.refreshTable("view_refresh") + val newCount = sql("select count(*) from view_refresh").first().getLong(0) + assert(newCount > 0 && newCount < 100) + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index b028d49aff58d..12d250d4fb604 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -255,13 +255,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) // Discard the cached relation. - sessionState.invalidateTable("jsonTable") + sessionState.refreshTable("jsonTable") checkAnswer( sql("SELECT * FROM jsonTable"), sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) - sessionState.invalidateTable("jsonTable") + sessionState.refreshTable("jsonTable") val expectedSchema = StructType(StructField("c_!@(3)", IntegerType, true) :: Nil) assert(expectedSchema === table("jsonTable").schema) @@ -349,7 +349,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) // Discard the cached relation. - sessionState.invalidateTable("ctasJsonTable") + sessionState.refreshTable("ctasJsonTable") // Schema should not be changed. assert(table("ctasJsonTable").schema === table("jsonTable").schema) @@ -424,7 +424,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"), (6 to 10).map(i => Row(i, s"str$i"))) - sessionState.invalidateTable("savedJsonTable") + sessionState.refreshTable("savedJsonTable") checkAnswer( sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"), @@ -710,7 +710,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv options = Map("path" -> tempDir.getCanonicalPath), isExternal = false) - sessionState.invalidateTable("wide_schema") + sessionState.refreshTable("wide_schema") val actualSchema = table("wide_schema").schema assert(schema === actualSchema) @@ -743,7 +743,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sharedState.externalCatalog.createTable("default", hiveTable, ignoreIfExists = false) - sessionState.invalidateTable(tableName) + sessionState.refreshTable(tableName) val actualSchema = table(tableName).schema assert(schema === actualSchema) @@ -758,7 +758,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv withTable(tableName) { df.write.format("parquet").partitionBy("d", "b").saveAsTable(tableName) - sessionState.invalidateTable(tableName) + sessionState.refreshTable(tableName) val metastoreTable = sharedState.externalCatalog.getTable("default", tableName) val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) @@ -793,7 +793,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .bucketBy(8, "d", "b") .sortBy("c") .saveAsTable(tableName) - sessionState.invalidateTable(tableName) + sessionState.refreshTable(tableName) val metastoreTable = sharedState.externalCatalog.getTable("default", tableName) val expectedBucketByColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val expectedSortByColumns = StructType(df.schema("c") :: Nil) 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 fe7253d7354d5..a78a28536cc3b 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 @@ -462,7 +462,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { checkCached(tableIdentifier) // For insert into non-partitioned table, we will do the conversion, // so the converted test_insert_parquet should be cached. - sessionState.invalidateTable("test_insert_parquet") + sessionState.refreshTable("test_insert_parquet") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) sql( """ @@ -475,7 +475,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql("select * from test_insert_parquet"), sql("select a, b from jt").collect()) // Invalidate the cache. - sessionState.invalidateTable("test_insert_parquet") + sessionState.refreshTable("test_insert_parquet") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) // Create a partitioned table. @@ -525,7 +525,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { |select b, '2015-04-02', a FROM jt """.stripMargin).collect()) - sessionState.invalidateTable("test_parquet_partitioned_cache_test") + sessionState.refreshTable("test_parquet_partitioned_cache_test") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) dropTables("test_insert_parquet", "test_parquet_partitioned_cache_test") From cabee23241922d55179c3e725f24397eccc75471 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 5 Jul 2016 11:45:54 -0700 Subject: [PATCH 0889/1470] [SPARK-16212][STREAMING][KAFKA] use random port for embedded kafka ## What changes were proposed in this pull request? Testing for 0.10 uncovered an issue with a fixed port number being used in KafkaTestUtils. This is making a roughly equivalent fix for the 0.8 connector ## How was this patch tested? Unit tests, manual tests Author: cody koeninger Closes #14018 from koeninger/kafka-0-8-test-port. (cherry picked from commit 1fca9da95dc9b9aaf9ae75fd7456378861d8b409) Signed-off-by: Tathagata Das --- .../org/apache/spark/streaming/kafka/KafkaTestUtils.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index d9d4240c056a5..abfd7aad4c5c6 100644 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -35,6 +35,7 @@ import kafka.serializer.StringEncoder import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.{ZKStringSerializer, ZkUtils} import org.I0Itec.zkclient.ZkClient +import org.apache.commons.lang3.RandomUtils import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.apache.spark.SparkConf @@ -62,7 +63,8 @@ private[kafka] class KafkaTestUtils extends Logging { // Kafka broker related configurations private val brokerHost = "localhost" - private var brokerPort = 9092 + // 0.8.2 server doesn't have a boundPort method, so can't use 0 for a random port + private var brokerPort = RandomUtils.nextInt(1024, 65536) private var brokerConf: KafkaConfig = _ // Kafka broker server @@ -112,7 +114,7 @@ private[kafka] class KafkaTestUtils extends Logging { brokerConf = new KafkaConfig(brokerConfiguration) server = new KafkaServer(brokerConf) server.startup() - (server, port) + (server, brokerPort) }, new SparkConf(), "KafkaBroker") brokerReady = true From 9c1596b6cf5ca6d0b2529e31dc072a16db014683 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 5 Jul 2016 16:42:43 -0700 Subject: [PATCH 0890/1470] [SPARK-15730][SQL] Respect the --hiveconf in the spark-sql command line ## What changes were proposed in this pull request? This PR makes spark-sql (backed by SparkSQLCLIDriver) respects confs set by hiveconf, which is what we do in previous versions. The change is that when we start SparkSQLCLIDriver, we explicitly set confs set through --hiveconf to SQLContext's conf (basically treating those confs as a SparkSQL conf). ## How was this patch tested? A new test in CliSuite. Closes #13542 Author: Cheng Hao Author: Yin Huai Closes #14058 from yhuai/hiveConfThriftServer. (cherry picked from commit 920cb5fe4ed0eb008cd14bf0ea45ed5b225b5022) Signed-off-by: Reynold Xin --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 8 ++++++++ .../apache/spark/sql/hive/thriftserver/CliSuite.scala | 11 +++++++++++ 2 files changed, 19 insertions(+) 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 7389e18aefb1b..5dafec1c3021b 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 @@ -156,6 +156,14 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) + // Respect the configurations set by --hiveconf from the command line + // (based on Hive's CliDriver). + val it = sessionState.getOverriddenConfigurations.entrySet().iterator() + while (it.hasNext) { + val kv = it.next() + SparkSQLEnv.sqlContext.setConf(kv.getKey, kv.getValue) + } + if (sessionState.execString != null) { System.exit(cli.processLine(sessionState.execString)) } 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 75535cad1b18e..d3cec11bd7567 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 @@ -91,6 +91,8 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath | --hiveconf ${ConfVars.SCRATCHDIR}=$scratchDirPath + | --hiveconf conf1=conftest + | --hiveconf conf2=1 """.stripMargin.split("\\s+").toSeq ++ extraArgs } @@ -272,4 +274,13 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { s"LIST FILE $dataFilePath;" -> "small_kv.txt" ) } + + test("apply hiveconf from cli command") { + runCliWithin(2.minute)( + "SET conf1;" -> "conftest", + "SET conf2;" -> "1", + "SET conf3=${hiveconf:conf1};" -> "conftest", + "SET conf3;" -> "conftest" + ) + } } From 801fb7994d890fa4112b97fc339520f5ce3ab6cb Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 5 Jul 2016 16:44:15 -0700 Subject: [PATCH 0891/1470] [SPARK-16359][STREAMING][KAFKA] unidoc skip kafka 0.10 ## What changes were proposed in this pull request? during sbt unidoc task, skip the streamingKafka010 subproject and filter kafka 0.10 classes from the classpath, so that at least existing kafka 0.8 doc can be included in unidoc without error ## How was this patch tested? sbt spark/scalaunidoc:doc | grep -i error Author: cody koeninger Closes #14041 from koeninger/SPARK-16359. (cherry picked from commit 1f0d021308f2201366111f8390015114710d4f9b) Signed-off-by: Tathagata Das --- project/SparkBuild.scala | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6018b22bae600..b1a9f393423bd 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -701,15 +701,29 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive/test"))) } + private def ignoreClasspaths(classpaths: Seq[Classpath]): Seq[Classpath] = { + classpaths + .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka-clients-0\.10.*"""))) + .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka_2\..*-0\.10.*"""))) + } + val unidocSourceBase = settingKey[String]("Base URL of source links in Scaladoc.") lazy val settings = scalaJavaUnidocSettings ++ Seq ( publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010), + + unidocAllClasspaths in (ScalaUnidoc, unidoc) := { + ignoreClasspaths((unidocAllClasspaths in (ScalaUnidoc, unidoc)).value) + }, + + unidocAllClasspaths in (JavaUnidoc, unidoc) := { + ignoreClasspaths((unidocAllClasspaths in (JavaUnidoc, unidoc)).value) + }, // Skip actual catalyst, but include the subproject. // Catalyst is not public API and contains quasiquotes which break scaladoc. From a2ef13a7d3b3daf888f9aba5e28fcdf813c4171c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 5 Jul 2016 16:55:22 -0700 Subject: [PATCH 0892/1470] [SPARK-16385][CORE] Catch correct exception when calling method via reflection. Using "Method.invoke" causes an exception to be thrown, not an error, so Utils.waitForProcess() was always throwing an exception when run on Java 7. Author: Marcelo Vanzin Closes #14056 from vanzin/SPARK-16385. (cherry picked from commit 59f9c1bd1adfea7069e769fb68351c228c37c8fc) Signed-off-by: Marcelo Vanzin --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0c23f3cd35458..156cf1748b2a6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1813,7 +1813,7 @@ private[spark] object Utils extends Logging { .invoke(process, timeoutMs.asInstanceOf[java.lang.Long], TimeUnit.MILLISECONDS) .asInstanceOf[Boolean] } catch { - case _: NoSuchMethodError => + case _: NoSuchMethodException => // Otherwise implement it manually var terminated = false val startTime = System.currentTimeMillis From 0fe2a8c161894ba7edbd89689e69a368392d564b Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 5 Jul 2016 17:00:24 -0700 Subject: [PATCH 0893/1470] [SPARK-16348][ML][MLLIB][PYTHON] Use full classpaths for pyspark ML JVM calls ## What changes were proposed in this pull request? Issue: Omitting the full classpath can cause problems when calling JVM methods or classes from pyspark. This PR: Changed all uses of jvm.X in pyspark.ml and pyspark.mllib to use full classpath for X ## How was this patch tested? Existing unit tests. Manual testing in an environment where this was an issue. Author: Joseph K. Bradley Closes #14023 from jkbradley/SPARK-16348. (cherry picked from commit fdde7d0aa0ef69d0e9a88cf712601bba1d5b0706) Signed-off-by: Joseph K. Bradley --- python/pyspark/ml/common.py | 10 +++++----- python/pyspark/ml/tests.py | 8 ++++---- python/pyspark/mllib/clustering.py | 5 +++-- python/pyspark/mllib/common.py | 10 +++++----- python/pyspark/mllib/feature.py | 2 +- python/pyspark/mllib/fpm.py | 2 +- python/pyspark/mllib/recommendation.py | 2 +- python/pyspark/mllib/tests.py | 15 ++++++++------- 8 files changed, 28 insertions(+), 26 deletions(-) diff --git a/python/pyspark/ml/common.py b/python/pyspark/ml/common.py index 256e91e14165e..7d449aaccb44f 100644 --- a/python/pyspark/ml/common.py +++ b/python/pyspark/ml/common.py @@ -63,7 +63,7 @@ def _to_java_object_rdd(rdd): RDD is serialized in batch or not. """ rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) - return rdd.ctx._jvm.MLSerDe.pythonToJava(rdd._jrdd, True) + return rdd.ctx._jvm.org.apache.spark.ml.python.MLSerDe.pythonToJava(rdd._jrdd, True) def _py2java(sc, obj): @@ -82,7 +82,7 @@ def _py2java(sc, obj): pass else: data = bytearray(PickleSerializer().dumps(obj)) - obj = sc._jvm.MLSerDe.loads(data) + obj = sc._jvm.org.apache.spark.ml.python.MLSerDe.loads(data) return obj @@ -95,17 +95,17 @@ def _java2py(sc, r, encoding="bytes"): clsName = 'JavaRDD' if clsName == 'JavaRDD': - jrdd = sc._jvm.MLSerDe.javaToPython(r) + jrdd = sc._jvm.org.apache.spark.ml.python.MLSerDe.javaToPython(r) return RDD(jrdd, sc) if clsName == 'Dataset': return DataFrame(r, SQLContext.getOrCreate(sc)) if clsName in _picklable_classes: - r = sc._jvm.MLSerDe.dumps(r) + r = sc._jvm.org.apache.spark.ml.python.MLSerDe.dumps(r) elif isinstance(r, (JavaArray, JavaList)): try: - r = sc._jvm.MLSerDe.dumps(r) + r = sc._jvm.org.apache.spark.ml.python.MLSerDe.dumps(r) except Py4JJavaError: pass # not pickable diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 981ed9dda042c..24efce812b3b3 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -1195,12 +1195,12 @@ class VectorTests(MLlibTestCase): def _test_serialize(self, v): self.assertEqual(v, ser.loads(ser.dumps(v))) - jvec = self.sc._jvm.MLSerDe.loads(bytearray(ser.dumps(v))) - nv = ser.loads(bytes(self.sc._jvm.MLSerDe.dumps(jvec))) + jvec = self.sc._jvm.org.apache.spark.ml.python.MLSerDe.loads(bytearray(ser.dumps(v))) + nv = ser.loads(bytes(self.sc._jvm.org.apache.spark.ml.python.MLSerDe.dumps(jvec))) self.assertEqual(v, nv) vs = [v] * 100 - jvecs = self.sc._jvm.MLSerDe.loads(bytearray(ser.dumps(vs))) - nvs = ser.loads(bytes(self.sc._jvm.MLSerDe.dumps(jvecs))) + jvecs = self.sc._jvm.org.apache.spark.ml.python.MLSerDe.loads(bytearray(ser.dumps(vs))) + nvs = ser.loads(bytes(self.sc._jvm.org.apache.spark.ml.python.MLSerDe.dumps(jvecs))) self.assertEqual(vs, nvs) def test_serialize(self): diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 95f7278dc64ce..93a0b64569b13 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -507,7 +507,7 @@ def load(cls, sc, path): Path to where the model is stored. """ model = cls._load_java(sc, path) - wrapper = sc._jvm.GaussianMixtureModelWrapper(model) + wrapper = sc._jvm.org.apache.spark.mllib.api.python.GaussianMixtureModelWrapper(model) return cls(wrapper) @@ -638,7 +638,8 @@ def load(cls, sc, path): Load a model from the given path. """ model = cls._load_java(sc, path) - wrapper = sc._jvm.PowerIterationClusteringModelWrapper(model) + wrapper =\ + sc._jvm.org.apache.spark.mllib.api.python.PowerIterationClusteringModelWrapper(model) return PowerIterationClusteringModel(wrapper) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index 31afdf576b677..21f0e09ea7742 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -66,7 +66,7 @@ def _to_java_object_rdd(rdd): RDD is serialized in batch or not. """ rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) - return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) + return rdd.ctx._jvm.org.apache.spark.mllib.api.python.SerDe.pythonToJava(rdd._jrdd, True) def _py2java(sc, obj): @@ -85,7 +85,7 @@ def _py2java(sc, obj): pass else: data = bytearray(PickleSerializer().dumps(obj)) - obj = sc._jvm.SerDe.loads(data) + obj = sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads(data) return obj @@ -98,17 +98,17 @@ def _java2py(sc, r, encoding="bytes"): clsName = 'JavaRDD' if clsName == 'JavaRDD': - jrdd = sc._jvm.SerDe.javaToPython(r) + jrdd = sc._jvm.org.apache.spark.mllib.api.python.SerDe.javaToPython(r) return RDD(jrdd, sc) if clsName == 'Dataset': return DataFrame(r, SQLContext.getOrCreate(sc)) if clsName in _picklable_classes: - r = sc._jvm.SerDe.dumps(r) + r = sc._jvm.org.apache.spark.mllib.api.python.SerDe.dumps(r) elif isinstance(r, (JavaArray, JavaList)): try: - r = sc._jvm.SerDe.dumps(r) + r = sc._jvm.org.apache.spark.mllib.api.python.SerDe.dumps(r) except Py4JJavaError: pass # not pickable diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index e31c75c1e8675..aef91a8ddc1f1 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -553,7 +553,7 @@ def load(cls, sc, path): """ jmodel = sc._jvm.org.apache.spark.mllib.feature \ .Word2VecModel.load(sc._jsc.sc(), path) - model = sc._jvm.Word2VecModelWrapper(jmodel) + model = sc._jvm.org.apache.spark.mllib.api.python.Word2VecModelWrapper(jmodel) return Word2VecModel(model) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index ab4066f7d68ba..fb226e84e5d50 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -64,7 +64,7 @@ def load(cls, sc, path): Load a model from the given path. """ model = cls._load_java(sc, path) - wrapper = sc._jvm.FPGrowthModelWrapper(model) + wrapper = sc._jvm.org.apache.spark.mllib.api.python.FPGrowthModelWrapper(model) return FPGrowthModel(wrapper) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 7e60255d43ead..732300ee9c2c9 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -207,7 +207,7 @@ def rank(self): def load(cls, sc, path): """Load a model from the given path""" model = cls._load_java(sc, path) - wrapper = sc._jvm.MatrixFactorizationModelWrapper(model) + wrapper = sc._jvm.org.apache.spark.mllib.api.python.MatrixFactorizationModelWrapper(model) return MatrixFactorizationModel(wrapper) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 72fa8b5f3d477..99bf50b5a1640 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -150,12 +150,12 @@ class VectorTests(MLlibTestCase): 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(bytes(self.sc._jvm.SerDe.dumps(jvec))) + jvec = self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads(bytearray(ser.dumps(v))) + nv = ser.loads(bytes(self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.dumps(jvec))) self.assertEqual(v, nv) vs = [v] * 100 - jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) - nvs = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvecs))) + jvecs = self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads(bytearray(ser.dumps(vs))) + nvs = ser.loads(bytes(self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.dumps(jvecs))) self.assertEqual(vs, nvs) def test_serialize(self): @@ -1650,8 +1650,8 @@ class ALSTests(MLlibTestCase): def test_als_ratings_serialize(self): r = Rating(7, 1123, 3.14) - jr = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(r))) - nr = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jr))) + jr = self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads(bytearray(ser.dumps(r))) + nr = ser.loads(bytes(self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.dumps(jr))) self.assertEqual(r.user, nr.user) self.assertEqual(r.product, nr.product) self.assertAlmostEqual(r.rating, nr.rating, 2) @@ -1659,7 +1659,8 @@ def test_als_ratings_serialize(self): def test_als_ratings_id_long_error(self): r = Rating(1205640308657491975, 50233468418, 1.0) # rating user id exceeds max int value, should fail when pickled - self.assertRaises(Py4JJavaError, self.sc._jvm.SerDe.loads, bytearray(ser.dumps(r))) + self.assertRaises(Py4JJavaError, self.sc._jvm.org.apache.spark.mllib.api.python.SerDe.loads, + bytearray(ser.dumps(r))) class HashingTFTest(MLlibTestCase): From 4a55b2326c8cf50f772907a8b73fd5e7b3d1aa06 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 5 Jul 2016 18:40:45 -0700 Subject: [PATCH 0894/1470] Preparing Spark release v2.0.0-rc2 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 507ddc778383d..5f546bb9ee697 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bc3b0fe73f6eb..2eaa8100a61d4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 2fb5835305a25..f068d9d4ac428 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 07d9f1c58f7a3..fd221883b1fba 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5e02efdc45e67..a17aba5abe548 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e7fc6a2a02417..0bd8846f452c5 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 24f0e75f2f045..f495b0dddee54 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index bb27ec916c65a..093ca69abb866 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d2227944d92dd..fe9f85ae5a085 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 18e14c7981d80..ca5e4980eb8c7 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 3847b39e17315..b4cdbda6b613b 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7fc8a4b00dc20..4e7e17de6a417 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index dcdbcc55d8785..cc5a13721ee3f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1eb349ff4c3b9..636b233a0de04 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 59f41f1e17f30..093cfb76b6aea 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 26965612cc0ab..b4e9eba6efd4d 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a346d76210932..1438e2ba59e6a 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 2aa7d82e8e064..dc704375dcc44 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 58c57c1c539df..af0b81a0cff82 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b524001d04719..919778bac01e9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 93ac8b6e664d7..f7588d1917ba5 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4f8af77792b0b..f8dd7f7fd751c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index b6591598ee127..e015b95cafd15 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1d8f7f4d9bbed..86edf650b5c7b 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40fde1bab7ad1..407cb6da8281c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 9f3d7f003584c..ff7b36f495a85 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 0b5ec1a08c821..9ca84e986a3dc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0bfdb13cec941..259e56e4afef9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02a18b33b087a..0b06c29302d6e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 672425c86ecbe..ce66c71ccae6e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index add4375364b1a..5ad0c51ad05c8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index e7614fe66110d..21d8d062bd380 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 3f4cce1ca354e..a2124104f4fb9 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 7dba1a829fb94..1ca840d8c0968 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml From 6e8fa86ebf30a9b850f4a66810d5d38d1f188b33 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 5 Jul 2016 18:40:51 -0700 Subject: [PATCH 0895/1470] Preparing development version 2.0.1-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5f546bb9ee697..507ddc778383d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2eaa8100a61d4..bc3b0fe73f6eb 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f068d9d4ac428..2fb5835305a25 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index fd221883b1fba..07d9f1c58f7a3 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index a17aba5abe548..5e02efdc45e67 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 0bd8846f452c5..e7fc6a2a02417 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index f495b0dddee54..24f0e75f2f045 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 093ca69abb866..bb27ec916c65a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index fe9f85ae5a085..d2227944d92dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ca5e4980eb8c7..18e14c7981d80 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index b4cdbda6b613b..3847b39e17315 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 4e7e17de6a417..7fc8a4b00dc20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index cc5a13721ee3f..dcdbcc55d8785 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 636b233a0de04..1eb349ff4c3b9 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 093cfb76b6aea..59f41f1e17f30 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index b4e9eba6efd4d..26965612cc0ab 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 1438e2ba59e6a..a346d76210932 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dc704375dcc44..2aa7d82e8e064 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index af0b81a0cff82..58c57c1c539df 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 919778bac01e9..b524001d04719 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index f7588d1917ba5..93ac8b6e664d7 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f8dd7f7fd751c..4f8af77792b0b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e015b95cafd15..b6591598ee127 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 86edf650b5c7b..1d8f7f4d9bbed 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 407cb6da8281c..40fde1bab7ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ff7b36f495a85..9f3d7f003584c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 9ca84e986a3dc..0b5ec1a08c821 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 259e56e4afef9..0bfdb13cec941 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0b06c29302d6e..02a18b33b087a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index ce66c71ccae6e..672425c86ecbe 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5ad0c51ad05c8..add4375364b1a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 21d8d062bd380..e7614fe66110d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2124104f4fb9..3f4cce1ca354e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 1ca840d8c0968..7dba1a829fb94 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml From 521fc7186a2637321f7a7cfac713537de73ae66f Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Wed, 6 Jul 2016 09:18:04 +0100 Subject: [PATCH 0896/1470] [SPARK-16339][CORE] ScriptTransform does not print stderr when outstream is lost MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Currently, if due to some failure, the outstream gets destroyed or closed and later `outstream.close()` leads to IOException in such case. Due to this, the `stderrBuffer` does not get logged and there is no way for users to see why the job failed. The change is to first display the stderr buffer and then try closing the outstream. ## How was this patch tested? The correct way to test this fix would be to grep the log to see if the `stderrBuffer` gets logged but I dont think having test cases which do that is a good idea. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) … Author: Tejas Patil Closes #13834 from tejasapatil/script_transform. (cherry picked from commit 5f342049cce9102fb62b4de2d8d8fa691c2e8ac4) Signed-off-by: Sean Owen --- .../spark/sql/hive/execution/ScriptTransformation.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 9e25e1d40ce81..dfb12512a40fc 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 @@ -312,15 +312,15 @@ private class ScriptTransformationWriterThread( } threwException = false } catch { - case NonFatal(e) => + case t: Throwable => // An error occurred while writing input, so kill the child process. According to the // Javadoc this call will not throw an exception: - _exception = e + _exception = t proc.destroy() - throw e + throw t } finally { try { - outputStream.close() + Utils.tryLogNonFatalError(outputStream.close()) if (proc.waitFor() != 0) { logError(stderrBuffer.toString) // log the stderr circular buffer } From 25006c8bcc397c9f070cc5d685ffeb5b8fb0a341 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Wed, 6 Jul 2016 01:30:47 -0700 Subject: [PATCH 0897/1470] [SPARK-16249][ML] Change visibility of Object ml.clustering.LDA to public for loading ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-16249 Change visibility of Object ml.clustering.LDA to public for loading, thus users can invoke LDA.load("path"). ## How was this patch tested? existing ut and manually test for load ( saved with current code) Author: Yuhao Yang Author: Yuhao Yang Closes #13941 from hhbyyh/ldapublic. (cherry picked from commit 5497242c769b40338bfa57d64f2c64996dfa57e8) Signed-off-by: Yanbo Liang --- .../scala/org/apache/spark/ml/clustering/LDA.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index b333d59258238..778cd0fee71c0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -880,11 +880,13 @@ class LDA @Since("1.6.0") ( } } - -private[clustering] object LDA extends DefaultParamsReadable[LDA] { +@Since("2.0.0") +object LDA extends DefaultParamsReadable[LDA] { /** Get dataset for spark.mllib LDA */ - def getOldDataset(dataset: Dataset[_], featuresCol: String): RDD[(Long, OldVector)] = { + private[clustering] def getOldDataset( + dataset: Dataset[_], + featuresCol: String): RDD[(Long, OldVector)] = { dataset .withColumn("docId", monotonicallyIncreasingId()) .select("docId", featuresCol) @@ -894,6 +896,6 @@ private[clustering] object LDA extends DefaultParamsReadable[LDA] { } } - @Since("1.6.0") + @Since("2.0.0") override def load(path: String): LDA = super.load(path) } From d5d2457e41661a3402a8258026856454222a2f54 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 6 Jul 2016 21:35:56 +0800 Subject: [PATCH 0898/1470] [SPARK-15968][SQL] Nonempty partitioned metastore tables are not cached This PR backports your fix (https://github.com/apache/spark/pull/13818) to branch 2.0. This PR addresses [SPARK-15968](https://issues.apache.org/jira/browse/SPARK-15968). ## What changes were proposed in this pull request? The `getCached` method of [HiveMetastoreCatalog](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala) computes `pathsInMetastore` from the metastore relation's catalog table. This only returns the table base path, which is incomplete/inaccurate for a nonempty partitioned table. As a result, cached lookups on nonempty partitioned tables always miss. Rather than get `pathsInMetastore` from metastoreRelation.catalogTable.storage.locationUri.toSeq I modified the `getCached` method to take a `pathsInMetastore` argument. Calls to this method pass in the paths computed from calls to the Hive metastore. This is how `getCached` was implemented in Spark 1.5: https://github.com/apache/spark/blob/e0c3212a9b42e3e704b070da4ac25b68c584427f/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala#L444. I also added a call in `InsertIntoHiveTable.scala` to invalidate the table from the SQL session catalog. ## How was this patch tested? I've added a new unit test to `parquetSuites.scala`: SPARK-15968: nonempty partitioned metastore Parquet table lookup should use cached relation Note that the only difference between this new test and the one above it in the file is that the new test populates its partitioned table with a single value, while the existing test leaves the table empty. This reveals a subtle, unexpected hole in test coverage present before this patch. Note I also modified a different but related unit test in `parquetSuites.scala`: SPARK-15248: explicitly added partitions should be readable This unit test asserts that Spark SQL should return data from a table partition which has been placed there outside a metastore query immediately after it is added. I changed the test so that, instead of adding the data as a parquet file saved in the partition's location, the data is added through a SQL `INSERT` query. I made this change because I could find no way to efficiently support partitioned table caching without failing that test. In addition to my primary motivation, I can offer a few reasons I believe this is an acceptable weakening of that test. First, it still validates a fix for [SPARK-15248](https://issues.apache.org/jira/browse/SPARK-15248), the issue for which it was written. Second, the assertion made is stronger than that required for non-partitioned tables. If you write data to the storage location of a non-partitioned metastore table without using a proper SQL DML query, a subsequent call to show that data will not return it. I believe this is an intentional limitation put in place to make table caching feasible, but I'm only speculating. Building a large `HadoopFsRelation` requires `stat`-ing all of its data files. In our environment, where we have tables with 10's of thousands of partitions, the difference between using a cached relation versus a new one is a matter of seconds versus minutes. Caching partitioned table metadata vastly improves the usability of Spark SQL for these cases. Author: Reynold Xin Author: Michael Allman Closes #14064 from yhuai/spark-15968-branch-2.0. --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 16 ++++- .../hive/execution/InsertIntoHiveTable.scala | 1 + .../apache/spark/sql/hive/parquetSuites.scala | 61 +++++++++++++------ 3 files changed, 59 insertions(+), 19 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 d6c532563ff9c..789f94aff3036 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 @@ -187,6 +187,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log private def getCached( tableIdentifier: QualifiedTableName, + pathsInMetastore: Seq[String], metastoreRelation: MetastoreRelation, schemaInMetastore: StructType, expectedFileFormat: Class[_ <: FileFormat], @@ -196,7 +197,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => None // Cache miss case logical @ LogicalRelation(relation: HadoopFsRelation, _, _) => - val pathsInMetastore = metastoreRelation.catalogTable.storage.locationUri.toSeq val cachedRelationFileFormatClass = relation.fileFormat.getClass expectedFileFormat match { @@ -261,9 +261,22 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log PartitionDirectory(values, location) } val partitionSpec = PartitionSpec(partitionSchema, partitions) + val partitionPaths = partitions.map(_.path.toString) + + // By convention (for example, see MetaStorePartitionedTableFileCatalog), the definition of a + // partitioned table's paths depends on whether that table has any actual partitions. + // Partitioned tables without partitions use the location of the table's base path. + // Partitioned tables with partitions use the locations of those partitions' data locations, + // _omitting_ the table's base path. + val paths = if (partitionPaths.isEmpty) { + Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) + } else { + partitionPaths + } val cached = getCached( tableIdentifier, + paths, metastoreRelation, metastoreSchema, fileFormatClass, @@ -308,6 +321,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val paths = Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) val cached = getCached(tableIdentifier, + paths, metastoreRelation, metastoreSchema, fileFormatClass, 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 97cd29f541ed9..3d58d490a51e2 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 @@ -298,6 +298,7 @@ case class InsertIntoHiveTable( // Invalidate the cache. sqlContext.sharedState.cacheManager.invalidateCache(table) + sqlContext.sessionState.catalog.refreshTable(table.catalogTable.identifier) // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which 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 a78a28536cc3b..96beb2d3427b1 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 @@ -389,17 +389,18 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { test("SPARK-7749: non-partitioned metastore Parquet table lookup should use cached relation") { withTable("nonPartitioned") { sql( - s"""CREATE TABLE nonPartitioned ( - | key INT, - | value STRING - |) - |STORED AS PARQUET - """.stripMargin) + """ + |CREATE TABLE nonPartitioned ( + | key INT, + | value STRING + |) + |STORED AS PARQUET + """.stripMargin) // First lookup fills the cache - val r1 = collectHadoopFsRelation (table("nonPartitioned")) + val r1 = collectHadoopFsRelation(table("nonPartitioned")) // Second lookup should reuse the cache - val r2 = collectHadoopFsRelation (table("nonPartitioned")) + val r2 = collectHadoopFsRelation(table("nonPartitioned")) // They should be the same instance assert(r1 eq r2) } @@ -408,18 +409,42 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { test("SPARK-7749: partitioned metastore Parquet table lookup should use cached relation") { withTable("partitioned") { sql( - s"""CREATE TABLE partitioned ( - | key INT, - | value STRING - |) - |PARTITIONED BY (part INT) - |STORED AS PARQUET - """.stripMargin) + """ + |CREATE TABLE partitioned ( + | key INT, + | value STRING + |) + |PARTITIONED BY (part INT) + |STORED AS PARQUET + """.stripMargin) + + // First lookup fills the cache + val r1 = collectHadoopFsRelation(table("partitioned")) + // Second lookup should reuse the cache + val r2 = collectHadoopFsRelation(table("partitioned")) + // They should be the same instance + assert(r1 eq r2) + } + } + + test("SPARK-15968: nonempty partitioned metastore Parquet table lookup should use cached " + + "relation") { + withTable("partitioned") { + sql( + """ + |CREATE TABLE partitioned ( + | key INT, + | value STRING + |) + |PARTITIONED BY (part INT) + |STORED AS PARQUET + """.stripMargin) + sql("INSERT INTO TABLE partitioned PARTITION(part=0) SELECT 1 as key, 'one' as value") // First lookup fills the cache - val r1 = collectHadoopFsRelation (table("partitioned")) + val r1 = collectHadoopFsRelation(table("partitioned")) // Second lookup should reuse the cache - val r2 = collectHadoopFsRelation (table("partitioned")) + val r2 = collectHadoopFsRelation(table("partitioned")) // They should be the same instance assert(r1 eq r2) } @@ -557,7 +582,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { Seq(("foo", 0), ("bar", 0)).toDF("a", "b")) // Add data files to partition directory and check whether they can be read - Seq("baz").toDF("a").write.mode(SaveMode.Overwrite).parquet(partitionDir) + sql("INSERT INTO TABLE test_added_partitions PARTITION (b=1) select 'baz' as a") checkAnswer( sql("SELECT * FROM test_added_partitions"), Seq(("foo", 0), ("bar", 0), ("baz", 1)).toDF("a", "b")) From e956bd7750882ce259a278e9eac7f64b4fb76286 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 6 Jul 2016 21:43:55 +0800 Subject: [PATCH 0899/1470] [SPARK-16229][SQL] Drop Empty Table After CREATE TABLE AS SELECT fails #### What changes were proposed in this pull request? In `CREATE TABLE AS SELECT`, if the `SELECT` query failed, the table should not exist. For example, ```SQL CREATE TABLE tab STORED AS TEXTFILE SELECT 1 AS a, (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b ``` The above query failed as expected but an empty table `t` is created. This PR is to drop the created table when hitting any non-fatal exception. #### How was this patch tested? Added a test case to verify the behavior Author: gatorsmile Closes #13926 from gatorsmile/dropTableAfterException. (cherry picked from commit 21eadd1d8cbf029197e73ffca1cba54d5a890c01) Signed-off-by: Wenchen Fan --- .../CreateHiveTableAsSelectCommand.scala | 13 +++++++++++-- .../spark/sql/hive/execution/HiveDDLSuite.scala | 15 +++++++++++++++ 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index b8099385a466b..15a5d79dcb085 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import scala.util.control.NonFatal + import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} @@ -87,8 +89,15 @@ case class CreateHiveTableAsSelectCommand( throw new AnalysisException(s"$tableIdentifier already exists.") } } else { - sparkSession.sessionState.executePlan(InsertIntoTable( - metastoreRelation, Map(), query, overwrite = true, ifNotExists = false)).toRdd + try { + sparkSession.sessionState.executePlan(InsertIntoTable( + metastoreRelation, Map(), query, overwrite = true, ifNotExists = false)).toRdd + } catch { + case NonFatal(e) => + // drop the created table. + sparkSession.sessionState.catalog.dropTable(tableIdentifier, ignoreIfNotExists = true) + throw e + } } Seq.empty[Row] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 89f69c8e4d7f1..9d3c4cd3d570c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -554,6 +554,21 @@ class HiveDDLSuite } } + test("Create Cataloged Table As Select - Drop Table After Runtime Exception") { + withTable("tab") { + intercept[RuntimeException] { + sql( + """ + |CREATE TABLE tab + |STORED AS TEXTFILE + |SELECT 1 AS a, (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b + """.stripMargin) + } + // After hitting runtime exception, we should drop the created table. + assert(!spark.sessionState.catalog.tableExists(TableIdentifier("tab"))) + } + } + test("desc table for data source table") { withTable("tab1") { val tabName = "tab1" From 091cd5f265166512a450333946c62c3eb3440e79 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 6 Jul 2016 10:41:48 -0700 Subject: [PATCH 0900/1470] [DOC][SQL] update out-of-date code snippets using SQLContext in all documents. ## What changes were proposed in this pull request? I search the whole documents directory using SQLContext, and update the following places: - docs/configuration.md, sparkR code snippets. - docs/streaming-programming-guide.md, several example code. ## How was this patch tested? N/A Author: WeichenXu Closes #14025 from WeichenXu123/WIP_SQLContext_update. (cherry picked from commit b1310425b30cbd711e4834d65a0accb3c5a8403a) Signed-off-by: Reynold Xin --- docs/configuration.md | 4 +-- docs/streaming-programming-guide.md | 39 ++++++++++++++++------------- 2 files changed, 23 insertions(+), 20 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index cee59cf2aa05f..1e95b862441f5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1564,8 +1564,8 @@ spark.sql("SET -v").show(n=200, truncate=False)

      {% highlight r %} -# sqlContext is an existing sqlContext. -properties <- sql(sqlContext, "SET -v") +sparkR.session() +properties <- sql("SET -v") showDF(properties, numRows = 200, truncate = FALSE) {% endhighlight %} diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index db06a65b994be..2ee3b80185c2f 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1534,7 +1534,7 @@ See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/ma *** ## DataFrame and SQL Operations -You can easily use [DataFrames and SQL](sql-programming-guide.html) operations on streaming data. You have to create a SQLContext using the SparkContext that the StreamingContext is using. Furthermore this has to done such that it can be restarted on driver failures. This is done by creating a lazily instantiated singleton instance of SQLContext. This is shown in the following example. It modifies the earlier [word count example](#a-quick-example) to generate word counts using DataFrames and SQL. Each RDD is converted to a DataFrame, registered as a temporary table and then queried using SQL. +You can easily use [DataFrames and SQL](sql-programming-guide.html) operations on streaming data. You have to create a SparkSession using the SparkContext that the StreamingContext is using. Furthermore this has to done such that it can be restarted on driver failures. This is done by creating a lazily instantiated singleton instance of SparkSession. This is shown in the following example. It modifies the earlier [word count example](#a-quick-example) to generate word counts using DataFrames and SQL. Each RDD is converted to a DataFrame, registered as a temporary table and then queried using SQL.
      @@ -1546,9 +1546,9 @@ val words: DStream[String] = ... words.foreachRDD { rdd => - // Get the singleton instance of SQLContext - val sqlContext = SQLContext.getOrCreate(rdd.sparkContext) - import sqlContext.implicits._ + // Get the singleton instance of SparkSession + val spark = SparkSession.builder.config(rdd.sparkContext.getConf).getOrCreate() + import spark.implicits._ // Convert RDD[String] to DataFrame val wordsDataFrame = rdd.toDF("word") @@ -1558,7 +1558,7 @@ words.foreachRDD { rdd => // Do word count on DataFrame using SQL and print it val wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() } @@ -1593,8 +1593,8 @@ words.foreachRDD( @Override public Void call(JavaRDD rdd, Time time) { - // Get the singleton instance of SQLContext - SQLContext sqlContext = SQLContext.getOrCreate(rdd.context()); + // Get the singleton instance of SparkSession + SparkSession spark = SparkSession.builder().config(rdd.sparkContext().getConf()).getOrCreate(); // Convert RDD[String] to RDD[case class] to DataFrame JavaRDD rowRDD = rdd.map(new Function() { @@ -1604,14 +1604,14 @@ words.foreachRDD( return record; } }); - DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class); + DataFrame wordsDataFrame = spark.createDataFrame(rowRDD, JavaRow.class); // Creates a temporary view using the DataFrame wordsDataFrame.createOrReplaceTempView("words"); // Do word count on table using SQL and print it DataFrame wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word"); + spark.sql("select word, count(*) as total from words group by word"); wordCountsDataFrame.show(); return null; } @@ -1624,11 +1624,14 @@ See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/ma
      {% highlight python %} -# Lazily instantiated global instance of SQLContext -def getSqlContextInstance(sparkContext): - if ('sqlContextSingletonInstance' not in globals()): - globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) - return globals()['sqlContextSingletonInstance'] +# Lazily instantiated global instance of SparkSession +def getSparkSessionInstance(sparkConf): + if ('sparkSessionSingletonInstance' not in globals()): + globals()['sparkSessionSingletonInstance'] = SparkSession\ + .builder\ + .config(conf=sparkConf)\ + .getOrCreate() + return globals()['sparkSessionSingletonInstance'] ... @@ -1639,18 +1642,18 @@ words = ... # DStream of strings def process(time, rdd): print("========= %s =========" % str(time)) try: - # Get the singleton instance of SQLContext - sqlContext = getSqlContextInstance(rdd.context) + # Get the singleton instance of SparkSession + spark = getSparkSessionInstance(rdd.context.getConf()) # Convert RDD[String] to RDD[Row] to DataFrame rowRdd = rdd.map(lambda w: Row(word=w)) - wordsDataFrame = sqlContext.createDataFrame(rowRdd) + wordsDataFrame = spark.createDataFrame(rowRdd) # Creates a temporary view using the DataFrame wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it - wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame = spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() except: pass From 03f336d8921e1f22ee4d1f6fa8869163b1f29ea9 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 6 Jul 2016 10:45:51 -0700 Subject: [PATCH 0901/1470] [MINOR][PYSPARK][DOC] Fix wrongly formatted examples in PySpark documentation ## What changes were proposed in this pull request? This PR fixes wrongly formatted examples in PySpark documentation as below: - **`SparkSession`** - **Before** ![2016-07-06 11 34 41](https://cloud.githubusercontent.com/assets/6477701/16605847/ae939526-436d-11e6-8ab8-6ad578362425.png) - **After** ![2016-07-06 11 33 56](https://cloud.githubusercontent.com/assets/6477701/16605845/ace9ee78-436d-11e6-8923-b76d4fc3e7c3.png) - **`Builder`** - **Before** ![2016-07-06 11 34 44](https://cloud.githubusercontent.com/assets/6477701/16605844/aba60dbc-436d-11e6-990a-c87bc0281c6b.png) - **After** ![2016-07-06 1 26 37](https://cloud.githubusercontent.com/assets/6477701/16607562/586704c0-437d-11e6-9483-e0af93d8f74e.png) This PR also fixes several similar instances across the documentation in `sql` PySpark module. ## How was this patch tested? N/A Author: hyukjinkwon Closes #14063 from HyukjinKwon/minor-pyspark-builder. (cherry picked from commit 4e14199ff740ea186eb2cec2e5cf901b58c5f90e) Signed-off-by: Reynold Xin --- python/pyspark/mllib/clustering.py | 14 +++++++------- python/pyspark/sql/dataframe.py | 8 ++++---- python/pyspark/sql/functions.py | 8 ++++---- python/pyspark/sql/group.py | 2 ++ python/pyspark/sql/session.py | 13 +++++++------ python/pyspark/sql/types.py | 4 ++-- 6 files changed, 26 insertions(+), 23 deletions(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 93a0b64569b13..c38c543972d13 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -571,14 +571,14 @@ class PowerIterationClusteringModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> import math >>> def genCircle(r, n): - ... points = [] - ... for i in range(0, n): - ... theta = 2.0 * math.pi * i / n - ... points.append((r * math.cos(theta), r * math.sin(theta))) - ... return points + ... points = [] + ... for i in range(0, n): + ... theta = 2.0 * math.pi * i / n + ... points.append((r * math.cos(theta), r * math.sin(theta))) + ... return points >>> def sim(x, y): - ... dist2 = (x[0] - y[0]) * (x[0] - y[0]) + (x[1] - y[1]) * (x[1] - y[1]) - ... return math.exp(-dist2 / 2.0) + ... dist2 = (x[0] - y[0]) * (x[0] - y[0]) + (x[1] - y[1]) * (x[1] - y[1]) + ... return math.exp(-dist2 / 2.0) >>> r1 = 1.0 >>> n1 = 10 >>> r2 = 4.0 diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index e6e70291da5b3..c7d704a18adaf 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1033,10 +1033,10 @@ def dropDuplicates(self, subset=None): :func:`drop_duplicates` is an alias for :func:`dropDuplicates`. >>> from pyspark.sql import Row - >>> df = sc.parallelize([ \ - Row(name='Alice', age=5, height=80), \ - Row(name='Alice', age=5, height=80), \ - Row(name='Alice', age=10, height=80)]).toDF() + >>> df = sc.parallelize([ \\ + ... Row(name='Alice', age=5, height=80), \\ + ... Row(name='Alice', age=5, height=80), \\ + ... Row(name='Alice', age=10, height=80)]).toDF() >>> df.dropDuplicates().show() +---+------+-----+ |age|height| name| diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 15cefc8cf1125..1feca6e8caafa 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1550,8 +1550,8 @@ def translate(srcCol, matching, replace): The translate will happen when any character in the string matching with the character in the `matching`. - >>> spark.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123")\ - .alias('r')).collect() + >>> spark.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123") \\ + ... .alias('r')).collect() [Row(r=u'1a2s3ae')] """ sc = SparkContext._active_spark_context @@ -1649,8 +1649,8 @@ def get_json_object(col, path): >>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')] >>> df = spark.createDataFrame(data, ("key", "jstring")) - >>> df.select(df.key, get_json_object(df.jstring, '$.f1').alias("c0"), \ - get_json_object(df.jstring, '$.f2').alias("c1") ).collect() + >>> df.select(df.key, get_json_object(df.jstring, '$.f1').alias("c0"), \\ + ... get_json_object(df.jstring, '$.f2').alias("c1") ).collect() [Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)] """ sc = SparkContext._active_spark_context diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index a4232065540ea..f2092f9c63054 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -179,10 +179,12 @@ def pivot(self, pivot_col, values=None): :param values: List of values that will be translated to columns in the output DataFrame. # Compute the sum of earnings for each year by course with each course as a separate column + >>> df4.groupBy("year").pivot("course", ["dotNET", "Java"]).sum("earnings").collect() [Row(year=2012, dotNET=15000, Java=20000), Row(year=2013, dotNET=48000, Java=30000)] # Or without specifying column values (less efficient) + >>> df4.groupBy("year").pivot("course").sum("earnings").collect() [Row(year=2012, Java=20000, dotNET=15000), Row(year=2013, Java=30000, dotNET=48000)] """ diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 55f86a16f50a2..a360fbefa492c 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -66,12 +66,11 @@ class SparkSession(object): tables, execute SQL over tables, cache tables, and read parquet files. To create a SparkSession, use the following builder pattern: - >>> spark = SparkSession.builder \ - .master("local") \ - .appName("Word Count") \ - .config("spark.some.config.option", "some-value") \ - .getOrCreate() - + >>> spark = SparkSession.builder \\ + ... .master("local") \\ + ... .appName("Word Count") \\ + ... .config("spark.some.config.option", "some-value") \\ + ... .getOrCreate() """ class Builder(object): @@ -87,11 +86,13 @@ def config(self, key=None, value=None, conf=None): both :class:`SparkConf` and :class:`SparkSession`'s own configuration. For an existing SparkConf, use `conf` parameter. + >>> from pyspark.conf import SparkConf >>> SparkSession.builder.config(conf=SparkConf()) >> SparkSession.builder.config("spark.some.config.option", "some-value") >> struct1 = StructType().add("f1", StringType(), True).add("f2", StringType(), True, None) - >>> struct2 = StructType([StructField("f1", StringType(), True),\ - StructField("f2", StringType(), True, None)]) + >>> struct2 = StructType([StructField("f1", StringType(), True), \\ + ... StructField("f2", StringType(), True, None)]) >>> struct1 == struct2 True >>> struct1 = StructType().add(StructField("f1", StringType(), True)) From 2465f0728e95109ab851ab09b5badd697928ba2b Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 6 Jul 2016 12:42:16 -0700 Subject: [PATCH 0902/1470] [SPARK-16371][SQL] Do not push down filters incorrectly when inner name and outer name are the same in Parquet ## What changes were proposed in this pull request? Currently, if there is a schema as below: ``` root |-- _1: struct (nullable = true) | |-- _1: integer (nullable = true) ``` and if we execute the codes below: ```scala df.filter("_1 IS NOT NULL").count() ``` This pushes down a filter although this filter is being applied to `StructType`.(If my understanding is correct, Spark does not pushes down filters for those). The reason is, `ParquetFilters.getFieldMap` produces results below: ``` (_1,StructType(StructField(_1,IntegerType,true))) (_1,IntegerType) ``` and then it becomes a `Map` ``` (_1,IntegerType) ``` Now, because of ` ....lift(dataTypeOf(name)).map(_(name, value))`, this pushes down filters for `_1` which Parquet thinks is `IntegerType`. However, it is actually `StructType`. So, Parquet filter2 produces incorrect results, for example, the codes below: ``` df.filter("_1 IS NOT NULL").count() ``` produces always 0. This PR prevents this by not finding nested fields. ## How was this patch tested? Unit test in `ParquetFilterSuite`. Author: hyukjinkwon Closes #14067 from HyukjinKwon/SPARK-16371. (cherry picked from commit 4f8ceed59367319300e4bfa5b957c387be81ffa3) Signed-off-by: Reynold Xin --- .../datasources/parquet/ParquetFileFormat.scala | 2 +- .../datasources/parquet/ParquetFilters.scala | 5 ++++- .../datasources/parquet/ParquetFilterSuite.scala | 14 ++++++++++++++ 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index f38bf81e52c0f..8cbdaebac1797 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -436,7 +436,7 @@ private[sql] class ParquetOutputWriterFactory( ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) // We want to clear this temporary metadata from saving into Parquet file. - // This metadata is only useful for detecting optional columns when pushdowning filters. + // This metadata is only useful for detecting optional columns when pushing down filters. val dataSchemaToWrite = StructType.removeMetadata( StructType.metadataKeyForOptionalField, dataSchema).asInstanceOf[StructType] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 95afdc789f322..70ae829219d59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -215,10 +215,13 @@ private[sql] object ParquetFilters { */ private def getFieldMap(dataType: DataType): Array[(String, DataType)] = dataType match { case StructType(fields) => + // Here we don't flatten the fields in the nested schema but just look up through + // root fields. Currently, accessing to nested fields does not push down filters + // and it does not support to create filters for them. fields.filter { f => !f.metadata.contains(StructType.metadataKeyForOptionalField) || !f.metadata.getBoolean(StructType.metadataKeyForOptionalField) - }.map(f => f.name -> f.dataType) ++ fields.flatMap { f => getFieldMap(f.dataType) } + }.map(f => f.name -> f.dataType) case _ => Array.empty[(String, DataType)] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 45fd6a5d80dea..35d6915af13e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -545,4 +545,18 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } } } + + test("Do not push down filters incorrectly when inner name and outer name are the same") { + withParquetDataFrame((1 to 4).map(i => Tuple1(Tuple1(i)))) { implicit df => + // Here the schema becomes as below: + // + // root + // |-- _1: struct (nullable = true) + // | |-- _1: integer (nullable = true) + // + // The inner column name, `_1` and outer column name `_1` are the same. + // Obviously this should not push down filters because the outer column is struct. + assert(df.filter("_1 IS NOT NULL").count() === 4) + } + } } From d7926da5e72ee2015e3ebe39a5fd0b322e9d1334 Mon Sep 17 00:00:00 2001 From: tmnd1991 Date: Wed, 6 Jul 2016 12:56:26 -0700 Subject: [PATCH 0903/1470] [SPARK-15740][MLLIB] Word2VecSuite "big model load / save" caused OOM in maven jenkins builds ## What changes were proposed in this pull request? "test big model load / save" in Word2VecSuite, lately resulted into OOM. Therefore we decided to make the partitioning adaptive (not based on spark default "spark.kryoserializer.buffer.max" conf) and then testing it using a small buffer size in order to trigger partitioning without allocating too much memory for the test. ## How was this patch tested? It was tested running the following unit test: org.apache.spark.mllib.feature.Word2VecSuite Author: tmnd1991 Closes #13509 from tmnd1991/SPARK-15740. (cherry picked from commit 040f6f9f468f153e4c4db78c26ced0299245fb6f) Signed-off-by: Joseph K. Bradley --- .../apache/spark/mllib/feature/Word2Vec.scala | 16 ++++++------ .../spark/mllib/feature/Word2VecSuite.scala | 25 ++++++++++++++++--- 2 files changed, 31 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 2f52825c6cb01..f2211df3f943d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -629,14 +629,16 @@ object Word2VecModel extends Loader[Word2VecModel] { ("vectorSize" -> vectorSize) ~ ("numWords" -> numWords))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) - // We want to partition the model in partitions of size 32MB - val partitionSize = (1L << 25) + // We want to partition the model in partitions smaller than + // spark.kryoserializer.buffer.max + val bufferSize = Utils.byteStringAsBytes( + spark.conf.get("spark.kryoserializer.buffer.max", "64m")) // We calculate the approximate size of the model - // We only calculate the array size, not considering - // the string size, the formula is: - // floatSize * numWords * vectorSize - val approxSize = 4L * numWords * vectorSize - val nPartitions = ((approxSize / partitionSize) + 1).toInt + // We only calculate the array size, considering an + // average string size of 15 bytes, the formula is: + // (floatSize * vectorSize + 15) * numWords + val approxSize = (4L * vectorSize + 15) * numWords + val nPartitions = ((approxSize / bufferSize) + 1).toInt val dataArray = model.toSeq.map { case (w, v) => Data(w, v) } spark.createDataFrame(dataArray).repartition(nPartitions).write.parquet(Loader.dataPath(path)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index c9fb9768c1b45..22de4c4ac40e6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -91,11 +91,23 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { } - ignore("big model load / save") { - // create a model bigger than 32MB since 9000 * 1000 * 4 > 2^25 - val word2VecMap = Map((0 to 9000).map(i => s"$i" -> Array.fill(1000)(0.1f)): _*) + test("big model load / save") { + // backupping old values + val oldBufferConfValue = spark.conf.get("spark.kryoserializer.buffer.max", "64m") + val oldBufferMaxConfValue = spark.conf.get("spark.kryoserializer.buffer", "64k") + + // setting test values to trigger partitioning + spark.conf.set("spark.kryoserializer.buffer", "50b") + spark.conf.set("spark.kryoserializer.buffer.max", "50b") + + // create a model bigger than 50 Bytes + val word2VecMap = Map((0 to 10).map(i => s"$i" -> Array.fill(10)(0.1f)): _*) val model = new Word2VecModel(word2VecMap) + // est. size of this model, given the formula: + // (floatSize * vectorSize + 15) * numWords + // (4 * 10 + 15) * 10 = 550 + // therefore it should generate multiple partitions val tempDir = Utils.createTempDir() val path = tempDir.toURI.toString @@ -103,9 +115,16 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { model.save(sc, path) val sameModel = Word2VecModel.load(sc, path) assert(sameModel.getVectors.mapValues(_.toSeq) === model.getVectors.mapValues(_.toSeq)) + } + catch { + case t: Throwable => fail("exception thrown persisting a model " + + "that spans over multiple partitions", t) } finally { Utils.deleteRecursively(tempDir) + spark.conf.set("spark.kryoserializer.buffer", oldBufferConfValue) + spark.conf.set("spark.kryoserializer.buffer.max", oldBufferMaxConfValue) } + } test("test similarity for word vectors with large values is not Infinity or NaN") { From 88be66b933a7b1f0f71b1eb6c88bf01ecbf1923c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 6 Jul 2016 13:36:07 -0700 Subject: [PATCH 0904/1470] [SPARK-16379][CORE][MESOS] Spark on mesos is broken due to race condition in Logging ## What changes were proposed in this pull request? The commit https://github.com/apache/spark/commit/044971eca0ff3c2ce62afa665dbd3072d52cbbec introduced a lazy val to simplify code in Logging. Simple enough, though one side effect is that accessing log now means grabbing the instance's lock. This in turn turned up a form of deadlock in the Mesos code. It was arguably a bit of a problem in how this code is structured, but, in any event the safest thing to do seems to be to revert the commit, and that's 90% of the change here; it's just not worth the risk of similar more subtle issues. What I didn't revert here was the removal of this odd override of log in the Mesos code. In retrospect it might have been put in place at some stage as a defense against this type of problem. After all the Logging code still involved a lock at initialization before the change in question. Even after the revert, it doesn't seem like it does anything, given how Logging works now, so I left it removed. However, I also removed the particular log message that ended up playing a part in this problem anyway, maybe being paranoid, to make sure this type of problem can't happen even with how the current locking works in logging initialization. ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #14069 from srowen/SPARK-16379. (cherry picked from commit a8f89df3b391e7a3fa9f73d9ec730d6eaa95bb09) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/internal/Logging.scala | 14 ++++++++++---- .../mesos/MesosCoarseGrainedSchedulerBackend.scala | 1 - 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index c51050c13d3a1..66a0cfec6296d 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -32,10 +32,7 @@ private[spark] trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine - @transient lazy val log: Logger = { - initializeLogIfNecessary(false) - LoggerFactory.getLogger(logName) - } + @transient private var log_ : Logger = null // Method to get the logger name for this object protected def logName = { @@ -43,6 +40,15 @@ private[spark] trait Logging { this.getClass.getName.stripSuffix("$") } + // Method to get or create the logger for this object + protected def log: Logger = { + if (log_ == null) { + initializeLogIfNecessary(false) + log_ = LoggerFactory.getLogger(logName) + } + log_ + } + // Log methods that take only a String protected def logInfo(msg: => String) { if (log.isInfoEnabled) log.info(msg) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index e88e4ad4750d7..99e6d39583747 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -244,7 +244,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( d: org.apache.mesos.SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { appId = frameworkId.getValue mesosExternalShuffleClient.foreach(_.init(appId)) - logInfo("Registered as framework ID " + appId) markRegistered() } From 2c2b8f121a213618ef47cb030d17b9bd323f0d9e Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Wed, 6 Jul 2016 15:02:45 -0700 Subject: [PATCH 0905/1470] [MESOS] expand coarse-grained mode docs ## What changes were proposed in this pull request? docs ## How was this patch tested? viewed the docs in github Author: Michael Gummelt Closes #14059 from mgummelt/coarse-grained. (cherry picked from commit 9c041990cf4d0138d9104207b5c2e7a319b42615) Signed-off-by: Reynold Xin --- docs/running-on-mesos.md | 77 ++++++++++++++++++++++++++-------------- 1 file changed, 51 insertions(+), 26 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 4a0ab623c1082..8ab5f30220afc 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -180,30 +180,53 @@ Note that jars or python files that are passed to spark-submit should be URIs re # Mesos Run Modes -Spark can run over Mesos in two modes: "coarse-grained" (default) and "fine-grained". - -The "coarse-grained" mode will launch only *one* long-running Spark task on each Mesos -machine, and dynamically schedule its own "mini-tasks" within it. The benefit is much lower startup -overhead, but at the cost of reserving the Mesos resources for the complete duration of the -application. - -Coarse-grained is the default mode. You can also set `spark.mesos.coarse` property to true -to turn it on explicitly in [SparkConf](configuration.html#spark-properties): - -{% highlight scala %} -conf.set("spark.mesos.coarse", "true") -{% endhighlight %} - -In addition, for coarse-grained mode, you can control the maximum number of resources Spark will -acquire. By default, it will acquire *all* cores in the cluster (that get offered by Mesos), which -only makes sense if you run just one application at a time. You can cap the maximum number of cores -using `conf.set("spark.cores.max", "10")` (for example). - -In "fine-grained" mode, each Spark task runs as a separate Mesos task. This allows -multiple instances of Spark (and other frameworks) to share machines at a very fine granularity, -where each application gets more or fewer machines as it ramps up and down, but it comes with an -additional overhead in launching each task. This mode may be inappropriate for low-latency -requirements like interactive queries or serving web requests. +Spark can run over Mesos in two modes: "coarse-grained" (default) and +"fine-grained". + +## Coarse-Grained + +In "coarse-grained" mode, each Spark executor runs as a single Mesos +task. Spark executors are sized according to the following +configuration variables: + +* Executor memory: `spark.executor.memory` +* Executor cores: `spark.executor.cores` +* Number of executors: `spark.cores.max`/`spark.executor.cores` + +Please see the [Spark Configuration](configuration.html) page for +details and default values. + +Executors are brought up eagerly when the application starts, until +`spark.cores.max` is reached. If you don't set `spark.cores.max`, the +Spark application will reserve all resources offered to it by Mesos, +so we of course urge you to set this variable in any sort of +multi-tenant cluster, including one which runs multiple concurrent +Spark applications. + +The scheduler will start executors round-robin on the offers Mesos +gives it, but there are no spread guarantees, as Mesos does not +provide such guarantees on the offer stream. + +The benefit of coarse-grained mode is much lower startup overhead, but +at the cost of reserving Mesos resources for the complete duration of +the application. To configure your job to dynamically adjust to its +resource requirements, look into +[Dynamic Allocation](#dynamic-resource-allocation-with-mesos). + +## Fine-Grained + +In "fine-grained" mode, each Spark task inside the Spark executor runs +as a separate Mesos task. This allows multiple instances of Spark (and +other frameworks) to share cores at a very fine granularity, where +each application gets more or fewer cores as it ramps up and down, but +it comes with an additional overhead in launching each task. This mode +may be inappropriate for low-latency requirements like interactive +queries or serving web requests. + +Note that while Spark tasks in fine-grained will relinquish cores as +they terminate, they will not relinquish memory, as the JVM does not +give memory back to the Operating System. Neither will executors +terminate when they're idle. To run in fine-grained mode, set the `spark.mesos.coarse` property to false in your [SparkConf](configuration.html#spark-properties): @@ -212,7 +235,9 @@ To run in fine-grained mode, set the `spark.mesos.coarse` property to false in y conf.set("spark.mesos.coarse", "false") {% endhighlight %} -You may also make use of `spark.mesos.constraints` to set attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. +You may also make use of `spark.mesos.constraints` to set +attribute-based constraints on Mesos resource offers. By default, all +resource offers will be accepted. {% highlight scala %} conf.set("spark.mesos.constraints", "os:centos7;us-east-1:false") @@ -246,7 +271,7 @@ In either case, HDFS runs separately from Hadoop MapReduce, without being schedu # Dynamic Resource Allocation with Mesos -Mesos supports dynamic allocation only with coarse-grain mode, which can resize the number of +Mesos supports dynamic allocation only with coarse-grained mode, which can resize the number of executors based on statistics of the application. For general information, see [Dynamic Resource Allocation](job-scheduling.html#dynamic-resource-allocation). From 05ddc75179acc582c615da01b9c0e7e049a5ecf0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 6 Jul 2016 15:04:37 -0700 Subject: [PATCH 0906/1470] [SPARK-16371][SQL] Two follow-up tasks ## What changes were proposed in this pull request? This is a small follow-up for SPARK-16371: 1. Hide removeMetadata from public API. 2. Add JIRA ticket number to test case name. ## How was this patch tested? Updated a test comment. Author: Reynold Xin Closes #14074 from rxin/parquet-filter. (cherry picked from commit 8e3e4ed6c090d18675d49eec46b3ee572457db95) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/sql/types/StructType.scala | 4 ++-- .../execution/datasources/parquet/ParquetFilterSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 436512ff69335..effef542f7ffb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -378,10 +378,10 @@ object StructType extends AbstractDataType { StructType(fields.asScala) } - protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = + private[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - def removeMetadata(key: String, dt: DataType): DataType = + private[sql] def removeMetadata(key: String, dt: DataType): DataType = dt match { case StructType(fields) => val newFields = fields.map { f => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 35d6915af13e0..2a89773cf5341 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -546,7 +546,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } } - test("Do not push down filters incorrectly when inner name and outer name are the same") { + test("SPARK-16371 Do not push down filters when inner name and outer name are the same") { withParquetDataFrame((1 to 4).map(i => Tuple1(Tuple1(i)))) { implicit df => // Here the schema becomes as below: // From 920162a1e0b43b558ba2242868a44cad06bef946 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 6 Jul 2016 16:21:41 -0700 Subject: [PATCH 0907/1470] [SPARK-16212][STREAMING][KAFKA] apply test tweaks from 0-10 to 0-8 as well ## What changes were proposed in this pull request? Bring the kafka-0-8 subproject up to date with some test modifications from development on 0-10. Main changes are - eliminating waits on concurrent queue in favor of an assert on received results, - atomics instead of volatile (although this probably doesn't matter) - increasing uniqueness of topic names ## How was this patch tested? Unit tests Author: cody koeninger Closes #14073 from koeninger/kafka-0-8-test-direct-cleanup. (cherry picked from commit b8ebf63c1e1fa1ab53ea760fa293051c08ce5f59) Signed-off-by: Tathagata Das --- .../kafka/DirectKafkaStreamSuite.scala | 41 +++++++++---------- .../spark/streaming/kafka/KafkaRDDSuite.scala | 8 ++-- 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index cb782d27fe22c..ab1c5055a253f 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -244,12 +244,9 @@ class DirectKafkaStreamSuite ) // Send data to Kafka and wait for it to be received - def sendDataAndWaitForReceive(data: Seq[Int]) { + def sendData(data: Seq[Int]) { val strings = data.map { _.toString} kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) - } } // Setup the streaming context @@ -264,21 +261,21 @@ class DirectKafkaStreamSuite } ssc.checkpoint(testDir.getAbsolutePath) - // This is to collect the raw data received from Kafka - kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => - val data = rdd.map { _._2 }.collect() - DirectKafkaStreamSuite.collectedData.addAll(Arrays.asList(data: _*)) - } - // This is ensure all the data is eventually receiving only once stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => - rdd.collect().headOption.foreach { x => DirectKafkaStreamSuite.total = x._2 } + rdd.collect().headOption.foreach { x => + DirectKafkaStreamSuite.total.set(x._2) + } } ssc.start() - // Send some data and wait for them to be received + // Send some data for (i <- (1 to 10).grouped(4)) { - sendDataAndWaitForReceive(i) + sendData(i) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) } ssc.stop() @@ -302,23 +299,26 @@ class DirectKafkaStreamSuite val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] // Verify offset ranges have been recovered - val recoveredOffsetRanges = getOffsetRanges(recoveredStream) + val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => (x._1, x._2.toSet) } assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") - val earlierOffsetRangesAsSets = offsetRangesAfterStop.map { x => (x._1, x._2.toSet) } + val earlierOffsetRanges = offsetRangesAfterStop.map { x => (x._1, x._2.toSet) } assert( recoveredOffsetRanges.forall { or => - earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) + earlierOffsetRanges.contains((or._1, or._2)) }, "Recovered ranges are not the same as the ones generated\n" + s"recoveredOffsetRanges: $recoveredOffsetRanges\n" + - s"earlierOffsetRangesAsSets: $earlierOffsetRangesAsSets" + s"earlierOffsetRanges: $earlierOffsetRanges" ) // Restart context, give more data and verify the total at the end // If the total is write that means each records has been received only once ssc.start() - sendDataAndWaitForReceive(11 to 20) + for (i <- (11 to 20).grouped(4)) { + sendData(i) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { - assert(DirectKafkaStreamSuite.total === (1 to 20).sum) + assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) } ssc.stop() } @@ -488,8 +488,7 @@ class DirectKafkaStreamSuite } object DirectKafkaStreamSuite { - val collectedData = new ConcurrentLinkedQueue[String]() - @volatile var total = -1L + val total = new AtomicLong(-1L) class InputInfoCollector extends StreamingListener { val numRecordsSubmitted = new AtomicLong(0L) diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 5e539c1d790cc..809699a739962 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -53,13 +53,13 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { } test("basic usage") { - val topic = s"topicbasic-${Random.nextInt}" + val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" kafkaTestUtils.createTopic(topic) val messages = Array("the", "quick", "brown", "fox") kafkaTestUtils.sendMessages(topic, messages) val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}") + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) @@ -92,12 +92,12 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { test("iterator boundary conditions") { // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd - val topic = s"topicboundary-${Random.nextInt}" + val topic = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) kafkaTestUtils.createTopic(topic) val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}") + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") val kc = new KafkaCluster(kafkaParams) From d63428af6d1c7c0a0533567a0a7ccb5817a65de3 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 7 Jul 2016 00:07:25 -0700 Subject: [PATCH 0908/1470] [SPARK-16368][SQL] Fix Strange Errors When Creating View With Unmatched Column Num #### What changes were proposed in this pull request? When creating a view, a common user error is the number of columns produced by the `SELECT` clause does not match the number of column names specified by `CREATE VIEW`. For example, given Table `t1` only has 3 columns ```SQL create view v1(col2, col4, col3, col5) as select * from t1 ``` Currently, Spark SQL reports the following error: ``` requirement failed java.lang.IllegalArgumentException: requirement failed at scala.Predef$.require(Predef.scala:212) at org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:90) ``` This error message is very confusing. This PR is to detect the error and issue a meaningful error message. #### How was this patch tested? Added test cases Author: gatorsmile Closes #14047 from gatorsmile/viewMismatchedColumns. (cherry picked from commit ab05db0b48f395543cd7d91e2ad9dd760516868b) Signed-off-by: Reynold Xin --- .../spark/sql/execution/command/views.scala | 6 ++++- .../sql/execution/command/DDLSuite.scala | 23 +++++++++++++++++++ .../sql/hive/execution/HiveDDLSuite.scala | 23 +++++++++++++++++++ 3 files changed, 51 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 088f684365db3..6533d796e8063 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -88,7 +88,11 @@ case class CreateViewCommand( qe.assertAnalyzed() val analyzedPlan = qe.analyzed - require(tableDesc.schema == Nil || tableDesc.schema.length == analyzedPlan.output.length) + if (tableDesc.schema != Nil && tableDesc.schema.length != analyzedPlan.output.length) { + throw new AnalysisException(s"The number of columns produced by the SELECT clause " + + s"(num: `${analyzedPlan.output.length}`) does not match the number of column names " + + s"specified by CREATE VIEW (num: `${tableDesc.schema.length}`).") + } val sessionState = sparkSession.sessionState if (isTemporary) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 0ee8d179d79eb..7d1f1d1e62fc7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1314,6 +1314,29 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("create temporary view with mismatched schema") { + withTable("tab1") { + spark.range(10).write.saveAsTable("tab1") + withView("view1") { + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW view1 (col1, col3) AS SELECT * FROM tab1") + }.getMessage + assert(e.contains("the SELECT clause (num: `1`) does not match") + && e.contains("CREATE VIEW (num: `2`)")) + } + } + } + + test("create temporary view with specified schema") { + withView("view1") { + sql("CREATE TEMPORARY VIEW view1 (col1, col2) AS SELECT 1, 2") + checkAnswer( + sql("SELECT * FROM view1"), + Row(1, 2) :: Nil + ) + } + } + test("truncate table - external table, temporary table, view (not allowed)") { import testImplicits._ val path = Utils.createTempDir().getAbsolutePath diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 9d3c4cd3d570c..93e50f4ee907b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -391,6 +391,29 @@ class HiveDDLSuite } } + test("create view with mismatched schema") { + withTable("tab1") { + spark.range(10).write.saveAsTable("tab1") + withView("view1") { + val e = intercept[AnalysisException] { + sql("CREATE VIEW view1 (col1, col3) AS SELECT * FROM tab1") + }.getMessage + assert(e.contains("the SELECT clause (num: `1`) does not match") + && e.contains("CREATE VIEW (num: `2`)")) + } + } + } + + test("create view with specified schema") { + withView("view1") { + sql("CREATE VIEW view1 (col1, col2) AS SELECT 1, 2") + checkAnswer( + sql("SELECT * FROM view1"), + Row(1, 2) :: Nil + ) + } + } + test("desc table for Hive table") { withTable("tab1") { val tabName = "tab1" From 24933355c7211bbf6bc5bebfad91ed783d6b6a51 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Thu, 7 Jul 2016 11:28:04 +0100 Subject: [PATCH 0909/1470] [SPARK-16372][MLLIB] Retag RDD to tallSkinnyQR of RowMatrix ## What changes were proposed in this pull request? The following Java code because of type erasing: ```Java JavaRDD rows = jsc.parallelize(...); RowMatrix mat = new RowMatrix(rows.rdd()); QRDecomposition result = mat.tallSkinnyQR(true); ``` We should use retag to restore the type to prevent the following exception: ```Java java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to [Lorg.apache.spark.mllib.linalg.Vector; ``` ## How was this patch tested? Java unit test Author: Xusen Yin Closes #14051 from yinxusen/SPARK-16372. (cherry picked from commit 4c6f00d09c016dfc1d2de6e694dff219c9027fa0) Signed-off-by: Sean Owen --- .../mllib/api/python/PythonMLLibAPI.scala | 2 +- .../mllib/linalg/distributed/RowMatrix.scala | 2 +- .../distributed/JavaRowMatrixSuite.java | 44 +++++++++++++++++++ 3 files changed, 46 insertions(+), 2 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/linalg/distributed/JavaRowMatrixSuite.java 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 f4819f77ebdb8..a80cca70f4b28 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 @@ -1127,7 +1127,7 @@ private[python] class PythonMLLibAPI extends Serializable { * Wrapper around RowMatrix constructor. */ def createRowMatrix(rows: JavaRDD[Vector], numRows: Long, numCols: Int): RowMatrix = { - new RowMatrix(rows.rdd.retag(classOf[Vector]), numRows, numCols) + new RowMatrix(rows.rdd, numRows, numCols) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index cd5209d0ebe20..1c94479ef02af 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -537,7 +537,7 @@ class RowMatrix @Since("1.0.0") ( def tallSkinnyQR(computeQ: Boolean = false): QRDecomposition[RowMatrix, Matrix] = { val col = numCols().toInt // split rows horizontally into smaller matrices, and compute QR for each of them - val blockQRs = rows.glom().map { partRows => + val blockQRs = rows.retag(classOf[Vector]).glom().map { partRows => val bdm = BDM.zeros[Double](partRows.length, col) var i = 0 partRows.foreach { row => diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/distributed/JavaRowMatrixSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/distributed/JavaRowMatrixSuite.java new file mode 100644 index 0000000000000..c01af405491b7 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/distributed/JavaRowMatrixSuite.java @@ -0,0 +1,44 @@ +/* + * 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.linalg.distributed; + +import java.util.Arrays; + +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.QRDecomposition; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +public class JavaRowMatrixSuite extends SharedSparkSession { + + @Test + public void rowMatrixQRDecomposition() { + Vector v1 = Vectors.dense(1.0, 10.0, 100.0); + Vector v2 = Vectors.dense(2.0, 20.0, 200.0); + Vector v3 = Vectors.dense(3.0, 30.0, 300.0); + + JavaRDD rows = jsc.parallelize(Arrays.asList(v1, v2, v3), 1); + RowMatrix mat = new RowMatrix(rows.rdd()); + + QRDecomposition result = mat.tallSkinnyQR(true); + } +} From cbfd94eacf46b61011f1bd8d30f0c134cab37b09 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Thu, 7 Jul 2016 10:40:42 -0700 Subject: [PATCH 0910/1470] [SPARK-16350][SQL] Fix support for incremental planning in wirteStream.foreach() ## What changes were proposed in this pull request? There are cases where `complete` output mode does not output updated aggregated value; for details please refer to [SPARK-16350](https://issues.apache.org/jira/browse/SPARK-16350). The cause is that, as we do `data.as[T].foreachPartition { iter => ... }` in `ForeachSink.addBatch()`, `foreachPartition()` does not support incremental planning for now. This patches makes `foreachPartition()` support incremental planning in `ForeachSink`, by making a special version of `Dataset` with its `rdd()` method supporting incremental planning. ## How was this patch tested? Added a unit test which failed before the change Author: Liwei Lin Closes #14030 from lw-lin/fix-foreach-complete. (cherry picked from commit 0f7175def985a7f1e37198680f893e749612ab76) Signed-off-by: Shixiong Zhu --- .../sql/execution/streaming/ForeachSink.scala | 40 ++++++++- .../streaming/IncrementalExecution.scala | 4 +- .../streaming/ForeachSinkSuite.scala | 86 +++++++++++++++++-- 3 files changed, 117 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala index 14b9b1cb09317..082664aa23f04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.TaskContext -import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Encoder, ForeachWriter} +import org.apache.spark.sql.catalyst.plans.logical.CatalystSerde /** * A [[Sink]] that forwards all data into [[ForeachWriter]] according to the contract defined by @@ -30,7 +32,41 @@ import org.apache.spark.sql.{DataFrame, Encoder, ForeachWriter} class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Serializable { override def addBatch(batchId: Long, data: DataFrame): Unit = { - data.as[T].foreachPartition { iter => + // TODO: Refine this method when SPARK-16264 is resolved; see comments below. + + // This logic should've been as simple as: + // ``` + // data.as[T].foreachPartition { iter => ... } + // ``` + // + // Unfortunately, doing that would just break the incremental planing. The reason is, + // `Dataset.foreachPartition()` would further call `Dataset.rdd()`, but `Dataset.rdd()` just + // does not support `IncrementalExecution`. + // + // So as a provisional fix, below we've made a special version of `Dataset` with its `rdd()` + // method supporting incremental planning. But in the long run, we should generally make newly + // created Datasets use `IncrementalExecution` where necessary (which is SPARK-16264 tries to + // resolve). + + val datasetWithIncrementalExecution = + new Dataset(data.sparkSession, data.logicalPlan, implicitly[Encoder[T]]) { + override lazy val rdd: RDD[T] = { + val objectType = exprEnc.deserializer.dataType + val deserialized = CatalystSerde.deserialize[T](logicalPlan) + + // was originally: sparkSession.sessionState.executePlan(deserialized) ... + val incrementalExecution = new IncrementalExecution( + this.sparkSession, + deserialized, + data.queryExecution.asInstanceOf[IncrementalExecution].outputMode, + data.queryExecution.asInstanceOf[IncrementalExecution].checkpointLocation, + data.queryExecution.asInstanceOf[IncrementalExecution].currentBatchId) + incrementalExecution.toRdd.mapPartitions { rows => + rows.map(_.get(0, objectType)) + }.asInstanceOf[RDD[T]] + } + } + datasetWithIncrementalExecution.foreachPartition { iter => if (writer.open(TaskContext.getPartitionId(), batchId)) { var isFailed = false try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 0ce00552bf6cb..7367c68d0a0e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -30,8 +30,8 @@ import org.apache.spark.sql.streaming.OutputMode class IncrementalExecution private[sql]( sparkSession: SparkSession, logicalPlan: LogicalPlan, - outputMode: OutputMode, - checkpointLocation: String, + val outputMode: OutputMode, + val checkpointLocation: String, val currentBatchId: Long) extends QueryExecution(sparkSession, logicalPlan) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala index 6ff597c16bb28..7928b8e8775c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.scalatest.BeforeAndAfter import org.apache.spark.sql.ForeachWriter -import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.streaming.{OutputMode, StreamTest} import org.apache.spark.sql.test.SharedSQLContext class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { @@ -35,35 +35,103 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf sqlContext.streams.active.foreach(_.stop()) } - test("foreach") { + test("foreach() with `append` output mode") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(2).writeStream .option("checkpointLocation", checkpointDir.getCanonicalPath) + .outputMode(OutputMode.Append) .foreach(new TestForeachWriter()) .start() + + // -- batch 0 --------------------------------------- input.addData(1, 2, 3, 4) query.processAllAvailable() - val expectedEventsForPartition0 = Seq( + var expectedEventsForPartition0 = Seq( ForeachSinkSuite.Open(partition = 0, version = 0), ForeachSinkSuite.Process(value = 1), ForeachSinkSuite.Process(value = 3), ForeachSinkSuite.Close(None) ) - val expectedEventsForPartition1 = Seq( + var expectedEventsForPartition1 = Seq( ForeachSinkSuite.Open(partition = 1, version = 0), ForeachSinkSuite.Process(value = 2), ForeachSinkSuite.Process(value = 4), ForeachSinkSuite.Close(None) ) - val allEvents = ForeachSinkSuite.allEvents() + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert(allEvents.toSet === Set(expectedEventsForPartition0, expectedEventsForPartition1)) + + ForeachSinkSuite.clear() + + // -- batch 1 --------------------------------------- + input.addData(5, 6, 7, 8) + query.processAllAvailable() + + expectedEventsForPartition0 = Seq( + ForeachSinkSuite.Open(partition = 0, version = 1), + ForeachSinkSuite.Process(value = 5), + ForeachSinkSuite.Process(value = 7), + ForeachSinkSuite.Close(None) + ) + expectedEventsForPartition1 = Seq( + ForeachSinkSuite.Open(partition = 1, version = 1), + ForeachSinkSuite.Process(value = 6), + ForeachSinkSuite.Process(value = 8), + ForeachSinkSuite.Close(None) + ) + + allEvents = ForeachSinkSuite.allEvents() assert(allEvents.size === 2) - assert { - allEvents === Seq(expectedEventsForPartition0, expectedEventsForPartition1) || - allEvents === Seq(expectedEventsForPartition1, expectedEventsForPartition0) - } + assert(allEvents.toSet === Set(expectedEventsForPartition0, expectedEventsForPartition1)) + + query.stop() + } + } + + test("foreach() with `complete` output mode") { + withTempDir { checkpointDir => + val input = MemoryStream[Int] + + val query = input.toDS() + .groupBy().count().as[Long].map(_.toInt) + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .outputMode(OutputMode.Complete) + .foreach(new TestForeachWriter()) + .start() + + // -- batch 0 --------------------------------------- + input.addData(1, 2, 3, 4) + query.processAllAvailable() + + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 1) + var expectedEvents = Seq( + ForeachSinkSuite.Open(partition = 0, version = 0), + ForeachSinkSuite.Process(value = 4), + ForeachSinkSuite.Close(None) + ) + assert(allEvents === Seq(expectedEvents)) + + ForeachSinkSuite.clear() + + // -- batch 1 --------------------------------------- + input.addData(5, 6, 7, 8) + query.processAllAvailable() + + allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 1) + expectedEvents = Seq( + ForeachSinkSuite.Open(partition = 0, version = 1), + ForeachSinkSuite.Process(value = 8), + ForeachSinkSuite.Close(None) + ) + assert(allEvents === Seq(expectedEvents)) + query.stop() } } From 30cb3f1d3a1d413568d586e6b8df56f74f05d80e Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Thu, 7 Jul 2016 11:08:06 -0700 Subject: [PATCH 0911/1470] [SPARK-16415][SQL] fix catalog string error ## What changes were proposed in this pull request? In #13537 we truncate `simpleString` if it is a long `StructType`. But sometimes we need `catalogString` to reconstruct `TypeInfo`, for example in description of [SPARK-16415 ](https://issues.apache.org/jira/browse/SPARK-16415). So we need to keep the implementation of `catalogString` not affected by our truncate. ## How was this patch tested? added a test case. Author: Daoyuan Wang Closes #14089 from adrian-wang/catalogstring. (cherry picked from commit 28710b42b0d18a55bd64d597558649537259b127) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/types/StructType.scala | 6 ++++++ .../spark/sql/hive/HiveMetastoreCatalogSuite.scala | 14 +++++++++++--- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index effef542f7ffb..55fdfbe3e0464 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -298,6 +298,12 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru Utils.truncatedString(fieldTypes, "struct<", ",", ">") } + override def catalogString: String = { + // in catalogString, we should not truncate + val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.catalogString}") + s"struct<${fieldTypes.mkString(",")}>" + } + override def sql: String = { val fieldTypes = fields.map(f => s"${quoteIdentifier(f.name)}: ${f.dataType.sql}") s"STRUCT<${fieldTypes.mkString(", ")}>" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index b420781e51bd3..754aabb5ac936 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -26,15 +26,15 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} -import org.apache.spark.sql.types.{DecimalType, StringType, StructType} +import org.apache.spark.sql.types.{DecimalType, StringType, StructField, StructType} class HiveMetastoreCatalogSuite extends TestHiveSingleton { import spark.implicits._ test("struct field should accept underscore in sub-column name") { val hiveTypeStr = "struct" - val dateType = CatalystSqlParser.parseDataType(hiveTypeStr) - assert(dateType.isInstanceOf[StructType]) + val dataType = CatalystSqlParser.parseDataType(hiveTypeStr) + assert(dataType.isInstanceOf[StructType]) } test("udt to metastore type conversion") { @@ -49,6 +49,14 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton { logInfo(df.queryExecution.toString) df.as('a).join(df.as('b), $"a.key" === $"b.key") } + + test("should not truncate struct type catalog string") { + def field(n: Int): StructField = { + StructField("col" + n, StringType) + } + val dataType = StructType((1 to 100).map(field)) + assert(CatalystSqlParser.parseDataType(dataType.catalogString) == dataType) + } } class DataSourceWithHiveMetastoreCatalogSuite From 5828da41cb2d815708191bd9a5cf3bd82795aa41 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 7 Jul 2016 15:21:57 -0700 Subject: [PATCH 0912/1470] [SPARK-16310][SPARKR] R na.string-like default for csv source ## What changes were proposed in this pull request? Apply default "NA" as null string for R, like R read.csv na.string parameter. https://stat.ethz.ch/R-manual/R-devel/library/utils/html/read.table.html na.strings = "NA" An user passing a csv file with NA value should get the same behavior with SparkR read.df(... source = "csv") (couldn't open JIRA, will do that later) ## How was this patch tested? unit tests shivaram Author: Felix Cheung Closes #13984 from felixcheung/rcsvnastring. (cherry picked from commit f4767bcc7a9d1bdd301f054776aa45e7c9f344a7) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/SQLContext.R | 10 +++++-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 32 ++++++++++++++++++----- 2 files changed, 34 insertions(+), 8 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 8df73db36e956..bc0daa25c9f6a 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -714,11 +714,14 @@ dropTempView <- function(viewName) { #' #' 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. +#' "spark.sql.sources.default" will be used. \cr +#' Similar to R read.csv, when `source` is "csv", by default, a value of "NA" will be interpreted +#' as NA. #' #' @param path The path of files to load #' @param source The name of external data source #' @param schema The data schema defined in structType +#' @param na.strings Default string value for NA when source is "csv" #' @return SparkDataFrame #' @rdname read.df #' @name read.df @@ -735,7 +738,7 @@ dropTempView <- function(viewName) { #' @name read.df #' @method read.df default #' @note read.df since 1.4.0 -read.df.default <- function(path = NULL, source = NULL, schema = NULL, ...) { +read.df.default <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA", ...) { sparkSession <- getSparkSession() options <- varargsToEnv(...) if (!is.null(path)) { @@ -744,6 +747,9 @@ read.df.default <- function(path = NULL, source = NULL, schema = NULL, ...) { if (is.null(source)) { source <- getDefaultSqlSource() } + if (source == "csv" && is.null(options[["nullValue"]])) { + options[["nullValue"]] <- na.strings + } if (!is.null(schema)) { stopifnot(class(schema) == "structType") sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "loadDF", sparkSession, source, diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index d22baf6a20f0f..003fcce4c87dd 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -213,15 +213,35 @@ test_that("read csv as DataFrame", { mockLinesCsv <- c("year,make,model,comment,blank", "\"2012\",\"Tesla\",\"S\",\"No comment\",", "1997,Ford,E350,\"Go get one now they are going fast\",", - "2015,Chevy,Volt") + "2015,Chevy,Volt", + "NA,Dummy,Placeholder") writeLines(mockLinesCsv, csvPath) - # default "header" is false - df <- read.df(csvPath, "csv", header = "true") - expect_equal(count(df), 3) + # default "header" is false, inferSchema to handle "year" as "int" + df <- read.df(csvPath, "csv", header = "true", inferSchema = "true") + expect_equal(count(df), 4) expect_equal(columns(df), c("year", "make", "model", "comment", "blank")) - expect_equal(sort(unlist(collect(where(df, df$year == "2015")))), - sort(unlist(list(year = "2015", make = "Chevy", model = "Volt")))) + expect_equal(sort(unlist(collect(where(df, df$year == 2015)))), + sort(unlist(list(year = 2015, make = "Chevy", model = "Volt")))) + + # since "year" is "int", let's skip the NA values + withoutna <- na.omit(df, how = "any", cols = "year") + expect_equal(count(withoutna), 3) + + unlink(csvPath) + csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") + mockLinesCsv <- c("year,make,model,comment,blank", + "\"2012\",\"Tesla\",\"S\",\"No comment\",", + "1997,Ford,E350,\"Go get one now they are going fast\",", + "2015,Chevy,Volt", + "Empty,Dummy,Placeholder") + writeLines(mockLinesCsv, csvPath) + + df2 <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.string = "Empty") + expect_equal(count(df2), 4) + withoutna2 <- na.omit(df2, how = "any", cols = "year") + expect_equal(count(withoutna2), 3) + expect_equal(count(where(withoutna2, withoutna2$make == "Dummy")), 0) unlink(csvPath) }) From 73c764a047f795c85909c7a7ea4324f286d2aafa Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 7 Jul 2016 17:47:29 -0700 Subject: [PATCH 0913/1470] [SPARK-16425][R] `describe()` should not fail with non-numeric columns ## What changes were proposed in this pull request? This PR prevents ERRORs when `summary(df)` is called for `SparkDataFrame` with not-numeric columns. This failure happens only in `SparkR`. **Before** ```r > df <- createDataFrame(faithful) > df <- withColumn(df, "boolean", df$waiting==79) > summary(df) 16/07/07 14:15:16 ERROR RBackendHandler: describe on 34 failed Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: cannot resolve 'avg(`boolean`)' due to data type mismatch: function average requires numeric types, not BooleanType; ``` **After** ```r > df <- createDataFrame(faithful) > df <- withColumn(df, "boolean", df$waiting==79) > summary(df) SparkDataFrame[summary:string, eruptions:string, waiting:string] ``` ## How was this patch tested? Pass the Jenkins with a updated testcase. Author: Dongjoon Hyun Closes #14096 from dongjoon-hyun/SPARK-16425. (cherry picked from commit 6aa7d09f4e126f42e41085dec169c813379ed354) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 3 +-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 8 ++++++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 17474d4c81559..ec09aab6f9698 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2617,8 +2617,7 @@ setMethod("describe", setMethod("describe", signature(x = "SparkDataFrame"), function(x) { - colList <- as.list(c(columns(x))) - sdf <- callJMethod(x@sdf, "describe", colList) + sdf <- callJMethod(x@sdf, "describe", list()) dataFrame(sdf) }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 003fcce4c87dd..755aded2272e1 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1816,13 +1816,17 @@ test_that("describe() and summarize() on a DataFrame", { expect_equal(collect(stats)[2, "age"], "24.5") expect_equal(collect(stats)[3, "age"], "7.7781745930520225") stats <- describe(df) - expect_equal(collect(stats)[4, "name"], "Andy") + expect_equal(collect(stats)[4, "name"], NULL) expect_equal(collect(stats)[5, "age"], "30") stats2 <- summary(df) - expect_equal(collect(stats2)[4, "name"], "Andy") + expect_equal(collect(stats2)[4, "name"], NULL) expect_equal(collect(stats2)[5, "age"], "30") + # SPARK-16425: SparkR summary() fails on column of type logical + df <- withColumn(df, "boolean", df$age == 30) + summary(df) + # Test base::summary is working expect_equal(length(summary(attenu, digits = 4)), 35) }) From 88603bd4f9a665ad02df40ed8a0dd78b65c9d152 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Fri, 1 Jul 2016 07:57:48 +0800 Subject: [PATCH 0914/1470] [SPARK-16276][SQL] Implement elt SQL function This patch implements the elt function, as it is implemented in Hive. Added expression unit test in StringExpressionsSuite and end-to-end test in StringFunctionsSuite. Author: petermaxlee Closes #13966 from petermaxlee/SPARK-16276. (cherry picked from commit 85f2303ecadd9bf6d9694a2743dda075654c5ccf) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/ExpectsInputTypes.scala | 3 +- .../expressions/stringExpressions.scala | 41 +++++++++++++++++++ .../expressions/StringExpressionsSuite.scala | 23 +++++++++++ .../spark/sql/StringFunctionsSuite.scala | 14 +++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- 6 files changed, 82 insertions(+), 2 deletions(-) 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 0bde48ce57c86..95be0d610e1fa 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 @@ -265,6 +265,7 @@ object FunctionRegistry { expression[Concat]("concat"), expression[ConcatWs]("concat_ws"), expression[Decode]("decode"), + expression[Elt]("elt"), expression[Encode]("encode"), expression[FindInSet]("find_in_set"), expression[FormatNumber]("format_number"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala index c15a2df50855a..98f25a9ad7597 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala @@ -57,7 +57,8 @@ trait ExpectsInputTypes extends Expression { /** - * A mixin for the analyzer to perform implicit type casting using [[ImplicitTypeCasts]]. + * A mixin for the analyzer to perform implicit type casting using + * [[org.apache.spark.sql.catalyst.analysis.TypeCoercion.ImplicitTypeCasts]]. */ trait ImplicitCastInputTypes extends ExpectsInputTypes { // No other methods diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 44ff7fda8ef49..b0df957637f1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -21,6 +21,7 @@ import java.text.{DecimalFormat, DecimalFormatSymbols} import java.util.{HashMap, Locale, Map => JMap} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.types._ @@ -162,6 +163,46 @@ case class ConcatWs(children: Seq[Expression]) } } +@ExpressionDescription( + usage = "_FUNC_(n, str1, str2, ...) - returns the n-th string, e.g. returns str2 when n is 2", + extended = "> SELECT _FUNC_(1, 'scala', 'java') FROM src LIMIT 1;\n" + "'scala'") +case class Elt(children: Seq[Expression]) + extends Expression with ImplicitCastInputTypes with CodegenFallback { + + private lazy val indexExpr = children.head + private lazy val stringExprs = children.tail.toArray + + /** This expression is always nullable because it returns null if index is out of range. */ + override def nullable: Boolean = true + + override def dataType: DataType = StringType + + override def inputTypes: Seq[DataType] = IntegerType +: Seq.fill(children.size - 1)(StringType) + + override def checkInputDataTypes(): TypeCheckResult = { + if (children.size < 2) { + TypeCheckResult.TypeCheckFailure("elt function requires at least two arguments") + } else { + super[ImplicitCastInputTypes].checkInputDataTypes() + } + } + + override def eval(input: InternalRow): Any = { + val indexObj = indexExpr.eval(input) + if (indexObj == null) { + null + } else { + val index = indexObj.asInstanceOf[Int] + if (index <= 0 || index > stringExprs.length) { + null + } else { + stringExprs(index - 1).eval(input) + } + } + } +} + + trait String2StringExpression extends ImplicitCastInputTypes { self: UnaryExpression => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 29bf15bf524b9..5f01561986f19 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -75,6 +75,29 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // scalastyle:on } + test("elt") { + def testElt(result: String, n: java.lang.Integer, args: String*): Unit = { + checkEvaluation( + Elt(Literal.create(n, IntegerType) +: args.map(Literal.create(_, StringType))), + result) + } + + testElt("hello", 1, "hello", "world") + testElt(null, 1, null, "world") + testElt(null, null, "hello", "world") + + // Invalid ranages + testElt(null, 3, "hello", "world") + testElt(null, 0, "hello", "world") + testElt(null, -1, "hello", "world") + + // type checking + assert(Elt(Seq.empty).checkInputDataTypes().isFailure) + assert(Elt(Seq(Literal(1))).checkInputDataTypes().isFailure) + assert(Elt(Seq(Literal(1), Literal("A"))).checkInputDataTypes().isSuccess) + assert(Elt(Seq(Literal(1), Literal(2))).checkInputDataTypes().isFailure) + } + test("StringComparison") { val row = create_row("abc", null) val c1 = 'a.string.at(0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 1de2d9b5adab1..dff4226051494 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -48,6 +48,20 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("a||b")) } + test("string elt") { + val df = Seq[(String, String, String, Int)](("hello", "world", null, 15)) + .toDF("a", "b", "c", "d") + + checkAnswer( + df.selectExpr("elt(0, a, b, c)", "elt(1, a, b, c)", "elt(4, a, b, c)"), + Row(null, "hello", null)) + + // check implicit type cast + checkAnswer( + df.selectExpr("elt(4, a, b, c, d)", "elt('2', a, b, c, d)"), + Row("15", "world")) + } + test("string Levenshtein distance") { val df = Seq(("kitten", "sitting"), ("frog", "fog")).toDF("l", "r") checkAnswer(df.select(levenshtein($"l", $"r")), Seq(Row(3), Row(1))) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index ea818b5ebca75..1479554196d41 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -235,7 +235,7 @@ private[sql] class HiveSessionCatalog( // parse_url_tuple, posexplode, reflect2, // str_to_map, windowingtablefunction. private val hiveFunctions = Seq( - "elt", "hash", "java_method", "histogram_numeric", + "hash", "java_method", "histogram_numeric", "map_keys", "map_values", "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "stack", "str_to_map", "xpath", "xpath_boolean", "xpath_double", "xpath_float", "xpath_int", "xpath_long", From 7ef1d1c618100313dbbdb6f615d9f87ff67e895d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 3 Jul 2016 16:59:40 +0800 Subject: [PATCH 0915/1470] [SPARK-16278][SPARK-16279][SQL] Implement map_keys/map_values SQL functions This PR adds `map_keys` and `map_values` SQL functions in order to remove Hive fallback. Pass the Jenkins tests including new testcases. Author: Dongjoon Hyun Closes #13967 from dongjoon-hyun/SPARK-16278. (cherry picked from commit 54b27c1797fcd32b3f3e9d44e1a149ae396a61e6) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../expressions/collectionOperations.scala | 48 +++++++++++++++++++ .../CollectionFunctionsSuite.scala | 13 +++++ .../spark/sql/DataFrameFunctionsSuite.scala | 16 +++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 1 - 5 files changed, 79 insertions(+), 1 deletion(-) 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 95be0d610e1fa..27c3a09571a9f 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 @@ -170,6 +170,8 @@ object FunctionRegistry { expression[IsNotNull]("isnotnull"), expression[Least]("least"), expression[CreateMap]("map"), + expression[MapKeys]("map_keys"), + expression[MapValues]("map_values"), expression[CreateNamedStruct]("named_struct"), expression[NaNvl]("nanvl"), expression[NullIf]("nullif"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index c71cb73d65bf6..2e8ea1107cee0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -43,6 +43,54 @@ case class Size(child: Expression) extends UnaryExpression with ExpectsInputType } } +/** + * Returns an unordered array containing the keys of the map. + */ +@ExpressionDescription( + usage = "_FUNC_(map) - Returns an unordered array containing the keys of the map.", + extended = " > SELECT _FUNC_(map(1, 'a', 2, 'b'));\n [1,2]") +case class MapKeys(child: Expression) + extends UnaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(MapType) + + override def dataType: DataType = ArrayType(child.dataType.asInstanceOf[MapType].keyType) + + override def nullSafeEval(map: Any): Any = { + map.asInstanceOf[MapData].keyArray() + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + nullSafeCodeGen(ctx, ev, c => s"${ev.value} = ($c).keyArray();") + } + + override def prettyName: String = "map_keys" +} + +/** + * Returns an unordered array containing the values of the map. + */ +@ExpressionDescription( + usage = "_FUNC_(map) - Returns an unordered array containing the values of the map.", + extended = " > SELECT _FUNC_(map(1, 'a', 2, 'b'));\n [\"a\",\"b\"]") +case class MapValues(child: Expression) + extends UnaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(MapType) + + override def dataType: DataType = ArrayType(child.dataType.asInstanceOf[MapType].valueType) + + override def nullSafeEval(map: Any): Any = { + map.asInstanceOf[MapData].valueArray() + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + nullSafeCodeGen(ctx, ev, c => s"${ev.value} = ($c).valueArray();") + } + + override def prettyName: String = "map_values" +} + /** * Sorts the input array in ascending / descending order according to the natural ordering of * the array elements and returns it. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala index 1aae4678d6278..a5f784fdcc13c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala @@ -44,6 +44,19 @@ class CollectionFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.create(null, ArrayType(StringType)), null) } + test("MapKeys/MapValues") { + val m0 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, StringType)) + val m1 = Literal.create(Map[String, String](), MapType(StringType, StringType)) + val m2 = Literal.create(null, MapType(StringType, StringType)) + + checkEvaluation(MapKeys(m0), Seq("a", "b")) + checkEvaluation(MapValues(m0), Seq("1", "2")) + checkEvaluation(MapKeys(m1), Seq()) + checkEvaluation(MapValues(m1), Seq()) + checkEvaluation(MapKeys(m2), null) + checkEvaluation(MapValues(m2), null) + } + test("Sort Array") { val a0 = Literal.create(Seq(2, 1, 3), ArrayType(IntegerType)) val a1 = Literal.create(Seq[Integer](), ArrayType(IntegerType)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 73d77651a027e..0f6c49e759590 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -352,6 +352,22 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { ) } + test("map_keys/map_values function") { + val df = Seq( + (Map[Int, Int](1 -> 100, 2 -> 200), "x"), + (Map[Int, Int](), "y"), + (Map[Int, Int](1 -> 100, 2 -> 200, 3 -> 300), "z") + ).toDF("a", "b") + checkAnswer( + df.selectExpr("map_keys(a)"), + Seq(Row(Seq(1, 2)), Row(Seq.empty), Row(Seq(1, 2, 3))) + ) + checkAnswer( + df.selectExpr("map_values(a)"), + Seq(Row(Seq(100, 200)), Row(Seq.empty), Row(Seq(100, 200, 300))) + ) + } + test("array contains function") { val df = Seq( (Seq[Int](1, 2), "x"), diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 1479554196d41..4c986b0f5a200 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -236,7 +236,6 @@ private[sql] class HiveSessionCatalog( // str_to_map, windowingtablefunction. private val hiveFunctions = Seq( "hash", "java_method", "histogram_numeric", - "map_keys", "map_values", "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "stack", "str_to_map", "xpath", "xpath_boolean", "xpath_double", "xpath_float", "xpath_int", "xpath_long", "xpath_number", "xpath_short", "xpath_string", From a049754577aa78a5a26b38821233861a4dfd8e8a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 30 Jun 2016 12:03:54 -0700 Subject: [PATCH 0916/1470] [SPARK-16289][SQL] Implement posexplode table generating function This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive. **Before** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7 ``` **After** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show +---+---+-----+ |pos|key|value| +---+---+-----+ | 0| a| 1| | 1| b| 2| +---+---+-----+ ``` For `array` argument, `after` is the same with `before`. ``` scala> sql("select posexplode(array(1, 2, 3))").show +---+---+ |pos|col| +---+---+ | 0| 1| | 1| 2| | 2| 3| +---+---+ ``` Pass the Jenkins tests with newly added testcases. Author: Dongjoon Hyun Closes #13971 from dongjoon-hyun/SPARK-16289. (cherry picked from commit 46395db80e3304e3f3a1ebdc8aadb8f2819b48b4) Signed-off-by: Reynold Xin --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 17 ++++ R/pkg/R/generics.R | 4 + R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- python/pyspark/sql/functions.py | 21 +++++ .../catalyst/analysis/FunctionRegistry.scala | 1 + .../sql/catalyst/expressions/generators.scala | 66 +++++++++++-- .../ExpressionTypeCheckingSuite.scala | 2 + .../GeneratorExpressionSuite.scala | 71 ++++++++++++++ .../scala/org/apache/spark/sql/Column.scala | 1 + .../org/apache/spark/sql/functions.scala | 8 ++ .../spark/sql/ColumnExpressionSuite.scala | 60 ------------ .../spark/sql/GeneratorFunctionSuite.scala | 92 +++++++++++++++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- 14 files changed, 276 insertions(+), 72 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 9fd2568078e63..bc3aceba22568 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -235,6 +235,7 @@ exportMethods("%in%", "over", "percent_rank", "pmod", + "posexplode", "quarter", "rand", "randn", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 09e5afa97060e..52d46f9d76120 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2934,3 +2934,20 @@ setMethod("sort_array", jc <- callJStatic("org.apache.spark.sql.functions", "sort_array", x@jc, asc) column(jc) }) + +#' posexplode +#' +#' Creates a new row for each element with position in the given array or map column. +#' +#' @rdname posexplode +#' @name posexplode +#' @family collection_funcs +#' @export +#' @examples \dontrun{posexplode(df$c)} +#' @note posexplode since 2.1.0 +setMethod("posexplode", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "posexplode", x@jc) + column(jc) + }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index b0f25deaf3e82..e4ec508795a14 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1054,6 +1054,10 @@ setGeneric("percent_rank", function(x) { standardGeneric("percent_rank") }) #' @export setGeneric("pmod", function(y, x) { standardGeneric("pmod") }) +#' @rdname posexplode +#' @export +setGeneric("posexplode", function(x) { standardGeneric("posexplode") }) + #' @rdname quarter #' @export setGeneric("quarter", function(x) { standardGeneric("quarter") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 755aded2272e1..bd7b5f062e6d4 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1085,7 +1085,7 @@ test_that("column functions", { c4 <- explode(c) + expm1(c) + factorial(c) + first(c) + floor(c) + hex(c) c5 <- hour(c) + initcap(c) + last(c) + last_day(c) + length(c) c6 <- log(c) + (c) + log1p(c) + log2(c) + lower(c) + ltrim(c) + max(c) + md5(c) - c7 <- mean(c) + min(c) + month(c) + negate(c) + quarter(c) + c7 <- mean(c) + min(c) + month(c) + negate(c) + posexplode(c) + quarter(c) c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + sha1(c) + monotonically_increasing_id() c9 <- signum(c) + sin(c) + sinh(c) + size(c) + stddev(c) + soundex(c) + sqrt(c) + sum(c) c10 <- sumDistinct(c) + tan(c) + tanh(c) + toDegrees(c) + toRadians(c) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 1feca6e8caafa..92d709ee40e1f 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1637,6 +1637,27 @@ def explode(col): return Column(jc) +@since(2.1) +def posexplode(col): + """Returns a new row for each element with position in the given array or map. + + >>> from pyspark.sql import Row + >>> eDF = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})]) + >>> eDF.select(posexplode(eDF.intlist)).collect() + [Row(pos=0, col=1), Row(pos=1, col=2), Row(pos=2, col=3)] + + >>> eDF.select(posexplode(eDF.mapfield)).show() + +---+---+-----+ + |pos|key|value| + +---+---+-----+ + | 0| a| b| + +---+---+-----+ + """ + sc = SparkContext._active_spark_context + jc = sc._jvm.functions.posexplode(_to_java_column(col)) + return Column(jc) + + @ignore_unicode_prefix @since(1.6) def get_json_object(col, path): 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 27c3a09571a9f..346cdd88441aa 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 @@ -177,6 +177,7 @@ object FunctionRegistry { expression[NullIf]("nullif"), expression[Nvl]("nvl"), expression[Nvl2]("nvl2"), + expression[PosExplode]("posexplode"), expression[Rand]("rand"), expression[Randn]("randn"), expression[CreateStruct]("struct"), 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 12c35644e564c..4e91cc5aec645 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 @@ -94,13 +94,10 @@ case class UserDefinedGenerator( } /** - * Given an input array produces a sequence of rows for each value in the array. + * A base class for Explode and PosExplode */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(a) - Separates the elements of array a into multiple rows, or the elements of a map into multiple rows and columns.") -// scalastyle:on line.size.limit -case class Explode(child: Expression) extends UnaryExpression with Generator with CodegenFallback { +abstract class ExplodeBase(child: Expression, position: Boolean) + extends UnaryExpression with Generator with CodegenFallback with Serializable { override def children: Seq[Expression] = child :: Nil @@ -115,9 +112,26 @@ case class Explode(child: Expression) extends UnaryExpression with Generator wit // hive-compatible default alias for explode function ("col" for array, "key", "value" for map) override def elementSchema: StructType = child.dataType match { - case ArrayType(et, containsNull) => new StructType().add("col", et, containsNull) + case ArrayType(et, containsNull) => + if (position) { + new StructType() + .add("pos", IntegerType, false) + .add("col", et, containsNull) + } else { + new StructType() + .add("col", et, containsNull) + } case MapType(kt, vt, valueContainsNull) => - new StructType().add("key", kt, false).add("value", vt, valueContainsNull) + if (position) { + new StructType() + .add("pos", IntegerType, false) + .add("key", kt, false) + .add("value", vt, valueContainsNull) + } else { + new StructType() + .add("key", kt, false) + .add("value", vt, valueContainsNull) + } } override def eval(input: InternalRow): TraversableOnce[InternalRow] = { @@ -129,7 +143,7 @@ case class Explode(child: Expression) extends UnaryExpression with Generator wit } else { val rows = new Array[InternalRow](inputArray.numElements()) inputArray.foreach(et, (i, e) => { - rows(i) = InternalRow(e) + rows(i) = if (position) InternalRow(i, e) else InternalRow(e) }) rows } @@ -141,7 +155,7 @@ case class Explode(child: Expression) extends UnaryExpression with Generator wit val rows = new Array[InternalRow](inputMap.numElements()) var i = 0 inputMap.foreach(kt, vt, (k, v) => { - rows(i) = InternalRow(k, v) + rows(i) = if (position) InternalRow(i, k, v) else InternalRow(k, v) i += 1 }) rows @@ -149,3 +163,35 @@ case class Explode(child: Expression) extends UnaryExpression with Generator wit } } } + +/** + * Given an input array produces a sequence of rows for each value in the array. + * + * {{{ + * SELECT explode(array(10,20)) -> + * 10 + * 20 + * }}} + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(a) - Separates the elements of array a into multiple rows, or the elements of map a into multiple rows and columns.", + extended = "> SELECT _FUNC_(array(10,20));\n 10\n 20") +// scalastyle:on line.size.limit +case class Explode(child: Expression) extends ExplodeBase(child, position = false) + +/** + * Given an input array produces a sequence of rows for each position and value in the array. + * + * {{{ + * SELECT posexplode(array(10,20)) -> + * 0 10 + * 1 20 + * }}} + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(a) - Separates the elements of array a into multiple rows with positions, or the elements of a map into multiple rows and columns with positions.", + extended = "> SELECT _FUNC_(array(10,20));\n 0\t10\n 1\t20") +// scalastyle:on line.size.limit +case class PosExplode(child: Expression) extends ExplodeBase(child, position = true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 54436ea9a4a72..76e42d9afa4c3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -166,6 +166,8 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertError(new Murmur3Hash(Nil), "function hash requires at least one argument") assertError(Explode('intField), "input to function explode should be array or map type") + assertError(PosExplode('intField), + "input to function explode should be array or map type") } test("check types for CreateNamedStruct") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala new file mode 100644 index 0000000000000..2aba84141b2d0 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.unsafe.types.UTF8String + +class GeneratorExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + private def checkTuple(actual: ExplodeBase, expected: Seq[InternalRow]): Unit = { + assert(actual.eval(null).toSeq === expected) + } + + private final val int_array = Seq(1, 2, 3) + private final val str_array = Seq("a", "b", "c") + + test("explode") { + val int_correct_answer = Seq(Seq(1), Seq(2), Seq(3)) + val str_correct_answer = Seq( + Seq(UTF8String.fromString("a")), + Seq(UTF8String.fromString("b")), + Seq(UTF8String.fromString("c"))) + + checkTuple( + Explode(CreateArray(Seq.empty)), + Seq.empty) + + checkTuple( + Explode(CreateArray(int_array.map(Literal(_)))), + int_correct_answer.map(InternalRow.fromSeq(_))) + + checkTuple( + Explode(CreateArray(str_array.map(Literal(_)))), + str_correct_answer.map(InternalRow.fromSeq(_))) + } + + test("posexplode") { + val int_correct_answer = Seq(Seq(0, 1), Seq(1, 2), Seq(2, 3)) + val str_correct_answer = Seq( + Seq(0, UTF8String.fromString("a")), + Seq(1, UTF8String.fromString("b")), + Seq(2, UTF8String.fromString("c"))) + + checkTuple( + PosExplode(CreateArray(Seq.empty)), + Seq.empty) + + checkTuple( + PosExplode(CreateArray(int_array.map(Literal(_)))), + int_correct_answer.map(InternalRow.fromSeq(_))) + + checkTuple( + PosExplode(CreateArray(str_array.map(Literal(_)))), + str_correct_answer.map(InternalRow.fromSeq(_))) + } +} 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 9f35107e5bb6d..a46d1949e94ae 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 @@ -159,6 +159,7 @@ class Column(protected[sql] val expr: Expression) extends Logging { // Leave an unaliased generator with an empty list of names since the analyzer will generate // the correct defaults after the nested expression's type has been resolved. case explode: Explode => MultiAlias(explode, Nil) + case explode: PosExplode => MultiAlias(explode, Nil) case jt: JsonTuple => MultiAlias(jt, Nil) 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 e8bd489be3410..c8782df146df6 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 @@ -2721,6 +2721,14 @@ object functions { */ def explode(e: Column): Column = withExpr { Explode(e.expr) } + /** + * Creates a new row for each element with position in the given array or map column. + * + * @group collection_funcs + * @since 2.1.0 + */ + def posexplode(e: Column): Column = withExpr { PosExplode(e.expr) } + /** * Extracts json object from a json string based on json path specified, and returns json string * of the extracted json object. It will return null if the input json string is invalid. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a66c83dea00b2..a170fae577c1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -122,66 +122,6 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { assert(newCol.expr.asInstanceOf[NamedExpression].metadata.getString("key") === "value") } - test("single explode") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - checkAnswer( - df.select(explode('intList)), - Row(1) :: Row(2) :: Row(3) :: Nil) - } - - test("explode and other columns") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - - checkAnswer( - df.select($"a", explode('intList)), - Row(1, 1) :: - Row(1, 2) :: - Row(1, 3) :: Nil) - - checkAnswer( - df.select($"*", explode('intList)), - Row(1, Seq(1, 2, 3), 1) :: - Row(1, Seq(1, 2, 3), 2) :: - Row(1, Seq(1, 2, 3), 3) :: Nil) - } - - test("aliased explode") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - - checkAnswer( - df.select(explode('intList).as('int)).select('int), - Row(1) :: Row(2) :: Row(3) :: Nil) - - checkAnswer( - df.select(explode('intList).as('int)).select(sum('int)), - Row(6) :: Nil) - } - - test("explode on map") { - val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") - - checkAnswer( - df.select(explode('map)), - Row("a", "b")) - } - - test("explode on map with aliases") { - val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") - - checkAnswer( - df.select(explode('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), - Row("a", "b")) - } - - test("self join explode") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - val exploded = df.select(explode('intList).as('i)) - - checkAnswer( - exploded.join(exploded, exploded("i") === exploded("i")).agg(count("*")), - Row(3) :: Nil) - } - test("collect on column produced by a binary operator") { val df = Seq((1, 2, 3)).toDF("a", "b", "c") checkAnswer(df.select(df("a") + df("b")), Seq(Row(3))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala new file mode 100644 index 0000000000000..1f0ef34ec1935 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext + +class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("single explode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + checkAnswer( + df.select(explode('intList)), + Row(1) :: Row(2) :: Row(3) :: Nil) + } + + test("single posexplode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + checkAnswer( + df.select(posexplode('intList)), + Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Nil) + } + + test("explode and other columns") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + + checkAnswer( + df.select($"a", explode('intList)), + Row(1, 1) :: + Row(1, 2) :: + Row(1, 3) :: Nil) + + checkAnswer( + df.select($"*", explode('intList)), + Row(1, Seq(1, 2, 3), 1) :: + Row(1, Seq(1, 2, 3), 2) :: + Row(1, Seq(1, 2, 3), 3) :: Nil) + } + + test("aliased explode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + + checkAnswer( + df.select(explode('intList).as('int)).select('int), + Row(1) :: Row(2) :: Row(3) :: Nil) + + checkAnswer( + df.select(explode('intList).as('int)).select(sum('int)), + Row(6) :: Nil) + } + + test("explode on map") { + val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") + + checkAnswer( + df.select(explode('map)), + Row("a", "b")) + } + + test("explode on map with aliases") { + val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") + + checkAnswer( + df.select(explode('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), + Row("a", "b")) + } + + test("self join explode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + val exploded = df.select(explode('intList).as('i)) + + checkAnswer( + exploded.join(exploded, exploded("i") === exploded("i")).agg(count("*")), + Row(3) :: Nil) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 4c986b0f5a200..9fe0bf44ba570 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -241,6 +241,6 @@ private[sql] class HiveSessionCatalog( "xpath_number", "xpath_short", "xpath_string", // table generating function - "inline", "posexplode" + "inline" ) } From 144aa84ce0f3463d95c06c78df6e9996ad42240a Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Tue, 28 Jun 2016 21:07:52 -0700 Subject: [PATCH 0917/1470] [SPARK-16271][SQL] Implement Hive's UDFXPathUtil This patch ports Hive's UDFXPathUtil over to Spark, which can be used to implement xpath functionality in Spark in the near future. Added two new test suites UDFXPathUtilSuite and ReusableStringReaderSuite. They have been ported over from Hive (but rewritten in Scala in order to leverage ScalaTest). Author: petermaxlee Closes #13961 from petermaxlee/xpath. (cherry picked from commit 153c2f9ac12846367a09684fd875c496d350a603) Signed-off-by: Reynold Xin --- .../expressions/xml/UDFXPathUtil.java | 192 ++++++++++++++++++ .../xml/ReusableStringReaderSuite.scala | 103 ++++++++++ .../expressions/xml/UDFXPathUtilSuite.scala | 99 +++++++++ 3 files changed, 394 insertions(+) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/ReusableStringReaderSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java new file mode 100644 index 0000000000000..01a11f9bdca2d --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.xml; + +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; + +import javax.xml.namespace.QName; +import javax.xml.xpath.XPath; +import javax.xml.xpath.XPathConstants; +import javax.xml.xpath.XPathExpression; +import javax.xml.xpath.XPathExpressionException; +import javax.xml.xpath.XPathFactory; + +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; +import org.xml.sax.InputSource; + +/** + * Utility class for all XPath UDFs. Each UDF instance should keep an instance of this class. + * + * This is based on Hive's UDFXPathUtil implementation. + */ +public class UDFXPathUtil { + private XPath xpath = XPathFactory.newInstance().newXPath(); + private ReusableStringReader reader = new ReusableStringReader(); + private InputSource inputSource = new InputSource(reader); + private XPathExpression expression = null; + private String oldPath = null; + + public Object eval(String xml, String path, QName qname) { + if (xml == null || path == null || qname == null) { + return null; + } + + if (xml.length() == 0 || path.length() == 0) { + return null; + } + + if (!path.equals(oldPath)) { + try { + expression = xpath.compile(path); + } catch (XPathExpressionException e) { + expression = null; + } + oldPath = path; + } + + if (expression == null) { + return null; + } + + reader.set(xml); + + try { + return expression.evaluate(inputSource, qname); + } catch (XPathExpressionException e) { + throw new RuntimeException ("Invalid expression '" + oldPath + "'", e); + } + } + + public Boolean evalBoolean(String xml, String path) { + return (Boolean) eval(xml, path, XPathConstants.BOOLEAN); + } + + public String evalString(String xml, String path) { + return (String) eval(xml, path, XPathConstants.STRING); + } + + public Double evalNumber(String xml, String path) { + return (Double) eval(xml, path, XPathConstants.NUMBER); + } + + public Node evalNode(String xml, String path) { + return (Node) eval(xml, path, XPathConstants.NODE); + } + + public NodeList evalNodeList(String xml, String path) { + return (NodeList) eval(xml, path, XPathConstants.NODESET); + } + + /** + * Reusable, non-threadsafe version of {@link StringReader}. + */ + public static class ReusableStringReader extends Reader { + + private String str = null; + private int length = -1; + private int next = 0; + private int mark = 0; + + public ReusableStringReader() { + } + + public void set(String s) { + this.str = s; + this.length = s.length(); + this.mark = 0; + this.next = 0; + } + + /** Check to make sure that the stream has not been closed */ + private void ensureOpen() throws IOException { + if (str == null) + throw new IOException("Stream closed"); + } + + @Override + public int read() throws IOException { + ensureOpen(); + if (next >= length) + return -1; + return str.charAt(next++); + } + + @Override + public int read(char cbuf[], int off, int len) throws IOException { + ensureOpen(); + if ((off < 0) || (off > cbuf.length) || (len < 0) + || ((off + len) > cbuf.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException(); + } else if (len == 0) { + return 0; + } + if (next >= length) + return -1; + int n = Math.min(length - next, len); + str.getChars(next, next + n, cbuf, off); + next += n; + return n; + } + + @Override + public long skip(long ns) throws IOException { + ensureOpen(); + if (next >= length) + return 0; + // Bound skip by beginning and end of the source + long n = Math.min(length - next, ns); + n = Math.max(-next, n); + next += n; + return n; + } + + @Override + public boolean ready() throws IOException { + ensureOpen(); + return true; + } + + @Override + public boolean markSupported() { + return true; + } + + @Override + public void mark(int readAheadLimit) throws IOException { + if (readAheadLimit < 0) { + throw new IllegalArgumentException("Read-ahead limit < 0"); + } + ensureOpen(); + mark = next; + } + + @Override + public void reset() throws IOException { + ensureOpen(); + next = mark; + } + + @Override + public void close() { + str = null; + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/ReusableStringReaderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/ReusableStringReaderSuite.scala new file mode 100644 index 0000000000000..e06d209c474be --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/ReusableStringReaderSuite.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.xml + +import java.io.IOException + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.xml.UDFXPathUtil.ReusableStringReader + +/** + * Unit tests for [[UDFXPathUtil.ReusableStringReader]]. + * + * Loosely based on Hive's TestReusableStringReader.java. + */ +class ReusableStringReaderSuite extends SparkFunSuite { + + private val fox = "Quick brown fox jumps over the lazy dog." + + test("empty reader") { + val reader = new ReusableStringReader + + intercept[IOException] { + reader.read() + } + + intercept[IOException] { + reader.ready() + } + + reader.close() + } + + test("mark reset") { + val reader = new ReusableStringReader + + if (reader.markSupported()) { + reader.asInstanceOf[ReusableStringReader].set(fox) + assert(reader.ready()) + + val cc = new Array[Char](6) + var read = reader.read(cc) + assert(read == 6) + assert("Quick " == new String(cc)) + + reader.mark(100) + + read = reader.read(cc) + assert(read == 6) + assert("brown " == new String(cc)) + + reader.reset() + read = reader.read(cc) + assert(read == 6) + assert("brown " == new String(cc)) + } + reader.close() + } + + test("skip") { + val reader = new ReusableStringReader + reader.asInstanceOf[ReusableStringReader].set(fox) + + // skip entire the data: + var skipped = reader.skip(fox.length() + 1) + assert(fox.length() == skipped) + assert(-1 == reader.read()) + + reader.asInstanceOf[ReusableStringReader].set(fox) // reset the data + val cc = new Array[Char](6) + var read = reader.read(cc) + assert(read == 6) + assert("Quick " == new String(cc)) + + // skip some piece of data: + skipped = reader.skip(30) + assert(skipped == 30) + read = reader.read(cc) + assert(read == 4) + assert("dog." == new String(cc, 0, read)) + + // skip when already at EOF: + skipped = reader.skip(300) + assert(skipped == 0, skipped) + assert(reader.read() == -1) + + reader.close() + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala new file mode 100644 index 0000000000000..a5614f83844e0 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.xml + +import javax.xml.xpath.XPathConstants.STRING + +import org.w3c.dom.Node +import org.w3c.dom.NodeList + +import org.apache.spark.SparkFunSuite + +/** + * Unit tests for [[UDFXPathUtil]]. Loosely based on Hive's TestUDFXPathUtil.java. + */ +class UDFXPathUtilSuite extends SparkFunSuite { + + private lazy val util = new UDFXPathUtil + + test("illegal arguments") { + // null args + assert(util.eval(null, "a/text()", STRING) == null) + assert(util.eval("b1b2b3c1c2", null, STRING) == null) + assert( + util.eval("b1b2b3c1c2", "a/text()", null) == null) + + // empty String args + assert(util.eval("", "a/text()", STRING) == null) + assert(util.eval("b1b2b3c1c2", "", STRING) == null) + + // wrong expression: + assert( + util.eval("b1b2b3c1c2", "a/text(", STRING) == null) + } + + test("generic eval") { + val ret = + util.eval("b1b2b3c1c2", "a/c[2]/text()", STRING) + assert(ret == "c2") + } + + test("boolean eval") { + var ret = + util.evalBoolean("truefalseb3c1c2", "a/b[1]/text()") + assert(ret == true) + + ret = util.evalBoolean("truefalseb3c1c2", "a/b[4]") + assert(ret == false) + } + + test("string eval") { + var ret = + util.evalString("truefalseb3c1c2", "a/b[3]/text()") + assert(ret == "b3") + + ret = + util.evalString("truefalseb3c1c2", "a/b[4]/text()") + assert(ret == "") + + ret = util.evalString( + "trueFALSEb3c1c2", "a/b[2]/@k") + assert(ret == "foo") + } + + test("number eval") { + var ret = + util.evalNumber("truefalseb3c1-77", "a/c[2]") + assert(ret == -77.0d) + + ret = util.evalNumber( + "trueFALSEb3c1c2", "a/b[2]/@k") + assert(ret.isNaN) + } + + test("node eval") { + val ret = util.evalNode("truefalseb3c1-77", "a/c[2]") + assert(ret != null && ret.isInstanceOf[Node]) + } + + test("node list eval") { + val ret = util.evalNodeList("truefalseb3c1-77", "a/*") + assert(ret != null && ret.isInstanceOf[NodeList]) + assert(ret.asInstanceOf[NodeList].getLength == 5) + } +} From bb4b0419b1dcd2b1926a829488a5a1d1b43756e0 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 30 Jun 2016 09:27:48 +0800 Subject: [PATCH 0918/1470] [SPARK-16274][SQL] Implement xpath_boolean This patch implements xpath_boolean expression for Spark SQL, a xpath function that returns true or false. The implementation is modelled after Hive's xpath_boolean, except that how the expression handles null inputs. Hive throws a NullPointerException at runtime if either of the input is null. This implementation returns null if either of the input is null. Created two new test suites. One for unit tests covering the expression, and the other for end-to-end test in SQL. Author: petermaxlee Closes #13964 from petermaxlee/SPARK-16274. (cherry picked from commit d3af6731fa270842818ed91d6b4d14708ddae2db) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../expressions/xml/XPathBoolean.scala | 58 ++++++++++++++++++ .../xml/XPathExpressionSuite.scala | 61 +++++++++++++++++++ .../apache/spark/sql/XmlFunctionsSuite.scala | 32 ++++++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- 5 files changed, 154 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala 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 346cdd88441aa..e7f335f4fb4ed 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 @@ -25,6 +25,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.expressions.xml._ import org.apache.spark.sql.catalyst.util.StringKeyHashMap @@ -305,6 +306,7 @@ object FunctionRegistry { expression[UnBase64]("unbase64"), expression[Unhex]("unhex"), expression[Upper]("upper"), + expression[XPathBoolean]("xpath_boolean"), // datetime functions expression[AddMonths]("add_months"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala new file mode 100644 index 0000000000000..2a5256c7f56fd --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.xml + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType} +import org.apache.spark.unsafe.types.UTF8String + + +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.", + extended = "> SELECT _FUNC_('1','a/b');\ntrue") +case class XPathBoolean(xml: Expression, path: Expression) + extends BinaryExpression with ExpectsInputTypes with CodegenFallback { + + @transient private lazy val xpathUtil = new UDFXPathUtil + + // If the path is a constant, cache the path string so that we don't need to convert path + // from UTF8String to String for every row. + @transient lazy val pathLiteral: String = path match { + case Literal(str: UTF8String, _) => str.toString + case _ => null + } + + override def prettyName: String = "xpath_boolean" + + override def dataType: DataType = BooleanType + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType) + + override def left: Expression = xml + override def right: Expression = path + + override protected def nullSafeEval(xml: Any, path: Any): Any = { + val xmlString = xml.asInstanceOf[UTF8String].toString + if (pathLiteral ne null) { + xpathUtil.evalBoolean(xmlString, pathLiteral) + } else { + xpathUtil.evalBoolean(xmlString, path.asInstanceOf[UTF8String].toString) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala new file mode 100644 index 0000000000000..f7c65c667efbd --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.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.sql.catalyst.expressions.xml + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} +import org.apache.spark.sql.types.StringType + +/** + * Test suite for various xpath functions. + */ +class XPathExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + + private def testBoolean[T](xml: String, path: String, expected: T): Unit = { + checkEvaluation( + XPathBoolean(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + test("xpath_boolean") { + testBoolean("b", "a/b", true) + testBoolean("b", "a/c", false) + testBoolean("b", "a/b = \"b\"", true) + testBoolean("b", "a/b = \"c\"", false) + testBoolean("10", "a/b < 10", false) + testBoolean("10", "a/b = 10", true) + + // null input + testBoolean(null, null, null) + testBoolean(null, "a", null) + testBoolean("10", null, null) + + // exception handling for invalid input + intercept[Exception] { + testBoolean("/a>", "a", null) + } + } + + test("xpath_boolean path cache invalidation") { + // This is a test to ensure the expression is not reusing the path for different strings + val expr = XPathBoolean(Literal("b"), 'path.string.at(0)) + checkEvaluation(expr, true, create_row("a/b")) + checkEvaluation(expr, false, create_row("a/c")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala new file mode 100644 index 0000000000000..532d48cc265ac --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext + +/** + * End-to-end tests for XML expressions. + */ +class XmlFunctionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("xpath_boolean") { + val df = Seq("b" -> "a/b").toDF("xml", "path") + checkAnswer(df.selectExpr("xpath_boolean(xml, path)"), Row(true)) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 9fe0bf44ba570..115098a8fbcc4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -237,7 +237,7 @@ private[sql] class HiveSessionCatalog( private val hiveFunctions = Seq( "hash", "java_method", "histogram_numeric", "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "stack", "str_to_map", - "xpath", "xpath_boolean", "xpath_double", "xpath_float", "xpath_int", "xpath_long", + "xpath", "xpath_double", "xpath_float", "xpath_int", "xpath_long", "xpath_number", "xpath_short", "xpath_string", // table generating function From e32c29d86d4cc7ebe8e485c4221b5a10366b3d7d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 4 Jul 2016 01:57:45 +0800 Subject: [PATCH 0919/1470] [SPARK-16288][SQL] Implement inline table generating function This PR implements `inline` table generating function. Pass the Jenkins tests with new testcase. Author: Dongjoon Hyun Closes #13976 from dongjoon-hyun/SPARK-16288. (cherry picked from commit 88134e736829f5f93a82879c08cb191f175ff8af) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../sql/catalyst/expressions/generators.scala | 35 +++++++++++ .../GeneratorExpressionSuite.scala | 59 +++++++++--------- .../spark/sql/GeneratorFunctionSuite.scala | 60 +++++++++++++++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 5 +- 5 files changed, 124 insertions(+), 36 deletions(-) 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 e7f335f4fb4ed..021bec7f5f120 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 @@ -165,6 +165,7 @@ object FunctionRegistry { expression[Explode]("explode"), expression[Greatest]("greatest"), expression[If]("if"), + expression[Inline]("inline"), expression[IsNaN]("isnan"), expression[IfNull]("ifnull"), expression[IsNull]("isnull"), 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 4e91cc5aec645..99b97c8ea2358 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 @@ -195,3 +195,38 @@ case class Explode(child: Expression) extends ExplodeBase(child, position = fals extended = "> SELECT _FUNC_(array(10,20));\n 0\t10\n 1\t20") // scalastyle:on line.size.limit case class PosExplode(child: Expression) extends ExplodeBase(child, position = true) + +/** + * Explodes an array of structs into a table. + */ +@ExpressionDescription( + usage = "_FUNC_(a) - Explodes an array of structs into a table.", + extended = "> SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b')));\n [1,a]\n [2,b]") +case class Inline(child: Expression) extends UnaryExpression with Generator with CodegenFallback { + + override def children: Seq[Expression] = child :: Nil + + override def checkInputDataTypes(): TypeCheckResult = child.dataType match { + case ArrayType(et, _) if et.isInstanceOf[StructType] => + TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure( + s"input to function $prettyName should be array of struct type, not ${child.dataType}") + } + + override def elementSchema: StructType = child.dataType match { + case ArrayType(et : StructType, _) => et + } + + private lazy val numFields = elementSchema.fields.length + + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { + val inputArray = child.eval(input).asInstanceOf[ArrayData] + if (inputArray == null) { + Nil + } else { + for (i <- 0 until inputArray.numElements()) + yield inputArray.getStruct(i, numFields) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala index 2aba84141b2d0..e79f89b4970b6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala @@ -19,53 +19,48 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.sql.types._ class GeneratorExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { - private def checkTuple(actual: ExplodeBase, expected: Seq[InternalRow]): Unit = { - assert(actual.eval(null).toSeq === expected) + private def checkTuple(actual: Expression, expected: Seq[InternalRow]): Unit = { + assert(actual.eval(null).asInstanceOf[TraversableOnce[InternalRow]].toSeq === expected) } - private final val int_array = Seq(1, 2, 3) - private final val str_array = Seq("a", "b", "c") + private final val empty_array = CreateArray(Seq.empty) + private final val int_array = CreateArray(Seq(1, 2, 3).map(Literal(_))) + private final val str_array = CreateArray(Seq("a", "b", "c").map(Literal(_))) test("explode") { - val int_correct_answer = Seq(Seq(1), Seq(2), Seq(3)) - val str_correct_answer = Seq( - Seq(UTF8String.fromString("a")), - Seq(UTF8String.fromString("b")), - Seq(UTF8String.fromString("c"))) + val int_correct_answer = Seq(create_row(1), create_row(2), create_row(3)) + val str_correct_answer = Seq(create_row("a"), create_row("b"), create_row("c")) - checkTuple( - Explode(CreateArray(Seq.empty)), - Seq.empty) + checkTuple(Explode(empty_array), Seq.empty) + checkTuple(Explode(int_array), int_correct_answer) + checkTuple(Explode(str_array), str_correct_answer) + } - checkTuple( - Explode(CreateArray(int_array.map(Literal(_)))), - int_correct_answer.map(InternalRow.fromSeq(_))) + test("posexplode") { + val int_correct_answer = Seq(create_row(0, 1), create_row(1, 2), create_row(2, 3)) + val str_correct_answer = Seq(create_row(0, "a"), create_row(1, "b"), create_row(2, "c")) - checkTuple( - Explode(CreateArray(str_array.map(Literal(_)))), - str_correct_answer.map(InternalRow.fromSeq(_))) + checkTuple(PosExplode(CreateArray(Seq.empty)), Seq.empty) + checkTuple(PosExplode(int_array), int_correct_answer) + checkTuple(PosExplode(str_array), str_correct_answer) } - test("posexplode") { - val int_correct_answer = Seq(Seq(0, 1), Seq(1, 2), Seq(2, 3)) - val str_correct_answer = Seq( - Seq(0, UTF8String.fromString("a")), - Seq(1, UTF8String.fromString("b")), - Seq(2, UTF8String.fromString("c"))) + test("inline") { + val correct_answer = Seq(create_row(0, "a"), create_row(1, "b"), create_row(2, "c")) checkTuple( - PosExplode(CreateArray(Seq.empty)), + Inline(Literal.create(Array(), ArrayType(new StructType().add("id", LongType)))), Seq.empty) checkTuple( - PosExplode(CreateArray(int_array.map(Literal(_)))), - int_correct_answer.map(InternalRow.fromSeq(_))) - - checkTuple( - PosExplode(CreateArray(str_array.map(Literal(_)))), - str_correct_answer.map(InternalRow.fromSeq(_))) + Inline(CreateArray(Seq( + CreateStruct(Seq(Literal(0), Literal("a"))), + CreateStruct(Seq(Literal(1), Literal("b"))), + CreateStruct(Seq(Literal(2), Literal("c"))) + ))), + correct_answer) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index 1f0ef34ec1935..d8a0aa4d52942 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -89,4 +89,64 @@ class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { exploded.join(exploded, exploded("i") === exploded("i")).agg(count("*")), Row(3) :: Nil) } + + test("inline raises exception on array of null type") { + val m = intercept[AnalysisException] { + spark.range(2).selectExpr("inline(array())") + }.getMessage + assert(m.contains("data type mismatch")) + } + + test("inline with empty table") { + checkAnswer( + spark.range(0).selectExpr("inline(array(struct(10, 100)))"), + Nil) + } + + test("inline on literal") { + checkAnswer( + spark.range(2).selectExpr("inline(array(struct(10, 100), struct(20, 200), struct(30, 300)))"), + Row(10, 100) :: Row(20, 200) :: Row(30, 300) :: + Row(10, 100) :: Row(20, 200) :: Row(30, 300) :: Nil) + } + + test("inline on column") { + val df = Seq((1, 2)).toDF("a", "b") + + checkAnswer( + df.selectExpr("inline(array(struct(a), struct(a)))"), + Row(1) :: Row(1) :: Nil) + + checkAnswer( + df.selectExpr("inline(array(struct(a, b), struct(a, b)))"), + Row(1, 2) :: Row(1, 2) :: Nil) + + // Spark think [struct, struct] is heterogeneous due to name difference. + val m = intercept[AnalysisException] { + df.selectExpr("inline(array(struct(a), struct(b)))") + }.getMessage + assert(m.contains("data type mismatch")) + + checkAnswer( + df.selectExpr("inline(array(struct(a), named_struct('a', b)))"), + Row(1) :: Row(2) :: Nil) + + // Spark think [struct, struct] is heterogeneous due to name difference. + val m2 = intercept[AnalysisException] { + df.selectExpr("inline(array(struct(a), struct(2)))") + }.getMessage + assert(m2.contains("data type mismatch")) + + checkAnswer( + df.selectExpr("inline(array(struct(a), named_struct('a', 2)))"), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + df.selectExpr("struct(a)").selectExpr("inline(array(*))"), + Row(1) :: Nil) + + checkAnswer( + df.selectExpr("array(struct(a), named_struct('a', b))").selectExpr("inline(*)"), + Row(1) :: Row(2) :: Nil) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 115098a8fbcc4..ebb6711f6ab75 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -238,9 +238,6 @@ private[sql] class HiveSessionCatalog( "hash", "java_method", "histogram_numeric", "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "stack", "str_to_map", "xpath", "xpath_double", "xpath_float", "xpath_int", "xpath_long", - "xpath_number", "xpath_short", "xpath_string", - - // table generating function - "inline" + "xpath_number", "xpath_short", "xpath_string" ) } From 565e18cf7670231b1fa9db84f907654da79e6cef Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 6 Jul 2016 10:54:43 +0800 Subject: [PATCH 0920/1470] [SPARK-16286][SQL] Implement stack table generating function This PR implements `stack` table generating function. Pass the Jenkins tests including new testcases. Author: Dongjoon Hyun Closes #14033 from dongjoon-hyun/SPARK-16286. (cherry picked from commit d0d28507cacfca5919dbfb4269892d58b62e8662) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../sql/catalyst/expressions/generators.scala | 53 +++++++++++++++++++ .../GeneratorExpressionSuite.scala | 18 +++++++ .../spark/sql/GeneratorFunctionSuite.scala | 53 +++++++++++++++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- 5 files changed, 126 insertions(+), 1 deletion(-) 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 021bec7f5f120..f6ebcaeded484 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 @@ -182,6 +182,7 @@ object FunctionRegistry { expression[PosExplode]("posexplode"), expression[Rand]("rand"), expression[Randn]("randn"), + expression[Stack]("stack"), expression[CreateStruct]("struct"), expression[CaseWhen]("when"), 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 99b97c8ea2358..9d5c856a23e2a 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 @@ -93,6 +93,59 @@ case class UserDefinedGenerator( override def toString: String = s"UserDefinedGenerator(${children.mkString(",")})" } +/** + * Separate v1, ..., vk into n rows. Each row will have k/n columns. n must be constant. + * {{{ + * SELECT stack(2, 1, 2, 3) -> + * 1 2 + * 3 NULL + * }}} + */ +@ExpressionDescription( + usage = "_FUNC_(n, v1, ..., vk) - Separate v1, ..., vk into n rows.", + extended = "> SELECT _FUNC_(2, 1, 2, 3);\n [1,2]\n [3,null]") +case class Stack(children: Seq[Expression]) + extends Expression with Generator with CodegenFallback { + + private lazy val numRows = children.head.eval().asInstanceOf[Int] + private lazy val numFields = Math.ceil((children.length - 1.0) / numRows).toInt + + override def checkInputDataTypes(): TypeCheckResult = { + if (children.length <= 1) { + TypeCheckResult.TypeCheckFailure(s"$prettyName requires at least 2 arguments.") + } else if (children.head.dataType != IntegerType || !children.head.foldable || numRows < 1) { + TypeCheckResult.TypeCheckFailure("The number of rows must be a positive constant integer.") + } else { + for (i <- 1 until children.length) { + val j = (i - 1) % numFields + if (children(i).dataType != elementSchema.fields(j).dataType) { + return TypeCheckResult.TypeCheckFailure( + s"Argument ${j + 1} (${elementSchema.fields(j).dataType}) != " + + s"Argument $i (${children(i).dataType})") + } + } + TypeCheckResult.TypeCheckSuccess + } + } + + override def elementSchema: StructType = + StructType(children.tail.take(numFields).zipWithIndex.map { + case (e, index) => StructField(s"col$index", e.dataType) + }) + + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { + val values = children.tail.map(_.eval(input)).toArray + for (row <- 0 until numRows) yield { + val fields = new Array[Any](numFields) + for (col <- 0 until numFields) { + val index = row * numFields + col + fields.update(col, if (index < values.length) values(index) else null) + } + InternalRow(fields: _*) + } + } +} + /** * A base class for Explode and PosExplode */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala index e79f89b4970b6..e29dfa41f1cc5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratorExpressionSuite.scala @@ -63,4 +63,22 @@ class GeneratorExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { ))), correct_answer) } + + test("stack") { + checkTuple(Stack(Seq(1, 1).map(Literal(_))), Seq(create_row(1))) + checkTuple(Stack(Seq(1, 1, 2).map(Literal(_))), Seq(create_row(1, 2))) + checkTuple(Stack(Seq(2, 1, 2).map(Literal(_))), Seq(create_row(1), create_row(2))) + checkTuple(Stack(Seq(2, 1, 2, 3).map(Literal(_))), Seq(create_row(1, 2), create_row(3, null))) + checkTuple(Stack(Seq(3, 1, 2, 3).map(Literal(_))), Seq(1, 2, 3).map(create_row(_))) + checkTuple(Stack(Seq(4, 1, 2, 3).map(Literal(_))), Seq(1, 2, 3, null).map(create_row(_))) + + checkTuple( + Stack(Seq(3, 1, 1.0, "a", 2, 2.0, "b", 3, 3.0, "c").map(Literal(_))), + Seq(create_row(1, 1.0, "a"), create_row(2, 2.0, "b"), create_row(3, 3.0, "c"))) + + assert(Stack(Seq(Literal(1))).checkInputDataTypes().isFailure) + assert(Stack(Seq(Literal(1.0))).checkInputDataTypes().isFailure) + assert(Stack(Seq(Literal(1), Literal(1), Literal(1.0))).checkInputDataTypes().isSuccess) + assert(Stack(Seq(Literal(2), Literal(1), Literal(1.0))).checkInputDataTypes().isFailure) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index d8a0aa4d52942..aedc0a8d6f70b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -23,6 +23,59 @@ import org.apache.spark.sql.test.SharedSQLContext class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { import testImplicits._ + test("stack") { + val df = spark.range(1) + + // Empty DataFrame suppress the result generation + checkAnswer(spark.emptyDataFrame.selectExpr("stack(1, 1, 2, 3)"), Nil) + + // Rows & columns + checkAnswer(df.selectExpr("stack(1, 1, 2, 3)"), Row(1, 2, 3) :: Nil) + checkAnswer(df.selectExpr("stack(2, 1, 2, 3)"), Row(1, 2) :: Row(3, null) :: Nil) + checkAnswer(df.selectExpr("stack(3, 1, 2, 3)"), Row(1) :: Row(2) :: Row(3) :: Nil) + checkAnswer(df.selectExpr("stack(4, 1, 2, 3)"), Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + + // Various column types + checkAnswer(df.selectExpr("stack(3, 1, 1.1, 'a', 2, 2.2, 'b', 3, 3.3, 'c')"), + Row(1, 1.1, "a") :: Row(2, 2.2, "b") :: Row(3, 3.3, "c") :: Nil) + + // Repeat generation at every input row + checkAnswer(spark.range(2).selectExpr("stack(2, 1, 2, 3)"), + Row(1, 2) :: Row(3, null) :: Row(1, 2) :: Row(3, null) :: Nil) + + // The first argument must be a positive constant integer. + val m = intercept[AnalysisException] { + df.selectExpr("stack(1.1, 1, 2, 3)") + }.getMessage + assert(m.contains("The number of rows must be a positive constant integer.")) + val m2 = intercept[AnalysisException] { + df.selectExpr("stack(-1, 1, 2, 3)") + }.getMessage + assert(m2.contains("The number of rows must be a positive constant integer.")) + + // The data for the same column should have the same type. + val m3 = intercept[AnalysisException] { + df.selectExpr("stack(2, 1, '2.2')") + }.getMessage + assert(m3.contains("data type mismatch: Argument 1 (IntegerType) != Argument 2 (StringType)")) + + // stack on column data + val df2 = Seq((2, 1, 2, 3)).toDF("n", "a", "b", "c") + checkAnswer(df2.selectExpr("stack(2, a, b, c)"), Row(1, 2) :: Row(3, null) :: Nil) + + val m4 = intercept[AnalysisException] { + df2.selectExpr("stack(n, a, b, c)") + }.getMessage + assert(m4.contains("The number of rows must be a positive constant integer.")) + + val df3 = Seq((2, 1, 2.0)).toDF("n", "a", "b") + val m5 = intercept[AnalysisException] { + df3.selectExpr("stack(2, a, b)") + }.getMessage + assert(m5.contains("data type mismatch: Argument 1 (IntegerType) != Argument 2 (DoubleType)")) + + } + test("single explode") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") checkAnswer( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index ebb6711f6ab75..fdc4c18e70d69 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -236,7 +236,7 @@ private[sql] class HiveSessionCatalog( // str_to_map, windowingtablefunction. private val hiveFunctions = Seq( "hash", "java_method", "histogram_numeric", - "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "stack", "str_to_map", + "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "str_to_map", "xpath", "xpath_double", "xpath_float", "xpath_int", "xpath_long", "xpath_number", "xpath_short", "xpath_string" ) From 18ace015e967910ce363937ac2fa67011e0a3bba Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 7 Jul 2016 23:19:41 -0700 Subject: [PATCH 0921/1470] [SPARK-16430][SQL][STREAMING] Add option maxFilesPerTrigger ## What changes were proposed in this pull request? An option that limits the file stream source to read 1 file at a time enables rate limiting. It has the additional convenience that a static set of files can be used like a stream for testing as this will allows those files to be considered one at a time. This PR adds option `maxFilesPerTrigger`. ## How was this patch tested? New unit test Author: Tathagata Das Closes #14094 from tdas/SPARK-16430. (cherry picked from commit 5bce4580939c27876f11cd75f0dc2190fb9fa908) Signed-off-by: Tathagata Das --- .../streaming/FileStreamSource.scala | 40 ++++++---- .../sql/streaming/DataStreamReader.scala | 10 +++ .../sql/streaming/FileStreamSourceSuite.scala | 76 +++++++++++++++++++ 3 files changed, 112 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 11bf3c0bd2e00..72b335a42ed34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming -import scala.collection.mutable.ArrayBuffer +import scala.util.Try import org.apache.hadoop.fs.Path @@ -46,6 +46,9 @@ class FileStreamSource( private val metadataLog = new HDFSMetadataLog[Seq[String]](sparkSession, metadataPath) private var maxBatchId = metadataLog.getLatest().map(_._1).getOrElse(-1L) + /** Maximum number of new files to be considered in each batch */ + private val maxFilesPerBatch = getMaxFilesPerBatch() + private val seenFiles = new OpenHashSet[String] metadataLog.get(None, Some(maxBatchId)).foreach { case (batchId, files) => files.foreach(seenFiles.add) @@ -58,19 +61,17 @@ class FileStreamSource( * there is no race here, so the cost of `synchronized` should be rare. */ private def fetchMaxOffset(): LongOffset = synchronized { - val filesPresent = fetchAllFiles() - val newFiles = new ArrayBuffer[String]() - filesPresent.foreach { file => - if (!seenFiles.contains(file)) { - logDebug(s"new file: $file") - newFiles.append(file) - seenFiles.add(file) - } else { - logDebug(s"old file: $file") - } + val newFiles = fetchAllFiles().filter(!seenFiles.contains(_)) + val batchFiles = + if (maxFilesPerBatch.nonEmpty) newFiles.take(maxFilesPerBatch.get) else newFiles + batchFiles.foreach { file => + seenFiles.add(file) + logDebug(s"New file: $file") } - - if (newFiles.nonEmpty) { + logTrace(s"Number of new files = ${newFiles.size})") + logTrace(s"Number of files selected for batch = ${batchFiles.size}") + logTrace(s"Number of seen files = ${seenFiles.size}") + if (batchFiles.nonEmpty) { maxBatchId += 1 metadataLog.add(maxBatchId, newFiles) logInfo(s"Max batch id increased to $maxBatchId with ${newFiles.size} new files") @@ -118,7 +119,7 @@ class FileStreamSource( val startTime = System.nanoTime val globbedPaths = SparkHadoopUtil.get.globPathIfNecessary(qualifiedBasePath) val catalog = new ListingFileCatalog(sparkSession, globbedPaths, options, Some(new StructType)) - val files = catalog.allFiles().map(_.getPath.toUri.toString) + val files = catalog.allFiles().sortBy(_.getModificationTime).map(_.getPath.toUri.toString) val endTime = System.nanoTime val listingTimeMs = (endTime.toDouble - startTime) / 1000000 if (listingTimeMs > 2000) { @@ -131,6 +132,17 @@ class FileStreamSource( files } + private def getMaxFilesPerBatch(): Option[Int] = { + new CaseInsensitiveMap(options) + .get("maxFilesPerTrigger") + .map { str => + Try(str.toInt).toOption.filter(_ > 0).getOrElse { + throw new IllegalArgumentException( + s"Invalid value '$str' for option 'maxFilesPerBatch', must be a positive integer") + } + } + } + override def getOffset: Option[Offset] = Some(fetchMaxOffset()).filterNot(_.offset == -1) override def toString: String = s"FileStreamSource[$qualifiedBasePath]" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 248247a257d94..2e606b21bdf30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -161,6 +161,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * schema in advance, use the version that specifies the schema to avoid the extra scan. * * You can set the following JSON-specific options to deal with non-standard JSON files: + *
    1. `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    2. *
    3. `primitivesAsString` (default `false`): infers all primitive values as a string type
    4. *
    5. `prefersDecimal` (default `false`): infers all floating-point values as a decimal * type. If the values do not fit in decimal, then it infers them as doubles.
    6. @@ -199,6 +201,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * specify the schema explicitly using [[schema]]. * * You can set the following CSV-specific options to deal with CSV files: + *
    7. `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    8. *
    9. `sep` (default `,`): sets the single character as a separator for each * field and value.
    10. *
    11. `encoding` (default `UTF-8`): decodes the CSV files by the given encoding @@ -251,6 +255,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * Loads a Parquet file stream, returning the result as a [[DataFrame]]. * * You can set the following Parquet-specific option(s) for reading Parquet files: + *
    12. `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    13. *
    14. `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets * whether we should merge schemas collected from all Parquet part-files. This will override * `spark.sql.parquet.mergeSchema`.
    15. @@ -276,6 +282,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * spark.readStream().text("/path/to/directory/") * }}} * + * You can set the following text-specific options to deal with text files: + *
    16. `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    17. + * * @since 2.0.0 */ @Experimental diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 8a34cf95f918d..29ce578bcde34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -585,6 +585,82 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + test("max files per trigger") { + withTempDir { case src => + var lastFileModTime: Option[Long] = None + + /** Create a text file with a single data item */ + def createFile(data: Int): File = { + val file = stringToFile(new File(src, s"$data.txt"), data.toString) + if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000) + lastFileModTime = Some(file.lastModified) + file + } + + createFile(1) + createFile(2) + createFile(3) + + // Set up a query to read text files 2 at a time + val df = spark + .readStream + .option("maxFilesPerTrigger", 2) + .text(src.getCanonicalPath) + val q = df + .writeStream + .format("memory") + .queryName("file_data") + .start() + .asInstanceOf[StreamExecution] + q.processAllAvailable() + val memorySink = q.sink.asInstanceOf[MemorySink] + val fileSource = q.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => + source.asInstanceOf[FileStreamSource] + }.head + + /** Check the data read in the last batch */ + def checkLastBatchData(data: Int*): Unit = { + val schema = StructType(Seq(StructField("value", StringType))) + val df = spark.createDataFrame( + spark.sparkContext.makeRDD(memorySink.latestBatchData), schema) + checkAnswer(df, data.map(_.toString).toDF("value")) + } + + /** Check how many batches have executed since the last time this check was made */ + var lastBatchId = -1L + def checkNumBatchesSinceLastCheck(numBatches: Int): Unit = { + require(lastBatchId >= 0) + assert(memorySink.latestBatchId.get === lastBatchId + numBatches) + lastBatchId = memorySink.latestBatchId.get + } + + checkLastBatchData(3) // (1 and 2) should be in batch 1, (3) should be in batch 2 (last) + lastBatchId = memorySink.latestBatchId.get + + fileSource.withBatchingLocked { + createFile(4) + createFile(5) // 4 and 5 should be in a batch + createFile(6) + createFile(7) // 6 and 7 should be in the last batch + } + q.processAllAvailable() + checkLastBatchData(6, 7) + checkNumBatchesSinceLastCheck(2) + + fileSource.withBatchingLocked { + createFile(8) + createFile(9) // 8 and 9 should be in a batch + createFile(10) + createFile(11) // 10 and 11 should be in a batch + createFile(12) // 12 should be in the last batch + } + q.processAllAvailable() + checkLastBatchData(12) + checkNumBatchesSinceLastCheck(3) + } + } + test("explain") { withTempDirs { case (src, tmp) => src.mkdirs() From 221a4a7fba6720246a1ada970137181af95abfd3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 8 Jul 2016 17:05:24 +0800 Subject: [PATCH 0922/1470] [SPARK-16285][SQL] Implement sentences SQL functions ## What changes were proposed in this pull request? This PR implements `sentences` SQL function. ## How was this patch tested? Pass the Jenkins tests with a new testcase. Author: Dongjoon Hyun Closes #14004 from dongjoon-hyun/SPARK_16285. (cherry picked from commit a54438cb23c80f7c7fc35da273677c39317cb1a5) Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/stringExpressions.scala | 68 ++++++++++++++++++- .../expressions/StringExpressionsSuite.scala | 23 +++++++ .../spark/sql/StringFunctionsSuite.scala | 20 ++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- 5 files changed, 111 insertions(+), 3 deletions(-) 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 f6ebcaeded484..842c9c63ce147 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 @@ -296,6 +296,7 @@ object FunctionRegistry { expression[RLike]("rlike"), expression[StringRPad]("rpad"), expression[StringTrimRight]("rtrim"), + expression[Sentences]("sentences"), expression[SoundEx]("soundex"), expression[StringSpace]("space"), expression[StringSplit]("split"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index b0df957637f1a..894e12d4a38ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.catalyst.expressions -import java.text.{DecimalFormat, DecimalFormatSymbols} +import java.text.{BreakIterator, DecimalFormat, DecimalFormatSymbols} import java.util.{HashMap, Locale, Map => JMap} +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{ByteArray, UTF8String} @@ -1188,3 +1190,65 @@ case class FormatNumber(x: Expression, d: Expression) override def prettyName: String = "format_number" } + +/** + * Splits a string into arrays of sentences, where each sentence is an array of words. + * The 'lang' and 'country' arguments are optional, and if omitted, the default locale is used. + */ +@ExpressionDescription( + usage = "_FUNC_(str[, lang, country]) - Splits str into an array of array of words.", + extended = "> SELECT _FUNC_('Hi there! Good morning.');\n [['Hi','there'], ['Good','morning']]") +case class Sentences( + str: Expression, + language: Expression = Literal(""), + country: Expression = Literal("")) + extends Expression with ImplicitCastInputTypes with CodegenFallback { + + def this(str: Expression) = this(str, Literal(""), Literal("")) + def this(str: Expression, language: Expression) = this(str, language, Literal("")) + + override def nullable: Boolean = true + override def dataType: DataType = + ArrayType(ArrayType(StringType, containsNull = false), containsNull = false) + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, StringType) + override def children: Seq[Expression] = str :: language :: country :: Nil + + override def eval(input: InternalRow): Any = { + val string = str.eval(input) + if (string == null) { + null + } else { + val languageStr = language.eval(input).asInstanceOf[UTF8String] + val countryStr = country.eval(input).asInstanceOf[UTF8String] + val locale = if (languageStr != null && countryStr != null) { + new Locale(languageStr.toString, countryStr.toString) + } else { + Locale.getDefault + } + getSentences(string.asInstanceOf[UTF8String].toString, locale) + } + } + + private def getSentences(sentences: String, locale: Locale) = { + val bi = BreakIterator.getSentenceInstance(locale) + bi.setText(sentences) + var idx = 0 + val result = new ArrayBuffer[GenericArrayData] + while (bi.next != BreakIterator.DONE) { + val sentence = sentences.substring(idx, bi.current) + idx = bi.current + + val wi = BreakIterator.getWordInstance(locale) + var widx = 0 + wi.setText(sentence) + val words = new ArrayBuffer[UTF8String] + while (wi.next != BreakIterator.DONE) { + val word = sentence.substring(widx, wi.current) + widx = wi.current + if (Character.isLetterOrDigit(word.charAt(0))) words += UTF8String.fromString(word) + } + result += new GenericArrayData(words) + } + new GenericArrayData(result) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 5f01561986f19..256ce85743c61 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -725,4 +725,27 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(FindInSet(Literal("abf"), Literal("abc,b,ab,c,def")), 0) checkEvaluation(FindInSet(Literal("ab,"), Literal("abc,b,ab,c,def")), 0) } + + test("Sentences") { + val nullString = Literal.create(null, StringType) + checkEvaluation(Sentences(nullString, nullString, nullString), null) + checkEvaluation(Sentences(nullString, nullString), null) + checkEvaluation(Sentences(nullString), null) + checkEvaluation(Sentences(Literal.create(null, NullType)), null) + checkEvaluation(Sentences("", nullString, nullString), Seq.empty) + checkEvaluation(Sentences("", nullString), Seq.empty) + checkEvaluation(Sentences(""), Seq.empty) + + val answer = Seq( + Seq("Hi", "there"), + Seq("The", "price", "was"), + Seq("But", "not", "now")) + + checkEvaluation(Sentences("Hi there! The price was $1,234.56.... But, not now."), answer) + checkEvaluation(Sentences("Hi there! The price was $1,234.56.... But, not now.", "en"), answer) + checkEvaluation(Sentences("Hi there! The price was $1,234.56.... But, not now.", "en", "US"), + answer) + checkEvaluation(Sentences("Hi there! The price was $1,234.56.... But, not now.", "XXX", "YYY"), + answer) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index dff4226051494..433a23bcb9422 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -347,4 +347,24 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { df2.filter("b>0").selectExpr("format_number(a, b)"), Row("5.0000") :: Row("4.000") :: Row("4.000") :: Row("4.000") :: Row("3.00") :: Nil) } + + test("string sentences function") { + val df = Seq(("Hi there! The price was $1,234.56.... But, not now.", "en", "US")) + .toDF("str", "language", "country") + + checkAnswer( + df.selectExpr("sentences(str, language, country)"), + Row(Seq(Seq("Hi", "there"), Seq("The", "price", "was"), Seq("But", "not", "now")))) + + // Type coercion + checkAnswer( + df.selectExpr("sentences(null)", "sentences(10)", "sentences(3.14)"), + Row(null, Seq(Seq("10")), Seq(Seq("3.14")))) + + // Argument number exception + val m = intercept[AnalysisException] { + df.selectExpr("sentences()") + }.getMessage + assert(m.contains("Invalid number of arguments for function sentences")) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index fdc4c18e70d69..6f05f0f3058cf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -236,7 +236,7 @@ private[sql] class HiveSessionCatalog( // str_to_map, windowingtablefunction. private val hiveFunctions = Seq( "hash", "java_method", "histogram_numeric", - "parse_url", "percentile", "percentile_approx", "reflect", "sentences", "str_to_map", + "parse_url", "percentile", "percentile_approx", "reflect", "str_to_map", "xpath", "xpath_double", "xpath_float", "xpath_int", "xpath_long", "xpath_number", "xpath_short", "xpath_string" ) From 8c8180605b0faa1f6223af9c4ffbbdb1c81486c4 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 8 Jul 2016 14:23:57 +0100 Subject: [PATCH 0923/1470] [SPARK-16369][MLLIB] tallSkinnyQR of RowMatrix should aware of empty partition ## What changes were proposed in this pull request? tallSkinnyQR of RowMatrix should aware of empty partition, which could cause exception from Breeze qr decomposition. See the [archived dev mail](https://mail-archives.apache.org/mod_mbox/spark-dev/201510.mbox/%3CCAF7ADNrycvPL3qX-VZJhq4OYmiUUhoscut_tkOm63Cm18iK1tQmail.gmail.com%3E) for more details. ## How was this patch tested? Scala unit test. Author: Xusen Yin Closes #14049 from yinxusen/SPARK-16369. (cherry picked from commit 255d74fe4a0db2cc842177ec735bbde07c7c8732) Signed-off-by: Sean Owen --- .../mllib/linalg/distributed/RowMatrix.scala | 5 +++-- .../linalg/distributed/RowMatrixSuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 1c94479ef02af..ec32e37afb792 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -537,7 +537,7 @@ class RowMatrix @Since("1.0.0") ( def tallSkinnyQR(computeQ: Boolean = false): QRDecomposition[RowMatrix, Matrix] = { val col = numCols().toInt // split rows horizontally into smaller matrices, and compute QR for each of them - val blockQRs = rows.retag(classOf[Vector]).glom().map { partRows => + val blockQRs = rows.retag(classOf[Vector]).glom().filter(_.length != 0).map { partRows => val bdm = BDM.zeros[Double](partRows.length, col) var i = 0 partRows.foreach { row => @@ -548,10 +548,11 @@ class RowMatrix @Since("1.0.0") ( } // combine the R part from previous results vertically into a tall matrix - val combinedR = blockQRs.treeReduce{ (r1, r2) => + val combinedR = blockQRs.treeReduce { (r1, r2) => val stackedR = BDM.vertcat(r1, r2) breeze.linalg.qr.reduced(stackedR).r } + val finalR = Matrices.fromBreeze(combinedR.toDenseMatrix) val finalQ = if (computeQ) { try { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 7c4c6d8409c6c..7c9e14f8cee70 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors} import org.apache.spark.mllib.random.RandomRDDs import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -281,6 +282,22 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(cov(i, j) === cov(j, i)) } } + + test("QR decomposition should aware of empty partition (SPARK-16369)") { + val mat: RowMatrix = new RowMatrix(sc.parallelize(denseData, 1)) + val qrResult = mat.tallSkinnyQR(true) + + val matWithEmptyPartition = new RowMatrix(sc.parallelize(denseData, 8)) + val qrResult2 = matWithEmptyPartition.tallSkinnyQR(true) + + assert(qrResult.Q.numCols() === qrResult2.Q.numCols(), "Q matrix ncol not match") + assert(qrResult.Q.numRows() === qrResult2.Q.numRows(), "Q matrix nrow not match") + qrResult.Q.rows.collect().zip(qrResult2.Q.rows.collect()) + .foreach(x => assert(x._1 ~== x._2 relTol 1E-8, "Q matrix not match")) + + qrResult.R.toArray.zip(qrResult2.R.toArray) + .foreach(x => assert(x._1 ~== x._2 relTol 1E-8, "R matrix not match")) + } } class RowMatrixClusterSuite extends SparkFunSuite with LocalClusterSparkContext { From 8dee2eca8eb819c8c76eda6606e110f6267dd75c Mon Sep 17 00:00:00 2001 From: Jurriaan Pruis Date: Fri, 8 Jul 2016 11:45:41 -0700 Subject: [PATCH 0924/1470] [SPARK-13638][SQL] Add quoteAll option to CSV DataFrameWriter ## What changes were proposed in this pull request? Adds an quoteAll option for writing CSV which will quote all fields. See https://issues.apache.org/jira/browse/SPARK-13638 ## How was this patch tested? Added a test to verify the output columns are quoted for all fields in the Dataframe Author: Jurriaan Pruis Closes #13374 from jurriaan/csv-quote-all. (cherry picked from commit 38cf8f2a50068f80350740ac28e31c8accd20634) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 7 +++-- .../apache/spark/sql/DataFrameWriter.scala | 2 ++ .../datasources/csv/CSVOptions.scala | 2 ++ .../execution/datasources/csv/CSVParser.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 26 +++++++++++++++++++ 5 files changed, 36 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 78d992e415489..f7c354f51330c 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -633,7 +633,7 @@ def text(self, path, compression=None): @since(2.0) def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, - header=None, nullValue=None, escapeQuotes=None): + header=None, nullValue=None, escapeQuotes=None, quoteAll=None): """Saves the content of the :class:`DataFrame` in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -658,6 +658,9 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No :param escapeQuotes: A flag indicating whether values containing quotes should always be enclosed in quotes. If None is set, it uses the default value ``true``, escaping all values containing a quote character. + :param quoteAll: A flag indicating whether all values should always be enclosed in + quotes. If None is set, it uses the default value ``false``, + only escaping values containing a quote character. :param header: writes the names of columns as the first line. If None is set, it uses the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses @@ -667,7 +670,7 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No """ self.mode(mode) self._set_opts(compression=compression, sep=sep, quote=quote, escape=escape, header=header, - nullValue=nullValue, escapeQuotes=escapeQuotes) + nullValue=nullValue, escapeQuotes=escapeQuotes, quoteAll=quoteAll) self._jwrite.csv(path) @since(1.5) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index f77af76d2bf3a..12b304623d30b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -537,6 +537,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
    18. `escapeQuotes` (default `true`): a flag indicating whether values containing * quotes should always be enclosed in quotes. Default is to escape all values containing * a quote character.
    19. + *
    20. `quoteAll` (default `false`): A flag indicating whether all values should always be + * enclosed in quotes. Default is to only escape values containing a quote character.
    21. *
    22. `header` (default `false`): writes the names of columns as the first line.
    23. *
    24. `nullValue` (default empty string): sets the string representation of a null value.
    25. *
    26. `compression` (default `null`): compression codec to use when saving to file. This can be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 581eda7e09a3e..22fb8163b1c0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -115,6 +115,8 @@ private[sql] class CSVOptions(@transient private val parameters: Map[String, Str val maxMalformedLogPerPartition = getInt("maxMalformedLogPerPartition", 10) + val quoteAll = getBool("quoteAll", false) + val inputBufferSize = 128 val isCommentSet = this.comment != '\u0000' diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index b06f12369dd0a..7929ebbd90f71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -73,7 +73,7 @@ private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten writerSettings.setNullValue(params.nullValue) writerSettings.setEmptyValue(params.nullValue) writerSettings.setSkipEmptyLines(true) - writerSettings.setQuoteAllFields(false) + writerSettings.setQuoteAllFields(params.quoteAll) writerSettings.setHeaders(headers: _*) writerSettings.setQuoteEscapingEnabled(params.escapeQuotes) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f170065132acd..311f1fa8d2aff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -366,6 +366,32 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } + test("save csv with quoteAll enabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escape", "\"") + .option("quoteAll", "true") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "\"test \"\"quote\"\"\",\"123\",\"it \"\"works\"\"!\",\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + test("save csv with quote escaping enabled") { withTempDir { dir => val csvDir = new File(dir, "csv").getCanonicalPath From 0e9333b275c8307a24bb6c7e8409ea48d4bac3d6 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Fri, 8 Jul 2016 12:37:26 -0700 Subject: [PATCH 0925/1470] [SPARK-16420] Ensure compression streams are closed. ## What changes were proposed in this pull request? This uses the try/finally pattern to ensure streams are closed after use. `UnsafeShuffleWriter` wasn't closing compression streams, causing them to leak resources until garbage collected. This was causing a problem with codecs that use off-heap memory. ## How was this patch tested? Current tests are sufficient. This should not change behavior. Author: Ryan Blue Closes #14093 from rdblue/SPARK-16420-unsafe-shuffle-writer-leak. (cherry picked from commit 67e085ef6dd62774095f3187844c091db1a6a72c) Signed-off-by: Reynold Xin --- .../network/util/LimitedInputStream.java | 23 +++++++++++++++++++ .../shuffle/sort/UnsafeShuffleWriter.java | 17 ++++++++++---- .../spark/broadcast/TorrentBroadcast.scala | 13 ++++++++--- .../serializer/GenericAvroSerializer.scala | 15 +++++++++--- 4 files changed, 57 insertions(+), 11 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java b/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java index 922c37a10efdd..e79eef0325897 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java @@ -48,11 +48,27 @@ * use this functionality in both a Guava 11 environment and a Guava >14 environment. */ public final class LimitedInputStream extends FilterInputStream { + private final boolean closeWrappedStream; private long left; private long mark = -1; public LimitedInputStream(InputStream in, long limit) { + this(in, limit, true); + } + + /** + * Create a LimitedInputStream that will read {@code limit} bytes from {@code in}. + *

      + * If {@code closeWrappedStream} is true, this will close {@code in} when it is closed. + * Otherwise, the stream is left open for reading its remaining content. + * + * @param in a {@link InputStream} to read from + * @param limit the number of bytes to read + * @param closeWrappedStream whether to close {@code in} when {@link #close} is called + */ + public LimitedInputStream(InputStream in, long limit, boolean closeWrappedStream) { super(in); + this.closeWrappedStream = closeWrappedStream; Preconditions.checkNotNull(in); Preconditions.checkArgument(limit >= 0, "limit must be non-negative"); left = limit; @@ -102,4 +118,11 @@ public LimitedInputStream(InputStream in, long limit) { left -= skipped; return skipped; } + + @Override + public void close() throws IOException { + if (closeWrappedStream) { + super.close(); + } + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index daa63d47e6aed..44e6aa73d9755 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -346,12 +346,19 @@ private long[] mergeSpillsWithFileStream( for (int i = 0; i < spills.length; i++) { final long partitionLengthInSpill = spills[i].partitionLengths[partition]; if (partitionLengthInSpill > 0) { - InputStream partitionInputStream = - new LimitedInputStream(spillInputStreams[i], partitionLengthInSpill); - if (compressionCodec != null) { - partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); + InputStream partitionInputStream = null; + boolean innerThrewException = true; + try { + partitionInputStream = + new LimitedInputStream(spillInputStreams[i], partitionLengthInSpill, false); + if (compressionCodec != null) { + partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); + } + ByteStreams.copy(partitionInputStream, mergedFileOutputStream); + innerThrewException = false; + } finally { + Closeables.close(partitionInputStream, innerThrewException); } - ByteStreams.copy(partitionInputStream, mergedFileOutputStream); } } mergedFileOutputStream.flush(); diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 632b0ae9c2c37..e8d6d587b4824 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -232,7 +232,11 @@ private object TorrentBroadcast extends Logging { val out = compressionCodec.map(c => c.compressedOutputStream(cbbos)).getOrElse(cbbos) val ser = serializer.newInstance() val serOut = ser.serializeStream(out) - serOut.writeObject[T](obj).close() + Utils.tryWithSafeFinally { + serOut.writeObject[T](obj) + } { + serOut.close() + } cbbos.toChunkedByteBuffer.getChunks() } @@ -246,8 +250,11 @@ private object TorrentBroadcast extends Logging { val in: InputStream = compressionCodec.map(c => c.compressedInputStream(is)).getOrElse(is) val ser = serializer.newInstance() val serIn = ser.deserializeStream(in) - val obj = serIn.readObject[T]() - serIn.close() + val obj = Utils.tryWithSafeFinally { + serIn.readObject[T]() + } { + serIn.close() + } obj } diff --git a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala index d17a7894fd8a8..f0ed41f6903f4 100644 --- a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala @@ -32,6 +32,7 @@ import org.apache.commons.io.IOUtils import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils /** * Custom serializer used for generic Avro records. If the user registers the schemas @@ -72,8 +73,11 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) def compress(schema: Schema): Array[Byte] = compressCache.getOrElseUpdate(schema, { val bos = new ByteArrayOutputStream() val out = codec.compressedOutputStream(bos) - out.write(schema.toString.getBytes(StandardCharsets.UTF_8)) - out.close() + Utils.tryWithSafeFinally { + out.write(schema.toString.getBytes(StandardCharsets.UTF_8)) + } { + out.close() + } bos.toByteArray }) @@ -86,7 +90,12 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) schemaBytes.array(), schemaBytes.arrayOffset() + schemaBytes.position(), schemaBytes.remaining()) - val bytes = IOUtils.toByteArray(codec.compressedInputStream(bis)) + val in = codec.compressedInputStream(bis) + val bytes = Utils.tryWithSafeFinally { + IOUtils.toByteArray(in) + } { + in.close() + } new Schema.Parser().parse(new String(bytes, StandardCharsets.UTF_8)) }) From e3424fd7716d0c3f6ce82acd200bda704e42d3eb Mon Sep 17 00:00:00 2001 From: wujian Date: Fri, 8 Jul 2016 14:38:05 -0700 Subject: [PATCH 0926/1470] [SPARK-16281][SQL] Implement parse_url SQL function ## What changes were proposed in this pull request? This PR adds parse_url SQL functions in order to remove Hive fallback. A new implementation of #13999 ## How was this patch tested? Pass the exist tests including new testcases. Author: wujian Closes #14008 from janplus/SPARK-16281. (cherry picked from commit f5fef69143b2a83bb8b168b7417e92659af0c72c) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/stringExpressions.scala | 150 ++++++++++++++++++ .../expressions/StringExpressionsSuite.scala | 51 ++++++ .../spark/sql/StringFunctionsSuite.scala | 15 ++ .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- 5 files changed, 218 insertions(+), 1 deletion(-) 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 842c9c63ce147..c8bbbf88532dc 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 @@ -288,6 +288,7 @@ object FunctionRegistry { expression[StringLPad]("lpad"), expression[StringTrimLeft]("ltrim"), expression[JsonTuple]("json_tuple"), + expression[ParseUrl]("parse_url"), expression[FormatString]("printf"), expression[RegExpExtract]("regexp_extract"), expression[RegExpReplace]("regexp_replace"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 894e12d4a38ed..61549c9a23685 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.catalyst.expressions +import java.net.{MalformedURLException, URL} import java.text.{BreakIterator, DecimalFormat, DecimalFormatSymbols} import java.util.{HashMap, Locale, Map => JMap} +import java.util.regex.Pattern import scala.collection.mutable.ArrayBuffer @@ -654,6 +656,154 @@ case class StringRPad(str: Expression, len: Expression, pad: Expression) override def prettyName: String = "rpad" } +object ParseUrl { + private val HOST = UTF8String.fromString("HOST") + private val PATH = UTF8String.fromString("PATH") + private val QUERY = UTF8String.fromString("QUERY") + private val REF = UTF8String.fromString("REF") + private val PROTOCOL = UTF8String.fromString("PROTOCOL") + private val FILE = UTF8String.fromString("FILE") + private val AUTHORITY = UTF8String.fromString("AUTHORITY") + private val USERINFO = UTF8String.fromString("USERINFO") + private val REGEXPREFIX = "(&|^)" + private val REGEXSUBFIX = "=([^&]*)" +} + +/** + * Extracts a part from a URL + */ +@ExpressionDescription( + usage = "_FUNC_(url, partToExtract[, key]) - extracts a part from a URL", + extended = """Parts: HOST, PATH, QUERY, REF, PROTOCOL, AUTHORITY, FILE, USERINFO. + Key specifies which query to extract. + Examples: + > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'HOST') + 'spark.apache.org' + > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY') + 'query=1' + > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY', 'query') + '1'""") +case class ParseUrl(children: Seq[Expression]) + extends Expression with ExpectsInputTypes with CodegenFallback { + + override def nullable: Boolean = true + override def inputTypes: Seq[DataType] = Seq.fill(children.size)(StringType) + override def dataType: DataType = StringType + override def prettyName: String = "parse_url" + + // If the url is a constant, cache the URL object so that we don't need to convert url + // from UTF8String to String to URL for every row. + @transient private lazy val cachedUrl = children(0) match { + case Literal(url: UTF8String, _) if url ne null => getUrl(url) + case _ => null + } + + // If the key is a constant, cache the Pattern object so that we don't need to convert key + // from UTF8String to String to StringBuilder to String to Pattern for every row. + @transient private lazy val cachedPattern = children(2) match { + case Literal(key: UTF8String, _) if key ne null => getPattern(key) + case _ => null + } + + // If the partToExtract is a constant, cache the Extract part function so that we don't need + // to check the partToExtract for every row. + @transient private lazy val cachedExtractPartFunc = children(1) match { + case Literal(part: UTF8String, _) => getExtractPartFunc(part) + case _ => null + } + + import ParseUrl._ + + override def checkInputDataTypes(): TypeCheckResult = { + if (children.size > 3 || children.size < 2) { + TypeCheckResult.TypeCheckFailure(s"$prettyName function requires two or three arguments") + } else { + super[ExpectsInputTypes].checkInputDataTypes() + } + } + + private def getPattern(key: UTF8String): Pattern = { + Pattern.compile(REGEXPREFIX + key.toString + REGEXSUBFIX) + } + + private def getUrl(url: UTF8String): URL = { + try { + new URL(url.toString) + } catch { + case e: MalformedURLException => null + } + } + + private def getExtractPartFunc(partToExtract: UTF8String): URL => String = { + partToExtract match { + case HOST => _.getHost + case PATH => _.getPath + case QUERY => _.getQuery + case REF => _.getRef + case PROTOCOL => _.getProtocol + case FILE => _.getFile + case AUTHORITY => _.getAuthority + case USERINFO => _.getUserInfo + case _ => (url: URL) => null + } + } + + private def extractValueFromQuery(query: UTF8String, pattern: Pattern): UTF8String = { + val m = pattern.matcher(query.toString) + if (m.find()) { + UTF8String.fromString(m.group(2)) + } else { + null + } + } + + private def extractFromUrl(url: URL, partToExtract: UTF8String): UTF8String = { + if (cachedExtractPartFunc ne null) { + UTF8String.fromString(cachedExtractPartFunc.apply(url)) + } else { + UTF8String.fromString(getExtractPartFunc(partToExtract).apply(url)) + } + } + + private def parseUrlWithoutKey(url: UTF8String, partToExtract: UTF8String): UTF8String = { + if (cachedUrl ne null) { + extractFromUrl(cachedUrl, partToExtract) + } else { + val currentUrl = getUrl(url) + if (currentUrl ne null) { + extractFromUrl(currentUrl, partToExtract) + } else { + null + } + } + } + + override def eval(input: InternalRow): Any = { + val evaluated = children.map{e => e.eval(input).asInstanceOf[UTF8String]} + if (evaluated.contains(null)) return null + if (evaluated.size == 2) { + parseUrlWithoutKey(evaluated(0), evaluated(1)) + } else { + // 3-arg, i.e. QUERY with key + assert(evaluated.size == 3) + if (evaluated(1) != QUERY) { + return null + } + + val query = parseUrlWithoutKey(evaluated(0), evaluated(1)) + if (query eq null) { + return null + } + + if (cachedPattern ne null) { + extractValueFromQuery(query, cachedPattern) + } else { + extractValueFromQuery(query, getPattern(evaluated(2))) + } + } + } +} + /** * Returns the input formatted according do printf-style format strings */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 256ce85743c61..8f7b1041fad32 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -726,6 +726,57 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(FindInSet(Literal("ab,"), Literal("abc,b,ab,c,def")), 0) } + test("ParseUrl") { + def checkParseUrl(expected: String, urlStr: String, partToExtract: String): Unit = { + checkEvaluation( + ParseUrl(Seq(Literal(urlStr), Literal(partToExtract))), expected) + } + def checkParseUrlWithKey( + expected: String, + urlStr: String, + partToExtract: String, + key: String): Unit = { + checkEvaluation( + ParseUrl(Seq(Literal(urlStr), Literal(partToExtract), Literal(key))), expected) + } + + checkParseUrl("spark.apache.org", "http://spark.apache.org/path?query=1", "HOST") + checkParseUrl("/path", "http://spark.apache.org/path?query=1", "PATH") + checkParseUrl("query=1", "http://spark.apache.org/path?query=1", "QUERY") + checkParseUrl("Ref", "http://spark.apache.org/path?query=1#Ref", "REF") + checkParseUrl("http", "http://spark.apache.org/path?query=1", "PROTOCOL") + checkParseUrl("/path?query=1", "http://spark.apache.org/path?query=1", "FILE") + checkParseUrl("spark.apache.org:8080", "http://spark.apache.org:8080/path?query=1", "AUTHORITY") + checkParseUrl("userinfo", "http://userinfo@spark.apache.org/path?query=1", "USERINFO") + checkParseUrlWithKey("1", "http://spark.apache.org/path?query=1", "QUERY", "query") + + // Null checking + checkParseUrl(null, null, "HOST") + checkParseUrl(null, "http://spark.apache.org/path?query=1", null) + checkParseUrl(null, null, null) + checkParseUrl(null, "test", "HOST") + checkParseUrl(null, "http://spark.apache.org/path?query=1", "NO") + checkParseUrl(null, "http://spark.apache.org/path?query=1", "USERINFO") + checkParseUrlWithKey(null, "http://spark.apache.org/path?query=1", "HOST", "query") + checkParseUrlWithKey(null, "http://spark.apache.org/path?query=1", "QUERY", "quer") + checkParseUrlWithKey(null, "http://spark.apache.org/path?query=1", "QUERY", null) + checkParseUrlWithKey(null, "http://spark.apache.org/path?query=1", "QUERY", "") + + // exceptional cases + intercept[java.util.regex.PatternSyntaxException] { + evaluate(ParseUrl(Seq(Literal("http://spark.apache.org/path?"), + Literal("QUERY"), Literal("???")))) + } + + // arguments checking + assert(ParseUrl(Seq(Literal("1"))).checkInputDataTypes().isFailure) + assert(ParseUrl(Seq(Literal("1"), Literal("2"), Literal("3"), Literal("4"))) + .checkInputDataTypes().isFailure) + assert(ParseUrl(Seq(Literal("1"), Literal(2))).checkInputDataTypes().isFailure) + assert(ParseUrl(Seq(Literal(1), Literal("2"))).checkInputDataTypes().isFailure) + assert(ParseUrl(Seq(Literal("1"), Literal("2"), Literal(3))).checkInputDataTypes().isFailure) + } + test("Sentences") { val nullString = Literal.create(null, StringType) checkEvaluation(Sentences(nullString, nullString, nullString), null) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 433a23bcb9422..cbe480b525641 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -226,6 +226,21 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("???hi", "hi???", "h", "h")) } + test("string parse_url function") { + val df = Seq[String](("http://userinfo@spark.apache.org/path?query=1#Ref")) + .toDF("url") + + checkAnswer( + df.selectExpr( + "parse_url(url, 'HOST')", "parse_url(url, 'PATH')", + "parse_url(url, 'QUERY')", "parse_url(url, 'REF')", + "parse_url(url, 'PROTOCOL')", "parse_url(url, 'FILE')", + "parse_url(url, 'AUTHORITY')", "parse_url(url, 'USERINFO')", + "parse_url(url, 'QUERY', 'query')"), + Row("spark.apache.org", "/path", "query=1", "Ref", + "http", "/path?query=1", "userinfo@spark.apache.org", "userinfo", "1")) + } + test("string repeat function") { val df = Seq(("hi", 2)).toDF("a", "b") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 6f05f0f3058cf..9c7f461362d84 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -236,7 +236,7 @@ private[sql] class HiveSessionCatalog( // str_to_map, windowingtablefunction. private val hiveFunctions = Seq( "hash", "java_method", "histogram_numeric", - "parse_url", "percentile", "percentile_approx", "reflect", "str_to_map", + "percentile", "percentile_approx", "reflect", "str_to_map", "xpath", "xpath_double", "xpath_float", "xpath_int", "xpath_long", "xpath_number", "xpath_short", "xpath_string" ) From 07f562f5881f1896a41077a367c31af704551d78 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 8 Jul 2016 15:56:46 -0700 Subject: [PATCH 0927/1470] [SPARK-16453][BUILD] release-build.sh is missing hive-thriftserver for scala 2.10 ## What changes were proposed in this pull request? This PR adds hive-thriftserver profile to scala 2.10 build created by release-build.sh. Author: Yin Huai Closes #14108 from yhuai/SPARK-16453. (cherry picked from commit 60ba436b7010436c77dfe5219a9662accc25bffa) Signed-off-by: Yin Huai --- dev/create-release/release-build.sh | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 65e80fc76056a..2833dc7651117 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -80,7 +80,7 @@ NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads BASE_DIR=$(pwd) MVN="build/mvn --force" -PUBLISH_PROFILES="-Pyarn -Phive -Phadoop-2.2" +PUBLISH_PROFILES="-Pyarn -Phive -Phive-thriftserver -Phadoop-2.2" PUBLISH_PROFILES="$PUBLISH_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl" rm -rf spark @@ -254,8 +254,7 @@ if [[ "$1" == "publish-snapshot" ]]; then # Generate random point for Zinc export ZINC_PORT=$(python -S -c "import random; print random.randrange(3030,4030)") - $MVN -DzincPort=$ZINC_PORT --settings $tmp_settings -DskipTests $PUBLISH_PROFILES \ - -Phive-thriftserver deploy + $MVN -DzincPort=$ZINC_PORT --settings $tmp_settings -DskipTests $PUBLISH_PROFILES deploy ./dev/change-scala-version.sh 2.10 $MVN -DzincPort=$ZINC_PORT -Dscala-2.10 --settings $tmp_settings \ -DskipTests $PUBLISH_PROFILES clean deploy @@ -291,8 +290,7 @@ if [[ "$1" == "publish-release" ]]; then # Generate random point for Zinc export ZINC_PORT=$(python -S -c "import random; print random.randrange(3030,4030)") - $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $PUBLISH_PROFILES \ - -Phive-thriftserver clean install + $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $PUBLISH_PROFILES clean install ./dev/change-scala-version.sh 2.10 From 463cbf72fd6db1d0646df432f56cd121b0eed625 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 8 Jul 2016 16:07:12 -0700 Subject: [PATCH 0928/1470] [SPARK-16387][SQL] JDBC Writer should use dialect to quote field names. ## What changes were proposed in this pull request? Currently, JDBC Writer uses dialects to get datatypes, but doesn't to quote field names. This PR uses dialects to quote the field names, too. **Reported Error Scenario (MySQL case)** ```scala scala> val url="jdbc:mysql://localhost:3306/temp" scala> val prop = new java.util.Properties scala> prop.setProperty("user","root") scala> spark.createDataset(Seq("a","b","c")).toDF("order") scala> df.write.mode("overwrite").jdbc(url, "temptable", prop) ...MySQLSyntaxErrorException: ... near 'order TEXT ) ``` ## How was this patch tested? Pass the Jenkins tests and manually do the above case. Author: Dongjoon Hyun Closes #14107 from dongjoon-hyun/SPARK-16387. (cherry picked from commit 3b22291b5f0317609cd71ce7af78e4c5063d66e8) Signed-off-by: Reynold Xin --- .../spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 9 +++++---- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 6 ++++++ 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 3529ee6e3b6ad..d3e1efc562777 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -100,8 +100,9 @@ object JdbcUtils extends Logging { /** * Returns a PreparedStatement that inserts a row into table via conn. */ - def insertStatement(conn: Connection, table: String, rddSchema: StructType): PreparedStatement = { - val columns = rddSchema.fields.map(_.name).mkString(",") + def insertStatement(conn: Connection, table: String, rddSchema: StructType, dialect: JdbcDialect) + : PreparedStatement = { + val columns = rddSchema.fields.map(x => dialect.quoteIdentifier(x.name)).mkString(",") val placeholders = rddSchema.fields.map(_ => "?").mkString(",") val sql = s"INSERT INTO $table ($columns) VALUES ($placeholders)" conn.prepareStatement(sql) @@ -177,7 +178,7 @@ object JdbcUtils extends Logging { if (supportsTransactions) { conn.setAutoCommit(false) // Everything in the same db transaction. } - val stmt = insertStatement(conn, table, rddSchema) + val stmt = insertStatement(conn, table, rddSchema, dialect) try { var rowCount = 0 while (iterator.hasNext) { @@ -260,7 +261,7 @@ object JdbcUtils extends Logging { val sb = new StringBuilder() val dialect = JdbcDialects.get(url) df.schema.fields foreach { field => - val name = field.name + val name = dialect.quoteIdentifier(field.name) val typ: String = getJdbcType(field.dataType, dialect).databaseTypeDefinition val nullable = if (field.nullable) "" else "NOT NULL" sb.append(s", $name $typ $nullable") 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 11e66ad08009c..228e4250f3c66 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 @@ -764,4 +764,10 @@ class JDBCSuite extends SparkFunSuite assertEmptyQuery(s"SELECT * FROM tempFrame where $FALSE2") } } + + test("SPARK-16387: Reserved SQL words are not escaped by JDBC writer") { + val df = spark.createDataset(Seq("a", "b", "c")).toDF("order") + val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") + assert(schema.contains("`order` TEXT")) + } } From c425230fdf1654aecaa84aba02b6844923c56d61 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 8 Jul 2016 17:47:58 -0700 Subject: [PATCH 0929/1470] [SPARK-13569][STREAMING][KAFKA] pattern based topic subscription ## What changes were proposed in this pull request? Allow for kafka topic subscriptions based on a regex pattern. ## How was this patch tested? Unit tests, manual tests Author: cody koeninger Closes #14026 from koeninger/SPARK-13569. (cherry picked from commit fd6e8f0e2269a2e7f24f79d5c2041816ea308c86) Signed-off-by: Tathagata Das --- .../streaming/kafka010/ConsumerStrategy.scala | 178 +++++++++++++++++- .../kafka010/JavaConsumerStrategySuite.java | 15 ++ .../kafka010/DirectKafkaStreamSuite.scala | 74 +++++++- 3 files changed, 258 insertions(+), 9 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index 70c3f1a98d97a..60255fc655e5f 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -22,10 +22,11 @@ import java.{ lang => jl, util => ju } import scala.collection.JavaConverters._ import org.apache.kafka.clients.consumer._ +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition import org.apache.spark.annotation.Experimental - +import org.apache.spark.internal.Logging /** * :: Experimental :: @@ -47,7 +48,9 @@ abstract class ConsumerStrategy[K, V] { /** * Must return a fully configured Kafka Consumer, including subscribed or assigned topics. + * See Kafka docs. * This consumer will be used on the driver to query for offsets only, not messages. + * The consumer must be returned in a state that it is safe to call poll(0) on. * @param currentOffsets A map from TopicPartition to offset, indicating how far the driver * has successfully read. Will be empty on initial start, possibly non-empty on restart from * checkpoint. @@ -72,15 +75,83 @@ private case class Subscribe[K, V]( topics: ju.Collection[jl.String], kafkaParams: ju.Map[String, Object], offsets: ju.Map[TopicPartition, jl.Long] - ) extends ConsumerStrategy[K, V] { + ) extends ConsumerStrategy[K, V] with Logging { def executorKafkaParams: ju.Map[String, Object] = kafkaParams def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { val consumer = new KafkaConsumer[K, V](kafkaParams) consumer.subscribe(topics) - if (currentOffsets.isEmpty) { - offsets.asScala.foreach { case (topicPartition, offset) => + val toSeek = if (currentOffsets.isEmpty) { + offsets + } else { + currentOffsets + } + if (!toSeek.isEmpty) { + // work around KAFKA-3370 when reset is none + // poll will throw if no position, i.e. auto offset reset none and no explicit position + // but cant seek to a position before poll, because poll is what gets subscription partitions + // So, poll, suppress the first exception, then seek + val aor = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + val shouldSuppress = aor != null && aor.asInstanceOf[String].toUpperCase == "NONE" + try { + consumer.poll(0) + } catch { + case x: NoOffsetForPartitionException if shouldSuppress => + logWarning("Catching NoOffsetForPartitionException since " + + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is none. See KAFKA-3370") + } + toSeek.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } + + consumer + } +} + +/** + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ +private case class SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long] + ) extends ConsumerStrategy[K, V] with Logging { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.subscribe(pattern, new NoOpConsumerRebalanceListener()) + val toSeek = if (currentOffsets.isEmpty) { + offsets + } else { + currentOffsets + } + if (!toSeek.isEmpty) { + // work around KAFKA-3370 when reset is none, see explanation in Subscribe above + val aor = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + val shouldSuppress = aor != null && aor.asInstanceOf[String].toUpperCase == "NONE" + try { + consumer.poll(0) + } catch { + case x: NoOffsetForPartitionException if shouldSuppress => + logWarning("Catching NoOffsetForPartitionException since " + + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is none. See KAFKA-3370") + } + toSeek.asScala.foreach { case (topicPartition, offset) => consumer.seek(topicPartition, offset) } } @@ -113,8 +184,14 @@ private case class Assign[K, V]( def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { val consumer = new KafkaConsumer[K, V](kafkaParams) consumer.assign(topicPartitions) - if (currentOffsets.isEmpty) { - offsets.asScala.foreach { case (topicPartition, offset) => + val toSeek = if (currentOffsets.isEmpty) { + offsets + } else { + currentOffsets + } + if (!toSeek.isEmpty) { + // this doesn't need a KAFKA-3370 workaround, because partitions are known, no poll needed + toSeek.asScala.foreach { case (topicPartition, offset) => consumer.seek(topicPartition, offset) } } @@ -215,6 +292,95 @@ object ConsumerStrategies { new Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, jl.Long]()) } + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V]( + pattern, + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) + } + + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V]( + pattern, + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V](pattern, kafkaParams, offsets) + } + + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V]( + pattern, + kafkaParams, + ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + /** * :: Experimental :: * Assign a fixed collection of TopicPartitions diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java index ac8d64b180f0d..ba57b6beb247d 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -19,6 +19,7 @@ import java.io.Serializable; import java.util.*; +import java.util.regex.Pattern; import scala.collection.JavaConverters; @@ -32,6 +33,7 @@ public class JavaConsumerStrategySuite implements Serializable { @Test public void testConsumerStrategyConstructors() { final String topic1 = "topic1"; + final Pattern pat = Pattern.compile("top.*"); final Collection topics = Arrays.asList(topic1); final scala.collection.Iterable sTopics = JavaConverters.collectionAsScalaIterableConverter(topics).asScala(); @@ -69,6 +71,19 @@ public Object apply(Long x) { sub1.executorKafkaParams().get("bootstrap.servers"), sub3.executorKafkaParams().get("bootstrap.servers")); + final ConsumerStrategy psub1 = + ConsumerStrategies.SubscribePattern(pat, sKafkaParams, sOffsets); + final ConsumerStrategy psub2 = + ConsumerStrategies.SubscribePattern(pat, sKafkaParams); + final ConsumerStrategy psub3 = + ConsumerStrategies.SubscribePattern(pat, kafkaParams, offsets); + final ConsumerStrategy psub4 = + ConsumerStrategies.SubscribePattern(pat, kafkaParams); + + Assert.assertEquals( + psub1.executorKafkaParams().get("bootstrap.servers"), + psub3.executorKafkaParams().get("bootstrap.servers")); + final ConsumerStrategy asn1 = ConsumerStrategies.Assign(sParts, sKafkaParams, sOffsets); final ConsumerStrategy asn2 = diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 0a53259802d1e..c9e15bcba0a9e 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -103,7 +103,9 @@ class DirectKafkaStreamSuite kafkaTestUtils.createTopic(t) kafkaTestUtils.sendMessages(t, data) } - val totalSent = data.values.sum * topics.size + val offsets = Map(new TopicPartition("basic3", 0) -> 2L) + // one topic is starting 2 messages later + val expectedTotal = (data.values.sum * topics.size) - 2 val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") ssc = new StreamingContext(sparkConf, Milliseconds(200)) @@ -111,7 +113,7 @@ class DirectKafkaStreamSuite KafkaUtils.createDirectStream[String, String]( ssc, preferredHosts, - ConsumerStrategies.Subscribe[String, String](topics, kafkaParams.asScala)) + ConsumerStrategies.Subscribe[String, String](topics, kafkaParams.asScala, offsets)) } val allReceived = new ConcurrentLinkedQueue[(String, String)]() @@ -149,13 +151,78 @@ class DirectKafkaStreamSuite } ssc.start() eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { - assert(allReceived.size === totalSent, + assert(allReceived.size === expectedTotal, "didn't get expected number of messages, messages:\n" + allReceived.asScala.mkString("\n")) } ssc.stop() } + test("pattern based subscription") { + val topics = List("pat1", "pat2", "advanced3") + // Should match 2 out of 3 topics + val pat = """pat\d""".r.pattern + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + val offsets = Map(new TopicPartition("pat2", 0) -> 3L) + // 2 matching topics, one of which starts 3 messages later + val expectedTotal = (data.values.sum * 2) - 3 + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.SubscribePattern[String, String](pat, kafkaParams.asScala, offsets)) + } + val allReceived = new ConcurrentLinkedQueue[(String, String)]() + + // hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + val tf = stream.transform { rdd => + // Get the offset ranges in the RDD + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd.map(r => (r.key, r.value)) + } + + tf.foreachRDD { rdd => + for (o <- offsetRanges) { + logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + val collected = rdd.mapPartitionsWithIndex { (i, iter) => + // For each partition, get size of the range in the partition, + // and the number of items in the partition + val off = offsetRanges(i) + val all = iter.toSeq + val partSize = all.size + val rangeSize = off.untilOffset - off.fromOffset + Iterator((partSize, rangeSize)) + }.collect + + // Verify whether number of elements in each partition + // matches with the corresponding offset range + collected.foreach { case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") + } + } + + stream.foreachRDD { rdd => + allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) + } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === expectedTotal, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + } + ssc.stop() + } + + test("receiving from largest starting offset") { val topic = "latest" val topicPartition = new TopicPartition(topic, 0) @@ -228,6 +295,7 @@ class DirectKafkaStreamSuite kc.close() // Setup context and kafka stream with largest offset + kafkaParams.put("auto.offset.reset", "none") ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { val s = new DirectKafkaInputDStream[String, String]( From 16202ba684eae8d200e063abfe154c3d1b8106a5 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 8 Jul 2016 20:17:50 -0700 Subject: [PATCH 0930/1470] [SPARK-16376][WEBUI][SPARK WEB UI][APP-ID] HTTP ERROR 500 when using rest api "/applications//jobs" if array "stageIds" is empty ## What changes were proposed in this pull request? Avoid error finding max of empty Seq when stageIds is empty. It does fix the immediate problem; I don't know if it results in meaningful output, but not an error at least. ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #14105 from srowen/SPARK-16376. (cherry picked from commit 6cef0183c0f0392dad78fec54635afdb9341b7f3) Signed-off-by: Reynold Xin --- .../org/apache/spark/status/api/v1/AllJobsResource.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala index 5783df5d8220c..b21d36d4a8d8a 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala @@ -68,7 +68,12 @@ private[v1] object AllJobsResource { listener: JobProgressListener, includeStageDetails: Boolean): JobData = { listener.synchronized { - val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageInfo = + if (job.stageIds.isEmpty) { + None + } else { + listener.stageIdToInfo.get(job.stageIds.max) + } val lastStageData = lastStageInfo.flatMap { s => listener.stageIdToData.get((s.stageId, s.attemptId)) } From 5024c4cb8f08019197670151d9bf9299e30586e4 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Fri, 8 Jul 2016 20:18:49 -0700 Subject: [PATCH 0931/1470] [SPARK-16432] Empty blocks fail to serialize due to assert in ChunkedByteBuffer ## What changes were proposed in this pull request? It's possible to also change the callers to not pass in empty chunks, but it seems cleaner to just allow `ChunkedByteBuffer` to handle empty arrays. cc JoshRosen ## How was this patch tested? Unit tests, also checked that the original reproduction case in https://github.com/apache/spark/pull/11748#issuecomment-230760283 is resolved. Author: Eric Liang Closes #14099 from ericl/spark-16432. (cherry picked from commit d8b06f18dc3e35938d15099beac98221d6f528b5) Signed-off-by: Reynold Xin --- .../org/apache/spark/util/io/ChunkedByteBuffer.scala | 9 ++++----- .../org/apache/spark/io/ChunkedByteBufferSuite.scala | 12 ++++-------- 2 files changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index fb4706e78d38f..89b0874e3865a 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -31,14 +31,13 @@ import org.apache.spark.storage.StorageUtils * Read-only byte buffer which is physically stored as multiple chunks rather than a single * contiguous array. * - * @param chunks an array of [[ByteBuffer]]s. Each buffer in this array must be non-empty and have - * position == 0. Ownership of these buffers is transferred to the ChunkedByteBuffer, - * so if these buffers may also be used elsewhere then the caller is responsible for - * copying them as needed. + * @param chunks an array of [[ByteBuffer]]s. Each buffer in this array must have position == 0. + * Ownership of these buffers is transferred to the ChunkedByteBuffer, so if these + * buffers may also be used elsewhere then the caller is responsible for copying + * them as needed. */ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { require(chunks != null, "chunks must not be null") - require(chunks.forall(_.limit() > 0), "chunks must be non-empty") require(chunks.forall(_.position() == 0), "chunks' positions must be 0") private[this] var disposed: Boolean = false diff --git a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala index f205d4f0d60b5..38b48a4c9e654 100644 --- a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala @@ -38,12 +38,6 @@ class ChunkedByteBufferSuite extends SparkFunSuite { emptyChunkedByteBuffer.toInputStream(dispose = true).close() } - test("chunks must be non-empty") { - intercept[IllegalArgumentException] { - new ChunkedByteBuffer(Array(ByteBuffer.allocate(0))) - } - } - test("getChunks() duplicates chunks") { val chunkedByteBuffer = new ChunkedByteBuffer(Array(ByteBuffer.allocate(8))) chunkedByteBuffer.getChunks().head.position(4) @@ -63,8 +57,9 @@ class ChunkedByteBufferSuite extends SparkFunSuite { } test("toArray()") { + val empty = ByteBuffer.wrap(Array[Byte]()) val bytes = ByteBuffer.wrap(Array.tabulate(8)(_.toByte)) - val chunkedByteBuffer = new ChunkedByteBuffer(Array(bytes, bytes)) + val chunkedByteBuffer = new ChunkedByteBuffer(Array(bytes, bytes, empty)) assert(chunkedByteBuffer.toArray === bytes.array() ++ bytes.array()) } @@ -79,9 +74,10 @@ class ChunkedByteBufferSuite extends SparkFunSuite { } test("toInputStream()") { + val empty = ByteBuffer.wrap(Array[Byte]()) val bytes1 = ByteBuffer.wrap(Array.tabulate(256)(_.toByte)) val bytes2 = ByteBuffer.wrap(Array.tabulate(128)(_.toByte)) - val chunkedByteBuffer = new ChunkedByteBuffer(Array(bytes1, bytes2)) + val chunkedByteBuffer = new ChunkedByteBuffer(Array(empty, bytes1, bytes2)) assert(chunkedByteBuffer.size === bytes1.limit() + bytes2.limit()) val inputStream = chunkedByteBuffer.toInputStream(dispose = false) From 50d7002b6aa95bad2a89f771f02e629ca7fc524f Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 8 Jul 2016 20:20:26 -0700 Subject: [PATCH 0932/1470] [SPARK-11857][MESOS] Deprecate fine grained ## What changes were proposed in this pull request? Documentation changes to indicate that fine-grained mode is now deprecated. No code changes were made, and all fine-grained mode instructions were left in place. We can remove all of that once the deprecation cycle completes (Does Spark have a standard deprecation cycle? One major version?) Blocked on https://github.com/apache/spark/pull/14059 ## How was this patch tested? Viewed in Github Author: Michael Gummelt Closes #14078 from mgummelt/deprecate-fine-grained. (cherry picked from commit b1db26acc51003e68e4e8d7d324cf74e3aa03cfd) Signed-off-by: Reynold Xin --- docs/running-on-mesos.md | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 8ab5f30220afc..5219e99fee73e 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -181,7 +181,7 @@ Note that jars or python files that are passed to spark-submit should be URIs re # Mesos Run Modes Spark can run over Mesos in two modes: "coarse-grained" (default) and -"fine-grained". +"fine-grained" (deprecated). ## Coarse-Grained @@ -213,7 +213,12 @@ the application. To configure your job to dynamically adjust to its resource requirements, look into [Dynamic Allocation](#dynamic-resource-allocation-with-mesos). -## Fine-Grained +## Fine-Grained (deprecated) + +**NOTE:** Fine-grained mode is deprecated as of Spark 2.0.0. Consider + using [Dynamic Allocation](#dynamic-resource-allocation-with-mesos) + for some of the benefits. For a full explanation see + [SPARK-11857](https://issues.apache.org/jira/browse/SPARK-11857) In "fine-grained" mode, each Spark task inside the Spark executor runs as a separate Mesos task. This allows multiple instances of Spark (and From a33643cbf0f8b68bde5bd6f9a706ee0f5be377f9 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 9 Jul 2016 20:35:45 +0800 Subject: [PATCH 0933/1470] [SPARK-16401][SQL] Data Source API: Enable Extending RelationProvider and CreatableRelationProvider without Extending SchemaRelationProvider #### What changes were proposed in this pull request? When users try to implement a data source API with extending only `RelationProvider` and `CreatableRelationProvider`, they will hit an error when resolving the relation. ```Scala spark.read .format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") .load() .write. format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") .save() ``` The error they hit is like ``` org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.; org.apache.spark.sql.AnalysisException: org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema does not allow user-specified schemas.; at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:319) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:494) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:211) ``` Actually, the bug fix is simple. [`DataSource.createRelation(sparkSession.sqlContext, mode, options, data)`](https://github.com/gatorsmile/spark/blob/dd644f8117e889cebd6caca58702a7c7e3d88bef/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L429) already returns a BaseRelation. We should not assign schema to `userSpecifiedSchema`. That schema assignment only makes sense for the data sources that extend `FileFormat`. #### How was this patch tested? Added a test case. Author: gatorsmile Closes #14075 from gatorsmile/dataSource. (cherry picked from commit 7374e518e2641fddfe57003340db410224b37581) Signed-off-by: Wenchen Fan --- .../execution/datasources/DataSource.scala | 5 ++- .../sql/test/DataFrameReaderWriterSuite.scala | 32 +++++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 6dc27c19521ea..f572b93991e0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -485,12 +485,11 @@ case class DataSource( data.logicalPlan, mode) sparkSession.sessionState.executePlan(plan).toRdd + // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring it. + copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() case _ => sys.error(s"${providingClass.getCanonicalName} does not allow create table as select.") } - - // We replace the schema with that of the DataFrame we just wrote out to avoid re-inferring it. - copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 7308f85a883d8..27a0a2a776c36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -82,6 +82,29 @@ class DefaultSource } } +/** Dummy provider with only RelationProvider and CreatableRelationProvider. */ +class DefaultSourceWithoutUserSpecifiedSchema + extends RelationProvider + with CreatableRelationProvider { + + case class FakeRelation(sqlContext: SQLContext) extends BaseRelation { + override def schema: StructType = StructType(Seq(StructField("a", StringType))) + } + + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + FakeRelation(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + FakeRelation(sqlContext) + } +} class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { @@ -120,6 +143,15 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be .save() } + test("resolve default source without extending SchemaRelationProvider") { + spark.read + .format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") + .load() + .write + .format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") + .save() + } + test("resolve full class") { spark.read .format("org.apache.spark.sql.test.DefaultSource") From 139d5eae038d846081e8c92518bdf7923d984afa Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 10 Jul 2016 22:05:16 -0700 Subject: [PATCH 0934/1470] [SPARK-16476] Restructure MimaExcludes for easier union excludes ## What changes were proposed in this pull request? It is currently fairly difficult to have proper mima excludes when we cut a version branch. I'm proposing a small change to take the exclude list out of the exclude function, and put it in a variable so we can easily union excludes. After this change, we can bump pom.xml version to 2.1.0-SNAPSHOT, without bumping the diff base version. Note that I also deleted all the exclude rules for version 1.x, to cut down the size of the file. ## How was this patch tested? N/A - this is a build infra change. Author: Reynold Xin Closes #14128 from rxin/SPARK-16476. (cherry picked from commit 52b5bb0b7fabe6cc949f514c548f9fbc6a4fa181) Signed-off-by: Reynold Xin --- project/MimaExcludes.scala | 2270 ++++++++++++------------------------ 1 file changed, 744 insertions(+), 1526 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a6209d78e168c..4bd6156288592 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -33,1538 +33,756 @@ import com.typesafe.tools.mima.core.ProblemFilters._ * For a new Spark version, please update MimaBuild.scala to reflect the previous version. */ object MimaExcludes { - def excludes(version: String) = version match { - case v if v.startsWith("2.0") => - Seq( - excludePackage("org.apache.spark.rpc"), - excludePackage("org.spark-project.jetty"), - excludePackage("org.apache.spark.unused"), - excludePackage("org.apache.spark.unsafe"), - excludePackage("org.apache.spark.memory"), - excludePackage("org.apache.spark.util.collection.unsafe"), - excludePackage("org.apache.spark.sql.catalyst"), - excludePackage("org.apache.spark.sql.execution"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.status.api.v1.ApplicationAttemptInfo.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.status.api.v1.ApplicationAttemptInfo.$default$5"), - // SPARK-14042 Add custom coalescer support - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.coalesce"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.rdd.PartitionCoalescer$LocationIterator"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.rdd.PartitionCoalescer"), - // SPARK-15532 Remove isRootContext flag from SQLContext. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.isRootContext"), - // SPARK-12600 Remove SQL deprecated methods - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$QueryExecution"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$SparkPlanner"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.applySchema"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.parquetFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jdbc"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.load"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.dialectClassName"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.getSQLDialect"), - // SPARK-13664 Replace HadoopFsRelation with FileFormat - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.LibSVMRelation"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelationProvider"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache"), - // SPARK-15543 Rename DefaultSources to make them more self-describing - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.DefaultSource") - ) ++ Seq( - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.SparkContext.emptyRDD"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.broadcast.HttpBroadcastFactory"), - // SPARK-14358 SparkListener from trait to abstract class - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.addSparkListener"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.JavaSparkListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.SparkFirehoseListener"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.scheduler.SparkListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.jobs.JobProgressListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.exec.ExecutorsListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.env.EnvironmentListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.storage.StorageListener"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.storage.StorageStatusListener") - ) ++ - Seq( - // SPARK-3369 Fix Iterable/Iterator in Java API - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.function.FlatMapFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.function.FlatMapFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.function.FlatMapFunction2.call"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.function.FlatMapFunction2.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.function.PairFlatMapFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.function.PairFlatMapFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.function.CoGroupFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.function.CoGroupFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.function.MapPartitionsFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.function.MapPartitionsFunction.call"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.function.FlatMapGroupsFunction.call"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.function.FlatMapGroupsFunction.call") - ) ++ - Seq( - // [SPARK-6429] Implement hashCode and equals together - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Partition.org$apache$spark$Partition$$super=uals") - ) ++ - Seq( - // SPARK-4819 replace Guava Optional - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getCheckpointDir"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getSparkHome"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner") - ) ++ - Seq( - // SPARK-12481 Remove Hadoop 1.x - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.mapred.SparkHadoopMapRedUtil"), - // SPARK-12615 Remove deprecated APIs in core - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.$default$6"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.numericRDDToDoubleRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intToIntWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.writableWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToAsyncRDDActions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.boolToBoolWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longToLongWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToOrderedRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.booleanWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringToText"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleToDoubleWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToSequenceFileRDDFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesToBytesWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringWritableConverter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatToFloatWritable"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions$default$4"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addOnCompleteCallback"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.runningLocally"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.attemptId"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.defaultMinSplits"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.runJob"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.runJob"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.tachyonFolderName"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.initLocalProperties"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearJars"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearFiles"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith$default$2"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.toArray"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith$default$2"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.filterWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.foreachWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit$default$2"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.SequenceFileRDDFunctions.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.splits"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.toArray"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.defaultMinSplits"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearJars"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearFiles"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.externalBlockStoreFolderName"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockManager"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore") - ) ++ Seq( - // SPARK-12149 Added new fields to ExecutorSummary - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") - ) ++ - // SPARK-12665 Remove deprecated and unused classes - Seq( - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.graphx.GraphKryoRegistrator"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$Multiplier"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$") - ) ++ Seq( - // SPARK-12591 Register OpenHashMapBasedStateMap for Kryo - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoInputDataInputBridge"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoOutputDataOutputBridge") - ) ++ Seq( - // SPARK-12510 Refactor ActorReceiver to support Java - ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") - ) ++ Seq( - // SPARK-12895 Implement TaskMetrics using accumulators - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.internalMetricsToAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectInternalAccumulators"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectAccumulators") - ) ++ Seq( - // SPARK-12896 Send only accumulator updates to driver, not TaskMetrics - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulable.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Accumulator.this"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.initialValue") - ) ++ Seq( - // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log__="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log__="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log__="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=") - ) ++ Seq( - // SPARK-12689 Migrate DDL parsing to the newly absorbed parser - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLParser"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLException"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.ddlParser") - ) ++ Seq( - // SPARK-7799 Add "streaming-akka" project - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$6"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$5"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$4"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$3"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.actorStream"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.streaming.zeromq.ZeroMQReceiver"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver$Supervisor") - ) ++ Seq( - // SPARK-12348 Remove deprecated Streaming APIs. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.dstream.DStream.foreach"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions$default$4"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.awaitTermination"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.networkStream"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.api.java.JavaStreamingContextFactory"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.awaitTermination"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.sc"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.reduceByWindow"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreachRDD"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreach"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.getOrCreate") - ) ++ Seq( - // SPARK-12847 Remove StreamingListenerBus and post all Streaming events to the same thread as Spark events - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus") - ) ++ Seq( - // SPARK-11622 Make LibSVMRelation extends HadoopFsRelation and Add LibSVMOutputWriter - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.source.libsvm.DefaultSource"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.source.libsvm.DefaultSource.createRelation") - ) ++ Seq( - // SPARK-6363 Make Scala 2.11 the default Scala version - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.cleanup"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metadataCleaner"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnDriverEndpoint"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint") - ) ++ Seq( - // SPARK-7889 - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.org$apache$spark$deploy$history$HistoryServer$@tachSparkUI"), - // SPARK-13296 - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.UDFRegistration.register"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction$") - ) ++ Seq( - // SPARK-12995 Remove deprecated APIs in graphx - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.lib.SVDPlusPlus.runSVDPlusPlus"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets$default$3"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.impl.GraphImpl.mapReduceTriplets") - ) ++ Seq( - // SPARK-13426 Remove the support of SIMR - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkMasterRegex.SIMR_REGEX") - ) ++ Seq( - // SPARK-13413 Remove SparkContext.metricsSystem/schedulerBackend_ setter - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metricsSystem"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.schedulerBackend_=") - ) ++ Seq( - // SPARK-13220 Deprecate yarn-client and yarn-cluster mode - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") - ) ++ Seq( - // SPARK-13465 TaskContext. - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addTaskFailureListener") - ) ++ Seq ( - // SPARK-7729 Executor which has been killed should also be displayed on Executor Tab - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") - ) ++ Seq( - // SPARK-13526 Move SQLContext per-session states to new class - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.sql.UDFRegistration.this") - ) ++ Seq( - // [SPARK-13486][SQL] Move SQLConf into an internal package - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry$") - ) ++ Seq( - //SPARK-11011 UserDefinedType serialization should be strongly typed - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.serialize"), - // SPARK-12073: backpressure rate controller consumes events preferentially from lagging partitions - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.KafkaTestUtils.createTopic"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.DirectKafkaInputDStream.maxMessagesPerPartition") - ) ++ Seq( - // [SPARK-13244][SQL] Migrates DataFrame to Dataset - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.tables"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.sql"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.baseRelationToDataFrame"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.table"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrame.apply"), - - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.LegacyFunctions"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.localSeqToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.stringRddToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.rddToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.longRddToDataFrameHolder"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.intRddToDataFrameHolder"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.GroupedDataset"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.subtract"), - - // [SPARK-14451][SQL] Move encoder definition into Aggregator interface - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.toColumn"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.bufferEncoder"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.outputEncoder"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MultilabelMetrics.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions") - ) ++ Seq( - // [SPARK-13686][MLLIB][STREAMING] Add a constructor parameter `reqParam` to (Streaming)LinearRegressionWithSGD - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LinearRegressionWithSGD.this") - ) ++ Seq( - // SPARK-15250 Remove deprecated json API in DataFrameReader - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameReader.json") - ) ++ Seq( - // SPARK-13920: MIMA checks should apply to @Experimental and @DeveloperAPI APIs - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineCombinersByKey"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineValuesByKey"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.run"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.runJob"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.actorSystem"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.cacheManager"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getConfigurationFromJobContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTaskAttemptIDFromTaskAttemptContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.newConfiguration"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback_="), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.canEqual"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productArity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productElement"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productIterator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productPrefix"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.setBytesReadCallback"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.updateBytesRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.canEqual"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productArity"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productElement"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productIterator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productPrefix"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decFetchWaitTime"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decLocalBlocksFetched"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRecordsRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBlocksFetched"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBytesRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleBytesWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleRecordsWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleWriteTime"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleBytesWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleRecordsWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleWriteTime"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.setShuffleRecordsWritten"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.PCAModel.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.AccumulableInfo.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.taskMetrics"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.TaskInfo.attempt"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.ExperimentalMethods.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUDF"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUdf"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.cumeDist"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.denseRank"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.inputFileName"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.isNaN"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.percentRank"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.rowNumber"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.sparkPartitionId"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.externalBlockStoreSize"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsedByRdd"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatusListener.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.scheduler.BatchInfo.streamIdToNumRecords"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.storageStatusList"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.storage.StorageListener.storageStatusList"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.apply"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.copy"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.InputMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.OutputMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Pipeline.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PipelineModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transformImpl"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.extractLabeledPoints"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Transformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.GBTClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayes.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRest.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRestModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.RandomForestClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeans.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.computeCost"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logLikelihood"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logPerplexity"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.RegressionEvaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Binarizer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Bucketizer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelector.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.HashingTF.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDF.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDFModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IndexToString.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Interaction.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScaler.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.OneHotEncoder.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCA.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCAModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormula.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormulaModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.SQLTransformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScaler.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StopWordsRemover.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorAssembler.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexer.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorSlicer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2Vec.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALS.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.GBTRegressor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.extractWeightedLabeledPoints"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.extractWeightedLabeledPoints"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegression.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionTrainingSummary.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidator.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidatorModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplit.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplitModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.BinaryClassificationMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.RegressionMetrics.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameWriter.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.broadcast"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.callUDF"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.InsertableRelation.insert"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.fMeasureByThreshold"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.pr"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.precisionByThreshold"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.predictions"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.recallByThreshold"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.roc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.describeTopics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.getVectors"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.itemFactors"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.userFactors"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.predictions"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.residuals"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.name"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.value"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.drop"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.fill"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.replace"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.jdbc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.json"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.load"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.orc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.parquet"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.table"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.text"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.crosstab"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.freqItems"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.sampleBy"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.emptyDataFrame"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.range"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.functions.udf"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.JobLogger"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorHelper"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics$"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics$"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics$"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.functions$"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Estimator.fit"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Predictor.train"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Transformer.transform"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListener.onOtherEvent"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.InsertableRelation.insert") - ) ++ Seq( - // [SPARK-13926] Automatically use Kryo serializer when shuffling RDDs with simple types - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ShuffleDependency.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ShuffleDependency.serializer"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.Serializer$") - ) ++ Seq( - // SPARK-13927: add row/column iterator to local matrices - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.rowIter"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.colIter") - ) ++ Seq( - // SPARK-13948: MiMa Check should catch if the visibility change to `private` - // TODO(josh): Some of these may be legitimate incompatibilities; we should follow up before the 2.0.0 release - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.toDS"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.askTimeout"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.lookupTimeout"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.UnaryTransformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassifier.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegression.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressor.train"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.select"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.toDF"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Logging.initializeLogIfNecessary"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerEvent.logEvent"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance") - ) ++ Seq( - // [SPARK-14014] Replace existing analysis.Catalog with SessionCatalog - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.this") - ) ++ Seq( - // [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Logging"), - (problem: Problem) => problem match { - case MissingTypesProblem(_, missing) - if missing.map(_.fullName).sameElements(Seq("org.apache.spark.Logging")) => false - case _ => true - } - ) ++ Seq( - // [SPARK-13990] Automatically pick serializer when caching RDDs - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.uploadBlock") - ) ++ Seq( - // [SPARK-14089][CORE][MLLIB] Remove methods that has been deprecated since 1.1, 1.2, 1.3, 1.4, and 1.5 - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.getThreadLocal"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeReduce"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeAggregate"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.tree.configuration.Strategy.defaultStategy"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.saveLabeledData"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLabeledData"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.optimization.LBFGS.setMaxNumIterations"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.setScoreCol") - ) ++ Seq( - // [SPARK-14205][SQL] remove trait Queryable - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.Dataset") - ) ++ Seq( - // [SPARK-11262][ML] Unit test for gradient, loss layers, memory management - // for multilayer perceptron. - // This class is marked as `private`. - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.ann.SoftmaxFunction") - ) ++ Seq( - // [SPARK-13674][SQL] Add wholestage codegen support to Sample - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.util.random.PoissonSampler.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.random.PoissonSampler.this") - ) ++ Seq( - // [SPARK-13430][ML] moved featureCol from LinearRegressionModelSummary to LinearRegressionSummary - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this") - ) ++ Seq( - // [SPARK-14437][Core] Use the address that NettyBlockTransferService listens to create BlockManagerId - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.this") - ) ++ Seq( - // [SPARK-13048][ML][MLLIB] keepLastCheckpoint option for LDA EM optimizer - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.DistributedLDAModel.this") - ) ++ Seq( - // [SPARK-14475] Propagate user-defined context from driver to executors - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getLocalProperty"), - // [SPARK-14617] Remove deprecated APIs in TaskMetrics - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.InputMetrics$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.OutputMetrics$"), - // [SPARK-14628] Simplify task metrics by always tracking read/write metrics - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.readMethod"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.writeMethod") - ) ++ Seq( - // SPARK-14628: Always track input/output/shuffle metrics - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.totalBlocksFetched"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.inputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.outputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleWriteMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleReadMetrics"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.inputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.outputMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleWriteMetrics"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleReadMetrics"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.this") - ) ++ Seq( - // SPARK-13643: Move functionality from SQLContext to SparkSession - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.getSchema") - ) ++ Seq( - // [SPARK-14407] Hides HadoopFsRelation related data source API into execution package - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriter"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriterFactory") - ) ++ Seq( - // SPARK-14734: Add conversions between mllib and ml Vector, Matrix types - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asML"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asML") - ) ++ Seq( - // SPARK-14704: Create accumulators in TaskMetrics - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.this") - ) ++ Seq( - // SPARK-14861: Replace internal usages of SQLContext with SparkSession - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.clustering.LocalLDAModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.clustering.DistributedLDAModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.clustering.LDAModel.this"), - ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.ml.clustering.LDAModel.sqlContext"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.sql.Dataset.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.sql.DataFrameReader.this") - ) ++ Seq( - // SPARK-14542 configurable buffer size for pipe RDD - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.pipe"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.pipe") - ) ++ Seq( - // [SPARK-4452][Core]Shuffle data structures can starve others on the same thread for memory - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.util.collection.Spillable") - ) ++ Seq( - // [SPARK-14952][Core][ML] Remove methods deprecated in 1.6 - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.input.PortableDataStream.close"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.weights"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionModel.weights") - ) ++ Seq( - // [SPARK-10653] [Core] Remove unnecessary things from SparkEnv - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.sparkFilesDir"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.blockTransferService") - ) ++ Seq( - // SPARK-14654: New accumulator API - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ExceptionFailure$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.metrics"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.remoteBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.totalBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") - ) ++ Seq( - // [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.getOldDocConcentration"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.estimatedDocConcentration"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.topicsMatrix"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.clusterCenters"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.decodeLabel"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.encodeLabeledPoint"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.weights"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.predict"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.predictRaw"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.raw2probabilityInPlace"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.theta"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.pi"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.probability2prediction"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predictRaw"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2prediction"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2probabilityInPlace"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predict"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.coefficients"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.raw2prediction"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.getScalingVec"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.setScalingVec"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.PCAModel.pc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMax"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMin"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.IDFModel.idf"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.mean"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.std"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predict"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.coefficients"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predictQuantiles"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.predictions"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.boundaries"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.predict"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.coefficients"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.this") - ) ++ Seq( - // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Private"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.AlphaComponent"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Experimental"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.DeveloperApi") - ) ++ Seq( - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asBreeze"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asBreeze") - ) ++ Seq( - // [SPARK-15914] Binary compatibility is broken since consolidation of Dataset and DataFrame - // in Spark 2.0. However, source level compatibility is still maintained. - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.load"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonFile"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jdbc"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema") - ) - case v if v.startsWith("1.6") => - Seq( - MimaBuild.excludeSparkPackage("deploy"), - MimaBuild.excludeSparkPackage("network"), - MimaBuild.excludeSparkPackage("unsafe"), - // These are needed if checking against the sbt build, since they are part of - // the maven-generated artifacts in 1.3. - excludePackage("org.spark-project.jetty"), - MimaBuild.excludeSparkPackage("unused"), - // SQL execution is considered private. - excludePackage("org.apache.spark.sql.execution"), - // SQL columnar is considered private. - excludePackage("org.apache.spark.sql.columnar"), - // The shuffle package is considered private. - excludePackage("org.apache.spark.shuffle"), - // The collections utilities are considered private. - excludePackage("org.apache.spark.util.collection") - ) ++ - MimaBuild.excludeSparkClass("streaming.flume.FlumeTestUtils") ++ - MimaBuild.excludeSparkClass("streaming.flume.PollingFlumeTestUtils") ++ - Seq( - // MiMa does not deal properly with sealed traits - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.ml.classification.LogisticRegressionSummary.featuresCol") - ) ++ Seq( - // SPARK-11530 - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this") - ) ++ Seq( - // SPARK-10381 Fix types / units in private AskPermissionToCommitOutput RPC message. - // This class is marked as `private` but MiMa still seems to be confused by the change. - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.task"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.copy$default$2"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.copy"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.taskAttempt"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.copy$default$3"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.apply") - ) ++ Seq( - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.shuffle.FileShuffleBlockResolver$ShuffleFileGroup") - ) ++ Seq( - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.ml.regression.LeastSquaresAggregator.add"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.ml.regression.LeastSquaresCostFun.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.clearLastInstantiatedContext"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.setLastInstantiatedContext"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.SQLContext$SQLSession"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.detachSession"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.tlSession"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.defaultSession"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.currentSession"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.openSession"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.setSession"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.createSession") - ) ++ Seq( - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.SparkContext.preferredNodeLocationData_="), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.rdd.MapPartitionsWithPreparationRDD"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.rdd.MapPartitionsWithPreparationRDD$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SparkSQLParser") - ) ++ Seq( - // SPARK-11485 - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.DataFrameHolder.df"), - // SPARK-11541 mark various JDBC dialects as private - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.NoopDialect.productElement"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.NoopDialect.productArity"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.NoopDialect.canEqual"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.NoopDialect.productIterator"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.NoopDialect.productPrefix"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.NoopDialect.toString"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.NoopDialect.hashCode"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.jdbc.PostgresDialect$"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.PostgresDialect.productElement"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.PostgresDialect.productArity"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.PostgresDialect.canEqual"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.PostgresDialect.productIterator"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.PostgresDialect.productPrefix"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.PostgresDialect.toString"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.jdbc.PostgresDialect.hashCode"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.jdbc.NoopDialect$") - ) ++ Seq ( - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.status.api.v1.ApplicationInfo.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.status.api.v1.StageData.this") - ) ++ Seq( - // SPARK-11766 add toJson to Vector - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Vector.toJson") - ) ++ Seq( - // SPARK-9065 Support message handler in Kafka Python API - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper.createDirectStream"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper.createRDD") - ) ++ Seq( - // SPARK-4557 Changed foreachRDD to use VoidFunction - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.api.java.JavaDStreamLike.foreachRDD") - ) ++ Seq( - // SPARK-11996 Make the executor thread dump work again - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.ExecutorEndpoint"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.ExecutorEndpoint$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.storage.BlockManagerMessages$GetRpcHostPortForExecutor"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.storage.BlockManagerMessages$GetRpcHostPortForExecutor$") - ) ++ Seq( - // SPARK-3580 Add getNumPartitions method to JavaRDD - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.getNumPartitions") - ) ++ Seq( - // SPARK-12149 Added new fields to ExecutorSummary - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") - ) ++ - // SPARK-11314: YARN backend moved to yarn sub-module and MiMA complains even though it's a - // private class. - MimaBuild.excludeSparkClass("scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint") - case v if v.startsWith("1.5") => - Seq( - MimaBuild.excludeSparkPackage("network"), - MimaBuild.excludeSparkPackage("deploy"), - // These are needed if checking against the sbt build, since they are part of - // the maven-generated artifacts in 1.3. - excludePackage("org.spark-project.jetty"), - MimaBuild.excludeSparkPackage("unused"), - // JavaRDDLike is not meant to be extended by user programs - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.partitioner"), - // Modification of private static method - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.streaming.kafka.KafkaUtils.org$apache$spark$streaming$kafka$KafkaUtils$$leadersForRanges"), - // Mima false positive (was a private[spark] class) - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.util.collection.PairIterator"), - // Removing a testing method from a private class - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.kafka.KafkaTestUtils.waitUntilLeaderOffset"), - // While private MiMa is still not happy about the changes, - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.ml.regression.LeastSquaresAggregator.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.ml.regression.LeastSquaresCostFun.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.ml.classification.LogisticCostFun.this"), - // SQL execution is considered private. - excludePackage("org.apache.spark.sql.execution"), - // The old JSON RDD is removed in favor of streaming Jackson - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.json.JsonRDD$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.json.JsonRDD"), - // local function inside a method - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.org$apache$spark$sql$SQLContext$$needsConversion$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.UDFRegistration.org$apache$spark$sql$UDFRegistration$$builder$24") - ) ++ Seq( - // SPARK-8479 Add numNonzeros and numActives to Matrix. - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrix.numNonzeros"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrix.numActives") - ) ++ Seq( - // SPARK-8914 Remove RDDApi - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.RDDApi") - ) ++ Seq( - // SPARK-7292 Provide operator to truncate lineage cheaply - ProblemFilters.exclude[AbstractClassProblem]( - "org.apache.spark.rdd.RDDCheckpointData"), - ProblemFilters.exclude[AbstractClassProblem]( - "org.apache.spark.rdd.CheckpointRDD") - ) ++ Seq( - // SPARK-8701 Add input metadata in the batch page. - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.streaming.scheduler.InputInfo$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.streaming.scheduler.InputInfo") - ) ++ Seq( - // SPARK-6797 Support YARN modes for SparkR - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.PairwiseRRDD.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.RRDD.createRWorker"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.RRDD.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.StringRRDD.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.BaseRRDD.this") - ) ++ Seq( - // SPARK-7422 add argmax for sparse vectors - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Vector.argmax") - ) ++ Seq( - // SPARK-8906 Move all internal data source classes into execution.datasources - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.ResolvedDataSource"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PreInsertCastAndRename$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTableUsingAsSelect$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.InsertIntoDataSource$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PartitioningUtils$PartitionValues$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DefaultWriterContainer"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PartitioningUtils$PartitionValues"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.RefreshTable$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTempTableUsing$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PartitionSpec"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DynamicPartitionWriterContainer"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTableUsingAsSelect"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DescribeCommand$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PartitioningUtils$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PreInsertCastAndRename"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Partition$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LogicalRelation$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PartitioningUtils"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.LogicalRelation"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.Partition"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.BaseWriterContainer"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PreWriteCheck"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTableUsing"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.RefreshTable"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DataSourceStrategy$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTempTableUsing"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTempTableUsingAsSelect$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTempTableUsingAsSelect"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CreateTableUsing$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.ResolvedDataSource$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PreWriteCheck$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.InsertIntoDataSource"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.InsertIntoHadoopFsRelation"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DDLParser"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.CaseInsensitiveMap"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.InsertIntoHadoopFsRelation$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DataSourceStrategy"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.PartitionSpec$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DescribeCommand"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.DDLException"), - // SPARK-9763 Minimize exposure of internal SQL classes - excludePackage("org.apache.spark.sql.parquet"), - excludePackage("org.apache.spark.sql.json"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCRDD$DecimalConversion$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCPartition"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JdbcUtils$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCRDD$DecimalConversion"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCPartitioningInfo$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCPartition$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.package"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCRDD$JDBCConversion"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCRDD$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.package$DriverWrapper"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCRDD"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCPartitioningInfo"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JdbcUtils"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.DefaultSource"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCRelation$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.package$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.JDBCRelation") - ) ++ Seq( - // SPARK-4751 Dynamic allocation for standalone mode - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.SparkContext.supportDynamicAllocation") - ) ++ Seq( - // SPARK-9580: Remove SQL test singletons - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.test.LocalSQLContext$SQLSession"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.test.LocalSQLContext"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.test.TestSQLContext"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.test.TestSQLContext$") - ) ++ Seq( - // SPARK-9704 Made ProbabilisticClassifier, Identifiable, VectorUDT public APIs - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.mllib.linalg.VectorUDT.serialize") - ) ++ Seq( - // SPARK-10381 Fix types / units in private AskPermissionToCommitOutput RPC message. - // This class is marked as `private` but MiMa still seems to be confused by the change. - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.task"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.copy$default$2"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.copy"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.taskAttempt"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.copy$default$3"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.scheduler.AskPermissionToCommitOutput.apply") - ) + // Exclude rules for 2.1.x + lazy val v21excludes = v20excludes - case v if v.startsWith("1.4") => - Seq( - MimaBuild.excludeSparkPackage("deploy"), - MimaBuild.excludeSparkPackage("ml"), - // SPARK-7910 Adding a method to get the partitioner to JavaRDD, - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner"), - // SPARK-5922 Adding a generalized diff(other: RDD[(VertexId, VD)]) to VertexRDD - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.diff"), - // These are needed if checking against the sbt build, since they are part of - // the maven-generated artifacts in 1.3. - excludePackage("org.spark-project.jetty"), - MimaBuild.excludeSparkPackage("unused"), - ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.rdd.JdbcRDD.compute"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.scheduler.OutputCommitCoordinator$OutputCommitCoordinatorEndpoint") - ) ++ Seq( - // SPARK-4655 - Making Stage an Abstract class broke binary compatibility even though - // the stage class is defined as private[spark] - ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.scheduler.Stage") - ) ++ Seq( - // SPARK-6510 Add a Graph#minus method acting as Set#difference - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.minus") - ) ++ Seq( - // 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") - )++ Seq( - // SPARK-6703 Add getOrCreate method to SparkContext - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.SparkContext.org$apache$spark$SparkContext$$activeContext") - )++ Seq( - // SPARK-7090 Introduce LDAOptimizer to LDA to further improve extensibility - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.mllib.clustering.LDA$EMOptimizer") - ) ++ Seq( - // SPARK-6756 add toSparse, toDense, numActives, numNonzeros, and compressed to Vector - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Vector.compressed"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Vector.toDense"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Vector.numNonzeros"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Vector.toSparse"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Vector.numActives"), - // SPARK-7681 add SparseVector support for gemv - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrix.multiply"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.DenseMatrix.multiply"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.SparseMatrix.multiply") - ) ++ Seq( - // Execution should never be included as its always internal. - MimaBuild.excludeSparkPackage("sql.execution"), - // This `protected[sql]` method was removed in 1.3.1 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.sql.SQLContext.checkAnalysis"), - // These `private[sql]` class were removed in 1.4.0: - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.execution.AddExchange"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.execution.AddExchange$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.PartitionSpec"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.PartitionSpec$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.Partition"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.Partition$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetRelation2$PartitionValues"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetRelation2$PartitionValues$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetRelation2"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetRelation2$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetRelation2$MetadataCache"), - // These test support classes were moved out of src/main and into src/test: - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetTestData"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetTestData$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.TestGroupWriteSupport"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.CachedData"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.CachedData$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.CacheManager"), - // TODO: Remove the following rule once ParquetTest has been moved to src/test. - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetTest") - ) ++ Seq( - // SPARK-7530 Added StreamingContext.getState() - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.StreamingContext.state_=") - ) ++ Seq( - // SPARK-7081 changed ShuffleWriter from a trait to an abstract class and removed some - // unnecessary type bounds in order to fix some compiler warnings that occurred when - // implementing this interface in Java. Note that ShuffleWriter is private[spark]. - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.shuffle.ShuffleWriter") - ) ++ Seq( - // SPARK-6888 make jdbc driver handling user definable - // This patch renames some classes to API friendly names. - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.DriverQuirks$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.DriverQuirks"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.PostgresQuirks"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.NoQuirks"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.jdbc.MySQLQuirks") - ) + // Exclude rules for 2.0.x + lazy val v20excludes = { + Seq( + excludePackage("org.apache.spark.rpc"), + excludePackage("org.spark-project.jetty"), + excludePackage("org.apache.spark.unused"), + excludePackage("org.apache.spark.unsafe"), + excludePackage("org.apache.spark.memory"), + excludePackage("org.apache.spark.util.collection.unsafe"), + excludePackage("org.apache.spark.sql.catalyst"), + excludePackage("org.apache.spark.sql.execution"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.status.api.v1.ApplicationAttemptInfo.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.status.api.v1.ApplicationAttemptInfo.$default$5"), + // SPARK-14042 Add custom coalescer support + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.coalesce"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.rdd.PartitionCoalescer$LocationIterator"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.rdd.PartitionCoalescer"), + // SPARK-15532 Remove isRootContext flag from SQLContext. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.isRootContext"), + // SPARK-12600 Remove SQL deprecated methods + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$QueryExecution"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$SparkPlanner"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.applySchema"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.parquetFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jdbc"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.load"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.dialectClassName"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.getSQLDialect"), + // SPARK-13664 Replace HadoopFsRelation with FileFormat + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.LibSVMRelation"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelationProvider"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.HadoopFsRelation$FileStatusCache"), + // SPARK-15543 Rename DefaultSources to make them more self-describing + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.source.libsvm.DefaultSource") + ) ++ Seq( + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.SparkContext.emptyRDD"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.broadcast.HttpBroadcastFactory"), + // SPARK-14358 SparkListener from trait to abstract class + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.addSparkListener"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.JavaSparkListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.SparkFirehoseListener"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.scheduler.SparkListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.jobs.JobProgressListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.exec.ExecutorsListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.env.EnvironmentListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ui.storage.StorageListener"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.storage.StorageStatusListener") + ) ++ + Seq( + // SPARK-3369 Fix Iterable/Iterator in Java API + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.DoubleFlatMapFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapFunction2.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapFunction2.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.PairFlatMapFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.PairFlatMapFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.CoGroupFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.CoGroupFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.MapPartitionsFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.MapPartitionsFunction.call"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.function.FlatMapGroupsFunction.call"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.function.FlatMapGroupsFunction.call") + ) ++ + Seq( + // [SPARK-6429] Implement hashCode and equals together + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Partition.org$apache$spark$Partition$$super=uals") + ) ++ + Seq( + // SPARK-4819 replace Guava Optional + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getCheckpointDir"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.api.java.JavaSparkContext.getSparkHome"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.getCheckpointFile"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner") + ) ++ + Seq( + // SPARK-12481 Remove Hadoop 1.x + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.mapred.SparkHadoopMapRedUtil"), + // SPARK-12615 Remove deprecated APIs in core + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.$default$6"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.numericRDDToDoubleRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intToIntWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.intWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.writableWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToAsyncRDDActions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.boolToBoolWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longToLongWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToOrderedRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.booleanWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringToText"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.doubleToDoubleWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToSequenceFileRDDFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.bytesToBytesWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.longWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.stringWritableConverter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.floatToFloatWritable"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.rddToPairRDDFunctions$default$4"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addOnCompleteCallback"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.runningLocally"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.attemptId"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.defaultMinSplits"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.runJob"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.runJob"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.tachyonFolderName"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.initLocalProperties"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearJars"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.clearFiles"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.SparkContext.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith$default$2"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.toArray"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith$default$2"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.flatMapWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.filterWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.foreachWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapWith"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithSplit$default$2"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.SequenceFileRDDFunctions.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.splits"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.toArray"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.defaultMinSplits"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearJars"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.clearFiles"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.externalBlockStoreFolderName"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockManager"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.storage.ExternalBlockStore") + ) ++ Seq( + // SPARK-12149 Added new fields to ExecutorSummary + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") + ) ++ + // SPARK-12665 Remove deprecated and unused classes + Seq( + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.graphx.GraphKryoRegistrator"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$Multiplier"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$") + ) ++ Seq( + // SPARK-12591 Register OpenHashMapBasedStateMap for Kryo + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoInputDataInputBridge"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.KryoOutputDataOutputBridge") + ) ++ Seq( + // SPARK-12510 Refactor ActorReceiver to support Java + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") + ) ++ Seq( + // SPARK-12895 Implement TaskMetrics using accumulators + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.internalMetricsToAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectInternalAccumulators"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.collectAccumulators") + ) ++ Seq( + // SPARK-12896 Send only accumulator updates to driver, not TaskMetrics + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulable.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Accumulator.this"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.initialValue") + ) ++ Seq( + // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=") + ) ++ Seq( + // SPARK-12689 Migrate DDL parsing to the newly absorbed parser + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLException"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.sql.SQLContext.ddlParser") + ) ++ Seq( + // SPARK-7799 Add "streaming-akka" project + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$6"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.zeromq.ZeroMQUtils.createStream$default$5"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$4"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream$default$3"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.actorStream"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.actorStream"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.streaming.zeromq.ZeroMQReceiver"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver$Supervisor") + ) ++ Seq( + // SPARK-12348 Remove deprecated Streaming APIs. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.dstream.DStream.foreach"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions$default$4"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.awaitTermination"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.networkStream"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.api.java.JavaStreamingContextFactory"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.awaitTermination"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.sc"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.reduceByWindow"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreachRDD"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.api.java.JavaDStreamLike.foreach"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.getOrCreate") + ) ++ Seq( + // SPARK-12847 Remove StreamingListenerBus and post all Streaming events to the same thread as Spark events + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.AsynchronousListenerBus") + ) ++ Seq( + // SPARK-11622 Make LibSVMRelation extends HadoopFsRelation and Add LibSVMOutputWriter + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.source.libsvm.DefaultSource"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.source.libsvm.DefaultSource.createRelation") + ) ++ Seq( + // SPARK-6363 Make Scala 2.11 the default Scala version + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.cleanup"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metadataCleaner"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnDriverEndpoint"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint") + ) ++ Seq( + // SPARK-7889 + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.org$apache$spark$deploy$history$HistoryServer$@tachSparkUI"), + // SPARK-13296 + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.UDFRegistration.register"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedPythonFunction"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UserDefinedFunction$") + ) ++ Seq( + // SPARK-12995 Remove deprecated APIs in graphx + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.lib.SVDPlusPlus.runSVDPlusPlus"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.Graph.mapReduceTriplets$default$3"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.impl.GraphImpl.mapReduceTriplets") + ) ++ Seq( + // SPARK-13426 Remove the support of SIMR + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkMasterRegex.SIMR_REGEX") + ) ++ Seq( + // SPARK-13413 Remove SparkContext.metricsSystem/schedulerBackend_ setter + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.metricsSystem"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.schedulerBackend_=") + ) ++ Seq( + // SPARK-13220 Deprecate yarn-client and yarn-cluster mode + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") + ) ++ Seq( + // SPARK-13465 TaskContext. + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.TaskContext.addTaskFailureListener") + ) ++ Seq ( + // SPARK-7729 Executor which has been killed should also be displayed on Executor Tab + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorSummary.this") + ) ++ Seq( + // SPARK-13526 Move SQLContext per-session states to new class + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.sql.UDFRegistration.this") + ) ++ Seq( + // [SPARK-13486][SQL] Move SQLConf into an internal package + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLConf$SQLConfEntry$") + ) ++ Seq( + //SPARK-11011 UserDefinedType serialization should be strongly typed + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.serialize"), + // SPARK-12073: backpressure rate controller consumes events preferentially from lagging partitions + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.KafkaTestUtils.createTopic"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.kafka.DirectKafkaInputDStream.maxMessagesPerPartition") + ) ++ Seq( + // [SPARK-13244][SQL] Migrates DataFrame to Dataset + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.tables"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.sql"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.baseRelationToDataFrame"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.table"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrame.apply"), - case v if v.startsWith("1.3") => - Seq( - MimaBuild.excludeSparkPackage("deploy"), - MimaBuild.excludeSparkPackage("ml"), - // These are needed if checking against the sbt build, since they are part of - // the maven-generated artifacts in the 1.2 build. - MimaBuild.excludeSparkPackage("unused"), - ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional") - ) ++ Seq( - // SPARK-2321 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.SparkStageInfoImpl.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.SparkStageInfo.submissionTime") - ) ++ Seq( - // SPARK-4614 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrices.randn"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrices.rand") - ) ++ Seq( - // SPARK-5321 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.SparseMatrix.transposeMultiply"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrix.transpose"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.DenseMatrix.transposeMultiply"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix." + - "org$apache$spark$mllib$linalg$Matrix$_setter_$isTransposed_="), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrix.isTransposed"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.linalg.Matrix.foreachActive") - ) ++ Seq( - // SPARK-5540 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.solveLeastSquares"), - // SPARK-5536 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateBlock") - ) ++ Seq( - // SPARK-3325 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.api.java.JavaDStreamLike.print"), - // SPARK-2757 - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." + - "removeAndGetProcessor") - ) ++ Seq( - // SPARK-5123 (SparkSQL data type change) - alpha component only - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.ml.feature.HashingTF.outputDataType"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.ml.feature.Tokenizer.outputDataType"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.feature.Tokenizer.validateInputType"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.classification.LogisticRegressionModel.validateAndTransformSchema"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.ml.classification.LogisticRegression.validateAndTransformSchema") - ) ++ Seq( - // SPARK-4014 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.TaskContext.taskAttemptId"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.TaskContext.attemptNumber") - ) ++ Seq( - // SPARK-5166 Spark SQL API stabilization - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Transformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.Transformer.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Pipeline.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PipelineModel.transform"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.Estimator.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Evaluator.evaluate"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.Evaluator.evaluate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidator.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidatorModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScaler.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.transform"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegression.fit"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.evaluate") - ) ++ Seq( - // SPARK-5270 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.isEmpty") - ) ++ Seq( - // SPARK-5430 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.treeReduce"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.treeAggregate") - ) ++ Seq( - // SPARK-5297 Java FileStream do not work with custom key/values - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.api.java.JavaStreamingContext.fileStream") - ) ++ Seq( - // SPARK-5315 Spark Streaming Java API returns Scala DStream - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.api.java.JavaDStreamLike.reduceByWindow") - ) ++ Seq( - // SPARK-5461 Graph should have isCheckpointed, getCheckpointFiles methods - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.graphx.Graph.getCheckpointFiles"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.graphx.Graph.isCheckpointed") - ) ++ Seq( - // SPARK-4789 Standardize ML Prediction APIs - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.mllib.linalg.VectorUDT"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.serialize"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.sqlType") - ) ++ Seq( - // SPARK-5814 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$wrapDoubleArray"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$fillFullMatrix"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$iterations"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$makeOutLinkBlock"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$computeYtY"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$makeLinkRDDs"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$alpha"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$randomFactor"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$makeInLinkBlock"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$dspr"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$lambda"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$implicitPrefs"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$rank") - ) ++ Seq( - // SPARK-4682 - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.RealClock"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Clock"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.TestClock") - ) ++ Seq( - // SPARK-5922 Adding a generalized diff(other: RDD[(VertexId, VD)]) to VertexRDD - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.diff") - ) + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrame$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.LegacyFunctions"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataFrameHolder$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.localSeqToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.stringRddToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.rddToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.longRddToDataFrameHolder"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLImplicits.intRddToDataFrameHolder"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.GroupedDataset"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.subtract"), - case v if v.startsWith("1.2") => - Seq( - MimaBuild.excludeSparkPackage("deploy"), - MimaBuild.excludeSparkPackage("graphx") - ) ++ - MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ - MimaBuild.excludeSparkClass("mllib.linalg.Vector") ++ - Seq( - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.scheduler.TaskLocation"), - // Added normL1 and normL2 to trait MultivariateStatisticalSummary - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL2"), - // MapStatus should be private[spark] - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.scheduler.MapStatus"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.network.netty.PathResolver"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.network.netty.client.BlockClientListener"), + // [SPARK-14451][SQL] Move encoder definition into Aggregator interface + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.toColumn"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.bufferEncoder"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.expressions.Aggregator.outputEncoder"), - // TaskContext was promoted to Abstract class - ProblemFilters.exclude[AbstractClassProblem]( - "org.apache.spark.TaskContext"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.util.collection.SortDataFormat") - ) ++ Seq( - // Adding new methods to the JavaRDDLike trait: - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.takeAsync"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.foreachPartitionAsync"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.countAsync"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.foreachAsync"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.collectAsync") - ) ++ Seq( - // SPARK-3822 - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") - ) ++ Seq( - // SPARK-1209 - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.hadoop.mapred.SparkHadoopMapRedUtil"), - ProblemFilters.exclude[MissingTypesProblem]( - "org.apache.spark.rdd.PairRDDFunctions") - ) ++ Seq( - // SPARK-4062 - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.kafka.KafkaReceiver#MessageHandler.this") - ) + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MultilabelMetrics.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionSummary.predictions") + ) ++ Seq( + // [SPARK-13686][MLLIB][STREAMING] Add a constructor parameter `reqParam` to (Streaming)LinearRegressionWithSGD + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LinearRegressionWithSGD.this") + ) ++ Seq( + // SPARK-15250 Remove deprecated json API in DataFrameReader + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameReader.json") + ) ++ Seq( + // SPARK-13920: MIMA checks should apply to @Experimental and @DeveloperAPI APIs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineCombinersByKey"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Aggregator.combineValuesByKey"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.run"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.runJob"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ComplexFutureAction.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.actorSystem"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.cacheManager"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getConfigurationFromJobContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTaskAttemptIDFromTaskAttemptContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.newConfiguration"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.bytesReadCallback_="), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.productPrefix"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.setBytesReadCallback"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.updateBytesRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.productPrefix"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decFetchWaitTime"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decLocalBlocksFetched"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRecordsRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBlocksFetched"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleReadMetrics.decRemoteBytesRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleBytesWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleRecordsWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.decShuffleWriteTime"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleBytesWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleRecordsWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.incShuffleWriteTime"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.ShuffleWriteMetrics.setShuffleRecordsWritten"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.feature.PCAModel.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.mapPartitionsWithContext"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.AccumulableInfo.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.taskMetrics"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.TaskInfo.attempt"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.ExperimentalMethods.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUDF"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.callUdf"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.cumeDist"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.denseRank"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.inputFileName"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.isNaN"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.percentRank"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.rowNumber"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.functions.sparkPartitionId"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.externalBlockStoreSize"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.BlockStatus.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatus.offHeapUsedByRdd"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatusListener.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.scheduler.BatchInfo.streamIdToNumRecords"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.storageStatusList"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.storage.StorageListener.storageStatusList"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.InputMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.executor.OutputMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Pipeline.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PipelineModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.PredictionModel.transformImpl"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.extractLabeledPoints"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Predictor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Transformer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.GBTClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayes.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRest.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.OneVsRestModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.RandomForestClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeans.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.computeCost"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logLikelihood"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.logPerplexity"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.clustering.LDAModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.evaluation.RegressionEvaluator.evaluate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Binarizer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Bucketizer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelector.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ChiSqSelectorModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.CountVectorizerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.HashingTF.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDF.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IDFModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.IndexToString.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Interaction.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScaler.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.OneHotEncoder.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCA.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.PCAModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormula.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.RFormulaModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.SQLTransformer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScaler.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StopWordsRemover.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StringIndexerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorAssembler.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexer.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorIndexerModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.VectorSlicer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2Vec.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALS.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.GBTRegressor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.extractWeightedLabeledPoints"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegression.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.extractWeightedLabeledPoints"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegression.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionTrainingSummary.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidator.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.CrossValidatorModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplit.fit"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.tuning.TrainValidationSplitModel.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.BinaryClassificationMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.MulticlassMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.evaluation.RegressionMetrics.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.DataFrameWriter.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.broadcast"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.functions.callUDF"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.sources.InsertableRelation.insert"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.fMeasureByThreshold"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.pr"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.precisionByThreshold"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.recallByThreshold"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.roc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.describeTopics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.getVectors"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.itemFactors"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.recommendation.ALSModel.userFactors"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.residuals"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.name"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.AccumulableInfo.value"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.drop"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.fill"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameNaFunctions.replace"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.jdbc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.json"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.load"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.orc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.parquet"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.table"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameReader.text"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.crosstab"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.freqItems"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.DataFrameStatFunctions.sampleBy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.emptyDataFrame"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.range"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.functions.udf"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.scheduler.JobLogger"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorHelper"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorSupervisorStrategy$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.Statistics$"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.InputMetrics$"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.executor.OutputMetrics$"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.functions$"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Estimator.fit"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Predictor.train"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.Transformer.transform"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.evaluation.Evaluator.evaluate"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListener.onOtherEvent"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.CreatableRelationProvider.createRelation"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.InsertableRelation.insert") + ) ++ Seq( + // [SPARK-13926] Automatically use Kryo serializer when shuffling RDDs with simple types + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ShuffleDependency.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ShuffleDependency.serializer"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.serializer.Serializer$") + ) ++ Seq( + // SPARK-13927: add row/column iterator to local matrices + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.rowIter"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.colIter") + ) ++ Seq( + // SPARK-13948: MiMa Check should catch if the visibility change to `private` + // TODO(josh): Some of these may be legitimate incompatibilities; we should follow up before the 2.0.0 release + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.toDS"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.askTimeout"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.lookupTimeout"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.UnaryTransformer.transform"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassifier.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegression.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressor.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.groupBy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.select"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.toDF"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Logging.initializeLogIfNecessary"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerEvent.logEvent"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance") + ) ++ Seq( + // [SPARK-14014] Replace existing analysis.Catalog with SessionCatalog + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.this") + ) ++ Seq( + // [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Logging"), + (problem: Problem) => problem match { + case MissingTypesProblem(_, missing) + if missing.map(_.fullName).sameElements(Seq("org.apache.spark.Logging")) => false + case _ => true + } + ) ++ Seq( + // [SPARK-13990] Automatically pick serializer when caching RDDs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.uploadBlock") + ) ++ Seq( + // [SPARK-14089][CORE][MLLIB] Remove methods that has been deprecated since 1.1, 1.2, 1.3, 1.4, and 1.5 + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.getThreadLocal"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeReduce"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.rdd.RDDFunctions.treeAggregate"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.tree.configuration.Strategy.defaultStategy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLibSVMFile"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.saveLabeledData"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.util.MLUtils.loadLabeledData"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.optimization.LBFGS.setMaxNumIterations"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.evaluation.BinaryClassificationEvaluator.setScoreCol") + ) ++ Seq( + // [SPARK-14205][SQL] remove trait Queryable + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.sql.Dataset") + ) ++ Seq( + // [SPARK-11262][ML] Unit test for gradient, loss layers, memory management + // for multilayer perceptron. + // This class is marked as `private`. + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.ann.SoftmaxFunction") + ) ++ Seq( + // [SPARK-13674][SQL] Add wholestage codegen support to Sample + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.util.random.PoissonSampler.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.random.PoissonSampler.this") + ) ++ Seq( + // [SPARK-13430][ML] moved featureCol from LinearRegressionModelSummary to LinearRegressionSummary + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this") + ) ++ Seq( + // [SPARK-14437][Core] Use the address that NettyBlockTransferService listens to create BlockManagerId + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.NettyBlockTransferService.this") + ) ++ Seq( + // [SPARK-13048][ML][MLLIB] keepLastCheckpoint option for LDA EM optimizer + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.DistributedLDAModel.this") + ) ++ Seq( + // [SPARK-14475] Propagate user-defined context from driver to executors + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getLocalProperty"), + // [SPARK-14617] Remove deprecated APIs in TaskMetrics + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.InputMetrics$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.executor.OutputMetrics$"), + // [SPARK-14628] Simplify task metrics by always tracking read/write metrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.readMethod"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.writeMethod") + ) ++ Seq( + // SPARK-14628: Always track input/output/shuffle metrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.totalBlocksFetched"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.inputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.outputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleWriteMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.shuffleReadMetrics"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetrics.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.inputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.outputMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleWriteMetrics"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleReadMetrics"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.this") + ) ++ Seq( + // SPARK-13643: Move functionality from SQLContext to SparkSession + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.getSchema") + ) ++ Seq( + // [SPARK-14407] Hides HadoopFsRelation related data source API into execution package + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriter"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriterFactory") + ) ++ Seq( + // SPARK-14734: Add conversions between mllib and ml Vector, Matrix types + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asML"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asML") + ) ++ Seq( + // SPARK-14704: Create accumulators in TaskMetrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.this") + ) ++ Seq( + // SPARK-14861: Replace internal usages of SQLContext with SparkSession + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.clustering.LocalLDAModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.clustering.DistributedLDAModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.clustering.LDAModel.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]( + "org.apache.spark.ml.clustering.LDAModel.sqlContext"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.sql.Dataset.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.sql.DataFrameReader.this") + ) ++ Seq( + // SPARK-14542 configurable buffer size for pipe RDD + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.pipe"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.pipe") + ) ++ Seq( + // [SPARK-4452][Core]Shuffle data structures can starve others on the same thread for memory + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.util.collection.Spillable") + ) ++ Seq( + // [SPARK-14952][Core][ML] Remove methods deprecated in 1.6 + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.input.PortableDataStream.close"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.weights"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionModel.weights") + ) ++ Seq( + // [SPARK-10653] [Core] Remove unnecessary things from SparkEnv + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.sparkFilesDir"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkEnv.blockTransferService") + ) ++ Seq( + // SPARK-14654: New accumulator API + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ExceptionFailure$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.metrics"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExceptionFailure.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.remoteBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.totalBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.executor.ShuffleReadMetrics.localBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.remoteBlocksFetched"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.localBlocksFetched") + ) ++ Seq( + // [SPARK-14615][ML] Use the new ML Vector and Matrix in the ML pipeline based algorithms + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.getOldDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.estimatedDocConcentration"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.LDAModel.topicsMatrix"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.clustering.KMeansModel.clusterCenters"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.decodeLabel"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LabelConverter.encodeLabeledPoint"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.weights"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.predict"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.theta"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.pi"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.probability2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predictRaw"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.raw2probabilityInPlace"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegressionModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.raw2prediction"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.classification.ClassificationModel.predictRaw"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.getScalingVec"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.setScalingVec"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.PCAModel.pc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMax"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.originalMin"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.MinMaxScalerModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.Word2VecModel.findSynonyms"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.IDFModel.idf"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.mean"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.StandardScalerModel.std"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.predictQuantiles"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.predictions"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.IsotonicRegressionModel.boundaries"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.predict"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.coefficients"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.LinearRegressionModel.this") + ) ++ Seq( + // [SPARK-15290] Move annotations, like @Since / @DeveloperApi, into spark-tags + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.package"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Private"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.AlphaComponent"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.Experimental"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.annotation.DeveloperApi") + ) ++ Seq( + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.asBreeze"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.asBreeze") + ) ++ Seq( + // [SPARK-15914] Binary compatibility is broken since consolidation of Dataset and DataFrame + // in Spark 2.0. However, source level compatibility is still maintained. + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.load"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonRDD"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jsonFile"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jdbc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema") + ) + } - case v if v.startsWith("1.1") => - Seq( - MimaBuild.excludeSparkPackage("deploy"), - MimaBuild.excludeSparkPackage("graphx") - ) ++ - Seq( - // Adding new method to JavaRDLike trait - we should probably mark this as a developer API. - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitions"), - // Should probably mark this as Experimental - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.foreachAsync"), - // We made a mistake earlier (ed06500d3) in the Java API to use default parameter values - // for countApproxDistinct* functions, which does not work in Java. We later removed - // them, and use the following to tell Mima to not care about them. - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDD.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.storage.DiskStore.getValues"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.storage.MemoryStore.Entry") - ) ++ - Seq( - // Serializer interface change. See SPARK-3045. - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.serializer.DeserializationStream"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.serializer.Serializer"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.serializer.SerializationStream"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.serializer.SerializerInstance") - )++ - Seq( - // Renamed putValues -> putArray + putIterator - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.storage.MemoryStore.putValues"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.storage.DiskStore.putValues"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.storage.TachyonStore.putValues") - ) ++ - Seq( - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.streaming.flume.FlumeReceiver.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.streaming.kafka.KafkaUtils.createStream"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.streaming.kafka.KafkaReceiver.this") - ) ++ - Seq( // Ignore some private methods in ALS. - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures"), - ProblemFilters.exclude[MissingMethodProblem]( // The only public constructor is the one without arguments. - "org.apache.spark.mllib.recommendation.ALS.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures") - ) ++ - MimaBuild.excludeSparkClass("mllib.linalg.distributed.ColumnStatisticsAggregator") ++ - MimaBuild.excludeSparkClass("rdd.ZippedRDD") ++ - MimaBuild.excludeSparkClass("rdd.ZippedPartition") ++ - MimaBuild.excludeSparkClass("util.SerializableHyperLogLog") ++ - MimaBuild.excludeSparkClass("storage.Values") ++ - MimaBuild.excludeSparkClass("storage.Entry") ++ - MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") ++ - // Class was missing "@DeveloperApi" annotation in 1.0. - MimaBuild.excludeSparkClass("scheduler.SparkListenerApplicationStart") ++ - Seq( - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.tree.impurity.Gini.calculate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.tree.impurity.Entropy.calculate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.tree.impurity.Variance.calculate") - ) ++ - Seq( // Package-private classes removed in SPARK-2341 - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") - ) ++ - Seq( // package-private classes removed in MLlib - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.regression.GeneralizedLinearAlgorithm.org$apache$spark$mllib$regression$GeneralizedLinearAlgorithm$$prependOne") - ) ++ - Seq( // new Vector methods in MLlib (binary compatible assuming users do not implement Vector) - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.copy") - ) ++ - Seq( // synthetic methods generated in LabeledPoint - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.mllib.regression.LabeledPoint$"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.regression.LabeledPoint.apply"), - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LabeledPoint.toString") - ) ++ - Seq ( // Scala 2.11 compatibility fix - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.$default$2") - ) - case v if v.startsWith("1.0") => - Seq( - MimaBuild.excludeSparkPackage("api.java"), - MimaBuild.excludeSparkPackage("mllib"), - MimaBuild.excludeSparkPackage("streaming") - ) ++ - MimaBuild.excludeSparkClass("rdd.ClassTags") ++ - MimaBuild.excludeSparkClass("util.XORShiftRandom") ++ - MimaBuild.excludeSparkClass("graphx.EdgeRDD") ++ - MimaBuild.excludeSparkClass("graphx.VertexRDD") ++ - MimaBuild.excludeSparkClass("graphx.impl.GraphImpl") ++ - MimaBuild.excludeSparkClass("graphx.impl.RoutingTable") ++ - MimaBuild.excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++ - MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++ - MimaBuild.excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ - MimaBuild.excludeSparkClass("mllib.optimization.SquaredGradient") ++ - MimaBuild.excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ - MimaBuild.excludeSparkClass("mllib.regression.LassoWithSGD") ++ - MimaBuild.excludeSparkClass("mllib.regression.LinearRegressionWithSGD") + def excludes(version: String) = version match { + case v if v.startsWith("2.1") => v21excludes + case v if v.startsWith("2.0") => v20excludes case _ => Seq() } } From aa8cbcd199b5dcfd95b6a5e6f214f291e27d5781 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 11 Jul 2016 16:21:13 +0800 Subject: [PATCH 0935/1470] [SPARK-16355][SPARK-16354][SQL] Fix Bugs When LIMIT/TABLESAMPLE is Non-foldable, Zero or Negative #### What changes were proposed in this pull request? **Issue 1:** When a query containing LIMIT/TABLESAMPLE 0, the statistics could be zero. Results are correct but it could cause a huge performance regression. For example, ```Scala Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") .createOrReplaceTempView("test") val df1 = spark.table("test") val df2 = spark.table("test").limit(0) val df = df1.join(df2, Seq("k"), "left") ``` The statistics of both `df` and `df2` are zero. The statistics values should never be zero; otherwise `sizeInBytes` of `BinaryNode` will also be zero (product of children). This PR is to increase it to `1` when the num of rows is equal to 0. **Issue 2:** When a query containing negative LIMIT/TABLESAMPLE, we should issue exceptions. Negative values could break the implementation assumption of multiple parts. For example, statistics calculation. Below is the example query. ```SQL SELECT * FROM testData TABLESAMPLE (-1 rows) SELECT * FROM testData LIMIT -1 ``` This PR is to issue an appropriate exception in this case. **Issue 3:** Spark SQL follows the restriction of LIMIT clause in Hive. The argument to the LIMIT clause must evaluate to a constant value. It can be a numeric literal, or another kind of numeric expression involving operators, casts, and function return values. You cannot refer to a column or use a subquery. Currently, we do not detect whether the expression in LIMIT clause is foldable or not. If non-foldable, we might issue a strange error message. For example, ```SQL SELECT * FROM testData LIMIT rand() > 0.2 ``` Then, a misleading error message is issued, like ``` assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2) +- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203] +- LocalLimit (_nondeterministic#202 > 0.2) +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202] +- LogicalRDD [key#11, value#12] java.lang.AssertionError: assertion failed: No plan for GlobalLimit (_nondeterministic#203 > 0.2) +- Project [key#11, value#12, rand(-1441968339187861415) AS _nondeterministic#203] +- LocalLimit (_nondeterministic#202 > 0.2) +- Project [key#11, value#12, rand(-1308350387169017676) AS _nondeterministic#202] +- LogicalRDD [key#11, value#12] ``` This PR detects it and then issues a meaningful error message. #### How was this patch tested? Added test cases. Author: gatorsmile Closes #14034 from gatorsmile/limit. (cherry picked from commit e22627894126dceb7491300b63f1fe028b1e2e2c) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/CheckAnalysis.scala | 19 ++++++++ .../plans/logical/basicLogicalOperators.scala | 16 ++++++- .../analysis/AnalysisErrorSuite.scala | 6 +++ .../org/apache/spark/sql/SQLQuerySuite.scala | 37 +++++++++++++++- .../apache/spark/sql/StatisticsSuite.scala | 44 +++++++++++++++++++ 5 files changed, 118 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 7b30fcc6c5314..8b87a4e41c23d 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,6 +46,21 @@ trait CheckAnalysis extends PredicateHelper { }).length > 1 } + private def checkLimitClause(limitExpr: Expression): Unit = { + limitExpr match { + case e if !e.foldable => failAnalysis( + "The limit expression must evaluate to a constant value, but got " + + limitExpr.sql) + case e if e.dataType != IntegerType => failAnalysis( + s"The limit expression must be integer type, but got " + + e.dataType.simpleString) + case e if e.eval().asInstanceOf[Int] < 0 => failAnalysis( + "The limit expression must be equal to or greater than 0, but got " + + e.eval().asInstanceOf[Int]) + case e => // OK + } + } + def checkAnalysis(plan: LogicalPlan): Unit = { // We transform up and order the rules so as to catch the first possible failure instead // of the result of cascading resolution failures. @@ -251,6 +266,10 @@ trait CheckAnalysis extends PredicateHelper { s"but one table has '${firstError.output.length}' columns and another table has " + s"'${s.children.head.output.length}' columns") + case GlobalLimit(limitExpr, _) => checkLimitClause(limitExpr) + + case LocalLimit(limitExpr, _) => checkLimitClause(limitExpr) + case p if p.expressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) => p match { case _: Filter | _: Aggregate | _: Project => // Ok diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 79f9a210a30b5..c0e400f61777f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -660,7 +660,13 @@ case class GlobalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryN } override lazy val statistics: Statistics = { val limit = limitExpr.eval().asInstanceOf[Int] - val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum + val sizeInBytes = if (limit == 0) { + // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero + // (product of children). + 1 + } else { + (limit: Long) * output.map(a => a.dataType.defaultSize).sum + } child.statistics.copy(sizeInBytes = sizeInBytes) } } @@ -675,7 +681,13 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo } override lazy val statistics: Statistics = { val limit = limitExpr.eval().asInstanceOf[Int] - val sizeInBytes = (limit: Long) * output.map(a => a.dataType.defaultSize).sum + val sizeInBytes = if (limit == 0) { + // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero + // (product of children). + 1 + } else { + (limit: Long) * output.map(a => a.dataType.defaultSize).sum + } child.statistics.copy(sizeInBytes = sizeInBytes) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index a9cde1e19efc8..ff112c51697ad 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -352,6 +352,12 @@ class AnalysisErrorSuite extends AnalysisTest { "Generators are not supported outside the SELECT clause, but got: Sort" :: Nil ) + errorTest( + "num_rows in limit clause must be equal to or greater than 0", + listRelation.limit(-1), + "The limit expression must be equal to or greater than 0, but got -1" :: Nil + ) + errorTest( "more than one generators in SELECT", listRelation.select(Explode('list), Explode('list)), 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 dca9e5e503c72..ede7d9a0c95b9 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 @@ -660,11 +660,11 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("limit") { checkAnswer( - sql("SELECT * FROM testData LIMIT 10"), + sql("SELECT * FROM testData LIMIT 9 + 1"), testData.take(10).toSeq) checkAnswer( - sql("SELECT * FROM arrayData LIMIT 1"), + sql("SELECT * FROM arrayData LIMIT CAST(1 AS Integer)"), arrayData.collect().take(1).map(Row.fromTuple).toSeq) checkAnswer( @@ -672,6 +672,39 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { mapData.collect().take(1).map(Row.fromTuple).toSeq) } + test("non-foldable expressions in LIMIT") { + val e = intercept[AnalysisException] { + sql("SELECT * FROM testData LIMIT key > 3") + }.getMessage + assert(e.contains("The limit expression must evaluate to a constant value, " + + "but got (testdata.`key` > 3)")) + } + + test("Expressions in limit clause are not integer") { + var e = intercept[AnalysisException] { + sql("SELECT * FROM testData LIMIT true") + }.getMessage + assert(e.contains("The limit expression must be integer type, but got boolean")) + + e = intercept[AnalysisException] { + sql("SELECT * FROM testData LIMIT 'a'") + }.getMessage + assert(e.contains("The limit expression must be integer type, but got string")) + } + + test("negative in LIMIT or TABLESAMPLE") { + val expected = "The limit expression must be equal to or greater than 0, but got -1" + var e = intercept[AnalysisException] { + sql("SELECT * FROM testData TABLESAMPLE (-1 rows)") + }.getMessage + assert(e.contains(expected)) + + e = intercept[AnalysisException] { + sql("SELECT * FROM testData LIMIT -1") + }.getMessage + assert(e.contains(expected)) + } + test("CTE feature") { checkAnswer( sql("with q1 as (select * from testData limit 10) select * from q1"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index 4de3cf605caa1..ab55242ec0683 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.plans.logical.{GlobalLimit, Join, LocalLimit} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ class StatisticsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) @@ -31,4 +33,46 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { spark.sessionState.conf.autoBroadcastJoinThreshold) } + test("estimates the size of limit") { + withTempTable("test") { + Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") + .createOrReplaceTempView("test") + Seq((0, 1), (1, 24), (2, 48)).foreach { case (limit, expected) => + val df = sql(s"""SELECT * FROM test limit $limit""") + + val sizesGlobalLimit = df.queryExecution.analyzed.collect { case g: GlobalLimit => + g.statistics.sizeInBytes + } + assert(sizesGlobalLimit.size === 1, s"Size wrong for:\n ${df.queryExecution}") + assert(sizesGlobalLimit.head === BigInt(expected), + s"expected exact size $expected for table 'test', got: ${sizesGlobalLimit.head}") + + val sizesLocalLimit = df.queryExecution.analyzed.collect { case l: LocalLimit => + l.statistics.sizeInBytes + } + assert(sizesLocalLimit.size === 1, s"Size wrong for:\n ${df.queryExecution}") + assert(sizesLocalLimit.head === BigInt(expected), + s"expected exact size $expected for table 'test', got: ${sizesLocalLimit.head}") + } + } + } + + test("estimates the size of a limit 0 on outer join") { + withTempTable("test") { + Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") + .createOrReplaceTempView("test") + val df1 = spark.table("test") + val df2 = spark.table("test").limit(0) + val df = df1.join(df2, Seq("k"), "left") + + val sizes = df.queryExecution.analyzed.collect { case g: Join => + g.statistics.sizeInBytes + } + + assert(sizes.size === 1, s"number of Join nodes is wrong:\n ${df.queryExecution}") + assert(sizes.head === BigInt(96), + s"expected exact size 96 for table 'test', got: ${sizes.head}") + } + } + } From 7e4ba66d938a8bf312e991dfa034d420a0b7b360 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Mon, 11 Jul 2016 20:05:28 +0800 Subject: [PATCH 0936/1470] [SPARK-16381][SQL][SPARKR] Update SQL examples and programming guide for R language binding https://issues.apache.org/jira/browse/SPARK-16381 ## What changes were proposed in this pull request? Update SQL examples and programming guide for R language binding. Here I just follow example https://github.com/apache/spark/compare/master...liancheng:example-snippet-extraction, created a separate R file to store all the example code. ## How was this patch tested? Manual test on my local machine. Screenshot as below: ![screen shot 2016-07-06 at 4 52 25 pm](https://cloud.githubusercontent.com/assets/3925641/16638180/13925a58-439a-11e6-8d57-8451a63dcae9.png) Author: Xin Ren Closes #14082 from keypointt/SPARK-16381. (cherry picked from commit 9cb1eb7af779e74165552977002158a7dad9bb09) Signed-off-by: Cheng Lian --- docs/sql-programming-guide.md | 155 ++----------------- examples/src/main/r/RSparkSQLExample.R | 197 +++++++++++++++++++++++++ examples/src/main/r/dataframe.R | 2 +- examples/src/main/r/ml.R | 2 +- 4 files changed, 212 insertions(+), 144 deletions(-) create mode 100644 examples/src/main/r/RSparkSQLExample.R diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 68419e1331594..448251cfdc697 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -86,9 +86,7 @@ The entry point into all functionality in Spark is the [`SparkSession`](api/pyth The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: -{% highlight r %} -sparkR.session() -{% endhighlight %} +{% include_example init_session r/RSparkSQLExample.R %} Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around.

    27. @@ -155,12 +153,7 @@ from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: -{% highlight r %} -df <- read.json("examples/src/main/resources/people.json") - -# Displays the content of the DataFrame -showDF(df) -{% endhighlight %} +{% include_example create_DataFrames r/RSparkSQLExample.R %}
      @@ -343,50 +336,8 @@ In addition to simple column references and expressions, DataFrames also have a
      -{% highlight r %} -# Create the DataFrame -df <- read.json("examples/src/main/resources/people.json") - -# Show the content of the DataFrame -showDF(df) -## age name -## null Michael -## 30 Andy -## 19 Justin - -# Print the schema in a tree format -printSchema(df) -## root -## |-- age: long (nullable = true) -## |-- name: string (nullable = true) -# Select only the "name" column -showDF(select(df, "name")) -## name -## Michael -## Andy -## Justin - -# Select everybody, but increment the age by 1 -showDF(select(df, df$name, df$age + 1)) -## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 - -# Select people older than 21 -showDF(where(df, df$age > 21)) -## age name -## 30 Andy - -# Count people by age -showDF(count(groupBy(df, "age"))) -## age count -## null 1 -## 19 1 -## 30 1 - -{% endhighlight %} +{% include_example dataframe_operations r/RSparkSQLExample.R %} For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). @@ -429,12 +380,10 @@ df = spark.sql("SELECT * FROM table")
      The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. -{% highlight r %} -df <- sql("SELECT * FROM table") -{% endhighlight %} -
      +{% include_example sql_query r/RSparkSQLExample.R %}
      + ## Creating Datasets @@ -888,10 +837,7 @@ df.select("name", "favorite_color").write.save("namesAndFavColors.parquet")
      -{% highlight r %} -df <- read.df("examples/src/main/resources/users.parquet") -write.df(select(df, "name", "favorite_color"), "namesAndFavColors.parquet") -{% endhighlight %} +{% include_example source_parquet r/RSparkSQLExample.R %}
      @@ -937,12 +883,7 @@ df.select("name", "age").write.save("namesAndAges.parquet", format="parquet")
      -{% highlight r %} - -df <- read.df("examples/src/main/resources/people.json", "json") -write.df(select(df, "name", "age"), "namesAndAges.parquet", "parquet") - -{% endhighlight %} +{% include_example source_json r/RSparkSQLExample.R %}
      @@ -978,9 +919,7 @@ df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet
      -{% highlight r %} -df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") -{% endhighlight %} +{% include_example direct_query r/RSparkSQLExample.R %}
      @@ -1133,26 +1072,7 @@ for teenName in teenNames.collect():
      -{% highlight r %} - -schemaPeople # The SparkDataFrame from the previous example. - -# SparkDataFrame can be saved as Parquet files, maintaining the schema information. -write.parquet(schemaPeople, "people.parquet") - -# 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. -parquetFile <- read.parquet("people.parquet") - -# Parquet files can also be used to create a temporary view and then used in SQL statements. -createOrReplaceTempView(parquetFile, "parquetFile") -teenagers <- sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") -schema <- structType(structField("name", "string")) -teenNames <- dapply(df, function(p) { cbind(paste("Name:", p$name)) }, schema) -for (teenName in collect(teenNames)$name) { - cat(teenName, "\n") -} -{% endhighlight %} +{% include_example load_programmatically r/RSparkSQLExample.R %}
      @@ -1315,27 +1235,7 @@ df3.printSchema()
      -{% highlight r %} - -# Create a simple DataFrame, stored into a partition directory -write.df(df1, "data/test_table/key=1", "parquet", "overwrite") - -# Create another DataFrame in a new partition directory, -# adding a new column and dropping an existing column -write.df(df2, "data/test_table/key=2", "parquet", "overwrite") - -# Read the partitioned table -df3 <- read.df("data/test_table", "parquet", mergeSchema="true") -printSchema(df3) - -# The final schema consists of all 3 columns in the Parquet files together -# with the partitioning column appeared in the partition directory paths. -# root -# |-- single: int (nullable = true) -# |-- double: int (nullable = true) -# |-- triple: int (nullable = true) -# |-- key : int (nullable = true) -{% endhighlight %} +{% include_example schema_merging r/RSparkSQLExample.R %}
      @@ -1601,25 +1501,8 @@ Note that the file that is offered as _a json file_ is not a typical JSON file. line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. -{% highlight r %} -# A JSON dataset is pointed to by path. -# The path can be either a single text file or a directory storing text files. -path <- "examples/src/main/resources/people.json" -# Create a DataFrame from the file(s) pointed to by path -people <- read.json(path) +{% include_example load_json_file r/RSparkSQLExample.R %} -# The inferred schema can be visualized using the printSchema() method. -printSchema(people) -# root -# |-- age: long (nullable = true) -# |-- name: string (nullable = true) - -# Register this DataFrame as a table. -createOrReplaceTempView(people, "people") - -# SQL statements can be run by using the sql methods. -teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") -{% endhighlight %}
      @@ -1734,16 +1617,8 @@ results = spark.sql("FROM src SELECT key, value").collect() When working with Hive one must instantiate `SparkSession` with Hive support. This adds support for finding tables in the MetaStore and writing queries using HiveQL. -{% highlight r %} -# enableHiveSupport defaults to TRUE -sparkR.session(enableHiveSupport = TRUE) -sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") - -# Queries can be expressed in HiveQL. -results <- collect(sql("FROM src SELECT key, value")) -{% endhighlight %} +{% include_example hive_table r/RSparkSQLExample.R %}
      @@ -1920,11 +1795,7 @@ df = spark.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbtable='
      -{% highlight r %} - -df <- read.jdbc("jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password") - -{% endhighlight %} +{% include_example jdbc r/RSparkSQLExample.R %}
      diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R new file mode 100644 index 0000000000000..eba3f1b91e2df --- /dev/null +++ b/examples/src/main/r/RSparkSQLExample.R @@ -0,0 +1,197 @@ +# +# 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) + +# $example on:init_session$ +sparkR.session(appName = "MyApp", sparkConfig = list(spark.executor.memory = "1g")) +# $example off:init_session$ + + +# $example on:create_DataFrames$ +df <- read.json("examples/src/main/resources/people.json") + +# Displays the content of the DataFrame +head(df) + +# Another method to print the first few rows and optionally truncate the printing of long values +showDF(df) +# $example off:create_DataFrames$ + + +# $example on:dataframe_operations$ +# Create the DataFrame +df <- read.json("examples/src/main/resources/people.json") + +# Show the content of the DataFrame +head(df) +## age name +## null Michael +## 30 Andy +## 19 Justin + +# Print the schema in a tree format +printSchema(df) +## root +## |-- age: long (nullable = true) +## |-- name: string (nullable = true) + +# Select only the "name" column +head(select(df, "name")) +## name +## Michael +## Andy +## Justin + +# Select everybody, but increment the age by 1 +head(select(df, df$name, df$age + 1)) +## name (age + 1) +## Michael null +## Andy 31 +## Justin 20 + +# Select people older than 21 +head(where(df, df$age > 21)) +## age name +## 30 Andy + +# Count people by age +head(count(groupBy(df, "age"))) +## age count +## null 1 +## 19 1 +## 30 1 +# $example off:dataframe_operations$ + + +# Register this DataFrame as a table. +createOrReplaceTempView(df, "table") +# $example on:sql_query$ +df <- sql("SELECT * FROM table") +# $example off:sql_query$ + + +# $example on:source_parquet$ +df <- read.df("examples/src/main/resources/users.parquet") +write.df(select(df, "name", "favorite_color"), "namesAndFavColors.parquet") +# $example off:source_parquet$ + + +# $example on:source_json$ +df <- read.df("examples/src/main/resources/people.json", "json") +namesAndAges <- select(df, "name", "age") +write.df(namesAndAges, "namesAndAges.parquet", "parquet") +# $example off:source_json$ + + +# $example on:direct_query$ +df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") +# $example off:direct_query$ + + +# $example on:load_programmatically$ +df <- read.df("examples/src/main/resources/people.json", "json") + +# SparkDataFrame can be saved as Parquet files, maintaining the schema information. +write.parquet(df, "people.parquet") + +# 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. +parquetFile <- read.parquet("people.parquet") + +# Parquet files can also be used to create a temporary view and then used in SQL statements. +createOrReplaceTempView(parquetFile, "parquetFile") +teenagers <- sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +head(teenagers) +## name +## 1 Justin + +# We can also run custom R-UDFs on Spark DataFrames. Here we prefix all the names with "Name:" +schema <- structType(structField("name", "string")) +teenNames <- dapply(df, function(p) { cbind(paste("Name:", p$name)) }, schema) +for (teenName in collect(teenNames)$name) { + cat(teenName, "\n") +} +## Name: Michael +## Name: Andy +## Name: Justin +# $example off:load_programmatically$ + + +# $example on:schema_merging$ +df1 <- createDataFrame(data.frame(single=c(12, 29), double=c(19, 23))) +df2 <- createDataFrame(data.frame(double=c(19, 23), triple=c(23, 18))) + +# Create a simple DataFrame, stored into a partition directory +write.df(df1, "data/test_table/key=1", "parquet", "overwrite") + +# Create another DataFrame in a new partition directory, +# adding a new column and dropping an existing column +write.df(df2, "data/test_table/key=2", "parquet", "overwrite") + +# Read the partitioned table +df3 <- read.df("data/test_table", "parquet", mergeSchema="true") +printSchema(df3) + +# The final schema consists of all 3 columns in the Parquet files together +# with the partitioning column appeared in the partition directory paths. +# root +# |-- single: double (nullable = true) +# |-- double: double (nullable = true) +# |-- triple: double (nullable = true) +# |-- key : int (nullable = true) +# $example off:schema_merging$ + + +# $example on:load_json_file$ +# A JSON dataset is pointed to by path. +# The path can be either a single text file or a directory storing text files. +path <- "examples/src/main/resources/people.json" +# Create a DataFrame from the file(s) pointed to by path +people <- read.json(path) + +# The inferred schema can be visualized using the printSchema() method. +printSchema(people) +# root +# |-- age: long (nullable = true) +# |-- name: string (nullable = true) + +# Register this DataFrame as a table. +createOrReplaceTempView(people, "people") + +# SQL statements can be run by using the sql methods. +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +head(teenagers) +## name +## 1 Justin +# $example off:load_json_file$ + + +# $example on:hive_table$ +# enableHiveSupport defaults to TRUE +sparkR.session(enableHiveSupport = TRUE) +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + +# Queries can be expressed in HiveQL. +results <- collect(sql("FROM src SELECT key, value")) +# $example off:hive_table$ + + +# $example on:jdbc$ +df <- read.jdbc("jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password") +# $example off:jdbc$ diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R index a377d6e864d2b..295f9b427622c 100644 --- a/examples/src/main/r/dataframe.R +++ b/examples/src/main/r/dataframe.R @@ -18,7 +18,7 @@ library(SparkR) # Initialize SparkSession -sc <- sparkR.session(appName="SparkR-DataFrame-example") +sc <- sparkR.session(appName = "SparkR-DataFrame-example") # Create a simple local data.frame localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) diff --git a/examples/src/main/r/ml.R b/examples/src/main/r/ml.R index 940c98dcb97a1..65242e68b3c81 100644 --- a/examples/src/main/r/ml.R +++ b/examples/src/main/r/ml.R @@ -22,7 +22,7 @@ library(SparkR) # Initialize SparkSession -sparkR.session(appName="SparkR-ML-example") +sparkR.session(appName = "SparkR-ML-example") # $example on$ ############################ spark.glm and glm ############################################## From f97dd8a8fd61ab1964b4a7dc4fd0ddecf801c612 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 11 Jul 2016 15:15:47 +0200 Subject: [PATCH 0937/1470] [SPARK-16459][SQL] Prevent dropping current database This PR prevents dropping the current database to avoid errors like the followings. ```scala scala> sql("create database delete_db") scala> sql("use delete_db") scala> sql("drop database delete_db") scala> sql("create table t as select 1") org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database `delete_db` not found; ``` Pass the Jenkins tests including an updated testcase. Author: Dongjoon Hyun Closes #14115 from dongjoon-hyun/SPARK-16459. (cherry picked from commit 7ac79da0e4607f7f89a3617edf53c2b174b378e8) Signed-off-by: Herman van Hovell --- .../sql/catalyst/catalog/SessionCatalog.scala | 20 ++++++++++++------- .../sql/execution/command/DDLSuite.scala | 9 +++++++++ .../hive/HiveContextCompatibilitySuite.scala | 1 + .../sql/hive/execution/HiveDDLSuite.scala | 2 ++ 4 files changed, 25 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e1d49912c311f..0f7e15ff27bff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -34,6 +34,10 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.util.StringUtils +object SessionCatalog { + val DEFAULT_DATABASE = "default" +} + /** * An internal catalog that is used by a Spark Session. This internal catalog serves as a * proxy to the underlying metastore (e.g. Hive Metastore) and it also manages temporary @@ -47,6 +51,7 @@ class SessionCatalog( functionRegistry: FunctionRegistry, conf: CatalystConf, hadoopConf: Configuration) extends Logging { + import SessionCatalog._ import CatalogTypes.TablePartitionSpec // For testing only. @@ -77,7 +82,7 @@ class SessionCatalog( // the corresponding item in the current database. @GuardedBy("this") protected var currentDb = { - val defaultName = "default" + val defaultName = DEFAULT_DATABASE val defaultDbDefinition = CatalogDatabase(defaultName, "default database", conf.warehousePath, Map()) // Initialize default database if it doesn't already exist @@ -146,8 +151,10 @@ class SessionCatalog( def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { val dbName = formatDatabaseName(db) - if (dbName == "default") { + if (dbName == DEFAULT_DATABASE) { throw new AnalysisException(s"Can not drop default database") + } else if (dbName == getCurrentDatabase) { + throw new AnalysisException(s"Can not drop current database `${dbName}`") } externalCatalog.dropDatabase(dbName, ignoreIfNotExists, cascade) } @@ -878,14 +885,14 @@ class SessionCatalog( * This is mainly used for tests. */ private[sql] def reset(): Unit = synchronized { - val default = "default" - listDatabases().filter(_ != default).foreach { db => + setCurrentDatabase(DEFAULT_DATABASE) + listDatabases().filter(_ != DEFAULT_DATABASE).foreach { db => dropDatabase(db, ignoreIfNotExists = false, cascade = true) } - listTables(default).foreach { table => + listTables(DEFAULT_DATABASE).foreach { table => dropTable(table, ignoreIfNotExists = false) } - listFunctions(default).map(_._1).foreach { func => + listFunctions(DEFAULT_DATABASE).map(_._1).foreach { func => if (func.database.isDefined) { dropFunction(func, ignoreIfNotExists = false) } else { @@ -902,7 +909,6 @@ class SessionCatalog( require(functionBuilder.isDefined, s"built-in function '$f' is missing function builder") functionRegistry.registerFunction(f, expressionInfo.get, functionBuilder.get) } - setCurrentDatabase(default) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 7d1f1d1e62fc7..b4294ed7ff1aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1270,6 +1270,15 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { "WITH SERDEPROPERTIES ('spark.sql.sources.me'='anything')") } + test("drop current database") { + sql("CREATE DATABASE temp") + sql("USE temp") + val m = intercept[AnalysisException] { + sql("DROP DATABASE temp") + }.getMessage + assert(m.contains("Can not drop current database `temp`")) + } + test("drop default database") { Seq("true", "false").foreach { caseSensitive => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala index 3aa8174702513..57363b7259c61 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -93,6 +93,7 @@ class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEac hc.sql("DROP TABLE mee_table") val tables2 = hc.sql("SHOW TABLES IN mee_db").collect().map(_.getString(0)) assert(tables2.isEmpty) + hc.sql("USE default") hc.sql("DROP DATABASE mee_db CASCADE") val databases3 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) assert(databases3.toSeq == Seq("default")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 93e50f4ee907b..343d7bae98bff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -472,6 +472,7 @@ class HiveDDLSuite sql(s"DROP TABLE $tabName") assert(tmpDir.listFiles.isEmpty) + sql("USE default") sql(s"DROP DATABASE $dbName") assert(!fs.exists(new Path(tmpDir.toString))) } @@ -526,6 +527,7 @@ class HiveDDLSuite assert(!tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) } + sql(s"USE default") val sqlDropDatabase = s"DROP DATABASE $dbName ${if (cascade) "CASCADE" else "RESTRICT"}" if (tableExists && !cascade) { val message = intercept[AnalysisException] { From 72cf743240c2f36fb45f5bf44be2ca16367320fc Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Mon, 11 Jul 2016 12:42:43 -0700 Subject: [PATCH 0938/1470] [SPARK-16318][SQL] Implement all remaining xpath functions (branch-2.0) ## What changes were proposed in this pull request? This patch implements all remaining xpath functions that Hive supports and not natively supported in Spark: xpath_int, xpath_short, xpath_long, xpath_float, xpath_double, xpath_string, and xpath. This is based on https://github.com/apache/spark/pull/13991 but for branch-2.0. ## How was this patch tested? Added unit tests and end-to-end tests. Author: petermaxlee Closes #14131 from petermaxlee/xpath-branch-2.0. --- .../expressions/xml/UDFXPathUtil.java | 34 +-- .../catalyst/analysis/FunctionRegistry.scala | 8 + .../expressions/xml/XPathBoolean.scala | 58 ------ .../sql/catalyst/expressions/xml/xpath.scala | 174 ++++++++++++++++ .../expressions/ExpressionEvalHelper.scala | 4 + .../expressions/xml/UDFXPathUtilSuite.scala | 5 +- .../xml/XPathExpressionSuite.scala | 195 +++++++++++++++--- .../spark/sql/XPathFunctionsSuite.scala | 62 ++++++ .../apache/spark/sql/XmlFunctionsSuite.scala | 32 --- .../spark/sql/hive/HiveSessionCatalog.scala | 4 +- 10 files changed, 441 insertions(+), 135 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java index 01a11f9bdca2d..d224332d8a6c9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.io.Reader; -import java.io.StringReader; import javax.xml.namespace.QName; import javax.xml.xpath.XPath; @@ -44,7 +43,7 @@ public class UDFXPathUtil { private XPathExpression expression = null; private String oldPath = null; - public Object eval(String xml, String path, QName qname) { + public Object eval(String xml, String path, QName qname) throws XPathExpressionException { if (xml == null || path == null || qname == null) { return null; } @@ -57,7 +56,7 @@ public Object eval(String xml, String path, QName qname) { try { expression = xpath.compile(path); } catch (XPathExpressionException e) { - expression = null; + throw new RuntimeException("Invalid XPath '" + path + "'" + e.getMessage(), e); } oldPath = path; } @@ -67,36 +66,35 @@ public Object eval(String xml, String path, QName qname) { } reader.set(xml); - try { return expression.evaluate(inputSource, qname); } catch (XPathExpressionException e) { - throw new RuntimeException ("Invalid expression '" + oldPath + "'", e); + throw new RuntimeException("Invalid XML document: " + e.getMessage() + "\n" + xml, e); } } - public Boolean evalBoolean(String xml, String path) { + public Boolean evalBoolean(String xml, String path) throws XPathExpressionException { return (Boolean) eval(xml, path, XPathConstants.BOOLEAN); } - public String evalString(String xml, String path) { + public String evalString(String xml, String path) throws XPathExpressionException { return (String) eval(xml, path, XPathConstants.STRING); } - public Double evalNumber(String xml, String path) { + public Double evalNumber(String xml, String path) throws XPathExpressionException { return (Double) eval(xml, path, XPathConstants.NUMBER); } - public Node evalNode(String xml, String path) { + public Node evalNode(String xml, String path) throws XPathExpressionException { return (Node) eval(xml, path, XPathConstants.NODE); } - public NodeList evalNodeList(String xml, String path) { + public NodeList evalNodeList(String xml, String path) throws XPathExpressionException { return (NodeList) eval(xml, path, XPathConstants.NODESET); } /** - * Reusable, non-threadsafe version of {@link StringReader}. + * Reusable, non-threadsafe version of {@link java.io.StringReader}. */ public static class ReusableStringReader extends Reader { @@ -117,20 +115,22 @@ public void set(String s) { /** Check to make sure that the stream has not been closed */ private void ensureOpen() throws IOException { - if (str == null) + if (str == null) { throw new IOException("Stream closed"); + } } @Override public int read() throws IOException { ensureOpen(); - if (next >= length) + if (next >= length) { return -1; + } return str.charAt(next++); } @Override - public int read(char cbuf[], int off, int len) throws IOException { + public int read(char[] cbuf, int off, int len) throws IOException { ensureOpen(); if ((off < 0) || (off > cbuf.length) || (len < 0) || ((off + len) > cbuf.length) || ((off + len) < 0)) { @@ -138,8 +138,9 @@ public int read(char cbuf[], int off, int len) throws IOException { } else if (len == 0) { return 0; } - if (next >= length) + if (next >= length) { return -1; + } int n = Math.min(length - next, len); str.getChars(next, next + n, cbuf, off); next += n; @@ -149,8 +150,9 @@ public int read(char cbuf[], int off, int len) throws IOException { @Override public long skip(long ns) throws IOException { ensureOpen(); - if (next >= length) + if (next >= length) { return 0; + } // Bound skip by beginning and end of the source long n = Math.min(length - next, ns); n = Math.max(-next, n); 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 c8bbbf88532dc..54568b7445df8 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 @@ -310,7 +310,15 @@ object FunctionRegistry { expression[UnBase64]("unbase64"), expression[Unhex]("unhex"), expression[Upper]("upper"), + expression[XPathList]("xpath"), expression[XPathBoolean]("xpath_boolean"), + expression[XPathDouble]("xpath_double"), + expression[XPathDouble]("xpath_number"), + expression[XPathFloat]("xpath_float"), + expression[XPathInt]("xpath_int"), + expression[XPathLong]("xpath_long"), + expression[XPathShort]("xpath_short"), + expression[XPathString]("xpath_string"), // datetime functions expression[AddMonths]("add_months"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala deleted file mode 100644 index 2a5256c7f56fd..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions.xml - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType} -import org.apache.spark.unsafe.types.UTF8String - - -@ExpressionDescription( - usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.", - extended = "> SELECT _FUNC_('1','a/b');\ntrue") -case class XPathBoolean(xml: Expression, path: Expression) - extends BinaryExpression with ExpectsInputTypes with CodegenFallback { - - @transient private lazy val xpathUtil = new UDFXPathUtil - - // If the path is a constant, cache the path string so that we don't need to convert path - // from UTF8String to String for every row. - @transient lazy val pathLiteral: String = path match { - case Literal(str: UTF8String, _) => str.toString - case _ => null - } - - override def prettyName: String = "xpath_boolean" - - override def dataType: DataType = BooleanType - - override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType) - - override def left: Expression = xml - override def right: Expression = path - - override protected def nullSafeEval(xml: Any, path: Any): Any = { - val xmlString = xml.asInstanceOf[UTF8String].toString - if (pathLiteral ne null) { - xpathUtil.evalBoolean(xmlString, pathLiteral) - } else { - xpathUtil.evalBoolean(xmlString, path.asInstanceOf[UTF8String].toString) - } - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala new file mode 100644 index 0000000000000..47f039e6a4cc4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/xpath.scala @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.xml + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * Base class for xpath_boolean, xpath_double, xpath_int, etc. + * + * This is not the world's most efficient implementation due to type conversion, but works. + */ +abstract class XPathExtract extends BinaryExpression with ExpectsInputTypes with CodegenFallback { + override def left: Expression = xml + override def right: Expression = path + + /** XPath expressions are always nullable, e.g. if the xml string is empty. */ + override def nullable: Boolean = true + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType) + + override def checkInputDataTypes(): TypeCheckResult = { + if (!path.foldable) { + TypeCheckFailure("path should be a string literal") + } else { + super.checkInputDataTypes() + } + } + + @transient protected lazy val xpathUtil = new UDFXPathUtil + @transient protected lazy val pathString: String = path.eval().asInstanceOf[UTF8String].toString + + /** Concrete implementations need to override the following three methods. */ + def xml: Expression + def path: Expression +} + +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.", + extended = "> SELECT _FUNC_('1','a/b');\ntrue") +case class XPathBoolean(xml: Expression, path: Expression) extends XPathExtract { + + override def prettyName: String = "xpath_boolean" + override def dataType: DataType = BooleanType + + override def nullSafeEval(xml: Any, path: Any): Any = { + xpathUtil.evalBoolean(xml.asInstanceOf[UTF8String].toString, pathString) + } +} + +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Returns a short value that matches the xpath expression", + extended = "> SELECT _FUNC_('12','sum(a/b)');\n3") +case class XPathShort(xml: Expression, path: Expression) extends XPathExtract { + override def prettyName: String = "xpath_int" + override def dataType: DataType = ShortType + + override def nullSafeEval(xml: Any, path: Any): Any = { + val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) + if (ret eq null) null else ret.shortValue() + } +} + +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Returns an integer value that matches the xpath expression", + extended = "> SELECT _FUNC_('12','sum(a/b)');\n3") +case class XPathInt(xml: Expression, path: Expression) extends XPathExtract { + override def prettyName: String = "xpath_int" + override def dataType: DataType = IntegerType + + override def nullSafeEval(xml: Any, path: Any): Any = { + val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) + if (ret eq null) null else ret.intValue() + } +} + +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Returns a long value that matches the xpath expression", + extended = "> SELECT _FUNC_('12','sum(a/b)');\n3") +case class XPathLong(xml: Expression, path: Expression) extends XPathExtract { + override def prettyName: String = "xpath_long" + override def dataType: DataType = LongType + + override def nullSafeEval(xml: Any, path: Any): Any = { + val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) + if (ret eq null) null else ret.longValue() + } +} + +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Returns a float value that matches the xpath expression", + extended = "> SELECT _FUNC_('12','sum(a/b)');\n3.0") +case class XPathFloat(xml: Expression, path: Expression) extends XPathExtract { + override def prettyName: String = "xpath_float" + override def dataType: DataType = FloatType + + override def nullSafeEval(xml: Any, path: Any): Any = { + val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) + if (ret eq null) null else ret.floatValue() + } +} + +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Returns a double value that matches the xpath expression", + extended = "> SELECT _FUNC_('12','sum(a/b)');\n3.0") +case class XPathDouble(xml: Expression, path: Expression) extends XPathExtract { + override def prettyName: String = "xpath_float" + override def dataType: DataType = DoubleType + + override def nullSafeEval(xml: Any, path: Any): Any = { + val ret = xpathUtil.evalNumber(xml.asInstanceOf[UTF8String].toString, pathString) + if (ret eq null) null else ret.doubleValue() + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Returns the text contents of the first xml node that matches the xpath expression", + extended = "> SELECT _FUNC_('bcc','a/c');\ncc") +// scalastyle:on line.size.limit +case class XPathString(xml: Expression, path: Expression) extends XPathExtract { + override def prettyName: String = "xpath_string" + override def dataType: DataType = StringType + + override def nullSafeEval(xml: Any, path: Any): Any = { + val ret = xpathUtil.evalString(xml.asInstanceOf[UTF8String].toString, pathString) + UTF8String.fromString(ret) + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression", + extended = "> SELECT _FUNC_('b1b2b3c1c2','a/b/text()');\n['b1','b2','b3']") +// scalastyle:on line.size.limit +case class XPathList(xml: Expression, path: Expression) extends XPathExtract { + override def prettyName: String = "xpath" + override def dataType: DataType = ArrayType(StringType, containsNull = false) + + override def nullSafeEval(xml: Any, path: Any): Any = { + val nodeList = xpathUtil.evalNodeList(xml.asInstanceOf[UTF8String].toString, pathString) + if (nodeList ne null) { + val ret = new Array[UTF8String](nodeList.getLength) + var i = 0 + while (i < nodeList.getLength) { + ret(i) = UTF8String.fromString(nodeList.item(i).getNodeValue) + i += 1 + } + new GenericArrayData(ret) + } else { + null + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index e58a0df317fe9..58e9d6f8bdf75 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -63,6 +63,10 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { expected.asInstanceOf[Spread[Double]].isWithin(result) case (result: MapData, expected: MapData) => result.keyArray() == expected.keyArray() && result.valueArray() == expected.valueArray() + case (result: Double, expected: Double) => + if (expected.isNaN) result.isNaN else expected == result + case (result: Float, expected: Float) => + if (expected.isNaN) result.isNaN else expected == result case _ => result == expected } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala index a5614f83844e0..c4cde7091154b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala @@ -43,8 +43,9 @@ class UDFXPathUtilSuite extends SparkFunSuite { assert(util.eval("b1b2b3c1c2", "", STRING) == null) // wrong expression: - assert( - util.eval("b1b2b3c1c2", "a/text(", STRING) == null) + intercept[RuntimeException] { + util.eval("b1b2b3c1c2", "a/text(", STRING) + } } test("generic eval") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala index f7c65c667efbd..bfa18a0919e45 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.xml import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.StringType /** @@ -27,35 +26,183 @@ import org.apache.spark.sql.types.StringType */ class XPathExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { - private def testBoolean[T](xml: String, path: String, expected: T): Unit = { - checkEvaluation( - XPathBoolean(Literal.create(xml, StringType), Literal.create(path, StringType)), - expected) + /** A helper function that tests null and error behaviors for xpath expressions. */ + private def testNullAndErrorBehavior[T <: AnyRef](testExpr: (String, String, T) => Unit): Unit = { + // null input should lead to null output + testExpr("b1b2", null, null.asInstanceOf[T]) + testExpr(null, "a", null.asInstanceOf[T]) + testExpr(null, null, null.asInstanceOf[T]) + + // Empty input should also lead to null output + testExpr("", "a", null.asInstanceOf[T]) + testExpr("", "", null.asInstanceOf[T]) + testExpr("", "", null.asInstanceOf[T]) + + // Test error message for invalid XML document + val e1 = intercept[RuntimeException] { testExpr("/a>", "a", null.asInstanceOf[T]) } + assert(e1.getCause.getMessage.contains("Invalid XML document") && + e1.getCause.getMessage.contains("/a>")) + + // Test error message for invalid xpath + val e2 = intercept[RuntimeException] { testExpr("", "!#$", null.asInstanceOf[T]) } + assert(e2.getCause.getMessage.contains("Invalid XPath") && + e2.getCause.getMessage.contains("!#$")) } test("xpath_boolean") { - testBoolean("b", "a/b", true) - testBoolean("b", "a/c", false) - testBoolean("b", "a/b = \"b\"", true) - testBoolean("b", "a/b = \"c\"", false) - testBoolean("10", "a/b < 10", false) - testBoolean("10", "a/b = 10", true) + def testExpr[T](xml: String, path: String, expected: java.lang.Boolean): Unit = { + checkEvaluation( + XPathBoolean(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + testExpr("b", "a/b", true) + testExpr("b", "a/c", false) + testExpr("b", "a/b = \"b\"", true) + testExpr("b", "a/b = \"c\"", false) + testExpr("10", "a/b < 10", false) + testExpr("10", "a/b = 10", true) - // null input - testBoolean(null, null, null) - testBoolean(null, "a", null) - testBoolean("10", null, null) + testNullAndErrorBehavior(testExpr) + } - // exception handling for invalid input - intercept[Exception] { - testBoolean("/a>", "a", null) + test("xpath_short") { + def testExpr[T](xml: String, path: String, expected: java.lang.Short): Unit = { + checkEvaluation( + XPathShort(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) } + + testExpr("this is not a number", "a", 0.toShort) + testExpr("try a boolean", "a = 10", 0.toShort) + testExpr( + "10000248", + "sum(a/b[@class=\"odd\"])", + 10004.toShort) + + testNullAndErrorBehavior(testExpr) } - test("xpath_boolean path cache invalidation") { - // This is a test to ensure the expression is not reusing the path for different strings - val expr = XPathBoolean(Literal("b"), 'path.string.at(0)) - checkEvaluation(expr, true, create_row("a/b")) - checkEvaluation(expr, false, create_row("a/c")) + test("xpath_int") { + def testExpr[T](xml: String, path: String, expected: java.lang.Integer): Unit = { + checkEvaluation( + XPathInt(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + testExpr("this is not a number", "a", 0) + testExpr("try a boolean", "a = 10", 0) + testExpr( + "100000248", + "sum(a/b[@class=\"odd\"])", + 100004) + + testNullAndErrorBehavior(testExpr) + } + + test("xpath_long") { + def testExpr[T](xml: String, path: String, expected: java.lang.Long): Unit = { + checkEvaluation( + XPathLong(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + testExpr("this is not a number", "a", 0L) + testExpr("try a boolean", "a = 10", 0L) + testExpr( + "9000000000248", + "sum(a/b[@class=\"odd\"])", + 9000000004L) + + testNullAndErrorBehavior(testExpr) + } + + test("xpath_float") { + def testExpr[T](xml: String, path: String, expected: java.lang.Float): Unit = { + checkEvaluation( + XPathFloat(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + testExpr("this is not a number", "a", Float.NaN) + testExpr("try a boolean", "a = 10", 0.0F) + testExpr("1248", + "sum(a/b[@class=\"odd\"])", + 5.0F) + + testNullAndErrorBehavior(testExpr) + } + + test("xpath_double") { + def testExpr[T](xml: String, path: String, expected: java.lang.Double): Unit = { + checkEvaluation( + XPathDouble(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + testExpr("this is not a number", "a", Double.NaN) + testExpr("try a boolean", "a = 10", 0.0) + testExpr("1248", + "sum(a/b[@class=\"odd\"])", + 5.0) + + testNullAndErrorBehavior(testExpr) + } + + test("xpath_string") { + def testExpr[T](xml: String, path: String, expected: String): Unit = { + checkEvaluation( + XPathString(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + testExpr("bbcc", "a", "bbcc") + testExpr("bbcc", "a/b", "bb") + testExpr("bbcc", "a/c", "cc") + testExpr("bbcc", "a/d", "") + testExpr("b1b2", "//b", "b1") + testExpr("b1b2", "a/b[1]", "b1") + testExpr("b1b2", "a/b[@id='b_2']", "b2") + + testNullAndErrorBehavior(testExpr) + } + + test("xpath") { + def testExpr[T](xml: String, path: String, expected: Seq[String]): Unit = { + checkEvaluation( + XPathList(Literal.create(xml, StringType), Literal.create(path, StringType)), + expected) + } + + testExpr("b1b2b3c1c2", "a/text()", Seq.empty[String]) + testExpr("b1b2b3c1c2", "a/*/text()", + Seq("b1", "b2", "b3", "c1", "c2")) + testExpr("b1b2b3c1c2", "a/b/text()", + Seq("b1", "b2", "b3")) + testExpr("b1b2b3c1c2", "a/c/text()", Seq("c1", "c2")) + testExpr("b1b2b3c1c2", + "a/*[@class='bb']/text()", Seq("b1", "c1")) + + testNullAndErrorBehavior(testExpr) + } + + test("accept only literal path") { + def testExpr(exprCtor: (Expression, Expression) => Expression): Unit = { + // Validate that literal (technically this is foldable) paths are supported + val litPath = exprCtor(Literal("abcd"), Concat(Literal("/") :: Literal("/") :: Nil)) + assert(litPath.checkInputDataTypes().isSuccess) + + // Validate that non-foldable paths are not supported. + val nonLitPath = exprCtor(Literal("abcd"), NonFoldableLiteral("/")) + assert(nonLitPath.checkInputDataTypes().isFailure) + } + + testExpr(XPathBoolean) + testExpr(XPathShort) + testExpr(XPathInt) + testExpr(XPathLong) + testExpr(XPathFloat) + testExpr(XPathDouble) + testExpr(XPathString) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala new file mode 100644 index 0000000000000..1d33e7970be8e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext + +/** + * End-to-end tests for xpath expressions. + */ +class XPathFunctionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("xpath_boolean") { + val df = Seq("b").toDF("xml") + checkAnswer(df.selectExpr("xpath_boolean(xml, 'a/b')"), Row(true)) + } + + test("xpath_short, xpath_int, xpath_long") { + val df = Seq("12").toDF("xml") + checkAnswer( + df.selectExpr( + "xpath_short(xml, 'sum(a/b)')", + "xpath_int(xml, 'sum(a/b)')", + "xpath_long(xml, 'sum(a/b)')"), + Row(3.toShort, 3, 3L)) + } + + test("xpath_float, xpath_double, xpath_number") { + val df = Seq("1.02.1").toDF("xml") + checkAnswer( + df.selectExpr( + "xpath_float(xml, 'sum(a/b)')", + "xpath_double(xml, 'sum(a/b)')", + "xpath_number(xml, 'sum(a/b)')"), + Row(3.1.toFloat, 3.1, 3.1)) + } + + test("xpath_string") { + val df = Seq("bcc").toDF("xml") + checkAnswer(df.selectExpr("xpath_string(xml, 'a/c')"), Row("cc")) + } + + test("xpath") { + val df = Seq("b1b2b3c1c2").toDF("xml") + checkAnswer(df.selectExpr("xpath(xml, 'a/*/text()')"), Row(Seq("b1", "b2", "b3", "c1", "c2"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala deleted file mode 100644 index 532d48cc265ac..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.spark.sql.test.SharedSQLContext - -/** - * End-to-end tests for XML expressions. - */ -class XmlFunctionsSuite extends QueryTest with SharedSQLContext { - import testImplicits._ - - test("xpath_boolean") { - val df = Seq("b" -> "a/b").toDF("xml", "path") - checkAnswer(df.selectExpr("xpath_boolean(xml, path)"), Row(true)) - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 9c7f461362d84..6f36abc4db0ed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -236,8 +236,6 @@ private[sql] class HiveSessionCatalog( // str_to_map, windowingtablefunction. private val hiveFunctions = Seq( "hash", "java_method", "histogram_numeric", - "percentile", "percentile_approx", "reflect", "str_to_map", - "xpath", "xpath_double", "xpath_float", "xpath_int", "xpath_long", - "xpath_number", "xpath_short", "xpath_string" + "percentile", "percentile_approx", "reflect", "str_to_map" ) } From aea33bf05fef49683eaa858f653aad5a30f37e4a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 11 Jul 2016 22:45:22 +0200 Subject: [PATCH 0939/1470] [SPARK-16458][SQL] SessionCatalog should support `listColumns` for temporary tables ## What changes were proposed in this pull request? Temporary tables are used frequently, but `spark.catalog.listColumns` does not support those tables. This PR make `SessionCatalog` supports temporary table column listing. **Before** ```scala scala> spark.range(10).createOrReplaceTempView("t1") scala> spark.catalog.listTables().collect() res1: Array[org.apache.spark.sql.catalog.Table] = Array(Table[name=`t1`, tableType=`TEMPORARY`, isTemporary=`true`]) scala> spark.catalog.listColumns("t1").collect() org.apache.spark.sql.AnalysisException: Table `t1` does not exist in database `default`.; ``` **After** ``` scala> spark.catalog.listColumns("t1").collect() res2: Array[org.apache.spark.sql.catalog.Column] = Array(Column[name='id', description='id', dataType='bigint', nullable='false', isPartition='false', isBucket='false']) ``` ## How was this patch tested? Pass the Jenkins tests including a new testcase. Author: Dongjoon Hyun Closes #14114 from dongjoon-hyun/SPARK-16458. (cherry picked from commit 840853ed06d63694bf98b21a889a960aac6ac0ac) Signed-off-by: Herman van Hovell --- .../sql/catalyst/catalog/SessionCatalog.scala | 32 ++++++++++++++---- .../catalog/SessionCatalogSuite.scala | 33 +++++++++++++++++++ .../apache/spark/sql/catalog/Catalog.scala | 3 +- .../spark/sql/internal/CatalogImpl.scala | 8 +++-- .../spark/sql/internal/CatalogSuite.scala | 5 +++ 5 files changed, 71 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 0f7e15ff27bff..3a2e574a1d9aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -22,7 +22,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException @@ -253,9 +253,27 @@ class SessionCatalog( def getTableMetadata(name: TableIdentifier): CatalogTable = { val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase)) val table = formatTableName(name.table) - requireDbExists(db) - requireTableExists(TableIdentifier(table, Some(db))) - externalCatalog.getTable(db, table) + val tid = TableIdentifier(table) + if (isTemporaryTable(name)) { + CatalogTable( + identifier = tid, + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = tempTables(table).output.map { c => + CatalogColumn( + name = c.name, + dataType = c.dataType.catalogString, + nullable = c.nullable, + comment = Option(c.name) + ) + }, + properties = Map(), + viewText = None) + } else { + requireDbExists(db) + requireTableExists(TableIdentifier(table, Some(db))) + externalCatalog.getTable(db, table) + } } /** @@ -432,10 +450,10 @@ class SessionCatalog( def tableExists(name: TableIdentifier): Boolean = synchronized { val db = formatDatabaseName(name.database.getOrElse(currentDb)) val table = formatTableName(name.table) - if (name.database.isDefined || !tempTables.contains(table)) { - externalCatalog.tableExists(db, table) + if (isTemporaryTable(name)) { + true } else { - true // it's a temporary table + externalCatalog.tableExists(db, table) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index c8e7c5103b6eb..67ca0aadcc138 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -432,6 +432,39 @@ class SessionCatalogSuite extends SparkFunSuite { assert(catalog.tableExists(TableIdentifier("tbl3"))) } + test("tableExists on temporary views") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempTable = Range(1, 10, 2, 10) + assert(!catalog.tableExists(TableIdentifier("view1"))) + assert(!catalog.tableExists(TableIdentifier("view1", Some("default")))) + catalog.createTempView("view1", tempTable, overrideIfExists = false) + assert(catalog.tableExists(TableIdentifier("view1"))) + assert(!catalog.tableExists(TableIdentifier("view1", Some("default")))) + } + + test("getTableMetadata on temporary views") { + val catalog = new SessionCatalog(newBasicCatalog()) + val tempTable = Range(1, 10, 2, 10) + val m = intercept[AnalysisException] { + catalog.getTableMetadata(TableIdentifier("view1")) + }.getMessage + assert(m.contains("Table or view 'view1' not found in database 'default'")) + + val m2 = intercept[AnalysisException] { + catalog.getTableMetadata(TableIdentifier("view1", Some("default"))) + }.getMessage + assert(m2.contains("Table or view 'view1' not found in database 'default'")) + + catalog.createTempView("view1", tempTable, overrideIfExists = false) + assert(catalog.getTableMetadata(TableIdentifier("view1")).identifier.table == "view1") + assert(catalog.getTableMetadata(TableIdentifier("view1")).schema(0).name == "id") + + val m3 = intercept[AnalysisException] { + catalog.getTableMetadata(TableIdentifier("view1", Some("default"))) + }.getMessage + assert(m3.contains("Table or view 'view1' not found in database 'default'")) + } + test("list tables without pattern") { val catalog = new SessionCatalog(newBasicCatalog()) val tempTable = Range(1, 10, 2, 10) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 91ed9b3258a12..1aed245fdd332 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -85,7 +85,8 @@ abstract class Catalog { def listFunctions(dbName: String): Dataset[Function] /** - * Returns a list of columns for the given table in the current database. + * Returns a list of columns for the given table in the current database or + * the given temporary table. * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 44babcc93a1de..a6ae6fe2aad2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -138,7 +138,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { */ @throws[AnalysisException]("table does not exist") override def listColumns(tableName: String): Dataset[Column] = { - listColumns(currentDatabase, tableName) + listColumns(TableIdentifier(tableName, None)) } /** @@ -147,7 +147,11 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { @throws[AnalysisException]("database or table does not exist") override def listColumns(dbName: String, tableName: String): Dataset[Column] = { requireTableExists(dbName, tableName) - val tableMetadata = sessionCatalog.getTableMetadata(TableIdentifier(tableName, Some(dbName))) + listColumns(TableIdentifier(tableName, Some(dbName))) + } + + private def listColumns(tableIdentifier: TableIdentifier): Dataset[Column] = { + val tableMetadata = sessionCatalog.getTableMetadata(tableIdentifier) val partitionColumnNames = tableMetadata.partitionColumnNames.toSet val bucketColumnNames = tableMetadata.bucketColumnNames.toSet val columns = tableMetadata.schema.map { c => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index d862e4cfa943a..d75df56dd608a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -234,6 +234,11 @@ class CatalogSuite testListColumns("tab1", dbName = None) } + test("list columns in temporary table") { + createTempTable("temp1") + spark.catalog.listColumns("temp1") + } + test("list columns in database") { createDatabase("db1") createTable("tab1", Some("db1")) From b938ca76ebd92e17233addfc29cb7c3692957a7b Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 11 Jul 2016 14:31:11 -0700 Subject: [PATCH 0940/1470] [SPARKR][DOC] SparkR ML user guides update for 2.0 ## What changes were proposed in this pull request? * Update SparkR ML section to make them consistent with SparkR API docs. * Since #13972 adds labelling support for the ```include_example``` Jekyll plugin, so that we can split the single ```ml.R``` example file into multiple line blocks with different labels, and include them in different algorithms/models in the generated HTML page. ## How was this patch tested? Only docs update, manually check the generated docs. Author: Yanbo Liang Closes #14011 from yanboliang/r-user-guide-update. (cherry picked from commit 2ad031be67c7a0f0c4895c084c891330a9ec935e) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/mllib.R | 8 +++++--- docs/sparkr.md | 43 +++++++++++++++++++++++----------------- examples/src/main/r/ml.R | 22 ++++++++++---------- 3 files changed, 41 insertions(+), 32 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 4fe73671f80df..e9fd0c75c1081 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -55,8 +55,9 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' Generalized Linear Models #' -#' Fits generalized linear model against a Spark DataFrame. Users can print, make predictions on the -#' produced model and save the model to the input path. +#' Fits generalized linear model against a Spark DataFrame. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' #' @param data SparkDataFrame for training. #' @param formula A symbolic description of the model to be fitted. Currently only a few formula @@ -270,7 +271,8 @@ setMethod("summary", signature(object = "NaiveBayesModel"), #' K-Means Clustering Model #' #' Fits a k-means clustering model against a Spark DataFrame, similarly to R's kmeans(). -#' Users can print, make predictions on the produced model and save the model to the input path. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' #' @param data SparkDataFrame for training #' @param formula A symbolic description of the model to be fitted. Currently only a few formula diff --git a/docs/sparkr.md b/docs/sparkr.md index 32ef815eb11c4..b4acb230403a2 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -355,32 +355,39 @@ head(teenagers) # Machine Learning -SparkR supports the following Machine Learning algorithms. +SparkR supports the following machine learning algorithms currently: `Generalized Linear Model`, `Accelerated Failure Time (AFT) Survival Regression Model`, `Naive Bayes Model` and `KMeans Model`. +Under the hood, SparkR uses MLlib to train the model. +Users can call `summary` to print a summary of the fitted model, [predict](api/R/predict.html) to make predictions on new data, and [write.ml](api/R/write.ml.html)/[read.ml](api/R/read.ml.html) to save/load fitted models. +SparkR supports a subset of the available R formula operators for model fitting, including ‘~’, ‘.’, ‘:’, ‘+’, and ‘-‘. -* Generalized Linear Regression Model [spark.glm()](api/R/spark.glm.html) -* Naive Bayes [spark.naiveBayes()](api/R/spark.naiveBayes.html) -* KMeans [spark.kmeans()](api/R/spark.kmeans.html) -* AFT Survival Regression [spark.survreg()](api/R/spark.survreg.html) +## Algorithms -[Generalized Linear Regression](api/R/spark.glm.html) can be used to train a model from a specified family. Currently the Gaussian, Binomial, Poisson and Gamma families are supported. We support a subset of the available R formula operators for model fitting, including '~', '.', ':', '+', and '-'. +### Generalized Linear Model -The [summary()](api/R/summary.html) function gives the summary of a model produced by different algorithms listed above. -It produces the similar result compared with R summary function. +[spark.glm()](api/R/spark.glm.html) or [glm()](api/R/glm.html) fits generalized linear model against a Spark DataFrame. +Currently "gaussian", "binomial", "poisson" and "gamma" families are supported. +{% include_example glm r/ml.R %} -## Model persistence +### Accelerated Failure Time (AFT) Survival Regression Model + +[spark.survreg()](api/R/spark.survreg.html) fits an accelerated failure time (AFT) survival regression model on a SparkDataFrame. +Note that the formula of [spark.survreg()](api/R/spark.survreg.html) does not support operator '.' currently. +{% include_example survreg r/ml.R %} + +### Naive Bayes Model -* [write.ml](api/R/write.ml.html) allows users to save a fitted model in a given input path -* [read.ml](api/R/read.ml.html) allows users to read/load the model which was saved using write.ml in a given path +[spark.naiveBayes()](api/R/spark.naiveBayes.html) fits a Bernoulli naive Bayes model against a SparkDataFrame. Only categorical data is supported. +{% include_example naiveBayes r/ml.R %} -Model persistence is supported for all Machine Learning algorithms for all families. +### KMeans Model -The examples below show how to build several models: -* GLM using the Gaussian and Binomial model families -* AFT survival regression model -* Naive Bayes model -* K-Means model +[spark.kmeans()](api/R/spark.kmeans.html) fits a k-means clustering model against a Spark DataFrame, similarly to R's kmeans(). +{% include_example kmeans r/ml.R %} + +## Model persistence -{% include_example r/ml.R %} +The following example shows how to save/load a MLlib model by SparkR. +{% include_example read_write r/ml.R %} # R Function Name Conflicts diff --git a/examples/src/main/r/ml.R b/examples/src/main/r/ml.R index 65242e68b3c81..a8a1274ac902a 100644 --- a/examples/src/main/r/ml.R +++ b/examples/src/main/r/ml.R @@ -24,9 +24,8 @@ library(SparkR) # Initialize SparkSession sparkR.session(appName = "SparkR-ML-example") -# $example on$ ############################ spark.glm and glm ############################################## - +# $example on:glm$ irisDF <- suppressWarnings(createDataFrame(iris)) # Fit a generalized linear model of family "gaussian" with spark.glm gaussianDF <- irisDF @@ -55,8 +54,9 @@ summary(binomialGLM) # Prediction binomialPredictions <- predict(binomialGLM, binomialTestDF) showDF(binomialPredictions) - +# $example off:glm$ ############################ spark.survreg ############################################## +# $example on:survreg$ # Use the ovarian dataset available in R survival package library(survival) @@ -72,9 +72,9 @@ summary(aftModel) # Prediction aftPredictions <- predict(aftModel, aftTestDF) showDF(aftPredictions) - +# $example off:survreg$ ############################ spark.naiveBayes ############################################## - +# $example on:naiveBayes$ # Fit a Bernoulli naive Bayes model with spark.naiveBayes titanic <- as.data.frame(Titanic) titanicDF <- createDataFrame(titanic[titanic$Freq > 0, -5]) @@ -88,9 +88,9 @@ summary(nbModel) # Prediction nbPredictions <- predict(nbModel, nbTestDF) showDF(nbPredictions) - +# $example off:naiveBayes$ ############################ spark.kmeans ############################################## - +# $example on:kmeans$ # Fit a k-means model with spark.kmeans irisDF <- suppressWarnings(createDataFrame(iris)) kmeansDF <- irisDF @@ -107,9 +107,9 @@ showDF(fitted(kmeansModel)) # Prediction kmeansPredictions <- predict(kmeansModel, kmeansTestDF) showDF(kmeansPredictions) - +# $example off:kmeans$ ############################ model read/write ############################################## - +# $example on:read_write$ irisDF <- suppressWarnings(createDataFrame(iris)) # Fit a generalized linear model of family "gaussian" with spark.glm gaussianDF <- irisDF @@ -120,7 +120,7 @@ gaussianGLM <- spark.glm(gaussianDF, Sepal_Length ~ Sepal_Width + Species, famil modelPath <- tempfile(pattern = "ml", fileext = ".tmp") write.ml(gaussianGLM, modelPath) gaussianGLM2 <- read.ml(modelPath) -# $example off$ + # Check model summary summary(gaussianGLM2) @@ -129,7 +129,7 @@ gaussianPredictions <- predict(gaussianGLM2, gaussianTestDF) showDF(gaussianPredictions) unlink(modelPath) - +# $example off:read_write$ ############################ fit models with spark.lapply ##################################### # Perform distributed training of multiple models with spark.lapply From cb463b6db30491e4e881b8fb5981dfdbf9e73d34 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 11 Jul 2016 14:34:48 -0700 Subject: [PATCH 0941/1470] [SPARK-16144][SPARKR] update R API doc for mllib ## What changes were proposed in this pull request? From SPARK-16140/PR #13921 - the issue is we left write.ml doc empty: ![image](https://cloud.githubusercontent.com/assets/8969467/16481934/856dd0ea-3e62-11e6-9474-e4d57d1ca001.png) Here's what I meant as the fix: ![image](https://cloud.githubusercontent.com/assets/8969467/16481943/911f02ec-3e62-11e6-9d68-17363a9f5628.png) ![image](https://cloud.githubusercontent.com/assets/8969467/16481950/9bc057aa-3e62-11e6-8127-54870701c4b1.png) I didn't realize there was already a JIRA on this. mengxr yanboliang ## How was this patch tested? check doc generated. Author: Felix Cheung Closes #13993 from felixcheung/rmllibdoc. (cherry picked from commit 7f38b9d5f469b2550bc481cbf9adb9acc3779712) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/generics.R | 2 -- R/pkg/R/mllib.R | 36 ++++++++++++++++++++++++++++++------ 2 files changed, 30 insertions(+), 8 deletions(-) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index e4ec508795a14..df057bda99f8e 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1255,7 +1255,6 @@ setGeneric("spark.glm", function(data, formula, ...) { standardGeneric("spark.gl #' @export setGeneric("glm") -#' predict #' @rdname predict #' @export setGeneric("predict", function(object, ...) { standardGeneric("predict") }) @@ -1280,7 +1279,6 @@ setGeneric("spark.naiveBayes", function(data, formula, ...) { standardGeneric("s #' @export setGeneric("spark.survreg", function(data, formula, ...) { standardGeneric("spark.survreg") }) -#' write.ml #' @rdname write.ml #' @export setGeneric("write.ml", function(object, path, ...) { standardGeneric("write.ml") }) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index e9fd0c75c1081..94e1f654f5196 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -53,6 +53,29 @@ setClass("AFTSurvivalRegressionModel", representation(jobj = "jobj")) #' @note KMeansModel since 2.0.0 setClass("KMeansModel", representation(jobj = "jobj")) +#' Saves the MLlib model to the input path +#' +#' Saves the MLlib model to the input path. For more information, see the specific +#' MLlib model below. +#' @rdname write.ml +#' @name write.ml +#' @export +#' @seealso \link{spark.glm}, \link{glm} +#' @seealso \link{spark.kmeans}, \link{spark.naiveBayes}, \link{spark.survreg} +#' @seealso \link{read.ml} +NULL + +#' Makes predictions from a MLlib model +#' +#' Makes predictions from a MLlib model. For more information, see the specific +#' MLlib model below. +#' @rdname predict +#' @name predict +#' @export +#' @seealso \link{spark.glm}, \link{glm} +#' @seealso \link{spark.kmeans}, \link{spark.naiveBayes}, \link{spark.survreg} +NULL + #' Generalized Linear Models #' #' Fits generalized linear model against a Spark DataFrame. @@ -146,7 +169,7 @@ setMethod("glm", signature(formula = "formula", family = "ANY", data = "SparkDat }) # Returns the summary of a model produced by glm() or spark.glm(), similarly to R's summary(). -#' + #' @param object A fitted generalized linear model #' @return \code{summary} returns a summary object of the fitted model, a list of components #' including at least the coefficients, null/residual deviance, null/residual degrees @@ -186,7 +209,7 @@ setMethod("summary", signature(object = "GeneralizedLinearRegressionModel"), }) # Prints the summary of GeneralizedLinearRegressionModel -#' + #' @rdname spark.glm #' @param x Summary object of fitted generalized linear model returned by \code{summary} function #' @export @@ -345,7 +368,7 @@ setMethod("fitted", signature(object = "KMeansModel"), }) # Get the summary of a k-means model -#' + #' @param object A fitted k-means model #' @return \code{summary} returns the model's coefficients, size and cluster #' @rdname spark.kmeans @@ -372,7 +395,7 @@ setMethod("summary", signature(object = "KMeansModel"), }) # Predicted values based on a k-means model -#' + #' @return \code{predict} returns the predicted values based on a k-means model #' @rdname spark.kmeans #' @export @@ -465,7 +488,7 @@ setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "c }) # Saves the generalized linear model to the input path. -#' + #' @param path The directory where the model is saved #' @param overwrite Overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. @@ -483,7 +506,7 @@ setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", pat }) # Save fitted MLlib model to the input path -#' + #' @param path The directory where the model is saved #' @param overwrite Overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. @@ -508,6 +531,7 @@ setMethod("write.ml", signature(object = "KMeansModel", path = "character"), #' @rdname read.ml #' @name read.ml #' @export +#' @seealso \link{write.ml} #' @examples #' \dontrun{ #' path <- "path/to/model" From 02d584ccbb95daae3607bd733ab37764ec454c84 Mon Sep 17 00:00:00 2001 From: James Thomas Date: Mon, 11 Jul 2016 17:57:51 -0700 Subject: [PATCH 0942/1470] [SPARK-16114][SQL] structured streaming event time window example ## What changes were proposed in this pull request? A structured streaming example with event time windowing. ## How was this patch tested? Run locally Author: James Thomas Closes #13957 from jjthomas/current. (cherry picked from commit 9e2c763dbb5ac6fc5d2eb0759402504d4b9073a4) Signed-off-by: Tathagata Das --- .../JavaStructuredNetworkWordCount.java | 4 +- ...avaStructuredNetworkWordCountWindowed.java | 116 ++++++++++++++++++ .../streaming/structured_network_wordcount.py | 3 +- .../structured_network_wordcount_windowed.py | 102 +++++++++++++++ .../StructuredNetworkWordCount.scala | 2 +- .../StructuredNetworkWordCountWindowed.scala | 103 ++++++++++++++++ .../sql/execution/streaming/socket.scala | 47 +++++-- .../streaming/TextSocketStreamSuite.scala | 52 +++++++- 8 files changed, 415 insertions(+), 14 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java create mode 100644 examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java index a2cf9389543e8..346d2182c70b0 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java @@ -24,7 +24,7 @@ import java.util.Iterator; /** - * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Counts words in UTF8 encoded, '\n' delimited text received from the network. * * Usage: JavaStructuredNetworkWordCount * and describe the TCP server that Structured Streaming @@ -40,7 +40,7 @@ public final class JavaStructuredNetworkWordCount { public static void main(String[] args) throws Exception { if (args.length < 2) { - System.err.println("Usage: JavaNetworkWordCount "); + System.err.println("Usage: JavaStructuredNetworkWordCount "); System.exit(1); } diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java new file mode 100644 index 0000000000000..557d36cff30d7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql.streaming; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.streaming.StreamingQuery; +import scala.Tuple2; + +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network over a + * sliding window of configurable duration. Each line from the network is tagged + * with a timestamp that is used to determine the windows into which it falls. + * + * Usage: JavaStructuredNetworkWordCountWindowed + * [] + * and describe the TCP server that Structured Streaming + * would connect to receive data. + * gives the size of window, specified as integer number of seconds + * gives the amount of time successive windows are offset from one another, + * given in the same units as above. should be less than or equal to + * . If the two are equal, successive windows have no overlap. If + * is not provided, it defaults to . + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example sql.streaming.JavaStructuredNetworkWordCountWindowed + * localhost 9999 []` + * + * One recommended , pair is 10, 5 + */ +public final class JavaStructuredNetworkWordCountWindowed { + + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: JavaStructuredNetworkWordCountWindowed " + + " []"); + System.exit(1); + } + + String host = args[0]; + int port = Integer.parseInt(args[1]); + int windowSize = Integer.parseInt(args[2]); + int slideSize = (args.length == 3) ? windowSize : Integer.parseInt(args[3]); + if (slideSize > windowSize) { + System.err.println(" must be less than or equal to "); + } + String windowDuration = windowSize + " seconds"; + String slideDuration = slideSize + " seconds"; + + SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCountWindowed") + .getOrCreate(); + + // Create DataFrame representing the stream of input lines from connection to host:port + Dataset> lines = spark + .readStream() + .format("socket") + .option("host", host) + .option("port", port) + .option("includeTimestamp", true) + .load().as(Encoders.tuple(Encoders.STRING(), Encoders.TIMESTAMP())); + + // Split the lines into words, retaining timestamps + Dataset words = lines.flatMap( + new FlatMapFunction, Tuple2>() { + @Override + public Iterator> call(Tuple2 t) { + List> result = new ArrayList<>(); + for (String word : t._1.split(" ")) { + result.add(new Tuple2<>(word, t._2)); + } + return result.iterator(); + } + }, + Encoders.tuple(Encoders.STRING(), Encoders.TIMESTAMP()) + ).toDF("word", "timestamp"); + + // Group the data by window and word and compute the count of each group + Dataset windowedCounts = words.groupBy( + functions.window(words.col("timestamp"), windowDuration, slideDuration), + words.col("word") + ).count().orderBy("window"); + + // Start running the query that prints the windowed word counts to the console + StreamingQuery query = windowedCounts.writeStream() + .outputMode("complete") + .format("console") + .option("truncate", "false") + .start(); + + query.awaitTermination(); + } +} diff --git a/examples/src/main/python/sql/streaming/structured_network_wordcount.py b/examples/src/main/python/sql/streaming/structured_network_wordcount.py index 32d63c52c9191..afde2550587ca 100644 --- a/examples/src/main/python/sql/streaming/structured_network_wordcount.py +++ b/examples/src/main/python/sql/streaming/structured_network_wordcount.py @@ -16,7 +16,7 @@ # """ - Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Counts words in UTF8 encoded, '\n' delimited text received from the network. Usage: structured_network_wordcount.py and describe the TCP server that Structured Streaming would connect to receive data. @@ -58,6 +58,7 @@ # Split the lines into words words = lines.select( + # explode turns each item in an array into a separate row explode( split(lines.value, ' ') ).alias('word') diff --git a/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py b/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py new file mode 100644 index 0000000000000..02a7d3363d780 --- /dev/null +++ b/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py @@ -0,0 +1,102 @@ +# +# 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. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network over a + sliding window of configurable duration. Each line from the network is tagged + with a timestamp that is used to determine the windows into which it falls. + + Usage: structured_network_wordcount_windowed.py + [] + and describe the TCP server that Structured Streaming + would connect to receive data. + gives the size of window, specified as integer number of seconds + gives the amount of time successive windows are offset from one another, + given in the same units as above. should be less than or equal to + . If the two are equal, successive windows have no overlap. If + is not provided, it defaults to . + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit + examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py + localhost 9999 []` + + One recommended , pair is 10, 5 +""" +from __future__ import print_function + +import sys + +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split +from pyspark.sql.functions import window + +if __name__ == "__main__": + if len(sys.argv) != 5 and len(sys.argv) != 4: + msg = ("Usage: structured_network_wordcount_windowed.py " + " []") + print(msg, file=sys.stderr) + exit(-1) + + host = sys.argv[1] + port = int(sys.argv[2]) + windowSize = int(sys.argv[3]) + slideSize = int(sys.argv[4]) if (len(sys.argv) == 5) else windowSize + if slideSize > windowSize: + print(" must be less than or equal to ", file=sys.stderr) + windowDuration = '{} seconds'.format(windowSize) + slideDuration = '{} seconds'.format(slideSize) + + spark = SparkSession\ + .builder\ + .appName("StructuredNetworkWordCountWindowed")\ + .getOrCreate() + + # Create DataFrame representing the stream of input lines from connection to host:port + lines = spark\ + .readStream\ + .format('socket')\ + .option('host', host)\ + .option('port', port)\ + .option('includeTimestamp', 'true')\ + .load() + + # Split the lines into words, retaining timestamps + # split() splits each line into an array, and explode() turns the array into multiple rows + words = lines.select( + explode(split(lines.value, ' ')).alias('word'), + lines.timestamp + ) + + # Group the data by window and word and compute the count of each group + windowedCounts = words.groupBy( + window(words.timestamp, windowDuration, slideDuration), + words.word + ).count().orderBy('window') + + # Start running the query that prints the windowed word counts to the console + query = windowedCounts\ + .writeStream\ + .outputMode('complete')\ + .format('console')\ + .option('truncate', 'false')\ + .start() + + query.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala index 433f7a181bbf8..364bff227bc55 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.SparkSession /** - * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Counts words in UTF8 encoded, '\n' delimited text received from the network. * * Usage: StructuredNetworkWordCount * and describe the TCP server that Structured Streaming diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala new file mode 100644 index 0000000000000..333b0a9d24f40 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala @@ -0,0 +1,103 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.sql.streaming + +import java.sql.Timestamp + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions._ + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network over a + * sliding window of configurable duration. Each line from the network is tagged + * with a timestamp that is used to determine the windows into which it falls. + * + * Usage: StructuredNetworkWordCountWindowed + * [] + * and describe the TCP server that Structured Streaming + * would connect to receive data. + * gives the size of window, specified as integer number of seconds + * gives the amount of time successive windows are offset from one another, + * given in the same units as above. should be less than or equal to + * . If the two are equal, successive windows have no overlap. If + * is not provided, it defaults to . + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example sql.streaming.StructuredNetworkWordCountWindowed + * localhost 9999 []` + * + * One recommended , pair is 10, 5 + */ +object StructuredNetworkWordCountWindowed { + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: StructuredNetworkWordCountWindowed " + + " []") + System.exit(1) + } + + val host = args(0) + val port = args(1).toInt + val windowSize = args(2).toInt + val slideSize = if (args.length == 3) windowSize else args(3).toInt + if (slideSize > windowSize) { + System.err.println(" must be less than or equal to ") + } + val windowDuration = s"$windowSize seconds" + val slideDuration = s"$slideSize seconds" + + val spark = SparkSession + .builder + .appName("StructuredNetworkWordCountWindowed") + .getOrCreate() + + import spark.implicits._ + + // Create DataFrame representing the stream of input lines from connection to host:port + val lines = spark.readStream + .format("socket") + .option("host", host) + .option("port", port) + .option("includeTimestamp", true) + .load().as[(String, Timestamp)] + + // Split the lines into words, retaining timestamps + val words = lines.flatMap(line => + line._1.split(" ").map(word => (word, line._2)) + ).toDF("word", "timestamp") + + // Group the data by window and word and compute the count of each group + val windowedCounts = words.groupBy( + window($"timestamp", windowDuration, slideDuration), $"word" + ).count().orderBy("window") + + // Start running the query that prints the windowed word counts to the console + val query = windowedCounts.writeStream + .outputMode("complete") + .format("console") + .option("truncate", "false") + .start() + + query.awaitTermination() + } +} +// scalastyle:on println diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala index d07d88dcdcc44..fb15239f9af98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala @@ -19,17 +19,24 @@ package org.apache.spark.sql.execution.streaming import java.io.{BufferedReader, InputStreamReader, IOException} import java.net.Socket +import java.sql.Timestamp +import java.text.SimpleDateFormat +import java.util.Calendar import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.ArrayBuffer +import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} object TextSocketSource { - val SCHEMA = StructType(StructField("value", StringType) :: Nil) + val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil) + val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) :: + StructField("timestamp", TimestampType) :: Nil) + val DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } /** @@ -37,7 +44,7 @@ object TextSocketSource { * This source will *not* work in production applications due to multiple reasons, including no * support for fault recovery and keeping all of the text read in memory forever. */ -class TextSocketSource(host: String, port: Int, sqlContext: SQLContext) +class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlContext: SQLContext) extends Source with Logging { @GuardedBy("this") @@ -47,7 +54,7 @@ class TextSocketSource(host: String, port: Int, sqlContext: SQLContext) private var readThread: Thread = null @GuardedBy("this") - private var lines = new ArrayBuffer[String] + private var lines = new ArrayBuffer[(String, Timestamp)] initialize() @@ -67,7 +74,10 @@ class TextSocketSource(host: String, port: Int, sqlContext: SQLContext) return } TextSocketSource.this.synchronized { - lines += line + lines += ((line, + Timestamp.valueOf( + TextSocketSource.DATE_FORMAT.format(Calendar.getInstance().getTime())) + )) } } } catch { @@ -79,7 +89,8 @@ class TextSocketSource(host: String, port: Int, sqlContext: SQLContext) } /** Returns the schema of the data from this source */ - override def schema: StructType = TextSocketSource.SCHEMA + override def schema: StructType = if (includeTimestamp) TextSocketSource.SCHEMA_TIMESTAMP + else TextSocketSource.SCHEMA_REGULAR /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = synchronized { @@ -92,7 +103,11 @@ class TextSocketSource(host: String, port: Int, sqlContext: SQLContext) val endIdx = end.asInstanceOf[LongOffset].offset.toInt + 1 val data = synchronized { lines.slice(startIdx, endIdx) } import sqlContext.implicits._ - data.toDF("value") + if (includeTimestamp) { + data.toDF("value", "timestamp") + } else { + data.map(_._1).toDF("value") + } } /** Stop this source. */ @@ -111,6 +126,14 @@ class TextSocketSource(host: String, port: Int, sqlContext: SQLContext) } class TextSocketSourceProvider extends StreamSourceProvider with DataSourceRegister with Logging { + private def parseIncludeTimestamp(params: Map[String, String]): Boolean = { + Try(params.getOrElse("includeTimestamp", "false").toBoolean) match { + case Success(bool) => bool + case Failure(_) => + throw new AnalysisException("includeTimestamp must be set to either \"true\" or \"false\"") + } + } + /** Returns the name and schema of the source that can be used to continually read data. */ override def sourceSchema( sqlContext: SQLContext, @@ -125,7 +148,13 @@ class TextSocketSourceProvider extends StreamSourceProvider with DataSourceRegis if (!parameters.contains("port")) { throw new AnalysisException("Set a port to read from with option(\"port\", ...).") } - ("textSocket", TextSocketSource.SCHEMA) + val schema = + if (parseIncludeTimestamp(parameters)) { + TextSocketSource.SCHEMA_TIMESTAMP + } else { + TextSocketSource.SCHEMA_REGULAR + } + ("textSocket", schema) } override def createSource( @@ -136,7 +165,7 @@ class TextSocketSourceProvider extends StreamSourceProvider with DataSourceRegis parameters: Map[String, String]): Source = { val host = parameters("host") val port = parameters("port").toInt - new TextSocketSource(host, port, sqlContext) + new TextSocketSource(host, port, parseIncludeTimestamp(parameters), sqlContext) } /** String that represents the format that this data source provider uses. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala index ca577631854ef..6b0ba7acb4804 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming import java.io.{IOException, OutputStreamWriter} import java.net.ServerSocket +import java.sql.Timestamp import java.util.concurrent.LinkedBlockingQueue import org.scalatest.BeforeAndAfterEach @@ -27,7 +28,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} class TextSocketStreamSuite extends StreamTest with SharedSQLContext with BeforeAndAfterEach { import testImplicits._ @@ -85,6 +86,47 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before } } + test("timestamped usage") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString, + "includeTimestamp" -> "true") + val schema = provider.sourceSchema(sqlContext, None, "", parameters)._2 + assert(schema === StructType(StructField("value", StringType) :: + StructField("timestamp", TimestampType) :: Nil)) + + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val offset1 = source.getOffset.get + val batch1 = source.getBatch(None, offset1) + val batch1Seq = batch1.as[(String, Timestamp)].collect().toSeq + assert(batch1Seq.map(_._1) === Seq("hello")) + val batch1Stamp = batch1Seq(0)._2 + + serverThread.enqueue("world") + while (source.getOffset.get === offset1) { + Thread.sleep(10) + } + val offset2 = source.getOffset.get + val batch2 = source.getBatch(Some(offset1), offset2) + val batch2Seq = batch2.as[(String, Timestamp)].collect().toSeq + assert(batch2Seq.map(_._1) === Seq("world")) + val batch2Stamp = batch2Seq(0)._2 + assert(!batch2Stamp.before(batch1Stamp)) + + // Try stopping the source to make sure this does not block forever. + source.stop() + source = null + } + } + test("params not given") { val provider = new TextSocketSourceProvider intercept[AnalysisException] { @@ -98,6 +140,14 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before } } + test("non-boolean includeTimestamp") { + val provider = new TextSocketSourceProvider + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map("host" -> "localhost", + "port" -> "1234", "includeTimestamp" -> "fasle")) + } + } + test("no server up") { val provider = new TextSocketSourceProvider val parameters = Map("host" -> "localhost", "port" -> "0") From 81d7f484ac3b68792e49e47c2b7c9994cf17487a Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Mon, 11 Jul 2016 18:09:14 -0700 Subject: [PATCH 0943/1470] [MINOR][STREAMING][DOCS] Minor changes on kinesis integration ## What changes were proposed in this pull request? Some minor changes for documentation page "Spark Streaming + Kinesis Integration". Moved "streaming-kinesis-arch.png" before the bullet list, not in between the bullets. ## How was this patch tested? Tested manually, on my local machine. Author: Xin Ren Closes #14097 from keypointt/kinesisDoc. (cherry picked from commit 05d7151ccbccdd977ec2f2301d5b12566018c988) Signed-off-by: Tathagata Das --- docs/streaming-kinesis-integration.md | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 5b9a7554d2e64..96198ddf537b6 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -111,7 +111,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m - `[checkpoint interval]`: The interval (e.g., Duration(2000) = 2 seconds) at which the Kinesis Client Library saves its position in the stream. For starters, set it to the same as the batch interval of the streaming application. - - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see Kinesis Checkpointing section and Amazon Kinesis API documentation for more details). + - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see [`Kinesis Checkpointing`](#kinesis-checkpointing) section and [`Amazon Kinesis API documentation`](http://docs.aws.amazon.com/streams/latest/dev/developing-consumers-with-sdk.html) for more details). - `[message handler]`: A function that takes a Kinesis `Record` and outputs generic `T`. @@ -128,14 +128,6 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kinesis-asl-assembly` from the [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kinesis-asl-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. - *Points to remember at runtime:* - - - Kinesis data processing is ordered per partition and occurs at-least once per message. - - - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamoDB. - - - A single Kinesis stream shard is processed by one input DStream at a time. -

      + *Points to remember at runtime:* + + - Kinesis data processing is ordered per partition and occurs at-least once per message. + + - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamoDB. + + - A single Kinesis stream shard is processed by one input DStream at a time. + - A single Kinesis input DStream can read from multiple shards of a Kinesis stream by creating multiple KinesisRecordProcessor threads. - Multiple input DStreams running in separate processes/instances can read from a Kinesis stream. @@ -173,7 +173,7 @@ To run the example, - Set up Kinesis stream (see earlier section) within AWS. Note the name of the Kinesis stream and the endpoint URL corresponding to the region where the stream was created. -- Set up the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_KEY with your AWS credentials. +- Set up the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_KEY` with your AWS credentials. - In the Spark root directory, run the example as @@ -216,6 +216,6 @@ de-aggregate records during consumption. - Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. -- If no Kinesis checkpoint info exists when the input DStream starts, it will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPositionInStream.LATEST). This is configurable. -- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no input DStreams are running (and no checkpoint info is being stored). -- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. +- If no Kinesis checkpoint info exists when the input DStream starts, it will start either from the oldest record available (`InitialPositionInStream.TRIM_HORIZON`) or from the latest tip (`InitialPositionInStream.LATEST`). This is configurable. + - `InitialPositionInStream.LATEST` could lead to missed records if data is added to the stream while no input DStreams are running (and no checkpoint info is being stored). + - `InitialPositionInStream.TRIM_HORIZON` may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. From b716e104b917a598d4e56abcfa1517a36b9232a6 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 11 Jul 2016 18:11:06 -0700 Subject: [PATCH 0944/1470] [SPARK-16433][SQL] Improve StreamingQuery.explain when no data arrives ## What changes were proposed in this pull request? Display `No physical plan. Waiting for data.` instead of `N/A` for StreamingQuery.explain when no data arrives because `N/A` doesn't provide meaningful information. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu Closes #14100 from zsxwing/SPARK-16433. (cherry picked from commit 91a443b849e4d1ccc50a32b25fdd2bb502cf9b84) Signed-off-by: Tathagata Das --- .../spark/sql/execution/streaming/StreamExecution.scala | 2 +- .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 4 ++-- .../scala/org/apache/spark/sql/streaming/StreamSuite.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index f1af79e738faf..c90dcc5680743 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -477,7 +477,7 @@ class StreamExecution( /** Expose for tests */ def explainInternal(extended: Boolean): String = { if (lastExecution == null) { - "N/A" + "No physical plan. Waiting for data." } else { val explain = ExplainCommand(lastExecution.logical, extended = extended) sparkSession.sessionState.executePlan(explain).executedPlan.executeCollect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 29ce578bcde34..3d28d4f99cfc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -672,8 +672,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val q = df.writeStream.queryName("file_explain").format("memory").start() .asInstanceOf[StreamExecution] try { - assert("N/A" === q.explainInternal(false)) - assert("N/A" === q.explainInternal(true)) + assert("No physical plan. Waiting for data." === q.explainInternal(false)) + assert("No physical plan. Waiting for data." === q.explainInternal(true)) val tempFile = Utils.tempFileWith(new File(tmp, "text")) val finalFile = new File(src, tempFile.getName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 28170f30646ab..1caafb9d74440 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -251,8 +251,8 @@ class StreamSuite extends StreamTest { val q = df.writeStream.queryName("memory_explain").format("memory").start() .asInstanceOf[StreamExecution] try { - assert("N/A" === q.explainInternal(false)) - assert("N/A" === q.explainInternal(true)) + assert("No physical plan. Waiting for data." === q.explainInternal(false)) + assert("No physical plan. Waiting for data." === q.explainInternal(true)) inputData.addData("abc") q.processAllAvailable() From b37177c22f5c0f927b8d9f3a38dba9617d36c944 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 11 Jul 2016 18:41:36 -0700 Subject: [PATCH 0945/1470] [SPARK-16430][SQL][STREAMING] Fixed bug in the maxFilesPerTrigger in FileStreamSource ## What changes were proposed in this pull request? Incorrect list of files were being allocated to a batch. This caused a file to read multiple times in the multiple batches. ## How was this patch tested? Added unit tests Author: Tathagata Das Closes #14143 from tdas/SPARK-16430-1. (cherry picked from commit e50efd53f073890d789a8448f850cc219cca7708) Signed-off-by: Tathagata Das --- .../streaming/FileStreamSource.scala | 6 ++-- .../sql/streaming/FileStreamSourceSuite.scala | 35 +++++++++++++++++-- 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 72b335a42ed34..0cfad659dc92c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -73,8 +73,8 @@ class FileStreamSource( logTrace(s"Number of seen files = ${seenFiles.size}") if (batchFiles.nonEmpty) { maxBatchId += 1 - metadataLog.add(maxBatchId, newFiles) - logInfo(s"Max batch id increased to $maxBatchId with ${newFiles.size} new files") + metadataLog.add(maxBatchId, batchFiles) + logInfo(s"Max batch id increased to $maxBatchId with ${batchFiles.size} new files") } new LongOffset(maxBatchId) @@ -138,7 +138,7 @@ class FileStreamSource( .map { str => Try(str.toInt).toOption.filter(_ > 0).getOrElse { throw new IllegalArgumentException( - s"Invalid value '$str' for option 'maxFilesPerBatch', must be a positive integer") + s"Invalid value '$str' for option 'maxFilesPerTrigger', must be a positive integer") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 3d28d4f99cfc1..47260a23c7ee3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -627,6 +627,13 @@ class FileStreamSourceSuite extends FileStreamSourceTest { checkAnswer(df, data.map(_.toString).toDF("value")) } + def checkAllData(data: Seq[Int]): Unit = { + val schema = StructType(Seq(StructField("value", StringType))) + val df = spark.createDataFrame( + spark.sparkContext.makeRDD(memorySink.allData), schema) + checkAnswer(df, data.map(_.toString).toDF("value")) + } + /** Check how many batches have executed since the last time this check was made */ var lastBatchId = -1L def checkNumBatchesSinceLastCheck(numBatches: Int): Unit = { @@ -636,6 +643,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } checkLastBatchData(3) // (1 and 2) should be in batch 1, (3) should be in batch 2 (last) + checkAllData(1 to 3) lastBatchId = memorySink.latestBatchId.get fileSource.withBatchingLocked { @@ -645,8 +653,9 @@ class FileStreamSourceSuite extends FileStreamSourceTest { createFile(7) // 6 and 7 should be in the last batch } q.processAllAvailable() - checkLastBatchData(6, 7) checkNumBatchesSinceLastCheck(2) + checkLastBatchData(6, 7) + checkAllData(1 to 7) fileSource.withBatchingLocked { createFile(8) @@ -656,8 +665,30 @@ class FileStreamSourceSuite extends FileStreamSourceTest { createFile(12) // 12 should be in the last batch } q.processAllAvailable() - checkLastBatchData(12) checkNumBatchesSinceLastCheck(3) + checkLastBatchData(12) + checkAllData(1 to 12) + + q.stop() + } + } + + test("max files per trigger - incorrect values") { + withTempDir { case src => + def testMaxFilePerTriggerValue(value: String): Unit = { + val df = spark.readStream.option("maxFilesPerTrigger", value).text(src.getCanonicalPath) + val e = intercept[IllegalArgumentException] { + testStream(df)() + } + Seq("maxFilesPerTrigger", value, "positive integer").foreach { s => + assert(e.getMessage.contains(s)) + } + } + + testMaxFilePerTriggerValue("not-a-integer") + testMaxFilePerTriggerValue("-1") + testMaxFilePerTriggerValue("0") + testMaxFilePerTriggerValue("10.1") } } From 689261465ad1dd443ebf764ad837243418b986ef Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Mon, 11 Jul 2016 20:26:01 -0700 Subject: [PATCH 0946/1470] [SPARK-16488] Fix codegen variable namespace collision in pmod and partitionBy This patch fixes a variable namespace collision bug in pmod and partitionBy Regression test for one possible occurrence. A more general fix in `ExpressionEvalHelper.checkEvaluation` will be in a subsequent PR. Author: Sameer Agarwal Closes #14144 from sameeragarwal/codegen-bug. (cherry picked from commit 9cc74f95edb6e4f56151966139cd0dc24e377949) Signed-off-by: Reynold Xin --- .../sql/catalyst/expressions/arithmetic.scala | 25 ++++++++++--------- .../sql/test/DataFrameReaderWriterSuite.scala | 14 +++++++++++ 2 files changed, 27 insertions(+), 12 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 4db1352291e0b..91ffac0ba2a60 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 @@ -498,34 +498,35 @@ case class Pmod(left: Expression, right: Expression) extends BinaryArithmetic wi override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, (eval1, eval2) => { + val remainder = ctx.freshName("remainder") dataType match { case dt: DecimalType => val decimalAdd = "$plus" s""" - ${ctx.javaType(dataType)} r = $eval1.remainder($eval2); - if (r.compare(new org.apache.spark.sql.types.Decimal().set(0)) < 0) { - ${ev.value} = (r.$decimalAdd($eval2)).remainder($eval2); + ${ctx.javaType(dataType)} $remainder = $eval1.remainder($eval2); + if ($remainder.compare(new org.apache.spark.sql.types.Decimal().set(0)) < 0) { + ${ev.value} = ($remainder.$decimalAdd($eval2)).remainder($eval2); } else { - ${ev.value} = r; + ${ev.value} = $remainder; } """ // byte and short are casted into int when add, minus, times or divide case ByteType | ShortType => s""" - ${ctx.javaType(dataType)} r = (${ctx.javaType(dataType)})($eval1 % $eval2); - if (r < 0) { - ${ev.value} = (${ctx.javaType(dataType)})((r + $eval2) % $eval2); + ${ctx.javaType(dataType)} $remainder = (${ctx.javaType(dataType)})($eval1 % $eval2); + if ($remainder < 0) { + ${ev.value} = (${ctx.javaType(dataType)})(($remainder + $eval2) % $eval2); } else { - ${ev.value} = r; + ${ev.value} = $remainder; } """ case _ => s""" - ${ctx.javaType(dataType)} r = $eval1 % $eval2; - if (r < 0) { - ${ev.value} = (r + $eval2) % $eval2; + ${ctx.javaType(dataType)} $remainder = $eval1 % $eval2; + if ($remainder < 0) { + ${ev.value} = ($remainder + $eval2) % $eval2; } else { - ${ev.value} = r; + ${ev.value} = $remainder; } """ } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 27a0a2a776c36..75834a3a4056b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -424,6 +424,20 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be spark.range(10).write.orc(dir) } + test("pmod with partitionBy") { + val spark = this.spark + import spark.implicits._ + + case class Test(a: Int, b: String) + val data = Seq((0, "a"), (1, "b"), (1, "a")) + spark.createDataset(data).createOrReplaceTempView("test") + sql("select * from test distribute by pmod(_1, 2)") + .write + .partitionBy("_2") + .mode("overwrite") + .parquet(dir) + } + private def testRead( df: => DataFrame, expectedResult: Seq[String], From 9e0d2e22637f6cef2ab91aadcdeb8f06f677e397 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 12 Jul 2016 09:23:59 +0100 Subject: [PATCH 0947/1470] [MINOR][ML] update comment where is inconsistent with code in ml.regression.LinearRegression ## What changes were proposed in this pull request? In `train` method of `ml.regression.LinearRegression` when handling situation `std(label) == 0` the code replace `std(label)` with `mean(label)` but the relative comment is inconsistent, I update it. ## How was this patch tested? N/A Author: WeichenXu Closes #14121 from WeichenXu123/update_lr_comment. (cherry picked from commit fc11c509e234c5414687f7fbd13af113a1f52f10) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/ml/regression/LinearRegression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0a4d98cab64aa..0477f71f32043 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 @@ -263,7 +263,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String } // if y is constant (rawYStd is zero), then y cannot be scaled. In this case - // setting yStd=1.0 ensures that y is not scaled anymore in l-bfgs algorithm. + // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) val featuresMean = featuresSummarizer.mean.toArray val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) From 7b63e7d924cb82db37fe5d0f9b35f556bab37d39 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 12 Jul 2016 13:04:34 +0100 Subject: [PATCH 0948/1470] [SPARK-16470][ML][OPTIMIZER] Check linear regression training whether actually reach convergence and add warning if not ## What changes were proposed in this pull request? In `ml.regression.LinearRegression`, it use breeze `LBFGS` and `OWLQN` optimizer to do data training, but do not check whether breeze's optimizer returned result actually reached convergence. The `LBFGS` and `OWLQN` optimizer in breeze finish iteration may result the following situations: 1) reach max iteration number 2) function reach value convergence 3) objective function stop improving 4) gradient reach convergence 5) search failed(due to some internal numerical error) I add warning printing code so that if the iteration result is (1) or (3) or (5) in above, it will print a warning with respective reason string. ## How was this patch tested? Manual. Author: WeichenXu Closes #14122 from WeichenXu123/add_lr_not_convergence_warn. (cherry picked from commit 6cb75db9ab1a4f227069bec2763b89546b88b0ee) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/regression/LinearRegression.scala | 5 +++++ 1 file changed, 5 insertions(+) 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 0477f71f32043..6b82ae14e1cef 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 @@ -327,6 +327,11 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String throw new SparkException(msg) } + if (!state.actuallyConverged) { + logWarning("LinearRegression training fininshed but the result " + + s"is not converged because: ${state.convergedReason.get.reason}") + } + /* The coefficients are trained in the scaled space; we're converting them back to the original space. From f419476546f133040a21d7662b6509185f1a5d53 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Jul 2016 10:07:23 -0700 Subject: [PATCH 0949/1470] [SPARK-16489][SQL] Guard against variable reuse mistakes in expression code generation In code generation, it is incorrect for expressions to reuse variable names across different instances of itself. As an example, SPARK-16488 reports a bug in which pmod expression reuses variable name "r". This patch updates ExpressionEvalHelper test harness to always project two instances of the same expression, which will help us catch variable reuse problems in expression unit tests. This patch also fixes the bug in crc32 expression. This is a test harness change, but I also created a new test suite for testing the test harness. Author: Reynold Xin Closes #14146 from rxin/SPARK-16489. (cherry picked from commit c377e49e38a290e5c4fbc178278069788674dfb7) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/expressions/misc.scala | 7 +-- .../expressions/ExpressionEvalHelper.scala | 15 ++++-- .../ExpressionEvalHelperSuite.scala | 54 +++++++++++++++++++ .../sql/test/DataFrameReaderWriterSuite.scala | 14 ----- 4 files changed, 68 insertions(+), 22 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 1c0787bf9227f..d2c94ec1df4d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -175,11 +175,12 @@ case class Crc32(child: Expression) extends UnaryExpression with ImplicitCastInp override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val CRC32 = "java.util.zip.CRC32" + val checksum = ctx.freshName("checksum") nullSafeCodeGen(ctx, ev, value => { s""" - $CRC32 checksum = new $CRC32(); - checksum.update($value, 0, $value.length); - ${ev.value} = checksum.getValue(); + $CRC32 $checksum = new $CRC32(); + $checksum.update($value, 0, $value.length); + ${ev.value} = $checksum.getValue(); """ }) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 58e9d6f8bdf75..d6a9672d1f186 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -132,9 +132,13 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { expression: Expression, expected: Any, inputRow: InternalRow = EmptyRow): Unit = { - + // SPARK-16489 Explicitly doing code generation twice so code gen will fail if + // some expression is reusing variable names across different instances. + // This behavior is tested in ExpressionEvalHelperSuite. val plan = generateProject( - GenerateUnsafeProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), + GenerateUnsafeProjection.generate( + Alias(expression, s"Optimized($expression)1")() :: + Alias(expression, s"Optimized($expression)2")() :: Nil), expression) val unsafeRow = plan(inputRow) @@ -142,13 +146,14 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { if (expected == null) { if (!unsafeRow.isNullAt(0)) { - val expectedRow = InternalRow(expected) + val expectedRow = InternalRow(expected, expected) fail("Incorrect evaluation in unsafe mode: " + s"$expression, actual: $unsafeRow, expected: $expectedRow$input") } } else { - val lit = InternalRow(expected) - val expectedRow = UnsafeProjection.create(Array(expression.dataType)).apply(lit) + val lit = InternalRow(expected, expected) + val expectedRow = + UnsafeProjection.create(Array(expression.dataType, expression.dataType)).apply(lit) if (unsafeRow != expectedRow) { fail("Incorrect evaluation in unsafe mode: " + s"$expression, actual: $unsafeRow, expected: $expectedRow$input") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala new file mode 100644 index 0000000000000..64b65e2070ed6 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.types.{DataType, IntegerType} + +/** + * A test suite for testing [[ExpressionEvalHelper]]. + * + * Yes, we should write test cases for test harnesses, in case + * they have behaviors that are easy to break. + */ +class ExpressionEvalHelperSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("SPARK-16489 checkEvaluation should fail if expression reuses variable names") { + val e = intercept[RuntimeException] { checkEvaluation(BadCodegenExpression(), 10) } + assert(e.getMessage.contains("some_variable")) + } +} + +/** + * An expression that generates bad code (variable name "some_variable" is not unique across + * instances of the expression. + */ +case class BadCodegenExpression() extends LeafExpression { + override def nullable: Boolean = false + override def eval(input: InternalRow): Any = 10 + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + ev.copy(code = + s""" + |int some_variable = 11; + |int ${ev.value} = 10; + """.stripMargin) + } + override def dataType: DataType = IntegerType +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 75834a3a4056b..27a0a2a776c36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -424,20 +424,6 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be spark.range(10).write.orc(dir) } - test("pmod with partitionBy") { - val spark = this.spark - import spark.implicits._ - - case class Test(a: Int, b: String) - val data = Seq((0, "a"), (1, "b"), (1, "a")) - spark.createDataset(data).createOrReplaceTempView("test") - sql("select * from test distribute by pmod(_1, 2)") - .write - .partitionBy("_2") - .mode("overwrite") - .parquet(dir) - } - private def testRead( df: => DataFrame, expectedResult: Seq[String], From 2f47b37784288b533405d7ef1cad1f7bac324ec0 Mon Sep 17 00:00:00 2001 From: sharkd Date: Tue, 12 Jul 2016 10:10:35 -0700 Subject: [PATCH 0950/1470] [SPARK-16414][YARN] Fix bugs for "Can not get user config when calling SparkHadoopUtil.get.conf on yarn cluser mode" ## What changes were proposed in this pull request? The `SparkHadoopUtil` singleton was instantiated before `ApplicationMaster` in `ApplicationMaster.main` when deploying spark on yarn cluster mode, the `conf` in the `SparkHadoopUtil` singleton didn't include user's configuration. So, we should load the properties file with the Spark configuration and set entries as system properties before `SparkHadoopUtil` first instantiate. ## How was this patch tested? Add a test case Author: sharkd Author: sharkdtu Closes #14088 from sharkdtu/master. (cherry picked from commit d513c99c19e229f72d03006e251725a43c13fefd) --- .../spark/deploy/yarn/ApplicationMaster.scala | 17 +++---- .../spark/deploy/yarn/YarnClusterSuite.scala | 45 +++++++++++++++++++ 2 files changed, 54 insertions(+), 8 deletions(-) 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 b6f45dd63473b..c371ad616a47a 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 @@ -50,14 +50,6 @@ private[spark] class ApplicationMaster( client: YarnRMClient) extends Logging { - // Load the properties file with the Spark configuration and set entries as system properties, - // so that user code run inside the AM also has access to them. - if (args.propertiesFile != null) { - Utils.getPropertiesFromFile(args.propertiesFile).foreach { case (k, v) => - sys.props(k) = v - } - } - // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. @@ -743,6 +735,15 @@ object ApplicationMaster extends Logging { def main(args: Array[String]): Unit = { SignalUtils.registerLogger(log) val amArgs = new ApplicationMasterArguments(args) + + // Load the properties file with the Spark configuration and set entries as system properties, + // so that user code run inside the AM also has access to them. + // Note: we must do this before SparkHadoopUtil instantiated + if (amArgs.propertiesFile != null) { + Utils.getPropertiesFromFile(amArgs.propertiesFile).foreach { case (k, v) => + sys.props(k) = v + } + } SparkHadoopUtil.get.runAsSparkUser { () => master = new ApplicationMaster(amArgs, new YarnRMClient) System.exit(master.run()) 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 9085fca1d3cc0..874e3045b4058 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 @@ -32,6 +32,7 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.launcher._ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, @@ -106,6 +107,10 @@ class YarnClusterSuite extends BaseYarnClusterSuite { )) } + test("run Spark in yarn-cluster mode with using SparkHadoopUtil.conf") { + testYarnAppUseSparkHadoopUtilConf() + } + test("run Spark in yarn-client mode with additional jar") { testWithAddJar(true) } @@ -181,6 +186,15 @@ class YarnClusterSuite extends BaseYarnClusterSuite { checkResult(finalState, result) } + private def testYarnAppUseSparkHadoopUtilConf(): Unit = { + val result = File.createTempFile("result", null, tempDir) + val finalState = runSpark(false, + mainClassName(YarnClusterDriverUseSparkHadoopUtilConf.getClass), + appArgs = Seq("key=value", result.getAbsolutePath()), + extraConf = Map("spark.hadoop.key" -> "value")) + checkResult(finalState, result) + } + private def testWithAddJar(clientMode: Boolean): Unit = { val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir) val driverResult = File.createTempFile("driver", null, tempDir) @@ -274,6 +288,37 @@ private object YarnClusterDriverWithFailure extends Logging with Matchers { } } +private object YarnClusterDriverUseSparkHadoopUtilConf extends Logging with Matchers { + def main(args: Array[String]): Unit = { + if (args.length != 2) { + // scalastyle:off println + System.err.println( + s""" + |Invalid command line: ${args.mkString(" ")} + | + |Usage: YarnClusterDriverUseSparkHadoopUtilConf [hadoopConfKey=value] [result file] + """.stripMargin) + // scalastyle:on println + System.exit(1) + } + + val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) + .setAppName("yarn test using SparkHadoopUtil's conf")) + + val kv = args(0).split("=") + val status = new File(args(1)) + var result = "failure" + try { + SparkHadoopUtil.get.conf.get(kv(0)) should be (kv(1)) + result = "success" + } finally { + Files.write(result, status, StandardCharsets.UTF_8) + sc.stop() + } + } +} + private object YarnClusterDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 From 4303d292b55fc8709780994b05b41e73a52c001a Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Wed, 13 Jul 2016 08:05:20 +0800 Subject: [PATCH 0951/1470] [SPARK-16284][SQL] Implement reflect SQL function ## What changes were proposed in this pull request? This patch implements reflect SQL function, which can be used to invoke a Java method in SQL. Slightly different from Hive, this implementation requires the class name and the method name to be literals. This implementation also supports only a smaller number of data types, and requires the function to be static, as suggested by rxin in #13969. java_method is an alias for reflect, so this should also resolve SPARK-16277. ## How was this patch tested? Added expression unit tests and an end-to-end test. Author: petermaxlee Closes #14138 from petermaxlee/reflect-static. (cherry picked from commit 56bd399a86c4e92be412d151200cb5e4a5f6a48a) Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../expressions/CallMethodViaReflection.scala | 164 ++++++++++++++++++ .../CallMethodViaReflectionSuite.scala | 102 +++++++++++ .../apache/spark/sql/MiscFunctionsSuite.scala | 38 ++++ .../spark/sql/hive/HiveSessionCatalog.scala | 7 +- .../sql/hive/execution/SQLQuerySuite.scala | 23 --- 6 files changed, 311 insertions(+), 25 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala 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 54568b7445df8..65a90d8099b7d 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 @@ -368,6 +368,8 @@ object FunctionRegistry { expression[InputFileName]("input_file_name"), expression[MonotonicallyIncreasingID]("monotonically_increasing_id"), expression[CurrentDatabase]("current_database"), + expression[CallMethodViaReflection]("reflect"), + expression[CallMethodViaReflection]("java_method"), // grouping sets expression[Cube]("cube"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala new file mode 100644 index 0000000000000..fe24c0489fc98 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflection.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import java.lang.reflect.{Method, Modifier} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.Utils + +/** + * An expression that invokes a method on a class via reflection. + * + * For now, only types defined in `Reflect.typeMapping` are supported (basically primitives + * and string) as input types, and the output is turned automatically to a string. + * + * Note that unlike Hive's reflect function, this expression calls only static methods + * (i.e. does not support calling non-static methods). + * + * We should also look into how to consolidate this expression with + * [[org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke]] in the future. + * + * @param children the first element should be a literal string for the class name, + * and the second element should be a literal string for the method name, + * and the remaining are input arguments to the Java method. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(class,method[,arg1[,arg2..]]) calls method with reflection", + extended = "> SELECT _FUNC_('java.util.UUID', 'randomUUID');\n c33fb387-8500-4bfa-81d2-6e0e3e930df2") +// scalastyle:on line.size.limit +case class CallMethodViaReflection(children: Seq[Expression]) + extends Expression with CodegenFallback { + + override def prettyName: String = "reflect" + + override def checkInputDataTypes(): TypeCheckResult = { + if (children.size < 2) { + TypeCheckFailure("requires at least two arguments") + } else if (!children.take(2).forall(e => e.dataType == StringType && e.foldable)) { + // The first two arguments must be string type. + TypeCheckFailure("first two arguments should be string literals") + } else if (!classExists) { + TypeCheckFailure(s"class $className not found") + } else if (method == null) { + TypeCheckFailure(s"cannot find a static method that matches the argument types in $className") + } else { + TypeCheckSuccess + } + } + + override def deterministic: Boolean = false + override def nullable: Boolean = true + override val dataType: DataType = StringType + + override def eval(input: InternalRow): Any = { + var i = 0 + while (i < argExprs.length) { + buffer(i) = argExprs(i).eval(input).asInstanceOf[Object] + // Convert if necessary. Based on the types defined in typeMapping, string is the only + // type that needs conversion. If we support timestamps, dates, decimals, arrays, or maps + // in the future, proper conversion needs to happen here too. + if (buffer(i).isInstanceOf[UTF8String]) { + buffer(i) = buffer(i).toString + } + i += 1 + } + val ret = method.invoke(null, buffer : _*) + UTF8String.fromString(String.valueOf(ret)) + } + + @transient private lazy val argExprs: Array[Expression] = children.drop(2).toArray + + /** Name of the class -- this has to be called after we verify children has at least two exprs. */ + @transient private lazy val className = children(0).eval().asInstanceOf[UTF8String].toString + + /** True if the class exists and can be loaded. */ + @transient private lazy val classExists = CallMethodViaReflection.classExists(className) + + /** The reflection method. */ + @transient lazy val method: Method = { + val methodName = children(1).eval(null).asInstanceOf[UTF8String].toString + CallMethodViaReflection.findMethod(className, methodName, argExprs.map(_.dataType)).orNull + } + + /** A temporary buffer used to hold intermediate results returned by children. */ + @transient private lazy val buffer = new Array[Object](argExprs.length) +} + +object CallMethodViaReflection { + /** Mapping from Spark's type to acceptable JVM types. */ + val typeMapping = Map[DataType, Seq[Class[_]]]( + BooleanType -> Seq(classOf[java.lang.Boolean], classOf[Boolean]), + ByteType -> Seq(classOf[java.lang.Byte], classOf[Byte]), + ShortType -> Seq(classOf[java.lang.Short], classOf[Short]), + IntegerType -> Seq(classOf[java.lang.Integer], classOf[Int]), + LongType -> Seq(classOf[java.lang.Long], classOf[Long]), + FloatType -> Seq(classOf[java.lang.Float], classOf[Float]), + DoubleType -> Seq(classOf[java.lang.Double], classOf[Double]), + StringType -> Seq(classOf[String]) + ) + + /** + * Returns true if the class can be found and loaded. + */ + private def classExists(className: String): Boolean = { + try { + Utils.classForName(className) + true + } catch { + case e: ClassNotFoundException => false + } + } + + /** + * Finds a Java static method using reflection that matches the given argument types, + * and whose return type is string. + * + * The types sequence must be the valid types defined in [[typeMapping]]. + * + * This is made public for unit testing. + */ + def findMethod(className: String, methodName: String, argTypes: Seq[DataType]): Option[Method] = { + val clazz: Class[_] = Utils.classForName(className) + clazz.getMethods.find { method => + val candidateTypes = method.getParameterTypes + if (method.getName != methodName) { + // Name must match + false + } else if (!Modifier.isStatic(method.getModifiers)) { + // Method must be static + false + } else if (candidateTypes.length != argTypes.length) { + // Argument length must match + false + } else { + // Argument type must match. That is, either the method's argument type matches one of the + // acceptable types defined in typeMapping, or it is a super type of the acceptable types. + candidateTypes.zip(argTypes).forall { case (candidateType, argType) => + typeMapping(argType).exists(candidateType.isAssignableFrom) + } + } + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala new file mode 100644 index 0000000000000..43367c7e14c34 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CallMethodViaReflectionSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure +import org.apache.spark.sql.types.{IntegerType, StringType} + +/** A static class for testing purpose. */ +object ReflectStaticClass { + def method1(): String = "m1" + def method2(v1: Int): String = "m" + v1 + def method3(v1: java.lang.Integer): String = "m" + v1 + def method4(v1: Int, v2: String): String = "m" + v1 + v2 +} + +/** A non-static class for testing purpose. */ +class ReflectDynamicClass { + def method1(): String = "m1" +} + +/** + * Test suite for [[CallMethodViaReflection]] and its companion object. + */ +class CallMethodViaReflectionSuite extends SparkFunSuite with ExpressionEvalHelper { + + import CallMethodViaReflection._ + + // Get rid of the $ so we are getting the companion object's name. + private val staticClassName = ReflectStaticClass.getClass.getName.stripSuffix("$") + private val dynamicClassName = classOf[ReflectDynamicClass].getName + + test("findMethod via reflection for static methods") { + assert(findMethod(staticClassName, "method1", Seq.empty).exists(_.getName == "method1")) + assert(findMethod(staticClassName, "method2", Seq(IntegerType)).isDefined) + assert(findMethod(staticClassName, "method3", Seq(IntegerType)).isDefined) + assert(findMethod(staticClassName, "method4", Seq(IntegerType, StringType)).isDefined) + } + + test("findMethod for a JDK library") { + assert(findMethod(classOf[java.util.UUID].getName, "randomUUID", Seq.empty).isDefined) + } + + test("class not found") { + val ret = createExpr("some-random-class", "method").checkInputDataTypes() + assert(ret.isFailure) + val errorMsg = ret.asInstanceOf[TypeCheckFailure].message + assert(errorMsg.contains("not found") && errorMsg.contains("class")) + } + + test("method not found because name does not match") { + val ret = createExpr(staticClassName, "notfoundmethod").checkInputDataTypes() + assert(ret.isFailure) + val errorMsg = ret.asInstanceOf[TypeCheckFailure].message + assert(errorMsg.contains("cannot find a static method")) + } + + test("method not found because there is no static method") { + val ret = createExpr(dynamicClassName, "method1").checkInputDataTypes() + assert(ret.isFailure) + val errorMsg = ret.asInstanceOf[TypeCheckFailure].message + assert(errorMsg.contains("cannot find a static method")) + } + + test("input type checking") { + assert(CallMethodViaReflection(Seq.empty).checkInputDataTypes().isFailure) + assert(CallMethodViaReflection(Seq(Literal(staticClassName))).checkInputDataTypes().isFailure) + assert(CallMethodViaReflection( + Seq(Literal(staticClassName), Literal(1))).checkInputDataTypes().isFailure) + assert(createExpr(staticClassName, "method1").checkInputDataTypes().isSuccess) + } + + test("invoking methods using acceptable types") { + checkEvaluation(createExpr(staticClassName, "method1"), "m1") + checkEvaluation(createExpr(staticClassName, "method2", 2), "m2") + checkEvaluation(createExpr(staticClassName, "method3", 3), "m3") + checkEvaluation(createExpr(staticClassName, "method4", 4, "four"), "m4four") + } + + private def createExpr(className: String, methodName: String, args: Any*) = { + CallMethodViaReflection( + Literal.create(className, StringType) +: + Literal.create(methodName, StringType) +: + args.map(Literal.apply) + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala new file mode 100644 index 0000000000000..a5b08f717767f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext + +class MiscFunctionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("reflect and java_method") { + val df = Seq((1, "one")).toDF("a", "b") + val className = ReflectClass.getClass.getName.stripSuffix("$") + checkAnswer( + df.selectExpr( + s"reflect('$className', 'method1', a, b)", + s"java_method('$className', 'method1', a, b)"), + Row("m1one", "m1one")) + } +} + +object ReflectClass { + def method1(v1: Int, v2: String): String = "m" + v1 + v2 +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 6f36abc4db0ed..b8a75850b1d5e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -235,7 +235,10 @@ private[sql] class HiveSessionCatalog( // parse_url_tuple, posexplode, reflect2, // str_to_map, windowingtablefunction. private val hiveFunctions = Seq( - "hash", "java_method", "histogram_numeric", - "percentile", "percentile_approx", "reflect", "str_to_map" + "hash", + "histogram_numeric", + "percentile", + "percentile_approx", + "str_to_map" ) } 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 e8af4fbe876e1..22dcbc4c6010d 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 @@ -996,29 +996,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer(sql("SELECT CAST('775983671874188101' as BIGINT)"), Row(775983671874188101L)) } - // `Math.exp(1.0)` has different result for different jdk version, so not use createQueryTest - test("udf_java_method") { - checkAnswer(sql( - """ - |SELECT java_method("java.lang.String", "valueOf", 1), - | java_method("java.lang.String", "isEmpty"), - | java_method("java.lang.Math", "max", 2, 3), - | java_method("java.lang.Math", "min", 2, 3), - | java_method("java.lang.Math", "round", 2.5D), - | java_method("java.lang.Math", "exp", 1.0D), - | java_method("java.lang.Math", "floor", 1.9D) - |FROM src tablesample (1 rows) - """.stripMargin), - Row( - "1", - "true", - java.lang.Math.max(2, 3).toString, - java.lang.Math.min(2, 3).toString, - java.lang.Math.round(2.5).toString, - java.lang.Math.exp(1.0).toString, - java.lang.Math.floor(1.9).toString)) - } - test("dynamic partition value test") { try { sql("set hive.exec.dynamic.partition.mode=nonstrict") From 41df62c595474d7afda6dbe76a558d8cb3be7ff2 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 12 Jul 2016 23:09:02 -0700 Subject: [PATCH 0952/1470] [SPARK-16514][SQL] Fix various regex codegen bugs ## What changes were proposed in this pull request? RegexExtract and RegexReplace currently crash on non-nullable input due use of a hard-coded local variable name (e.g. compiles fail with `java.lang.Exception: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 85, Column 26: Redefinition of local variable "m" `). This changes those variables to use fresh names, and also in a few other places. ## How was this patch tested? Unit tests. rxin Author: Eric Liang Closes #14168 from ericl/sc-3906. (cherry picked from commit 1c58fa905b6543d366d00b2e5394dfd633987f6d) Signed-off-by: Reynold Xin --- .../expressions/regexpExpressions.scala | 48 +++++++++++++------ .../expressions/StringExpressionsSuite.scala | 6 +++ 2 files changed, 39 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 541b8601a344b..be82b3b8f45f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -108,10 +108,11 @@ case class Like(left: Expression, right: Expression) """) } } else { + val rightStr = ctx.freshName("rightStr") nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" - String rightStr = ${eval2}.toString(); - ${patternClass} $pattern = ${patternClass}.compile($escapeFunc(rightStr)); + String $rightStr = ${eval2}.toString(); + ${patternClass} $pattern = ${patternClass}.compile($escapeFunc($rightStr)); ${ev.value} = $pattern.matcher(${eval1}.toString()).matches(); """ }) @@ -157,10 +158,11 @@ case class RLike(left: Expression, right: Expression) """) } } else { + val rightStr = ctx.freshName("rightStr") nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" - String rightStr = ${eval2}.toString(); - ${patternClass} $pattern = ${patternClass}.compile(rightStr); + String $rightStr = ${eval2}.toString(); + ${patternClass} $pattern = ${patternClass}.compile($rightStr); ${ev.value} = $pattern.matcher(${eval1}.toString()).find(0); """ }) @@ -259,6 +261,8 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio val classNamePattern = classOf[Pattern].getCanonicalName val classNameStringBuffer = classOf[java.lang.StringBuffer].getCanonicalName + val matcher = ctx.freshName("matcher") + ctx.addMutableState("UTF8String", termLastRegex, s"${termLastRegex} = null;") ctx.addMutableState(classNamePattern, termPattern, s"${termPattern} = null;") ctx.addMutableState("String", termLastReplacement, s"${termLastReplacement} = null;") @@ -267,6 +271,12 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio ctx.addMutableState(classNameStringBuffer, termResult, s"${termResult} = new $classNameStringBuffer();") + val setEvNotNull = if (nullable) { + s"${ev.isNull} = false;" + } else { + "" + } + nullSafeCodeGen(ctx, ev, (subject, regexp, rep) => { s""" if (!$regexp.equals(${termLastRegex})) { @@ -280,14 +290,14 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio ${termLastReplacement} = ${termLastReplacementInUTF8}.toString(); } ${termResult}.delete(0, ${termResult}.length()); - java.util.regex.Matcher m = ${termPattern}.matcher($subject.toString()); + java.util.regex.Matcher ${matcher} = ${termPattern}.matcher($subject.toString()); - while (m.find()) { - m.appendReplacement(${termResult}, ${termLastReplacement}); + while (${matcher}.find()) { + ${matcher}.appendReplacement(${termResult}, ${termLastReplacement}); } - m.appendTail(${termResult}); + ${matcher}.appendTail(${termResult}); ${ev.value} = UTF8String.fromString(${termResult}.toString()); - ${ev.isNull} = false; + $setEvNotNull """ }) } @@ -334,10 +344,18 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio val termLastRegex = ctx.freshName("lastRegex") val termPattern = ctx.freshName("pattern") val classNamePattern = classOf[Pattern].getCanonicalName + val matcher = ctx.freshName("matcher") + val matchResult = ctx.freshName("matchResult") ctx.addMutableState("UTF8String", termLastRegex, s"${termLastRegex} = null;") ctx.addMutableState(classNamePattern, termPattern, s"${termPattern} = null;") + val setEvNotNull = if (nullable) { + s"${ev.isNull} = false;" + } else { + "" + } + nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => { s""" if (!$regexp.equals(${termLastRegex})) { @@ -345,15 +363,15 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio ${termLastRegex} = $regexp.clone(); ${termPattern} = ${classNamePattern}.compile(${termLastRegex}.toString()); } - java.util.regex.Matcher m = + java.util.regex.Matcher ${matcher} = ${termPattern}.matcher($subject.toString()); - if (m.find()) { - java.util.regex.MatchResult mr = m.toMatchResult(); - ${ev.value} = UTF8String.fromString(mr.group($idx)); - ${ev.isNull} = false; + if (${matcher}.find()) { + java.util.regex.MatchResult ${matchResult} = ${matcher}.toMatchResult(); + ${ev.value} = UTF8String.fromString(${matchResult}.group($idx)); + $setEvNotNull } else { ${ev.value} = UTF8String.EMPTY_UTF8; - ${ev.isNull} = false; + $setEvNotNull }""" }) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 8f7b1041fad32..5b9ed83a4eb35 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -631,6 +631,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(expr, null, row4) checkEvaluation(expr, null, row5) checkEvaluation(expr, null, row6) + + val nonNullExpr = RegExpReplace(Literal("100-200"), Literal("(\\d+)"), Literal("num")) + checkEvaluation(nonNullExpr, "num-num", row1) } test("RegexExtract") { @@ -657,6 +660,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val expr1 = new RegExpExtract(s, p) checkEvaluation(expr1, "100", row1) + + val nonNullExpr = RegExpExtract(Literal("100-200"), Literal("(\\d+)-(\\d+)"), Literal(1)) + checkEvaluation(nonNullExpr, "100", row1) } test("SPLIT") { From 5173f847c55a7b810d1c494c8b23c740ba110c39 Mon Sep 17 00:00:00 2001 From: aokolnychyi Date: Wed, 13 Jul 2016 16:12:05 +0800 Subject: [PATCH 0953/1470] [SPARK-16303][DOCS][EXAMPLES] Updated SQL programming guide and examples - Hard-coded Spark SQL sample snippets were moved into source files under examples sub-project. - Removed the inconsistency between Scala and Java Spark SQL examples - Scala and Java Spark SQL examples were updated The work is still in progress. All involved examples were tested manually. An additional round of testing will be done after the code review. ![image](https://cloud.githubusercontent.com/assets/6235869/16710314/51851606-462a-11e6-9fbe-0818daef65e4.png) Author: aokolnychyi Closes #14119 from aokolnychyi/spark_16303. (cherry picked from commit 772c213ec702c80d0f25aa6f30b2dffebfbe2d0d) Signed-off-by: Cheng Lian --- docs/sql-programming-guide.md | 572 ++---------------- .../spark/examples/sql/JavaSparkSQL.java | 186 ------ .../examples/sql/JavaSparkSqlExample.java | 336 ++++++++++ .../sql/JavaSqlDataSourceExample.java | 217 +++++++ .../sql/hive/JavaSparkHiveExample.java | 131 ++++ .../spark/examples/sql/SparkSqlExample.scala | 254 ++++++++ .../examples/sql/SqlDataSourceExample.scala | 148 +++++ .../examples/sql/hive/HiveFromSpark.scala | 83 --- .../examples/sql/hive/SparkHiveExample.scala | 107 ++++ 9 files changed, 1228 insertions(+), 806 deletions(-) delete mode 100644 examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java create mode 100644 examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSqlExample.java create mode 100644 examples/src/main/java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java create mode 100644 examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java create mode 100644 examples/src/main/scala/org/apache/spark/examples/sql/SparkSqlExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 448251cfdc697..f5d1fee943b7c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -65,14 +65,14 @@ Throughout this document, we will often refer to Scala/Java Datasets of `Row`s a The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: -{% include_example init_session scala/org/apache/spark/examples/sql/RDDRelation.scala %} +{% include_example init_session scala/org/apache/spark/examples/sql/SparkSqlExample.scala %}
      The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: -{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQL.java %} +{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %}
      @@ -105,14 +105,7 @@ from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: -{% highlight scala %} -val spark: SparkSession // An existing SparkSession. -val df = spark.read.json("examples/src/main/resources/people.json") - -// Displays the content of the DataFrame to stdout -df.show() -{% endhighlight %} - +{% include_example create_df scala/org/apache/spark/examples/sql/SparkSqlExample.scala %}
      @@ -121,14 +114,7 @@ from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: -{% highlight java %} -SparkSession spark = ...; // An existing SparkSession. -Dataset df = spark.read().json("examples/src/main/resources/people.json"); - -// Displays the content of the DataFrame to stdout -df.show(); -{% endhighlight %} - +{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %}
      @@ -169,110 +155,20 @@ Here we include some basic examples of structured data processing using Datasets
      -{% highlight scala %} -val spark: SparkSession // An existing SparkSession - -// Create the DataFrame -val df = spark.read.json("examples/src/main/resources/people.json") - -// Show the content of the DataFrame -df.show() -// age name -// null Michael -// 30 Andy -// 19 Justin - -// Print the schema in a tree format -df.printSchema() -// root -// |-- age: long (nullable = true) -// |-- name: string (nullable = true) - -// Select only the "name" column -df.select("name").show() -// name -// Michael -// Andy -// Justin - -// Select everybody, but increment the age by 1 -df.select(df("name"), df("age") + 1).show() -// name (age + 1) -// Michael null -// Andy 31 -// Justin 20 - -// Select people older than 21 -df.filter(df("age") > 21).show() -// age name -// 30 Andy - -// Count people by age -df.groupBy("age").count().show() -// age count -// null 1 -// 19 1 -// 30 1 -{% endhighlight %} +{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). - -
      -{% highlight java %} -SparkSession spark = ...; // An existing SparkSession - -// Create the DataFrame -Dataset df = spark.read().json("examples/src/main/resources/people.json"); - -// Show the content of the DataFrame -df.show(); -// age name -// null Michael -// 30 Andy -// 19 Justin - -// Print the schema in a tree format -df.printSchema(); -// root -// |-- age: long (nullable = true) -// |-- name: string (nullable = true) - -// Select only the "name" column -df.select("name").show(); -// name -// Michael -// Andy -// Justin - -// Select everybody, but increment the age by 1 -df.select(df.col("name"), df.col("age").plus(1)).show(); -// name (age + 1) -// Michael null -// Andy 31 -// Justin 20 - -// Select people older than 21 -df.filter(df.col("age").gt(21)).show(); -// age name -// 30 Andy - -// Count people by age -df.groupBy("age").count().show(); -// age count -// null 1 -// 19 1 -// 30 1 -{% endhighlight %} + +{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). -
      @@ -353,19 +249,13 @@ In addition to simple column references and expressions, DataFrames also have a
      The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. -{% highlight scala %} -val spark = ... // An existing SparkSession -val df = spark.sql("SELECT * FROM table") -{% endhighlight %} +{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
      The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. -{% highlight java %} -SparkSession spark = ... // An existing SparkSession -Dataset df = spark.sql("SELECT * FROM table") -{% endhighlight %} +{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %}
      @@ -397,53 +287,11 @@ the bytes back into an object.
      - -{% highlight scala %} -// Encoders for most common types are automatically provided by importing spark.implicits._ -val ds = Seq(1, 2, 3).toDS() -ds.map(_ + 1).collect() // Returns: Array(2, 3, 4) - -// Encoders are also created for case classes. -case class Person(name: String, age: Long) -val ds = Seq(Person("Andy", 32)).toDS() - -// DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name. -val path = "examples/src/main/resources/people.json" -val people = spark.read.json(path).as[Person] - -{% endhighlight %} - +{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSqlExample.scala %}
      - -{% highlight java %} -SparkSession spark = ... // An existing SparkSession - -// Encoders for most common types are provided in class Encoders. -Dataset ds = spark.createDataset(Arrays.asList(1, 2, 3), Encoders.INT()); -ds.map(new MapFunction() { - @Override - public Integer call(Integer value) throws Exception { - return value + 1; - } -}, Encoders.INT()); // Returns: [2, 3, 4] - -Person person = new Person(); -person.setName("Andy"); -person.setAge(32); - -// Encoders are also created for Java beans. -Dataset ds = spark.createDataset( - Collections.singletonList(person), - Encoders.bean(Person.class) -); - -// DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name. -String path = "examples/src/main/resources/people.json"; -Dataset people = spark.read().json(path).as(Encoders.bean(Person.class)); -{% endhighlight %} - +{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %}
      @@ -470,38 +318,7 @@ reflection and become the names of the columns. Case classes can also be nested types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be registered as a table. Tables can be used in subsequent SQL statements. -{% highlight scala %} -val spark: SparkSession // An existing SparkSession -// this is used to implicitly convert an RDD to a DataFrame. -import spark.implicits._ - -// Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, -// you can use custom classes that implement the Product interface. -case class Person(name: String, age: Int) - -// Create an RDD of Person objects and register it as a temporary view. -val people = sc - .textFile("examples/src/main/resources/people.txt") - .map(_.split(",")) - .map(p => Person(p(0), p(1).trim.toInt)) - .toDF() -people.createOrReplaceTempView("people") - -// SQL statements can be run by using the sql methods provided by spark. -val teenagers = spark.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19") - -// The columns of a row in the result can be accessed by field index: -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) - -// or by field name: -teenagers.map(t => "Name: " + t.getAs[String]("name")).collect().foreach(println) - -// row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T] -teenagers.map(_.getValuesMap[Any](List("name", "age"))).collect().foreach(println) -// Map("name" -> "Justin", "age" -> 19) -{% endhighlight %} - +{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSqlExample.scala %}
      @@ -513,68 +330,7 @@ does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `Li fields are supported though. You can create a JavaBean by creating a class that implements Serializable and has getters and setters for all of its fields. -{% highlight java %} - -public static class Person implements Serializable { - private String name; - private int age; - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public int getAge() { - return age; - } - - public void setAge(int age) { - this.age = age; - } -} - -{% endhighlight %} - - -A schema can be applied to an existing RDD by calling `createDataFrame` and providing the Class object -for the JavaBean. - -{% highlight java %} -SparkSession spark = ...; // An existing SparkSession - -// Load a text file and convert each line to a JavaBean. -JavaRDD people = spark.sparkContext.textFile("examples/src/main/resources/people.txt").map( - new Function() { - public Person call(String line) throws Exception { - String[] parts = line.split(","); - - Person person = new Person(); - person.setName(parts[0]); - person.setAge(Integer.parseInt(parts[1].trim())); - - return person; - } - }); - -// Apply a schema to an RDD of JavaBeans and register it as a table. -Dataset schemaPeople = spark.createDataFrame(people, Person.class); -schemaPeople.createOrReplaceTempView("people"); - -// SQL can be run over RDDs that have been registered as tables. -Dataset teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -// The columns of a row in the result can be accessed by ordinal. -List teenagerNames = teenagers.map(new MapFunction() { - public String call(Row row) { - return "Name: " + row.getString(0); - } -}).collectAsList(); - -{% endhighlight %} - +{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %}
      @@ -628,43 +384,8 @@ a `DataFrame` can be created programmatically with three steps. by `SparkSession`. For example: -{% highlight scala %} -val spark: SparkSession // An existing SparkSession - -// Create an RDD -val people = sc.textFile("examples/src/main/resources/people.txt") - -// The schema is encoded in a string -val schemaString = "name age" - -// Import Row. -import org.apache.spark.sql.Row; - -// Import Spark SQL data types -import org.apache.spark.sql.types.{StructType, StructField, StringType}; - -// Generate the schema based on the string of schema -val schema = StructType(schemaString.split(" ").map { fieldName => - StructField(fieldName, StringType, true) -}) - -// Convert records of the RDD (people) to Rows. -val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) - -// Apply the schema to the RDD. -val peopleDataFrame = spark.createDataFrame(rowRDD, schema) - -// Creates a temporary view using the DataFrame. -peopleDataFrame.createOrReplaceTempView("people") - -// SQL statements can be run by using the sql methods provided by spark. -val results = spark.sql("SELECT name FROM people") - -// The columns of a row in the result can be accessed by field index or by field name. -results.map(t => "Name: " + t(0)).collect().foreach(println) -{% endhighlight %} - +{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSqlExample.scala %}
      @@ -681,62 +402,8 @@ a `Dataset` can be created programmatically with three steps. by `SparkSession`. For example: -{% highlight java %} -import org.apache.spark.api.java.function.Function; -// Import factory methods provided by DataTypes. -import org.apache.spark.sql.types.DataTypes; -// Import StructType and StructField -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.types.StructField; -// Import Row. -import org.apache.spark.sql.Row; -// Import RowFactory. -import org.apache.spark.sql.RowFactory; - -SparkSession spark = ...; // An existing SparkSession. -JavaSparkContext sc = spark.sparkContext - -// Load a text file and convert each line to a JavaBean. -JavaRDD people = sc.textFile("examples/src/main/resources/people.txt"); - -// The schema is encoded in a string -String schemaString = "name age"; - -// Generate the schema based on the string of schema -List fields = new ArrayList<>(); -for (String fieldName: schemaString.split(" ")) { - fields.add(DataTypes.createStructField(fieldName, DataTypes.StringType, true)); -} -StructType schema = DataTypes.createStructType(fields); - -// Convert records of the RDD (people) to Rows. -JavaRDD rowRDD = people.map( - new Function() { - public Row call(String record) throws Exception { - String[] fields = record.split(","); - return RowFactory.create(fields[0], fields[1].trim()); - } - }); - -// Apply the schema to the RDD. -Dataset peopleDataFrame = spark.createDataFrame(rowRDD, schema); - -// Creates a temporary view using the DataFrame. -peopleDataFrame.createOrReplaceTempView("people"); - -// SQL can be run over a temporary view created using DataFrames. -Dataset results = spark.sql("SELECT name FROM people"); - -// 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. -List names = results.javaRDD().map(new Function() { - public String call(Row row) { - return "Name: " + row.getString(0); - } -}).collect(); - -{% endhighlight %} +{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %}
      @@ -805,23 +472,11 @@ In the simplest form, the default data source (`parquet` unless otherwise config
      - -{% highlight scala %} -val df = spark.read.load("examples/src/main/resources/users.parquet") -df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") -{% endhighlight %} - +{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %}
      - -{% highlight java %} - -Dataset df = spark.read().load("examples/src/main/resources/users.parquet"); -df.select("name", "favorite_color").write().save("namesAndFavColors.parquet"); - -{% endhighlight %} - +{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %}
      @@ -852,23 +507,11 @@ using this syntax.
      - -{% highlight scala %} -val df = spark.read.format("json").load("examples/src/main/resources/people.json") -df.select("name", "age").write.format("parquet").save("namesAndAges.parquet") -{% endhighlight %} - +{% include_example manual_load_options scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %}
      - -{% highlight java %} - -Dataset df = spark.read().format("json").load("examples/src/main/resources/people.json"); -df.select("name", "age").write().format("parquet").save("namesAndAges.parquet"); - -{% endhighlight %} - +{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %}
      @@ -895,18 +538,11 @@ file directly with SQL.
      - -{% highlight scala %} -val df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") -{% endhighlight %} - +{% include_example direct_sql scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %}
      - -{% highlight java %} -Dataset df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); -{% endhighlight %} +{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %}
      @@ -997,53 +633,11 @@ Using the data from the above example:
      - -{% highlight scala %} -// spark from the previous example is used in this example. -// This is used to implicitly convert an RDD to a DataFrame. -import spark.implicits._ - -val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. - -// The RDD is implicitly converted to a DataFrame by implicits, allowing it to be stored using Parquet. -people.write.parquet("people.parquet") - -// 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. -val parquetFile = spark.read.parquet("people.parquet") - -// Parquet files can also be used to create a temporary view and then used in SQL statements. -parquetFile.createOrReplaceTempView("parquetFile") -val teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) -{% endhighlight %} - +{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %}
      - -{% highlight java %} -// spark from the previous example is used in this example. - -Dataset schemaPeople = ... // The DataFrame from the previous example. - -// DataFrames can be saved as Parquet files, maintaining the schema information. -schemaPeople.write().parquet("people.parquet"); - -// 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. -Dataset parquetFile = spark.read().parquet("people.parquet"); - -// Parquet files can also be used to create a temporary view and then used in SQL statements. -parquetFile.createOrReplaceTempView("parquetFile"); -Dataset teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); -List teenagerNames = teenagers.javaRDD().map(new Function() { - public String call(Row row) { - return "Name: " + row.getString(0); - } -}).collect(); -{% endhighlight %} - +{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %}
      @@ -1172,34 +766,11 @@ turned it off by default starting from 1.5.0. You may enable it by
      +{% include_example schema_merging scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %} +
      -{% highlight scala %} -// spark from the previous example is used in this example. -// This is used to implicitly convert an RDD to a DataFrame. -import spark.implicits._ - -// Create a simple DataFrame, stored into a partition directory -val df1 = sc.makeRDD(1 to 5).map(i => (i, i * 2)).toDF("single", "double") -df1.write.parquet("data/test_table/key=1") - -// Create another DataFrame in a new partition directory, -// adding a new column and dropping an existing column -val df2 = sc.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple") -df2.write.parquet("data/test_table/key=2") - -// Read the partitioned table -val df3 = spark.read.option("mergeSchema", "true").parquet("data/test_table") -df3.printSchema() - -// The final schema consists of all 3 columns in the Parquet files together -// with the partitioning column appeared in the partition directory paths. -// root -// |-- single: int (nullable = true) -// |-- double: int (nullable = true) -// |-- triple: int (nullable = true) -// |-- key : int (nullable = true) -{% endhighlight %} - +
      +{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %}
      @@ -1280,8 +851,8 @@ metadata.
      {% highlight scala %} -// spark is an existing HiveContext -spark.refreshTable("my_table") +// spark is an existing SparkSession +spark.catalog.refreshTable("my_table") {% endhighlight %}
      @@ -1289,8 +860,8 @@ spark.refreshTable("my_table")
      {% highlight java %} -// spark is an existing HiveContext -spark.refreshTable("my_table") +// spark is an existing SparkSession +spark.catalog().refreshTable("my_table"); {% endhighlight %}
      @@ -1390,33 +961,7 @@ Note that the file that is offered as _a json file_ is not a typical JSON file. line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. -{% highlight scala %} -val spark: SparkSession // An existing SparkSession - -// A JSON dataset is pointed to by path. -// The path can be either a single text file or a directory storing text files. -val path = "examples/src/main/resources/people.json" -val people = spark.read.json(path) - -// The inferred schema can be visualized using the printSchema() method. -people.printSchema() -// root -// |-- age: long (nullable = true) -// |-- name: string (nullable = true) - -// Creates a temporary view using the DataFrame -people.createOrReplaceTempView("people") - -// SQL statements can be run by using the sql methods provided by spark. -val teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -// Alternatively, a DataFrame can be created for a JSON dataset represented by -// an RDD[String] storing one JSON object per string. -val anotherPeopleRDD = sc.parallelize( - """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) -val anotherPeople = spark.read.json(anotherPeopleRDD) -{% endhighlight %} - +{% include_example json_dataset scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %}
      @@ -1428,33 +973,7 @@ Note that the file that is offered as _a json file_ is not a typical JSON file. line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. -{% highlight java %} -// sc is an existing JavaSparkContext. -SparkSession spark = new org.apache.spark.sql.SparkSession(sc); - -// A JSON dataset is pointed to by path. -// The path can be either a single text file or a directory storing text files. -Dataset people = spark.read().json("examples/src/main/resources/people.json"); - -// The inferred schema can be visualized using the printSchema() method. -people.printSchema(); -// root -// |-- age: long (nullable = true) -// |-- name: string (nullable = true) - -// Creates a temporary view using the DataFrame -people.createOrReplaceTempView("people"); - -// SQL statements can be run by using the sql methods provided by spark. -Dataset teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); - -// Alternatively, a DataFrame can be created for a JSON dataset represented by -// an RDD[String] storing one JSON object per string. -List jsonData = Arrays.asList( - "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); -JavaRDD anotherPeopleRDD = sc.parallelize(jsonData); -Dataset anotherPeople = spark.read().json(anotherPeopleRDD); -{% endhighlight %} +{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %}
      @@ -1549,18 +1068,7 @@ the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated sin Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. You may need to grant write privilege to the user who starts the spark application. -{% highlight scala %} -// warehouse_location points to the default location for managed databases and tables -val conf = new SparkConf().setAppName("HiveFromSpark").set("spark.sql.warehouse.dir", warehouse_location) -val spark = SparkSession.builder.config(conf).enableHiveSupport().getOrCreate() - -spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") - -// Queries are expressed in HiveQL -spark.sql("FROM src SELECT key, value").collect().foreach(println) -{% endhighlight %} - +{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %}
      @@ -1575,17 +1083,7 @@ the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated sin Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. You may need to grant write privilege to the user who starts the spark application. -{% highlight java %} -SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate(); - -spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); - -// Queries are expressed in HiveQL. -List results = spark.sql("FROM src SELECT key, value").collectAsList(); - -{% endhighlight %} - +{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %}
      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 deleted file mode 100644 index 7fc6c007b6843..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ /dev/null @@ -1,186 +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.examples.sql; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; - -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -// $example on:init_session$ -import org.apache.spark.sql.SparkSession; -// $example off:init_session$ - -public class JavaSparkSQL { - public static class Person implements Serializable { - private String name; - private int age; - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public int getAge() { - return age; - } - - public void setAge(int age) { - this.age = age; - } - } - - public static void main(String[] args) throws Exception { - // $example on:init_session$ - SparkSession spark = SparkSession - .builder() - .appName("JavaSparkSQL") - .config("spark.some.config.option", "some-value") - .getOrCreate(); - // $example off:init_session$ - - System.out.println("=== Data source: RDD ==="); - // Load a text file and convert each line to a Java Bean. - String file = "examples/src/main/resources/people.txt"; - JavaRDD people = spark.read().textFile(file).javaRDD().map( - new Function() { - @Override - public Person call(String line) { - String[] parts = line.split(","); - - Person person = new Person(); - person.setName(parts[0]); - person.setAge(Integer.parseInt(parts[1].trim())); - - return person; - } - }); - - // Apply a schema to an RDD of Java Beans and create a temporary view - Dataset schemaPeople = spark.createDataFrame(people, Person.class); - schemaPeople.createOrReplaceTempView("people"); - - // SQL can be run over RDDs which backs a temporary view. - Dataset teenagers = spark.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. - List teenagerNames = teenagers.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { - return "Name: " + row.getString(0); - } - }).collect(); - for (String name: teenagerNames) { - System.out.println(name); - } - - System.out.println("=== Data source: Parquet File ==="); - // DataFrames can be saved as parquet files, maintaining the schema information. - schemaPeople.write().parquet("people.parquet"); - - // 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. - Dataset parquetFile = spark.read().parquet("people.parquet"); - - // A temporary view can be created by using Parquet files and then used in SQL statements. - parquetFile.createOrReplaceTempView("parquetFile"); - Dataset teenagers2 = - spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); - teenagerNames = teenagers2.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { - return "Name: " + row.getString(0); - } - }).collect(); - for (String name: teenagerNames) { - System.out.println(name); - } - - System.out.println("=== Data source: JSON Dataset ==="); - // A JSON dataset is pointed by path. - // 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 - Dataset peopleFromJsonFile = spark.read().json(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. - peopleFromJsonFile.printSchema(); - // The schema of people is ... - // root - // |-- age: IntegerType - // |-- name: StringType - - // Creates a temporary view using the DataFrame - peopleFromJsonFile.createOrReplaceTempView("people"); - - // SQL statements can be run by using the sql methods provided by `spark` - Dataset teenagers3 = spark.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. - teenagerNames = teenagers3.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { return "Name: " + row.getString(0); } - }).collect(); - for (String name: teenagerNames) { - System.out.println(name); - } - - // Alternatively, a DataFrame can be created for a JSON dataset represented by - // a RDD[String] storing one JSON object per string. - List jsonData = Arrays.asList( - "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); - JavaRDD anotherPeopleRDD = spark - .createDataFrame(jsonData, String.class).toJSON().javaRDD(); - Dataset peopleFromJsonRDD = spark.read().json(anotherPeopleRDD); - - // Take a look at the schema of this new DataFrame. - peopleFromJsonRDD.printSchema(); - // The schema of anotherPeople is ... - // root - // |-- address: StructType - // | |-- city: StringType - // | |-- state: StringType - // |-- name: StringType - - peopleFromJsonRDD.createOrReplaceTempView("people2"); - - Dataset peopleWithCity = spark.sql("SELECT name, address.city FROM people2"); - List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { - return "Name: " + row.getString(0) + ", City: " + row.getString(1); - } - }).collect(); - for (String name: nameAndCity) { - System.out.println(name); - } - - spark.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSqlExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSqlExample.java new file mode 100644 index 0000000000000..586d6e3a3e478 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSqlExample.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql; + +// $example on:programmatic_schema$ +import java.util.ArrayList; +import java.util.List; +// $example off:programmatic_schema$ +// $example on:create_ds$ +import java.util.Arrays; +import java.util.Collections; +import java.io.Serializable; +// $example off:create_ds$ + +// $example on:schema_inferring$ +// $example on:programmatic_schema$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +// $example off:programmatic_schema$ +// $example on:create_ds$ +import org.apache.spark.api.java.function.MapFunction; +// $example on:create_df$ +// $example on:run_sql$ +// $example on:programmatic_schema$ +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off:programmatic_schema$ +// $example off:create_df$ +// $example off:run_sql$ +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +// $example off:create_ds$ +// $example off:schema_inferring$ +import org.apache.spark.sql.RowFactory; +// $example on:init_session$ +import org.apache.spark.sql.SparkSession; +// $example off:init_session$ +// $example on:programmatic_schema$ +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off:programmatic_schema$ + +// $example on:untyped_ops$ +// col("...") is preferable to df.col("...") +import static org.apache.spark.sql.functions.col; +// $example off:untyped_ops$ + +public class JavaSparkSqlExample { + // $example on:create_ds$ + public static class Person implements Serializable { + private String name; + private int age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getAge() { + return age; + } + + public void setAge(int age) { + this.age = age; + } + } + // $example off:create_ds$ + + public static void main(String[] args) { + // $example on:init_session$ + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL Example") + .config("spark.some.config.option", "some-value") + .getOrCreate(); + // $example off:init_session$ + + runBasicDataFrameExample(spark); + runDatasetCreationExample(spark); + runInferSchemaExample(spark); + runProgrammaticSchemaExample(spark); + + spark.stop(); + } + + private static void runBasicDataFrameExample(SparkSession spark) { + // $example on:create_df$ + Dataset df = spark.read().json("examples/src/main/resources/people.json"); + + // Displays the content of the DataFrame to stdout + df.show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_df$ + + // $example on:untyped_ops$ + // Print the schema in a tree format + df.printSchema(); + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Select only the "name" column + df.select("name").show(); + // +-------+ + // | name| + // +-------+ + // |Michael| + // | Andy| + // | Justin| + // +-------+ + + // Select everybody, but increment the age by 1 + df.select(col("name"), col("age").plus(1)).show(); + // +-------+---------+ + // | name|(age + 1)| + // +-------+---------+ + // |Michael| null| + // | Andy| 31| + // | Justin| 20| + // +-------+---------+ + + // Select people older than 21 + df.filter(col("age").gt(21)).show(); + // +---+----+ + // |age|name| + // +---+----+ + // | 30|Andy| + // +---+----+ + + // Count people by age + df.groupBy("age").count().show(); + // +----+-----+ + // | age|count| + // +----+-----+ + // | 19| 1| + // |null| 1| + // | 30| 1| + // +----+-----+ + // $example off:untyped_ops$ + + // $example on:run_sql$ + // Register the DataFrame as a SQL temporary view + df.createOrReplaceTempView("people"); + + Dataset sqlDF = spark.sql("SELECT * FROM people"); + sqlDF.show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:run_sql$ + } + + private static void runDatasetCreationExample(SparkSession spark) { + // $example on:create_ds$ + // Create an instance of a Bean class + Person person = new Person(); + person.setName("Andy"); + person.setAge(32); + + // Encoders are created for Java beans + Encoder personEncoder = Encoders.bean(Person.class); + Dataset javaBeanDS = spark.createDataset( + Collections.singletonList(person), + personEncoder + ); + javaBeanDS.show(); + // +---+----+ + // |age|name| + // +---+----+ + // | 32|Andy| + // +---+----+ + + // Encoders for most common types are provided in class Encoders + Encoder integerEncoder = Encoders.INT(); + Dataset primitiveDS = spark.createDataset(Arrays.asList(1, 2, 3), integerEncoder); + Dataset transformedDS = primitiveDS.map(new MapFunction() { + @Override + public Integer call(Integer value) throws Exception { + return value + 1; + } + }, integerEncoder); + transformedDS.collect(); // Returns [2, 3, 4] + + // DataFrames can be converted to a Dataset by providing a class. Mapping based on name + String path = "examples/src/main/resources/people.json"; + Dataset peopleDS = spark.read().json(path).as(personEncoder); + peopleDS.show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_ds$ + } + + private static void runInferSchemaExample(SparkSession spark) { + // $example on:schema_inferring$ + // Create an RDD of Person objects from a text file + JavaRDD peopleRDD = spark.read() + .textFile("examples/src/main/resources/people.txt") + .javaRDD() + .map(new Function() { + @Override + public Person call(String line) throws Exception { + String[] parts = line.split(","); + Person person = new Person(); + person.setName(parts[0]); + person.setAge(Integer.parseInt(parts[1].trim())); + return person; + } + }); + + // Apply a schema to an RDD of JavaBeans to get a DataFrame + Dataset peopleDF = spark.createDataFrame(peopleRDD, Person.class); + // Register the DataFrame as a temporary view + peopleDF.createOrReplaceTempView("people"); + + // SQL statements can be run by using the sql methods provided by spark + Dataset teenagersDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19"); + + // The columns of a row in the result can be accessed by field index + Encoder stringEncoder = Encoders.STRING(); + Dataset teenagerNamesByIndexDF = teenagersDF.map(new MapFunction() { + @Override + public String call(Row row) throws Exception { + return "Name: " + row.getString(0); + } + }, stringEncoder); + teenagerNamesByIndexDF.show(); + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + + // or by field name + Dataset teenagerNamesByFieldDF = teenagersDF.map(new MapFunction() { + @Override + public String call(Row row) throws Exception { + return "Name: " + row.getAs("name"); + } + }, stringEncoder); + teenagerNamesByFieldDF.show(); + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + // $example off:schema_inferring$ + } + + private static void runProgrammaticSchemaExample(SparkSession spark) { + // $example on:programmatic_schema$ + // Create an RDD + JavaRDD peopleRDD = spark.sparkContext() + .textFile("examples/src/main/resources/people.txt", 1) + .toJavaRDD(); + + // The schema is encoded in a string + String schemaString = "name age"; + + // Generate the schema based on the string of schema + List fields = new ArrayList<>(); + for (String fieldName : schemaString.split(" ")) { + StructField field = DataTypes.createStructField(fieldName, DataTypes.StringType, true); + fields.add(field); + } + StructType schema = DataTypes.createStructType(fields); + + // Convert records of the RDD (people) to Rows + JavaRDD rowRDD = peopleRDD.map(new Function() { + @Override + public Row call(String record) throws Exception { + String[] attributes = record.split(","); + return RowFactory.create(attributes[0], attributes[1].trim()); + } + }); + + // Apply the schema to the RDD + Dataset peopleDataFrame = spark.createDataFrame(rowRDD, schema); + + // Creates a temporary view using the DataFrame + peopleDataFrame.createOrReplaceTempView("people"); + + // SQL can be run over a temporary view created using DataFrames + Dataset results = spark.sql("SELECT name FROM people"); + + // 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 field index or by field name + Dataset namesDS = results.map(new MapFunction() { + @Override + public String call(Row row) throws Exception { + return "Name: " + row.getString(0); + } + }, Encoders.STRING()); + namesDS.show(); + // +-------------+ + // | value| + // +-------------+ + // |Name: Michael| + // | Name: Andy| + // | Name: Justin| + // +-------------+ + // $example off:programmatic_schema$ + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java new file mode 100644 index 0000000000000..4db5e1b0af83d --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql; + +// $example on:schema_merging$ +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +// $example off:schema_merging$ + +// $example on:basic_parquet_example$ +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Encoders; +// import org.apache.spark.sql.Encoders; +// $example on:schema_merging$ +// $example on:json_dataset$ +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off:json_dataset$ +// $example off:schema_merging$ +// $example off:basic_parquet_example$ +import org.apache.spark.sql.SparkSession; + +public class JavaSqlDataSourceExample { + + // $example on:schema_merging$ + public static class Square implements Serializable { + private int value; + private int square; + + // Getters and setters... + // $example off:schema_merging$ + public int getValue() { + return value; + } + + public void setValue(int value) { + this.value = value; + } + + public int getSquare() { + return square; + } + + public void setSquare(int square) { + this.square = square; + } + // $example on:schema_merging$ + } + // $example off:schema_merging$ + + // $example on:schema_merging$ + public static class Cube implements Serializable { + private int value; + private int cube; + + // Getters and setters... + // $example off:schema_merging$ + public int getValue() { + return value; + } + + public void setValue(int value) { + this.value = value; + } + + public int getCube() { + return cube; + } + + public void setCube(int cube) { + this.cube = cube; + } + // $example on:schema_merging$ + } + // $example off:schema_merging$ + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL Data Sources Example") + .config("spark.some.config.option", "some-value") + .getOrCreate(); + + runBasicDataSourceExample(spark); + runBasicParquetExample(spark); + runParquetSchemaMergingExample(spark); + runJsonDatasetExample(spark); + + spark.stop(); + } + + private static void runBasicDataSourceExample(SparkSession spark) { + // $example on:generic_load_save_functions$ + Dataset usersDF = spark.read().load("examples/src/main/resources/users.parquet"); + usersDF.select("name", "favorite_color").write().save("namesAndFavColors.parquet"); + // $example off:generic_load_save_functions$ + // $example on:manual_load_options$ + Dataset peopleDF = spark.read().format("json").load("examples/src/main/resources/people.json"); + peopleDF.select("name", "age").write().format("parquet").save("namesAndAges.parquet"); + // $example off:manual_load_options$ + // $example on:direct_sql$ + Dataset sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); + // $example off:direct_sql$ + } + + private static void runBasicParquetExample(SparkSession spark) { + // $example on:basic_parquet_example$ + Dataset peopleDF = spark.read().json("examples/src/main/resources/people.json"); + + // DataFrames can be saved as Parquet files, maintaining the schema information + peopleDF.write().parquet("people.parquet"); + + // 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 + Dataset parquetFileDF = spark.read().parquet("people.parquet"); + + // Parquet files can also be used to create a temporary view and then used in SQL statements + parquetFileDF.createOrReplaceTempView("parquetFile"); + Dataset namesDF = spark.sql("SELECT name FROM parquetFile WHERE age BETWEEN 13 AND 19"); + Dataset namesDS = namesDF.map(new MapFunction() { + public String call(Row row) { + return "Name: " + row.getString(0); + } + }, Encoders.STRING()); + namesDS.show(); + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + // $example off:basic_parquet_example$ + } + + private static void runParquetSchemaMergingExample(SparkSession spark) { + // $example on:schema_merging$ + List squares = new ArrayList<>(); + for (int value = 1; value <= 5; value++) { + Square square = new Square(); + square.setValue(value); + square.setSquare(value * value); + squares.add(square); + } + + // Create a simple DataFrame, store into a partition directory + Dataset squaresDF = spark.createDataFrame(squares, Square.class); + squaresDF.write().parquet("data/test_table/key=1"); + + List cubes = new ArrayList<>(); + for (int value = 6; value <= 10; value++) { + Cube cube = new Cube(); + cube.setValue(value); + cube.setCube(value * value * value); + cubes.add(cube); + } + + // Create another DataFrame in a new partition directory, + // adding a new column and dropping an existing column + Dataset cubesDF = spark.createDataFrame(cubes, Cube.class); + cubesDF.write().parquet("data/test_table/key=2"); + + // Read the partitioned table + Dataset mergedDF = spark.read().option("mergeSchema", true).parquet("data/test_table"); + mergedDF.printSchema(); + + // The final schema consists of all 3 columns in the Parquet files together + // with the partitioning column appeared in the partition directory paths + // root + // |-- value: int (nullable = true) + // |-- square: int (nullable = true) + // |-- cube: int (nullable = true) + // |-- key : int (nullable = true) + // $example off:schema_merging$ + } + + private static void runJsonDatasetExample(SparkSession spark) { + // $example on:json_dataset$ + // A JSON dataset is pointed to by path. + // The path can be either a single text file or a directory storing text files + Dataset people = spark.read().json("examples/src/main/resources/people.json"); + + // The inferred schema can be visualized using the printSchema() method + people.printSchema(); + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Creates a temporary view using the DataFrame + people.createOrReplaceTempView("people"); + + // SQL statements can be run by using the sql methods provided by spark + Dataset namesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19"); + namesDF.show(); + // +------+ + // | name| + // +------+ + // |Justin| + // +------+ + // $example off:json_dataset$ + } + +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java new file mode 100644 index 0000000000000..493d759a916f8 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql.hive; + +// $example on:spark_hive$ +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off:spark_hive$ + +public class JavaSparkHiveExample { + + // $example on:spark_hive$ + public static class Record implements Serializable { + private int key; + private String value; + + public int getKey() { + return key; + } + + public void setKey(int key) { + this.key = key; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + } + // $example off:spark_hive$ + + public static void main(String[] args) { + // $example on:spark_hive$ + // warehouseLocation points to the default location for managed databases and tables + String warehouseLocation = "file:" + System.getProperty("user.dir") + "spark-warehouse"; + SparkSession spark = SparkSession + .builder() + .appName("Java Spark Hive Example") + .config("spark.sql.warehouse.dir", warehouseLocation) + .enableHiveSupport() + .getOrCreate(); + + spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); + spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); + + // Queries are expressed in HiveQL + spark.sql("SELECT * FROM src").show(); + // +---+-------+ + // |key| value| + // +---+-------+ + // |238|val_238| + // | 86| val_86| + // |311|val_311| + // ... + + // Aggregation queries are also supported. + spark.sql("SELECT COUNT(*) FROM src").show(); + // +--------+ + // |count(1)| + // +--------+ + // | 500 | + // +--------+ + + // The results of SQL queries are themselves DataFrames and support all normal functions. + Dataset sqlDF = spark.sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key"); + + // The items in DaraFrames are of type Row, which lets you to access each column by ordinal. + Dataset stringsDS = sqlDF.map(new MapFunction() { + @Override + public String call(Row row) throws Exception { + return "Key: " + row.get(0) + ", Value: " + row.get(1); + } + }, Encoders.STRING()); + stringsDS.show(); + // +--------------------+ + // | value| + // +--------------------+ + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // ... + + // You can also use DataFrames to create temporary views within a HiveContext. + List records = new ArrayList<>(); + for (int key = 1; key < 100; key++) { + Record record = new Record(); + record.setKey(key); + record.setValue("val_" + key); + records.add(record); + } + Dataset recordsDF = spark.createDataFrame(records, Record.class); + recordsDF.createOrReplaceTempView("records"); + + // Queries can then join DataFrames data with data stored in Hive. + spark.sql("SELECT * FROM records r JOIN src s ON r.key = s.key").show(); + // +---+------+---+------+ + // |key| value|key| value| + // +---+------+---+------+ + // | 2| val_2| 2| val_2| + // | 2| val_2| 2| val_2| + // | 4| val_4| 4| val_4| + // ... + // $example off:spark_hive$ + + spark.stop(); + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSqlExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSqlExample.scala new file mode 100644 index 0000000000000..cf3f864267197 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSqlExample.scala @@ -0,0 +1,254 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql + +// $example on:schema_inferring$ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.Encoder +// $example off:schema_inferring$ +import org.apache.spark.sql.Row +// $example on:init_session$ +import org.apache.spark.sql.SparkSession +// $example off:init_session$ +// $example on:programmatic_schema$ +import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.StructField +import org.apache.spark.sql.types.StructType +// $example off:programmatic_schema$ + +object SparkSqlExample { + + // $example on:create_ds$ + // Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, + // you can use custom classes that implement the Product interface + case class Person(name: String, age: Long) + // $example off:create_ds$ + + def main(args: Array[String]) { + // $example on:init_session$ + val spark = SparkSession + .builder() + .appName("Spark SQL Example") + .config("spark.some.config.option", "some-value") + .getOrCreate() + + // For implicit conversions like converting RDDs to DataFrames + import spark.implicits._ + // $example off:init_session$ + + runBasicDataFrameExample(spark) + runDatasetCreationExample(spark) + runInferSchemaExample(spark) + runProgrammaticSchemaExample(spark) + + spark.stop() + } + + private def runBasicDataFrameExample(spark: SparkSession): Unit = { + // $example on:create_df$ + val df = spark.read.json("examples/src/main/resources/people.json") + + // Displays the content of the DataFrame to stdout + df.show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_df$ + + // $example on:untyped_ops$ + // This import is needed to use the $-notation + import spark.implicits._ + // Print the schema in a tree format + df.printSchema() + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Select only the "name" column + df.select("name").show() + // +-------+ + // | name| + // +-------+ + // |Michael| + // | Andy| + // | Justin| + // +-------+ + + // Select everybody, but increment the age by 1 + df.select($"name", $"age" + 1).show() + // +-------+---------+ + // | name|(age + 1)| + // +-------+---------+ + // |Michael| null| + // | Andy| 31| + // | Justin| 20| + // +-------+---------+ + + // Select people older than 21 + df.filter($"age" > 21).show() + // +---+----+ + // |age|name| + // +---+----+ + // | 30|Andy| + // +---+----+ + + // Count people by age + df.groupBy("age").count().show() + // +----+-----+ + // | age|count| + // +----+-----+ + // | 19| 1| + // |null| 1| + // | 30| 1| + // +----+-----+ + // $example off:untyped_ops$ + + // $example on:run_sql$ + // Register the DataFrame as a SQL temporary view + df.createOrReplaceTempView("people") + + val sqlDF = spark.sql("SELECT * FROM people") + sqlDF.show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:run_sql$ + } + + private def runDatasetCreationExample(spark: SparkSession): Unit = { + import spark.implicits._ + // $example on:create_ds$ + // Encoders are created for case classes + val caseClassDS = Seq(Person("Andy", 32)).toDS() + caseClassDS.show() + // +----+---+ + // |name|age| + // +----+---+ + // |Andy| 32| + // +----+---+ + + // Encoders for most common types are automatically provided by importing spark.implicits._ + val primitiveDS = Seq(1, 2, 3).toDS() + primitiveDS.map(_ + 1).collect() // Returns: Array(2, 3, 4) + + // DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name + val path = "examples/src/main/resources/people.json" + val peopleDS = spark.read.json(path).as[Person] + peopleDS.show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_ds$ + } + + private def runInferSchemaExample(spark: SparkSession): Unit = { + // $example on:schema_inferring$ + // For implicit conversions from RDDs to DataFrames + import spark.implicits._ + + // Create an RDD of Person objects from a text file, convert it to a Dataframe + val peopleDF = spark.sparkContext + .textFile("examples/src/main/resources/people.txt") + .map(_.split(",")) + .map(attributes => Person(attributes(0), attributes(1).trim.toInt)) + .toDF() + // Register the DataFrame as a temporary view + peopleDF.createOrReplaceTempView("people") + + // SQL statements can be run by using the sql methods provided by Spark + val teenagersDF = spark.sql("SELECT name, age FROM people WHERE age BETWEEN 13 AND 19") + + // The columns of a row in the result can be accessed by field index + teenagersDF.map(teenager => "Name: " + teenager(0)).show() + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + + // or by field name + teenagersDF.map(teenager => "Name: " + teenager.getAs[String]("name")).show() + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + + // No pre-defined encoders for Dataset[Map[K,V]], define explicitly + implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]] + // Primitive types and case classes can be also defined as + implicit val stringIntMapEncoder: Encoder[Map[String, Int]] = ExpressionEncoder() + + // row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T] + teenagersDF.map(teenager => teenager.getValuesMap[Any](List("name", "age"))).collect() + // Array(Map("name" -> "Justin", "age" -> 19)) + // $example off:schema_inferring$ + } + + private def runProgrammaticSchemaExample(spark: SparkSession): Unit = { + import spark.implicits._ + // $example on:programmatic_schema$ + // Create an RDD + val peopleRDD = spark.sparkContext.textFile("examples/src/main/resources/people.txt") + + // The schema is encoded in a string + val schemaString = "name age" + + // Generate the schema based on the string of schema + val fields = schemaString.split(" ") + .map(fieldName => StructField(fieldName, StringType, nullable = true)) + val schema = StructType(fields) + + // Convert records of the RDD (people) to Rows + val rowRDD = peopleRDD + .map(_.split(",")) + .map(attributes => Row(attributes(0), attributes(1).trim)) + + // Apply the schema to the RDD + val peopleDF = spark.createDataFrame(rowRDD, schema) + + // Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + // SQL can be run over a temporary view created using DataFrames + val results = spark.sql("SELECT name FROM people") + + // 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 field index or by field name + results.map(attributes => "Name: " + attributes(0)).show() + // +-------------+ + // | value| + // +-------------+ + // |Name: Michael| + // | Name: Andy| + // | Name: Justin| + // +-------------+ + // $example off:programmatic_schema$ + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala new file mode 100644 index 0000000000000..61dea6ad2ca90 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql + +import org.apache.spark.sql.SparkSession + +object SqlDataSourceExample { + + case class Person(name: String, age: Long) + + def main(args: Array[String]) { + val spark = SparkSession + .builder() + .appName("Spark SQL Data Soures Example") + .config("spark.some.config.option", "some-value") + .getOrCreate() + + runBasicDataSourceExample(spark) + runBasicParquetExample(spark) + runParquetSchemaMergingExample(spark) + runJsonDatasetExample(spark) + + spark.stop() + } + + private def runBasicDataSourceExample(spark: SparkSession): Unit = { + // $example on:generic_load_save_functions$ + val usersDF = spark.read.load("examples/src/main/resources/users.parquet") + usersDF.select("name", "favorite_color").write.save("namesAndFavColors.parquet") + // $example off:generic_load_save_functions$ + // $example on:manual_load_options$ + val peopleDF = spark.read.format("json").load("examples/src/main/resources/people.json") + peopleDF.select("name", "age").write.format("parquet").save("namesAndAges.parquet") + // $example off:manual_load_options$ + // $example on:direct_sql$ + val sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") + // $example off:direct_sql$ + } + + private def runBasicParquetExample(spark: SparkSession): Unit = { + // $example on:basic_parquet_example$ + // Encoders for most common types are automatically provided by importing spark.implicits._ + import spark.implicits._ + + val peopleDF = spark.read.json("examples/src/main/resources/people.json") + + // DataFrames can be saved as Parquet files, maintaining the schema information + peopleDF.write.parquet("people.parquet") + + // 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 + val parquetFileDF = spark.read.parquet("people.parquet") + + // Parquet files can also be used to create a temporary view and then used in SQL statements + parquetFileDF.createOrReplaceTempView("parquetFile") + val namesDF = spark.sql("SELECT name FROM parquetFile WHERE age BETWEEN 13 AND 19") + namesDF.map(attributes => "Name: " + attributes(0)).show() + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + // $example off:basic_parquet_example$ + } + + private def runParquetSchemaMergingExample(spark: SparkSession): Unit = { + // $example on:schema_merging$ + // This is used to implicitly convert an RDD to a DataFrame. + import spark.implicits._ + + // Create a simple DataFrame, store into a partition directory + val squaresDF = spark.sparkContext.makeRDD(1 to 5).map(i => (i, i * i)).toDF("value", "square") + squaresDF.write.parquet("data/test_table/key=1") + + // Create another DataFrame in a new partition directory, + // adding a new column and dropping an existing column + val cubesDF = spark.sparkContext.makeRDD(6 to 10).map(i => (i, i * i * i)).toDF("value", "cube") + cubesDF.write.parquet("data/test_table/key=2") + + // Read the partitioned table + val mergedDF = spark.read.option("mergeSchema", "true").parquet("data/test_table") + mergedDF.printSchema() + + // The final schema consists of all 3 columns in the Parquet files together + // with the partitioning column appeared in the partition directory paths + // root + // |-- value: int (nullable = true) + // |-- square: int (nullable = true) + // |-- cube: int (nullable = true) + // |-- key : int (nullable = true) + // $example off:schema_merging$ + } + + private def runJsonDatasetExample(spark: SparkSession): Unit = { + // $example on:json_dataset$ + // A JSON dataset is pointed to by path. + // The path can be either a single text file or a directory storing text files + val path = "examples/src/main/resources/people.json" + val peopleDF = spark.read.json(path) + + // The inferred schema can be visualized using the printSchema() method + peopleDF.printSchema() + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + // SQL statements can be run by using the sql methods provided by spark + val teenagerNamesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19") + teenagerNamesDF.show() + // +------+ + // | name| + // +------+ + // |Justin| + // +------+ + + // Alternatively, a DataFrame can be created for a JSON dataset represented by + // an RDD[String] storing one JSON object per string + val otherPeopleRDD = spark.sparkContext.makeRDD( + """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) + val otherPeople = spark.read.json(otherPeopleRDD) + otherPeople.show() + // +---------------+----+ + // | address|name| + // +---------------+----+ + // |[Columbus,Ohio]| Yin| + // +---------------+----+ + // $example off:json_dataset$ + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala deleted file mode 100644 index 2343f98c8d07c..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ /dev/null @@ -1,83 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.sql.hive - -import java.io.File - -import com.google.common.io.{ByteStreams, Files} - -import org.apache.spark.sql._ - -object HiveFromSpark { - case class Record(key: Int, value: String) - - // Copy kv1.txt file from classpath to temporary directory - val kv1Stream = HiveFromSpark.getClass.getResourceAsStream("/kv1.txt") - val kv1File = File.createTempFile("kv1", "txt") - kv1File.deleteOnExit() - ByteStreams.copy(kv1Stream, Files.newOutputStreamSupplier(kv1File)) - - def main(args: Array[String]) { - // When working with Hive, one must instantiate `SparkSession` with Hive support, including - // connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined - // functions. Users who do not have an existing Hive deployment can still enable Hive support. - // When not configured by the hive-site.xml, the context automatically creates `metastore_db` - // in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, - // which defaults to the directory `spark-warehouse` in the current directory that the spark - // application is started. - val spark = SparkSession.builder - .appName("HiveFromSpark") - .enableHiveSupport() - .getOrCreate() - - import spark.implicits._ - import spark.sql - - sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") - - // Queries are expressed in HiveQL - println("Result of 'SELECT *': ") - sql("SELECT * FROM src").collect().foreach(println) - - // Aggregation queries are also supported. - val count = sql("SELECT COUNT(*) FROM src").collect().head.getLong(0) - println(s"COUNT(*): $count") - - // The results of SQL queries are themselves RDDs and support all normal RDD functions. The - // items in the RDD are of type Row, which allows you to access each column by ordinal. - val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") - - println("Result of RDD.map:") - val rddAsStrings = rddFromSql.rdd.map { - case Row(key: Int, value: String) => s"Key: $key, Value: $value" - } - - // You can also use RDDs to create temporary views within a HiveContext. - val rdd = spark.sparkContext.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.toDF().createOrReplaceTempView("records") - - // Queries can then join RDD data with data stored in Hive. - println("Result of SELECT *:") - sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) - - spark.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala new file mode 100644 index 0000000000000..e897c2d066cfb --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala @@ -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. + */ +package org.apache.spark.examples.sql.hive + +// $example on:spark_hive$ +import org.apache.spark.sql.Row +import org.apache.spark.sql.SparkSession +// $example off:spark_hive$ + +object SparkHiveExample { + + // $example on:spark_hive$ + case class Record(key: Int, value: String) + // $example off:spark_hive$ + + def main(args: Array[String]) { + // When working with Hive, one must instantiate `SparkSession` with Hive support, including + // connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined + // functions. Users who do not have an existing Hive deployment can still enable Hive support. + // When not configured by the hive-site.xml, the context automatically creates `metastore_db` + // in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, + // which defaults to the directory `spark-warehouse` in the current directory that the spark + // application is started. + + // $example on:spark_hive$ + // warehouseLocation points to the default location for managed databases and tables + val warehouseLocation = "file:${system:user.dir}/spark-warehouse" + + val spark = SparkSession + .builder() + .appName("Spark Hive Example") + .config("spark.sql.warehouse.dir", warehouseLocation) + .enableHiveSupport() + .getOrCreate() + + import spark.implicits._ + import spark.sql + + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + + // Queries are expressed in HiveQL + sql("SELECT * FROM src").show() + // +---+-------+ + // |key| value| + // +---+-------+ + // |238|val_238| + // | 86| val_86| + // |311|val_311| + // ... + + // Aggregation queries are also supported. + sql("SELECT COUNT(*) FROM src").show() + // +--------+ + // |count(1)| + // +--------+ + // | 500 | + // +--------+ + + // The results of SQL queries are themselves DataFrames and support all normal functions. + val sqlDF = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + + // The items in DaraFrames are of type Row, which allows you to access each column by ordinal. + val stringsDS = sqlDF.map { + case Row(key: Int, value: String) => s"Key: $key, Value: $value" + } + stringsDS.show() + // +--------------------+ + // | value| + // +--------------------+ + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // ... + + // You can also use DataFrames to create temporary views within a HiveContext. + val recordsDF = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) + recordsDF.createOrReplaceTempView("records") + + // Queries can then join DataFrame data with data stored in Hive. + sql("SELECT * FROM records r JOIN src s ON r.key = s.key").show() + // +---+------+---+------+ + // |key| value|key| value| + // +---+------+---+------+ + // | 2| val_2| 2| val_2| + // | 2| val_2| 2| val_2| + // | 4| val_4| 4| val_4| + // ... + // $example off:spark_hive$ + + spark.stop() + } +} From 4b93a833b75d72043fd7770250c25247e690666d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 13 Jul 2016 10:44:07 +0100 Subject: [PATCH 0954/1470] [SPARK-15889][STREAMING] Follow-up fix to erroneous condition in StreamTest ## What changes were proposed in this pull request? A second form of AssertQuery now actually invokes the condition; avoids a build warning too ## How was this patch tested? Jenkins; running StreamTest Author: Sean Owen Closes #14133 from srowen/SPARK-15889.2. (cherry picked from commit c190d89bd3cf677400c49238498207b87da9ee78) Signed-off-by: Sean Owen --- .../test/scala/org/apache/spark/sql/streaming/StreamTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index f9496520f3836..af2b58116b2aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -189,7 +189,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } def apply(message: String)(condition: StreamExecution => Unit): AssertOnQuery = { - new AssertOnQuery(s => { condition; true }, message) + new AssertOnQuery(s => { condition(s); true }, message) } } From 5301efc1779c9d4a14ea1238bbbfa4aab25d3163 Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Wed, 13 Jul 2016 10:45:06 +0100 Subject: [PATCH 0955/1470] [SPARK-16375][WEB UI] Fixed misassigned var: numCompletedTasks was assigned to numSkippedTasks ## What changes were proposed in this pull request? I fixed a misassigned var, numCompletedTasks was assigned to numSkippedTasks in the convertJobData method ## How was this patch tested? dev/run-tests Author: Alex Bozarth Closes #14141 from ajbozarth/spark16375. (cherry picked from commit f156136dae5df38f73a25cf3fb48f98f417ef059) Signed-off-by: Sean Owen --- .../org/apache/spark/status/api/v1/AllJobsResource.scala | 2 +- ...job_list_from_multi_attempt_app_json_1__expectation.json | 2 +- ...job_list_from_multi_attempt_app_json_2__expectation.json | 2 +- .../job_list_json_expectation.json | 6 +++--- .../HistoryServerExpectations/one_job_json_expectation.json | 2 +- .../succeeded_failed_job_list_json_expectation.json | 6 +++--- .../succeeded_job_list_json_expectation.json | 4 ++-- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala index b21d36d4a8d8a..d0d9ef1165e81 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala @@ -91,7 +91,7 @@ private[v1] object AllJobsResource { numTasks = job.numTasks, numActiveTasks = job.numActiveTasks, numCompletedTasks = job.numCompletedTasks, - numSkippedTasks = job.numCompletedTasks, + numSkippedTasks = job.numSkippedTasks, numFailedTasks = job.numFailedTasks, numActiveStages = job.numActiveStages, numCompletedStages = job.completedStageIndices.size, diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json index bb6bf434be90b..c108fa61a4318 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json index bb6bf434be90b..c108fa61a4318 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json index 1583e5ddef565..3d7407004d262 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -20,7 +20,7 @@ "numTasks" : 16, "numActiveTasks" : 0, "numCompletedTasks" : 15, - "numSkippedTasks" : 15, + "numSkippedTasks" : 0, "numFailedTasks" : 1, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -34,7 +34,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, diff --git a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json index f1f0ec885587b..10c7e1c0b36fd 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json index 1583e5ddef565..3d7407004d262 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -20,7 +20,7 @@ "numTasks" : 16, "numActiveTasks" : 0, "numCompletedTasks" : 15, - "numSkippedTasks" : 15, + "numSkippedTasks" : 0, "numFailedTasks" : 1, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -34,7 +34,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json index c232c98323755..6a9bafd6b2191 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -20,7 +20,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, From 934e2aa4f8aea409c8814f394f760f5952bd48f1 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Wed, 13 Jul 2016 10:47:07 +0100 Subject: [PATCH 0956/1470] [MINOR] Fix Java style errors and remove unused imports Fix Java style errors and remove unused imports, which are randomly found Tested on my local machine. Author: Xin Ren Closes #14161 from keypointt/SPARK-16437. (cherry picked from commit f73891e0b9640e14455bdbfd999a8ff10b78a819) Signed-off-by: Sean Owen --- .../sql/execution/datasources/parquet/ParquetFileFormat.scala | 3 +-- .../scala/org/apache/spark/sql/sources/DataSourceTest.scala | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 8cbdaebac1797..f1c78bb60c4f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -778,8 +778,7 @@ private[sql] object ParquetFileFormat extends Logging { val assumeBinaryIsString = sparkSession.sessionState.conf.isParquetBinaryAsString val assumeInt96IsTimestamp = sparkSession.sessionState.conf.isParquetINT96AsTimestamp val writeLegacyParquetFormat = sparkSession.sessionState.conf.writeLegacyParquetFormat - val serializedConf = - new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()) + val serializedConf = new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()) // !! HACK ALERT !! // diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 206d03ea98e69..cc77d3c4b91ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.sources import org.apache.spark.sql._ -import org.apache.spark.sql.internal.SQLConf private[sql] abstract class DataSourceTest extends QueryTest { From 38787ec9361bde444ba00cac6822c491acd14fcc Mon Sep 17 00:00:00 2001 From: Maciej Brynski Date: Wed, 13 Jul 2016 10:50:26 +0100 Subject: [PATCH 0957/1470] [SPARK-16439] Fix number formatting in SQL UI ## What changes were proposed in this pull request? Spark SQL UI display numbers greater than 1000 with u00A0 as grouping separator. Problem exists when server locale has no-breaking space as separator. (for example pl_PL) This patch turns off grouping and remove this separator. The problem starts with this PR. https://github.com/apache/spark/pull/12425/files#diff-803f475b01acfae1c5c96807c2ea9ddcR125 ## How was this patch tested? Manual UI tests. Screenshot attached. ![image](https://cloud.githubusercontent.com/assets/4006010/16749556/5cb5a372-47cb-11e6-9a95-67fd3f9d1c71.png) Author: Maciej Brynski Closes #14142 from maver1ck/master. (cherry picked from commit 83879ebc5850b74369a5b066c65fa9929bbdb21c) Signed-off-by: Sean Owen --- .../org/apache/spark/sql/execution/metric/SQLMetrics.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index edfdf7cd6b7f1..9817a56f499a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -101,7 +101,9 @@ private[sql] object SQLMetrics { */ def stringValue(metricsType: String, values: Seq[Long]): String = { if (metricsType == SUM_METRIC) { - NumberFormat.getInstance().format(values.sum) + val numberFormat = NumberFormat.getInstance() + numberFormat.setGroupingUsed(false) + numberFormat.format(values.sum) } else { val strFormat: Long => String = if (metricsType == SIZE_METRIC) { Utils.bytesToString From a34a54435f6af572b33017945dd34a1b5898bf31 Mon Sep 17 00:00:00 2001 From: sandy Date: Wed, 13 Jul 2016 11:33:46 +0100 Subject: [PATCH 0958/1470] [SPARK-16438] Add Asynchronous Actions documentation ## What changes were proposed in this pull request? Add Asynchronous Actions documentation inside action of programming guide ## How was this patch tested? check the documentation indentation and formatting with md preview. Author: sandy Closes #14104 from phalodi/SPARK-16438. (cherry picked from commit bf107f1e6522f9138d454b0723089c24626e775a) Signed-off-by: Sean Owen --- docs/programming-guide.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 3872aecff25d7..2bc49120a0e24 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1101,6 +1101,9 @@ for details.

      XlDs2lrya=$EyDbj#*64R71DvqD4cJ*JQfkr$NY_k4OcHB za?{JjIsUOeh2>L@bozqwqXOgfT}}V$rpYql@PC!IN3XDUR`J}gWb6EQ>HVz5s=^i; zB40uR{UOFJQyG25{u9hD*Au@gX^rCroNJ7tp z@*OOgAzb@z`Tdep+9`t(<`pHLY~vNU+%m29#%BRGH7A;#!0n@?KH!S znKbmYwaJ55UM4g4T_jWY+*6-W@(WN=TeHTF42VZChKjqUo7+W!)ky?P_Yl__-@bmW zta@7T`~m4-a)VUqxgRb2pOQ##i)2X)W>q@}ix&YBjlJe> zAgTiiyP>JSM9@u`b(P&`vg6oDHS4st|CJ2Z%M%?9*b zv)nmD13GcVn^MwVM_jArOw`>d)yjXJzMKFztUlcmj0rdzJ>0}Ri|(b~gNB^*O6Qln zHviUM7Z4%#Z%YpsnLN(+VcatRw-)UjgqC%7c5&f1?2I&4km9JcU99~(9Yei|?|l8S zsI_eQN_jyS<(I8oX@{Dw&|R#n^hF-)bO)>LV%0%?}^DJCCyo1BoZ48pc17f8dn1p&U)pl`=)XkRQf`Y2}W zf$vg1ao%Juu$hOdhj`L0FjLf2a-7mJbRjGQ6vSCZoDLeq6${Kqn3g{;Z;HGj`&b_2 zC2!*cV)C#={bHOEZ2K#gFD@@m2e+}PKVEj#`s&kUtG;bh);%Kwt8S4lJ?D&%kA>|A zI07O4)gsI6i#iZe7q&5|gE&m?^srZ5FuPXAS+z3Z(0|s)GI3^*OHWgqY2IAwR&KGD zcWX=caV&>?T3waJyJlH-zUxbgGcusx=Tupgde=HC<1@u;t2f_5CMKl+#<^4E7T)yYpE(g zU3}nkgdkKLys-Q4u6uRvuJf4Vjyn!1J2gCg)kPBSRr<<>x5&xgIbnw_tfUEl?6JoR z7R@l9ebrtIUA=o-*zQN*YKMV-ozv-@454115E|6^-HM0qw@>4QiPAcKhBQr?qMI-# z>$tAPe2{*`x-c7q0eD(~p=o>_YxMN#8#Hwj#fA;GSk|_BmGtQkng~t0Bd$k}^g+Gg zD5Dx3AL0jHoOx~KsEoeLa(#!gHqJ^vG^ll{+gtDa+kM6hI`8=47NzB+kI3cMe0Zns z5KNPOirOA$?%nsmqq^awN1E}b(P}qGhP|zSOzfWH;G4YK2R`roKGz_S&-DdK9pZB>tWRfmWq0{N@(cLDxr|o#SAG z{_?6FD{NkgiF4q8^kL9v{_omb*8W|G1m9qv2LC`m75WQtgGfey<@132@7e!h^7Qm` zOXQ)4pOD9%d`g~vc8PwTvjwwq>w6|on+%H`v|Zf7dAoxI`W^m9V8u)NQ1B8T&1RUcj(*sCdl}M z-mQ8DLnGJJl-Awm0nxAZjbuIDZ-wD!YEJo3;t0TI|g+eMju71M?74ebmx z{assS(z2_y_ZgOPuYI3XkDqP*4cmdU18kGueqi4uSVPb&W5MDJk+Kn{<_|UFb{9N8WZ#U70C`AM>TbSiZQtI2}9?S1eFA!nFK(c~j&K z*~juAFL@guuRrcxE&7AbP;ZAWP(P&$)VE2`^doJ7IrvLA-$RcY@iE67!%RB|4lI$2-u>e|-j81JUvk^$ zPmtrL>LyyS&-}4Uj8ns1@vx%b47TQ1IX@fqUlOZ%THOj2H^K7oRMHjd_ zbfK$jv!3VmR8BOURf`up;RSEle1U!+^EZ2nN#~umgH6X<13Lf2QviK>_N=`*J;%c4 zgcJ2?s0owxX((NYd-~xA^-*K71v|)thUc0Ke&Td>i0rISEc&6&2ecls==Z=QkINo= z>>)E}x+8&tqIMjr8_2qq+s@k`SX@*r20s4y<2q+}R*pREAgLbIADr=rx2y!62>hGD z^(UAsxqbv~2e@95M6Yvr5}ASoBg2%`ZA7tQV!^CeCkEa6Sf&hgN%OvM&wIm^39hnG z0SE@10umo-I?X@`5kou{xu85aAY@Qz0kQ%2_KN9A6Z7RY^2Yq&NeE# zzwv$f(a(NaR=deq2Wb#1?%0Wec7W=r)mtvgunf5=|`k(0K3F+A>BSvhEraW?DOVlBz}VrV?- z-C9Xlq#w{Tvxan_Iyj{Ljpji=mGn1Ok)TxseTz4TZOK>M`Wf^$`hx)a8Z4CABc%RU zbt6Z+_4#payIt9qH6HyO+rO0q8l$(|dF{Wdwm}cUf2VA|>xa_w@TF2c@ryR;2PG%| zdHsZAF($qt7iA-aV9*uUfLOG}u13_$_?3TFd+Q|r06p`qy1`5sD-xMjlU?2NW*_oK z-a`6(j`_KB)rRU|2++7VlaXZ@y|4PjaS(|@Mi|TuJP-4Uf_zwDbb2|P=4*hrcEK;0 zm$Ct-h5X!OOaD}Py=`*(F{i5oH|M=BTrSULkLXP=AcK?eR<;Nugtc=awurV(jAC znPJt9Ka~rtfMey4Pkl}Pyw9O}eY2c@$vtx1Ifv}DqW9FtEAgY^b1y8j#U&g|fdws$ zt!yC*H#QA?!INFU!StSnv0SUZKn{IZkG;H3clGNoCv3#P=8Mkmjb=;BvyCq|Ueh!A zlwZ9rvf&3!{IH456iMA^8NlJ(I4T5nguV&&h4CLC>l!q`hx7#i7(;@7{f3Q#%`e9u z_l9jqvSX57^5W5U7Ed2tyyST~{E!1p_evcXt2L|}g}o3GJGsle>%`u$wWnpACoc-N zOK%inj2d2g}UM#BYaSoq(M4yK!k0kYZ2&(jn2MP=;0su(+M*ICK~Ro zCy>`t+%E{;zu{+@|HBC0pjrEOwkV_i53llAMit1tuAo6=MZ!!Hu7yxe`6e!hc3YTZl{o+?{8QZnvpHip+Dn2(4cEz=z)yZ z!y0{iX8mplOV5kP(1Ten$pfD#t1qg`2%hxr>9&!1g zPGq;Nk%p#+#u>yI1|_=1-Yr|(q^+&P-gu3L+U90`xlL<}w6^FEQM;u6u&2`>5aFfN zf9-_X(y+Tu2ABO!YBoM9y%YD>1+!|cN9QMT9kb3{iRJUnc17syl^aVzFD%$swk?(B zj%Rdt*>2J}|ES0xbNZrP(B3XKzcy$mK!ieimy6%|xpTaIb|cGcRZ#}YlF)nk%t^vR zh?jr`{37~7eewm$^w0SgOjlW>Kk)2-s%*LE$1-lsjnXmhEL*@2{gF5!@`X6A%OQky zUrc_s3)2VArXQmv)C8M>s9Rx<^!&2rc3A$-J6=>ITQc$N92UP50R0jM*CHa=U@Sc%^Y+jxE{hzFWgu4$4%I; zK^M8O11)!VaI_dl*by37&E zh1g48G_#dWbrc-_-~Z&5wp!b=uRnOsDiT$yfxh7{p%EM~&`fEdCFwnGamFD@vMV!$ zw@#9a!(2xnUxoZJzC*en=XvqM9I?P&h6J!|$f=<~9;ZjbF=jW$^uO||-|u=0X0B$m zLHPwNS0xs!eVVx*AcFTc*D3i3>$_OJphJ~=`ULmsqvq%RVOW11a6uzXK74uHoYWTp zB1?DFh5qIU@<{SCXnyc=6q*0S5?T9yIr_iO&Mx`X7tWVUes!f=sPQhmD+2U2!$bO5 zwTAFsJ?j2pC-pZbn05n@5Q7T5HOk%`rQL#C#y1K`4+;-mT=Sl9(zDGTm;PsdFWtKx zE_F@glL^U42O5sk%lB;qVCZP~TL)P>#x5Mc-~|!mH~dXiuF|)%9j|qmqXiG6D_DHmE>nq|+0^X+rk>kg7* zjyy~j%-`L3ao1VmBfJldG5zgK8&!#QO?hEQ#zEtkTK|m;-yq$~@0VuvvySGyZN~$6 z!9L*6@?06L9bA=npzUBsOe!%~n6&nGQ>tarNzS(3nG1Xq{gOs%Ru|mBMI&8=@y1!M zAkvUpSDuGsKz=Si_#zK}v_!zmtnhNH7xgU7HhsZNhdxHts|SKN=>)k&7l-o1@cL-dd#=-;r-4gUGk=-I94NVV?qduV5kJHBG-5L6uC@Ej8E7}v% zINaGM&H}w{9OWgB?GO5wTmQN#3#4wwA`v}@vu(rE(m#1`TcF18R0yme>Ij+3F@$v$ z>JZ}8k@}f$slEyH_rQKVuAxtNTCZNSRu=BDC&_mT+8v>+}q3v}0{(wyR6;aa}qd z7}hhzAO-FV?u^)|?Itai{^MGMy)4Skrp>iqlxuB6Z`4BrIeLxNl%%!m70W;leX*3@ zhp*DNonv<#&b!9WIP5e78T%aaz+;(^wWp_7?tSnP%MV?5)j-`=tX!iPMxrW2MyT+cIm$&q49}2^_#~ zY8qz{MH2@dLx?ku5NQFw1VmV%-xKE-=c)ui`xaX+-D=q}{+KkmH#@2#hvX=fwmd~w&s2DKlxMW{%{vC`i-v2t|{|7RO;5d8!~40SSC z+ayg3kJYzn{y}<|+$7cdjk~Rt)48-_l?ATy48o--bWBm>Lb1WtLj_SzYeg& z%B$L!$x!1y$pjuofjyEs3UFtz^7y6>grU@k!w2((`ks|iJ9QtaZJq5GZJjNd4-f%k zA}k04A`l0(YuFtB-u4#f^LhdWM&K?r4K~ICF~T#?Es+~`A%_5Bn(%g3_7oi_t4dlGl;{UG#6-;|=!mnwCxXN&Cb-b+IAWwoDB=6l4r=q?J>k z?1cQ$19iZ92;9+Cv+8SO6zC0H6}IYCLM+mZ8|UtSJI4;~a_!eX?pPoDBROerzol%# z>*Wid+h1<}+M{ymSr^IwdCzxb!AoA1*tX=<@AF^&#x@D3>$$K$ITag#XiGK%LSzqc z7}|{K;_N=65e_P;+8XmwKo^+TL(8vetT2UG#6Rb!zx4E@2h?@+Imdnu>eiu)oA7(H ztM_N~YuYhTh3y0|#7UAC$FvxZ+C3p88YO-?!g(Nl&;G&urD?%2${S8#-s7Qyyf|Fv zr{`ah8V)zI;0C4O3Qo-LV2*=J%UuT;3~WO95@UfPdCjrm{H&N#^kZEAT7NKoV39G7 zeomJ5e^qu@&wcvWAC`ThLnRa8wK_RKO5!{c`c^>)heXyX=!pYbh$75?G;WmsV4Wyg zY9ahX6n#2L-UdQiK!jL)mo*XIU54MUaQ}Dh8HV#-5mj%21H8ogb~OXT=NFKF#riA% z{%_0H_E*e;8EVyb>*ym_X<VYk;w=aO4w@P(VCXYR4~3n(A9_GB8GZJdsy2#u<2c$Kw2G3KMq7!-+eNX1MUj z|D$J_UMhI}az};bD^|%jzkiW^4n6n)Ip^aak+&RwteyXfXM)_@sG+sW`!ebeGlccQ zVH_ZWH>bDOj;;Ut*{_uzeZwF8t#{HMI+?F>h3Yqb6A>Yv5R?HD`X1XD;xJ#p2Xlq` z&2F1X;xIx5j+jrPb0P(@2Pa8IDvLga6yYgCjXzwJGiiYgZ(Jf< zIycC0<0M<4hJ3tV0XG>ayj_*i7Yqp%2uCEkT3q!-Hu$hTPN;1Hj~`;e3~?=Z)P5(0 zo~Qp;&brny|N42S$b_Jrqdxrqi+t@=jk!SnAA45x7pOptM+8*l2nmP^D&idll@K8wiNPakjGDtg#y~ux;(<}4@j}D{6-7j@W!dAe zuc~XN-|Tz)-tNAIx6AhKysGK0uI{Sp?&+!N>ACCaiFECd?p|v8&>G8!>m|QZZfXXU z5Q#u@vi@KsK8%o8F`ebL<`|UYGpQk7V_9p+e_N8bWTz&=2Hsx|Z*1YU*h+r&6=OU; zjvxx=bhPzELHYt44iF(aekIC>_!>LSPzVkzwcf;gnPsj0nDRUAK{>5DGCMQ3LF?Pv zSy3QgzE~JsO!NjrQ!s=&5y9VDAur7<+9QYws1DK#29F4YIK+t?Z=t}+Lv`G%@#`^wk@-HvUKHQE6v=Qv>$0h zTyNdyk4hM4|DJtpaEcips!5L&ucu5e_P2t) z{2;K(2(j8fi}zt_-s2$J@cuujWcjmH(BgRcIxtpV*tJl|Gv&ub6a}kH=p#6AgMb4! z^2H^e%Ete>?Rv6GoF1HRG5>;^1VS#hos7}aXAprxtN>t~?m2-1J%WUT%ma*vJGXd1 zn->8}yyF$#xW@9gvMe37fpbBia&WlHZSsHr{W`ko*7-E=&fn2S-q1J@NAjfi=cQL( zr&C6qPks03L)Tw*2@M`}kjN>oN};-*KI&8c_)?ke%s{L;D5DL>beZ0UmY^-OCJ8Q-h^}H zQ&UuJ&5{o18U6t!!r_R6{+KyFxPspB+2>!->|fnU_bmJamGDQ)(it{yE~Q5v|C=1U z@d~ubh|7x}=*V<>&gjVDN# zv4~3xMhmAiE?BaoBBn>7cf5jR7DR2NH5Pt|lR~8cwT-b8ep^R=tQy|JPP4r4-ldYf z0W#KOyg{(G8O0^Bq5P=I!^Yyt1^G-yYirV|5HB`X2o};Nc%D(pZ5RP#QvHN7dTRDa z$9g7>8Wj7P%$)AW(|x0eMlNLE+0;MKxFJ)hOFdMirt+aJ%e>Cj#WiI;^G>W!7>rkS zkKB4l06||}-gx|y#o@KidJRSS^~%0F=IG0CfXm1GSaJ4;HVqsfc90Ns05_H)9rS$2 z{DYzO70~;*046F!t9>GmegP<1N znw{}>`HjwClpUX=4py4sS}zbISSs+?`iHV8jn&<1gLI2S?r8o@m%m-H@AFp)GWT~h zjXAzoSiOS2H0h3^v~a9@gb}8&8Q3b=XNcoqxzJud5%5@ws`iTJjinE{`)$ZCBbHt+ zR45bm3xb!|+Oa{<3U2f!jmu+n2Eo&3$EVijkz}~ai5LVg1ImlhSZsM(3v2V5*Wz%d z+Oj-a9)82}EU#athnpM*Q6Iszl(23tkI@-K6?}G5qo)3ED%o20&(Eu`2^b8VJoKA* zA3ZVcNXq3Wg*6FVwrrwJ->;>wKmUxDz5fQixM(pgczkhE7V;s03SN1|l+d-c)~Xq! z6h>_pisq;ry+7Vq2@3HW8>1yBF0ZiL0NU`uBHlCR6)NBTn3&Zv(OPfw^915yVq_Di z%F=bbAI3V$?XWwmLNj2bQk*5g7>#&0RvRj$Rg;68ME^!UrZEUB@Ko)XZESS1n@~Ur z5qgQYI3_SzY_g}LEXG%tAHX>m5QX?B{`wS6zHAnKw(<)tgMY2KAfg_Sx88Y|j^o$0 zhn;jB-Ej4#)OCl>tOiwt!FrV+GNCxpi3+H;R=r3J8&>1AUg@?!oJEzq-x6(JOBMX} zG)&yhb_P)oAJ>{TfM3(AL04ti6i4>vZz!j-gz`G<#q#L~mlz2eUwt^Jv0vf!vwuam zu)_=v^7;<%efSZL9zTU0((`D}^;gp2haJpCLLmsz#>esXEI-y)T0jt%R-r;0C^Qg%UW%5B>-ECPKO z^#epOTYv-+d8Tbyw+4mhuJQ<~CL2dgFpG=3b)9jfbMiE>P%1ZGRUF~h12DZ02P--_btHb_^htq-T z4T!#IO$7X+{1Y{R@-mymSwjKzag>V?g?=1xE-9;%d0DR62C1%&pv&rdq=E(u3hNUw?7RUaw45QemecC zWDR}z&g=Bl?{A|CbK*GYxd>bnF%V#Eu`l+bY)_5DF9+kr5ML$n@0e)SHOtX@IXuDd*)##29&M>A#4udkorVISKH zPbq1`Dye90Q5TWgrL&J59-zmUxxjQC?>&Z(a{TUrM-msX6;d94?60)w*?-fWb8n&}e>jNS52m15 z;y^<2>6YJY5{Tw(8Jv4aecL~$Xb0JEi8g#iWi8uF`@ok-5rQ`2Z0c33VW6g02dzy4 zqTgZfK!n_OJ=n%O~5nSax`pHHVx zKVK>ITl43GWgm-!_>>cd(3~5tq;_qKDVzV~CJhxus0aDo!0q_6KK!wO6QLg+-8D_d z6VM=t>R}f>wY+9WqYc`r<0Upk7O-(P5f0!ZadO6}@($?hQT}FmEh)Q2d&=MRl{iar zZMN5;A&<#`n)$faH1mJWCI;ixSmcZ(7(Jkmqg`UN`E~2yYD|B4?yeh2U(Du34<~tx z9=xJj^&L*P4jxMr786aq?jgG1u`!hHmqpEy)UkbA?u#~OQC(O&z*yDJIT11F$OoQW zJ<*&k__*|C^dk&f9Dr(nl2eKHlI5K>P{L&?!8~%}(9|fjSKOPUURF1x__*0n3X^QS zHyy7W$4c`0vZdAzzYAz!SLX>0BjaNIZg`2_n8II>PE#=1nimvGI~IM*z~vM#15a#W z*5(aDkD>l_JVXvZE|x=hma@|<%%WK3*Kbo+*(NIB&uR0N$4hpjZZP^HiQ`-=!=P!7 z3QdU}%q5%mUMhs}1+=_2_=gR~%5~)dNU*Y9y41tsP%TxoLS%3kbE$})Yu)|)9Pi|b z*0)N1;*@7HDEDnEfA^ibq8be#45eu!`c|zcZzOr?6^GF?S3MJxhK`hIPcxqoR@D{| z#+qrvn8cXAOvyLOL4$~;HCu=6)ur>`tig3&w0V0EsHy!2-KeSmH{9_Lt5{d^SDCg+ z;4h!0E94XuQ=k5a%W>Y6|Ij_tC(^}t#f~@STi7}1UID4D+<4XY>9_2zlk}#IcUDaERP!cP<3>wTK1+eOZ9x z1{4RJ7Wt)XM2g&&om8(;xy24>zbY)+E^vVym8@+b-O9S^`Z;vdZM<@28;3{#{C|Yk zlYhTp4jnvjf05g&}v{ z!qvxV)5f|bbn}eG!fw<(yyzjJ$Dx>heBBK+Vc=9=#Wa@w`0h{W?C1)KI#%k-FTWJW z_3bxaMY%cb##QHAc+O}y;*IqG&3i(UD*f$YQv}R&3T$MD)!&yelvD}Jz;bFn6ufIh% z@YV~EwSGfC8alLR@*=)pPCa)_?*}GKwxXhzqEl>ng+(%|f78C|RTRUu3~a-C_Hke5 z$%RphgiSae0~Cg3&M;2ds<&lBwERwcqDodb&E^_IJRYz+*E(oG0nTv}EU&?UcMt&v zN*m`_E*#vf01;&3!+6-#;$({9XS@|AYMVJx zgJ({r0SO*G3_52#?Zr;B|N7PCPE>&#S=d6_oJ%wukOSJZ4O$u#`x|XPN!vq&f3$qd zU(TNiMnH#`HX(#uS@Z+{#tQL}L4Sa7kO)O1zs4UBAzJ?kxjXgw8LluXzgTkRXK`@~ z%e3}iwZc~UCi$V8y4Zh*&K)x7AMg4f`e+r@B;Gr~d?>(sh{)f^*1NzhM zy?FBNutY&#I)*Try!QTyk0z}FCSsN(DY@h8qpctV#ib@E>|lOi(o2je|% z;xUKG;a71Hn&lK~ZN(;`u5Ao8C~AUoO+{=AdBp|xk-!P^;KLCZZVy91V|fL~iC>-) z#E`^hh~cl73BG<=agtph-@t)@x?vK1%Ba!w&FXKFno;3~+4Ja=Pd}%5H(yN!1qJf- zUlhcz?^y_RFA*%L>6Fjo$sf-AkiqFUMu)=4{h02Z{p#8NSvk5)zLMJ=+93K2>;!0` zy19taWFqu$L{LARLpHfr;J%h!rmI+*6||8wZT4&*r+S&Eut5{e)1%`MObTbTy@FHD z8$z#HvzAUj=R$h!g_ko%MR;g*)JY@hm)A_E(dUeiJM_wOi%VaDO2e0-%rk68U`yIAk}tW7p2~`kU_%UVXzgJze3g z`{$Ca&ELOQb6T-%8O7f)BZqQY_X&zUXZBxc)cn(8=3YT!Es=NMeV12n@d`vZp?J*A zX2+TCIjCb*F0b^$>zKSDp_Tys$aOSwj|f$x-T7k|Uu$jba!t~jrqD{9B_MkYt|wtz zM0r^Kmn(HeeRJS6bJq#8uk3%z8{n61QO8~><2!ZjB6WZ7z4rvx+t1Jo=|Nr%s8_Ul zDTaXZL8+n)dzwY9l{3ZadOz|9LB9 z-T<&S7e7uocuQJW+o5#BeUs_@;e)8yyIS@- z!rotXJgA#ml8-k&f0-z|aVDIkg%nBc&o>^q#oeiN^+&X2Bj1ql0~g!`f>$u+gzPd7 zLmdMhROo8NtZdsf*Q{hPqFcE!;W&X1&2Z3J@NtVXh%~c5;tWqvo@EPQEWNfdyM8G@ z#KF9nr@gS`H9C3Z`SO*ifJKHO4?OZXtzNT+9=vO=>`{W7g)H*|R;g=h@~d7|AIhR8 z`t|cg>{UnmTRY3*cBf9wu+R8?$H%LvpaDf#KZC4U75o5`LspKvA(t_T?P6mQ`Vt3i z(}INE0}0Y^d7|X^J$9ht>wns=zW#1iKcGDF z{*W^&j~i?i0#!CaU~;Z#lJt;Md@N~xYuyeCJrAUb>Ju%+oC5}t?&HizgF&2mc(VE> zOLyE8f~5JaE)EKP68Zb>*(bA|W}~mBBZI-SjGn%0HZ8ucCoP$FxLbWzK*wIpH?Mb( z5xs5V)sW42Km^R=!AH7~n`}vkU_9s;ZouP)tU5E4)Z#GCiU;1LX_=)*{-+qry zJ8MivJIqXgr~dgYjhQ%&emn1GYL?CNxf@?xhfSwO>lr7EGAyv6)3{|0R2GR4dX57` zh$L5yh*~5zL+n```-6Z3RsfpQOdLuqr>!EeDk){d{k6Vj{Va=Df=S?2sv4X!MCpi$ z=vKx@jsW2od*_~#ZOM`)X>*#bcx*oJe5GOVs8JNUII7mrVw&?O z99~+;4=(v{`d!b`iPI0aq&mTuUV2IL@3M1uS=F4!tGHV5>Sk7q=%$?82%Jx%6J9|t zvjo<$T~iUyXTn2qe`Ohb9%zl0uDJE=Div4agk2x>ut|#+g@k>(3JdeuX@*sjc%Xz- z#~JnGGdwuJH9setx69(a>EHl;_0?D9t-zXoF%;ZW4K|XeN}nz|vt{pIJ;Zis+j^h` z;8s$ANV=ptTEhcMm&tfpd_0(v+~{mL(An^MCML`nlJbgsRN;vRgl`?1-je; ztvjy{7_C9j`gk<2r5Q{{FV#m&G<_Sc{UOS}%DI(eup!95-w*oHo%cRiMV3s7xNp)? z;WFJd&(e+8VOz;JY3?82q5)^?K4%5H9Zxq6A^OGt5#9Xk3pD8vUYS;qPwRPUJKAbC zw_kkQj&EtH8QZd%yb2JSIg}NKH=`rgaBTpvD_(#x`x*|YLGyzG1Fs&uAu~r~m@UK8 zMHg~-B~AJq z4S}vKq8ezl20`lM(X5tcFd2P1?Z4OVeWKNAz~_(C$bBED#q<76bI#p2xLCIua1?D= zaW8GyQbFr^;*!g+UF&$m;|`BG?5E;3&tiw>j_3dhK7rwYYFis)aBg^Xya6S0Af8|R zl5wYuCj#X>-UuhYLuC%3Mg4 zUdg!#7ox_|Xp;j^oAKKJr=NW(t8TvM&Ho!pzy|{G>dZAWrtp3<`Wh8ppAi!!kw1`8 z^y=%EQ-Zvpataoxb~l@AYCYTERmSXvG7-?`EO@9zHE0M~@t|Go2uTd7(v+hSv9?k} zY8-gtCJ_L`Ks>+gef8?^Xy}O1F@)Olx_zZo_6OW4njJMq<;Pj?ZDb zuDF%c=b8-kfn8JvZ)f8l-R9)~QaA3t_)~ z`chHL7V_F-D~75IzAxj|Wo6vfWv{c%C4?Oqb)12-AMFZva_Em7j|mpTLp1;>w>SseHyz%IqDi>Q$AVOz9lF4shGL435`qV9MQn$53JZ!RsR z{{8l(NB{D7dgYZ@=%9lRQihG>7uS+!o_R(bgB{wp6MdLyfsUPmaBUYu89_DBnM`yW z)o(OgjYm98IBBDPbt=1Mhvb#A*5sfxApWsv6I7dz##iV4j;X$#I#O%? zB8t9@6H1Cf4f@4xC2WGB5kUqsm!>UHV!bZ!nIcCG>nnMx`>7#6tBYl8$ny8wYq#n& z(;(CCyzpRp>ElP}=}V8~rkF(qop+#?!_T4*@4c2*f0ILf_{JamW$==-P(JRPHpUpV zu?GEDh6?GwIv$0+4VY^Q+zLOP@|{pJao4OF@oV(~#DR~)0OBS?#!jv!G@x2A zZCKT9q6~;MXbF^owt{>@z-KW~$T%(_XFg(V<);Z&2In+7gJ9<2urNxnwqv-Jin?vt zQby-o^fSWhn}!zV{^oal^R_n)JNdX!?i$LkGMh$JL(q!`Fbnk^&*m1V^hT@pN{juO zvD}Q=^_rpGTxppN_gCgq=v$sla=#!VsSVKlaX`<~4+GnP$n2!%7aFGy+_dVAC2WBd zJj5@fpIk7B-g^68N0s>ivR^jqChE3R7aDdFxAQ<|$evjKP_is)*Qr~0TF=<#P3!6F zFP6tGrERMM>fE+v+_d_LeD9Ohw7$fj#p@#^DGO~*8dj)7s{`7Zg;?BT8y@ugn$#YB z3S$aHHE|6}6#6Clo`>50GAOHAAqjQ?U5%~T0-_sF;2C~xJObiX1 z4{86NE`41=$JGTJBBU+AKGlSjoo1*IE`V6VSm{31p2MH@g|WT%5a5Z*wRnT5TRc2o zoX7zMAsCF{md_xVqeECBh6_Yq4gsy6#{62E!6?14cC`{-78k?Fofsd3QGOP0Fq$67 zoksb0@7;^{Z>-x~N^5yD%{BZLmeqW$TECvY;3xij?f(NBe9V#JB+Jdsk&iE6gGVeA z#=G_MbrhDt>~0Wc_;`KTKZbuqC`&UKt&g?!hTQ&r*&^Zxy7y^(yh(m-pYiP9hsPCs zm1!-10=qGV|K4~Sjo;xJnsa(@*)+GHWh>g{j5F!ZV@K2LygFu=cI~KV+cvZ#KPlXS z_oe8-j`DUq*0kWI^X?_3w5p?&ugf{SjK88Fk1BaQtKbI&m96;8Lakd<4lnI5;PDOP zOaVW&lzlw-i6p+nBwu6Vn)vNGS}{vAi1h5d=DeS2b81%dP4HnL%nfX!U6aX5w7hhm z+r$!3CRHqm+{i;9n<+`5eouDN`ca@kw8crD=#M1=B1AJdzA+lNIM7$0U+b?<2~eZ{ zCYQ|3*kN|n4Rhp+eUje5e$Z~l{bD-p({B&LtKB$TOVr!P$bFbKn6R z+;k#1oyeIP1aykK+ajUE$}-biDeob9qAADl!N)VJiu&PHM&n<_3fK z3$IwAw0QAiI^u{U>Y$@F^Z>ttEMe#!G_+5GFSsQtp?CmIo;jEv=dW%)K6XC6^^^J3 zm*1qUZNUCsOP4OCg9q$Ot$9ucS+Q4G0bYB=JJPHS{a{HsZN`>9`Su|Ij9vP`#0^$K z6%skjKp^8CbRDdUjP2XfMs*-ieue{D;5x~N0C-M^`$XI`!d}wKT^k3E9N6znuK)Z% z5Vkycz540^FpiA^LMDVCveOJ7tZ3#OpECDky_ilw<7pG8F~Rx^AOiZ4 z#l;ZDIjND;OqW^naL!Md@dd4Bg{{?yDAZ%_<6d;Vpb*(tl;J@gOmhZN3m=cvDk;AR zy_I7S$vz$-Py1=o`kJf;Q3D@$H9Gq6L+N)9{4wqIs}sY1H_#aa!nL`}9-T`IBMj$E zzn6Y?+@;hB9VV*_ojzq8-E!{S2o=#c8#c=EYL%2QqJUouDd52=p9j)hOk|iRZY;5K z1Q3%pJMa#CIiF)Mqs_cLb|VkANQ)O*vV-n|qX+XKts6Sx^%Hb>+{j`u#D&W)e(~~G z7l#tj-mGN9gEnXh2Eok7aqa~q84pAo9`N&ME!~GvJVKuK)1>t^Sq-8FJ|3x2QhpJ7 z(~ChQ`*?&r?WalWYqA9TJZ#^75Du@?)Qc4EYHVSz+(-T-DmUa zG#zhLgBVZBdE%w8f=c;801k{boA{=2Juk(Flm5Dy)2KVIs6}8ltp6;2ii!zwwoIt< zxqdR%a9P+J1LY(dZ&I~C9+*|E{Y;+3+QcI&Zzi0ZoPa(qb&eUYt<@Uj+H)QW265u! z(vd)89m;Ucr)N5mLspxV1^NOn)FI%h`C!2?c(~CS1QFwy*kjWC+8b}tZFk%qW^G^? z^s;XK2AXimRrL7%cLt7xKwLu%I{8gZy^(=4CbWme36145DETe7!Q}cjO0_%a!}7v7 zwIZA!0Y__Z2EoX(Iw)<_;=+O&1m?GIY2z%oK~D)z7So&44rI0{P#Er#T_)j&w*JDB zR|tEA!IZu_jV_a+wezzt_!<54i3fNtsPjy`su{G01oCIHTwAwp)=3P;R~Y(}w81!) zOB5f6w82Hi$OaX0Cy8~$YMX=80V|X+S3Bm&!^CNJ@4ffd$!WIkt%qpBVpZ7<(}q)c z2ba1U2c0*O`mxjOOWbLu=R zE6b*q{E+tGf&0=wpLu~^e)(lO;D7@X71N;Nu{GEO4?I9^dGX7D{{5uQW@F40ZK`eL zTA-+>Cl%B06aHM#lwfgoo(QJoG(%yyP>DB_odh{$8+oJkLNQjA_QEAuqljCmwU$6I zrm{F>Fg#IuT&C4TM))PGue5X52aKy##m;@0jGs>{c%(jovY<4CWa&zk`fr3^qWT_p z)ZrQDG@EwADC!tUx`OT+Y1>K$k-W9p47%_6^XcaYx0jpOT~GKiolkS=Hy%lz#j>@<5#Raxb?jIW_W z=?eY#-i!9!t+!69Fn)kB|KQaFJb}z*hZ&x#Vnf)L{H04c&ETlVWZ6FAV?&H&1p|(d zaR3nVNs`5BC~|R) zy-87aAPIuu1|yRJVa>#rtW)IB4P*P%l<{N|uVqfYY!=Vaj5;W#!6E;8?nQd|Pk*7| zryMV@rQlh=TpL;Mdh2H`vZ{uhK|Gm_7AdMbb)=zPC*$X_5fr?vOh#*A|0qXmN&y5x zSmi3@nH*s^9X5}_eS^gX^_F*%SU^#wgOX@MJ1EVVao3z-GpyXeiV$q3KjD&@az7IA zZGgz@Z@x`;+;bm|JO3wqbBr70uy3o}q5KU{yn3#(I>X!*uS$Dz%_jD`6hvy(q-|PU zJ=bCH;{ZX9<(xc#);kg6JJqT9#?%1U>J^b*)^vhrVk~(z2_DeMlCG_VT z$7-g2Gij8!=%GfOx*SW>M-vU>TfPOu?xIU8uchuaVxYvE`K_P3@4j2+K7$V#Agi{r zc@H-nSe1qLBl`;Cp(XCY7&c?5fg1;pVQ`*_{uoM-ai7xwIp9}xD`$+yp%=P$fHLRfIo=Rn1Ounpym|BL$7xoiHQk!Ok38}S zee=yXG;HWG@-YX@W90)0*rXh<4$AyX6eHD&=>r@SJ+E!uRr@LAhJkUwah6qHDxZ$V zxlB781dbMD5P0;EhQjg2Pr5F6yFUnyKT&{(2kAx!8a}igD)e!r;gIqij9XvuaJ2F0 zr9NDbpgs<_`ZpH&ha7RJ3@j-$g{?#~kc$8$DopK>DcHU;RGsi?=E)006NJ23{=Vf;J#$V0bT->dh6AG(nK zZ`l@pve%M|ii)Uc$ANTWUEXS_MTN(lGlJUicv9S^C^*`+EvDk)*3^dMS{HHMc+A0g zgRv%`CmNVM-~pYrdxK_ogJySw{hw?H#rJ{L!Aaiwc@1UNZ8!$Zcv#=eyuD(K5AwE zkfrq(gJDWz#uDky({l~BFyqDhp@S_H=B#gt?{_=MEY~@P%fQRg#6i05= zMS~YN%>smDLj1mnHWUeNJe7gK$0KB!eCl6ks~b0>X&41Dh8hG8hY=pu=gR(G`r!UF zc1Ud6w25ZVo?R<7l&$y+jdZUAO}zRn3h&@*$}WeFpLCDQbRFHhsCHWi!D060m;7Oz zV~>!pQNqcB#R{@Wf#t^vf9rChGZ}`o?Z8Q2&QAJDz6KctUcsx+3<59b!=T_o&vBr2 zc?}vLgUVa^AukT8kCWV*DNbQKgMy>3{3nh-{>4WO@D@`ZFTxnO-`=!y_nm0r!iDtl z#~(|+wEiq#OW&68A`|}7XtHD7x^?`pqi&_cHrHHpjW{+>J#k2|&_rG@#!5TP7780rr8s_(E#ge_JxAzU{3B7S1iR`zzn z{!VT9CKg|9Y0CmH(u9tG+F+KF417x3$R=jT- zz6jNtonvj&G#%Lrg~GMKPZCL9P%U_tc+GYmJ&2%wTypAMSpg3r^(gFIFj zKt6=VLEw-JU(G7rQcAPu-5F$S3<&mD{rw*vv`Ld8|U01?HuxnthkOhh2$EjMooS{_jL-qAqR{VGG4G42HnskW2DIz zGp0afK16bZc26NIl-PNe(@Y!@#~(GA+VOs@bLYrAJGOg@ z`Qnom^f{I87^-PAW<5YPb(>+8+4a|7FY}&ZLyzVCq}4Hq6yaq!)#y6H1K2(G{W$}F>g5O?kS{4kU zj0nz-KUpouHpLgG;Sj_eZ`AOi^5x=duf0}yrS<2{snnw|mkRkFHras(%D23R?~&F% zJo@OP^!D3t(~*ZA%pa}b4_oo;wK=&gr`S^E7hSPJg8r_?4GN9v6G5yZrsp7UB;7wv z$!X?iz`#zGW$ZNL&GIk+;cng_IwFW26ocZ_JETGIIbgg4ybv!ufeVpBDaIukPb32m zc)9#0yrgklK<{u%M@CVYV_*Xjpd8@u$%6`km({N^$$#FcOj)5diYA@X2U?b}Zt0(b zcjPTD3eTDQcPhmN08dWZ9zfHsJuFJH)z1g>W`!e%4W%}GQ`;J=QRL>8Z(w<#Dc}ve zbNPt}Ue}cO)VYnJ38GyY6kOV6Q~PJNNSD+89F8eakHkTBqCVB1^>3wb>i^J#uM5); zlvdy4AD&(|*6`n(UZk(MFJs~W=hYquAGuXT#l&ysrITcThjzTGteE$CfWr)qc^pL; zZ`fhhik*3ImK9)MGoCbJeAK->Lc7<>{=BCtLPcbynef^w8c1GkYYA68Y5#|OCZj=#(i2?0 z^$HF$jgxj>UM8;~|N8Fx#L?6c^_0l2_donNt@!+Nk*^%9>DUpDVDcx*x4p-wsvof09MUFkE~1!2J?CTyPJ9`;h|PDyE1Z>W@5aC}DNbC6`=MGfkC! zNPoKSP02GlX3;~<*o4%rX!x{S^>p!F^xDdrEmwlW?2}JEp_7k4ia&JMo(eG!;*S7` zQ%M|6x>!N(;radx!wt5q$Zg(&vUli6_)hD(GX5epk3E&VZ*>Jj9EJnp!0IrAwpz@G zmS*%m58=tnumG_)_F6dLxZM0+J#-bE_#sd)uAbCSq2NphXbUdn-k0Zh1^GN5WqTgY zZ;$op)|KwO^G!q3WN8mo)duZlb<2rbhE!xGY?0Bg~V3%!HDCD}pX zBDXb{p6Ww8r*KF=U^+lPWai_b z&VpQ=S<{nU+%tsiC3Q=3&DFio5VcqB-lq7N1 zH<-`*O&e*!Z|~d{S{Eib{t4CD{BQeXCEu-tMyN)`vZ%b#LaWXr~bRSo|&Zi?V4?-oTi^P!!%kw8r>iyc@3kQznd|v&K$V;kU`}U>bkMfS_dm9r++KF7hT#zxP=>ar)uX9TeJ2mOi=#`)#fGRnCS%i^4p~7&#Bj`U+H;-t;_h=d*|*n=%D@SxffmuOw@Rgx9F?y zN-2A1emcocv$nl?Uo#f&iFy>U3s2@|UNS-UM8vgFu79pPyG{bi;cTZ=+!pj=nb4lF zJH{kHghbarO%(z`Z9EI&GZgXb=|l`_igQo&07qQBVDSb)6AzCUC;V)jGKfeIw|oY{ z-1;t@NL%;|svmvwDJ^;RKfo<)e+R%qd^;wqjU}8+>{B3!xS1* z`Dm|;NeoYzOmxCfK6JsWF#hu7)AZfi??akKggyAjC+L!iWBDPgzFH-FEkV8%)zyep zk+LW@S2i_SlT!w&IJ+LkV=C9VcnJ+K&g7J9z7x#io3UP- zu`>-bWPBJ2L5D%`Myy6)N6kvM3l7dj!Rx zxbUGY!!aRpj`=5h^W>pvA+M-I-hKH)Q9l}d2tRy!p02+7YP#ZzE3C){0iS>Vd3nXQ zGw%(4*6Ble&YLHz=w$yk_XSj$2cUgJ@pvs(4Zym9f;8Cm70}0nK4OF^z@$zypNAC| z#G!uZpawYsG>8jmt>O-1@-8$c8R&s~>A)q<`0<(XgFZCB)%0?+T;p3 z3>Sz9DNCW8@(bn+~G7xzHyvGM~<6LG-p^!dDBURWwGb;N)Txxu3e*@AvCrBrvAS@j(->Z^sGke zG>eQJylX*yhq*V;S1?2EnkOk3hCuJGU$wseftd?v&vsZ` zro!Ml&J`5>consCgGlmmkF;J;XPR*C!KGN76zFV*{hef!VxGAF@v*<9r7ovWxsJy4 z4VHJO#It10O8VfxuhNr`-c3{IJf2dXni65Mzm^>(1^MkoX1?DDCud-0?Klo&)18sP zmKGO>Jg~KIb03eFr>=3<9u0z@kJpv|w#<()0YRV{!BxoYV9|+eqvHyARM10@{nh2J zJ;+q&@M^v(|L3zW(#gji#W(JzmDJ>>NJrr`$T$sf}mnE?IF;^NU~avbXI*Wcs- z^zlFjzr&f5teG*y!-$-m3i;|bzDn3{&)w+slaHeZAN>oRfByOOhd=xwV3p|zUMqe1 z<(KJ*Lk}W+F*A=>-^(6YSV`wOs)CCF^uP!7iC6vPjRAJ9;R6YIo!e6h?+?4Fg!ggh z{S=#h$Lr2EY@o`GC3RZ4Rlz#0;J1;=c^+56+iGOBZOtEQ;w^hPJ`W#e5Qh-ImBjT; zcs(#X8iWjR`mqf1m2_Uk2HD4-e-^#<_Iq^IRaeoWhaO6U2M^A46H5K?rOY$VI78&0 zI&qAAa4U!9fkP3m*y4>X2_`>Id>nRl!E^|S1APQJPxpjFHu zmtqx(SD*+5S|=Q?t1JWOF=|5$qCJOrK#WWrP?iadbLE9NoN+q5z~B!Lq=m4sAa5XB zq(UZ(L1E#s3aaMC#n4)r22n52TKyQ;Ivas8G6?bz4~U(B>6RbMbAK^@42_+1nN?`3 z!s%m<^vYX7&o1vtqec&;R9<@8_|;eRuO%yK&=CD*OHQxj>Aq1Ee9gsUTyN+OojTIp zx8K0#bq4lJf*CX1{?ZY>25D{Cporr&$DnC?aWS;M{J@*!&*=VNU-m!g*dNjEyyX0U z?=OqCh{ko!8)3xh`Dv5!+>yiRxFZM4bqh@6kN>!)Xf5jR`llTp*QRRwd)g$g9V1^q z!0iy-MOa2I{n4Q+Ld~a?m~DDFai2c0O#_+Kog+otYIQx{fKNG+1^ z>Kaz4o_ze#(xzn|$9m8?A0YaGGKQLg0wPEk;0V_hG){iw|7;cebJ1pg>o*2bE8y05 zncQHCdlPUHx$*T6Y;o{7CCu29d@%?o+aOMN>AfzRKewZ@>FKJ@n8+wCA3C(u^51e2y83<7=6>+;R(b+p!B>bp9x6&hp?R zcaTe0nX$8loh1^SE@TsZ)Bex1ZNtr_|D?%R2Hv><%_c%Sxg`+$v8iIcc_w`VU^&`+x1j%U9uS0t$CVJ0jI$c z)LcFcmqRUE@t4kd%MI+?g?OPuor4R9t3KCH?CdUZXv?^Nhx4<5^J3K;c)|%db>e86 zGUEmsHf$LE```bj-FDkenPliMzQ}p<$tTm^91Vv606+jqL_t*Nyw~HoBTu8f_v}N> zv3D@9petnOqBs@VX(sKLe-SVj#W|n^XJ1K!J-h?D^81hBMxzYk|3+s<@u^&_F*c%9 z1{~z^D`-3nT3RArHO(KThu*jfPOW@`1||_s(UJN^>r;iFQ+A^Inr;k6^Wz-BV6;5T zXRxOEqx7|re-v+Cf75OAxBhCgQTNgD-Tfxni>`g_F;hh?U;&*l=?ztp2)qcy<>Q&Ku=N@#^)zgG1kv*!|i>d45`sZCytFQDn zOf?-DOjH%5i{9es3X@-%GsMX(c_Nc9o3DY~R;=5qK3w@NPSW{D;&8%2<`i6>Ktvdh zI_N2W#c5W?6ZG}#*3;5A{wrx|`zcvU#}@5Ni>qVUZ!k?9c@~{<>QEZcOO|z0L6`nC z<@a0Yz(X#gL&q+tE_aR7o_+CUI^h_0Cdds8UV+iMTa~dbNYNt1bc{OnNiJXChO2g! ztK|_6Y}<6&UuzZn8?Ox^t|10r7U2Hj_Y24-dS(NfsA5ZwSIesO<{*aC?D-d8sng01 zq6SncZuiwbpay?GOFI$k$cHOyT|IUV{_Zo3^RI~T8j z^Ty{TpHu0|m#Az5ob`1G)>Wbn$i>IBK`sX87JS^|wl`?k8gV4?+kMy%HJ?8SP*_k- zEt;3oWk0)+CSG=pyw=yLQzsfTW=zJa=!qwukX3Q`5Ym-XE|#x|7Z&i>9ogC9JD^PF zs32$>SY3wxSHSY)ycs*8^4Nie2UMH+E7kbArFW~qD+Ma@}r!@-{z3pI`t&_J=YI&nkQG|(U@y?4O~ z__~H6{K9nOZTZ0?W-w%WY5jnb7F!-#o#xeaZJk0X^i|HK8nBKgH%9q$`Nr?o>t@jL zXZ+OZVC#T8rkvoOb*HbqR{c+=TL+I#`t{9plz2VwV|ECCOu=u9P3^y_|JxW9uZ>xn zwVB2|{+)ixaWwO$+vuZ@KaHn~1{;^pPYNxReEb$Iw39SKOlI%<&1`DLPe5^fv+HTJ z{h-LLa{aRlU>arIR;-{)(WmK^A%f`HL5Ya`$Q%}VBJsf~_=1<4d+HTHPX^%d;$cHb2SF-{o)m-%UUE0#&dv~V3Jz7(9c8VZu{P+W!{+Dm`q&|On zWB8T>Mp29I-Q=z6szsec6>DClZ~t){)n*v8wgkR*uMCI~ug4m_L7_{~w}nY#`+=-B zW9y^J%3oEYV^-Oj`Vda=O2hHZ5{7{RmxMLk;@vEwEV8OA4m1mnb=gJs_ zJ<@R9E#;ME70d(IVfg?6=fUdl>VnQOpnjceU`3el=eZ;9xA9sgzIus04dJwuR}gdg zBXPXKOx`H9hK)K`6Cej{DIc6;2>G#?1CGNXhYzAPYu3_lA9z$=)BMLj{y{tMymK|& zwtf;;n4Nj%nFI%D|GoF5Nf(?W?J&O}S7gb{*AHB2dqw@?^>nLu5@i`#4PiLjuVF?Z zN1d>IlHO-bMtQM|LD)!c%tK(8!R}{ z#tV#%px5&#y^aszcxJSK7JNKP4;tfb^gc{JUZrpR!vtXFv`I8-%+LHPG;n+r9n?D< zx3+9qS1T+@=4tsJ-#Ga$p)Ly^+9k000~#xBq?2f6M!XyBib_Q&XklpiCYQ+w->O!;P z{iyxPm(kY`U&HIqY8JGYx;$uCuD>G2b>W&ZY}C|tp)tt9qFL#n$p8)|ta@TcP(Iom zuLtD*hZRe`*oN0lpG=p|x`oD#8%JM$`6bPqIaB*_2Iy?wym?YDyo!0{)bX_UUVW&D z_X^A92aPRS6tdF=PAT03OWK7soQV8rG|kvKmBsxdKaW>|xm9KbdA!1m9b~0tTd17J zBm0^&R-M6l20dCojd7l#O|pKPvC|4calB?($xbo6o{92!|1qu`UN?ku4~{h8NXur3 zJor$91Z&@b@dAiBNmgV--U@y_Q$Fla#`}R`gQH3saoPzop5D(N*g5E+gXqa8pQIoB z;0N)zZS}Y{Yu3<7C!IvkKKm@sb$2H?KU=dMv43w1-v14+XTqT<9|aQ4sJuZ%o&1)h z$_P@^hSbSvCKozx7zR-*!vPU&;Fk+!oOH9lLFBQ#z^exEHSHwlUp2ou@W8{$vN&*d z@|fnWC?&74d<|Ltn`d1{PyOq8+OT0`RW+tLX699AkiO^v#w+N$-3L-#o>kBM=T|h{ zeNmv};b+k_nnBm4xz5J!-~s#6{MoZG=GM4d3X9 zV}ffx<{KEFwX%}Ts7nqLL4?4BGVgI2jLslPVsOXY=1NjfOo2KHNAZ}ffAsMuV37gg zO*&^_SNi^?uW0-Le8s1LPP_V_)NjENl#UNr6hs4%fR^54vu~Wd2Cp?KErFELysBDcyVTz0|H_`+xfuQuZcSF9K4298BA(t9>tedd@2z*a@!>YdFi`M@6!ka0lQ zczqJMY;KP@K1Y2)2YbLS5)c6#P=Z7-7DOlt5_YsPD|y`$qy_o;{1sw;2*E4m&OG&a zYF*Tl=Kt;wbl`yp(rvfhMx#fM7LMEMXUUQ!@=E546)R}af&0+}UVs9J8D6W!S5NbJ zC0-66P>1M5M8*kfEI(Rx57owHytS2vXaDWJ{Wr<4Z8p$DFz#8K4#&EhjDNRYKb`x_R`|?h(EsZXnKtrZ z@0Icr`tOTrT3+L-e9!^?>Gm6DWbE~?{xrMwINO{9w_U&i@yALJAaI#SIB0YRh2HWZ zuZJraj&r<&$q(qO+>_EUoeG9L?S@G%TTTe+Tlr&-BWLj#T zh>nNT?3sg8$dww&haazCnM>v71$yvpWBM^$0OCL}2pn_l|R%& z$_zlr`;l8H=hvOyduIhb`A+>m__i_)mM_1bRzGtsefjVtTJz#>spRulsQmktRQB~^ z`u?96)7mdK7%62JcM9-S5p;2JF1SuEjuDsHb^laZHGVpDCMd-qR!9|rZ%lxi9-nWZPlW=ywZpL!QeQ9bIN3fE(`}0oPYI|XgIF0IX|!davSnv z6&qGMwP@a)`GPm=hV|37b!%$N4m*_Frl^SEyen?)4u`kFIpb}M+lV8ms1@|tO7z=; z>j>W1;9g!Mg>#6@7Dt~r%$(g|I|PGr0%IZoh6lQXtW?8FIlT7CtFxreuw_fE+-gB5 z9rq);eDa0jkitH7!-o%-6@IpP$;#kok^d{|( z>)o7Jnql=V9A|jlRQtVEG?d-fgQ&zv(g^G1G*kO&b5m=EIMQ(7iHSkmFfV|(Fv|SL zLEua;aRQ+X_Tn-g@>kB<%;DihibCMZ+xQ@#;(~a%mV>$}|9~IYLF16m_!tajG5M4l z+^t@UTRO@CZsoxvt#xGB0OUV!)NsP;kopW0A09#aVh2iDMGI%msC#jXC(uK|44$ah z@#APz-MPkaz?U-r`p{j}zAe_sHBS3$yA9i~wPkDHjlKOh$sao&r^Eic?%b0mUwCdz zUuu{q(`d|)?qStS-l50S=rh~7`%dhT|CPKV5soi5&gd0uTV4M`p&J2)X#yW|)8!y= zL>m+mT(3b2$cM}m90ZQZh(RA$OJR-@<^hgi5V+)ZQ1fV9FxD~xT#DkFtY07AbRv4& zhlTX>Ux!~fJZRWY(^jQ*>(;3~z|kqhbm2~&{y81Gu z-GzPa>@W2~{kP+Q8#)|n4~4#+%yyuukUi`wX(g{R%%o7C&fzb<{90DlK{L>W=__^l z%lPJVExq^F3cCCLS7^$euhW7>E9vDgH>J)ZU6f_Zsr2)ArF+F=S+cBdua`@^ah=z2 zK*2F6G=*3VU=Tx)!3@m{ZA>9>J3wMi7p#7Q(*>`^VN?0`#lj|eEY^e6kqXGNwO>?ff zjP}@V7kc>NhiTVcccoizy_NS#I3e*~%?xLLA3 z^e`L{NQ$hjmN;L{k?Br*%Tc&i*r%7>9zsH0h3^c2;Lt+@>}C^$&MacmA}o z1lQoDIq;7=MdQ^z!4P=#KosJ_B*Ou<;Tc3Zew+7kse}Az9%jmCuB|&LDJTncuE3`D zueyL9R2LysMIoJ!L_rMBiO%41%0KQ@^el0qfdk!}-3=RvSe8 z%zuuf?p{q-JoS58=$+REZQrh#o_JsZ0cLdnuZxfwk{9Es9C%O%9FPwT2nJ4RW53{A&SM+Q)=Zq3> zu%7H+o$j`aG-nGL(-;TDfoOxk4y^S71P_Q99J#lnzbgc8<7OQS zXkb|}<{`MTFTe#!@PnR)00`aj!P()|kq-j!Ynd&06;$h@7QB+GrL301YkgS7HTHVpf_Ycc#EBE-;{bE#&ZSM8 z>c`QJ)roL=9)0xDwCuwV>9`;LkZ$9*PxjlVFMm0-Q0mr(SLPM*D!i6_6he>K_YJmW zd#EX^mtS<{bYa>_7p#-h4C56fFsRaSq)3HCbVLM&g3G{$C{E`?@;azd(TS!_IF()+ z783&cxbc$$!3c%%A%mxbz!7Z_IPw_`cp$$WkOep(A3zQIxTQ0((z=E%yc@XuSlW62 zopbn$BHrdb&|MnBs5{0{pI~Rbb@a#CW7ELZu~;~g{<17^n^FJalWA}oxMSJqlMmfZ zd-mBiXbX+m{@Z)|Z<0U#;~$=S-M8QtS$>^{F1httjJl5|9v-fbC0a)Fht_2m#!v*j z0`SCr3#d20`i-_H*Dzdc`0%fJJ}zw(G`QdxWW2tve{y*XtK5Jvy(0S+wz)(yh&-Zm zmsxbnI3NzB8H~rPruoH;Zq8vQ1>{2Hzy|4=7Z%ducihGZIbG?%G(D%J@CK-bj&fwP>&ux==R%hmsc+@IA;Xic;zps z-yXZk>yp?cUN(w%jxso?f_=gGx`3c#b?WlFq21}iDyqD&Uzi@ao&X2bRb2+MnJ zY!cFHemKjZZ_AHTfF}xmofL;Wpss3jiRDE98i!SbaG=4N*n&S002{(d)rw!)gyZV= z>o2Dhh8#|-zxjqPzW8G5+_|&tb@tqI&&flKMDd?Me9At3p;Z;nuHE}v}ebuQ5SCuLUGMn56RRoB>#5N|Lep4z4)`>^XFVcXP!5~HKv9FCer_YGlIfz zW)i)7|BY$+&e^r23>h71b}*=YBJ22W-rnZEvO5h2qyHQ7oLpw^3iT zbLSjh@OUEH)Z>Fq#<6c-qYZti&5+>~+G1HGPWUX%fDRYPoUl%*Ckd5|MEZ)^^$J0NU^s=yIiNrn6h zX0Fs54n~CRY|wO|b4Q*^ryPGY{p}zBqJKTVgnsjz-^kIXO&j{*4}T~Q(LMIqgJ8>U z-MUGggn{?-4LjZN>g>DkzDwA9ZqcGe^zp|Z3k4S53_fIlyjI(7ryY0>!>`ivDzX;b zCYrN7@wLyE3~gned>n!64jJs)STp&(ilqq+#;scM6C0t`aBgQkX(A$^<7f@S!Ptfd z8l9>J4GVmjsM7HZaM#ep@>-x2;K;3bX$Tgl+=WbXSO#Gg@aI^`5K9MAu5F(1>?0St_+Jx99y^h#r20bRt3ggjhx=qB5x8?yog3L`6b^LsKw2CZw>WezW zyZLZgGXxaV9(=>ZcR+Qn4<^!f)02oNv&7K1FAYv4NT*PFnoyy`sW{p`!jq|agHK_-U!a1LX|3|>uTMKLVrCZk{5%S8EW zS0@0cKv=(an4NSfzj9@{%V^zmH?VuXhJo9W_D^*#M#t7vxOsPJsgO5d8uUNbmq=g` zC%|nC({mwlV-OrHputKrea%r{_si$k5OZ^y$!o-g>vP!wQ%;*oHq+oi1L%+e`_dO* zt)k~%dX-*(^MCZ8|GZAGyb}9WR9v@O^6I7*ywVA;0Ajz&jghSe;N-zW)qVQz&Q7`g zs2gvCgE-kRpErcZ10#$J*h8#U%R)(m!z_pEr7OB@P!Hv*sr-6jK>IRYhKFsAUnJ-9 zjI(=a#&HPh0a||JVr?iJ`USx-7FpTusR}##aFu3T5xh| zTrTS}haF{GwiL*#rT9X13EQw_Gd>#7oPKh~Ni_1b6X=b%|408`vXtI>=Y4q%7q8-4 zK>L6lTk%H?V4L;p*NZKqd!k`8|AG7M!z;`B(ZK!pVFww%a>=iYqFoj8#})FpJ;9c6 zpcUYC*4#Yd2WJ}OlLdKBmfFd0H4b556K9+1u18@m#p>iV(?N^HVp};|cxJMqB8FvR zF`P6O&{%_1h+jC&EH-FijvnRMvchBZr+xtG-!HQ?IK;(=>ff zZCqI)i}=`Ln)TokI(+;9NzUnV6iw%iR)*E^9vAO^|CJqPeR}r{Yz~IooBg-<_TMCb z*yhvK{;!?!bNNEr{f|5$eKb>lrrq}v%@{Gj`&uSyx{_`@;|N+@$4X;`*+ci-PDlOl zkl>0TEYaW)9Mtg!*AR_%z^*PrhB(U>9iQw<Q`H{;R=ze z>A`F=()dAVI!7f9Ph{U@={2{1T2#}pn7{^N2mUcAx(}}_&@&uI2C+X;W{31RhoKHQ z%@F*y*=QW8Ee2D&W6$B6ViA#l%4p;Bm(%*%e58Pth5nYXuUZk+z(%9jYGe$Pyo-{H zRQ=()q7l*Y&BZcM0@>G#9VHd)Ac4aJ9T>0s;j5eY%B`&OWztR^+R>>e97RKqIg&P) zZl?D?{FpxZ>Lq(k;Iz}PS;DD;{lT#DzU~!6*KftJ<)!%+g?=M?HpRN3oZTU5QyXHG_mcrS#rL=-tvAsI9Z7o}_bm`oI zx^(VDJNMX;9iCmq35L~d*w7!ggrf|&t|(JmVpSPz31^(u88VrCsC8^G!4$@mKbAf` zL*JMAL0}75#q1K&#l!04G!t>Sz6Mc%i5M0iD_B4r62#Oi;-f;VQLBscn3T0CiVU=h zI9IR$twErG2))_B*adlAHD%zCVl+azahV*4Adr*n9MupSmtSzBlN>HV3UUdSNz!9~ z8iM>2FW?IwU-^D^*|lnmSP@b?8+|n$*(Gd#OMZJ}tyX`fQrO}Nb7|@50n}gSUIldg zq7#$1Iyba|Mu7a(`|oTyne&4 zX|Dcu4gG!5bJCA8>1XJ~JLu|Z=TYB|;dmwWD_cRgoz|nyA1=Xb1%LYeuj$Z(4$ywZ zoy$F=VT4bller$pcqUx`GRZ#as*6t0pmH1CV)1A>=8Q2Ymw04#*McZL_}Yz$rTJx9 zoI%hc7_|JBj+={246Tanu6PT5u{=Ew!g}zg;n0KW0&+X`1TflF;@TsDZNSfLVbG5Q z9t~!9gK^9AGkNXGkKUee(=F`iw`aoveH?Uc+qTJY6{W{HjG+S?bjuB$r=OsGWw;iq zt6p}`3#t8q1HEqpuBCO0u3(4Rx7B4!oF+pw+Db@*cGDCRX`t%B7ac!&1a#x8>JjP; zY2dhmgRDaC{jeISoL53(?U&o7g7Iyd&t}d@n$A_ zl^$iFpJTNWY6<5Ua5%!?Aj3H}*ayUlBy75muPWy8inDBXr~x9tsfSf&kV6B`fmf9s z>7i^&ok)JGZ*B zKasnDnsDceLt!q9L9|b4G$t<+fCmJzc(ft<7fi?6F|;#dq)_rRb5k(NhOVU!oCWSK zP@3CIaj!bcCWwffN}(2KHSnRO8Qfmvzwq3V)UjP#I(ytD+_`GLA@O}vjtciQShs|( zOV=keed7f9LYn>PW%T) zx3#^d_ODt($7t&RVMBpdg24h~RpOfmRUqiS3_)g%Yb@g*o?K$j#A8qYDZTLW(m?dI zAX_%qm`%UIqv*7=hS9KNkD@*utFCk?TeXCK@uPuM*Tu1T<@+&qnC;)cpVYCzT>tzg zS!3;HRtC%$Y|da1H8dO$L0=m~f7RD7G*W)`r|J_gj`s&Y%c~5l^2a;|<(P5-HNUnC zt%Giinh#Wyd}H&4wYtZAur;9*gmi8_>4VdF_#C9N?==vb$7 zNxx5ttWX9YxD$rX!n^iZO<0=fm6PPU$N2>Exxe}o{W6McsUlZ34yP5$ z$I2w^gI2*~a~XdPwv1O00WW8q*Q8+p2n%Ok_=6LL0k1&9tAos&oo4xXRgax2`Rs7P z91%{mJf_R$)kxXQAI=!CLy)l=P|k#v;MPRyD!;E|6cN>z6l;a!Br+d}Uw)VY^~71C z`f56C7(ZBE$2ZtTK@?hapy=;lo470~~EZzG96U zdj;~9nFWAT4qsqZd%}h&L+l4TVx<||5$B+jm4eN3L@ueC;}_LeZRC%ysk+71!D;3i zB?~C+))a^O_;~ajX=Y6!%@o+~Q#4sYb zMOa;ozc5eyl*M2q+&YppfWIkKSSs_3GJO z$TI6^A`RNb^__L^)01hqu=niFtIT+fifGn!A^I@AKuKmc+&G3B@o~|i=rc?Yj1e#Z zPi2s9`+KDZXiR(LSasxi0tA$e13}Zq0TmmBWQJQhZpz>sM#INy60E9RzT&ftQbM|& z!w{|qw{phjD+nuiC0M>V z#WurkaC-9ol5m>Ac3Yf%vC`01n+YTEVjIC(idAemY(L0`)n#(dw!|ty;8}0`Vi^QK!Hr*%{7K6$jtu07A9^4ydHRnu;`~YUpSS*psN~_z8Pp?dMsm8eQp}-Ua_6MP5nsr}>tc8( zU-V@<;oMRAw|DzD$-jN~|8(0Q`&c~8dwX5*^DF4y2mh2(e@#l1Q%v1^6(=RDmY`$} zz4gMAG57b z2`#8>rQ@b!fK4QL-j$YL);`NVb?P*yZ-C?e*BlZbOocH1$-8^yVb~gYwTFXo>Phma zx~i$avOdcCq?!>Yb!T|gwWJMzd1$! z!OF6*=87vzVeJo%%KEPejUJAsK8@$XrBz+)lGdBCWA&-g_hS^=d)S$~pS^0MzUZ&e>iUeJ;BT|sX$&Rz%F@a(R;gMhIwD54-a@6AwqGgs^GTd|9b>Wy(zZ)*S z{94tlU&2q}H7UN1xzv&$@%nc`{yr5n+Nt34ietg`>Q^VdH|*b(zYjGaTMM$N6t837 zdj8Rf_UDsEPFjCt(S^gA7hKju<#zv-aTV0jF(pj?@PqK?8!v}vo_H+WbLY+BtRMBl zl9^b**Z1EueCOn&Y>=rkYeLytkc25dr|e&|!i|-X4u(>^qL$VWpiB;^;vhD|;~BraGV7wJ$XK~1q@Doi;t^g`W13%tWd;%NwKKM}Y>>E~5V0R<1 zg8by&mWjDO47Z%sJ73ve>l3RO&$9fqCkNfKtJ#LOi}sSPUHh?GF^y+jq+&E8P~JYW zzn8{>V_oJVKpW6Lg0)1GYTZ_#l06dLGs@N#M*DE3+1jhG67KlJ-L=@x_V~V-9B#!I zZB>Wkw}^Ie+0CJ`<-M5MAnBGe<)T z>Ee0npNcE+Z15BH{gR_AGT20hU5ltKUZ1=J60YkLJHr9$6rl&<25abrlU&Sa^%%ej z6h9vbeX|{#h0uOE*}`I5@I^!`H?fcUb@t`9f(`j>WfCXLY~qD0m(b7SkL6kYL8Ee& zpZpnT&`_Q*%AwT-xaE)Wii?Dq%$6_eqBHr$oAlx({)!{)jZo%a(cjL3{KgA2b^~X? zel#I$HWN4}KL=sLa|S26@Z$Kko7ZvK%ChUJjmB3KLKrJ8QH^zvYDVU zs&nY6{A8Qe5p^f7*NEL=i~@(mc^FkA%;Kb3Y9)6}n<{1DTGPpbW0KV&&7w5L8G$Mu zGeJOaeI!CCC0iO>30tTI;R!wCm0ZHfo@oXx%`f~JziMkbQ(ol?)uWH)vC0<6OL5gA z`icDf`rCzPo`BB|uNw|O?%Uzj*WYyc`q7b3?r*>T$M*ib8kxHlj8w~?oAxg{@I31O zSqIaUAO6rudr@E4h5qm4W4>;ms@~`8$A!1w8Q-m*)Vk_Ixz;N3qr&8|1w}bspnj}4FsV-!P+%}ESW8U(*y`MyrwH>7i*8# zv1}R|lD8#pP#k&j0Fgmv63&bkL`#TrjNETFco&`4er@RjW~Ke4jCu0#D`9G(Rm*qaezl<=ruX<0Bcjd&n| z_eGg`4M${5?{)iU1<}@iL&2JN=f)vQ+K@nFCc+diRF>DrOwT}~t=#!>6GkI5fQ&cJ zg9gnoxbE6(czh!q`)G2Q6X3(zhK9j-b%|==e1z+=dqRKkzh}cYzxQnSGW7xV^ykjQ zhd}y(ab!syvBzbg*Qoc+A{ZzCvdI2_P}|ZqO??OBh1Zf&6Fh>XOL60Yn1aq+y~dKX zu48YbhQkPgc&XCxA#nv5Csk|oZh`-9m)SA)L zv=-;t5l7zc2_z6bq0UoS#(N}x)QNaWCZ1cdteY2a)iWE#`ifTam+MJD>GbmW?tch%+D2rG>NTbBX4$Sh+;ZJDqo6MOD8?! zMQ8cDAV2A<2O5F(AXG?k#Z^N0zRLKn$Ix>b2<<1FkO>HaK+PBt%7Fx(j(n6!83h?% z3U$kO!?}VIoGkg1-U#_)q#$w92paCEjI~g=d{8ML(79dy?YI1Fc%jY06)#y>!?9 zTYP52@E~6Q-eBFeawqDC*g9){!sOU#yDhEu)Pv~)t|I9DO7xvt3M|WBxBtP#3@rVk zmf(!zkJmauUH1f;z%)U_BR!)VJlG(ZgL>#F(>LmD`cCRjaY1Hc3Yy52mnz79xH1?{ z2n2ZqO^?hhX26M(u|*jgamJB0#uY@q8o=g@Kc z)52r#-W)!YeGU7CaMoG}^h#wmUjI|}r7fE%w414%t!pr<9MRMf;|yA&CLp6m3Cy4l zOs9y;=p$}Q2Tl_kdBkb!S+PG$S8!e|faN$;T7QJq{JE+eQKzPK;!*i#F5qTAMMub+ z=*93cfMu}<abxK^u!#f2WxhG*1#h?-c>9DCv0buu5jvh%x3j)K zr|Msr`ixrrh}vT+Rn!u6RsD~i#}vY9f8@n@1>vGCr}LbXFb@5iMSu3boUdSF`is{w zK*KS~V*8u%$q4C^NtU!MV~B%lLYU%8FT>d_e_b2JT4(x$5GH37NO+cRa!S&o%!(B( z%g*zj%1=9(Ex^NY*hh8~Ol>LqQvR|ZN++z3$roWNtEmNoq;V-N^D;mOJEsE9P#omCKE8s3-XT~IXry#)T42ecw-p1!8$H@KRF)p ztuw>Bh9oc_X0SBokM!9STRi#YCcE4ed69(|2y3sg zDog5}(5*MSHVGHq^3O2&<`cuJ*_MIh!xdj!DxB3jwF%p7wn^Ls8mo@5S+NapqrRXP zvj?I5aGF6T!A?d3sCc-A3imZl)_jQ?lQ1^Cp_FX0;4y30rKkq1enB*oTwDTs6Ac z#_QY48m_LP-n5^Y^#gN8;>IpDX^TYZP!{EAR(JKEjF?w+S(MWom+1(NOgf@IJdPNH zHNFTrQt^C_RXh6q5qGxoA8EK+6fE%LL<+B2wy8hcG4?F84gK+oCSTK}&kGXwm8G0W z)6Z*X{e;z~T%Th3q-OcCcsp|rvsZp;%IlNsTXl0?@zuttFF?a%Cz$&y($z7}A^A=B z5QKKLS7p%`^E%8QZO3tl!xV=tke9U$HVV#ygdB%fq#axF8X1(^G|;z#T*aZ zJK~F<-#Wbb@~h!bfB9RXKCiv;W;o%D3&Ic1J;i3z*;(jj#R!}BkL_3O1&u>Cr-FnN zGO+CsRT~l>lbh`yiZGo>Ayit5mlRk}zggbML%hYZ#Lmhx+e8B423Z$`6u0sbm)RGI zsuO92XX%W?PdtQY^#NjWiZCWT#K4SLrjN~_5Bu6z!p_@m6^=alJK?5Ve)|{-f=7N`ss#hlnKS&RqoZRmUiaNH{OzBQTAo=x3~LK3ZL%7+ z1s_|T)u_e8C)Qd!Y_a*~Vg0pM2qV)GCVvI+IPkyy+HVWV$7epVE2eV=w<= zUQ2sN*y6JrhRd$F&NiZ+Wes$Da`@b^|8{EzY_A_f?@kVH`02I7==nyOuXA#N?cBzl zRFIUUqMOh)kULBER-V0`hg56lQBOVr>5ZOve-4s`ZLH82bwMbYW5hUBGEj(#f#Gy&+Web`_;@ zMwWRAW9XE>hy?4E$9lv%S{C6+eGBOzid8pq)kTnz1w8VbN9E`H&+DwUTDajS7loto zp@1J>b%UvGP8>##7#{Z8bqCv1sI7Hac=452!^8OE=X>uLzZ`VUjkks`?yz;(2m_gx zLG&LpYF_20{S#e8WfXiqQIPf+m`DA;vVVhaq!V+ia{rL4*0E;)r~aP<#YGkx8?O5P z*xS7Z@9s89RSiY}BZ* z5U#spus0cX+m*o0hu@tvISd&!B20YnkuYGQ=zRiI^(x~7B=;+{EK9A*t3P2n+nT!& zC!H9~t72bBN3+O^1M_(QrX=Pjv2)3BG3Y#9ya*ZU@h?wgICsh5Jkq0TR;xCwuh%?M zSOy%FA!XKu`ZeV@1wvC&VB^_8HEG%=(s<$;BM6q$)AL!JG)tA{?pV)}yrHlY^J)yh zN;AbVrreOcq2&XWy8OXIwIV;qk)J0BREpwwUT&DAP%fscZh3u+1FbTBo;NQskE_OS z?vz{LP>?8HYAXuxm921B}&}d8Z)tc+h=0A;h&Q)UsL8cV&$WI&T(olxe zNL>jKhu0tOD>7&xHs-~uE!MXcAL04RZeIxBI{mz^aH&c&W#VMSkF1(jj^x*6KYaJy z@cRG$9q#}0o#E=^&JK^a4GTN1^T}|)vP;`)vMdkkJ9NazFm(9vFu1KP3?AAV1`Qs9 z*EaA98YTzmz8oXW#3$`LrcSlrw8@h~>%<9R+Qf-r>i=G_-#dT(duYL{*^A;;)Fn6g z6nG9dT~Hqy3`^I7v~37{#eF*bgMPl}7k3E1`{TVuJ0&dYqq8w$)g9h8{Y_=B^^+9DdLv;;6KkU0L2#=@L zGSD{}1l9Jc(f0Sav>&6?6rF8Nkn*YR2~wNxaIQqVH%20%(ML?93FqZ))Ce+Xjr3L? z@0UEa3{@9GhMDMK#nU1f&rUa{N1#*C7yHgwj6A~ZEE?zElwUfRK2kV0+Mn_%q-{cR zg1T<0Z%}+SIE#~Jbo8PlmM6jTM(8fjbjCSkm4qwZtEq-WL%Ja4CZTiYbDqvE^3R(N zO6V-zYv`p`9#s!PD?_apCr$aGKj=dpV$uGtcMghw@U!c~yYIbUq2T@Y+$D^`vh;sE z@-M9VdNPc!LQ|2c3lYm<)7b5|`fS(*uVH>-l@;vkEgg9MqrIcu_JWy?S3-v1-s~dZ z002M$Nkl3jg*P5~BCNIl9-$4B)XC5; zD;c30vS_ z>}$E&?Dv2AORXY0jql0i@EgxP6&|?z58R5PA)ju)GVt!N2cQ9^3v^6?|M4=<%gbbd{e%OQe z$;^@~Gsi%qlL!f75yrr9MCSa8=_*&Sf4+wgW>c?4}G~)^Ys#wq)m^I+oqRNg~(b7sSuWG{sjSNZA zorlNF#G_j8F#K_#aIY7I)}zJvxX|(|t}@*l9^8Z#L2!O-n5u#DIzNwZ$}jy`M+;pM zPRC@~jOjDO8*jfIet5<8uF&MT7+$;h(vI7NF{4L?uYKiAsF9KB{+pL9m;InGa)Q;(eVe(>SP!^z*d*wV^}!w>jM*aROQ z`8#O-c<%$Y&r>`0!s(09f(iG{HeN4$6)RY`-gG0})Uj`vm#%;Et-|wfRm8Bcr+|EZoSaLO^Na6c10m9Q;dYOY=$yS^9?aEC~%6L{Lhf_Yig>SX# z#$h!))NO5T#kjQ9H%f<3n?K91daJN`iBGLInCPIpzLR8aI8?F_+Ar&Dyn8ZNl0QXg zT^DeviqRIANT)1};^orskDWkx*54vJrFW|QXqQ`-7YnS&ufk)xF;iT^J1awoAZb!O z7M(>2iO^UBuTh-kRu-N#$w;-pz#kQ!}e;k;o? zJU_$bs>LcPR*=oGn7>3H3Dpppn&Nqra6&1jR(+LM(D~)H=`O#@`jZ!pm-1vO9uzBB znO~HVJ`xt|o6?Ar!`Ui7#N;Zoi+*@DCZr~&l`?!UEVXaNYfRkayQO8Y!4;NWCak*R z^0raGEm4oU>c9%I@$Y>QCQZh`sePI~d2Aat46D>e+JM7#O16|UZ5+A${2I}~fivvM z{PaOH?CUMuNO{4rW5a=XrRtnZeq!~~fvr}~MH zN^_`K!OHw4`bgN2_P45*+b=6TkNAI6ezSG4|L(s1=`OtBy^UAG_CEO7sw>I1!o<~< z*eH4gUIE)_yDe%PHkgw6iQoUHQlkyfds>ZIj;s(bF<7e)^E4=-`|Ax_vFOC&w3x(lh z=Etj;nDC(dde#V(BBY`S&#z&*qC4clzH*y_OS@f-_wAhB^gR`kpv#l8Q6OvEYo@y1KM%ad^62Yo;QH7*o zs7LQq`BA)U=BIfW1c@ei7GEt6v}OL8xg$M66aiQzeLZ=s-e6L*{>HWTphl0lp)OGq?rbcf?k>$ay*Mwh%K=sY1y$VJA>T$ z$*B1HpT*!n&DP)_Q>1(25Bj{0!bL%zX;3q)&focnhv&Zj_%6$O)&8 zj_Kj$*WL_QT>HyZp@uJwO?@}r_|uq-Y7K4palZHj_#&b;ulaMFH|)|j~qEX49CP-8z#AVzTH-z2|IpnGrK~toj>)T|Ai->dL}&g!pq?WtZIGp z?RPL4_g>&Why17jSI6>+{|KylU3`&+!bh=Ub(IyD3u~^na#(qVWo>1P%R)KS?ghP4 z^OF4&Gc+VhTj+*Db`i`gOy6Y39(~hbo?5`<#)#CDOF>?o4Ch6{0{9tDz6gpIOyg2K zOD7WR#B}1|$5QwzGFMq~)fxSqGLFE$VY_UNq;m4Jk{6ZkZOk0tTFKD*=o@VWnr-W&f&>kmz7}_UuuQd;(A3g#f z2l(*AFclBZrc9Y;*BaX_9%)>ESJ0OR4}=V5buvOWxAds%zP3NgqP_TI5G>=C*pCmA z%VaIfVYWe}Y-8!sPQ+;%{=vUFF*+a&u{Q+T(TDR5f$g{46l1j~!hH`uY&rYm;poHm z3kzXPI}BfQ=l+C4FezxSXJXdO`lv)5{ZYQYuCw}fD~D~C`YN?u`X1@j=L<4ViW`r9 zbV%Els;xMk7a(+_OOZh&S`$P&%bKA4Ekf%qnJ+Rj`k0SV-A#{dGQeVthhlV+6<8uD z)z9+dplHe^nT#b8Bs0&2H|00&WV)K0?eCIS9@X{*)lYQVh9a9;n>5ST+{@eDf%CH} zl|?h*z>*h42A*;4zClLN(IX@LjP;$JGwNs^;WI(N%IlOF(peYm&Kx!?O~m^@`lq5((u{qoK>*ym&! zpS0OiHuPrx2b#=WunTbW14e_d>9D8v2M-RiF>1$V-9xN0lUt|S3X~%b*gNcZ`0=Rk zREyW)k{@3kw%hWv;ghScY@0B1;#T!3`&ZNch1Mn&X3%U0gGztR_79I~_J8n)*x;?L z!(DXBNi$5opZERC!<~P=FY_ZIoOD0_8wZBb_(~sdK89hV@m6f8%zd5)@y48OJzl4H zW3oD4;pQHPx>#B{?D|z65c#OeC_wOP_Ww|p>9nAkhCI;WyQqao7fe^oem`tEdLtF7d0Q?P|GQ>WTapb}Kgu{PU;S(}^i;Rj(dR+dlZ#22om6R;oIgdWsjFxu;IwClFE;h_~1f+Mg$ z+Eh%U{qO02W7~)6VH#GS51BsLWH&odDP|ig2tl~)mBm?Vv@dYlfnm1TpXFF-bKQY` zD*cgA{i|RuEQQC8=&$dG@!ARDt+(GT zWa|$nJ8idRSpQRNg;qY?!F7wP=r{?^Hl67#%HJR4nf-P2>#((1E1Yeg3R7FoYgF$0 zrC(hSa?1XLshvrN6bhFvs1egjRE<rm0aqSA2Pq9vTHfZGuogPw``#p2kzngpKEEmR9FE5e1fnYBsj5$vR)xNg<%cp&pEIyJ_2I!&A(`7^n=zK&+#EaQMig3x|F2 zddv5Ia!sMmTrqdpx4s?zeDhVPD_$eThXJf!c9AA&(SD3k{fVbxZnwYW7EEO)-ju)D z|EUkleWTcZ*tXP1@Z`H+3K7M`V3_xOQ~en21LgViC}#$zbHzRJFyUp7DK=tIIWr<`N^n-$7w z;FEPfxq8@hS8Pp!w!*J0+oT!VFSjVMwxGI4`E}ZasRp1N(+>{{Dei5)q7M;iKZfZw zP4X#DVyXS{oOh4>SzA*(V}kJ5=1iT8W*UN;qM-GDWaG&d{8R^(t$c#4t7x1*uVWr3 zmMWX)R$hcWCm+>;=gM2wfyyuEZ^{pim=dWO1mV#@)V*}3D(PVJQ@uFBKl?T-*Y20`Z+4GbzbsZn?m zW0E`tUa(mJP?)k+0zM!%-_Ow=5=z53>iac^Qq1 z@fH6Qk31-h!g!Ng32+=d3~iTVQOtkiKEQve{EaKkoUR%D2X(0WVOb8Bf7G+sg%YDIS$z_jN!ye^x4(46 zeS!Y->Qig1hR=uN$(lAl#fmS^`ewF>KEZ5R9596(i~2`eckE9- zR#!0MjWz4RCz#wDX6TS9;hP8Tjs25Os+jOQ?+2HMZMWPcthBuEZDf8=|7iMurvI|N zc+MbL(Vm*^-}{(m|7U(;{sCQZLb?Mt%KIOFVtDhdcPuSEEWhm1*bI6{TVXQ9Gk5Z91wGS48IsU>TE2+WLp{(-E7Q5%fUUy8s_oVe ztTr1u9TQYj!pA*iIxq;9@xHge6<;`%`wu3{PYZ+%e4@6M_KoTv z1@r3D2>i*9bKlP6NL`uEef$PuKiPo;;olfP^5HBe(N@Rn_IySEI~V>iOu@dCjT%;6 z>Eq#BM;v5(53>GzYk@07ZB;aV*c?+HoXRg9xn3rIIDWQ?*fFiRRY_X#q$wTY?e~jc zoKMqx8*hib)8OsZbyIG;DP4w3wSrP{M$lgn`U?ArVqeDE(}(gl=Yq0#!F|E6x&QdKy>eS8g~eSCDN+C5orV_?hSg)3S)yD=(8ZHcsQyl;2FunS4%|al`F1 z&cD#USD1%PbZ4_~Q2Ar$4_juQ)Ic$i9Bu8R3p!{MhriU=W9=;+ao?rd;3L z8=L-5Wm4~MBiD!G2pMEY!LmL*zjyxC>2r-6LpxNKCH^T7;`PeQY~5`CQM1r(FZO@r zWfNxY)5EFfem~rUFJG1eR~Vd#4|DMAW8D0l8~3;2$G(I&B6btacA15y{iO=|y&&UL zyu1D{zKsbX8Qqiq3CICa;a8}%55Vcn8aSCQl0iJZh|NCtrUHYQ$~aQxw0`FKT)&o# zLy@CyAT+@9y0bo_7bl1O8Z=Ac>x|rPfH&eZv673cayWs-J^c8=opVn+Hk^eCg~$H& zR92!m_9nE~GvnYipLwL{-#7rRGG3z5%bF=E3th-;_txKY%7anWwVcvOb+h zU)evecWsPi%l@^g8Z$-pCQR{awXO6C=5c#DeQHmS=W4~lgZU8*|B^Yk_TZR}{r-BN zS~Hw=(vjih?_3-{_^>BC=vAxbtjEdc)?0gx@U3qgU~gCOy2Z&du8`&!*j86^nOUKA zrt%Zd!}zye&QInbHh#jCz~7t--ha4m9QQU}t3K6yQgOBNG-kZF71quBK0@_*-uAo? zOXILDi1jTFJ+EKul((XTEH%=+tXwOKV+_X;DBU2>IgYb95UC77rk8{IT|uTZzk>1# zGCsv!zU2DN@}tb86)yhFt2|8aZTY1O!fHWv7bHyuZIV@Vf<~VhK(gfW^^W{pJ?>>0 zK%blYmB;uoEnjtz{YjxX<&hkMjPGsvRTj%me~jn3$}U6Ys|I^KX-47rmv-%5fUNA$ z)16Kg$eRI`gv#TmT*6bDJe3?o>9AgYucJ3+=FUs=5}-FIjtY#a#t`*&_9!hj{Ok!Q z2~}#D8+pWx5jyqUFF)nBC(Tgi6aW5C`1SAp#VgJHsW3ZdpTe!Iwq|6bevyD zG8+{1<$BGtN~eqw?vO`WgsH*_89KASrD3eX%~uDt48$h0Ej9q-Kj+dSNJ-`~SC z|9in1#p8nyJr*v#;yU|U-$1O0;rck$!AXks^ZLyF_LsiP_G!v5AI#k-*OhC_b>!*7 zQ<_|-q$&EBd{43E{GItfxiwnCTDw zA=neq_Auj01d!wBtx~#g_W$$)4@D2@x&155(tr_=I*Lumfl=XCsxf`f2U0#r6i(>D z#{&6kC&rUj>5(h~#R}3J{U$)RAYqRCDmAuA#qrumJm(Nu8z|P7lV{xb39mNdhMW^- zl$8?;3yvKVPCe$(aKp{N4cGtbb}Axe(`z^%nsG1FY9TGreKBd15>Z4HS8>KHk0|n2 z2HKV7`edQ(C9?^VRf-$yfnYMAyS@$@OgGs1k z%8>r@y1UI>##~nV1moN{$gSi8kpXGU&(Yt{oay`c(MI(1m6l&7oOk-M;hal;f;SzW z&x@R+*nqI>4qJyWec^NVS}EJ5?YoHwwS()Jc@(;z>v=IX9Heeto z=CRM{oDPFp{3BlKKdo-4l=Rn?`neBwg|e||tewPug8fCcP%U0@g3OoVsn1y&4vOP> zTCS)2^-OKncp-zfL*O~_C;#(o_}5eawR@>&pL;31k4fzh@wM_v_)0QA-ZWx(n>_&i z7`}qe1q&-Iw{%z)Uo3CIWH>75Z3FE?xWQWWN%|1gk&r>M3Z`+yB`YCA*{8ZHe@*_Z zTeO>(P&_Iv%b&_@6>?q7PWe?9>oj5Fr0~#VPll)e`#fy1JJXJQ zh|TfFavxjLY`OX>D~81uS~fIL zi8K46Wz3Q4voGy$4HOo`1A5q>HyObi{vh0ROrAV79DV4%;lLwLwb#T_Va~kZhhghY zHww!x{ZXt`gLT>PV&jvo1(`3!=T`emKay3jDSy-csV~&>e|7Kt=*G*hz8?0+>(RWZ zl>=`;Hdtp(+s|e=F47~hw+(NYI1$eap)P{Tr?Le-KJQ^z1kY5Ia8W3qU^zd}MeDeq zrSvIYOa7E5<=2<}AM4v2@{>;^2J7D;pkVDf($6%25_IaKKglwJ%tze))O3l{%$IOY zC=Sd(T}X#F(EpN2GUru;f1xzy^Kq00q2$fBz)1z-tzfT3!1*6_x=`CZ$Z{S z#d$6niCcF?IoU7;BTbr5IH5u{ybu@ltT;Y6dH;hC(f0lre)p%phI=0PXVs67CZdgEtkoAdA%bL;ZTmugmcxlo^5%~do;Tk5`|#(#J{bP-=#y1##gCb= zU!F9n`m5T@ee9!Q+N&|$747k)nQ?S6=TZft1g5dP`@lsj7k90SXZa< z`YVoP#nLKc%H>OL!Tv5kC(CFn?k~}f0qy;NegzA&g8C;x__BUoYCkA+=#+$Q|||86bFG7*EbBTo2E_|=t{ znD27qVqfAyvZ6=K?e;H9+myd)|Ds29#sBFwY;)|Br%ag=_B!Oa@b38c8C?k66K1b3 z?qE+RdBedr%8!Te>w)Y?)Q33k9ZI?UX8&Ivxarzfy~owHu6+F~7^Xo_Qd`px_+t>v ztIc@+ioTntJ4XK^3;9{ux3nrOp9FJy@lm;gEYETGRng_p>+5yn(d*~XsYJmuzOKm? z+~nDU>oM!ekL^sw>zf@NGi*gQKXP;S$w!7e|9l??!*^7@p~=hdCdLU6Y|r4ebZu7W zP^9n?gnOFkNs)(~UaMRrx1iJN%wcMOOXuXIv2yMlgUP==e_l>rc8>Ly{mGGa<4?FD z{q#V8}x5AffN878IT!qor#w`T!HUn2iA9nOFL!D8l7qq9yd;`md&6rsvvr>LFO!*Vc+RL&Az->D50I zrnmF8O!07<;!$ZTu*|WXUd&y2mggVb$}jWsn$I@mlx6wT{7g&Jr4!Pmc$%ltan)O} zGg+wVRNGNynd})45E)KlhFs--o;VTdNT&Ngp5OnEN5Z8)`$f3z_jh9}8=v4^YN@5d z=Xdx*SaZ!a!wM^`5SCkRxv;cY{GI$%chU)}KHv*nzdC7x8*Cs+gHxSP%agWK5mG#T(Oe)E(NiwS|#^JQx&#t>U zTz$i>;Zb}5kAdUtPk;K;VdITA4lA#`a#()(fSr7|maPLfCf*vHdU~kCJdQ}T7k3rB( zg($*$Vb&(iQme$mC@|4XaGY3Njmog@sxT`nT$T6Et}Gm#FBdIAO$pA!mBPxHVlpBY z$h@BJ<)eloM`kZm8Po_O0RcDdj;B%B$dl3Yru=L$TqSbm1wYCS6EQwT+iTa)+k_cs zrTL{V`@B2`4txS`T8sL0+qoYw`|PkkZ#K{;pezi)P&W=@CSfHUSHOJA?PY6 z^H?9Fv4}_??cw|q{ZmH#NiOHXJRZtYq;%Ac^drbz#8bM+s-Xev%b)5_D47Iv0jQN! zGr7KU8go(baLD+SH_zGU;~0kECT~z2L-PJ>U|fmD&GlDY?qlJ*ryL#r^q0SdU*7gR ztYGfjk701!xIQN8cG`A}@Uf+rM7_A0#;;K$v5y(Q`f2-aqHS{A%e@HcvHk-{0$sEy z|G+S+ZE&Vn&m%oDqj17%f#ryX>B759j~nR0cvyvB8dCdPe2Kla6Ua*Fp_@xkqKn#Zqm4GoNMC#H zwQ$o-H-)RNx+?tU512s1Z_U+K3`ZZbZ`gK=aoL26HIyt@jz~9xr0X^LQH*Oohb%e0 zcFPWgmIl2k<WAL7k)OE84E+&lp8Sd<5&sKN+vfe*g08!zDktCcO3byS6{@4m<2% zw%mO4&2431xs3Fj^&E*+w46}mZ@cZbvt*9zAAIn^aLqN>gln(8He7&hSH6Gwwc*P< zea>vT#Nvx&dXdb`D127sR~<7wAcDB~=jp;ahdrJ&vj$@YvjP;*3ldJqz-Xc|Im2VJ zR>J~ro6Ykt{y-WY%LD-dG_5xwXpY_9e%CDaPvmDWT-q~Y+Ubt8;FRGky=j${-zWN4M zrw};le%xXEg~bl24z`)H%8~YOMLD<5THMjw>`zGYFi=n}Ug_Iqm(o@MTI6LKM zbyTik8W*eEDX{v2f{=lHTn0b)$OI=%a;kC#yDEReqG3@DiUe6v;)0pZa1FzhE%b&# zHU0?mkZUWaiLaCyyVv0e&@l+buT1HEezCQ}SH6`>%hu3G27t?b+A) zcmw5{iDK0NtS=={y-Aj9nP;`bOngz-gZe0k&Qms2Es7=inew^5+~%o0nO?nSy<`7a zd-#L~@q$cAai{D*N7bD$#S0Q-hL(;68a4(ZN zi|J+Y>(Q6fjW4Pr{_Oj#R++tu?L9y1sr@^bzn~lH8b1cXGH!|e_>i_Y;i(QxG#XpLU;HBMy8n@3-1=*W^G`d*_Ri7+Q%+K=6|r35gcc-gBA@HR zeM7Q#Ccjt_GMCEWdAUuw>r-D$We~22G6$#9an|dT*U@;EJ(SEK2~Oxx`;p1Q@kwS+$w(uUI#h*lAl(Xgp*@m z-g#RaIM~-RF<{`P-F!MoTgYB^V#;GAeo@ENrJYvMImydG{CZ-4HX!>b01D*><6L^o zC-y5Y|FN(m-mAay*TuaY_?4z3Pd+F7{L-`0@%Z44<<5)tPi0Q=%$77F!rbcrsmv+f zlpjWC+pS@L4m8+co_pb?aLBRWri)aBy>{IJ9}rp5Zix6=<`DdNBf?i)>;kLCj+}|SDUOs732TFXcTp!~fc`|>R&edjQ&WVG8__F3q zd#dBi_=RikJxbWluTg?-d2GJCELO+tz59;gjW^y7k3aRF@cfIfgx6kwE4=o`TefjC zCn6L$ImdlT$BZ6@SJ39O=o#2MjT6`q+yztJDWu|vmh^0ylS3*8qfcEFF=rc&?DzLTJST&4) z?}KpP-yg!)WSuto4{(JCt}7uH&R<*-J)HgOV~E6Hr@0JIJ6jms}P z^IA1{$RKNr)-1hgnl9u^E_e6kk1h%M*mqHdIS&(B3e9xu>7q4A8b+&~S{_cSF+}=8 z9Ao?M1hxE8erlriql2|GSVxdmS6?JCB%`0hhV%>hhZ{CCcRU3T_eNRX^B^_xDLxos zxmW$-)^PM`=h;T^8*aE^xagvb!WwI=q2hZTuDa@~wo2*LQ%?;?9(iQA_10U%x?6oI zoN&a~!Z!}s)B7Jx&akdC<7xcbBsPYg#Lb(F0bBa7Y+d}zm2h+qEl zm&5hfUvIWN{k+S=snWZ_+s zGp(m^K{844(rI1(${S&YpRkSz&$|3To7P8p2^m<7;&|R&ei?7on@m!?bXu3c@V)d47;EI7fCes))pH;mr(vq@6 zn$%bF`)i_D-j9xV$dCbe5|4owUS*j!vLj5x;A@}VzkpSA58DkyRxu#Ix%2Mue>dL| zzK9LV`N{zKP$uyst~zJ=Qh|gEk|@PXr{YD(P{Ch$BkU{t_m)3pvZ~%Dl!=xd(xj&R zxy?;gP>ytB3X`%TY-IcMrk6J~{F2fh2Ok?a30)3rePZRX$1dC1Q>gjo8)^HQ`D-p* zHNz+N#g(09e`;j;tJ$j%7nLEC@>I5-)9pW2t)YQ#DgNBX~pN z!Mq8m_kN^jO3JSqt~lM;h@a=ztxC=(r}vUndR_-%c|ANP;QIcy`GRI;x_W5C+p z+F@;o6K4E+87Iy7TE=ATE2Di$N8$@uA6;^BYnzOl!F|e53AWW~(=agx-n>q770)DW zENxsr|bDr(JRTnAo%)m>2X3Nl_e#~M24#2o`C2&FSYrzzS` zb;@YK)bPk$xYEn^Doa8-W)(fltcEL2GLTN8Jb0&Wdoqw0zea+RLy&c&e3DVn;!T&G zwS|%`;{>H9hX)LfU)97UF(1mdVO+|$8amqH=fftOk9WX7>E|Oy3=a!0v?=x-`)ruj zJ}tcX@@wH0yt?_y>u-d&-WhM3lXD`C6U(jGPK2+IjU3U2{dg9_m*5r(OE0+?CW^

    z&UH@ho0`QDy0v7kska}ksR8|0tp71 zXFNG{0#HM`l&SBBhN~H+euC4(ycqtNdXX$i9gjNlkG7o*FC9in^Cq|-(BVuSrGBi{ z`6E*f4ikXI^-e)W`FuN!d+#%Sqod#*Xk)|HGn&K@oa0&B-xvbi+%&twi;sv%&@uLw znK}=i{l(0@PIaSIalDB=o>EIxO6*gnpktZmxBGVLm+33L^D#QU;88*^vzb>9Cs&dJ znCW#I*}M3Kw{LyRin8B%d7l59qmI3})a3t38JgzEq*CtQZ+dmojD?c!gqR=6rB%j< zzlNPe@RIJIHH}W?wqa@%nEctoBfVPmeG5dlac2amACkIotuLJ|k%iHMj^^CiT4M5q zwQ~JWEr-MQr*MONFQttY94ftEFe@7vOf!Cw*Q%~P#T)dx#qh55T|Ky~1v|X>wX>(+ zB`$C2H`!Q4@+X%(v%xn&a99XQg^@@*sZhzQ%@&Jw4oQ%_yTbI!!iRoVQ`wruBd6CZ zvdCS8He^l^JWy8e=t>FdBKg{#XSTAwzNLVyR1?m%ZY-)LgUdY!!I%<~G|g6x+$B?D z5opL$iveUXygSQ}1bY3jdvARC;d|=YI<=k7uxu?F<~HHbxE-zO#xfhFtnI1cs5!7^ zr9IU$W-HGNi3?*@L(YL^Vf?;SM!rI&vo9!R(Hn1hcSd_oQCBhGSq}|&KiODIr!(6_ zMa*^~zf1UO)9vj;x8Z}OY&_FQm@)*BP`vlfw!&QzQigNCi;?BbX;6g-LI^iTepCSv zA&NbA2t3IRZ=|7WUWRqd2+jBly^GszvhoGGZ+0BEbHnP%#mRx|YhiA`zm~Qvg$!(s zdII_vYX#1br$mb2iW>B@)~Y_4S?N@?x$D&PIj=6Zs^FW`c0}51+Q-g#np~c`JLb0R z4ox#p8HHnMV@6C=%vJVgd5c_bmEPO^Xd^mWttgd}_QK^U=L9t5vdxxjO-IEhr=;vG zr9ftfQC6Mn589t|H4ro4#lTLBbcPc5$D4he$D(1*HFI}pqgQO~!XHIZo~8k_sQ2Ix z%vq|bi>7W9Eio@F(t(Pt&Z7yv7BxV{ju#-Xp`O^J8ksGC&i4KyPCWk3;_$!3W|pS! zw`cupg^FUZ{q>kFKKaMmikDTl$di;^Xk*_%hbBg z-?9ewYO|5>K5(>3w@0!I<`nt*c+Fs3u)e)gzuqX0-l>4iui2vP`fzdVsI#_$vzZa! zWkgf2Web@;)l3nyNEe8M;S#FI%RfwECQh(`YR|)s05=KZ8JORf>5cn))Y6Y(@11^` zf6gGkbjfdH`zA>KnIfz?D}oi|-*y zVAr{$-4KTsIKT4Zme`Z_Itg+|oxuHM{N3dGX{UPGiB-RIq{|uRSTXp97aB9QrQhsu zS{0?$1zM&~wdZYL<^13>G1qGD(_*5_F3HvY>u)Wl7<=0OuU~|Q)u>sWp=iTg3}3L2 z2?*3wHubL}g+Le7k&s8h z$=Ztou1(ysk0u@g+nZfN*fWEJ4mRw%XLO4FoQl1L7V>v2m+|qPOZTJzoK;8`vCa#2 zbFVYQmxyk&aC-J-o&l3d@Ir4h32VhZS8=BYN4!Xn>zV%22Ux2$m_o>%f=x%|!rSIZ zdyF8!r9ynPF{uqe{_tJ01mlMxgQKjD-#plO;t@*3_pr7F1O4?%a8Bg$9}V@&tD<~X zJ-|#yMjh?wk2o98*DEd^q?h=CD9OVYsih$KKmXE(P%eWHrl2GQM5$$*VMAV z#5m%?&VX8g20#pd)QKtxP!e7}ThMnUC}lLmliLq=0C6HC^Q z`=u7uIae{H108Gzz&ie7JmIX8$2z~_>miQGnW3viNI+%Ot-dB&zEE^J(7|X(Tg0Zr z(>^sbY`*hoa28TCrQ4P>*T3|_M&;z(XywWn8Q%YJb8kH*v#H&^(T@xP znelfXps3R=23IvY|EKX3X@ALs)&jls!D{*`DCZPDWpb!wP*#TPE5 z<>ImhBRWC7@FHuDw_Yv&rgTA<1pClge>{Dgg1dUt4f~OKU6we`(N24oOs-nHd}w>B zjb|cOwU4&(sR4Gi!`bqa(&%he7duFzFl^2)?F#*N)Y(wS^+-_)6*VpVteqZ`O|2l` z&Ev7oQPm@plj}k_EQQRlPAA^eFCA>~FNgn2lTxxE6U(;XJZ z2oBjr*v&xFJZIms%E|_}lSZ8r$g5i=syjz+p!&n%^VO7GA12l2mYom?S@ZZmKkQQ! z3>Gq%=`=U1c&Ah$Za+yCLsAq8JAVa@#io~d^{gpeCsKdC1HUO#ME#N+TV*gN_S4>{ zT8(h`2umW4JF!b*U<9qy2X%GVI$@y=`M00$HwjNL5b>tKXG;hqN8T`PvUWMC+#tSs zxgVuqTh-j)B*d^o#@D6GFL7(!iDE<4!u1E50YiR!c*ryU$V*HUq^+hP)Wa2FoHmp` zrAG=N;OB==zFA=tD}xM2kYQicVfy=}B#d~U9Xd}VUm%jRxp3>Syn4MA&V%36xY_4Z z?55(1I&@>j2EbTw`m7ph~?~Bn`8S9iH$}O~Mz<1xo z;CU@Xz+J!A|Eg;_uUk9&iWq9mXJrjr$gqFRUo=&`4!ZmQ&UFY3p9^%@EOswu$1wO8j^K`vcUiN3kX;#i_D2!mM0+i z854r$>RG})C@|ibXF!Nyg?t`K8mdFoX1=N4p8|a|y11aPC@T`dCb$wZB({-`0?;Onb(zWwvWWQPXt{T?Fg@1-n)nq#Jbkdqwvm2cE zU|!o`zktz!F-U+qc6McOKEo< z)BleChP$l_Ik~;$Io#^cf3A(dr?-tAOWlOy`r3Tp=GF)Ug6jJQ`i2bP{bu|V-`(C>~Kp$CJadkCZR+EkDp%@Hhm*i{!u1g_oetrD*YqEr zh9x0C&(ZKegrq0r*0!F~xBTPd?O><9?eXgY1f!3c(DFFm58ufkEm3r>d$LaYs;Zhu zn}(Y-SEb*ATie9?lU-ids_Jkq4UAR2G>6qTV%ftNz1_~Jcx4z3unRQqAbhWK>^>B6 z0^{!|0jKNgjv?HgvR$$NaP$7h5^2scZT_let=^WX>}G^rEvAnSI+ee#P;Ql9#*JPG z6G$6TU0huJ`24ucNY5=P>U9@uh9`H*?oEPc=Gd0YL6+5@GLs_i(=tK!buYvvD3pbU z!S78MNv*DT-mK)zY%trbRCXGhbBm-=CM@h1WH0z2B;I=T-r6W30DiO>++OiqA0rMPYld6g8jUgHfBwj3sxwpEr-?d^Ffq@T~gTYRdHxwfsC$!yXOSh&kp!P=$~u-cwFFmBS}!7+CK|55do zQBi-<+AHbI5?=L7VF$Gb*^-f#XT%jSxj%2xOa-$CI z^-qE4pzfRAUSIC0R%v8rWlF4Lz#yP-QB9;3@}S2-4=|{Mgd@EeNjml&Vx?)1nK|+g zZ@wwq*hNN^uD z#Lya7WBGndT~%)-5zBc1W8h1jW%S|edL=|DzGYb=8N715 z!n-dAEv&~&#y3phF)6&j31WFmggPxM8?^q?Uw-1 z+t^uw&+$#O`J7tB>(7$=HW~Adl}gF4nNL}ktyMMg zeGUR3q3L3xby_97USR+1(wZT-TUXF&ziz^EOzryokQ!z4i^{&sL8V!eRjm2?ebT9C z`EUC9@@FOEcE{ccFD*mverv_dwuj1kf#qI^w&zGlz}CATepIC)wl!I<@rx~QmM3sH zwidSAL#-M56Qf;m@?Ig-V_*=CXRh}!OQqTWeO{*$-CW|N8a-RTr?m64CB$4kqqqS} ztJSu01D}z;zfr5KtLp^=;pb1gwtT=r@&Kd|TK|{JDE>MOVPQ=Ko+SuD$i|AaOO{gC=F6S~JQpzdM4nl2 zOCp2HgU>6*!P_zy`r$d-miNq_DWS=A={m9t*B{ z-N=F+#kyYHPBJ`-tMz?j1fu)wpsn%7B-o*C?v1+F%^hn=%YbMAQ#Upk{{vei@;o@2 zw(ZTYu~W%Vyuu%3FEGIXRLDR4t{na-<3%9H=eLj~ZJ1yzF#e?@b;&SqhS?7{44(?U z6=g4f;7#((Qv+qJCjhj2-;>&bAFb`V9K`YNkNL?;K;4r~tUd+zY)XGuJKw)=6>26r zwdTytF~}Qxv`GcMKMYg zn;T!GbKgN+S*L`3@7#wk3rTBZj6mca+wpc`m_o+KJ(CO_DqXn~NA@fgDXjB$3_|j^ z38VJZhR2Z;w2Ok&CpF)cI*#G2X%G|D; z%;lOj1$3j>5K&VZ_>0(=QT7@K@QJC{LKFwT1?~G=B2Ykq$~#uccqDG#x@EYyu|mXY zKeF##E^N$(9?%Z7kZ^DzQqhTCI$5@#Jryu~_Y$Sa(4|Y#n%~xTJ(X$W0R`l$G00^% zA~RyXRx#v5k)Aqp;@l+@Yu=c7&ju`F75TQk6=Y4UmC}#`y%K$LNM>E8Z`R(tzM921 z?$c(NDD&Xhm>pocI^W~Ijh~OU=N%`i;V^-#vHS$|1e6}T75AZRz`N&XmzlJ<_zh|qCTW$JRdW0LZvb+qOh zvPz6ERO9nT4W)CWC?Zzbg1}q|(p0#@DN=(oYDUyR#>1E_90r|b(}o&9LTA3r&|iG$ zb$W40|}Z z%JUb>-`Z*^(VoYijo~rU?T3sID-UkUYL7mHlu3<`$N$1Bhb+P;F zf4ugy?+-z_)h>SvuS&|WP_NuWeJcm-!ltJZ%}GA!N0|x9^w_P_*!`&w4{HSb%Zmy& zEqY=wY2af1VOR3A#XwcgJzc9->m!{^K8?3!vm}h#X0)+{GgMrPqAw&)5{OTO6lBsF z!T7<+;PPyqyQxO~vMPoQGL@L+V~P*$nV^9+6p4!%8`NM~NH)@oA-jq8R;q#KtY7h2 z+e>WON<0YCFgpHsL-&Ek7s_Oa$)tMjq;Nmx^{V?~Tb(I9N-}$s1x-24|ygX#wZ{1d`rT+76lo4oF}UY4Q<;rjw(Fl&XM&CEttDm3QGOU}A`Fy`FgD z;gqDzLt?b2s{P6<-Ip_hnjo0eQ7b_}Ha!u$Ome1YRT%&9a`0XfgY5D(S9hDFtF!MJKy zfDA^zjDOzSA$poK*GY4LyjlM9B8Y0mi-A?5$~w&7a(XxBZo60*j2nj3A;*ubUeM;< zg!?5NXN7VFPqI-qB)Wriz9R$>OQ6-* zkBDf%78Fa~oUir|{jfE_RwVw9h8P%8M#{z%k2J!Z?^ivJRfq4u6NM&HPW#ZU>rCd= z_}}=`=%qraM`pbl!0{Zv(5q@0cc!AFoq5TcWzFn>j{^#jgW3i0%hsV1F(GYODvCGB==>%d9BvYX#H-|#w| z-*&Jp-H!yBerNC#h%r+h4m z7wDH=nUUiSdS#4^21?M!2!Yz*rXmV1%r@vadG95r+&EEmeUixCmC!Ah5d=q(H+aKq zeP|wgx{h(#^W^*jNF|maSOB9WQKtBq0L$+3ZTQBEJR||Gf5aB`LV-SSpdFcTx@h{1 zOpXFI&QAh4m!psDdEbk%K2$GOIs+Qvt!*{OmCTu zkLxh#^OMAe$W#6rzEc37_f9CiSDPwVLdM%A$Ybo1_bNVqN#t65sQ()U8fu!u{pSw3 zD^#21650`bFjwyQ80cGX8`6- z3~fn%+Zf)g8yC4&Es~>i<{)($qhH%^|M{Q=UoZCX5hD*{MUhc3+c>Q$qB=9r;B*_YU?zpV{PxYlKDH+JRtr|?mF0)JyHyn(4DWkq3kYuKE;e5%^QMb+ey}{9{Kq?8aV=C zHSP7*H@V%Y#fPNz6H7TRjMXjLsBZjRh+T`mHAS<$Ue_@{KXI(@@D{9+zn|fvqk` zbqDsFRXK~(si*F_*R_r##W{l%%NTFv?FUH?QyYf6ns9*d0kn@^ z0wBNU+^0APwskt5L_&t7_+qT^Byb4Y(DuuZ?nSz=j>hAnmylR@kL{I4pXe1mo<{PI zW9pi!G#Q;L{V@M>*mfug>Gtm-)f>MdLUYH}Xz9kqliY?D{l@_$1nNZv91vYmL_&M~ zARiqZDPmv6hFZQ9%WePbqTj=mQ(+$Qa|nSo!mtA`X}tWuzoe6cwIY1C%uAt#IXbjy zbnl0cpt4T}q&l?%9NrM<{dt#X)p4zlGCF84Muzv!Qw<95h%<)Sx*`_9oEoN!1T@D+ zH~~SyuBdxIiC33*l-R_32MhM^W+@(6e}eP}$icy-*&1R03i4a&Hr9@PYtgd+uk915 z{jzeA>r%5LCbHyvQ`AqWlHB%o$ww}wLvLXSVSWpE-)Y`Hp#1Az)zxx*y8(~NM{KB& z9eA<;snvS19Jw1?LzfA^N`@(J5>q*dT49E~XLw_VSrxjG{v&9WlCPWI4gsK06jSE! zaiCwWnV?}aYjjl9-X6-^b#H0M_OF{ZA(a5lWnxg+lwQ&g8AoJp=z01 zP98F2*>@g0rC-O`jJGIX?BfdRyf@^Ko=Qw*2|KX?&;s}=T%}q_p6%;U?(Sq9fr%4w z))@to$71+93(OZT7wmsM5(<0rYij#?`+~2~eEdrE;CX0QKj+hOEh*D4ifkq-#;yCo zFb=!}y_gho+jO25YU$()43^WzTVQZ$5l}A4`O2^aP&a3UZ_~$)vr26vKr<^vZCQ`w^`TegaQ$hxyMX(%F$DCOn9& zw+v_2{L*&m^Q8M1uXN2a;0~Y1 z<_DMEVJar>#tQ~8>>@BCJQPzp$OL-wW!{no6|8{%dx7|>T(WLDkuT}~S8<(~)cnysIOZM5qZJS}Ov4yy7|UPHE7#`r=&^8H)~ z9E%!V{MHtJ)(&S%>4j>T9uNT#EkS-}X_4->tf?VHyS$ilSsUAriWB<^$V-_oiWKk* z7_udblmBs^8y({t6RT#G=VW4X9R2)^KR?M<)<};{L?x5c%nSuXi-MwUY4CyZg zaAT|T3-JNi(&Bdv=tTW`X;1ShQ61gB{Orf8%i&3XPU**Lc*PnBwYRUM4<3aUE{X+u znfBW2-qj#GK6gA}y(z1;GvY(G+}AJFvldiUIOAr6`NzXEa1RoG4vm+(c*A-FRyh2N z&0$Eot{t;VI*eVQsh-`~qYE|7q#T(2Q7vb>i`W&jL0`5K51#litCT%vgv0#WfNX;)vDaFNq-G$m|vAXcP(znum!I1d5GBh;7^y$aB`g zmT)Gwz}l-sO}@H?97rLkLxNEh_kATQA?BncEy=_lX0MoMK>p<=7BwX*C^1){lKx6S zLxJc)jZN}$3W+! zMz5Mzqb}6GqU}e);sq$tYlxeCdY2GvoJznklTpYRzkuo0_O{heh6PU}Jv{%19r5?o z!YyC7#&8^E*{2J%sF<^B_1y9>T?#;{D#?R6Uauo*M|JOTN~2NSy=+2x08L@mbquVv zVGC{r#Xg2gFcI`=C20u6PC>15wSTXrVVWpoC0Y$l~Qhh_9Pb-yFM7@{Re#4_jrd_jE*=Q&R91d;5-9W10tNozK@cyPPp(9Xk3V_?aPVeG=7v*&w_Dumirsj7 zhEJ6(al#dcKXZAU^~&+=NCacJ6q#g(_QAZ^;k4qf1huDq+Z~7YG!`rP!YzwU=UZ?d zqD6Fm-5A9c?g0)^&VA838#+a!Q@52ojf7JAFRE=S!lDx$v@Jq2G|ILeD!|5OuaM#1 zi|tY#|B!VctT4g`PsIxV5tb}Ews7LZ3g-y#XUoXZ>8hlP*30?J*gSJE8sni0`@XN+ z`XHOtX_V{^$)YK^9=a`m7bV;N(@Ke&+Oxm9{58Ho+Av2;?i|pGcLCOr% zhd)_x-N}g|t5s;zc%wbo7I4nT#rK1{-?UB8mFI1Ed13!DrPkw66aww11<=di36;{1 z5f+p7c$y^iy;G;l5OMCf$M<2I-ox*Nzx#Qt{Amfr*K1`$Wu5g^ZP8&u6V=6G-Dj6% z;_P;5xhw0t0nN|-)~wZ#Nc-C$N@oik!ACg(7qT{&=<@mT!;qp$2Knb*jC-_^CB3ld z3YQT4G}EIa9rrvsjWxF)%Z)m2S_zW|0`!0PGS%}@&C09*VuFsTgB+mEO!0t252 zz_G-~my;hp_^w(*OZ53r=IIQrcj$JR#J}e7a>MnOY|};_96X7AqzuuQ$O*3v4J@3j z<{n^>Co*S9@*iP%Fv8#+FCLaTUtm^%Lx&T75?MbHNpFv^n3d}`By|6X*ZF~du6#BVFNVV=OwtSYDrqn6RBPfMVcx2#Xmk70#1otKD z6rl(iMrp{5Kal8@wpl9B&rf0=uYm~@9cv0C(03dV)%FP8B$ap1{q1I~^pCKWFRP;=Esevpv6MlhzU!M-9oHm1oVy6kC+)1L-I-$HaG`P$cj#8^>%J@0V zeNV{5HjwvYGXlEHvFFp<478 zxJcJFkfGtgD>)6>)9JvsuA-Z|Om{i*j?!J*XyHNH5KNc4N;c4RsbbxirF!ej_65QK zs4k)9f0oY<;9zg1Yy`hO5ak6%ejE_$bEK&u=qTp7TKlYqPIzy`{GH|eMU&#efo33O zbUE}`T!bS{pw&7=z z3ulD%rwKPrRhLhtio6vD#)zh4{(%7+mVT?s?lwpK4U>PzIY8aqbN}HVM$~{`7X5O0 zyhY~;5-tRj2#b%bJoF&Pd3cp=-W<|#INus#RKLFwSKhMWjJ4)U=P z#A+_NTDZt>l9UIWcfb+fvH}X!C@YVUK!-BYAVK4m_{7GttCNqW(aV)JH-HDOmOR$l- z9-IE>)gR1%=0~;Q@hxuF+$}B9U~Qa$U&Y6=YA}8P$G~-Pc4P5i3XD6<6`=8PnD7aV zJq~eKTU`6y^T;&-P$>b}#c`T=DDX|NFGEH$WqgwKvQ&e^F!xI0Ath$;RT%hXJ-cp| zvt+d!OlUBAY^K`A!uW9J#mWewe9ac)J02D635@uo=Jq^;Gypn$d}T3%ox`oW_tI=- z8>lMU)HOg0!NuFO*tjRgf@lFZFcWW+Y|A$yDJ|@@eSOEiSz%kJ@V`0I6qX~2w<5A0 ztuCJu#MK#+2Tb_E7{LrW8e2ubMbpmLL!Rpso_taoRPODe@940C{ZGgWBB9f%>kkqq zM8eW8bVrL~nWtEUG|xWCDTTCyUP*rNJ(mbQrYzYmT>O$X9Oeod7_*n0S`%n_oN1b3 z$s}%=B9(7&pC6bS53FZ!@rU|7ilzYvu=~X?aHJxSN=-m|F`Q!4phtMQ+o>PB)3ms1 z(-z`6Ni%Nz?I)7ZxY=<2d-)lGha8Cx{pE_|CT>wrSkm>jdq<^K-}7~_9Yq4 zo*-!eA@$h*VR{Dqwh@HYdT`3PaHaT5dUJ?`@$#&0L%Z|e89D0*0ENsKQ-Ov zEV7!7Nla#6VhM?@j6kINEj4&Kd;IKD5K%O6!y?rGfhZZwhvlxi_xsSj^A>|XS(Z2> z<>esGUOPM>cxq%8YP^sA8H#hx4bdu8ACqWlRsN-m)LZsYFo>mk>5bsE0de@#-}cZ} zG0!$q$fK^R0U#h}F2xT0v9L&Q0y|J6kz@Cg#M_EP)*DG+9gyq%+4u>jwr&qzqm%MH(q#(R6!zR!TozEH z0hXQ*y!?!PMZXjq>bLz{c439>qpB!DjupF6%TsG@5~gm>%cdqpR`nl`-upB6O;qEn z40xJL&ky0%wz&e7!IB-2SO>o{eI#IHGa;!#R=oF0do*+XhX@r5(Ef-2s92@c_|ftB z=PRMGWj@r)k*H#J@t(=n{VQDO^Sb{K3GWo3CDIk~sd`+)NPmm8o#g7G(FE3)f9s|= zCmu6bA2*CT&Mjv0>^0l6EdZX}8^8hsbocej@lB|PHcpy1A(LV!DVN8eJdNmpm6w!U zWjoIzBtQ6^_iR-@uaPiMpoPIEZ)!-j%;5))C>ahh`^F4l%@FV-NAt6|@<6f6$_x$8 zW1zo!_&(MHYz}MAU^2CVjbf6wLew7dc@;Rgl%`IfwW@`Zl(0z(Zr6`^@o$O-T9-e+ zK#uZ?+mp!P7*EzL0y#hYG1m)BB1;l9q#PZ!{u2uI50?s?lo3dEio3fpsWVc$T;uOL zxEw|Lzo$nU%$hQ#MpFU|*E3vZT(HmIDEs{UPS7|SGRiMfJ-!uSD9!N25sl>!m51AH|4$S!@ zZ_&6j2>+?oUCKx|s-p!e?mYIE@Fd6@0K2>mb`m!8p!x&{55xpfmMLdzc8RP7>D<4M z`5A}P(f@2hGf1`-EYbMUF5^c#HgKo>gH?D@AWm=|C{c;3F*Ol#GFZj0(~5BR(LCCd z3?v=aOKgveuRneOg0}6XSYn7JQ>xrJ3iPZXMMmY>^Eh z0uc-JOxw4*8ygRpPkRzh2QKv(&!CzV$A(y9IU1;uYu@d-fRQ!aN@OBy4S-oFEDQog z|LQu(F(N=qlf78*KUxCXM(ac0S#T5O(QNRhICBx* zy?vh^zh%Q~;WuNY?5@zsN2R=n#^G5ABlP+<*|yS$2gv;8cw_(JLIA2?HlLSnmbBxbXp-EH>^*4wND?R% z@OCc&YR-U%Tn7n9^qsdT)pL|*8jr?U^XHXAZqP2s6`9d|)*&elJXR)TTxMXF<1{=B zW1tNK+=3lg^n48Mj`&8D^9^*v0+DufDN(9fVnQ>t7|+Luq$^ecd{S*wuGsu{4NspC zM4=yLP?|Bb7D6=u`w!`qNV<*#PpZDasRhs`nQe~4CcyuJ38KJ0Ds}kqGfSY%f@y;e z^zxKe`|DNXtM~~4`eD*KJd`3s9t$eQU{rgW8rG?o9;A`?ZK~hV* z_-wCzKrT6YCen|&4Jr!QG9(#(?lI?L+JIN&7|C%wSPJxPd|2t7^4CQnL}s%lKXe7V z`8lZ9Q$#C5YG6^vE4U3*_Y{SIPhP!DH|<&7k)L@X*{s$z&R6;8WiKFia-g-++uh=d zSj)tX055Avv_aI*3s;7a`Y()qP7jAQMDu~tMBtA9+YaYK-l1&2pd6gwgd%J2qnoY7 z061i+i^k*3UHJo&@2thh>gye;Tr=hnGWNp*(RECkp2@OOKePDdiDrvPMa9R0CkDrB zxf|!iE5oW?n@xkiBOm(=Z#H)=Xp=O88dJk5&I$6m^WZWg?w!rh((#(xY4aHQ$n~Qz z?w*%f&V7#1_$u^9tCK4QRQIK<0`D>ML0~!y0Nw`!HZNepBn9=Ik~X8VS#@f;CaiH{ z5<`v`9ErFMe8*UmH^22~bm%;o0WY*06hSp2M5u!Ip?mDROR<>y1m27kp1QKgbMap{ zr;`2E-pcB?uRJ_K)0TgTIHdVV(GPqcN1Ex#h@2L0@8PsVlh*iK?{cb#FALkiPLJKD zC`c7H^^RFH+4`qQDcuc{T7MG1gV!@gE3Zyw<1-B89E{%08ptGUnIfDvtiL|&5)a&5 z=s!&R3q({EEl>{+rRDLH)u7ffj2DWGdRKDzB{_wvy02>V%XH!8BSnz>NNQ-iS$EU_ zO4|DLGM$;sTRcF|GQeGivX!Kc$8UeGy8*qMKdA0#TjK#`l7aZbB0)w}-@}Rl>}rv= z0<qK5eT#S?ftMJMVuLpx7j ziPz3-aOGqyjR41sti-}%oBlCo>H&p_C*XjXH z*%(kl3}Pldqr}VGNygHc8YKo2h%U`>jEo@~xS9oX<<8<8Z4VcXLMEO(rkX=_za5_X zN-u+6(hrv7&Un=#efR^!;HuXhT#e7P1Dh#C2Fn|~8}uq4>7&M!Nj_KMV8m!8xvu3H za1d4#bHsnlX5pegVCk(QFZStU=|sl_*NAn+A6fvNVd60GN#ZCR@Yy6A_ zqO*R|x*xA+!8hSq*zCg;TZ0iBtv%ZxnQ`W#oz8vj8Z^{x&hIrw@n#ttwwUZFdVlUF z0gp%NBu|j^a$5JRyj18oU+({-O&GIGT0+=o&ldc><(+$(K{1AZ8S+CO0_uPlKppVh z^*wNI958+ebPQW2MPQX*@C7>OYd#Bd=VwJ)#GRbd5H&c37@{09sy%=w6>AWR#I8ip z830;OW8!Hj`p@VKVXQRH`!={M=borz+pW-8r2jCk#Vgb`i66QURI8nVL@JdEb(L2l zE9k~ON(>UtyF7~ag~89jQQplj;T2qREAk@^pI!s@UoHHXRPZrbV*h-SmOJF~hL&_S zc5}7PoYWr_q0nD6dj7mx+t+{Af-m(>2Qnv<1o$P${|Pu2r%Zce9|V$@)bpgV-RxGW{3#IbfHMt>v{ko^gJP7mW2rUVYVfUGE}z+nnYI zLq))h0MOnfrUs*35YP_fkCCWVsmV7XGVUT|E!JVsRvuNvl<`%gp z4huHDSbql}i(cg`02l51~>p*99l0f*Yn!P4bcySjIu)7%5RpCO?;Rm8BbF%wB! z5s$Nr`i1#rbWd;jBMrcsE#QeSQMkw-13e}-n0v?$QkHjshT6lL!v4}J-@rT_ozeEt zcbqL^4hbg*q4se3#wdMGwh_=ByCJifH(>jx%URCFA?19$B(UmhIMvVBQ z-$CHgfWqp82#?`Wh`z+oD5`@jHl*2SC0(A2DH~M__|ni+o4aD0vw=t-_q14Lq=IL?7 z6u1O$S_*^wQm-u3#|ah*M9Xmg+$>6GLTY%6Yzc=)n?%qqIOHDC1uyWeBrs)dm8l?m zbb)lraKYYD;;6_hJx-neD1kW)XYhXZuNt#U$&SH7E^MEOjbtp9bTdXET`K__2@+p} z&s3oB(=Qycrf{a|x|%K%(K8Rj=*xg~Ee2Zn$~Up68<3l=8Ok5<yZ5MS(?FYRnO2%ST`( z-v;D2boY!uuxmH=z6YJWv8qFWQ5pf5t+%3DV@|!oDa>MVE@F{>Cm0RD9ioVBEV>7D z5tWeGvtb;KhSf?(>(JtPUpMy%rb}JE8Fbv){Tls%!z#(S9#N;z-eY_T3^IRA?tg(^ z^%dm2la_dm{2Gf5RrN|*^}hiK%`^1$?)c9%bj2LFKR09o3y(*J@@*%?{;0gkdbEL69@U#aa+e8#L&LuXxQxuzA%j2Hoe`_%oBIDfBWwNMFHTvZW zf3>J#>3F`$M_kFJXsZ#Ons_%tT>I0M+~LkgD#}0AKQ4iI5jDxo*oV_9=Z$nhBn6jn z@riv%pLfB!XCJoY&=;*`cPHMKe7nut11(CD7*14-T+5-B_fIMh_S`kr;-pxxV_)Xl zcT4;|E?H{-Eq$Ba%~PFSv@g{h*`VNG;#y^YpYf3(O>}f(hULJ*(8uttPLwpw0dTyi zi8A~h)My00wu14-;e%Q62NL$qQ|#X?;j&uM0|pZTD*PK<12*4?v3}vffd!dF{e$y( zFaQ|?qOj3|{4Du}F0?*7iGLG<{@LHH1^@4!+V?MwP7Rw442@DgobuG3j!1dB!LJr_YhPzIe8lS~R+U(y%;9N?#r48vdAAAx2d#Uuq zF~m5(>Bp(3+wKhmr%v{$Lz{LU+nen<*3RS!Prf&q`8$%cNWAZ}vn$b&auLREzIO`~ zzeWgN5}MZ6D6kjdN?~w3Kk*TljIFH$b992J)#}8@#%gX#NRbN$ zqsKn0IrLXI1BeetgWIZ0cn;3<8#=X`QLzS>pb3MqWI3pN#>ws*aU;Po5nu-lfE}3d zpuvl$Z^Mt(5Ak{%D>C0{X4ND242WM!^qcDp1x~*r9w!3*U{sQ!WVKf*3n42bXQh=8 z*ALsCIJ%~YVvM|;9q9QsBmV_>^htbs(9`m~cJl@Q^IFiy-Q7%q%`?)*g$R8tH?szH zuEfGC^sztCo*7ngTGGrD+YnkaLAiB#;&CMRJ+ZJ)$weo;A-M+B*v7jg@4~EzODF$C z^+=Gp6y?t+rAfIs8zf7H;!Z7C-uZX|?AtHNepvrzje7rq#Kp~ms zcb}-=M|rU+kBhOB&akSkU4!`w8^@vFNlAISK_l?D6T`&23jHLrDpu9g5x0Q3n1HUG ze=bB6@{qo-jZL#zr{L3KV+g|gkxrv#FNn?Q(&Ij4vl&R2U!zk>DQ zKy2L3vuTOBR%*L8aKwrLrS;yP+~;bophV-Ap7O-awOUEHq()ue@CZ>>LezL}fd%wX z3RgX;gz~bF;HHJ)9u9}MOFw@9CnZ{t8{^-t>-DjS$}M5fgsVl#<4~`t8oKG;{FCV9 zXr2ne-h3=19W2qdQ_#Zd+SO|bI4nA1C{A^5m?>j!hWlU-*ZHsmA|whU10LFM+irP= zywhrqJ&!(Xa$t8MUZ^E=7^qoEfm$=p(17sX%}lT}&{aQ*m#K^W!wSnwW=`qx=E4)v zkj)p(JDg6~{aod)iUUc+Uo7?7@%WW);JQ2rY9>wj0}aFt>-=2HIgqbT9h0}{v65{OqJO*Ppd^E0Zh3!xRwFFdEWCO4;v&+EVqS4w zJ-RnvXU>)m1%#N*{PT}E`?;h{AP`avB_UDs-`vdtZ+RUs#N`Zud*ovaE~=yBO-!P9 z`bH7xuQC3XX(D4Nnl4k>B&VA0r>8JesGMOaUg~1KjBErMlTc*X;)ICs4mPCaPTzru zqB`toLUwHUWY<+~$E5wt>9*11kSNW~7cZx{+Xa6uq2d6q3{JH~;ZxX&bUKuo&K+}T zI3(AlDY%I_fjvn++s=zZSQmbk2yLia3b2|Uu24e414U^LENJ#Zbf$(b`1wCBQH85Q ziyKP6qzb>adH30Az`1lvfjnk1CZc{C-1O6lI4YA@>U^#zPY@9jp8AB0d8fV{7l@uL z$0^2U`Y(#|tPcY`WPbC%+iESejqfpI%AT}0T1AjeqG)^-;`YY=eY+q^`!C0DJ>Plx z;$3OY_ls=1M6CWo2a4|jJrP83<)$LJwby6;%E_9bq@ zXa`4Q7O-f zKhbwkE5SXc<$p0L@TOx#_s#yXvNe`Nv9-m|7R7|jQ`+4;gMMmG)#Fw#bQ5S zwLwP`4ba6-5M443IOpJ9zr7c1#Nv`nuLEz}yev6*ZHYyv0@Yr^KbTU+wAy%=KR*%U zrYhv7dC~!`5WTv@D zT}C9yN423MD6*pB9T%0*xM|4F)a8_%Iy*HE3rf{^$3eDjP|sIe7|RG zJf~C?Z(X>^I5v^3apHQjdDac2o$KP8h87{4pQZ|k*JG4uw^+c{6q4>LPCq$*=h{<~ zR&y>Y1fJP1q?@VlG%1}qW-gtx7pVREE1sm)^&vKp57TtfDL)imGvhBKi{mzd2Q7lR zA>LP(a5I6u*zo`O$3|u(bq?8i!Rfp|@2FPyR8a4|F>bb(`kbQAUZcT;_X1Tu;1Jm_ zME&LE6kY%?V1jT0l6^R6N!Gu9cF?c!Kh%>6botG!sKsyfCnMHqM||J_ukhffE3w;*==x+Yq?&kS=?bjR8Ml-jqvc9MARnFAnQQ4 zuz#1fkm+xBTJ^zjMG~`b=*F{+TuZs-{BBYuOAK!{3CJo&QL&8qH5{5ZT{Z9`{DfZd z(O27VJWb(|bq}Rs#=dgVg`y?JrZ2T5H90uga17K8PkB$zR@>zq(ThTx>0#WfH7{6{ zF4E&%LBf^OtE=i?nbeH7W+*xHbN|uo@ExR6fh(;_>6t$5&Hvhfqs#(SmdrqXVb2jO z`ULJC#n4Wxs=j|9B*T57zxQ5?3WW<%#A3GM5W~cBpltZBw}^=zmu6qZr~TunXB8gy zk?zL+2L8>!#U$Ycap1X6dJ0?J6vRhz%4NhUcU$07{++WaVyYB6q)4s=QrV>Z&hFbf zZnBa4QB%c|cmeAM^97yt6XMcoUXAr9As@{LKmA0Aa8s2J*yDUaH{we~!7me$N91f}g&u(eL@!^Z1 zYwi4=ObbwP<3YPe!nSkb?vwL<_@^c&C$t~Qg>M>jlKQ3q)YzeKsNmOuiW@TkC^M4~v+vdmh=)$|ll`pFP~vN7CNYWd2OToI zp4poWkQ4h5o0xw?XW91G;?&7u%Ri9~5isW@6D#?OM494nzbxO1u!^u?-qUDmF*%0J z*}Q|8mOT*vyn}`u@;k+Ist+uJ#7M@Y@7&CB!n^3epr)Cj=SeSMbY3Nzdr+hmW8c{9 zRH*&SM^qaCy{`Ppl@P?9-?S(`Elnfh!NNh&Y-6xA>Eb==&5!Ut4g&Jv3miZK37Dr? zqUk@u?9EJb4$&S*PpHTE3}S=H8+wa9NO5y`Bh^Ak%A0k_J{JhrQmdHw4SA-F zcIlpq$%%ZPSU*EbXfLbaQ2WRhui*^@Zg1bB_3tpj@yuz~e!?}gU;G640l^jIrvgNf zvs43NZ_!VFe=>t4ky=+Q1I4A_;G4$QEg@HhK2Q%mfy)Cq)bZj`34sv*XhO#4GZ%@G zJBmGSo0uxzUg)wCmND%;7t+-FPHS*oumee%>|=6s^`c>H{KQ1}FEn zUoMcm&V4T609{s`VRC1#i@EEV?ZkJVjhqbbtqT%2*1=&Fw~t2{PAZfh^4Z})%U$o1 z^$(UU{7e*@3cLgQ?Hvh?RblB-(&Z~MWD5}?*{l?r&Ec3I92y-XL2Run>+r7|<+eV5 zFJ0X#HQgFSxCvJ2mLI6*MonYN`r_Yl=^UMrTZun%^ zF}62ba^f_bhhH&())&8Rq5XFs7G?5Jv$mHtarI#zr*!qqA)9Ld^q+yae?si#(rgL z3X!6;#hxKYPvnq|zUJldIx_lC$E3o`)=3w@G~)tER!N2oMM9dnJX2qEBB!(_^~lPO zb*B_JOB_@oxH})1&R!BkjGx!An=#qFTj)Z=sW#|#IBd!*-9(R-<>#Iun3Ih%Gf|Q+ zG)xTx&pGeAB;Y$+E{X{6{cdg2w{R9}U)i7YV$O^*URAu`GEAQ3XTE+elRb2&5}dF}7M z=KS6%(~?l;@~8KlDR^!%XVo9Vnpb(k@Yu|3(Jh-(gfqpaMpqqj)y^oc^~T_3R-~aq z)jwHGlZ#K3LpGT)YQ5`0=b(^HSNcAF?a&EMBjO=;Ss&q*mMo`|{T$%>Wr_HDpKb@8 zT6PpGi@QLav;A-N-)$mW6vvEcE(-3;2V|3Y)w+cXb_)vKO{K4l=3Jx|5Ag($&PNm= zNe33j)LNAeQ*(PamJ}Uk#tm^ek*X9$P4^uT1E->M z0|XY1;CO&y+g3b65L2|R5R_-y-oy`UzUr!`tM-)H#00ds$0+>MTD3W@gySlP%(&s^ zi7oiQS)nl_dr#2Op%dNKqDb#_(BRkF^dDhNxyCdgG>}sx_p1XsmTAQ}TfDzM zM{qvCrNy&WCLDC{qa;6=mnQw{M*y?qYVh;dCESsTNMlHyzX`}-U>^+a{!6tIS2G^C zZJg+W5wb~{9UcWQ8w`V~sDZ*tEAp4m6Oazhk15MhD>V33HDMk55i?i(FpBahCpGQ- zr{oipQ(uRPd5<-h(cV?xeB_by# zu`suE(xs`#1gFSAI>oc9=iW^F)7Ul{Bgt<2itut!a%;&Ce11U<0*>ady5?K`&Tg~j zd6$w?lc`DTXXjV=V;m^J&A9_TVGD1M%=yGyiL0F~x4+i~#s`Rg0@Z&qdB^bjw~ATy zbYCSkJZJ|6+z;Z}=@xMt`g+dN+$f2Gf+z_A7(QiR#NRXJB!gDV)&$=bA!95T`d7)a zh{;9bRl-uu8GPi2@_5f-Mp%3OO9_%xE15*mM-$J`7xgvP@6eVZ&CrN$=Bizn^=E$# zIiP6j=kqO%<6bKSo<&7?rtX>OdGB&*$v*CJuk`SF7Ckjmq$|rA9u-C#?M?mhhWNF# zIe$h7^V#{Q?gy-{w`36?&j$Nht_RML z{&qJ@PxGcQ&DFmxJmL6of!-`pc7#Gna*lQ7lKsOiu{Xt`ri=ZQPZe&U#(E0JJ!j76 zQl{Isy`X?unjfqWwwoR9DdIl}ds#i5*jUf|wv`qPu3#v0P_V&hog1ghmMm`DniXce}hEXD)02`Gz(j#{2NX!kn@UXtsrE zt~dj-(4QAxlake--C(fdabPhk`GxeZA=56w`U~m%%+5fwvTouHM6a${eBKZ0Wln=~ zgHN|!&&aNGtvZ|EY35oVeK>sspqI6r7eD{R_gHcs=8oPHS`?3t#nUxqax4WWx>mJ_ zRIQ%EUw{8gI@Dp2!Y-3lKi`G=(W&)fEu>M~@Nc;Af)0vj*(nj<|rYJpPIziX^7J%;#nzDq(vbaZ53=(7h_TGpSjw9 z$8cq=BH^%S{e-`r8a95v^y9)}N@wp|IdS!Di~(0_j;(kI!KYglDmZ8nGFVRjPESqO z|5vEhUvK047cN?m@tXWRHI}&E{Djy`6{b@1)S#6R{~H?Zk$R6s#Wq`SLe)1h>3as!)|j!k#owNcM`-}8Up``i!r zez{+Gc-Ee4tvSXV`5R--718cWLer6xrQ&b+TAY#+2?|q&;iA}Q+Sc7MjDLpCOW$HO zT&^sM|L#B>k9W@|wZLplOv40AWe!Dw97m|ge6^*X)3ZutE^V4^h10pT% zeL)58D-to=()Y%OW44;IdT{k24IEBOjj5d!q24QwUQvQk1Vgsey_xxbgaRALx5J!XF~`XG%1)-~P^1G~ z|5*5(eEA`F+BYUtOc$-@J%=@hN2n9`V=^b!6%UEg8|HXD{Olh`9G#gIy|P~_tI=Ad z4F1rICL7SRO+o?qoC6ucUX<}Vl0Kjg!PQ=eszXnIH;WUT&@RWFl*gebWO9A_Dt)bf zQYL+ZBJDkKZHG>{*-$LS@TVeq2PO(#aa(jjO5f`5H8J+g99}y?f`ysEqWIkDje980=y8?hTPDfuRe}!vFcNK0LaJ?AyA6MFF`&wH>{Jh#r;R(B ztK67tGs#twvpYtPO~J-SI-RIBR{YOju`D)xs-;q-M7_hPMoId`{25-grb?B=VLvzD z?8Qk_ouAJ2$5_)XBX(!C7#t#dRg!P9&1-Q}V`9f#X#wam}m@s^m{I$ z;$7q5WeIt{(^Z2u{HS2EiLtVbVwR&aRW-AZ`WAZL5jNXODp?emB49@*%jjm$J zRcVP#7#3WKOy}!y&ZB1X1C<0O8=nwTB=qY~@y|cX4z`|gfHX20g{(eG;YZx^eT^pM zzEhxvS90(t^Ia#ihlwn?IpK5xsLKW<`1WK%Ll5F4MSyAye8Nsm%(UkK+IU;=>u_Xw z*$cc*$H^`Zb1H=32)ZxogMrx@G$2L3A5)wx7Z6CS)+UuR(8>?uTG;BS(;Ao3Yl}L< zE-WdP;f|4K8BV>AwG3J1r>1z+c%?f$OIZ22M#aRuWo6^w@GQr*)Vgn)X}O~d!ok^_ z_&nOogU7@zE!1i$g5mmItd&2XnMJgBHg2r`EJSR@d6j=N*{e`z@rkwI#);!;shINt zk{L&6diZNa4UF#b{SWXR<(5P65!N7%3xYKr4l)+_H-n-mI?6dL0K{&AmKQX8!ruxV zma~aX9%KDk6F7^n?f-((N2C-h#&Dg->GTf5W{tc|?2_pWhm*+j{!LX9h)K{s&)&6#irJ{e-26Q0wb%uVyIs5JmO-jUMaZJ$pfhI(Jkzj2?ftH-zNmviveYR;VUr zH#WGs$S=n!H7s8+*d!XjzNn7()MP~_PuMRgyzs2Bx(ISYyOB;nKG2`LacJ~R;> zj;W(>8XDW#c>_e5qGz&ECRnjPBIJOzkhZA1yozFJXuDm&6b06Xa92W#tUR(&z9p|K zY)SD>d(Y;t3R`-R2X;O|27YL--&~!{)w(6T-D)4hvEgt&$N$ZAaUclR0nUm7Qk+}9 zo0y-;Jw2qPJ{^W>m^`CBhwLr9%TnpBN_9->>(YwG6Wo2)_~XMI>a5y_ciGJMMgW^3 zTY@-5lQX(KN)dOL1BO15^jo zGZY)(J@W>Ey2v6IUdh5tt-5L_n!c$=z30o`_o7&sF3Kr(Q=JPH zZ$ekSCQCC>*Y5>R#-H;2Iis9kVPq=Mz&?FHhm2_@)sK%u4(=71sl1NqzH=V}zN-xZ{?x9%~F_)JPj@wpHfMI z$HOGDGn=o$Q&)F-p6nb$zt22Xr`w}z+=7>-v^!*kP1U-g*By*tKa*>+Rk6exjR|#rmCptjgurPqHl4eua91E0YQ{x>AOvY9W=3t)VC z#l~{HF6GjFgNzXetsO5BsyK5d+L;vK#cB5u8+-C`2`;fJKXSd+vgeY#Ic&1D_-U{m)E`D}W1*(J;`&#kaWt0|R!y4NeoZh&bwDVgW^xM>rptuEuREBNP? z(y&GPc$Qyvb0=o!e$WI^DpB#-w}vWGvT+w!Kvlhp=a7tP-n7I_%4x)G3&)bN7YGFC z)=|bN5;x{ox zQ8YW|0o}2xruLW4VIIZ%ve_N;Bvqp>VeL?^c5EhxA7Uf{ATBb-<)TQJd96e<`H)8` z97?7^np<~+4Y{u57RX4v0gY{b;7hBZH&K(}xZ!-DRo9nblGAfAo7Zu+U=ng0XZ88B z>h86@33+{PuU3}Yke+l_Mp@&?4fKNMeboj3S>+k$@yJ|e#gCwwkvg#ho^1R|D|d@2 zWqx|N1$~>H=cTCrEY2qZ$2TF?Qe!iSyWOy<27@gFDVS)Ec;xnBvdv0h2tKb;kAO_} z{d)`yzGZt~uz^Z8S46JYhF>C@Bagdss|1=O!1B{sx_S z>3MZAUqm?qy1>r3vIDPx!59<6Opm#isE@bKJ@7Fqsp}FmY)d5 zr_c?*+XR2&)BcWikX%JN6T`3b1Kr;4+yF>h6H+)dtEKQ)+otC6Q@VzpQuuyFmxfol zKTGV7YJadAGi>3AbCJDUE-}MQ%KziFpnFat3+PWK8*xu^5bO4ETZKyc3P1IN%;EeF zyv8oIeTJl`l8^7g<;0QRv*9(rq0pHB#sfsm#A9t$PhBH>-HtV_q>cke6ARCxsG@VM zVABo0ct^-MwIhk}SbqxtVP$gr*0}|SD+QteHkQ3+sl@$4^!BQvfoZiW$7NdbmYN%nup} z2?e5=rFblA30k>?$D)b>sKfp7WL(Ajvx6&IYa5SPdK`2+Xlhp}J*PGFuataU zak?K!O|pD@YNGLU*mfU;B>d1L?IO>1c%1kqrORI&s)!EO9C(5i!w(#S=hW8Qhmgo` zt(6=b=NPVBkDMW+y;n4*8oRu7xI}NZ@=5jjUYnYHcV16SYI{4#v~SQ2L#i8h5_TB5 zZr5=5B*OF1QNPcWP1ky*v-Z#)Xs(FbXvKA$d5JIr>MNd8*^M-{np$*}ip;2L3>AqG zl_IsVC*_c_JEqlfjIqd`D|A*xl7Y9DEKK`$ts1P)vY6@?O_JdPD`2ci1&X}z6;Cf7z`;7F41n@V z#UzizQtc_43NGRAtvKo1_6gs8OYvGM_*V3wxvV#II-h&}^y>Sa;0%myPU>pCq| zG6O7EacB5G!xP^y-TqJ)8 z(s1~?%H7+Z88q4`c3M>1=!{p$Ni@thDGwQTD@6RLg$=)LgCGWkPYQ!v-JURc@&pny z)mzZ{4arA@z}@YNmTlj6tr&*~La5#2`X3~krcD6F<3KEa4K9R^9Pgk*$;cnk~6lDx55lBy{d;we?+bE~PL{GtJsEZdE5tH`*kLZ`dJ zb&zp&1=^~MR8X*AJ;U5g7aJQa$GOKaEJn(Iv zrw0b1iKaUB>PBk2<}?A>6#KdfZpB<<`h+ZW~|l6w1*-6+i-jK#D@ zt0CP#gd~#~$Vic!09eQPMkSHV4v;YWNl zXkIfoVc^jFhX!7vd5Pl;8&IZZT6S!F1^7NAI(FA(o!r9wh#@@Z#;$yejm041M{!0F zRSgnA+vO&L?qNS;70Kb}#~{ei-#=TQ>QVBmprmDle4_6e#V-5WlgBiOhc z%?cSdc;hTZ$eu{OMz5PzHD3oh4D#=Zs?+x;5k%3kb-T%{92df}QkA=c-KsxL%*U@$}3af1=>I&&F;DLmk8BSc(?DnXPc2?xI6@#+r8$>GAU& zspx&NcXxGI7iIglQG%C!d6{(jCxa$-#b80&nJ3)`bJs$4g)K`-w%bG$l-x%Z&9)t6 zV;VwyIwXuP3mm@I-cHvKPXaWt1=VB%7kE)ZR4yTH9;!1$l@+4F`X_lK1dtr`Jz;Vg zrF|7wG&{PTilVNBwgx@J`4IK5obqamuYoL2_7a>l+=iER4eF;pwUxgDs+RJ}d6A=7 zJ!=l_f$JovdmR(aKr# zVE#u>->cO7k}`AG4vhAy&A}wrqvJzi?*tva(pHo_XDd0Fj8%w$F4w8IZoaMaIHP`Na=eHQH@=N+G~pe9@e6j|XbwsW14-!yY z5ay1HNOJ}`xT8__eNikx-AE$SQ6jpX$OtbCgZQ#q_`RTaqIg&!&5s*Rv^nLyItDQo zi~&L1LRNfnF5fTPK1L@!U5sOwHSytvlRl&?1hu$_``O6AStU!BL!A@<-cE)O=shjLm0Yya>A}fAuY~)f5o8A_$VpUPaz1@qjFSWRvT!5{sF=IWhU0S|(>AJ+GS7OcY4WMjy*zmG^*ZxG_uHjGv&kE^WI+X^j1x9nk@t{teOurc)S)l_@vJkqG-ufG?iRBKk!-D z1YC^?CQF}rWN7X!MkdNkR(ZdY@DXYqJOjHn)aGrS6w#w|k`sO(bld;x@~%_PZO1kf zJHN@$J6uM$vx8St;BZ6*4t-4_|7;#DWS;tAy_@u#36*b^8Nm+WU+^p(ji(a~K)^SK z?=#54QQ`gTl_-ZTTk^?54Us|}nLJ2L*!K|@0J(F#4$Und?qI6mGCGI@%4rJ_S{nXG z?$bNm_^tsWCa90HO_tSf$rKkmer(vGQ>v7ekWpM9`UDsRlqPJOfS}nb$|VU(PL%zN z_hZ~~D|Pj$;r)c-lt%g~KgKuoa&q%0<=H&FURpb8j(#VJ-XLDa$Y8CgiRRaqJ?-1a zv^vk*+6e>eXC7(|y-56Cevr4FdA1qd%)WjtV!5RrgJF^Q4U z?xfmtuNY>0InG=d#5ud1HkJTr_MyZ^of`at{<`cSOT=7V#HD=m>utaRi!^tj>SCaI zkNrrl0oo^{E1xh4Yw%OJNj0zM6&JQbyhb_6BchA+4&XpbfzDdA`FdnYY6?zQwxe50 zf$Ax-+u+Q~~q!&de4mlh^$x+jXUu&-(MZqoP-s;nxsLZq7 z8vlpf9ZlAx>kf6W=9jU;S9>`5E0BCnDF2Ri;S`fAMT6K9INbAqXVR2=458Ae$*+~Q zKF)fHOL!_YYK&|zK~5|W)y?{c`U-PBiA9#f|q@xhT#2SKr2v&suWa4ipi^ z9vD;}h>z~BMD`iV@b%v8=K#=UU#pH8t+f|C=#gzG)WffWHjApNb&5qmTyk^E&0G0+ny$3w|4*|lPeK|n<)eAgZ<+iy-O|NPv z3UWB*80wjbt>>orWiSiy-P}2xXdUZYup2hbsrzbb*?4*}0s}w$;3Ui6M0fx7Pv=V@ zDv>?VApAwf4J|-xDka*%R#ZDx8zGxsFBvO(scJ47a@sf0Z3(YxOle$Gyj0W~r*(xIB5yFG*a*NlBcaFAk2#gmT=EB=1= zy`3x{W6faaYM1g4KXvft=jII{xpMKuRucNxvB)?Wdz^%t2{fE_*ZKc8fv`et`~N|p z;<_~Jzt9(^ibA$#;-aQ5vW%3VaBMSv_j0@nm>HW^MNVz!^Bf_7HDP79I&v9R!tebX zp?@vXzYzcb^|cfc^a9jEqS3z={I>L57k%=bS5F*%e4^)R43)iULKOjg2-uWu_z~Qq zA}=q`O7MKMT-_*&Mh0mN+o+mt&224B&4I3pT&%BIQJ-J`7N}dOrvxR;78)aG@0GS) zZ)9@roe!iu+Ja|VRt@;H(OGumz`OxFCUT+hfT%m-76&Pi&PY=#zOKwH`};9J|7*Qp z!u^siIx6ZZ$xF2N>VJtHfK090NW}(z_~#8jW&M}I_5jl4A9d`=+R4AHY05`52g!tX zex1Dh{F;``j}SrCzcg|R4+w);q;8V!f4OI;T*NQ{TP*O@l~dX2_roZ?0c}`>CUO7r z5C5)izoy`eIWVt`tkM4K#DI|*Y;P<+GWU_O-fV>Vt$Yc<=pQeGD*izgI-8=3N(-mX zZ7(Jj(ciCc_5`S#;-!`Nec{(blCdwXf13;eq*0LIL2LULMj&tK%>PN* zk|TeK-R~zgR{@lrI1MY={J!#gg8#~sKp^MUo51_}$G@|e(i#=w-`OoAE$!kj^MA5t zeoyXQ-ieNjBLOfCM==$k8T{uQDT9Dn7~PL9_?wD5(C2DCfY~jzsQ6nIxyRCE0?_B-^*yk060p2ALfwX7k-)1d#FyLBk!dD zUe852z=+ommK*$@_%CJ^V*(2?KCcWcU6_I11pD8N+6f0(Cj{3Z_=jZv8<*=DffeJe z!p{6lQ-4z5nd(zWRmPVzj~71o_x;VEfJe$#NdDyoe$6%q;32_!RHoiU)wP}v-W0rxuq;u{R}0dT zzD~ZrFinhF=EMai$3Ots2W(>*W!!(8Xo@6SlFN@z!$}@{UHH>EMO_jK%s5UoYfMx0+}>lrXrnIRPRp4cVFiFhJrjh)wgt|fPn6VE}VPxja&^%a?OK`^+k zf5V`#jJ6VV+-|_OB{@Gj6JO($^fjg1=`oS1x0t(R$!lTAg$jF5t$Y*BBPD4tni%DzN z+T6n89EXI<%p}i|H<4>;Y3X{Xf46D1#LyYZl<16F<~8K10LehY`Z0^-DG*ji1H0_L z$beyfza{zcy@7m;zR{s_F7g!meIfVF(heu-u!=6Di2e-T9#BH2bJaz@6sph-Z$6+O zutuv_He5&wZz^hOoy8K9Fh7V>N`D0|NMzXtnhWHb%CfSSVXvcT?jjk)0OTly-BHLG zBMcibQFf`le81(y_c5xN|D9YMzZbl*MsCX1G@ftv&zD`9Sy)V#;~nTc#cE0AgjiTw z1b_uki9xSuG1d^ftTUTD4Vr<$AR0nIibF)r{^Y{9qwc0q>+Nmdq zhfTCRWXXWx@in+N_pI_E^&F8coFcJ9P41Gn#N zd-YMB0{h+5EQ2KV5`dm)-5W5isZRy3opVS*Mxn_-#FXa*sVAD8NZZd-FVt$AX5ldN zczMhB2OdCLHH5cP9O$M*t8Sqcg?urW+!)j#&u?kbiYh4W^u<1m2- zz5&h*E$?b26f(o;^K^6WL1%gV7HNqU36LVvW|H*=?G6q~SW=m-WQ`aYRFB}3W>IzY{WKo%-MY#=EC7-EMJ zjX!@Iw$8NXFe94kq=;Hun0B{5b}Sg= znwtKC_Gtp3PzOoj*ijY)Fa)wdGmNQT2C@uu!y7y5uZB`pxA5%*km4l%05_UGFN3H2;CRJ_{oTvw!3W0~>RI@Q5}Xs9^l`zPC_tF&|eW zA0SWyW&qMR0{><#`h=Y#95NoNTAnl4N~h|nU-EhjRSvMf;Pr3xNZNon=eyU>5D8Ed z7Y)i8Bu9SC-+eU*{2Gl4FKQR+F&;?~eeZQ>Y|OxgR8AUTrR~xB%z**ws8PkzGH-uQ zgF)FhsQuAiqMB#<>iVQXU*J{m`S%AU2PCnH=6@E}30e165_O^_1Y;Ax(Pru25sMACyd>y{sTTzPHji39OkP zFS{W2?RK?i0?A!f@1Y^RVy`%E?WTAkGvzdkxt#`ZvFWI~n|w=YUdtW3m?_8~S};cg z*h!b}A9}O=CL|R18TfpM((*YT{FJ!n08|Lr5%vdMZBTs7a5p$zc)u|W{l(M#Mbe3C zC*1wK4(rt54ey{#%@u!rySRXfY#w<|A__K>*m7Sw%STsZ``27wA*l%{JzI;X)aNBW z5Ek8ojJH7Fcz&su)%}-^uAqFgfvKUQ z-9WV*c_Wr#WWL0i__VE(k`v!(0o%f$fs93*hLt%0J0k|DBiDci+Koc{CSYrn2{9&J8I7e+mNI!r8Tat)kq#>(F>q$Nl z_XLTPWL})?+9cKK(; z-7vY4C?<`2oU4}bvZ{{#jAr}3q~@D%dzw$3x*Obn6zc?ng|#XSiawtnv4T|K{X~w} z;4Us#$FrOmu!T1M^p={vUrw z4hmEQOYVCFB#@O(p zK$gr1ms4o_%V7V{$2GGc^T;ocDSrNRTXC*R-z$2X&YLY?tw_4giAlZaYnQ$!(`bI4 zN_PwB?CaCM7u39#34}dtJn|djz%Hl4zD>=w(bz%Bi)%+MlXJYQgdKmI9w}s zug`g}KOTSS)U>@Tg;V~^FjSD+D%HYVtw->>T4BcJlx0nEP_(- zYT5KEG~|Jg4wo4X;<^4fRoa_kQs_}xporiV^3YuFOQcuV_0XMKGN6Zzj&_Ng>eFQw zN*jMSE$%Ud3Q^+=dCCD<3Gvp4QgB@twfr!z=Cc;u8VE<5oMR##SP0yEwojIHay&fm z$u_$rS>Z)L`BaHqmW|o>^)VKFa`T$ysk>Ut;*EE0C67d0-Ti<<}mPbyYEKNHjwMt8(JH zq4WztQ6^Zuj&7U}!5?-lTWNHEtYbcZAzrqP;vTU5VhxtAZCD$U_+&Awt|%IMU=?ZI zm@xHBa4`OO_EH2mQY3#W#ku@z6i~ms(Y!fF$QDimojHQmklG;&AuTrP+El-%D8?-UE|(HSS9(0Frr9tMUNH%tdq8D8!Bs9#@URYKcLgHqnrXlZV`8i4 z5g19-+`8hYu>7)B|BcAp%hRa8;Q3Rt=C!hoD0lyb2VzB5UqWy(3GzgqfCGnlzJuI~ z;aLVZHYJvS1^4+zhQ0B%E>3uzbpzW}(pKHD@6+YAQn@@G6%Tp6{k z(0(9@9_7O(vkev;m`*bhT!%k2xOctg9O{spezgc0z^**@zfG+#H>$q(X>fwa8I}dz zYMr*kOpzFt2g9UgRitHI3s0HRE}mwkd8B3iTD#nkxR7OiSGz$wArJlgUUozIVTrw6 zxCOw%o=u|od38NDipJy8i}NPjF11NZ;-V~K==uElRUs73MYO+m?(LJ`N&0m=2k_11p$JGej( zW|;I(8Jfu-$sNCuAz6%0N(9&QD>Q_N>j60^r@NGA*KQMz?Da z$z^8W3{IOmwj=KHXf=f|^v7=v%EtvQwP8U6vJs6dMQ5KY$CkR3pdSLzpPru4woc++ zC_8n>3i`U7qj%&T&H%5Qe(Gh$FPHdCO}E^50fAw$yoZB=HU$)V+U|EkwLf^C++N4j{0+JUo$h<6J#pNE;GbZ-JlGxFH0#tKGzsx#5rIMeG z0e+>1wDskimQwS}tlnj>u%Z4J)c;JpeI3vm<1ZSfycVYCZRdKrwP^dk-(jkBdHVx@ zC5zK!?MvoRrS_POo;*;%Zw`W9E9gDkS1#8q6)WJ937ba)5-ar;AU<;BlM*&P($NI0 zG|yH|ctqP`JR5z__#oSdWTKL*KR3FTwZTQsovXS^a1=2l4we96l%K@RURwZT!za(0 zbaySwp7%M(T9<(oB=R*N70P<9yWGUt2t?omCL?UH1@G8=M>kV*Z2}Ma7X>VLowOad9(0(oti-7g?qq~^boJyjbaMWg;j_4o`E4# z-Yf5Ojl@XMZYxtJ^0a|S=eaxu%;1yU(7eEvh(b_S!`-hLFj(AE*>kVj3-AEC-S}X@ zYDsd7aYP@?0d{dQTs-l_L~UPgh5ySaYIqGj+Ry}B0>lqkiub9AUj0dszXtQN;iy9+ z&F<0LEC38yM`Alx;?h6E-azR$r+W|Vw&tkpHhq9Ez+FjX=o|w9NQUmY(_T7`Zdu=v zHm~+p$C_}L*7Vqy&grK>EY+h(Ur_a}cK)HY9#2-L#X<$M)%Y{r?1+Pnq9*y13Bjhz zvpUP#Q#vUk&x6$K58GC*2Ad2Z{TU&U8y>RFb?x`Mo(|8@H4SqnX=e?)SwU=1Qpa)^ z%iRvmi3N6KiQRyWwZLH#QM+z&4ox(A#Y9WMCvc}Pzk{?_xjgH&2A!6(5zI#bX-$8V zma9)4;g*zf(`*2w7RlHJS~b%B{Em_?C*jz|&t{dty{bkSWKKTyr|Xlp(e4|SSav(} z7+n4KagDa$1RA;k3lQF5W!PPH8WrD#fiEnZYmPq~_qgM-vMIfvupFle$1?)&_NPES z%7|a9)yOMP_frX3s%CVAt`~cDrqjFDj{qn1VVj)GxhjLCehRj1CoDM{JACPXfJsmb6g2!A{cs{y9OJwQZuu_u z>Ur;rkagz|^#WI_nl*OD5#oLOLVE6N!8;eL#DMW*i|;uG)`O*Z4BX#1=iqwM=pB%} z^rgTm-yx>v%g_n0hWEv{Y+Iu3k(zWse%NeRIPL?udG+8=GY^d}B}LL>9N%y}D^q*C zN~dsfzLqh1yQ-3QcM{$fMA&nXE6mlC-dK29IdR~j(AnV?w#`?5wTSvV-=QxScTz@X zJVts|jO~V_9-A2Q$D~6vik-*=dk>ar^)py}Mvzlj7%6k*R5S8=tP^oOOPz@&A0lCT zz9W3iA_45HielD-KRZ=8-(-ykm*(o&cd>Jb#mBtCxnOIWGhb^s?=Lgl)(Zypk!G*O zGbg0uA|t_~F)+@viqOr9o)A0brf+*az1;L0FH*xkyC0!HUpnC=*;eFmJl*zw#XF%+ z$g1&J{Z7p*J=gisYA*iGiU!%Z9j^s}EhkXMTrd=U%3LJ@avw_=VAi!t+@-(UAsw;w2oy*-kOKR z4{Ob=8;pCG&0UI@H|lpy3X1jF)ve%j{c?M|-6#y!mCx45dOW-9Jr7!TWulPv;=QKP6wj|d;u{rO=^7&(-8x6Mvpzldb#6hlkSc5?axTgOZYZ{FTOOl3r0(F0u}rj| z!k=+#Cadwfkn!>fb7;+uEHZY@-2T=fBi<3X^K@ax1)^E&A!}31vaj?C7OVYl8K}ap zoBO223YpQtbn~u)q%A}t0vjK01lzNxKh#ywrfSyR!f zX}4$x{azMg`cR>+GiMODXO%w5;;3!hX5sGQCY5PtGmXNXs_Cr#rFT93_6K*WOkz!* zQ6+9m`sW+%3Sd1uEy^m_w<&--w9xT~IF|{^_Ul>m%Mk~yX>ccQf8?MB+`Bt!l)Gj) zaG2W-I%gy^_njjkWK0;|@p8p`*@{=fyLZxkpS^vesV}Op3eIdr+{y(&vW=44vK>?tOMwrVF9zb>NUA=Yp_Q%;{QYY-6&$krWt z6SJd1Me{*yAx&5g7N_Qt(c&Qg;@cF#d8 z7KWzoTl>~BSzY2W5a;*A8EuOdSt%*0dLy~4zIcANsSxY3Q@StG=Tk=W{Uz*>o`u;stkgyY6i+^2hZK)wx?Wo>z|*n?DW5wMF;;{M<-Gq~Og#uh%3_ zcqP1;RFJWFJE9WzC)QHAC#9CuOV{&#Z)Xp^LR{t^M@?C=G<%g4#$)1RMWvu&`Uc3;@oR#_sZu3i;2t^d>k3 z{$wCKsrk&Ak^72nMEF1bd5JYJkLUMI`+s6-f5UZ@WWd{olnjagw}YgCpo`(o%&*}1 zKfiMgsFi#PA?f)~Gx_V965vo?zTNppv)bR&LH;Kxk;0Ye_g8hyj_J)O%LjGs(T&~= zPI$@pC8gdt>N-A~?uIWZTb?U{bLe!A*J~pibI$YN5BeavV04HzPdFAUrUrkj$sX); zcE;mog)9vsA$}r_GIKAN)ZBgH@Y!g2=Oo|Is1n){4Y+#Nr8e^=v?mo(bhZ6Lme{?` zW7!ZDh3Rp_<7MR;Db5bb{4X%9s5ehhbFs$_crXX)^`)=kS`d+dc^@K{x{b-#EAGJ9F4SPhIz3_SW{&eX0ss zD-E!m0IyW+?ex5;OdZ|(;fxy)W(&qhq*>Zv5^n}TNr=sq~ea3qV`@_Qb|K4YF%yz{IS zA7nn`EdiGGFu{rK{Gv^OOehZNSsFsBJZ`?t++3a8soHYdke5uWdy-y~h&g$>phX<& z%wvK3l!I~AxNsS~LT3SHvv3~INy(R6vZGNX$^&oOOyr#x4b@hFt<(~-D?cairHs2_ zoDsq~W>Xzk1R|*`M_G$CVf8KI&tA#Ng}(M-hW>465)#?KkM{*me)!M&4$R`8VB&uq`u}oDepEZ{8`j-# zG02r3My3)XrxbuLt8D?-`Prjuj#|zJH1JPtzsYrTRyM~DL-uL^qbz1{|HO3}x_78P z+u0q-G+eX?B&Ul#(5A0Ir%&eB0Kicg$N9ZP2Ix@~mOxxO0xNRT;-1Y;=5!D7IP1y= zFdWp`F~8f363xNCQVcJnms2IST28{_F%VcqO&gw>X^^o@wi*;J9%QsM7D1cQ5gUu9 z-6WRZc&@h|#Vp4N1jH)cTtT0v?Ci8&7W1iIvx}*%S;s2|PELXvq<}NsF_By8#&Lf- z4hB?Y<>7R57`oUV&f5ivx28Xyan%2=H8wILxl=n|CR04Uc%ae`AeD8}kU?YCx)Zxx z9kp1SnsMT+_r^P2*s2E78{9+37pW#(fozv&(3i|PM&LC{QfJ+W5sPMP~|IKaM}* zIrk3gm5*9KVh4e&gWox$A_|-#;yf~Y8&dBx_u8W4i>Uh(wH;-myR6?szb;uf&=m5( z6>Aeb{m~Adm9Ga^eINE(x6iRWT60&17itgJ|L}O~tbb9|kq#VcAHl;6)T7|s`LHsYbja@qZf zi&q%i5i<==kx=1t9k=bFx~_|HTUeH-Px~@wEP$pUYT_#Dsr%!ngv$N#Bn6bN{68^O7c%CX#%~@^!B8wo( z!BEh31it*4#l@W%trFB+uO^Un`E^z2rctgHZbZ}(jKPI|px@0@+o=38;&?`;n+MKN zAJ`UpDe)pw)mFV~FNn>!t@Su_!*4|@bbgHc3W4xCT-xig^^RP!ZerRmY7&PvoI|R6 zBoqqV_7P|OQHjo9liUVhrhq4RHX-htboK|t6yA5ps7Qldvuy9*2*$1)Ah5f?1vZ3G zYy@4OC5tH~mWqtT4L0j9DHY9%m%|V}%IVFR0#|x~_K&VgN;Xt&a6@FwLpF!a5hbTd zuzGC4f-(Jujq6quw9Ci$kMHW&5-X#Pp6+Xt*BTuA>HIngS)u)$>f6(0K!Uq(!&fu9(C6u;ls)ZDF(oaT_0a!C*jGTsl`UOQ2th&ufdC0ka3?r4?ry=I;Eh}3 zt|7tQ-Q8V6aDq3k3EpUMhyOB@dGp?9|5|tTYP;R%+*7A&SJl2%Ue$A^htf%eh$4tU zh2OOM>n+>h@5g9-*WTwNPx(G)zg)BNAv;}$FPd<-nNSvaqA0SLO$^8Wx$qL%Z?=+l z7E7d3KP!alvwGoYxaTA~Zc1(_0h5QW9CK?2_RdXh+V<{ETN@XZEi5%PEDuX5i^DA} zYxo(|imN<`~fQmAA2B5_sBflBhHF$KN6q_R6~F&{5pCO~zU zW!QG_m$_qdDK@1$PR?7_VRrI(yHC%$97#ZJ(Uc2cP$T%-tiLu>ao~<7<LybE-=zYZ*F2*oDQNbnlGYF!ZiE8sGqDhTa+ibvElYF za_PB>vRDnW8s61!Ww`3*_ z7;=DD_*nX9d+9YOxZQOr#g8mg@-5U-ubq6?skJ4ZL(FXuxa&omomujAKuPx+Rmsay zib&3Wodf1a``y$yE20;tL}Xf3y~}*HTJF$`3p7s!vYwk*%^lWph-dwzIaP~IqwF~v z&O4Aw+NQ&Gfv9d&s^(?PEws9APejY}4-4Icc9_#ufePYPfh%_~D&L9U%t;Lk32M!A z{2u4STUlI5WKK+_s$PmPpgGbFvdcqI{N2nZ)?YoDfX8p4UyI{PjK_HWwWWBSPS=iE zmf3jQ=hedrd1*|K{PV^M(xVe;&uXDw|#^2iWe3NOxh0nKmrysx8(eEnx7zk#kif#W|xK$ z;Y`>m4^J%86L%57xqPG?XHBBkF!!28_lK+S_R)NAhM1K!x|26vYl+n({X-NFuv+(y zhxQM;$-lB?rMe9QNS}DmW0qI}!soQH35cF?11X+o*3TX5wp^Qz1*6RbJ&ATl#P1^n z+%FL?GQ3NV)MX(;7tK4GI3SNtLsxF~+mqVn^Xz9W`*L%w;F?g6(~k)brXGC4%kHMt z9v596eaXtfs07gq)~z}RL{1r}t(!Cnr&)aZ+;}_O`W3qw3$0UB3YKjryX9tIJC7Lq zaU3}_Ixl8c8RUj&(7EJglqP%6t+jHtsX4(PMf=}k)_<52F6U$%myma zOiwJ>8Ow8BTp$btM|qMT?l3Y<6-(Xf?XGXHqmgZ_Q%9&s_pnqwmD_QtEn&KRsbv3^ z07IuLyPN(-G+Zv3q~+JDe2=7!)DhlwV=7UElN*{Y!so#RH7R~hna?Y;;*PmfQ5?%r z@S#Htc3T7(S_Sp%6Q+8_`s@9;LwDui)O-5r+YI{ z8t$OE+m^+G@roY1>URrq_71uS6%$V;CDqbhqc1u@r<7=%iW!> z$#0L2I&)#GVe?_XWF`n-unl;&B;r{gzrxg8HgKX8jn7XV@*(6uYJTMw* zLvi+qaD1y;XTx6J%@1kCQ^kV`D??GJR-mSU6vT%vy!m-+R8Rd+I7XHb&{)%o2Xz`_ zdw1e;4K^rM78>>0Cg-lq<)bKz#GwAs5wcAy$&6ua76Cqkr71x}5VtkFx z&J(;JI8>iIH_05w(f#DS`$b!UlOs0Rahe*7Kf8jYW|}VN+fb*W#f2Cw^Gs#~>$!A_ zlx0C|SF-oQ4v*BeZ1;08P-~A&g}m7&gT`9~CCYmXnj3nnpizMx#*a zbCPAKS5P}X(vjWs8T2jHnXogv;iNs4a-m`07QW)jp-g+RJ#B5)wP2}d--vv3qu#&m z*WQ;8RWjt@>B_*NbT0ADp%mbDMo6XbxO&igGX1>Y>&fAvI;&3KvNTJl*Q`B#+ayNl zV%FGKyuC7Uo45&G!hUFdyQ^cB(0S1uF2)8`=_pKIbou`Bcu@NF5HXj_ugs%{Miq;h z5p!9E23;MlV2*5OL;3&(+=V%XiH3{1*Y8(%`%@03FsJ=#A&2@2-{QU9QngPPFpAEX zw8a}?5(B%F;Db-ya_x@dG84Y6!C$6MM_lK?D>RgV7D6cyu4uK!Y}+%80HxBK=wvgk zif#;`w*4}nca4Z};?x>)6s_1{2n0FD8p(LO@A;z*>l?ab%i-1uB{)robvbHs==6!s zL`ouevg+Eo2uc-T(9Y(^#p2pW|-^=wa`70%nY(kEBxZD0$ zyYkd4=g%QcLodIuH8Ul{xM6kVgT0k(jLEItNGe^3|&xQ+m!sm82i4abe2$HD|r+{f*!?g+H^aOdS+(f zQ%WW#rY^3X^@(2!4Gos2EQ`1ZK0bUIH%!A|n*3^d{{&Drt#+iZw3o~lW*i@IyFrNQ z(tAc}SF`k>Jo%nrwBWi^a1rl)hTkDW^K?jyBdBDr)(6;93a_{{+G>;dkfFpB!e*5s z;`x^rA#cxOJ@yhsHbx`R=@Q&{WYZYTk9Ij4ySM1$*gw#m?cpTY@{VnN$ly)bViYd^ zfw-+~aC3kMBmrpK)7azGe^q?L2RcK0p#O+9Xy1Mx{`o^WFl8zD(g6OJ z$`nVfB^Hj)w}-!8T`>}Gc&V5Jn*~?5&flj^X2HbKi^C>_N!}sO>NF65Ninawr>q3jJOmQ=8Xi+RKh7qL}r`Y1ShEs3I^7dI+n+ov#vn5xZ8N@%!q-S#n9 zNl`=gkXoK)uYcF(yP!vfT?0)Gf5Nu}h!Yy>yd80BsV+|<@US4`jk=$q=@=)~+e~ia zVzS_X$BrXuP^`?JI+{KqZrJ^Kz9}q4CmY3Z{rlXC>Y$yBzZRj=Oj16n48~ELhIrz7 zZP!MJoxJpFxXy=zqCRp)FPkF&(T_0Ho>*u>SSHWgq;iXvF#zNE0TkARAIC=4e|2Cb zi!4FHpY;~7ZYin=^}Yv#gIp``n9@#oQPXyh-tuJ-*?ewF0coAewoDjH;VG&@Teb@N z%8syYpV3(+#b6`hm299O;FZjr#__#-x4*x&ooJ>vz}#IFBfFBWdc}O#Spb%8d0~Si z_stP0hY*Kep6?{!=PxL#P+IyPXL?Rb)gR>qW%!#Y-o~ra%r0Mf9UEtl5aw!Sd~2yy z7m!L|yjceBDd!E}U#I8?f!!_Y%BFTx_RA`&-=a{i9agT%dXagp!tCr1GjG;~LfI`R zN<-|fQmNWHyo_(a_cG;|++UlKHl$z5T(A{d!#k*_pN3x-UGcq=^pQM zNBQMK^SgEMx(Qst5&L8r$1(-SvWNd^Q&q%@3cYT2BPNY@zsvV}yO>(WlM%|@g@$=W zgYb9&Uvn`!bTBVVC~d?(8|*c|^6Tu~Ve`!EsQe&mm^+}`TBSxB!%>)aGR~2Mf=NZp zU4Qts_9G-UNBjINZ1+I@bw6v`xBOi!9pjx?ust}K;lpz%``%oQ8c_b*U|NlZ9GBl{ z-<h2B=oB%*2VdxH%SxM6jTJ<$f zO|QB1!edT4VxFS$n#&|9yI$A6uz+0BL@{AJ9ldXoO(A~*+l@)>%w>>knOAa9m zmxHL@PN%#GR^V%;5(j$UBJug2tW^4Ca5((g)AbQLA}PU{9*JtF(KoA?%WWC;IARK; z$&}<;sJqxVdfc2fOmu+`wi0&(cpevPOZV8!^+&;nnTqYJa*q~ zJ9#n|`rk#yvGQ&WVQcr8MY&wqWskkAI?$sIIw;H&t z=07mluDv60c59cMZp!ShC&65J5o}DB%jkVh(;L`gL!Hzz)Xf*?EunHpgGAoM&nCo; zu26{4^80D$@1ApAO>tl}7$sqLCX01(ca*W!FAt5M_MWX?Pai| zG#QC{ZEK)>b&NvoAxS9-a#1+OWIVX|M4K?=XPbGE1t+HAUZb?a5J`6HITpp?dgO=y zi09prY_r?peLDuSMGq?TUB?;K)Llp@I8RJ%9mzOJ4*OLC(Cw8*WXKHRgE%ijzz@Jpl^k@a|SSmuJn4RqEJ%Sr7yzh(yHEv%l zFm4US1#OSfsaNU;aUw+eND{xb`aH${&D^b~l&kF}pl6NiU#AfAy7$kqm>MrHR`9B> zWt2U9MQ&ph)~0m4R4LLFU3Pc>5XAXXLHztQf-eq{5PuV+0}1tL6Q&f5&G2Csdjz|_ zhN0mTBC+jtpJ?^g^nEW?z~`8i5yz#NM&pkDk@T5?Dlk#YFIv>T?D;HHy;w{cUyJ!F zH2$c7eeS43^6a#Wd<|KfbRfx62>=Z5LN3^+_~@rMcs=)r1E%ByRC); zu&%XR?kq@;3(#rNDfU|5zaYDhU{lA-w+|X8HH~NBnAj9-DT=SKoQver3WfW{4{Q{z zS3i*6Xe}aO`~AihdxFHs4sXyJ4Kq`_@Vki&J|k?w#9HHQNxWKt1cB-C{x_S@z%DkO z0%?%cJ_us8-PX0fFKL65wm$F5ep&9GYt_O&F=17%Ujl23Bj@h&@98zi+3(GN@bNnC zTt~}Yv2tErZl@0XMUj?V&VMxaD|o{(75XhzOlICY!Kq0^Sxv{aPNqwHSEWRSjpVe+ z<+jSM`jrjIeWlAs&DGNfJPt5j1TRR{OZzfr99{htl)Vr}GFq&VTrSUAxgyYHGwW%; z51Em=I_3IVC&O}-pHt4l;(josfE_hc&*iIM_m`m)_GMDHERA21vz*kp=@=(9=A2fb z)^4oScDl+FgTxN2>5asw>3U45?=825gFb9sC1fO6xSzAn58MPT@4DEnj!t8{I1v>YFinQmSq|4hoiC( zv9V6mbPmw$Xz8`^=XXR7X>TBfn4;;FcKd0(Zqik|<(CZh7bth`H~{mTHZ)F?6iAaf-$-iP?Su>yP5GCPhT4wY26u zJzu?0iXv3Q9gR?=;`9dG!H`{*7@;(fOKLuy@J}28II>8K$Bep#H94)&?H+oSTo1Q`AEs9HtyzGqDT(d(>75_5=*tHlK`F6HcuOzGcBf%S;+{LAQkIQyL zxeNX=-<*?9B9LSMPuubE7>7VMWc>Md1FkO9zbKgAIFSvsA+Zaw^RojTH2G zlBI9Bk*rByLiZG?K?Vki4qx!oN8TLb&_If3Sj6m7y9?t)9DPD@xuwh}TP1>)I2H}P zY}#*vWWFs};dtF1B1>2V(k%JERPx#zU zLmL{CbyT?Q=2U2Ww?5j7KMfL2>{l~S^3pb1h&tG&xm^r~VtTjnAS6~x)l-Et1uXf} z&`nu~Y^v6vFeLRWkD9jO`wmgngJ3=}Ckq>!L+=*Mu1QvBwAQ}2^aZ=+mYvP7_vag8 zTvtW{Sxf@@3iQNh?vN1AU)4fi>n54i#<^FrcP2HmFy9B7=Bgdm7DcDZdzs(HOcd#u zYE+vQjV~wL?bs?J4o$VW4fcx@or7ePlK2te7z5_LV$;U($qPNhDAv`GvyYP925VJs9joB?>c7>&!ysMC=sQ<&mei-e8rR8 z^x;zJ{w||LSawvY=6KdQhSX4Zav;KLz0ILy)S_cpX@abhXPRr#4p1gl+J@U9T9OP zRpdry!LFef11w1%kXT7?1oSWe!3jhWG+9vVDJE zAyIXkm*y|joS~6o;6*XFY;wo0VVpFFwN;iu=1rhZlQ7vRIIq%1Q?t%+1kzbM-{)T6 zjv`gZE2~n+fn@K;+`laUg#H9Hl}HKQ+S%me_&$a%ym4HvBceAn=A~%GJ+Q~N1fDbYRc`-Y2DF!v-_Yu_{Zj`?B2;; z4udbxFA4qa((;I{s6xAusHd9zutiDxmfw3IUzm0JY@tqP#P1f_LFb3r5Q5$-Hv)$5(+7f^TzpTP&&o+>Mcv93CaI*HdIiN$ zMRxIJmxaXq@=Gg@E-*`{1)oWG4gYyU>?G~s<@MjS0F2gdn!My2YI?|1W09o~oaqhB zcidlJd+^V)MT(QXa)syoMt*$2Wg+uhvk8(byd^c-SuJC-$#>{%u!at5IR@F5z*vm* zb9{yXAXB_F3$ID&`K48LUz4w-g5Rkx+-ot>M5Z84k%b!vQUaX-_pRMlbi_?1(sb(q zRb{>)ywxamVG1`vjwFz;5hJc)@7Xb(Fn6=a8=jj?s;bh&<*phBe`62|&KfKR-%K8? z*7M1yTwT2=QYonaX*nJ+0u52^O@3C~mAE4*P8BpbI}-HS<5I8T4>6A`<}{Pi%mw9! zPMJemM(!Tt-T`xsGW)vZ4u05_;s+!p94$602gC8@Vh@egWlwvf)s)_5rof2>4buXh ztokx?n<|59=dVlAjiwb9rEkRnUH=_veZctaBjooM{1f@vKk;2suVj5<* z@#J$&m#ZNQlh8+yj|W7f0*1>B1b?H;$KSC>+Wv&)+4T$H^tqLt#zgQNx5kz2Vm&(J zzyU`(mJ5Dex__QWrf*IzU+>T-=X$mg!n}RZ!XwHO`uG)+XaQeUda~y^l@W6H0^y#l zONnAb@KGLnjrbn-#$2D+SRhuJ(U$zOtc%eCBCmk347y#4hUD*n_2aM@;XwPdH&BWJ zA8v%fI#0I&EH&MCSTwrC1@am3!zuOslhn{$8Q28#{u~R+5ah8@g)%?Mts5uT7bsTD-vjx;mUE6~~%i=XwrU6W1@BA61C3;~%fl zo`gVw>FqWYT_ZTml%83liTMS~)eCXj59Ey0trz6s&iC-_@3Th;@gC}u8lNs@go zkkh%o+fqzd_JR1sVcbF){hrg1@IS)^1mMpD?~!waz~L+U_Z$33lg{!ikd4;|f;<#Y zq~S-%(xW9j!&hTgzttZox@IS9Hjm5vtwa79Y$WNQwS$IQk$%S|fb02q_@p6=h)UvJ z9Lwr5i5-jBVVgKJ@YKuw(6(>2o2Vvm*l5sM(Htd~8Few7ds@%vDrIgT#GK7*mhvw0 zNjh?_6!(SVY(}XUww0;wC=5pm?g@sA#g*`rbR52dr2X++tem4N`;_IcCQ}l!vIFYb zRNyqG$ajz0Zre7b)qMmv=C5J={n&^uAshGBvAQ`iJRlorS&vLck?~q3H!(c-h5VP# z$fNdK9!2BUVl8@Xe6UdWR}qrL-{HSC@K))b)r1t3Wf(=U%sS1l@l|5JqX5{TpKShM zl>g5ad*ueFs901YlAX+%kgHYp?LWfrulElPtCFcy0I(-}x{`POA_nFJ!21&K$ zq@*MxoA8!dKQENr-$irK2z$4d726*#Drz)BlBJV?0#Rzs@V~y&`cf&XDkgAJHtuC> z<+=D)j|eVO`uZT{=28r#o>ky&>FONnhK#(%Q{ek)KT+C2nORsg(RMCC{mSZ){ARN1 zdN|)%XKZyNZi?Xv53w&2(D+X}VA(}A)?UueQo`QM?h8Q98q3qAH{6AanW3(PJ4D*9osK!Jbnct zzV^|K5p4tvu8aJaU`|evuF*ac(d>~%`BrsS?;`Do)SHmtfoch2F7kmWVPwuDgX2wS z{H@+Js#qCW^~j zMfdLv&R=`-pU1Cya0iSem+pknsK15f_u2o`%UqxhabN`8f6Ds*^hKk`dR`jv{m_4O z|DVd>pFjTRCF&c#r~mAz|3wdwMT!FE#0Z^Op3)bn%kl=Y_Yy0_r`bfcXHP$hz|}+* zt@ppk{J&2FJ_#6(Aj06D1o0b;Mr26xGnJ7*NR4LRpC@aUy@CE8C;w|MUC9EvvF({? zfsl=FmA{^OETZUL3~(`cznxAIrTDE_{!%Xs=x;%`0Ylf0;roLC^^gZ(F9OkD8E30362=~ODIXg#FX+_ji)Sj|Y;#zel}k#!X}KjjxI=G1sL z|4s4CiR+J+w^e^!5)0uk(Gz@HQ)4Xh07ya!#? z2}*JbNqec!aY~8t@b;+!@8=OqGMr0b4){{nK&z=<^SLe{#O8LD!uQO|#K0iCV;~7g zV*p6KKS6r8$2`gh3+{S;8l&O$X^kQ_4H^s=uPZBS%(9RuH|>E{4X(?!@y!IR53Xv< zHo5|0AVzx6$~rE-`Ed2g5c3~7G2-^k%_Y-#m_g7u?ZQa9n@h02(PgOc9ug7~=b? zN`vIhh}8xxAtMriD;MiQ!o5=0FL2=BzIx8CT3$3&J;*G8$ukW1q~u~uXj;ac6i)Sk zWnpCRp~YasYum899x6tZf10h{7JOdvL|O#y=VLP9SDQ1P>n|iaG1Yd^(V0|O7Lx)I zua8bf{m%zE_{n#9;UxkD3>DlP8{ms4@PvN7?w^g`{xi#2->+&c9q~yp=zDTD?D z>OKPkZa0}AR1bw>c#Y;rE`(@Mmw7Cao(Tq3Vhr#*A3q_cFrm6>7cT%GmQ35Wn0W|c}6W|)9l2&3T zG^1<&Q3Tzv{QJI$~m^@Tm%z5OT^5q($Q--$XNz!Yrt zK(Zt?wZ}izB&X->A~~5o*7Ee4brimIQFxT=Ju5!arF+JllgYyhmZ$D*8&Pt+Rl$eK z42`DXb6VcVwRtQk_^aKU&T|A~Z00p$zFxJ(h%Sh(Tr5~xeUSni02P1fldq`ojyn~; z7fY}zD$y@kQ53^?Ul`g`l7>@w^~#&;TLA77L{hU-{|U?qqL##QgO>n6e;v7zEPs6f zs4v>;I~;dYci}WRU+|So8M-|(9Q>yE)bCogWQm^7b{aeqiA-~Li!B$Lb?&w6#2mwL za39y-hyx(aR;Ro_8MT5x&lEz;%5MUIZ=rlA@j5z#MsAR_poZMvD8%;{*(*FeTGNq6 zrx&n!I+py=_{quI3X+%!)C0T5d{Q0EdWWi*4xLgMzGr|txX+!Jwe731cc5~JZxyk4HJKj=e; zA}^QMUSoG(pY+t1;^tfyv|`l?b4MCjG6QHhaq85B&uQVhtJgeff5-$HG(7PTrgVC2Ye`7{M2|h6LBMheqp^G-5F2Cdb&E~r{$IVz8iS5HVY-l_=^`)eihWT zaFo7HXZ5tq#`EUpY9`$ddFu&s03N_j`bz=t4-OFlJQ!#w^#Tn?FryFiHmC>i7t#q&mDk-ms(&zeLN_jcoU8#G23(yAWtLhxJ9?(|q>l z(eQ+Ce_9tjBA`VKv3%h-886iJ4$|JZpoLV3pkf>G%eHjHk9^FiKb2uvn7KmGn^X*- zA9Cy!rpl&qz2QU*(AOgnL=zsYK+0Wo7TzI-Ar{F7A^=YYhbDh!Uz|MIMwTd0;zlL_ zNE=Z~up2!?JUWyO2dq-id!m2s2N@(Ft6m_V9`6>m)12E{ba?h5-x6G;(GrASZh`Zd z(5rI#B5i=b?e>12%lq;ddXd5r9FWoD<>$t~`QB@;u|Kq#)R!LP!u=C zZ~~UD+olRPI5(-N;6;mIhSW*M+XvzaSG@E1@e;!7NTimg9g zv=?ymA0>d~*sly9r?K4<^j*)?2{^kQq$ap#w25&d5J?t=GKL#`GtLD1E8Hz>@>lPP zCiG?-gm*B(&voFzsp(qJlAmn^3xN}UtkSC z4>K+UGJtOBP@aWS_pR5QdVg?pCXd!=>s)7IX1~m$#!IDGCElX^kovH#{7^nmqu`XI zZrE@sozv@sTpABK+wNqfIa~4P@Nhf!{nR~|qV`=;Qb1Me=Xq9MyEY^*Tf;vxRh(L? z$BW(^HaBE^sp)C5zTkC?sy&qRH;pbF_jE`AmMhPyk3ER=#zRK4 zr5OU{pUC&L*wX(?YaIo#GXZ-0QvS!$Y3`2tFQXm&ak+CU3GRMK?SNu`EkHhcvV#B? zf1v7L5+luI&K-%5ASQTcn4JEAlRJ{g5Luzf{%OQX%b&Jz&CKdL4z+;DB9WSffmbrz zhW+Pfe!-WS@5ZG1lZwLmc2GQ@!4{tnva@LU!0+stqw*r$!I2A^KM3x;ut2o|(UQ$}Q+VmR?znX?O@jlOEdYRaO>a;Zo+s2Z&fN&N7Txy7npfOvU1E14u z18}8X_XhzzML*H1!a}WH=`%wEb7W5*Vdb5s?wRZ~nE!0tk)@=t!u>5pXq#se){|=0 z#6*uuz%U}yNTds2T8pO2f2ayw2usC!R7Ps-DdQCNhoy(2ZuKZMu?e_!4AH}QqYY_k z47*g{-y8JI)mm9DWYw`+jEk`AyD_JEjn`s*_l}{U$Gx%47_MCeA^;-Tz8+C3I@9s+ za1@!Sv7M1lv%9}Wq<3`z)a>7+2tl9SE^`5n^bRj@bX|hSALm2#_0Pr4PHqg(w{62%Deiqg*6aKX|A^~XbrTSooD7r1>71e{dCXoT^-!??J8 zw-NB)!hEKFs}=d%S*QFvg5z$A*^h|l4*(QMg_JI=(tBHRH(1RvOz_f+ZZQz72Nvi? z1rxagr6D~G3=Ns0l__b^OTSD>sFtaRv1v`DLOZ^4bw&9i;NAoC{~-q4YQ;g!L-vFc z&bU$!+5CB!&yL`3R!SyW|B^Yezyq^mHwo_5x|$VMg5gY5!nhNO7cA)u zd+#zUs`34HuA|z3hHzY3Lv&tU;o)%2=x&%&com@pD7{;QaC8*ELsGuBgEBXy3F}O6uRV}LdiYuY1)m213!jk8W|_5K^2 zvEkYLpX9Ln+>XLc(cWz=J+QO&3Vn0Dbi2paO2Oo_%!H5oPuO$}GY#@Shl6cuV-ySe z$9vMJ8oj>w`aT4{^A%Y53xV)v87<)KBTLr@=l~XVAR^(F)8XP8i?b z44(9hHNOm|Z$CeXrvmpIycVvF0W{ecOEq@L&P+7F>OJ+~w-i|R2=K}&q+F*VK9{WE zEkStiHQhFqBm2I{sP~2*I3LlKD&t{rtIW zjsF8-Y+1^KAZY)9%Omb_APNYJsM5U%lK^6W~5@7OUBUs!nDb|7~xE zO2Dn;Ov|@O_AQij2^U~mz+V}jMVg@~BTk4;+>P7iI(U;C(0*eGb`q)(od3v=JyAVY zEtg4L-S~3irljTee`;E8WQfCvr{dcb7( zpq}mqJCZ5jKT^&sDc@%}+49<|j;zT4!c9Pl(ymK|duXW&<<3ZnxdDW!uGPsXN6Ax1 z{YvP{?6VhS*v#E&A4}(z``F!6W)Y{>Qh%U6g_I%u_ntIu0)~B1JzGA;LO@{Ncf%ty zog6?AtsZCH^>wEmkgs*{ypmIx*2Kz#TrSP$_cj^XfA%G8 z3S%KEti9-NmL*~^wxK`>syz)JNA2Xkei7;AZv)uVCB)$`j=i-8(q5W0JF!J74f!aj zkjmH|+-FKFX`5&CR%zvO8S=QuvmpYNzdZ%vPP#w&5aE`SOxZgNH*z^IS$A{2iTWZu zbsu3Vp_fd;4@ezzVvth7>g~G!XfE%xr5y*s%fhX4ox6lu^OEJ}pk z_avrOj2&@09i9h$ED9NPyur1tX!c_L`{_*op@NNG2v0Y;D-x9WrCOg)@ypy5glhpc z0cm2-Fj?`AF=k3z#^_d5s{luDc1P?%WZDd`D9>+_RA7OVZ#7{i+iKvBdJeEYy_XU>=%;+Qx5*n&xXODY-g*?9BI5lxg-- z3w6nS!U#_g0{p__EX?bKdMH?OAxt{6p;sA$h3JOH6Dfz%OV<#imE}A8^C1nKvvpE? zE zF5b6%@4BrC_KCY0s%N=*C|7SHx)zzO+xKw#revGy0TlY}&=`zo{L+!qBl^Hs?7#gz zP){Nxnw*DkEbTq$SDD&M9C}WW@h6)=C&=ZG;9RU}W?F?zfgl$GCZ$XPucn*xcK~*o ze};}*;kg5sx+m{1Ux{y^qZTr}w-!c_wMN zgebYtPgcu~Q?|{Rbct1@{dZ&~u%*!ksGSKw^BXmg)VafJ`radyn1- z+kxNfJab6BAO<&;qtxnTv72vY_mDx80zPPD=h}rZ1a~WSZ~j_1&+*+u2F{j4jUj){ zU$E?YuTZJ-cnF{t?82wW+Dk>tXq+V~hH@uI#3Z&}m0=I4I~(`ar`aEOORgQ}P>P3a z$atpOp6_XBbU695T1-vaid{)0zcW7`RD3&-NYcQHLmp`G+2m7$($~3e=Uys7=cZAj7Sjz=971jLG5R*H7`Gzp%r!Yq>2q{%?orj z{1Abd6~5j!-cC9-w8W1OXjAV~G&;l-IXhdJW8MZ);S7RV+(jH6Wr2KaS^BBg47z9zfs;D~7gzhR ztc8%hB7sY)`^z&3E}Zw$VP|3YGG2}C54UuL9}#6M;f`Z=cb>|MiP?8NpAKpQ0zvRX z#PYM9aX9VU;QA`A*Ex>}`DI)76p0oYww)@$v5|xCNTx)U zN;FhXxV!_pf4q7Bg5n7W9NG`{g6!l!T=961saGA{7zmVL z^e?mhOe~^eXE{h_OnX-0udC{Ge$^3@_1q5Nq5R}&c5?x#$JS++(q zB{N({&aZH@8%@tEFM0mTb^nu?_&bsWWIp|YSbdaThQ{~O*R$btx_%6PVr~O+1?W&% zNwFYlYwi){Y{o!hPNbrzggE~;rol)(+g)aZLuT-uG&=i~unV!^PACX94sCtilBdr| zkvqnMYV@7LuFOQzc-$}nkrsrH_eok1AU_Eju{!A`LxTe@f3M+xBo-du{A31zE|JrH z(+zqI7i^fbaI;HxINg!aA@MIXGoDkpb-rtOK~$x_p|kLuT7vykHAde zVf;@Av??gN6I?UVas8#cUbiRm5^|4V`<}L2E{NP?jz*;3fAC5ww#$yzt_{=d{kqj@ zSUCIycuhCJF(8_s9WBK1W!g7g3|fCt1w)C9z^9QMC&3j2>9jT*rkwvo!v8(_^5=6P zr~R$(F&S)&AW0NZyQVv%aAY*!RAJFL7oQGQ;G48CjQqIeZoR*%xe|BTHCTSyaeqfjnBCDS>x z;~l-;>urmEBIr$y9KO!jz2Y`T;fqNe+^2V_Fl7VKgd_o^eTb;t$)((5%Gd5UkpCM$ z{PXhF5xhs6C()5Dk`Fx3tHhAX3}T3Cp`j@E5pcT}oVAa2aC(4l@Z$b^xOD$XrsyA- zMLAl{f-^!91t@rLzH~*&>Ex5rwbH{sz^~TgGYS#`>iY#EK^7a#H;GF-o^+{W41U{0 zB8@NIk^wGQ*ljo?4Z{qFLy(^{aAUn3r0qCg7ZvG_<3ub&KxQLc6uF!*By&{~BT67M zD!fK)T9C9^gFRvT)iGaxrsV4T4C-jyjMV$Txuw6s_P>cgN!p{--@}VI-F1kZ<-iGK z=8)Z1p+!lk;F=?-1lI%J%6dVc> z9C7Dgl=)2x8YGw3ty;9j^`TdMSU_Atc__bljjT^DqBj zl$eHQt+sF>ygcsq>Uod~ue!QB1}A#Px$bCIV_WrAq?gW6TYPHvAt-MyzN>`XMp^e{ zEPjCn`I=>5DSNOb_}46pZ{g(Ru^+KvD|s2i5A+!yQ^c7 zl;9@&l;q@T-m3k6&5+@Qm1&5K#B})!=PwhmDKHrz2igxbVmzpDfAb4|@8(Y;a9t)& zcPunCQhK<5YP+u$ugkSEca;{HVj$1`ikKPYe^nvBN9Ei9n9es^@dl9Ke_#Ipc!wAX{`lED zG?Tw+82^&_pK?EJl)oCkOdy0HN^~Y>|I}rSUqOIZJC z$X$m|I=M*cedfl|s&_o^5w#n0ByQZf3APsaFw^wx*BHu^Qt-XAGNGMD*T%7CgVYh6 zaa(GqJ`g4qkTS0xy54153($Re5Vzf0b)SJBL~UB0MpYC|qBey`^(-4@p&8*Q#MEdx z$a(2|<1(*LAK9yrM?yq*%LwSd8h@tbA~&t$pM+W%)6+aJY0#WS&aH}1MP1Ep&Nz^Nm0nc zB+-8qV1DUe7?Juf!~1XQ6B#}R*o&RH1FiqDD*v`w|Mfobqd*Te*98sm|9AQIADfpX zvL?OxKNR<$9|?#&ev`z8n^yKec_05>{9efb)(Jj_;^a3p{BP##UpwNH#G@Tsw=&K9 z|B?j!B!T>&{)Cb4zr^nMlUINErg1wB&a-(B%+;7a<>cOvWicJ;&nS6_y}Z2iIjLL= z0o>a@w<;On1P#hpq36C{O* z&QP}gWa|hBfXCt3L7_VP1#YmZ(7e7`lpowiC%gL9yChp?ue2(-Ck9Ot-d5S(0Dr*LB%f(~0 zfoE=R_35S?geDe(iLUh+mG-|gqqG;K(F<~S3PYD3LAMtxotAeV@X<|i?r$aTd4hq! z6}PX)a?V@f|2h>i{lKqV8D8&*LeZJ`VcNil5>dMfi?4Q@G6yebH%YhNbjtJJ(=9YuC5U$1S^^a~_}JN1my>Se zYi*VkRd26TYOM$j&Ik{{j7d4=ln-QbCl?lk0$4Nu*Lm91V= ziqKl#f1B5rf6c4G;s(4yXwGW}h*~G5&34oxX5_ zIN1NQiKq z=0vkj2#40eN()8M3?O*QNZQ@ga}|&YZm^!C>3FE0Fx9Nj;A3Kr1{j#X^$X0BwXbPa#(rh&zH^M@g>FYmOhcR^$!RW7&`kxl~y^j)7n$ ze&*?q4V+|`Fl)P?E7f6{mDog7rt|vgkRMx14TU-12-p{W@?1XaPS*h@6_f{J8 zthYULSPP*Nz5HCdkGE(}mW&JV-M_cO<*tbal`W=Jw2@mSH^h9=a*j+$_tq$HJ~jt0 ztQyP(p)2OoYgNnOPJ`)4#d%2h>24K^RSP%AOk`NZNPTTdf5_i_tv|WdfjqR>^YuQo z*_v_Ju@lMOsGF;lJeHr3On2XO1%K#BT6A0*w%31nk}!DRX3aZ4y^tq&=4L;dd1Ew` zW0>bu*L=L{FfVGmY~5B>f!^*sstcRj#}!{FvMBtMKAG~U?e}g*=MfXW-t}Aao+c~R zgTNQ)1#63yT7SfC8)cDz(lq@7PqXASk)DmI38Dzg5<^nQ_hfx8hL$ga^ZpFE0%_gf zl{r94`>uH0*LGJ|eWDYMcBJ6N+A4frH|3FZURpNuY0iv(aD`YHZsN|yr>gufaxD2y zn5YTNa-`a)w{Xu+cXcik`7Pm!!j|R=pNgbWG+gY`LOU)+jyI%lxGD(p4aLIZXj}B& zAC^+mezk|bsf(Q!9LV!svw2~1zhMo$D=O44;qW9--PcQFXh)UUoBl6S>DlQ zcpO!c(mP8K<0D@}`6~B$yiJ!L%2OkbyhjhL3Z~aPf1-Vl7X4y8(mFSh`O!1Sz8PUY zAc1nw6w3eq*!$|RsJgCir9}xzr4b21KuKXpK|oqkTImjfp+Q7MKtd_Wp}S*fQ0eX% zhLC1}0qLCKJLvt~ulMu#K7W3Hz1QXCb%r@}&faV9^;@;qT5-5^RQ7La#k_CXq!L7b zj1arOt&(V}IFxr2$jFQ?s5de>Z)`lQGnAQ95xeM|9M9L4HSax{KCZXLx=fk!f6p3u zNWi)9kQg|V5ig;$q{}?c8qNcr zHc?A6#y+yOb=~Anl)$%h6JVl0hs@jF4mew~6>b~;AY}9iw`)-)JSG>Y)AT^}7XQk# zc=S+CyNfVT2DG1e;0g(z1qK>QWD6RYGdpVLST#AHtMG!?dLv%N^V)=;A9j?u&!>LK zF!tD6iNTd!zyD)ZDS5oPST`jNRjAdNe}h)6^%k>2iz4y=G7b$W_-m$JPm z`Uj|ceiaBJ#=;OD>${*kr5E<6i++QBy>4l;NLJ7Tg`A*qn6}P&i5Ond=8em2_XpR+ z-Paj*tod9<&aM$@A{)`AR6TpwX8eK?bZ8fMa=rPZ<%ih8yd3YMy_YLD^NYv@{G*5Z zZDY9%*{LTSqUV|RQ8B%JQ58=Hh6&p#igY{0^)9+ZNV6II9{|+{d|tpk(Cu{>214|U z7b(eb<)T1kdI8f(6jwimQPtkKFVt${om~#M;|GAL953_iaPhYyC1UY1`-T`uwq?15 zQO_PF)^@MUYZ$ve(r%T7sob$Ii#T=+E{9jdCNFn!9^KbZ;f(!t+=L+!&wF>s#bl}k zs4St(x`KEm>eq`zUMzYHdcztZ;nJDXRGwWY5MId=y2Z#vM%4j(3--9CVy?* z0o#MBvi&X^4O4t}V6WMW!bJ=+%t*6c&XE&y0F+CAWc?_ybQ93pE|VW$FfEK&BH=ikqEfK;@2_oNC_;`=RhKf$E!{C_Bv{y2t)iQHg)I?VLulZ zje%^!Xp@fc;OxB)I?p9WtC_;EGOiERt_J2nU7V)l`x#R0KJpxX#TLwAngUw8hpa{7 zw&z`}gM5jC_OS_7*3X4etvjT<%bVQGBPwEoG+YPk6XXCxuq6ph1Xvsdyx+&`s0*QaavXRJH1m$Jxf zLuZGW#4(xr&RgtjKZaKt$N>!$?*Xn#1-!z%dUxr2(f6#s)Z;IL{+MS(opyo;F*^g| zoPD}{d&!SPwN}$3LP1PBJ3F7m{m?g7b1R;Jf-0jr>48k`klYNjjaa|TXN&LXRDdG? z{No;ZYY(~VF)&|}Zk8XnuuM+4^s7+1jU&FCR~@)#oULp)#28K(1`H{X&KKq(9=XEB z;WTH{jQ)e@fV&Qs<{C#aO{ z|7b4t!5y}W=i%L>^a6wQMPh2P@Fys(@ewv2!YBJKmaIV5S(5>g!Hs)o&1uVsZ|ymV zDh{dc!w+%MDcw0G>4vyfZ@Q)63}zroVcZiQ13?)tc^)quKBA(;*=E1AqR5KCQypin z&@EhDIu&mp(F*i5?`k;$)ktQIfpD&((8X<&+5+QWZ4xv2+N)5KJObG(uE{>!M7UPb zo{|njm)-tS@***@lp*0!dJYXr_6;XE2xk-e=91@tUUzwP(ss7;36-E#^t=cGH=sQp z^Xkf|2+U1~g&(4MpUv$9sqfg`ER)N7K}w}1h$M)x;y5bHNRy|!wN?7EFqp?q3?+{u zecTC5Q4(Wbied25Sm>WwRIN^qVb|sv7ll_j$BpJ2aEydBquSws21&o$_r^o5Ms8kl zg!k&DrR09+J1VgZJrDY{(>4~HQQtznJR8q5J9|VkMKMr6-zIUnSCLV;{b7VWz z%=QUM)?+|!a@*q=!#NF8NW0G=Q>DP3O`Fe}Cd zZrkJ2T0hF+Qqwi8QR)P|6O(mEAwrLZ`GFI3_Ry{{pQQ*d=r!PLbnC z^3F=-$#@LSpu@+fN>6F|txKecB%8WRCg`s7zaE!L>237(R5?gkw|r)yamR%;^AOiX zw9_#D z`c=g0Y!vF<7Xi9EZ{p+3jcB^N25N;v=KQCm)}K#fkOx+ZYUttL$^N zmcSLB2LN=V@a~>U=s8eqp!qv4!mGnBHi^?#+GyO9J~`x80f z2ZVvylEUng$y^5vuB~euMC%U_aZPP30R%E~3Z0w#zCx(yEgBJ$)^QS^)0#vtjHJw| zoV~JcM(s3H;0+TFaHO844yyviqkkRf(#5t)5>)R96xI`aKP4#)a>(t6VanK`PUrLL@9PmA3VI+DYWNLe}bV$*P{;~P^&>bMSXm^-bNS2D#;RgUtg9^vG z;a?!W$2$9*Mh@BPS-Al$Mq;S$Te?7*6b}AUP-0J=KKn8^8&OVZlfxn$sO;s{7$h4` z^ASkOH1AB6?jre?*Ut0e7!RdTMAxr^DDeAqmG@$_GtnzNO$EPw-kl){fGrKl)v)zR9LeT+pL_vARQo!+d| zAbO$e=IZ?+(DSjUimOYg)oG7fkH?iddUg!#mL_Qy;X1iJx@$s4>f2*2gEo<~5JOdP zqCpCOMT&oHhlu#Cfx`GWF~a^~$ujJ+~hWr(5i>Tr5ZE~uF<3%R9uy==HmQ*Ku}`8@CS3~cQbbD(j*uwi$&V0Q{AG$St` zlHqxFgp5l(o4zyGd)AI;GgTOz^vLO3&A>`7xzfeV`Sub}qfbQmFhRhE9R%KtM#B63 zkPio3_1}Z;zQFutqh`-2s`zOaPMg~YEpKn8Nd+yX_#bs8={Sp|TZOw$_&z_5VhXp*!NdkE6ywXv^<~v5l`MM5bNcto#CT%Sm&u2zTB+)sw;1NStIgLZ+Rph(p^a;7OgwBE*B(_{2@Hslb^cN9}{<}gIWB2 zQ(hu1#bw&XZFvFouKiGX;C=o5#+_+>joNJekwx9w8ckfpa9)AFZ`<-P?MP1^Le$%0 z7tUi2WJ~7Slh&fAs57ksR(8kY_Lh1a%zI*HN_ri;tj7wv-v`l0awT)eZ4}oUCJ0(v zJ+kGw+dF@S({&b##ctS*6ZnaG<>WK@Jo`=4Y2#ynCz zlmbtsnhJ65Oh}ES^$uH$DKRF00&qM#3a|d1sQ*P(z%lOHE2g_YE=FN$cp(A%eeTHA z@mJO}RF5a5glVI|qlUOeffx@fdi$Kp$9v*L-*?8P-W)7wBDqt{RC&u2YxpK_N^b)g zQ`&k)BtAYH`;NAk{C5f{X~)#pnQqQ$kn!JfD2T?) zh%Yr8X&nETy!Fbh(T1mwgIl&7E<(t9jHaAu)@cnGJO{RB(l;92d6H#pQKtl#HvK&f zx#nVz>Hu6Ks*%vF%uIRFC2b(LXFIrsfrVEp(=auDvarAn`+Z>$@;&dBC?Q%ItjY?C48#xg8u_=&GvT9er<+WOZBev06hzjb-FV z?DUBihGAK`0` z#DptvUqVaGrX?s!{nJt{WFMt|zjf-wAh=Kai~L8P%&}*j@S%2Ep;x1;bKQWq_Ld)$ z%|sDWTQ8SMLC|jceqLv@7#asxn1A(Gp1SGnsh8@XhSCJn?t{AL=bOuel7-dneyG;4~4&R)p1U zjsU*b4dRG)w)XO^EAA~hUNbj@eFb1eJj1>v)g}!_{3`~| za072Gm)xPBg3}jp{${x*YWa?3+QQ1yyE$WTQ*#{Lz!d}i%@VB^aHMKXMt zyNGi(T5~P$MkeB!jssqw{ms_u(;IK4;CUI3ZAJ^Po)jBob;n3Am$bFE$}v5H{!{@| zffAFqAebaSlvCN#M|Dz}1n{N&8y5meF)L3WsnFtaphS#lrRvi{y)!3nQs`9O>wH1H zv}OBnPw88~`XD@V+=hA=<6pG^BxT1Y=4I+{=VqMxEk)*g&~rX zuz^P=9p))DlNJjEJJT7E`kJezG|^F7M3iuhu`3P5NX-@FyIDvwd5THrrO5b4X}3k21jm@97%-^+D5r8S_w!>R-_b~O#F zRyJ}lgUxf^32GTTslZZW)pC-HusXa;yfu&@t}!-$a1$uHYJh`SosJ@x{*F0lXjznl zg?HWDFwOLWay=b6bw#dLgeC^|NDXm zI47dO^7>YC!L<*P$dtD$j|C-nB9m<~@iRt(krnD`_hZ}cdwGPeKJ@4(=_CNC(5lSz zQJBPeA3hXB&$oHHCFxNC!P80KN+G^{K5rOkL+^e)! z#2ZXvpo~{76)W5j&_`9;`@(kL87##z!{1q?imGy~>2%UxmoWsJHe<(3RK?M~&6ntY z_rJY6^0Frf&(+p7eT7s^YYNOh|4(+h3L{UQYL$9cRm!NChHe93V%2nQO;?hc(TB@< z>hirryso8d(K@23jFG~;91@z0W`4|>Oh)HrFP6NT9>e7m%;S#yFkbu z6ppNMzJq?BV{B$gB*JX`Sx$p}N|`UUAN;+QP$({UduB+E!3OJ-`am;%{n?~>a2IuA z)eSpeuysCKnjGfkT|oAM%XnkBowePJjb45L%8N;kBYo#8QX=i6#9flf1VhBsB-j2` ziDu#TI>Mr~4}MDCNo&mVIb0(~dgwx{^yV$ssS3fgn}ZdqY)D3Z{p70=67jAL?A1pM zC;>jIbTj2I-Yi~Bhyt-sMj9Ogl}|#=;%xS|tef z4d0Lw#%HY8H()*9DD`7}p9uSE1 z=czAcEG1zcdqx?Mc9d2TS)b=TkD1GxX(NVZlb@s6SwN~m*j>1HM4Ng0N?4G*LmvAJ9Gp&cS>=y6JN3Rdo}d zDsECSZ8~95Z?FA8Ld>qIR<*=J-1oaE5ybnjZOpDOh@=-QF@R@?F}GuH@VQ*F*Qyb9 zi+>bBi!b?_LC%A%pT#GNnO`fvd34lG9?Hq=vAg8@YiAJ0IjWI-T4eTX`liO98yD5v znC)OmA2H!3FCDVV&2elCoB9NZ{3Rh&?W$?sZ8&Dm^DSXgRW19Y8adl*@Ce&i#TZ9O z1qoblnh0@AvQl?6kEO~qoxmA|XzC|gP@>j^@ri;$O0LM&<&WyDS((|L!N)gdC~h|? zENghU12qLm0nXfX5qut-bhq?6r5C;@&G1dGIs*cPhEZ~A|KP!(jNIJh&`lQMU4~oP zCG0N#6cR0r$>W80U9kDcg$d%bmD73{k&!;P*01UX=nCXGyA{b z4Em^o%=_wrwBu&tW5H5fX` zIj9{e+@i6gO1nlUN9!HR=$FEOH{;Uc1k#Nqa{8_J7xg#wL$_RUQjkO5r~J84OJ8ES zuuqz67|q>6R&i85SuSb#e?2Z&%B7nfWp)3R66)+h)^qgh-D`1CEG0C|d5)(0xx8}t zZDduhWdkfJu@M*EQ&k`GcKr^wFqK2@AQz7-me@jG5a#AGgj{7}`E>1I|AU3F^P_{_ zoRZnR&j+7i>H{t-yk6`wlTWaC*=1C*9^ScwhevXR60ZPl^ltVlug4=`Cu0UeGh>=# znsZyX#US}bTTVOIHW^jpF^bqL;QLhm?_aCb99w^^AosZf5>OoPKMA=|+y6RAPZoeL z#<)Pr3Fw^Rd7~HI)2uus@CD3oxNK;{9pknQa?*h60d3ji*FKCadgtC--vfFh*iV-R z{6jj}S>6pqmlmWGS(BEN*2uw7o@b3t4v&YlE1Lrr>JJbVB;l-Z0(fJro!6&>dk{W| zVb<7pKz!^3tbSg!5P0y6-s$TZnzGH0 z?bLzv>d&rT|NM_3>3o#+{8RXZj(Zdx1t4}KqX@68yJ69dLplZ&Gv{tRk~Tsb;-N}gOy2uWlwE_@_A#Of=qwZD!3Macd66x=S|CrN9ZOa>Fyit?U(5eI^%XZ+ozv~ni_qN$Lwep$<=FdJ-Ur=lr=Opc zMeDSHOMq(G(X_qazif3o$E5p>v&poaugRctw$B{CNw?*Yi>ryKJp_Ma?yS>r9ain# zKB2Vsm;Kpj>fbtKNzh?%?y@O7`_+7?`^ong{4V(mKkieB7euhvc( zkI_Y4=8-EL<(+QTZLFI0xw=GqrmG}J7-@UECT1G zuY&#Ze0eE3b;(|YooJJ{y4}6dQpLoGVIIscuTyaN)l{hz!Bg{)A&6lEKliD|&o+;$ z^8Rh1l<=TM@$!T?Ludz`_cvd^-Zb!jud{b55Ecy7W?|D|DGRI(8vq!S==1%Bk=N*o!|@V`h9vzosIFL#qx>q z@u`)KgKB04ijFKc5(oVftT)?pBJbUEIoU3yrf3 z{PHQYDCJ~P5wtwXYu86t0B?+#v6|7|r0Y5wDTIM%5v%apkcm04>u`BlutMzpvXOA{ z9%1mh<(NZb>eIw=D9{lXwAku=C+fjP4T8NOKi`^-2k2X_SNg)gQSd~`23Bq+<@rz` z#BR~2dp{|qI$FTj_wg%#+v1nIRC1iS{)#2M&B=imZf)CfiX}m-%he;~-$>t_hQG~` zt;F)*B?UJ2dGT$|pBwwHJ?|fhzqwc6*q0F`p6qSkt=0b-oq>E{2?8BvW-RQw=zwq6 zFC-|~#o4)4?`$_X_z2FhZMiS>e|~0y4DC`!?oiT&C_;~S1nl6GMkUSDzPy%3)JN^0 zafgCFs*n>2#W=HiGN1A`YE|1r+kS+Hg4Ns`vu{idge38SxWcJ2%B!F7)Pw9iwz z_C4JVeL)6{B8&jt-94ohh}p!i&d#g(PJE#F$wPnfm_+>bk}y+v$*aDa74svL;K{utW&;fMhb7MUX zl3={VmrO3#-^X+XLkO?Z4nDrp7fOF?*R4Xt`@&>ppy%j2gb7ko##m1%@A>X){f)}F zl!9=Oz+Q*Wxa-OC!f>}8D`}$sDVaGno>(_MfuS4-k}cjjV@ zB21geSLXV?wrj+96Wvd*`IQ%xG#EA~N^NS0_K#>Ux8hv*0PVldjH9Ta5#pP7sVm#K z2+qo=9c;)E0KBxb1p0ztZg=;?viz%z*@<9^aE?%8fGRh#K#qWICIp_cz#2KnCewapUXS3ms3)(A+~(W_$6$ zN?gg;3i^OX5~ZjhI#W2&^mu!dc=y)vcG2Pa=N#laP;nUt_i5tH@jxPVT|JBV>j`k19!OU}hu^~FIoB;d7#(DlDaq5bGzq!^#;-&1a8*p|ApLh7j?E>wgN6NU9`&J{=!a^^89IiyHDc-p5tMfuV$J?sM)nMHF z75l>vJ||=4XETb2v`;rF&ww7dkSp7cT>}xr|G<8UR17p*S*>R+3dhH*W9+QH<*<5- zicQxImGaTWPuZ(OnU;EXh9pi0gckcu!Kph*B;H%9sbwJa%6+^Svj1jL2zBkSHeF4=Wom9QY$SIvq@dms)?R|lF{`$83#qM&z}j#S-(cTRgb7RM8Z2< zUq#a01jEF+?Qh-K4UW#HBhu2;H0$A-Xfs!Y45>z_T&6VQUm}3Z2nf_xihd9%3HR;G_C^QNf_oXC)>p3R+DsarZCh0v`suao z&_9j4K!L5G`YS*{Ed-$e zHPgLIp=?CGV{5K{_RJBtgvbwxa zU`{$>@4WMTCv@n?)aMDis6Ija-6Np{!5a{mx9qn;!>LPF{UKNOs>!pG_J4G+t5^wP zubUH>>En!S1m}FDx;x&sfx#Y)p=K)|jwvtN>5jTSL(Ea8J#I!~0z%7&^vaGF&s;NA7Iu7Oky7O9VI5D|wbLZSxl9%DYC4+|nCY1B z`SqeV!D1}7wZ%O;kS3qas)r7f%Jl@{(Fd~gH(;)+re1K`>aw2x=3%t^c<@J*)ts+` zxc5bXT?0%;DJ3WX@2w^Sd=A$a6zayTume9q3eLfRvM+XFbOnO2XBWsMz8&X^Q@2^7 zTjhd^b4q7ZdL@lTZqQLB<%qE*#oEFfy@#OTyEM2r zB^Zh>jiKPqe`=RxT3>J*RfN|yKu75n2U}qTyuz^Enb-&bVHJGfP#bve2NmcslXQB( zlQ5pUcMF?VtjCoP6jS)xJRK%E<}GKo%=K(__m3KujH4=dJnotsN68-C3mvx}R_uvX zqdUUn)YU);>n-WZ@@_qH2km|Rvi7zov82gt)xcYM$q~{-Jt=79Qy|;ZH_WseTUF+1 z3lo4l&>ktrS&Dk!AqM;FU&_(@vznZ34mlV0r|U`)N`b90%83q*wLYk2K>?pBmyIT4 zH|mU-*J6^i1WlffE(eqEuEjm%ZZ;31cYD6?XesV|68;v-%Xh<{=O&=HC~}eYN&l{< zuiX2wH?X(a%A``(csMh1IGux5vEyB{jx&o3Zul{)Eooc&Y+Hu;oZGFwXpl2wjK^@2 z)k5SrbV5B)Q`@Lbgo0zqn$+d&uMHu*QEal^d?p#CsdD0-e^Re+T$0#)<#Z`SIW&6D z?M7kUX+A+jZCi#tVl#5r!Xp8dkxWr*I!PG2*T*GoNz!9ExNm za>MOzuTQ!iUUUE;TJ8lENK^k(p?~RWsXF+_i}lLF3uJmF6LsKqAy$hq*^3PWKgCgn zfiOi~`_{cN-FelcTX~tcNk{+>e=wS7(kP$qU1WsPAd)*qsYGl8RwFnUEV9C>32Fa8 zWz%?CNbly3n%#g6Gb{Ar(lBjX3`xAS{ozJ^?4|GhVBb~LvoOz%wl&s-+P@TU=_dm0 z6dT|31))xho4=lZoNLk$MWO4rkt5e2_vSpEba2*0xd3<--F@FqbkH9=Da(1<3Kh@a zE66jlNk6R`6j#0v*_NZh2LbZ43fS5@QO|x`)JuEI=*9fEC*bZfi|0VU)y+@MMxPIN zSiAabIvj$gey~D9y*`$#o_TBy9G;JGC(uXW(7!nHPuG|Sl%`;jH|*dn>HLO3WgT-9<4M{I|m91*xpP766lf%o6}&C;9%1h%@F+KF3& z_GBNNc7G!f*rcs~HOR56$yKtJ{JbR0qj7%7_^FEOLMuZP6Y~~Ej%;e-rSiDZBfI#I zm8HR^;5M|iIG?xGt9g-zTFfq>B3Qq>G1BbN&;p)wa%l8g%z4*>w#-f0C^GRATYRin zY5DB#46bfJdeP%j*S3xFr50RDK1A|?Ot)K*h>jrmCS-*#^9=N&nC4K>YvaI)VEe@K z(epQ9skY@-p3m*hxvMT-!D_)B&0A{cDlH~+4Gv-q?#I5e@S7bMVh9swoV7Az?g$z? zK$aM8yDmQReq)B+nDM<>4;{5zQVDX|B(4=-#4it!2*!SM>hrz2V(T%>nHzJW0?qiG zGUV2d`=prc6JN{>pItwC?&k08)cnSkC#2G+Y1KmiHiZ1f)k2P=d2x zzTGG~3*Br6j+%9vh*2VFvToFR%gVzH`@dOuvn!K&r*HpZW|PT%AHt=pRzQ1u zNFhThxyx2lJipox8x4E@xVl-k9T8$H?tUbtg?b$3ssjy0PCC5rJM-{ey2 zX>2pBUn)pi&HMEX{Q%Em&}~ncN1R+jb3_ZPEqcGh+3x#Fy)+%?0bp@Izo3OedrF!f zWwTWH5S`jB8E|V`zFM9h>94MuFyPo7-d7PzdI7g?YiViOcE}YhesKkqRob$vb6^>Ftvm$ovpzv<{(Z>3PZVqWJG3Gvv@2!B*=b!la0Gq6Hy(^2FWee^Uf#XSEEU++UHU4bP{8cv;c28?;m~K_P8c!yUF~C&@(DL9WL4NTxi;I@MQP8bY?MT4d*Cw>MI+@Su8xRdig(4QX2+ z7{0>R~L}iI9{=1iMG4B#uPT7C|>VGzaeb@5K1G=dl@GD#}MwL$ChJ zO?bf)1(B5j=KpliWsyy??JSK&(N)HB>awy_L`aNd@UE(aM#n!uDV>=gW)}9usr&2d zO$|=3vd)@mQfxKv={LB%l=8};*3_&D^Ce2}IQ~GHn_q;q(bRMbE>ZOOHdSd$9oJ5X zQHwXKYbRAkDc;Ds-^@x7*5{M~g>D@=FJ)1?-WzkAgmpA7G) zx*CHU&)^?^nNN84n#OssqMZHkehz0&j!oEg+$QKAG^WK(&aPVTIiN@>bzv zsu02N&WX3$-=}>aB^i^9Dz#!QO6{Q>A8=fB=_2 zEf%}RB`MYdbKt)lum9B58HkerJ?E>A8tLz&`{!=N)5rzxA@`g|{K=F2FNuf63QW^6 zcJ!Yj@Sjg2X9w;HG<2^E``54j`Tfsve!T{8X%aI()$U{^~YG8^O_TwDK)}AOyXjCEl~)}vW&|1Pmhca{1V-zzS-#3 z0S7=dbbJ;_E`os^noXL|cNK>E<9mYNly*VGUzK&W@ka4fDa1{s2VUpxlBWYK<91*e ze2bw96aR@i{-xLN8#YYNdhT)7pMT;g9H^t7I+&P_EbqF`pDMa}9%{Q6R$ZAvB;JnR zE_AlTj1k({udwUi6uQ^LOxL-RWH`?HJX~ax4PL3z+<79n)tHy;xETz!97=D^$DA3+ zPL`ZpQj^WI#R#-Fk1=ze5M04 zTO&_~q5^fdghPvFyx~g1 zbC0}`V-rkPfyXM@NFbgNCoFHJj<>y{W|4iQ$enL9^s5=U6iw_6Nh zA`~4yAF?WYS(4@j?-QIMNgOTp`QFrA(bOntH5oDUBStCFev(@t*bW%em(Py#EdM6j zetu;0@~@uth6^C8b@au#j`sq=D|r^MW4DfuEQ`9m#R%~!fK`fC>qMg{5I}mm>6N)T z1j!2VTP0)Kmrx;{?fu%-VlQM?0wpqL!U%dHdX}k_bRJw(x_}Q|6u{i9-_GXOFHUxw zf#n}e*0lcU;clK>8eoO*_PFswY7}npGuhbwP!1rCg2`U-$EO6u|2Ky#brFB__up_4 zUH`>hRNl6K69(O3@mZ{UwKOCsRs=Fy`)`ZpF5~LZdI;o}X8* z+u(g;(V=}=wG$jk3GwSn@j&0xvq91k#I=>o8(lcX5K!@SzDxFdCUx3hzh!D-Q(P3Y|Z9Z|aQP-)kwMb}MvlxXxQ^yJ6aRb=L6*+XaR=H>h4<;%r$m z@z;`2oA)J|(fjR}*)=a7mUaEaJ`uGKUjN#2ze5(F8Ew_C`kpzpM}W_RJFsW388wyz zCH=Skb|@>nnjXAkuvHtosg+11eym!chLhbuQ&0@eKUi;U@@TT0^oUF5%OkmXUS?es z4RiWV2wfi6oATOjqCE0Vjgt+7azdxR34=Fp6}?dbHXXyAL44dl@Y_|a?q}ac8xxlo zu5*>)($|VFn>6ntw2#Q^;2-FH<#Ym7puI^#BO*rmKYsYpLxbqNYlh~7=>2k^=|*0Y zoaED#B^Ho8oY*)$Ehs^*LRJyajBWUJ4unM~%C12aOGS^Sm_7~AbAZ1>f?!YL#>w|Q zr=T_kV@fl3t|dCm4z!1RXhskhcv*ysLv1_z8=AA8niAeoO#`@Afhmqn zy4g}F6E%yz_dyT6vRGQ02sME75*iFi8?+fki+>79U-8HCuXXl=N)d{T5{j%{fp337 zk%Z$^*KfY+(FGy`RaC~t!}6j&ZRCJW>DotSQ~VNcEg}AprRc%yRt8(1JSM!vD|OV` zZPo=P=16#Tbr)o!5yHInqPIVDfCYBiIPM^41;_#9qZhJ!F&uOoa08Oq5Ty+wH$;*VqlG=y|KAb{~V|NecIM*dmc&)2j(>jzIpg3 zP=0)`)N%eX=sc3C(0R2Vj0~H|^l{N3PM+L;joK`er3>}4s+$10_3L}>24BE@aW|=3 zt#*q6g4u51W%TK+CYep37ljjfy_D5P3b-x`kWH(q8+k`Ht?w8?L}d?4K9o5*L{v0U zJD3*fwlxQ**~chUHHuhQo-noJU9Kw+U}JpBi+uN=u*X2m{xgm}6hv=xo#K8ae5$(xEW2VQuQR1;z{>$YwJ$O`Bx~Sxyr6= znd0`CdZ*$~p>@j+hv0Uu>}!HbQ7|e@DPAPYf2IUhYXPy~7Q09{a6y#5+h2PpPGyhX z2Dh&j-EuM>S`?dKFq`vSRD#=|*=2MQK+tQof!8;-BHl>@q)lDG`s|ojqo5r0pTzZd z#GCdy@`1`%k5MfwMu@%C*7r1PLDz};`V6$wR?k=s5cu^2nI9$_6|*u>wF%8auQ+n* zwwx62x-{p-S>KLxAKH-|6$iNULFcUrI#l+HQs9S|Jg zG=5Y91(bGWg{#3bE1Gsbh2~^#*fWwWrmn>{ea`oh{d~)O-_G z@^nRRk0fmzxy_4{+Vtr%%!qoQ7P08}+e=$D(9NhgI6n+ijtqQ=RKMY89 z|C{w;#sXkVyupKn&_8g?zu;ybIEKLNJ@J;}znLHLfQh1JPWfTwPiKSv8k)p0mTq9l z(@J2Tze~}dl9gc-L(^Q36}sc^pS{f{;?7YY!>!W|oCY9AcomV|6!>*fs_EEE;1heok_SC z_%dSNlZVt?p7NcVImg!JyWkp@`cFpri4Lj97)0?^c@Mo_zJ;ZQr^Yhk<-Zdd zC7}>A^;CgJYaq|%82{i0nyf-6L0{;pul7b#HRr>P>e0>TcH-K-0*+i*Jmi)!Rx-PP zghC*Wx@Dfh{|K%UP}xB^dc|%Mu5boObgQdAkbwgJvy!F(zq|pVVgw1zlT;UsLdSc_ zq87jV5dXr(gsRspqTy^`t)svPxkCfyc8aT1*Q3{(Y^I6Tb}0t8J3{hJ&DvOZi$CBK zv%LZ4`9xxc>_6uz!HmDgJyJ6KePq0C^US!y?qK2ve63{O;r@z;khVLPywhA@&QV`6 zvyFSi{;e{H1cQM!)e5rjAC3EV%ZKm%=TfBZVS$?TP6`lB$be#H6(G1U}V*7 ze;YYl86#1egznHGoL}y(-)^)j2Igl~tR3t72U2B`=MKvU5N6q7Sd>Uy-KxiLSaZ1z z1T0IxG2hpI1O5*_l&BzeA1)l^yqfntKE_k-&UhxDL%_(hI zWwAUSvfgU3-m~el>fT-0xNF(ZyqD_cP?#zErl_z3S82L!bG27t1;PtQGlNXTCcJ83 zJf4VN-MT>t|CpPDgZD7aIF$?_{OImA4;Tb6LRcKCU(N8y5w%fHOxL2j>NvvRs11Rto=MN z&LKRq$U(w@&7EKJw%KXpefOY8k}$lrGnGGi67b|DFkS5U9evYq415b}n${0?DE(;q z>pe$@k}>=2m0h_ADBRXe!L|6Cc_KDmVlg_YsGX?>&(|=_lvwuBwMOQI!Znf3&7-_s z_fxqFUcLtnb5|)UYx@e<|f;trq zbU<`@p_t(4YIXNU#7W3IY|K7^;H_L=zgkp$x7Rzd`tclY&G_J{rk8aMgbzk0{mdl~ zXp53^HfVgeWWGjh=nEZjmwnSOcC)3fEK;za$W@YEuXs`f)A!-ZIh4iOM0xufSuSkJ zFT>Q!f^zQls(UM)sHn}V+_`t{B3LwnP;=~aa`w4JL3qr11=lFL=#ALo@#$01n4LrY zbde53MZl_h3O(7t=s^?wC=vf^Rn_++gG!=WsGzPP>2oB%ziWKM`P8v=Y|zQBr=B1Z zTWX&?f2j3yQygp0c6R3w4LWMuA)wfX;i$1`&i6uJOqCQoWPf~j7T=t^Ai`v{qap*5 zCk7GJGZ?t$C3!M1XgXWbwTlp`-w?UTG2z&7ruFbmLv}8u9yZ^8k)6K7D97ZHg)(f1 zB#UCLW9eS;|ARt6{NXjuwKGedjtweekth6O={NTb_}1zYa47r=eEdo}3HZ%b2tyk_ zNq9Y+e^Q~u|4mT&X%O3Nb~(thswIKSQbc-LTy+YSe_?O*A1ZaMl&0U&q1e}o)SFr0Q&lIL6&}u-Oh-1P;aX^ z_hg&nlBdquyO3QKdbovk4J27vC3j?yxqMPN!4av=t;BbHgnF(c#!&F}RNr92T%49J z<94t`^r~{!y?aN*tquWG#FIXdNm3NU{bwkL?s=jF#i8jJBa%^^XzF39xdZKlPN63U zCl2o~#E!$zk%vspOjC_LBgRsyGWon2+h&`!Fc}7|)R!zg0v|6Oe*fU%w936s?$heX(v)zTKT!Y%0l!RxHtv4Mm~Gba`MO+H|sfZU5yH~3gV`IoXERa)9(J*e`?hTAK_&N7v) z9E&g_BNjdXBlKi$^;TG5k&sO*@(>>}_A^cX_291I^n4W|-EUaro0ke7-&HrZt6@Q zybSr<3jMvIDQ3&Xl5)&S@HVC0ki{Dz6-#qH;q^gaywxPOXkwU|H{H3hm*{?rjaZmHbZ zalh+w*CHp9w=$?Q*ru`YX_IX5A>T59^5{;Qb+w8x!e~Uc+NshKwrXQ&84^kR$`db* z5MJHmhE;w1B6pd1a(eQ(_WMmbJ2p_f+dsYhfrt0+UDy2#OTV_Zy~GiapD)U}1ia@! z26s5Ds&!~fqSvSm4%JY0(<0RJiBY5m)q~yTQg--^+#xR;W=K-rbe402BV5QwB{e3lxsER z!&%p(9~6T}Tbc}at-urwHxn(`;UF#r)=udM}RNA~SM5w7Wg& zZQ6=8o1Pbyi7cGHl9h@KQM1fS=ZZDjw_3_VK!Ch0PZLl@AIq5AC*>ZVS2Tz3S1;~x z0l04Q?k8F2*x}DN+L;|fB1cflGY^oC%C)gbdLg{dmdxChvEx78$D9isD**(D(J=oM z9)pbX#$>N)a~K6_*i#9JuMfI|uJ&IBW0ukL6nxM9!Gwc5HmA-j8I479Z*SEV+;7(> zsM%qj4tIWuNp{8Kr@m&pcdH*g+z8X;uA%`Vr2K&*H^sCIrv~N3deb6)10FO^SslI8 z{;#tD&bp7nPk&eEp8RqwRFpXlt+A{=;pK<6d=%nK{ zzI?7_hq?R5VH}t0b^WWf$vZ)3?MLOl?T5O}WKsNU?R<818C_;Vc+=L36M@p-_}tU| zJZpkfFuRi(1q)&$Ctev;)?SO`PG4<^;d-uShFP3L@N~nEQ2cEek4^NLt6Wd{N}EN( zgMCuw_!IhG&CPYIzjuF&l{HpR$*!U_qhpDF-tNk2@Vzai0sN-wmgeHFV!l*BZ}YfI zDi_jmGh($y{f)XII?`OFWM(~3wS3k4aQR^Uwww|ipJ!qfz}y3akU}hlPNl+KC%ZAi zkoBk)k&4#PKB5nFAjneApF!Ke!Fu{o{=(SH=keAp5d{gR8JC6am=U=2(=p4p@8snQ zr`QBO0PMWFH>^R|71!sN)_!k5OSRO9e#0tzhDx=wDe^n|`!-+hkZdiMQ|`I-XdkD` zOeR0k<6U4EPT87OD#LwYIiNka$U4RW>DtL$Ti||fJG^}|4Ne-_$t96{g|gp=OE=SdTG$y8Qg>YR;y)o zE4e{c>$dT-$Ll1*g{8Kl$MFWHTLDddX)5}gSxT~@k0MD>;dj#&48!wr<)^l+>mO+D zkcQvnPcP^>wX=kg?r%q;aO{zTKv{4I5`>`qN3W zIS2NPsF(Xp_78A|RTxO!@cDa}p+J^%x2;t*kK_ewdW=0to2~6;3~-(M@FKn_@Qyu9 zy@LbPD|e%^NV#5R-PWfH;o6SSV;QEqf|I%vXE%N?Mo}LkhNx z`er}bm0GBO)C@ z5-I`(bhb+|P68HGYY#r}+$|5kAfxIsvi}J&X|T~5j;3bK1B$u*<35b&#kA(~#%4$w zRWH&BDOJ)b8qdg_tL{*!l}Fo+G07X-d}ozh_BOv2hLiYJtzAaKuK(r-M-5JLWaX9D zi(|g7J22f2)SIlU7*blDD&X@xn1W8E^*|mV&C*Fx`b8>TL zjomhmlz{f9+>^c49zE`0F^#Ew+rv_>SHm6ZMK$nAoZOO<8{Nb{TO~yM1GY z_SMK?LKC6qlck(}y7j>Zp%`syJ}?ZVx=}CJX;hGb$UM|xjY02$Yp9)&1>XGfMK#Ti z0yQ^ncn+arhVvWnUMtq;dQVSJWtq^vCGfh@QW9*j?bfCy*9$c}##7YcvqdIKTnP$K z>HAFCTfJpozsZiV-EpRaxf;!{0NM9<0Sxgh9{dM*uBPdk{Mu$byPU%_ZjTc`K7UDH zllsKxXh(hz1*XyN^sXUA(H$GhYga1QG3KtyOglyZaqdoa$IZV?=7rM&^H3&F<>BZe zLJ>V`w2pMC_w?sGmY{0Ih_YO1XFlLxRSBFFTan-G~ zpV2}S(sfP7`&x*}RI9#m|2|WkO)9W09PTq zH(>ev_M@wB|5|4O{pRHVJ3P}kb>19*=`Xo#StRxj`;Z1o%s_*+{!iNa?=bZQ`(0zP znd)W(LQmAB-mkj3|9;tjeA2jiX79#~=gzI(WLE_Xt;n?-Crs$AT=#G8{m(Xye}uo^ zY&AByI_{^{R+GEza)VsVUubZiT~&4dKTODvbyP2%MA~~Nnf-XJc6HNx8dx>G$(RMs zu#HdEDpdJZN(#q6sQG-G=2ezSf5fB5ei)9(UJ@^Y?HZ*QYzzJ*qu|3!46^hq1@ z2MTL{+UwGoIw|{c?C53_lNO>r-E+{;!nG!*5+Hl(UtcshX_Ka_>>>1rEdvc1*41b7 zHW2Q^>fr<)_dr`2yu)fIvD6yN>?#o`O%<8=s^&J?5PH{xOdw}-rWfyh1BvYBNr_IQif(28P<3+mJES|%zUVbiZ z_QNp1&?1#9oE)2MFQ+^1^|xi2;2=a401*jWgMKPVqGtKg+_YfTln%0ud+WTUtE14h zw(vpW$tz0{kA}Mw+nXgkSzU?l-24Bl>A96mKP3gR3KrlpP;0mkxP|kOOWvxcPz^dD z15bZGZ*s{VQ268xK(@$UQBI>4%xv3KQwkLDmGt|c!KVyLj3I9O57;?w3y81ucVv2? zySxFMS;BZRk27p~^i5fIz4XvOS)g;*3$L_6YmXldj{{D~g8aJsoI?Q2J7fG!`R^>Q zxMr`3xJ{pGDXXzO=N+K`IuScizx#Y?vKn$JG#|BIZX+|b{>G*W*zDvQUA}#j7ZRyZ z=JDgLYHqN22h>7B-HU47d#}V}3?4qHgFlOQ6Bs5Ey!_?hs8`!(-qQoo+0%cr-|;6s zhel8zaGd~vp!@Pin*n0O_7T+N)Sa{0vs3d#52xZFgGTk&IQL!_rwOOF!{GSor(?1N zzT!g`w8!I~BNu{tNC+Tz=go=-Zo=x_txa`{6|>>SgTkl=7Fo^Pw$u{j*C={)YPvsD zj{EuaC{rS>z5J_p%FKdPK3`A%^ff|HQE*bDAd{T-Z9wS4^_v5J*$blmhQiRmIJNea ztGi}eUaFF{Ka_-Be}a+JOea433fS2`qC|nQeF$O2Js&76HlyOGGN1tb6)}-XWkJ7_ zsbP3RUoN)r<7_-i4qs4x0wB9y{#5O6SPbJQD}yv62IH{m`sKb%z~93qS@Fv)7%_Q? zKy;$mE=iYw^Z-F`jw?0~@;Ps6h!9+iJ$hv)S_3G~TnXp_+KS+spZ# znw6EkTCmCk*4yw~@H{;;Q(U2L2|zh3-YnrB8_-NSa8!IaTQunJa}kBjRGsX(!_q7V z@MXS$uJ7erZsPO#LQON@9y9ckFT-)H_I@|4zc6u?Ln)%kO<M} zsLOan11PoC58MHL5Gt0flOR?-OCCEkFCY$=2aRgN`ErCY+YO*JbYT6;{%%vsFtMd1 zoOoq)Q~gh2-m$3QUuq}OmdJ0I&KmiQpZNQ0Q8({$8Db;*=ak%mSe;v%a zFt+IROF?d)VM!BPCES)(C)3*kZ;x-P8Z~YRR8!jKsAm&_i`o6g{qlz4bI!Klc^&p) zs~$(AT4PJT)x9#k9$SSpNi(}gCVl07WrEY3HVXR+I?-9atdaxSz% zZ#MS)uN9`}^2DC6I~RWsD!y5K_h2fd4z7{Ze%O4YgoCi_#j#`` zAG;QV^jwZM8d|!X&OF*_mjDgx(Hq}ZqU9+0im4WOmwmZ_bX5&lH!s7~Fh=ACs|3<2 zge<>T=I0ND{WStAaV>ZlA$%adxWp#_90_iQ9^_q_(?ab{)zpsyQ2}a&)l9XAX zSM8QGQX*%5+Jh4;SpBQt^+cDqs6h=?(wC1Wac`i7bZAwv`;lwW2pPDHwu4ysJSE0_ zlxd7p_2FT)2F%*|c7_cO6*q#oUx#&4$y(AU7l z&Mov0ha5Re6zV(%SQaJ9z9=1j@WhI#3--M#T5hU7F_8_jjMk0*oE>z_&RF?6!>*@W z%$KIs*`w>^n>JcwOE`KlbS$;H&v8=i7gieEmqNhj^3@m&PP&Gk8ip3x8AST{PS_${ zb#C?X13K~~a~!ir@^)5P{BYSj4;Q~I2jqW2-ri3KhNmZ0oP3-QWDy%talAcXm7q~G zuqLXi{ht{*o!Q?sH8$fCBlkNQ!>z{oLmUC3t9c}lCK~QJXw_|JS+F$dAV0o4z$GZP z`ci)!Q7^i0o%4WNOeT7l z5OPlWedccxE?JunAMnWD4jL?dJ2GLv-;+3;1KgkLC`Kbg?`OAiZxGx=cg!{^(*_D1 zJ<64<0%wTf-1hVz==&ijKV7GXVejnD&io-ne`)-DfSqFf z6i7$r__@0sFe7*W=7qKVbp0fVQQ>Nu1|SAItTfPQfA~1^-pVg&U~4LRv+mO&ZQ<=D zFC`Q3f%uS_^Qm{!R12RC%))Q;Q1Q>)uKRhoj4c2mK?Vn(~5 zu@6DB=U(h{1*3>v9G=>Jk6l8Bh$4wb?}OFAsY0t?AfzjO@$(W8A20Jn_3P`E;se^e zH_MT#@w_gU8|60j&2O)TU*D)G7-pXOY~cS;F~+0OUeSI{o)ogQCg$WskK3N&wU15^(DvFKCYvxq=A#Glxo?2l; z-(5@aZl37djXSDrNIBI)^3MxbRNWk~>R7sF4%dmZZywho8m&b$&e4%i8=DFDPi46P zxt|zIR`-@rCaOxa2W`0rHxItH0@zl=6h_kd0U4vVcA6 zX#1*C<-EK$O}(E?w|uIC2mdih0xXIO6YC$McU*-A|ct&fm%6 z1((SinZ!1Yr`gnRxUA_mttomen!{OfsE{?w!MCz#t2FN--Pj(%I)m6{R|>tFgfzTi zF#*={0RmdP@~eTtDfg`Jq-87TvevoeyPN z{?#b)=OzT*EvQ*cYtw*i4jw@Kvpc6;mG;NqJ1B?#D zWhO+skw<}w8#TJ6(V#rJigf$Rv6pc|#Unp0Om+1<1!72cAJ+?F`@&zv)X&|ElOf%0 zh+$1cZDMWJv|DZhB%4EEd0q)C``O04$0vGj{24wNKewlsd{6FYH_X!Q8v6bQHWhoj z7d9|q4il)B7RF;Y;5l_=1e$2K z-pWKX*l{N_l6k!3lR0*5rpv5cy=ETWQoQP4MlzdGjLUIvbop>m(W}$sTU&U^IR?Em zO(AMN{`v2+PY0}3GDhXn(UAVEgunivF3V@!FCKU>Wja>z8ckl#Fft?A?dMy-AOQ!q zOdUNLqKi$v;*B4bJLDtOI9Y!4)0;<+Xh*t18+u76nohBFNgWnjF4j$un7N zy3cptV#76A&2jLvusZm}&X2LSoO3|j_w=y~KDha(O%JI)ON%vST(6>BX;ovLQ30C= z1(`S4 zg*vjA>F{|}%9))pYj8hkQa@UMd}blvJh81Mo?Tf~2;A!9n`36_Hn;@))K)7g68|U9 z{x1y5UGH%UGvz$-O#G=y%M^8@!baJv_@?BP`}`2Ux;71JXVWeA!^Zl2!s8RND(-B8RZ|n_=h2)7S8`< z&REFt|2lBVy`-MmboRb4NSg+WHw(AjQtj1_tJ2cvSuK74c42^&6}svlco?|ecak6; z4|v$vojpzwduA3ngx*S6=!<)Gwa~$Y^sMfEn$s2SO=33HuATJ#;6ae`aH<7S62C%%uHKB z8pUNcc`wf$ro~-bH$iD3V^Z1K4zsPqDBvXVy-n@vr~TVH)Jy3{LS8pKez#k{wB(}c zaX$Wfi=B@@3P0IYuX6Mjuyt=Zj|uY?vx>}kvwao2Y2EJZk;k2OB==F+#v!3z z0Y1bYtx7)vcfR6M+!%#bU3hKEkR(i`_n-;ee{$lwuZ-)&EXAy#c8Kg*$A!H z4hiNH+)%GnnSS|hMgS{VryhZ=z!7Yu_6fcZAxf zimH{bGL#&l*3>>^iC-)JVuoMrr0g8f-s-;LB^o)FC5d0KSYU~J!%i!@`*2LyE&OMo zZSNra;gmt|96OOoP2e_Q42=V;J2bR8?S#)y-ieDi;O#YHo2#z6$T7i>9y5AQRy z1|L7(V_Cv>$h&NK6NUN#5SFs?qDnKnQqSL`d`-Y zdKAB~ol>Qv<1JYIYT|?*dv(C~Ky5z$L#t|oJ8OvuEJ^3A3)imkgMyu`>i1yQSFcy6 zSmM;y8ogQf$9dxN;ir$_hLC*?o`SN~LnDPBmpLZ1T5hmg1bpwb;oLCWVHb}nAE@2w zW}73L^>WaqUVw7Rr>2b<(Mw-I@GS(G-N%2FRr1yCpBADKK1P8_Y)@{K!7Z+pUCn!| zo7~dB*Mqkl0P>m#3$@xJ!Hn7%Bd-u8vYGXe(^?vpo0Yz1cqt(7= zuf5c_T_*MXT>zajyJLUXhO;ZeaVtgtClO$}qqGZucLJDUv|XC>d<*yKu}x)QaCg5} zz;`sD$71%jb@|KYu^I9~m~g4^1|z|mUiP*Pi)&R}J)2PoMLi>-cJ{x#$#kYQQ@UWE zZlRLu^hXbeTjo;l{hr`y35;H6t1G;8MwTzC3Kz$sReoM=>}O>$^zPNdtIl`L(&|cr z8;Xs(O3w$0MYmDev?Ie!T7O(%e`LGG*&8DxzD}Ur`P?<%YMK*jra#79;JwAJ#}~{n z{!$s22+S37ETUWrussiUEPFDr_I#A_=&l?~WF6Igy9b4@G-})!bFP2P!Z?R{Y^&q* zyC7&JpR!@w56@*jelmjns%=mE3{*D+2(n*#If|0stQ8GNN{eiD)`3bTkQR7JY zQOn+!e*L&NO_f^{lrAqfldnw_&vWXS^KqLZ=}K&$WHZ##ig_?d1=(-ZD&xb4sL6M8 zR_(J4YXhY+KezFsBWAf9%A6-7V^Smetl<2>h9>86j8j|epP>AxbIs?P{q9GCkL6a&E#|^_;%46S`G4Q1~;!^w*T$(5Li`-Muw{F``j{O~GuZ z?`2ue^R+^iZZ>e4wR{aawfmL;rY`*dICSveF;L;VUt2Ge88c!=g^3*pg$u9a zjxy`+u@{)<);xN9yW&W&6YuZIEc{k&S9qcrruFHb04=l|Q4Ug-J+H4^Oz0;yU89)1 zrHWehIHeQM&h&`omj!1nw8yo?#XkA_@=@tt(L!46X?j=vABaw%6W!~(BAzBNdT*}Q za>qi5MyGbvIl4qo^3mQyxy{m5hDlgT`RJ2I!`sHbfL_$O=Z(D8O^EXOx_8Z=(J4l& zMW3GhG(g2@VXW3h_~_*b{f#XPigC6g4uy9OePP8tthigf6Krm`c2)3cdgHox_|?A&u#c$D^5Ksb`9$k|eSP zETLC?Qw*7JKKs!3bT5?cFv!)uH$CoXO3RV3{p$Lk(t*M+CnWU3BMcKY{4YJy;n~`p zz{jbdo~>Gavo5m_OF*@|iELC%$$SqKtQmo0e;n&|xo-RMP*}YDmAB4wn>8=diU_uC zn4T1S{x!Asg~T&m4qH!q72{ZLIn)LhEiBScLp~Gr2d2K%; zv$XGZx@890fMLth-!>TR+SOy);CwqtsmlL!y* zU0nU)htm<*tkEzzZ#UgaA|;Cd45{S(U5+&PkI~ogghuJPlY+N3#~hsqiANqKnUdG4 zE(BE!nnxa>DR2WmG^lC2r{sO&!bg(J`TZY}9-BWd^zy=zg=&izrNqi}o}YW;A%ISP zEjs#nc`{LJ&MK;y<9#C#hj;A$fGrCBgR6`>M$Zk-9!nOznDnCOOXy1{;oct6@^fm> zDSt2c#L@KAL_IEOVww?T+X%@x|*73dSfL;SV>Gn3mywTY5ul2qUR-^%@ z)%uif{i#>mSTM>`TZyGYlinGc=sNwuD3#XrQo7HBrrK-9jtO>&c=laB&6A@p9499h3yIgw)!%dSg78A1v)qDCzxcLq`D3<>OBe9&>Qr zv-=CHv6t0k2)5<>PBrkIP?BZQbQ~+Yum~kcewr*MPtaLbG`WLL@=%u^z7lq1K;^c5 z{f8Eb!rGJ71Jy0E98ueserozSYL4IAI!(V~A=Gi}2diK^BMr7|I{V0%saGVfguTaN zNU*lrAV=g`;a#gO8oHBp&ue4f;k%%Kz~q;FA>Qi^UnZyG#LhBly|1h;zNM9>UsY$J zKl8@Vgc^uMhS@(unEKpIx6wLtDa_)c^~Hvp#n+1jLe(+Ytl0iL*jz=L+XxXePY-z@ zGHxphJy{s)kRr`0qDKZ>6`MTM!|2v0iFz2__7L|7gw+S*kG?pE4cOeV{~F?*IH*Uq zPdvOz?3fXG4(gjuUs~+%imf)E{^5eT)-YdEAry0y1gj&?ORS~aZ#%=C^-Uq!I)WCP zYPxei&U6JgZW6z&%rb_2-A_Uw`LuDnC6R zRoWkI#s7SI-Avu;5v}}<*8lSMGZ&|tPk@5-4p08Du;Mpnl~mVsDU*kj}%ofOqc|0371i9@4rF}w){U0_+M}O{oVhc|BHP6 zhvREAz-O}?b<_F2j!ja-uT}1#(67U-L2-W&>!=pN=X!=KSg*C0^8M}4F@jP%Tx;cc zpZKnehSC4={3G=f%5?>1pz@RQr5}98Yr5r~;Q^lA=j^0~Bhr>A{k-@V){&fJQA}dr zH&iSKTT7VOOAijfmDBV%G_la|1;vm&`yIhm~;g8CsO`&g+qeiI|tgc1O#~a zw`HHq|A1La^}M|wdp>stC`&~|!0jIQh*fMzLEz+dVpOfy=@!1}CU9N7&5Fjl4S{HL z7wYmJ#Ul=Vh!bAG*&JwZuPxd4;~_yp1-dw{dV-@qvteqy)VsI%I*U$6ju|+tuSHDt z@>Tv{Q(#`r7X42f6Gakvj>!#V+U05au%_NTjqM9L_;!LiS2n#Vr%F8toCJ#~Dz%%! zcUw9N zVhPzs((Ryd2e&v-DN3iTkHl%N&MIG8i=y9EKq+VbPmA~1{+ykdn>~X{g~an+px)f3 zAV@*uaA|y_KWTf&y!xrC^TTD3sT+M7_?M3@4E+trITfefZi0MZ^6hrgrPtE+S{%WH zvpvVyfACQ3 z;acLPY@S9q586U7z}-$;-k8*&3R51p9)UEJZk`L%mwLzsbjBlL*#Yv3H1U{ziGaP> zn{;sFXLgP`-9wP@?MagS^wy2W;`(l7L8`)FWU`Kn|2dT#bM^R-F9%n`4@ueoA$L>! zCxC|Po=I3<28zFi8Mz^3w$nOd>&0R902#t}p|;bZTEdGzuzM%6WyD{cY*oepA{4x{ zp(=0C9b2nT3n|_Fe9JIMke_mnT!v{>$M+=)HW&Y36xGg^zee#~lu=MtT@()*0=%sxmT#+~T8l;&(GfM}i&WfAJ7ii4+R{T@WqnK{I8+^SMn{e@w z!S}s#%IMjQ4zsdDj$NImgTm5?gxX0XF(35Ak;$sQOCK0QOv(P1WjPZHsu-Hi)5{pz zwudN zhPYg8OUNOj>S$8E*&g#c-)h(J71uXq;r;opf$4~~(j%T&A|%^5#|p=n++**DE_Wg= z`}gbW;t}3lJMRq08)sF7jBue!k|B{npZ^}5s7Ki@T_u@?%U!bGEDO3>Av7_tYbWKC zV>O~RJt=35`jljfaxI%TQqOPv+^jZtcE=v zLV?4mvTKhDvV!ymy3~O987TZCCC3E1SPS3DYdZAnXHW4GvS<|qZ?;i}XfE_4E(7N= z!>tiy-?#A8-96=nO<(e?#P2fStiiOGn}gtYXIe56ll$8|S_4Qh4>nKt8l@f21A*U2 zdEn+6fKMlp#~!u#52X?axYqfm8TOFc*mSO;x%AABzDD}h5)>P596-i#l zc1i$qp0rOwEq00m(>*^K zVdZ3eW?!^MPmDWb*GQ~w;drhH$ih2kp(NmxIE3x?H;my{{AcA(*7DNzQaf=O;~P<*1j6^(QxnN6(tlVvtnpt)3$W(K*+^RhEz(Le}&My%c_EUI7uBjo(ok%tSf-` z>-kC;PaG5H!y5k(pegpC(zy}A??CEK2z%8Z&k9c*A+kbh1m&`J#+@zO4do|&y?S7# zSW8SP%|lvoYto8#$=aHr3ZL%YMjYVIb23xEQG;Qo0JvoMRGFVh4oSAJ|wTUJ{mg&iizmH9fa9g&>IobEZYt zFbYtoR`sGPYAPqF?-}ssW>kYCxq)gPP8B3st3VN6kD$vTx^rN~8gZ{rF4lDBdD)^c zK`9MKym0Bp`2XqNe}#ANBv007Hq|O$@O%z~+DVFeeb|dv<>pHQ$$j7MQ$>3spQg+l zeGio`Yi3ZOM?xb8%J&!#@XuNFaeMH3VY7h(H!w1UoJyFmL>LJGz8 zFke-fZaT2fuhfNVr&kw9b-54wam?U55=iAyF^|5IOk*5=nG0`8(~W|GWxw^edGZ6K zNy0+ib}8lX>o4`fDu6cbG{833^t0*L$F=!7&5eXaL~JL=lIAL&wzZ{@KTJWXQ(#;u zlXZl>DAFsUKrt;t?nh%S-5R&3${cbLH@B_{<(&OzSaSRX{@!;-|7^LcMqBtAZ$3!w zF@j9Aw}MkT!WMXR9rrQT*3=DjTZ1)K1t z_g(CzK)EhP@5Q(CYl-av+uf!0@rRK9#^n08XPB9fw9g%4!s)<#?DBScED3CsI6P1t zhJ?kd;thv7Litcj`i_e7VtwAgv0n@eBl)hYS>(J)jVbrbQhPRHNWgCTtRTD;+FG0D z!v+rwIQ)^&VRt{GYPJAvt`Pj zbROn`z{q2DO-EE+fMM-}{b&B89<01_{f%MUM@#1ca}B)DvR~t{zMj%J^m%#8czFc) zMnvO*z9oYH$X-|>ARpDr^+*+FZACTDk{`8Ycqb=9&m<6Zy>0cQXqX0JE=UL~uzNkP zw@qK0S2e`=DL>ia19J;c+TVdYsteOxoC^9YKsuTQYtG$tra}6w&%^PQx`L6KMoeqW zo5P?R_CTLL9I;_zT~WD;Jl4Fu@4aW^2Q8iVZ_=C2D>!&;SvJxNbZ(!S?QE*=yc-zi zgT9Ahc-pf8I&dLX5zt+I{@LfkhHKuxBzO^IT;?&0H!r%U{6i5f^$UT8!7U@bzGrJ+ zYbf8wkO?!{&lN=!{Rd)^-+!b*7=&9<_7_k_YKjdJI%Z{%C_amu7Jzt)st$^y9hIKsXLgqq{j zb>!^q-o5${o-8d&TKhtlf*^D~AeIa?_h_P9C9yqp{*)cA+cUjcKI4`44-S=|L(ure zPRciZ*aBdMh3)F|NeYGzatzq$>D`L{qW%hHlIKNQ3cZsLAGot1g}wHUXAgQdp+O%V z2(|@}nFbw0IBNr{?bH+%*3wIeOOm>P!tm}+fx({SYe`+LO|XrXyYf}QBouaT=>@RG zVmZ@?rDqotXxt@K;q^V(y$WA3kX}bSL`K@=xV)#~ZXN0q6h@%;_aCYqJI1Y`dtcLh zqG#X9s|VuNY41G@%~uh513yGg5F9)K>Av0GEk z=v-H`m+q_V0fBVJ1$5_bhGv!O&4B|ZC(C>y&*${=>^?79dFZt)vz(z^i)Ft54?wR* z^6@7Zq~i(UcLOKn+-ZlUj0qjf%)^ju^DIlX6ebeW%Doh7)4Oa|NkY3pnD^<@ z^gBN>5uJNW9;XYMV2u6s^bBAvmKB7_!%FFU7B5+_qCGE{>ro*9biPk#*z&UwKeY3< zVVWnlYpF!nvE8$2J08q^x;E$9cl&Z`*TlZ1*^~LgAhaRCl9bl3MlUjOo|A@VXpcPA zLHa~8;%!t<&pt%if`F(c7(ks?A98Kmcyq8ukbcJJk{T%@JYai$X|;4CF2^!InJGi> zD8%MfEnjChXAEvmb{Fq(+S;-JCkMW7>L@>@Pz8@=5su6Y7u`4YbM<>}inl89ZiS~J zZwDIu15nJoPUp+Hx5ym5?|VC~5uaS)rAK*DqH;e>F?y=mh5N-8d z#-V#qtx^hbDJ8;{cuS}V1M5Wsy`&d5-2BNM^zNfr+C{Is09gx76r zX1yKbvh173WcAH&Zh9x*d1mQTJK*p$ac|f)Bspfc!g&TvH&MJ#XBLICndr{8yhUc0 zvI~0VNY+LJ$zEvd02+^dPdBw{bL4G&KzAB`4k9MR_izR;^+rY?95ER|)_0I8b7flZ zxfSBKJ-Y;l&$X&moO<2M04~-%;5_MS0|2pkSmwI_G2OH50CEb z>o%tAQLRTK5sPxoCICL=L$PQvYl%JdFSfxHq7pp3E6$UH5gD3 z6yajd(CN}i?i!G3RhL5#5Pq29#6!DVzlybU;Moj~%I&2R;jK!jKB0U1)cpE_mTXH7 z!qc|A46JV6#Jvu&oaITU8+JfsS=&t{D*HaC&9fwWoa^f!SgE^FZ3a@$ofVi?ofc8# z5ApSTvVG|MZR?jO#6Mj}soKxmd|jVNZ+?`%Y!9&~EctST4GLJ>f*etuBr7q@Jfd?L zYB_Aj<%ABET$FM7wk525dvkNk84UlYTjV*9`0!Heant!`$mKRq{`#c893j4I$ILLZ z^?u(bY0ovrB62#XePYyeXRi4c+`P6w2*{yA3kh?LMgYd#{M>?_v>*ks&6(*( z(I6|sd6YHEzP8g2hq?OR#%11!h!?=LmYueZP_&^3KF?-&L7RnpEwlzc7A8oV_=}#BTmijz`ypmcepR}0Etj;9 znEw%Ii9@=Pbs&*Oo~B#+x`#^^Oe?6cAUcnpNyYHkeT|qyGti^^jIyyBAsrzj!x^$A z2g=TR#cd!PU(0&^f2O%nEjNuZ5$0#S1EaX!cJjA_7<|zxlXJ)J3EAP?Y$I~7RUQAiYeVsbpKamr$;%f#UuQFnCj|MG%SDAEwA{Mtwrz>VsLu>3 zSG;XO4K#Y=wz?20#MjvGfhk0pdXqGLxWClYUrP>@cQpU{Y_s_FX-Rt0gL=}pK9TPM z4abjsy>ip5yE%Vw&%hcF4vP2;Y2 zvPra63jqQe5DW>3f=~hpvILAFD5WDZT9!n@8cU0X@8*-cR5Az4zVwx%YnVJ?9Pm)H7JN4^m(*Le~?Z^OMzxNgtT? zOA*b&T!wZl3u_1zdB+BRe_hi(IQOVE!lZO`8|wE=C)xLM^SHhqpyD`F?eVrS)$$JM zeq?R0eerDXGg)(U9k`plQ!)+)O+XN0DZ^<27iX$C7oag z=&mcYGa*e}iJLeKQB_XEm~(@qgh#XZGKs(S50b9s;g1!g9R~d%bn_fQ2B2CCu)zh1 zqszy8PW_Y&)J^kAhI2J}&`pF^P)%g5H6`)_6J|I<9SQ8$w2kgUgfh2xgW1v?}fbp zeZD?>_Xf>UsHqemO3WxtGx_zGHXrl6U55z<%kG@s;m8Qkg zDz5m0xTi%h_yLnH<}1eM963yhnpp%}06dc^hvEq8@KZWd%VAWR0JQ1Ea`<>2SsoxjT!5xt$d%1#L@6 znJj@xha8FZck^Ivq7HmdcK+6Y1>q)Zpy zL2BAavj7eO>mw^edXmb62>>z|BpJdbQojKTJDT9lqGz|8tX?Lf_zkzl0Gf?p(=AKg zZv2_q!6_gL@ImOF52)uf8Lg2#38=-781~#I_Lwqp8>R>7>aO?AS1y3G4=8MRZqJCz zA2OQ#TWEV)<7-7)1r^s36*oLLXKc+jZra|tHJn9!eBEMS(mU16<( zPYr`0kVP;Wae7t;d4K8_IHHOsyq0>R3mIBP6%yX2H8rF|shPV8*pUf-;rEO>lLxCj zppmv&hHi<8^~gpZ7J!867G2DDeO~3-Ud>mRal@hG5od(-rZ1WMo|= zJ(@DA@PWbYG{wbemT2e>--TI{{)DSOR3YxO0vf2lNM;ubYYpxRDb1>AQDbUWdP-L< z3k`NOLmAx~=$Ke_{2Gc4c{I{yTMx(w0^|S$cDBRJBlnG(?@tl<-ee3fKRBaYm^>?w z)(^zIqcdnTtshiTmRK>q9`?mqr9%Ba%!5rdPqzmPbjqksPJSPx3(^MlD~n)A@$h^I zt;9jlU)OSS#nxV3%ah3Dbc6zBv&nU%hR&ad=mM%rg|k_S2-Z`RQ&R=b1%_OkV`;GXs2vcU1OLHq}>BuMh_AnA&S@Gpy z9|uu^93EL!Ss6k+rnBY$$c|)nkOX&>NhDlilK!mickN1(EAnrY_cY5amFuJ-Z;E5X zfPL-eFzo-gt3rjO6`(Lx0GM5JAyTiOr%Hkc`l&y_|9mXP|_i@?W&_CQ!L zCG!%$eb>Y8VGzlp0qNZ`?tmK8VhVA-jMrYISIf7s?;WD~PwP|3^I)tOz>E(Q$tcLd zeivD;9Z!h{5l$|ATRT>wUf9J}b=D4L&|^Z8e9b5}X{%(QIOReHSO#qya6OPIAKEz1 zWw^!Moi2^#aTd>pEVRVR+g0b95~d%-RG}J{!jT1%u{|S~l{%ZOX1B|%vzY?1BQc$G zQfzY9ho|@<<~dBFJO$n^b~tomYYRsOaFCi64$`G389@mrwafbr07RFe>zF#j_!gJR zl<$Sa3>PU5{+;*7Gt}b8`T8<8`ROi$u5;|fz^Y_@b7(`o>He_i>_q|SK$Jta!7__f z)5&dH%BfXPx$oXuBUB7#@teuEQ|3ejW#@jFI9lomAw?lVMY-iS8(?|1!iZKU0jVfe zKN!N4JP+sN_&SQg4Gt}*)%if7lB|9m-NjJ*ipM-JDm`8Pm}9dMi*m<%`Phk6uE`Wa=_;vg%>up3XLX9{veqI9D43s9_7k0>WeZLaaVlq z?-S0?)m#N43M5aFUp?agJWXbu{ObnF^lk<^m_LzaonqQ)8gG`_Sfkdi3hSRQ8?q z8iDPbzaACk{l@rihzFshe}^K%##xh^_y6VRydKc`c0Ho#7tM~J&Mr@6oWC5cc(1bW zBWavu%7RAf!?6S}8+NV;x2?DMo$$jO0y>>;)H(km{$~0558^jvz4{t*D^AJ#JHrc| zublHrC(`P}u*vq?c+scZSB+Tz<4RP>6Zsirm8zG@s(w=eR)DMq8^;q_lG$%+)l0AL z)uMjDv4GgxnFQQ|TKDu1^M{|8L2kbbXyf!w#sNWwAFUenz5~9a|0BK{)-hKE>F}Q@ zUD~&@-ZByfoHf72>zwS%IYIv3w0*1Usow@0$Jw{>f-Kq2+2VZDzd^ii zIf&A7alZc9yBnCrduMU^Br=C$=m_VQRgOY19VJnadl z6BrDJ7FPIO9fLU{fWc6A9XkSAxXygh0{@|MQJ24o$!H|L3Bp3I!|@iT$XY->-;=|DoFa|}gRvf_hxabh z&?KHjcCtp-g&zL`-YB1@#$Ze~-j@(uJrxzen55gssW2E<8%1QEs5~P)Y< znF5#K?wPDT1q@Z;rtir7;X z(kT~rEMvRv^joT=BQ5SAIIg}+DN7z1kg>AOnLQXOxf?18sLF7K;$G)_`A8 zLrIYNnFGd6ewkJ|J@yddN)uKMoiVM9d(N1nP)gkS;TTaZW$FB*`Ka}_W_tB@3EVwp zwmAijz@-DeT}E}Io4SOO-2uYp*2>#u{@j+JwEHP95-txsSRVF4n?USn`v-VsiQ~N&!l%s<+M4BNT?H-Rtk-9 zw}MAFN-F}#dEjgn=PiuG{w7q=v8;y6NI7#;x-am$w;+j5eW80Xq$+080b6;8K%s=+ z-kcA-Anm70i5!m^nXt)b>AC!OGcoS8}FlrPa7+pJM^~X!BkLH+~je1B2D;($D|il?hNfM z?-y^o`_cp5R29yZy_4b0SqQyU$Dk1qY|HYg=fQHgX-C!Q*a?h(9HaIb4CXlp+%jLg zMz7A)&1TR-pi7xe#BY`DF2HrkO>DW6qT1Wm=lrj8l^2a- zEzIH!tr;^nxD}l5AxY{JkEiTs!%eeWvhGyvUbH;4xSo3E{4ioMhpB$ik*+f|K6VOB zuJ8nqACUq@WCX%r8Go@l5ZG3_FP3t8w-%mo%kwURwxP{YjJ*s`%1ciI{iImcR;g{o z$Z}HkYCV4M0!a+Mrt0jJ<}iG>((m3WQ|}f9${w`Mv3V=aGB9-|byK5xxIA7ls&T9n z8p-k9yI^UyAaV4YTa4Dc2aLfa96gwG^EtTq?cQ!Zq@fXS>Jc&Nf(_|3R&h3nh?Oz7 zNA}@T;(iBp?(SpAgKN|8>iw#H);}1*@J8W+fwiNs^uUjuH`phS*aNOhmvEJGll&(7 zfTSWZ@q}x`#&Lqmo$l}p6=!c`J*7#2^5M$F?r*Knrpwfhq9UZGU!N-6?jd<LY`F0o2`|Kk1j6MM}*&*Y3(>Vcy6x@n|M4#$!6T&2$R znbc2)g+I&bvGH^qm?;`6WgF{bqxbT#8VbME_Axw;eQaAG&R8XC@H_@Xuuuh7cINvg zT-E+#yM(x7{d$*{o>-A4zSyX-U?C%u;_=x@ed6>+2gboq30Xxl?<_N}_Tb}mKbvm{ zOb*}u?0AFryUQI!OxJzkg(zn!J4vLLyelbWX#4)nysdN62PGTD?)i2szlHU7CetU$Q$3*fKxzuNH zMC~^LU;SO4%C4SyzZmk%@Z;fL=KRkOD61+*StX`ytviSWwRVPoY-s87Dw85dGAh;sb3bwetro{AELrcy%gtW(b=_jaZ&l4?Y--S%j2h3yD#?d`Cz4V zT2#)Zr%B8v3-7FVf9T4qqg}$s#abkVmv4nCP#(SclvZm9`^F3QG%^!+ge=<^#iI;9 zo0qe8_X(w>HO6m?#Vx{7!Miatl4XJdBp6Ho47fPpc+<0pl2Iv@{4H~7yQeg9!9zsJ zSaLta1&LiDjnanpKz$-Zt?XZW=7|;~;iWY9lFu+-dmHDZ0@O2Jd3v7GZ&YVm5I%f( z9!TTxOe6l-F86tB%(+}(#pdYdE#^od*Rh8-H zJbsc5Y!sIwb+wK|j60>|lT*9f#Xoa$#-h_V1`%>iz`Ur6vxQ>Ys}ej6_nHbq=2xc3U9U8Zw!dboQ1^rNHf&AKI#kJ`!KTEwN;?b^lm_dDJWD~ zuq@JVM6YWJ{MESqXUSyW)DPQT8Jdp|yfi*BYE&{XSlLkfIPDe(uKzw+PjxkSBm&xx zBG=6<>LcV2GcgRwP4jsTn5q{VL<23Baw*c_%w!BSqOe{m{FASyeG8jMvqIDO>(Qg% zli-D>e2=71C(q0{XZg)~gj>-je4%w-f|)2N*<}i#N~_#n=2ISx0LB?UYPUEyv4vLR z&V$z#fn2FHt!_F9180=r)1??SmgTT;qw_om{*Lj`QLDc4x=zm6b{1%r`7(yEN+ z?tI2t$69B{MtI!eeGXu?!LwR|Z{QbTIULLu5>`E{1n(6enq^bp-Hl#Fd)~piNM~LG zJ#)Vwz?w(a_Q-NU;d+fBJS={5ub77@S#@Z(`Z}o+OFX+?zt-9M8d}6TyDMHi5E*}_ zZRd#C7{cV~l@g!JDT#zJ&TgMutLxR(Y$a*F($VBLX!iJFlnvR+O7X^)M@tK**VeOM zl2gNYT=Yk8;uPFr5Rpu^!1DXZU@u z-P~_k)d*0`tEi>oe(yl*7TFVI$7JK9glEIW$`(4C@10s%NDSdYL4gA0iX%xbPR0PBKQO(C{2A z8ixPE9%9kp+-(C>GQdDhicy2iKl{rTLQ&D=u14L(N4k17I*0{Z;=zt!jL=($-3q){q5YQjLv_Z)!V^EK@qblqXg~CsV8~tl)^sxo zzIImd=P3L;mz9Ff$h+#TjnFpsYY-d`nlUub_45lAmUaUsJrwg3O#8GSqvtqavbe|c z=N=EO-)+Ug&Q`U{+Fa+Z8XUZ-G3mS5c#5Gb&Y}^T6xrGP9cg~@*Eh=YsK1i&*xC*G z_{%|On&Q?;&{S7HeG@d)$lCk{R+o_LK-U-n*(gy$AaG#F^i`c&+3x5PpVXTDIlq-Y z>giIh+qh79VoaDtKL5`h*|iB-$@W~*K|&$jTgMJ`$N%Av_Dr26GgX90%)H|IlRsH; z>E_?~;uI((DC79KgEoG~7KPT&L0m*{PRm~GL6iTD<#kuX3ePK~NhFiw_n)7d*um2_ zZ8{aNBgrS>?z5~1ttM~nBT_yGSs~tHq_Uq!5DZOG4-^$owu8GFP91bg;B{BbPvk%N z_rLXjH4WJuBym{p$MT?vu)|SKdCMMRo*8gW=If;s$c<4G=mgDw2FC5oLbr&7~tWbWXTwpQUWzm^jB5 z7Fv%nbbicG4+!oDpmZ-|i&gsWdvsOeR8CU|1d`yrq-U#Dkw8ChI1~!!wKPE^r>|fx zADFG0+~vE-h-lvM2r)C(g6HBRJ!>bk*l5K5>RpW5Wivn|*gz;Fp842e8fIn)ufT{& zQ2PX08B5(2@-L8F!AhBqg^LO8T|D#8{$9z-P=(s%gta=3(hH%*35QFgcZL^c3{-#!G;fX0NXHn(yx6=6JKRRCs z1tAqTFPrp=7jN+n!b2lKBcImV}C;RCy&y2MRT5uCZA2j_$Q-haV@Z0h1QCxTpV(o?na}PTNQE%T`XVrP;5l;BDF_t~ zj4bg$w!y#D`DPx^9k0 zMydb)`lW?$QRowRxW{H7lCv~J&(|2K&+seZ+SEhxRN^LWVczs&0ECJmkGuPh01F`S zFdDQ2Y|2m@ezL{^Old`IQlWf+GVV6&6Dux}A(2*{-H+#(*%ez&$s z;%s>^8E!_ZfEE?6XP1s(lz1}t9^L~DA;e^sCR>qv-|2XxGN;ZiD>tbdQMy7`M z5VB#H-!Gg{-zAbA5rjcjAY)ii3z!c7H*G@HCXZ283hI#x0n#0~8%J54`RHeaNFY0z zdV77)0Hk}_0xXA$+ZmxM&NG-=ji6sKx##UT`mm%4ctO$G1vx4|H9}CN;R5@_{>(H5 z6CmbdhRKyIAns@ZrSmuaeF@#_+hJ<~M={^LJaeV`v8O71UQ>gi+vow;CL0nd(SFpK z+$$BJpa1D5mV^s|RAQ3hgaLZFRuly49JiLF5~KMtnppVfEt<~a;&CcQAt1 zO(vY|DQaiW%RD(x6FpI=a#bLN>nE+Ut7eytKnv!tJ_>HUe>LpOj~)MFDtn49<;A6d z@2Cy`!TmR(-7C)-Fs_NWfO-CsAHtb^`UarSh_N*d{}rW$!MG(qJ%B`*PT)okTGW*O zX=;Fw96Uw;y-w;GfOr^r>?Q_N6-brfsP`C9G3m8i-JdJeg@Uu9=>8FmHyw3N*d5@&FTnkV?1%gpO)+ny?|=c(r}O1Mr;>#>@z2G! zH&;_(Omu`=E@3dw^4|Zo3SaJn6asi;dHxq?`fsmU&6g5AnMEhfi-$2IwHSWJGho8E zs~!>OdT`tQIsI%vu2B1)(8|eyzfL+XmDw@;W2)si%$mM-v{Ok)jAnxs-JQ27bg9C@ z2Cprg85+H8C8G=+dg93Y$2ri=``LrBn3E&wRXfI61K|v-R@u`}j>i z>zSKY@>5eupC3{n41-aVVDWfZ%!QY1e9=0jIY#pBBCxdj$sRLRs1Rvn!&UcPs`G{S zloUR!^P`J2Bs_e$KMt5z`V7lhW}Iqfgooth0*>OONN)-)%M%XxqmhB*U@kM?f@C zrL@F*JS^L$53pKju>3iF{<&3GhPUN1rk)eo21mfQfQS9I$A9Q%=IZu$H;K}pYfyqXL7ye)z=2+*N|KWXEe=Uj5_Bglv0IzqIX3zJ-Vnn1X?@8X^q z3B+q~&W6Y~az4u94&8u?GUX*Cn!NH{oSno2FR0|(rMb0cb>n>yA|g5T!}9^(da0xY z2i{Cbyzut)m(B=@dem4&vQbJopK`jYCQV4CDAo7R-k$e;zf@R?B z<0pM&E%v#C#B)W(wjXmbLb!-KxCoX2w@7H6Yg2yJ%bdC)rE<_p$X9*WVF-lVelwKZ z&x7Jsu{tp^b_dq(4;J^sZw&F9xh!z(lU~%SsA&1m2`Z_ zUi1p73ogb{x}glNzH6lPE{_dMxZ6f&zZcTi95FY}tQwGD^;;a5jR{*Zy=LgQ3#m`^D0C|9`zshMggk=zj%u2UN!N2j7134q0V%N7l*ZV?K$~9 z7h(XBZpEZ}7j=TFq>SZoQ0ui!FNf8ghh2*yw3m;rI>$FNMp`J-HR@|DG%eWMBaz9% zybfU&BT)3+{kRB=vG6~dESxUd-ZXz(rxpYTp&f;p!_0oE-_$U=nIopvcxPcMvP2Qu30LM)P=5 z^2nr_burq|G(u_X!(x?&STaaP>ens{Ar&C+Mb-h#%9>lZY6aD)%}nlMr7p^JG*i|f zkG0(0Ce{c;#A4DYrbBkFphOj>c#1Z*D3=}|z3;^Czi=QJ|HynprpMNDO7%92dsy0- zCKcdy^}?s-Jg}7)iVZ%vFg^8I>t-YFWJX)!(gwSBS)JGqhEsh8hO=4>Pi&r`tErky zSrJd)miDsqRSeCZuMEMI^<4P{7!eGgLe%~BV5+ivanbGH9{7Z;_c2tfI_ov4iG_Me z2s{=JSc?P8+)xW2SkR1Zx`0T&bw`lh#>A zj@7ov2q(x#KHaQ?WE7q4B3e{50)0*;H>*FtigA@gQ@XeR(1q8gYS#i&n@%ppW9cW> z0w4BH%Jvm*y*vFRv6C$%vHIdygt!;>GwN9~V%)Pmy_ zDhYwPlz&uMonTkGn+K5Q7D9#*8upG7JXBL}LuNs9&FKL^}`>(-W~%a_%V z2x!)|z+1h6|7GRUACLs+W>U#%*>%(&Gak>z*%(ZzI0a8~=!#l8&;Hidc3m))ZM}~e znC|0l6)Qcr?Evq6bNHRsfFUH2N$#AZbflp7_nX+vlw&3Bx=6v00 z-+ycmFBC|GLQ%x_f}{d0%&U^GNH^fr-MpLytExC_xv@~oih_X&*UGj+lSkJvGY$kK zo<_2*mPXo8_F5=vNd~^Q;l=h79B7`)ZY*CDGj{kOkMe->=LPV z$bLN#D(C3S-KtC(^~x>*l=mSq4$HWZyW2xaThXlss>88S**_> zUw{E@4^Rgbf)xi4|F7&my2@X z6N2U!aL!)HBj&j1-g=)ka%TALP$F}jJ{M_Ui4r&NtqqifR=4;@nn=oxNd{Q;`Gb{r zfwO-K+%>;k+m;7Q>-yA7MUZE%$h=U+D(}4ytl0HL;_uGxvx9S;5s{n%SG6^}GbFEw zzcF$Uk|33Dzi*B6PTew!k0VdmR*xu9rlAoftg5?Jb@=Giy<}xxU;T!!cp(jf+PQ|) zCSrrSG_p>W5-C)RTsNsYKQ(Kb*OX#I4wrC7#k618d<7Y+V#Qs@ze1BKXV~^$wO_t2 zO0vO*1XRzPMXy2i9HsoEhuB+-AsPf7?zwSkFN%)h2l5=Wn7+0h6V7^h|D{DBYw@lj z!gu$!o{kTlJL5(L+a0u#fn@rkbc!}^8S|M z=va!TuWv@sJg)*}i`yY)Fu$Rrb=5RpYEv0`*R|xRl<%&1XPF)Ljau+}1EmP8Xm;Tx z!_+Z-!YL0oXtDHRwQ8GYt5VB@;Wq~~;kb7ZLxw&w9^)WYs+#YB;JZ8~zT5Wg4tZDe zq*qm^YKeK_Rx3v$!4AC(HwtZiw^tQiV83L#$XXm5Nj8V2U$)QM8MsL=S6X7;bX@uc zBSkPog47r8@22199?OP6M*VD}zqAbwSu5>%`a2SVm00g_n_<$F`%pQ(mpz}I9Z~}4 z-)I>6R`6!f(HWUnXX3~v<%c}29J0dIX>Xs8CpSP2jk^09SX$nrn$4@yet`$PBeu4qcH%1nL z*}RvcJmGF#G-nK-QWZ28+m?8@70mid@ZAO$)`A5^EtMx~Gu+}c_jZHEW5!Al`v}3j zIf}AsRPJ^;Jlt>7rvhu)ZTM8rv`viyl&6pBtV`0}Oa35RX+!g5)an7k^Mop45uDh+ z%G5zoVdPra_YDwf=9HDYo~nwA&-fsKFms|ec35I9)*(MMMq!N0w zRmz7OTN3Mv`AUekR`vd$Ta`fwretK4^4$P- zp;_&m%7fSxrn8ntK*~vdZi@69oQgltS7GR^-y^0}8W69>+z#MLkMDSLNatAyj?#Ggegy+0CP3=KhH%vrY;;MW`00nKcg1nIMG31n z_sYT-0dYo2l1F;$sj!k&@!?)z)0~Y#y#mdCxy`GBeTDp&k&Gr&ATGo&=z2!LasD1& zA9?Wcm*D%Lu*&b3^G+jBs|r>*XV%h^qfi6UtzGe)<=Vs}eq6c|q>2E`XlT<73FQL!E2N`W&f$bO`HI4Cb;h0l7uqdF1~7VOK9GH8Cf>gkdi*ziHqLbdn) zr)}Ghi4CVQ@9k>vU3JFmusGd8!0}d^gs3(yz?3);YoubSoI}f8caua8ZgD)tY5Kp{bqt?$sx&>=x z`8rjUZ}RCDUzfN-3RDjWRA>l09nH1rg6u1vJCvGJFL8lX_}le;=E}!mkeDdRFIhQr zyB}h+*C}>-Wy}&6O)dv~tFL}w%8*Ybv!K~$B3;}fO0_mbxYy7uGuM*p6|>MKYd z4gK#!n_dk=WdnB^w)bBNN_K9yyDUDxyAN+TE3yxD!O}?kv2|rdxj_lyTwJ@x*gC9g z_kE}8r87gPlX{&;l|ahH`|;t!=W+Ps4{x+bv`b_reUnIwUVVX!%5|Akot2CrS8Fg` zuF~*n;hh>=DqhR4IpLLlv3|mXs81w%OOTFZMZH{g5HcnF`*$~N$cvHru_fs{Qd=F3 zpq7w)9PnPehL;c{Jpj2Xb;hVkl>doOn729mD*2iSs2;eyC8roU*{B{6MvzQ-y)@3b zzP=QGyNnizAqw8@3I@Cp)j9X}e2NxgIM+NLC@z^r&&1s0DwRsvpj_f(QxnSI5-TOR zuAlSJLmqTlaj4|!*I58l=hEO z>@wXRsqq*;P-_fftstDl;5Bp4+qtNvS!H2))uxJT&>UL{X{nUDZ1h52X~K2$YK8_m zPjk{}_E(T_iRY(q)6bT^hnL5TOUjV+qkBiRiIQBFIuRb6cooDgzmV~e8iRypeAnb^ zZp%)1oCO~}@7t_Mj!h#1>mT#wT}AEJL?qU6$%C@a#gV7+`U4e)VS+N&)c>fgUg;=pRz9bGBB3QUMff(iZVREKV@IdI<|~;p z8Bw4r&rV)e4`|DLq@7H2xJ>>{bOj@M{_A@g(orVgk&7wooyO`D)ZTTC$N4ue) z#3+V2yURsMoLH%d$Q6o?hWf&Fhwh6$*Ux4urzSU5ciHng4cCvc5Zea|&&j}1h3~KG zr_tbC3?k+?Pf&`h@B^b|P>zc8EhIFhX_l(yf(8ihm{{I-YGC==SYX<~*z1G6!mV)b zMXv5bL%>9nO4P4jwUDUr%f|J!B~Q_08lj$G{c^l-GgPg7?#d)L-$jbM4%xw~^{&{? zFzPXmqroAqBPCE@`m@#I?C$xX!?5pVGQUY_g&ShbH@ROLy)MUV99u=?W?{p%xb*rrHXh!o3{}< z5y^OnPr+Fe89;V$GAvHFLR8PGI%a@^K^rCPz09u9a=h$LvoA{`)0B_wEkgsYXa+Mr z^$ZtoO^}*~_7;u**5-k;)W3|;Qt(2?kP7sSovnlwcBK)`yFgM>XToM@N9cS)mI!I#!!E;%deLLUvo$e~!dk{$LGuC8+C2;CU<|9eNXV=G;v_6eupENfUCBp?JTjrop*QMTWrI3oOTf@TuS=k!K zvyMJptF=N#_xuwNQnYgEQgHvI5&J)2z0|ouww`xHqJ5;`POS(WH)j^xhOHFnR&;hy zeAFPzO1F76RC}-!(oYq>Jlx+bZU@2*q(@i}XTEda#x4eq|M=1}kcTCP$#7YINuS1k zR=@;M%3%=33;-KY7u2z5xAO}iapP2@DdsljKS?kAGac~%G_&z<$NNzPrKW^tANW_m z{JGJX>(^&!%3p3a=jIP??Sn#;pA8PJZ(`)VYOD1txIwSZhX;iL|0j|XKbibLye9u& z8?F;X=4woAot&z1XCiEEZ2A<=fOSu>`w7M|n19KF{qGi2b}fDV=nG)<8vbsSD+RaB z@yJlAwO(^=zcJ>Kji#6KtYS|6g*NF-vekA``30bF4F*kdGUj#bJ5_SPl)hG9_l9c& z_65VIB<6Bk-lfHfZ&qzDxX}Kx5?FoWMoIRshcK8eIgnz-JbG7Ixm)!?dO}m%P-J4z zr8V|PNBo}acc1m4p8ods_NsbTFYfCrU*#rQ+~;cmSl&jq922}nz9@y zTDTr+;NE9x-YT(J@l0mBua&bYPReT~874+l%xLBiCZ4jtcwMr;x6Q{WMOdjZ4hG>| zXT|_wr!xZLhS9)HsFf^oUIa&O4a4V*A!qrFVn%isJjPmN_IHj~?~Lgz%3ftQ?+3eA(w8a+0g@+J+n3h5-K$&>0YOkYCk?X>9V&u ziSr!U9xmxm3YFgS8&h;B8FD{4vbR-j#L%>jqZ}F)oHK* zgkjr^tnT5CHKhn{lEm=R3(e}?d8Zubt?4wGwc?UW2csO`x}VGO=6@{5UxV1Z4Psme zl0CKPJt;xoD!n}i1G#gW-B=onJn#AKVe5I1!3Vg-(#g1!+7`y>2ytEurEBvgBg5ie zYjc4FxBn=U&96~?TRO|F6Xwz)X7%x^(bDrGRO42qcc*bV;t#%mTI*1im|r@|>iI2^ zZSO~5XpS`%JA6BY>6P3mF^gd^De|+dnX{ z$ed|z=({;?4g_eih=E71vz>0zXW!o-xTw(nZe(Pg$R(P^P;D`q%Est}=;heaFX6F1 z5g4_rs3&3X2CP@NE46EA@c!_*Sf#R26b1zdHx@Bb4uIY;kx5F`3e{ST-E!e}ua zNH*Kfx1;on70WH%=Y;MBS53xw>FMj~QGhKn?>@q8pPQWFfQl;Eg5zk}D-9LULI2ZE z-I=;cK$$!;%T~Mfb37;{uAHqgwtdgh2t2q3`!}5hPyNPpFF-xAf zfvO)*_zX5c&k*tX!Vay|Fd)L{{d3cr1nM)dnb!q(EAf_EtbJxM-s#N--@Z*rDSh#q zzGLb5chTz2W?cM(x^u+e{{cQk<3J!zX6UopW$wE(#)cCIauk34n?Lp0m3wSC-s4v% zWq?6STCCpp?NI?A@PFB{IuE>LcG*n20Ng*an(ZMqG~ik?YG`3Q~XGUlD-(F>1=h*+zd8$&-K zLKnqSFl!sRD?=G4XEqV++455D+(XLf$-QaaH zDDrE0(;GcM(BJPojGt5)1UfR%t!ov_@voy61I~)6PhI~OMN}2HDFXJu^vPtQ=l=Fc z^$alcIS;tvy(tE5AmQl$Aq`Xy1zTIix^d((Fh7+W#0aN3*w zJyq8>MRM@X6~O9dE&t0J|CxoleVMJcP*%f2>pP7AOO&sT2C`LcvF&fv=PggR`gNBH zS+$&Bt=`|RrZ4Wd%~+LQdWOp^n=c;V-5Or6T7Q2T)ugZ1rw?IF_=`$PMnuol`v8H{ zmj+a`DOpZFcCT&)oh=Gxktp_=>3rt^v~cz{3$VAH1kMtaYgvw3}& zTEw?@B;BT+lVaoH;bHC?Pk9h>taf`$^OW2oCDms^CVC_^W@JzW+Or=ul%2u=V&a90U-7|5&bp@z?ujO~*S>J?Q^(2EaB4snt#m?deb- zut~b1L6^aoG`t5UlJMihQTbS(wLT)DeE0Ej`iM**zVjJCi2w0(cY8n%$Tp)Ke0_8P zFTgx)2UmNHofuWou`GZ>VWfJ2V`AVS0sCa9fEN7!)$9rEz8+$+Fj$%$4O&gYHe0V_ zWa+X?C*BCa`OVh`xc@pGMH*EufHCiXn+BePF)%Qm~ zMu@pg?2M?eW@P3!Aas5kwK(r@)NG!5!ocX8{6K3A*t-Gz7(g9k@x2~4p?k{Rh8{zo zfvNuRJ*&7seg19Ii#H`ekES1@enKu8)CT&0=o}vQjlV4XZ+IIJH#~U$>#89jhPI%t z#{}|PX_ME_e*gXX1E9+?bxkp1E-U7tQrB!h{Bf9v*`cuQJ6O3)hajF}>jASMo_gGRgnKWh_LrIPU((;~Ku3 zjmE0h6K||`HkRg{e3zr+#g{(O$M;(hRnGD`?KX>!cn^hK>$hGSk2IDp$1#}yyPEy= zRxs-M_P)qV3y#YJOZ5pw4)(xE1{wYMgMqu>wL@j#+KBgZ@jHz)UL&vhB8Cya{oN$1 zrq^wR^&yXvz3q{GR-f@#+Aq8|Mg!$T&k3dOewCy1*-8RPj0oJ1Wy$AY^bnRM&*tO0 zvNGLawKi=AE}j1bwod^d0=XdnHy{WJd{PX4B8vV6=q>`BL+63$Nsd0IkCU9st&KY; zGSk=LKMw@X0Y2-xQ)Tku4{u<$;{hc1eekNF736+V)g66)HQUo7+M(EHc42=b=q%4a zKwU6~Q5My<+mDx9W&Cyo>dzO{_2!um@V)=`N1y1BYey#gi|cyDA}c>rA5LnSFufWPg$KsH~`6#%3Q)&P{OYPx@-z^c^|j~Xgmtay>i3M%H` zP%v6M{ERE^x@Aj@Sduh2uC?Ij?TjU=W!CUtZyasj-;^!~mn>IP|xfor&h0u1(DpDM7GJ$k4fy#`Ee zBzAXq4|*+DCb85S$v&sPsetchhdr2b}fT;ucsysIY;43I~?gSS~L1q?3!69DX zpK;x0@!c3*K4P{n&*IkE0nS}hQ?ohMzD)p%^3QbT_^#@K75MTNJ>ub3iu_T{T**XD z_~_h1`66p91g6pP=fmE8-txKp%do2SCRuZj5?LJ_F9DJ~;5^{ak{HNpd-=d-nV^WU zwjV%zc`gzP(mXG$RgVTR^Qt1ih(^-OhcI>R0lzq#M+b}t?ir}c#n4EiEOu3$EqDxU+e z(hrFKgh!?hz**XjfTfNvU1?XUHzrJ{jBUA$RBsP0))~3B-J+)xBZsANsk}*MKx<-Y z^3Prb3~6|IZTo-{n`pcx|o~4ZOtEZ+-RPApMmvl^B>U?a*JidawD7$QD>DPgfA3 z;r}_)(}vXG^epmgEi6)o9zFcX>}*laK+!^2qzJCt%)>td@BrA(end^tFltg$Rb%_F znSOi>WFc+5MPkuD79Fg7v8sF(=eO1IUWu>L)fNn6vc>^|dB7e09KE9>hun zV(x;2HHfMH9pC7XWe`X=C&#$7MBQ(|rYT1Og0EYGz;<-*0W=_!AwL7AC%vCcR3$j? z@2(s3>nwo+eKJ3d14tvdmR$o?+X;?;#c;sgXIGsH&j)+JKfI{?{`_j4p-Z!fIZ(IJ zJ1!GV=S2MWb|O%4yp$U4*pC9!B9&Niw0SRfS~>tgW_sVM>(Uu;ULSe8k;tsv|wAyE{@9&kkxb*hv@RRBo%-d(6002%i-8F>C{t zi`(OGY$3L3p2EHDp7k{1qhVxnQlt#~Zn640A?%OQm@p$87fDWq^w)D$g+jRWwYTxv z;M=LGkhYf@J@=+r`eI!?iMIH97lLWcR~wb6fr%JyeQ$%8i=)T8);zSf(epgxofZ|! z?}#?IT!atd%E>O?PgMIDwgsno$Hoed+!}LXSvD4yPGdMila-WVu%lhZdMZ6_aa1LJ zzMpuWBOFkuQCO*fKuWkyS`1c78y^yhE4(|)_pZ3-Bz1t+-e?^>N zUdV-qQhIxv)_*i+tPUg{m?{2`vC%2CkW6F#vW07xryvLEv#vXhugR3#LP%akpb?`Q zoxD|_*_c=4veeEnw|;o%$phBg06kaXfWbE@;5$djzB0Yxr5Mng&2Y7DY9W!~NT(V@+vC)^^V2IM1PXB8d46XQj-B+-UY3xrh z0Vzp8cS}t}J9!e8(ypsFuHgCSH37&59H;#+cJcm_t?}~jy93$h%uizyjyOiUC0&5} z2(b=_ss2v8ZODMTa|h2`b^s*M=kH4d|EZltZyIdXE+gk(Tl|)YnJS6^b&9%j;D+je z2ub=QhQi=q1@hMkGUwlrxxi)g6aGfbv1EPt+OObBDyXQ;`q{+}R4B2As0wBWjxRq( z!*G}*0RVSNdAT8JUY<=jL3WZcdj0zDMQRfas5!b1t|&MiTtoO`jCljKYxqm2X)WLqO3?oa|=f+LE9hJP*#vG6m+Y@U=|27vaB z3m(bgwThi5dJB_obS)k|IZ&|Up(vnmn%d;E@;?rRHTv%y);V`h^+dwHK|DDOs)n%^ z6^0518`N@+28bciJ~lZe+AS68V<&B?%sJge=8+11(x+Eb%P!ssgUC5{#>>nceU%M; z*@ej~FNUzBUe2|W8{qel`dm56&-QbZPQD2~{l$3mBtIFjv(@t*a?0ABK+LdZD{8iBt(pfCkKN&m7AK1 zNlHnvh-6;Oa)lJhS7k?PGL?pXbqnIM?88SHRm3=Wd9ktx#OTbAm2{2lf=drjDIdwM zqKoktwhW4+E3&A$*t3kp^x9NL%#_M&)MdzI5EkR()ct4e>mhBl>X0GbuT^ncpdVlO zM?AP9K{x4n7rFXoL*E@fr@qVvE)A4=+t3fJuSrTrkUOxnaa>%b5+EOKfLJ&pY-uzv zsVUlfhdLuJftt7SoU*XtJD1}u|NIs6XhpX`1>xhun=39pEFwzw<+fbaIKj-osVkN8 z(5yN9<_8XVZMv|Ph*WTpTHw?anv3rVNHw{y_UH@S@5_!i6>D24=cCo_H#HDG&c58y zrFG8iRWMt6qBl><==e&wgi(CF^U--OqCuEo-*uYgJk7XRz4662@HJ2%nvg`?0e2HM z2nr5HmD-|;rE=RPv1_I7ic3rDaCcU*E*Gf^TwW(+x4nnV3D@hpU5CE*_Z4;FS#)-X zUpF2V$@6h)1wZr?32kb>s^8WU7);pW9=M)j>YiW1d@$ozAR2z8&H?N3(Kz(Ox<4Y=7g6cWd%NNj@s8Y4fFJoK;mFqkdFQ9_NgK* z#Yg>R__d+hlhKk4{`X>uWLnrU%h$L)*WL_Sgz&f`qoidGCPV4ScF%SrDL-Y!q`i6C zS!rVXb)m4hbl+ml5iXY_0s$GCBYdHvlAC+LmYnm)c z6Lw$FF~qsPHgxdPR;(SvWlYj3*eLH)pD%_vA8p@HKJ&|7O#4RXD<{o*$2zjDG25Iax1*x7Qc-9!jXq^kH8dXI zh}oU90>7CtHMfhUWuJ5{*xANC_iWS(=>yziS{i z{vy!_sSrwu{MZc_E9c*K%k$Gg?9wd_cLNj5&##GUGptB(M(OcLaHnJ};o1!trXLn+ zE6SPFpTAiB*4N20^2*zRKSxW!?G^oNXZxWP1wUQIR%u3s zYFw#6@>M>CQn@V(YjJgz2Cwn|)!uhUHI?^$#&Im8D9%_=>Zk|;0)q6ahA8l*aT+J)3GR+Igb* z*rlNP+GvZEVJo69>lg^Ji#s)9u2ni&7+8L(Q-=Kcu??AP)$q=AW>Xp_#b_DznAvvR3b zt$I5Rvp`EWx-i8=<;m{*yC2ReYUa8s2>z*KUM9_M!QKn!J`ZE7uZ#6YckUrO|uZcVHB@#uT! z5~qiWmYa#&>9`%Q_p_!y`W}^B)db4VP)+>uT8>mU`Ej{2aozfA+JG^OV)GBW4c9? z-n4k1gQP1niQKZmjAESN z@u7e}|2c<>o_IIDv0?IVv5h_zlA6{ie`2jNKM*r(uNgV94)i6PjgY@*%k4=Evlb=b zBOZfZhDWB<7p~o2z~azCr#Ifj(|{S)RtwE>GZWR2J;^oai}tsj**m4D)ADWyDt%?ea%7`6XCj_vF8m->g zfH=SMI_*q{k#U`WsNt|xla`@o_qw^kT%pkD;P<4oW+Xp%#$T>8JGxXy1OyMi+cWVe zBrud4q95H>ZjC55{QP6$EYX+U+w6ce>4~6VOTTFoFl|Te{&XXF)db~1$~+_NQWgZ; zOP1O6Hc1}sHV?Y=F2P56KCM@R9;E-bg~juKjAb0U-JU`VF!;Z+N8Ks+F_ zJNioj=+|o7yxjAl*v%viR%2mW9oTd6R=+{GIf)6Lcs8>M^kb^F5WB3Oh-S)+QcZV% zE^Cy7DaFVYo!5Wu;b_|9@+a!5m3S5amehE$SWVBce2fS9C&Tk6`KdE$7bFe`+w=sB zWES^5{bD=KpQ2ARZ5eD{O(CZQj0ETP3z-jQGrtGlHX-2NiRJ-FMHN=~Zw7NesWbedzBuMU z(7-}fQ0OKvP0Z>>zIX)VgQ!c=e?g_&inuxq34YQ^bk8wuPAtW0HR@+ z|FpEMS{1JYZ~-C*4j6T(Rh{{m zqcreVzvyHrI7I~J6n>@Wv-vQVnUgz1Hs8)|)lNRlDVmVeJ=zu2~GseI*!upG>o4-tusaWS#1)U^t7Xf*7VQYx>fJx?>DYNFL%8Dyx!+NX^T zoR^pfDMkV=zUiy@2k3JF39x9hhhJ2L*9?r%ry^6+w3V0X9!R9|sc`S5U3#=Ks{``a zWgBI?m*qZY7$MD1$#_2md_X7Y>uvHq!?m6+>-ZXYjLBI?`>$_#UZJ;p9M;;0_(y1U zj+#&YOerIchMzCDwl{8k8s?Fq@v<<1upO9M>ZE>IuOSCvh~ApOY4@vd!Pe3m&K7<9 zJPP#R+2u3p%iF09XJ~^jhbn6WO@!0FroFy|KeuV0vPpRhz>GvrgtRE81mvlHnIju# ztX1;%65sXqrJZrLWGt-^p_Gj@qYrg51HKRCefdKltGneopC_3%9^`h*@bSm>lPzr< zkvBV8BjIweRO$jSL*Sp*DX^Jq8&bzWTem$-o=@A$#UHppo3h(()l5ZU91kcOAQzN|(24CJtJM)2Ws@jwgNOjHaOm1=r4@!|u0lMj( z$N@-iV0I4>Y)fSt1ALR~LYYC53eD8o%IgnY4t$3;!!%t6!mlVbpOd#P^L`R?1`;k@ ze{c#LE%aioDCnzWimEJrkFmis)%y$0DT_{l_198JGMl$k=H)uA)dyTd{8D-a6A?sX z3*#3FUaivDa{%s-luZ{-ujuB`=map(Q62d%`D}`rsocw!CLO${HT-T)_RzyOvi5d{ zZqL1~gclHJyR(2WX|);{B^|Ba;>bpEKwmnh=rx;%U&neddgV0RfymH(1gI{>#=tyM z42zF}fD!0srLU0}y2Vlf93|P5=+>vu+p?Ph(3Pgj$xYpq#mpI&u|>ILu70j+08zwR zw4iBqg|653Y$nH|xBSLxiZsai(Z^g4o5~g5o1KzoyiJO9>|{Q*KA?eP zuLJ2K&?`Y#+m{X}<8YJtQz7;}dMAW68kb~@!_8h4a=Ozz+shnzM{Y{k0ZQJ)GO^J( zs3ZyZ1}_uonsUup$!VZzK`=zid&aSP@_TZmSY^sx%arlky$J^Jaa}UJOEWXl`!=L;;&WY`WNEgH_POpE!5DBbpq3YB&l&N)Zbc}Y z$vJIb0D`U)=OP0E6q^>3snEp;K^oP)5E#-b^@wRfr7SvgIhMxYS;6>@NeVY%WhKcJ z@YoV$HeJy^eC2M-6g}88!-~s14cay1nB18SX&Qh?my$-Y0otnLGDY3L%>#wq_8XV{ zeimnBNME;0PHidtH0u%B5-HcJ7(@D4tt{QB-rd~V_lfEx$>@>Rof+MeSSZBUAgASJ zfrNe%=8aE7NzyQ2_TQ(244_vTnM_}E zONQC^Y_R*Y(!Gl_kE8&Y1&!x@tT@tKi4}P2t-F^iqN)hN%c@eKFZ7L0J!~jSNgERo7t2sX5g|R{U%q3%)10r9((ggw^*5pX5m7cZ|{c~Zi6_~{i zt;+mu)0wl)bvIE?9#7q_ZL-kP={}=;uKk+;2+rLHP&-sp!qschnnm}0#3mn%jma6C zrZC|SzA-76uq_p-sFdgnMIm-)&#%_=Hgzv`ljW`@su)u;y)}Uuk#7Pp8h0O6#qKqr z%jS0<)j`@XB42beA(p2Ol*0CMoS)!dgcwW58*BICnM ztDl4@dw?u(sZepD*-1F$=mI6%n`t1)$V%%r$zDlK>&D-o3#j|4vBkLOP_!vLvQ$n^ zzu_hHhH%%GQwlVtg<|^GwnX@5wlou%Ea;h#pjxbUrR9l&gnk$Bw%v&^Yy2zf#qLAB zj<8NOQu|POQ`X%t68=liVj=13md_^@C?65aCX z6V)Rfm&-k0bbj-RG!1UNYcG(50WzMc&M5)8Go;=|BT9=?y>fZEWPAI}s-1Z$@bYGW zxk$ssX#;RRkjrVUSzcK^jZCoRh11s4CW}BP-39u~z0fhuJ3tdq#i93l9=#!Jt9B0q zh-eAZyAZvzg=U50qc}{ui?NQ&9fM+v`hMaoS=oqGy%xgjl-leymM@z<~Kwww+-+S*t8^Rnfs^4V|kG*K$Y ztt51dVO$J#q7BRK>8>v_`!Fum<$SBT=$MiP0E(AFOaVqfyq;u=ZaQPP5jx%Z*6{n{ zsosBdR%fJ%od)D-BkPNp`wpJr#UTJ3E7`ah01xSr$Y%sO#CVuYPra5qbpC&s;rQ~@ zO6D{p;@-rpHg>YHw*}Y}knPW$DKlQa5$bAk)u-!6;K7s@>7tv#P+(-&77WX*86(IN-`L3#bC^2NyD5s+GJ{J&eRTEIMFAwdp3^FIN7-e3-c+H2l5gMS7<#9TooyMe@*;P`h9U;iv&{`Vk0|8>%T z(dqv&R*6mT1uCp-!);C}Ee6%~0{907x1PNiaG++Mv&VZC&zBh4bY-PhZBWainBpBn*tP3QbgbyDTa%5|4J$XpkAz(Lb44HV=2h&O-#d=}@;j$3 zPNG9{_anqF9+rPBa|+-Fr<7s02l{ftd>g3@T9^TXF~6+iO0?A{+Y$kU()K@=$YS^W za+Zm_3X${YJ7cQ_FoTtDi@>*K$>3M7gm(Ar>^7#Vg7ONBSo>*@&&iwN0s^gPJ4QXE z?>W$O@ztV$nv3=|>wAd@kEMBB9u;)Ik1)Q{4{bRa|NL(!1jN0A|N9F9UZnrW&u{;` z&yB7B;RXQ*qu6^}QPR8qv4hy%qy=Y$6=C-#Q)hJd*TA>e>+>f5-b=ts{1c*KCE4$< zFGn6%V=nA&e;_XX3kXyX8AB9zdRB;Oga{HLRB#xD0=g!mL-+wM%Y zKbKjRkyoxtpU3v7f&xa~3%&-9kt8+GzyCF!V)S>p(%pGT&=mk20j8q=(5`{TX9Xo; zfQpKKfUhDr>dv7AHD`d_b%i^K8^B^2|6FY7bqdJ+suGy%4t0Tc)5C~J+lq>TbU}j^ zs4@~*_ABiraT=p*FGA_ss<0uHOFAO1gbHrkjt44*?%~33DX!!)ouaL`^g{Y0`aX=? zTZ*Bgp#yj0gyh0*Z2pMPJWg!-XTb&Xz+L<1R)|M25F1#@@?gaDlmyD7kmACHvaS+$ zV@z!bfok^x;`Z6VMtx}Rns|U{pjg{GA3zS{xtIhysj?uu)=<TcB5xwlO;Fb4DYkBrPGi$eR*9lk(BD*8i~yd=4o09G z5vlgR5qnBP`_B=-*3)mPmsf22T^cuwhukG53?nkUrE(gI!+Q z)?Q*9w!SZd%)3voAzoj1g2mCNh(%(kFI^gQ-yN275ZeRkXau!kvH5KAG`EdzMU2AM z+d?YZbUH(VpgF)sK_hz(rvR97O|~pL{S=W9bTRQER@lzoUu|!+kJNa}Yn|6k+d`8P z>TxSy=rK`coSxbk_53oWdJ-(TJO($I6&%cZg+JnO-OktEtvo5ej_br4CO-YTkz1B@ zbWtq|GoZ_~k*n%^ZLig>uh3JeD%69i7HYrGtH1FZsPy~XaOkDhZ9)abDR(66*ml|; zNrtc`iDhc1?60P#2d@YmaP^>GE#5GA+eE32UOC2Qib5`LyR$@y2gQ!&@>gQC_GLfb z`Ry3p93!FFbOgV4dqbvE8{ru7G~h!7skWuBcH7fy1MS$_28Thdpkv?#1AFemF-os! zr+@3cv9>^QSpgZsF9L0drHj?Wt83=*VO!wIKzpPuxt0;w?y#Jwbkb=7<|B9+pc4RB zxA!CkC1!KnvV~~Vy&dBNye1XnbtlSD#MLPZG2MG-J14H~oLABYMnt7fMu#{?G=DVI zm>X<(93W5sYaXi-{Z!ic`d0 zR{!xj?TW#Xs?-bWo;h_-r*;HLL$F1taCU|;xxFpH24{`H!YAa%5yd)9O`i{&WH#GJ zIW_+3CHuGF1R4RmB_U{{YT&zsO|ux@L3M+Nl`M zIMLDd;^um3xmu@&$XiKQMC(?o#F2lzT;SRLqrzBXL?2Pe?NVD6eiq#i6?uA@(gRVr zu!*K)i52#lV2^B$4FJIXdJ8YqwG9Aq5vvydqT%c&UJcJ<{-wz(sBY#K%maJC~0JSJ=DY%z$j_@yQO4|*A( znqq3UIwgMl`*A9-p4-*lmj`Zbf!nYQMbo#VI5yLYafSgxjcVBIfE@B_;d9^xup7A= zPFmyF+}_WKBR(wt0e-j0Vc1FKOjXsR&AWIb_?x#xTiO%6S|5ga^1qwkd zG{P$;aT-&ACD@_u(HipGn)DJn994f}AV^KT1$lH*!kJvzbeGjlKZJ^po0W9cf77&F zP1UJY3HX9SN3@oy=zXzx{GY0TfKMzE^K;-n8dIN)_UJe-{Gii`uTun!JA(SBWfQDgms)QJd$waP(*z$g zy)aj$ckW*VG^j7l?l%@}rTsU6Qx{+k&;t4fbTMp_&cRG;tTfw~2Z3E<;#dimUC<>B zjX~yVBFg?3WI<9%8?LD@aa>7$vCLw+f9`$}SGj(%nOS$SOt9Js3qb4^%N<**x10d= zewFBpm9-Ud^vD;pfs_oCgY9w+mY5B86+iQ<=!R&dTPu92mVvbMwNqZk*>6yBTYs{) z=m?@C5kcC|pB=_Q99k>4R_`VHTI5s7-f;3O`0%|8e4V(>$Mof>Kp2W%OSdLo!y?Pu zDi^~t%^#6Z6tCQ7wx=xERb;R83G_s{GD!ynEQuyC+V10n0_ow{Fz(n$QVW~Bp^xck z%(x9QCC&43_T=zU8Z>)*Y#k zmp_-Z#}I4#xm%Riap19&XXxPykN7!wv2$!cwSmPYJL`>lh+61kE8kiyeKSh;_15Kr z`?NhA2PB%_Xx$VOGs;dd3uy?OR{U%3WVrJ)e0KkC`xjGN8*~-moZy1@v?>%n^O8uG z-37{jbnc$0mI^%-IDf&hOqspyW4t#i(VwL%DtWqG_TI!I1ZAjAp~7D%b?udrB}d9- z_Vj`s0x}Cnc|0CiesCJKuCCc(s5y7jp_?QI(TTnr$ExrXa$9=_EhgpL^1rqz4EVPk z|G}e^G=i1pB-YOiN7O`qdrV9a>$F(v-L!TJ_!ueR#S!Jh=|6+8P2SV}vCV3%iI?w- zL|Zo)mH6?p5_nG~sTZBiFTHkh6Vi zD#pgEJTAMUimV{osZiE+rUPsv3egZ2*Z;@@Un=JC^mYJ$IVeqmbG78R`4y0(GQZ9B zQ*{VV^%7~J_fomx-J>gf2dD#d^He>~f>j1AY6K-hC40|w=D`GuUEuP~#q^fTZU~_8 zV;cozqnDFQN_ZVQ>`g0w`&iORrSCI`B**j2%kw|_qGvMf3Kdvk5QuYi zeRt~=e-QeCtBQJDjZ>$2IeL#Mesj=F-Dd%5IT!NhG%AstBlulVkb)e%L zHoJUzyp3kFVOH?#%nw9q6ofTSyo5EJP-s0IsJ6HDUDLL~I>G}=!bcxApOn%^SF-B? z<%_oPXsK#4^{QSj4z|c!M~=$laH&IrkyQ8mop{58m@B3~ZAi_k`F4=lEPA|ygl(%P z8CaIYc#qMZqE0_(8#*~hLlIG!OLZ45r~zsn(Qn~yfW%wcN%@y zr{i5)vi61}5l1J(P>(Ux@vOov0v4A&1c4wgN)Tj4nL0VP`kodZkhuPt`(YSoP0{W= z%_8YHD@EaRunpEDN%N59zG}2Y<*6}?#?m14BA)R zDy9-!R%GQPW%KftkV5CHQu$txy8!W4KtkuCCx?UGS7r7FC`U=(|JTK!woE|OL3I={ z>Gw~-%D>!whvryZL+hsiH3cXKU-TeITnwRe+;g_`N20@n>B{=0!)ZgW18DG&Iz zA_TdXY#X$tM@;?J+!}*mw@>$D=I}38xZ198b>25z9=D=E{_YfHxlur$R zLTl(WEEDFDqip5c+Eh49*mXGu2KA?u>~<1IFi(payitYT!W$R>Ab1(bwuusN-77;?{Vmas2CdXx z4)JUo!^+3OJWoe0WMk)$J6c$lRd4N3KT#C9hC#(w>7z_=$@i{Y9X*I)EA~cJKx{TT zEL!6~F_txf#j|RP4t##NUbXxR|9PLRU44n9YpzNl5)6&%?$QVyH=6y7w7D+f?z5q9 z-5A}ow^OTj%Z0V9=Gk{Q4|US?xMwCcZj(7hOeo&qIzwBJ*EQc|k%q(F-Y&m0YGCKr zwZ1jYuN^TW9oZyv1rhPVN~g2(v<;D0Yu(l_sc02d5(4 z;Uxy~(=$zJ&uDgF_0flj5*n5KZdAyao044p8aTQhg+fL{>y46UsV_tFMQlhFF{MO> z^yFuOg>l=5LBZ|@TD#gyfkaeQlQHyG_bz9;ClRL>AE_Ie%aI)XerWr_ZRFX5x2x2y z_x7S&w6T}a1gFgtey4G}LE@JIn@|ak!murnSIcF!-A!E!x1w43sJwO&=dL!Gzs)oK zumirbTjduj70Gv;+910;uwT+oj4z3v0WQBV0x(kSp$G`8lXysUdPN!L;MJC>tcH&Q zhgx}UbB~Hnx8q)IZOG$7;^-VvNVH26{5orWzFt!&hGi={Z9=VVGGo1^+Y-_CBfQ1d zul;9(L@+^@BsxQesZ%gD{Kpt}VuF>`TlaP`KBd z@Amd?#bo}&qG77z84;WELM5T z?!`Jorr?!-RXaf!)0BgpTs`H_La+v(_Z&|aJH&rN>~DUxkEM%&*`eQU#6N#1nhJ9B zY?8Eb#j1EyQ!3tt3ok_$a$3qsQMz7E;3x~f@EZm9m#2uc#f+r)f6JRcUoJsPF_rc@*SzJ2L$%Uzq}^zgqti`H}vvw_|I$n)wbrcs^=7XLa0! z7Zk6zGQmw-&Ta))%-kN1VFVdKVsO6kYR+5Y+6h_)?&-J?g~$)J1WQ0(R9r8fT5B_) zVq$!y=IdY7I@q*mWGBzvQeu+;3eB;v_45M`ErKnfERg1OUh586!E9$u&Vndvm8CQW}_ z(4vSQe?+sbcroyP1(nwWC@oxR=(jeGc77RJD8Fry&)51$8#%SkWz9QSyyWq{$|`>K z`qk$h{kmxD2ez5dEK-AFrRAxkEL=*tScWuWy}w-%yFF+WM~WmKMc*?FbsR~~|B5Ig zDlBP^{}s|KCn@Z5g0xSOvwVNJYB~8HwT}f?Pf%76MrtJhV1de}*)a1--YD3wZmsdN z_*BDpqeK8~^Fv`SvB$$@~|R4M`5O@0a9=QtIGF+(#h~3icp8zue9-w2^K;Tn7nU z0>HoX>bNZ1x<)XIH9xuf?3APl>1h2^jrf-lrUmtLCWq+{B**x^_=IWXOZi;})IF^x z>~yfHK0l`L*8C=!uH58){!J@)w3chgsGQDvXEo_Ktbz>0{BYL8@sOy%&^G?kAFM-9X6hXqitOhe#&@`Q42{zhWd|)fHN08*HuReqN)qlb z(KU2rzdiWugd0G|I|Wk#Wd}-4&4h6@_4x8pxh@s|qK-VHs}oRPE!m3fIvgja&v!wt zvsX;zuwNpS`&E6tdF-B$u=6^(6m{~cJT^fyK4R3G9ERU;E4%ePd|9aL$Dq+HpLlye zTLe8Fu`RP7gM&>IWbLH}Hvg^-MUhpQN5($YJ72B3ld|3HHqXYuBu6(vZo^nKN{!a+3_7q>V3I5==ul}i|WWV=IN zxXtoqN9z)h_8&TFy`g@|AkDz*3T+lr`~w*sKfSRgB)qCFf0^)NeRJtl^0bsw^P9kC ziLTQ01NlL9PhN^%)m?lm+1R7O91uLW?;?wQ6z!!QlFrSLKbI*?Rq;QIf73(1iVk+G z9Z=hF@VCFrqa0hm>MO*olyk*VkWzTx*3G~ZH5#55o19Y@Igm3H-mLCfDvP4IDv3lM8eQ5%P-s(^8yMQ3cRZE4KDek#IVob6 z?>uZEOZf3E+`>OOWMU1py(G7&Po|8UtL(H8+u}TrT z-t(IIj^yCn$$(lbn0|=MpFVt*VQoEM2@)<6{qc>M%5C%M$jshr;H%+LF>W8!hY*D2 zW-2IF5mqS;()z}|Qjp15CFHB)_r#ccMN`-|xSOrbDnYA2=9jHb?D8F1-U5C*g7;44 z^b&k~v$m!Cz=CDK+8>+_9m$B~JFM3~e?1C3aZhbaM@}cVZ*M(rN=ZM-5fL8GtSX7F z`;}2jq4pd?X}&Et@H$0t^I`3^hPMPX;roUI(UoqL2n7#2sTUdcehZ4LqK$^-e!a-) zB$Uw#8g}L3hHi0#Yn0(OudDut9;Hqt=gUFSs}qYxHO2qpPe_9QOc0JSS_bH`N;e7l zUqFW=6b?xR$4sklSgOhRMsKY>)5SVQX@LEmV%&Fk@m}M-Q6#6?`#)BPcTZkON7_Ty zB6weNrZg6)Q~v7luNs`7%y~}1Ty!3;_WcrUA*7Gqduf~ZW_8p^A4sDc5yxVJmZ*=> z)s(>?=nrDp6N(1O^)4s)Wcq}neCE@Z_{H_@w&SEvyO-L0HK`C#dkjz_iB-^q?^lAw zur}8%iwIV;oe7$i4DI2EYqprPVn^>PvA@#sMB__(i>|dQ@!QSLZjcWwosvO?tsr8V zSSSC2`SxZ%eq+>hyni@v4+J-ytW! zm^-`tLW=w%gIS#mBa9Kh$Zd@VBW2*Vq#IasE9l9hovz%wUMJIg>sEi*>y925$s)jB z0<7iFh^-};Zz7N`XNA`|dBCqm>;iF6K%&~9@uP5%ss1Cw>i1m3iIP{%x4jmrvw3#h zP_L8^{5xwmz}=mqskS`MgQ>^})>9%x1FjTUrv{DW8}3uD^$VM&b8l$M7T#PhFes3&2JtqKj0ncPe0M8u&3J5DK}5 z=W5-)zZs1#{s>sUcd(mdLG{~hpZp7fT_XQ&mvv)9yGM*r;_|eUmHS=v72AY!o`R5H-u*J^D{It4tT2 zZ=ay1E_sFFYQJ>`Fd3BD7c(NMDJ@5F0bUhCiY_z)aLd_d|1<^-dzjk#nG>z zCZRFqD%(AJ64dV=5oW9p^RGjbhuancpa$dPObl8rK24`+P>d-_WTL&sEPPmN_=__3 zfGhrLq4&ztMC1Mw2-JiH5AeRChLl>2qAPDzMR>9-kZhXMziGr@NjCr+_!Y2$iy}Y* za^$XQDiMs>%>g&)ly6nw8+tgQ4qidY${Kj@n81STt1!{&J@)!&mG^TZCywA`ErnN; zF_KI77*5W)g!OeI{ThnWdR;AkL90n+o*k}$mCDbsce*Kz|dZx zuHY(~ej`CGUMu3d*VOp6hA(STGz~D{7dA$Zu1oQ?;}@vBpi77y?QIJc%ODP;XNe*a zh{o6cGf68fqqt=|yIN)QXk?#w=k~YiuWn8a~gvExS}rEQZoI~1`CH7OWc%@jNS%`z~| z3$KhRZS!@aV7JQVHRmD`+StJ?=3n%J^5^4YRVeVhl;3RCdZx^8VDBOZx_H)yR7JQ( za^RQa$1k$8cpA|Ghr1dnwGMx_)Lq?)OX_Lt$ozxA>ze~!--QXF3nNxBJlyNk{i~v> z&I5FgSo#%TwRokGOlPxR`=DDB88VeFghwr1pMKLFu%p$a=M0~;no7VtwI_@f`>+6A zW~KxtBPAxmZ*)RYfo*pBr7GKasYat^5m=C!Q}pDa#V8VxHMgzXoTzP?`Dr?MuC`?$ zahzHW|0X&f?bq`#K^=EG{A>1!r+<-*`kGyoRz<7FnVc3v94Bu#GiLWr)d@8I-c1AV z)8<<+6sVYq;oaS0KXi(GGi7PnJ6(4kEcG&J2~Lc=J;G}v8ulFC*(g!3W-x1oqM;$T z#*@b;=VahUvyO!;WF-ehrxRoVh_eWr9!kCSlL+)<$N zeA`JW^!uwzqQ`lfkfj6sY_)ipwtsvu-}?XuE!x@AKP6M-B;BH_6-wsQjKrsqpQiBpWSRDR}8M z;UL!D`l9XxMss^uFql<1PqhqM`$5g*sL6M@8GH;=q|zT~it)q=8;Ye@=4XE;w`^4z z6)k!>McqkfYs&NThDx4o-UG_mSfIQY#M+#PQK%uleK!el!f8)wZeA(QM~swkZs7Fn8IYgUn&h8l5AwOofw;DtZyzud-O`!UJ&f zK23NpuRmvaape`?te6IBWo!NZt(4_t{9)P9JaXe>U(`32SN*WOliw zLQS`?73IH9!xvBdXB#H-lT2gKMXLROy43h?X{NK>hhkM6P{Zr9UTU(g~<=$r z${>V$I+8PcHQ>LO94~P13FP^*e>!aH}Te z9v1efXnOs18>tuN_oWsdoNoFm#`o8VCp1~H+=e$feLp2}dr&I83gR8T?aKu)e#^i# zvA8ATkAl?=@^GWeJ*Nw*U}qrR!B+RKpJPYn0g87gjBxmPt~g=a8he)1$f(eOm$&|@ zdM1At=A0DH8(O(ak#@P~HyU(tI(4OAc|cc*}dPL_YmI%KejXjakC`s53W z1z2X}%s$hfK;dtTxPs$PKfLzO9+V}dy}v2qtr}xgR+i2`##erG+AS; zV~{lllV$9~c}J1@zRq=>>zs3)KhE!uu3j_m^W6LL-1q&=vuir4%=?b*qoSf>R#&^C zPeny{o{DNu2R$9|3Hb0wBj7)34}H~3R5{HjCV)Q}?y8x1P*JfRr2M0HNN59=C{d|j zQNDhkdJ4aH-D})m@!R;bkNppg4;&D9cj&ay!((r}o~IsG$vW4eZ+7@(Mv-Q=Vu!v& z=2gegLbr-ncL!#uog-vZ(dt(p7)FQz>oJG zzj%g*em{@W11f6J{~rFwg#T&6|E%DDR`5S7prLOxwI+C46HUA?t?f1m<0H`LnfXbJ z@NF%zo}l51mG}Ml(hp)&>^HI>7s!X3ik0K{nct7Y{e>>2E!KS$%jfXC!iIEUhp$n4 zp-tm&4z2!-JJ$42gZw7RiaXxpjRU8BG$TigrbX>Ci=M}H-JHy0hU(E4tJ9WfbG?!i zT_KLdKfCnb5sQ)MnzwqK95iqyO2!RQn&bm2%{ZYpn490Zg5eWd`KV-O`fMBX2WZf~ z(nGhXS!09`ZDZD`mB`gIUmIPNa}#v@VkMY<69<~Di*SVhJ(d_3CB+^bfvB)2yQ zX0FJwIIuB%H$8AcAZf9{ij0p4>zyg|FDC^ZocGpDk<@&=(!<9(P@yv>H6pU3?;Z3Gpc% z*TtrYl94{?SS|Y{GJ3;Sf6p=er$iP~?O26ZrY+}uS9?oCO1}uN2uI$=al5LyepJmW z!LbA~AE>Yi54Y+pR}PzSwlyIU6S)XX`RQ79vj}#GRstd}{(_=7slY72-$s7}|0yqC zJ3zwP-B{nHqfCE#lYd2juJp>upGvTcq52G2-us6qde(^iq(^veHK)5@0@W?(6-6C2 zvb}H4r#2A2LqW(|z^#KU6dP?Nw9dF6xB31^aWiwfK@nA}j&XRBHZkFY}=w9h?##avl%iEeys9HJ6DasBFS z*6O1A3!TaFxknfaSSH_+nq@e4Wsw_`=Y7!5;L(oa?S7c&`!?n|J_7MX3hN}imQzG3 zhq|(9tufp5N#i=f7*-vb@YU-pCYX4{QTu*CP_(nXL|xtr-s#bM(X*)|3%2!(k9-uL z)-V@X&a0E_z0z;JG^tti*eoh0cBQ?N2~!lbwb~niR+V5tV2DHX7m4#=nbW~Hh=VN8tK?&2EKaRdBE zEEQJ57XOy5%%u*Uyo~2)v%CtqotsN0r<46v&t;MQt^I19i6x&1+-|ZuzG-aRpQtVf0u4Bkpl{Wp(2+Jy0Fm-is*_h@)dNH>(IOTu**UXVebjpL1M-^9-&_<;$_x z%Q}YGG%VHCa87uj_ltF}+VdOdUj$amlUE{bmqt`w0y5t}UtL)-!L7R&;*9LNO57S@ zC=C`3g}&m>%X8H;bt_u9brV#8p3Jp@(OK9?wsY?2dTKIVeQ!g0d{ID$>?ib1zl2Kg zHnzr>sDs%x?_Zlf_(Sg2?`bDN-z=Tiz9B0uWs$mfL@r6oA+kcSq^EK9U^aejq;4@v z0jWEODvj=0l=-x1mX>n}hH}j;=)xWH6N+oicVC#_Ghr6+q~<+F(`?n}krwxB?2>+A zB!IA`L&=iFx5bH(7TBV4=&`7^5?d~je(vZ%Jb#({J+-;y5u|!Ry;=abTC}9y^%fU> zy)E=yPY6sEjr~9qKY4LM=qh%itof) zxz&H3@%5i{SXv*^=vX!_)3YQv+(Oycv0I{E$|#6E_LVVoA8pK1e%l+?!fVE-%>fws zYV+-L^E>LPlo*S)#~+Q--PTtn>rWs_Ex+75j`Im(Nqun@ z8-PdXg-K-)4=ii7WryaIOAIDlF~azB3GkKS$#Go1;;0Wig6w-cmEe%tj*uO8DoH7e zhjV{y6NgV47;wFQHVn(XAU?KEa+nwphpF=K<;}KARmd3k)*sNpoTF8RtTX3CzAfB7Q*CDU4Wr(QA2u~!HM=C6rJbEr|CQR67y=yg3ozSFzu z%0Hbln z_7_8YTT<~3c0(VMh4Z^SnVlBSB^6g#Cafw%9y3PPt9G<_lKrhDIUJ(vc&?I`{u*I+ z;pFa^A4cZVsTR9rcA5RmfKZ3NNg)UHqPFv^{ZX%!LMojGt-HlE6wPq! zSNi=G%FgsuL6jpW^m;izq*e9owY^FX?Rs$4Vu3rQ@6>T_Y{aekjRu!_Cxifh@auB3 z5k|}krU53)*MyLhP$Si2&Z^@jK06LNA8xt6f+I4WO>hG*>Bb~<$%~QmM*J`g7==G5 zBa-1xcm$A}*xXMMu6sdvCsuIm!$;ve%U#LBtDQ=h3f!i1QJ8#hK3VXz9dGJ2yTyrW^EOMmGBqo!59Z|D@=2SOGj`p9#l=2%`>$Im^vD#nEaXkI z47K+!jEWDBjJ!=QQyQplP9!S6I^t}#7bCwn`vRe3@IwOMfR?b9)62;;byC?2(ihC4 zdWl$;gMk%uvcC+n*m#VX(^c`G{4f)ZUOPbICUldvOnO z{a%Ip^+Kciy(6x?0ugO(*DE7T}erP*z?4Kz8nJ){V7(xpD{W^xiw2P6kKQsk6!p z20t6P-thQI&o0-?Bl;IQz9*IIP&<=NrQcJTF?V>7TH1y;CugQ^Y~=f*&SD^?f?ROv zIJ-|y&ZWa!VZrxA`{9bDvpYOAST?@_iHU`j52V_D5w&J*%Y?!Ry%M+VhJ`ycIg$2m(O902 z3fSu0;$GfABPr2VvUT zu@S)eT+eL0Hdy3XvBIHvv3+l2o=6P;4wiX`L5Pjv!o#xm;2kQ)u3FD8jEG}gx_qw> z>~Sv`VRI%%&W<|uAXhI0GWTN_95L<~Z}_psbVhs?aKu{5z~y5Yu$#F#h&V)?Oqo4a zYbSpF@-TexzWO;Q1w^27s+)Mu-yN5K`F6L`up1>-A0HY@=J=2)b-A6uPvStZohwch0xuZ@h8|+PzZyZ4Z4W%N)OD}L{*n2ZV?ut z|1WZOmLEE>Z0>}AquWGE!E5dKg6L)Q)ca{t&7%*pE!1<9{)s zk86$&vT5skG^-fhFQ@l^LdTK&UQTXwXS@ilZF@~465=QDvRSu|{-^7s>|#9yBI-PE z;;Bn0?ATxvYH&i7J74}!zqBO`EtNr>V+zNI`_7snk}V5vrV*so=Fo^Ht%JJ-|BKwN zvUQ6Y76#s2V(rT4JDCeO*7who#Hwn0#YBE*(GHf-XN-Lpgf8qk(nNrW5iNw~TRtuP zu}}l-5*354Y10?K|G6@-*rh)>wevXh&(!{R%sOc{gmaq)Rnod7U;f5>_`z;}{Ue!? z+XPsjrE$gH5)p3BXM}h0n}KmzRh^!MHLJ0J?D^*8&KrN0!)eC2*T-b=(DNTF+Y6?} z;nP=V9_{<{>q4LkTi=)O|6YUyzM77Q-x<)4RR8IPf8!Inn}$VAdwS2|s{Vv%QbNL% zR54fj(`}spW#X;MgX+;;&aH1vyo@^iZsr?2k!k#m?p**vEe4+>-*KdWUGUY3;EurF zN;~{#M6fi~FC%1_E#qQWo`5TKvcbpxym1!8oknN8<7b(t?w0*1xiXu#e>N`rQ+Klq zdjZdJu02;RM-Co;`}Lp6J9U?+wJGBRxfdR>!oL%!{>5wYBc(p3s9r6~~ z+h&(IV5IWP%J({=m#*xk-_Ncx7}0Z|Xj?xbi#d-`2l7C)|GqonuAEU*Wt~;TQ4#?& z;Y_UhH|#6c1-6?Ja(wMGALt)mvR0;4Ym_d2A&`t`61g2Dvk6k4j<;QI)XV%{jltT@r+kRHfA5cLJ z52x#T-atCc3eeCCK2}2Hf<><0Ai=g4XjuX^=8@+`yEPbU!q9ht87P`IXUQa5fM`1T z^87k;h>gob<;D48TVB}MhEj3xzdLD~bi~AvAIt#O z)g0tadifokDge$iu4s3-hdMHvpJndksY^@4=PlCjH;zPmnXCbIzVk(9_qs+R|4Zon zcS|aBML4TeaQ;B@X)FgVNoz>ss}Qv8B1NIw7_r2jTt3K)P9pwT;eDNagKb|T1e*D9 z`@N|8E++`(J5YLgaab4b07e7^TOHZHvkEIr9o~sp4muQY^ zCejHhyu{+!{Dh3UND)*}`aU_Ns%BL%-7dy{S4(s%i0=GFiStj;rx+>)5mh{8^QR!D zRt)<~L(wOFZVbTn_8j{ck`I(Z0F5}x1gANwHK;e?X0hx)^pIrc4XnySbNQH!&m9t$ z#mWBLcWe+_i}1&h&K6B|b{C2W3@3-Mbz5UOfDKYKmt=r&9!?cwgT0sW}#fwi0} zl(wa}3lND?H%msVfEYJ+d9ZV$E{I>1pH*8}2k=v4x`T5QZF^ur#}l2G%+1CH_GCF9 z+1_fKXXv(lSS1yC55JcuiVF96g`$V{VIw!JWW!QkzvQD9yln(iom4Xpyg$Y-DfV;B3w>6gFg^5rv(f2kWXW=f*GwhEl zMME9m%zQ|n|7M7J0ylu4s{pfV#3#hX^@&M~|B{cj>RA&Svvn`LfaxUUWr*PYeeRKW zIulF+-$Z)uzN|<$?JBF!d*1Vc3i^)0TtcEsT3Qx4KVO+VQ+Yep>gB|$;?rL&C>{j1 zb0h?SQfsoI@(0rrwb5*TCD8l8 zt56AOsU*3z6ZRVxzfxr$h;;uf5=f?e)3B_{-W5f+#8z0mM$w=oga}hU?Quw z3(f*#Y#bzP9d*6S(Vty`>a4r+gwbINWs#4;}e*+VJ0;&lF*D>ac zNYKC24GYMCG`pE>q-!O>u{{5-888=XTOY2j(b-$1W{v z0h(M~A+_Rqudmm$$xk)kW!qo<8<+>b=mb!7J=4wa<9N?&ZozgfSp)CrK0Y#%OvgSG zeceYxf6wt;^_=(%qjb^B8v2mxyy%``2ud*XJ*?bdgaIS0Zs2q`X;pvExqAKz{y!6> z3TkKnAa!cZEbL*qP9!Ih^tRcKm@Xl%TxmpS^!6kB|MR2@&9WWE)7#&h|0CsJ0hS!w?j)~@BnhTP$4|%o{LXF*j1oK za$^PC`wzY%tMJCyrrlieNV_^}4wjW?I0q&h6@HJG6gBy&QKEo@eXLi&3`jZk^tV{* z7->5fgmPy#u{HhPb4{y@EA!RS23G6j2<-^Zcy~|xuE@}?!6(01q8`np@O&eNCGI<hQGV)R zj$03}Q#g6l5CHd+Vl#|46iCXglD^Ab$L;>4}2((U@eGREl z^#w4b2G#7t^3>(1mxyd|65b??zkk4DT1Xnr?bZJ{iaIdKUC?c65wG)(8_Hp zz#1*RL1ssS9_A*d8B+jbgzKcx$u4U4w>y4$*erX29Jujt(V}FRCUob$;b>gGC(5F& zK({5mqh@FXdq}|gRxlktS}Jv<)7u7%{?HC<;R}k#2GJDjENaZaD}`5d=04em?1Yx@ zE|cLxFh@0&fBcKTtAP)!>47Zna$?V40Fj!)-$JGaf?-d*qQP{ai z{x#n5Up?QLULr^1eUhA)P69>hi70g4My7Evg=a?Q)?HJZO>lYmFx-t#Q9I(@i5SG3 z6w*#Ee{p9X!rG8l&dA9ytP;=(YeyP?a9!I84OzgBpwAfMB4QNJzw0&JU9y7CGO|~) zqmajTF@_QW8fk>p8?@~^8JM@mDxgP>+z#B^MaYO~FBr~dsnJqX)LP>@-g$A2LQWPq^UQh`6`Ibc@sr^N!>&;CD!M6`Q z?jHz?Q8=_*1w7R}2AT($Xymzg{L>a*|aBcd+eaI^#G)} zrT6m}o^-PamjQ@iSo82^zSQG)HI7c^ zFYu9cU%>vFL6xP1SGZCt@^KKk#C==>p_# zfP~{0B=hN@K1c7ew9J40ED0jh#?_#>KZxG{yg6Qb>?b=ZkgP{(%;ikVF1tR7)Iv*F z+U~{9WW{YUG;A!LyYQZBKMO^mN>_6z=$q@`u}?TtUcupR{*OxAiB71gg2+}T4kfkMK3 z>DMDWP=9?e_5qAqVDmK9PrQF6vM~*Cdn?qpsCGbJ1Bsx2{L!d| zCVq+j>`ZafG-DRl!1 z5^Y?jvR8lod}nLG?Rnb>F3z_5?dHqt2jdfK(txV`O5wJF@2viR=K)OxI==(M)NkY8q@(Z(eFi4 z=vn$@c{y5JZ3+=+n|*hRqwt;1yNTXSopleOE`nZ8s%$9x*qT&@fVGHFi;aLZ4&aE z0U7H-T?z)$cw464ZARLfz+E#t7QtY{1WKik;lj9E6`=Huk-*w{h}^>C-xfz*hNh+I zLfq%8FkYOiT?EkWY%5%cOc4I2SpiV=s#41JxW8D{z6V6J&b281FXR1s@>&8&UINL8 zsORBjwooO2Y z{3t9rlw^4R7mERMZYEH60w|ji!N%8OihH-nm?(jYketUqdPg_<2+c$@*+;6nDE^>} z7kZwNY1QxktzMGf{l?{tpEE_@DLr~Qkz|nAM5PiiP}=~%++`U{MBR?)%KT2;8zqmK z&XqEYCxM0Wxzi!e;tL>w)pF9F(?!x&W?i#>Qc}Mk;qul& z(+MCK;g{GY%RV8;>dm*&R=G3(7;hvn`blkqvd3K!X`}#&>=aU*3&M&ZF<#_RW>|Lt z+CyI$-;h0?Jy261&1JefncedB*Am;)l*~ku3w75P#!qlvu`pGKc6xLz^=6HPJwR9< z1pX)^d%LDB1+t1@pe4tDbwdx()*!qBZSTZC&vac`?VW@`?fY*ldI#h-UBDJ8TLUPM z-%A@nHOZl#)Dv^B{^stiscGoQKoK)Y82uhw8-DPs=ZtUsh6b6wyNB<zum`e+wU}xDjxT86!rB(d zLFJCo@cm!CXNoOpvK`0gG}eu~F|>!jdJD^XvwcgP{9X|K=>SC)pq2?B(nQOwbmlW) zD%lrNF};VqdDNYd83w@gMKm`ZyCZV}1*`P`-n=6=)~h6Er=qO@Gp$zpxFB1%)s_1syZ@ni2l2Tm;mocLz7% zxJWZ;kT{?*X?Rk}+MUK!@jaf&cA0l+=!Bv?Un>wN`_%6LuBh~$2TJuLSL_e*hz@|7 zz@@RuObH|L#NiZgnsUHB_G#Do|E67A$-_~jM3O6$_f5g6o7g`!fPm9{6P_I~rRHJh z++Q)>;%?)W!Fk6bADYlEO(6wI_~|!#AOhRn4X@3$2A|4}1hAQw6;^q&+CTug<7C6K4ddrm-#-!EOG!lsdA{ys=)b>_hXhhh zZSBTi+8MjI8&d-Ez7r>CEPmS0k9Z*X4qe?-;wLz}(IY_-SQO>3&M=e_vkk*~hK4JW z%ffxeH{~)=pHM~Dwvuj}M56ua`{eQW|MB-!5uq^F`ZEoXA@v?;x7rpvRFB3I{ybM& zgSMDY$$AVV{J432|JR*ueF36U9kMA0{9lxj3i_FrEBqV_SJg2I$`|NU^z0vu|4K>@ z+j(*ROrD_(>6On^l)?d?JYj0K%U!j0b|89E49-sS%ONKMS04kHVsGHN@h6*uF_0ztHoB8aVB52m)BYPVW5Obnos61r8 z6JuDXAFBcEl6MNA)@Vctr~PA7^uB#l;Q`eu`F0J}tiA{ATJCsc+7cpMaNsTv>Xc&6 zzD?LqwlKyS9lgqUj>ulHi(MP%dO1 zyIEDqg)P>RCnvAo2tdjxyjmCZ@tCVgzB7Z^x=7fnYF(hvAheyS0;H&cj<@%uffaSD zVU_a8kh4VXoQSvtPXQ9`iST_p@&w0VK)cZrdG4ghCEXB)>)b{b?aQuy4uzO z?pLd_AA$emVM(S*#7%wQzCQX3qWlId)VH4lU4c?N9&k&|85rrj=>x>RHRvPG|N>Hyaa9=FtX>ZYXT07Qv`%_tTIxE;bYTehRykBx& ztn6kLr1K(NP&Q>Q7EIK263y2jk)LDW@EVv73MIZZ2M0YU`t-~A&N3}H8%eE8lOhU3F;kk`0Pyzt4wXX0L87?q8_Z?4aFZSs_Mc==s@HqBIOsq7YEV zpeZMVS$=v^sLH|a^9V9-q!y!F;kmxt=tg(3_lC5Hc#QJ1MY~M|2ZZedq?c{MNAa1= z+Jzp-Y+C+Euth{CkFNaC&{>O!SZ(VVz-cXZDqIM$V48NZomilqf$K?4m3J|AqP_8 zN%Akmp2?_Abk*n2Ho$z3`jjvxCt=;pK*9S;*=5qPsf03iN^ELB&@J0kY0sqHVF6vz zX3?5__ql$2%hfIUG3SMBb<008FGg1e4yr+Fb)O) zi({1`Z<1Dee90gZZO08Y9rMjp6wZdA#ThRrg(Lz!wfBLB_sr*7ZKGsq!FmqmVz2bW z-ev_ke0Liv=;7pYbd>hCD&<`R`K6YCUvv+zx;_=*=hhI`m;`-tfjRSn!8NbdvC>+K zt1j*SpVLFmkdp-ylF_BYkE>-6>o@OhCUdpVMI|>6JMh=ME@zjSk$Eyv&w~RndRK?#2UKx>%?q?(;`Byv#&F;S*)G>#MZTq`oz$$*)yk0`T%FG3jN^cS>k%-{DX-5{}v05h(G{kWwm}d{qL}cegwtboAgdYD&k#ymWeh1tj>6 z$7phOpRasM+2FkumgCH34o@-yB{PNJURSILcV)X#o;@J5`Oaq^)5(#_(5Qy4y~cLg zVIe84=Ku{$OEGErgPAw-R!d`U#Ua`7V@kO|uRlD~b}Yfm^ra7E@0fPE;PK*S;SQH& zpQCZnSp2E!6t3VUE6n0FqiKH^r#?ah-veCTNPw?;hi-L zjJn~abu|hh)wGlFdTy=waBLc%Y#lIW#_FO|LP2lz0j6(F4*9}ZmiNeeVc7k zw`f#6;~1#hv6;jL!&)4u2ZsHF?8o-NdUMLI8dipUddK7%n`H)t17_YE^9kY=!oG4SJ9l6@ zrwd~MvqWXOMvV@rnTi~AiC==eN{?UKoEiac{fRmQiI@cwg4B|@_~eijZ)}w)a-amI z_CmvnYUqGEK4q4uo~FNtMMSMX61!@p75KVOR+`7hr44yqmPDUQ_-xdPjPUx#$ z`ELz9Ht66a$ddfzGVlRfjvlM>sGR* z!U~J!$A1cPc;5iOmrl-NJmUv;;58fv`hVz3`V+7h*p}NC$(1}Sf%mFLg5W^z7No!D z?sN+2pV2blW*F0w#-za@K>LXykC(=8mvd(s4+-vze|oU3(R+(8y&f! z9)1Ysz2W85u4oA}*?ir1Mp^_pP{YP&IJtQo5s2y-U?#0)HmArq1OX`oIhD!4p}-}r zYgKmiy*jx+o4jUp?_S&MiVKH0Eu9nMBiCK-!EGCo7iYV^PYLY-Sp0Qsn(eFb-2uT> zrMcH5hWHz17`}RkT*X+qdwfg%I+TN|1^vKXOpg_Z)pJJk_Bzf*XJ5rhl!3Rp3gk_q zLX%056|;UuQvfWJUlp7N+>e;fZ+i|eKclxdpLA*{50V;_W0xt24D2LC0zEv-Q5Aq~ z<{P(NQp)|3R@bVM{a=5fjYDxx)!pZ^67J|~ab2rzWy!X*H$rS8gdbblr?9w{TQFrC zKFKb)ERKrpn6~*!G`bz#E!PhDT)aq#O75>tDPH}} zKxxl94n$(h_}+GYe4yJa>iHdDavOS@B1;aw+O;%@2xgN;Z7HRbUjO9Dp7t-RA!8es`VgJOqKN?hDozIY6Jp zx|y~(dVXW8Df{b-z^KsuIP61iB|KQkT9_s$+R|LeG2r#bqdrzvIrkfEOOm;?zdavO zgbnakX{4=A<~+%8LyhdiNWzm~zKg3S>L~pdE!TKNzZ-rPf6Xe&N8_G3?o&~MEN=zB z#Qwn1kwtZ9q)0)wL*Xxl1!F^~Z}P|zJaBW<%#eEx$ONEJAd>tzS$Q5=E)evw*Wbft z$fD67gB?_n_pYq|YxbmicrG8SR^TXWp(H13W_s7YlMwc!6YFTYvdq<8yUxV}uhU z0St9~nN-Ki&(4jEVd`OlL7SGsgg*Y2OLK2WBrwykj5NB_P7g-HV8HZ7g=~DhR#M;j z=Y6Nj2&T6r?&}lHJeG>=EVm@9Kx6sJRnAeN{4E;|J=57%b0!y*rq{>GJKHT8;}gL7 zQ?U76n*q@Ju^J4X7$U)Bc}eMMo%-Ioaze-oiH%cbZu-HTRiDNSzHPz+;J0u_>n8mu z5oEo3z$t0DmHYYx-TV_(Ml0c+mbwkjB<2683y69_0~2Vt{DRL>=sw6e6(3OH(x&j> zb9qB2|m~NN77Qr>Nb5VOcPuZntKs{GE^x5Z(C@ScY z!}o75t|Tk01^7p7oG@zrP01Pi(gDIdg>N((!6@~Suk``e!*T{ z_b2uE{?3WD3fQs2(pz(ckq09mN3_++_&58FFGAfDRLTBz`2M4g-;MwUgfxEA@_;-{ z#(`j7ef2x+P#OUC>HAF-U)Tr&ZYR96n9L@A>CSHuD*#L?PY{H2P@Oqni-^AN=iQ=U zu84$AtkRTRvhJX9jptN={4Fcryg2zeo?AmlD<>*0@wu4Mw$u3o>Fpd79k82ThhmN3ub zAEcxkW?o{!q0PFKFeI3SE;i^#suwK{MUd|YH596YGEa#jJu+hlq;M5H&nqQYDn^Ei z>4Kt`bk!B&PDHGW>Cp>@pru-pl(hgJ9BR8QA5%iE%$7xR#m2$e^=Je!w$33LT(7+5{^Br$La_D>eRmGR&g(+4B#Qm(Lhjm5mvu1oX z8y3a=i(|Fwiu8>YXy4{+Q}4~>nS1QTS!q>)E?878i*B#TUl(x!GKY@C%W)bf^Rz$& z>9Dcs%Ix85VXr2=`a2Of71f>gU#6HZl!G4ViPL?Jp_uCfP=!O48_A?ZxUz~zS`s;) z<}Fa4EmRt0NifO9e?sIBy5Nr%H|FZ{BBG}{#aeg)6R_vI8**VCYFl5#QW#ji;t|~? zBW1=tv=W=l#i}->8hB>!ltytc?PatAr^QLmXrapJu zec!g%GGPRNR}E%RrF5Cy9C^eE;|62n&()q}=?F}*@~cmKD6o)e>*8ZEvfe{Q=sc|u z6n+5+^m)$==^KlbQ1iDrnsE)|^ybpY3f$IoWE2!|9UO_ylr!s|ed`XLe%f=DBXKM^ ztYWg_>pos2b*SZLdd{YK<%N~`5l-ON-bJp6m0ZmYL7!V~x&C|tCthAxq+8MTL#+x@ zCb;rIRxOW76CaP6FJIAT6OYwrV*>u2REWUxxLfScM%7PVa}{VIxz? zbZ6WWuqcn2xIU2y$(NsVt5%mf5_p4O2Q_Ns9<#1JZi|?P<+9}E7}5(q1jtVBd7`4= z#vP835?Zk-5Vzb4O68O)C?T6tdd*TD3^!RhfY}4@0$Zt7M^C}AL2cX!?C_iT7;Mmx zU-)WEK(@zieKy*7I98?9iceB2KB|wuEa={qx%^}+q^|Ua^4x54JkdM=$5&=HSDZ|; zx!h0bnny(`F<-~xZ#F$})Al~6`>9bLMuv3)cU+c&USVnfmgFNX-3fSGVaAbMMS$SC zfx>_Rni5@8qJJ$GB^1&;^Kr^SkQ7n%;ao@1=!n(V<((`^Z#*bOhaEGftjg?Cn`w;DbNX!S$?KYPAgH6HZO4= z;6sMEOKMMYcM;B#p4AF~C9L%l&d5lW-P8|0sS4Ic`h+?5FWeW3%mjPuW+W`>FxLljixsj$lkd9Ai^2p%pc1~rjsi-EV+CX3)a~y#M=CK zO|iu7e%UP6%Z>Szki~7woMuy0i6OCKLOjH(w=l zxszHfI?u-f1EhjOKDa=+$(azDC!1pDpMK66mIpj=Di4sY@B0`-$qlx0lBhCtNqaDX zr8Q*rgJ-s^2JlNUL7XWd+(DjJ!^Gwm8C}Ir$a77UU)Ib&>?HFXRlVnPQW-A!0nE2M zmr~_V1A#sReF{;6f1xK2%b&q!5xeHT1iNaCQF4V#pjhp6%MOa1e*h~+(>gO-R$s2M zYN!-~twD@C8#AW4BCsC0d(T$&9=G)aiao6+iOtzERf)a&>PrTYK`1Qy6!%}U3nfz_sDGb zG0*Js@$VRUMM}NgBRy~VQchC3q-IX1z_+N&ksLTbNqn`+66*UJTLn|*Ru4Bk0%8FXYcyJ8DA|P! zO@9o8RZYjZ{-P13Aq&;yp8dA0B*!l;EG>hCK0Zd1u0LSt{v=sgAL}vWY$9);A)M29 z7Ri{n8O~%n`RX7b=fe0-QidCeFI{`mppT^TwrS0`%K7bnV|!?M$QJ{3`SQ^ab-X%h zB>60{=?e2bIbP&^N%Ob?tcBN2eI~;EHIwOTk>qXpqW<^l%*iv+adFsN*%_OP#FxkI z;x7UHf>ng-ynlb^W`!#pJ0P8)9T?@WfMmVuuejo_zZtk}_#>^QnLH%4Gg1L0NDMC& zNiP*7D=Hf%n;0dL3k*cEsMbmMQP&Dmd|4C{NEVx3TmA7Wwvr?;Gl;is-1$YB9qat@ zV1Et}cIx`EKfX}T!2aPY|1E_d*lp|bPo*{hAc&84N&V@CR&C%)lq3gmndW!c-yI?} znld@L>rNs-*Dr5ET>_6rHX+8k>wHP2@bQPocx!+*_Kvnz*a9%SA1P=OtcmYo-62s5$x@Yq%zW_-pheppi z|Htix@807pf1XbX?N1D}bghAL0PWoKLcflzLTL}&1=l@1aR&dx@kAI621R(aJRW%0 z!iqu){UQeIXyCy0PD{0&GBi%1Eac4H3h9XjJ|S{2K?bu{0F`7R`?08uZry$k)} z^>;I1>#eA0il~2#+Wr@oU-^LF(u#ax_XiRv_)@Gs-vVH+3Do(Nb|Ea=GmF9n{@ISBcqhR+xRrtT73U?F= zBDURoRx+9|(tK_>yl+-mw#t3D&e_w%_Sb>;{lKjV@Bi=bRRB@x(>-u4A zIy+W3>6vm8UC))1=adgd(p+)BdB9fhXb;DIH*1n-PD zTzAIXxTXqz{!;MuYu1Sm1KNHJBw8P2WMXE<<0on=*5?}15&{-`3Q+S1V&gNKx74y)oLW zme!VczfMUhj^R_V~L4w_$!TY5K!4o>mbMjGcThmD53h zmiPRz+Zr|(l-_OqCQbP{o@bv8f273FjYD6(4kqz)k)!U-*0QxMk4d7W$NdJ@`lnXr z$92taJ(n=58ak4Df9mP!7smGnp?if97hO!VeQ{iT=XK+yKP`Wc)H<6WY0D#N4llRsjkfFNG;n5p8=F+3}iW{?y9?jAYi;vhWOJ&}?>-QyO zH%4>#T>i#ZIY`^^H_2=Tpd&nHQ6(`K*kyHfL3Y`FpPJ2u2k;NKAJ`nAq(7d90N~8+ z6A~GGt(N!dtYDUDkrRpOxm(a0&ZAjzbG>J2AZW9&=_2kUC6k zw{ayw(?;uh>mty7V^p(Q*5eWNq6W~nhp{tXqYP0{L303@p5Z@^afzEW8Sx*|5wGkT zBD68l97LZMzs`Vf@AoE@=3M7JA5k^!;^w!Ip7g@B?e8@QH5=>)`GY*7%D|Qkpx#Om ziWuU&cC#nds|d1lGC9Y87Z2%T<_cP$CEUr!@VA6ssr8L@EuZ)#5;z;e7`TQT=wIq- z3G`C#2MqlOxpP%F z5vLw1ORV(hMUio+rf1P?&V7(Bi{JwLCQ}F?zVWP(+O?^Imi%{due5AGTs+?AKi?EQ zKi;xAY+Rh(5&j`v{XVaf^+Uj*4$&0)SW)y)YI?Vll!CABfTQ=kWIfTAV$du%-`GPD zU00Z&$({9Qy8bqZ(z8pgXVc>H2LYEs9yW166j^66gibAC{}1-QGpfm@`}YVaA}R)u zBGp7drGxYmREjhaP{{P~hA1 z>Zz;>4@?L%iD*dSm9>uNU$RYuL{?0smF|C}c@N5qeq}-qAH04N1L7BQ`}O)R*Aqfl zQH3kLX80qvipp3PE8zL?PQTjsikH{OrX|uE$3sl>@E9%OC;Rdxhgy&ua~>nJ%dgB`ejCP zClL8N*Wr|psBwRM-dciVBalYvOy_@ww_ESt9PN|1-`xa=o%7F76GL`$b$jHhOeU|3 zv4k(w!EpV^*&QB)OMB+i(EmP$BslH$PAW9}<*x5E=-C*SyWiYPKXDtrz0`B-R{~)L z%peIEq0H`e?LOh95#;6bPZ|UH(a37LtzrBwA6eOex#m8GKBa))mvhdf49MFl`5A0~ z0PyS7XO$09UgYZyxvy#$w}@_W^OC5W^pV&WPf%x#@ z!I#83Dw(~Nzub?SaokfG)UBONf_(F21?}< zYU1iaqbz_2%*)rmF#~u3re1B|K_(}N*b`F#MFa>G^xHNLTNDiNvg!b!b>;Qeq-sTUK|Nzk zhUvpf*+{k*Tz20+Nv8kdOXy_*%ZNeD`~XAeCdw;Ioflu1Z_ili8gI@N46P-H-qslJ z8`@ck1zI?$9k9OFJx9Pb!@lR4FE}iG;%v~ioe(nvT32K0fz|%;dV-f(i1z(%^F5UZO{_#j z_ptT4Y;Aj9WckvoAa=EPdc7j6{Uh$}PJdY(>CDMUKOBT181C6`dzYGDITooaD&oSw z6zEuLfN^pyZoe(W+G3Ga^ewtS@@Q(x!#e<~cy&kb*W?N^TQ7f6ZRs{mf2-2`Ldemi zl+_B$#ddv~mPLlvrJL`LQiVelPz6)e z*opK%8esNb27E55RkH7uDrSBMr<-C z3oWA99J$}5qra{gvB8!@{=NhBD%-n}K%q?ji3BfNp|Agtbd~|#Q0u1Pw_3hhHl=0% zn;#BbWE9yJ&^vP;&QklQ|&~VErW4DTK$J)qZ{Z>#A zd}l4}mR80xKreqcsg_ONlMNJE>1Kd^DfqqT#L06eK|q|kYR!cIm~$$!vcJgAyBr`7 z5emL_TK?fjxV-#Z^HuRSZ4%^7b>b54MY}tjDLb1LyPMN9`F)lnLGIHAQ^Ko7E!sPI zS?(6s-9@sFs@dj~{+o=EFTTg^cXUP6Gjd$J_Rx_p@O!WnE#=z0H@vhH%Z+(lFF6fr z{9nE;7<8#V!`IK@9;xl1?odnPD5v>O??di%pWswR&Lq(OZfk2JLAt{Brmhu>oE#kQ z)cJ%jwa({rlMF!yE#FPQ%#6D!#RE}@JjX2i51WYSf26R- zc;d~Y-DkgI^#K|Ej%2G+Ow9uf1o^@;=SziKp^Z^I)1!et}>IGx?W=~`MV(fSZ zKTwI0=DBT>G)2J$mn3IC{cqsX@0hyy1u8YV@RH=0X5`Mf33b5eaU%ADgA(B3#M-YY zD|dC{q_)VlL&SVUmmb3@V*z~HXsj@FgzN`QSb2%;8R|KTsq5wMuK9z%|Ao*2Uzixn zNtrt$M-KQHYYW~+8`zob%aeG*e(S0;UQF?B3mR|r__vqZ$#eDLE7y-Oei->P)Mcwz zj6ViyHNf#)tD<`yXH!FI=3mg4W89*X*DN@7S|a1ZoG7+*>)Xv(={Ut|<=oh>tRS-dlCNUDDJURo*;ez+>C@&a znMQqPXU=O-j7Q^-{g9R^VPso1YuI6!ItyQT8x@ z>Bp&*!$I8Ex&7w6)*i0*TxyfT$Lbk;jqHjI_rln_Wymf-FXaUOsZT@BBjUO0Lj>Sc zFXdTVrJoiZQ1?I4<2`cz*}@UpGGN&o>)18%uvMuqhrar~w*$v_yi1&n16S@e}-bVgDgr4omTl#W=ixZdkiOPHO zQw*Qgq%_lu+~%e4?cNSOhUC)zycB&5zj4z!3TYrZFVRfx|E!!ut@z>hus+KYi{5&s z_|k8*KxcmiYLI&kYqEUB3c-!DRmYAXV>ZIm4n^)w2cPybT7Q2P+3YkhrBk5Ob0UIO zllPtV*N;r)quHGrW^Y|~fdw%SFj{%F;p$6lv#6h23{njQ!BOv2@7>4DcQ_I1IYwtJ zuo|(!ftihp#}F`?MiCbJ^jnt9s8HK?D9UzfJ8E-tG~m2Ba$Gc6I9k!$JqHR@PkisR zNM*DdGvl8)!JA%pkvx=V5T-JkJaGcSd6J^api(Kk^4*VPAVYr~ft8G93%{I!dDIwj zaWCP&iygd4z#o%hr9Tr2_M!0s7JPG&-bXQDF#)9c9(?rsgV+neFEq#_Tl`ZFi1FZ^ zNN+n+cHqX$l^O;vAL_!T&$7llSX}6_yS=L53Xb7xHtphvdO1zJnnhn!G~ad`#aO7+ zI`ml-*aq-)ULK?`33SX@%rF))tH%0zs^ytAj>;^#TY7poH2e1Y&Vbo+ zNFr|i)h|*bUvqD}cq)@>s}f)I9oBurFXX^6=^BUKG8*U6;2@8AFFLn}eysUw0vbud zmED&~E=FqE)^D{<2IhL`>4E-nosQPpoCxkWs`#6)uHfJ`5KvT&#dbjNkgADTJ|)vu z*Xmqt!S5U_GaB9&lm7_Aa?8Vem7cc_k@N%{;lRWRSRwxb-A1c-=T2uqIgjy|H}`Nk z`}EhbpSv}(7H9=?Ghs1N_#P~9oh>rf=%6XxUg^W11Fnt-K%e$}+MF%1>cyUxvqIDF zs_Ay-z3F-YwPF~#R{)+}Hi)Rrr2<-T$0ru}wR|ReK)cY#V5_i-l>zI|5tIJa>0qmL zI7Rwa83aIt7Au;A3lT+)k?eHcHnlX7)pGt)gKIT>qP+H#?N+$?jcg?u$o$C8-XK{y z`Sp#i3eoYL`0Z&8<5-)TXsn}Ty72TD-0b*_uGG`wLIDw)M+30Aw}!uawiTGvIvjwl ziR4Ow@mTJu=LUgMLRbi4*COJ$v)rW%p93yDljoHt9a&M_#5|^I#%NWiX#%; zSsTG+cmFQiRWX%Wa-m$eA&Q!_BQ`ye zO`j>O7W0q5JiJrD)Hr?`C@d`JBL+#4F*aFcK(M66Z*-)Pm2Z5>qIJYca$1TyxxSPF zFXNbX=G+cbzO9~9ILoZN-f)3gC4-{ds$yH?vx?2o5$paKj>0M7G0i6mY%2#L;Z|CN z%22m!&wE2^U?oDaJmXl#vGxCIz<@CT`)zE#g74?f*$f1bvV=s65 zRz(CCUY@Ta(&cJ%=jq+S`VqH=y@+m?e2X@FJp`;PWw0XE%-u>{BDw7eDfq^`I0;2S zZ_dHw4}^}BlHPoMO56d&aZwmv>9dNDC>sPmugHR=RUqGMC%5AsewH@do;HTZShlbBgY`+Ge(4~T0 z$LVWq;rG>6EPC|@l73aBH!y$t@Gil9VdnG(7$eFerTdnUiyZx&e6V&L+|xsIunYAn zS!He&VCR_-O<*;aW32<$$hVW64g*J;PS8M18tFD^z86Uz2-1u1*1yGM#a)5$^;k9< za%ENQY!# z$XBex+aK$^w;P;)%EzJu-0Gh`nlNO#AMDbPWgv)ga2Y0356nlCsLIyX)&bVOp7}s@ zh=LbzkWNVgtDkbCn%#rb3hO@QY~#%YAAi8Q&8g0wK7Cr-V|{E9YZmAC^9!$3^oVcv zCeHXtUzmev{$t9ACwN)!wBtEV?E{Bm{#jTwGPtS+RlS;d| z#n|&$VM`_gY#?P_jIU~jXP364{Deqz6W@^V%oy<2ZZt-)#w={ld3t)35L3M6ci22o zSo>-(R9)|e=dgkiMeV*ETZ*XkVTxO;D**%PS$f}s}Ah`+4OIYxT7>K>zV2ia?b1<$`slOG?AfK^IfpM zFkf{_vs#ANMshK{78CYat^>e_x%wV!VU26ap_e50G$9EoFkt8tnl^#`oAlv9iT$okx-x}(jJPkdyS29v53;oLgCf3o)V@Gro0t6G@*+Ao*cqg80 zrlR<++veo?KXWBp1%S|MRyX_2|1<(AWZ*5nrtgO_*Xm|0XxZ-9Q2Tb}D4F+o|0+8N zsYr&zm{7VG-o~9_IQlsqvOB9jyQ{`Zl=o`Uz!q}x!rNSbkQPAb&-=7w@H)*O66c|E zr8m<+M;9;XUFG0=H65xVUYc@!Ik?8Uvt`D1vp?v@BQfLS`7CYyw+sqB7Fj3BoemqL z9O$nXz7v8=*Y8=qFrsH_h(o~4*Z$i!{eDY>0a-SXV zflb$fI?`UIJ23%@R9UK0%9^M7R5c@l)NP$Jvsn2EVfRRJ1|SsvMvH^I#*(C=LDhpD zHKrD)$-BlD``G|ZM|bwBihd;r$Uxv13jFpJsmd$xrcG419xQ)Lr2wpd70)+^)3*Pc zJKKT>c{Qh2$*zP=lsd(Lkg*2iGaM2Xsq0e{hR!SyxzR|3DH7cwH3?aI_CBq&>cH+}v>NHaal;WVWSabu6m(&qoz zyWo})uI1?=en9tPW-vq&uVjo!^pm!M#aH$?Jcy(bX^U8+16<2eg3i#RTYbWW*lPxiBoiCTXyo=u9;}X8%y==sSLZj{~Q=1 zhx__`31=C+&ntNAwS#D9;!d7AMd(iL$73rAVo^O5V1~m!U&f@_TTOT+XLg5-z?|ha z=m_zTM5IoV9-lNe0C$}x=L=7PP62ln^m4JkzB?ED9fBa`7ff97G>@uHvV|QDU1QUl z0*t^<&sDSdd#tBf$Ch2@_-uYUhrg7R1XqdS>$GVDdb-()q) z-KF$*1Q13!oS#0X_NRTmh@$}E7u4a|#JLW*NBIym96Y!)-7(^oZ2nm10;*KLoHx0K zCD~*gy9n$K$z^Wrf1)Gyf+&Yn0(gi0xMU0%Oj378jemlcG&LUkAEG#M?O9pC9;tpg zY?JZhE`Khs00-7o!$Fv2<;gn-HZP_-k((-d$$J!@33`J2oAVf)3tyZj zBk)DVy9=>m%UI@K0QVUsIN}8cW_Gqz)^!zWVGG`VJ3vciVo7`N+YUPZh?cajheE(% z@yD+&?9ITjIX>=Z&Df8%(VY9S5d2o2>j`IYtW2v!XF`BvBBZ8~2U8t2`(r_O2oSPK zV#^L_1yWYm*f| z7_`@(z0MOo7M&BgSE8rItq_B-m2JR)9Q{$8<=}c zdpPzV4{>btKA49jvIm@T z|LYhjAX7(p07ZcG z877#Ifdy}_!oBhEtV|GzJAb}z)0;-Et^7NpA0c4D7yr_Ln?A8DT-3rF$;haA3iqe0oK# z*`V*LegTq8QeF3A>nU*Z&imF_nUT(b9J)Ac?opBQYf!FoS4+=uX|DULXn8K$eMWNi z&AhiOU!Z0T@VwuqH@+}Fw#DU~t$<>Mj7x1n^_!6TJ#0E#!stO{iuaef&(^5Ot%@pq}$~VvfOe9m&Z&2$T3|_LoFO zxWO`sVd-`%rBMjuDkcdjJewuiPif9`BQ1ce*#WmyV^(Y?r))9@CX6+XzaDsx>=k|%jv2K9MnHjDiC@7#i% zJQPq_Pb0t7*of}vIrZEk8kv#0-~rW_L`g#mq_!@hA0s6oLxF0{H!ilC9vfMeRs{cm z^A@lh8)C~qZm(dt^VF!f(7Lzv4kAPNo?f%EF1DdRBLC#*Tr>?#lpM2xj1_rz9R-8r zEqr!b@Ix*_;~>Fw9GT?7K99Go=L{j$A&o=S*{-Ow5NCo9{j<%N)q&~M zP`XcnDucx5^ZnMOQN}P{Ruucl`Yl7~cHNHWMJOEHIq{g!$LY4X2ks6Xv^jvE5lr#kbVmUt2`OrN z3c(jIy#$Rs-}uRiv#c;-)|pY{0)x!t8~GwZvilWhX>}B-8wI=^bGZgRlD`tdGJWf8 zVr!$Gt_yocTF?gjg!8~iao&=0c*J*H^N7!#$70M8;yOO9y(-RJo#EHrh8{P0cT?Tn z!UnA9rsCn$%xKQtx-EN@6|^BNQu38H%og4VT?iWOuN{Ll$#|?qxR+$8Kl!+@RRP^} z)J4q^Ha3bj!9OqF%vy#m;O1>vC+zf9^~`a(`k{Epfk|XnOJHcFbb6wKEH?LS~ss`$$sNX45X+!fxmj3)7S~ z%dUVY>${DQKQ>HPsX`Z4*O_3=z7#XNMX8DBC<-dm^&92*NkfgQFr4 zcR7`f0jdk8LmEurv!%yM0x=fPfV5?dQuW45Y;mBgAT54Q(x?i_`!|MDAnRY1lNMg7 zJ-3LwvffnM-!&AT)}PS4oIULbDGFj?52_olPTfrsoVo(k#&e|Dj!qf_$K>kbpxQG! z^G+rX)8z%n1tW-nP?E~RD#nSpC~w9UgMqbu*9YO1(<#+{ACOgkgd$9>ClGR1cicm_ zGie&RDKRTZ@OKrFNcW5A-3~I8L3vgY6EyVMwi25!uEmOunN5Ze4|>hxgLN#;c@7Vz zW{a)c{jIcM26#BiOdmM;@qi||D^JgE|BTLK+u@(U8QQ5#Unh0cZa|&-DS=S^K4cMO zy>^2Nmg!$-zs#^pzm|oB2k~=3cj;TK=q+i2r)>L5?~3W1xMZyLFlU-7U>qM2l~BL1 zjdauY$F;@w2ttsZ8uj-z8e_~zRThQ!ZjLFT_WND#18+lDV-Y?7Na^M&pYH9|$aMRV32>(XdZeZPN<~ps?#a0<0 zPs5`FRt>j!9YB|8gft@ue|=!>SL|M%!eKh&2!&@@-R1iFjPB`J0j^^ZSXA|U4xL!y zU;xpGW{4z9nG)4*BgxmV_X1A(bQ!eeg_mx&AdMwQ@#P0tDkv@8s=n54(W4tI9d7jj zmHwpi4J6RkTP?D}YnAfl?VZ{F_wROSFk5C2J}EaLbUUO=V#%(E@2rkeO1vkIpxrbq zV2mb-ut3uPTEybElt*@wilq+|zLXmS)Y#gM7fl5j@Tefwp3tbzj%BE|%jr=~XI}HJ zq`<1L#{o&j7@@P*VgNL91^(7D$L1vv*P1+aTD)uU0;UI_*X@bBGl(;^P`?B44NHlJLnfw^W}NOJ)*5$YGe z!b0*k!4@ivjklT2~fFlgiKH*z80 zF?{4T1A@N}wZ&_$X^+t+@n%x|Oy4EZLDlmN)q+9_IMuXX(hhHz2 zdL%Y>oEJ%JNtP`uzH{uw(CoH__>N*JG_l;Agm93?MnhcZsX=ZCpTEd8D4c(@_QqIj zQ?PWX|I3fIdof78`v@uhu6)Q$KFod1@=+SxAKhl}St{}E(UPQi(x>W?{ZQdlGp$n#So26A` zZbaATA}u}tP94~;+dS2;CS($XS4`jPn;kn)&Y?etjQu-2N7s*`Rb*i2?0YtV9j}u-ITgU@EOzuT!qcU+d{WGEjdNu{UuvuisjDzQ1m}Y zIm>Iz>o3(aD-zY86pslsqBD_iry_P`OPlc;dVuWeSvvUxpb=;he;K6aypy?s%=4f$ zTO@w$sh%26hxe_VkA;OulwzVzwv5n*VxK4Dw4+ZUNXdjD*iMd; zOB+d+79Y3B@^>GNl=C$ZaEuu-Npky~e2r>c{*3})bi<6f0gZtm_%d2yx15J|r=YOX z4u|F(&}ZVbQ**&g1zYW7{sBXY$Z^oSLIwbKX=e=@(qlLl=hhUb7f0-ahbo&ceKdjy zNrYVy5`Pr~i=)Z5t5>5~&iB0>F;wffwK7Ko(M1+Api#(azdfiAZl9)3=kwur7&TSM zctP0`z3tINM7!X)QKrxl$2Ep>E*hyiKx1pRTlx(eB&9K_#_;)i=njuvawPtEAk0fixVqeF&~&vsJ?>OrB% zvU`DDlsLO-{-fJanWOklpeSo%oA)lWf%D4Swq2%Os>XP_><8iz)5bYO463RWJI2D> zsTQ6Z!u3&}Q(ZvecsyUwg=k8;aR4x6foPGy=fYSUqB8AY)MFOHk>h9B>HKRKVII`M zbG5eAUu2qDHl~oCZ&)cjKHb;e>5spw7FXX`5x_5s-X)75j~69nzYs9ys_*RalEJg3 zLng^3&pxjfvjQtE=Y_I_&>cYk8;I!G!D14bENA->(VRTCk#rlVPN$;U#9s>H_$P7Cj)DDDP zQj0iS{)ZW)K)%+Zmg|ZthXCvqxb|g!akbij@r>zM4CWGi%hdo?ill-qIGTG{7aO9k zLu8@8W1$!z7t~h^AgdBvb|@xjIa#npXQZ6kE(f&F_4+`A-H5$T@s-uc?N~ZYCbF%; z2^)Y7*QhTcJz#&*zwB@K|N(b8ang_IU`Fg$+!IlQ34uMVXb_=RP? z{6P!SjeR8vc^m={sFVG;t$^}|mD4J+METeKQzZno?KkA2ul>7xwZS3AeJ#G?CIBzn zS0i3J*=V7%Aw@G*&{?175KbjZE4XbuJdmFYos8^nF@!iz$NSb9Y{~(6Lt-ZHFg!R%iQG68G-f093ygsR@e?kFWMM zmNIY6xIq|UMZUQvhH8U2N=~20cq8@KolCo~>W88~GPZODjIDdms=^kCW+lgxvgc9A zaq&@a`v4gaGfub~KP$Qk0x+g0aUD_a>;7x(o&lHpte)!z>S&)ZRS=vnRLKn<`q(UP zH*>SasOj6>*7T?#Wc|9)_FR|RQrLnq=Bnr1_*@Ma-NtyN|4QJRJIWoh%)tI6#{%^N z5~X%ov*vTqhwcX+@4{-G-NnlX1@w(j_aH%1mpzF&JgBxVJ$yiItgx_J!wYHg1rlTh zc@F^Oa-xP+w)LG$p$}o(<}JR<>!R#*+yw8S`ne&SEu1lPyOWL4*?K8RYE@N9VPeJC z{&{qCK`%sw(9gcU7~jk`U;TzGc^bLA=*8JTJni`4G*yKHoV;ePU7iJKM}^}HL_hAS zWMq6s=VBN#QUT>^!}ubN7~s@>0M1bmT}!|AVQgLMdkk!Wq^(8LO@GZ^SXgm^Xl^S> zea|w2yF0D5hn!m%+U;VPGRkv(nl#P0jxb9i5;p5IPIODY@BCJ0dqizhv0rqzg`$y@ zykI$2jOq9S?svJrjKZ!&{X@+PQSVb=Bj5hwA={=MtNCooC7quWu) zhdzGB^jiU9T=J92yXL+8n*(3VzH1`<98h1V2d$vhB%@Dd$q4^KpG|y_32)=qm~1xo zrZ8o%m@hu}Sed&}arwy9c<0Fa0G5`m_A2BqEeE=Hfq5~yL)~ci`-ci}-jBQ*PZH!4 zQ}ono0Kw=lF`S>e6@@Ha!3^>AaIjxoeu;bw9c)j&9aS9@ks8(&-ekr%J{?p_dMmqw zEyvp0=5Md@&Fc5<7g*?866NMu{JWimsrrsy{REKaF33RHn2QwxanxRBYuo9aOk`=c zL-Qk(ySD?LI2&{*NX8N?!eyakB8pr+HWMUSJ4>T4Fwdls6#>szQY6mox~sA@iMpkK z=*=61x<=*mbJ`d$CwFO73M>=3 zb_>M}wS8Ju4DqDI1R%}fQ-+Dj<)g1&0LwcA*EdHwW zLvv%~dO)4IC;+$FkMgWsX}x5t-TXA^65#X#rC|K*f%Az-xxeneN7q7@$6l(`)<-go7(g3H91t2 z{du;4E_VO5rul`o5DIvZ%c_K%C^`TVb?d?1=Ab9FQukxX$*H3t3Jw`N-@oRPi0D@jQ2Go3F*{N?p{gxNE!D;ZkUuS%nJr1IFww={M zaZWt>OpcPv{+Ng-*?;SNzG?tvF!pmR89h52j zfj5&J!}CWJ?cgY01L)Mm(%_(<@)0*Uux+q~25e}v!)tDie!~QKecSqLE?~jU!ApOA zNbtk)(l*y@|K-pAZ0~O0?O{NtK++|%GEV?!Bgu1`ZD)Dl_>qNqkOM!L6c#MKVST2% z<9-_@3h^YvY$EBzw>WR?iPN(LAW8kANz>sl59NlRn|z-S%;Pnb5$e@s-tZI$n19x% zkY8UC=*)OLqv{Vj-5+o*rCiy4c81Plf{{G1PCqA4lof0x;Lb@o6@-I-N^nMR+n}zv zS5)qr4r&(C2-Bglu|t*oz-<%H9z{{mJnsB?!BH)qLqNng`;5FR%HVfKZM(QngT69h z)_lK?)1F)k6=PgHJ3V}?ptx)#agHvCp9NiOUyCWgY`~#$G;Q@ms`dl(A!2=a8X|F$ zQzq$sn~f&(6|g}{8zb!zcKRd->_P*c?ONgxt9$U$%%I~dH=-ZfIf ztHm!>LGM1?;{xUjGS(*zOG`0)ASwkPWdKqCV=obbr~Jpf{4+MUkgSZYRoS_G2qSZu zggLw#!TkKQFH5&%2fg%#4Kro}I&1h6Kjy*iHjLKXhGi7^im?*{RDI84>+JEJ7qHkz z>H6A(%s=|Go;?YQ4)CAx95QVy1d)>z9x47Qpw3KXR~umBS5Y1+u8<&Sg2*khu{cP9 zxcP-(zW!}*q-S6q5xXRFQl6vPG<}w*?(5O9`?KuIiZnRx^zWpoM;u|Dl%k#0PYjxE zv`RWO*qYhDZM%!>#9e8`1%RQ>N@h16pz*yI+?m$%ZgWsl^VI&jeoKKaAUw~2IxWr~ z&qZ1<7jsR$#%gf1uc^%SJCm|5C)@yaEUF z*b3E#2sx(=7Da*3)b$dXiyTpSl2i(L1~X%rm!F0BwwAt^jx0}_xXP0uLNBcclPDigZK-OP=w}J-xteK*28Ro8tl2KvrDDju4 zueA;wl6uN#b}hp0-k`N(BXR)dYftOj(TIsx>ZHH7K>)7|21=F`4Lu}PV{S5OY=;AHB=N4E@?7=4j0y;>_ zjZ6PFfcs)R$KxOaf)Dt83HhUeCk#Sq@-Aaes~OVfx$*2)`@qZ4#W)cbVx{(^WS4-=znT_^$|pMc+_2&83y zkek>+ew7RicnccKe1V@sOQ%24Ks-fdMA88Z#fb%uZHaNRALmo5$LPGuj*Wb{yN#7Es z&EOJhA9pWt@YcPB@(a#6U%AHMiAsa!AfA~JTdAV>c3N!^I(ZxL-`BQ`v8{n%?!tx$ zKJjWJ3-^ni78ID?_d?1+AmTLf1rw8^k9C#`lVZk@eo))jPxm&*0%XaVz%sQD3RzCx zmC;jN0$B_4uuPg)#_B1b@RMOPjD2;DJz-b)!qSq#twIVXN7wo{eN>y`N7c&t64jMn zu16?j*7v9nxh{;QwmgmQa8hM~U(ydkO2u?!KNo|aTtctCg|iR&9*+uhvqAMNGw)qS zi|7QM1adpu2M_hi8`k-fiNO`fuE-6(MUjOZ==)jmhV$7rV6I|Q$8Fs#RrL!52AKpS zxP5F?eDWpVgmANqLV|O0W~tK>?eR9m5xq!t*L$? zV0=AWxYn-OQVo` zID(nMx)N6*d1IJ|4VI8dseV#S7R0de)=c<-RQ6!5Q?|-QQ@U2N$T57TQ;2#tYXxdO zoy@V;j|ei)6*x(wo!l;$zIU-;RDaz+qnfY?($0XThq)_~z4>j$TOwgTenjH2B-xgz zjWhZ)YOl{dH^)f|NVys`6dSPKj*P9gmqSe>HR@MMtpe9H4MfXStVxx;v04@N`dk~l~Y%~GW zS9m+9TN5hrB-OpqPUr8Q>bwS80y%v|>Gc&GNUh|SD4Gwl{$au&$s!BWp#ae;>e?bU zcID6!mkOku+zx%BIpy23_nT>7gt+4}C^6rWh7dw1t;xBRTCOLR_em%$gzvd;dGOw@ zGlO#d9lX1%LlR8T?hU<-TuVo`gFqh8wiA!UjGmo8)Vs3;qjc-ameG#dt1fq5MidOq z7v&9t*yChpc8&#X*5AaTX>wVAU=g}=&PEya28oH_nh9_m%1?j&2-7!Qn(IVuOc-^t z`wFn0!vV7Gh_lQ64dVA z0~xpU<^M#8{};Lh00xNiANvJh9s1eGbj6ubU8!Zi>s zH(@jo_)oLLj@Nn*5b6IhBj_dcocg(qf>4f*U=^1v$egn4T=$mma*XV$r@^~JD_nqPF<+<@hgW0QL;8<-vq^xbS$VkK1 z25)ImRDtIo)(BzU8%GK)yMw)?J$KoI`^#oEE&t!ew-RnwQe@&{La;gNC3`vQ*bNqB z2DYDZS>DldwL_|g0lj;f;MIkHB__5GyxGOLE(oHPGbHHIoUzCsoIPS%A z=f93{2U|jMDa`agu*x2dK$<476@E12Kb-cj4++VDVWcmAIiTp#7a^Ll&7(Rs&5r)z zV5(x+9Q%+vZ+@=j*MnjbZ=%(VOOQn#G6K-E|CI;;2>w?h{Ff2- zEQ0?s!vB|yz#)yNLs84wnrKkJU78@V`JU>R9uPdo3<^w*#XYeQCczkBewS}|zhBg-eFE;$@H*Dr$LgF&I+S~Mfav10~~I8MAr$KQ}jSg~l;_=tw1Tg_JMQ&}5p zSwMXVd1TC!V6Ib$Sh_4eH(POYz@NoIJG=QtdsDt~xN4zVM6xI>b<%WJ}T)pWHam4S6qcic9I!}KvN z!9&$birLJHP6B_`%5Ivh*qL8{gtaHj_kPIg+&10D;$OZAv8g-{>=9;<#}9+B6|4mZ z6|BD)gNQ`2Br7joSv^q35VjZ#I-dA{HI2nDqvucol*!PaJZgKoGv8V1O8_M&GeJn) ztU|CB3`w#pUNj(tydKNXqOg67lk|9l2USNIhp#CJBm)aa>xJD*CcGM`-gKaZtoH9z z69Y|WB{z9ZGF3r3M<)30fL)R?o+L4#(&y}>eZ%NESl;Oh7n47w!^bIO2FSBf`NxAj&Sv$LWDaLP zVba+KS&*;}Na&*f(690`PR;CPF_ohf!865K2E5h<+;?R#Fp=gWaB(DHu!7XDe;BO^ zK9wHXrTkPdU67NvgthpD#$^-;NRRsuBt8XsPruDY>=hncz#u6JejNNeCP(JQl@}LV zslchS{5BWt25w84paFw8HisXn7eSO0Kg)IEdeByvnB_r1ZLV+f0I!xq& zkVOVBU3$RIE-M>%1k=nrYdtV(YI&vDa!IQ9{sa)x?9b?k<% z{Qfam92s%2g7(+*2sT2BbOWJdH4ELP9ax@0bP~#0Y_rYmH`P3{BiLzi*Z`}|hpKnK ziReCi{X(zS6W|NKhO&+br2l{3fcU^W-OdIiVk;4jS?%>%l5BMpHz*MR*+!p{W7sU4 zBa8+pb1Jej+o==125vdK5 zvOyugfo-gm)b&SVo5w{d<=w98+n^euXF+B?xvE4z1e1%z zKvE>kA#LB^OQujwi@Anko(opKJ`7eKkV|Y17~M-i%zzG@f-sJAO>R*A2PohV4!o?q zV5=-8{t4tth^I8(->UTFqE$}uWXnC$BnuqbD}YJk1hoZaABmCUoeki1^jUL3l^K}t zK2G(tN5w#Q-2D4)=WfLM%qBeUQ>Za`L!ZO;ha2Gyj6>~3<^;Cl=PDBitIHA*2rBQ` z#FjO>K8U5lV30<9y@LXsO}>RJAXTZNV383G64c5w7hYprW{}G;F6lBw@xi1dwTNoJ z4Tl9=)uI{pQ|IkEZ>~75BA20=euYqU1JWBN$vhzXx^2T)0l zWq!9B9WNdWvAK_Ol3^sV^zUYbh8`n?PByj}d{BYJbYz*#y9}sBY=Zb-w*8cyb0%Hd z{e=;bfFXJzO95GjC%v+|h71zVIG=qP{SwIvdoN6EiD$Umj6ZooV*x?EkAe(Gp0N5q{j71^!m^y)YB&`8uME{aL>2 znn!O$5eAs?=@efj)1}R}5FaEJo(pfwd~Y?)@0kUMp`~^`T^(`Yy7#)yY#md_6NjK^ z)=P#1Q^hw-t`>!T!%;VV^7{VzT_e+|2xhh@mDg#@-J!3qm_|oOxU!`t;6T{uuhJvv zp~#abed$}q3GECijN)BAE24(x+;y^uxb5bO#^V;_T)o$axVbBm884VVPOR7}+`T@u zz0oNhuo_-JSk_1MEx*R&fzr_ATG8m2`puAyo8z8Q54FiCmAl;A|GafdtQLonQ}9Hm z@G%@_*7=;WK~IAZim{*j%6q45iAM;_w)@{Xy56>&8jpX?>Yain58aqpB*3M55nelY z`%PIiy|Xj8)O^xWos32~(!Kr2T@pYHgZJ;ebF8h526;xWj$8ImxpNb6&gn(Z#H@U? zNST`1VG}Bp;^IfNRSYRFCagbrYrK+WeYE8d_C0-Ip1ovWKodGDX}zw~uRZuy!Oa49 z7v%#@t)d#IJ{QTpp3vJy-F3f-=B#IZotpI@#1x_kPfq_+wGj*lkJqR2Hs$z}dLnXa ztL+Tsa8s+KFg|1YWyfnPeV1{$&i*0d5Mv}9KG|+9KC}?}cfvSB3aSpbGfVU$Ee8Ru z`Ysm+6Kfsx0vJmk6mugnND@w}Im{(E&^Kx~j)hXlt|dv)v^3H=X5e$*y9ZW-4iOJZ za$H{&)pOa)=>C*l2l|aS;*qp}z_Qx~kWVvn%B{SsKQWX7JL=4et-D5vja7khM$7C$ z6V+eTrylpo=cw+}E$=x&OPQ=nNZA=Wek#Q)=z zgWI^DqtwdB#e+_sQ=>%~d=776tO%RtHZC~Z?6CayNf#4&fXT>bmaO`~dsmw-)eyi0 z%X|$kaFY7{A=vRcWh*b$xYYNt9|IQy&-d#{?V1Ok)n)G6lD4&32;uCDkFqae>@8?#OQ!mfhx(r04lgX9IC#H8C9%*!Pr5gwH>2w zojy7jTAT`hbIns#KN=pgJ**h8#0sx)^z<$_v=|hj4;WxwA0Lf{oXP5aE_u4@M;DZX z?Zr|Ku}lg$C2nIy{!!(dlN|+|`7)xx{RR!U;TM;8E*8-{t8-<>c5ANJ*@VBJ-KmdZ z>`40pmur@dhL>wMzP?iMJW)yBzm%hh!+GWTMkCj2PVL;ohAfBc(CY!*z_Q7m40>L& zcvX(K`=ngrlbej83}vL!2@|xEo1DAR(ZUw6k$*!)TX8UA*8{%pHF`porRbX{ZKTAU zya)682Od>j1LqySo}F zrNw)`srS-kkYd7TaFnx*5*A*!R@%&}4~?cywNjV6nkE0lA~fjICQ}K6D+7O7bV%%M)W3QlyDi>cRIkhAeb)S;6YUV+`VFh~YoQc# zvI8j$Uv%7EldnXMHLHeRsMSle3c5R_8{45EG4uS|*h2rh(!#`Oozd5S`V1-B|6$Yk zGnMV#Cmc;yyYVG#G(?u=o%@E0C!Imi{jSlz`YZLE(%SR!1=AsCR&P(nP%K+KjppYU zjVkC+5F6krvR`N#MJ6-KZpg~>3d4)+K2Mv)^k>Bf6uT;oys<)dDCjgPcN)koF17RG zbt(`0b_K|`q(+{51?m1~w8`sOW^8e(pDq27ol_n62sQ0v{Wcdue${cO6S0kU2|fQ{ z)v(6-RKL4>GV&AKe3JRX_a45O!fNLs$Q`MsOq57x2|V64L{c~uAO zgRuYQO{+|!bd)$uQTQo5-`&M2_sPDi;N5(%(mw=q01(TPACH>N|3~_;=081tgze+FV_2=rmcM3=l zeaT<482BcW?E{Xs6Ts_K2QL+bA9Db?4T$sSy9Ks02`3shPMOCK4Dt3DxtIL?MyzQbEV4+ZI?N?)v!%4U*tBsiyb3SrUiSHhra zG|`l&ylNaGV$^N)ZZb~YI&1W`7qd&yRFJT0#q9G>i{5clROcKOB=h3$*@Z_xU+Wot zs%$PYb*YBMLLqx|_sc6xVy#%tw*1Bk+XqA2(vE%k(Yf%cKhzo_AsGAl9bCMV=1@eR zaYpH zzC@Smmqkxg zB^3pEHd)4S-VDZ>s-w=4M`BD^4EgsL;P0clGi*cI?#X12ajF>Is9z4zRhW&Zjy@eR zRR0(j3T_=gS7xTvC&HPw5_rop>@@T8Ti=b&n(YuGra2kQxart_p(175TS{3YS#q>5 zghhK8(mcwb>nWq;p?|;X&8iQED~N`;38z`v^wXz_im(e@S|7h3r!*oYyne@aB)LY( z-Sf7Rif8P#S8NS%MrS9h4>@+5kIFlaZ|!i&_HFl_8Jz1aTq&sIyweBgO@ljl*eVT$ zq!~;UOFM7NEbQFZdeCUvFPib2>j8W8A2PnXXUuAbZK_KpHLOC}fT_P&-x&HqiM+~D zw){&$BIC*OSYd{BPxr={l4bmishD%RGLttJ^_=gLbyV+ghv0^ordM=~ifdQ?ulBw> zs>!VD_tmkCW1AUA0YR}qGNOR=4vuvKgc53`iS!yNp@$4(VU(r`y+tLI&}%{u>IjB9 zNJ0w`loCw{(O`%qhTI2qzHjDz@3-zhcddK>7}jDf0wm8l`|PvN-oO3ZXRmw0kgV%# z#Sk=sayLM6|B~Byn;VC$p?8KL#ue_#%sod~l~0e0yXF-n5#mASJNlFxW=}()@7g18 zcpWuSSR2DTld+n`!mA!pzA%QSjQwIztfP~QtZK}jbW6QX&FJC#TLKHbv|g>eo?nC< z*(olDR;))%b1%>G{8wLUq`ocT9}Gg~<#FgArea4gKjqYaqnns$s4@8J_>YIGNWX+3 zQ+>Kx41s=E8%E@zT|(=>K6m_$N^tN&0;i>0YT0Vh8Odl@>tspiZ8d-nRbE(Ns2m;0 z3!~J+BrjUs0Ds;ZX_A|d%g>U_r#_cF;jNzenDgW|DowRtyi?&B=xq69DMq;V*oSsg zrk8K+2C4*RXnbY$hZ9C5S zY^8^f)^9XHtySB`V&YhNTlkSO)gR|ae|R>jjAZ3~pAoWvg-1t)T5%(4gQ2N$VK>~l zA^c+FXf*FPep53D4E)a+#Q*8_TI!{Dz0IM%_MOKG5H18+QQ(643L8tisaD)MVQo^U zW%l+lr#e~6Xh+0ebG^}_ThKY2R@K|$5F0+Qb3oYo&D}Z=0$< zVjgPGjP{+IeR(|t!k^|A^$O;#IC~6s~!h}b}wO~XWB#$<KmVyeFvP4 zwOy++bVZHSn$DK9O9 zQzezVdke*h&K^(B{Cl{iZV-k7i=3I$a|^#ZrJXstZmK<;?48 z1diBtoQY(Ww)vBv^_~dYA5hchm2~tgJ_$DXbD^dW7j5DuhoayJ(QNpm1 zpdpJ0XgQS~?1HaB_}dy-LPjfW-DV!NtF_p5s-q_RS;_m80$}b~oC|YfB!0Mo4t_D%x z3u-;v@TSpn>2WG+zUt+b7+RUypDlm>%b^PlO=$h4c;jW4XF~4%C|};Kc>{=e;MWGM zsshqIo;x24K}W-nemW2QA%C6?p%HLS<-12B%`Bj!70&5L7%FSQq0TVIuOGcbLqm(^ zmnmS3dok$vprysn!5aAUsJ?k7MyTBjhln4sTFcgN&f8IBF+ABb=)&zZ+z6&dfvGrT zDeP+x5|HZb1yXuDR3Ko1E&Z#GtU)vH4cYFvR67 z&NjmQwn&km@=I=@v|R+{UhcPRkkB~wK;DTY^wsK&SdwLMfhJbo#6|Ch&nR)Hw%bTz zlH6DPPg8mXL{qw8JEhVqqtg2b{c;&PdhcuCe46;$!!KwLeJYSqqRfTM?jrRm&B`6+ z^#h00L&wIMt%EeZ!YxdmoUisK%guiKr<=8Q7Ab&EUGv(u1FU9WGN*5P?dCVlW$N>H zk;9%}-@EjG{Dgeliu}X>aq9oM$dli{gC-LdngAC%fwu88EvEBB#N2M@_mn?UO!`}{ zyfI==g=;&X+`@bpy{z!`kSq-yZhu=Z$W2XF?*hcS8o-EwE+UDx&{2tnNf=J+7!uZH7q{(OSP z_Tc`nO{9Xd4Bi&wTW%6>HCaS33IE}lq~Fk)lB+N zqk7IJ0pT~)5fflpV>0P|VyH+MEn7d`zcsDz5Pu+52ZW}P_zzG5ou|_~J-@ZIWR=GH zKP_2<(07MP<{;4dPXVFu$KN_4xAch0O>P#6$Z!TC`B2^8?x;Ajo2u}9fwKP&KEAMc zl+T(GWBz$lX2(PlH+xS~#pR z@#}RyXa>K_=_YDMUsQItPQTM1^Zxglx!fOP5uwg6xIA+mA6mE?G0(KSbr7xJU;!Zy$Qe0A(T6Rn?iuN1h#5MTLBX}lpv%Axa<+1 ztM981_unhDX1>e9=&QOJFxOd5Ek25F=H*AQ(m!QevGNXh(YhJG`>Oml9sbnVA%u?{ zd0V6C#Ya$P0PIRC-Yv_vgR7Wk(VYSI@}gDY!Y7t)KYxAYoD)_$Rqz~_WasPN(8+MT z&AcrCmjRAH+&TwRyR$c%edZGfuX2<|QfEEIp8uswxQnXysSqC@^JdAj5Vs5UGqEB} z2=4$sEg{=#GXHm7W08_1y0C56w6lQ5hbA4SGb|>sU)j{8zV*Na^#t0OnAvf1e{bf2 zTOE`297`N&qY8KCP~QZ8X;HUj>&iFO?O!fnFh6N!vK?l!h4&iYS^Rz!*bLWD+pDqD z^0`yT)}HINhFTF@&JqyYehg6kqg{#|$cdbOdz6Ide}xg=p~{0t(p$fd%bNd=lGYPg zJCcpe3p$vz5Y%ccqzqEtTo`Z@Oj%yqydU;E9wJ)4Y8{codwnluDYs7}<~T-Ad7 z()9rVn%<}4;}@$y3f`xXTN;hUu2N}<;>xKrK?JjhZV%UFm9_NjdlnlPl5H!KPpeG1 z6+e5el*DgmM!8v{L|4@q(YgA)~TmX|YPy`SI2 z#JE~OSo<%@XU3W2*k)GAoBW(e)+-q)5;`%?m*K9Ldw7_^hN6%7B!ZF2{Z|he4SG+w zr4V`_K?>Z+`R$51C5GA=gQlzzo(Q~>sWM1o^`&6+joHyHt}y0{VTuE0@#&l<)2!UB zXv)xRIYSfousZDB`1~!rpFyO-DP+5g4nd8^{wc#ifsaK&TaHBz`W{Vg4tj>7>Z!5W z{A*H{9OdS@*?W5OKn|X^&81};Kh~W&H5%~Y+S=7aADjj?y4j^E`s=JuA6?a<1c{{` zi|^q`zT8w7Yo8t|mMm*Spr%K&+Ery%_1k~D`pqx-DwvY)7ZiKm4puT;dh3(|^P z7>i-nHDR(G)R1=^d00;NZ;Y?nlC4@#tQh5GyZsw7$%8wZIM~EMN2wbD(H@kB z2x(2o)QR?8-hVpQ^OZZq2Jw!lvd_pkr;OWB)K9kksMi*Wt`)EMNHF?huLpJ0 z>U+f(U{b|G!7y6#rZPa~kA>wIFF(w!p|s)+1Va!ZJz!aKC6hblKw4hTh%@wKeP9$~ z7PO)@31^&4jDZU;o(Y|>QecpH4zWCTd)^KAhqr`9Ol?@#?wPI$`^fJ2Mk~QiEWW{g zkI$1WIS>68HQIXN2_!|N#lP13@_EbEw(78J)H*oIP%qjFwlMCQc{jQBtl%v^I#sn# zY`XIC#S{aRe0m^6^KFzYty{MK@dTrcjy}>9lF5A_mzg{nkvaCl`^=D8terQiqcgN3 zV4VcQm80FviJgnAoUFK0PRWP4v%Du`thnU zm(;LqiTR0q=}hb4F~e?|NlPGPz%ac;sc5nf+oA^|&a>grs)B@Cr&L=Y#RhMM@R z`?2wp0ED?42k;C)B(tM=wz0J3L`)S%Db3tpeIbBy0NeV974BI4HQpeeCHb+%3?Ldf znUr*tSq0m%wz?n5!iEPd0&{^t`3_QrHsjIo zgH?XLs>Lt$ATk{%TF|f>WhkAWKi0}s+z-8+vDl0YZ%>)*9BepzrC5eNBFv=mSnS%D z#8{2YXxExVehu8dMhsTu8;UPA+(@9`tPk>7y**VEAt^z?8>)jzX2L#0k5A7utwd3w z=nDsyhk(Vo--z^viOM8b)?Mo@0f&l=^-fb4?!a#-@;Qqoor*q(pK8&zV9aY}=5- z5w;Tsr&Y{Z*CRBZs8uc-deGJO7Uat1OocfXb$Ia&+m91E*BLYQ);ds=nappvkWMUB zVW<}73I@@FU8dFs@P%y-U^?iTsf%2t5Ft;jM@-c7JtmXBK@3RuF8?}ThnkaVk3DxN z{lZ$+LWuTwnspjmuG*vG6mpL(ntgr&&s($AeN+R}7#;hdEQ=&ngz?=c++E(4&#mMX zMa(#@XHOMoQKQ)BC-5&DQCF2_e9RYeujzu995e231{1S z1JSJxm{NS_s4%9Zf?3k-P~$fSnf#G!dXlziR1z3is1|{&BeyzB-K&sjx!WHg6q$-8 ze0i+?;AJt|&;IkGq=?)h1uLbBNrhQ{uS+?|_lIGew9^%F?PJZ8aVpx+M1_zV$$(y+ z^$a4}V72P6Am_Q#tt%GS4NrBH?>^g;{Kf^?mqJRC8748UY>eEjG8(5r*zG#voLz2d zG+OFVAy30uFC2$kBaPL|&og*Jg>c%DNNc~sI+{xG1M{#4^`L`;5dYVuo!^};*A$nn z7h;z5%C95VxQj&AD7c9IbOMVFS%`tdvHEtAX=(?vs^mvpZGTM4{jF^!yH2aZ7;9iB z>GFPZpH56`INm1R>U-n+B9-v?(|Z2CoQgv;pvVC8p$`AygXkaHGoiV^zhOC7g$d{J z!gpn!xv;r1${h$+nig?HBn567!1I2$EsJjtzxe#~ei{GdpJe%>XmSicmRLQNSJ%&r zK{ahLk9x<3yJ2r@0nN}$6$})T(j&d*{1Lel2WtGHjE94isHR|DvnLwrZEnMAKWgXQ z^v2(LMVf?ijaWOZ-iyAmR(_NlWFoatKg3qN5a$ zN7WyA-mpH%$=YdR&N?9~L*M61o^h1LvS}@DQoAcuYh-PFZqPUl4^`@M1<9?mlHSUS z2>JWwO2|iV9?#o&C=F^5)du#u@b|;7qShB41=Ki?e<)IML(*<>q+Cm%S(%glXimD%AcOb#1sn3cb@EB zX=rCrSH{x63N8I2T5-e7I)f4J+e>dfX6X0*+(ZX4Kz<}W*W6HxaJPN>PT(JM0&FVj zc%G}ra3faQrA5bsz>J?9V+ZWH{`6P6m)IS`DGIgVF)2`NbZ@29GILX&ZuhrjJuzU) ze7qm@cVXU&pMPv;I75$Dx%@iWvF;u>7Syq>!~FenG+FI6q2bSdwwrP0^zMH3yLFMR zD|hh56lbYx`%D3|C?A}KKR=DLWu0(`Cfm|=@@%88q4)&h3MOG;;Mud#(oycrnd}>} z;iF@>`*Qz~wg-|9wb~%%*f=Fy98SID`bZi?^4l`vu}ZIf+lXsB+O=j~@)qGayI#)s zC|sW*G2f+q<-tWqIo4P>^}x~ex#clE=b->}i_=l+O5D$qrwJmrzPXb-GNBrGhiNTW zP3X4FX-f=hhfwS7D?B9GW#n8F%Cvp}-z^-~Ibie*bdIT)D$MPioc05$wT9NDYI@|X z@`e5dxc5)QF!I_Cmm@_@Ll>}ZpoVm~>eP@%w~ET@82f@4UEOWOVZT$zf-AR5JMcs7 z=ZHgnfK>(=?JOFt>3u)58m89kF~#fg@#+8t;sr(ndTM>5Mz46X+VWb%=LAsY-LBDq zdMSg=9E#VWGRxmbOXEZ!7~gZ&+M@=Aix~jmEmp+diw)xW!-_ z$>NA@;UkNY<;FZsK7F~TQ6!0|dELw`s+bF$G+rGu(B7YFxsT;G9!Va|po2#8pP}+x zLI5k<1X5wOi+o`AHAMDCv@w%F3zXMVzJ$J&aaRohreuK?ns?dPb_`txuL797&X)g zs#uNpm<%C&rrlrtE_>mCU)7S;$&DO9ZTQxr`0K^S2VLB5nkW-YOvNR5m2DPI@rGW{ z4L-0?UcEvwjdfrlOA#t7X0zhOH{X>|1~3{JNU_TDDCxWf&MIU>IE(jae93E;go6== zFLz*+_>`C0eUUzal?1Z2NKLt?rQ_b5T&wB8<5J|owoooNB(KI#%-Pq5c0P18 z-_Ri1pC_W6#3q8!_uGx%KCWDlPUMq~y&vhhh-e7etZ1+;mD>vhFRtKaRkdlsLYPDzeD%$52g@;j7|#QTL> zg?hR%YcfRSOsb_&6KjjA0MBq(&Owt^5DKwRIc9Hg0Ud{X^j?t+ftVX{{|G zAI`}l`sWx>*ZXnD2#BC$rJ;d)L0E~jo%@pox{nA9LZ?$p+tBH8>IBPv`3!2Q7CTQ> zVCLUaufG*x?vS;hd%9@^V+l@!u%ZG$3#&uoR$a-cXW>g)%jClL7t^s{RWW@FUyctn z1Ub>e7iR1whk{DuO&2LZ9vKZS#I>ugh1NeX4LhT_9V})9iE#yG1&ZJC+{U(^(^#ec+C^-k)r2Xn2^aUMa*}Ps!_iigGSl9Lsovi`f zLv-OMpwF1P0_ng73M3asHX!gB#gs|j6{%Di$;K7 z{EWvaXR?wJr4`>moDLjo)L^d$*h!`*hAS|oU{#BK=GrhnS8m=M>SRjLXk8=9RlCHc z{Z79=(T}a#x1r6$$qZ59<;T->FCd6q{`$-HCYJ$byYv|uT2fZLHecc}A}Mznos8s2!pNz|fCu+){<^BVE-uFUrk@kP5lFSElUPZ%yD z?h*_OkeFcgj&Sz`zVYCWw=)^rYoIJ}vH$lT^!W#rSd9KE^OQ?LMJZmx1h%-gxU&hPY^m_dY}y$Ic8Y9<7Hll^;h$z8?G9ZTHG-~UEQOGYiA zB|>}rEPM}b8&H?qaH6y#xUnTh zTNf)xfGRG^-mdzriphW~dT+BS*npp|O8Kh%!O^(V7wePP`hwc1HhvM+yMlty+7O{J zyjr}<37TbP+;IZJD?c~fI>M}|iJW_Jjja;#aG>~GNA6tU59gtdpZr75@n+}y=MPZY zi#Khl4%0a_W^F8xA@VrUmtE&>8SDbo4Mzrta!i9-*PP=34+I7FOE(?Tz(S; z@@_VbKSEU|b;Zd06zy;^IcX=QjMgFSp5^PP$743vqGDPp&0<}bFm{t@+Jth2VfBVA zEllmJ!WTM|_vx6HvYe#BAZF^D+2W1`7mHBpt;h^nw`dI-IuEg$QvT_7W%G~Go-heQ zR}bFU??3#!c~Ry9o$1x&P@FV&|FqNcBDwEIJ4x&u#TXf=d)e3uptQN`EuS&=f?eTj zi{u82O@8Dy?ewlSnE`=KTW*D(P{2>ov!gNW`lXIAi(I<;FhNflNF9 zz>!RClQjd5G0n;RT1Q%gbw>@E3p2@xzyb6UC+sCt2PP}zEp;b`HHYro zl+9ZLz-ha6{m4z^na&&Mr~Mp;Hs)Ev2|#yz_Me0zqowt7`3+s15$rKAopd4s)0~uj zl=)c`zXq)R$|H0ci?G2Ty$d)>g93mq}QrpPPof%>`-`wBdXZ~hLW~heR1!wfQ?1V+fjJM z2g{hvy)j389F|cZ(I$D%>EKA2&X~nwz%usW!N`463TtsNso*OaZUC1ky$si_?`K=w zxqv*n?g~3_4*_N6Infn?K=}8*0BmAv*$oI{ty%kWpuXUB^%QgYkKgvH4)GEdaFAte z{(ZyK6N)l+EXK(+GvoZFbVDNW@;)6@YMIA)2EkRR!nHzTyv2l zY%bqc#o~79s*>0#*eyL-C>Y-dcfjgj0iiSZoNl-pu2t=h%4@9Ttkn07-M99b_^~H8 zZ|G7J|NZZxLhCC7*Zca2z3QEe=6av?&n?dNzaK9tiqQJ}!u#MyTkU2a9&!HuygLOA z!a)a3{asK>tKUIf(yNk8u<`M%k5mHXLF+NNRBmTRJ^=r|RW%s|pzbEzd4!*78k|eK3UWwc&+$Ug4@p}r;yHET)SelD@;;T5kfU)@g>!eBM!8gqZ9+oWKXSn8UPy~ z9wkHi{QChL+*ETDQ|)@8Tu1e_yNd zeeE7?VTXk`)C1iZ{X8EqUD1wc!5XPYK8Uc*DV671dNfH|eRZLFXEVyz1G;Tpo8tRN z7%s8DmGBUC01z#Th5qYns%ry?3_04L)`x5N1U)GKeEcj!Ws6^tX|mpOp_8q zHk|~lYxE=xdt4Rbf8u&AC&syEzr3mcV^8NbQDqgk*jv4TmfNw&m4j}1*HXiH^;JZVtQ$GentS>rZsju ztVPwRHSwJTR|Q5Z`3L$w#e0mu50=@m72i7AB*537G?pA`(Aao64)h?ElyzW)jVl{T zlT@gRd5JQ~;`xvQp}a;m^fp-$ zXFTiL$!|6s_{gU zUZBdBo{7bn&Xml}0ahmdB9LW23dU^?<4dv48^>A85g;(;-qlzxUGIt%50A=&1^FRP zG#%V8bAc;H%**DR|2lfLOO(=dh-9)klrPkR%7iXXecEn}yn>)06iqViIEl^u6m|ab zx7iA_x_JtoUy96Y%YY{*eKT43MNpgXHdDGG`M^#DDpturZJ~jp*lWzN*@4%M??i=I z^FCZWezpo2z|~Xy&mvBU-rmm0b{LHK>*yUr6WU7A2s8f^MV;=}Yp$W^dDAmL_-{#xt8nALT!ZZ7`oF#e%${v(aQs`fw9`2S8Cm@B&s z|MjmerI3q1+5WS6znJeo$@G6c#QvZEDZzcKL|)FxIr#OP-mrfGUyw@{7pu=-`}KbT D)j8rz diff --git a/docs/img/structured-streaming-stream-as-a-table.png b/docs/img/structured-streaming-stream-as-a-table.png index 81812161178fa9ce89e94d9dbf326de8da385be8..bc63524464099b5a7ca57a563c0f40c4413e25f1 100644 GIT binary patch literal 47791 zcmeEuc{r5)|L-kA3sQ;*m3E1;Wt~cBlg1h{mXa-0vNJt)vi03deu z%xMb%Sla{uf-<7Q;CE)kPp5(Z2>4r=o&w6-cT9m_th;%}&L04zH}n4rxMz2PN9wH3 zo<3<6jGq}2d49!uKx9tN-eb+~%O_5*m3$o;7bz-hZ2V@I?8kFAph}NEJT-eM^x@w9 z$h&K@voZ4z#6F6zJ0nuGZYSo9NdAMkI%3!{_cpbe*>5w$FEU?nI9a5K*=tMfnf@IZ z?Mha5C28AQkc$8R=l^ry|0V}cxDOoi>Oi(k##Ij-TUyb@_n}E|$$T3C>Y7eMIna)- ztv>$Fvx_7T2VB-@xf}q5Hh+bBi(8bCWdUc=lCf-Q8-;iN`^QM-_)m=W3wTuOnmNsT z%OSNd>@gcUZsEiLz`!M$9ZhkjJNM9`*u*q>^k1`UfM(WW$9n3Hcm6e>I~@D}ISP?j zT)dqZ;3EC^%M?~b2#UsS$YLF6tVx9O^u305!~#}(dtUXd9DPDzngt^$0-q!Vj?-xt zfT4lcJGB(y#6g&Q=2FlPo!L48AZP!~BgE^Wn)c9+cj_InP0Gu1vnJdH_2)2sF%82Uq}&0UN3hf`@WSDA0t%K=#w@ zrC4EHwhTI~=biChni<->Cn+GS=2njlw0;byrvoW3m$#?k6GN?g8XAVK<{9olM|@#b zdiGq@bUMHI8lg2!fcoe4Z7)ddNJx832)ZMFq}t~;#FUU6O?eM~{8IMOO;u}p?hybO_J@7)T8S9h042uj?wKxI;S^;>?WAtEc5=}3q9XHh z8=nwcgA(n7Vj?PxXOoo{!tkbxVN5SELqxhTyr2kfKA3+UVLud$&QhQ!4I)}hBap~J z_3Nj&aHr1j&sRQmD3^H=ZhjSQGG+POD7Vi)5*l%nHv)hIZ3-VhubCTr${r+%Rkm-* z6_2k|MX^rwR3Hh9Sk|J%vJvO@a%wUCUa^A%4NrgNBz#fT=Gz0f3$*slcvH1`EA~dp z8_1OOLxdHli`CmOi;eqgc9y60K@~+d<{U?{vXRPpQ!87iWX8YY1<+7>RTKd7 z33)OhfT12_smv&xhh7nxrM2Ru7gZ5kiW{>{km1^XqaSuvzwzk_grdHYGoSJt9SK#F zXfmfF%&5l^N-ZA9%OT2@S!E+vKLt1+ml&Os#ETcOC_1d z+hYq7J%s|+XR9XTbN*_19jIuL4tauDIp-66feXjiI4-J^cwbPN-;kl4{kA@;oHb$a z;feiMz5+hINq~M?r3rL9nt7?`r6yHyDuUu4f|05&>uF-Z6aub8S4MOb;ll^9ygdi+ zhx_+r$A8su^Ox&S&5ZvlQaQ!ZXj8@n^s>gD#C<(5Mp$57cFOcY?*>#WC7#gUaeG5J zcdpW;PnTWkEuh)nB1xr4P&$-lH@s+F4#~In@$NYzIA_Dou%6Hv_5fggM}>gMXZ?J> zaVOY=D@Ngn%G_g{uf@AmKY|}!@f>cqd0)D;?D|pkxyjQNe6-GCh~JV+pPhP5UIIr4 zIemPn`$QP79Xoi%b|XZ3OnFiDL~PAVMG?l2pq11QFS$4Ld`2NUqR&&8f@)lt$NyUE zd3yQkm3F|VyBuKSpg8IG$vK9{Q1$MDgs|h&$xsLJO5Zfyv9AqWt@ouhupapp1CJV) zzVlIwO(NBPGS@LSYe(GZKk&5?2Tu&ww|A_a;)IQ;)@OC!%eJ-etiGP00r-9Fuye~A zP5fgm9xn{y9*K;odW@u^OlF?e_7hpU$ zvC?#|)$+@qkCdA2JCg-jH_t*5w2m(!L)dL8P~-P@PjN%7;~T$gQPwwg-|tK$-%$PQ z{#11L7P)?ytQ+BTm7U%g9jpFL*K|C9;o%%MrP^X~Da+(bx#72tZDpcOt<&$LvQ-K+ z`#(j;Gk5vhK~Pn>K5ZU!Gqv;;osw_<4Ud3^A&CDP;6yj$fQdpRop0?Qj@2Pf!IlGba?b>E59S<0gVpZ;qPsRnuoYR0~5ZPgkV?5;*) zG0>1;xMO)>_fYcHUg4S`=cAxM>I40eheM^ke5CfGzl|L!Sc$MBK3fM2Bc5SWN)k=o z5~EY*+f`Lw${)N0)BU#Our*S90o0gWB(Jd9_yJ3OZo?U+Ja6HN>%`gzqsy0$G9R(& z%r`Hbz75-H7(OO%{}?A}4pf=eubDG>-%uI<j=GQ4EX+%e`&-{HQe`xrH_)zqbO{B4weQCISdx`V%{)Bf6F5E6sJT!d~H)2~G) z;A^iw4`>x{N)MsDd-Yw@Eb8&Z!wi^);ZKiOtnP2Bnog~hwEoj!UHT6TtKKT655z{L z$7pSiM5W_6jjh{rd~{U+bL|xP+YgkzDAjvh?^k|fRLw9W>_@1CR5Wkxz-5t<_w*mQ z#p1G^)W;8xB7V7fOOk%GI~Gj41d-ltf=n?fP{X~oeR!It%zcKMwp&s-Su%_ zhh|WR4*8oEkP7R23OjcOi7gIx3XkN4P+sb5PscT6%H@jR4?P#%;o}Bmad+soRbtGx zq-6O!#L&A$|YaT>bfsx`f<1cK`k?OtO${H~>!Ex*_T0Ox8R)J;4icYM=JS^W`i-*j%!V9|enDlO|#)ows>`Fu^0Q3K9U?QVqPb3OKh z=y22%!MQtvpIGUGtK=r@2^iWy+%<0}X;-uzLrn1f`#_$I9l(9YZrB1^PTKo9#TAj! z6wqN|6sObICA7jRkC@A9KE7@M{01!OJ+vA-1DA>vq$Nekx$tfab#ScUv(*N!%Qv6BratutRzH0o>7svKFm3DEg_13OLNun2@IV7zjW7 zx3qqt7mp%?xCy~I4W{1E+&)h2M4wSwdnA#2iFm>J3d1}3QybJa`8FbrlRt4a zWGE=1g5tK62kgkHxngNGaTr$kC3{y_&n|?@J*=n8exVV>;X~TH2Av;LDGnMCkVB5@C(*qU7_tJ5MkaNVwD69CwI zCFiaH;V^hRF4oQ+4kM-P+gdZhk?c)LO>8dyeXGHmP%hdr;P6_1;O6cl|PQ`Fj)qS2j?kJ4TZ=; zN!b1|I<<6h1HMV>AggEKy6ieFgXx#Ka8XVw`#r*-ESO@S$o}Gw=hC6liMJ6Z3+LOa z8n|p|hTI=6zRDKtifur6&qW*^8l-+=BN$qHu~bdjNNmBoz=EY&4HWIkJv+baoy(%b zDtTNxD-(pdjBBO@iPBbDS$0RT4J%C%YkEQ)qfow&u$JYGIS*?SK&JMB;dc+<*ZVvC zW*GezHyD1I0>NpiH*rYFc!1wm?B`xq{&BNMw>^*2hGx1Y2C-kUABHS?BwixY5h3Bt zG7W*~sg+dDz{fd%t#;(%hLd?L^~X-G-PtKl53~I}<|id5d$Vjd!j&p2SWrOxd)T%y zUNkBpv{Q|o&trA8li`hPsbPu2BfPIDEMpTne`(A=?$a6!Z8+^zhsIy0>t6i0^o~)! z!a0gz+(NDhD>rC7_7I0 z!MfO|P39~xT%29`Rlh}~oa$clgD;#`Fq3lTg!D8Q+e-yp;JQuDHe6iqoqRv^N;Cn+ zPgK~g#LyLy6`Fg!(-(pn@w^N3mN92j=!m(dYFye=!hAyRvD?=v+pvLXS{n8nfp-;N zhTEC~iG`{1HZv008|HtyPCjgS!P)+$A&D%lY%-IbQd1wn-Rr%`W|~T;$f8p%!G-*)PVk-x`UiLkhKc?qRao z@JbiB;>BGIu@I;0TFZ4)7@lni!Bc7>HtyWz<+0q&H2Eil6c{yM?1rDSSCZ8C#!BR% z_ujwmp*m1cnyI#90o@TM`0bI>wPr;;i=APYxpqI#u3RqUo>#oBf7z-aBqXTw@}G>x zAS@b7Ew9^zDd|DZf~q%~ zVD7*6qOwiGGp1GNWzkGHfOvb|fmh6Qd!2%EB%2xUZBQuXb*Ussw#F|>U{G>#GsB)r@Jy=P&vk^o{%K1)Ai=6a{>I>S~P?w{$>~-3^Ia3q9 zE2(!ic6)KO8X3-A*{^3M>5tg5gstzBA8&8tCYUh&r)|d@&uCx4du&YJ1z-+xG`lYv zyNv}EOG~G87G@rZIFo5o1XshhlsPnOX+OZLQ(6A;pG@i{u48hKrI^Y>xeV89ijg(; zX%*x7E2M0VY2%|c-m;bcVNxc)yy+anOqGgPa-On1cd+*n{dLeG@SBm6 zaW}^oTaBJAI&~5eoEj@s?|_5#sjAk3sd!lF#}ca~YOm>JJgh`nym$w}GTx!VQUE-b za7pN}h>C>Wk+eY~f)G-Sn;xVb+J(;+@07+|K5tXDgP0C4R}C#IXMJiCos;`>=pk?g zC>I(z2Z)1@`2Iv_WX;XJfbE)_d^)oB0Me1l0&n?M7XYB`5qJw+S=W>T>;S*Y*}tx7 zEqDw(3w{L{oCS{XkMTWr5@1S0~IjM{WR*foJ)2hCW{P$`h)>h{A0w zMz+>G?Q(g!woDB!YW0sLz#mKbQ6QWL5;WBp!JfP?A9X`xZiUE;u6jGoc@8XW`Dgk2 z@m5qr4xF0XaCOfzKJ>3!LI1=F%B`bTgawd`oz) zg6owl|Mi)ip!H8hu_ZCcTokYlPlyCtGJgV5{;b(8tTfKw-Mn6r{*4D zljN?J=N3**{kM;qCIB5jmNaY2b@!c`E3*{kF3PNHtQgL9M+vTyRiwg3E>|v6vEG&W zgqI`)(TW{$(BHIcIhiTFRe9=Y=)!-!4^`4K$0~NjIvJ%~hvk*FDG51oSf_6OPajlj ziY+Q>>?a~7EEOjqK4?FN^hY5BQ}h4MPU%P|ddk*TD_eyDA}_Y^!#$wWB(SN(t zxWIBHmE-r>;7Yz4`>WD!A>wh9yXN{P)K2x)y3uqW{*r6XeOYEr&xU!XHr_od-h{d+ z5M<_PjV>)$#)21o(v^5L`FphJZ{6eTtuvc@YKVwvn>Xs=S*eJWOb6tam)nc$CsCHp z)zhrE_~=$&M)PnV?a!rH#K+g5WCXb5+^wzCEt)RD_S~sO4yO=e*tU*q$0I2HR(HOs zM9wQj;_hYBi z;+zpXIr+Ca{j!Ts{s7D&I#IYW^f}cj;!5wr+rC6Eg}lJ2l6?dkQL+wgt?s%6wbpB{ zxyUKoosD^c^mK>XOY1G6K$b34i>8TGpC5 z@1&BRUhCGA?>md#l*q|iJ(SlVY1`J0D_!k4!LP_cS9bdF&J#dlKo;ZcKx%Zm zdEC|9Zx9+Wx|iKzqmo>_ymr%Su8B2~t%q)+ zjbF~n@Sg%2k9u&gaDkafg&7<>Os-+lXAthQyo%`ELQaeCxOvVh5WWQ*bqBtJMWXszxthsk@!BZgj2kZSY@>h-3_(v^4N zYYEf*2)5-=%Tlsxw=~9vMAg_7N#w`E91seHYTmtxetM}!9aK=Ekyi3o)w0Vr5Cv72 zTc}xHSBT)Q<=Nrwli!!fUw4eO4ikvHrZOMl`n*K&M=<<3=}&R4uCHcd81_mX_6 z!q#eWi#*xhK|4y9tyy(Z=ZPv`XZ*Fqz%Yt$@-Nnn<$)sGfN@y;0N946R-{c#9pCUZ zn}LQYpz@bD3pnL$?-!R0t!d!~^>CfoFTvO2c!ZxJ!$(&!z3J`xCh{QL_uHwVKG4ag zNJZi{98WLLD}H9$8(y{*rTw*nH&nxuM!d%Is;sGro7g(Tnyb#As98H=)Q)zLf2fd& z4Z1T*E^NtL7x0V2U=7yigq@|Wl?v)J8FtHRRKUG5%43R*v!N_f5au(yk5ry{f)LO9 zLTm*(NPLw8`fet$eBaLo*L0=vZ+5f^k0@jJJ&M4FjV^FZ*@5WVm(3Njt^jnuSGa)olY|Qts$WTBoDza)gq;4jaeLO8wcxr zd;L2EcNjn-T7lvqlnrtA-~svJ%$2VUH<`tz7cV zwhxpP%BQYv|-v`(K~>cn8oAcCm8wSH%IEt?&c|GEW}_ea_Im-+rcAlS}QVx!I9GLG>H zJ$*pua_@%2WTA{?JRC#Cg>dQ!C1sDDK;9C$*;p1tY4Gh!-k7Tq_0T_;1AseTz;ImA z+OMJkdaoRr#ic+)uYg-6!TVN@H?O6Io6Rdm&_D<~%_VM4UTnYnC$58-`h+&93G0gS zlGD{i;K2bcu{Uk4$(P?W=%f5sB;QvR!4hw?ddW|_vK|xMdG+qR4#WykNeH{Uz{swn ze#0N5rjbv~2)01HKM@Is_;lJ5X$kerIl`cE|WE==9Ej7|+hfE6CY?|LYrCYGq3=oAV5> zz2=Pu@>F@@F(U-7ipkw<1^>I|w4`Xh5Z8_{jee0QG?IAlrBV?z(QCF8j2(}viwcU3 zKHXc>4Yi?=dxzka5&dIK1|poot9Z|3b%eGPe|~!Y3*tQt1?v>hj=pdjNfZjQ#Xzh%jt>8J<@LG-L$`lhev2BL~ez}R?$&*JIU}$yR z-U036nW+{qKspb4U%o12ZjJ;Ync}-xK;Dso9bG~g16*&a?k@%YouO430E&kW8eOeY6jMQ3F2rpweR`)ecX3uYQGRFDEK4%UV8VMIn!QJ@w4obgd009A894ZU4{)e zUkXa#)8}Kk!89MT`?da!j}^>Yc2V9_D0g1VN3N;_)@iPW+ie5UGl8T2tK<&@ilQea z9N+9HEyisx^~%mx_?xoFr&j9ny1WIyhemm30(5AUTN^pC+Q2QLqK1d5;kottBY|VAEZ=_)@EpuAzrenxTlM5qj}V6S^0CT-u8cYZbYE@G(Oih@V9a zvBa?x;q@jfNP6{SjsQFm+^FjSoD0@6_H?vK$sd50=F`1tF-bzJWBDc#WgAfA)U2WQ z-b}fY9bozf!N6)Wcn!8}rDkdg3t_9@uD9BiWlBXlabKKMVS)H}fq9|#c8q3mNSR*A zZ%;o2fVk;KMVgof*#J8QRDEIe^M(n$#UPe@jGXp|s{L}IwlDvp;;@cfg!lTci%#Zr zdlgvz=G@Hcg2u@&qYhTnv;dkhs@Kz7Vh0fsT(}091e+#U7f(#Fm}Lo8>vW=*mH)5o zK<=xytXf}sEc<)$_)ASExiMCNDZ5V}I}_5ahz1!m#ewiN53(B%78r!&21(g7;h0yK z*~Cp|2lK58d*AbeUfphhXVtr437&RI**qO+0!B@+zb))rHSn!rytRgKVW)%N@S^0P z%$=7Q8e%E$si8zM-w5u2vY`>-YNEw*ED*OqxZPi7uSY~QzXc!L_Ul_dqr5P2MqSK`Mf236?i?^fFr`Kkb z9bplNSiF`RZaw15p`ayRWB)fj(Z9pvTQC^q$gP>%^}eHcyaEKz49@t=>0Q=ar zWyJeV^HMMofdzgZvG(K8f~`PM{+m#-tta|6oG^7LgJZ7O`0>JAd8*+ph}~?icMQ=X zjILd@9scZo`;}a z2R?H(>OoCpgP?6l5>|TKkLMRp3dXH|^3b+ORfFJbX#E2QZx890G4Ur7z3h|uam7pP zsp6rW(z4Jf4>Rl7;$9c>r4B?76CHH}<5dw+1|xE7f_SQfdTXVZjF&n3RGB}JhhHkm zNeBItb?dw0(9H3GOT7y@X{hml);|Df0AO2Eit@9syeWkbR^ZszdKaF677$&*D4f7$+8=hk=Pa31TcH2QV{87;Q)go#WSgwdZgs$?kvq|)V zk^cl#lx@C9abBs7ABgORLDc+C%0_?FkEJ8Sd;bG>i;SF?9@j;BgZVq4poEctnT0jF}SwPRIqZOvM(fdwI~d>Mk-!L>%X4xzPkOl zF935aw^*>$RFrSsny={;_Q79H%)=W-g%T9UnPwt&46HYU_frCV9C0g(HXFIBBow+2 zfV_UEo8z@Tz^-?pf{ls#7q#`?NWUklkqUPO!{Yp_hEcy%v%$4+$h*Os}Xb!#SH0^z(XB33GvH|uzD zZuHSzSSR$3VEw;^e51R<-0GL}Xu}D!xE>`iu+~E-R}~KY_KbLrlLFj4{57xYXWOvk zzhX+jM@^ZgB1w~Rodp8YwZnwNAY7b-v<_|ZX7_|yoxUMsx6)HRx0ZBMa}TpGYGP$EQWaA2A&+W z=3m8JGxxZ3YXD+4nh4gtb@u{Jbx!#pQWCe{Q1WcUV%>&VI6PFp7~1v+gtK#s$4`Pb zbTV?_jj&!&WQwDzJ#Rz?3{e6pK6aV~K^?@CV+5#m7k|5Kcs1e9?6? zA4$Hg7+w~gbY4AT4J`V@<{=a3LpMf`sjMh6^q-6h+Xm)?B}D` z6bz@cy5*NAc(MJ*QX0R+Hl;Ky@Gt*qrvbQG>|bqD!a1sm_O;RYo(qcWnxYgoHmN#1 z^mgez_E$Ej+$&*%t@rwzj(_zu(B*r_R{$G|MtapEC%mKArdhYTX*wzIRV%CE;2C{3 znG6E&4X9Vwjh8WfV2uk!JIZ7ITT-2T0kEsZH#F%%f8Rw7gz`JeilzgE9T2an$$#Q^ z%2qHX(oYw=Ou%5E!fzRf8w^CqGNxUSQ&@Jag~$Han}KB@tP=q9`a^c?`tz+w3lq&C zFj`wz3M}V7E*^-injs|z4Dg^-UVfz#8@3e1-CJ^)hBf@hgT+3>BQ~y?Lu_a|Pf-#$ z{@ovYiMOBkz3);c6P-%qk$OJi%n5$B_1uW*m}k``16nHFP%=TIB9PLR)EMW)uwfQj z@FQiI7I6&V)h*5-`BZsN;jccEuwpKhZX35A*sgl zo9TVX2TJ{1^}K@9@^S%ptm-6L)kdZ16A8n6z0!qiF#a$^uW;W`?t_5XMH({bwEWh9 z@SfP*EJe*-2rTLgpiiHgg^G|OwR#PqdVL|opP&xo5$8}rx8YYD^}snqIi12TGD_zj zX~=`=@frv(EngSO(3@)#q-JaR;PLYPWw!*_JIUW{6436*)LZ(Z zBIcE(VncxxT?VV(Ubp5MT=nc*e!Q-zoO<24a<09Zqpo%{m2SDr+0G05SJd|&Ha8+{ zI4n(Q%>*#VK&CH_T1k%qZzcJIpb%BDLv_cdi`mB`C=O6>v4w^^mCf~eHpP{kvSK_v zP8jbG>rt^j8Bp1kC~uAoZ42{ekg5g?UWf^gnBUhuJl)v96@<3l&X#ESgu6Yb`y6Xk zrfoHZ&&BzUWfW`Qt+OUMZjZ1caNjiU1Mmvd!@BP68T$jBkzu3Fe{IO_a`9ny!&#K2 z1o4@Y$~;uy9j$K7cVN@i!B%KQtS>(T><9RPL8aJfRhJ<{xcRGQ?u4;{RDg#SJ?AAo z&oC8NUlDaPv>xf5$K9r>0>SDEWysM{$kse9WXLRZC15%ccJbt5CNyYVE$FzXvrXEX z(GsPZJdI}7^kBJC9_*5pNr}WvM@eq`1E#4sp1gJh)g}utkudDX#^iJIxnHn*i(remFp^_uL*W^$HmaXMwr(xC3@9w&9NR#uz zNsP)gejK=9E;N!V|Jqe&Zuzkk9Kwy_ek@VQTl&`rh}}!UCe;~=zF1u?dmYmP4iwzb&i#fvq6Dat8<0%n%J=9i5i{c8)HVY>bKU9Gm(i7E|b1ckhRZFk+t{)lIA+FwJ65VtJW@UHXJo zBP2H2$W94D9fDHZP})}r-Jhwkc_znvCY!hyyl--%;}?AI)>;ntN$$N@@75TI@t?Ny z(Wl03W+A&lo2ia1EHn+SEX(TvY>aco-u11+ zgm>~p#-YWA5_hmDyWepO;3F1beqVCLkJ1zAwHW9G%?^bsAp|*PwQzmyMB<3+vY6Uv4 z_iN2Fr-bho_JmW;8oSv)xEdSWUXgoSuvx%UVInMUt9C(Ywcjw7l$7Jwn3c_{x(**Y zY@!&6vb6GT&;jQ-4Y7G2Zq|ImDYWTWPlVW-mk7=UUg6W6#?!vry%gY_!>aYb&4`mr zl26wsQ9g{yM4)%#gPy9Vm5Xw@dTYY5TvOWww&7@zgvm0elDkEi6QvkB`fpR#Gv)gG z#;B>Y!u}U~_Cg67&?z09SVh@l{MVk%XstF%gmEt}u)`^=b0?XXi@XqZY_q3*yoPMe zius7S5~ zG=5%>sG@Ai>?yco+3ta>%?#dgRraa72_j$d?uD%>^|$#aH>~{vB9Q*K~dH|N=JU% zCt)+85rT9|(UI9U7OfH)0MnKDW_JX+Jh|8 zueWE^nh1?ZT*x+zaNV!a>OokMjG_K0Eznyt*LGECxtYuB%RjonDs}(xnfd@rg;f8L zTlg39dAjun1wprC>TW%u$DFK2JNn{iX{c-&`r^qxb8T<^`U!8wD#njmF~>Ga?z&3(}c$12fO^M zD8}-khn=>x+@2cy^2AK;q`L?L+)$8RiSPu)?cIt6vCk{8)C`UaWLyz(a#X0us5C?H zDz<|afQ0>E1pv}$|4(->0?Y62EmOc6fa7pco&5_J;uJLKSx>pfnw7qvk6eP7B`sJJ ziuz}!x<9fW-itHbN(Xm`5?+f+FW{$I{3wdCVX=hCT^}UVnxA;0Rzd15B z#0;oX(a2PIIVG)E=6LjRh(#5dcG26mYHCun$^FjU{FNPan7fv|0(%??KPir_iJJ>U ztGnW#^wRm`5SRa>R~Udk_P+7zVzt)cA|EG}Lzd4=pYbsM!(4S?yPy&_220;U}$MAGze)ol7wdy5M2w3Fc*DC5y)kb zw`e;AZ?7K|V}s~_4Hk>A(T$TIWn~25owLkb+;p<4qG%J9ezCXr>qZuT@h0gpo>9<% zaa_Q5N1)2LU*~AZci33vKFu zF)B<(%5_0v#*nwrRR#S&u)ai_UW&8IA90%MYiikqbMkJ@HR9BEF98Tk=*DRBA^M zzoDLgoD*1PvpKd}Z!sS~&Vj23Az?&6XNTWx4-_70qd^Ouwg6?)O81;LyG$wSYJ4`( zTtl+GbGdTgt4C9wF`&r5J+_}b|!0A?8I>GzddxXN9x=&J#nQdGh z6lQkbwjUm5KgKVmBljA+`BF7|o;V!-mt$0IsFo#*?DdzBeU8@Lx$uG6v`XA6hs9^Z=icRr}<+<*)598`MD zN5S68_lnNbLJ*jU&UvMi}wopWJU4GOiW=#k-`;t5658)ysi3}*8aG!N)iuZT2 zkAkQ=`LsMPio*2hsnO%SHMzvgL|)DO0c4u~Jrz{wW-ocVpcB7;jFM$-WwOIB+X&Z| z*98=5q?}z}#k*4a_RF}lRx9Bw$xcbT>PiOAtRuweYy_=6_qKYw)Oy8dgzURNrIM(M z0f9ZyQ0jxwprheH*5R)ph&w{m^ujb??g(~B;fa~p8n>QXVlm(Sz7gsVcv(PS@YvgH zi8$YR?_^qrR}e6KK{0@-De(PS4(+GRQFa69Bhs?=;sZ?z(cXQfc5j#x9214ae-ZkXOC&z*9^S5;>T)*^!K}dbJk$ksJ#l)Tm$8s29)*xs!Lp(MqES$PjHztEPj1L>X zyLLm?NP>t=*s2*dpUnb`uSA>7$sekX>yzTr1|ImF*M5=U9AaOygEZN~=?n^c|D1hA1QklA1TLfyR_T&8RBm^Q1UJU7j5zu$;C_^iW!El`#E@9f|SIph0b z9#&4^$q78%cz*|4p2H|YUagydYf7AbQE=-)XkrPF6Y&pj1pxd369MzYO*Dk=8=vk% z^K2QPdXy!pG)vRFPgwFhG1GS+KC#6RkGWd6&Z|enaUDR}^*5>nTYDx)HwRn9ye$m! zV(rc_&`GqJ?P275+UF+C{2ayFRq?(e^SVXE%S9vmR*ge?H88P!wEW*X7Wn9J$9hd9krFU*oJJ~y$MmRER>_aN~3(Xb!Xqm06R7ym(rb0 zSa95zh~sWwpZ0?DnTPOoVW-n0hzjOX9i#0 za;>7ai%x4(#tajauiTf`dhZ@H-;=u7-->_Q>$iZjR{4UsuON5Ctbwe zB6;L<;;$}5BjsEX#Mwh@WU~Hw$TAmQ?|}ds{d&^FkP=}&2*9II=8K-{sc-M~YQfdi z;F^Teq`>wB=j9B(0Q<f09hlhKG)Q7-mxV@_q%m7%mhfD#Z67o$D8Y`1dFD4Wre(s~7SN5H7A0q=P6#G< zk~?I~NfzrHN5h$?)`Rl|(u=u%lj{XP>8-jPFP9^*jEx^&pYV%MH5%B!m=xmVnAU`x z>hQo1rCx6q-3Tge_TrcueLnEwN2>#J9~%(|r^}J%ZJ_-}*vhKG{<$cvs?ZdpbV#gU z*Kpbh+L4_05KFYxkd8D@z#B}D>|hq)_j|SIb`Xwxl^`7#$e9_6wm5MWZ~+Z-$O|?+ z`v$6*gl`<<3}|?PQ>2&!)0I%-MQpiAnfe8}?CqL+Yp9dRB~u^57(A(3QG^iNViIxK zkBrFMcG@7|J_)s%&OoXfN2pU5;~k5e-ac#os^JkY%<3@2&r*)>YugrhHLwg{->4j*e51O z3du9&$ROMeJ$f%}g@rRNgEez^JjI#+SXx>jH>!HIu{kYb3mKH#uQQtGJ)vY=)h%D~ z|J(-`r1#uw8KV|P5YWhQU@n`D|B_IQL)`9Vbcq}#k$m9I;~NqjB)`M;@8U;wF`1p8 z6pbAqeP$7IF4R6x!#No-%uq0-!}TJqZQ=f0*ltmXE{D)7k`N8BpPu_%VX^muCB`~_ z|JWv7$LkVU zp2K~Amb=f4{`ckX?V`y`iFJihjDpb>Qtsv^Kd^py@V>eKBgNr+@E!fJL}N!AtLDXb zuYAB(Lqd3YK4-}1x#2~UkE|qDCiSS=k#Who8uKjuknxMY?t|CnS329%b2A<3?s}5T zjfROZ>1CS_pO;%q5j!3m%G}r8W!Mw^5+)F8&3o)Rk1*26e`=9wBN0}Zdp7&akFhFY z9OUf*>b~cC<3937zv;FKe**JoXN%)ue-^lzH_5$bJ33M|U;CylOn}?lc_j|Nx3`bd z?k@`LCk62AtxPH;+@Y`_RggMg<4qxOp{gf!Ihl~;Cfa+)eyn1`9QRN+qV!?m{Q>Q1 z=486%)fHLH8d@4SFqH^48>`7+e>&iBc4=toGpPeJ2Vj)A+BRwnR5oo8~G*uE&^5O4|3Vl)Zqlq$ntK98*PMT{>eTw`@KnIy7|9 z921<|v-a!x*cJZvvFv4m*fYTokG?pZo(uLrx?p~tun)hIE_#88OXTbjpw4%4pIAODv*zkQ=5e>{)3%2Pxb++a8FQWCGP3VwqR_&p)V}D^9*wJ)x6F90EDhT~oT<2cpOL@ua=( z9&?%lMIemR z>2a$wjbDvZxX9cvDPgSAN?`@{pmGCIiw!oyBTJ-!^fGTyT;SrcO7-0Y=h+?||>@mg4Y*t7p_zRRgp0IL{ zcrWOHi<1YN*g?baG+apHu0Dq=PZc9m>G(wa(P@ztO-qMVsT4VjXKx^FF)cyTDUw5S z+IHq1w;=1A1ngB3D;!@tzgw(-StfZm>ya-AoFn9Sal2g=ed8jY|NQ2C<(n(@nF7n)kj(7UWia=!`1sp`bN{g`{ZIkimENL| ztq|aTmSu7M_e+O@Upo{`+#gx~CGlg_eEJwwqM%H&~bO( z=2mK9ZF4;x1@5{GO79WOLggVQeV zW6n3wqD~!#NV+Uv=WE=7FF%a;o(TV+OID*3$P&|9$XQU~TNdi_I3)|eH(SN0?nQg< z@zJgdXwT3wPk6o9?m<TZCw3fAqFJ??NTs zcG{?|%}8)v-c;AgISY=;baV2KeD2Vx3Z{wDyv*a%2Ji8g<*UvM#h4a^Jgv{i99{dh zRjI!r@axlqB|%}tmc*ml360p}43c{LhlP(Kdg&%a#<=0d#lH|on&(JU+d%>as0pDt z$e0(g#`#f~4~|^;vCM_Kn;OW_vyeg3U^VW4vGwKgP=5XUDn*NsHEZ}tc4`>=lx!_% z+LV1sk(lfTV<=@emF!y?%czhD*=3osGZSTv$qZvRGj=n-d-QyI`o4b8fBu=q6W(5+jXy`7!s~w$>xtLN157|M- zw8tqiL_S)^pk4!7dmG8{(bJEbtZ<-Xqy0qoGXS6aUSJbR>`a}-)@O+O8aTZ4lJB7u z;n;NHbeeqs29Mpaeh)D7noTBtZ2)rI`I0x(#cMeg-LrMY81c}8pZ*P)t@{;;a$V-S zMP&ca=6rEadbju!H9x+sOf<@E$l>^pj(0&kx1a5x>!IE#>G>W2>fbZ7FJ?Bvg_H>& zp8Yr>2rM*1vX7d^jZdssc(}sSK1h#pc{5Lg^H1m~K&mIagS0%f8su;%NFW0;jNPC- znO?MVggYoW*{{=k`hgv>k!v6-a2H*&?%hR~tgi1JHuZ#b%~%~^%qM@ZJ-C8tx6&2(sTH?UUs>{Rvw_ISgz2UF)T}6+^kA7`RWD2 zCPwMV5qYfrCUu5~pb@OSI`a;s7G3s%2EaAtSJT9GA|?h4gcFPqTJ@{?b^Cx0W6_ly zw_fQP1szh~-_}QbD_J#ClpJqZEqfQQ=6YY~-rbH5`v-j}nY z%wJABa<3~!Z(ap|)bxn_6eRRfnBr2>cL^)0Ke*OXTBS&K?(S68uB?cr+X<2e_d_d< zMa3B*T>u|qfzxI_px4>a_E((LubQBta#uJ1Yc=Su@hJNhjIFkq>o3nkO-`DvKQt@) zttp-gciTSs1u8IvI(VEm#(-P44s?5kJ<(K|cTtqQEfETKpF&y5vx&)JMT znGd;LzLlSH$a%+B`=so{IE1#;>T-?inD;$bl8HA78^N0}xgM+(-}B+tm=NvT`cRfP zsT&b1w)>dw(-MzrQK9`dX*u$RvUD`zA_*t%SanL@@iq{zz`2uq39YPT06e3BZs(ajT4R zqjSOY-v|jF@yD^Gp3>|s0b<625R*-jj33qWoC`^V<-*V53kX_4p9s?I6$I~|*LWvLU40FmK) zvl7flxV&Z}iK9j6#%oKNdzaXGNUJNV7bWh`kzPXwvGw)6w}%r6!=m;{h`K$>K$Ds|Y7Wz5x2U zv_v4EQ=z|0AhWK{KN#n-q$9W8#v57SB-+8*wwCq!cVDtf8e|OuZ;;@!#VPsEc7K4z z&EKO*Lpo+Bw|NF7l~ww;o?0rvs)pE!EW+~7>WuT!YL-71a!0~$RB6gF+khXOM>kSN z+AR!JG@NTi%D#`+QK-t|%B-&m`e~=7<2Ql`mODmxHK;$f0hyJr8cq3`=ZVIvQJMQF z%MnTa@_JBpxz%v?=Pe!M)v8a>J4X%7V|Mh^x*`@7#O(MLZY_mtu&u^=|9r2()*U^y z4@znyD7Sd6jC}G2_g`W@1SlX0Gj{Q_%CdYdZ#X>uEI-<$>1TkZ2XkO$^%cnV7rx5 z)^Y(^n%S(jCrpg+8_+$?;ol>&4Zf^RE&FZfn`i5(uI}{ul(lF)tBDTVg5Q>Fix#{59r-sJS`_eGAlH)F+g9f>BLdeXzZbSaZq&f^;ck z&4Z;E67mbzZt7U1fhGVJ9V@6VyL4?b_mT`T5G7dzwXDQmxrsuh-w= z;?&V*Cb%U7T|pE>umjfOWwu%G{mg@JqWxSVi@O(+;Z7>Fed}HMHrxfNWx!;|Z-4X( zhHZXgvf~TF0K&q-tjc$}6@H_@=oam{$2f|)2#IcaUN|(?wxwz+HGvjiML7lC3;wcP z%DkHVx?J$B-!8FbWA!s;t#!jc01xYTSq^UQ!lCN{%Y-D4DpwCrt;~O6Pe9UzihQ2Y zV+MW^D|B_PmY8^wJ_RFytrjM$YW?MRaF#QQi~I$oRVJ`1{ixy|Q8U_Z_RCF!_Zrfh zgo^3po;+xEV7OT?0AXIJj&jdPM*!3=rYvWnTdJ)mYV&4B&GP)dG5!uxUc*YSTY1c1 zZRWfHzH<4s=b_X<2c+Xa>#N*pqug5*N#jb%xOb*uj$#rFAzl4~i#T+y8+b?BxPF;< z5Guf|)3}761dSu#ozdP)1^*I$Se#EfmGIYdQBh}Sfcxb1>zyBpae?stL z$a$aqoU-q>=qb)VNe@nPU!D7bLY-Y&*V*a9Gbcx`#k891*p@~^WfX($q_LuiWBu}? zAws>L(T5ow8yhR;bXX>H;W0TQ~mSo+uWIY?P`dts9wcpl*MYD=`bqb!S9L-&x zKxHQwZrUsKfa%@0x?`qPmyxX(paNJzkmH6uerkCg3}qz1+Be29Hn>O%``5wVn7Y4w zHD38Q_sZlJs$Q+aUE1Q?qJ9HG9;V4PLB*`8J?ZFT(`$yE>EOkg!0=_I9kt8VsI&D2 zjcn+n@tc9xQVpnTxh~x-dP8jWiF$eh2HoGK>9kS7`>sHRpsBJIu^_rLT)n4=NC z$MVYXtR;(<1bgQ3ux{lb*X*n8d|R|K42<6+8l207c4GC?vd`C<8;7n5*6lri>Ds|o z#DR*KS)wU)D<4inn|;B(+twtD&)Bj;3ycvm^=qj<3EiM?7t_*OuglN#A@nMrHyrpw zg@sA(ENm}Q(a^8712%^nQ!b%%r7*XJa1KN}#Zu20OB1+0Tt%hzp<~sS6Iz zJcz=K5a+Zvr}Ul;of>VqSrTtebwg4~?oeFB?4>^wWF2r4Lmj)sa`Y^4=o>M*^3b%T z%TwE_Etclk(9QD*d@}AqT)pKBJ+2E<)-?8)$~$?6X?}8_7X+|usG!{R{(NCSBT;VJ zXcavqoj5Hn2o*yyEJagD=|4hFk5kt&_l3c&8-x#cT9}9%<+237%E>M^_pattH;w!A zZSlflw95B?{8+sIm6L(W(mZb?btr^JqZJ8n`TDV*u=9B-tS4a0L&IXcrCtR+5eq9# z(EAcIyL23Qc2;&U+&(M+_HhN39`7Vhp%kpTtr}NK0lX(hEc2usgYOE`-B8?`8?ih< z-r@gu-1o2uvQ74^iVIIPnsUcqCVxc4d@3G{>Lql3Z%`H>vcmA;yvGc)ZBjk<3Zndb zrF%CH?4__*`3tdk5;kpAxKgkyU|AJZwI&^bRABJ=+8ye>rjbFve=@0{Wu;)|aUYd& zSAjEb2of>9XwCrVa<)&Z`I&jA7YNU^VOBW`E~kD7ETEnja)sPtvArSCrdO+8B>&$} zOU3Nh^M*kpG)z}ILIN<*ARLwmXkOtk92fU>5;7H%GR`itcvW!OgaOId#7e0k@78}M zb@U@7`XiUws~{pBnozcanOw&WNa^-D1#>e-0kMxX*s>CTOOKcKP^VsI^av;^IZrq| zzgIU&rfp?*ro_{6qYwlMd=QjXpYulxo_KOOW@=x}csRWqKQTT{m?ud?J>gj&wDwa+L=J8l-5N|uM*Wc{X1JR8&_ z;8ZXfSHowS`r zxWN|kD%ZF@OMPvl&sSft{N58zr^|`X<+wRfo3}+t$yQ^%|*GeN0KR;L@=>bxIy_$gP)x{}SADhjx z72A=9VDEA}tX_x(yWVM4jwV75Cqw`dV38)S@n}hAn(x&${NY9dqX^ly^47K9$NR6z z7$GU5As6>z;oHt7uv-W6K2m=(0bv~6B7tk0v5q=1vrrz}a3bJ??VdKv%grA)xSH0+ z`x!S}k1mJ#G8MqPHb$?$L&zB~X5V2leHB?m<<0q43NYYRvR*#CNQz5Swo6xdtL6<_ z&kLX{g$`ytS^}YhW8aKiE@5$!reAdKNL$((12E*e6JmBX#sB>@U6+O%a3~|`L17=@ zsftEl=3jc8R%Zg7fxhBShhBb@#_WL zq|AE>ac`v3zF{Jt`rk#60eHqK6pASU{H{+7=ndGEr@vz>#Ryq|h}Fs~DS_D-BdGmd z*QI>Y1Q3B%cuvAHg?gx0X{O}LBqJ+Tbuy?ZaQ4u@BXM?l2vg6GCsy-^#wQmXye+hc zXj@mCy?f+erF>z(taO25sl77w<=V`gFe2z0UU^a{@HsHNQGec)rBOrR_0-{hlimBa zr^wvw9Q%D(qxL+~xNruH`{%0vrZcK04)9Cy1joNi&z+Z5>vm}WlK_3o!SBk4oFgW=GL$8`~r`7(a`WdiKeASP2+f{7xFp@rc=c zR%Ugr#jK7B+uE$#cOgCN%ygj#7>Kgy>Ui0@a zxMh)-MpMR$QBEs<*nKY70j|zd%o+3Y>Kh8#VevX}-~S{C->PC4EkOzM*E`hCZzG`r*R9Mj~aL`F-2J-!0%JI~U|7u{jG2 z8*{>XLt5CE?ioY2W%a)~IkhYcb;oum<(oti0Fww?Ml$f%rdgV#G{WY0I~j?__$;A` ziTi|4;wvN57Saa`q zCe6q&_Q1H5K5jUN$V*II@Z*^jL3(c5ONrt%2798WIyJ_@18U44sT?&pvEI&p02asn zA$>M3I^}?K!gMmkpkdpKc~KdZ8ddh#to917T8`Gpw~MRR79GdjHhRiz>seF`^n-Zh zm8JF+;b24^^!Z*Ky9sAhwdQ>b-TrG>zmM$AcA}*1(wXLLAD%9awwOt+??%57NX}@M zr85ASd|~3%*e!`i{mCS*?Pt8vv(LvqFh9DTx@8wtq2L(yb3L(on>&DV3;a!rK&*lZVt00+5R?-Vrhw!d{DK|P;RimU zlMi;wxrPp?Jauf3=#u%>%gJRSJl>fzJILXhKrQ=voZg+Y>x);;L^^z-#B7DXul$%jto`W;_mKF;}aEJ8Y`EmkNxk?JKn>KKz z-WNJ3c5du2t*!T@X!EkI%hV$&giOznU>R~ZNJ}Tl?lSI3-g5%; z^{rH^M2}n{QX7RJlFCDSi`iNU zUdfI>yUlmzUbppmW#9#;eIYg_rCkkH{+?*Xp2lY5TPFJ1^z* zglGc0Xu&MEkgEGo`A5JxZozXbSg$)f#&IM07G2@~xS%Hefi8gh^(L+=Yk}`9{HoYf zZV~=aJg!>nQI*V0+~&&2Pw)QPp5;BCAAyD}N_t~K$bbxOSe*Ctcc1yZ28GB?4}8Sr zY)|x!lhe`wnfv0v5vE#`8q_$9uIHlFVb14_38{Iq4kPd4Ut*ll@$rF3JX5VnPkWe* zaTgj{?S}8%5WeLfb8?_)5TYEt`2b&uCK?r8Bl(UW_cg#Xj1Zvv5hGYbw$~7cpDXBYq6n(UN*qoU+g4W=7Gpv&#V=#-qDi(!4E(m&fg9guPmvn)a=0)KU z6bd+_QQn5!o0U)FOf{}9nRgLByBPhmNcnSBo@@$EIBsTkcKDyoxq=yD1h-}Vb(G50 z8D$mVC3A+Zx~Y8%n3# zZwZ7#x_h?F@rh82DFA~Xmyv25@Pherg+T7`m%eeahlP|&4pRL zv#*XY4@r(07O*G^`3vUoipXRPMTc6$tSl6*V`b^RDSPMAxvSNm;6gLZBn_x}vE{uT zOwSbU>iZkYFGrQgLpjHens@YwzXuHO)iU+59q9YW9u2vM?xVi8Qv2=wjYe-=|NL!2 zM~OVn^bdAriCvy&-!UR24K_^=EXIp}^5 zlKYDVBTunxcLb*kpZ z_U0XY3}yMk)$ep720}t)Y9AODnOEy=cRPcK^z^l5e8C9tCN^jYx(vScg5+`{x_5&+ zz7O8~$YCEG^=kew^j>F{ZjfSKUlrbt`?456!aqo1+%Dzwqhnow3RsmpvO%u9=rKV= zTa9fJ8LK5{n&-GF2{^zDv;PPY^+l*|V}40ds)LS}bq^UzpStWYGXTVUwC5ZA@>y8)kt{A*sAb> zjS%gviTB&I(FnN5p>446IYu5S_j!}mkutMGn#i=__c!)oB>`v6_ubX5o6qDao5SVs zbadSTOSpS}A7eMgW7#BZYkW5iV{0fHm=u?IB(G~L?X8fO^}&gDMV(mr)aQTCP@LY> z#o?M*c&(sV2$w%6II%XTF&J{p*CNAzFt!p2Ek3HsG%1D?yZ5j=@#X>K9d8Xzs<}|pHjo@5F8Ha= zc;Bp^I@^>h^}+lXvft|Yz_?-Ty3UAfB_r}cyrIyAu`k!7nb{yXqI>+RkPGdjW>J%q z*Ta24;==FOp5@w3VaIo;GEZ&3*vQSr^RA!({5`36*4L7I$=*svk+8(ii#n+Few~Zk zoX`mD;99iX#!PjT>cPhU>tp;Tf8W^uQLM6em#9@00g~0?%me?#3DO5u!e|~FaPv+F z%fRh=2U{6a;}3UU?ofsg3?TeG)-+8YAZ^F!A&-MgY<3=c0#P-3NyvF^iOPvV4w0C9 zYNexxy~REH^WBqtIm*dHmnqfYK&kKhX2Uz3=iTIo4xDAZ7hx3X=XrT?IiRku77yz~ zjz?E3ef~N)!#yOsv)|adS-AsTrK?KL$7mDAWsDW0y}kUs2XIL&!JeLTE6=931!!w_ zTerWt`3~F#IF9~GruM2;-XeZ6o1dw@>R2lT%$JjUqAq*yfuZ;O1O%^d9-Twp_`+1Y z_MVAj*R3bcE-F9G?w&ZTSuXm_=u4JLOSKgTkBT}D5*PbW^mm>J?o|%xWBbPM_cVYa zH)H)wwmy`%c&i<+x;>dbT;BDeIP617*C)8uwY2X&%yf+1v3h23#m^qkr#kw%Ezs$X zq)RZI6=h)rp`Y?lsg4qJc$fR>GeIZVDFc-qyXlU%n;ZvNFo;nUQm2@bxzoDWIVN=K z)s#d>f8EyBVhxmLnb~kn8c^?C+RPDf`+7f-5HzUCl*J7s954zU|C#w_CbY8e*?ew} zpjS9T=wKR=nEAbBwm>t{yl}&^n)7m6_r!OJtg?5Qk#$((b4>$V_%q+NuC28Zb!rcI zrMl~A+bt2=S-?5}`#d?qN2-08Qko8AogAt>GRvEoE^ekICl$JQN|ObN21DWmox0CD2`qi9S;2!N>E?=N4Ye zI!^jkLl9y(rmdb5gK+smb?EoQC&go~Y)daUbyZg%VtSzLdQr2CUH_u4qKLKc$BQ!y z%2?Gcf6Hn?lAD|7>g!!S7Ukd6R|L_AVh-G92pBq(QGV1OxMrxwfaivqx@dEVNz*#t z#>UL%1@L(A#lzi|pi@&eK=o3l1z%z)q>lL5(Aj6nMjFWdIrTK#s&>vj3C{5n?w*yt z<)3%#Jn_ToM?Uv`uPj*|#N)5bwK0#(g#e zB=)w7K4CNu`hI0k)8s~9cFi=mmbH*9Jyyd`#u#w0pim{uvhUV0i*&3NOeS&wahNt{YOExy6pb+spJ3f?_GPnBG54CaNkA=v7x}C-g zJM@ZMZSCVt2du~vT^j-()$yiquTGTLW_IL^9sk(IH4DU2jq5>3OTe7!bbHn5MyUsU z1E2n<&5eLbbUNDO(w^cm!8-OMYti&tJu>UH&R+LPWqmOL(n7IoS}q1=WOIyGS-X9g zzUcv92tv*SXo?0CG~MlTz5?S!-+lemGXq6x5-_3H+k+Uyf!oNtM{f_7Yz`IV(2vtn z)M@5J3^t=Bp?%&FkkWly4ly~W2fgo)(+r)<{jN0#5lQra?$fkpDS+PVqWa{0-h%e(#Ic%7eMZNB z0S|9`<9DDWLxfa4L|JbMeqQK?*tOz!?*;?Y|LyNFsrDm3;=6{2-se^z)Y#hJP5~)Q zm5kVP1EX?CSjn4`yp1w*S0~1iIYyglecNpy5hKE19}=tsTXQqjkum+cf)aVDzEZY4 znX}EZ9lD)iGgRJnx+Ls$NtZauUALkI2s=Tz8xFGwnz33WTxA#Qu$YbW+SS2w3({d} zcTbWZGx5j^CB5>Jf_5@=uRB=YSS&*{y90-@wo-EhN<%}Izj`X_Xm=SF4+a9JjKAGV zcN_jkrknO^nFl&}q%1^h!LA1-ZgJg6zsJLgF_%7=+268?^Wj3qY@Ux;JMZO<|7QBG z!OCL|D{PJQgod!tu{mDoE{;E~Zl+Y&S~&t?PW2Tz?kxC(=6Az6W4Yk=1mm83fW(XE z6tpcio_1#5Vx%%D&_xqU3>J=b#7{vv4Q*0kwHt*#xbL?}t;~aCj_s_{zi3|o>eQJ# zm!!_&Oo%=SHK!M{U(g>>#Wm+H_6Ii`bC!h&Jx|L&d;$0Lve>!znnUypke9z zCjIa9J}c#zORRvmtZ37WK1ptmo%!70mIFh+$dB%oij`qrCDn+6<67hX_KYh2`b3yW zkTHJW%)Xpk^iy#sQTE%-o^iF3U1-=vzCYGc{^wY-E_V#4+g;rX0SpitQ;6keQmJ;3 z&gy+qd6iufOB_zUkTWrD#=5ZE*NH^>c_^d)bkSUdQT~Gq^EGlhcrhP3E${Uw+ zY``w9nb4GbV&k_6xCwCH`5_Sf8b(gdPt+`ETCH$QceY_Jq1wdxQJW*WGJ(%~o|I<- zQrFaup~0Kw9HD@rf0v|7wHDdF4qDF}Z2W4iH5lFbmGxfyS^5?>uO5L;hq*{j~n=)|6qrNyw z|Ca$yLtSn1S4|e3lWP-kP-XXr=|DeF1wJ)&&gOk85-_Y&T5Jx~OxZR%N)-@@8}0ls zlv6toYDWy#cJT%-CO+P%p{(`0tP_Co=j@ox)BGx6lsBV>3w?9JaO!+S`S{Gf&{R{A z@zc!t&>QV`X6H#z^=tAtMXy0s93@H#$r7d6LsMUnMmd1FZnIeABFtj?wEUjosTEW3 zd$+h0e*_NbEp5rrG+~^2bfxbXR6QNzAMBkIc~($(z55b))l}*W zu1k}!{Nh%Kc2agcEx(m8SO6t0Ge!v1JJ4MC4>W8OZHA)53xU;#X$A^#{Vc%-62qaq z*KPi{kU6qqKyd%h3qrvvQsmWhtGaPjbU(aP`p8}REeX?fuRboU+>;lFqn~$CGWJq- zau$T|)VMB8yS7G$31!O(W$z2#5qYC`tE`LhYv;QIE4K38zvtXxZOEdyL6)jtexwF7Zu;$?(lBy z{16G{#KW8#{Qrbs98cuPvE`#p-R=57`Df z%r_PM?Ad3!#V_v@?rTpBDah9vw@zf_j|7~uHFrR(e!fH_F0g2?Ib-w^oxSMMB!J8hPGR zwA)0g6=!$v*X(j_K(LdTDSdE#EzvdCM~Cj9fQqLPCiaL&KSQ8ELc04G;Fjp()DY=a z0!3}l`bzZ9=Iyz0U}5F`mwyba5Snq>_Ji@TRNJK?!DJ%)dD~ZDOZm<9R(ShU=b5(9 zC-B(UQn?M`7m%gggVRc-!I!MF#Z4*eSX$r%1>`Z`yE^qJ$zuWAC7{nb=1pGb>PtzS zXu6CTQYq;$j^bpsXyToLT``5*7R_S<1{IS=+%|GR%^ z^xgzKja|G=R@y*zzTuz}oUr@M;%H8{IUj<4Uw7}Q)DEq)7ETcprcqO1+Z*ukjdZDL z`q`|K?F3g=INpD9R)(~sGde;U&>7lZobeR%3~|^D!7uh><}iz-Ew*4;axcbkMsF{4 zE4R?9-&B#nhy@$eNyc3?r(_oV)EipCgJ7G?g2L$MizsvNR|Quf``6F!|9JzbVTX+@ zmC1<9{^9D~0dz6c?-FI=Xy=V0@Q~D_x4Jl95Vuz*|HQZMK-NGzps2HnmLQosCb|DR zJTTEB*Is*+dWKvFXWh)e?4#(6*~M<&PwUVj7q}`dV$NI*&3a4))%C~CFqVQ7dTVw; zys|KsV7O?x3j>+Cl{x=MQus<2AWv0{!sc)*Capbu`6QPZP#Q?jC%`<-Od^h7jgR9> z(mG_S>cGCGvhMY^n0hzXe+Bt2!hol7Y$CN_2cAXd8Pc_z4^U?oHsYqYrnl0sa(NhN zs90`1&2HX5UXU+q-Cs@o?;8XD{h_5G_{y(u{o5aB%;#{{9|Cjh5aMBYHgs{kr3md} z-QHM(wBevgVf%c}AAWCaE&fzH4KdSqMD6vfQ>|E~kQFSSx@x(+;V0GLA1kpehIa{fj|IrKKXNfV$J zs})e*ta^X5Gxf}1uY9A+Urqoj3%i9Zd)6;NFWv3Q+6mA-T`_smZ#YjALE%;NZg!#` zQaSc#>I2oQ&pqPdSMVI%)%AccbRDP?mPP=7UeeCt?@Cj1+gG<2^vA~mAcNk_RgQzq z%7@4OD8au|!LeD*#fL*}58lH02+!P&bv<-cw?Fl6XG-Z9;HL%wKUxwd+{#SzxL?gA zcBn^U&Hwbd&$)mvORPULk-JT3-6c`4&|26ndSW#{gZrg2-IzS80av|+NQrk&aWr;o z7ZudqNBD2mp$(VVX`MX*r0m^B@Hmhku{Xiu;Z6Si!kNmV1?)`><@g^qrQ4skHz<{O zc@;f_Dz9|_dC3`-dbp8F{-{OL1b~2LkBnu)&&4&A~g z2#a!B8}{+WVS3scz_>HLL6X)RP&@>^5LJDXW<8pXS}fYkk(m|WSRVsINQyn$v8yG_ z=U3>njLonj<_F*IS-)ko2rT7koYyFr$#k^eH>pu)Ks=I;_0$IKpD5in#i!#`zX7~! zY}nS}Jplzg)Q@&02Yx#F)*Wu#BbX|9ZhM9IQGtqe=n$g+`YF%tF)=LcLYCyzGtg(d z9)QmC4(GC!}q5R)>wd2tjLw&rM){vn0nTo)D`ah%uq+~8=?DDSIJm=V0 z9#d`pqN2g{@mBh~7h9y`Kmclp@^B&1t*E&dwp~E zr?2sMHneV1=Gb2uceAcGi;mQMXRb7nPF4P!XQjw(Rj$N6%S~-<%=bCAlZRn7ZvIx(tivdU zLf%=fWnpID&=EKAhcCI1e#e)?*1|B(&`OKCn};TigsZZ-eJ?$pEl##uTXz_^&@=*h zOW#Qw$=>-i54QNoc$9N2fZw+wc>r-Ae-T;+NO4L=t#E6FCANbt^~t^p+I5_4?|6Lm zzj-fqZLMc-F9O8HEgfcdxY-tkP#ZGTov1dveG_l#gs0okD4$9uK@X0f>Vx_l3j(E^V6#ju-y6C>CC4du;g2?W`Iw9A-dE}V zzV2sQy#3>=WyyAi&PtP@yr#x4cp1RO=U@#C~dn8de3)HfLK96GnJEs(mHQJnbO z_p-7u5JDo->vZlvRS7&cK&-s? z*45Bj|4eVW{R>ECazv+Vy6ihsbaF)cu-5(_*RcS`H#bcil z-(@RjBB2k^HzwSLpq7jGXuPNMT?Ne^*ax}tdPw2w=AI2Q%!E!viS z+YL0X_Tx{w@@^#z<10=VubnPwx|z`oXp)^prC79XjNDpI<$h@^C^2*JcPYczv2*J& zgmtm>0+S8X?JXNs!}c`hH*r9XCsMZtbD8I%8P9Nba%F?v;WY`eYZiQ)ES(Nkw)Nk{72QR0mi!h=~xZGG^S0J4#(?qM89&G?r^9QZ-HSzZL(i{{+k&v{op2kuU>EYtklk@XUv`+%YgWy^9G zX`msKZK>vwPC94EIha|gBa9a2HpDD=Ig*olc1{>RdKY%AQiB|pv)yzY0I7x90FWAe zZ&9$dqD`yT6f7OQo4noAllmjy>Bm>ZI<2;-Ky5d9*rzM^Q##xV>tuqy9jLUc_$lFng#HNJ764 zWppC&9&65-*aM3QChDvizX3S1h|>lEZSBTqC-O^QKvmmVHE{WaAF^*Gqi-Ma*I)pR zeYgq%J$P{u@ID&`&|+RO04=_L-AGCq*oHo;f9=^gmmcpdv*s`~erY!)nzFO81!Qpl zk8UY){AURVWDOp3n08gwZND3MCpfZUZ@}e44TSFuP{afzD)8QB^%zj5?4Ne9WdRTl z=eZ4h-WF08Z}KiM+|ed~!1x;#skQ{h0Q+XqJjMGc49Dz-TIKBXT^OJf^e&4_K47M8 z|Nrx}dL$h1Jc>JDj{bZYY${^W&%UIwuozE3UOm+8v_ZI4k6_9Z0nKBm=WAegP;*W* zyvq&{(?c8+dWIp8MH%SZ!Q1|x5-cUO`0b2Bcdew%k%*ZCackgD%q(hrgn!aQ4ggYg zwqA6$d+eOu3`Cm0b7DQK{M(8QZ5H)>!W<}CF}JfDuU>Ze5;Q5Y4i`I4N3XBheg?9g zvY5RNL}L?9+7aI6Y;5G?teY99TWgz!d0k zSPw#*bvvvxTG{@A0b3tA4JaOZHJj(karG>9=fHp|&Neswr;P=O-|M7Z`!LcxVLmN8 zd#(Mzpq+Ir5ML4;QfCU3S(M~Z_AP<=o7pmpG?;^*C(onQ^iwi?F+ z`O(~^oSf4@^z~2x^8kTNsH{A{Z#eKao|$hAx3s-SKLxDy<;;x}SWcK5XZLgF8K_R~ za~G*4a{O@rbwNOkI%ck6z@wFd$B zMq*-)(wAA$v@M@gOvQRzMcitj@)mO|tvyk)Hm5Ek#S-`<_d>$WsL(SjUHf3DP>M}p zxaG3IzPGKKAFMK@jEzURd~a-TEv1;7EZbUGlxn*;t4rfP{8WsDF4;G znKL1Y*$GC)As7tn6>}xIi9S_B>0i~Vv!=Suem{pn{LEcbW;qv&U zeJb(tz@W(N!(GgMxSR>K5=cBdM}s4;J=}_}UFd54&iXT3Nv_G0Jj{DEzpqi4NJ!G3 z6OO)ErCHWAsJL~@&r6(tH+b4U=$76ZPV=MFLPHr^OhFD;TU+CzaK{!x4zFFfy;1$6 z9uwY{P8!sx#*AE9>1pk~)?J?Cj;Ee2Kl~t7`oj^|)w0bQ0u7tR9Pb|MjB`!Cy2p;Cy|* z4&+rmx@Udg#&X*yq}y<+k(b>qkb0bvJa76ru$($wU=NU`tJ>c( z;m84DbVVHXKgXjCBB((P0t_o~CJD3=nh_2o$^Zpb;n@vP)(Tyb$)iGHeuBS4kF9Ac&X&A=E;&e06bVYda~mzTqhahMAd0Pk{n5 zZpzaVZc^tgIzFaa`u^?k$?Ukk3FTFXL(B@z-U}xU9&XgLtHNTqC=^`2sg+f7Z|6oC z;88I@xviq-!n?4zK zB;y@5MDH;*T_Sq?DzmDPg~k00vv989c@0#+n2Ro{tDh=K4Y3AFpyDXA>+DS>s43IZ z?Qu*cc1<0Bp812b9eK}A>wis`dZnYTzlK}BFA9=|Ml=@9nC4yW8VtJ3{YC2F z!N7ye|8wd__={n{GwB?D(J3ujk;?9VT4MZ)w`E4qcn9+N66sb5Bu)s$J=tT!oMY8d zr)k-i!6it~X6bi{JT?t;0J26xmvKtzQDc0_gHjw$Ns#;{677Ud)S5-zAXL z=R~RG2nAAIc>^QO35fZ=4NUX$OXF{WC{z9tYt4F#kgGuQlj1C3Z4aaEw$w1~t>)gg zGIVzwR(pn9>)fGQLdJ+9`5F7?6CWoeYv$Rc<>OME*W z3O0_^$Wwu6*hta?9Y!@^{&5sP+wh+<#D@D-+DQ%3Q!>EloR8&2zQ`gY{vcOCaqD|Y zgJ1{N2&4&EIu;8QAXGzH$IK&tZuxqOiSv>ufV}w(c>rUFZQzU<0%9=eOp(T*Ro&kh zqKgS@I4q`GZ*Mar=ESmVV7P|>a%$gg6rey%R9O&ymIut zIfwsXdBA)HL;_Ayyw)j1Hq^5v+^$K4?d#-RVfQvWEgkvpqtCn^TbLIuUJM#w}1ezq3M7$jT6Lx_$R3I-`UPe0va1$F8a3`*CPSn?=dg5;Qga8rV219A-nyZ2!#fAO`^QW2X(=G&3 zXTgo2c1$M=3|N`}!P1!vMXY0{U=u!82fR|bHVkX%q8?#qTSiiIqaUnYjDBKmF z46K$Qi$js!Xe>iHDZ;ob)@@GFwHzL}5Rig`%sb5?k;USiYYzWsB#to`cy0aKot5A( zSfjI*rM6+uFEzoc<8%CC-DZopLwcgMB#TRUC{($b^92onlTvU(Hw;TK$5sQAeOBt! zn0lPv;_9Ajr~6h>ALs`_rGQ?u!P__P^J>rtk?ZpS14t7yN1q|oOR26s8Rf> zz<7G6ALF0rdVR8b%S!<%))7NtyUK}eSd5(NzopcT{)&p4I=7sGm|Px~yc`182>~pd zyt7jB!DK+6RcJr(j;TYOpgLGptIjn3mskQ;aI4%Cik}OWi6^#CfVUr)a!YKqyZ#e^ zAK>;ek9PC&5|a+G$0tv2GG&udQ+cJeisWqQT1!JoC$4l`V}BQ%zJkD66H7C7oi^G* zTDqnej&+y<@c5^YL%%vakjxv+dDfr%Xjjt2PR^tMLeqizS;3f4vfIpEB=^;>sAms2 z>h_Ad>mMqU`W155x835F_{pgDIQGWL_p%UdZG3`(xBIa`|CDvNABhGdjbchuGi9P`ZF!=$Ue_BtHsFup|ZL;D;O>9~=Gi)MGW0FLO zutv2Fzjkl>E3^sjdHs{1i?OA+BNU;Lk8Jt4@A^&}>XH78Yo^bAy`Fy^4l~PuXtQ^L z?&8iBM?WUWjy}(GI488wX>S?gYqqb^j&b@Yjs;TkS0Z1(v6qgD+<&;(02A^Ksv`KL zzIS2q9^|~~DUW-$q@ovgHAS)^XS2dgTHZwbb|D+Sth&^7*{^>C?}9>!s|E07bNP~Q zP&qf)cnqBsY(r5$sJ&E%JZ)OTk>8^DMymBxUbw-q1UA)m+4`k_P-$7rOOE0nU55ol z5SWX}#x3Nqcqy*fFLw~}YPWIpHmAAt1QVd)!@$kB6v9YZ;MNVqSR^Nyt;4#A4@Jg?G|={eC{xPRYTo95y~rXrD4G^#Dv5Oj z0X+?7%L63F$mo~4Lra9=b|}|s7TS=}Y{|C?@#jGG+45?qXmzrN)Y;a7-1Uh?)S~@4 zt}8z8!9E2I5gcp%88dq_pu)U8f@6z?c|)Ku5BBIIz}@0DO1_bbuikxwjc%Mw0q;5W zf0W^TU5AVMv6CBg`mCddpxR8 z*2A=}4-Z=Xf)PquG#>YcCjdcjBBP#nd1et-8MC}@VBH7S(oO{~_ zI1;RY4;_qyHV+{C9;qe_1U?Im26hc5qy@k#mqS4qFnG9ItG4)T1F(Q6C}wc1s|HFJ zsww%it_x=52EjEq;n@yml34fXLH-fBEkiTc{`xQy3KNpz_A!r2Vgtp$ZXr!7LREWa zAW{RyCw1jU@Z=H6g`0TdkqDIrHU+|Y+2r!q#Gq<&q*}P6_@IaBI&7Fm!d=yq<&?kX z1z|QD(R8_Y@u;J6aCD#Z&$sSxkK3mRL+voVMU!cMg-M<k5ikzvoqX7>~NcN&bR*DR@0WPgx|!!uG%4vLXF1b zMR)dw^P%F}&PDVv7%v*qQFpA)kye=TLhci4mI{43zd><>g1_G$iZX6sNtv!&XvT-- zVP! z6vaivGt>=%*iHrHVPO*ln|kz5yXe!@$w9g6Gp(wygoDBx8)KR0g!8JqD2rwmSZSr( z@CJb05(XyXldh&@^)Ru}I^@eDl-i$vxmtJiRg2`?pjfuSOZG2qcNu8RY{yJ*4D(|K z-_8hvID!s4D7B)4WI*-9Y~LertW)PE&P?;jrA5cl&uOeuZoI16XF6k~c$keS961T% zyZXjAI6?fWDeuSkpx3~@FIQa`c@5yX#OW)v5!csjgTm79jG;#SfXZ|0v5ni$ITrI z_jc@j<({%4^o}F4oxLAa9$QZTU=g`epqoxV6Liygx_Jc3;D?N!-Kch8^yq9u>y;O7 z*7@^=hcuikuXIQfXd8g5@!Ra%4#Tf1B%%k2e=U2FIqpcFS?ehOQe&WoC?d$7F7BE& zIDNQmjF=CMwbW5vn)X0q(O0H8ibtx{r4} z$#b4=u|bbS|Z=0 z{M;bNwAptfRWY`nbhHSaZfQ80ji(&}xz;4O>cUQ=E`ue>8X&&|O1PMjB%o!l&m1Z> z)cdF;-<*+$-KdSaNP%viDZ8M*@dyA1M5!SygkFF{caN6fJ04HmtJZJeAj~3mL})&- z4YBX{3^-la^=0W7&cTs6GasbyqW0fl>6ppEr{3b#A1`daK2valc8|QOW20Ir^}?S? zzky4dVE|?4`Xa21-rI_}&x5SHY{C8$s5b81!VK^1G#|p&1d!ghPy{uf$S3zMRro>m zwHqI^d5?1?Mx2g_g$ZNvw4(#5zLgK3*y$+~bar=#yobz+PR#SAss+q@FNE0Ot~CSN zXcPkX3?{ENw%Xt$Qoearo;8jfW2(C_bcV3cs&L;TqG-RO_I`f#67>~^%U!&x)l!&% z{(424Aid)@GD+G=eU#_s(?Cf{hgzA((y#CCk`PN`xOgn%&Von zz9%qHm|4cqu)~%ms$J9`ZQ!jZ7tWFtS7SZIX{EY@W6~=DVC|_+)K^+ieu-f!@q-RS zXa{s^v8js2IZf+4yy7nOtBh&A<@2BsJ$?q@XmlbW8=iR0Wr@N8`);-26Oq<0!uFRP@KteulXy+!*&=y8$cGGfjvz^jWB~ccg3^{0WT$J^$i&pK85#v8<nFC${{XVkqx0j@G+cx-ejRu4GSagcWM`tv_I zYS8zxmwJfr{H(#Q!gIjE=h=PSCF6GKkNlT^C|KpR8lh7L= z+EKZ40+AL#w9Be82X#>2rxufrK7gd7Su?1Y{BNwSpgq_$#GFvw36Fn~37jXPdWpFz5ozJb=Xy@-imSJ^=-n)=Zw(fM6>A zcl>{&fQUxTf?1yGlz{mjTqnNq4Stt+^4nEXpQNN~AH!F89$a_g#KseBNf}(G^oc|F zgMT>@``J?=*2C+_ng46T& z<{o5IC$dK#ri(Jhq2uu1G@_1{oWKpZ|M1*(q4lQwkD}+7E?PhEg(kE25I>Gn1J6Nj z+|R6rS6muPPv0;s3;lBZesuqjL>M|OpdlQX7-zP#?r};teSM2;jq|&R%X9y69uEsu zRM4r}Sx=#6?xlf8Hop@wV(Ohzo4jX5Ql_CL#>OP&+iM=}T2b{yX&GU9IC>VCSS1HX z#b-j*FZb;rwIbKX`MKfO)BpG}g`CdvuSsa}4*4dwE$`^4x`rBf>UHc2_Ss6-u_!PB z+s2D4*o=uS^mJ!qqjL!3rfY}|n^jb6k44-%ssyI78M(}O6<=-zWDDH&OoG`Qi08DH zlxch@laEKs{ontY0V|eE;|m&+dm~KGWahJXOl{hn0+#OBC_banAGEU`2>)G=FNq1` zk%SEXe4c8V;8}^T`4BY6B7FtEmkBap!Jx%Ptk%xSV~Ehf%)Ac+-k3AnT{aB9zsDs| z%6cO6;DSb71rnO1t4TV6l$XZ5l@|D{qwhhM$8mlaeu-%eu3uS-O&Iqavqnx!bvbb( z<>)Q&w$)wfqY*sq*P;bvx#P45AzxJdny5lj4-h-2-PagwimPA*``}#s`+C)o)VFgf zB>pUtDo-Dn-9HUK;n0iq$MJ_n6FKA>O{KPqJa!(l{vlaO7xozl!}#?bTYFk5kzj>E^oyuM@+Vp*HvYeDGM_m5Nj*y*7?x zG4VTjh!-AD+)lcg=E5Y%R4b)ex|#@l#x$rHLFNYf&U#CP1K5br1J+`OMuQ7BRMC6I z4^$NtjJ7egTM8|YzFRt_7APdE3HBs>47+S58`D&YeKdjIEqJEc7<^`k&@DBs!i@CI zOcE!$l+&yC@mnxuKJ;U~D9jBfk~_&Nw1;P8xrbwq&H6T- zaR|ny7}Kh=Z8|RVEy%lFGzrG})C`Q_(}M`t?6I)3_aa=A_#x99=T1=szYAGY{s^q0 zEoD+qNU)8NEsXQ#hV1)Mxu8Mfc7KOU^@gTgI&`-z`CXF#m!~E*`Tp-OdD6VI7-5%l z0v=n6)$;rRXP-MvGLKU67@kkq`Z6L3?)yRN+sGm9h!i~~hm$+@DO~_5_!3Ps!7rF| zdxamrnP6&nkhQCXP7({WbC)|Y2I-+kUyKQILr?kE7`76((4{cKqc|t-FFfpWfg`Gi zh*S~V6UP~Ydys>VV-MQAPyA@_$tC1k>?)k}y~Byl{NAHQa?K6p?)9!wSZ(u&uT2dk zQnwXVNYRqXdz8%N5twG%y5;kkCCNJp=Lh?Z_p1E%BF^uKb!UFX34T0hs3h4Id+d5h zmzjvHm-FPy#2b3*wv4u1&Ee|y*m2R?*@8K)u_O_Eu=zHNTm3=Lq`Z(J z)9&Pa{F1SeEJ2ECveK_}4c%RT~5N~8Lk@_;2r^YxnIsUS?t!vHPG|qXtTYBCyfce3B8f{hv+NcM!_0wKRkmGh`q>3t$ zln6m|ac)FKp!`H>MC+XcE7P%il4p+h zIGxbJf1St6^h}4b=v09JQXd_c0VATRqTa*fdYTk|nY?_fz>~a})hs}{t-KTaCjZUs z8or)t8miQ7Ho5+oub?2BW4}R$RgLyqaerK<^I;;kf)U}`RU*e~WAL-shTFPCiOi0C z=KDdvzEZck7-od^);#YB6%R{(;S2s`>b&7jjpricmXWHKs@bL9=-nY&k9iW^*a*N%60RjIG_8pR$>Z>1{@g z81555`>qg?!)8xQiujCo?ic-Gr{S0>p8b@ZW7$*d^G1G$=>rV2=23)~clr?ny)@Lt z2JRI*k!7|`Qnrdqd>@8u%##0Wu9kJ?j`ylu6+@Yu$_sb)Ynn{`I#%b@?V@?wuU7_6 z-#r<)XP{WV*MFNSkgjMG!R1S6e%9HlYg!LCSdC@h`e&+A>BRR^{jwW9+Ym7Yh%1@H zZ$mn+t$t;;i;qbMw@@UlMEEAx=_TiE2yu&*XKl?+lhNV^9jhqf-ZWI;zbDw5>-6=~ zCk-W!3?7q!;Z-dRH&~zue2I`g3H?sNU@gmoQV&P%@g#uHQ?RkISCg$1-b|GWZgKx;H<`Fw*wMMQ5>G=dT z3xr&_cUHl-%Sj8kq#sYybVJ^+>~l@8$m-0c@+4ZzR(}ZUu7?xPG+fsRTY;1&9G({IC|&FXRL0qdMsVK|+}${i%lhW5~|5sA`KCH2;^*`ZoH1Je)} ze5yR2Mm5qdi@aIE5)aOb4aVZBo4sY)S_Cdyxtxi_)m6G>mQjdxbLLTL`ZSA7nl>>we#$F%oTw+;Q#X1#e&vn z&Iai%cb){=d|l)lac~MzJG3nO2atBYJFtprHkaC6*B5E0H_Ur#0|Npo```zjHQsV$02Ms!Ocj;PUksz(pa)qxgP=xG-a@>Vfrx^D6aYi1{$l z%0A7RnzoWyEb+~;G^E?P$Mb6~}S&^a}3SVcU&vUh>2MM`6ghh2r=JYmjgr+hN* z>+RI&#|YBFQJ)z5DMylEXie0&C-X6N5!Dem1aqIS4_nVl{%(z7Cci8G#kDn-=J7dA z!AIT@%B^H-jh-!F++27_DmNLH!<+3M4o{DvD=(wAo^_jXY?=}p=hJnu26nW4l^vN0 zoG=nR=0AM;PZ72emjpv?XJufwkH?#Q2;Y~cSKyZQ_lNzmen+V&tv#U8Ndcta!k8p) z`r&|}8`-R2du91rwHAix`Y5d*G-K+=`;@lTD%Ybn+}1qhf1Y~HKPFTso|6HmIsbwY zGD8HVawNC^um_9hb?RMLQrE|Ekazf1#zJ79HNWPDf>FF3WLI)9*`sAxtQLG$(%>mM{CLexeH#<`wQ^5gv zf3&JUPp0#Whk=Ezab)+?mAu!M?dP~~ooKa?Qq1)Qo;H=62+NtCJunJwf&GqU3q8(c z^y?mGoZG_s!6~OAk=FSz3mA1Hor1jPbCyr`A5eLA9Z6O4C91Brxx041+8@qyGQA#7 z8R@seE~6nHE=W)ZV&+u4(z1mTIO@H?2$SM3wR$}1j6}?ZJM}@)1&BhK0zej96^#Uq zPm~wtH`5pq#`cf?16^46$0{F#tEZ(-yTw7wux@1Tx$X1Mb-f8W@=0S0w)d*X@`o!h zmtUHaDhcvsv-D3chhOJ^0)k~x5A=55ym)gz2~0D70j6~PP@j0PMX*SV#idFM3UuB^ zO4Enovf6k7yCiwr))YxYg)~9OuT#@s;#2aGdipAg>G;UnD-^Ma@pP?Y7&}cb?hj-s^Ucg+1H88!w^VTQ_UJ-+NXs#rHNdvou+_zYi0zUHcHtr9B$d$V-L0-tz8 z;sD-Yw-=|tnZN9kCk~O_oXxiEHTqVTd z%dOTAIyDB{pK)~3RXRc4n6C5@mBJ4ZsQp&HrYkhhYt3h=TNmyyC7AwbpPBjr2Lt=b z8`oo|3*&bQfM(Nvn)J#WU4!)&{;qfD_vr|7Y+JP%F2MbV&Kpj4A>xh37g77-ja{~W z@O=&=)OgEnO(jmhtrFdMJ-r01B=|0{jZ{flx09`W&qwN}&AZ{kbUfeE55G<}smA8K z{4mS;WLtMAV|1ZWMwWiR&o-g2`t%36jdU~Qi+Ik|CyhUxC!uwjJzQLud&YI{wV#UA z(+z`6&3n8n>C~O`7qJgsAP#rPHYf`pb#d(F>5%IKLd*-3r_mcurd% zQIUto@kuq2NVslIPen8f3GL^e>#ocb}!HS9qNKQw0T-YgVkw zok|iu;(vjYZ+}MnA(ZDIQ3_0qRg}i8S6ceoJg(U0Z1E+?xxwXd#Uw9aS&U$cG@5V0#4SEKEjCVZ5Q&1c1CN}i2u z(eI$lAl4P>oJ&~M<#f@>CRZ*`KD{G>XoRvGD|(`|TYgh|@ra={nw&x~UFbBlA0gMb zXM{Vo*88EqaS5HXWu9d38r!;*(980#jE3_Eyxzx{k%NyK%l84oyV$h%J&hTzMr^>m zTE30WD`#V?K6+U{&ve1=gNE5*)*Oia{&)OOqrgE0kyJ;p#3PZN S?R#i#=0~lMyf}RB#(x2x0N*A6 literal 82251 zcmeFZXIN8d7d31dyMP^~>Zl+BA}Sq%Wh`{*gc6#72q7Z9B&aATRYek{1(hPbcSvv$ zA_N2}2{kAI0tqdF5E4ju4>->AJg;+ozu)V-=7-0SoRjQx@4eSvYwtUGep~-0-)^DZ zn>KCY)4KJC(WXt?8a8d(dUWS@;F)RuKN5leHhUS}yt=92i^vr4$FBRgz+Rg+2_9Jg zx7h{V1zd7@lhz+sO#C-b5AUjzxoN{%1Nn8-D+O(_*Xv`2>pj}pf6HR*BW~yKXG&U5 zs=qt%V)*HA2@yZzL=has=^UZ*^Hl2Wwmv)F;-m#;33k&=d(>R6Z_m(!Cy zl{)pN`(n4(mn=wOVor8J0jIZCO~s44lwx)0)NlJPZ`%Ct;omj*HxB+ygMXpmUnKk& z3I9dHf06KCB>Wc%{~tlZG^;IsrT4`g+21^#mgl^$cDtmR9-2G-z@7X+Z&VEZDS=!N zyYSDLm3HHs69IFVEvA5T^|1Y>D{M5ux)(BeIJR%Rui0kk)GS>qN@bR#Ebueo6CxVI zvfKVV)qTVQcF?0BVew)4u*p~bgIjhUf4B)aq;)(^l9{Wh3;82v8!hC;c)fhAzv}S* zgqnx5=EIfJTEP{N=VFxKH_inP#s@5vG6E8uYqntRx@Yn}D>YZ>-whP6vLz z>HMLSos@9Ns4JdX{YG7{#5yJd zwJrF{fyEoS;!ySxq(ND5;<&jLB63`_Xmu;TTi>Pc3Ca?IE0av^49kMC7>8F> zZSl~u8NRSgy}fKeG0J=}Ff%~zyx`w!wC<^&F38_qw3&34Vl9w4x0z`Ea#FlAc>a+2 zJ3UtKYqP;Xv+iQW0O`8Cz)bVqyOeEVR?xGcWqh zjl*Ec@^sx)IvBY!n(1?QLmHjkRC^hrG9waVyvOnA%GEc^n4GG31u+Nq89e7&LXs8e zaNke>!_;Ib;qxaA>7JF(kw z#XK*#ew-$2e!3}>*SaDj4PuA%KjjTkDW~Er3HJanZMQPx&$BzHy+$<%?G z)-((Cy4>bz^*$$5bXB{B$tKO&m&aO}xvR<21VP*jv&B>?_6k_PyNVX@hHn%*L_?93 zED7oZZJ1WZIXtIH${-xvSSyoTnCi0ukB96D9hiF5ZNAG*^ zE|Mwvg$NQ&9UfM7iNJ&iw-3yVyuOsm5pxMh8K-~=E(gk8OIJoMh(U1YfU@z-sr$6$ zLZnXtqZiptU-U8;CO5v~-Dcq$Rh!5hc0u(Qu}&=X8d^JP1y(sti$@bC(XxwdiU$(y z28Pc%1z6l8FGNfv{Ndi{K%`5{&;%xMT2RE7S1eEDU=5Co6^2iUA_r>18YjLh|5rks z+>`9#GTF#Db`lLf)DcVJX}4Fe4G%DNUMgN%Yb`;rt%|0n^vwcto!ZygDo&AZHE)JB=7RvnRBTX)&Hj&_M6rD(sS}J#vw;xjt#`Qt ztB540_7!0y*UcoREHHKKeY6uaMibB2kJONA7PKTxEGWr{J5`@8pPF>L-4=2WEW9`3 zyB%KJaxfcN{_(BtJB7+FFPRk$Z(3(p&{sJkx>5$EIi*04SWoctrysexKcubA?EoW+ z$Ax9F0}nV}moifPeXvIcn9V^ICpm7O*r0wjOMX42=@|O7VAbH}*{=%ZkwP z!G7ul;d?lc5{tTczsfp0@YE4xNP9_Y7onqA5G?juqIfeY^Q{9GJw9his0wjq^N>U< zvo4p&a4TM(6Zb9mDY6&Vx7}YDZSJ1<{8Of9Qo3|rH!;z>jWb#LMO#yP@ji#YSrNK;HYVp1HE>mMpN*sjSn;#=0<<#E z_M2Qqo^3Pj70>Ppc5H1Vksqqc5dt}QO`$%i^|ob~yKc1>-aN3_87OXFeYRy8hb%0f z7M?QY-o3VN*In~(Q;Nw5evoIBJ+t~Uh4=BH7uThEM}Eg?~Bl{0IagJ=kxjIVHQ1=LIutc-DA|1*n5j zw5Yv~$S6v0pECzz0IUFJdb)7GmEMs5&W$G-v$As5%j)RtXw&8^&Im{p6U+ir7w> z(z>KEywtoM-g_PUZR8FZ`zb2vhG&m$iWu@-q-f$|Z~gn8JLLDoIP-N9^eA9|XGfNV zmZs7GPlWC)>{UOWQ)nCDt-gdaQ`8G|uonK#QPF7c344oy+(66nK9yh`-beQkDDhgK z2+f=U5l6sJ@7vTS#b6iofy8dHUcB;?=w$w0WPi5we2e<7p+6t@ibt*9Z%Y#>UMQQ& zk^Ed~l{(?4Tw|SWlNvCHggqBp(3_bJ(r3yZF3NI zV4{5wQ1ffpiFw@}Oblo1t_|AQFmR}I+r1X4(mcf*IYe{NmS6D7$bC2AUfmhn^lDzQL{-uZ=lBP);xpkqxrAH(2a}f&1(@J5XOZd*bhV=-2|! zynB(W%ol*R$)W>pvFDJy=^(nK4SSprl=w`0u zl(L;A{|0sOHn9tdaw8w5Pvu^E9UMifovo)nzk8Y+U8z1L@CKWzr08F$dD*<{;26|H zBK(Tv@;iU58~bvkR~a+F=|FLLRojhg?Nl#9T2P(VH>gdcgxA*{e`k0EKy!`lQ**UE z1mBC6ZRW|)-iJg(goksm-Hd2t*2_%!%2Xt0FhTrI29MriTd}$mFK9uSs6yB$J9lZ$ zGF9+_kMF)tFE`0>RW@ta5niwRm;JUaDuuu1ryV|7&ru0{T!WuVIBMOi`DZq|9P?7M zRcKWtMbp6b>tma{D&HzVg}T32AW^FKtH6jaLf)|gdsW5U|9Odrbl=%NPTlvOTbTG^ z?eoA|A0tR6cTNwP74Iz1a&h;JtqgX-%N4Jh;Khqqy^ta90hU?@=~FFbIktT_qwSBx zyl=i0vXPEcd#pvXL?|P+RbX`@E&;LF?p)_Mz0@8t%K-40l;sjsf% z+nDv#=#<^OHmJuEJ}JXJ`9j+i!26pubZSy4$I3mRt78C)gvE;=Wk7_#1v0oL5eQC_ zMUwPDZD%@mCacg~t**&#;D^Q5&NH`dmVa2R{tp)O&G&QsTzxgTTqFDr8Cpwxj`w2) zkCN4M_?28$-6AN`D+|xTF5NJ&mriw+2>E@@UOKrt4PbQmQ2!%dyfA zk2;S5U0bair^(Ppx+Wi}4nOWDy7fRGN-r5A2&5s-$Y6;gU+ycO-s_Vxv;|}0tJD&+ zE~bZc`Hwg)Z@xnI{=Ock!MCRodHHUWWW&FO?L|Smt-%whw2PbTKL1{g8V@P6X6Ynr z?qBLfzQcfGyl#im2U2T4zAm@nb;nFf&GUg{4GD26s*5v&dEu$&f~v>k)wKwxac?6t zx30%%L(evn+r@heD@`ZcR{x*{iR0Jo!67e5y^E1Y5-_7<;xe}2{bOq|Cqynp`!##p>2#?KgefXm`DIT9D@Uxj7 zw9=9t1cJsaneXgiO<3r}x1;Ne{1?VcADPE;{F@)XBn%cC%Z3ax-i?b}6679)zDf-w z!N3L&X@V^|>5|DHJ8;rR%?fMglXwM`7S?~JI1hVO{fwT&tTzbSH&?P6j4#R78QAv% z#kUWCS>m#1W7Xj==`On zQx3V2)j8bVZawwqwN(kKbCp8`7bh%fgPpN7cE|u3p=`JNM@{&#wnY~)~x4dtsV2gs&$0ep@Xz2@!cVZB1wKj&VjOIrMP1em?wA-080v4;- z%!?%4B*Xg|Ey%;T6q&V{@fkpYBI{kfX1(3p**s;QV7@w@@m^Nk5$x6@XTN3##_kJg zdoV1)K0~A1-b>^iaO)16+uxg6Bm#nh^#zLZG+$J`2V|TQAozJN$Muj4-bjVAaljAy z&lgHIr`ql(t)s^90gIyh5PL9sf7u(>cDzAlUi*EnI?ztSR^*`-1`>ku$7|4$z8`eg z-J!*)dq=0eip|0{RWvdLWU!wYQpnhf;F598l2tg~67MCeq{Zo9+l=Rc*(dNFWx)pc z%8EQqYl4eZQwCeVL|rDuRrC^}35mYTo<+i3*n%={Vo(Iy89Zf#?;hGjOpee(D1i~t zPH<$^)A1c(B%1faF>74X3h~M*->GwZm8i$~LPRq_W`ruWgaGkGPIUs`Ko`|@b+iI< z>u}ttS|dXd?=!Vit*EOb*~hVa1HylVe?!DJ{mT7|9~C<|rh7rpEwTE@*7 z6m1xCpL-gwhZ$|GiFf^YucJ7)dfZ#Rm#NfziTE9vlsQr!6W}gHBO{F)3;S0}Wr-OT zd~h-N5q6zZKnAUM#Y>h3xgA*MBr0C9xd5YWP!arm+*`L$V5QC>>*=NJzTnjwBl=We zRN??Hxpzqe({yC`^9qp#<&L6W;W7dQH;Ni~jpaWaTJ10??6)VhpC;0g_ zprrN2g$$TP8xd6 zxiqiWqE+beK`YGX65f3>4w;nhs;u}t^arIIKGHezrADVMVsTL3EwsqZwRD^JT!LH$ z<0_sAhG^B~r>tH?3Y4#I1L7?=eD@8#@cvx`$^_>Uu~W+tPSL#?>4olJDdJ5ZXt~(K zEDgN0o;15|)XU(kqBwTuff2fNpbDqd?*Ih-j@yen^z@8`0Xnb#A3Fa}-N=8R`$<^; z9TWZRssEIJ|Hj7utFe)Jyy*N>m4q!n?X&RO5=l8SCq+0MTk>?SESQ_P)q=YR^e3{a zpP_(+S#t3VFsegI<@)lvc|D-~~ z!v@Z1r-OVjLAB?GxkSGHqxa^N5)0EqRfiAcnte+%t-WyMr{Kl|YSp&+qwvo#N<00i zX>^ZbmsZnK_cXXMtL)O5M(-QnO3eGbMn;Oi@a$KWG2{ z^euD^?dWx*QB8XkLxx&qjgyhtzke64G<-e-!tcmjBrqUPVqb|F4>^PlCfp@c}s z^fOMZ=+C#i={_9)TX6ok3iy<$_4dww_3rgoav_&(AC=sd+e0^;KZ)>iD`tDJq${8S zSdCp6*WN<(wqGxA0~QF)K7D)>kZ^YKd+r`oKQ*{hbM#onE=1OjBT6Hykd<(Gr94J_ zsrt{9{2SR5ih=ITMa(AMSm56C0yk@HDoB4Y%z3BGsB}NL$}b>AfFde2c6spq!=_TZ z3+ox!9B;UhJ&zx1a7D#7#b!|V4E(kxI{2IB(w(3KtF}iv&Mt#L4L|y(cS1g*y%W8PgFw|iz$BW=reP#X>W`)uBeOGyFV-VtJO~ctK)7v>8`)>=u8?P z?6h1Q=P`WSvSGz83_andpun(q40B#iXP3+Tub;d#&jj#P{O$*!^+*J!&%9WNum|tq zqOD$)hiM+Xgwa(!*iq>5z`wcSPgiZ)61pCZ(~X83pIp3c`i%$l0R>EEOH@&5m4}+X zb_|;3X>#b1g6@cv#9RNl?&qV+!dsSD9a;+#Ma#vb51Z6m3U^yrD+erwye^3J^pTes zhS@~Bk$6_eNh5S^sApU@n4-JYL8_M+#ck z27UjeAtz*iqvjVAHf@0evB7%T2EK+VJmmYbEUbvugH0&OtWOz3z&z zDfE&J5jO1z+XCCN9YxFgv}C^2k(_<#<*&y)I~EKHwVSYN(3k z%qv`+0LURYfQ9+K5Vraxn5Tv9Ah%F*qhW4UFmH?Og0^tmko&8uugx`F=&LlCrLyY? zK4|Ow-<`Yvl!+*b2F=GRiRONRl(i4ZSOHaN6bCLcdXO6_4>>&r9 z?a05|Q}fy+liB$J<&No`g*s)Wzv6#Ia1_*32-EPcx-$k*5PrxMj(qc)l}@TL6o-Ud z3w+)c*x%i38S)JJ7<=EU*kdJ9P_A$OPsYB(zhvHJyWaqKyxBlCF^38PU46F1YnX0! zELCr%(EFtx<&CqoO_bIbo9NsYjkLaaDxp)&axHNri#i20bz#fLyAH2pLS2Vr3ok*$ zarpQcRC59WzFMcdlI}Bys>txs#B}+?MdZv?AwrIpEPB0l$Opit&zOTwgF1js%hs^W zKmNw9zu8IR`hEzL95?P_QAWut6rS{n|9e9Hr%HUy6D3(}&zFN+id!7Q=LwL}!RO7U zYBzT}D9XEjw29J5dTD>{g4DG%u_A4qsZ1x!IZ;ubDs_8Kh(hd|-nRB{(6?fRF}O+tuU^Az2rwhMQkwLk(O%O`&9AQ6-(0xz()1W zm&(EIHq|Pq9#hj=r2u&LiPS8F_V*_iNXe%s$@R-e!N)R-1!1Z8tU<13)O_4a0_yR! zsN9LGS$z|Z@jGxIex9*(6eZ-a7v!XTq9R_B{qd{zh#ld44yqA+-3PWase^EPDb@$N z-5k0z3;MR8SRu_ETy;N3D5Cb}xUeMR@3uA>$WM3foTCf}l!MrnCv6{j<^KV=L< zpwstCEOH8*<*o?^N$KQ1aV*xQVP3we$Y9oxsd3IhgOOVC5?UXno*oZ+N~J@qkD#yZ zFk$gSSY%ZLA5G0q`7A=3NKDg({Meu&3bQMd+LWaomwq|k#F8Hd+}tyHWKM1<%5|7x z&|wHaLL2eFV<^JBx^h!NX>ELzO+&x?p!LgrJZw1sh_6xgejRy>{X)BMwuxEBVvCum zK$5lA8+0KLnjSn9+y?#xAm> z2R*SjSN)&KexWcgk(5m*Q)7v4)`9t|AyapKps(pVN&c@2W;)2f`=J+M7~kLI?oY=_k;)PZ5$LD?`p-=V=S%(#m0 z`JP@W?;_OVZ?HW(;v9>yXe+f;w7D_IvU}}Zoow>x9w)A;NWG#`i63H$dM?XWQqJeG zZR3N-%AdoM`iXDVVQ!wiZlo&zk9t9Z`ZA>bHS>=z3vN3Sp%$(wP)px){jIKL$`&Hm z%J58_Syn@QNU18ZDd&{kTIbH6`?3g+0S>Sb7%wRB>2tYZdf*V4q4!=wSWD>RO2Dz3 zqP3F_02E2}`7H{C^#waB^?$N%(HNx!zWHJpP|B`@gdIE^v?NrIMg@exSQj)?`DQ!= zkJV&RoVnrkOwT$I)zpr0CuREv8Bzzn4qF-YM!X|zNx6RSaJN?}uaU%2hVYAaP#71I zi!R}B1btZgJW{GRzqT;Z<7c<2~cSqf}+}}rk z^RCaY+-7j)P*bK=#f4L$-nYJ41~IHc9{YTjddd|(w0@qMOwruHyFR)UpBFg-=de!g z)^RUk4}iXu(F=ULyOf^QnT(WyCapyy4NN)PIkzbMLW7R;M(}P7xv~`IYQw;^Hok!j*%hT)XF6QAI*TEZq!4*g%%P6TuMxcmv zO5;iZxHqJ{Afz!ra&+DWCadVqK{dr560q8m@l(A17aT0E({h&DN2Pibv+?=+MWV)l zt%&jFaKASM&IRn0poPd*~4tYLs75v7#G_(2EaiTDCKCOk zKqcQTul3OO(vyeDQ@N*OmQ(VZn)T)?^yZDv8f5*~Zj--y286tgJen$Z?YZN4))u`~ z``RQd&(p;GSZPdbQt;jxa(XE%wE97)J;|#KgWt z63VUPsMOpsN|45-fyghWXL0j2tz{mttLUItL;N5r*-LsrlQ~Y{B+&nIS@eji9I6(gDNINm$%yxy5z1m9S!8Wc|1H=w<&0dWRADD2)qGxw? z*U{j-GT9j#CD&F+$C4$Vhy*#S8mMAp<0>-X++d~-P^du7XEI`wzQi|wbIgFc+G^Tz zy8zA=D}V1zhihej*A0(k18AeeJUJlSze`yGX_Sy<&d0Q{@%-);AR^Q`n6ur5t8rN6 zi8~B`VZdjG1H-HfjI{yhq;GQoRFHGeNkjiD*nZT|qNVkiBf8BB8%Nms7+|s~!tj7> z%suIjSYd0-LPE^AV~i1!+^n)X1YYOvV_i*E>Jsq7F+lvy#V)O!vL=pUr2TSxkweBH zQc>5O7C^Z=ba1*fgt~@gK0ts?R_`s0`OM)slm!l~Iug(nj(I$3uIPHrfUE7L7pVM;J zG%!+1JwEDFFJiO2>hzfE>}pUdC#RsRW5I|r!R4yUb%SzNs(r}gxbj2g*^5pA*>ncG zTd{Y6-US?9#|lnV>@v-CWCsL<*jA#b6u5~2?iNio^L0gbL3!X5ZOMoWpZ275A9ZcT z?eW4fVHh_K-0g@v{!1J`M67rj>T2NU z8l8}*WE4s!3zI%ss%zFQ$TKsk91b^&-6M%aB%)Nyrc)VPAz}Rek>UVi^$xG3Gy8~h zNx{QBQ1XRu!J@W`Aez9b@1S|wY=+AARtwc!6keuxa!1{Cw6!}McHMI_PT-s(bhQeI z*I`HLZNJ9EdXtO7eC$?uW?Ha6GZuw8hC(9p3b2Z?J(}*n2f(+T@mWZh&jz(u0XXqY z?HFog&-X2-uLL%nm7OYjRd|JEaCc$B=eCalBo@a_U0As~eAP;a%SD{SpP)%OGMB>_ z1>DLlOv7qYDB~r?mX;kKAmcT{1uoz=)tiy@3o>nj16H>O#%N0q`Yq~bby!!UTzZ&M z10zK2IB8FlJ=V7fFkMoxLw0)4pT82kqV?VonkZescftuSSb+K-3o7d?qF;=p z9LBL-U;heL9H7r&>qE4ty{&yVzFcc8gBnng*_rv^RrW@0v(l5w}T%+6N00nRxv7+C(*R-wYkMqM>iMI%7tll{zz3JcCr*K7nyGCv4i zCYz7KK}YXn;&)$zw;ILw5ptt>ImPj(}Qm)=?4OC$aoK_DqF4XL>X|!Yc6MKVUQD2R8Rkym-}G zWcd2E{hE<*7&T20|29OX9?M^u@~67u2W>P?Cl-fyZCbAvq^DJkqg_=d9mW{@Y%(b` zaKyi*K8q&GX*TjxnhN zU(Q3U9WDNd`YM_;+B>gmu&%muvgE@MzMc3SdY2dQ_Mp_5AS;sj-1;5O(I`24nMcub zXObPEXXAm0y&X&GfBRrKCuq0hfN=a&wJ8~@8E~zb`|_q>5>VpnMvsa=KJas>@wd)5 z%i5qSHgLzA^xWANpr#G))dr4o_~Fyy_dj*vx?OXE7&YPsfZPP0H9vrg&9E?GjrnM% zHoqir{UHUXB$~dF1L~zcGw8476O*X+?H-dE3V0nYrw@bjaz6T#LipR22ne-K{b_Yx z@zg4uORfeLbYxuUnQ6jbc+uIZ_8z9rBzksGk&8fiuMDW>4T{yg?eiUIkMsfPGiG!W zGGF)K?XXmIZPm};0@%p!hI@yA83L*B-pGv<+0TQCdHFDML5mh2$A2%t`9h7ap|n25 z1LnFMi<{5~ZlpBEt$&rh?dwG0z-ZY0XKE1I?Mg=olKYB>AE+DFiOvSnPBjzgAcZ$? zmjfYN%ShjPI!~t<`$T`w=mm##{K1bHu9rzUQ0N3D%{uimzsjGEch$J0=&yx~V$KMR zSh%aP6C7414CD~$$|WCLJua|-Q?(|HOhRu_$ zo2qvY>B)4Ug~Dvy2UfLKR9`o~I%)D>yU%|Fie3CpAe~u)7e*$(u3U0^M9OM$D5n!| zFjUriW)t%uy$QFI=g@LMH?8CMOz$t3kY%Y$Z(f>yt4uKAWXzXn-3kyN$y?;pjm`kF z6h52DU0{b}P*|fnetXRRQCf=iS}P|=(WZ(wxQ3e;P1_-6o(B***5U9T+eo zRYmR|c5>BYHz#re0mP1$Oc9R3(N|p^nWEA@>j-~psGvEcuY}J|As(93Oh$-DTb}D< ztKM8sucN4l6u>$%AWnnp9^^-yjLvWdiDvQH1qE)`rECh8&ui$4``JXyE!S<#B>-P#S9D>;(wwjE!flY3KgIs zBC>GZ>rDJYGe>4!n0bsjRVmsO8C^ztAG=n8Dk+O7c#ji;y=S!CPeD~Yb@PyOIwp|G zvOz#Eg;i#Q9vBsApO`#q5T&MX+uTR1T2)Z1_Y-$hR{KU;v4)1EXs!oBvr?(Gs}E!gu-NQ;*NE@&MCB+q z$(7|3=Yr_VT?yUQ_1Fp1;W#4KKTuy487s~KWSdt~Ocp|@(v@+WHX$&xG>Yp^=q_L8 zTq?w+09f?*dBGxb-E@A$D$=1_W6n^dWc+<3Opd?n$A04f>9Dm`ZonV6tmS>4=qWi< zfh5)9y-6UjMzRCciBY4Cx6fmIn~eX_)E`_I2E%GOD;nwkjrAE38PgS0%yM5%TQ~rS zKzEJpui7yf997na6FyR?2^CHKR1tSg1TxU@9d5)g+~H`PowrbLV$*^zn_WnU=1|BP zz@Y`VL=S7ibAlc>7Obgm)mSv2(VDL%)havV!>0kEOj)e^m3Olb0O)}!AR3ho%o`ET zC85S+lICJ$eQb01`d#-p$}+{O2~F!n(XRHW`7z_D4%jAahKZR|O4br17^~@nJq?&^ zze9P+RKvZm`9gk>W;B@`R--rTB=_Rz-7dKq@x+!&WyO^%elB<9s)<?=oJrx4yT4TN}u`gk81(p#uG_LgvP=DIO(lk9=Phl#q6B z43g9_LmzWe9>(PfN${P`H)obttkZ(5YnjZ%?_jKLZXyk9rzs==Z@to#sit03#hwVs zbsGDhAlbklZ;0JB%E?m=LYqg%ySB$rF>mAiSdKp2QZj~)6zE=t5392`_1B$1z>IAR zadfjo<|6N5G2rGgH=0YtAZai@V9D>!X>tVar!|Gg>dh!PcOPPyL zaU2v6eLT4hUs#T!z9KLYtLZJjPP@)$t*f1>r&gHnGf8(Xbuoq$xVj$5m(6%g7I~(4 z(V0)dj6wH)03m)OAyIKrcnZ}>*R=S9o)-Pf=T;>?cwqdAalU{k0OlmP-Lz3kw3Rdc z4ov6E1C4(5Yb#1%16AiZUY_P*E3iiG!~VgSg;<_=05{3+aIGB2n%+H(Eh_0#7XF1u zI7zV*@Q!?W9C3dyFP&5E|twAkY%OOffBlh#OA zD|ILEOQv`rJclFM9dBIW0j;7H0cx~ES)bGX^I=&br#q=%S8%D4v+ARirrWz z1rZs2aefqyUsP9H?0WipX>vr&k_(ev)h{^-sNZv+9JckZvb zR{9(;*Xpeb-eLG?%x4=QoMR<5&9=|%2+U9lI;L1UqsRUfkQW}*t4~Kj?!7J}3Hwa- z#s<7$i7VPx7kI-1Y(lf3I^A;IoAKLUh_IaH!kft4x;A3mUeTQOmgrg$yVWOyhNlUs zLv=#u9bGrIc*2$*Sg5a;B#oBNw^ZQl6@jD<;$cPXh_7QYCCOIm+z9vl;3_G_peg7V zWtVGd3eWgEU0Q&qBSP$?euA1 z@d51KW`i>~sis#&eA~!|Zk8QuwO`2HMWhKzU=j(ngPaDcK;>y3Xae>f60zJ&hYoWr zWT=W_GPsJ7zs>k{pzI|J#M2N2Q{{PZ)PRb4XK^n-j ztI6Jw!WDV6<{Vm_7(g;-+H)7ks*-x_W&2^@1GG7*Dbmu~jt{3(moP;VQ%)Z?Dv9qA=|Gff$rod@$9z zc(2AC5#2Lt8~g4;9Wk<(yNcn}H;&U8BBp0wb&QjKxfDDM^A_Wzkhpjum)E@hsd4eg@vH>C&m6rL5nmo| zb~LlY#%OwE;lKeeRU4+;fO@@R{%eS*1| z#i!QW-ClQOf6Y;ILB|6Dg5;c9y;Y=CtXnCOlyIHR%%C#F=r99aW@Doxaz;L0FD@5Fb0q@_bbJ7^h}JEC!%+90kU+`+bZ;l25h zTIj;c_a>+MJ#Erk4j!GI+*yx-bG?Q4t2Om&sI7jDaG*6pxA}d z^9lY}w+OD~@VOi89+_ZXFaG{{m)@U67x(wU4GmY9n-V!$7vG?@{?cf6RDcc`q}1i_ zLd|@NKL6p%!tnK6XISIg{WoZH0~;K0O!cUFO?DDtVb4tE3+;?m5F;Z1EK;OT%`Wjy0Ov7F)JvlgiOZ_8}n(;-8W*;?nTCE-i zLK6Yb=YKVc&KzM*ahtvfSlbMhu^#P-J%dQLHq68<#dsU z4s3$&rWf4BhfhKUwk+as(7ndT(7ld*%Ee*Yb>j6q6X(VMN)=smpk+4| zXkf26ViF~+2VX%{?8=C~3IAXoaUHy5DLMb+?j~#ZnU6#}dIlur6hk}?&@kFDZpU01 zw!d>R#v`4XUNW-O8)boKuIcy-yK%2iLr9eXO+U#ms=rhn%!4@k^T|SK^Vu z`EN}^%Y%|g6H;xkXi$il98D>Cm+hJjx*UN0s!Zj5irA0@+ut|74e>v^se!sK=RFGD zYHCq(zpET!(U6L}$rFD597W9z(#m$5rnk;|Ekkh0Q@Bha^9wK+2czRQQPR;;*|fl^ zd|Rcb=**(GZhYfm_XBUw(CblwwRwf&&z$}w=_(5PvBRIKavpy8!)ql#$Uytl$cL9XI(0q_J zRVFoQfDbFbBOI_^eM2S-V5%ZUd)VHS(*q zExr5rn~mlgqYvxUiz68ey(1Kc*8+<~O53|8YN4w~ym&7J&`uug*nV&4jvIR3$E8P`SmLXk9yaqE_D#&k-prBkZ%}!jA1*nD?Iwn4VfD9r@BhAmb93 z=JuQsP~LMvGcUkmZLDA|71lq&(j2)x-S1ZdMT98S3e{czWdNjbYTXpuX?x*NcC*!p z-!Y&K6i`w*wViyvaziwa^<`HDJ6b%-@ToV56JIdGbp;L(sR-ME?4Wb*TWF+iPqlB* z@9?`es&dfC@a=+PIyP_KQ@EFEDoxwRaCIVGBgC1ELWR{z7fK0lsbFn(4)B$gZ(rO< z*1!Y{l~W-04--G%^~#Q=_Vy;cqM$rIE4#h28tZl)QmkLE(dU60r^v*dueEK1AV^U;9y{u$z}%ti4=us zKTT!sOjy~rICh#f7}I()1U3WIj*9!KfI%5S z3czRA=R$5hAFF~-67j2MyMA}}7jx_Xy6%DlV(eJ;41iJR{ z02pGg-edLE8(4B`>UCqY#hN2CNI{dN5#j;uIoJ;P7sK^*Z+e~CTgR(MEh%DIvl_#S zSWV=)B-LNI;d&Esh_sbAmUrZ~DN8XPqb67@0b5CJWAdr;shKeXd>)t%4`v=ib@XQ> zP#v@eHD;wO*-Qb}v-aEzPtt4ZrDAb`)gRqb{!{@*c|a@9ZU*5oIXY9^Q+yv0Jm(cp zcERC;a_vLJt(;7$Nh7KONu2PSlsuzsIZ!z`CLDFvsj;hC6+3-2cE!4!Y^7;?t(Og_ z-{Eqt<4v<+2-+BwDo*u%01YNcw!QtLjq#A{eciJ_2BBETD#SEH=3=MSuxkkOV%VJA z-~=nYv8V2r9`2?s&l_se^?k&)1vlPu`XHGBI}o2C+o{xG5>~@x8F5#awPiOLePJcKf?ZEB7u1&}*?Jnf{^u90fUVBXTl~sBVvB=smCi0w(;9|lBy1W( z)s70EIC0(!0Dc$5u4cZ=wn_FX?pvC>lzQs&1}xUM^7T(kD}iv)T9l0t?lF`eR{Nf+ zt^G6+h_>;J+4qK}&5J?uQR0%FaoDMDtPhzp1ng&F84PySha5c8}FT-y;)vPhzRV_{1TrZ}y>=BUV<%i?`XLDeb)V#mioG3C@bRk$j zslMT$%x^Z0gI_?0e!rZyJ`DGpON=`5684TQVOXPrBG6*oDympVB=l+Tub0&MZGeIi z?T$4@CAaH=zpzekWBp+rtV>}V{mnCbVOWxGhbN8MSLz7G&apnI**H?hWku7Zp2`<5c54K2<;O+rJ z@C1TuaCdhC!8HVTcXxO95E$Iu-3Ixa`|a-C-TN<3Kb^xcneKB=$y;w#*H~1!K7Srg zTwEcYOTsk&;}?U*t?Hxg9KEBaPJpv0FPdqC>>^Y9qdW6R{&g3C@RR%mf%ld@_L7Yc zh)tCeUMvG}0rzu~sO001Y(Zgyxz4Di5GzR+e2%z-u(ULL)G7PqdzI*G0n4~GfCH#J z(A5Ab5NOvQoqE!7N{Q$_3*dS9Ex2RN2~)mG9a@9Jc7AB-8bKfIKchbN+jC1 zCZts;r^X1m^V=ky!a3eDa45@Bil{NU7buO(N+mJ8iarc0n|ii@nyOEaMF%9!_f48T zYfH;&&jp0#C|ZBRIX*Fn=2eqVQ)iAt*A?!G&z;+f$)QpK(-%^JT2{Vmkffqz+EvIn z5BBooh=~F#Pi3dDSMHiqmR=-0$(t-Sw?@*FGm&W;j-l%rX zo@Y^*r{LGv;@ohM&ZB--;la1KUI=+86DV{Wgg{)h3e&C;w{IGNq!(9xvqXSHX&m6+ zczX5$$qRpz^ZWnQDT7opvn{Eq05duqI4-|^IQSNQ)y6IP@vlnS?v9lF<%?h~MaHtR zjRwBn?H6%fc|1*)y;sN?q)d%3e!ZF6mMI7Q@@&0Y7IX^+@0d(7dZCx=`W)p^3f=WW7~5y$J! z1?Ru=q_rXpu4)frm73*dW7*0dXSJR@z!w~C=m(yUodfVoDq7^UfAumMyd+u2 z0##^BvDZ^>X?;W0C53Gfo0E`|?bXYkB=qm-k;KNbvB$yT_HSkx zK|QycNd3CIiJ$tM5#W8c07d7l+f$NzDz&uua64HU{1KudsdZP;_!z@$)I!$lT9VWU zta;*dYJC+bbQwf0L*7oxCUbT>7~;Pdg$sc zo*F_`D}y+@aGT$2VM~>{c!h0S>}@*4;0%BV?ZEWmXPixOvk>6sd4{y_hM^{IcyP$asF+vN%-keL%^-<-k}~x@zG%ysuR#qDJ!CYiW7x(st$+h&;Zm?>eUz zj>^oxKTJ#IrS}X=xdNyy$44K+gMnkt1NzraEw-!K9$vBHU4LU%jb+-Of?dpDIPi?k zdj zsm?K-ofus%`;Ey32^j%8dOVzC4C?P3ScMl`#~S53|3Ub;+3B{l=1}IAa!4!lHVgyo zKytHnTIVrUq~)-fok>8ETO@E(0>6A4Q|7*@7#%FJ9!>z3(smupIfmSwqPt!eX?yH; zuiYtC6x8QfsC|EBEV8n)h8xl%=iGCujsscN01uK0p!hy=0bF0O8d%p#CB zBD#9^9&Sjj?K{Wr=DGclT7<64b03qHxpGgRO(PY7nMxv0o!bvXay7;F;hT-jZH9dt z8xmj{Zkf2*B=v06iq@YEbp9IC*=~C|vBxI2ZL!3vuB-(ROB4T0m7q^N;TphVixR*) z!19m*DoOpetJ`;`!eYMmkYh-cvt1l;Zfk1}gdFDM=o>*6-Vqp+iu>R)!Oqd79k`o( z8`UQ%PRj*nvlzYP`JEr%jQr7q?$STJ)9HDBMD|XMlZ&I0J4nha8+a|@Rx=Hq!&^e8WDkDK^nZNh%q zXz5#xR`Lx!QT=qzrvR|xLqtIJ-AhOFrDjzxo+3esL;1!fjt=x1Z_k;G^wQe+3*mKM z;$VpkjmGkFplur_^f_btanW^|@Pz5-Wq>;Yx0o1JJT-DMCO2O3+T{UyD2ZNh{%kjP*`$GX*QerIWjcC%ZsoCYlTFjiO3MOh7kUOY z)Mj;iP4D*eeJMvf$rl%(Cr0@G{bj}Rc}cNU-mH&9Y>kVc=}%YeXwN#2!c8wgS4+eR z$4P4Fv#RLPrIxGFwt?o0dv$a?`%XL5s!N8q%(0j7Xv^ivxr}|BoFO@nOhQRJUGju> z=t+7%){kpPEf$;T7>xpNKR6WFH)FN8ni=xMFBrWf%xDwpU zF__tTsWkTU+<)IJ9I+BqUT>job;;$d{WxlSRCqGqcwE@4Ws-Eo3NnA5YOy_Wa)eM3 z;FjwKXGb~?g0=|hD@QH7)uqAn;IJLUlealV;1cybLCRyb#{RWNaqw4i5UfAaOxNP- zH7es}Qvg_`0o;1EF~S=HquiFoWpm8xA1)66m$)?+)^o8?P|a0cb#iMLoqjp@%=pv7 z#9YlM_wdbgV9Uy?(bVY$nRkul?ET7EUD9J5+Sd1iG*a?VEevvm3Ag56JpI_*|7GL1 zv45MIV&~X(UTw3ebXSdP|IL6OO5l*Kq+!1Pn95-Cy}2iPik@&dWVt(gy00D~loAL@ zRm|<2^ySQhJ`MFEp9h6b7-BN3-ajp7z3uR-;%aXR3UWWI_==vWduRe*SqG zYN=)eI3?5%@0ud#+R1sh-ET6s&9o+NibJlC1v6MxV804s`JDWybGVS-HfD970I@VY zGF+uB3qAe?arAfyqMnkgDwa9t+ugOwv1xsR9JMHfoa=FyyhE>B96MX5-?vH!yUb@yPttFL zs^f*P7t_ykZOlKKTRdcc%0}Q>_ z9HCwTsx?1lDRu9&Dy2d{KIT)0&v+GGf+x2kD(|ZR;{naGWT1+!epJm@z&6w%t~!|x zvHW2}HR1HpyfI+z#!45{hLs9h@0lxpJ@1~PpOniv^WJWoqn#ml|In>uzm>#-s{hLG z(rfPeyPySCGOvutBgsRXL_VGO;B$5-=(9Ai+8w7{-+MAT zZ7tc2-HzHF`E9fqwIY`p6*#`EjCn~*zTydeMQ8mW9|CT97&l0M%Fq7MggR`422hoD-&boskzU_SVx5hrfeO8K(UAb+XV9gb(tYhd8$p@3dR z`~^3e0$Z1K+6W$5dX@jU7uNoDP+>B>(5=wpMxjO-R#fP}2c}4Ru;o_fVPb(ur=JmA zeauIs=(MGWYT5b?XdoM7U;$T?w07 zj{IfCCL{4zmr3^pL6iv|eR_PBmXiZxeo1`=GxCAnO5Sus>DcS`%KP%LlfSmnR5jTNDUWE|Y3H#FhEyb_k{4&8IOHdr`j1zt#G1|Gqs{ z@yljkK>z4|ulVjgcIso8kI-fCxP&27VW2lgd<6Z&0Qf5R+a~d7-4p5XuT*^VA@?}F z@%RAVh2oLMNNS6T%DKRn>4$h;T6d<-Q0TI0vPL*cKHeMUj`VBuDd3td~R8v!Dr!@Br#+4hH4h6_8rrWybFUScIc{ek;!*dT&ECL9I=CR5e1wLZFq2L1x>e(&RUn(wG^>|nRj zPf+NAs~ZwK{S`?+d%sfXG9jOFSwj zCCUxOdnp!3_|wk9tZ#H0qEqEH5ewW1yfX zxkuN_+Nx`0IInQ9@bf(SNQsV-Yro*)y`aEmG5&?GBH z2Z03fhJ@?PH9}~1`btemGpdoVRd2RW^^f+x4mk78%S8r@&~u;>3T6LmMb%L&byUmT z3*BGk^g4Yk_T(%u?sWQXY#SLWL)#r#=mrqy;xIO@V}zxoj4`k-rm3eo9-x4Cay@Go zM&vm7ro(S3NMRNXurMR6>0crpHjchX<8j!NRAL2hY;RvDvKS1WIfwGca;zHAwIl0t zb7E{`zkc1y?0J9V)MUSVrhM1LgW*w#juDt3bW1dR{0$izM^XHX=RW#Kyzri=9HueO zP4?R(S#db%mON)$1f)g`Xk@RxIX;w=68O`e%Spn5e38`O%I`2LXDY3$u79H^0eeLK zjQ*gBWzRjveLwKxX@Z^kO{6|neZUK(as7)Irn}@6QkuGSZ4vJcz2}3EdRuAr`((wg zZk&1Za9p8#)DF;=tnDdn2Vc}-XDA8Wn@p~Gf#KAR3B{yjuZhLtO7njby_bbQxG?J6 zz4?`jz?S6wprd%o4K=)Q{E?9pEjnhmRQ=^>7Zq+t;d%b2tj4COgr3ieiUxGhXU?R# z{Hy9>unQ78lY7%X!7rtt1(h!4c*#Btt;1o_~KY1rDdYR82=8hM@?(B+bA+&w=Fy~Dl6a41?%!%5g5YuX-xV)Bfr3t zE?{R7J6I9!oq^)K*1UkG<6=L!}o7<7l$Z)%VAz!eOR% zVD;ggrIftvTs84=7t{o7rN}S~Hmuu+&vIWEHH)Q;?66?(9?PlQ8&!-6O>BU?l3cGi zozQ=&1c@()U2f;ZAh;gEOjf3MMa&c&9XUAJ_jqPaU z=2X}7t>GSlnf+2qM!qYR&E#hr)1KhLOvQSQ1gdCCh!7sMiTKM>8D8BH8vYX&6rw{+ zD2!Ks9M*(TtWFH@H|A{6s(YAH+%B(Jy1=i0r1)D3rIA7i>9NFUBkbD4CY5wcke48Ak8#-#{ZmNsSG>LV{=h0^@`&YzpHD>mnVlKQg_y8a=t zFAs&;v4+0ku6REG9Q-V*ckbZj=YAo%4553)%SXjCTNG&0d#V#%^RTSX_d1X^!iZJr&ZAhfj$XeT zgBz>3Iuh=q;B|zpwDF2aq(xWwg2xhD`)+Ae)vr_xKI^V$w{^ena0@*01v|oq>oy@! z+}%_GHj^sy_5V!`lz8*v{cZ~uwQTZXu1}(=;7xv0G@YpF2#r+BT9|3H%8B7`hd7fIsQq1CmE19YgY2-d)Miy};_ zzv|XyToDZKX{gIKog8m=$>QBwvYy}YdTp|bAJ(ozV*0AG8VZU^O$?SCbEh}zVLW_C z;er{Efsh;u;p33IjF=tG3;f`mndKu0g}P21h(XjUOATA90-FYso-!U6_@n`2*z8XS zn2=I*&LcClEa%?iyfINe?g9&3%yA@ZTvf&|q)cw}4cBBjJMhyO7@`Pm~+h!}XN#q>v&^2z5 zpa<33fP_U;q&Ld%_olD)t|?Y-$l+!5Y>dgr4~%C_6~vBHipsN^vdXwhSIeVB+R>@0 zkLI49t&7A;7=LKaANTc#F8TbRCWYY>d+U=9b<~Kn;-5AnU^!J7v-x1F_~z}~x76v} zv$M0Y+as6XVphBYHrp&HZ~CJsG~?uZFJIuK|HVZ91!V?$A0za?xL*9>Px7iapH3ki zHp?R*W*U{m7Q>SiacM||=eRTZN#tu&SOc%p*ZD5tRI0K&ED?Vc6B0WSo=^Wpd;ib~ zcOfCTdUPu0L^8l>dHN14yrHJf(`38EQq9ZET*F7X(?dc+(s1H$JsYHx!wA3hK2&;hZnrym(Z&YkawT1wNMA+H;9Re zT`@&jsaSaEe4i0Ls4XrohGV_W1^#IcjL~rRj(>Wma$I9FxrM`zH~9@KJMtg5{^zfl z?*JqJj;SdNOUm~v^Bxb(!p^HW{>xifr^q>$;2JZq7JL({ip-+h5XF&4pXuF5wg`SL zpiJs6Oj~Apec_AXmI+#xKX&hbb~O+kz&xj`d(%pBuj1)Yt(Ck4#Y!!d305d6DT`J( zLKIgh7I`Y7*48@W!JSt~O&v^@3-!_DQLL0|5`Mg zE_O+{xw1DK-8(xwGh>PxBTMP8m3WCGvK>t==lKfj{IDSTtI>3d)@|Ilu2i$BmDz9@PuY{O zosbPD=ig)p_gU&Y>AA7>K7E?tQSxCtT55qjqOtE9vlf5!OuW%B^Oj3sF?b2nO2NzP z=9dK{r=wR~tL7Rmw~P=uTq4p?=l* zb_OkPpIyNM7y>o=g)l*X8lf=RA1Law$zFt)y%b?DY zFydMKS#Wm5<~^3Wjg8H1cMuk1_7Mu`9y*U!g)jKfIxL~7;2(Da}2Jt zT6P*heKhOyE+G3?W%z&ztd0b9+J>cO5`1z1o6>jP4bbprNc3?2#L?3=x+&+_t>!PPXrc*|+3VP42Z9mm5XBa6LZ=jrkP)agHi zby7?y?F7dtxPk|D%yymBvDDJr+gtuxzO_`R_=_jgslqABqaYgh-^!a&VVW_sZWve1 zoZT1;v{K(PGA{dmqBon&9f2=)Bw=;GIjLO&#bo}Qet*D$W@%S2$gLwXVzAQg(h)Ep zT0C+Qca3Udj+G z<}})7y*CASbGMM{zo_|l?TEyKRvsTw4cet8Yq_!DL-fg&1?KH3Aq02Bn2_BKQlf7Q zPS~AeqnXT=Q6Hf_p=M!a^*X*nB*Ph`?&e|#!pld1l{9SRUs73LCrglG?i?#9D7b~; zq%Q4-^+KI|N>`bmbor+$l0t`L-Vm0gtQm(4SPWaOOT3l2d<*?X8rZvFSpm5EqI2*0 z>R0P+Olq2Ga6P(|+K!0+4`2_-6+`W{=UcqhG9AC9QxD|Z4ic0t_~p?tz3KnOm@u$& zE~LF%emixiY(@u#bkO80q=4|DbzaCTn1M?@LYyI+8yodkg4kzrf&m51I4*hDSJ3l= zgM%SK=lV+T(J$ULGlW*Wo_xAOCFHosNMW3-wSWk$w@#0^Y^rlwK%(Wfs z!EV=`b_WXQ1$iS>{{TpcSj>x^+h^0S?{Jv}V2-{PXD*|o=|hRvko<|i(#!o zs39)ywKT^}`rvNyovHMh{6L{S%nwqF8D<;tG4qj)Yh9Z|ao5T0W=b{cLFJh@0>gI= z*$ee^CCnV5A@RrLTVdVzeg1v&tNiGF3#s=jS@`?M^#isdVq!h)weJLQdP|vu0c0o? z2ta)u=(4_-wNxTX$8Rzg=kC6>cP_t#-oHt*?Ms68DZQ$z2`)ou_J#ixB3^~NB)-Mb zN}x7~o=e1N)*<2lMRm^GJ3)q)d|U(Ckp2-XVZyivW$Jibwx2IHZo*(k)E;#X*&~n~ zhqYKdy(1r{@|*V1xRSA%eh#*3k|BtbzG!x$VRaPlxNXy;3=VQ&Aq7F3+<4vOxSf(P z8jyu*jKSjRc-|gdLEyhs!`T6>=fQPpRKK(_>sKq`xOAER$JaGU*oIWfnRh)ii~h3W zhD?m_9Vy7z(w??1)1pSCve&T13A<>1;2-SB*MiAI7E2K!=mWh*k6~d`rDY#?$3ncw z^dLB<5$5JfhM7dzM|TzL0CVdGeU+gk@d+Bhm&;5Sa;{egF4ajGCrrZw-A*`kh4b~? zqjwcCadGpEi~--1bS-(B`TZ+Szv35YnSQVYbuG57( z7?QMqWX;%-@(4I@RIIqo<9<&#MGe97;54s#bYo50PH*jhe9Ix?sIA{yNz98rE0YiZ z@uBTfj~IRA7k(e0p+T}DhNMoSj;tL@Q=DvV7LVov7E5o%7eA6MY2ZjM{GH>zo4PHw)NUWhh^e*pXmu@J7MQ{oE zao98&4#o#=Xp=Xbx~R^`&=lg~_iaE3!?`_{);0nV*_;+O<6>54=jYd5(w^~LPUNF2 z+P}R;VtPk|cipx*&QPo?_|BhkxnXc%HW9w;5tZr+Sr4C+-OyTLy#(Q=TK!q}_+-+r zdW7&FjlJa1-*x;a()U9Am5qupLyU%l5g6-T($AiCXKSyAO)Zl>s`4uy3M9trpoq4) ztQ?h4Zg}EPH^Ls;RfA!v&E^rnJGQone~IA46znlVEZ5Ih%lpf*s$F`&SJGG@BKIe9 znBvCFL;qv#@=K3zx^E_|QNlVb<5e`4Ie$(Ng{SyUr|V73)5yPB@m|DcioVy%CWn3_ z4tUQ&8Q|+fLnR;_a29BA7PE{IjYjA;0{&m_@ceg2j)DqzKLkSLid{!3iTg7Y@?KDDERd?YYbajI=^-UUyjw_)MuznL0 zd1F>wa$|aGsC10G#$#6P-`u)8TpjF%@R=#g;)B6B?Z<>^!L>q- zr%+`In_zGJhT661;jzWYgkl*SRwCp`EVz0X&e}#dP^CTbs-zw2y0wMhJg&_0>Zn^< zN=qdEO3f-QTSZyJI$}cK*v}&ynQG$1v8Dni8!U&zlhz$gHcT1uiXX;Y{=Mab#4284 zF-2U)MF+sQQ|3sT6cY_};6^RBzcDS&3=i_wM$Cpd=nyDZA@H@sAdjtuRvM9zs6Y@EqlvsKi8;Q{4H9fV%E_1N5+M)Fxdjn+}XZ{ZIhaQm~4lr;i^{ast;ao`< zGz$F~THQprWm+wFO9`w-O1T7BVL(Es-3#`oC*e)vrvxA?&@TN7=LvKC{?5tLLSOsF z&6z-x2q9S_DZv;sxcA;pi1DWdrRFrC@cemH85UHT5s)~BMq1EIm1rG0n6pV?`~QNRznyF zY3Opv8iXTb^)Tp)l+T5ReWM8_CR@%=x`VnC1+7=wr)9v7%BW{?PYE-em&eObBuBx+ z#1q!hEBN-h1_qm7;#XC7fvm2cpWrYITLkqx<>h1>cwsTJzXP*Z)X)#cV5Jv~VG588 z!zvdW0hUQ;<~xiQU&#p$np9Buq=))_Uk2m&auk2y1W10@U$LJfEhI9hYL-`WR^uLt zT`_qJub?gQ1AONJ`c&BVC04FFK-ylv8t*wyZTMjN8(uiN&ZrszwsKr&XQy)D`B19| z=l4Zmetse72` zwkC{n^gfZYUXh41X2xzzZC%7m;uwvH&9f`K_NxDm5WadhC4lkDhX&DkR4CL|qi|Xp zdgIGuq=G46z8Tu$i-tA^qSK6UZqV^ow;OrJ5jr$IE~l2UAVHyIBos~`{Y>XHBVK^F z_kll0m$jlB-WZj;1rj1yWetgk6V2vmDYjt$lTXP*beHwzJ)IX87V6=3iAtYzXc399=u&AvNQRyiWQ$d3 zE4|O?s;sOO=t5`%+kSI9Rfif_$rlj&GvxfwAW>|E)l^*#lNxtfe{u=$0;#gCi=ods zQ<4Qz%dGZz7&4*QZ%L1xDf*hV-6F?BiUcXaJUmE0w^g)#@|liuy-P*~DclMcm)0~@ zoZgwn=|NSI-zXB$0pU=}L+8qXE;>mMEQxmlFA@wmHMj(m5-Rj)1aI|fe0aG`wQ5T_ z`AT&W-0cVwi;H_S?i3cDuCwbsDaGufZX>QO(ZYG`w7UXTt5+s^S_feaBI9!8}5pIq<-a) z#!yfXgs&Q-PZXy|KfgPEcX^BMeqWrKnJLJu;e%-D)TWpyu`*{?_5iRExrkU)L@EF$ z#KLJa_r<>rZj8k~O1X`4TpFeDAK}_7L1=Wiq}HDuZ{O{T3)<(8)O0w`y`M;(^bnsg z=%zik>sIo~wUPqIDiNMWpcnU7k+Vm@?Pn; zc^D!MZ6Ja>WuWuS({+4YyzJn>T)EhtjEll*7To;^7mY^Q7FEde3)bh2Law_XzUSuI z=Fs66DsV>CO>DbzF;W{A>wPv93>@2iPGQh%2bI48Uf^6_EK=Wo10i>7TR*me0Qrbj{2Sp7aRVx0<)+B|NCAI1D!zbCmJb8^M?JRY-Dg z`c80VVGH3njggrS*C4|nqPQ-l0K)T6&qN}7;jjF9-Jakk{M}W+Z!9e#Q+VEP#*N$` zg*CV&;;w~Z>wWpM3I@s+cg98n1a+om13$|_vM87U($qBcBw)N?b92+9{?i#1Ctfd+ z%SP04`!$s%+l=zmK^(*X*o#2__IjNhZgWDZ*{H5lqmM4-p4lVnL^GjpMWU1}uGeO; zjNvMAhGb&u3Uo}=T-iP`gFL{;KT+IR1H#1i@8AiVTfbh{-a%U7AR<;-#B%wbcD0-e zx0-A(aWZ;eOC@{~c*RbCx;0ZS*a8qNfl+{p;&t3!vW0e`;(c?<$;)$F>j*V4FkmlE zt$osvfqOEK>e*7w=@r_mw`QPpKiX^nvtEt<5dBb?1Q@1%;Y{D=2tcw1J85C+M9t*RumjFdvD$1K`z0FZGW+|X@2c9*Cb;ZZe^Nn1LoWY;H zU-A|Z29poXYPEGI={>YwsQpK%+9jZ~-iOVh>RI2ebod?$$^!AlFl5464t8ENvMT`n zcu-~O{S}G18rGBp*L!GJAsgJJ>LzI=w&t}2eKdEd&RJ6l*{d@s6BLO}WxL?aDEa&d1Zn1q4(x@?sX4gZwH-mH zJ59p4KCCmRph6O&p zwzKfkiH?pAeWT;MHrIlCXA+RR-2V|1b4f$&)nq%o1m_6N!XNsz0%==C8V#GqV{4Ctc3C}xqIocx`lpr`F+PP# z;umi!+u+-TCw_=phf<+U0a-2t5dh!g(aiNY)%1q(B2%1LWryt?0?cH!r>59z=Uj)^?lIu8#P%stAC^3O%zjaQtk z_a$78CAt6bG=FaG8jzTBCQ84{7y#^2_Vw|y*PGtuQ!Uk(X8tzdQcc0m`g=M$FvE5N zBT~^RHB;m{bBC8NV3Y?HF-5(TXdy}XvQCD9bzQH#`iD=ZOVry!Ra?1*zvL7C6FmHp z(7=1nt7B-;Cil{wK=;m~vfh?y;UC8{>&f;e)2lW)9tr^Y9>Nw(Cj}ebZmqyuHPM#y z_z2v@!m{;Itd+SvGefR(9se809So`HN}Z0c+W?2Jx#=EZ`ftg%^c=u7*8ND27^7x9 ztKATL)Y%-2;azZE4~q>L#`=ac>GVbV(fiXqryMMiL1g^O1GDzAo;ihTj5fihN#Sew z|BE$;hoz;Wbm_XT_071?N#+5Mg~e((e8YjJSt#+ZQw5SL5m0KhDPI%rU-T|b#t+mv z8!UABqwX&N+jc#$d7n2BO)YpBQ=b(St=R9O^6j3>f1iJYhoq0YLI9gG@#r%(611|W zgJPZI;ap00a&o89CYP-0SJ3vITXT8&39*)0c;-qz?f^paHwAl2cv4vR~zVheIzVWC+N;f8{;$V2T zEI+Z_BbO@iWH1lK!Tef2wIV2%vroIJZYY%nf+~QI6~2Vrap# zS!(K2zYLK8Lm>3Iv_R=2mpa)2pqjZ%O>c38*n{~cmpOiy{n3PT#O&6=47s}5&>tNz!{(tAa4_w&k0lR@hNQoeV{QDxa%9xY5 z=!CfN0Y&};VQ6m_G7=JpxL2-|sqV$~V1Iu-E=7&|Vxz-0gyFcs$&kGE1B27emjGf% zCjWYV(R1f58z+=gW!2dy6#tt^g0*#GaP5wba+mz@9UQ-o)KB-vq;)A8dQ z!{luxA3fK)&JXS&1sgP%;gC#FbsueFkn2X2xGdR075=_Z?_F%4f4a3VERcTX0o*fj z7^0BuVngtrM7<}Fr9hrqnuNT`_&im-94x&3u!@?@ss&LUpn9`SS2jWz=A&1s9%NS2 z(BIj?!D03}W~={SNB>6=k|h~Xn#RDo>cjP}sJ*DL@J=p;=MoFuuURZ`V^AQCjmD)& zsf4Hf_%?=F+6l+USqs{SMPQ`YOQ3LftL}jK5~CO=S?gSpKu?lfotRME(EAACP_n zgh_67Ai`irOZz2gxa6KHcZCgQ^`{D+^C6LZriSP+>sP(Et2Ln?%Vcy}*K2;f$-r4l zcIDa|zHb}7Qmid2(Qfm6i!Ql1Gi<1+`8W9Rr-9%DNUZwpT>4umz5FeVo~7c6Umi84f6%#!eg&?!8ip5z0Amqg+^W?Or-EDcgieP~t(M>0i#2;;w zfC~l7U9OiU4ea301@10-W}=-P+=Kr^=KXoaKL_L(00IsQy%PUTr>MQjVZRm@=9WgD zjy_Ojq6ePIPp${0xivR;s~rIE#nv|Y;op=#ejGOi(qi24soZ$!-0@FNT+{UgXV_3u z5mkq^^B^7ndi-@*FB-K7t#!KxTJV4w@ZB_z)>j9!Nqp=~FM)*AAM*3p0V=SsRxf~} z9tz+#nB}j@aBo~DQdV2lHdsjlZk-l@m?0wdIcotS50w)whFhWoa&XJlt@!}?!vZqC z6LEKyFVZ4!=cI6?jJWyYUW&d67nHCzf9eV4-A;ZrKCI*#3}o*p0I`92Qb~n>TNtxQoRE4o9v8dsA@LaCv?v|wVN&rR($sg z7Z1oRUl%2fXMB2*C>XfEK`MO$`%%tFCN$J3E&ZJVriiyOoHX+vDE6W+wcuzGT zjy)-ZwP8Ak4tjznP(ghUN6Ju+D*~v<-1)}~3%s@wD^hFNG;N6bF6vhOpjVgFE=G-! z;k9pjgb56EUp5;es@CtCBkud?%Tit-8zIb*2gxvy4WQ8>TIoX+hUmcuYD%cNqboIdlbg?aalveG_fz z)0WxNFT4>R=fik);05cupAQuBe&nxsedcgs4DgH|^sI!9{!encql8f5oZl18D^LEp zk8ht%Atl!86cZdeD0#6+x$I?e>LZNdG@TZ4K#z>T$%MIQ6@bg2s*J{D2Vw;N19vL) z`pAa|qnT))IDnGufM&kD;6--WpYN@mLYt)`z?!xiST^)U83?X@b{AU- zn8zPAoW4mTMyR&jF3QNc*C}j6hn*HM#Z%Jrc9L-cLI0tWZ=QiKlC*7eKtJtCl`1!% z>Erz^yz@eRWq|IeKbSUjx#M(eYwKzN)syS}CcUo4=TDzb!2l~!wm?*-drswZe}50U zB7(xA$Y<@Dwe%HkeWiCVQcF^ycHafiKoRz+u6E%H{Cjhi?XY;E$o3#Sgo|Mr2ck#9 zCzjN-mQ@Q+QyDZ^p464L)114&*B}CfUrO-My!@{b^e;X*5Ihn*y=UQk+tS}#jRNBi z|IEPuSqO*loCx5eodcUgulGfGK~?Q}hOQ?idQ^tU;<4y|fLJazw!Ht~#1f1D7Kvds z0|lr)sj^ybb-|1B>heqWBlGkS7KW0__$*55345SU$?tkGlLDfTy8)SB>oif%bL>z= zC#>yIfEMneY(qpudujjx-Q;MRj3^CP&q}m77mQ+EH!3}}{vB$`*56e4cX8!=4_Nj- zs^OB=%I|g&p`oGMDh0C<@VbvH(_C3WAyZ@X-`j3*P#&PAPv*0@`^Fr;q9eAv1RZ{G zV2vl*J-&5mCP?B%v?jLfk%{L4p6Re*?P;KLGE$fACOMr`3Z3tA#hVdhYEG%OnXt(UbbE=4GJ7^`r4w|ue<j)oE-oz%x3%ylpNEN# z#ig3yrh*U2v5IB1?PD5|)#EF3 z^Wc;19;012N=Qfmk58{!<9Ch#4G?n4p^qq-fU1(x!}+>n-_w?&pnV72Og1}rqw3dF z3G0TzNkA^YIuPF4Gqw=D{ab(}(Q^y}^O=+V4a|Z})v7x^OM8mZUhHcF?L9X*8`Joubx~@V+-JSjOA%nTwmfKl)rmaQCe6Etf+sn z&SZ49F%a_dJuKflQU*>=Cw0q>U-;{W;aE;7&Mo|gH{j;wPNMDeHsiaCUG2)YX$rm1 z@xL1nj4~!*u|kS@=VG9a!d(Kzv7ZAKMT9l)ah@f%n zi?`mMBHJZ1JBUfuFC7}nEf(6SbC`p@d!}Ui5+RR+WmISdN=Ocbu z+?PMH*;46l1L&Y+tEcj^a!IJ$#FL6)>?&L+k-o=Z4?NL)t&qo0ukpi~OW_^Fd z^fU9vLuKq$p!X9(B%v2wsfRjyD}~^7tY& zP^2m)eHQjek%ckv;`){y@BzzM^e_u;=XAiyt|$tqg?WY-|DjANyeE!gG1^fMqoSft zKPf6seJPHwb#H8?iC?MH)xC7x&54s`prb1~?MWvB!MaS2We(uuxu1UsVbPtdHmRjm z5`&eGZI2`qnLGDhS-O`lX6=FcREL_d(Y%aSBHqR{aO+YPIyMoP=Jj}M3mR_-;-LkR z19v16jU$!M(yo`Wm0}05;YOv2gas%kTbK$S7{+gWzOY%_OA#X4J?dYLpjK%&J>$X^ zTtFY6po1K_#9mO?bHRi)b!o>eq+!*JNuqQB#Mw6;mRoozjEP1)Gib zyS9|)UX%e(5(I&70MNPL`FZLsKms7B&}INR>80#f!TR5wvgYdY-?jvoP;uHNkEtv= zKHOc_25S^3VcN4ty{2>O3C00)URsdiuOKEy>N`-t@TqQKO_rgYbX&{5XJl!Q7S{C0 znS-YtS_AvX~V3ghI>v*}Aw;Y1k%(SX~$o~}79=GcPm@mg;qOm0qA zEqEOdZrDPXdlyKK;HE9Ze5{}^bf@;Q*LYK_Y4n({C1XeyN~&_|7o@$3Ht;o@92aB9 zK8i*YiSYR{yZgRr{SKycc}Qv5s=7WmJNsTHx?GWY@22!axxr=FGCa%*#;4x#CG+dI zD&;G&ALs!);_{j}PzJ6Z=9T7xA5h{gCyOi+4Yl;nn@1CPE`(IyoA(xVPdYOaYU!Y0 z9FaV-5m@rPqXlaH4In~?1AxO^joI~yLijxJw4DQ0P6w-yyBL-8WZci4i7a$d@w2I4 z2mU5({!{0*!CGQK0#+Ehp`n48so(8Xb$@ibm{-uP?)~C!=jUsl`sJm0qlvR{Fd!*f zCWZ|yoLzhub0~IhEyfd>F;UW5qY?$WoD{B?dza@UFf$A<+hIdbcy+495($6>ZDZ5- zhPZt-@P;^fKA$YvS?Vyk0@|-4M67eFxdqV<|HJfS< z{NuSeJDc7a$!>>1`Xr+jUxf(1vqopv8tD@%mAb$(iZe&}Vu1;e^?Hvm6V`lnI1k3( zGz_f+Y6(Yawz21xujj7dwZZsXhW>W|c>wsEwVtiU zK+0sh(f<~&ha?hd6B=c#@-^FLqr!m0Vw470P8X>q0#U&4g@@5qHtxV=& z*)T#MlPfFQL;f?z1&`>5a-ROC_5aaz6<|@eOB;~x?nb1gySuwX+NDdnK?J0`L6Gil zq+7bATe_wDfAM^AzH`p+<+az!;_kll&NI)vq5RjqccECcHUtyXai2Tvao=1j?=f!^IojCndLwq@C=`A+xyjs zcq&1CyB>{bFF0)clNs2ihWidAfpzN*jn)1K;s8jZ5KloAQ*c$Nu;mY}>-Q(^t6}Q? z2PPKXWjl}~(cC@PaBpr5>sdN;l1tZKiNh#6^LZqxA(-ULrTkLHEGT}_kaEK2o}E|m ze{~rc9zOJ=v8K`Cv7ah=we7O(DR+kKvQ+Q#t%~7tl7Mwm6q@&PpJ1PWyCCSPc;3#P z>S&Q_2I>0jyx_R;h^nu7Ga(^i2KS5|pf5S!w>@4h;>$X8!Jn4%!EC@RSQdY~S`>() zU-3h#47Y62Xu2=2c4n$f(y~fz=#os75h? zy7txRIco(Zw=tAmbdtyO2OdSz?{`c1t{8@2LeR4`d(0TCwd~7Sz%pyH!e)Vd+LiVN zLt#D;cxh29VdzM_PCK0?n(+2PxspbOi?mLxqetLh_sGs_-sC(9c#`m7FOb-hIO|lc z{_@h*RZbvk@xtdih)xx1Au10oZliVo42U#PemE=Fa>x(RL zv`S4@9=ILxPhgJgV_xhJMQ@5dkC{*;hK}cJb|@g9kWfTh8rQqW`8cy6Vfai3qOWvq z>Shv3r)$0TujB3vS?IcwsJIZFsb2Qi2sEk9uWhaFhQ#i-{gZUaMGI>DO#?V z%G)*kB*BLr)d8ol{ho?d3HZYLFSYl0YhO~|{#9Nppqg)qQCNyXW|JwYsnteS#2Lse z`)a+NHy=Gy$a#IIYtgRuZ@w{K>1zC|} z=nJy&nwR5}@#TvNL&)bhZoH|;Q@zD`UOnQt@$(fJuX%L+`1@=BVUT|n6VYP88jFJ! z#A^A%=l<(F{J#CC;$sU*zX7n9JM2y76)@M2Ca+!3X`xa7M!zwG+@4su zn0iA&VIm-Lvw92N>2h|cY4bum5@LhAnK#{2@lj@KV_3bBi#+KWtCYQcwmoB{wfIf5 z{#PlVOYC2Tcq<=#$#X>X#eF-ga0BUK3F)I@DPSn*1k%OP*G%k+6j zV!ZS|2CTd#0BHYt3fMaCU6!3^9kq#U=^)fog|h%m*{b+C0&6X>6uP|_m}YVkC;22i zoM(Y6o5_(wqmK2`SMP^CWylO_=Pk=SBLFINB^5or(yf)q7!A?CtRg^GiLl89#@z~5 zQ=)idjtjg1N4j=_k_Gvx^zAo=4wXPdV`O7O&Uel;>dW5*zSy6QrE}MO04gvuhf*xi zv-RXx)1Z<9Z+Q$-lpA>4_@97|;fIgIT~P2!t3R604xom-WB0DN{B4FB$bG!E)pOF( zDj<2E_s3ezG4018W{LNA^}D;=0cR0PQaRQYvTS-84a(ou8X%gXfHqdv-cqZx70roMNo#e z*m+x(kzWJEXq6Zt9a>}^6Biip$xo&+Z+^bh7+*I~Xr3if<|Yvr78a%(jzxFEq+3;l z&%d&7%fYmYNkf=l(!0SZVRpp5gVB?WYl@`DN&hXDZ~XY1tLpz}x#=t}2j3+OG4#|$ zk+zBqS)W1(>%`aD88j152TRY#hFWR1aD1Uf8vS+0p~YAs9p0GY%lu?mY2AAOis0p% zDP8!h5}=(j5T_F%RzlBlJ{ zpg(oML7SFHB}Y%PI3>gI9(H!(T`8=u4}iW(Qn4UkQvPpfjC@TWSV_ygoHS^7bS|r% zjo&DFdGASlNSI6*`cB6-56;I-K3-qnyJ4^k$g&5Z`3%W2w${=~xY2l3^uzk%^zGEh z@KGo`6q5XD^1p2V$CK-qe7TKVy6JNwqk#b4<_BHGp-vx?Z+RX@r8~xxD^b#(k*ob_ z17|OhzT+Bs*k0F25Wf#6hJ4GBD+*u0tMLo!mw`bggTGw;A5YfMU%)07v?4<2BKl%& zJa1yUPAZd+CkL+BxV4OGnwlitTxLuwbdLfnufr>_`xK*Hwz+IM(#4)zoOZ{J25-(b z?QmaO0n(o(-~Cw){U`eVIt17hU;Aq|=%ObB@kZoIZ?t&%8Igda^gL1W@!Ix#JPnDu zni}DEH^#nXhANo%cXsiEBxrP9f<1`sx%v6|jG!A7psr-BA%8&o^$A7F$0_um&ECHt zQ!&yNNYm!tw9193R4|DxYq{@|fWX3umnu)(J6rO%G&T^+&IVvl#wJ_&Qg`AO|9yP2+ zc0is)GN%wkT`m9E1}GM@!o2ZeFubbmQ z=Asf(R{ngQS^@|8bW$!v>TM;NrePxeMyZMk^=7}qU&v;yq&5d)k%}AN`S*_ca^s`H zfCYj9rv0u*;&;D#dvi1W5uDEH+}Nr&%`67NYX?x{J{xxc!^cm*Ccf!Sqd0S!2J-zy zfFNqQ&ONBf`GdsEX_sX}A7Ls4^6AB7A zDx&!-5+uDO?vo+PT7qaVOH2dMU7SRxZrJ^__IaJqh=|9M>wSH^Y(wcb2B<;O*-iSw z?NluuZ1E5Lz+X+G0AP=*ZiDS3j@&7KNLjqqFYNNcv1QBxW5CDkaRA>vPDK%?cnNEN z;`@KxEr6{6)D_MkZ??B8UmsF|qcGGoG>BOMlxEC}t=2rqc#o^E65Y#QUVcP7Ffh9B?$ zpRq7*V&4xX66pYc&S~rR!?_#K5X16<+aT1c^r*Hq9)_~w;$}Ml53*W9%{2bUqWmjR`>Rm`aErRx zO28)ac8?E&qXRymb0a1c2@@p-s1N->Ca?k(fDDuuuL=}0nyY&wih+y-z&IEy-5g_(Qd0jQrx&&(3 zHGLpj5E%ENYlHeaVKpXAro(x*)8+om^+-CmqfPSu!vDK}q5__X?61ZeLg{zwiN}v@ zCVoWrBJ9eTTUnmc1z;fJ+*!A10tm)Zpzha_mL8P27Aug#N3X=B0SEy6%#E}UT9ISx z5~D{5IB&gdUs_rCKW07uT15YR;-3iEZiU5%hc^V$jd~lgSz@(8+FhY0CUWBTXOla9M%}yvcz}98e`#$J6I*Z`c7};Nv8BdSKY#WN-tS)4$qn|>bU|HluaKfb^cYq|Fp1EtwELjlkRJxpHV2{jmx zZ1q}wA7d6^C@uxUrvtIDumTEzeDF=8i<81H*tr0GRu-0FSWeL;ABj~0J-qUS$H6r0 zOZxwn;H)VF!H=KE0sh_>av)&&Rre{9T-QYMn(On4lw}+KAH<&WK3@JJpbyEXfi@;D zkG?{}UUGNckg%S(5H#^HaJ2$V*zg?md``KREI|9;3svX=z)zzDO)i0GiyPQtf+-NS z?ui{#v8AMN+hut=W%PlSwEPXF472W5!qu^QOdq=sH7EP132eJb%k;4QWbV?~9V09*AZRp=;^`UA{OqF`n>NdS69D~)~^+fn{*QT=7x1c0!X_3CnfZk!EJ z>`(LvN+H5RJ39Pd>0a+m?GyqXzBD`^9*G->dK93X=Fd%7S0+=V{HK5YD=7G50^lBB z;2U17pzpNvg4O%(Jpg2cLi5sggEz6Y3eV1M`f>_L; zgJw#~?-NUejR74pz~;(C%#o&tlM4}=BJ`4@t7H>)5iS)Z=j>l< zRQN9<^1m)H>SZsQ2vMDjK$ju_TKbREg~?1tT~HA}AkbcS>E#}~5|959!=SAEZhE@#TfDW{l=bM25Q(?FA&7J2q+N_Cv z)U>zP^7I|BSY(*RIDbq}>#1mIHL__}6|VN6Roa0wog^|Y4WCjlbsI=J7MGOl?5RBT68SC?kz!Bnu#M`G0@Y6cC#FI{2m$mZI(CYxe*bA4+Z1lI35bk zXsz?x1UK$iZZ*6S%Tr&m!s2Ct*U@Qdi~b4g{bzXGa2OA-TkyJ=c9O~30BRu4OL^?Y zWsM~LUzeD*1=%+x2L7VdTT)vhOiSa+lLLAws-R+3*R13t$EQu~hbw1@=x_)c;y> z{$d#RHG_xSbLjkv`!5tSXS`fQaJf@argLJD_j_OA>7(Ccv>dw?{_wk;uo7KP1krh#YK_LF)?-|(|C~khF zs@IHT+A;@-Xq=&vFppA(ON|cgMtTj#3q-+H2lX)QjqvU-?GqqfZ0O!UPfbl70>H)U zE_HC)!2YQ24%wD5cAgk5-LS3>p0OM`|xR9-bMNiX_|CPT9slYbQo4Pn!OQ)2o zov1;(6&rxl@cDdrheDot2C2b8B8|M6M-xt7{59^?9oS_51A^%J0U(tg`i(KFQ5CDm zVEk?_1Nj&6U>ycak81~N1L4Ei*}P|@9hO)8q#v8xfBMqt8~>_L{AfpkOIJr8oc-G) z^H*-uz!oUr58wVCQVY(9n*p)^GcN>%t^1J9`SX0|(poojr{f@!{;xr*Ow5|phVRc5 zv#=TDhPT}{05flUTQ!4Bq2(xsU+G*t{7myEsMAY-mi!;Yb63TnH?s>Deeu6@t;j_CI zmo0e|O!?)zoxn|#O>QDQ&dm?3)ydsh2iF~0Jk@no?Gf8wrF^X@PV{HYi{Tj z(rTIt^C-ih>B}}c&F>9O$cQ-@XA>%}6!DMx2=m7mu5B38`1y;BDHKuVyED^%NVv_U^*02!52Xw;?j7 zQTK`z>IPazf2Q0G7|B9@ml1PuZta9A;Gi+sJKCFtl{9UOJ~g&H)&yb^mlaSgDg$y` zy|*hlD!Pnv89uT3X&d878PQY~Qe>c~Ro@C{VG{yWgyO)3;?8B+z#SXw>c*1a z@SKfSbLFNWHRdL*zOVjCYZa3TpNceF@G{)b_oiow6s*DwK}rvdK$G!ID|Fe7TEZo+ zV?j5{+9>>;!PhIhO=neKf;l~0S2`DOy$}(b)cHql_&T$DMPc~_w_3b?)*oX+yIx@o z#F_FfXpVyLRV*%-f@vJHSGV}Yr8QKG`7*No_?CjiRe=kD3=K%fALPNNY|w)iTX0)4 z9{5$vBsl>U$COb6rb}M4{~GSu-2bsPV zLc*ba%<%p(R-tUtJKU78>uI`eY0|3!Z6^ZK=WO_NlYHxqQvLK+czz(XZn=_hG?FnU z!hvr;crrRcfA@M<*K`|146lhk_!R-1UYw&IL5W&JAKJQYRNUpkaT=Pv;34QcsB%He zjGXoHprIY%J=qx1s{&F_MSR)4m zzo><%e?hpuM(9M({asT(W?5OCND*i2fo#NnM)N_dLo0WMipr7Zn`h zS=^A8LlE^WLr>BAoten05s3mi@UA!f$;^RF-#=!OND^ohpZY*6iE5S4v3fG;ukjw>S_^RQ} zyzi$xfZUS_sF>*Q7g9oDd%}e9>3Uo@Q+Xj&A5eL?VK4g-eKU{r>K%cZ96P_X>jen` za@1EyO9RAsqx4Hf^Md_<@rp=wAlPx&@Fp}A&OFHCj=;FSVL*j37M>GTuMXj3vnmTp zfVlLRP7RZ<@2gK8F1$>h|pA<>=*ci{kJC5%5r;3;&1FrstfN z%pfPdIf#(}qse5sZ1jYPGC70+UiMw;#6i!&q3UtZn_TbDSpdKC(TTt}T@jXuEoaM@ zXMm3M9j@btT|lBq5_aq~gYFN)DBQZ!>Cn#CcB#*kx0uS0ctm$Y&A$;67-E&mn1p6s z`Y=7R{LTGbuWgI@dcUNiK0%b098l-gIUAH5k(iIQrj!@-SCG`D6GG74#HilV{1|fQ z%eKk<*^{NeDQT!JU}FalS#OlKLgVvTxYdJE*t4VO(5mew_@>Qq>{<(L3$`s!{{+_g zNU%rF$?DT4l52ZE^1&=(R4;FGU0pKUDy>_{`8Ip##=PmunRPu} z23pl@u6xY=T8Vl?>IGxEKF62{*AD@4=)LPucddY2yX(vrRYPwj^{kC>UXNS*IIGGk zI`0b3%&&h^)5M1760eXX2_tia)nZ&AEl!DT+x78@qSeK+^z1W(%J9O8Dv&>W^Hkw$ z48a<1Y#V<}GBY!4XY;t&HF=ryWUC!k=0c>a2eVou~ zh~f#3Fd-)In8#S#V!A{+K7a{Ksu2n>V7b2ma1D(8yYg+rWmotJX;C}$<80$>7_UZc-RTgZeSnt2SXMlE_ovM6gJ&_Y?2OD1# zJ%e~!+Z?=k3(|T^Fd(xrFDN3{Qqzt;-}T8nH08?`XO99B?bv!NQB6X}I->UCGZ%b9 zfx81Z%XtdQHTKrcS_@VE+~Nc01bsS7&GPp&L7VJ5*H?AJ!-8P*wB`h^g|9=qk(|ui zhnFHGveIHy-_Xdj>+pP-w)LB1A18JeQb?ri>?2>U4auwMTX5QEMU~Zy@aACXjZDCK z5t&w~nnXxiMjwh7fgwkk34tRNlGbPyfLltH8=kQdizHR|wLJ;3CuWw~V_ z{5a@6j7##wLxTs*7x*^8&qoKBSBtW&9ql3@mj0Hx8yhqU$Mz8O;X4Xei==ENY0ThW zC-c2`$CAF1+`Af11b67%+SL}r8V_H>k;On>iY6p+sJu7QYSwphm|B_82u!ZQya) z8*baS9My9csa*9!t_CM;>}2yxW&U~6i3Q?+8_e6T-99U%ykdF*?gNB7_*r+d&PUTjvC0%pGFj=2=`p0AN z#x;iL?97$kga#~=&b@jUS}ueUedY_DLL&fEmrK|)=qqAIabEmI6jZ&~07bw2HRX|! zG^}&bxFoym;?81opgp2)@26QujA1~;T-|%RSx(-$!r$J5jI1i}{(@gK4#yhH(fUir zy;%-7()|KZ)c8O*(%*JG<7h`Hg>>L`)g1(A-qS<%)-e%WOmD6PDVlzr`6lLWUE~BLE z-%7ELM99PIvZeuz8;*jx?=wM^rVqqp{z1@$@jTq8-@VK#^%73rj4YFhJz2%|~^4b@QLm zl|7(&9axSmfar9KXCJG`5HT;~Nfq|n2nvy^Htv3a5m4k}MdSp)HRx7Dj9S2J4rmaU zR0rW~+iQz@gh&UDm2BUnBPecKqEHU2hNj>e?#GLvRwiuJZ}hS8PLr*|&WENM-y3?R zG5cW7{<2qjJRH8XZa-ZNebb_Rh1PU}6D_hOtG~!-^KZ(u=zYlTR~;I{yha>tqP(l* ztOg@|b;iW~%;w>(7D``!dNc2bYCuJ4>qJ>zSf6@-5ipGwnH7@ z9xx^dgeKc`qrQU{*X>hrAosw{2%}XZ9~$i(6>zAW0%FH6)-}Tc|1k?|Nh*R|hJrr) zs5wI7;#5ao02sH;BRpy(da~vlBg4K;o^ZQVfL0r0PxY? zSf%t?+x)h(-{kYEj*`-v>2Vb<#k2)ED!;$sU7UNm(Or{axNGjk=8l9x`H>IqS^)Q4 z1;Di751{Ki!V$d>^b|l@)~j?o+Ys+`q8DuVit{%RvxWgd0@H6qS8?KED3|!OiGE>q z!BRc521=Z7xW>o6_qM|0K}!K|!>XQ{bEj7B^*O0A-ej!9QI02`o+oHslzPW$UzvuW znN5KoQnOB0k_L<-7-VW^_%ao>xe`Yjo`D7>M#q-iw^tBaY{5xKDctReN{e)n{vqO; z<+3>mqEge)%nxuKrgA>&y=X$JW;AqW13Ubx7^W$j+O{Nk4LM2?$fl1{i<46PH^+ko0?}?~%EoE41%(7oaryRtD;;jWt ztPJ?{>ja6EsJY`^8;9Y+N0%i&S=kT0F|O)G$37|99sOMABeRaKh&+*NU6lskLa~V# zack2J_n0voJUccQ3ER8Cq2DgCg|6p;*2^@RfWo(b1V^ zp}C_yA%m_=C8MlzFENFG%JW2&`2@J@9I)3t}Ohn z>F4&g%Ua)}!kh4u4Cc;jin8xQ`X{C|>Nllc(p!|E=$bOt0f6CaTGB6wf`2U7koRv3 zDEc)7=hR7_IsE>|VXtu0pw z@?>SZ%?6kc^`ddg43=7vM*ZFc8bt@_VGIH^y1F6)+U;6E7RrV3FP71RDioAga9g4jt*Wm6wZAu2P~lLCz)lI#M%p2tGc4eJH%Ah3 z#cDw{e3mhSYu18s|AaD6f>{6E0`LAPX@*f-_=&Q=y@&KU_JWpfZg!ePLrwaq3u(;J zs6I9G`pU6=LPb}jq1aq6>t5Wy<0=lntR#T=b>*(!kMP#Mf`dDutb-zALO&7P_>ONG zw;@_su!o%xjl=p88Z*LCXw<%$KaIEJpcC_u3Rw^MIQ()y@%rl@ouuxt*enj(*I&>uzh&Kv*EU$r_KjE&2T{EB7qS?%< znYKhL-Y3p&yBl_G6-yg;fAPS=L@&n3w7QyO4Yq(rX~SfA}SBCJC)?` z7`?S1J-I_Jg>EqTtl;l>7K%kzZkBflz2m78M!qHp!KYTQ?(AIm$lS!AnR9fDXcHx^ z-_Sd}{N=HAem3NY6ZHAKc@!4~?9=+RXg7LCT$DHl%>KY{$)AVyth9!2+R##jE&A+D zdJ$DvjYr1mElbJ!C9;bO(QN|1E6d6_-hR4c5!{A1ZP&CWTEJVfHN7Xr57mIV7Ox4R z3TB1vcp_fpl_Tx5hiBQGZ{#Sb&97!25GTSnjheo}1@5qi~EvN;<0Bx4fhv&B~hL zc!*M~3IXCPzP^~OyX(jJJ|Kkw;+kj=fh>Vbj+Ei>$5Kz-WXU*9q*KK`Y0Ws zFe2u|o9>L|Hk_r&w>>ER{j9rbdBHoQqd!Yem-JCfm>5qdz|}#9}NM%6_tr$W*LNFm)mw zeadxPwpQqfgrM|<@)=1W01#_W7k_D(6xHMro_JTKe3sgUev$||mgn>%Kv9h_e=b+6 z*iFMoTC;&tTVyV_HL3E4+^w@#$%WiC2lY zTqEDw0~o;&pBg8+>=9mwM`dm4=R$!aZ&oEMTD>oMmS*c^{7w!IYs!q%D0}PPe)GsvjU0Hzcco?i|g)ggbwD ze8~lcTJiEt2cTqv=BREkGa=()1>eCn4py(|CgGMjyPr&PV1}?|N^vnj_@m+i6=DG} zokH8#OEX&!$x0^v;Bq0JewKOBW#+MdK?mKd^~&2?_sD8t!F##Bz0hL8tRchuPQfn< zpdhzA{m`|myCgbSxp)%Loqcww4lA;`T5+t1y)&uR8@77g5l16#P6Brdiv)@-j|Nlx zh#vFqbq57Yw9_m9TnL}nC68QYlYKQ9;E)d(q~OPy3}$aXfKy7pMeQ6(J-cZQ!Jrb* z`bpU4YOu1iM;>=5QcvhVOmg;^tp^4U$%j_i{$&Ek!DV`o=ipVD0x;R=imEarUeR{J z(qM|iAv1P7#Mw9zbJjsN+iqgBet|=M_r|tA{l30#{IY((!mZw_^#Yo*hT!6hN`>|b z62K0#8E0XxhYT&_Pbohf@8oTB-LU&XGQGY`az>fFS195Y4(aw8W=woa+OI_j-U17K zjf{vZHNuY$TrhUVCw)rYtnd>U#0p$A*!5^ED_Lc*ABZZmRO~xR0$)mQ%9`VKVF&UU zRIS8Kpp1^Z68b@RP*6~ePSCdW7-JV4nh>cHgr=T@cnP3OsHoAuK1zVK+x&z-FLBaY zW|M~N!guBqx|qOYkg~sk!+lltTNcpSxxc z!x>Ge}U*d*Vk+Y?uYJzG!SJNVhHos_;3>*q3^x_v3i zc9L4{n`++|6+I$~!Wi)XvKpbjvpTg)obc}Kz8X_ac+gM$`ZgHt-%i&Ayu=F|@GDlP z0?7_Ypf!7Uw$U3lHdET!3^cj~C7y>K@=lY9qV0Pfl@vPi~ZUStDa{9sZ zOcMkIteCRU$!7iQ@5JKLC&6{LH+wQ(nPrTd-qum@hyVO2RO-(_82MP~IAM&LEa>Y~ zhE9=Ge8PUmQs)btm-A3*>_jb=Iie3&l$J!V{UNv9x5hXzc5RIMFib`pz8E#2iyaNN z$PazF1%0jAX3NK0c{{`->bAJeajhuYB@9II=P8)Vs5fOi6OvVv-J>@S3MjzYVW>hK z5kmQbo6e5N`=ow;p8gV)oh6rB$#lS|3v5fzuORB@p~vo`lrTT^btI36jL=n@+edjD z$tYl8U^co|TmoS`l@DK%?5P$c>gPtk5wKZLiGX_7MbF@sZi5K7q&?z4Pbfsi_HUS0 zLSPId*}16pPDcQT)PrSuwFHH(j(Vyi1Q%yO=kOu0Oup>(&3paK%=7n{=F`?A_gGXG zgh7Q@&%m&ldv?m+_EkW+5E>qp511FT%x{I2xU{T{Uxozall0SdLI7!(>(7sSx#Uk$ zeu9rE6NVGhyKv?Vrz-%n`?|y|ie9VI5FlO3dtZ3qMH24?d}d|OsR3r(mfh?DeH&kX zKub`zZq8pO2Ifx+&-SCIujHMHv8MrX7!eoQXZc+DCfIL2()9JxB`X&X?({#l2%6GE z`k|HA+{@HvwFtt9&YkEr=J`J}^?*I#&fUMy02oAB#+;1dS}U+m?O@O3mp zXg9aM&_oF(Lx8s()?l;f658w{1yC7@30O=A1kesa)Xljm=?Lr;4?{>uNI1ovX08BL zJTP`cRDB^h`xP5H_`jWN`NuKphA80ZYo5`Sulyh2<$!4r7>_iq@(wq9xn zLUfdY>$3AQ{1ke5Dm9bi_OP=K{`;<2N?8xf=XZC}#>E1;ZsNelz08W^se1drO+W^i z0C!V}4s}l65FoP>Bv?N(9z9qAbWUt_O-)}P;MgRZ(y%EE>8D2l_OUBD4Fdo|ggv$0JCG5Mtj1leR0awPZd zsNZo8NI~jT7ST{i!HNBaa}i>&%xm%bWvZVl0fHTPng!wfm!Mh`{LFr=)$wX}GxKyF zpq=1Rg}&YpLCF2FBm+>q}b zM%9kYe5C*@20N{pyPUyNci!R1R$2~5_{)^9Ojj7Rf8W^i=JF_hZzyLfEYLVsRu(*i~J!U=ucUWAR6!hy%VcoM7&9k2mV}`-*@{T z4?>NQ-@9k@bi#zr;DlHI8F=w_z|0ss7J3DrX?$OyG_d8*z`nQg64c3q3pW1lDJ9u z==N^Wp!I6O>UuuO5}O?{Lz1)$A_Bpf-NW@5WBu(UJK#d3!w{E1TzTE;O47O2eE+HI zsa*yxzr&5)DtrSsP~dCMPhrE-a%P1yyxyTu(geK)jl$U$k})H}QM_D-djyElo|=CJ>fYC>doTL~7Y{;)2F9aMnnbS-L|XXC2@7 znO4v1HszWwyBXl*XSVz+5>$9#!;{WpzKUs9EvRf?(7W8EgvHmJ zn-{_zZTuJBGcL1n4_qpoPa1lmV!F@W-fARcQ%urY8<+7)R_jO6# zw~?AVp-ntpId+1OeJ_b1|_{&~mluBYZ}?m&LUnnw+%ipA~T zgXNOhtjsYZWixvH2<1KmW$>5V)#2%nK0-`f+OubAb0jDn9}SO)&sJF1clsfaA?nu^ zAQJC4*JykU4*dA{4&bK=L2@9L_)`{#B)3V+jHrck{p{)yhYsVIamq$Ed`E|#Sx~e} z*vO{tWLB$Gi>Tf1-d0)b19#FdMgAM_B&yw950~Y%sp$sB9a*_7AMxvvDPl7lUrwLJ zW8II_=wJ3iGsgC>)aGEY1F%3w_w@6vEWk@?GqYH>VVWj*PNMzY0)k8a4kJA4Z~ zDPP^#CGg&M8~xrRR-xjl`GL*xeW-yB`OG!K2XF%@iAcg*yofDEOcGkWC&^x@$NTGw z)6491!hhdZA~mTLRnki;bC)wWuWd3h`uAL_KLok7$QO2VSHdvwut+>qB>^~gRE{!s~`D(9BE{uCX^Es z!wrnu1d=2l7+J4=(1YG$e_|QqN{I)#zlbu#+G3_mT8*AZ(JPy_W6O97aU`e&d8%+CRM59?Wo|5=TDg~=RrP;I8hqZOYv33 z@o&M8)6cOvsZ=vCHg*Vpl(sn%GcNu(L~l6;oqMcalHzbKeS>cGLDsEd$(X7daSqo* zRRVU!y&sA2mT49^5U{2+Ou%fbJQDAViJi+zK+~W;W|d#LoG&J3*!2nS`K2zN!QXtu zCm`^Br=s#f8c5Jq)I}b!y>BKomusItEB%oSzNCexB5MacAq*%+tR~CYY45FTI+llV zT|Gy)FX^Q%446`^K{n^jw{3Pq=8ZHZYJ&$?&zYkZn+evp30aZvS?CxMZQSwFxCMoc zh6db}54ml29wLU~TC+TU;K+V|gDtp_==S8bbalXaQhylqSl&9Y zwRza|T;KJjBBVE}p7P7;*rHE6^b~*a^2JaI^9M(UeMmI`&=P_P0+FpH`GEqSr46EQ z^?=dlc8frX)f`XMt83@0%#*LCp-WT0dbymK z^JS6N-}lJr`+we}Xc+&#gNpVi9AGlFuhzlG)okrf9833jI(E^eHm3k|VgK9n9nL~s z6y4nlum!6n#&L9-M;_PX2S>_ot6JYDVBYFeE{YV#J_poan;)3JDMa0Zg%z=xZhHUR zW(~vaF1RCR;%@!oHBCJ&bqZ8-Z^^f$TuZV(7HQUB^(D%hzdV9k@dSXKabg}bt>}x^ zgSqa;ts^ipY}T-Or!vjQon6AVTs*@MJ{k;Vw_}_h@)u>+dnS!t9dCAohyGNkCp$`! zei~*0&HFMk%*+IM7;p^i*3x)0hy7t zxAewd_Dy;0T9gIIO;IaUv?Eb)BAB=#ibU|~0hlayJ#O%7$!;tFKxhF(u*>_-{%5RJ zc~jq0Y1F@GS(I@Lv{z%XUFd7|E;q&W#Iq94k~Ck`^m0#mZ*6Kmlwc~;D*PG;jd3#i zgujnQviX(ts6W-jErxtOQdwerLO5!Ka+zbp%~7%mI$G-Y*iuz>?tkXV;6*X!Hiq z1pdv}f5CKUW+`7QDcAd7x8ul-PH?{I?B{BW6ZZX3P)MlWlSfBXecrwyN5!>QJ9JtD z_mpF!`=jY~D}IjY4#+?XK!t-%v&`Q>++vR>8NF5JRu4Ia(jj;dOuhAd-?rFs`H=!u zF=#dW(~V}G+GwYTK3^${RNHKsZo~EEQlsYJ=@0-dS^M5e{vA(YGyNDCs`V#E#HVcF z;v7Q1gj!*xgqB5F`)UJ6!yl#4a`Lv(`#cYT?$cK57&!>DZuHJyX|K4JyBY=cdkgp5 zjVSexb;C4qHF>Ww=^L%?v9gGJ3-+-L1Sx)Bw}U4TtD{ z77$;H7BWT-mhluT3?1RA_F1U$}Mx^Z^5B%5EfgR*QU=K++Ix0DDJ?VTp$5)LI=~C-MH6 z0V|-43*Z+qCT^%Ekz;5mn#r)Vx|~Be$ep65C075UF?uwf$uFx!BAd3H_|h81JVkX( zsaJ`OQhjuM6_VwaRL7+k4XW zC~L72rb@B-w%wSdwwV*eyNCx5^1nXbw@lvhV8&L72EHl9NG>w0663(J%j{LhG1Q^)yXR*TthHp z`eVr8p~4;q;9aI2bcy|9R_i%3bU&YQ^@7m9+#4A834eg@ar>qokGIRUKKbpX=Mgn)DrJ|3v`TK2(== zkM$$cQ@~E+aE*jl&L?-UcydGQkKRwSCG8*lHH-~9wg>x`j>L`WeYAFJ8aFLu19bIS z|Lh%W z^p1`|T~vy|(dR{|4f`t8s($BdupOn#tQqUN#2HDqA;7NdkL@DrxiDE#Pnbh2-E#CG0(x9q#o}Cc$ zr_lU?34hFjVpQzVHV$VP|+vbuAYIx*)UPMp$cW*iXW`KteXcREdmHc?>A6V?MR}*ku1f-*l z0N$BGWXZFvEo(Gb#^hi-)^IQgY6S*(U#OB+)lP=w6!8*UMb^~RpyU(oe675gnnd{fYzK(k0Qk3* zv(_vH-dnOu!amzqK}d&Eve{I21whI`>)A8PYh7M+yK6ABNC_L7FxKdOE$R+4|7%K=nn2n#%wuJiA8&{J)18Y{tbCr2CpmR4nr-*wsX{w2mDLKX zMBxBLcZ9q{Wzy;1sf$85uv~2r84BLm*icjeUB+BjEkCLLV>+;EaSI5DASpnEIq`HF z(!~J>{wDKP&*vt&=8wmo$Dek2o=!0{gml=M%}Mo(CZDum?x1M2g1LGvvG=$c+u1FO z`I42VGda!N{RJR7etk)-h&=rxvG^JpKCZ9KHyItoA1#S?W@I>@fG|JSAJ} z3*MbNeM^7+yokwD=u)Ioj&+lx0N@ok1IH)#>&`WR^7Fj9Vanh3SX4UgvbK z?XwNdRaYSqM?GE}^5sp2-iwt-ptBGPA<{locAo4Lo*;KA=$X`V67UhFh}4shQ@*vi z$PR+y`A<)StfIXdl|O||0gXqzgJd02vcI0Y{9)m^XRnx!YB(OKYqT57S7+4~sAIlb zEU_J0bC4Z96}BE2fiA~tvlA!~7z+CwB5ZL{3D2j_IwI6{jw;Oc%BvmR9-{a+Lq`|F z>^wn2#KDBEZ=%kzGP%$EqEJz6t7xQ8Gg7i#Qw z)PenobJ!^va7R`6uvt)p(=p+fxxNYT&r;|<5i7R~0Vk_3+QB&+cgQUB0(?uqS>+_V zh4+;p^HLb+J*{;UAq`rstCE8pT{@Ft;N*3Lgj|m0AVXF^R$2EigXnnh)B`-hnkhN( zc$&dV#JBYQ$zsqOuuZ+4+A-=$@#~M$_1!2{7I%FQols|a$%3EC#6if0^nG{ zNMKhOXFa{0t*r7OP3(N5edH5>c5(+QAG!+q3HU3;h)DKWfydF?f~U|K{YvOLo=_1I z@G{Is{Si1=OC{u00U@~_ts;A<6R5i0E_C@1#-Vdue znMnyT5J}y5@?3#IiROgj^7tITWr>%7v`k|nfYJCpv02!#jIH&uOZW?6Je`7TAFprh zABOMF-$2{d#xD)U+cgw=$ZQz`qQ+VqINe51^9+aa;&tMclE|bsw--3>#FSmuH{OWz zJ94~|$2+DY9B+?LnPMljZkrZM4HBuQ+6BTHoF7UE;i&}5iF`~>WWt7s)Zd-7|? z;2nqg&fzKZ^5y7YBjWE5V~ED=Wt`-m+90+IU>J}#JKP15-bVpZd&4h}CunqP($xiP zOsEUjp5u`Ad3X#^1NYI|nd8K~iD5;S;+@2GK|{k5Af)xAs?qRab`JrawGQ1G*c2FaSNz5eS|3KkQ}B_jcB-z zVa(2>S93IC4Bzoi5uRJ<0ru?r-G{@`6Y?@n^*l*mw$I7IT3$@ShwOT_5q<}|W+1^n zK<1ew==KcXgRbjr{WVXgveVUc2qt@mO#Edu96uF z?2te5cE)6C>ody5omSVSbVx&xccsqcbJJL(9~7r zqoZW7tGf&D&SY$TYfse7*^@K`EeZEZ;u0@skPcnUddGm9qQHpP0<47*S@)wrd=KiR zXY|)_>AvG;?wp95DVE~k%|#c8gNJ+ZlTNF8GQaCCJ!Ja{v**^!XtjsOHsFnXwiOZn z{WpCWq-m7_G>!t4a*JN9@l8XFP|UB_cu8%Il@=|bGB7Uz>#e~fr>xA`wFfh$c_iD5 z`D0_`1O3Ltn;A8F#B-dnP>KP6nE;UTOymbD_J6fpAn3>FxyaXIv%zWPo=^?NrS9;R zc}DqH>jqWclJn=8k^&%pu?vgE7in@au`ei~pQ5K^i3El^YK{US@6?y=5y~)Ba`InP2v@%$hjQfYq*Dbl0Ej% zaiRvFcF~^*$Q^)ybxKPR;FA%RzX6300<04}YQ*^kF+sKRhIseP0C+s#ILtU0DWXKc z#^*Bw&Ywu*%&ELQg%y*fb6xZBS;XdI&P?VWz0Hx19pO;jXh|%mZ#CUI3!zjAU1!w^ULX@M#1z7p+Y>_ z=y+PUeq;Tf!}`i|JHoHz8;}O^-t(^2$i?y^modbkg)1-Q<_A~*h{&SeK9hR#Z&zGX zA^;WVO6Ay`{cxFCTeJI^uX{I^@1fpOeg|vO0nn$D`YmXv=ZT>;Vn3#JTE1Fiqq&KH zqZ`!1cMArIzor00jt|BzBbMaj#d%_y&+$EGo`nB#!qIwG0IZH4fu|rKFUf*4e`};X zJ$LSL?-t0awD*IP$2u+AUsZCeC8w1VQN)Y1fF7{B@wo9L*_XVK3J zq05_I`bcEaNYq$gwo?WKZC7w0PwZ@wIuf-!=Bl^HW@P|x>5umt{J4~@lwPk8x``M( zO4~GJN%UH*|5ANA(rdg>LmwrIC-_3+Qq&6^(y82P2QFgAEe+!W89YJT>-W7+Qr_>i zfr0J_N&-$&BhxLwipy9J_Px-v*&WmKssf5AhVW@UsE%xpq$5~X;RmSP+(fM+Ztk0P_AOxtI(&dt?S&?8lrow-5DMM5(#i^jW5udywWOqBW}ZU`Ha$ zJ9w|jF9EU%F2JS`oI4M6*Zs=&drqZl`*ggHN--S$dd)l_r< z7d*=G6S(;JQPspT-!>iTz^+Raq-z8MyH0)ogAK#p!ME{NZquYO6<(g8i{BW+74~j~ zOD75k&yUEGm?H6fO4*Fi7j<-&TpSzU_qGt|cOTE$$QW0?zmbw3B8(ClG@RT=32I`u zqCc;B)~jdFWpw~ zAmyIZ=Ad; z<>EQoVu?^z;9TBv>>)TDx6Wo=C1Q9+6$qBOxEM4B5M4gY`Q_`MdiX!{1lUgujNqnx ztE)!%WK0)v*(1s%eCr>Ct~_r0@Zkx<24kNGh*Yz{aj^bl#v45;0zJ><7I>MQu{-qQ3sfZJpCq%57Kc?gDaK;uAnvkv5{1 z5&vt>etn-;K*HA~u&b`akoJ{SSm~ONBZ$tQ`0$))P5QAuU8$V{~I`Y%~#2tT% z&-h~;Vn=!X>DtFp@UKcYwxhL<3W zc+AI9L4u1f_J~%NodKLryz$!YKLweYgu|B(b@CXqvein|{kg1KD73Z<8fj7$q43!-cdU$*j3 z?9iVs{+V_DYkJX;H)mERsU5t)Pzfkrg5rrQHTwY|^$?L?Sh(NNN1O_m*!I?;Gu!OS zl3Pv%JYI3!#Fng|+@QaP6ZoF^)+s?}aoG?ce=~TXOXHp?NEGw38Psjxf}k8eeP@Es zjnru3aQq8m>U?+7yL=ChGo^3r7L{SUo5z<`xt>@ zc#Ro3u4QY0gHm|nQp~d9Np^lyq=abPs|BCW>T|<^9qZbY7 zuedKPtThvqYI(Ly&iluB>+OwpjB+>YNL44UpZy`K!jw!q>o<|d!y#dJ1bxI^yQwEI zxwHqB2HXB1&23?Gb`?cgqpwGgi(f`%sYg~2%H6zcdL)hYtbkS$2JS{Tl)?n{Lgapo zNF)&J1A5?8ft`=x*#mHb3$8i{%?=}@$4|M|>B(gcR{_ju=y?0n_63t6vVu=-W3pxf zDGiIkH!SEw=g|BCPp-&CU~Qy->xxW) z4$~){>)x5oZjZ^w36U!_g2$``pmbsY>=2ZAEU5~j#d`!5Jlg|tL>Su@)yjJPwv2t9 zwhT6Ag|l8*v;WV9AgZDQT=7xZEAYLDC;_n3u8!K;1Ef7qS!2$C(%5!zzDr%a_5=a# zMUImw>epM~0-`zH_}kZ3*sJdZf&=E`Qk|MF8*;LJXQv9w14S4j)CC759pIPXLLycP8>W$lc(VX8cPCf0~^@ z8eny%16ss^qz0xDtmf*5?Lu0;d#^=sqNOqmxc8%r!n#D87((@+o?3KG>!W}ca zT`6aZ|1o*kSl1|mun0T*c-KDxmT#Fsy;t%>xy)Q;Lrp=m))n;v4WV10rz3ly71P=PS||Rck961r+!5>mguNFahIgUR*uFSA@76(1V235^lh_?po9Co*LOPZ!{{KK8>iu~^Fb zbPUl^+&RPCT=n6cpeo2aJ9CeWKmW(B`$Zi**`T`m5-hj$cNzmb0#%v0hm#=T4A`6hX+1=-)b1eVw6j?K4}C!){-&krOFXv4lc`o=8~hCF6rWd$h(*{Adc2@|9RM_Ud9 zWnW(dzr~j0=IPzPz__39ad;Y39*wmUH-U$iz8-kjg zR}ZGg|4}n?JOdPPpb(R^-d`8YomD{vev4C&#e&tk{ZN`*a$jP#$8RP;ur8U9^@_mHS@5!bTh@BlBEjyp_K^9xIl2Vk(j;pA z%^mz}NFwW4gw;QM_9|Au*YN;{svn#77(Q7U!N5XIv3(~<6BRbK6AJ3IQHJZ%IRBJ! zA_lS(+*GXwc<*3~*GQ9S%c9Bos);B2L>rsj|aDLlsrQGxb++eS=goDiVaE zn2sm#?S))Tyt->1?dg&~CUm?KNCsSzj~*)-z;VO63%TaOTcr?C(z%>5#Iv?WtShcJ zM1T#R3fZ~vuk(JdcFkQ~?eEc&vDGSU3og~#O8w$?hmKY-Fcu0!BLu0!7NHy$=S@Mi z&vSB*4O`b7j*9}54O)dJsnV`RGXmvipkV1c%U<4rLF+*2DcI;KoRhhQiKNvL?MhSq zh~{NPxEkW;@h_q(FMxWMtrF8J@(G0AhcrN)bL+BCq&+kh7k`&tX@e04RXKI+8u2Pk zcn}@xce2`hT7`3I*PcAJ#K_qZ5X8cY!zvbnmcx3?>o3HY}KD(BRkJOzuh_h@-yX2&@o_dZbX0e z=?3s{zCXPlxgBmEnwox3eRZn5gTFuuR9+eMu&qkMENlxBzBr(t8 zh)gBGIN*`u#2@FQpQap_`}7uo;`g4nfGU-(50Y;RDtGox(f-W1!dM`HcG5NFvzsz- zqPSi#DjufI;_g4j4*l=V2m6T+XnJSDvkNERKGQZ@8m>-ME4WD~Px_ZR=F^^knhGd3 zy`UHZrJ*w17hx~^bjS^4k~^^n|Gvm3Tp|7awnU~7u(7dnu$%x)KZQ1pzgf2dS< zs1%UA5_51*0Y=tl4Rv)|c;+@Wq}8zRKr@EP0oGffi7MtCEVS_=JNoC#f_E2D!K#FI zf9UZ@M2zt(IJRZ=2PW-p0{bp-3VeTmom5q2|I$e1OpfWchwtZaqN|*PAkS-a#Jbzq zuDK3LVr>DLLGi7BrAaC3!33xKli(cB51A=$5#wmuCVNB+eG=l_rCVwFz?9a5%wnXn z*f9B&rgpTR%u2CrIOpes?53(El%B5cp6q@Ye{x7U))n|=#Lf<(}x8=ncUutb6A!v1h2{$-=ez@38Rb>Xn*;Iz1bA zp_f;FTRI|%2Ve(o`XgrPc$O%T0p?x~F>g>)rKcZ)CXf2q`e5SKhmej#?tDy;9ga9joP_YjDpz_hG{Q?j3o zDh-W?O=si~VwxUeJh3Ed<+fa!{K|Kvy&d8)`-g>yqt4A_*FNo$z4h6(l%?f38sv?g z-HrKpn~S?o^ch*AujqImFz$XD*YXv)r7>SQL`e8su00S3yZfw+f4X}zi~AQ}0&W^( zO|#2t|DrR`P&GhZgkor*5?sxs>dwQ_Us91ngZ@v+-wxR!s@iEVhm^4F%U^iBeZbL} z1=sjhYhriz^)KrcR`&UqgWMZ&A-rLz`-#9mW>X{$ED@pgJ4+0jpc|&8?7O?sf`Iny&czp`LdO7-+5?UU^Q9Oz@4oX-3Au-jL#L~{ZBD*+zWDO? z!)EH_rp7)R2Lss{hB4jwNOxe+YojMSHi<@-GVq8%hU2LlS3N-ua@8AaTwi~0TBdGI0!|hY7nOU+4F<(Pi_|UiSEId2P@|T zK4(ZN_EFEG!qMMu6toEcV^(ki1+l>Yb)NVietV!0!6bMWvAs&6D8CvDK#nVly)h=& zR{Th|;x_k+-J*VgxDT zS2S;GB+ra^RD4N-fP9=KeBcX6IWPtmYk!k(bxJN#X`Ls5e6Wi{7OjFhTel%|zk(FI z+Un{OI+t;-<&?T*1G~Ovg~68g>$$&~=J61)TSL|O7=Nr0zH%BiOMa#2R+zg^mk9g2 z?^lG1HEUW}WDZ|Pc2;lp@Y(SrcTFwX+-z&+Xnxx}R+Ix!A2!a%tDdYZs13xoz3K6a zt{y%x^%!Y8&^5I?52uHJ*+SfPK6gay1UvWB(rquSC~s@cHkn(>?*P>#+&Mx+yN+ zq*NWs`N*cK0fSe$yd*fai)uEhbny0d>(I0eY<`l=uH|jRrqS2%sc2%?fTX9`xudV6 z`t8LZ@0!kTgo;!CIV8c0f1!9DJ~|4=dE4;_5nq`xtzh1qNwyzS0QPEgEP=rC$`1hu z(o7THG%aL9!1-4i$R}7EhtBJysHNR-3*U5Csmor~kFPi<@Qzi9TtGeo`Sn`r=k2~H z$mje6#Pozm{D2e6E^Qq-uq_4@InkWG1~-0X-p;MsOO`*xKf%0@Uz= zUy6V#>fXN-lVHX~J`>ZV)A6#(PNqU$W3E&F-rn?f@1V<2Id@m~TQ0%25-G*{4}9A%FvlOKx}>ka=}f+R zkLDys04;k!bpDR1Tk65<2BIa$D;xcT6s@cFo7mi{Tu-@B)aF! zV3X&A6Ae`s+qh9__Y9oU8vTlebGD-z`>5cmnJi}fMuvR@qMwBf@^TPYCXp9-5qfQr zO|qhh5Gtr9QNxX zplt~?=<5iDQ60Ds_C`;?SnIb%8dM1MR>kJto@TbaoRtFGqU2;v0nKM@s^dEN-!{N{ zVNv=ig#~lkTDsBo@P4cPCB?NZk=jvBN=D`dZYx8vI_cHIMmv<6Ym~{@SEFe`ZDU^q z2ASO5oaT9@g4K$+z{$j7fzM=_a8qz0wtkhk#_oCTbmOBX{3ScY$gcNwMi6XPU)I zI6p2=B6myYz|>eKWC^AQDxO;D;%yE0C>U4MX_*z0A8nZp+O!JhL=0W9A8McuhS(>@ z>+O0yRWP5XqlH?r+Bqoq6XGcd*zR#lmYJZ2p|2h1bjaT~Rv%T^Z>d!=8+ZvEP`_}T zpr6^~5AH@ZEv8nYcGkun7roge4`K%$T17a+;t!B+mFqh(cByr??qyEpTVZCfrd4Nq zhn_L zK<4U?lxgy+)FympR!Zy>t7A`7ZsLAaL2psU<(4MEr9GXj7+SY;tv)IxyZCoIi_mN6`DL9ACVV2y}}p^@Foj}-P!pIa;4 z`bbDI&AO@Fzuj-+k=rPvaVUqZWgezjJCrtVaA)CA?XO(mwqGWmqF82L*j}bgrkIuO z4^M9B-&?j0-stwbsb4gccOuiQvaLBC7X?SJ&#_ zMPY*0yj&7-w!HLcU4B$f{B8I>$+nd+C5GSSbUcdao9U0%D54xP4jhGakK(&MTmZtCCe;h+mO`m8nryK!+OmQT;QalT_cP7sMNo~Cw zTb}*2@BQcYXBy25mzt8c_`IPdq~u+(4zuH4!P-4p1(%@TlJ@LIIyZ!R6lRk;qY%xB zNzN|tNG;Uq9dIlfCdkE}m)OyQrj8mDGxIy%wIaoU!QYPWW!r=Bjc`6EoUKa4uZ7&i zFthDE@nMm1@>+`7y@b)mT!<~+axbOl@}-%c+4xnM<=1|QUiq=|gX*boNujP~o<9sAXs=XH zhmkmSQr*r#W*OKef{biF!PK4^;>!K0BlbF{F)a1D;;f*?x&CVAepEH$ojINkpWk#g z!;wy7vpimD*xUf;jsuHdzN6~GFiKe~dLT^LeCpuOVyn*D2k67-@y8xnQ1g5m=u&th z71tYmgR(rG>_dUY?~VM4Zf5z3c_X1wNGF%w=T*=FJGpy0d`8nZpB?S?((7KkndppS zfh(^o(>(tYxQW}V2>M-Ih^A4joad&klK-rJ`<^-^b<5joSjMhMqra1N$V9Ya%gOKe~+ulC4a}?}yGrSmP0o+&mr_eW8b;k zyX|dp)o5mIKc1`~i8O8C5-|As<)iv(gX=F=ovdHicM9cue4gs_I-r}g&=p$p3R9fM zc!zK^b%XI_{5BpYQVCYeAKk0ol9eQs#qIm4eolLj=aldO3@h$!2JBG`&3J<%0E5o0FFo?3ro_9KF zQ5Gmd+)wK8m!rQxF>^V@k*m8ue~`^7QQjjk9X^R4yk_R0`^Y=F5k^(s>Yl>9gW9nS zVZUNzV@j(2Q8iqBn4A=8kiX>Z6m7mK+B~4GXE)|ClEOjvf^^xm-SPo~tG&T?K8LT@ zKBR8O|7wb&L4`LjQM|ibRUuWJwcpnC!M&h09fe0u(RpyjrAMAa=3Nb0C8+O~@b_-a zBtkbFATLU%jAVS_VN*-J-nprE8#T*$9h$jY=W>K*4K64oKhTA@aCdL~h?@SHwFG2V zuVy&vnz+D%;tC25+RE@e^r)S5LQ|l*V44)0r)}ti7q<$&)Y3leeQwBlpSj_}n1+pR z#&-3ju;lBu(bYeOyBeQlCibQ{EMIhciwyA`Ir0%o1$&Ldo1`ikBfDW^)D60h1v`t@ z^c(73OJz;fkT+2(Y_hss{e%Y(SiFxV>VPgQt zJkbq&i8PBkTF{G3s-R<;vW01(vy)R@Fs)?Z^A20Eb&J5L?=(^>vHO zW|13CK-*0_b$FI}Sf=FeY9#AA1IPn6+SbxKZGe!kqLO2zg_fT!RrviV({ z-)&O4pdO3URL3jQXMHScT2**(L`9V@UT3^y z`!yz7&B|V1$eaP;B2IpR~ zM{!EcT2=Zd=zKCymbd&p*E@E=+&;ie{UDVEl9yDD{KZWvaiLwFA~5DsIf};na(@w= zhtR_Ogo&Cmm`T(sq(fYP4$+{uR1HR=>)efj*^R23)%ZV%d_=_}xAz(Ot9C1nna~WEdF2jq8 zW=HB4vS@k4>lAKhfB>vBCq5tE78L_zAu*CnSl>kib+viWJ*~ z#!stW9ZGJkfV4ovq3;cJl{q%wCaewWzsk($nBS|q(Q;>zUxRYNoqn38eLkes!>n$V zeTN~<6O}mT?vvzX=_fd}=ZD}PLR96^S8PQ-fD`ljmhl@z6%5U*kRIo!0!2XhOxkqo z6VY)1jCWcuZ9X-vgj2d!{dplxe)6Wvnicw+DW!A&w9jgKVrrWo;(7?jg4c}rs;wE?&+oHtu_QB0-go+L%K7E7GA|G z14qj4i6*km{gE|CX)W!JhEiUkXb-DzushNDhX;pRh^66ex%@ox6z z!%`RT9js{|+2l`56%h4at`bXru=UE(W*o#L^&H1!X@Cp_1tc0)ocgTv1GEWH#N%mO z#H_kR_3`w^XYd(1YsX0n40rVD(LyaOsHQi|X*oESYd$=o;4YuFfvIMW)GJF#G28YH zS5$D1(4GlH){dGQvN9bgZurRdW~5aJsPC1H92xB6+Q)D5$!+*2ElQH;jTw}KO@pFU zQKbT%LO4aSDJ&>Dfk;Y+QL&7w^nbtDD*H0rP?_*Bz2uAgk!i9_DLi9y=#KgwHj|NmqkZdzz3&{nX!B0r%#xOUjps{F zagGD&C5G2nDHA@#=9j1&Pd6}&n7Hxc#jCwoga@_j-TaVOpa=rcLdv`c{XR!4P$KCbqofyoD^1FbX$&Z~06XV=iH3+EU?q_)Ok= zwN2GvD$31Ex*u5eOEFQJ%dg#Np+m2V={rho{g$>5MkiWq8a?aHcKLwb@i*B1Gr^7j zooW0g?x@V-4#Fg`ksLQpVkfVW{8qw;RQ0!XTmtfvd(oBNlsqr??Sp$eV{o862A$Ct z?4#%XD#hc+GJF)h3rRQS_IA8!MtoZoA7SQ%)1|*A*cpx-nxMSpOiDYgLVS&k7ps+@ zceonT9#v1;z)aDkV9pB2h#l8f%8m@6snHSf<(+MA0@JpwRi4jrkn^M({cAhRB#AqT zh5JaiK|+T$%6R5NzpEF<-45(v&tYnEWfgs;-V_?{u_peK*EOzC zkp+|tvLnt-QwIxe8VCC_f17}f!cJ4T$;;IYE=cS_TsVV!+hcGvW3vtQik4o$!YEUf zlQLdr-&20BwQnmm>AK*SU=jeDfa>hgY3gO%lXoha^gu&kEfjA1v}j@2Beb84`w%AX z22HXJeb$xEX1Liy2SbBhcJz7BCsiF*4U;Fa>12~En+$Q6+f~oSuZlm|n%ar6{b-vS zzvx!{IofHB1$WdSLaVrBdX5}z6kIR0uB=z&m@<#*StJSeO?&8y%8*vypNn!C9h+_3 z#Eq$6*1_8}2LXPn`aTrhNzq2pr{Ep6lA81LO`OES}1M>-Hk6D}S zg1Om;33H?8@%>CwH^lw#Z18NleZTFQqNk_;@%=FCGpl_^TZ$bszTBSaH={WqZN2hU zl%0)8om$l~`uWk@ecVn^JNuS{&AS1iuHy8V&=K7mA^|wKc=M23^J4^< zf><4K?63HJifMi{7=~T*??QRJ=M5RTjYg0?V_&_v=53wt4#*nZFnpnH7Vh>jzn~8j z_3gCGSw~uhxrl9Qqn((ZA&a&+VPAc>v2tPe8&~FvK2wM9CHDj<2yF&S% z_at_PM7ztJ^~1Q1A{Qj5Y8d0~My3!iyk_=zF6WvQ*JImR!0_E3SWG z11FCz!0smJ3(eLtv-eXqw#mX$$KxX+V>li+{(m39YX3J0|4qVw>hPZy{HF!~X~F*+ vS|C)Nk%EJRQz0WEs`me-&FCW0KEcJVSC!e-UrWIINw94GJQnFa!jo1(lHQ8lY0r-LL_o+>#P9dMX``?lkBaJ-P;r zp2V1J44(Jy`}6sJ|AFV1=QtiY)Ps7*d7amJzT&(-=xVDn(zDZ3P*5Vu*7S_=L_w&CHvEf`ys<_ksg_RG;F4Nx%9tCBuN3 zoq5`{Yn>e3`@_MsOnSN%Qij(n!oP-D3cbWan~U!8Y7O#B?!3Oppx^n{`Dh`S=|-91 z^U7Nu7dh@9!u_kIz5UfecF8VDSWMu8YQyZEOm$X&e7!i@1Gw$~|MUMN@P9i3SD{Xb z+_(&EfVl}&0y&ahRmLqb1ht%;0C`RB7?#$Aa#JVI>5#C9BGWV3TGZHS_mDV$#iKtUMHizs$1=`Z%+5h{yO}S?K z=cUqqbXNWG4-=g4kKFVVK5>y-#8m(9#s0Fu=VYDLL|2s<=d4KU5!Tgbc&E3c0^7nh zjDmnWy79QPJ;t_TX^9ZUP%0+q=^25!qCrWQ%#TE%GHnY7&mC8suNP zCD~m~cM1GDh*N#(bqc3qpXVL&)hWVOrIl(Qdz=u7c23hN_=yf$H$9@1V;;?wts5S* z?QIq%+>IETiIV5uzN4P1MFz9ngOd4cs%lH1l z`oiFa$8Hukt5KR09d_F#AMSfkiBp5f4{-;%?{Ym{gE!vtXW*b(oq)$}4kF{WN)JbzxMBnOsNdQ555+zITuQvr#t>Sb}=t9=Wb{>Pm>+^_cki^R~xQ20BI+l0y9;|JhUbb+!vB1Jc}YGMCHcxQ&*0%PVg=6p2ct;~jRj<(lR?o`Vc~ z(30gnjGMl;*OLb_+g@+cR>Ebo$y>tWeve;{Ny;`TrD6@=TVEXN?fgr~Vj5{1E91%G+1lkKT z9+mJ+IL}CUOds#tPVAOlF46v$(bXiO%H>z#G}G`U2;Zq>7x4kZ({9Z*(v9C%w0^VI$8v>Bmpx~)zw4fI^%5uKYr%L9b$Qwgh`eU8r5--bmXdzW{ zQkaroATBns)7!CsvDLXg1(9t%Bnc7u@-(m9vAg3r<;VB9?Gi|4GC;S`b;EQx|vwwdf2=jEi@qJQA;&(`n>z z1;9Q1zv3}sTu~u6hZO2zfdfS-@$r!;p4ZfJ3vg14+j}SIsHoGAW~A7UixgqwCa-kd zx~g+NcYcfZveYRyix28NFW1&xV(zPa5Uw%Zbojo1{I>Ard!AIEs13^%Nq>80vEK7Y zL|f2lOM=d^tR_QrdyOYACS2N8_$R0L7b5R`ABY*E)i^zUYXX00#W#FH9UgNT96M|` zaktj_WZy>-2+b4ev-J5L0@S{~#%Sqazf7;ICwg>@auxDBqUoe0CYtJc=@u|tI?C|-3%oj{NGGM7o%4MWd4BS$ai-jps`ef>yNHT~E&X6P8$}ZmD&2@< zX~-Hu#06C?&O>+v4Dx#C--|tu#3HY_)WI|0B){}s)Mm1ebmLu*aIRwM_!t}>!($oj z>slyO%3E!D=ophy)Y^rSMvZ*1q6&BE-?4m3D|9Wkf*;jd|4scI&F;9Jw!T?^{3~{503vr6{nrVG)@gtqGUm; z!4?hA`bsxGkaylK2>hsnvvs&=Mix(RK@IW>KdVVjFnLRhyYD|Mjb40I@QuDME|MN- z4KJsi@5*>2;)Qb6=_LnZ2#vNfh*5)&?>HKIPEpK3y>GrgneF-Dwb@~3yuRJ#Q0*}G zBP^krSn%LQme6^PTT_XCW716X;Q71jY`AJWJMM6X9oB}UduNyRuHR@`f7~?c+yVdU zf`J5kS@sX=2{*?ahz)`E9Iml8JPYPb9}16&jpY8*A$$QU(w%Td$&!vRE*M~xoN%oz zF?vzv8Gwp|l?rgT&0~+B=Iq$Srrq`s6!6$LJ}bc(+S~I7@dI!B`RIdlPIaz=<9s2e z)%YAFJ$N%bhMFR(-ic|n0@64oVsMpuR%H*uAQ|_xp*(#9DE>+t5Uaw!c|E`h*3bOBL)w>81Gda14p+4XH!vJ3O^pbj2(XGX~|5el|5|4eyf#DcSKH+$G zheg(QVJ}Sx@?t6Cjfm-hU)Jjo=*>-u{JQx3vXY5xxpRM@i*B&z?+xwB&d|~}BHz)T)a;mwa#+*A~v9^g! z!a0t=!2Nc48tD*8rSO;=-HDq(i@10FWFsPO=9s0S2Yr8|n^fuRIgDvHuTL(;W_Lp2 zVuTyMKSZNorc%Qe4)6tI#?TzW&L8w(IfGbNy=Ie~Cn98k*VF5ac<@vk49$G}yE*jm zKEIsD31s)WP36dldxJ6rlHS$Fv>NH$jj4`Zq-{b!OhRe;Eb)$UQD+8y@d=gjFRR@9 zP*!=c_I2Mb=fO}cj zvM|wl8u&ocxe6cJq4+cnL*s!p_xwJN3SjvmhpSgF*mT* zZuxVyfHmBK*S)&&x#T6z=RR&5&a*BwmyBO+Z#pIsH__G{v`2J*fukjbbp^PCpEdGv zQ)=YipZ3L0ChEx)*2!yU&#m9fy)3)>nb|<)*CUT!Yq7Tqk@U?F*1AYC z(xw;nIPpAECNRF(LQe2Wr5XHPtO#pwe}1FvuNKmfSv8ui!8DLF9T6VGS!i5hsu$X7 zt85BzCzTZ_59!@i^n?amE0w}tlukj()vLjCM))yS*u8eJf__z0H(?HZ~;klqj+}Q~9e#Z+Se{ zvz$%9IiE!z7PBw*X>zm|3qVUCPEE@(n>@@uHS$crh%1>Rz6(DiVJ zaE9kg%KVN~Ro=yf(6j9vn~7eX()M~yhTTd}dhlyca{ccIx_Vq<|C!rdqD?35nSbqK z$Mbm4wLfCR1d;RR<54lb3cI+}2cVs-JET{mG@vR(fBk(ZP91CpfT3F+>IunH*02(J zgB%I%*XIilM-C6EFM_<1f*kAPySO60#7_R!F68$w=$#R6yAYXm)c43@p?Q4O_Rk8! zAys-LdR1cgW`mk==0RJDMRRJ9-%=>Qwr+i>oSCJ^kmG$K2@{QKC^jifWj`!v{&QJ# zTEYDnLzkhY7o`Q^F!7|u&T5CTze*$PH_a@MboeJyy5#W$v#aoOoaXd!$@4?8@R;$L z>0G0?LB6q@ogvmXYc*G5D>MrpJRq@R9fZSUL*5^l(VFK*b-ep0PcDIrR*RuftIv?k zldqmxgxMO*xZ*74{n?+#pX4d`C*#SFt>KrrPn<2b`yEw1uNH6u1wgiPh(L z#VuWQ+OX8sWTsKCxAC!F`|}V09hTU~Id$!CAll`s6*On|`Xxy_VCH<#3yRnscYg#q=6EnZ?AWp7lYj|$!M|tlc-C| zObxonrO>*;(cE%*sf#29%nckizH?b#HS&M?sZK#hXYkvPbN7sLgCx*X#%_~|Ba>b( zC!2GX7l1Cl@Co@$fgv!KQ>Skxesn#2(zX#mp5u@>j+B#~JVvB6`nrV3wksuzvubn8 z*Y!cP3@1fi(cb#G8qa6J*NAu!46%616|zO>Jq^8)5#8}F`tZZfF(6Wv@9K0v?=a20Fdkd&u|qB|rSu zPCs%>7ksawllfrmdvBb%*T9zi`xK;Fyd}1|Hj0VhVkwc>(K#~$I-E#g$n?m+NO1`~ z`k|)x#!Y$D_oli`SNXAJw)s&d)N)hMMvxVX`4C^~!^PemN)@gVq{Ckq{Cxj9BlKIp z&f}fyl4qmEvR99)R)o0pOFYEZ8JD|puO(I zKrXE@T21dQe#=StgqAs%BN+OokIgMFHqrZXU5x@Bk3pc?xjSI}XAY?fDD+7{L*y6S z2HH*$Dkms!PFH0z5akkZPg z#*=W?=oR2O#rs{u5UIYTyu_SmOI@pZ(ZuQ=pTP=5&KZzQ$|ijU!GYo?V2ZE~ey&qn zy`ufF=`MU!1L<~?TijS%cn5zg%Po>;&W!lJI@;EFy&F``2vlW_r16ge#3J#TTDr%U z>0r%N!A|=G=-ZBWFE#;eJZ2%e7Ao;O+#)vSDL@$LgSz+l@N+NA5`6xn^*EG~e8{jo zg7HupqD5zlXsD>JR530z&=ETeo?pX!GJ8;ZWKI>nMS}70d8kJK*PKlBy$)z{8b4Dl z`^R}07Vqb59b>hUbD>s%^a1lK;7UGs*Wx;S(cx;#^~6kP*kAY=B`hu0FHq~7)!_9* ziYH4+qtQb25@No)V@8Ybi8Pe3Bl#~AV`8TY=eQL|_ER@iAVf)b_*wi!b|*&~-UzYy z!gSc&2Bw9bo8&@JzFSLgGm1Uk741mQ*BW#s8Nds{jvtw_)Os)G6%rTG{<3C|Oef`= z0Dh&RnOn78D&0ldIgi}FKDymIqw^m*3ANI3h;)aso^uCVdAXUg6t2F83BBhWyvIQa z`}66jyh?-`lwv1ENvi5$^>mFCHlQtqw!vKoy{`chP>O-KK#Sd}qQRt7y6Bz+?j#~1 z)<(VY!sq!9svM!&WpQs3n4p5dHg@_#2n;kpv{6umYm^XBUv+JA5m=Wdcc9hH&JgYD z9PU>-zKv_9Ba58+U)CMkzXi=tj-||K-?PVe!7BJ5Oox#N_x#L~ce#&dq#??Ez zOI47w$G8EC^H;zABV4hvWGXjqKmtl{+7&P*L%oHs%vlQ$aNRucu@xItIb#&hH}#?h zi8TS3uBznhxF_Xr&=FPox zkvkrhEp4b=%Nfs$G!*)>9*)J^HTsIBCEM(lBHILARB}_$R>{pD&V=AL^Cq`(iDxKQ zOOj7^ZmHGZ*xWe8@ohH?GgNxh<-^0~^`>|+sK~C^#A&sA?Um{{%r-&QWcQw8sTzLJ zIv8H!T!9z?8deF%s&B%uZ#Cje)55TCCgR$UIsX%@9)bi&{9aO%j!D#29xPV=kw-v* zV=^Yl8bx7AOM5b~MkF*L64!$jvOD zk%|vY=DYYcDAeXV_+t-^4JV@^XZy4Zkwrz@x1$^q0|Hxw zSlc@>C#x8icfS=WK{;|n3iZ@%iLM+I_CeXuP z^aI7sHgNd^ZC`9O+wqUE6~`Z|?Gnz02Jk`RFxFcr`!vUCvrjHU(w*f$Z+e3ZD z*V4GXw+Te?u_-*p#o#Ar;6W4uKkN+Y-?505o9eP#6<+Q;dsJB?8$ z-DrXUDTv(SC3(kkG*jw1#CI)eiREdT{F1RSC<&jgqCqV76C}Cg7MZ62 zuUrG@VG%)_n;olPTRAV6<%m`VaIu~=`|sX_j#c+y*OVa@s4Z_iuFEhiwe1{$rSJhZ z8_m|^oets)YC-1DYD8(`oYiS{d%LxLq}W+OK_Zi~*w>A|7#U)=)3>17lC5rJ!Sm}4 zY81RmI_*|cY{Vsw95T!!RYC0ckz`}R*8s`TAM2FZ{X)%v&}bdJ#iY`ptM`rn4<;kA z^Hsk!9Kza+;`?_Q4$Ny!NmX7@kTvBKCmN8rkE?`87VSYve626-Lf2bxRX!O z?(Q1@QChCY+i>)g!``Ca?UtzNrb*dT9a;>d{&D(Z6l9&ad-LUN#=5_Ky^hDzrKBx^ z`eu`&Mf;YQSyOwKifyOeE<4Uvmt0E53C;IPoz%T`ni0}bfJH{wNj(Rx>{ux)KFxpj zuTuHuV&fE+csO^(N>|F-SI;wymt}W*5^>{qpe=Q5Y}g+^Twd7hX`5Xl)1{EcHzA`J zK@K&}xj8%j^^1$WHI`xoOw7CVPDL^R?wi05iOuHB96zqrPGC+Nd2)VQ(eevh5Ep8c z9k9d1q$B7z=_tS!u6}PeNEP%6G+jtym(qxFrkY z>}vIUh<^FRc3kP+m~^dFJx}tl@R$w9_&>k3J11wELUym0QnAnl4>7zVIt=b2FTa}A zjB+>{3fHUYcU<3YW&RQTXMic-SscpHdp0D>Ay50Ew)bo)9e6bNj1W-c0NI%?kCe3g z36nbak9eVH_>j~Mdw(NQ zd=~y0FhJ^mVS^Vv<0{{H15QbZf|!xT_xBrxPR%tI-C_nG0T#i83zWTCx`=7pqO&NF zD?K}Lv){4Cqk1`Wb&E}At+cuoRz+G7{aoNK>acNo@XY%^Sr)$E{#rhXr?XMCyA!MV zYJ&Fz70ZYws=o>1eOj*)cS`yQ?6qp;sbk$JhgO&_ocQdpktgn}mTg9*HD?Z!;aY1{ zvT{IM)Szo{RE4Xm_iWpjoqGZw)FBbgp*d|aztz#ycsSM}>bQZibx^iEb0fxBO4oQI zi8ROrau~kZYqgCyU|X(AYa<5{Oo-ltSvp~NhUzW>aU?8%GpEkFTXxiRC#vT0tqnwGMRFtohxz_Qe{r^F)ui2!4L%=bvig zNb|T&tNVj}&Th_;KGO|_q>i=cn71Xwz-(Sf0`uK(84Ajb&^1rrQ@yAX2bwvJCFSR# zLWJ2POq|WgqJ3M@8Oc#8#3~pQOHRf!8FDffvSjjsOGf|klUKzF%Upnu_T&XM=nKuA z{(d?|UOI44WBd2`%| zRDL_)^BO+JAtyJTkgE=IeBUtb1;?bON04%C^LveDf6iv=QZ5F$-I}5cuCGu@$h*6f z9-@WUAAmjYlecpy6^q#b$sO=vWBS=@Ke~j+m`GUGZxZ@KpM7WKS514}7kx-%9z zhdSQv%KXkd193~O{XR}&uk)i2T9=XcSRO1qg*~K%1+CYi0l?aP3`tyT%=DP?c3MQN zk?mJNCbD}=u-Tu@xfL?=jt;J`3EJyd#^x@1ygNOWZO;p!Z303B9A$QZ^jF|IQk2iVi@+iGrh5}UVGUbydOsKaUonIo{iI#t)a%3+8V=WM#WMd-Gq>MN*g$z zNJkGgek`}0S3{S(Dr%t|;AWP07G!DT>*^6ONk}9mZBE#x43yt&jjH~63G9?UYtP(} zcMP6hkflFgcqHA@P38+^-08A+eCC$w;M1QcFwySqEbz!faHgiiG_AFTiQ{ZCxY(ey zKvb-25zDgrm(k#{GZ2!h0`*yRLhRs00xK;DDL<)U8L0PLu=8$0kQrC#pWAlIabz*| zG;qG<%7nB-mi`L9YQ2A|K1b;6ROK&E70!m9>o4U+q^0uk)&8?$Hjf5XpTaM*E&z{{ z7+Z0+om{!Drch6C-epBQcrF6ij!XjXAdK8gCrV2_Hkkn;|C*_x<{{6aUNjBJikKN5 zC8i4bCF>z5R?k>3NgFuiXK%rhw%qsZAGE(s5#{g*83HhKdw*RKw)Mtb-E*m(|9O4Q zKG{LEwe!M5RNo8Dr2}H>^R2QmZGEZZhK@_RKSCJ;=RVWAk7WX5CgM4CQ zz0{}9$Q0pf!yOw^G0M3yP9qgys?7Zz`PnCFFnu`>rtZy|((M>P{Fmj*VR|t5v#j7k zj*jvbGq`>0&+57taTK#a#)%fUOX=XUHnbt;Bw|1Be=u{^)5VroNrmn?GN!x4S3W-g zYi#QucYxngHZ|?|epUuRBPRem8#lY2^!+J7I=YSwzfU?OK3O^5ZW6Ft;tl`1tXn_m zcA&efd19{9L>Xcd$z!ta?>`^V$bS5TB6nADEEgOC6}S)+dB@oy!C0VPx)19e1=umy z?ZhJ9{QhjU<|a$}dAnpSoJgC+zL$rNzjtU!&`tOyz4P%c02QlV#aO{RIqvu0b2e+| z|3Z8QaBzTyyWm=LRLZzu4=w<*#AJAX?YhmL+S`uKSMRycX)fJ+&DJ0^TKn@W{&(6* zQ)5#JtXepn;l^8823YB0fueiX&K`;P2bI@_LkrzgW!s?anI8dHJ5{6E z_mN6{WYhcTNCp7ZJC@C_1nXH-a-=uCD9Asck!JCi9}6J8L<%Gbx^rD`rUB_QM?Weu z=}xm6@b?++bL3%tH3s+{B1?ugVpW*ow^xpc!F1}jPL)p*0NZIBkVhhG68Z~T%E@yr zWcPdDED+kZi|*Zu@YXh%#wwb$aO<7yODww}DXCC0+E&83&oluyBOv`NB7Gx^zX06`a zpb~&#m3Za9fD~$F|K6+aOlaIWt&W9d5qvZV5~%9A&Ocg8~NC*)@fuejUGS*_ZA%IoD9ng+*;R}+uLF9_VTm& zh3Jl?H9jW;mf^?a{cZkhu_KR}Thu!4UWnd7b0Cj|nLesry`u;#?ridNa@P#?`}v85 zHP;Osiap|3ke?(<@)b$r2mH1^-lQ!8tUN$1400_=cS5E(#3U%5&sYB9CaV0KdYncT zTfURd7bqxU0W=^US-F@psms5Kp6lMmT8SO}-613u&&%LFsrpvQJ`&bm0r~W{o6eH5 zS?~|y&#aY;FV>JuiY))}EzG2v>`-1d?4p~Qc73X7j40>wYu_8L<}ZbgxHs@(aZ@WF zuqE=DH}68A=<5vul3cO;>54 z>@6q01}u^WQ9HoK>?B3xa1ZkO-(Zgsh04h^VE}E8(2@UBP@q`QEH|mZHOUo92QKp1 zDzCMX^M@dzq;|M~-cpj`4NS+o*Nm{5y;%hglaO2_yi`?1p(=y?d@8v8r1X+CC0gc& zUl7B7|NGF+UT_9A1tkjr3l5(KfXsfbUbB4OlyPx-4|FIU3_A?T_9d|?FY)?iDm}^D z+9phg$SIpK#*eT?OGrakN0f&-+NAV=#lMzF_^^ z9@8~$jpo$>?+KyZ>wm-1_12QbpBM^C6e07>qsNQT!%A}*ha`ese^cIN@JxOO$YMd&MRuIjo~I+6Lv( zpJZb~30pC5ebeML>7z&qn~HZyFNc^{@xb_j>ekuU@w_h3%p{v)wpaPp1Lc-K)+Q!x z2(9>zvH1DVrNhhVAZMLR;r&9E!Gol~$Il1wg4kt#)!%~1`C~x z_b;Bzj#o7%Pjskq>k_f3XN-PV=o@yeo7;>Mv$fl0;pm8#PI7ON=^GVw;=i1?rlW z9sEq)h~<`owQd9%>UnZZR}qXP+^y_D>OelnOoaC{tUo`-`ZD?isK9$tj2$+EG*R{T zMZ>bcP@z9U7KED+W4<+Pm%uJoQ?QCPu=gzX=_PQBL3-%xz2ve&uQX|=Tt8OVka&dq z0451ftV859$!s{~ZhHavBGlEb+RDV40WX{+;mw|LJ5tUq{iJ1g;PFBw1u#hRjCIHuG4w2j4 z!TL9IS)fYAb)3uH8qe<6hN1;J6&GuFHGKC^EDiB7Lkyd^)jLN&+%QF@Diwe*9WsmoEz zibIFsl<{qjQ{ouqzYUSUkyry~DOt2{U1igg-)-EOWT**TBKt5NR^oU6jq|=~dH%}@ zMsPH+5n-l?<33TSV{>A*9Z65g2 zT|+VR=Qs1LHysvR_;cm(px zv-@eY3R(bLYc<_0=*?jWN`{LTAS!K^RxD6Q8{EICy-0m0Tj{YEAkcauvE1-20XKdS zXwVnRXvVPm`>DmJbQKpQGBZ{>on@#;P~4>2{SsREWcoulPw4`5+m4du7zPX$DE?)A zl`RKP@4Oog4eo0IgpOip+_w0^;$-J%i z$o#QfOz+~(Z_V3x`GPN3^tN5(9joz`z}IV}5obG2HP_&h1d76c4<6^L2lE^_FF+1`x4u~U&j4dr z&8#e%NMnn3Q?t`*KUbZ5QXL-|E*vxrZkt3N$%wsBTj#eHg*q8}Cky;jyqGMrPW|4q zy;uAOkhp&aE`TWPWd;w_7Tr-xMI-a3@c$ac<|i3UT+K6b9*KJL)@4?oQwMV1lp)oC zY98A2&#VB5mXczHt%2hB3Ygwlr*Ov%3JUfw%{o0(Ag;}qyv+ASwgwJ&1Io6oRw_$Q zP|HB}yPsCUZGS+Dswj*lJ6leNXx<%q4hO&^=B4+TUC<8;2Liy`0Yt+~oFq-GZz zJZ>3O_i|;*C>h7GOFrD1$${n1oxSI75K32qKKoSmX$)5*dorqcj@vIPC%GN|tl%eC z7Q?^8`xA0mM$7LMoDpmXV8I7Da}KeS76(KA!~>H%$n!(Lp^WSefbrXylv;~_wK3tK z1(hpcB;zk6NN;&e>>pk3@?2emfnRE3)|<0AAGDpiykEpT#?DnwgwsAcBsM=6_l{%w zt$wY+g{Sh~mvh9OCv<_f{UtEd@{0iGr@ZC3ijme?NZuy+>*Q=><*#LRTd-FRdi&Jd zEb+1oYv|BU?q*fT3KCaWdt_mCr}NEz8k;jG^DJFhyQ8OpwMBS;T2KWL93m#16JiA z*OdFeI_&Du9zm&p`UZCNitiyw1tYMvM*NNSCIbJwmQ6J$j+v_=(rd?Q%~>U6$7!r% zt-v|B#3n&O+4a=LY#+JGdY&`uE~(74=(t!+`i?gw|YpDG!Sz-K*BrMckkOY9yHwAdeJxCe$n1KH@6_m9T96!1OR{ zLl1bO)_FomJnqR}!}8YpcF;}QdtSUo_1e2(NIw5-zzmV-(uWPji;ElvmW@2y&Sek8 z91a4u7uw$elZ8i@9$7}Vs5UYTTZege74MEGCn#>=JvkIConkL1337x$LH^W3@Khsv zqVI$CcOl9dE<}x`2({3h%`)>8%*-8<_bb0HEhP|j&PHRygl;j9y@ByW#vw4>hniQh z$WsOU1xJ3d_Ozv>=zzS zH0V9Gxo-|_Z(k#njstDi>J%@65;Bx1F%cO`NV}y7lxDH>m!BQV|mt z<9nY3hW`_#Pe0`FeCu3MnN;)gTyem-Ys2PZW3B78hMtQjV>9qFnr7Nh&EK+Y`yLs6 zubVDc=d1PlJ}#yR8)5$Rz({OQCWVRl;%s~6G0Eb3!$4++UE~WfbAhbq8xC5hJptHY z$YnV^ZvMQv_*d_?PNsr^(R~x+V1Ht}?ns4FXzkkJC&AiwYS7Pql(+ZcCC@< z3&0Sh=l_g-vO3@Ay$l)ToB~G)!2<}Qc#(J02!mu?tJ=~5zHAaOFMmk;#jQ&S%lg&6 z0%q_A`-(ojG?*|hHJ@m>&`lIi_cM;?39V1&fONH;7kI`G=X-ZEx6cAS7QN23Xro05 zyGIF&Fse3Oj#fF!qX6(Gg}R;pWJfCDPC!D|B^Fy*WK5?sU{4LX$n(d~$d*S51B7ZZ zK#vRZrZj9U@>5#_6u>tTZhZdzCz)MJlo#k;dntr0HhdJh9dCdd=od#Oe#l)FUCvRH6g7M`l z=UJH}!W$PE0QZa6qjFZ%KK_t;9vbQ%Dcp!CKd8?7mUKcoO+?}E28uv68}IIgYdr2o zjIVGip5!FBcVK1_H!Hc`Y29fuWN`J?x9 zywL$}1{@%OU7~`WP9RGM244I(9;!OGC8y<>xUa=vqzWEn{`sXH7za)DQ4d(Evfuy9 zW;fx+Vd49lR!tFpfj8k0k?kHCVvtrN5{E9(nfNYz+0ec1H1-=M%)-6F9M^`8 zub}!X5PTjLd@*#+ghxb-k|90k2eH~fTSmz)F0g)}PR!Nho&@y+kxpbV_FCaJ!VB4b?7fJ2vW>EeqKNIV~VlT#^ykC!OyScn= zKVYO!en|o>mm{peJ^^$_+?BE}v@AX}N~^d#cbA!pPrW z1nbFh`86bj39p(A7XoF&F>>iQFMO2ih9QEFSDViF%NRGf1)cTyiNT}TwF}t&owmRm z=$j4CfZ}1d#JpcwI!MW4maO9Tk-lyyZoF1rM7+T0@XRtEYxn5zI1bJVg<^0l;{X}B zcSdp+k+X}9*gtCp*crQX`_+a>#4{iWPSrEDrvX3^p77R=Pw1IwWl%n8uq|N^Wp0@_ z{aYG^YfsDe!8BH_N}OeQswh*zzQQF-t|che%`sgLFNjNOW+0KQKwVMZ%>2aLn44Zv zS(cQrZlb(xjZMRaE^i%$0>lnPZ@614=V4zO)g0cTx&6E^Eq;02&o!=@I{&hBsM-fX zPJKs!YF_>DLk@K>7dm;KFF|pKs&LF~L{MXXUu-$=+sa^ui zEALk;Od*SU?%~RM%zoM-l7^j z;2dX>Lxe*ylj^2`>=-||RVJ=uBxN=B^YJo0USierEkR)e?=Mz6(ds|PdEnhUZu9BB z@lo?#wXc*gJ#348)~&vyH_OzlUft+3qJ-#-)G#_|-nZ1V{xcTo)>Kz0%F^f`mjah6 z*VrWSdnh#=YkF{(6NjMr=bFb7S19tLT+>^fHxh-XbM%A}?etz^ZffZY(o$$M8y^j~ zuF;iqlkUgB*!a{$Uk5)K${?1#%!$eUn)o#fMHkc35|dJUYJoOID0UDQJ2D|QU#3~U zE=zA-g1ap|v5tE%zAd7#i_ZeYxyR(GQKN<}vwSLxAmGp}#hq^vLJqXZPC5L-S2{$R zmWHVu>BldH(pV}2AYzK$cv9~@qz?lR*ACF>7fJA8pQ3F`jy@ECLll39**k4!sdp;& z13jzOLq)p-6STi65c{x0+aS7^=+tD57V|$kK{$`*dFF~;cn_xdn2d4|qxlm$SK!-E zTF|;Dki8~sp7Da*fPahZH5O#D2*V2{D5lzwqhq66Hhnj8I0EOIqrI{m^812`65cPp zNed7d{+T9}fu8WQ zq@1)y3RQK*#YQG3j{b9W?n%bJ5Mh%b#BCtIac#V|%jfl&NT?)JcuZEFRnRLb^yTjd zH+zp#!Lf=f6_Z^v!VB9Nc?P zXMMNFjUTXR)&j?}Bz$_jyV|t+*$lpt6gbaPW!8&+8kN4aNTGX}HwntlXCE(j&_t~T z(-yR-N=tv*Y=5EHHJi%p(SzZ2W!qwZ+}i}yj!Q;LZYom98S}jmHh#4jii+y5V9baN z=fdmwj0MBT-_!M3+;PDiuMqx!zGSQwv21sAmi(6&0&OMFt%2~u_q0-#t=p&V0xZ@e zRq~@AD7=6@S(5X3=Fkoz(4EQRC)>^XeAoT@$GJ=6MXvz#0c6E3qfMMSO`1R_N8Mlk z#t9SKJ44>tZ>%1j;^5N~h?@HziN)D~%%~5pqA1YxULpnmq`}41hvKB`MKLv5tK;P+ zCPoOu-ao;c2bM8U_q3DYy_L|BQ!AaT`OO_R@N9_hHTG@n6L@-9|UV0bNxtOT6G2Z{S?KQCNChy= zBU%dC2Yr5ilS;qbylwldz7Cve>2H!YfvH>U-@i|@l)mvXf|>tIDklm6bl}40D&+b= zYNKcWbJ@f_*9MzyLRf>Bd$-rQ!ppa)PWCF<0cElG$LH?_2}zE>ZX)}{wGn;6ATt-sy-QiLcFG;H z>pOtLs6*d=V*5eVRk4TFOXv&DJlm&0% zpkO08X$r{4i*(L+&OoW*k13wbTdy^^rQZ4>H^~8y4xP-ALm37_-!-y{^e%q;cc9ZF zAx=ozuhwyCMK;KivV^}{z`Ppq`qJ2PpQu9xj+XA9x3QJ;JaxAWW2y|vMv7+};Vxxx z9hi%a@~2b6@=92#L6wMlt3c!Vz->mQ`ozGB1eukJqZw;TSWW#awQ?uI%9?R*!cc!h z{LKq5O=Mp7{ys92iS6kRP&zMCE@|yP<#a%ej>!UN0xRvN<;e09If6yK7v&B?zp^WH z4wT?oR*nn#t&@+H_yPS%bi2zrs0d{hP%`)N>7-z>igE=YbRzTfS=E=q^@RRJxyVRUI zZ2r2f496C_>D_u!!jA|SOkgp=ib@Ns_K#Ez99f{~_^iXeh?|q%&EVNa5%|X%+ZB{Z zs_Qrq=eCQ>e{NRmkN5BTS|(5{J!CrUd(tA<#W_=@()Sa;n19*L^D{gC`|KteS2$|n z2SqZ+4mERuk{Nf|;_?J(Txz|hW|rJs;3e3Jv;|Ql?%TeVHyA}cizf>x|DK9&(?ABk zS8$vw)cJTmI>yV%bcj0Cj!E9m&<%-r0X(XH$+}Lze(PS-L|SwyuvNGA&gX442Wv4H ztxa-%IXaVZcN6l|fZ%9$7!9mvDY{1pwYlJXVw>@eD{zQPpFjerWV%5?sD4n8Dw%Tr zTU7?xVo;bS1u3eJ-RS)cf^XZyf=+;>R{WtR>o$PCqx4zf@aduQx`#PcSSz70wshc$ zDWRx0=(_i=A*6l1#<4jLpP4<9e}w&4mL%s$jiR2PyxV(W>)z{hm5|mSE~w&l%W%aZ zGNy$8vwncHdhe-94J+SO+(j+6O=X~tGI;+wQ1Bj1MTv_C8@_mpUN;#;PZgg*0BZpF zATL#3Wdz;c&m%fCPuB;p_`|Y4Sw&4z=>jEhHU3>E;G)0%J>Ba9F@MBsXeeYqa@e;x zpKrEpT8C=lb!A$xrvpk+Jx9$4IsOjKGk<2^|IY%b**p*2lV0Z?z;>Z=b#}*yFI2D) zzNKD29;2222?U+k?$;6P^+x|SAUDGR%=o5RjOSZn!kT@$6!O$K4ZWA6(mj~oMI7>M zWqI=tt^YU1S~u1k+$2R-N&0E??Rs)n2Jp{T?rrN;Vgf~noL5-jKA}N^z~Ozjvgo%$ zGklHP$qZ)<;C1Pev6y^Gv%k&opWqo^{!rCi3VSyDY12h71!IoQW|ve%26{EM;3I@k zf;pRGVCc?(7dPu0S&7+^8sLCF+-jgwqk_;-U6_5-@Qd({`N4ER8`5EviX2EKwT`bk zY5oD)Y2t_&)OL{S`1Q4hy?fpfGK>2@`*io8>wR~ApL zRgOxz=8Yce0OPS@T+h?5jW@AK$8TKa~G;)7vYwr8;>&-!NaT=gkecm5F(HCB+WG!}ob&^&il@AQ1D zF`g(5q0{%~3LUIt*9e}f>tctG$aQ}|iUgkL3B_~iq3VI+TW9xre7M=`QG?!!uX2hK z)(wDjnj||!@Mq~yQ|u%ZJrLzzI8_87@&{mN_)DV}^!T$Ep!W6&FPa$J9#Z`dK2ZTK zM?n$YTl0vJx9=?NIPSlVOF(Ow4EtYnJIE??$oeU8f#QtKKw-V{A{NVWzlR z)b7)2e+)~0!$|Sr_gIfYZQd1tE-Cu9`}nfgzjc_;aKW#Ih?B2E(fQB8Mg!$jet*H6 z_Kn&J80T$Y3zZBd9U3?_#UB`a&Bg9$Oe|CfvHjSb`bo=f5BsoYf%qd;-&6CnT{%9> zS>?rts|SFGYtwwgaPyS`i_U#lbG2N-oeAbSLv$wrsK zKca?wk?3R!igWrZaG~16Zz6tYZQ}ibw}dHgAwqS5(#=(MJ*_(?##X-A$L$JT+u)L5 zj63Z=Pt|^{3xEbk>S%49jYLnQQytTJjn!HPFH5NdM~Ww5ZDL4?#f?IoxkU|4Si`ZR zG(z^soFAI`s(*^{X^@cy=J>bf&+RkSnY@R{Q0m32|2!R3V5AWty8g9rY=ty)@!DKe zdaeb%-LjqnBgMT^pmpmXYHxALs>*JU@=~niMdYaLupK_uGwB*TsP=KcK7TiP5RF2e zM3b*YaVGm}U})oY){|{Prvq_XuB5PQ##r^!<`8xYU1M@0$*`W!alQ)>0p69HjO9?d zW@3ZrU#&p@uOGfIHZwdu0@f%kx;NX%Bm7T>{J}nxLA8#U?ua8vYAx1_n?A|fiEbeO5Z7 z#IeR0eVTv5%PAnvZv84OF?arUyx8j3+WCYseUbb`1z2NDu2OPn$F=CQ`sJ)slA-CE z?I!rRAnSJ5AwinxZwjv*Q?chf+k1PaYYGRF7r)nYuOw6yo`h7(A@{-7=lPLWgckA= z#bAv!m=;%;9$Kd$#pegdH=FO1>i6ADcJ9E@-^S|Oc?e^5#5Ff zMXak}JA+;p{fzl}r{ZoHU&}BIRUo*s(%%^Z6jo>W;{?ixI}+%zTG*F;ArI$4b@?}G zS0f>7>+atRzDkIgw#9Jv=5xD-xDKea%T_KdkIKv@H1Y3-c~e>pu=?XnNFXqST#;4W zqM>cP58QMIz@4$Jl1t0o9+I$a&0)p##x#W;>fQpBkxC4NR|g*@00s0PW}#86rLB)t zdMstZZYV+WGh3atJ)q%F`$A!aJyl3==@I`PbR|QukEYU8TN;3z7%;#tKZD6e4(u#= zY&kzRS!hz_-ghOULxR-#vFd9FW;KJ=_Q39~_Io@Ebax;L=+WJ{{B0%QqpwN2q~OBE zZ*7(DwJum|rh_@qIrgm6=k2Pu+N0aaJZ>UP z!>PfaNwB19ZjQuHNZ-6uK10#i-gx|nyz}jCZl7xc7bn`=xx$Z>b62rP0(~1;gqWuw z!LA+EOWmwFTVTft;(_acW+E-1p;w{f=X7 zZdi^_tG2+wKri|QQmh|M`!S3u#HrT_GohSaeFwbPSk0u!6TM);nfFshPR?!EBXvN@ za=$Y=joHPf=&b?oPU*Qg#Vyz7Y(2`mvjgXo&Nq*>#oc>8A{5;@>_5u3+A7J*5v}CdjUphW*I{J)D&Ti;Rs zc8Yy0Y;iXr?9&W4PMB_X5vY{K@kuj-OzKCTq7Ahqfx`xdhUNXHqN~3NZQp zjLB!C#957?!za(mVw|)1~Q_pZ5T;CEQId;pd+? zN!t1~3-eXm&J-d%H1OSbLqY_{7B$^tf&0*2x=Tt7koiFG);` zX7G(Zio(mj)bd|fZ<<-&qG=yf%JMyE*O10_uGpVgV`V6glTBG;= z)r@p@^Vp+<(&mazvWqswQJ0WMVBaj{$#fv#6EeMc7lbI+A%}1>u-X$`;U5JO5wGRn z4jfkTLLmB@BIB6CZ;AF=Xm`SsKrquTNKJbc@gsrO>K3NP2wSme?CwNSlE;}%u0Azt zdIS8#l}G1W9jaukgHI5H4@4llFL!g?u3iwO`&Gc}m9%5pPj|F@mEwH9`FQUs_^=t; zRORK*3!NhIeoIQbd=JaZne3fp0!A@u7*b7L7R&*?Ey4D-NWl^jomQ632gg|j@7F-(5G%Xhm>%FFd)?=du}Uu zrSC~#yJf+C9X3p{$xKGO?^!z2)uWVsoW-9s<$)GVrIUx66RGy2?m=!n3s@q280Q z)_GA!k^F8%6LNa7PhR&_s8-R;^?rVsdHhXiAzu@C$ns;-)qSq9%_G1LUrn#MB6wL+ zYRPifUATffySXRU=sZZt?+3yFn1U6%L1w46dryc-lqYNEYH;}CO>1ubmf02mWb15m zP8ivudOnu^#&n%cpHE(W#-7t1nqBaR1#7vMQ{JDZVTZmElHiXPRb9-2>c2V3-N?~V z4z-=V4`YU?0TGul#-$7|-(9PWyt-gsP3sN38OvY%Y8_%vZPS3uG=&K_Vnb1ctrbBL z0_B8Nc?Q~^rWJ*+d#zPuut%5455@s4^|(vJzaPv0L+&Jh0LZ0=oVT~$SBt=>nWf*U z9k?M$mG0HmB>U+H$w~o)2)F~>L`Lol0hht0D)#ctDCC5da*j>`2+;OAt#n=6Z4WK! z8)VB)h|X^->w5BK7PVq92#-zCf40LZPsdS~Sj3un?q^s%5=sp{`j*tDopXt^uF6LT6ptwmIdHEUMqdh+VsH}4|8bP= zeWz(7l3CByLvX>&8Q8W@66I|U@W|OOHR3sQ9P_!#IgQzQhKP;}?s9xEC`k9c${(0X zIJ&qx9pA(5I@$~&q|;mT=(z;&4k8Ho5aP|~m;KVN9_9(Xy3tzx z!7uNd-`1r*7?zZ49tvlLY^CqjjEtxg7UQ(_;QRiMn-Q0HA_$9LtjH6-&gV3CvlDe^ z)%`bgJ3j(5WE34Ob8&V8g_R+eRg!1b1!}b%>Uehrv53`;dtJW ziHR8qOK`3ciMgw`>bCnY&BE=GXmip|u&y*D7lHi3UjjAdBr6P?y)QTu-Ahl)+V;94 zu=~c`Hz%${>f6wc!);QgHnMtrg3a9%?cMU+Ae#c8_tPtz<5wXfwfA!xjX)G+8eOh(=P2Y#4*emj z-QfE&n@LPQ*l*V{41}w$#0`3`w#?z6_`8*#;j|5Pel*B-dQbmYk6$kjY=3b)e)g34 z{*JmQXArkJLe-tdmfK;Q?0P$CVNV>h@VK~ICw|yV&NG(3E-+8ntFV)`<+Au*FuiqzyYnyZ@=HOK`(K5Jel8R|~ zj{mVXu1(Hp`q>2n!o`+dceURPv5QADJ@KiA;L6f{?*}fpQ;?as5;nOOiKZ@h2vlp}?J+Y2GJ|wU$Shcg% zz8=J)D`wI14$GfckJ|AVEzVEhROO4#*A$4Dh15kA_Evto{@Gm`$>riaf;;_wyR#$F?oJJ(MiI1`FcbI;c{ISu1EWMon4gMTmAd#Oi zJvwQ<4(PK%=P9m7BYs@fqr2PdeY)8_qOJ?w#xYxd>kg{x;$nsQoNqei1nOH!d19@d z!A`>27-i(V#VY!IL&E1idT+U0RzDkC$D7BbiC((*Ay7`6OiJET#-giB3~j(G(~eD| zY_C#wVUFeK7Qbc63p+t+N~`EFndSH+F*-kV%uNts>p2p88p>R8IAc+UYgoAb*4bWU zi?gKHOGa>I-1KCM36I;{85Cs}+zggC6e6JdH>>)Y`=@%svy=s2*CgiHoFFaMT*rP0`(U}(F4HF~l&$;gjhci7D1mdPO zFw( zs!lY7+9rGv(Dqmy%G!n127NHVL`f>CTA?Elg5O%MBh{`fkwP_w*!gDvYT?JI=1wNj5DO8*EDgbMR9Sc>~|(fjUQL_YLI@p6So4p;7vuO$rHyx z*miIBii_Ii1cqJDAq`R+Mhml%=OT~smjM`}BQQt~QpU!N6L3W$YAD0Q&%yCL{0D93 z*XU=s-uwBBGbhbtNeb$7Iites75Qq;{mj24b*G7=M<(4pVn*Vcn^~mJgQT|0$pJ$n z=C>;nyLnbh80XAW#*A~Q#7_|bze*D#_?W4lU6ZjEA9^Kw&Tv6jyNh-r?#4D*u!SHI-Y>F58X~e9Ws~N9g7(nRaKf4Ma6@^=&^14EbE*vPKqX{$x)<5URhh-nB4Buh;sMOtGoWQk1js(#ZVSan9cSU zR^%%PU!7aecoJwbo70Xlr9z6HtS}=rF$&+2vqS05aj522{LQYT^5BrLbX|D>9J#KW z4&rxEmN!keOypI|m+fW0rbB>Byq`O&nj<`5{*F%y79rns8x*5h`AdXiF$f*{ z(Av*lb2ew5kBpCA>&nd1?!Y-YoMJ2!D1jf0zVoe@<(Yv3L_pp9 zBB@Co{5av!Wl#+~9~-5UeqN|K6G*j#^giW*EYGNiJY4)FclHX;h4-$ssQb&?#6&YL z_~G>ambpvQPRK*w$>s!9%MV>#_EpAb+ui-wUt>R?twKLm(S#V@ov9d~1W=9&QUWf*AI?~Q5b z1UZPLPW+R-xoFUAP6&^D(GijX(&!F#?6So{z@622juc>X2R9SDnKj5)(&Ra2ELmh% zI(H?QQsbW(xt(aqh+?IM1(|-F-GgOVryZB&FEOEQ8VQcS5w)^Q-7wfNpb9) z{&+vod)9H;otx#CXLJtbtf5F*7&?$sL_*@X+M&;=J+oaxN`;q@ zt7^;`k4HbU?G}17uWRwKhCed~h(kZ_rXHbpO{P z^6ojQ(!I1?6^d@o8kVpnX?cA5nL+M$JV{t+64U)e8Bs@BD}kUV9Swf$YUF*mGNfZ} zzjdte$IX<&wmzBr7Y^U{bQDbx%b3)=#8h;yw~n|`q!`O~I^jFSxfUB)yvkwOtt-om zLVk-;BOT2K1U$SVhD7p(g@*g6h0ap2bBZ}jWxkpao>P1FD11_aB@t8z+Vo)b%FtS5 ztWp+)GgZDLRb6U}4k&tbKo#MK>zxp@NlGVmT zd)bIv5g#{9W0^(`uCKwP`D3%hdXkMLN9UL#BzU+xoD-|m>;PFt(eVwr&6VxV@}greS4^awY4zCMTIbt4EBofQy@x(vE0_gt{fCrz`w>4$ zYXTG59SBfH-GynrIzqWz!q!#xjOjGe{g%Ha==G4IHljqy%8^ct^4V3QmavwhDg~bkflaasCb2P$y6l_0c zc*h&lC#muS;Y&044f*_c0tYw}a{~@8e+HemPq<%lDGV48Sx$M4Iztq^6 zK;a3RZ0n-X52Vv_HL0*)Sz%~7OVO}7%S zh5U!8sE&Pf)wtLA@FoCpo((=M6@`Dl*;2kO5+9OK{-j!1l7Hl@q)gU`kjntLP2z&e zja#yPS7mH-d$2seils!rZA`lzD-%|*mnSXbp;v=JwHbTfu2W>-H|&xr#LA29EtM(^ zS-KSw*q!{%-0d-^pVpo?RvT~TMzJoJ-8hE8=}*rzf&g6!M9*dD#f5^2TX%}SS+J@N z{-o&Nsru!j$^~Jds|W9Ut}d>ii`Rgb;iBpBlpTN7xN*9DL4%z7_0y4}aw&{UvaH0W z;zhYO_lF{NehmlgXiNP6#3<7r|uQa0u29ol$*C3at8PhU`4y}ZLSh*R)be%>bjfHyW95=R(UP9+N|J zKdGyDgL=8gr_80++?L8?%qZVYo&u@ic(T}gUZ7viIUw~YPtKr|Ht-WT)$u_9CnCwS zam-XgDhi3Sg8;ZL;90K`e|y%uT{ly-MO4NLT8dQ*6K?LC>TLx#B65uZe&t6{B+Ogd4?kN^3>M`_sNnJ2QLeU+!F zUN45UpJT=DVuew91e7t>V|kIx;Fddj_P>AL9{nqbt&t0DHyVASq{3UR{Wyy zI5;F0C#8-rE2lg8Kmo22Wypez&E6W8-HvcFG$%|H#gAthZjj?roBP}s6q<@GLfTZU zyWqoxI|+d=?}OXg9p~F{+x6A69yi%~pX!_q2eL_V@9BifYb6uUujG1$IXD#8g^o#d z9&In2!a`L_8>Ti>Cdi}T9uTS@hLSj^ClEPmS$N3?Jp5JbkRAJvigApS?WK%@MMNIF zK>#0%FfbHW+&QQ(-LA6YHwUMT3Nx!judozkO`*43O+^~TqdI%B-j zK8DJM)U|IO@EQN@8*r8rzm5L$IluGFS8B@2`&cXo}jiH)I4j-hJnlb+&XP=gob6x4?WUjs&s1i+&p(J#6JzzOx0B`Jb6M2GqLW zInSX1NOt8*1af8USDEEuUZBw77F6Q;wp6FnK@r8QS@py01>dd~cp<_t5PB`kN^6x) zuk&NQ8Md+$`#rubBJXnJeOMq)F0!PY17b)dBvTD=@bpoRb;zX2HI{zh6n4FXqbgS;-w1P`0nHK4PHI`^35zEsUqiwuq6I z2CTch8u*z`bKXVU?434tbYQtF=y9@-;$b?>L82(PNT*$rZtfEaj*^fene9L@Pdzqg z-(9x9Nq9$wQZcpGCDqSqw*Jka2R6$Q%fJLpa0?NrbY+SxljKW6H=lU^25%-)Sv zfjggwFr;(l2(Gp*45hwUyx{9ZHyy`bq}?@6BE;(|N7Y} z)b7N6vu)SnRs})YLoI~pA(7Cty_frgT-%~nVe^S*qo!|9CvXc_1`O|gAT`iqg%i{V z7HT2)04n-eE%Ps_v1zF8hHVCbUc`WKNxAemn>EU&;jIuJ!|j>jT%y{tglVkbJ(=>4 z;9l-Fs|Id<-Oge43@ZC=C34keDk^j&gJ^Ads$H%Pb7waDD;BnRnxE>iKPGa;f3xW! z+w|IO^};mkr0HlPDPOwP6YqSpe7VNd-EKqTl^&67kFL`ZzO}GlY~fsAFVT%HcmVFX zef}(bofUTX&NJh3p}xYe=A~^5GF6>UmN!qALq!+jAyP{Qr(V8JJT;~B6>cakaZOk; zkG(^j+uR&KgYT$K>TKQ;~L~mIE#>N?3rV+Ul({cI5C9GKO zm1>`}G767vXe6iX4bsZaov6}un$qKQXDXRnUEf?EE-XA>;&W7w+4qeX9otLLx`%yn zpTV=(0tX%6dEUDp>dpM~y-}{S1lGs;jCSQx(r1lc8rJ0H5g82iojEy8s|8V8{i{ce z_cS&XMAO9*3x5O_k^%;Z@wr?cipQnU;+Q`3eDoJLV;u3YY~*4dn8IOtkOASa=J)Lj zD`}4r+4YwB>AlT79`w;$;%nyg+vh3>OlwGwkg8F1VU$N$TqVK#5x1x6{^jL?vyldL z7BVo_CAP{#HBhCEtxKb<{YJa`*1oW;>2eTirx&4BO!dn3$`^l=kid%Ie2~#7xPP5NQr7+y#ERRYPT3=03*u@lD z;%Hz~+go@?kCGm>b6vy7uKECDr~`Dcj77Q&7bS^&&Xx=2(5BNaOIj4I7NF zyHW8q8#L8$(>qr9_Oca;GwH|_x-HbAV=^(6(>O?Wz4Nga9ZYKqq^cWawCv?P1wih&O1`t%P*o0Q=E0og#i*@--xlzLI)g6bt`pq8Km{j zP}#eV_%^3S6;Sj0#(m*zE}^+r^Zl&!r-(h5uXD9Hj6E5f5ye}UX4cMkx|aFd3grD% z7u;^8yed8Kd;SgRic)uvz=gpLc8P)6O?+|FD?`PXk?fw2b$G z6XXQ&m@@O1%WiEr?eb-vI$_vcCUihYq+=T>*qt1XoS3uPKzBSRE0pHSKv7nOmq)XK zwx=HJjbpjs=Drc^^qaJ&--|(m-J%}pcB_+5PDu)EA_c;CyG%tlU&v8AP#{5l{Tf|x zsq^~464_;Fw)t0l()rei{;hAXE|Zoex#0V3l-1P-T;kPAH*wR!_yqZp;nIA9bIlZz z+#f&ORenriC=WabG)DlfUwD=gUqKge&E&~cnd3tAqpsr@48A9xzi5;FtnA`Nxe-fU zp9-nAEGum0y^)8X4hq#9TfaJ`e}0=!oKu(PjvFl#08fML8@Q|z1X`hNP|tzOzC791 zBVPpMglkg2bTj%G%RIf}h$oa(etEzDN-`nfQb6EZ;t@>~s-cOlPvq+0YUu^D9MtAg z+s9(0Ots3TwopY$WLFI7wTkkc+oBCIKd*=1D;drpx(aGTO$or&E@}tP7*uofCF{-O zx|WW98R$64w}~i|$*r0q<$U=du-^G#OUOBtQ{SMJAv}|ingJ`xMiU#IpB|&-@@ZB) z6VqKc(5-xjvU-$p4#yZf@TB9EzW1j4+-GWU`LX7#PlndXuMP`>Yr{{!fG&{bb3J9c z+G5}$?%^`Zs!>GzG4C~3JN=prg65?hWYfXoIRq_iBPv@fO$5gla8$Q-Ky32|mpU0v zl5ys=FYZ&Dz07JBf5SR%W?KB=Cw23slv^$IzO+lY=} zZ%JYD5qH75gG${DZ1O$^^B3RalXplk1uf{icEhr~HVJn->J_eDt@vOQISV?%N*Rhd zb7m7C8rscJl(8(R>?If9aB$hG6Zet0#lIx=YsJJ#q_b#*+HCEU&T0vbQIVOI$Q@xrTHM9ab-<_hEW;(LH19xtp^9km8zTpB@7t}Gf zfVwOf#g&vMQQIUwz?;jEvID{_PltovVW(Vd4*rs>=A8B(z~}fz#c@od78qxL#-RgU zs@TcHi!wGRdU9fz4Otb0t;~n;(W<*u0WFhrSpLco;g^vHWVqR#g(;LcI6^eK424w6 zXBa6%p#MpbPpC)f9Ju8XdN{~=FzHLg(REf{wprt|vRmyfZf{uWo3jer`IKZIyr)Z0(s-tZ7k-fSIMB*(A8V5nCnH|;PtaF)ulwCV#X=`U8<%Z zj?kz&<`d8%>XE{(iZhJVAv7ZWSWa9n$z&-m(FNN2vbleXmzMm)A-w7hd zA`%j(>H!~)sqXLrc8E!;B#wb_v=P3-_>=pl>VEP!XG4l@2QBP}Op6x@U|HT2#60tp z;Setws|-vHZ(Smts_CxN;WJ_eYaZ00q}zU4N+2?RAL)=(XUV5+{1`{X>rAR4I7IlM z(|k4AjpL?Os@=^<<=asWu1~x}as&kShG;w~dH@_LD>%Y&8-&h>Ib{v9@JX(xegrT- z32+<@1=U&AcA%Q=tS+t>y75dw2y~G$zL^sWjZJUrN1x1t;v89oJr5IX?Ss{eKy1X1 zPJ)PGRM3@()s@Mg(i+j}>6@4FN+KQ#ewIhgQ2L0=DEV!14Q_3+P^$}<3e$^;M~!z2 z>)!xPfx!lbLFO28Sx`mU_oyzc&_#|KiTU)D{s;`P>ehsi{TU0WC;=!1EY(3ypQG{s zyGZIuaN5q1!NXg(vyI?uz!qO`*EuA(-3ETvFfe)(2pV`nu%d6I(U&O}J2@$@^Ae%Kv)T>;C6f@dN&JN|COD zO5uO<=NEN|yOuWh>&F)|s9gN-Z*>2)%1Mudk@Mr;u6>#69iW=!pR#bj?nRp?sTK|u z`f+6ZhejM`3$(xZ=Re6~G2WzkbH$(&-Ep9TCPV`gSXkV*-LM9dIk-IbPZBJE?v?{N z0%nsNTxk546yU%&fG^vp|Fe&fmhD3$`M`+4e=MAwlyQtYFoFr;kSN~sF9KV{?Yn(! zmHvD2Cp~A{wEp$-C~wz5z-(Es_0Ruewzr(}zh3dS+S&4r59nFipKQM zIc~{$qK7cDNjWz`gV*4b{XtrLfIR)toLF`lkCi+M{i@`p1my%pW1X5eT6;j@uy1PM zf;D$}CT1aWTPFtFAr+YYh#{>i%_xIkXC7;D`Zh7Sh$?Sc?#_li2wO0Z zB%YLVp6QBFmu^0b3b<7@aSCYE)NdQ{9!6RC?AcLOfPXEr8Z0z4?iDA!#ie%?*4mR| z&?!o7QCt0V&F}TH4x{DDn+BGtMaS;<1N8r(cxhb5!l->j@@eC{qqp7Tp7W2y@)zNp zu%}`4>C`r}XFgRL>3f-`gunQ)$mz)0=518sR&nGEzFqclt-rOlzR7EW>#86jT->-^ zf@O^)8~P*vCT!n0#&vZq8pT4>-mWk%oL{{g)#*`fjNn?{pSKT)-yg}jhW8{VV!qaj zESq`aKd+62RPWt&`YLMMKrefnW4q~;U7uiWd|l;w;?|g)ENzk7L$CN*>FNLQkN3qx zI7@cVvRPydgT4cx`n96pIm6ABB;x9xhA)$3kJds01#?!}F2!fP;CML)i|L4TD$$|6AAo`6J%?u~)m4%QO06r*pm6;_NB9 zh{KCwdI$e9alFwt;4FL3FPxkF$@YP}fYvZ`IxR7k#O|fIdgSCY{^ltoYq4C3d}% zfB#s4zH(>j`dc1^!m2bd!vJy^tfdG3a!_~Aq>;EL!R8i57`08cKYY>lKyS~VrQg5g zY9~CuApgDerL~kBjIzjxwsyiJfajOaV4vwxpuSqaCs1CZ2@WhQiU$X&ixkFs{}_A zTir(X+Yo)W--hA0nfOABP^wAV2Ri7EqD;eQrB)sGBK9|Dv99v#dr>~-UBD$A4cO*H z#L`nKdB}BhiOD0Ih7zwk|o@2|rDtfB- z2BSaEm>aX>Q)M#KRsznf#p@d4mk>$URnyu{`h;eV1FL!i&_6SuMQU#fc1geq+ARqCNZ*gGyfjWB&Q5@qI|G|1pf=Z zKD5#DydpS_#bT`2Ha+tl!T$o*T{WnrQ|$}cyB4q7&g)JJRjEsFZ<(W;sIF5fIPuZj zReiSHRkiAc?5pBbu8w(IuMCP>jmQ)($C{w>b^Glk#uaCrv9bN;2<^ERHhLu>shdrQ zKf+c{qVqdW#>~Yh>;pYbsZ`TzQ@DStnVVh1ZlxFQs;^W4ibH)(s}4nJQ+(H!ZLBX` zOxF9=Z#p4$)&6N{sV6D0E@8TACK|5raE`R%GO+1$8=qKJG}*Jiayz*_ZX}E z08}X<*5R4}mU zcLUl#j|X3$wod}1kN+2vfWZ9g7vNDKKiVNF)#$qjVv$$Oo1jxR9s5nqUdyNCQE=?l zdx)AR?)S^p2H)5-4WApZfrin%oMbeZvC}9)^V#cKQc1gBC_6F;uEb}J!H-l9l0Bh_ z09?goHjo`Fz8BnaDrsTHCDv**^@B&|Hr=iY!)nVk)Kvbd{ucrT{}~}+|M7Ytyf&D8UC)E5ie%aeXZqK zr8JfB(P}G3xBQ?%1MM=S!9zLwBy<{DrK#R*p!fnIts8%rgG5f2gwKK7|{!^RIF6AoT#i3d-{&x+!6@ry3HNN~R?j`)04Aj*nYPA`WEgJ>q5VP=y z9GGoTX}hiWDO&Y$KUP_@WV)m88zV`f43T@0%jPI(h?i{>cER2fp>s)}LGXuH1ij$w z;mR&t5AGc3QI?`V$6DD)09*GIQ~##!W1T2yX?-bFGjE^4&wjA0K(qi0Uj{Zfl7}bM zSoY0|Eg)tD(dg>ch1f@-)YiO5KBonGBz(T^@+9G8^lF_$nBjQ& z$7!>iZiz3iY%lQ>AVtYK0iK<%=yNKk-fSu|us?I|#~q~pczK(gv4z2}Pwty^Hp`QN z?Bz@bPgiBY@S6odm1r$R6-*7N=FVvX4M}zWFy^#<(RX>3>1PX~negc9%(z$Fml6bT zZ0Xl}xP*&}_L7`}T4*=KECv%BTkc;U8Cm7M_NknaEN0Sw4GdcnS)t}?qw}pCe%;I- z%i=%fBRi4X+dmpEdhdCpm`R*uwKBhnM}SF)oE|5M8Y?OkMpZ` zQhrX+Os9|?v0|BO<)d`Jnd!TK~9d07jr{P_+GhgX(S;Qq$I`LyU7#YHgI`S5q ztCvV~#w+uXYa-Q693KQYJ;IT>+ZUAi+1?bo)BYiV=d0_Yr>;u#5?P@%$ z$xUk?JcMvf9F2 z0C}=|%~jg}V3`e1bf6TgjfvTJ5$R=4&1zMfGW+?s_Zg1A$je*o2u*4)XtTY;I$ZiA zA8WJkjsaZVJTH{4BJvC#KOf9|AsnwN!XWsfjsd6yC&XYQoF!=8e7`xnh}j@_?k@%( z`RX4N=N8vh9t_#BDO>7zWl|HT>F4qffie^4f&(l(tK%Er_Wt2*YyDKJD}O|#=#`&G z>t`n(_0q^tt5hXw{hO_8WfXldjATe|mPOmE=sc(rswV21rJq(#ujdrtZ*P&+WjeMTswzlWp!tsS4$0@(p6PM*@O*dPl}6&R_U_sRnrv)@)bgMEVQW|M zanOGF+$c6ka|E=f|7I9m*DLzk^QXA=(6QBlcV`G|o_3m=*;+e8Z2m(_q#~E+U6%z{ zt;Tr)CRv}W?xh|}Uk|vffAGr!hUVVz{}a00lCJ^hF(Lo+i+|Es^fd?D2;oOET_h}` z`VN32KkKZEht1RaOA2Hy+l`G;a|3`;>Bnu3w$+=h1z0xKj9S+*lIYjJ_@C9p5DTuV zc8wri15jH7*1=LNIghgL1HThC@PdhELs+fQ+9Mcg=wkT|ntd=tc|%w<-ql zXasNTZyM{z)UG-Elld@^Hx28AsI*$!q0-2`2$h1hIb(;ESw3q*h>f*H8>z%Z8ctV* z3ELMz1~*I(I3{x|a(wq*!4}utlVHT37J~mz4ACI1uD{WL#AQ-g@x&MW7fugjU{fYk zFZ!Ulg|`g*-Ox>k?d<$4Px9CR;zYGa?9 z<{Tt{SV;a{sF3_c@6H`jFucVieXXDmVQ#qkGxinTTk#gwN}UbJ z(t^5gnfNvc_=0J%6L@ za_koAGMr2kvTq%0|9q$Ilda#21@CMVyy$ZwTg*aORpMC{$?$08r4G##XkGwPsBId|z64&g|nLf2^tja;MIX(y#8FMv2J^3wq* z5JIENKo=yJLm{#Y=T{$WGqjGm&SZ)|0xo#BD#@v@NHN0F4{z*qr|f#=n|UC7UM!$t zGUVv*BkOv>Ad=`1&Aj~vKfC{Ow0y5Wo|8BY8HJ{?2T7v}#Q14>)vKn=<~XL?j=n;@ z$Zvh;#_6_8I_E8F>d;I|Kb|AsebmVTYX5N%Vl@~CT7`GHHjPR^T`6evmRo)JR#LYj zlAJFI?x>_&{;~;uV~x)Y6s)e-xjukX${h{oN0Z}Ez65re@Az(D*jIpI-*1rGG#&h) z%nJLkmWs2R%OT^7Ppn3pBa@6c*PbrM>@OqsT@!?PQ11_Im646K@>s0R=*81Uo(w1Z zH%T(o(cDsLd73Yjd2w{RBl}nkW~6#^NTr}*EwoYph-*gbzpjo>G_|~_n0$E;@_5^V z%Qc=A^T@?_MTKeY3I-XrHkw%v`6ftd#Hl0i!n7pdm8Q2bb?W{JAVFZe5j!2AAHmX45P)PLOUdcsIDmrah+RM}E_*(fgSh@;qH1rk>g5!Ey8@I$98$6r9A@(vk zH1=w%j{f(Oyqwz}En;4P1m*<$BZ(xgA~5^*ci>0 zdwWpkp(M87=Vy2moKVz)P2KcFxK77|ygDJ?J(9MKzA}FeOn`pm@%0e3bJ4IwxdO>y z{)?1QoGw?@mLXRqI)$@aTx<@K0=Kf%)X*uBsJ8{n=PxF=QYQ^3r22$NOCT9P zD#vMCLsoRE=Ks3&EZ@5W=X{WE_PEL6N@sR`FdaMZlnU~u?n9vt_xg3!gZi8Dt!S)- zvFiCP@vHc@%OfPxBu4hUR0CfQu^Qk5RX(>OvC0+3zzzg8{BK|b zq)PuUpol=+Nh$9jD^JIragJcA0Eh29h`g1mzYq^)D~_C<3=wvLt1_2j`X;9KaRg-{ z17HYsG>5<#2{>waZMi~tNLJlUVWbfyaP)9x(3Zs~E2oA>uK1T(mF z^RZ>WJ$Nd+S5*$oK{zf`@LMrY*UDTIi9V#z>BJezjg6l~VE(X*WnIBX}5TyKKF|`+2>`r4g$L z#g|a0_`R1(NIn~?@$k>?gq}7*b@b%+vHmqJt(uW@p!cQb2uKdND>rMXiv5eZAx-An zEqHFWZ3Dr}((AwIn%;z{;A}w5^yM!?!F*$*|02}-A6|I}Z9p#-1GFJ9Lg>M(gcG3W z=_xerPJfQEnhH`>EVb%`eub^(GszcuBjP!9ERqjY;v(4k*NU>;cKDV9@$K2`wzxMX zG$Z-5usn)2!x=2Q-C@wxHtn2h2{Vd0q+6PkB)c)AJ^Z(?=5M|{aOHUSQa|S zJS#?7^JyfJ7CgTjxlDA3%a|+W@*7zKb7ySyZodn&hl^p^-vD*dpnY;4N0aE!o2i_axv%s z6`1IBTk;oy_tvfxn7+c*TYRkhGLu*@$K%Norrk|w4{|!l{-uR~i3RKdBg?^_q|%6wV?h~leBN8b z`nQ9y`}zd^!NfRJh@=ZqUgo497>@l1!bkzLB5!3HWLBf$LMFScNf>QJal<~1>_=HY z#qB>ZdG>&ilf!g0ZGJ>tX+)-{KrOntMq3QKwtY@@5w=yzJ9EcrWX$%+dC_47N@g?4-^Hn ztbp`0guf`MT!q)kp*h(G_##uQlkNo~HZRdQ29&kESEpiA{F@i7v!C`D70kFR_#RFffAL^FhhiRJ9NKc(7op@i08wj3N78*4y5p!Btc~@RBqV)W_ z&Cvx2lLW6dnNJu;a2@bG^DvsyH0#YXQ!1gEqa`M|*TcsZ*y@eiqxyu|&uZ-OGs9=> zKOk}+WraB~WQ?Avb04|WmH-CX(x9BBk6;J-w3>3ls@`3I`!NjPp$voCgV&??!AQ*_ z!i~`Cjq=!g23I$|zDR0TIhvKxuq2wTF&ebApDxj4JZSKA2zoCtNeUMo^VfR;Ft4%r zH%tr$FdgH!awKaJ2l8&6WEEobH{It`XfwGWOzPY7-?)oB=)^BQ@L4s64&=7+Zyht7 z3-Pjb9S(fy`5)eBnO)@c8EIuK8nv|Wt5*Rgy?AH`wJxFEePbe<)iF#%JMtU zG7&j46_+@&&SWs@;Hd3V*c{9y-V-I#OeG88>_pP}mn*K68f1oh%f(gqf^e&`7@Uj$ z&WOdRop%AUCi}0XeVpOufFeao=)D9%)PpqXB2Bu~NQXcYP>QsmKxmspzNtsp$i(Qk1SeJcBT=uPR7RS94=yr&QFfKcwO!cm{l<2V7b6S7zRII#ZwC1=W|E2PynBDzAhU~W`(!F}58qiZ98aUfO zg_1V_MNt5X0+js^*$ePUEt`DXHdiqoCP-H6yVV)?{ep`hOVKLIKzv|f;kxSG^xuny z>jZQnU4uo=u*}?2kGF7H&Z;}lQXr3Qb!;>c)>Xw>tuU+0;~! zQ1hT;Xo(&=A;ZPH?brxJUsO!BwO$TPln;kbZ|^V|U0!HdZjcb_>dbh9X{h+rGI^)3 z!44ywc}`|CDl!DEm!q9~S4k!Dc{woX@R%BI4`5BOT_l-rC!i5KdSV=YLE3-Z_g6^` z3Nb5xu0&K#>n4KuEa7DUg>AXwEywHvI6hsWIlp|E@qb}NsyD1FzJL9UKrbovu9&o# z^cnA4t8>9-P)wY@KfUs35yR7@3Glz(c~%M|jEFz(=TgFrrocWA)&z@z zGrB{m;X@HY(vB9ft)ufY$%P3shMs-g^==+bbWeL$kXwmd+ye_t4vf~?_y8|P_Moyr z9?-UkXEJ?jo1tCTpxh23tnS`pYZ$yv=ginMp2Ls(*?1lXU0gw3zvyPrQZHBO^rh|& zV#OkdP0Z@E1C!ii_A)rkhJUi^hg0f7*2*C0x;`-gN?x6ll;^g)6z~i8JMa>@miP3&JD*)B&a(~zU`j0?F6dp!j(|)JS5)k!lVhGb?_Xy*T z&pIM+**c2{k&|}0k>PLr)Z(T_W`H1THsS;d2o$7aDlaBdrd~R6ut_gLLk)TGMDZy% zFaB$%e^=XpXRBsNZ&q|(o5O|3WKVugZe0^{Gif&#kr+|TgKk8+My*FSCU`zM`h3XF z!_IH2GKTgOyM!$?6P1yOop;V`PE1Jr!dGwFf2{J$B?=F#++?xTCvYZu?|7(2WTtw_ zgpATi1Qe`d!JT}-{GwKKi;_Fa6MlmB9>_1mSYxZzZ$-(wV5D%#1vj?z=!_;DIvzCr z9q0Du6hp<266lR6aWZTiENM2a_(5_F>UI`i5H8AOKO|@5+z-j0oU6F9QkbI+4DmnZ zQZQtN5Pf&y-Wyj$5^E;TMJf$EV_Dt&IxvLQHo^z&c(YK$HPG9D&h4jaC6lJwd{%41 zdFS01l|crX_3tCyr4FnQiCtZ|)~Pk%+L+%ExY|h2;dMU2s&Dgz$lMwir9A~s+KgSL z2g`ZZF8?I7nw2Kjx4LzP{Plth*G{Cs6Tt&2U&;vfR$gMR+aJcNCQv&IAKP`#v06;t z85G>Bl6nTx{C#KW4V<~+u%*jLL|$W#4q}wC<2S|cL6Gbs{+qG#^RdNiAP zCfstfY2*i-n@X)7Wov-wwXk5?m@Ut?da~gYUKV&Vm)_C0+yeW*k`4y~3_FGWnnDiF zPgx-7cYwLj<)NxalN_L{zvK+iy}(N7Re{ z4+1dJ6p4P-y0-(8m;>eEVzj2PgTo3bbibfNfUf!}xKb(U2V}(AtjN7Yg)b&rBI)%$b_K%J9~)M=XvCxOe|0-C zR)B&Wmpz|;(dSZu0&nm5@!h|FOj)I1}us{K_%lg*X(2O|fmH&pKnB1w(Csul6)b6-IE ztSK?<8YvvDR}o}P&HKTK&w>olCr@d=5V)^-P}DvQ0bCITod&;X!((Y$hz`qsapd2O z<+tqOUrdB5AL)g;&K~MlM;j<0Z6$g`kybRRM9P~;iyA&)9Q6>$Om?<)o#o@aL zW8co6p@Z&AC;q`Dzn~1*P`FvxdO%P?!}5vYgHC?XnD2ilmf%)2nEiqlW?T(5kojg; zqJj^neSSe0O$Z7mbzQEEO|Pe{?XoMq@mYvq3}Ke}j0;LOt9&WZ)2)zya+pvp#LKeB{&5d?1z@+R z_J*x^&3SFl`u7xUZ3f=1gZJ{Lo4URx1NtKrAUBGcQ8UOh_K{ob8WUiM#d7WIJ#L|? z`#{j~Fr?2g+3R919)wyyU`X^#e{w1p9||0|+CYBc9b3KXMKT;*9$Gu!$rNegtGtKr z!FheHyh$a4HnE1^%z#E`ZeQ9n=jl{SGcF@$Uf%*BoO~a$dSm!U)^I_|&Msur{D^hM zz1m?m2)L`S7_a7tt$Dn5{cvunL2Sl0Wg*cAy=0_o`{XIb#}MUldY{38;n@3!mc=fD zW{$t!7x-%$nooy+p)gSA@ctvS+5bOR{ithzS{0=Olbl+{`z4E^p;6kOfYk8f>w|v; zH2eSLCF6g+Ou+P2!&pMjU9Gxw!579fG&cLI>wkXpEq4)TKEuWz%)*uo{h+YR{$H4+bw?5sMN`7!v`z9UUpK2(6{kBrK)O2D#kQJzn_sLy?tc9Y>R3DPjo%?(X#e>82}(N^)p9BPlRxRW@zCV*Q=g;?#laXu(T zMV}$<%bEtofz#O=0$s)5XzweQc>RJ({gm_evxJxOgBMbcLhf{X$JxqF72L4OdUVJy z`TSvstA`rU24w1~@VVAzOw(4^giCn&dwww%!e9)_;3+u$-KTyFw9c(3?42A_R7oU#g;oh_J$wZ=+fUQ1j=v^_ErzmWSs+Q~HFsIC%1N z72?sH+ux4V`-1SS%IL=NUr?zWL@7as1&Sbd5|O1qB%&tVZSkV3BE@qye_F{#f+{6J zzzxR_KA)X_FN2?~5WQO-iRpRa>nSq{ucfNs7+Jt|fvx~Y)fKF^yVuP~7;p|LmBM@@ za`baErODOUhE;1cUU6@0hRJ7JuJN2jV&+F8CV6-`q}6lyZ2jGR8?!h=1Wrc(g zF+4{K!Q60JK2hR)lER?+vGG#m+}!jc=xlRdBwD}9k>fIo0wCO<)1tLNG2BL~}D zE||%^wGqW_7u%JV&#V-2*=J`G4jMv=RWpd4xJz}7Uih!&C@TU|vF9(t=dJC8^M-3* zy|MNOWAAw*fEv}(l>xer$0){G>Zy^Oa~x`+kQw)BLM)(htwK(hHj6ith6$Ccm-~ha zSWn!wq~g|Y7?bKq1W-o_pLs;uCC47qymBkNtRjn?$jzb{6BM+Yc_KpQ$K*+J0c(Q6 z`qwx#p$h1xs46|GC4N?DKt(K1d@OGGwPffT<(<5lTKci|yr*7tM z$Ef%8AjLoU#w3SyX!Cy`pB$rbIpK$$^IhCoGIY%qXPh)&HbQ=u-mD{Q4tt~BxQ-uo zrOa!6+WMK=&i}C{BfEgr`OHU3$%7u&lkG3KNpPi&N2LkVOJ()7$ZK<+z21;beZOy_ z%S{yTYpgl?#rNZ~)s4K!+&ENYeLZWz#28hzbWkpA zzyn%nq9A6y*#!vLxWczbc%F#TQzm`lAk5@Vbo58=g?sJn+RlBI9Va@*K^iH=D-dE1 z(QAV!f^YuX1WLc`Zt*Q#J!p)V*92ki`}5GJO2P+Yzx44@l>{iXC;$_Q!gg#7^+J2z zDd4Qb?iEItpJvmndp;f2Hrn}~r6LR0ZC2y&Q?^M-2F;)aCRH1PvnHXa*y$k6br-h) z&Wal%oTBpI#d!%}^SqlOdB#d_&fmirY^LAR)O{`=X7`C;V2GJR-qwAs8HVZKMnMP2 z63skGel2uUlm!!8(WdPxyVAm$6|f_}^2aFM-Ss(7)gHc#{>S%ogT=`5xe3$DWeOIC z%56W!6b-Y$Iasd)0%UY&wrWS?%=7L3Zr3*&iI^dQT}XUw5WgU3f^-Uu{gZYn_~y{FEUOD|1` zbC>BOZT+lVW-cYsWDN#3JUN2KV5mwJ8P7p#_jpEe(7kqFb9JAs;vwhNG@XMxaG*=d zQNmrj|6*Y$P6c<>r-0-tF$hO_5=*l}$R2ZX7n)ad8Qm%C?bL})^PaeGrQUo9n`Igy zm$or1IR2FGmF6^T4O$Y~+#F<>vWEUOu_WeFdUCM}^rttXNbRz7GxCG{zC%sq+Qi~T z{Ns*j3-sHxJ1>GTELkrP=y+Vu>BUH;dwf06o{W=Tv+{KD_a!A+0LmAwI!_#rAbCN*On(jJwx8nOVdU^ zMB&WZCD6@x`03+yd$6#;l;WmNsW$4AQnJ_`}w^P^Q#NLHpgRC z41H&$zjU>8U*D8^M+SHzuCtZAsB{c=>9SeQo5h5MwuQ!feQ4K>Uu)~Hir{2v9pfL_ zSv|wY7Sx``-B*HKquoN~ku_58(27}10;o*Z1p=;g>HOz!|FFw=y~9+UST=HJ$l#T& zs0Tds^p_1aCLXt9jL7_J#`cwlMcokjmc*&XM0=Gb(o{jhGw4Hsn{8+P816iK`qGB3Ki8iY8rx6^D5(emN+ z{Xnh8(u|~4U3MSos2kO9zr%86Z(u+NvT>B{nf>Pi%-3TV)NUlN{M3}+d)y-w8lffk z`c)0L$eC8x(=m3=pWL_6jY8;I9WoC`BY~X< zJvM#vfQtjt=*%T@aBW!h2P=n{1dZp+!DLd~M+wnxI5plqK(n1yTM*gW4D90!TRCN;q7?U6Lv1F`?b2=#mv|GmR zTLQswzm{f1%vpw*Sc!<*;p%Nihlj+c>txF5@nJ!SQC=z>6;%5LN0Bf9M>#kaKh1g8 zR@Q>PoR@Oep5-fT3wTXEn0t9B{+w@&rAvSEP(dBZtc_JawgwMT$IMeh|GiS%;Y_5I`nHX#oKFi|c>x%WT$jvfpzP>BaDHRGlIPU;fG}Am%US0n^!R;E@ z!u@N`w=w+Caz1fQ`&HYp&-9`|4hKCQ`>ylhQ)Wt$VscY4TXEGx?Nd%`9opQj)R~`z zJlXy*lLILZY~ig`%b*U3IA^EK*XfcLGCj@9sF=?+W^EEJUH@q9DjqTut-p^GtM`22 zzPt=6#!g!806R`7e0zbX%0A8T_t+}FGfRNzS)`xqhj`DRW2r-y>ufSLc*;LPo z!ne%=9D<$f9(e8hRRT~MFw<+exd49z60I~+FhDVJI)$sw#%M7++;Q#FC)$@Kmv$;C zo4n2!ef)&$Qv-J0lN)xBh zBi7%d$O*2S`)i-^bj9~o?kqQH?Zm+ht+*iRl{2fg{F;cF=i1Q<9d!*O)67LMB9g>Q{19w1E+J$9m-v$N{LU0dVg^vB#7zkFAy zrR!{r)=Q&`lDVOQ03yqDW|4w#*q@W0JAwQ3>`>`gkOJJ-_qCC)!jSH4a=Wo;u6uI( zO@ve2u=4liUag&w@arPT_xlU}k*3>&#hOVPLxri=vmY3i2M!wLcy95nplF9i0fyO7Wk1rqlUXQq?Pu z+FfHOUlvK(plr(B5zX$K=J=*@M5p%U8rPKz-A5q{t&P>pqfY&kUZBsqQZbjk!%cEU zj%J3VLsKg148M-Rt`};{C1_tQ=nxqI{gACyUIr?+5ySCUjX1Wv11l#dEtsawqumWF zzwlrpt31?i7Qf0F{)BdLsP`Jpb$(riI`ex&$sjcl(-e8FieKkFSE~#%0*#|(FKkt4 z*MTthl^XwJAgguh(~jNRK?xiYgo%sW-K%L=Ihd&X2KD|6RT}udh8qoz*_e@;l>zm* zhqnY_T(F;8i#qo?uMyh5!y&uQwl$4|ofNe+Rdd7#ohi3mv{6pZ<9w4~F8bAvB8lTb zTp9iY$Vc`|C^gB%(B(t1Ip7S@vqeM&f@6|h_&NHzj?t6qpD$5RxY2WQ8f<>bieYh- znA}`XDLj@sb)eRAQ=!L_J0ZjTG&Gt|1nyVp85SKY5_rk6NcP-{(dhdY!f@x(P2&%E zd}XtB{FmhRv9dsM`=#!pHHHv{&)hEqA5=9zHY<)%Eo?g$tH}&!A|6oT55USR@ z9S%LLl;-q?av>VDth9b8FNm;?Jq;1|pzbxNkw?ljpMutHPPs<5^x1x{G>nv-r`!## zIq)`Ev-jNCN|#$YD41#0?o2KQuteYKg`>Z1)l5392wU9W_-DCCfGQi%3erO(0QKuo z76IpdEXSMyMdmZqJg^Uhs4NwK%R1qM{4ZF4#6#J$cJ_>V?#s(_S`Z)L{z^(iUoJfq zYdZzM$vn2cKZ7_4{aLvVe-;x^4FB(#XK{{r?54eC2I17o9p_RS{P$x4h9}!iCUIk5 z_OzUL78V8ki3EiepVH%_QU$JhOcjqM%FnTWBeN@xbPNw5AeU_Aho8Yl9JAB5FXmML zW4S?C`D@=nK-fPQJwW3Qo`9p`Sb62n^V?;WjRTp!%oLn#0PLEitToJgY}(B&j`E`_ zVR~kMF;Dfm!U&hEjI5tR-aW<&s3no&bE(8a=q>MK25x&P`GB5+XK*ycORY=862fc zE|a;#0Nr(Pzt4y}`6aQWmj1zuONv()tl#cC@&Dt<3)!C@I?beiyUg-4f9HU0+&58V zYNAK&-lMk`AG1LsTtDDz9|g6Zyl&Xp+SuIBsrJ@l@Wal9q%|Gs88&b(51oko_xcsy z*6H3{D7Ban2qLqIU?~&EMb+1Ps$reYNMqjMGC&@(z zJ#Wt53>Oq14k}3t5D-c1$0J=oVq`RXu0MDI`v1OEe6*H$PAy?!QDujlWN`nM;&t9p zEz)1>LS5^n-++6jf_Cb3I`AK4_~nZIX{Ylk`3#O}-$ZXlE^EoO29ehe$*zltRxFbB zDt`#2aWxyXr^R|Kp;pIPXQgW`Waxy!T^7l;giSqFMI)-mMeyXIZ@vkD4^?X4BxHq=leE1qB^eB+eL%(BkxI(y~3)b5~GOjm%I9jh-wL>CjjOd{9jtpt;|Mw>4ILhg|J zW`hojob2GIiXUlc&W#q8wZ1;|F14fdf92!qE<654`yIBxz8NYZXH*0`7^tkCy5E(C zW)W~+@PqPUs=-yzpxQ9-sbT%~!)dB$j~Q&s-2Yn|V`=+y+FM_1^Mpij5h0y?fCre_ z+KtPs4QaXoO)IUjy1=eP$zr@>+3>I>p>eC~g=7?XDsmV}I^CI~sxJ}YOOE0$eeo>{ zY%bkKtn6rRpbyY=|d?Wp@U^R~z9)R?R4iMF&se;o+_a*T=T%ooCw9s+7;13K(! z(S3fs&!TZ+$Z36ZFKOCmb^c~04|deYqI-X8Jic4|`qRjr+SmTd_RP>@^bls~)nBjk zrPbAnzoK+v28^1t@x_&fd6nibJ~BSY&S6&Ux>E%H8tG@=-M|0jDi>dE-0+E_b?BcP{;LHlF9Zbfj&32`iMtWBJ+5 zr%(4Dvq1gN-=&9Sue?%A4Jp>r(mc9$Lgl>0*li{) z73kAc&NlEq-`grW^OaGOswP$KTYY@WE3!2t0kzrb@myS66RR6!q__qG|33T6h6<*giR#A0b^XS=lE zv|D4Cv>K=LcR!q5q-p$>w;c&Lf^%X11)@!5fmI_#tGVr~>8w1`fmoAH_g2o%zS5qQXF$eQFhG{4A_gRQ)qJpg1!v;dakN5&rcRzS2|D!7zh-ZQ zL!o1kAg{T1nSY@84b^FT<;jIYJZ zQ*g14kmt>-ib&OXah#mjM&Mg#1k4LJm!6VbMpulzg6AplZ;|1w&T#83MA&v$$B9nW zNfQdv@TJySh^nQ2t4_t9hpbR%G!f~li)zlvomrsRB4qM9ZX;0G{;f3$mYU+uhGikw7W)Cy2 zwzFEri8h2GV~vuskDlb1sV?pGDJFWZ2#WWFu!-epe2TQ~H)U%&)U-Jq5BB%I%ljfu&lmoM~M))8@m!_fw^@wJw1!iQ+OPFV6+JYVjf=P^&GwFgPV~;!%qe*TIJ)^$pBG6cwZTA81i@@s6zkI$9lQL7fj0J0hYnb9`ugB4(nF43 z@O)1!GPlV35Fr)X9ygtk(k@5Jn3Nb6U!o8E& zw@!>dhIqUhH#f=Efj6y|(5~f_If?$BqMckjlkCBDFKaL2RkP_Qns_b6#^W@P3XC=R zu2ob^TD)0_nx7tq)pr@@Drv;oMdRdGq8wW6(h26fuTrDu$A1ttl4t^au5kfiI}pOwT; zk}WiN!KPTC9xqioH!VX#-E|vwX3AnXI~!@#UA~yOsrFTs@%`(D&8u$9ybpiBwbBu; z#wQV;%INJIj`&k-BA@TGno?IyKy=K$p{{mo(6H|V<}C+R&nRw=_0d2e`*VJ+Dc^J$!-E`QRf*~VUDjt96Sx)i_rPRIdYiOfxt)bs^+%7!5Y~>c+kR@l zxp$uGK1{;@bywl3>gKv8w}Z6pAWuGXQf=kj1e43RHz)Xjk2*jT_3*ua3#n9gRxc|(3N0xGx5+CHi!`k@!yH;)8%+GbnhZzl8T$-Idj@eB%-9m#oYsw# zmSeI|#dts99Am$9JfcqTN;38{ZEX3lZ|Cfs?95gcT2g`2ySG;DnJs>y9QWXYQ@Ych z&5ut9s1t_fB_o}t{U+u~lCxYp(fYwjuepo`Xzs$a9L*>H=qkYL-`dF>S|0LjY(vMA zPRwsll9Q*m(Y-Z4Dw8H){UzC*J-%qmA$K=FxcPFY7#9r20XS&{%ozm+~;}oq2dQm0C%WFp-cE?f1KWM-OpyhJ5W25s^A&(EQ3L{^29`2 zrEX_NgUw#-n#y^TJMh3kn=yN)_^l6!;y@$)WZ3gDdZtrxl7^MfpKdsy2H(D+vnk5#DB^+mcgc1lS)5>C zp%kJ`Nwr+^8J>q2pw(3t_{2UOQsdeOt3X&{Hu1XH8k|VM9rW0Kyc50u{x_-bpH-?m zgk8dQ$Z8qqGhRPUXe;Yg5!#t6PN-+6PHfN=+~vAMX&?q~x+t&W{dCNWQe39^-MjvE ztVm|dnXukjIG;#jT{&%8aYkU)g0pYjMsFSO`rc&mQPms#*Z1_8_OK7Thhfu{PMnn2G;`WIa}>#kaFA zh-X*lDaA;rwy4=5QQ4rav~&B}Sbi|!zDZ~*;T8=ulu_OOp#(3gY+-WMNhmPo`p$T= zcmGVn@CDK37e2k=Z?}wMs_A}y*0{X0pl+PSWqGy~Hl`w}0E-6$c252lW6fsL_jWaw z7OjQXLorRZZ(9&Z6*A`+ChAft&7@9O+X0#Jv){}*+az$T#_({2rI%fgfw9*&F^}Px zCT;uhCAev@z^@TjIw7_culjNLg^4oLnh4sskwxXolH{d!k#4b{-|9}gB6AnojocO2 zRTtWK7Mz)iGb3k`EH2!%@w*+7Ap%z-k~$P1=V zgGOHhM>LtCC4&tu(FV+sK8AdkQ>xQcyZ6k(vLyM&KbX6E@!07Ll(bY|&GZja>AY8I zcXYd@E?S=u_1q;oQLR4@PDc-!l^SH|*bX(-nyq#d5d9dEjM@R!mIny?Vum zTdu9SK)vy?Ae`;5%MTH7(0m6SD1^E z%UP;zGp-a~9zSKuGsa!WGpMSlt7-HVR`clqc1v^`Lu{~i0eMS8-TUL8i9Qg5{8n$8 z^IL7{#|EyI5pNtyRo}bt42lEO%GloiArAx@&}6sTN~{A}F4hm_Hit<=lTy5L8nX1V z>y@x*clGj?)l+hw7$Y~7OX62mR~h$12JK!eYgUF!kL+*v(?bSi2;C^rQ2(I1ABk@z z$BRsd#wX5KEOoOxBZlyg2T2q8ytA4DPVw>^t2a^8f8TT&hWj{zH>=9EDL3~kSclA- zF-4Bkt3*DoophAhIVqRvbEncy-&x@O-_a8CM-*&M@j+w}1BH)cvOgN~>^vPiKZyw{ zGND^_(_Sj)%j~uEEmb7mfqdyCG6{Q-R4?0|&oeSjj3|jo{Sk1m6Sxz`o=TRU^}{zU zcFq@XmVME%Nr;VyJxenSjL{AyK(XbdO4OkmM+au;m8C09XN9i#8Dii&Imu?w6L$UJ zn1tR^>1Yh^%-iGaW8N`?tf(tx6IJ4{{uw!$RR^R*_{6)| zd#%UMRk+6+8Gv(fVLP^JsfrOR-`ocEG3D+WCW>DjOMblt>J^Q=#O3K2P) zx59bL5fxj@R*LAi$?o4bp-7gV8X%_cfu^|-EW$|Q`klXcCvNBcG- z5#I1#vR+N5zp(_5w^R@NP)*M4`#QZVP^`Fhhi=jF40KGkXRJ5r`m{~qbDlw)9ZLq5 z`gxOB90pjK9+%xv6aqKSa*C$l%8-4ajOkw!cMhFWcH@gQ`tUi{r#OA4XlTzgywfaH z1q0G>t(!HJ#QVGALv=q_}+>h5varRR4Bo|%*$Y?@h#qCCbp z({^8<@Z1h{M(%VC{arN@G4*Lgv%Kq#KZAzXWpLB|uc+~RN9iLX z{)|`LhDxRStLeylwF;TxSu?Mk+qe%ow+E+4{`EF~URvm+l<#}k`VmvImc4$T8bz

    +The Spark RDD API also exposes asynchronous versions of some actions, like `foreachAsync` for `foreach`, which immediately return a `FutureAction` to the caller instead of blocking on completion of the action. This can be used to manage or wait for the asynchronous execution of the action. + + ### Shuffle operations Certain operations within Spark trigger an event known as the shuffle. The shuffle is Spark's From 74ad486dc886d8899c79eae8a78622eff05aeab6 Mon Sep 17 00:00:00 2001 From: sharkd Date: Wed, 13 Jul 2016 11:36:02 +0100 Subject: [PATCH 0959/1470] [MINOR][YARN] Fix code error in yarn-cluster unit test ## What changes were proposed in this pull request? Fix code error in yarn-cluster unit test. ## How was this patch tested? Use exist tests Author: sharkd Closes #14166 from sharkdtu/master. (cherry picked from commit 3d6f679cfe5945a9f72841727342af39e9410e0a) Signed-off-by: Sean Owen --- .../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 874e3045b4058..1ccd7e5993f56 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 @@ -97,7 +97,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { } test("run Spark in yarn-cluster mode with different configurations") { - testBasicYarnApp(true, + testBasicYarnApp(false, Map( "spark.driver.memory" -> "512m", "spark.driver.cores" -> "1", From 5a71a05015ac7aabfb6c4aa8753abc87ead20718 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 13 Jul 2016 11:39:32 +0100 Subject: [PATCH 0960/1470] [SPARK-16440][MLLIB] Undeleted broadcast variables in Word2Vec causing OoM for long runs ## What changes were proposed in this pull request? Unpersist broadcasted vars in Word2Vec.fit for more timely / reliable resource cleanup ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #14153 from srowen/SPARK-16440. (cherry picked from commit 51ade51a9fd64fc2fe651c505a286e6f29f59d40) Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/mllib/feature/Word2Vec.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index f2211df3f943d..6b9c8ee2e3efc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -434,6 +434,9 @@ class Word2Vec extends Serializable with Logging { bcSyn1Global.unpersist(false) } newSentences.unpersist() + expTable.unpersist() + bcVocab.unpersist() + bcVocabHash.unpersist() val wordArray = vocab.map(_.word) new Word2VecModel(wordArray.zipWithIndex.toMap, syn0Global) From 7d9bd951b0b5767ef2c95eb7467f35c9409e7d8c Mon Sep 17 00:00:00 2001 From: oraviv Date: Wed, 13 Jul 2016 14:47:08 +0100 Subject: [PATCH 0961/1470] [SPARK-16469] enhanced simulate multiply ## What changes were proposed in this pull request? We have a use case of multiplying very big sparse matrices. we have about 1000x1000 distributed block matrices multiplication and the simulate multiply goes like O(n^4) (n being 1000). it takes about 1.5 hours. We modified it slightly with classical hashmap and now run in about 30 seconds O(n^2). ## How was this patch tested? We have added a performance test and verified the reduced time. Author: oraviv Closes #14068 from uzadude/master. (cherry picked from commit ea06e4ef34c860219a9aeec81816ef53ada96253) Signed-off-by: Sean Owen --- .../mllib/linalg/distributed/BlockMatrix.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 639295c695255..9782350587061 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -426,16 +426,21 @@ class BlockMatrix @Since("1.3.0") ( partitioner: GridPartitioner): (BlockDestinations, BlockDestinations) = { val leftMatrix = blockInfo.keys.collect() // blockInfo should already be cached val rightMatrix = other.blocks.keys.collect() + + val rightCounterpartsHelper = rightMatrix.groupBy(_._1).mapValues(_.map(_._2)) val leftDestinations = leftMatrix.map { case (rowIndex, colIndex) => - val rightCounterparts = rightMatrix.filter(_._1 == colIndex) - val partitions = rightCounterparts.map(b => partitioner.getPartition((rowIndex, b._2))) + val rightCounterparts = rightCounterpartsHelper.getOrElse(colIndex, Array()) + val partitions = rightCounterparts.map(b => partitioner.getPartition((rowIndex, b))) ((rowIndex, colIndex), partitions.toSet) }.toMap + + val leftCounterpartsHelper = leftMatrix.groupBy(_._2).mapValues(_.map(_._1)) val rightDestinations = rightMatrix.map { case (rowIndex, colIndex) => - val leftCounterparts = leftMatrix.filter(_._2 == rowIndex) - val partitions = leftCounterparts.map(b => partitioner.getPartition((b._1, colIndex))) + val leftCounterparts = leftCounterpartsHelper.getOrElse(rowIndex, Array()) + val partitions = leftCounterparts.map(b => partitioner.getPartition((b, colIndex))) ((rowIndex, colIndex), partitions.toSet) }.toMap + (leftDestinations, rightDestinations) } From 90f0e8132bb40158d6d1b6be77e6b512d837466b Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 13 Jul 2016 13:24:47 -0500 Subject: [PATCH 0962/1470] [SPARK-16435][YARN][MINOR] Add warning log if initialExecutors is less than minExecutors ## What changes were proposed in this pull request? Currently if `spark.dynamicAllocation.initialExecutors` is less than `spark.dynamicAllocation.minExecutors`, Spark will automatically pick the minExecutors without any warning. While in 1.6 Spark will throw exception if configured like this. So here propose to add warning log if these parameters are configured invalidly. ## How was this patch tested? Unit test added to verify the scenario. Author: jerryshao Closes #14149 from jerryshao/SPARK-16435. (cherry picked from commit d8220c1e5e94abbdb9643672b918f0d748206db9) Signed-off-by: Tom Graves --- .../scala/org/apache/spark/util/Utils.scala | 19 ++++++++++++++++++- .../org/apache/spark/util/UtilsSuite.scala | 3 +++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 156cf1748b2a6..a79d1959979a7 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2338,10 +2338,27 @@ private[spark] object Utils extends Logging { * Return the initial number of executors for dynamic allocation. */ def getDynamicAllocationInitialExecutors(conf: SparkConf): Int = { - Seq( + if (conf.get(DYN_ALLOCATION_INITIAL_EXECUTORS) < conf.get(DYN_ALLOCATION_MIN_EXECUTORS)) { + logWarning(s"${DYN_ALLOCATION_INITIAL_EXECUTORS.key} less than " + + s"${DYN_ALLOCATION_MIN_EXECUTORS.key} is invalid, ignoring its setting, " + + "please update your configs.") + } + + if (conf.get(EXECUTOR_INSTANCES).getOrElse(0) < conf.get(DYN_ALLOCATION_MIN_EXECUTORS)) { + logWarning(s"${EXECUTOR_INSTANCES.key} less than " + + s"${DYN_ALLOCATION_MIN_EXECUTORS.key} is invalid, ignoring its setting, " + + "please update your configs.") + } + + val initialExecutors = Seq( conf.get(DYN_ALLOCATION_MIN_EXECUTORS), conf.get(DYN_ALLOCATION_INITIAL_EXECUTORS), conf.get(EXECUTOR_INSTANCES).getOrElse(0)).max + + logInfo(s"Using initial executors = $initialExecutors, max of " + + s"${DYN_ALLOCATION_INITIAL_EXECUTORS.key}, ${DYN_ALLOCATION_MIN_EXECUTORS.key} and " + + s"${EXECUTOR_INSTANCES.key}") + initialExecutors } def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = { 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 f5d0fb00b732d..30952a9458345 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -782,6 +782,9 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { conf.set("spark.dynamicAllocation.initialExecutors", "3")) === 4) assert(Utils.getDynamicAllocationInitialExecutors( // should use initialExecutors conf.set("spark.dynamicAllocation.initialExecutors", "5")) === 5) + assert(Utils.getDynamicAllocationInitialExecutors( // should use minExecutors + conf.set("spark.dynamicAllocation.initialExecutors", "2") + .set("spark.executor.instances", "1")) === 3) } From 2e97f3a08e3b48ce8ad0d669ef844210d0a3d2be Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 13 Jul 2016 12:33:39 -0700 Subject: [PATCH 0963/1470] [SPARK-14812][ML][MLLIB][PYTHON] Experimental, DeveloperApi annotation audit for ML ## What changes were proposed in this pull request? General decisions to follow, except where noted: * spark.mllib, pyspark.mllib: Remove all Experimental annotations. Leave DeveloperApi annotations alone. * spark.ml, pyspark.ml ** Annotate Estimator-Model pairs of classes and companion objects the same way. ** For all algorithms marked Experimental with Since tag <= 1.6, remove Experimental annotation. ** For all algorithms marked Experimental with Since tag = 2.0, leave Experimental annotation. * DeveloperApi annotations are left alone, except where noted. * No changes to which types are sealed. Exceptions where I am leaving items Experimental in spark.ml, pyspark.ml, mainly because the items are new: * Model Summary classes * MLWriter, MLReader, MLWritable, MLReadable * Evaluator and subclasses: There is discussion of changes around evaluating multiple metrics at once for efficiency. * RFormula: Its behavior may need to change slightly to match R in edge cases. * AFTSurvivalRegression * MultilayerPerceptronClassifier DeveloperApi changes: * ml.tree.Node, ml.tree.Split, and subclasses should no longer be DeveloperApi ## How was this patch tested? N/A Note to reviewers: * spark.ml.clustering.LDA underwent significant changes (additional methods), so let me know if you want me to leave it Experimental. * Be careful to check for cases where a class should no longer be Experimental but has an Experimental method, val, or other feature. I did not find such cases, but please verify. Author: Joseph K. Bradley Closes #14147 from jkbradley/experimental-audit. (cherry picked from commit 01f09b161217193b797c8c85969d17054c958615) Signed-off-by: Joseph K. Bradley --- .../scala/org/apache/spark/ml/Pipeline.scala | 6 +- .../DecisionTreeClassifier.scala | 7 +- .../ml/classification/GBTClassifier.scala | 7 +- .../classification/LogisticRegression.scala | 4 -- .../spark/ml/classification/NaiveBayes.scala | 6 +- .../spark/ml/classification/OneVsRest.scala | 7 +- .../RandomForestClassifier.scala | 7 +- .../apache/spark/ml/feature/Binarizer.scala | 4 +- .../apache/spark/ml/feature/Bucketizer.scala | 4 +- .../spark/ml/feature/ChiSqSelector.scala | 6 +- .../spark/ml/feature/CountVectorizer.scala | 6 +- .../org/apache/spark/ml/feature/DCT.scala | 4 +- .../spark/ml/feature/ElementwiseProduct.scala | 4 +- .../apache/spark/ml/feature/HashingTF.scala | 4 +- .../org/apache/spark/ml/feature/IDF.scala | 6 +- .../apache/spark/ml/feature/Interaction.scala | 4 +- .../spark/ml/feature/LabeledPoint.scala | 2 + .../spark/ml/feature/MinMaxScaler.scala | 6 +- .../org/apache/spark/ml/feature/NGram.scala | 4 +- .../apache/spark/ml/feature/Normalizer.scala | 4 +- .../spark/ml/feature/OneHotEncoder.scala | 4 +- .../org/apache/spark/ml/feature/PCA.scala | 7 +- .../ml/feature/PolynomialExpansion.scala | 4 +- .../ml/feature/QuantileDiscretizer.scala | 4 +- .../spark/ml/feature/SQLTransformer.scala | 4 +- .../spark/ml/feature/StandardScaler.scala | 6 +- .../spark/ml/feature/StopWordsRemover.scala | 4 +- .../spark/ml/feature/StringIndexer.scala | 8 +-- .../apache/spark/ml/feature/Tokenizer.scala | 6 +- .../spark/ml/feature/VectorAssembler.scala | 4 +- .../spark/ml/feature/VectorIndexer.scala | 6 +- .../spark/ml/feature/VectorSlicer.scala | 4 +- .../apache/spark/ml/feature/Word2Vec.scala | 7 +- .../org/apache/spark/ml/param/params.scala | 9 +-- .../apache/spark/ml/recommendation/ALS.scala | 8 +-- .../ml/regression/DecisionTreeRegressor.scala | 7 +- .../spark/ml/regression/GBTRegressor.scala | 6 -- .../ml/regression/IsotonicRegression.scala | 6 +- .../ml/regression/LinearRegression.scala | 4 -- .../ml/regression/RandomForestRegressor.scala | 7 +- .../scala/org/apache/spark/ml/tree/Node.scala | 10 +-- .../org/apache/spark/ml/tree/Split.scala | 8 +-- .../spark/ml/tuning/CrossValidator.scala | 6 +- .../spark/ml/tuning/ParamGridBuilder.scala | 4 +- .../ml/tuning/TrainValidationSplit.scala | 6 +- .../mllib/clustering/BisectingKMeans.scala | 8 +-- .../clustering/BisectingKMeansModel.scala | 5 +- .../spark/mllib/clustering/LDAModel.scala | 8 +-- .../spark/mllib/fpm/AssociationRules.scala | 8 +-- .../apache/spark/mllib/fpm/PrefixSpan.scala | 6 +- .../linalg/SingularValueDecomposition.scala | 4 +- .../mllib/optimization/GradientDescent.scala | 4 +- .../spark/mllib/pmml/PMMLExportable.scala | 10 +-- .../spark/mllib/stat/test/StreamingTest.scala | 4 +- .../spark/mllib/stat/test/TestResult.scala | 6 +- .../spark/mllib/tree/configuration/Algo.scala | 4 +- .../spark/mllib/tree/impurity/Entropy.scala | 4 +- .../spark/mllib/tree/impurity/Gini.scala | 4 +- .../spark/mllib/tree/impurity/Impurity.scala | 4 +- .../spark/mllib/tree/impurity/Variance.scala | 4 +- python/pyspark/ml/classification.py | 24 ++----- python/pyspark/ml/feature.py | 64 ------------------- python/pyspark/ml/regression.py | 34 ++-------- python/pyspark/ml/tuning.py | 6 -- python/pyspark/mllib/classification.py | 6 -- python/pyspark/mllib/clustering.py | 16 ----- python/pyspark/mllib/feature.py | 16 ----- python/pyspark/mllib/fpm.py | 8 --- python/pyspark/mllib/linalg/__init__.py | 2 - python/pyspark/mllib/linalg/distributed.py | 14 ---- python/pyspark/mllib/stat/KernelDensity.py | 2 - python/pyspark/mllib/stat/_statistics.py | 4 -- python/pyspark/mllib/tree.py | 12 ---- 73 files changed, 74 insertions(+), 468 deletions(-) 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 a1d08b3a6e780..d18fb697994fa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -27,7 +27,7 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.param.{Param, ParamMap, Params} import org.apache.spark.ml.util._ @@ -78,7 +78,6 @@ abstract class PipelineStage extends Params with Logging { } /** - * :: Experimental :: * A simple pipeline, which acts as an estimator. A Pipeline consists of a sequence of stages, each * of which is either an [[Estimator]] or a [[Transformer]]. When [[Pipeline#fit]] is called, the * stages are executed in order. If a stage is an [[Estimator]], its [[Estimator#fit]] method will @@ -90,7 +89,6 @@ abstract class PipelineStage extends Params with Logging { * an identity transformer. */ @Since("1.2.0") -@Experimental class Pipeline @Since("1.4.0") ( @Since("1.4.0") override val uid: String) extends Estimator[PipelineModel] with MLWritable { @@ -282,11 +280,9 @@ object Pipeline extends MLReadable[Pipeline] { } /** - * :: Experimental :: * Represents a fitted pipeline. */ @Since("1.2.0") -@Experimental class PipelineModel private[ml] ( @Since("1.4.0") override val uid: String, @Since("1.4.0") val stages: Array[Transformer]) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index c65d3d5b54423..082848c9ded5b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap @@ -36,14 +36,12 @@ import org.apache.spark.sql.Dataset /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm * for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. */ @Since("1.4.0") -@Experimental class DecisionTreeClassifier @Since("1.4.0") ( @Since("1.4.0") override val uid: String) extends ProbabilisticClassifier[Vector, DecisionTreeClassifier, DecisionTreeClassificationModel] @@ -127,7 +125,6 @@ class DecisionTreeClassifier @Since("1.4.0") ( } @Since("1.4.0") -@Experimental object DecisionTreeClassifier extends DefaultParamsReadable[DecisionTreeClassifier] { /** Accessor for supported impurities: entropy, gini */ @Since("1.4.0") @@ -138,13 +135,11 @@ object DecisionTreeClassifier extends DefaultParamsReadable[DecisionTreeClassifi } /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. */ @Since("1.4.0") -@Experimental class DecisionTreeClassificationModel private[ml] ( @Since("1.4.0")override val uid: String, @Since("1.4.0")override val rootNode: Node, diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 4e534baddc633..5946a12933ff2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -21,7 +21,7 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.feature.LabeledPoint @@ -40,7 +40,6 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] * learning algorithm for classification. * It supports binary labels, as well as both continuous and categorical features. @@ -57,7 +56,6 @@ import org.apache.spark.sql.types.DoubleType * [https://issues.apache.org/jira/browse/SPARK-4240] */ @Since("1.4.0") -@Experimental class GBTClassifier @Since("1.4.0") ( @Since("1.4.0") override val uid: String) extends Predictor[Vector, GBTClassifier, GBTClassificationModel] @@ -149,7 +147,6 @@ class GBTClassifier @Since("1.4.0") ( } @Since("1.4.0") -@Experimental object GBTClassifier extends DefaultParamsReadable[GBTClassifier] { /** Accessor for supported loss settings: logistic */ @@ -161,7 +158,6 @@ object GBTClassifier extends DefaultParamsReadable[GBTClassifier] { } /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] * model for classification. * It supports binary labels, as well as both continuous and categorical features. @@ -171,7 +167,6 @@ object GBTClassifier extends DefaultParamsReadable[GBTClassifier] { * @param _treeWeights Weights for the decision trees in the ensemble. */ @Since("1.6.0") -@Experimental class GBTClassificationModel private[ml]( @Since("1.6.0") override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], 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 9c9f5ced4e35c..e157bdeb5b7ee 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 @@ -151,13 +151,11 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas } /** - * :: Experimental :: * Logistic regression. * Currently, this class only supports binary classification. It will support multiclass * in the future. */ @Since("1.2.0") -@Experimental class LogisticRegression @Since("1.2.0") ( @Since("1.4.0") override val uid: String) extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] @@ -475,11 +473,9 @@ object LogisticRegression extends DefaultParamsReadable[LogisticRegression] { } /** - * :: Experimental :: * Model produced by [[LogisticRegression]]. */ @Since("1.4.0") -@Experimental class LogisticRegressionModel private[spark] ( @Since("1.4.0") override val uid: String, @Since("2.0.0") val coefficients: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index c99ae30155e3f..ab977c8802e36 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.classification import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} @@ -63,7 +63,6 @@ private[ml] trait NaiveBayesParams extends PredictorParams { } /** - * :: Experimental :: * Naive Bayes Classifiers. * It supports both Multinomial NB * ([[http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html]]) @@ -74,7 +73,6 @@ private[ml] trait NaiveBayesParams extends PredictorParams { * The input feature values must be nonnegative. */ @Since("1.5.0") -@Experimental class NaiveBayes @Since("1.5.0") ( @Since("1.5.0") override val uid: String) extends ProbabilisticClassifier[Vector, NaiveBayes, NaiveBayesModel] @@ -121,14 +119,12 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { } /** - * :: Experimental :: * Model produced by [[NaiveBayes]] * @param pi log of class priors, whose dimension is C (number of classes) * @param theta log of class conditional probabilities, whose dimension is C (number of classes) * by D (number of features) */ @Since("1.5.0") -@Experimental class NaiveBayesModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("2.0.0") val pi: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 047a378b79aa7..f4ab0a074c420 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -29,7 +29,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.Vector @@ -117,7 +117,6 @@ private[ml] object OneVsRestParams extends ClassifierTypeTrait { } /** - * :: Experimental :: * Model produced by [[OneVsRest]]. * This stores the models resulting from training k binary classifiers: one for each class. * Each example is scored against all k models, and the model with the highest score @@ -130,7 +129,6 @@ private[ml] object OneVsRestParams extends ClassifierTypeTrait { * (taking label 0). */ @Since("1.4.0") -@Experimental final class OneVsRestModel private[ml] ( @Since("1.4.0") override val uid: String, private[ml] val labelMetadata: Metadata, @@ -260,8 +258,6 @@ object OneVsRestModel extends MLReadable[OneVsRestModel] { } /** - * :: Experimental :: - * * Reduction of Multiclass Classification to Binary Classification. * Performs reduction using one against all strategy. * For a multiclass classification with k classes, train k models (one per class). @@ -269,7 +265,6 @@ object OneVsRestModel extends MLReadable[OneVsRestModel] { * is picked to label the example. */ @Since("1.4.0") -@Experimental final class OneVsRest @Since("1.4.0") ( @Since("1.4.0") override val uid: String) extends Estimator[OneVsRestModel] with OneVsRestParams with MLWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 9a26a5c5b1431..4ab132e5f2940 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.classification import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap @@ -36,14 +36,12 @@ import org.apache.spark.sql.functions._ /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] learning algorithm for * classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. */ @Since("1.4.0") -@Experimental class RandomForestClassifier @Since("1.4.0") ( @Since("1.4.0") override val uid: String) extends ProbabilisticClassifier[Vector, RandomForestClassifier, RandomForestClassificationModel] @@ -124,7 +122,6 @@ class RandomForestClassifier @Since("1.4.0") ( } @Since("1.4.0") -@Experimental object RandomForestClassifier extends DefaultParamsReadable[RandomForestClassifier] { /** Accessor for supported impurity settings: entropy, gini */ @Since("1.4.0") @@ -140,7 +137,6 @@ object RandomForestClassifier extends DefaultParamsReadable[RandomForestClassifi } /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] model for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. @@ -149,7 +145,6 @@ object RandomForestClassifier extends DefaultParamsReadable[RandomForestClassifi * Warning: These have null parents. */ @Since("1.4.0") -@Experimental class RandomForestClassificationModel private[ml] ( @Since("1.5.0") override val uid: String, private val _trees: Array[DecisionTreeClassificationModel], diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index fa9634fdfa7e9..2b0862c60fdf7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable.ArrayBuilder -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.BinaryAttribute import org.apache.spark.ml.linalg._ @@ -31,10 +31,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** - * :: Experimental :: * Binarize a column of continuous features given a threshold. */ -@Experimental @Since("1.4.0") final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index caffc39e2be14..100d9e7f6cbcc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import java.{util => ju} import org.apache.spark.SparkException -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Model import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param._ @@ -31,10 +31,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} /** - * :: Experimental :: * `Bucketizer` maps a column of continuous features to a column of feature buckets. */ -@Experimental @Since("1.4.0") final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Model[Bucketizer] with HasInputCol with HasOutputCol with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 712634dffbf17..bd053e886f171 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.attribute.{AttributeGroup, _} import org.apache.spark.ml.linalg.{Vector, VectorUDT} @@ -57,11 +57,9 @@ private[feature] trait ChiSqSelectorParams extends Params } /** - * :: Experimental :: * Chi-Squared feature selection, which selects categorical features to use for predicting a * categorical label. */ -@Experimental @Since("1.6.0") final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Estimator[ChiSqSelectorModel] with ChiSqSelectorParams with DefaultParamsWritable { @@ -116,10 +114,8 @@ object ChiSqSelector extends DefaultParamsReadable[ChiSqSelector] { } /** - * :: Experimental :: * Model fitted by [[ChiSqSelector]]. */ -@Experimental @Since("1.6.0") final class ChiSqSelectorModel private[ml] ( @Since("1.6.0") override val uid: String, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 96e6f1c512e90..6299f74a6bf96 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{Vectors, VectorUDT} @@ -116,10 +116,8 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit } /** - * :: Experimental :: * Extracts a vocabulary from document collections and generates a [[CountVectorizerModel]]. */ -@Experimental @Since("1.5.0") class CountVectorizer @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Estimator[CountVectorizerModel] with CountVectorizerParams with DefaultParamsWritable { @@ -201,11 +199,9 @@ object CountVectorizer extends DefaultParamsReadable[CountVectorizer] { } /** - * :: Experimental :: * Converts a text document to a sparse vector of token counts. * @param vocabulary An Array over terms. Only the terms in the vocabulary will be counted. */ -@Experimental @Since("1.5.0") class CountVectorizerModel( @Since("1.5.0") override val uid: String, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 9605145e12c27..6ff36b35ca4c1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import edu.emory.mathcs.jtransforms.dct._ -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.BooleanParam @@ -27,7 +27,6 @@ import org.apache.spark.ml.util._ import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * A feature transformer that takes the 1D discrete cosine transform of a real vector. No zero * padding is performed on the input vector. * It returns a real vector of the same length representing the DCT. The return vector is scaled @@ -35,7 +34,6 @@ import org.apache.spark.sql.types.DataType * * More information on [[https://en.wikipedia.org/wiki/Discrete_cosine_transform#DCT-II Wikipedia]]. */ -@Experimental @Since("1.5.0") class DCT @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends UnaryTransformer[Vector, Vector, DCT] with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index d07833e5805df..f860b3a787b4d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.Param @@ -27,12 +27,10 @@ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a * provided "weight" vector. In other words, it scales each column of the dataset by a scalar * multiplier. */ -@Experimental @Since("1.4.0") class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[Vector, Vector, ElementwiseProduct] with DefaultParamsWritable { 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 6ca7336cd048e..a8792a35ff4ae 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 @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.param._ @@ -29,7 +29,6 @@ import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{ArrayType, StructType} /** - * :: Experimental :: * Maps a sequence of terms to their term frequencies using the hashing trick. * Currently we use Austin Appleby's MurmurHash 3 algorithm (MurmurHash3_x86_32) * to calculate the hash code value for the term object. @@ -37,7 +36,6 @@ import org.apache.spark.sql.types.{ArrayType, StructType} * it is advisable to use a power of two as the numFeatures parameter; * otherwise the features will not be mapped evenly to the columns. */ -@Experimental @Since("1.2.0") class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 5d6287f0e3f15..6386dd8a10801 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ @@ -61,10 +61,8 @@ private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol } /** - * :: Experimental :: * Compute the Inverse Document Frequency (IDF) given a collection of documents. */ -@Experimental @Since("1.4.0") final class IDF @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Estimator[IDFModel] with IDFBase with DefaultParamsWritable { @@ -111,10 +109,8 @@ object IDF extends DefaultParamsReadable[IDF] { } /** - * :: Experimental :: * Model fitted by [[IDF]]. */ -@Experimental @Since("1.4.0") class IDFModel private[ml] ( @Since("1.4.0") override val uid: String, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index dca28b5c5d34f..7b11f86279b9d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkException -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.attribute._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -32,7 +32,6 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** - * :: Experimental :: * Implements the feature interaction transform. This transformer takes in Double and Vector type * columns and outputs a flattened vector of their feature interactions. To handle interaction, * we first one-hot encode any nominal features. Then, a vector of the feature cross-products is @@ -42,7 +41,6 @@ import org.apache.spark.sql.types._ * `Vector(6, 8)` if all input features were numeric. If the first feature was instead nominal * with four categories, the output would then be `Vector(0, 0, 0, 0, 3, 4, 0, 0)`. */ -@Experimental @Since("1.6.0") class Interaction @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Transformer with HasInputCols with HasOutputCol with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala index f7f1d42039599..6cefa7086c881 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala @@ -23,6 +23,8 @@ import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.linalg.Vector /** + * :: Experimental :: + * * Class that represents the features and labels of a data point. * * @param label Label for this data point. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index d5ad5abced469..7b03f0c0f341e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamMap, Params} @@ -74,7 +74,6 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H } /** - * :: Experimental :: * Rescale each feature individually to a common range [min, max] linearly using column summary * statistics, which is also known as min-max normalization or Rescaling. The rescaled value for * feature E is calculated as, @@ -85,7 +84,6 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H * Note that since zero values will probably be transformed to non-zero values, output of the * transformer will be DenseVector even for sparse input. */ -@Experimental @Since("1.5.0") class MinMaxScaler @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Estimator[MinMaxScalerModel] with MinMaxScalerParams with DefaultParamsWritable { @@ -138,7 +136,6 @@ object MinMaxScaler extends DefaultParamsReadable[MinMaxScaler] { } /** - * :: Experimental :: * Model fitted by [[MinMaxScaler]]. * * @param originalMin min value for each original column during fitting @@ -146,7 +143,6 @@ object MinMaxScaler extends DefaultParamsReadable[MinMaxScaler] { * * TODO: The transformer does not yet set the metadata in the output column (SPARK-8529). */ -@Experimental @Since("1.5.0") class MinMaxScalerModel private[ml] ( @Since("1.5.0") override val uid: String, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index 9c1f1ad443bba..4463aea0097e2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -17,14 +17,13 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param._ import org.apache.spark.ml.util._ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} /** - * :: Experimental :: * A feature transformer that converts the input array of strings into an array of n-grams. Null * values in the input array are ignored. * It returns an array of n-grams where each n-gram is represented by a space-separated string of @@ -34,7 +33,6 @@ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} * When the input array length is less than n (number of elements per n-gram), no n-grams are * returned. */ -@Experimental @Since("1.5.0") class NGram @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends UnaryTransformer[Seq[String], Seq[String], NGram] with DefaultParamsWritable { 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 f9cbad90c9f3f..eb0690058013f 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 @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamValidators} @@ -27,10 +27,8 @@ import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * Normalize a vector to have unit norm using the given p-norm. */ -@Experimental @Since("1.4.0") class Normalizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[Vector, Vector, Normalizer] with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 01828ede6bc69..8b04b5de6fd2b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.Vectors @@ -29,7 +29,6 @@ import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} /** - * :: Experimental :: * A one-hot encoder that maps a column of category indices to a column of binary vectors, with * at most a single one-value per row that indicates the input category index. * For example with 5 categories, an input value of 2.0 would map to an output vector of @@ -42,7 +41,6 @@ import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} * * @see [[StringIndexer]] for converting categorical values into category indices */ -@Experimental @Since("1.4.0") class OneHotEncoder @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index ef8b08545db2a..6b913480fdc28 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ @@ -59,12 +59,11 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC } } + /** - * :: Experimental :: * PCA trains a model to project vectors to a lower dimensional space of the top [[PCA!.k]] * principal components. */ -@Experimental @Since("1.5.0") class PCA @Since("1.5.0") ( @Since("1.5.0") override val uid: String) @@ -116,14 +115,12 @@ object PCA extends DefaultParamsReadable[PCA] { } /** - * :: Experimental :: * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. * * @param pc A principal components Matrix. Each column is one principal component. * @param explainedVariance A vector of proportions of variance explained by * each principal component. */ -@Experimental @Since("1.5.0") class PCAModel private[ml] ( @Since("1.5.0") override val uid: String, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 7b35fdeaf40c6..72fb35bd79ad7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} @@ -27,14 +27,12 @@ import org.apache.spark.ml.util._ import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * Perform feature expansion in a polynomial space. As said in wikipedia of Polynomial Expansion, * which is available at [[http://en.wikipedia.org/wiki/Polynomial_expansion]], "In mathematics, an * expansion of a product of sums expresses it as a sum of products by using the fact that * multiplication distributes over addition". Take a 2-variable feature vector as an example: * `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. */ -@Experimental @Since("1.4.0") class PolynomialExpansion @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[Vector, Vector, PolynomialExpansion] with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 96b8e7d9f7faf..9a636bd8a5e46 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.attribute.NominalAttribute @@ -64,7 +64,6 @@ private[feature] trait QuantileDiscretizerBase extends Params } /** - * :: Experimental :: * `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned * categorical features. The number of bins can be set using the `numBuckets` parameter. * The bin ranges are chosen using an approximate algorithm (see the documentation for @@ -73,7 +72,6 @@ private[feature] trait QuantileDiscretizerBase extends Params * `relativeError` parameter. The lower and upper bin bounds will be `-Infinity` and `+Infinity`, * covering all real values. */ -@Experimental @Since("1.6.0") final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Estimator[Bucketizer] with QuantileDiscretizerBase with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index b8715746fee5b..289037640fd45 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.Transformer import org.apache.spark.ml.util._ @@ -25,7 +25,6 @@ import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.types.StructType /** - * :: Experimental :: * Implements the transformations which are defined by SQL statement. * Currently we only support SQL syntax like 'SELECT ... FROM __THIS__ ...' * where '__THIS__' represents the underlying table of the input dataset. @@ -37,7 +36,6 @@ import org.apache.spark.sql.types.StructType * - SELECT a, SQRT(b) AS b_sqrt FROM __THIS__ where a > 5 * - SELECT a, b, SUM(c) AS c_sum FROM __THIS__ GROUP BY a, b */ -@Experimental @Since("1.6.0") class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Transformer with DefaultParamsWritable { 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 b4be95494fd10..2494cf51a2bd6 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 @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ @@ -76,7 +76,6 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with } /** - * :: Experimental :: * Standardizes features by removing the mean and scaling to unit variance using column summary * statistics on the samples in the training set. * @@ -85,7 +84,6 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with * corrected sample standard deviation]], * which is computed as the square root of the unbiased sample variance. */ -@Experimental @Since("1.2.0") class StandardScaler @Since("1.4.0") ( @Since("1.4.0") override val uid: String) @@ -138,13 +136,11 @@ object StandardScaler extends DefaultParamsReadable[StandardScaler] { } /** - * :: Experimental :: * Model fitted by [[StandardScaler]]. * * @param std Standard deviation of the StandardScalerModel * @param mean Mean of the StandardScalerModel */ -@Experimental @Since("1.2.0") class StandardScalerModel private[ml] ( @Since("1.4.0") override val uid: String, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala index 1a6f42f773cd7..666070037cdd8 100755 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.param.{BooleanParam, ParamMap, StringArrayParam} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} @@ -27,12 +27,10 @@ import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{ArrayType, StringType, StructType} /** - * :: Experimental :: * A feature transformer that filters out stop words from input. * Note: null values from input array are preserved unless adding null to stopWords explicitly. * @see [[http://en.wikipedia.org/wiki/Stop_words]] */ -@Experimental @Since("1.5.0") class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { 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 028e540fe5356..fe79e2ec808ac 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 @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model, Transformer} import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} import org.apache.spark.ml.param._ @@ -55,7 +55,6 @@ private[feature] trait StringIndexerBase extends Params with HasInputCol with Ha } /** - * :: Experimental :: * A label indexer that maps a string column of labels to an ML column of label indices. * If the input column is numeric, we cast it to string and index the string values. * The indices are in [0, numLabels), ordered by label frequencies. @@ -63,7 +62,6 @@ private[feature] trait StringIndexerBase extends Params with HasInputCol with Ha * * @see [[IndexToString]] for the inverse transformation */ -@Experimental @Since("1.4.0") class StringIndexer @Since("1.4.0") ( @Since("1.4.0") override val uid: String) extends Estimator[StringIndexerModel] @@ -112,7 +110,6 @@ object StringIndexer extends DefaultParamsReadable[StringIndexer] { } /** - * :: Experimental :: * Model fitted by [[StringIndexer]]. * * NOTE: During transformation, if the input column does not exist, @@ -121,7 +118,6 @@ object StringIndexer extends DefaultParamsReadable[StringIndexer] { * * @param labels Ordered list of labels, corresponding to indices to be assigned. */ -@Experimental @Since("1.4.0") class StringIndexerModel ( @Since("1.4.0") override val uid: String, @@ -250,7 +246,6 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { } /** - * :: Experimental :: * A [[Transformer]] that maps a column of indices back to a new column of corresponding * string values. * The index-string mapping is either from the ML attributes of the input column, @@ -258,7 +253,6 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { * * @see [[StringIndexer]] for converting strings into indices */ -@Experimental @Since("1.5.0") class IndexToString private[ml] (@Since("1.5.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { 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 010c948749f3b..45d8fa94a8f8f 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 @@ -17,19 +17,17 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param._ import org.apache.spark.ml.util._ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} /** - * :: Experimental :: * A tokenizer that converts the input string to lowercase and then splits it by white spaces. * * @see [[RegexTokenizer]] */ -@Experimental @Since("1.2.0") class Tokenizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[String, Seq[String], Tokenizer] with DefaultParamsWritable { @@ -59,13 +57,11 @@ object Tokenizer extends DefaultParamsReadable[Tokenizer] { } /** - * :: Experimental :: * A regex based tokenizer that extracts tokens either by using the provided regex pattern to split * the text (default) or repeatedly matching the regex (if `gaps` is false). * Optional parameters also allow filtering tokens using a minimal length. * It returns an array of strings that can be empty. */ -@Experimental @Since("1.4.0") class RegexTokenizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends UnaryTransformer[String, Seq[String], RegexTokenizer] with DefaultParamsWritable { 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 4939dabd987ec..142a2ae44c690 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 @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkException -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute, UnresolvedAttribute} import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} @@ -32,10 +32,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** - * :: Experimental :: * A feature transformer that merges multiple columns into a vector column. */ -@Experimental @Since("1.4.0") class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer with HasInputCols with HasOutputCol with DefaultParamsWritable { 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 5656a9f979fc1..d1a5c2e82581e 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 @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, VectorUDT} @@ -59,7 +59,6 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu } /** - * :: Experimental :: * Class for indexing categorical feature columns in a dataset of [[Vector]]. * * This has 2 usage modes: @@ -93,7 +92,6 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu * - Add warning if a categorical feature has only 1 category. * - Add option for allowing unknown categories. */ -@Experimental @Since("1.4.0") class VectorIndexer @Since("1.4.0") ( @Since("1.4.0") override val uid: String) @@ -247,7 +245,6 @@ object VectorIndexer extends DefaultParamsReadable[VectorIndexer] { } /** - * :: Experimental :: * Model fitted by [[VectorIndexer]]. Transform categorical features to use 0-based indices * instead of their original values. * - Categorical features are mapped to indices. @@ -263,7 +260,6 @@ object VectorIndexer extends DefaultParamsReadable[VectorIndexer] { * 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. */ -@Experimental @Since("1.4.0") class VectorIndexerModel private[ml] ( @Since("1.4.0") override val uid: String, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index 6769e490c51c7..966ccb85d0e0e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup} import org.apache.spark.ml.linalg._ @@ -29,7 +29,6 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType /** - * :: Experimental :: * This class takes a feature vector and outputs a new feature vector with a subarray of the * original features. * @@ -40,7 +39,6 @@ import org.apache.spark.sql.types.StructType * The output vector will order features with the selected indices first (in the order given), * followed by the selected names (in the order given). */ -@Experimental @Since("1.5.0") final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 0cac3fa2d7e57..c2b434c3d5cb1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -19,8 +19,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ @@ -115,11 +114,9 @@ private[feature] trait Word2VecBase extends Params } /** - * :: Experimental :: * Word2Vec trains a model of `Map(String, Vector)`, i.e. transforms a word into a code for further * natural language processing or machine learning process. */ -@Experimental @Since("1.4.0") final class Word2Vec @Since("1.4.0") ( @Since("1.4.0") override val uid: String) @@ -202,10 +199,8 @@ object Word2Vec extends DefaultParamsReadable[Word2Vec] { } /** - * :: Experimental :: * Model fitted by [[Word2Vec]]. */ -@Experimental @Since("1.4.0") class Word2VecModel private[ml] ( @Since("1.4.0") override val uid: String, 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 ecec61a72f823..e7780cf1c39f9 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 @@ -28,9 +28,9 @@ import scala.collection.JavaConverters._ import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} -import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.ml.linalg.JsonVectorConverter +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.util.Identifiable /** @@ -510,11 +510,9 @@ class IntArrayParam(parent: Params, name: String, doc: String, isValid: Array[In } /** - * :: Experimental :: * A param and its value. */ @Since("1.2.0") -@Experimental case class ParamPair[T] @Since("1.2.0") ( @Since("1.2.0") param: Param[T], @Since("1.2.0") value: T) { @@ -797,11 +795,9 @@ trait Params extends Identifiable with Serializable { abstract class JavaParams extends Params /** - * :: Experimental :: * A param to value map. */ @Since("1.2.0") -@Experimental final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) extends Serializable { @@ -952,7 +948,6 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) } @Since("1.2.0") -@Experimental object ParamMap { /** 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 5dc2433e55c39..a2c4c2691190e 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 @@ -26,12 +26,12 @@ import scala.util.{Sorting, Try} import scala.util.hashing.byteswap64 import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ @@ -222,14 +222,12 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w } /** - * :: Experimental :: * Model fitted by ALS. * * @param rank rank of the matrix factorization model * @param userFactors a DataFrame that stores user factors in two columns: `id` and `features` * @param itemFactors a DataFrame that stores item factors in two columns: `id` and `features` */ -@Experimental @Since("1.3.0") class ALSModel private[ml] ( @Since("1.4.0") override val uid: String, @@ -333,7 +331,6 @@ object ALSModel extends MLReadable[ALSModel] { } /** - * :: Experimental :: * Alternating Least Squares (ALS) matrix factorization. * * ALS attempts to estimate the ratings matrix `R` as the product of two lower-rank matrices, @@ -362,7 +359,6 @@ object ALSModel extends MLReadable[ALSModel] { * indicated user * preferences rather than explicit ratings given to items. */ -@Experimental @Since("1.3.0") class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] with ALSParams with DefaultParamsWritable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 7ff6d0afd55c2..ebc6c12ddcf92 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.Vector @@ -38,13 +38,11 @@ import org.apache.spark.sql.functions._ /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm * for regression. * It supports both continuous and categorical features. */ @Since("1.4.0") -@Experimental class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Predictor[Vector, DecisionTreeRegressor, DecisionTreeRegressionModel] with DecisionTreeRegressorParams with DefaultParamsWritable { @@ -125,7 +123,6 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S } @Since("1.4.0") -@Experimental object DecisionTreeRegressor extends DefaultParamsReadable[DecisionTreeRegressor] { /** Accessor for supported impurities: variance */ final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities @@ -135,13 +132,11 @@ object DecisionTreeRegressor extends DefaultParamsReadable[DecisionTreeRegressor } /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression. * It supports both continuous and categorical features. * @param rootNode Root of the decision tree */ @Since("1.4.0") -@Experimental class DecisionTreeRegressionModel private[ml] ( override val uid: String, override val rootNode: Node, diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 6223555504d71..ce355938ec1c7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] * learning algorithm for regression. * It supports both continuous and categorical features. @@ -56,7 +55,6 @@ import org.apache.spark.sql.functions._ * [https://issues.apache.org/jira/browse/SPARK-4240] */ @Since("1.4.0") -@Experimental class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Predictor[Vector, GBTRegressor, GBTRegressionModel] with GBTRegressorParams with DefaultParamsWritable with Logging { @@ -135,7 +133,6 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) } @Since("1.4.0") -@Experimental object GBTRegressor extends DefaultParamsReadable[GBTRegressor] { /** Accessor for supported loss settings: squared (L2), absolute (L1) */ @@ -147,8 +144,6 @@ object GBTRegressor extends DefaultParamsReadable[GBTRegressor] { } /** - * :: Experimental :: - * * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] * model for regression. * It supports both continuous and categorical features. @@ -156,7 +151,6 @@ object GBTRegressor extends DefaultParamsReadable[GBTRegressor] { * @param _treeWeights Weights for the decision trees in the ensemble. */ @Since("1.4.0") -@Experimental class GBTRegressionModel private[ml]( override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 9b9429a328d08..35396446edc16 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.regression import org.apache.hadoop.fs.Path -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} @@ -120,7 +120,6 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures } /** - * :: Experimental :: * Isotonic regression. * * Currently implemented using parallelized pool adjacent violators algorithm. @@ -129,7 +128,6 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures * Uses [[org.apache.spark.mllib.regression.IsotonicRegression]]. */ @Since("1.5.0") -@Experimental class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Estimator[IsotonicRegressionModel] with IsotonicRegressionBase with DefaultParamsWritable { @@ -192,7 +190,6 @@ object IsotonicRegression extends DefaultParamsReadable[IsotonicRegression] { } /** - * :: Experimental :: * Model fitted by IsotonicRegression. * Predicts using a piecewise linear function. * @@ -202,7 +199,6 @@ object IsotonicRegression extends DefaultParamsReadable[IsotonicRegression] { * model trained by [[org.apache.spark.mllib.regression.IsotonicRegression]]. */ @Since("1.5.0") -@Experimental class IsotonicRegressionModel private[ml] ( override val uid: String, private val oldModel: MLlibIsotonicRegressionModel) 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 6b82ae14e1cef..c57e9eb0bfd15 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 @@ -54,7 +54,6 @@ private[regression] trait LinearRegressionParams extends PredictorParams with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver /** - * :: Experimental :: * Linear regression. * * The learning objective is to minimize the squared error, with regularization. @@ -68,7 +67,6 @@ private[regression] trait LinearRegressionParams extends PredictorParams * - L2 + L1 (elastic net) */ @Since("1.3.0") -@Experimental class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) extends Regressor[Vector, LinearRegression, LinearRegressionModel] with LinearRegressionParams with DefaultParamsWritable with Logging { @@ -387,11 +385,9 @@ object LinearRegression extends DefaultParamsReadable[LinearRegression] { } /** - * :: Experimental :: * Model produced by [[LinearRegression]]. */ @Since("1.3.0") -@Experimental class LinearRegressionModel private[ml] ( @Since("1.4.0") override val uid: String, @Since("2.0.0") val coefficients: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 4f4d3d27841da..0ad00aa6f9280 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.regression import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.Vector @@ -37,12 +37,10 @@ import org.apache.spark.sql.functions._ /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] learning algorithm for regression. * It supports both continuous and categorical features. */ @Since("1.4.0") -@Experimental class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Predictor[Vector, RandomForestRegressor, RandomForestRegressionModel] with RandomForestRegressorParams with DefaultParamsWritable { @@ -118,7 +116,6 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S } @Since("1.4.0") -@Experimental object RandomForestRegressor extends DefaultParamsReadable[RandomForestRegressor]{ /** Accessor for supported impurity settings: variance */ @Since("1.4.0") @@ -135,7 +132,6 @@ object RandomForestRegressor extends DefaultParamsReadable[RandomForestRegressor } /** - * :: Experimental :: * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] model for regression. * It supports both continuous and categorical features. * @@ -143,7 +139,6 @@ object RandomForestRegressor extends DefaultParamsReadable[RandomForestRegressor * @param numFeatures Number of features used by this model */ @Since("1.4.0") -@Experimental class RandomForestRegressionModel private[ml] ( override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index d5e5c454605b7..8144bcb7d46f1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -17,17 +17,14 @@ package org.apache.spark.ml.tree -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.{ImpurityStats, InformationGainStats => OldInformationGainStats, Node => OldNode, Predict => OldPredict} /** - * :: DeveloperApi :: * Decision tree node interface. */ -@DeveloperApi sealed abstract class Node extends Serializable { // TODO: Add aggregate stats (once available). This will happen after we move the DecisionTree @@ -109,12 +106,10 @@ private[ml] object Node { } /** - * :: DeveloperApi :: * Decision tree leaf node. * @param prediction Prediction this node makes * @param impurity Impurity measure at this node (for training data) */ -@DeveloperApi class LeafNode private[ml] ( override val prediction: Double, override val impurity: Double, @@ -147,7 +142,6 @@ class LeafNode private[ml] ( } /** - * :: DeveloperApi :: * Internal Decision Tree node. * @param prediction Prediction this node would make if it were a leaf node * @param impurity Impurity measure at this node (for training data) @@ -157,7 +151,6 @@ class LeafNode private[ml] ( * @param rightChild Right-hand child node * @param split Information about the test used to split to the left or right child. */ -@DeveloperApi class InternalNode private[ml] ( override val prediction: Double, override val impurity: Double, @@ -167,6 +160,9 @@ class InternalNode private[ml] ( val split: Split, override private[ml] val impurityStats: ImpurityCalculator) extends Node { + // Note to developers: The constructor argument impurityStats should be reconsidered before we + // make the constructor public. We may be able to improve the representation. + override def toString: String = { s"InternalNode(prediction = $prediction, impurity = $impurity, split = $split)" } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index 9704e15cd838f..47fe3524f2296 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -19,18 +19,16 @@ package org.apache.spark.ml.tree import java.util.Objects -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} import org.apache.spark.mllib.tree.model.{Split => OldSplit} /** - * :: DeveloperApi :: * Interface for a "Split," which specifies a test made at a decision tree node * to choose the left or right path. */ -@DeveloperApi sealed trait Split extends Serializable { /** Index of feature which this split tests */ @@ -67,14 +65,12 @@ private[tree] object Split { } /** - * :: DeveloperApi :: * Split which tests a categorical feature. * @param featureIndex Index of the feature to test * @param _leftCategories If the feature value is in this set of categories, then the split goes * left. Otherwise, it goes right. * @param numCategories Number of categories for this feature. */ -@DeveloperApi class CategoricalSplit private[ml] ( override val featureIndex: Int, _leftCategories: Array[Double], @@ -153,13 +149,11 @@ class CategoricalSplit private[ml] ( } /** - * :: DeveloperApi :: * Split which tests a continuous feature. * @param featureIndex Index of the feature to test * @param threshold If the feature value is <= this threshold, then the split goes left. * Otherwise, it goes right. */ -@DeveloperApi class ContinuousSplit private[ml] (override val featureIndex: Int, val threshold: Double) extends Split { 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 7d42da4a2ffae..520557849b9e2 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 @@ -25,7 +25,7 @@ import com.github.fommil.netlib.F2jBLAS import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.evaluation.Evaluator @@ -55,11 +55,9 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { } /** - * :: Experimental :: * K-fold cross validation. */ @Since("1.2.0") -@Experimental class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) extends Estimator[CrossValidatorModel] with CrossValidatorParams with MLWritable with Logging { @@ -190,7 +188,6 @@ object CrossValidator extends MLReadable[CrossValidator] { } /** - * :: Experimental :: * Model from k-fold cross validation. * * @param bestModel The best model selected from k-fold cross validation. @@ -198,7 +195,6 @@ object CrossValidator extends MLReadable[CrossValidator] { * [[CrossValidator.estimatorParamMaps]], in the corresponding order. */ @Since("1.2.0") -@Experimental class CrossValidatorModel private[ml] ( @Since("1.4.0") override val uid: String, @Since("1.2.0") val bestModel: Model[_], diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala index 7d12f447f7963..d369e7a61cdc5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala @@ -20,15 +20,13 @@ package org.apache.spark.ml.tuning import scala.annotation.varargs import scala.collection.mutable -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.ml.param._ /** - * :: Experimental :: * Builder for a param grid used in grid search-based model selection. */ @Since("1.2.0") -@Experimental class ParamGridBuilder @Since("1.2.0") { private val paramGrid = mutable.Map.empty[Param[_], Iterable[_]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index f6f2bad401a17..0fdba1cb8814a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -25,7 +25,7 @@ import scala.language.existentials import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator @@ -54,14 +54,12 @@ private[ml] trait TrainValidationSplitParams extends ValidatorParams { } /** - * :: Experimental :: * Validation for hyper-parameter tuning. * Randomly splits the input dataset into train and validation sets, * and uses evaluation metric on the validation set to select the best model. * Similar to [[CrossValidator]], but only splits the set once. */ @Since("1.5.0") -@Experimental class TrainValidationSplit @Since("1.5.0") (@Since("1.5.0") override val uid: String) extends Estimator[TrainValidationSplitModel] with TrainValidationSplitParams with MLWritable with Logging { @@ -188,7 +186,6 @@ object TrainValidationSplit extends MLReadable[TrainValidationSplit] { } /** - * :: Experimental :: * Model from train validation split. * * @param uid Id. @@ -196,7 +193,6 @@ object TrainValidationSplit extends MLReadable[TrainValidationSplit] { * @param validationMetrics Evaluated validation metrics. */ @Since("1.5.0") -@Experimental class TrainValidationSplitModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("1.5.0") val bestModel: Model[_], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index 91edcf2a7925b..f1664ce4ab3f8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -22,7 +22,7 @@ import java.util.Random import scala.annotation.tailrec import scala.collection.mutable -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} @@ -31,8 +31,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** - * :: Experimental :: - * * A bisecting k-means algorithm based on the paper "A comparison of document clustering techniques" * by Steinbach, Karypis, and Kumar, with modification to fit Spark. * The algorithm starts from a single cluster that contains all points. @@ -54,7 +52,6 @@ import org.apache.spark.storage.StorageLevel * KDD Workshop on Text Mining, 2000.]] */ @Since("1.6.0") -@Experimental class BisectingKMeans private ( private var k: Int, private var maxIterations: Int, @@ -398,8 +395,6 @@ private object BisectingKMeans extends Serializable { } /** - * :: Experimental :: - * * Represents a node in a clustering tree. * * @param index node index, negative for internal nodes and non-negative for leaf nodes @@ -411,7 +406,6 @@ private object BisectingKMeans extends Serializable { * @param children children nodes */ @Since("1.6.0") -@Experimental private[clustering] class ClusteringTreeNode private[clustering] ( val index: Int, val size: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index 11fd940b8b205..8438015ccecea 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -23,7 +23,7 @@ import org.json4s.jackson.JsonMethods._ import org.json4s.JsonDSL._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector @@ -32,8 +32,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession} /** - * :: Experimental :: - * * Clustering model produced by [[BisectingKMeans]]. * The prediction is done level-by-level from the root node to a leaf node, and at each node among * its children the closest to the input point is selected. @@ -41,7 +39,6 @@ import org.apache.spark.sql.{Row, SparkSession} * @param root the root node of the clustering tree */ @Since("1.6.0") -@Experimental class BisectingKMeansModel private[clustering] ( private[clustering] val root: ClusteringTreeNode ) extends Serializable with Saveable with Logging { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index d295826300419..9ebba1de0dad4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -25,7 +25,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} import org.apache.spark.graphx.{Edge, EdgeContext, Graph, VertexId} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} @@ -426,13 +426,10 @@ class LocalLDAModel private[spark] ( } /** - * :: Experimental :: - * * Local (non-distributed) model fitted by [[LDA]]. * * This model stores the inferred topics only; it does not store info about the training dataset. */ -@Experimental @Since("1.5.0") object LocalLDAModel extends Loader[LocalLDAModel] { @@ -822,15 +819,12 @@ class DistributedLDAModel private[clustering] ( } /** - * :: Experimental :: - * * Distributed model fitted by [[LDA]]. * This type of model is currently only produced by Expectation-Maximization (EM). * * This model stores the inferred topics, the full training dataset, and the topic distribution * for each training document. */ -@Experimental @Since("1.5.0") object DistributedLDAModel extends Loader[DistributedLDAModel] { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 9a63cc29dacb5..3c26d2670841b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.fpm import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.internal.Logging @@ -28,14 +28,11 @@ import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset import org.apache.spark.rdd.RDD /** - * :: Experimental :: - * * Generates association rules from a [[RDD[FreqItemset[Item]]]. This method only generates * association rules which have a single item as the consequent. * */ @Since("1.5.0") -@Experimental class AssociationRules private[fpm] ( private var minConfidence: Double) extends Logging with Serializable { @@ -95,8 +92,6 @@ class AssociationRules private[fpm] ( object AssociationRules { /** - * :: Experimental :: - * * An association rule between sets of items. * @param antecedent hypotheses of the rule. Java users should call [[Rule#javaAntecedent]] * instead. @@ -106,7 +101,6 @@ object AssociationRules { * */ @Since("1.5.0") - @Experimental class Rule[Item] private[fpm] ( @Since("1.5.0") val antecedent: Array[Item], @Since("1.5.0") val consequent: Array[Item], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index c13c794775fec..7382000791cfb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -30,7 +30,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.internal.Logging @@ -42,8 +42,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel /** - * :: Experimental :: - * * A parallel PrefixSpan algorithm to mine frequent sequential patterns. * The PrefixSpan algorithm is described in J. Pei, et al., PrefixSpan: Mining Sequential Patterns * Efficiently by Prefix-Projected Pattern Growth ([[http://doi.org/10.1109/ICDE.2001.914830]]). @@ -60,7 +58,6 @@ import org.apache.spark.storage.StorageLevel * @see [[https://en.wikipedia.org/wiki/Sequential_Pattern_Mining Sequential Pattern Mining * (Wikipedia)]] */ -@Experimental @Since("1.5.0") class PrefixSpan private ( private var minSupport: Double, @@ -230,7 +227,6 @@ class PrefixSpan private ( } -@Experimental @Since("1.5.0") object PrefixSpan extends Logging { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 4591cb88ef152..8024b1c0031fe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.linalg -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since /** * Represents singular value decomposition (SVD) factors. @@ -26,10 +26,8 @@ import org.apache.spark.annotation.{Experimental, Since} case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) /** - * :: Experimental :: * Represents QR factors. */ @Since("1.5.0") -@Experimental case class QRDecomposition[QType, RType](Q: QType, R: RType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 480a64548cb70..f372355005656 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{norm, DenseVector => BDV} -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD @@ -53,11 +53,9 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va } /** - * :: Experimental :: * Set fraction of data to be used for each SGD iteration. * Default 1.0 (corresponding to deterministic/classical gradient descent) */ - @Experimental def setMiniBatchFraction(fraction: Double): this.type = { require(fraction > 0 && fraction <= 1.0, s"Fraction for mini-batch SGD must be in range (0, 1] but got ${fraction}") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 274ac7c99553b..5d61796f1de60 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -23,7 +23,7 @@ import javax.xml.transform.stream.StreamResult import org.jpmml.model.JAXBUtil import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory /** @@ -45,20 +45,16 @@ trait PMMLExportable { } /** - * :: Experimental :: * Export the model to a local file in PMML format */ - @Experimental @Since("1.4.0") def toPMML(localPath: String): Unit = { toPMML(new StreamResult(new File(localPath))) } /** - * :: Experimental :: * Export the model to a directory on a distributed file system in PMML format */ - @Experimental @Since("1.4.0") def toPMML(sc: SparkContext, path: String): Unit = { val pmml = toPMML() @@ -66,20 +62,16 @@ trait PMMLExportable { } /** - * :: Experimental :: * Export the model to the OutputStream in PMML format */ - @Experimental @Since("1.4.0") def toPMML(outputStream: OutputStream): Unit = { toPMML(new StreamResult(outputStream)) } /** - * :: Experimental :: * Export the model to a String in PMML format */ - @Experimental @Since("1.4.0") def toPMML(): String = { val writer = new StringWriter diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala index 4c382d7c2b791..97c032de7a813 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.stat.test import scala.beans.BeanInfo -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.streaming.api.java.JavaDStream import org.apache.spark.streaming.dstream.DStream @@ -42,7 +42,6 @@ case class BinarySample @Since("1.6.0") ( } /** - * :: Experimental :: * Performs online 2-sample significance testing for a stream of (Boolean, Double) pairs. The * Boolean identifies which sample each observation comes from, and the Double is the numeric value * of the observation. @@ -67,7 +66,6 @@ case class BinarySample @Since("1.6.0") ( * .registerStream(DStream) * }}} */ -@Experimental @Since("1.6.0") class StreamingTest @Since("1.6.0") () extends Logging with Serializable { private var peacePeriod: Int = 0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala index 8a29fd39a9106..5cfc05a3dd2d2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.stat.test -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since /** * Trait for hypothesis test results. @@ -94,10 +94,8 @@ class ChiSqTestResult private[stat] (override val pValue: Double, } /** - * :: Experimental :: * Object containing the test results for the Kolmogorov-Smirnov test. */ -@Experimental @Since("1.5.0") class KolmogorovSmirnovTestResult private[stat] ( @Since("1.5.0") override val pValue: Double, @@ -113,10 +111,8 @@ class KolmogorovSmirnovTestResult private[stat] ( } /** - * :: Experimental :: * Object containing the test results for streaming testing. */ -@Experimental @Since("1.6.0") private[stat] class StreamingTestResult @Since("1.6.0") ( @Since("1.6.0") override val pValue: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 853c7319ec44d..2436ce40866e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -17,14 +17,12 @@ package org.apache.spark.mllib.tree.configuration -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since /** - * :: Experimental :: * Enum to select the algorithm for the decision tree */ @Since("1.0.0") -@Experimental object Algo extends Enumeration { @Since("1.0.0") type Algo = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 3a731f45d6a07..d4448da9eef51 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -17,14 +17,12 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: * Class for calculating entropy during multiclass classification. */ @Since("1.0.0") -@Experimental object Entropy extends Impurity { private[tree] def log2(x: Double) = scala.math.log(x) / scala.math.log(2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 7730c0a8c1117..22e70278a6656 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -17,16 +17,14 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during multiclass classification. */ @Since("1.0.0") -@Experimental object Gini extends Impurity { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 65f0163ec6059..a5bdc2c6d2c94 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -17,17 +17,15 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: * Trait for calculating information gain. * This trait is used for * (a) setting the impurity parameter in [[org.apache.spark.mllib.tree.configuration.Strategy]] * (b) calculating impurity values from sufficient statistics. */ @Since("1.0.0") -@Experimental trait Impurity extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 2423516123b82..c9bf0db4de3c2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -17,14 +17,12 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: * Class for calculating variance during regression */ @Since("1.0.0") -@Experimental object Variance extends Impurity { /** diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index c035942f73863..3c4af90acac85 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -49,8 +49,6 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti HasElasticNetParam, HasFitIntercept, HasStandardization, HasThresholds, HasWeightCol, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Logistic regression. Currently, this class only supports binary classification. @@ -216,8 +214,6 @@ def _checkThresholdConsistency(self): class LogisticRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by LogisticRegression. .. versionadded:: 1.3.0 @@ -277,6 +273,8 @@ def evaluate(self, dataset): class LogisticRegressionSummary(JavaWrapper): """ + .. note:: Experimental + Abstraction for Logistic Regression Results for a given model. .. versionadded:: 2.0.0 @@ -321,6 +319,8 @@ def featuresCol(self): @inherit_doc class LogisticRegressionTrainingSummary(LogisticRegressionSummary): """ + .. note:: Experimental + Abstraction for multinomial Logistic Regression Training results. Currently, the training summary ignores the training weights except for the objective trace. @@ -501,8 +501,6 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred TreeClassifierParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - `Decision tree `_ learning algorithm for classification. It supports both binary and multiclass labels, as well as both continuous and categorical @@ -599,8 +597,6 @@ def _create_model(self, java_model): @inherit_doc class DecisionTreeClassificationModel(DecisionTreeModel, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by DecisionTreeClassifier. .. versionadded:: 1.4.0 @@ -634,8 +630,6 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred RandomForestParams, TreeClassifierParams, HasCheckpointInterval, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - `Random Forest `_ learning algorithm for classification. It supports both binary and multiclass labels, as well as both continuous and categorical @@ -730,8 +724,6 @@ def _create_model(self, java_model): class RandomForestClassificationModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by RandomForestClassifier. .. versionadded:: 1.4.0 @@ -764,8 +756,6 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol GBTParams, HasCheckpointInterval, HasStepSize, HasSeed, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - `Gradient-Boosted Trees (GBTs) `_ learning algorithm for classification. It supports binary labels, as well as both continuous and categorical features. @@ -885,8 +875,6 @@ def getLossType(self): class GBTClassificationModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by GBTClassifier. .. versionadded:: 1.4.0 @@ -918,8 +906,6 @@ def trees(self): class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, HasRawPredictionCol, HasThresholds, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Naive Bayes Classifiers. It supports both Multinomial and Bernoulli NB. `Multinomial NB `_ @@ -1043,8 +1029,6 @@ def getModelType(self): class NaiveBayesModel(JavaModel, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by NaiveBayes. .. versionadded:: 1.5.0 diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index bbbb94f9a0a04..2881380152c8d 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -60,8 +60,6 @@ @inherit_doc class Binarizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Binarize a column of continuous features given a threshold. >>> df = spark.createDataFrame([(0.5,)], ["values"]) @@ -125,8 +123,6 @@ def getThreshold(self): @inherit_doc class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Maps a column of continuous features to a column of feature buckets. >>> df = spark.createDataFrame([(0.1,), (0.4,), (1.2,), (1.5,)], ["values"]) @@ -200,8 +196,6 @@ def getSplits(self): @inherit_doc class CountVectorizer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Extracts a vocabulary from document collections and generates a :py:attr:`CountVectorizerModel`. >>> df = spark.createDataFrame( @@ -348,8 +342,6 @@ def _create_model(self, java_model): class CountVectorizerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`CountVectorizer`. .. versionadded:: 1.6.0 @@ -367,8 +359,6 @@ def vocabulary(self): @inherit_doc class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A feature transformer that takes the 1D discrete cosine transform of a real vector. No zero padding is performed on the input vector. It returns a real vector of the same length representing the DCT. @@ -439,8 +429,6 @@ def getInverse(self): class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a provided "weight" vector. In other words, it scales each column of the dataset by a scalar multiplier. @@ -505,8 +493,6 @@ def getScalingVec(self): class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Maps a sequence of terms to their term frequencies using the hashing trick. Currently we use Austin Appleby's MurmurHash 3 algorithm (MurmurHash3_x86_32) to calculate the hash code value for the term object. @@ -576,8 +562,6 @@ def getBinary(self): @inherit_doc class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Compute the Inverse Document Frequency (IDF) given a collection of documents. >>> from pyspark.ml.linalg import DenseVector @@ -653,8 +637,6 @@ def _create_model(self, java_model): class IDFModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`IDF`. .. versionadded:: 1.4.0 @@ -752,8 +734,6 @@ def maxAbs(self): @inherit_doc class MinMaxScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Rescale each feature individually to a common range [min, max] linearly using column summary statistics, which is also known as min-max normalization or Rescaling. The rescaled value for feature E is calculated as, @@ -859,8 +839,6 @@ def _create_model(self, java_model): class MinMaxScalerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`MinMaxScaler`. .. versionadded:: 1.6.0 @@ -887,8 +865,6 @@ def originalMax(self): @ignore_unicode_prefix class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A feature transformer that converts the input array of strings into an array of n-grams. Null values in the input array are ignored. It returns an array of n-grams where each n-gram is represented by a space-separated string of @@ -965,8 +941,6 @@ def getN(self): @inherit_doc class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Normalize a vector to have unit norm using the given p-norm. >>> from pyspark.ml.linalg import Vectors @@ -1031,8 +1005,6 @@ def getP(self): @inherit_doc class OneHotEncoder(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A one-hot encoder that maps a column of category indices to a column of binary vectors, with at most a single one-value per row that indicates the input category index. @@ -1114,8 +1086,6 @@ def getDropLast(self): class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Perform feature expansion in a polynomial space. As said in `wikipedia of Polynomial Expansion `_, "In mathematics, an expansion of a product of sums expresses it as a sum of products by using the fact that @@ -1287,8 +1257,6 @@ def _create_model(self, java_model): @ignore_unicode_prefix class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A regex based tokenizer that extracts tokens either by using the provided regex pattern (in Java dialect) to split the text (default) or repeatedly matching the regex (if gaps is false). @@ -1418,8 +1386,6 @@ def getToLowercase(self): @inherit_doc class SQLTransformer(JavaTransformer, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Implements the transforms which are defined by SQL statement. Currently we only support SQL syntax like 'SELECT ... FROM __THIS__' where '__THIS__' represents the underlying table of the input dataset. @@ -1479,8 +1445,6 @@ def getStatement(self): @inherit_doc class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the training set. @@ -1576,8 +1540,6 @@ def _create_model(self, java_model): class StandardScalerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`StandardScaler`. .. versionadded:: 1.4.0 @@ -1604,8 +1566,6 @@ def mean(self): class StringIndexer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInvalid, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A label indexer that maps a string column of labels to an ML column of label indices. If the input column is numeric, we cast it to string and index the string values. The indices are in [0, numLabels), ordered by label frequencies. @@ -1668,8 +1628,6 @@ def _create_model(self, java_model): class StringIndexerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`StringIndexer`. .. versionadded:: 1.4.0 @@ -1687,8 +1645,6 @@ def labels(self): @inherit_doc class IndexToString(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A :py:class:`Transformer` that maps a column of indices back to a new column of corresponding string values. The index-string mapping is either from the ML attributes of the input column, @@ -1741,8 +1697,6 @@ def getLabels(self): class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A feature transformer that filters out stop words from input. Note: null values from input array are preserved unless adding null to stopWords explicitly. @@ -1833,8 +1787,6 @@ def loadDefaultStopWords(language): @ignore_unicode_prefix class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A tokenizer that converts the input string to lowercase and then splits it by white spaces. @@ -1888,8 +1840,6 @@ def setParams(self, inputCol=None, outputCol=None): @inherit_doc class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - A feature transformer that merges multiple columns into a vector column. >>> df = spark.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) @@ -1934,8 +1884,6 @@ def setParams(self, inputCols=None, outputCol=None): @inherit_doc class VectorIndexer(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Class for indexing categorical feature columns in a dataset of `Vector`. This has 2 usage modes: @@ -2050,8 +1998,6 @@ def _create_model(self, java_model): class VectorIndexerModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`VectorIndexer`. Transform categorical features to use 0-based indices instead of their original values. @@ -2089,8 +2035,6 @@ def categoryMaps(self): @inherit_doc class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - This class takes a feature vector and outputs a new feature vector with a subarray of the original features. @@ -2183,8 +2127,6 @@ def getNames(self): class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Word2Vec trains a model of `Map(String, Vector)`, i.e. transforms a word into a code for further natural language processing or machine learning process. @@ -2352,8 +2294,6 @@ def _create_model(self, java_model): class Word2VecModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`Word2Vec`. .. versionadded:: 1.4.0 @@ -2383,8 +2323,6 @@ def findSynonyms(self, word, num): @inherit_doc class PCA(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - PCA trains a model to project vectors to a lower dimensional space of the top :py:attr:`k` principal components. @@ -2458,8 +2396,6 @@ def _create_model(self, java_model): class PCAModel(JavaModel, JavaMLReadable, JavaMLWritable): """ - .. note:: Experimental - Model fitted by :py:class:`PCA`. Transforms vectors to a lower dimensional space. .. versionadded:: 1.5.0 diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 8de9ad85311fa..d88dc75353598 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -41,8 +41,6 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction HasRegParam, HasTol, HasElasticNetParam, HasFitIntercept, HasStandardization, HasSolver, HasWeightCol, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Linear regression. The learning objective is to minimize the squared error, with regularization. @@ -130,8 +128,6 @@ def _create_model(self, java_model): class LinearRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by :class:`LinearRegression`. .. versionadded:: 1.4.0 @@ -411,8 +407,6 @@ def totalIterations(self): class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasWeightCol, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Currently implemented using parallelized pool adjacent violators algorithm. Only univariate (single feature) algorithm supported. @@ -439,6 +433,8 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti True >>> model.predictions == model2.predictions True + + .. versionadded:: 1.6.0 """ isotonic = \ @@ -505,13 +501,13 @@ def getFeatureIndex(self): class IsotonicRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by :class:`IsotonicRegression`. + + .. versionadded:: 1.6.0 """ @property - @since("2.0.0") + @since("1.6.0") def boundaries(self): """ Boundaries in increasing order for which predictions are known. @@ -519,7 +515,7 @@ def boundaries(self): return self._call_java("boundaries") @property - @since("2.0.0") + @since("1.6.0") def predictions(self): """ Predictions associated with the boundaries at the same index, monotone because of isotonic @@ -642,8 +638,6 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi DecisionTreeParams, TreeRegressorParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable, HasVarianceCol): """ - .. note:: Experimental - `Decision tree `_ learning algorithm for regression. It supports both continuous and categorical features. @@ -727,8 +721,6 @@ def _create_model(self, java_model): @inherit_doc class DecisionTreeModel(JavaModel): """ - .. note:: Experimental - Abstraction for Decision Tree models. .. versionadded:: 1.5.0 @@ -759,11 +751,9 @@ def __repr__(self): @inherit_doc class TreeEnsembleModels(JavaModel): """ - .. note:: Experimental + (private abstraction) Represents a tree ensemble model. - - .. versionadded:: 1.5.0 """ @property @@ -803,8 +793,6 @@ def __repr__(self): @inherit_doc class DecisionTreeRegressionModel(DecisionTreeModel, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by :class:`DecisionTreeRegressor`. .. versionadded:: 1.4.0 @@ -837,8 +825,6 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi RandomForestParams, TreeRegressorParams, HasCheckpointInterval, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - `Random Forest `_ learning algorithm for regression. It supports both continuous and categorical features. @@ -925,8 +911,6 @@ def _create_model(self, java_model): class RandomForestRegressionModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by :class:`RandomForestRegressor`. .. versionadded:: 1.4.0 @@ -959,8 +943,6 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, GBTParams, HasCheckpointInterval, HasStepSize, HasSeed, JavaMLWritable, JavaMLReadable, TreeRegressorParams): """ - .. note:: Experimental - `Gradient-Boosted Trees (GBTs) `_ learning algorithm for regression. It supports both continuous and categorical features. @@ -1067,8 +1049,6 @@ def getLossType(self): class GBTRegressionModel(TreeEnsembleModels, JavaMLWritable, JavaMLReadable): """ - .. note:: Experimental - Model fitted by :class:`GBTRegressor`. .. versionadded:: 1.4.0 diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index f857c5e8c86b6..298314d46caf6 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -33,8 +33,6 @@ class ParamGridBuilder(object): r""" - .. note:: Experimental - Builder for a param grid used in grid search-based model selection. >>> from pyspark.ml.classification import LogisticRegression @@ -145,8 +143,6 @@ def getEvaluator(self): class CrossValidator(Estimator, ValidatorParams): """ - .. note:: Experimental - K-fold cross validation. >>> from pyspark.ml.classification import LogisticRegression @@ -264,8 +260,6 @@ def copy(self, extra=None): class CrossValidatorModel(Model, ValidatorParams): """ - .. note:: Experimental - Model from k-fold cross validation. .. versionadded:: 1.4.0 diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 3734f87405e5a..9f53ed098202b 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -48,8 +48,6 @@ def __init__(self, weights, intercept): @since('1.4.0') def setThreshold(self, value): """ - .. note:: Experimental - Sets the threshold that separates positive predictions from negative predictions. An example with prediction score greater than or equal to this threshold is identified as a positive, @@ -62,8 +60,6 @@ def setThreshold(self, value): @since('1.4.0') def threshold(self): """ - .. note:: Experimental - Returns the threshold (if any) used for converting raw prediction scores into 0/1 predictions. It is used for binary classification only. @@ -73,8 +69,6 @@ def threshold(self): @since('1.4.0') def clearThreshold(self): """ - .. note:: Experimental - Clears the threshold so that `predict` will output raw prediction scores. It is used for binary classification only. """ diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index c38c543972d13..c8c3c42774f21 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -47,8 +47,6 @@ @inherit_doc class BisectingKMeansModel(JavaModelWrapper): """ - .. note:: Experimental - A clustering model derived from the bisecting k-means method. >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4, 2) @@ -120,8 +118,6 @@ def computeCost(self, x): class BisectingKMeans(object): """ - .. note:: Experimental - A bisecting k-means algorithm based on the paper "A comparison of document clustering techniques" by Steinbach, Karypis, and Kumar, with modification to fit Spark. @@ -366,8 +362,6 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" class GaussianMixtureModel(JavaModelWrapper, JavaSaveable, JavaLoader): """ - .. note:: Experimental - A clustering model derived from the Gaussian Mixture Model method. >>> from pyspark.mllib.linalg import Vectors, DenseMatrix @@ -513,8 +507,6 @@ def load(cls, sc, path): class GaussianMixture(object): """ - .. note:: Experimental - Learning algorithm for Gaussian Mixtures using the expectation-maximization algorithm. .. versionadded:: 1.3.0 @@ -565,8 +557,6 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia class PowerIterationClusteringModel(JavaModelWrapper, JavaSaveable, JavaLoader): """ - .. note:: Experimental - Model produced by [[PowerIterationClustering]]. >>> import math @@ -645,8 +635,6 @@ def load(cls, sc, path): class PowerIterationClustering(object): """ - .. note:: Experimental - Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by [[http://www.icml2010.org/papers/387.pdf Lin and Cohen]]. From the abstract: PIC finds a very low-dimensional embedding of a @@ -693,8 +681,6 @@ class Assignment(namedtuple("Assignment", ["id", "cluster"])): class StreamingKMeansModel(KMeansModel): """ - .. note:: Experimental - Clustering model which can perform an online update of the centroids. The update formula for each centroid is given by @@ -794,8 +780,6 @@ def update(self, data, decayFactor, timeUnit): class StreamingKMeans(object): """ - .. note:: Experimental - Provides methods to set k, decayFactor, timeUnit to configure the KMeans algorithm for fitting and predicting on incoming dstreams. More details on how the centroids are updated are provided under the diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index aef91a8ddc1f1..c8a6e33f4d9a4 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -60,8 +60,6 @@ def transform(self, vector): class Normalizer(VectorTransformer): """ - .. note:: Experimental - Normalizes samples individually to unit L\ :sup:`p`\ norm For any 1 <= `p` < float('inf'), normalizes samples using @@ -131,8 +129,6 @@ def transform(self, vector): class StandardScalerModel(JavaVectorTransformer): """ - .. note:: Experimental - Represents a StandardScaler model that can transform vectors. .. versionadded:: 1.2.0 @@ -207,8 +203,6 @@ def mean(self): class StandardScaler(object): """ - .. note:: Experimental - Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the training set. @@ -262,8 +256,6 @@ def fit(self, dataset): class ChiSqSelectorModel(JavaVectorTransformer): """ - .. note:: Experimental - Represents a Chi Squared selector model. .. versionadded:: 1.4.0 @@ -282,8 +274,6 @@ def transform(self, vector): class ChiSqSelector(object): """ - .. note:: Experimental - Creates a ChiSquared feature selector. :param numTopFeatures: number of features that selector will select. @@ -361,8 +351,6 @@ def fit(self, data): class HashingTF(object): """ - .. note:: Experimental - Maps a sequence of terms to their term frequencies using the hashing trick. @@ -448,8 +436,6 @@ def idf(self): class IDF(object): """ - .. note:: Experimental - Inverse document frequency (IDF). The standard formulation is used: `idf = log((m + 1) / (d(t) + 1))`, @@ -697,8 +683,6 @@ def fit(self, data): class ElementwiseProduct(VectorTransformer): """ - .. note:: Experimental - Scales each column of the vector, with the supplied weight vector. i.e the elementwise product. diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index fb226e84e5d50..f58ea5dfb0874 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -31,8 +31,6 @@ @ignore_unicode_prefix class FPGrowthModel(JavaModelWrapper, JavaSaveable, JavaLoader): """ - .. note:: Experimental - A FP-Growth model for mining frequent itemsets using the Parallel FP-Growth algorithm. @@ -70,8 +68,6 @@ def load(cls, sc, path): class FPGrowth(object): """ - .. note:: Experimental - A Parallel FP-growth algorithm to mine frequent itemsets. .. versionadded:: 1.4.0 @@ -108,8 +104,6 @@ class FreqItemset(namedtuple("FreqItemset", ["items", "freq"])): @ignore_unicode_prefix class PrefixSpanModel(JavaModelWrapper): """ - .. note:: Experimental - Model fitted by PrefixSpan >>> data = [ @@ -133,8 +127,6 @@ def freqSequences(self): class PrefixSpan(object): """ - .. note:: Experimental - A parallel PrefixSpan algorithm to mine frequent sequential patterns. The PrefixSpan algorithm is described in J. Pei, et al., PrefixSpan: Mining Sequential Patterns Efficiently by Prefix-Projected Pattern Growth diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index 15dc53a959d6d..9672dbde823f2 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -1338,8 +1338,6 @@ def fromML(mat): class QRDecomposition(object): """ - .. note:: Experimental - Represents QR factors. """ def __init__(self, Q, R): diff --git a/python/pyspark/mllib/linalg/distributed.py b/python/pyspark/mllib/linalg/distributed.py index ea4f27cf4ffe9..538cada7d163d 100644 --- a/python/pyspark/mllib/linalg/distributed.py +++ b/python/pyspark/mllib/linalg/distributed.py @@ -40,8 +40,6 @@ class DistributedMatrix(object): """ - .. note:: Experimental - Represents a distributively stored matrix backed by one or more RDDs. @@ -57,8 +55,6 @@ def numCols(self): class RowMatrix(DistributedMatrix): """ - .. note:: Experimental - Represents a row-oriented distributed Matrix with no meaningful row indices. @@ -306,8 +302,6 @@ def tallSkinnyQR(self, computeQ=False): class IndexedRow(object): """ - .. note:: Experimental - Represents a row of an IndexedRowMatrix. Just a wrapper over a (long, vector) tuple. @@ -334,8 +328,6 @@ def _convert_to_indexed_row(row): class IndexedRowMatrix(DistributedMatrix): """ - .. note:: Experimental - Represents a row-oriented distributed Matrix with indexed rows. :param rows: An RDD of IndexedRows or (long, vector) tuples. @@ -536,8 +528,6 @@ def toBlockMatrix(self, rowsPerBlock=1024, colsPerBlock=1024): class MatrixEntry(object): """ - .. note:: Experimental - Represents an entry of a CoordinateMatrix. Just a wrapper over a (long, long, float) tuple. @@ -566,8 +556,6 @@ def _convert_to_matrix_entry(entry): class CoordinateMatrix(DistributedMatrix): """ - .. note:: Experimental - Represents a matrix in coordinate format. :param entries: An RDD of MatrixEntry inputs or @@ -795,8 +783,6 @@ def _convert_to_matrix_block_tuple(block): class BlockMatrix(DistributedMatrix): """ - .. note:: Experimental - Represents a distributed matrix in blocks of local matrices. :param blocks: An RDD of sub-matrix blocks diff --git a/python/pyspark/mllib/stat/KernelDensity.py b/python/pyspark/mllib/stat/KernelDensity.py index 7da921976d4d2..3b1c5519bd87e 100644 --- a/python/pyspark/mllib/stat/KernelDensity.py +++ b/python/pyspark/mllib/stat/KernelDensity.py @@ -28,8 +28,6 @@ class KernelDensity(object): """ - .. note:: Experimental - Estimate probability density at required points given a RDD of samples from the population. diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index b0a85240b289a..67d5f0e44f41c 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -160,8 +160,6 @@ def corr(x, y=None, method=None): @ignore_unicode_prefix def chiSqTest(observed, expected=None): """ - .. note:: Experimental - If `observed` is Vector, conduct Pearson's chi-squared goodness of fit test of the observed data against the expected distribution, or againt the uniform distribution (by default), with each category @@ -246,8 +244,6 @@ def chiSqTest(observed, expected=None): @ignore_unicode_prefix def kolmogorovSmirnovTest(data, distName="norm", *params): """ - .. note:: Experimental - Performs the Kolmogorov-Smirnov (KS) test for data sampled from a continuous distribution. It tests the null hypothesis that the data is generated from a particular distribution. diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 8be76fcefe542..b3011d42e56af 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -76,8 +76,6 @@ def toDebugString(self): class DecisionTreeModel(JavaModelWrapper, JavaSaveable, JavaLoader): """ - .. note:: Experimental - A decision tree model for classification or regression. .. versionadded:: 1.1.0 @@ -130,8 +128,6 @@ def _java_loader_class(cls): class DecisionTree(object): """ - .. note:: Experimental - Learning algorithm for a decision tree model for classification or regression. @@ -283,8 +279,6 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, @inherit_doc class RandomForestModel(TreeEnsembleModel, JavaLoader): """ - .. note:: Experimental - Represents a random forest model. .. versionadded:: 1.2.0 @@ -297,8 +291,6 @@ def _java_loader_class(cls): class RandomForest(object): """ - .. note:: Experimental - Learning algorithm for a random forest model for classification or regression. @@ -486,8 +478,6 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, numTrees, featureSubsetSt @inherit_doc class GradientBoostedTreesModel(TreeEnsembleModel, JavaLoader): """ - .. note:: Experimental - Represents a gradient-boosted tree model. .. versionadded:: 1.3.0 @@ -500,8 +490,6 @@ def _java_loader_class(cls): class GradientBoostedTrees(object): """ - .. note:: Experimental - Learning algorithm for a gradient boosted trees model for classification or regression. From 7de183d975c1a46bde6a9020b339673d953dd1a1 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 13 Jul 2016 12:54:57 -0700 Subject: [PATCH 0964/1470] [SPARK-16531][SQL][TEST] Remove timezone setting from DataFrameTimeWindowingSuite ## What changes were proposed in this pull request? It's unnecessary. `QueryTest` already sets it. Author: Burak Yavuz Closes #14170 from brkyvz/test-tz. (cherry picked from commit 0744d84c91d6e494dea77a35e6410bc4b1849e71) Signed-off-by: Michael Armbrust --- .../apache/spark/sql/DataFrameTimeWindowingSuite.scala | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index a15b4e1221d3b..4296ec543e275 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -29,16 +29,6 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with B import testImplicits._ - override def beforeEach(): Unit = { - super.beforeEach() - TimeZone.setDefault(TimeZone.getTimeZone("UTC")) - } - - override def afterEach(): Unit = { - super.beforeEach() - TimeZone.setDefault(null) - } - test("tumbling window groupBy statement") { val df = Seq( ("2016-03-27 19:39:34", 1, "a"), From 86adc5cfbe286eb4d6071ec9ee09b6d0960a8509 Mon Sep 17 00:00:00 2001 From: James Thomas Date: Wed, 13 Jul 2016 13:26:23 -0700 Subject: [PATCH 0965/1470] [SPARK-16114][SQL] updated structured streaming guide ## What changes were proposed in this pull request? Updated structured streaming programming guide with new windowed example. ## How was this patch tested? Docs Author: James Thomas Closes #14183 from jjthomas/ss_docs_update. (cherry picked from commit 51a6706b1339bb761602e33276a469f71be2cd90) Signed-off-by: Tathagata Das --- .../structured-streaming-programming-guide.md | 49 +++++++++---------- 1 file changed, 23 insertions(+), 26 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 79493968db274..3ef39e4885be8 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -626,52 +626,49 @@ The result tables would look something like the following. ![Window Operations](img/structured-streaming-window.png) -Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window()` operations to express windowed aggregations. +Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window()` operations to express windowed aggregations. You can see the full code for the below examples in +[Scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala)/ +[Java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java)/ +[Python]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py).
    {% highlight scala %} -// Number of events in every 1 minute time windows -df.groupBy(window(df.col("time"), "1 minute")) - .count() +import spark.implicits._ +val words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String } -// Average number of events for each device type in every 1 minute time windows -df.groupBy( - df.col("type"), - window(df.col("time"), "1 minute")) - .avg("signal") +// Group the data by window and word and compute the count of each group +val windowedCounts = words.groupBy( + window($"timestamp", "10 minutes", "5 minutes"), + $"word" +).count() {% endhighlight %}
    {% highlight java %} -import static org.apache.spark.sql.functions.window; - -// Number of events in every 1 minute time windows -df.groupBy(window(df.col("time"), "1 minute")) - .count(); - -// Average number of events for each device type in every 1 minute time windows -df.groupBy( - df.col("type"), - window(df.col("time"), "1 minute")) - .avg("signal"); +Dataset words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String } +// Group the data by window and word and compute the count of each group +Dataset windowedCounts = words.groupBy( + functions.window(words.col("timestamp"), "10 minutes", "5 minutes"), + words.col("word") +).count(); {% endhighlight %}
    {% highlight python %} -from pyspark.sql.functions import window - -# Number of events in every 1 minute time windows -df.groupBy(window("time", "1 minute")).count() +words = ... # streaming DataFrame of schema { timestamp: Timestamp, word: String } -# Average number of events for each device type in every 1 minute time windows -df.groupBy("type", window("time", "1 minute")).avg("signal") +# Group the data by window and word and compute the count of each group +windowedCounts = words.groupBy( + window(words.timestamp, '10 minutes', '5 minutes'), + words.word +).count() {% endhighlight %}
    From 18255a9345dd711bf630993c582511efa74b7919 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Wed, 13 Jul 2016 13:33:34 -0700 Subject: [PATCH 0966/1470] [SPARKR][MINOR] R examples and test updates ## What changes were proposed in this pull request? Minor example updates ## How was this patch tested? manual shivaram Author: Felix Cheung Closes #14171 from felixcheung/rexample. (cherry picked from commit b4baf086ca380a46d953f2710184ad9eee3a045e) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/jarTest.R | 2 +- R/pkg/inst/tests/testthat/packageInAJarTest.R | 2 +- examples/src/main/r/RSparkSQLExample.R | 3 +++ examples/src/main/r/dataframe.R | 2 +- 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/R/pkg/inst/tests/testthat/jarTest.R b/R/pkg/inst/tests/testthat/jarTest.R index 84e4845f180b3..51754a465062f 100644 --- a/R/pkg/inst/tests/testthat/jarTest.R +++ b/R/pkg/inst/tests/testthat/jarTest.R @@ -16,7 +16,7 @@ # library(SparkR) -sparkSession <- sparkR.session() +sparkR.session() helloTest <- SparkR:::callJStatic("sparkR.test.hello", "helloWorld", diff --git a/R/pkg/inst/tests/testthat/packageInAJarTest.R b/R/pkg/inst/tests/testthat/packageInAJarTest.R index 940c91f376cd5..4bc935c79eb0f 100644 --- a/R/pkg/inst/tests/testthat/packageInAJarTest.R +++ b/R/pkg/inst/tests/testthat/packageInAJarTest.R @@ -17,7 +17,7 @@ library(SparkR) library(sparkPackageTest) -sparkSession <- sparkR.session() +sparkR.session() run1 <- myfunc(5L) diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index eba3f1b91e2df..f20875ca9efb5 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -195,3 +195,6 @@ results <- collect(sql("FROM src SELECT key, value")) # $example on:jdbc$ df <- read.jdbc("jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password") # $example off:jdbc$ + +# Stop the SparkSession now +sparkR.session.stop() diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R index 295f9b427622c..82b85f2f590f6 100644 --- a/examples/src/main/r/dataframe.R +++ b/examples/src/main/r/dataframe.R @@ -18,7 +18,7 @@ library(SparkR) # Initialize SparkSession -sc <- sparkR.session(appName = "SparkR-DataFrame-example") +sparkR.session(appName = "SparkR-DataFrame-example") # Create a simple local data.frame localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) From 9e3a598582c747194188f8ad15b43aca03907bae Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Wed, 13 Jul 2016 15:09:23 -0700 Subject: [PATCH 0967/1470] [SPARKR][DOCS][MINOR] R programming guide to include csv data source example ## What changes were proposed in this pull request? Minor documentation update for code example, code style, and missed reference to "sparkR.init" ## How was this patch tested? manual shivaram Author: Felix Cheung Closes #14178 from felixcheung/rcsvprogrammingguide. (cherry picked from commit fb2e8eeb0b1e56bea535165f7a3bec6558b3f4a3) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- docs/sparkr.md | 27 +++++++++++++++-------- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index bd7b5f062e6d4..e26b015d9371b 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -237,7 +237,7 @@ test_that("read csv as DataFrame", { "Empty,Dummy,Placeholder") writeLines(mockLinesCsv, csvPath) - df2 <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.string = "Empty") + df2 <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.strings = "Empty") expect_equal(count(df2), 4) withoutna2 <- na.omit(df2, how = "any", cols = "year") expect_equal(count(withoutna2), 3) diff --git a/docs/sparkr.md b/docs/sparkr.md index b4acb230403a2..9fda0ec0e6542 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -111,19 +111,17 @@ head(df) SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. SparkR supports reading JSON, CSV and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by -specifying `--packages` with `spark-submit` or `sparkR` commands, or if creating context through `init` -you can specify the packages with the `packages` argument. +specifying `--packages` with `spark-submit` or `sparkR` commands, or if initializing SparkSession with `sparkPackages` parameter when in an interactive R shell or from RStudio.
    {% highlight r %} -sc <- sparkR.session(sparkPackages="com.databricks:spark-avro_2.11:3.0.0") +sc <- sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") {% endhighlight %}
    We can see how to use data sources using an example JSON input file. Note that the file that is used here is _not_ a typical JSON file. Each line in the file must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail.
    - {% highlight r %} people <- read.df("./examples/src/main/resources/people.json", "json") head(people) @@ -138,6 +136,18 @@ printSchema(people) # |-- age: long (nullable = true) # |-- name: string (nullable = true) +# Similarly, multiple files can be read with read.json +people <- read.json(c("./examples/src/main/resources/people.json", "./examples/src/main/resources/people2.json")) + +{% endhighlight %} +
    + +The data sources API natively supports CSV formatted input files. For more information please refer to SparkR [read.df](api/R/read.df.html) API documentation. + +
    +{% highlight r %} +df <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.strings = "NA") + {% endhighlight %}
    @@ -146,7 +156,7 @@ to a Parquet file using `write.df`.
    {% highlight r %} -write.df(people, path="people.parquet", source="parquet", mode="overwrite") +write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite") {% endhighlight %}
    @@ -264,14 +274,14 @@ In SparkR, we support several kinds of User-Defined Functions: Apply a function to each partition of a `SparkDataFrame`. The function to be applied to each partition of the `SparkDataFrame` and should have only one parameter, to which a `data.frame` corresponds to each partition will be passed. The output of function should be a `data.frame`. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match the R function's output. +
    {% highlight r %} - # Convert waiting time from hours to seconds. # Note that we can apply UDF to DataFrame. schema <- structType(structField("eruptions", "double"), structField("waiting", "double"), structField("waiting_secs", "double")) -df1 <- dapply(df, function(x) {x <- cbind(x, x$waiting * 60)}, schema) +df1 <- dapply(df, function(x) { x <- cbind(x, x$waiting * 60) }, schema) head(collect(df1)) ## eruptions waiting waiting_secs ##1 3.600 79 4740 @@ -313,9 +323,9 @@ Similar to `lapply` in native R, `spark.lapply` runs a function over a list of e Applies a function in a manner that is similar to `doParallel` or `lapply` to elements of a list. The results of all the computations should fit in a single machine. If that is not the case they can do something like `df <- createDataFrame(list)` and then use `dapply` +
    {% highlight r %} - # Perform distributed training of multiple models with spark.lapply. Here, we pass # a read-only list of arguments which specifies family the generalized linear model should be. families <- c("gaussian", "poisson") @@ -436,4 +446,3 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` - From 550d0e7dc6339fac0fe3bb5a8d6038681fd3fec3 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 13 Jul 2016 15:23:37 -0700 Subject: [PATCH 0968/1470] [SPARK-16482][SQL] Describe Table Command for Tables Requiring Runtime Inferred Schema #### What changes were proposed in this pull request? If we create a table pointing to a parquet/json datasets without specifying the schema, describe table command does not show the schema at all. It only shows `# Schema of this table is inferred at runtime`. In 1.6, describe table does show the schema of such a table. ~~For data source tables, to infer the schema, we need to load the data source tables at runtime. Thus, this PR calls the function `lookupRelation`.~~ For data source tables, we infer the schema before table creation. Thus, this PR set the inferred schema as the table schema when table creation. #### How was this patch tested? Added test cases Author: gatorsmile Closes #14148 from gatorsmile/describeSchema. (cherry picked from commit c5ec879828369ec1d21acd7f18a792306634ff74) Signed-off-by: Yin Huai --- .../spark/sql/execution/command/tables.scala | 28 +++++++++---------- .../sql/hive/execution/HiveDDLSuite.scala | 16 ++++++----- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 14836044cabee..b2300b416d346 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -413,29 +413,29 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF } else { val metadata = catalog.getTableMetadata(table) + if (DDLUtils.isDatasourceTable(metadata)) { + DDLUtils.getSchemaFromTableProperties(metadata) match { + case Some(userSpecifiedSchema) => describeSchema(userSpecifiedSchema, result) + case None => describeSchema(catalog.lookupRelation(table).schema, result) + } + } else { + describeSchema(metadata.schema, result) + } + if (isExtended) { describeExtended(metadata, result) } else if (isFormatted) { describeFormatted(metadata, result) } else { - describe(metadata, result) + describePartitionInfo(metadata, result) } } result } - // Shows data columns and partitioned columns (if any) - private def describe(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { + private def describePartitionInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { if (DDLUtils.isDatasourceTable(table)) { - val schema = DDLUtils.getSchemaFromTableProperties(table) - - if (schema.isEmpty) { - append(buffer, "# Schema of this table is inferred at runtime", "", "") - } else { - schema.foreach(describeSchema(_, buffer)) - } - val partCols = DDLUtils.getPartitionColumnsFromTableProperties(table) if (partCols.nonEmpty) { append(buffer, "# Partition Information", "", "") @@ -443,8 +443,6 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF partCols.foreach(col => append(buffer, col, "", "")) } } else { - describeSchema(table.schema, buffer) - if (table.partitionColumns.nonEmpty) { append(buffer, "# Partition Information", "", "") append(buffer, s"# ${output.head.name}", output(1).name, output(2).name) @@ -454,14 +452,14 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF } private def describeExtended(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { - describe(table, buffer) + describePartitionInfo(table, buffer) append(buffer, "", "", "") append(buffer, "# Detailed Table Information", table.toString, "") } private def describeFormatted(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { - describe(table, buffer) + describePartitionInfo(table, buffer) append(buffer, "", "", "") append(buffer, "# Detailed Table Information", "", "") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 343d7bae98bff..92282420214d0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -612,15 +612,17 @@ class HiveDDLSuite } test("desc table for data source table - no user-defined schema") { - withTable("t1") { - withTempPath { dir => - val path = dir.getCanonicalPath - spark.range(1).write.parquet(path) - sql(s"CREATE TABLE t1 USING parquet OPTIONS (PATH '$path')") + Seq("parquet", "json", "orc").foreach { fileFormat => + withTable("t1") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(1).write.format(fileFormat).save(path) + sql(s"CREATE TABLE t1 USING $fileFormat OPTIONS (PATH '$path')") - val desc = sql("DESC FORMATTED t1").collect().toSeq + val desc = sql("DESC FORMATTED t1").collect().toSeq - assert(desc.contains(Row("# Schema of this table is inferred at runtime", "", ""))) + assert(desc.contains(Row("id", "bigint", ""))) + } } } } From abb8023599df4a9b5133accf547607eda5ca45d2 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 13 Jul 2016 15:40:44 -0700 Subject: [PATCH 0969/1470] [SPARK-16485][ML][DOC] Fix privacy of GLM members, rename sqlDataTypes for ML, doc fixes ## What changes were proposed in this pull request? Fixing issues found during 2.0 API checks: * GeneralizedLinearRegressionModel: linkObj, familyObj, familyAndLink should not be exposed * sqlDataTypes: name does not follow conventions. Do we need to expose it? * Evaluator: inconsistent doc between evaluate and isLargerBetter * MinMaxScaler: math rendering --> hard to make it great, but I'll change it a little * GeneralizedLinearRegressionSummary: aic doc is incorrect --> will change to use more common name ## How was this patch tested? Existing unit tests. Docs generated locally. (MinMaxScaler is improved a tiny bit.) Author: Joseph K. Bradley Closes #14187 from jkbradley/final-api-check-2.0. (cherry picked from commit a5f51e21627c1bcfc62829a3a962707abf41a452) Signed-off-by: Joseph K. Bradley --- .../org/apache/spark/ml/evaluation/Evaluator.scala | 7 +++++-- .../org/apache/spark/ml/feature/MinMaxScaler.scala | 4 ++-- .../ml/linalg/{dataTypes.scala => SQLDataTypes.scala} | 5 +++-- .../ml/regression/GeneralizedLinearRegression.scala | 10 +++++----- .../apache/spark/ml/linalg/JavaSQLDataTypesSuite.java | 2 +- .../org/apache/spark/ml/linalg/SQLDataTypesSuite.scala | 4 ++-- 6 files changed, 18 insertions(+), 14 deletions(-) rename mllib/src/main/scala/org/apache/spark/ml/linalg/{dataTypes.scala => SQLDataTypes.scala} (92%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala index 5f765c071b9cd..dfbc3e5222c2f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala @@ -30,7 +30,8 @@ import org.apache.spark.sql.Dataset abstract class Evaluator extends Params { /** - * Evaluates model output and returns a scalar metric (larger is better). + * Evaluates model output and returns a scalar metric. + * The value of [[isLargerBetter]] specifies whether larger values are better. * * @param dataset a dataset that contains labels/observations and predictions. * @param paramMap parameter map that specifies the input columns and output metrics @@ -42,7 +43,9 @@ abstract class Evaluator extends Params { } /** - * Evaluates the output. + * Evaluates model output and returns a scalar metric. + * The value of [[isLargerBetter]] specifies whether larger values are better. + * * @param dataset a dataset that contains labels/observations and predictions. * @return metric */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 7b03f0c0f341e..9ed8d83324cf6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -78,9 +78,9 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H * statistics, which is also known as min-max normalization or Rescaling. The rescaled value for * feature E is calculated as, * - * Rescaled(e_i) = \frac{e_i - E_{min}}{E_{max} - E_{min}} * (max - min) + min + * `Rescaled(e_i) = \frac{e_i - E_{min}}{E_{max} - E_{min}} * (max - min) + min` * - * For the case E_{max} == E_{min}, Rescaled(e_i) = 0.5 * (max + min) + * For the case `E_{max} == E_{min}`, `Rescaled(e_i) = 0.5 * (max + min)`. * Note that since zero values will probably be transformed to non-zero values, output of the * transformer will be DenseVector even for sparse input. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/dataTypes.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala similarity index 92% rename from mllib/src/main/scala/org/apache/spark/ml/linalg/dataTypes.scala rename to mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala index 52a6fd25e2fa7..a66ba27a7b9c5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/dataTypes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala @@ -17,15 +17,16 @@ package org.apache.spark.ml.linalg -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.sql.types.DataType /** * :: DeveloperApi :: * SQL data types for vectors and matrices. */ +@Since("2.0.0") @DeveloperApi -object sqlDataTypes { +object SQLDataTypes { /** Data type for [[Vector]]. */ val VectorType: DataType = new VectorUDT diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index a23e90d9e1259..2bdc09e1db246 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -376,7 +376,7 @@ object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLine def deviance(y: Double, mu: Double, weight: Double): Double /** - * Akaike's 'An Information Criterion'(AIC) value of the family for a given dataset. + * Akaike Information Criterion (AIC) value of the family for a given dataset. * * @param predictions an RDD of (y, mu, weight) of instances in evaluation dataset * @param deviance the deviance for the fitted model in evaluation dataset @@ -702,13 +702,13 @@ class GeneralizedLinearRegressionModel private[ml] ( import GeneralizedLinearRegression._ - lazy val familyObj = Family.fromName($(family)) - lazy val linkObj = if (isDefined(link)) { + private lazy val familyObj = Family.fromName($(family)) + private lazy val linkObj = if (isDefined(link)) { Link.fromName($(link)) } else { familyObj.defaultLink } - lazy val familyAndLink = new FamilyAndLink(familyObj, linkObj) + private lazy val familyAndLink = new FamilyAndLink(familyObj, linkObj) override protected def predict(features: Vector): Double = { val eta = predictLink(features) @@ -1021,7 +1021,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( rss / degreesOfFreedom } - /** Akaike's "An Information Criterion"(AIC) for the fitted model. */ + /** Akaike Information Criterion (AIC) for the fitted model. */ @Since("2.0.0") lazy val aic: Double = { val w = weightCol diff --git a/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java b/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java index b09e13112f124..bd64a7186eac0 100644 --- a/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java @@ -20,7 +20,7 @@ import org.junit.Assert; import org.junit.Test; -import static org.apache.spark.ml.linalg.sqlDataTypes.*; +import static org.apache.spark.ml.linalg.SQLDataTypes.*; public class JavaSQLDataTypesSuite { @Test diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala index 13bf3d3015f64..0bd0c32f19d04 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite class SQLDataTypesSuite extends SparkFunSuite { test("sqlDataTypes") { - assert(sqlDataTypes.VectorType === new VectorUDT) - assert(sqlDataTypes.MatrixType === new MatrixUDT) + assert(SQLDataTypes.VectorType === new VectorUDT) + assert(SQLDataTypes.MatrixType === new MatrixUDT) } } From 47eb9a621e1ca37ecd31fa03f8bb42b83c366c01 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 13 Jul 2016 22:27:07 -0700 Subject: [PATCH 0970/1470] Preparing Spark release v2.0.0-rc3 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 507ddc778383d..5f546bb9ee697 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bc3b0fe73f6eb..2eaa8100a61d4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 2fb5835305a25..f068d9d4ac428 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 07d9f1c58f7a3..fd221883b1fba 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5e02efdc45e67..a17aba5abe548 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e7fc6a2a02417..0bd8846f452c5 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 24f0e75f2f045..f495b0dddee54 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index bb27ec916c65a..093ca69abb866 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d2227944d92dd..fe9f85ae5a085 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 18e14c7981d80..ca5e4980eb8c7 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 3847b39e17315..b4cdbda6b613b 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7fc8a4b00dc20..4e7e17de6a417 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index dcdbcc55d8785..cc5a13721ee3f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1eb349ff4c3b9..636b233a0de04 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 59f41f1e17f30..093cfb76b6aea 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 26965612cc0ab..b4e9eba6efd4d 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a346d76210932..1438e2ba59e6a 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 2aa7d82e8e064..dc704375dcc44 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 58c57c1c539df..af0b81a0cff82 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b524001d04719..919778bac01e9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 93ac8b6e664d7..f7588d1917ba5 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4f8af77792b0b..f8dd7f7fd751c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index b6591598ee127..e015b95cafd15 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1d8f7f4d9bbed..86edf650b5c7b 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40fde1bab7ad1..407cb6da8281c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 9f3d7f003584c..ff7b36f495a85 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 0b5ec1a08c821..9ca84e986a3dc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0bfdb13cec941..259e56e4afef9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02a18b33b087a..0b06c29302d6e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 672425c86ecbe..ce66c71ccae6e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index add4375364b1a..5ad0c51ad05c8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index e7614fe66110d..21d8d062bd380 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 3f4cce1ca354e..a2124104f4fb9 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 7dba1a829fb94..1ca840d8c0968 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml From 5244f86cfc8532587b70cf1cc20b0684583b9c23 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 13 Jul 2016 22:27:15 -0700 Subject: [PATCH 0971/1470] Preparing development version 2.0.1-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5f546bb9ee697..507ddc778383d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2eaa8100a61d4..bc3b0fe73f6eb 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f068d9d4ac428..2fb5835305a25 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index fd221883b1fba..07d9f1c58f7a3 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index a17aba5abe548..5e02efdc45e67 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 0bd8846f452c5..e7fc6a2a02417 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index f495b0dddee54..24f0e75f2f045 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 093ca69abb866..bb27ec916c65a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index fe9f85ae5a085..d2227944d92dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ca5e4980eb8c7..18e14c7981d80 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index b4cdbda6b613b..3847b39e17315 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 4e7e17de6a417..7fc8a4b00dc20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index cc5a13721ee3f..dcdbcc55d8785 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 636b233a0de04..1eb349ff4c3b9 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 093cfb76b6aea..59f41f1e17f30 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index b4e9eba6efd4d..26965612cc0ab 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 1438e2ba59e6a..a346d76210932 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dc704375dcc44..2aa7d82e8e064 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index af0b81a0cff82..58c57c1c539df 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 919778bac01e9..b524001d04719 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index f7588d1917ba5..93ac8b6e664d7 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f8dd7f7fd751c..4f8af77792b0b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e015b95cafd15..b6591598ee127 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 86edf650b5c7b..1d8f7f4d9bbed 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 407cb6da8281c..40fde1bab7ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ff7b36f495a85..9f3d7f003584c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 9ca84e986a3dc..0b5ec1a08c821 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 259e56e4afef9..0bfdb13cec941 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0b06c29302d6e..02a18b33b087a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index ce66c71ccae6e..672425c86ecbe 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5ad0c51ad05c8..add4375364b1a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 21d8d062bd380..e7614fe66110d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2124104f4fb9..3f4cce1ca354e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 1ca840d8c0968..7dba1a829fb94 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml From f6eda6b3020a5be0ff898111bf84c71e36e9433b Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Wed, 13 Jul 2016 22:30:46 -0700 Subject: [PATCH 0972/1470] [SPARK-16503] SparkSession should provide Spark version ## What changes were proposed in this pull request? This patch enables SparkSession to provide spark version. ## How was this patch tested? Manual test: ``` scala> sc.version res0: String = 2.1.0-SNAPSHOT scala> spark.version res1: String = 2.1.0-SNAPSHOT ``` ``` >>> sc.version u'2.1.0-SNAPSHOT' >>> spark.version u'2.1.0-SNAPSHOT' ``` Author: Liwei Lin Closes #14165 from lw-lin/add-version. (cherry picked from commit 39c836e976fcae51568bed5ebab28e148383b5d4) Signed-off-by: Reynold Xin --- python/pyspark/sql/session.py | 6 ++++++ .../main/scala/org/apache/spark/sql/SparkSession.scala | 9 ++++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index a360fbefa492c..594f9375f7678 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -232,6 +232,12 @@ def sparkContext(self): """Returns the underlying :class:`SparkContext`.""" return self._sc + @property + @since(2.0) + def version(self): + """The version of Spark on which this application is running.""" + return self._jsparkSession.version() + @property @since(2.0) def conf(self): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index a3fd39d42eeb9..df0950de1525e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -25,7 +25,7 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging @@ -79,6 +79,13 @@ class SparkSession private( sparkContext.assertNotStopped() + /** + * The version of Spark on which this application is running. + * + * @since 2.0.0 + */ + def version: String = SPARK_VERSION + /* ----------------------- * | Session-related state | * ----------------------- */ From 48d1fa3e736b2ffbb660839735cbb8867f00fee2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 13 Jul 2016 22:32:45 -0700 Subject: [PATCH 0973/1470] Preparing Spark release v2.0.0-rc3 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 507ddc778383d..5f546bb9ee697 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bc3b0fe73f6eb..2eaa8100a61d4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 2fb5835305a25..f068d9d4ac428 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 07d9f1c58f7a3..fd221883b1fba 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5e02efdc45e67..a17aba5abe548 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e7fc6a2a02417..0bd8846f452c5 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 24f0e75f2f045..f495b0dddee54 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index bb27ec916c65a..093ca69abb866 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d2227944d92dd..fe9f85ae5a085 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 18e14c7981d80..ca5e4980eb8c7 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 3847b39e17315..b4cdbda6b613b 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7fc8a4b00dc20..4e7e17de6a417 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index dcdbcc55d8785..cc5a13721ee3f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1eb349ff4c3b9..636b233a0de04 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 59f41f1e17f30..093cfb76b6aea 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 26965612cc0ab..b4e9eba6efd4d 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a346d76210932..1438e2ba59e6a 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 2aa7d82e8e064..dc704375dcc44 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 58c57c1c539df..af0b81a0cff82 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b524001d04719..919778bac01e9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 93ac8b6e664d7..f7588d1917ba5 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4f8af77792b0b..f8dd7f7fd751c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index b6591598ee127..e015b95cafd15 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1d8f7f4d9bbed..86edf650b5c7b 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40fde1bab7ad1..407cb6da8281c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 9f3d7f003584c..ff7b36f495a85 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 0b5ec1a08c821..9ca84e986a3dc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0bfdb13cec941..259e56e4afef9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02a18b33b087a..0b06c29302d6e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 672425c86ecbe..ce66c71ccae6e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index add4375364b1a..5ad0c51ad05c8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index e7614fe66110d..21d8d062bd380 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 3f4cce1ca354e..a2124104f4fb9 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 7dba1a829fb94..1ca840d8c0968 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml From b3ebecbb7a5f2ce55ba0392bc59b26780ff69e13 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 13 Jul 2016 22:32:55 -0700 Subject: [PATCH 0974/1470] Preparing development version 2.0.1-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5f546bb9ee697..507ddc778383d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2eaa8100a61d4..bc3b0fe73f6eb 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f068d9d4ac428..2fb5835305a25 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index fd221883b1fba..07d9f1c58f7a3 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index a17aba5abe548..5e02efdc45e67 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 0bd8846f452c5..e7fc6a2a02417 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index f495b0dddee54..24f0e75f2f045 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 093ca69abb866..bb27ec916c65a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index fe9f85ae5a085..d2227944d92dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ca5e4980eb8c7..18e14c7981d80 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index b4cdbda6b613b..3847b39e17315 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 4e7e17de6a417..7fc8a4b00dc20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index cc5a13721ee3f..dcdbcc55d8785 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 636b233a0de04..1eb349ff4c3b9 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 093cfb76b6aea..59f41f1e17f30 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index b4e9eba6efd4d..26965612cc0ab 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 1438e2ba59e6a..a346d76210932 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dc704375dcc44..2aa7d82e8e064 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index af0b81a0cff82..58c57c1c539df 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 919778bac01e9..b524001d04719 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index f7588d1917ba5..93ac8b6e664d7 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f8dd7f7fd751c..4f8af77792b0b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e015b95cafd15..b6591598ee127 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 86edf650b5c7b..1d8f7f4d9bbed 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 407cb6da8281c..40fde1bab7ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ff7b36f495a85..9f3d7f003584c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 9ca84e986a3dc..0b5ec1a08c821 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 259e56e4afef9..0bfdb13cec941 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0b06c29302d6e..02a18b33b087a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index ce66c71ccae6e..672425c86ecbe 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5ad0c51ad05c8..add4375364b1a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 21d8d062bd380..e7614fe66110d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2124104f4fb9..3f4cce1ca354e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 1ca840d8c0968..7dba1a829fb94 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml From 240c42b284b3f4bd302984fa51513c249f6d7648 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Thu, 14 Jul 2016 09:11:04 +0100 Subject: [PATCH 0975/1470] [SPARK-16500][ML][MLLIB][OPTIMIZER] add LBFGS convergence warning for all used place in MLLib ## What changes were proposed in this pull request? Add warning_for the following case when LBFGS training not actually convergence: 1) LogisticRegression 2) AFTSurvivalRegression 3) LBFGS algorithm wrapper in mllib package ## How was this patch tested? N/A Author: WeichenXu Closes #14157 from WeichenXu123/add_lbfgs_convergence_warning_for_all_used_place. (cherry picked from commit 252d4f27f23b547777892bcea25a2cea62d8cbab) Signed-off-by: Sean Owen --- .../apache/spark/ml/classification/LogisticRegression.scala | 5 +++++ .../apache/spark/ml/regression/AFTSurvivalRegression.scala | 5 +++++ .../scala/org/apache/spark/mllib/optimization/LBFGS.scala | 6 ++++++ 3 files changed, 16 insertions(+) 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 e157bdeb5b7ee..4bab801bb3621 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 @@ -424,6 +424,11 @@ class LogisticRegression @Since("1.2.0") ( throw new SparkException(msg) } + if (!state.actuallyConverged) { + logWarning("LogisticRegression training fininshed but the result " + + s"is not converged because: ${state.convergedReason.get.reason}") + } + /* The coefficients are trained in the scaled space; we're converting them back to the original space. diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 7c51845a25815..366448fc56bf5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -245,6 +245,11 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S throw new SparkException(msg) } + if (!state.actuallyConverged) { + logWarning("AFTSurvivalRegression training fininshed but the result " + + s"is not converged because: ${state.convergedReason.get.reason}") + } + state.x.toArray.clone() } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index ec6ffe6e19439..c61b2db6c961c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -212,6 +212,12 @@ object LBFGS extends Logging { state = states.next() } lossHistory += state.value + + if (!state.actuallyConverged) { + logWarning("LBFGS training fininshed but the result " + + s"is not converged because: ${state.convergedReason.get.reason}") + } + val weights = Vectors.fromBreeze(state.x) val lossHistoryArray = lossHistory.result() From 4e9080f44431bc4b91118d53f9234420f2f3d584 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Thu, 14 Jul 2016 09:38:42 -0700 Subject: [PATCH 0976/1470] [SPARK-16509][SPARKR] Rename window.partitionBy and window.orderBy to windowPartitionBy and windowOrderBy. ## What changes were proposed in this pull request? Rename window.partitionBy and window.orderBy to windowPartitionBy and windowOrderBy to pass CRAN package check. ## How was this patch tested? SparkR unit tests. Author: Sun Rui Closes #14192 from sun-rui/SPARK-16509. (cherry picked from commit 093ebbc628699b40f091b5b7083c119fffa9314b) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 4 +- R/pkg/R/WindowSpec.R | 4 +- R/pkg/R/generics.R | 8 ++-- R/pkg/R/window.R | 54 ++++++++++++++--------- R/pkg/inst/tests/testthat/test_sparkSQL.R | 8 ++-- 5 files changed, 44 insertions(+), 34 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index bc3aceba22568..fe52905e0a850 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -341,5 +341,5 @@ export("partitionBy", "rowsBetween", "rangeBetween") -export("window.partitionBy", - "window.orderBy") +export("windowPartitionBy", + "windowOrderBy") diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index 9f3b1e4be5609..e20d05decefc7 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -22,10 +22,10 @@ NULL #' S4 class that represents a WindowSpec #' -#' WindowSpec can be created by using window.partitionBy() or window.orderBy() +#' WindowSpec can be created by using windowPartitionBy() or windowOrderBy() #' #' @rdname WindowSpec -#' @seealso \link{window.partitionBy}, \link{window.orderBy} +#' @seealso \link{windowPartitionBy}, \link{windowOrderBy} #' #' @param sws A Java object reference to the backing Scala WindowSpec #' @export diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index df057bda99f8e..8416e5cdb8bf4 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -779,13 +779,13 @@ setGeneric("rowsBetween", function(x, start, end) { standardGeneric("rowsBetween #' @export setGeneric("rangeBetween", function(x, start, end) { standardGeneric("rangeBetween") }) -#' @rdname window.partitionBy +#' @rdname windowPartitionBy #' @export -setGeneric("window.partitionBy", function(col, ...) { standardGeneric("window.partitionBy") }) +setGeneric("windowPartitionBy", function(col, ...) { standardGeneric("windowPartitionBy") }) -#' @rdname window.orderBy +#' @rdname windowOrderBy #' @export -setGeneric("window.orderBy", function(col, ...) { standardGeneric("window.orderBy") }) +setGeneric("windowOrderBy", function(col, ...) { standardGeneric("windowOrderBy") }) ###################### Expression Function Methods ########################## diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R index e4bc933b9aaba..d9d069cde9309 100644 --- a/R/pkg/R/window.R +++ b/R/pkg/R/window.R @@ -17,23 +17,28 @@ # window.R - Utility functions for defining window in DataFrames -#' window.partitionBy +#' windowPartitionBy #' #' Creates a WindowSpec with the partitioning defined. #' -#' @rdname window.partitionBy -#' @name window.partitionBy +#' @param col A column name or Column by which rows are partitioned to +#' windows. +#' @param ... Optional column names or Columns in addition to col, by +#' which rows are partitioned to windows. +#' +#' @rdname windowPartitionBy +#' @name windowPartitionBy #' @export #' @examples #' \dontrun{ -#' ws <- window.partitionBy("key1", "key2") +#' ws <- windowPartitionBy("key1", "key2") #' df1 <- select(df, over(lead("value", 1), ws)) #' -#' ws <- window.partitionBy(df$key1, df$key2) +#' ws <- windowPartitionBy(df$key1, df$key2) #' df1 <- select(df, over(lead("value", 1), ws)) #' } -#' @note window.partitionBy(character) since 2.0.0 -setMethod("window.partitionBy", +#' @note windowPartitionBy(character) since 2.0.0 +setMethod("windowPartitionBy", signature(col = "character"), function(col, ...) { windowSpec( @@ -43,11 +48,11 @@ setMethod("window.partitionBy", list(...))) }) -#' @rdname window.partitionBy -#' @name window.partitionBy +#' @rdname windowPartitionBy +#' @name windowPartitionBy #' @export -#' @note window.partitionBy(Column) since 2.0.0 -setMethod("window.partitionBy", +#' @note windowPartitionBy(Column) since 2.0.0 +setMethod("windowPartitionBy", signature(col = "Column"), function(col, ...) { jcols <- lapply(list(col, ...), function(c) { @@ -59,23 +64,28 @@ setMethod("window.partitionBy", jcols)) }) -#' window.orderBy +#' windowOrderBy #' #' Creates a WindowSpec with the ordering defined. #' -#' @rdname window.orderBy -#' @name window.orderBy +#' @param col A column name or Column by which rows are ordered within +#' windows. +#' @param ... Optional column names or Columns in addition to col, by +#' which rows are ordered within windows. +#' +#' @rdname windowOrderBy +#' @name windowOrderBy #' @export #' @examples #' \dontrun{ -#' ws <- window.orderBy("key1", "key2") +#' ws <- windowOrderBy("key1", "key2") #' df1 <- select(df, over(lead("value", 1), ws)) #' -#' ws <- window.orderBy(df$key1, df$key2) +#' ws <- windowOrderBy(df$key1, df$key2) #' df1 <- select(df, over(lead("value", 1), ws)) #' } -#' @note window.orderBy(character) since 2.0.0 -setMethod("window.orderBy", +#' @note windowOrderBy(character) since 2.0.0 +setMethod("windowOrderBy", signature(col = "character"), function(col, ...) { windowSpec( @@ -85,11 +95,11 @@ setMethod("window.orderBy", list(...))) }) -#' @rdname window.orderBy -#' @name window.orderBy +#' @rdname windowOrderBy +#' @name windowOrderBy #' @export -#' @note window.orderBy(Column) since 2.0.0 -setMethod("window.orderBy", +#' @note windowOrderBy(Column) since 2.0.0 +setMethod("windowOrderBy", signature(col = "Column"), function(col, ...) { jcols <- lapply(list(col, ...), function(c) { diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index e26b015d9371b..1bfdc3436e639 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2368,7 +2368,7 @@ test_that("gapply() and gapplyCollect() on a DataFrame", { test_that("Window functions on a DataFrame", { df <- createDataFrame(list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), schema = c("key", "value")) - ws <- orderBy(window.partitionBy("key"), "value") + ws <- orderBy(windowPartitionBy("key"), "value") result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) names(result) <- c("key", "value") expected <- data.frame(key = c(1L, NA, 2L, NA), @@ -2376,17 +2376,17 @@ test_that("Window functions on a DataFrame", { stringsAsFactors = FALSE) expect_equal(result, expected) - ws <- orderBy(window.partitionBy(df$key), df$value) + ws <- orderBy(windowPartitionBy(df$key), df$value) result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) names(result) <- c("key", "value") expect_equal(result, expected) - ws <- partitionBy(window.orderBy("value"), "key") + ws <- partitionBy(windowOrderBy("value"), "key") result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) names(result) <- c("key", "value") expect_equal(result, expected) - ws <- partitionBy(window.orderBy(df$value), df$key) + ws <- partitionBy(windowOrderBy(df$value), df$key) result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) names(result) <- c("key", "value") expect_equal(result, expected) From 29281bc40cb83ce2946b0395981c8dce5630910c Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 14 Jul 2016 09:45:30 -0700 Subject: [PATCH 0977/1470] [SPARK-16538][SPARKR] fix R call with namespace operator on SparkSession functions ## What changes were proposed in this pull request? Fix function routing to work with and without namespace operator `SparkR::createDataFrame` ## How was this patch tested? manual, unit tests shivaram Author: Felix Cheung Closes #14195 from felixcheung/rroutedefault. (cherry picked from commit 12005c88fb24168d57b577cff73eddcd9d8963fc) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/SQLContext.R | 4 +++- R/pkg/inst/tests/testthat/test_sparkSQL.R | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index bc0daa25c9f6a..d2ea1553c6a2b 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -48,7 +48,9 @@ getInternalType <- function(x) { #' @return whatever the target returns #' @noRd dispatchFunc <- function(newFuncSig, x, ...) { - funcName <- as.character(sys.call(sys.parent())[[1]]) + # When called with SparkR::createDataFrame, sys.call()[[1]] returns c(::, SparkR, createDataFrame) + callsite <- as.character(sys.call(sys.parent())[[1]]) + funcName <- callsite[[length(callsite)]] f <- get(paste0(funcName, ".default")) # Strip sqlContext from list of parameters and then pass the rest along. contextNames <- c("org.apache.spark.sql.SQLContext", diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 1bfdc3436e639..20c750a50c70a 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2397,7 +2397,8 @@ test_that("createDataFrame sqlContext parameter backward compatibility", { a <- 1:3 b <- c("a", "b", "c") ldf <- data.frame(a, b) - df <- suppressWarnings(createDataFrame(sqlContext, ldf)) + # Call function with namespace :: operator - SPARK-16538 + df <- suppressWarnings(SparkR::createDataFrame(sqlContext, ldf)) expect_equal(columns(df), c("a", "b")) expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) expect_equal(count(df), 3) From e5f8c1117e0c48499f54d62b556bc693435afae0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 14 Jul 2016 09:50:07 -0700 Subject: [PATCH 0978/1470] Preparing Spark release v2.0.0-rc4 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 507ddc778383d..5f546bb9ee697 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bc3b0fe73f6eb..2eaa8100a61d4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 2fb5835305a25..f068d9d4ac428 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 07d9f1c58f7a3..fd221883b1fba 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5e02efdc45e67..a17aba5abe548 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e7fc6a2a02417..0bd8846f452c5 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 24f0e75f2f045..f495b0dddee54 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index bb27ec916c65a..093ca69abb866 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d2227944d92dd..fe9f85ae5a085 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 18e14c7981d80..ca5e4980eb8c7 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 3847b39e17315..b4cdbda6b613b 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7fc8a4b00dc20..4e7e17de6a417 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index dcdbcc55d8785..cc5a13721ee3f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1eb349ff4c3b9..636b233a0de04 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 59f41f1e17f30..093cfb76b6aea 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 26965612cc0ab..b4e9eba6efd4d 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a346d76210932..1438e2ba59e6a 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 2aa7d82e8e064..dc704375dcc44 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 58c57c1c539df..af0b81a0cff82 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b524001d04719..919778bac01e9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 93ac8b6e664d7..f7588d1917ba5 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4f8af77792b0b..f8dd7f7fd751c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index b6591598ee127..e015b95cafd15 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1d8f7f4d9bbed..86edf650b5c7b 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40fde1bab7ad1..407cb6da8281c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 9f3d7f003584c..ff7b36f495a85 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 0b5ec1a08c821..9ca84e986a3dc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0bfdb13cec941..259e56e4afef9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02a18b33b087a..0b06c29302d6e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 672425c86ecbe..ce66c71ccae6e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index add4375364b1a..5ad0c51ad05c8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index e7614fe66110d..21d8d062bd380 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 3f4cce1ca354e..a2124104f4fb9 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 7dba1a829fb94..1ca840d8c0968 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml From 0a651aa26b27ec679a5878154e75c92111689981 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 14 Jul 2016 09:50:16 -0700 Subject: [PATCH 0979/1470] Preparing development version 2.0.1-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5f546bb9ee697..507ddc778383d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2eaa8100a61d4..bc3b0fe73f6eb 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f068d9d4ac428..2fb5835305a25 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index fd221883b1fba..07d9f1c58f7a3 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index a17aba5abe548..5e02efdc45e67 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 0bd8846f452c5..e7fc6a2a02417 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index f495b0dddee54..24f0e75f2f045 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 093ca69abb866..bb27ec916c65a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index fe9f85ae5a085..d2227944d92dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ca5e4980eb8c7..18e14c7981d80 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index b4cdbda6b613b..3847b39e17315 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 4e7e17de6a417..7fc8a4b00dc20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index cc5a13721ee3f..dcdbcc55d8785 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 636b233a0de04..1eb349ff4c3b9 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 093cfb76b6aea..59f41f1e17f30 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index b4e9eba6efd4d..26965612cc0ab 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 1438e2ba59e6a..a346d76210932 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dc704375dcc44..2aa7d82e8e064 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index af0b81a0cff82..58c57c1c539df 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 919778bac01e9..b524001d04719 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index f7588d1917ba5..93ac8b6e664d7 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f8dd7f7fd751c..4f8af77792b0b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e015b95cafd15..b6591598ee127 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 86edf650b5c7b..1d8f7f4d9bbed 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 407cb6da8281c..40fde1bab7ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ff7b36f495a85..9f3d7f003584c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 9ca84e986a3dc..0b5ec1a08c821 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 259e56e4afef9..0bfdb13cec941 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0b06c29302d6e..02a18b33b087a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index ce66c71ccae6e..672425c86ecbe 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5ad0c51ad05c8..add4375364b1a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 21d8d062bd380..e7614fe66110d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2124104f4fb9..3f4cce1ca354e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 1ca840d8c0968..7dba1a829fb94 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml From 7418019218e5a2bd4ae948bb1984816f161925cf Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 15 Jul 2016 00:51:11 +0800 Subject: [PATCH 0980/1470] [SPARK-16529][SQL][TEST] `withTempDatabase` should set `default` database before dropping ## What changes were proposed in this pull request? `SQLTestUtils.withTempDatabase` is a frequently used test harness to setup a temporary table and clean up finally. This issue improves like the following for usability. ```scala - try f(dbName) finally spark.sql(s"DROP DATABASE $dbName CASCADE") + try f(dbName) finally { + if (spark.catalog.currentDatabase == dbName) { + spark.sql(s"USE ${DEFAULT_DATABASE}") + } + spark.sql(s"DROP DATABASE $dbName CASCADE") + } ``` In case of forgetting to reset the databaes, `withTempDatabase` will not raise Exception. ## How was this patch tested? This improves test harness. Author: Dongjoon Hyun Closes #14184 from dongjoon-hyun/SPARK-16529. (cherry picked from commit c576f9fb90853cce2e8e5dcc32a536a0f49cbbd8) Signed-off-by: Cheng Lian --- .../scala/org/apache/spark/sql/test/SQLTestUtils.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 853dd0ff3f601..26bd3fb7eb27b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -29,6 +29,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ @@ -196,7 +197,12 @@ private[sql] trait SQLTestUtils fail("Failed to create temporary database", cause) } - try f(dbName) finally spark.sql(s"DROP DATABASE $dbName CASCADE") + try f(dbName) finally { + if (spark.catalog.currentDatabase == dbName) { + spark.sql(s"USE ${DEFAULT_DATABASE}") + } + spark.sql(s"DROP DATABASE $dbName CASCADE") + } } /** From 23e1ab9c7d56946647cb2081bd384d174bce1882 Mon Sep 17 00:00:00 2001 From: Jacek Lewandowski Date: Thu, 14 Jul 2016 10:18:31 -0700 Subject: [PATCH 0981/1470] [SPARK-16528][SQL] Fix NPE problem in HiveClientImpl ## What changes were proposed in this pull request? There are some calls to methods or fields (getParameters, properties) which are then passed to Java/Scala collection converters. Unfortunately those fields can be null in some cases and then the conversions throws NPE. We fix it by wrapping calls to those fields and methods with option and then do the conversion. ## How was this patch tested? Manually tested with a custom Hive metastore. Author: Jacek Lewandowski Closes #14200 from jacek-lewandowski/SPARK-16528. (cherry picked from commit 31ca741aef9dd138529e064785c8e58b86140ff5) Signed-off-by: Reynold Xin --- .../spark/sql/hive/client/HiveClientImpl.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 1c89d8c62a3ad..6cdf3ef545009 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -293,7 +293,7 @@ private[hive] class HiveClientImpl( database.name, database.description, database.locationUri, - database.properties.asJava), + Option(database.properties).map(_.asJava).orNull), ignoreIfExists) } @@ -311,7 +311,7 @@ private[hive] class HiveClientImpl( database.name, database.description, database.locationUri, - database.properties.asJava)) + Option(database.properties).map(_.asJava).orNull)) } override def getDatabaseOption(name: String): Option[CatalogDatabase] = withHiveState { @@ -320,7 +320,7 @@ private[hive] class HiveClientImpl( name = d.getName, description = d.getDescription, locationUri = d.getLocationUri, - properties = d.getParameters.asScala.toMap) + properties = Option(d.getParameters).map(_.asScala.toMap).orNull) } } @@ -353,7 +353,7 @@ private[hive] class HiveClientImpl( unsupportedFeatures += "bucketing" } - val properties = h.getParameters.asScala.toMap + val properties = Option(h.getParameters).map(_.asScala.toMap).orNull CatalogTable( identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), @@ -390,7 +390,8 @@ private[hive] class HiveClientImpl( outputFormat = Option(h.getOutputFormatClass).map(_.getName), serde = Option(h.getSerializationLib), compressed = h.getTTable.getSd.isCompressed, - serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap + serdeProperties = Option(h.getTTable.getSd.getSerdeInfo.getParameters) + .map(_.asScala.toMap).orNull ), properties = properties, viewOriginalText = Option(h.getViewOriginalText), @@ -815,6 +816,7 @@ private[hive] class HiveClientImpl( outputFormat = Option(apiPartition.getSd.getOutputFormat), serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), compressed = apiPartition.getSd.isCompressed, - serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) + serdeProperties = Option(apiPartition.getSd.getSerdeInfo.getParameters) + .map(_.asScala.toMap).orNull)) } } From 1fe0bcdd0bf39dd4993bf2ec35f66eec1b949f5b Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 14 Jul 2016 10:40:59 -0700 Subject: [PATCH 0982/1470] [SPARK-16540][YARN][CORE] Avoid adding jars twice for Spark running on yarn ## What changes were proposed in this pull request? Currently when running spark on yarn, jars specified with --jars, --packages will be added twice, one is Spark's own file server, another is yarn's distributed cache, this can be seen from log: for example: ``` ./bin/spark-shell --master yarn-client --jars examples/target/scala-2.11/jars/scopt_2.11-3.3.0.jar ``` If specified the jar to be added is scopt jar, it will added twice: ``` ... 16/07/14 15:06:48 INFO Server: Started 5603ms 16/07/14 15:06:48 INFO Utils: Successfully started service 'SparkUI' on port 4040. 16/07/14 15:06:48 INFO SparkUI: Bound SparkUI to 0.0.0.0, and started at http://192.168.0.102:4040 16/07/14 15:06:48 INFO SparkContext: Added JAR file:/Users/sshao/projects/apache-spark/examples/target/scala-2.11/jars/scopt_2.11-3.3.0.jar at spark://192.168.0.102:63996/jars/scopt_2.11-3.3.0.jar with timestamp 1468480008637 16/07/14 15:06:49 INFO RMProxy: Connecting to ResourceManager at /0.0.0.0:8032 16/07/14 15:06:49 INFO Client: Requesting a new application from cluster with 1 NodeManagers 16/07/14 15:06:49 INFO Client: Verifying our application has not requested more than the maximum memory capability of the cluster (8192 MB per container) 16/07/14 15:06:49 INFO Client: Will allocate AM container, with 896 MB memory including 384 MB overhead 16/07/14 15:06:49 INFO Client: Setting up container launch context for our AM 16/07/14 15:06:49 INFO Client: Setting up the launch environment for our AM container 16/07/14 15:06:49 INFO Client: Preparing resources for our AM container 16/07/14 15:06:49 WARN Client: Neither spark.yarn.jars nor spark.yarn.archive is set, falling back to uploading libraries under SPARK_HOME. 16/07/14 15:06:50 INFO Client: Uploading resource file:/private/var/folders/tb/8pw1511s2q78mj7plnq8p9g40000gn/T/spark-a446300b-84bf-43ff-bfb1-3adfb0571a42/__spark_libs__6486179704064718817.zip -> hdfs://localhost:8020/user/sshao/.sparkStaging/application_1468468348998_0009/__spark_libs__6486179704064718817.zip 16/07/14 15:06:51 INFO Client: Uploading resource file:/Users/sshao/projects/apache-spark/examples/target/scala-2.11/jars/scopt_2.11-3.3.0.jar -> hdfs://localhost:8020/user/sshao/.sparkStaging/application_1468468348998_0009/scopt_2.11-3.3.0.jar 16/07/14 15:06:51 INFO Client: Uploading resource file:/private/var/folders/tb/8pw1511s2q78mj7plnq8p9g40000gn/T/spark-a446300b-84bf-43ff-bfb1-3adfb0571a42/__spark_conf__326416236462420861.zip -> hdfs://localhost:8020/user/sshao/.sparkStaging/application_1468468348998_0009/__spark_conf__.zip ... ``` So here try to avoid adding jars to Spark's fileserver unnecessarily. ## How was this patch tested? Manually verified both in yarn client and cluster mode, also in standalone mode. Author: jerryshao Closes #14196 from jerryshao/SPARK-16540. (cherry picked from commit 91575cac32e470d7079a55fb86d66332aba599d0) Signed-off-by: Marcelo Vanzin --- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 ++-- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- .../src/main/scala/org/apache/spark/repl/Main.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index a79d1959979a7..be1ae401d9504 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2405,9 +2405,9 @@ private[spark] object Utils extends Logging { * "spark.yarn.dist.jars" properties, while in other modes it returns the jar files pointed by * only the "spark.jars" property. */ - def getUserJars(conf: SparkConf): Seq[String] = { + def getUserJars(conf: SparkConf, isShell: Boolean = false): Seq[String] = { val sparkJars = conf.getOption("spark.jars") - if (conf.get("spark.master") == "yarn") { + if (conf.get("spark.master") == "yarn" && isShell) { val yarnJars = conf.getOption("spark.yarn.dist.jars") unionFileLists(sparkJars, yarnJars).toSeq } else { diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index e871004173704..16f330a320a4b 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1066,7 +1066,7 @@ class SparkILoop( logWarning("ADD_JARS environment variable is deprecated, use --jar spark submit argument instead") } val jars = { - val userJars = Utils.getUserJars(conf) + val userJars = Utils.getUserJars(conf, isShell = true) if (userJars.isEmpty) { envJars.getOrElse("") } else { diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 28fe84d6fe9bd..5dfe18ad49822 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -54,7 +54,7 @@ object Main extends Logging { // Visible for testing private[repl] def doMain(args: Array[String], _interp: SparkILoop): Unit = { interp = _interp - val jars = Utils.getUserJars(conf).mkString(File.pathSeparator) + val jars = Utils.getUserJars(conf, isShell = true).mkString(File.pathSeparator) val interpArguments = List( "-Yrepl-class-based", "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", From 5c56bc00ce6d873107010e574f53a4fa5a23bd27 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 14 Jul 2016 14:19:30 -0700 Subject: [PATCH 0983/1470] [SPARK-16553][DOCS] Fix SQL example file name in docs ## What changes were proposed in this pull request? Fixes a typo in the sql programming guide ## How was this patch tested? Building docs locally (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Shivaram Venkataraman Closes #14208 from shivaram/spark-sql-doc-fix. (cherry picked from commit 01c4c1fa539a6c601ea0d8960363e895c17a8f76) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index f5d1fee943b7c..a4127dacee88f 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -249,7 +249,7 @@ In addition to simple column references and expressions, DataFrames also have a
    The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. -{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSqlExample.scala %}
    From aa4690b1bbf86f5f927ca0038dc80dc17182b268 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 14 Jul 2016 15:55:36 -0700 Subject: [PATCH 0984/1470] [SPARK-16555] Work around Jekyll error-handling bug which led to silent failures If a custom Jekyll template tag throws Ruby's equivalent of a "file not found" exception, then Jekyll will stop the doc building process but will exit with a successful status, causing our doc publishing jobs to silently fail. This is caused by https://github.com/jekyll/jekyll/issues/5104, a case of bad error-handling logic in Jekyll. This patch works around this by updating our `include_example.rb` plugin to catch the exception and exit rather than allowing it to bubble up and be ignored by Jekyll. I tested this manually with ``` rm ./examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala cd docs SKIP_API=1 jekyll build echo $? ``` Author: Josh Rosen Closes #14209 from JoshRosen/fix-doc-building. (cherry picked from commit 972673aca562b24c885801d2ac48e0df95cde9eb) Signed-off-by: Reynold Xin --- docs/_plugins/include_example.rb | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb index 306888801df21..6ea1d438f529e 100644 --- a/docs/_plugins/include_example.rb +++ b/docs/_plugins/include_example.rb @@ -45,7 +45,15 @@ def render(context) @file = File.join(@code_dir, snippet_file) @lang = snippet_file.split('.').last - code = File.open(@file).read.encode("UTF-8") + begin + code = File.open(@file).read.encode("UTF-8") + rescue => e + # We need to explicitly exit on execptions here because Jekyll will silently swallow + # them, leading to silent build failures (see https://github.com/jekyll/jekyll/issues/5104) + puts(e) + puts(e.backtrace) + exit 1 + end code = select_lines(code) rendered_code = Pygments.highlight(code, :lexer => @lang) From c5f935582c07787271dcabcfbd6a7b8e776d607a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 14 Jul 2016 19:24:42 -0700 Subject: [PATCH 0985/1470] [SPARK-16557][SQL] Remove stale doc in sql/README.md ## What changes were proposed in this pull request? Most of the documentation in https://github.com/apache/spark/blob/master/sql/README.md is stale. It would be useful to keep the list of projects to explain what's going on, and everything else should be removed. ## How was this patch tested? N/A Author: Reynold Xin Closes #14211 from rxin/SPARK-16557. (cherry picked from commit 2e4075e2ece9574100c79558cab054485e25c2ee) Signed-off-by: Reynold Xin --- sql/README.md | 75 +-------------------------------------------------- 1 file changed, 1 insertion(+), 74 deletions(-) diff --git a/sql/README.md b/sql/README.md index b0903980a59f3..58e9097ed4db1 100644 --- a/sql/README.md +++ b/sql/README.md @@ -1,83 +1,10 @@ Spark SQL ========= -This module provides support for executing relational queries expressed in either SQL or a LINQ-like Scala DSL. +This module provides support for executing relational queries expressed in either SQL or the DataFrame/Dataset API. Spark SQL is broken up into four subprojects: - Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions. - Execution (sql/core) - A query planner / execution engine for translating Catalyst's logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files. - Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs. - HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server. - - -Other dependencies for developers ---------------------------------- -In order to create new hive test cases (i.e. a test suite based on `HiveComparisonTest`), -you will need to setup your development environment based on the following instructions. - -If you are working with Hive 0.12.0, you will need to set several environmental variables as follows. - -``` -export HIVE_HOME="/hive/build/dist" -export HIVE_DEV_HOME="/hive/" -export HADOOP_HOME="/hadoop" -``` - -If you are working with Hive 0.13.1, the following steps are needed: - -1. Download Hive's [0.13.1](https://archive.apache.org/dist/hive/hive-0.13.1) and set `HIVE_HOME` with `export HIVE_HOME=""`. Please do not set `HIVE_DEV_HOME` (See [SPARK-4119](https://issues.apache.org/jira/browse/SPARK-4119)). -2. Set `HADOOP_HOME` with `export HADOOP_HOME=""` -3. Download all Hive 0.13.1a jars (Hive jars actually used by Spark) from [here](http://mvnrepository.com/artifact/org.spark-project.hive) and replace corresponding original 0.13.1 jars in `$HIVE_HOME/lib`. -4. Download [Kryo 2.21 jar](http://mvnrepository.com/artifact/com.esotericsoftware.kryo/kryo/2.21) (Note: 2.22 jar does not work) and [Javolution 5.5.1 jar](http://mvnrepository.com/artifact/javolution/javolution/5.5.1) to `$HIVE_HOME/lib`. -5. This step is optional. But, when generating golden answer files, if a Hive query fails and you find that Hive tries to talk to HDFS or you find weird runtime NPEs, set the following in your test suite... - -``` -val testTempDir = Utils.createTempDir() -// We have to use kryo to let Hive correctly serialize some plans. -sql("set hive.plan.serialization.format=kryo") -// Explicitly set fs to local fs. -sql(s"set fs.default.name=file://$testTempDir/") -// Ask Hive to run jobs in-process as a single map and reduce task. -sql("set mapred.job.tracker=local") -``` - -Using the console -================= -An interactive scala console can be invoked by running `build/sbt hive/console`. -From here you can execute queries with HiveQl and manipulate DataFrame by using DSL. - -```scala -$ build/sbt hive/console - -[info] Starting scala interpreter... -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.dsl._ -import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.hive.test.TestHive.implicits._ -import org.apache.spark.sql.types._ -Type in expressions to have them evaluated. -Type :help for more information. - -scala> val query = sql("SELECT * FROM (SELECT * FROM src) a") -query: org.apache.spark.sql.DataFrame = [key: int, value: string] -``` - -Query results are `DataFrames` and can be operated as such. -``` -scala> query.collect() -res0: Array[org.apache.spark.sql.Row] = Array([238,val_238], [86,val_86], [311,val_311], [27,val_27]... -``` - -You can also build further queries on top of these `DataFrames` using the query DSL. -``` -scala> query.where(query("key") > 30).select(avg(query("key"))).collect() -res1: Array[org.apache.spark.sql.Row] = Array([274.79025423728814]) -``` From 90686abbddcbfa85a53abb6b87a1f53f1e9adbed Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 15 Jul 2016 13:38:23 -0700 Subject: [PATCH 0986/1470] [SPARK-14817][ML][MLLIB][DOC] Made DataFrame-based API primary in MLlib guide ## What changes were proposed in this pull request? Made DataFrame-based API primary * Spark doc menu bar and other places now link to ml-guide.html, not mllib-guide.html * mllib-guide.html keeps RDD-specific list of features, with a link at the top redirecting people to ml-guide.html * ml-guide.html includes a "maintenance mode" announcement about the RDD-based API * **Reviewers: please check this carefully** * (minor) Titles for DF API no longer include "- spark.ml" suffix. Titles for RDD API have "- RDD-based API" suffix * Moved migration guide to ml-guide from mllib-guide * Also moved past guides from mllib-migration-guides to ml-migration-guides, with a redirect link on mllib-migration-guides * **Reviewers**: I did not change any of the content of the migration guides. Reorganized DataFrame-based guide: * ml-guide.html mimics the old mllib-guide.html page in terms of content: overview, migration guide, etc. * Moved Pipeline description into ml-pipeline.html and moved tuning into ml-tuning.html * **Reviewers**: I did not change the content of these guides, except some intro text. * Sidebar remains the same, but with pipeline and tuning sections added Other: * ml-classification-regression.html: Moved text about linear methods to new section in page ## How was this patch tested? Generated docs locally Author: Joseph K. Bradley Closes #14213 from jkbradley/ml-guide-2.0. (cherry picked from commit 5ffd5d3838da40ad408a6f40071fe6f4dcacf2a1) Signed-off-by: Joseph K. Bradley --- docs/_data/menu-ml.yaml | 6 +- docs/_includes/nav-left-wrapper-ml.html | 4 +- docs/_layouts/global.html | 2 +- docs/index.md | 4 +- docs/ml-advanced.md | 4 +- docs/ml-ann.md | 4 +- docs/ml-classification-regression.md | 60 +-- docs/ml-clustering.md | 8 +- docs/ml-collaborative-filtering.md | 4 +- docs/ml-decision-tree.md | 4 +- docs/ml-ensembles.md | 4 +- docs/ml-features.md | 4 +- docs/ml-guide.md | 461 +++++++++--------------- docs/ml-linear-methods.md | 4 +- docs/ml-migration-guides.md | 159 ++++++++ docs/ml-pipeline.md | 245 +++++++++++++ docs/ml-survival-regression.md | 4 +- docs/ml-tuning.md | 121 +++++++ docs/mllib-classification-regression.md | 4 +- docs/mllib-clustering.md | 4 +- docs/mllib-collaborative-filtering.md | 4 +- docs/mllib-data-types.md | 4 +- docs/mllib-decision-tree.md | 4 +- docs/mllib-dimensionality-reduction.md | 4 +- docs/mllib-ensembles.md | 4 +- docs/mllib-evaluation-metrics.md | 4 +- docs/mllib-feature-extraction.md | 4 +- docs/mllib-frequent-pattern-mining.md | 4 +- docs/mllib-guide.md | 219 +---------- docs/mllib-isotonic-regression.md | 4 +- docs/mllib-linear-methods.md | 4 +- docs/mllib-migration-guides.md | 158 +------- docs/mllib-naive-bayes.md | 4 +- docs/mllib-optimization.md | 4 +- docs/mllib-pmml-model-export.md | 4 +- docs/mllib-statistics.md | 4 +- docs/programming-guide.md | 2 +- docs/streaming-programming-guide.md | 4 +- python/pyspark/ml/__init__.py | 4 +- python/pyspark/ml/tests.py | 2 +- python/pyspark/mllib/__init__.py | 5 +- 41 files changed, 814 insertions(+), 746 deletions(-) create mode 100644 docs/ml-migration-guides.md create mode 100644 docs/ml-pipeline.md create mode 100644 docs/ml-tuning.md diff --git a/docs/_data/menu-ml.yaml b/docs/_data/menu-ml.yaml index 3fd3ee2823f75..0c6b9b20a6e4b 100644 --- a/docs/_data/menu-ml.yaml +++ b/docs/_data/menu-ml.yaml @@ -1,5 +1,5 @@ -- text: "Overview: estimators, transformers and pipelines" - url: ml-guide.html +- text: Pipelines + url: ml-pipeline.html - text: Extracting, transforming and selecting features url: ml-features.html - text: Classification and Regression @@ -8,5 +8,7 @@ url: ml-clustering.html - text: Collaborative filtering url: ml-collaborative-filtering.html +- text: Model selection and tuning + url: ml-tuning.html - text: Advanced topics url: ml-advanced.html diff --git a/docs/_includes/nav-left-wrapper-ml.html b/docs/_includes/nav-left-wrapper-ml.html index e2d7eda027c6e..00ac6cc0dbc7d 100644 --- a/docs/_includes/nav-left-wrapper-ml.html +++ b/docs/_includes/nav-left-wrapper-ml.html @@ -1,8 +1,8 @@
    -

    spark.ml package

    +

    MLlib: Main Guide

    {% include nav-left.html nav=include.nav-ml %} -

    spark.mllib package

    +

    MLlib: RDD-based API Guide

    {% include nav-left.html nav=include.nav-mllib %}
    \ No newline at end of file diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 2d0c3fd71293d..d3bf082aa751a 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -74,7 +74,7 @@
  • Spark Streaming
  • DataFrames, Datasets and SQL
  • Structured Streaming
  • -
  • MLlib (Machine Learning)
  • +
  • MLlib (Machine Learning)
  • GraphX (Graph Processing)
  • SparkR (R on Spark)
  • diff --git a/docs/index.md b/docs/index.md index 7157afc411bc5..0cb8803783a0f 100644 --- a/docs/index.md +++ b/docs/index.md @@ -8,7 +8,7 @@ description: Apache Spark SPARK_VERSION_SHORT documentation homepage Apache Spark is a fast and general-purpose cluster computing system. It provides high-level APIs in Java, Scala, Python and R, and an optimized engine that supports general execution graphs. -It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](mllib-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). +It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](ml-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). # Downloading @@ -87,7 +87,7 @@ options for deployment: * Modules built on Spark: * [Spark Streaming](streaming-programming-guide.html): processing real-time data streams * [Spark SQL, Datasets, and DataFrames](sql-programming-guide.html): support for structured data and relational queries - * [MLlib](mllib-guide.html): built-in machine learning library + * [MLlib](ml-guide.html): built-in machine learning library * [GraphX](graphx-programming-guide.html): Spark's new API for graph processing **API Docs:** diff --git a/docs/ml-advanced.md b/docs/ml-advanced.md index 1c5f844b08e40..f5804fdeee5aa 100644 --- a/docs/ml-advanced.md +++ b/docs/ml-advanced.md @@ -1,7 +1,7 @@ --- layout: global -title: Advanced topics - spark.ml -displayTitle: Advanced topics - spark.ml +title: Advanced topics +displayTitle: Advanced topics --- * Table of contents diff --git a/docs/ml-ann.md b/docs/ml-ann.md index c2d9bd200f62f..7c460c4af6f41 100644 --- a/docs/ml-ann.md +++ b/docs/ml-ann.md @@ -1,7 +1,7 @@ --- layout: global -title: Multilayer perceptron classifier - spark.ml -displayTitle: Multilayer perceptron classifier - spark.ml +title: Multilayer perceptron classifier +displayTitle: Multilayer perceptron classifier --- > This section has been moved into the diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index 3d6106b532ff9..7c2437eacde3f 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -1,7 +1,7 @@ --- layout: global -title: Classification and regression - spark.ml -displayTitle: Classification and regression - spark.ml +title: Classification and regression +displayTitle: Classification and regression --- @@ -22,37 +22,14 @@ displayTitle: Classification and regression - spark.ml \newcommand{\zero}{\mathbf{0}} \]` +This page covers algorithms for Classification and Regression. It also includes sections +discussing specific classes of algorithms, such as linear methods, trees, and ensembles. + **Table of Contents** * This will become a table of contents (this text will be scraped). {:toc} -In `spark.ml`, we implement popular linear methods such as logistic -regression and linear least squares with $L_1$ or $L_2$ regularization. -Refer to [the linear methods in mllib](mllib-linear-methods.html) for -details about implementation and tuning. We also include a DataFrame API for [Elastic -net](http://en.wikipedia.org/wiki/Elastic_net_regularization), a hybrid -of $L_1$ and $L_2$ regularization proposed in [Zou et al, Regularization -and variable selection via the elastic -net](http://users.stat.umn.edu/~zouxx019/Papers/elasticnet.pdf). -Mathematically, it is defined as a convex combination of the $L_1$ and -the $L_2$ regularization terms: -`\[ -\alpha \left( \lambda \|\wv\|_1 \right) + (1-\alpha) \left( \frac{\lambda}{2}\|\wv\|_2^2 \right) , \alpha \in [0, 1], \lambda \geq 0 -\]` -By setting $\alpha$ properly, elastic net contains both $L_1$ and $L_2$ -regularization as special cases. For example, if a [linear -regression](https://en.wikipedia.org/wiki/Linear_regression) model is -trained with the elastic net parameter $\alpha$ set to $1$, it is -equivalent to a -[Lasso](http://en.wikipedia.org/wiki/Least_squares#Lasso_method) model. -On the other hand, if $\alpha$ is set to $0$, the trained model reduces -to a [ridge -regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. -We implement Pipelines API for both linear regression and logistic -regression with elastic net regularization. - - # Classification ## Logistic regression @@ -760,7 +737,34 @@ Refer to the [`IsotonicRegression` Python docs](api/python/pyspark.ml.html#pyspa
    +# Linear methods + +We implement popular linear methods such as logistic +regression and linear least squares with $L_1$ or $L_2$ regularization. +Refer to [the linear methods guide for the RDD-based API](mllib-linear-methods.html) for +details about implementation and tuning; this information is still relevant. +We also include a DataFrame API for [Elastic +net](http://en.wikipedia.org/wiki/Elastic_net_regularization), a hybrid +of $L_1$ and $L_2$ regularization proposed in [Zou et al, Regularization +and variable selection via the elastic +net](http://users.stat.umn.edu/~zouxx019/Papers/elasticnet.pdf). +Mathematically, it is defined as a convex combination of the $L_1$ and +the $L_2$ regularization terms: +`\[ +\alpha \left( \lambda \|\wv\|_1 \right) + (1-\alpha) \left( \frac{\lambda}{2}\|\wv\|_2^2 \right) , \alpha \in [0, 1], \lambda \geq 0 +\]` +By setting $\alpha$ properly, elastic net contains both $L_1$ and $L_2$ +regularization as special cases. For example, if a [linear +regression](https://en.wikipedia.org/wiki/Linear_regression) model is +trained with the elastic net parameter $\alpha$ set to $1$, it is +equivalent to a +[Lasso](http://en.wikipedia.org/wiki/Least_squares#Lasso_method) model. +On the other hand, if $\alpha$ is set to $0$, the trained model reduces +to a [ridge +regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. +We implement Pipelines API for both linear regression and logistic +regression with elastic net regularization. # Decision trees diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 8656eb4001f4b..8a0a61cb595e7 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -1,10 +1,12 @@ --- layout: global -title: Clustering - spark.ml -displayTitle: Clustering - spark.ml +title: Clustering +displayTitle: Clustering --- -In this section, we introduce the pipeline API for [clustering in mllib](mllib-clustering.html). +This page describes clustering algorithms in MLlib. +The [guide for clustering in the RDD-based API](mllib-clustering.html) also has relevant information +about these algorithms. **Table of Contents** diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md index 8bd75f3bcf7a7..1d02d6933cb48 100644 --- a/docs/ml-collaborative-filtering.md +++ b/docs/ml-collaborative-filtering.md @@ -1,7 +1,7 @@ --- layout: global -title: Collaborative Filtering - spark.ml -displayTitle: Collaborative Filtering - spark.ml +title: Collaborative Filtering +displayTitle: Collaborative Filtering --- * Table of contents diff --git a/docs/ml-decision-tree.md b/docs/ml-decision-tree.md index a721d55bc675b..5e1eeb95e4724 100644 --- a/docs/ml-decision-tree.md +++ b/docs/ml-decision-tree.md @@ -1,7 +1,7 @@ --- layout: global -title: Decision trees - spark.ml -displayTitle: Decision trees - spark.ml +title: Decision trees +displayTitle: Decision trees --- > This section has been moved into the diff --git a/docs/ml-ensembles.md b/docs/ml-ensembles.md index 303773e8038fc..97f1bdc803d01 100644 --- a/docs/ml-ensembles.md +++ b/docs/ml-ensembles.md @@ -1,7 +1,7 @@ --- layout: global -title: Tree ensemble methods - spark.ml -displayTitle: Tree ensemble methods - spark.ml +title: Tree ensemble methods +displayTitle: Tree ensemble methods --- > This section has been moved into the diff --git a/docs/ml-features.md b/docs/ml-features.md index 88fd291b4be50..e7d7ddfe28a87 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1,7 +1,7 @@ --- layout: global -title: Extracting, transforming and selecting features - spark.ml -displayTitle: Extracting, transforming and selecting features - spark.ml +title: Extracting, transforming and selecting features +displayTitle: Extracting, transforming and selecting features --- This section covers algorithms for working with features, roughly divided into these groups: diff --git a/docs/ml-guide.md b/docs/ml-guide.md index dae86d84804d0..5abec63b7ab45 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -1,323 +1,214 @@ --- layout: global -title: "Overview: estimators, transformers and pipelines - spark.ml" -displayTitle: "Overview: estimators, transformers and pipelines - spark.ml" +title: "MLlib: Main Guide" +displayTitle: "Machine Learning Library (MLlib) Guide" --- +MLlib is Spark's machine learning (ML) library. +Its goal is to make practical machine learning scalable and easy. +At a high level, it provides tools such as: -`\[ -\newcommand{\R}{\mathbb{R}} -\newcommand{\E}{\mathbb{E}} -\newcommand{\x}{\mathbf{x}} -\newcommand{\y}{\mathbf{y}} -\newcommand{\wv}{\mathbf{w}} -\newcommand{\av}{\mathbf{\alpha}} -\newcommand{\bv}{\mathbf{b}} -\newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} -\newcommand{\ind}{\mathbf{1}} -\newcommand{\0}{\mathbf{0}} -\newcommand{\unit}{\mathbf{e}} -\newcommand{\one}{\mathbf{1}} -\newcommand{\zero}{\mathbf{0}} -\]` +* ML Algorithms: common learning algorithms such as classification, regression, clustering, and collaborative filtering +* Featurization: feature extraction, transformation, dimensionality reduction, and selection +* Pipelines: tools for constructing, evaluating, and tuning ML Pipelines +* Persistence: saving and load algorithms, models, and Pipelines +* Utilities: linear algebra, statistics, data handling, etc. +# Announcement: DataFrame-based API is primary API -The `spark.ml` package aims to provide a uniform set of high-level APIs built on top of -[DataFrames](sql-programming-guide.html#dataframes) that help users create and tune practical -machine learning pipelines. -See the [algorithm guides](#algorithm-guides) section below for guides on sub-packages of -`spark.ml`, including feature transformers unique to the Pipelines API, ensembles, and more. +**The MLlib RDD-based API is now in maintenance mode.** -**Table of contents** +As of Spark 2.0, the [RDD](programming-guide.html#resilient-distributed-datasets-rdds)-based APIs in the `spark.mllib` package have entered maintenance mode. +The primary Machine Learning API for Spark is now the [DataFrame](sql-programming-guide.html)-based API in the `spark.ml` package. -* This will become a table of contents (this text will be scraped). -{:toc} +*What are the implications?* +* MLlib will still support the RDD-based API in `spark.mllib` with bug fixes. +* MLlib will not add new features to the RDD-based API. +* In the Spark 2.x releases, MLlib will add features to the DataFrames-based API to reach feature parity with the RDD-based API. +* After reaching feature parity (roughly estimated for Spark 2.2), the RDD-based API will be deprecated. +* The RDD-based API is expected to be removed in Spark 3.0. -# Main concepts in Pipelines +*Why is MLlib switching to the DataFrame-based API?* -Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple -algorithms into a single pipeline, or workflow. -This section covers the key concepts introduced by the Spark ML API, where the pipeline concept is -mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. +* DataFrames provide a more user-friendly API than RDDs. The many benefits of DataFrames include Spark Datasources, SQL/DataFrame queries, Tungsten and Catalyst optimizations, and uniform APIs across languages. +* The DataFrame-based API for MLlib provides a uniform API across ML algorithms and across multiple languages. +* DataFrames facilitate practical ML Pipelines, particularly feature transformations. See the [Pipelines guide](ml-pipeline.md) for details. -* **[`DataFrame`](ml-guide.html#dataframe)**: Spark ML uses `DataFrame` from Spark SQL as an ML - dataset, which can hold a variety of data types. - E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. +# Dependencies -* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. -E.g., an ML model is a `Transformer` which transforms a `DataFrame` with features into a `DataFrame` with predictions. +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on +[netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing. +If native libraries[^1] are not available at runtime, you will see a warning message and a pure JVM +implementation will be used instead. -* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. -E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. +Due to licensing issues with runtime proprietary binaries, we do not include `netlib-java`'s native +proxies by default. +To configure `netlib-java` / Breeze to use system optimised binaries, include +`com.github.fommil.netlib:all:1.1.2` (or build Spark with `-Pnetlib-lgpl`) as a dependency of your +project and read the [netlib-java](https://github.com/fommil/netlib-java) documentation for your +platform's additional installation instructions. -* **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. +To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. -* **[`Parameter`](ml-guide.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. +[^1]: To learn more about the benefits and background of system optimised natives, you may wish to + watch Sam Halliday's ScalaX talk on [High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/). -## DataFrame +# Migration guide -Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. -Spark ML adopts the `DataFrame` from Spark SQL in order to support a variety of data types. +MLlib is under active development. +The APIs marked `Experimental`/`DeveloperApi` may change in future releases, +and the migration guide below will explain all changes between releases. -`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. -In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. +## From 1.6 to 2.0 -A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. +### Breaking changes -Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." +There were several breaking changes in Spark 2.0, which are outlined below. -## Pipeline components +**Linear algebra classes for DataFrame-based APIs** -### Transformers +Spark's linear algebra dependencies were moved to a new project, `mllib-local` +(see [SPARK-13944](https://issues.apache.org/jira/browse/SPARK-13944)). +As part of this change, the linear algebra classes were copied to a new package, `spark.ml.linalg`. +The DataFrame-based APIs in `spark.ml` now depend on the `spark.ml.linalg` classes, +leading to a few breaking changes, predominantly in various model classes +(see [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810) for a full list). -A `Transformer` is an abstraction that includes feature transformers and learned models. -Technically, a `Transformer` implements a method `transform()`, which converts one `DataFrame` into -another, generally by appending one or more columns. -For example: +**Note:** the RDD-based APIs in `spark.mllib` continue to depend on the previous package `spark.mllib.linalg`. -* A feature transformer might take a `DataFrame`, read a column (e.g., text), map it into a new - column (e.g., feature vectors), and output a new `DataFrame` with the mapped column appended. -* A learning model might take a `DataFrame`, read the column containing feature vectors, predict the - label for each feature vector, and output a new `DataFrame` with predicted labels appended as a - column. +_Converting vectors and matrices_ -### Estimators +While most pipeline components support backward compatibility for loading, +some existing `DataFrames` and pipelines in Spark versions prior to 2.0, that contain vector or matrix +columns, may need to be migrated to the new `spark.ml` vector and matrix types. +Utilities for converting `DataFrame` columns from `spark.mllib.linalg` to `spark.ml.linalg` types +(and vice versa) can be found in `spark.mllib.util.MLUtils`. -An `Estimator` abstracts the concept of a learning algorithm or any algorithm that fits or trains on -data. -Technically, an `Estimator` implements a method `fit()`, which accepts a `DataFrame` and produces a -`Model`, which is a `Transformer`. -For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling -`fit()` trains a `LogisticRegressionModel`, which is a `Model` and hence a `Transformer`. - -### Properties of pipeline components - -`Transformer.transform()`s and `Estimator.fit()`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. - -Each instance of a `Transformer` or `Estimator` has a unique ID, which is useful in specifying parameters (discussed below). - -## Pipeline - -In machine learning, it is common to run a sequence of algorithms to process and learn from data. -E.g., a simple text document processing workflow might include several stages: - -* Split each document's text into words. -* Convert each document's words into a numerical feature vector. -* Learn a prediction model using the feature vectors and labels. - -Spark ML represents such a workflow as a `Pipeline`, which consists of a sequence of -`PipelineStage`s (`Transformer`s and `Estimator`s) to be run in a specific order. -We will use this simple workflow as a running example in this section. - -### How it works - -A `Pipeline` is specified as a sequence of stages, and each stage is either a `Transformer` or an `Estimator`. -These stages are run in order, and the input `DataFrame` is transformed as it passes through each stage. -For `Transformer` stages, the `transform()` method is called on the `DataFrame`. -For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the `DataFrame`. - -We illustrate this for the simple text document workflow. The figure below is for the *training time* usage of a `Pipeline`. - -

    - Spark ML Pipeline Example -

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

    - Spark ML PipelineModel Example -

    - -In the figure above, the `PipelineModel` has the same number of stages as the original `Pipeline`, but all `Estimator`s in the original `Pipeline` have become `Transformer`s. -When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed -through the fitted pipeline in order. -Each stage's `transform()` method updates the dataset and passes it to the next stage. - -`Pipeline`s and `PipelineModel`s help to ensure that training and test data go through identical feature processing steps. - -### Details - -*DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. - -*Runtime checking*: Since `Pipeline`s can operate on `DataFrame`s with varied types, they cannot use -compile-time type checking. -`Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. -This type checking is done using the `DataFrame` *schema*, a description of the data types of columns in the `DataFrame`. - -*Unique Pipeline stages*: A `Pipeline`'s stages should be unique instances. E.g., the same instance -`myHashingTF` should not be inserted into the `Pipeline` twice since `Pipeline` stages must have -unique IDs. However, different instances `myHashingTF1` and `myHashingTF2` (both of type `HashingTF`) -can be put into the same `Pipeline` since different instances will be created with different IDs. - -## Parameters - -Spark ML `Estimator`s and `Transformer`s use a uniform API for specifying parameters. - -A `Param` is a named parameter with self-contained documentation. -A `ParamMap` is a set of (parameter, value) pairs. - -There are two main ways to pass parameters to an algorithm: - -1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could - call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. - This API resembles the API used in `spark.mllib` package. -2. Pass a `ParamMap` to `fit()` or `transform()`. Any parameters in the `ParamMap` will override parameters previously specified via setter methods. - -Parameters belong to specific instances of `Estimator`s and `Transformer`s. -For example, if we have two `LogisticRegression` instances `lr1` and `lr2`, then we can build a `ParamMap` with both `maxIter` parameters specified: `ParamMap(lr1.maxIter -> 10, lr2.maxIter -> 20)`. -This is useful if there are two algorithms with the `maxIter` parameter in a `Pipeline`. - -## Saving and Loading Pipelines - -Often times it is worth it to save a model or a pipeline to disk for later use. In Spark 1.6, a model import/export functionality was added to the Pipeline API. Most basic transformers are supported as well as some of the more basic ML models. Please refer to the algorithm's API documentation to see if saving and loading is supported. - -# Code examples - -This section gives code examples illustrating the functionality discussed above. -For more info, please refer to the API documentation -([Scala](api/scala/index.html#org.apache.spark.ml.package), -[Java](api/java/org/apache/spark/ml/package-summary.html), -and [Python](api/python/pyspark.ml.html)). -Some Spark ML algorithms are wrappers for `spark.mllib` algorithms, and the -[MLlib programming guide](mllib-guide.html) has details on specific algorithms. - -## Example: Estimator, Transformer, and Param - -This example covers the concepts of `Estimator`, `Transformer`, and `Param`. +There are also utility methods available for converting single instances of +vectors and matrices. Use the `asML` method on a `mllib.linalg.Vector` / `mllib.linalg.Matrix` +for converting to `ml.linalg` types, and +`mllib.linalg.Vectors.fromML` / `mllib.linalg.Matrices.fromML` +for converting to `mllib.linalg` types.
    +
    -
    -{% include_example scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala %} -
    +{% highlight scala %} +import org.apache.spark.mllib.util.MLUtils -
    -{% include_example java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java %} -
    +// convert DataFrame columns +val convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) +val convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) +// convert a single vector or matrix +val mlVec: org.apache.spark.ml.linalg.Vector = mllibVec.asML +val mlMat: org.apache.spark.ml.linalg.Matrix = mllibMat.asML +{% endhighlight %} -
    -{% include_example python/ml/estimator_transformer_param_example.py %} -
    - -
    - -## Example: Pipeline - -This example follows the simple text document `Pipeline` illustrated in the figures above. - -
    - -
    -{% include_example scala/org/apache/spark/examples/ml/PipelineExample.scala %} -
    - -
    -{% include_example java/org/apache/spark/examples/ml/JavaPipelineExample.java %} -
    - -
    -{% include_example python/ml/pipeline_example.py %} -
    - -
    - -## Example: model selection via cross-validation - -An important task in ML is *model selection*, or using data to find the best model or parameters for a given task. This is also called *tuning*. -`Pipeline`s facilitate model selection by making it easy to tune an entire `Pipeline` at once, rather than tuning each element in the `Pipeline` separately. - -Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) class, which takes an `Estimator`, a set of `ParamMap`s, and an [`Evaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.Evaluator). -`CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets; e.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. -`CrossValidator` iterates through the set of `ParamMap`s. For each `ParamMap`, it trains the given `Estimator` and evaluates it using the given `Evaluator`. - -The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) -for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) -for binary data, or a [`MulticlassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) -for multiclass problems. The default metric used to choose the best `ParamMap` can be overridden by the `setMetricName` -method in each of these evaluators. - -The `ParamMap` which produces the best evaluation metric (averaged over the `$k$` folds) is selected as the best model. -`CrossValidator` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. - -The following example demonstrates using `CrossValidator` to select from a grid of parameters. -To help construct the parameter grid, we use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridBuilder) utility. - -Note that cross-validation over a grid of parameters is expensive. -E.g., in the example below, the parameter grid has 3 values for `hashingTF.numFeatures` and 2 values for `lr.regParam`, and `CrossValidator` uses 2 folds. This multiplies out to `$(3 \times 2) \times 2 = 12$` different models being trained. -In realistic settings, it can be common to try many more parameters and use more folds (`$k=3$` and `$k=10$` are common). -In other words, using `CrossValidator` can be very expensive. -However, it is also a well-established method for choosing parameters which is more statistically sound than heuristic hand-tuning. - -
    - -
    -{% include_example scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala %} -
    - -
    -{% include_example java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java %} -
    - -
    - -{% include_example python/ml/cross_validator.py %} -
    - -
    - -## Example: model selection via train validation split -In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. -`TrainValidationSplit` only evaluates each combination of parameters once, as opposed to k times in - the case of `CrossValidator`. It is therefore less expensive, - but will not produce as reliable results when the training dataset is not sufficiently large. - -`TrainValidationSplit` takes an `Estimator`, a set of `ParamMap`s provided in the `estimatorParamMaps` parameter, -and an `Evaluator`. -It begins by splitting the dataset into two parts using the `trainRatio` parameter -which are used as separate training and test datasets. For example with `$trainRatio=0.75$` (default), -`TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. -Similar to `CrossValidator`, `TrainValidationSplit` also iterates through the set of `ParamMap`s. -For each combination of parameters, it trains the given `Estimator` and evaluates it using the given `Evaluator`. -The `ParamMap` which produces the best evaluation metric is selected as the best option. -`TrainValidationSplit` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. - -
    - -
    -{% include_example scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala %} +Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) for further detail.
    -{% include_example java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java %} -
    -
    -{% include_example python/ml/train_validation_split.py %} -
    +{% highlight java %} +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.sql.Dataset; + +// convert DataFrame columns +Dataset convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF); +Dataset convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF); +// convert a single vector or matrix +org.apache.spark.ml.linalg.Vector mlVec = mllibVec.asML(); +org.apache.spark.ml.linalg.Matrix mlMat = mllibMat.asML(); +{% endhighlight %} + +Refer to the [`MLUtils` Java docs](api/java/org/apache/spark/mllib/util/MLUtils.html) for further detail. +
    + +
    + +{% highlight python %} +from pyspark.mllib.util import MLUtils + +# convert DataFrame columns +convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) +convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) +# convert a single vector or matrix +mlVec = mllibVec.asML() +mlMat = mllibMat.asML() +{% endhighlight %} + +Refer to the [`MLUtils` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.util.MLUtils) for further detail. +
    +
    + +**Deprecated methods removed** + +Several deprecated methods were removed in the `spark.mllib` and `spark.ml` packages: + +* `setScoreCol` in `ml.evaluation.BinaryClassificationEvaluator` +* `weights` in `LinearRegression` and `LogisticRegression` in `spark.ml` +* `setMaxNumIterations` in `mllib.optimization.LBFGS` (marked as `DeveloperApi`) +* `treeReduce` and `treeAggregate` in `mllib.rdd.RDDFunctions` (these functions are available on `RDD`s directly, and were marked as `DeveloperApi`) +* `defaultStategy` in `mllib.tree.configuration.Strategy` +* `build` in `mllib.tree.Node` +* libsvm loaders for multiclass and load/save labeledData methods in `mllib.util.MLUtils` + +A full list of breaking changes can be found at [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810). + +### Deprecations and changes of behavior + +**Deprecations** + +Deprecations in the `spark.mllib` and `spark.ml` packages include: + +* [SPARK-14984](https://issues.apache.org/jira/browse/SPARK-14984): + In `spark.ml.regression.LinearRegressionSummary`, the `model` field has been deprecated. +* [SPARK-13784](https://issues.apache.org/jira/browse/SPARK-13784): + In `spark.ml.regression.RandomForestRegressionModel` and `spark.ml.classification.RandomForestClassificationModel`, + the `numTrees` parameter has been deprecated in favor of `getNumTrees` method. +* [SPARK-13761](https://issues.apache.org/jira/browse/SPARK-13761): + In `spark.ml.param.Params`, the `validateParams` method has been deprecated. + We move all functionality in overridden methods to the corresponding `transformSchema`. +* [SPARK-14829](https://issues.apache.org/jira/browse/SPARK-14829): + In `spark.mllib` package, `LinearRegressionWithSGD`, `LassoWithSGD`, `RidgeRegressionWithSGD` and `LogisticRegressionWithSGD` have been deprecated. + We encourage users to use `spark.ml.regression.LinearRegresson` and `spark.ml.classification.LogisticRegresson`. +* [SPARK-14900](https://issues.apache.org/jira/browse/SPARK-14900): + In `spark.mllib.evaluation.MulticlassMetrics`, the parameters `precision`, `recall` and `fMeasure` have been deprecated in favor of `accuracy`. +* [SPARK-15644](https://issues.apache.org/jira/browse/SPARK-15644): + In `spark.ml.util.MLReader` and `spark.ml.util.MLWriter`, the `context` method has been deprecated in favor of `session`. +* In `spark.ml.feature.ChiSqSelectorModel`, the `setLabelCol` method has been deprecated since it was not used by `ChiSqSelectorModel`. + +**Changes of behavior** + +Changes of behavior in the `spark.mllib` and `spark.ml` packages include: + +* [SPARK-7780](https://issues.apache.org/jira/browse/SPARK-7780): + `spark.mllib.classification.LogisticRegressionWithLBFGS` directly calls `spark.ml.classification.LogisticRegresson` for binary classification now. + This will introduce the following behavior changes for `spark.mllib.classification.LogisticRegressionWithLBFGS`: + * The intercept will not be regularized when training binary classification model with L1/L2 Updater. + * If users set without regularization, training with or without feature scaling will return the same solution by the same convergence rate. +* [SPARK-13429](https://issues.apache.org/jira/browse/SPARK-13429): + In order to provide better and consistent result with `spark.ml.classification.LogisticRegresson`, + the default value of `spark.mllib.classification.LogisticRegressionWithLBFGS`: `convergenceTol` has been changed from 1E-4 to 1E-6. +* [SPARK-12363](https://issues.apache.org/jira/browse/SPARK-12363): + Fix a bug of `PowerIterationClustering` which will likely change its result. +* [SPARK-13048](https://issues.apache.org/jira/browse/SPARK-13048): + `LDA` using the `EM` optimizer will keep the last checkpoint by default, if checkpointing is being used. +* [SPARK-12153](https://issues.apache.org/jira/browse/SPARK-12153): + `Word2Vec` now respects sentence boundaries. Previously, it did not handle them correctly. +* [SPARK-10574](https://issues.apache.org/jira/browse/SPARK-10574): + `HashingTF` uses `MurmurHash3` as default hash algorithm in both `spark.ml` and `spark.mllib`. +* [SPARK-14768](https://issues.apache.org/jira/browse/SPARK-14768): + The `expectedType` argument for PySpark `Param` was removed. +* [SPARK-14931](https://issues.apache.org/jira/browse/SPARK-14931): + Some default `Param` values, which were mismatched between pipelines in Scala and Python, have been changed. +* [SPARK-13600](https://issues.apache.org/jira/browse/SPARK-13600): + `QuantileDiscretizer` now uses `spark.sql.DataFrameStatFunctions.approxQuantile` to find splits (previously used custom sampling logic). + The output buckets will differ for same input data and params. + +## Previous Spark versions + +Earlier migration guides are archived [on this page](ml-migration-guides.html). -
    +--- diff --git a/docs/ml-linear-methods.md b/docs/ml-linear-methods.md index a8754835cab95..eb39173505aed 100644 --- a/docs/ml-linear-methods.md +++ b/docs/ml-linear-methods.md @@ -1,7 +1,7 @@ --- layout: global -title: Linear methods - spark.ml -displayTitle: Linear methods - spark.ml +title: Linear methods +displayTitle: Linear methods --- > This section has been moved into the diff --git a/docs/ml-migration-guides.md b/docs/ml-migration-guides.md new file mode 100644 index 0000000000000..82bf9d7760fb4 --- /dev/null +++ b/docs/ml-migration-guides.md @@ -0,0 +1,159 @@ +--- +layout: global +title: Old Migration Guides - MLlib +displayTitle: Old Migration Guides - MLlib +description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +--- + +The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). + +## From 1.5 to 1.6 + +There are no breaking API changes in the `spark.mllib` or `spark.ml` packages, but there are +deprecations and changes of behavior. + +Deprecations: + +* [SPARK-11358](https://issues.apache.org/jira/browse/SPARK-11358): + In `spark.mllib.clustering.KMeans`, the `runs` parameter has been deprecated. +* [SPARK-10592](https://issues.apache.org/jira/browse/SPARK-10592): + In `spark.ml.classification.LogisticRegressionModel` and + `spark.ml.regression.LinearRegressionModel`, the `weights` field has been deprecated in favor of + the new name `coefficients`. This helps disambiguate from instance (row) "weights" given to + algorithms. + +Changes of behavior: + +* [SPARK-7770](https://issues.apache.org/jira/browse/SPARK-7770): + `spark.mllib.tree.GradientBoostedTrees`: `validationTol` has changed semantics in 1.6. + Previously, it was a threshold for absolute change in error. Now, it resembles the behavior of + `GradientDescent`'s `convergenceTol`: For large errors, it uses relative error (relative to the + previous error); for small errors (`< 0.01`), it uses absolute error. +* [SPARK-11069](https://issues.apache.org/jira/browse/SPARK-11069): + `spark.ml.feature.RegexTokenizer`: Previously, it did not convert strings to lowercase before + tokenizing. Now, it converts to lowercase by default, with an option not to. This matches the + behavior of the simpler `Tokenizer` transformer. + +## From 1.4 to 1.5 + +In the `spark.mllib` package, there are no breaking API changes but several behavior changes: + +* [SPARK-9005](https://issues.apache.org/jira/browse/SPARK-9005): + `RegressionMetrics.explainedVariance` returns the average regression sum of squares. +* [SPARK-8600](https://issues.apache.org/jira/browse/SPARK-8600): `NaiveBayesModel.labels` become + sorted. +* [SPARK-3382](https://issues.apache.org/jira/browse/SPARK-3382): `GradientDescent` has a default + convergence tolerance `1e-3`, and hence iterations might end earlier than 1.4. + +In the `spark.ml` package, there exists one breaking API change and one behavior change: + +* [SPARK-9268](https://issues.apache.org/jira/browse/SPARK-9268): Java's varargs support is removed + from `Params.setDefault` due to a + [Scala compiler bug](https://issues.scala-lang.org/browse/SI-9013). +* [SPARK-10097](https://issues.apache.org/jira/browse/SPARK-10097): `Evaluator.isLargerBetter` is + added to indicate metric ordering. Metrics like RMSE no longer flip signs as in 1.4. + +## From 1.3 to 1.4 + +In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: + +* Gradient-Boosted Trees + * *(Breaking change)* The signature of the [`Loss.gradient`](api/scala/index.html#org.apache.spark.mllib.tree.loss.Loss) method was changed. This is only an issues for users who wrote their own losses for GBTs. + * *(Breaking change)* The `apply` and `copy` methods for the case class [`BoostingStrategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.BoostingStrategy) have been changed because of a modification to the case class fields. This could be an issue for users who use `BoostingStrategy` to set GBT parameters. +* *(Breaking change)* The return value of [`LDA.run`](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) has changed. It now returns an abstract class `LDAModel` instead of the concrete class `DistributedLDAModel`. The object of type `LDAModel` can still be cast to the appropriate concrete type, which depends on the optimization algorithm. + +In the `spark.ml` package, several major API changes occurred, including: + +* `Param` and other APIs for specifying parameters +* `uid` unique IDs for Pipeline components +* Reorganization of certain classes + +Since the `spark.ml` API was an alpha component in Spark 1.3, we do not list all changes here. +However, since 1.4 `spark.ml` is no longer an alpha component, we will provide details on any API +changes for future releases. + +## From 1.2 to 1.3 + +In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. + +* *(Breaking change)* In [`ALS`](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS), the extraneous method `solveLeastSquares` has been removed. The `DeveloperApi` method `analyzeBlocks` was also removed. +* *(Breaking change)* [`StandardScalerModel`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScalerModel) remains an Alpha component. In it, the `variance` method has been replaced with the `std` method. To compute the column variance values returned by the original `variance` method, simply square the standard deviation values returned by `std`. +* *(Breaking change)* [`StreamingLinearRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD) remains an Experimental component. In it, there were two changes: + * The constructor taking arguments was removed in favor of a builder pattern using the default constructor plus parameter setter methods. + * Variable `model` is no longer public. +* *(Breaking change)* [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) remains an Experimental component. In it and its associated classes, there were several changes: + * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) + * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. +* `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. +* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. + So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. + +In the `spark.ml` package, the main API changes are from Spark SQL. We list the most important changes here: + +* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in `spark.ml` which used to use SchemaRDD now use DataFrame. +* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`. +* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details. + +Other changes were in `LogisticRegression`: + +* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). +* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. + +## From 1.1 to 1.2 + +The only API changes in MLlib v1.2 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.2: + +1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number +of classes. In MLlib v1.1, this argument was called `numClasses` in Python and +`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. +This `numClasses` parameter is specified either via +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Breaking change)* The API for +[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. +This should generally not affect user code, unless the user manually constructs decision trees +(instead of using the `trainClassifier` or `trainRegressor` methods). +The tree `Node` now includes more information, including the probability of the predicted label +(for classification). + +3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. + +Examples in the Spark distribution and examples in the +[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. + +## From 1.0 to 1.1 + +The only API changes in MLlib v1.1 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.1: + +1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match +the implementations of trees in +[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) +and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). +In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. +In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. +This depth is specified by the `maxDepth` parameter in +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` +methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +rather than using the old parameter class `Strategy`. These new training methods explicitly +separate classification and regression, and they replace specialized parameter types with +simple `String` types. + +Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the +[Decision Trees Guide](mllib-decision-tree.html#examples). + +## From 0.9 to 1.0 + +In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few +breaking changes. If your data is sparse, please store it in a sparse format instead of dense to +take advantage of sparsity in both storage and computation. Details are described below. + diff --git a/docs/ml-pipeline.md b/docs/ml-pipeline.md new file mode 100644 index 0000000000000..adb057ba7e250 --- /dev/null +++ b/docs/ml-pipeline.md @@ -0,0 +1,245 @@ +--- +layout: global +title: ML Pipelines +displayTitle: ML Pipelines +--- + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +In this section, we introduce the concept of ***ML Pipelines***. +ML Pipelines provide a uniform set of high-level APIs built on top of +[DataFrames](sql-programming-guide.html) that help users create and tune practical +machine learning pipelines. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Main concepts in Pipelines + +MLlib standardizes APIs for machine learning algorithms to make it easier to combine multiple +algorithms into a single pipeline, or workflow. +This section covers the key concepts introduced by the Pipelines API, where the pipeline concept is +mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. + +* **[`DataFrame`](ml-guide.html#dataframe)**: This ML API uses `DataFrame` from Spark SQL as an ML + dataset, which can hold a variety of data types. + E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. + +* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. +E.g., an ML model is a `Transformer` which transforms a `DataFrame` with features into a `DataFrame` with predictions. + +* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. +E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. + +* **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. + +* **[`Parameter`](ml-guide.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. + +## DataFrame + +Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. +This API adopts the `DataFrame` from Spark SQL in order to support a variety of data types. + +`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. +In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. + +A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. + +Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." + +## Pipeline components + +### Transformers + +A `Transformer` is an abstraction that includes feature transformers and learned models. +Technically, a `Transformer` implements a method `transform()`, which converts one `DataFrame` into +another, generally by appending one or more columns. +For example: + +* A feature transformer might take a `DataFrame`, read a column (e.g., text), map it into a new + column (e.g., feature vectors), and output a new `DataFrame` with the mapped column appended. +* A learning model might take a `DataFrame`, read the column containing feature vectors, predict the + label for each feature vector, and output a new `DataFrame` with predicted labels appended as a + column. + +### Estimators + +An `Estimator` abstracts the concept of a learning algorithm or any algorithm that fits or trains on +data. +Technically, an `Estimator` implements a method `fit()`, which accepts a `DataFrame` and produces a +`Model`, which is a `Transformer`. +For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling +`fit()` trains a `LogisticRegressionModel`, which is a `Model` and hence a `Transformer`. + +### Properties of pipeline components + +`Transformer.transform()`s and `Estimator.fit()`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. + +Each instance of a `Transformer` or `Estimator` has a unique ID, which is useful in specifying parameters (discussed below). + +## Pipeline + +In machine learning, it is common to run a sequence of algorithms to process and learn from data. +E.g., a simple text document processing workflow might include several stages: + +* Split each document's text into words. +* Convert each document's words into a numerical feature vector. +* Learn a prediction model using the feature vectors and labels. + +MLlib represents such a workflow as a `Pipeline`, which consists of a sequence of +`PipelineStage`s (`Transformer`s and `Estimator`s) to be run in a specific order. +We will use this simple workflow as a running example in this section. + +### How it works + +A `Pipeline` is specified as a sequence of stages, and each stage is either a `Transformer` or an `Estimator`. +These stages are run in order, and the input `DataFrame` is transformed as it passes through each stage. +For `Transformer` stages, the `transform()` method is called on the `DataFrame`. +For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the `DataFrame`. + +We illustrate this for the simple text document workflow. The figure below is for the *training time* usage of a `Pipeline`. + +

    + ML Pipeline Example +

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

    + ML PipelineModel Example +

    + +In the figure above, the `PipelineModel` has the same number of stages as the original `Pipeline`, but all `Estimator`s in the original `Pipeline` have become `Transformer`s. +When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed +through the fitted pipeline in order. +Each stage's `transform()` method updates the dataset and passes it to the next stage. + +`Pipeline`s and `PipelineModel`s help to ensure that training and test data go through identical feature processing steps. + +### Details + +*DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. + +*Runtime checking*: Since `Pipeline`s can operate on `DataFrame`s with varied types, they cannot use +compile-time type checking. +`Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. +This type checking is done using the `DataFrame` *schema*, a description of the data types of columns in the `DataFrame`. + +*Unique Pipeline stages*: A `Pipeline`'s stages should be unique instances. E.g., the same instance +`myHashingTF` should not be inserted into the `Pipeline` twice since `Pipeline` stages must have +unique IDs. However, different instances `myHashingTF1` and `myHashingTF2` (both of type `HashingTF`) +can be put into the same `Pipeline` since different instances will be created with different IDs. + +## Parameters + +MLlib `Estimator`s and `Transformer`s use a uniform API for specifying parameters. + +A `Param` is a named parameter with self-contained documentation. +A `ParamMap` is a set of (parameter, value) pairs. + +There are two main ways to pass parameters to an algorithm: + +1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could + call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. + This API resembles the API used in `spark.mllib` package. +2. Pass a `ParamMap` to `fit()` or `transform()`. Any parameters in the `ParamMap` will override parameters previously specified via setter methods. + +Parameters belong to specific instances of `Estimator`s and `Transformer`s. +For example, if we have two `LogisticRegression` instances `lr1` and `lr2`, then we can build a `ParamMap` with both `maxIter` parameters specified: `ParamMap(lr1.maxIter -> 10, lr2.maxIter -> 20)`. +This is useful if there are two algorithms with the `maxIter` parameter in a `Pipeline`. + +## Saving and Loading Pipelines + +Often times it is worth it to save a model or a pipeline to disk for later use. In Spark 1.6, a model import/export functionality was added to the Pipeline API. Most basic transformers are supported as well as some of the more basic ML models. Please refer to the algorithm's API documentation to see if saving and loading is supported. + +# Code examples + +This section gives code examples illustrating the functionality discussed above. +For more info, please refer to the API documentation +([Scala](api/scala/index.html#org.apache.spark.ml.package), +[Java](api/java/org/apache/spark/ml/package-summary.html), +and [Python](api/python/pyspark.ml.html)). + +## Example: Estimator, Transformer, and Param + +This example covers the concepts of `Estimator`, `Transformer`, and `Param`. + +
    + +
    +{% include_example scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala %} +
    + +
    +{% include_example java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java %} +
    + +
    +{% include_example python/ml/estimator_transformer_param_example.py %} +
    + +
    + +## Example: Pipeline + +This example follows the simple text document `Pipeline` illustrated in the figures above. + +
    + +
    +{% include_example scala/org/apache/spark/examples/ml/PipelineExample.scala %} +
    + +
    +{% include_example java/org/apache/spark/examples/ml/JavaPipelineExample.java %} +
    + +
    +{% include_example python/ml/pipeline_example.py %} +
    + +
    + +## Model selection (hyperparameter tuning) + +A big benefit of using ML Pipelines is hyperparameter optimization. See the [ML Tuning Guide](ml-tuning.html) for more information on automatic model selection. diff --git a/docs/ml-survival-regression.md b/docs/ml-survival-regression.md index 856ceb2f4e7f6..efa3c21c7ca1b 100644 --- a/docs/ml-survival-regression.md +++ b/docs/ml-survival-regression.md @@ -1,7 +1,7 @@ --- layout: global -title: Survival Regression - spark.ml -displayTitle: Survival Regression - spark.ml +title: Survival Regression +displayTitle: Survival Regression --- > This section has been moved into the diff --git a/docs/ml-tuning.md b/docs/ml-tuning.md new file mode 100644 index 0000000000000..2ca90c7092fd3 --- /dev/null +++ b/docs/ml-tuning.md @@ -0,0 +1,121 @@ +--- +layout: global +title: "ML Tuning" +displayTitle: "ML Tuning: model selection and hyperparameter tuning" +--- + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +This section describes how to use MLlib's tooling for tuning ML algorithms and Pipelines. +Built-in Cross-Validation and other tooling allow users to optimize hyperparameters in algorithms and Pipelines. + +**Table of contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Model selection (a.k.a. hyperparameter tuning) + +An important task in ML is *model selection*, or using data to find the best model or parameters for a given task. This is also called *tuning*. +Tuning may be done for individual `Estimator`s such as `LogisticRegression`, or for entire `Pipeline`s which include multiple algorithms, featurization, and other steps. Users can tune an entire `Pipeline` at once, rather than tuning each element in the `Pipeline` separately. + +MLlib supports model selection using tools such as [`CrossValidator`](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) and [`TrainValidationSplit`](api/scala/index.html#org.apache.spark.ml.tuning.TrainValidationSplit). +These tools require the following items: + +* [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator): algorithm or `Pipeline` to tune +* Set of `ParamMap`s: parameters to choose from, sometimes called a "parameter grid" to search over +* [`Evaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.Evaluator): metric to measure how well a fitted `Model` does on held-out test data + +At a high level, these model selection tools work as follows: + +* They split the input data into separate training and test datasets. +* For each (training, test) pair, they iterate through the set of `ParamMap`s: + * For each `ParamMap`, they fit the `Estimator` using those parameters, get the fitted `Model`, and evaluate the `Model`'s performance using the `Evaluator`. +* They select the `Model` produced by the best-performing set of parameters. + +The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) +for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) +for binary data, or a [`MulticlassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) +for multiclass problems. The default metric used to choose the best `ParamMap` can be overridden by the `setMetricName` +method in each of these evaluators. + +To help construct the parameter grid, users can use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridBuilder) utility. + +# Cross-Validation + +`CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets. E.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. To evaluate a particular `ParamMap`, `CrossValidator` computes the average evaluation metric for the 3 `Model`s produced by fitting the `Estimator` on the 3 different (training, test) dataset pairs. + +After identifying the best `ParamMap`, `CrossValidator` finally re-fits the `Estimator` using the best `ParamMap` and the entire dataset. + +## Example: model selection via cross-validation + +The following example demonstrates using `CrossValidator` to select from a grid of parameters. + +Note that cross-validation over a grid of parameters is expensive. +E.g., in the example below, the parameter grid has 3 values for `hashingTF.numFeatures` and 2 values for `lr.regParam`, and `CrossValidator` uses 2 folds. This multiplies out to `$(3 \times 2) \times 2 = 12$` different models being trained. +In realistic settings, it can be common to try many more parameters and use more folds (`$k=3$` and `$k=10$` are common). +In other words, using `CrossValidator` can be very expensive. +However, it is also a well-established method for choosing parameters which is more statistically sound than heuristic hand-tuning. + +
    + +
    +{% include_example scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala %} +
    + +
    +{% include_example java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java %} +
    + +
    + +{% include_example python/ml/cross_validator.py %} +
    + +
    + +# Train-Validation Split + +In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. +`TrainValidationSplit` only evaluates each combination of parameters once, as opposed to k times in + the case of `CrossValidator`. It is therefore less expensive, + but will not produce as reliable results when the training dataset is not sufficiently large. + +Unlike `CrossValidator`, `TrainValidationSplit` creates a single (training, test) dataset pair. +It splits the dataset into these two parts using the `trainRatio` parameter. For example with `$trainRatio=0.75$`, +`TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. + +Like `CrossValidator`, `TrainValidationSplit` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. + +## Example: model selection via train validation split + +
    + +
    +{% include_example scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala %} +
    + +
    +{% include_example java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java %} +
    + +
    +{% include_example python/ml/train_validation_split.py %} +
    + +
    diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index aaf8bd465c9ab..a7b90de09369c 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -1,7 +1,7 @@ --- layout: global -title: Classification and Regression - spark.mllib -displayTitle: Classification and Regression - spark.mllib +title: Classification and Regression - RDD-based API +displayTitle: Classification and Regression - RDD-based API --- The `spark.mllib` package supports various methods for diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 073927c30bc63..d5f6ae379a85e 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -1,7 +1,7 @@ --- layout: global -title: Clustering - spark.mllib -displayTitle: Clustering - spark.mllib +title: Clustering - RDD-based API +displayTitle: Clustering - RDD-based API --- [Clustering](https://en.wikipedia.org/wiki/Cluster_analysis) is an unsupervised learning problem whereby we aim to group subsets diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 5c33292aaf086..0f891a09a6e61 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -1,7 +1,7 @@ --- layout: global -title: Collaborative Filtering - spark.mllib -displayTitle: Collaborative Filtering - spark.mllib +title: Collaborative Filtering - RDD-based API +displayTitle: Collaborative Filtering - RDD-based API --- * Table of contents diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index ef56aebbc3608..7dd3c97a83e4d 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -1,7 +1,7 @@ --- layout: global -title: Data Types - MLlib -displayTitle: Data Types - MLlib +title: Data Types - RDD-based API +displayTitle: Data Types - RDD-based API --- * Table of contents diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 11f5de1fc95ee..0e753b8dd04a2 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -1,7 +1,7 @@ --- layout: global -title: Decision Trees - spark.mllib -displayTitle: Decision Trees - spark.mllib +title: Decision Trees - RDD-based API +displayTitle: Decision Trees - RDD-based API --- * Table of contents diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index cceddce9f79a6..539cbc1b3163a 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -1,7 +1,7 @@ --- layout: global -title: Dimensionality Reduction - spark.mllib -displayTitle: Dimensionality Reduction - spark.mllib +title: Dimensionality Reduction - RDD-based API +displayTitle: Dimensionality Reduction - RDD-based API --- * Table of contents diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index 5543262a8967c..e1984b6c8d5a5 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -1,7 +1,7 @@ --- layout: global -title: Ensembles - spark.mllib -displayTitle: Ensembles - spark.mllib +title: Ensembles - RDD-based API +displayTitle: Ensembles - RDD-based API --- * Table of contents diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index c49bc4ff124bd..ac82f43cfb79d 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -1,7 +1,7 @@ --- layout: global -title: Evaluation Metrics - spark.mllib -displayTitle: Evaluation Metrics - spark.mllib +title: Evaluation Metrics - RDD-based API +displayTitle: Evaluation Metrics - RDD-based API --- * Table of contents diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 67c033e9e4003..867be7f2932ed 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -1,7 +1,7 @@ --- layout: global -title: Feature Extraction and Transformation - spark.mllib -displayTitle: Feature Extraction and Transformation - spark.mllib +title: Feature Extraction and Transformation - RDD-based API +displayTitle: Feature Extraction and Transformation - RDD-based API --- * Table of contents diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md index a7b55dc5e5668..93e3f0b2d2267 100644 --- a/docs/mllib-frequent-pattern-mining.md +++ b/docs/mllib-frequent-pattern-mining.md @@ -1,7 +1,7 @@ --- layout: global -title: Frequent Pattern Mining - spark.mllib -displayTitle: Frequent Pattern Mining - spark.mllib +title: Frequent Pattern Mining - RDD-based API +displayTitle: Frequent Pattern Mining - RDD-based API --- Mining frequent items, itemsets, subsequences, or other substructures is usually among the diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 17fd3e1edf4b4..30112c72c9c31 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -1,32 +1,12 @@ --- layout: global -title: MLlib -displayTitle: Machine Learning Library (MLlib) Guide -description: MLlib machine learning library overview for Spark SPARK_VERSION_SHORT +title: "MLlib: RDD-based API" +displayTitle: "MLlib: RDD-based API" --- -MLlib is Spark's machine learning (ML) library. -Its goal is to make practical machine learning scalable and easy. -It consists of common learning algorithms and utilities, including classification, regression, -clustering, collaborative filtering, dimensionality reduction, as well as lower-level optimization -primitives and higher-level pipeline APIs. - -It divides into two packages: - -* [`spark.mllib`](mllib-guide.html#data-types-algorithms-and-utilities) contains the original API - built on top of [RDDs](programming-guide.html#resilient-distributed-datasets-rdds). -* [`spark.ml`](ml-guide.html) provides higher-level API - built on top of [DataFrames](sql-programming-guide.html#dataframes) for constructing ML pipelines. - -Using `spark.ml` is recommended because with DataFrames the API is more versatile and flexible. -But we will keep supporting `spark.mllib` along with the development of `spark.ml`. -Users should be comfortable using `spark.mllib` features and expect more features coming. -Developers should contribute new algorithms to `spark.ml` if they fit the ML pipeline concept well, -e.g., feature extractors and transformers. - -We list major functionality from both below, with links to detailed guides. - -# spark.mllib: data types, algorithms, and utilities +This page documents sections of the MLlib guide for the RDD-based API (the `spark.mllib` package). +Please see the [MLlib Main Guide](ml-guide.html) for the DataFrame-based API (the `spark.ml` package), +which is now the primary API for MLlib. * [Data types](mllib-data-types.html) * [Basic statistics](mllib-statistics.html) @@ -65,192 +45,3 @@ We list major functionality from both below, with links to detailed guides. * [stochastic gradient descent](mllib-optimization.html#stochastic-gradient-descent-sgd) * [limited-memory BFGS (L-BFGS)](mllib-optimization.html#limited-memory-bfgs-l-bfgs) -# spark.ml: high-level APIs for ML pipelines - -* [Overview: estimators, transformers and pipelines](ml-guide.html) -* [Extracting, transforming and selecting features](ml-features.html) -* [Classification and regression](ml-classification-regression.html) -* [Clustering](ml-clustering.html) -* [Collaborative filtering](ml-collaborative-filtering.html) -* [Advanced topics](ml-advanced.html) - -Some techniques are not available yet in spark.ml, most notably dimensionality reduction -Users can seamlessly combine the implementation of these techniques found in `spark.mllib` with the rest of the algorithms found in `spark.ml`. - -# Dependencies - -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing. -If natives libraries[^1] are not available at runtime, you will see a warning message and a pure JVM -implementation will be used instead. - -Due to licensing issues with runtime proprietary binaries, we do not include `netlib-java`'s native -proxies by default. -To configure `netlib-java` / Breeze to use system optimised binaries, include -`com.github.fommil.netlib:all:1.1.2` (or build Spark with `-Pnetlib-lgpl`) as a dependency of your -project and read the [netlib-java](https://github.com/fommil/netlib-java) documentation for your -platform's additional installation instructions. - -To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. - -[^1]: To learn more about the benefits and background of system optimised natives, you may wish to - watch Sam Halliday's ScalaX talk on [High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/). - -# Migration guide - -MLlib is under active development. -The APIs marked `Experimental`/`DeveloperApi` may change in future releases, -and the migration guide below will explain all changes between releases. - -## From 1.6 to 2.0 - -### Breaking changes - -There were several breaking changes in Spark 2.0, which are outlined below. - -**Linear algebra classes for DataFrame-based APIs** - -Spark's linear algebra dependencies were moved to a new project, `mllib-local` -(see [SPARK-13944](https://issues.apache.org/jira/browse/SPARK-13944)). -As part of this change, the linear algebra classes were copied to a new package, `spark.ml.linalg`. -The DataFrame-based APIs in `spark.ml` now depend on the `spark.ml.linalg` classes, -leading to a few breaking changes, predominantly in various model classes -(see [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810) for a full list). - -**Note:** the RDD-based APIs in `spark.mllib` continue to depend on the previous package `spark.mllib.linalg`. - -_Converting vectors and matrices_ - -While most pipeline components support backward compatibility for loading, -some existing `DataFrames` and pipelines in Spark versions prior to 2.0, that contain vector or matrix -columns, may need to be migrated to the new `spark.ml` vector and matrix types. -Utilities for converting `DataFrame` columns from `spark.mllib.linalg` to `spark.ml.linalg` types -(and vice versa) can be found in `spark.mllib.util.MLUtils`. - -There are also utility methods available for converting single instances of -vectors and matrices. Use the `asML` method on a `mllib.linalg.Vector` / `mllib.linalg.Matrix` -for converting to `ml.linalg` types, and -`mllib.linalg.Vectors.fromML` / `mllib.linalg.Matrices.fromML` -for converting to `mllib.linalg` types. - -
    -
    - -{% highlight scala %} -import org.apache.spark.mllib.util.MLUtils - -// convert DataFrame columns -val convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) -val convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) -// convert a single vector or matrix -val mlVec: org.apache.spark.ml.linalg.Vector = mllibVec.asML -val mlMat: org.apache.spark.ml.linalg.Matrix = mllibMat.asML -{% endhighlight %} - -Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) for further detail. -
    - -
    - -{% highlight java %} -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.sql.Dataset; - -// convert DataFrame columns -Dataset convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF); -Dataset convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF); -// convert a single vector or matrix -org.apache.spark.ml.linalg.Vector mlVec = mllibVec.asML(); -org.apache.spark.ml.linalg.Matrix mlMat = mllibMat.asML(); -{% endhighlight %} - -Refer to the [`MLUtils` Java docs](api/java/org/apache/spark/mllib/util/MLUtils.html) for further detail. -
    - -
    - -{% highlight python %} -from pyspark.mllib.util import MLUtils - -# convert DataFrame columns -convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) -convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) -# convert a single vector or matrix -mlVec = mllibVec.asML() -mlMat = mllibMat.asML() -{% endhighlight %} - -Refer to the [`MLUtils` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.util.MLUtils) for further detail. -
    -
    - -**Deprecated methods removed** - -Several deprecated methods were removed in the `spark.mllib` and `spark.ml` packages: - -* `setScoreCol` in `ml.evaluation.BinaryClassificationEvaluator` -* `weights` in `LinearRegression` and `LogisticRegression` in `spark.ml` -* `setMaxNumIterations` in `mllib.optimization.LBFGS` (marked as `DeveloperApi`) -* `treeReduce` and `treeAggregate` in `mllib.rdd.RDDFunctions` (these functions are available on `RDD`s directly, and were marked as `DeveloperApi`) -* `defaultStategy` in `mllib.tree.configuration.Strategy` -* `build` in `mllib.tree.Node` -* libsvm loaders for multiclass and load/save labeledData methods in `mllib.util.MLUtils` - -A full list of breaking changes can be found at [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810). - -### Deprecations and changes of behavior - -**Deprecations** - -Deprecations in the `spark.mllib` and `spark.ml` packages include: - -* [SPARK-14984](https://issues.apache.org/jira/browse/SPARK-14984): - In `spark.ml.regression.LinearRegressionSummary`, the `model` field has been deprecated. -* [SPARK-13784](https://issues.apache.org/jira/browse/SPARK-13784): - In `spark.ml.regression.RandomForestRegressionModel` and `spark.ml.classification.RandomForestClassificationModel`, - the `numTrees` parameter has been deprecated in favor of `getNumTrees` method. -* [SPARK-13761](https://issues.apache.org/jira/browse/SPARK-13761): - In `spark.ml.param.Params`, the `validateParams` method has been deprecated. - We move all functionality in overridden methods to the corresponding `transformSchema`. -* [SPARK-14829](https://issues.apache.org/jira/browse/SPARK-14829): - In `spark.mllib` package, `LinearRegressionWithSGD`, `LassoWithSGD`, `RidgeRegressionWithSGD` and `LogisticRegressionWithSGD` have been deprecated. - We encourage users to use `spark.ml.regression.LinearRegresson` and `spark.ml.classification.LogisticRegresson`. -* [SPARK-14900](https://issues.apache.org/jira/browse/SPARK-14900): - In `spark.mllib.evaluation.MulticlassMetrics`, the parameters `precision`, `recall` and `fMeasure` have been deprecated in favor of `accuracy`. -* [SPARK-15644](https://issues.apache.org/jira/browse/SPARK-15644): - In `spark.ml.util.MLReader` and `spark.ml.util.MLWriter`, the `context` method has been deprecated in favor of `session`. -* In `spark.ml.feature.ChiSqSelectorModel`, the `setLabelCol` method has been deprecated since it was not used by `ChiSqSelectorModel`. - -**Changes of behavior** - -Changes of behavior in the `spark.mllib` and `spark.ml` packages include: - -* [SPARK-7780](https://issues.apache.org/jira/browse/SPARK-7780): - `spark.mllib.classification.LogisticRegressionWithLBFGS` directly calls `spark.ml.classification.LogisticRegresson` for binary classification now. - This will introduce the following behavior changes for `spark.mllib.classification.LogisticRegressionWithLBFGS`: - * The intercept will not be regularized when training binary classification model with L1/L2 Updater. - * If users set without regularization, training with or without feature scaling will return the same solution by the same convergence rate. -* [SPARK-13429](https://issues.apache.org/jira/browse/SPARK-13429): - In order to provide better and consistent result with `spark.ml.classification.LogisticRegresson`, - the default value of `spark.mllib.classification.LogisticRegressionWithLBFGS`: `convergenceTol` has been changed from 1E-4 to 1E-6. -* [SPARK-12363](https://issues.apache.org/jira/browse/SPARK-12363): - Fix a bug of `PowerIterationClustering` which will likely change its result. -* [SPARK-13048](https://issues.apache.org/jira/browse/SPARK-13048): - `LDA` using the `EM` optimizer will keep the last checkpoint by default, if checkpointing is being used. -* [SPARK-12153](https://issues.apache.org/jira/browse/SPARK-12153): - `Word2Vec` now respects sentence boundaries. Previously, it did not handle them correctly. -* [SPARK-10574](https://issues.apache.org/jira/browse/SPARK-10574): - `HashingTF` uses `MurmurHash3` as default hash algorithm in both `spark.ml` and `spark.mllib`. -* [SPARK-14768](https://issues.apache.org/jira/browse/SPARK-14768): - The `expectedType` argument for PySpark `Param` was removed. -* [SPARK-14931](https://issues.apache.org/jira/browse/SPARK-14931): - Some default `Param` values, which were mismatched between pipelines in Scala and Python, have been changed. -* [SPARK-13600](https://issues.apache.org/jira/browse/SPARK-13600): - `QuantileDiscretizer` now uses `spark.sql.DataFrameStatFunctions.approxQuantile` to find splits (previously used custom sampling logic). - The output buckets will differ for same input data and params. - -## Previous Spark versions - -Earlier migration guides are archived [on this page](mllib-migration-guides.html). - ---- diff --git a/docs/mllib-isotonic-regression.md b/docs/mllib-isotonic-regression.md index 8ede4407d5843..d90905a86ade9 100644 --- a/docs/mllib-isotonic-regression.md +++ b/docs/mllib-isotonic-regression.md @@ -1,7 +1,7 @@ --- layout: global -title: Isotonic regression - spark.mllib -displayTitle: Regression - spark.mllib +title: Isotonic regression - RDD-based API +displayTitle: Regression - RDD-based API --- ## Isotonic regression diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 17d781ac23f81..6fcd3ae85700c 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -1,7 +1,7 @@ --- layout: global -title: Linear Methods - spark.mllib -displayTitle: Linear Methods - spark.mllib +title: Linear Methods - RDD-based API +displayTitle: Linear Methods - RDD-based API --- * Table of contents diff --git a/docs/mllib-migration-guides.md b/docs/mllib-migration-guides.md index 970c6697f433e..ea6f93fcf67f3 100644 --- a/docs/mllib-migration-guides.md +++ b/docs/mllib-migration-guides.md @@ -1,159 +1,9 @@ --- layout: global -title: Old Migration Guides - spark.mllib -displayTitle: Old Migration Guides - spark.mllib -description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +title: Old Migration Guides - MLlib +displayTitle: Old Migration Guides - MLlib --- -The migration guide for the current Spark version is kept on the [MLlib Programming Guide main page](mllib-guide.html#migration-guide). - -## From 1.5 to 1.6 - -There are no breaking API changes in the `spark.mllib` or `spark.ml` packages, but there are -deprecations and changes of behavior. - -Deprecations: - -* [SPARK-11358](https://issues.apache.org/jira/browse/SPARK-11358): - In `spark.mllib.clustering.KMeans`, the `runs` parameter has been deprecated. -* [SPARK-10592](https://issues.apache.org/jira/browse/SPARK-10592): - In `spark.ml.classification.LogisticRegressionModel` and - `spark.ml.regression.LinearRegressionModel`, the `weights` field has been deprecated in favor of - the new name `coefficients`. This helps disambiguate from instance (row) "weights" given to - algorithms. - -Changes of behavior: - -* [SPARK-7770](https://issues.apache.org/jira/browse/SPARK-7770): - `spark.mllib.tree.GradientBoostedTrees`: `validationTol` has changed semantics in 1.6. - Previously, it was a threshold for absolute change in error. Now, it resembles the behavior of - `GradientDescent`'s `convergenceTol`: For large errors, it uses relative error (relative to the - previous error); for small errors (`< 0.01`), it uses absolute error. -* [SPARK-11069](https://issues.apache.org/jira/browse/SPARK-11069): - `spark.ml.feature.RegexTokenizer`: Previously, it did not convert strings to lowercase before - tokenizing. Now, it converts to lowercase by default, with an option not to. This matches the - behavior of the simpler `Tokenizer` transformer. - -## From 1.4 to 1.5 - -In the `spark.mllib` package, there are no breaking API changes but several behavior changes: - -* [SPARK-9005](https://issues.apache.org/jira/browse/SPARK-9005): - `RegressionMetrics.explainedVariance` returns the average regression sum of squares. -* [SPARK-8600](https://issues.apache.org/jira/browse/SPARK-8600): `NaiveBayesModel.labels` become - sorted. -* [SPARK-3382](https://issues.apache.org/jira/browse/SPARK-3382): `GradientDescent` has a default - convergence tolerance `1e-3`, and hence iterations might end earlier than 1.4. - -In the `spark.ml` package, there exists one breaking API change and one behavior change: - -* [SPARK-9268](https://issues.apache.org/jira/browse/SPARK-9268): Java's varargs support is removed - from `Params.setDefault` due to a - [Scala compiler bug](https://issues.scala-lang.org/browse/SI-9013). -* [SPARK-10097](https://issues.apache.org/jira/browse/SPARK-10097): `Evaluator.isLargerBetter` is - added to indicate metric ordering. Metrics like RMSE no longer flip signs as in 1.4. - -## From 1.3 to 1.4 - -In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: - -* Gradient-Boosted Trees - * *(Breaking change)* The signature of the [`Loss.gradient`](api/scala/index.html#org.apache.spark.mllib.tree.loss.Loss) method was changed. This is only an issues for users who wrote their own losses for GBTs. - * *(Breaking change)* The `apply` and `copy` methods for the case class [`BoostingStrategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.BoostingStrategy) have been changed because of a modification to the case class fields. This could be an issue for users who use `BoostingStrategy` to set GBT parameters. -* *(Breaking change)* The return value of [`LDA.run`](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) has changed. It now returns an abstract class `LDAModel` instead of the concrete class `DistributedLDAModel`. The object of type `LDAModel` can still be cast to the appropriate concrete type, which depends on the optimization algorithm. - -In the `spark.ml` package, several major API changes occurred, including: - -* `Param` and other APIs for specifying parameters -* `uid` unique IDs for Pipeline components -* Reorganization of certain classes - -Since the `spark.ml` API was an alpha component in Spark 1.3, we do not list all changes here. -However, since 1.4 `spark.ml` is no longer an alpha component, we will provide details on any API -changes for future releases. - -## From 1.2 to 1.3 - -In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. - -* *(Breaking change)* In [`ALS`](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS), the extraneous method `solveLeastSquares` has been removed. The `DeveloperApi` method `analyzeBlocks` was also removed. -* *(Breaking change)* [`StandardScalerModel`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScalerModel) remains an Alpha component. In it, the `variance` method has been replaced with the `std` method. To compute the column variance values returned by the original `variance` method, simply square the standard deviation values returned by `std`. -* *(Breaking change)* [`StreamingLinearRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD) remains an Experimental component. In it, there were two changes: - * The constructor taking arguments was removed in favor of a builder pattern using the default constructor plus parameter setter methods. - * Variable `model` is no longer public. -* *(Breaking change)* [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) remains an Experimental component. In it and its associated classes, there were several changes: - * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) - * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. -* `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. -* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. - So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. - -In the `spark.ml` package, the main API changes are from Spark SQL. We list the most important changes here: - -* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in Spark ML which used to use SchemaRDD now use DataFrame. -* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`. -* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details. - -Other changes were in `LogisticRegression`: - -* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). -* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. - -## From 1.1 to 1.2 - -The only API changes in MLlib v1.2 are in -[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -which continues to be an experimental API in MLlib 1.2: - -1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number -of classes. In MLlib v1.1, this argument was called `numClasses` in Python and -`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. -This `numClasses` parameter is specified either via -[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) -or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) -static `trainClassifier` and `trainRegressor` methods. - -2. *(Breaking change)* The API for -[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. -This should generally not affect user code, unless the user manually constructs decision trees -(instead of using the `trainClassifier` or `trainRegressor` methods). -The tree `Node` now includes more information, including the probability of the predicted label -(for classification). - -3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. - -Examples in the Spark distribution and examples in the -[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. - -## From 1.0 to 1.1 - -The only API changes in MLlib v1.1 are in -[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -which continues to be an experimental API in MLlib 1.1: - -1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match -the implementations of trees in -[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) -and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). -In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. -In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. -This depth is specified by the `maxDepth` parameter in -[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) -or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) -static `trainClassifier` and `trainRegressor` methods. - -2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` -methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -rather than using the old parameter class `Strategy`. These new training methods explicitly -separate classification and regression, and they replace specialized parameter types with -simple `String` types. - -Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the -[Decision Trees Guide](mllib-decision-tree.html#examples). - -## From 0.9 to 1.0 - -In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few -breaking changes. If your data is sparse, please store it in a sparse format instead of dense to -take advantage of sparsity in both storage and computation. Details are described below. +The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). +Past migration guides are now stored at [ml-migration-guides.html](ml-migration-guides.html). diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index d0d594af6a4ad..7471d18a0dddc 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -1,7 +1,7 @@ --- layout: global -title: Naive Bayes - spark.mllib -displayTitle: Naive Bayes - spark.mllib +title: Naive Bayes - RDD-based API +displayTitle: Naive Bayes - RDD-based API --- [Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) is a simple diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index f90b66f8e2c44..eefd7dcf1108b 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -1,7 +1,7 @@ --- layout: global -title: Optimization - spark.mllib -displayTitle: Optimization - spark.mllib +title: Optimization - RDD-based API +displayTitle: Optimization - RDD-based API --- * Table of contents diff --git a/docs/mllib-pmml-model-export.md b/docs/mllib-pmml-model-export.md index 7f2347dc0b769..d3530908706d0 100644 --- a/docs/mllib-pmml-model-export.md +++ b/docs/mllib-pmml-model-export.md @@ -1,7 +1,7 @@ --- layout: global -title: PMML model export - spark.mllib -displayTitle: PMML model export - spark.mllib +title: PMML model export - RDD-based API +displayTitle: PMML model export - RDD-based API --- * Table of contents diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index 329855e565b24..12797bd8688e1 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -1,7 +1,7 @@ --- layout: global -title: Basic Statistics - spark.mllib -displayTitle: Basic Statistics - spark.mllib +title: Basic Statistics - RDD-based API +displayTitle: Basic Statistics - RDD-based API --- * Table of contents diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 2bc49120a0e24..888c12f18635c 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1571,7 +1571,7 @@ have changed from returning (key, list of values) pairs to (key, iterable of val
    Migration guides are also available for [Spark Streaming](streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x), -[MLlib](mllib-guide.html#migration-guide) and [GraphX](graphx-programming-guide.html#migrating-from-spark-091). +[MLlib](ml-guide.html#migration-guide) and [GraphX](graphx-programming-guide.html#migrating-from-spark-091). # Where to Go from Here diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 2ee3b80185c2f..de82a064d1d3e 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -15,7 +15,7 @@ like Kafka, Flume, Kinesis, or TCP sockets, and can be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark's -[machine learning](mllib-guide.html) and +[machine learning](ml-guide.html) and [graph processing](graphx-programming-guide.html) algorithms on data streams.

    @@ -1673,7 +1673,7 @@ See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more abo *** ## MLlib Operations -You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. [Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. +You can also easily use machine learning algorithms provided by [MLlib](ml-guide.html). First of all, there are streaming machine learning algorithms (e.g. [Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](ml-guide.html) guide for more details. *** diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 05f3be5f0d6e7..1d42d49a8816b 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -16,8 +16,8 @@ # """ -Spark ML is a component that adds a new set of machine learning APIs to let users quickly -assemble and configure practical machine learning pipelines. +DataFrame-based machine learning APIs to let users quickly assemble and configure practical +machine learning pipelines. """ from pyspark.ml.base import Estimator, Model, Transformer from pyspark.ml.pipeline import Pipeline, PipelineModel diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 24efce812b3b3..4bcb2c400c4aa 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -16,7 +16,7 @@ # """ -Unit tests for Spark ML Python APIs. +Unit tests for MLlib Python DataFrame-based APIs. """ import sys if sys.version > '3': diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index acba3a717d21a..ae26521ea96bf 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -16,7 +16,10 @@ # """ -Python bindings for MLlib. +RDD-based machine learning APIs for Python (in maintenance mode). + +The `pyspark.mllib` package is in maintenance mode as of the Spark 2.0.0 release to encourage +migration to the DataFrame-based APIs under the `pyspark.ml` package. """ from __future__ import absolute_import From e833c906f4f3e6b82a1fc03f69cf94de06d85c61 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 15 Jul 2016 13:58:57 -0700 Subject: [PATCH 0987/1470] [SPARK-16538][SPARKR] Add more tests for namespace call to SparkSession functions ## What changes were proposed in this pull request? More tests I don't think this is critical for Spark 2.0.0 RC, maybe Spark 2.0.1 or 2.1.0. ## How was this patch tested? unit tests shivaram dongjoon-hyun Author: Felix Cheung Closes #14206 from felixcheung/rroutetests. (cherry picked from commit 611a8ca5895357059f1e7c035d946e0718b26a5a) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 20c750a50c70a..7e59fdf4620e1 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2416,6 +2416,13 @@ test_that("createDataFrame sqlContext parameter backward compatibility", { before <- suppressWarnings(createDataFrame(sqlContext, iris)) after <- suppressWarnings(createDataFrame(iris)) expect_equal(collect(before), collect(after)) + + # more tests for SPARK-16538 + createOrReplaceTempView(df, "table") + SparkR::tables() + SparkR::sql("SELECT 1") + suppressWarnings(SparkR::sql(sqlContext, "SELECT * FROM table")) + suppressWarnings(SparkR::dropTempTable(sqlContext, "table")) }) test_that("randomSplit", { From 34ac45a34d5673112c84ed464a7a23c68c7bd8fe Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Fri, 15 Jul 2016 14:27:16 -0700 Subject: [PATCH 0988/1470] [SPARK-16230][CORE] CoarseGrainedExecutorBackend to self kill if there is an exception while creating an Executor ## What changes were proposed in this pull request? With the fix from SPARK-13112, I see that `LaunchTask` is always processed after `RegisteredExecutor` is done and so it gets chance to do all retries to startup an executor. There is still a problem that if `Executor` creation itself fails and there is some exception, it gets unnoticed and the executor is killed when it tries to process the `LaunchTask` as `executor` is null : https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala#L88 So if one looks at the logs, it does not tell that there was problem during `Executor` creation and thats why it was killed. This PR explicitly catches exception in `Executor` creation, logs a proper message and then exits the JVM. Also, I have changed the `exitExecutor` method to accept `reason` so that backends can use that reason and do stuff like logging to a DB to get an aggregate of such exits at a cluster level ## How was this patch tested? I am relying on existing tests Author: Tejas Patil Closes #14202 from tejasapatil/exit_executor_failure. (cherry picked from commit b2f24f94591082d3ff82bd3db1760b14603b38aa) Signed-off-by: Shixiong Zhu --- .../CoarseGrainedExecutorBackend.scala | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index ccc6c36e9c79a..e30839c49c04f 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable import scala.util.{Failure, Success} +import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.TaskState.TaskState @@ -64,8 +65,7 @@ private[spark] class CoarseGrainedExecutorBackend( case Success(msg) => // Always receive `true`. Just ignore it case Failure(e) => - logError(s"Cannot register with driver: $driverUrl", e) - exitExecutor(1) + exitExecutor(1, s"Cannot register with driver: $driverUrl", e) }(ThreadUtils.sameThread) } @@ -78,16 +78,19 @@ private[spark] class CoarseGrainedExecutorBackend( override def receive: PartialFunction[Any, Unit] = { case RegisteredExecutor => logInfo("Successfully registered with driver") - executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + try { + executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + } catch { + case NonFatal(e) => + exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) + } case RegisterExecutorFailed(message) => - logError("Slave registration failed: " + message) - exitExecutor(1) + exitExecutor(1, "Slave registration failed: " + message) case LaunchTask(data) => if (executor == null) { - logError("Received LaunchTask command but executor was null") - exitExecutor(1) + exitExecutor(1, "Received LaunchTask command but executor was null") } else { val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) @@ -97,8 +100,7 @@ private[spark] class CoarseGrainedExecutorBackend( case KillTask(taskId, _, interruptThread) => if (executor == null) { - logError("Received KillTask command but executor was null") - exitExecutor(1) + exitExecutor(1, "Received KillTask command but executor was null") } else { executor.killTask(taskId, interruptThread) } @@ -127,8 +129,7 @@ private[spark] class CoarseGrainedExecutorBackend( if (stopping.get()) { logInfo(s"Driver from $remoteAddress disconnected during shutdown") } else if (driver.exists(_.address == remoteAddress)) { - logError(s"Driver $remoteAddress disassociated! Shutting down.") - exitExecutor(1) + exitExecutor(1, s"Driver $remoteAddress disassociated! Shutting down.") } else { logWarning(s"An unknown ($remoteAddress) driver disconnected.") } @@ -147,7 +148,14 @@ private[spark] class CoarseGrainedExecutorBackend( * executor exits differently. For e.g. when an executor goes down, * back-end may not want to take the parent process down. */ - protected def exitExecutor(code: Int): Unit = System.exit(code) + protected def exitExecutor(code: Int, reason: String, throwable: Throwable = null) = { + if (throwable != null) { + logError(reason, throwable) + } else { + logError(reason) + } + System.exit(code) + } } private[spark] object CoarseGrainedExecutorBackend extends Logging { From 5d495292ba662c2081cdcb5fd5c595522bfd5c74 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Sat, 16 Jul 2016 13:24:00 -0700 Subject: [PATCH 0989/1470] [SPARK-16582][SQL] Explicitly define isNull = false for non-nullable expressions ## What changes were proposed in this pull request? This patch is just a slightly safer way to fix the issue we encountered in https://github.com/apache/spark/pull/14168 should this pattern re-occur at other places in the code. ## How was this patch tested? Existing tests. Also, I manually tested that it fixes the problem in SPARK-16514 without having the proposed change in https://github.com/apache/spark/pull/14168 Author: Sameer Agarwal Closes #14227 from sameeragarwal/codegen. (cherry picked from commit a1ffbada8a266a4130de6fffc4a5efd085a29ae4) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/expressions/Expression.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 10a141254f54e..1f37b68846ae4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -377,6 +377,7 @@ abstract class UnaryExpression extends Expression { """) } else { ev.copy(code = s""" + boolean ${ev.isNull} = false; ${childGen.code} ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; $resultCode""", isNull = "false") @@ -475,6 +476,7 @@ abstract class BinaryExpression extends Expression { """) } else { ev.copy(code = s""" + boolean ${ev.isNull} = false; ${leftGen.code} ${rightGen.code} ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; @@ -617,6 +619,7 @@ abstract class TernaryExpression extends Expression { $nullSafeEval""") } else { ev.copy(code = s""" + boolean ${ev.isNull} = false; ${leftGen.code} ${midGen.code} ${rightGen.code} From cad4693f963fe1d1ca67803b3c3d8958797e034c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 16 Jul 2016 13:26:58 -0700 Subject: [PATCH 0990/1470] [SPARK-3359][DOCS] More changes to resolve javadoc 8 errors that will help unidoc/genjavadoc compatibility ## What changes were proposed in this pull request? These are yet more changes that resolve problems with unidoc/genjavadoc and Java 8. It does not fully resolve the problem, but gets rid of as many errors as we can from this end. ## How was this patch tested? Jenkins build of docs Author: Sean Owen Closes #14221 from srowen/SPARK-3359.3. (cherry picked from commit 5ec0d692b0789a1d06db35134ee6eac2ecce47c3) Signed-off-by: Reynold Xin --- .../spark/graphx/util/GraphGenerators.scala | 2 +- .../org/apache/spark/ml/linalg/Vectors.scala | 2 +- .../scala/org/apache/spark/ml/Pipeline.scala | 2 +- .../scala/org/apache/spark/ml/Predictor.scala | 2 +- .../spark/ml/classification/Classifier.scala | 6 +++--- .../classification/DecisionTreeClassifier.scala | 4 ++-- .../spark/ml/classification/GBTClassifier.scala | 8 ++++---- .../ml/classification/LogisticRegression.scala | 14 +++++++------- .../classification/ProbabilisticClassifier.scala | 10 +++++----- .../apache/spark/ml/evaluation/Evaluator.scala | 2 +- .../apache/spark/ml/feature/ChiSqSelector.scala | 4 ++-- .../scala/org/apache/spark/ml/param/params.scala | 13 ++++++------- .../org/apache/spark/ml/recommendation/ALS.scala | 16 ++++++++-------- .../scala/org/apache/spark/ml/tree/Node.scala | 4 ++-- .../scala/org/apache/spark/ml/tree/Split.scala | 4 ++-- .../org/apache/spark/ml/tree/treeModels.scala | 4 ++-- .../spark/mllib/feature/ChiSqSelector.scala | 4 ++-- .../org/apache/spark/mllib/feature/PCA.scala | 4 ++-- .../spark/mllib/feature/StandardScaler.scala | 5 ++--- .../apache/spark/mllib/tree/impurity/Gini.scala | 4 ++-- .../apache/spark/mllib/util/modelSaveLoad.scala | 2 +- 21 files changed, 57 insertions(+), 59 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 4da1ecb2a9af3..2b3e5f98c4fe5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -119,7 +119,7 @@ object GraphGenerators extends Logging { * A random graph generator using the R-MAT model, proposed in * "R-MAT: A Recursive Model for Graph Mining" by Chakrabarti et al. * - * See [[http://www.cs.cmu.edu/~christos/PUBLICATIONS/siam04.pdf]]. + * See http://www.cs.cmu.edu/~christos/PUBLICATIONS/siam04.pdf. */ def rmatGraph(sc: SparkContext, requestedNumVertices: Int, numEdges: Int): Graph[Int, Int] = { // let N = requestedNumVertices diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index c74e5d44a328d..0659324aad1fa 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -66,7 +66,7 @@ sealed trait Vector extends Serializable { /** * Returns a hash code value for the vector. The hash code is based on its size and its first 128 - * nonzero entries, using a hash algorithm similar to [[java.util.Arrays.hashCode]]. + * nonzero entries, using a hash algorithm similar to `java.util.Arrays.hashCode`. */ override def hashCode(): Int = { // This is a reference implementation. It calls return in foreachActive, which is slow. 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 d18fb697994fa..195a93e086725 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -212,7 +212,7 @@ object Pipeline extends MLReadable[Pipeline] { } } - /** Methods for [[MLReader]] and [[MLWriter]] shared between [[Pipeline]] and [[PipelineModel]] */ + /** Methods for `MLReader` and `MLWriter` shared between [[Pipeline]] and [[PipelineModel]] */ private[ml] object SharedReadWrite { import org.json4s.JsonDSL._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index 569a5fb993768..e29d7f48a1d6b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -165,7 +165,7 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, } /** - * Transforms dataset by reading from [[featuresCol]], calling [[predict()]], and storing + * Transforms dataset by reading from [[featuresCol]], calling `predict`, and storing * the predictions as a new column [[predictionCol]]. * * @param dataset input dataset 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 e35b04a1cf423..6decea72719fd 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 @@ -50,7 +50,7 @@ private[spark] trait ClassifierParams * Single-label binary or multiclass classification. * Classes are indexed {0, 1, ..., numClasses - 1}. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam E Concrete Estimator type * @tparam M Concrete Model type */ @@ -134,7 +134,7 @@ abstract class Classifier[ * Model produced by a [[Classifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam M Concrete Model type */ @DeveloperApi @@ -151,7 +151,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by * parameters: * - predicted labels as [[predictionCol]] of type [[Double]] - * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]]. + * - raw predictions (confidences) as [[rawPredictionCol]] of type `Vector`. * * @param dataset input dataset * @return transformed dataset diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 082848c9ded5b..71293017e0527 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.Dataset /** - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * Decision tree learning algorithm (http://en.wikipedia.org/wiki/Decision_tree_learning) * for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. @@ -135,7 +135,7 @@ object DecisionTreeClassifier extends DefaultParamsReadable[DecisionTreeClassifi } /** - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for classification. + * Decision tree model (http://en.wikipedia.org/wiki/Decision_tree_learning) for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 5946a12933ff2..ba70293273f94 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType /** - * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * Gradient-Boosted Trees (GBTs) (http://en.wikipedia.org/wiki/Gradient_boosting) * learning algorithm for classification. * It supports binary labels, as well as both continuous and categorical features. * Note: Multiclass labels are not currently supported. @@ -158,7 +158,7 @@ object GBTClassifier extends DefaultParamsReadable[GBTClassifier] { } /** - * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * Gradient-Boosted Trees (GBTs) (http://en.wikipedia.org/wiki/Gradient_boosting) * model for classification. * It supports binary labels, as well as both continuous and categorical features. * Note: Multiclass labels are not currently supported. @@ -233,8 +233,8 @@ class GBTClassificationModel private[ml]( * The importance vector is normalized to sum to 1. This method is suggested by Hastie et al. * (Hastie, Tibshirani, Friedman. "The Elements of Statistical Learning, 2nd Edition." 2001.) * and follows the implementation from scikit-learn. - * - * @see [[DecisionTreeClassificationModel.featureImportances]] + + * See `DecisionTreeClassificationModel.featureImportances` */ @Since("2.0.0") lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) 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 4bab801bb3621..1fed5fd42987e 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 @@ -863,10 +863,10 @@ class BinaryLogisticRegressionSummary private[classification] ( * Returns the receiver operating characteristic (ROC) curve, * which is a Dataframe having two fields (FPR, TPR) * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. + * See http://en.wikipedia.org/wiki/Receiver_operating_characteristic * - * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. + * Note: This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. * This will change in later Spark versions. - * @see http://en.wikipedia.org/wiki/Receiver_operating_characteristic */ @Since("1.5.0") @transient lazy val roc: DataFrame = binaryMetrics.roc().toDF("FPR", "TPR") @@ -874,7 +874,7 @@ class BinaryLogisticRegressionSummary private[classification] ( /** * Computes the area under the receiver operating characteristic (ROC) curve. * - * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. + * Note: This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. * This will change in later Spark versions. */ @Since("1.5.0") @@ -884,7 +884,7 @@ class BinaryLogisticRegressionSummary private[classification] ( * Returns the precision-recall curve, which is a Dataframe containing * two fields recall, precision with (0.0, 1.0) prepended to it. * - * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. + * Note: This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. * This will change in later Spark versions. */ @Since("1.5.0") @@ -893,7 +893,7 @@ class BinaryLogisticRegressionSummary private[classification] ( /** * Returns a dataframe with two fields (threshold, F-Measure) curve with beta = 1.0. * - * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. + * Note: This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. * This will change in later Spark versions. */ @Since("1.5.0") @@ -906,7 +906,7 @@ class BinaryLogisticRegressionSummary private[classification] ( * Every possible probability obtained in transforming the dataset are used * as thresholds used in calculating the precision. * - * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. + * Note: This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. * This will change in later Spark versions. */ @Since("1.5.0") @@ -919,7 +919,7 @@ class BinaryLogisticRegressionSummary private[classification] ( * Every possible probability obtained in transforming the dataset are used * as thresholds used in calculating the recall. * - * Note: This ignores instance weights (setting all to 1.0) from [[LogisticRegression.weightCol]]. + * Note: This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. * This will change in later Spark versions. */ @Since("1.5.0") 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 59277d0f42b34..88642abf63221 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 @@ -45,7 +45,7 @@ private[classification] trait ProbabilisticClassifierParams * * Single-label binary or multiclass classifier which can output class conditional probabilities. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam E Concrete Estimator type * @tparam M Concrete Model type */ @@ -70,7 +70,7 @@ abstract class ProbabilisticClassifier[ * Model produced by a [[ProbabilisticClassifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam M Concrete Model type */ @DeveloperApi @@ -89,8 +89,8 @@ abstract class ProbabilisticClassificationModel[ * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by * parameters: * - predicted labels as [[predictionCol]] of type [[Double]] - * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]] - * - probability of each class as [[probabilityCol]] of type [[Vector]]. + * - raw predictions (confidences) as [[rawPredictionCol]] of type `Vector` + * - probability of each class as [[probabilityCol]] of type `Vector`. * * @param dataset input dataset * @return transformed dataset @@ -210,7 +210,7 @@ private[ml] object ProbabilisticClassificationModel { /** * Normalize a vector of raw predictions to be a multinomial probability vector, in place. * - * The input raw predictions should be >= 0. + * The input raw predictions should be nonnegative. * The output vector sums to 1, unless the input vector is all-0 (in which case the output is * all-0 too). * diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala index dfbc3e5222c2f..e7b949ddce344 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala @@ -53,7 +53,7 @@ abstract class Evaluator extends Params { def evaluate(dataset: Dataset[_]): Double /** - * Indicates whether the metric returned by [[evaluate()]] should be maximized (true, default) + * Indicates whether the metric returned by `evaluate` should be maximized (true, default) * or minimized (false). * A given evaluator may support multiple metrics which may be maximized or minimized. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index bd053e886f171..1482eb3d1f7a6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -42,8 +42,8 @@ private[feature] trait ChiSqSelectorParams extends Params /** * Number of features that selector will select (ordered by statistic value descending). If the - * number of features is < numTopFeatures, then this will select all features. The default value - * of numTopFeatures is 50. + * number of features is less than numTopFeatures, then this will select all features. + * The default value of numTopFeatures is 50. * @group param */ final val numTopFeatures = new IntParam(this, "numTopFeatures", 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 e7780cf1c39f9..9245931b27ca6 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 @@ -552,7 +552,7 @@ trait Params extends Identifiable with Serializable { * * This only needs to check for interactions between parameters. * Parameter value checks which do not depend on other parameters are handled by - * [[Param.validate()]]. This method does not handle input/output column parameters; + * `Param.validate()`. This method does not handle input/output column parameters; * those are checked during schema validation. * @deprecated Will be removed in 2.1.0. All the checks should be merged into transformSchema */ @@ -580,8 +580,7 @@ trait Params extends Identifiable with Serializable { } /** - * Explains all params of this instance. - * @see [[explainParam()]] + * Explains all params of this instance. See `explainParam()`. */ def explainParams(): String = { params.map(explainParam).mkString("\n") @@ -678,7 +677,7 @@ trait Params extends Identifiable with Serializable { /** * Sets default values for a list of params. * - * Note: Java developers should use the single-parameter [[setDefault()]]. + * Note: Java developers should use the single-parameter `setDefault`. * Annotating this with varargs can cause compilation failures due to a Scala compiler bug. * See SPARK-9268. * @@ -712,8 +711,7 @@ trait Params extends Identifiable with Serializable { /** * Creates a copy of this instance with the same UID and some extra params. * Subclasses should implement this method and set the return type properly. - * - * @see [[defaultCopy()]] + * See `defaultCopy()`. */ def copy(extra: ParamMap): Params @@ -730,7 +728,8 @@ trait Params extends Identifiable with Serializable { /** * 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 < extra. + * conflicts, i.e., with ordering: + * default param values less than user-supplied values less than extra. */ final def extractParamMap(extra: ParamMap): ParamMap = { defaultParamMap ++ paramMap ++ extra 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 a2c4c2691190e..02e2384afe530 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 @@ -99,7 +99,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w with HasPredictionCol with HasCheckpointInterval with HasSeed { /** - * Param for rank of the matrix factorization (>= 1). + * Param for rank of the matrix factorization (positive). * Default: 10 * @group param */ @@ -109,7 +109,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w def getRank: Int = $(rank) /** - * Param for number of user blocks (>= 1). + * Param for number of user blocks (positive). * Default: 10 * @group param */ @@ -120,7 +120,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w def getNumUserBlocks: Int = $(numUserBlocks) /** - * Param for number of item blocks (>= 1). + * Param for number of item blocks (positive). * Default: 10 * @group param */ @@ -141,7 +141,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w def getImplicitPrefs: Boolean = $(implicitPrefs) /** - * Param for the alpha parameter in the implicit preference formulation (>= 0). + * Param for the alpha parameter in the implicit preference formulation (nonnegative). * Default: 1.0 * @group param */ @@ -174,7 +174,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w /** * Param for StorageLevel for intermediate datasets. Pass in a string representation of - * [[StorageLevel]]. Cannot be "NONE". + * `StorageLevel`. Cannot be "NONE". * Default: "MEMORY_AND_DISK". * * @group expertParam @@ -188,7 +188,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w /** * Param for StorageLevel for ALS model factors. Pass in a string representation of - * [[StorageLevel]]. + * `StorageLevel`. * Default: "MEMORY_AND_DISK". * * @group expertParam @@ -351,11 +351,11 @@ object ALSModel extends MLReadable[ALSModel] { * * For implicit preference data, the algorithm used is based on * "Collaborative Filtering for Implicit Feedback Datasets", available at - * [[http://dx.doi.org/10.1109/ICDM.2008.22]], adapted for the blocked approach used here. + * http://dx.doi.org/10.1109/ICDM.2008.22, adapted for the blocked approach used here. * * Essentially instead of finding the low-rank approximations to the rating matrix `R`, * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if - * r > 0 and 0 if r <= 0. The ratings then act as 'confidence' values related to strength of + * r > 0 and 0 if r <= 0. The ratings then act as 'confidence' values related to strength of * indicated user * preferences rather than explicit ratings given to items. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index 8144bcb7d46f1..07e98a142b10e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -145,8 +145,8 @@ class LeafNode private[ml] ( * Internal Decision Tree node. * @param prediction Prediction this node would make if it were a leaf node * @param impurity Impurity measure at this node (for training data) - * @param gain Information gain value. - * Values < 0 indicate missing values; this quirk will be removed with future updates. + * @param gain Information gain value. Values less than 0 indicate missing values; + * this quirk will be removed with future updates. * @param leftChild Left-hand child node * @param rightChild Right-hand child node * @param split Information about the test used to split to the left or right child. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index 47fe3524f2296..dff44e2d49ec8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -151,8 +151,8 @@ class CategoricalSplit private[ml] ( /** * Split which tests a continuous feature. * @param featureIndex Index of the feature to test - * @param threshold If the feature value is <= this threshold, then the split goes left. - * Otherwise, it goes right. + * @param threshold If the feature value is less than or equal to this threshold, then the + * split goes left. Otherwise, it goes right. */ class ContinuousSplit private[ml] (override val featureIndex: Int, val threshold: Double) extends Split { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index 5b6fcc53c2dd5..d3cbc363799a5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -415,12 +415,12 @@ private[ml] object EnsembleModelReadWrite { /** * Helper method for loading a tree ensemble from disk. * This reconstructs all trees, returning the root nodes. - * @param path Path given to [[saveImpl()]] + * @param path Path given to `saveImpl` * @param className Class name for ensemble model type * @param treeClassName Class name for tree model type in the ensemble * @return (ensemble metadata, array over trees of (tree metadata, root node)), * where the root node is linked with all descendents - * @see [[saveImpl()]] for how the model was saved + * @see `saveImpl` for how the model was saved */ def loadImpl( path: String, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index c8c2823bbaf04..56fb2d33c2ca0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -173,8 +173,8 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { * Creates a ChiSquared feature selector. * @param numTopFeatures number of features that selector will select * (ordered by statistic value descending) - * Note that if the number of features is < numTopFeatures, then this will - * select all features. + * Note that if the number of features is less than numTopFeatures, + * then this will select all features. */ @Since("1.3.0") class ChiSqSelector @Since("1.3.0") ( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala index 15b72205ac17a..aaecfa8d45dc0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala @@ -70,7 +70,7 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { } /** - * Java-friendly version of [[fit()]] + * Java-friendly version of `fit()`. */ @Since("1.4.0") def fit(sources: JavaRDD[Vector]): PCAModel = fit(sources.rdd) @@ -91,7 +91,7 @@ class PCAModel private[spark] ( * Transform a vector by computed Principal Components. * * @param vector vector to be transformed. - * Vector must be the same length as the source vectors given to [[PCA.fit()]]. + * Vector must be the same length as the source vectors given to `PCA.fit()`. * @return transformed vector. Vector will be of length k. */ @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index b7d6c6056803a..3e86c6c59c953 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -27,9 +27,8 @@ import org.apache.spark.rdd.RDD * Standardizes features by removing the mean and scaling to unit std using column summary * statistics on the samples in the training set. * - * The "unit std" is computed using the - * [[https://en.wikipedia.org/wiki/Standard_deviation#Corrected_sample_standard_deviation - * corrected sample standard deviation]], + * The "unit std" is computed using the corrected sample standard deviation + * (https://en.wikipedia.org/wiki/Standard_deviation#Corrected_sample_standard_deviation), * which is computed as the square root of the unbiased sample variance. * * @param withMean False by default. Centers the data with mean before scaling. It will build a diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 22e70278a6656..c5e34ffa4f2e5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -20,8 +20,8 @@ package org.apache.spark.mllib.tree.impurity import org.apache.spark.annotation.{DeveloperApi, Since} /** - * Class for calculating the - * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] + * Class for calculating the Gini impurity + * (http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity) * during multiclass classification. */ @Since("1.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala index 4d71d534a0774..c881c8ea50c09 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala @@ -45,7 +45,7 @@ trait Saveable { * - human-readable (JSON) model metadata to path/metadata/ * - Parquet formatted data to path/data/ * - * The model may be loaded using [[Loader.load]]. + * The model may be loaded using `Loader.load`. * * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. From 8c2ec443baaf84a7bc008c0fdcddc039a65ad948 Mon Sep 17 00:00:00 2001 From: Narine Kokhlikyan Date: Sat, 16 Jul 2016 16:56:16 -0700 Subject: [PATCH 0991/1470] [SPARK-16112][SPARKR] Programming guide for gapply/gapplyCollect ## What changes were proposed in this pull request? Updates programming guide for spark.gapply/spark.gapplyCollect. Similar to other examples I used `faithful` dataset to demonstrate gapply's functionality. Please, let me know if you prefer another example. ## How was this patch tested? Existing test cases in R Author: Narine Kokhlikyan Closes #14090 from NarineK/gapplyProgGuide. (cherry picked from commit 416730483643a0a92dbd6ae4ad07e80ceb3c5285) Signed-off-by: Shivaram Venkataraman --- docs/sparkr.md | 138 +++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 134 insertions(+), 4 deletions(-) diff --git a/docs/sparkr.md b/docs/sparkr.md index 9fda0ec0e6542..a5235b2bf66a4 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -272,11 +272,11 @@ In SparkR, we support several kinds of User-Defined Functions: ##### dapply Apply a function to each partition of a `SparkDataFrame`. The function to be applied to each partition of the `SparkDataFrame` -and should have only one parameter, to which a `data.frame` corresponds to each partition will be passed. The output of function -should be a `data.frame`. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match the R function's output. +and should have only one parameter, to which a `data.frame` corresponds to each partition will be passed. The output of function should be a `data.frame`. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match to [data types](#data-type-mapping-between-r-and-spark) of returned value.

    {% highlight r %} + # Convert waiting time from hours to seconds. # Note that we can apply UDF to DataFrame. schema <- structType(structField("eruptions", "double"), structField("waiting", "double"), @@ -295,8 +295,8 @@ head(collect(df1)) ##### dapplyCollect Like `dapply`, apply a function to each partition of a `SparkDataFrame` and collect the result back. The output of function -should be a `data.frame`. But, Schema is not required to be passed. Note that `dapplyCollect` only can be used if the -output of UDF run on all the partitions can fit in driver memory. +should be a `data.frame`. But, Schema is not required to be passed. Note that `dapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. +
    {% highlight r %} @@ -316,6 +316,136 @@ head(ldf, 3) {% endhighlight %}
    +#### Run a given function on a large dataset grouping by input column(s) and using `gapply` or `gapplyCollect` + +##### gapply +Apply a function to each group of a `SparkDataFrame`. The function is to be applied to each group of the `SparkDataFrame` and should have only two parameters: grouping key and R `data.frame` corresponding to +that key. The groups are chosen from `SparkDataFrame`s column(s). +The output of function should be a `data.frame`. Schema specifies the row format of the resulting +`SparkDataFrame`. It must represent R function's output schema on the basis of Spark data types. The column names of the returned `data.frame` are set by user. Below is the data type mapping between R +and Spark. + +#### Data type mapping between R and Spark + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    RSpark
    bytebyte
    integerinteger
    floatfloat
    doubledouble
    numericdouble
    characterstring
    stringstring
    binarybinary
    rawbinary
    logicalboolean
    POSIXcttimestamp
    POSIXlttimestamp
    Datedate
    arrayarray
    listarray
    envmap
    + +
    +{% highlight r %} + +# Determine six waiting times with the largest eruption time in minutes. +schema <- structType(structField("waiting", "double"), structField("max_eruption", "double")) +result <- gapply( + df, + "waiting", + function(key, x) { + y <- data.frame(key, max(x$eruptions)) + }, + schema) +head(collect(arrange(result, "max_eruption", decreasing = TRUE))) + +## waiting max_eruption +##1 64 5.100 +##2 69 5.067 +##3 71 5.033 +##4 87 5.000 +##5 63 4.933 +##6 89 4.900 +{% endhighlight %} +
    + +##### gapplyCollect +Like `gapply`, applies a function to each partition of a `SparkDataFrame` and collect the result back to R data.frame. The output of the function should be a `data.frame`. But, the schema is not required to be passed. Note that `gapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +
    +{% highlight r %} + +# Determine six waiting times with the largest eruption time in minutes. +result <- gapplyCollect( + df, + "waiting", + function(key, x) { + y <- data.frame(key, max(x$eruptions)) + colnames(y) <- c("waiting", "max_eruption") + y + }) +head(result[order(result$max_eruption, decreasing = TRUE), ]) + +## waiting max_eruption +##1 64 5.100 +##2 69 5.067 +##3 71 5.033 +##4 87 5.000 +##5 63 4.933 +##6 89 4.900 + +{% endhighlight %} +
    + #### Run local R functions distributed using `spark.lapply` ##### spark.lapply From c527e9ed4ae3d45e2df3c7bdb1a4d44afca652d7 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 16 Jul 2016 17:06:44 -0700 Subject: [PATCH 0992/1470] [SPARK-16507][SPARKR] Add a CRAN checker, fix Rd aliases ## What changes were proposed in this pull request? Add a check-cran.sh script that runs `R CMD check` as CRAN. Also fixes a number of issues pointed out by the check. These include - Updating `DESCRIPTION` to be appropriate - Adding a .Rbuildignore to ignore lintr, src-native, html that are non-standard files / dirs - Adding aliases to all S4 methods in DataFrame, Column, GroupedData etc. This is required as stated in https://cran.r-project.org/doc/manuals/r-release/R-exts.html#Documenting-S4-classes-and-methods - Other minor fixes ## How was this patch tested? SparkR unit tests, running the above mentioned script Author: Shivaram Venkataraman Closes #14173 from shivaram/sparkr-cran-changes. (cherry picked from commit c33e4b0d96d424568963c7e716c20f02949c72d1) Signed-off-by: Shivaram Venkataraman --- R/check-cran.sh | 52 ++++++ R/pkg/.Rbuildignore | 5 + R/pkg/DESCRIPTION | 8 +- R/pkg/NAMESPACE | 9 + R/pkg/R/DataFrame.R | 105 ++++++++++- R/pkg/R/SQLContext.R | 3 + R/pkg/R/WindowSpec.R | 6 + R/pkg/R/column.R | 27 ++- R/pkg/R/functions.R | 436 ++++++++++++++++++++++++++++++++++++++++++- R/pkg/R/generics.R | 8 +- R/pkg/R/group.R | 9 +- R/pkg/R/mllib.R | 4 +- R/pkg/R/schema.R | 16 +- R/pkg/R/stats.R | 6 + R/pkg/R/utils.R | 20 +- R/pkg/R/window.R | 4 + dev/.rat-excludes | 1 + 17 files changed, 676 insertions(+), 43 deletions(-) create mode 100755 R/check-cran.sh create mode 100644 R/pkg/.Rbuildignore diff --git a/R/check-cran.sh b/R/check-cran.sh new file mode 100755 index 0000000000000..b3a6860961c1e --- /dev/null +++ b/R/check-cran.sh @@ -0,0 +1,52 @@ +#!/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. +# + +set -o pipefail +set -e + +FWDIR="$(cd `dirname $0`; pwd)" +pushd $FWDIR > /dev/null + +if [ ! -z "$R_HOME" ] + then + R_SCRIPT_PATH="$R_HOME/bin" + else + # if system wide R_HOME is not found, then exit + if [ ! `command -v R` ]; then + echo "Cannot find 'R_HOME'. Please specify 'R_HOME' or make sure R is properly installed." + exit 1 + fi + R_SCRIPT_PATH="$(dirname $(which R))" +fi +echo "USING R_HOME = $R_HOME" + +# Build the latest docs +$FWDIR/create-docs.sh + +# Build a zip file containing the source package +"$R_SCRIPT_PATH/"R CMD build $FWDIR/pkg + +# Run check as-cran. +# TODO(shivaram): Remove the skip tests once we figure out the install mechanism + +VERSION=`grep Version $FWDIR/pkg/DESCRIPTION | awk '{print $NF}'` + +"$R_SCRIPT_PATH/"R CMD check --as-cran --no-tests SparkR_"$VERSION".tar.gz + +popd > /dev/null diff --git a/R/pkg/.Rbuildignore b/R/pkg/.Rbuildignore new file mode 100644 index 0000000000000..544d203a6dce6 --- /dev/null +++ b/R/pkg/.Rbuildignore @@ -0,0 +1,5 @@ +^.*\.Rproj$ +^\.Rproj\.user$ +^\.lintr$ +^src-native$ +^html$ diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 963a1bb5806a7..ac73d6c79891e 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,12 +1,10 @@ Package: SparkR Type: Package -Title: R frontend for Spark +Title: R Frontend for Apache Spark Version: 2.0.0 -Date: 2013-09-09 +Date: 2016-07-07 Author: The Apache Software Foundation Maintainer: Shivaram Venkataraman -Imports: - methods Depends: R (>= 3.0), methods, @@ -14,7 +12,7 @@ Suggests: testthat, e1071, survival -Description: R frontend for Spark +Description: The SparkR package provides an R frontend for Apache Spark. License: Apache License (== 2.0) Collate: 'schema.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index fe52905e0a850..1d74c6d95578f 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -343,3 +343,12 @@ export("partitionBy", export("windowPartitionBy", "windowOrderBy") + +S3method(print, jobj) +S3method(print, structField) +S3method(print, structType) +S3method(print, summary.GeneralizedLinearRegressionModel) +S3method(structField, character) +S3method(structField, jobj) +S3method(structType, jobj) +S3method(structType, structField) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ec09aab6f9698..92c10f1be1336 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -74,6 +74,7 @@ dataFrame <- function(sdf, isCached = FALSE) { #' @family SparkDataFrame functions #' @rdname printSchema #' @name printSchema +#' @aliases printSchema,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -99,6 +100,7 @@ setMethod("printSchema", #' @family SparkDataFrame functions #' @rdname schema #' @name schema +#' @aliases schema,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -121,6 +123,7 @@ setMethod("schema", #' @param x A SparkDataFrame #' @param extended Logical. If extended is FALSE, explain() only prints the physical plan. #' @family SparkDataFrame functions +#' @aliases explain,SparkDataFrame-method #' @rdname explain #' @name explain #' @export @@ -154,6 +157,7 @@ setMethod("explain", #' @family SparkDataFrame functions #' @rdname isLocal #' @name isLocal +#' @aliases isLocal,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -179,6 +183,7 @@ setMethod("isLocal", #' truncated and all cells will be aligned right #' #' @family SparkDataFrame functions +#' @aliases showDF,SparkDataFrame-method #' @rdname showDF #' @name showDF #' @export @@ -205,6 +210,7 @@ setMethod("showDF", #' #' @family SparkDataFrame functions #' @rdname show +#' @aliases show,SparkDataFrame-method #' @name show #' @export #' @examples @@ -233,6 +239,7 @@ setMethod("show", "SparkDataFrame", #' @family SparkDataFrame functions #' @rdname dtypes #' @name dtypes +#' @aliases dtypes,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -259,7 +266,7 @@ setMethod("dtypes", #' @family SparkDataFrame functions #' @rdname columns #' @name columns - +#' @aliases columns,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -280,6 +287,7 @@ setMethod("columns", #' @rdname columns #' @name names +#' @aliases names,SparkDataFrame-method #' @note names since 1.5.0 setMethod("names", signature(x = "SparkDataFrame"), @@ -288,6 +296,7 @@ setMethod("names", }) #' @rdname columns +#' @aliases names<-,SparkDataFrame-method #' @name names<- #' @note names<- since 1.5.0 setMethod("names<-", @@ -300,6 +309,7 @@ setMethod("names<-", }) #' @rdname columns +#' @aliases colnames,SparkDataFrame-method #' @name colnames #' @note colnames since 1.6.0 setMethod("colnames", @@ -309,6 +319,7 @@ setMethod("colnames", }) #' @rdname columns +#' @aliases colnames<-,SparkDataFrame-method #' @name colnames<- #' @note colnames<- since 1.6.0 setMethod("colnames<-", @@ -345,6 +356,7 @@ setMethod("colnames<-", #' @param x A SparkDataFrame #' @return value A character vector with the column types of the given SparkDataFrame #' @rdname coltypes +#' @aliases coltypes,SparkDataFrame-method #' @name coltypes #' @family SparkDataFrame functions #' @export @@ -400,6 +412,7 @@ setMethod("coltypes", #' to keep that column as-is. #' @rdname coltypes #' @name coltypes<- +#' @aliases coltypes<-,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ @@ -448,6 +461,7 @@ setMethod("coltypes<-", #' @family SparkDataFrame functions #' @rdname createOrReplaceTempView #' @name createOrReplaceTempView +#' @aliases createOrReplaceTempView,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ @@ -474,6 +488,7 @@ setMethod("createOrReplaceTempView", #' @seealso \link{createOrReplaceTempView} #' @rdname registerTempTable-deprecated #' @name registerTempTable +#' @aliases registerTempTable,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ @@ -503,6 +518,7 @@ setMethod("registerTempTable", #' @family SparkDataFrame functions #' @rdname insertInto #' @name insertInto +#' @aliases insertInto,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ @@ -529,6 +545,7 @@ setMethod("insertInto", #' @param x A SparkDataFrame #' #' @family SparkDataFrame functions +#' @aliases cache,SparkDataFrame-method #' @rdname cache #' @name cache #' @export @@ -559,6 +576,7 @@ setMethod("cache", #' @family SparkDataFrame functions #' @rdname persist #' @name persist +#' @aliases persist,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ @@ -586,6 +604,7 @@ setMethod("persist", #' #' @family SparkDataFrame functions #' @rdname unpersist-methods +#' @aliases unpersist,SparkDataFrame-method #' @name unpersist #' @export #' @examples @@ -622,6 +641,7 @@ setMethod("unpersist", #' @family SparkDataFrame functions #' @rdname repartition #' @name repartition +#' @aliases repartition,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -665,6 +685,7 @@ setMethod("repartition", #' #' @param x A SparkDataFrame #' @return A StringRRDD of JSON objects +#' @aliases toJSON,SparkDataFrame-method #' @noRd #' @examples #'\dontrun{ @@ -692,6 +713,7 @@ setMethod("toJSON", #' @family SparkDataFrame functions #' @rdname write.json #' @name write.json +#' @aliases write.json,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ @@ -717,6 +739,7 @@ setMethod("write.json", #' @param path The directory where the file is saved #' #' @family SparkDataFrame functions +#' @aliases write.orc,SparkDataFrame,character-method #' @rdname write.orc #' @name write.orc #' @export @@ -746,6 +769,7 @@ setMethod("write.orc", #' @family SparkDataFrame functions #' @rdname write.parquet #' @name write.parquet +#' @aliases write.parquet,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ @@ -765,6 +789,7 @@ setMethod("write.parquet", #' @rdname write.parquet #' @name saveAsParquetFile +#' @aliases saveAsParquetFile,SparkDataFrame,character-method #' @export #' @note saveAsParquetFile since 1.4.0 setMethod("saveAsParquetFile", @@ -784,6 +809,7 @@ setMethod("saveAsParquetFile", #' @param path The directory where the file is saved #' #' @family SparkDataFrame functions +#' @aliases write.text,SparkDataFrame,character-method #' @rdname write.text #' @name write.text #' @export @@ -809,6 +835,7 @@ setMethod("write.text", #' @param x A SparkDataFrame #' #' @family SparkDataFrame functions +#' @aliases distinct,SparkDataFrame-method #' @rdname distinct #' @name distinct #' @export @@ -829,6 +856,7 @@ setMethod("distinct", #' @rdname distinct #' @name unique +#' @aliases unique,SparkDataFrame-method #' @note unique since 1.5.0 setMethod("unique", signature(x = "SparkDataFrame"), @@ -846,6 +874,7 @@ setMethod("unique", #' @param seed Randomness seed value #' #' @family SparkDataFrame functions +#' @aliases sample,SparkDataFrame,logical,numeric-method #' @rdname sample #' @name sample #' @export @@ -874,6 +903,7 @@ setMethod("sample", }) #' @rdname sample +#' @aliases sample_frac,SparkDataFrame,logical,numeric-method #' @name sample_frac #' @note sample_frac since 1.4.0 setMethod("sample_frac", @@ -890,6 +920,7 @@ setMethod("sample_frac", #' @family SparkDataFrame functions #' @rdname nrow #' @name count +#' @aliases count,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -907,6 +938,7 @@ setMethod("count", #' @name nrow #' @rdname nrow +#' @aliases nrow,SparkDataFrame-method #' @note nrow since 1.5.0 setMethod("nrow", signature(x = "SparkDataFrame"), @@ -921,6 +953,7 @@ setMethod("nrow", #' @family SparkDataFrame functions #' @rdname ncol #' @name ncol +#' @aliases ncol,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -943,6 +976,7 @@ setMethod("ncol", #' #' @family SparkDataFrame functions #' @rdname dim +#' @aliases dim,SparkDataFrame-method #' @name dim #' @export #' @examples @@ -967,6 +1001,7 @@ setMethod("dim", #' #' @family SparkDataFrame functions #' @rdname collect +#' @aliases collect,SparkDataFrame-method #' @name collect #' @export #' @examples @@ -1040,6 +1075,7 @@ setMethod("collect", #' @family SparkDataFrame functions #' @rdname limit #' @name limit +#' @aliases limit,SparkDataFrame,numeric-method #' @export #' @examples #' \dontrun{ @@ -1061,6 +1097,7 @@ setMethod("limit", #' @family SparkDataFrame functions #' @rdname take #' @name take +#' @aliases take,SparkDataFrame,numeric-method #' @export #' @examples #'\dontrun{ @@ -1088,6 +1125,7 @@ setMethod("take", #' @return A data.frame #' #' @family SparkDataFrame functions +#' @aliases head,SparkDataFrame-method #' @rdname head #' @name head #' @export @@ -1111,6 +1149,7 @@ setMethod("head", #' @param x A SparkDataFrame #' #' @family SparkDataFrame functions +#' @aliases first,SparkDataFrame-method #' @rdname first #' @name first #' @export @@ -1161,6 +1200,7 @@ setMethod("toRDD", #' @param x a SparkDataFrame #' @return a GroupedData #' @family SparkDataFrame functions +#' @aliases groupBy,SparkDataFrame-method #' @rdname groupBy #' @name groupBy #' @export @@ -1188,6 +1228,7 @@ setMethod("groupBy", #' @rdname groupBy #' @name group_by +#' @aliases group_by,SparkDataFrame-method #' @note group_by since 1.4.0 setMethod("group_by", signature(x = "SparkDataFrame"), @@ -1201,7 +1242,8 @@ setMethod("group_by", #' #' @param x a SparkDataFrame #' @family SparkDataFrame functions -#' @rdname agg +#' @aliases agg,SparkDataFrame-method +#' @rdname summarize #' @name agg #' @export #' @note agg since 1.4.0 @@ -1211,8 +1253,9 @@ setMethod("agg", agg(groupBy(x), ...) }) -#' @rdname agg +#' @rdname summarize #' @name summarize +#' @aliases summarize,SparkDataFrame-method #' @note summarize since 1.4.0 setMethod("summarize", signature(x = "SparkDataFrame"), @@ -1251,6 +1294,7 @@ dapplyInternal <- function(x, func, schema) { #' It must match the output of func. #' @family SparkDataFrame functions #' @rdname dapply +#' @aliases dapply,SparkDataFrame,function,structType-method #' @name dapply #' @seealso \link{dapplyCollect} #' @export @@ -1289,7 +1333,7 @@ setMethod("dapply", #' dapplyCollect #' #' Apply a function to each partition of a SparkDataFrame and collect the result back -#’ to R as a data.frame. +#' to R as a data.frame. #' #' @param x A SparkDataFrame #' @param func A function to be applied to each partition of the SparkDataFrame. @@ -1298,6 +1342,7 @@ setMethod("dapply", #' The output of func should be a R data.frame. #' @family SparkDataFrame functions #' @rdname dapplyCollect +#' @aliases dapplyCollect,SparkDataFrame,function-method #' @name dapplyCollect #' @seealso \link{dapply} #' @export @@ -1353,6 +1398,7 @@ setMethod("dapplyCollect", #' output column with preferred output column name and corresponding data type. #' @return a SparkDataFrame #' @family SparkDataFrame functions +#' @aliases gapply,SparkDataFrame-method #' @rdname gapply #' @name gapply #' @seealso \link{gapplyCollect} @@ -1441,6 +1487,7 @@ setMethod("gapply", #' The output of `func` is a local R data.frame. #' @return a data.frame #' @family SparkDataFrame functions +#' @aliases gapplyCollect,SparkDataFrame-method #' @rdname gapplyCollect #' @name gapplyCollect #' @seealso \link{gapply} @@ -1587,6 +1634,7 @@ getColumn <- function(x, c) { #' @rdname select #' @name $ +#' @aliases $,SparkDataFrame-method #' @note $ since 1.4.0 setMethod("$", signature(x = "SparkDataFrame"), function(x, name) { @@ -1595,6 +1643,7 @@ setMethod("$", signature(x = "SparkDataFrame"), #' @rdname select #' @name $<- +#' @aliases $<-,SparkDataFrame-method #' @note $<- since 1.4.0 setMethod("$<-", signature(x = "SparkDataFrame"), function(x, name, value) { @@ -1613,6 +1662,7 @@ setClassUnion("numericOrcharacter", c("numeric", "character")) #' @rdname subset #' @name [[ +#' @aliases [[,SparkDataFrame,numericOrcharacter-method #' @note [[ since 1.4.0 setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"), function(x, i) { @@ -1625,6 +1675,7 @@ setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"), #' @rdname subset #' @name [ +#' @aliases [,SparkDataFrame-method #' @note [ since 1.4.0 setMethod("[", signature(x = "SparkDataFrame"), function(x, i, j, ..., drop = F) { @@ -1672,6 +1723,7 @@ setMethod("[", signature(x = "SparkDataFrame"), #' @return A new SparkDataFrame containing only the rows that meet the condition with selected columns #' @export #' @family SparkDataFrame functions +#' @aliases subset,SparkDataFrame-method #' @rdname subset #' @name subset #' @family subsetting functions @@ -1709,6 +1761,7 @@ setMethod("subset", signature(x = "SparkDataFrame"), #' @export #' @family SparkDataFrame functions #' @rdname select +#' @aliases select,SparkDataFrame,character-method #' @name select #' @family subsetting functions #' @examples @@ -1738,6 +1791,7 @@ setMethod("select", signature(x = "SparkDataFrame", col = "character"), #' @rdname select #' @export +#' @aliases select,SparkDataFrame,Column-method #' @note select(SparkDataFrame, Column) since 1.4.0 setMethod("select", signature(x = "SparkDataFrame", col = "Column"), function(x, col, ...) { @@ -1750,6 +1804,7 @@ setMethod("select", signature(x = "SparkDataFrame", col = "Column"), #' @rdname select #' @export +#' @aliases select,SparkDataFrame,list-method #' @note select(SparkDataFrame, list) since 1.4.0 setMethod("select", signature(x = "SparkDataFrame", col = "list"), @@ -1774,6 +1829,7 @@ setMethod("select", #' @param ... Additional expressions #' @return A SparkDataFrame #' @family SparkDataFrame functions +#' @aliases selectExpr,SparkDataFrame,character-method #' @rdname selectExpr #' @name selectExpr #' @export @@ -1803,6 +1859,7 @@ setMethod("selectExpr", #' @param col A Column expression. #' @return A SparkDataFrame with the new column added or the existing column replaced. #' @family SparkDataFrame functions +#' @aliases withColumn,SparkDataFrame,character,Column-method #' @rdname withColumn #' @name withColumn #' @seealso \link{rename} \link{mutate} @@ -1832,6 +1889,7 @@ setMethod("withColumn", #' @param col a named argument of the form name = col #' @return A new SparkDataFrame with the new columns added or replaced. #' @family SparkDataFrame functions +#' @aliases mutate,SparkDataFrame-method #' @rdname mutate #' @name mutate #' @seealso \link{rename} \link{withColumn} @@ -1907,6 +1965,7 @@ setMethod("mutate", #' @export #' @rdname mutate +#' @aliases transform,SparkDataFrame-method #' @name transform #' @note transform since 1.5.0 setMethod("transform", @@ -1926,6 +1985,7 @@ setMethod("transform", #' @family SparkDataFrame functions #' @rdname rename #' @name withColumnRenamed +#' @aliases withColumnRenamed,SparkDataFrame,character,character-method #' @seealso \link{mutate} #' @export #' @examples @@ -1952,6 +2012,7 @@ setMethod("withColumnRenamed", #' @param ... A named pair of the form new_column_name = existing_column #' @rdname rename #' @name rename +#' @aliases rename,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -1994,6 +2055,7 @@ setClassUnion("characterOrColumn", c("character", "Column")) #' a character vector is specified for col #' @return A SparkDataFrame where all elements are sorted. #' @family SparkDataFrame functions +#' @aliases arrange,SparkDataFrame,Column-method #' @rdname arrange #' @name arrange #' @export @@ -2021,6 +2083,7 @@ setMethod("arrange", #' @rdname arrange #' @name arrange +#' @aliases arrange,SparkDataFrame,character-method #' @export #' @note arrange(SparkDataFrame, character) since 1.4.0 setMethod("arrange", @@ -2054,6 +2117,7 @@ setMethod("arrange", #' @rdname arrange #' @name orderBy +#' @aliases orderBy,SparkDataFrame,characterOrColumn-method #' @export #' @note orderBy(SparkDataFrame, characterOrColumn) since 1.4.0 setMethod("orderBy", @@ -2071,6 +2135,7 @@ setMethod("orderBy", #' or a string containing a SQL statement #' @return A SparkDataFrame containing only the rows that meet the condition. #' @family SparkDataFrame functions +#' @aliases filter,SparkDataFrame,characterOrColumn-method #' @rdname filter #' @name filter #' @family subsetting functions @@ -2096,6 +2161,7 @@ setMethod("filter", #' @rdname filter #' @name where +#' @aliases where,SparkDataFrame,characterOrColumn-method #' @note where since 1.4.0 setMethod("where", signature(x = "SparkDataFrame", condition = "characterOrColumn"), @@ -2113,6 +2179,7 @@ setMethod("where", #' If the first argument contains a character vector, the followings are ignored. #' @return A SparkDataFrame with duplicate rows removed. #' @family SparkDataFrame functions +#' @aliases dropDuplicates,SparkDataFrame-method #' @rdname dropDuplicates #' @name dropDuplicates #' @export @@ -2159,6 +2226,7 @@ setMethod("dropDuplicates", #' 'right_outer', 'rightouter', 'right', and 'leftsemi'. The default joinType is "inner". #' @return A SparkDataFrame containing the result of the join operation. #' @family SparkDataFrame functions +#' @aliases join,SparkDataFrame,SparkDataFrame-method #' @rdname join #' @name join #' @seealso \link{merge} @@ -2218,6 +2286,7 @@ setMethod("join", #' outer join will be returned. If all.x and all.y are set to TRUE, a full #' outer join will be returned. #' @family SparkDataFrame functions +#' @aliases merge,SparkDataFrame,SparkDataFrame-method #' @rdname merge #' @seealso \link{join} #' @export @@ -2355,6 +2424,7 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { #' @family SparkDataFrame functions #' @rdname union #' @name union +#' @aliases union,SparkDataFrame,SparkDataFrame-method #' @seealso \link{rbind} #' @export #' @examples @@ -2376,6 +2446,7 @@ setMethod("union", #' unionAll is deprecated - use union instead #' @rdname union #' @name unionAll +#' @aliases unionAll,SparkDataFrame,SparkDataFrame-method #' @export #' @note unionAll since 1.4.0 setMethod("unionAll", @@ -2394,6 +2465,7 @@ setMethod("unionAll", #' @param ... Additional SparkDataFrame #' @return A SparkDataFrame containing the result of the union. #' @family SparkDataFrame functions +#' @aliases rbind,SparkDataFrame-method #' @rdname rbind #' @name rbind #' @seealso \link{union} @@ -2423,6 +2495,7 @@ setMethod("rbind", #' @param y A SparkDataFrame #' @return A SparkDataFrame containing the result of the intersect. #' @family SparkDataFrame functions +#' @aliases intersect,SparkDataFrame,SparkDataFrame-method #' @rdname intersect #' @name intersect #' @export @@ -2450,6 +2523,7 @@ setMethod("intersect", #' @param y A SparkDataFrame #' @return A SparkDataFrame containing the result of the except operation. #' @family SparkDataFrame functions +#' @aliases except,SparkDataFrame,SparkDataFrame-method #' @rdname except #' @name except #' @export @@ -2493,6 +2567,7 @@ setMethod("except", #' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) #' #' @family SparkDataFrame functions +#' @aliases write.df,SparkDataFrame,character-method #' @rdname write.df #' @name write.df #' @export @@ -2524,6 +2599,7 @@ setMethod("write.df", #' @rdname write.df #' @name saveDF +#' @aliases saveDF,SparkDataFrame,character-method #' @export #' @note saveDF since 1.4.0 setMethod("saveDF", @@ -2553,6 +2629,7 @@ setMethod("saveDF", #' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) #' #' @family SparkDataFrame functions +#' @aliases saveAsTable,SparkDataFrame,character-method #' @rdname saveAsTable #' @name saveAsTable #' @export @@ -2590,6 +2667,7 @@ setMethod("saveAsTable", #' @param ... Additional expressions #' @return A SparkDataFrame #' @family SparkDataFrame functions +#' @aliases describe,SparkDataFrame,character-method describe,SparkDataFrame,ANY-method #' @rdname summary #' @name describe #' @export @@ -2613,6 +2691,7 @@ setMethod("describe", #' @rdname summary #' @name describe +#' @aliases describe,SparkDataFrame-method #' @note describe(SparkDataFrame) since 1.4.0 setMethod("describe", signature(x = "SparkDataFrame"), @@ -2623,6 +2702,7 @@ setMethod("describe", #' @rdname summary #' @name summary +#' @aliases summary,SparkDataFrame-method #' @note summary(SparkDataFrame) since 1.5.0 setMethod("summary", signature(object = "SparkDataFrame"), @@ -2648,6 +2728,7 @@ setMethod("summary", #' #' @family SparkDataFrame functions #' @rdname nafunctions +#' @aliases dropna,SparkDataFrame-method #' @name dropna #' @export #' @examples @@ -2677,6 +2758,7 @@ setMethod("dropna", #' @rdname nafunctions #' @name na.omit +#' @aliases na.omit,SparkDataFrame-method #' @export #' @note na.omit since 1.5.0 setMethod("na.omit", @@ -2702,6 +2784,7 @@ setMethod("na.omit", #' #' @rdname nafunctions #' @name fillna +#' @aliases fillna,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -2765,6 +2848,7 @@ setMethod("fillna", #' @param x a SparkDataFrame #' @return a data.frame #' @family SparkDataFrame functions +#' @aliases as.data.frame,SparkDataFrame-method #' @rdname as.data.frame #' @examples \dontrun{ #' @@ -2786,6 +2870,7 @@ setMethod("as.data.frame", #' #' @family SparkDataFrame functions #' @rdname attach +#' @aliases attach,SparkDataFrame-method #' @param what (SparkDataFrame) The SparkDataFrame to attach #' @param pos (integer) Specify position in search() where to attach. #' @param name (character) Name to use for the attached SparkDataFrame. Names @@ -2816,6 +2901,7 @@ setMethod("attach", #' #' @rdname with #' @family SparkDataFrame functions +#' @aliases with,SparkDataFrame-method #' @param data (SparkDataFrame) SparkDataFrame to use for constructing an environment. #' @param expr (expression) Expression to evaluate. #' @param ... arguments to be passed to future methods. @@ -2839,6 +2925,7 @@ setMethod("with", #' #' @name str #' @rdname str +#' @aliases str,SparkDataFrame-method #' @family SparkDataFrame functions #' @param object a SparkDataFrame #' @examples \dontrun{ @@ -2920,6 +3007,7 @@ setMethod("str", #' @family SparkDataFrame functions #' @rdname drop #' @name drop +#' @aliases drop,SparkDataFrame-method #' @export #' @examples #'\dontrun{ @@ -2945,6 +3033,10 @@ setMethod("drop", }) # Expose base::drop +#' @name drop +#' @rdname drop +#' @aliases drop,ANY-method +#' @export setMethod("drop", signature(x = "ANY"), function(x) { @@ -2961,6 +3053,7 @@ setMethod("drop", #' @param colname the name of the column to build the histogram from. #' @return a data.frame with the histogram statistics, i.e., counts and centroids. #' @rdname histogram +#' @aliases histogram,SparkDataFrame,characterOrColumn-method #' @family SparkDataFrame functions #' @export #' @examples @@ -3020,7 +3113,7 @@ setMethod("histogram", # columns AND all of them have names 100 characters long (which is very unlikely), # AND they run 1 billion histograms, the probability of collision will roughly be # 1 in 4.4 x 10 ^ 96 - colname <- paste(base:::sample(c(letters, LETTERS), + colname <- paste(base::sample(c(letters, LETTERS), size = min(max(nchar(colnames(df))) + 1, 100), replace = TRUE), collapse = "") @@ -3095,6 +3188,7 @@ setMethod("histogram", #' @family SparkDataFrame functions #' @rdname write.jdbc #' @name write.jdbc +#' @aliases write.jdbc,SparkDataFrame,character,character-method #' @export #' @examples #'\dontrun{ @@ -3122,6 +3216,7 @@ setMethod("write.jdbc", #' @param seed A seed to use for random split #' #' @family SparkDataFrame functions +#' @aliases randomSplit,SparkDataFrame,numeric-method #' @rdname randomSplit #' @name randomSplit #' @export diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index d2ea1553c6a2b..a14bcd91b3eac 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -269,6 +269,9 @@ as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { createDataFrame(data, schema, samplingRatio) } +#' @rdname createDataFrame +#' @aliases as.DataFrame +#' @export as.DataFrame <- function(x, ...) { dispatchFunc("as.DataFrame(data, schema = NULL, samplingRatio = 1.0)", x, ...) } diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index e20d05decefc7..4746380096245 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -58,6 +58,7 @@ setMethod("show", "WindowSpec", #' @return a WindowSpec #' @rdname partitionBy #' @name partitionBy +#' @aliases partitionBy,WindowSpec-method #' @family windowspec_method #' @export #' @examples @@ -89,6 +90,7 @@ setMethod("partitionBy", #' @return a WindowSpec #' @rdname arrange #' @name orderBy +#' @aliases orderBy,WindowSpec,character-method #' @family windowspec_method #' @export #' @examples @@ -105,6 +107,7 @@ setMethod("orderBy", #' @rdname arrange #' @name orderBy +#' @aliases orderBy,WindowSpec,Column-method #' @export #' @note orderBy(WindowSpec, Column) since 2.0.0 setMethod("orderBy", @@ -131,6 +134,7 @@ setMethod("orderBy", #' The frame is unbounded if this is the maximum long value. #' @return a WindowSpec #' @rdname rowsBetween +#' @aliases rowsBetween,WindowSpec,numeric,numeric-method #' @name rowsBetween #' @family windowspec_method #' @export @@ -162,6 +166,7 @@ setMethod("rowsBetween", #' The frame is unbounded if this is the maximum long value. #' @return a WindowSpec #' @rdname rangeBetween +#' @aliases rangeBetween,WindowSpec,numeric,numeric-method #' @name rangeBetween #' @family windowspec_method #' @export @@ -187,6 +192,7 @@ setMethod("rangeBetween", #' #' @rdname over #' @name over +#' @aliases over,Column,WindowSpec-method #' @family colum_func #' @export #' @note over since 2.0.0 diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 1a65912d3aed1..0edb9d2ae5c45 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -44,6 +44,9 @@ setMethod("initialize", "Column", function(.Object, jc) { .Object }) +#' @rdname column +#' @name column +#' @aliases column,jobj-method setMethod("column", signature(x = "jobj"), function(x) { @@ -52,6 +55,7 @@ setMethod("column", #' @rdname show #' @name show +#' @aliases show,Column-method #' @export #' @note show(Column) since 1.4.0 setMethod("show", "Column", @@ -131,8 +135,12 @@ createMethods() #' #' Set a new name for a column #' +#' @param object Column to rename +#' @param data new name to use +#' #' @rdname alias #' @name alias +#' @aliases alias,Column-method #' @family colum_func #' @export #' @note alias since 1.4.0 @@ -153,6 +161,7 @@ setMethod("alias", #' @rdname substr #' @name substr #' @family colum_func +#' @aliases substr,Column-method #' #' @param start starting position #' @param stop ending position @@ -171,8 +180,9 @@ setMethod("substr", signature(x = "Column"), #' @rdname startsWith #' @name startsWith #' @family colum_func +#' @aliases startsWith,Column-method #' -#' @param x vector of character string whose “starts” are considered +#' @param x vector of character string whose "starts" are considered #' @param prefix character vector (often of length one) #' @note startsWith since 1.4.0 setMethod("startsWith", signature(x = "Column"), @@ -189,8 +199,9 @@ setMethod("startsWith", signature(x = "Column"), #' @rdname endsWith #' @name endsWith #' @family colum_func +#' @aliases endsWith,Column-method #' -#' @param x vector of character string whose “ends” are considered +#' @param x vector of character string whose "ends" are considered #' @param suffix character vector (often of length one) #' @note endsWith since 1.4.0 setMethod("endsWith", signature(x = "Column"), @@ -206,6 +217,7 @@ setMethod("endsWith", signature(x = "Column"), #' @rdname between #' @name between #' @family colum_func +#' @aliases between,Column-method #' #' @param bounds lower and upper bounds #' @note between since 1.5.0 @@ -224,10 +236,10 @@ setMethod("between", signature(x = "Column"), #' @rdname cast #' @name cast #' @family colum_func +#' @aliases cast,Column-method #' #' @examples \dontrun{ #' cast(df$age, "string") -#' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE)) #' } #' @note cast since 1.4.0 setMethod("cast", @@ -235,12 +247,8 @@ setMethod("cast", 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") + stop("dataType should be character") } }) @@ -248,7 +256,7 @@ setMethod("cast", #' #' @rdname match #' @name %in% -#' @aliases %in% +#' @aliases %in%,Column-method #' @return a matched values as a result of comparing with given values. #' @export #' @examples @@ -272,6 +280,7 @@ setMethod("%in%", #' @rdname otherwise #' @name otherwise #' @family colum_func +#' @aliases otherwise,Column-method #' @export #' @note otherwise since 1.5.0 setMethod("otherwise", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 52d46f9d76120..573c915a5c67a 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -27,6 +27,7 @@ NULL #' @rdname lit #' @name lit #' @export +#' @aliases lit,ANY-method #' @examples #' \dontrun{ #' lit(df$name) @@ -46,11 +47,14 @@ setMethod("lit", signature("ANY"), #' #' Computes the absolute value. #' +#' @param x Column to compute on. +#' #' @rdname abs #' @name abs #' @family normal_funcs #' @export #' @examples \dontrun{abs(df$c)} +#' @aliases abs,Column-method #' @note abs since 1.5.0 setMethod("abs", signature(x = "Column"), @@ -64,11 +68,14 @@ setMethod("abs", #' Computes the cosine inverse of the given value; the returned angle is in the range #' 0.0 through pi. #' +#' @param x Column to compute on. +#' #' @rdname acos #' @name acos #' @family math_funcs #' @export #' @examples \dontrun{acos(df$c)} +#' @aliases acos,Column-method #' @note acos since 1.5.0 setMethod("acos", signature(x = "Column"), @@ -82,10 +89,13 @@ setMethod("acos", #' Returns the approximate number of distinct items in a group. This is a column #' aggregate function. #' +#' @param x Column to compute on. +#' #' @rdname approxCountDistinct #' @name approxCountDistinct #' @return the approximate number of distinct items in a group. #' @export +#' @aliases approxCountDistinct,Column-method #' @examples \dontrun{approxCountDistinct(df$c)} #' @note approxCountDistinct(Column) since 1.4.0 setMethod("approxCountDistinct", @@ -100,10 +110,13 @@ setMethod("approxCountDistinct", #' Computes the numeric value of the first character of the string column, and returns the #' result as a int column. #' +#' @param x Column to compute on. +#' #' @rdname ascii #' @name ascii #' @family string_funcs #' @export +#' @aliases ascii,Column-method #' @examples \dontrun{\dontrun{ascii(df$c)}} #' @note ascii since 1.5.0 setMethod("ascii", @@ -118,10 +131,13 @@ setMethod("ascii", #' Computes the sine inverse of the given value; the returned angle is in the range #' -pi/2 through pi/2. #' +#' @param x Column to compute on. +#' #' @rdname asin #' @name asin #' @family math_funcs #' @export +#' @aliases asin,Column-method #' @examples \dontrun{asin(df$c)} #' @note asin since 1.5.0 setMethod("asin", @@ -135,10 +151,13 @@ setMethod("asin", #' #' Computes the tangent inverse of the given value. #' +#' @param x Column to compute on. +#' #' @rdname atan #' @name atan #' @family math_funcs #' @export +#' @aliases atan,Column-method #' @examples \dontrun{atan(df$c)} #' @note atan since 1.5.0 setMethod("atan", @@ -152,10 +171,13 @@ setMethod("atan", #' #' Aggregate function: returns the average of the values in a group. #' +#' @param x Column to compute on. +#' #' @rdname avg #' @name avg #' @family agg_funcs #' @export +#' @aliases avg,Column-method #' @examples \dontrun{avg(df$c)} #' @note avg since 1.4.0 setMethod("avg", @@ -170,10 +192,13 @@ setMethod("avg", #' Computes the BASE64 encoding of a binary column and returns it as a string column. #' This is the reverse of unbase64. #' +#' @param x Column to compute on. +#' #' @rdname base64 #' @name base64 #' @family string_funcs #' @export +#' @aliases base64,Column-method #' @examples \dontrun{base64(df$c)} #' @note base64 since 1.5.0 setMethod("base64", @@ -188,10 +213,13 @@ setMethod("base64", #' An expression that returns the string representation of the binary value of the given long #' column. For example, bin("12") returns "1100". #' +#' @param x Column to compute on. +#' #' @rdname bin #' @name bin #' @family math_funcs #' @export +#' @aliases bin,Column-method #' @examples \dontrun{bin(df$c)} #' @note bin since 1.5.0 setMethod("bin", @@ -205,10 +233,13 @@ setMethod("bin", #' #' Computes bitwise NOT. #' +#' @param x Column to compute on. +#' #' @rdname bitwiseNOT #' @name bitwiseNOT #' @family normal_funcs #' @export +#' @aliases bitwiseNOT,Column-method #' @examples \dontrun{bitwiseNOT(df$c)} #' @note bitwiseNOT since 1.5.0 setMethod("bitwiseNOT", @@ -222,10 +253,13 @@ setMethod("bitwiseNOT", #' #' Computes the cube-root of the given value. #' +#' @param x Column to compute on. +#' #' @rdname cbrt #' @name cbrt #' @family math_funcs #' @export +#' @aliases cbrt,Column-method #' @examples \dontrun{cbrt(df$c)} #' @note cbrt since 1.4.0 setMethod("cbrt", @@ -239,10 +273,13 @@ setMethod("cbrt", #' #' Computes the ceiling of the given value. #' +#' @param x Column to compute on. +#' #' @rdname ceil #' @name ceil #' @family math_funcs #' @export +#' @aliases ceil,Column-method #' @examples \dontrun{ceil(df$c)} #' @note ceil since 1.5.0 setMethod("ceil", @@ -263,11 +300,14 @@ col <- function(x) { #' Returns a Column based on the given column name #' #' Returns a Column based on the given column name. +# +#' @param x Character column name. #' #' @rdname column #' @name column #' @family normal_funcs #' @export +#' @aliases column,character-method #' @examples \dontrun{column(df)} #' @note column since 1.6.0 setMethod("column", @@ -279,10 +319,13 @@ setMethod("column", #' #' Computes the Pearson Correlation Coefficient for two Columns. #' +#' @param x Column to compute on. +#' #' @rdname corr #' @name corr #' @family math_funcs #' @export +#' @aliases corr,Column-method #' @examples \dontrun{corr(df$c, df$d)} #' @note corr since 1.6.0 setMethod("corr", signature(x = "Column"), @@ -296,10 +339,13 @@ setMethod("corr", signature(x = "Column"), #' #' Compute the sample covariance between two expressions. #' +#' @param x Column to compute on. +#' #' @rdname cov #' @name cov #' @family math_funcs #' @export +#' @aliases cov,characterOrColumn-method #' @examples #' \dontrun{ #' cov(df$c, df$d) @@ -315,7 +361,11 @@ setMethod("cov", signature(x = "characterOrColumn"), }) #' @rdname cov +#' +#' @param col1 First column to compute cov_samp. +#' @param col2 Second column to compute cov_samp. #' @name covar_samp +#' @aliases covar_samp,characterOrColumn,characterOrColumn-method #' @note covar_samp since 2.0.0 setMethod("covar_samp", signature(col1 = "characterOrColumn", col2 = "characterOrColumn"), function(col1, col2) { @@ -332,10 +382,14 @@ setMethod("covar_samp", signature(col1 = "characterOrColumn", col2 = "characterO #' #' Compute the population covariance between two expressions. #' +#' @param col1 First column to compute cov_pop. +#' @param col2 Second column to compute cov_pop. +#' #' @rdname covar_pop #' @name covar_pop #' @family math_funcs #' @export +#' @aliases covar_pop,characterOrColumn,characterOrColumn-method #' @examples #' \dontrun{ #' covar_pop(df$c, df$d) @@ -357,9 +411,12 @@ setMethod("covar_pop", signature(col1 = "characterOrColumn", col2 = "characterOr #' #' Computes the cosine of the given value. #' +#' @param x Column to compute on. +#' #' @rdname cos #' @name cos #' @family math_funcs +#' @aliases cos,Column-method #' @export #' @examples \dontrun{cos(df$c)} #' @note cos since 1.5.0 @@ -374,9 +431,12 @@ setMethod("cos", #' #' Computes the hyperbolic cosine of the given value. #' +#' @param x Column to compute on. +#' #' @rdname cosh #' @name cosh #' @family math_funcs +#' @aliases cosh,Column-method #' @export #' @examples \dontrun{cosh(df$c)} #' @note cosh since 1.5.0 @@ -391,9 +451,12 @@ setMethod("cosh", #' #' Returns the number of items in a group. This is a column aggregate function. #' -#' @rdname count +#' @param x Column to compute on. +#' +#' @rdname nrow #' @name count #' @family agg_funcs +#' @aliases count,Column-method #' @export #' @examples \dontrun{count(df$c)} #' @note count since 1.4.0 @@ -409,9 +472,12 @@ setMethod("count", #' Calculates the cyclic redundancy check value (CRC32) of a binary column and #' returns the value as a bigint. #' +#' @param x Column to compute on. +#' #' @rdname crc32 #' @name crc32 #' @family misc_funcs +#' @aliases crc32,Column-method #' @export #' @examples \dontrun{crc32(df$c)} #' @note crc32 since 1.5.0 @@ -426,9 +492,12 @@ setMethod("crc32", #' #' Calculates the hash code of given columns, and returns the result as a int column. #' +#' @param x Column to compute on. +#' #' @rdname hash #' @name hash #' @family misc_funcs +#' @aliases hash,Column-method #' @export #' @examples \dontrun{hash(df$c)} #' @note hash since 2.0.0 @@ -447,9 +516,12 @@ setMethod("hash", #' #' Extracts the day of the month as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname dayofmonth #' @name dayofmonth #' @family datetime_funcs +#' @aliases dayofmonth,Column-method #' @export #' @examples \dontrun{dayofmonth(df$c)} #' @note dayofmonth since 1.5.0 @@ -464,9 +536,12 @@ setMethod("dayofmonth", #' #' Extracts the day of the year as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname dayofyear #' @name dayofyear #' @family datetime_funcs +#' @aliases dayofyear,Column-method #' @export #' @examples \dontrun{dayofyear(df$c)} #' @note dayofyear since 1.5.0 @@ -482,9 +557,13 @@ setMethod("dayofyear", #' Computes the first argument into a string from a binary using the provided character set #' (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). #' +#' @param x Column to compute on. +#' @param charset Character set to use +#' #' @rdname decode #' @name decode #' @family string_funcs +#' @aliases decode,Column,character-method #' @export #' @examples \dontrun{decode(df$c, "UTF-8")} #' @note decode since 1.6.0 @@ -500,9 +579,13 @@ setMethod("decode", #' Computes the first argument into a binary from a string using the provided character set #' (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). #' +#' @param x Column to compute on. +#' @param charset Character set to use +#' #' @rdname encode #' @name encode #' @family string_funcs +#' @aliases encode,Column,character-method #' @export #' @examples \dontrun{encode(df$c, "UTF-8")} #' @note encode since 1.6.0 @@ -517,9 +600,12 @@ setMethod("encode", #' #' Computes the exponential of the given value. #' +#' @param x Column to compute on. +#' #' @rdname exp #' @name exp #' @family math_funcs +#' @aliases exp,Column-method #' @export #' @examples \dontrun{exp(df$c)} #' @note exp since 1.5.0 @@ -534,8 +620,11 @@ setMethod("exp", #' #' Computes the exponential of the given value minus one. #' +#' @param x Column to compute on. +#' #' @rdname expm1 #' @name expm1 +#' @aliases expm1,Column-method #' @family math_funcs #' @export #' @examples \dontrun{expm1(df$c)} @@ -551,8 +640,11 @@ setMethod("expm1", #' #' Computes the factorial of the given value. #' +#' @param x Column to compute on. +#' #' @rdname factorial #' @name factorial +#' @aliases factorial,Column-method #' @family math_funcs #' @export #' @examples \dontrun{factorial(df$c)} @@ -571,8 +663,11 @@ setMethod("factorial", #' The function by default returns the first values it sees. It will return the first non-missing #' value it sees when na.rm is set to true. If all values are missing, then NA is returned. #' +#' @param x Column to compute on. +#' #' @rdname first #' @name first +#' @aliases first,characterOrColumn-method #' @family agg_funcs #' @export #' @examples @@ -597,8 +692,11 @@ setMethod("first", #' #' Computes the floor of the given value. #' +#' @param x Column to compute on. +#' #' @rdname floor #' @name floor +#' @aliases floor,Column-method #' @family math_funcs #' @export #' @examples \dontrun{floor(df$c)} @@ -614,9 +712,12 @@ setMethod("floor", #' #' Computes hex value of the given column. #' +#' @param x Column to compute on. +#' #' @rdname hex #' @name hex #' @family math_funcs +#' @aliases hex,Column-method #' @export #' @examples \dontrun{hex(df$c)} #' @note hex since 1.5.0 @@ -631,8 +732,11 @@ setMethod("hex", #' #' Extracts the hours as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname hour #' @name hour +#' @aliases hour,Column-method #' @family datetime_funcs #' @export #' @examples \dontrun{hour(df$c)} @@ -651,9 +755,12 @@ setMethod("hour", #' #' For example, "hello world" will become "Hello World". #' +#' @param x Column to compute on. +#' #' @rdname initcap #' @name initcap #' @family string_funcs +#' @aliases initcap,Column-method #' @export #' @examples \dontrun{initcap(df$c)} #' @note initcap since 1.5.0 @@ -668,9 +775,12 @@ setMethod("initcap", #' #' Return true if the column is NaN, alias for \link{isnan} #' +#' @param x Column to compute on. +#' #' @rdname is.nan #' @name is.nan #' @family normal_funcs +#' @aliases is.nan,Column-method #' @export #' @examples #' \dontrun{ @@ -686,6 +796,7 @@ setMethod("is.nan", #' @rdname is.nan #' @name isnan +#' @aliases isnan,Column-method #' @note isnan since 2.0.0 setMethod("isnan", signature(x = "Column"), @@ -698,8 +809,11 @@ setMethod("isnan", #' #' Aggregate function: returns the kurtosis of the values in a group. #' +#' @param x Column to compute on. +#' #' @rdname kurtosis #' @name kurtosis +#' @aliases kurtosis,Column-method #' @family agg_funcs #' @export #' @examples \dontrun{kurtosis(df$c)} @@ -718,8 +832,11 @@ setMethod("kurtosis", #' The function by default returns the last values it sees. It will return the last non-missing #' value it sees when na.rm is set to true. If all values are missing, then NA is returned. #' +#' @param x Column to compute on. +#' #' @rdname last #' @name last +#' @aliases last,characterOrColumn-method #' @family agg_funcs #' @export #' @examples @@ -746,8 +863,11 @@ setMethod("last", #' For example, input "2015-07-27" returns "2015-07-31" since July 31 is the last day of the #' month in July 2015. #' +#' @param x Column to compute on. +#' #' @rdname last_day #' @name last_day +#' @aliases last_day,Column-method #' @family datetime_funcs #' @export #' @examples \dontrun{last_day(df$c)} @@ -763,8 +883,11 @@ setMethod("last_day", #' #' Computes the length of a given string or binary column. #' +#' @param x Column to compute on. +#' #' @rdname length #' @name length +#' @aliases length,Column-method #' @family string_funcs #' @export #' @examples \dontrun{length(df$c)} @@ -780,8 +903,11 @@ setMethod("length", #' #' Computes the natural logarithm of the given value. #' +#' @param x Column to compute on. +#' #' @rdname log #' @name log +#' @aliases log,Column-method #' @family math_funcs #' @export #' @examples \dontrun{log(df$c)} @@ -797,9 +923,12 @@ setMethod("log", #' #' Computes the logarithm of the given value in base 10. #' +#' @param x Column to compute on. +#' #' @rdname log10 #' @name log10 #' @family math_funcs +#' @aliases log10,Column-method #' @export #' @examples \dontrun{log10(df$c)} #' @note log10 since 1.5.0 @@ -814,9 +943,12 @@ setMethod("log10", #' #' Computes the natural logarithm of the given value plus one. #' +#' @param x Column to compute on. +#' #' @rdname log1p #' @name log1p #' @family math_funcs +#' @aliases log1p,Column-method #' @export #' @examples \dontrun{log1p(df$c)} #' @note log1p since 1.5.0 @@ -831,9 +963,12 @@ setMethod("log1p", #' #' Computes the logarithm of the given column in base 2. #' +#' @param x Column to compute on. +#' #' @rdname log2 #' @name log2 #' @family math_funcs +#' @aliases log2,Column-method #' @export #' @examples \dontrun{log2(df$c)} #' @note log2 since 1.5.0 @@ -848,9 +983,12 @@ setMethod("log2", #' #' Converts a string column to lower case. #' +#' @param x Column to compute on. +#' #' @rdname lower #' @name lower #' @family string_funcs +#' @aliases lower,Column-method #' @export #' @examples \dontrun{lower(df$c)} #' @note lower since 1.4.0 @@ -865,9 +1003,12 @@ setMethod("lower", #' #' Trim the spaces from left end for the specified string value. #' +#' @param x Column to compute on. +#' #' @rdname ltrim #' @name ltrim #' @family string_funcs +#' @aliases ltrim,Column-method #' @export #' @examples \dontrun{ltrim(df$c)} #' @note ltrim since 1.5.0 @@ -882,9 +1023,12 @@ setMethod("ltrim", #' #' Aggregate function: returns the maximum value of the expression in a group. #' +#' @param x Column to compute on. +#' #' @rdname max #' @name max #' @family agg_funcs +#' @aliases max,Column-method #' @export #' @examples \dontrun{max(df$c)} #' @note max since 1.5.0 @@ -900,9 +1044,12 @@ setMethod("max", #' Calculates the MD5 digest of a binary column and returns the value #' as a 32 character hex string. #' +#' @param x Column to compute on. +#' #' @rdname md5 #' @name md5 #' @family misc_funcs +#' @aliases md5,Column-method #' @export #' @examples \dontrun{md5(df$c)} #' @note md5 since 1.5.0 @@ -918,9 +1065,12 @@ setMethod("md5", #' Aggregate function: returns the average of the values in a group. #' Alias for avg. #' +#' @param x Column to compute on. +#' #' @rdname mean #' @name mean #' @family agg_funcs +#' @aliases mean,Column-method #' @export #' @examples \dontrun{mean(df$c)} #' @note mean since 1.5.0 @@ -935,8 +1085,11 @@ setMethod("mean", #' #' Aggregate function: returns the minimum value of the expression in a group. #' +#' @param x Column to compute on. +#' #' @rdname min #' @name min +#' @aliases min,Column-method #' @family agg_funcs #' @export #' @examples \dontrun{min(df$c)} @@ -952,8 +1105,11 @@ setMethod("min", #' #' Extracts the minutes as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname minute #' @name minute +#' @aliases minute,Column-method #' @family datetime_funcs #' @export #' @examples \dontrun{minute(df$c)} @@ -981,6 +1137,7 @@ setMethod("minute", #' This is equivalent to the MONOTONICALLY_INCREASING_ID function in SQL. #' #' @rdname monotonically_increasing_id +#' @aliases monotonically_increasing_id,missing-method #' @name monotonically_increasing_id #' @family misc_funcs #' @export @@ -996,8 +1153,11 @@ setMethod("monotonically_increasing_id", #' #' Extracts the month as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname month #' @name month +#' @aliases month,Column-method #' @family datetime_funcs #' @export #' @examples \dontrun{month(df$c)} @@ -1013,9 +1173,12 @@ setMethod("month", #' #' Unary minus, i.e. negate the expression. #' +#' @param x Column to compute on. +#' #' @rdname negate #' @name negate #' @family normal_funcs +#' @aliases negate,Column-method #' @export #' @examples \dontrun{negate(df$c)} #' @note negate since 1.5.0 @@ -1030,9 +1193,12 @@ setMethod("negate", #' #' Extracts the quarter as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname quarter #' @name quarter #' @family datetime_funcs +#' @aliases quarter,Column-method #' @export #' @examples \dontrun{quarter(df$c)} #' @note quarter since 1.5.0 @@ -1047,9 +1213,12 @@ setMethod("quarter", #' #' Reverses the string column and returns it as a new string column. #' +#' @param x Column to compute on. +#' #' @rdname reverse #' @name reverse #' @family string_funcs +#' @aliases reverse,Column-method #' @export #' @examples \dontrun{reverse(df$c)} #' @note reverse since 1.5.0 @@ -1065,9 +1234,12 @@ setMethod("reverse", #' Returns the double value that is closest in value to the argument and #' is equal to a mathematical integer. #' +#' @param x Column to compute on. +#' #' @rdname rint #' @name rint #' @family math_funcs +#' @aliases rint,Column-method #' @export #' @examples \dontrun{rint(df$c)} #' @note rint since 1.5.0 @@ -1082,9 +1254,12 @@ setMethod("rint", #' #' Returns the value of the column `e` rounded to 0 decimal places using HALF_UP rounding mode. #' +#' @param x Column to compute on. +#' #' @rdname round #' @name round #' @family math_funcs +#' @aliases round,Column-method #' @export #' @examples \dontrun{round(df$c)} #' @note round since 1.5.0 @@ -1102,9 +1277,12 @@ setMethod("round", #' Also known as Gaussian rounding or bankers' rounding that rounds to the nearest even number. #' bround(2.5, 0) = 2, bround(3.5, 0) = 4. #' +#' @param x Column to compute on. +#' #' @rdname bround #' @name bround #' @family math_funcs +#' @aliases bround,Column-method #' @export #' @examples \dontrun{bround(df$c, 0)} #' @note bround since 2.0.0 @@ -1120,9 +1298,12 @@ setMethod("bround", #' #' Trim the spaces from right end for the specified string value. #' +#' @param x Column to compute on. +#' #' @rdname rtrim #' @name rtrim #' @family string_funcs +#' @aliases rtrim,Column-method #' @export #' @examples \dontrun{rtrim(df$c)} #' @note rtrim since 1.5.0 @@ -1137,9 +1318,12 @@ setMethod("rtrim", #' #' Aggregate function: alias for \link{stddev_samp} #' +#' @param x Column to compute on. +#' #' @rdname sd #' @name sd #' @family agg_funcs +#' @aliases sd,Column-method #' @seealso \link{stddev_pop}, \link{stddev_samp} #' @export #' @examples @@ -1160,9 +1344,12 @@ setMethod("sd", #' #' Extracts the seconds as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname second #' @name second #' @family datetime_funcs +#' @aliases second,Column-method #' @export #' @examples \dontrun{second(df$c)} #' @note second since 1.5.0 @@ -1178,9 +1365,12 @@ setMethod("second", #' Calculates the SHA-1 digest of a binary column and returns the value #' as a 40 character hex string. #' +#' @param x Column to compute on. +#' #' @rdname sha1 #' @name sha1 #' @family misc_funcs +#' @aliases sha1,Column-method #' @export #' @examples \dontrun{sha1(df$c)} #' @note sha1 since 1.5.0 @@ -1195,8 +1385,11 @@ setMethod("sha1", #' #' Computes the signum of the given value. #' +#' @param x Column to compute on. +#' #' @rdname sign #' @name signum +#' @aliases signum,Column-method #' @family math_funcs #' @export #' @examples \dontrun{signum(df$c)} @@ -1212,9 +1405,12 @@ setMethod("signum", #' #' Computes the sine of the given value. #' +#' @param x Column to compute on. +#' #' @rdname sin #' @name sin #' @family math_funcs +#' @aliases sin,Column-method #' @export #' @examples \dontrun{sin(df$c)} #' @note sin since 1.5.0 @@ -1229,9 +1425,12 @@ setMethod("sin", #' #' Computes the hyperbolic sine of the given value. #' +#' @param x Column to compute on. +#' #' @rdname sinh #' @name sinh #' @family math_funcs +#' @aliases sinh,Column-method #' @export #' @examples \dontrun{sinh(df$c)} #' @note sinh since 1.5.0 @@ -1246,9 +1445,12 @@ setMethod("sinh", #' #' Aggregate function: returns the skewness of the values in a group. #' +#' @param x Column to compute on. +#' #' @rdname skewness #' @name skewness #' @family agg_funcs +#' @aliases skewness,Column-method #' @export #' @examples \dontrun{skewness(df$c)} #' @note skewness since 1.6.0 @@ -1263,9 +1465,12 @@ setMethod("skewness", #' #' Return the soundex code for the specified expression. #' +#' @param x Column to compute on. +#' #' @rdname soundex #' @name soundex #' @family string_funcs +#' @aliases soundex,Column-method #' @export #' @examples \dontrun{soundex(df$c)} #' @note soundex since 1.5.0 @@ -1286,6 +1491,7 @@ setMethod("soundex", #' #' @rdname spark_partition_id #' @name spark_partition_id +#' @aliases spark_partition_id,missing-method #' @export #' @examples #' \dontrun{select(df, spark_partition_id())} @@ -1298,6 +1504,7 @@ setMethod("spark_partition_id", }) #' @rdname sd +#' @aliases stddev,Column-method #' @name stddev #' @note stddev since 1.6.0 setMethod("stddev", @@ -1311,9 +1518,12 @@ setMethod("stddev", #' #' Aggregate function: returns the population standard deviation of the expression in a group. #' +#' @param x Column to compute on. +#' #' @rdname stddev_pop #' @name stddev_pop #' @family agg_funcs +#' @aliases stddev_pop,Column-method #' @seealso \link{sd}, \link{stddev_samp} #' @export #' @examples \dontrun{stddev_pop(df$c)} @@ -1329,9 +1539,12 @@ setMethod("stddev_pop", #' #' Aggregate function: returns the unbiased sample standard deviation of the expression in a group. #' +#' @param x Column to compute on. +#' #' @rdname stddev_samp #' @name stddev_samp #' @family agg_funcs +#' @aliases stddev_samp,Column-method #' @seealso \link{stddev_pop}, \link{sd} #' @export #' @examples \dontrun{stddev_samp(df$c)} @@ -1347,9 +1560,12 @@ setMethod("stddev_samp", #' #' Creates a new struct column that composes multiple input columns. #' +#' @param x Column to compute on. +#' #' @rdname struct #' @name struct #' @family normal_funcs +#' @aliases struct,characterOrColumn-method #' @export #' @examples #' \dontrun{ @@ -1373,9 +1589,12 @@ setMethod("struct", #' #' Computes the square root of the specified float value. #' +#' @param x Column to compute on. +#' #' @rdname sqrt #' @name sqrt #' @family math_funcs +#' @aliases sqrt,Column-method #' @export #' @examples \dontrun{sqrt(df$c)} #' @note sqrt since 1.5.0 @@ -1390,9 +1609,12 @@ setMethod("sqrt", #' #' Aggregate function: returns the sum of all values in the expression. #' +#' @param x Column to compute on. +#' #' @rdname sum #' @name sum #' @family agg_funcs +#' @aliases sum,Column-method #' @export #' @examples \dontrun{sum(df$c)} #' @note sum since 1.5.0 @@ -1407,9 +1629,12 @@ setMethod("sum", #' #' Aggregate function: returns the sum of distinct values in the expression. #' +#' @param x Column to compute on. +#' #' @rdname sumDistinct #' @name sumDistinct #' @family agg_funcs +#' @aliases sumDistinct,Column-method #' @export #' @examples \dontrun{sumDistinct(df$c)} #' @note sumDistinct since 1.4.0 @@ -1424,9 +1649,12 @@ setMethod("sumDistinct", #' #' Computes the tangent of the given value. #' +#' @param x Column to compute on. +#' #' @rdname tan #' @name tan #' @family math_funcs +#' @aliases tan,Column-method #' @export #' @examples \dontrun{tan(df$c)} #' @note tan since 1.5.0 @@ -1441,9 +1669,12 @@ setMethod("tan", #' #' Computes the hyperbolic tangent of the given value. #' +#' @param x Column to compute on. +#' #' @rdname tanh #' @name tanh #' @family math_funcs +#' @aliases tanh,Column-method #' @export #' @examples \dontrun{tanh(df$c)} #' @note tanh since 1.5.0 @@ -1458,9 +1689,12 @@ setMethod("tanh", #' #' Converts an angle measured in radians to an approximately equivalent angle measured in degrees. #' +#' @param x Column to compute on. +#' #' @rdname toDegrees #' @name toDegrees #' @family math_funcs +#' @aliases toDegrees,Column-method #' @export #' @examples \dontrun{toDegrees(df$c)} #' @note toDegrees since 1.4.0 @@ -1475,9 +1709,12 @@ setMethod("toDegrees", #' #' Converts an angle measured in degrees to an approximately equivalent angle measured in radians. #' +#' @param x Column to compute on. +#' #' @rdname toRadians #' @name toRadians #' @family math_funcs +#' @aliases toRadians,Column-method #' @export #' @examples \dontrun{toRadians(df$c)} #' @note toRadians since 1.4.0 @@ -1492,9 +1729,12 @@ setMethod("toRadians", #' #' Converts the column into DateType. #' +#' @param x Column to compute on. +#' #' @rdname to_date #' @name to_date #' @family datetime_funcs +#' @aliases to_date,Column-method #' @export #' @examples \dontrun{to_date(df$c)} #' @note to_date since 1.5.0 @@ -1509,9 +1749,12 @@ setMethod("to_date", #' #' Trim the spaces from both ends for the specified string column. #' +#' @param x Column to compute on. +#' #' @rdname trim #' @name trim #' @family string_funcs +#' @aliases trim,Column-method #' @export #' @examples \dontrun{trim(df$c)} #' @note trim since 1.5.0 @@ -1527,9 +1770,12 @@ setMethod("trim", #' Decodes a BASE64 encoded string column and returns it as a binary column. #' This is the reverse of base64. #' +#' @param x Column to compute on. +#' #' @rdname unbase64 #' @name unbase64 #' @family string_funcs +#' @aliases unbase64,Column-method #' @export #' @examples \dontrun{unbase64(df$c)} #' @note unbase64 since 1.5.0 @@ -1545,9 +1791,12 @@ setMethod("unbase64", #' Inverse of hex. Interprets each pair of characters as a hexadecimal number #' and converts to the byte representation of number. #' +#' @param x Column to compute on. +#' #' @rdname unhex #' @name unhex #' @family math_funcs +#' @aliases unhex,Column-method #' @export #' @examples \dontrun{unhex(df$c)} #' @note unhex since 1.5.0 @@ -1562,9 +1811,12 @@ setMethod("unhex", #' #' Converts a string column to upper case. #' +#' @param x Column to compute on. +#' #' @rdname upper #' @name upper #' @family string_funcs +#' @aliases upper,Column-method #' @export #' @examples \dontrun{upper(df$c)} #' @note upper since 1.4.0 @@ -1579,9 +1831,12 @@ setMethod("upper", #' #' Aggregate function: alias for \link{var_samp}. #' +#' @param x Column to compute on. +#' #' @rdname var #' @name var #' @family agg_funcs +#' @aliases var,Column-method #' @seealso \link{var_pop}, \link{var_samp} #' @export #' @examples @@ -1599,6 +1854,7 @@ setMethod("var", }) #' @rdname var +#' @aliases variance,Column-method #' @name variance #' @note variance since 1.6.0 setMethod("variance", @@ -1612,9 +1868,12 @@ setMethod("variance", #' #' Aggregate function: returns the population variance of the values in a group. #' +#' @param x Column to compute on. +#' #' @rdname var_pop #' @name var_pop #' @family agg_funcs +#' @aliases var_pop,Column-method #' @seealso \link{var}, \link{var_samp} #' @export #' @examples \dontrun{var_pop(df$c)} @@ -1630,8 +1889,11 @@ setMethod("var_pop", #' #' Aggregate function: returns the unbiased variance of the values in a group. #' +#' @param x Column to compute on. +#' #' @rdname var_samp #' @name var_samp +#' @aliases var_samp,Column-method #' @family agg_funcs #' @seealso \link{var_pop}, \link{var} #' @export @@ -1648,8 +1910,11 @@ setMethod("var_samp", #' #' Extracts the week number as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname weekofyear #' @name weekofyear +#' @aliases weekofyear,Column-method #' @family datetime_funcs #' @export #' @examples \dontrun{weekofyear(df$c)} @@ -1665,9 +1930,12 @@ setMethod("weekofyear", #' #' Extracts the year as an integer from a given date/timestamp/string. #' +#' @param x Column to compute on. +#' #' @rdname year #' @name year #' @family datetime_funcs +#' @aliases year,Column-method #' @export #' @examples \dontrun{year(df$c)} #' @note year since 1.5.0 @@ -1682,10 +1950,14 @@ setMethod("year", #' #' Returns the angle theta from the conversion of rectangular coordinates (x, y) to #' polar coordinates (r, theta). +# +#' @param x Column to compute on. +#' @param y Column to compute on. #' #' @rdname atan2 #' @name atan2 #' @family math_funcs +#' @aliases atan2,Column-method #' @export #' @examples \dontrun{atan2(df$c, x)} #' @note atan2 since 1.5.0 @@ -1702,8 +1974,12 @@ setMethod("atan2", signature(y = "Column"), #' #' Returns the number of days from `start` to `end`. #' +#' @param x start Column to use. +#' @param y end Column to use. +#' #' @rdname datediff #' @name datediff +#' @aliases datediff,Column-method #' @family datetime_funcs #' @export #' @examples \dontrun{datediff(df$c, x)} @@ -1720,10 +1996,14 @@ setMethod("datediff", signature(y = "Column"), #' hypot #' #' Computes "sqrt(a^2 + b^2)" without intermediate overflow or underflow. +# +#' @param x Column to compute on. +#' @param y Column to compute on. #' #' @rdname hypot #' @name hypot #' @family math_funcs +#' @aliases hypot,Column-method #' @export #' @examples \dontrun{hypot(df$c, x)} #' @note hypot since 1.4.0 @@ -1740,9 +2020,13 @@ setMethod("hypot", signature(y = "Column"), #' #' Computes the Levenshtein distance of the two given string columns. #' +#' @param x Column to compute on. +#' @param y Column to compute on. +#' #' @rdname levenshtein #' @name levenshtein #' @family string_funcs +#' @aliases levenshtein,Column-method #' @export #' @examples \dontrun{levenshtein(df$c, x)} #' @note levenshtein since 1.5.0 @@ -1759,9 +2043,13 @@ setMethod("levenshtein", signature(y = "Column"), #' #' Returns number of months between dates `date1` and `date2`. #' +#' @param x start Column to use. +#' @param y end Column to use. +#' #' @rdname months_between #' @name months_between #' @family datetime_funcs +#' @aliases months_between,Column-method #' @export #' @examples \dontrun{months_between(df$c, x)} #' @note months_between since 1.5.0 @@ -1779,9 +2067,13 @@ setMethod("months_between", signature(y = "Column"), #' Returns col1 if it is not NaN, or col2 if col1 is NaN. #' Both inputs should be floating point columns (DoubleType or FloatType). #' +#' @param x first Column. +#' @param y second Column. +#' #' @rdname nanvl #' @name nanvl #' @family normal_funcs +#' @aliases nanvl,Column-method #' @export #' @examples \dontrun{nanvl(df$c, x)} #' @note nanvl since 1.5.0 @@ -1798,10 +2090,14 @@ setMethod("nanvl", signature(y = "Column"), #' #' Returns the positive value of dividend mod divisor. #' +#' @param x divisor Column. +#' @param y dividend Column. +#' #' @rdname pmod #' @name pmod #' @docType methods #' @family math_funcs +#' @aliases pmod,Column-method #' @export #' @examples \dontrun{pmod(df$c, x)} #' @note pmod since 1.5.0 @@ -1817,6 +2113,10 @@ setMethod("pmod", signature(y = "Column"), #' @rdname approxCountDistinct #' @name approxCountDistinct +#' +#' @param rsd maximum estimation error allowed (default = 0.05) +#' +#' @aliases approxCountDistinct,Column-method #' @export #' @examples \dontrun{approxCountDistinct(df$c, 0.02)} #' @note approxCountDistinct(Column, numeric) since 1.4.0 @@ -1829,9 +2129,13 @@ setMethod("approxCountDistinct", #' Count Distinct #' +#' @param x Column to compute on +#' @param ... other columns +#' #' @family agg_funcs #' @rdname countDistinct #' @name countDistinct +#' @aliases countDistinct,Column-method #' @return the number of distinct items in a group. #' @export #' @examples \dontrun{countDistinct(df$c)} @@ -1852,10 +2156,14 @@ setMethod("countDistinct", #' concat #' #' Concatenates multiple input string columns together into a single string column. +#' +#' @param x Column to compute on +#' @param ... other columns #' #' @family string_funcs #' @rdname concat #' @name concat +#' @aliases concat,Column-method #' @export #' @examples \dontrun{concat(df$strings, df$strings2)} #' @note concat since 1.5.0 @@ -1875,9 +2183,13 @@ setMethod("concat", #' Returns the greatest value of the list of column names, skipping null values. #' This function takes at least 2 parameters. It will return null if all parameters are null. #' +#' @param x Column to compute on +#' @param ... other columns +#' #' @family normal_funcs #' @rdname greatest #' @name greatest +#' @aliases greatest,Column-method #' @export #' @examples \dontrun{greatest(df$c, df$d)} #' @note greatest since 1.5.0 @@ -1898,8 +2210,12 @@ setMethod("greatest", #' Returns the least value of the list of column names, skipping null values. #' This function takes at least 2 parameters. It will return null if all parameters are null. #' +#' @param x Column to compute on +#' @param ... other columns +#' #' @family normal_funcs #' @rdname least +#' @aliases least,Column-method #' @name least #' @export #' @examples \dontrun{least(df$c, df$d)} @@ -1917,7 +2233,9 @@ setMethod("least", }) #' @rdname ceil +#' #' @name ceiling +#' @aliases ceiling,Column-method #' @export #' @examples \dontrun{ceiling(df$c)} #' @note ceiling since 1.5.0 @@ -1928,7 +2246,10 @@ setMethod("ceiling", }) #' @rdname sign +#' @param x Column to compute on +#' #' @name sign +#' @aliases sign,Column-method #' @export #' @examples \dontrun{sign(df$c)} #' @note sign since 1.5.0 @@ -1941,8 +2262,12 @@ setMethod("sign", signature(x = "Column"), #' #' Aggregate function: returns the number of distinct items in a group. #' +#' @param x Column to compute on +#' @param ... other columns +#' #' @rdname countDistinct #' @name n_distinct +#' @aliases n_distinct,Column-method #' @export #' @examples \dontrun{n_distinct(df$c)} #' @note n_distinct since 1.4.0 @@ -1951,8 +2276,11 @@ setMethod("n_distinct", signature(x = "Column"), countDistinct(x, ...) }) -#' @rdname count +#' @rdname nrow +#' @param x Column to compute on +#' #' @name n +#' @aliases n,Column-method #' @export #' @examples \dontrun{n(df$c)} #' @note n since 1.4.0 @@ -1972,9 +2300,13 @@ setMethod("n", signature(x = "Column"), #' NOTE: Use when ever possible specialized functions like \code{year}. These benefit from a #' specialized implementation. #' +#' @param y Column to compute on +#' @param x date format specification +#' #' @family datetime_funcs #' @rdname date_format #' @name date_format +#' @aliases date_format,Column,character-method #' @export #' @examples \dontrun{date_format(df$t, 'MM/dd/yyy')} #' @note date_format since 1.5.0 @@ -1988,9 +2320,13 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' #' Assumes given timestamp is UTC and converts to given timezone. #' +#' @param y Column to compute on +#' @param x time zone to use +#' #' @family datetime_funcs #' @rdname from_utc_timestamp #' @name from_utc_timestamp +#' @aliases from_utc_timestamp,Column,character-method #' @export #' @examples \dontrun{from_utc_timestamp(df$t, 'PST')} #' @note from_utc_timestamp since 1.5.0 @@ -2011,6 +2347,7 @@ setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), #' @param y column to check #' @param x substring to check #' @family string_funcs +#' @aliases instr,Column,character-method #' @rdname instr #' @name instr #' @export @@ -2033,9 +2370,13 @@ setMethod("instr", signature(y = "Column", x = "character"), #' Day of the week parameter is case insensitive, and accepts first three or two characters: #' "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". #' +#' @param y Column to compute on +#' @param x Day of the week string +#' #' @family datetime_funcs #' @rdname next_day #' @name next_day +#' @aliases next_day,Column,character-method #' @export #' @examples #'\dontrun{ @@ -2053,9 +2394,13 @@ setMethod("next_day", signature(y = "Column", x = "character"), #' #' Assumes given timestamp is in given timezone and converts to UTC. #' +#' @param y Column to compute on +#' @param x timezone to use +#' #' @family datetime_funcs #' @rdname to_utc_timestamp #' @name to_utc_timestamp +#' @aliases to_utc_timestamp,Column,character-method #' @export #' @examples \dontrun{to_utc_timestamp(df$t, 'PST')} #' @note to_utc_timestamp since 1.5.0 @@ -2069,9 +2414,13 @@ setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), #' #' Returns the date that is numMonths after startDate. #' +#' @param y Column to compute on +#' @param x Number of months to add +#' #' @name add_months #' @family datetime_funcs #' @rdname add_months +#' @aliases add_months,Column,numeric-method #' @export #' @examples \dontrun{add_months(df$d, 1)} #' @note add_months since 1.5.0 @@ -2085,9 +2434,13 @@ setMethod("add_months", signature(y = "Column", x = "numeric"), #' #' Returns the date that is `days` days after `start` #' +#' @param y Column to compute on +#' @param x Number of days to add +#' #' @family datetime_funcs #' @rdname date_add #' @name date_add +#' @aliases date_add,Column,numeric-method #' @export #' @examples \dontrun{date_add(df$d, 1)} #' @note date_add since 1.5.0 @@ -2101,9 +2454,13 @@ setMethod("date_add", signature(y = "Column", x = "numeric"), #' #' Returns the date that is `days` days before `start` #' +#' @param y Column to compute on +#' @param x Number of days to substract +#' #' @family datetime_funcs #' @rdname date_sub #' @name date_sub +#' @aliases date_sub,Column,numeric-method #' @export #' @examples \dontrun{date_sub(df$d, 1)} #' @note date_sub since 1.5.0 @@ -2126,6 +2483,7 @@ setMethod("date_sub", signature(y = "Column", x = "numeric"), #' @family string_funcs #' @rdname format_number #' @name format_number +#' @aliases format_number,Column,numeric-method #' @export #' @examples \dontrun{format_number(df$n, 4)} #' @note format_number since 1.5.0 @@ -2147,6 +2505,7 @@ setMethod("format_number", signature(y = "Column", x = "numeric"), #' @family misc_funcs #' @rdname sha2 #' @name sha2 +#' @aliases sha2,Column,numeric-method #' @export #' @examples \dontrun{sha2(df$c, 256)} #' @note sha2 since 1.5.0 @@ -2161,9 +2520,13 @@ setMethod("sha2", signature(y = "Column", x = "numeric"), #' Shift the given value numBits left. If the given value is a long value, this function #' will return a long value else it will return an integer value. #' +#' @param y column to compute on. +#' @param x number of bits to shift. +#' #' @family math_funcs #' @rdname shiftLeft #' @name shiftLeft +#' @aliases shiftLeft,Column,numeric-method #' @export #' @examples \dontrun{shiftLeft(df$c, 1)} #' @note shiftLeft since 1.5.0 @@ -2180,9 +2543,13 @@ setMethod("shiftLeft", signature(y = "Column", x = "numeric"), #' Shift the given value numBits right. If the given value is a long value, it will return #' a long value else it will return an integer value. #' +#' @param y column to compute on. +#' @param x number of bits to shift. +#' #' @family math_funcs #' @rdname shiftRight #' @name shiftRight +#' @aliases shiftRight,Column,numeric-method #' @export #' @examples \dontrun{shiftRight(df$c, 1)} #' @note shiftRight since 1.5.0 @@ -2199,9 +2566,13 @@ setMethod("shiftRight", signature(y = "Column", x = "numeric"), #' Unsigned shift the given value numBits right. If the given value is a long value, #' it will return a long value else it will return an integer value. #' +#' @param y column to compute on. +#' @param x number of bits to shift. +#' #' @family math_funcs #' @rdname shiftRightUnsigned #' @name shiftRightUnsigned +#' @aliases shiftRightUnsigned,Column,numeric-method #' @export #' @examples \dontrun{shiftRightUnsigned(df$c, 1)} #' @note shiftRightUnsigned since 1.5.0 @@ -2218,9 +2589,14 @@ setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), #' Concatenates multiple input string columns together into a single string column, #' using the given separator. #' +#' @param x column to concatenate. +#' @param sep separator to use. +#' @param ... other columns to concatenate. +#' #' @family string_funcs #' @rdname concat_ws #' @name concat_ws +#' @aliases concat_ws,character,Column-method #' @export #' @examples \dontrun{concat_ws('-', df$s, df$d)} #' @note concat_ws since 1.5.0 @@ -2235,8 +2611,13 @@ setMethod("concat_ws", signature(sep = "character", x = "Column"), #' #' Convert a number in a string column from one base to another. #' +#' @param x column to convert. +#' @param fromBase base to convert from. +#' @param toBase base to convert to. +#' #' @family math_funcs #' @rdname conv +#' @aliases conv,Column,numeric,numeric-method #' @name conv #' @export #' @examples \dontrun{conv(df$n, 2, 16)} @@ -2258,6 +2639,7 @@ setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeri #' #' @family normal_funcs #' @rdname expr +#' @aliases expr,character-method #' @name expr #' @export #' @examples \dontrun{expr('length(name)')} @@ -2275,6 +2657,7 @@ setMethod("expr", signature(x = "character"), #' @family string_funcs #' @rdname format_string #' @name format_string +#' @aliases format_string,character,Column-method #' @export #' @examples \dontrun{format_string('%d %s', df$a, df$b)} #' @note format_string since 1.5.0 @@ -2296,6 +2679,7 @@ setMethod("format_string", signature(format = "character", x = "Column"), #' @family datetime_funcs #' @rdname from_unixtime #' @name from_unixtime +#' @aliases from_unixtime,Column-method #' @export #' @examples #'\dontrun{ @@ -2334,6 +2718,7 @@ setMethod("from_unixtime", signature(x = "Column"), #' @family datetime_funcs #' @rdname window #' @name window +#' @aliases window,Column-method #' @export #' @examples #'\dontrun{ @@ -2383,6 +2768,7 @@ setMethod("window", signature(x = "Column"), #' #' @family string_funcs #' @rdname locate +#' @aliases locate,character,Column-method #' @name locate #' @export #' @examples \dontrun{locate('b', df$c, 1)} @@ -2401,6 +2787,7 @@ setMethod("locate", signature(substr = "character", str = "Column"), #' #' @family string_funcs #' @rdname lpad +#' @aliases lpad,Column,numeric,character-method #' @name lpad #' @export #' @examples \dontrun{lpad(df$c, 6, '#')} @@ -2420,6 +2807,7 @@ setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), #' @family normal_funcs #' @rdname rand #' @name rand +#' @aliases rand,missing-method #' @export #' @examples \dontrun{rand()} #' @note rand since 1.5.0 @@ -2431,6 +2819,7 @@ setMethod("rand", signature(seed = "missing"), #' @rdname rand #' @name rand +#' @aliases rand,numeric-method #' @export #' @note rand(numeric) since 1.5.0 setMethod("rand", signature(seed = "numeric"), @@ -2446,6 +2835,7 @@ setMethod("rand", signature(seed = "numeric"), #' @family normal_funcs #' @rdname randn #' @name randn +#' @aliases randn,missing-method #' @export #' @examples \dontrun{randn()} #' @note randn since 1.5.0 @@ -2457,6 +2847,7 @@ setMethod("randn", signature(seed = "missing"), #' @rdname randn #' @name randn +#' @aliases randn,numeric-method #' @export #' @note randn(numeric) since 1.5.0 setMethod("randn", signature(seed = "numeric"), @@ -2472,6 +2863,7 @@ setMethod("randn", signature(seed = "numeric"), #' @family string_funcs #' @rdname regexp_extract #' @name regexp_extract +#' @aliases regexp_extract,Column,character,numeric-method #' @export #' @examples \dontrun{regexp_extract(df$c, '(\d+)-(\d+)', 1)} #' @note regexp_extract since 1.5.0 @@ -2491,6 +2883,7 @@ setMethod("regexp_extract", #' @family string_funcs #' @rdname regexp_replace #' @name regexp_replace +#' @aliases regexp_replace,Column,character,character-method #' @export #' @examples \dontrun{regexp_replace(df$c, '(\\d+)', '--')} #' @note regexp_replace since 1.5.0 @@ -2510,6 +2903,7 @@ setMethod("regexp_replace", #' @family string_funcs #' @rdname rpad #' @name rpad +#' @aliases rpad,Column,numeric,character-method #' @export #' @examples \dontrun{rpad(df$c, 6, '#')} #' @note rpad since 1.5.0 @@ -2530,6 +2924,7 @@ setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), #' #' @family string_funcs #' @rdname substring_index +#' @aliases substring_index,Column,character,numeric-method #' @name substring_index #' @export #' @examples @@ -2557,6 +2952,7 @@ setMethod("substring_index", #' @family string_funcs #' @rdname translate #' @name translate +#' @aliases translate,Column,character,character-method #' @export #' @examples \dontrun{translate(df$c, 'rnlt', '123')} #' @note translate since 1.5.0 @@ -2575,6 +2971,7 @@ setMethod("translate", #' @family datetime_funcs #' @rdname unix_timestamp #' @name unix_timestamp +#' @aliases unix_timestamp,missing,missing-method #' @export #' @examples #'\dontrun{ @@ -2591,6 +2988,7 @@ setMethod("unix_timestamp", signature(x = "missing", format = "missing"), #' @rdname unix_timestamp #' @name unix_timestamp +#' @aliases unix_timestamp,Column,missing-method #' @export #' @note unix_timestamp(Column) since 1.5.0 setMethod("unix_timestamp", signature(x = "Column", format = "missing"), @@ -2601,6 +2999,7 @@ setMethod("unix_timestamp", signature(x = "Column", format = "missing"), #' @rdname unix_timestamp #' @name unix_timestamp +#' @aliases unix_timestamp,Column,character-method #' @export #' @note unix_timestamp(Column, character) since 1.5.0 setMethod("unix_timestamp", signature(x = "Column", format = "character"), @@ -2616,6 +3015,7 @@ setMethod("unix_timestamp", signature(x = "Column", format = "character"), #' @family normal_funcs #' @rdname when #' @name when +#' @aliases when,Column-method #' @seealso \link{ifelse} #' @export #' @examples \dontrun{when(df$age == 2, df$age + 1)} @@ -2636,6 +3036,7 @@ setMethod("when", signature(condition = "Column", value = "ANY"), #' @family normal_funcs #' @rdname ifelse #' @name ifelse +#' @aliases ifelse,Column-method #' @seealso \link{when} #' @export #' @examples \dontrun{ @@ -2671,6 +3072,7 @@ setMethod("ifelse", #' @rdname cume_dist #' @name cume_dist #' @family window_funcs +#' @aliases cume_dist,missing-method #' @export #' @examples \dontrun{cume_dist()} #' @note cume_dist since 1.6.0 @@ -2694,6 +3096,7 @@ setMethod("cume_dist", #' @rdname dense_rank #' @name dense_rank #' @family window_funcs +#' @aliases dense_rank,missing-method #' @export #' @examples \dontrun{dense_rank()} #' @note dense_rank since 1.6.0 @@ -2714,6 +3117,7 @@ setMethod("dense_rank", #' #' @rdname lag #' @name lag +#' @aliases lag,characterOrColumn-method #' @family window_funcs #' @export #' @examples \dontrun{lag(df$c)} @@ -2739,10 +3143,15 @@ setMethod("lag", #' an `offset` of one will return the next row at any given point in the window partition. #' #' This is equivalent to the LEAD function in SQL. +#' +#' @param x Column to compute on +#' @param offset Number of rows to offset +#' @param defaultValue (Optional) default value to use #' #' @rdname lead #' @name lead #' @family window_funcs +#' @aliases lead,characterOrColumn,numeric-method #' @export #' @examples \dontrun{lead(df$c)} #' @note lead since 1.6.0 @@ -2763,13 +3172,16 @@ setMethod("lead", #' ntile #' #' Window function: returns the ntile group id (from 1 to `n` inclusive) in an ordered window -#' partition. Fow example, if `n` is 4, the first quarter of the rows will get value 1, the second +#' partition. For example, if `n` is 4, the first quarter of the rows will get value 1, the second #' quarter will get 2, the third quarter will get 3, and the last quarter will get 4. #' #' This is equivalent to the NTILE function in SQL. #' +#' @param x Number of ntile groups +#' #' @rdname ntile #' @name ntile +#' @aliases ntile,numeric-method #' @family window_funcs #' @export #' @examples \dontrun{ntile(1)} @@ -2794,6 +3206,7 @@ setMethod("ntile", #' @rdname percent_rank #' @name percent_rank #' @family window_funcs +#' @aliases percent_rank,missing-method #' @export #' @examples \dontrun{percent_rank()} #' @note percent_rank since 1.6.0 @@ -2818,6 +3231,7 @@ setMethod("percent_rank", #' @rdname rank #' @name rank #' @family window_funcs +#' @aliases rank,missing-method #' @export #' @examples \dontrun{rank()} #' @note rank since 1.6.0 @@ -2829,6 +3243,10 @@ setMethod("rank", }) # Expose rank() in the R base package +#' @name rank +#' @rdname rank +#' @aliases rank,ANY-method +#' @export setMethod("rank", signature(x = "ANY"), function(x, ...) { @@ -2843,6 +3261,7 @@ setMethod("rank", #' #' @rdname row_number #' @name row_number +#' @aliases row_number,missing-method #' @family window_funcs #' @export #' @examples \dontrun{row_number()} @@ -2863,6 +3282,7 @@ setMethod("row_number", #' @param x A Column #' @param value A value to be checked if contained in the column #' @rdname array_contains +#' @aliases array_contains,Column-method #' @name array_contains #' @family collection_funcs #' @export @@ -2879,9 +3299,12 @@ setMethod("array_contains", #' #' Creates a new row for each element in the given array or map column. #' +#' @param x Column to compute on +#' #' @rdname explode #' @name explode #' @family collection_funcs +#' @aliases explode,Column-method #' @export #' @examples \dontrun{explode(df$c)} #' @note explode since 1.5.0 @@ -2895,9 +3318,12 @@ setMethod("explode", #' size #' #' Returns length of array or map. +#' +#' @param x Column to compute on #' #' @rdname size #' @name size +#' @aliases size,Column-method #' @family collection_funcs #' @export #' @examples \dontrun{size(df$c)} @@ -2920,6 +3346,7 @@ setMethod("size", #' FALSE, sorting is in descending order. #' @rdname sort_array #' @name sort_array +#' @aliases sort_array,Column-method #' @family collection_funcs #' @export #' @examples @@ -2939,9 +3366,12 @@ setMethod("sort_array", #' #' Creates a new row for each element with position in the given array or map column. #' +#' @param x Column to compute on +#' #' @rdname posexplode #' @name posexplode #' @family collection_funcs +#' @aliases posexplode,Column-method #' @export #' @examples \dontrun{posexplode(df$c)} #' @note posexplode since 2.1.0 diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 8416e5cdb8bf4..e7444ac2467d8 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -51,7 +51,7 @@ setGeneric("collectPartition", standardGeneric("collectPartition") }) -# @rdname count +# @rdname nrow # @export setGeneric("count", function(x) { standardGeneric("count") }) @@ -395,7 +395,7 @@ setGeneric("value", function(bcast) { standardGeneric("value") }) #################### SparkDataFrame Methods ######################## -#' @rdname agg +#' @rdname summarize #' @export setGeneric("agg", function (x, ...) { standardGeneric("agg") }) @@ -654,7 +654,7 @@ setGeneric("showDF", function(x, ...) { standardGeneric("showDF") }) # @export setGeneric("subset", function(x, ...) { standardGeneric("subset") }) -#' @rdname agg +#' @rdname summarize #' @export setGeneric("summarize", function(x, ...) { standardGeneric("summarize") }) @@ -1022,7 +1022,7 @@ setGeneric("month", function(x) { standardGeneric("month") }) #' @export setGeneric("months_between", function(y, x) { standardGeneric("months_between") }) -#' @rdname count +#' @rdname nrow #' @export setGeneric("n", function(x) { standardGeneric("n") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 5ed7e8abb43de..85348ae76baa7 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -47,6 +47,7 @@ groupedData <- function(sgd) { #' @rdname show +#' @aliases show,GroupedData-method #' @note show(GroupedData) since 1.4.0 setMethod("show", "GroupedData", function(object) { @@ -61,6 +62,7 @@ setMethod("show", "GroupedData", #' @param x a GroupedData #' @return a SparkDataFrame #' @rdname count +#' @aliases count,GroupedData-method #' @export #' @examples #' \dontrun{ @@ -84,6 +86,7 @@ setMethod("count", #' @param x a GroupedData #' @return a SparkDataFrame #' @rdname summarize +#' @aliases agg,GroupedData-method #' @name agg #' @family agg_funcs #' @export @@ -121,6 +124,7 @@ setMethod("agg", #' @rdname summarize #' @name summarize +#' @aliases summarize,GroupedData-method #' @note summarize since 1.4.0 setMethod("summarize", signature(x = "GroupedData"), @@ -146,6 +150,7 @@ methods <- c("avg", "max", "mean", "min", "sum") #' @param values A value or a list/vector of distinct values for the output columns. #' @return GroupedData object #' @rdname pivot +#' @aliases pivot,GroupedData,character-method #' @name pivot #' @export #' @examples @@ -198,6 +203,7 @@ createMethods() #' #' @param x A GroupedData #' @rdname gapply +#' @aliases gapply,GroupedData-method #' @name gapply #' @export #' @note gapply(GroupedData) since 2.0.0 @@ -212,6 +218,7 @@ setMethod("gapply", #' #' @param x A GroupedData #' @rdname gapplyCollect +#' @aliases gapplyCollect,GroupedData-method #' @name gapplyCollect #' @export #' @note gapplyCollect(GroupedData) since 2.0.0 @@ -243,4 +250,4 @@ gapplyInternal <- function(x, func, schema) { broadcastArr, if (class(schema) == "structType") { schema$jobj } else { NULL }) dataFrame(sdf) -} \ No newline at end of file +} diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 94e1f654f5196..50c601fcd9e1b 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -91,6 +91,7 @@ NULL #' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. #' @param tol Positive convergence tolerance of iterations. #' @param maxIter Integer giving the maximal number of IRLS iterations. +#' @aliases spark.glm,SparkDataFrame,formula-method #' @return \code{spark.glm} returns a fitted generalized linear model #' @rdname spark.glm #' @name spark.glm @@ -306,6 +307,7 @@ setMethod("summary", signature(object = "NaiveBayesModel"), #' @param initMode The initialization algorithm choosen to fit the model #' @return \code{spark.kmeans} returns a fitted k-means model #' @rdname spark.kmeans +#' @aliases spark.kmeans,SparkDataFrame,formula-method #' @name spark.kmeans #' @export #' @examples @@ -418,6 +420,7 @@ setMethod("predict", signature(object = "KMeansModel"), #' @param smoothing Smoothing parameter #' @return \code{spark.naiveBayes} returns a fitted naive Bayes model #' @rdname spark.naiveBayes +#' @aliases spark.naiveBayes,SparkDataFrame,formula-method #' @name spark.naiveBayes #' @seealso e1071: \url{https://cran.r-project.org/web/packages/e1071/} #' @export @@ -512,7 +515,6 @@ setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", pat #' which means throw exception if the output path exists. #' #' @rdname spark.kmeans -#' @name write.ml #' @export #' @note write.ml(KMeansModel, character) since 2.0.0 setMethod("write.ml", signature(object = "KMeansModel", path = "character"), diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R index a91e9980df937..b429f5de13b87 100644 --- a/R/pkg/R/schema.R +++ b/R/pkg/R/schema.R @@ -26,6 +26,7 @@ #' @param x a structField object (created with the field() function) #' @param ... additional structField objects #' @return a structType object +#' @rdname structType #' @export #' @examples #'\dontrun{ @@ -40,13 +41,19 @@ structType <- function(x, ...) { UseMethod("structType", x) } -structType.jobj <- function(x) { +#' @rdname structType +#' @method structType jobj +#' @export +structType.jobj <- function(x, ...) { obj <- structure(list(), class = "structType") obj$jobj <- x obj$fields <- function() { lapply(callJMethod(obj$jobj, "fields"), structField) } obj } +#' @rdname structType +#' @method structType structField +#' @export structType.structField <- function(x, ...) { fields <- list(x, ...) if (!all(sapply(fields, inherits, "structField"))) { @@ -104,7 +111,10 @@ structField <- function(x, ...) { UseMethod("structField", x) } -structField.jobj <- function(x) { +#' @rdname structField +#' @method structField jobj +#' @export +structField.jobj <- function(x, ...) { obj <- structure(list(), class = "structField") obj$jobj <- x obj$name <- function() { callJMethod(x, "name") } @@ -179,7 +189,7 @@ checkType <- function(type) { #' @param nullable A logical vector indicating whether or not the field is nullable #' @rdname structField #' @export -structField.character <- function(x, type, nullable = TRUE) { +structField.character <- function(x, type, nullable = TRUE, ...) { if (class(x) != "character") { stop("Field name must be a string.") } diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index c92352e1b063d..2b4ce195cbddb 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -34,6 +34,7 @@ setOldClass("jobj") #' #' @rdname crosstab #' @name crosstab +#' @aliases crosstab,SparkDataFrame,character,character-method #' @family stat functions #' @export #' @examples @@ -59,6 +60,7 @@ setMethod("crosstab", #' #' @rdname cov #' @name cov +#' @aliases cov,SparkDataFrame-method #' @family stat functions #' @export #' @examples @@ -88,6 +90,7 @@ setMethod("cov", #' #' @rdname corr #' @name corr +#' @aliases corr,SparkDataFrame-method #' @family stat functions #' @export #' @examples @@ -120,6 +123,7 @@ setMethod("corr", #' #' @rdname freqItems #' @name freqItems +#' @aliases freqItems,SparkDataFrame,character-method #' @family stat functions #' @export #' @examples @@ -158,6 +162,7 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), #' #' @rdname approxQuantile #' @name approxQuantile +#' @aliases approxQuantile,SparkDataFrame,character,numeric,numeric-method #' @family stat functions #' @export #' @examples @@ -188,6 +193,7 @@ setMethod("approxQuantile", #' @return A new SparkDataFrame that represents the stratified sample #' #' @rdname sampleBy +#' @aliases sampleBy,SparkDataFrame,character,list,numeric-method #' @name sampleBy #' @family stat functions #' @export diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index e75bfbf037fbb..240b9f669bdd7 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -126,20 +126,16 @@ hashCode <- function(key) { as.integer(bitwXor(intBits[2], intBits[1])) } else if (class(key) == "character") { # TODO: SPARK-7839 means we might not have the native library available - if (is.loaded("stringHashCode")) { - .Call("stringHashCode", key) + n <- nchar(key) + if (n == 0) { + 0L } else { - n <- nchar(key) - if (n == 0) { - 0L - } else { - asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) - hashC <- 0 - for (k in 1:length(asciiVals)) { - hashC <- mult31AndAdd(hashC, asciiVals[k]) - } - as.integer(hashC) + asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) + hashC <- 0 + for (k in 1:length(asciiVals)) { + hashC <- mult31AndAdd(hashC, asciiVals[k]) } + as.integer(hashC) } } else { warning(paste("Could not hash object, returning 0", sep = "")) diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R index d9d069cde9309..215d0e7b5cfbe 100644 --- a/R/pkg/R/window.R +++ b/R/pkg/R/window.R @@ -28,6 +28,7 @@ #' #' @rdname windowPartitionBy #' @name windowPartitionBy +#' @aliases windowPartitionBy,character-method #' @export #' @examples #' \dontrun{ @@ -50,6 +51,7 @@ setMethod("windowPartitionBy", #' @rdname windowPartitionBy #' @name windowPartitionBy +#' @aliases windowPartitionBy,Column-method #' @export #' @note windowPartitionBy(Column) since 2.0.0 setMethod("windowPartitionBy", @@ -75,6 +77,7 @@ setMethod("windowPartitionBy", #' #' @rdname windowOrderBy #' @name windowOrderBy +#' @aliases windowOrderBy,character-method #' @export #' @examples #' \dontrun{ @@ -97,6 +100,7 @@ setMethod("windowOrderBy", #' @rdname windowOrderBy #' @name windowOrderBy +#' @aliases windowOrderBy,Column-method #' @export #' @note windowOrderBy(Column) since 2.0.0 setMethod("windowOrderBy", diff --git a/dev/.rat-excludes b/dev/.rat-excludes index fb582dec56d51..0c866717a3f43 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -99,3 +99,4 @@ spark-deps-.* .*tsv org.apache.spark.scheduler.ExternalClusterManager .*\.sql +.Rbuildignore From a4bf13a74bbe180c0f1848f960096dd94af8afd1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 16 Jul 2016 23:42:28 -0700 Subject: [PATCH 0993/1470] [SPARK-16584][SQL] Move regexp unit tests to RegexpExpressionsSuite ## What changes were proposed in this pull request? This patch moves regexp related unit tests from StringExpressionsSuite to RegexpExpressionsSuite to match the file name for regexp expressions. ## How was this patch tested? This is a test only change. Author: Reynold Xin Closes #14230 from rxin/SPARK-16584. (cherry picked from commit 7b84758034b9bceca1168438ef5d0beefd5b5273) Signed-off-by: Reynold Xin --- .../expressions/RegexpExpressionsSuite.scala | 194 ++++++++++++++++++ .../expressions/StringExpressionsSuite.scala | 164 --------------- 2 files changed, 194 insertions(+), 164 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala new file mode 100644 index 0000000000000..5299549e7b4da --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types.StringType + +/** + * Unit tests for regular expression (regexp) related SQL expressions. + */ +class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("LIKE literal Regular Expression") { + checkEvaluation(Literal.create(null, StringType).like("a"), null) + checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) + checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null) + checkEvaluation( + Literal.create("a", StringType).like(NonFoldableLiteral.create("a", StringType)), true) + checkEvaluation( + Literal.create("a", StringType).like(NonFoldableLiteral.create(null, StringType)), null) + checkEvaluation( + Literal.create(null, StringType).like(NonFoldableLiteral.create("a", StringType)), null) + checkEvaluation( + Literal.create(null, StringType).like(NonFoldableLiteral.create(null, StringType)), null) + + checkEvaluation("abdef" like "abdef", true) + checkEvaluation("a_%b" like "a\\__b", true) + checkEvaluation("addb" like "a_%b", true) + checkEvaluation("addb" like "a\\__b", false) + checkEvaluation("addb" like "a%\\%b", false) + checkEvaluation("a_%b" like "a%\\%b", true) + checkEvaluation("addb" like "a%", true) + checkEvaluation("addb" like "**", false) + checkEvaluation("abc" like "a%", true) + checkEvaluation("abc" like "b%", false) + checkEvaluation("abc" like "bc%", false) + checkEvaluation("a\nb" like "a_b", true) + checkEvaluation("ab" like "a%b", true) + checkEvaluation("a\nb" like "a%b", true) + } + + test("LIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + 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") { + checkEvaluation(Literal.create(null, StringType) rlike "abdef", null) + checkEvaluation("abdef" rlike Literal.create(null, StringType), null) + checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null) + checkEvaluation("abdef" rlike NonFoldableLiteral.create("abdef", StringType), true) + checkEvaluation("abdef" rlike NonFoldableLiteral.create(null, StringType), null) + checkEvaluation( + Literal.create(null, StringType) rlike NonFoldableLiteral.create("abdef", StringType), null) + checkEvaluation( + Literal.create(null, StringType) rlike NonFoldableLiteral.create(null, StringType), null) + + checkEvaluation("abdef" rlike "abdef", true) + checkEvaluation("abbbbc" rlike "a.*c", true) + + checkEvaluation("fofo" rlike "^fo", true) + checkEvaluation("fo\no" rlike "^fo\no$", true) + checkEvaluation("Bn" rlike "^Ba*n", true) + checkEvaluation("afofo" rlike "fo", true) + checkEvaluation("afofo" rlike "^fo", false) + checkEvaluation("Baan" rlike "^Ba?n", false) + checkEvaluation("axe" rlike "pi|apa", false) + checkEvaluation("pip" rlike "^(pi)*$", false) + + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + + intercept[java.util.regex.PatternSyntaxException] { + evaluate("abbbbc" rlike "**") + } + } + + test("RLIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + 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, create_row("**")) + } + } + + + test("RegexReplace") { + val row1 = create_row("100-200", "(\\d+)", "num") + val row2 = create_row("100-200", "(\\d+)", "###") + val row3 = create_row("100-200", "(-)", "###") + val row4 = create_row(null, "(\\d+)", "###") + val row5 = create_row("100-200", null, "###") + val row6 = create_row("100-200", "(-)", null) + + val s = 's.string.at(0) + val p = 'p.string.at(1) + val r = 'r.string.at(2) + + val expr = RegExpReplace(s, p, r) + checkEvaluation(expr, "num-num", row1) + checkEvaluation(expr, "###-###", row2) + checkEvaluation(expr, "100###200", row3) + checkEvaluation(expr, null, row4) + checkEvaluation(expr, null, row5) + checkEvaluation(expr, null, row6) + + val nonNullExpr = RegExpReplace(Literal("100-200"), Literal("(\\d+)"), Literal("num")) + checkEvaluation(nonNullExpr, "num-num", row1) + } + + test("RegexExtract") { + val row1 = create_row("100-200", "(\\d+)-(\\d+)", 1) + val row2 = create_row("100-200", "(\\d+)-(\\d+)", 2) + val row3 = create_row("100-200", "(\\d+).*", 1) + val row4 = create_row("100-200", "([a-z])", 1) + val row5 = create_row(null, "([a-z])", 1) + val row6 = create_row("100-200", null, 1) + val row7 = create_row("100-200", "([a-z])", null) + + val s = 's.string.at(0) + val p = 'p.string.at(1) + val r = 'r.int.at(2) + + val expr = RegExpExtract(s, p, r) + checkEvaluation(expr, "100", row1) + checkEvaluation(expr, "200", row2) + checkEvaluation(expr, "100", row3) + checkEvaluation(expr, "", row4) // will not match anything, empty string get + checkEvaluation(expr, null, row5) + checkEvaluation(expr, null, row6) + checkEvaluation(expr, null, row7) + + val expr1 = new RegExpExtract(s, p) + checkEvaluation(expr1, "100", row1) + + val nonNullExpr = RegExpExtract(Literal("100-200"), Literal("(\\d+)-(\\d+)"), Literal(1)) + checkEvaluation(nonNullExpr, "100", row1) + } + + test("SPLIT") { + val s1 = 'a.string.at(0) + val s2 = 'b.string.at(1) + val row1 = create_row("aa2bb3cc", "[1-9]+") + val row2 = create_row(null, "[1-9]+") + val row3 = create_row("aa2bb3cc", null) + + checkEvaluation( + StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+")), Seq("aa", "bb", "cc"), row1) + checkEvaluation( + StringSplit(s1, s2), Seq("aa", "bb", "cc"), row1) + checkEvaluation(StringSplit(s1, s2), null, row2) + checkEvaluation(StringSplit(s1, s2), null, row3) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 5b9ed83a4eb35..fdb9fa31f09c8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -254,102 +254,6 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { SubstringIndex(Literal("www||apache||org"), Literal( "||"), Literal(2)), "www||apache") } - test("LIKE literal Regular Expression") { - checkEvaluation(Literal.create(null, StringType).like("a"), null) - checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) - checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null) - checkEvaluation( - Literal.create("a", StringType).like(NonFoldableLiteral.create("a", StringType)), true) - checkEvaluation( - Literal.create("a", StringType).like(NonFoldableLiteral.create(null, StringType)), null) - checkEvaluation( - Literal.create(null, StringType).like(NonFoldableLiteral.create("a", StringType)), null) - checkEvaluation( - Literal.create(null, StringType).like(NonFoldableLiteral.create(null, StringType)), null) - - checkEvaluation("abdef" like "abdef", true) - checkEvaluation("a_%b" like "a\\__b", true) - checkEvaluation("addb" like "a_%b", true) - checkEvaluation("addb" like "a\\__b", false) - checkEvaluation("addb" like "a%\\%b", false) - checkEvaluation("a_%b" like "a%\\%b", true) - checkEvaluation("addb" like "a%", true) - checkEvaluation("addb" like "**", false) - checkEvaluation("abc" like "a%", true) - checkEvaluation("abc" like "b%", false) - checkEvaluation("abc" like "bc%", false) - checkEvaluation("a\nb" like "a_b", true) - checkEvaluation("ab" like "a%b", true) - checkEvaluation("a\nb" like "a%b", true) - } - - test("LIKE Non-literal Regular Expression") { - val regEx = 'a.string.at(0) - 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") { - checkEvaluation(Literal.create(null, StringType) rlike "abdef", null) - checkEvaluation("abdef" rlike Literal.create(null, StringType), null) - checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null) - checkEvaluation("abdef" rlike NonFoldableLiteral.create("abdef", StringType), true) - checkEvaluation("abdef" rlike NonFoldableLiteral.create(null, StringType), null) - checkEvaluation( - Literal.create(null, StringType) rlike NonFoldableLiteral.create("abdef", StringType), null) - checkEvaluation( - Literal.create(null, StringType) rlike NonFoldableLiteral.create(null, StringType), null) - - checkEvaluation("abdef" rlike "abdef", true) - checkEvaluation("abbbbc" rlike "a.*c", true) - - checkEvaluation("fofo" rlike "^fo", true) - checkEvaluation("fo\no" rlike "^fo\no$", true) - checkEvaluation("Bn" rlike "^Ba*n", true) - checkEvaluation("afofo" rlike "fo", true) - checkEvaluation("afofo" rlike "^fo", false) - checkEvaluation("Baan" rlike "^Ba?n", false) - checkEvaluation("axe" rlike "pi|apa", false) - checkEvaluation("pip" rlike "^(pi)*$", false) - - checkEvaluation("abc" rlike "^ab", true) - checkEvaluation("abc" rlike "^bc", false) - checkEvaluation("abc" rlike "^ab", true) - checkEvaluation("abc" rlike "^bc", false) - - intercept[java.util.regex.PatternSyntaxException] { - evaluate("abbbbc" rlike "**") - } - } - - test("RLIKE Non-literal Regular Expression") { - val regEx = 'a.string.at(0) - 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, create_row("**")) - } - } - test("ascii for string") { val a = 'a.string.at(0) checkEvaluation(Ascii(Literal("efg")), 101, create_row("abdef")) @@ -612,74 +516,6 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringSpace(s1), null, row2) } - test("RegexReplace") { - val row1 = create_row("100-200", "(\\d+)", "num") - val row2 = create_row("100-200", "(\\d+)", "###") - val row3 = create_row("100-200", "(-)", "###") - val row4 = create_row(null, "(\\d+)", "###") - val row5 = create_row("100-200", null, "###") - val row6 = create_row("100-200", "(-)", null) - - val s = 's.string.at(0) - val p = 'p.string.at(1) - val r = 'r.string.at(2) - - val expr = RegExpReplace(s, p, r) - checkEvaluation(expr, "num-num", row1) - checkEvaluation(expr, "###-###", row2) - checkEvaluation(expr, "100###200", row3) - checkEvaluation(expr, null, row4) - checkEvaluation(expr, null, row5) - checkEvaluation(expr, null, row6) - - val nonNullExpr = RegExpReplace(Literal("100-200"), Literal("(\\d+)"), Literal("num")) - checkEvaluation(nonNullExpr, "num-num", row1) - } - - test("RegexExtract") { - val row1 = create_row("100-200", "(\\d+)-(\\d+)", 1) - val row2 = create_row("100-200", "(\\d+)-(\\d+)", 2) - val row3 = create_row("100-200", "(\\d+).*", 1) - val row4 = create_row("100-200", "([a-z])", 1) - val row5 = create_row(null, "([a-z])", 1) - val row6 = create_row("100-200", null, 1) - val row7 = create_row("100-200", "([a-z])", null) - - val s = 's.string.at(0) - val p = 'p.string.at(1) - val r = 'r.int.at(2) - - val expr = RegExpExtract(s, p, r) - checkEvaluation(expr, "100", row1) - checkEvaluation(expr, "200", row2) - checkEvaluation(expr, "100", row3) - checkEvaluation(expr, "", row4) // will not match anything, empty string get - checkEvaluation(expr, null, row5) - checkEvaluation(expr, null, row6) - checkEvaluation(expr, null, row7) - - val expr1 = new RegExpExtract(s, p) - checkEvaluation(expr1, "100", row1) - - val nonNullExpr = RegExpExtract(Literal("100-200"), Literal("(\\d+)-(\\d+)"), Literal(1)) - checkEvaluation(nonNullExpr, "100", row1) - } - - test("SPLIT") { - val s1 = 'a.string.at(0) - val s2 = 'b.string.at(1) - val row1 = create_row("aa2bb3cc", "[1-9]+") - val row2 = create_row(null, "[1-9]+") - val row3 = create_row("aa2bb3cc", null) - - checkEvaluation( - StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+")), Seq("aa", "bb", "cc"), row1) - checkEvaluation( - StringSplit(s1, s2), Seq("aa", "bb", "cc"), row1) - checkEvaluation(StringSplit(s1, s2), null, row2) - checkEvaluation(StringSplit(s1, s2), null, row3) - } - test("length for string / binary") { val a = 'a.string.at(0) val b = 'b.binary.at(0) From 808d69aaaf5024c139ce9f6ab4e1f62496f7118c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 17 Jul 2016 22:48:00 -0700 Subject: [PATCH 0994/1470] [SPARK-16588][SQL] Deprecate monotonicallyIncreasingId in Scala/Java This patch deprecates monotonicallyIncreasingId in Scala/Java, as done in Python. This patch was originally written by HyukjinKwon. Closes #14236. (cherry picked from commit 480c870644595a71102be6597146d80b1c0816e4) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/ml/clustering/LDA.scala | 4 ++-- .../main/scala/org/apache/spark/sql/functions.scala | 1 + .../apache/spark/sql/ColumnExpressionSuite.scala | 13 ++++++------- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 778cd0fee71c0..034f2c3fa2fd9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -37,7 +37,7 @@ import org.apache.spark.mllib.linalg.MatrixImplicits._ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.functions.{col, monotonicallyIncreasingId, udf} +import org.apache.spark.sql.functions.{col, monotonically_increasing_id, udf} import org.apache.spark.sql.types.StructType @@ -888,7 +888,7 @@ object LDA extends DefaultParamsReadable[LDA] { dataset: Dataset[_], featuresCol: String): RDD[(Long, OldVector)] = { dataset - .withColumn("docId", monotonicallyIncreasingId()) + .withColumn("docId", monotonically_increasing_id()) .select("docId", featuresCol) .rdd .map { case Row(docId: Long, features: Vector) => 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 c8782df146df6..ab09ef7450b04 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 @@ -978,6 +978,7 @@ object functions { * @group normal_funcs * @since 1.4.0 */ + @deprecated("Use monotonically_increasing_id()", "2.0.0") def monotonicallyIncreasingId(): Column = monotonically_increasing_id() /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a170fae577c1b..26e1a9f75da13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -508,18 +508,17 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { Row("ab", "cde")) } - test("monotonicallyIncreasingId") { + test("monotonically_increasing_id") { // Make sure we have 2 partitions, each with 2 records. val df = sparkContext.parallelize(Seq[Int](), 2).mapPartitions { _ => Iterator(Tuple1(1), Tuple1(2)) }.toDF("a") checkAnswer( - df.select(monotonicallyIncreasingId()), - Row(0L) :: Row(1L) :: Row((1L << 33) + 0L) :: Row((1L << 33) + 1L) :: Nil - ) - checkAnswer( - df.select(expr("monotonically_increasing_id()")), - Row(0L) :: Row(1L) :: Row((1L << 33) + 0L) :: Row((1L << 33) + 1L) :: Nil + df.select(monotonically_increasing_id(), expr("monotonically_increasing_id()")), + Row(0L, 0L) :: + Row(1L, 1L) :: + Row((1L << 33) + 0L, (1L << 33) + 0L) :: + Row((1L << 33) + 1L, (1L << 33) + 1L) :: Nil ) } From 2365d635262c42f2c60633d75abb3eddd251b40f Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Mon, 18 Jul 2016 09:11:53 +0100 Subject: [PATCH 0995/1470] [MINOR][TYPO] fix fininsh typo ## What changes were proposed in this pull request? fininsh => finish ## How was this patch tested? N/A Author: WeichenXu Closes #14238 from WeichenXu123/fix_fininsh_typo. (cherry picked from commit a529fc944209e7255ec5858b33490212884d6c60) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/classification/LogisticRegression.scala | 2 +- .../org/apache/spark/ml/regression/AFTSurvivalRegression.scala | 2 +- .../scala/org/apache/spark/ml/regression/LinearRegression.scala | 2 +- .../main/scala/org/apache/spark/mllib/optimization/LBFGS.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) 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 1fed5fd42987e..91eee0e69d635 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 @@ -425,7 +425,7 @@ class LogisticRegression @Since("1.2.0") ( } if (!state.actuallyConverged) { - logWarning("LogisticRegression training fininshed but the result " + + logWarning("LogisticRegression training finished but the result " + s"is not converged because: ${state.convergedReason.get.reason}") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 366448fc56bf5..700a92cc261be 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -246,7 +246,7 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S } if (!state.actuallyConverged) { - logWarning("AFTSurvivalRegression training fininshed but the result " + + logWarning("AFTSurvivalRegression training finished but the result " + s"is not converged because: ${state.convergedReason.get.reason}") } 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 c57e9eb0bfd15..401f2c673f51c 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 @@ -326,7 +326,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String } if (!state.actuallyConverged) { - logWarning("LinearRegression training fininshed but the result " + + logWarning("LinearRegression training finished but the result " + s"is not converged because: ${state.convergedReason.get.reason}") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index c61b2db6c961c..fd09f35277a09 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -214,7 +214,7 @@ object LBFGS extends Logging { lossHistory += state.value if (!state.actuallyConverged) { - logWarning("LBFGS training fininshed but the result " + + logWarning("LBFGS training finished but the result " + s"is not converged because: ${state.convergedReason.get.reason}") } From 085f3cc855c091f0e5ea2699ddc9790e0b4cd83a Mon Sep 17 00:00:00 2001 From: krishnakalyan3 Date: Mon, 18 Jul 2016 09:46:23 -0700 Subject: [PATCH 0996/1470] [SPARK-16055][SPARKR] warning added while using sparkPackages with spark-submit ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-16055 sparkPackages - argument is passed and we detect that we are in the R script mode, we should print some warning like --packages flag should be used with with spark-submit ## How was this patch tested? In my system locally Author: krishnakalyan3 Closes #14179 from krishnakalyan3/spark-pkg. (cherry picked from commit 8ea3f4eaec65ee4277f9943063fcc9488d3fa924) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/sparkR.R | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 62659b0c0ce5f..ff5297ffd51cb 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -155,6 +155,10 @@ sparkR.sparkContext <- function( existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") if (existingPort != "") { + if (length(packages) != 0) { + warning(paste("sparkPackages has no effect when using spark-submit or sparkR shell", + " please use the --packages commandline instead", sep = ",")) + } backendPort <- existingPort } else { path <- tempfile(pattern = "backend_port") From 33d92f7f39136bd399e1f7cabd264e7eeca9b958 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 18 Jul 2016 13:58:12 -0700 Subject: [PATCH 0997/1470] [SPARK-16515][SQL] set default record reader and writer for script transformation ## What changes were proposed in this pull request? In ScriptInputOutputSchema, we read default RecordReader and RecordWriter from conf. Since Spark 2.0 has deleted those config keys from hive conf, we have to set default reader/writer class name by ourselves. Otherwise we will get None for LazySimpleSerde, the data written would not be able to read by script. The test case added worked fine with previous version of Spark, but would fail now. ## How was this patch tested? added a test case in SQLQuerySuite. Closes #14169 Author: Daoyuan Wang Author: Yin Huai Closes #14249 from yhuai/scriptTransformation. (cherry picked from commit 96e9afaae93318250334211cc80ed0fee3d055b9) Signed-off-by: Yin Huai --- .../spark/sql/execution/SparkSqlParser.scala | 16 +++++++++---- sql/hive/src/test/resources/test_script.sh | 23 +++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 11 +++++++++ 3 files changed, 45 insertions(+), 5 deletions(-) create mode 100755 sql/hive/src/test/resources/test_script.sh diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 42ec210baa2d6..3573a86d4e83f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1315,7 +1315,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Decode and input/output format. type Format = (Seq[(String, String)], Option[String], Seq[(String, String)], Option[String]) - def format(fmt: RowFormatContext, configKey: String): Format = fmt match { + def format( + fmt: RowFormatContext, + configKey: String, + defaultConfigValue: String): Format = fmt match { case c: RowFormatDelimitedContext => // TODO we should use the visitRowFormatDelimited function here. However HiveScriptIOSchema // expects a seq of pairs in which the old parsers' token names are used as keys. @@ -1338,7 +1341,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // SPARK-10310: Special cases LazySimpleSerDe val recordHandler = if (name == "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") { - Try(conf.getConfString(configKey)).toOption + Option(conf.getConfString(configKey, defaultConfigValue)) } else { None } @@ -1349,15 +1352,18 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val name = conf.getConfString("hive.script.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") val props = Seq("field.delim" -> "\t") - val recordHandler = Try(conf.getConfString(configKey)).toOption + val recordHandler = Option(conf.getConfString(configKey, defaultConfigValue)) (Nil, Option(name), props, recordHandler) } val (inFormat, inSerdeClass, inSerdeProps, reader) = - format(inRowFormat, "hive.script.recordreader") + format( + inRowFormat, "hive.script.recordreader", "org.apache.hadoop.hive.ql.exec.TextRecordReader") val (outFormat, outSerdeClass, outSerdeProps, writer) = - format(outRowFormat, "hive.script.recordwriter") + format( + outRowFormat, "hive.script.recordwriter", + "org.apache.hadoop.hive.ql.exec.TextRecordWriter") ScriptInputOutputSchema( inFormat, outFormat, diff --git a/sql/hive/src/test/resources/test_script.sh b/sql/hive/src/test/resources/test_script.sh new file mode 100755 index 0000000000000..ab998c41b22ce --- /dev/null +++ b/sql/hive/src/test/resources/test_script.sh @@ -0,0 +1,23 @@ +#!/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. +# + +while read line +do + echo "$line" | sed 's/\t/_/' +done < /dev/stdin 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 22dcbc4c6010d..84fa7f966f66b 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 @@ -63,6 +63,17 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ import spark.implicits._ + test("script") { + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query1 = sql( + """ + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash src/test/resources/test_script.sh' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query1, Row("x1_y1") :: Row("x2_y2") :: Nil) + } + test("UDTF") { withUserDefinedFunction("udtf_count2" -> true) { sql(s"ADD JAR ${hiveContext.getHiveFile("TestUDTF.jar").getCanonicalPath()}") From 7889585ccb9b99eb9dc3a80b8381ae8d2329e26c Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 18 Jul 2016 16:01:57 -0700 Subject: [PATCH 0998/1470] [SPARKR][DOCS] minor code sample update in R programming guide ## What changes were proposed in this pull request? Fix code style from ad hoc review of RC4 doc ## How was this patch tested? manual shivaram Author: Felix Cheung Closes #14250 from felixcheung/rdocs2rc4. (cherry picked from commit 75f0efe74d0c9a7acb525339c5184b99fee4dafc) Signed-off-by: Shivaram Venkataraman --- docs/sparkr.md | 4 ++-- examples/src/main/r/RSparkSQLExample.R | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sparkr.md b/docs/sparkr.md index a5235b2bf66a4..dfa5278ef8491 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -54,7 +54,7 @@ if (nchar(Sys.getenv("SPARK_HOME")) < 1) { Sys.setenv(SPARK_HOME = "/home/spark") } library(SparkR, lib.loc = c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"))) -sc <- sparkR.session(master = "local[*]", sparkConfig = list(spark.driver.memory="2g")) +sparkR.session(master = "local[*]", sparkConfig = list(spark.driver.memory = "2g")) {% endhighlight %}
    @@ -115,7 +115,7 @@ specifying `--packages` with `spark-submit` or `sparkR` commands, or if initiali
    {% highlight r %} -sc <- sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") +sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") {% endhighlight %}
    diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index f20875ca9efb5..33e88e15fd47a 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -144,7 +144,7 @@ write.df(df1, "data/test_table/key=1", "parquet", "overwrite") write.df(df2, "data/test_table/key=2", "parquet", "overwrite") # Read the partitioned table -df3 <- read.df("data/test_table", "parquet", mergeSchema="true") +df3 <- read.df("data/test_table", "parquet", mergeSchema = "true") printSchema(df3) # The final schema consists of all 3 columns in the Parquet files together From aac860802efbae2c61387dbcb8989b3b226a57ff Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 18 Jul 2016 17:17:37 -0700 Subject: [PATCH 0999/1470] [SPARK-16590][SQL] Improve LogicalPlanToSQLSuite to check generated SQL directly ## What changes were proposed in this pull request? This PR improves `LogicalPlanToSQLSuite` to check the generated SQL directly by **structure**. So far, `LogicalPlanToSQLSuite` relies on `checkHiveQl` to ensure the **successful SQL generation** and **answer equality**. However, it does not guarantee the generated SQL is the same or will not be changed unnoticeably. ## How was this patch tested? Pass the Jenkins. This is only a testsuite change. Author: Dongjoon Hyun Closes #14235 from dongjoon-hyun/SPARK-16590. (cherry picked from commit ea78edb80bf46e925d53e2aec29666c4eeb66188) Signed-off-by: Reynold Xin --- sql/hive/src/test/resources/sqlgen/agg1.sql | 4 + sql/hive/src/test/resources/sqlgen/agg2.sql | 4 + sql/hive/src/test/resources/sqlgen/agg3.sql | 4 + .../sqlgen/aggregate_functions_and_window.sql | 4 + sql/hive/src/test/resources/sqlgen/case.sql | 4 + .../test/resources/sqlgen/case_with_else.sql | 4 + .../test/resources/sqlgen/case_with_key.sql | 4 + .../sqlgen/case_with_key_and_else.sql | 4 + .../src/test/resources/sqlgen/cluster_by.sql | 4 + .../sqlgen/data_source_json_parquet_t0.sql | 4 + .../sqlgen/data_source_orc_parquet_t0.sql | 4 + .../sqlgen/data_source_parquet_parquet_t0.sql | 4 + .../resources/sqlgen/distinct_aggregation.sql | 4 + .../test/resources/sqlgen/distribute_by.sql | 4 + .../sqlgen/distribute_by_with_sort_by.sql | 4 + sql/hive/src/test/resources/sqlgen/except.sql | 4 + .../sqlgen/filter_after_subquery.sql | 4 + .../sqlgen/generate_with_other_1.sql | 8 + .../sqlgen/generate_with_other_2.sql | 10 + .../sqlgen/generator_in_lateral_view_1.sql | 4 + .../sqlgen/generator_in_lateral_view_2.sql | 4 + .../generator_non_referenced_table_1.sql | 4 + .../generator_non_referenced_table_2.sql | 4 + .../resources/sqlgen/generator_non_udtf_1.sql | 4 + .../resources/sqlgen/generator_non_udtf_2.sql | 4 + .../sqlgen/generator_referenced_table_1.sql | 4 + .../sqlgen/generator_referenced_table_2.sql | 4 + .../generator_with_ambiguous_names_1.sql | 6 + .../generator_with_ambiguous_names_2.sql | 6 + .../sqlgen/generator_without_from_1.sql | 4 + .../sqlgen/generator_without_from_2.sql | 4 + .../test/resources/sqlgen/grouping_sets_1.sql | 6 + .../resources/sqlgen/grouping_sets_2_1.sql | 4 + .../resources/sqlgen/grouping_sets_2_2.sql | 4 + .../resources/sqlgen/grouping_sets_2_3.sql | 4 + .../resources/sqlgen/grouping_sets_2_4.sql | 4 + .../resources/sqlgen/grouping_sets_2_5.sql | 5 + sql/hive/src/test/resources/sqlgen/in.sql | 4 + .../src/test/resources/sqlgen/intersect.sql | 4 + .../test/resources/sqlgen/join_2_tables.sql | 7 + .../sqlgen/json_tuple_generator_1.sql | 6 + .../sqlgen/json_tuple_generator_2.sql | 6 + .../test/resources/sqlgen/multi_distinct.sql | 4 + .../nested_generator_in_lateral_view_1.sql | 7 + .../nested_generator_in_lateral_view_2.sql | 7 + sql/hive/src/test/resources/sqlgen/not_in.sql | 4 + .../src/test/resources/sqlgen/not_like.sql | 4 + .../sqlgen/regular_expressions_and_window.sql | 4 + .../test/resources/sqlgen/rollup_cube_1_1.sql | 4 + .../test/resources/sqlgen/rollup_cube_1_2.sql | 4 + .../test/resources/sqlgen/rollup_cube_2_1.sql | 4 + .../test/resources/sqlgen/rollup_cube_2_2.sql | 4 + .../test/resources/sqlgen/rollup_cube_3_1.sql | 4 + .../test/resources/sqlgen/rollup_cube_3_2.sql | 4 + .../test/resources/sqlgen/rollup_cube_4_1.sql | 5 + .../test/resources/sqlgen/rollup_cube_4_2.sql | 5 + .../test/resources/sqlgen/rollup_cube_5_1.sql | 6 + .../test/resources/sqlgen/rollup_cube_5_2.sql | 6 + .../test/resources/sqlgen/rollup_cube_6_1.sql | 4 + .../test/resources/sqlgen/rollup_cube_6_2.sql | 4 + .../test/resources/sqlgen/rollup_cube_6_3.sql | 4 + .../test/resources/sqlgen/rollup_cube_6_4.sql | 4 + .../test/resources/sqlgen/rollup_cube_6_5.sql | 4 + .../test/resources/sqlgen/rollup_cube_6_6.sql | 4 + .../test/resources/sqlgen/rollup_cube_7_1.sql | 4 + .../test/resources/sqlgen/rollup_cube_7_2.sql | 4 + .../test/resources/sqlgen/rollup_cube_7_3.sql | 4 + .../test/resources/sqlgen/rollup_cube_8_1.sql | 6 + .../test/resources/sqlgen/rollup_cube_8_2.sql | 6 + .../test/resources/sqlgen/rollup_cube_9_1.sql | 8 + .../test/resources/sqlgen/rollup_cube_9_2.sql | 8 + .../sqlgen/script_transformation_1.sql | 4 + .../sqlgen/script_transformation_2.sql | 4 + .../script_transformation_alias_list.sql | 4 + ...pt_transformation_alias_list_with_type.sql | 6 + ...ipt_transformation_row_format_multiple.sql | 8 + .../script_transformation_row_format_one.sql | 6 + ...script_transformation_row_format_serde.sql | 10 + ...ransformation_row_format_without_serde.sql | 8 + .../test/resources/sqlgen/select_distinct.sql | 4 + .../resources/sqlgen/select_orc_table.sql | 4 + .../resources/sqlgen/select_parquet_table.sql | 4 + .../src/test/resources/sqlgen/self_join.sql | 4 + .../sqlgen/self_join_with_group_by.sql | 4 + .../resources/sqlgen/sort_by_after_having.sql | 4 + .../test/resources/sqlgen/tablesample_1.sql | 4 + .../test/resources/sqlgen/tablesample_2.sql | 4 + .../test/resources/sqlgen/tablesample_3.sql | 4 + .../test/resources/sqlgen/tablesample_4.sql | 4 + .../test/resources/sqlgen/tablesample_5.sql | 4 + .../test/resources/sqlgen/tablesample_6.sql | 4 + .../resources/sqlgen/three_child_union.sql | 6 + .../test/resources/sqlgen/type_widening.sql | 4 + .../test/resources/sqlgen/union_distinct.sql | 4 + .../test/resources/sqlgen/window_basic_1.sql | 4 + .../test/resources/sqlgen/window_basic_2.sql | 5 + .../test/resources/sqlgen/window_basic_3.sql | 5 + .../resources/sqlgen/window_with_join.sql | 5 + .../window_with_the_same_window_with_agg.sql | 7 + ...w_with_the_same_window_with_agg_filter.sql | 7 + ...ith_the_same_window_with_agg_functions.sql | 6 + ...w_with_the_same_window_with_agg_having.sql | 6 + .../sql/catalyst/LogicalPlanToSQLSuite.scala | 484 ++++++++++++------ 103 files changed, 820 insertions(+), 153 deletions(-) create mode 100644 sql/hive/src/test/resources/sqlgen/agg1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/agg2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/agg3.sql create mode 100644 sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql create mode 100644 sql/hive/src/test/resources/sqlgen/case.sql create mode 100644 sql/hive/src/test/resources/sqlgen/case_with_else.sql create mode 100644 sql/hive/src/test/resources/sqlgen/case_with_key.sql create mode 100644 sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql create mode 100644 sql/hive/src/test/resources/sqlgen/cluster_by.sql create mode 100644 sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql create mode 100644 sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql create mode 100644 sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql create mode 100644 sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql create mode 100644 sql/hive/src/test/resources/sqlgen/distribute_by.sql create mode 100644 sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql create mode 100644 sql/hive/src/test/resources/sqlgen/except.sql create mode 100644 sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql create mode 100644 sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql create mode 100644 sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql create mode 100644 sql/hive/src/test/resources/sqlgen/in.sql create mode 100644 sql/hive/src/test/resources/sqlgen/intersect.sql create mode 100644 sql/hive/src/test/resources/sqlgen/join_2_tables.sql create mode 100644 sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/multi_distinct.sql create mode 100644 sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/not_in.sql create mode 100644 sql/hive/src/test/resources/sqlgen/not_like.sql create mode 100644 sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql create mode 100644 sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql create mode 100644 sql/hive/src/test/resources/sqlgen/select_distinct.sql create mode 100644 sql/hive/src/test/resources/sqlgen/select_orc_table.sql create mode 100644 sql/hive/src/test/resources/sqlgen/select_parquet_table.sql create mode 100644 sql/hive/src/test/resources/sqlgen/self_join.sql create mode 100644 sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql create mode 100644 sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql create mode 100644 sql/hive/src/test/resources/sqlgen/tablesample_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/tablesample_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/tablesample_3.sql create mode 100644 sql/hive/src/test/resources/sqlgen/tablesample_4.sql create mode 100644 sql/hive/src/test/resources/sqlgen/tablesample_5.sql create mode 100644 sql/hive/src/test/resources/sqlgen/tablesample_6.sql create mode 100644 sql/hive/src/test/resources/sqlgen/three_child_union.sql create mode 100644 sql/hive/src/test/resources/sqlgen/type_widening.sql create mode 100644 sql/hive/src/test/resources/sqlgen/union_distinct.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_basic_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_basic_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_basic_3.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_with_join.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql diff --git a/sql/hive/src/test/resources/sqlgen/agg1.sql b/sql/hive/src/test/resources/sqlgen/agg1.sql new file mode 100644 index 0000000000000..9953640a6fa1c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/agg1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/agg2.sql b/sql/hive/src/test/resources/sqlgen/agg2.sql new file mode 100644 index 0000000000000..0caea28b0240a --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/agg2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY MAX(key) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` ORDER BY `gen_attr` ASC) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/agg3.sql b/sql/hive/src/test/resources/sqlgen/agg3.sql new file mode 100644 index 0000000000000..437afa7790773 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/agg3.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql new file mode 100644 index 0000000000000..3a29bcf2ae442 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT MAX(c) + COUNT(a) OVER () FROM parquet_t2 GROUP BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `(max(c) + count(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, count(`gen_attr`) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/case.sql b/sql/hive/src/test/resources/sqlgen/case.sql new file mode 100644 index 0000000000000..f6a8c32cf0121 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/case.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((id % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END` FROM (SELECT CASE WHEN ((`gen_attr` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((`gen_attr` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_else.sql b/sql/hive/src/test/resources/sqlgen/case_with_else.sql new file mode 100644 index 0000000000000..8f1595d4b8cee --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/case_with_else.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT CASE WHEN id % 2 > 0 THEN 0 ELSE 1 END FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END` FROM (SELECT CASE WHEN ((`gen_attr` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key.sql b/sql/hive/src/test/resources/sqlgen/case_with_key.sql new file mode 100644 index 0000000000000..88353e321e1d2 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/case_with_key.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM (SELECT CASE WHEN (`gen_attr` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr` = CAST(1 AS BIGINT)) THEN "bar" END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql new file mode 100644 index 0000000000000..bc44f25d1ff8b --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM (SELECT CASE WHEN (`gen_attr` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr` = CAST(1 AS BIGINT)) THEN "bar" ELSE "baz" END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/cluster_by.sql b/sql/hive/src/test/resources/sqlgen/cluster_by.sql new file mode 100644 index 0000000000000..3bc1dfc581882 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/cluster_by.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM parquet_t0 CLUSTER BY id +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql new file mode 100644 index 0000000000000..a4f46da9ffb75 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM json_parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`json_parquet_t0`) AS gen_subquery_0) AS json_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql new file mode 100644 index 0000000000000..214dbcf23625d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM orc_parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`orc_parquet_t0`) AS gen_subquery_0) AS orc_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql new file mode 100644 index 0000000000000..eccf03eb122cb --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM parquet_parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_parquet_t0`) AS gen_subquery_0) AS parquet_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql b/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql new file mode 100644 index 0000000000000..0fa21ad3ba423 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(DISTINCT id) FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `count(DISTINCT id)` FROM (SELECT count(DISTINCT `gen_attr`) AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/distribute_by.sql b/sql/hive/src/test/resources/sqlgen/distribute_by.sql new file mode 100644 index 0000000000000..75a1a2a6b60bc --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/distribute_by.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM parquet_t0 DISTRIBUTE BY id +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 DISTRIBUTE BY `gen_attr`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql b/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql new file mode 100644 index 0000000000000..a8e766e81db91 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM parquet_t0 DISTRIBUTE BY id SORT BY id +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/except.sql b/sql/hive/src/test/resources/sqlgen/except.sql new file mode 100644 index 0000000000000..b1c64288d6e6b --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/except.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT * FROM t0 EXCEPT SELECT * FROM t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 ) EXCEPT ( SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql b/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql new file mode 100644 index 0000000000000..0156dffd0a9f7 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a > 5 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a` FROM (SELECT `gen_attr` FROM (SELECT (`gen_attr` + CAST(1 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t WHERE (`gen_attr` > CAST(5 AS BIGINT))) AS t diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql new file mode 100644 index 0000000000000..15d23c67aec3c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(arr) AS val, id +FROM parquet_t3 +WHERE id > 2 +ORDER BY val, id +LIMIT 5 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 WHERE (`gen_attr` > CAST(2 AS BIGINT))) AS gen_subquery_1 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC LIMIT 5) AS parquet_t3 diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql new file mode 100644 index 0000000000000..e2b94ce6739a6 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql @@ -0,0 +1,10 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT val, id +FROM parquet_t3 +LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array +LATERAL VIEW EXPLODE(nested_array) exp1 AS val +WHERE val > 2 +ORDER BY val, id +LIMIT 5 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW explode(`gen_attr`) gen_subquery_3 AS `gen_attr` WHERE (`gen_attr` > CAST(2 AS BIGINT)) ORDER BY `gen_attr` ASC, `gen_attr` ASC LIMIT 5) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql new file mode 100644 index 0000000000000..c62b895c67403 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT val, id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS val +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql new file mode 100644 index 0000000000000..4c68eef214fb1 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT val, id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS val +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql new file mode 100644 index 0000000000000..059c45ec47c02 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(ARRAY(1,2,3)) FROM t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql new file mode 100644 index 0000000000000..8f03dedbf971f --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(ARRAY(1,2,3)) AS val FROM t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql new file mode 100644 index 0000000000000..0bfe38036ebf7 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(arr), id FROM parquet_t3 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `col`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_1 AS `gen_attr`) AS parquet_t3 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql new file mode 100644 index 0000000000000..e4edb6876cac1 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(arr) AS val, id as a FROM parquet_t3 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val`, `gen_attr` AS `a` FROM (SELECT `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql new file mode 100644 index 0000000000000..5cf57f82d2df2 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(arr) FROM parquet_t3 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql new file mode 100644 index 0000000000000..ed42894807d25 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(arr) AS val FROM parquet_t3 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql new file mode 100644 index 0000000000000..5a4aec961a5e0 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT exp.id, parquet_t3.id +FROM parquet_t3 +LATERAL VIEW EXPLODE(arr) exp AS id +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql new file mode 100644 index 0000000000000..3c3e858deda7c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT exp.id, parquet_t3.id +FROM parquet_t3 +LATERAL VIEW OUTER EXPLODE(arr) exp AS id +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql b/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql new file mode 100644 index 0000000000000..d56e89c8bb45a --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(ARRAY(1,2,3)) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_0 diff --git a/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql b/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql new file mode 100644 index 0000000000000..d7acb6924076a --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT EXPLODE(ARRAY(1,2,3)) AS val +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_0 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql new file mode 100644 index 0000000000000..72e429fe05b32 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id() AS k3 +FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 +GROUPING SETS (key % 5, key - 5) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))))) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql new file mode 100644 index 0000000000000..25e86e73cbb7e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a, b) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`), (`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql new file mode 100644 index 0000000000000..24bf02043488d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql new file mode 100644 index 0000000000000..f72bc8acd69a4 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (b) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql new file mode 100644 index 0000000000000..2fac7f661809e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (()) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS(()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql new file mode 100644 index 0000000000000..2e60c9c4d2c6c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b +GROUPING SETS ((), (a), (a, b)) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((), (`gen_attr`), (`gen_attr`, `gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/in.sql b/sql/hive/src/test/resources/sqlgen/in.sql new file mode 100644 index 0000000000000..1e0251e586e20 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/in.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM parquet_t0 WHERE id IN (1, 2, 3) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 WHERE (CAST(`gen_attr` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT)))) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/intersect.sql b/sql/hive/src/test/resources/sqlgen/intersect.sql new file mode 100644 index 0000000000000..608fa59f8afbc --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/intersect.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT * FROM t0 INTERSECT SELECT * FROM t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 ) INTERSECT ( SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/join_2_tables.sql b/sql/hive/src/test/resources/sqlgen/join_2_tables.sql new file mode 100644 index 0000000000000..bb99797b8bcc6 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/join_2_tables.sql @@ -0,0 +1,7 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(a.value), b.KEY, a.KEY +FROM parquet_t1 a, parquet_t1 b +GROUP BY a.KEY, b.KEY +HAVING MAX(a.KEY) > 0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `count(value)`, `gen_attr` AS `KEY`, `gen_attr` AS `KEY` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, `gen_attr`, `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 GROUP BY `gen_attr`, `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql new file mode 100644 index 0000000000000..9e5fe1156e5de --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT c0, c1, c2 +FROM parquet_t3 +LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `c0`, `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr`, `gen_attr`, `gen_attr`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql new file mode 100644 index 0000000000000..55dd491a32a32 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, c +FROM parquet_t3 +LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt AS a, b, c +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `c` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr`, `gen_attr`, `gen_attr`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/multi_distinct.sql b/sql/hive/src/test/resources/sqlgen/multi_distinct.sql new file mode 100644 index 0000000000000..b5c7cf0b0ce1a --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/multi_distinct.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM parquet_t2 GROUP BY a +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `count(DISTINCT b)`, `gen_attr` AS `count(DISTINCT c)`, `gen_attr` AS `sum(d)` FROM (SELECT `gen_attr`, count(DISTINCT `gen_attr`) AS `gen_attr`, count(DISTINCT `gen_attr`) AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`) AS parquet_t2 diff --git a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql new file mode 100644 index 0000000000000..3d79f95654585 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql @@ -0,0 +1,7 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT val, id +FROM parquet_t3 +LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array +LATERAL VIEW EXPLODE(nested_array) exp1 AS val +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW explode(`gen_attr`) gen_subquery_3 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql new file mode 100644 index 0000000000000..18b29a7bc5eea --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql @@ -0,0 +1,7 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT val, id +FROM parquet_t3 +LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array +LATERAL VIEW OUTER EXPLODE(nested_array) exp1 AS val +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_3 AS `gen_attr`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/not_in.sql b/sql/hive/src/test/resources/sqlgen/not_in.sql new file mode 100644 index 0000000000000..a50ee282e534e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/not_in.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM t0 WHERE id NOT IN (1, 2, 3) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT (CAST(`gen_attr` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT))))) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/not_like.sql b/sql/hive/src/test/resources/sqlgen/not_like.sql new file mode 100644 index 0000000000000..2f66695189cba --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/not_like.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM t0 WHERE id + 5 NOT LIKE '1%' +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT CAST((`gen_attr` + CAST(5 AS BIGINT)) AS STRING) LIKE "1%")) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql new file mode 100644 index 0000000000000..7d837d490f125 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT MAX(key) OVER (PARTITION BY key % 3) + key FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql new file mode 100644 index 0000000000000..4b2c78ea40c78 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `cnt`, `gen_attr` AS `(key % CAST(5 AS BIGINT))`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql new file mode 100644 index 0000000000000..62cf1bb9d8e0c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `cnt`, `gen_attr` AS `(key % CAST(5 AS BIGINT))`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql new file mode 100644 index 0000000000000..d58910af6939b --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql new file mode 100644 index 0000000000000..9a58c15203d1e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql new file mode 100644 index 0000000000000..cd4d428b07ede --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `count(value)`, `gen_attr` AS `grouping_id()` FROM (SELECT `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql new file mode 100644 index 0000000000000..4f42d7eda8901 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `count(value)`, `gen_attr` AS `grouping_id()` FROM (SELECT `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql new file mode 100644 index 0000000000000..be4908ea16e52 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 +GROUP BY key % 5, key - 5 WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql new file mode 100644 index 0000000000000..eebef6a5f3b89 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 +GROUP BY key % 5, key - 5 WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql new file mode 100644 index 0000000000000..9474233cb7528 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - 5) AS k3 +FROM (SELECT key, key%2, key - 5 FROM parquet_t1) t GROUP BY key%5, key-5 +WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql new file mode 100644 index 0000000000000..d36f43d665cda --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - 5) AS k3 +FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 +WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql new file mode 100644 index 0000000000000..de980784c7215 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(c) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql new file mode 100644 index 0000000000000..8956ac4c42f86 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(c) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql new file mode 100644 index 0000000000000..b9cef62dcfdfa --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(a) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql new file mode 100644 index 0000000000000..ba6457ab1a54d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, sum(a) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql new file mode 100644 index 0000000000000..763fecd9517fa --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `(a + b)`, `gen_attr` AS `b`, `gen_attr` AS `sum((a - b))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum((`gen_attr` - `gen_attr`)) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr` + `gen_attr`), `gen_attr` GROUPING SETS(((`gen_attr` + `gen_attr`), `gen_attr`), ((`gen_attr` + `gen_attr`)), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql new file mode 100644 index 0000000000000..00ecbf6078e00 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `(a + b)`, `gen_attr` AS `b`, `gen_attr` AS `sum((a - b))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum((`gen_attr` - `gen_attr`)) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr` + `gen_attr`), `gen_attr` GROUPING SETS(((`gen_attr` + `gen_attr`), `gen_attr`), ((`gen_attr` + `gen_attr`)), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql new file mode 100644 index 0000000000000..6f57130f3ea2c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, grouping_id(a, b) FROM parquet_t2 GROUP BY cube(a, b) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping_id(a, b)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql new file mode 100644 index 0000000000000..dab7852a3e967 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, grouping(b) FROM parquet_t2 GROUP BY cube(a, b) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping(b)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql new file mode 100644 index 0000000000000..50964e5b01b6c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a, b, grouping(a) FROM parquet_t2 GROUP BY cube(a, b) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql new file mode 100644 index 0000000000000..84af580a36250 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid +FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 +WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `hgid` FROM (SELECT `gen_attr` AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, hash(grouping_id()) AS `gen_attr` FROM (SELECT hash(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql new file mode 100644 index 0000000000000..b0f65b7d41bfb --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid +FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 +WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `hgid` FROM (SELECT `gen_attr` AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, hash(grouping_id()) AS `gen_attr` FROM (SELECT hash(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql new file mode 100644 index 0000000000000..171400dd78cd1 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT t.key - 5, cnt, SUM(cnt) +FROM (SELECT x.key, COUNT(*) as cnt +FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t +GROUP BY cnt, t.key - 5 +WITH ROLLUP +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `(key - CAST(5 AS BIGINT))`, `gen_attr` AS `cnt`, `gen_attr` AS `sum(cnt)` FROM (SELECT (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ())) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql new file mode 100644 index 0000000000000..fe98b6c343cc6 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT t.key - 5, cnt, SUM(cnt) +FROM (SELECT x.key, COUNT(*) as cnt +FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t +GROUP BY cnt, t.key - 5 +WITH CUBE +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `(key - CAST(5 AS BIGINT))`, `gen_attr` AS `cnt`, `gen_attr` AS `sum(cnt)` FROM (SELECT (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql new file mode 100644 index 0000000000000..75e43d0dc1cfe --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT TRANSFORM (a, b, c, d) USING 'cat' FROM parquet_t2 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) USING 'cat' AS (`gen_attr` string, `gen_attr` string) FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql new file mode 100644 index 0000000000000..6a68edc72ebc3 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT TRANSFORM (*) USING 'cat' FROM parquet_t2 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) USING 'cat' AS (`gen_attr` string, `gen_attr` string) FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql new file mode 100644 index 0000000000000..111df4bdf2299 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT TRANSFORM (a, b, c, d) USING 'cat' AS (d1, d2, d3, d4) FROM parquet_t2 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `d1`, `gen_attr` AS `d2`, `gen_attr` AS `d3`, `gen_attr` AS `d4` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr` string, `gen_attr` string, `gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql new file mode 100644 index 0000000000000..4ab412cfd171b --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +FROM +(FROM parquet_t1 SELECT TRANSFORM(key, value) USING 'cat' AS (thing1 int, thing2 string)) t +SELECT thing1 + 1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `(thing1 + 1)` FROM (SELECT (`gen_attr` + 1) AS `gen_attr` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr` int, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql new file mode 100644 index 0000000000000..1e30e45d45d31 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT TRANSFORM (key) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' +USING 'cat' AS (tKey) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' +FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' USING 'cat' AS (`gen_attr` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql new file mode 100644 index 0000000000000..dd622898780ac --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' +FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (`gen_attr` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql new file mode 100644 index 0000000000000..2ad369815742f --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql @@ -0,0 +1,10 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT TRANSFORM (key, value) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +WITH SERDEPROPERTIES('field.delim' = '|') +USING 'cat' AS (tKey, tValue) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +WITH SERDEPROPERTIES('field.delim' = '|') +FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `tKey`, `gen_attr` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') USING 'cat' AS (`gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql new file mode 100644 index 0000000000000..a90b42d9b4aed --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT TRANSFORM (key, value) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +USING 'cat' AS (tKey, tValue) +ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `tKey`, `gen_attr` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (`gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/select_distinct.sql b/sql/hive/src/test/resources/sqlgen/select_distinct.sql new file mode 100644 index 0000000000000..3bc8e55379281 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/select_distinct.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT DISTINCT id FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT DISTINCT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/select_orc_table.sql b/sql/hive/src/test/resources/sqlgen/select_orc_table.sql new file mode 100644 index 0000000000000..eae67f9540381 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/select_orc_table.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * from orc_t +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `c1` AS `gen_attr`, `c2` AS `gen_attr` FROM `default`.`orc_t`) AS gen_subquery_0) AS orc_t diff --git a/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql b/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql new file mode 100644 index 0000000000000..a085bab84c9df --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * from parquet_t +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `c1` AS `gen_attr`, `c2` AS `gen_attr` FROM `default`.`parquet_t`) AS gen_subquery_0) AS parquet_t diff --git a/sql/hive/src/test/resources/sqlgen/self_join.sql b/sql/hive/src/test/resources/sqlgen/self_join.sql new file mode 100644 index 0000000000000..8947ccda820a5 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/self_join.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT x.key FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key` FROM (SELECT `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`)) AS x diff --git a/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql b/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql new file mode 100644 index 0000000000000..6db053f286028 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT x.key, COUNT(*) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key group by x.key +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `count(1)` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS x diff --git a/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql new file mode 100644 index 0000000000000..e9a6afdc7eda1 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 SORT BY key +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_1 SORT BY `gen_attr` ASC) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_1.sql b/sql/hive/src/test/resources/sqlgen/tablesample_1.sql new file mode 100644 index 0000000000000..54c8deeb65679 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/tablesample_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT s.id FROM parquet_t0 TABLESAMPLE(100 PERCENT) s +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_2.sql b/sql/hive/src/test/resources/sqlgen/tablesample_2.sql new file mode 100644 index 0000000000000..13dcadb41a87f --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/tablesample_2.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT * FROM parquet_t0 TABLESAMPLE(100 PERCENT) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_3.sql b/sql/hive/src/test/resources/sqlgen/tablesample_3.sql new file mode 100644 index 0000000000000..c1ed57f5e3f2e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/tablesample_3.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT s.id FROM t0 TABLESAMPLE(100 PERCENT) s +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_4.sql b/sql/hive/src/test/resources/sqlgen/tablesample_4.sql new file mode 100644 index 0000000000000..c22b20802c41a --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/tablesample_4.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT * FROM t0 TABLESAMPLE(100 PERCENT) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_5.sql b/sql/hive/src/test/resources/sqlgen/tablesample_5.sql new file mode 100644 index 0000000000000..fa69dd46c7462 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/tablesample_5.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT s.id FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) s WHERE 1=0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_6.sql b/sql/hive/src/test/resources/sqlgen/tablesample_6.sql new file mode 100644 index 0000000000000..bc72560c42905 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/tablesample_6.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT * FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) WHERE 1=0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/three_child_union.sql b/sql/hive/src/test/resources/sqlgen/three_child_union.sql new file mode 100644 index 0000000000000..c3781117d814d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/three_child_union.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM parquet_t0 +UNION ALL SELECT id FROM parquet_t0 +UNION ALL SELECT id FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_1) UNION ALL (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_2)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/type_widening.sql b/sql/hive/src/test/resources/sqlgen/type_widening.sql new file mode 100644 index 0000000000000..96e6cc2fec90d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/type_widening.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT id FROM parquet_t0 UNION ALL SELECT CAST(id AS INT) AS id FROM parquet_t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT CAST(CAST(`gen_attr` AS INT) AS BIGINT) AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_1)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/union_distinct.sql b/sql/hive/src/test/resources/sqlgen/union_distinct.sql new file mode 100644 index 0000000000000..c5895ae766d2e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/union_distinct.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT * FROM t0 UNION SELECT * FROM t0 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0) UNION DISTINCT (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_1.sql b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql new file mode 100644 index 0000000000000..73f343d1018c1 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT MAX(value) OVER (PARTITION BY key % 3) FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql new file mode 100644 index 0000000000000..3ddb87f91b88d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, ROUND(AVG(key) OVER (), 2) +FROM parquet_t1 ORDER BY key +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `round(avg(key) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr`, `gen_attr`, round(`gen_attr`, 2) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, avg(`gen_attr`) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr` ASC) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_3.sql b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql new file mode 100644 index 0000000000000..43f992882ce28 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT value, MAX(key + 1) OVER (PARTITION BY key % 5 ORDER BY key % 7) AS max +FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` + CAST(1 AS BIGINT)) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` % CAST(7 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_join.sql b/sql/hive/src/test/resources/sqlgen/window_with_join.sql new file mode 100644 index 0000000000000..00c45c8feafaa --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_with_join.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT x.key, MAX(y.key) OVER (PARTITION BY x.key % 5 ORDER BY x.key) +FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_2.`gen_attr`, gen_subquery_2.`gen_attr`, gen_subquery_2.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`)) AS gen_subquery_2) AS gen_subquery_3) AS x diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql new file mode 100644 index 0000000000000..65d3d3aeb623e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql @@ -0,0 +1,7 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, +DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, +COUNT(key) +FROM parquet_t1 GROUP BY key, value +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `dr`, `gen_attr` AS `count(key)` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, DENSE_RANK() OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql new file mode 100644 index 0000000000000..03e0962f44943 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql @@ -0,0 +1,7 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, +DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, +COUNT(key) OVER(DISTRIBUTE BY key SORT BY key, value) AS ca +FROM parquet_t1 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `dr`, `gen_attr` AS `ca` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, DENSE_RANK() OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr`, count(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql new file mode 100644 index 0000000000000..090207f9b8352 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, +MAX(value) OVER (PARTITION BY key % 5 ORDER BY key) AS max +FROM parquet_t1 GROUP BY key, value +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql new file mode 100644 index 0000000000000..fcc2cf7de80a6 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, +MAX(value) OVER (PARTITION BY key % 5 ORDER BY key DESC) AS max +FROM parquet_t1 GROUP BY key, value HAVING key > 5 +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` HAVING (`gen_attr` > CAST(5 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 0827b04252bc4..698c7c3e3c2fa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -17,15 +17,33 @@ package org.apache.spark.sql.catalyst +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, NoSuchFileException, Paths} + import scala.util.control.NonFatal import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SQLTestUtils +/** + * A test suite for LogicalPlan-to-SQL conversion. + * + * Each query has a golden generated SQL file in test/resources/sqlgen. The test suite also has + * built-in functionality to automatically generate these golden files. + * + * To re-generate golden files, run: + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "hive/test-only *LogicalPlanToSQLSuite" + */ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { import testImplicits._ + // Used for generating new query answer files by saving + private val regenerateGoldenFiles = + Option(System.getenv("SPARK_GENERATE_GOLDEN_FILES")).contains("1") + private val goldenSQLPath = "src/test/resources/sqlgen/" + protected override def beforeAll(): Unit = { super.beforeAll() sql("DROP TABLE IF EXISTS parquet_t0") @@ -76,22 +94,60 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } } - private def checkHiveQl(hiveQl: String): Unit = { - val df = sql(hiveQl) + /** + * Compare the generated SQL with the expected answer string. + * Note that there exists a normalization for both arguments for the convenience. + * - Remove the id from the generated attributes, e.g., `gen_attr_1` -> `gen_attr`. + */ + private def checkSQLStructure(originalSQL: String, convertedSQL: String, answerFile: String) = { + val normalizedGenSQL = convertedSQL.replaceAll("`gen_attr_\\d+`", "`gen_attr`") + if (answerFile != null) { + val separator = "-" * 80 + if (regenerateGoldenFiles) { + val path = Paths.get(s"$goldenSQLPath/$answerFile.sql") + val header = "-- This file is automatically generated by LogicalPlanToSQLSuite." + val answerText = s"$header\n${originalSQL.trim()}\n${separator}\n$normalizedGenSQL\n" + Files.write(path, answerText.getBytes(StandardCharsets.UTF_8)) + } else { + val goldenFileName = s"sqlgen/$answerFile.sql" + val resourceFile = getClass.getClassLoader.getResource(goldenFileName) + if (resourceFile == null) { + throw new NoSuchFileException(goldenFileName) + } + val path = resourceFile.getPath + val answerText = new String(Files.readAllBytes(Paths.get(path)), StandardCharsets.UTF_8) + val sqls = answerText.split(separator) + assert(sqls.length == 2, "Golden sql files should have a separator.") + val normalizedExpectSQL = sqls(1).trim() + assert(normalizedGenSQL == normalizedExpectSQL) + } + } + } + + /** + * 1. Checks if SQL parsing succeeds. + * 2. Checks if SQL generation succeeds. + * 3. Checks the generated SQL against golden files. + * 4. Verifies the execution result stays the same. + */ + private def checkSQL(sqlString: String, answerFile: String = null): Unit = { + val df = sql(sqlString) val convertedSQL = try new SQLBuilder(df).toSQL catch { case NonFatal(e) => fail( - s"""Cannot convert the following HiveQL query plan back to SQL query string: + s"""Cannot convert the following SQL query plan back to SQL query string: | - |# Original HiveQL query string: - |$hiveQl + |# Original SQL query string: + |$sqlString | |# Resolved query plan: |${df.queryExecution.analyzed.treeString} """.stripMargin, e) } + checkSQLStructure(sqlString, convertedSQL, answerFile) + try { checkAnswer(sql(convertedSQL), df) } catch { case cause: Throwable => @@ -101,8 +157,8 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { |# Converted SQL query string: |$convertedSQL | - |# Original HiveQL query string: - |$hiveQl + |# Original SQL query string: + |$sqlString | |# Resolved query plan: |${df.queryExecution.analyzed.treeString} @@ -110,24 +166,57 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } } + // When saving golden files, these tests should be ignored to prevent making files. + if (!regenerateGoldenFiles) { + test("Test should fail if the SQL query cannot be parsed") { + val m = intercept[ParseException] { + checkSQL("SELE", "NOT_A_FILE") + }.getMessage + assert(m.contains("mismatched input")) + } + + test("Test should fail if the golden file cannot be found") { + val m2 = intercept[NoSuchFileException] { + checkSQL("SELECT 1", "NOT_A_FILE") + }.getMessage + assert(m2.contains("NOT_A_FILE")) + } + + test("Test should fail if the SQL query cannot be regenerated") { + spark.range(10).createOrReplaceTempView("not_sql_gen_supported_table_so_far") + sql("select * from not_sql_gen_supported_table_so_far") + val m3 = intercept[org.scalatest.exceptions.TestFailedException] { + checkSQL("select * from not_sql_gen_supported_table_so_far", "in") + }.getMessage + assert(m3.contains("Cannot convert the following SQL query plan back to SQL query string")) + } + + test("Test should fail if the SQL query did not equal to the golden SQL") { + val m4 = intercept[org.scalatest.exceptions.TestFailedException] { + checkSQL("SELECT 1", "in") + }.getMessage + assert(m4.contains("did not equal")) + } + } + test("in") { - checkHiveQl("SELECT id FROM parquet_t0 WHERE id IN (1, 2, 3)") + checkSQL("SELECT id FROM parquet_t0 WHERE id IN (1, 2, 3)", "in") } test("not in") { - checkHiveQl("SELECT id FROM t0 WHERE id NOT IN (1, 2, 3)") + checkSQL("SELECT id FROM t0 WHERE id NOT IN (1, 2, 3)", "not_in") } test("not like") { - checkHiveQl("SELECT id FROM t0 WHERE id + 5 NOT LIKE '1%'") + checkSQL("SELECT id FROM t0 WHERE id + 5 NOT LIKE '1%'", "not_like") } test("aggregate function in having clause") { - checkHiveQl("SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0") + checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0", "agg1") } test("aggregate function in order by clause") { - checkHiveQl("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY MAX(key)") + checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY MAX(key)", "agg2") } // When there are multiple aggregate functions in ORDER BY clause, all of them are extracted into @@ -135,61 +224,67 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { // execution since these aliases have different expression ID. But this introduces name collision // when converting resolved plans back to SQL query strings as expression IDs are stripped. test("aggregate function in order by clause with multiple order keys") { - checkHiveQl("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key)") + checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key)", "agg3") } test("type widening in union") { - checkHiveQl("SELECT id FROM parquet_t0 UNION ALL SELECT CAST(id AS INT) AS id FROM parquet_t0") + checkSQL("SELECT id FROM parquet_t0 UNION ALL SELECT CAST(id AS INT) AS id FROM parquet_t0", + "type_widening") } test("union distinct") { - checkHiveQl("SELECT * FROM t0 UNION SELECT * FROM t0") + checkSQL("SELECT * FROM t0 UNION SELECT * FROM t0", "union_distinct") } test("three-child union") { - checkHiveQl( + checkSQL( """ |SELECT id FROM parquet_t0 |UNION ALL SELECT id FROM parquet_t0 |UNION ALL SELECT id FROM parquet_t0 - """.stripMargin) + """.stripMargin, + "three_child_union") } test("intersect") { - checkHiveQl("SELECT * FROM t0 INTERSECT SELECT * FROM t0") + checkSQL("SELECT * FROM t0 INTERSECT SELECT * FROM t0", "intersect") } test("except") { - checkHiveQl("SELECT * FROM t0 EXCEPT SELECT * FROM t0") + checkSQL("SELECT * FROM t0 EXCEPT SELECT * FROM t0", "except") } test("self join") { - checkHiveQl("SELECT x.key FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key") + checkSQL("SELECT x.key FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key", "self_join") } test("self join with group by") { - checkHiveQl( - "SELECT x.key, COUNT(*) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key group by x.key") + checkSQL( + "SELECT x.key, COUNT(*) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key group by x.key", + "self_join_with_group_by") } test("case") { - checkHiveQl("SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM parquet_t0") + checkSQL("SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM parquet_t0", + "case") } test("case with else") { - checkHiveQl("SELECT CASE WHEN id % 2 > 0 THEN 0 ELSE 1 END FROM parquet_t0") + checkSQL("SELECT CASE WHEN id % 2 > 0 THEN 0 ELSE 1 END FROM parquet_t0", "case_with_else") } test("case with key") { - checkHiveQl("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM parquet_t0") + checkSQL("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM parquet_t0", + "case_with_key") } test("case with key and else") { - checkHiveQl("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM parquet_t0") + checkSQL("SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM parquet_t0", + "case_with_key_and_else") } test("select distinct without aggregate functions") { - checkHiveQl("SELECT DISTINCT id FROM parquet_t0") + checkSQL("SELECT DISTINCT id FROM parquet_t0", "select_distinct") } test("rollup/cube #1") { @@ -213,146 +308,195 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { // FROM `default`.`t1` // GROUP BY (`t1`.`key` % CAST(5 AS BIGINT)) // GROUPING SETS (((`t1`.`key` % CAST(5 AS BIGINT))), ()) - checkHiveQl( - "SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH ROLLUP") - checkHiveQl( - "SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH CUBE") + checkSQL( + "SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH ROLLUP", + "rollup_cube_1_1") + + checkSQL( + "SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH CUBE", + "rollup_cube_1_2") } test("rollup/cube #2") { - checkHiveQl("SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH ROLLUP") - checkHiveQl("SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH CUBE") + checkSQL("SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH ROLLUP", + "rollup_cube_2_1") + + checkSQL("SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH CUBE", + "rollup_cube_2_2") } test("rollup/cube #3") { - checkHiveQl( - "SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH ROLLUP") - checkHiveQl( - "SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH CUBE") + checkSQL( + "SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH ROLLUP", + "rollup_cube_3_1") + + checkSQL( + "SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH CUBE", + "rollup_cube_3_2") } test("rollup/cube #4") { - checkHiveQl( + checkSQL( s""" |SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 |GROUP BY key % 5, key - 5 WITH ROLLUP - """.stripMargin) - checkHiveQl( + """.stripMargin, + "rollup_cube_4_1") + + checkSQL( s""" |SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 |GROUP BY key % 5, key - 5 WITH CUBE - """.stripMargin) + """.stripMargin, + "rollup_cube_4_2") } test("rollup/cube #5") { - checkHiveQl( + checkSQL( s""" |SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - 5) AS k3 |FROM (SELECT key, key%2, key - 5 FROM parquet_t1) t GROUP BY key%5, key-5 |WITH ROLLUP - """.stripMargin) - checkHiveQl( + """.stripMargin, + "rollup_cube_5_1") + + checkSQL( s""" |SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - 5) AS k3 |FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 |WITH CUBE - """.stripMargin) + """.stripMargin, + "rollup_cube_5_2") } test("rollup/cube #6") { - checkHiveQl("SELECT a, b, sum(c) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b") - checkHiveQl("SELECT a, b, sum(c) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b") - checkHiveQl("SELECT a, b, sum(a) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b") - checkHiveQl("SELECT a, b, sum(a) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b") - checkHiveQl("SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH ROLLUP") - checkHiveQl("SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH CUBE") + checkSQL("SELECT a, b, sum(c) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b", + "rollup_cube_6_1") + + checkSQL("SELECT a, b, sum(c) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b", + "rollup_cube_6_2") + + checkSQL("SELECT a, b, sum(a) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b", + "rollup_cube_6_3") + + checkSQL("SELECT a, b, sum(a) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b", + "rollup_cube_6_4") + + checkSQL("SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH ROLLUP", + "rollup_cube_6_5") + + checkSQL("SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH CUBE", + "rollup_cube_6_6") } test("rollup/cube #7") { - checkHiveQl("SELECT a, b, grouping_id(a, b) FROM parquet_t2 GROUP BY cube(a, b)") - checkHiveQl("SELECT a, b, grouping(b) FROM parquet_t2 GROUP BY cube(a, b)") - checkHiveQl("SELECT a, b, grouping(a) FROM parquet_t2 GROUP BY cube(a, b)") + checkSQL("SELECT a, b, grouping_id(a, b) FROM parquet_t2 GROUP BY cube(a, b)", + "rollup_cube_7_1") + + checkSQL("SELECT a, b, grouping(b) FROM parquet_t2 GROUP BY cube(a, b)", + "rollup_cube_7_2") + + checkSQL("SELECT a, b, grouping(a) FROM parquet_t2 GROUP BY cube(a, b)", + "rollup_cube_7_3") } test("rollup/cube #8") { // grouping_id() is part of another expression - checkHiveQl( + checkSQL( s""" |SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid |FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 |WITH ROLLUP - """.stripMargin) - checkHiveQl( + """.stripMargin, + "rollup_cube_8_1") + + checkSQL( s""" |SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid |FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 |WITH CUBE - """.stripMargin) + """.stripMargin, + "rollup_cube_8_2") } test("rollup/cube #9") { // self join is used as the child node of ROLLUP/CUBE with replaced quantifiers - checkHiveQl( + checkSQL( s""" |SELECT t.key - 5, cnt, SUM(cnt) |FROM (SELECT x.key, COUNT(*) as cnt |FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t |GROUP BY cnt, t.key - 5 |WITH ROLLUP - """.stripMargin) - checkHiveQl( + """.stripMargin, + "rollup_cube_9_1") + + checkSQL( s""" |SELECT t.key - 5, cnt, SUM(cnt) |FROM (SELECT x.key, COUNT(*) as cnt |FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t |GROUP BY cnt, t.key - 5 |WITH CUBE - """.stripMargin) + """.stripMargin, + "rollup_cube_9_2") } test("grouping sets #1") { - checkHiveQl( + checkSQL( s""" |SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id() AS k3 |FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 |GROUPING SETS (key % 5, key - 5) - """.stripMargin) + """.stripMargin, + "grouping_sets_1") } test("grouping sets #2") { - checkHiveQl( - "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a, b) ORDER BY a, b") - checkHiveQl( - "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a) ORDER BY a, b") - checkHiveQl( - "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (b) ORDER BY a, b") - checkHiveQl( - "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (()) ORDER BY a, b") - checkHiveQl( + checkSQL( + "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a, b) ORDER BY a, b", + "grouping_sets_2_1") + + checkSQL( + "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a) ORDER BY a, b", + "grouping_sets_2_2") + + checkSQL( + "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (b) ORDER BY a, b", + "grouping_sets_2_3") + + checkSQL( + "SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (()) ORDER BY a, b", + "grouping_sets_2_4") + + checkSQL( s""" |SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b |GROUPING SETS ((), (a), (a, b)) ORDER BY a, b - """.stripMargin) + """.stripMargin, + "grouping_sets_2_5") } test("cluster by") { - checkHiveQl("SELECT id FROM parquet_t0 CLUSTER BY id") + checkSQL("SELECT id FROM parquet_t0 CLUSTER BY id", "cluster_by") } test("distribute by") { - checkHiveQl("SELECT id FROM parquet_t0 DISTRIBUTE BY id") + checkSQL("SELECT id FROM parquet_t0 DISTRIBUTE BY id", "distribute_by") } test("distribute by with sort by") { - checkHiveQl("SELECT id FROM parquet_t0 DISTRIBUTE BY id SORT BY id") + checkSQL("SELECT id FROM parquet_t0 DISTRIBUTE BY id SORT BY id", + "distribute_by_with_sort_by") } test("SPARK-13720: sort by after having") { - checkHiveQl("SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 SORT BY key") + checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 SORT BY key", + "sort_by_after_having") } test("distinct aggregation") { - checkHiveQl("SELECT COUNT(DISTINCT id) FROM parquet_t0") + checkSQL("SELECT COUNT(DISTINCT id) FROM parquet_t0", "distinct_aggregation") } test("TABLESAMPLE") { @@ -361,33 +505,34 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { // +- Subquery s // +- Subquery parquet_t0 // +- Relation[id#2L] ParquetRelation - checkHiveQl("SELECT s.id FROM parquet_t0 TABLESAMPLE(100 PERCENT) s") + checkSQL("SELECT s.id FROM parquet_t0 TABLESAMPLE(100 PERCENT) s", "tablesample_1") // Project [id#2L] // +- Sample 0.0, 1.0, false, ... // +- Subquery parquet_t0 // +- Relation[id#2L] ParquetRelation - checkHiveQl("SELECT * FROM parquet_t0 TABLESAMPLE(100 PERCENT)") + checkSQL("SELECT * FROM parquet_t0 TABLESAMPLE(100 PERCENT)", "tablesample_2") // Project [id#21L] // +- Sample 0.0, 1.0, false, ... // +- MetastoreRelation default, t0, Some(s) - checkHiveQl("SELECT s.id FROM t0 TABLESAMPLE(100 PERCENT) s") + checkSQL("SELECT s.id FROM t0 TABLESAMPLE(100 PERCENT) s", "tablesample_3") // Project [id#24L] // +- Sample 0.0, 1.0, false, ... // +- MetastoreRelation default, t0, None - checkHiveQl("SELECT * FROM t0 TABLESAMPLE(100 PERCENT)") + checkSQL("SELECT * FROM t0 TABLESAMPLE(100 PERCENT)", "tablesample_4") // When a sampling fraction is not 100%, the returned results are random. // Thus, added an always-false filter here to check if the generated plan can be successfully // executed. - checkHiveQl("SELECT s.id FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) s WHERE 1=0") - checkHiveQl("SELECT * FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) WHERE 1=0") + checkSQL("SELECT s.id FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) s WHERE 1=0", "tablesample_5") + checkSQL("SELECT * FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) WHERE 1=0", "tablesample_6") } test("multi-distinct columns") { - checkHiveQl("SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM parquet_t2 GROUP BY a") + checkSQL("SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM parquet_t2 GROUP BY a", + "multi_distinct") } test("persisted data source relations") { @@ -395,48 +540,54 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { val tableName = s"${format}_parquet_t0" withTable(tableName) { spark.range(10).write.format(format).saveAsTable(tableName) - checkHiveQl(s"SELECT id FROM $tableName") + checkSQL(s"SELECT id FROM $tableName", s"data_source_$tableName") } } } test("script transformation - schemaless") { - checkHiveQl("SELECT TRANSFORM (a, b, c, d) USING 'cat' FROM parquet_t2") - checkHiveQl("SELECT TRANSFORM (*) USING 'cat' FROM parquet_t2") + checkSQL("SELECT TRANSFORM (a, b, c, d) USING 'cat' FROM parquet_t2", + "script_transformation_1") + checkSQL("SELECT TRANSFORM (*) USING 'cat' FROM parquet_t2", + "script_transformation_2") } test("script transformation - alias list") { - checkHiveQl("SELECT TRANSFORM (a, b, c, d) USING 'cat' AS (d1, d2, d3, d4) FROM parquet_t2") + checkSQL("SELECT TRANSFORM (a, b, c, d) USING 'cat' AS (d1, d2, d3, d4) FROM parquet_t2", + "script_transformation_alias_list") } test("script transformation - alias list with type") { - checkHiveQl( + checkSQL( """FROM |(FROM parquet_t1 SELECT TRANSFORM(key, value) USING 'cat' AS (thing1 int, thing2 string)) t |SELECT thing1 + 1 - """.stripMargin) + """.stripMargin, + "script_transformation_alias_list_with_type") } test("script transformation - row format delimited clause with only one format property") { - checkHiveQl( + checkSQL( """SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' |FROM parquet_t1 - """.stripMargin) + """.stripMargin, + "script_transformation_row_format_one") } test("script transformation - row format delimited clause with multiple format properties") { - checkHiveQl( + checkSQL( """SELECT TRANSFORM (key) |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' |USING 'cat' AS (tKey) |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' |FROM parquet_t1 - """.stripMargin) + """.stripMargin, + "script_transformation_row_format_multiple") } test("script transformation - row format serde clauses with SERDEPROPERTIES") { - checkHiveQl( + checkSQL( """SELECT TRANSFORM (key, value) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES('field.delim' = '|') @@ -444,17 +595,19 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES('field.delim' = '|') |FROM parquet_t1 - """.stripMargin) + """.stripMargin, + "script_transformation_row_format_serde") } test("script transformation - row format serde clauses without SERDEPROPERTIES") { - checkHiveQl( + checkSQL( """SELECT TRANSFORM (key, value) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |USING 'cat' AS (tKey, tValue) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |FROM parquet_t1 - """.stripMargin) + """.stripMargin, + "script_transformation_row_format_without_serde") } test("plans with non-SQL expressions") { @@ -464,7 +617,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { test("named expression in column names shouldn't be quoted") { def checkColumnNames(query: String, expectedColNames: String*): Unit = { - checkHiveQl(query) + checkSQL(query) assert(sql(query).columns === expectedColNames) } @@ -521,21 +674,25 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } test("window basic") { - checkHiveQl("SELECT MAX(value) OVER (PARTITION BY key % 3) FROM parquet_t1") - checkHiveQl( + checkSQL("SELECT MAX(value) OVER (PARTITION BY key % 3) FROM parquet_t1", "window_basic_1") + + checkSQL( """ |SELECT key, value, ROUND(AVG(key) OVER (), 2) |FROM parquet_t1 ORDER BY key - """.stripMargin) - checkHiveQl( + """.stripMargin, + "window_basic_2") + + checkSQL( """ |SELECT value, MAX(key + 1) OVER (PARTITION BY key % 5 ORDER BY key % 7) AS max |FROM parquet_t1 - """.stripMargin) + """.stripMargin, + "window_basic_3") } test("multiple window functions in one expression") { - checkHiveQl( + checkSQL( """ |SELECT | MAX(key) OVER (ORDER BY key DESC, value) / MIN(key) OVER (PARTITION BY key % 3) @@ -544,15 +701,17 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } test("regular expressions and window functions in one expression") { - checkHiveQl("SELECT MAX(key) OVER (PARTITION BY key % 3) + key FROM parquet_t1") + checkSQL("SELECT MAX(key) OVER (PARTITION BY key % 3) + key FROM parquet_t1", + "regular_expressions_and_window") } test("aggregate functions and window functions in one expression") { - checkHiveQl("SELECT MAX(c) + COUNT(a) OVER () FROM parquet_t2 GROUP BY a, b") + checkSQL("SELECT MAX(c) + COUNT(a) OVER () FROM parquet_t2 GROUP BY a, b", + "aggregate_functions_and_window") } test("window with different window specification") { - checkHiveQl( + checkSQL( """ |SELECT key, value, |DENSE_RANK() OVER (ORDER BY key, value) AS dr, @@ -562,45 +721,49 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } test("window with the same window specification with aggregate + having") { - checkHiveQl( + checkSQL( """ |SELECT key, value, |MAX(value) OVER (PARTITION BY key % 5 ORDER BY key DESC) AS max |FROM parquet_t1 GROUP BY key, value HAVING key > 5 - """.stripMargin) + """.stripMargin, + "window_with_the_same_window_with_agg_having") } test("window with the same window specification with aggregate functions") { - checkHiveQl( + checkSQL( """ |SELECT key, value, |MAX(value) OVER (PARTITION BY key % 5 ORDER BY key) AS max |FROM parquet_t1 GROUP BY key, value - """.stripMargin) + """.stripMargin, + "window_with_the_same_window_with_agg_functions") } test("window with the same window specification with aggregate") { - checkHiveQl( + checkSQL( """ |SELECT key, value, |DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, |COUNT(key) |FROM parquet_t1 GROUP BY key, value - """.stripMargin) + """.stripMargin, + "window_with_the_same_window_with_agg") } test("window with the same window specification without aggregate and filter") { - checkHiveQl( + checkSQL( """ |SELECT key, value, |DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, |COUNT(key) OVER(DISTRIBUTE BY key SORT BY key, value) AS ca |FROM parquet_t1 - """.stripMargin) + """.stripMargin, + "window_with_the_same_window_with_agg_filter") } test("window clause") { - checkHiveQl( + checkSQL( """ |SELECT key, MAX(value) OVER w1 AS MAX, MIN(value) OVER w2 AS min |FROM parquet_t1 @@ -609,7 +772,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } test("special window functions") { - checkHiveQl( + checkSQL( """ |SELECT | RANK() OVER w, @@ -626,107 +789,120 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } test("window with join") { - checkHiveQl( + checkSQL( """ |SELECT x.key, MAX(y.key) OVER (PARTITION BY x.key % 5 ORDER BY x.key) |FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key - """.stripMargin) + """.stripMargin, + "window_with_join") } test("join 2 tables and aggregate function in having clause") { - checkHiveQl( + checkSQL( """ |SELECT COUNT(a.value), b.KEY, a.KEY |FROM parquet_t1 a, parquet_t1 b |GROUP BY a.KEY, b.KEY |HAVING MAX(a.KEY) > 0 - """.stripMargin) + """.stripMargin, + "join_2_tables") } test("generator in project list without FROM clause") { - checkHiveQl("SELECT EXPLODE(ARRAY(1,2,3))") - checkHiveQl("SELECT EXPLODE(ARRAY(1,2,3)) AS val") + checkSQL("SELECT EXPLODE(ARRAY(1,2,3))", "generator_without_from_1") + checkSQL("SELECT EXPLODE(ARRAY(1,2,3)) AS val", "generator_without_from_2") } test("generator in project list with non-referenced table") { - checkHiveQl("SELECT EXPLODE(ARRAY(1,2,3)) FROM t0") - checkHiveQl("SELECT EXPLODE(ARRAY(1,2,3)) AS val FROM t0") + checkSQL("SELECT EXPLODE(ARRAY(1,2,3)) FROM t0", "generator_non_referenced_table_1") + checkSQL("SELECT EXPLODE(ARRAY(1,2,3)) AS val FROM t0", "generator_non_referenced_table_2") } test("generator in project list with referenced table") { - checkHiveQl("SELECT EXPLODE(arr) FROM parquet_t3") - checkHiveQl("SELECT EXPLODE(arr) AS val FROM parquet_t3") + checkSQL("SELECT EXPLODE(arr) FROM parquet_t3", "generator_referenced_table_1") + checkSQL("SELECT EXPLODE(arr) AS val FROM parquet_t3", "generator_referenced_table_2") } test("generator in project list with non-UDTF expressions") { - checkHiveQl("SELECT EXPLODE(arr), id FROM parquet_t3") - checkHiveQl("SELECT EXPLODE(arr) AS val, id as a FROM parquet_t3") + checkSQL("SELECT EXPLODE(arr), id FROM parquet_t3", "generator_non_udtf_1") + checkSQL("SELECT EXPLODE(arr) AS val, id as a FROM parquet_t3", "generator_non_udtf_2") } test("generator in lateral view") { - checkHiveQl("SELECT val, id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS val") - checkHiveQl("SELECT val, id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS val") + checkSQL("SELECT val, id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS val", + "generator_in_lateral_view_1") + checkSQL("SELECT val, id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS val", + "generator_in_lateral_view_2") } test("generator in lateral view with ambiguous names") { - checkHiveQl( + checkSQL( """ |SELECT exp.id, parquet_t3.id |FROM parquet_t3 |LATERAL VIEW EXPLODE(arr) exp AS id - """.stripMargin) - checkHiveQl( + """.stripMargin, + "generator_with_ambiguous_names_1") + + checkSQL( """ |SELECT exp.id, parquet_t3.id |FROM parquet_t3 |LATERAL VIEW OUTER EXPLODE(arr) exp AS id - """.stripMargin) + """.stripMargin, + "generator_with_ambiguous_names_2") } test("use JSON_TUPLE as generator") { - checkHiveQl( + checkSQL( """ |SELECT c0, c1, c2 |FROM parquet_t3 |LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt - """.stripMargin) - checkHiveQl( + """.stripMargin, + "json_tuple_generator_1") + + checkSQL( """ |SELECT a, b, c |FROM parquet_t3 |LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt AS a, b, c - """.stripMargin) + """.stripMargin, + "json_tuple_generator_2") } test("nested generator in lateral view") { - checkHiveQl( + checkSQL( """ |SELECT val, id |FROM parquet_t3 |LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array |LATERAL VIEW EXPLODE(nested_array) exp1 AS val - """.stripMargin) + """.stripMargin, + "nested_generator_in_lateral_view_1") - checkHiveQl( + checkSQL( """ |SELECT val, id |FROM parquet_t3 |LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array |LATERAL VIEW OUTER EXPLODE(nested_array) exp1 AS val - """.stripMargin) + """.stripMargin, + "nested_generator_in_lateral_view_2") } test("generate with other operators") { - checkHiveQl( + checkSQL( """ |SELECT EXPLODE(arr) AS val, id |FROM parquet_t3 |WHERE id > 2 |ORDER BY val, id |LIMIT 5 - """.stripMargin) + """.stripMargin, + "generate_with_other_1") - checkHiveQl( + checkSQL( """ |SELECT val, id |FROM parquet_t3 @@ -735,24 +911,26 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { |WHERE val > 2 |ORDER BY val, id |LIMIT 5 - """.stripMargin) + """.stripMargin, + "generate_with_other_2") } test("filter after subquery") { - checkHiveQl("SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a > 5") + checkSQL("SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a > 5", + "filter_after_subquery") } test("SPARK-14933 - select parquet table") { withTable("parquet_t") { sql("create table parquet_t stored as parquet as select 1 as c1, 'abc' as c2") - checkHiveQl("select * from parquet_t") + checkSQL("select * from parquet_t", "select_parquet_table") } } test("SPARK-14933 - select orc table") { withTable("orc_t") { sql("create table orc_t stored as orc as select 1 as c1, 'abc' as c2") - checkHiveQl("select * from orc_t") + checkSQL("select * from orc_t", "select_orc_table") } } } From 1dd152656a56b83c6daabda22148c495357ea3e3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Jul 2016 17:56:36 -0700 Subject: [PATCH 1000/1470] [HOTFIX] Fix Scala 2.10 compilation (cherry picked from commit c4524f5193e1b3ce1c56c5aed126f4121ce26d23) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 698c7c3e3c2fa..1f5078dc9c7d4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -40,8 +40,8 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { import testImplicits._ // Used for generating new query answer files by saving - private val regenerateGoldenFiles = - Option(System.getenv("SPARK_GENERATE_GOLDEN_FILES")).contains("1") + private val regenerateGoldenFiles: Boolean = + Option(System.getenv("SPARK_GENERATE_GOLDEN_FILES")) == Some("1") private val goldenSQLPath = "src/test/resources/sqlgen/" protected override def beforeAll(): Unit = { From 24ea875198ffcef4a4c3ba28aba128d6d7d9a395 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Jul 2016 18:03:35 -0700 Subject: [PATCH 1001/1470] [SPARK-16615][SQL] Expose sqlContext in SparkSession ## What changes were proposed in this pull request? This patch removes the private[spark] qualifier for SparkSession.sqlContext, as discussed in http://apache-spark-developers-list.1001551.n3.nabble.com/Re-transtition-SQLContext-to-SparkSession-td18342.html ## How was this patch tested? N/A - this is a visibility change. Author: Reynold Xin Closes #14252 from rxin/SPARK-16615. (cherry picked from commit 69c773052acc627eb033614797de9b913dfa35c1) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index df0950de1525e..946d8cbc6bf4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -115,9 +115,11 @@ class SparkSession private( /** * A wrapped version of this session in the form of a [[SQLContext]], for backward compatibility. + * + * @since 2.0.0 */ @transient - private[spark] val sqlContext: SQLContext = new SQLContext(this) + val sqlContext: SQLContext = new SQLContext(this) /** * Runtime configuration interface for Spark. From ef2a6f1310777bb6ea2b157a873c3785231b104a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 18 Jul 2016 23:07:59 -0700 Subject: [PATCH 1002/1470] [SPARK-16303][DOCS][EXAMPLES] Minor Scala/Java example update ## What changes were proposed in this pull request? This PR moves one and the last hard-coded Scala example snippet from the SQL programming guide into `SparkSqlExample.scala`. It also renames all Scala/Java example files so that all "Sql" in the file names are updated to "SQL". ## How was this patch tested? Manually verified the generated HTML page. Author: Cheng Lian Closes #14245 from liancheng/minor-scala-example-update. (cherry picked from commit 1426a080528bdb470b5e81300d892af45dd188bf) Signed-off-by: Yin Huai --- docs/sql-programming-guide.md | 57 +++++++++---------- ...ple.java => JavaSQLDataSourceExample.java} | 2 +- ...lExample.java => JavaSparkSQLExample.java} | 2 +- ...ample.scala => SQLDataSourceExample.scala} | 2 +- ...SqlExample.scala => SparkSQLExample.scala} | 8 +-- 5 files changed, 35 insertions(+), 36 deletions(-) rename examples/src/main/java/org/apache/spark/examples/sql/{JavaSqlDataSourceExample.java => JavaSQLDataSourceExample.java} (99%) rename examples/src/main/java/org/apache/spark/examples/sql/{JavaSparkSqlExample.java => JavaSparkSQLExample.java} (99%) rename examples/src/main/scala/org/apache/spark/examples/sql/{SqlDataSourceExample.scala => SQLDataSourceExample.scala} (99%) rename examples/src/main/scala/org/apache/spark/examples/sql/{SparkSqlExample.scala => SparkSQLExample.scala} (98%) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a4127dacee88f..a88efb7e12ba3 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -65,14 +65,14 @@ Throughout this document, we will often refer to Scala/Java Datasets of `Row`s a The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: -{% include_example init_session scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} +{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: -{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} +{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    @@ -105,7 +105,7 @@ from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: -{% include_example create_df scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} +{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    @@ -114,7 +114,7 @@ from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: -{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} +{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    @@ -155,7 +155,7 @@ Here we include some basic examples of structured data processing using Datasets
    -{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} +{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). @@ -164,7 +164,7 @@ In addition to simple column references and expressions, Datasets also have a ri
    -{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} +{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). @@ -249,13 +249,13 @@ In addition to simple column references and expressions, DataFrames also have a
    The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. -{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} +{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. -{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} +{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    @@ -287,11 +287,11 @@ the bytes back into an object.
    -{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} +{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    -{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} +{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    @@ -318,7 +318,7 @@ reflection and become the names of the columns. Case classes can also be nested types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be registered as a table. Tables can be used in subsequent SQL statements. -{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} +{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    @@ -330,7 +330,7 @@ does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `Li fields are supported though. You can create a JavaBean by creating a class that implements Serializable and has getters and setters for all of its fields. -{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} +{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    @@ -385,7 +385,7 @@ by `SparkSession`. For example: -{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSqlExample.scala %} +{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    @@ -403,7 +403,7 @@ by `SparkSession`. For example: -{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSqlExample.java %} +{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    @@ -472,11 +472,11 @@ In the simplest form, the default data source (`parquet` unless otherwise config
    -{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %} +{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    -{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %} +{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    @@ -507,11 +507,11 @@ using this syntax.
    -{% include_example manual_load_options scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %} +{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    -{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %} +{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    @@ -538,11 +538,11 @@ file directly with SQL.
    -{% include_example direct_sql scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %} +{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    -{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %} +{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    @@ -633,11 +633,11 @@ Using the data from the above example:
    -{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %} +{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    -{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %} +{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    @@ -766,11 +766,11 @@ turned it off by default starting from 1.5.0. You may enable it by
    -{% include_example schema_merging scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %} +{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    -{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %} +{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    @@ -961,7 +961,7 @@ Note that the file that is offered as _a json file_ is not a typical JSON file. line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. -{% include_example json_dataset scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala %} +{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    @@ -973,7 +973,7 @@ Note that the file that is offered as _a json file_ is not a typical JSON file. line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. -{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java %} +{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    @@ -1867,9 +1867,8 @@ Spark SQL and DataFrames support the following data types: All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. You can access them by doing -{% highlight scala %} -import org.apache.spark.sql.types._ -{% endhighlight %} + +{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java similarity index 99% rename from examples/src/main/java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java rename to examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index 4db5e1b0af83d..2b94b9f114e2d 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSqlDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -35,7 +35,7 @@ // $example off:basic_parquet_example$ import org.apache.spark.sql.SparkSession; -public class JavaSqlDataSourceExample { +public class JavaSQLDataSourceExample { // $example on:schema_merging$ public static class Square implements Serializable { diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSqlExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java similarity index 99% rename from examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSqlExample.java rename to examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java index 586d6e3a3e478..afc18078d471e 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSqlExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java @@ -60,7 +60,7 @@ import static org.apache.spark.sql.functions.col; // $example off:untyped_ops$ -public class JavaSparkSqlExample { +public class JavaSparkSQLExample { // $example on:create_ds$ public static class Person implements Serializable { private String name; diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala similarity index 99% rename from examples/src/main/scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala rename to examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 61dea6ad2ca90..0caba12af0bd0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SqlDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples.sql import org.apache.spark.sql.SparkSession -object SqlDataSourceExample { +object SQLDataSourceExample { case class Person(name: String, age: Long) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSqlExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/examples/sql/SparkSqlExample.scala rename to examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index cf3f864267197..952c074d03457 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSqlExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -25,12 +25,12 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.SparkSession // $example off:init_session$ // $example on:programmatic_schema$ -import org.apache.spark.sql.types.StringType -import org.apache.spark.sql.types.StructField -import org.apache.spark.sql.types.StructType +// $example on:data_types$ +import org.apache.spark.sql.types._ +// $example off:data_types$ // $example off:programmatic_schema$ -object SparkSqlExample { +object SparkSQLExample { // $example on:create_ds$ // Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, From 504aa6f7a87973de0955aa8c124e2a036f8b3369 Mon Sep 17 00:00:00 2001 From: Mortada Mehyar Date: Mon, 18 Jul 2016 23:49:47 -0700 Subject: [PATCH 1003/1470] [DOC] improve python doc for rdd.histogram and dataframe.join ## What changes were proposed in this pull request? doc change only ## How was this patch tested? doc change only Author: Mortada Mehyar Closes #14253 from mortada/histogram_typos. (cherry picked from commit 6ee40d2cc5f467c78be662c1639fc3d5b7f796cf) Signed-off-by: Reynold Xin --- python/pyspark/rdd.py | 18 +++++++++--------- python/pyspark/sql/dataframe.py | 10 +++++----- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6afe769662221..0508235c1c9ee 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1027,20 +1027,20 @@ def histogram(self, buckets): If your histogram is evenly spaced (e.g. [0, 10, 20, 30]), this can be switched from an O(log n) inseration to O(1) per - element(where n = # buckets). + element (where n is the number of buckets). - Buckets must be sorted and not contain any duplicates, must be + Buckets must be sorted, not contain any duplicates, and have at least two elements. - If `buckets` is a number, it will generates buckets which are + If `buckets` is a number, it will generate buckets which are evenly spaced between the minimum and maximum of the RDD. For - example, if the min value is 0 and the max is 100, given buckets - as 2, the resulting buckets will be [0,50) [50,100]. buckets must - be at least 1 If the RDD contains infinity, NaN throws an exception - If the elements in RDD do not vary (max == min) always returns - a single bucket. + example, if the min value is 0 and the max is 100, given `buckets` + as 2, the resulting buckets will be [0,50) [50,100]. `buckets` must + be at least 1. An exception is raised if the RDD contains infinity. + If the elements in the RDD do not vary (max == min), a single bucket + will be used. - It will return a tuple of buckets and histogram. + The return value is a tuple of buckets and histogram. >>> rdd = sc.parallelize(range(51)) >>> rdd.histogram(2) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c7d704a18adaf..b9f50ff021a35 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -601,16 +601,16 @@ def alias(self, alias): def join(self, other, on=None, how=None): """Joins with another :class:`DataFrame`, using the given join expression. - The following performs a full outer join between ``df1`` and ``df2``. - :param other: Right side of the join - :param on: a string for join column name, a list of column names, - , a join expression (Column) or a list of Columns. - If `on` is a string or a list of string indicating the name of the join column(s), + :param on: a string for the join column name, a list of column names, + a join expression (Column), or a list of Columns. + If `on` is a string or a list of strings indicating the name of the join column(s), the column(s) must exist on both sides, and this performs an equi-join. :param how: str, default 'inner'. One of `inner`, `outer`, `left_outer`, `right_outer`, `leftsemi`. + The following performs a full outer join between ``df1`` and ``df2``. + >>> 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)] From eb1c20fa0b43dde6da69bdb21ad8b3f9124e2607 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 19 Jul 2016 11:51:43 +0100 Subject: [PATCH 1004/1470] [MINOR][BUILD] Fix Java Linter `LineLength` errors This PR fixes four java linter `LineLength` errors. Those are all `LineLength` errors, but we had better remove all java linter errors before release. After pass the Jenkins, `./dev/lint-java`. Author: Dongjoon Hyun Closes #14255 from dongjoon-hyun/minor_java_linter. (cherry picked from commit 556a9437ac7b55079f5a8a91e669dcc36ca02696) Signed-off-by: Sean Owen --- .../org/apache/spark/network/yarn/YarnShuffleService.java | 2 +- .../apache/spark/examples/sql/JavaSQLDataSourceExample.java | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 4bc3c1a3c8a64..b6feb55e2192b 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -114,7 +114,7 @@ private boolean isAuthenticationEnabled() { protected void serviceInit(Configuration conf) { // In case this NM was killed while there were running spark applications, we need to restore - // lost state for the existing executors. We look for an existing file in the NM's local dirs. + // lost state for the existing executors. We look for an existing file in the NM's local dirs. // If we don't find one, then we choose a file to use to save the state next time. Even if // an application was stopped while the NM was down, we expect yarn to call stopApplication() // when it comes back diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index 2b94b9f114e2d..ec02c8bbb8efe 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -110,11 +110,13 @@ private static void runBasicDataSourceExample(SparkSession spark) { usersDF.select("name", "favorite_color").write().save("namesAndFavColors.parquet"); // $example off:generic_load_save_functions$ // $example on:manual_load_options$ - Dataset peopleDF = spark.read().format("json").load("examples/src/main/resources/people.json"); + Dataset peopleDF = + spark.read().format("json").load("examples/src/main/resources/people.json"); peopleDF.select("name", "age").write().format("parquet").save("namesAndAges.parquet"); // $example off:manual_load_options$ // $example on:direct_sql$ - Dataset sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); + Dataset sqlDF = + spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); // $example off:direct_sql$ } From 929fa287e700c0e112f43e0c7b9bc746b5546c64 Mon Sep 17 00:00:00 2001 From: Ahmed Mahran Date: Tue, 19 Jul 2016 12:01:54 +0100 Subject: [PATCH 1005/1470] [MINOR][SQL][STREAMING][DOCS] Fix minor typos, punctuations and grammar ## What changes were proposed in this pull request? Minor fixes correcting some typos, punctuations, grammar. Adding more anchors for easy navigation. Fixing minor issues with code snippets. ## How was this patch tested? `jekyll serve` Author: Ahmed Mahran Closes #14234 from ahmed-mahran/b-struct-streaming-docs. (cherry picked from commit 6caa22050e221cf14e2db0544fd2766dd1102bda) Signed-off-by: Sean Owen --- .../structured-streaming-programming-guide.md | 154 ++++++++---------- 1 file changed, 71 insertions(+), 83 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 3ef39e4885be8..aac88171fe3a4 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -22,14 +22,49 @@ Let’s say you want to maintain a running word count of text data received from
    +{% highlight scala %} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.SparkSession + +val spark = SparkSession + .builder + .appName("StructuredNetworkWordCount") + .getOrCreate() + +import spark.implicits._ +{% endhighlight %}
    +{% highlight java %} +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.util.Arrays; +import java.util.Iterator; + +SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCount") + .getOrCreate(); +{% endhighlight %}
    +{% highlight python %} +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +spark = SparkSession\ + .builder()\ + .appName("StructuredNetworkWordCount")\ + .getOrCreate() +{% endhighlight %} +
    @@ -38,18 +73,6 @@ Next, let’s create a streaming DataFrame that represents text data received fr
    -{% highlight scala %} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.SparkSession - -val spark = SparkSession - .builder - .appName("StructuredNetworkWordCount") - .getOrCreate() -{% endhighlight %} - -Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. - {% highlight scala %} // Create DataFrame representing the stream of input lines from connection to localhost:9999 val lines = spark.readStream @@ -65,29 +88,11 @@ val words = lines.as[String].flatMap(_.split(" ")) val wordCounts = words.groupBy("value").count() {% endhighlight %} -This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as[String]`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream.
    -{% highlight java %} -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.sql.*; -import org.apache.spark.sql.streaming.StreamingQuery; - -import java.util.Arrays; -import java.util.Iterator; - -SparkSession spark = SparkSession - .builder() - .appName("JavaStructuredNetworkWordCount") - .getOrCreate(); - -import spark.implicits._ -{% endhighlight %} - -Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. - {% highlight java %} // Create DataFrame representing the stream of input lines from connection to localhost:9999 Dataset lines = spark @@ -117,19 +122,6 @@ This `lines` DataFrame represents an unbounded table containing the streaming te
    -{% highlight python %} -from pyspark.sql import SparkSession -from pyspark.sql.functions import explode -from pyspark.sql.functions import split - -spark = SparkSession\ - .builder()\ - .appName("StructuredNetworkWordCount")\ - .getOrCreate() -{% endhighlight %} - -Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. - {% highlight python %} # Create DataFrame representing the stream of input lines from connection to localhost:9999 lines = spark\ @@ -223,7 +215,7 @@ $ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetwor {% endhighlight %}
    - {% highlight bash %} +{% highlight bash %} $ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999 {% endhighlight %}
    @@ -389,7 +381,7 @@ The “Output” is defined as what gets written out to the external storage. Th Note that each mode is applicable on certain types of queries. This is discussed in detail [later](#output-modes). To illustrate the use of this model, let’s understand the model in context of -the Quick Example above. The first `lines` DataFrame is the input table, and +the [Quick Example](#quick-example) above. The first `lines` DataFrame is the input table, and the final `wordCounts` DataFrame is the result table. Note that the query on streaming `lines` DataFrame to generate `wordCounts` is *exactly the same* as it would be a static DataFrame. However, when this query is started, Spark @@ -410,15 +402,14 @@ see how this model handles event-time based processing and late arriving data. ## Handling Event-time and Late Data Event-time is the time embedded in the data itself. For many applications, you may want to operate on this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time is a column value in the row. This allows window-based aggregations (e.g. number of event every minute) to be just a special type of grouping and aggregation on the even-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on a data stream, making the life of the user much easier. -Furthermore this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0, event-time watermarking will be used to manage this data. These are explained later in more details in the [Window Operations](#window-operations-on-event-time) section. +Furthermore, this model naturally handles data that has arrived later than expected based on its event-time. Since Spark is updating the Result Table, it has full control over updating/cleaning up the aggregates when there is late data. While not yet implemented in Spark 2.0, event-time watermarking will be used to manage this data. These are explained later in more details in the [Window Operations](#window-operations-on-event-time) section. ## Fault Tolerance Semantics Delivering end-to-end exactly-once semantics was one of key goals behind the design of Structured Streaming. To achieve that, we have designed the Structured Streaming sources, the sinks and the execution engine to reliably track the exact progress of the processing so that it can handle any kind of failure by restarting and/or reprocessing. Every streaming source is assumed to have offsets (similar to Kafka offsets, or Kinesis sequence numbers) to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, using replayable sources and idempotant sinks, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. # API using Datasets and DataFrames -Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` ( -[Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/ +Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` ([Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/ [Java](api/java/org/apache/spark/sql/SparkSession.html)/ [Python](api/python/pyspark.sql.html#pyspark.sql.SparkSession) docs) to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the [DataFrame/Dataset Programming Guide](sql-programming-guide.html). @@ -427,9 +418,9 @@ Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as Streaming DataFrames can be created through the `DataStreamReader` interface ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/ [Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source - data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. - - **File sources** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. + - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. @@ -439,7 +430,7 @@ Here are some examples.
    {% highlight scala %} -val spark: SparkSession = … +val spark: SparkSession = ... // Read text from socket val socketDF = spark @@ -493,7 +484,7 @@ Dataset[Row] csvDF = spark
    {% highlight python %} -spark = SparkSession. …. +spark = SparkSession. ... # Read text from socket socketDF = spark \ @@ -519,10 +510,10 @@ csvDF = spark \
    -These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the SQL Programming Guide for more details. Additionally, more details on the supported streaming sources are discussed later in the document. +These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the [SQL Programming Guide](sql-programming-guide.html) for more details. Additionally, more details on the supported streaming sources are discussed later in the document. ## Operations on streaming DataFrames/Datasets -You can apply all kinds of operations on streaming DataFrames/Datasets - ranging from untyped, SQL-like operations (e.g. select, where, groupBy), to typed RDD-like operations (e.g. map, filter, flatMap). See the [SQL programming guide](sql-programming-guide.html) for more details. Let’s take a look at a few example operations that you can use. +You can apply all kinds of operations on streaming DataFrames/Datasets – ranging from untyped, SQL-like operations (e.g. `select`, `where`, `groupBy`), to typed RDD-like operations (e.g. `map`, `filter`, `flatMap`). See the [SQL programming guide](sql-programming-guide.html) for more details. Let’s take a look at a few example operations that you can use. ### Basic Operations - Selection, Projection, Aggregation Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are [discussed later](#unsupported-operations) in this section. @@ -608,7 +599,7 @@ ds.groupByKey(new MapFunction() { // using typed API df = ... # streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType } -# Select the devices which have signal more than 11 +# Select the devices which have signal more than 10 df.select("device").where("signal > 10") # Running count of the number of updates for each device type @@ -620,7 +611,7 @@ df.groupBy("type").count() ### Window Operations on Event Time Aggregations over a sliding event-time window are straightforward with Structured Streaming. The key idea to understand about window-based aggregations are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration. -Imagine our quick example is modified and the stream now contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). +Imagine our [quick example](#quick-example) is modified and the stream now contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). The result tables would look something like the following. @@ -677,7 +668,7 @@ windowedCounts = words.groupBy( Now consider what happens if one of the events arrives late to the application. For example, a word that was generated at 12:04 but it was received at 12:11. -Since this windowing is based on the time in the data, the time 12:04 should be considered for windowing. This occurs naturally in our window-based grouping - the late data is automatically placed in the proper windows and the correct aggregates updated as illustrated below. +Since this windowing is based on the time in the data, the time 12:04 should be considered for windowing. This occurs naturally in our window-based grouping – the late data is automatically placed in the proper windows and the correct aggregates are updated as illustrated below. ![Handling Late Data](img/structured-streaming-late-data.png) @@ -711,8 +702,8 @@ streamingDf.join(staticDf, "type", "right_join"); // right outer join with a st
    {% highlight python %} -staticDf = spark.read. … -streamingDf = spark.readStream. … +staticDf = spark.read. ... +streamingDf = spark.readStream. ... streamingDf.join(staticDf, "type") # inner equi-join with a static DF streamingDf.join(staticDf, "type", "right_join") # right outer join with a static DF {% endhighlight %} @@ -741,7 +732,7 @@ However, note that all of the operations applicable on static DataFrames/Dataset - Any kind of joins between two streaming Datasets are not yet supported. -In addition, there are some Dataset methods that will not work on streaming Datasets. They are actions that will immediately run queries and return results, which does not makes sense on a streaming Dataset. Rather those functionalities can be done by explicitly starting a streaming query (see the next section regarding that). +In addition, there are some Dataset methods that will not work on streaming Datasets. They are actions that will immediately run queries and return results, which does not make sense on a streaming Dataset. Rather, those functionalities can be done by explicitly starting a streaming query (see the next section regarding that). - `count()` - Cannot return a single count from a streaming Dataset. Instead, use `ds.groupBy.count()` which returns a streaming Dataset containing a running count. @@ -753,10 +744,9 @@ If you try any of these operations, you will see an AnalysisException like "oper ## Starting Streaming Queries Once you have defined the final result DataFrame/Dataset, all that is left is for you start the streaming computation. To do that, you have to use the -`DataStreamWriter` ( -[Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/ +`DataStreamWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/ [Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) returned through `Dataset.writeSteram()`. You will have to specify one or more of the following in this interface. +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) returned through `Dataset.writeStream()`. You will have to specify one or more of the following in this interface. - *Details of the output sink:* Data format, location, etc. @@ -766,12 +756,12 @@ Once you have defined the final result DataFrame/Dataset, all that is left is fo - *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will attempt to trigger at the next trigger point, not immediately after the processing has completed. -- *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in a HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. +- *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in an HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. #### Output Modes There are two types of output mode currently implemented. -- **Append mode (default)** - This is the default mode, where only the new rows added to the result table since the last trigger will be outputted to the sink. This is only applicable to queries that *do not have any aggregations* (e.g. queries with only select, where, map, flatMap, filter, join, etc.). +- **Append mode (default)** - This is the default mode, where only the new rows added to the result table since the last trigger will be outputted to the sink. This is only applicable to queries that *do not have any aggregations* (e.g. queries with only `select`, `where`, `map`, `flatMap`, `filter`, `join`, etc.). - **Complete mode** - The whole result table will be outputted to the sink.This is only applicable to queries that *have aggregations*. @@ -826,7 +816,7 @@ Here is a table of all the sinks, and the corresponding settings.
    -Finally, you have to call `start()` to actually to start the execution of the query. This returns a StreamingQuery object which is a handle to the continuously running execution. You can use this object to manage the query, which we will discuss in the next subsection. For now, let’s understand all this with a few examples. +Finally, you have to call `start()` to actually start the execution of the query. This returns a StreamingQuery object which is a handle to the continuously running execution. You can use this object to manage the query, which we will discuss in the next subsection. For now, let’s understand all this with a few examples.
    @@ -858,7 +848,7 @@ aggDF .format("console") .start() -// Have all the aggregates in an in memory table +// Have all the aggregates in an in-memory table aggDF .writeStream .queryName("aggregates") // this query name will be the table name @@ -874,7 +864,7 @@ spark.sql("select * from aggregates").show() // interactively query in-memory {% highlight java %} // ========== DF with no aggregations ========== -Dataset noAggDF = deviceDataDf.select("device").where("signal > 10") +Dataset noAggDF = deviceDataDf.select("device").where("signal > 10"); // Print new data to console noAggDF @@ -898,7 +888,7 @@ aggDF .format("console") .start(); -// Have all the aggregates in an in memory table +// Have all the aggregates in an in-memory table aggDF .writeStream() .queryName("aggregates") // this query name will be the table name @@ -954,7 +944,7 @@ spark.sql("select * from aggregates").show() # interactively query in-memory t #### Using Foreach The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.0, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/ -[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), which has methods that gets called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. +[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), which has methods that get called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. - The writer must be serializable, as it will be serialized and sent to the executors for execution. @@ -989,7 +979,7 @@ query.awaitTermination() // block until query is terminated, with stop() or wi query.exception() // the exception if the query has been terminated with error -query.souceStatus() // progress information about data has been read from the input sources +query.sourceStatus() // progress information about data has been read from the input sources query.sinkStatus() // progress information about data written to the output sink {% endhighlight %} @@ -1013,7 +1003,7 @@ query.awaitTermination(); // block until query is terminated, with stop() or w query.exception(); // the exception if the query has been terminated with error -query.souceStatus(); // progress information about data has been read from the input sources +query.sourceStatus(); // progress information about data has been read from the input sources query.sinkStatus(); // progress information about data written to the output sink @@ -1037,7 +1027,7 @@ query.awaitTermination() # block until query is terminated, with stop() or wit query.exception() # the exception if the query has been terminated with error -query.souceStatus() # progress information about data has been read from the input sources +query.sourceStatus() # progress information about data has been read from the input sources query.sinkStatus() # progress information about data written to the output sink @@ -1046,8 +1036,7 @@ query.sinkStatus() # progress information about data written to the output sin
    -You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the `StreamingQueryManager` ( -[Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryManager)/ +You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the `StreamingQueryManager` ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryManager)/ [Java](api/java/org/apache/spark/sql/streaming/StreamingQueryManager.html)/ [Python](api/python/pyspark.sql.html#pyspark.sql.streaming.StreamingQueryManager) docs) that can be used to manage the currently active queries. @@ -1055,7 +1044,7 @@ You can start any number of queries in a single SparkSession. They will all be r
    {% highlight scala %} -val spark: SparkSession = … +val spark: SparkSession = ... spark.streams.active // get the list of currently active streaming queries @@ -1070,11 +1059,11 @@ spark.streams.awaitAnyTermination() // block until any one of them terminates {% highlight java %} SparkSession spark = ... -spark.streams().active() // get the list of currently active streaming queries +spark.streams().active(); // get the list of currently active streaming queries -spark.streams().get(id) // get a query object by its unique id +spark.streams().get(id); // get a query object by its unique id -spark.streams().awaitAnyTermination() // block until any one of them terminates +spark.streams().awaitAnyTermination(); // block until any one of them terminates {% endhighlight %}
    @@ -1093,12 +1082,11 @@ spark.streams().awaitAnyTermination() # block until any one of them terminates
    -Finally, for asynchronous monitoring of streaming queries, you can create and attach a `StreamingQueryListener` ( -[Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryListener)/ +Finally, for asynchronous monitoring of streaming queries, you can create and attach a `StreamingQueryListener` ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryListener)/ [Java](api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html) docs), which will give you regular callback-based updates when queries are started and terminated. ## Recovering from Failures with Checkpointing -In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure a query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger), and the running aggregates (e.g. word counts in the quick example) will be saved the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in a HDFS compatible file system, and can be set as an option in the DataStreamWriter when [starting a query](#starting-streaming-queries). +In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure a query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger) and the running aggregates (e.g. word counts in the [quick example](#quick-example)) to the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in an HDFS compatible file system, and can be set as an option in the DataStreamWriter when [starting a query](#starting-streaming-queries).
    From 2c74b6d73beab4510fa7933dde9c0a5c218cce92 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 19 Jul 2016 12:07:40 +0100 Subject: [PATCH 1006/1470] [SPARK-16600][MLLIB] fix some latex formula syntax error ## What changes were proposed in this pull request? `\partial\x` ==> `\partial x` `har{x_i}` ==> `hat{x_i}` ## How was this patch tested? N/A Author: WeichenXu Closes #14246 from WeichenXu123/fix_formular_err. (cherry picked from commit 8310c0741c0ca805ec74c1a78ba4a0f18e82d459) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/regression/LinearRegression.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 401f2c673f51c..0a155e1844f62 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 @@ -794,16 +794,16 @@ class LinearRegressionSummary private[regression] ( * * Now, the first derivative of the objective function in scaled space is * {{{ - * \frac{\partial L}{\partial\w_i} = diff/N (x_i - \bar{x_i}) / \hat{x_i} + * \frac{\partial L}{\partial w_i} = diff/N (x_i - \bar{x_i}) / \hat{x_i} * }}} * However, ($x_i - \bar{x_i}$) will densify the computation, so it's not * an ideal formula when the training dataset is sparse format. * - * This can be addressed by adding the dense \bar{x_i} / \har{x_i} terms + * This can be addressed by adding the dense \bar{x_i} / \hat{x_i} terms * in the end by keeping the sum of diff. The first derivative of total * objective function from all the samples is * {{{ - * \frac{\partial L}{\partial\w_i} = + * \frac{\partial L}{\partial w_i} = * 1/N \sum_j diff_j (x_{ij} - \bar{x_i}) / \hat{x_i} * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - diffSum \bar{x_i}) / \hat{x_i}) * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + correction_i) @@ -822,7 +822,7 @@ class LinearRegressionSummary private[regression] ( * the training dataset, which can be easily computed in distributed fashion, and is * sparse format friendly. * {{{ - * \frac{\partial L}{\partial\w_i} = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + * \frac{\partial L}{\partial w_i} = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) * }}}, * * @param coefficients The coefficients corresponding to the features. From 6ca1d941b0b417f10533ab3506a9f3cf60e6a7fe Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Tue, 19 Jul 2016 10:24:48 -0700 Subject: [PATCH 1007/1470] [SPARK-16620][CORE] Add back the tokenization process in `RDD.pipe(command: String)` ## What changes were proposed in this pull request? Currently `RDD.pipe(command: String)`: - works only when the command is specified without any options, such as `RDD.pipe("wc")` - does NOT work when the command is specified with some options, such as `RDD.pipe("wc -l")` This is a regression from Spark 1.6. This patch adds back the tokenization process in `RDD.pipe(command: String)` to fix this regression. ## How was this patch tested? Added a test which: - would pass in `1.6` - _[prior to this patch]_ would fail in `master` - _[after this patch]_ would pass in `master` Author: Liwei Lin Closes #14256 from lw-lin/rdd-pipe. (cherry picked from commit 0bd76e872b60cb80295fc12654e370cf22390056) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/rdd/RDD.scala | 8 ++++++-- .../org/apache/spark/rdd/PipedRDDSuite.scala | 16 ++++++++++++++++ 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index b7a5b222087e1..0804cdeb048a3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -699,14 +699,18 @@ abstract class RDD[T: ClassTag]( * Return an RDD created by piping elements to a forked external process. */ def pipe(command: String): RDD[String] = withScope { - pipe(command) + // Similar to Runtime.exec(), if we are given a single string, split it into words + // using a standard StringTokenizer (i.e. by spaces) + pipe(PipedRDD.tokenize(command)) } /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: String, env: Map[String, String]): RDD[String] = withScope { - pipe(command, env) + // Similar to Runtime.exec(), if we are given a single string, split it into words + // using a standard StringTokenizer (i.e. by spaces) + pipe(PipedRDD.tokenize(command), env) } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 27cfdc7aced56..5d56fc19f00e1 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -51,6 +51,22 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("basic pipe with tokenization") { + if (testCommandAvailable("wc")) { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + + // verify that both RDD.pipe(command: String) and RDD.pipe(command: String, env) work good + for (piped <- Seq(nums.pipe("wc -l"), nums.pipe("wc -l", Map[String, String]()))) { + val c = piped.collect() + assert(c.size === 2) + assert(c(0).trim === "2") + assert(c(1).trim === "2") + } + } else { + assert(true) + } + } + test("failure in iterating over pipe input") { if (testCommandAvailable("cat")) { val nums = From f18f9ca5b22ca11712694b1106463ae6efc1d646 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 19 Jul 2016 10:28:17 -0700 Subject: [PATCH 1008/1470] [SPARK-16602][SQL] `Nvl` function should support numeric-string cases ## What changes were proposed in this pull request? `Nvl` function should support numeric-straing cases like Hive/Spark1.6. Currently, `Nvl` finds the tightest common types among numeric types. This PR extends that to consider `String` type, too. ```scala - TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeToString(left.dataType, right.dataType).map { dtype => ``` **Before** ```scala scala> sql("select nvl('0', 1)").collect() org.apache.spark.sql.AnalysisException: cannot resolve `nvl("0", 1)` due to data type mismatch: input to function coalesce should all be the same type, but it's [string, int]; line 1 pos 7 ``` **After** ```scala scala> sql("select nvl('0', 1)").collect() res0: Array[org.apache.spark.sql.Row] = Array([0]) ``` ## How was this patch tested? Pass the Jenkins tests. Author: Dongjoon Hyun Closes #14251 from dongjoon-hyun/SPARK-16602. (cherry picked from commit 162d04a30e38bb83d35865679145f8ea80b84c26) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/TypeCoercion.scala | 2 +- .../catalyst/expressions/nullExpressions.scala | 2 +- .../catalyst/expressions/NullFunctionsSuite.scala | 15 +++++++++++++++ 3 files changed, 17 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index baec6d14a212a..9a040f8644fb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -100,7 +100,7 @@ object TypeCoercion { } /** Similar to [[findTightestCommonType]], but can promote all the way to StringType. */ - private def findTightestCommonTypeToString(left: DataType, right: DataType): Option[DataType] = { + def findTightestCommonTypeToString(left: DataType, right: DataType): Option[DataType] = { findTightestCommonTypeOfTwo(left, right).orElse((left, right) match { case (StringType, t2: AtomicType) if t2 != BinaryType && t2 != BooleanType => Some(StringType) case (t1: AtomicType, StringType) if t1 != BinaryType && t1 != BooleanType => Some(StringType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 523fb053972dd..1c18265e0fed4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -134,7 +134,7 @@ case class Nvl(left: Expression, right: Expression) extends RuntimeReplaceable { override def replaceForTypeCoercion(): Expression = { if (left.dataType != right.dataType) { - TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype => + TypeCoercion.findTightestCommonTypeToString(left.dataType, right.dataType).map { dtype => copy(left = Cast(left, dtype), right = Cast(right, dtype)) }.getOrElse(this) } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala index ace6c15dc8418..712fe35f477b3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala @@ -77,6 +77,21 @@ class NullFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("SPARK-16602 Nvl should support numeric-string cases") { + val intLit = Literal.create(1, IntegerType) + val doubleLit = Literal.create(2.2, DoubleType) + val stringLit = Literal.create("c", StringType) + val nullLit = Literal.create(null, NullType) + + assert(Nvl(intLit, doubleLit).replaceForTypeCoercion().dataType == DoubleType) + assert(Nvl(intLit, stringLit).replaceForTypeCoercion().dataType == StringType) + assert(Nvl(stringLit, doubleLit).replaceForTypeCoercion().dataType == StringType) + + assert(Nvl(nullLit, intLit).replaceForTypeCoercion().dataType == IntegerType) + assert(Nvl(doubleLit, nullLit).replaceForTypeCoercion().dataType == DoubleType) + assert(Nvl(nullLit, stringLit).replaceForTypeCoercion().dataType == StringType) + } + test("AtLeastNNonNulls") { val mix = Seq(Literal("x"), Literal.create(null, StringType), From 80ab8b666f007de15fa9427f9734ed91238605b0 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 19 Jul 2016 12:58:08 -0700 Subject: [PATCH 1009/1470] [SPARK-15705][SQL] Change the default value of spark.sql.hive.convertMetastoreOrc to false. ## What changes were proposed in this pull request? In 2.0, we add a new logic to convert HiveTableScan on ORC tables to Spark's native code path. However, during this conversion, we drop the original metastore schema (https://issues.apache.org/jira/browse/SPARK-15705). Because of this regression, I am changing the default value of `spark.sql.hive.convertMetastoreOrc` to false. Author: Yin Huai Closes #14267 from yhuai/SPARK-15705-changeDefaultValue. (cherry picked from commit 2ae7b88a07140e012b6c60db3c4a2a8ca360c684) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 9ed357c587c35..bdec611453b2d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -97,10 +97,11 @@ private[spark] object HiveUtils extends Logging { .createWithDefault(false) val CONVERT_METASTORE_ORC = SQLConfigBuilder("spark.sql.hive.convertMetastoreOrc") + .internal() .doc("When set to false, Spark SQL will use the Hive SerDe for ORC tables instead of " + "the built in support.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val HIVE_METASTORE_SHARED_PREFIXES = SQLConfigBuilder("spark.sql.hive.metastore.sharedPrefixes") .doc("A comma separated list of class prefixes that should be loaded using the classloader " + From 13650fc58e1fcf2cf2a26ba11c819185ae1acc1f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 19 Jul 2016 14:02:27 -0700 Subject: [PATCH 1010/1470] Preparing Spark release v2.0.0-rc5 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 507ddc778383d..5f546bb9ee697 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bc3b0fe73f6eb..2eaa8100a61d4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 2fb5835305a25..f068d9d4ac428 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 07d9f1c58f7a3..fd221883b1fba 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5e02efdc45e67..a17aba5abe548 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e7fc6a2a02417..0bd8846f452c5 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 24f0e75f2f045..f495b0dddee54 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index bb27ec916c65a..093ca69abb866 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d2227944d92dd..fe9f85ae5a085 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 18e14c7981d80..ca5e4980eb8c7 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 3847b39e17315..b4cdbda6b613b 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7fc8a4b00dc20..4e7e17de6a417 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index dcdbcc55d8785..cc5a13721ee3f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1eb349ff4c3b9..636b233a0de04 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 59f41f1e17f30..093cfb76b6aea 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 26965612cc0ab..b4e9eba6efd4d 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a346d76210932..1438e2ba59e6a 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 2aa7d82e8e064..dc704375dcc44 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 58c57c1c539df..af0b81a0cff82 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b524001d04719..919778bac01e9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 93ac8b6e664d7..f7588d1917ba5 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4f8af77792b0b..f8dd7f7fd751c 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index b6591598ee127..e015b95cafd15 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1d8f7f4d9bbed..86edf650b5c7b 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40fde1bab7ad1..407cb6da8281c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 9f3d7f003584c..ff7b36f495a85 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 0b5ec1a08c821..9ca84e986a3dc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0bfdb13cec941..259e56e4afef9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02a18b33b087a..0b06c29302d6e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 672425c86ecbe..ce66c71ccae6e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index add4375364b1a..5ad0c51ad05c8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index e7614fe66110d..21d8d062bd380 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 3f4cce1ca354e..a2124104f4fb9 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 7dba1a829fb94..1ca840d8c0968 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.0 ../pom.xml From 307f8922be5c781d83c295edbbe9ad0f0d2095e3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 19 Jul 2016 14:02:33 -0700 Subject: [PATCH 1011/1470] Preparing development version 2.0.1-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5f546bb9ee697..507ddc778383d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2eaa8100a61d4..bc3b0fe73f6eb 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f068d9d4ac428..2fb5835305a25 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index fd221883b1fba..07d9f1c58f7a3 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index a17aba5abe548..5e02efdc45e67 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 0bd8846f452c5..e7fc6a2a02417 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index f495b0dddee54..24f0e75f2f045 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 093ca69abb866..bb27ec916c65a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index fe9f85ae5a085..d2227944d92dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ca5e4980eb8c7..18e14c7981d80 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index b4cdbda6b613b..3847b39e17315 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 4e7e17de6a417..7fc8a4b00dc20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index cc5a13721ee3f..dcdbcc55d8785 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 636b233a0de04..1eb349ff4c3b9 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 093cfb76b6aea..59f41f1e17f30 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index b4e9eba6efd4d..26965612cc0ab 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 1438e2ba59e6a..a346d76210932 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dc704375dcc44..2aa7d82e8e064 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index af0b81a0cff82..58c57c1c539df 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 919778bac01e9..b524001d04719 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index f7588d1917ba5..93ac8b6e664d7 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f8dd7f7fd751c..4f8af77792b0b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e015b95cafd15..b6591598ee127 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 86edf650b5c7b..1d8f7f4d9bbed 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 407cb6da8281c..40fde1bab7ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ff7b36f495a85..9f3d7f003584c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 9ca84e986a3dc..0b5ec1a08c821 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 259e56e4afef9..0bfdb13cec941 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0b06c29302d6e..02a18b33b087a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index ce66c71ccae6e..672425c86ecbe 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5ad0c51ad05c8..add4375364b1a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 21d8d062bd380..e7614fe66110d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a2124104f4fb9..3f4cce1ca354e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 1ca840d8c0968..7dba1a829fb94 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.0 + 2.0.1-SNAPSHOT ../pom.xml From f58fd4620f703fba0c8be0724c0150b08e984a2b Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 19 Jul 2016 18:48:41 -0700 Subject: [PATCH 1012/1470] [SPARK-16568][SQL][DOCUMENTATION] update sql programming guide refreshTable API in python code ## What changes were proposed in this pull request? update `refreshTable` API in python code of the sql-programming-guide. This API is added in SPARK-15820 ## How was this patch tested? N/A Author: WeichenXu Closes #14220 from WeichenXu123/update_sql_doc_catalog. (cherry picked from commit 9674af6f6f81066139ea675de724f951bd0d49c9) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a88efb7e12ba3..8d92a4311eeb3 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -869,8 +869,8 @@ spark.catalog().refreshTable("my_table");
    {% highlight python %} -# spark is an existing HiveContext -spark.refreshTable("my_table") +# spark is an existing SparkSession +spark.catalog.refreshTable("my_table") {% endhighlight %}
    From 6f209c8faad0c928368852c881e2aaabe100b152 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 19 Jul 2016 19:28:08 -0700 Subject: [PATCH 1013/1470] [SPARK-10683][SPARK-16510][SPARKR] Move SparkR include jar test to SparkSubmitSuite ## What changes were proposed in this pull request? This change moves the include jar test from R to SparkSubmitSuite and uses a dynamically compiled jar. This helps us remove the binary jar from the R package and solves both the CRAN warnings and the lack of source being available for this jar. ## How was this patch tested? SparkR unit tests, SparkSubmitSuite, check-cran.sh Author: Shivaram Venkataraman Closes #14243 from shivaram/sparkr-jar-move. (cherry picked from commit fc23263623d5dcd1167fa93c094fe41ace77c326) Signed-off-by: Shivaram Venkataraman --- .../test_support/sparktestjar_2.10-1.0.jar | Bin 2886 -> 0 bytes R/pkg/inst/tests/testthat/jarTest.R | 10 ++--- R/pkg/inst/tests/testthat/test_includeJAR.R | 36 ----------------- .../scala/org/apache/spark/api/r/RUtils.scala | 9 +++++ .../spark/deploy/SparkSubmitSuite.scala | 38 ++++++++++++++++++ 5 files changed, 52 insertions(+), 41 deletions(-) delete mode 100644 R/pkg/inst/test_support/sparktestjar_2.10-1.0.jar delete mode 100644 R/pkg/inst/tests/testthat/test_includeJAR.R diff --git a/R/pkg/inst/test_support/sparktestjar_2.10-1.0.jar b/R/pkg/inst/test_support/sparktestjar_2.10-1.0.jar deleted file mode 100644 index 1d5c2af631aa3ae88aa7836e8db598e59cbcf1b7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2886 zcmaJ@2T)UK7Y!kyL@*d49Sj{JRS@YAsv!nq=*S|~fYJj-g+=LYfzbORNV8FN{bdCk zg0zIbAfgZyq{;FFilHpKi8@1Yb?=)u^SycZJMX@^=brE2Fzg^WfQyR@ut^-V0I&oc z00Lmm?NG{SYYSB@${KB9ZfmE4wbuIiGP0M_cNecVtU;Sur6_lz zsaWb^v=SR+A;CLuy3$1vE+`}5lL)YnkQMN#MIA*nA&%;3C~FAI<>zOe_O2pl73Db< z*~X|~OI8rlVFtFrMKPnCLMpTwAOMHqFeX~AEe^t??EK`oE#4vGUh9FCIkePXoe5stL%}& zNVAhqLvP+N2Ki!4axnH{pCBmT>;l@Gm+pFq9mvWPc4#D^ejDn^&%HvUK^OB z)EgN^0iUSck}R0#%xqa6FDyH=u4lyK#mnchVHF8GuTYWvvwq8}Wg!P7M*Y|;8%rrT zSMkG(9`ZZmI9>ms=PmBAWd9<1I7oL}szYkqA=5BaS>~7Fg4Fu;I@PigC*Z->SEaFo z)mA(U+Et_0mnO!HE930f3)`UwV_k-Irmgy4i(gJx=yLphKC9Z|?D+e)*g9|J@Rsv7 zk~1Pi$~vua^rxVv3bbNK;S<5F#?d~J`xnxM^8*4`fzJ2NyI^8D1%q_oI-HdKqvn++ z@j;zGC;;QDtGL>2zA`gW*&5@)PLxO097`UtpP@;T8Uuo*gl5_07hrMQ`oFAcb9Zpv zU;eNo|IP=^hg}#?g_&HGTvq1g0WHlng}gi$D0%CHfbvHm+#?yqW(9U9_w{s}bsD60 zJbKlZTGQ2eSt2gWZfi(kU)^7K5xfcIeb*Fv%>>#q_2YJy&1isVF^WX@#n6hj6z{`$ z%jC^tPLH<4yHC$bm@gpo;104PgXP`zRac*-ITr)B!AzcE1SyllYyf~A@C(nrPYj{& z5kuw+Gpg{PnPDKR7ye#IBnz)F3M|+5yB;wmr()vg;(b}QHGS>#( z3vS3(@8`Xdz_ZHZ5;L)1`Z2Y^eK=$BWymyG{N|B)PX$>RK0=zub0-sStx+Pv>1NSF zCCw=kgsKW_ELQGn0T9Y&+I!bpHCO`y5xKugYBh#)_91|y|bv3&bbNn zhl?VRCe)^$J}q+=Qe+qHvz>9J*ohAqsNP^``th`sTtvmSm>alzorsQR37?+mp&3oj zY& z-k5K3sz5^m;*OY4n|5OT1|(XAJm)oLyn_?h$3hGPO(hQPO*jb4N2-&N?h;NAI z=|S}KvIr0K3iAsJ{7SfRa*uVZF+Ab#XH`{T@KaGilFF5MkgL^;S{WGyZiRC-RlWyO z=G$vROpZ`kR$=NeU#AyYCMvqt$Fy_Aie4LY#jBp$cw7PqDEPtTh@#xpf`=2-BF zNjKdZWD>bJ*>Rv?-)y70+L^rhn@RkT%g8=`TM8KUP}ueLr+ORi+?~H)>@&F$&>PUP zt`~RVo|c@#Qa{x=v06Gob2O>Tdzy1dQPO$o<5>ff*2{Mcw5%I^VYT-&l@prsYNVsX zI3RXuY@8*{uGUH>j~-8trp2Gz=N0(6E0(-8o*pi$#M6@|wdpI|RQ<=js%)_?XnEIF z5HtmslKq+Jh;CpZU$T;;eF? zxpCQ&39>;eWaswZk0LCAGCDUJ2Jb_)35SFHJ|10PT-J6CIOVOCTx+gwW=oG1vhuKp z`HXD|GMhCjwbXK&p$fcvRN>oZ9r~|keG!N7%#%Sd6ki7+&^etwJrwvqoSM$1N*KS2 zyVq^A?D&|c{p8Sqd+4LiG{mX-(qIxc^8cb6)3H!PI@>lVO62We8|R@1UGFJIYdx3m zYCPWArgH<7Usxz`l+icezS#d7@teFp-%!*v)^sWYi7;6&nhcHTdKhm|MDoF3kZT-3_HO6kWVS)v`toS%e?3*2|~%cZro` z)VXA?Azv-4#lxOOl5G8Ij-%eTgldL}ZV-}0)X%oM3#AHbR)oLKXTqvz+lZ{3N@(p3yvbfn5G-99m*{uhm!9X}zvQGyzicGsj=}Bjwy);9H?Ff==SdETklfOmO6)|W6!QWPkfel(kTtipW5d1 z@LrE>9H@+1qE^&56Lbj5@sZ7BeuDcHvXXB&dthawfgqWa`03h`<{Wtd!F$TDA5oZ- z8_l=4sUaviPan9nu-=YOd855*67tq9$@oN`%9_5>v?#yMX4~T`V}5wj(|1=tWTzjQ ztwaJc#Q{sA-k9desi&iQNycsy5F9t4QbsJloP&HNu~-XC-^XN1_zY>^YX(ysKQo05 z2nXz*AgmsSX{+|ek4zR0!v=%^e(ZO4QEu<@@4q%N{m*U;GL~O0(^ogNw`kS_k?Dta zW1F#L-O1vPn4f3;b5^lqo}IgKkRgBn0{JRztSHP`W1T|8E(Bv01>TGDJ(>I#jkQzE i$=!{^3>(Q>(;l=hbBx1)IhY$b8P_ + new File(Seq(pkgDir, "SparkR").mkString(File.separator)).exists + }.isDefined + } + /** * Get the list of paths for R packages in various deployment modes, of which the first * path is for the SparkR package itself. The second path is for R packages built as diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 0b020592b06d3..b2bc8861083bb 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate import org.apache.spark.internal.Logging +import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.util.{ResetSystemProperties, Utils} // Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch @@ -417,6 +418,8 @@ class SparkSubmitSuite // See https://gist.github.com/shivaram/3a2fecce60768a603dac for a error log ignore("correctly builds R packages included in a jar with --packages") { assume(RUtils.isRInstalled, "R isn't installed on this machine.") + // Check if the SparkR package is installed + assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.") val main = MavenCoordinate("my.great.lib", "mylib", "0.1") val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val rScriptDir = @@ -435,6 +438,41 @@ class SparkSubmitSuite } } + test("include an external JAR in SparkR") { + assume(RUtils.isRInstalled, "R isn't installed on this machine.") + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) + // Check if the SparkR package is installed + assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.") + val rScriptDir = + Seq(sparkHome, "R", "pkg", "inst", "tests", "testthat", "jarTest.R").mkString(File.separator) + assert(new File(rScriptDir).exists) + + // compile a small jar containing a class that will be called from R code. + val tempDir = Utils.createTempDir() + val srcDir = new File(tempDir, "sparkrtest") + srcDir.mkdirs() + val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").getAbsolutePath, + """package sparkrtest; + | + |public class DummyClass implements java.io.Serializable { + | public static String helloWorld(String arg) { return "Hello " + arg; } + | public static int addStuff(int arg1, int arg2) { return arg1 + arg2; } + |} + """.stripMargin) + val excFile = TestUtils.createCompiledClass("DummyClass", srcDir, excSource, Seq.empty) + val jarFile = new File(tempDir, "sparkRTestJar-%s.jar".format(System.currentTimeMillis())) + val jarURL = TestUtils.createJar(Seq(excFile), jarFile, directoryPrefix = Some("sparkrtest")) + + val args = Seq( + "--name", "testApp", + "--master", "local", + "--jars", jarURL.toString, + "--verbose", + "--conf", "spark.ui.enabled=false", + rScriptDir) + runSparkSubmit(args) + } + test("resolves command line argument paths correctly") { val jars = "/jar1,/jar2" // --jars val files = "hdfs:/file1,file2" // --files From c2b5b3ca538aaaef946653e60bd68e38c58dc41f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 20 Jul 2016 13:00:22 +0800 Subject: [PATCH 1014/1470] [SPARK-16632][SQL] Respect Hive schema when merging parquet schema. When Hive (or at least certain versions of Hive) creates parquet files containing tinyint or smallint columns, it stores them as int32, but doesn't annotate the parquet field as containing the corresponding int8 / int16 data. When Spark reads those files using the vectorized reader, it follows the parquet schema for these fields, but when actually reading the data it tries to use the type fetched from the metastore, and then fails because data has been loaded into the wrong fields in OnHeapColumnVector. So instead of blindly trusting the parquet schema, check whether the Catalyst-provided schema disagrees with it, and adjust the types so that the necessary metadata is present when loading the data into the ColumnVector instance. Tested with unit tests and with tests that create byte / short columns in Hive and try to read them from Spark. Author: Marcelo Vanzin Closes #14272 from vanzin/SPARK-16632. (cherry picked from commit 75146be6ba5e9f559f5f15430310bb476ee0812c) Signed-off-by: Cheng Lian --- .../parquet/ParquetReadSupport.scala | 18 +++++++++ .../parquet/ParquetSchemaSuite.scala | 39 +++++++++++++++++++ 2 files changed, 57 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 12f497421f4b7..1628e4c8ee660 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -26,6 +26,8 @@ import org.apache.parquet.hadoop.api.{InitContext, ReadSupport} import org.apache.parquet.hadoop.api.ReadSupport.ReadContext import org.apache.parquet.io.api.RecordMaterializer import org.apache.parquet.schema._ +import org.apache.parquet.schema.OriginalType._ +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition import org.apache.spark.internal.Logging @@ -116,6 +118,12 @@ private[parquet] object ParquetReadSupport { } private def clipParquetType(parquetType: Type, catalystType: DataType): Type = { + val primName = if (parquetType.isPrimitive()) { + parquetType.asPrimitiveType().getPrimitiveTypeName() + } else { + null + } + catalystType match { case t: ArrayType if !isPrimitiveCatalystType(t.elementType) => // Only clips array types with nested type as element type. @@ -130,6 +138,16 @@ private[parquet] object ParquetReadSupport { case t: StructType => clipParquetGroup(parquetType.asGroupType(), t) + case _: ByteType if primName == INT32 => + // SPARK-16632: Handle case where Hive stores bytes in a int32 field without specifying + // the original type. + Types.primitive(INT32, parquetType.getRepetition()).as(INT_8).named(parquetType.getName()) + + case _: ShortType if primName == INT32 => + // SPARK-16632: Handle case where Hive stores shorts in a int32 field without specifying + // the original type. + Types.primitive(INT32, parquetType.getRepetition()).as(INT_16).named(parquetType.getName()) + case _ => // UDTs and primitive types are not clipped. For UDTs, a clipped version might not be able // to be mapped to desired user-space types. So UDTs shouldn't participate schema merging. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 51bb236fe8441..215c1387ef00e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -1573,4 +1573,43 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | } |} """.stripMargin) + + testSchemaClipping( + "int32 parquet field with byte schema field", + + parquetSchema = + """message root { + | optional int32 value; + |} + """.stripMargin, + + catalystSchema = + new StructType() + .add("value", ByteType, nullable = true), + + expectedSchema = + """message root { + | optional int32 value (INT_8); + |} + """.stripMargin) + + testSchemaClipping( + "int32 parquet field with short schema field", + + parquetSchema = + """message root { + | optional int32 value; + |} + """.stripMargin, + + catalystSchema = + new StructType() + .add("value", ShortType, nullable = true), + + expectedSchema = + """message root { + | optional int32 value (INT_16); + |} + """.stripMargin) + } From 3f6b272db3fe65633dd21ddc03fa97c816b816fb Mon Sep 17 00:00:00 2001 From: Anthony Truchet Date: Wed, 20 Jul 2016 10:39:59 +0100 Subject: [PATCH 1015/1470] [SPARK-16440][MLLIB] Destroy broadcasted variables even on driver ## What changes were proposed in this pull request? Forgotten broadcasted variables were persisted into a previous #PR 14153). This PR turns those `unpersist()` into `destroy()` so that memory is freed even on the driver. ## How was this patch tested? Unit Tests in Word2VecSuite were run locally. This contribution is done on behalf of Criteo, according to the terms of the Apache license 2.0. Author: Anthony Truchet Closes #14268 from AnthonyTruchet/SPARK-16440. (cherry picked from commit 0dc79ffd1cbb45e69a35e3f5334c9a13290037a0) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 6b9c8ee2e3efc..bc75646d532dc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -434,9 +434,9 @@ class Word2Vec extends Serializable with Logging { bcSyn1Global.unpersist(false) } newSentences.unpersist() - expTable.unpersist() - bcVocab.unpersist() - bcVocabHash.unpersist() + expTable.destroy() + bcVocab.destroy() + bcVocabHash.destroy() val wordArray = vocab.map(_.word) new Word2VecModel(wordArray.zipWithIndex.toMap, syn0Global) From 83b957e6a546c817b95948110b3044f89304cf60 Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Wed, 20 Jul 2016 14:26:26 +0100 Subject: [PATCH 1016/1470] =?UTF-8?q?[SPARK-15923][YARN]=20Spark=20Applica?= =?UTF-8?q?tion=20rest=20api=20returns=20'no=20such=20app:=20=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Update monitoring.md. …' Author: Weiqing Yang Closes #14163 from Sherry302/master. (cherry picked from commit 95abbe537751929353d18b733f6267c3287b6047) Signed-off-by: Sean Owen --- docs/monitoring.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index fa6c899a40b68..ee932cfc6d705 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -225,9 +225,10 @@ for the history server, they would typically be accessible at `http:// EndpointMeaning From b177e082f196d6d06247d801a8441929a2ff93bf Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 20 Jul 2016 09:48:52 -0700 Subject: [PATCH 1017/1470] [SPARK-16613][CORE] RDD.pipe returns values for empty partitions ## What changes were proposed in this pull request? Document RDD.pipe semantics; don't execute process for empty input partitions. Note this includes the fix in https://github.com/apache/spark/pull/14256 because it's necessary to even test this. One or the other will merge the fix. ## How was this patch tested? Jenkins tests including new test. Author: Sean Owen Closes #14260 from srowen/SPARK-16613. (cherry picked from commit 4b079dc3964dbe0f4d7839d39512d0400122b520) Signed-off-by: Reynold Xin --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 8 +++++++- .../test/scala/org/apache/spark/rdd/PipedRDDSuite.scala | 8 ++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 0804cdeb048a3..a4905dd51b940 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -714,7 +714,13 @@ abstract class RDD[T: ClassTag]( } /** - * Return an RDD created by piping elements to a forked external process. + * Return an RDD created by piping elements to a forked external process. The resulting RDD + * is computed by executing the given process once per partition. All elements + * of each input partition are written to a process's stdin as lines of input separated + * by a newline. The resulting partition consists of the process's stdout output, with + * each line of stdout resulting in one element of the output partition. A process is invoked + * even for empty partitions. + * * The print behavior can be customized by providing two functions. * * @param command command to run in forked process. diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 5d56fc19f00e1..f8d523fa2c6ae 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -138,6 +138,14 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("pipe with empty partition") { + val data = sc.parallelize(Seq("foo", "bing"), 8) + val piped = data.pipe("wc -c") + assert(piped.count == 8) + val charCounts = piped.map(_.trim.toInt).collect().toSet + assert(Set(0, 4, 5) == charCounts) + } + test("pipe with env variable") { if (testCommandAvailable("printenv")) { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) From 81004f13f0c173b46b82b731204874c493346d94 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 20 Jul 2016 10:38:44 -0700 Subject: [PATCH 1018/1470] [SPARK-16634][SQL] Workaround JVM bug by moving some code out of ctor. Some 1.7 JVMs have a bug that is triggered by certain Scala-generated bytecode. GenericArrayData suffers from that and fails to load in certain JVMs. Moving the offending code out of the constructor and into a helper method avoids the issue. Author: Marcelo Vanzin Closes #14271 from vanzin/SPARK-16634. (cherry picked from commit e3cd5b3050711af69fc1dfc518b11bf1a86b6a4c) Signed-off-by: Marcelo Vanzin --- .../sql/catalyst/util/GenericArrayData.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index 3a665d370830f..7ee9581b63af5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -23,6 +23,16 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{DataType, Decimal} import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +private object GenericArrayData { + + // SPARK-16634: Workaround for JVM bug present in some 1.7 versions. + def anyToSeq(seqOrArray: Any): Seq[Any] = seqOrArray match { + case seq: Seq[Any] => seq + case array: Array[_] => array.toSeq + } + +} + class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(seq: Seq[Any]) = this(seq.toArray) @@ -37,10 +47,7 @@ class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(primitiveArray: Array[Byte]) = this(primitiveArray.toSeq) def this(primitiveArray: Array[Boolean]) = this(primitiveArray.toSeq) - def this(seqOrArray: Any) = this(seqOrArray match { - case seq: Seq[Any] => seq - case array: Array[_] => array.toSeq - }) + def this(seqOrArray: Any) = this(GenericArrayData.anyToSeq(seqOrArray)) override def copy(): ArrayData = new GenericArrayData(array.clone()) From a804c92604253720fdccf117915db58c7768a6b8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 20 Jul 2016 18:37:15 -0700 Subject: [PATCH 1019/1470] [SPARK-16644][SQL] Aggregate should not propagate constraints containing aggregate expressions aggregate expressions can only be executed inside `Aggregate`, if we propagate it up with constraints, the parent operator can not execute it and will fail at runtime. new test in SQLQuerySuite Author: Wenchen Fan Author: Yin Huai Closes #14281 from cloud-fan/bug. (cherry picked from commit cfa5ae84ed0f48b3b108d0614dbf6fcd79ef5179) Signed-off-by: Yin Huai --- .../plans/logical/basicLogicalOperators.scala | 6 ++++-- .../plans/ConstraintPropagationSuite.scala | 6 ++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 17 +++++++++++++++++ 3 files changed, 25 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index c0e400f61777f..b31f5aa11c229 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -483,8 +483,10 @@ case class Aggregate( override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) override def maxRows: Option[Long] = child.maxRows - override def validConstraints: Set[Expression] = - child.constraints.union(getAliasedConstraints(aggregateExpressions)) + override def validConstraints: Set[Expression] = { + val nonAgg = aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) + child.constraints.union(getAliasedConstraints(nonAgg)) + } override lazy val statistics: Statistics = { if (groupingExpressions.isEmpty) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index 0b73b5e009b79..5a76969235acd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -79,13 +79,15 @@ class ConstraintPropagationSuite extends SparkFunSuite { assert(tr.analyze.constraints.isEmpty) val aliasedRelation = tr.where('c.attr > 10 && 'a.attr < 5) - .groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")).select('c1, 'a).analyze + .groupBy('a, 'c, 'b)('a, 'c.as("c1"), count('a).as("a3")).select('c1, 'a, 'a3).analyze + // SPARK-16644: aggregate expression count(a) should not appear in the constraints. verifyConstraints(aliasedRelation.analyze.constraints, ExpressionSet(Seq(resolveColumn(aliasedRelation.analyze, "c1") > 10, IsNotNull(resolveColumn(aliasedRelation.analyze, "c1")), resolveColumn(aliasedRelation.analyze, "a") < 5, - IsNotNull(resolveColumn(aliasedRelation.analyze, "a"))))) + IsNotNull(resolveColumn(aliasedRelation.analyze, "a")), + IsNotNull(resolveColumn(aliasedRelation.analyze, "a3"))))) } test("propagating constraints in expand") { 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 ede7d9a0c95b9..8a5ff2ce6a63d 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 @@ -2929,4 +2929,21 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SELECT '$literal' AS DUMMY"), Row(s"$expected") :: Nil) } + + test("SPARK-16644: Aggregate should not put aggregate expressions to constraints") { + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet") + checkAnswer(sql( + """ + |SELECT + | a, + | MAX(b) AS c1, + | b AS c2 + |FROM tbl + |WHERE a = b + |GROUP BY a, b + |HAVING c1 = 1 + """.stripMargin), Nil) + } + } } From c2b4228d7dcd275909969f8e1a938cae02820326 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 21 Jul 2016 09:17:38 +0100 Subject: [PATCH 1020/1470] [MINOR][DOCS][STREAMING] Minor docfix schema of csv rather than parquet in comments ## What changes were proposed in this pull request? Fix parquet to csv in a comment to match the input format being read. ## How was this patch tested? N/A (doc change only) Author: Holden Karau Closes #14274 from holdenk/minor-docfix-schema-of-csv-rather-than-parquet. (cherry picked from commit 1bf13ba3a24aa002bfaa660588490e527c5bf243) Signed-off-by: Sean Owen --- docs/structured-streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index aac88171fe3a4..8c14c3d220a23 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -449,7 +449,7 @@ val userSchema = new StructType().add("name", "string").add("age", "integer") val csvDF = spark .readStream .option("sep", ";") - .schema(userSchema) // Specify schema of the parquet files + .schema(userSchema) // Specify schema of the csv files .csv("/path/to/directory") // Equivalent to format("csv").load("/path/to/directory") {% endhighlight %} @@ -476,7 +476,7 @@ StructType userSchema = new StructType().add("name", "string").add("age", "integ Dataset[Row] csvDF = spark .readStream() .option("sep", ";") - .schema(userSchema) // Specify schema of the parquet files + .schema(userSchema) // Specify schema of the csv files .csv("/path/to/directory"); // Equivalent to format("csv").load("/path/to/directory") {% endhighlight %} From f9367d6a045ca171f86845b92c0def1d212a4fcc Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 21 Jul 2016 17:15:07 +0800 Subject: [PATCH 1021/1470] [SPARK-16632][SQL] Use Spark requested schema to guide vectorized Parquet reader initialization In `SpecificParquetRecordReaderBase`, which is used by the vectorized Parquet reader, we convert the Parquet requested schema into a Spark schema to guide column reader initialization. However, the Parquet requested schema is tailored from the schema of the physical file being scanned, and may have inaccurate type information due to bugs of other systems (e.g. HIVE-14294). On the other hand, we already set the real Spark requested schema into Hadoop configuration in [`ParquetFileFormat`][1]. This PR simply reads out this schema to replace the converted one. New test case added in `ParquetQuerySuite`. [1]: https://github.com/apache/spark/blob/v2.0.0-rc5/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L292-L294 Author: Cheng Lian Closes #14278 from liancheng/spark-16632-simpler-fix. (cherry picked from commit 8674054d3402b400a4766fe1c9214001cebf2106) Signed-off-by: Cheng Lian --- .../SpecificParquetRecordReaderBase.java | 5 +++- .../parquet/ParquetQuerySuite.scala | 24 +++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index 1a25679d19f90..0d624d17f4cd7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -60,6 +60,7 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Types; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.StructType$; /** * Base class for custom RecordReaders for Parquet that directly materialize to `T`. @@ -136,7 +137,9 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont ReadSupport.ReadContext readContext = readSupport.init(new InitContext( taskAttemptContext.getConfiguration(), toSetMultiMap(fileMetadata), fileSchema)); this.requestedSchema = readContext.getRequestedSchema(); - this.sparkSchema = new ParquetSchemaConverter(configuration).convert(requestedSchema); + String sparkRequestedSchemaString = + configuration.get(ParquetReadSupport$.MODULE$.SPARK_ROW_REQUESTED_SCHEMA()); + this.sparkSchema = StructType$.MODULE$.fromString(sparkRequestedSchemaString); this.reader = new ParquetFileReader(configuration, file, blocks, requestedSchema.getColumns()); for (BlockMetaData block : blocks) { this.totalRowCount += block.getRowCount(); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 83d10010f9dcb..3201f8e0dea1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -668,6 +668,30 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } } + + test("SPARK-16632: read Parquet int32 as ByteType and ShortType") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + withTempPath { dir => + val path = dir.getCanonicalPath + + // When being written to Parquet, `TINYINT` and `SMALLINT` should be converted into + // `int32 (INT_8)` and `int32 (INT_16)` respectively. However, Hive doesn't add the `INT_8` + // and `INT_16` annotation properly (HIVE-14294). Thus, when reading files written by Hive + // using Spark with the vectorized Parquet reader enabled, we may hit error due to type + // mismatch. + // + // Here we are simulating Hive's behavior by writing a single `INT` field and then read it + // back as `TINYINT` and `SMALLINT` in Spark to verify this issue. + Seq(1).toDF("f").write.parquet(path) + + val withByteField = new StructType().add("f", ByteType) + checkAnswer(spark.read.schema(withByteField).parquet(path), Row(1: Byte)) + + val withShortField = new StructType().add("f", ShortType) + checkAnswer(spark.read.schema(withShortField).parquet(path), Row(1: Short)) + } + } + } } object TestingUDT { From 933d76a2265176e0efee2d2b03ea53b235f2e175 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 21 Jul 2016 22:08:34 +0800 Subject: [PATCH 1022/1470] [SPARK-16632][SQL] Revert PR #14272: Respect Hive schema when merging parquet schema ## What changes were proposed in this pull request? PR #14278 is a more general and simpler fix for SPARK-16632 than PR #14272. After merging #14278, we no longer need changes made in #14272. So here I revert them. This PR targets both master and branch-2.0. ## How was this patch tested? Existing tests. Author: Cheng Lian Closes #14300 from liancheng/revert-pr-14272. (cherry picked from commit 69626adddc0441a4834b70a32e2d95b11d69a219) Signed-off-by: Cheng Lian --- .../parquet/ParquetReadSupport.scala | 18 --------- .../parquet/ParquetSchemaSuite.scala | 39 ------------------- 2 files changed, 57 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 1628e4c8ee660..12f497421f4b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -26,8 +26,6 @@ import org.apache.parquet.hadoop.api.{InitContext, ReadSupport} import org.apache.parquet.hadoop.api.ReadSupport.ReadContext import org.apache.parquet.io.api.RecordMaterializer import org.apache.parquet.schema._ -import org.apache.parquet.schema.OriginalType._ -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition import org.apache.spark.internal.Logging @@ -118,12 +116,6 @@ private[parquet] object ParquetReadSupport { } private def clipParquetType(parquetType: Type, catalystType: DataType): Type = { - val primName = if (parquetType.isPrimitive()) { - parquetType.asPrimitiveType().getPrimitiveTypeName() - } else { - null - } - catalystType match { case t: ArrayType if !isPrimitiveCatalystType(t.elementType) => // Only clips array types with nested type as element type. @@ -138,16 +130,6 @@ private[parquet] object ParquetReadSupport { case t: StructType => clipParquetGroup(parquetType.asGroupType(), t) - case _: ByteType if primName == INT32 => - // SPARK-16632: Handle case where Hive stores bytes in a int32 field without specifying - // the original type. - Types.primitive(INT32, parquetType.getRepetition()).as(INT_8).named(parquetType.getName()) - - case _: ShortType if primName == INT32 => - // SPARK-16632: Handle case where Hive stores shorts in a int32 field without specifying - // the original type. - Types.primitive(INT32, parquetType.getRepetition()).as(INT_16).named(parquetType.getName()) - case _ => // UDTs and primitive types are not clipped. For UDTs, a clipped version might not be able // to be mapped to desired user-space types. So UDTs shouldn't participate schema merging. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 215c1387ef00e..51bb236fe8441 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -1573,43 +1573,4 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | } |} """.stripMargin) - - testSchemaClipping( - "int32 parquet field with byte schema field", - - parquetSchema = - """message root { - | optional int32 value; - |} - """.stripMargin, - - catalystSchema = - new StructType() - .add("value", ByteType, nullable = true), - - expectedSchema = - """message root { - | optional int32 value (INT_8); - |} - """.stripMargin) - - testSchemaClipping( - "int32 parquet field with short schema field", - - parquetSchema = - """message root { - | optional int32 value; - |} - """.stripMargin, - - catalystSchema = - new StructType() - .add("value", ShortType, nullable = true), - - expectedSchema = - """message root { - | optional int32 value (INT_16); - |} - """.stripMargin) - } From cd41e6a33ed5ebbb256400ae504f3e0676f57387 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 21 Jul 2016 12:10:26 -0700 Subject: [PATCH 1023/1470] [SPARK-16656][SQL] Try to make CreateTableAsSelectSuite more stable ## What changes were proposed in this pull request? https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62593/testReport/junit/org.apache.spark.sql.sources/CreateTableAsSelectSuite/create_a_table__drop_it_and_create_another_one_with_the_same_name/ shows that `create a table, drop it and create another one with the same name` failed. But other runs were good. Seems it is a flaky test. This PR tries to make this test more stable. Author: Yin Huai Closes #14289 from yhuai/SPARK-16656. (cherry picked from commit 9abd99b3c318d0ec8b91124d40f3ab9e9d835dcf) Signed-off-by: Yin Huai --- .../sources/CreateTableAsSelectSuite.scala | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index f9a07dbdf0be0..251a25665a421 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.sources import java.io.File -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.TableIdentifier @@ -29,14 +29,16 @@ import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils -class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter { +class CreateTableAsSelectSuite + extends DataSourceTest + with SharedSQLContext + with BeforeAndAfterEach { protected override lazy val sql = spark.sql _ private var path: File = null override def beforeAll(): Unit = { super.beforeAll() - path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) spark.read.json(rdd).createOrReplaceTempView("jt") } @@ -44,18 +46,21 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with override def afterAll(): Unit = { try { spark.catalog.dropTempView("jt") - if (path.exists()) { - Utils.deleteRecursively(path) - } + Utils.deleteRecursively(path) } finally { super.afterAll() } } - before { - if (path.exists()) { - Utils.deleteRecursively(path) - } + override def beforeEach(): Unit = { + super.beforeEach() + path = Utils.createTempDir() + path.delete() + } + + override def afterEach(): Unit = { + Utils.deleteRecursively(path) + super.afterEach() } test("CREATE TABLE USING AS SELECT") { From 4cb8ff73fc91869716a579972166f41984fcdbf4 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Thu, 21 Jul 2016 15:34:32 -0700 Subject: [PATCH 1024/1470] [SPARK-16334] Maintain single dictionary per row-batch in vectorized parquet reader ## What changes were proposed in this pull request? As part of the bugfix in https://github.com/apache/spark/pull/12279, if a row batch consist of both dictionary encoded and non-dictionary encoded pages, we explicitly decode the dictionary for the values that are already dictionary encoded. Currently we reset the dictionary while reading every page that can potentially cause ` java.lang.ArrayIndexOutOfBoundsException` while decoding older pages. This patch fixes the problem by maintaining a single dictionary per row-batch in vectorized parquet reader. ## How was this patch tested? Manual Tests against a number of hand-generated parquet files. Author: Sameer Agarwal Closes #14225 from sameeragarwal/vectorized. (cherry picked from commit 46f80a307392bee6743e5847eb5243bf5fcd00a4) Signed-off-by: Reynold Xin --- .../parquet/VectorizedColumnReader.java | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index a18b881c78a09..6c47dc09a8637 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -59,7 +59,7 @@ public class VectorizedColumnReader { /** * If true, the current page is dictionary encoded. */ - private boolean useDictionary; + private boolean isCurrentPageDictionaryEncoded; /** * Maximum definition level for this column. @@ -100,13 +100,13 @@ public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader if (dictionaryPage != null) { try { this.dictionary = dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage); - this.useDictionary = true; + this.isCurrentPageDictionaryEncoded = true; } catch (IOException e) { throw new IOException("could not decode the dictionary for " + descriptor, e); } } else { this.dictionary = null; - this.useDictionary = false; + this.isCurrentPageDictionaryEncoded = false; } this.totalValueCount = pageReader.getTotalValueCount(); if (totalValueCount == 0) { @@ -136,6 +136,13 @@ private boolean next() throws IOException { */ void readBatch(int total, ColumnVector column) throws IOException { int rowId = 0; + ColumnVector dictionaryIds = null; + if (dictionary != null) { + // SPARK-16334: We only maintain a single dictionary per row batch, so that it can be used to + // decode all previous dictionary encoded pages if we ever encounter a non-dictionary encoded + // page. + dictionaryIds = column.reserveDictionaryIds(total); + } while (total > 0) { // Compute the number of values we want to read in this page. int leftInPage = (int) (endOfPageValueCount - valuesRead); @@ -144,12 +151,10 @@ void readBatch(int total, ColumnVector column) throws IOException { leftInPage = (int) (endOfPageValueCount - valuesRead); } int num = Math.min(total, leftInPage); - if (useDictionary) { + if (isCurrentPageDictionaryEncoded) { // Read and decode dictionary ids. - ColumnVector dictionaryIds = column.reserveDictionaryIds(total); defColumn.readIntegers( num, dictionaryIds, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); - if (column.hasDictionary() || (rowId == 0 && (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 || descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64 || @@ -461,13 +466,13 @@ private void initDataReader(Encoding dataEncoding, byte[] bytes, int offset) thr throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); } this.dataColumn = new VectorizedRleValuesReader(); - this.useDictionary = true; + this.isCurrentPageDictionaryEncoded = true; } else { if (dataEncoding != Encoding.PLAIN) { throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); } this.dataColumn = new VectorizedPlainValuesReader(); - this.useDictionary = false; + this.isCurrentPageDictionaryEncoded = false; } try { From 70bf8ce72a31f7c8b290abea5207244eb4b1fbab Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Fri, 22 Jul 2016 10:05:21 +0800 Subject: [PATCH 1025/1470] [SPARK-16287][SQL] Implement str_to_map SQL function ## What changes were proposed in this pull request? This PR adds `str_to_map` SQL function in order to remove Hive fallback. ## How was this patch tested? Pass the Jenkins tests with newly added. Author: Sandeep Singh Closes #13990 from techaddict/SPARK-16287. (cherry picked from commit df2c6d59d0e1a3db9942dbc5e4993cf3babc2d60) Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/complexTypeCreator.scala | 52 ++++++++++++++++++- .../expressions/ComplexTypeSuite.scala | 36 +++++++++++++ .../spark/sql/StringFunctionsSuite.scala | 23 ++++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 3 +- 5 files changed, 112 insertions(+), 3 deletions(-) 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 65a90d8099b7d..65168998c8aee 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 @@ -228,6 +228,7 @@ object FunctionRegistry { expression[Signum]("signum"), expression[Sin]("sin"), expression[Sinh]("sinh"), + expression[StringToMap]("str_to_map"), expression[Sqrt]("sqrt"), expression[Tan]("tan"), expression[Tanh]("tanh"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index d603d3c73ecbc..b3c5c585c5a52 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, TypeUtils} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData, TypeUtils} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -393,3 +393,53 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression override def prettyName: String = "named_struct_unsafe" } + +/** + * Creates a map after splitting the input text into key/value pairs using delimiters + */ +@ExpressionDescription( + usage = "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text " + + "into key/value pairs using delimiters. " + + "Default delimiters are ',' for pairDelim and ':' for keyValueDelim.", + extended = """ > SELECT _FUNC_('a:1,b:2,c:3',',',':');\n map("a":"1","b":"2","c":"3") """) +case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: Expression) + extends TernaryExpression with CodegenFallback with ExpectsInputTypes { + + def this(child: Expression, pairDelim: Expression) = { + this(child, pairDelim, Literal(":")) + } + + def this(child: Expression) = { + this(child, Literal(","), Literal(":")) + } + + override def children: Seq[Expression] = Seq(text, pairDelim, keyValueDelim) + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, StringType) + + override def dataType: DataType = MapType(StringType, StringType, valueContainsNull = false) + + override def checkInputDataTypes(): TypeCheckResult = { + if (Seq(pairDelim, keyValueDelim).exists(! _.foldable)) { + TypeCheckResult.TypeCheckFailure(s"$prettyName's delimiters must be foldable.") + } else { + super.checkInputDataTypes() + } + } + + override def nullSafeEval(str: Any, delim1: Any, delim2: Any): Any = { + val array = str.asInstanceOf[UTF8String] + .split(delim1.asInstanceOf[UTF8String], -1) + .map { kv => + val arr = kv.split(delim2.asInstanceOf[UTF8String], 2) + if (arr.length < 2) { + Array(arr(0), null) + } else { + arr + } + } + ArrayBasedMapData(array.map(_ (0)), array.map(_ (1))) + } + + override def prettyName: String = "str_to_map" +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index ec7be4d4b849d..0c307b2b8576b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -246,4 +246,40 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { checkMetadata(CreateStructUnsafe(Seq(a, b))) checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) } + + test("StringToMap") { + val s0 = Literal("a:1,b:2,c:3") + val m0 = Map("a" -> "1", "b" -> "2", "c" -> "3") + checkEvaluation(new StringToMap(s0), m0) + + val s1 = Literal("a: ,b:2") + val m1 = Map("a" -> " ", "b" -> "2") + checkEvaluation(new StringToMap(s1), m1) + + val s2 = Literal("a=1,b=2,c=3") + val m2 = Map("a" -> "1", "b" -> "2", "c" -> "3") + checkEvaluation(StringToMap(s2, Literal(","), Literal("=")), m2) + + val s3 = Literal("") + val m3 = Map[String, String]("" -> null) + checkEvaluation(StringToMap(s3, Literal(","), Literal("=")), m3) + + val s4 = Literal("a:1_b:2_c:3") + val m4 = Map("a" -> "1", "b" -> "2", "c" -> "3") + checkEvaluation(new StringToMap(s4, Literal("_")), m4) + + // arguments checking + assert(new StringToMap(Literal("a:1,b:2,c:3")).checkInputDataTypes().isSuccess) + assert(new StringToMap(Literal(null)).checkInputDataTypes().isFailure) + assert(new StringToMap(Literal("a:1,b:2,c:3"), Literal(null)).checkInputDataTypes().isFailure) + assert(StringToMap(Literal("a:1,b:2,c:3"), Literal(null), Literal(null)) + .checkInputDataTypes().isFailure) + assert(new StringToMap(Literal(null), Literal(null)).checkInputDataTypes().isFailure) + + assert(new StringToMap(Literal("a:1_b:2_c:3"), NonFoldableLiteral("_")) + .checkInputDataTypes().isFailure) + assert( + new StringToMap(Literal("a=1_b=2_c=3"), Literal("_"), NonFoldableLiteral("=")) + .checkInputDataTypes().isFailure) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index cbe480b525641..7bea2f6ad0dbb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -382,4 +382,27 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { }.getMessage assert(m.contains("Invalid number of arguments for function sentences")) } + + test("str_to_map function") { + val df1 = Seq( + ("a=1,b=2", "y"), + ("a=1,b=2,c=3", "y") + ).toDF("a", "b") + + checkAnswer( + df1.selectExpr("str_to_map(a,',','=')"), + Seq( + Row(Map("a" -> "1", "b" -> "2")), + Row(Map("a" -> "1", "b" -> "2", "c" -> "3")) + ) + ) + + val df2 = Seq(("a:1,b:2,c:3", "y")).toDF("a", "b") + + checkAnswer( + df2.selectExpr("str_to_map(a)"), + Seq(Row(Map("a" -> "1", "b" -> "2", "c" -> "3"))) + ) + + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index b8a75850b1d5e..c59ac3dcafea4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -238,7 +238,6 @@ private[sql] class HiveSessionCatalog( "hash", "histogram_numeric", "percentile", - "percentile_approx", - "str_to_map" + "percentile_approx" ) } From 0cc36cad1c6c1fbee6308f80178e9bbeebd4053a Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Fri, 22 Jul 2016 12:37:30 +0100 Subject: [PATCH 1026/1470] [SPARK-16287][HOTFIX][BUILD][SQL] Fix annotation argument needs to be a constant ## What changes were proposed in this pull request? Build fix for [SPARK-16287][SQL] Implement str_to_map SQL function that has introduced this compilation error: ``` /Users/jacek/dev/oss/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala:402: error: annotation argument needs to be a constant; found: "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text ".+("into key/value pairs using delimiters. ").+("Default delimiters are \',\' for pairDelim and \':\' for keyValueDelim.") "into key/value pairs using delimiters. " + ^ ``` ## How was this patch tested? Local build Author: Jacek Laskowski Closes #14315 from jaceklaskowski/build-fix-complexTypeCreator. (cherry picked from commit e1bd70f44b11141b000821e9754efeabc14f24a5) Signed-off-by: Sean Owen --- .../spark/sql/catalyst/expressions/complexTypeCreator.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index b3c5c585c5a52..0ca715f42472a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -397,11 +397,11 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression /** * Creates a map after splitting the input text into key/value pairs using delimiters */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text " + - "into key/value pairs using delimiters. " + - "Default delimiters are ',' for pairDelim and ':' for keyValueDelim.", + usage = "_FUNC_(text[, pairDelim, keyValueDelim]) - Creates a map after splitting the text into key/value pairs using delimiters. Default delimiters are ',' for pairDelim and ':' for keyValueDelim.", extended = """ > SELECT _FUNC_('a:1,b:2,c:3',',',':');\n map("a":"1","b":"2","c":"3") """) +// scalastyle:on line.size.limit case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: Expression) extends TernaryExpression with CodegenFallback with ExpectsInputTypes { From fb944a1e85a4d0e618cf7485afb0d0b39367fbda Mon Sep 17 00:00:00 2001 From: Tom Graves Date: Fri, 22 Jul 2016 12:41:38 +0100 Subject: [PATCH 1027/1470] [SPARK-16650] Improve documentation of spark.task.maxFailures Clarify documentation on spark.task.maxFailures No tests run as its documentation Author: Tom Graves Closes #14287 from tgravescs/SPARK-16650. (cherry picked from commit 6c56fff118ff2380c661456755db17976040de66) Signed-off-by: Sean Owen --- docs/configuration.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 1e95b862441f5..86a9bd97d3342 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1188,7 +1188,9 @@ Apart from these, the following properties are also available, and may be useful spark.task.maxFailures 4 - Number of individual task failures before giving up on the job. + Number of failures of any particular task before giving up on the job. + The total number of failures spread across different tasks will not cause the job + to fail; a particular task has to fail this number of attempts. Should be greater than or equal to 1. Number of allowed retries = this value - 1. From 28bb2b0447e9b47c4c568de983adde4a49b29263 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 22 Jul 2016 13:20:06 +0100 Subject: [PATCH 1028/1470] [SPARK-16651][PYSPARK][DOC] Make `withColumnRenamed/drop` description more consistent with Scala API ## What changes were proposed in this pull request? `withColumnRenamed` and `drop` is a no-op if the given column name does not exists. Python documentation also describe that, but this PR adds more explicit line consistently with Scala to reduce the ambiguity. ## How was this patch tested? It's about docs. Author: Dongjoon Hyun Closes #14288 from dongjoon-hyun/SPARK-16651. (cherry picked from commit 47f5b88db4d65f1870b16745d3c93d01051ba20b) Signed-off-by: Sean Owen --- python/pyspark/sql/dataframe.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index b9f50ff021a35..6aff938354955 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1376,6 +1376,7 @@ def withColumn(self, colName, col): @since(1.3) def withColumnRenamed(self, existing, new): """Returns a new :class:`DataFrame` by renaming an existing column. + This is a no-op if schema doesn't contain the given column name. :param existing: string, name of the existing column to rename. :param col: string, new name of the column. @@ -1389,6 +1390,7 @@ def withColumnRenamed(self, existing, new): @ignore_unicode_prefix def drop(self, col): """Returns a new :class:`DataFrame` that drops the specified column. + This is a no-op if schema doesn't contain the given column name(s). :param col: a string name of the column to drop, or a :class:`Column` to drop. From da34e8e8faaf7239f6dfe248812c83e1b2e2c1fd Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 23 Jul 2016 11:41:24 -0700 Subject: [PATCH 1029/1470] [SPARK-16380][EXAMPLES] Update SQL examples and programming guide for Python language binding This PR is based on PR #14098 authored by wangmiao1981. ## What changes were proposed in this pull request? This PR replaces the original Python Spark SQL example file with the following three files: - `sql/basic.py` Demonstrates basic Spark SQL features. - `sql/datasource.py` Demonstrates various Spark SQL data sources. - `sql/hive.py` Demonstrates Spark SQL Hive interaction. This PR also removes hard-coded Python example snippets in the SQL programming guide by extracting snippets from the above files using the `include_example` Liquid template tag. ## How was this patch tested? Manually tested. Author: wm624@hotmail.com Author: Cheng Lian Closes #14317 from liancheng/py-examples-update. (cherry picked from commit 53b2456d1de38b9d4f18509e7b36eb3fbe09e050) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 229 +----------------- .../sql/hive/JavaSparkHiveExample.java | 2 +- examples/src/main/python/sql.py | 83 ------- examples/src/main/python/sql/basic.py | 194 +++++++++++++++ examples/src/main/python/sql/datasource.py | 154 ++++++++++++ examples/src/main/python/sql/hive.py | 96 ++++++++ .../examples/sql/hive/SparkHiveExample.scala | 4 +- 7 files changed, 460 insertions(+), 302 deletions(-) delete mode 100644 examples/src/main/python/sql.py create mode 100644 examples/src/main/python/sql/basic.py create mode 100644 examples/src/main/python/sql/datasource.py create mode 100644 examples/src/main/python/sql/hive.py diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 8d92a4311eeb3..e92596bbc5b53 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -79,7 +79,7 @@ The entry point into all functionality in Spark is the [`SparkSession`](api/java The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: -{% include_example init_session python/sql.py %} +{% include_example init_session python/sql/basic.py %}
    @@ -123,14 +123,7 @@ from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: -{% highlight python %} -# spark is an existing SparkSession -df = spark.read.json("examples/src/main/resources/people.json") - -# Displays the content of the DataFrame to stdout -df.show() -{% endhighlight %} - +{% include_example create_df python/sql/basic.py %}
    @@ -178,53 +171,7 @@ interactive data exploration, users are highly encouraged to use the latter form, which is future proof and won't break with column names that are also attributes on the DataFrame class. -{% highlight python %} -# spark is an existing SparkSession - -# Create the DataFrame -df = spark.read.json("examples/src/main/resources/people.json") - -# Show the content of the DataFrame -df.show() -## age name -## null Michael -## 30 Andy -## 19 Justin - -# Print the schema in a tree format -df.printSchema() -## root -## |-- age: long (nullable = true) -## |-- name: string (nullable = true) - -# Select only the "name" column -df.select("name").show() -## name -## Michael -## Andy -## Justin - -# Select everybody, but increment the age by 1 -df.select(df['name'], df['age'] + 1).show() -## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 - -# Select people older than 21 -df.filter(df['age'] > 21).show() -## age name -## 30 Andy - -# Count people by age -df.groupBy("age").count().show() -## age count -## null 1 -## 19 1 -## 30 1 - -{% endhighlight %} - +{% include_example untyped_ops python/sql/basic.py %} For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). @@ -261,10 +208,7 @@ The `sql` function on a `SparkSession` enables applications to run SQL queries p
    The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. -{% highlight python %} -# spark is an existing SparkSession -df = spark.sql("SELECT * FROM table") -{% endhighlight %} +{% include_example run_sql python/sql/basic.py %}
    @@ -339,29 +283,7 @@ Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the dataty key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, and the types are inferred by sampling the whole datase, similar to the inference that is performed on JSON files. -{% highlight python %} -# spark is an existing SparkSession. -from pyspark.sql import Row -sc = spark.sparkContext - -# Load a text file and convert each line to a Row. -lines = sc.textFile("examples/src/main/resources/people.txt") -parts = lines.map(lambda l: l.split(",")) -people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) - -# Infer the schema, and register the DataFrame as a table. -schemaPeople = spark.createDataFrame(people) -schemaPeople.createOrReplaceTempView("people") - -# SQL can be run over DataFrames that have been registered as a table. -teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -# The results of SQL queries are RDDs and support all the normal RDD operations. -teenNames = teenagers.map(lambda p: "Name: " + p.name) -for teenName in teenNames.collect(): - print(teenName) -{% endhighlight %} - +{% include_example schema_inferring python/sql/basic.py %}
    @@ -419,39 +341,8 @@ tuples or lists in the RDD created in the step 1. 3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. For example: -{% highlight python %} -# Import SparkSession and data types -from pyspark.sql.types import * - -# spark is an existing SparkSession. -sc = spark.sparkContext - -# Load a text file and convert each line to a tuple. -lines = sc.textFile("examples/src/main/resources/people.txt") -parts = lines.map(lambda l: l.split(",")) -people = parts.map(lambda p: (p[0], p[1].strip())) - -# The schema is encoded in a string. -schemaString = "name age" - -fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()] -schema = StructType(fields) - -# Apply the schema to the RDD. -schemaPeople = spark.createDataFrame(people, schema) - -# Creates a temporary view using the DataFrame -schemaPeople.createOrReplaceTempView("people") - -# SQL can be run over DataFrames that have been registered as a table. -results = spark.sql("SELECT name FROM people") - -# The results of SQL queries are RDDs and support all the normal RDD operations. -names = results.map(lambda p: "Name: " + p.name) -for name in names.collect(): - print(name) -{% endhighlight %} +{% include_example programmatic_schema python/sql/basic.py %}
    @@ -481,13 +372,7 @@ In the simplest form, the default data source (`parquet` unless otherwise config
    -{% highlight python %} - -df = spark.read.load("examples/src/main/resources/users.parquet") -df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") - -{% endhighlight %} - +{% include_example generic_load_save_functions python/sql/datasource.py %}
    @@ -516,13 +401,7 @@ using this syntax.
    -{% highlight python %} - -df = spark.read.load("examples/src/main/resources/people.json", format="json") -df.select("name", "age").write.save("namesAndAges.parquet", format="parquet") - -{% endhighlight %} - +{% include_example manual_load_options python/sql/datasource.py %}
    @@ -547,10 +426,7 @@ file directly with SQL.
    -{% highlight python %} -df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") -{% endhighlight %} - +{% include_example direct_sql python/sql/datasource.py %}
    @@ -642,26 +518,7 @@ Using the data from the above example:
    -{% highlight python %} -# spark from the previous example is used in this example. - -schemaPeople # The DataFrame from the previous example. - -# DataFrames can be saved as Parquet files, maintaining the schema information. -schemaPeople.write.parquet("people.parquet") - -# 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. -parquetFile = spark.read.parquet("people.parquet") - -# Parquet files can also be used to create a temporary view and then used in SQL statements. -parquetFile.createOrReplaceTempView("parquetFile"); -teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") -teenNames = teenagers.map(lambda p: "Name: " + p.name) -for teenName in teenNames.collect(): - print(teenName) -{% endhighlight %} - +{% include_example basic_parquet_example python/sql/datasource.py %}
    @@ -775,33 +632,7 @@ turned it off by default starting from 1.5.0. You may enable it by
    -{% highlight python %} -# spark from the previous example is used in this example. - -# Create a simple DataFrame, stored into a partition directory -df1 = spark.createDataFrame(sc.parallelize(range(1, 6))\ - .map(lambda i: Row(single=i, double=i * 2))) -df1.write.parquet("data/test_table/key=1") - -# Create another DataFrame in a new partition directory, -# adding a new column and dropping an existing column -df2 = spark.createDataFrame(sc.parallelize(range(6, 11)) - .map(lambda i: Row(single=i, triple=i * 3))) -df2.write.parquet("data/test_table/key=2") - -# Read the partitioned table -df3 = spark.read.option("mergeSchema", "true").parquet("data/test_table") -df3.printSchema() - -# The final schema consists of all 3 columns in the Parquet files together -# with the partitioning column appeared in the partition directory paths. -# root -# |-- single: int (nullable = true) -# |-- double: int (nullable = true) -# |-- triple: int (nullable = true) -# |-- key : int (nullable = true) -{% endhighlight %} - +{% include_example schema_merging python/sql/datasource.py %}
    @@ -984,31 +815,7 @@ Note that the file that is offered as _a json file_ is not a typical JSON file. line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. -{% highlight python %} -# spark is an existing SparkSession. - -# A JSON dataset is pointed to by path. -# The path can be either a single text file or a directory storing text files. -people = spark.read.json("examples/src/main/resources/people.json") - -# The inferred schema can be visualized using the printSchema() method. -people.printSchema() -# root -# |-- age: long (nullable = true) -# |-- name: string (nullable = true) - -# Creates a temporary view using the DataFrame. -people.createOrReplaceTempView("people") - -# SQL statements can be run by using the sql methods provided by `spark`. -teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -# Alternatively, a DataFrame can be created for a JSON dataset represented by -# an RDD[String] storing one JSON object per string. -anotherPeopleRDD = sc.parallelize([ - '{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}']) -anotherPeople = spark.jsonRDD(anotherPeopleRDD) -{% endhighlight %} +{% include_example json_dataset python/sql/datasource.py %}
    @@ -1098,17 +905,7 @@ the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated sin Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. You may need to grant write privilege to the user who starts the spark application. -{% highlight python %} -# spark is an existing SparkSession - -spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") - -# Queries can be expressed in HiveQL. -results = spark.sql("FROM src SELECT key, value").collect() - -{% endhighlight %} - +{% include_example spark_hive python/sql/hive.py %}
    diff --git a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java index 493d759a916f8..76dd160d5568b 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java @@ -104,7 +104,7 @@ public String call(Row row) throws Exception { // |Key: 0, Value: val_0| // ... - // You can also use DataFrames to create temporary views within a HiveContext. + // You can also use DataFrames to create temporary views within a SparkSession. List records = new ArrayList<>(); for (int key = 1; key < 100; key++) { Record record = new Record(); diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py deleted file mode 100644 index ea11d2c4c7b33..0000000000000 --- a/examples/src/main/python/sql.py +++ /dev/null @@ -1,83 +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. -# - -from __future__ import print_function - -import os -import sys - -# $example on:init_session$ -from pyspark.sql import SparkSession -# $example off:init_session$ -from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType - - -if __name__ == "__main__": - # $example on:init_session$ - spark = SparkSession\ - .builder\ - .appName("PythonSQL")\ - .config("spark.some.config.option", "some-value")\ - .getOrCreate() - # $example off:init_session$ - - # A list of Rows. Infer schema from the first row, create a DataFrame and print the schema - rows = [Row(name="John", age=19), Row(name="Smith", age=23), Row(name="Sarah", age=18)] - some_df = spark.createDataFrame(rows) - some_df.printSchema() - - # A list of tuples - tuples = [("John", 19), ("Smith", 23), ("Sarah", 18)] - # Schema with two fields - person_name and person_age - schema = StructType([StructField("person_name", StringType(), False), - StructField("person_age", IntegerType(), False)]) - # Create a DataFrame by applying the schema to the RDD and print the schema - another_df = spark.createDataFrame(tuples, schema) - another_df.printSchema() - # root - # |-- age: long (nullable = true) - # |-- name: string (nullable = true) - - # A JSON dataset is pointed to by path. - # The path can be either a single text file or a directory storing text files. - if len(sys.argv) < 2: - path = "file://" + \ - os.path.join(os.environ['SPARK_HOME'], "examples/src/main/resources/people.json") - else: - path = sys.argv[1] - # Create a DataFrame from the file(s) pointed to by path - people = spark.read.json(path) - # root - # |-- person_name: string (nullable = false) - # |-- person_age: integer (nullable = false) - - # The inferred schema can be visualized using the printSchema() method. - people.printSchema() - # root - # |-- age: long (nullable = true) - # |-- name: string (nullable = true) - - # Creates a temporary view using the DataFrame. - people.createOrReplaceTempView("people") - - # SQL statements can be run by using the sql methods provided by `spark` - teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - - for each in teenagers.collect(): - print(each[0]) - - spark.stop() diff --git a/examples/src/main/python/sql/basic.py b/examples/src/main/python/sql/basic.py new file mode 100644 index 0000000000000..74f5009581e43 --- /dev/null +++ b/examples/src/main/python/sql/basic.py @@ -0,0 +1,194 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on:init_session$ +from pyspark.sql import SparkSession +# $example off:init_session$ + +# $example on:schema_inferring$ +from pyspark.sql import Row +# $example off:schema_inferring$ + +# $example on:programmatic_schema$ +# Import data types +from pyspark.sql.types import * +# $example off:programmatic_schema$ + +""" +A simple example demonstrating basic Spark SQL features. +Run with: + ./bin/spark-submit examples/src/main/python/sql/basic.py +""" + + +def basic_df_example(spark): + # $example on:create_df$ + # spark is an existing SparkSession + df = spark.read.json("examples/src/main/resources/people.json") + # Displays the content of the DataFrame to stdout + df.show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + # $example off:create_df$ + + # $example on:untyped_ops$ + # spark, df are from the previous example + # Print the schema in a tree format + df.printSchema() + # root + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) + + # Select only the "name" column + df.select("name").show() + # +-------+ + # | name| + # +-------+ + # |Michael| + # | Andy| + # | Justin| + # +-------+ + + # Select everybody, but increment the age by 1 + df.select(df['name'], df['age'] + 1).show() + # +-------+---------+ + # | name|(age + 1)| + # +-------+---------+ + # |Michael| null| + # | Andy| 31| + # | Justin| 20| + # +-------+---------+ + + # Select people older than 21 + df.filter(df['age'] > 21).show() + # +---+----+ + # |age|name| + # +---+----+ + # | 30|Andy| + # +---+----+ + + # Count people by age + df.groupBy("age").count().show() + # +----+-----+ + # | age|count| + # +----+-----+ + # | 19| 1| + # |null| 1| + # | 30| 1| + # +----+-----+ + # $example off:untyped_ops$ + + # $example on:run_sql$ + # Register the DataFrame as a SQL temporary view + df.createOrReplaceTempView("people") + + sqlDF = spark.sql("SELECT * FROM people") + sqlDF.show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + # $example off:run_sql$ + + +def schema_inference_example(spark): + # $example on:schema_inferring$ + sc = spark.sparkContext + + # Load a text file and convert each line to a Row. + lines = sc.textFile("examples/src/main/resources/people.txt") + parts = lines.map(lambda l: l.split(",")) + people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) + + # Infer the schema, and register the DataFrame as a table. + schemaPeople = spark.createDataFrame(people) + schemaPeople.createOrReplaceTempView("people") + + # SQL can be run over DataFrames that have been registered as a table. + teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + + # The results of SQL queries are Dataframe objects. + # rdd returns the content as an :class:`pyspark.RDD` of :class:`Row`. + teenNames = teenagers.rdd.map(lambda p: "Name: " + p.name).collect() + for name in teenNames: + print(name) + # Name: Justin + # $example off:schema_inferring$ + + +def programmatic_schema_example(spark): + # $example on:programmatic_schema$ + sc = spark.sparkContext + + # Load a text file and convert each line to a Row. + lines = sc.textFile("examples/src/main/resources/people.txt") + parts = lines.map(lambda l: l.split(",")) + # Each line is converted to a tuple. + people = parts.map(lambda p: (p[0], p[1].strip())) + + # The schema is encoded in a string. + schemaString = "name age" + + fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()] + schema = StructType(fields) + + # Apply the schema to the RDD. + schemaPeople = spark.createDataFrame(people, schema) + + # Creates a temporary view using the DataFrame + schemaPeople.createOrReplaceTempView("people") + + # Creates a temporary view using the DataFrame + schemaPeople.createOrReplaceTempView("people") + + # SQL can be run over DataFrames that have been registered as a table. + results = spark.sql("SELECT name FROM people") + + results.show() + # +-------+ + # | name| + # +-------+ + # |Michael| + # | Andy| + # | Justin| + # +-------+ + # $example off:programmatic_schema$ + +if __name__ == "__main__": + # $example on:init_session$ + spark = SparkSession \ + .builder \ + .appName("PythonSQL") \ + .config("spark.some.config.option", "some-value") \ + .getOrCreate() + # $example off:init_session$ + + basic_df_example(spark) + schema_inference_example(spark) + programmatic_schema_example(spark) + + spark.stop() diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py new file mode 100644 index 0000000000000..0bdc3d66ff984 --- /dev/null +++ b/examples/src/main/python/sql/datasource.py @@ -0,0 +1,154 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on:schema_merging$ +from pyspark.sql import Row +# $example off:schema_merging$ + +""" +A simple example demonstrating Spark SQL data sources. +Run with: + ./bin/spark-submit examples/src/main/python/sql/datasource.py +""" + + +def basic_datasource_example(spark): + # $example on:generic_load_save_functions$ + df = spark.read.load("examples/src/main/resources/users.parquet") + df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") + # $example off:generic_load_save_functions$ + + # $example on:manual_load_options$ + df = spark.read.load("examples/src/main/resources/people.json", format="json") + df.select("name", "age").write.save("namesAndAges.parquet", format="parquet") + # $example off:manual_load_options$ + + # $example on:direct_sql$ + df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") + # $example off:direct_sql$ + + +def parquet_example(spark): + # $example on:basic_parquet_example$ + peopleDF = spark.read.json("examples/src/main/resources/people.json") + + # DataFrames can be saved as Parquet files, maintaining the schema information. + peopleDF.write.parquet("people.parquet") + + # 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. + parquetFile = spark.read.parquet("people.parquet") + + # Parquet files can also be used to create a temporary view and then used in SQL statements. + parquetFile.createOrReplaceTempView("parquetFile") + teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") + teenagers.show() + # +------+ + # | name| + # +------+ + # |Justin| + # +------+ + # $example off:basic_parquet_example$ + + +def parquet_schema_merging_example(spark): + # $example on:schema_merging$ + # spark is from the previous example. + # Create a simple DataFrame, stored into a partition directory + sc = spark.sparkContext + + squaresDF = spark.createDataFrame(sc.parallelize(range(1, 6)) + .map(lambda i: Row(single=i, double=i ** 2))) + squaresDF.write.parquet("data/test_table/key=1") + + # Create another DataFrame in a new partition directory, + # adding a new column and dropping an existing column + cubesDF = spark.createDataFrame(sc.parallelize(range(6, 11)) + .map(lambda i: Row(single=i, triple=i ** 3))) + cubesDF.write.parquet("data/test_table/key=2") + + # Read the partitioned table + mergedDF = spark.read.option("mergeSchema", "true").parquet("data/test_table") + mergedDF.printSchema() + + # The final schema consists of all 3 columns in the Parquet files together + # with the partitioning column appeared in the partition directory paths. + # root + # |-- double: long (nullable = true) + # |-- single: long (nullable = true) + # |-- triple: long (nullable = true) + # |-- key: integer (nullable = true) + # $example off:schema_merging$ + + +def json_dataset_examplg(spark): + # $example on:json_dataset$ + # spark is from the previous example. + sc = spark.sparkContext + + # A JSON dataset is pointed to by path. + # The path can be either a single text file or a directory storing text files + path = "examples/src/main/resources/people.json" + peopleDF = spark.read.json(path) + + # The inferred schema can be visualized using the printSchema() method + peopleDF.printSchema() + # root + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) + + # Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + # SQL statements can be run by using the sql methods provided by spark + teenagerNamesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19") + teenagerNamesDF.show() + # +------+ + # | name| + # +------+ + # |Justin| + # +------+ + + # Alternatively, a DataFrame can be created for a JSON dataset represented by + # an RDD[String] storing one JSON object per string + jsonStrings = ['{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}'] + otherPeopleRDD = sc.parallelize(jsonStrings) + otherPeople = spark.read.json(otherPeopleRDD) + otherPeople.show() + # +---------------+----+ + # | address|name| + # +---------------+----+ + # |[Columbus,Ohio]| Yin| + # +---------------+----+ + # $example off:json_dataset$ + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("PythonSQL") \ + .getOrCreate() + + basic_datasource_example(spark) + parquet_example(spark) + parquet_schema_merging_example(spark) + json_dataset_examplg(spark) + + spark.stop() diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py new file mode 100644 index 0000000000000..d9ce5cef1f2b0 --- /dev/null +++ b/examples/src/main/python/sql/hive.py @@ -0,0 +1,96 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on:spark_hive$ +from os.path import expanduser, join + +from pyspark.sql import SparkSession +from pyspark.sql import Row +# $example off:spark_hive$ + +""" +A simple example demonstrating Spark SQL Hive integration. +Run with: + ./bin/spark-submit examples/src/main/python/sql/hive.py +""" + + +if __name__ == "__main__": + # $example on:spark_hive$ + # warehouse_location points to the default location for managed databases and tables + warehouse_location = 'file:${system:user.dir}/spark-warehouse' + + spark = SparkSession \ + .builder \ + .appName("PythonSQL") \ + .config("spark.sql.warehouse.dir", warehouse_location) \ + .enableHiveSupport() \ + .getOrCreate() + + # spark is an existing SparkSession + spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + + # Queries are expressed in HiveQL + spark.sql("SELECT * FROM src").show() + # +---+-------+ + # |key| value| + # +---+-------+ + # |238|val_238| + # | 86| val_86| + # |311|val_311| + # ... + + # Aggregation queries are also supported. + spark.sql("SELECT COUNT(*) FROM src").show() + # +--------+ + # |count(1)| + # +--------+ + # | 500 | + # +--------+ + + # The results of SQL queries are themselves DataFrames and support all normal functions. + sqlDF = spark.sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + + # The items in DaraFrames are of type Row, which allows you to access each column by ordinal. + stringsDS = sqlDF.rdd.map(lambda row: "Key: %d, Value: %s" % (row.key, row.value)) + for record in stringsDS.collect(): + print(record) + # Key: 0, Value: val_0 + # Key: 0, Value: val_0 + # Key: 0, Value: val_0 + # ... + + # You can also use DataFrames to create temporary views within a SparkSession. + Record = Row("key", "value") + recordsDF = spark.createDataFrame(map(lambda i: Record(i, "val_" + str(i)), range(1, 101))) + recordsDF.createOrReplaceTempView("records") + + # Queries can then join DataFrame data with data stored in Hive. + spark.sql("SELECT * FROM records r JOIN src s ON r.key = s.key").show() + # +---+------+---+------+ + # |key| value|key| value| + # +---+------+---+------+ + # | 2| val_2| 2| val_2| + # | 4| val_4| 4| val_4| + # | 5| val_5| 5| val_5| + # ... + # $example off:spark_hive$ + + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala index e897c2d066cfb..11e84c0e45632 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala @@ -87,7 +87,7 @@ object SparkHiveExample { // |Key: 0, Value: val_0| // ... - // You can also use DataFrames to create temporary views within a HiveContext. + // You can also use DataFrames to create temporary views within a SparkSession. val recordsDF = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) recordsDF.createOrReplaceTempView("records") @@ -97,8 +97,8 @@ object SparkHiveExample { // |key| value|key| value| // +---+------+---+------+ // | 2| val_2| 2| val_2| - // | 2| val_2| 2| val_2| // | 4| val_4| 4| val_4| + // | 5| val_5| 5| val_5| // ... // $example off:spark_hive$ From 31c3bcb46cb56b57d3cdcb8c42e7056dab0f7601 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 23 Jul 2016 11:39:48 -0700 Subject: [PATCH 1030/1470] [SPARK-16690][TEST] rename SQLTestUtils.withTempTable to withTempView after https://github.com/apache/spark/pull/12945, we renamed the `registerTempTable` to `createTempView`, as we do create a view actually. This PR renames `SQLTestUtils.withTempTable` to reflect this change. N/A Author: Wenchen Fan Closes #14318 from cloud-fan/minor4. (cherry picked from commit 86c275206605c44e1ebca2f166d62868e44bf029) Signed-off-by: Reynold Xin --- .../apache/spark/sql/CachedTableSuite.scala | 22 ++++++------ .../apache/spark/sql/MetadataCacheSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 34 +++++++++---------- .../apache/spark/sql/StatisticsSuite.scala | 4 +-- .../org/apache/spark/sql/SubquerySuite.scala | 4 +-- .../spark/sql/execution/PlannerSuite.scala | 8 ++--- .../sql/execution/command/DDLSuite.scala | 6 ++-- .../datasources/json/JsonSuite.scala | 6 ++-- .../ParquetPartitionDiscoverySuite.scala | 10 +++--- .../datasources/parquet/ParquetTest.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 8 ++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../apache/spark/sql/test/SQLTestUtils.scala | 2 +- .../spark/sql/hive/CachedTableSuite.scala | 10 +++--- .../sql/hive/HiveMetadataCacheSuite.scala | 2 +- .../spark/sql/hive/HiveParquetSuite.scala | 4 +-- .../sql/hive/MetastoreDataSourcesSuite.scala | 8 ++--- .../hive/ParquetHiveCompatibilitySuite.scala | 2 +- .../execution/AggregationQuerySuite.scala | 2 +- .../sql/hive/execution/HiveCommandSuite.scala | 4 +-- .../sql/hive/execution/HiveExplainSuite.scala | 2 +- .../hive/execution/HiveTableScanSuite.scala | 2 +- .../sql/hive/execution/HiveUDFSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 8 ++--- .../spark/sql/hive/orc/OrcQuerySuite.scala | 6 ++-- .../apache/spark/sql/hive/orc/OrcTest.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 2 +- .../sql/sources/HadoopFsRelationTest.scala | 12 +++---- 28 files changed, 89 insertions(+), 89 deletions(-) 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 6f6abfa93c1d8..f42402e1cc7d2 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 @@ -73,7 +73,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("cache temp table") { - withTempTable("tempTable") { + withTempView("tempTable") { testData.select('key).createOrReplaceTempView("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) spark.catalog.cacheTable("tempTable") @@ -97,7 +97,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("cache table as select") { - withTempTable("tempTable") { + withTempView("tempTable") { sql("CACHE TABLE tempTable AS SELECT key FROM testData") assertCached(sql("SELECT COUNT(*) FROM tempTable")) spark.catalog.uncacheTable("tempTable") @@ -227,7 +227,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { - withTempTable("testCacheTable") { + withTempView("testCacheTable") { sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") assertCached(spark.table("testCacheTable")) @@ -244,7 +244,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } test("CACHE TABLE tableName AS SELECT ...") { - withTempTable("testCacheTable") { + withTempView("testCacheTable") { sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10") assertCached(spark.table("testCacheTable")) @@ -413,7 +413,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // Set up two tables distributed in the same way. Try this with the data distributed into // different number of partitions. for (numPartitions <- 1 until 10 by 4) { - withTempTable("t1", "t2") { + withTempView("t1", "t2") { testData.repartition(numPartitions, $"key").createOrReplaceTempView("t1") testData2.repartition(numPartitions, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") @@ -435,7 +435,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } // Distribute the tables into non-matching number of partitions. Need to shuffle one side. - withTempTable("t1", "t2") { + withTempView("t1", "t2") { testData.repartition(6, $"key").createOrReplaceTempView("t1") testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") @@ -452,7 +452,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } // One side of join is not partitioned in the desired way. Need to shuffle one side. - withTempTable("t1", "t2") { + withTempView("t1", "t2") { testData.repartition(6, $"value").createOrReplaceTempView("t1") testData2.repartition(6, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") @@ -468,7 +468,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext spark.catalog.uncacheTable("t2") } - withTempTable("t1", "t2") { + withTempView("t1", "t2") { testData.repartition(6, $"value").createOrReplaceTempView("t1") testData2.repartition(12, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") @@ -487,7 +487,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // One side of join is not partitioned in the desired way. Since the number of partitions of // the side that has already partitioned is smaller than the side that is not partitioned, // we shuffle both side. - withTempTable("t1", "t2") { + withTempView("t1", "t2") { testData.repartition(6, $"value").createOrReplaceTempView("t1") testData2.repartition(3, $"a").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") @@ -504,7 +504,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // repartition's column ordering is different from group by column ordering. // But they use the same set of columns. - withTempTable("t1") { + withTempView("t1") { testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1") spark.catalog.cacheTable("t1") @@ -520,7 +520,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext // We will still shuffle because hashcodes of a row depend on the column ordering. // If we do not shuffle, we may actually partition two tables in totally two different way. // See PartitioningSuite for more details. - withTempTable("t1", "t2") { + withTempView("t1", "t2") { val df1 = testData df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1") val df2 = testData2.select($"a", $"b".cast("string")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala index 3f8cc8164d040..eacf254cd183d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -60,7 +60,7 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { } test("SPARK-16337 temporary view refresh") { - withTempTable("view_refresh") { withTempPath { (location: File) => + withTempView("view_refresh") { withTempPath { (location: File) => // Create a Parquet directory spark.range(start = 0, end = 100, step = 1, numPartitions = 3) .write.parquet(location.getAbsolutePath) 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 8a5ff2ce6a63d..f1a24109c8e96 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 @@ -805,7 +805,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("count of empty table") { - withTempTable("t") { + withTempView("t") { Seq.empty[(Int, Int)].toDF("a", "b").createOrReplaceTempView("t") checkAnswer( sql("select count(a) from t"), @@ -1671,7 +1671,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-7952: fix the equality check between boolean and numeric types") { - withTempTable("t") { + withTempView("t") { // numeric field i, boolean field j, result of i = j, result of i <=> j Seq[(Integer, java.lang.Boolean, java.lang.Boolean, java.lang.Boolean)]( (1, true, true, true), @@ -1691,7 +1691,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-7067: order by queries for complex ExtractValue chain") { - withTempTable("t") { + withTempView("t") { spark.read.json(sparkContext.makeRDD( """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).createOrReplaceTempView("t") checkAnswer(sql("SELECT a.b FROM t ORDER BY b[0].d"), Row(Seq(Row(1)))) @@ -1699,14 +1699,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-8782: ORDER BY NULL") { - withTempTable("t") { + withTempView("t") { Seq((1, 2), (1, 2)).toDF("a", "b").createOrReplaceTempView("t") checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), Row(1, 2))) } } test("SPARK-8837: use keyword in column name") { - withTempTable("t") { + withTempView("t") { val df = Seq(1 -> "a").toDF("count", "sort") checkAnswer(df.filter("count > 0"), Row(1, "a")) df.createOrReplaceTempView("t") @@ -1820,7 +1820,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-9511: error with table starting with number") { - withTempTable("1one") { + withTempView("1one") { sparkContext.parallelize(1 to 10).map(i => (i, i.toString)) .toDF("num", "str") .createOrReplaceTempView("1one") @@ -1864,7 +1864,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-10130 type coercion for IF should have children resolved first") { - withTempTable("src") { + withTempView("src") { Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer( sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) @@ -1872,7 +1872,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-10389: order by non-attribute grouping expression on Aggregate") { - withTempTable("src") { + withTempView("src") { Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1"), Seq(Row(1), Row(1))) @@ -1976,7 +1976,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-11032: resolve having correctly") { - withTempTable("src") { + withTempView("src") { Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("src") checkAnswer( sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t HAVING(COUNT(1) > 0)"), @@ -2081,7 +2081,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) // Try with a temporary view - withTempTable("nestedStructTable") { + withTempView("nestedStructTable") { nestedStructData.createOrReplaceTempView("nestedStructTable") checkAnswer( sql("SELECT record.* FROM nestedStructTable"), @@ -2104,7 +2104,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { | SELECT struct(`col$.a_`, `a.b.c.`) as `r&&b.c` FROM | (SELECT struct(a, b) as `col$.a_`, struct(b, a) as `a.b.c.` FROM testData2) tmp """.stripMargin) - withTempTable("specialCharacterTable") { + withTempView("specialCharacterTable") { specialCharacterPath.createOrReplaceTempView("specialCharacterTable") checkAnswer( specialCharacterPath.select($"`r&&b.c`.*"), @@ -2128,7 +2128,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Struct Star Expansion - Name conflict") { // Create a data set that contains a naming conflict val nameConflict = sql("SELECT struct(a, b) as nameConflict, a as a FROM testData2") - withTempTable("nameConflict") { + withTempView("nameConflict") { nameConflict.createOrReplaceTempView("nameConflict") // Unqualified should resolve to table. checkAnswer(sql("SELECT nameConflict.* FROM nameConflict"), @@ -2149,7 +2149,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("Star Expansion - table with zero column") { - withTempTable("temp_table_no_cols") { + withTempView("temp_table_no_cols") { val rddNoCols = sparkContext.parallelize(1 to 10).map(_ => Row.empty) val dfNoCols = spark.createDataFrame(rddNoCols, StructType(Seq.empty)) dfNoCols.createTempView("temp_table_no_cols") @@ -2464,7 +2464,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-13056: Null in map value causes NPE") { val df = Seq(1 -> Map("abc" -> "somestring", "cba" -> null)).toDF("key", "value") - withTempTable("maptest") { + withTempView("maptest") { df.createOrReplaceTempView("maptest") // local optimization will by pass codegen code, so we should keep the filter `key=1` checkAnswer(sql("SELECT value['abc'] FROM maptest where key = 1"), Row("somestring")) @@ -2474,7 +2474,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("hash function") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") - withTempTable("tbl") { + withTempView("tbl") { df.createOrReplaceTempView("tbl") checkAnswer( df.select(hash($"i", $"j")), @@ -2526,7 +2526,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("natural join") { val df1 = Seq(("one", 1), ("two", 2), ("three", 3)).toDF("k", "v1") val df2 = Seq(("one", 1), ("two", 22), ("one", 5)).toDF("k", "v2") - withTempTable("nt1", "nt2") { + withTempView("nt1", "nt2") { df1.createOrReplaceTempView("nt1") df2.createOrReplaceTempView("nt2") checkAnswer( @@ -2554,7 +2554,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ("r2c1", "r2c2", "t2r2c3"), ("r3c1y", "r3c2", "t2r3c3")).toDF("c1", "c2", "c3") val df3 = Seq((null, "r1c2", "t3r1c3"), ("r2c1", "r2c2", "t3r2c3"), ("r3c1y", "r3c2", "t3r3c3")).toDF("c1", "c2", "c3") - withTempTable("t1", "t2", "t3") { + withTempView("t1", "t2", "t3") { df1.createOrReplaceTempView("t1") df2.createOrReplaceTempView("t2") df3.createOrReplaceTempView("t3") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index ab55242ec0683..2c81cbf15f088 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -34,7 +34,7 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { } test("estimates the size of limit") { - withTempTable("test") { + withTempView("test") { Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") .createOrReplaceTempView("test") Seq((0, 1), (1, 24), (2, 48)).foreach { case (limit, expected) => @@ -58,7 +58,7 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { } test("estimates the size of a limit 0 on outer join") { - withTempTable("test") { + withTempView("test") { Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") .createOrReplaceTempView("test") val df1 = spark.table("test") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 1d9ff21dbf5d9..afed342ff8e2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -128,7 +128,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } test("SPARK-15677: Queries against local relations with scalar subquery in Select list") { - withTempTable("t1", "t2") { + withTempView("t1", "t2") { Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") @@ -267,7 +267,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } test("SPARK-15832: Test embedded existential predicate sub-queries") { - withTempTable("t1", "t2", "t3", "t4", "t5") { + withTempView("t1", "t2", "t3", "t4", "t5") { Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") Seq((1, 1), (2, 2), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t3") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index c96239e682018..13490c35679a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -71,7 +71,7 @@ class PlannerSuite extends SharedSQLContext { test("sizeInBytes estimation of limit operator for broadcast hash join optimization") { def checkPlan(fieldTypes: Seq[DataType]): Unit = { - withTempTable("testLimit") { + withTempView("testLimit") { val fields = fieldTypes.zipWithIndex.map { case (dataType, index) => StructField(s"c${index}", dataType, true) } :+ StructField("key", IntegerType, true) @@ -131,7 +131,7 @@ class PlannerSuite extends SharedSQLContext { test("InMemoryRelation statistics propagation") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "81920") { - withTempTable("tiny") { + withTempView("tiny") { testData.limit(3).createOrReplaceTempView("tiny") sql("CACHE TABLE tiny") @@ -157,7 +157,7 @@ class PlannerSuite extends SharedSQLContext { val df = spark.read.parquet(path) df.createOrReplaceTempView("testPushed") - withTempTable("testPushed") { + withTempView("testPushed") { val exp = sql("select * from testPushed where key = 15").queryExecution.sparkPlan assert(exp.toString.contains("PushedFilters: [IsNotNull(key), EqualTo(key,15)]")) } @@ -198,7 +198,7 @@ class PlannerSuite extends SharedSQLContext { } test("PartitioningCollection") { - withTempTable("normal", "small", "tiny") { + withTempView("normal", "small", "tiny") { testData.createOrReplaceTempView("normal") testData.limit(10).createOrReplaceTempView("small") testData.limit(3).createOrReplaceTempView("tiny") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b4294ed7ff1aa..f2ec393c30eca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -497,7 +497,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("rename temporary table - destination table with database name") { - withTempTable("tab1") { + withTempView("tab1") { sql( """ |CREATE TEMPORARY TABLE tab1 @@ -522,7 +522,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("rename temporary table - destination table already exists") { - withTempTable("tab1", "tab2") { + withTempView("tab1", "tab2") { sql( """ |CREATE TEMPORARY TABLE tab1 @@ -677,7 +677,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("show tables") { - withTempTable("show1a", "show2b") { + withTempView("show1a", "show2b") { sql( """ |CREATE TEMPORARY TABLE show1a diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 6c72019702c3d..0b0e64ac72730 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -1082,7 +1082,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("Corrupt records: PERMISSIVE mode") { // Test if we can query corrupt records. withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { - withTempTable("jsonTable") { + withTempView("jsonTable") { val jsonDF = spark.read.json(corruptRecords) jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( @@ -1518,7 +1518,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-12057 additional corrupt records do not throw exceptions") { // Test if we can query corrupt records. withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { - withTempTable("jsonTable") { + withTempView("jsonTable") { val schema = StructType( StructField("_unparsed", StringType, true) :: StructField("dummy", StringType, true) :: Nil) @@ -1635,7 +1635,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Casting long as timestamp") { - withTempTable("jsonTable") { + withTempView("jsonTable") { val schema = (new StructType).add("ts", TimestampType) val jsonDF = spark.read.schema(schema).json(timestampAsLong) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 133ffedf12812..8d18be9300f7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -404,7 +404,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -488,7 +488,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -537,7 +537,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) parquetRelation.createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -577,7 +577,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) parquetRelation.createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -613,7 +613,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha .load(base.getCanonicalPath) .createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), (1 to 10).map(i => Row(i, null, 1)) ++ (1 to 10).map(i => Row(i, i.toString, 2))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index 9fb34e03cb201..85efca3c4b24d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -91,7 +91,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { (f: => Unit): Unit = { withParquetDataFrame(data, testVectorized) { df => df.createOrReplaceTempView(tableName) - withTempTable(tableName)(f) + withTempView(tableName)(f) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 579a095ff000f..bba40c6510cfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -133,7 +133,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) testDataForJoin.createOrReplaceTempView("testDataForJoin") - withTempTable("testDataForJoin") { + withTempView("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) val df = spark.sql( @@ -151,7 +151,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // this test should use the deterministic number of partitions. val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) testDataForJoin.createOrReplaceTempView("testDataForJoin") - withTempTable("testDataForJoin") { + withTempView("testDataForJoin") { // Assume the execution plan is // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) val df = spark.sql( @@ -206,7 +206,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) testDataForJoin.createOrReplaceTempView("testDataForJoin") withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { - withTempTable("testDataForJoin") { + withTempView("testDataForJoin") { // Assume the execution plan is // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) val df = spark.sql( @@ -236,7 +236,7 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) testDataForJoin.createOrReplaceTempView("testDataForJoin") - withTempTable("testDataForJoin") { + withTempView("testDataForJoin") { // Assume the execution plan is // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) val df = spark.sql( 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 228e4250f3c66..995b1200a2294 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 @@ -752,7 +752,7 @@ class JDBCSuite extends SparkFunSuite assertEmptyQuery(s"SELECT * FROM foobar WHERE $FALSE1 AND ($FALSE2 OR $TRUE)") // Tests JDBCPartition whereClause clause push down. - withTempTable("tempFrame") { + withTempView("tempFrame") { val jdbcPartitionWhereClause = s"$FALSE1 OR $TRUE" val df = spark.read.jdbc( urlWithUserAndPass, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 26bd3fb7eb27b..5286ee5bc23d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -150,7 +150,7 @@ private[sql] trait SQLTestUtils /** * Drops temporary table `tableName` after calling `f`. */ - protected def withTempTable(tableNames: String*)(f: => Unit): Unit = { + protected def withTempView(tableNames: String*)(f: => Unit): Unit = { try f finally { // If the test failed part way, we don't want to mask the failure by failing to remove // temp tables that never got created. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index f7c3e347b61e1..7d4ef6f26a600 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -129,7 +129,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { - withTempTable("testCacheTable") { + withTempView("testCacheTable") { sql("CACHE TABLE testCacheTable AS SELECT * FROM src") assertCached(table("testCacheTable")) @@ -144,7 +144,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("CACHE TABLE tableName AS SELECT ...") { - withTempTable("testCacheTable") { + withTempView("testCacheTable") { sql("CACHE TABLE testCacheTable AS SELECT key FROM src LIMIT 10") assertCached(table("testCacheTable")) @@ -177,7 +177,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("CACHE TABLE with Hive UDF") { - withTempTable("udfTest") { + withTempView("udfTest") { sql("CACHE TABLE udfTest AS SELECT * FROM src WHERE floor(key) = 1") assertCached(table("udfTest")) uncacheTable("udfTest") @@ -276,7 +276,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto test("Cache/Uncache Qualified Tables") { withTempDatabase { db => - withTempTable("cachedTable") { + withTempView("cachedTable") { sql(s"CREATE TABLE $db.cachedTable STORED AS PARQUET AS SELECT 1") sql(s"CACHE TABLE $db.cachedTable") assertCached(spark.table(s"$db.cachedTable")) @@ -298,7 +298,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto test("Cache Table As Select - having database name") { withTempDatabase { db => - withTempTable("cachedTable") { + withTempView("cachedTable") { val e = intercept[ParseException] { sql(s"CACHE TABLE $db.cachedTable AS SELECT 1") }.getMessage diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala index 5714d06f0fe7a..3414f5e0409a1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.test.SQLTestUtils class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-16337 temporary view refresh") { - withTempTable("view_refresh") { + withTempView("view_refresh") { withTable("view_table") { // Create a Parquet directory spark.range(start = 0, end = 100, step = 1, numPartitions = 3) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index 33252ad07add9..09c15473b21c1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -52,7 +52,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton withTempPath { dir => sql("SELECT * FROM src").write.parquet(dir.getCanonicalPath) spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("p") - withTempTable("p") { + withTempView("p") { checkAnswer( sql("SELECT * FROM src ORDER BY key"), sql("SELECT * from p ORDER BY key").collect().toSeq) @@ -66,7 +66,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton withTempPath { file => sql("SELECT * FROM t LIMIT 1").write.parquet(file.getCanonicalPath) spark.read.parquet(file.getCanonicalPath).createOrReplaceTempView("p") - withTempTable("p") { + withTempView("p") { // let's do three overwrites for good measure sql("INSERT OVERWRITE TABLE p SELECT * FROM t") sql("INSERT OVERWRITE TABLE p SELECT * FROM t") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 12d250d4fb604..af071f95e69fa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -79,7 +79,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv |) """.stripMargin) - withTempTable("expectedJsonTable") { + withTempView("expectedJsonTable") { read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM jsonTable"), @@ -109,7 +109,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert(expectedSchema === table("jsonTable").schema) - withTempTable("expectedJsonTable") { + withTempView("expectedJsonTable") { read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT b, ``.`=` FROM jsonTable"), @@ -247,7 +247,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv |) """.stripMargin) - withTempTable("expectedJsonTable") { + withTempView("expectedJsonTable") { read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( @@ -553,7 +553,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("scan a parquet table created through a CTAS statement") { withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { - withTempTable("jt") { + withTempView("jt") { (1 to 10).map(i => i -> s"str$i").toDF("a", "b").createOrReplaceTempView("jt") withTable("test_parquet_ctas") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index ac89bbbf8e19d..7d429f4723f51 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -54,7 +54,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // Don't convert Hive metastore Parquet tables to let Hive write those Parquet files. withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { - withTempTable("data") { + withTempView("data") { val fields = hiveTypes.zipWithIndex.map { case (typ, index) => s" col_$index $typ" } val ddl = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index a16fe3228b1fc..2dcf13c02a466 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -923,7 +923,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te } test("udaf without specifying inputSchema") { - withTempTable("noInputSchemaUDAF") { + withTempView("noInputSchemaUDAF") { spark.udf.register("noInputSchema", new ScalaAggregateFunctionWithoutInputSchema) val data = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 741abcb7513cb..5d510197c4d95 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -139,7 +139,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("show tblproperties for spark temporary table - empty row") { - withTempTable("parquet_temp") { + withTempView("parquet_temp") { sql( """ |CREATE TEMPORARY TABLE parquet_temp (c1 INT, c2 STRING) @@ -397,7 +397,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("show partitions - empty row") { - withTempTable("parquet_temp") { + withTempView("parquet_temp") { sql( """ |CREATE TEMPORARY TABLE parquet_temp (c1 INT, c2 STRING) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index a43eed9a2a4fd..98afd99a203ac 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -78,7 +78,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") { - withTempTable("jt") { + withTempView("jt") { val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) spark.read.json(rdd).createOrReplaceTempView("jt") val outputs = sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 76d3f3dbab01f..5b464764f0a99 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -102,7 +102,7 @@ class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestH test("Verify SQLConf HIVE_METASTORE_PARTITION_PRUNING") { val view = "src" - withTempTable(view) { + withTempView(view) { spark.range(1, 5).createOrReplaceTempView(view) val table = "table_with_partition" withTable(table) { 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 def4601cf6156..f690035c845f7 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 @@ -358,7 +358,7 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } test("Hive UDF in group by") { - withTempTable("tab1") { + withTempView("tab1") { Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") val count = sql("select testUDFToDate(cast(test_date as timestamp))" + 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 84fa7f966f66b..9a36fd6c99432 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 @@ -122,7 +122,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("SPARK-13651: generator outputs shouldn't be resolved from its child's output") { - withTempTable("src") { + withTempView("src") { Seq(("id1", "value1")).toDF("key", "value").createOrReplaceTempView("src") val query = sql("SELECT genoutput.* FROM src " + @@ -952,7 +952,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("Sorting columns are not in Generate") { - withTempTable("data") { + withTempView("data") { spark.range(1, 5) .select(array($"id", $"id" + 1).as("a"), $"id".as("b"), (lit(10) - $"id").as("c")) .createOrReplaceTempView("data") @@ -1229,7 +1229,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-10741: Sort on Aggregate using parquet") { withTable("test10741") { - withTempTable("src") { + withTempView("src") { Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF("c1", "c2").createOrReplaceTempView("src") sql("CREATE TABLE test10741 STORED AS PARQUET AS SELECT * FROM src") } @@ -1483,7 +1483,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("multi-insert with lateral view") { - withTempTable("t1") { + withTempView("t1") { spark.range(10) .select(array($"id", $"id" + 1).as("arr"), $"id") .createOrReplaceTempView("source") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index cd41da7214a23..46595ee8186aa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -92,7 +92,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("Creating case class RDD table") { val data = (1 to 100).map(i => (i, s"val_$i")) sparkContext.parallelize(data).toDF().createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer(sql("SELECT * FROM t"), data.toDF().collect()) } } @@ -309,7 +309,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { val path = dir.getCanonicalPath withTable("empty_orc") { - withTempTable("empty", "single") { + withTempView("empty", "single") { spark.sql( s"""CREATE TABLE empty_orc(key INT, value STRING) |STORED AS ORC @@ -401,7 +401,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } test("Verify the ORC conversion parameter: CONVERT_METASTORE_ORC") { - withTempTable("single") { + withTempView("single") { val singleRowDF = Seq((0, "foo")).toDF("key", "value") singleRowDF.createOrReplaceTempView("single") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala index 2a647115b7e01..7226ed521ef32 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala @@ -62,7 +62,7 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton { (f: => Unit): Unit = { withOrcDataFrame(data) { df => df.createOrReplaceTempView(tableName) - withTempTable(tableName)(f) + withTempView(tableName)(f) } } 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 96beb2d3427b1..31b6197d56fc7 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 @@ -702,7 +702,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { } test("Verify the PARQUET conversion parameter: CONVERT_METASTORE_PARQUET") { - withTempTable("single") { + withTempView("single") { val singleRowDF = Seq((0, "foo")).toDF("key", "value") singleRowDF.createOrReplaceTempView("single") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 62998572eaf4b..047b08c4ccf65 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -92,7 +92,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes // Self-join df.createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql( """SELECT l.a, r.b, l.p1, r.p2 @@ -339,7 +339,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { intercept[AnalysisException] { testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).saveAsTable("t") } @@ -349,7 +349,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes test("saveAsTable()/load() - non-partitioned table - Ignore") { Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t") assert(spark.table("t").collect().isEmpty) } @@ -461,7 +461,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes test("saveAsTable()/load() - partitioned table - ErrorIfExists") { Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { intercept[AnalysisException] { partitionedTestDF.write .format(dataSourceName) @@ -476,7 +476,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes test("saveAsTable()/load() - partitioned table - Ignore") { Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { partitionedTestDF.write .format(dataSourceName) .mode(SaveMode.Ignore) @@ -722,7 +722,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes 'p3.cast(FloatType).as('pf1), 'f) - withTempTable("t") { + withTempView("t") { input .write .format(dataSourceName) From 198b0426e07f3d4b1fbbef21d39daa32a75da36c Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Sun, 24 Jul 2016 08:35:57 +0100 Subject: [PATCH 1031/1470] [SPARK-16515][SQL][FOLLOW-UP] Fix test `script` on OS X/Windows... The current `sed` in `test_script.sh` is missing a `$`, leading to the failure of `script` test on OS X: ``` == Results == !== Correct Answer - 2 == == Spark Answer - 2 == ![x1_y1] [x1] ![x2_y2] [x2] ``` In addition, this `script` test would also fail on systems like Windows where we couldn't be able to invoke `bash` or `echo | sed`. This patch - fixes `sed` in `test_script.sh` - adds command guards so that the `script` test would pass on systems like Windows - Jenkins - Manually verified tests pass on OS X Author: Liwei Lin Closes #14280 from lw-lin/osx-sed. (cherry picked from commit d6795c7a254b83d4ae4785f3add74981e5273c91) Signed-off-by: Sean Owen --- sql/hive/src/test/resources/test_script.sh | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 26 +++++++++++++------ 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/sql/hive/src/test/resources/test_script.sh b/sql/hive/src/test/resources/test_script.sh index ab998c41b22ce..eb0c50e98292c 100755 --- a/sql/hive/src/test/resources/test_script.sh +++ b/sql/hive/src/test/resources/test_script.sh @@ -19,5 +19,5 @@ while read line do - echo "$line" | sed 's/\t/_/' + echo "$line" | sed $'s/\t/_/' done < /dev/stdin 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 9a36fd6c99432..c4d9e0aee911f 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 @@ -19,6 +19,9 @@ package org.apache.spark.sql.hive.execution import java.sql.{Date, Timestamp} +import scala.sys.process.Process +import scala.util.Try + import org.apache.hadoop.fs.Path import org.apache.spark.sql._ @@ -64,14 +67,17 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import spark.implicits._ test("script") { - val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") - df.createOrReplaceTempView("script_table") - val query1 = sql( - """ - |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table - |REDUCE c1, c2, c3 USING 'bash src/test/resources/test_script.sh' AS - |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) - checkAnswer(query1, Row("x1_y1") :: Row("x2_y2") :: Nil) + if (testCommandAvailable("bash") && testCommandAvailable("echo | sed")) { + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query1 = sql( + """ + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash src/test/resources/test_script.sh' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query1, Row("x1_y1") :: Row("x2_y2") :: Nil) + } + // else skip this test } test("UDTF") { @@ -1677,4 +1683,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } } + + def testCommandAvailable(command: String): Boolean = { + Try(Process(command) !!).isSuccess + } } From d226dce12babcd9f30db033417b2b9ce79f44312 Mon Sep 17 00:00:00 2001 From: Qifan Pu Date: Sun, 24 Jul 2016 21:53:21 -0700 Subject: [PATCH 1032/1470] [SPARK-16699][SQL] Fix performance bug in hash aggregate on long string keys ## What changes were proposed in this pull request? In the following code in `VectorizedHashMapGenerator.scala`: ``` def hashBytes(b: String): String = { val hash = ctx.freshName("hash") s""" |int $result = 0; |for (int i = 0; i < $b.length; i++) { | ${genComputeHash(ctx, s"$b[i]", ByteType, hash)} | $result = ($result ^ (0x9e3779b9)) + $hash + ($result << 6) + ($result >>> 2); |} """.stripMargin } ``` when b=input.getBytes(), the current 2.0 code results in getBytes() being called n times, n being length of input. getBytes() involves memory copy is thus expensive and causes a performance degradation. Fix is to evaluate getBytes() before the for loop. ## How was this patch tested? Performance bug, no additional test added. Author: Qifan Pu Closes #14337 from ooq/SPARK-16699. --- .../execution/aggregate/VectorizedHashMapGenerator.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala index 8a3f466ccfef3..b4a9059299539 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/VectorizedHashMapGenerator.scala @@ -313,10 +313,12 @@ class VectorizedHashMapGenerator( def hashLong(l: String): String = s"long $result = $l;" def hashBytes(b: String): String = { val hash = ctx.freshName("hash") + val bytes = ctx.freshName("bytes") s""" |int $result = 0; - |for (int i = 0; i < $b.length; i++) { - | ${genComputeHash(ctx, s"$b[i]", ByteType, hash)} + |byte[] $bytes = $b; + |for (int i = 0; i < $bytes.length; i++) { + | ${genComputeHash(ctx, s"$bytes[i]", ByteType, hash)} | $result = ($result ^ (0x9e3779b9)) + $hash + ($result << 6) + ($result >>> 2); |} """.stripMargin From fcbb7f653df11d923a208c5af03c0a6b9a472376 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 25 Jul 2016 17:22:29 +0800 Subject: [PATCH 1033/1470] [SPARK-16648][SQL] Make ignoreNullsExpr a child expression of First and Last ## What changes were proposed in this pull request? Default `TreeNode.withNewChildren` implementation doesn't work for `Last` and when both constructor arguments are the same, e.g.: ```sql LAST_VALUE(FALSE) -- The 2nd argument defaults to FALSE LAST_VALUE(FALSE, FALSE) LAST_VALUE(TRUE, TRUE) ``` This is because although `Last` is a unary expression, both of its constructor arguments, `child` and `ignoreNullsExpr`, are `Expression`s. When they have the same value, `TreeNode.withNewChildren` treats both of them as child nodes by mistake. `First` is also affected by this issue in exactly the same way. This PR fixes this issue by making `ignoreNullsExpr` a child expression of `First` and `Last`. ## How was this patch tested? New test case added in `WindowQuerySuite`. Author: Cheng Lian Closes #14295 from liancheng/spark-16648-last-value. (cherry picked from commit 68b4020d0c0d4f063facfbf4639ef4251dcfda8b) Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/aggregate/First.scala | 4 ++-- .../sql/catalyst/expressions/aggregate/Last.scala | 4 ++-- .../spark/sql/hive/execution/WindowQuerySuite.scala | 12 ++++++++++++ 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala index 946b3d446a40f..d702c08cfd342 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala @@ -43,7 +43,7 @@ case class First(child: Expression, ignoreNullsExpr: Expression) extends Declara throw new AnalysisException("The second argument of First should be a boolean literal.") } - override def children: Seq[Expression] = child :: Nil + override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil override def nullable: Boolean = true @@ -54,7 +54,7 @@ case class First(child: Expression, ignoreNullsExpr: Expression) extends Declara override def dataType: DataType = child.dataType // Expected input data type. - override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType) + override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, BooleanType) private lazy val first = AttributeReference("first", child.dataType)() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index 53b4b761ae514..af8840305805f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -40,7 +40,7 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) extends Declarat throw new AnalysisException("The second argument of First should be a boolean literal.") } - override def children: Seq[Expression] = child :: Nil + override def children: Seq[Expression] = child :: ignoreNullsExpr :: Nil override def nullable: Boolean = true @@ -51,7 +51,7 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) extends Declarat override def dataType: DataType = child.dataType // Expected input data type. - override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType) + override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, BooleanType) private lazy val last = AttributeReference("last", child.dataType)() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala index c6b7eb63662c5..0ff3511c87a4f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala @@ -247,4 +247,16 @@ class WindowQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleto |from part """.stripMargin)) } + + test("SPARK-16646: LAST_VALUE(FALSE) OVER ()") { + checkAnswer(sql("SELECT LAST_VALUE(FALSE) OVER ()"), Row(false)) + checkAnswer(sql("SELECT LAST_VALUE(FALSE, FALSE) OVER ()"), Row(false)) + checkAnswer(sql("SELECT LAST_VALUE(TRUE, TRUE) OVER ()"), Row(true)) + } + + test("SPARK-16646: FIRST_VALUE(FALSE) OVER ()") { + checkAnswer(sql("SELECT FIRST_VALUE(FALSE) OVER ()"), Row(false)) + checkAnswer(sql("SELECT FIRST_VALUE(FALSE, FALSE) OVER ()"), Row(false)) + checkAnswer(sql("SELECT FIRST_VALUE(TRUE, TRUE) OVER ()"), Row(true)) + } } From b52e639a84a851e0b9159a0f6dae92664425042e Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 25 Jul 2016 22:51:30 +0800 Subject: [PATCH 1034/1470] [SPARK-16698][SQL] Field names having dots should be allowed for datasources based on FileFormat ## What changes were proposed in this pull request? It seems this is a regression assuming from https://issues.apache.org/jira/browse/SPARK-16698. Field name having dots throws an exception. For example the codes below: ```scala val path = "/tmp/path" val json =""" {"a.b":"data"}""" spark.sparkContext .parallelize(json :: Nil) .saveAsTextFile(path) spark.read.json(path).collect() ``` throws an exception as below: ``` Unable to resolve a.b given [a.b]; org.apache.spark.sql.AnalysisException: Unable to resolve a.b given [a.b]; at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134) at scala.Option.getOrElse(Option.scala:121) ``` This problem was introduced in https://github.com/apache/spark/commit/17eec0a71ba8713c559d641e3f43a1be726b037c#diff-27c76f96a7b2733ecfd6f46a1716e153R121 When extracting the data columns, it does not count that it can contains dots in field names. Actually, it seems the fields name are not expected as quoted when defining schema. So, It not have to consider whether this is wrapped with quotes because the actual schema (inferred or user-given schema) would not have the quotes for fields. For example, this throws an exception. (**Loading JSON from RDD is fine**) ```scala val json =""" {"a.b":"data"}""" val rdd = spark.sparkContext.parallelize(json :: Nil) spark.read.schema(StructType(Seq(StructField("`a.b`", StringType, true)))) .json(rdd).select("`a.b`").printSchema() ``` as below: ``` cannot resolve '```a.b```' given input columns: [`a.b`]; org.apache.spark.sql.AnalysisException: cannot resolve '```a.b```' given input columns: [`a.b`]; at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42) ``` ## How was this patch tested? Unit tests in `FileSourceStrategySuite`. Author: hyukjinkwon Closes #14339 from HyukjinKwon/SPARK-16698-regression. (cherry picked from commit 79826f3c7936ee27457d030c7115d5cac69befd7) Signed-off-by: Cheng Lian --- .../sql/catalyst/plans/logical/LogicalPlan.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) 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 d0b2b5d7b2df6..6d7799151d93b 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 @@ -127,7 +127,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { */ def resolve(schema: StructType, resolver: Resolver): Seq[Attribute] = { schema.map { field => - resolveQuoted(field.name, resolver).map { + resolve(field.name :: Nil, resolver).map { case a: AttributeReference => a case other => sys.error(s"can not handle nested schema yet... plan $this") }.getOrElse { 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 f1a24109c8e96..be84dff09239d 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 @@ -2946,4 +2946,19 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { """.stripMargin), Nil) } } + + test("SPARK-16674: field names containing dots for both fields and partitioned fields") { + withTempPath { path => + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "a" else "b")) + .toDF("col.1", "col.2", "part.col1", "part.col2") + data.write + .format("parquet") + .partitionBy("part.col1", "part.col2") + .save(path.getCanonicalPath) + val readBack = spark.read.format("parquet").load(path.getCanonicalPath) + checkAnswer( + readBack.selectExpr("`part.col1`", "`col.1`"), + data.selectExpr("`part.col1`", "`col.1`")) + } + } } From 57d65e5111e281d3d5224c5ea11005c89718f791 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 25 Jul 2016 09:42:39 -0700 Subject: [PATCH 1035/1470] [SPARK-16703][SQL] Remove extra whitespace in SQL generation for window functions ## What changes were proposed in this pull request? This PR fixes a minor formatting issue of `WindowSpecDefinition.sql` when no partitioning expressions are present. Before: ```sql ( ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) ``` After: ```sql (ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) ``` ## How was this patch tested? New test case added in `ExpressionSQLBuilderSuite`. Author: Cheng Lian Closes #14334 from liancheng/window-spec-sql-format. (cherry picked from commit 7ea6d282b925819ddb3874a67b3c9da8cc41f131) Signed-off-by: Reynold Xin --- .../expressions/windowExpressions.scala | 6 ++-- .../sqlgen/aggregate_functions_and_window.sql | 2 +- .../sqlgen/regular_expressions_and_window.sql | 2 +- .../test/resources/sqlgen/window_basic_1.sql | 2 +- .../test/resources/sqlgen/window_basic_2.sql | 2 +- .../catalyst/ExpressionSQLBuilderSuite.scala | 35 +++++++++++++++++-- 6 files changed, 40 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index c0b453dccf5e9..e35192ca2d26c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -82,16 +82,16 @@ case class WindowSpecDefinition( val partition = if (partitionSpec.isEmpty) { "" } else { - "PARTITION BY " + partitionSpec.map(_.sql).mkString(", ") + "PARTITION BY " + partitionSpec.map(_.sql).mkString(", ") + " " } val order = if (orderSpec.isEmpty) { "" } else { - "ORDER BY " + orderSpec.map(_.sql).mkString(", ") + "ORDER BY " + orderSpec.map(_.sql).mkString(", ") + " " } - s"($partition $order ${frameSpecification.toString})" + s"($partition$order${frameSpecification.toString})" } } diff --git a/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql index 3a29bcf2ae442..c94f53b0a9f44 100644 --- a/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql +++ b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(c) + COUNT(a) OVER () FROM parquet_t2 GROUP BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(max(c) + count(a) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, count(`gen_attr`) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr` AS `(max(c) + count(a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, count(`gen_attr`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql index 7d837d490f125..a7909485fbc5f 100644 --- a/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql +++ b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(key) OVER (PARTITION BY key % 3) + key FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr` AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_1.sql b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql index 73f343d1018c1..f86ca5e086b52 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_1.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(value) OVER (PARTITION BY key % 3) FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr` AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql index 3ddb87f91b88d..15559ac8e15d4 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql @@ -2,4 +2,4 @@ SELECT key, value, ROUND(AVG(key) OVER (), 2) FROM parquet_t1 ORDER BY key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `round(avg(key) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr`, `gen_attr`, round(`gen_attr`, 2) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, avg(`gen_attr`) OVER ( ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr` ASC) AS parquet_t1 +SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr`, `gen_attr`, round(`gen_attr`, 2) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, avg(`gen_attr`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr` ASC) AS parquet_t1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index c8bf20d13bdba..fef726c5d801d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.catalyst import java.sql.Timestamp import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{If, Literal} - +import org.apache.spark.sql.catalyst.expressions.{If, Literal, SpecifiedWindowFrame, WindowSpecDefinition} class ExpressionSQLBuilderSuite extends SQLBuilderTest { test("literal") { @@ -79,4 +78,36 @@ class ExpressionSQLBuilderSuite extends SQLBuilderTest { checkSQL(-'a.int, "(-`a`)") checkSQL(-('a.int + 'b.int), "(-(`a` + `b`))") } + + test("window specification") { + val frame = SpecifiedWindowFrame.defaultWindowFrame( + hasOrderSpecification = true, + acceptWindowFrame = true + ) + + checkSQL( + WindowSpecDefinition('a.int :: Nil, Nil, frame), + s"(PARTITION BY `a` $frame)" + ) + + checkSQL( + WindowSpecDefinition('a.int :: 'b.string :: Nil, Nil, frame), + s"(PARTITION BY `a`, `b` $frame)" + ) + + checkSQL( + WindowSpecDefinition(Nil, 'a.int.asc :: Nil, frame), + s"(ORDER BY `a` ASC $frame)" + ) + + checkSQL( + WindowSpecDefinition(Nil, 'a.int.asc :: 'b.string.desc :: Nil, frame), + s"(ORDER BY `a` ASC, `b` DESC $frame)" + ) + + checkSQL( + WindowSpecDefinition('a.int :: 'b.string :: Nil, 'c.int.asc :: 'd.string.desc :: Nil, frame), + s"(PARTITION BY `a`, `b` ORDER BY `c` ASC, `d` DESC $frame)" + ) + } } From d9bd066b9f37cfd18037b9a600371d0342703c0f Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 25 Jul 2016 11:25:41 -0700 Subject: [PATCH 1036/1470] [SPARKR][DOCS] fix broken url in doc ## What changes were proposed in this pull request? Fix broken url, also, sparkR.session.stop doc page should have it in the header, instead of saying "sparkR.stop" ![image](https://cloud.githubusercontent.com/assets/8969467/17080129/26d41308-50d9-11e6-8967-79d6c920313f.png) Data type section is in the middle of a list of gapply/gapplyCollect subsections: ![image](https://cloud.githubusercontent.com/assets/8969467/17080122/f992d00a-50d8-11e6-8f2c-fd5786213920.png) ## How was this patch tested? manual test Author: Felix Cheung Closes #14329 from felixcheung/rdoclinkfix. (cherry picked from commit b73defdd790cb823a4f9958ca89cec06fd198051) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 2 +- R/pkg/R/sparkR.R | 16 +++---- docs/sparkr.md | 107 ++++++++++++++++++++++---------------------- 3 files changed, 62 insertions(+), 63 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 92c10f1be1336..aa211b326a167 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -35,7 +35,7 @@ setOldClass("structType") #' @slot env An R environment that stores bookkeeping states of the SparkDataFrame #' @slot sdf A Java object reference to the backing Scala DataFrame #' @seealso \link{createDataFrame}, \link{read.json}, \link{table} -#' @seealso \url{https://spark.apache.org/docs/latest/sparkr.html#sparkdataframe} +#' @seealso \url{https://spark.apache.org/docs/latest/sparkr.html#sparkr-dataframes} #' @export #' @examples #'\dontrun{ diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index ff5297ffd51cb..524f7c4a26b67 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -28,14 +28,6 @@ connExists <- function(env) { }) } -#' @rdname sparkR.session.stop -#' @name sparkR.stop -#' @export -#' @note sparkR.stop since 1.4.0 -sparkR.stop <- function() { - sparkR.session.stop() -} - #' Stop the Spark Session and Spark Context #' #' Stop the Spark Session and Spark Context. @@ -90,6 +82,14 @@ sparkR.session.stop <- function() { clearJobjs() } +#' @rdname sparkR.session.stop +#' @name sparkR.stop +#' @export +#' @note sparkR.stop since 1.4.0 +sparkR.stop <- function() { + sparkR.session.stop() +} + #' (Deprecated) Initialize a new Spark Context #' #' This function initializes a new SparkContext. diff --git a/docs/sparkr.md b/docs/sparkr.md index dfa5278ef8491..4bbc362c52086 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -322,8 +322,59 @@ head(ldf, 3) Apply a function to each group of a `SparkDataFrame`. The function is to be applied to each group of the `SparkDataFrame` and should have only two parameters: grouping key and R `data.frame` corresponding to that key. The groups are chosen from `SparkDataFrame`s column(s). The output of function should be a `data.frame`. Schema specifies the row format of the resulting -`SparkDataFrame`. It must represent R function's output schema on the basis of Spark data types. The column names of the returned `data.frame` are set by user. Below is the data type mapping between R -and Spark. +`SparkDataFrame`. It must represent R function's output schema on the basis of Spark [data types](#data-type-mapping-between-r-and-spark). The column names of the returned `data.frame` are set by user. + +
    +{% highlight r %} + +# Determine six waiting times with the largest eruption time in minutes. +schema <- structType(structField("waiting", "double"), structField("max_eruption", "double")) +result <- gapply( + df, + "waiting", + function(key, x) { + y <- data.frame(key, max(x$eruptions)) + }, + schema) +head(collect(arrange(result, "max_eruption", decreasing = TRUE))) + +## waiting max_eruption +##1 64 5.100 +##2 69 5.067 +##3 71 5.033 +##4 87 5.000 +##5 63 4.933 +##6 89 4.900 +{% endhighlight %} +
    + +##### gapplyCollect +Like `gapply`, applies a function to each partition of a `SparkDataFrame` and collect the result back to R data.frame. The output of the function should be a `data.frame`. But, the schema is not required to be passed. Note that `gapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +
    +{% highlight r %} + +# Determine six waiting times with the largest eruption time in minutes. +result <- gapplyCollect( + df, + "waiting", + function(key, x) { + y <- data.frame(key, max(x$eruptions)) + colnames(y) <- c("waiting", "max_eruption") + y + }) +head(result[order(result$max_eruption, decreasing = TRUE), ]) + +## waiting max_eruption +##1 64 5.100 +##2 69 5.067 +##3 71 5.033 +##4 87 5.000 +##5 63 4.933 +##6 89 4.900 + +{% endhighlight %} +
    #### Data type mapping between R and Spark @@ -394,58 +445,6 @@ and Spark.
    -
    -{% highlight r %} - -# Determine six waiting times with the largest eruption time in minutes. -schema <- structType(structField("waiting", "double"), structField("max_eruption", "double")) -result <- gapply( - df, - "waiting", - function(key, x) { - y <- data.frame(key, max(x$eruptions)) - }, - schema) -head(collect(arrange(result, "max_eruption", decreasing = TRUE))) - -## waiting max_eruption -##1 64 5.100 -##2 69 5.067 -##3 71 5.033 -##4 87 5.000 -##5 63 4.933 -##6 89 4.900 -{% endhighlight %} -
    - -##### gapplyCollect -Like `gapply`, applies a function to each partition of a `SparkDataFrame` and collect the result back to R data.frame. The output of the function should be a `data.frame`. But, the schema is not required to be passed. Note that `gapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. - -
    -{% highlight r %} - -# Determine six waiting times with the largest eruption time in minutes. -result <- gapplyCollect( - df, - "waiting", - function(key, x) { - y <- data.frame(key, max(x$eruptions)) - colnames(y) <- c("waiting", "max_eruption") - y - }) -head(result[order(result$max_eruption, decreasing = TRUE), ]) - -## waiting max_eruption -##1 64 5.100 -##2 69 5.067 -##3 71 5.033 -##4 87 5.000 -##5 63 4.933 -##6 89 4.900 - -{% endhighlight %} -
    - #### Run local R functions distributed using `spark.lapply` ##### spark.lapply From f0d05f669b4e7be017d8d0cfba33c3a61a1eef8f Mon Sep 17 00:00:00 2001 From: Shuai Lin Date: Mon, 25 Jul 2016 20:26:55 +0100 Subject: [PATCH 1037/1470] [SPARK-16485][DOC][ML] Fixed several inline formatting in ml features doc ## What changes were proposed in this pull request? Fixed several inline formatting in ml features doc. Before: screen shot 2016-07-14 at 12 24 57 pm After: screen shot 2016-07-14 at 12 25 48 pm ## How was this patch tested? Genetate the docs locally by `SKIP_API=1 jekyll build` and view it in the browser. Author: Shuai Lin Closes #14194 from lins05/fix-docs-formatting. (cherry picked from commit 3b6e1d094e153599e158331b10d33d74a667be5a) Signed-off-by: Sean Owen --- docs/ml-features.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index e7d7ddfe28a87..6020114845486 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -216,7 +216,7 @@ for more details on the API. [RegexTokenizer](api/scala/index.html#org.apache.spark.ml.feature.RegexTokenizer) allows more advanced tokenization based on regular expression (regex) matching. - By default, the parameter "pattern" (regex, default: \\s+) is used as delimiters to split the input text. + By default, the parameter "pattern" (regex, default: `"\\s+"`) is used as delimiters to split the input text. Alternatively, users can set parameter "gaps" to false indicating the regex "pattern" denotes "tokens" rather than splitting gaps, and find all matching occurrences as the tokenization result. @@ -815,7 +815,7 @@ The rescaled value for a feature E is calculated as, `\begin{equation} Rescaled(e_i) = \frac{e_i - E_{min}}{E_{max} - E_{min}} * (max - min) + min \end{equation}` -For the case `E_{max} == E_{min}`, `Rescaled(e_i) = 0.5 * (max + min)` +For the case `$E_{max} == E_{min}$`, `$Rescaled(e_i) = 0.5 * (max + min)$` Note that since zero values will probably be transformed to non-zero values, output of the transformer will be `DenseVector` even for sparse input. From 1b4f7cf135eebc46f07649509a027b6d422dcfdf Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Mon, 25 Jul 2016 15:08:58 -0700 Subject: [PATCH 1038/1470] [SQL][DOC] Fix a default name for parquet compression ## What changes were proposed in this pull request? This pr is to fix a wrong description for parquet default compression. Author: Takeshi YAMAMURO Closes #14351 from maropu/FixParquetDoc. (cherry picked from commit cda4603de340d533c49feac1b244ddfd291f9bcf) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e92596bbc5b53..33b170e50a00c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -749,7 +749,7 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession spark.sql.parquet.compression.codec - gzip + snappy Sets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo. From 41e72f65929c345aa21ebd4e00dadfbfb5acfdf3 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 25 Jul 2016 16:08:29 -0700 Subject: [PATCH 1039/1470] [SPARK-16715][TESTS] Fix a potential ExprId conflict for SubexpressionEliminationSuite."Semantic equals and hash" ## What changes were proposed in this pull request? SubexpressionEliminationSuite."Semantic equals and hash" assumes the default AttributeReference's exprId wont' be "ExprId(1)". However, that depends on when this test runs. It may happen to use "ExprId(1)". This PR detects the conflict and makes sure we create a different ExprId when the conflict happens. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #14350 from zsxwing/SPARK-16715. (cherry picked from commit 12f490b5c85cdee26d47eb70ad1a1edd00504f21) Signed-off-by: Shixiong Zhu --- .../expressions/SubexpressionEliminationSuite.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala index 90e97d718a9fc..1e39b24fe8770 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala @@ -21,8 +21,12 @@ import org.apache.spark.sql.types.IntegerType class SubexpressionEliminationSuite extends SparkFunSuite { test("Semantic equals and hash") { - val id = ExprId(1) val a: AttributeReference = AttributeReference("name", IntegerType)() + val id = { + // Make sure we use a "ExprId" different from "a.exprId" + val _id = ExprId(1) + if (a.exprId == _id) ExprId(2) else _id + } val b1 = a.withName("name2").withExprId(id) val b2 = a.withExprId(id) val b3 = a.withQualifier(Some("qualifierName")) From b17fe4e412d27a4f3e8ad86ac5d8c2c108654eb3 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 25 Jul 2016 16:09:22 -0700 Subject: [PATCH 1040/1470] [SPARK-14131][STREAMING] SQL Improved fix for avoiding potential deadlocks in HDFSMetadataLog ## What changes were proposed in this pull request? Current fix for deadlock disables interrupts in the StreamExecution which getting offsets for all sources, and when writing to any metadata log, to avoid potential deadlocks in HDFSMetadataLog(see JIRA for more details). However, disabling interrupts can have unintended consequences in other sources. So I am making the fix more narrow, by disabling interrupt it only in the HDFSMetadataLog. This is a narrower fix for something risky like disabling interrupt. ## How was this patch tested? Existing tests. Author: Tathagata Das Closes #14292 from tdas/SPARK-14131. (cherry picked from commit c979c8bba02bc89cb9ad81b212f085a8a5490a07) Signed-off-by: Tathagata Das --- .../execution/streaming/HDFSMetadataLog.scala | 31 +++++++++---- .../execution/streaming/StreamExecution.scala | 28 ++++-------- .../streaming/FileStreamSinkLogSuite.scala | 4 +- .../streaming/HDFSMetadataLogSuite.scala | 10 +++-- .../apache/spark/sql/test/SQLTestUtils.scala | 43 ++++++++++++++++++- 5 files changed, 80 insertions(+), 36 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 069e41b6cedd6..698f07b0a187f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -32,6 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.JavaSerializer import org.apache.spark.sql.SparkSession +import org.apache.spark.util.UninterruptibleThread /** @@ -91,18 +92,30 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) serializer.deserialize[T](ByteBuffer.wrap(bytes)) } + /** + * Store the metadata for the specified batchId and return `true` if successful. If the batchId's + * metadata has already been stored, this method will return `false`. + * + * Note that this method must be called on a [[org.apache.spark.util.UninterruptibleThread]] + * so that interrupts can be disabled while writing the batch file. This is because there is a + * potential dead-lock in Hadoop "Shell.runCommand" before 2.5.0 (HADOOP-10622). If the thread + * running "Shell.runCommand" is interrupted, then the thread can get deadlocked. In our + * case, `writeBatch` creates a file using HDFS API and calls "Shell.runCommand" to set the + * file permissions, and can get deadlocked if the stream execution thread is stopped by + * interrupt. Hence, we make sure that this method is called on [[UninterruptibleThread]] which + * allows us to disable interrupts here. Also see SPARK-14131. + */ override def add(batchId: Long, metadata: T): Boolean = { get(batchId).map(_ => false).getOrElse { - // Only write metadata when the batch has not yet been written. - try { - writeBatch(batchId, serialize(metadata)) - true - } catch { - case e: IOException if "java.lang.InterruptedException" == e.getMessage => - // create may convert InterruptedException to IOException. Let's convert it back to - // InterruptedException so that this failure won't crash StreamExecution - throw new InterruptedException("Creating file is interrupted") + // Only write metadata when the batch has not yet been written + Thread.currentThread match { + case ut: UninterruptibleThread => + ut.runUninterruptibly { writeBatch(batchId, serialize(metadata)) } + case _ => + throw new IllegalStateException( + "HDFSMetadataLog.add() must be executed on a o.a.spark.util.UninterruptibleThread") } + true } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index c90dcc5680743..af2229a46bebb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -110,7 +110,11 @@ class StreamExecution( /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() - /** The thread that runs the micro-batches of this stream. */ + /** + * The thread that runs the micro-batches of this stream. Note that this thread must be + * [[org.apache.spark.util.UninterruptibleThread]] to avoid potential deadlocks in using + * [[HDFSMetadataLog]]. See SPARK-14131 for more details. + */ private[sql] val microBatchThread = new UninterruptibleThread(s"stream execution thread for $name") { override def run(): Unit = { @@ -269,19 +273,11 @@ class StreamExecution( * batchId counter is incremented and a new log entry is written with the newest offsets. */ private def constructNextBatch(): Unit = { - // There is a potential dead-lock in Hadoop "Shell.runCommand" before 2.5.0 (HADOOP-10622). - // If we interrupt some thread running Shell.runCommand, we may hit this issue. - // As "FileStreamSource.getOffset" will create a file using HDFS API and call "Shell.runCommand" - // to set the file permission, we should not interrupt "microBatchThread" when running this - // method. See SPARK-14131. - // // Check to see what new data is available. val hasNewData = { awaitBatchLock.lock() try { - val newData = microBatchThread.runUninterruptibly { - uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - } + val newData = uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) availableOffsets ++= newData if (dataAvailable) { @@ -295,16 +291,8 @@ class StreamExecution( } } if (hasNewData) { - // There is a potential dead-lock in Hadoop "Shell.runCommand" before 2.5.0 (HADOOP-10622). - // If we interrupt some thread running Shell.runCommand, we may hit this issue. - // As "offsetLog.add" will create a file using HDFS API and call "Shell.runCommand" to set - // the file permission, we should not interrupt "microBatchThread" when running this method. - // See SPARK-14131. - microBatchThread.runUninterruptibly { - assert( - offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), - s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") - } + assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), + s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") logInfo(s"Committed offsets for batch $currentBatchId.") } else { awaitBatchLock.lock() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala index a7b2cfe7d0a49..39fd1f0cd37bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -190,7 +190,7 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { } } - test("compact") { + testWithUninterruptibleThread("compact") { withSQLConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3") { withFileStreamSinkLog { sinkLog => for (batchId <- 0 to 10) { @@ -210,7 +210,7 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { } } - test("delete expired file") { + testWithUninterruptibleThread("delete expired file") { // Set FILE_SINK_LOG_CLEANUP_DELAY to 0 so that we can detect the deleting behaviour // deterministically withSQLConf( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index ef2b479a5636f..ab5a2d253b94a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.sql.execution.streaming.FakeFileSystem._ import org.apache.spark.sql.execution.streaming.HDFSMetadataLog.{FileContextManager, FileManager, FileSystemManager} import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.UninterruptibleThread class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { @@ -56,7 +57,7 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { } } - test("HDFSMetadataLog: basic") { + testWithUninterruptibleThread("HDFSMetadataLog: basic") { withTempDir { temp => val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir val metadataLog = new HDFSMetadataLog[String](spark, dir.getAbsolutePath) @@ -81,7 +82,8 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { } } - testQuietly("HDFSMetadataLog: fallback from FileContext to FileSystem") { + testWithUninterruptibleThread( + "HDFSMetadataLog: fallback from FileContext to FileSystem", quietly = true) { spark.conf.set( s"fs.$scheme.impl", classOf[FakeFileSystem].getName) @@ -101,7 +103,7 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { } } - test("HDFSMetadataLog: restart") { + testWithUninterruptibleThread("HDFSMetadataLog: restart") { withTempDir { temp => val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) assert(metadataLog.add(0, "batch0")) @@ -124,7 +126,7 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { val waiter = new Waiter val maxBatchId = 100 for (id <- 0 until 10) { - new Thread() { + new UninterruptibleThread(s"HDFSMetadataLog: metadata directory collision - thread $id") { override def run(): Unit = waiter { val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 5286ee5bc23d3..d4d8e3e4e83d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -22,6 +22,7 @@ import java.util.UUID import scala.language.implicitConversions import scala.util.Try +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.scalatest.BeforeAndAfterAll @@ -34,7 +35,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.FilterExec -import org.apache.spark.util.Utils +import org.apache.spark.util.{UninterruptibleThread, Utils} /** * Helper trait that should be extended by all SQL test suites. @@ -247,6 +248,46 @@ private[sql] trait SQLTestUtils } } } + + /** Run a test on a separate [[UninterruptibleThread]]. */ + protected def testWithUninterruptibleThread(name: String, quietly: Boolean = false) + (body: => Unit): Unit = { + val timeoutMillis = 10000 + @transient var ex: Throwable = null + + def runOnThread(): Unit = { + val thread = new UninterruptibleThread(s"Testing thread for test $name") { + override def run(): Unit = { + try { + body + } catch { + case NonFatal(e) => + ex = e + } + } + } + thread.setDaemon(true) + thread.start() + thread.join(timeoutMillis) + if (thread.isAlive) { + thread.interrupt() + // If this interrupt does not work, then this thread is most likely running something that + // is not interruptible. There is not much point to wait for the thread to termniate, and + // we rather let the JVM terminate the thread on exit. + fail( + s"Test '$name' running on o.a.s.util.UninterruptibleThread timed out after" + + s" $timeoutMillis ms") + } else if (ex != null) { + throw ex + } + } + + if (quietly) { + testQuietly(name) { runOnThread() } + } else { + test(name) { runOnThread() } + } + } } private[sql] object SQLTestUtils { From 9d581dc61951eccf0f06868e0d3f10134f433e82 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 25 Jul 2016 18:26:29 -0700 Subject: [PATCH 1041/1470] [SPARK-16722][TESTS] Fix a StreamingContext leak in StreamingContextSuite when eventually fails ## What changes were proposed in this pull request? This PR moves `ssc.stop()` into `finally` for `StreamingContextSuite.createValidCheckpoint` to avoid leaking a StreamingContext since leaking a StreamingContext will fail a lot of tests and make us hard to find the real failure one. ## How was this patch tested? Jenkins unit tests Author: Shixiong Zhu Closes #14354 from zsxwing/ssc-leak. (cherry picked from commit e164a04b2ba3503e5c14cd1cd4beb40e0b79925a) Signed-off-by: Tathagata Das --- .../apache/spark/streaming/StreamingContextSuite.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 806e181f61980..f1482e5c06cdc 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -819,10 +819,13 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo ssc.checkpoint(checkpointDirectory) ssc.textFileStream(testDirectory).foreachRDD { rdd => rdd.count() } ssc.start() - eventually(timeout(10000 millis)) { - assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) + try { + eventually(timeout(30000 millis)) { + assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) + } + } finally { + ssc.stop() } - ssc.stop() checkpointDirectory } From 3d35474872d3b117abc3fc7debcb1eb6409769d6 Mon Sep 17 00:00:00 2001 From: Nicholas Brown Date: Mon, 25 Jul 2016 19:18:27 -0700 Subject: [PATCH 1042/1470] Fix description of spark.speculation.quantile ## What changes were proposed in this pull request? Minor doc fix regarding the spark.speculation.quantile configuration parameter. It incorrectly states it should be a percentage, when it should be a fraction. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) I tried building the documentation but got some unidoc errors. I also got them when building off origin/master, so I don't think I caused that problem. I did run the web app and saw the changes reflected as expected. Author: Nicholas Brown Closes #14352 from nwbvt/master. (cherry picked from commit ba0aade6d517364363e07ed09278c2b44110c33b) Signed-off-by: Reynold Xin --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 86a9bd97d3342..bf10b2481951a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1174,7 +1174,7 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.quantile 0.75 - Percentage of tasks which must be complete before speculation is enabled for a particular stage. + Fraction of tasks which must be complete before speculation is enabled for a particular stage. From aeb6d5c053d4e848df0e7842a3994154df464647 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 25 Jul 2016 19:52:17 -0700 Subject: [PATCH 1043/1470] [SPARK-16672][SQL] SQLBuilder should not raise exceptions on EXISTS queries ## What changes were proposed in this pull request? Currently, `SQLBuilder` raises `empty.reduceLeft` exceptions on *unoptimized* `EXISTS` queries. We had better prevent this. ```scala scala> sql("CREATE TABLE t1(a int)") scala> val df = sql("select * from t1 b where exists (select * from t1 a)") scala> new org.apache.spark.sql.catalyst.SQLBuilder(df).toSQL java.lang.UnsupportedOperationException: empty.reduceLeft ``` ## How was this patch tested? Pass the Jenkins tests with a new test suite. Author: Dongjoon Hyun Closes #14307 from dongjoon-hyun/SPARK-16672. (cherry picked from commit 8a8d26f1e27db5c2228307b1c3609b4713b9d0db) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/SQLBuilder.scala | 9 +++++++-- .../src/test/resources/sqlgen/predicate_subquery.sql | 4 ++++ .../spark/sql/catalyst/LogicalPlanToSQLSuite.scala | 10 ++++++++++ 3 files changed, 21 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/resources/sqlgen/predicate_subquery.sql diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index a8cc72f2e7b3a..9a02e3c8f31f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -512,8 +512,13 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { ScalarSubquery(rewrite, Seq.empty, exprId) case PredicateSubquery(query, conditions, false, exprId) => - val plan = Project(Seq(Alias(Literal(1), "1")()), - Filter(conditions.reduce(And), addSubqueryIfNeeded(query))) + val subquery = addSubqueryIfNeeded(query) + val plan = if (conditions.isEmpty) { + subquery + } else { + Project(Seq(Alias(Literal(1), "1")()), + Filter(conditions.reduce(And), subquery)) + } Exists(plan, exprId) case PredicateSubquery(query, conditions, true, exprId) => diff --git a/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql b/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql new file mode 100644 index 0000000000000..2e06b4fe77ee6 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * from t1 b where exists (select * from t1 a) +-------------------------------------------------------------------------------- +SELECT `gen_attr` AS `a` FROM (SELECT `gen_attr` FROM (SELECT `a` AS `gen_attr` FROM `default`.`t1`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr` AS `a` FROM ((SELECT `gen_attr` FROM (SELECT `a` AS `gen_attr` FROM `default`.`t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_1)) AS b diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 1f5078dc9c7d4..ebece38773c23 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -25,6 +25,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils /** @@ -927,6 +928,15 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } } + test("predicate subquery") { + withTable("t1") { + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + sql("CREATE TABLE t1(a int)") + checkSQL("select * from t1 b where exists (select * from t1 a)", "predicate_subquery") + } + } + } + test("SPARK-14933 - select orc table") { withTable("orc_t") { sql("create table orc_t stored as orc as select 1 as c1, 'abc' as c2") From 4b38a6a534d93b1eab3b19f62a2f78474be1d8bc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 25 Jul 2016 20:41:24 -0700 Subject: [PATCH 1044/1470] [SPARK-16724] Expose DefinedByConstructorParams We don't generally make things in catalyst/execution private. Instead they are just undocumented due to their lack of stability guarantees. Author: Michael Armbrust Closes #14356 from marmbrus/patch-1. (cherry picked from commit f99e34e8e58c97ff30c6e054875533350d99fe5b) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/catalyst/ScalaReflection.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 78c145d4fd936..8affb033d8287 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 @@ -30,7 +30,7 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} * for classes whose fields are entirely defined by constructor params but should not be * case classes. */ -private[sql] trait DefinedByConstructorParams +trait DefinedByConstructorParams /** From 4391d4a3c60d59df625cbfdb918aa67c51ebcbc1 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 25 Jul 2016 20:58:07 -0700 Subject: [PATCH 1045/1470] [SPARK-16633][SPARK-16642][SPARK-16721][SQL] Fixes three issues related to lead and lag functions ## What changes were proposed in this pull request? This PR contains three changes. First, this PR changes the behavior of lead/lag back to Spark 1.6's behavior, which is described as below: 1. lead/lag respect null input values, which means that if the offset row exists and the input value is null, the result will be null instead of the default value. 2. If the offset row does not exist, the default value will be used. 3. OffsetWindowFunction's nullable setting also considers the nullability of its input (because of the first change). Second, this PR fixes the evaluation of lead/lag when the input expression is a literal. This fix is a result of the first change. In current master, if a literal is used as the input expression of a lead or lag function, the result will be this literal even if the offset row does not exist. Third, this PR makes ResolveWindowFrame not fire if a window function is not resolved. ## How was this patch tested? New tests in SQLWindowFunctionSuite Author: Yin Huai Closes #14284 from yhuai/lead-lag. (cherry picked from commit 815f3eece5f095919a329af8cbd762b9ed71c7a8) Signed-off-by: Yin Huai --- .../sql/catalyst/analysis/Analyzer.scala | 3 +- .../expressions/windowExpressions.scala | 45 +++++++------ .../spark/sql/execution/WindowExec.scala | 34 ++++++++-- .../execution/SQLWindowFunctionSuite.scala | 66 +++++++++++++++---- 4 files changed, 108 insertions(+), 40 deletions(-) rename sql/{hive/src/test/scala/org/apache/spark/sql/hive => core/src/test/scala/org/apache/spark/sql}/execution/SQLWindowFunctionSuite.scala (85%) 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 d1d2c59caed9a..61162ccdba810 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 @@ -1787,7 +1787,8 @@ class Analyzer( s @ WindowSpecDefinition(_, o, UnspecifiedFrame)) if wf.frame != UnspecifiedFrame => WindowExpression(wf, s.copy(frameSpecification = wf.frame)) - case we @ WindowExpression(e, s @ WindowSpecDefinition(_, o, UnspecifiedFrame)) => + case we @ WindowExpression(e, s @ WindowSpecDefinition(_, o, UnspecifiedFrame)) + if e.resolved => val frame = SpecifiedWindowFrame.defaultWindowFrame(o.nonEmpty, acceptWindowFrame = true) we.copy(windowSpec = s.copy(frameSpecification = frame)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index e35192ca2d26c..6806591f68bc1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -321,8 +321,7 @@ abstract class OffsetWindowFunction val input: Expression /** - * Default result value for the function when the input expression returns NULL. The default will - * evaluated against the current row instead of the offset row. + * Default result value for the function when the 'offset'th row does not exist. */ val default: Expression @@ -348,7 +347,7 @@ abstract class OffsetWindowFunction */ override def foldable: Boolean = false - override def nullable: Boolean = default == null || default.nullable + override def nullable: Boolean = default == null || default.nullable || input.nullable override lazy val frame = { // This will be triggered by the Analyzer. @@ -373,20 +372,22 @@ abstract class OffsetWindowFunction } /** - * The Lead function returns the value of 'x' at 'offset' rows after the current row in the window. - * Offsets start at 0, which is the current row. The offset must be constant integer value. The - * default offset is 1. When the value of 'x' is null at the offset, or when the offset is larger - * than the window, the default expression is evaluated. - * - * This documentation has been based upon similar documentation for the Hive and Presto projects. + * The Lead function returns the value of 'x' at the 'offset'th row after the current row in + * the window. Offsets start at 0, which is the current row. The offset must be constant + * integer value. The default offset is 1. When the value of 'x' is null at the 'offset'th row, + * null is returned. If there is no such offset row, the default expression is evaluated. * * @param input expression to evaluate 'offset' rows after the current row. * @param offset rows to jump ahead in the partition. - * @param default to use when the input value is null or when the offset is larger than the window. + * @param default to use when the offset is larger than the window. The default value is null. */ @ExpressionDescription(usage = - """_FUNC_(input, offset, default) - LEAD returns the value of 'x' at 'offset' rows - after the current row in the window""") + """_FUNC_(input, offset, default) - LEAD returns the value of 'x' at the 'offset'th row + after the current row in the window. + The default value of 'offset' is 1 and the default value of 'default' is null. + If the value of 'x' at the 'offset'th row is null, null is returned. + If there is no such offset row (e.g. when the offset is 1, the last row of the window + does not have any subsequent row), 'default' is returned.""") case class Lead(input: Expression, offset: Expression, default: Expression) extends OffsetWindowFunction { @@ -400,20 +401,22 @@ case class Lead(input: Expression, offset: Expression, default: Expression) } /** - * The Lag function returns the value of 'x' at 'offset' rows before the current row in the window. - * Offsets start at 0, which is the current row. The offset must be constant integer value. The - * default offset is 1. When the value of 'x' is null at the offset, or when the offset is smaller - * than the window, the default expression is evaluated. - * - * This documentation has been based upon similar documentation for the Hive and Presto projects. + * The Lag function returns the value of 'x' at the 'offset'th row before the current row in + * the window. Offsets start at 0, which is the current row. The offset must be constant + * integer value. The default offset is 1. When the value of 'x' is null at the 'offset'th row, + * null is returned. If there is no such offset row, the default expression is evaluated. * * @param input expression to evaluate 'offset' rows before the current row. * @param offset rows to jump back in the partition. - * @param default to use when the input value is null or when the offset is smaller than the window. + * @param default to use when the offset row does not exist. */ @ExpressionDescription(usage = - """_FUNC_(input, offset, default) - LAG returns the value of 'x' at 'offset' rows - before the current row in the window""") + """_FUNC_(input, offset, default) - LAG returns the value of 'x' at the 'offset'th row + before the current row in the window. + The default value of 'offset' is 1 and the default value of 'default' is null. + If the value of 'x' at the 'offset'th row is null, null is returned. + If there is no such offset row (e.g. when the offset is 1, the first row of the window + does not have any previous row), 'default' is returned.""") case class Lag(input: Expression, offset: Expression, default: Expression) extends OffsetWindowFunction { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala index e01094a7c8e3b..3927a5011ba4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala @@ -582,25 +582,43 @@ private[execution] final class OffsetWindowFunctionFrame( /** Row used to combine the offset and the current row. */ private[this] val join = new JoinedRow - /** Create the projection. */ + /** + * Create the projection used when the offset row exists. + * Please note that this project always respect null input values (like PostgreSQL). + */ private[this] val projection = { // Collect the expressions and bind them. val inputAttrs = inputSchema.map(_.withNullability(true)) - val numInputAttributes = inputAttrs.size val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { case e: OffsetWindowFunction => val input = BindReferences.bindReference(e.input, inputAttrs) + input + case e => + BindReferences.bindReference(e, inputAttrs) + } + + // Create the projection. + newMutableProjection(boundExpressions, Nil).target(target) + } + + /** Create the projection used when the offset row DOES NOT exists. */ + private[this] val fillDefaultValue = { + // Collect the expressions and bind them. + val inputAttrs = inputSchema.map(_.withNullability(true)) + val numInputAttributes = inputAttrs.size + val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { + case e: OffsetWindowFunction => if (e.default == null || e.default.foldable && e.default.eval() == null) { - // Without default value. - input + // The default value is null. + Literal.create(null, e.dataType) } else { - // With default value. + // The default value is an expression. val default = BindReferences.bindReference(e.default, inputAttrs).transform { // Shift the input reference to its default version. case BoundReference(o, dataType, nullable) => BoundReference(o + numInputAttributes, dataType, nullable) } - org.apache.spark.sql.catalyst.expressions.Coalesce(input :: default :: Nil) + default } case e => BindReferences.bindReference(e, inputAttrs) @@ -625,10 +643,12 @@ private[execution] final class OffsetWindowFunctionFrame( if (inputIndex >= 0 && inputIndex < input.size) { val r = input.next() join(r, current) + projection(join) } else { join(emptyRow, current) + // Use default values since the offset row does not exist. + fillDefaultValue(join) } - projection(join) inputIndex += 1 } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala similarity index 85% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala index 77e97dff8c221..d3cfa953a3123 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLWindowFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.execution +package org.apache.spark.sql.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row} -import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.test.SQLTestUtils - +import org.apache.spark.sql.test.SharedSQLContext case class WindowData(month: Int, area: String, product: Int) @@ -28,8 +26,9 @@ case class WindowData(month: Int, area: String, product: Int) /** * Test suite for SQL window functions. */ -class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { - import spark.implicits._ +class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext { + + import testImplicits._ test("window function: udaf with aggregate expression") { val data = Seq( @@ -357,14 +356,59 @@ class SQLWindowFunctionSuite extends QueryTest with SQLTestUtils with TestHiveSi } test("SPARK-7595: Window will cause resolve failed with self join") { - sql("SELECT * FROM src") // Force loading of src table. - checkAnswer(sql( """ |with - | v1 as (select key, count(value) over (partition by key) cnt_val from src), + | v0 as (select 0 as key, 1 as value), + | v1 as (select key, count(value) over (partition by key) cnt_val from v0), | v2 as (select v1.key, v1_lag.cnt_val from v1, v1 v1_lag where v1.key = v1_lag.key) - | select * from v2 order by key limit 1 - """.stripMargin), Row(0, 3)) + | select key, cnt_val from v2 order by key limit 1 + """.stripMargin), Row(0, 1)) + } + + test("SPARK-16633: lead/lag should return the default value if the offset row does not exist") { + checkAnswer(sql( + """ + |SELECT + | lag(123, 100, 321) OVER (ORDER BY id) as lag, + | lead(123, 100, 321) OVER (ORDER BY id) as lead + |FROM (SELECT 1 as id) tmp + """.stripMargin), + Row(321, 321)) + + checkAnswer(sql( + """ + |SELECT + | lag(123, 100, a) OVER (ORDER BY id) as lag, + | lead(123, 100, a) OVER (ORDER BY id) as lead + |FROM (SELECT 1 as id, 2 as a) tmp + """.stripMargin), + Row(2, 2)) + } + + test("lead/lag should respect null values") { + checkAnswer(sql( + """ + |SELECT + | b, + | lag(a, 1, 321) OVER (ORDER BY b) as lag, + | lead(a, 1, 321) OVER (ORDER BY b) as lead + |FROM (SELECT cast(null as int) as a, 1 as b + | UNION ALL + | select cast(null as int) as id, 2 as b) tmp + """.stripMargin), + Row(1, 321, null) :: Row(2, null, 321) :: Nil) + + checkAnswer(sql( + """ + |SELECT + | b, + | lag(a, 1, c) OVER (ORDER BY b) as lag, + | lead(a, 1, c) OVER (ORDER BY b) as lead + |FROM (SELECT cast(null as int) as a, 1 as b, 3 as c + | UNION ALL + | select cast(null as int) as id, 2 as b, 4 as c) tmp + """.stripMargin), + Row(1, 3, null) :: Row(2, null, 4) :: Nil) } } From 44234b1c4266ac7be56892817d043fe6d9ea62f7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 26 Jul 2016 00:41:46 -0700 Subject: [PATCH 1046/1470] [TEST][STREAMING] Fix flaky Kafka rate controlling test ## What changes were proposed in this pull request? The current test is incorrect, because - The expected number of messages does not take into account that the topic has 2 partitions, and rate is set per partition. - Also in some cases, the test ran out of data in Kafka while waiting for the right amount of data per batch. The PR - Reduces the number of partitions to 1 - Adds more data to Kafka - Runs with 0.5 second so that batches are created slowly ## How was this patch tested? Ran many times locally, going to run it many times in Jenkins (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Tathagata Das Closes #14361 from tdas/kafka-rate-test-fix. (cherry picked from commit 03c27435aee4e319abe290771ba96e69469109ac) Signed-off-by: Tathagata Das --- .../streaming/kafka010/DirectKafkaStreamSuite.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index c9e15bcba0a9e..b1d90b8a82d59 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -544,15 +544,14 @@ class DirectKafkaStreamSuite test("using rate controller") { val topic = "backpressure" - val topicPartitions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) - kafkaTestUtils.createTopic(topic, 2) + kafkaTestUtils.createTopic(topic, 1) val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") val executorKafkaParams = new JHashMap[String, Object](kafkaParams) KafkaUtils.fixKafkaParams(executorKafkaParams) - val batchIntervalMilliseconds = 100 + val batchIntervalMilliseconds = 500 val estimator = new ConstantEstimator(100) - val messages = Map("foo" -> 200) + val messages = Map("foo" -> 5000) kafkaTestUtils.sendMessages(topic, messages) val sparkConf = new SparkConf() @@ -596,7 +595,7 @@ class DirectKafkaStreamSuite estimator.updateRate(rate) // Set a new rate. // Expect blocks of data equal to "rate", scaled by the interval length in secs. val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) - eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { + eventually(timeout(5.seconds), interval(10 milliseconds)) { // Assert that rate estimator values are used to determine maxMessagesPerPartition. // Funky "-" in message makes the complete assertion message read better. assert(collectedData.asScala.exists(_.size == expectedSize), From be9965b077cded3d30a2d35342f3440f4708c357 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 27 Jul 2016 13:23:59 +0800 Subject: [PATCH 1047/1470] [SPARK-16621][SQL] Generate stable SQLs in SQLBuilder Currently, the generated SQLs have not-stable IDs for generated attributes. The stable generated SQL will give more benefit for understanding or testing the queries. This PR provides stable SQL generation by the followings. - Provide unique ids for generated subqueries, `gen_subquery_xxx`. - Provide unique and stable ids for generated attributes, `gen_attr_xxx`. **Before** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res0: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4`) AS gen_subquery_0 ``` **After** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res2: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 ``` Pass the existing Jenkins tests. Author: Dongjoon Hyun Closes #14257 from dongjoon-hyun/SPARK-16621. (cherry picked from commit 5b8e848bbfbc0c99a5faf758e40b188b0bbebb7b) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/SQLBuilder.scala | 23 +++++++++++++++---- sql/hive/src/test/resources/sqlgen/agg1.sql | 2 +- sql/hive/src/test/resources/sqlgen/agg2.sql | 2 +- sql/hive/src/test/resources/sqlgen/agg3.sql | 2 +- .../sqlgen/aggregate_functions_and_window.sql | 2 +- sql/hive/src/test/resources/sqlgen/case.sql | 2 +- .../test/resources/sqlgen/case_with_else.sql | 2 +- .../test/resources/sqlgen/case_with_key.sql | 2 +- .../sqlgen/case_with_key_and_else.sql | 2 +- .../src/test/resources/sqlgen/cluster_by.sql | 2 +- .../sqlgen/data_source_json_parquet_t0.sql | 2 +- .../sqlgen/data_source_orc_parquet_t0.sql | 2 +- .../sqlgen/data_source_parquet_parquet_t0.sql | 2 +- .../resources/sqlgen/distinct_aggregation.sql | 2 +- .../test/resources/sqlgen/distribute_by.sql | 2 +- .../sqlgen/distribute_by_with_sort_by.sql | 2 +- sql/hive/src/test/resources/sqlgen/except.sql | 2 +- .../sqlgen/filter_after_subquery.sql | 2 +- .../sqlgen/generate_with_other_1.sql | 2 +- .../sqlgen/generate_with_other_2.sql | 2 +- .../sqlgen/generator_in_lateral_view_1.sql | 2 +- .../sqlgen/generator_in_lateral_view_2.sql | 2 +- .../generator_non_referenced_table_1.sql | 2 +- .../generator_non_referenced_table_2.sql | 2 +- .../resources/sqlgen/generator_non_udtf_1.sql | 2 +- .../resources/sqlgen/generator_non_udtf_2.sql | 2 +- .../sqlgen/generator_referenced_table_1.sql | 2 +- .../sqlgen/generator_referenced_table_2.sql | 2 +- .../generator_with_ambiguous_names_1.sql | 2 +- .../generator_with_ambiguous_names_2.sql | 2 +- .../sqlgen/generator_without_from_1.sql | 2 +- .../sqlgen/generator_without_from_2.sql | 2 +- .../test/resources/sqlgen/grouping_sets_1.sql | 2 +- .../resources/sqlgen/grouping_sets_2_1.sql | 2 +- .../resources/sqlgen/grouping_sets_2_2.sql | 2 +- .../resources/sqlgen/grouping_sets_2_3.sql | 2 +- .../resources/sqlgen/grouping_sets_2_4.sql | 2 +- .../resources/sqlgen/grouping_sets_2_5.sql | 2 +- sql/hive/src/test/resources/sqlgen/in.sql | 2 +- .../src/test/resources/sqlgen/intersect.sql | 2 +- .../test/resources/sqlgen/join_2_tables.sql | 2 +- .../sqlgen/json_tuple_generator_1.sql | 2 +- .../sqlgen/json_tuple_generator_2.sql | 2 +- .../test/resources/sqlgen/multi_distinct.sql | 2 +- .../nested_generator_in_lateral_view_1.sql | 2 +- .../nested_generator_in_lateral_view_2.sql | 2 +- sql/hive/src/test/resources/sqlgen/not_in.sql | 2 +- .../src/test/resources/sqlgen/not_like.sql | 2 +- .../resources/sqlgen/predicate_subquery.sql | 2 +- .../sqlgen/regular_expressions_and_window.sql | 2 +- .../test/resources/sqlgen/rollup_cube_1_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_1_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_2_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_2_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_3_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_3_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_4_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_4_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_5_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_5_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_3.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_4.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_5.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_6.sql | 2 +- .../test/resources/sqlgen/rollup_cube_7_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_7_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_7_3.sql | 2 +- .../test/resources/sqlgen/rollup_cube_8_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_8_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_9_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_9_2.sql | 2 +- .../sqlgen/script_transformation_1.sql | 2 +- .../sqlgen/script_transformation_2.sql | 2 +- .../script_transformation_alias_list.sql | 2 +- ...pt_transformation_alias_list_with_type.sql | 2 +- ...ipt_transformation_row_format_multiple.sql | 2 +- .../script_transformation_row_format_one.sql | 2 +- ...script_transformation_row_format_serde.sql | 2 +- ...ransformation_row_format_without_serde.sql | 2 +- .../test/resources/sqlgen/select_distinct.sql | 2 +- .../resources/sqlgen/select_orc_table.sql | 2 +- .../resources/sqlgen/select_parquet_table.sql | 2 +- .../src/test/resources/sqlgen/self_join.sql | 2 +- .../sqlgen/self_join_with_group_by.sql | 2 +- .../resources/sqlgen/sort_by_after_having.sql | 2 +- .../test/resources/sqlgen/tablesample_1.sql | 2 +- .../test/resources/sqlgen/tablesample_2.sql | 2 +- .../test/resources/sqlgen/tablesample_3.sql | 2 +- .../test/resources/sqlgen/tablesample_4.sql | 2 +- .../test/resources/sqlgen/tablesample_5.sql | 2 +- .../test/resources/sqlgen/tablesample_6.sql | 2 +- .../resources/sqlgen/three_child_union.sql | 2 +- .../test/resources/sqlgen/type_widening.sql | 2 +- .../test/resources/sqlgen/union_distinct.sql | 2 +- .../test/resources/sqlgen/window_basic_1.sql | 2 +- .../test/resources/sqlgen/window_basic_2.sql | 2 +- .../test/resources/sqlgen/window_basic_3.sql | 2 +- .../resources/sqlgen/window_with_join.sql | 2 +- .../window_with_the_same_window_with_agg.sql | 2 +- ...w_with_the_same_window_with_agg_filter.sql | 2 +- ...ith_the_same_window_with_agg_functions.sql | 2 +- ...w_with_the_same_window_with_agg_having.sql | 2 +- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 9 +++----- 105 files changed, 124 insertions(+), 114 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 9a02e3c8f31f9..5d93419f357ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst import java.util.concurrent.atomic.AtomicLong +import scala.collection.mutable.Map import scala.util.control.NonFatal import org.apache.spark.internal.Logging @@ -38,14 +39,23 @@ import org.apache.spark.sql.types.{ByteType, DataType, IntegerType, NullType} * representations (e.g. logical plans that operate on local Scala collections), or are simply not * supported by this builder (yet). */ -class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { +class SQLBuilder private ( + logicalPlan: LogicalPlan, + nextSubqueryId: AtomicLong, + nextGenAttrId: AtomicLong, + exprIdMap: Map[Long, Long]) extends Logging { require(logicalPlan.resolved, "SQLBuilder only supports resolved logical query plans. Current plan:\n" + logicalPlan) + def this(logicalPlan: LogicalPlan) = + this(logicalPlan, new AtomicLong(0), new AtomicLong(0), Map.empty[Long, Long]) + def this(df: Dataset[_]) = this(df.queryExecution.analyzed) - private val nextSubqueryId = new AtomicLong(0) private def newSubqueryName(): String = s"gen_subquery_${nextSubqueryId.getAndIncrement()}" + private def normalizedName(n: NamedExpression): String = synchronized { + "gen_attr_" + exprIdMap.getOrElseUpdate(n.exprId.id, nextGenAttrId.getAndIncrement()) + } def toSQL: String = { val canonicalizedPlan = Canonicalizer.execute(logicalPlan) @@ -70,7 +80,7 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { try { val replaced = finalPlan.transformAllExpressions { case s: SubqueryExpression => - val query = new SQLBuilder(s.query).toSQL + val query = new SQLBuilder(s.query, nextSubqueryId, nextGenAttrId, exprIdMap).toSQL val sql = s match { case _: ListQuery => query case _: Exists => s"EXISTS($query)" @@ -169,6 +179,11 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { qualifiedName + " TABLESAMPLE(" + fraction + " PERCENT)" }.getOrElse(qualifiedName) + case relation: CatalogRelation => + val m = relation.catalogTable + val qualifiedName = s"${quoteIdentifier(m.database)}.${quoteIdentifier(m.identifier.table)}" + qualifiedName + case Sort(orders, _, RepartitionByExpression(partitionExprs, child, _)) if orders.map(_.child) == partitionExprs => build(toSQL(child), "CLUSTER BY", partitionExprs.map(_.sql).mkString(", ")) @@ -376,8 +391,6 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { ) } - private def normalizedName(n: NamedExpression): String = "gen_attr_" + n.exprId.id - object Canonicalizer extends RuleExecutor[LogicalPlan] { override protected def batches: Seq[Batch] = Seq( Batch("Prepare", FixedPoint(100), diff --git a/sql/hive/src/test/resources/sqlgen/agg1.sql b/sql/hive/src/test/resources/sqlgen/agg1.sql index 9953640a6fa1c..05403a9dd8927 100644 --- a/sql/hive/src/test/resources/sqlgen/agg1.sql +++ b/sql/hive/src/test/resources/sqlgen/agg1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_2`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_2` HAVING (`gen_attr_1` > CAST(0 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/agg2.sql b/sql/hive/src/test/resources/sqlgen/agg2.sql index 0caea28b0240a..65d71714fe850 100644 --- a/sql/hive/src/test/resources/sqlgen/agg2.sql +++ b/sql/hive/src/test/resources/sqlgen/agg2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY MAX(key) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` ORDER BY `gen_attr` ASC) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_2`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_2` ORDER BY `gen_attr_1` ASC) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/agg3.sql b/sql/hive/src/test/resources/sqlgen/agg3.sql index 437afa7790773..14b19392cdce3 100644 --- a/sql/hive/src/test/resources/sqlgen/agg3.sql +++ b/sql/hive/src/test/resources/sqlgen/agg3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_3` AS `gen_attr_1`, max(`gen_attr_3`) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_3` ORDER BY `gen_attr_1` ASC, `gen_attr_2` ASC) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql index c94f53b0a9f44..e3e372d5eccdd 100644 --- a/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql +++ b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(c) + COUNT(a) OVER () FROM parquet_t2 GROUP BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(max(c) + count(a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, count(`gen_attr`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `(max(c) + count(a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT (`gen_attr_1` + `gen_attr_2`) AS `gen_attr_0` FROM (SELECT gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, count(`gen_attr_3`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_2` FROM (SELECT max(`gen_attr_5`) AS `gen_attr_1`, `gen_attr_3` FROM (SELECT `a` AS `gen_attr_3`, `b` AS `gen_attr_4`, `c` AS `gen_attr_5`, `d` AS `gen_attr_6` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_3`, `gen_attr_4`) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/case.sql b/sql/hive/src/test/resources/sqlgen/case.sql index f6a8c32cf0121..99630e88cff66 100644 --- a/sql/hive/src/test/resources/sqlgen/case.sql +++ b/sql/hive/src/test/resources/sqlgen/case.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((id % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END` FROM (SELECT CASE WHEN ((`gen_attr` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((`gen_attr` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((id % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END` FROM (SELECT CASE WHEN ((`gen_attr_1` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((`gen_attr_1` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_else.sql b/sql/hive/src/test/resources/sqlgen/case_with_else.sql index 8f1595d4b8cee..aed8f08804807 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_else.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_else.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE WHEN id % 2 > 0 THEN 0 ELSE 1 END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END` FROM (SELECT CASE WHEN ((`gen_attr` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END` FROM (SELECT CASE WHEN ((`gen_attr_1` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key.sql b/sql/hive/src/test/resources/sqlgen/case_with_key.sql index 88353e321e1d2..dff65f10835f3 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_key.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_key.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM (SELECT CASE WHEN (`gen_attr` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr` = CAST(1 AS BIGINT)) THEN "bar" END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN "bar" END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql index bc44f25d1ff8b..af3e169b54315 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM (SELECT CASE WHEN (`gen_attr` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr` = CAST(1 AS BIGINT)) THEN "bar" ELSE "baz" END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN "bar" ELSE "baz" END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/cluster_by.sql b/sql/hive/src/test/resources/sqlgen/cluster_by.sql index 3bc1dfc581882..3154791c3c5fd 100644 --- a/sql/hive/src/test/resources/sqlgen/cluster_by.sql +++ b/sql/hive/src/test/resources/sqlgen/cluster_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 CLUSTER BY id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr`) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr_0`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql index a4f46da9ffb75..e41b645937d37 100644 --- a/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql +++ b/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM json_parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`json_parquet_t0`) AS gen_subquery_0) AS json_parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`json_parquet_t0`) AS gen_subquery_0) AS json_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql index 214dbcf23625d..f5ceccde8c65b 100644 --- a/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql +++ b/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM orc_parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`orc_parquet_t0`) AS gen_subquery_0) AS orc_parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`orc_parquet_t0`) AS gen_subquery_0) AS orc_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql index eccf03eb122cb..2bccefe55e417 100644 --- a/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql +++ b/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_parquet_t0`) AS gen_subquery_0) AS parquet_parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_parquet_t0`) AS gen_subquery_0) AS parquet_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql b/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql index 0fa21ad3ba423..bced711caedf4 100644 --- a/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql +++ b/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(DISTINCT id) FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(DISTINCT id)` FROM (SELECT count(DISTINCT `gen_attr`) AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `count(DISTINCT id)` FROM (SELECT count(DISTINCT `gen_attr_1`) AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/distribute_by.sql b/sql/hive/src/test/resources/sqlgen/distribute_by.sql index 75a1a2a6b60bc..72863dcaf5c9c 100644 --- a/sql/hive/src/test/resources/sqlgen/distribute_by.sql +++ b/sql/hive/src/test/resources/sqlgen/distribute_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 DISTRIBUTE BY id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 DISTRIBUTE BY `gen_attr`) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 DISTRIBUTE BY `gen_attr_0`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql b/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql index a8e766e81db91..96b9b2dae87aa 100644 --- a/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql +++ b/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 DISTRIBUTE BY id SORT BY id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr`) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr_0`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/except.sql b/sql/hive/src/test/resources/sqlgen/except.sql index b1c64288d6e6b..7a7d27fcd6336 100644 --- a/sql/hive/src/test/resources/sqlgen/except.sql +++ b/sql/hive/src/test/resources/sqlgen/except.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 EXCEPT SELECT * FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 ) EXCEPT ( SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 ) EXCEPT ( SELECT `gen_attr_1` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql b/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql index 0156dffd0a9f7..9cd6514d771ff 100644 --- a/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a > 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a` FROM (SELECT `gen_attr` FROM (SELECT (`gen_attr` + CAST(1 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t WHERE (`gen_attr` > CAST(5 AS BIGINT))) AS t +SELECT `gen_attr_0` AS `a` FROM (SELECT `gen_attr_0` FROM (SELECT (`gen_attr_1` + CAST(1 AS BIGINT)) AS `gen_attr_0` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t WHERE (`gen_attr_0` > CAST(5 AS BIGINT))) AS t diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql index 15d23c67aec3c..805197a4ea11b 100644 --- a/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql @@ -5,4 +5,4 @@ WHERE id > 2 ORDER BY val, id LIMIT 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 WHERE (`gen_attr` > CAST(2 AS BIGINT))) AS gen_subquery_1 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC LIMIT 5) AS parquet_t3 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_0.`gen_attr_2`, gen_subquery_0.`gen_attr_3`, gen_subquery_0.`gen_attr_4`, gen_subquery_0.`gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 WHERE (`gen_attr_1` > CAST(2 AS BIGINT))) AS gen_subquery_1 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC LIMIT 5) AS parquet_t3 diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql index e2b94ce6739a6..ef9a596197b8b 100644 --- a/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql @@ -7,4 +7,4 @@ WHERE val > 2 ORDER BY val, id LIMIT 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW explode(`gen_attr`) gen_subquery_3 AS `gen_attr` WHERE (`gen_attr` > CAST(2 AS BIGINT)) ORDER BY `gen_attr` ASC, `gen_attr` ASC LIMIT 5) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0` WHERE (`gen_attr_0` > CAST(2 AS BIGINT)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC LIMIT 5) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql index c62b895c67403..2f6596ef422b0 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT val, id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql index 4c68eef214fb1..239980dd80bda 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT val, id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql index 059c45ec47c02..7fe0298c8e171 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `col` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql index 8f03dedbf971f..8db834acc73a1 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) AS val FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql index 0bfe38036ebf7..fef65e006867c 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr), id FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_1 AS `gen_attr`) AS parquet_t3 +SELECT `gen_attr_0` AS `col`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_1 AS `gen_attr_0`) AS parquet_t3 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql index e4edb6876cac1..e0e310888f11f 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr) AS val, id as a FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `a` FROM (SELECT `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `a` FROM (SELECT `gen_attr_0`, `gen_attr_2` AS `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_3`, `arr2` AS `gen_attr_4`, `json` AS `gen_attr_5`, `id` AS `gen_attr_2` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql index 5cf57f82d2df2..ea5db850bef8a 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr) FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `col` FROM (SELECT `gen_attr_0` FROM (SELECT `arr` AS `gen_attr_1`, `arr2` AS `gen_attr_2`, `json` AS `gen_attr_3`, `id` AS `gen_attr_4` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_1`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql index ed42894807d25..8f75b825476e0 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr) AS val FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val` FROM (SELECT `gen_attr_0` FROM (SELECT `arr` AS `gen_attr_1`, `arr2` AS `gen_attr_2`, `json` AS `gen_attr_3`, `id` AS `gen_attr_4` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_1`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql index 5a4aec961a5e0..984cce8a2ca83 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql @@ -3,4 +3,4 @@ SELECT exp.id, parquet_t3.id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `id`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql index 3c3e858deda7c..5c55b164c7feb 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql @@ -3,4 +3,4 @@ SELECT exp.id, parquet_t3.id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `id`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql b/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql index d56e89c8bb45a..ee22fe8728995 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_0 +SELECT `gen_attr_0` AS `col` FROM (SELECT `gen_attr_0` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_0 diff --git a/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql b/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql index d7acb6924076a..0acded74b3eee 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_0 +SELECT `gen_attr_0` AS `val` FROM (SELECT `gen_attr_0` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_0 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql index 72e429fe05b32..db2b2cc732889 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql @@ -3,4 +3,4 @@ SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id() AS k3 FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 GROUPING SETS (key % 5, key - 5) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))))) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `k3` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `gen_attr_7`, (`gen_attr_7` % CAST(2 AS BIGINT)) AS `gen_attr_8`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_9` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_12` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT))), ((`gen_attr_7` - CAST(5 AS BIGINT))))) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql index 25e86e73cbb7e..b2c426c660d80 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`), (`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`), (`gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql index 24bf02043488d..96ee8e85951e8 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql index f72bc8acd69a4..9b8b230c879c2 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql index 2fac7f661809e..c35db74a5c5b5 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (()) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS(()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS(()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql index 2e60c9c4d2c6c..e47f6d5dcf465 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql @@ -2,4 +2,4 @@ SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS ((), (a), (a, b)) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((), (`gen_attr`), (`gen_attr`, `gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((), (`gen_attr_5`), (`gen_attr_5`, `gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/in.sql b/sql/hive/src/test/resources/sqlgen/in.sql index 1e0251e586e20..7cff62b1af7df 100644 --- a/sql/hive/src/test/resources/sqlgen/in.sql +++ b/sql/hive/src/test/resources/sqlgen/in.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 WHERE id IN (1, 2, 3) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 WHERE (CAST(`gen_attr` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT)))) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 WHERE (CAST(`gen_attr_0` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT)))) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/intersect.sql b/sql/hive/src/test/resources/sqlgen/intersect.sql index 608fa59f8afbc..4143a6208d4b5 100644 --- a/sql/hive/src/test/resources/sqlgen/intersect.sql +++ b/sql/hive/src/test/resources/sqlgen/intersect.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 INTERSECT SELECT * FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 ) INTERSECT ( SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 ) INTERSECT ( SELECT `gen_attr_1` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/join_2_tables.sql b/sql/hive/src/test/resources/sqlgen/join_2_tables.sql index bb99797b8bcc6..9dd200c3c0cfa 100644 --- a/sql/hive/src/test/resources/sqlgen/join_2_tables.sql +++ b/sql/hive/src/test/resources/sqlgen/join_2_tables.sql @@ -4,4 +4,4 @@ FROM parquet_t1 a, parquet_t1 b GROUP BY a.KEY, b.KEY HAVING MAX(a.KEY) > 0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)`, `gen_attr` AS `KEY`, `gen_attr` AS `KEY` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, `gen_attr`, `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 GROUP BY `gen_attr`, `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `count(value)`, `gen_attr_1` AS `KEY`, `gen_attr_2` AS `KEY` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, max(`gen_attr_2`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_5` FROM `default`.`parquet_t1`) AS gen_subquery_1 GROUP BY `gen_attr_2`, `gen_attr_1` HAVING (`gen_attr_3` > CAST(0 AS BIGINT))) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql index 9e5fe1156e5de..6f5562a20cccd 100644 --- a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql @@ -3,4 +3,4 @@ SELECT c0, c1, c2 FROM parquet_t3 LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `c0`, `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr`, `gen_attr`, `gen_attr`) AS jt +SELECT `gen_attr_0` AS `c0`, `gen_attr_1` AS `c1`, `gen_attr_2` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql index 55dd491a32a32..0d4f67f18426b 100644 --- a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql @@ -3,4 +3,4 @@ SELECT a, b, c FROM parquet_t3 LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt AS a, b, c -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `c` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr`, `gen_attr`, `gen_attr`) AS jt +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_2` AS `c` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/multi_distinct.sql b/sql/hive/src/test/resources/sqlgen/multi_distinct.sql index b5c7cf0b0ce1a..3ca526fcc4415 100644 --- a/sql/hive/src/test/resources/sqlgen/multi_distinct.sql +++ b/sql/hive/src/test/resources/sqlgen/multi_distinct.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM parquet_t2 GROUP BY a -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `count(DISTINCT b)`, `gen_attr` AS `count(DISTINCT c)`, `gen_attr` AS `sum(d)` FROM (SELECT `gen_attr`, count(DISTINCT `gen_attr`) AS `gen_attr`, count(DISTINCT `gen_attr`) AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`) AS parquet_t2 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `count(DISTINCT b)`, `gen_attr_3` AS `count(DISTINCT c)`, `gen_attr_5` AS `sum(d)` FROM (SELECT `gen_attr_0`, count(DISTINCT `gen_attr_2`) AS `gen_attr_1`, count(DISTINCT `gen_attr_4`) AS `gen_attr_3`, sum(`gen_attr_6`) AS `gen_attr_5` FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_2`, `c` AS `gen_attr_4`, `d` AS `gen_attr_6` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_0`) AS parquet_t2 diff --git a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql index 3d79f95654585..e681c2b6354c0 100644 --- a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql +++ b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql @@ -4,4 +4,4 @@ FROM parquet_t3 LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array LATERAL VIEW EXPLODE(nested_array) exp1 AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW explode(`gen_attr`) gen_subquery_3 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql index 18b29a7bc5eea..e9d6522c91680 100644 --- a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql +++ b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql @@ -4,4 +4,4 @@ FROM parquet_t3 LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array LATERAL VIEW OUTER EXPLODE(nested_array) exp1 AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_3 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW OUTER explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/not_in.sql b/sql/hive/src/test/resources/sqlgen/not_in.sql index a50ee282e534e..797d22e8e9154 100644 --- a/sql/hive/src/test/resources/sqlgen/not_in.sql +++ b/sql/hive/src/test/resources/sqlgen/not_in.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM t0 WHERE id NOT IN (1, 2, 3) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT (CAST(`gen_attr` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT))))) AS t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT (CAST(`gen_attr_0` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT))))) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/not_like.sql b/sql/hive/src/test/resources/sqlgen/not_like.sql index 2f66695189cba..da39a62225a53 100644 --- a/sql/hive/src/test/resources/sqlgen/not_like.sql +++ b/sql/hive/src/test/resources/sqlgen/not_like.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM t0 WHERE id + 5 NOT LIKE '1%' -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT CAST((`gen_attr` + CAST(5 AS BIGINT)) AS STRING) LIKE "1%")) AS t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT CAST((`gen_attr_0` + CAST(5 AS BIGINT)) AS STRING) LIKE "1%")) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql b/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql index 2e06b4fe77ee6..6e5bd9860008c 100644 --- a/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from t1 b where exists (select * from t1 a) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a` FROM (SELECT `gen_attr` FROM (SELECT `a` AS `gen_attr` FROM `default`.`t1`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr` AS `a` FROM ((SELECT `gen_attr` FROM (SELECT `a` AS `gen_attr` FROM `default`.`t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_1)) AS b +SELECT `gen_attr_0` AS `a` FROM (SELECT `gen_attr_0` FROM (SELECT `a` AS `gen_attr_0` FROM `default`.`t1`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_1` AS `a` FROM ((SELECT `gen_attr_1` FROM (SELECT `a` AS `gen_attr_1` FROM `default`.`t1`) AS gen_subquery_2) AS gen_subquery_1) AS gen_subquery_1)) AS b diff --git a/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql index a7909485fbc5f..37cd5568baa7f 100644 --- a/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql +++ b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(key) OVER (PARTITION BY key % 3) + key FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT (`gen_attr_1` + `gen_attr_2`) AS `gen_attr_0` FROM (SELECT gen_subquery_1.`gen_attr_2`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_1` FROM (SELECT `gen_attr_2`, (`gen_attr_2` % CAST(3 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql index 4b2c78ea40c78..c54963ab5c550 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `(key % CAST(5 AS BIGINT))`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_2` AS `cnt`, `gen_attr_3` AS `(key % CAST(5 AS BIGINT))`, `gen_attr_4` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, grouping_id() AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_6` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_5` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_5` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql index 62cf1bb9d8e0c..6c869063c3bec 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `(key % CAST(5 AS BIGINT))`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_2` AS `cnt`, `gen_attr_3` AS `(key % CAST(5 AS BIGINT))`, `gen_attr_4` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, grouping_id() AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_6` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_5` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_5` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql index d58910af6939b..9628e38572940 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_3` AS `count(value)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_4` AS `gen_attr_1`, count(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_4` GROUPING SETS((`gen_attr_5`, `gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql index 9a58c15203d1e..d6b61929df0ad 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_3` AS `count(value)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_4` AS `gen_attr_1`, count(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_4` GROUPING SETS((`gen_attr_5`, `gen_attr_4`), (`gen_attr_5`), (`gen_attr_4`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql index cd4d428b07ede..d04b6578fc1ce 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `count(value)`, `gen_attr` AS `grouping_id()` FROM (SELECT `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_3` AS `count(value)`, `gen_attr_5` AS `grouping_id()` FROM (SELECT `gen_attr_6` AS `gen_attr_0`, count(`gen_attr_4`) AS `gen_attr_3`, grouping_id() AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_6`, `gen_attr_4` GROUPING SETS((`gen_attr_6`, `gen_attr_4`), (`gen_attr_6`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql index 4f42d7eda8901..80a5d93438f2a 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `count(value)`, `gen_attr` AS `grouping_id()` FROM (SELECT `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_3` AS `count(value)`, `gen_attr_5` AS `grouping_id()` FROM (SELECT `gen_attr_6` AS `gen_attr_0`, count(`gen_attr_4`) AS `gen_attr_3`, grouping_id() AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_6`, `gen_attr_4` GROUPING SETS((`gen_attr_6`, `gen_attr_4`), (`gen_attr_6`), (`gen_attr_4`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql index be4908ea16e52..619a554875ff0 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql @@ -2,4 +2,4 @@ SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 GROUP BY key % 5, key - 5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_8` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql index eebef6a5f3b89..8bf164519165c 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql @@ -2,4 +2,4 @@ SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 GROUP BY key % 5, key - 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_8` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ((`gen_attr_7` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql index 9474233cb7528..17e78a0a706a5 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql @@ -3,4 +3,4 @@ SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - FROM (SELECT key, key%2, key - 5 FROM parquet_t1) t GROUP BY key%5, key-5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `k3` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `gen_attr_7`, (`gen_attr_7` % CAST(2 AS BIGINT)) AS `gen_attr_8`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_9` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_12` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql index d36f43d665cda..72506ef72aecd 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql @@ -3,4 +3,4 @@ SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `k3` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `gen_attr_7`, (`gen_attr_7` % CAST(2 AS BIGINT)) AS `gen_attr_8`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_9` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_12` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ((`gen_attr_7` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql index de980784c7215..22df578518ef3 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql index 8956ac4c42f86..f44b652343acb 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), (`gen_attr_6`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql index b9cef62dcfdfa..40f6924913765 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(a) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql index ba6457ab1a54d..608e644dee6d0 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(a) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql index 763fecd9517fa..26885a26e2b96 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(a + b)`, `gen_attr` AS `b`, `gen_attr` AS `sum((a - b))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum((`gen_attr` - `gen_attr`)) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr` + `gen_attr`), `gen_attr` GROUPING SETS(((`gen_attr` + `gen_attr`), `gen_attr`), ((`gen_attr` + `gen_attr`)), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `(a + b)`, `gen_attr_1` AS `b`, `gen_attr_4` AS `sum((a - b))` FROM (SELECT (`gen_attr_5` + `gen_attr_6`) AS `gen_attr_3`, `gen_attr_6` AS `gen_attr_1`, sum((`gen_attr_5` - `gen_attr_6`)) AS `gen_attr_4` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_7`, `d` AS `gen_attr_8` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr_5` + `gen_attr_6`), `gen_attr_6` GROUPING SETS(((`gen_attr_5` + `gen_attr_6`), `gen_attr_6`), ((`gen_attr_5` + `gen_attr_6`)), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql index 00ecbf6078e00..dd97c976afe61 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(a + b)`, `gen_attr` AS `b`, `gen_attr` AS `sum((a - b))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum((`gen_attr` - `gen_attr`)) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr` + `gen_attr`), `gen_attr` GROUPING SETS(((`gen_attr` + `gen_attr`), `gen_attr`), ((`gen_attr` + `gen_attr`)), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `(a + b)`, `gen_attr_1` AS `b`, `gen_attr_4` AS `sum((a - b))` FROM (SELECT (`gen_attr_5` + `gen_attr_6`) AS `gen_attr_3`, `gen_attr_6` AS `gen_attr_1`, sum((`gen_attr_5` - `gen_attr_6`)) AS `gen_attr_4` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_7`, `d` AS `gen_attr_8` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr_5` + `gen_attr_6`), `gen_attr_6` GROUPING SETS(((`gen_attr_5` + `gen_attr_6`), `gen_attr_6`), ((`gen_attr_5` + `gen_attr_6`)), (`gen_attr_6`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql index 6f57130f3ea2c..aae2d75d794be 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, grouping_id(a, b) FROM parquet_t2 GROUP BY cube(a, b) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping_id(a, b)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `grouping_id(a, b)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, grouping_id() AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql index dab7852a3e967..9958c8f38bc87 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, grouping(b) FROM parquet_t2 GROUP BY cube(a, b) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping(b)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `grouping(b)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, grouping(`gen_attr_5`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql index 50964e5b01b6c..fd012043cf6cb 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, grouping(a) FROM parquet_t2 GROUP BY cube(a, b) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `grouping(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, grouping(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql index 84af580a36250..61c27067e1521 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql @@ -3,4 +3,4 @@ SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `hgid` FROM (SELECT `gen_attr` AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, hash(grouping_id()) AS `gen_attr` FROM (SELECT hash(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `k1`, `gen_attr_4` AS `k2`, `gen_attr_5` AS `hgid` FROM (SELECT `gen_attr_6` AS `gen_attr_3`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_4`, hash(grouping_id()) AS `gen_attr_5` FROM (SELECT hash(`gen_attr_10`) AS `gen_attr_6`, `gen_attr_10` AS `gen_attr_7` FROM (SELECT `key` AS `gen_attr_10`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT))), (`gen_attr_6`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql index b0f65b7d41bfb..16f254fa41f78 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql @@ -3,4 +3,4 @@ SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `hgid` FROM (SELECT `gen_attr` AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, hash(grouping_id()) AS `gen_attr` FROM (SELECT hash(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `k1`, `gen_attr_4` AS `k2`, `gen_attr_5` AS `hgid` FROM (SELECT `gen_attr_6` AS `gen_attr_3`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_4`, hash(grouping_id()) AS `gen_attr_5` FROM (SELECT hash(`gen_attr_10`) AS `gen_attr_6`, `gen_attr_10` AS `gen_attr_7` FROM (SELECT `key` AS `gen_attr_10`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT))), (`gen_attr_6`), ((`gen_attr_7` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql index 171400dd78cd1..cfce1758434de 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql @@ -5,4 +5,4 @@ FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t GROUP BY cnt, t.key - 5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(key - CAST(5 AS BIGINT))`, `gen_attr` AS `cnt`, `gen_attr` AS `sum(cnt)` FROM (SELECT (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ())) AS gen_subquery_2 +SELECT `gen_attr_3` AS `(key - CAST(5 AS BIGINT))`, `gen_attr_0` AS `cnt`, `gen_attr_4` AS `sum(cnt)` FROM (SELECT (`gen_attr_6` - CAST(5 AS BIGINT)) AS `gen_attr_3`, `gen_attr_5` AS `gen_attr_0`, sum(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `gen_attr_6`, count(1) AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_10` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_6` = `gen_attr_9`) GROUP BY `gen_attr_6`) AS t GROUP BY `gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT))), (`gen_attr_5`), ())) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql index fe98b6c343cc6..d950674b74c19 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql @@ -5,4 +5,4 @@ FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t GROUP BY cnt, t.key - 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(key - CAST(5 AS BIGINT))`, `gen_attr` AS `cnt`, `gen_attr` AS `sum(cnt)` FROM (SELECT (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_2 +SELECT `gen_attr_3` AS `(key - CAST(5 AS BIGINT))`, `gen_attr_0` AS `cnt`, `gen_attr_4` AS `sum(cnt)` FROM (SELECT (`gen_attr_6` - CAST(5 AS BIGINT)) AS `gen_attr_3`, `gen_attr_5` AS `gen_attr_0`, sum(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `gen_attr_6`, count(1) AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_10` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_6` = `gen_attr_9`) GROUP BY `gen_attr_6`) AS t GROUP BY `gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT))), (`gen_attr_5`), ((`gen_attr_6` - CAST(5 AS BIGINT))), ())) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql index 75e43d0dc1cfe..1736d74b0cfa9 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT TRANSFORM (a, b, c, d) USING 'cat' FROM parquet_t2 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) USING 'cat' AS (`gen_attr` string, `gen_attr` string) FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_4` AS `key`, `gen_attr_5` AS `value` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3`) USING 'cat' AS (`gen_attr_4` string, `gen_attr_5` string) FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_1`, `c` AS `gen_attr_2`, `d` AS `gen_attr_3` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql index 6a68edc72ebc3..07f59d6bffddc 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT TRANSFORM (*) USING 'cat' FROM parquet_t2 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) USING 'cat' AS (`gen_attr` string, `gen_attr` string) FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_4` AS `key`, `gen_attr_5` AS `value` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3`) USING 'cat' AS (`gen_attr_4` string, `gen_attr_5` string) FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_1`, `c` AS `gen_attr_2`, `d` AS `gen_attr_3` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql index 111df4bdf2299..fc0cabec237bc 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT TRANSFORM (a, b, c, d) USING 'cat' AS (d1, d2, d3, d4) FROM parquet_t2 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `d1`, `gen_attr` AS `d2`, `gen_attr` AS `d3`, `gen_attr` AS `d4` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr` string, `gen_attr` string, `gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_4` AS `d1`, `gen_attr_5` AS `d2`, `gen_attr_6` AS `d3`, `gen_attr_7` AS `d4` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr_4` string, `gen_attr_5` string, `gen_attr_6` string, `gen_attr_7` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_1`, `c` AS `gen_attr_2`, `d` AS `gen_attr_3` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql index 4ab412cfd171b..a45f9a2c625f6 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql @@ -3,4 +3,4 @@ FROM (FROM parquet_t1 SELECT TRANSFORM(key, value) USING 'cat' AS (thing1 int, thing2 string)) t SELECT thing1 + 1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(thing1 + 1)` FROM (SELECT (`gen_attr` + 1) AS `gen_attr` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr` int, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t) AS gen_subquery_1 +SELECT `gen_attr_0` AS `(thing1 + 1)` FROM (SELECT (`gen_attr_1` + 1) AS `gen_attr_0` FROM (SELECT TRANSFORM (`gen_attr_2`, `gen_attr_3`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr_1` int, `gen_attr_4` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql index 1e30e45d45d31..30d37c78b58e1 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql @@ -5,4 +5,4 @@ USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' USING 'cat' AS (`gen_attr` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_1` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr_0`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' USING 'cat' AS (`gen_attr_1` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql index dd622898780ac..0b694e0d6dafa 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql @@ -3,4 +3,4 @@ SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (`gen_attr` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_1` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr_0`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (`gen_attr_1` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql index 2ad369815742f..14cff373852dd 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql @@ -7,4 +7,4 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey`, `gen_attr` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') USING 'cat' AS (`gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_2` AS `tKey`, `gen_attr_3` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') USING 'cat' AS (`gen_attr_2` string, `gen_attr_3` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql index a90b42d9b4aed..d20caf7afcf0f 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql @@ -5,4 +5,4 @@ USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey`, `gen_attr` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (`gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_2` AS `tKey`, `gen_attr_3` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (`gen_attr_2` string, `gen_attr_3` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/select_distinct.sql b/sql/hive/src/test/resources/sqlgen/select_distinct.sql index 3bc8e55379281..09d93cac8e5fd 100644 --- a/sql/hive/src/test/resources/sqlgen/select_distinct.sql +++ b/sql/hive/src/test/resources/sqlgen/select_distinct.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT DISTINCT id FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT DISTINCT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT DISTINCT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/select_orc_table.sql b/sql/hive/src/test/resources/sqlgen/select_orc_table.sql index eae67f9540381..18ff021798972 100644 --- a/sql/hive/src/test/resources/sqlgen/select_orc_table.sql +++ b/sql/hive/src/test/resources/sqlgen/select_orc_table.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from orc_t -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `c1` AS `gen_attr`, `c2` AS `gen_attr` FROM `default`.`orc_t`) AS gen_subquery_0) AS orc_t +SELECT `gen_attr_0` AS `c1`, `gen_attr_1` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `c1` AS `gen_attr_0`, `c2` AS `gen_attr_1` FROM `default`.`orc_t`) AS gen_subquery_0) AS orc_t diff --git a/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql b/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql index a085bab84c9df..d2eac9c08f56c 100644 --- a/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql +++ b/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from parquet_t -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `c1` AS `gen_attr`, `c2` AS `gen_attr` FROM `default`.`parquet_t`) AS gen_subquery_0) AS parquet_t +SELECT `gen_attr_0` AS `c1`, `gen_attr_1` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `c1` AS `gen_attr_0`, `c2` AS `gen_attr_1` FROM `default`.`parquet_t`) AS gen_subquery_0) AS parquet_t diff --git a/sql/hive/src/test/resources/sqlgen/self_join.sql b/sql/hive/src/test/resources/sqlgen/self_join.sql index 8947ccda820a5..d6dcee2f67dbd 100644 --- a/sql/hive/src/test/resources/sqlgen/self_join.sql +++ b/sql/hive/src/test/resources/sqlgen/self_join.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT x.key FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key` FROM (SELECT `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`)) AS x +SELECT `gen_attr_0` AS `key` FROM (SELECT `gen_attr_0` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_1`)) AS x diff --git a/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql b/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql index 6db053f286028..1dedb44dbff65 100644 --- a/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql +++ b/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT x.key, COUNT(*) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key group by x.key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `count(1)` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS x +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_2`) GROUP BY `gen_attr_0`) AS x diff --git a/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql index e9a6afdc7eda1..da60204297a21 100644 --- a/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql +++ b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 SORT BY key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_1 SORT BY `gen_attr` ASC) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_1`) AS `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING (`gen_attr_2` > CAST(0 AS BIGINT))) AS gen_subquery_1 SORT BY `gen_attr_1` ASC) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_1.sql b/sql/hive/src/test/resources/sqlgen/tablesample_1.sql index 54c8deeb65679..291f2f59d7378 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_1.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT s.id FROM parquet_t0 TABLESAMPLE(100 PERCENT) s -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_2.sql b/sql/hive/src/test/resources/sqlgen/tablesample_2.sql index 13dcadb41a87f..6a92d7aef72f1 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_2.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM parquet_t0 TABLESAMPLE(100 PERCENT) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_3.sql b/sql/hive/src/test/resources/sqlgen/tablesample_3.sql index c1ed57f5e3f2e..4a17d7105eec6 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_3.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT s.id FROM t0 TABLESAMPLE(100 PERCENT) s -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_4.sql b/sql/hive/src/test/resources/sqlgen/tablesample_4.sql index c22b20802c41a..873de051a6bd5 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_4.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 TABLESAMPLE(100 PERCENT) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_5.sql b/sql/hive/src/test/resources/sqlgen/tablesample_5.sql index fa69dd46c7462..f958b2f111ba2 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_5.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_5.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT s.id FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) s WHERE 1=0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS s +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_6.sql b/sql/hive/src/test/resources/sqlgen/tablesample_6.sql index bc72560c42905..688a102d1da4e 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_6.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_6.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) WHERE 1=0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/three_child_union.sql b/sql/hive/src/test/resources/sqlgen/three_child_union.sql index c3781117d814d..713c7502f5a1a 100644 --- a/sql/hive/src/test/resources/sqlgen/three_child_union.sql +++ b/sql/hive/src/test/resources/sqlgen/three_child_union.sql @@ -3,4 +3,4 @@ SELECT id FROM parquet_t0 UNION ALL SELECT id FROM parquet_t0 UNION ALL SELECT id FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_1) UNION ALL (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_2)) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_1) UNION ALL (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_2)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/type_widening.sql b/sql/hive/src/test/resources/sqlgen/type_widening.sql index 96e6cc2fec90d..ebb8a92afd345 100644 --- a/sql/hive/src/test/resources/sqlgen/type_widening.sql +++ b/sql/hive/src/test/resources/sqlgen/type_widening.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 UNION ALL SELECT CAST(id AS INT) AS id FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT CAST(CAST(`gen_attr` AS INT) AS BIGINT) AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_1)) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT CAST(CAST(`gen_attr_0` AS INT) AS BIGINT) AS `gen_attr_1` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_1)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/union_distinct.sql b/sql/hive/src/test/resources/sqlgen/union_distinct.sql index c5895ae766d2e..46644b89ebb04 100644 --- a/sql/hive/src/test/resources/sqlgen/union_distinct.sql +++ b/sql/hive/src/test/resources/sqlgen/union_distinct.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 UNION SELECT * FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0) UNION DISTINCT (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0) UNION DISTINCT (SELECT `gen_attr_1` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_1.sql b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql index f86ca5e086b52..000c4e735ac6e 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_1.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(value) OVER (PARTITION BY key % 3) FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT `gen_attr_0` FROM (SELECT gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_2`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_2` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_0` FROM (SELECT `gen_attr_1`, (`gen_attr_3` % CAST(3 AS BIGINT)) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql index 15559ac8e15d4..ec55d4b7146f2 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql @@ -2,4 +2,4 @@ SELECT key, value, ROUND(AVG(key) OVER (), 2) FROM parquet_t1 ORDER BY key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr`, `gen_attr`, round(`gen_attr`, 2) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, avg(`gen_attr`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr` ASC) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, round(`gen_attr_3`, 2) AS `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, avg(`gen_attr_0`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr_0` ASC) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_3.sql b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql index 43f992882ce28..c0ac9541e67ee 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_3.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql @@ -2,4 +2,4 @@ SELECT value, MAX(key + 1) OVER (PARTITION BY key % 5 ORDER BY key % 7) AS max FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` + CAST(1 AS BIGINT)) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` % CAST(7 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `value`, `gen_attr_1` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_2`, gen_subquery_1.`gen_attr_3`, gen_subquery_1.`gen_attr_4`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_4` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, (`gen_attr_5` + CAST(1 AS BIGINT)) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, (`gen_attr_5` % CAST(7 AS BIGINT)) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_0` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_join.sql b/sql/hive/src/test/resources/sqlgen/window_with_join.sql index 00c45c8feafaa..030a4c0907a1c 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_join.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_join.sql @@ -2,4 +2,4 @@ SELECT x.key, MAX(y.key) OVER (PARTITION BY x.key % 5 ORDER BY x.key) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_2.`gen_attr`, gen_subquery_2.`gen_attr`, gen_subquery_2.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`)) AS gen_subquery_2) AS gen_subquery_3) AS x +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_2.`gen_attr_0`, gen_subquery_2.`gen_attr_2`, gen_subquery_2.`gen_attr_3`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_2`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_2`)) AS gen_subquery_2) AS gen_subquery_3) AS x diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql index 65d3d3aeb623e..7b99539a05480 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql @@ -4,4 +4,4 @@ DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, COUNT(key) FROM parquet_t1 GROUP BY key, value -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `dr`, `gen_attr` AS `count(key)` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, DENSE_RANK() OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `count(key)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, count(`gen_attr_0`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql index 03e0962f44943..591a654a3888e 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql @@ -4,4 +4,4 @@ DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, COUNT(key) OVER(DISTRIBUTE BY key SORT BY key, value) AS ca FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `dr`, `gen_attr` AS `ca` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, DENSE_RANK() OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr`, count(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `ca` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2`, count(`gen_attr_0`) OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql index 090207f9b8352..d9169eab6e46a 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql @@ -3,4 +3,4 @@ SELECT key, value, MAX(value) OVER (PARTITION BY key % 5 ORDER BY key) AS max FROM parquet_t1 GROUP BY key, value -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql index fcc2cf7de80a6..f0a820811ee0a 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql @@ -3,4 +3,4 @@ SELECT key, value, MAX(value) OVER (PARTITION BY key % 5 ORDER BY key DESC) AS max FROM parquet_t1 GROUP BY key, value HAVING key > 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` HAVING (`gen_attr` > CAST(5 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1` HAVING (`gen_attr_0` > CAST(5 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index ebece38773c23..aaff272bba8ce 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -97,17 +97,14 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { /** * Compare the generated SQL with the expected answer string. - * Note that there exists a normalization for both arguments for the convenience. - * - Remove the id from the generated attributes, e.g., `gen_attr_1` -> `gen_attr`. */ private def checkSQLStructure(originalSQL: String, convertedSQL: String, answerFile: String) = { - val normalizedGenSQL = convertedSQL.replaceAll("`gen_attr_\\d+`", "`gen_attr`") if (answerFile != null) { val separator = "-" * 80 if (regenerateGoldenFiles) { val path = Paths.get(s"$goldenSQLPath/$answerFile.sql") val header = "-- This file is automatically generated by LogicalPlanToSQLSuite." - val answerText = s"$header\n${originalSQL.trim()}\n${separator}\n$normalizedGenSQL\n" + val answerText = s"$header\n${originalSQL.trim()}\n${separator}\n$convertedSQL\n" Files.write(path, answerText.getBytes(StandardCharsets.UTF_8)) } else { val goldenFileName = s"sqlgen/$answerFile.sql" @@ -119,8 +116,8 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { val answerText = new String(Files.readAllBytes(Paths.get(path)), StandardCharsets.UTF_8) val sqls = answerText.split(separator) assert(sqls.length == 2, "Golden sql files should have a separator.") - val normalizedExpectSQL = sqls(1).trim() - assert(normalizedGenSQL == normalizedExpectSQL) + val expectedSQL = sqls(1).trim() + assert(convertedSQL == expectedSQL) } } } From 4e98e6905f0dd35902207d40e68befc0b0040b7d Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 27 Jul 2016 11:24:28 +0100 Subject: [PATCH 1048/1470] [MINOR][ML] Fix some mistake in LinearRegression formula. ## What changes were proposed in this pull request? Fix some mistake in ```LinearRegression``` formula. ## How was this patch tested? Documents change, no tests. Author: Yanbo Liang Closes #14369 from yanboliang/LiR-formula. (cherry picked from commit 3c3371bbd6361011b138cce88f6396a2aa4e2cb9) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/regression/LinearRegression.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 0a155e1844f62..f82f2c3a82b68 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 @@ -805,16 +805,16 @@ class LinearRegressionSummary private[regression] ( * {{{ * \frac{\partial L}{\partial w_i} = * 1/N \sum_j diff_j (x_{ij} - \bar{x_i}) / \hat{x_i} - * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - diffSum \bar{x_i}) / \hat{x_i}) + * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - diffSum \bar{x_i} / \hat{x_i}) * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + correction_i) * }}}, - * where correction_i = - diffSum \bar{x_i}) / \hat{x_i} + * where correction_i = - diffSum \bar{x_i} / \hat{x_i} * * A simple math can show that diffSum is actually zero, so we don't even * need to add the correction terms in the end. From the definition of diff, * {{{ * diffSum = \sum_j (\sum_i w_i(x_{ij} - \bar{x_i}) / \hat{x_i} - (y_j - \bar{y}) / \hat{y}) - * = N * (\sum_i w_i(\bar{x_i} - \bar{x_i}) / \hat{x_i} - (\bar{y_j} - \bar{y}) / \hat{y}) + * = N * (\sum_i w_i(\bar{x_i} - \bar{x_i}) / \hat{x_i} - (\bar{y} - \bar{y}) / \hat{y}) * = 0 * }}} * From 8bc2877d8c7cad6831de73a3f7c032b7dd73ae78 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Wed, 27 Jul 2016 16:04:43 +0800 Subject: [PATCH 1049/1470] [SPARK-16729][SQL] Throw analysis exception for invalid date casts Spark currently throws exceptions for invalid casts for all other data types except date type. Somehow date type returns null. It should be consistent and throws analysis exception as well. Added a unit test case in CastSuite. Author: petermaxlee Closes #14358 from petermaxlee/SPARK-16729. (cherry picked from commit ef0ccbcb07252db0ead8509e70d1a9a670d41616) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/expressions/Cast.scala | 9 ++------- .../sql/catalyst/expressions/CastSuite.scala | 16 ++++++++++++++-- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index b1e89b5de833f..a12fba047b3d4 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 @@ -52,7 +52,8 @@ object Cast { case (DateType, TimestampType) => true case (_: NumericType, TimestampType) => true - case (_, DateType) => true + case (StringType, DateType) => true + case (TimestampType, DateType) => true case (StringType, CalendarIntervalType) => true @@ -228,18 +229,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. buildCast[Long](_, t => DateTimeUtils.millisToDays(t / 1000L)) - // Hive throws this exception as a Semantic Exception - // It is never possible to compare result when hive return with exception, - // so we can return null - // NULL is more reasonable here, since the query itself obeys the grammar. - case _ => _ => null } // IntervalConverter private[this] def castToInterval(from: DataType): Any => Any = from match { case StringType => buildCast[UTF8String](_, s => CalendarInterval.fromString(s.toString)) - case _ => _ => null } // LongConverter diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index dfda7c50f2c05..5ae0527a9c7ae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -70,7 +70,8 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkNullCast(DateType, TimestampType) numericTypes.foreach(dt => checkNullCast(dt, TimestampType)) - atomicTypes.foreach(dt => checkNullCast(dt, DateType)) + checkNullCast(StringType, DateType) + checkNullCast(TimestampType, DateType) checkNullCast(StringType, CalendarIntervalType) numericTypes.foreach(dt => checkNullCast(StringType, dt)) @@ -366,7 +367,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast("2012-12-11", DoubleType), null) checkEvaluation(cast(123, IntegerType), 123) - checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null) } @@ -783,4 +783,16 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast("abc", BooleanType), null) checkEvaluation(cast("", BooleanType), null) } + + test("SPARK-16729 type checking for casting to date type") { + assert(cast("1234", DateType).checkInputDataTypes().isSuccess) + assert(cast(new Timestamp(1), DateType).checkInputDataTypes().isSuccess) + assert(cast(false, DateType).checkInputDataTypes().isFailure) + assert(cast(1.toByte, DateType).checkInputDataTypes().isFailure) + assert(cast(1.toShort, DateType).checkInputDataTypes().isFailure) + assert(cast(1, DateType).checkInputDataTypes().isFailure) + assert(cast(1L, DateType).checkInputDataTypes().isFailure) + assert(cast(1.0.toFloat, DateType).checkInputDataTypes().isFailure) + assert(cast(1.0, DateType).checkInputDataTypes().isFailure) + } } From 2f4e06e381465289680348a78006a2e24be86e62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bartek=20Wis=CC=81niewski?= Date: Wed, 27 Jul 2016 10:53:22 -0700 Subject: [PATCH 1050/1470] [MINOR][DOC] missing keyword new MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? added missing keyword for java example ## How was this patch tested? wasn't Author: Bartek Wiśniewski Closes #14381 from wedi-dev/quickfix/missing_keyword. (cherry picked from commit bc4851adeb386edc5bef47027a12ca44eda82b09) Signed-off-by: Reynold Xin --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index de82a064d1d3e..e80f1c94ff1b4 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -477,7 +477,7 @@ import org.apache.spark.*; import org.apache.spark.streaming.api.java.*; SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); -JavaStreamingContext ssc = new JavaStreamingContext(conf, Duration(1000)); +JavaStreamingContext ssc = new JavaStreamingContext(conf, new Duration(1000)); {% endhighlight %} The `appName` parameter is a name for your application to show on the cluster UI. From 2d56a213622f699dd6c65b1c79621178a597bbf7 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 28 Jul 2016 13:13:17 +0800 Subject: [PATCH 1051/1470] [SPARK-16730][SQL] Implement function aliases for type casts ## What changes were proposed in this pull request? Spark 1.x supports using the Hive type name as function names for doing casts, e.g. ```sql SELECT int(1.0); SELECT string(2.0); ``` The above query would work in Spark 1.x because Spark 1.x fail back to Hive for unimplemented functions, and break in Spark 2.0 because the fall back was removed. This patch implements function aliases using an analyzer rule for the following cast functions: - boolean - tinyint - smallint - int - bigint - float - double - decimal - date - timestamp - binary - string ## How was this patch tested? Added end-to-end tests in SQLCompatibilityFunctionSuite. Author: petermaxlee Closes #14364 from petermaxlee/SPARK-16730-2. (cherry picked from commit 11d427c924d303e20af90c0179a105f6ff4d89e2) Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 51 ++++++++++++++++--- .../spark/sql/catalyst/expressions/Cast.scala | 3 ++ .../sql/SQLCompatibilityFunctionSuite.scala | 26 ++++++++++ 3 files changed, 73 insertions(+), 7 deletions(-) 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 65168998c8aee..c5f91c1590542 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.xml._ import org.apache.spark.sql.catalyst.util.StringKeyHashMap +import org.apache.spark.sql.types._ /** @@ -408,8 +409,21 @@ object FunctionRegistry { expression[BitwiseAnd]("&"), expression[BitwiseNot]("~"), expression[BitwiseOr]("|"), - expression[BitwiseXor]("^") - + expression[BitwiseXor]("^"), + + // Cast aliases (SPARK-16730) + castAlias("boolean", BooleanType), + castAlias("tinyint", ByteType), + castAlias("smallint", ShortType), + castAlias("int", IntegerType), + castAlias("bigint", LongType), + castAlias("float", FloatType), + castAlias("double", DoubleType), + castAlias("decimal", DecimalType.USER_DEFAULT), + castAlias("date", DateType), + castAlias("timestamp", TimestampType), + castAlias("binary", BinaryType), + castAlias("string", StringType) ) val builtin: SimpleFunctionRegistry = { @@ -452,14 +466,37 @@ object FunctionRegistry { } } - val clazz = tag.runtimeClass + (name, (expressionInfo[T](name), builder)) + } + + /** + * Creates a function registry lookup entry for cast aliases (SPARK-16730). + * For example, if name is "int", and dataType is IntegerType, this means int(x) would become + * an alias for cast(x as IntegerType). + * See usage above. + */ + private def castAlias( + name: String, + dataType: DataType): (String, (ExpressionInfo, FunctionBuilder)) = { + val builder = (args: Seq[Expression]) => { + if (args.size != 1) { + throw new AnalysisException(s"Function $name accepts only one argument") + } + Cast(args.head, dataType) + } + (name, (expressionInfo[Cast](name), builder)) + } + + /** + * Creates an [[ExpressionInfo]] for the function as defined by expression T using the given name. + */ + private def expressionInfo[T <: Expression : ClassTag](name: String): ExpressionInfo = { + val clazz = scala.reflect.classTag[T].runtimeClass val df = clazz.getAnnotation(classOf[ExpressionDescription]) if (df != null) { - (name, - (new ExpressionInfo(clazz.getCanonicalName, name, df.usage(), df.extended()), - builder)) + new ExpressionInfo(clazz.getCanonicalName, name, df.usage(), df.extended()) } else { - (name, (new ExpressionInfo(clazz.getCanonicalName, name), builder)) + new ExpressionInfo(clazz.getCanonicalName, name) } } } 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 a12fba047b3d4..c452765af2dd9 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 @@ -113,6 +113,9 @@ object Cast { } /** Cast the child expression to the target data type. */ +@ExpressionDescription( + usage = " - Cast value v to the target data type.", + extended = "> SELECT _FUNC_('10' as int);\n 10") case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with NullIntolerant { override def toString: String = s"cast($child as ${dataType.simpleString})" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala index 1e3239550fb81..27b60e0d9def8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLCompatibilityFunctionSuite.scala @@ -17,10 +17,14 @@ package org.apache.spark.sql +import java.math.BigDecimal +import java.sql.Timestamp + import org.apache.spark.sql.test.SharedSQLContext /** * A test suite for functions added for compatibility with other databases such as Oracle, MSSQL. + * * These functions are typically implemented using the trait * [[org.apache.spark.sql.catalyst.expressions.RuntimeReplaceable]]. */ @@ -69,4 +73,26 @@ class SQLCompatibilityFunctionSuite extends QueryTest with SharedSQLContext { sql("SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d)"), Row(2.1, 1.0)) } + + test("SPARK-16730 cast alias functions for Hive compatibility") { + checkAnswer( + sql("SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1)"), + Row(true, 1.toByte, 1.toShort, 1, 1L)) + + checkAnswer( + sql("SELECT float(1), double(1), decimal(1)"), + Row(1.toFloat, 1.0, new BigDecimal(1))) + + checkAnswer( + sql("SELECT date(\"2014-04-04\"), timestamp(date(\"2014-04-04\"))"), + Row(new java.util.Date(114, 3, 4), new Timestamp(114, 3, 4, 0, 0, 0, 0))) + + checkAnswer( + sql("SELECT string(1)"), + Row("1")) + + // Error handling: only one argument + val errorMsg = intercept[AnalysisException](sql("SELECT string(1, 2)")).getMessage + assert(errorMsg.contains("Function string accepts only one argument")) + } } From 0fd2dfb6dee9d7eaa277d6806e56f1b0531afa51 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 27 Jul 2016 23:29:26 -0700 Subject: [PATCH 1052/1470] [SPARK-15232][SQL] Add subquery SQL building tests to LogicalPlanToSQLSuite ## What changes were proposed in this pull request? We currently test subquery SQL building using the `HiveCompatibilitySuite`. The is not desired since SQL building is actually a part of `sql/core` and because we are slowly reducing our dependency on Hive. This PR adds the same tests from the whitelist of `HiveCompatibilitySuite` into `LogicalPlanToSQLSuite`. ## How was this patch tested? This adds more testcases. Pass the Jenkins tests. Author: Dongjoon Hyun Closes #14383 from dongjoon-hyun/SPARK-15232. (cherry picked from commit 5c2ae79bfcf448d8dc9217efafa1409997c739de) Signed-off-by: Reynold Xin --- .../sqlgen/broadcast_join_subquery.sql | 8 + sql/hive/src/test/resources/sqlgen/subq2.sql | 8 + .../resources/sqlgen/subquery_exists_1.sql | 8 + .../resources/sqlgen/subquery_exists_2.sql | 9 + .../sqlgen/subquery_exists_having_1.sql | 9 + .../sqlgen/subquery_exists_having_2.sql | 10 ++ .../sqlgen/subquery_exists_having_3.sql | 9 + .../src/test/resources/sqlgen/subquery_in.sql | 6 + .../resources/sqlgen/subquery_in_having_1.sql | 8 + .../resources/sqlgen/subquery_in_having_2.sql | 10 ++ .../sqlgen/subquery_not_exists_1.sql | 8 + .../sqlgen/subquery_not_exists_2.sql | 8 + .../sqlgen/subquery_not_exists_having_1.sql | 9 + .../sqlgen/subquery_not_exists_having_2.sql | 9 + .../sql/catalyst/LogicalPlanToSQLSuite.scala | 163 ++++++++++++++++++ 15 files changed, 282 insertions(+) create mode 100644 sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subq2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_in.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql create mode 100644 sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql diff --git a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql new file mode 100644 index 0000000000000..3e2111d58a3c6 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT /*+ MAPJOIN(srcpart) */ subq.key1, z.value +FROM (SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + FROM src1 x JOIN src y ON (x.key = y.key)) subq +JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) +ORDER BY subq.key1, z.value +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = "2008-04-08")) AND (CAST(`gen_attr_4` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/subq2.sql b/sql/hive/src/test/resources/sqlgen/subq2.sql new file mode 100644 index 0000000000000..ee7e80c1fc9e2 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subq2.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT a.k, a.c +FROM (SELECT b.key as k, count(1) as c + FROM src b + GROUP BY b.key) a +WHERE a.k >= 90 +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `k`, `gen_attr_1` AS `c` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_2` AS `gen_attr_0`, count(1) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_2`) AS a WHERE (`gen_attr_0` >= 90)) AS a diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql new file mode 100644 index 0000000000000..d598e4c036a29 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * +from src b +where exists (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_9")) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql new file mode 100644 index 0000000000000..a353c33af21a6 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql @@ -0,0 +1,9 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * +from (select * + from src b + where exists (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9')) a +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_9")) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS a) AS a diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql new file mode 100644 index 0000000000000..f6873d24e16ec --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql @@ -0,0 +1,9 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select b.key, count(*) +from src b +group by b.key +having exists (select a.key + from src a + where a.key = b.key and a.value > 'val_9') +-------------------------------------------------------------------------------- +SELECT `gen_attr_1` AS `key`, `gen_attr_2` AS `count(1)` FROM (SELECT `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_0` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > "val_9")) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3)) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql new file mode 100644 index 0000000000000..8452ef946f61d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql @@ -0,0 +1,10 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * +from (select b.key, count(*) + from src b + group by b.key + having exists (select a.key + from src a + where a.key = b.key and a.value > 'val_9')) a +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > "val_9")) AS gen_subquery_1 WHERE (`gen_attr_2` = `gen_attr_0`)) AS gen_subquery_3)) AS a) AS a diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql new file mode 100644 index 0000000000000..2ef38ce42944f --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql @@ -0,0 +1,9 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select b.key, min(b.value) +from src b +group by b.key +having exists (select a.key + from src a + where a.value > 'val_9' and a.value = min(b.value)) +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_4`) AS `gen_attr_1`, min(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_5` AS `1` FROM (SELECT 1 AS `gen_attr_5` FROM (SELECT `gen_attr_6`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_2` > "val_9")) AS gen_subquery_2 WHERE (`gen_attr_2` = `gen_attr_3`)) AS gen_subquery_4)) AS gen_subquery_1) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in.sql b/sql/hive/src/test/resources/sqlgen/subquery_in.sql new file mode 100644 index 0000000000000..0fe62248dbfec --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_in.sql @@ -0,0 +1,6 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key +FROM src +WHERE key in (SELECT max(key) FROM src) +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key` FROM (SELECT `gen_attr_0` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_0 WHERE (`gen_attr_0` IN (SELECT `gen_attr_3` AS `_c0` FROM (SELECT `gen_attr_1` AS `gen_attr_3` FROM (SELECT max(`gen_attr_4`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2) AS gen_subquery_1) AS gen_subquery_3))) AS src diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql new file mode 100644 index 0000000000000..bfa58211b12f1 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select key, count(*) +from src +group by key +having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key) +order by key +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (CAST(`gen_attr_6` AS DOUBLE) = CAST("90" AS DOUBLE)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS src diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql new file mode 100644 index 0000000000000..f7503bce068f8 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql @@ -0,0 +1,10 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select b.key, min(b.value) +from src b +group by b.key +having b.key in (select a.key + from src a + where a.value > 'val_9' and a.value = min(b.value)) +order by b.key +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > "val_9")) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql new file mode 100644 index 0000000000000..54a38ec0edb4c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * +from src b +where not exists (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_2') +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_2")) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql new file mode 100644 index 0000000000000..c05bb5d991b4b --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * +from src b +where not exists (select a.key + from src a + where b.value = a.value and a.value > 'val_2') +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT `gen_attr_4`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_2")) AS gen_subquery_1 WHERE (`gen_attr_1` = `gen_attr_2`)) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql new file mode 100644 index 0000000000000..d6047c52f20fc --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql @@ -0,0 +1,9 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * +from src b +group by key, value +having not exists (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_12') +-------------------------------------------------------------------------------- +SELECT `gen_attr_3` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_3`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_3`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > "val_12")) AS gen_subquery_1 WHERE ((`gen_attr_0` = `gen_attr_1`) AND (`gen_attr_2` = `gen_attr_3`))) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql new file mode 100644 index 0000000000000..8b5402d8aa77f --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql @@ -0,0 +1,9 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * +from src b +group by key, value +having not exists (select distinct a.key + from src a + where b.value = a.value and a.value > 'val_12') +-------------------------------------------------------------------------------- +SELECT `gen_attr_2` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_2`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_2`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT DISTINCT `gen_attr_4`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > "val_12")) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index aaff272bba8ce..d8ab864ca6fce 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -934,6 +934,169 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } } + test("broadcast join") { + checkSQL( + """ + |SELECT /*+ MAPJOIN(srcpart) */ subq.key1, z.value + |FROM (SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 + | FROM src1 x JOIN src y ON (x.key = y.key)) subq + |JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) + |ORDER BY subq.key1, z.value + """.stripMargin, + "broadcast_join_subquery") + } + + test("subquery using single table") { + checkSQL( + """ + |SELECT a.k, a.c + |FROM (SELECT b.key as k, count(1) as c + | FROM src b + | GROUP BY b.key) a + |WHERE a.k >= 90 + """.stripMargin, + "subq2") + } + + test("correlated subqueries using EXISTS on where clause") { + checkSQL( + """ + |select * + |from src b + |where exists (select a.key + | from src a + | where b.value = a.value and a.key = b.key and a.value > 'val_9') + """.stripMargin, + "subquery_exists_1") + + checkSQL( + """ + |select * + |from (select * + | from src b + | where exists (select a.key + | from src a + | where b.value = a.value and a.key = b.key and a.value > 'val_9')) a + """.stripMargin, + "subquery_exists_2") + } + + test("correlated subqueries using EXISTS on having clause") { + checkSQL( + """ + |select b.key, count(*) + |from src b + |group by b.key + |having exists (select a.key + | from src a + | where a.key = b.key and a.value > 'val_9') + """.stripMargin, + "subquery_exists_having_1") + + checkSQL( + """ + |select * + |from (select b.key, count(*) + | from src b + | group by b.key + | having exists (select a.key + | from src a + | where a.key = b.key and a.value > 'val_9')) a + """.stripMargin, + "subquery_exists_having_2") + + checkSQL( + """ + |select b.key, min(b.value) + |from src b + |group by b.key + |having exists (select a.key + | from src a + | where a.value > 'val_9' and a.value = min(b.value)) + """.stripMargin, + "subquery_exists_having_3") + } + + test("correlated subqueries using NOT EXISTS on where clause") { + checkSQL( + """ + |select * + |from src b + |where not exists (select a.key + | from src a + | where b.value = a.value and a.key = b.key and a.value > 'val_2') + """.stripMargin, + "subquery_not_exists_1") + + checkSQL( + """ + |select * + |from src b + |where not exists (select a.key + | from src a + | where b.value = a.value and a.value > 'val_2') + """.stripMargin, + "subquery_not_exists_2") + } + + test("correlated subqueries using NOT EXISTS on having clause") { + checkSQL( + """ + |select * + |from src b + |group by key, value + |having not exists (select a.key + | from src a + | where b.value = a.value and a.key = b.key and a.value > 'val_12') + """.stripMargin, + "subquery_not_exists_having_1") + + checkSQL( + """ + |select * + |from src b + |group by key, value + |having not exists (select distinct a.key + | from src a + | where b.value = a.value and a.value > 'val_12') + """.stripMargin, + "subquery_not_exists_having_2") + } + + test("subquery using IN on where clause") { + checkSQL( + """ + |SELECT key + |FROM src + |WHERE key in (SELECT max(key) FROM src) + """.stripMargin, + "subquery_in") + } + + test("subquery using IN on having clause") { + checkSQL( + """ + |select key, count(*) + |from src + |group by key + |having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key) + |order by key + """.stripMargin, + "subquery_in_having_1") + + checkSQL( + """ + |select b.key, min(b.value) + |from src b + |group by b.key + |having b.key in (select a.key + | from src a + | where a.value > 'val_9' and a.value = min(b.value)) + |order by b.key + """.stripMargin, + "subquery_in_having_2") + } + test("SPARK-14933 - select orc table") { withTable("orc_t") { sql("create table orc_t stored as orc as select 1 as c1, 'abc' as c2") From 825c8371784468ff976526deffd97ad7df997738 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 28 Jul 2016 22:33:33 +0800 Subject: [PATCH 1053/1470] [SPARK-16639][SQL] The query with having condition that contains grouping by column should work ## What changes were proposed in this pull request? The query with having condition that contains grouping by column will be failed during analysis. E.g., create table tbl(a int, b string); select count(b) from tbl group by a + 1 having a + 1 = 2; Having condition should be able to use grouping by column. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #14296 from viirya/having-contains-grouping-column. (cherry picked from commit 9ade77c3fa2e1bf436b79368a97d5980c12fe215) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 15 ++++++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 22 ++++++++++++++----- 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 61162ccdba810..2efa997ff22d2 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 @@ -246,7 +246,7 @@ class Analyzer( }.isDefined } - private def hasGroupingFunction(e: Expression): Boolean = { + private[sql] def hasGroupingFunction(e: Expression): Boolean = { e.collectFirst { case g: Grouping => g case g: GroupingID => g @@ -1207,6 +1207,19 @@ class Analyzer( val alias = Alias(ae, ae.toString)() aggregateExpressions += alias alias.toAttribute + // Grouping functions are handled in the rule [[ResolveGroupingAnalytics]]. + case e: Expression if grouping.exists(_.semanticEquals(e)) && + !ResolveGroupingAnalytics.hasGroupingFunction(e) && + !aggregate.output.exists(_.semanticEquals(e)) => + e match { + case ne: NamedExpression => + aggregateExpressions += ne + ne.toAttribute + case _ => + val alias = Alias(e, e.toString)() + aggregateExpressions += alias + alias.toAttribute + } } // Push the aggregate expressions into the aggregate (if any). 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 be84dff09239d..d9659012fac5c 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 @@ -39,11 +39,23 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { setupTestData() test("having clause") { - Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") - .createOrReplaceTempView("hav") - checkAnswer( - sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"), - Row("one", 6) :: Row("three", 3) :: Nil) + withTempView("hav") { + Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") + .createOrReplaceTempView("hav") + checkAnswer( + sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"), + Row("one", 6) :: Row("three", 3) :: Nil) + } + } + + test("having condition contains grouping column") { + withTempView("hav") { + Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") + .createOrReplaceTempView("hav") + checkAnswer( + sql("SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2"), + Row(1) :: Nil) + } } test("SPARK-8010: promote numeric to string") { From f46a074510e47206de9d3b3ac6902af321923ce8 Mon Sep 17 00:00:00 2001 From: Sylvain Zimmer Date: Thu, 28 Jul 2016 09:51:45 -0700 Subject: [PATCH 1054/1470] [SPARK-16740][SQL] Fix Long overflow in LongToUnsafeRowMap Avoid overflow of Long type causing a NegativeArraySizeException a few lines later. Unit tests for HashedRelationSuite still pass. I can confirm the python script I included in https://issues.apache.org/jira/browse/SPARK-16740 works fine with this patch. Unfortunately I don't have the knowledge/time to write a Scala test case for HashedRelationSuite right now. As the patch is pretty obvious I hope it can be included without this. Thanks! Author: Sylvain Zimmer Closes #14373 from sylvinus/master. (cherry picked from commit 1178d61ede816bf1c8d5bb3dbb3b965c9b944407) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/execution/joins/HashedRelation.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 412e8c54ca308..cf4454c033384 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 @@ -608,7 +608,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap def optimize(): Unit = { val range = maxKey - minKey // Convert to dense mode if it does not require more memory or could fit within L1 cache - if (range < array.length || range < 1024) { + // SPARK-16740: Make sure range doesn't overflow if minKey has a large negative value + if (range >= 0 && (range < array.length || range < 1024)) { try { ensureAcquireMemory((range + 1) * 8L) } catch { From fb09a693d6f58d71ec042224b8ea66b972c1adc2 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Thu, 28 Jul 2016 13:04:19 -0700 Subject: [PATCH 1055/1470] [SPARK-16764][SQL] Recommend disabling vectorized parquet reader on OutOfMemoryError ## What changes were proposed in this pull request? We currently don't bound or manage the data array size used by column vectors in the vectorized reader (they're just bound by INT.MAX) which may lead to OOMs while reading data. As a short term fix, this patch intercepts the OutOfMemoryError exception and suggest the user to disable the vectorized parquet reader. ## How was this patch tested? Existing Tests Author: Sameer Agarwal Closes #14387 from sameeragarwal/oom. (cherry picked from commit 3fd39b87bda77f3c3a4622d854f23d4234683571) Signed-off-by: Reynold Xin --- .../execution/vectorized/ColumnVector.java | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index bbbb796aca0de..59173d253b298 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -282,16 +282,30 @@ public void reserve(int requiredCapacity) { if (requiredCapacity > capacity) { int newCapacity = (int) Math.min(MAX_CAPACITY, requiredCapacity * 2L); if (requiredCapacity <= newCapacity) { - reserveInternal(newCapacity); + try { + reserveInternal(newCapacity); + } catch (OutOfMemoryError outOfMemoryError) { + throwUnsupportedException(newCapacity, requiredCapacity, outOfMemoryError); + } } else { - throw new RuntimeException("Cannot reserve more than " + newCapacity + - " bytes in the vectorized reader (requested = " + requiredCapacity + " bytes). As a " + - "workaround, you can disable the vectorized reader by setting " - + SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key() + " to false."); + throwUnsupportedException(newCapacity, requiredCapacity, null); } } } + private void throwUnsupportedException(int newCapacity, int requiredCapacity, Throwable cause) { + String message = "Cannot reserve more than " + newCapacity + + " bytes in the vectorized reader (requested = " + requiredCapacity + " bytes). As a" + + " workaround, you can disable the vectorized reader by setting " + + SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key() + " to false."; + + if (cause != null) { + throw new RuntimeException(message, cause); + } else { + throw new RuntimeException(message); + } + } + /** * Ensures that there is enough storage to store capcity elements. That is, the put() APIs * must work for all rowIds < capcity. From 5cd79c396f98660e12b02c0151a084b4d1599b6b Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 28 Jul 2016 14:57:15 -0700 Subject: [PATCH 1056/1470] [SPARK-16772] Correct API doc references to PySpark classes + formatting fixes ## What's Been Changed The PR corrects several broken or missing class references in the Python API docs. It also correct formatting problems. For example, you can see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.registerFunction) how Sphinx is not picking up the reference to `DataType`. That's because the reference is relative to the current module, whereas `DataType` is in a different module. You can also see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.createDataFrame) how the formatting for byte, tinyint, and so on is italic instead of monospace. That's because in ReST single backticks just make things italic, unlike in Markdown. ## Testing I tested this PR by [building the Python docs](https://github.com/apache/spark/tree/master/docs#generating-the-documentation-html) and reviewing the results locally in my browser. I confirmed that the broken or missing class references were resolved, and that the formatting was corrected. Author: Nicholas Chammas Closes #14393 from nchammas/python-docstring-fixes. (cherry picked from commit 274f3b9ec86e4109c7678eef60f990d41dc3899f) Signed-off-by: Reynold Xin --- python/pyspark/sql/catalog.py | 2 +- python/pyspark/sql/context.py | 44 ++++++++++++++++++-------------- python/pyspark/sql/dataframe.py | 2 +- python/pyspark/sql/functions.py | 21 +++++++++------ python/pyspark/sql/readwriter.py | 8 +++--- python/pyspark/sql/session.py | 41 ++++++++++++++++------------- python/pyspark/sql/streaming.py | 8 +++--- python/pyspark/sql/types.py | 7 ++--- 8 files changed, 75 insertions(+), 58 deletions(-) diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 4af930a3cd563..3c5030722f307 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -193,7 +193,7 @@ def registerFunction(self, name, f, returnType=StringType()): :param name: name of the UDF :param f: python function - :param returnType: a :class:`DataType` object + :param returnType: a :class:`pyspark.sql.types.DataType` object >>> spark.catalog.registerFunction("stringLengthString", lambda x: len(x)) >>> spark.sql("SELECT stringLengthString('test')").collect() diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 4cfdf799f6f42..afb9b54432100 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -152,9 +152,9 @@ def udf(self): @since(1.4) def range(self, start, end=None, step=1, numPartitions=None): """ - Create a :class:`DataFrame` with single LongType column named `id`, - containing elements in a range from `start` to `end` (exclusive) with - step value `step`. + Create a :class:`DataFrame` with single :class:`pyspark.sql.types.LongType` column named + ``id``, containing elements in a range from ``start`` to ``end`` (exclusive) with + step value ``step``. :param start: the start value :param end: the end value (exclusive) @@ -184,7 +184,7 @@ def registerFunction(self, name, f, returnType=StringType()): :param name: name of the UDF :param f: python function - :param returnType: a :class:`DataType` object + :param returnType: a :class:`pyspark.sql.types.DataType` object >>> sqlContext.registerFunction("stringLengthString", lambda x: len(x)) >>> sqlContext.sql("SELECT stringLengthString('test')").collect() @@ -209,7 +209,7 @@ def _inferSchema(self, rdd, samplingRatio=None): :param rdd: an RDD of Row or tuple :param samplingRatio: sampling ratio, or no sampling (default) - :return: StructType + :return: :class:`pyspark.sql.types.StructType` """ return self.sparkSession._inferSchema(rdd, samplingRatio) @@ -226,28 +226,34 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): from ``data``, which should be an RDD of :class:`Row`, or :class:`namedtuple`, or :class:`dict`. - When ``schema`` is :class:`DataType` or datatype string, it must match the real data, or - exception will be thrown at runtime. If the given schema is not StructType, it will be - wrapped into a StructType as its only field, and the field name will be "value", each record - will also be wrapped into a tuple, which can be converted to row later. + When ``schema`` is :class:`pyspark.sql.types.DataType` or + :class:`pyspark.sql.types.StringType`, it must match the + real data, or an exception will be thrown at runtime. If the given schema is not + :class:`pyspark.sql.types.StructType`, it will be wrapped into a + :class:`pyspark.sql.types.StructType` as its only field, and the field name will be "value", + each record will also be wrapped into a tuple, which can be converted to row later. If schema inference is needed, ``samplingRatio`` is used to determined the ratio of rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``. - :param data: an RDD of any kind of SQL data representation(e.g. row, tuple, int, boolean, - etc.), or :class:`list`, or :class:`pandas.DataFrame`. - :param schema: a :class:`DataType` or a datatype string or a list of column names, default - is None. The data type string format equals to `DataType.simpleString`, except that - top level struct type can omit the `struct<>` and atomic types use `typeName()` as - their format, e.g. use `byte` instead of `tinyint` for ByteType. We can also use `int` - as a short name for IntegerType. + :param data: an RDD of any kind of SQL data representation(e.g. :class:`Row`, + :class:`tuple`, ``int``, ``boolean``, etc.), or :class:`list`, or + :class:`pandas.DataFrame`. + :param schema: a :class:`pyspark.sql.types.DataType` or a + :class:`pyspark.sql.types.StringType` or a list of + column names, default is None. The data type string format equals to + :class:`pyspark.sql.types.DataType.simpleString`, except that top level struct type can + omit the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use + ``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`. + We can also use ``int`` as a short name for :class:`pyspark.sql.types.IntegerType`. :param samplingRatio: the sample ratio of rows used for inferring :return: :class:`DataFrame` .. versionchanged:: 2.0 - The schema parameter can be a DataType or a datatype string after 2.0. If it's not a - StructType, it will be wrapped into a StructType and each record will also be wrapped - into a tuple. + The ``schema`` parameter can be a :class:`pyspark.sql.types.DataType` or a + :class:`pyspark.sql.types.StringType` after 2.0. + If it's not a :class:`pyspark.sql.types.StructType`, it will be wrapped into a + :class:`pyspark.sql.types.StructType` and each record will also be wrapped into a tuple. >>> l = [('Alice', 1)] >>> sqlContext.createDataFrame(l).collect() diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 6aff938354955..f19bd3a57a4e0 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -196,7 +196,7 @@ def writeStream(self): @property @since(1.3) def schema(self): - """Returns the schema of this :class:`DataFrame` as a :class:`types.StructType`. + """Returns the schema of this :class:`DataFrame` as a :class:`pyspark.sql.types.StructType`. >>> df.schema StructType(List(StructField(age,IntegerType,true),StructField(name,StringType,true))) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 92d709ee40e1f..e422363ec1f55 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -142,7 +142,7 @@ def _(): _binary_mathfunctions = { 'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' + 'polar coordinates (r, theta).', - 'hypot': 'Computes `sqrt(a^2 + b^2)` without intermediate overflow or underflow.', + 'hypot': 'Computes ``sqrt(a^2 + b^2)`` without intermediate overflow or underflow.', 'pow': 'Returns the value of the first argument raised to the power of the second argument.', } @@ -958,7 +958,8 @@ def months_between(date1, date2): @since(1.5) def to_date(col): """ - Converts the column of StringType or TimestampType into DateType. + Converts the column of :class:`pyspark.sql.types.StringType` or + :class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType`. >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_date(df.t).alias('date')).collect() @@ -1074,18 +1075,18 @@ def window(timeColumn, windowDuration, slideDuration=None, startTime=None): [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in the order of months are not supported. - The time column must be of TimestampType. + The time column must be of :class:`pyspark.sql.types.TimestampType`. Durations are provided as strings, e.g. '1 second', '1 day 12 hours', '2 minutes'. Valid interval strings are 'week', 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. - If the `slideDuration` is not provided, the windows will be tumbling windows. + If the ``slideDuration`` is not provided, the windows will be tumbling windows. The startTime is the offset with respect to 1970-01-01 00:00:00 UTC with which to start window intervals. For example, in order to have hourly tumbling windows that start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide `startTime` as `15 minutes`. The output column will be a struct called 'window' by default with the nested columns 'start' - and 'end', where 'start' and 'end' will be of `TimestampType`. + and 'end', where 'start' and 'end' will be of :class:`pyspark.sql.types.TimestampType`. >>> df = spark.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val") >>> w = df.groupBy(window("date", "5 seconds")).agg(sum("val").alias("sum")) @@ -1367,7 +1368,7 @@ def locate(substr, str, pos=1): could not be found in str. :param substr: a string - :param str: a Column of StringType + :param str: a Column of :class:`pyspark.sql.types.StringType` :param pos: start position (zero based) >>> df = spark.createDataFrame([('abcd',)], ['s',]) @@ -1506,8 +1507,9 @@ def bin(col): @ignore_unicode_prefix @since(1.5) def hex(col): - """Computes hex value of the given column, which could be StringType, - BinaryType, IntegerType or LongType. + """Computes hex value of the given column, which could be :class:`pyspark.sql.types.StringType`, + :class:`pyspark.sql.types.BinaryType`, :class:`pyspark.sql.types.IntegerType` or + :class:`pyspark.sql.types.LongType`. >>> spark.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() [Row(hex(a)=u'414243', hex(b)=u'3')] @@ -1781,6 +1783,9 @@ def udf(f, returnType=StringType()): duplicate invocations may be eliminated or the function may even be invoked more times than it is present in the query. + :param f: python function + :param returnType: a :class:`pyspark.sql.types.DataType` object + >>> from pyspark.sql.types import IntegerType >>> slen = udf(lambda s: len(s), IntegerType()) >>> df.select(slen(df.name).alias('slen')).collect() diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index f7c354f51330c..4020bb3fa45b0 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -96,7 +96,7 @@ def schema(self, schema): By specifying the schema here, the underlying data source can skip the schema inference step, and thus speed up data loading. - :param schema: a StructType object + :param schema: a :class:`pyspark.sql.types.StructType` object """ if not isinstance(schema, StructType): raise TypeError("schema should be StructType") @@ -125,7 +125,7 @@ def load(self, path=None, format=None, schema=None, **options): :param path: optional string or a list of string for file-system backed data sources. :param format: optional string for format of the data source. Default to 'parquet'. - :param schema: optional :class:`StructType` for the input schema. + :param schema: optional :class:`pyspark.sql.types.StructType` for the input schema. :param options: all other string options >>> df = spark.read.load('python/test_support/sql/parquet_partitioned', opt1=True, @@ -166,7 +166,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param path: string represents path to the JSON dataset, or RDD of Strings storing JSON objects. - :param schema: an optional :class:`StructType` for the input schema. + :param schema: an optional :class:`pyspark.sql.types.StructType` for the input schema. :param primitivesAsString: infers all primitive values as a string type. If None is set, it uses the default value, ``false``. :param prefersDecimal: infers all floating-point values as a decimal type. If the values @@ -294,7 +294,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non ``inferSchema`` option or specify the schema explicitly using ``schema``. :param path: string, or list of strings, for input path(s). - :param schema: an optional :class:`StructType` for the input schema. + :param schema: an optional :class:`pyspark.sql.types.StructType` for the input schema. :param sep: sets the single character as a separator for each field and value. If None is set, it uses the default value, ``,``. :param encoding: decodes the CSV files by the given encoding type. If None is set, diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 594f9375f7678..10bd89b03fe33 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -47,7 +47,7 @@ def toDF(self, schema=None, sampleRatio=None): This is a shorthand for ``spark.createDataFrame(rdd, schema, sampleRatio)`` - :param schema: a StructType or list of names of columns + :param schema: a :class:`pyspark.sql.types.StructType` or list of names of columns :param samplingRatio: the sample ratio of rows used for inferring :return: a DataFrame @@ -274,9 +274,9 @@ def udf(self): @since(2.0) def range(self, start, end=None, step=1, numPartitions=None): """ - Create a :class:`DataFrame` with single LongType column named `id`, - containing elements in a range from `start` to `end` (exclusive) with - step value `step`. + Create a :class:`DataFrame` with single :class:`pyspark.sql.types.LongType` column named + ``id``, containing elements in a range from ``start`` to ``end`` (exclusive) with + step value ``step``. :param start: the start value :param end: the end value (exclusive) @@ -307,7 +307,7 @@ def _inferSchemaFromList(self, data): Infer schema from list of Row or tuple. :param data: list of Row or tuple - :return: StructType + :return: :class:`pyspark.sql.types.StructType` """ if not data: raise ValueError("can not infer schema from empty dataset") @@ -326,7 +326,7 @@ def _inferSchema(self, rdd, samplingRatio=None): :param rdd: an RDD of Row or tuple :param samplingRatio: sampling ratio, or no sampling (default) - :return: StructType + :return: :class:`pyspark.sql.types.StructType` """ first = rdd.first() if not first: @@ -414,28 +414,33 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): from ``data``, which should be an RDD of :class:`Row`, or :class:`namedtuple`, or :class:`dict`. - When ``schema`` is :class:`DataType` or datatype string, it must match the real data, or - exception will be thrown at runtime. If the given schema is not StructType, it will be - wrapped into a StructType as its only field, and the field name will be "value", each record - will also be wrapped into a tuple, which can be converted to row later. + When ``schema`` is :class:`pyspark.sql.types.DataType` or + :class:`pyspark.sql.types.StringType`, it must match the + real data, or an exception will be thrown at runtime. If the given schema is not + :class:`pyspark.sql.types.StructType`, it will be wrapped into a + :class:`pyspark.sql.types.StructType` as its only field, and the field name will be "value", + each record will also be wrapped into a tuple, which can be converted to row later. If schema inference is needed, ``samplingRatio`` is used to determined the ratio of rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``. :param data: an RDD of any kind of SQL data representation(e.g. row, tuple, int, boolean, etc.), or :class:`list`, or :class:`pandas.DataFrame`. - :param schema: a :class:`DataType` or a datatype string or a list of column names, default - is None. The data type string format equals to `DataType.simpleString`, except that - top level struct type can omit the `struct<>` and atomic types use `typeName()` as - their format, e.g. use `byte` instead of `tinyint` for ByteType. We can also use `int` - as a short name for IntegerType. + :param schema: a :class:`pyspark.sql.types.DataType` or a + :class:`pyspark.sql.types.StringType` or a list of + column names, default is ``None``. The data type string format equals to + :class:`pyspark.sql.types.DataType.simpleString`, except that top level struct type can + omit the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use + ``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`. We can also use + ``int`` as a short name for ``IntegerType``. :param samplingRatio: the sample ratio of rows used for inferring :return: :class:`DataFrame` .. versionchanged:: 2.0 - The schema parameter can be a DataType or a datatype string after 2.0. If it's not a - StructType, it will be wrapped into a StructType and each record will also be wrapped - into a tuple. + The ``schema`` parameter can be a :class:`pyspark.sql.types.DataType` or a + :class:`pyspark.sql.types.StringType` after 2.0. If it's not a + :class:`pyspark.sql.types.StructType`, it will be wrapped into a + :class:`pyspark.sql.types.StructType` and each record will also be wrapped into a tuple. >>> l = [('Alice', 1)] >>> spark.createDataFrame(l).collect() diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 8bac347e13084..a364555003027 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -269,7 +269,7 @@ def schema(self, schema): .. note:: Experimental. - :param schema: a StructType object + :param schema: a :class:`pyspark.sql.types.StructType` object >>> s = spark.readStream.schema(sdf_schema) """ @@ -310,7 +310,7 @@ def load(self, path=None, format=None, schema=None, **options): :param path: optional string for file-system backed data sources. :param format: optional string for format of the data source. Default to 'parquet'. - :param schema: optional :class:`StructType` for the input schema. + :param schema: optional :class:`pyspark.sql.types.StructType` for the input schema. :param options: all other string options >>> json_sdf = spark.readStream.format("json")\ @@ -349,7 +349,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param path: string represents path to the JSON dataset, or RDD of Strings storing JSON objects. - :param schema: an optional :class:`StructType` for the input schema. + :param schema: an optional :class:`pyspark.sql.types.StructType` for the input schema. :param primitivesAsString: infers all primitive values as a string type. If None is set, it uses the default value, ``false``. :param prefersDecimal: infers all floating-point values as a decimal type. If the values @@ -461,7 +461,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non .. note:: Experimental. :param path: string, or list of strings, for input path(s). - :param schema: an optional :class:`StructType` for the input schema. + :param schema: an optional :class:`pyspark.sql.types.StructType` for the input schema. :param sep: sets the single character as a separator for each field and value. If None is set, it uses the default value, ``,``. :param encoding: decodes the CSV files by the given encoding type. If None is set, diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index eea80684e2dfc..1ca4bbc379b42 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -786,9 +786,10 @@ def _parse_struct_fields_string(s): def _parse_datatype_string(s): """ Parses the given data type string to a :class:`DataType`. The data type string format equals - to `DataType.simpleString`, except that top level struct type can omit the `struct<>` and - atomic types use `typeName()` as their format, e.g. use `byte` instead of `tinyint` for - ByteType. We can also use `int` as a short name for IntegerType. + to :class:`DataType.simpleString`, except that top level struct type can omit + the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use ``byte`` instead + of ``tinyint`` for :class:`ByteType`. We can also use ``int`` as a short name + for :class:`IntegerType`. >>> _parse_datatype_string("int ") IntegerType From ed03d0a690c9a7920a21c858df7f42f9a41f28d7 Mon Sep 17 00:00:00 2001 From: Wesley Tang Date: Fri, 29 Jul 2016 04:26:05 -0700 Subject: [PATCH 1057/1470] =?UTF-8?q?[SPARK-16664][SQL]=20Fix=20persist=20?= =?UTF-8?q?call=20on=20Data=20frames=20with=20more=20than=20200=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? f12f11e578169b47e3f8b18b299948c0670ba585 introduced this bug, missed foreach as map ## How was this patch tested? Test added Author: Wesley Tang Closes #14324 from breakdawn/master. (cherry picked from commit d1d5069aa3744d46abd3889abab5f15e9067382a) Signed-off-by: Sean Owen --- .../sql/execution/columnar/GenerateColumnAccessor.scala | 4 ++-- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ .../execution/columnar/InMemoryColumnarQuerySuite.scala | 3 ++- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 7a14879b8b9df..96bd338f092e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -127,7 +127,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera val groupedAccessorsItr = initializeAccessors.grouped(numberOfStatementsThreshold) val groupedExtractorsItr = extractors.grouped(numberOfStatementsThreshold) var groupedAccessorsLength = 0 - groupedAccessorsItr.zipWithIndex.map { case (body, i) => + groupedAccessorsItr.zipWithIndex.foreach { case (body, i) => groupedAccessorsLength += 1 val funcName = s"accessors$i" val funcCode = s""" @@ -137,7 +137,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera """.stripMargin ctx.addNewFunction(funcName, funcCode) } - groupedExtractorsItr.zipWithIndex.map { case (body, i) => + groupedExtractorsItr.zipWithIndex.foreach { case (body, i) => val funcName = s"extractors$i" val funcCode = s""" |private void $funcName() { 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 5151532ed2e1f..55edbe2d44686 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 @@ -1550,4 +1550,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(joined, Row("x", null, null)) checkAnswer(joined.filter($"new".isNull), Row("x", null, null)) } + + test("SPARK-16664: persist with more than 200 columns") { + val size = 201L + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(Seq.range(0, size)))) + val schemas = List.range(0, size).map(a => StructField("name" + a, LongType, true)) + val df = spark.createDataFrame(rdd, StructType(schemas), false) + assert(df.persist.take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index af3ed14c122d2..937839644ad5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -227,7 +227,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val columnTypes1 = List.fill(length1)(IntegerType) val columnarIterator1 = GenerateColumnAccessor.generate(columnTypes1) - val length2 = 10000 + // SPARK-16664: the limit of janino is 8117 + val length2 = 8117 val columnTypes2 = List.fill(length2)(IntegerType) val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) } From efad4aa1468867b36cffb1e8c91f9731c48eca81 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 29 Jul 2016 04:40:20 -0700 Subject: [PATCH 1058/1470] [SPARK-16750][ML] Fix GaussianMixture training failed due to feature column type mistake ## What changes were proposed in this pull request? ML ```GaussianMixture``` training failed due to feature column type mistake. The feature column type should be ```ml.linalg.VectorUDT``` but got ```mllib.linalg.VectorUDT``` by mistake. See [SPARK-16750](https://issues.apache.org/jira/browse/SPARK-16750) for how to reproduce this bug. Why the unit tests did not complain this errors? Because some estimators/transformers missed calling ```transformSchema(dataset.schema)``` firstly during ```fit``` or ```transform```. I will also add this function to all estimators/transformers who missed in this PR. ## How was this patch tested? No new tests, should pass existing ones. Author: Yanbo Liang Closes #14378 from yanboliang/spark-16750. (cherry picked from commit 0557a45452f6e73877e5ec972110825ce8f3fbc5) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/clustering/BisectingKMeans.scala | 2 ++ .../org/apache/spark/ml/clustering/GaussianMixture.scala | 8 +++++--- .../scala/org/apache/spark/ml/clustering/KMeans.scala | 2 ++ .../scala/org/apache/spark/ml/feature/Interaction.scala | 1 + .../scala/org/apache/spark/ml/feature/MinMaxScaler.scala | 1 + .../org/apache/spark/ml/feature/QuantileDiscretizer.scala | 3 ++- .../main/scala/org/apache/spark/ml/feature/RFormula.scala | 1 + .../org/apache/spark/ml/feature/SQLTransformer.scala | 1 + .../spark/ml/regression/AFTSurvivalRegression.scala | 4 ++-- .../apache/spark/ml/regression/IsotonicRegression.scala | 3 ++- 10 files changed, 19 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index afb1080b9b7d5..a97bd0fb16fd7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -99,6 +99,7 @@ class BisectingKMeansModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val predictUDF = udf((vector: Vector) => predict(vector)) dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) } @@ -222,6 +223,7 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): BisectingKMeansModel = { + transformSchema(dataset.schema, logging = true) val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => OldVectors.fromML(point) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 81749055c7613..69f060ad7711e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -30,7 +30,7 @@ import org.apache.spark.ml.stat.distribution.MultivariateGaussian import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{GaussianMixture => MLlibGM} import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, - Vector => OldVector, Vectors => OldVectors, VectorUDT => OldVectorUDT} + Vector => OldVector, Vectors => OldVectors} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, udf} @@ -61,9 +61,9 @@ private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter w * @return output schema */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(featuresCol), new OldVectorUDT) + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType) - SchemaUtils.appendColumn(schema, $(probabilityCol), new OldVectorUDT) + SchemaUtils.appendColumn(schema, $(probabilityCol), new VectorUDT) } } @@ -95,6 +95,7 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val predUDF = udf((vector: Vector) => predict(vector)) val probUDF = udf((vector: Vector) => predictProbability(vector)) dataset.withColumn($(predictionCol), predUDF(col($(featuresCol)))) @@ -317,6 +318,7 @@ class GaussianMixture @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): GaussianMixtureModel = { + transformSchema(dataset.schema, logging = true) val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => OldVectors.fromML(point) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 9fb7d6a9a21ae..6c46be719674b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -120,6 +120,7 @@ class KMeansModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val predictUDF = udf((vector: Vector) => predict(vector)) dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) } @@ -304,6 +305,7 @@ class KMeans @Since("1.5.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): KMeansModel = { + transformSchema(dataset.schema, logging = true) val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => OldVectors.fromML(point) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index 7b11f86279b9d..96d0bdee9e2b9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -68,6 +68,7 @@ class Interaction @Since("1.6.0") (@Since("1.6.0") override val uid: String) ext @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val inputFeatures = $(inputCols).map(c => dataset.schema(c)) val featureEncoders = getFeatureEncoders(inputFeatures) val featureAttrs = getFeatureAttrs(inputFeatures) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 9ed8d83324cf6..068f11a2a573a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -170,6 +170,7 @@ class MinMaxScalerModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val originalRange = (originalMax.asBreeze - originalMin.asBreeze).toArray val minArray = originalMin.toArray diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 9a636bd8a5e46..558a7bbf0a2df 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -97,7 +97,7 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(inputCol)) val inputFields = schema.fields require(inputFields.forall(_.name != $(outputCol)), s"Output column ${$(outputCol)} already exists.") @@ -108,6 +108,7 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui @Since("2.0.0") override def fit(dataset: Dataset[_]): Bucketizer = { + transformSchema(dataset.schema, logging = true) val splits = dataset.stat.approxQuantile($(inputCol), (0.0 to 1.0 by 1.0/$(numBuckets)).toArray, $(relativeError)) splits(0) = Double.NegativeInfinity diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index c95dacfce8cfa..2ee899bcca564 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -112,6 +112,7 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) @Since("2.0.0") override def fit(dataset: Dataset[_]): RFormulaModel = { + transformSchema(dataset.schema, logging = true) require(isDefined(formula), "Formula must be defined first.") val parsedFormula = RFormulaParser.parse($(formula)) val resolvedFormula = parsedFormula.resolve(dataset.schema) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 289037640fd45..259be2679ce19 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -63,6 +63,7 @@ class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val tableName = Identifiable.randomUID(uid) dataset.createOrReplaceTempView(tableName) val realStatement = $(statement).replace(tableIdentifier, tableName) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 700a92cc261be..3ebaba6368837 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -196,7 +196,7 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S @Since("2.0.0") override def fit(dataset: Dataset[_]): AFTSurvivalRegressionModel = { - validateAndTransformSchema(dataset.schema, fitting = true) + transformSchema(dataset.schema, logging = true) val instances = extractAFTPoints(dataset) val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) @@ -332,7 +332,7 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema) + transformSchema(dataset.schema, logging = true) val predictUDF = udf { features: Vector => predict(features) } val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} if (hasQuantilesCol) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 35396446edc16..cd7b4f2a9c56e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -164,7 +164,7 @@ class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: Stri @Since("2.0.0") override def fit(dataset: Dataset[_]): IsotonicRegressionModel = { - validateAndTransformSchema(dataset.schema, fitting = true) + transformSchema(dataset.schema, logging = true) // Extract columns from data. If dataset is persisted, do not persist oldDataset. val instances = extractWeightedLabeledPoints(dataset) val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE @@ -234,6 +234,7 @@ class IsotonicRegressionModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val predict = dataset.schema($(featuresCol)).dataType match { case DoubleType => udf { feature: Double => oldModel.predict(feature) } From 268bf144004952385e4573a11d981b3440f31f5d Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Fri, 29 Jul 2016 04:43:01 -0700 Subject: [PATCH 1059/1470] [SPARK-16751] Upgrade derby to 10.12.1.1 Version of derby upgraded based on important security info at VersionEye. Test scope added so we don't include it in our final package anyway. NB: I think this should be backported to all previous releases as it is a security problem https://www.versioneye.com/java/org.apache.derby:derby/10.11.1.1 The CVE number is 2015-1832. I also suggest we add a SECURITY tag for JIRAs Existing tests with the change making sure that we see no new failures. I checked derby 10.12.x and not derby 10.11.x is downloaded to our ~/.m2 folder. I then used dev/make-distribution.sh and checked the dist/jars folder for Spark 2.0: no derby jar is present. I don't know if this would also remove it from the assembly jar in our 1.x branches. Author: Adam Roberts Closes #14379 from a-roberts/patch-4. (cherry picked from commit 04a2c072d94874f3f7ae9dd94c026e8826a75ccd) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- pom.xml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 96001eade028b..e1d4051e7e766 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -46,7 +46,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.11.1.1.jar +derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 9f3d9ad97a9f7..78d93334fff99 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.11.1.1.jar +derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 3df292ee9956e..abc644da38065 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -48,7 +48,7 @@ curator-recipes-2.4.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.11.1.1.jar +derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar guava-14.0.1.jar guice-3.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 9540f5856bce0..10ecd482b03e8 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.11.1.1.jar +derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index b5c3de75a9c8f..e84689ffd89a7 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -52,7 +52,7 @@ curator-recipes-2.6.0.jar datanucleus-api-jdo-3.2.6.jar datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar -derby-10.11.1.1.jar +derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar gson-2.2.4.jar guava-14.0.1.jar diff --git a/pom.xml b/pom.xml index 9f3d7f003584c..910f1a8959b90 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ 1.2.1.spark2 1.2.1 - 10.11.1.1 + 10.12.1.1 1.7.0 1.6.0 9.2.16.v20160414 From a32531a72cc2b6a0ff95a7a73b256ffc5d9eff60 Mon Sep 17 00:00:00 2001 From: Sun Dapeng Date: Fri, 29 Jul 2016 06:01:23 -0700 Subject: [PATCH 1060/1470] [SPARK-16761][DOC][ML] Fix doc link in docs/ml-guide.md ## What changes were proposed in this pull request? Fix the link at http://spark.apache.org/docs/latest/ml-guide.html. ## How was this patch tested? None Author: Sun Dapeng Closes #14386 from sundapeng/doclink. (cherry picked from commit 2c15323ad026da64caa68787c5d103a8595f63a0) Signed-off-by: Sean Owen --- docs/ml-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 5abec63b7ab45..4607ad3ba681a 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -33,7 +33,7 @@ The primary Machine Learning API for Spark is now the [DataFrame](sql-programmin * DataFrames provide a more user-friendly API than RDDs. The many benefits of DataFrames include Spark Datasources, SQL/DataFrame queries, Tungsten and Catalyst optimizations, and uniform APIs across languages. * The DataFrame-based API for MLlib provides a uniform API across ML algorithms and across multiple languages. -* DataFrames facilitate practical ML Pipelines, particularly feature transformations. See the [Pipelines guide](ml-pipeline.md) for details. +* DataFrames facilitate practical ML Pipelines, particularly feature transformations. See the [Pipelines guide](ml-pipeline.html) for details. # Dependencies From 7d87fc9649b141a1888b89363a8e311690d0fb56 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 29 Jul 2016 19:59:35 -0700 Subject: [PATCH 1061/1470] [SPARK-16748][SQL] SparkExceptions during planning should not wrapped in TreeNodeException ## What changes were proposed in this pull request? We do not want SparkExceptions from job failures in the planning phase to create TreeNodeException. Hence do not wrap SparkException in TreeNodeException. ## How was this patch tested? New unit test Author: Tathagata Das Closes #14395 from tdas/SPARK-16748. (cherry picked from commit bbc247548ac6faeca15afc05c266cee37ef13416) Signed-off-by: Yin Huai --- .../org/apache/spark/sql/catalyst/errors/package.scala | 8 +++++++- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 +++++++++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala index 0420b4b5387c0..0d45f371fa0cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.catalyst +import scala.util.control.NonFatal + import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.SparkException /** * Functions for attaching and retrieving trees that are associated with errors. @@ -47,7 +50,10 @@ package object errors { */ def attachTree[TreeType <: TreeNode[_], A](tree: TreeType, msg: String = "")(f: => A): A = { try f catch { - case e: Exception => throw new TreeNodeException(tree, msg, e) + // SPARK-16748: We do not want SparkExceptions from job failures in the planning phase + // to create TreeNodeException. Hence, wrap exception only if it is not SparkException. + case NonFatal(e) if !e.isInstanceOf[SparkException] => + throw new TreeNodeException(tree, msg, e) } } } 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 d9659012fac5c..b4614e6b6eda2 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,7 +20,7 @@ package org.apache.spark.sql import java.math.MathContext import java.sql.Timestamp -import org.apache.spark.AccumulatorSuite +import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.plans.logical.Aggregate @@ -1339,6 +1339,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) } + testQuietly( + "SPARK-16748: SparkExceptions during planning should not wrapped in TreeNodeException") { + intercept[SparkException] { + val df = spark.range(0, 5).map(x => (1 / x).toString).toDF("a").orderBy("a") + df.queryExecution.toRdd // force physical planning, but not execution of the plan + } + } + test("Test to check we can use Long.MinValue") { checkAnswer( sql(s"SELECT ${Long.MinValue} FROM testData ORDER BY key LIMIT 1"), Row(Long.MinValue) From 26da5a7fc37ac961e7b4d8f423e8e58aefb5c2bc Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Sat, 30 Jul 2016 08:08:33 -0700 Subject: [PATCH 1062/1470] [SPARK-16800][EXAMPLES][ML] Fix Java examples that fail to run due to exception ## What changes were proposed in this pull request? Some Java examples are using mllib.linalg.Vectors instead of ml.linalg.Vectors and causes an exception when run. Also there are some Java examples that incorrectly specify data types in the schema, also causing an exception. ## How was this patch tested? Ran corrected examples locally Author: Bryan Cutler Closes #14405 from BryanCutler/java-examples-ml.Vectors-fix-SPARK-16800. (cherry picked from commit a6290e51e402e8434d6207d553db1f551e714fde) Signed-off-by: Sean Owen --- .../ml/JavaAFTSurvivalRegressionExample.java | 8 +++- .../examples/ml/JavaBinarizerExample.java | 2 +- .../examples/ml/JavaChiSqSelectorExample.java | 4 +- .../spark/examples/ml/JavaDCTExample.java | 4 +- .../JavaEstimatorTransformerParamExample.java | 43 +++++++++++-------- ...LinearRegressionWithElasticNetExample.java | 2 +- .../examples/ml/JavaOneHotEncoderExample.java | 2 +- .../spark/examples/ml/JavaPCAExample.java | 4 +- .../ml/JavaPolynomialExpansionExample.java | 4 +- .../spark/examples/ml/JavaTfIdfExample.java | 8 ++-- .../ml/JavaVectorAssemblerExample.java | 4 +- .../examples/ml/JavaVectorSlicerExample.java | 2 +- 12 files changed, 49 insertions(+), 38 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java index b0115756cf45f..3f034588c9527 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java @@ -23,12 +23,16 @@ import org.apache.spark.ml.regression.AFTSurvivalRegression; import org.apache.spark.ml.regression.AFTSurvivalRegressionModel; -import org.apache.spark.mllib.linalg.*; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.types.*; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; // $example off$ /** diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java index 5f964aca92096..a954dbd20c12f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -47,7 +47,7 @@ public static void main(String[] args) { RowFactory.create(2, 0.2) ); StructType schema = new StructType(new StructField[]{ - new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) }); Dataset continuousDataFrame = spark.createDataFrame(data, schema); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java index f8f2fb14be1f1..fcf90d8d18748 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java @@ -25,8 +25,8 @@ import java.util.List; import org.apache.spark.ml.feature.ChiSqSelector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.DataTypes; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java index eee92c77a8c58..66ce23b49d361 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java @@ -25,8 +25,8 @@ import java.util.List; import org.apache.spark.ml.feature.DCT; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.Metadata; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java index 889f5785dfd8b..9e07a0c2f899a 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java @@ -19,16 +19,20 @@ // $example on$ import java.util.Arrays; -// $example off$ +import java.util.List; -// $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; // $example off$ import org.apache.spark.sql.SparkSession; @@ -44,15 +48,17 @@ public static void main(String[] args) { // $example on$ // Prepare training data. - // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans into - // DataFrames, where it uses the bean metadata to infer the schema. - Dataset training = spark.createDataFrame( - Arrays.asList( - new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)) - ), LabeledPoint.class); + List dataTraining = Arrays.asList( + RowFactory.create(1.0, Vectors.dense(0.0, 1.1, 0.1)), + RowFactory.create(0.0, Vectors.dense(2.0, 1.0, -1.0)), + RowFactory.create(0.0, Vectors.dense(2.0, 1.3, 1.0)), + RowFactory.create(1.0, Vectors.dense(0.0, 1.2, -0.5)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset training = spark.createDataFrame(dataTraining, schema); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -87,11 +93,12 @@ public static void main(String[] args) { System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); // Prepare test documents. - Dataset test = spark.createDataFrame(Arrays.asList( - new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), - new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), - new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)) - ), LabeledPoint.class); + List dataTest = Arrays.asList( + RowFactory.create(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + RowFactory.create(0.0, Vectors.dense(3.0, 2.0, -0.1)), + RowFactory.create(1.0, Vectors.dense(0.0, 2.2, -1.5)) + ); + Dataset test = spark.createDataFrame(dataTest, schema); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java index dcd209e28e2b8..a561b6d39ba83 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java @@ -21,7 +21,7 @@ import org.apache.spark.ml.regression.LinearRegression; import org.apache.spark.ml.regression.LinearRegressionModel; import org.apache.spark.ml.regression.LinearRegressionTrainingSummary; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java index 5d29e54549213..a15e5f84a1871 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -53,7 +53,7 @@ public static void main(String[] args) { ); StructType schema = new StructType(new StructField[]{ - new StructField("id", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), new StructField("category", DataTypes.StringType, false, Metadata.empty()) }); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java index ffa979ee013ad..d597a9a2ed0b7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java @@ -25,8 +25,8 @@ import org.apache.spark.ml.feature.PCA; import org.apache.spark.ml.feature.PCAModel; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java index 7afcd0e50cd95..67180df65c721 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -24,8 +24,8 @@ import java.util.List; import org.apache.spark.ml.feature.PolynomialExpansion; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index 6e0753959efd6..800e42c949cbe 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -25,7 +25,7 @@ import org.apache.spark.ml.feature.IDF; import org.apache.spark.ml.feature.IDFModel; import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -45,9 +45,9 @@ public static void main(String[] args) { // $example on$ List data = Arrays.asList( - RowFactory.create(0, "Hi I heard about Spark"), - RowFactory.create(0, "I wish Java could use case classes"), - RowFactory.create(1, "Logistic regression models are neat") + RowFactory.create(0.0, "Hi I heard about Spark"), + RowFactory.create(0.0, "I wish Java could use case classes"), + RowFactory.create(1.0, "Logistic regression models are neat") ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java index 41f1d8750ac40..9bb0f93d3a6a1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java @@ -23,8 +23,8 @@ import java.util.Arrays; import org.apache.spark.ml.feature.VectorAssembler; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java index 24959c0e10f2b..19b8bc83be6e1 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java @@ -28,7 +28,7 @@ import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; import org.apache.spark.ml.feature.VectorSlicer; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.ml.linalg.Vectors; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; From 75dd78130d29154a3147490c57bce6883c992469 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Jul 2016 23:05:03 -0700 Subject: [PATCH 1063/1470] [SPARK-16812] Open up SparkILoop.getAddedJars ## What changes were proposed in this pull request? This patch makes SparkILoop.getAddedJars a public developer API. It is a useful function to get the list of jars added. ## How was this patch tested? N/A - this is a simple visibility change. Author: Reynold Xin Closes #14417 from rxin/SPARK-16812. (cherry picked from commit 7c27d075c39ebaf3e762284e2536fe7be0e3da87) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 16f330a320a4b..e017aa42a4c18 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1059,7 +1059,8 @@ class SparkILoop( @deprecated("Use `process` instead", "2.9.0") private def main(settings: Settings): Unit = process(settings) - private[repl] def getAddedJars(): Array[String] = { + @DeveloperApi + def getAddedJars(): Array[String] = { val conf = new SparkConf().setMaster(getMaster()) val envJars = sys.env.get("ADD_JARS") if (envJars.isDefined) { From d357ca3023c84e472927380bed65b1cee33c4e03 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 31 Jul 2016 16:31:06 +0800 Subject: [PATCH 1064/1470] [SPARK-16813][SQL] Remove private[sql] and private[spark] from catalyst package The catalyst package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime. This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.catalyst. N/A - just visibility changes. Author: Reynold Xin Closes #14418 from rxin/SPARK-16813. (cherry picked from commit 064d91ff7342002414d3274694a8e2e37f154986) Signed-off-by: Reynold Xin --- .../sql/catalyst/CatalystTypeConverters.scala | 4 ++-- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- .../spark/sql/catalyst/analysis/Analyzer.scala | 4 ++-- .../spark/sql/catalyst/analysis/TypeCoercion.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 6 +++--- .../spark/sql/catalyst/encoders/package.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 2 +- .../expressions/MonotonicallyIncreasingID.scala | 2 +- .../catalyst/expressions/SparkPartitionID.scala | 2 +- .../expressions/aggregate/interfaces.scala | 14 +++++++------- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../catalyst/expressions/complexTypeCreator.scala | 4 ++-- .../expressions/complexTypeExtractors.scala | 2 +- .../spark/sql/catalyst/expressions/misc.scala | 2 +- .../sql/catalyst/expressions/predicates.scala | 4 ++-- .../spark/sql/catalyst/expressions/rows.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 6 +++--- .../catalyst/util/AbstractScalaRowIterator.scala | 2 +- 18 files changed, 32 insertions(+), 32 deletions(-) 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 9cc7b2ac79205..f542f5cf40506 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 @@ -382,7 +382,7 @@ object CatalystTypeConverters { * 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 createToCatalystConverter(dataType: DataType): Any => Any = { if (isPrimitive(dataType)) { // Although the `else` branch here is capable of handling inbound conversion of primitives, // we add some special-case handling for those types here. The motivation for this relates to @@ -409,7 +409,7 @@ object CatalystTypeConverters { * 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 = { + def createToScalaConverter(dataType: DataType): Any => Any = { if (isPrimitive(dataType)) { identity } else { 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 8affb033d8287..dd36468583b1b 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 @@ -720,7 +720,7 @@ object ScalaReflection extends ScalaReflection { /** * Whether the fields of the given type is defined entirely by its constructor parameters. */ - private[sql] def definedByConstructorParams(tpe: Type): Boolean = { + def definedByConstructorParams(tpe: Type): Boolean = { tpe <:< localTypeOf[Product] || tpe <:< localTypeOf[DefinedByConstructorParams] } 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 2efa997ff22d2..660f523698e7f 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 @@ -246,7 +246,7 @@ class Analyzer( }.isDefined } - private[sql] def hasGroupingFunction(e: Expression): Boolean = { + private[analysis] def hasGroupingFunction(e: Expression): Boolean = { e.collectFirst { case g: Grouping => g case g: GroupingID => g @@ -1412,7 +1412,7 @@ class Analyzer( * Construct the output attributes for a [[Generator]], given a list of names. If the list of * names is empty names are assigned from field names in generator. */ - private[sql] def makeGeneratorOutput( + private[analysis] def makeGeneratorOutput( generator: Generator, names: Seq[String]): Seq[Attribute] = { val elementAttrs = generator.elementSchema.toAttributes diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 9a040f8644fb5..8503b8dcf81a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -63,7 +63,7 @@ object TypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: - private[sql] val numericPrecedence = + val numericPrecedence = IndexedSeq( ByteType, ShortType, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 3a2e574a1d9aa..977426a221fcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -746,7 +746,7 @@ class SessionCatalog( * * This performs reflection to decide what type of [[Expression]] to return in the builder. */ - private[sql] def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = { + def makeFunctionBuilder(name: String, functionClassName: String): FunctionBuilder = { // TODO: at least support UDAFs here throw new UnsupportedOperationException("Use sqlContext.udf.register(...) instead.") } @@ -790,7 +790,7 @@ class SessionCatalog( /** * Look up the [[ExpressionInfo]] associated with the specified function, assuming it exists. */ - private[spark] def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = synchronized { + def lookupFunctionInfo(name: FunctionIdentifier): ExpressionInfo = synchronized { // TODO: just make function registry take in FunctionIdentifier instead of duplicating this val database = name.database.orElse(Some(currentDb)).map(formatDatabaseName) val qualifiedName = name.copy(database = database) @@ -902,7 +902,7 @@ class SessionCatalog( * * This is mainly used for tests. */ - private[sql] def reset(): Unit = synchronized { + def reset(): Unit = synchronized { setCurrentDatabase(DEFAULT_DATABASE) listDatabases().filter(_ != DEFAULT_DATABASE).foreach { db => dropDatabase(db, ignoreIfNotExists = false, cascade = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala index 03708fb7afd44..59f7969e56144 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala @@ -26,7 +26,7 @@ package object encoders { * references from a specific schema.) This requirement allows us to preserve whether a given * object type is being bound by name or by ordinal when doing resolution. */ - private[sql] def encoderFor[A : Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match { + def encoderFor[A : Encoder]: ExpressionEncoder[A] = implicitly[Encoder[A]] match { case e: ExpressionEncoder[A] => e.assertUnresolved() e diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 1f37b68846ae4..7abbbe257d830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -526,7 +526,7 @@ abstract class BinaryOperator extends BinaryExpression with ExpectsInputTypes { } -private[sql] object BinaryOperator { +object BinaryOperator { def unapply(e: BinaryOperator): Option[(Expression, Expression)] = Some((e.left, e.right)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index 75c6bb2d84dfb..5b4922e0cf2b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.types.{DataType, LongType} represent the record number within each partition. The assumption is that the data frame has less than 1 billion partitions, and each partition has less than 8 billion records.""", extended = "> SELECT _FUNC_();\n 0") -private[sql] case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterministic { +case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterministic { /** * Record ID within each partition. By being transient, count's value is reset to 0 every time diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala index 71af59a7a8529..1f675d5b07270 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.{DataType, IntegerType} @ExpressionDescription( usage = "_FUNC_() - Returns the current partition id of the Spark task", extended = "> SELECT _FUNC_();\n 0") -private[sql] case class SparkPartitionID() extends LeafExpression with Nondeterministic { +case class SparkPartitionID() extends LeafExpression with Nondeterministic { override def nullable: Boolean = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index 504cea52797de..7a39e568fa289 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -24,14 +24,14 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.types._ /** The mode of an [[AggregateFunction]]. */ -private[sql] sealed trait AggregateMode +sealed trait AggregateMode /** * An [[AggregateFunction]] with [[Partial]] mode is used for partial aggregation. * This function updates the given aggregation buffer with the original input of this * function. When it has processed all input rows, the aggregation buffer is returned. */ -private[sql] case object Partial extends AggregateMode +case object Partial extends AggregateMode /** * An [[AggregateFunction]] with [[PartialMerge]] mode is used to merge aggregation buffers @@ -39,7 +39,7 @@ private[sql] case object Partial extends AggregateMode * This function updates the given aggregation buffer by merging multiple aggregation buffers. * When it has processed all input rows, the aggregation buffer is returned. */ -private[sql] case object PartialMerge extends AggregateMode +case object PartialMerge extends AggregateMode /** * An [[AggregateFunction]] with [[Final]] mode is used to merge aggregation buffers @@ -47,7 +47,7 @@ private[sql] case object PartialMerge extends AggregateMode * This function updates the given aggregation buffer by merging multiple aggregation buffers. * When it has processed all input rows, the final result of this function is returned. */ -private[sql] case object Final extends AggregateMode +case object Final extends AggregateMode /** * An [[AggregateFunction]] with [[Complete]] mode is used to evaluate this function directly @@ -55,13 +55,13 @@ private[sql] case object Final extends AggregateMode * This function updates the given aggregation buffer with the original input of this * function. When it has processed all input rows, the final result of this function is returned. */ -private[sql] case object Complete extends AggregateMode +case object Complete extends AggregateMode /** * A place holder expressions used in code-gen, it does not change the corresponding value * in the row. */ -private[sql] case object NoOp extends Expression with Unevaluable { +case object NoOp extends Expression with Unevaluable { override def nullable: Boolean = true override def dataType: DataType = NullType override def children: Seq[Expression] = Nil @@ -84,7 +84,7 @@ object AggregateExpression { * A container for an [[AggregateFunction]] with its [[AggregateMode]] and a field * (`isDistinct`) indicating if DISTINCT keyword is specified for this function. */ -private[sql] case class AggregateExpression( +case class AggregateExpression( aggregateFunction: AggregateFunction, mode: AggregateMode, isDistinct: Boolean, 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 91ffac0ba2a60..7ff8795d4f05e 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 @@ -125,7 +125,7 @@ abstract class BinaryArithmetic extends BinaryOperator { } } -private[sql] object BinaryArithmetic { +object BinaryArithmetic { def unapply(e: BinaryArithmetic): Option[(Expression, Expression)] = Some((e.left, e.right)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 0ca715f42472a..09e22aaf3e3d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -84,8 +84,8 @@ case class CreateArray(children: Seq[Expression]) extends Expression { @ExpressionDescription( usage = "_FUNC_(key0, value0, key1, value1...) - Creates a map with the given key/value pairs.") case class CreateMap(children: Seq[Expression]) extends Expression { - private[sql] lazy val keys = children.indices.filter(_ % 2 == 0).map(children) - private[sql] lazy val values = children.indices.filter(_ % 2 != 0).map(children) + lazy val keys = children.indices.filter(_ % 2 == 0).map(children) + lazy val values = children.indices.filter(_ % 2 != 0).map(children) override def foldable: Boolean = children.forall(_.foldable) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index 3b4468f55ca73..abb5594bfa7f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -106,7 +106,7 @@ trait ExtractValue extends Expression case class GetStructField(child: Expression, ordinal: Int, name: Option[String] = None) extends UnaryExpression with ExtractValue { - private[sql] lazy val childSchema = child.dataType.asInstanceOf[StructType] + lazy val childSchema = child.dataType.asInstanceOf[StructType] override def dataType: DataType = childSchema(ordinal).dataType override def nullable: Boolean = child.nullable || childSchema(ordinal).nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index d2c94ec1df4d5..369207587d860 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -554,7 +554,7 @@ object XxHash64Function extends InterpretedHashFunction { @ExpressionDescription( usage = "_FUNC_() - Returns the current database.", extended = "> SELECT _FUNC_()") -private[sql] case class CurrentDatabase() extends LeafExpression with Unevaluable { +case class CurrentDatabase() extends LeafExpression with Unevaluable { override def dataType: DataType = StringType override def foldable: Boolean = true override def nullable: Boolean = false 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 a3b098afe5728..100087ed58918 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 @@ -393,13 +393,13 @@ abstract class BinaryComparison extends BinaryOperator with Predicate { } -private[sql] object BinaryComparison { +object BinaryComparison { def unapply(e: BinaryComparison): Option[(Expression, Expression)] = Some((e.left, e.right)) } /** An extractor that matches both standard 3VL equality and null-safe equality. */ -private[sql] object Equality { +object Equality { def unapply(e: BinaryComparison): Option[(Expression, Expression)] = e match { case EqualTo(l, r) => Some((l, r)) case EqualNullSafe(l, r) => Some((l, r)) 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 e036982e70f99..73dceb35ac50e 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 @@ -218,7 +218,7 @@ class GenericRowWithSchema(values: Array[Any], override val schema: StructType) * Note that, while the array is not copied, and thus could technically be mutated after creation, * this is not allowed. */ -class GenericInternalRow(private[sql] val values: Array[Any]) extends BaseGenericInternalRow { +class GenericInternalRow(val values: Array[Any]) extends BaseGenericInternalRow { /** No-arg constructor for serialization. */ protected def this() = this(null) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b31f5aa11c229..eb612c4c12c75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -127,7 +127,7 @@ abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends Binar } } -private[sql] object SetOperation { +object SetOperation { def unapply(p: SetOperation): Option[(LogicalPlan, LogicalPlan)] = Some((p.left, p.right)) } @@ -365,7 +365,7 @@ case class InsertIntoTable( override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty - private[spark] lazy val expectedColumns = { + lazy val expectedColumns = { if (table.output.isEmpty) { None } else { @@ -509,7 +509,7 @@ case class Window( def windowOutputSet: AttributeSet = AttributeSet(windowExpressions.map(_.toAttribute)) } -private[sql] object Expand { +object Expand { /** * Extract attribute set according to the grouping id. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala index 6d35f140cf23f..0c7205b3c6651 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala @@ -23,7 +23,7 @@ package org.apache.spark.sql.catalyst.util * `Row` in order to work around a spurious IntelliJ compiler error. This cannot be an abstract * class because that leads to compilation errors under Scala 2.11. */ -private[spark] class AbstractScalaRowIterator[T] extends Iterator[T] { +class AbstractScalaRowIterator[T] extends Iterator[T] { override def hasNext: Boolean = throw new NotImplementedError override def next(): T = throw new NotImplementedError From c651ff53adefd0c74c84500e929ed37f8ad668d2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 31 Jul 2016 18:21:06 -0700 Subject: [PATCH 1065/1470] [SPARK-16805][SQL] Log timezone when query result does not match ## What changes were proposed in this pull request? It is useful to log the timezone when query result does not match, especially on build machines that have different timezone from AMPLab Jenkins. ## How was this patch tested? This is a test-only change. Author: Reynold Xin Closes #14413 from rxin/SPARK-16805. (cherry picked from commit 579fbcf3bd9717003025caecc0c0b85bcff7ac7f) Signed-off-by: Yin Huai --- sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala | 3 +++ 1 file changed, 3 insertions(+) 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 b15f38c2a71e2..e8480a7001760 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 @@ -394,6 +394,9 @@ object QueryTest { sameRows(expectedAnswer, sparkAnswer, isSorted).map { results => s""" |Results do not match for query: + |Timezone: ${TimeZone.getDefault} + |Timezone Env: ${sys.env("TZ")} + | |${df.queryExecution} |== Results == |$results From 4bdc558989ef4a9490ca42e7330c10136151134b Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 1 Aug 2016 06:55:31 -0700 Subject: [PATCH 1066/1470] [SPARK-16778][SQL][TRIVIAL] Fix deprecation warning with SQLContext ## What changes were proposed in this pull request? Change to non-deprecated constructor for SQLContext. ## How was this patch tested? Existing tests Author: Holden Karau Closes #14406 from holdenk/SPARK-16778-fix-use-of-deprecated-SQLContext-constructor. (cherry picked from commit 1e9b59b73bdb8aacf5a85e0eed29efc6485a3bc3) Signed-off-by: Sean Owen --- .../test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 2cd3f475b6c0d..5d348044515af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -28,7 +28,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { test("propagate from spark conf") { // We create a new context here to avoid order dependence with other tests that might call // clear(). - val newContext = new SQLContext(sparkContext) + val newContext = new SQLContext(SparkSession.builder().sparkContext(sparkContext).getOrCreate()) assert(newContext.getConf("spark.sql.testkey", "false") === "true") } From b49091e10100dfbefeabdd2dfe0b64cdf613a052 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 1 Aug 2016 06:56:52 -0700 Subject: [PATCH 1067/1470] [SPARK-16776][STREAMING] Replace deprecated API in KafkaTestUtils for 0.10.0. ## What changes were proposed in this pull request? This PR replaces the old Kafka API to 0.10.0 ones in `KafkaTestUtils`. The change include: - `Producer` to `KafkaProducer` - Change configurations to equalvant ones. (I referred [here](http://kafka.apache.org/documentation.html#producerconfigs) for 0.10.0 and [here](http://kafka.apache.org/082/documentation.html#producerconfigs ) for old, 0.8.2). This PR will remove the build warning as below: ```scala [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:71: class Producer in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.KafkaProducer instead. [WARNING] private var producer: Producer[String, String] = _ [WARNING] ^ [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:181: class Producer in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.KafkaProducer instead. [WARNING] producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) [WARNING] ^ [WARNING] .../spark/streaming/kafka010/KafkaTestUtils.scala:181: class ProducerConfig in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.ProducerConfig instead. [WARNING] producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) [WARNING] ^ [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:182: class KeyedMessage in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.ProducerRecord instead. [WARNING] producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) [WARNING] ^ [WARNING] four warnings found [WARNING] warning: [options] bootstrap class path not set in conjunction with -source 1.7 [WARNING] 1 warning ``` ## How was this patch tested? Existing tests that use `KafkaTestUtils` should cover this. Author: hyukjinkwon Closes #14416 from HyukjinKwon/SPARK-16776. (cherry picked from commit f93ad4fe7c9728c8dd67a8095de3d39fad21d03f) Signed-off-by: Sean Owen --- .../streaming/kafka010/KafkaTestUtils.scala | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala index 19192e4b95945..ecabe1c365b41 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala @@ -30,10 +30,10 @@ import scala.util.control.NonFatal import kafka.admin.AdminUtils import kafka.api.Request -import kafka.producer.{KeyedMessage, Producer, ProducerConfig} -import kafka.serializer.StringEncoder import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.ZkUtils +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.serialization.StringSerializer import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.apache.spark.SparkConf @@ -68,7 +68,7 @@ private[kafka010] class KafkaTestUtils extends Logging { private var server: KafkaServer = _ // Kafka producer - private var producer: Producer[String, String] = _ + private var producer: KafkaProducer[String, String] = _ // Flag to test whether the system is correctly started private var zkReady = false @@ -178,8 +178,10 @@ private[kafka010] class KafkaTestUtils extends Logging { /** 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 = new KafkaProducer[String, String](producerConfiguration) + messages.foreach { message => + producer.send(new ProducerRecord[String, String](topic, message)) + } producer.close() producer = null } @@ -198,10 +200,12 @@ private[kafka010] class KafkaTestUtils extends Logging { private def producerConfiguration: Properties = { val props = new Properties() - props.put("metadata.broker.list", brokerAddress) - props.put("serializer.class", classOf[StringEncoder].getName) + props.put("bootstrap.servers", brokerAddress) + props.put("value.serializer", classOf[StringSerializer].getName) + // Key serializer is required. + props.put("key.serializer", classOf[StringSerializer].getName) // wait for all in-sync replicas to ack sends - props.put("request.required.acks", "-1") + props.put("acks", "all") props } From 1523bf69a0ef87f36b0b3995ce2d7a33aaff6046 Mon Sep 17 00:00:00 2001 From: eyal farago Date: Mon, 1 Aug 2016 22:43:32 +0800 Subject: [PATCH 1068/1470] [SPARK-16791][SQL] cast struct with timestamp field fails ## What changes were proposed in this pull request? a failing test case + fix to SPARK-16791 (https://issues.apache.org/jira/browse/SPARK-16791) ## How was this patch tested? added a failing test case to CastSuit, then fixed the Cast code and rerun the entire CastSuit Author: eyal farago Author: Eyal Farago Closes #14400 from eyalfa/SPARK-16791_cast_struct_with_timestamp_field_fails. (cherry picked from commit 338a98d65c8efe0c41f39a8dddeab7040dcda125) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/expressions/Cast.scala | 2 +- .../spark/sql/catalyst/expressions/CastSuite.scala | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index c452765af2dd9..70fff51956255 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 @@ -416,7 +416,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } private[this] def cast(from: DataType, to: DataType): Any => Any = to match { - case dt if dt == child.dataType => identity[Any] + case dt if dt == from => identity[Any] case StringType => castToString(from) case BinaryType => castToBinary(from) case DateType => castToDate(from) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 5ae0527a9c7ae..5c35baacef2fa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -727,6 +727,16 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("cast struct with a timestamp field") { + val originalSchema = new StructType().add("tsField", TimestampType, nullable = false) + // nine out of ten times I'm casting a struct, it's to normalize its fields nullability + val targetSchema = new StructType().add("tsField", TimestampType, nullable = true) + + val inp = Literal.create(InternalRow(0L), originalSchema) + val expected = InternalRow(0L) + checkEvaluation(cast(inp, targetSchema), expected) + } + test("complex casting") { val complex = Literal.create( Row( From 4e73cb8ebdb0dcb1be4dce562bac9214e9905b8e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 1 Aug 2016 13:57:05 -0700 Subject: [PATCH 1069/1470] [SPARK-16774][SQL] Fix use of deprecated timestamp constructor & improve timezone handling ## What changes were proposed in this pull request? Removes the deprecated timestamp constructor and incidentally fixes the use which was using system timezone rather than the one specified when working near DST. This change also causes the roundtrip tests to fail since it now actually uses all the timezones near DST boundaries where it didn't before. Note: this is only a partial the solution, longer term we should follow up with https://issues.apache.org/jira/browse/SPARK-16788 to avoid this problem & simplify our timezone handling code. ## How was this patch tested? New tests for two timezones added so even if user timezone happens to coincided with one, the other tests should still fail. Important note: this (temporarily) disables the round trip tests until we can fix the issue more thoroughly. Author: Holden Karau Closes #14398 from holdenk/SPARK-16774-fix-use-of-deprecated-timestamp-constructor. (cherry picked from commit ab1e761f9691b41385e2ed2202c5a671c63c963d) Signed-off-by: Sean Owen --- .../spark/sql/catalyst/util/DateTimeUtils.scala | 14 ++++++++------ .../sql/catalyst/util/DateTimeUtilsSuite.scala | 3 ++- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index df480a1d65bc9..0b643a5b84268 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -852,8 +852,10 @@ object DateTimeUtils { /** * Lookup the offset for given millis seconds since 1970-01-01 00:00:00 in given timezone. + * TODO: Improve handling of normalization differences. + * TODO: Replace with JSR-310 or similar system - see SPARK-16788 */ - private def getOffsetFromLocalMillis(millisLocal: Long, tz: TimeZone): Long = { + private[sql] def getOffsetFromLocalMillis(millisLocal: Long, tz: TimeZone): Long = { var guess = tz.getRawOffset // the actual offset should be calculated based on milliseconds in UTC val offset = tz.getOffset(millisLocal - guess) @@ -875,11 +877,11 @@ object DateTimeUtils { val hh = seconds / 3600 val mm = seconds / 60 % 60 val ss = seconds % 60 - val nano = millisOfDay % 1000 * 1000000 - - // create a Timestamp to get the unix timestamp (in UTC) - val timestamp = new Timestamp(year - 1900, month - 1, day, hh, mm, ss, nano) - guess = (millisLocal - timestamp.getTime).toInt + val ms = millisOfDay % 1000 + val calendar = Calendar.getInstance(tz) + calendar.set(year, month - 1, day, hh, mm, ss) + calendar.set(Calendar.MILLISECOND, ms) + guess = (millisLocal - calendar.getTimeInMillis()).toInt } } guess diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 059a5b7d07cde..4f516d006458e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -551,7 +551,8 @@ class DateTimeUtilsSuite extends SparkFunSuite { val skipped = skipped_days.getOrElse(tz.getID, Int.MinValue) (-20000 to 20000).foreach { d => if (d != skipped) { - assert(millisToDays(daysToMillis(d)) === d) + assert(millisToDays(daysToMillis(d)) === d, + s"Round trip of ${d} did not work in tz ${tz}") } } } From 1813bbd9bf7cb9afd29e1385f0dc52e8fcc4f132 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 1 Aug 2016 14:41:22 -0700 Subject: [PATCH 1070/1470] [SPARK-15869][STREAMING] Fix a potential NPE in StreamingJobProgressListener.getBatchUIData ## What changes were proposed in this pull request? Moved `asScala` to a `map` to avoid NPE. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu Closes #14443 from zsxwing/SPARK-15869. (cherry picked from commit 03d46aafe561b03e25f4e25cf01e631c18dd827c) Signed-off-by: Shixiong Zhu --- .../spark/streaming/ui/StreamingJobProgressListener.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c086df47d9835..61f852a0d31a7 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 @@ -259,7 +259,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) // We use an Iterable rather than explicitly converting to a seq so that updates // will propagate val outputOpIdToSparkJobIds: Iterable[OutputOpIdAndSparkJobId] = - Option(batchTimeToOutputOpIdSparkJobIdPair.get(batchTime).asScala) + Option(batchTimeToOutputOpIdSparkJobIdPair.get(batchTime)).map(_.asScala) .getOrElse(Seq.empty) _batchUIData.outputOpIdSparkJobIdPairs = outputOpIdToSparkJobIds } From 5fbf5f93ee5aa4d1aca0fa0c8fb769a085dd7b93 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 1 Aug 2016 19:46:20 -0700 Subject: [PATCH 1071/1470] [SPARK-16818] Exchange reuse incorrectly reuses scans over different sets of partitions https://github.com/apache/spark/pull/14425 rebased for branch-2.0 Author: Eric Liang Closes #14427 from ericl/spark-16818-br-2. --- .../datasources/FileSourceStrategy.scala | 2 ++ .../datasources/FileSourceStrategySuite.scala | 35 ++++++++++++++++++- 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 13a86bfb3896f..8af9562330e81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -202,7 +202,9 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { partitions } + // These metadata values make scan plans uniquely identifiable for equality checking. val meta = Map( + "PartitionFilters" -> partitionKeyFilters.mkString("[", ", ", "]"), "Format" -> files.fileFormat.toString, "ReadSchema" -> prunedDataSchema.simpleString, PUSHED_FILTERS -> pushedDownFilters.mkString("[", ", ", "]"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 8d8a18fa9332b..7a24f214aa101 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet, PredicateHelper} import org.apache.spark.sql.catalyst.util -import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.{DataSourceScanExec, SparkPlan} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ @@ -407,6 +407,39 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } } + test("[SPARK-16818] partition pruned file scans implement sameResult correctly") { + withTempPath { path => + val tempDir = path.getCanonicalPath + spark.range(100) + .selectExpr("id", "id as b") + .write + .partitionBy("id") + .parquet(tempDir) + val df = spark.read.parquet(tempDir) + def getPlan(df: DataFrame): SparkPlan = { + df.queryExecution.executedPlan + } + assert(getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 2")))) + assert(!getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 3")))) + } + } + + test("[SPARK-16818] exchange reuse respects differences in partition pruning") { + spark.conf.set("spark.sql.exchange.reuse", true) + withTempPath { path => + val tempDir = path.getCanonicalPath + spark.range(10) + .selectExpr("id % 2 as a", "id % 3 as b", "id as c") + .write + .partitionBy("a") + .parquet(tempDir) + val df = spark.read.parquet(tempDir) + val df1 = df.where("a = 0").groupBy("b").agg("c" -> "sum") + val df2 = df.where("a = 1").groupBy("b").agg("c" -> "sum") + checkAnswer(df1.join(df2, "b"), Row(0, 6, 12) :: Row(1, 4, 8) :: Row(2, 10, 5) :: Nil) + } + } + // Helpers for checking the arguments passed to the FileFormat. protected val checkPartitionSchema = From 9d9956e8f8abd41a603fde2347384428b7ec715c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 2 Aug 2016 15:02:40 +0800 Subject: [PATCH 1072/1470] [SPARK-16734][EXAMPLES][SQL] Revise examples of all language bindings ## What changes were proposed in this pull request? This PR makes various minor updates to examples of all language bindings to make sure they are consistent with each other. Some typos and missing parts (JDBC example in Scala/Java/Python) are also fixed. ## How was this patch tested? Manually tested. Author: Cheng Lian Closes #14368 from liancheng/revise-examples. (cherry picked from commit 10e1c0e638774f5d746771b6dd251de2480f94eb) Signed-off-by: Wenchen Fan --- docs/sql-programming-guide.md | 56 +++------ .../sql/JavaSQLDataSourceExample.java | 23 +++- .../examples/sql/JavaSparkSQLExample.java | 2 +- examples/src/main/python/sql/basic.py | 2 +- examples/src/main/python/sql/datasource.py | 32 +++-- examples/src/main/python/sql/hive.py | 2 +- examples/src/main/r/RSparkSQLExample.R | 113 ++++++++++-------- .../examples/sql/SQLDataSourceExample.scala | 22 +++- .../spark/examples/sql/SparkSQLExample.scala | 2 +- 9 files changed, 137 insertions(+), 117 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 33b170e50a00c..82b03a2b0edb2 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -132,7 +132,7 @@ from a Hive table, or from [Spark data sources](#data-sources). As an example, the following creates a DataFrame based on the content of a JSON file: -{% include_example create_DataFrames r/RSparkSQLExample.R %} +{% include_example create_df r/RSparkSQLExample.R %}
    @@ -180,7 +180,7 @@ In addition to simple column references and expressions, DataFrames also have a
    -{% include_example dataframe_operations r/RSparkSQLExample.R %} +{% include_example untyped_ops r/RSparkSQLExample.R %} For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). @@ -214,7 +214,7 @@ The `sql` function on a `SparkSession` enables applications to run SQL queries p
    The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. -{% include_example sql_query r/RSparkSQLExample.R %} +{% include_example run_sql r/RSparkSQLExample.R %}
    @@ -377,7 +377,7 @@ In the simplest form, the default data source (`parquet` unless otherwise config
    -{% include_example source_parquet r/RSparkSQLExample.R %} +{% include_example generic_load_save_functions r/RSparkSQLExample.R %}
    @@ -400,13 +400,11 @@ using this syntax.
    - {% include_example manual_load_options python/sql/datasource.py %}
    -
    - -{% include_example source_json r/RSparkSQLExample.R %} +
    +{% include_example manual_load_options r/RSparkSQLExample.R %}
    @@ -425,13 +423,11 @@ file directly with SQL.
    - {% include_example direct_sql python/sql/datasource.py %}
    - -{% include_example direct_query r/RSparkSQLExample.R %} +{% include_example direct_sql r/RSparkSQLExample.R %}
    @@ -523,7 +519,7 @@ Using the data from the above example:
    -{% include_example load_programmatically r/RSparkSQLExample.R %} +{% include_example basic_parquet_example r/RSparkSQLExample.R %}
    @@ -827,7 +823,7 @@ Note that the file that is offered as _a json file_ is not a typical JSON file. line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. -{% include_example load_json_file r/RSparkSQLExample.R %} +{% include_example json_dataset r/RSparkSQLExample.R %}
    @@ -913,7 +909,7 @@ You may need to grant write privilege to the user who starts the spark applicati When working with Hive one must instantiate `SparkSession` with Hive support. This adds support for finding tables in the MetaStore and writing queries using HiveQL. -{% include_example hive_table r/RSparkSQLExample.R %} +{% include_example spark_hive r/RSparkSQLExample.R %}
    @@ -1055,43 +1051,19 @@ the Data Sources API. The following options are supported:
    - -{% highlight scala %} -val jdbcDF = spark.read.format("jdbc").options( - Map("url" -> "jdbc:postgresql:dbserver", - "dbtable" -> "schema.tablename")).load() -{% endhighlight %} - +{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    - -{% highlight java %} - -Map options = new HashMap<>(); -options.put("url", "jdbc:postgresql:dbserver"); -options.put("dbtable", "schema.tablename"); - -Dataset jdbcDF = spark.read().format("jdbc"). options(options).load(); -{% endhighlight %} - - +{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    - -{% highlight python %} - -df = spark.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbtable='schema.tablename').load() - -{% endhighlight %} - +{% include_example jdbc_dataset python/sql/datasource.py %}
    - -{% include_example jdbc r/RSparkSQLExample.R %} - +{% include_example jdbc_dataset r/RSparkSQLExample.R %}
    diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index ec02c8bbb8efe..52e3b62b79dd2 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -25,7 +25,6 @@ // $example on:basic_parquet_example$ import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Encoders; -// import org.apache.spark.sql.Encoders; // $example on:schema_merging$ // $example on:json_dataset$ import org.apache.spark.sql.Dataset; @@ -92,7 +91,7 @@ public void setCube(int cube) { public static void main(String[] args) { SparkSession spark = SparkSession .builder() - .appName("Java Spark SQL Data Sources Example") + .appName("Java Spark SQL data sources example") .config("spark.some.config.option", "some-value") .getOrCreate(); @@ -100,6 +99,7 @@ public static void main(String[] args) { runBasicParquetExample(spark); runParquetSchemaMergingExample(spark); runJsonDatasetExample(spark); + runJdbcDatasetExample(spark); spark.stop(); } @@ -183,10 +183,10 @@ private static void runParquetSchemaMergingExample(SparkSession spark) { // The final schema consists of all 3 columns in the Parquet files together // with the partitioning column appeared in the partition directory paths // root - // |-- value: int (nullable = true) - // |-- square: int (nullable = true) - // |-- cube: int (nullable = true) - // |-- key : int (nullable = true) + // |-- value: int (nullable = true) + // |-- square: int (nullable = true) + // |-- cube: int (nullable = true) + // |-- key: int (nullable = true) // $example off:schema_merging$ } @@ -216,4 +216,15 @@ private static void runJsonDatasetExample(SparkSession spark) { // $example off:json_dataset$ } + private static void runJdbcDatasetExample(SparkSession spark) { + // $example on:jdbc_dataset$ + Dataset jdbcDF = spark.read() + .format("jdbc") + .option("url", "jdbc:postgresql:dbserver") + .option("dbtable", "schema.tablename") + .option("user", "username") + .option("password", "password") + .load(); + // $example off:jdbc_dataset$ + } } diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java index afc18078d471e..cff9032f52b5a 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java @@ -88,7 +88,7 @@ public static void main(String[] args) { // $example on:init_session$ SparkSession spark = SparkSession .builder() - .appName("Java Spark SQL Example") + .appName("Java Spark SQL basic example") .config("spark.some.config.option", "some-value") .getOrCreate(); // $example off:init_session$ diff --git a/examples/src/main/python/sql/basic.py b/examples/src/main/python/sql/basic.py index 74f5009581e43..fdc017aed97c1 100644 --- a/examples/src/main/python/sql/basic.py +++ b/examples/src/main/python/sql/basic.py @@ -182,7 +182,7 @@ def programmatic_schema_example(spark): # $example on:init_session$ spark = SparkSession \ .builder \ - .appName("PythonSQL") \ + .appName("Python Spark SQL basic example") \ .config("spark.some.config.option", "some-value") \ .getOrCreate() # $example off:init_session$ diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 0bdc3d66ff984..b36c901d2b403 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -92,14 +92,14 @@ def parquet_schema_merging_example(spark): # The final schema consists of all 3 columns in the Parquet files together # with the partitioning column appeared in the partition directory paths. # root - # |-- double: long (nullable = true) - # |-- single: long (nullable = true) - # |-- triple: long (nullable = true) - # |-- key: integer (nullable = true) + # |-- double: long (nullable = true) + # |-- single: long (nullable = true) + # |-- triple: long (nullable = true) + # |-- key: integer (nullable = true) # $example off:schema_merging$ -def json_dataset_examplg(spark): +def json_dataset_example(spark): # $example on:json_dataset$ # spark is from the previous example. sc = spark.sparkContext @@ -112,8 +112,8 @@ def json_dataset_examplg(spark): # The inferred schema can be visualized using the printSchema() method peopleDF.printSchema() # root - # |-- age: long (nullable = true) - # |-- name: string (nullable = true) + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) # Creates a temporary view using the DataFrame peopleDF.createOrReplaceTempView("people") @@ -140,15 +140,29 @@ def json_dataset_examplg(spark): # +---------------+----+ # $example off:json_dataset$ + +def jdbc_dataset_example(spark): + # $example on:jdbc_dataset$ + jdbcDF = spark.read \ + .format("jdbc") \ + .option("url", "jdbc:postgresql:dbserver") \ + .option("dbtable", "schema.tablename") \ + .option("user", "username") \ + .option("password", "password") \ + .load() + # $example off:jdbc_dataset$ + + if __name__ == "__main__": spark = SparkSession \ .builder \ - .appName("PythonSQL") \ + .appName("Python Spark SQL data source example") \ .getOrCreate() basic_datasource_example(spark) parquet_example(spark) parquet_schema_merging_example(spark) - json_dataset_examplg(spark) + json_dataset_example(spark) + jdbc_dataset_example(spark) spark.stop() diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index d9ce5cef1f2b0..9b2a2c4e6a16b 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -38,7 +38,7 @@ spark = SparkSession \ .builder \ - .appName("PythonSQL") \ + .appName("Python Spark SQL Hive integration example") \ .config("spark.sql.warehouse.dir", warehouse_location) \ .enableHiveSupport() \ .getOrCreate() diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index 33e88e15fd47a..de489e1bda2c3 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -18,31 +18,43 @@ library(SparkR) # $example on:init_session$ -sparkR.session(appName = "MyApp", sparkConfig = list(spark.executor.memory = "1g")) +sparkR.session(appName = "MyApp", sparkConfig = list(spark.some.config.option = "some-value")) # $example off:init_session$ -# $example on:create_DataFrames$ +# $example on:create_df$ df <- read.json("examples/src/main/resources/people.json") # Displays the content of the DataFrame head(df) +## age name +## 1 NA Michael +## 2 30 Andy +## 3 19 Justin # Another method to print the first few rows and optionally truncate the printing of long values showDF(df) -# $example off:create_DataFrames$ +## +----+-------+ +## | age| name| +## +----+-------+ +## |null|Michael| +## | 30| Andy| +## | 19| Justin| +## +----+-------+ +## $example off:create_df$ -# $example on:dataframe_operations$ +# $example on:untyped_ops$ # Create the DataFrame df <- read.json("examples/src/main/resources/people.json") # Show the content of the DataFrame head(df) -## age name -## null Michael -## 30 Andy -## 19 Justin +## age name +## 1 NA Michael +## 2 30 Andy +## 3 19 Justin + # Print the schema in a tree format printSchema(df) @@ -52,58 +64,58 @@ printSchema(df) # Select only the "name" column head(select(df, "name")) -## name -## Michael -## Andy -## Justin +## name +## 1 Michael +## 2 Andy +## 3 Justin # Select everybody, but increment the age by 1 head(select(df, df$name, df$age + 1)) -## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 +## name (age + 1.0) +## 1 Michael NA +## 2 Andy 31 +## 3 Justin 20 # Select people older than 21 head(where(df, df$age > 21)) -## age name -## 30 Andy +## age name +## 1 30 Andy # Count people by age head(count(groupBy(df, "age"))) -## age count -## null 1 -## 19 1 -## 30 1 -# $example off:dataframe_operations$ +## age count +## 1 19 1 +## 2 NA 1 +## 3 30 1 +# $example off:untyped_ops$ # Register this DataFrame as a table. createOrReplaceTempView(df, "table") -# $example on:sql_query$ +# $example on:run_sql$ df <- sql("SELECT * FROM table") -# $example off:sql_query$ +# $example off:run_sql$ -# $example on:source_parquet$ +# $example on:generic_load_save_functions$ df <- read.df("examples/src/main/resources/users.parquet") write.df(select(df, "name", "favorite_color"), "namesAndFavColors.parquet") -# $example off:source_parquet$ +# $example off:generic_load_save_functions$ -# $example on:source_json$ +# $example on:manual_load_options$ df <- read.df("examples/src/main/resources/people.json", "json") namesAndAges <- select(df, "name", "age") write.df(namesAndAges, "namesAndAges.parquet", "parquet") -# $example off:source_json$ +# $example off:manual_load_options$ -# $example on:direct_query$ +# $example on:direct_sql$ df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") -# $example off:direct_query$ +# $example off:direct_sql$ -# $example on:load_programmatically$ +# $example on:basic_parquet_example$ df <- read.df("examples/src/main/resources/people.json", "json") # SparkDataFrame can be saved as Parquet files, maintaining the schema information. @@ -117,7 +129,7 @@ parquetFile <- read.parquet("people.parquet") createOrReplaceTempView(parquetFile, "parquetFile") teenagers <- sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") head(teenagers) -## name +## name ## 1 Justin # We can also run custom R-UDFs on Spark DataFrames. Here we prefix all the names with "Name:" @@ -129,7 +141,7 @@ for (teenName in collect(teenNames)$name) { ## Name: Michael ## Name: Andy ## Name: Justin -# $example off:load_programmatically$ +# $example off:basic_parquet_example$ # $example on:schema_merging$ @@ -146,18 +158,17 @@ write.df(df2, "data/test_table/key=2", "parquet", "overwrite") # Read the partitioned table df3 <- read.df("data/test_table", "parquet", mergeSchema = "true") printSchema(df3) - # The final schema consists of all 3 columns in the Parquet files together -# with the partitioning column appeared in the partition directory paths. -# root -# |-- single: double (nullable = true) -# |-- double: double (nullable = true) -# |-- triple: double (nullable = true) -# |-- key : int (nullable = true) +# with the partitioning column appeared in the partition directory paths +## root +## |-- single: double (nullable = true) +## |-- double: double (nullable = true) +## |-- triple: double (nullable = true) +## |-- key: integer (nullable = true) # $example off:schema_merging$ -# $example on:load_json_file$ +# $example on:json_dataset$ # A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. path <- "examples/src/main/resources/people.json" @@ -166,9 +177,9 @@ people <- read.json(path) # The inferred schema can be visualized using the printSchema() method. printSchema(people) -# root -# |-- age: long (nullable = true) -# |-- name: string (nullable = true) +## root +## |-- age: long (nullable = true) +## |-- name: string (nullable = true) # Register this DataFrame as a table. createOrReplaceTempView(people, "people") @@ -176,12 +187,12 @@ createOrReplaceTempView(people, "people") # SQL statements can be run by using the sql methods. teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") head(teenagers) -## name +## name ## 1 Justin -# $example off:load_json_file$ +# $example off:json_dataset$ -# $example on:hive_table$ +# $example on:spark_hive$ # enableHiveSupport defaults to TRUE sparkR.session(enableHiveSupport = TRUE) sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") @@ -189,12 +200,12 @@ sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src # Queries can be expressed in HiveQL. results <- collect(sql("FROM src SELECT key, value")) -# $example off:hive_table$ +# $example off:spark_hive$ -# $example on:jdbc$ +# $example on:jdbc_dataset$ df <- read.jdbc("jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password") -# $example off:jdbc$ +# $example off:jdbc_dataset$ # Stop the SparkSession now sparkR.session.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 0caba12af0bd0..dc3915a4882b0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -25,7 +25,7 @@ object SQLDataSourceExample { def main(args: Array[String]) { val spark = SparkSession .builder() - .appName("Spark SQL Data Soures Example") + .appName("Spark SQL data sources example") .config("spark.some.config.option", "some-value") .getOrCreate() @@ -33,6 +33,7 @@ object SQLDataSourceExample { runBasicParquetExample(spark) runParquetSchemaMergingExample(spark) runJsonDatasetExample(spark) + runJdbcDatasetExample(spark) spark.stop() } @@ -99,10 +100,10 @@ object SQLDataSourceExample { // The final schema consists of all 3 columns in the Parquet files together // with the partitioning column appeared in the partition directory paths // root - // |-- value: int (nullable = true) - // |-- square: int (nullable = true) - // |-- cube: int (nullable = true) - // |-- key : int (nullable = true) + // |-- value: int (nullable = true) + // |-- square: int (nullable = true) + // |-- cube: int (nullable = true) + // |-- key: int (nullable = true) // $example off:schema_merging$ } @@ -145,4 +146,15 @@ object SQLDataSourceExample { // $example off:json_dataset$ } + private def runJdbcDatasetExample(spark: SparkSession): Unit = { + // $example on:jdbc_dataset$ + val jdbcDF = spark.read + .format("jdbc") + .option("url", "jdbc:postgresql:dbserver") + .option("dbtable", "schema.tablename") + .option("user", "username") + .option("password", "password") + .load() + // $example off:jdbc_dataset$ + } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index 952c074d03457..5cd437d017f6f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -42,7 +42,7 @@ object SparkSQLExample { // $example on:init_session$ val spark = SparkSession .builder() - .appName("Spark SQL Example") + .appName("Spark SQL basic example") .config("spark.some.config.option", "some-value") .getOrCreate() From c5516ab60da860320693bbc245818cb6d8a282c8 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Tue, 2 Aug 2016 07:28:46 -0700 Subject: [PATCH 1073/1470] [SPARK-16558][EXAMPLES][MLLIB] examples/mllib/LDAExample should use MLVector instead of MLlib Vector ## What changes were proposed in this pull request? mllib.LDAExample uses ML pipeline and MLlib LDA algorithm. The former transforms original data into MLVector format, while the latter uses MLlibVector format. ## How was this patch tested? Test manually. Author: Xusen Yin Closes #14212 from yinxusen/SPARK-16558. (cherry picked from commit dd8514fa2059a695143073f852b1abee50e522bd) Signed-off-by: Yanbo Liang --- .../scala/org/apache/spark/examples/mllib/LDAExample.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 3fbf8e03339e8..ef67841f0cbee 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -24,8 +24,9 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel, RegexTokenizer, StopWordsRemover} +import org.apache.spark.ml.linalg.{Vector => MLVector} import org.apache.spark.mllib.clustering.{DistributedLDAModel, EMLDAOptimizer, LDA, OnlineLDAOptimizer} -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession} @@ -225,7 +226,7 @@ object LDAExample { val documents = model.transform(df) .select("features") .rdd - .map { case Row(features: Vector) => features } + .map { case Row(features: MLVector) => Vectors.fromML(features) } .zipWithIndex() .map(_.swap) From fc18e259a311c0f1dffe47edef0e42182afca8e9 Mon Sep 17 00:00:00 2001 From: Maciej Brynski Date: Tue, 2 Aug 2016 08:07:08 -0700 Subject: [PATCH 1074/1470] [SPARK-15541] Casting ConcurrentHashMap to ConcurrentMap (master branch) ## What changes were proposed in this pull request? Casting ConcurrentHashMap to ConcurrentMap allows to run code compiled with Java 8 on Java 7 ## How was this patch tested? Compilation. Existing automatic tests Author: Maciej Brynski Closes #14459 from maver1ck/spark-15541-master. (cherry picked from commit 511dede1118f20a7756f614acb6fc88af52c9de9) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/rpc/netty/Dispatcher.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index d305de2e1340e..a02cf30a5d831 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -17,7 +17,7 @@ package org.apache.spark.rpc.netty -import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ @@ -42,8 +42,10 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging { val inbox = new Inbox(ref, endpoint) } - private val endpoints = new ConcurrentHashMap[String, EndpointData] - private val endpointRefs = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef] + private val endpoints: ConcurrentMap[String, EndpointData] = + new ConcurrentHashMap[String, EndpointData] + private val endpointRefs: ConcurrentMap[RpcEndpoint, RpcEndpointRef] = + new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef] // Track the receivers whose inboxes may contain messages. private val receivers = new LinkedBlockingQueue[EndpointData] From 22f0899bc78e1f2021084c6397a4c05ad6317bae Mon Sep 17 00:00:00 2001 From: Tom Magrino Date: Tue, 2 Aug 2016 09:16:44 -0700 Subject: [PATCH 1075/1470] [SPARK-16837][SQL] TimeWindow incorrectly drops slideDuration in constructors ## What changes were proposed in this pull request? Fix of incorrect arguments (dropping slideDuration and using windowDuration) in constructors for TimeWindow. The JIRA this addresses is here: https://issues.apache.org/jira/browse/SPARK-16837 ## How was this patch tested? Added a test to TimeWindowSuite to check that the results of TimeWindow object apply and TimeWindow class constructor are equivalent. Author: Tom Magrino Closes #14441 from tmagrino/windowing-fix. (cherry picked from commit 1dab63d8d3c59a3d6b4ee8e777810c44849e58b8) Signed-off-by: Sean Owen --- .../spark/sql/catalyst/expressions/TimeWindow.scala | 4 ++-- .../sql/catalyst/expressions/TimeWindowSuite.scala | 12 ++++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 66c4bf29ea4b2..7ff61ee479452 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -45,12 +45,12 @@ case class TimeWindow( slideDuration: Expression, startTime: Expression) = { this(timeColumn, TimeWindow.parseExpression(windowDuration), - TimeWindow.parseExpression(windowDuration), TimeWindow.parseExpression(startTime)) + TimeWindow.parseExpression(slideDuration), TimeWindow.parseExpression(startTime)) } def this(timeColumn: Expression, windowDuration: Expression, slideDuration: Expression) = { this(timeColumn, TimeWindow.parseExpression(windowDuration), - TimeWindow.parseExpression(windowDuration), 0) + TimeWindow.parseExpression(slideDuration), 0) } def this(timeColumn: Expression, windowDuration: Expression) = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index b82cf8d1693e2..d6c8fcf291842 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -108,4 +108,16 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper with Priva TimeWindow.invokePrivate(parseExpression(Rand(123))) } } + + test("SPARK-16837: TimeWindow.apply equivalent to TimeWindow constructor") { + val slideLength = "1 second" + for (windowLength <- Seq("10 second", "1 minute", "2 hours")) { + val applyValue = TimeWindow(Literal(10L), windowLength, slideLength, "0 seconds") + val constructed = new TimeWindow(Literal(10L), + Literal(windowLength), + Literal(slideLength), + Literal("0 seconds")) + assert(applyValue == constructed) + } + } } From ef7927e8e77558f9a18eacc8491b0c28231e2769 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 2 Aug 2016 10:08:18 -0700 Subject: [PATCH 1076/1470] [SPARK-16062] [SPARK-15989] [SQL] Fix two bugs of Python-only UDTs ## What changes were proposed in this pull request? There are two related bugs of Python-only UDTs. Because the test case of second one needs the first fix too. I put them into one PR. If it is not appropriate, please let me know. ### First bug: When MapObjects works on Python-only UDTs `RowEncoder` will use `PythonUserDefinedType.sqlType` for its deserializer expression. If the sql type is `ArrayType`, we will have `MapObjects` working on it. But `MapObjects` doesn't consider `PythonUserDefinedType` as its input data type. It causes error like: import pyspark.sql.group from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT from pyspark.sql.types import * schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT()) df = spark.createDataFrame([(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], schema=schema) df.show() File "/home/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o36.showString. : java.lang.RuntimeException: Error while decoding: scala.MatchError: org.apache.spark.sql.types.PythonUserDefinedTypef4ceede8 (of class org.apache.spark.sql.types.PythonUserDefinedType) ... ### Second bug: When Python-only UDTs is the element type of ArrayType import pyspark.sql.group from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT from pyspark.sql.types import * schema = StructType().add("key", LongType()).add("val", ArrayType(PythonOnlyUDT())) df = spark.createDataFrame([(i % 3, [PythonOnlyPoint(float(i), float(i))]) for i in range(10)], schema=schema) df.show() ## How was this patch tested? PySpark's sql tests. Author: Liang-Chi Hsieh Closes #13778 from viirya/fix-pyudt. (cherry picked from commit 146001a9ffefc7aaedd3d888d68c7a9b80bca545) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 35 +++++++++++++++++++ .../sql/catalyst/encoders/RowEncoder.scala | 9 ++++- .../expressions/objects/objects.scala | 17 +++++++-- 3 files changed, 58 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index a8ca386e1ce31..87dbb5049565f 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -575,6 +575,41 @@ def check_datatype(datatype): _verify_type(PythonOnlyPoint(1.0, 2.0), PythonOnlyUDT()) self.assertRaises(ValueError, lambda: _verify_type([1.0, 2.0], PythonOnlyUDT())) + def test_simple_udt_in_df(self): + schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT()) + df = self.spark.createDataFrame( + [(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], + schema=schema) + df.show() + + def test_nested_udt_in_df(self): + schema = StructType().add("key", LongType()).add("val", ArrayType(PythonOnlyUDT())) + df = self.spark.createDataFrame( + [(i % 3, [PythonOnlyPoint(float(i), float(i))]) for i in range(10)], + schema=schema) + df.collect() + + schema = StructType().add("key", LongType()).add("val", + MapType(LongType(), PythonOnlyUDT())) + df = self.spark.createDataFrame( + [(i % 3, {i % 3: PythonOnlyPoint(float(i + 1), float(i + 1))}) for i in range(10)], + schema=schema) + df.collect() + + def test_complex_nested_udt_in_df(self): + from pyspark.sql.functions import udf + + schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT()) + df = self.spark.createDataFrame( + [(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], + schema=schema) + df.collect() + + gd = df.groupby("key").agg({"val": "collect_list"}) + gd.collect() + udf = udf(lambda k, v: [(k, v[0])], ArrayType(df.schema)) + gd.select(udf(*gd)).collect() + def test_udt_with_none(self): df = self.spark.range(0, 10, 1, 1) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 67fca153b551a..2a6fcd03a26b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -206,6 +206,7 @@ object RowEncoder { case _: ArrayType => ObjectType(classOf[scala.collection.Seq[_]]) case _: MapType => ObjectType(classOf[scala.collection.Map[_, _]]) case _: StructType => ObjectType(classOf[Row]) + case p: PythonUserDefinedType => externalDataTypeFor(p.sqlType) case udt: UserDefinedType[_] => ObjectType(udt.userClass) } @@ -220,9 +221,15 @@ object RowEncoder { CreateExternalRow(fields, schema) } - private def deserializerFor(input: Expression): Expression = input.dataType match { + private def deserializerFor(input: Expression): Expression = { + deserializerFor(input, input.dataType) + } + + private def deserializerFor(input: Expression, dataType: DataType): Expression = dataType match { case dt if ScalaReflection.isNativeType(dt) => input + case p: PythonUserDefinedType => deserializerFor(input, p.sqlType) + case udt: UserDefinedType[_] => val annotation = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]) val udtClass: Class[_] = if (annotation != null) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index ea4dee174e74e..9621db1d38762 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -346,6 +346,13 @@ case class LambdaVariable(value: String, isNull: String, dataType: DataType) ext object MapObjects { private val curId = new java.util.concurrent.atomic.AtomicInteger() + /** + * Construct an instance of MapObjects case class. + * + * @param function The function applied on the collection elements. + * @param inputData An expression that when evaluated returns a collection object. + * @param elementType The data type of elements in the collection. + */ def apply( function: Expression => Expression, inputData: Expression, @@ -433,8 +440,14 @@ case class MapObjects private( case _ => "" } + // The data with PythonUserDefinedType are actually stored with the data type of its sqlType. + // When we want to apply MapObjects on it, we have to use it. + val inputDataType = inputData.dataType match { + case p: PythonUserDefinedType => p.sqlType + case _ => inputData.dataType + } - val (getLength, getLoopVar) = inputData.dataType match { + val (getLength, getLoopVar) = inputDataType match { case ObjectType(cls) if classOf[Seq[_]].isAssignableFrom(cls) => s"${genInputData.value}.size()" -> s"${genInputData.value}.apply($loopIndex)" case ObjectType(cls) if cls.isArray => @@ -448,7 +461,7 @@ case class MapObjects private( s"$seq == null ? $array[$loopIndex] : $seq.apply($loopIndex)" } - val loopNullCheck = inputData.dataType match { + val loopNullCheck = inputDataType match { case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" // The element of primitive array will never be null. case ObjectType(cls) if cls.isArray && cls.getComponentType.isPrimitive => From a937c9ee44e0766194fc8ca4bce2338453112a53 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 2 Aug 2016 10:09:47 -0700 Subject: [PATCH 1077/1470] [SPARK-16836][SQL] Add support for CURRENT_DATE/CURRENT_TIMESTAMP literals ## What changes were proposed in this pull request? In Spark 1.6 (with Hive support) we could use `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions as literals (without adding braces), for example: ```SQL select /* Spark 1.6: */ current_date, /* Spark 1.6 & Spark 2.0: */ current_date() ``` This was accidentally dropped in Spark 2.0. This PR reinstates this functionality. ## How was this patch tested? Added a case to ExpressionParserSuite. Author: Herman van Hovell Closes #14442 from hvanhovell/SPARK-16836. (cherry picked from commit 2330f3ecbbd89c7eaab9cc0d06726aa743b16334) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 5 ++++- .../spark/sql/catalyst/parser/AstBuilder.scala | 13 +++++++++++++ .../sql/catalyst/parser/ExpressionParserSuite.scala | 5 +++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 ++++++++++- 4 files changed, 32 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4c15f9cec6573..de98a871b3358 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -493,6 +493,7 @@ valueExpression primaryExpression : constant #constantDefault + | name=(CURRENT_DATE | CURRENT_TIMESTAMP) #timeFunctionCall | ASTERISK #star | qualifiedName '.' ASTERISK #star | '(' expression (',' expression)+ ')' #rowConstructor @@ -653,7 +654,7 @@ nonReserved | NULL | ORDER | OUTER | TABLE | TRUE | WITH | RLIKE | AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN | UNBOUNDED | WHEN - | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT + | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT | CURRENT_DATE | CURRENT_TIMESTAMP ; SELECT: 'SELECT'; @@ -873,6 +874,8 @@ OPTION: 'OPTION'; ANTI: 'ANTI'; LOCAL: 'LOCAL'; INPATH: 'INPATH'; +CURRENT_DATE: 'CURRENT_DATE'; +CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP'; STRING : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c7420a1c5965d..1a0e7ab32a6c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1022,6 +1022,19 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { } } + /** + * Create a current timestamp/date expression. These are different from regular function because + * they do not require the user to specify braces when calling them. + */ + override def visitTimeFunctionCall(ctx: TimeFunctionCallContext): Expression = withOrigin(ctx) { + ctx.name.getType match { + case SqlBaseParser.CURRENT_DATE => + CurrentDate() + case SqlBaseParser.CURRENT_TIMESTAMP => + CurrentTimestamp() + } + } + /** * Create a function database (optional) and name pair. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index e73592c7afa28..849d96212822c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -502,4 +502,9 @@ class ExpressionParserSuite extends PlanTest { assertEqual("1 - f('o', o(bar))", Literal(1) - 'f.function("o", 'o.function('bar))) intercept("1 - f('o', o(bar)) hello * world", "mismatched input '*'") } + + test("current date/timestamp braceless expressions") { + assertEqual("current_date", CurrentDate()) + assertEqual("current_timestamp", CurrentTimestamp()) + } } 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 b4614e6b6eda2..393b4af285498 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql import java.math.MathContext -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.sql.catalyst.analysis.UnresolvedException @@ -2981,4 +2981,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { data.selectExpr("`part.col1`", "`col.1`")) } } + + test("current_date and current_timestamp literals") { + // NOTE that I am comparing the result of the literal with the result of the function call. + // This is done to prevent the test from failing because we are comparing a result to an out + // dated timestamp (quite likely) or date (very unlikely - but equally annoying). + checkAnswer( + sql("select current_date = current_date(), current_timestamp = current_timestamp()"), + Seq(Row(true, true))) + } } From f190bb83beaafb65c8e6290e9ecaa61ac51e04bb Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Tue, 2 Aug 2016 19:32:35 +0800 Subject: [PATCH 1078/1470] [SPARK-16850][SQL] Improve type checking error message for greatest/least Greatest/least function does not have the most friendly error message for data types. This patch improves the error message to not show the Seq type, and use more human readable data types. Before: ``` org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST (ArrayBuffer(DecimalType(2,1), StringType)).; line 1 pos 7 ``` After: ``` org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST(decimal(2,1), string).; line 1 pos 7 ``` Manually verified the output and also added unit tests to ConditionalExpressionSuite. Author: petermaxlee Closes #14453 from petermaxlee/SPARK-16850. (cherry picked from commit a1ff72e1cce6f22249ccc4905e8cef30075beb2f) Signed-off-by: Reynold Xin --- .../expressions/conditionalExpressions.scala | 4 ++-- .../expressions/ConditionalExpressionSuite.scala | 13 +++++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index e97e08947a500..5f2585fc40b06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -299,7 +299,7 @@ case class Least(children: Seq[Expression]) extends Expression { } else if (children.map(_.dataType).distinct.count(_ != NullType) > 1) { TypeCheckResult.TypeCheckFailure( s"The expressions should all have the same type," + - s" got LEAST (${children.map(_.dataType)}).") + s" got LEAST(${children.map(_.dataType.simpleString).mkString(", ")}).") } else { TypeUtils.checkForOrderingExpr(dataType, "function " + prettyName) } @@ -359,7 +359,7 @@ case class Greatest(children: Seq[Expression]) extends Expression { } else if (children.map(_.dataType).distinct.count(_ != NullType) > 1) { TypeCheckResult.TypeCheckFailure( s"The expressions should all have the same type," + - s" got GREATEST (${children.map(_.dataType)}).") + s" got GREATEST(${children.map(_.dataType.simpleString).mkString(", ")}).") } else { TypeUtils.checkForOrderingExpr(dataType, "function " + prettyName) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala index 3c581ecdaf068..36185b8c637a1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala @@ -21,6 +21,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.types._ @@ -181,6 +182,12 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper Literal(Timestamp.valueOf("2015-07-01 10:00:00")))), Timestamp.valueOf("2015-07-01 08:00:00"), InternalRow.empty) + // Type checking error + assert( + Least(Seq(Literal(1), Literal("1"))).checkInputDataTypes() == + TypeCheckFailure("The expressions should all have the same type, " + + "got LEAST(int, string).")) + DataTypeTestUtils.ordered.foreach { dt => checkConsistencyBetweenInterpretedAndCodegen(Least, dt, 2) } @@ -227,6 +234,12 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper Literal(Timestamp.valueOf("2015-07-01 10:00:00")))), Timestamp.valueOf("2015-07-01 10:00:00"), InternalRow.empty) + // Type checking error + assert( + Greatest(Seq(Literal(1), Literal("1"))).checkInputDataTypes() == + TypeCheckFailure("The expressions should all have the same type, " + + "got GREATEST(int, string).")) + DataTypeTestUtils.ordered.foreach { dt => checkConsistencyBetweenInterpretedAndCodegen(Greatest, dt, 2) } From 063a507fce862d14061b0c0464b7a51a0afde066 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 2 Aug 2016 12:02:11 -0700 Subject: [PATCH 1079/1470] [SPARK-16787] SparkContext.addFile() should not throw if called twice with the same file ## What changes were proposed in this pull request? The behavior of `SparkContext.addFile()` changed slightly with the introduction of the Netty-RPC-based file server, which was introduced in Spark 1.6 (where it was disabled by default) and became the default / only file server in Spark 2.0.0. Prior to 2.0, calling `SparkContext.addFile()` with files that have the same name and identical contents would succeed. This behavior was never explicitly documented but Spark has behaved this way since very early 1.x versions. In 2.0 (or 1.6 with the Netty file server enabled), the second `addFile()` call will fail with a requirement error because NettyStreamManager tries to guard against duplicate file registration. This problem also affects `addJar()` in a more subtle way: the `fileServer.addJar()` call will also fail with an exception but that exception is logged and ignored; I believe that the problematic exception-catching path was mistakenly copied from some old code which was only relevant to very old versions of Spark and YARN mode. I believe that this change of behavior was unintentional, so this patch weakens the `require` check so that adding the same filename at the same path will succeed. At file download time, Spark tasks will fail with exceptions if an executor already has a local copy of a file and that file's contents do not match the contents of the file being downloaded / added. As a result, it's important that we prevent files with the same name and different contents from being served because allowing that can effectively brick an executor by preventing it from successfully launching any new tasks. Before this patch's change, this was prevented by forbidding `addFile()` from being called twice on files with the same name. Because Spark does not defensively copy local files that are passed to `addFile` it is vulnerable to files' contents changing, so I think it's okay to rely on an implicit assumption that these files are intended to be immutable (since if they _are_ mutable then this can lead to either explicit task failures or implicit incorrectness (in case new executors silently get newer copies of the file while old executors continue to use an older version)). To guard against this, I have decided to only update the file addition timestamps on the first call to `addFile()`; duplicate calls will succeed but will not update the timestamp. This behavior is fine as long as we assume files are immutable, which seems reasonable given the behaviors described above. As part of this change, I also improved the thread-safety of the `addedJars` and `addedFiles` maps; this is important because these maps may be concurrently read by a task launching thread and written by a driver thread in case the user's driver code is multi-threaded. ## How was this patch tested? I added regression tests in `SparkContextSuite`. Author: Josh Rosen Closes #14396 from JoshRosen/SPARK-16787. (cherry picked from commit e9fc0b6a8b4ce62cab56d18581f588c67b811f5b) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/SparkContext.scala | 36 ++++++------- .../spark/rpc/netty/NettyStreamManager.scala | 12 +++-- .../org/apache/spark/scheduler/Task.scala | 5 +- .../org/apache/spark/SparkContextSuite.scala | 51 +++++++++++++++++++ 4 files changed, 78 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fe15052b62478..d3e8de31b7f84 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -21,7 +21,7 @@ import java.io._ import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID} -import java.util.concurrent.ConcurrentMap +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} import scala.collection.JavaConverters._ @@ -262,8 +262,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli 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]() - private[spark] val addedJars = HashMap[String, Long]() + private[spark] val addedFiles = new ConcurrentHashMap[String, Long]().asScala + private[spark] val addedJars = new ConcurrentHashMap[String, Long]().asScala // Keeps track of all persisted RDDs private[spark] val persistentRdds = { @@ -1429,14 +1429,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli schemeCorrectedPath } val timestamp = System.currentTimeMillis - addedFiles(key) = timestamp - - // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, - hadoopConfiguration, timestamp, useCache = false) - - logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) - postEnvironmentUpdate() + if (addedFiles.putIfAbsent(key, timestamp).isEmpty) { + logInfo(s"Added file $path at $key with timestamp $timestamp") + // Fetch the file locally so that closures which are run on the driver can still use the + // SparkFiles API to access files. + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, + hadoopConfiguration, timestamp, useCache = false) + postEnvironmentUpdate() + } } /** @@ -1704,12 +1704,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli case exc: FileNotFoundException => logError(s"Jar not found at $path") null - case e: Exception => - // For now just log an error but allow to go through so spark examples work. - // The spark examples don't really need the jar distributed since its also - // the app jar. - logError("Error adding jar (" + e + "), was the --addJars option used?") - null } } // A JAR file which exists locally on every worker node @@ -1720,11 +1714,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } if (key != null) { - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + val timestamp = System.currentTimeMillis + if (addedJars.putIfAbsent(key, timestamp).isEmpty) { + logInfo(s"Added JAR $path at $key with timestamp $timestamp") + postEnvironmentUpdate() + } } } - postEnvironmentUpdate() } /** diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyStreamManager.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyStreamManager.scala index afcb023a99daa..780fadd5bda8e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyStreamManager.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyStreamManager.scala @@ -66,14 +66,18 @@ private[netty] class NettyStreamManager(rpcEnv: NettyRpcEnv) } override def addFile(file: File): String = { - require(files.putIfAbsent(file.getName(), file) == null, - s"File ${file.getName()} already registered.") + val existingPath = files.putIfAbsent(file.getName, file) + require(existingPath == null || existingPath == file, + s"File ${file.getName} was already registered with a different path " + + s"(old path = $existingPath, new path = $file") s"${rpcEnv.address.toSparkURL}/files/${Utils.encodeFileNameToURIRawPath(file.getName())}" } override def addJar(file: File): String = { - require(jars.putIfAbsent(file.getName(), file) == null, - s"JAR ${file.getName()} already registered.") + val existingPath = jars.putIfAbsent(file.getName, file) + require(existingPath == null || existingPath == file, + s"File ${file.getName} was already registered with a different path " + + s"(old path = $existingPath, new path = $file") s"${rpcEnv.address.toSparkURL}/jars/${Utils.encodeFileNameToURIRawPath(file.getName())}" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 15f863b66c6ee..35c4dafe9c19c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -21,6 +21,7 @@ import java.io.{DataInputStream, DataOutputStream} import java.nio.ByteBuffer import java.util.Properties +import scala.collection.mutable import scala.collection.mutable.HashMap import org.apache.spark._ @@ -198,8 +199,8 @@ private[spark] object Task { */ def serializeWithDependencies( task: Task[_], - currentFiles: HashMap[String, Long], - currentJars: HashMap[String, Long], + currentFiles: mutable.Map[String, Long], + currentJars: mutable.Map[String, Long], serializer: SerializerInstance) : ByteBuffer = { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 4fa3cab18184c..f8d143dc610cb 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -216,6 +216,57 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { } } + test("cannot call addFile with different paths that have the same filename") { + val dir = Utils.createTempDir() + try { + val subdir1 = new File(dir, "subdir1") + val subdir2 = new File(dir, "subdir2") + assert(subdir1.mkdir()) + assert(subdir2.mkdir()) + val file1 = new File(subdir1, "file") + val file2 = new File(subdir2, "file") + Files.write("old", file1, StandardCharsets.UTF_8) + Files.write("new", file2, StandardCharsets.UTF_8) + sc = new SparkContext("local-cluster[1,1,1024]", "test") + sc.addFile(file1.getAbsolutePath) + def getAddedFileContents(): String = { + sc.parallelize(Seq(0)).map { _ => + scala.io.Source.fromFile(SparkFiles.get("file")).mkString + }.first() + } + assert(getAddedFileContents() === "old") + intercept[IllegalArgumentException] { + sc.addFile(file2.getAbsolutePath) + } + assert(getAddedFileContents() === "old") + } finally { + Utils.deleteRecursively(dir) + } + } + + // Regression tests for SPARK-16787 + for ( + schedulingMode <- Seq("local-mode", "non-local-mode"); + method <- Seq("addJar", "addFile") + ) { + val jarPath = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar").toString + val master = schedulingMode match { + case "local-mode" => "local" + case "non-local-mode" => "local-cluster[1,1,1024]" + } + test(s"$method can be called twice with same file in $schedulingMode (SPARK-16787)") { + sc = new SparkContext(master, "test") + method match { + case "addJar" => + sc.addJar(jarPath) + sc.addJar(jarPath) + case "addFile" => + sc.addFile(jarPath) + sc.addFile(jarPath) + } + } + } + test("Cancelling job group should not cause SparkContext to shutdown (SPARK-6414)") { try { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) From d9d3504b91ebf4fc9f06551f1b23fada0f4e1b0e Mon Sep 17 00:00:00 2001 From: =^_^= Date: Wed, 3 Aug 2016 04:18:28 -0700 Subject: [PATCH 1080/1470] [SPARK-16831][PYTHON] Fixed bug in CrossValidator.avgMetrics ## What changes were proposed in this pull request? avgMetrics was summed, not averaged, across folds Author: =^_^= Closes #14456 from pkch/pkch-patch-1. (cherry picked from commit 639df046a250873c26446a037cb832ab28cb5272) Signed-off-by: Sean Owen --- python/pyspark/ml/tuning.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 298314d46caf6..e17d13d4d2bd2 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -160,6 +160,8 @@ class CrossValidator(Estimator, ValidatorParams): >>> evaluator = BinaryClassificationEvaluator() >>> cv = CrossValidator(estimator=lr, estimatorParamMaps=grid, evaluator=evaluator) >>> cvModel = cv.fit(dataset) + >>> cvModel.avgMetrics[0] + 0.5 >>> evaluator.evaluate(cvModel.transform(dataset)) 0.8333... @@ -228,7 +230,7 @@ def _fit(self, dataset): model = est.fit(train, epm[j]) # TODO: duplicate evaluator to take extra params from input metric = eva.evaluate(model.transform(validation, epm[j])) - metrics[j] += metric + metrics[j] += metric/nFolds if eva.isLargerBetter(): bestIndex = np.argmax(metrics) From 969313bb20a6695dee0959cabab7e5265f8de311 Mon Sep 17 00:00:00 2001 From: Artur Sukhenko Date: Tue, 2 Aug 2016 16:13:12 -0700 Subject: [PATCH 1081/1470] [SPARK-16796][WEB UI] Visible passwords on Spark environment page ## What changes were proposed in this pull request? Mask spark.ssl.keyPassword, spark.ssl.keyStorePassword, spark.ssl.trustStorePassword in Web UI environment page. (Changes their values to ***** in env. page) ## How was this patch tested? I've built spark, run spark shell and checked that this values have been masked with *****. Also run tests: ./dev/run-tests [info] ScalaTest [info] Run completed in 1 hour, 9 minutes, 5 seconds. [info] Total number of tests run: 2166 [info] Suites: completed 65, aborted 0 [info] Tests: succeeded 2166, failed 0, canceled 0, ignored 590, pending 0 [info] All tests passed. ![mask](https://cloud.githubusercontent.com/assets/15244468/17262154/7641e132-55e2-11e6-8a6c-30ead77c7372.png) Author: Artur Sukhenko Closes #14409 from Devian-ua/maskpass. (cherry picked from commit 3861273771c2631e88e1f37a498c644ad45ac1c0) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/ui/env/EnvironmentPage.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index f0a1174a71d34..22136a6f10743 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -26,11 +26,15 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { private val listener = parent.listener + private def removePass(kv: (String, String)): (String, String) = { + if (kv._1.toLowerCase.contains("password")) (kv._1, "******") else kv + } + def render(request: HttpServletRequest): Seq[Node] = { val runtimeInformationTable = UIUtils.listingTable( propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable( - propertyHeader, propertyRow, listener.sparkProperties, fixedWidth = true) + propertyHeader, propertyRow, listener.sparkProperties.map(removePass), fixedWidth = true) val systemPropertiesTable = UIUtils.listingTable( propertyHeader, propertyRow, listener.systemProperties, fixedWidth = true) val classpathEntriesTable = UIUtils.listingTable( From 2daab33c4ed2cdbe1025252ae10bf19320df9d25 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 3 Aug 2016 11:15:09 -0700 Subject: [PATCH 1082/1470] [SPARK-16714][SPARK-16735][SPARK-16646] array, map, greatest, least's type coercion should handle decimal type ## What changes were proposed in this pull request? Here is a table about the behaviours of `array`/`map` and `greatest`/`least` in Hive, MySQL and Postgres: | |Hive|MySQL|Postgres| |---|---|---|---|---| |`array`/`map`|can find a wider type with decimal type arguments, and will truncate the wider decimal type if necessary|can find a wider type with decimal type arguments, no truncation problem|can find a wider type with decimal type arguments, no truncation problem| |`greatest`/`least`|can find a wider type with decimal type arguments, and truncate if necessary, but can't do string promotion|can find a wider type with decimal type arguments, no truncation problem, but can't do string promotion|can find a wider type with decimal type arguments, no truncation problem, but can't do string promotion| I think these behaviours makes sense and Spark SQL should follow them. This PR fixes `array` and `map` by using `findWiderCommonType` to get the wider type. This PR fixes `greatest` and `least` by add a `findWiderTypeWithoutStringPromotion`, which provides similar semantic of `findWiderCommonType`, but without string promotion. ## How was this patch tested? new tests in `TypeCoersionSuite` Author: Wenchen Fan Author: Yin Huai Closes #14439 from cloud-fan/bug. (cherry picked from commit b55f34370f695de355b72c1518b5f2a45c324af0) Signed-off-by: Yin Huai --- .../sql/catalyst/analysis/TypeCoercion.scala | 47 ++++++++----- .../ExpressionTypeCheckingSuite.scala | 1 - .../catalyst/analysis/TypeCoercionSuite.scala | 67 +++++++++++++++++++ 3 files changed, 97 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 8503b8dcf81a7..021952e7166f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -108,18 +108,6 @@ object TypeCoercion { }) } - /** - * Similar to [[findTightestCommonType]], if can not find the TightestCommonType, try to use - * [[findTightestCommonTypeToString]] to find the TightestCommonType. - */ - private def findTightestCommonTypeAndPromoteToString(types: Seq[DataType]): Option[DataType] = { - types.foldLeft[Option[DataType]](Some(NullType))((r, c) => r match { - case None => None - case Some(d) => - findTightestCommonTypeToString(d, c) - }) - } - /** * Find the tightest common type of a set of types by continuously applying * `findTightestCommonTypeOfTwo` on these types. @@ -157,6 +145,28 @@ object TypeCoercion { }) } + /** + * Similar to [[findWiderCommonType]], but can't promote to string. This is also similar to + * [[findTightestCommonType]], but can handle decimal types. If the wider decimal type exceeds + * system limitation, this rule will truncate the decimal type before return it. + */ + private def findWiderTypeWithoutStringPromotion(types: Seq[DataType]): Option[DataType] = { + types.foldLeft[Option[DataType]](Some(NullType))((r, c) => r match { + case Some(d) => findTightestCommonTypeOfTwo(d, c).orElse((d, c) match { + case (t1: DecimalType, t2: DecimalType) => + Some(DecimalPrecision.widerDecimalType(t1, t2)) + case (t: IntegralType, d: DecimalType) => + Some(DecimalPrecision.widerDecimalType(DecimalType.forType(t), d)) + case (d: DecimalType, t: IntegralType) => + Some(DecimalPrecision.widerDecimalType(DecimalType.forType(t), d)) + case (_: FractionalType, _: DecimalType) | (_: DecimalType, _: FractionalType) => + Some(DoubleType) + case _ => None + }) + case None => None + }) + } + private def haveSameType(exprs: Seq[Expression]): Boolean = exprs.map(_.dataType).distinct.length == 1 @@ -440,7 +450,7 @@ object TypeCoercion { case a @ CreateArray(children) if !haveSameType(children) => val types = children.map(_.dataType) - findTightestCommonTypeAndPromoteToString(types) match { + findWiderCommonType(types) match { case Some(finalDataType) => CreateArray(children.map(Cast(_, finalDataType))) case None => a } @@ -451,7 +461,7 @@ object TypeCoercion { m.keys } else { val types = m.keys.map(_.dataType) - findTightestCommonTypeAndPromoteToString(types) match { + findWiderCommonType(types) match { case Some(finalDataType) => m.keys.map(Cast(_, finalDataType)) case None => m.keys } @@ -461,7 +471,7 @@ object TypeCoercion { m.values } else { val types = m.values.map(_.dataType) - findTightestCommonTypeAndPromoteToString(types) match { + findWiderCommonType(types) match { case Some(finalDataType) => m.values.map(Cast(_, finalDataType)) case None => m.values } @@ -494,16 +504,19 @@ object TypeCoercion { case None => c } + // When finding wider type for `Greatest` and `Least`, we should handle decimal types even if + // we need to truncate, but we should not promote one side to string if the other side is + // string.g case g @ Greatest(children) if !haveSameType(children) => val types = children.map(_.dataType) - findTightestCommonType(types) match { + findWiderTypeWithoutStringPromotion(types) match { case Some(finalDataType) => Greatest(children.map(Cast(_, finalDataType))) case None => g } case l @ Least(children) if !haveSameType(children) => val types = children.map(_.dataType) - findTightestCommonType(types) match { + findWiderTypeWithoutStringPromotion(types) match { case Some(finalDataType) => Least(children.map(Cast(_, finalDataType))) case None => l } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 76e42d9afa4c3..3aefb3cfc3336 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -216,7 +216,6 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { for (operator <- Seq[(Seq[Expression] => Expression)](Greatest, Least)) { assertError(operator(Seq('booleanField)), "requires at least 2 arguments") assertError(operator(Seq('intField, 'stringField)), "should all have the same type") - assertError(operator(Seq('intField, 'decimalField)), "should all have the same type") assertError(operator(Seq('mapField, 'mapField)), "does not support ordering") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 971c99b671671..a13c45fe2ffee 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -283,6 +283,24 @@ class TypeCoercionSuite extends PlanTest { :: Cast(Literal(1), StringType) :: Cast(Literal("a"), StringType) :: Nil)) + + ruleTest(TypeCoercion.FunctionArgumentConversion, + CreateArray(Literal.create(null, DecimalType(5, 3)) + :: Literal(1) + :: Nil), + CreateArray(Literal.create(null, DecimalType(5, 3)).cast(DecimalType(13, 3)) + :: Literal(1).cast(DecimalType(13, 3)) + :: Nil)) + + ruleTest(TypeCoercion.FunctionArgumentConversion, + CreateArray(Literal.create(null, DecimalType(5, 3)) + :: Literal.create(null, DecimalType(22, 10)) + :: Literal.create(null, DecimalType(38, 38)) + :: Nil), + CreateArray(Literal.create(null, DecimalType(5, 3)).cast(DecimalType(38, 38)) + :: Literal.create(null, DecimalType(22, 10)).cast(DecimalType(38, 38)) + :: Literal.create(null, DecimalType(38, 38)).cast(DecimalType(38, 38)) + :: Nil)) } test("CreateMap casts") { @@ -298,6 +316,17 @@ class TypeCoercionSuite extends PlanTest { :: Cast(Literal.create(2.0, FloatType), FloatType) :: Literal("b") :: Nil)) + ruleTest(TypeCoercion.FunctionArgumentConversion, + CreateMap(Literal.create(null, DecimalType(5, 3)) + :: Literal("a") + :: Literal.create(2.0, FloatType) + :: Literal("b") + :: Nil), + CreateMap(Literal.create(null, DecimalType(5, 3)).cast(DoubleType) + :: Literal("a") + :: Literal.create(2.0, FloatType).cast(DoubleType) + :: Literal("b") + :: Nil)) // type coercion for map values ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) @@ -310,6 +339,17 @@ class TypeCoercionSuite extends PlanTest { :: Literal(2) :: Cast(Literal(3.0), StringType) :: Nil)) + ruleTest(TypeCoercion.FunctionArgumentConversion, + CreateMap(Literal(1) + :: Literal.create(null, DecimalType(38, 0)) + :: Literal(2) + :: Literal.create(null, DecimalType(38, 38)) + :: Nil), + CreateMap(Literal(1) + :: Literal.create(null, DecimalType(38, 0)).cast(DecimalType(38, 38)) + :: Literal(2) + :: Literal.create(null, DecimalType(38, 38)).cast(DecimalType(38, 38)) + :: Nil)) // type coercion for both map keys and values ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) @@ -344,6 +384,33 @@ class TypeCoercionSuite extends PlanTest { :: Cast(Literal(1), DecimalType(22, 0)) :: Cast(Literal(new java.math.BigDecimal("1000000000000000000000")), DecimalType(22, 0)) :: Nil)) + ruleTest(TypeCoercion.FunctionArgumentConversion, + operator(Literal(1.0) + :: Literal.create(null, DecimalType(10, 5)) + :: Literal(1) + :: Nil), + operator(Literal(1.0).cast(DoubleType) + :: Literal.create(null, DecimalType(10, 5)).cast(DoubleType) + :: Literal(1).cast(DoubleType) + :: Nil)) + ruleTest(TypeCoercion.FunctionArgumentConversion, + operator(Literal.create(null, DecimalType(15, 0)) + :: Literal.create(null, DecimalType(10, 5)) + :: Literal(1) + :: Nil), + operator(Literal.create(null, DecimalType(15, 0)).cast(DecimalType(20, 5)) + :: Literal.create(null, DecimalType(10, 5)).cast(DecimalType(20, 5)) + :: Literal(1).cast(DecimalType(20, 5)) + :: Nil)) + ruleTest(TypeCoercion.FunctionArgumentConversion, + operator(Literal.create(2L, LongType) + :: Literal(1) + :: Literal.create(null, DecimalType(10, 5)) + :: Nil), + operator(Literal.create(2L, LongType).cast(DecimalType(25, 5)) + :: Literal(1).cast(DecimalType(25, 5)) + :: Literal.create(null, DecimalType(10, 5)).cast(DecimalType(25, 5)) + :: Nil)) } } From b44da5b4e2a62023b127fdd8b81c6ae95d2cdbc7 Mon Sep 17 00:00:00 2001 From: Kevin McHale Date: Wed, 3 Aug 2016 13:15:13 -0700 Subject: [PATCH 1083/1470] [SPARK-14204][SQL] register driverClass rather than user-specified class This is a pull request that was originally merged against branch-1.6 as #12000, now being merged into master as well. srowen zzcclp JoshRosen This pull request fixes an issue in which cluster-mode executors fail to properly register a JDBC driver when the driver is provided in a jar by the user, but the driver class name is derived from a JDBC URL (rather than specified by the user). The consequence of this is that all JDBC accesses under the described circumstances fail with an IllegalStateException. I reported the issue here: https://issues.apache.org/jira/browse/SPARK-14204 My proposed solution is to have the executors register the JDBC driver class under all circumstances, not only when the driver is specified by the user. This patch was tested manually. I built an assembly jar, deployed it to a cluster, and confirmed that the problem was fixed. Author: Kevin McHale Closes #14420 from mchalek/mchalek-jdbc_driver_registration. (cherry picked from commit 685b08e2611b69f8db60a00c0c94aecd315e2a3e) Signed-off-by: Sean Owen --- .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index d3e1efc562777..1328ac741c216 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -54,7 +54,7 @@ object JdbcUtils extends Logging { DriverManager.getDriver(url).getClass.getCanonicalName } () => { - userSpecifiedDriverClass.foreach(DriverRegistry.register) + DriverRegistry.register(driverClass) val driver: Driver = DriverManager.getDrivers.asScala.collectFirst { case d: DriverWrapper if d.wrapped.getClass.getCanonicalName == driverClass => d case d if d.getClass.getCanonicalName == driverClass => d From bb30a3d0f4616c871016ba772515e9461a9048eb Mon Sep 17 00:00:00 2001 From: Stefan Schulze Date: Wed, 3 Aug 2016 17:07:10 -0700 Subject: [PATCH 1084/1470] =?UTF-8?q?[SPARK-16770][BUILD]=20Fix=20JLine=20?= =?UTF-8?q?dependency=20management=20and=20version=20(Sca=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? As of Scala 2.11.x there is no longer a org.scala-lang:jline version aligned to the scala version itself. Scala console now uses the plain jline:jline module. Spark's dependency management did not reflect this change properly, causing Maven to pull in Jline via transitive dependency. Unfortunately Jline 2.12 contained a minor but very annoying bug rendering the shell almost useless for developers with german keyboard layout. This request contains the following chages: - Exclude transitive dependency 'jline:jline' from hive-exec module - Remove global properties 'jline.version' and 'jline.groupId' - Add both properties and dependency to 'scala-2.11' profile - Add explicit dependency on 'jline:jline' to module 'spark-repl' ## How was this patch tested? - Running mvn dependency:tree and checking for correct Jline version 2.12.1 - Running full builds with assembly and checking for jline-2.12.1.jar in 'lib' folder of generated tarball Author: Stefan Schulze Closes #14429 from stsc-pentasys/SPARK-16770. (cherry picked from commit 4775eb414fa8285cfdc301e52dac52a2ef64c9e1) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- pom.xml | 22 +++++++++++----------- repl/pom.xml | 11 ++++------- 7 files changed, 20 insertions(+), 23 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index e1d4051e7e766..f4a39ab47f0ee 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -98,7 +98,7 @@ jersey-media-jaxb-2.22.2.jar jersey-server-2.22.2.jar jets3t-0.7.1.jar jetty-util-6.1.26.jar -jline-2.12.jar +jline-2.12.1.jar joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 78d93334fff99..ba93f90b72781 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -103,7 +103,7 @@ jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar jetty-util-6.1.26.jar -jline-2.12.jar +jline-2.12.1.jar joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index abc644da38065..19459b6fc30f9 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -103,7 +103,7 @@ jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar jetty-util-6.1.26.jar -jline-2.12.jar +jline-2.12.1.jar joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 10ecd482b03e8..f7022a17e26d9 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -111,7 +111,7 @@ jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar jetty-util-6.1.26.jar -jline-2.12.jar +jline-2.12.1.jar joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index e84689ffd89a7..36f0fa592c7e7 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -111,7 +111,7 @@ jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar jetty-util-6.1.26.jar -jline-2.12.jar +jline-2.12.1.jar joda-time-2.9.3.jar jodd-core-3.5.2.jar jpam-1.1.jar diff --git a/pom.xml b/pom.xml index 910f1a8959b90..ded596336e094 100644 --- a/pom.xml +++ b/pom.xml @@ -159,8 +159,6 @@ 3.2.2 2.11.8 2.11 - ${scala.version} - org.scala-lang 1.9.13 2.6.5 1.1.2.4 @@ -1428,6 +1426,10 @@ org.codehaus.groovy groovy-all + + jline + jline + @@ -1832,6 +1834,11 @@ antlr4-runtime ${antlr4.version} + + ${jline.groupid} + jline + ${jline.version} + @@ -2537,15 +2544,6 @@ ${scala.version} org.scala-lang - - - - ${jline.groupid} - jline - ${jline.version} - - - @@ -2644,6 +2642,8 @@ 2.11.8 2.11 + 2.12.1 + jline diff --git a/repl/pom.xml b/repl/pom.xml index 0b5ec1a08c821..6fe1d7d7cd45e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -72,6 +72,10 @@ ${scala.version} + ${jline.groupid} + jline + + org.slf4j jul-to-slf4j @@ -161,13 +165,6 @@ scala-2.10 - - - ${jline.groupid} - jline - ${jline.version} - - From 11854e5a1baa7682d91bfce4e8bba57566f22b3a Mon Sep 17 00:00:00 2001 From: sharkd Date: Wed, 3 Aug 2016 19:20:34 -0700 Subject: [PATCH 1085/1470] [SPARK-16873][CORE] Fix SpillReader NPE when spillFile has no data ## What changes were proposed in this pull request? SpillReader NPE when spillFile has no data. See follow logs: 16/07/31 20:54:04 INFO collection.ExternalSorter: spill memory to file:/data4/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-db5f46c3-d7a4-4f93-8b77-565e469696fb/09/temp_shuffle_ec3ece08-4569-4197-893a-4a5dfcbbf9fa, fileSize:0.0 B 16/07/31 20:54:04 WARN memory.TaskMemoryManager: leak 164.3 MB memory from org.apache.spark.util.collection.ExternalSorter3db4b52d 16/07/31 20:54:04 ERROR executor.Executor: Managed memory leak detected; size = 190458101 bytes, TID = 2358516/07/31 20:54:04 ERROR executor.Executor: Exception in task 1013.0 in stage 18.0 (TID 23585) java.lang.NullPointerException at org.apache.spark.util.collection.ExternalSorter$SpillReader.cleanup(ExternalSorter.scala:624) at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:539) at org.apache.spark.util.collection.ExternalSorter$SpillReader.(ExternalSorter.scala:507) at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:816) at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:251) at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:109) at org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:154) at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249) at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:346) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:367) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:89) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:227) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) 16/07/31 20:54:30 INFO executor.Executor: Executor is trying to kill task 1090.1 in stage 18.0 (TID 23793) 16/07/31 20:54:30 INFO executor.CoarseGrainedExecutorBackend: Driver commanded a shutdown ## How was this patch tested? Manual test. Author: sharkd Author: sharkdtu Closes #14479 from sharkdtu/master. (cherry picked from commit 583d91a1957f4258a64184cc6b9007588791d332) Signed-off-by: Reynold Xin --- .../org/apache/spark/util/collection/ExternalSorter.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 4067acee738ed..6ea7307c3c6e7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -622,7 +622,9 @@ private[spark] class ExternalSorter[K, V, C]( val ds = deserializeStream deserializeStream = null fileStream = null - ds.close() + if (ds != null) { + ds.close() + } // NOTE: We don't do file.delete() here because that is done in ExternalSorter.stop(). // This should also be fixed in ExternalAppendOnlyMap. } From 182991eddebfaeb60fc44ecb8fc457ea6dd6f56a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 4 Aug 2016 11:20:17 -0700 Subject: [PATCH 1086/1470] [SPARK-16802] [SQL] fix overflow in LongToUnsafeRowMap ## What changes were proposed in this pull request? This patch fix the overflow in LongToUnsafeRowMap when the range of key is very wide (the key is much much smaller then minKey, for example, key is Long.MinValue, minKey is > 0). ## How was this patch tested? Added regression test (also for SPARK-16740) Author: Davies Liu Closes #14464 from davies/fix_overflow. (cherry picked from commit 9d4e6212fa8d434089d32bff1217f39919abe44d) Signed-off-by: Davies Liu --- .../sql/execution/joins/HashedRelation.scala | 16 ++++--- .../execution/joins/HashedRelationSuite.scala | 45 +++++++++++++++++++ 2 files changed, 55 insertions(+), 6 deletions(-) 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 cf4454c033384..08975733ff5d1 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 @@ -459,9 +459,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def getValue(key: Long, resultRow: UnsafeRow): UnsafeRow = { if (isDense) { - val idx = (key - minKey).toInt - if (idx >= 0 && key <= maxKey && array(idx) > 0) { - return getRow(array(idx), resultRow) + if (key >= minKey && key <= maxKey) { + val value = array((key - minKey).toInt) + if (value > 0) { + return getRow(value, resultRow) + } } } else { var pos = firstSlot(key) @@ -497,9 +499,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def get(key: Long, resultRow: UnsafeRow): Iterator[UnsafeRow] = { if (isDense) { - val idx = (key - minKey).toInt - if (idx >=0 && key <= maxKey && array(idx) > 0) { - return valueIter(array(idx), resultRow) + if (key >= minKey && key <= maxKey) { + val value = array((key - minKey).toInt) + if (value > 0) { + return valueIter(value, resultRow) + } } } else { var pos = firstSlot(key) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 40864c80ebc81..1196f5ec7b3a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -152,6 +152,51 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { } } + test("LongToUnsafeRowMap with very wide range") { + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) + + { + // SPARK-16740 + val keys = Seq(0L, Long.MaxValue, Long.MaxValue) + val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + assert(map.getValue(k, row) eq row) + assert(row.getLong(0) === k) + } + map.free() + } + + + { + // SPARK-16802 + val keys = Seq(Long.MaxValue, Long.MaxValue - 10) + val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + assert(map.getValue(k, row) eq row) + assert(row.getLong(0) === k) + } + assert(map.getValue(Long.MinValue, row) eq null) + map.free() + } + } + test("Spark-14521") { val ser = new KryoSerializer( (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() From ddbff011eaa79aa4c96184bfd15682fbb220f8e7 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Thu, 4 Aug 2016 21:39:45 +0100 Subject: [PATCH 1087/1470] [SPARK-16875][SQL] Add args checking for DataSet randomSplit and sample ## What changes were proposed in this pull request? Add the missing args-checking for randomSplit and sample ## How was this patch tested? unit tests Author: Zheng RuiFeng Closes #14478 from zhengruifeng/fix_randomSplit. (cherry picked from commit be8ea4b2f7ddf1196111acb61fe1a79866376003) Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/rdd/RDD.scala | 37 +++++++++++++------ .../scala/org/apache/spark/sql/Dataset.scala | 14 ++++++- 2 files changed, 37 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a4905dd51b940..2ee13dc4db5fd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -474,12 +474,17 @@ abstract class RDD[T: ClassTag]( def sample( withReplacement: Boolean, fraction: Double, - seed: Long = Utils.random.nextLong): RDD[T] = withScope { - require(fraction >= 0.0, "Negative fraction value: " + fraction) - if (withReplacement) { - new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) - } else { - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), true, seed) + seed: Long = Utils.random.nextLong): RDD[T] = { + require(fraction >= 0, + s"Fraction must be nonnegative, but got ${fraction}") + + withScope { + require(fraction >= 0.0, "Negative fraction value: " + fraction) + if (withReplacement) { + new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) + } else { + new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), true, seed) + } } } @@ -493,14 +498,22 @@ abstract class RDD[T: ClassTag]( */ def randomSplit( weights: Array[Double], - seed: Long = Utils.random.nextLong): Array[RDD[T]] = withScope { - val sum = weights.sum - val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) - normalizedCumWeights.sliding(2).map { x => - randomSampleWithRange(x(0), x(1), seed) - }.toArray + seed: Long = Utils.random.nextLong): Array[RDD[T]] = { + require(weights.forall(_ >= 0), + s"Weights must be nonnegative, but got ${weights.mkString("[", ",", "]")}") + require(weights.sum > 0, + s"Sum of weights must be positive, but got ${weights.mkString("[", ",", "]")}") + + withScope { + val sum = weights.sum + val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) + normalizedCumWeights.sliding(2).map { x => + randomSampleWithRange(x(0), x(1), seed) + }.toArray + } } + /** * Internal method exposed for Random Splits in DataFrames. Samples an RDD given a probability * range. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 067cbec4bf618..6ca0138adf0e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1500,8 +1500,13 @@ class Dataset[T] private[sql]( * @group typedrel * @since 1.6.0 */ - def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = withTypedPlan { - Sample(0.0, fraction, withReplacement, seed, logicalPlan)() + def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = { + require(fraction >= 0, + s"Fraction must be nonnegative, but got ${fraction}") + + withTypedPlan { + Sample(0.0, fraction, withReplacement, seed, logicalPlan)() + } } /** @@ -1529,6 +1534,11 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ def randomSplit(weights: Array[Double], seed: Long): Array[Dataset[T]] = { + require(weights.forall(_ >= 0), + s"Weights must be nonnegative, but got ${weights.mkString("[", ",", "]")}") + require(weights.sum > 0, + s"Sum of weights must be positive, but got ${weights.mkString("[", ",", "]")}") + // It is possible that the underlying dataframe doesn't guarantee the ordering of rows in its // constituent partitions each time a split is materialized which could result in // overlapping splits. To prevent this, we explicitly sort each input partition to make the From c66338b3a994bb2701fbd0cd4e632d0cf0e51b7f Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Thu, 4 Aug 2016 21:41:35 +0100 Subject: [PATCH 1088/1470] [SPARK-16880][ML][MLLIB] make ann training data persisted if needed ## What changes were proposed in this pull request? To Make sure ANN layer input training data to be persisted, so that it can avoid overhead cost if the RDD need to be computed from lineage. ## How was this patch tested? Existing Tests. Author: WeichenXu Closes #14483 from WeichenXu123/add_ann_persist_training_data. (cherry picked from commit 462784ffad77e43455dd0364064ce4994826a426) Signed-off-by: Sean Owen --- mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 576584c62797d..88909a9fb953f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -26,6 +26,7 @@ import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.optimization._ import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom /** @@ -810,9 +811,13 @@ private[ml] class FeedForwardTrainer( getWeights } // TODO: deprecate standard optimizer because it needs Vector - val newWeights = optimizer.optimize(dataStacker.stack(data).map { v => + val trainData = dataStacker.stack(data).map { v => (v._1, OldVectors.fromML(v._2)) - }, w) + } + val handlePersistence = trainData.getStorageLevel == StorageLevel.NONE + if (handlePersistence) trainData.persist(StorageLevel.MEMORY_AND_DISK) + val newWeights = optimizer.optimize(trainData, w) + if (handlePersistence) trainData.unpersist() topology.model(newWeights) } From 818ddcf98f1091f9133ebbf3d2e2217e17c44a0f Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 4 Aug 2016 21:43:05 +0100 Subject: [PATCH 1089/1470] [SPARK-16877][BUILD] Add rules for preventing to use Java annotations (Deprecated and Override) ## What changes were proposed in this pull request? This PR adds both rules for preventing to use `Deprecated` and `Override`. - Java's `Override` It seems Scala compiler just ignores this. Apparently, `override` modifier is only mandatory for " that override some other **concrete member definition** in a parent class" but not for for **incomplete member definition** (such as ones from trait or abstract), see (http://www.scala-lang.org/files/archive/spec/2.11/05-classes-and-objects.html#override) For a simple example, - Normal class - needs `override` modifier ```bash scala> class A { def say = {}} defined class A scala> class B extends A { def say = {}} :8: error: overriding method say in class A of type => Unit; method say needs `override' modifier class B extends A { def say = {}} ^ ``` - Trait - does not need `override` modifier ```bash scala> trait A { def say } defined trait A scala> class B extends A { def say = {}} defined class B ``` To cut this short, this case below is possible, ```bash scala> class B extends A { | Override | def say = {} | } defined class B ``` we can write `Override` annotation (meaning nothing) which might confuse engineers that Java's annotation is working fine. It might be great if we prevent those potential confusion. - Java's `Deprecated` When `Deprecated` is used, it seems Scala compiler recognises this correctly but it seems we use Scala one `deprecated` across codebase. ## How was this patch tested? Manually tested, by inserting both `Override` and `Deprecated`. This will shows the error messages as below: ```bash Scalastyle checks failed at following occurrences: [error] ... : deprecated should be used instead of java.lang.Deprecated. ``` ```basg Scalastyle checks failed at following occurrences: [error] ... : override modifier should be used instead of java.lang.Override. ``` Author: hyukjinkwon Closes #14490 from HyukjinKwon/SPARK-16877. (cherry picked from commit 1d781572e832058e2ef54bccd76ef71bc1fd548c) Signed-off-by: Sean Owen --- scalastyle-config.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 9a35183c63733..7fe0697202cd1 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -250,6 +250,14 @@ This file is divided into 3 sections: Omit braces in case clauses. + + + ^Override$ + override modifier should be used instead of @java.lang.Override. + + + + From 824d6268d1b0fb10dd9a23b0a7d5452b72ec71ef Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Thu, 4 Aug 2016 21:44:54 +0100 Subject: [PATCH 1090/1470] [SPARK-16863][ML] ProbabilisticClassifier.fit check threshoulds' length ## What changes were proposed in this pull request? Add threshoulds' length checking for Classifiers which extends ProbabilisticClassifier ## How was this patch tested? unit tests and manual tests Author: Zheng RuiFeng Closes #14470 from zhengruifeng/classifier_check_setThreshoulds_length. (cherry picked from commit 0e2e5d7d0b42226c61c3200fd63d2831c558519d) Signed-off-by: Sean Owen --- .../spark/ml/classification/DecisionTreeClassifier.scala | 7 +++++++ .../spark/ml/classification/LogisticRegression.scala | 6 ++++++ .../org/apache/spark/ml/classification/NaiveBayes.scala | 8 ++++++++ .../spark/ml/classification/RandomForestClassifier.scala | 7 +++++++ 4 files changed, 28 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 71293017e0527..bb192ab5f25ab 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -84,6 +84,13 @@ class DecisionTreeClassifier @Since("1.4.0") ( val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val numClasses: Int = getNumClasses(dataset) + + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, numClasses) val strategy = getOldStrategy(categoricalFeatures, numClasses) 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 91eee0e69d635..cca337487d6da 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 @@ -292,6 +292,12 @@ class LogisticRegression @Since("1.2.0") ( val numClasses = histogram.length val numFeatures = summarizer.mean.size + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + instr.logNumClasses(numClasses) instr.logNumFeatures(numFeatures) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index ab977c8802e36..f939a1c6808e6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -101,6 +101,14 @@ class NaiveBayes @Since("1.5.0") ( setDefault(modelType -> OldNaiveBayes.Multinomial) override protected def train(dataset: Dataset[_]): NaiveBayesModel = { + val numClasses = getNumClasses(dataset) + + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + val oldDataset: RDD[OldLabeledPoint] = extractLabeledPoints(dataset).map(OldLabeledPoint.fromML) val oldModel = OldNaiveBayes.train(oldDataset, $(smoothing), $(modelType)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 4ab132e5f2940..52345b0626c47 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -100,6 +100,13 @@ class RandomForestClassifier @Since("1.4.0") ( val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val numClasses: Int = getNumClasses(dataset) + + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, numClasses) val strategy = super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity) From dae08fb5accf2ad1e4fae6225395cc16623011a0 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Fri, 5 Aug 2016 11:19:20 +0800 Subject: [PATCH 1091/1470] [SPARK-16907][SQL] Fix performance regression for parquet table when vectorized parquet record reader is not being used ## What changes were proposed in this pull request? For non-partitioned parquet table, if the vectorized parquet record reader is not being used, Spark 2.0 adds an extra unnecessary memory copy to append partition values for each row. There are several typical cases that vectorized parquet record reader is not being used: 1. When the table schema is not flat, like containing nested fields. 2. When `spark.sql.parquet.enableVectorizedReader = false` By fixing this bug, we get about 20% - 30% performance gain in test case like this: ``` // Generates parquet table with nested columns spark.range(100000000).select(struct($"id").as("nc")).write.parquet("/tmp/data4") def time[R](block: => R): Long = { val t0 = System.nanoTime() val result = block // call-by-name val t1 = System.nanoTime() println("Elapsed time: " + (t1 - t0)/1000000 + "ms") (t1 - t0)/1000000 } val x = ((0 until 20).toList.map(x => time(spark.read.parquet("/tmp/data4").filter($"nc.id" < 100).collect()))).sum/20 ``` ## How was this patch tested? After a few times warm up, we get 26% performance improvement Before fix: ``` Average: 4584ms, raw data (10 tries): 4726ms 4509ms 4454ms 4879ms 4586ms 4733ms 4500ms 4361ms 4456ms 4640ms ``` After fix: ``` Average: 3614ms, raw data(10 tries): 3554ms 3740ms 4019ms 3439ms 3460ms 3664ms 3557ms 3584ms 3612ms 3531ms ``` Test env: Intel(R) Core(TM) i7-6700 CPU 3.40GHz, Intel SSD SC2KW24 Author: Sean Zhong Closes #14445 from clockfly/fix_parquet_regression_2. (cherry picked from commit 1fa644497aed0a6d22f5fc7bf8e752508053b75b) Signed-off-by: Wenchen Fan --- .../execution/datasources/parquet/ParquetFileFormat.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index f1c78bb60c4f7..5397d50b6c7a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -366,6 +366,7 @@ private[sql] class ParquetFileFormat vectorizedReader } else { logDebug(s"Falling back to parquet-mr") + // ParquetRecordReader returns UnsafeRow val reader = pushed match { case Some(filter) => new ParquetRecordReader[InternalRow]( @@ -392,8 +393,13 @@ private[sql] class ParquetFileFormat // This is a horrible erasure hack... if we type the iterator above, then it actually check // the type in next() and we get a class cast exception. If we make that function return // Object, then we can defer the cast until later! - iter.asInstanceOf[Iterator[InternalRow]] + if (partitionSchema.length == 0) { + // There is no partition columns + iter.asInstanceOf[Iterator[InternalRow]] + } else { + iter.asInstanceOf[Iterator[InternalRow]] .map(d => appendPartitionColumns(joinedRow(d, file.partitionValues))) + } } } } From b4a89c1c1245cb764e7e214220518442b2225ce5 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 5 Aug 2016 10:13:32 +0100 Subject: [PATCH 1092/1470] [SPARK-16312][STREAMING][KAFKA][DOC] Doc for Kafka 0.10 integration ## What changes were proposed in this pull request? Doc for the Kafka 0.10 integration ## How was this patch tested? Scala code examples were taken from my example repo, so hopefully they compile. Author: cody koeninger Closes #14385 from koeninger/SPARK-16312. (cherry picked from commit c9f2501af278241f780a38b9562e193755ed5af3) Signed-off-by: Sean Owen --- docs/streaming-kafka-0-10-integration.md | 192 +++++++++++++++++ docs/streaming-kafka-0-8-integration.md | 210 +++++++++++++++++++ docs/streaming-kafka-integration.md | 253 +++++------------------ docs/streaming-programming-guide.md | 4 +- 4 files changed, 452 insertions(+), 207 deletions(-) create mode 100644 docs/streaming-kafka-0-10-integration.md create mode 100644 docs/streaming-kafka-0-8-integration.md diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md new file mode 100644 index 0000000000000..44c39e39446de --- /dev/null +++ b/docs/streaming-kafka-0-10-integration.md @@ -0,0 +1,192 @@ +--- +layout: global +title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) +--- + +The Spark Streaming integration for Kafka 0.10 is similar in design to the 0.8 [Direct Stream approach](streaming-kafka-0-8-integration.html#approach-2-direct-approach-no-receivers). It provides simple parallelism, 1:1 correspondence between Kafka partitions and Spark partitions, and access to offsets and metadata. However, because the newer integration uses the [new Kafka consumer API](http://kafka.apache.org/documentation.html#newconsumerapi) instead of the simple API, there are notable differences in usage. This version of the integration is marked as experimental, so the API is potentially subject to change. + +### Linking +For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +### Creating a Direct Stream + Note that the namespace for the import includes the version, org.apache.spark.streaming.kafka010 + +
    +
    + import org.apache.kafka.clients.consumer.ConsumerRecord + import org.apache.kafka.common.serialization.StringDeserializer + import org.apache.spark.streaming.kafka010._ + import org.apache.spark.streaming.kafka010.LocationStrategies.PreferConsistent + import org.apache.spark.streaming.kafka010.ConsumerStrategies.Subscribe + + val kafkaParams = Map[String, Object]( + "bootstrap.servers" -> "localhost:9092,anotherhost:9092", + "key.deserializer" -> classOf[StringDeserializer], + "value.deserializer" -> classOf[StringDeserializer], + "group.id" -> "example", + "auto.offset.reset" -> "latest", + "enable.auto.commit" -> (false: java.lang.Boolean) + ) + + val topics = Array("topicA", "topicB") + val stream = KafkaUtils.createDirectStream[String, String]( + streamingContext, + PreferConsistent, + Subscribe[String, String](topics, kafkaParams) + ) + + stream.map(record => (record.key, record.value)) + +Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/ConsumerRecord.html) +
    +
    +
    +
    + +For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). +Note that enable.auto.commit is disabled, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. + +### LocationStrategies +The new Kafka consumer API will pre-fetch messages into buffers. Therefore it is important for performance reasons that the Spark integration keep cached consumers on executors (rather than recreating them for each batch), and prefer to schedule partitions on the host locations that have the appropriate consumers. + +In most cases, you should use `LocationStrategies.PreferConsistent` as shown above. This will distribute partitions evenly across available executors. If your executors are on the same hosts as your Kafka brokers, use `PreferBrokers`, which will prefer to schedule partitions on the Kafka leader for that partition. Finally, if you have a significant skew in load among partitions, use `PreferFixed`. This allows you to specify an explicit mapping of partitions to hosts (any unspecified partitions will use a consistent location). + +The cache for consumers has a default maximum size of 64. If you expect to be handling more than (64 * number of executors) Kafka partitions, you can change this setting via `spark.streaming.kafka.consumer.cache.maxCapacity` + +### ConsumerStrategies +The new Kafka consumer API has a number of different ways to specify topics, some of which require considerable post-object-instantiation setup. `ConsumerStrategies` provides an abstraction that allows Spark to obtain properly configured consumers even after restart from checkpoint. + +`ConsumerStrategies.Subscribe`, as shown above, allows you to subscribe to a fixed collection of topics. `SubscribePattern` allows you to use a regex to specify topics of interest. Note that unlike the 0.8 integration, using `Subscribe` or `SubscribePattern` should respond to adding partitions during a running stream. Finally, `Assign` allows you to specify a fixed collection of partitions. All three strategies have overloaded constructors that allow you to specify the starting offset for a particular partition. + +If you have specific consumer setup needs that are not met by the options above, `ConsumerStrategy` is a public class that you can extend. + +### Creating an RDD +If you have a use case that is better suited to batch processing, you can create an RDD for a defined range of offsets. + +
    +
    + // Import dependencies and create kafka params as in Create Direct Stream above + + val offsetRanges = Array( + // topic, partition, inclusive starting offset, exclusive ending offset + OffsetRange("test", 0, 0, 100), + OffsetRange("test", 1, 0, 100) + ) + + val rdd = KafkaUtils.createRDD[String, String](sparkContext, kafkaParams, offsetRanges, PreferConsistent) + +
    +
    +
    +
    + +Note that you cannot use `PreferBrokers`, because without the stream there is not a driver-side consumer to automatically look up broker metadata for you. Use `PreferFixed` with your own metadata lookups if necessary. + +### Obtaining Offsets + +
    +
    + stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd.foreachPartition { iter => + val o: OffsetRange = offsetRanges(TaskContext.get.partitionId) + println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + } +
    +
    +
    +
    + +Note that the typecast to `HasOffsetRanges` will only succeed if it is done in the first method called on the result of `createDirectStream`, not later down a chain of methods. Be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). + +### Storing Offsets +Kafka delivery semantics in the case of failure depend on how and when offsets are stored. Spark output operations are [at-least-once](streaming-programming-guide.html#semantics-of-output-operations). So if you want the equivalent of exactly-once semantics, you must either store offsets after an idempotent output, or store offsets in an atomic transaction alongside output. With this integration, you have 3 options, in order of increasing reliablity (and code complexity), for how to store offsets. + +#### Checkpoints +If you enable Spark [checkpointing](streaming-programming-guide.html#checkpointing), offsets will be stored in the checkpoint. This is easy to enable, but there are drawbacks. Your output operation must be idempotent, since you will get repeated outputs; transactions are not an option. Furthermore, you cannot recover from a checkpoint if your application code has changed. For planned upgrades, you can mitigate this by running the new code at the same time as the old code (since outputs need to be idempotent anyway, they should not clash). But for unplanned failures that require code changes, you will lose data unless you have another way to identify known good starting offsets. + +#### Kafka itself +Kafka has an offset commit API that stores offsets in a special Kafka topic. By default, the new consumer will periodically auto-commit offsets. This is almost certainly not what you want, because messages successfully polled by the consumer may not yet have resulted in a Spark output operation, resulting in undefined semantics. This is why the stream example above sets "enable.auto.commit" to false. However, you can commit offsets to Kafka after you know your output has been stored, using the `commitAsync` API. The benefit as compared to checkpoints is that Kafka is a durable store regardless of changes to your application code. However, Kafka is not transactional, so your outputs must still be idempotent. + +
    +
    + stream.foreachRDD { rdd => + val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + + // some time later, after outputs have completed + stream.asInstanceOf[CanCommitOffsets].commitAsync(offsets) + } + +As with HasOffsetRanges, the cast to CanCommitOffsets will only succeed if called on the result of createDirectStream, not after transformations. The commitAsync call is threadsafe, but must occur after outputs if you want meaningful semantics. +
    +
    +
    +
    + +#### Your own data store +For data stores that support transactions, saving offsets in the same transaction as the results can keep the two in sync, even in failure situations. If you're careful about detecting repeated or skipped offset ranges, rolling back the transaction prevents duplicated or lost messages from affecting results. This gives the equivalent of exactly-once semantics. It is also possible to use this tactic even for outputs that result from aggregations, which are typically hard to make idempotent. + +
    +
    + // The details depend on your data store, but the general idea looks like this + + // begin from the the offsets committed to the database + val fromOffsets = selectOffsetsFromYourDatabase.map { resultSet => + new TopicPartition(resultSet.string("topic")), resultSet.int("partition")) -> resultSet.long("offset") + }.toMap + + val stream = KafkaUtils.createDirectStream[String, String]( + streamingContext, + PreferConsistent, + Assign[String, String](fromOffsets.keys.toList, kafkaParams, fromOffsets) + ) + + stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + + val results = yourCalculation(rdd) + + yourTransactionBlock { + // update results + + // update offsets where the end of existing offsets matches the beginning of this batch of offsets + + // assert that offsets were updated correctly + } + } +
    +
    +
    +
    + +### SSL / TLS +The new Kafka consumer [supports SSL](http://kafka.apache.org/documentation.html#security_ssl). To enable it, set kafkaParams appropriately before passing to `createDirectStream` / `createRDD`. Note that this only applies to communication between Spark and Kafka brokers; you are still responsible for separately [securing](security.html) Spark inter-node communication. + + +
    +
    + val kafkaParams = Map[String, Object]( + // the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS + "security.protocol" -> "SSL", + "ssl.truststore.location" -> "/some-directory/kafka.client.truststore.jks", + "ssl.truststore.password" -> "test1234", + "ssl.keystore.location" -> "/some-directory/kafka.client.keystore.jks", + "ssl.keystore.password" -> "test1234", + "ssl.key.password" -> "test1234" + ) +
    +
    +
    +
    + +### Deploying + +As with any Spark applications, `spark-submit` is used to launch your application. + +For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + diff --git a/docs/streaming-kafka-0-8-integration.md b/docs/streaming-kafka-0-8-integration.md new file mode 100644 index 0000000000000..da4a845fe2d41 --- /dev/null +++ b/docs/streaming-kafka-0-8-integration.md @@ -0,0 +1,210 @@ +--- +layout: global +title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.8.2.1 or higher) +--- +Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. Both approaches are considered stable APIs as of the current version of Spark. + +## Approach 1: Receiver-based Approach +This approach uses a Receiver to receive the data. The Receiver is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. + +However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs. + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val kafkaStream = KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairReceiverInputDStream kafkaStream = + KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); + + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). + +
    +
    + from pyspark.streaming.kafka import KafkaUtils + + kafkaStream = KafkaUtils.createStream(streamingContext, \ + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py). +
    +
    + + **Points to remember:** + + - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. + + - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. + + - If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use +`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). + +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. + + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + For Python applications which lack SBT/Maven project management, `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + + ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-0-8-assembly` from the + [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-0-8-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. + +## Approach 2: Direct Approach (No Receivers) +This new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this feature was introduced in Spark 1.3 for the Scala and Java API, in Spark 1.4 for the Python API. + +This approach has the following advantages over the receiver-based approach (i.e. Approach 1). + +- *Simplified Parallelism:* No need to create multiple input Kafka streams and union them. With `directStream`, Spark Streaming will create as many RDD partitions as there are Kafka partitions to consume, which will all read data from Kafka in parallel. So there is a one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. + +- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminates the problem as there is no receiver, and hence no need for Write Ahead Logs. As long as you have sufficient Kafka retention, messages can be recovered from Kafka. + +- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semantics of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). + +Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val directKafkaStream = KafkaUtils.createDirectStream[ + [key class], [value class], [key decoder class], [value decoder class] ]( + streamingContext, [map of Kafka parameters], [set of topics to consume]) + + You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairInputDStream directKafkaStream = + KafkaUtils.createDirectStream(streamingContext, + [key class], [value class], [key decoder class], [value decoder class], + [map of Kafka parameters], [set of topics to consume]); + + You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java). + +
    +
    + from pyspark.streaming.kafka import KafkaUtils + directKafkaStream = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) + + You can also pass a `messageHandler` to `createDirectStream` to access `KafkaMessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/direct_kafka_wordcount.py). +
    +
    + + In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. + By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. + + You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. + +
    +
    + // Hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + + directKafkaStream.transform { rdd => + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd + }.map { + ... + }.foreachRDD { rdd => + for (o <- offsetRanges) { + println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + ... + } +
    +
    + // Hold a reference to the current offset ranges, so it can be used downstream + final AtomicReference offsetRanges = new AtomicReference<>(); + + directKafkaStream.transformToPair( + new Function, JavaPairRDD>() { + @Override + public JavaPairRDD call(JavaPairRDD rdd) throws Exception { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + return rdd; + } + } + ).map( + ... + ).foreachRDD( + new Function, Void>() { + @Override + public Void call(JavaPairRDD rdd) throws IOException { + for (OffsetRange o : offsetRanges.get()) { + System.out.println( + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() + ); + } + ... + return null; + } + } + ); +
    +
    + offsetRanges = [] + + def storeOffsetRanges(rdd): + global offsetRanges + offsetRanges = rdd.offsetRanges() + return rdd + + def printOffsetRanges(rdd): + for o in offsetRanges: + print "%s %s %s %s" % (o.topic, o.partition, o.fromOffset, o.untilOffset) + + directKafkaStream\ + .transform(storeOffsetRanges)\ + .foreachRDD(printOffsetRanges) +
    +
    + + You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. + + Note that the typecast to HasOffsetRanges will only succeed if it is done in the first method called on the directKafkaStream, not later down a chain of methods. You can use transform() instead of foreachRDD() as your first method call in order to access offsets, then call further Spark methods. However, be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). + + Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate (in messages per second) at which each Kafka partition will be read by this direct API. + +3. **Deploying:** This is same as the first approach. diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index e0d3f4f69be8f..a8f3667a49850 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -2,209 +2,52 @@ layout: global title: Spark Streaming + Kafka Integration Guide --- -[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new experimental approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. -## Approach 1: Receiver-based Approach -This approach uses a Receiver to receive the data. The Receiver is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. - -However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs. - -Next, we discuss how to use this approach in your streaming application. - -1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - - groupId = org.apache.spark - artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - - For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. - -2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. - -
    -
    - import org.apache.spark.streaming.kafka._ - - val kafkaStream = KafkaUtils.createStream(streamingContext, - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - - You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). -
    -
    - import org.apache.spark.streaming.kafka.*; - - JavaPairReceiverInputDStream kafkaStream = - KafkaUtils.createStream(streamingContext, - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); - - You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). - -
    -
    - from pyspark.streaming.kafka import KafkaUtils - - kafkaStream = KafkaUtils.createStream(streamingContext, \ - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - - By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py). -
    -
    - - **Points to remember:** - - - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. - - - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. - - - If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use -`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). - -3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. - - For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - - For Python applications which lack SBT/Maven project management, `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, - - ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... - - Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-0-8-assembly` from the - [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-0-8-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. - -## Approach 2: Direct Approach (No Receivers) -This new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this is an experimental feature introduced in Spark 1.3 for the Scala and Java API, in Spark 1.4 for the Python API. - -This approach has the following advantages over the receiver-based approach (i.e. Approach 1). - -- *Simplified Parallelism:* No need to create multiple input Kafka streams and union them. With `directStream`, Spark Streaming will create as many RDD partitions as there are Kafka partitions to consume, which will all read data from Kafka in parallel. So there is a one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. - -- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminates the problem as there is no receiver, and hence no need for Write Ahead Logs. As long as you have sufficient Kafka retention, messages can be recovered from Kafka. - -- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semantics of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). - -Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). - -Next, we discuss how to use this approach in your streaming application. - -1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - - groupId = org.apache.spark - artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - -2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. - -
    -
    - import org.apache.spark.streaming.kafka._ - - val directKafkaStream = KafkaUtils.createDirectStream[ - [key class], [value class], [key decoder class], [value decoder class] ]( - streamingContext, [map of Kafka parameters], [set of topics to consume]) - - You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. - See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala). -
    -
    - import org.apache.spark.streaming.kafka.*; - - JavaPairInputDStream directKafkaStream = - KafkaUtils.createDirectStream(streamingContext, - [key class], [value class], [key decoder class], [value decoder class], - [map of Kafka parameters], [set of topics to consume]); - - You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. - See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java). - -
    -
    - from pyspark.streaming.kafka import KafkaUtils - directKafkaStream = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) - - You can also pass a `messageHandler` to `createDirectStream` to access `KafkaMessageAndMetadata` that contains metadata about the current message and transform it to any desired type. - By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/direct_kafka_wordcount.py). -
    -
    - - In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. - By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. - - You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. - -
    -
    - // Hold a reference to the current offset ranges, so it can be used downstream - var offsetRanges = Array[OffsetRange]() - - directKafkaStream.transform { rdd => - offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - rdd - }.map { - ... - }.foreachRDD { rdd => - for (o <- offsetRanges) { - println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") - } - ... - } -
    -
    - // Hold a reference to the current offset ranges, so it can be used downstream - final AtomicReference offsetRanges = new AtomicReference<>(); - - directKafkaStream.transformToPair( - new Function, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaPairRDD rdd) throws Exception { - OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - offsetRanges.set(offsets); - return rdd; - } - } - ).map( - ... - ).foreachRDD( - new Function, Void>() { - @Override - public Void call(JavaPairRDD rdd) throws IOException { - for (OffsetRange o : offsetRanges.get()) { - System.out.println( - o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() - ); - } - ... - return null; - } - } - ); -
    -
    - offsetRanges = [] - - def storeOffsetRanges(rdd): - global offsetRanges - offsetRanges = rdd.offsetRanges() - return rdd - - def printOffsetRanges(rdd): - for o in offsetRanges: - print "%s %s %s %s" % (o.topic, o.partition, o.fromOffset, o.untilOffset) - - directKafkaStream\ - .transform(storeOffsetRanges)\ - .foreachRDD(printOffsetRanges) -
    -
    - - You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. - - Note that the typecast to HasOffsetRanges will only succeed if it is done in the first method called on the directKafkaStream, not later down a chain of methods. You can use transform() instead of foreachRDD() as your first method call in order to access offsets, then call further Spark methods. However, be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). - - Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate (in messages per second) at which each Kafka partition will be read by this direct API. - -3. **Deploying:** This is same as the first approach. +[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Please read the [Kafka documentation](http://kafka.apache.org/documentation.html) thoroughly before starting an integration using Spark. + +The Kafka project introduced a new consumer api between versions 0.8 and 0.10, so there are 2 separate corresponding Spark Streaming packages available. Please choose the correct package for your brokers and desired features; note that the 0.8 integration is compatible with later 0.9 and 0.10 brokers, but the 0.10 integration is not compatible with earlier brokers. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    spark-streaming-kafka-0-8spark-streaming-kafka-0-10
    Broker Version0.8.2.1 or higher0.10.0 or higher
    Api StabilityStableExperimental
    Language SupportScala, Java, PythonScala, Java
    Receiver DStreamYesNo
    Direct DStreamYesYes
    SSL / TLS SupportNoYes
    Offset Commit ApiNoYes
    Dynamic Topic SubscriptionNoYes
    diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index e80f1c94ff1b4..902df6ada8790 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -683,7 +683,7 @@ and add it to the classpath. Some of these advanced sources are as follows. -- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka 0.8.2.1. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.8.2.1 or higher. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. - **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.6.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. @@ -2350,7 +2350,7 @@ The following table summarizes the semantics under failures: ### With Kafka Direct API {:.no_toc} -In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach (experimental as of Spark {{site.SPARK_VERSION_SHORT}}) is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). +In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). ## Semantics of output operations {:.no_toc} From 7fbac48f011f286ac779dfac6b2e8bbfcc2dd3a8 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Fri, 5 Aug 2016 11:06:36 +0100 Subject: [PATCH 1093/1470] [MINOR] Update AccumulatorV2 doc to not mention "+=". ## What changes were proposed in this pull request? As reported by Bryan Cutler on the mailing list, AccumulatorV2 does not have a += method, yet the documentation still references it. ## How was this patch tested? N/A Author: petermaxlee Closes #14466 from petermaxlee/accumulator. (cherry picked from commit e026064143367e4614cb866e321cc521fdde3170) Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 044dd69cc92c7..a9167ce6edf90 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -131,7 +131,7 @@ abstract class AccumulatorV2[IN, OUT] extends Serializable { def reset(): Unit /** - * Takes the inputs and accumulates. e.g. it can be a simple `+=` for counter accumulator. + * Takes the inputs and accumulates. */ def add(v: IN): Unit From d99d90982bb7bd2c9783dd007d5e16aa8703df6d Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 5 Aug 2016 22:07:59 +0100 Subject: [PATCH 1094/1470] [SPARK-16750][FOLLOW-UP][ML] Add transformSchema for StringIndexer/VectorAssembler and fix failed tests. ## What changes were proposed in this pull request? This is follow-up for #14378. When we add ```transformSchema``` for all estimators and transformers, I found there are tests failed for ```StringIndexer``` and ```VectorAssembler```. So I moved these parts of work separately in this PR, to make it more clear to review. The corresponding tests should throw ```IllegalArgumentException``` at schema validation period after we add ```transformSchema```. It's efficient that to throw exception at the start of ```fit``` or ```transform``` rather than during the process. ## How was this patch tested? Modified unit tests. Author: Yanbo Liang Closes #14455 from yanboliang/transformSchema. (cherry picked from commit 6cbde337a539e5bb170d0eb81f715a95ee9c9af3) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/feature/StringIndexer.scala | 4 +++- .../apache/spark/ml/feature/VectorAssembler.scala | 1 + .../apache/spark/ml/feature/StringIndexerSuite.scala | 12 ++++++++++-- .../spark/ml/feature/VectorAssemblerSuite.scala | 4 ++-- 4 files changed, 16 insertions(+), 5 deletions(-) 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 fe79e2ec808ac..80fe46796f807 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 @@ -85,6 +85,7 @@ class StringIndexer @Since("1.4.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): StringIndexerModel = { + transformSchema(dataset.schema, logging = true) val counts = dataset.select(col($(inputCol)).cast(StringType)) .rdd .map(_.getString(0)) @@ -160,7 +161,7 @@ class StringIndexerModel ( "Skip StringIndexerModel.") return dataset.toDF } - validateAndTransformSchema(dataset.schema) + transformSchema(dataset.schema, logging = true) val indexer = udf { label: String => if (labelToIndex.contains(label)) { @@ -305,6 +306,7 @@ class IndexToString private[ml] (@Since("1.5.0") override val uid: String) @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val inputColSchema = dataset.schema($(inputCol)) // If the labels array is empty use column metadata val values = if (!isDefined(labels) || $(labels).isEmpty) { 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 142a2ae44c690..ca900536bc7b8 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 @@ -51,6 +51,7 @@ class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) // Schema transformation. val schema = dataset.schema lazy val first = dataset.toDF.first() 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 index c221d4aa558a6..b478fea5e74ec 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala @@ -120,12 +120,20 @@ class StringIndexerSuite test("StringIndexerModel can't overwrite output column") { val df = spark.createDataFrame(Seq((1, 2), (3, 4))).toDF("input", "output") + intercept[IllegalArgumentException] { + new StringIndexer() + .setInputCol("input") + .setOutputCol("output") + .fit(df) + } + val indexer = new StringIndexer() .setInputCol("input") - .setOutputCol("output") + .setOutputCol("indexedInput") .fit(df) + intercept[IllegalArgumentException] { - indexer.transform(df) + indexer.setOutputCol("output").transform(df) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index 14973e79bf345..561493fbafd6c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -74,10 +74,10 @@ class VectorAssemblerSuite val assembler = new VectorAssembler() .setInputCols(Array("a", "b", "c")) .setOutputCol("features") - val thrown = intercept[SparkException] { + val thrown = intercept[IllegalArgumentException] { assembler.transform(df) } - assert(thrown.getMessage contains "VectorAssembler does not support the StringType type") + assert(thrown.getMessage contains "Data type StringType is not supported") } test("ML attributes") { From b5d65b45dfd34a7f451465ed3aac923077675166 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 5 Aug 2016 15:52:02 -0700 Subject: [PATCH 1095/1470] [SPARK-16901] Hive settings in hive-site.xml may be overridden by Hive's default values ## What changes were proposed in this pull request? When we create the HiveConf for metastore client, we use a Hadoop Conf as the base, which may contain Hive settings in hive-site.xml (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L49). However, HiveConf's initialize function basically ignores the base Hadoop Conf and always its default values (i.e. settings with non-null default values) as the base (https://github.com/apache/hive/blob/release-1.2.1/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java#L2687). So, even a user put javax.jdo.option.ConnectionURL in hive-site.xml, it is not used and Hive will use its default, which is jdbc:derby:;databaseName=metastore_db;create=true. This issue only shows up when `spark.sql.hive.metastore.jars` is not set to builtin. ## How was this patch tested? New test in HiveSparkSubmitSuite. Author: Yin Huai Closes #14497 from yhuai/SPARK-16901. (cherry picked from commit e679bc3c1cd418ef0025d2ecbc547c9660cac433) Signed-off-by: Yin Huai --- .../sql/hive/client/HiveClientImpl.scala | 24 +++++- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 80 +++++++++++++++++++ 2 files changed, 101 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 6cdf3ef545009..1d408950d358a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -139,14 +139,32 @@ private[hive] class HiveClientImpl( // so we should keep `conf` and reuse the existing instance of `CliSessionState`. originalState } else { - val hiveConf = new HiveConf(hadoopConf, classOf[SessionState]) + val hiveConf = new HiveConf(classOf[SessionState]) + // 1: we set all confs in the hadoopConf to this hiveConf. + // This hadoopConf contains user settings in Hadoop's core-site.xml file + // and Hive's hive-site.xml file. Note, we load hive-site.xml file manually in + // SharedState and put settings in this hadoopConf instead of relying on HiveConf + // to load user settings. Otherwise, HiveConf's initialize method will override + // settings in the hadoopConf. This issue only shows up when spark.sql.hive.metastore.jars + // is not set to builtin. When spark.sql.hive.metastore.jars is builtin, the classpath + // has hive-site.xml. So, HiveConf will use that to override its default values. + hadoopConf.iterator().asScala.foreach { entry => + val key = entry.getKey + val value = entry.getValue + if (key.toLowerCase.contains("password")) { + logDebug(s"Applying Hadoop and Hive config to Hive Conf: $key=xxx") + } else { + logDebug(s"Applying Hadoop and Hive config to Hive Conf: $key=$value") + } + hiveConf.set(key, value) + } // HiveConf is a Hadoop Configuration, which has a field of classLoader and // the initial value will be the current thread's context class loader // (i.e. initClassLoader at here). // We call initialConf.setClassLoader(initClassLoader) at here to make // this action explicit. hiveConf.setClassLoader(initClassLoader) - // First, we set all spark confs to this hiveConf. + // 2: we set all spark confs to this hiveConf. sparkConf.getAll.foreach { case (k, v) => if (k.toLowerCase.contains("password")) { logDebug(s"Applying Spark config to Hive Conf: $k=xxx") @@ -155,7 +173,7 @@ private[hive] class HiveClientImpl( } hiveConf.set(k, v) } - // Second, we set all entries in config to this hiveConf. + // 3: we set all entries in config to this hiveConf. extraConfig.foreach { case (k, v) => if (k.toLowerCase.contains("password")) { logDebug(s"Applying extra config to HiveConf: $k=xxx") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 9bca720a94736..dd8fec0c15ffa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -253,6 +253,47 @@ class HiveSparkSubmitSuite runSparkSubmit(args) } + test("SPARK-16901: set javax.jdo.option.ConnectionURL") { + // In this test, we set javax.jdo.option.ConnectionURL and set metastore version to + // 0.13. This test will make sure that javax.jdo.option.ConnectionURL will not be + // overridden by hive's default settings when we create a HiveConf object inside + // HiveClientImpl. Please see SPARK-16901 for more details. + + val metastoreLocation = Utils.createTempDir() + metastoreLocation.delete() + val metastoreURL = + s"jdbc:derby:memory:;databaseName=${metastoreLocation.getAbsolutePath};create=true" + val hiveSiteXmlContent = + s""" + | + | + | javax.jdo.option.ConnectionURL + | $metastoreURL + | + | + """.stripMargin + + // Write a hive-site.xml containing a setting of hive.metastore.warehouse.dir. + val hiveSiteDir = Utils.createTempDir() + val file = new File(hiveSiteDir.getCanonicalPath, "hive-site.xml") + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(hiveSiteXmlContent) + bw.close() + + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SetMetastoreURLTest.getClass.getName.stripSuffix("$"), + "--name", "SetMetastoreURLTest", + "--master", "local[1]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", s"spark.sql.test.expectedMetastoreURL=$metastoreURL", + "--conf", s"spark.driver.extraClassPath=${hiveSiteDir.getCanonicalPath}", + "--driver-java-options", "-Dderby.system.durability=test", + unusedJar.toString) + runSparkSubmit(args) + } + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. // This is copied from org.apache.spark.deploy.SparkSubmitSuite private def runSparkSubmit(args: Seq[String]): Unit = { @@ -313,6 +354,45 @@ class HiveSparkSubmitSuite } } +object SetMetastoreURLTest extends Logging { + def main(args: Array[String]): Unit = { + Utils.configTestLog4j("INFO") + + val sparkConf = new SparkConf(loadDefaults = true) + val builder = SparkSession.builder() + .config(sparkConf) + .config("spark.ui.enabled", "false") + .config("spark.sql.hive.metastore.version", "0.13.1") + // The issue described in SPARK-16901 only appear when + // spark.sql.hive.metastore.jars is not set to builtin. + .config("spark.sql.hive.metastore.jars", "maven") + .enableHiveSupport() + + val spark = builder.getOrCreate() + val expectedMetastoreURL = + spark.conf.get("spark.sql.test.expectedMetastoreURL") + logInfo(s"spark.sql.test.expectedMetastoreURL is $expectedMetastoreURL") + + if (expectedMetastoreURL == null) { + throw new Exception( + s"spark.sql.test.expectedMetastoreURL should be set.") + } + + // HiveSharedState is used when Hive support is enabled. + val actualMetastoreURL = + spark.sharedState.asInstanceOf[HiveSharedState] + .metadataHive + .getConf("javax.jdo.option.ConnectionURL", "this_is_a_wrong_URL") + logInfo(s"javax.jdo.option.ConnectionURL is $actualMetastoreURL") + + if (actualMetastoreURL != expectedMetastoreURL) { + throw new Exception( + s"Expected value of javax.jdo.option.ConnectionURL is $expectedMetastoreURL. But, " + + s"the actual value is $actualMetastoreURL") + } + } +} + object SetWarehouseLocationTest extends Logging { def main(args: Array[String]): Unit = { Utils.configTestLog4j("INFO") From 90e046024d341e93f529804323c8ee6e848c4eeb Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 6 Aug 2016 05:02:59 +0100 Subject: [PATCH 1096/1470] [SPARK-16772][PYTHON][DOCS] Fix API doc references to UDFRegistration + Update "important classes" ## Proposed Changes * Update the list of "important classes" in `pyspark.sql` to match 2.0. * Fix references to `UDFRegistration` so that the class shows up in the docs. It currently [doesn't](http://spark.apache.org/docs/latest/api/python/pyspark.sql.html). * Remove some unnecessary whitespace in the Python RST doc files. I reused the [existing JIRA](https://issues.apache.org/jira/browse/SPARK-16772) I created last week for similar API doc fixes. ## How was this patch tested? * I ran `lint-python` successfully. * I ran `make clean build` on the Python docs and confirmed the results are as expected locally in my browser. Author: Nicholas Chammas Closes #14496 from nchammas/SPARK-16772-UDFRegistration. (cherry picked from commit 2dd03886173f2f3b5c20fe14e9cdbd33480c1f36) Signed-off-by: Sean Owen --- python/docs/index.rst | 1 - python/docs/pyspark.sql.rst | 2 -- python/pyspark/sql/__init__.py | 11 +++++------ 3 files changed, 5 insertions(+), 9 deletions(-) diff --git a/python/docs/index.rst b/python/docs/index.rst index 306ffdb0e0f13..421c8de86a3cc 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -50,4 +50,3 @@ Indices and tables ================== * :ref:`search` - diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst index 3be9533c126d2..09848b880194d 100644 --- a/python/docs/pyspark.sql.rst +++ b/python/docs/pyspark.sql.rst @@ -8,14 +8,12 @@ Module Context :members: :undoc-members: - pyspark.sql.types module ------------------------ .. automodule:: pyspark.sql.types :members: :undoc-members: - pyspark.sql.functions module ---------------------------- .. automodule:: pyspark.sql.functions diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index cff73ff192e51..22ec416f6c584 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -18,7 +18,7 @@ """ Important classes of Spark SQL and DataFrames: - - :class:`pyspark.sql.SQLContext` + - :class:`pyspark.sql.SparkSession` Main entry point for :class:`DataFrame` and SQL functionality. - :class:`pyspark.sql.DataFrame` A distributed collection of data grouped into named columns. @@ -26,8 +26,6 @@ A column expression in a :class:`DataFrame`. - :class:`pyspark.sql.Row` A row of data in a :class:`DataFrame`. - - :class:`pyspark.sql.HiveContext` - Main entry point for accessing data stored in Apache Hive. - :class:`pyspark.sql.GroupedData` Aggregation methods, returned by :func:`DataFrame.groupBy`. - :class:`pyspark.sql.DataFrameNaFunctions` @@ -45,7 +43,7 @@ from pyspark.sql.types import Row -from pyspark.sql.context import SQLContext, HiveContext +from pyspark.sql.context import SQLContext, HiveContext, UDFRegistration from pyspark.sql.session import SparkSession from pyspark.sql.column import Column from pyspark.sql.dataframe import DataFrame, DataFrameNaFunctions, DataFrameStatFunctions @@ -55,7 +53,8 @@ __all__ = [ - 'SparkSession', 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', - 'Row', 'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec', + 'SparkSession', 'SQLContext', 'HiveContext', 'UDFRegistration', + 'DataFrame', 'GroupedData', 'Column', 'Row', + 'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec', 'DataFrameReader', 'DataFrameWriter' ] From d2334319252dd774467a8d54c2efb5eca603c073 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 6 Aug 2016 19:29:19 -0700 Subject: [PATCH 1097/1470] [SPARK-16925] Master should call schedule() after all executor exit events, not only failures ## What changes were proposed in this pull request? This patch fixes a bug in Spark's standalone Master which could cause applications to hang if tasks cause executors to exit with zero exit codes. As an example of the bug, run ``` sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) } ``` on a standalone cluster which has a single Spark application. This will cause all executors to die but those executors won't be replaced unless another Spark application or worker joins or leaves the cluster (or if an executor exits with a non-zero exit code). This behavior is caused by a bug in how the Master handles the `ExecutorStateChanged` event: the current implementation calls `schedule()` only if the executor exited with a non-zero exit code, so a task which causes a JVM to unexpectedly exit "cleanly" will skip the `schedule()` call. This patch addresses this by modifying the `ExecutorStateChanged` to always unconditionally call `schedule()`. This should be safe because it should always be safe to call `schedule()`; adding extra `schedule()` calls can only affect performance and should not introduce correctness bugs. ## How was this patch tested? I added a regression test in `DistributedSuite`. Author: Josh Rosen Closes #14510 from JoshRosen/SPARK-16925. (cherry picked from commit 4f5f9b670e1f1783f43feb22490613e72dcff852) Signed-off-by: Josh Rosen --- .../org/apache/spark/deploy/master/Master.scala | 17 +++++++---------- .../org/apache/spark/DistributedSuite.scala | 15 +++++++++++++++ 2 files changed, 22 insertions(+), 10 deletions(-) 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 f8aac3008cefa..fded8475a0916 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 @@ -265,19 +265,16 @@ private[deploy] class Master( val normalExit = exitStatus == Some(0) // Only retry certain number of times so we don't go into an infinite loop. - if (!normalExit) { - if (appInfo.incrementRetryCount() < ApplicationState.MAX_NUM_RETRY) { - schedule() - } else { - val execs = appInfo.executors.values - if (!execs.exists(_.state == ExecutorState.RUNNING)) { - logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + - s"${appInfo.retryCount} times; removing it") - removeApplication(appInfo, ApplicationState.FAILED) - } + if (!normalExit && appInfo.incrementRetryCount() >= ApplicationState.MAX_NUM_RETRY) { + val execs = appInfo.executors.values + if (!execs.exists(_.state == ExecutorState.RUNNING)) { + logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + + s"${appInfo.retryCount} times; removing it") + removeApplication(appInfo, ApplicationState.FAILED) } } } + schedule() case None => logWarning(s"Got status update for unknown executor $appId/$execId") } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 0515e6e3a6319..6beae842b04d1 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -134,6 +134,21 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } } + test("repeatedly failing task that crashes JVM with a zero exit code (SPARK-16925)") { + // Ensures that if a task which causes the JVM to exit with a zero exit code will cause the + // Spark job to eventually fail. + sc = new SparkContext(clusterUrl, "test") + failAfter(Span(100000, Millis)) { + val thrown = intercept[SparkException] { + sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) } + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("failed 4 times")) + } + // Check that the cluster is still usable: + sc.parallelize(1 to 10).count() + } + test("caching") { sc = new SparkContext(clusterUrl, "test") val data = sc.parallelize(1 to 1000, 10).cache() From 58e7038b8ff95244c0a4d940ad87e3ee15bc2059 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Sun, 7 Aug 2016 08:59:04 +0100 Subject: [PATCH 1098/1470] document that Mesos cluster mode supports python update docs to be consistent with SPARK-14645 https://issues.apache.org/jira/browse/SPARK-14645 Author: Michael Gummelt Closes #14514 from mgummelt/fix-docs. (cherry picked from commit 7aaa5a01c1ee142663d28e98eb501fffc5a5cc46) Signed-off-by: Sean Owen --- docs/submitting-applications.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 100ff0b147efd..6fe3049995876 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -58,7 +58,8 @@ for applications that involve the REPL (e.g. Spark shell). Alternatively, if your application is submitted from a machine far from the worker machines (e.g. locally on your laptop), it is common to use `cluster` mode to minimize network latency between -the drivers and the executors. Currently only YARN supports cluster mode for Python applications. +the drivers and the executors. Currently, standalone mode does not support cluster mode for Python +applications. For Python applications, simply pass a `.py` file in the place of `` instead of a JAR, and add Python `.zip`, `.egg` or `.py` files to the search path with `--py-files`. From c0364485e7cc26a12ead7d62964998b6872dc616 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Sun, 7 Aug 2016 09:06:59 +0100 Subject: [PATCH 1099/1470] [SPARK-16932][DOCS] Changed programming guide to not reference old accumulator API in Scala ## What changes were proposed in this pull request? In the programming guide, the accumulator section mixes up both the old and new APIs causing it to be confusing. This is not necessary for Scala, so all references to the old API are removed. For Java, it is somewhat fixed up except for the example of a custom accumulator because I don't think an API exists yet. Python has not currently implemented the new API. ## How was this patch tested? built doc locally Author: Bryan Cutler Closes #14516 from BryanCutler/fixup-accumulator-programming-guide-SPARK-15702. (cherry picked from commit b1ebe182ca10f6d6fdd427f4ea4a8f6cd229ccd1) Signed-off-by: Sean Owen --- docs/programming-guide.md | 41 ++++++++++++++++++++++++++------------- 1 file changed, 27 insertions(+), 14 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 888c12f18635c..5fcd4d3647cfe 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1348,17 +1348,17 @@ running stages (NOTE: this is not yet supported in Python). Accumulators in the Spark UI

    -An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks -running on a cluster can then add to it using the `add` method or the `+=` operator (in Scala and Python). -However, they cannot read its value. -Only the driver program can read the accumulator's value, using its `value` method. - -The code below shows an accumulator being used to add up the elements of an array: -
    +A numeric accumulator can be created by calling `SparkContext.longAccumulator()` or `SparkContext.doubleAccumulator()` +to accumulate values of type Long or Double, respectively. Tasks running on a cluster can then add to it using +the `add` method. However, they cannot read its value. Only the driver program can read the accumulator's value, +using its `value` method. + +The code below shows an accumulator being used to add up the elements of an array: + {% highlight scala %} scala> val accum = sc.longAccumulator("My Accumulator") accum: org.apache.spark.util.LongAccumulator = LongAccumulator(id: 0, name: Some(My Accumulator), value: 0) @@ -1395,14 +1395,21 @@ val myVectorAcc = new VectorAccumulatorV2 sc.register(myVectorAcc, "MyVectorAcc1") {% endhighlight %} -Note that, when programmers define their own type of AccumulatorV2, the resulting type can be same or not same with the elements added. +Note that, when programmers define their own type of AccumulatorV2, the resulting type can be different than that of the elements added.
    +A numeric accumulator can be created by calling `SparkContext.longAccumulator()` or `SparkContext.doubleAccumulator()` +to accumulate values of type Long or Double, respectively. Tasks running on a cluster can then add to it using +the `add` method. However, they cannot read its value. Only the driver program can read the accumulator's value, +using its `value` method. + +The code below shows an accumulator being used to add up the elements of an array: + {% highlight java %} -LongAccumulator accum = sc.sc().longAccumulator(); +LongAccumulator accum = jsc.sc().longAccumulator(); sc.parallelize(Arrays.asList(1, 2, 3, 4)).foreach(x -> accum.add(x)); // ... @@ -1412,8 +1419,8 @@ accum.value(); // returns 10 {% endhighlight %} -While this code used the built-in support for accumulators of type Integer, programmers can also -create their own types by subclassing [AccumulatorParam](api/java/index.html?org/apache/spark/AccumulatorParam.html). +Programmers can also create their own types by subclassing +[AccumulatorParam](api/java/index.html?org/apache/spark/AccumulatorParam.html). The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class representing mathematical vectors, we could write: @@ -1440,6 +1447,12 @@ a list by collecting together elements).
    +An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks +running on a cluster can then add to it using the `add` method or the `+=` operator. However, they cannot read its value. +Only the driver program can read the accumulator's value, using its `value` method. + +The code below shows an accumulator being used to add up the elements of an array: + {% highlight python %} >>> accum = sc.accumulator(0) Accumulator @@ -1485,15 +1498,15 @@ Accumulators do not change the lazy evaluation model of Spark. If they are being
    {% highlight scala %} -val accum = sc.accumulator(0) -data.map { x => accum += x; x } +val accum = sc.longAccumulator +data.map { x => accum.add(x); x } // Here, accum is still 0 because no actions have caused the map operation to be computed. {% endhighlight %}
    {% highlight java %} -LongAccumulator accum = sc.sc().longAccumulator(); +LongAccumulator accum = jsc.sc().longAccumulator(); data.map(x -> { accum.add(x); return f(x); }); // Here, accum is still 0 because no actions have caused the `map` to be computed. {% endhighlight %} From 3f8a95be8f096e13cade1af50f3d4e699ce45e07 Mon Sep 17 00:00:00 2001 From: keliang Date: Sun, 7 Aug 2016 09:28:32 +0100 Subject: [PATCH 1100/1470] =?UTF-8?q?[SPARK-16870][DOCS]=20Summary:add=20"?= =?UTF-8?q?spark.sql.broadcastTimeout"=20into=20docs/sql-programming-gu?= =?UTF-8?q?=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? default value for spark.sql.broadcastTimeout is 300s. and this property do not show in any docs of spark. so add "spark.sql.broadcastTimeout" into docs/sql-programming-guide.md to help people to how to fix this timeout error when it happenned ## How was this patch tested? not need (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) …ide.md JIRA_ID:SPARK-16870 Description:default value for spark.sql.broadcastTimeout is 300s. and this property do not show in any docs of spark. so add "spark.sql.broadcastTimeout" into docs/sql-programming-guide.md to help people to how to fix this timeout error when it happenned Test:done Author: keliang Closes #14477 from biglobster/keliang. (cherry picked from commit 1275f646964d2fdb5b96a9429760b4fac4340521) Signed-off-by: Sean Owen --- docs/sql-programming-guide.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 82b03a2b0edb2..3b01dc82bd7b5 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1147,6 +1147,15 @@ that these options will be deprecated in future release as more optimizations ar scheduled first). + + spark.sql.broadcastTimeout + 300 + +

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

    + + spark.sql.autoBroadcastJoinThreshold 10485760 (10 MB) From 739a333f6eeab030e01c028a77fade0e7a634768 Mon Sep 17 00:00:00 2001 From: Shivansh Date: Sun, 7 Aug 2016 09:30:18 +0100 Subject: [PATCH 1101/1470] [SPARK-16911] Fix the links in the programming guide ## What changes were proposed in this pull request? Fix the broken links in the programming guide of the Graphx Migration and understanding closures ## How was this patch tested? By running the test cases and checking the links. Author: Shivansh Closes #14503 from shiv4nsh/SPARK-16911. (cherry picked from commit 6c1ecb191bc086290e33d56b6a5706d962e84a3a) Signed-off-by: Sean Owen --- docs/graphx-programming-guide.md | 17 ----------- docs/programming-guide.md | 45 +---------------------------- docs/streaming-programming-guide.md | 45 ----------------------------- 3 files changed, 1 insertion(+), 106 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 2e9966c0a2b60..bf4b968eb8b78 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -67,23 +67,6 @@ operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operato [aggregateMessages](#aggregateMessages)) as well as an optimized variant of the [Pregel](#pregel) API. In addition, GraphX includes a growing collection of graph [algorithms](#graph_algorithms) and [builders](#graph_builders) to simplify graph analytics tasks. - -## Migrating from Spark 1.1 - -GraphX in Spark 1.2 contains a few user facing API changes: - -1. To improve performance we have introduced a new version of -[`mapReduceTriplets`][Graph.mapReduceTriplets] called -[`aggregateMessages`][Graph.aggregateMessages] which takes the messages previously returned from -[`mapReduceTriplets`][Graph.mapReduceTriplets] through a callback ([`EdgeContext`][EdgeContext]) -rather than by return value. -We are deprecating [`mapReduceTriplets`][Graph.mapReduceTriplets] and encourage users to consult -the [transition guide](#mrTripletsTransition). - -2. In Spark 1.0 and 1.1, the type signature of [`EdgeRDD`][EdgeRDD] switched from -`EdgeRDD[ED]` to `EdgeRDD[ED, VD]` to enable some caching optimizations. We have since discovered -a more elegant solution and have restored the type signature to the more natural `EdgeRDD[ED]` type. - # Getting Started To get started you first need to import Spark and GraphX into your project, as follows: diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 5fcd4d3647cfe..f82832905ef44 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1097,7 +1097,7 @@ for details. foreach(func) Run a function func on each element of the dataset. This is usually done for side effects such as updating an Accumulator or interacting with external storage systems. -
    Note: modifying variables other than Accumulators outside of the foreach() may result in undefined behavior. See Understanding closures for more details. +
    Note: modifying variables other than Accumulators outside of the foreach() may result in undefined behavior. See Understanding closures for more details. @@ -1544,49 +1544,6 @@ and then call `SparkContext.stop()` to tear it down. Make sure you stop the context within a `finally` block or the test framework's `tearDown` method, as Spark does not support two contexts running concurrently in the same program. -# Migrating from pre-1.0 Versions of Spark - -
    - -
    - -Spark 1.0 freezes the API of Spark Core for the 1.X series, in that any API available today that is -not marked "experimental" or "developer API" will be supported in future versions. -The only change for Scala users is that the grouping operations, e.g. `groupByKey`, `cogroup` and `join`, -have changed from returning `(Key, Seq[Value])` pairs to `(Key, Iterable[Value])`. - -
    - -
    - -Spark 1.0 freezes the API of Spark Core for the 1.X series, in that any API available today that is -not marked "experimental" or "developer API" will be supported in future versions. -Several changes were made to the Java API: - -* The Function classes in `org.apache.spark.api.java.function` became interfaces in 1.0, meaning that old - code that `extends Function` should `implement Function` instead. -* New variants of the `map` transformations, like `mapToPair` and `mapToDouble`, were added to create RDDs - of special data types. -* Grouping operations like `groupByKey`, `cogroup` and `join` have changed from returning - `(Key, List)` pairs to `(Key, Iterable)`. - -
    - -
    - -Spark 1.0 freezes the API of Spark Core for the 1.X series, in that any API available today that is -not marked "experimental" or "developer API" will be supported in future versions. -The only change for Python users is that the grouping operations, e.g. `groupByKey`, `cogroup` and `join`, -have changed from returning (key, list of values) pairs to (key, iterable of values). - -
    - -
    - -Migration guides are also available for [Spark Streaming](streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x), -[MLlib](ml-guide.html#migration-guide) and [GraphX](graphx-programming-guide.html#migrating-from-spark-091). - - # Where to Go from Here You can see some [example Spark programs](http://spark.apache.org/examples.html) on the Spark website. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 902df6ada8790..3d40b2c3136eb 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -2378,51 +2378,6 @@ additional effort may be necessary to achieve exactly-once semantics. There are *************************************************************************************************** *************************************************************************************************** -# Migration Guide from 0.9.1 or below to 1.x -Between Spark 0.9.1 and Spark 1.0, there were a few API changes made to ensure future API stability. -This section elaborates the steps required to migrate your existing code to 1.0. - -**Input DStreams**: All operations that create an input stream (e.g., `StreamingContext.socketStream`, `FlumeUtils.createStream`, etc.) now returns -[InputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.InputDStream) / -[ReceiverInputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.ReceiverInputDStream) -(instead of DStream) for Scala, and [JavaInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaInputDStream.html) / -[JavaPairInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairInputDStream.html) / -[JavaReceiverInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaReceiverInputDStream.html) / -[JavaPairReceiverInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.html) -(instead of JavaDStream) for Java. This ensures that functionality specific to input streams can -be added to these classes in the future without breaking binary compatibility. -Note that your existing Spark Streaming applications should not require any change -(as these new classes are subclasses of DStream/JavaDStream) but may require recompilation with Spark 1.0. - -**Custom Network Receivers**: Since the release to Spark Streaming, custom network receivers could be defined -in Scala using the class NetworkReceiver. However, the API was limited in terms of error handling -and reporting, and could not be used from Java. Starting Spark 1.0, this class has been -replaced by [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) which has -the following advantages. - -* Methods like `stop` and `restart` have been added to for better control of the lifecycle of a receiver. See -the [custom receiver guide](streaming-custom-receivers.html) for more details. -* Custom receivers can be implemented using both Scala and Java. - -To migrate your existing custom receivers from the earlier NetworkReceiver to the new Receiver, you have -to do the following. - -* Make your custom receiver class extend -[`org.apache.spark.streaming.receiver.Receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) -instead of `org.apache.spark.streaming.dstream.NetworkReceiver`. -* Earlier, a BlockGenerator object had to be created by the custom receiver, to which received data was -added for being stored in Spark. It had to be explicitly started and stopped from `onStart()` and `onStop()` -methods. The new Receiver class makes this unnecessary as it adds a set of methods named `store()` -that can be called to store the data in Spark. So, to migrate your custom network receiver, remove any -BlockGenerator object (does not exist any more in Spark 1.0 anyway), and use `store(...)` methods on -received data. - -**Actor-based Receivers**: The Actor-based Receiver APIs have been moved to [DStream Akka](https://github.com/spark-packages/dstream-akka). -Please refer to the project for more details. - -*************************************************************************************************** -*************************************************************************************************** - # Where to Go from Here * Additional guides - [Kafka Integration Guide](streaming-kafka-integration.html) From fd828e14ba968122c22a52c87fc3979dce853157 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 7 Aug 2016 12:20:07 +0100 Subject: [PATCH 1102/1470] [SPARK-16409][SQL] regexp_extract with optional groups causes NPE ## What changes were proposed in this pull request? regexp_extract actually returns null when it shouldn't when a regex matches but the requested optional group did not. This makes it return an empty string, as apparently designed. ## How was this patch tested? Additional unit test Author: Sean Owen Closes #14504 from srowen/SPARK-16409. (cherry picked from commit 8d8725208771a8815a60160a5a30dc6ea87a7e6a) Signed-off-by: Sean Owen --- python/pyspark/sql/functions.py | 3 +++ .../catalyst/expressions/regexpExpressions.scala | 13 +++++++++++-- .../org/apache/spark/sql/StringFunctionsSuite.scala | 8 ++++++++ 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index e422363ec1f55..8a01805ec831b 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1445,6 +1445,9 @@ def regexp_extract(str, pattern, idx): >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_extract('str', '(\d+)-(\d+)', 1).alias('d')).collect() [Row(d=u'100')] + >>> df = spark.createDataFrame([('aaaac',)], ['str']) + >>> df.select(regexp_extract('str', '(a+)(b)?(c)', 2).alias('d')).collect() + [Row(d=u'')] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.regexp_extract(_to_java_column(str), pattern, idx) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index be82b3b8f45f6..d25da3fd587b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -329,7 +329,12 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio val m = pattern.matcher(s.toString) if (m.find) { val mr: MatchResult = m.toMatchResult - UTF8String.fromString(mr.group(r.asInstanceOf[Int])) + val group = mr.group(r.asInstanceOf[Int]) + if (group == null) { // Pattern matched, but not optional group + UTF8String.EMPTY_UTF8 + } else { + UTF8String.fromString(group) + } } else { UTF8String.EMPTY_UTF8 } @@ -367,7 +372,11 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio ${termPattern}.matcher($subject.toString()); if (${matcher}.find()) { java.util.regex.MatchResult ${matchResult} = ${matcher}.toMatchResult(); - ${ev.value} = UTF8String.fromString(${matchResult}.group($idx)); + if (${matchResult}.group($idx) == null) { + ${ev.value} = UTF8String.EMPTY_UTF8; + } else { + ${ev.value} = UTF8String.fromString(${matchResult}.group($idx)); + } $setEvNotNull } else { ${ev.value} = UTF8String.EMPTY_UTF8; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 7bea2f6ad0dbb..73bd7e6414350 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -92,6 +92,14 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("300", "100") :: Row("400", "100") :: Row("400-400", "100") :: Nil) } + test("non-matching optional group") { + val df = Seq("aaaac").toDF("s") + checkAnswer( + df.select(regexp_extract($"s", "(a+)(b)?(c)", 2)), + Row("") + ) + } + test("string ascii function") { val df = Seq(("abc", "")).toDF("a", "b") checkAnswer( From f37ed6e703f287907667899cd8f4a7ccee65fa91 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 7 Aug 2016 20:51:54 +0100 Subject: [PATCH 1103/1470] [SPARK-16939][SQL] Fix build error by using `Tuple1` explicitly in StringFunctionsSuite ## What changes were proposed in this pull request? This PR aims to fix a build error on branch 1.6 at https://github.com/apache/spark/commit/8d8725208771a8815a60160a5a30dc6ea87a7e6a, but I think we had better have this consistently in master branch, too. It's because there exist other ongoing PR (https://github.com/apache/spark/pull/14525) about this. https://amplab.cs.berkeley.edu/jenkins/job/spark-branch-1.6-compile-maven-with-yarn-2.3/286/console ```scala [error] /home/jenkins/workspace/spark-branch-1.6-compile-maven-with-yarn-2.3/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala:82: value toDF is not a member of Seq[String] [error] val df = Seq("aaaac").toDF("s") [error] ^ ``` ## How was this patch tested? After passing Jenkins, run compilation test on branch 1.6. ``` build/mvn -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install ``` Author: Dongjoon Hyun Closes #14526 from dongjoon-hyun/SPARK-16939. (cherry picked from commit a16983c97b4c6539f97e5d26f163fed49872df2b) Signed-off-by: Sean Owen --- .../test/scala/org/apache/spark/sql/StringFunctionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 73bd7e6414350..6edd7b0c25b97 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -93,7 +93,7 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { } test("non-matching optional group") { - val df = Seq("aaaac").toDF("s") + val df = Seq(Tuple1("aaaac")).toDF("s") checkAnswer( df.select(regexp_extract($"s", "(a+)(b)?(c)", 2)), Row("") From ca0c6e6734583d9677251896f318a4457fb4bdbe Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 8 Aug 2016 22:26:44 +0800 Subject: [PATCH 1104/1470] [SPARK-16457][SQL] Fix Wrong Messages when CTAS with a Partition By Clause #### What changes were proposed in this pull request? When doing a CTAS with a Partition By clause, we got a wrong error message. For example, ```SQL CREATE TABLE gen__tmp PARTITIONED BY (key string) AS SELECT key, value FROM mytable1 ``` The error message we get now is like ``` Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 2, pos 0) ``` However, based on the code, the message we should get is like ``` Operation not allowed: A Create Table As Select (CTAS) statement is not allowed to create a partitioned table using Hive's file formats. Please use the syntax of "CREATE TABLE tableName USING dataSource OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a CTAS statement.(line 2, pos 0) ``` Currently, partitioning columns is part of the schema. This PR fixes the bug by changing the detection orders. #### How was this patch tested? Added test cases. Author: gatorsmile Closes #14113 from gatorsmile/ctas. (cherry picked from commit ab126909ce381842dbb057d480a1f9bee1b4f38e) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/SparkSqlParser.scala | 12 +++---- .../sql/hive/execution/SQLQuerySuite.scala | 36 +++++++++++++------ 2 files changed, 32 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3573a86d4e83f..91c01e1eecf0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -992,12 +992,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { selectQuery match { case Some(q) => - // Just use whatever is projected in the select statement as our schema - if (schema.nonEmpty) { - operationNotAllowed( - "Schema may not be specified in a Create Table As Select (CTAS) statement", - ctx) - } // Hive does not allow to use a CTAS statement to create a partitioned table. if (tableDesc.partitionColumnNames.nonEmpty) { val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " + @@ -1007,6 +1001,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { "CTAS statement." operationNotAllowed(errorMessage, ctx) } + // Just use whatever is projected in the select statement as our schema + if (schema.nonEmpty) { + operationNotAllowed( + "Schema may not be specified in a Create Table As Select (CTAS) statement", + ctx) + } val hasStorageProperties = (ctx.createFileFormat != null) || (ctx.rowFormat != null) if (conf.convertCTAS && !hasStorageProperties) { 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 c4d9e0aee911f..404b27596fa4e 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 @@ -642,19 +642,35 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("specifying the column list for CTAS") { - Seq((1, "111111"), (2, "222222")).toDF("key", "value").createOrReplaceTempView("mytable1") + withTempView("mytable1") { + Seq((1, "111111"), (2, "222222")).toDF("key", "value").createOrReplaceTempView("mytable1") + withTable("gen__tmp") { + sql("create table gen__tmp as select key as a, value as b from mytable1") + checkAnswer( + sql("SELECT a, b from gen__tmp"), + sql("select key, value from mytable1").collect()) + } - sql("create table gen__tmp as select key as a, value as b from mytable1") - checkAnswer( - sql("SELECT a, b from gen__tmp"), - sql("select key, value from mytable1").collect()) - sql("DROP TABLE gen__tmp") + withTable("gen__tmp") { + val e = intercept[AnalysisException] { + sql("create table gen__tmp(a int, b string) as select key, value from mytable1") + }.getMessage + assert(e.contains("Schema may not be specified in a Create Table As Select (CTAS)")) + } - intercept[AnalysisException] { - sql("create table gen__tmp(a int, b string) as select key, value from mytable1") + withTable("gen__tmp") { + val e = intercept[AnalysisException] { + sql( + """ + |CREATE TABLE gen__tmp + |PARTITIONED BY (key string) + |AS SELECT key, value FROM mytable1 + """.stripMargin) + }.getMessage + assert(e.contains("A Create Table As Select (CTAS) statement is not allowed to " + + "create a partitioned table using Hive's file formats")) + } } - - sql("drop table mytable1") } test("command substitution") { From b8a7958aaf5d0898b2759f5ee2f0ab5194d42edb Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 8 Aug 2016 22:34:28 +0800 Subject: [PATCH 1105/1470] [SPARK-16936][SQL] Case Sensitivity Support for Refresh Temp Table ### What changes were proposed in this pull request? Currently, the `refreshTable` API is always case sensitive. When users use the view name without the exact case match, the API silently ignores the call. Users might expect the command has been successfully completed. However, when users run the subsequent SQL commands, they might still get the exception, like ``` Job aborted due to stage failure: Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 7, localhost): java.io.FileNotFoundException: File file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-bd4b9ea6-9aec-49c5-8f05-01cff426211e/part-r-00000-0c84b915-c032-4f2e-abf5-1d48fdbddf38.snappy.parquet does not exist ``` This PR is to fix the issue. ### How was this patch tested? Added a test case. Author: gatorsmile Closes #14523 from gatorsmile/refreshTempTable. (cherry picked from commit 5959df217df53196607b7fa744cdc2b36311360e) Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/SessionCatalog.scala | 4 +-- .../spark/sql/execution/SparkSqlParser.scala | 2 +- .../apache/spark/sql/MetadataCacheSuite.scala | 25 +++++++++++++++++++ 3 files changed, 28 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 977426a221fcc..8db817cb152ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -495,7 +495,7 @@ class SessionCatalog( // If the database is defined, this is definitely not a temp table. // If the database is not defined, there is a good chance this is a temp table. if (name.database.isEmpty) { - tempTables.get(name.table).foreach(_.refresh()) + tempTables.get(formatTableName(name.table)).foreach(_.refresh()) } } @@ -512,7 +512,7 @@ class SessionCatalog( * For testing only. */ private[catalog] def getTempTable(name: String): Option[LogicalPlan] = synchronized { - tempTables.get(name) + tempTables.get(formatTableName(name)) } // ---------------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 91c01e1eecf0f..f3cd9236e086a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1209,7 +1209,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * For example: * {{{ - * CREATE [TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name + * CREATE [OR REPLACE] [TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name * [(column_name [COMMENT column_comment], ...) ] * [COMMENT view_comment] * [TBLPROPERTIES (property_name = property_value, ...)] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala index eacf254cd183d..98aa447fc0560 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.io.File import org.apache.spark.SparkException +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext /** @@ -85,4 +86,28 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { assert(newCount > 0 && newCount < 100) }} } + + test("case sensitivity support in temporary view refresh") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTempView("view_refresh") { + withTempPath { (location: File) => + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.parquet(location.getAbsolutePath) + + // Read the directory in + spark.read.parquet(location.getAbsolutePath).createOrReplaceTempView("view_refresh") + + // Delete a file + deleteOneFileInDirectory(location) + intercept[SparkException](sql("select count(*) from view_refresh").first()) + + // Refresh and we should be able to read it again. + spark.catalog.refreshTable("vIeW_reFrEsH") + val newCount = sql("select count(*) from view_refresh").first().getLong(0) + assert(newCount > 0 && newCount < 100) + } + } + } + } } From 69e278ebc503eb037d51d28108541032e2cffbf9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 8 Aug 2016 10:34:54 -0700 Subject: [PATCH 1106/1470] [SPARK-16586][CORE] Handle JVM errors printed to stdout. Some very rare JVM errors are printed to stdout, and that confuses the code in spark-class. So add a check so that those cases are detected and the proper error message is shown to the user. Tested by running spark-submit after setting "ulimit -v 32000". Closes #14231 Author: Marcelo Vanzin Closes #14508 from vanzin/SPARK-16586. (cherry picked from commit 1739e75fecf0cb9507dc950bba1716f40be1e609) Signed-off-by: Marcelo Vanzin --- bin/spark-class | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/bin/spark-class b/bin/spark-class index 658e076bc0462..377c8d1add3f6 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -80,6 +80,15 @@ done < <(build_command "$@") COUNT=${#CMD[@]} LAST=$((COUNT - 1)) LAUNCHER_EXIT_CODE=${CMD[$LAST]} + +# Certain JVM failures result in errors being printed to stdout (instead of stderr), which causes +# the code that parses the output of the launcher to get confused. In those cases, check if the +# exit code is an integer, and if it's not, handle it as a special error case. +if ! [[ $LAUNCHER_EXIT_CODE =~ ^[0-9]+$ ]]; then + echo "${CMD[@]}" | head -n-1 1>&2 + exit 1 +fi + if [ $LAUNCHER_EXIT_CODE != 0 ]; then exit $LAUNCHER_EXIT_CODE fi From 9748a29289cf35a739854f17e72d039009abedbe Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 8 Aug 2016 12:52:04 -0700 Subject: [PATCH 1107/1470] [SPARK-16953] Make requestTotalExecutors public Developer API to be consistent with requestExecutors/killExecutors ## What changes were proposed in this pull request? RequestExecutors and killExecutor are public developer APIs for managing the number of executors allocated to the SparkContext. For consistency, requestTotalExecutors should also be a public Developer API, as it provides similar functionality. In fact, using requestTotalExecutors is more convenient that requestExecutors as the former is idempotent and the latter is not. Author: Tathagata Das Closes #14541 from tdas/SPARK-16953. (cherry picked from commit 8650239050ade91689ffa0672ea094de2594e37c) Signed-off-by: Tathagata Das --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d3e8de31b7f84..699dc516b17af 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1472,7 +1472,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * This includes running, pending, and completed tasks. * @return whether the request is acknowledged by the cluster manager. */ - private[spark] override def requestTotalExecutors( + @DeveloperApi + override def requestTotalExecutors( numExecutors: Int, localityAwareTasks: Int, hostToLocalTaskCount: scala.collection.immutable.Map[String, Int] From 6fc54b776419317dc55754a76b68a5ba7eecdcf3 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 8 Aug 2016 16:07:51 -0700 Subject: [PATCH 1108/1470] Update docs to include SASL support for RPC ## What changes were proposed in this pull request? Update docs to include SASL support for RPC Evidence: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala#L63 ## How was this patch tested? Docs change only Author: Michael Gummelt Closes #14549 from mgummelt/sasl. (cherry picked from commit 53d1c7877967f03cc9c8c7e7394f380d1bbefc27) Signed-off-by: Reynold Xin --- docs/configuration.md | 7 ++++--- docs/security.md | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index bf10b2481951a..8facd0ecf367a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1204,7 +1204,7 @@ Apart from these, the following properties are also available, and may be useful false Whether to use dynamic resource allocation, which scales the number of executors registered - with this application up and down based on the workload. + with this application up and down based on the workload. For more detail, see the description here.

    @@ -1345,8 +1345,9 @@ Apart from these, the following properties are also available, and may be useful spark.authenticate.enableSaslEncryption false - Enable encrypted communication when authentication is enabled. This option is currently - only supported by the block transfer service. + Enable encrypted communication when authentication is + enabled. This is supported by the block transfer service and the + RPC endpoints. diff --git a/docs/security.md b/docs/security.md index d2708a80703ec..baadfefbec826 100644 --- a/docs/security.md +++ b/docs/security.md @@ -27,7 +27,8 @@ If your applications are using event logging, the directory where the event logs ## Encryption -Spark supports SSL for HTTP protocols. SASL encryption is supported for the block transfer service. +Spark supports SSL for HTTP protocols. SASL encryption is supported for the block transfer service +and the RPC endpoints. Encryption is not yet supported for data stored by Spark in temporary local storage, such as shuffle files, cached data, and other application files. If encrypting this data is desired, a workaround is From 601c649d0134e6791f1c0e0aaa25d6aad3c541d4 Mon Sep 17 00:00:00 2001 From: Alice Date: Mon, 8 Aug 2016 18:00:04 -0700 Subject: [PATCH 1109/1470] [SPARK-16563][SQL] fix spark sql thrift server FetchResults bug ## What changes were proposed in this pull request? Add a constant iterator which point to head of result. The header will be used to reset iterator when fetch result from first row repeatedly. JIRA ticket https://issues.apache.org/jira/browse/SPARK-16563 ## How was this patch tested? This bug was found when using Cloudera HUE connecting to spark sql thrift server, currently SQL statement result can be only fetched for once. The fix was tested manually with Cloudera HUE, With this fix, HUE can fetch spark SQL results repeatedly through thrift server. Author: Alice Author: Alice Closes #14218 from alicegugu/SparkSQLFetchResultsBug. (cherry picked from commit e17a76efdb44837c38388a4d0e62436065cd4dc9) Signed-off-by: Reynold Xin --- .../SparkExecuteStatementOperation.scala | 12 +++++ .../HiveThriftServer2Suites.scala | 48 +++++++++++++++++++ 2 files changed, 60 insertions(+) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index e8bcdd76efd7a..b2717ec54e693 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -51,6 +51,7 @@ private[hive] class SparkExecuteStatementOperation( private var result: DataFrame = _ private var iter: Iterator[SparkRow] = _ + private var iterHeader: Iterator[SparkRow] = _ private var dataTypes: Array[DataType] = _ private var statementId: String = _ @@ -110,6 +111,14 @@ private[hive] class SparkExecuteStatementOperation( assertState(OperationState.FINISHED) setHasResultSet(true) val resultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) + + // Reset iter to header when fetching start from first row + if (order.equals(FetchOrientation.FETCH_FIRST)) { + val (ita, itb) = iterHeader.duplicate + iter = ita + iterHeader = itb + } + if (!iter.hasNext) { resultRowSet } else { @@ -228,6 +237,9 @@ private[hive] class SparkExecuteStatementOperation( result.collect().iterator } } + val (itra, itrb) = iter.duplicate + iterHeader = itra + iter = itrb dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray } catch { case e: HiveSQLException => 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 e388c2a082f18..8f2c4fafa0b43 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 @@ -36,6 +36,8 @@ import org.apache.hive.service.auth.PlainSaslHelper import org.apache.hive.service.cli.GetInfoType import org.apache.hive.service.cli.thrift.TCLIService.Client import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient +import org.apache.hive.service.cli.FetchOrientation +import org.apache.hive.service.cli.FetchType import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket import org.scalatest.BeforeAndAfterAll @@ -91,6 +93,52 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } } + test("SPARK-16563 ThriftCLIService FetchResults repeat fetching result") { + withCLIServiceClient { client => + val user = System.getProperty("user.name") + val sessionHandle = client.openSession(user, "") + + withJdbcStatement { statement => + val queries = Seq( + "DROP TABLE IF EXISTS test_16563", + "CREATE TABLE test_16563(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_16563") + + queries.foreach(statement.execute) + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val operationHandle = client.executeStatement( + sessionHandle, + "SELECT * FROM test_16563", + confOverlay) + + // Fetch result first time + assertResult(5, "Fetching result first time from next row") { + + val rows_next = client.fetchResults( + operationHandle, + FetchOrientation.FETCH_NEXT, + 1000, + FetchType.QUERY_OUTPUT) + + rows_next.numRows() + } + + // Fetch result second time from first row + assertResult(5, "Repeat fetching result from first row") { + + val rows_first = client.fetchResults( + operationHandle, + FetchOrientation.FETCH_FIRST, + 1000, + FetchType.QUERY_OUTPUT) + + rows_first.numRows() + } + statement.executeQuery("DROP TABLE IF EXISTS test_16563") + } + } + } + test("JDBC query execution") { withJdbcStatement { statement => val queries = Seq( From bbbd3cb15d8c3c096fb3d81be8e447294a248062 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 9 Aug 2016 10:23:54 +0800 Subject: [PATCH 1110/1470] [SPARK-16610][SQL] Add `orc.compress` as an alias for `compression` option. ## What changes were proposed in this pull request? For ORC source, Spark SQL has a writer option `compression`, which is used to set the codec and its value will be also set to `orc.compress` (the orc conf used for codec). However, if a user only set `orc.compress` in the writer option, we should not use the default value of `compression` (snappy) as the codec. Instead, we should respect the value of `orc.compress`. This PR makes ORC data source not ignoring `orc.compress` when `comperssion` is unset. So, here is the behaviour, 1. Check `compression` and use this if it is set. 2. If `compression` is not set, check `orc.compress` and use it. 3. If `compression` and `orc.compress` are not set, then use the default snappy. ## How was this patch tested? Unit test in `OrcQuerySuite`. Author: hyukjinkwon Closes #14518 from HyukjinKwon/SPARK-16610. (cherry picked from commit bb2b9d0a428b86bf366ee9916e26402f8c00912f) Signed-off-by: Wenchen Fan --- .../spark/sql/hive/orc/OrcOptions.scala | 12 +++++++--- .../spark/sql/hive/orc/OrcQuerySuite.scala | 23 +++++++++++++++++++ 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala index 91cf0dc960d58..c2a126d3bf9c0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.hive.orc /** * Options for the ORC data source. */ -private[orc] class OrcOptions( - @transient private val parameters: Map[String, String]) +private[orc] class OrcOptions(@transient private val parameters: Map[String, String]) extends Serializable { import OrcOptions._ @@ -31,7 +30,14 @@ private[orc] class OrcOptions( * Acceptable values are defined in [[shortOrcCompressionCodecNames]]. */ val compressionCodec: String = { - val codecName = parameters.getOrElse("compression", "snappy").toLowerCase + // `orc.compress` is a ORC configuration. So, here we respect this as an option but + // `compression` has higher precedence than `orc.compress`. It means if both are set, + // we will use `compression`. + val orcCompressionConf = parameters.get(OrcRelation.ORC_COMPRESSION) + val codecName = parameters + .get("compression") + .orElse(orcCompressionConf) + .getOrElse("snappy").toLowerCase if (!shortOrcCompressionCodecNames.contains(codecName)) { val availableCodecs = shortOrcCompressionCodecNames.keys.map(_.toLowerCase) throw new IllegalArgumentException(s"Codec [$codecName] " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 46595ee8186aa..8f8768217788e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -160,6 +160,29 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } } + test("SPARK-16610: Respect orc.compress option when compression is unset") { + // Respect `orc.compress`. + withTempPath { file => + spark.range(0, 10).write + .option("orc.compress", "ZLIB") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + } + + // `compression` overrides `orc.compress`. + withTempPath { file => + spark.range(0, 10).write + .option("compression", "ZLIB") + .option("orc.compress", "SNAPPY") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + } + } + // Hive supports zlib, snappy and none for Hive 1.2.1. test("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") { withTempPath { file => From 41d9dcac3e1d3ee1a27fe094ebb60c1c18d6bcff Mon Sep 17 00:00:00 2001 From: Mariusz Strzelecki Date: Tue, 9 Aug 2016 09:44:43 -0700 Subject: [PATCH 1111/1470] [SPARK-16950] [PYSPARK] fromOffsets parameter support in KafkaUtils.createDirectStream for python3 ## What changes were proposed in this pull request? Ability to use KafkaUtils.createDirectStream with starting offsets in python 3 by using java.lang.Number instead of Long during param mapping in scala helper. This allows py4j to pass Integer or Long to the map and resolves ClassCastException problems. ## How was this patch tested? unit tests jerryshao - could you please look at this PR? Author: Mariusz Strzelecki Closes #14540 from szczeles/kafka_pyspark. (cherry picked from commit 29081b587f3423bf5a3e0066357884d0c26a04bf) Signed-off-by: Davies Liu --- .../apache/spark/streaming/kafka/KafkaUtils.scala | 8 ++++---- python/pyspark/streaming/kafka.py | 3 +++ python/pyspark/streaming/tests.py | 12 +++--------- 3 files changed, 10 insertions(+), 13 deletions(-) diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index edaafb912c5c5..b17e198077949 100644 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.kafka import java.io.OutputStream -import java.lang.{Integer => JInt, Long => JLong} +import java.lang.{Integer => JInt, Long => JLong, Number => JNumber} import java.nio.charset.StandardCharsets import java.util.{List => JList, Map => JMap, Set => JSet} @@ -682,7 +682,7 @@ private[kafka] class KafkaUtilsPythonHelper { jssc: JavaStreamingContext, kafkaParams: JMap[String, String], topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JLong]): JavaDStream[(Array[Byte], Array[Byte])] = { + fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[(Array[Byte], Array[Byte])] = { val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) new JavaDStream(createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler)) @@ -692,7 +692,7 @@ private[kafka] class KafkaUtilsPythonHelper { jssc: JavaStreamingContext, kafkaParams: JMap[String, String], topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JLong]): JavaDStream[Array[Byte]] = { + fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[Array[Byte]] = { val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => new PythonMessageAndMetadata(mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) val stream = createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler). @@ -704,7 +704,7 @@ private[kafka] class KafkaUtilsPythonHelper { jssc: JavaStreamingContext, kafkaParams: JMap[String, String], topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JLong], + fromOffsets: JMap[TopicAndPartition, JNumber], messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): DStream[V] = { val currentFromOffsets = if (!fromOffsets.isEmpty) { diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index 2c1a667fc80c4..bf27d8047a753 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -287,6 +287,9 @@ def __eq__(self, other): def __ne__(self, other): return not self.__eq__(other) + def __hash__(self): + return (self._topic, self._partition).__hash__() + class Broker(object): """ diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 360ba1e7167cb..5ac007cd598b9 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -41,6 +41,9 @@ else: import unittest +if sys.version >= "3": + long = int + from pyspark.context import SparkConf, SparkContext, RDD from pyspark.storagelevel import StorageLevel from pyspark.streaming.context import StreamingContext @@ -1058,7 +1061,6 @@ def test_kafka_direct_stream(self): stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) self._validateStreamResult(sendData, stream) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_from_offset(self): """Test the Python direct Kafka stream API with start offset specified.""" topic = self._randomTopic() @@ -1072,7 +1074,6 @@ def test_kafka_direct_stream_from_offset(self): stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams, fromOffsets) self._validateStreamResult(sendData, stream) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd(self): """Test the Python direct Kafka RDD API.""" topic = self._randomTopic() @@ -1085,7 +1086,6 @@ def test_kafka_rdd(self): rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) self._validateRddResult(sendData, rdd) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd_with_leaders(self): """Test the Python direct Kafka RDD API with leaders.""" topic = self._randomTopic() @@ -1100,7 +1100,6 @@ def test_kafka_rdd_with_leaders(self): rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders) self._validateRddResult(sendData, rdd) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd_get_offsetRanges(self): """Test Python direct Kafka RDD get OffsetRanges.""" topic = self._randomTopic() @@ -1113,7 +1112,6 @@ def test_kafka_rdd_get_offsetRanges(self): rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) self.assertEqual(offsetRanges, rdd.offsetRanges()) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_foreach_get_offsetRanges(self): """Test the Python direct Kafka stream foreachRDD get offsetRanges.""" topic = self._randomTopic() @@ -1138,7 +1136,6 @@ def getOffsetRanges(_, rdd): self.assertEqual(offsetRanges, [OffsetRange(topic, 0, long(0), long(6))]) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_transform_get_offsetRanges(self): """Test the Python direct Kafka stream transform get offsetRanges.""" topic = self._randomTopic() @@ -1176,7 +1173,6 @@ def test_topic_and_partition_equality(self): self.assertNotEqual(topic_and_partition_a, topic_and_partition_c) self.assertNotEqual(topic_and_partition_a, topic_and_partition_d) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_transform_with_checkpoint(self): """Test the Python direct Kafka stream transform with checkpoint correctly recovered.""" topic = self._randomTopic() @@ -1225,7 +1221,6 @@ def setup(): finally: shutil.rmtree(tmpdir) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd_message_handler(self): """Test Python direct Kafka RDD MessageHandler.""" topic = self._randomTopic() @@ -1242,7 +1237,6 @@ def getKeyAndDoubleMessage(m): messageHandler=getKeyAndDoubleMessage) self._validateRddResult({"aa": 1, "bb": 1, "cc": 2}, rdd) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_message_handler(self): """Test the Python direct Kafka stream MessageHandler.""" topic = self._randomTopic() From 44115e90ef2a80d8ecf3965b97ce7bee21e29158 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 9 Aug 2016 11:21:45 -0700 Subject: [PATCH 1112/1470] [SPARK-16956] Make ApplicationState.MAX_NUM_RETRY configurable ## What changes were proposed in this pull request? This patch introduces a new configuration, `spark.deploy.maxExecutorRetries`, to let users configure an obscure behavior in the standalone master where the master will kill Spark applications which have experienced too many back-to-back executor failures. The current setting is a hardcoded constant (10); this patch replaces that with a new cluster-wide configuration. **Background:** This application-killing was added in 6b5980da796e0204a7735a31fb454f312bc9daac (from September 2012) and I believe that it was designed to prevent a faulty application whose executors could never launch from DOS'ing the Spark cluster via an infinite series of executor launch attempts. In a subsequent patch (#1360), this feature was refined to prevent applications which have running executors from being killed by this code path. **Motivation for making this configurable:** Previously, if a Spark Standalone application experienced more than `ApplicationState.MAX_NUM_RETRY` executor failures and was left with no executors running then the Spark master would kill that application, but this behavior is problematic in environments where the Spark executors run on unstable infrastructure and can all simultaneously die. For instance, if your Spark driver runs on an on-demand EC2 instance while all workers run on ephemeral spot instances then it's possible for all executors to die at the same time while the driver stays alive. In this case, it may be desirable to keep the Spark application alive so that it can recover once new workers and executors are available. In order to accommodate this use-case, this patch modifies the Master to never kill faulty applications if `spark.deploy.maxExecutorRetries` is negative. I'd like to merge this patch into master, branch-2.0, and branch-1.6. ## How was this patch tested? I tested this manually using `spark-shell` and `local-cluster` mode. This is a tricky feature to unit test and historically this code has not changed very often, so I'd prefer to skip the additional effort of adding a testing framework and would rather rely on manual tests and review for now. Author: Josh Rosen Closes #14544 from JoshRosen/add-setting-for-max-executor-failures. (cherry picked from commit b89b3a5c8e391fcaebe7ef3c77ef16bb9431d6ab) Signed-off-by: Josh Rosen --- .../spark/deploy/master/ApplicationState.scala | 2 -- .../org/apache/spark/deploy/master/Master.scala | 7 ++++++- docs/spark-standalone.md | 15 +++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index 37bfcdfdf4777..097728c821570 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -22,6 +22,4 @@ private[master] object ApplicationState extends Enumeration { type ApplicationState = Value val WAITING, RUNNING, FINISHED, FAILED, KILLED, UNKNOWN = Value - - val MAX_NUM_RETRY = 10 } 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 fded8475a0916..dfffc47703ab4 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 @@ -58,6 +58,7 @@ private[deploy] class Master( private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") + private val MAX_EXECUTOR_RETRIES = conf.getInt("spark.deploy.maxExecutorRetries", 10) val workers = new HashSet[WorkerInfo] val idToApp = new HashMap[String, ApplicationInfo] @@ -265,7 +266,11 @@ private[deploy] class Master( val normalExit = exitStatus == Some(0) // Only retry certain number of times so we don't go into an infinite loop. - if (!normalExit && appInfo.incrementRetryCount() >= ApplicationState.MAX_NUM_RETRY) { + // Important note: this code path is not exercised by tests, so be very careful when + // changing this `if` condition. + if (!normalExit + && appInfo.incrementRetryCount() >= MAX_EXECUTOR_RETRIES + && MAX_EXECUTOR_RETRIES >= 0) { // < 0 disables this application-killing path val execs = appInfo.executors.values if (!execs.exists(_.state == ExecutorState.RUNNING)) { logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index c864c9030835e..5ae63fe4e6e07 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -195,6 +195,21 @@ SPARK_MASTER_OPTS supports the following system properties: the whole cluster by default.
    + + spark.deploy.maxExecutorRetries + 10 + + Limit on the maximum number of back-to-back executor failures that can occur before the + standalone cluster manager removes a faulty application. An application will never be removed + if it has any running executors. If an application experiences more than + spark.deploy.maxExecutorRetries failures in a row, no executors + successfully start running in between those failures, and the application has no running + executors then the standalone cluster manager will remove the application and mark it as failed. + To disable this automatic removal, set spark.deploy.maxExecutorRetries to + -1. +
    + + spark.worker.timeout 60 From 2d136dba415a40a04598068ac2cea0490a6fd091 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 9 Aug 2016 10:04:36 -0700 Subject: [PATCH 1113/1470] [SPARK-16905] SQL DDL: MSCK REPAIR TABLE MSCK REPAIR TABLE could be used to recover the partitions in external catalog based on partitions in file system. Another syntax is: ALTER TABLE table RECOVER PARTITIONS The implementation in this PR will only list partitions (not the files with a partition) in driver (in parallel if needed). Added unit tests for it and Hive compatibility test suite. Author: Davies Liu Closes #14500 from davies/repair_table. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 6 +- .../spark/sql/execution/SparkSqlParser.scala | 27 ++++ .../spark/sql/execution/command/ddl.scala | 116 +++++++++++++++++- .../execution/command/DDLCommandSuite.scala | 8 ++ .../sql/execution/command/DDLSuite.scala | 49 ++++++++ .../spark/sql/hive/HiveDDLCommandSuite.scala | 10 +- 6 files changed, 209 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index de98a871b3358..279a1cee0ad21 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -84,6 +84,7 @@ statement | ALTER VIEW tableIdentifier DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation + | ALTER TABLE tableIdentifier RECOVER PARTITIONS #recoverPartitions | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? tableIdentifier #dropTable | CREATE (OR REPLACE)? TEMPORARY? VIEW (IF NOT EXISTS)? tableIdentifier @@ -121,6 +122,7 @@ statement | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData | TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable + | MSCK REPAIR TABLE tableIdentifier #repairTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration @@ -154,7 +156,6 @@ unsupportedHiveNativeCommands | kw1=UNLOCK kw2=DATABASE | kw1=CREATE kw2=TEMPORARY kw3=MACRO | kw1=DROP kw2=TEMPORARY kw3=MACRO - | kw1=MSCK kw2=REPAIR kw3=TABLE | kw1=ALTER kw2=TABLE tableIdentifier kw3=NOT kw4=CLUSTERED | kw1=ALTER kw2=TABLE tableIdentifier kw3=CLUSTERED kw4=BY | kw1=ALTER kw2=TABLE tableIdentifier kw3=NOT kw4=SORTED @@ -646,7 +647,7 @@ nonReserved | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT | DBPROPERTIES | DFS | TRUNCATE | COMPUTE | LIST | STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER - | REVOKE | GRANT | LOCK | UNLOCK | MSCK | REPAIR | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE + | REVOKE | GRANT | LOCK | UNLOCK | MSCK | REPAIR | RECOVER | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE | ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEX | INDEXES | LOCKS | OPTION | LOCAL | INPATH | ASC | DESC | LIMIT | RENAME | SETS | AT | NULLS | OVERWRITE | ALL | ALTER | AS | BETWEEN | BY | CREATE | DELETE @@ -859,6 +860,7 @@ LOCK: 'LOCK'; UNLOCK: 'UNLOCK'; MSCK: 'MSCK'; REPAIR: 'REPAIR'; +RECOVER: 'RECOVER'; EXPORT: 'EXPORT'; IMPORT: 'IMPORT'; LOAD: 'LOAD'; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index f3cd9236e086a..d6185d0bfce14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -405,6 +405,20 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } + /** + * Create a [[AlterTableRecoverPartitionsCommand]] command. + * + * For example: + * {{{ + * MSCK REPAIR TABLE tablename + * }}} + */ + override def visitRepairTable(ctx: RepairTableContext): LogicalPlan = withOrigin(ctx) { + AlterTableRecoverPartitionsCommand( + visitTableIdentifier(ctx.tableIdentifier), + "MSCK REPAIR TABLE") + } + /** * Convert a table property list into a key-value map. * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. @@ -763,6 +777,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.EXISTS != null) } + /** + * Create an [[AlterTableDiscoverPartitionsCommand]] command + * + * For example: + * {{{ + * ALTER TABLE table RECOVER PARTITIONS; + * }}} + */ + override def visitRecoverPartitions( + ctx: RecoverPartitionsContext): LogicalPlan = withOrigin(ctx) { + AlterTableRecoverPartitionsCommand(visitTableIdentifier(ctx.tableIdentifier)) + } + /** * Create an [[AlterTableSetLocationCommand]] command * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 226f61ef404ae..aac70e90b883c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -17,19 +17,24 @@ package org.apache.spark.sql.execution.command +import scala.collection.GenSeq +import scala.collection.parallel.ForkJoinTaskSupport +import scala.concurrent.forkjoin.ForkJoinPool import scala.util.control.NonFatal +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable} -import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, CatalogTableType, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTablePartition, CatalogTableType, SessionCatalog} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.types._ - // Note: The definition of these commands are based on the ones described in // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL @@ -424,6 +429,111 @@ case class AlterTableDropPartitionCommand( } +/** + * Recover Partitions in ALTER TABLE: recover all the partition in the directory of a table and + * update the catalog. + * + * The syntax of this command is: + * {{{ + * ALTER TABLE table RECOVER PARTITIONS; + * MSCK REPAIR TABLE table; + * }}} + */ +case class AlterTableRecoverPartitionsCommand( + tableName: TableIdentifier, + cmd: String = "ALTER TABLE RECOVER PARTITIONS") extends RunnableCommand { + override def run(spark: SparkSession): Seq[Row] = { + val catalog = spark.sessionState.catalog + if (!catalog.tableExists(tableName)) { + throw new AnalysisException(s"Table $tableName in $cmd does not exist.") + } + val table = catalog.getTableMetadata(tableName) + if (catalog.isTemporaryTable(tableName)) { + throw new AnalysisException( + s"Operation not allowed: $cmd on temporary tables: $tableName") + } + if (DDLUtils.isDatasourceTable(table)) { + throw new AnalysisException( + s"Operation not allowed: $cmd on datasource tables: $tableName") + } + if (table.tableType != CatalogTableType.EXTERNAL) { + throw new AnalysisException( + s"Operation not allowed: $cmd only works on external tables: $tableName") + } + if (!DDLUtils.isTablePartitioned(table)) { + throw new AnalysisException( + s"Operation not allowed: $cmd only works on partitioned tables: $tableName") + } + if (table.storage.locationUri.isEmpty) { + throw new AnalysisException( + s"Operation not allowed: $cmd only works on table with location provided: $tableName") + } + + val root = new Path(table.storage.locationUri.get) + val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) + // Dummy jobconf to get to the pathFilter defined in configuration + // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) + val jobConf = new JobConf(spark.sparkContext.hadoopConfiguration, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + val partitionSpecsAndLocs = scanPartitions( + spark, fs, pathFilter, root, Map(), table.partitionColumnNames.map(_.toLowerCase)) + val parts = partitionSpecsAndLocs.map { case (spec, location) => + // inherit table storage format (possibly except for location) + CatalogTablePartition(spec, table.storage.copy(locationUri = Some(location.toUri.toString))) + } + spark.sessionState.catalog.createPartitions(tableName, + parts.toArray[CatalogTablePartition], ignoreIfExists = true) + Seq.empty[Row] + } + + @transient private lazy val evalTaskSupport = new ForkJoinTaskSupport(new ForkJoinPool(8)) + + private def scanPartitions( + spark: SparkSession, + fs: FileSystem, + filter: PathFilter, + path: Path, + spec: TablePartitionSpec, + partitionNames: Seq[String]): GenSeq[(TablePartitionSpec, Path)] = { + if (partitionNames.length == 0) { + return Seq(spec -> path) + } + + val statuses = fs.listStatus(path) + val threshold = spark.conf.get("spark.rdd.parallelListingThreshold", "10").toInt + val statusPar: GenSeq[FileStatus] = + if (partitionNames.length > 1 && statuses.length > threshold || partitionNames.length > 2) { + val parArray = statuses.par + parArray.tasksupport = evalTaskSupport + parArray + } else { + statuses + } + statusPar.flatMap { st => + val name = st.getPath.getName + if (st.isDirectory && name.contains("=")) { + val ps = name.split("=", 2) + val columnName = PartitioningUtils.unescapePathName(ps(0)).toLowerCase + // TODO: Validate the value + val value = PartitioningUtils.unescapePathName(ps(1)) + // comparing with case-insensitive, but preserve the case + if (columnName == partitionNames(0)) { + scanPartitions( + spark, fs, filter, st.getPath, spec ++ Map(columnName -> value), partitionNames.drop(1)) + } else { + logWarning(s"expect partition column ${partitionNames(0)}, but got ${ps(0)}, ignore it") + Seq() + } + } else { + if (name != "_SUCCESS" && name != "_temporary" && !name.startsWith(".")) { + logWarning(s"ignore ${new Path(path, name)}") + } + Seq() + } + } + } +} + /** * A command that sets the location of a table or a partition. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 7b96f4c99ab5a..8d74884df9273 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -564,6 +564,14 @@ class DDLCommandSuite extends PlanTest { comparePlans(parsed2, expected2) } + test("alter table: recover partitions") { + val sql = "ALTER TABLE table_name RECOVER PARTITIONS" + val parsed = parser.parsePlan(sql) + val expected = AlterTableRecoverPartitionsCommand( + TableIdentifier("table_name", None)) + comparePlans(parsed, expected) + } + test("alter view: add partition (not supported)") { assertUnsupported( """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index f2ec393c30eca..b276e2d0c37e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -628,6 +628,55 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { testAddPartitions(isDatasourceTable = true) } + test("alter table: recover partitions (sequential)") { + withSQLConf("spark.rdd.parallelListingThreshold" -> "1") { + testRecoverPartitions() + } + } + + test("alter table: recover partition (parallel)") { + withSQLConf("spark.rdd.parallelListingThreshold" -> "10") { + testRecoverPartitions() + } + } + + private def testRecoverPartitions() { + val catalog = spark.sessionState.catalog + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist RECOVER PARTITIONS") + } + + val tableIdent = TableIdentifier("tab1") + createTable(catalog, tableIdent) + val part1 = Map("a" -> "1", "b" -> "5") + createTablePartition(catalog, part1, tableIdent) + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1)) + + val part2 = Map("a" -> "2", "b" -> "6") + val root = new Path(catalog.getTableMetadata(tableIdent).storage.locationUri.get) + val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) + // valid + fs.mkdirs(new Path(new Path(root, "a=1"), "b=5")) + fs.mkdirs(new Path(new Path(root, "A=2"), "B=6")) + // invalid + fs.mkdirs(new Path(new Path(root, "a"), "b")) // bad name + fs.mkdirs(new Path(new Path(root, "b=1"), "a=1")) // wrong order + fs.mkdirs(new Path(root, "a=4")) // not enough columns + fs.createNewFile(new Path(new Path(root, "a=1"), "b=4")) // file + fs.createNewFile(new Path(new Path(root, "a=1"), "_SUCCESS")) // _SUCCESS + fs.mkdirs(new Path(new Path(root, "a=1"), "_temporary")) // _temporary + fs.mkdirs(new Path(new Path(root, "a=1"), ".b=4")) // start with . + + try { + sql("ALTER TABLE tab1 RECOVER PARTITIONS") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2)) + } finally { + fs.delete(root, true) + } + } + test("alter table: add partition is not supported for views") { assertUnsupported("ALTER VIEW dbx.tab1 ADD IF NOT EXISTS PARTITION (b='2')") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 867aadb5f5569..54009d4b4130a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -520,8 +521,13 @@ class HiveDDLCommandSuite extends PlanTest { } } - test("MSCK repair table (not supported)") { - assertUnsupported("MSCK REPAIR TABLE tab1") + test("MSCK REPAIR table") { + val sql = "MSCK REPAIR TABLE tab1" + val parsed = parser.parsePlan(sql) + val expected = AlterTableRecoverPartitionsCommand( + TableIdentifier("tab1", None), + "MSCK REPAIR TABLE") + comparePlans(parsed, expected) } test("create table like") { From 475ee38150ee5a234156a903e4de227954b0063e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Kie=C5=82bowicz?= Date: Tue, 9 Aug 2016 23:01:50 -0700 Subject: [PATCH 1114/1470] Fixed typo MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Fixed small typo - "value ... ~~in~~ is null" ## How was this patch tested? Still compiles! Author: Michał Kiełbowicz Closes #14569 from jupblb/typo-fix. (cherry picked from commit 9dc3e602d77ccdf670f1b6648e5674066d189cc0) Signed-off-by: Reynold Xin --- sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d83eef7a41629..e16850efbea5f 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 @@ -463,6 +463,6 @@ trait Row extends Serializable { * @throws NullPointerException when value is null. */ private def getAnyValAs[T <: AnyVal](i: Int): T = - if (isNullAt(i)) throw new NullPointerException(s"Value at index $i in null") + if (isNullAt(i)) throw new NullPointerException(s"Value at index $i is null") else getAs[T](i) } From 2285de7347653ea6b3d35d58639ac70563f3c57a Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Wed, 10 Aug 2016 10:01:29 +0100 Subject: [PATCH 1115/1470] [SPARK-16522][MESOS] Spark application throws exception on exit. This is backport of https://github.com/apache/spark/pull/14175 to branch 2.0 Author: Sun Rui Closes #14575 from sun-rui/SPARK-16522-branch-2.0. --- .../MesosCoarseGrainedSchedulerBackend.scala | 7 +++- ...osCoarseGrainedSchedulerBackendSuite.scala | 33 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 99e6d39583747..2dcd67c7b89fa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -552,7 +552,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( taskId: String, reason: String): Unit = { stateLock.synchronized { - removeExecutor(taskId, SlaveLost(reason)) + // Do not call removeExecutor() after this scheduler backend was stopped because + // removeExecutor() internally will send a message to the driver endpoint but + // the driver endpoint is not available now, otherwise an exception will be thrown. + if (!stopCalled) { + removeExecutor(taskId, SlaveLost(reason)) + } slaves(slaveId).taskIDs.remove(taskId) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 7f21d4c623afc..f6ec167a187de 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -21,6 +21,7 @@ import java.util.Collections import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} import org.apache.mesos.Protos._ @@ -34,6 +35,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor import org.apache.spark.scheduler.TaskSchedulerImpl class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite @@ -47,6 +49,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite private var backend: MesosCoarseGrainedSchedulerBackend = _ private var externalShuffleClient: MesosExternalShuffleClient = _ private var driverEndpoint: RpcEndpointRef = _ + @volatile private var stopCalled = false test("mesos supports killing and limiting executors") { setBackend() @@ -252,6 +255,32 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite backend.start() } + test("Do not call removeExecutor() after backend is stopped") { + setBackend() + + // launches a task on a valid offer + val offers = List((backend.executorMemory(sc), 1)) + offerResources(offers) + verifyTaskLaunched("o1") + + // launches a thread simulating status update + val statusUpdateThread = new Thread { + override def run(): Unit = { + while (!stopCalled) { + Thread.sleep(100) + } + + val status = createTaskStatus("0", "s1", TaskState.TASK_FINISHED) + backend.statusUpdate(driver, status) + } + }.start + + backend.stop() + // Any method of the backend involving sending messages to the driver endpoint should not + // be called after the backend is stopped. + verify(driverEndpoint, never()).askWithRetry(isA(classOf[RemoveExecutor]))(any[ClassTag[_]]) + } + private def verifyDeclinedOffer(driver: SchedulerDriver, offerId: OfferID, filter: Boolean = false): Unit = { @@ -350,6 +379,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite mesosDriver = newDriver } + override def stopExecutors(): Unit = { + stopCalled = true + } + markRegistered() } backend.start() From 20efb7969ac8b313cd0895b57789e47d657453a4 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 10 Aug 2016 10:14:43 +0100 Subject: [PATCH 1116/1470] [SPARK-16324][SQL] regexp_extract should doc that it returns empty string when match fails ## What changes were proposed in this pull request? Doc that regexp_extract returns empty string when regex or group does not match ## How was this patch tested? Jenkins test, with a few new test cases Author: Sean Owen Closes #14525 from srowen/SPARK-16324. (cherry picked from commit 0578ff9681edbaab4ae68f67272dc3d4d890d53b) Signed-off-by: Sean Owen --- python/pyspark/sql/functions.py | 6 +++++- .../src/main/scala/org/apache/spark/sql/functions.scala | 3 ++- .../scala/org/apache/spark/sql/StringFunctionsSuite.scala | 4 ++++ 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 8a01805ec831b..4ea83e24bbc9a 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1440,11 +1440,15 @@ def split(str, pattern): @ignore_unicode_prefix @since(1.5) def regexp_extract(str, pattern, idx): - """Extract a specific(idx) group identified by a java regex, from the specified string column. + """Extract a specific group matched by a Java regex, from the specified string column. + If the regex did not match, or the specified group did not match, an empty string is returned. >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_extract('str', '(\d+)-(\d+)', 1).alias('d')).collect() [Row(d=u'100')] + >>> df = spark.createDataFrame([('foo',)], ['str']) + >>> df.select(regexp_extract('str', '(\d+)', 1).alias('d')).collect() + [Row(d=u'')] >>> df = spark.createDataFrame([('aaaac',)], ['str']) >>> df.select(regexp_extract('str', '(a+)(b)?(c)', 2).alias('d')).collect() [Row(d=u'')] 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 ab09ef7450b04..4e185b85e7660 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 @@ -2175,7 +2175,8 @@ object functions { def ltrim(e: Column): Column = withExpr {StringTrimLeft(e.expr) } /** - * Extract a specific(idx) group identified by a java regex, from the specified string column. + * Extract a specific group matched by a Java regex, from the specified string column. + * If the regex did not match, or the specified group did not match, an empty string is returned. * * @group string_funcs * @since 1.5.0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 6edd7b0c25b97..9be2de9c7d719 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -94,6 +94,10 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { test("non-matching optional group") { val df = Seq(Tuple1("aaaac")).toDF("s") + checkAnswer( + df.select(regexp_extract($"s", "(foo)", 1)), + Row("") + ) checkAnswer( df.select(regexp_extract($"s", "(a+)(b)?(c)", 2)), Row("") From 719ac5f37ccf32c34c70524b8cf9a2699c71a353 Mon Sep 17 00:00:00 2001 From: avulanov Date: Wed, 10 Aug 2016 10:25:00 +0100 Subject: [PATCH 1117/1470] [SPARK-15899][SQL] Fix the construction of the file path with hadoop Path ## What changes were proposed in this pull request? Fix the construction of the file path. Previous way of construction caused the creation of incorrect path on Windows. ## How was this patch tested? Run SQL unit tests on Windows Author: avulanov Closes #13868 from avulanov/SPARK-15899-file. (cherry picked from commit 11a6844bebbad1968bcdc295ab2de31c60dc0874) Signed-off-by: Sean Owen # Conflicts: # sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala # sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala --- .../apache/spark/sql/internal/SQLConf.scala | 7 ++- .../sql/execution/command/DDLSuite.scala | 44 ++++++++++--------- .../spark/sql/internal/SQLConfSuite.scala | 4 +- 3 files changed, 29 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1a9bb6a0b54e1..1d30f714000b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.immutable +import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.internal.Logging @@ -55,7 +56,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("file:${system:user.dir}/spark-warehouse") + .createWithDefault("${system:user.dir}/spark-warehouse") val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() @@ -690,9 +691,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH) - def warehousePath: String = { - getConf(WAREHOUSE_PATH).replace("${system:user.dir}", System.getProperty("user.dir")) - } + def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString override def orderByOrdinal: Boolean = getConf(ORDER_BY_ORDINAL) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b276e2d0c37e0..1652c4de3c4a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -111,10 +111,6 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) } - private def appendTrailingSlash(path: String): String = { - if (!path.endsWith(File.separator)) path + File.separator else path - } - test("the qualified path of a database is stored in the catalog") { val catalog = spark.sessionState.catalog @@ -122,18 +118,19 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val path = tmpDir.toString // The generated temp path is not qualified. assert(!path.startsWith("file:/")) - sql(s"CREATE DATABASE db1 LOCATION '$path'") + val uri = tmpDir.toURI + sql(s"CREATE DATABASE db1 LOCATION '$uri'") val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri assert("file" === pathInCatalog.getScheme) - val expectedPath = if (path.endsWith(File.separator)) path.dropRight(1) else path - assert(expectedPath === pathInCatalog.getPath) + val expectedPath = new Path(path).toUri + assert(expectedPath.getPath === pathInCatalog.getPath) withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { sql(s"CREATE DATABASE db2") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri - assert("file" === pathInCatalog.getScheme) - val expectedPath = appendTrailingSlash(spark.sessionState.conf.warehousePath) + "db2.db" - assert(expectedPath === pathInCatalog.getPath) + val pathInCatalog2 = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + assert("file" === pathInCatalog2.getScheme) + val expectedPath2 = new Path(spark.sessionState.conf.warehousePath + "/" + "db2.db").toUri + assert(expectedPath2.getPath === pathInCatalog2.getPath) } sql("DROP DATABASE db1") @@ -141,6 +138,13 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + private def makeQualifiedPath(path: String): String = { + // copy-paste from SessionCatalog + val hadoopPath = new Path(path) + val fs = hadoopPath.getFileSystem(sparkContext.hadoopConfiguration) + fs.makeQualified(hadoopPath).toString + } + test("Create/Drop Database") { withTempDir { tmpDir => val path = tmpDir.toString @@ -154,8 +158,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -181,8 +184,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) val expectedLocation = - "file:" + appendTrailingSlash(System.getProperty("user.dir")) + - s"spark-warehouse/$dbName.db" + makeQualifiedPath(s"${System.getProperty("user.dir")}/spark-warehouse" + + "/" + s"$dbName.db") assert(db1 == CatalogDatabase( dbName, "", @@ -200,17 +203,17 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => - val path = tmpDir.toString - val dbPath = "file:" + path + val path = new Path(tmpDir.toString).toUri.toString databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + val expPath = makeQualifiedPath(tmpDir.toString) assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", - if (dbPath.endsWith(File.separator)) dbPath.dropRight(1) else dbPath, + expPath, Map.empty)) sql(s"DROP DATABASE $dbName CASCADE") assert(!catalog.databaseExists(dbNameWithoutBackTicks)) @@ -233,8 +236,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -275,7 +277,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + val location = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") sql(s"CREATE DATABASE $dbName") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 5d348044515af..761bbe3576c71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.internal +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} @@ -214,7 +216,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) assert(spark.sessionState.conf.warehousePath - === s"file:${System.getProperty("user.dir")}/spark-warehouse") + === new Path(s"${System.getProperty("user.dir")}/spark-warehouse").toString) } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") } From 15637f735f4b27b291f40bbeadb98c5e0318bf70 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 10 Aug 2016 16:48:57 +0100 Subject: [PATCH 1118/1470] Revert "[SPARK-15899][SQL] Fix the construction of the file path with hadoop Path" This reverts commit 719ac5f37ccf32c34c70524b8cf9a2699c71a353. --- .../apache/spark/sql/internal/SQLConf.scala | 7 +-- .../sql/execution/command/DDLSuite.scala | 44 +++++++++---------- .../spark/sql/internal/SQLConfSuite.scala | 4 +- 3 files changed, 26 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1d30f714000b9..1a9bb6a0b54e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -23,7 +23,6 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.immutable -import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.internal.Logging @@ -56,7 +55,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("${system:user.dir}/spark-warehouse") + .createWithDefault("file:${system:user.dir}/spark-warehouse") val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() @@ -691,7 +690,9 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH) - def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString + def warehousePath: String = { + getConf(WAREHOUSE_PATH).replace("${system:user.dir}", System.getProperty("user.dir")) + } override def orderByOrdinal: Boolean = getConf(ORDER_BY_ORDINAL) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 1652c4de3c4a1..b276e2d0c37e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -111,6 +111,10 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) } + private def appendTrailingSlash(path: String): String = { + if (!path.endsWith(File.separator)) path + File.separator else path + } + test("the qualified path of a database is stored in the catalog") { val catalog = spark.sessionState.catalog @@ -118,19 +122,18 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val path = tmpDir.toString // The generated temp path is not qualified. assert(!path.startsWith("file:/")) - val uri = tmpDir.toURI - sql(s"CREATE DATABASE db1 LOCATION '$uri'") + sql(s"CREATE DATABASE db1 LOCATION '$path'") val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri assert("file" === pathInCatalog.getScheme) - val expectedPath = new Path(path).toUri - assert(expectedPath.getPath === pathInCatalog.getPath) + val expectedPath = if (path.endsWith(File.separator)) path.dropRight(1) else path + assert(expectedPath === pathInCatalog.getPath) withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { sql(s"CREATE DATABASE db2") - val pathInCatalog2 = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri - assert("file" === pathInCatalog2.getScheme) - val expectedPath2 = new Path(spark.sessionState.conf.warehousePath + "/" + "db2.db").toUri - assert(expectedPath2.getPath === pathInCatalog2.getPath) + val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + assert("file" === pathInCatalog.getScheme) + val expectedPath = appendTrailingSlash(spark.sessionState.conf.warehousePath) + "db2.db" + assert(expectedPath === pathInCatalog.getPath) } sql("DROP DATABASE db1") @@ -138,13 +141,6 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } - private def makeQualifiedPath(path: String): String = { - // copy-paste from SessionCatalog - val hadoopPath = new Path(path) - val fs = hadoopPath.getFileSystem(sparkContext.hadoopConfiguration) - fs.makeQualified(hadoopPath).toString - } - test("Create/Drop Database") { withTempDir { tmpDir => val path = tmpDir.toString @@ -158,7 +154,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = + "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -184,8 +181,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) val expectedLocation = - makeQualifiedPath(s"${System.getProperty("user.dir")}/spark-warehouse" + - "/" + s"$dbName.db") + "file:" + appendTrailingSlash(System.getProperty("user.dir")) + + s"spark-warehouse/$dbName.db" assert(db1 == CatalogDatabase( dbName, "", @@ -203,17 +200,17 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => - val path = new Path(tmpDir.toString).toUri.toString + val path = tmpDir.toString + val dbPath = "file:" + path databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expPath = makeQualifiedPath(tmpDir.toString) assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", - expPath, + if (dbPath.endsWith(File.separator)) dbPath.dropRight(1) else dbPath, Map.empty)) sql(s"DROP DATABASE $dbName CASCADE") assert(!catalog.databaseExists(dbNameWithoutBackTicks)) @@ -236,7 +233,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = + "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -277,7 +275,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val location = "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" sql(s"CREATE DATABASE $dbName") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 761bbe3576c71..5d348044515af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.internal -import org.apache.hadoop.fs.Path - import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} @@ -216,7 +214,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) assert(spark.sessionState.conf.warehousePath - === new Path(s"${System.getProperty("user.dir")}/spark-warehouse").toString) + === s"file:${System.getProperty("user.dir")}/spark-warehouse") } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") } From 977fbbfcae705dbdbf203bd0a6e7c75a12156d3f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 10 Aug 2016 10:03:55 -0700 Subject: [PATCH 1119/1470] [SPARK-15639] [SPARK-16321] [SQL] Push down filter at RowGroups level for parquet reader The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader. The benchmark that excludes the time of writing Parquet file: test("Benchmark for Parquet") { val N = 500 << 12 withParquetTable((0 until N).map(i => (101, i)), "t") { val benchmark = new Benchmark("Parquet reader", N) benchmark.addCase("reading Parquet file", 10) { iter => sql("SELECT _1 FROM t where t._1 < 100").collect() } benchmark.run() } } `withParquetTable` in default will run tests for vectorized reader non-vectorized readers. I only let it run vectorized reader. When we set the block size of parquet as 1024 to have multiple row groups. The benchmark is: Before this patch: The retrieved row groups: 8063 Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 825 / 1233 2.5 402.6 1.0X After this patch: The retrieved row groups: 0 Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 306 / 503 6.7 149.6 1.0X Next, I run the benchmark for non-pushdown case using the same benchmark code but with disabled pushdown configuration. This time the parquet block size is default value. Before this patch: Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 136 / 238 15.0 66.5 1.0X After this patch: Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 124 / 193 16.5 60.7 1.0X For non-pushdown case, from the results, I think this patch doesn't affect normal code path. I've manually output the `totalRowCount` in `SpecificParquetRecordReaderBase` to see if this patch actually filter the row-groups. When running the above benchmark: After this patch: `totalRowCount = 0` Before this patch: `totalRowCount = 1024000` Existing tests should be passed. Author: Liang-Chi Hsieh Closes #13701 from viirya/vectorized-reader-push-down-filter2. (cherry picked from commit 19af298bb6d264adcf02f6f84c8dc1542b408507) Signed-off-by: Davies Liu --- .../apache/spark/executor/TaskMetrics.scala | 9 + .../org/apache/spark/util/AccumulatorV2.scala | 12 ++ .../SpecificParquetRecordReaderBase.java | 18 ++ .../parquet/ParquetFileFormat.scala | 6 + .../parquet/ParquetFilterSuite.scala | 165 +++++++++++------- 5 files changed, 143 insertions(+), 67 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 5bb505bf09f17..dd149a919fe55 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -225,6 +225,15 @@ class TaskMetrics private[spark] () extends Serializable { } private[spark] def accumulators(): Seq[AccumulatorV2[_, _]] = internalAccums ++ externalAccums + + /** + * Looks for a registered accumulator by accumulator name. + */ + private[spark] def lookForAccumulatorByName(name: String): Option[AccumulatorV2[_, _]] = { + accumulators.find { acc => + acc.name.isDefined && acc.name.get == name + } + } } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index a9167ce6edf90..d130a37db5b5d 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -23,6 +23,8 @@ import java.util.ArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong +import scala.collection.JavaConverters._ + import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext} import org.apache.spark.scheduler.AccumulableInfo @@ -257,6 +259,16 @@ private[spark] object AccumulatorContext { originals.clear() } + /** + * Looks for a registered accumulator by accumulator name. + */ + private[spark] def lookForAccumulatorByName(name: String): Option[AccumulatorV2[_, _]] = { + originals.values().asScala.find { ref => + val acc = ref.get + acc != null && acc.name.isDefined && acc.name.get == name + }.map(_.get) + } + // Identifier for distinguishing SQL metrics from other accumulators private[spark] val SQL_ACCUM_IDENTIFIER = "sql" } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index 0d624d17f4cd7..b903aeeb5c9ea 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -31,6 +31,8 @@ import java.util.Map; import java.util.Set; +import scala.Option; + import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups; import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; import static org.apache.parquet.format.converter.ParquetMetadataConverter.range; @@ -59,8 +61,12 @@ import org.apache.parquet.hadoop.util.ConfigurationUtil; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Types; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.StructType$; +import org.apache.spark.util.AccumulatorV2; +import org.apache.spark.util.LongAccumulator; /** * Base class for custom RecordReaders for Parquet that directly materialize to `T`. @@ -144,6 +150,18 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont for (BlockMetaData block : blocks) { this.totalRowCount += block.getRowCount(); } + + // For test purpose. + // If the predefined accumulator exists, the row group number to read will be updated + // to the accumulator. So we can check if the row groups are filtered or not in test case. + TaskContext taskContext = TaskContext$.MODULE$.get(); + if (taskContext != null) { + Option> accu = (Option>) taskContext.taskMetrics() + .lookForAccumulatorByName("numRowGroups"); + if (accu.isDefined()) { + ((LongAccumulator)accu.get()).add((long)blocks.size()); + } + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 5397d50b6c7a0..7e819c744c638 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -46,6 +46,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -355,6 +356,11 @@ private[sql] class ParquetFileFormat val hadoopAttemptContext = new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId) + // Try to push down filters when filter push-down is enabled. + // Notice: This push-down is RowGroups level, not individual records. + if (pushed.isDefined) { + ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get) + } val parquetReader = if (enableVectorizedReader) { val vectorizedReader = new VectorizedParquetRecordReader() vectorizedReader.initialize(split, hadoopAttemptContext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 2a89773cf5341..ab9250045f5b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.util.{AccumulatorContext, LongAccumulator} /** * A test suite that tests Parquet filter2 API based filter pushdown optimization. @@ -370,73 +371,75 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex test("SPARK-11103: Filter applied on merged Parquet schema with new column fails") { import testImplicits._ - - withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { dir => - val pathOne = s"${dir.getCanonicalPath}/table1" - (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(pathOne) - val pathTwo = s"${dir.getCanonicalPath}/table2" - (1 to 3).map(i => (i, i.toString)).toDF("c", "b").write.parquet(pathTwo) - - // If the "c = 1" filter gets pushed down, this query will throw an exception which - // Parquet emits. This is a Parquet issue (PARQUET-389). - val df = spark.read.parquet(pathOne, pathTwo).filter("c = 1").selectExpr("c", "b", "a") - checkAnswer( - df, - Row(1, "1", null)) - - // The fields "a" and "c" only exist in one Parquet file. - assert(df.schema("a").metadata.getBoolean(StructType.metadataKeyForOptionalField)) - assert(df.schema("c").metadata.getBoolean(StructType.metadataKeyForOptionalField)) - - val pathThree = s"${dir.getCanonicalPath}/table3" - df.write.parquet(pathThree) - - // We will remove the temporary metadata when writing Parquet file. - val schema = spark.read.parquet(pathThree).schema - assert(schema.forall(!_.metadata.contains(StructType.metadataKeyForOptionalField))) - - val pathFour = s"${dir.getCanonicalPath}/table4" - val dfStruct = sparkContext.parallelize(Seq((1, 1))).toDF("a", "b") - dfStruct.select(struct("a").as("s")).write.parquet(pathFour) - - val pathFive = s"${dir.getCanonicalPath}/table5" - val dfStruct2 = sparkContext.parallelize(Seq((1, 1))).toDF("c", "b") - dfStruct2.select(struct("c").as("s")).write.parquet(pathFive) - - // If the "s.c = 1" filter gets pushed down, this query will throw an exception which - // Parquet emits. - val dfStruct3 = spark.read.parquet(pathFour, pathFive).filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - - // The fields "s.a" and "s.c" only exist in one Parquet file. - val field = dfStruct3.schema("s").dataType.asInstanceOf[StructType] - assert(field("a").metadata.getBoolean(StructType.metadataKeyForOptionalField)) - assert(field("c").metadata.getBoolean(StructType.metadataKeyForOptionalField)) - - val pathSix = s"${dir.getCanonicalPath}/table6" - dfStruct3.write.parquet(pathSix) - - // We will remove the temporary metadata when writing Parquet file. - val forPathSix = spark.read.parquet(pathSix).schema - assert(forPathSix.forall(!_.metadata.contains(StructType.metadataKeyForOptionalField))) - - // sanity test: make sure optional metadata field is not wrongly set. - val pathSeven = s"${dir.getCanonicalPath}/table7" - (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(pathSeven) - val pathEight = s"${dir.getCanonicalPath}/table8" - (4 to 6).map(i => (i, i.toString)).toDF("a", "b").write.parquet(pathEight) - - val df2 = spark.read.parquet(pathSeven, pathEight).filter("a = 1").selectExpr("a", "b") - checkAnswer( - df2, - Row(1, "1")) - - // The fields "a" and "b" exist in both two Parquet files. No metadata is set. - assert(!df2.schema("a").metadata.contains(StructType.metadataKeyForOptionalField)) - assert(!df2.schema("b").metadata.contains(StructType.metadataKeyForOptionalField)) + Seq("true", "false").map { vectorized => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + withTempPath { dir => + val pathOne = s"${dir.getCanonicalPath}/table1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(pathOne) + val pathTwo = s"${dir.getCanonicalPath}/table2" + (1 to 3).map(i => (i, i.toString)).toDF("c", "b").write.parquet(pathTwo) + + // If the "c = 1" filter gets pushed down, this query will throw an exception which + // Parquet emits. This is a Parquet issue (PARQUET-389). + val df = spark.read.parquet(pathOne, pathTwo).filter("c = 1").selectExpr("c", "b", "a") + checkAnswer( + df, + Row(1, "1", null)) + + // The fields "a" and "c" only exist in one Parquet file. + assert(df.schema("a").metadata.getBoolean(StructType.metadataKeyForOptionalField)) + assert(df.schema("c").metadata.getBoolean(StructType.metadataKeyForOptionalField)) + + val pathThree = s"${dir.getCanonicalPath}/table3" + df.write.parquet(pathThree) + + // We will remove the temporary metadata when writing Parquet file. + val schema = spark.read.parquet(pathThree).schema + assert(schema.forall(!_.metadata.contains(StructType.metadataKeyForOptionalField))) + + val pathFour = s"${dir.getCanonicalPath}/table4" + val dfStruct = sparkContext.parallelize(Seq((1, 1))).toDF("a", "b") + dfStruct.select(struct("a").as("s")).write.parquet(pathFour) + + val pathFive = s"${dir.getCanonicalPath}/table5" + val dfStruct2 = sparkContext.parallelize(Seq((1, 1))).toDF("c", "b") + dfStruct2.select(struct("c").as("s")).write.parquet(pathFive) + + // If the "s.c = 1" filter gets pushed down, this query will throw an exception which + // Parquet emits. + val dfStruct3 = spark.read.parquet(pathFour, pathFive).filter("s.c = 1") + .selectExpr("s") + checkAnswer(dfStruct3, Row(Row(null, 1))) + + // The fields "s.a" and "s.c" only exist in one Parquet file. + val field = dfStruct3.schema("s").dataType.asInstanceOf[StructType] + assert(field("a").metadata.getBoolean(StructType.metadataKeyForOptionalField)) + assert(field("c").metadata.getBoolean(StructType.metadataKeyForOptionalField)) + + val pathSix = s"${dir.getCanonicalPath}/table6" + dfStruct3.write.parquet(pathSix) + + // We will remove the temporary metadata when writing Parquet file. + val forPathSix = spark.read.parquet(pathSix).schema + assert(forPathSix.forall(!_.metadata.contains(StructType.metadataKeyForOptionalField))) + + // sanity test: make sure optional metadata field is not wrongly set. + val pathSeven = s"${dir.getCanonicalPath}/table7" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(pathSeven) + val pathEight = s"${dir.getCanonicalPath}/table8" + (4 to 6).map(i => (i, i.toString)).toDF("a", "b").write.parquet(pathEight) + + val df2 = spark.read.parquet(pathSeven, pathEight).filter("a = 1").selectExpr("a", "b") + checkAnswer( + df2, + Row(1, "1")) + + // The fields "a" and "b" exist in both two Parquet files. No metadata is set. + assert(!df2.schema("a").metadata.contains(StructType.metadataKeyForOptionalField)) + assert(!df2.schema("b").metadata.contains(StructType.metadataKeyForOptionalField)) + } } } } @@ -559,4 +562,32 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex assert(df.filter("_1 IS NOT NULL").count() === 4) } } + + test("Fiters should be pushed down for vectorized Parquet reader at row group level") { + import testImplicits._ + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/table" + (1 to 1024).map(i => (101, i)).toDF("a", "b").write.parquet(path) + + Seq(("true", (x: Long) => x == 0), ("false", (x: Long) => x > 0)).map { case (push, func) => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> push) { + val accu = new LongAccumulator + accu.register(sparkContext, Some("numRowGroups")) + + val df = spark.read.parquet(path).filter("a < 100") + df.foreachPartition(_.foreach(v => accu.add(0))) + df.collect + + val numRowGroups = AccumulatorContext.lookForAccumulatorByName("numRowGroups") + assert(numRowGroups.isDefined) + assert(func(numRowGroups.get.asInstanceOf[LongAccumulator].value)) + AccumulatorContext.remove(accu.id) + } + } + } + } + } } From d3a30d2f0531049b60d1b321b3b8b3d0a6d716d2 Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Wed, 10 Aug 2016 11:18:23 -0700 Subject: [PATCH 1120/1470] [SPARK-16579][SPARKR] add install.spark function Add an install_spark function to the SparkR package. User can run `install_spark()` to install Spark to a local directory within R. Updates: Several changes have been made: - `install.spark()` - check existence of tar file in the cache folder, and download only if not found - trial priority of mirror_url look-up: user-provided -> preferred mirror site from apache website -> hardcoded backup option - use 2.0.0 - `sparkR.session()` - can install spark when not found in `SPARK_HOME` Manual tests, running the check-cran.sh script added in #14173. Author: Junyang Qian Closes #14258 from junyangq/SPARK-16579. (cherry picked from commit 214ba66a030bc3a718c567a742b0db44bf911d61) Signed-off-by: Shivaram Venkataraman --- R/check-cran.sh | 2 +- R/pkg/DESCRIPTION | 3 +- R/pkg/NAMESPACE | 2 + R/pkg/R/install.R | 235 ++++++++++++++++++++++++++++++++++++++++++++++ R/pkg/R/sparkR.R | 17 ++++ R/pkg/R/utils.R | 8 ++ 6 files changed, 265 insertions(+), 2 deletions(-) create mode 100644 R/pkg/R/install.R diff --git a/R/check-cran.sh b/R/check-cran.sh index b3a6860961c1e..5c90fd07f28e4 100755 --- a/R/check-cran.sh +++ b/R/check-cran.sh @@ -47,6 +47,6 @@ $FWDIR/create-docs.sh VERSION=`grep Version $FWDIR/pkg/DESCRIPTION | awk '{print $NF}'` -"$R_SCRIPT_PATH/"R CMD check --as-cran --no-tests SparkR_"$VERSION".tar.gz +"$R_SCRIPT_PATH/"R CMD check --as-cran SparkR_"$VERSION".tar.gz popd > /dev/null diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index ac73d6c79891e..357ab007931f5 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -7,7 +7,7 @@ Author: The Apache Software Foundation Maintainer: Shivaram Venkataraman Depends: R (>= 3.0), - methods, + methods Suggests: testthat, e1071, @@ -31,6 +31,7 @@ Collate: 'context.R' 'deserialize.R' 'functions.R' + 'install.R' 'mllib.R' 'serialize.R' 'sparkR.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 1d74c6d95578f..aaab92f5cfc7b 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -352,3 +352,5 @@ S3method(structField, character) S3method(structField, jobj) S3method(structType, jobj) S3method(structType, structField) + +export("install.spark") diff --git a/R/pkg/R/install.R b/R/pkg/R/install.R new file mode 100644 index 0000000000000..987bac7bebc0e --- /dev/null +++ b/R/pkg/R/install.R @@ -0,0 +1,235 @@ +# +# 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. +# + +# Functions to install Spark in case the user directly downloads SparkR +# from CRAN. + +#' Download and Install Apache Spark to a Local Directory +#' +#' \code{install.spark} downloads and installs Spark to a local directory if +#' it is not found. The Spark version we use is the same as the SparkR version. +#' Users can specify a desired Hadoop version, the remote mirror site, and +#' the directory where the package is installed locally. +#' +#' The full url of remote file is inferred from \code{mirrorUrl} and \code{hadoopVersion}. +#' \code{mirrorUrl} specifies the remote path to a Spark folder. It is followed by a subfolder +#' named after the Spark version (that corresponds to SparkR), and then the tar filename. +#' The filename is composed of four parts, i.e. [Spark version]-bin-[Hadoop version].tgz. +#' For example, the full path for a Spark 2.0.0 package for Hadoop 2.7 from +#' \code{http://apache.osuosl.org} has path: +#' \code{http://apache.osuosl.org/spark/spark-2.0.0/spark-2.0.0-bin-hadoop2.7.tgz}. +#' For \code{hadoopVersion = "without"}, [Hadoop version] in the filename is then +#' \code{without-hadoop}. +#' +#' @param hadoopVersion Version of Hadoop to install. Default is \code{"2.7"}. It can take other +#' version number in the format of "x.y" where x and y are integer. +#' If \code{hadoopVersion = "without"}, "Hadoop free" build is installed. +#' See +#' \href{http://spark.apache.org/docs/latest/hadoop-provided.html}{ +#' "Hadoop Free" Build} for more information. +#' Other patched version names can also be used, e.g. \code{"cdh4"} +#' @param mirrorUrl base URL of the repositories to use. The directory layout should follow +#' \href{http://www.apache.org/dyn/closer.lua/spark/}{Apache mirrors}. +#' @param localDir a local directory where Spark is installed. The directory contains +#' version-specific folders of Spark packages. Default is path to +#' the cache directory: +#' \itemize{ +#' \item Mac OS X: \file{~/Library/Caches/spark} +#' \item Unix: \env{$XDG_CACHE_HOME} if defined, otherwise \file{~/.cache/spark} +#' \item Windows: \file{\%LOCALAPPDATA\%\\spark\\spark\\Cache}. See +#' \href{https://www.microsoft.com/security/portal/mmpc/shared/variables.aspx}{ +#' Windows Common Folder Variables} about \%LOCALAPPDATA\% +#' } +#' @param overwrite If \code{TRUE}, download and overwrite the existing tar file in localDir +#' and force re-install Spark (in case the local directory or file is corrupted) +#' @return \code{install.spark} returns the local directory where Spark is found or installed +#' @rdname install.spark +#' @name install.spark +#' @aliases install.spark +#' @export +#' @examples +#'\dontrun{ +#' install.spark() +#'} +#' @note install.spark since 2.1.0 +#' @seealso See available Hadoop versions: +#' \href{http://spark.apache.org/downloads.html}{Apache Spark} +install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, + localDir = NULL, overwrite = FALSE) { + version <- paste0("spark-", packageVersion("SparkR")) + hadoopVersion <- tolower(hadoopVersion) + hadoopVersionName <- hadoop_version_name(hadoopVersion) + packageName <- paste(version, "bin", hadoopVersionName, sep = "-") + localDir <- ifelse(is.null(localDir), spark_cache_path(), + normalizePath(localDir, mustWork = FALSE)) + + if (is.na(file.info(localDir)$isdir)) { + dir.create(localDir, recursive = TRUE) + } + + packageLocalDir <- file.path(localDir, packageName) + + if (overwrite) { + message(paste0("Overwrite = TRUE: download and overwrite the tar file", + "and Spark package directory if they exist.")) + } + + # can use dir.exists(packageLocalDir) under R 3.2.0 or later + if (!is.na(file.info(packageLocalDir)$isdir) && !overwrite) { + fmt <- "Spark %s for Hadoop %s is found, and SPARK_HOME set to %s" + msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion), + packageLocalDir) + message(msg) + Sys.setenv(SPARK_HOME = packageLocalDir) + return(invisible(packageLocalDir)) + } + + packageLocalPath <- paste0(packageLocalDir, ".tgz") + tarExists <- file.exists(packageLocalPath) + + if (tarExists && !overwrite) { + message("tar file found.") + } else { + robust_download_tar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) + } + + message(sprintf("Installing to %s", localDir)) + untar(tarfile = packageLocalPath, exdir = localDir) + if (!tarExists || overwrite) { + unlink(packageLocalPath) + } + message("DONE.") + Sys.setenv(SPARK_HOME = packageLocalDir) + message(paste("SPARK_HOME set to", packageLocalDir)) + invisible(packageLocalDir) +} + +robust_download_tar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { + # step 1: use user-provided url + if (!is.null(mirrorUrl)) { + msg <- sprintf("Use user-provided mirror site: %s.", mirrorUrl) + message(msg) + success <- direct_download_tar(mirrorUrl, version, hadoopVersion, + packageName, packageLocalPath) + if (success) return() + } else { + message("Mirror site not provided.") + } + + # step 2: use url suggested from apache website + message("Looking for site suggested from apache website...") + mirrorUrl <- get_preferred_mirror(version, packageName) + if (!is.null(mirrorUrl)) { + success <- direct_download_tar(mirrorUrl, version, hadoopVersion, + packageName, packageLocalPath) + if (success) return() + } else { + message("Unable to find suggested mirror site.") + } + + # step 3: use backup option + message("To use backup site...") + mirrorUrl <- default_mirror_url() + success <- direct_download_tar(mirrorUrl, version, hadoopVersion, + packageName, packageLocalPath) + if (success) { + return(packageLocalPath) + } else { + msg <- sprintf(paste("Unable to download Spark %s for Hadoop %s.", + "Please check network connection, Hadoop version,", + "or provide other mirror sites."), + version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion)) + stop(msg) + } +} + +get_preferred_mirror <- function(version, packageName) { + jsonUrl <- paste0("http://www.apache.org/dyn/closer.cgi?path=", + file.path("spark", version, packageName), + ".tgz&as_json=1") + textLines <- readLines(jsonUrl, warn = FALSE) + rowNum <- grep("\"preferred\"", textLines) + linePreferred <- textLines[rowNum] + matchInfo <- regexpr("\"[A-Za-z][A-Za-z0-9+-.]*://.+\"", linePreferred) + if (matchInfo != -1) { + startPos <- matchInfo + 1 + endPos <- matchInfo + attr(matchInfo, "match.length") - 2 + mirrorPreferred <- base::substr(linePreferred, startPos, endPos) + mirrorPreferred <- paste0(mirrorPreferred, "spark") + message(sprintf("Preferred mirror site found: %s", mirrorPreferred)) + } else { + mirrorPreferred <- NULL + } + mirrorPreferred +} + +direct_download_tar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { + packageRemotePath <- paste0( + file.path(mirrorUrl, version, packageName), ".tgz") + fmt <- paste("Downloading Spark %s for Hadoop %s from:\n- %s") + msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion), + packageRemotePath) + message(msg) + + isFail <- tryCatch(download.file(packageRemotePath, packageLocalPath), + error = function(e) { + message(sprintf("Fetch failed from %s", mirrorUrl)) + print(e) + TRUE + }) + !isFail +} + +default_mirror_url <- function() { + "http://www-us.apache.org/dist/spark" +} + +hadoop_version_name <- function(hadoopVersion) { + if (hadoopVersion == "without") { + "without-hadoop" + } else if (grepl("^[0-9]+\\.[0-9]+$", hadoopVersion, perl = TRUE)) { + paste0("hadoop", hadoopVersion) + } else { + hadoopVersion + } +} + +# The implementation refers to appdirs package: https://pypi.python.org/pypi/appdirs and +# adapt to Spark context +spark_cache_path <- function() { + if (.Platform$OS.type == "windows") { + winAppPath <- Sys.getenv("%LOCALAPPDATA%", unset = NA) + if (is.na(winAppPath)) { + msg <- paste("%LOCALAPPDATA% not found.", + "Please define the environment variable", + "or restart and enter an installation path in localDir.") + stop(msg) + } else { + path <- file.path(winAppPath, "spark", "spark", "Cache") + } + } else if (.Platform$OS.type == "unix") { + if (Sys.info()["sysname"] == "Darwin") { + path <- file.path(Sys.getenv("HOME"), "Library/Caches", "spark") + } else { + path <- file.path( + Sys.getenv("XDG_CACHE_HOME", file.path(Sys.getenv("HOME"), ".cache")), "spark") + } + } else { + stop(sprintf("Unknown OS: %s", .Platform$OS.type)) + } + normalizePath(path, mustWork = FALSE) +} diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 524f7c4a26b67..f8bdee739ef02 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -365,6 +365,23 @@ sparkR.session <- function( } overrideEnvs(sparkConfigMap, paramMap) } + # do not download if it is run in the sparkR shell + if (!nzchar(master) || is_master_local(master)) { + if (!is_sparkR_shell()) { + if (is.na(file.info(sparkHome)$isdir)) { + msg <- paste0("Spark not found in SPARK_HOME: ", + sparkHome, + " .\nTo search in the cache directory. ", + "Installation will start if not found.") + message(msg) + packageLocalDir <- install.spark() + sparkHome <- packageLocalDir + } else { + msg <- paste0("Spark package is found in SPARK_HOME: ", sparkHome) + message(msg) + } + } + } if (!exists(".sparkRjsc", envir = .sparkREnv)) { sparkExecutorEnvMap <- new.env() diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 240b9f669bdd7..d78c0a7a539a8 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -689,3 +689,11 @@ getSparkContext <- function() { sc <- get(".sparkRjsc", envir = .sparkREnv) sc } + +is_master_local <- function(master) { + grepl("^local(\\[([0-9]+|\\*)\\])?$", master, perl = TRUE) +} + +is_sparkR_shell <- function() { + grepl(".*shell\\.R$", Sys.getenv("R_PROFILE_USER"), perl = TRUE) +} From 1e4013571b18ca337ea664838f7f8e781c8de7aa Mon Sep 17 00:00:00 2001 From: Tao Wang Date: Wed, 10 Aug 2016 22:30:18 -0700 Subject: [PATCH 1121/1470] [SPARK-17010][MINOR][DOC] Wrong description in memory management document ## What changes were proposed in this pull request? change the remain percent to right one. ## How was this patch tested? Manual review Author: Tao Wang Closes #14591 from WangTaoTheTonic/patch-1. (cherry picked from commit 7a6a3c3fbcea889ca20beae9d4198df2fe53bd1b) Signed-off-by: Reynold Xin --- docs/tuning.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tuning.md b/docs/tuning.md index 1ed14091c0546..976f2eb8a7b23 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -115,7 +115,7 @@ Although there are two relevant configurations, the typical user should not need as the default values are applicable to most workloads: * `spark.memory.fraction` expresses the size of `M` as a fraction of the (JVM heap space - 300MB) -(default 0.6). The rest of the space (25%) is reserved for user data structures, internal +(default 0.6). The rest of the space (40%) is reserved for user data structures, internal metadata in Spark, and safeguarding against OOM errors in the case of sparse and unusually large records. * `spark.memory.storageFraction` expresses the size of `R` as a fraction of `M` (default 0.5). From 8611bc2058eb7397c372de39b59934494569623c Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Wed, 10 Aug 2016 17:17:21 +0800 Subject: [PATCH 1122/1470] [SPARK-16866][SQL] Infrastructure for file-based SQL end-to-end tests ## What changes were proposed in this pull request? This patch introduces SQLQueryTestSuite, a basic framework for end-to-end SQL test cases defined in spark/sql/core/src/test/resources/sql-tests. This is a more standard way to test SQL queries end-to-end in different open source database systems, because it is more manageable to work with files. This is inspired by HiveCompatibilitySuite, but simplified for general Spark SQL tests. Once this is merged, I can work towards porting SQLQuerySuite over, and eventually also move the existing HiveCompatibilitySuite to use this framework. Unlike HiveCompatibilitySuite, SQLQueryTestSuite compares both the output schema and the output data (in string form). When there is a mismatch, the error message looks like the following: ``` [info] - blacklist.sql !!! IGNORED !!! [info] - number-format.sql *** FAILED *** (2 seconds, 405 milliseconds) [info] Expected "...147483648 -214748364[8]", but got "...147483648 -214748364[9]" Result should match for query #1 (SQLQueryTestSuite.scala:171) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495) [info] at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) [info] at org.scalatest.Assertions$class.assertResult(Assertions.scala:1171) ``` ## How was this patch tested? This is a test infrastructure change. Author: petermaxlee Closes #14472 from petermaxlee/SPARK-16866. (cherry picked from commit b9f8a117097bc102e261b68f38a679d16e19f2e2) Signed-off-by: Wenchen Fan --- .../resources/sql-tests/inputs/blacklist.sql | 4 + .../sql-tests/inputs/number-format.sql | 13 ++ .../sql-tests/results/number-format.sql.out | 34 +++ .../org/apache/spark/sql/SQLQuerySuite.scala | 36 --- .../apache/spark/sql/SQLQueryTestSuite.scala | 215 ++++++++++++++++++ .../sql/catalyst/LogicalPlanToSQLSuite.scala | 3 +- 6 files changed, 267 insertions(+), 38 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/blacklist.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/number-format.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/number-format.sql.out create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala diff --git a/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql b/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql new file mode 100644 index 0000000000000..d69f8147a5264 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql @@ -0,0 +1,4 @@ +-- This is a query file that has been blacklisted. +-- It includes a query that should crash Spark. +-- If the test case is run, the whole suite would fail. +some random not working query that should crash Spark. diff --git a/sql/core/src/test/resources/sql-tests/inputs/number-format.sql b/sql/core/src/test/resources/sql-tests/inputs/number-format.sql new file mode 100644 index 0000000000000..60076a8431582 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/number-format.sql @@ -0,0 +1,13 @@ +-- Verifies how we parse numbers + +-- parse as ints +select 1, -1; + +-- parse as longs +select 2147483648, -2147483649; + +-- parse as decimals +select 9223372036854775808, -9223372036854775809; + +-- various floating point (decimal) formats +select 0.3, -0.8, .5, -.18, 0.1111; diff --git a/sql/core/src/test/resources/sql-tests/results/number-format.sql.out b/sql/core/src/test/resources/sql-tests/results/number-format.sql.out new file mode 100644 index 0000000000000..4b800b7d92560 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/number-format.sql.out @@ -0,0 +1,34 @@ +-- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite +-- Number of queries: 4 + + +-- !query 0 +select 1, -1 +-- !query 0 schema +struct<1:int,(-1):int> +-- !query 0 output +1 -1 + + +-- !query 1 +select 2147483648, -2147483649 +-- !query 1 schema +struct<2147483648:bigint,(-2147483649):bigint> +-- !query 1 output +2147483648 -2147483649 + + +-- !query 2 +select 9223372036854775808, -9223372036854775809 +-- !query 2 schema +struct<9223372036854775808:decimal(19,0),(-9223372036854775809):decimal(19,0)> +-- !query 2 output +9223372036854775808 -9223372036854775809 + + +-- !query 3 +select 0.3, -0.8, .5, -.18, 0.1111 +-- !query 3 schema +struct<0.3:decimal(1,1),(-0.8):decimal(1,1),0.5:decimal(1,1),(-0.18):decimal(2,2),0.1111:decimal(4,4)> +-- !query 3 output +0.3 -0.8 0.5 -0.18 0.1111 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 393b4af285498..01c7e861bf8a3 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 @@ -1358,42 +1358,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } - test("Floating point number format") { - checkAnswer( - sql("SELECT 0.3"), Row(BigDecimal(0.3)) - ) - - checkAnswer( - sql("SELECT -0.8"), Row(BigDecimal(-0.8)) - ) - - checkAnswer( - sql("SELECT .5"), Row(BigDecimal(0.5)) - ) - - checkAnswer( - sql("SELECT -.18"), Row(BigDecimal(-0.18)) - ) - } - - test("Auto cast integer type") { - checkAnswer( - sql(s"SELECT ${Int.MaxValue + 1L}"), Row(Int.MaxValue + 1L) - ) - - checkAnswer( - sql(s"SELECT ${Int.MinValue - 1L}"), Row(Int.MinValue - 1L) - ) - - checkAnswer( - sql("SELECT 9223372036854775808"), Row(new java.math.BigDecimal("9223372036854775808")) - ) - - checkAnswer( - sql("SELECT -9223372036854775809"), Row(new java.math.BigDecimal("-9223372036854775809")) - ) - } - test("Test to check we can apply sign to expression") { checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala new file mode 100644 index 0000000000000..08b8432d68eb3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.io.File +import java.util.{Locale, TimeZone} + +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} +import org.apache.spark.sql.test.SharedSQLContext + +/** + * End-to-end test cases for SQL queries. + * + * Each case is loaded from a file in "spark/sql/core/src/test/resources/sql-tests/inputs". + * Each case has a golden result file in "spark/sql/core/src/test/resources/sql-tests/results". + * + * To re-generate golden files, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" + * }}} + * + * The format for input files is simple: + * 1. A list of SQL queries separated by semicolon. + * 2. Lines starting with -- are treated as comments and ignored. + * + * For example: + * {{{ + * -- this is a comment + * select 1, -1; + * select current_date; + * }}} + * + * The format for golden result files look roughly like: + * {{{ + * -- some header information + * + * -- !query 0 + * select 1, -1 + * -- !query 0 schema + * struct<...schema...> + * -- !query 0 output + * ... data row 1 ... + * ... data row 2 ... + * ... + * + * -- !query 1 + * ... + * }}} + */ +class SQLQueryTestSuite extends QueryTest with SharedSQLContext { + + private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" + + private val baseResourcePath = { + // If regenerateGoldenFiles is true, we must be running this in SBT and we use hard-coded + // relative path. Otherwise, we use classloader's getResource to find the location. + if (regenerateGoldenFiles) { + java.nio.file.Paths.get("src", "test", "resources", "sql-tests").toFile + } else { + val res = getClass.getClassLoader.getResource("sql-tests") + new File(res.getFile) + } + } + + private val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath + private val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath + + /** List of test cases to ignore, in lower cases. */ + private val blackList = Set( + "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. + ) + + // Create all the test cases. + listTestCases().foreach(createScalaTestCase) + + /** A test case. */ + private case class TestCase(name: String, inputFile: String, resultFile: String) + + /** A single SQL query's output. */ + private case class QueryOutput(sql: String, schema: String, output: String) { + def toString(queryIndex: Int): String = { + // We are explicitly not using multi-line string due to stripMargin removing "|" in output. + s"-- !query $queryIndex\n" + + sql + "\n" + + s"-- !query $queryIndex schema\n" + + schema + "\n" + + s"-- !query $queryIndex output\n" + + output + } + } + + private def createScalaTestCase(testCase: TestCase): Unit = { + if (blackList.contains(testCase.name.toLowerCase)) { + // Create a test case to ignore this case. + ignore(testCase.name) { /* Do nothing */ } + } else { + // Create a test case to run this case. + test(testCase.name) { runTest(testCase) } + } + } + + /** Run a test case. */ + private def runTest(testCase: TestCase): Unit = { + val input = fileToString(new File(testCase.inputFile)) + + // List of SQL queries to run + val queries: Seq[String] = { + val cleaned = input.split("\n").filterNot(_.startsWith("--")).mkString("\n") + // note: this is not a robust way to split queries using semicolon, but works for now. + cleaned.split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq + } + + // Run the SQL queries preparing them for comparison. + val outputs: Seq[QueryOutput] = queries.map { sql => + val df = spark.sql(sql) + // We might need to do some query canonicalization in the future. + QueryOutput( + sql = sql, + schema = df.schema.catalogString, + output = df.queryExecution.hiveResultString().mkString("\n")) + } + + if (regenerateGoldenFiles) { + // Again, we are explicitly not using multi-line string due to stripMargin removing "|". + val goldenOutput = { + s"-- Automatically generated by ${getClass.getName}\n" + + s"-- Number of queries: ${outputs.size}\n\n\n" + + outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n" + } + stringToFile(new File(testCase.resultFile), goldenOutput) + } + + // Read back the golden file. + val expectedOutputs: Seq[QueryOutput] = { + val goldenOutput = fileToString(new File(testCase.resultFile)) + val segments = goldenOutput.split("-- !query.+\n") + + // each query has 3 segments, plus the header + assert(segments.size == outputs.size * 3 + 1, + s"Expected ${outputs.size * 3 + 1} blocks in result file but got ${segments.size}. " + + s"Try regenerate the result files.") + Seq.tabulate(outputs.size) { i => + QueryOutput( + sql = segments(i * 3 + 1).trim, + schema = segments(i * 3 + 2).trim, + output = segments(i * 3 + 3).trim + ) + } + } + + // Compare results. + assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") { + outputs.size + } + + outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) => + assertResult(expected.sql, s"SQL query should match for query #$i") { output.sql } + assertResult(expected.schema, s"Schema should match for query #$i") { output.schema } + assertResult(expected.output, s"Result should match for query #$i") { output.output } + } + } + + private def listTestCases(): Seq[TestCase] = { + listFilesRecursively(new File(inputFilePath)).map { file => + val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" + TestCase(file.getName, file.getAbsolutePath, resultFile) + } + } + + /** Returns all the files (not directories) in a directory, recursively. */ + private def listFilesRecursively(path: File): Seq[File] = { + val (dirs, files) = path.listFiles().partition(_.isDirectory) + files ++ dirs.flatMap(listFilesRecursively) + } + + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + + override def beforeAll(): Unit = { + super.beforeAll() + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + RuleExecutor.resetTime() + } + + override def afterAll(): Unit = { + try { + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + + // For debugging dump some statistics about how much time was spent in various optimizer rules + logWarning(RuleExecutor.dumpTimeSpent()) + } finally { + super.afterAll() + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index d8ab864ca6fce..4e5a51155defd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -41,8 +41,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { import testImplicits._ // Used for generating new query answer files by saving - private val regenerateGoldenFiles: Boolean = - Option(System.getenv("SPARK_GENERATE_GOLDEN_FILES")) == Some("1") + private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" private val goldenSQLPath = "src/test/resources/sqlgen/" protected override def beforeAll(): Unit = { From 51b1016682a805e06b857a6b1f160a877839dbd5 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Wed, 10 Aug 2016 21:05:32 -0700 Subject: [PATCH 1123/1470] [SPARK-17008][SPARK-17009][SQL] Normalization and isolation in SQLQueryTestSuite. ## What changes were proposed in this pull request? This patch enhances SQLQueryTestSuite in two ways: 1. SPARK-17009: Use a new SparkSession for each test case to provide stronger isolation (e.g. config changes in one test case does not impact another). That said, we do not currently isolate catalog changes. 2. SPARK-17008: Normalize query output using sorting, inspired by HiveComparisonTest. I also ported a few new test cases over from SQLQuerySuite. ## How was this patch tested? This is a test harness update. Author: petermaxlee Closes #14590 from petermaxlee/SPARK-17008. (cherry picked from commit 425c7c2dbd2923094712e1215dd29272fb09cd79) Signed-off-by: Wenchen Fan --- .../resources/sql-tests/inputs/datetime.sql | 4 ++ .../resources/sql-tests/inputs/having.sql | 15 +++++ .../sql-tests/inputs/natural-join.sql | 20 ++++++ .../sql-tests/results/datetime.sql.out | 10 +++ .../sql-tests/results/having.sql.out | 40 ++++++++++++ .../sql-tests/results/natural-join.sql.out | 64 +++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 62 ------------------ .../apache/spark/sql/SQLQueryTestSuite.scala | 30 ++++++++- 8 files changed, 180 insertions(+), 65 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/datetime.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/having.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/natural-join.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/datetime.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/having.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/natural-join.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql new file mode 100644 index 0000000000000..3fd1c37e71795 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -0,0 +1,4 @@ +-- date time functions + +-- [SPARK-16836] current_date and current_timestamp literals +select current_date = current_date(), current_timestamp = current_timestamp(); diff --git a/sql/core/src/test/resources/sql-tests/inputs/having.sql b/sql/core/src/test/resources/sql-tests/inputs/having.sql new file mode 100644 index 0000000000000..364c022d959dc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/having.sql @@ -0,0 +1,15 @@ +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v); + +-- having clause +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2; + +-- having condition contains grouping column +SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2; + +-- SPARK-11032: resolve having correctly +SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0); diff --git a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql new file mode 100644 index 0000000000000..71a50157b766c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql @@ -0,0 +1,20 @@ +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + + +SELECT * FROM nt1 natural join nt2 where k = "one"; + +SELECT * FROM nt1 natural left join nt2 order by v1, v2; + +SELECT * FROM nt1 natural right join nt2 order by v1, v2; + +SELECT count(*) FROM nt1 natural full outer join nt2; diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out new file mode 100644 index 0000000000000..51746579b1314 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -0,0 +1,10 @@ +-- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query 0 +select current_date = current_date(), current_timestamp = current_timestamp() +-- !query 0 schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean> +-- !query 0 output +true true diff --git a/sql/core/src/test/resources/sql-tests/results/having.sql.out b/sql/core/src/test/resources/sql-tests/results/having.sql.out new file mode 100644 index 0000000000000..0bc8be66be638 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/having.sql.out @@ -0,0 +1,40 @@ +-- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite +-- Number of queries: 4 + + +-- !query 0 +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 +-- !query 1 schema +struct +-- !query 1 output +one 6 +three 3 + + +-- !query 2 +SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2 +-- !query 2 schema +struct +-- !query 2 output +1 + + +-- !query 3 +SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0) +-- !query 3 schema +struct +-- !query 3 output +1 diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out new file mode 100644 index 0000000000000..d4954dabea8c9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 natural join nt2 where k = "one" +-- !query 2 schema +struct +-- !query 2 output +one 1 1 +one 1 5 + + +-- !query 3 +SELECT * FROM nt1 natural left join nt2 order by v1, v2 +-- !query 3 schema +struct +-- !query 3 output +one 1 1 +one 1 5 +two 2 22 +three 3 NULL + + +-- !query 4 +SELECT * FROM nt1 natural right join nt2 order by v1, v2 +-- !query 4 schema +struct +-- !query 4 output +one 1 1 +one 1 5 +two 2 22 + + +-- !query 5 +SELECT count(*) FROM nt1 natural full outer join nt2 +-- !query 5 schema +struct +-- !query 5 output +4 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 01c7e861bf8a3..6b58159da0c43 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 @@ -38,26 +38,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { setupTestData() - test("having clause") { - withTempView("hav") { - Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") - .createOrReplaceTempView("hav") - checkAnswer( - sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"), - Row("one", 6) :: Row("three", 3) :: Nil) - } - } - - test("having condition contains grouping column") { - withTempView("hav") { - Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v") - .createOrReplaceTempView("hav") - checkAnswer( - sql("SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2"), - Row(1) :: Nil) - } - } - test("SPARK-8010: promote numeric to string") { val df = Seq((1, 1)).toDF("key", "value") df.createOrReplaceTempView("src") @@ -1959,15 +1939,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } - test("SPARK-11032: resolve having correctly") { - withTempView("src") { - Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("src") - checkAnswer( - sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t HAVING(COUNT(1) > 0)"), - Row(1)) - } - } - test("SPARK-11303: filter should not be pushed down into sample") { val df = spark.range(100) List(true, false).foreach { withReplacement => @@ -2507,30 +2478,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } - test("natural join") { - val df1 = Seq(("one", 1), ("two", 2), ("three", 3)).toDF("k", "v1") - val df2 = Seq(("one", 1), ("two", 22), ("one", 5)).toDF("k", "v2") - withTempView("nt1", "nt2") { - df1.createOrReplaceTempView("nt1") - df2.createOrReplaceTempView("nt2") - checkAnswer( - sql("SELECT * FROM nt1 natural join nt2 where k = \"one\""), - Row("one", 1, 1) :: Row("one", 1, 5) :: Nil) - - checkAnswer( - sql("SELECT * FROM nt1 natural left join nt2 order by v1, v2"), - Row("one", 1, 1) :: Row("one", 1, 5) :: Row("two", 2, 22) :: Row("three", 3, null) :: Nil) - - checkAnswer( - sql("SELECT * FROM nt1 natural right join nt2 order by v1, v2"), - Row("one", 1, 1) :: Row("one", 1, 5) :: Row("two", 2, 22) :: Nil) - - checkAnswer( - sql("SELECT count(*) FROM nt1 natural full outer join nt2"), - Row(4) :: Nil) - } - } - test("join with using clause") { val df1 = Seq(("r1c1", "r1c2", "t1r1c3"), ("r2c1", "r2c2", "t1r2c3"), ("r3c1x", "r3c2", "t1r3c3")).toDF("c1", "c2", "c3") @@ -2945,13 +2892,4 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { data.selectExpr("`part.col1`", "`col.1`")) } } - - test("current_date and current_timestamp literals") { - // NOTE that I am comparing the result of the literal with the result of the function call. - // This is done to prevent the test from failing because we are comparing a result to an out - // dated timestamp (quite likely) or date (very unlikely - but equally annoying). - checkAnswer( - sql("select current_date = current_date(), current_timestamp = current_timestamp()"), - Seq(Row(true, true))) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 08b8432d68eb3..14a029ed50a69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql import java.io.File import java.util.{Locale, TimeZone} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType /** * End-to-end test cases for SQL queries. @@ -126,14 +129,18 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { cleaned.split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq } + // Create a local SparkSession to have stronger isolation between different test cases. + // This does not isolate catalog changes. + val localSparkSession = spark.newSession() + // Run the SQL queries preparing them for comparison. val outputs: Seq[QueryOutput] = queries.map { sql => - val df = spark.sql(sql) + val (schema, output) = getNormalizedResult(localSparkSession, sql) // We might need to do some query canonicalization in the future. QueryOutput( sql = sql, - schema = df.schema.catalogString, - output = df.queryExecution.hiveResultString().mkString("\n")) + schema = schema.catalogString, + output = output.mkString("\n")) } if (regenerateGoldenFiles) { @@ -176,6 +183,23 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } } + /** Executes a query and returns the result as (schema of the output, normalized output). */ + private def getNormalizedResult(session: SparkSession, sql: String): (StructType, Seq[String]) = { + // Returns true if the plan is supposed to be sorted. + def isSorted(plan: LogicalPlan): Boolean = plan match { + case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false + case PhysicalOperation(_, _, Sort(_, true, _)) => true + case _ => plan.children.iterator.exists(isSorted) + } + + val df = session.sql(sql) + val schema = df.schema + val answer = df.queryExecution.hiveResultString() + + // If the output is not pre-sorted, sort it. + if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) + } + private def listTestCases(): Seq[TestCase] = { listFilesRecursively(new File(inputFilePath)).map { file => val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" From ea8a198b9838f731458456f369b700815f02198a Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Wed, 10 Aug 2016 21:26:46 -0700 Subject: [PATCH 1124/1470] [SPARK-17007][SQL] Move test data files into a test-data folder ## What changes were proposed in this pull request? This patch moves all the test data files in sql/core/src/test/resources to sql/core/src/test/resources/test-data, so we don't clutter the top level sql/core/src/test/resources. Also deleted sql/core/src/test/resources/old-repeated.parquet since it is no longer used. The change will make it easier to spot sql-tests directory. ## How was this patch tested? This is a test-only change. Author: petermaxlee Closes #14589 from petermaxlee/SPARK-17007. (cherry picked from commit 665e175328130ab3eb0370cdd2a43ed5a7bed1d6) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/JavaDataFrameSuite.java | 12 +++---- .../src/test/resources/old-repeated.parquet | Bin 432 -> 0 bytes .../test/resources/{ => test-data}/bool.csv | 0 .../{ => test-data}/cars-alternative.csv | 0 .../cars-blank-column-name.csv | 0 .../{ => test-data}/cars-malformed.csv | 0 .../resources/{ => test-data}/cars-null.csv | 0 .../cars-unbalanced-quotes.csv | 0 .../test/resources/{ => test-data}/cars.csv | 0 .../test/resources/{ => test-data}/cars.tsv | 0 .../{ => test-data}/cars_iso-8859-1.csv | 0 .../resources/{ => test-data}/comments.csv | 0 .../test/resources/{ => test-data}/dates.csv | 0 .../{ => test-data}/dec-in-fixed-len.parquet | Bin .../{ => test-data}/dec-in-i32.parquet | Bin .../{ => test-data}/dec-in-i64.parquet | Bin .../resources/{ => test-data}/decimal.csv | 0 .../{ => test-data}/disable_comments.csv | 0 .../test/resources/{ => test-data}/empty.csv | 0 .../nested-array-struct.parquet | Bin .../resources/{ => test-data}/numbers.csv | 0 .../{ => test-data}/old-repeated-int.parquet | Bin .../old-repeated-message.parquet | Bin .../parquet-thrift-compat.snappy.parquet | Bin .../proto-repeated-string.parquet | Bin .../proto-repeated-struct.parquet | Bin .../proto-struct-with-array-many.parquet | Bin .../proto-struct-with-array.parquet | Bin .../{ => test-data}/simple_sparse.csv | 0 .../text-partitioned/year=2014/data.txt | 0 .../text-partitioned/year=2015/data.txt | 0 .../resources/{ => test-data}/text-suite.txt | 0 .../resources/{ => test-data}/text-suite2.txt | 0 .../{ => test-data}/unescaped-quotes.csv | 0 .../sql/execution/command/DDLSuite.scala | 3 +- .../execution/datasources/csv/CSVSuite.scala | 34 +++++++++--------- .../datasources/parquet/ParquetIOSuite.scala | 6 ++-- .../ParquetProtobufCompatibilitySuite.scala | 14 ++++---- .../ParquetThriftCompatibilitySuite.scala | 4 +-- .../datasources/text/TextSuite.scala | 6 ++-- 40 files changed, 40 insertions(+), 39 deletions(-) delete mode 100644 sql/core/src/test/resources/old-repeated.parquet rename sql/core/src/test/resources/{ => test-data}/bool.csv (100%) rename sql/core/src/test/resources/{ => test-data}/cars-alternative.csv (100%) rename sql/core/src/test/resources/{ => test-data}/cars-blank-column-name.csv (100%) rename sql/core/src/test/resources/{ => test-data}/cars-malformed.csv (100%) rename sql/core/src/test/resources/{ => test-data}/cars-null.csv (100%) rename sql/core/src/test/resources/{ => test-data}/cars-unbalanced-quotes.csv (100%) rename sql/core/src/test/resources/{ => test-data}/cars.csv (100%) rename sql/core/src/test/resources/{ => test-data}/cars.tsv (100%) rename sql/core/src/test/resources/{ => test-data}/cars_iso-8859-1.csv (100%) rename sql/core/src/test/resources/{ => test-data}/comments.csv (100%) rename sql/core/src/test/resources/{ => test-data}/dates.csv (100%) rename sql/core/src/test/resources/{ => test-data}/dec-in-fixed-len.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/dec-in-i32.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/dec-in-i64.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/decimal.csv (100%) rename sql/core/src/test/resources/{ => test-data}/disable_comments.csv (100%) rename sql/core/src/test/resources/{ => test-data}/empty.csv (100%) rename sql/core/src/test/resources/{ => test-data}/nested-array-struct.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/numbers.csv (100%) rename sql/core/src/test/resources/{ => test-data}/old-repeated-int.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/old-repeated-message.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/parquet-thrift-compat.snappy.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/proto-repeated-string.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/proto-repeated-struct.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/proto-struct-with-array-many.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/proto-struct-with-array.parquet (100%) rename sql/core/src/test/resources/{ => test-data}/simple_sparse.csv (100%) rename sql/core/src/test/resources/{ => test-data}/text-partitioned/year=2014/data.txt (100%) rename sql/core/src/test/resources/{ => test-data}/text-partitioned/year=2015/data.txt (100%) rename sql/core/src/test/resources/{ => test-data}/text-suite.txt (100%) rename sql/core/src/test/resources/{ => test-data}/text-suite2.txt (100%) rename sql/core/src/test/resources/{ => test-data}/unescaped-quotes.csv (100%) 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 318b53cdbbaa0..c44fc3d393862 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 @@ -327,23 +327,23 @@ private String getResource(String resource) { @Test public void testGenericLoad() { - Dataset df1 = spark.read().format("text").load(getResource("text-suite.txt")); + Dataset df1 = spark.read().format("text").load(getResource("test-data/text-suite.txt")); Assert.assertEquals(4L, df1.count()); Dataset df2 = spark.read().format("text").load( - getResource("text-suite.txt"), - getResource("text-suite2.txt")); + getResource("test-data/text-suite.txt"), + getResource("test-data/text-suite2.txt")); Assert.assertEquals(5L, df2.count()); } @Test public void testTextLoad() { - Dataset ds1 = spark.read().textFile(getResource("text-suite.txt")); + Dataset ds1 = spark.read().textFile(getResource("test-data/text-suite.txt")); Assert.assertEquals(4L, ds1.count()); Dataset ds2 = spark.read().textFile( - getResource("text-suite.txt"), - getResource("text-suite2.txt")); + getResource("test-data/text-suite.txt"), + getResource("test-data/text-suite2.txt")); Assert.assertEquals(5L, ds2.count()); } diff --git a/sql/core/src/test/resources/old-repeated.parquet b/sql/core/src/test/resources/old-repeated.parquet deleted file mode 100644 index 213f1a90291b30a8a3161b51c38f008f3ae9f6e5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 432 zcmZWm!D@p*5ZxNF!5+)X3PMGioUA16Ei?y9g$B|h;-#ms>Ld--Xm{5`DgF13A<&42 znSH!BJNtMWhsm50I-@h68VC$(I7}ZALYRJm-NGUo*2p;a%Z@xEJgH{;FE=Sj6^mNc zS-TAqXn-pyRtNP8Qt};84d*60yAuBru{7JUo$1)Y6%&KlJ(Uv6u!JS1zOP)cw zaM$5ewB9699EEB0jJ*18aDDn7N1N4K`fzXlnuJ~V?c^nwk}Yeo3wXox4+#3Y!pMU2 V+-`?%2{TWZ?kYh(G4~k%>JK8=aDe~- diff --git a/sql/core/src/test/resources/bool.csv b/sql/core/src/test/resources/test-data/bool.csv similarity index 100% rename from sql/core/src/test/resources/bool.csv rename to sql/core/src/test/resources/test-data/bool.csv diff --git a/sql/core/src/test/resources/cars-alternative.csv b/sql/core/src/test/resources/test-data/cars-alternative.csv similarity index 100% rename from sql/core/src/test/resources/cars-alternative.csv rename to sql/core/src/test/resources/test-data/cars-alternative.csv diff --git a/sql/core/src/test/resources/cars-blank-column-name.csv b/sql/core/src/test/resources/test-data/cars-blank-column-name.csv similarity index 100% rename from sql/core/src/test/resources/cars-blank-column-name.csv rename to sql/core/src/test/resources/test-data/cars-blank-column-name.csv diff --git a/sql/core/src/test/resources/cars-malformed.csv b/sql/core/src/test/resources/test-data/cars-malformed.csv similarity index 100% rename from sql/core/src/test/resources/cars-malformed.csv rename to sql/core/src/test/resources/test-data/cars-malformed.csv diff --git a/sql/core/src/test/resources/cars-null.csv b/sql/core/src/test/resources/test-data/cars-null.csv similarity index 100% rename from sql/core/src/test/resources/cars-null.csv rename to sql/core/src/test/resources/test-data/cars-null.csv diff --git a/sql/core/src/test/resources/cars-unbalanced-quotes.csv b/sql/core/src/test/resources/test-data/cars-unbalanced-quotes.csv similarity index 100% rename from sql/core/src/test/resources/cars-unbalanced-quotes.csv rename to sql/core/src/test/resources/test-data/cars-unbalanced-quotes.csv diff --git a/sql/core/src/test/resources/cars.csv b/sql/core/src/test/resources/test-data/cars.csv similarity index 100% rename from sql/core/src/test/resources/cars.csv rename to sql/core/src/test/resources/test-data/cars.csv diff --git a/sql/core/src/test/resources/cars.tsv b/sql/core/src/test/resources/test-data/cars.tsv similarity index 100% rename from sql/core/src/test/resources/cars.tsv rename to sql/core/src/test/resources/test-data/cars.tsv diff --git a/sql/core/src/test/resources/cars_iso-8859-1.csv b/sql/core/src/test/resources/test-data/cars_iso-8859-1.csv similarity index 100% rename from sql/core/src/test/resources/cars_iso-8859-1.csv rename to sql/core/src/test/resources/test-data/cars_iso-8859-1.csv diff --git a/sql/core/src/test/resources/comments.csv b/sql/core/src/test/resources/test-data/comments.csv similarity index 100% rename from sql/core/src/test/resources/comments.csv rename to sql/core/src/test/resources/test-data/comments.csv diff --git a/sql/core/src/test/resources/dates.csv b/sql/core/src/test/resources/test-data/dates.csv similarity index 100% rename from sql/core/src/test/resources/dates.csv rename to sql/core/src/test/resources/test-data/dates.csv diff --git a/sql/core/src/test/resources/dec-in-fixed-len.parquet b/sql/core/src/test/resources/test-data/dec-in-fixed-len.parquet similarity index 100% rename from sql/core/src/test/resources/dec-in-fixed-len.parquet rename to sql/core/src/test/resources/test-data/dec-in-fixed-len.parquet diff --git a/sql/core/src/test/resources/dec-in-i32.parquet b/sql/core/src/test/resources/test-data/dec-in-i32.parquet similarity index 100% rename from sql/core/src/test/resources/dec-in-i32.parquet rename to sql/core/src/test/resources/test-data/dec-in-i32.parquet diff --git a/sql/core/src/test/resources/dec-in-i64.parquet b/sql/core/src/test/resources/test-data/dec-in-i64.parquet similarity index 100% rename from sql/core/src/test/resources/dec-in-i64.parquet rename to sql/core/src/test/resources/test-data/dec-in-i64.parquet diff --git a/sql/core/src/test/resources/decimal.csv b/sql/core/src/test/resources/test-data/decimal.csv similarity index 100% rename from sql/core/src/test/resources/decimal.csv rename to sql/core/src/test/resources/test-data/decimal.csv diff --git a/sql/core/src/test/resources/disable_comments.csv b/sql/core/src/test/resources/test-data/disable_comments.csv similarity index 100% rename from sql/core/src/test/resources/disable_comments.csv rename to sql/core/src/test/resources/test-data/disable_comments.csv diff --git a/sql/core/src/test/resources/empty.csv b/sql/core/src/test/resources/test-data/empty.csv similarity index 100% rename from sql/core/src/test/resources/empty.csv rename to sql/core/src/test/resources/test-data/empty.csv diff --git a/sql/core/src/test/resources/nested-array-struct.parquet b/sql/core/src/test/resources/test-data/nested-array-struct.parquet similarity index 100% rename from sql/core/src/test/resources/nested-array-struct.parquet rename to sql/core/src/test/resources/test-data/nested-array-struct.parquet diff --git a/sql/core/src/test/resources/numbers.csv b/sql/core/src/test/resources/test-data/numbers.csv similarity index 100% rename from sql/core/src/test/resources/numbers.csv rename to sql/core/src/test/resources/test-data/numbers.csv diff --git a/sql/core/src/test/resources/old-repeated-int.parquet b/sql/core/src/test/resources/test-data/old-repeated-int.parquet similarity index 100% rename from sql/core/src/test/resources/old-repeated-int.parquet rename to sql/core/src/test/resources/test-data/old-repeated-int.parquet diff --git a/sql/core/src/test/resources/old-repeated-message.parquet b/sql/core/src/test/resources/test-data/old-repeated-message.parquet similarity index 100% rename from sql/core/src/test/resources/old-repeated-message.parquet rename to sql/core/src/test/resources/test-data/old-repeated-message.parquet diff --git a/sql/core/src/test/resources/parquet-thrift-compat.snappy.parquet b/sql/core/src/test/resources/test-data/parquet-thrift-compat.snappy.parquet similarity index 100% rename from sql/core/src/test/resources/parquet-thrift-compat.snappy.parquet rename to sql/core/src/test/resources/test-data/parquet-thrift-compat.snappy.parquet diff --git a/sql/core/src/test/resources/proto-repeated-string.parquet b/sql/core/src/test/resources/test-data/proto-repeated-string.parquet similarity index 100% rename from sql/core/src/test/resources/proto-repeated-string.parquet rename to sql/core/src/test/resources/test-data/proto-repeated-string.parquet diff --git a/sql/core/src/test/resources/proto-repeated-struct.parquet b/sql/core/src/test/resources/test-data/proto-repeated-struct.parquet similarity index 100% rename from sql/core/src/test/resources/proto-repeated-struct.parquet rename to sql/core/src/test/resources/test-data/proto-repeated-struct.parquet diff --git a/sql/core/src/test/resources/proto-struct-with-array-many.parquet b/sql/core/src/test/resources/test-data/proto-struct-with-array-many.parquet similarity index 100% rename from sql/core/src/test/resources/proto-struct-with-array-many.parquet rename to sql/core/src/test/resources/test-data/proto-struct-with-array-many.parquet diff --git a/sql/core/src/test/resources/proto-struct-with-array.parquet b/sql/core/src/test/resources/test-data/proto-struct-with-array.parquet similarity index 100% rename from sql/core/src/test/resources/proto-struct-with-array.parquet rename to sql/core/src/test/resources/test-data/proto-struct-with-array.parquet diff --git a/sql/core/src/test/resources/simple_sparse.csv b/sql/core/src/test/resources/test-data/simple_sparse.csv similarity index 100% rename from sql/core/src/test/resources/simple_sparse.csv rename to sql/core/src/test/resources/test-data/simple_sparse.csv diff --git a/sql/core/src/test/resources/text-partitioned/year=2014/data.txt b/sql/core/src/test/resources/test-data/text-partitioned/year=2014/data.txt similarity index 100% rename from sql/core/src/test/resources/text-partitioned/year=2014/data.txt rename to sql/core/src/test/resources/test-data/text-partitioned/year=2014/data.txt diff --git a/sql/core/src/test/resources/text-partitioned/year=2015/data.txt b/sql/core/src/test/resources/test-data/text-partitioned/year=2015/data.txt similarity index 100% rename from sql/core/src/test/resources/text-partitioned/year=2015/data.txt rename to sql/core/src/test/resources/test-data/text-partitioned/year=2015/data.txt diff --git a/sql/core/src/test/resources/text-suite.txt b/sql/core/src/test/resources/test-data/text-suite.txt similarity index 100% rename from sql/core/src/test/resources/text-suite.txt rename to sql/core/src/test/resources/test-data/text-suite.txt diff --git a/sql/core/src/test/resources/text-suite2.txt b/sql/core/src/test/resources/test-data/text-suite2.txt similarity index 100% rename from sql/core/src/test/resources/text-suite2.txt rename to sql/core/src/test/resources/test-data/text-suite2.txt diff --git a/sql/core/src/test/resources/unescaped-quotes.csv b/sql/core/src/test/resources/test-data/unescaped-quotes.csv similarity index 100% rename from sql/core/src/test/resources/unescaped-quotes.csv rename to sql/core/src/test/resources/test-data/unescaped-quotes.csv diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index b276e2d0c37e0..8ca4722b2b803 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -436,7 +436,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("create temporary view using") { - val csvFile = Thread.currentThread().getContextClassLoader.getResource("cars.csv").toString() + val csvFile = + Thread.currentThread().getContextClassLoader.getResource("test-data/cars.csv").toString withView("testview") { sql(s"CREATE OR REPLACE TEMPORARY VIEW testview (c1: String, c2: String) USING " + "org.apache.spark.sql.execution.datasources.csv.CSVFileFormat " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 311f1fa8d2aff..8cd76ddf20f04 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -33,23 +33,23 @@ import org.apache.spark.sql.types._ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { import testImplicits._ - private val carsFile = "cars.csv" - private val carsMalformedFile = "cars-malformed.csv" - private val carsFile8859 = "cars_iso-8859-1.csv" - private val carsTsvFile = "cars.tsv" - private val carsAltFile = "cars-alternative.csv" - private val carsUnbalancedQuotesFile = "cars-unbalanced-quotes.csv" - private val carsNullFile = "cars-null.csv" - private val carsBlankColName = "cars-blank-column-name.csv" - private val emptyFile = "empty.csv" - private val commentsFile = "comments.csv" - private val disableCommentsFile = "disable_comments.csv" - private val boolFile = "bool.csv" - private val decimalFile = "decimal.csv" - private val simpleSparseFile = "simple_sparse.csv" - private val numbersFile = "numbers.csv" - private val datesFile = "dates.csv" - private val unescapedQuotesFile = "unescaped-quotes.csv" + private val carsFile = "test-data/cars.csv" + private val carsMalformedFile = "test-data/cars-malformed.csv" + private val carsFile8859 = "test-data/cars_iso-8859-1.csv" + private val carsTsvFile = "test-data/cars.tsv" + private val carsAltFile = "test-data/cars-alternative.csv" + private val carsUnbalancedQuotesFile = "test-data/cars-unbalanced-quotes.csv" + private val carsNullFile = "test-data/cars-null.csv" + private val carsBlankColName = "test-data/cars-blank-column-name.csv" + private val emptyFile = "test-data/empty.csv" + private val commentsFile = "test-data/comments.csv" + private val disableCommentsFile = "test-data/disable_comments.csv" + private val boolFile = "test-data/bool.csv" + private val decimalFile = "test-data/decimal.csv" + private val simpleSparseFile = "test-data/simple_sparse.csv" + private val numbersFile = "test-data/numbers.csv" + private val datesFile = "test-data/dates.csv" + private val unescapedQuotesFile = "test-data/unescaped-quotes.csv" private def testFile(fileName: String): String = { Thread.currentThread().getContextClassLoader.getResource(fileName).toString diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index fc9ce6bb3041b..a95de2ea9135f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -556,7 +556,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { checkAnswer( // Decimal column in this file is encoded using plain dictionary - readResourceParquetFile("dec-in-i32.parquet"), + readResourceParquetFile("test-data/dec-in-i32.parquet"), spark.range(1 << 4).select('id % 10 cast DecimalType(5, 2) as 'i32_dec)) } } @@ -567,7 +567,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { checkAnswer( // Decimal column in this file is encoded using plain dictionary - readResourceParquetFile("dec-in-i64.parquet"), + readResourceParquetFile("test-data/dec-in-i64.parquet"), spark.range(1 << 4).select('id % 10 cast DecimalType(10, 2) as 'i64_dec)) } } @@ -578,7 +578,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { checkAnswer( // Decimal column in this file is encoded using plain dictionary - readResourceParquetFile("dec-in-fixed-len.parquet"), + readResourceParquetFile("test-data/dec-in-fixed-len.parquet"), spark.range(1 << 4).select('id % 10 cast DecimalType(10, 2) as 'fixed_len_dec)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala index 98333e58cada8..fa88019298a69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala @@ -22,12 +22,12 @@ import org.apache.spark.sql.test.SharedSQLContext class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { test("unannotated array of primitive type") { - checkAnswer(readResourceParquetFile("old-repeated-int.parquet"), Row(Seq(1, 2, 3))) + checkAnswer(readResourceParquetFile("test-data/old-repeated-int.parquet"), Row(Seq(1, 2, 3))) } test("unannotated array of struct") { checkAnswer( - readResourceParquetFile("old-repeated-message.parquet"), + readResourceParquetFile("test-data/old-repeated-message.parquet"), Row( Seq( Row("First inner", null, null), @@ -35,14 +35,14 @@ class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with Sh Row(null, null, "Third inner")))) checkAnswer( - readResourceParquetFile("proto-repeated-struct.parquet"), + readResourceParquetFile("test-data/proto-repeated-struct.parquet"), Row( Seq( Row("0 - 1", "0 - 2", "0 - 3"), Row("1 - 1", "1 - 2", "1 - 3")))) checkAnswer( - readResourceParquetFile("proto-struct-with-array-many.parquet"), + readResourceParquetFile("test-data/proto-struct-with-array-many.parquet"), Seq( Row( Seq( @@ -60,13 +60,13 @@ class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with Sh test("struct with unannotated array") { checkAnswer( - readResourceParquetFile("proto-struct-with-array.parquet"), + readResourceParquetFile("test-data/proto-struct-with-array.parquet"), Row(10, 9, Seq.empty, null, Row(9), Seq(Row(9), Row(10)))) } test("unannotated array of struct with unannotated array") { checkAnswer( - readResourceParquetFile("nested-array-struct.parquet"), + readResourceParquetFile("test-data/nested-array-struct.parquet"), Seq( Row(2, Seq(Row(1, Seq(Row(3))))), Row(5, Seq(Row(4, Seq(Row(6))))), @@ -75,7 +75,7 @@ class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with Sh test("unannotated array of string") { checkAnswer( - readResourceParquetFile("proto-repeated-string.parquet"), + readResourceParquetFile("test-data/proto-repeated-string.parquet"), Seq( Row(Seq("hello", "world")), Row(Seq("good", "bye")), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala index ff5706999a6dd..4157a5b46dc42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.test.SharedSQLContext class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { import ParquetCompatibilityTest._ - private val parquetFilePath = - Thread.currentThread().getContextClassLoader.getResource("parquet-thrift-compat.snappy.parquet") + private val parquetFilePath = Thread.currentThread().getContextClassLoader.getResource( + "test-data/parquet-thrift-compat.snappy.parquet") test("Read Parquet file generated by parquet-thrift") { logInfo( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 71d3da915840a..d11c2acb815d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -66,7 +66,7 @@ class TextSuite extends QueryTest with SharedSQLContext { test("reading partitioned data using read.textFile()") { val partitionedData = Thread.currentThread().getContextClassLoader - .getResource("text-partitioned").toString + .getResource("test-data/text-partitioned").toString val ds = spark.read.textFile(partitionedData) val data = ds.collect() @@ -76,7 +76,7 @@ class TextSuite extends QueryTest with SharedSQLContext { test("support for partitioned reading using read.text()") { val partitionedData = Thread.currentThread().getContextClassLoader - .getResource("text-partitioned").toString + .getResource("test-data/text-partitioned").toString val df = spark.read.text(partitionedData) val data = df.filter("year = '2015'").select("value").collect() @@ -155,7 +155,7 @@ class TextSuite extends QueryTest with SharedSQLContext { } private def testFile: String = { - Thread.currentThread().getContextClassLoader.getResource("text-suite.txt").toString + Thread.currentThread().getContextClassLoader.getResource("test-data/text-suite.txt").toString } /** Verifies data and schema. */ From 4b434e7dadffd83fe701668a23f0ece03e3f08bb Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Wed, 10 Aug 2016 23:22:14 -0700 Subject: [PATCH 1125/1470] [SPARK-17011][SQL] Support testing exceptions in SQLQueryTestSuite ## What changes were proposed in this pull request? This patch adds exception testing to SQLQueryTestSuite. When there is an exception in query execution, the query result contains the the exception class along with the exception message. As part of this, I moved some additional test cases for limit from SQLQuerySuite over to SQLQueryTestSuite. ## How was this patch tested? This is a test harness change. Author: petermaxlee Closes #14592 from petermaxlee/SPARK-17011. (cherry picked from commit 0db373aaf87991207a7a8a09853b6fa602f0f45b) Signed-off-by: Wenchen Fan --- .../test/resources/sql-tests/inputs/limit.sql | 20 +++++ .../sql-tests/inputs/number-format.sql | 7 +- .../sql-tests/results/datetime.sql.out | 2 +- .../sql-tests/results/having.sql.out | 2 +- .../resources/sql-tests/results/limit.sql.out | 83 +++++++++++++++++++ .../sql-tests/results/natural-join.sql.out | 2 +- .../sql-tests/results/number-format.sql.out | 22 +++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 50 ----------- .../apache/spark/sql/SQLQueryTestSuite.scala | 41 +++++++-- 9 files changed, 161 insertions(+), 68 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/limit.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/limit.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/limit.sql new file mode 100644 index 0000000000000..892a1bb4b559f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/limit.sql @@ -0,0 +1,20 @@ + +-- limit on various data types +select * from testdata limit 2; +select * from arraydata limit 2; +select * from mapdata limit 2; + +-- foldable non-literal in limit +select * from testdata limit 2 + 1; + +select * from testdata limit CAST(1 AS int); + +-- limit must be non-negative +select * from testdata limit -1; + +-- limit must be foldable +select * from testdata limit key > 3; + +-- limit must be integer +select * from testdata limit true; +select * from testdata limit 'a'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/number-format.sql b/sql/core/src/test/resources/sql-tests/inputs/number-format.sql index 60076a8431582..a32d0688f813a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/number-format.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/number-format.sql @@ -3,10 +3,13 @@ -- parse as ints select 1, -1; --- parse as longs +-- parse as longs (Int.MaxValue + 1, and Int.MinValue - 1) select 2147483648, -2147483649; --- parse as decimals +-- parse long min and max value +select 9223372036854775807, -9223372036854775808; + +-- parse as decimals (Long.MaxValue + 1, and Long.MinValue - 1) select 9223372036854775808, -9223372036854775809; -- various floating point (decimal) formats diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 51746579b1314..032e4258500fb 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,4 +1,4 @@ --- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite +-- Automatically generated by SQLQueryTestSuite -- Number of queries: 1 diff --git a/sql/core/src/test/resources/sql-tests/results/having.sql.out b/sql/core/src/test/resources/sql-tests/results/having.sql.out index 0bc8be66be638..e0923832673cb 100644 --- a/sql/core/src/test/resources/sql-tests/results/having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/having.sql.out @@ -1,4 +1,4 @@ --- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite +-- Automatically generated by SQLQueryTestSuite -- Number of queries: 4 diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/limit.sql.out new file mode 100644 index 0000000000000..b71b05886986c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out @@ -0,0 +1,83 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 9 + + +-- !query 0 +select * from testdata limit 2 +-- !query 0 schema +struct +-- !query 0 output +1 1 +2 2 + + +-- !query 1 +select * from arraydata limit 2 +-- !query 1 schema +struct,nestedarraycol:array>> +-- !query 1 output +[1,2,3] [[1,2,3]] +[2,3,4] [[2,3,4]] + + +-- !query 2 +select * from mapdata limit 2 +-- !query 2 schema +struct> +-- !query 2 output +{1:"a1",2:"b1",3:"c1",4:"d1",5:"e1"} +{1:"a2",2:"b2",3:"c2",4:"d2"} + + +-- !query 3 +select * from testdata limit 2 + 1 +-- !query 3 schema +struct +-- !query 3 output +1 1 +2 2 +3 3 + + +-- !query 4 +select * from testdata limit CAST(1 AS int) +-- !query 4 schema +struct +-- !query 4 output +1 1 + + +-- !query 5 +select * from testdata limit -1 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +The limit expression must be equal to or greater than 0, but got -1; + + +-- !query 6 +select * from testdata limit key > 3 +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +The limit expression must evaluate to a constant value, but got (testdata.`key` > 3); + + +-- !query 7 +select * from testdata limit true +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +The limit expression must be integer type, but got boolean; + + +-- !query 8 +select * from testdata limit 'a' +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +The limit expression must be integer type, but got string; diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out index d4954dabea8c9..43f2f9af61d9b 100644 --- a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out @@ -1,4 +1,4 @@ --- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite +-- Automatically generated by SQLQueryTestSuite -- Number of queries: 6 diff --git a/sql/core/src/test/resources/sql-tests/results/number-format.sql.out b/sql/core/src/test/resources/sql-tests/results/number-format.sql.out index 4b800b7d92560..82a1d39c0a0be 100644 --- a/sql/core/src/test/resources/sql-tests/results/number-format.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/number-format.sql.out @@ -1,5 +1,5 @@ --- Automatically generated by org.apache.spark.sql.SQLQueryTestSuite --- Number of queries: 4 +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 -- !query 0 @@ -19,16 +19,24 @@ struct<2147483648:bigint,(-2147483649):bigint> -- !query 2 -select 9223372036854775808, -9223372036854775809 +select 9223372036854775807, -9223372036854775808 -- !query 2 schema -struct<9223372036854775808:decimal(19,0),(-9223372036854775809):decimal(19,0)> +struct<9223372036854775807:bigint,(-9223372036854775808):decimal(19,0)> -- !query 2 output -9223372036854775808 -9223372036854775809 +9223372036854775807 -9223372036854775808 -- !query 3 -select 0.3, -0.8, .5, -.18, 0.1111 +select 9223372036854775808, -9223372036854775809 -- !query 3 schema -struct<0.3:decimal(1,1),(-0.8):decimal(1,1),0.5:decimal(1,1),(-0.18):decimal(2,2),0.1111:decimal(4,4)> +struct<9223372036854775808:decimal(19,0),(-9223372036854775809):decimal(19,0)> -- !query 3 output +9223372036854775808 -9223372036854775809 + + +-- !query 4 +select 0.3, -0.8, .5, -.18, 0.1111 +-- !query 4 schema +struct<0.3:decimal(1,1),(-0.8):decimal(1,1),0.5:decimal(1,1),(-0.18):decimal(2,2),0.1111:decimal(4,4)> +-- !query 4 output 0.3 -0.8 0.5 -0.18 0.1111 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 6b58159da0c43..3de5086b15c9c 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 @@ -650,51 +650,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sortTest() } - test("limit") { - checkAnswer( - sql("SELECT * FROM testData LIMIT 9 + 1"), - testData.take(10).toSeq) - - checkAnswer( - sql("SELECT * FROM arrayData LIMIT CAST(1 AS Integer)"), - arrayData.collect().take(1).map(Row.fromTuple).toSeq) - - checkAnswer( - sql("SELECT * FROM mapData LIMIT 1"), - mapData.collect().take(1).map(Row.fromTuple).toSeq) - } - - test("non-foldable expressions in LIMIT") { - val e = intercept[AnalysisException] { - sql("SELECT * FROM testData LIMIT key > 3") - }.getMessage - assert(e.contains("The limit expression must evaluate to a constant value, " + - "but got (testdata.`key` > 3)")) - } - - test("Expressions in limit clause are not integer") { - var e = intercept[AnalysisException] { - sql("SELECT * FROM testData LIMIT true") - }.getMessage - assert(e.contains("The limit expression must be integer type, but got boolean")) - - e = intercept[AnalysisException] { - sql("SELECT * FROM testData LIMIT 'a'") - }.getMessage - assert(e.contains("The limit expression must be integer type, but got string")) - } - test("negative in LIMIT or TABLESAMPLE") { val expected = "The limit expression must be equal to or greater than 0, but got -1" var e = intercept[AnalysisException] { sql("SELECT * FROM testData TABLESAMPLE (-1 rows)") }.getMessage assert(e.contains(expected)) - - e = intercept[AnalysisException] { - sql("SELECT * FROM testData LIMIT -1") - }.getMessage - assert(e.contains(expected)) } test("CTE feature") { @@ -1327,17 +1288,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } - test("Test to check we can use Long.MinValue") { - checkAnswer( - sql(s"SELECT ${Long.MinValue} FROM testData ORDER BY key LIMIT 1"), Row(Long.MinValue) - ) - - checkAnswer( - sql(s"SELECT key FROM testData WHERE key > ${Long.MinValue}"), - (1 to 100).map(Row(_)).toSeq - ) - } - test("Test to check we can apply sign to expression") { checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 14a029ed50a69..1022c38e262b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql import java.io.File import java.util.{Locale, TimeZone} +import scala.util.control.NonFatal + import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -132,6 +134,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // Create a local SparkSession to have stronger isolation between different test cases. // This does not isolate catalog changes. val localSparkSession = spark.newSession() + loadTestData(localSparkSession) // Run the SQL queries preparing them for comparison. val outputs: Seq[QueryOutput] = queries.map { sql => @@ -146,7 +149,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { if (regenerateGoldenFiles) { // Again, we are explicitly not using multi-line string due to stripMargin removing "|". val goldenOutput = { - s"-- Automatically generated by ${getClass.getName}\n" + + s"-- Automatically generated by ${getClass.getSimpleName}\n" + s"-- Number of queries: ${outputs.size}\n\n\n" + outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n" } @@ -192,12 +195,19 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { case _ => plan.children.iterator.exists(isSorted) } - val df = session.sql(sql) - val schema = df.schema - val answer = df.queryExecution.hiveResultString() + try { + val df = session.sql(sql) + val schema = df.schema + val answer = df.queryExecution.hiveResultString() + + // If the output is not pre-sorted, sort it. + if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) - // If the output is not pre-sorted, sort it. - if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) + } catch { + case NonFatal(e) => + // If there is an exception, put the exception class followed by the message. + (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage)) + } } private def listTestCases(): Seq[TestCase] = { @@ -213,6 +223,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { files ++ dirs.flatMap(listFilesRecursively) } + /** Load built-in test tables into the SparkSession. */ + private def loadTestData(session: SparkSession): Unit = { + import session.implicits._ + + (1 to 100).map(i => (i, i.toString)).toDF("key", "value").createOrReplaceTempView("testdata") + + ((Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: (Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) + .toDF("arraycol", "nestedarraycol") + .createOrReplaceTempView("arraydata") + + (Tuple1(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: + Tuple1(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: + Tuple1(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: + Tuple1(Map(1 -> "a4", 2 -> "b4")) :: + Tuple1(Map(1 -> "a5")) :: Nil) + .toDF("mapcol") + .createOrReplaceTempView("mapdata") + } + private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault From 0ed6236e94318ae0b56363ee1aef4a5577eeebd3 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 11 Aug 2016 11:26:57 +0100 Subject: [PATCH 1126/1470] Correct example value for spark.ssl.YYY.XXX settings Docs adjustment to: - link to other relevant section of docs - correct statement about the only value when actually other values are supported Author: Andrew Ash Closes #14581 from ash211/patch-10. (cherry picked from commit 8a6b7037bb058d00cc767895c3292509576ea2f9) Signed-off-by: Sean Owen --- docs/configuration.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8facd0ecf367a..500a6dad113da 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1449,8 +1449,10 @@ Apart from these, the following properties are also available, and may be useful the properties must be overwritten in the protocol-specific namespace.

    Use spark.ssl.YYY.XXX settings to overwrite the global configuration for - particular protocol denoted by YYY. Currently YYY can be - only fs for file server.

    + particular protocol denoted by YYY. Example values for YYY + include fs, ui, standalone, and + historyServer. See SSL + Configuration for details on hierarchical SSL configuration for services.

    From 33a213f330bd746fb54783b16ea90c91b23a02a6 Mon Sep 17 00:00:00 2001 From: avulanov Date: Thu, 11 Aug 2016 13:07:14 +0100 Subject: [PATCH 1127/1470] [SPARK-15899][SQL] Fix the construction of the file path with hadoop Path for Spark 2.0 This PR contains the adaptation of https://github.com/apache/spark/pull/13868 for Spark 2.0 ## What changes were proposed in this pull request? Fix the construction of the file path in `SQLConf.scala` and unit tests that rely on this: `SQLConfSuite` and `DDLSuite`. Previous way of construction caused the creation of incorrect path on Windows. ## How was this patch tested? Run unit tests on Windows Author: avulanov Closes #14600 from avulanov/SPARK-15899-file-2.0. --- .../apache/spark/sql/internal/SQLConf.scala | 7 +-- .../sql/execution/command/DDLSuite.scala | 44 ++++++++++--------- .../spark/sql/internal/SQLConfSuite.scala | 4 +- 3 files changed, 30 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1a9bb6a0b54e1..0666a99cfc43e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.immutable +import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.internal.Logging @@ -55,7 +56,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("file:${system:user.dir}/spark-warehouse") + .createWithDefault("${system:user.dir}/spark-warehouse") val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() @@ -691,9 +692,9 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH) def warehousePath: String = { - getConf(WAREHOUSE_PATH).replace("${system:user.dir}", System.getProperty("user.dir")) + new Path(getConf(WAREHOUSE_PATH).replace("${system:user.dir}", + System.getProperty("user.dir"))).toString } - override def orderByOrdinal: Boolean = getConf(ORDER_BY_ORDINAL) override def groupByOrdinal: Boolean = getConf(GROUP_BY_ORDINAL) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 8ca4722b2b803..d70cae74bc6c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -111,10 +111,6 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) } - private def appendTrailingSlash(path: String): String = { - if (!path.endsWith(File.separator)) path + File.separator else path - } - test("the qualified path of a database is stored in the catalog") { val catalog = spark.sessionState.catalog @@ -122,18 +118,19 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val path = tmpDir.toString // The generated temp path is not qualified. assert(!path.startsWith("file:/")) - sql(s"CREATE DATABASE db1 LOCATION '$path'") + val uri = tmpDir.toURI + sql(s"CREATE DATABASE db1 LOCATION '$uri'") val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri assert("file" === pathInCatalog.getScheme) - val expectedPath = if (path.endsWith(File.separator)) path.dropRight(1) else path - assert(expectedPath === pathInCatalog.getPath) + val expectedPath = new Path(path).toUri + assert(expectedPath.getPath === pathInCatalog.getPath) withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { sql(s"CREATE DATABASE db2") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri - assert("file" === pathInCatalog.getScheme) - val expectedPath = appendTrailingSlash(spark.sessionState.conf.warehousePath) + "db2.db" - assert(expectedPath === pathInCatalog.getPath) + val pathInCatalog2 = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + assert("file" === pathInCatalog2.getScheme) + val expectedPath2 = new Path(spark.sessionState.conf.warehousePath + "/" + "db2.db").toUri + assert(expectedPath2.getPath === pathInCatalog2.getPath) } sql("DROP DATABASE db1") @@ -141,6 +138,13 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + private def makeQualifiedPath(path: String): String = { + // copy-paste from SessionCatalog + val hadoopPath = new Path(path) + val fs = hadoopPath.getFileSystem(sparkContext.hadoopConfiguration) + fs.makeQualified(hadoopPath).toString + } + test("Create/Drop Database") { withTempDir { tmpDir => val path = tmpDir.toString @@ -154,8 +158,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -181,8 +184,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) val expectedLocation = - "file:" + appendTrailingSlash(System.getProperty("user.dir")) + - s"spark-warehouse/$dbName.db" + makeQualifiedPath(s"${System.getProperty("user.dir")}/spark-warehouse" + + "/" + s"$dbName.db") assert(db1 == CatalogDatabase( dbName, "", @@ -200,17 +203,17 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => - val path = tmpDir.toString - val dbPath = "file:" + path + val path = new Path(tmpDir.toString).toUri.toString databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + val expPath = makeQualifiedPath(tmpDir.toString) assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", - if (dbPath.endsWith(File.separator)) dbPath.dropRight(1) else dbPath, + expPath, Map.empty)) sql(s"DROP DATABASE $dbName CASCADE") assert(!catalog.databaseExists(dbNameWithoutBackTicks)) @@ -233,8 +236,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -275,7 +277,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = "file:" + appendTrailingSlash(path) + s"$dbNameWithoutBackTicks.db" + val location = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") sql(s"CREATE DATABASE $dbName") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 5d348044515af..761bbe3576c71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.internal +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} @@ -214,7 +216,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) assert(spark.sessionState.conf.warehousePath - === s"file:${System.getProperty("user.dir")}/spark-warehouse") + === new Path(s"${System.getProperty("user.dir")}/spark-warehouse").toString) } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") } From 6bf20cd9460fd27c3e1e434b1cf31a3778ec3443 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 11 Aug 2016 01:43:08 -0700 Subject: [PATCH 1128/1470] [SPARK-17015][SQL] group-by/order-by ordinal and arithmetic tests This patch adds three test files: 1. arithmetic.sql.out 2. order-by-ordinal.sql 3. group-by-ordinal.sql This includes https://github.com/apache/spark/pull/14594. This is a test case change. Author: petermaxlee Closes #14595 from petermaxlee/SPARK-17015. (cherry picked from commit a7b02db457d5fc663ce6a1ef01bf04689870e6b4) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/Analyzer.scala | 24 +- .../resources/sql-tests/inputs/arithmetic.sql | 26 +++ .../sql-tests/inputs/group-by-ordinal.sql | 50 ++++ .../sql-tests/inputs/order-by-ordinal.sql | 36 +++ .../sql-tests/results/arithmetic.sql.out | 178 ++++++++++++++ .../results/group-by-ordinal.sql.out | 168 +++++++++++++ .../results/order-by-ordinal.sql.out | 143 ++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 220 ------------------ 8 files changed, 613 insertions(+), 232 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/order-by-ordinal.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out 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 660f523698e7f..57c3d9aece80c 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 @@ -547,8 +547,7 @@ class Analyzer( case a: Aggregate if containsStar(a.aggregateExpressions) => if (conf.groupByOrdinal && a.groupingExpressions.exists(IntegerIndex.unapply(_).nonEmpty)) { failAnalysis( - "Group by position: star is not allowed to use in the select list " + - "when using ordinals in group by") + "Star (*) is not allowed in select list when GROUP BY ordinal position is used") } else { a.copy(aggregateExpressions = buildExpandedProjectList(a.aggregateExpressions, a.child)) } @@ -723,9 +722,9 @@ class Analyzer( if (index > 0 && index <= child.output.size) { SortOrder(child.output(index - 1), direction) } else { - throw new UnresolvedException(s, - s"Order/sort By position: $index does not exist " + - s"The Select List is indexed from 1 to ${child.output.size}") + s.failAnalysis( + s"ORDER BY position $index is not in select list " + + s"(valid range is [1, ${child.output.size}])") } case o => o } @@ -737,17 +736,18 @@ class Analyzer( if conf.groupByOrdinal && aggs.forall(_.resolved) && groups.exists(IntegerIndex.unapply(_).nonEmpty) => val newGroups = groups.map { - case IntegerIndex(index) if index > 0 && index <= aggs.size => + case ordinal @ IntegerIndex(index) if index > 0 && index <= aggs.size => aggs(index - 1) match { case e if ResolveAggregateFunctions.containsAggregate(e) => - throw new UnresolvedException(a, - s"Group by position: the '$index'th column in the select contains an " + - s"aggregate function: ${e.sql}. Aggregate functions are not allowed in GROUP BY") + ordinal.failAnalysis( + s"GROUP BY position $index is an aggregate function, and " + + "aggregate functions are not allowed in GROUP BY") case o => o } - case IntegerIndex(index) => - throw new UnresolvedException(a, - s"Group by position: '$index' exceeds the size of the select list '${aggs.size}'.") + case ordinal @ IntegerIndex(index) => + ordinal.failAnalysis( + s"GROUP BY position $index is not in select list " + + s"(valid range is [1, ${aggs.size}])") case o => o } Aggregate(newGroups, aggs, child) diff --git a/sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql b/sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql new file mode 100644 index 0000000000000..cbe40410cdc10 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql @@ -0,0 +1,26 @@ + +-- unary minus and plus +select -100; +select +230; +select -5.2; +select +6.8e0; +select -key, +key from testdata where key = 2; +select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1; +select -max(key), +max(key) from testdata; +select - (-10); +select + (-key) from testdata where key = 32; +select - (+max(key)) from testdata; +select - - 3; +select - + 20; +select + + 100; +select - - max(key) from testdata; +select + - key from testdata where key = 33; + +-- other arithmetics +select 1 + 2; +select 1 - 2; +select 2 * 5; +select 5 / 2; +select 5 div 2; +select 5 % 3; +select pmod(-7, 3); diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql new file mode 100644 index 0000000000000..36b469c61788c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql @@ -0,0 +1,50 @@ +-- group by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +-- basic case +select a, sum(b) from data group by 1; + +-- constant case +select 1, 2, sum(b) from data group by 1, 2; + +-- duplicate group by column +select a, 1, sum(b) from data group by a, 1; +select a, 1, sum(b) from data group by 1, 2; + +-- group by a non-aggregate expression's ordinal +select a, b + 2, count(2) from data group by a, 2; + +-- with alias +select a as aa, b + 2 as bb, count(2) from data group by 1, 2; + +-- foldable non-literal: this should be the same as no grouping. +select sum(b) from data group by 1 + 0; + +-- negative cases: ordinal out of range +select a, b from data group by -1; +select a, b from data group by 0; +select a, b from data group by 3; + +-- negative case: position is an aggregate expression +select a, b, sum(b) from data group by 3; +select a, b, sum(b) + 2 from data group by 3; + +-- negative case: nondeterministic expression +select a, rand(0), sum(b) from data group by a, 2; + +-- negative case: star +select * from data group by a, b, 1; + +-- turn of group by ordinal +set spark.sql.groupByOrdinal=false; + +-- can now group by negative literal +select sum(b) from data group by -1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/order-by-ordinal.sql b/sql/core/src/test/resources/sql-tests/inputs/order-by-ordinal.sql new file mode 100644 index 0000000000000..8d733e77fa8d3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/order-by-ordinal.sql @@ -0,0 +1,36 @@ +-- order by and sort by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +select * from data order by 1 desc; + +-- mix ordinal and column name +select * from data order by 1 desc, b desc; + +-- order by multiple ordinals +select * from data order by 1 desc, 2 desc; + +-- 1 + 0 is considered a constant (not an ordinal) and thus ignored +select * from data order by 1 + 0 desc, b desc; + +-- negative cases: ordinal position out of range +select * from data order by 0; +select * from data order by -1; +select * from data order by 3; + +-- sort by ordinal +select * from data sort by 1 desc; + +-- turn off order by ordinal +set spark.sql.orderByOrdinal=false; + +-- 0 is now a valid literal +select * from data order by 0; +select * from data sort by 0; diff --git a/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out new file mode 100644 index 0000000000000..50ea254b0b648 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out @@ -0,0 +1,178 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +select -100 +-- !query 0 schema +struct<(-100):int> +-- !query 0 output +-100 + + +-- !query 1 +select +230 +-- !query 1 schema +struct<230:int> +-- !query 1 output +230 + + +-- !query 2 +select -5.2 +-- !query 2 schema +struct<(-5.2):decimal(2,1)> +-- !query 2 output +-5.2 + + +-- !query 3 +select +6.8e0 +-- !query 3 schema +struct<6.8:double> +-- !query 3 output +6.8 + + +-- !query 4 +select -key, +key from testdata where key = 2 +-- !query 4 schema +struct<(-key):int,key:int> +-- !query 4 output +-2 2 + + +-- !query 5 +select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 +-- !query 5 schema +struct<(-(key + 1)):int,((-key) + 1):int,(key + 5):int> +-- !query 5 output +-2 0 6 + + +-- !query 6 +select -max(key), +max(key) from testdata +-- !query 6 schema +struct<(-max(key)):int,max(key):int> +-- !query 6 output +-100 100 + + +-- !query 7 +select - (-10) +-- !query 7 schema +struct<(-(-10)):int> +-- !query 7 output +10 + + +-- !query 8 +select + (-key) from testdata where key = 32 +-- !query 8 schema +struct<(-key):int> +-- !query 8 output +-32 + + +-- !query 9 +select - (+max(key)) from testdata +-- !query 9 schema +struct<(-max(key)):int> +-- !query 9 output +-100 + + +-- !query 10 +select - - 3 +-- !query 10 schema +struct<(-(-3)):int> +-- !query 10 output +3 + + +-- !query 11 +select - + 20 +-- !query 11 schema +struct<(-20):int> +-- !query 11 output +-20 + + +-- !query 12 +select + + 100 +-- !query 12 schema +struct<100:int> +-- !query 12 output +100 + + +-- !query 13 +select - - max(key) from testdata +-- !query 13 schema +struct<(-(-max(key))):int> +-- !query 13 output +100 + + +-- !query 14 +select + - key from testdata where key = 33 +-- !query 14 schema +struct<(-key):int> +-- !query 14 output +-33 + + +-- !query 15 +select 1 + 2 +-- !query 15 schema +struct<(1 + 2):int> +-- !query 15 output +3 + + +-- !query 16 +select 1 - 2 +-- !query 16 schema +struct<(1 - 2):int> +-- !query 16 output +-1 + + +-- !query 17 +select 2 * 5 +-- !query 17 schema +struct<(2 * 5):int> +-- !query 17 output +10 + + +-- !query 18 +select 5 / 2 +-- !query 18 schema +struct<(CAST(5 AS DOUBLE) / CAST(2 AS DOUBLE)):double> +-- !query 18 output +2.5 + + +-- !query 19 +select 5 div 2 +-- !query 19 schema +struct +-- !query 19 output +2 + + +-- !query 20 +select 5 % 3 +-- !query 20 schema +struct<(5 % 3):int> +-- !query 20 output +2 + + +-- !query 21 +select pmod(-7, 3) +-- !query 21 schema +struct +-- !query 21 output +2 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out new file mode 100644 index 0000000000000..2f10b7ebc6d32 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out @@ -0,0 +1,168 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select a, sum(b) from data group by 1 +-- !query 1 schema +struct +-- !query 1 output +1 3 +2 3 +3 3 + + +-- !query 2 +select 1, 2, sum(b) from data group by 1, 2 +-- !query 2 schema +struct<1:int,2:int,sum(b):bigint> +-- !query 2 output +1 2 9 + + +-- !query 3 +select a, 1, sum(b) from data group by a, 1 +-- !query 3 schema +struct +-- !query 3 output +1 1 3 +2 1 3 +3 1 3 + + +-- !query 4 +select a, 1, sum(b) from data group by 1, 2 +-- !query 4 schema +struct +-- !query 4 output +1 1 3 +2 1 3 +3 1 3 + + +-- !query 5 +select a, b + 2, count(2) from data group by a, 2 +-- !query 5 schema +struct +-- !query 5 output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query 6 +select a as aa, b + 2 as bb, count(2) from data group by 1, 2 +-- !query 6 schema +struct +-- !query 6 output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query 7 +select sum(b) from data group by 1 + 0 +-- !query 7 schema +struct +-- !query 7 output +9 + + +-- !query 8 +select a, b from data group by -1 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +GROUP BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 31 + + +-- !query 9 +select a, b from data group by 0 +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +GROUP BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 31 + + +-- !query 10 +select a, b from data group by 3 +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 31 + + +-- !query 11 +select a, b, sum(b) from data group by 3 +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is an aggregate function, and aggregate functions are not allowed in GROUP BY; line 1 pos 39 + + +-- !query 12 +select a, b, sum(b) + 2 from data group by 3 +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is an aggregate function, and aggregate functions are not allowed in GROUP BY; line 1 pos 43 + + +-- !query 13 +select a, rand(0), sum(b) from data group by a, 2 +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +nondeterministic expression rand(0) should not appear in grouping expression.; + + +-- !query 14 +select * from data group by a, b, 1 +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Star (*) is not allowed in select list when GROUP BY ordinal position is used; + + +-- !query 15 +set spark.sql.groupByOrdinal=false +-- !query 15 schema +struct +-- !query 15 output +spark.sql.groupByOrdinal + + +-- !query 16 +select sum(b) from data group by -1 +-- !query 16 schema +struct +-- !query 16 output +9 diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out new file mode 100644 index 0000000000000..03a4e72d0fa3e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out @@ -0,0 +1,143 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select * from data order by 1 desc +-- !query 1 schema +struct +-- !query 1 output +3 1 +3 2 +2 1 +2 2 +1 1 +1 2 + + +-- !query 2 +select * from data order by 1 desc, b desc +-- !query 2 schema +struct +-- !query 2 output +3 2 +3 1 +2 2 +2 1 +1 2 +1 1 + + +-- !query 3 +select * from data order by 1 desc, 2 desc +-- !query 3 schema +struct +-- !query 3 output +3 2 +3 1 +2 2 +2 1 +1 2 +1 1 + + +-- !query 4 +select * from data order by 1 + 0 desc, b desc +-- !query 4 schema +struct +-- !query 4 output +1 2 +2 2 +3 2 +1 1 +2 1 +3 1 + + +-- !query 5 +select * from data order by 0 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +ORDER BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 28 + + +-- !query 6 +select * from data order by -1 +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +ORDER BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 28 + + +-- !query 7 +select * from data order by 3 +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +ORDER BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 28 + + +-- !query 8 +select * from data sort by 1 desc +-- !query 8 schema +struct +-- !query 8 output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 + + +-- !query 9 +set spark.sql.orderByOrdinal=false +-- !query 9 schema +struct +-- !query 9 output +spark.sql.orderByOrdinal + + +-- !query 10 +select * from data order by 0 +-- !query 10 schema +struct +-- !query 10 output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 + + +-- !query 11 +select * from data sort by 0 +-- !query 11 schema +struct +-- !query 11 output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 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 3de5086b15c9c..d1d5875588718 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 @@ -487,103 +487,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Seq(Row(1, 3), Row(2, 3), Row(3, 3))) } - test("Group By Ordinal - basic") { - checkAnswer( - sql("SELECT a, sum(b) FROM testData2 GROUP BY 1"), - sql("SELECT a, sum(b) FROM testData2 GROUP BY a")) - - // duplicate group-by columns - checkAnswer( - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a, 1"), - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a")) - - checkAnswer( - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY 1, 2"), - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a")) - } - - test("Group By Ordinal - non aggregate expressions") { - checkAnswer( - sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, 2"), - sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, b + 2")) - - checkAnswer( - sql("SELECT a, b + 2 as c, count(2) FROM testData2 GROUP BY a, 2"), - sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, b + 2")) - } - - test("Group By Ordinal - non-foldable constant expression") { - checkAnswer( - sql("SELECT a, b, sum(b) FROM testData2 GROUP BY a, b, 1 + 0"), - sql("SELECT a, b, sum(b) FROM testData2 GROUP BY a, b")) - - checkAnswer( - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a, 1 + 2"), - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a")) - } - - test("Group By Ordinal - alias") { - checkAnswer( - sql("SELECT a, (b + 2) as c, count(2) FROM testData2 GROUP BY a, 2"), - sql("SELECT a, b + 2, count(2) FROM testData2 GROUP BY a, b + 2")) - - checkAnswer( - sql("SELECT a as b, b as a, sum(b) FROM testData2 GROUP BY 1, 2"), - sql("SELECT a, b, sum(b) FROM testData2 GROUP BY a, b")) - } - - test("Group By Ordinal - constants") { - checkAnswer( - sql("SELECT 1, 2, sum(b) FROM testData2 GROUP BY 1, 2"), - sql("SELECT 1, 2, sum(b) FROM testData2")) - } - - test("Group By Ordinal - negative cases") { - intercept[UnresolvedException[Aggregate]] { - sql("SELECT a, b FROM testData2 GROUP BY -1") - } - - intercept[UnresolvedException[Aggregate]] { - sql("SELECT a, b FROM testData2 GROUP BY 3") - } - - var e = intercept[UnresolvedException[Aggregate]]( - sql("SELECT SUM(a) FROM testData2 GROUP BY 1")) - assert(e.getMessage contains - "Invalid call to Group by position: the '1'th column in the select contains " + - "an aggregate function") - - e = intercept[UnresolvedException[Aggregate]]( - sql("SELECT SUM(a) + 1 FROM testData2 GROUP BY 1")) - assert(e.getMessage contains - "Invalid call to Group by position: the '1'th column in the select contains " + - "an aggregate function") - - var ae = intercept[AnalysisException]( - sql("SELECT a, rand(0), sum(b) FROM testData2 GROUP BY a, 2")) - assert(ae.getMessage contains - "nondeterministic expression rand(0) should not appear in grouping expression") - - ae = intercept[AnalysisException]( - sql("SELECT * FROM testData2 GROUP BY a, b, 1")) - assert(ae.getMessage contains - "Group by position: star is not allowed to use in the select list " + - "when using ordinals in group by") - } - - test("Group By Ordinal: spark.sql.groupByOrdinal=false") { - withSQLConf(SQLConf.GROUP_BY_ORDINAL.key -> "false") { - // If spark.sql.groupByOrdinal=false, ignore the position number. - intercept[AnalysisException] { - sql("SELECT a, sum(b) FROM testData2 GROUP BY 1") - } - // '*' is not allowed to use in the select list when users specify ordinals in group by - checkAnswer( - sql("SELECT * FROM testData2 GROUP BY a, b, 1"), - sql("SELECT * FROM testData2 GROUP BY a, b")) - } - } - test("aggregates with nulls") { checkAnswer( sql("SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a)," + @@ -1288,89 +1191,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } - test("Test to check we can apply sign to expression") { - - checkAnswer( - sql("SELECT -100"), Row(-100) - ) - - checkAnswer( - sql("SELECT +230"), Row(230) - ) - - checkAnswer( - sql("SELECT -5.2"), Row(BigDecimal(-5.2)) - ) - - checkAnswer( - sql("SELECT +6.8e0"), Row(6.8d) - ) - - checkAnswer( - sql("SELECT -key FROM testData WHERE key = 2"), Row(-2) - ) - - checkAnswer( - sql("SELECT +key FROM testData WHERE key = 3"), Row(3) - ) - - checkAnswer( - sql("SELECT -(key + 1) FROM testData WHERE key = 1"), Row(-2) - ) - - checkAnswer( - sql("SELECT - key + 1 FROM testData WHERE key = 10"), Row(-9) - ) - - checkAnswer( - sql("SELECT +(key + 5) FROM testData WHERE key = 5"), Row(10) - ) - - checkAnswer( - sql("SELECT -MAX(key) FROM testData"), Row(-100) - ) - - checkAnswer( - sql("SELECT +MAX(key) FROM testData"), Row(100) - ) - - checkAnswer( - sql("SELECT - (-10)"), Row(10) - ) - - checkAnswer( - sql("SELECT + (-key) FROM testData WHERE key = 32"), Row(-32) - ) - - checkAnswer( - sql("SELECT - (+Max(key)) FROM testData"), Row(-100) - ) - - checkAnswer( - sql("SELECT - - 3"), Row(3) - ) - - checkAnswer( - sql("SELECT - + 20"), Row(-20) - ) - - checkAnswer( - sql("SELEcT - + 45"), Row(-45) - ) - - checkAnswer( - sql("SELECT + + 100"), Row(100) - ) - - checkAnswer( - sql("SELECT - - Max(key) FROM testData"), Row(100) - ) - - checkAnswer( - sql("SELECT + - key FROM testData WHERE key = 33"), Row(-33) - ) - } - test("Multiple join") { checkAnswer( sql( @@ -2388,46 +2208,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } - test("order by ordinal number") { - checkAnswer( - sql("SELECT * FROM testData2 ORDER BY 1 DESC"), - sql("SELECT * FROM testData2 ORDER BY a DESC")) - // If the position is not an integer, ignore it. - checkAnswer( - sql("SELECT * FROM testData2 ORDER BY 1 + 0 DESC, b ASC"), - sql("SELECT * FROM testData2 ORDER BY b ASC")) - checkAnswer( - sql("SELECT * FROM testData2 ORDER BY 1 DESC, b ASC"), - sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC")) - checkAnswer( - sql("SELECT * FROM testData2 SORT BY 1 DESC, 2"), - sql("SELECT * FROM testData2 SORT BY a DESC, b ASC")) - checkAnswer( - sql("SELECT * FROM testData2 ORDER BY 1 ASC, b ASC"), - Seq(Row(1, 1), Row(1, 2), Row(2, 1), Row(2, 2), Row(3, 1), Row(3, 2))) - } - - test("order by ordinal number - negative cases") { - intercept[UnresolvedException[SortOrder]] { - sql("SELECT * FROM testData2 ORDER BY 0") - } - intercept[UnresolvedException[SortOrder]] { - sql("SELECT * FROM testData2 ORDER BY -1 DESC, b ASC") - } - intercept[UnresolvedException[SortOrder]] { - sql("SELECT * FROM testData2 ORDER BY 3 DESC, b ASC") - } - } - - test("order by ordinal number with conf spark.sql.orderByOrdinal=false") { - withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "false") { - // If spark.sql.orderByOrdinal=false, ignore the position number. - checkAnswer( - sql("SELECT * FROM testData2 ORDER BY 1 DESC, b ASC"), - sql("SELECT * FROM testData2 ORDER BY b ASC")) - } - } - test("join with using clause") { val df1 = Seq(("r1c1", "r1c2", "t1r1c3"), ("r2c1", "r2c2", "t1r2c3"), ("r3c1x", "r3c2", "t1r3c3")).toDF("c1", "c2", "c3") From bc683f037d4e84f2a42eb7b1aaa9e0e4fd5f833a Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 11 Aug 2016 13:55:10 -0700 Subject: [PATCH 1129/1470] [SPARK-17018][SQL] literals.sql for testing literal parsing ## What changes were proposed in this pull request? This patch adds literals.sql for testing literal parsing end-to-end in SQL. ## How was this patch tested? The patch itself is only about adding test cases. Author: petermaxlee Closes #14598 from petermaxlee/SPARK-17018-2. (cherry picked from commit cf9367826c38e5f34ae69b409f5d09c55ed1d319) Signed-off-by: Reynold Xin --- .../resources/sql-tests/inputs/literals.sql | 92 +++++ .../sql-tests/inputs/number-format.sql | 16 - .../sql-tests/results/literals.sql.out | 374 ++++++++++++++++++ .../sql-tests/results/number-format.sql.out | 42 -- .../apache/spark/sql/SQLQueryTestSuite.scala | 14 +- 5 files changed, 476 insertions(+), 62 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/literals.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/number-format.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/literals.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/number-format.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql new file mode 100644 index 0000000000000..62f0d3d0599c6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -0,0 +1,92 @@ +-- Literal parsing + +-- null +select null, Null, nUll; + +-- boolean +select true, tRue, false, fALse; + +-- byte (tinyint) +select 1Y; +select 127Y, -128Y; + +-- out of range byte +select 128Y; + +-- short (smallint) +select 1S; +select 32767S, -32768S; + +-- out of range short +select 32768S; + +-- long (bigint) +select 1L, 2147483648L; +select 9223372036854775807L, -9223372036854775808L; + +-- out of range long +select 9223372036854775808L; + +-- integral parsing + +-- parse int +select 1, -1; + +-- parse int max and min value as int +select 2147483647, -2147483648; + +-- parse long max and min value as long +select 9223372036854775807, -9223372036854775808; + +-- parse as decimals (Long.MaxValue + 1, and Long.MinValue - 1) +select 9223372036854775808, -9223372036854775809; + +-- out of range decimal numbers +select 1234567890123456789012345678901234567890; +select 1234567890123456789012345678901234567890.0; + +-- double +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1; +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5; +-- negative double +select .e3; +-- inf and -inf +select 1E309, -1E309; + +-- decimal parsing +select 0.3, -0.8, .5, -.18, 0.1111, .1111; + +-- super large scientific notation numbers should still be valid doubles +select 123456789012345678901234567890123456789e10, 123456789012345678901234567890123456789.1e10; + +-- string +select "Hello Peter!", 'hello lee!'; +-- multi string +select 'hello' 'world', 'hello' " " 'lee'; +-- single quote within double quotes +select "hello 'peter'"; +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%'; +select '\'', '"', '\n', '\r', '\t', 'Z'; +-- "Hello!" in octals +select '\110\145\154\154\157\041'; +-- "World :)" in unicode +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029'; + +-- date +select dAte '2016-03-12'; +-- invalid date +select date 'mar 11 2016'; + +-- timestamp +select tImEstAmp '2016-03-11 20:54:00.000'; +-- invalid timestamp +select timestamp '2016-33-11 20:54:00.000'; + +-- interval +select interval 13.123456789 seconds, interval -13.123456789 second; +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond; +-- ns is not supported +select interval 10 nanoseconds; + +-- unsupported data type +select GEO '(10,-6)'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/number-format.sql b/sql/core/src/test/resources/sql-tests/inputs/number-format.sql deleted file mode 100644 index a32d0688f813a..0000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/number-format.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Verifies how we parse numbers - --- parse as ints -select 1, -1; - --- parse as longs (Int.MaxValue + 1, and Int.MinValue - 1) -select 2147483648, -2147483649; - --- parse long min and max value -select 9223372036854775807, -9223372036854775808; - --- parse as decimals (Long.MaxValue + 1, and Long.MinValue - 1) -select 9223372036854775808, -9223372036854775809; - --- various floating point (decimal) formats -select 0.3, -0.8, .5, -.18, 0.1111; diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out new file mode 100644 index 0000000000000..6d5fabdf62151 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -0,0 +1,374 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 38 + + +-- !query 0 +select null, Null, nUll +-- !query 0 schema +struct +-- !query 0 output +NULL NULL NULL + + +-- !query 1 +select true, tRue, false, fALse +-- !query 1 schema +struct +-- !query 1 output +true true false false + + +-- !query 2 +select 1Y +-- !query 2 schema +struct<1:tinyint> +-- !query 2 output +1 + + +-- !query 3 +select 127Y, -128Y +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.catalyst.parser.ParseException + +Value out of range. Value:"128" Radix:10(line 1, pos 14) + +== SQL == +select 127Y, -128Y +--------------^^^ + + +-- !query 4 +select 128Y +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.catalyst.parser.ParseException + +Value out of range. Value:"128" Radix:10(line 1, pos 7) + +== SQL == +select 128Y +-------^^^ + + +-- !query 5 +select 1S +-- !query 5 schema +struct<1:smallint> +-- !query 5 output +1 + + +-- !query 6 +select 32767S, -32768S +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.catalyst.parser.ParseException + +Value out of range. Value:"32768" Radix:10(line 1, pos 16) + +== SQL == +select 32767S, -32768S +----------------^^^ + + +-- !query 7 +select 32768S +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.catalyst.parser.ParseException + +Value out of range. Value:"32768" Radix:10(line 1, pos 7) + +== SQL == +select 32768S +-------^^^ + + +-- !query 8 +select 1L, 2147483648L +-- !query 8 schema +struct<1:bigint,2147483648:bigint> +-- !query 8 output +1 2147483648 + + +-- !query 9 +select 9223372036854775807L, -9223372036854775808L +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.catalyst.parser.ParseException + +For input string: "9223372036854775808"(line 1, pos 30) + +== SQL == +select 9223372036854775807L, -9223372036854775808L +------------------------------^^^ + + +-- !query 10 +select 9223372036854775808L +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.catalyst.parser.ParseException + +For input string: "9223372036854775808"(line 1, pos 7) + +== SQL == +select 9223372036854775808L +-------^^^ + + +-- !query 11 +select 1, -1 +-- !query 11 schema +struct<1:int,(-1):int> +-- !query 11 output +1 -1 + + +-- !query 12 +select 2147483647, -2147483648 +-- !query 12 schema +struct<2147483647:int,(-2147483648):bigint> +-- !query 12 output +2147483647 -2147483648 + + +-- !query 13 +select 9223372036854775807, -9223372036854775808 +-- !query 13 schema +struct<9223372036854775807:bigint,(-9223372036854775808):decimal(19,0)> +-- !query 13 output +9223372036854775807 -9223372036854775808 + + +-- !query 14 +select 9223372036854775808, -9223372036854775809 +-- !query 14 schema +struct<9223372036854775808:decimal(19,0),(-9223372036854775809):decimal(19,0)> +-- !query 14 output +9223372036854775808 -9223372036854775809 + + +-- !query 15 +select 1234567890123456789012345678901234567890 +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.parser.ParseException + +DecimalType can only support precision up to 38 +== SQL == +select 1234567890123456789012345678901234567890 + + +-- !query 16 +select 1234567890123456789012345678901234567890.0 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.parser.ParseException + +DecimalType can only support precision up to 38 +== SQL == +select 1234567890123456789012345678901234567890.0 + + +-- !query 17 +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 +-- !query 17 schema +struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double,10000.0:double,90.0:double,90.0:double,90.0:double,90.0:double> +-- !query 17 output +1.0 1.2 1.0E10 150000.0 0.1 0.1 10000.0 90.0 90.0 90.0 90.0 + + +-- !query 18 +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 +-- !query 18 schema +struct<(-1.0):double,(-1.2):double,(-1.0E10):double,(-150000.0):double,(-0.1):double,(-0.1):double,(-10000.0):double> +-- !query 18 output +-1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 + + +-- !query 19 +select .e3 +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'select .'(line 1, pos 7) + +== SQL == +select .e3 +-------^^^ + + +-- !query 20 +select 1E309, -1E309 +-- !query 20 schema +struct +-- !query 20 output +Infinity -Infinity + + +-- !query 21 +select 0.3, -0.8, .5, -.18, 0.1111, .1111 +-- !query 21 schema +struct<0.3:decimal(1,1),(-0.8):decimal(1,1),0.5:decimal(1,1),(-0.18):decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> +-- !query 21 output +0.3 -0.8 0.5 -0.18 0.1111 0.1111 + + +-- !query 22 +select 123456789012345678901234567890123456789e10, 123456789012345678901234567890123456789.1e10 +-- !query 22 schema +struct<1.2345678901234568E48:double,1.2345678901234568E48:double> +-- !query 22 output +1.2345678901234568E48 1.2345678901234568E48 + + +-- !query 23 +select "Hello Peter!", 'hello lee!' +-- !query 23 schema +struct +-- !query 23 output +Hello Peter! hello lee! + + +-- !query 24 +select 'hello' 'world', 'hello' " " 'lee' +-- !query 24 schema +struct +-- !query 24 output +helloworld hello lee + + +-- !query 25 +select "hello 'peter'" +-- !query 25 schema +struct +-- !query 25 output +hello 'peter' + + +-- !query 26 +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' +-- !query 26 schema +struct +-- !query 26 output +pattern% no-pattern\% pattern\% pattern\\% + + +-- !query 27 +select '\'', '"', '\n', '\r', '\t', 'Z' +-- !query 27 schema +struct<':string,":string, +:string, :string, :string,Z:string> +-- !query 27 output +' " + Z + + +-- !query 28 +select '\110\145\154\154\157\041' +-- !query 28 schema +struct +-- !query 28 output +Hello! + + +-- !query 29 +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' +-- !query 29 schema +struct +-- !query 29 output +World :) + + +-- !query 30 +select dAte '2016-03-12' +-- !query 30 schema +struct +-- !query 30 output +2016-03-12 + + +-- !query 31 +select date 'mar 11 2016' +-- !query 31 schema +struct<> +-- !query 31 output +java.lang.IllegalArgumentException +null + + +-- !query 32 +select tImEstAmp '2016-03-11 20:54:00.000' +-- !query 32 schema +struct +-- !query 32 output +2016-03-11 20:54:00 + + +-- !query 33 +select timestamp '2016-33-11 20:54:00.000' +-- !query 33 schema +struct<> +-- !query 33 output +java.lang.IllegalArgumentException +Timestamp format must be yyyy-mm-dd hh:mm:ss[.fffffffff] + + +-- !query 34 +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query 34 schema +struct<> +-- !query 34 output +scala.MatchError +(interval 13 seconds 123 milliseconds 456 microseconds,CalendarIntervalType) (of class scala.Tuple2) + + +-- !query 35 +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond +-- !query 35 schema +struct<> +-- !query 35 output +scala.MatchError +(interval 1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseconds,CalendarIntervalType) (of class scala.Tuple2) + + +-- !query 36 +select interval 10 nanoseconds +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.catalyst.parser.ParseException + +No interval can be constructed(line 1, pos 16) + +== SQL == +select interval 10 nanoseconds +----------------^^^ + + +-- !query 37 +select GEO '(10,-6)' +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.catalyst.parser.ParseException + +Literals of type 'GEO' are currently not supported.(line 1, pos 7) + +== SQL == +select GEO '(10,-6)' +-------^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/number-format.sql.out b/sql/core/src/test/resources/sql-tests/results/number-format.sql.out deleted file mode 100644 index 82a1d39c0a0be..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/number-format.sql.out +++ /dev/null @@ -1,42 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 5 - - --- !query 0 -select 1, -1 --- !query 0 schema -struct<1:int,(-1):int> --- !query 0 output -1 -1 - - --- !query 1 -select 2147483648, -2147483649 --- !query 1 schema -struct<2147483648:bigint,(-2147483649):bigint> --- !query 1 output -2147483648 -2147483649 - - --- !query 2 -select 9223372036854775807, -9223372036854775808 --- !query 2 schema -struct<9223372036854775807:bigint,(-9223372036854775808):decimal(19,0)> --- !query 2 output -9223372036854775807 -9223372036854775808 - - --- !query 3 -select 9223372036854775808, -9223372036854775809 --- !query 3 schema -struct<9223372036854775808:decimal(19,0),(-9223372036854775809):decimal(19,0)> --- !query 3 output -9223372036854775808 -9223372036854775809 - - --- !query 4 -select 0.3, -0.8, .5, -.18, 0.1111 --- !query 4 schema -struct<0.3:decimal(1,1),(-0.8):decimal(1,1),0.5:decimal(1,1),(-0.18):decimal(2,2),0.1111:decimal(4,4)> --- !query 4 output -0.3 -0.8 0.5 -0.18 0.1111 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 1022c38e262b9..069a9b665eb36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -143,7 +143,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { QueryOutput( sql = sql, schema = schema.catalogString, - output = output.mkString("\n")) + output = output.mkString("\n").trim) } if (regenerateGoldenFiles) { @@ -180,9 +180,15 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) => - assertResult(expected.sql, s"SQL query should match for query #$i") { output.sql } - assertResult(expected.schema, s"Schema should match for query #$i") { output.schema } - assertResult(expected.output, s"Result should match for query #$i") { output.output } + assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") { + output.sql + } + assertResult(expected.schema, s"Schema did not match for query #$i\n${expected.sql}") { + output.schema + } + assertResult(expected.output, s"Result dit not match for query #$i\n${expected.sql}") { + output.output + } } } From 0fb01496c09defa1436dbb7f5e1cbc5461617a31 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 11 Aug 2016 15:09:23 -0700 Subject: [PATCH 1130/1470] [SPARK-17022][YARN] Handle potential deadlock in driver handling messages ## What changes were proposed in this pull request? We directly send RequestExecutors to AM instead of transfer it to yarnShedulerBackend first, to avoid potential deadlock. ## How was this patch tested? manual tests Author: WangTaoTheTonic Closes #14605 from WangTaoTheTonic/lock. (cherry picked from commit ea0bf91b4a2ca3ef472906e50e31fd6268b6f53e) Signed-off-by: Marcelo Vanzin --- .../cluster/YarnSchedulerBackend.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 6b3c831e60472..ea63ff5dc1580 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -125,8 +125,20 @@ private[spark] abstract class YarnSchedulerBackend( * This includes executors already pending or running. */ override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - yarnSchedulerEndpointRef.askWithRetry[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) + val r = RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) + yarnSchedulerEndpoint.amEndpoint match { + case Some(am) => + try { + am.askWithRetry[Boolean](r) + } catch { + case NonFatal(e) => + logError(s"Sending $r to AM was unsuccessful", e) + return false + } + case None => + logWarning("Attempted to request executors before the AM has registered!") + return false + } } /** @@ -209,7 +221,7 @@ private[spark] abstract class YarnSchedulerBackend( */ private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging { - private var amEndpoint: Option[RpcEndpointRef] = None + var amEndpoint: Option[RpcEndpointRef] = None private val askAmThreadPool = ThreadUtils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-thread-pool") From b4047fc21cefcf6a43c1ee88af330a042f02bebc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 12 Aug 2016 14:40:12 +0800 Subject: [PATCH 1131/1470] [SPARK-16975][SQL] Column-partition path starting '_' should be handled correctly Currently, Spark ignores path names starting with underscore `_` and `.`. This causes read-failures for the column-partitioned file data sources whose partition column names starts from '_', e.g. `_col`. **Before** ```scala scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet") scala> spark.read.parquet("/tmp/parquet") org.apache.spark.sql.AnalysisException: Unable to infer schema for ParquetFormat at /tmp/parquet20. It must be specified manually; ``` **After** ```scala scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet") scala> spark.read.parquet("/tmp/parquet") res2: org.apache.spark.sql.DataFrame = [id: bigint, _locality_code: int] ``` Pass the Jenkins with a new test case. Author: Dongjoon Hyun Closes #14585 from dongjoon-hyun/SPARK-16975-PARQUET. (cherry picked from commit abff92bfdc7d4c9d2308794f0350561fe0ceb4dd) Signed-off-by: Cheng Lian --- .../datasources/PartitioningAwareFileCatalog.scala | 2 +- .../sql/execution/datasources/fileSourceInterfaces.scala | 2 +- .../sql/execution/datasources/json/JsonFileFormat.scala | 2 +- .../datasources/parquet/ParquetFileFormat.scala | 3 ++- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +++++++++ 5 files changed, 14 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index 811e96c99a96d..cef9d4d9c7f1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -204,6 +204,6 @@ abstract class PartitioningAwareFileCatalog( private def isDataPath(path: Path): Boolean = { val name = path.getName - !(name.startsWith("_") || name.startsWith(".")) + !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 0b5a19fe9384b..438fccb25c778 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -363,7 +363,7 @@ private[sql] object HadoopFsRelation extends Logging { // We filter everything that starts with _ and ., except _common_metadata and _metadata // because Parquet needs to find those metadata files from leaf files returned by this method. // We should refactor this logic to not mix metadata files with data files. - (pathName.startsWith("_") || pathName.startsWith(".")) && + ((pathName.startsWith("_") && !pathName.contains("=")) || pathName.startsWith(".")) && !pathName.startsWith("_common_metadata") && !pathName.startsWith("_metadata") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 86aef1f7d4411..c58de3ae6f9e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -55,7 +55,7 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { .getOrElse(sparkSession.sessionState.conf.columnNameOfCorruptRecord) val jsonFiles = files.filterNot { status => val name = status.getPath.getName - name.startsWith("_") || name.startsWith(".") + (name.startsWith("_") && !name.contains("=")) || name.startsWith(".") }.toArray val jsonSchema = InferSchema.infer( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 7e819c744c638..260071ea6fee7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -234,7 +234,8 @@ private[sql] class ParquetFileFormat // Lists `FileStatus`es of all leaf nodes (files) under all base directories. val leaves = allFiles.filter { f => isSummaryFile(f.getPath) || - !(f.getPath.getName.startsWith("_") || f.getPath.getName.startsWith(".")) + !((f.getPath.getName.startsWith("_") && !f.getPath.getName.contains("=")) || + f.getPath.getName.startsWith(".")) }.toArray.sortBy(_.getPath.toString) FileTypes( 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 d1d5875588718..de1a811d642bb 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 java.io.File import java.math.MathContext import java.sql.{Date, Timestamp} @@ -2591,6 +2592,14 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(s"$expected") :: Nil) } + test("SPARK-16975: Column-partition path starting '_' should be handled correctly") { + withTempDir { dir => + val parquetDir = new File(dir, "parquet").getCanonicalPath + spark.range(10).withColumn("_col", $"id").write.partitionBy("_col").save(parquetDir) + spark.read.parquet(parquetDir) + } + } + test("SPARK-16644: Aggregate should not put aggregate expressions to constraints") { withTable("tbl") { sql("CREATE TABLE tbl(a INT, b INT) USING parquet") From bde94cd71086fd348f3ba96de628d6df3f87dba5 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 11 Aug 2016 23:56:55 -0700 Subject: [PATCH 1132/1470] [SPARK-17013][SQL] Parse negative numeric literals ## What changes were proposed in this pull request? This patch updates the SQL parser to parse negative numeric literals as numeric literals, instead of unary minus of positive literals. This allows the parser to parse the minimal value for each data type, e.g. "-32768S". ## How was this patch tested? Updated test cases. Author: petermaxlee Closes #14608 from petermaxlee/SPARK-17013. (cherry picked from commit 00e103a6edd1a1f001a94d41dd1f7acc40a1e30f) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 14 +++--- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../sql-tests/results/arithmetic.sql.out | 26 +++++------ .../sql-tests/results/literals.sql.out | 44 ++++++------------- .../catalyst/ExpressionSQLBuilderSuite.scala | 4 +- 5 files changed, 37 insertions(+), 55 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 279a1cee0ad21..aca728234ad99 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -618,13 +618,13 @@ quotedIdentifier ; number - : DECIMAL_VALUE #decimalLiteral - | SCIENTIFIC_DECIMAL_VALUE #scientificDecimalLiteral - | INTEGER_VALUE #integerLiteral - | BIGINT_LITERAL #bigIntLiteral - | SMALLINT_LITERAL #smallIntLiteral - | TINYINT_LITERAL #tinyIntLiteral - | DOUBLE_LITERAL #doubleLiteral + : MINUS? DECIMAL_VALUE #decimalLiteral + | MINUS? SCIENTIFIC_DECIMAL_VALUE #scientificDecimalLiteral + | MINUS? INTEGER_VALUE #integerLiteral + | MINUS? BIGINT_LITERAL #bigIntLiteral + | MINUS? SMALLINT_LITERAL #smallIntLiteral + | MINUS? TINYINT_LITERAL #tinyIntLiteral + | MINUS? DOUBLE_LITERAL #doubleLiteral ; nonReserved 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 7ff8795d4f05e..fa459aa2e5d72 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 @@ -57,7 +57,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression } } - override def sql: String = s"(-${child.sql})" + override def sql: String = s"(- ${child.sql})" } @ExpressionDescription( @@ -75,7 +75,7 @@ case class UnaryPositive(child: Expression) protected override def nullSafeEval(input: Any): Any = input - override def sql: String = s"(+${child.sql})" + override def sql: String = s"(+ ${child.sql})" } /** diff --git a/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out index 50ea254b0b648..f2b40a00d062d 100644 --- a/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out @@ -5,7 +5,7 @@ -- !query 0 select -100 -- !query 0 schema -struct<(-100):int> +struct<-100:int> -- !query 0 output -100 @@ -21,7 +21,7 @@ struct<230:int> -- !query 2 select -5.2 -- !query 2 schema -struct<(-5.2):decimal(2,1)> +struct<-5.2:decimal(2,1)> -- !query 2 output -5.2 @@ -37,7 +37,7 @@ struct<6.8:double> -- !query 4 select -key, +key from testdata where key = 2 -- !query 4 schema -struct<(-key):int,key:int> +struct<(- key):int,key:int> -- !query 4 output -2 2 @@ -45,7 +45,7 @@ struct<(-key):int,key:int> -- !query 5 select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 -- !query 5 schema -struct<(-(key + 1)):int,((-key) + 1):int,(key + 5):int> +struct<(- (key + 1)):int,((- key) + 1):int,(key + 5):int> -- !query 5 output -2 0 6 @@ -53,7 +53,7 @@ struct<(-(key + 1)):int,((-key) + 1):int,(key + 5):int> -- !query 6 select -max(key), +max(key) from testdata -- !query 6 schema -struct<(-max(key)):int,max(key):int> +struct<(- max(key)):int,max(key):int> -- !query 6 output -100 100 @@ -61,7 +61,7 @@ struct<(-max(key)):int,max(key):int> -- !query 7 select - (-10) -- !query 7 schema -struct<(-(-10)):int> +struct<(- -10):int> -- !query 7 output 10 @@ -69,7 +69,7 @@ struct<(-(-10)):int> -- !query 8 select + (-key) from testdata where key = 32 -- !query 8 schema -struct<(-key):int> +struct<(- key):int> -- !query 8 output -32 @@ -77,7 +77,7 @@ struct<(-key):int> -- !query 9 select - (+max(key)) from testdata -- !query 9 schema -struct<(-max(key)):int> +struct<(- max(key)):int> -- !query 9 output -100 @@ -85,7 +85,7 @@ struct<(-max(key)):int> -- !query 10 select - - 3 -- !query 10 schema -struct<(-(-3)):int> +struct<(- -3):int> -- !query 10 output 3 @@ -93,7 +93,7 @@ struct<(-(-3)):int> -- !query 11 select - + 20 -- !query 11 schema -struct<(-20):int> +struct<(- 20):int> -- !query 11 output -20 @@ -109,7 +109,7 @@ struct<100:int> -- !query 13 select - - max(key) from testdata -- !query 13 schema -struct<(-(-max(key))):int> +struct<(- (- max(key))):int> -- !query 13 output 100 @@ -117,7 +117,7 @@ struct<(-(-max(key))):int> -- !query 14 select + - key from testdata where key = 33 -- !query 14 schema -struct<(-key):int> +struct<(- key):int> -- !query 14 output -33 @@ -173,6 +173,6 @@ struct<(5 % 3):int> -- !query 21 select pmod(-7, 3) -- !query 21 schema -struct +struct -- !query 21 output 2 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 6d5fabdf62151..b964a6fc0921f 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -29,15 +29,9 @@ struct<1:tinyint> -- !query 3 select 127Y, -128Y -- !query 3 schema -struct<> +struct<127:tinyint,-128:tinyint> -- !query 3 output -org.apache.spark.sql.catalyst.parser.ParseException - -Value out of range. Value:"128" Radix:10(line 1, pos 14) - -== SQL == -select 127Y, -128Y ---------------^^^ +127 -128 -- !query 4 @@ -65,15 +59,9 @@ struct<1:smallint> -- !query 6 select 32767S, -32768S -- !query 6 schema -struct<> +struct<32767:smallint,-32768:smallint> -- !query 6 output -org.apache.spark.sql.catalyst.parser.ParseException - -Value out of range. Value:"32768" Radix:10(line 1, pos 16) - -== SQL == -select 32767S, -32768S -----------------^^^ +32767 -32768 -- !query 7 @@ -101,15 +89,9 @@ struct<1:bigint,2147483648:bigint> -- !query 9 select 9223372036854775807L, -9223372036854775808L -- !query 9 schema -struct<> +struct<9223372036854775807:bigint,-9223372036854775808:bigint> -- !query 9 output -org.apache.spark.sql.catalyst.parser.ParseException - -For input string: "9223372036854775808"(line 1, pos 30) - -== SQL == -select 9223372036854775807L, -9223372036854775808L -------------------------------^^^ +9223372036854775807 -9223372036854775808 -- !query 10 @@ -129,7 +111,7 @@ select 9223372036854775808L -- !query 11 select 1, -1 -- !query 11 schema -struct<1:int,(-1):int> +struct<1:int,-1:int> -- !query 11 output 1 -1 @@ -137,7 +119,7 @@ struct<1:int,(-1):int> -- !query 12 select 2147483647, -2147483648 -- !query 12 schema -struct<2147483647:int,(-2147483648):bigint> +struct<2147483647:int,-2147483648:int> -- !query 12 output 2147483647 -2147483648 @@ -145,7 +127,7 @@ struct<2147483647:int,(-2147483648):bigint> -- !query 13 select 9223372036854775807, -9223372036854775808 -- !query 13 schema -struct<9223372036854775807:bigint,(-9223372036854775808):decimal(19,0)> +struct<9223372036854775807:bigint,-9223372036854775808:bigint> -- !query 13 output 9223372036854775807 -9223372036854775808 @@ -153,7 +135,7 @@ struct<9223372036854775807:bigint,(-9223372036854775808):decimal(19,0)> -- !query 14 select 9223372036854775808, -9223372036854775809 -- !query 14 schema -struct<9223372036854775808:decimal(19,0),(-9223372036854775809):decimal(19,0)> +struct<9223372036854775808:decimal(19,0),-9223372036854775809:decimal(19,0)> -- !query 14 output 9223372036854775808 -9223372036854775809 @@ -193,7 +175,7 @@ struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double -- !query 18 select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 -- !query 18 schema -struct<(-1.0):double,(-1.2):double,(-1.0E10):double,(-150000.0):double,(-0.1):double,(-0.1):double,(-10000.0):double> +struct<-1.0:double,-1.2:double,-1.0E10:double,-150000.0:double,-0.1:double,-0.1:double,-10000.0:double> -- !query 18 output -1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 @@ -215,7 +197,7 @@ select .e3 -- !query 20 select 1E309, -1E309 -- !query 20 schema -struct +struct -- !query 20 output Infinity -Infinity @@ -223,7 +205,7 @@ Infinity -Infinity -- !query 21 select 0.3, -0.8, .5, -.18, 0.1111, .1111 -- !query 21 schema -struct<0.3:decimal(1,1),(-0.8):decimal(1,1),0.5:decimal(1,1),(-0.18):decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> +struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> -- !query 21 output 0.3 -0.8 0.5 -0.18 0.1111 0.1111 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index fef726c5d801d..7249df813b17f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -75,8 +75,8 @@ class ExpressionSQLBuilderSuite extends SQLBuilderTest { checkSQL('a.int / 'b.int, "(`a` / `b`)") checkSQL('a.int % 'b.int, "(`a` % `b`)") - checkSQL(-'a.int, "(-`a`)") - checkSQL(-('a.int + 'b.int), "(-(`a` + `b`))") + checkSQL(-'a.int, "(- `a`)") + checkSQL(-('a.int + 'b.int), "(- (`a` + `b`))") } test("window specification") { From 38378f59f2c91a6f07366aa2013522c334066c69 Mon Sep 17 00:00:00 2001 From: Jagadeesan Date: Sat, 13 Aug 2016 11:25:03 +0100 Subject: [PATCH 1133/1470] =?UTF-8?q?[SPARK-12370][DOCUMENTATION]=20Docume?= =?UTF-8?q?ntation=20should=20link=20to=20examples=20=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? When documentation is built is should reference examples from the same build. There are times when the docs have links that point to files in the GitHub head which may not be valid on the current release. Changed that in URLs to make them point to the right tag in git using ```SPARK_VERSION_SHORT``` …from its own release version] [Streaming programming guide] Author: Jagadeesan Closes #14596 from jagadeesanas2/SPARK-12370. (cherry picked from commit e46cb78b3b9fd04a50b5ae50f360db612d656a48) Signed-off-by: Sean Owen --- docs/ml-advanced.md | 4 ++-- docs/streaming-custom-receivers.md | 4 ++-- docs/streaming-flume-integration.md | 2 +- docs/streaming-kafka-0-8-integration.md | 12 +++++----- docs/streaming-programming-guide.md | 22 +++++++++---------- .../structured-streaming-programming-guide.md | 12 +++++----- 6 files changed, 28 insertions(+), 28 deletions(-) diff --git a/docs/ml-advanced.md b/docs/ml-advanced.md index f5804fdeee5aa..12a03d3c91984 100644 --- a/docs/ml-advanced.md +++ b/docs/ml-advanced.md @@ -49,7 +49,7 @@ MLlib L-BFGS solver calls the corresponding implementation in [breeze](https://g ## Normal equation solver for weighted least squares -MLlib implements normal equation solver for [weighted least squares](https://en.wikipedia.org/wiki/Least_squares#Weighted_least_squares) by [WeightedLeastSquares](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala). +MLlib implements normal equation solver for [weighted least squares](https://en.wikipedia.org/wiki/Least_squares#Weighted_least_squares) by [WeightedLeastSquares]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala). Given $n$ weighted observations $(w_i, a_i, b_i)$: @@ -73,7 +73,7 @@ In order to make the normal equation approach efficient, WeightedLeastSquares re ## Iteratively reweighted least squares (IRLS) -MLlib implements [iteratively reweighted least squares (IRLS)](https://en.wikipedia.org/wiki/Iteratively_reweighted_least_squares) by [IterativelyReweightedLeastSquares](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala). +MLlib implements [iteratively reweighted least squares (IRLS)](https://en.wikipedia.org/wiki/Iteratively_reweighted_least_squares) by [IterativelyReweightedLeastSquares]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala). It can be used to find the maximum likelihood estimates of a generalized linear model (GLM), find M-estimator in robust regression and other optimization problems. Refer to [Iteratively Reweighted Least Squares for Maximum Likelihood Estimation, and some Robust and Resistant Alternatives](http://www.jstor.org/stable/2345503) for more information. diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 479140f519103..f52bf348fcc99 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -181,7 +181,7 @@ val words = lines.flatMap(_.split(" ")) ... {% endhighlight %} -The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala). +The full source code is in the example [CustomReceiver.scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala).
    @@ -193,7 +193,7 @@ JavaDStream words = lines.flatMap(new FlatMapFunction() ... {% endhighlight %} -The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java). +The full source code is in the example [JavaCustomReceiver.java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java).
    diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index 8eeeee75dbf40..767e1f9402e01 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -63,7 +63,7 @@ configuring Flume agents. By default, the Python API will decode Flume event body as UTF8 encoded strings. You can specify your custom decoding function to decode the body byte arrays in Flume events to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/flume_wordcount.py). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/flume_wordcount.py).
    diff --git a/docs/streaming-kafka-0-8-integration.md b/docs/streaming-kafka-0-8-integration.md index da4a845fe2d41..f8f7b95cf7458 100644 --- a/docs/streaming-kafka-0-8-integration.md +++ b/docs/streaming-kafka-0-8-integration.md @@ -29,7 +29,7 @@ Next, we discuss how to use this approach in your streaming application. [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
    import org.apache.spark.streaming.kafka.*; @@ -39,7 +39,7 @@ Next, we discuss how to use this approach in your streaming application. [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java).
    @@ -49,7 +49,7 @@ Next, we discuss how to use this approach in your streaming application. [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/kafka_wordcount.py).
    @@ -106,7 +106,7 @@ Next, we discuss how to use this approach in your streaming application. You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala).
    import org.apache.spark.streaming.kafka.*; @@ -118,7 +118,7 @@ Next, we discuss how to use this approach in your streaming application. You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java).
    @@ -127,7 +127,7 @@ Next, we discuss how to use this approach in your streaming application. You can also pass a `messageHandler` to `createDirectStream` to access `KafkaMessageAndMetadata` that contains metadata about the current message and transform it to any desired type. By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/direct_kafka_wordcount.py). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/direct_kafka_wordcount.py).
    diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 3d40b2c3136eb..14e17443e362c 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -126,7 +126,7 @@ ssc.awaitTermination() // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala). +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala).
    @@ -216,7 +216,7 @@ jssc.awaitTermination(); // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java). +[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
    @@ -277,7 +277,7 @@ ssc.awaitTermination() # Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/network_wordcount.py). +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/network_wordcount.py).
    @@ -854,7 +854,7 @@ JavaPairDStream runningCounts = pairs.updateStateByKey(updateFu The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete Java code, take a look at the example -[JavaStatefulNetworkWordCount.java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming +[JavaStatefulNetworkWordCount.java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming /JavaStatefulNetworkWordCount.java).
    @@ -877,7 +877,7 @@ runningCounts = pairs.updateStateByKey(updateFunction) The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete Python code, take a look at the example -[stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/stateful_network_wordcount.py). +[stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/stateful_network_wordcount.py).
    @@ -1428,7 +1428,7 @@ wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala).
    {% highlight java %} @@ -1491,7 +1491,7 @@ wordCounts.foreachRDD(new Function2, Time, Void>() {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java).
    {% highlight python %} @@ -1526,7 +1526,7 @@ wordCounts.foreachRDD(echo) {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/recoverable_network_wordcount.py). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/recoverable_network_wordcount.py).
    @@ -1564,7 +1564,7 @@ words.foreachRDD { rdd => {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala).
    {% highlight java %} @@ -1619,7 +1619,7 @@ words.foreachRDD( ); {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java).
    {% highlight python %} @@ -1661,7 +1661,7 @@ def process(time, rdd): words.foreachRDD(process) {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/sql_network_wordcount.py). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/sql_network_wordcount.py).
    diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 8c14c3d220a23..811e8c408cb45 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -14,9 +14,9 @@ Structured Streaming is a scalable and fault-tolerant stream processing engine b # Quick Example Let’s say you want to maintain a running word count of text data received from a data server listening on a TCP socket. Let’s see how you can express this using Structured Streaming. You can see the full code in -[Scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/ -[Java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/ -[Python]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/sql/streaming/structured_network_wordcount.py). And if you +[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/ +[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/ +[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount.py). And if you [download Spark](http://spark.apache.org/downloads.html), you can directly run the example. In any case, let’s walk through the example step-by-step and understand how it works. First, we have to import the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark.
    @@ -618,9 +618,9 @@ The result tables would look something like the following. ![Window Operations](img/structured-streaming-window.png) Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window()` operations to express windowed aggregations. You can see the full code for the below examples in -[Scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala)/ -[Java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java)/ -[Python]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py). +[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala)/ +[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java)/ +[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py).
    From a21ecc9964bbd6e41a5464dcc85db1529de14d67 Mon Sep 17 00:00:00 2001 From: Luciano Resende Date: Sat, 13 Aug 2016 11:42:38 +0100 Subject: [PATCH 1134/1470] [SPARK-17023][BUILD] Upgrade to Kafka 0.10.0.1 release ## What changes were proposed in this pull request? Update Kafka streaming connector to use Kafka 0.10.0.1 release ## How was this patch tested? Tested via Spark unit and integration tests Author: Luciano Resende Closes #14606 from lresende/kafka-upgrade. (cherry picked from commit 67f025d90e6ba8c039ff45e26d34f20d24b92e6a) Signed-off-by: Sean Owen --- external/kafka-0-10/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 26965612cc0ab..f9776fc63686e 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -51,7 +51,7 @@ org.apache.kafka kafka_${scala.binary.version} - 0.10.0.0 + 0.10.0.1 com.sun.jmx From 750f8804540df5ad68a732f68598c4a2dbbc4761 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 13 Aug 2016 15:40:43 -0700 Subject: [PATCH 1135/1470] [SPARK-16966][SQL][CORE] App Name is a randomUUID even when "spark.app.name" exists ## What changes were proposed in this pull request? Don't override app name specified in `SparkConf` with a random app name. Only set it if the conf has no app name even after options have been applied. See also https://github.com/apache/spark/pull/14602 This is similar to Sherry302 's original proposal in https://github.com/apache/spark/pull/14556 ## How was this patch tested? Jenkins test, with new case reproducing the bug Author: Sean Owen Closes #14630 from srowen/SPARK-16966.2. (cherry picked from commit cdaa562c9a09e2e83e6df4e84d911ce1428a7a7c) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/SparkSession.scala | 11 +++++++---- .../apache/spark/sql/SparkSessionBuilderSuite.scala | 1 + 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 946d8cbc6bf4a..c88206c81a04e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -822,16 +822,19 @@ object SparkSession { // No active nor global default session. Create a new one. val sparkContext = userSuppliedContext.getOrElse { // set app name if not given - if (!options.contains("spark.app.name")) { - options += "spark.app.name" -> java.util.UUID.randomUUID().toString - } - + val randomAppName = java.util.UUID.randomUUID().toString val sparkConf = new SparkConf() options.foreach { case (k, v) => sparkConf.set(k, v) } + if (!sparkConf.contains("spark.app.name")) { + sparkConf.setAppName(randomAppName) + } val sc = SparkContext.getOrCreate(sparkConf) // maybe this is an existing SparkContext, update its SparkConf which maybe used // by SparkSession options.foreach { case (k, v) => sc.conf.set(k, v) } + if (!sc.conf.contains("spark.app.name")) { + sc.conf.setAppName(randomAppName) + } sc } session = new SparkSession(sparkContext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 418345b9ee8f2..386d13d07a95f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -100,6 +100,7 @@ class SparkSessionBuilderSuite extends SparkFunSuite { assert(session.conf.get("key2") == "value2") assert(session.sparkContext.conf.get("key1") == "value1") assert(session.sparkContext.conf.get("key2") == "value2") + assert(session.sparkContext.conf.get("spark.app.name") == "test") session.stop() } From e02d0d0852c5d56558ddfd13c675b3f2d70a7eea Mon Sep 17 00:00:00 2001 From: zero323 Date: Sun, 14 Aug 2016 11:59:24 +0100 Subject: [PATCH 1136/1470] [SPARK-17027][ML] Avoid integer overflow in PolynomialExpansion.getPolySize ## What changes were proposed in this pull request? Replaces custom choose function with o.a.commons.math3.CombinatoricsUtils.binomialCoefficient ## How was this patch tested? Spark unit tests Author: zero323 Closes #14614 from zero323/SPARK-17027. (cherry picked from commit 0ebf7c1bff736cf54ec47957d71394d5b75b47a7) Signed-off-by: Sean Owen --- .../ml/feature/PolynomialExpansion.scala | 10 ++++---- .../ml/feature/PolynomialExpansionSuite.scala | 24 +++++++++++++++++++ 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 72fb35bd79ad7..6e872c1f2cada 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -19,6 +19,8 @@ package org.apache.spark.ml.feature import scala.collection.mutable +import org.apache.commons.math3.util.CombinatoricsUtils + import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.linalg._ @@ -84,12 +86,12 @@ class PolynomialExpansion @Since("1.4.0") (@Since("1.4.0") override val uid: Str @Since("1.6.0") object PolynomialExpansion extends DefaultParamsReadable[PolynomialExpansion] { - private def choose(n: Int, k: Int): Int = { - Range(n, n - k, -1).product / Range(k, 1, -1).product + private def getPolySize(numFeatures: Int, degree: Int): Int = { + val n = CombinatoricsUtils.binomialCoefficient(numFeatures + degree, degree) + require(n <= Integer.MAX_VALUE) + n.toInt } - private def getPolySize(numFeatures: Int, degree: Int): Int = choose(numFeatures + degree, degree) - private def expandDense( values: Array[Double], lastIdx: Int, diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala index 8e1f9ddb36cbe..9ecd321b128f6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala @@ -116,5 +116,29 @@ class PolynomialExpansionSuite .setDegree(3) testDefaultReadWrite(t) } + + test("SPARK-17027. Integer overflow in PolynomialExpansion.getPolySize") { + val data: Array[(Vector, Int, Int)] = Array( + (Vectors.dense(1.0, 2.0, 3.0, 4.0, 5.0), 3002, 4367), + (Vectors.sparse(5, Seq((0, 1.0), (4, 5.0))), 3002, 4367), + (Vectors.dense(1.0, 2.0, 3.0, 4.0, 5.0, 6.0), 8007, 12375) + ) + + val df = spark.createDataFrame(data) + .toDF("features", "expectedPoly10size", "expectedPoly11size") + + val t = new PolynomialExpansion() + .setInputCol("features") + .setOutputCol("polyFeatures") + + for (i <- Seq(10, 11)) { + val transformed = t.setDegree(i) + .transform(df) + .select(s"expectedPoly${i}size", "polyFeatures") + .rdd.map { case Row(expected: Int, v: Vector) => expected == v.size } + + assert(transformed.collect.forall(identity)) + } + } } From 8f4cacd3a7a077a43bc82b887498cde9f6fb20e3 Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Mon, 15 Aug 2016 11:03:03 -0700 Subject: [PATCH 1137/1470] [SPARK-16508][SPARKR] Split docs for arrange and orderBy methods This PR splits arrange and orderBy methods according to their functionality (the former for sorting sparkDataFrame and the latter for windowSpec). ![screen shot 2016-08-06 at 6 39 19 pm](https://cloud.githubusercontent.com/assets/15318264/17459969/51eade28-5c05-11e6-8ca1-8d8a8e344bab.png) ![screen shot 2016-08-06 at 6 39 29 pm](https://cloud.githubusercontent.com/assets/15318264/17459966/51e3c246-5c05-11e6-8d35-3e905ca48676.png) ![screen shot 2016-08-06 at 6 40 02 pm](https://cloud.githubusercontent.com/assets/15318264/17459967/51e650ec-5c05-11e6-8698-0f037f5199ff.png) Author: Junyang Qian Closes #14522 from junyangq/SPARK-16508-0. (cherry picked from commit 564fe614c11deb657e0ac9e6b75e65370c48b7fe) Signed-off-by: Shivaram Venkataraman --- .gitignore | 5 +++++ R/pkg/R/DataFrame.R | 11 +++++------ R/pkg/R/WindowSpec.R | 18 ++++++++++-------- R/pkg/R/generics.R | 2 +- 4 files changed, 21 insertions(+), 15 deletions(-) diff --git a/.gitignore b/.gitignore index 9f8cd0b4cb232..a2639766a592e 100644 --- a/.gitignore +++ b/.gitignore @@ -77,3 +77,8 @@ spark-warehouse/ # For R session data .RData .RHistory +.Rhistory +*.Rproj +*.Rproj.* + +.Rproj.user diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index aa211b326a167..5efc891ec2fe3 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2044,14 +2044,14 @@ setMethod("rename", setClassUnion("characterOrColumn", c("character", "Column")) -#' Arrange +#' Arrange Rows by Variables #' #' Sort a SparkDataFrame by the specified column(s). #' -#' @param x A SparkDataFrame to be sorted. -#' @param col A character or Column object vector indicating the fields to sort on -#' @param ... Additional sorting fields -#' @param decreasing A logical argument indicating sorting order for columns when +#' @param x a SparkDataFrame to be sorted. +#' @param col a character or Column object indicating the fields to sort on +#' @param ... additional sorting fields +#' @param decreasing a logical argument indicating sorting order for columns when #' a character vector is specified for col #' @return A SparkDataFrame where all elements are sorted. #' @family SparkDataFrame functions @@ -2116,7 +2116,6 @@ setMethod("arrange", }) #' @rdname arrange -#' @name orderBy #' @aliases orderBy,SparkDataFrame,characterOrColumn-method #' @export #' @note orderBy(SparkDataFrame, characterOrColumn) since 1.4.0 diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index 4746380096245..751ba3fde954d 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -82,16 +82,18 @@ setMethod("partitionBy", } }) -#' orderBy +#' Ordering Columns in a WindowSpec #' #' Defines the ordering columns in a WindowSpec. -#' #' @param x a WindowSpec -#' @return a WindowSpec -#' @rdname arrange +#' @param col a character or Column object indicating an ordering column +#' @param ... additional sorting fields +#' @return A WindowSpec. #' @name orderBy +#' @rdname orderBy #' @aliases orderBy,WindowSpec,character-method #' @family windowspec_method +#' @seealso See \link{arrange} for use in sorting a SparkDataFrame #' @export #' @examples #' \dontrun{ @@ -105,7 +107,7 @@ setMethod("orderBy", windowSpec(callJMethod(x@sws, "orderBy", col, list(...))) }) -#' @rdname arrange +#' @rdname orderBy #' @name orderBy #' @aliases orderBy,WindowSpec,Column-method #' @export @@ -122,7 +124,7 @@ setMethod("orderBy", #' rowsBetween #' #' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). -#' +#' #' Both `start` and `end` are relative positions from the current row. For example, "0" means #' "current row", while "-1" means the row before the current row, and "5" means the fifth row #' after the current row. @@ -154,7 +156,7 @@ setMethod("rowsBetween", #' rangeBetween #' #' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). -#' +#' #' Both `start` and `end` are relative from the current row. For example, "0" means "current row", #' while "-1" means one off before the current row, and "5" means the five off after the #' current row. @@ -188,7 +190,7 @@ setMethod("rangeBetween", #' over #' -#' Define a windowing column. +#' Define a windowing column. #' #' @rdname over #' @name over diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index e7444ac2467d8..10a09129ec927 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -551,7 +551,7 @@ setGeneric("merge") #' @export setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) -#' @rdname arrange +#' @rdname orderBy #' @export setGeneric("orderBy", function(x, col, ...) { standardGeneric("orderBy") }) From 45036327fdbdb0167b3c53245fce9dc2be67ffe9 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 15 Aug 2016 15:55:32 -0700 Subject: [PATCH 1138/1470] [SPARK-17065][SQL] Improve the error message when encountering an incompatible DataSourceRegister ## What changes were proposed in this pull request? Add an instruction to ask the user to remove or upgrade the incompatible DataSourceRegister in the error message. ## How was this patch tested? Test command: ``` build/sbt -Dscala-2.10 package SPARK_SCALA_VERSION=2.10 bin/spark-shell --packages ai.h2o:sparkling-water-core_2.10:1.6.5 scala> Seq(1).toDS().write.format("parquet").save("foo") ``` Before: ``` java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at java.util.ServiceLoader.fail(ServiceLoader.java:232) at java.util.ServiceLoader.access$100(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) ... Caused by: java.lang.NoClassDefFoundError: org/apache/spark/Logging at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:760) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:467) at java.net.URLClassLoader.access$100(URLClassLoader.java:73) at java.net.URLClassLoader$1.run(URLClassLoader.java:368) at java.net.URLClassLoader$1.run(URLClassLoader.java:362) at java.security.AccessController.doPrivileged(Native Method) ... ``` After: ``` java.lang.ClassNotFoundException: Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at org.apache.spark.sql.execution.datasources.DataSource.lookupDataSource(DataSource.scala:178) at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:441) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:213) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:196) ... ``` Author: Shixiong Zhu Closes #14651 from zsxwing/SPARK-17065. (cherry picked from commit 268b71d0d792f875fcfaec5314862236754a00d6) Signed-off-by: Yin Huai --- .../execution/datasources/DataSource.scala | 91 +++++++++++-------- 1 file changed, 52 insertions(+), 39 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index f572b93991e0c..f5727da387d13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources -import java.util.ServiceLoader +import java.util.{ServiceConfigurationError, ServiceLoader} import scala.collection.JavaConverters._ import scala.language.{existentials, implicitConversions} @@ -123,50 +123,63 @@ case class DataSource( val loader = Utils.getContextOrSparkClassLoader val serviceLoader = ServiceLoader.load(classOf[DataSourceRegister], loader) - serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList match { - // the provider format did not match any given registered aliases - case Nil => - try { - Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match { - case Success(dataSource) => - // Found the data source using fully qualified path - dataSource - case Failure(error) => - if (provider.toLowerCase == "orc" || + try { + serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList match { + // the provider format did not match any given registered aliases + case Nil => + try { + Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match { + case Success(dataSource) => + // Found the data source using fully qualified path + dataSource + case Failure(error) => + if (provider.toLowerCase == "orc" || provider.startsWith("org.apache.spark.sql.hive.orc")) { - throw new AnalysisException( - "The ORC data source must be used with Hive support enabled") - } else if (provider.toLowerCase == "avro" || + throw new AnalysisException( + "The ORC data source must be used with Hive support enabled") + } else if (provider.toLowerCase == "avro" || provider == "com.databricks.spark.avro") { - throw new AnalysisException( - s"Failed to find data source: ${provider.toLowerCase}. Please use Spark " + - "package http://spark-packages.org/package/databricks/spark-avro") + throw new AnalysisException( + s"Failed to find data source: ${provider.toLowerCase}. Please use Spark " + + "package http://spark-packages.org/package/databricks/spark-avro") + } else { + throw new ClassNotFoundException( + s"Failed to find data source: $provider. Please find packages at " + + "http://spark-packages.org", + error) + } + } + } catch { + case e: NoClassDefFoundError => // This one won't be caught by Scala NonFatal + // NoClassDefFoundError's class name uses "/" rather than "." for packages + val className = e.getMessage.replaceAll("/", ".") + if (spark2RemovedClasses.contains(className)) { + throw new ClassNotFoundException(s"$className was removed in Spark 2.0. " + + "Please check if your library is compatible with Spark 2.0", e) } else { - throw new ClassNotFoundException( - s"Failed to find data source: $provider. Please find packages at " + - "http://spark-packages.org", - error) + throw e } } - } catch { - case e: NoClassDefFoundError => // This one won't be caught by Scala NonFatal - // NoClassDefFoundError's class name uses "/" rather than "." for packages - val className = e.getMessage.replaceAll("/", ".") - if (spark2RemovedClasses.contains(className)) { - throw new ClassNotFoundException(s"$className was removed in Spark 2.0. " + - "Please check if your library is compatible with Spark 2.0", e) - } else { - throw e - } + case head :: Nil => + // there is exactly one registered alias + head.getClass + case sources => + // There are multiple registered aliases for the input + sys.error(s"Multiple sources found for $provider " + + s"(${sources.map(_.getClass.getName).mkString(", ")}), " + + "please specify the fully qualified class name.") + } + } catch { + case e: ServiceConfigurationError if e.getCause.isInstanceOf[NoClassDefFoundError] => + // NoClassDefFoundError's class name uses "/" rather than "." for packages + val className = e.getCause.getMessage.replaceAll("/", ".") + if (spark2RemovedClasses.contains(className)) { + throw new ClassNotFoundException(s"Detected an incompatible DataSourceRegister. " + + "Please remove the incompatible library from classpath or upgrade it. " + + s"Error: ${e.getMessage}", e) + } else { + throw e } - case head :: Nil => - // there is exactly one registered alias - head.getClass - case sources => - // There are multiple registered aliases for the input - sys.error(s"Multiple sources found for $provider " + - s"(${sources.map(_.getClass.getName).mkString(", ")}), " + - "please specify the fully qualified class name.") } } From 2e2c787bf588e129eaaadc792737fd9d2892939c Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 16 Aug 2016 01:12:27 -0700 Subject: [PATCH 1139/1470] [SPARK-16964][SQL] Remove private[hive] from sql.hive.execution package ## What changes were proposed in this pull request? This PR is a small follow-up to https://github.com/apache/spark/pull/14554. This also widens the visibility of a few (similar) Hive classes. ## How was this patch tested? No test. Only a visibility change. Author: Herman van Hovell Closes #14654 from hvanhovell/SPARK-16964-hive. (cherry picked from commit 8fdc6ce400f9130399fbdd004df48b3ba95bcd6a) Signed-off-by: Reynold Xin --- .../sql/hive/execution/CreateHiveTableAsSelectCommand.scala | 1 - .../apache/spark/sql/hive/execution/ScriptTransformation.scala | 3 --- .../scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala | 3 +-- 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 15a5d79dcb085..3a8b0f1b8ebdf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.hive.MetastoreRelation * @param ignoreIfExists allow continue working if it's already exists, otherwise * raise exception */ -private[hive] case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, 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 dfb12512a40fc..9747abbf15a55 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 @@ -51,7 +51,6 @@ import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfig * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ -private[hive] case class ScriptTransformation( input: Seq[Expression], script: String, @@ -336,7 +335,6 @@ private class ScriptTransformationWriterThread( } } -private[hive] object HiveScriptIOSchema { def apply(input: ScriptInputOutputSchema): HiveScriptIOSchema = { HiveScriptIOSchema( @@ -355,7 +353,6 @@ object HiveScriptIOSchema { /** * The wrapper class of Hive input and output schema properties */ -private[hive] case class HiveScriptIOSchema ( inputRowFormat: Seq[(String, String)], outputRowFormat: Seq[(String, String)], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index a2c8092e01bb9..894c71c97944e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -47,8 +47,7 @@ import org.apache.spark.util.SerializableConfiguration * [[FileFormat]] for reading ORC files. If this is moved or renamed, please update * [[DataSource]]'s backwardCompatibilityMap. */ -private[sql] class OrcFileFormat - extends FileFormat with DataSourceRegister with Serializable { +class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable { override def shortName(): String = "orc" From 237ae54c960d52b35b4bc673609aed9998c2bd45 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 16 Aug 2016 01:14:53 -0700 Subject: [PATCH 1140/1470] Revert "[SPARK-16964][SQL] Remove private[hive] from sql.hive.execution package" This reverts commit 2e2c787bf588e129eaaadc792737fd9d2892939c. --- .../sql/hive/execution/CreateHiveTableAsSelectCommand.scala | 1 + .../apache/spark/sql/hive/execution/ScriptTransformation.scala | 3 +++ .../scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala | 3 ++- 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 3a8b0f1b8ebdf..15a5d79dcb085 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.hive.MetastoreRelation * @param ignoreIfExists allow continue working if it's already exists, otherwise * raise exception */ +private[hive] case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, 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 9747abbf15a55..dfb12512a40fc 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 @@ -51,6 +51,7 @@ import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfig * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ +private[hive] case class ScriptTransformation( input: Seq[Expression], script: String, @@ -335,6 +336,7 @@ private class ScriptTransformationWriterThread( } } +private[hive] object HiveScriptIOSchema { def apply(input: ScriptInputOutputSchema): HiveScriptIOSchema = { HiveScriptIOSchema( @@ -353,6 +355,7 @@ object HiveScriptIOSchema { /** * The wrapper class of Hive input and output schema properties */ +private[hive] case class HiveScriptIOSchema ( inputRowFormat: Seq[(String, String)], outputRowFormat: Seq[(String, String)], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 894c71c97944e..a2c8092e01bb9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -47,7 +47,8 @@ import org.apache.spark.util.SerializableConfiguration * [[FileFormat]] for reading ORC files. If this is moved or renamed, please update * [[DataSource]]'s backwardCompatibilityMap. */ -class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable { +private[sql] class OrcFileFormat + extends FileFormat with DataSourceRegister with Serializable { override def shortName(): String = "orc" From 1c56971167a0ebb3c422ccc7cc3d6904015fe2ec Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 16 Aug 2016 01:15:31 -0700 Subject: [PATCH 1141/1470] [SPARK-16964][SQL] Remove private[sql] and private[spark] from sql.execution package [Backport] ## What changes were proposed in this pull request? This PR backports https://github.com/apache/spark/pull/14554 to branch-2.0. I have also changed the visibility of a few similar Hive classes. ## How was this patch tested? (Only a package visibility change) Author: Herman van Hovell Author: Reynold Xin Closes #14652 from hvanhovell/SPARK-16964. --- .../spark/sql/execution/CacheManager.scala | 22 ++++++++--------- .../spark/sql/execution/ExistingRDD.scala | 18 +++++++------- .../spark/sql/execution/ExpandExec.scala | 2 +- .../spark/sql/execution/FileRelation.scala | 2 +- .../spark/sql/execution/GenerateExec.scala | 2 +- .../sql/execution/LocalTableScanExec.scala | 4 ++-- .../spark/sql/execution/RowIterator.scala | 2 +- .../spark/sql/execution/SQLExecution.scala | 2 +- .../apache/spark/sql/execution/SortExec.scala | 6 ++--- .../spark/sql/execution/SparkPlan.scala | 14 +++++------ .../spark/sql/execution/SparkPlanInfo.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 6 ++--- .../sql/execution/UnsafeRowSerializer.scala | 4 ++-- .../sql/execution/WholeStageCodegenExec.scala | 2 +- .../aggregate/HashAggregateExec.scala | 2 +- .../aggregate/SortAggregateExec.scala | 2 +- .../spark/sql/execution/aggregate/udaf.scala | 6 ++--- .../execution/basicPhysicalOperators.scala | 6 ++--- .../execution/columnar/InMemoryRelation.scala | 8 +++---- .../columnar/InMemoryTableScanExec.scala | 4 ++-- .../sql/execution/command/commands.scala | 4 ++-- .../datasources/DataSourceStrategy.scala | 8 +++---- .../datasources/FileSourceStrategy.scala | 2 +- .../InsertIntoDataSourceCommand.scala | 2 +- .../InsertIntoHadoopFsRelationCommand.scala | 2 +- .../datasources/PartitioningUtils.scala | 24 ++++++++++--------- .../datasources/WriterContainer.scala | 8 +++---- .../sql/execution/datasources/bucket.scala | 2 +- .../datasources/csv/CSVOptions.scala | 2 +- .../execution/datasources/csv/CSVParser.scala | 4 ++-- .../datasources/csv/CSVRelation.scala | 4 ++-- .../datasources/fileSourceInterfaces.scala | 6 ++--- .../execution/datasources/jdbc/JDBCRDD.scala | 8 +++---- .../parquet/ParquetFileFormat.scala | 17 ++++++------- .../datasources/parquet/ParquetFilters.scala | 2 +- .../datasources/parquet/ParquetOptions.scala | 6 ++--- .../sql/execution/datasources/rules.scala | 6 ++--- .../spark/sql/execution/debug/package.scala | 2 +- .../exchange/BroadcastExchangeExec.scala | 2 +- .../exchange/ExchangeCoordinator.scala | 4 ++-- .../execution/exchange/ShuffleExchange.scala | 9 +++---- .../joins/BroadcastHashJoinExec.scala | 2 +- .../joins/BroadcastNestedLoopJoinExec.scala | 2 +- .../joins/CartesianProductExec.scala | 5 ++-- .../joins/ShuffledHashJoinExec.scala | 2 +- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../sql/execution/metric/SQLMetrics.scala | 10 ++++---- .../execution/python/ExtractPythonUDFs.scala | 4 ++-- .../execution/r/MapPartitionsRWrapper.scala | 4 ++-- .../sql/execution/stat/FrequentItems.scala | 4 ++-- .../sql/execution/stat/StatFunctions.scala | 8 +++---- .../streaming/IncrementalExecution.scala | 2 +- .../execution/streaming/StreamExecution.scala | 19 +++++++-------- .../execution/streaming/StreamProgress.scala | 2 +- .../streaming/state/StateStore.scala | 2 +- .../state/StateStoreCoordinator.scala | 4 ++-- .../sql/execution/ui/ExecutionPage.scala | 2 +- .../spark/sql/execution/ui/SQLListener.scala | 6 ++--- .../spark/sql/execution/ui/SQLTab.scala | 4 ++-- .../sql/execution/ui/SparkPlanGraph.scala | 6 ++--- .../spark/sql/internal/SharedState.scala | 2 -- .../CreateHiveTableAsSelectCommand.scala | 1 - .../hive/execution/HiveTableScanExec.scala | 2 +- .../hive/execution/ScriptTransformation.scala | 3 --- .../spark/sql/hive/orc/OrcFileFormat.scala | 2 +- 65 files changed, 166 insertions(+), 174 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index de2503a87ab7d..83b7c779ab818 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK /** Holds a cached logical plan and its data */ -private[sql] case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) +case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) /** * Provides support in a SQLContext for caching query results and automatically using these cached @@ -41,7 +41,7 @@ private[sql] case class CachedData(plan: LogicalPlan, cachedRepresentation: InMe * * Internal to Spark SQL. */ -private[sql] class CacheManager extends Logging { +class CacheManager extends Logging { @transient private val cachedData = new scala.collection.mutable.ArrayBuffer[CachedData] @@ -68,13 +68,13 @@ private[sql] class CacheManager extends Logging { } /** Clears all cached tables. */ - private[sql] def clearCache(): Unit = writeLock { + def clearCache(): Unit = writeLock { cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist()) cachedData.clear() } /** Checks if the cache is empty. */ - private[sql] def isEmpty: Boolean = readLock { + def isEmpty: Boolean = readLock { cachedData.isEmpty } @@ -83,7 +83,7 @@ private[sql] class CacheManager extends Logging { * Unlike `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because * recomputing the in-memory columnar representation of the underlying table is expensive. */ - private[sql] def cacheQuery( + def cacheQuery( query: Dataset[_], tableName: Option[String] = None, storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock { @@ -108,7 +108,7 @@ private[sql] class CacheManager extends Logging { * Tries to remove the data for the given [[Dataset]] from the cache. * No operation, if it's already uncached. */ - private[sql] def uncacheQuery(query: Dataset[_], blocking: Boolean = true): Boolean = writeLock { + def uncacheQuery(query: Dataset[_], blocking: Boolean = true): Boolean = writeLock { val planToCache = query.queryExecution.analyzed val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) val found = dataIndex >= 0 @@ -120,17 +120,17 @@ private[sql] class CacheManager extends Logging { } /** Optionally returns cached data for the given [[Dataset]] */ - private[sql] def lookupCachedData(query: Dataset[_]): Option[CachedData] = readLock { + def lookupCachedData(query: Dataset[_]): Option[CachedData] = readLock { lookupCachedData(query.queryExecution.analyzed) } /** Optionally returns cached data for the given [[LogicalPlan]]. */ - private[sql] def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { + def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { cachedData.find(cd => plan.sameResult(cd.plan)) } /** Replaces segments of the given logical plan with cached versions where possible. */ - private[sql] def useCachedData(plan: LogicalPlan): LogicalPlan = { + def useCachedData(plan: LogicalPlan): LogicalPlan = { plan transformDown { case currentFragment => lookupCachedData(currentFragment) @@ -143,7 +143,7 @@ private[sql] class CacheManager extends Logging { * Invalidates the cache of any data that contains `plan`. Note that it is possible that this * function will over invalidate. */ - private[sql] def invalidateCache(plan: LogicalPlan): Unit = writeLock { + def invalidateCache(plan: LogicalPlan): Unit = writeLock { cachedData.foreach { case data if data.plan.collect { case p if p.sameResult(plan) => p }.nonEmpty => data.cachedRepresentation.recache() @@ -155,7 +155,7 @@ private[sql] class CacheManager extends Logging { * Invalidates the cache of any data that contains `resourcePath` in one or more * `HadoopFsRelation` node(s) as part of its logical plan. */ - private[sql] def invalidateCachedPath( + def invalidateCachedPath( sparkSession: SparkSession, resourcePath: String): Unit = writeLock { val (fs, qualifiedPath) = { val path = new Path(resourcePath) 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 09203e69983da..ba30bed0b450e 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 @@ -75,7 +75,7 @@ object RDDConversions { } /** Logical plan node for scanning data from an RDD. */ -private[sql] case class LogicalRDD( +case class LogicalRDD( output: Seq[Attribute], rdd: RDD[InternalRow])(session: SparkSession) extends LogicalPlan with MultiInstanceRelation { @@ -106,12 +106,12 @@ private[sql] case class LogicalRDD( } /** Physical plan node for scanning data from an RDD. */ -private[sql] case class RDDScanExec( +case class RDDScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], override val nodeName: String) extends LeafExecNode { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) protected override def doExecute(): RDD[InternalRow] = { @@ -130,7 +130,7 @@ private[sql] case class RDDScanExec( } } -private[sql] trait DataSourceScanExec extends LeafExecNode { +trait DataSourceScanExec extends LeafExecNode { val rdd: RDD[InternalRow] val relation: BaseRelation val metastoreTableIdentifier: Option[TableIdentifier] @@ -147,7 +147,7 @@ private[sql] trait DataSourceScanExec extends LeafExecNode { } /** Physical plan node for scanning data from a relation. */ -private[sql] case class RowDataSourceScanExec( +case class RowDataSourceScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], @transient relation: BaseRelation, @@ -156,7 +156,7 @@ private[sql] case class RowDataSourceScanExec( override val metastoreTableIdentifier: Option[TableIdentifier]) extends DataSourceScanExec with CodegenSupport { - private[sql] override lazy val metrics = + override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) val outputUnsafeRows = relation match { @@ -222,7 +222,7 @@ private[sql] case class RowDataSourceScanExec( } /** Physical plan node for scanning data from a batched relation. */ -private[sql] case class BatchedDataSourceScanExec( +case class BatchedDataSourceScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], @transient relation: BaseRelation, @@ -231,7 +231,7 @@ private[sql] case class BatchedDataSourceScanExec( override val metastoreTableIdentifier: Option[TableIdentifier]) extends DataSourceScanExec with CodegenSupport { - private[sql] override lazy val metrics = + override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) @@ -337,7 +337,7 @@ private[sql] case class BatchedDataSourceScanExec( } } -private[sql] object DataSourceScanExec { +object DataSourceScanExec { // Metadata keys val INPUT_PATHS = "InputPaths" val PUSHED_FILTERS = "PushedFilters" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index 4c046f7bdca48..d5603b3b00914 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -39,7 +39,7 @@ case class ExpandExec( child: SparkPlan) extends UnaryExecNode with CodegenSupport { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) // The GroupExpressions can output data with arbitrary partitioning, so set it diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala index 7a2a9eed5807d..a299fed7fd14a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/FileRelation.scala @@ -22,7 +22,7 @@ package org.apache.spark.sql.execution * the list of paths that it returns will be returned to a user who calls `inputPaths` on any * DataFrame that queries this relation. */ -private[sql] trait FileRelation { +trait FileRelation { /** Returns the list of files that will be read when scanning this relation. */ def inputFiles: Array[String] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 8b62c5507c0c8..39189a2b0c72c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -55,7 +55,7 @@ case class GenerateExec( child: SparkPlan) extends UnaryExecNode { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def producedAttributes: AttributeSet = AttributeSet(output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index df2f238d8c2e0..9f53a99346caa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -26,11 +26,11 @@ import org.apache.spark.sql.execution.metric.SQLMetrics /** * Physical plan node for scanning data from a local collection. */ -private[sql] case class LocalTableScanExec( +case class LocalTableScanExec( output: Seq[Attribute], rows: Seq[InternalRow]) extends LeafExecNode { - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) private val unsafeRows: Array[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala index 7462dbc4eba3a..717ff93eab5d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.InternalRow * iterator to consume the next row, whereas RowIterator combines these calls into a single * [[advanceNext()]] method. */ -private[sql] abstract class RowIterator { +abstract class RowIterator { /** * Advance this iterator by a single row. Returns `false` if this iterator has no more rows * and `true` otherwise. If this returns `true`, then the new row can be retrieved by calling diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 6cb1a44a2044a..ec07aab359ac6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} -private[sql] object SQLExecution { +object SQLExecution { val EXECUTION_ID_KEY = "spark.sql.execution.id" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 66a16ac576b3a..cde3ed48ffeaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -22,11 +22,9 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, GenerateUnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types._ -import org.apache.spark.util.collection.unsafe.sort.RadixSort; /** * Performs (external) sorting. @@ -52,7 +50,7 @@ case class SortExec( private val enableRadixSort = sqlContext.conf.enableRadixSort - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "sortTime" -> SQLMetrics.createTimingMetric(sparkContext, "sort time"), "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")) 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 045ccc7bd6eae..79cb40948b982 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,24 +72,24 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Return all metadata that describes more details of this SparkPlan. */ - private[sql] def metadata: Map[String, String] = Map.empty + def metadata: Map[String, String] = Map.empty /** * Return all metrics containing metrics of this SparkPlan. */ - private[sql] def metrics: Map[String, SQLMetric] = Map.empty + def metrics: Map[String, SQLMetric] = Map.empty /** * Reset all the metrics. */ - private[sql] def resetMetrics(): Unit = { + def resetMetrics(): Unit = { metrics.valuesIterator.foreach(_.reset()) } /** * Return a LongSQLMetric according to the name. */ - private[sql] def longMetric(name: String): SQLMetric = metrics(name) + def longMetric(name: String): SQLMetric = metrics(name) // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ @@ -395,7 +395,7 @@ object SparkPlan { ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) } -private[sql] trait LeafExecNode extends SparkPlan { +trait LeafExecNode extends SparkPlan { override def children: Seq[SparkPlan] = Nil override def producedAttributes: AttributeSet = outputSet } @@ -407,7 +407,7 @@ object UnaryExecNode { } } -private[sql] trait UnaryExecNode extends SparkPlan { +trait UnaryExecNode extends SparkPlan { def child: SparkPlan override def children: Seq[SparkPlan] = child :: Nil @@ -415,7 +415,7 @@ private[sql] trait UnaryExecNode extends SparkPlan { override def outputPartitioning: Partitioning = child.outputPartitioning } -private[sql] trait BinaryExecNode extends SparkPlan { +trait BinaryExecNode extends SparkPlan { def left: SparkPlan def right: SparkPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index f84070a0c4bcb..7aa93126fdabd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -47,7 +47,7 @@ class SparkPlanInfo( } } -private[sql] object SparkPlanInfo { +private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan 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 b619d4edc30de..6d7c193fd42c8 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 @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.InternalRow @@ -43,13 +42,12 @@ import org.apache.spark.sql.streaming.StreamingQuery * writing libraries should instead consider using the stable APIs provided in * [[org.apache.spark.sql.sources]] */ -@DeveloperApi abstract class SparkStrategy extends GenericStrategy[SparkPlan] { override protected def planLater(plan: LogicalPlan): SparkPlan = PlanLater(plan) } -private[sql] case class PlanLater(plan: LogicalPlan) extends LeafExecNode { +case class PlanLater(plan: LogicalPlan) extends LeafExecNode { override def output: Seq[Attribute] = plan.output @@ -58,7 +56,7 @@ private[sql] case class PlanLater(plan: LogicalPlan) extends LeafExecNode { } } -private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { +abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SparkPlanner => /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala index 484923428f4ad..8ab553369de6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala @@ -40,12 +40,12 @@ import org.apache.spark.unsafe.Platform * * @param numFields the number of fields in the row being serialized. */ -private[sql] class UnsafeRowSerializer( +class UnsafeRowSerializer( numFields: Int, dataSize: SQLMetric = null) extends Serializer with Serializable { override def newInstance(): SerializerInstance = new UnsafeRowSerializerInstance(numFields, dataSize) - override private[spark] def supportsRelocationOfSerializedObjects: Boolean = true + override def supportsRelocationOfSerializedObjects: Boolean = true } private class UnsafeRowSerializerInstance( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index ac4c3aae5f8ee..fb57ed7692de4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -295,7 +295,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co override def outputPartitioning: Partitioning = child.outputPartitioning override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext, WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 54d7340d8acd0..cfc47aba889aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -54,7 +54,7 @@ case class HashAggregateExec( child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ aggregateExpressions.flatMap(_.aggregateFunction.inputAggBufferAttributes) - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 05dbacf07a178..7c41e5e4c28aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -49,7 +49,7 @@ case class SortAggregateExec( AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ AttributeSet(aggregateBufferAttributes) - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index b047bc0641dd2..586e1456ac69e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -204,7 +204,7 @@ sealed trait BufferSetterGetterUtils { /** * A Mutable [[Row]] representing a mutable aggregation buffer. */ -private[sql] class MutableAggregationBufferImpl ( +private[aggregate] class MutableAggregationBufferImpl( schema: StructType, toCatalystConverters: Array[Any => Any], toScalaConverters: Array[Any => Any], @@ -266,7 +266,7 @@ private[sql] class MutableAggregationBufferImpl ( /** * A [[Row]] representing an immutable aggregation buffer. */ -private[sql] class InputAggregationBuffer private[sql] ( +private[aggregate] class InputAggregationBuffer( schema: StructType, toCatalystConverters: Array[Any => Any], toScalaConverters: Array[Any => Any], @@ -319,7 +319,7 @@ private[sql] class InputAggregationBuffer private[sql] ( * The internal wrapper used to hook a [[UserDefinedAggregateFunction]] `udaf` in the * internal aggregation code path. */ -private[sql] case class ScalaUDAF( +case class ScalaUDAF( children: Seq[Expression], udaf: UserDefinedAggregateFunction, mutableAggBufferOffset: Int = 0, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 185c79f899e68..e6f7081f2916d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -102,7 +102,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) } } - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -228,7 +228,7 @@ case class SampleExec( child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) protected override def doExecute(): RDD[InternalRow] = { @@ -317,7 +317,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) override val output: Seq[Attribute] = range.output - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) // output attributes should not affect the results diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 079e122a5a85a..479934a7afc75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -34,7 +34,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.CollectionAccumulator -private[sql] object InMemoryRelation { +object InMemoryRelation { def apply( useCompression: Boolean, batchSize: Int, @@ -55,15 +55,15 @@ private[sql] object InMemoryRelation { private[columnar] case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) -private[sql] case class InMemoryRelation( +case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, batchSize: Int, storageLevel: StorageLevel, @transient child: SparkPlan, tableName: Option[String])( - @transient private[sql] var _cachedColumnBuffers: RDD[CachedBatch] = null, - private[sql] val batchStats: CollectionAccumulator[InternalRow] = + @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, + val batchStats: CollectionAccumulator[InternalRow] = child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) extends logical.LeafNode with MultiInstanceRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 183e4947b6d72..e63b313cb1d5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.UserDefinedType -private[sql] case class InMemoryTableScanExec( +case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @transient relation: InMemoryRelation) @@ -36,7 +36,7 @@ private[sql] case class InMemoryTableScanExec( override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = attributes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 7eaad81a81615..cce1489abd301 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.types._ * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { +trait RunnableCommand extends LogicalPlan with logical.Command { override def output: Seq[Attribute] = Seq.empty override def children: Seq[LogicalPlan] = Seq.empty def run(sparkSession: SparkSession): Seq[Row] @@ -45,7 +45,7 @@ private[sql] trait RunnableCommand extends LogicalPlan with logical.Command { * A physical operator that executes the run method of a `RunnableCommand` and * saves the result to prevent multiple executions. */ -private[sql] case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkPlan { +case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkPlan { /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 27133f0a43f2e..bd65d0251197b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -43,7 +43,7 @@ import org.apache.spark.unsafe.types.UTF8String * Replaces generic operations with specific variants that are designed to work with Spark * SQL Data Sources. */ -private[sql] case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { +case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { def resolver: Resolver = { if (conf.caseSensitiveAnalysis) { @@ -54,7 +54,7 @@ private[sql] case class DataSourceAnalysis(conf: CatalystConf) extends Rule[Logi } // The access modifier is used to expose this method to tests. - private[sql] def convertStaticPartitions( + def convertStaticPartitions( sourceAttributes: Seq[Attribute], providedPartitions: Map[String, Option[String]], targetAttributes: Seq[Attribute], @@ -202,7 +202,7 @@ private[sql] case class DataSourceAnalysis(conf: CatalystConf) extends Rule[Logi * Replaces [[SimpleCatalogRelation]] with data source table if its table property contains data * source information. */ -private[sql] class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { +class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { private def readDataSourceTable(sparkSession: SparkSession, table: CatalogTable): LogicalPlan = { val userSpecifiedSchema = DDLUtils.getSchemaFromTableProperties(table) @@ -242,7 +242,7 @@ private[sql] class FindDataSourceTable(sparkSession: SparkSession) extends Rule[ /** * A Strategy for planning scans over data sources defined using the sources API. */ -private[sql] object DataSourceStrategy extends Strategy with Logging { +object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _)) => pruneFilterProjectRaw( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 8af9562330e81..74510f9c08b6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -54,7 +54,7 @@ import org.apache.spark.sql.execution.SparkPlan * is under the threshold with the addition of the next file, add it. If not, open a new bucket * and add it. Proceed to the next file. */ -private[sql] object FileSourceStrategy extends Strategy with Logging { +object FileSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projects, filters, l @ LogicalRelation(files: HadoopFsRelation, _, table)) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index 8549ae96e2f39..b2ff68a833fea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.sources.InsertableRelation /** * Inserts the results of `query` in to a relation that extends [[InsertableRelation]]. */ -private[sql] case class InsertIntoDataSourceCommand( +case class InsertIntoDataSourceCommand( logicalRelation: LogicalRelation, query: LogicalPlan, overwrite: Boolean) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 1426dcf4697ff..d8b8fae3bf2d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -55,7 +55,7 @@ import org.apache.spark.sql.internal.SQLConf * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is * thrown during job commitment, also aborts the job. */ -private[sql] case class InsertIntoHadoopFsRelationCommand( +case class InsertIntoHadoopFsRelationCommand( outputPath: Path, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index c3561099d6842..504464216e5a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types._ +// TODO: We should tighten up visibility of the classes here once we clean up Hive coupling. object PartitionDirectory { def apply(values: InternalRow, path: String): PartitionDirectory = @@ -41,22 +42,23 @@ object PartitionDirectory { * Holds a directory in a partitioned collection of files as well as as the partition values * in the form of a Row. Before scanning, the files at `path` need to be enumerated. */ -private[sql] case class PartitionDirectory(values: InternalRow, path: Path) +case class PartitionDirectory(values: InternalRow, path: Path) -private[sql] case class PartitionSpec( +case class PartitionSpec( partitionColumns: StructType, partitions: Seq[PartitionDirectory]) -private[sql] object PartitionSpec { +object PartitionSpec { val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[PartitionDirectory]) } -private[sql] object PartitioningUtils { +object PartitioningUtils { // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since sql/core doesn't // depend on Hive. - private[sql] val DEFAULT_PARTITION_NAME = "__HIVE_DEFAULT_PARTITION__" + val DEFAULT_PARTITION_NAME = "__HIVE_DEFAULT_PARTITION__" - private[sql] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { + private[datasources] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) + { require(columnNames.size == literals.size) } @@ -83,7 +85,7 @@ private[sql] object PartitioningUtils { * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28"))) * }}} */ - private[sql] def parsePartitions( + private[datasources] def parsePartitions( paths: Seq[Path], defaultPartitionName: String, typeInference: Boolean, @@ -166,7 +168,7 @@ private[sql] object PartitioningUtils { * hdfs://:/path/to/partition * }}} */ - private[sql] def parsePartition( + private[datasources] def parsePartition( path: Path, defaultPartitionName: String, typeInference: Boolean, @@ -249,7 +251,7 @@ private[sql] object PartitioningUtils { * DoubleType -> StringType * }}} */ - private[sql] def resolvePartitions( + def resolvePartitions( pathsWithPartitionValues: Seq[(Path, PartitionValues)]): Seq[PartitionValues] = { if (pathsWithPartitionValues.isEmpty) { Seq.empty @@ -275,7 +277,7 @@ private[sql] object PartitioningUtils { } } - private[sql] def listConflictingPartitionColumns( + private[datasources] def listConflictingPartitionColumns( pathWithPartitionValues: Seq[(Path, PartitionValues)]): String = { val distinctPartColNames = pathWithPartitionValues.map(_._2.columnNames).distinct @@ -308,7 +310,7 @@ private[sql] object PartitioningUtils { * [[IntegerType]], [[LongType]], [[DoubleType]], [[DecimalType.SYSTEM_DEFAULT]], and * [[StringType]]. */ - private[sql] def inferPartitionColumnValue( + private[datasources] def inferPartitionColumnValue( raw: String, defaultPartitionName: String, typeInference: Boolean): Literal = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 9a0b46c1a4a5e..e25924b1ba1ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -40,14 +40,14 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter /** A container for all the details required when writing to a table. */ -case class WriteRelation( +private[datasources] case class WriteRelation( sparkSession: SparkSession, dataSchema: StructType, path: String, prepareJobForWrite: Job => OutputWriterFactory, bucketSpec: Option[BucketSpec]) -private[sql] abstract class BaseWriterContainer( +private[datasources] abstract class BaseWriterContainer( @transient val relation: WriteRelation, @transient private val job: Job, isAppend: Boolean) @@ -234,7 +234,7 @@ private[sql] abstract class BaseWriterContainer( /** * A writer that writes all of the rows in a partition to a single file. */ -private[sql] class DefaultWriterContainer( +private[datasources] class DefaultWriterContainer( relation: WriteRelation, job: Job, isAppend: Boolean) @@ -293,7 +293,7 @@ private[sql] class DefaultWriterContainer( * done by maintaining a HashMap of open files until `maxFiles` is reached. If this occurs, the * writer externally sorts the remaining rows and then writes out them out one file at a time. */ -private[sql] class DynamicPartitionWriterContainer( +private[datasources] class DynamicPartitionWriterContainer( relation: WriteRelation, job: Job, partitionColumns: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index 6008d73717f77..2bafe967993b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -31,7 +31,7 @@ private[sql] case class BucketSpec( bucketColumnNames: Seq[String], sortColumnNames: Seq[String]) -private[sql] object BucketingUtils { +object BucketingUtils { // The file name of bucketed data should have 3 parts: // 1. some other information in the head of file name // 2. bucket id part, some numbers, starts with "_" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 22fb8163b1c0a..10fe541a2c575 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.{CompressionCodecs, ParseModes} -private[sql] class CSVOptions(@transient private val parameters: Map[String, String]) +private[csv] class CSVOptions(@transient private val parameters: Map[String, String]) extends Logging with Serializable { private def getChar(paramName: String, default: Char): Char = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index 7929ebbd90f71..0a996547d2536 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging * @param params Parameters object * @param headers headers for the columns */ -private[sql] abstract class CsvReader(params: CSVOptions, headers: Seq[String]) { +private[csv] abstract class CsvReader(params: CSVOptions, headers: Seq[String]) { protected lazy val parser: CsvParser = { val settings = new CsvParserSettings() @@ -60,7 +60,7 @@ private[sql] abstract class CsvReader(params: CSVOptions, headers: Seq[String]) * @param params Parameters object for configuration * @param headers headers for columns */ -private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) extends Logging { +private[csv] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) extends Logging { private val writerSettings = new CsvWriterSettings private val format = writerSettings.getFormat diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 083ac3350ef02..10d84f4a70d5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -159,7 +159,7 @@ object CSVRelation extends Logging { } } -private[sql] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { +private[csv] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { override def newInstance( path: String, bucketId: Option[Int], @@ -170,7 +170,7 @@ private[sql] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWrit } } -private[sql] class CsvOutputWriter( +private[csv] class CsvOutputWriter( path: String, dataSchema: StructType, context: TaskAttemptContext, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 438fccb25c778..4351bed99460a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -76,7 +76,7 @@ abstract class OutputWriterFactory extends Serializable { * through the [[OutputWriterFactory]] implementation. * @since 2.0.0 */ - private[sql] def newWriter(path: String): OutputWriter = { + def newWriter(path: String): OutputWriter = { throw new UnsupportedOperationException("newInstance with just path not supported") } } @@ -249,7 +249,7 @@ trait FileFormat { * appends partition values to [[InternalRow]]s produced by the reader function [[buildReader]] * returns. */ - private[sql] def buildReaderWithPartitionValues( + def buildReaderWithPartitionValues( sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -356,7 +356,7 @@ trait FileCatalog { /** * Helper methods for gathering metadata from HDFS. */ -private[sql] object HadoopFsRelation extends Logging { +object HadoopFsRelation extends Logging { /** Checks if we should filter out this path name. */ def shouldFilterOut(pathName: String): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 24e2c1a5fd2f6..f655155287974 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -38,11 +38,11 @@ import org.apache.spark.unsafe.types.UTF8String /** * Data corresponding to one partition of a JDBCRDD. */ -private[sql] case class JDBCPartition(whereClause: String, idx: Int) extends Partition { +case class JDBCPartition(whereClause: String, idx: Int) extends Partition { override def index: Int = idx } -private[sql] object JDBCRDD extends Logging { +object JDBCRDD extends Logging { /** * Maps a JDBC type to a Catalyst type. This function is called only when @@ -192,7 +192,7 @@ private[sql] object JDBCRDD extends Logging { * Turns a single Filter into a String representing a SQL expression. * Returns None for an unhandled filter. */ - private[jdbc] def compileFilter(f: Filter): Option[String] = { + def compileFilter(f: Filter): Option[String] = { Option(f match { case EqualTo(attr, value) => s"$attr = ${compileValue(value)}" case EqualNullSafe(attr, value) => @@ -275,7 +275,7 @@ private[sql] object JDBCRDD extends Logging { * driver code and the workers must be able to access the database; the driver * needs to fetch the schema while the workers need to fetch the data. */ -private[sql] class JDBCRDD( +private[jdbc] class JDBCRDD( sc: SparkContext, getConnection: () => Connection, schema: StructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 260071ea6fee7..94980886c6265 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -52,7 +52,7 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration -private[sql] class ParquetFileFormat +class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging @@ -268,7 +268,7 @@ private[sql] class ParquetFileFormat true } - override private[sql] def buildReaderWithPartitionValues( + override def buildReaderWithPartitionValues( sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -429,7 +429,7 @@ private[sql] class ParquetFileFormat * writes the data to the path used to generate the output writer. Callers of this factory * has to ensure which files are to be considered as committed. */ -private[sql] class ParquetOutputWriterFactory( +private[parquet] class ParquetOutputWriterFactory( sqlConf: SQLConf, dataSchema: StructType, hadoopConf: Configuration, @@ -478,7 +478,7 @@ private[sql] class ParquetOutputWriterFactory( * Returns a [[OutputWriter]] that writes data to the give path without using * [[OutputCommitter]]. */ - override private[sql] def newWriter(path: String): OutputWriter = new OutputWriter { + override def newWriter(path: String): OutputWriter = new OutputWriter { // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter private val hadoopTaskAttempId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) @@ -525,7 +525,7 @@ private[sql] class ParquetOutputWriterFactory( // NOTE: This class is instantiated and used on executor side only, no need to be serializable. -private[sql] class ParquetOutputWriter( +private[parquet] class ParquetOutputWriter( path: String, bucketId: Option[Int], context: TaskAttemptContext) @@ -563,12 +563,13 @@ private[sql] class ParquetOutputWriter( override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") - override protected[sql] def writeInternal(row: InternalRow): Unit = recordWriter.write(null, row) + override def writeInternal(row: InternalRow): Unit = recordWriter.write(null, row) override def close(): Unit = recordWriter.close(context) } -private[sql] object ParquetFileFormat extends Logging { + +object ParquetFileFormat extends Logging { /** * If parquet's block size (row group size) setting is larger than the min split size, * we use parquet's block size setting as the min split size. Otherwise, we will create @@ -715,7 +716,7 @@ private[sql] object ParquetFileFormat extends Logging { * distinguish binary and string). This method generates a correct schema by merging Metastore * schema data types and Parquet schema field names. */ - private[sql] def mergeMetastoreParquetSchema( + def mergeMetastoreParquetSchema( metastoreSchema: StructType, parquetSchema: StructType): StructType = { def schemaConflictMessage: String = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 70ae829219d59..2edd2757428aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -28,7 +28,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.spark.sql.sources import org.apache.spark.sql.types._ -private[sql] object ParquetFilters { +object ParquetFilters { case class SetInFilter[T <: Comparable[T]]( valueSet: Set[T]) extends UserDefinedPredicate[T] with Serializable { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index dd2e915e7b7f9..3eec582714e15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.internal.SQLConf /** * Options for the Parquet data source. */ -private[sql] class ParquetOptions( +private[parquet] class ParquetOptions( @transient private val parameters: Map[String, String], @transient private val sqlConf: SQLConf) extends Serializable { @@ -56,8 +56,8 @@ private[sql] class ParquetOptions( } -private[sql] object ParquetOptions { - private[sql] val MERGE_SCHEMA = "mergeSchema" +object ParquetOptions { + val MERGE_SCHEMA = "mergeSchema" // The parquet compression short names private val shortParquetCompressionCodecNames = Map( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 15b9d14bd73fe..05908d908fd20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} /** * Try to replaces [[UnresolvedRelation]]s with [[ResolveDataSource]]. */ -private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { +class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case u: UnresolvedRelation if u.tableIdentifier.database.isDefined => try { @@ -67,7 +67,7 @@ private[sql] class ResolveDataSource(sparkSession: SparkSession) extends Rule[Lo * table. It also does data type casting and field renaming, to make sure that the columns to be * inserted have the correct data type and fields have the correct names. */ -private[sql] case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { +case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { private def preprocess( insert: InsertIntoTable, tblName: String, @@ -147,7 +147,7 @@ private[sql] case class PreprocessTableInsertion(conf: SQLConf) extends Rule[Log /** * A rule to do various checks before inserting into or writing to a data source table. */ -private[sql] case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) +case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) extends (LogicalPlan => Unit) { def failAnalysis(msg: String): Unit = { throw new AnalysisException(msg) } 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 e89f792496d6a..082f97a8808fa 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 @@ -104,7 +104,7 @@ package object debug { } } - private[sql] case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { + case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { def output: Seq[Attribute] = child.output class SetAccumulator[T] extends AccumulatorV2[T, HashSet[T]] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index bd0841db7e8ab..a809076de5419 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -38,7 +38,7 @@ case class BroadcastExchangeExec( mode: BroadcastMode, child: SparkPlan) extends Exchange { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)"), "buildTime" -> SQLMetrics.createMetric(sparkContext, "time to build (ms)"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index 2ea6ee38a932a..57da85fa84f99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -79,7 +79,7 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} * - post-shuffle partition 1: pre-shuffle partition 2 * - post-shuffle partition 2: pre-shuffle partition 3 and 4 */ -private[sql] class ExchangeCoordinator( +class ExchangeCoordinator( numExchanges: Int, advisoryTargetPostShuffleInputSize: Long, minNumPostShufflePartitions: Option[Int] = None) @@ -112,7 +112,7 @@ private[sql] class ExchangeCoordinator( * Estimates partition start indices for post-shuffle partitions based on * mapOutputStatistics provided by all pre-shuffle stages. */ - private[sql] def estimatePartitionStartIndices( + def estimatePartitionStartIndices( mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = { // If we have mapOutputStatistics.length < numExchange, it is because we do not submit // a stage when the number of partitions of this dependency is 0. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala index afe0fbea73bd9..7a4a251370706 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala @@ -40,7 +40,7 @@ case class ShuffleExchange( child: SparkPlan, @transient coordinator: Option[ExchangeCoordinator]) extends Exchange { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size")) override def nodeName: String = { @@ -81,7 +81,8 @@ case class ShuffleExchange( * the partitioning scheme defined in `newPartitioning`. Those partitions of * the returned ShuffleDependency will be the input of shuffle. */ - private[sql] def prepareShuffleDependency(): ShuffleDependency[Int, InternalRow, InternalRow] = { + private[exchange] def prepareShuffleDependency() + : ShuffleDependency[Int, InternalRow, InternalRow] = { ShuffleExchange.prepareShuffleDependency( child.execute(), child.output, newPartitioning, serializer) } @@ -92,7 +93,7 @@ case class ShuffleExchange( * partition start indices array. If this optional array is defined, the returned * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array. */ - private[sql] def preparePostShuffleRDD( + private[exchange] def preparePostShuffleRDD( shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow], specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = { // If an array of partition start indices is provided, we need to use this array @@ -194,7 +195,7 @@ object ShuffleExchange { * the partitioning scheme defined in `newPartitioning`. Those partitions of * the returned ShuffleDependency will be the input of shuffle. */ - private[sql] def prepareShuffleDependency( + def prepareShuffleDependency( rdd: RDD[InternalRow], outputAttributes: Seq[Attribute], newPartitioning: Partitioning, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 7c194ab72643a..0f24baacd18d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -45,7 +45,7 @@ case class BroadcastHashJoinExec( right: SparkPlan) extends BinaryExecNode with HashJoin with CodegenSupport { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index 4d43765f8fcd3..6a9965f1a24cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -37,7 +37,7 @@ case class BroadcastNestedLoopJoinExec( condition: Option[Expression], withinBroadcastThreshold: Boolean = true) extends BinaryExecNode { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) /** BuildRight means the right relation <=> the broadcast relation. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 3a0b6efdfc910..c97fffe88b719 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -34,7 +34,6 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter * will be much faster than building the right partition for every row in left RDD, it also * materialize the right RDD (in case of the right RDD is nondeterministic). */ -private[spark] class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numFieldsOfRight: Int) extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) { @@ -78,7 +77,7 @@ class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numField for (x <- rdd1.iterator(partition.s1, context); y <- createIter()) yield (x, y) CompletionIterator[(UnsafeRow, UnsafeRow), Iterator[(UnsafeRow, UnsafeRow)]]( - resultIter, sorter.cleanupResources) + resultIter, sorter.cleanupResources()) } } @@ -89,7 +88,7 @@ case class CartesianProductExec( condition: Option[Expression]) extends BinaryExecNode { override def output: Seq[Attribute] = left.output ++ right.output - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) protected override def doPrepare(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 0036f9aadc5d9..afb6e5e3dd235 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -39,7 +39,7 @@ case class ShuffledHashJoinExec( right: SparkPlan) extends BinaryExecNode with HashJoin { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index fac6b8de8ed5e..5c9c1e6062f0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -40,7 +40,7 @@ case class SortMergeJoinExec( left: SparkPlan, right: SparkPlan) extends BinaryExecNode with CodegenSupport { - override private[sql] lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def output: Seq[Attribute] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 9817a56f499a5..15afa0b1a5391 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -55,17 +55,17 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato override def value: Long = _value // Provide special identifier as metadata so we can tell that this is a `SQLMetric` later - private[spark] override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { + override def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { new AccumulableInfo( id, name, update, value, true, true, Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) } } -private[sql] object SQLMetrics { - private[sql] val SUM_METRIC = "sum" - private[sql] val SIZE_METRIC = "size" - private[sql] val TIMING_METRIC = "timing" +object SQLMetrics { + private val SUM_METRIC = "sum" + private val SIZE_METRIC = "size" + private val TIMING_METRIC = "timing" def createMetric(sc: SparkContext, name: String): SQLMetric = { val acc = new SQLMetric(SUM_METRIC) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index 829bcae6f95d4..16e44845d5283 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.SparkPlan * Extracts all the Python UDFs in logical aggregate, which depends on aggregate expression or * grouping key, evaluate them after aggregate. */ -private[spark] object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { +object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { /** * Returns whether the expression could only be evaluated within aggregate. @@ -90,7 +90,7 @@ private[spark] object ExtractPythonUDFFromAggregate extends Rule[LogicalPlan] { * This has the limitation that the input to the Python UDF is not allowed include attributes from * multiple child operators. */ -private[spark] object ExtractPythonUDFs extends Rule[SparkPlan] { +object ExtractPythonUDFs extends Rule[SparkPlan] { private def hasPythonUDF(e: Expression): Boolean = { e.find(_.isInstanceOf[PythonUDF]).isDefined diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index 70539da348b0e..d2178e971ec20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -21,12 +21,12 @@ import org.apache.spark.api.r._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.api.r.SQLUtils._ import org.apache.spark.sql.Row -import org.apache.spark.sql.types.{BinaryType, StructField, StructType} +import org.apache.spark.sql.types.StructType /** * A function wrapper that applies the given R function to each partition. */ -private[sql] case class MapPartitionsRWrapper( +case class MapPartitionsRWrapper( func: Array[Byte], packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index b19344f04383f..b9dbfcf7734c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ -private[sql] object FrequentItems extends Logging { +object FrequentItems extends Logging { /** A helper class wrapping `MutableMap[Any, Long]` for simplicity. */ private class FreqItemCounter(size: Int) extends Serializable { @@ -79,7 +79,7 @@ private[sql] object FrequentItems extends Logging { * than 1e-4. * @return A Local DataFrame with the Array of frequent items for each column. */ - private[sql] def singlePassFreqItems( + def singlePassFreqItems( df: DataFrame, cols: Seq[String], support: Double): DataFrame = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index ea58df70b3252..50eecb409830f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -private[sql] object StatFunctions extends Logging { +object StatFunctions extends Logging { import QuantileSummaries.Stats @@ -337,7 +337,7 @@ private[sql] object StatFunctions extends Logging { } /** Calculate the Pearson Correlation Coefficient for the given columns */ - private[sql] def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = { + def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = { val counts = collectStatisticalData(df, cols, "correlation") counts.Ck / math.sqrt(counts.MkX * counts.MkY) } @@ -407,13 +407,13 @@ private[sql] object StatFunctions extends Logging { * @param cols the column names * @return the covariance of the two columns. */ - private[sql] def calculateCov(df: DataFrame, cols: Seq[String]): Double = { + def calculateCov(df: DataFrame, cols: Seq[String]): Double = { val counts = collectStatisticalData(df, cols, "covariance") counts.cov } /** Generate a table of frequencies for the elements of two columns. */ - private[sql] def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = { + def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = { val tableName = s"${col1}_$col2" val counts = df.groupBy(col1, col2).agg(count("*")).take(1e6.toInt) if (counts.length == 1e6.toInt) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 7367c68d0a0e5..05294df2673dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.streaming.OutputMode * A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]] * plan incrementally. Possibly preserving state in between each execution. */ -class IncrementalExecution private[sql]( +class IncrementalExecution( sparkSession: SparkSession, logicalPlan: LogicalPlan, val outputMode: OutputMode, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index af2229a46bebb..66fb5a4bdeb7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -49,10 +49,10 @@ class StreamExecution( override val id: Long, override val name: String, checkpointRoot: String, - private[sql] val logicalPlan: LogicalPlan, + val logicalPlan: LogicalPlan, val sink: Sink, val trigger: Trigger, - private[sql] val triggerClock: Clock, + val triggerClock: Clock, val outputMode: OutputMode) extends StreamingQuery with Logging { @@ -74,7 +74,7 @@ class StreamExecution( * input source. */ @volatile - private[sql] var committedOffsets = new StreamProgress + var committedOffsets = new StreamProgress /** * Tracks the offsets that are available to be processed, but have not yet be committed to the @@ -102,10 +102,10 @@ class StreamExecution( private var state: State = INITIALIZED @volatile - private[sql] var lastExecution: QueryExecution = null + var lastExecution: QueryExecution = null @volatile - private[sql] var streamDeathCause: StreamingQueryException = null + var streamDeathCause: StreamingQueryException = null /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() @@ -115,7 +115,7 @@ class StreamExecution( * [[org.apache.spark.util.UninterruptibleThread]] to avoid potential deadlocks in using * [[HDFSMetadataLog]]. See SPARK-14131 for more details. */ - private[sql] val microBatchThread = + val microBatchThread = new UninterruptibleThread(s"stream execution thread for $name") { override def run(): Unit = { // To fix call site like "run at :0", we bridge the call site from the caller @@ -131,8 +131,7 @@ class StreamExecution( * processing is done. Thus, the Nth record in this log indicated data that is currently being * processed and the N-1th entry indicates which offsets have been durably committed to the sink. */ - private[sql] val offsetLog = - new HDFSMetadataLog[CompositeOffset](sparkSession, checkpointFile("offsets")) + val offsetLog = new HDFSMetadataLog[CompositeOffset](sparkSession, checkpointFile("offsets")) /** Whether the query is currently active or not */ override def isActive: Boolean = state == ACTIVE @@ -159,7 +158,7 @@ class StreamExecution( * Starts the execution. This returns only after the thread has started and [[QueryStarted]] event * has been posted to all the listeners. */ - private[sql] def start(): Unit = { + def start(): Unit = { microBatchThread.setDaemon(true) microBatchThread.start() startLatch.await() // Wait until thread started and QueryStart event has been posted @@ -518,7 +517,7 @@ class StreamExecution( case object TERMINATED extends State } -private[sql] object StreamExecution { +object StreamExecution { private val _nextId = new AtomicLong(0) def nextId: Long = _nextId.getAndIncrement() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala index 405a5f0387a7e..db0bd9e6bc6f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -26,7 +26,7 @@ class StreamProgress( val baseMap: immutable.Map[Source, Offset] = new immutable.HashMap[Source, Offset]) extends scala.collection.immutable.Map[Source, Offset] { - private[sql] def toCompositeOffset(source: Seq[Source]): CompositeOffset = { + def toCompositeOffset(source: Seq[Source]): CompositeOffset = { CompositeOffset(source.map(get)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 066765324ac94..a67fdceb3cee6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -113,7 +113,7 @@ case class KeyRemoved(key: UnsafeRow) extends StoreUpdate * the store is the active instance. Accordingly, it either keeps it loaded and performs * maintenance, or unloads the store. */ -private[sql] object StateStore extends Logging { +object StateStore extends Logging { val MAINTENANCE_INTERVAL_CONFIG = "spark.sql.streaming.stateStore.maintenanceInterval" val MAINTENANCE_INTERVAL_DEFAULT_SECS = 60 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala index e418217238cca..d945d7aff2da4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala @@ -45,7 +45,7 @@ private object StopCoordinator extends StateStoreCoordinatorMessage /** Helper object used to create reference to [[StateStoreCoordinator]]. */ -private[sql] object StateStoreCoordinatorRef extends Logging { +object StateStoreCoordinatorRef extends Logging { private val endpointName = "StateStoreCoordinator" @@ -77,7 +77,7 @@ private[sql] object StateStoreCoordinatorRef extends Logging { * Reference to a [[StateStoreCoordinator]] that can be used to coordinate instances of * [[StateStore]]s across all the executors, and get their locations for job scheduling. */ -private[sql] class StateStoreCoordinatorRef private(rpcEndpointRef: RpcEndpointRef) { +class StateStoreCoordinatorRef private(rpcEndpointRef: RpcEndpointRef) { private[state] def reportActiveInstance( storeId: StateStoreId, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala index 4b4fa126b85f3..23fc0bd0bce13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} -private[sql] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { +class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { private val listener = parent.listener diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 6e94791901762..60f13432d78d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -46,14 +46,14 @@ case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) case class SparkListenerDriverAccumUpdates(executionId: Long, accumUpdates: Seq[(Long, Long)]) extends SparkListenerEvent -private[sql] class SQLHistoryListenerFactory extends SparkHistoryListenerFactory { +class SQLHistoryListenerFactory extends SparkHistoryListenerFactory { override def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener] = { List(new SQLHistoryListener(conf, sparkUI)) } } -private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Logging { +class SQLListener(conf: SparkConf) extends SparkListener with Logging { private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) @@ -333,7 +333,7 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi /** * A [[SQLListener]] for rendering the SQL UI in the history server. */ -private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) +class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) extends SQLListener(conf) { private var sqlTabAttached = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala index e8675ce749a2b..d0376af3e31ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.ui import org.apache.spark.internal.Logging import org.apache.spark.ui.{SparkUI, SparkUITab} -private[sql] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) +class SQLTab(val listener: SQLListener, sparkUI: SparkUI) extends SparkUITab(sparkUI, "SQL") with Logging { val parent = sparkUI @@ -32,6 +32,6 @@ private[sql] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) parent.addStaticHandler(SQLTab.STATIC_RESOURCE_DIR, "/static/sql") } -private[sql] object SQLTab { +object SQLTab { private val STATIC_RESOURCE_DIR = "org/apache/spark/sql/execution/ui/static" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index 8f5681bfc7cc6..4bb9d6fef4c1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.sql.execution.{SparkPlanInfo, WholeStageCodegenExec} -import org.apache.spark.sql.execution.metric.SQLMetrics + /** * A graph used for storing information of an executionPlan of DataFrame. @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * Each graph is defined with a set of nodes and a set of edges. Each node represents a node in the * SparkPlan tree, and each edge represents a parent-child relationship between two nodes. */ -private[ui] case class SparkPlanGraph( +case class SparkPlanGraph( nodes: Seq[SparkPlanGraphNode], edges: Seq[SparkPlanGraphEdge]) { def makeDotFile(metrics: Map[Long, String]): String = { @@ -55,7 +55,7 @@ private[ui] case class SparkPlanGraph( } } -private[sql] object SparkPlanGraph { +object SparkPlanGraph { /** * Build a SparkPlanGraph from the root of a SparkPlan tree. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 6c43fe3177d65..54aee5e02bb9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.internal -import org.apache.hadoop.conf.Configuration - import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SQLContext} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 15a5d79dcb085..3a8b0f1b8ebdf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.hive.MetastoreRelation * @param ignoreIfExists allow continue working if it's already exists, otherwise * raise exception */ -private[hive] case class CreateHiveTableAsSelectCommand( tableDesc: CatalogTable, query: LogicalPlan, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index cc3e74b4e8ccc..a716a3eab6219 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -54,7 +54,7 @@ case class HiveTableScanExec( require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, "Partition pruning predicates only supported for partitioned tables.") - private[sql] override lazy val metrics = Map( + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def producedAttributes: AttributeSet = outputSet ++ 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 dfb12512a40fc..9747abbf15a55 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 @@ -51,7 +51,6 @@ import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfig * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ -private[hive] case class ScriptTransformation( input: Seq[Expression], script: String, @@ -336,7 +335,6 @@ private class ScriptTransformationWriterThread( } } -private[hive] object HiveScriptIOSchema { def apply(input: ScriptInputOutputSchema): HiveScriptIOSchema = { HiveScriptIOSchema( @@ -355,7 +353,6 @@ object HiveScriptIOSchema { /** * The wrapper class of Hive input and output schema properties */ -private[hive] case class HiveScriptIOSchema ( inputRowFormat: Seq[(String, String)], outputRowFormat: Seq[(String, String)], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index a2c8092e01bb9..9843f0774af96 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -47,7 +47,7 @@ import org.apache.spark.util.SerializableConfiguration * [[FileFormat]] for reading ORC files. If this is moved or renamed, please update * [[DataSource]]'s backwardCompatibilityMap. */ -private[sql] class OrcFileFormat +class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable { override def shortName(): String = "orc" From 022230c20905a29483cfd4cc76b74fe5f208c2c8 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 16 Aug 2016 11:19:18 -0700 Subject: [PATCH 1142/1470] [SPARK-16519][SPARKR] Handle SparkR RDD generics that create warnings in R CMD check Rename RDD functions for now to avoid CRAN check warnings. Some RDD functions are sharing generics with DataFrame functions (hence the problem) so after the renames we need to add new generics, for now. unit tests Author: Felix Cheung Closes #14626 from felixcheung/rrddfunctions. (cherry picked from commit c34b546d674ce186f13d9999b97977bc281cfedf) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/RDD.R | 100 +++++----- R/pkg/R/SQLContext.R | 2 +- R/pkg/R/context.R | 2 +- R/pkg/R/generics.R | 91 +++++---- R/pkg/R/pairRDD.R | 40 ++-- R/pkg/inst/tests/testthat/test_binaryFile.R | 8 +- .../tests/testthat/test_binary_function.R | 18 +- R/pkg/inst/tests/testthat/test_broadcast.R | 4 +- R/pkg/inst/tests/testthat/test_context.R | 7 +- .../inst/tests/testthat/test_includePackage.R | 4 +- .../tests/testthat/test_parallelize_collect.R | 26 +-- R/pkg/inst/tests/testthat/test_rdd.R | 172 +++++++++--------- R/pkg/inst/tests/testthat/test_shuffle.R | 34 ++-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 28 +-- R/pkg/inst/tests/testthat/test_take.R | 32 ++-- R/pkg/inst/tests/testthat/test_textFile.R | 26 +-- R/pkg/inst/tests/testthat/test_utils.R | 6 +- 17 files changed, 313 insertions(+), 287 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 72a805256523e..6b254bb0d302c 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -67,7 +67,7 @@ setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, .Object }) -setMethod("show", "RDD", +setMethod("showRDD", "RDD", function(object) { cat(paste(callJMethod(getJRDD(object), "toString"), "\n", sep = "")) }) @@ -215,7 +215,7 @@ setValidity("RDD", #' @rdname cache-methods #' @aliases cache,RDD-method #' @noRd -setMethod("cache", +setMethod("cacheRDD", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "cache") @@ -235,12 +235,12 @@ setMethod("cache", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10, 2L) -#' persist(rdd, "MEMORY_AND_DISK") +#' persistRDD(rdd, "MEMORY_AND_DISK") #'} #' @rdname persist #' @aliases persist,RDD-method #' @noRd -setMethod("persist", +setMethod("persistRDD", signature(x = "RDD", newLevel = "character"), function(x, newLevel = "MEMORY_ONLY") { callJMethod(getJRDD(x), "persist", getStorageLevel(newLevel)) @@ -259,12 +259,12 @@ setMethod("persist", #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10, 2L) #' cache(rdd) # rdd@@env$isCached == TRUE -#' unpersist(rdd) # rdd@@env$isCached == FALSE +#' unpersistRDD(rdd) # rdd@@env$isCached == FALSE #'} #' @rdname unpersist-methods #' @aliases unpersist,RDD-method #' @noRd -setMethod("unpersist", +setMethod("unpersistRDD", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "unpersist") @@ -345,13 +345,13 @@ setMethod("numPartitions", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10, 2L) -#' collect(rdd) # list from 1 to 10 +#' collectRDD(rdd) # list from 1 to 10 #' collectPartition(rdd, 0L) # list from 1 to 5 #'} #' @rdname collect-methods #' @aliases collect,RDD-method #' @noRd -setMethod("collect", +setMethod("collectRDD", signature(x = "RDD"), function(x, flatten = TRUE) { # Assumes a pairwise RDD is backed by a JavaPairRDD. @@ -397,7 +397,7 @@ setMethod("collectPartition", setMethod("collectAsMap", signature(x = "RDD"), function(x) { - pairList <- collect(x) + pairList <- collectRDD(x) map <- new.env() lapply(pairList, function(i) { assign(as.character(i[[1]]), i[[2]], envir = map) }) as.list(map) @@ -411,30 +411,30 @@ setMethod("collectAsMap", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) -#' count(rdd) # 10 +#' countRDD(rdd) # 10 #' length(rdd) # Same as count #'} #' @rdname count #' @aliases count,RDD-method #' @noRd -setMethod("count", +setMethod("countRDD", signature(x = "RDD"), function(x) { countPartition <- function(part) { as.integer(length(part)) } valsRDD <- lapplyPartition(x, countPartition) - vals <- collect(valsRDD) + vals <- collectRDD(valsRDD) sum(as.integer(vals)) }) #' Return the number of elements in the RDD #' @rdname count #' @noRd -setMethod("length", +setMethod("lengthRDD", signature(x = "RDD"), function(x) { - count(x) + countRDD(x) }) #' Return the count of each unique value in this RDD as a list of @@ -460,7 +460,7 @@ setMethod("countByValue", signature(x = "RDD"), function(x) { ones <- lapply(x, function(item) { list(item, 1L) }) - collect(reduceByKey(ones, `+`, getNumPartitions(x))) + collectRDD(reduceByKey(ones, `+`, getNumPartitions(x))) }) #' Apply a function to all elements @@ -479,7 +479,7 @@ setMethod("countByValue", #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) -#' collect(multiplyByTwo) # 2,4,6... +#' collectRDD(multiplyByTwo) # 2,4,6... #'} setMethod("lapply", signature(X = "RDD", FUN = "function"), @@ -512,7 +512,7 @@ setMethod("map", #' 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... +#' collectRDD(multiplyByTwo) # 2,20,4,40,6,60... #'} #' @rdname flatMap #' @aliases flatMap,RDD,function-method @@ -541,7 +541,7 @@ setMethod("flatMap", #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) -#' collect(partitionSum) # 15, 40 +#' collectRDD(partitionSum) # 15, 40 #'} #' @rdname lapplyPartition #' @aliases lapplyPartition,RDD,function-method @@ -576,7 +576,7 @@ setMethod("mapPartitions", #' rdd <- parallelize(sc, 1:10, 5L) #' prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { #' partIndex * Reduce("+", part) }) -#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#' collectRDD(prod, flatten = FALSE) # 0, 7, 22, 45, 76 #'} #' @rdname lapplyPartitionsWithIndex #' @aliases lapplyPartitionsWithIndex,RDD,function-method @@ -607,7 +607,7 @@ setMethod("mapPartitionsWithIndex", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) -#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#' unlist(collectRDD(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) #'} # nolint end #' @rdname filterRDD @@ -656,7 +656,7 @@ setMethod("reduce", Reduce(func, part) } - partitionList <- collect(lapplyPartition(x, reducePartition), + partitionList <- collectRDD(lapplyPartition(x, reducePartition), flatten = FALSE) Reduce(func, partitionList) }) @@ -736,7 +736,7 @@ setMethod("foreach", lapply(x, func) NULL } - invisible(collect(mapPartitions(x, partition.func))) + invisible(collectRDD(mapPartitions(x, partition.func))) }) #' Applies a function to each partition in an RDD, and forces evaluation. @@ -753,7 +753,7 @@ setMethod("foreach", setMethod("foreachPartition", signature(x = "RDD", func = "function"), function(x, func) { - invisible(collect(mapPartitions(x, func))) + invisible(collectRDD(mapPartitions(x, func))) }) #' Take elements from an RDD. @@ -768,13 +768,13 @@ setMethod("foreachPartition", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) -#' take(rdd, 2L) # list(1, 2) +#' takeRDD(rdd, 2L) # list(1, 2) #'} # nolint end #' @rdname take #' @aliases take,RDD,numeric-method #' @noRd -setMethod("take", +setMethod("takeRDD", signature(x = "RDD", num = "numeric"), function(x, num) { resList <- list() @@ -817,13 +817,13 @@ setMethod("take", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) -#' first(rdd) +#' firstRDD(rdd) #' } #' @noRd -setMethod("first", +setMethod("firstRDD", signature(x = "RDD"), function(x) { - take(x, 1)[[1]] + takeRDD(x, 1)[[1]] }) #' Removes the duplicates from RDD. @@ -838,13 +838,13 @@ setMethod("first", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, c(1,2,2,3,3,3)) -#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#' sort(unlist(collectRDD(distinctRDD(rdd)))) # c(1, 2, 3) #'} # nolint end #' @rdname distinct #' @aliases distinct,RDD-method #' @noRd -setMethod("distinct", +setMethod("distinctRDD", signature(x = "RDD"), function(x, numPartitions = SparkR:::getNumPartitions(x)) { identical.mapped <- lapply(x, function(x) { list(x, NULL) }) @@ -868,8 +868,8 @@ setMethod("distinct", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) -#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements -#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#' collectRDD(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +#' collectRDD(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates #'} #' @rdname sampleRDD #' @aliases sampleRDD,RDD @@ -942,7 +942,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", fraction <- 0.0 total <- 0 multiplier <- 3.0 - initialCount <- count(x) + initialCount <- countRDD(x) maxSelected <- 0 MAXINT <- .Machine$integer.max @@ -964,7 +964,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", } set.seed(seed) - samples <- collect(sampleRDD(x, withReplacement, fraction, + samples <- collectRDD(sampleRDD(x, withReplacement, fraction, as.integer(ceiling(runif(1, -MAXINT, MAXINT))))) @@ -972,7 +972,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", # 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, + samples <- collectRDD(sampleRDD(x, withReplacement, fraction, as.integer(ceiling(runif(1, -MAXINT, MAXINT))))) @@ -990,7 +990,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", #'\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)) +#' collectRDD(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) #'} # nolint end #' @rdname keyBy @@ -1019,12 +1019,12 @@ setMethod("keyBy", #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) #' getNumPartitions(rdd) # 4 -#' getNumPartitions(repartition(rdd, 2L)) # 2 +#' getNumPartitions(repartitionRDD(rdd, 2L)) # 2 #'} #' @rdname repartition #' @aliases repartition,RDD #' @noRd -setMethod("repartition", +setMethod("repartitionRDD", signature(x = "RDD"), function(x, numPartitions) { if (!is.null(numPartitions) && is.numeric(numPartitions)) { @@ -1064,7 +1064,7 @@ setMethod("coalesce", }) } shuffled <- lapplyPartitionsWithIndex(x, func) - repartitioned <- partitionBy(shuffled, numPartitions) + repartitioned <- partitionByRDD(shuffled, numPartitions) values(repartitioned) } else { jrdd <- callJMethod(getJRDD(x), "coalesce", numPartitions, shuffle) @@ -1135,7 +1135,7 @@ setMethod("saveAsTextFile", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(3, 2, 1)) -#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#' collectRDD(sortBy(rdd, function(x) { x })) # list (1, 2, 3) #'} # nolint end #' @rdname sortBy @@ -1304,7 +1304,7 @@ setMethod("aggregateRDD", Reduce(seqOp, part, zeroValue) } - partitionList <- collect(lapplyPartition(x, partitionFunc), + partitionList <- collectRDD(lapplyPartition(x, partitionFunc), flatten = FALSE) Reduce(combOp, partitionList, zeroValue) }) @@ -1322,7 +1322,7 @@ setMethod("aggregateRDD", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) -#' collect(pipeRDD(rdd, "more") +#' pipeRDD(rdd, "more") #' Output: c("1", "2", ..., "10") #'} #' @aliases pipeRDD,RDD,character-method @@ -1397,7 +1397,7 @@ setMethod("setName", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithUniqueId(rdd)) +#' collectRDD(zipWithUniqueId(rdd)) #' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) #'} # nolint end @@ -1440,7 +1440,7 @@ setMethod("zipWithUniqueId", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -#' collect(zipWithIndex(rdd)) +#' collectRDD(zipWithIndex(rdd)) #' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) #'} # nolint end @@ -1452,7 +1452,7 @@ setMethod("zipWithIndex", function(x) { n <- getNumPartitions(x) if (n > 1) { - nums <- collect(lapplyPartition(x, + nums <- collectRDD(lapplyPartition(x, function(part) { list(length(part)) })) @@ -1488,7 +1488,7 @@ setMethod("zipWithIndex", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, as.list(1:4), 2L) -#' collect(glom(rdd)) +#' collectRDD(glom(rdd)) #' # list(list(1, 2), list(3, 4)) #'} # nolint end @@ -1556,7 +1556,7 @@ setMethod("unionRDD", #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, 0:4) #' rdd2 <- parallelize(sc, 1000:1004) -#' collect(zipRDD(rdd1, rdd2)) +#' collectRDD(zipRDD(rdd1, rdd2)) #' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) #'} # nolint end @@ -1628,7 +1628,7 @@ setMethod("cartesian", #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) #' rdd2 <- parallelize(sc, list(2, 4)) -#' collect(subtract(rdd1, rdd2)) +#' collectRDD(subtract(rdd1, rdd2)) #' # list(1, 1, 3) #'} # nolint end @@ -1662,7 +1662,7 @@ setMethod("subtract", #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) #' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) -#' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +#' collectRDD(sortBy(intersection(rdd1, rdd2), function(x) { x })) #' # list(1, 2, 3) #'} # nolint end @@ -1699,7 +1699,7 @@ setMethod("intersection", #' rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 #' rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 #' rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 -#' collect(zipPartitions(rdd1, rdd2, rdd3, +#' collectRDD(zipPartitions(rdd1, rdd2, rdd3, #' func = function(x, y, z) { list(list(x, y, z))} )) #' # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) #'} diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index a14bcd91b3eac..0c06bba639d9b 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -218,7 +218,7 @@ createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { } if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { - row <- first(rdd) + row <- firstRDD(rdd) names <- if (is.null(schema)) { names(row) } else { diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 2538bb25073e1..13ade49eabfa6 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -267,7 +267,7 @@ spark.lapply <- function(list, func) { sc <- getSparkContext() rdd <- parallelize(sc, list, length(list)) results <- map(rdd, func) - local <- collect(results) + local <- collectRDD(results) local } diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 10a09129ec927..52ab730e215c2 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -23,9 +23,7 @@ setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) -# @rdname cache-methods -# @export -setGeneric("cache", function(x) { standardGeneric("cache") }) +setGeneric("cacheRDD", function(x) { standardGeneric("cacheRDD") }) # @rdname coalesce # @seealso repartition @@ -36,9 +34,7 @@ setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coales # @export setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) -# @rdname collect-methods -# @export -setGeneric("collect", function(x, ...) { standardGeneric("collect") }) +setGeneric("collectRDD", function(x, ...) { standardGeneric("collectRDD") }) # @rdname collect-methods # @export @@ -51,9 +47,9 @@ setGeneric("collectPartition", standardGeneric("collectPartition") }) -# @rdname nrow -# @export -setGeneric("count", function(x) { standardGeneric("count") }) +setGeneric("countRDD", function(x) { standardGeneric("countRDD") }) + +setGeneric("lengthRDD", function(x) { standardGeneric("lengthRDD") }) # @rdname countByValue # @export @@ -74,17 +70,13 @@ setGeneric("approxQuantile", standardGeneric("approxQuantile") }) -# @rdname distinct -# @export -setGeneric("distinct", function(x, numPartitions = 1) { standardGeneric("distinct") }) +setGeneric("distinctRDD", function(x, numPartitions = 1) { standardGeneric("distinctRDD") }) # @rdname filterRDD # @export setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") }) -# @rdname first -# @export -setGeneric("first", function(x, ...) { standardGeneric("first") }) +setGeneric("firstRDD", function(x, ...) { standardGeneric("firstRDD") }) # @rdname flatMap # @export @@ -110,6 +102,8 @@ setGeneric("glom", function(x) { standardGeneric("glom") }) # @export setGeneric("histogram", function(df, col, nbins=10) { standardGeneric("histogram") }) +setGeneric("joinRDD", function(x, y, ...) { standardGeneric("joinRDD") }) + # @rdname keyBy # @export setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) @@ -152,9 +146,7 @@ setGeneric("getNumPartitions", function(x) { standardGeneric("getNumPartitions") # @export setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) -# @rdname persist -# @export -setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) +setGeneric("persistRDD", function(x, newLevel) { standardGeneric("persistRDD") }) # @rdname pipeRDD # @export @@ -168,10 +160,7 @@ setGeneric("pivot", function(x, colname, values = list()) { standardGeneric("piv # @export setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) -# @rdname repartition -# @seealso coalesce -# @export -setGeneric("repartition", function(x, ...) { standardGeneric("repartition") }) +setGeneric("repartitionRDD", function(x, ...) { standardGeneric("repartitionRDD") }) # @rdname sampleRDD # @export @@ -193,6 +182,8 @@ setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile # @export setGeneric("setName", function(x, name) { standardGeneric("setName") }) +setGeneric("showRDD", function(object, ...) { standardGeneric("showRDD") }) + # @rdname sortBy # @export setGeneric("sortBy", @@ -200,9 +191,7 @@ setGeneric("sortBy", standardGeneric("sortBy") }) -# @rdname take -# @export -setGeneric("take", function(x, num) { standardGeneric("take") }) +setGeneric("takeRDD", function(x, num) { standardGeneric("takeRDD") }) # @rdname takeOrdered # @export @@ -223,9 +212,7 @@ setGeneric("top", function(x, num) { standardGeneric("top") }) # @export setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) -# @rdname unpersist-methods -# @export -setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) +setGeneric("unpersistRDD", function(x, ...) { standardGeneric("unpersistRDD") }) # @rdname zipRDD # @export @@ -343,9 +330,7 @@ setGeneric("join", function(x, y, ...) { standardGeneric("join") }) # @export setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) -#' @rdname partitionBy -#' @export -setGeneric("partitionBy", function(x, ...) { standardGeneric("partitionBy") }) +setGeneric("partitionByRDD", function(x, ...) { standardGeneric("partitionByRDD") }) # @rdname reduceByKey # @seealso groupByKey @@ -414,6 +399,14 @@ setGeneric("as.data.frame", #' @export setGeneric("attach") +#' @rdname cache +#' @export +setGeneric("cache", function(x) { standardGeneric("cache") }) + +#' @rdname collect +#' @export +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) + #' @rdname columns #' @export setGeneric("colnames", function(x, do.NULL = TRUE, prefix = "col") { standardGeneric("colnames") }) @@ -434,6 +427,10 @@ setGeneric("coltypes<-", function(x, value) { standardGeneric("coltypes<-") }) #' @export setGeneric("columns", function(x) {standardGeneric("columns") }) +#' @rdname nrow +#' @export +setGeneric("count", function(x) { standardGeneric("count") }) + #' @rdname cov #' @export setGeneric("cov", function(x, ...) {standardGeneric("cov") }) @@ -477,6 +474,10 @@ setGeneric("gapplyCollect", function(x, ...) { standardGeneric("gapplyCollect") #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) +#' @rdname distinct +#' @export +setGeneric("distinct", function(x) { standardGeneric("distinct") }) + #' @rdname drop #' @export setGeneric("drop", function(x, ...) { standardGeneric("drop") }) @@ -519,6 +520,10 @@ setGeneric("fillna", function(x, value, cols = NULL) { standardGeneric("fillna") #' @export setGeneric("filter", function(x, condition) { standardGeneric("filter") }) +#' @rdname first +#' @export +setGeneric("first", function(x, ...) { standardGeneric("first") }) + #' @rdname groupBy #' @export setGeneric("group_by", function(x, ...) { standardGeneric("group_by") }) @@ -555,17 +560,25 @@ setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) #' @export setGeneric("orderBy", function(x, col, ...) { standardGeneric("orderBy") }) +#' @rdname persist +#' @export +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) + #' @rdname printSchema #' @export setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) +#' @rdname registerTempTable-deprecated +#' @export +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) + #' @rdname rename #' @export setGeneric("rename", function(x, ...) { standardGeneric("rename") }) -#' @rdname registerTempTable-deprecated +#' @rdname repartition #' @export -setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) +setGeneric("repartition", function(x, ...) { standardGeneric("repartition") }) #' @rdname sample #' @export @@ -592,6 +605,10 @@ setGeneric("saveAsTable", function(df, tableName, source = NULL, mode = "error", #' @export setGeneric("str") +#' @rdname take +#' @export +setGeneric("take", function(x, num) { standardGeneric("take") }) + #' @rdname mutate #' @export setGeneric("transform", function(`_data`, ...) {standardGeneric("transform") }) @@ -674,6 +691,10 @@ setGeneric("union", function(x, y) { standardGeneric("union") }) #' @export setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) +#' @rdname unpersist-methods +#' @export +setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) + #' @rdname filter #' @export setGeneric("where", function(x, condition) { standardGeneric("where") }) @@ -771,6 +792,10 @@ setGeneric("over", function(x, window) { standardGeneric("over") }) ###################### WindowSpec Methods ########################## +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, ...) { standardGeneric("partitionBy") }) + #' @rdname rowsBetween #' @export setGeneric("rowsBetween", function(x, start, end) { standardGeneric("rowsBetween") }) diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index d39775cabef88..f0605db1e9e83 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -49,7 +49,7 @@ setMethod("lookup", lapply(filtered, function(i) { i[[2]] }) } valsRDD <- lapplyPartition(x, partitionFunc) - collect(valsRDD) + collectRDD(valsRDD) }) #' Count the number of elements for each key, and return the result to the @@ -85,7 +85,7 @@ setMethod("countByKey", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(keys(rdd)) # list(1, 3) +#' collectRDD(keys(rdd)) # list(1, 3) #'} # nolint end #' @rdname keys @@ -108,7 +108,7 @@ setMethod("keys", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -#' collect(values(rdd)) # list(2, 4) +#' collectRDD(values(rdd)) # list(2, 4) #'} # nolint end #' @rdname values @@ -135,7 +135,7 @@ setMethod("values", #' sc <- sparkR.init() #' rdd <- parallelize(sc, 1:10) #' makePairs <- lapply(rdd, function(x) { list(x, x) }) -#' collect(mapValues(makePairs, function(x) { x * 2) }) +#' collectRDD(mapValues(makePairs, function(x) { x * 2) }) #' Output: list(list(1,2), list(2,4), list(3,6), ...) #'} #' @rdname mapValues @@ -162,7 +162,7 @@ setMethod("mapValues", #'\dontrun{ #' sc <- sparkR.init() #' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) -#' collect(flatMapValues(rdd, function(x) { x })) +#' collectRDD(flatMapValues(rdd, function(x) { x })) #' Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) #'} #' @rdname flatMapValues @@ -198,13 +198,13 @@ setMethod("flatMapValues", #' sc <- sparkR.init() #' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) #' rdd <- parallelize(sc, pairs) -#' parts <- partitionBy(rdd, 2L) +#' parts <- partitionByRDD(rdd, 2L) #' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) #'} #' @rdname partitionBy #' @aliases partitionBy,RDD,integer-method #' @noRd -setMethod("partitionBy", +setMethod("partitionByRDD", signature(x = "RDD"), function(x, numPartitions, partitionFunc = hashCode) { stopifnot(is.numeric(numPartitions)) @@ -261,7 +261,7 @@ setMethod("partitionBy", #' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) #' rdd <- parallelize(sc, pairs) #' parts <- groupByKey(rdd, 2L) -#' grouped <- collect(parts) +#' grouped <- collectRDD(parts) #' grouped[[1]] # Should be a list(1, list(2, 4)) #'} #' @rdname groupByKey @@ -270,7 +270,7 @@ setMethod("partitionBy", setMethod("groupByKey", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions) { - shuffled <- partitionBy(x, numPartitions) + shuffled <- partitionByRDD(x, numPartitions) groupVals <- function(part) { vals <- new.env() keys <- new.env() @@ -321,7 +321,7 @@ setMethod("groupByKey", #' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) #' rdd <- parallelize(sc, pairs) #' parts <- reduceByKey(rdd, "+", 2L) -#' reduced <- collect(parts) +#' reduced <- collectRDD(parts) #' reduced[[1]] # Should be a list(1, 6) #'} #' @rdname reduceByKey @@ -342,7 +342,7 @@ setMethod("reduceByKey", convertEnvsToList(keys, vals) } locallyReduced <- lapplyPartition(x, reduceVals) - shuffled <- partitionBy(locallyReduced, numToInt(numPartitions)) + shuffled <- partitionByRDD(locallyReduced, numToInt(numPartitions)) lapplyPartition(shuffled, reduceVals) }) @@ -430,7 +430,7 @@ setMethod("reduceByKeyLocally", #' 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 <- collectRDD(parts) #' combined[[1]] # Should be a list(1, 6) #'} # nolint end @@ -453,7 +453,7 @@ setMethod("combineByKey", convertEnvsToList(keys, combiners) } locallyCombined <- lapplyPartition(x, combineLocally) - shuffled <- partitionBy(locallyCombined, numToInt(numPartitions)) + shuffled <- partitionByRDD(locallyCombined, numToInt(numPartitions)) mergeAfterShuffle <- function(part) { combiners <- new.env() keys <- new.env() @@ -563,13 +563,13 @@ setMethod("foldByKey", #' 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)) +#' joinRDD(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) #'} # nolint end #' @rdname join-methods #' @aliases join,RDD,RDD-method #' @noRd -setMethod("join", +setMethod("joinRDD", signature(x = "RDD", y = "RDD"), function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) @@ -772,7 +772,7 @@ setMethod("cogroup", #'\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)) +#' collectRDD(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) #'} # nolint end #' @rdname sortByKey @@ -784,12 +784,12 @@ setMethod("sortByKey", rangeBounds <- list() if (numPartitions > 1) { - rddSize <- count(x) + rddSize <- countRDD(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))) + samples <- collectRDD(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) @@ -822,7 +822,7 @@ setMethod("sortByKey", sortKeyValueList(part, decreasing = !ascending) } - newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) + newRDD <- partitionByRDD(x, numPartitions, rangePartitionFunc) lapplyPartition(newRDD, partitionFunc) }) @@ -841,7 +841,7 @@ setMethod("sortByKey", #' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), #' list("b", 5), list("a", 2))) #' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) -#' collect(subtractByKey(rdd1, rdd2)) +#' collectRDD(subtractByKey(rdd1, rdd2)) #' # list(list("b", 4), list("b", 5)) #'} # nolint end diff --git a/R/pkg/inst/tests/testthat/test_binaryFile.R b/R/pkg/inst/tests/testthat/test_binaryFile.R index b69f017de81d1..f7a0510711da9 100644 --- a/R/pkg/inst/tests/testthat/test_binaryFile.R +++ b/R/pkg/inst/tests/testthat/test_binaryFile.R @@ -31,7 +31,7 @@ test_that("saveAsObjectFile()/objectFile() following textFile() works", { rdd <- textFile(sc, fileName1, 1) saveAsObjectFile(rdd, fileName2) rdd <- objectFile(sc, fileName2) - expect_equal(collect(rdd), as.list(mockFile)) + expect_equal(collectRDD(rdd), as.list(mockFile)) unlink(fileName1) unlink(fileName2, recursive = TRUE) @@ -44,7 +44,7 @@ test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { rdd <- parallelize(sc, l, 1) saveAsObjectFile(rdd, fileName) rdd <- objectFile(sc, fileName) - expect_equal(collect(rdd), l) + expect_equal(collectRDD(rdd), l) unlink(fileName, recursive = TRUE) }) @@ -64,7 +64,7 @@ test_that("saveAsObjectFile()/objectFile() following RDD transformations works", saveAsObjectFile(counts, fileName2) counts <- objectFile(sc, fileName2) - output <- collect(counts) + output <- collectRDD(counts) expected <- list(list("awesome.", 1), list("Spark", 2), list("pretty.", 1), list("is", 2)) expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) @@ -83,7 +83,7 @@ test_that("saveAsObjectFile()/objectFile() works with multiple paths", { saveAsObjectFile(rdd2, fileName2) rdd <- objectFile(sc, c(fileName1, fileName2)) - expect_equal(count(rdd), 2) + expect_equal(countRDD(rdd), 2) unlink(fileName1, recursive = TRUE) unlink(fileName2, recursive = TRUE) diff --git a/R/pkg/inst/tests/testthat/test_binary_function.R b/R/pkg/inst/tests/testthat/test_binary_function.R index 6f51d20687277..b780b9458545c 100644 --- a/R/pkg/inst/tests/testthat/test_binary_function.R +++ b/R/pkg/inst/tests/testthat/test_binary_function.R @@ -29,7 +29,7 @@ rdd <- parallelize(sc, nums, 2L) mockFile <- c("Spark is pretty.", "Spark is awesome.") test_that("union on two RDDs", { - actual <- collect(unionRDD(rdd, rdd)) + actual <- collectRDD(unionRDD(rdd, rdd)) expect_equal(actual, as.list(rep(nums, 2))) fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") @@ -37,13 +37,13 @@ test_that("union on two RDDs", { text.rdd <- textFile(sc, fileName) union.rdd <- unionRDD(rdd, text.rdd) - actual <- collect(union.rdd) + actual <- collectRDD(union.rdd) expect_equal(actual, c(as.list(nums), mockFile)) expect_equal(getSerializedMode(union.rdd), "byte") rdd <- map(text.rdd, function(x) {x}) union.rdd <- unionRDD(rdd, text.rdd) - actual <- collect(union.rdd) + actual <- collectRDD(union.rdd) expect_equal(actual, as.list(c(mockFile, mockFile))) expect_equal(getSerializedMode(union.rdd), "byte") @@ -54,14 +54,14 @@ 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) + actual <- collectRDD(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) + actual <- collectRDD(cogroup.rdd) expected <- list(list("b", list(list(), list(2))), list("a", list(list(1, 4), list(3)))) expect_equal(sortKeyValueList(actual), @@ -72,7 +72,7 @@ test_that("zipPartitions() on RDDs", { rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 - actual <- collect(zipPartitions(rdd1, rdd2, rdd3, + actual <- collectRDD(zipPartitions(rdd1, rdd2, rdd3, func = function(x, y, z) { list(list(x, y, z))} )) expect_equal(actual, list(list(1, c(1, 2), c(1, 2, 3)), list(2, c(3, 4), c(4, 5, 6)))) @@ -82,19 +82,19 @@ test_that("zipPartitions() on RDDs", { writeLines(mockFile, fileName) rdd <- textFile(sc, fileName, 1) - actual <- collect(zipPartitions(rdd, rdd, + actual <- collectRDD(zipPartitions(rdd, rdd, func = function(x, y) { list(paste(x, y, sep = "\n")) })) expected <- list(paste(mockFile, mockFile, sep = "\n")) expect_equal(actual, expected) rdd1 <- parallelize(sc, 0:1, 1) - actual <- collect(zipPartitions(rdd1, rdd, + actual <- collectRDD(zipPartitions(rdd1, rdd, func = function(x, y) { list(x + nchar(y)) })) expected <- list(0:1 + nchar(mockFile)) expect_equal(actual, expected) rdd <- map(rdd, function(x) { x }) - actual <- collect(zipPartitions(rdd, rdd1, + actual <- collectRDD(zipPartitions(rdd, rdd1, func = function(x, y) { list(y + nchar(x)) })) expect_equal(actual, expected) diff --git a/R/pkg/inst/tests/testthat/test_broadcast.R b/R/pkg/inst/tests/testthat/test_broadcast.R index cf1d43277105e..064249a57aed4 100644 --- a/R/pkg/inst/tests/testthat/test_broadcast.R +++ b/R/pkg/inst/tests/testthat/test_broadcast.R @@ -32,7 +32,7 @@ test_that("using broadcast variable", { useBroadcast <- function(x) { sum(SparkR:::value(randomMatBr) * x) } - actual <- collect(lapply(rrdd, useBroadcast)) + actual <- collectRDD(lapply(rrdd, useBroadcast)) expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) expect_equal(actual, expected) }) @@ -43,7 +43,7 @@ test_that("without using broadcast variable", { useBroadcast <- function(x) { sum(randomMat * x) } - actual <- collect(lapply(rrdd, useBroadcast)) + actual <- collectRDD(lapply(rrdd, useBroadcast)) expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) expect_equal(actual, expected) }) diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 2a1bd61b11118..66640c4b08459 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -58,7 +58,7 @@ test_that("repeatedly starting and stopping SparkR", { for (i in 1:4) { sc <- suppressWarnings(sparkR.init()) rdd <- parallelize(sc, 1:20, 2L) - expect_equal(count(rdd), 20) + expect_equal(countRDD(rdd), 20) suppressWarnings(sparkR.stop()) } }) @@ -94,8 +94,9 @@ test_that("rdd GC across sparkR.stop", { rm(rdd2) gc() - count(rdd3) - count(rdd4) + countRDD(rdd3) + countRDD(rdd4) + sparkR.session.stop() }) test_that("job group functions can be called", { diff --git a/R/pkg/inst/tests/testthat/test_includePackage.R b/R/pkg/inst/tests/testthat/test_includePackage.R index d6a3766539c02..025eb9b9fc9d6 100644 --- a/R/pkg/inst/tests/testthat/test_includePackage.R +++ b/R/pkg/inst/tests/testthat/test_includePackage.R @@ -37,7 +37,7 @@ test_that("include inside function", { } data <- lapplyPartition(rdd, generateData) - actual <- collect(data) + actual <- collectRDD(data) } }) @@ -53,6 +53,6 @@ test_that("use include package", { includePackage(sc, plyr) data <- lapplyPartition(rdd, generateData) - actual <- collect(data) + actual <- collectRDD(data) } }) diff --git a/R/pkg/inst/tests/testthat/test_parallelize_collect.R b/R/pkg/inst/tests/testthat/test_parallelize_collect.R index f79a8a70aafb1..1b230554f7a0e 100644 --- a/R/pkg/inst/tests/testthat/test_parallelize_collect.R +++ b/R/pkg/inst/tests/testthat/test_parallelize_collect.R @@ -67,22 +67,22 @@ test_that("parallelize() on simple vectors and lists returns an RDD", { test_that("collect(), following a parallelize(), gives back the original collections", { numVectorRDD <- parallelize(jsc, numVector, 10) - expect_equal(collect(numVectorRDD), as.list(numVector)) + expect_equal(collectRDD(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)) + expect_equal(collectRDD(numListRDD), as.list(numList)) + expect_equal(collectRDD(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)) + expect_equal(collectRDD(strVectorRDD), as.list(strVector)) + expect_equal(collectRDD(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)) + expect_equal(collectRDD(strListRDD), as.list(strList)) + expect_equal(collectRDD(strListRDD2), as.list(strList)) }) test_that("regression: collect() following a parallelize() does not drop elements", { @@ -90,7 +90,7 @@ test_that("regression: collect() following a parallelize() does not drop element collLen <- 10 numPart <- 6 expected <- runif(collLen) - actual <- collect(parallelize(jsc, expected, numPart)) + actual <- collectRDD(parallelize(jsc, expected, numPart)) expect_equal(actual, as.list(expected)) }) @@ -99,12 +99,12 @@ test_that("parallelize() and collect() work for lists of pairs (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) + expect_equal(collectRDD(numPairsRDDD1), numPairs) + expect_equal(collectRDD(numPairsRDDD2), numPairs) + expect_equal(collectRDD(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) + expect_equal(collectRDD(strPairsRDDD1), strPairs) + expect_equal(collectRDD(strPairsRDDD2), strPairs) }) diff --git a/R/pkg/inst/tests/testthat/test_rdd.R b/R/pkg/inst/tests/testthat/test_rdd.R index 429311d2924f0..d38a763bab8c6 100644 --- a/R/pkg/inst/tests/testthat/test_rdd.R +++ b/R/pkg/inst/tests/testthat/test_rdd.R @@ -34,14 +34,14 @@ test_that("get number of partitions in RDD", { }) test_that("first on RDD", { - expect_equal(first(rdd), 1) + expect_equal(firstRDD(rdd), 1) newrdd <- lapply(rdd, function(x) x + 1) - expect_equal(first(newrdd), 2) + expect_equal(firstRDD(newrdd), 2) }) test_that("count and length on RDD", { - expect_equal(count(rdd), 10) - expect_equal(length(rdd), 10) + expect_equal(countRDD(rdd), 10) + expect_equal(lengthRDD(rdd), 10) }) test_that("count by values and keys", { @@ -57,40 +57,40 @@ test_that("count by values and keys", { test_that("lapply on RDD", { multiples <- lapply(rdd, function(x) { 2 * x }) - actual <- collect(multiples) + actual <- collectRDD(multiples) expect_equal(actual, as.list(nums * 2)) }) test_that("lapplyPartition on RDD", { sums <- lapplyPartition(rdd, function(part) { sum(unlist(part)) }) - actual <- collect(sums) + actual <- collectRDD(sums) expect_equal(actual, list(15, 40)) }) test_that("mapPartitions on RDD", { sums <- mapPartitions(rdd, function(part) { sum(unlist(part)) }) - actual <- collect(sums) + actual <- collectRDD(sums) expect_equal(actual, list(15, 40)) }) test_that("flatMap() on RDDs", { flat <- flatMap(intRdd, function(x) { list(x, x) }) - actual <- collect(flat) + actual <- collectRDD(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) + actual <- collectRDD(filtered.rdd) expect_equal(actual, list(2, 4, 6, 8, 10)) filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) - actual <- collect(filtered.rdd) + actual <- collectRDD(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) + actual <- collectRDD(filtered.rdd) expect_equal(actual, list()) }) @@ -110,7 +110,7 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { part <- as.list(unlist(part) * partIndex + i) }) rdd2 <- lapply(rdd2, function(x) x + x) - actual <- collect(rdd2) + actual <- collectRDD(rdd2) expected <- list(24, 24, 24, 24, 24, 168, 170, 172, 174, 176) expect_equal(actual, expected) @@ -126,20 +126,20 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp part <- as.list(unlist(part) * partIndex) }) - cache(rdd2) + cacheRDD(rdd2) expect_true(rdd2@env$isCached) rdd2 <- lapply(rdd2, function(x) x) expect_false(rdd2@env$isCached) - unpersist(rdd2) + unpersistRDD(rdd2) expect_false(rdd2@env$isCached) - persist(rdd2, "MEMORY_AND_DISK") + persistRDD(rdd2, "MEMORY_AND_DISK") expect_true(rdd2@env$isCached) rdd2 <- lapply(rdd2, function(x) x) expect_false(rdd2@env$isCached) - unpersist(rdd2) + unpersistRDD(rdd2) expect_false(rdd2@env$isCached) tempDir <- tempfile(pattern = "checkpoint") @@ -152,7 +152,7 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp expect_false(rdd2@env$isCheckpointed) # make sure the data is collectable - collect(rdd2) + collectRDD(rdd2) unlink(tempDir) }) @@ -169,21 +169,21 @@ test_that("reduce on RDD", { test_that("lapply with dependency", { fa <- 5 multiples <- lapply(rdd, function(x) { fa * x }) - actual <- collect(multiples) + actual <- collectRDD(multiples) expect_equal(actual, as.list(nums * 5)) }) test_that("lapplyPartitionsWithIndex on RDDs", { func <- function(partIndex, part) { list(partIndex, Reduce("+", part)) } - actual <- collect(lapplyPartitionsWithIndex(rdd, func), flatten = FALSE) + actual <- collectRDD(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(partIndex, part) { list(partIndex, part) } - actual <- collect(lapplyPartitionsWithIndex( - partitionBy(pairsRDD, 2L, partitionByParity), + actual <- collectRDD(lapplyPartitionsWithIndex( + partitionByRDD(pairsRDD, 2L, partitionByParity), mkTup), FALSE) expect_equal(actual, list(list(0, list(list(1, 2), list(3, 4))), @@ -191,7 +191,7 @@ test_that("lapplyPartitionsWithIndex on RDDs", { }) test_that("sampleRDD() on RDDs", { - expect_equal(unlist(collect(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums) + expect_equal(unlist(collectRDD(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums) }) test_that("takeSample() on RDDs", { @@ -238,7 +238,7 @@ test_that("takeSample() on RDDs", { test_that("mapValues() on pairwise RDDs", { multiples <- mapValues(intRdd, function(x) { x * 2 }) - actual <- collect(multiples) + actual <- collectRDD(multiples) expected <- lapply(intPairs, function(x) { list(x[[1]], x[[2]] * 2) }) @@ -247,11 +247,11 @@ test_that("mapValues() on pairwise RDDs", { 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 })) + actual <- collectRDD(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) })) + actual <- collectRDD(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))) @@ -273,8 +273,8 @@ test_that("reduceByKeyLocally() on PairwiseRDDs", { 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))) + uniques <- distinctRDD(rdd.rep2) + actual <- sort(unlist(collectRDD(uniques))) expect_equal(actual, nums) }) @@ -296,7 +296,7 @@ test_that("sumRDD() on RDDs", { test_that("keyBy on RDDs", { func <- function(x) { x * x } keys <- keyBy(rdd, func) - actual <- collect(keys) + actual <- collectRDD(keys) expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) }) @@ -304,12 +304,12 @@ test_that("repartition/coalesce on RDDs", { rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements # repartition - r1 <- repartition(rdd, 2) + r1 <- repartitionRDD(rdd, 2) expect_equal(getNumPartitions(r1), 2L) count <- length(collectPartition(r1, 0L)) expect_true(count >= 8 && count <= 12) - r2 <- repartition(rdd, 6) + r2 <- repartitionRDD(rdd, 6) expect_equal(getNumPartitions(r2), 6L) count <- length(collectPartition(r2, 0L)) expect_true(count >= 0 && count <= 4) @@ -323,12 +323,12 @@ test_that("repartition/coalesce on RDDs", { test_that("sortBy() on RDDs", { sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE) - actual <- collect(sortedRdd) + actual <- collectRDD(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) + actual <- collectRDD(sortedRdd2) expect_equal(actual, as.list(nums)) }) @@ -380,13 +380,13 @@ test_that("aggregateRDD() on RDDs", { test_that("zipWithUniqueId() on RDDs", { rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) - actual <- collect(zipWithUniqueId(rdd)) + actual <- collectRDD(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)) + actual <- collectRDD(zipWithUniqueId(rdd)) expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) @@ -394,13 +394,13 @@ test_that("zipWithUniqueId() on RDDs", { test_that("zipWithIndex() on RDDs", { rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) - actual <- collect(zipWithIndex(rdd)) + actual <- collectRDD(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)) + actual <- collectRDD(zipWithIndex(rdd)) expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) @@ -408,35 +408,35 @@ test_that("zipWithIndex() on RDDs", { test_that("glom() on RDD", { rdd <- parallelize(sc, as.list(1:4), 2L) - actual <- collect(glom(rdd)) + actual <- collectRDD(glom(rdd)) expect_equal(actual, list(list(1, 2), list(3, 4))) }) test_that("keys() on RDDs", { keys <- keys(intRdd) - actual <- collect(keys) + actual <- collectRDD(keys) expect_equal(actual, lapply(intPairs, function(x) { x[[1]] })) }) test_that("values() on RDDs", { values <- values(intRdd) - actual <- collect(values) + actual <- collectRDD(values) expect_equal(actual, lapply(intPairs, function(x) { x[[2]] })) }) test_that("pipeRDD() on RDDs", { - actual <- collect(pipeRDD(rdd, "more")) + actual <- collectRDD(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")) + actual <- collectRDD(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")) + actual <- collectRDD(pipeRDD(rev.rdd, "sort")) expected <- as.list(as.character(c(5:9, 0:4))) expect_equal(actual, expected) }) @@ -444,7 +444,7 @@ test_that("pipeRDD() on RDDs", { test_that("zipRDD() on RDDs", { rdd1 <- parallelize(sc, 0:4, 2) rdd2 <- parallelize(sc, 1000:1004, 2) - actual <- collect(zipRDD(rdd1, rdd2)) + actual <- collectRDD(zipRDD(rdd1, rdd2)) expect_equal(actual, list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) @@ -453,17 +453,17 @@ test_that("zipRDD() on RDDs", { writeLines(mockFile, fileName) rdd <- textFile(sc, fileName, 1) - actual <- collect(zipRDD(rdd, rdd)) + actual <- collectRDD(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)) + actual <- collectRDD(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)) + actual <- collectRDD(zipRDD(rdd, rdd1)) expected <- lapply(mockFile, function(x) { list(x, x) }) expect_equal(actual, expected) @@ -472,7 +472,7 @@ test_that("zipRDD() on RDDs", { test_that("cartesian() on RDDs", { rdd <- parallelize(sc, 1:3) - actual <- collect(cartesian(rdd, rdd)) + actual <- collectRDD(cartesian(rdd, rdd)) expect_equal(sortKeyValueList(actual), list( list(1, 1), list(1, 2), list(1, 3), @@ -481,7 +481,7 @@ test_that("cartesian() on RDDs", { # test case where one RDD is empty emptyRdd <- parallelize(sc, list()) - actual <- collect(cartesian(rdd, emptyRdd)) + actual <- collectRDD(cartesian(rdd, emptyRdd)) expect_equal(actual, list()) mockFile <- c("Spark is pretty.", "Spark is awesome.") @@ -489,7 +489,7 @@ test_that("cartesian() on RDDs", { writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) - actual <- collect(cartesian(rdd, rdd)) + actual <- collectRDD(cartesian(rdd, rdd)) expected <- list( list("Spark is awesome.", "Spark is pretty."), list("Spark is awesome.", "Spark is awesome."), @@ -498,7 +498,7 @@ test_that("cartesian() on RDDs", { expect_equal(sortKeyValueList(actual), expected) rdd1 <- parallelize(sc, 0:1) - actual <- collect(cartesian(rdd1, rdd)) + actual <- collectRDD(cartesian(rdd1, rdd)) expect_equal(sortKeyValueList(actual), list( list(0, "Spark is pretty."), @@ -507,7 +507,7 @@ test_that("cartesian() on RDDs", { list(1, "Spark is awesome."))) rdd1 <- map(rdd, function(x) { x }) - actual <- collect(cartesian(rdd, rdd1)) + actual <- collectRDD(cartesian(rdd, rdd1)) expect_equal(sortKeyValueList(actual), expected) unlink(fileName) @@ -518,24 +518,24 @@ test_that("subtract() on RDDs", { rdd1 <- parallelize(sc, l) # subtract by itself - actual <- collect(subtract(rdd1, rdd1)) + actual <- collectRDD(subtract(rdd1, rdd1)) expect_equal(actual, list()) # subtract by an empty RDD rdd2 <- parallelize(sc, list()) - actual <- collect(subtract(rdd1, rdd2)) + actual <- collectRDD(subtract(rdd1, rdd2)) expect_equal(as.list(sort(as.vector(actual, mode = "integer"))), l) rdd2 <- parallelize(sc, list(2, 4)) - actual <- collect(subtract(rdd1, rdd2)) + actual <- collectRDD(subtract(rdd1, rdd2)) expect_equal(as.list(sort(as.vector(actual, mode = "integer"))), list(1, 1, 3)) l <- list("a", "a", "b", "b", "c", "d") rdd1 <- parallelize(sc, l) rdd2 <- parallelize(sc, list("b", "d")) - actual <- collect(subtract(rdd1, rdd2)) + actual <- collectRDD(subtract(rdd1, rdd2)) expect_equal(as.list(sort(as.vector(actual, mode = "character"))), list("a", "a", "c")) }) @@ -546,17 +546,17 @@ test_that("subtractByKey() on pairwise RDDs", { rdd1 <- parallelize(sc, l) # subtractByKey by itself - actual <- collect(subtractByKey(rdd1, rdd1)) + actual <- collectRDD(subtractByKey(rdd1, rdd1)) expect_equal(actual, list()) # subtractByKey by an empty RDD rdd2 <- parallelize(sc, list()) - actual <- collect(subtractByKey(rdd1, rdd2)) + actual <- collectRDD(subtractByKey(rdd1, rdd2)) expect_equal(sortKeyValueList(actual), sortKeyValueList(l)) rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) - actual <- collect(subtractByKey(rdd1, rdd2)) + actual <- collectRDD(subtractByKey(rdd1, rdd2)) expect_equal(actual, list(list("b", 4), list("b", 5))) @@ -564,76 +564,76 @@ test_that("subtractByKey() on pairwise RDDs", { list(2, 5), list(1, 2)) rdd1 <- parallelize(sc, l) rdd2 <- parallelize(sc, list(list(1, 3), list(3, 1))) - actual <- collect(subtractByKey(rdd1, rdd2)) + actual <- collectRDD(subtractByKey(rdd1, rdd2)) expect_equal(actual, list(list(2, 4), list(2, 5))) }) test_that("intersection() on RDDs", { # intersection with self - actual <- collect(intersection(rdd, rdd)) + actual <- collectRDD(intersection(rdd, rdd)) expect_equal(sort(as.integer(actual)), nums) # intersection with an empty RDD emptyRdd <- parallelize(sc, list()) - actual <- collect(intersection(rdd, emptyRdd)) + actual <- collectRDD(intersection(rdd, emptyRdd)) expect_equal(actual, list()) rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) - actual <- collect(intersection(rdd1, rdd2)) + actual <- collectRDD(intersection(rdd1, rdd2)) expect_equal(sort(as.integer(actual)), 1:3) }) 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)) + actual <- collectRDD(joinRDD(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)) + actual <- collectRDD(joinRDD(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)) + actual <- collectRDD(joinRDD(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)) + actual <- collectRDD(joinRDD(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)) + actual <- collectRDD(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)) + actual <- collectRDD(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)) + actual <- collectRDD(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)) + actual <- collectRDD(leftOuterJoin(rdd1, rdd2, 2L)) expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -642,26 +642,26 @@ test_that("leftOuterJoin() on pairwise RDDs", { 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)) + actual <- collectRDD(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)) + actual <- collectRDD(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)) + actual <- collectRDD(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)) + actual <- collectRDD(rightOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3))))) }) @@ -669,14 +669,14 @@ test_that("rightOuterJoin() on pairwise RDDs", { 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)) + actual <- collectRDD(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)) + actual <- collectRDD(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), @@ -684,14 +684,14 @@ test_that("fullOuterJoin() on pairwise RDDs", { 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)) + actual <- collectRDD(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)) + actual <- collectRDD(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))))) @@ -700,21 +700,21 @@ test_that("fullOuterJoin() on pairwise RDDs", { test_that("sortByKey() on pairwise RDDs", { numPairsRdd <- map(rdd, function(x) { list (x, x) }) sortedRdd <- sortByKey(numPairsRdd, ascending = FALSE) - actual <- collect(sortedRdd) + actual <- collectRDD(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) + actual <- collectRDD(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) + actual <- collectRDD(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 @@ -722,27 +722,27 @@ test_that("sortByKey() on pairwise RDDs", { # boundary case 1: the RDD to be sorted has only 1 partition rdd4 <- parallelize(sc, l, 1L) sortedRdd4 <- sortByKey(rdd4) - actual <- collect(sortedRdd4) + actual <- collectRDD(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) + actual <- collectRDD(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) + actual <- collectRDD(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) + actual <- collectRDD(sortedRdd7) expect_equal(actual, l3) }) @@ -766,7 +766,7 @@ test_that("collectAsMap() on a pairwise RDD", { test_that("show()", { rdd <- parallelize(sc, list(1:10)) - expect_output(show(rdd), "ParallelCollectionRDD\\[\\d+\\] at parallelize at RRDD\\.scala:\\d+") + expect_output(showRDD(rdd), "ParallelCollectionRDD\\[\\d+\\] at parallelize at RRDD\\.scala:\\d+") }) test_that("sampleByKey() on pairwise RDDs", { diff --git a/R/pkg/inst/tests/testthat/test_shuffle.R b/R/pkg/inst/tests/testthat/test_shuffle.R index 7d4f342016441..07f3b02df6649 100644 --- a/R/pkg/inst/tests/testthat/test_shuffle.R +++ b/R/pkg/inst/tests/testthat/test_shuffle.R @@ -39,7 +39,7 @@ strListRDD <- parallelize(sc, strList, 4) test_that("groupByKey for integers", { grouped <- groupByKey(intRdd, 2L) - actual <- collect(grouped) + actual <- collectRDD(grouped) expected <- list(list(2L, list(100, 1)), list(1L, list(-1, 200))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -48,7 +48,7 @@ test_that("groupByKey for integers", { test_that("groupByKey for doubles", { grouped <- groupByKey(doubleRdd, 2L) - actual <- collect(grouped) + actual <- collectRDD(grouped) expected <- list(list(1.5, list(-1, 200)), list(2.5, list(100, 1))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -57,7 +57,7 @@ test_that("groupByKey for doubles", { test_that("reduceByKey for ints", { reduced <- reduceByKey(intRdd, "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(2L, 101), list(1L, 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -65,7 +65,7 @@ test_that("reduceByKey for ints", { test_that("reduceByKey for doubles", { reduced <- reduceByKey(doubleRdd, "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(1.5, 199), list(2.5, 101)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -74,7 +74,7 @@ test_that("reduceByKey for doubles", { test_that("combineByKey for ints", { reduced <- combineByKey(intRdd, function(x) { x }, "+", "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(2L, 101), list(1L, 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -82,7 +82,7 @@ test_that("combineByKey for ints", { test_that("combineByKey for doubles", { reduced <- combineByKey(doubleRdd, function(x) { x }, "+", "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(1.5, 199), list(2.5, 101)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -94,7 +94,7 @@ test_that("combineByKey for characters", { list("other", 3L), list("max", 4L)), 2L) reduced <- combineByKey(stringKeyRDD, function(x) { x }, "+", "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list("max", 5L), list("min", 2L), list("other", 3L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -109,7 +109,7 @@ test_that("aggregateByKey", { combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) - actual <- collect(aggregatedRDD) + actual <- collectRDD(aggregatedRDD) expected <- list(list(1, list(3, 2)), list(2, list(7, 2))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -122,7 +122,7 @@ test_that("aggregateByKey", { combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) - actual <- collect(aggregatedRDD) + actual <- collectRDD(aggregatedRDD) expected <- list(list("a", list(3, 2)), list("b", list(7, 2))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -132,7 +132,7 @@ test_that("foldByKey", { # test foldByKey for int keys folded <- foldByKey(intRdd, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list(list(2L, 101), list(1L, 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -140,7 +140,7 @@ test_that("foldByKey", { # test foldByKey for double keys folded <- foldByKey(doubleRdd, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list(list(1.5, 199), list(2.5, 101)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -151,7 +151,7 @@ test_that("foldByKey", { stringKeyRDD <- parallelize(sc, stringKeyPairs) folded <- foldByKey(stringKeyRDD, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list(list("b", 101), list("a", 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -159,14 +159,14 @@ test_that("foldByKey", { # test foldByKey for empty pair RDD rdd <- parallelize(sc, list()) folded <- foldByKey(rdd, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(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) + actual <- collectRDD(folded) expected <- list(list(1, 1)) expect_equal(actual, expected) }) @@ -175,7 +175,7 @@ test_that("partitionBy() partitions data correctly", { # Partition by magnitude partitionByMagnitude <- function(key) { if (key >= 3) 1 else 0 } - resultRDD <- partitionBy(numPairsRdd, 2L, partitionByMagnitude) + resultRDD <- partitionByRDD(numPairsRdd, 2L, partitionByMagnitude) expected_first <- list(list(1, 100), list(2, 200)) # key less than 3 expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key greater than or equal 3 @@ -191,7 +191,7 @@ test_that("partitionBy works with dependencies", { partitionByParity <- function(key) { if (key %% 2 == kOne) 7 else 4 } # Partition by parity - resultRDD <- partitionBy(numPairsRdd, numPartitions = 2L, partitionByParity) + resultRDD <- partitionByRDD(numPairsRdd, numPartitions = 2L, partitionByParity) # keys even; 100 %% 2 == 0 expected_first <- list(list(2, 200), list(4, -1)) @@ -208,7 +208,7 @@ 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) + resultRDD <- partitionByRDD(wordCount, 2L) expected_first <- list(list("Dexter", 1), list("Dexter", 1)) expected_second <- list(list("and", 1), list("and", 1)) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 7e59fdf4620e1..0aea89ddcb076 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -490,7 +490,7 @@ test_that("read/write json files", { test_that("jsonRDD() on a RDD with json string", { sqlContext <- suppressWarnings(sparkRSQL.init(sc)) rdd <- parallelize(sc, mockLines) - expect_equal(count(rdd), 3) + expect_equal(countRDD(rdd), 3) df <- suppressWarnings(jsonRDD(sqlContext, rdd)) expect_is(df, "SparkDataFrame") expect_equal(count(df), 3) @@ -582,7 +582,7 @@ test_that("toRDD() returns an RRDD", { df <- read.json(jsonPath) testRDD <- toRDD(df) expect_is(testRDD, "RDD") - expect_equal(count(testRDD), 3) + expect_equal(countRDD(testRDD), 3) }) test_that("union on two RDDs created from DataFrames returns an RRDD", { @@ -592,7 +592,7 @@ test_that("union on two RDDs created from DataFrames returns an RRDD", { unioned <- unionRDD(RDD1, RDD2) expect_is(unioned, "RDD") expect_equal(getSerializedMode(unioned), "byte") - expect_equal(collect(unioned)[[2]]$name, "Andy") + expect_equal(collectRDD(unioned)[[2]]$name, "Andy") }) test_that("union on mixed serialization types correctly returns a byte RRDD", { @@ -614,14 +614,14 @@ test_that("union on mixed serialization types correctly returns a byte RRDD", { unionByte <- unionRDD(rdd, dfRDD) expect_is(unionByte, "RDD") expect_equal(getSerializedMode(unionByte), "byte") - expect_equal(collect(unionByte)[[1]], 1) - expect_equal(collect(unionByte)[[12]]$name, "Andy") + expect_equal(collectRDD(unionByte)[[1]], 1) + expect_equal(collectRDD(unionByte)[[12]]$name, "Andy") unionString <- unionRDD(textRDD, dfRDD) expect_is(unionString, "RDD") expect_equal(getSerializedMode(unionString), "byte") - expect_equal(collect(unionString)[[1]], "Michael") - expect_equal(collect(unionString)[[5]]$name, "Andy") + expect_equal(collectRDD(unionString)[[1]], "Michael") + expect_equal(collectRDD(unionString)[[5]]$name, "Andy") }) test_that("objectFile() works with row serialization", { @@ -633,7 +633,7 @@ test_that("objectFile() works with row serialization", { expect_is(objectIn, "RDD") expect_equal(getSerializedMode(objectIn), "byte") - expect_equal(collect(objectIn)[[2]]$age, 30) + expect_equal(collectRDD(objectIn)[[2]]$age, 30) }) test_that("lapply() on a DataFrame returns an RDD with the correct columns", { @@ -643,7 +643,7 @@ test_that("lapply() on a DataFrame returns an RDD with the correct columns", { row }) expect_is(testRDD, "RDD") - collected <- collect(testRDD) + collected <- collectRDD(testRDD) expect_equal(collected[[1]]$name, "Michael") expect_equal(collected[[2]]$newCol, 35) }) @@ -715,10 +715,10 @@ test_that("multiple pipeline transformations result in an RDD with the correct v row }) expect_is(second, "RDD") - expect_equal(count(second), 3) - expect_equal(collect(second)[[2]]$age, 35) - expect_true(collect(second)[[2]]$testCol) - expect_false(collect(second)[[3]]$testCol) + expect_equal(countRDD(second), 3) + expect_equal(collectRDD(second)[[2]]$age, 35) + expect_true(collectRDD(second)[[2]]$testCol) + expect_false(collectRDD(second)[[3]]$testCol) }) test_that("cache(), persist(), and unpersist() on a DataFrame", { @@ -1608,7 +1608,7 @@ test_that("toJSON() returns an RDD of the correct values", { testRDD <- toJSON(df) expect_is(testRDD, "RDD") expect_equal(getSerializedMode(testRDD), "string") - expect_equal(collect(testRDD)[[1]], mockLines[1]) + expect_equal(collectRDD(testRDD)[[1]], mockLines[1]) }) test_that("showDF()", { diff --git a/R/pkg/inst/tests/testthat/test_take.R b/R/pkg/inst/tests/testthat/test_take.R index daf5e41abe13f..dcf479363b9a8 100644 --- a/R/pkg/inst/tests/testthat/test_take.R +++ b/R/pkg/inst/tests/testthat/test_take.R @@ -36,32 +36,32 @@ sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", test_that("take() gives back the original elements in correct count and order", { numVectorRDD <- parallelize(sc, 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))) + expect_equal(takeRDD(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))) + expect_equal(takeRDD(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)) + expect_equal(takeRDD(numVectorRDD, length(numVector)), as.list(numVector)) + expect_equal(takeRDD(numVectorRDD, length(numVector) + 1), as.list(numVector)) numListRDD <- parallelize(sc, numList, 1) numListRDD2 <- parallelize(sc, 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) + expect_equal(takeRDD(numListRDD, 3), takeRDD(numListRDD2, 3)) + expect_equal(takeRDD(numListRDD, 5), takeRDD(numListRDD2, 5)) + expect_equal(takeRDD(numListRDD, 1), as.list(head(numList, n = 1))) + expect_equal(takeRDD(numListRDD2, 999), numList) strVectorRDD <- parallelize(sc, strVector, 2) strVectorRDD2 <- parallelize(sc, strVector, 3) - expect_equal(take(strVectorRDD, 4), as.list(strVector)) - expect_equal(take(strVectorRDD2, 2), as.list(head(strVector, n = 2))) + expect_equal(takeRDD(strVectorRDD, 4), as.list(strVector)) + expect_equal(takeRDD(strVectorRDD2, 2), as.list(head(strVector, n = 2))) strListRDD <- parallelize(sc, strList, 4) strListRDD2 <- parallelize(sc, 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_equal(takeRDD(strListRDD, 3), as.list(head(strList, n = 3))) + expect_equal(takeRDD(strListRDD2, 1), as.list(head(strList, n = 1))) - expect_equal(length(take(strListRDD, 0)), 0) - expect_equal(length(take(strVectorRDD, 0)), 0) - expect_equal(length(take(numListRDD, 0)), 0) - expect_equal(length(take(numVectorRDD, 0)), 0) + expect_equal(length(takeRDD(strListRDD, 0)), 0) + expect_equal(length(takeRDD(strVectorRDD, 0)), 0) + expect_equal(length(takeRDD(numListRDD, 0)), 0) + expect_equal(length(takeRDD(numVectorRDD, 0)), 0) }) diff --git a/R/pkg/inst/tests/testthat/test_textFile.R b/R/pkg/inst/tests/testthat/test_textFile.R index 7b2cc74753fe2..ba434a5d4127b 100644 --- a/R/pkg/inst/tests/testthat/test_textFile.R +++ b/R/pkg/inst/tests/testthat/test_textFile.R @@ -29,8 +29,8 @@ test_that("textFile() on a local file returns an RDD", { rdd <- textFile(sc, fileName) expect_is(rdd, "RDD") - expect_true(count(rdd) > 0) - expect_equal(count(rdd), 2) + expect_true(countRDD(rdd) > 0) + expect_equal(countRDD(rdd), 2) unlink(fileName) }) @@ -40,7 +40,7 @@ test_that("textFile() followed by a collect() returns the same content", { writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) - expect_equal(collect(rdd), as.list(mockFile)) + expect_equal(collectRDD(rdd), as.list(mockFile)) unlink(fileName) }) @@ -55,7 +55,7 @@ test_that("textFile() word count works as expected", { wordCount <- lapply(words, function(word) { list(word, 1L) }) counts <- reduceByKey(wordCount, "+", 2L) - output <- collect(counts) + output <- collectRDD(counts) expected <- list(list("pretty.", 1), list("is", 2), list("awesome.", 1), list("Spark", 2)) expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) @@ -72,7 +72,7 @@ test_that("several transformations on RDD created by textFile()", { # PipelinedRDD initially created from RDD rdd <- lapply(rdd, function(x) paste(x, x)) } - collect(rdd) + collectRDD(rdd) unlink(fileName) }) @@ -85,7 +85,7 @@ test_that("textFile() followed by a saveAsTextFile() returns the same content", rdd <- textFile(sc, fileName1, 1L) saveAsTextFile(rdd, fileName2) rdd <- textFile(sc, fileName2) - expect_equal(collect(rdd), as.list(mockFile)) + expect_equal(collectRDD(rdd), as.list(mockFile)) unlink(fileName1) unlink(fileName2) @@ -97,7 +97,7 @@ test_that("saveAsTextFile() on a parallelized list works as expected", { rdd <- parallelize(sc, l, 1L) saveAsTextFile(rdd, fileName) rdd <- textFile(sc, fileName) - expect_equal(collect(rdd), lapply(l, function(x) {toString(x)})) + expect_equal(collectRDD(rdd), lapply(l, function(x) {toString(x)})) unlink(fileName) }) @@ -117,7 +117,7 @@ test_that("textFile() and saveAsTextFile() word count works as expected", { saveAsTextFile(counts, fileName2) rdd <- textFile(sc, fileName2) - output <- collect(rdd) + output <- collectRDD(rdd) expected <- list(list("awesome.", 1), list("Spark", 2), list("pretty.", 1), list("is", 2)) expectedStr <- lapply(expected, function(x) { toString(x) }) @@ -134,7 +134,7 @@ test_that("textFile() on multiple paths", { writeLines("Spark is awesome.", fileName2) rdd <- textFile(sc, c(fileName1, fileName2)) - expect_equal(count(rdd), 2) + expect_equal(countRDD(rdd), 2) unlink(fileName1) unlink(fileName2) @@ -147,16 +147,16 @@ test_that("Pipelined operations on RDDs created using textFile", { rdd <- textFile(sc, fileName) lengths <- lapply(rdd, function(x) { length(x) }) - expect_equal(collect(lengths), list(1, 1)) + expect_equal(collectRDD(lengths), list(1, 1)) lengthsPipelined <- lapply(lengths, function(x) { x + 10 }) - expect_equal(collect(lengthsPipelined), list(11, 11)) + expect_equal(collectRDD(lengthsPipelined), list(11, 11)) lengths30 <- lapply(lengthsPipelined, function(x) { x + 20 }) - expect_equal(collect(lengths30), list(31, 31)) + expect_equal(collectRDD(lengths30), list(31, 31)) lengths20 <- lapply(lengths, function(x) { x + 20 }) - expect_equal(collect(lengths20), list(21, 21)) + expect_equal(collectRDD(lengths20), list(21, 21)) unlink(fileName) }) diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 21a119a06b937..42783fde5f25b 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -24,7 +24,7 @@ sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", 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 + # support generics well. Instead, we rely on collectRDD() returning a # JList. nums <- as.list(1:10) rdd <- parallelize(sc, nums, 1L) @@ -48,7 +48,7 @@ test_that("serializeToBytes on RDD", { text.rdd <- textFile(sc, fileName) expect_equal(getSerializedMode(text.rdd), "string") ser.rdd <- serializeToBytes(text.rdd) - expect_equal(collect(ser.rdd), as.list(mockFile)) + expect_equal(collectRDD(ser.rdd), as.list(mockFile)) expect_equal(getSerializedMode(ser.rdd), "byte") unlink(fileName) @@ -128,7 +128,7 @@ test_that("cleanClosure on R functions", { env <- environment(newF) expect_equal(ls(env), "t") expect_equal(get("t", envir = env, inherits = FALSE), t) - actual <- collect(lapply(rdd, f)) + actual <- collectRDD(lapply(rdd, f)) expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6))) expect_equal(actual, expected) From 6cb3eab7cc49ad8b8459ddc479a900de9dea1bcf Mon Sep 17 00:00:00 2001 From: sandy Date: Tue, 16 Aug 2016 12:50:55 -0700 Subject: [PATCH 1143/1470] [SPARK-17089][DOCS] Remove api doc link for mapReduceTriplets operator ## What changes were proposed in this pull request? Remove the api doc link for mapReduceTriplets operator because in latest api they are remove so when user link to that api they will not get mapReduceTriplets there so its more good to remove than confuse the user. ## How was this patch tested? Run all the test cases ![screenshot from 2016-08-16 23-08-25](https://cloud.githubusercontent.com/assets/8075390/17709393/8cfbf75a-6406-11e6-98e6-38f7b319d833.png) Author: sandy Closes #14669 from phalodi/SPARK-17089. (cherry picked from commit e28a8c5899c48ff065e2fd3bb6b10c82b4d39c2c) Signed-off-by: Reynold Xin --- docs/graphx-programming-guide.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index bf4b968eb8b78..07b38d9cc9a8f 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -24,7 +24,6 @@ description: GraphX graph processing library guide for Spark SPARK_VERSION_SHORT [Graph.outerJoinVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] [Graph.aggregateMessages]: api/scala/index.html#org.apache.spark.graphx.Graph@aggregateMessages[A]((EdgeContext[VD,ED,A])⇒Unit,(A,A)⇒A,TripletFields)(ClassTag[A]):VertexRDD[A] [EdgeContext]: api/scala/index.html#org.apache.spark.graphx.EdgeContext -[Graph.mapReduceTriplets]: api/scala/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] [GraphOps.collectNeighborIds]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] [GraphOps.collectNeighbors]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] [RDD Persistence]: programming-guide.html#rdd-persistence @@ -596,7 +595,7 @@ compute the average age of the more senior followers of each user. ### Map Reduce Triplets Transition Guide (Legacy) In earlier versions of GraphX neighborhood aggregation was accomplished using the -[`mapReduceTriplets`][Graph.mapReduceTriplets] operator: +`mapReduceTriplets` operator: {% highlight scala %} class Graph[VD, ED] { @@ -607,7 +606,7 @@ class Graph[VD, ED] { } {% endhighlight %} -The [`mapReduceTriplets`][Graph.mapReduceTriplets] operator takes a user defined map function which +The `mapReduceTriplets` operator takes a user defined map function which is applied to each triplet and can yield *messages* which are aggregated using the user defined `reduce` function. However, we found the user of the returned iterator to be expensive and it inhibited our ability to From 3e0163bee2354258899c82ce4cc4aacafd2a802d Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 16 Aug 2016 21:35:39 -0700 Subject: [PATCH 1144/1470] [SPARK-17084][SQL] Rename ParserUtils.assert to validate ## What changes were proposed in this pull request? This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state. ## How was this patch tested? Simple rename. Compilation should do. Author: Herman van Hovell Closes #14665 from hvanhovell/SPARK-17084. (cherry picked from commit 4a2c375be2bcd98cc7e00bea920fd6a0f68a4e14) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/AstBuilder.scala | 14 +++++++------- .../spark/sql/catalyst/parser/ParserUtils.scala | 4 ++-- .../spark/sql/execution/SparkSqlParser.scala | 5 ++--- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1a0e7ab32a6c3..aee8eb1f3877b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -132,7 +132,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // Build the insert clauses. val inserts = ctx.multiInsertQueryBody.asScala.map { body => - assert(body.querySpecification.fromClause == null, + validate(body.querySpecification.fromClause == null, "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements", body) @@ -591,7 +591,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // function takes X PERCENT as the input and the range of X is [0, 100], we need to // adjust the fraction. val eps = RandomSampler.roundingEpsilon - assert(fraction >= 0.0 - eps && fraction <= 1.0 + eps, + validate(fraction >= 0.0 - eps && fraction <= 1.0 + eps, s"Sampling fraction ($fraction) must be on interval [0, 1]", ctx) Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true) @@ -659,7 +659,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // Get the backing expressions. val expressions = ctx.expression.asScala.map { eCtx => val e = expression(eCtx) - assert(e.foldable, "All expressions in an inline table must be constants.", eCtx) + validate(e.foldable, "All expressions in an inline table must be constants.", eCtx) e } @@ -681,7 +681,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val baseAttributes = structType.toAttributes.map(_.withNullability(true)) val attributes = if (ctx.identifierList != null) { val aliases = visitIdentifierList(ctx.identifierList) - assert(aliases.size == baseAttributes.size, + validate(aliases.size == baseAttributes.size, "Number of aliases must match the number of fields in an inline table.", ctx) baseAttributes.zip(aliases).map(p => p._1.withName(p._2)) } else { @@ -1089,7 +1089,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // We currently only allow foldable integers. def value: Int = { val e = expression(ctx.expression) - assert(e.resolved && e.foldable && e.dataType == IntegerType, + validate(e.resolved && e.foldable && e.dataType == IntegerType, "Frame bound value must be a constant integer.", ctx) e.eval().asInstanceOf[Int] @@ -1342,7 +1342,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { */ override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) { val intervals = ctx.intervalField.asScala.map(visitIntervalField) - assert(intervals.nonEmpty, "at least one time unit should be given for interval literal", ctx) + validate(intervals.nonEmpty, "at least one time unit should be given for interval literal", ctx) Literal(intervals.reduce(_.add(_))) } @@ -1369,7 +1369,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { case (from, Some(t)) => throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx) } - assert(interval != null, "No interval can be constructed", ctx) + validate(interval != null, "No interval can be constructed", ctx) interval } catch { // Handle Exceptions thrown by CalendarInterval diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index b04ce58e233aa..bc35ae2f55409 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -77,8 +77,8 @@ object ParserUtils { Origin(Option(token.getLine), Option(token.getCharPositionInLine)) } - /** Assert if a condition holds. If it doesn't throw a parse exception. */ - def assert(f: => Boolean, message: String, ctx: ParserRuleContext): Unit = { + /** Validate the condition. If it doesn't throw a parse exception. */ + def validate(f: => Boolean, message: String, ctx: ParserRuleContext): Unit = { if (!f) { throw new ParseException(message, ctx) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index d6185d0bfce14..876b3341d217e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution import scala.collection.JavaConverters._ -import scala.util.Try import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.TerminalNode @@ -778,7 +777,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } /** - * Create an [[AlterTableDiscoverPartitionsCommand]] command + * Create an [[AlterTableRecoverPartitionsCommand]] command * * For example: * {{{ @@ -1179,7 +1178,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { entry("mapkey.delim", ctx.keysTerminatedBy) ++ Option(ctx.linesSeparatedBy).toSeq.map { token => val value = string(token) - assert( + validate( value == "\n", s"LINES TERMINATED BY only supports newline '\\n' right now: $value", ctx) From 68a24d3e7aa9b40d4557652d3179b0ccb0f8624e Mon Sep 17 00:00:00 2001 From: mvervuurt Date: Tue, 16 Aug 2016 23:12:59 -0700 Subject: [PATCH 1145/1470] [MINOR][DOC] Fix the descriptions for `properties` argument in the documenation for jdbc APIs ## What changes were proposed in this pull request? This should be credited to mvervuurt. The main purpose of this PR is - simply to include the change for the same instance in `DataFrameReader` just to match up. - just avoid duplicately verifying the PR (as I already did). The documentation for both should be the same because both assume the `properties` should be the same `dict` for the same option. ## How was this patch tested? Manually building Python documentation. This will produce the output as below: - `DataFrameReader` ![2016-08-17 11 12 00](https://cloud.githubusercontent.com/assets/6477701/17722764/b3f6568e-646f-11e6-8b75-4fb672f3f366.png) - `DataFrameWriter` ![2016-08-17 11 12 10](https://cloud.githubusercontent.com/assets/6477701/17722765/b58cb308-646f-11e6-841a-32f19800d139.png) Closes #14624 Author: hyukjinkwon Author: mvervuurt Closes #14677 from HyukjinKwon/typo-python. (cherry picked from commit 0f6aa8afaacdf0ceca9c2c1650ca26a5c167ae69) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 4020bb3fa45b0..64de33e8ec0a8 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -401,8 +401,9 @@ def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPar :param numPartitions: the number of partitions :param predicates: a list of expressions suitable for inclusion in WHERE clauses; each one defines one partition of the :class:`DataFrame` - :param properties: a dictionary of JDBC database connection arguments; normally, - at least a "user" and "password" property should be included + :param properties: a dictionary of JDBC database connection arguments. Normally at + least properties "user" and "password" with their corresponding values. + For example { 'user' : 'SYSTEM', 'password' : 'mypassword' } :return: a DataFrame """ if properties is None: @@ -716,9 +717,9 @@ def jdbc(self, url, table, mode=None, properties=None): * ``overwrite``: Overwrite existing data. * ``ignore``: Silently ignore this operation if data already exists. * ``error`` (default case): Throw an exception if data already exists. - :param properties: JDBC database connection arguments, a list of - arbitrary string tag/value. Normally at least a - "user" and "password" property should be included. + :param properties: a dictionary of JDBC database connection arguments. Normally at + least properties "user" and "password" with their corresponding values. + For example { 'user' : 'SYSTEM', 'password' : 'mypassword' } """ if properties is None: properties = dict() From 22c7660a8744049e27ea4cc4c08755ac95ea43f5 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 17 Aug 2016 21:34:57 +0800 Subject: [PATCH 1146/1470] [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB ## What changes were proposed in this pull request? This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method. Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf). ## How was this patch tested? Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10 Author: Kazuaki Ishizaki Closes #14670 from kiszk/SPARK-15285-2. (cherry picked from commit 56d86742d2600b8426d75bd87ab3c73332dca1d2) Signed-off-by: Wenchen Fan --- .../expressions/objects/objects.scala | 32 ++++++++++++--- .../spark/sql/DataFrameComplexTypeSuite.scala | 40 +++++++++++++++++++ 2 files changed, 66 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 9621db1d38762..37ec1a63394cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -232,27 +232,47 @@ case class NewInstance( override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val javaType = ctx.javaType(dataType) - val argGen = arguments.map(_.genCode(ctx)) - val argString = argGen.map(_.value).mkString(", ") + val argIsNulls = ctx.freshName("argIsNulls") + ctx.addMutableState("boolean[]", argIsNulls, + s"$argIsNulls = new boolean[${arguments.size}];") + val argValues = arguments.zipWithIndex.map { case (e, i) => + val argValue = ctx.freshName("argValue") + ctx.addMutableState(ctx.javaType(e.dataType), argValue, "") + argValue + } + + val argCodes = arguments.zipWithIndex.map { case (e, i) => + val expr = e.genCode(ctx) + expr.code + s""" + $argIsNulls[$i] = ${expr.isNull}; + ${argValues(i)} = ${expr.value}; + """ + } + val argCode = ctx.splitExpressions(ctx.INPUT_ROW, argCodes) val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) var isNull = ev.isNull val setIsNull = if (propagateNull && arguments.nonEmpty) { - s"final boolean $isNull = ${argGen.map(_.isNull).mkString(" || ")};" + s""" + boolean $isNull = false; + for (int idx = 0; idx < ${arguments.length}; idx++) { + if ($argIsNulls[idx]) { $isNull = true; break; } + } + """ } else { isNull = "false" "" } val constructorCall = outer.map { gen => - s"""${gen.value}.new ${cls.getSimpleName}($argString)""" + s"""${gen.value}.new ${cls.getSimpleName}(${argValues.mkString(", ")})""" }.getOrElse { - s"new $className($argString)" + s"new $className(${argValues.mkString(", ")})" } val code = s""" - ${argGen.map(_.code).mkString("\n")} + $argCode ${outer.map(_.code).getOrElse("")} $setIsNull final $javaType ${ev.value} = $isNull ? ${ctx.defaultValue(javaType)} : $constructorCall; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 72f676e6225ee..1230b921aa279 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.DefinedByConstructorParams import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext @@ -58,4 +59,43 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { val nullIntRow = df.selectExpr("i[1]").collect()(0) assert(nullIntRow == org.apache.spark.sql.Row(null)) } + + test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { + val ds100_5 = Seq(S100_5()).toDS() + ds100_5.rdd.count + } } + +class S100( + val s1: String = "1", val s2: String = "2", val s3: String = "3", val s4: String = "4", + val s5: String = "5", val s6: String = "6", val s7: String = "7", val s8: String = "8", + val s9: String = "9", val s10: String = "10", val s11: String = "11", val s12: String = "12", + val s13: String = "13", val s14: String = "14", val s15: String = "15", val s16: String = "16", + val s17: String = "17", val s18: String = "18", val s19: String = "19", val s20: String = "20", + val s21: String = "21", val s22: String = "22", val s23: String = "23", val s24: String = "24", + val s25: String = "25", val s26: String = "26", val s27: String = "27", val s28: String = "28", + val s29: String = "29", val s30: String = "30", val s31: String = "31", val s32: String = "32", + val s33: String = "33", val s34: String = "34", val s35: String = "35", val s36: String = "36", + val s37: String = "37", val s38: String = "38", val s39: String = "39", val s40: String = "40", + val s41: String = "41", val s42: String = "42", val s43: String = "43", val s44: String = "44", + val s45: String = "45", val s46: String = "46", val s47: String = "47", val s48: String = "48", + val s49: String = "49", val s50: String = "50", val s51: String = "51", val s52: String = "52", + val s53: String = "53", val s54: String = "54", val s55: String = "55", val s56: String = "56", + val s57: String = "57", val s58: String = "58", val s59: String = "59", val s60: String = "60", + val s61: String = "61", val s62: String = "62", val s63: String = "63", val s64: String = "64", + val s65: String = "65", val s66: String = "66", val s67: String = "67", val s68: String = "68", + val s69: String = "69", val s70: String = "70", val s71: String = "71", val s72: String = "72", + val s73: String = "73", val s74: String = "74", val s75: String = "75", val s76: String = "76", + val s77: String = "77", val s78: String = "78", val s79: String = "79", val s80: String = "80", + val s81: String = "81", val s82: String = "82", val s83: String = "83", val s84: String = "84", + val s85: String = "85", val s86: String = "86", val s87: String = "87", val s88: String = "88", + val s89: String = "89", val s90: String = "90", val s91: String = "91", val s92: String = "92", + val s93: String = "93", val s94: String = "94", val s95: String = "95", val s96: String = "96", + val s97: String = "97", val s98: String = "98", val s99: String = "99", val s100: String = "100") +extends DefinedByConstructorParams + +case class S100_5( + s1: S100 = new S100(), s2: S100 = new S100(), s3: S100 = new S100(), + s4: S100 = new S100(), s5: S100 = new S100()) + + From 394d5986617e65852422afeb8d755e38795bbe25 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 17 Aug 2016 09:31:22 -0700 Subject: [PATCH 1147/1470] [SPARK-17102][SQL] bypass UserDefinedGenerator for json format check ## What changes were proposed in this pull request? We use reflection to convert `TreeNode` to json string, and currently don't support arbitrary object. `UserDefinedGenerator` takes a function object, so we should skip json format test for it, or the tests can be flacky, e.g. `DataFrameSuite.simple explode`, this test always fail with scala 2.10(branch 1.6 builds with scala 2.10 by default), but pass with scala 2.11(master branch builds with scala 2.11 by default). ## How was this patch tested? N/A Author: Wenchen Fan Closes #14679 from cloud-fan/json. (cherry picked from commit 928ca1c6d12b23d84f9b6205e22d2e756311f072) Signed-off-by: Yin Huai --- sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 e8480a7001760..b2c051d11cf20 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 @@ -242,9 +242,10 @@ abstract class QueryTest extends PlanTest { case p if p.getClass.getSimpleName == "MetastoreRelation" => return case _: MemoryPlan => return }.transformAllExpressions { - case a: ImperativeAggregate => return + case _: ImperativeAggregate => return case _: TypedAggregateExpression => return case Literal(_, _: ObjectType) => return + case _: UserDefinedGenerator => return } // bypass hive tests before we fix all corner cases in hive module. From 9406f82db1e96c84bfacb4cac9b74aab6d4fde06 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 17 Aug 2016 13:31:34 -0700 Subject: [PATCH 1148/1470] [SPARK-17096][SQL][STREAMING] Improve exception string reported through the StreamingQueryListener ## What changes were proposed in this pull request? Currently, the stackTrace (as `Array[StackTraceElements]`) reported through StreamingQueryListener.onQueryTerminated is useless as it has the stack trace of where StreamingQueryException is defined, not the stack trace of underlying exception. For example, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.stackTrace` will have ``` org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:211) org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124) ``` This is basically useless, as it is location where the StreamingQueryException was defined. What we want is Here is the right way to reason about what should be posted as through StreamingQueryListener.onQueryTerminated - The actual exception could either be a SparkException, or an arbitrary exception. - SparkException reports the relevant executor stack trace of a failed task as a string in the the exception message. The `Array[StackTraceElements]` returned by `SparkException.stackTrace()` is mostly irrelevant. - For any arbitrary exception, the `Array[StackTraceElements]` returned by `exception.stackTrace()` may be relevant. - When there is an error in a streaming query, it's hard to reason whether the `Array[StackTraceElements]` is useful or not. In fact, it is not clear whether it is even useful to report the stack trace as this array of Java objects. It may be sufficient to report the strack trace as a string, along with the message. This is how Spark reported executor stra - Hence, this PR simplifies the API by removing the array `stackTrace` from `QueryTerminated`. Instead the `exception` returns a string containing the message and the stack trace of the actual underlying exception that failed the streaming query (i.e. not that of the StreamingQueryException). If anyone is interested in the actual stack trace as an array, can always access them through `streamingQuery.exception` which returns the exception object. With this change, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.exception` will be ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent failure: Lost task 1.0 in stage 0.0 (TID 1, localhost): java.lang.ArithmeticException: / by zero at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply$mcII$sp(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:232) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:226) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:86) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1429) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1417) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1416) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1416) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802) ... ``` It contains the relevant executor stack trace. In a case non-SparkException, if the streaming source MemoryStream throws an exception, exception message will have the relevant stack trace. ``` java.lang.RuntimeException: this is the exception message at org.apache.spark.sql.execution.streaming.MemoryStream.getBatch(memory.scala:103) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:316) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:313) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241) at org.apache.spark.sql.execution.streaming.StreamProgress.flatMap(StreamProgress.scala:25) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:313) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197) at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124) ``` Note that this change in the public `QueryTerminated` class is okay as the APIs are still experimental. ## How was this patch tested? Unit tests that test whether the right information is present in the exception message reported through QueryTerminated object. Author: Tathagata Das Closes #14675 from tdas/SPARK-17096. (cherry picked from commit d60af8f6aa53373de1333cc642cf2a9d7b39d912) Signed-off-by: Tathagata Das --- .../sql/execution/streaming/StreamExecution.scala | 5 +---- .../sql/streaming/StreamingQueryException.scala | 3 ++- .../sql/streaming/StreamingQueryListener.scala | 3 +-- .../sql/streaming/StreamingQueryListenerSuite.scala | 13 ++++++------- 4 files changed, 10 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 66fb5a4bdeb7f..4d05af0b60358 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -217,10 +217,7 @@ class StreamExecution( } finally { state = TERMINATED sparkSession.streams.notifyQueryTermination(StreamExecution.this) - postEvent(new QueryTerminated( - this.toInfo, - exception.map(_.getMessage), - exception.map(_.getStackTrace.toSeq).getOrElse(Nil))) + postEvent(new QueryTerminated(this.toInfo, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala index 90f95ca9d4229..bd3e5a5618ec4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -22,7 +22,8 @@ import org.apache.spark.sql.execution.streaming.{Offset, StreamExecution} /** * :: Experimental :: - * Exception that stopped a [[StreamingQuery]]. + * Exception that stopped a [[StreamingQuery]]. Use `cause` get the actual exception + * that caused the failure. * @param query Query that caused the exception * @param message Message of this exception * @param cause Internal cause of this exception diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 3b3cead3a66de..db606abb8ce43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -108,6 +108,5 @@ object StreamingQueryListener { @Experimental class QueryTerminated private[sql]( val queryInfo: StreamingQueryInfo, - val exception: Option[String], - val stackTrace: Seq[StackTraceElement]) extends Event + val exception: Option[String]) extends Event } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 7f4d28cf0598f..77602e8167fa3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -94,7 +94,6 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.id === query.id) assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) - assert(listener.terminationStackTrace.isEmpty) assert(listener.terminationException === None) } listener.checkAsyncErrors() @@ -147,7 +146,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } - test("exception should be reported in QueryTerminated") { + testQuietly("exception should be reported in QueryTerminated") { val listener = new QueryStatusCollector withListenerAdded(listener) { val input = MemoryStream[Int] @@ -159,8 +158,11 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { spark.sparkContext.listenerBus.waitUntilEmpty(10000) assert(listener.terminationStatus !== null) assert(listener.terminationException.isDefined) + // Make sure that the exception message reported through listener + // contains the actual exception and relevant stack trace + assert(!listener.terminationException.get.contains("StreamingQueryException")) assert(listener.terminationException.get.contains("java.lang.ArithmeticException")) - assert(listener.terminationStackTrace.nonEmpty) + assert(listener.terminationException.get.contains("StreamingQueryListenerSuite")) } ) } @@ -205,8 +207,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val exception = new RuntimeException("exception") val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( queryTerminatedInfo, - Some(exception.getMessage), - exception.getStackTrace) + Some(exception.getMessage)) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) @@ -262,7 +263,6 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { @volatile var startStatus: StreamingQueryInfo = null @volatile var terminationStatus: StreamingQueryInfo = null @volatile var terminationException: Option[String] = null - @volatile var terminationStackTrace: Seq[StackTraceElement] = null val progressStatuses = new ConcurrentLinkedQueue[StreamingQueryInfo] @@ -296,7 +296,6 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(startStatus != null, "onQueryTerminated called before onQueryStarted") terminationStatus = queryTerminated.queryInfo terminationException = queryTerminated.exception - terminationStackTrace = queryTerminated.stackTrace } asyncTestWaiter.dismiss() } From 585d1d95cb1c4419c716d3b3f595834927e0c175 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Wed, 17 Aug 2016 16:31:42 -0700 Subject: [PATCH 1149/1470] [SPARK-17038][STREAMING] fix metrics retrieval source of 'lastReceivedBatch' https://issues.apache.org/jira/browse/SPARK-17038 ## What changes were proposed in this pull request? StreamingSource's lastReceivedBatch_submissionTime, lastReceivedBatch_processingTimeStart, and lastReceivedBatch_processingTimeEnd all use data from lastCompletedBatch instead of lastReceivedBatch. In particular, this makes it impossible to match lastReceivedBatch_records with a batchID/submission time. This is apparent when looking at StreamingSource.scala, lines 89-94. ## How was this patch tested? Manually running unit tests on local laptop Author: Xin Ren Closes #14681 from keypointt/SPARK-17038. (cherry picked from commit e6bef7d52f0e19ec771fb0f3e96c7ddbd1a6a19b) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/streaming/StreamingSource.scala | 6 +++--- .../streaming/ui/StreamingJobProgressListenerSuite.scala | 3 +++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index 9697437dd2fe5..0b306a28d1a59 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -87,11 +87,11 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { // Gauge for last received batch, useful for monitoring the streaming job's running status, // displayed data -1 for any abnormal condition. registerGaugeWithOption("lastReceivedBatch_submissionTime", - _.lastCompletedBatch.map(_.submissionTime), -1L) + _.lastReceivedBatch.map(_.submissionTime), -1L) registerGaugeWithOption("lastReceivedBatch_processingStartTime", - _.lastCompletedBatch.flatMap(_.processingStartTime), -1L) + _.lastReceivedBatch.flatMap(_.processingStartTime), -1L) registerGaugeWithOption("lastReceivedBatch_processingEndTime", - _.lastCompletedBatch.flatMap(_.processingEndTime), -1L) + _.lastReceivedBatch.flatMap(_.processingEndTime), -1L) // Gauge for last received batch records. registerGauge("lastReceivedBatch_records", _.lastReceivedBatchRecords.values.sum, 0L) 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 26b757cc2d535..46ab3ac8de3d4 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 @@ -68,6 +68,7 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { listener.waitingBatches should be (List(BatchUIData(batchInfoSubmitted))) listener.runningBatches should be (Nil) listener.retainedCompletedBatches should be (Nil) + listener.lastReceivedBatch should be (Some(BatchUIData(batchInfoSubmitted))) listener.lastCompletedBatch should be (None) listener.numUnprocessedBatches should be (1) listener.numTotalCompletedBatches should be (0) @@ -81,6 +82,7 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { listener.waitingBatches should be (Nil) listener.runningBatches should be (List(BatchUIData(batchInfoStarted))) listener.retainedCompletedBatches should be (Nil) + listener.lastReceivedBatch should be (Some(BatchUIData(batchInfoStarted))) listener.lastCompletedBatch should be (None) listener.numUnprocessedBatches should be (1) listener.numTotalCompletedBatches should be (0) @@ -123,6 +125,7 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { listener.waitingBatches should be (Nil) listener.runningBatches should be (Nil) listener.retainedCompletedBatches should be (List(BatchUIData(batchInfoCompleted))) + listener.lastReceivedBatch should be (Some(BatchUIData(batchInfoCompleted))) listener.lastCompletedBatch should be (Some(BatchUIData(batchInfoCompleted))) listener.numUnprocessedBatches should be (0) listener.numTotalCompletedBatches should be (1) From 91aa53239570d7c89e771050d79a1a857797498b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 18 Aug 2016 13:24:12 +0800 Subject: [PATCH 1150/1470] [SPARK-16995][SQL] TreeNodeException when flat mapping RelationalGroupedDataset created from DataFrame containing a column created with lit/expr ## What changes were proposed in this pull request? A TreeNodeException is thrown when executing the following minimal example in Spark 2.0. import spark.implicits._ case class test (x: Int, q: Int) val d = Seq(1).toDF("x") d.withColumn("q", lit(0)).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show d.withColumn("q", expr("0")).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show The problem is at `FoldablePropagation`. The rule will do `transformExpressions` on `LogicalPlan`. The query above contains a `MapGroups` which has a parameter `dataAttributes:Seq[Attribute]`. One attributes in `dataAttributes` will be transformed to an `Alias(literal(0), _)` in `FoldablePropagation`. `Alias` is not an `Attribute` and causes the error. We can't easily detect such type inconsistency during transforming expressions. A direct approach to this problem is to skip doing `FoldablePropagation` on object operators as they should not contain such expressions. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #14648 from viirya/flat-mapping. (cherry picked from commit 10204b9d29cd69895f5a606e75510dc64cf2e009) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 13 +++++++++++++ .../scala/org/apache/spark/sql/DatasetSuite.scala | 13 +++++++++++++ 2 files changed, 26 insertions(+) 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 f24f8b78d476f..627f82994f8db 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 @@ -695,6 +695,19 @@ object FoldablePropagation extends Rule[LogicalPlan] { case j @ Join(_, _, LeftOuter | RightOuter | FullOuter, _) => stop = true j + + // These 3 operators take attributes as constructor parameters, and these attributes + // can't be replaced by alias. + case m: MapGroups => + stop = true + m + case f: FlatMapGroupsInR => + stop = true + f + case c: CoGroup => + stop = true + c + case p: LogicalPlan if !stop => p.transformExpressions { case a: AttributeReference if foldableMap.contains(a) => foldableMap(a) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 43cbc03b7aa0c..9aeeda4463afc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -844,6 +844,19 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds = spark.createDataset(data)(enc) checkDataset(ds, (("a", "b"), "c"), (null, "d")) } + + test("SPARK-16995: flat mapping on Dataset containing a column created with lit/expr") { + val df = Seq("1").toDF("a") + + import df.sparkSession.implicits._ + + checkDataset( + df.withColumn("b", lit(0)).as[ClassData] + .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) + checkDataset( + df.withColumn("b", expr("0")).as[ClassData] + .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) + } } case class Generic[T](id: T, value: Double) From 5735b8bd769c64e2b0e0fae75bad794cde3edc99 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 18 Aug 2016 16:37:25 +0800 Subject: [PATCH 1151/1470] [SPARK-16391][SQL] Support partial aggregation for reduceGroups ## What changes were proposed in this pull request? This patch introduces a new private ReduceAggregator interface that is a subclass of Aggregator. ReduceAggregator only requires a single associative and commutative reduce function. ReduceAggregator is also used to implement KeyValueGroupedDataset.reduceGroups in order to support partial aggregation. Note that the pull request was initially done by viirya. ## How was this patch tested? Covered by original tests for reduceGroups, as well as a new test suite for ReduceAggregator. Author: Reynold Xin Author: Liang-Chi Hsieh Closes #14576 from rxin/reduceAggregator. (cherry picked from commit 1748f824101870b845dbbd118763c6885744f98a) Signed-off-by: Wenchen Fan --- .../spark/sql/KeyValueGroupedDataset.scala | 10 +-- .../sql/expressions/ReduceAggregator.scala | 68 +++++++++++++++++ .../expressions/ReduceAggregatorSuite.scala | 73 +++++++++++++++++++ 3 files changed, 146 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index a6867a67eeade..8eec42aab4fa3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -21,10 +21,11 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function._ -import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, OuterScopes} +import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateStruct} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.expressions.ReduceAggregator /** * :: Experimental :: @@ -177,10 +178,9 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 1.6.0 */ def reduceGroups(f: (V, V) => V): Dataset[(K, V)] = { - val func = (key: K, it: Iterator[V]) => Iterator((key, it.reduce(f))) - - implicit val resultEncoder = ExpressionEncoder.tuple(kExprEnc, vExprEnc) - flatMapGroups(func) + val vEncoder = encoderFor[V] + val aggregator: TypedColumn[V, V] = new ReduceAggregator[V](f)(vEncoder).toColumn + agg(aggregator) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala new file mode 100644 index 0000000000000..174378304d4a5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.expressions + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +/** + * An aggregator that uses a single associative and commutative reduce function. This reduce + * function can be used to go through all input values and reduces them to a single value. + * If there is no input, a null value is returned. + * + * This class currently assumes there is at least one input row. + */ +private[sql] class ReduceAggregator[T: Encoder](func: (T, T) => T) + extends Aggregator[T, (Boolean, T), T] { + + private val encoder = implicitly[Encoder[T]] + + override def zero: (Boolean, T) = (false, null.asInstanceOf[T]) + + override def bufferEncoder: Encoder[(Boolean, T)] = + ExpressionEncoder.tuple( + ExpressionEncoder[Boolean](), + encoder.asInstanceOf[ExpressionEncoder[T]]) + + override def outputEncoder: Encoder[T] = encoder + + override def reduce(b: (Boolean, T), a: T): (Boolean, T) = { + if (b._1) { + (true, func(b._2, a)) + } else { + (true, a) + } + } + + override def merge(b1: (Boolean, T), b2: (Boolean, T)): (Boolean, T) = { + if (!b1._1) { + b2 + } else if (!b2._1) { + b1 + } else { + (true, func(b1._2, b2._2)) + } + } + + override def finish(reduction: (Boolean, T)): T = { + if (!reduction._1) { + throw new IllegalStateException("ReduceAggregator requires at least one input row") + } + reduction._2 + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala new file mode 100644 index 0000000000000..d826d3f54d922 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala @@ -0,0 +1,73 @@ +/* + * 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.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Encoders +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +class ReduceAggregatorSuite extends SparkFunSuite { + + test("zero value") { + val encoder: ExpressionEncoder[Int] = ExpressionEncoder() + val func = (v1: Int, v2: Int) => v1 + v2 + val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt) + assert(aggregator.zero == (false, null)) + } + + test("reduce, merge and finish") { + val encoder: ExpressionEncoder[Int] = ExpressionEncoder() + val func = (v1: Int, v2: Int) => v1 + v2 + val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt) + + val firstReduce = aggregator.reduce(aggregator.zero, 1) + assert(firstReduce == (true, 1)) + + val secondReduce = aggregator.reduce(firstReduce, 2) + assert(secondReduce == (true, 3)) + + val thirdReduce = aggregator.reduce(secondReduce, 3) + assert(thirdReduce == (true, 6)) + + val mergeWithZero1 = aggregator.merge(aggregator.zero, firstReduce) + assert(mergeWithZero1 == (true, 1)) + + val mergeWithZero2 = aggregator.merge(secondReduce, aggregator.zero) + assert(mergeWithZero2 == (true, 3)) + + val mergeTwoReduced = aggregator.merge(firstReduce, secondReduce) + assert(mergeTwoReduced == (true, 4)) + + assert(aggregator.finish(firstReduce)== 1) + assert(aggregator.finish(secondReduce) == 3) + assert(aggregator.finish(thirdReduce) == 6) + assert(aggregator.finish(mergeWithZero1) == 1) + assert(aggregator.finish(mergeWithZero2) == 3) + assert(aggregator.finish(mergeTwoReduced) == 4) + } + + test("requires at least one input row") { + val encoder: ExpressionEncoder[Int] = ExpressionEncoder() + val func = (v1: Int, v2: Int) => v1 + v2 + val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt) + + intercept[IllegalStateException] { + aggregator.finish(aggregator.zero) + } + } +} From ec5f157a32f0c65b5f93bdde7a6334e982b3b83c Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Thu, 18 Aug 2016 13:44:13 +0200 Subject: [PATCH 1152/1470] [SPARK-17117][SQL] 1 / NULL should not fail analysis ## What changes were proposed in this pull request? This patch fixes the problem described in SPARK-17117, i.e. "SELECT 1 / NULL" throws an analysis exception: ``` org.apache.spark.sql.AnalysisException: cannot resolve '(1 / NULL)' due to data type mismatch: differing types in '(1 / NULL)' (int and null). ``` The problem is that division type coercion did not take null type into account. ## How was this patch tested? A unit test for the type coercion, and a few end-to-end test cases using SQLQueryTestSuite. Author: petermaxlee Closes #14695 from petermaxlee/SPARK-17117. (cherry picked from commit 68f5087d2107d6afec5d5745f0cb0e9e3bdd6a0b) Signed-off-by: Herman van Hovell --- .../sql/catalyst/analysis/TypeCoercion.scala | 7 +- .../catalyst/analysis/TypeCoercionSuite.scala | 9 +- .../resources/sql-tests/inputs/arithmetic.sql | 12 ++- .../sql-tests/results/arithmetic.sql.out | 84 +++++++++++++++---- 4 files changed, 89 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 021952e7166f9..21e96aaf53844 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -543,11 +543,14 @@ object TypeCoercion { // Decimal and Double remain the same case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d - case Divide(left, right) if isNumeric(left) && isNumeric(right) => + case Divide(left, right) if isNumericOrNull(left) && isNumericOrNull(right) => Divide(Cast(left, DoubleType), Cast(right, DoubleType)) } - private def isNumeric(ex: Expression): Boolean = ex.dataType.isInstanceOf[NumericType] + private def isNumericOrNull(ex: Expression): Boolean = { + // We need to handle null types in case a query contains null literals. + ex.dataType.isInstanceOf[NumericType] || ex.dataType == NullType + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index a13c45fe2ffee..9560563a8ca56 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import java.sql.Timestamp -import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{Division, FunctionArgumentConversion} +import org.apache.spark.sql.catalyst.analysis.TypeCoercion._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -730,6 +730,13 @@ class TypeCoercionSuite extends PlanTest { // the right expression to Decimal. ruleTest(rules, sum(Divide(Decimal(4.0), 3)), sum(Divide(Decimal(4.0), 3))) } + + test("SPARK-17117 null type coercion in divide") { + val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts) + val nullLit = Literal.create(null, NullType) + ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) + ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql b/sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql index cbe40410cdc10..f62b10ca0037b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/arithmetic.sql @@ -16,11 +16,19 @@ select + + 100; select - - max(key) from testdata; select + - key from testdata where key = 33; +-- div +select 5 / 2; +select 5 / 0; +select 5 / null; +select null / 5; +select 5 div 2; +select 5 div 0; +select 5 div null; +select null div 5; + -- other arithmetics select 1 + 2; select 1 - 2; select 2 * 5; -select 5 / 2; -select 5 div 2; select 5 % 3; select pmod(-7, 3); diff --git a/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out index f2b40a00d062d..6abe048af477d 100644 --- a/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/arithmetic.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 22 +-- Number of queries: 28 -- !query 0 @@ -123,35 +123,35 @@ struct<(- key):int> -- !query 15 -select 1 + 2 +select 5 / 2 -- !query 15 schema -struct<(1 + 2):int> +struct<(CAST(5 AS DOUBLE) / CAST(2 AS DOUBLE)):double> -- !query 15 output -3 +2.5 -- !query 16 -select 1 - 2 +select 5 / 0 -- !query 16 schema -struct<(1 - 2):int> +struct<(CAST(5 AS DOUBLE) / CAST(0 AS DOUBLE)):double> -- !query 16 output --1 +NULL -- !query 17 -select 2 * 5 +select 5 / null -- !query 17 schema -struct<(2 * 5):int> +struct<(CAST(5 AS DOUBLE) / CAST(NULL AS DOUBLE)):double> -- !query 17 output -10 +NULL -- !query 18 -select 5 / 2 +select null / 5 -- !query 18 schema -struct<(CAST(5 AS DOUBLE) / CAST(2 AS DOUBLE)):double> +struct<(CAST(NULL AS DOUBLE) / CAST(5 AS DOUBLE)):double> -- !query 18 output -2.5 +NULL -- !query 19 @@ -163,16 +163,64 @@ struct -- !query 20 -select 5 % 3 +select 5 div 0 -- !query 20 schema -struct<(5 % 3):int> +struct -- !query 20 output -2 +NULL -- !query 21 -select pmod(-7, 3) +select 5 div null -- !query 21 schema -struct +struct -- !query 21 output +NULL + + +-- !query 22 +select null div 5 +-- !query 22 schema +struct +-- !query 22 output +NULL + + +-- !query 23 +select 1 + 2 +-- !query 23 schema +struct<(1 + 2):int> +-- !query 23 output +3 + + +-- !query 24 +select 1 - 2 +-- !query 24 schema +struct<(1 - 2):int> +-- !query 24 output +-1 + + +-- !query 25 +select 2 * 5 +-- !query 25 schema +struct<(2 * 5):int> +-- !query 25 output +10 + + +-- !query 26 +select 5 % 3 +-- !query 26 schema +struct<(5 % 3):int> +-- !query 26 output +2 + + +-- !query 27 +select pmod(-7, 3) +-- !query 27 schema +struct +-- !query 27 output 2 From 176af17a7213a4c2847a04f715137257657f2961 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Wed, 10 Aug 2016 00:49:06 -0700 Subject: [PATCH 1153/1470] [MINOR][SPARKR] R API documentation for "coltypes" is confusing ## What changes were proposed in this pull request? R API documentation for "coltypes" is confusing, found when working on another ticket. Current version http://spark.apache.org/docs/2.0.0/api/R/coltypes.html, where parameters have 2 "x" which is a duplicate, and also the example is not very clear ![current](https://cloud.githubusercontent.com/assets/3925641/17386808/effb98ce-59a2-11e6-9657-d477d258a80c.png) ![screen shot 2016-08-03 at 5 56 00 pm](https://cloud.githubusercontent.com/assets/3925641/17386884/91831096-59a3-11e6-84af-39890b3d45d8.png) ## How was this patch tested? Tested manually on local machine. And the screenshots are like below: ![screen shot 2016-08-07 at 11 29 20 pm](https://cloud.githubusercontent.com/assets/3925641/17471144/df36633c-5cf6-11e6-8238-4e32ead0e529.png) ![screen shot 2016-08-03 at 5 56 22 pm](https://cloud.githubusercontent.com/assets/3925641/17386896/9d36cb26-59a3-11e6-9619-6dae29f7ab17.png) Author: Xin Ren Closes #14489 from keypointt/rExample. (cherry picked from commit 1203c8415cd11540f79a235e66a2f241ca6c71e4) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 5efc891ec2fe3..92e60e7385fca 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -218,7 +218,7 @@ setMethod("showDF", #' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) -#' df +#' show(df) #'} #' @note show(SparkDataFrame) since 1.4.0 setMethod("show", "SparkDataFrame", @@ -363,7 +363,7 @@ setMethod("colnames<-", #' @examples #'\dontrun{ #' irisDF <- createDataFrame(iris) -#' coltypes(irisDF) +#' coltypes(irisDF) # get column types #'} #' @note coltypes since 1.6.0 setMethod("coltypes", @@ -406,7 +406,6 @@ setMethod("coltypes", #' #' Set the column types of a SparkDataFrame. #' -#' @param x A SparkDataFrame #' @param value A character vector with the target column types for the given #' SparkDataFrame. Column types can be one of integer, numeric/double, character, logical, or NA #' to keep that column as-is. @@ -419,8 +418,8 @@ setMethod("coltypes", #' sparkR.session() #' path <- "path/to/file.json" #' df <- read.json(path) -#' coltypes(df) <- c("character", "integer") -#' coltypes(df) <- c(NA, "numeric") +#' coltypes(df) <- c("character", "integer") # set column types +#' coltypes(df) <- c(NA, "numeric") # set column types #'} #' @note coltypes<- since 1.6.0 setMethod("coltypes<-", From ea684b69cd6934bc093f4a5a8b0d8470e92157cd Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Thu, 18 Aug 2016 13:33:55 +0200 Subject: [PATCH 1154/1470] [SPARK-17069] Expose spark.range() as table-valued function in SQL This adds analyzer rules for resolving table-valued functions, and adds one builtin implementation for range(). The arguments for range() are the same as those of `spark.range()`. Unit tests. cc hvanhovell Author: Eric Liang Closes #14656 from ericl/sc-4309. (cherry picked from commit 412dba63b511474a6db3c43c8618d803e604bc6b) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 1 + .../sql/catalyst/analysis/Analyzer.scala | 1 + .../ResolveTableValuedFunctions.scala | 132 ++++++++++++++++++ .../sql/catalyst/analysis/unresolved.scala | 11 ++ .../sql/catalyst/parser/AstBuilder.scala | 8 ++ .../sql/catalyst/parser/PlanParserSuite.scala | 8 +- .../inputs/table-valued-functions.sql | 20 +++ .../results/table-valued-functions.sql.out | 87 ++++++++++++ 8 files changed, 267 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala create mode 100644 sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index aca728234ad99..51f3804fbd793 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -426,6 +426,7 @@ relationPrimary | '(' queryNoWith ')' sample? (AS? strictIdentifier)? #aliasedQuery | '(' relation ')' sample? (AS? strictIdentifier)? #aliasedRelation | inlineTable #inlineTableDefault2 + | identifier '(' (expression (',' expression)*)? ')' #tableValuedFunction ; inlineTable 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 57c3d9aece80c..e0b81666b5cb4 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 @@ -86,6 +86,7 @@ class Analyzer( WindowsSubstitution, EliminateUnions), Batch("Resolution", fixedPoint, + ResolveTableValuedFunctions :: ResolveRelations :: ResolveReferences :: ResolveDeserializer :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala new file mode 100644 index 0000000000000..7fdf7fa0c06a3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala @@ -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. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Range} +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.{DataType, IntegerType, LongType} + +/** + * Rule that resolves table-valued function references. + */ +object ResolveTableValuedFunctions extends Rule[LogicalPlan] { + private lazy val defaultParallelism = + SparkContext.getOrCreate(new SparkConf(false)).defaultParallelism + + /** + * List of argument names and their types, used to declare a function. + */ + private case class ArgumentList(args: (String, DataType)*) { + /** + * Try to cast the expressions to satisfy the expected types of this argument list. If there + * are any types that cannot be casted, then None is returned. + */ + def implicitCast(values: Seq[Expression]): Option[Seq[Expression]] = { + if (args.length == values.length) { + val casted = values.zip(args).map { case (value, (_, expectedType)) => + TypeCoercion.ImplicitTypeCasts.implicitCast(value, expectedType) + } + if (casted.forall(_.isDefined)) { + return Some(casted.map(_.get)) + } + } + None + } + + override def toString: String = { + args.map { a => + s"${a._1}: ${a._2.typeName}" + }.mkString(", ") + } + } + + /** + * A TVF maps argument lists to resolver functions that accept those arguments. Using a map + * here allows for function overloading. + */ + private type TVF = Map[ArgumentList, Seq[Any] => LogicalPlan] + + /** + * TVF builder. + */ + private def tvf(args: (String, DataType)*)(pf: PartialFunction[Seq[Any], LogicalPlan]) + : (ArgumentList, Seq[Any] => LogicalPlan) = { + (ArgumentList(args: _*), + pf orElse { + case args => + throw new IllegalArgumentException( + "Invalid arguments for resolved function: " + args.mkString(", ")) + }) + } + + /** + * Internal registry of table-valued functions. + */ + private val builtinFunctions: Map[String, TVF] = Map( + "range" -> Map( + /* range(end) */ + tvf("end" -> LongType) { case Seq(end: Long) => + Range(0, end, 1, defaultParallelism) + }, + + /* range(start, end) */ + tvf("start" -> LongType, "end" -> LongType) { case Seq(start: Long, end: Long) => + Range(start, end, 1, defaultParallelism) + }, + + /* range(start, end, step) */ + tvf("start" -> LongType, "end" -> LongType, "step" -> LongType) { + case Seq(start: Long, end: Long, step: Long) => + Range(start, end, step, defaultParallelism) + }, + + /* range(start, end, step, numPartitions) */ + tvf("start" -> LongType, "end" -> LongType, "step" -> LongType, + "numPartitions" -> IntegerType) { + case Seq(start: Long, end: Long, step: Long, numPartitions: Int) => + Range(start, end, step, numPartitions) + }) + ) + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case u: UnresolvedTableValuedFunction if u.functionArgs.forall(_.resolved) => + builtinFunctions.get(u.functionName) match { + case Some(tvf) => + val resolved = tvf.flatMap { case (argList, resolver) => + argList.implicitCast(u.functionArgs) match { + case Some(casted) => + Some(resolver(casted.map(_.eval()))) + case _ => + None + } + } + resolved.headOption.getOrElse { + val argTypes = u.functionArgs.map(_.dataType.typeName).mkString(", ") + u.failAnalysis( + s"""error: table-valued function ${u.functionName} with alternatives: + |${tvf.keys.map(_.toString).toSeq.sorted.map(x => s" ($x)").mkString("\n")} + |cannot be applied to: (${argTypes})""".stripMargin) + } + case _ => + u.failAnalysis(s"could not resolve `${u.functionName}` to a table-valued function") + } + } +} 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 609089a302c88..a66a551bb21f8 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,6 +49,17 @@ case class UnresolvedRelation( override lazy val resolved = false } +/** + * Holds a table-valued function call that has yet to be resolved. + */ +case class UnresolvedTableValuedFunction( + functionName: String, functionArgs: Seq[Expression]) extends LeafNode { + + override def output: Seq[Attribute] = Nil + + override lazy val resolved = false +} + /** * Holds the name of an attribute that has yet to be resolved. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index aee8eb1f3877b..1f2cd4c4691ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -652,6 +652,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { table.optionalMap(ctx.sample)(withSample) } + /** + * Create a table-valued function call with arguments, e.g. range(1000) + */ + override def visitTableValuedFunction(ctx: TableValuedFunctionContext) + : LogicalPlan = withOrigin(ctx) { + UnresolvedTableValuedFunction(ctx.identifier.getText, ctx.expression.asScala.map(expression)) + } + /** * Create an inline table (a virtual table in Hive parlance). */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 456948d6455c9..05bd0cd0380ff 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.parser import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator +import org.apache.spark.sql.catalyst.analysis.{UnresolvedGenerator, UnresolvedTableValuedFunction} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -423,6 +423,12 @@ class PlanParserSuite extends PlanTest { assertEqual("table d.t", table("d", "t")) } + test("table valued function") { + assertEqual( + "select * from range(2)", + UnresolvedTableValuedFunction("range", Literal(2) :: Nil).select(star())) + } + test("inline table") { assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows( Seq('col1.int), diff --git a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql new file mode 100644 index 0000000000000..2e6dcd538b7ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql @@ -0,0 +1,20 @@ +-- unresolved function +select * from dummy(3); + +-- range call with end +select * from range(6 + cos(3)); + +-- range call with start and end +select * from range(5, 10); + +-- range call with step +select * from range(0, 10, 2); + +-- range call with numPartitions +select * from range(0, 10, 1, 200); + +-- range call error +select * from range(1, 1, 1, 1, 1); + +-- range call with null +select * from range(1, null); diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out new file mode 100644 index 0000000000000..d769bcef0aca7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 7 + + +-- !query 0 +select * from dummy(3) +-- !query 0 schema +struct<> +-- !query 0 output +org.apache.spark.sql.AnalysisException +could not resolve `dummy` to a table-valued function; line 1 pos 14 + + +-- !query 1 +select * from range(6 + cos(3)) +-- !query 1 schema +struct +-- !query 1 output +0 +1 +2 +3 +4 + + +-- !query 2 +select * from range(5, 10) +-- !query 2 schema +struct +-- !query 2 output +5 +6 +7 +8 +9 + + +-- !query 3 +select * from range(0, 10, 2) +-- !query 3 schema +struct +-- !query 3 output +0 +2 +4 +6 +8 + + +-- !query 4 +select * from range(0, 10, 1, 200) +-- !query 4 schema +struct +-- !query 4 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 5 +select * from range(1, 1, 1, 1, 1) +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +error: table-valued function range with alternatives: + (end: long) + (start: long, end: long) + (start: long, end: long, step: long) + (start: long, end: long, step: long, numPartitions: integer) +cannot be applied to: (integer, integer, integer, integer, integer); line 1 pos 14 + + +-- !query 6 +select * from range(1, null) +-- !query 6 schema +struct<> +-- !query 6 output +java.lang.IllegalArgumentException +Invalid arguments for resolved function: 1, null From c180d637a3caca0d4e46f4980c10d1005eb453bc Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Fri, 19 Aug 2016 09:19:47 +0800 Subject: [PATCH 1155/1470] [SPARK-16947][SQL] Support type coercion and foldable expression for inline tables This patch improves inline table support with the following: 1. Support type coercion. 2. Support using foldable expressions. Previously only literals were supported. 3. Improve error message handling. 4. Improve test coverage. Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql. Author: petermaxlee Closes #14676 from petermaxlee/SPARK-16947. (cherry picked from commit f5472dda51b980a726346587257c22873ff708e3) Signed-off-by: Reynold Xin --- .../sql/catalyst/analysis/Analyzer.scala | 1 + .../analysis/ResolveInlineTables.scala | 112 ++++++++++++++ .../sql/catalyst/analysis/TypeCoercion.scala | 2 +- .../sql/catalyst/analysis/unresolved.scala | 26 +++- .../sql/catalyst/parser/AstBuilder.scala | 41 ++--- .../analysis/ResolveInlineTablesSuite.scala | 101 ++++++++++++ .../sql/catalyst/parser/PlanParserSuite.scala | 22 +-- .../sql-tests/inputs/inline-table.sql | 48 ++++++ .../sql-tests/results/inline-table.sql.out | 145 ++++++++++++++++++ 9 files changed, 452 insertions(+), 46 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala create mode 100644 sql/core/src/test/resources/sql-tests/inputs/inline-table.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/inline-table.sql.out 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 e0b81666b5cb4..14e995e618bc3 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 @@ -108,6 +108,7 @@ class Analyzer( GlobalAggregates :: ResolveAggregateFunctions :: TimeWindowing :: + ResolveInlineTables :: TypeCoercion.typeCoercionRules ++ extendedResolutionRules : _*), Batch("Nondeterministic", Once, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala new file mode 100644 index 0000000000000..7323197b10f6e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala @@ -0,0 +1,112 @@ +/* + * 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.analysis + +import scala.util.control.NonFatal + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Cast +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.types.{StructField, StructType} + +/** + * An analyzer rule that replaces [[UnresolvedInlineTable]] with [[LocalRelation]]. + */ +object ResolveInlineTables extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case table: UnresolvedInlineTable if table.expressionsResolved => + validateInputDimension(table) + validateInputEvaluable(table) + convert(table) + } + + /** + * Validates the input data dimension: + * 1. All rows have the same cardinality. + * 2. The number of column aliases defined is consistent with the number of columns in data. + * + * This is package visible for unit testing. + */ + private[analysis] def validateInputDimension(table: UnresolvedInlineTable): Unit = { + if (table.rows.nonEmpty) { + val numCols = table.names.size + table.rows.zipWithIndex.foreach { case (row, ri) => + if (row.size != numCols) { + table.failAnalysis(s"expected $numCols columns but found ${row.size} columns in row $ri") + } + } + } + } + + /** + * Validates that all inline table data are valid expressions that can be evaluated + * (in this they must be foldable). + * + * This is package visible for unit testing. + */ + private[analysis] def validateInputEvaluable(table: UnresolvedInlineTable): Unit = { + table.rows.foreach { row => + row.foreach { e => + // Note that nondeterministic expressions are not supported since they are not foldable. + if (!e.resolved || !e.foldable) { + e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline table definition") + } + } + } + } + + /** + * Convert a valid (with right shape and foldable inputs) [[UnresolvedInlineTable]] + * into a [[LocalRelation]]. + * + * This function attempts to coerce inputs into consistent types. + * + * This is package visible for unit testing. + */ + private[analysis] def convert(table: UnresolvedInlineTable): LocalRelation = { + // For each column, traverse all the values and find a common data type and nullability. + val fields = table.rows.transpose.zip(table.names).map { case (column, name) => + val inputTypes = column.map(_.dataType) + val tpe = TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse { + table.failAnalysis(s"incompatible types found in column $name for inline table") + } + StructField(name, tpe, nullable = column.exists(_.nullable)) + } + val attributes = StructType(fields).toAttributes + assert(fields.size == table.names.size) + + val newRows: Seq[InternalRow] = table.rows.map { row => + InternalRow.fromSeq(row.zipWithIndex.map { case (e, ci) => + val targetType = fields(ci).dataType + try { + if (e.dataType.sameType(targetType)) { + e.eval() + } else { + Cast(e, targetType).eval() + } + } catch { + case NonFatal(ex) => + table.failAnalysis(s"failed to evaluate expression ${e.sql}: ${ex.getMessage}") + } + }) + } + + LocalRelation(attributes, newRows) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 21e96aaf53844..193c3ec4e585a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -150,7 +150,7 @@ object TypeCoercion { * [[findTightestCommonType]], but can handle decimal types. If the wider decimal type exceeds * system limitation, this rule will truncate the decimal type before return it. */ - private def findWiderTypeWithoutStringPromotion(types: Seq[DataType]): Option[DataType] = { + def findWiderTypeWithoutStringPromotion(types: Seq[DataType]): Option[DataType] = { types.foldLeft[Option[DataType]](Some(NullType))((r, c) => r match { case Some(d) => findTightestCommonTypeOfTwo(d, c).orElse((d, c) match { case (t1: DecimalType, t2: DecimalType) => 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 a66a551bb21f8..15239b99c946c 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 @@ -50,10 +50,30 @@ case class UnresolvedRelation( } /** - * Holds a table-valued function call that has yet to be resolved. + * An inline table that has not been resolved yet. Once resolved, it is turned by the analyzer into + * a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]]. + * + * @param names list of column names + * @param rows expressions for the data + */ +case class UnresolvedInlineTable( + names: Seq[String], + rows: Seq[Seq[Expression]]) + extends LeafNode { + + lazy val expressionsResolved: Boolean = rows.forall(_.forall(_.resolved)) + override lazy val resolved = false + override def output: Seq[Attribute] = Nil +} + +/** + * A table-valued function, e.g. + * {{{ + * select * from range(10); + * }}} */ -case class UnresolvedTableValuedFunction( - functionName: String, functionArgs: Seq[Expression]) extends LeafNode { +case class UnresolvedTableValuedFunction(functionName: String, functionArgs: Seq[Expression]) + extends LeafNode { override def output: Seq[Attribute] = Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1f2cd4c4691ae..02302941630c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -665,39 +665,24 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { */ override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) { // Get the backing expressions. - val expressions = ctx.expression.asScala.map { eCtx => - val e = expression(eCtx) - validate(e.foldable, "All expressions in an inline table must be constants.", eCtx) - e - } - - // Validate and evaluate the rows. - val (structType, structConstructor) = expressions.head.dataType match { - case st: StructType => - (st, (e: Expression) => e) - case dt => - val st = CreateStruct(Seq(expressions.head)).dataType - (st, (e: Expression) => CreateStruct(Seq(e))) - } - val rows = expressions.map { - case expression => - val safe = Cast(structConstructor(expression), structType) - safe.eval().asInstanceOf[InternalRow] + val rows = ctx.expression.asScala.map { e => + expression(e) match { + // inline table comes in two styles: + // style 1: values (1), (2), (3) -- multiple columns are supported + // style 2: values 1, 2, 3 -- only a single column is supported here + case CreateStruct(children) => children // style 1 + case child => Seq(child) // style 2 + } } - // Construct attributes. - val baseAttributes = structType.toAttributes.map(_.withNullability(true)) - val attributes = if (ctx.identifierList != null) { - val aliases = visitIdentifierList(ctx.identifierList) - validate(aliases.size == baseAttributes.size, - "Number of aliases must match the number of fields in an inline table.", ctx) - baseAttributes.zip(aliases).map(p => p._1.withName(p._2)) + val aliases = if (ctx.identifierList != null) { + visitIdentifierList(ctx.identifierList) } else { - baseAttributes + Seq.tabulate(rows.head.size)(i => s"col${i + 1}") } - // Create plan and add an alias if a name has been defined. - LocalRelation(attributes, rows).optionalMap(ctx.identifier)(aliasPlan) + val table = UnresolvedInlineTable(aliases, rows) + table.optionalMap(ctx.identifier)(aliasPlan) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala new file mode 100644 index 0000000000000..920c6ea50f4ba --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{Literal, Rand} +import org.apache.spark.sql.catalyst.expressions.aggregate.Count +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.types.{LongType, NullType} + +/** + * Unit tests for [[ResolveInlineTables]]. Note that there are also test cases defined in + * end-to-end tests (in sql/core module) for verifying the correct error messages are shown + * in negative cases. + */ +class ResolveInlineTablesSuite extends PlanTest with BeforeAndAfter { + + private def lit(v: Any): Literal = Literal(v) + + test("validate inputs are foldable") { + ResolveInlineTables.validateInputEvaluable( + UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(lit(1))))) + + // nondeterministic (rand) should not work + intercept[AnalysisException] { + ResolveInlineTables.validateInputEvaluable( + UnresolvedInlineTable(Seq("c1"), Seq(Seq(Rand(1))))) + } + + // aggregate should not work + intercept[AnalysisException] { + ResolveInlineTables.validateInputEvaluable( + UnresolvedInlineTable(Seq("c1"), Seq(Seq(Count(lit(1)))))) + } + + // unresolved attribute should not work + intercept[AnalysisException] { + ResolveInlineTables.validateInputEvaluable( + UnresolvedInlineTable(Seq("c1"), Seq(Seq(UnresolvedAttribute("A"))))) + } + } + + test("validate input dimensions") { + ResolveInlineTables.validateInputDimension( + UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2))))) + + // num alias != data dimension + intercept[AnalysisException] { + ResolveInlineTables.validateInputDimension( + UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(lit(1)), Seq(lit(2))))) + } + + // num alias == data dimension, but data themselves are inconsistent + intercept[AnalysisException] { + ResolveInlineTables.validateInputDimension( + UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(21), lit(22))))) + } + } + + test("do not fire the rule if not all expressions are resolved") { + val table = UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(UnresolvedAttribute("A")))) + assert(ResolveInlineTables(table) == table) + } + + test("convert") { + val table = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2L)))) + val converted = ResolveInlineTables.convert(table) + + assert(converted.output.map(_.dataType) == Seq(LongType)) + assert(converted.data.size == 2) + assert(converted.data(0).getLong(0) == 1L) + assert(converted.data(1).getLong(0) == 2L) + } + + test("nullability inference in convert") { + val table1 = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2L)))) + val converted1 = ResolveInlineTables.convert(table1) + assert(!converted1.schema.fields(0).nullable) + + val table2 = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(Literal(null, NullType)))) + val converted2 = ResolveInlineTables.convert(table2) + assert(converted2.schema.fields(0).nullable) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 05bd0cd0380ff..7fe056a588376 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -17,9 +17,8 @@ package org.apache.spark.sql.catalyst.parser -import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.analysis.{UnresolvedGenerator, UnresolvedTableValuedFunction} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedGenerator, UnresolvedInlineTable, UnresolvedTableValuedFunction} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -430,19 +429,14 @@ class PlanParserSuite extends PlanTest { } test("inline table") { - assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows( - Seq('col1.int), - Seq(1, 2, 3, 4).map(x => Row(x)))) + assertEqual("values 1, 2, 3, 4", + UnresolvedInlineTable(Seq("col1"), Seq(1, 2, 3, 4).map(x => Seq(Literal(x))))) + assertEqual( - "values (1, 'a'), (2, 'b'), (3, 'c') as tbl(a, b)", - LocalRelation.fromExternalRows( - Seq('a.int, 'b.string), - Seq((1, "a"), (2, "b"), (3, "c")).map(x => Row(x._1, x._2))).as("tbl")) - intercept("values (a, 'a'), (b, 'b')", - "All expressions in an inline table must be constants.") - intercept("values (1, 'a'), (2, 'b') as tbl(a, b, c)", - "Number of aliases must match the number of fields in an inline table.") - intercept[ArrayIndexOutOfBoundsException](parsePlan("values (1, 'a'), (2, 'b', 5Y)")) + "values (1, 'a'), (2, 'b') as tbl(a, b)", + UnresolvedInlineTable( + Seq("a", "b"), + Seq(Literal(1), Literal("a")) :: Seq(Literal(2), Literal("b")) :: Nil).as("tbl")) } test("simple select query with !> and !<") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql new file mode 100644 index 0000000000000..5107fa4d55537 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql @@ -0,0 +1,48 @@ + +-- single row, without table and column alias +select * from values ("one", 1); + +-- single row, without column alias +select * from values ("one", 1) as data; + +-- single row +select * from values ("one", 1) as data(a, b); + +-- single column multiple rows +select * from values 1, 2, 3 as data(a); + +-- three rows +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b); + +-- null type +select * from values ("one", null), ("two", null) as data(a, b); + +-- int and long coercion +select * from values ("one", 1), ("two", 2L) as data(a, b); + +-- foldable expressions +select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b); + +-- complex types +select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b); + +-- decimal and double coercion +select * from values ("one", 2.0), ("two", 3.0D) as data(a, b); + +-- error reporting: nondeterministic function rand +select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b); + +-- error reporting: different number of columns +select * from values ("one", 2.0), ("two") as data(a, b); + +-- error reporting: types that are incompatible +select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b); + +-- error reporting: number aliases different from number data values +select * from values ("one"), ("two") as data(a, b); + +-- error reporting: unresolved expression +select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b); + +-- error reporting: aggregate expression +select * from values ("one", count(1)), ("two", 2) as data(a, b); diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out new file mode 100644 index 0000000000000..de6f01b8de772 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -0,0 +1,145 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 16 + + +-- !query 0 +select * from values ("one", 1) +-- !query 0 schema +struct +-- !query 0 output +one 1 + + +-- !query 1 +select * from values ("one", 1) as data +-- !query 1 schema +struct +-- !query 1 output +one 1 + + +-- !query 2 +select * from values ("one", 1) as data(a, b) +-- !query 2 schema +struct +-- !query 2 output +one 1 + + +-- !query 3 +select * from values 1, 2, 3 as data(a) +-- !query 3 schema +struct +-- !query 3 output +1 +2 +3 + + +-- !query 4 +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query 4 schema +struct +-- !query 4 output +one 1 +three NULL +two 2 + + +-- !query 5 +select * from values ("one", null), ("two", null) as data(a, b) +-- !query 5 schema +struct +-- !query 5 output +one NULL +two NULL + + +-- !query 6 +select * from values ("one", 1), ("two", 2L) as data(a, b) +-- !query 6 schema +struct +-- !query 6 output +one 1 +two 2 + + +-- !query 7 +select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query 7 schema +struct +-- !query 7 output +one 1 +two 4 + + +-- !query 8 +select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query 8 schema +struct> +-- !query 8 output +one [0,1] +two [2,3] + + +-- !query 9 +select * from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query 9 schema +struct +-- !query 9 output +one 2.0 +two 3.0 + + +-- !query 10 +select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression rand(5) in inline table definition; line 1 pos 29 + + +-- !query 11 +select * from values ("one", 2.0), ("two") as data(a, b) +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 1; line 1 pos 14 + + +-- !query 12 +select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +incompatible types found in column b for inline table; line 1 pos 14 + + +-- !query 13 +select * from values ("one"), ("two") as data(a, b) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 0; line 1 pos 14 + + +-- !query 14 +select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 29 + + +-- !query 15 +select * from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression count(1) in inline table definition; line 1 pos 29 From 05b180faa4bd87498516c05d4769cc2f51d56aae Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 18 Aug 2016 19:02:32 -0700 Subject: [PATCH 1156/1470] HOTFIX: compilation broken due to protected ctor. (cherry picked from commit b482c09fa22c5762a355f95820e4ba3e2517fb77) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/expressions/literals.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 95ed68fbb0528..7040008769a32 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 @@ -163,8 +163,7 @@ object DecimalLiteral { /** * In order to do type checking, use Literal.create() instead of constructor */ -case class Literal protected (value: Any, dataType: DataType) - extends LeafExpression with CodegenFallback { +case class Literal (value: Any, dataType: DataType) extends LeafExpression with CodegenFallback { override def foldable: Boolean = true override def nullable: Boolean = value == null From d55d1f454e6739ccff9c748f78462d789b09991f Mon Sep 17 00:00:00 2001 From: Nick Lavers Date: Fri, 19 Aug 2016 10:11:59 +0100 Subject: [PATCH 1157/1470] [SPARK-16961][CORE] Fixed off-by-one error that biased randomizeInPlace JIRA issue link: https://issues.apache.org/jira/browse/SPARK-16961 Changed one line of Utils.randomizeInPlace to allow elements to stay in place. Created a unit test that runs a Pearson's chi squared test to determine whether the output diverges significantly from a uniform distribution. Author: Nick Lavers Closes #14551 from nicklavers/SPARK-16961-randomizeInPlace. (cherry picked from commit 5377fc62360d5e9b5c94078e41d10a96e0e8a535) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 35 +++++++++++++++++++ python/pyspark/ml/clustering.py | 12 +++---- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/tests.py | 2 +- 5 files changed, 44 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index be1ae401d9504..a0ef30e16bd5c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -824,7 +824,7 @@ private[spark] object Utils extends Logging { */ def randomizeInPlace[T](arr: Array[T], rand: Random = new Random): Array[T] = { for (i <- (arr.length - 1) to 1 by -1) { - val j = rand.nextInt(i) + val j = rand.nextInt(i + 1) val tmp = arr(j) arr(j) = arr(i) arr(i) = tmp 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 30952a9458345..4715fd29375d6 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -31,6 +31,7 @@ import scala.util.Random import com.google.common.io.Files import org.apache.commons.lang3.SystemUtils +import org.apache.commons.math3.stat.inference.ChiSquareTest import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -874,4 +875,38 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { } } } + + test("chi square test of randomizeInPlace") { + // Parameters + val arraySize = 10 + val numTrials = 1000 + val threshold = 0.05 + val seed = 1L + + // results(i)(j): how many times Utils.randomize moves an element from position j to position i + val results = Array.ofDim[Long](arraySize, arraySize) + + // This must be seeded because even a fair random process will fail this test with + // probability equal to the value of `threshold`, which is inconvenient for a unit test. + val rand = new java.util.Random(seed) + val range = 0 until arraySize + + for { + _ <- 0 until numTrials + trial = Utils.randomizeInPlace(range.toArray, rand) + i <- range + } results(i)(trial(i)) += 1L + + val chi = new ChiSquareTest() + + // We expect an even distribution; this array will be rescaled by `chiSquareTest` + val expected = Array.fill(arraySize * arraySize)(1.0) + val observed = results.flatten + + // Performs Pearson's chi-squared test. Using the sum-of-squares as the test statistic, gives + // the probability of a uniform distribution producing results as extreme as `observed` + val pValue = chi.chiSquareTest(expected, observed) + + assert(pValue > threshold) + } } diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 75d9a0e8cac18..4dab83362a0a4 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -99,9 +99,9 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte +--------------------+--------------------+ | mean| cov| +--------------------+--------------------+ - |[-0.0550000000000...|0.002025000000000...| - |[0.82499999999999...|0.005625000000000...| - |[-0.87,-0.7200000...|0.001600000000000...| + |[0.82500000140229...|0.005625000000006...| + |[-0.4777098016092...|0.167969502720916...| + |[-0.4472625243352...|0.167304119758233...| +--------------------+--------------------+ ... >>> transformed = model.transform(df).select("features", "prediction") @@ -124,9 +124,9 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte +--------------------+--------------------+ | mean| cov| +--------------------+--------------------+ - |[-0.0550000000000...|0.002025000000000...| - |[0.82499999999999...|0.005625000000000...| - |[-0.87,-0.7200000...|0.001600000000000...| + |[0.82500000140229...|0.005625000000006...| + |[-0.4777098016092...|0.167969502720916...| + |[-0.4472625243352...|0.167304119758233...| +--------------------+--------------------+ ... diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index c8c3c42774f21..29aa615125770 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -416,7 +416,7 @@ class GaussianMixtureModel(JavaModelWrapper, JavaSaveable, JavaLoader): ... 4.5605, 5.2043, 6.2734]) >>> clusterdata_2 = sc.parallelize(data.reshape(5,3)) >>> model = GaussianMixture.train(clusterdata_2, 2, convergenceTol=0.0001, - ... maxIterations=150, seed=10) + ... maxIterations=150, seed=4) >>> labels = model.predict(clusterdata_2).collect() >>> labels[0]==labels[1] True diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 99bf50b5a1640..3f3dfd186c10d 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -550,7 +550,7 @@ def test_gmm(self): [-6, -7], ]) clusters = GaussianMixture.train(data, 2, convergenceTol=0.001, - maxIterations=10, seed=56) + maxIterations=10, seed=1) labels = clusters.predict(data).collect() self.assertEqual(labels[0], labels[1]) self.assertEqual(labels[2], labels[3]) From e0c60f1850706faf2830b09af3dc6b52ffd9991e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 19 Aug 2016 21:11:35 +0800 Subject: [PATCH 1158/1470] [SPARK-16994][SQL] Whitelist operators for predicate pushdown ## What changes were proposed in this pull request? This patch changes predicate pushdown optimization rule (PushDownPredicate) from using a blacklist to a whitelist. That is to say, operators must be explicitly allowed. This approach is more future-proof: previously it was possible for us to introduce a new operator and then render the optimization rule incorrect. This also fixes the bug that previously we allowed pushing filter beneath limit, which was incorrect. That is to say, before this patch, the optimizer would rewrite ``` select * from (select * from range(10) limit 5) where id > 3 to select * from range(10) where id > 3 limit 5 ``` ## How was this patch tested? - a unit test case in FilterPushdownSuite - an end-to-end test in limit.sql Author: Reynold Xin Closes #14713 from rxin/SPARK-16994. (cherry picked from commit 67e59d464f782ff5f509234212aa072a7653d7bf) Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 23 ++++++++++++++----- .../optimizer/FilterPushdownSuite.scala | 6 +++++ .../test/resources/sql-tests/inputs/limit.sql | 3 +++ .../resources/sql-tests/results/limit.sql.out | 10 +++++++- 4 files changed, 35 insertions(+), 7 deletions(-) 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 627f82994f8db..19d3c39aeb616 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 @@ -1133,17 +1133,28 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { filter } - // two filters should be combine together by other rules - case filter @ Filter(_, _: Filter) => filter - // should not push predicates through sample, or will generate different results. - case filter @ Filter(_, _: Sample) => filter - - case filter @ Filter(condition, u: UnaryNode) if u.expressions.forall(_.deterministic) => + case filter @ Filter(condition, u: UnaryNode) + if canPushThrough(u) && u.expressions.forall(_.deterministic) => pushDownPredicate(filter, u.child) { predicate => u.withNewChildren(Seq(Filter(predicate, u.child))) } } + private def canPushThrough(p: UnaryNode): Boolean = p match { + // Note that some operators (e.g. project, aggregate, union) are being handled separately + // (earlier in this rule). + case _: AppendColumns => true + case _: BroadcastHint => true + case _: Distinct => true + case _: Generate => true + case _: Pivot => true + case _: RedistributeData => true + case _: Repartition => true + case _: ScriptTransformation => true + case _: Sort => true + case _ => false + } + private def pushDownPredicate( filter: Filter, grandchild: LogicalPlan)(insertFilter: Expression => LogicalPlan): LogicalPlan = { 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 9cb49e74ad34f..0e5c2acd4d327 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 @@ -112,6 +112,12 @@ class FilterPushdownSuite extends PlanTest { assert(optimized == correctAnswer) } + test("SPARK-16994: filter should not be pushed through limit") { + val originalQuery = testRelation.limit(10).where('a === 1).analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, originalQuery) + } + test("can't push without rewrite") { val originalQuery = testRelation diff --git a/sql/core/src/test/resources/sql-tests/inputs/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/limit.sql index 892a1bb4b559f..2ea35f7f3a5c8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/limit.sql @@ -18,3 +18,6 @@ select * from testdata limit key > 3; -- limit must be integer select * from testdata limit true; select * from testdata limit 'a'; + +-- limit within a subquery +select * from (select * from range(10) limit 5) where id > 3; diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/limit.sql.out index b71b05886986c..cb4e4d04810d0 100644 --- a/sql/core/src/test/resources/sql-tests/results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 9 +-- Number of queries: 10 -- !query 0 @@ -81,3 +81,11 @@ struct<> -- !query 8 output org.apache.spark.sql.AnalysisException The limit expression must be integer type, but got string; + + +-- !query 9 +select * from (select * from range(10) limit 5) where id > 3 +-- !query 9 schema +struct +-- !query 9 output +4 From d0707c6baeb4003735a508f981111db370984354 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 19 Aug 2016 10:11:25 -0500 Subject: [PATCH 1159/1470] [SPARK-11227][CORE] UnknownHostException can be thrown when NameNode HA is enabled. ## What changes were proposed in this pull request? If the following conditions are satisfied, executors don't load properties in `hdfs-site.xml` and UnknownHostException can be thrown. (1) NameNode HA is enabled (2) spark.eventLogging is disabled or logging path is NOT on HDFS (3) Using Standalone or Mesos for the cluster manager (4) There are no code to load `HdfsCondition` class in the driver regardless of directly or indirectly. (5) The tasks access to HDFS (There might be some more conditions...) For example, following code causes UnknownHostException when the conditions above are satisfied. ``` sc.textFile("").collect ``` ``` java.lang.IllegalArgumentException: java.net.UnknownHostException: hacluster at org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:378) at org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:310) at org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:176) at org.apache.hadoop.hdfs.DFSClient.(DFSClient.java:678) at org.apache.hadoop.hdfs.DFSClient.(DFSClient.java:619) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:149) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2653) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:92) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2687) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2669) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:371) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:170) at org.apache.hadoop.mapred.JobConf.getWorkingDirectory(JobConf.java:656) at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:438) at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:411) at org.apache.spark.SparkContext$$anonfun$hadoopFile$1$$anonfun$32.apply(SparkContext.scala:986) at org.apache.spark.SparkContext$$anonfun$hadoopFile$1$$anonfun$32.apply(SparkContext.scala:986) at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:177) at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:177) at scala.Option.map(Option.scala:146) at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:177) at org.apache.spark.rdd.HadoopRDD$$anon$1.(HadoopRDD.scala:213) at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:209) at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:102) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:85) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.net.UnknownHostException: hacluster ``` But following code doesn't cause the Exception because `textFile` method loads `HdfsConfiguration` indirectly. ``` sc.textFile("").collect ``` When a job includes some operations which access to HDFS, the object of `org.apache.hadoop.Configuration` is wrapped by `SerializableConfiguration`, serialized and broadcasted from driver to executors and each executor deserialize the object with `loadDefaults` false so HDFS related properties should be set before broadcasted. ## How was this patch tested? Tested manually on my standalone cluster. Author: Kousuke Saruta Closes #13738 from sarutak/SPARK-11227. (cherry picked from commit 071eaaf9d2b63589f2e66e5279a16a5a484de6f5) Signed-off-by: Tom Graves --- .../scala/org/apache/spark/SparkContext.scala | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 699dc516b17af..37e067839775a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -35,7 +35,7 @@ import scala.util.control.NonFatal import com.google.common.collect.MapMaker import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, @@ -960,6 +960,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli valueClass: Class[V], minPartitions: Int = defaultMinPartitions): RDD[(K, V)] = withScope { assertNotStopped() + + // This is a hack to enforce loading hdfs-site.xml. + // See SPARK-11227 for details. + FileSystem.getLocal(conf) + // Add necessary security credentials to the JobConf before broadcasting it. SparkHadoopUtil.get.addCredentials(conf) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minPartitions) @@ -980,6 +985,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli valueClass: Class[V], minPartitions: Int = defaultMinPartitions): RDD[(K, V)] = withScope { assertNotStopped() + + // This is a hack to enforce loading hdfs-site.xml. + // See SPARK-11227 for details. + FileSystem.get(new URI(path), hadoopConfiguration) + // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it. val confBroadcast = broadcast(new SerializableConfiguration(hadoopConfiguration)) val setInputPathsFunc = (jobConf: JobConf) => FileInputFormat.setInputPaths(jobConf, path) @@ -1064,6 +1074,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli vClass: Class[V], conf: Configuration = hadoopConfiguration): RDD[(K, V)] = withScope { assertNotStopped() + + // This is a hack to enforce loading hdfs-site.xml. + // See SPARK-11227 for details. + FileSystem.get(new URI(path), hadoopConfiguration) + // The call to NewHadoopJob automatically adds security credentials to conf, // so we don't need to explicitly add them ourselves val job = NewHadoopJob.getInstance(conf) @@ -1098,6 +1113,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli kClass: Class[K], vClass: Class[V]): RDD[(K, V)] = withScope { assertNotStopped() + + // This is a hack to enforce loading hdfs-site.xml. + // See SPARK-11227 for details. + FileSystem.getLocal(conf) + // Add necessary security credentials to the JobConf. Required to access secure HDFS. val jconf = new JobConf(conf) SparkHadoopUtil.get.addCredentials(jconf) From 3276ccfac807514d5a959415bcf58d2aa6ed8fbc Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 26 Jul 2016 12:00:01 +0800 Subject: [PATCH 1160/1470] [SPARK-16686][SQL] Remove PushProjectThroughSample since it is handled by ColumnPruning We push down `Project` through `Sample` in `Optimizer` by the rule `PushProjectThroughSample`. However, if the projected columns produce new output, they will encounter whole data instead of sampled data. It will bring some inconsistency between original plan (Sample then Project) and optimized plan (Project then Sample). In the extreme case such as attached in the JIRA, if the projected column is an UDF which is supposed to not see the sampled out data, the result of UDF will be incorrect. Since the rule `ColumnPruning` already handles general `Project` pushdown. We don't need `PushProjectThroughSample` anymore. The rule `ColumnPruning` also avoids the described issue. Jenkins tests. Author: Liang-Chi Hsieh Closes #14327 from viirya/fix-sample-pushdown. (cherry picked from commit 7b06a8948fc16d3c14e240fdd632b79ce1651008) Signed-off-by: Reynold Xin --- .../sql/catalyst/optimizer/Optimizer.scala | 12 --------- .../optimizer/ColumnPruningSuite.scala | 15 +++++++++++ .../optimizer/FilterPushdownSuite.scala | 17 ------------- .../org/apache/spark/sql/DatasetSuite.scala | 25 +++++++++++++++++++ 4 files changed, 40 insertions(+), 29 deletions(-) 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 19d3c39aeb616..88cc0e47fc8de 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 @@ -75,7 +75,6 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("Operator Optimizations", fixedPoint, // Operator push down PushThroughSetOperations, - PushProjectThroughSample, ReorderJoin, EliminateOuterJoin, PushPredicateThroughJoin, @@ -146,17 +145,6 @@ class SimpleTestOptimizer extends Optimizer( new SimpleCatalystConf(caseSensitiveAnalysis = true)), new SimpleCatalystConf(caseSensitiveAnalysis = true)) -/** - * Pushes projects down beneath Sample to enable column pruning with sampling. - */ -object PushProjectThroughSample extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - // Push down projection into sample - case Project(projectList, Sample(lb, up, replace, seed, child)) => - Sample(lb, up, replace, seed, Project(projectList, child))() - } -} - /** * Removes the Project only conducting Alias of its child node. * It is created mainly for removing extra Project added in EliminateSerialization rule, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala index b5664a5e699e8..589607e3ad5cb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -346,5 +346,20 @@ class ColumnPruningSuite extends PlanTest { comparePlans(Optimize.execute(plan1.analyze), correctAnswer1) } + test("push project down into sample") { + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val x = testRelation.subquery('x) + + val query1 = Sample(0.0, 0.6, false, 11L, x)().select('a) + val optimized1 = Optimize.execute(query1.analyze) + val expected1 = Sample(0.0, 0.6, false, 11L, x.select('a))() + comparePlans(optimized1, expected1.analyze) + + val query2 = Sample(0.0, 0.6, false, 11L, x)().select('a as 'aa) + val optimized2 = Optimize.execute(query2.analyze) + val expected2 = Sample(0.0, 0.6, false, 11L, x.select('a))().select('a as 'aa) + comparePlans(optimized2, expected2.analyze) + } + // todo: add more tests for column pruning } 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 0e5c2acd4d327..1dcabf9dcedd1 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 @@ -34,7 +34,6 @@ class FilterPushdownSuite extends PlanTest { Batch("Subqueries", Once, EliminateSubqueryAliases) :: Batch("Filter Pushdown", FixedPoint(10), - PushProjectThroughSample, CombineFilters, PushDownPredicate, BooleanSimplification, @@ -591,22 +590,6 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, originalQuery) } - test("push project and filter down into sample") { - val x = testRelation.subquery('x) - val originalQuery = - Sample(0.0, 0.6, false, 11L, x)().select('a) - - val originalQueryAnalyzed = - EliminateSubqueryAliases(analysis.SimpleAnalyzer.execute(originalQuery)) - - val optimized = Optimize.execute(originalQueryAnalyzed) - - val correctAnswer = - Sample(0.0, 0.6, false, 11L, x.select('a))() - - comparePlans(optimized, correctAnswer.analyze) - } - test("aggregate: push down filter when filter on group by expression") { val originalQuery = testRelation .groupBy('a)('a, count('b) as 'c) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 9aeeda4463afc..f897cfb26d3ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -422,6 +422,31 @@ class DatasetSuite extends QueryTest with SharedSQLContext { 3, 17, 27, 58, 62) } + test("SPARK-16686: Dataset.sample with seed results shouldn't depend on downstream usage") { + val simpleUdf = udf((n: Int) => { + require(n != 1, "simpleUdf shouldn't see id=1!") + 1 + }) + + val df = Seq( + (0, "string0"), + (1, "string1"), + (2, "string2"), + (3, "string3"), + (4, "string4"), + (5, "string5"), + (6, "string6"), + (7, "string7"), + (8, "string8"), + (9, "string9") + ).toDF("id", "stringData") + val sampleDF = df.sample(false, 0.7, 50) + // After sampling, sampleDF doesn't contain id=1. + assert(!sampleDF.select("id").collect.contains(1)) + // simpleUdf should not encounter id=1. + checkAnswer(sampleDF.select(simpleUdf($"id")), List.fill(sampleDF.count.toInt)(Row(1))) + } + test("SPARK-11436: we should rebind right encoder when join 2 datasets") { val ds1 = Seq("1", "2").toDS().as("a") val ds2 = Seq(2, 3).toDS().as("b") From ae89c8e170dd77e0b2adc04a2c85577f6df5cdef Mon Sep 17 00:00:00 2001 From: Sital Kedia Date: Fri, 19 Aug 2016 11:27:30 -0700 Subject: [PATCH 1161/1470] [SPARK-17113] [SHUFFLE] Job failure due to Executor OOM in offheap mode ## What changes were proposed in this pull request? This PR fixes executor OOM in offheap mode due to bug in Cooperative Memory Management for UnsafeExternSorter. UnsafeExternalSorter was checking if memory page is being used by upstream by comparing the base object address of the current page with the base object address of upstream. However, in case of offheap memory allocation, the base object addresses are always null, so there was no spilling happening and eventually the operator would OOM. Following is the stack trace this issue addresses - java.lang.OutOfMemoryError: Unable to acquire 1220 bytes of memory, got 0 at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:341) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:362) at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:93) at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:170) ## How was this patch tested? Tested by running the failing job. Author: Sital Kedia Closes #14693 from sitalkedia/fix_offheap_oom. (cherry picked from commit cf0cce90364d17afe780ff9a5426dfcefa298535) Signed-off-by: Davies Liu --- .../util/collection/unsafe/sort/UnsafeExternalSorter.java | 2 +- .../util/collection/unsafe/sort/UnsafeInMemorySorter.java | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 50f5b068b276d..0d67167569944 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -521,7 +521,7 @@ public long spill() throws IOException { // is accessing the current record. We free this page in that caller's next loadNext() // call. for (MemoryBlock page : allocatedPages) { - if (!loaded || page.getBaseObject() != upstream.getBaseObject()) { + if (!loaded || page.pageNumber != ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) { released += page.size(); freePage(page); } else { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index c7b070f519f88..9710529f2d052 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -225,6 +225,7 @@ public final class SortedIterator extends UnsafeSorterIterator implements Clonea private long baseOffset; private long keyPrefix; private int recordLength; + private long currentPageNumber; private SortedIterator(int numRecords, int offset) { this.numRecords = numRecords; @@ -239,6 +240,7 @@ public SortedIterator clone() { iter.baseOffset = baseOffset; iter.keyPrefix = keyPrefix; iter.recordLength = recordLength; + iter.currentPageNumber = currentPageNumber; return iter; } @@ -256,6 +258,7 @@ public boolean hasNext() { public void loadNext() { // This pointer points to a 4-byte record length, followed by the record's bytes final long recordPointer = array.get(offset + position); + currentPageNumber = memoryManager.decodePageNumber(recordPointer); baseObject = memoryManager.getPage(recordPointer); baseOffset = memoryManager.getOffsetInPage(recordPointer) + 4; // Skip over record length recordLength = Platform.getInt(baseObject, baseOffset - 4); @@ -269,6 +272,10 @@ public void loadNext() { @Override public long getBaseOffset() { return baseOffset; } + public long getCurrentPageNumber() { + return currentPageNumber; + } + @Override public int getRecordLength() { return recordLength; } From efe832200f2fdf90868f5d03b45f1d75502444b3 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Fri, 19 Aug 2016 18:14:45 -0700 Subject: [PATCH 1162/1470] [SPARK-17149][SQL] array.sql for testing array related functions ## What changes were proposed in this pull request? This patch creates array.sql in SQLQueryTestSuite for testing array related functions, including: - indexing - array creation - size - array_contains - sort_array ## How was this patch tested? The patch itself is about adding tests. Author: petermaxlee Closes #14708 from petermaxlee/SPARK-17149. (cherry picked from commit a117afa7c2d94f943106542ec53d74ba2b5f1058) Signed-off-by: Reynold Xin --- .../catalyst/analysis/FunctionRegistry.scala | 12 +- .../test/resources/sql-tests/inputs/array.sql | 86 +++++++++++ .../resources/sql-tests/results/array.sql.out | 144 ++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 16 -- .../apache/spark/sql/SQLQueryTestSuite.scala | 10 ++ .../execution/HiveCompatibilitySuite.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 9 -- 7 files changed, 248 insertions(+), 33 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/array.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/array.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c5f91c1590542..35fd800df4a4f 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 @@ -161,7 +161,6 @@ object FunctionRegistry { val expressions: Map[String, (ExpressionInfo, FunctionBuilder)] = Map( // misc non-aggregate functions expression[Abs]("abs"), - expression[CreateArray]("array"), expression[Coalesce]("coalesce"), expression[Explode]("explode"), expression[Greatest]("greatest"), @@ -172,10 +171,6 @@ object FunctionRegistry { expression[IsNull]("isnull"), expression[IsNotNull]("isnotnull"), expression[Least]("least"), - expression[CreateMap]("map"), - expression[MapKeys]("map_keys"), - expression[MapValues]("map_values"), - expression[CreateNamedStruct]("named_struct"), expression[NaNvl]("nanvl"), expression[NullIf]("nullif"), expression[Nvl]("nvl"), @@ -184,7 +179,6 @@ object FunctionRegistry { expression[Rand]("rand"), expression[Randn]("randn"), expression[Stack]("stack"), - expression[CreateStruct]("struct"), expression[CaseWhen]("when"), // math functions @@ -354,9 +348,15 @@ object FunctionRegistry { expression[TimeWindow]("window"), // collection functions + expression[CreateArray]("array"), expression[ArrayContains]("array_contains"), + expression[CreateMap]("map"), + expression[CreateNamedStruct]("named_struct"), + expression[MapKeys]("map_keys"), + expression[MapValues]("map_values"), expression[Size]("size"), expression[SortArray]("sort_array"), + expression[CreateStruct]("struct"), // misc functions expression[AssertTrue]("assert_true"), diff --git a/sql/core/src/test/resources/sql-tests/inputs/array.sql b/sql/core/src/test/resources/sql-tests/inputs/array.sql new file mode 100644 index 0000000000000..4038a0da41d2b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/array.sql @@ -0,0 +1,86 @@ +-- test cases for array functions + +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c); + +select * from data; + +-- index into array +select a, b[0], b[0] + b[1] from data; + +-- index into array of arrays +select a, c[0][0] + c[0][0 + 1] from data; + + +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +); + +select * from primitive_arrays; + +-- array_contains on all primitive types: result should alternate between true and false +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays; + +-- array_contains on nested arrays +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data; + +-- sort_array +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays; + +-- size +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays; diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out new file mode 100644 index 0000000000000..4a1d149c1f362 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -0,0 +1,144 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 10 + + +-- !query 0 +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select * from data +-- !query 1 schema +struct,c:array>> +-- !query 1 output +one [11,12,13] [[111,112,113],[121,122,123]] +two [21,22,23] [[211,212,213],[221,222,223]] + + +-- !query 2 +select a, b[0], b[0] + b[1] from data +-- !query 2 schema +struct +-- !query 2 output +one 11 23 +two 21 43 + + +-- !query 3 +select a, c[0][0] + c[0][0 + 1] from data +-- !query 3 schema +struct +-- !query 3 output +one 223 +two 423 + + +-- !query 4 +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +select * from primitive_arrays +-- !query 5 schema +struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,date_array:array,timestamp_array:array> +-- !query 5 output +[true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00.0,2016-11-12 20:54:00.0] + + +-- !query 6 +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays +-- !query 6 schema +struct +-- !query 6 output +true false true false true false true false true false true false true false true false true false true false + + +-- !query 7 +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data +-- !query 7 schema +struct +-- !query 7 output +false false +true true + + +-- !query 8 +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays +-- !query 8 schema +struct,sort_array(tinyint_array, true):array,sort_array(smallint_array, true):array,sort_array(int_array, true):array,sort_array(bigint_array, true):array,sort_array(decimal_array, true):array,sort_array(double_array, true):array,sort_array(float_array, true):array,sort_array(date_array, true):array,sort_array(timestamp_array, true):array> +-- !query 8 output +[true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00.0,2016-11-15 20:54:00.0] + + +-- !query 9 +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays +-- !query 9 schema +struct +-- !query 9 output +1 2 2 2 2 2 2 2 2 2 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 de1a811d642bb..df72afb03c90c 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 @@ -445,12 +445,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Nil) } - test("index into array") { - checkAnswer( - sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), - arrayData.map(d => Row(d.data, d.data(0), d.data(0) + d.data(1), d.data(1))).collect()) - } - test("left semi greater than predicate") { withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { checkAnswer( @@ -472,16 +466,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } - test("index into array of arrays") { - checkAnswer( - sql( - "SELECT nestedData, nestedData[0][0], nestedData[0][0] + nestedData[0][1] FROM arrayData"), - arrayData.map(d => - Row(d.nestedData, - d.nestedData(0)(0), - d.nestedData(0)(0) + d.nestedData(0)(1))).collect().toSeq) - } - test("agg") { checkAnswer( sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 069a9b665eb36..55d5a56f1040a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -35,6 +35,16 @@ import org.apache.spark.sql.types.StructType * Each case is loaded from a file in "spark/sql/core/src/test/resources/sql-tests/inputs". * Each case has a golden result file in "spark/sql/core/src/test/resources/sql-tests/results". * + * To run the entire test suite: + * {{{ + * build/sbt "sql/test-only *SQLQueryTestSuite" + * }}} + * + * To run a single test file upon change: + * {{{ + * build/sbt "~sql/test-only *SQLQueryTestSuite -- -z inline-table.sql" + * }}} + * * To re-generate golden files, run: * {{{ * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" 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 13d18fdec0e9d..a54d234876256 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 @@ -979,8 +979,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_PI", "udf_acos", "udf_add", - "udf_array", - "udf_array_contains", + // "udf_array", -- done in array.sql + // "udf_array_contains", -- done in array.sql "udf_ascii", "udf_asin", "udf_atan", 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 f8c55ec45650e..db6c64a1a130a 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 @@ -216,15 +216,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(new Timestamp(1000) == r1.getTimestamp(0)) } - createQueryTest("constant array", - """ - |SELECT sort_array( - | sort_array( - | array("hadoop distributed file system", - | "enterprise databases", "hadoop map-reduce"))) - |FROM src LIMIT 1; - """.stripMargin) - createQueryTest("null case", "SELECT case when(true) then 1 else null end FROM src LIMIT 1") From 379b1272925e534d99ddf4e4add054284900d200 Mon Sep 17 00:00:00 2001 From: Srinath Shankar Date: Fri, 19 Aug 2016 19:54:26 -0700 Subject: [PATCH 1163/1470] [SPARK-17158][SQL] Change error message for out of range numeric literals ## What changes were proposed in this pull request? Modifies error message for numeric literals to Numeric literal does not fit in range [min, max] for type ## How was this patch tested? Fixed up the error messages for literals.sql in SqlQueryTestSuite and re-ran via sbt. Also fixed up error messages in ExpressionParserSuite Author: Srinath Shankar Closes #14721 from srinathshankar/sc4296. (cherry picked from commit ba1737c21aab91ff3f1a1737aa2d6b07575e36a3) Signed-off-by: Reynold Xin --- .../sql/catalyst/parser/AstBuilder.scala | 29 ++++++++++++------- .../parser/ExpressionParserSuite.scala | 9 ++++-- .../sql-tests/results/literals.sql.out | 6 ++-- 3 files changed, 27 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 02302941630c8..aec312622a7ac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1273,10 +1273,17 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { } /** Create a numeric literal expression. */ - private def numericLiteral(ctx: NumberContext)(f: String => Any): Literal = withOrigin(ctx) { - val raw = ctx.getText + private def numericLiteral + (ctx: NumberContext, minValue: BigDecimal, maxValue: BigDecimal, typeName: String) + (converter: String => Any): Literal = withOrigin(ctx) { + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) try { - Literal(f(raw.substring(0, raw.length - 1))) + val rawBigDecimal = BigDecimal(rawStrippedQualifier) + if (rawBigDecimal < minValue || rawBigDecimal > maxValue) { + throw new ParseException(s"Numeric literal ${rawStrippedQualifier} does not " + + s"fit in range [${minValue}, ${maxValue}] for type ${typeName}", ctx) + } + Literal(converter(rawStrippedQualifier)) } catch { case e: NumberFormatException => throw new ParseException(e.getMessage, ctx) @@ -1286,29 +1293,29 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { /** * Create a Byte Literal expression. */ - override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = numericLiteral(ctx) { - _.toByte + override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = { + numericLiteral(ctx, Byte.MinValue, Byte.MaxValue, ByteType.simpleString)(_.toByte) } /** * Create a Short Literal expression. */ - override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = numericLiteral(ctx) { - _.toShort + override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = { + numericLiteral(ctx, Short.MinValue, Short.MaxValue, ShortType.simpleString)(_.toShort) } /** * Create a Long Literal expression. */ - override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = numericLiteral(ctx) { - _.toLong + override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = { + numericLiteral(ctx, Long.MinValue, Long.MaxValue, LongType.simpleString)(_.toLong) } /** * Create a Double Literal expression. */ - override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = numericLiteral(ctx) { - _.toDouble + override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = { + numericLiteral(ctx, Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 849d96212822c..401d9cd9d288c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -375,18 +375,21 @@ class ExpressionParserSuite extends PlanTest { // Tiny Int Literal assertEqual("10Y", Literal(10.toByte)) - intercept("-1000Y") + intercept("-1000Y", s"does not fit in range [${Byte.MinValue}, ${Byte.MaxValue}]") // Small Int Literal assertEqual("10S", Literal(10.toShort)) - intercept("40000S") + intercept("40000S", s"does not fit in range [${Short.MinValue}, ${Short.MaxValue}]") // Long Int Literal assertEqual("10L", Literal(10L)) - intercept("78732472347982492793712334L") + intercept("78732472347982492793712334L", + s"does not fit in range [${Long.MinValue}, ${Long.MaxValue}]") // Double Literal assertEqual("10.0D", Literal(10.0D)) + intercept("-1.8E308D", s"does not fit in range") + intercept("1.8E308D", s"does not fit in range") // TODO we need to figure out if we should throw an exception here! assertEqual("1E309", Literal(Double.PositiveInfinity)) } diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index b964a6fc0921f..67e6d78dfbf24 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -41,7 +41,7 @@ struct<> -- !query 4 output org.apache.spark.sql.catalyst.parser.ParseException -Value out of range. Value:"128" Radix:10(line 1, pos 7) +Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7) == SQL == select 128Y @@ -71,7 +71,7 @@ struct<> -- !query 7 output org.apache.spark.sql.catalyst.parser.ParseException -Value out of range. Value:"32768" Radix:10(line 1, pos 7) +Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7) == SQL == select 32768S @@ -101,7 +101,7 @@ struct<> -- !query 10 output org.apache.spark.sql.catalyst.parser.ParseException -For input string: "9223372036854775808"(line 1, pos 7) +Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7) == SQL == select 9223372036854775808L From f7458c71d3b02864acb33fc48c130a0a734e9723 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Sat, 20 Aug 2016 13:19:38 +0800 Subject: [PATCH 1164/1470] [SPARK-17150][SQL] Support SQL generation for inline tables ## What changes were proposed in this pull request? This patch adds support for SQL generation for inline tables. With this, it would be possible to create a view that depends on inline tables. ## How was this patch tested? Added a test case in LogicalPlanToSQLSuite. Author: petermaxlee Closes #14709 from petermaxlee/SPARK-17150. (cherry picked from commit 45d40d9f66c666eec6df926db23937589d67225d) Signed-off-by: Wenchen Fan --- .../catalyst/plans/logical/LocalRelation.scala | 17 +++++++++++++++-- .../apache/spark/sql/catalyst/SQLBuilder.scala | 3 +++ .../src/test/resources/sqlgen/inline_tables.sql | 4 ++++ .../sql/catalyst/LogicalPlanToSQLSuite.scala | 8 ++++++++ 4 files changed, 30 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/resources/sqlgen/inline_tables.sql 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 9d64f35efcc6a..890865d177845 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,8 +18,9 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.{analysis, CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal} import org.apache.spark.sql.types.{StructField, StructType} object LocalRelation { @@ -75,4 +76,16 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil) override lazy val statistics = Statistics(sizeInBytes = output.map(_.dataType.defaultSize).sum * data.length) + + def toSQL(inlineTableName: String): String = { + require(data.nonEmpty) + val types = output.map(_.dataType) + val rows = data.map { row => + val cells = row.toSeq(types).zip(types).map { case (v, tpe) => Literal(v, tpe).sql } + cells.mkString("(", ", ", ")") + } + "VALUES " + rows.mkString(", ") + + " AS " + inlineTableName + + output.map(_.name).mkString("(", ", ", ")") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 5d93419f357ef..9ac34801bf006 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -205,6 +205,9 @@ class SQLBuilder private ( case p: ScriptTransformation => scriptTransformationToSQL(p) + case p: LocalRelation => + p.toSQL(newSubqueryName()) + case OneRowRelation => "" diff --git a/sql/hive/src/test/resources/sqlgen/inline_tables.sql b/sql/hive/src/test/resources/sqlgen/inline_tables.sql new file mode 100644 index 0000000000000..602551e69da6e --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/inline_tables.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) where b > 1 +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (VALUES ("one", 1), ("two", 2), ("three", CAST(NULL AS INT)) AS gen_subquery_0(gen_attr_0, gen_attr_1)) AS data WHERE (`gen_attr_1` > 1)) AS data diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 4e5a51155defd..742b065891a8e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -1102,4 +1102,12 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSQL("select * from orc_t", "select_orc_table") } } + + test("inline tables") { + checkSQL( + """ + |select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) where b > 1 + """.stripMargin, + "inline_tables") + } } From 4c4c2753b1012e395ae3896396b6509d6082fdf2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Aug 2016 23:29:48 +0800 Subject: [PATCH 1165/1470] [SPARK-17104][SQL] LogicalRelation.newInstance should follow the semantics of MultiInstanceRelation ## What changes were proposed in this pull request? Currently `LogicalRelation.newInstance()` simply creates another `LogicalRelation` object with the same parameters. However, the `newInstance()` method inherited from `MultiInstanceRelation` should return a copy of object with unique expression ids. Current `LogicalRelation.newInstance()` can cause failure when doing self-join. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #14682 from viirya/fix-localrelation. (cherry picked from commit 31a015572024046f4deaa6cec66bb6fab110f31d) Signed-off-by: Wenchen Fan --- .../sql/execution/datasources/LogicalRelation.scala | 11 +++++++++-- .../org/apache/spark/sql/hive/parquetSuites.scala | 7 +++++++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 90711f2b1dde4..2a8e147011f55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -79,11 +79,18 @@ case class LogicalRelation( /** Used to lookup original attribute capitalization */ val attributeMap: AttributeMap[AttributeReference] = AttributeMap(output.map(o => (o, o))) - def newInstance(): this.type = + /** + * Returns a new instance of this LogicalRelation. According to the semantics of + * MultiInstanceRelation, this method returns a copy of this object with + * unique expression ids. We respect the `expectedOutputAttributes` and create + * new instances of attributes in it. + */ + override def newInstance(): this.type = { LogicalRelation( relation, - expectedOutputAttributes, + expectedOutputAttributes.map(_.map(_.newInstance())), metastoreTableIdentifier).asInstanceOf[this.type] + } override def refresh(): Unit = relation match { case fs: HadoopFsRelation => fs.refresh() 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 31b6197d56fc7..e92bbdea75a7b 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 @@ -589,6 +589,13 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { } } } + + test("self-join") { + val table = spark.table("normal_parquet") + val selfJoin = table.as("t1").join(table.as("t2")) + checkAnswer(selfJoin, + sql("SELECT * FROM normal_parquet x JOIN normal_parquet y")) + } } /** From 24dd9a702694db1d2c28ff4c41edac2b3112df60 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Sun, 21 Aug 2016 00:25:55 +0800 Subject: [PATCH 1166/1470] [SPARK-17124][SQL] RelationalGroupedDataset.agg should preserve order and allow multiple aggregates per column ## What changes were proposed in this pull request? This patch fixes a longstanding issue with one of the RelationalGroupedDataset.agg function. Even though the signature accepts vararg of pairs, the underlying implementation turns the seq into a map, and thus not order preserving nor allowing multiple aggregates per column. This change also allows users to use this function to run multiple different aggregations for a single column, e.g. ``` agg("age" -> "max", "age" -> "count") ``` ## How was this patch tested? Added a test case in DataFrameAggregateSuite. Author: petermaxlee Closes #14697 from petermaxlee/SPARK-17124. (cherry picked from commit 9560c8d29542a5dcaaa07b7af9ef5ddcdbb5d14d) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/RelationalGroupedDataset.scala | 6 ++++-- .../org/apache/spark/sql/DataFrameAggregateSuite.scala | 10 ++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 1aa5767038d53..6148ddfe05acb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -128,7 +128,7 @@ class RelationalGroupedDataset protected[sql]( } /** - * (Scala-specific) Compute aggregates by specifying a map from column name to + * (Scala-specific) Compute aggregates by specifying the column names and * aggregate methods. The resulting [[DataFrame]] will also contain the grouping columns. * * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. @@ -143,7 +143,9 @@ class RelationalGroupedDataset protected[sql]( * @since 1.3.0 */ def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { - agg((aggExpr +: aggExprs).toMap) + toDF((aggExpr +: aggExprs).map { case (colName, expr) => + strToExpr(expr)(df(colName).expr) + }) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 92aa7b95434dc..69a3b5f278fd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -87,6 +87,16 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { ) } + test("SPARK-17124 agg should be ordering preserving") { + val df = spark.range(2) + val ret = df.groupBy("id").agg("id" -> "sum", "id" -> "count", "id" -> "min") + assert(ret.schema.map(_.name) == Seq("id", "sum(id)", "count(id)", "min(id)")) + checkAnswer( + ret, + Row(0, 0, 1, 0) :: Row(1, 1, 1, 1) :: Nil + ) + } + test("rollup") { checkAnswer( courseSales.rollup("course", "year").sum("earnings"), From faff9297d154596e35de555c819049ba9a51d57d Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Sat, 20 Aug 2016 13:45:26 -0700 Subject: [PATCH 1167/1470] [SPARK-12666][CORE] SparkSubmit packages fix for when 'default' conf doesn't exist in dependent module ## What changes were proposed in this pull request? Adding a "(runtime)" to the dependency configuration will set a fallback configuration to be used if the requested one is not found. E.g. with the setting "default(runtime)", Ivy will look for the conf "default" in the module ivy file and if not found will look for the conf "runtime". This can help with the case when using "sbt publishLocal" which does not write a "default" conf in the published ivy.xml file. ## How was this patch tested? used spark-submit with --packages option for a package published locally with no default conf, and a package resolved from Maven central. Author: Bryan Cutler Closes #13428 from BryanCutler/fallback-package-conf-SPARK-12666. (cherry picked from commit 9f37d4eac28dd179dd523fa7d645be97bb52af9c) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 9feafc99ac07f..7b6d5a394bc35 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -897,9 +897,12 @@ private[spark] object SparkSubmitUtils { val localIvyRoot = new File(ivySettings.getDefaultIvyUserDir, "local") localIvy.setLocal(true) localIvy.setRepository(new FileRepository(localIvyRoot)) - val ivyPattern = Seq("[organisation]", "[module]", "[revision]", "[type]s", - "[artifact](-[classifier]).[ext]").mkString(File.separator) - localIvy.addIvyPattern(localIvyRoot.getAbsolutePath + File.separator + ivyPattern) + val ivyPattern = Seq(localIvyRoot.getAbsolutePath, "[organisation]", "[module]", "[revision]", + "ivys", "ivy.xml").mkString(File.separator) + localIvy.addIvyPattern(ivyPattern) + val artifactPattern = Seq(localIvyRoot.getAbsolutePath, "[organisation]", "[module]", + "[revision]", "[type]s", "[artifact](-[classifier]).[ext]").mkString(File.separator) + localIvy.addArtifactPattern(artifactPattern) localIvy.setName("local-ivy-cache") cr.add(localIvy) @@ -944,7 +947,7 @@ private[spark] object SparkSubmitUtils { artifacts.foreach { mvn => val ri = ModuleRevisionId.newInstance(mvn.groupId, mvn.artifactId, mvn.version) val dd = new DefaultDependencyDescriptor(ri, false, false) - dd.addDependencyConfiguration(ivyConfName, ivyConfName) + dd.addDependencyConfiguration(ivyConfName, ivyConfName + "(runtime)") // scalastyle:off println printStream.println(s"${dd.getDependencyId} added as a dependency") // scalastyle:on println From 26d5a8b0dab10310ec76b91465b3b4ff465e9746 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 21 Aug 2016 10:31:25 -0700 Subject: [PATCH 1168/1470] [MINOR][R] add SparkR.Rcheck/ and SparkR_*.tar.gz to R/.gitignore ## What changes were proposed in this pull request? Ignore temp files generated by `check-cran.sh`. Author: Xiangrui Meng Closes #14740 from mengxr/R-gitignore. (cherry picked from commit ab7143463daf2056736c85e3a943c826b5992623) Signed-off-by: Xiangrui Meng --- R/.gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/R/.gitignore b/R/.gitignore index 9a5889ba28b2a..c98504ab07781 100644 --- a/R/.gitignore +++ b/R/.gitignore @@ -4,3 +4,5 @@ lib pkg/man pkg/html +SparkR.Rcheck/ +SparkR_*.tar.gz From 0297896119e11f23da4b14f62f50ec72b5fac57f Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Sat, 20 Aug 2016 06:59:23 -0700 Subject: [PATCH 1169/1470] [SPARK-16508][SPARKR] Fix CRAN undocumented/duplicated arguments warnings. This PR tries to fix all the remaining "undocumented/duplicated arguments" warnings given by CRAN-check. One left is doc for R `stats::glm` exported in SparkR. To mute that warning, we have to also provide document for all arguments of that non-SparkR function. Some previous conversation is in #14558. R unit test and `check-cran.sh` script (with no-test). Author: Junyang Qian Closes #14705 from junyangq/SPARK-16508-master. (cherry picked from commit 01401e965b58f7e8ab615764a452d7d18f1d4bf0) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 219 +++++++++++++++++++++++++------------------ R/pkg/R/SQLContext.R | 30 +++--- R/pkg/R/WindowSpec.R | 11 ++- R/pkg/R/column.R | 18 +++- R/pkg/R/functions.R | 173 ++++++++++++++++++++++------------ R/pkg/R/generics.R | 61 +++++++++--- R/pkg/R/group.R | 7 +- R/pkg/R/mllib.R | 108 +++++++++++---------- R/pkg/R/schema.R | 5 +- R/pkg/R/sparkR.R | 21 +++-- R/pkg/R/stats.R | 25 +++-- 11 files changed, 415 insertions(+), 263 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 92e60e7385fca..0266939de37b1 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -120,8 +120,9 @@ setMethod("schema", #' #' Print the logical and physical Catalyst plans to the console for debugging. #' -#' @param x A SparkDataFrame +#' @param x a SparkDataFrame. #' @param extended Logical. If extended is FALSE, explain() only prints the physical plan. +#' @param ... further arguments to be passed to or from other methods. #' @family SparkDataFrame functions #' @aliases explain,SparkDataFrame-method #' @rdname explain @@ -177,11 +178,11 @@ setMethod("isLocal", #' #' Print the first numRows rows of a SparkDataFrame #' -#' @param x A SparkDataFrame -#' @param numRows The number of rows to print. Defaults to 20. -#' @param truncate Whether truncate long strings. If true, strings more than 20 characters will be -#' truncated and all cells will be aligned right -#' +#' @param x a SparkDataFrame. +#' @param numRows the number of rows to print. Defaults to 20. +#' @param truncate whether truncate long strings. If \code{TRUE}, strings more than +#' 20 characters will be truncated and all cells will be aligned right. +#' @param ... further arguments to be passed to or from other methods. #' @family SparkDataFrame functions #' @aliases showDF,SparkDataFrame-method #' @rdname showDF @@ -206,7 +207,7 @@ setMethod("showDF", #' #' Print the SparkDataFrame column names and types #' -#' @param x A SparkDataFrame +#' @param object a SparkDataFrame. #' #' @family SparkDataFrame functions #' @rdname show @@ -257,11 +258,11 @@ setMethod("dtypes", }) }) -#' Column names +#' Column Names of SparkDataFrame #' -#' Return all column names as a list +#' Return all column names as a list. #' -#' @param x A SparkDataFrame +#' @param x a SparkDataFrame. #' #' @family SparkDataFrame functions #' @rdname columns @@ -318,6 +319,8 @@ setMethod("colnames", columns(x) }) +#' @param value a character vector. Must have the same length as the number +#' of columns in the SparkDataFrame. #' @rdname columns #' @aliases colnames<-,SparkDataFrame-method #' @name colnames<- @@ -509,9 +512,10 @@ setMethod("registerTempTable", #' #' Insert the contents of a SparkDataFrame into a table registered in the current SparkSession. #' -#' @param x A SparkDataFrame -#' @param tableName A character vector containing the name of the table -#' @param overwrite A logical argument indicating whether or not to overwrite +#' @param x a SparkDataFrame. +#' @param tableName a character vector containing the name of the table. +#' @param overwrite a logical argument indicating whether or not to overwrite. +#' @param ... further arguments to be passed to or from other methods. #' the existing rows in the table. #' #' @family SparkDataFrame functions @@ -570,7 +574,9 @@ setMethod("cache", #' supported storage levels, refer to #' \url{http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence}. #' -#' @param x The SparkDataFrame to persist +#' @param x the SparkDataFrame to persist. +#' @param newLevel storage level chosen for the persistance. See available options in +#' the description. #' #' @family SparkDataFrame functions #' @rdname persist @@ -598,8 +604,9 @@ setMethod("persist", #' Mark this SparkDataFrame as non-persistent, and remove all blocks for it from memory and #' disk. #' -#' @param x The SparkDataFrame to unpersist -#' @param blocking Whether to block until all blocks are deleted +#' @param x the SparkDataFrame to unpersist. +#' @param blocking whether to block until all blocks are deleted. +#' @param ... further arguments to be passed to or from other methods. #' #' @family SparkDataFrame functions #' @rdname unpersist-methods @@ -633,9 +640,10 @@ setMethod("unpersist", #' \item{3.} {Return a new SparkDataFrame partitioned by the given column(s), #' using `spark.sql.shuffle.partitions` as number of partitions.} #'} -#' @param x A SparkDataFrame -#' @param numPartitions The number of partitions to use. -#' @param col The column by which the partitioning will be performed. +#' @param x a SparkDataFrame. +#' @param numPartitions the number of partitions to use. +#' @param col the column by which the partitioning will be performed. +#' @param ... additional column(s) to be used in the partitioning. #' #' @family SparkDataFrame functions #' @rdname repartition @@ -914,11 +922,10 @@ setMethod("sample_frac", #' Returns the number of rows in a SparkDataFrame #' -#' @param x A SparkDataFrame -#' +#' @param x a SparkDataFrame. #' @family SparkDataFrame functions #' @rdname nrow -#' @name count +#' @name nrow #' @aliases count,SparkDataFrame-method #' @export #' @examples @@ -994,9 +1001,10 @@ setMethod("dim", #' Collects all the elements of a SparkDataFrame and coerces them into an R data.frame. #' -#' @param x A SparkDataFrame -#' @param stringsAsFactors (Optional) A logical indicating whether or not string columns +#' @param x a SparkDataFrame. +#' @param stringsAsFactors (Optional) a logical indicating whether or not string columns #' should be converted to factors. FALSE by default. +#' @param ... further arguments to be passed to or from other methods. #' #' @family SparkDataFrame functions #' @rdname collect @@ -1091,8 +1099,10 @@ setMethod("limit", dataFrame(res) }) -#' Take the first NUM rows of a SparkDataFrame and return a the results as a R data.frame +#' Take the first NUM rows of a SparkDataFrame and return the results as a R data.frame #' +#' @param x a SparkDataFrame. +#' @param num number of rows to take. #' @family SparkDataFrame functions #' @rdname take #' @name take @@ -1119,9 +1129,9 @@ setMethod("take", #' then head() returns the first 6 rows in keeping with the current data.frame #' convention in R. #' -#' @param x A SparkDataFrame -#' @param num The number of rows to return. Default is 6. -#' @return A data.frame +#' @param x a SparkDataFrame. +#' @param num the number of rows to return. Default is 6. +#' @return A data.frame. #' #' @family SparkDataFrame functions #' @aliases head,SparkDataFrame-method @@ -1145,7 +1155,8 @@ setMethod("head", #' Return the first row of a SparkDataFrame #' -#' @param x A SparkDataFrame +#' @param x a SparkDataFrame or a column used in aggregation function. +#' @param ... further arguments to be passed to or from other methods. #' #' @family SparkDataFrame functions #' @aliases first,SparkDataFrame-method @@ -1196,8 +1207,9 @@ setMethod("toRDD", #' #' Groups the SparkDataFrame using the specified columns, so we can run aggregation on them. #' -#' @param x a SparkDataFrame -#' @return a GroupedData +#' @param x a SparkDataFrame. +#' @param ... variable(s) (character names(s) or Column(s)) to group on. +#' @return A GroupedData. #' @family SparkDataFrame functions #' @aliases groupBy,SparkDataFrame-method #' @rdname groupBy @@ -1239,7 +1251,6 @@ setMethod("group_by", #' #' Compute aggregates by specifying a list of columns #' -#' @param x a SparkDataFrame #' @family SparkDataFrame functions #' @aliases agg,SparkDataFrame-method #' @rdname summarize @@ -1386,16 +1397,15 @@ setMethod("dapplyCollect", #' Groups the SparkDataFrame using the specified columns and applies the R function to each #' group. #' -#' @param x A SparkDataFrame -#' @param cols Grouping columns -#' @param func A function to be applied to each group partition specified by grouping +#' @param cols grouping columns. +#' @param func a function to be applied to each group partition specified by grouping #' column of the SparkDataFrame. The function `func` takes as argument #' a key - grouping columns and a data frame - a local R data.frame. #' The output of `func` is a local R data.frame. -#' @param schema The schema of the resulting SparkDataFrame after the function is applied. +#' @param schema the schema of the resulting SparkDataFrame after the function is applied. #' The schema must match to output of `func`. It has to be defined for each #' output column with preferred output column name and corresponding data type. -#' @return a SparkDataFrame +#' @return A SparkDataFrame. #' @family SparkDataFrame functions #' @aliases gapply,SparkDataFrame-method #' @rdname gapply @@ -1478,13 +1488,12 @@ setMethod("gapply", #' Groups the SparkDataFrame using the specified columns, applies the R function to each #' group and collects the result back to R as data.frame. #' -#' @param x A SparkDataFrame -#' @param cols Grouping columns -#' @param func A function to be applied to each group partition specified by grouping +#' @param cols grouping columns. +#' @param func a function to be applied to each group partition specified by grouping #' column of the SparkDataFrame. The function `func` takes as argument #' a key - grouping columns and a data frame - a local R data.frame. #' The output of `func` is a local R data.frame. -#' @return a data.frame +#' @return A data.frame. #' @family SparkDataFrame functions #' @aliases gapplyCollect,SparkDataFrame-method #' @rdname gapplyCollect @@ -1631,6 +1640,7 @@ getColumn <- function(x, c) { column(callJMethod(x@sdf, "col", c)) } +#' @param name name of a Column (without being wrapped by \code{""}). #' @rdname select #' @name $ #' @aliases $,SparkDataFrame-method @@ -1640,6 +1650,7 @@ setMethod("$", signature(x = "SparkDataFrame"), getColumn(x, name) }) +#' @param value a Column or NULL. If NULL, the specified Column is dropped. #' @rdname select #' @name $<- #' @aliases $<-,SparkDataFrame-method @@ -1714,12 +1725,13 @@ setMethod("[", signature(x = "SparkDataFrame"), #' Subset #' #' Return subsets of SparkDataFrame according to given conditions -#' @param x A SparkDataFrame -#' @param subset (Optional) A logical expression to filter on rows -#' @param select expression for the single Column or a list of columns to select from the SparkDataFrame +#' @param x a SparkDataFrame. +#' @param i,subset (Optional) a logical expression to filter on rows. +#' @param j,select expression for the single Column or a list of columns to select from the SparkDataFrame. #' @param drop if TRUE, a Column will be returned if the resulting dataset has only one column. -#' Otherwise, a SparkDataFrame will always be returned. -#' @return A new SparkDataFrame containing only the rows that meet the condition with selected columns +#' Otherwise, a SparkDataFrame will always be returned. +#' @param ... currently not used. +#' @return A new SparkDataFrame containing only the rows that meet the condition with selected columns. #' @export #' @family SparkDataFrame functions #' @aliases subset,SparkDataFrame-method @@ -1754,9 +1766,12 @@ setMethod("subset", signature(x = "SparkDataFrame"), #' Select #' #' Selects a set of columns with names or Column expressions. -#' @param x A SparkDataFrame -#' @param col A list of columns or single Column or name -#' @return A new SparkDataFrame with selected columns +#' @param x a SparkDataFrame. +#' @param col a list of columns or single Column or name. +#' @param ... additional column(s) if only one column is specified in \code{col}. +#' If more than one column is assigned in \code{col}, \code{...} +#' should be left empty. +#' @return A new SparkDataFrame with selected columns. #' @export #' @family SparkDataFrame functions #' @rdname select @@ -1853,9 +1868,9 @@ setMethod("selectExpr", #' Return a new SparkDataFrame by adding a column or replacing the existing column #' that has the same name. #' -#' @param x A SparkDataFrame -#' @param colName A column name. -#' @param col A Column expression. +#' @param x a SparkDataFrame. +#' @param colName a column name. +#' @param col a Column expression. #' @return A SparkDataFrame with the new column added or the existing column replaced. #' @family SparkDataFrame functions #' @aliases withColumn,SparkDataFrame,character,Column-method @@ -1884,8 +1899,8 @@ setMethod("withColumn", #' #' Return a new SparkDataFrame with the specified columns added or replaced. #' -#' @param .data A SparkDataFrame -#' @param col a named argument of the form name = col +#' @param .data a SparkDataFrame. +#' @param ... additional column argument(s) each in the form name = col. #' @return A new SparkDataFrame with the new columns added or replaced. #' @family SparkDataFrame functions #' @aliases mutate,SparkDataFrame-method @@ -1962,6 +1977,7 @@ setMethod("mutate", do.call(select, c(x, colList, deDupCols)) }) +#' @param _data a SparkDataFrame. #' @export #' @rdname mutate #' @aliases transform,SparkDataFrame-method @@ -2273,11 +2289,18 @@ setMethod("join", #' specified, the common column names in \code{x} and \code{y} will be used. #' @param by.x a character vector specifying the joining columns for x. #' @param by.y a character vector specifying the joining columns for y. +#' @param all a boolean value setting \code{all.x} and \code{all.y} +#' if any of them are unset. #' @param all.x a boolean value indicating whether all the rows in x should #' be including in the join #' @param all.y a boolean value indicating whether all the rows in y should #' be including in the join #' @param sort a logical argument indicating whether the resulting columns should be sorted +#' @param suffixes a string vector of length 2 used to make colnames of +#' \code{x} and \code{y} unique. +#' The first element is appended to each colname of \code{x}. +#' The second element is appended to each colname of \code{y}. +#' @param ... additional argument(s) passed to the method. #' @details If all.x and all.y are set to FALSE, a natural join will be returned. If #' all.x is set to TRUE and all.y is set to FALSE, a left outer join will #' be returned. If all.x is set to FALSE and all.y is set to TRUE, a right @@ -2306,7 +2329,7 @@ setMethod("merge", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y, by = intersect(names(x), names(y)), by.x = by, by.y = by, all = FALSE, all.x = all, all.y = all, - sort = TRUE, suffixes = c("_x", "_y"), ... ) { + sort = TRUE, suffixes = c("_x", "_y"), ...) { if (length(suffixes) != 2) { stop("suffixes must have length 2") @@ -2459,8 +2482,10 @@ setMethod("unionAll", #' Union two or more SparkDataFrames. This is equivalent to `UNION ALL` in SQL. #' Note that this does not remove duplicate rows across the two SparkDataFrames. #' -#' @param x A SparkDataFrame -#' @param ... Additional SparkDataFrame +#' @param x a SparkDataFrame. +#' @param ... additional SparkDataFrame(s). +#' @param deparse.level currently not used (put here to match the signature of +#' the base implementation). #' @return A SparkDataFrame containing the result of the union. #' @family SparkDataFrame functions #' @aliases rbind,SparkDataFrame-method @@ -2517,8 +2542,8 @@ setMethod("intersect", #' Return a new SparkDataFrame containing rows in this SparkDataFrame #' but not in another SparkDataFrame. This is equivalent to `EXCEPT` in SQL. #' -#' @param x A SparkDataFrame -#' @param y A SparkDataFrame +#' @param x a SparkDataFrame. +#' @param y a SparkDataFrame. #' @return A SparkDataFrame containing the result of the except operation. #' @family SparkDataFrame functions #' @aliases except,SparkDataFrame,SparkDataFrame-method @@ -2559,10 +2584,11 @@ setMethod("except", #' and to not change the existing data. #' } #' -#' @param df A SparkDataFrame -#' @param path A name for the table -#' @param source A name for external data source -#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param df a SparkDataFrame. +#' @param path a name for the table. +#' @param source a name for external data source. +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. #' #' @family SparkDataFrame functions #' @aliases write.df,SparkDataFrame,character-method @@ -2621,10 +2647,11 @@ setMethod("saveDF", #' ignore: The save operation is expected to not save the contents of the SparkDataFrame #' and to not change the existing data. \cr #' -#' @param df A SparkDataFrame -#' @param tableName A name for the table -#' @param source A name for external data source -#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param df a SparkDataFrame. +#' @param tableName a name for the table. +#' @param source a name for external data source. +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default). +#' @param ... additional option(s) passed to the method. #' #' @family SparkDataFrame functions #' @aliases saveAsTable,SparkDataFrame,character-method @@ -2660,10 +2687,10 @@ setMethod("saveAsTable", #' Computes statistics for numeric columns. #' If no columns are given, this function computes statistics for all numerical columns. #' -#' @param x A SparkDataFrame to be computed. -#' @param col A string of name -#' @param ... Additional expressions -#' @return A SparkDataFrame +#' @param x a SparkDataFrame to be computed. +#' @param col a string of name. +#' @param ... additional expressions. +#' @return A SparkDataFrame. #' @family SparkDataFrame functions #' @aliases describe,SparkDataFrame,character-method describe,SparkDataFrame,ANY-method #' @rdname summary @@ -2698,6 +2725,7 @@ setMethod("describe", dataFrame(sdf) }) +#' @param object a SparkDataFrame to be summarized. #' @rdname summary #' @name summary #' @aliases summary,SparkDataFrame-method @@ -2713,16 +2741,20 @@ setMethod("summary", #' #' dropna, na.omit - Returns a new SparkDataFrame omitting rows with null values. #' -#' @param x A SparkDataFrame. +#' @param x a SparkDataFrame. #' @param how "any" or "all". #' if "any", drop a row if it contains any nulls. #' if "all", drop a row only if all its values are null. #' if minNonNulls is specified, how is ignored. -#' @param minNonNulls If specified, drop rows that have less than +#' @param minNonNulls if specified, drop rows that have less than #' minNonNulls non-null values. #' This overwrites the how parameter. -#' @param cols Optional list of column names to consider. -#' @return A SparkDataFrame +#' @param cols optional list of column names to consider. In `fillna`, +#' columns specified in cols that do not have matching data +#' type are ignored. For example, if value is a character, and +#' subset contains a non-character column, then the non-character +#' column is simply ignored. +#' @return A SparkDataFrame. #' #' @family SparkDataFrame functions #' @rdname nafunctions @@ -2754,6 +2786,8 @@ setMethod("dropna", dataFrame(sdf) }) +#' @param object a SparkDataFrame. +#' @param ... further arguments to be passed to or from other methods. #' @rdname nafunctions #' @name na.omit #' @aliases na.omit,SparkDataFrame-method @@ -2767,18 +2801,12 @@ setMethod("na.omit", #' fillna - Replace null values. #' -#' @param x A SparkDataFrame. -#' @param value Value to replace null values with. +#' @param value value to replace null values with. #' Should be an integer, numeric, character or named list. #' If the value is a named list, then cols is ignored and #' value must be a mapping from column name (character) to #' replacement value. The replacement value must be an #' integer, numeric or character. -#' @param cols optional list of column names to consider. -#' Columns specified in cols that do not have matching data -#' type are ignored. For example, if value is a character, and -#' subset contains a non-character column, then the non-character -#' column is simply ignored. #' #' @rdname nafunctions #' @name fillna @@ -2843,8 +2871,11 @@ setMethod("fillna", #' Since data.frames are held in memory, ensure that you have enough memory #' in your system to accommodate the contents. #' -#' @param x a SparkDataFrame -#' @return a data.frame +#' @param x a SparkDataFrame. +#' @param row.names NULL or a character vector giving the row names for the data frame. +#' @param optional If `TRUE`, converting column names is optional. +#' @param ... additional arguments to pass to base::as.data.frame. +#' @return A data.frame. #' @family SparkDataFrame functions #' @aliases as.data.frame,SparkDataFrame-method #' @rdname as.data.frame @@ -2998,9 +3029,10 @@ setMethod("str", #' Returns a new SparkDataFrame with columns dropped. #' This is a no-op if schema doesn't contain column name(s). #' -#' @param x A SparkDataFrame. -#' @param cols A character vector of column names or a Column. -#' @return A SparkDataFrame +#' @param x a SparkDataFrame. +#' @param col a character vector of column names or a Column. +#' @param ... further arguments to be passed to or from other methods. +#' @return A SparkDataFrame. #' #' @family SparkDataFrame functions #' @rdname drop @@ -3019,7 +3051,7 @@ setMethod("str", #' @note drop since 2.0.0 setMethod("drop", signature(x = "SparkDataFrame"), - function(x, col) { + function(x, col, ...) { stopifnot(class(col) == "character" || class(col) == "Column") if (class(col) == "Column") { @@ -3047,8 +3079,8 @@ setMethod("drop", #' #' @name histogram #' @param nbins the number of bins (optional). Default value is 10. +#' @param col the column as Character string or a Column to build the histogram from. #' @param df the SparkDataFrame containing the Column to build the histogram from. -#' @param colname the name of the column to build the histogram from. #' @return a data.frame with the histogram statistics, i.e., counts and centroids. #' @rdname histogram #' @aliases histogram,SparkDataFrame,characterOrColumn-method @@ -3179,10 +3211,11 @@ setMethod("histogram", #' and to not change the existing data. #' } #' -#' @param x A SparkDataFrame -#' @param url JDBC database url of the form `jdbc:subprotocol:subname` -#' @param tableName The name of the table in the external database -#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param x s SparkDataFrame. +#' @param url JDBC database url of the form `jdbc:subprotocol:subname`. +#' @param tableName yhe name of the table in the external database. +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default). +#' @param ... additional JDBC database connection properties. #' @family SparkDataFrame functions #' @rdname write.jdbc #' @name write.jdbc diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 0c06bba639d9b..a9cd2d85f898c 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -165,9 +165,9 @@ getDefaultSqlSource <- function() { #' #' Converts R data.frame or list into SparkDataFrame. #' -#' @param data An RDD or list or data.frame -#' @param schema a list of column names or named list (StructType), optional -#' @return a SparkDataFrame +#' @param data an RDD or list or data.frame. +#' @param schema a list of column names or named list (StructType), optional. +#' @return A SparkDataFrame. #' @rdname createDataFrame #' @export #' @examples @@ -257,23 +257,25 @@ createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { } createDataFrame <- function(x, ...) { - dispatchFunc("createDataFrame(data, schema = NULL, samplingRatio = 1.0)", x, ...) + dispatchFunc("createDataFrame(data, schema = NULL)", x, ...) } +#' @param samplingRatio Currently not used. #' @rdname createDataFrame #' @aliases createDataFrame #' @export #' @method as.DataFrame default #' @note as.DataFrame since 1.6.0 as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { - createDataFrame(data, schema, samplingRatio) + createDataFrame(data, schema) } +#' @param ... additional argument(s). #' @rdname createDataFrame #' @aliases as.DataFrame #' @export -as.DataFrame <- function(x, ...) { - dispatchFunc("as.DataFrame(data, schema = NULL, samplingRatio = 1.0)", x, ...) +as.DataFrame <- function(data, ...) { + dispatchFunc("as.DataFrame(data, schema = NULL)", data, ...) } #' toDF @@ -398,7 +400,7 @@ read.orc <- function(path) { #' #' Loads a Parquet file, returning the result as a SparkDataFrame. #' -#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param path path of file to read. A vector of multiple paths is allowed. #' @return SparkDataFrame #' @rdname read.parquet #' @export @@ -418,6 +420,7 @@ read.parquet <- function(x, ...) { dispatchFunc("read.parquet(...)", x, ...) } +#' @param ... argument(s) passed to the method. #' @rdname read.parquet #' @name parquetFile #' @export @@ -727,6 +730,7 @@ dropTempView <- function(viewName) { #' @param source The name of external data source #' @param schema The data schema defined in structType #' @param na.strings Default string value for NA when source is "csv" +#' @param ... additional external data source specific named properties. #' @return SparkDataFrame #' @rdname read.df #' @name read.df @@ -791,10 +795,11 @@ loadDF <- function(x, ...) { #' If `source` is not specified, the default data source configured by #' "spark.sql.sources.default" will be used. #' -#' @param tableName A name of the table -#' @param path The path of files to load -#' @param source the name of external data source -#' @return SparkDataFrame +#' @param tableName a name of the table. +#' @param path the path of files to load. +#' @param source the name of external data source. +#' @param ... additional argument(s) passed to the method. +#' @return A SparkDataFrame. #' @rdname createExternalTable #' @export #' @examples @@ -840,6 +845,7 @@ createExternalTable <- function(x, ...) { #' clause expressions used to split the column `partitionColumn` evenly. #' This defaults to SparkContext.defaultParallelism when unset. #' @param predicates a list of conditions in the where clause; each one defines one partition +#' @param ... additional JDBC database connection named propertie(s). #' @return SparkDataFrame #' @rdname read.jdbc #' @name read.jdbc diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index 751ba3fde954d..b55356b07d5e3 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -54,8 +54,10 @@ setMethod("show", "WindowSpec", #' #' Defines the partitioning columns in a WindowSpec. #' -#' @param x a WindowSpec -#' @return a WindowSpec +#' @param x a WindowSpec. +#' @param col a column to partition on (desribed by the name or Column). +#' @param ... additional column(s) to partition on. +#' @return A WindowSpec. #' @rdname partitionBy #' @name partitionBy #' @aliases partitionBy,WindowSpec-method @@ -86,7 +88,7 @@ setMethod("partitionBy", #' #' Defines the ordering columns in a WindowSpec. #' @param x a WindowSpec -#' @param col a character or Column object indicating an ordering column +#' @param col a character or Column indicating an ordering column #' @param ... additional sorting fields #' @return A WindowSpec. #' @name orderBy @@ -192,6 +194,9 @@ setMethod("rangeBetween", #' #' Define a windowing column. #' +#' @param x a Column, usually one returned by window function(s). +#' @param window a WindowSpec object. Can be created by `windowPartitionBy` or +#' `windowOrderBy` and configured by other WindowSpec methods. #' @rdname over #' @name over #' @aliases over,Column,WindowSpec-method diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 0edb9d2ae5c45..af486e1ce212d 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -163,8 +163,9 @@ setMethod("alias", #' @family colum_func #' @aliases substr,Column-method #' -#' @param start starting position -#' @param stop ending position +#' @param x a Column. +#' @param start starting position. +#' @param stop ending position. #' @note substr since 1.4.0 setMethod("substr", signature(x = "Column"), function(x, start, stop) { @@ -219,6 +220,7 @@ setMethod("endsWith", signature(x = "Column"), #' @family colum_func #' @aliases between,Column-method #' +#' @param x a Column #' @param bounds lower and upper bounds #' @note between since 1.5.0 setMethod("between", signature(x = "Column"), @@ -233,6 +235,11 @@ setMethod("between", signature(x = "Column"), #' Casts the column to a different data type. #' +#' @param x a Column. +#' @param dataType a character object describing the target data type. +#' See +#' \href{https://spark.apache.org/docs/latest/sparkr.html#data-type-mapping-between-r-and-spark}{ +#' Spark Data Types} for available data types. #' @rdname cast #' @name cast #' @family colum_func @@ -254,10 +261,12 @@ setMethod("cast", #' Match a column with given values. #' +#' @param x a Column. +#' @param table a collection of values (coercible to list) to compare with. #' @rdname match #' @name %in% #' @aliases %in%,Column-method -#' @return a matched values as a result of comparing with given values. +#' @return A matched values as a result of comparing with given values. #' @export #' @examples #' \dontrun{ @@ -277,6 +286,9 @@ setMethod("%in%", #' If values in the specified column are null, returns the value. #' Can be used in conjunction with `when` to specify a default value for expressions. #' +#' @param x a Column. +#' @param value value to replace when the corresponding entry in \code{x} is NA. +#' Can be a single value or a Column. #' @rdname otherwise #' @name otherwise #' @family colum_func diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 573c915a5c67a..b3c10de71f3fe 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -23,6 +23,7 @@ NULL #' A new \linkS4class{Column} is created to represent the literal value. #' If the parameter is a \linkS4class{Column}, it is returned unchanged. #' +#' @param x a literal value or a Column. #' @family normal_funcs #' @rdname lit #' @name lit @@ -89,8 +90,6 @@ setMethod("acos", #' Returns the approximate number of distinct items in a group. This is a column #' aggregate function. #' -#' @param x Column to compute on. -#' #' @rdname approxCountDistinct #' @name approxCountDistinct #' @return the approximate number of distinct items in a group. @@ -171,8 +170,6 @@ setMethod("atan", #' #' Aggregate function: returns the average of the values in a group. #' -#' @param x Column to compute on. -#' #' @rdname avg #' @name avg #' @family agg_funcs @@ -319,7 +316,7 @@ setMethod("column", #' #' Computes the Pearson Correlation Coefficient for two Columns. #' -#' @param x Column to compute on. +#' @param col2 a (second) Column. #' #' @rdname corr #' @name corr @@ -339,8 +336,6 @@ setMethod("corr", signature(x = "Column"), #' #' Compute the sample covariance between two expressions. #' -#' @param x Column to compute on. -#' #' @rdname cov #' @name cov #' @family math_funcs @@ -362,8 +357,8 @@ setMethod("cov", signature(x = "characterOrColumn"), #' @rdname cov #' -#' @param col1 First column to compute cov_samp. -#' @param col2 Second column to compute cov_samp. +#' @param col1 the first Column. +#' @param col2 the second Column. #' @name covar_samp #' @aliases covar_samp,characterOrColumn,characterOrColumn-method #' @note covar_samp since 2.0.0 @@ -451,9 +446,7 @@ setMethod("cosh", #' #' Returns the number of items in a group. This is a column aggregate function. #' -#' @param x Column to compute on. -#' -#' @rdname nrow +#' @rdname count #' @name count #' @family agg_funcs #' @aliases count,Column-method @@ -493,6 +486,7 @@ setMethod("crc32", #' Calculates the hash code of given columns, and returns the result as a int column. #' #' @param x Column to compute on. +#' @param ... additional Column(s) to be included. #' #' @rdname hash #' @name hash @@ -663,7 +657,8 @@ setMethod("factorial", #' The function by default returns the first values it sees. It will return the first non-missing #' value it sees when na.rm is set to true. If all values are missing, then NA is returned. #' -#' @param x Column to compute on. +#' @param na.rm a logical value indicating whether NA values should be stripped +#' before the computation proceeds. #' #' @rdname first #' @name first @@ -832,7 +827,10 @@ setMethod("kurtosis", #' The function by default returns the last values it sees. It will return the last non-missing #' value it sees when na.rm is set to true. If all values are missing, then NA is returned. #' -#' @param x Column to compute on. +#' @param x column to compute on. +#' @param na.rm a logical value indicating whether NA values should be stripped +#' before the computation proceeds. +#' @param ... further arguments to be passed to or from other methods. #' #' @rdname last #' @name last @@ -1143,7 +1141,7 @@ setMethod("minute", #' @export #' @examples \dontrun{select(df, monotonically_increasing_id())} setMethod("monotonically_increasing_id", - signature(x = "missing"), + signature("missing"), function() { jc <- callJStatic("org.apache.spark.sql.functions", "monotonically_increasing_id") column(jc) @@ -1272,13 +1270,16 @@ setMethod("round", #' bround #' -#' Returns the value of the column `e` rounded to `scale` decimal places using HALF_EVEN rounding -#' mode if `scale` >= 0 or at integral part when `scale` < 0. +#' Returns the value of the column \code{e} rounded to \code{scale} decimal places using HALF_EVEN rounding +#' mode if \code{scale} >= 0 or at integer part when \code{scale} < 0. #' Also known as Gaussian rounding or bankers' rounding that rounds to the nearest even number. #' bround(2.5, 0) = 2, bround(3.5, 0) = 4. #' #' @param x Column to compute on. -#' +#' @param scale round to \code{scale} digits to the right of the decimal point when \code{scale} > 0, +#' the nearest even number when \code{scale} = 0, and \code{scale} digits to the left +#' of the decimal point when \code{scale} < 0. +#' @param ... further arguments to be passed to or from other methods. #' @rdname bround #' @name bround #' @family math_funcs @@ -1319,7 +1320,7 @@ setMethod("rtrim", #' Aggregate function: alias for \link{stddev_samp} #' #' @param x Column to compute on. -#' +#' @param na.rm currently not used. #' @rdname sd #' @name sd #' @family agg_funcs @@ -1497,7 +1498,7 @@ setMethod("soundex", #' \dontrun{select(df, spark_partition_id())} #' @note spark_partition_id since 2.0.0 setMethod("spark_partition_id", - signature(x = "missing"), + signature("missing"), function() { jc <- callJStatic("org.apache.spark.sql.functions", "spark_partition_id") column(jc) @@ -1560,7 +1561,8 @@ setMethod("stddev_samp", #' #' Creates a new struct column that composes multiple input columns. #' -#' @param x Column to compute on. +#' @param x a column to compute on. +#' @param ... optional column(s) to be included. #' #' @rdname struct #' @name struct @@ -1831,8 +1833,8 @@ setMethod("upper", #' #' Aggregate function: alias for \link{var_samp}. #' -#' @param x Column to compute on. -#' +#' @param x a Column to compute on. +#' @param y,na.rm,use currently not used. #' @rdname var #' @name var #' @family agg_funcs @@ -2114,7 +2116,9 @@ setMethod("pmod", signature(y = "Column"), #' @rdname approxCountDistinct #' @name approxCountDistinct #' +#' @param x Column to compute on. #' @param rsd maximum estimation error allowed (default = 0.05) +#' @param ... further arguments to be passed to or from other methods. #' #' @aliases approxCountDistinct,Column-method #' @export @@ -2127,7 +2131,7 @@ setMethod("approxCountDistinct", column(jc) }) -#' Count Distinct +#' Count Distinct Values #' #' @param x Column to compute on #' @param ... other columns @@ -2156,7 +2160,7 @@ setMethod("countDistinct", #' concat #' #' Concatenates multiple input string columns together into a single string column. -#' +#' #' @param x Column to compute on #' @param ... other columns #' @@ -2246,7 +2250,6 @@ setMethod("ceiling", }) #' @rdname sign -#' @param x Column to compute on #' #' @name sign #' @aliases sign,Column-method @@ -2262,9 +2265,6 @@ setMethod("sign", signature(x = "Column"), #' #' Aggregate function: returns the number of distinct items in a group. #' -#' @param x Column to compute on -#' @param ... other columns -#' #' @rdname countDistinct #' @name n_distinct #' @aliases n_distinct,Column-method @@ -2276,9 +2276,7 @@ setMethod("n_distinct", signature(x = "Column"), countDistinct(x, ...) }) -#' @rdname nrow -#' @param x Column to compute on -#' +#' @rdname count #' @name n #' @aliases n,Column-method #' @export @@ -2300,8 +2298,8 @@ setMethod("n", signature(x = "Column"), #' NOTE: Use when ever possible specialized functions like \code{year}. These benefit from a #' specialized implementation. #' -#' @param y Column to compute on -#' @param x date format specification +#' @param y Column to compute on. +#' @param x date format specification. #' #' @family datetime_funcs #' @rdname date_format @@ -2320,8 +2318,8 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' #' Assumes given timestamp is UTC and converts to given timezone. #' -#' @param y Column to compute on -#' @param x time zone to use +#' @param y Column to compute on. +#' @param x time zone to use. #' #' @family datetime_funcs #' @rdname from_utc_timestamp @@ -2370,8 +2368,8 @@ setMethod("instr", signature(y = "Column", x = "character"), #' Day of the week parameter is case insensitive, and accepts first three or two characters: #' "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". #' -#' @param y Column to compute on -#' @param x Day of the week string +#' @param y Column to compute on. +#' @param x Day of the week string. #' #' @family datetime_funcs #' @rdname next_day @@ -2637,6 +2635,7 @@ setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeri #' Parses the expression string into the column that it represents, similar to #' SparkDataFrame.selectExpr #' +#' @param x an expression character object to be parsed. #' @family normal_funcs #' @rdname expr #' @aliases expr,character-method @@ -2654,6 +2653,9 @@ setMethod("expr", signature(x = "character"), #' #' Formats the arguments in printf-style and returns the result as a string column. #' +#' @param format a character object of format strings. +#' @param x a Column. +#' @param ... additional Column(s). #' @family string_funcs #' @rdname format_string #' @name format_string @@ -2676,6 +2678,11 @@ setMethod("format_string", signature(format = "character", x = "Column"), #' representing the timestamp of that moment in the current system time zone in the given #' format. #' +#' @param x a Column of unix timestamp. +#' @param format the target format. See +#' \href{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}{ +#' Customizing Formats} for available options. +#' @param ... further arguments to be passed to or from other methods. #' @family datetime_funcs #' @rdname from_unixtime #' @name from_unixtime @@ -2702,19 +2709,21 @@ setMethod("from_unixtime", signature(x = "Column"), #' [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in #' the order of months are not supported. #' -#' The time column must be of TimestampType. -#' -#' Durations are provided as strings, e.g. '1 second', '1 day 12 hours', '2 minutes'. Valid -#' interval strings are 'week', 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. -#' If the `slideDuration` is not provided, the windows will be tumbling windows. -#' -#' The startTime is the offset with respect to 1970-01-01 00:00:00 UTC with which to start -#' window intervals. For example, in order to have hourly tumbling windows that start 15 minutes -#' past the hour, e.g. 12:15-13:15, 13:15-14:15... provide `startTime` as `15 minutes`. -#' -#' The output column will be a struct called 'window' by default with the nested columns 'start' -#' and 'end'. -#' +#' @param x a time Column. Must be of TimestampType. +#' @param windowDuration a string specifying the width of the window, e.g. '1 second', +#' '1 day 12 hours', '2 minutes'. Valid interval strings are 'week', +#' 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. +#' @param slideDuration a string specifying the sliding interval of the window. Same format as +#' \code{windowDuration}. A new window will be generated every +#' \code{slideDuration}. Must be less than or equal to +#' the \code{windowDuration}. +#' @param startTime the offset with respect to 1970-01-01 00:00:00 UTC with which to start +#' window intervals. For example, in order to have hourly tumbling windows +#' that start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide +#' \code{startTime} as \code{"15 minutes"}. +#' @param ... further arguments to be passed to or from other methods. +#' @return An output column of struct called 'window' by default with the nested columns 'start' +#' and 'end'. #' @family datetime_funcs #' @rdname window #' @name window @@ -2766,6 +2775,10 @@ setMethod("window", signature(x = "Column"), #' NOTE: The position is not zero based, but 1 based index, returns 0 if substr #' could not be found in str. #' +#' @param substr a character string to be matched. +#' @param str a Column where matches are sought for each entry. +#' @param pos start position of search. +#' @param ... further arguments to be passed to or from other methods. #' @family string_funcs #' @rdname locate #' @aliases locate,character,Column-method @@ -2785,6 +2798,9 @@ setMethod("locate", signature(substr = "character", str = "Column"), #' #' Left-pad the string column with #' +#' @param x the string Column to be left-padded. +#' @param len maximum length of each output result. +#' @param pad a character string to be padded with. #' @family string_funcs #' @rdname lpad #' @aliases lpad,Column,numeric,character-method @@ -2804,6 +2820,7 @@ setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), #' #' Generate a random column with i.i.d. samples from U[0.0, 1.0]. #' +#' @param seed a random seed. Can be missing. #' @family normal_funcs #' @rdname rand #' @name rand @@ -2832,6 +2849,7 @@ setMethod("rand", signature(seed = "numeric"), #' #' Generate a column with i.i.d. samples from the standard normal distribution. #' +#' @param seed a random seed. Can be missing. #' @family normal_funcs #' @rdname randn #' @name randn @@ -2860,6 +2878,9 @@ setMethod("randn", signature(seed = "numeric"), #' #' Extract a specific(idx) group identified by a java regex, from the specified string column. #' +#' @param x a string Column. +#' @param pattern a regular expression. +#' @param idx a group index. #' @family string_funcs #' @rdname regexp_extract #' @name regexp_extract @@ -2880,6 +2901,9 @@ setMethod("regexp_extract", #' #' Replace all substrings of the specified string value that match regexp with rep. #' +#' @param x a string Column. +#' @param pattern a regular expression. +#' @param replacement a character string that a matched \code{pattern} is replaced with. #' @family string_funcs #' @rdname regexp_replace #' @name regexp_replace @@ -2900,6 +2924,9 @@ setMethod("regexp_replace", #' #' Right-padded with pad to a length of len. #' +#' @param x the string Column to be right-padded. +#' @param len maximum length of each output result. +#' @param pad a character string to be padded with. #' @family string_funcs #' @rdname rpad #' @name rpad @@ -2922,6 +2949,11 @@ setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), #' returned. If count is negative, every to the right of the final delimiter (counting from the #' right) is returned. substring_index performs a case-sensitive match when searching for delim. #' +#' @param x a Column. +#' @param delim a delimiter string. +#' @param count number of occurrences of \code{delim} before the substring is returned. +#' A positive number means counting from the left, while negative means +#' counting from the right. #' @family string_funcs #' @rdname substring_index #' @aliases substring_index,Column,character,numeric-method @@ -2949,6 +2981,11 @@ setMethod("substring_index", #' The translate will happen when any character in the string matching with the character #' in the matchingString. #' +#' @param x a string Column. +#' @param matchingString a source string where each character will be translated. +#' @param replaceString a target string where each \code{matchingString} character will +#' be replaced by the character in \code{replaceString} +#' at the same location, if any. #' @family string_funcs #' @rdname translate #' @name translate @@ -2997,6 +3034,10 @@ setMethod("unix_timestamp", signature(x = "Column", format = "missing"), column(jc) }) +#' @param x a Column of date, in string, date or timestamp type. +#' @param format the target format. See +#' \href{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}{ +#' Customizing Formats} for available options. #' @rdname unix_timestamp #' @name unix_timestamp #' @aliases unix_timestamp,Column,character-method @@ -3012,6 +3053,8 @@ setMethod("unix_timestamp", signature(x = "Column", format = "character"), #' Evaluates a list of conditions and returns one of multiple possible result expressions. #' For unmatched expressions null is returned. #' +#' @param condition the condition to test on. Must be a Column expression. +#' @param value result expression. #' @family normal_funcs #' @rdname when #' @name when @@ -3033,6 +3076,9 @@ setMethod("when", signature(condition = "Column", value = "ANY"), #' Evaluates a list of conditions and returns \code{yes} if the conditions are satisfied. #' Otherwise \code{no} is returned for unmatched conditions. #' +#' @param test a Column expression that describes the condition. +#' @param yes return values for \code{TRUE} elements of test. +#' @param no return values for \code{FALSE} elements of test. #' @family normal_funcs #' @rdname ifelse #' @name ifelse @@ -3074,10 +3120,14 @@ setMethod("ifelse", #' @family window_funcs #' @aliases cume_dist,missing-method #' @export -#' @examples \dontrun{cume_dist()} +#' @examples \dontrun{ +#' df <- createDataFrame(iris) +#' ws <- orderBy(windowPartitionBy("Species"), "Sepal_Length") +#' out <- select(df, over(cume_dist(), ws), df$Sepal_Length, df$Species) +#' } #' @note cume_dist since 1.6.0 setMethod("cume_dist", - signature(x = "missing"), + signature("missing"), function() { jc <- callJStatic("org.apache.spark.sql.functions", "cume_dist") column(jc) @@ -3101,7 +3151,7 @@ setMethod("cume_dist", #' @examples \dontrun{dense_rank()} #' @note dense_rank since 1.6.0 setMethod("dense_rank", - signature(x = "missing"), + signature("missing"), function() { jc <- callJStatic("org.apache.spark.sql.functions", "dense_rank") column(jc) @@ -3115,6 +3165,11 @@ setMethod("dense_rank", #' #' This is equivalent to the LAG function in SQL. #' +#' @param x the column as a character string or a Column to compute on. +#' @param offset the number of rows back from the current row from which to obtain a value. +#' If not specified, the default is 1. +#' @param defaultValue default to use when the offset row does not exist. +#' @param ... further arguments to be passed to or from other methods. #' @rdname lag #' @name lag #' @aliases lag,characterOrColumn-method @@ -3143,7 +3198,7 @@ setMethod("lag", #' an `offset` of one will return the next row at any given point in the window partition. #' #' This is equivalent to the LEAD function in SQL. -#' +#' #' @param x Column to compute on #' @param offset Number of rows to offset #' @param defaultValue (Optional) default value to use @@ -3211,7 +3266,7 @@ setMethod("ntile", #' @examples \dontrun{percent_rank()} #' @note percent_rank since 1.6.0 setMethod("percent_rank", - signature(x = "missing"), + signature("missing"), function() { jc <- callJStatic("org.apache.spark.sql.functions", "percent_rank") column(jc) @@ -3243,6 +3298,8 @@ setMethod("rank", }) # Expose rank() in the R base package +#' @param x a numeric, complex, character or logical vector. +#' @param ... additional argument(s) passed to the method. #' @name rank #' @rdname rank #' @aliases rank,ANY-method @@ -3267,7 +3324,7 @@ setMethod("rank", #' @examples \dontrun{row_number()} #' @note row_number since 1.6.0 setMethod("row_number", - signature(x = "missing"), + signature("missing"), function() { jc <- callJStatic("org.apache.spark.sql.functions", "row_number") column(jc) @@ -3318,7 +3375,7 @@ setMethod("explode", #' size #' #' Returns length of array or map. -#' +#' #' @param x Column to compute on #' #' @rdname size diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 52ab730e215c2..70db7cb418287 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -380,6 +380,9 @@ setGeneric("value", function(bcast) { standardGeneric("value") }) #################### SparkDataFrame Methods ######################## +#' @param x a SparkDataFrame or GroupedData. +#' @param ... further arguments to be passed to or from other methods. +#' @return A SparkDataFrame. #' @rdname summarize #' @export setGeneric("agg", function (x, ...) { standardGeneric("agg") }) @@ -407,6 +410,8 @@ setGeneric("cache", function(x) { standardGeneric("cache") }) #' @export setGeneric("collect", function(x, ...) { standardGeneric("collect") }) +#' @param do.NULL currently not used. +#' @param prefix currently not used. #' @rdname columns #' @export setGeneric("colnames", function(x, do.NULL = TRUE, prefix = "col") { standardGeneric("colnames") }) @@ -427,15 +432,24 @@ setGeneric("coltypes<-", function(x, value) { standardGeneric("coltypes<-") }) #' @export setGeneric("columns", function(x) {standardGeneric("columns") }) -#' @rdname nrow +#' @param x a GroupedData or Column. +#' @rdname count #' @export setGeneric("count", function(x) { standardGeneric("count") }) #' @rdname cov +#' @param x a Column object or a SparkDataFrame. +#' @param ... additional argument(s). If `x` is a Column object, a Column object +#' should be provided. If `x` is a SparkDataFrame, two column names should +#' be provided. #' @export setGeneric("cov", function(x, ...) {standardGeneric("cov") }) #' @rdname corr +#' @param x a Column object or a SparkDataFrame. +#' @param ... additional argument(s). If `x` is a Column object, a Column object +#' should be provided. If `x` is a SparkDataFrame, two column names should +#' be provided. #' @export setGeneric("corr", function(x, ...) {standardGeneric("corr") }) @@ -462,10 +476,14 @@ setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) #' @export setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") }) +#' @param x a SparkDataFrame or GroupedData. +#' @param ... additional argument(s) passed to the method. #' @rdname gapply #' @export setGeneric("gapply", function(x, ...) { standardGeneric("gapply") }) +#' @param x a SparkDataFrame or GroupedData. +#' @param ... additional argument(s) passed to the method. #' @rdname gapplyCollect #' @export setGeneric("gapplyCollect", function(x, ...) { standardGeneric("gapplyCollect") }) @@ -667,8 +685,8 @@ setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") #' @export setGeneric("showDF", function(x, ...) { standardGeneric("showDF") }) -# @rdname subset -# @export +#' @rdname subset +#' @export setGeneric("subset", function(x, ...) { standardGeneric("subset") }) #' @rdname summarize @@ -735,6 +753,8 @@ setGeneric("between", function(x, bounds) { standardGeneric("between") }) setGeneric("cast", function(x, dataType) { standardGeneric("cast") }) #' @rdname columnfunctions +#' @param x a Column object. +#' @param ... additional argument(s). #' @export setGeneric("contains", function(x, ...) { standardGeneric("contains") }) @@ -830,6 +850,8 @@ setGeneric("array_contains", function(x, value) { standardGeneric("array_contain #' @export setGeneric("ascii", function(x) { standardGeneric("ascii") }) +#' @param x Column to compute on or a GroupedData object. +#' @param ... additional argument(s) when `x` is a GroupedData object. #' @rdname avg #' @export setGeneric("avg", function(x, ...) { standardGeneric("avg") }) @@ -886,9 +908,10 @@ setGeneric("crc32", function(x) { standardGeneric("crc32") }) #' @export setGeneric("hash", function(x, ...) { standardGeneric("hash") }) +#' @param x empty. Should be used with no argument. #' @rdname cume_dist #' @export -setGeneric("cume_dist", function(x) { standardGeneric("cume_dist") }) +setGeneric("cume_dist", function(x = "missing") { standardGeneric("cume_dist") }) #' @rdname datediff #' @export @@ -918,9 +941,10 @@ setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") }) #' @export setGeneric("decode", function(x, charset) { standardGeneric("decode") }) +#' @param x empty. Should be used with no argument. #' @rdname dense_rank #' @export -setGeneric("dense_rank", function(x) { standardGeneric("dense_rank") }) +setGeneric("dense_rank", function(x = "missing") { standardGeneric("dense_rank") }) #' @rdname encode #' @export @@ -1034,10 +1058,11 @@ setGeneric("md5", function(x) { standardGeneric("md5") }) #' @export setGeneric("minute", function(x) { standardGeneric("minute") }) +#' @param x empty. Should be used with no argument. #' @rdname monotonically_increasing_id #' @export setGeneric("monotonically_increasing_id", - function(x) { standardGeneric("monotonically_increasing_id") }) + function(x = "missing") { standardGeneric("monotonically_increasing_id") }) #' @rdname month #' @export @@ -1047,7 +1072,7 @@ setGeneric("month", function(x) { standardGeneric("month") }) #' @export setGeneric("months_between", function(y, x) { standardGeneric("months_between") }) -#' @rdname nrow +#' @rdname count #' @export setGeneric("n", function(x) { standardGeneric("n") }) @@ -1071,9 +1096,10 @@ setGeneric("ntile", function(x) { standardGeneric("ntile") }) #' @export setGeneric("n_distinct", function(x, ...) { standardGeneric("n_distinct") }) +#' @param x empty. Should be used with no argument. #' @rdname percent_rank #' @export -setGeneric("percent_rank", function(x) { standardGeneric("percent_rank") }) +setGeneric("percent_rank", function(x = "missing") { standardGeneric("percent_rank") }) #' @rdname pmod #' @export @@ -1114,11 +1140,12 @@ setGeneric("reverse", function(x) { standardGeneric("reverse") }) #' @rdname rint #' @export -setGeneric("rint", function(x, ...) { standardGeneric("rint") }) +setGeneric("rint", function(x) { standardGeneric("rint") }) +#' @param x empty. Should be used with no argument. #' @rdname row_number #' @export -setGeneric("row_number", function(x) { standardGeneric("row_number") }) +setGeneric("row_number", function(x = "missing") { standardGeneric("row_number") }) #' @rdname rpad #' @export @@ -1176,9 +1203,10 @@ setGeneric("sort_array", function(x, asc = TRUE) { standardGeneric("sort_array") #' @export setGeneric("soundex", function(x) { standardGeneric("soundex") }) +#' @param x empty. Should be used with no argument. #' @rdname spark_partition_id #' @export -setGeneric("spark_partition_id", function(x) { standardGeneric("spark_partition_id") }) +setGeneric("spark_partition_id", function(x = "missing") { standardGeneric("spark_partition_id") }) #' @rdname sd #' @export @@ -1276,10 +1304,16 @@ setGeneric("year", function(x) { standardGeneric("year") }) #' @export setGeneric("spark.glm", function(data, formula, ...) { standardGeneric("spark.glm") }) +#' @param x,y For \code{glm}: logical values indicating whether the response vector +#' and model matrix used in the fitting process should be returned as +#' components of the returned value. +#' @inheritParams stats::glm #' @rdname glm #' @export setGeneric("glm") +#' @param object a fitted ML model object. +#' @param ... additional argument(s) passed to the method. #' @rdname predict #' @export setGeneric("predict", function(object, ...) { standardGeneric("predict") }) @@ -1302,8 +1336,11 @@ setGeneric("spark.naiveBayes", function(data, formula, ...) { standardGeneric("s #' @rdname spark.survreg #' @export -setGeneric("spark.survreg", function(data, formula, ...) { standardGeneric("spark.survreg") }) +setGeneric("spark.survreg", function(data, formula) { standardGeneric("spark.survreg") }) +#' @param object a fitted ML model object. +#' @param path the directory where the model is saved. +#' @param ... additional argument(s) passed to the method. #' @rdname write.ml #' @export setGeneric("write.ml", function(object, path, ...) { standardGeneric("write.ml") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 85348ae76baa7..3c85ada91a444 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -59,8 +59,7 @@ setMethod("show", "GroupedData", #' Count the number of rows for each group. #' The resulting SparkDataFrame will also contain the grouping columns. #' -#' @param x a GroupedData -#' @return a SparkDataFrame +#' @return A SparkDataFrame. #' @rdname count #' @aliases count,GroupedData-method #' @export @@ -83,8 +82,6 @@ setMethod("count", #' df2 <- agg(df, = ) #' df2 <- agg(df, newColName = aggFunction(column)) #' -#' @param x a GroupedData -#' @return a SparkDataFrame #' @rdname summarize #' @aliases agg,GroupedData-method #' @name agg @@ -201,7 +198,6 @@ createMethods() #' gapply #' -#' @param x A GroupedData #' @rdname gapply #' @aliases gapply,GroupedData-method #' @name gapply @@ -216,7 +212,6 @@ setMethod("gapply", #' gapplyCollect #' -#' @param x A GroupedData #' @rdname gapplyCollect #' @aliases gapplyCollect,GroupedData-method #' @name gapplyCollect diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 50c601fcd9e1b..008d92f87d020 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -82,15 +82,16 @@ NULL #' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make #' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' -#' @param data SparkDataFrame for training. -#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', '.', ':', '+', and '-'. -#' @param family A description of the error distribution and link function to be used in the model. +#' @param family a description of the error distribution and link function to be used in the model. #' This can be a character string naming a family function, a family function or #' the result of a call to a family function. Refer R family at #' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. -#' @param tol Positive convergence tolerance of iterations. -#' @param maxIter Integer giving the maximal number of IRLS iterations. +#' @param tol positive convergence tolerance of iterations. +#' @param maxIter integer giving the maximal number of IRLS iterations. +#' @param ... additional arguments passed to the method. #' @aliases spark.glm,SparkDataFrame,formula-method #' @return \code{spark.glm} returns a fitted generalized linear model #' @rdname spark.glm @@ -142,15 +143,15 @@ setMethod("spark.glm", signature(data = "SparkDataFrame", formula = "formula"), #' Generalized Linear Models (R-compliant) #' #' Fits a generalized linear model, similarly to R's glm(). -#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', '.', ':', '+', and '-'. -#' @param data SparkDataFrame for training. -#' @param family A description of the error distribution and link function to be used in the model. +#' @param data a SparkDataFrame or R's glm data for training. +#' @param family a description of the error distribution and link function to be used in the model. #' This can be a character string naming a family function, a family function or #' the result of a call to a family function. Refer R family at #' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. -#' @param epsilon Positive convergence tolerance of iterations. -#' @param maxit Integer giving the maximal number of IRLS iterations. +#' @param epsilon positive convergence tolerance of iterations. +#' @param maxit integer giving the maximal number of IRLS iterations. #' @return \code{glm} returns a fitted generalized linear model. #' @rdname glm #' @export @@ -171,7 +172,7 @@ setMethod("glm", signature(formula = "formula", family = "ANY", data = "SparkDat # Returns the summary of a model produced by glm() or spark.glm(), similarly to R's summary(). -#' @param object A fitted generalized linear model +#' @param object a fitted generalized linear model. #' @return \code{summary} returns a summary object of the fitted model, a list of components #' including at least the coefficients, null/residual deviance, null/residual degrees #' of freedom, AIC and number of iterations IRLS takes. @@ -212,7 +213,7 @@ setMethod("summary", signature(object = "GeneralizedLinearRegressionModel"), # Prints the summary of GeneralizedLinearRegressionModel #' @rdname spark.glm -#' @param x Summary object of fitted generalized linear model returned by \code{summary} function +#' @param x summary object of fitted generalized linear model returned by \code{summary} function #' @export #' @note print.summary.GeneralizedLinearRegressionModel since 2.0.0 print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { @@ -244,7 +245,7 @@ print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { # Makes predictions from a generalized linear model produced by glm() or spark.glm(), # similarly to R's predict(). -#' @param newData SparkDataFrame for testing +#' @param newData a SparkDataFrame for testing. #' @return \code{predict} returns a SparkDataFrame containing predicted labels in a column named #' "prediction" #' @rdname spark.glm @@ -258,7 +259,7 @@ setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), # Makes predictions from a naive Bayes model or a model produced by spark.naiveBayes(), # similarly to R package e1071's predict. -#' @param newData A SparkDataFrame for testing +#' @param newData a SparkDataFrame for testing. #' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named #' "prediction" #' @rdname spark.naiveBayes @@ -271,9 +272,9 @@ setMethod("predict", signature(object = "NaiveBayesModel"), # Returns the summary of a naive Bayes model produced by \code{spark.naiveBayes} -#' @param object A naive Bayes model fitted by \code{spark.naiveBayes} +#' @param object a naive Bayes model fitted by \code{spark.naiveBayes}. #' @return \code{summary} returns a list containing \code{apriori}, the label distribution, and -#' \code{tables}, conditional probabilities given the target label +#' \code{tables}, conditional probabilities given the target label. #' @rdname spark.naiveBayes #' @export #' @note summary(NaiveBayesModel) since 2.0.0 @@ -298,14 +299,15 @@ setMethod("summary", signature(object = "NaiveBayesModel"), #' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make #' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' -#' @param data SparkDataFrame for training -#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', '.', ':', '+', and '-'. #' Note that the response variable of formula is empty in spark.kmeans. -#' @param k Number of centers -#' @param maxIter Maximum iteration number -#' @param initMode The initialization algorithm choosen to fit the model -#' @return \code{spark.kmeans} returns a fitted k-means model +#' @param k number of centers. +#' @param maxIter maximum iteration number. +#' @param initMode the initialization algorithm choosen to fit the model. +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.kmeans} returns a fitted k-means model. #' @rdname spark.kmeans #' @aliases spark.kmeans,SparkDataFrame,formula-method #' @name spark.kmeans @@ -346,8 +348,11 @@ setMethod("spark.kmeans", signature(data = "SparkDataFrame", formula = "formula" #' Get fitted result from a k-means model, similarly to R's fitted(). #' Note: A saved-loaded model does not support this method. #' -#' @param object A fitted k-means model -#' @return \code{fitted} returns a SparkDataFrame containing fitted values +#' @param object a fitted k-means model. +#' @param method type of fitted results, \code{"centers"} for cluster centers +#' or \code{"classes"} for assigned classes. +#' @param ... additional argument(s) passed to the method. +#' @return \code{fitted} returns a SparkDataFrame containing fitted values. #' @rdname fitted #' @export #' @examples @@ -371,8 +376,8 @@ setMethod("fitted", signature(object = "KMeansModel"), # Get the summary of a k-means model -#' @param object A fitted k-means model -#' @return \code{summary} returns the model's coefficients, size and cluster +#' @param object a fitted k-means model. +#' @return \code{summary} returns the model's coefficients, size and cluster. #' @rdname spark.kmeans #' @export #' @note summary(KMeansModel) since 2.0.0 @@ -398,7 +403,8 @@ setMethod("summary", signature(object = "KMeansModel"), # Predicted values based on a k-means model -#' @return \code{predict} returns the predicted values based on a k-means model +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on a k-means model. #' @rdname spark.kmeans #' @export #' @note predict(KMeansModel) since 2.0.0 @@ -414,11 +420,12 @@ setMethod("predict", signature(object = "KMeansModel"), #' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' Only categorical data is supported. #' -#' @param data A \code{SparkDataFrame} of observations and labels for model fitting -#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' @param data a \code{SparkDataFrame} of observations and labels for model fitting. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', '.', ':', '+', and '-'. -#' @param smoothing Smoothing parameter -#' @return \code{spark.naiveBayes} returns a fitted naive Bayes model +#' @param smoothing smoothing parameter. +#' @param ... additional argument(s) passed to the method. Currently only \code{smoothing}. +#' @return \code{spark.naiveBayes} returns a fitted naive Bayes model. #' @rdname spark.naiveBayes #' @aliases spark.naiveBayes,SparkDataFrame,formula-method #' @name spark.naiveBayes @@ -454,8 +461,8 @@ setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "form # Saves the Bernoulli naive Bayes model to the input path. -#' @param path The directory where the model is saved -#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' @param path the directory where the model is saved +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. #' #' @rdname spark.naiveBayes @@ -473,10 +480,9 @@ setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), # Saves the AFT survival regression model to the input path. -#' @param path The directory where the model is saved -#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. -#' #' @rdname spark.survreg #' @export #' @note write.ml(AFTSurvivalRegressionModel, character) since 2.0.0 @@ -492,8 +498,8 @@ setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "c # Saves the generalized linear model to the input path. -#' @param path The directory where the model is saved -#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. #' #' @rdname spark.glm @@ -510,8 +516,8 @@ setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", pat # Save fitted MLlib model to the input path -#' @param path The directory where the model is saved -#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE #' which means throw exception if the output path exists. #' #' @rdname spark.kmeans @@ -528,8 +534,8 @@ setMethod("write.ml", signature(object = "KMeansModel", path = "character"), #' Load a fitted MLlib model from the input path. #' -#' @param path Path of the model to read. -#' @return a fitted MLlib model +#' @param path path of the model to read. +#' @return A fitted MLlib model. #' @rdname read.ml #' @name read.ml #' @export @@ -563,11 +569,11 @@ read.ml <- function(path) { #' \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} to #' save/load fitted models. #' -#' @param data A SparkDataFrame for training -#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula #' operators are supported, including '~', ':', '+', and '-'. -#' Note that operator '.' is not supported currently -#' @return \code{spark.survreg} returns a fitted AFT survival regression model +#' Note that operator '.' is not supported currently. +#' @return \code{spark.survreg} returns a fitted AFT survival regression model. #' @rdname spark.survreg #' @seealso survival: \url{https://cran.r-project.org/web/packages/survival/} #' @export @@ -591,7 +597,7 @@ read.ml <- function(path) { #' } #' @note spark.survreg since 2.0.0 setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula"), - function(data, formula, ...) { + function(data, formula) { formula <- paste(deparse(formula), collapse = "") jobj <- callJStatic("org.apache.spark.ml.r.AFTSurvivalRegressionWrapper", "fit", formula, data@sdf) @@ -602,14 +608,14 @@ setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula # Returns a summary of the AFT survival regression model produced by spark.survreg, # similarly to R's summary(). -#' @param object A fitted AFT survival regression model +#' @param object a fitted AFT survival regression model. #' @return \code{summary} returns a list containing the model's coefficients, #' intercept and log(scale) #' @rdname spark.survreg #' @export #' @note summary(AFTSurvivalRegressionModel) since 2.0.0 setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), - function(object, ...) { + function(object) { jobj <- object@jobj features <- callJMethod(jobj, "rFeatures") coefficients <- callJMethod(jobj, "rCoefficients") @@ -622,9 +628,9 @@ setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), # Makes predictions from an AFT survival regression model or a model produced by # spark.survreg, similarly to R package survival's predict. -#' @param newData A SparkDataFrame for testing +#' @param newData a SparkDataFrame for testing. #' @return \code{predict} returns a SparkDataFrame containing predicted values -#' on the original scale of the data (mean predicted value at scale = 1.0) +#' on the original scale of the data (mean predicted value at scale = 1.0). #' @rdname spark.survreg #' @export #' @note predict(AFTSurvivalRegressionModel) since 2.0.0 diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R index b429f5de13b87..cb5bdb90175bf 100644 --- a/R/pkg/R/schema.R +++ b/R/pkg/R/schema.R @@ -92,8 +92,9 @@ print.structType <- function(x, ...) { #' #' Create a structField object that contains the metadata for a single field in a schema. #' -#' @param x The name of the field -#' @return a structField object +#' @param x the name of the field. +#' @param ... additional argument(s) passed to the method. +#' @return A structField object. #' @rdname structField #' @export #' @examples diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index f8bdee739ef02..85815af1f3639 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -320,14 +320,15 @@ sparkRHive.init <- function(jsc = NULL) { #' For details on how to initialize and use SparkR, refer to SparkR programming guide at #' \url{http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparksession}. #' -#' @param master The Spark master URL -#' @param appName Application name to register with cluster manager -#' @param sparkHome Spark Home directory -#' @param sparkConfig Named list of Spark configuration to set on worker nodes -#' @param sparkJars Character vector of jar files to pass to the worker nodes -#' @param sparkPackages Character vector of packages from spark-packages.org -#' @param enableHiveSupport Enable support for Hive, fallback if not built with Hive support; once +#' @param master the Spark master URL. +#' @param appName application name to register with cluster manager. +#' @param sparkHome Spark Home directory. +#' @param sparkConfig named list of Spark configuration to set on worker nodes. +#' @param sparkJars character vector of jar files to pass to the worker nodes. +#' @param sparkPackages character vector of packages from spark-packages.org +#' @param enableHiveSupport enable support for Hive, fallback if not built with Hive support; once #' set, this cannot be turned off on an existing session +#' @param ... named Spark properties passed to the method. #' @export #' @examples #'\dontrun{ @@ -413,9 +414,9 @@ sparkR.session <- function( #' Assigns a group ID to all the jobs started by this thread until the group ID is set to a #' different value or cleared. #' -#' @param groupid the ID to be assigned to job groups -#' @param description description for the job group ID -#' @param interruptOnCancel flag to indicate if the job is interrupted on job cancellation +#' @param groupId the ID to be assigned to job groups. +#' @param description description for the job group ID. +#' @param interruptOnCancel flag to indicate if the job is interrupted on job cancellation. #' @rdname setJobGroup #' @name setJobGroup #' @examples diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index 2b4ce195cbddb..8ea24d81729ec 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -25,6 +25,7 @@ setOldClass("jobj") #' table. The number of distinct values for each column should be less than 1e4. At most 1e6 #' non-zero pair frequencies will be returned. #' +#' @param x a SparkDataFrame #' @param col1 name of the first column. Distinct items will make the first item of each row. #' @param col2 name of the second column. Distinct items will make the column names of the output. #' @return a local R data.frame representing the contingency table. The first column of each row @@ -53,10 +54,9 @@ setMethod("crosstab", #' Calculate the sample covariance of two numerical columns of a SparkDataFrame. #' -#' @param x A SparkDataFrame -#' @param col1 the name of the first column -#' @param col2 the name of the second column -#' @return the covariance of the two columns. +#' @param colName1 the name of the first column +#' @param colName2 the name of the second column +#' @return The covariance of the two columns. #' #' @rdname cov #' @name cov @@ -71,19 +71,18 @@ setMethod("crosstab", #' @note cov since 1.6.0 setMethod("cov", signature(x = "SparkDataFrame"), - function(x, col1, col2) { - stopifnot(class(col1) == "character" && class(col2) == "character") + function(x, colName1, colName2) { + stopifnot(class(colName1) == "character" && class(colName2) == "character") statFunctions <- callJMethod(x@sdf, "stat") - callJMethod(statFunctions, "cov", col1, col2) + callJMethod(statFunctions, "cov", colName1, colName2) }) #' Calculates the correlation of two columns of a SparkDataFrame. #' Currently only supports the Pearson Correlation Coefficient. #' For Spearman Correlation, consider using RDD methods found in MLlib's Statistics. #' -#' @param x A SparkDataFrame -#' @param col1 the name of the first column -#' @param col2 the name of the second column +#' @param colName1 the name of the first column +#' @param colName2 the name of the second column #' @param method Optional. A character specifying the method for calculating the correlation. #' only "pearson" is allowed now. #' @return The Pearson Correlation Coefficient as a Double. @@ -102,10 +101,10 @@ setMethod("cov", #' @note corr since 1.6.0 setMethod("corr", signature(x = "SparkDataFrame"), - function(x, col1, col2, method = "pearson") { - stopifnot(class(col1) == "character" && class(col2) == "character") + function(x, colName1, colName2, method = "pearson") { + stopifnot(class(colName1) == "character" && class(colName2) == "character") statFunctions <- callJMethod(x@sdf, "stat") - callJMethod(statFunctions, "corr", col1, col2, method) + callJMethod(statFunctions, "corr", colName1, colName2, method) }) From e62b29f29f44196a1cbe13004ff4abfd8e5be1c1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 21 Aug 2016 22:07:47 +0200 Subject: [PATCH 1170/1470] [SPARK-17098][SQL] Fix `NullPropagation` optimizer to handle `COUNT(NULL) OVER` correctly ## What changes were proposed in this pull request? Currently, `NullPropagation` optimizer replaces `COUNT` on null literals in a bottom-up fashion. During that, `WindowExpression` is not covered properly. This PR adds the missing propagation logic. **Before** ```scala scala> sql("SELECT COUNT(1 + NULL) OVER ()").show java.lang.UnsupportedOperationException: Cannot evaluate expression: cast(0 as bigint) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) ``` **After** ```scala scala> sql("SELECT COUNT(1 + NULL) OVER ()").show +----------------------------------------------------------------------------------------------+ |count((1 + CAST(NULL AS INT))) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)| +----------------------------------------------------------------------------------------------+ | 0| +----------------------------------------------------------------------------------------------+ ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun Closes #14689 from dongjoon-hyun/SPARK-17098. (cherry picked from commit 91c2397684ab791572ac57ffb2a924ff058bb64f) Signed-off-by: Herman van Hovell --- .../sql/catalyst/optimizer/Optimizer.scala | 2 + .../sql-tests/inputs/null-propagation.sql | 9 +++++ .../results/null-propagation.sql.out | 38 +++++++++++++++++++ 3 files changed, 49 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/null-propagation.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out 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 88cc0e47fc8de..4db541f692fdf 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 @@ -582,6 +582,8 @@ object NullPropagation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { + case e @ WindowExpression(Cast(Literal(0L, _), _), _) => + Cast(Literal(0L), e.dataType) case e @ AggregateExpression(Count(exprs), _, _, _) if !exprs.exists(nonNullLiteral) => Cast(Literal(0L), e.dataType) case e @ IsNull(c) if !c.nullable => Literal.create(false, BooleanType) diff --git a/sql/core/src/test/resources/sql-tests/inputs/null-propagation.sql b/sql/core/src/test/resources/sql-tests/inputs/null-propagation.sql new file mode 100644 index 0000000000000..66549da7971d3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/null-propagation.sql @@ -0,0 +1,9 @@ + +-- count(null) should be 0 +SELECT COUNT(NULL) FROM VALUES 1, 2, 3; +SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3; + +-- count(null) on window should be 0 +SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3; +SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3; + diff --git a/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out b/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out new file mode 100644 index 0000000000000..ed3a651aa6614 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out @@ -0,0 +1,38 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 4 + + +-- !query 0 +SELECT COUNT(NULL) FROM VALUES 1, 2, 3 +-- !query 0 schema +struct +-- !query 0 output +0 + + +-- !query 1 +SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3 +-- !query 1 schema +struct +-- !query 1 output +0 + + +-- !query 2 +SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3 +-- !query 2 schema +struct +-- !query 2 output +0 +0 +0 + + +-- !query 3 +SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3 +-- !query 3 schema +struct +-- !query 3 output +0 +0 +0 From 49cc44de3ad5495b2690633791941aa00a62b553 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 22 Aug 2016 16:16:03 +0800 Subject: [PATCH 1171/1470] [SPARK-17115][SQL] decrease the threshold when split expressions ## What changes were proposed in this pull request? In 2.0, we change the threshold of splitting expressions from 16K to 64K, which cause very bad performance on wide table, because the generated method can't be JIT compiled by default (above the limit of 8K bytecode). This PR will decrease it to 1K, based on the benchmark results for a wide table with 400 columns of LongType. It also fix a bug around splitting expression in whole-stage codegen (it should not split them). ## How was this patch tested? Added benchmark suite. Author: Davies Liu Closes #14692 from davies/split_exprs. (cherry picked from commit 8d35a6f68d6d733212674491cbf31bed73fada0f) Signed-off-by: Wenchen Fan --- .../expressions/codegen/CodeGenerator.scala | 9 ++-- .../aggregate/HashAggregateExec.scala | 2 - .../benchmark/BenchmarkWideTable.scala | 53 +++++++++++++++++++ 3 files changed, 59 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala 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 16fb1f683710f..4bd9ee03f96dd 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 @@ -584,15 +584,18 @@ class CodegenContext { * @param expressions the codes to evaluate expressions. */ def splitExpressions(row: String, expressions: Seq[String]): String = { - if (row == null) { + if (row == null || currentVars != null) { // Cannot split these expressions because they are not created from a row object. return expressions.mkString("\n") } val blocks = new ArrayBuffer[String]() val blockBuilder = new StringBuilder() for (code <- expressions) { - // We can't know how many byte code will be generated, so use the number of bytes as limit - if (blockBuilder.length > 64 * 1000) { + // We can't know how many bytecode will be generated, so use the length of source code + // as metric. A method should not go beyond 8K, otherwise it will not be JITted, should + // also not be too small, or it will have many function calls (for wide table), see the + // results in BenchmarkWideTable. + if (blockBuilder.length > 1024) { blocks.append(blockBuilder.toString()) blockBuilder.clear() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index cfc47aba889aa..bd7efa606e0ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -603,8 +603,6 @@ case class HashAggregateExec( // create grouping key ctx.currentVars = input - // make sure that the generated code will not be splitted as multiple functions - ctx.INPUT_ROW = null val unsafeRowKeyCode = GenerateUnsafeProjection.createCode( ctx, groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) val vectorizedRowKeys = ctx.generateExpressions( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala new file mode 100644 index 0000000000000..9dcaca0ca93ee --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.util.Benchmark + + +/** + * Benchmark to measure performance for wide table. + * To run this: + * build/sbt "sql/test-only *benchmark.BenchmarkWideTable" + * + * Benchmarks in this file are skipped in normal builds. + */ +class BenchmarkWideTable extends BenchmarkBase { + + ignore("project on wide table") { + val N = 1 << 20 + val df = sparkSession.range(N) + val columns = (0 until 400).map{ i => s"id as id$i"} + val benchmark = new Benchmark("projection on wide table", N) + benchmark.addCase("wide table", numIters = 5) { iter => + df.selectExpr(columns : _*).queryExecution.toRdd.count() + } + benchmark.run() + + /** + * Here are some numbers with different split threshold: + * + * Split threshold methods Rate(M/s) Per Row(ns) + * 10 400 0.4 2279 + * 100 200 0.6 1554 + * 1k 37 0.9 1116 + * 8k 5 0.5 2025 + * 64k 1 0.0 21649 + */ + } +} From 2add45fabeb0ea4f7b17b5bc4910161370e72627 Mon Sep 17 00:00:00 2001 From: Jagadeesan Date: Mon, 22 Aug 2016 09:30:31 +0100 Subject: [PATCH 1172/1470] [SPARK-17085][STREAMING][DOCUMENTATION AND ACTUAL CODE DIFFERS - UNSUPPORTED OPERATIONS] Changes in Spark Stuctured Streaming doc in this link https://spark.apache.org/docs/2.0.0/structured-streaming-programming-guide.html#unsupported-operations Author: Jagadeesan Closes #14715 from jagadeesanas2/SPARK-17085. (cherry picked from commit bd9655063bdba8836b4ec96ed115e5653e246b65) Signed-off-by: Sean Owen --- docs/structured-streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 811e8c408cb45..a2f1ee2406652 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -726,9 +726,9 @@ However, note that all of the operations applicable on static DataFrames/Dataset + Full outer join with a streaming Dataset is not supported - + Left outer join with a streaming Dataset on the left is not supported + + Left outer join with a streaming Dataset on the right is not supported - + Right outer join with a streaming Dataset on the right is not supported + + Right outer join with a streaming Dataset on the left is not supported - Any kind of joins between two streaming Datasets are not yet supported. From 79195982a4c6f8b1a3e02069dea00049cc806574 Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Mon, 22 Aug 2016 10:03:48 -0700 Subject: [PATCH 1173/1470] [SPARKR][MINOR] Fix Cache Folder Path in Windows ## What changes were proposed in this pull request? This PR tries to fix the scheme of local cache folder in Windows. The name of the environment variable should be `LOCALAPPDATA` rather than `%LOCALAPPDATA%`. ## How was this patch tested? Manual test in Windows 7. Author: Junyang Qian Closes #14743 from junyangq/SPARKR-FixWindowsInstall. (cherry picked from commit 209e1b3c0683a9106428e269e5041980b6cc327f) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/install.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/R/install.R b/R/pkg/R/install.R index 987bac7bebc0e..ff81e86835ff8 100644 --- a/R/pkg/R/install.R +++ b/R/pkg/R/install.R @@ -212,7 +212,7 @@ hadoop_version_name <- function(hadoopVersion) { # adapt to Spark context spark_cache_path <- function() { if (.Platform$OS.type == "windows") { - winAppPath <- Sys.getenv("%LOCALAPPDATA%", unset = NA) + winAppPath <- Sys.getenv("LOCALAPPDATA", unset = NA) if (is.na(winAppPath)) { msg <- paste("%LOCALAPPDATA% not found.", "Please define the environment variable", From 94eff08757cee70c5b31fff7095bbb1e6ebc7ecf Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 22 Aug 2016 11:15:53 -0700 Subject: [PATCH 1174/1470] [SPARK-16320][DOC] Document G1 heap region's effect on spark 2.0 vs 1.6 ## What changes were proposed in this pull request? Collect GC discussion in one section, and documenting findings about G1 GC heap region size. ## How was this patch tested? Jekyll doc build Author: Sean Owen Closes #14732 from srowen/SPARK-16320. (cherry picked from commit 342278c09cf6e79ed4f63422988a6bbd1e7d8a91) Signed-off-by: Yin Huai --- docs/tuning.md | 36 +++++++++++++++++------------------- 1 file changed, 17 insertions(+), 19 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 976f2eb8a7b23..cbf37213aa724 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -122,21 +122,8 @@ large records. `R` is the storage space within `M` where cached blocks immune to being evicted by execution. The value of `spark.memory.fraction` should be set in order to fit this amount of heap space -comfortably within the JVM's old or "tenured" generation. Otherwise, when much of this space is -used for caching and execution, the tenured generation will be full, which causes the JVM to -significantly increase time spent in garbage collection. See -Java GC sizing documentation -for more information. - -The tenured generation size is controlled by the JVM's `NewRatio` parameter, which defaults to 2, -meaning that the tenured generation is 2 times the size of the new generation (the rest of the heap). -So, by default, the tenured generation occupies 2/3 or about 0.66 of the heap. A value of -0.6 for `spark.memory.fraction` keeps storage and execution memory within the old generation with -room to spare. If `spark.memory.fraction` is increased to, say, 0.8, then `NewRatio` may have to -increase to 6 or more. - -`NewRatio` is set as a JVM flag for executors, which means adding -`spark.executor.extraJavaOptions=-XX:NewRatio=x` to a Spark job's configuration. +comfortably within the JVM's old or "tenured" generation. See the discussion of advanced GC +tuning below for details. ## Determining Memory Consumption @@ -217,14 +204,22 @@ temporary objects created during task execution. Some steps which may be useful * Check if there are too many garbage collections by collecting GC stats. If a full GC is invoked multiple times for before a task completes, it means that there isn't enough memory available for executing tasks. -* In the GC stats that are printed, if the OldGen is close to being full, reduce the amount of - memory used for caching by lowering `spark.memory.storageFraction`; it is better to cache fewer - objects than to slow down task execution! - * If there are too many minor collections but not many major GCs, allocating more memory for Eden would help. You can set the size of the Eden to be an over-estimate of how much memory each task will need. If the size of Eden is determined to be `E`, then you can set the size of the Young generation using the option `-Xmn=4/3*E`. (The scaling up by 4/3 is to account for space used by survivor regions as well.) + +* In the GC stats that are printed, if the OldGen is close to being full, reduce the amount of + memory used for caching by lowering `spark.memory.fraction`; it is better to cache fewer + objects than to slow down task execution. Alternatively, consider decreasing the size of + the Young generation. This means lowering `-Xmn` if you've set it as above. If not, try changing the + value of the JVM's `NewRatio` parameter. Many JVMs default this to 2, meaning that the Old generation + occupies 2/3 of the heap. It should be large enough such that this fraction exceeds `spark.memory.fraction`. + +* Try the G1GC garbage collector with `-XX:+UseG1GC`. It can improve performance in some situations where + garbage collection is a bottleneck. Note that with large executor heap sizes, it may be important to + increase the [G1 region size](https://blogs.oracle.com/g1gc/entry/g1_gc_tuning_a_case) + with `-XX:G1HeapRegionSize` * As an example, if your task is reading data from HDFS, the amount of memory used by the task can be estimated using the size of the data block read from HDFS. Note that the size of a decompressed block is often 2 or 3 times the @@ -237,6 +232,9 @@ Our experience suggests that the effect of GC tuning depends on your application There are [many more tuning options](http://www.oracle.com/technetwork/java/javase/gc-tuning-6-140523.html) described online, but at a high level, managing how frequently full GC takes place can help in reducing the overhead. +GC tuning flags for executors can be specified by setting `spark.executor.extraJavaOptions` in +a job's configuration. + # Other Considerations ## Level of Parallelism From 6dcc1a3f0cc8f2ed71f7bb6b1493852a58259d2f Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 22 Aug 2016 12:53:52 -0700 Subject: [PATCH 1175/1470] [SPARKR][MINOR] Add Xiangrui and Felix to maintainers ## What changes were proposed in this pull request? This change adds Xiangrui Meng and Felix Cheung to the maintainers field in the package description. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Shivaram Venkataraman Closes #14758 from shivaram/sparkr-maintainers. (cherry picked from commit 6f3cd36f93c11265449fdce3323e139fec8ab22d) Signed-off-by: Shivaram Venkataraman --- R/pkg/DESCRIPTION | 2 ++ 1 file changed, 2 insertions(+) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 357ab007931f5..d81f1a3d4de68 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -5,6 +5,8 @@ Version: 2.0.0 Date: 2016-07-07 Author: The Apache Software Foundation Maintainer: Shivaram Venkataraman + Xiangrui Meng + Felix Cheung Depends: R (>= 3.0), methods From 01a4d69f309a1cc8d370ce9f85e6a4f31b6db3b8 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 22 Aug 2016 15:48:35 -0700 Subject: [PATCH 1176/1470] [SPARK-17162] Range does not support SQL generation ## What changes were proposed in this pull request? The range operator previously didn't support SQL generation, which made it not possible to use in views. ## How was this patch tested? Unit tests. cc hvanhovell Author: Eric Liang Closes #14724 from ericl/spark-17162. (cherry picked from commit 84770b59f773f132073cd2af4204957fc2d7bf35) Signed-off-by: Reynold Xin --- .../ResolveTableValuedFunctions.scala | 11 ++++------ .../plans/logical/basicLogicalOperators.scala | 21 ++++++++++++------- .../spark/sql/catalyst/SQLBuilder.scala | 3 +++ .../execution/basicPhysicalOperators.scala | 2 +- .../spark/sql/execution/command/views.scala | 3 +-- sql/hive/src/test/resources/sqlgen/range.sql | 4 ++++ .../resources/sqlgen/range_with_splits.sql | 4 ++++ .../sql/catalyst/LogicalPlanToSQLSuite.scala | 14 ++++++++++++- 8 files changed, 44 insertions(+), 18 deletions(-) create mode 100644 sql/hive/src/test/resources/sqlgen/range.sql create mode 100644 sql/hive/src/test/resources/sqlgen/range_with_splits.sql diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala index 7fdf7fa0c06a3..6b3bb68538dd1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala @@ -28,9 +28,6 @@ import org.apache.spark.sql.types.{DataType, IntegerType, LongType} * Rule that resolves table-valued function references. */ object ResolveTableValuedFunctions extends Rule[LogicalPlan] { - private lazy val defaultParallelism = - SparkContext.getOrCreate(new SparkConf(false)).defaultParallelism - /** * List of argument names and their types, used to declare a function. */ @@ -84,25 +81,25 @@ object ResolveTableValuedFunctions extends Rule[LogicalPlan] { "range" -> Map( /* range(end) */ tvf("end" -> LongType) { case Seq(end: Long) => - Range(0, end, 1, defaultParallelism) + Range(0, end, 1, None) }, /* range(start, end) */ tvf("start" -> LongType, "end" -> LongType) { case Seq(start: Long, end: Long) => - Range(start, end, 1, defaultParallelism) + Range(start, end, 1, None) }, /* range(start, end, step) */ tvf("start" -> LongType, "end" -> LongType, "step" -> LongType) { case Seq(start: Long, end: Long, step: Long) => - Range(start, end, step, defaultParallelism) + Range(start, end, step, None) }, /* range(start, end, step, numPartitions) */ tvf("start" -> LongType, "end" -> LongType, "step" -> LongType, "numPartitions" -> IntegerType) { case Seq(start: Long, end: Long, step: Long, numPartitions: Int) => - Range(start, end, step, numPartitions) + Range(start, end, step, Some(numPartitions)) }) ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index eb612c4c12c75..07e39b029894a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -422,17 +422,20 @@ case class Sort( /** Factory for constructing new `Range` nodes. */ object Range { - def apply(start: Long, end: Long, step: Long, numSlices: Int): Range = { + def apply(start: Long, end: Long, step: Long, numSlices: Option[Int]): Range = { val output = StructType(StructField("id", LongType, nullable = false) :: Nil).toAttributes new Range(start, end, step, numSlices, output) } + def apply(start: Long, end: Long, step: Long, numSlices: Int): Range = { + Range(start, end, step, Some(numSlices)) + } } case class Range( start: Long, end: Long, step: Long, - numSlices: Int, + numSlices: Option[Int], output: Seq[Attribute]) extends LeafNode with MultiInstanceRelation { @@ -449,6 +452,14 @@ case class Range( } } + def toSQL(): String = { + if (numSlices.isDefined) { + s"SELECT id AS `${output.head.name}` FROM range($start, $end, $step, ${numSlices.get})" + } else { + s"SELECT id AS `${output.head.name}` FROM range($start, $end, $step)" + } + } + override def newInstance(): Range = copy(output = output.map(_.newInstance())) override lazy val statistics: Statistics = { @@ -457,11 +468,7 @@ case class Range( } override def simpleString: String = { - if (step == 1) { - s"Range ($start, $end, splits=$numSlices)" - } else { - s"Range ($start, $end, step=$step, splits=$numSlices)" - } + s"Range ($start, $end, step=$step, splits=$numSlices)" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 9ac34801bf006..5e0263ec5b4c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -208,6 +208,9 @@ class SQLBuilder private ( case p: LocalRelation => p.toSQL(newSubqueryName()) + case p: Range => + p.toSQL() + case OneRowRelation => "" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index e6f7081f2916d..90bf817d5c922 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -312,7 +312,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) def start: Long = range.start def step: Long = range.step - def numSlices: Int = range.numSlices + def numSlices: Int = range.numSlices.getOrElse(sparkContext.defaultParallelism) def numElements: BigInt = range.numElements override val output: Seq[Attribute] = range.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 6533d796e8063..9e263d1cd5510 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -191,8 +191,7 @@ case class CreateViewCommand( sparkSession.sql(viewSQL).queryExecution.assertAnalyzed() } catch { case NonFatal(e) => - throw new RuntimeException( - "Failed to analyze the canonicalized SQL. It is possible there is a bug in Spark.", e) + throw new RuntimeException(s"Failed to analyze the canonicalized SQL: ${viewSQL}", e) } val viewSchema: Seq[CatalogColumn] = { diff --git a/sql/hive/src/test/resources/sqlgen/range.sql b/sql/hive/src/test/resources/sqlgen/range.sql new file mode 100644 index 0000000000000..53c72ea71e6ac --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/range.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * from range(100) +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT id AS `gen_attr_0` FROM range(0, 100, 1)) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/range_with_splits.sql b/sql/hive/src/test/resources/sqlgen/range_with_splits.sql new file mode 100644 index 0000000000000..83d637d54a302 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/range_with_splits.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select * from range(1, 100, 20, 10) +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT id AS `gen_attr_0` FROM range(1, 100, 20, 10)) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 742b065891a8e..9c6da6a628dcf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -23,7 +23,10 @@ import java.nio.file.{Files, NoSuchFileException, Paths} import scala.util.control.NonFatal import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -180,7 +183,11 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } test("Test should fail if the SQL query cannot be regenerated") { - spark.range(10).createOrReplaceTempView("not_sql_gen_supported_table_so_far") + case class Unsupported() extends LeafNode with MultiInstanceRelation { + override def newInstance(): Unsupported = copy() + override def output: Seq[Attribute] = Nil + } + Unsupported().createOrReplaceTempView("not_sql_gen_supported_table_so_far") sql("select * from not_sql_gen_supported_table_so_far") val m3 = intercept[org.scalatest.exceptions.TestFailedException] { checkSQL("select * from not_sql_gen_supported_table_so_far", "in") @@ -196,6 +203,11 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } } + test("range") { + checkSQL("select * from range(100)", "range") + checkSQL("select * from range(1, 100, 20, 10)", "range_with_splits") + } + test("in") { checkSQL("SELECT id FROM parquet_t0 WHERE id IN (1, 2, 3)", "in") } From b65b041af8b64413c7d460d4ea110b2044d6f36e Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 22 Aug 2016 15:53:10 -0700 Subject: [PATCH 1177/1470] [SPARK-16508][SPARKR] doc updates and more CRAN check fixes replace ``` ` ``` in code doc with `\code{thing}` remove added `...` for drop(DataFrame) fix remaining CRAN check warnings create doc with knitr junyangq Author: Felix Cheung Closes #14734 from felixcheung/rdoccleanup. (cherry picked from commit 71afeeea4ec8e67edc95b5d504c557c88a2598b9) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 6 +++- R/pkg/R/DataFrame.R | 69 ++++++++++++++++++++++---------------------- R/pkg/R/RDD.R | 10 +++---- R/pkg/R/SQLContext.R | 30 +++++++++---------- R/pkg/R/WindowSpec.R | 23 ++++++++------- R/pkg/R/column.R | 2 +- R/pkg/R/functions.R | 36 +++++++++++------------ R/pkg/R/generics.R | 14 ++++----- R/pkg/R/group.R | 1 + R/pkg/R/mllib.R | 5 ++-- R/pkg/R/pairRDD.R | 6 ++-- R/pkg/R/stats.R | 14 ++++----- 12 files changed, 110 insertions(+), 106 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index aaab92f5cfc7b..cdb8834161f40 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -1,5 +1,9 @@ # Imports from base R -importFrom(methods, setGeneric, setMethod, setOldClass) +# Do not include stats:: "rpois", "runif" - causes error at runtime +importFrom("methods", "setGeneric", "setMethod", "setOldClass") +importFrom("methods", "is", "new", "signature", "show") +importFrom("stats", "gaussian", "setNames") +importFrom("utils", "download.file", "packageVersion", "untar") # Disable native libraries till we figure out how to package it # See SPARKR-7839 diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 0266939de37b1..f8a05c6830765 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -150,7 +150,7 @@ setMethod("explain", #' isLocal #' -#' Returns True if the `collect` and `take` methods can be run locally +#' Returns True if the \code{collect} and \code{take} methods can be run locally #' (without any Spark executors). #' #' @param x A SparkDataFrame @@ -635,10 +635,10 @@ setMethod("unpersist", #' The following options for repartition are possible: #' \itemize{ #' \item{1.} {Return a new SparkDataFrame partitioned by -#' the given columns into `numPartitions`.} -#' \item{2.} {Return a new SparkDataFrame that has exactly `numPartitions`.} +#' the given columns into \code{numPartitions}.} +#' \item{2.} {Return a new SparkDataFrame that has exactly \code{numPartitions}.} #' \item{3.} {Return a new SparkDataFrame partitioned by the given column(s), -#' using `spark.sql.shuffle.partitions` as number of partitions.} +#' using \code{spark.sql.shuffle.partitions} as number of partitions.} #'} #' @param x a SparkDataFrame. #' @param numPartitions the number of partitions to use. @@ -1125,9 +1125,8 @@ setMethod("take", #' Head #' -#' Return the first NUM rows of a SparkDataFrame as a R data.frame. If NUM is NULL, -#' then head() returns the first 6 rows in keeping with the current data.frame -#' convention in R. +#' Return the first \code{num} rows of a SparkDataFrame as a R data.frame. If \code{num} is not +#' specified, then head() returns the first 6 rows as with R data.frame. #' #' @param x a SparkDataFrame. #' @param num the number of rows to return. Default is 6. @@ -1399,11 +1398,11 @@ setMethod("dapplyCollect", #' #' @param cols grouping columns. #' @param func a function to be applied to each group partition specified by grouping -#' column of the SparkDataFrame. The function `func` takes as argument +#' column of the SparkDataFrame. The function \code{func} takes as argument #' a key - grouping columns and a data frame - a local R data.frame. -#' The output of `func` is a local R data.frame. +#' The output of \code{func} is a local R data.frame. #' @param schema the schema of the resulting SparkDataFrame after the function is applied. -#' The schema must match to output of `func`. It has to be defined for each +#' The schema must match to output of \code{func}. It has to be defined for each #' output column with preferred output column name and corresponding data type. #' @return A SparkDataFrame. #' @family SparkDataFrame functions @@ -1490,9 +1489,9 @@ setMethod("gapply", #' #' @param cols grouping columns. #' @param func a function to be applied to each group partition specified by grouping -#' column of the SparkDataFrame. The function `func` takes as argument +#' column of the SparkDataFrame. The function \code{func} takes as argument #' a key - grouping columns and a data frame - a local R data.frame. -#' The output of `func` is a local R data.frame. +#' The output of \code{func} is a local R data.frame. #' @return A data.frame. #' @family SparkDataFrame functions #' @aliases gapplyCollect,SparkDataFrame-method @@ -1650,7 +1649,7 @@ setMethod("$", signature(x = "SparkDataFrame"), getColumn(x, name) }) -#' @param value a Column or NULL. If NULL, the specified Column is dropped. +#' @param value a Column or \code{NULL}. If \code{NULL}, the specified Column is dropped. #' @rdname select #' @name $<- #' @aliases $<-,SparkDataFrame-method @@ -1740,7 +1739,7 @@ setMethod("[", signature(x = "SparkDataFrame"), #' @family subsetting functions #' @examples #' \dontrun{ -#' # Columns can be selected using `[[` and `[` +#' # Columns can be selected using [[ and [ #' df[[2]] == df[["age"]] #' df[,2] == df[,"age"] #' df[,c("name", "age")] @@ -1785,7 +1784,7 @@ setMethod("subset", signature(x = "SparkDataFrame"), #' select(df, df$name, df$age + 1) #' select(df, c("col1", "col2")) #' select(df, list(df$name, df$age + 1)) -#' # Similar to R data frames columns can also be selected using `$` +#' # Similar to R data frames columns can also be selected using $ #' df[,df$age] #' } #' @note select(SparkDataFrame, character) since 1.4.0 @@ -2436,7 +2435,7 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { #' Return a new SparkDataFrame containing the union of rows #' #' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame -#' and another SparkDataFrame. This is equivalent to `UNION ALL` in SQL. +#' and another SparkDataFrame. This is equivalent to \code{UNION ALL} in SQL. #' Note that this does not remove duplicate rows across the two SparkDataFrames. #' #' @param x A SparkDataFrame @@ -2479,7 +2478,7 @@ setMethod("unionAll", #' Union two or more SparkDataFrames #' -#' Union two or more SparkDataFrames. This is equivalent to `UNION ALL` in SQL. +#' Union two or more SparkDataFrames. This is equivalent to \code{UNION ALL} in SQL. #' Note that this does not remove duplicate rows across the two SparkDataFrames. #' #' @param x a SparkDataFrame. @@ -2512,7 +2511,7 @@ setMethod("rbind", #' Intersect #' #' Return a new SparkDataFrame containing rows only in both this SparkDataFrame -#' and another SparkDataFrame. This is equivalent to `INTERSECT` in SQL. +#' and another SparkDataFrame. This is equivalent to \code{INTERSECT} in SQL. #' #' @param x A SparkDataFrame #' @param y A SparkDataFrame @@ -2540,7 +2539,7 @@ setMethod("intersect", #' except #' #' Return a new SparkDataFrame containing rows in this SparkDataFrame -#' but not in another SparkDataFrame. This is equivalent to `EXCEPT` in SQL. +#' but not in another SparkDataFrame. This is equivalent to \code{EXCEPT} in SQL. #' #' @param x a SparkDataFrame. #' @param y a SparkDataFrame. @@ -2569,8 +2568,8 @@ setMethod("except", #' Save the contents of SparkDataFrame 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 +#' The data source is specified by the \code{source} and a set of options (...). +#' If \code{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 @@ -2606,7 +2605,7 @@ setMethod("except", #' @note write.df since 1.4.0 setMethod("write.df", signature(df = "SparkDataFrame", path = "character"), - function(df, path, source = NULL, mode = "error", ...){ + function(df, path, source = NULL, mode = "error", ...) { if (is.null(source)) { source <- getDefaultSqlSource() } @@ -2628,14 +2627,14 @@ setMethod("write.df", #' @note saveDF since 1.4.0 setMethod("saveDF", signature(df = "SparkDataFrame", path = "character"), - function(df, path, source = NULL, mode = "error", ...){ + function(df, path, source = NULL, mode = "error", ...) { write.df(df, path, source, mode, ...) }) #' Save the contents of the SparkDataFrame 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 +#' The data source is specified by the \code{source} and a set of options (...). +#' If \code{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 @@ -2668,7 +2667,7 @@ setMethod("saveDF", #' @note saveAsTable since 1.4.0 setMethod("saveAsTable", signature(df = "SparkDataFrame", tableName = "character"), - function(df, tableName, source = NULL, mode="error", ...){ + function(df, tableName, source = NULL, mode="error", ...) { if (is.null(source)) { source <- getDefaultSqlSource() } @@ -2745,11 +2744,11 @@ setMethod("summary", #' @param how "any" or "all". #' if "any", drop a row if it contains any nulls. #' if "all", drop a row only if all its values are null. -#' if minNonNulls is specified, how is ignored. +#' if \code{minNonNulls} is specified, how is ignored. #' @param minNonNulls if specified, drop rows that have less than -#' minNonNulls non-null values. +#' \code{minNonNulls} non-null values. #' This overwrites the how parameter. -#' @param cols optional list of column names to consider. In `fillna`, +#' @param cols optional list of column names to consider. In \code{fillna}, #' columns specified in cols that do not have matching data #' type are ignored. For example, if value is a character, and #' subset contains a non-character column, then the non-character @@ -2872,8 +2871,8 @@ setMethod("fillna", #' in your system to accommodate the contents. #' #' @param x a SparkDataFrame. -#' @param row.names NULL or a character vector giving the row names for the data frame. -#' @param optional If `TRUE`, converting column names is optional. +#' @param row.names \code{NULL} or a character vector giving the row names for the data frame. +#' @param optional If \code{TRUE}, converting column names is optional. #' @param ... additional arguments to pass to base::as.data.frame. #' @return A data.frame. #' @family SparkDataFrame functions @@ -3051,7 +3050,7 @@ setMethod("str", #' @note drop since 2.0.0 setMethod("drop", signature(x = "SparkDataFrame"), - function(x, col, ...) { + function(x, col) { stopifnot(class(col) == "character" || class(col) == "Column") if (class(col) == "Column") { @@ -3211,8 +3210,8 @@ setMethod("histogram", #' and to not change the existing data. #' } #' -#' @param x s SparkDataFrame. -#' @param url JDBC database url of the form `jdbc:subprotocol:subname`. +#' @param x a SparkDataFrame. +#' @param url JDBC database url of the form \code{jdbc:subprotocol:subname}. #' @param tableName yhe name of the table in the external database. #' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default). #' @param ... additional JDBC database connection properties. @@ -3230,7 +3229,7 @@ setMethod("histogram", #' @note write.jdbc since 2.0.0 setMethod("write.jdbc", signature(x = "SparkDataFrame", url = "character", tableName = "character"), - function(x, url, tableName, mode = "error", ...){ + function(x, url, tableName, mode = "error", ...) { jmode <- convertToJSaveMode(mode) jprops <- varargsToJProperties(...) write <- callJMethod(x@sdf, "write") diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 6b254bb0d302c..6cd0704003f1a 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -887,17 +887,17 @@ setMethod("sampleRDD", # Discards some random values to ensure each partition has a # different random seed. - runif(partIndex) + stats::runif(partIndex) for (elem in part) { if (withReplacement) { - count <- rpois(1, fraction) + count <- stats::rpois(1, fraction) if (count > 0) { res[ (len + 1) : (len + count) ] <- rep(list(elem), count) len <- len + count } } else { - if (runif(1) < fraction) { + if (stats::runif(1) < fraction) { len <- len + 1 res[[len]] <- elem } @@ -965,7 +965,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", set.seed(seed) samples <- collectRDD(sampleRDD(x, withReplacement, fraction, - as.integer(ceiling(runif(1, + as.integer(ceiling(stats::runif(1, -MAXINT, MAXINT))))) # If the first sample didn't turn out large enough, keep trying to @@ -973,7 +973,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", # multiplier for thei initial size while (length(samples) < total) samples <- collectRDD(sampleRDD(x, withReplacement, fraction, - as.integer(ceiling(runif(1, + as.integer(ceiling(stats::runif(1, -MAXINT, MAXINT))))) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index a9cd2d85f898c..572e71e25b80b 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -115,7 +115,7 @@ infer_type <- function(x) { #' Get Runtime Config from the current active SparkSession #' #' Get Runtime Config from the current active SparkSession. -#' To change SparkSession Runtime Config, please see `sparkR.session()`. +#' To change SparkSession Runtime Config, please see \code{sparkR.session()}. #' #' @param key (optional) The key of the config to get, if omitted, all config is returned #' @param defaultValue (optional) The default value of the config to return if they config is not @@ -720,11 +720,11 @@ dropTempView <- function(viewName) { #' #' Returns the dataset in a data source as a SparkDataFrame #' -#' The data source is specified by the `source` and a set of options(...). -#' If `source` is not specified, the default data source configured by +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by #' "spark.sql.sources.default" will be used. \cr -#' Similar to R read.csv, when `source` is "csv", by default, a value of "NA" will be interpreted -#' as NA. +#' Similar to R read.csv, when \code{source} is "csv", by default, a value of "NA" will be +#' interpreted as NA. #' #' @param path The path of files to load #' @param source The name of external data source @@ -791,8 +791,8 @@ loadDF <- function(x, ...) { #' Creates an external table based on the dataset in a data source, #' Returns a SparkDataFrame 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 +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by #' "spark.sql.sources.default" will be used. #' #' @param tableName a name of the table. @@ -830,22 +830,22 @@ createExternalTable <- function(x, ...) { #' Additional JDBC database connection properties can be set (...) #' #' Only one of partitionColumn or predicates should be set. Partitions of the table will be -#' retrieved in parallel based on the `numPartitions` or by the predicates. +#' retrieved in parallel based on the \code{numPartitions} or by the predicates. #' #' Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash #' your external database systems. #' -#' @param url JDBC database url of the form `jdbc:subprotocol:subname` +#' @param url JDBC database url of the form \code{jdbc:subprotocol:subname} #' @param tableName the name of the table in the external database #' @param partitionColumn the name of a column of integral type that will be used for partitioning -#' @param lowerBound the minimum value of `partitionColumn` used to decide partition stride -#' @param upperBound the maximum value of `partitionColumn` used to decide partition stride -#' @param numPartitions the number of partitions, This, along with `lowerBound` (inclusive), -#' `upperBound` (exclusive), form partition strides for generated WHERE -#' clause expressions used to split the column `partitionColumn` evenly. +#' @param lowerBound the minimum value of \code{partitionColumn} used to decide partition stride +#' @param upperBound the maximum value of \code{partitionColumn} used to decide partition stride +#' @param numPartitions the number of partitions, This, along with \code{lowerBound} (inclusive), +#' \code{upperBound} (exclusive), form partition strides for generated WHERE +#' clause expressions used to split the column \code{partitionColumn} evenly. #' This defaults to SparkContext.defaultParallelism when unset. #' @param predicates a list of conditions in the where clause; each one defines one partition -#' @param ... additional JDBC database connection named propertie(s). +#' @param ... additional JDBC database connection named properties. #' @return SparkDataFrame #' @rdname read.jdbc #' @name read.jdbc diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index b55356b07d5e3..ddd2ef2fcdee5 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -44,6 +44,7 @@ windowSpec <- function(sws) { } #' @rdname show +#' @export #' @note show(WindowSpec) since 2.0.0 setMethod("show", "WindowSpec", function(object) { @@ -125,11 +126,11 @@ setMethod("orderBy", #' rowsBetween #' -#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' Defines the frame boundaries, from \code{start} (inclusive) to \code{end} (inclusive). #' -#' Both `start` and `end` are relative positions from the current row. For example, "0" means -#' "current row", while "-1" means the row before the current row, and "5" means the fifth row -#' after the current row. +#' Both \code{start} and \code{end} are relative positions from the current row. For example, +#' "0" means "current row", while "-1" means the row before the current row, and "5" means the +#' fifth row after the current row. #' #' @param x a WindowSpec #' @param start boundary start, inclusive. @@ -157,12 +158,12 @@ setMethod("rowsBetween", #' rangeBetween #' -#' Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). +#' Defines the frame boundaries, from \code{start} (inclusive) to \code{end} (inclusive). +#' +#' Both \code{start} and \code{end} are relative from the current row. For example, "0" means +#' "current row", while "-1" means one off before the current row, and "5" means the five off +#' after the current row. #' -#' Both `start` and `end` are relative from the current row. For example, "0" means "current row", -#' while "-1" means one off before the current row, and "5" means the five off after the -#' current row. - #' @param x a WindowSpec #' @param start boundary start, inclusive. #' The frame is unbounded if this is the minimum long value. @@ -195,8 +196,8 @@ setMethod("rangeBetween", #' Define a windowing column. #' #' @param x a Column, usually one returned by window function(s). -#' @param window a WindowSpec object. Can be created by `windowPartitionBy` or -#' `windowOrderBy` and configured by other WindowSpec methods. +#' @param window a WindowSpec object. Can be created by \code{windowPartitionBy} or +#' \code{windowOrderBy} and configured by other WindowSpec methods. #' @rdname over #' @name over #' @aliases over,Column,WindowSpec-method diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index af486e1ce212d..539d91b0f8797 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -284,7 +284,7 @@ setMethod("%in%", #' otherwise #' #' If values in the specified column are null, returns the value. -#' Can be used in conjunction with `when` to specify a default value for expressions. +#' Can be used in conjunction with \code{when} to specify a default value for expressions. #' #' @param x a Column. #' @param value value to replace when the corresponding entry in \code{x} is NA. diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index b3c10de71f3fe..f042adddef91f 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1250,7 +1250,7 @@ setMethod("rint", #' round #' -#' Returns the value of the column `e` rounded to 0 decimal places using HALF_UP rounding mode. +#' Returns the value of the column \code{e} rounded to 0 decimal places using HALF_UP rounding mode. #' #' @param x Column to compute on. #' @@ -1974,7 +1974,7 @@ setMethod("atan2", signature(y = "Column"), #' datediff #' -#' Returns the number of days from `start` to `end`. +#' Returns the number of days from \code{start} to \code{end}. #' #' @param x start Column to use. #' @param y end Column to use. @@ -2043,7 +2043,7 @@ setMethod("levenshtein", signature(y = "Column"), #' months_between #' -#' Returns number of months between dates `date1` and `date2`. +#' Returns number of months between dates \code{date1} and \code{date2}. #' #' @param x start Column to use. #' @param y end Column to use. @@ -2430,7 +2430,7 @@ setMethod("add_months", signature(y = "Column", x = "numeric"), #' date_add #' -#' Returns the date that is `days` days after `start` +#' Returns the date that is \code{x} days after #' #' @param y Column to compute on #' @param x Number of days to add @@ -2450,7 +2450,7 @@ setMethod("date_add", signature(y = "Column", x = "numeric"), #' date_sub #' -#' Returns the date that is `days` days before `start` +#' Returns the date that is \code{x} days before #' #' @param y Column to compute on #' @param x Number of days to substract @@ -3113,7 +3113,7 @@ setMethod("ifelse", #' N = total number of rows in the partition #' cume_dist(x) = number of values before (and including) x / N #' -#' This is equivalent to the CUME_DIST function in SQL. +#' This is equivalent to the \code{CUME_DIST} function in SQL. #' #' @rdname cume_dist #' @name cume_dist @@ -3141,7 +3141,7 @@ setMethod("cume_dist", #' and had three people tie for second place, you would say that all three were in second #' place and that the next person came in third. #' -#' This is equivalent to the DENSE_RANK function in SQL. +#' This is equivalent to the \code{DENSE_RANK} function in SQL. #' #' @rdname dense_rank #' @name dense_rank @@ -3159,11 +3159,11 @@ setMethod("dense_rank", #' lag #' -#' Window function: returns the value that is `offset` rows before the current row, and -#' `defaultValue` if there is less than `offset` rows before the current row. For example, -#' an `offset` of one will return the previous row at any given point in the window partition. +#' Window function: returns the value that is \code{offset} rows before the current row, and +#' \code{defaultValue} if there is less than \code{offset} rows before the current row. For example, +#' an \code{offset} of one will return the previous row at any given point in the window partition. #' -#' This is equivalent to the LAG function in SQL. +#' This is equivalent to the \code{LAG} function in SQL. #' #' @param x the column as a character string or a Column to compute on. #' @param offset the number of rows back from the current row from which to obtain a value. @@ -3193,11 +3193,11 @@ setMethod("lag", #' lead #' -#' Window function: returns the value that is `offset` rows after the current row, and -#' `null` if there is less than `offset` rows after the current row. For example, -#' an `offset` of one will return the next row at any given point in the window partition. +#' Window function: returns the value that is \code{offset} rows after the current row, and +#' NULL if there is less than \code{offset} rows after the current row. For example, +#' an \code{offset} of one will return the next row at any given point in the window partition. #' -#' This is equivalent to the LEAD function in SQL. +#' This is equivalent to the \code{LEAD} function in SQL. #' #' @param x Column to compute on #' @param offset Number of rows to offset @@ -3226,11 +3226,11 @@ setMethod("lead", #' ntile #' -#' Window function: returns the ntile group id (from 1 to `n` inclusive) in an ordered window -#' partition. For example, if `n` is 4, the first quarter of the rows will get value 1, the second +#' Window function: returns the ntile group id (from 1 to n inclusive) in an ordered window +#' partition. For example, if n is 4, the first quarter of the rows will get value 1, the second #' quarter will get 2, the third quarter will get 3, and the last quarter will get 4. #' -#' This is equivalent to the NTILE function in SQL. +#' This is equivalent to the \code{NTILE} function in SQL. #' #' @param x Number of ntile groups #' diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 70db7cb418287..b54a92a3c6ddd 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -438,17 +438,17 @@ setGeneric("columns", function(x) {standardGeneric("columns") }) setGeneric("count", function(x) { standardGeneric("count") }) #' @rdname cov -#' @param x a Column object or a SparkDataFrame. -#' @param ... additional argument(s). If `x` is a Column object, a Column object -#' should be provided. If `x` is a SparkDataFrame, two column names should +#' @param x a Column or a SparkDataFrame. +#' @param ... additional argument(s). If \code{x} is a Column, a Column +#' should be provided. If \code{x} is a SparkDataFrame, two column names should #' be provided. #' @export setGeneric("cov", function(x, ...) {standardGeneric("cov") }) #' @rdname corr -#' @param x a Column object or a SparkDataFrame. -#' @param ... additional argument(s). If `x` is a Column object, a Column object -#' should be provided. If `x` is a SparkDataFrame, two column names should +#' @param x a Column or a SparkDataFrame. +#' @param ... additional argument(s). If \code{x} is a Column, a Column +#' should be provided. If \code{x} is a SparkDataFrame, two column names should #' be provided. #' @export setGeneric("corr", function(x, ...) {standardGeneric("corr") }) @@ -851,7 +851,7 @@ setGeneric("array_contains", function(x, value) { standardGeneric("array_contain setGeneric("ascii", function(x) { standardGeneric("ascii") }) #' @param x Column to compute on or a GroupedData object. -#' @param ... additional argument(s) when `x` is a GroupedData object. +#' @param ... additional argument(s) when \code{x} is a GroupedData object. #' @rdname avg #' @export setGeneric("avg", function(x, ...) { standardGeneric("avg") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 3c85ada91a444..e3479ef5fa583 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -48,6 +48,7 @@ groupedData <- function(sgd) { #' @rdname show #' @aliases show,GroupedData-method +#' @export #' @note show(GroupedData) since 1.4.0 setMethod("show", "GroupedData", function(object) { diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 008d92f87d020..25dcb3a3943e6 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -429,7 +429,7 @@ setMethod("predict", signature(object = "KMeansModel"), #' @rdname spark.naiveBayes #' @aliases spark.naiveBayes,SparkDataFrame,formula-method #' @name spark.naiveBayes -#' @seealso e1071: \url{https://cran.r-project.org/web/packages/e1071/} +#' @seealso e1071: \url{https://cran.r-project.org/package=e1071} #' @export #' @examples #' \dontrun{ @@ -575,7 +575,7 @@ read.ml <- function(path) { #' Note that operator '.' is not supported currently. #' @return \code{spark.survreg} returns a fitted AFT survival regression model. #' @rdname spark.survreg -#' @seealso survival: \url{https://cran.r-project.org/web/packages/survival/} +#' @seealso survival: \url{https://cran.r-project.org/package=survival} #' @export #' @examples #' \dontrun{ @@ -604,7 +604,6 @@ setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula return(new("AFTSurvivalRegressionModel", jobj = jobj)) }) - # Returns a summary of the AFT survival regression model produced by spark.survreg, # similarly to R's summary(). diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index f0605db1e9e83..4dee3245f9b75 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -917,19 +917,19 @@ setMethod("sampleByKey", len <- 0 # mixing because the initial seeds are close to each other - runif(10) + stats::runif(10) for (elem in part) { if (elem[[1]] %in% names(fractions)) { frac <- as.numeric(fractions[which(elem[[1]] == names(fractions))]) if (withReplacement) { - count <- rpois(1, frac) + count <- stats::rpois(1, frac) if (count > 0) { res[ (len + 1) : (len + count) ] <- rep(list(elem), count) len <- len + count } } else { - if (runif(1) < frac) { + if (stats::runif(1) < frac) { len <- len + 1 res[[len]] <- elem } diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index 8ea24d81729ec..dcd7198f41ea7 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -29,9 +29,9 @@ setOldClass("jobj") #' @param col1 name of the first column. Distinct items will make the first item of each row. #' @param col2 name of the second column. Distinct items will make the column names of the output. #' @return a local R data.frame representing the contingency table. The first column of each row -#' will be the distinct values of `col1` and the column names will be the distinct values -#' of `col2`. The name of the first column will be `$col1_$col2`. Pairs that have no -#' occurrences will have zero as their counts. +#' will be the distinct values of \code{col1} and the column names will be the distinct values +#' of \code{col2}. The name of the first column will be "\code{col1}_\code{col2}". Pairs +#' that have no occurrences will have zero as their counts. #' #' @rdname crosstab #' @name crosstab @@ -116,7 +116,7 @@ setMethod("corr", #' #' @param x A SparkDataFrame. #' @param cols A vector column names to search frequent items in. -#' @param support (Optional) The minimum frequency for an item to be considered `frequent`. +#' @param support (Optional) The minimum frequency for an item to be considered \code{frequent}. #' Should be greater than 1e-4. Default support = 0.01. #' @return a local R data.frame with the frequent items in each column #' @@ -142,9 +142,9 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), #' #' Calculates the approximate quantiles of a numerical column of a SparkDataFrame. #' The result of this algorithm has the following deterministic bound: -#' If the SparkDataFrame has N elements and if we request the quantile at probability `p` up to -#' error `err`, then the algorithm will return a sample `x` from the SparkDataFrame so that the -#' *exact* rank of `x` is close to (p * N). More precisely, +#' If the SparkDataFrame has N elements and if we request the quantile at probability p up to +#' error err, then the algorithm will return a sample x from the SparkDataFrame so that the +#' *exact* rank of x is close to (p * N). More precisely, #' floor((p - err) * N) <= rank(x) <= ceil((p + err) * N). #' This method implements a variation of the Greenwald-Khanna algorithm (with some speed #' optimizations). The algorithm was first present in [[http://dx.doi.org/10.1145/375663.375670 From ff2f873800fcc3d699e52e60fd0e69eb01d12503 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 22 Aug 2016 16:32:14 -0700 Subject: [PATCH 1178/1470] [SPARK-16550][SPARK-17042][CORE] Certain classes fail to deserialize in block manager replication ## What changes were proposed in this pull request? This is a straightforward clone of JoshRosen 's original patch. I have follow-up changes to fix block replication for repl-defined classes as well, but those appear to be flaking tests so I'm going to leave that for SPARK-17042 ## How was this patch tested? End-to-end test in ReplSuite (also more tests in DistributedSuite from the original patch). Author: Eric Liang Closes #14311 from ericl/spark-16550. (cherry picked from commit 8e223ea67acf5aa730ccf688802f17f6fc10907c) Signed-off-by: Reynold Xin --- .../spark/serializer/SerializerManager.scala | 14 +++- .../apache/spark/storage/BlockManager.scala | 13 +++- .../org/apache/spark/DistributedSuite.scala | 77 ++++++------------- .../org/apache/spark/repl/ReplSuite.scala | 14 ++++ 4 files changed, 60 insertions(+), 58 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 9dc274c9fe288..07caadbe40438 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -68,7 +68,7 @@ private[spark] class SerializerManager(defaultSerializer: Serializer, conf: Spar * loaded yet. */ private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) - private def canUseKryo(ct: ClassTag[_]): Boolean = { + def canUseKryo(ct: ClassTag[_]): Boolean = { primitiveAndPrimitiveArrayClassTags.contains(ct) || ct == stringClassTag } @@ -128,8 +128,18 @@ private[spark] class SerializerManager(defaultSerializer: Serializer, conf: Spar /** Serializes into a chunked byte buffer. */ def dataSerialize[T: ClassTag](blockId: BlockId, values: Iterator[T]): ChunkedByteBuffer = { + dataSerializeWithExplicitClassTag(blockId, values, implicitly[ClassTag[T]]) + } + + /** Serializes into a chunked byte buffer. */ + def dataSerializeWithExplicitClassTag( + blockId: BlockId, + values: Iterator[_], + classTag: ClassTag[_]): ChunkedByteBuffer = { val bbos = new ChunkedByteBufferOutputStream(1024 * 1024 * 4, ByteBuffer.allocate) - dataSerializeStream(blockId, bbos, values) + val byteStream = new BufferedOutputStream(bbos) + val ser = getSerializer(classTag).newInstance() + ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() bbos.toChunkedByteBuffer } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 83a9cbd63d391..a89ce857933ce 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -497,7 +497,8 @@ private[spark] class BlockManager( diskStore.getBytes(blockId) } else if (level.useMemory && memoryStore.contains(blockId)) { // The block was not found on disk, so serialize an in-memory copy: - serializerManager.dataSerialize(blockId, memoryStore.getValues(blockId).get) + serializerManager.dataSerializeWithExplicitClassTag( + blockId, memoryStore.getValues(blockId).get, info.classTag) } else { handleLocalReadFailure(blockId) } @@ -972,8 +973,16 @@ private[spark] class BlockManager( if (level.replication > 1) { val remoteStartTime = System.currentTimeMillis val bytesToReplicate = doGetLocalBytes(blockId, info) + // [SPARK-16550] Erase the typed classTag when using default serialization, since + // NettyBlockRpcServer crashes when deserializing repl-defined classes. + // TODO(ekl) remove this once the classloader issue on the remote end is fixed. + val remoteClassTag = if (!serializerManager.canUseKryo(classTag)) { + scala.reflect.classTag[Any] + } else { + classTag + } try { - replicate(blockId, bytesToReplicate, level, classTag) + replicate(blockId, bytesToReplicate, level, remoteClassTag) } finally { bytesToReplicate.dispose() } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 6beae842b04d1..4ee0e00fde506 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -149,61 +149,16 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex sc.parallelize(1 to 10).count() } - test("caching") { + private def testCaching(storageLevel: StorageLevel): Unit = { sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).cache() - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching on disk") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching in memory, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching in memory, serialized, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_SER_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching on disk, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching in memory and disk, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching in memory and disk, serialized, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2) - - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) + sc.jobProgressListener.waitUntilExecutorsUp(2, 30000) + val data = sc.parallelize(1 to 1000, 10) + val cachedData = data.persist(storageLevel) + assert(cachedData.count === 1000) + assert(sc.getExecutorStorageStatus.map(_.rddBlocksById(cachedData.id).size).sum === + storageLevel.replication * data.getNumPartitions) + assert(cachedData.count === 1000) + assert(cachedData.count === 1000) // Get all the locations of the first partition and try to fetch the partitions // from those locations. @@ -221,6 +176,20 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } } + Seq( + "caching" -> StorageLevel.MEMORY_ONLY, + "caching on disk" -> StorageLevel.DISK_ONLY, + "caching in memory, replicated" -> StorageLevel.MEMORY_ONLY_2, + "caching in memory, serialized, replicated" -> StorageLevel.MEMORY_ONLY_SER_2, + "caching on disk, replicated" -> StorageLevel.DISK_ONLY_2, + "caching in memory and disk, replicated" -> StorageLevel.MEMORY_AND_DISK_2, + "caching in memory and disk, serialized, replicated" -> StorageLevel.MEMORY_AND_DISK_SER_2 + ).foreach { case (testName, storageLevel) => + test(testName) { + testCaching(storageLevel) + } + } + test("compute without caching when no partitions fit in memory") { val size = 10000 val conf = new SparkConf() diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index c10db947bcb44..06b09f3158d77 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -396,6 +396,20 @@ class ReplSuite extends SparkFunSuite { assertContains("ret: Array[(Int, Iterable[Foo])] = Array((1,", output) } + test("replicating blocks of object with class defined in repl") { + val output = runInterpreter("local-cluster[2,1,1024]", + """ + |import org.apache.spark.storage.StorageLevel._ + |case class Foo(i: Int) + |val ret = sc.parallelize((1 to 100).map(Foo), 10).persist(MEMORY_ONLY_2) + |ret.count() + |sc.getExecutorStorageStatus.map(s => s.rddBlocksById(ret.id).size).sum + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains(": Int = 20", output) + } + test("line wrapper only initialized once when used as encoder outer scope") { val output = runInterpreter("local", """ From 225898961bc4bc71d56f33c027adbb2d0929ae5a Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 22 Aug 2016 17:09:32 -0700 Subject: [PATCH 1179/1470] [SPARK-16577][SPARKR] Add CRAN documentation checks to run-tests.sh ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) ## How was this patch tested? This change adds CRAN documentation checks to be run as a part of `R/run-tests.sh` . As this script is also used by Jenkins this means that we will get documentation checks on every PR going forward. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Shivaram Venkataraman Closes #14759 from shivaram/sparkr-cran-jenkins. (cherry picked from commit 920806ab272ba58a369072a5eeb89df5e9b470a6) Signed-off-by: Shivaram Venkataraman --- R/check-cran.sh | 18 +++++++++++++++--- R/run-tests.sh | 27 ++++++++++++++++++++++++--- 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/R/check-cran.sh b/R/check-cran.sh index 5c90fd07f28e4..bb331466ae931 100755 --- a/R/check-cran.sh +++ b/R/check-cran.sh @@ -43,10 +43,22 @@ $FWDIR/create-docs.sh "$R_SCRIPT_PATH/"R CMD build $FWDIR/pkg # Run check as-cran. -# TODO(shivaram): Remove the skip tests once we figure out the install mechanism - VERSION=`grep Version $FWDIR/pkg/DESCRIPTION | awk '{print $NF}'` -"$R_SCRIPT_PATH/"R CMD check --as-cran SparkR_"$VERSION".tar.gz +CRAN_CHECK_OPTIONS="--as-cran" + +if [ -n "$NO_TESTS" ] +then + CRAN_CHECK_OPTIONS=$CRAN_CHECK_OPTIONS" --no-tests" +fi + +if [ -n "$NO_MANUAL" ] +then + CRAN_CHECK_OPTIONS=$CRAN_CHECK_OPTIONS" --no-manual" +fi + +echo "Running CRAN check with $CRAN_CHECK_OPTIONS options" + +"$R_SCRIPT_PATH/"R CMD check $CRAN_CHECK_OPTIONS SparkR_"$VERSION".tar.gz popd > /dev/null diff --git a/R/run-tests.sh b/R/run-tests.sh index 9dcf0ace7d97e..1a1e8ab9ffe18 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -26,6 +26,17 @@ rm -f $LOGFILE SPARK_TESTING=1 $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.default.name="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE FAILED=$((PIPESTATUS[0]||$FAILED)) +# Also run the documentation tests for CRAN +CRAN_CHECK_LOG_FILE=$FWDIR/cran-check.out +rm -f $CRAN_CHECK_LOG_FILE + +NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE +FAILED=$((PIPESTATUS[0]||$FAILED)) + +NUM_CRAN_WARNING="$(grep -c WARNING$ $CRAN_CHECK_LOG_FILE)" +NUM_CRAN_ERROR="$(grep -c ERROR$ $CRAN_CHECK_LOG_FILE)" +NUM_CRAN_NOTES="$(grep -c NOTE$ $CRAN_CHECK_LOG_FILE)" + if [[ $FAILED != 0 ]]; then cat $LOGFILE echo -en "\033[31m" # Red @@ -33,7 +44,17 @@ if [[ $FAILED != 0 ]]; then echo -en "\033[0m" # No color exit -1 else - echo -en "\033[32m" # Green - echo "Tests passed." - echo -en "\033[0m" # No color + # We have 2 existing NOTEs for new maintainer, attach() + # We have one more NOTE in Jenkins due to "No repository set" + if [[ $NUM_CRAN_WARNING != 0 || $NUM_CRAN_ERROR != 0 || $NUM_CRAN_NOTES -gt 3 ]]; then + cat $CRAN_CHECK_LOG_FILE + echo -en "\033[31m" # Red + echo "Had CRAN check errors; 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 fi From eaea1c86b897d302107a9b6833a27a2b24ca31a0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 23 Aug 2016 09:11:47 +0800 Subject: [PATCH 1180/1470] [SPARK-17182][SQL] Mark Collect as non-deterministic ## What changes were proposed in this pull request? This PR marks the abstract class `Collect` as non-deterministic since the results of `CollectList` and `CollectSet` depend on the actual order of input rows. ## How was this patch tested? Existing test cases should be enough. Author: Cheng Lian Closes #14749 from liancheng/spark-17182-non-deterministic-collect. (cherry picked from commit 2cdd92a7cd6f85186c846635b422b977bdafbcdd) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/aggregate/collect.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index ac2cefaddcf59..896ff61b23093 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -54,6 +54,10 @@ abstract class Collect extends ImperativeAggregate { override def inputAggBufferAttributes: Seq[AttributeReference] = Nil + // Both `CollectList` and `CollectSet` are non-deterministic since their results depend on the + // actual order of input rows. + override def deterministic: Boolean = false + protected[this] val buffer: Growable[Any] with Iterable[Any] override def initialize(b: MutableRow): Unit = { From d16f9a0b7c464728d7b11899740908e23820a797 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 22 Aug 2016 20:15:03 -0700 Subject: [PATCH 1181/1470] [SPARKR][MINOR] Update R DESCRIPTION file ## What changes were proposed in this pull request? Update DESCRIPTION ## How was this patch tested? Run install and CRAN tests Author: Felix Cheung Closes #14764 from felixcheung/rpackagedescription. (cherry picked from commit d2b3d3e63e1a9217de6ef507c350308017664a62) Signed-off-by: Xiangrui Meng --- R/pkg/DESCRIPTION | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index d81f1a3d4de68..e5afed2d0a93e 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -3,10 +3,15 @@ Type: Package Title: R Frontend for Apache Spark Version: 2.0.0 Date: 2016-07-07 -Author: The Apache Software Foundation -Maintainer: Shivaram Venkataraman - Xiangrui Meng - Felix Cheung +Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), + email = "shivaram@cs.berkeley.edu"), + person("Xiangrui", "Meng", role = "aut", + email = "meng@databricks.com"), + person("Felix", "Cheung", role = "aut", + email = "felixcheung@apache.org"), + person(family = "The Apache Software Foundation", role = c("aut", "cph"))) +URL: http://www.apache.org/ http://spark.apache.org/ +BugReports: https://issues.apache.org/jira/secure/CreateIssueDetails!init.jspa?pid=12315420&components=12325400&issuetype=4 Depends: R (>= 3.0), methods From 811a2cef03647c5be29fef522c423921c79b1bc3 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 23 Aug 2016 09:45:13 -0700 Subject: [PATCH 1182/1470] [SPARK-13286] [SQL] add the next expression of SQLException as cause Some JDBC driver (for example PostgreSQL) does not use the underlying exception as cause, but have another APIs (getNextException) to access that, so it it's included in the error logging, making us hard to find the root cause, especially in batch mode. This PR will pull out the next exception and add it as cause (if it's different) or suppressed (if there is another different cause). Can't reproduce this on the default JDBC driver, so did not add a regression test. Author: Davies Liu Closes #14722 from davies/keep_cause. (cherry picked from commit 9afdfc94f49395e69a7959e881c19d787ce00c3e) Signed-off-by: Davies Liu --- .../sql/execution/datasources/jdbc/JdbcUtils.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 1328ac741c216..7a8b82509383b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Connection, Driver, DriverManager, PreparedStatement} +import java.sql.{Connection, Driver, DriverManager, PreparedStatement, SQLException} import java.util.Properties import scala.collection.JavaConverters._ @@ -233,6 +233,17 @@ object JdbcUtils extends Logging { conn.commit() } committed = true + } catch { + case e: SQLException => + val cause = e.getNextException + if (e.getCause != cause) { + if (e.getCause == null) { + e.initCause(cause) + } else { + e.addSuppressed(cause) + } + } + throw e } finally { if (!committed) { // The stage must fail. We got here through an exception path, so From cc4018996740b3a68d4a557615c59c67b8996ebb Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Tue, 23 Aug 2016 11:22:32 -0700 Subject: [PATCH 1183/1470] [SPARKR][MINOR] Remove reference link for common Windows environment variables ## What changes were proposed in this pull request? The PR removes reference link in the doc for environment variables for common Windows folders. The cran check gave code 503: service unavailable on the original link. ## How was this patch tested? Manual check. Author: Junyang Qian Closes #14767 from junyangq/SPARKR-RemoveLink. (cherry picked from commit 8fd63e808e15c8a7e78fef847183c86f332daa91) Signed-off-by: Felix Cheung --- R/pkg/R/install.R | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/R/pkg/R/install.R b/R/pkg/R/install.R index ff81e86835ff8..c6ed88e032a71 100644 --- a/R/pkg/R/install.R +++ b/R/pkg/R/install.R @@ -50,9 +50,7 @@ #' \itemize{ #' \item Mac OS X: \file{~/Library/Caches/spark} #' \item Unix: \env{$XDG_CACHE_HOME} if defined, otherwise \file{~/.cache/spark} -#' \item Windows: \file{\%LOCALAPPDATA\%\\spark\\spark\\Cache}. See -#' \href{https://www.microsoft.com/security/portal/mmpc/shared/variables.aspx}{ -#' Windows Common Folder Variables} about \%LOCALAPPDATA\% +#' \item Windows: \file{\%LOCALAPPDATA\%\\spark\\spark\\Cache}. #' } #' @param overwrite If \code{TRUE}, download and overwrite the existing tar file in localDir #' and force re-install Spark (in case the local directory or file is corrupted) From a2a7506d06fe9d878d55cf5498f5bfef9a69171c Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 23 Aug 2016 21:21:43 +0100 Subject: [PATCH 1184/1470] [MINOR][DOC] Use standard quotes instead of "curly quote" marks from Mac in structured streaming programming guides MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR fixes curly quotes (`“` and `”` ) to standard quotes (`"`). This will be a actual problem when users copy and paste the examples. This would not work. This seems only happening in `structured-streaming-programming-guide.md`. Manually built. This will change some examples to be correctly marked down as below: ![2016-08-23 3 24 13](https://cloud.githubusercontent.com/assets/6477701/17882878/2a38332e-694a-11e6-8e84-76bdb89151e0.png) to ![2016-08-23 3 26 06](https://cloud.githubusercontent.com/assets/6477701/17882888/376eaa28-694a-11e6-8b88-32ea83997037.png) Author: hyukjinkwon Closes #14770 from HyukjinKwon/minor-quotes. (cherry picked from commit 588559911de94bbe0932526ee1e1dd36a581a423) Signed-off-by: Sean Owen --- .../structured-streaming-programming-guide.md | 62 +++++++++---------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index a2f1ee2406652..94b7854449a35 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -88,7 +88,7 @@ val words = lines.as[String].flatMap(_.split(" ")) val wordCounts = words.groupBy("value").count() {% endhighlight %} -This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as[String]`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as[String]`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream.
    @@ -117,7 +117,7 @@ Dataset words = lines Dataset wordCounts = words.groupBy("value").count(); {% endhighlight %} -This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream.
    @@ -142,12 +142,12 @@ words = lines.select( wordCounts = words.groupBy('word').count() {% endhighlight %} -This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named “value”, and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used two built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function `alias` to name the new column as “word”. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used two built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function `alias` to name the new column as "word". Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream.
    -We have now set up the query on the streaming data. All that is left is to actually start receiving data and computing the counts. To do this, we set it up to print the complete set of counts (specified by `outputMode(“complete”)`) to the console every time they are updated. And then start the streaming computation using `start()`. +We have now set up the query on the streaming data. All that is left is to actually start receiving data and computing the counts. To do this, we set it up to print the complete set of counts (specified by `outputMode("complete")`) to the console every time they are updated. And then start the streaming computation using `start()`.
    @@ -361,16 +361,16 @@ table, and Spark runs it as an *incremental* query on the *unbounded* input table. Let’s understand this model in more detail. ## Basic Concepts -Consider the input data stream as the “Input Table”. Every data item that is +Consider the input data stream as the "Input Table". Every data item that is arriving on the stream is like a new row being appended to the Input Table. ![Stream as a Table](img/structured-streaming-stream-as-a-table.png "Stream as a Table") -A query on the input will generate the “Result Table”. Every trigger interval (say, every 1 second), new rows get appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want to write the changed result rows to an external sink. +A query on the input will generate the "Result Table". Every trigger interval (say, every 1 second), new rows get appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want to write the changed result rows to an external sink. ![Model](img/structured-streaming-model.png) -The “Output” is defined as what gets written out to the external storage. The output can be defined in different modes +The "Output" is defined as what gets written out to the external storage. The output can be defined in different modes - *Complete Mode* - The entire updated Result Table will be written to the external storage. It is up to the storage connector to decide how to handle writing of the entire table. @@ -386,7 +386,7 @@ the final `wordCounts` DataFrame is the result table. Note that the query on streaming `lines` DataFrame to generate `wordCounts` is *exactly the same* as it would be a static DataFrame. However, when this query is started, Spark will continuously check for new data from the socket connection. If there is -new data, Spark will run an “incremental” query that combines the previous +new data, Spark will run an "incremental" query that combines the previous running counts with the new data to compute updated counts, as shown below. ![Model](img/structured-streaming-example-model.png) @@ -682,8 +682,8 @@ Streaming DataFrames can be joined with static DataFrames to create new streamin val staticDf = spark.read. ... val streamingDf = spark.readStream. ... -streamingDf.join(staticDf, “type”) // inner equi-join with a static DF -streamingDf.join(staticDf, “type”, “right_join”) // right outer join with a static DF +streamingDf.join(staticDf, "type") // inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join") // right outer join with a static DF {% endhighlight %} @@ -789,7 +789,7 @@ Here is a table of all the sinks, and the corresponding settings. File Sink
    (only parquet in Spark 2.0) Append -
    writeStream
    .format(“parquet”)
    .start()
    +
    writeStream
    .format("parquet")
    .start()
    Yes Supports writes to partitioned tables. Partitioning by time may be useful. @@ -803,14 +803,14 @@ Here is a table of all the sinks, and the corresponding settings. Console Sink Append, Complete -
    writeStream
    .format(“console”)
    .start()
    +
    writeStream
    .format("console")
    .start()
    No Memory Sink Append, Complete -
    writeStream
    .format(“memory”)
    .queryName(“table”)
    .start()
    +
    writeStream
    .format("memory")
    .queryName("table")
    .start()
    No Saves the output data as a table, for interactive querying. Table name is the query name. @@ -839,7 +839,7 @@ noAggDF .start() // ========== DF with aggregation ========== -val aggDF = df.groupBy(“device”).count() +val aggDF = df.groupBy("device").count() // Print updated aggregations to console aggDF @@ -879,7 +879,7 @@ noAggDF .start(); // ========== DF with aggregation ========== -Dataset aggDF = df.groupBy(“device”).count(); +Dataset aggDF = df.groupBy("device").count(); // Print updated aggregations to console aggDF @@ -919,7 +919,7 @@ noAggDF\ .start() # ========== DF with aggregation ========== -aggDF = df.groupBy(“device”).count() +aggDF = df.groupBy("device").count() # Print updated aggregations to console aggDF\ @@ -1093,11 +1093,11 @@ In case of a failure or intentional shutdown, you can recover the previous progr {% highlight scala %} aggDF - .writeStream - .outputMode("complete") - .option(“checkpointLocation”, “path/to/HDFS/dir”) - .format("memory") - .start() + .writeStream + .outputMode("complete") + .option("checkpointLocation", "path/to/HDFS/dir") + .format("memory") + .start() {% endhighlight %}
    @@ -1105,11 +1105,11 @@ aggDF {% highlight java %} aggDF - .writeStream() - .outputMode("complete") - .option(“checkpointLocation”, “path/to/HDFS/dir”) - .format("memory") - .start(); + .writeStream() + .outputMode("complete") + .option("checkpointLocation", "path/to/HDFS/dir") + .format("memory") + .start(); {% endhighlight %}
    @@ -1117,11 +1117,11 @@ aggDF {% highlight python %} aggDF\ - .writeStream()\ - .outputMode("complete")\ - .option(“checkpointLocation”, “path/to/HDFS/dir”)\ - .format("memory")\ - .start() + .writeStream()\ + .outputMode("complete")\ + .option("checkpointLocation", "path/to/HDFS/dir")\ + .format("memory")\ + .start() {% endhighlight %}
    From a772b4b5dea46cda1204a50a4909d40f8933ad77 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Aug 2016 22:31:58 +0200 Subject: [PATCH 1185/1470] [SPARK-17194] Use single quotes when generating SQL for string literals When Spark emits SQL for a string literal, it should wrap the string in single quotes, not double quotes. Databases which adhere more strictly to the ANSI SQL standards, such as Postgres, allow only single-quotes to be used for denoting string literals (see http://stackoverflow.com/a/1992331/590203). Author: Josh Rosen Closes #14763 from JoshRosen/SPARK-17194. (cherry picked from commit bf8ff833e30b39e5e5e35ba8dcac31b79323838c) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/catalyst/expressions/literals.scala | 4 ++-- .../src/test/resources/sqlgen/broadcast_join_subquery.sql | 2 +- sql/hive/src/test/resources/sqlgen/case_with_key.sql | 2 +- .../src/test/resources/sqlgen/case_with_key_and_else.sql | 2 +- sql/hive/src/test/resources/sqlgen/inline_tables.sql | 2 +- .../src/test/resources/sqlgen/json_tuple_generator_1.sql | 2 +- .../src/test/resources/sqlgen/json_tuple_generator_2.sql | 2 +- sql/hive/src/test/resources/sqlgen/not_like.sql | 2 +- sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql | 2 +- sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql | 2 +- .../src/test/resources/sqlgen/subquery_exists_having_1.sql | 2 +- .../src/test/resources/sqlgen/subquery_exists_having_2.sql | 2 +- .../src/test/resources/sqlgen/subquery_exists_having_3.sql | 2 +- sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql | 2 +- sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql | 2 +- sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql | 2 +- sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql | 2 +- .../test/resources/sqlgen/subquery_not_exists_having_1.sql | 2 +- .../test/resources/sqlgen/subquery_not_exists_having_2.sql | 2 +- .../spark/sql/catalyst/ExpressionSQLBuilderSuite.scala | 5 +++-- 20 files changed, 23 insertions(+), 22 deletions(-) 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 7040008769a32..55fd9c0834fcc 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 @@ -245,8 +245,8 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression with case (_, NullType | _: ArrayType | _: MapType | _: StructType) if value == null => "NULL" case _ if value == null => s"CAST(NULL AS ${dataType.sql})" case (v: UTF8String, StringType) => - // Escapes all backslashes and double quotes. - "\"" + v.toString.replace("\\", "\\\\").replace("\"", "\\\"") + "\"" + // Escapes all backslashes and single quotes. + "'" + v.toString.replace("\\", "\\\\").replace("'", "\\'") + "'" case (v: Byte, ByteType) => v + "Y" case (v: Short, ShortType) => v + "S" case (v: Long, LongType) => v + "L" diff --git a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql index 3e2111d58a3c6..ec881a216e0b0 100644 --- a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql @@ -5,4 +5,4 @@ FROM (SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) ORDER BY subq.key1, z.value -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = "2008-04-08")) AND (CAST(`gen_attr_4` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_3 +SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = '2008-04-08')) AND (CAST(`gen_attr_4` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key.sql b/sql/hive/src/test/resources/sqlgen/case_with_key.sql index dff65f10835f3..e991ebafdc90e 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_key.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_key.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN "bar" END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN 'foo' WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN 'bar' END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql index af3e169b54315..492777e376ecc 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN "bar" ELSE "baz" END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN 'foo' WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN 'bar' ELSE 'baz' END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/inline_tables.sql b/sql/hive/src/test/resources/sqlgen/inline_tables.sql index 602551e69da6e..18803a3ee59b9 100644 --- a/sql/hive/src/test/resources/sqlgen/inline_tables.sql +++ b/sql/hive/src/test/resources/sqlgen/inline_tables.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) where b > 1 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (VALUES ("one", 1), ("two", 2), ("three", CAST(NULL AS INT)) AS gen_subquery_0(gen_attr_0, gen_attr_1)) AS data WHERE (`gen_attr_1` > 1)) AS data +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (VALUES ('one', 1), ('two', 2), ('three', CAST(NULL AS INT)) AS gen_subquery_0(gen_attr_0, gen_attr_1)) AS data WHERE (`gen_attr_1` > 1)) AS data diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql index 6f5562a20cccd..11e45a48f1b89 100644 --- a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql @@ -3,4 +3,4 @@ SELECT c0, c1, c2 FROM parquet_t3 LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `c0`, `gen_attr_1` AS `c1`, `gen_attr_2` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt +SELECT `gen_attr_0` AS `c0`, `gen_attr_1` AS `c1`, `gen_attr_2` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, 'f1', 'f2', 'f3') gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql index 0d4f67f18426b..d86b39df57442 100644 --- a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql @@ -3,4 +3,4 @@ SELECT a, b, c FROM parquet_t3 LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt AS a, b, c -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_2` AS `c` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_2` AS `c` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, 'f1', 'f2', 'f3') gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/not_like.sql b/sql/hive/src/test/resources/sqlgen/not_like.sql index da39a62225a53..22485045e212e 100644 --- a/sql/hive/src/test/resources/sqlgen/not_like.sql +++ b/sql/hive/src/test/resources/sqlgen/not_like.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM t0 WHERE id + 5 NOT LIKE '1%' -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT CAST((`gen_attr_0` + CAST(5 AS BIGINT)) AS STRING) LIKE "1%")) AS t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT CAST((`gen_attr_0` + CAST(5 AS BIGINT)) AS STRING) LIKE '1%')) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql index d598e4c036a29..bd28d8dca94c2 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql @@ -5,4 +5,4 @@ where exists (select a.key from src a where b.value = a.value and a.key = b.key and a.value > 'val_9') -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_9")) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_9')) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql index a353c33af21a6..d2965fc0b9b77 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql @@ -6,4 +6,4 @@ from (select * from src a where b.value = a.value and a.key = b.key and a.value > 'val_9')) a -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_9")) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS a) AS a +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_9')) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS a) AS a diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql index f6873d24e16ec..93ce902b75994 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql @@ -6,4 +6,4 @@ having exists (select a.key from src a where a.key = b.key and a.value > 'val_9') -------------------------------------------------------------------------------- -SELECT `gen_attr_1` AS `key`, `gen_attr_2` AS `count(1)` FROM (SELECT `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_0` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > "val_9")) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3)) AS b +SELECT `gen_attr_1` AS `key`, `gen_attr_2` AS `count(1)` FROM (SELECT `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_0` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > 'val_9')) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3)) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql index 8452ef946f61d..411e073f0d280 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql @@ -7,4 +7,4 @@ from (select b.key, count(*) from src a where a.key = b.key and a.value > 'val_9')) a -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > "val_9")) AS gen_subquery_1 WHERE (`gen_attr_2` = `gen_attr_0`)) AS gen_subquery_3)) AS a) AS a +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > 'val_9')) AS gen_subquery_1 WHERE (`gen_attr_2` = `gen_attr_0`)) AS gen_subquery_3)) AS a) AS a diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql index 2ef38ce42944f..b2ed0b0557aff 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql @@ -6,4 +6,4 @@ having exists (select a.key from src a where a.value > 'val_9' and a.value = min(b.value)) -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_4`) AS `gen_attr_1`, min(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_5` AS `1` FROM (SELECT 1 AS `gen_attr_5` FROM (SELECT `gen_attr_6`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_2` > "val_9")) AS gen_subquery_2 WHERE (`gen_attr_2` = `gen_attr_3`)) AS gen_subquery_4)) AS gen_subquery_1) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_4`) AS `gen_attr_1`, min(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_5` AS `1` FROM (SELECT 1 AS `gen_attr_5` FROM (SELECT `gen_attr_6`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_2` > 'val_9')) AS gen_subquery_2 WHERE (`gen_attr_2` = `gen_attr_3`)) AS gen_subquery_4)) AS gen_subquery_1) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql index bfa58211b12f1..9894f5ab39c76 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql @@ -5,4 +5,4 @@ group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key) order by key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (CAST(`gen_attr_6` AS DOUBLE) = CAST("90" AS DOUBLE)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS src +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (CAST(`gen_attr_6` AS DOUBLE) = CAST('90' AS DOUBLE)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS src diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql index f7503bce068f8..c3a122aa889b9 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql @@ -7,4 +7,4 @@ having b.key in (select a.key where a.value > 'val_9' and a.value = min(b.value)) order by b.key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > "val_9")) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql index 54a38ec0edb4c..eed20a5d311f3 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql @@ -5,4 +5,4 @@ where not exists (select a.key from src a where b.value = a.value and a.key = b.key and a.value > 'val_2') -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_2")) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3))) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_2')) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql index c05bb5d991b4b..7040e106e7ba2 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql @@ -5,4 +5,4 @@ where not exists (select a.key from src a where b.value = a.value and a.value > 'val_2') -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT `gen_attr_4`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > "val_2")) AS gen_subquery_1 WHERE (`gen_attr_1` = `gen_attr_2`)) AS gen_subquery_3))) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT `gen_attr_4`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_2')) AS gen_subquery_1 WHERE (`gen_attr_1` = `gen_attr_2`)) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql index d6047c52f20fc..3c0e90ed42223 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql @@ -6,4 +6,4 @@ having not exists (select a.key from src a where b.value = a.value and a.key = b.key and a.value > 'val_12') -------------------------------------------------------------------------------- -SELECT `gen_attr_3` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_3`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_3`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > "val_12")) AS gen_subquery_1 WHERE ((`gen_attr_0` = `gen_attr_1`) AND (`gen_attr_2` = `gen_attr_3`))) AS gen_subquery_3))) AS b +SELECT `gen_attr_3` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_3`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_3`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > 'val_12')) AS gen_subquery_1 WHERE ((`gen_attr_0` = `gen_attr_1`) AND (`gen_attr_2` = `gen_attr_3`))) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql index 8b5402d8aa77f..0c16f9e58b9b9 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql @@ -6,4 +6,4 @@ having not exists (select distinct a.key from src a where b.value = a.value and a.value > 'val_12') -------------------------------------------------------------------------------- -SELECT `gen_attr_2` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_2`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_2`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT DISTINCT `gen_attr_4`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > "val_12")) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3))) AS b +SELECT `gen_attr_2` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_2`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_2`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT DISTINCT `gen_attr_4`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > 'val_12')) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3))) AS b diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index 7249df813b17f..93dc0f493eb7b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{If, Literal, SpecifiedWindowFr class ExpressionSQLBuilderSuite extends SQLBuilderTest { test("literal") { - checkSQL(Literal("foo"), "\"foo\"") - checkSQL(Literal("\"foo\""), "\"\\\"foo\\\"\"") + checkSQL(Literal("foo"), "'foo'") + checkSQL(Literal("\"foo\""), "'\"foo\"'") + checkSQL(Literal("'foo'"), "'\\'foo\\''") checkSQL(Literal(1: Byte), "1Y") checkSQL(Literal(2: Short), "2S") checkSQL(Literal(4: Int), "4") From a6e6a047bb9215df55b009957d4c560624d886fc Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Tue, 23 Aug 2016 23:44:45 -0700 Subject: [PATCH 1186/1470] [MINOR][SQL] Remove implemented functions from comments of 'HiveSessionCatalog.scala' ## What changes were proposed in this pull request? This PR removes implemented functions from comments of `HiveSessionCatalog.scala`: `java_method`, `posexplode`, `str_to_map`. ## How was this patch tested? Manual. Author: Weiqing Yang Closes #14769 from Sherry302/cleanComment. (cherry picked from commit b9994ad05628077016331e6b411fbc09017b1e63) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/hive/HiveSessionCatalog.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index c59ac3dcafea4..1684e8debe3e0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -230,10 +230,8 @@ private[sql] class HiveSessionCatalog( // List of functions we are explicitly not supporting are: // compute_stats, context_ngrams, create_union, // current_user, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field, - // in_file, index, java_method, - // matchpath, ngrams, noop, noopstreaming, noopwithmap, noopwithmapstreaming, - // parse_url_tuple, posexplode, reflect2, - // str_to_map, windowingtablefunction. + // in_file, index, matchpath, ngrams, noop, noopstreaming, noopwithmap, + // noopwithmapstreaming, parse_url_tuple, reflect2, windowingtablefunction. private val hiveFunctions = Seq( "hash", "histogram_numeric", From df87f161c9e40a49235ea722f6a662a488b41c4c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 23 Aug 2016 23:46:09 -0700 Subject: [PATCH 1187/1470] [SPARK-17186][SQL] remove catalog table type INDEX ## What changes were proposed in this pull request? Actually Spark SQL doesn't support index, the catalog table type `INDEX` is from Hive. However, most operations in Spark SQL can't handle index table, e.g. create table, alter table, etc. Logically index table should be invisible to end users, and Hive also generates special table name for index table to avoid users accessing it directly. Hive has special SQL syntax to create/show/drop index tables. At Spark SQL side, although we can describe index table directly, but the result is unreadable, we should use the dedicated SQL syntax to do it(e.g. `SHOW INDEX ON tbl`). Spark SQL can also read index table directly, but the result is always empty.(Can hive read index table directly?) This PR remove the table type `INDEX`, to make it clear that Spark SQL doesn't support index currently. ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #14752 from cloud-fan/minor2. (cherry picked from commit 52fa45d62a5a0bc832442f38f9e634c5d8e29e08) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/catalog/interface.scala | 1 - .../org/apache/spark/sql/execution/command/tables.scala | 8 +++----- .../org/apache/spark/sql/hive/MetastoreRelation.scala | 1 - .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 4 ++-- .../spark/sql/hive/execution/HiveCommandSuite.scala | 2 +- 5 files changed, 6 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 6197acab33786..c083cf60ce6bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -203,7 +203,6 @@ case class CatalogTableType private(name: String) object CatalogTableType { val EXTERNAL = new CatalogTableType("EXTERNAL") val MANAGED = new CatalogTableType("MANAGED") - val INDEX = new CatalogTableType("INDEX") val VIEW = new CatalogTableType("VIEW") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index b2300b416d346..a5ccbcf27cb9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -678,12 +678,11 @@ case class ShowPartitionsCommand( * Validate and throws an [[AnalysisException]] exception under the following conditions: * 1. If the table is not partitioned. * 2. If it is a datasource table. - * 3. If it is a view or index table. + * 3. If it is a view. */ - if (tab.tableType == VIEW || - tab.tableType == INDEX) { + if (tab.tableType == VIEW) { throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a view or index table: ${tab.qualifiedName}") + s"SHOW PARTITIONS is not allowed on a view: ${tab.qualifiedName}") } if (!DDLUtils.isTablePartitioned(tab)) { @@ -765,7 +764,6 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman case EXTERNAL => " EXTERNAL TABLE" case VIEW => " VIEW" case MANAGED => " TABLE" - case INDEX => reportUnsupportedError(Seq("index table")) } builder ++= s"CREATE$tableTypeString ${table.quotedString}" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 58bca2059cacc..f8ebe088e9696 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -82,7 +82,6 @@ private[hive] case class MetastoreRelation( tTable.setTableType(catalogTable.tableType match { case CatalogTableType.EXTERNAL => HiveTableType.EXTERNAL_TABLE.toString case CatalogTableType.MANAGED => HiveTableType.MANAGED_TABLE.toString - case CatalogTableType.INDEX => HiveTableType.INDEX_TABLE.toString case CatalogTableType.VIEW => HiveTableType.VIRTUAL_VIEW.toString }) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 1d408950d358a..ccbd440bc4bbe 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -378,8 +378,9 @@ private[hive] class HiveClientImpl( tableType = h.getTableType match { case HiveTableType.EXTERNAL_TABLE => CatalogTableType.EXTERNAL case HiveTableType.MANAGED_TABLE => CatalogTableType.MANAGED - case HiveTableType.INDEX_TABLE => CatalogTableType.INDEX case HiveTableType.VIRTUAL_VIEW => CatalogTableType.VIEW + case HiveTableType.INDEX_TABLE => + throw new AnalysisException("Hive index table is not supported.") }, schema = schema, partitionColumnNames = partCols.map(_.name), @@ -760,7 +761,6 @@ private[hive] class HiveClientImpl( HiveTableType.EXTERNAL_TABLE case CatalogTableType.MANAGED => HiveTableType.MANAGED_TABLE - case CatalogTableType.INDEX => HiveTableType.INDEX_TABLE case CatalogTableType.VIEW => HiveTableType.VIRTUAL_VIEW }) // Note: In Hive the schema and partition columns must be disjoint sets diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 5d510197c4d95..37e78b0f08aa1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -422,7 +422,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message4 = intercept[AnalysisException] { sql("SHOW PARTITIONS parquet_view1") }.getMessage - assert(message4.contains("is not allowed on a view or index table")) + assert(message4.contains("is not allowed on a view")) } } From ce7dce1755a8d36ec7346adc3de26d8fdc4f05e9 Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Wed, 24 Aug 2016 10:12:44 +0100 Subject: [PATCH 1188/1470] [MINOR][BUILD] Fix Java CheckStyle Error As Spark 2.0.1 will be released soon (mentioned in the spark dev mailing list), besides the critical bugs, it's better to fix the code style errors before the release. Before: ``` ./dev/lint-java Checkstyle checks failed at following occurrences: [ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[525] (sizes) LineLength: Line is longer than 100 characters (found 119). [ERROR] src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103). ``` After: ``` ./dev/lint-java Using `mvn` from path: /usr/local/bin/mvn Checkstyle checks passed. ``` Manual. Author: Weiqing Yang Closes #14768 from Sherry302/fixjavastyle. (cherry picked from commit 673a80d2230602c9e6573a23e35fb0f6b832bfca) Signed-off-by: Sean Owen --- .../util/collection/unsafe/sort/UnsafeExternalSorter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 0d67167569944..999ded45f2e2a 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -521,7 +521,8 @@ public long spill() throws IOException { // is accessing the current record. We free this page in that caller's next loadNext() // call. for (MemoryBlock page : allocatedPages) { - if (!loaded || page.pageNumber != ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) { + if (!loaded || page.pageNumber != + ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) { released += page.size(); freePage(page); } else { From 33d79b58735770ac613540c21095a1e404f065b0 Mon Sep 17 00:00:00 2001 From: VinceShieh Date: Wed, 24 Aug 2016 10:16:58 +0100 Subject: [PATCH 1189/1470] [SPARK-17086][ML] Fix InvalidArgumentException issue in QuantileDiscretizer when some quantiles are duplicated ## What changes were proposed in this pull request? In cases when QuantileDiscretizerSuite is called upon a numeric array with duplicated elements, we will take the unique elements generated from approxQuantiles as input for Bucketizer. ## How was this patch tested? An unit test is added in QuantileDiscretizerSuite QuantileDiscretizer.fit will throw an illegal exception when calling setSplits on a list of splits with duplicated elements. Bucketizer.setSplits should only accept either a numeric vector of two or more unique cut points, although that may produce less number of buckets than requested. Signed-off-by: VinceShieh Author: VinceShieh Closes #14747 from VinceShieh/SPARK-17086. (cherry picked from commit 92c0eaf348b42b3479610da0be761013f9d81c54) Signed-off-by: Sean Owen --- .../ml/feature/QuantileDiscretizer.scala | 7 ++++++- .../ml/feature/QuantileDiscretizerSuite.scala | 19 +++++++++++++++++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 558a7bbf0a2df..e09800877c694 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -114,7 +114,12 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui splits(0) = Double.NegativeInfinity splits(splits.length - 1) = Double.PositiveInfinity - val bucketizer = new Bucketizer(uid).setSplits(splits) + val distinctSplits = splits.distinct + if (splits.length != distinctSplits.length) { + log.warn(s"Some quantiles were identical. Bucketing to ${distinctSplits.length - 1}" + + s" buckets as a result.") + } + val bucketizer = new Bucketizer(uid).setSplits(distinctSplits.sorted) copyValues(bucketizer.setParent(this)) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala index b73dbd62328cf..18f1e89ee8148 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala @@ -52,6 +52,25 @@ class QuantileDiscretizerSuite "Bucket sizes are not within expected relative error tolerance.") } + test("Test Bucketizer on duplicated splits") { + val spark = this.spark + import spark.implicits._ + + val datasetSize = 12 + val numBuckets = 5 + val df = sc.parallelize(Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0, 1.0, 3.0)) + .map(Tuple1.apply).toDF("input") + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setOutputCol("result") + .setNumBuckets(numBuckets) + val result = discretizer.fit(df).transform(df) + + val observedNumBuckets = result.select("result").distinct.count + assert(2 <= observedNumBuckets && observedNumBuckets <= numBuckets, + "Observed number of buckets are not within expected range.") + } + test("Test transform method on unseen data") { val spark = this.spark import spark.implicits._ From 29091d7cd60c20bf019dc9c1625a22e80ea50928 Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Wed, 24 Aug 2016 10:40:09 -0700 Subject: [PATCH 1190/1470] [SPARKR][MINOR] Fix doc for show method ## What changes were proposed in this pull request? The original doc of `show` put methods for multiple classes together but the text only talks about `SparkDataFrame`. This PR tries to fix this problem. ## How was this patch tested? Manual test. Author: Junyang Qian Closes #14776 from junyangq/SPARK-FixShowDoc. (cherry picked from commit d2932a0e987132c694ed59515b7c77adaad052e6) Signed-off-by: Felix Cheung --- R/pkg/R/DataFrame.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index f8a05c6830765..ab45d2c1a42c0 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -205,9 +205,9 @@ setMethod("showDF", #' show #' -#' Print the SparkDataFrame column names and types +#' Print class and type information of a Spark object. #' -#' @param object a SparkDataFrame. +#' @param object a Spark object. Can be a SparkDataFrame, Column, GroupedData, WindowSpec. #' #' @family SparkDataFrame functions #' @rdname show From 9f924a01b27ebba56080c9ad01b84fff026d5dcd Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 24 Aug 2016 20:04:09 +0100 Subject: [PATCH 1191/1470] [SPARK-16781][PYSPARK] java launched by PySpark as gateway may not be the same java used in the spark environment ## What changes were proposed in this pull request? Update to py4j 0.10.3 to enable JAVA_HOME support ## How was this patch tested? Pyspark tests Author: Sean Owen Closes #14748 from srowen/SPARK-16781. (cherry picked from commit 0b3a4be92ca6b38eef32ea5ca240d9f91f68aa65) Signed-off-by: Sean Owen --- LICENSE | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/docs/Makefile | 2 +- python/lib/py4j-0.10.1-src.zip | Bin 61356 -> 0 bytes python/lib/py4j-0.10.3-src.zip | Bin 0 -> 91275 bytes sbin/spark-config.sh | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 6 +++--- .../spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 16 files changed, 16 insertions(+), 16 deletions(-) delete mode 100644 python/lib/py4j-0.10.1-src.zip create mode 100644 python/lib/py4j-0.10.3-src.zip diff --git a/LICENSE b/LICENSE index 94fd46f568473..d68609cc28733 100644 --- a/LICENSE +++ b/LICENSE @@ -263,7 +263,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.1 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.3 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) diff --git a/bin/pyspark b/bin/pyspark index ac8aa04dba8a3..037645dbd64d7 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -65,7 +65,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.1-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.3-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 3e2ff100fb8af..1217a4f2f97a2 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.1-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.3-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index bb27ec916c65a..208659bcb92da 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -327,7 +327,7 @@ net.sf.py4j py4j - 0.10.1 + 0.10.3 org.apache.spark diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 64cf4981714c0..701097ace8974 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -32,7 +32,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.1-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.3-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index f4a39ab47f0ee..f5175f2483cc7 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -140,7 +140,7 @@ parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.1.jar +py4j-0.10.3.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index ba93f90b72781..49d35abe14da7 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -147,7 +147,7 @@ parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.1.jar +py4j-0.10.3.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 19459b6fc30f9..f7d5ab45b300f 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -147,7 +147,7 @@ parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.1.jar +py4j-0.10.3.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index f7022a17e26d9..f27e7bacc04de 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -155,7 +155,7 @@ parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.1.jar +py4j-0.10.3.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 36f0fa592c7e7..934626a5723af 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -156,7 +156,7 @@ parquet-jackson-1.7.0.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.1.jar +py4j-0.10.3.jar pyrolite-4.9.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/python/docs/Makefile b/python/docs/Makefile index 12e397e4507c5..de86e97d862f0 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -7,7 +7,7 @@ SPHINXBUILD ?= sphinx-build PAPER ?= BUILDDIR ?= _build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.1-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.3-src.zip) # User-friendly check for sphinx-build ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) diff --git a/python/lib/py4j-0.10.1-src.zip b/python/lib/py4j-0.10.1-src.zip deleted file mode 100644 index a54bcae03afb823da3b2b69814811d192db44630..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 61356 zcmb5VV~}XgvMt)SZQHhO+qUi1wr$&XueNR5-K%ZCwf8;e$M@pC*!#_hnlb-W&WNnc zF)Bw^DM$l@Kmq*gK~}Dk`1i;Eyg&hP031A7tmsu$AOV0^teDkB%b3+&J)i*qLC$~y z0RH___&Xit?*s?{amO5qn}#$mRagK3ATa;{jQ^c(W$0$8Z)|U4V`}VTX>aFD=ivEo zi>=ed>i-)5FN;??w)UGGNZ;4`{m#7-+N%TPoAXuZKSQZ< zZJSrwri-dX8gxfY(xyD2H%&C)2aj4upbYh^pP7Rt+?E$$;571sub?ytt`oS7->Y)1{vg?)^8gvqu ze+{{59`UlBAocpk7BJ6wLSJRbCBCvMToYZ!^m@jCk@{sGNmVSdo`~^OjTA@;SyCTYacj=b2?i1S3Yf*8 z{>-78VE7oXg@}zwM)l63q`@TRKmst83`CgQ|4U{|4rHHjk}OGe257$b;-$L#!KOWP z3LL=2ASR#y7O*!RohudxBgDwY>M$tZNX=O@6O{vfxmqQ;L{y6Wveq5-`(^I|i32FE zfl$b6Y0jF^nMjVcd0020&KDOKy{!r>87R3xiltIQc=D|jRF5bkxf`N1MxQ4ekOERi z;HfQ&ZS0Mp(;@1;|m)YSRo*1 zqDSPxy`TiUb3zdK+RZIk-e|kqPtwePo35+6=>_A8AQr_aOqa;POP4T0f?Y^lf8+-_ zV3Y}U9IPdR5T3n!Kv>TR z7Q8IGs$0bYAh;LB{14BYI^bu|pE^7I`!P^@pd&cGkoK;@v`dD^Gsg8#5I7-~M$33C zs_M$MuTRY4AJxcfA(tn9Ge;~0?K5&9i@8wH$PUfPgJ3#y)qy(iYN_yvXHiVipr)vK z{CNk40bOK^AorU=I>JY-XTq2S0?d_d8t=B3+^;D=6CywbqKWz{Qvi1sAi*;6q5xE2 zv~{V(hV3>nNWL7lLXc+ns^)n9+wO!+FRMqZH&&){I`l9Df$jBh+;R(r;TTIAa-Q9> zGQ#|cimWWme$S;NyIEOplxoY3-mMzg?9JwYwsJwYs#=rLwf>WbvR00U8lE)W8_m1ud2N$rz6 zBDw=>iuU~S2~hPDreb!@?+BP#<@ymXN`jQtmG}3#Gg|-Q#2K=Migs6KY`(Ad#9_*` z;eBejs%_o&`cHr>o9NDfu2DZe5x{y-0!`yrp*CQWzT&f&v2Pz5$^Ze*8YGFJQ&+)~ zg&G@d0-1tsNG+j#U3Jx?wZ8oKQGrRu(#~nJV3T}Cd8uL9U}YW_>PdF466sL2JXHYN zl>4uu$;l%A!b5g*bMez&(Nl4A;O9?ydF!-meSe6-wqCp9HcY!@#Vvtg5JHTm5A%(h zio4Hxw?L$`^Cee5yJWn+IkAwaqRE7kql#|C4%T4&G+#;s2wIxCpVht!FCS!z4BO%C zydQC<2y!!Skjp{%a}F!^LwI@lSnxPwKM*KXxAZ(J-Y<>qBZ7lZ?oM;z9N;MK)OtY_ z9DglU^`prNunOB@h@kg(b%O*d8xhbJipkr2R=iJx1L8-<(n_&E`;NM`;P5-0mcsdn zHt<;fOJF8uEdUOKIS8RB>hVr57Hxqbo2g_iviQ>gTzk*BqPMWK6onMkW`vLx===SZ zZDM06ysqCMU9HTD_jJd)1~#Ac^&);4#t=yuzbdI7RKRfrDQRppXf(X2ABJUFTduYh z(thzm;Q3iY)FN6(vBvY6a;#mnn@Xz)Kx(M6kkOKqA+o_}^O6xmSrF6-gos0AcQ6+o zR3zwFCq||{{*f5r4?lR%Z*51_`HanmqPIeMJ--Kn=}<9;Q5sRV%%Bn$6YfxH%RtTt z?J$T4Y#LUB?lKVH@8| z3qBZGaAz0b!p-LABH-oSbD&yMzEysJ)RE4)(3^Z~b9~_lC%(Rr#PJ5@&Tl_nE50UX zXl(#fO`{)rvisQ^eIe|MFieKj6=&{#)2o#=lrCg;m4A7EVOHgUYstsBiBU@&fG5KU zYB8|M4%3MlM=_jLsV7iJokw;}3QB6x5(MnYoVrUy9o=8FtiF!SuN=5qOg@*AuS_&& zW}sIz_w+$6;~_LOX}AByD8N4d!j<_kK`swZ*R3lnv-+~&eY&vugsBIxPI3u-k~xub zY>;)o>E%|-UNrK_lQm7U9euSFg`P2Fd`C4%C*$)u0a` zzqFbPls}c^bGk~WRb%JA489T^!x|`Ea%0qUx$>T`g(f3ESqWcu^?ne4^-+<{WCvqy zyvw=+kyr08B6=h`tjEv33GIj`t2WcIpRWrE<1mNk80>1Ai;pD>14cXxEF@VoAT0b~ zcjH?$5Za`xi?mc+hgDE{%U&amU&nVmNovOPk^@0ik-TQ3P5OIZZb0m=wohYQZZrPQ za>bgX6jtYXvLI!r^HUq=WR-|6k4k((5N@)i@#nW|dg_yf`hhNRPjSGr_A&bvc2)?x zs+D$H*Xw%W(Xt8lgx*#2bP2s*EA_9thg+odcKiaE5&5P7OS0X`{Sl|vZ}I0c^1*qu z62^pe;E$Wrr`_30{L}7lYnsSu_+VeiLG~r2*LAwCQF+?BP|)IFyEFs-u>`l~va_L$ zAoB8a-Ps(3m^3Z9GBp9UDMDUf91|WtXFng74N$E4{&EH3UV0H^e$(Pf_Kfv7PSFaB z?hWfChy&H}5JTJe>qzbDMBUHp=jIttqP?tCFkv1mp5iMiDRl!b+^-Q1NJ~W@$(aa8`|@y&gMDs?2~crWvI(} zMC9_*aC*7!O*-(iC+qBK1^XGttkli|YcY}=lJ@3BE1-7Tm?0_BG0y)n4EaCi_(uermmx-I5o9Ru4uqib&&!9h>+L`%$EY`=xF* zYc|@nA&!p~aOcoA7GRJH21*-h9TR&Wwt2#q7PjG^guSid9b0(8-vShyCQg zL9fS@;wZsDe@K)tcvo4}3~a?tWSuXEYda)wLeXHd)8?je8fL|LMbCP(ua&F=wbt6} z>iAi_&bGlGMYTUQ0=tMP6oqKMfCIxF4reHYsIbM%ecmXl8n0!?hKip}lB%xO4PW+0 zMq40}*P2}lKIjJNB?8H6HFBJEj2(T{mB0*x_63Ky&xX>HQ7ONyn)%}0xNLvf5!?zO z^);o;J#3N*;gci8Y5!yfR5Gh1I=HzY#(cM9JZcP}A5z-?IuXkl>`vSIwvcPz?n}Pj zayQYU=Z3GrOYOQ@yRa?K0WV|kxaMTBW95eTVZzeEi`miH@pkz*rDLnO{q@f>?ZSAo zs|_9iz=#n50O5bjGzTYp7yG|u+J7j|TClADmS^`}exULn!#nLl3t(N)yKaD^irVhD zo95iU`PyU6GMiRU4382QwC}t2p~VzRiAi>j=qR!xGf~Wgo|3J$+LOG*VJzdw{E(8# zHj(iOu$y3vlM*v8ffQpw+#bbuq4W(?Wnlz62}H)&jeJ^7hsV~ULk<3B@rbYDk~6=L zE8XhGC zm_j`8kGzoKOpA0;d%STH*YHdPUn^9zg58O%=~haNiD+{SsQ z!}EqT>kTQxafgIpStdIW)DTW-Vk{+AXr5mXW%-M!8hxE++3syP$&ngm_)QSB{V_~Q z))Q%7vmjl;5S=9|>SO^I$g+1+-#hZfB-4-3h(zh3-M53M)gk{A@c4zGKlyGCRH)HK|Yk%lmoqxX#I%#gj2JI`zhI7H`<$EAZ1IswU#Y zsUD}t5jLINHNPJujoifunBAQbXG+KZ)rw(sx%WG8I4>Og^z4nI{8%pa;bq4L>cC^M zYW3RTJf;?rt*vI?g()Cp-sNEWWHEzV|2Mm#&n8g9R$I`^&OlDDtV z)^*f8H^^((?6K$9$w;T0K8)R{H2in!+Bx~AMh=Xfj!syB{m)N!M-a zjkyp0+#bWVbbP4}l#Mk{PPfXe@n&bH+~~DIVd_esjh8p8+wuh1I=nc0P=i+I2=kOU zR>@TB9?SJs4h;3wQY+X4Cr-BxfFb&|67RKO_I#liob9CyC{0@;7d|<@pTwq-9ANu7el6 zN_)?)>b-qOqdAjG;H(2M4Q+kA_5A?E^y85&`dGpcuSj+)2hWuO zZB;Y%9Ds?I1c!)Qdu;jjKv-Q#Y6ztj&@=TmzMB9Ywh5aYV@^Tv?nT2cGpbG-^Cxsee?m6FlBozvTGUD;!*XE2Yr4w= zJNhwpfeLZ?$mS z32#}XIo0dHh1SVkfRFVW?e`PR)KyP4(|Sm{)_BU;08;8AUN;D0K2$#%`1Q(nGRiNq zGoOoIDZ(0}T^rEJt@^#41yC|4tjLuh*^dpKYAsDeFhoCNeLMg91aE%c70y&Jrd*T*{@-nD7q1&_^w8E4T zJ+%#pkVez|mg`^)OjhgxJ&DD4@#@OnzNdz5FdV;(tn0p2eUi#^5yGlrPxd)aFwT=n zOm`J^tX9`SUBiWi9ua)gmxH)c-Z)P@)nF?DZ6a;0+}%)UXuP={tp!=7ZexR4XG)jw zT3-tVd1{PMK+Uz?D>S`RSyf1?uVmYdZ!;9K-fCWGf-MU7IuEh+?$@t-`o<0 z#}DlWJ-K9bN!L+6=0nD%ehi5)p&>W~@tSHbB))qG-$PDND(1V15d8?nZOkVST*T=N z;EPuZ1~u@6`sOyBs~K4nF6$ttQNdl492VqWqkO8@58QpXqEb`V_>4+ICYij_uo-nz zuWYqMrqqhElP-{5PDOcit;j-`gv#NMfg|MTBcsgksCy)TqA^Wc856+fiYEy!%46MJ z>;j-TfC1Y$Vd1Q2;W;oCdC;LunkA3akO^h;aoZ0X8UE~N$wcYapN^zT!ay7RS^$Gz zld(3-k)P{!O9oYVXd(UF2SdIDC+7|p+?mHdVE(hzxm_Eu2nUW<)2AQ|o((?E+)`*E zzrBDBQ#M%)4Wd0~fpAz2>_ed}4B1ihL8EVA2rN89zyh;lhN^Hi^asvPb)vi%oHgj& zCzCPRcVra(NHZtmDAwePXqWG8`dX?L(^{g~e-b%zJcc|F{bv_hP$EAP6#;c1t^fwN zc_|Vs+BueBXCnoGtK@lKqJ$=Cj@SiHIL`7jA0sH@y3uNPNj?Q>)TdojA=*6KVv;O!4$tdHVQk+xL*eLY4r9 z>9_ePM7%Ub@ErFLzRZ#FgPh#XjVugqMnId)_z} zF}yL}L=_$bWy`Etg|VEDltF2I&j7Tb5W5Qb{>*W7i635B%v_((KM4EOKq=zr{}8d~ zihF&woQ@|WAq#?Ge9*K9am?=G8VQKyqzp~!7?b3;qAcV_C)#GHi?ExSpb9E4BXFnk z35Yq^6$CEGy_FV|bnE=yV^vD!aHPd9FcS6F;O~mHy3mTJP$gLzVh7;)5k-6kt*4GG zNUXgtOaq!HT*Sbd!hnf4eWlVv<5*zm?vz|lEph1AeDl)#TZ~zR0RE1RJ1eM)s?rgZ zh|KkuT0&xNd;a3Pec_Ui$DI(|QidVf%>&&Tp7rQeyYF9bVvyn5Rz;r~s3O7Q>ckmq zrwl9y`{{tl-gJjjKtm+LX<(66@^tLPpW(L}-bzIec7m?#g=Q8r_yS_r{qw8+)xzff z1ETAxVGn%hog%`|W8+7ox7utHmWkqh%{aqwI5}(-3QGV!N)@*=WpZ`};6MsPATD;3 zq(v5E5G zM@-KGl0f9;C$g9C|tbun_jOr!7is-NaEI;_&xp8?~eIn_xj_@h`YD*H|a2T1Jd+(Qd92m z2%4uvqOibz>L$^R1Za=)Tq&Xoz6g|b9$u(DsOnEVNMJg!M&0!4g7cc7l%{b(>=2aL zSTl6|``Y;5@WV8wO=5?!KXqs`Vs_7Eyl;?2)m&`59eU;|sL>W$| z=rnyV6;7F(+D@@=g-+pv&A)w`n735}&abK9f(fFi7oO&mf=I~eWdDWnMj1+>S49qE(lrQ)!zek&uan0 zS*M?T*|!$4-{r!q`7FQe_Oh;NT1(iibKm{>sq=p5>{oEqQcJScqLSL?el7JM&y70V zw|R#Kf_n-K?%79QYs}VIB#@V2<{D>fJqdL4<^6ix$Pb*80+~!vsfS_z|vjU)2XR~j21Z6!e0-4JUQ;0Wj ztQ>@=Z@S2QWua?XLR_uvI$*aqt$nuQG&KzdIA8Z=0`A%CIlZxR6T~qNOu1f}!{Ulq zb|bc~16&r7bW&Vc-NR9Ulk%x0k4^1X5ibHK*_?Nvk)D#QY7~mWsPe8#6@d*j1@#~r zCfvc&(p0n?)1*dn`s2hE3OM*MXU_zAtMLG!Ut}Cu79-yk zR}}QFiV2b(R}Nel^`}pRea9E^ZSwg*L0irLUdJ1zGv0{%I~70lcQctuSr@i$8Ol_6v1O4iZ{yIvcdLE1DlQ?b$(M^QhoYTc$;v%>M!8jc!cSg3 zV?>2 z+q}^KD*(8dI=eXk|7iARY3W)5fdc^i!Ug~!_@@zL8%t9=7ky__CpS|k{eKz$530hI z#^0td8^XVuzFhhweW;nu9}9HDN~@Ad$HU=0bi&#g%`UAON`+!dj@|mb#FXn^jyCLq zVQA@!EJ&X2-K(dunG%Re3M5$9tmei82PzV})k7==%JDTmu2`*Pp?ov#W)&!e6vK&@ z{Y&oDpe++4mae4I=_~mzlM#~Wb#_78UgF})m4o^%1!hE?HO0E*W3e1T#YhFn+%Dd1C z_M|V->Skbm$x8(ks>qb!i6TToOgv+JBsWnNXUAE-x|*y&F_z)`J3C^P`S2=VM?%Fh zm;9_&HO;Tjj}|HZSiBYp!we_#=)6Oz9s*SCBG}`pb4fxlSFGVi@8uSw?q4ozGz|6fiu0m51zg)c4)ugM!ft;A!BWJ~7HD`q;ODH;FzkXDsOkHi;OO-sN=(W|1CT~w^iU9G>o zO?LwA!1X>lISK9VB}!8lcJgucUBFJ)C#z7OkAzVgz++UfxrDf0m%{{h`#!XHGOVe_?WAl7E z3!g0Un_AGEiICFkrwwZ_!-9tqZlrt^jAd1iPK{5CVGr@bDwmw5ABNVpw76!Cu!YrFm;ap~*0Q+O)LFCoDY|i0@AQbP5J3IynHS*)9aS zLwI@?Gkz32Vgld~ZN*d_)JX1buyKKpM-0efjW`d@>VQk>*Vm9O#1Ku$Z)5cG@KDrz+51um+u*h7dWn9;e!;I27n@; zA49wwC6YP9lb=d>oXawFeKy9!2wQBN`?7c*t2)$%pW(XFKJH2lkLSBWyvJ=H-G_&3 z%X(|7SJYaKf;p>KyPE@pdIhWl0PU=y9KxvmK*yP6^<%n@siW|)g`Gt7??E&cry_?9 z0pik8{*cN1eMX<;D8GYTE%ws5qUfxZ}5eRN4!Q7-x7;|KkzDfXf6eFJ~ zR>l2}&C!Rxv1#9CP7qk&szE*H-u*^6ymd`!Gi;A|2E<_INLl}?JuGfUTxXBAxjEM; z+g?N0OvmgWmV(H+0Uy(v3WN=ZG#OYGy9PrR+f!HJn$9;F)Df@Hf`>jB9q0vdf)XWGxjzvN5q!l=;m0S%A z72L!c-GB^#9^f$0gqx4JiY{yR-$wL5m{Rnjd8I&Kop4Q+MTM9h9}*X=r0JBFoT>ae za+9^n*_m zg=L|A(-}Xb25A_7Ldi$^lI(D0=2|h8%i&uA6Y^1=346JolEWKbe7hn1Y>p`DUUnDW zCqH3U<2|32jj*qqYesMLjy*!%k90t}-8MvLA)eaCEZ6iRQW`wJC6Q=Ax>PDsn7sn zWwhINv|P$AR>PYIY6u11v8L?`hTQv%dqT7E2MVflV>m(qWa{OV89e#p4fhBKdT1Ib zGj&TcRWw$4So`tk*6AU>tLkU0?r4uuIlqzz zn28kVWv$`Q(FIS0sWeBce6dXaO2>1(AuaN>iNQWorpU>Rf}CnJqx747YOgPbyh!^z(WK-iQQ;fMly9FJu z?JOwJ3UC=Aitrn}TGQbSUeWS{SjQAq-LSwqwz#@ahf+kXoZq@U>dl5s(chM8@ zN$)qrqLfB(!45~;cPtVP40ZI%-18x~G85`rhAz$sA#uSysACw5+L4HE9XCOzTsY*{ zEuO4kN;w|>%$K=1s|i=}4j#$%bO(e<6LyLd7qcZMmg^0x5V$_7FIlf~Ojw)lbu_?F z*{qfJ#v_0TzV|}GwLwtx^Jj=vM6Y>8n_}ffi4@N!R|)U>(KP6(*UD&i*VRZA+9U_e z=9sAn;EmF`d@vg>b=4eU^VU^3t`CfFn&nN!jy;GLt>UeB=JU$!?N?ZcI^-l^zs8=((xGaFSvSW>bX<@oxd|4}$VS)?^zWvdjDvx-M(| z-IFEntuV*=oBiRW=hPE#6B#NB(*m?BlVAe`fmU65#7@*0KC?9*q7j^?!7Aev&S?R4 z>Bo3J#UcU5n1(ilv3`=tvPA8- z9F)wJ(PW=&D5uGLg%}dAW&IRzJRk}n&Y^gXgU%nZ69*8>{Gi@>I|y50C=wk(Mr4i5 zyw#N~icU?tl`7QB^L_*L(*t$YE-VF2V8(7wU&vQVuBG50ec2su;~A9GiIQWS0}P%S zYC_c0oqSeTH2sz1S^I?sHoNy74_r*`X4SZL(n%(3 zbm*)Gw(&EwE~(vh#U?AUm&SGaakuS$>U_u%OUD8M%$Wq+6$t9hZL)5`6Iu9D$ZB0t zW<%JlDGd7Li<0y$HJR!7+e)@7z*%ic%4%?@QQtH2pX#b$YXR3GmMS{^~Redkr*^=N&kLX zxhkj*#c)chv;a_~5h%`T&G>O^!7P*jY$3&j;jmr*tyZ}SgR35k!V2sMZj{Hwh%g%c z8N0N2v-SerH!6Spej`%&A7SgG5Swd(EPD|0_2Zn4I_QA8k7h%JDfbitl`6e6f<^Pn zwLgx3#tBq*7&#usrJuRnt}v9m$;`>O$W*+>({P{H*|!J$U)Ky-2_8uqU4d518n}6+ zNNG*yU(;z9CHJsm#5Wy-yPHisB1(^Sub}c;OUoB#fmZcx8+f8PQ_e!C8U@I*ZG7T2 zGvg}X5emx^H>8j-CG|>gi%Q7kolG;%q2&sG>kvFSo9Dsg+28h=ii-p~ zU6eBQW1bEu+{~SS-X<`YYRk0GYB`OdK_OsOktN2fJ~E0E6g2hj)iC{z0E~0%%5*cY+oC)3N;yoACUC*h_d72po1T1a8B{6<62Q5v1HdA z1n4`qDP4#&&45l}c^w|pLMUQ6Q>raE;<_$7^VZd*Funo)nLiSU!?#m^Swr+M;$i(W zfBwxH|3eo40px7sWUc>lMd-~pst`GPsN^UeEr=L9#vc}o8!_GEA=B?ZSG9q&Uv&~( zYxzRxJ#~1#1chFdGAZ!!MBVq~H0toTYb?|yxt8U=RMu2jrav50xNQSaVBV+tcT1M3>tD zS*P1980-G&pG7Qr3fsT<@&x^V#}{W)m;VM|w$*l{vIG$PUg|LZL3`lnX_)HaV103(`0o=8L*b=e~;G03n`kS_<5zT2Sk(2&IgLl1q z`wPtDyylb2xT?F}Mf2JSszh)B8xMnqtQD~|QtPHCqX97%mye8*YH`;c)k+Uc)iIkh z`Mq1LH}ow|OcGJu#1MYGLTJLwWIO5R8%1usuW$6CFM7PTU$wiCn$f}C{z6&;8q%1+ zVW}OBkaZurM(hr4r=9ele94ueBb-MBv%1*X_(3$Y)r%> z<^UbK1Xrow+|VW_r$6&RoUft8x%(P3Ex+!tX zw&JvoEXyv)M}Epk>0zh>oJLKkhH`)_TA1Rflt`$G=|G8tmE-`;)Fh@5?@)*p%@d_x zyxXO4G|rGw>k!Y*&x2XxrMk^*v{25WLDNPQt>|EVFPTA@3jvV)J_-7Jv#DRIB9SkA>)-ST_}enWY~}+L{^ioHkaq~ zys)0vy4Q@!>p@e2tazlJd;h{CF9eA>hdx;Xuwt%1tUsLPx~L;)Q(2$J%-rlL6`qx+ zhl*>3j1#==pSu_2hfKs(4O6eckyn(;4h!Z>OJ{c7Ep6ad2r*R92()J;ESL~CPUhsG zCB52nuc5RH^*<|mz)dU_641DP8>IUg8|51GslaC|7CXK1Puzv6A}^Z*W5Y&YJ=}M( zC_4e#k8@oO^5WJg|KM3GTfB1kfZj)T!InoC%VJU7`njX+2*iICGWa@#7A4KOwJTt4 z-U}jS>08Q-R!LV}xyJn=sKG@p$;uXd&;5!_>*w4^TE?OKls@n~^?7`HyZ15X9fkWS z^XVB~T@CkWp-a?@Bioka)7_gNRR5C4LYcF8|EY3^$lHhgXB-t_Q^-91jUyEh006vy z#*vYyi>bb|i<70D`F{h3Au5xxSqv!K52$V9V3m+ob2OC_>&>#%uJdYErCExmoDPs8 zBc;!z$b1#M^Z1vnAVLeH1ZdAsoTGfBahx=IbOfxL6))5tYrplopHm3hN`VOPL$RWS z1T>dNkj8SDD+y72`5|VW#L`60EY6fQ4dk#qyxbH&&uTX|4t?E4y67Wlt%#7ZyzMoKrvq zsz{#h$x6Sb(1bOZwm8(Ie(Y$UGibREV4m#7<6X=ZOw113iCvGdR{!w2V#Mrm+C(h? z{+Ix?92N&ajh~~ANj-_rp04hA&yfP~IW&))b=`<-+s2K{ddo%|9iEo~Q;-Y3TkJb`l? zY`K_Ww=XRnG9N!w1eWVFfUJMFuQ9yc!9jj)#5a-kn^$s+HiIKcDr3DQ)uUmY5}LcO zF0Tn%UW;0Lc38Rl)VJ(2O9eL6&NE2zJ5bUN>Cibem5m>(LMUV-YAeA0u}NgMKZ%m( z4S#DbdKq8e*IbPLdUJL!Y4ihex8*JI^b^HG^DWF^8W2ixTnU=-vdy)d^!J4257De-lY1YTAD_U=+XAI)bIJv2-#P!8RaJ78=*47Qq#dh$ce| zsKC-#8!nWJ1Qn#858R6biPT1ybiJv`ea>C?TDU=mbZi4Srb5&=P_-ixGV;2(Uoe4r zn#XJ#-NbpIy~Mxlf`bS|-nPyH6*ARpnFJ^iMJr;6Q;>Ab1xP0_$!ulTCag3$^J7Hj zx@oA+MJ!pstwXfjXVIr6=j?1-sKUt0cRB0r=nYNVmgHyMT3MB^cWjbqyUbl;{@|_E zGNXWk^g4IHwRD7!XG?Y%eO_Zt@uq_^R|Ak39%?NG7GWO2I4bW*DUEO@$8st9*Xqb< z>B`%i^J3t&;`$eSTZCKOt!6oU`Mr-)>SojNj|?^9ly9rhEy6>!A?0U-_Os_l=#h_- z^XJ?62M%$W;e?2*B_ow5rEX-Jv3Re}ZF+Ovy%*9yRs0CWaYReL^3va(0_-JeOgFF^ zBsG~i`uaL*r7Ag`1gq_@p6)lVaPyE#cA11LxWUr-#tu|_#xK1nHdTEs)xsJ?QYjT$ z(q*(9u}sFd>)8f$tz?fTpHCt=uK>r;jPZ~Wk!`Bwmr2CmjIou=WBc+&ECMQBj-`H9|rOe)dy?n-z>gK(FU%4cu)drm%_e=U;PKwru(nb_i@S zYO2Qe5#Lr`BZgDIALVR}x;Xb*|8E%k~I%k(}BhrC_#S{)w+V|w( zwe#O^4lf_yJNbDECOm)L{Q<@0>n7Tr2g)+4viEaqDONY{-Sz3@`nX+i{ta0DC#H1< zSqxTMxnDZ2*bLJaR(g9WH-_Wkws+2U6zLpUxuAWqMJ?c6tr?YE3`&Ycgv0D6qM3wY!^Kfp%Y;684fS}{0PNE=4dCfZ z08|gC2#sOHN*eICE(r?+mw`>+ObveCknXpWEf=2}(NI?WjrZ_xeG5 z*^)cdXH#}5NpW*usD$MjhZle87p2z${=d-a|Ma^4bte(k{}uFKHx&Oh(Edqv`udi3 zmM;4G|DhlvCQ$HXks?s}+r}bL_$#^@nwvuX69Gf)NV&)`000KO0RZs-`Hz2Z%gha3 zOx+DV{~Hv%;yriT6kGUtLNPuQ7YGxQP-w(mE~`^Zpw!mWrJj&$r9p=x2}7|#$amB* z+p)+q*04P^^p$s)pBgBb$s=Dlgu}*iLi*;(A<@GaMR1kmDO2r&L4IYv?r8i*kw>%J zo3R`!D0MlY)zJ^^KKf-B&VP#hX1PzciesGc9?=#(02yP<5IDqhDo^5r;e5uC?Nb;5 zvcc~XD;Jig%f!X~+Qr=2jd7jl0vbIOy?CcomNGZ&bn+%tcZ4;?qspP*2##6b7Vn?H zU102fF9}3POfP}+&Z-OrlS9Ii`m9si%CD8=(BQ%xFbxaDpMX7OndWilk)psJxrO@O zpd(lW2JC}Iu9#**e?@mUXs~12{v%;2#SImdf!_Z6g*(ROP5e3 z8yeNMn?P>MKq$wdG=bBd7`=QELg%sX$(Q9hTuk4h`1f13SD)EI7~|UQ@vXzp)m@3R zpPS>jWdV#bmvetjb_BsedlIBB862BgdLV+*H3LNZXs^_^MP(+c4;dsOkqdTCUF$AP z5d}!qp7@gV+e0!LyWyiGPk=&S`Q&E0V!q%$Wx?H$t`d1ZJ3;~ASu$9O_#F_@_C5+X zHa2$A{&3em=wjv{L~nBj3>zId@w5wM@59f$P5bPJsQvwU`tAFzq zf13h$^G@!ZySim8&BgQ{4RPRH{*piv+Hn}=^XV)-vI&MK${a~SCc|s|z zWY~+iPIW(dB*9QNH|}~iL^GZ#6xu!7tU&nC6EMQmIkc8Zv!7-M;xUDEf&g)6V(U|G zx(_jsxJ>qT)wVk}$?W9iCJ8r7rw2^?uWThK@8NS;_`l3mxhqP zwn5`0pyS+6+h_ZHGfIp3gkmZ?P|*hVE$r)f;f0LuS(WqyDr4*Te*TZPz4A1{li6_| zW2NcSzD(oJYIU;*?X6t|nnYPLeWGUb#&mV(I2F`EqGhK>H3YrU*eHt}Lps_uIaCdX zqLc9W5NxSQUW$Lf1X^_?jp>9H0IR@Lg1-<9w0+q3VQ()h2fwu zlkA!Cm^2;aGa`pJ6rnO+(bND4P~qv2K=Kq7$?2?6lWYazg zSl$bYEGpk22T}G?RrrnP$M;04;MVDHrE-MWSPa-cv<-`gY$$Y|C_#`>TIBQmI{W2? zsy+J)m=Bn6QZ)s;)v0o2Wl{P8LZb=65J4;(-7!m4_SeU`pbfa~4;M4X)imys-ZB-a z#nxHfPWbb?#1%V-9O=NYqJ7G{!kM+!exejH-g;99Qgl3P*$iBrS&kHBd|@M6XYeMi zG(`vloa!$^bpn-74-uG628+9~#`GK3$QI*A(#dKKhpZCX7;{SaGC~lc;e0}U`GrHH zkqo5Ph_!vcK~P-{9*;?5jYpR9Tt!X#)9GUS(Y!~fpkPT^5XKT&NAMH_Rlp5}i+LB? zz&GEabNKwv;9U$bup)f`cIw6Fq@DXc(*hkLeWdI2p^)fS1?CK(c+ln^l??&!(gF+H z!bz`QiOx#W~lC#sTWDq?N{-VB# zdX$+ex1-oPFc}MN7WLce)rf}YFRQi3s;jsOQy*7-w8nTd7_Fi%G7_mSYd@HzBFd3F zg+Hjw5Nw?bhXVg|##@VY?LEAtppaktsEG9tNayP=WLhUkW|JEltsad&Ze1xgfd*&W z_#Rb|*%=MR{Adn~8404;i3Yk#!`}gwSVaK-8)}cA;rEP06ix|ptHZe0q~riF;WAW>}*mp`wzje=<}iB1U8fn4TTSm4yc49q0nqC zT9Uhb};ab~c^cn74@f_fnp?n|TsFhA!Aj9FB^*WB+y?e3i1t1sAm{K>+I4 zbrT-y7igk|I@GuKO#E;6w#S_kkwW>4lJ0mRpu`w?ZsYmkMG5k}<4uS(L1;Ck~ z&jW*z+2S*?I87|$oj_bqc#N9lt*xcYv361?9QdRx*3E4bC#@}LU*lwb8WD-HKkE^R z1_#ojQKC^sP)7J!s>~d6NrGLzM6Tp}1v{#$GBp{}2<1?iQzLtO-_NLfnfE;vXk3OU>q@i4qa@`;927WCLY@CMdf7M4vuBo)*yj~mexVmenYR4TH7LsrX1hu7GL?$gN>`p0O zhafWE>=zLD%Q+FQ8s>}3uOpoieF`};qtv=G?GcbvucJySpou&>Ux;4pRd=(fFtAd8 zc?8ap=##(joz^*t3OeE?S`NIR9S1f$blE^^s)=`8F55;ZN{=1{$~HBwYj+@#V7~-+ ztPUo7wIrPsUf~xnRo#DHWHgj|-Nu^Y!cjjLc_wK6Ka9O&bZFnU?V0R2*|BZgwr$(C zZQHhO+qP}&7&}(wRJD8G`=7d1?|hiwS6g$nHRl+;_uqKeX-=9;_Lp>@8t4=?E?Qu# z(%06sGOz`|gr%2#lt%OTy`sN1aeKsmYM**si-4G-4vFG5*-DnOVW`!FNAtax&~e<^ zzlA~EXZ9t@k-4wY6=34uy8pY<9$6bi6(YCM5I1@v4erwyy(4lcA&!wDvay{xal==j zTv)5|#CNF9rwV*ClqY3ehg>Se(0+6QwOt;XZObZ>HR!oIBk>* zsNPdqXOJszUXe{`oRb3X$OznYb(U1NBJ-G7Tf`}#Gud{YPuP(+q#4Tnt#>dK#p+?6c!_XWO8M1G?884_iRh;YN(YqN1gt8nepBur_&LE8Wpbo0?AY?>oAF3q(PY>7RLRD(0AecgNIcPlc&9EZf> zIP+qbDW_|6s5Ci&&hHl%$AD*^V(S%$nxgUL$}c*w1|Z7*w_gfPl9_R-<-3Tm{oI3c7S%3H`kus zZ@mbNnnt_SX|vkxm$#v15*wEBeO?*NhE9*7AtiCw;5_Xf#8gJU_^9)!S%mPMNXRA0 zL?P+Rq!*gM$ZI}eX9CR{+A5qUOIgO4qu97y-%BC*tjaUDevD*A1?#}AYbO#GOq=s)`PACPSiCu2@s4>|u@19- z1Y3EUX(Yfu{$%mhUAlvNK0VS`RZ7COgfW>`KQ~T`@wVtcV=nnt6$N@*#z9!m_a17RaGi#_R7POTygRAt}8!XLePwJb(sCK?O2aY#@7^ z*wn-FdBoxHy^900YtaPnXt#Fm8b_7rY2DFdY&B=7%%mXOMA}s2rG9<7uQP}-P0RxM zir8E4Ov6suU&cWvgDZB~yU?{Cu(4!&eeO+@Ge*_B!aq@vAjr+phH(`qIAI$S##&D~ zup_ZW-BWcj0yrpvaUM8anJ`_sKJ3RvJ-xg)ZSi`#@jaGZ_XhUn5d^;mqrRJY#78*r z5e==U)VnO90i__pTQDw0JY85L#BZw(*p%MW2ku$$-F*i@%VP7<0kZAgo!MJIr5X^q z1m1W?(Q`+|X`8o5l1ZZ*E?V|ja7vuEjzy;@b6Ka?05~x<=7&c(2YRKQs*b~9wH;*o z<|e`E`Fq?YxfhIbeFR&EtBERCak&4gs{+Bj2Md)*Ke_a5AJ{u7Gv!eIB0lsxEIVJB zsm>rDoM7K4D1w`5843W>7m>NjCbBt^n$Q`p}8LBp=qCS_C_9tzXfa zKdLNFC5-biMTe_@8v%(S(s)UnB6W^U@pA>9ghdCD!T65cp95K#Ame84(QC}C^{EcB zJjq*b!kcnkhm7b>63M5)->Et@FM-+CyoG2T)eDEO*{d%B+em}bTin6FT!Kmgo#W#< zV=N8|>TE;dbB6n54P@vt0Ox_5hWM~z#PQ=)ICYJa^H%m7-P39_I$K&6#qks$xb!)g z7=On&M*FgK1;*&Y?tQwA%o?br8}wb6@F@>IbG;PMvlAT?deg6aF)Vvs^KI1HT6@6K z`V_&WF}f9rFjX!Sx=Wdsy0IFC>>f+sJI^YC-ep#ttOkb>+LdemX`?}%otR+mMxTgA zAfNQ$m(?Brg~}_=F8Ref)T&OpVR0FfoeObRbv)I&!4~aqlkpE@M+9s0G0df0QE+lE zXe*EyeG89NWzHzz4|2j37%|~vGicn{InM;V)5pZx$O-D$@^9CXnw0_<5Mfqa2)e7{ z$nrqIP7EP*C_Go#qz5Z5?T5}*ChZSNhf3-UH%xn}VL*zpbyp%)3ypWpCTb`&vprbL z%c_FX05Kc&iPQZgLAhHl{zw@PRosKUg`f()?KITXwXjiK;e) zrdyEu+GxWU#1TZE8xOzJK1=kA+qt#`Owo3CpwSN#Heh-^rtc(Ne|&`Jd>+nn<%o%z z6xwN0r;e-yC$BNT7c)_w9Fb^1&DkaaipQBdw|-mK=#;`ejJm-iafK*xxE9juxDcio zmA4#L;^8;~7A@Hg2+ZCu6q_(q$t^CgD(3c*@t^ z_!To40HHfc*-Y2lZ#%7m3#!N&Qq~I`_u*@v8Eq2M!%e4gDKo`@bZuIr6k}~qd*L!o zJ7{U(hX?(@3H1Kjh^j3!&cV*%ymN)yi|v9l=H-Q0RHvIpg3|dZko2o~+x(K@fUreM zU7E-x{UFM=8yONC_*Y?5 znxGkg3mfcRiuNbUM?u<+%#Qot5;845+gp2ij;<+~sY@?@B*XfESqT~h)*#3&=T$ou zcJRWyW-ot}dEXl`ZAL)X9k6=8`)W{P)76$V-frRqRE^5B4l>^dMMeTBAN zGis0{G7k{S*9^3aKTEd7$q=3eS_)w5nHN4S^ zikUkcGT6B|8Qox8ObTGf*HQdXAExEu=E7h_R_zxD(i>L@yRy*;6xB&7KP1PN68>E5 z-`%BaA8qEC)>J#+%bxCCIytrbdX#Tr-DbQ^-ZYyCZx6Y-S%WFAmh(0F^L%`%LIdj7 z1&Yu=y}|f!hPIWupASp!YKIHjH23?1=!syzuliKKxP{$r1)qd#RAuZ)BTo0rRTH1K zno{Lp#8w#@Gu=<*Pha1sU^wVWBa?h|JCz4A58)GJea+Z6ly!>H(+syb5zj@max>D2 zdVR;uiFCiF?Wz{&BO4jCxy3dLRM1cilYTnClsh#u&UJKRHWULgNz&--I}{xx0e}54 zMW8PDyJa8-GVkz}L%G%ob9VNjAYF)(>fH=mUZEax4%xu~#OE6=Fm7f`Cs*HDi#8HE`Cj)$Mt0e4 zs(b%n<*Bc918zw;1Cs3lPxL-{%k(HYYISW8TjhIH)4Tot>TZe(^vwK$JxR;p$JAXE z-%Ah==XWA0B|LLJ1tWu7znSyV$23PAk&1Q#S#L{2W$_ms+>hi#o93@6hA?vM=0z<8 zt+Ru{Z6DM6`&18*oHyj!8t z*7^K9(oL?unasJ53^ZtoR|&Pm%58Ix*(KV#`C7bW7Q!<~`gZ+ie^IqpV6RZ^AX{&3 z$S-Eg4pAXcPP~EO3bYUp!07chh~36R&UKUNPo_0y=k=GT_59cuiwcX z!_l1;kKrXY(o@zS=5K(F9C-teLs~llkZ^q*`dALhcDT_26pY|K@eI?AdjaD2OFsb; z6lO6*hZl1Obu}d9fO-t`M5Og>V`>ZzD&vREdj#!0=%Mfh?fsQ9-;#~diY0!Il0W;w zl*~lA_8w8iBEVlEOsQmev%)ov+!%;-BTooX`!~%Yo2~|L(m7+@+=2ai+7Vc{W6&G8 z?Y43_7Y5}rwemBpvDP1XAe}RWgz%2ODAfTyVvBbrn^f_dzL9yl4V4(Ne1mi51z{(# zP@*SVYk`NnFiUp~Em;$DRW@_XTgxKGkJ3l#1y)W1@5r^a$aRLGGPW{P8iRnn>e*$A z$4q=9-bbXyp>+&^-n>gskE(zRxTpzk zYy97xhSL2$MxmIsB*i{B!h*OQyRFUCnn7a<=k4bN>>feE=Lz@l1iJDJAoi)d^HnPy9ih_bKe zQEGvXbR1!I$iubIWd@YUsv_KEKj_!$eDPS~3F8=sm~erI82bf?^6$7%Y|Pw=wwf=V zYP8b257Y{HnCxk!37R5Z+mw9X5woRwBiS(^VTwegGGY!=OqNPXIK52=LBb;722l{< z0E#5=Xo3;}0zyNTy>O}0^p>Ojn89igCvbLIydI(=mY!Ov+7e2(^USHb5QF~c3Mc)- z9VW?gAyfkO0EsJiywN6}WR^*U5CM%)JbY@BMh0vxSo|p*V2kL#Ur**F$_w&~-B&0(&DTh~{95AP1L71o~r!0&>=aT@IIk z+dgEPkKENK8F}|7xlDxLhG5oO8!wS$jZd}9vPpXlbY?Dlda91+ne6Q?pm{*;G12l@ z>Kb5HM|W0GJy0OqS?~^&f8u7sDwY-Bofy$!usCTbYefgLL<6ZFdS{Njv*<`mX&zl2 z1iI7%?Z}9Edm*XV0Uo_pt{vMoA7%G3fu$Z;^lJ%D2=)r{Zfv^t?&$ooF73HrR9hO* zNEy~{eV3TrWbkt9mMBJ3_@vI@KH4$CoUS^H*wnh|IB6ZKn=C?)B0fchNAt_9IJCYq zx49y5*(P$`zp?VGeQL%u3H?4->so%FK@HY<=(@Sxwd&&J?&;d_r0sa3Z7}oL#JrPi zbO0xpkgaCG4lgCb6;5-XrcacB%cD=SgG=it0`D3$$ivomN+f>GRNn>O5=5f4uT={s zjXEDD3<8rBCy>dFYv)};C5W|9E)00vJvjZ`59Kvnf>mT4Wp2TD+ki=XVQ5y}56JNthd2L3zW^PgmAHU2w0+o5YDh4|4> z!^aBbh}9~+$|Q1WjlJQQN4%j=5RM_t{221}j9reJm~gbR@bx`>~Gmm7<9*x zn4X75h(fF-mNYzd3OJ0Ps1!%%46k0yHAOVsE<}opZk%rvzQ-f2h+;dj|Ik&e*j2J# z5jQS6?&yb!b}zXt=LC7MhHLmGv+utcVZ-IS~-ysx%dj2M;2kpc*S( zJ_M+ZxCihaOhnQnix^HENj7UV3y_ajih@*?Va4^J+w1PHE9H~3_iFUIB}}fan3ZG7 zu%}Aj6_P|kqi{$;%oA&PE;(|jQ@Z#HE+?~#`5Tv9A5N$ocoo1yJSm7SS*Lu_GoiMQ zA*|IH1s*9xTk}x? zKoc;}Ga>pm==jCiXJJ2G2XjRsT9zgBwQgSr41zdoXgl9zH zX($JkP;j&a9|%7 zak|`fZ%|@|ql5c}Ux)L0a*qmrPe-8Y+u5|j+2Ejf!Wu$`>A&i2Yr+Y(8zaI@v%p%9 z`_+8?qw;I#hClr?Ov;bu`%$7ao>=OS6)p3BZ9?`K5&*AbfETZA7}Wp-CM*R6`zFn1 zK#5`HA=ax*{^t|H zNu0aw&6n7(P+m~n4xBnn_%&C6Lf6r~6x#JLiwWn)dd)0#YaVTTfn!VUyV zh$?C=6?4NzCPylgiUyKdV>^Xq@`DkI%?6{A5d)(BWp|-IaFcea}Fbd@el0VsGmYN;K^gjJnkOqpLy$9IXc-F zZ(C-vHuuxC6p~B=2;60eG#b-_E?tv#fwe?yl*c|LBm4Hd8QHd4>*lplOvH7&BL5a= zq6!ckQMZ&A#kbyJM~0;)bpauux#dnMDClSt6HeU(>#qzPnTO1b|oS3Rk$_#0BzL0@61 z)}(B8Q^bb7G#Ka*(9P;MavREV#UBr9N}+r#2jI+fiBMk+X^yBU7~ZRynQ!jq>2}q) z#I91Ip5u5XwIp#QLtbQ7++B(v823sm^Ydn3+rC*#ol72klrZAX1YmxX`lW{~Pv`!+ zFRpB~w27g2F4u*ypy0z=U32v4_@Qf@8KBtKVYo)iYv6~W@YdU^O8M1{7y~cTzWKG- z0G8=->3uO35zQpX)~|+qPoJBYc8nWXqv#Bc-@B9?v(Z)gNT^9DX2zbsIMlkXaoy6E z(-HPwAriw~fRjCh7pVa!9eE%(6T|Z2szqLGAK&D}nb7UH)LLmN|AB&X&fN_2BGEdI zZLa(Snr!`Alm8|CZ54aj+H$!X*ocx!`E)hZbJ70!Q}9FW47CZfo9W>317|IT)6lq) z%w*TvPK?7wjaP~wJM#$t#`q*PkTb-kD)JkAP;qZNUirC&>J{{C%A%MYY}ft4mj-@Z z{{DEGTA3bQo#L=5`p`{E)=AiHh4S|NqTO+h)<{Tz{te9zG4j@T_4)97>paRD^qnpC z<>KuIea^d$Op#Z5M9(;C+16S+ig3UnsHNrx12^GM00yACJy0EijfC&CJ8>|bxvi7rU?WM3%N z5=6v0-Mbr)Oe)hs1U0F48VM(P$}M_B_=iXyDIODOi(f`r_=bP+>jDQ`>#-KxebSOu z^(14a&E7Fiu0|{3s10j9FUwpsM&fGI;wkn|+2!%jQu7e$A zO89XOvUD0?y~G~I3bR_ZP&lJp!Etp2Q-_<+)TJMn-IEVivd6;k+U}C&w=;paPAB5* zha6=D#PBFb)8t-ws>ZKF4MuAvP!wo&h&)6AF_QLXr-+Tt+C(2LETYwDp3q-^z?=Hj ztoTv5J7`*9Ze`_)IPJ$xR(gx83+*XMGyj&<1Q=x2$%Q~tRPd4FxRiVpMwe-AO=r^J zOo9qi$j9&0Rs}LlTNvg!D9R5AiIH1n5pjlN0N*i!1}6R%@PB{3LB_Sh&-OWUZL2=? z%Y3$YjdCLk{#-~Z5LovgD$ib}d48Mm@{DLnF>KP||=KhWs^MW(dQ~QNl!i!corMvzyKe_2e@Bu3@EcvZ#$5jGvNQ{bX zC?9A2mW3$7oKa zg+3BqXdr3Vxp+eS_OGusY;P!2P%`*_k54jEL@IAHCs`HKi+{$szrEFoC^%-5xq$|m z((@M!*crfr)`gMVd++`0(R-Lx4W6B6DYXu1uy*MI14c!@o=2y3LRAVELbg#9#E?Wf zpGAciMF@&Cx(={^5jDgfSa*B&2$K`VKSF7w_~+W|YI>YxSjtkezV?n9#+GfKv#YX; z=eEODco`rWH8t%3+72}RmkhbSk<0<4TRZ1BULR@_0!N&%dpRF)Z`Fz=Sw#iP7S2I# zl)bUQ+JQ8NVAvwl5%)Cyq2cT;!hltBBS^cX8$GDld$*1)prjK}RAmyqtIU1SBWVLQ zq}B|ucCw&rA&&+^1OXCp*ufE~#0|Qw5zZl%V6B75PTUa++513wdva9{S$98r5_KdE zquzIs4d`7%RnlMgdCB=D9+*~L*BTPN3X7X-@1ar!%5B-9CVFO?SSo}WsOv>qrcj>M zl7k6=#a69mP;KRffimb}%|tSJcBtiTjYV`tqeV}@YB)jkveD9W#4T;42{P(Z&K!*J zEMb^+*|W1fe8JMDK^3OI@(*#?ci~76qr`U4;IKW%T>e_ci9OzMAC!hnxr_pjcV4)Y z8-1&-nCPE6GX-m&!sI*RUlgUIIOC)PIyY!zGg0*$c&S1|h;gg04iHP7g>25Xt^|HU zh{7AdhxWQo8Ya{opPw%5qCN%7rU@S@1rSbZ)+&W2!HJ3@zmR9=H~50B>l%< z1TN06HHs13Q3;l|&=6Dy-hqf}h%0t~?X@?8g;`?D+2!{EF$IEVn|yxHmI0EHO>71( z*~=wxE@45zVC?O=-KsZUFs?u2d?S5^pX$ytPub@H8dQ=Ae!vp8|B5e0vq{=AdV*nf z>s%Gl+O3=>;V7f6q2#m70e&WhB#OGv#F7gYC9J<#T?k6Ae}R#Vy6gCQf*2-MV2oypQDgCO@u3F&&VJU}d)A}l(G_pbV{MT_D* zCzQrZ`3Ma7x&0H$!Q$}xF9XG}@bkjLi+L3??ey8$Tz^`PWcjM9CLMOeU02XC_x@g! zjRoSsOwRff>v^kli}2G$l(jH)>1=pR=@AwK5yjAm=uIX@3{JvR*_kVQ6FO(%OZ{?T z*HA5$(U*auyPRzV(fSK7op>&tJmek_A`88IF6u!S7;?ey12!=CD0Xv}LadpwLl(~J zM{)DdEyr@w*c;iGUS@Brlk0{}TCygGCJ!gRMtJY&L?QpRTmp+vZafUyBUPbcN^|SC2DM=ME5k+FDtp~^Tv0kc8sfR zQATg>JH5%FHmCKK3i;J@q(hewhL%Zn8cC2U5TceYiYkg!K5s_?6sb_!?3f;Aa|{ae zV*M>XwBH~HR%F!tjfjcGm75W$#J>*S0u9nU@YJT6|C-7DUJtOzusmzr&qM15v8h|V ztzeoN+mVUX^=9hPg=>KwJF;&O?w1rlzm~l)-Z$0%asJ#L4Ce-`%Mq@4?0S0Xgfc?y zIvM0EU$4RjR_bV8BS)X&WoHLqF;sN8_b?qLtgomgOU;F-2P&?`Y0|k`wqzUl{xEpl zR4T1)0>1ZxNvE%F-i-h^Z#>Z|pL%^!F`U}onMa4`T7iQt+#+^p<=HK}aJKiOcE@eU zR}Z*JTYkMpa!nL~{EpbgzbUdY2=o#?TP3wn6dwVg1vNX1q^2_PD*avN z?P$(D%NVMh*xWxi&wqG9*6;MJdiWValVe2HtG#hhXJxg~;I8Ju_0XoWxk+(}RyYc%v*8rC%IK&)d$CZMJYgzHhLB70O`kC8Q7IG&fIWoh| zom8WMPU=>q%WkaLWxy`5kmz^zuO{Dg(Itnzf8(mN1h;Cp|4hc3h5w)P+T~YMXR=#fSjDq;`F1F>K#I5wLLg#k3*+F&`&+_q`vkzC5F6YK zupxGyuNhY_8Wy8~RPe(p`DlPBcxp=;Bnyg+TO_(dNML=3n6c!eDtW(bVK^fb({A<- z9j!#-p8sCXRnq+j~F#{dc$*XU*Hyq`J#EC-6?IZxS#0(kvz z(+%ZuIl@3%4b_Nfgj*cq=y+%VmXPd&$Sr1BlObO5>=60e`rG?C210Xlz52xyk`Rh% zM<@q+4{Jv}hqyhe2C1snyw#y(X_RvWbH0U^gs!#R;eoS9Y_w`(qN%}rAQFSo8$_v+ zp3PvUaBcNxMukB|W5;8)^rl%OIDGhhr|@cehIP+e=p)FAHC5vn<9rprs{>}j34P!u znE)xNSwVCOc_-U(Vmcys0oKSk>wMo1khyYB^J~e;5EK)RDknp*asMhpYSBXR2+LX% zU{RWpM;KUMkGdn{=lR zVJ7X;(yRgt>i*0kr0e5b3}i<@)WvE4x12yP)V?XIfC0-5`!vep$(x5V%_V0INOI(S z^_*K!6RJL@L_Ti1&7S_1jj%p9Bd^AUp<1UIJ#EQuS!C?#w>Ib?YV|hjYAm#l z%RCiTrm-dmr&YL&5a7W2NcB~BRm*5YX3mI>w`=e4{xKuLE@~b{nV7a)=q#Gf^Jaej z;i_GxOI31_{=0X-lu|17g*>4%5ruWV>@U!zrvy+InYv>*ZlML77n&H0>E{{yS2*#) zh+MSsdwCUmq~9bB=q>Ep>;75=NhavvWk?cjUxIlGU%(hE&kfYKQLF;UH3PAr&qH>y zTUYt%6Wb|=c6^o{oGF7(XFa7Y<6c2XE%;hI0Wu$rXIQcC?Ih#SWKZDDW=O`deeZz(EYI@y;My*KGS+85eO~H+sq&i zWERP>bf8H->}1|(`O#kjLFOw!p%^|ohPF%7piE^$qfkwXB}_N0<&kxG`hL*P@`Deq zTyDk+!_v(G_W-CdB(PNSU)p;Y6VasCXB=&xSq!PSlVU!HmD@Y;We6(gZw-mk1Bn%G zTaFhXnGLq=@S7mNf1bz=K^05Z-6>H4JjCV5$_?%jB`5bG6hr%qDnGc4-r;;y@8b%d z#Y>JB443qGNsHxfBA%Ql5A^hV)j}_e66Q87Cpb%`GiE)nosE;ne#xYZP=Kn4gt({p z8{tI+;9jDxQzu3!=jBHwgzKxuyIbYw19>dRXut~S=PYY9E1cA=lvm!=(^7mCdJ~LU z=GDk^;^!WJD;VEe#Uj1b(7evPPCTao`<5~@<&Lkhp#$QoNI9`gSZe~z4MsO;E@}I6 zkrVUn+}$Kn6i3&qIk3`i0NRuf(GmftyV1B6Nv9e3n1>Ytxfn!pnP;?Du2`g6ei8~(w>+(zHZ z+~YsPZ7!?s{uKN_1QT8SE!0d9#LR&_IJNPWP=406j#Bv`t6a^}hGdC*f#=m5r0=AQ zX7Ns%YFq?y*N*pV&h2p*?NxONJq!f`35OXXcV5DIt}M^< zjH>yuIGXjmUArI~x{SEKp&OEQwFqzja!&45Ay1F#rP=8vxNIgJvXN`Z(j)v{H56As zMmE*Rv&nfQh3Pk?eo*ja+nxdC0E`*$1c8{B8c0_-YC`oO(BUt@#2kStW*-qLFZDB> z?r@)Svb5ko=`JpQ^&AQVWM>Q;S-D4wKK_#g^>LL#dIqVM@dWIiUEw~L&OB}`V{HSoUgqX@3W8BItilZUjix2ORKAx z0}X#FH-zVEP+$b2*B+tJ*aggPG89HH^k53rflYQbf(?G?5dc+t#qKY!#sC7Ohru}_ zT2DTz%xOcs6I=ywX#H#*)CG}gcA3=x*U$Wjy&@nGBSvncUW@q1F)udi0)KOUh`$-= z%})}3k_IA%5fRhw0J2!pe!CS{LtWA%65 z%3TYISLQfzfI!I+>lmLvI^SO+iHkj|KG7p6qCBKTSE7(0Wyr8Gu?04!=%S~x0K797 zZDO^SEJI$jPV@vUnfNYfFsS;E`W@1cEBI~p?)}}Fkuv~$vhH%3&1djlR=Zm7to?U* zE>z(*(ukTZZLjphfh*v(5pI-gbvn;crd{}Jr5;&!-wX0u+o)6jwd8?Q(=Cd={@^#r znt8KSPbQBS0`I>Jo&||2fA&^4F03;`t&(jE_O^!HbRjN64se zEdv!=O?oUtOpwvP|4pBBji%gS{GpKX{|qxl|L4Ek(b&qw(cIL=*oel?*!ur(l4*C! zI!vWl!oMIZyNkpwbj9q{#F6XI(=Rc@ety=CEiiB?P=Wk0|UBQCkuc|~CtugkXzL9_^R zdKPia^4yjJ`~n@O3Cd{lfR{dKs4O7b*Bf#b#cf_a83eoQ(m_`Bj&rKA2s{mnia}Wk z8}SzK@(??1ecx2d8i~_UWei<+q16?^f2nnr%!(!F21mnMefGc-5JPT5aBo!seF_*y zK4-ylO5T9zvFF@4d9Ko)WL9&!4{S@eN$?vrT^kf8`N}FI22H%K-2a<@Lzyz|x+-%^ z@AvUC5i0__wt^I8?uPi+g3LO8MTzL9duq6c0W5w1w7TS=c|Hak`HjH=Im}Obec4|^ zN*zIkXQD>o`CdlZSdHPfG^6yq@xdu?E-LAf^)_;{Uj}_8RP1TU(M_G zrDnw|Hbe{V3}ZE|^$58Se#4seSn;&RC(>?MR!H0q?F0O5B$v!(*wuG=pI zXIJxRn6YqtGkZJKteW@TK$ekP_BJ(u`?;aAf+eK`+V7(bNLb|J6=CA{Yb%W zL8tS`UUofG{;lXw)O!;Ip9*E%O7yMQUk;VsSwny#>S*&_kF0SmJ1$dxybEZqZM1*{%cH zxoKnOe96LQLhv?3^DD{3IYVh@@UFwT-{Ao<5Xde*Z#Jt+kHS4i2!XJFywq}|CJN0J z5VYcavR`Q^G!kNp?3l$^U8qXnn=NjBTLtd-8V=czrY9La?W!xBH(5uZ902|RC;WY( zz`KisQ;JU)AE`|tL^dD^-T7#?63qi)EhL_@8jyr2-!d#El3qF1Fgy>v50@mpX?>ko z@aUy+OhG(~9XY~QKct!BoN>S7CMK2=>O)~y?|*@o*yC_gczjZujO`p@HvaZfB}mgx zc0(^?5=jKZ1~wD0?}D3wlAWhWNqE$>TE6?aN{Q%rC;>;}JL?cA7&|MT<23lfiN)+y zT>}D@BBlZ^3A=b@Q$GVhZ%a?9=vD?J`T;t;5ISsmi43_Q{ zmE7(fIBV0SzliURx84X#7~0A&tNuArj+y25OIwO_d>CnHAOPH0!R0{dZ|J#TY)1KrD*v46)RToB#c2EH7e;8RR#& zw*B`vx{JobTlzFF+?VJavv#j@*y*?Nxv%<4>^`355agHF!%1vzw(2L7V6C}E&f#}? zxGn-aZAXuT;!CaR*WZYIGJ%r^F0)tU-0kXN#8@5jyp{%K3#r9R#(rr>(kxa$-lj6l~x2V zBLo@oS8zJu=Tp(sl;O3vZ$K+3^rq@rPJ=NyTP?ygYUuUzWZXD6E;L`=3D<6>Zdu zB`)R?Q~90-Zbi3qq-1~N_Z&4D*-9$ngcj!;A)mw_=iRErbvL>~e+CNVTu;xTGZuJ zsrg_1(|jJDxOEv9n7W={xSN;wZ0FsKwV$+?dB!zM(k%|{ytpuBb6o2O%J=LMPd?U2 zrvlvG^UMjTDLnGz0l9I1gRkb1Oc_Clv6xVgc=^TRXa5S#GmVIm7e|dB3$%6fA1cEO z?6hfkh-N}#=Zl>4q##7DH)IOE z_lbfv$@++E3876b&NwrNJ!YXY<}*GLvfYXK`ErYokwH3Z=n!yRH^OSFo{%CVR$yJ; z>cWuRNjLt5%4Kn57xUwePM}72;%`JnpSB+4Y}IvAKYt1CTDjmw?rad~63(5mdEi$( zo15-=1|p&D}%dW@KIlN?#O}QN_-nxG?4SWi7nWk1fo9`$fvl!0~biM^E+zomt zn@=&>G`K>A6$ZS!sskJeLdmIL$hS(NqlpAZ)6G~M7i0#U`L*FT9~6)CNL!YTypeju zjSqaShY9*@WFIV6<<3NtcO^eB9oB=)++7N!aGubWmjs%TCC_n9b3Sbq+0%+K2FY`O zlQ@YZ4tgY(5ZpMLRv{F;(do8@if%Bfn^O|Qb3BWL`$_{DPq0XDGLl!}M_1A|7Zp$C z4s%{m>ae2{>rKy!G}6&tf3A{zOkheViYWh9u~gnD3B;sh1au)K6|J#FT9u8SxMD3~ z<4k-!6wdSIzn%Zg>9}D30rtPP427>!j7Se?P5OtlWBarId$i=gb9S)A8CCza{>>1q zr~+Qbbf*9=CN(1?C3`GEIWemo7As3hO-?Nl9jiJvMLkAKAulFHF*8d}Egl_ui$C#K zufe!ncsTvlT}$ikPL4#blv+v+R{V!V_P-wMUl5>`35?ht5C8xejQ{Pi{sqvsvvv4C z&8ApZlZ^eDP4Q!jha1;7?t_oO+AxquJ);z!BOr?d@c{o7Oc}j8B~c`x&{^r5yD!Kf zEZ@Sq4O}J@!t`1C-G+>}263z$Pcb|62(ZxePt!JQvr#!pdZZVN<`5AWKff<`xO z0d^$?9@j9A55;)=p2kMwLHva%%ZNFb7dqb=Mc?M`UdIY^@X!Qf_#uv)peB&5p31Q}K z4B7&*HL|o=76J~a25tg(+_`BJLs-3NE9q>t+LiFOw2&>B0{W8YE$6HHThIXhxJ8FT zT4AWIxpTCptd^5xX5XA&4}8@qO|k?@s#3`zg;)z*Q9N>I?cdE2826}0`o++M+XA)_ zqX_=+)d+U=q~IUd-U2qir8z>;1Tyyyi>=~*7#8@93-+s{$!TU^9%dEk`-=$<@>oHG zV=|sjvle)dTTA%R&N~q-nzJwA)QgCV@9RV2W9g=(Md$?4*xDa@7Xs-8L>@=5Mt?X4 zh1A*;g>8TKo7E$9%G=y@mTqPa1WTqBfI45uoSg1Y8Jyk8{?gn5_!d~P#o<|&Q&JFtB76Y7po8{ggGtY>Tn{~8qW_Nox`pQIg3U21g_z|lM>1DHZ zQj#&ci4$#DSmLG(tw-t){NlKW!JS)GGx#Ddf!jI)gF<2@Z3t1rgHKwjc+}w#qY$($ z|7nlJsR3xSisrxmZDbJ<$zwODn3`hxm`KAp72fiC;8UgL-h*t?BMwHJY>ZGr@n~|4 z=$m^Y7jZGuLDYv|!_>hON@qP8)-r}xA!hG{5LqX)vCj~G`y{?!<=bL&N6D86S_xLr zmO-Q~ae>t7Go;P7sl9o+XDFD+6So`mV@9Z9$1T2Q8r!g$6T%koVd0F;6j`GgJj{+U zkS=xg=e^SA?$d(N`~1PyO&w#v2q7_<(A zPaAc|>{W`UU{%^2W@r0Us`~rtGC3_fI9O`F}1F zM=YT)4q9ZD^T6eD(xKlbskVR%3{R3|>w=@Wmcwn0+NywuCKY?R2NaQ4{U8}Np2((B z8;#f;mAvE1Ftl<$+Sl)^Vnvu%WPJ!uDRI55q%U!#zsY65mzSOt1*G+5mV-ruI)uAb&k9N8>-%rx`i^t+5W zeGx%oV`2ux3KSc^16@fxfNq!EmRX_A#wQNMPmLg-BuP#J)gdo)*BrYRW3ps(RAv#} zd9v1|zDyPWsuiKMJRDu!M6P!pKB#xv(V;c#J(3q%Ijmgex-7|NzqcBol&*6f^ zlq`xweY;6L0smm7+YUOHP5*fgD83a2E&eo67vKN@;Q#AJ@!ze}|3lyXN_9JOQv}X; zRhM21*8)|uhDYs&pEraqk>Dv#HB7YHEj)O9Tv1E!$ z46PMwQpxujKBc#)aLuVB8OKf10P0(Xyi~uDEvr7-fLBAc0ywWF<7}eAy&+cbh zXchE(g7-DTN{+_JRV-JAu2fpvR5`Q&>9R{|nUj_w*^_dBnhw(V*Tf*zjHt|u2bR=O z)Kj-Gdw*)4ygryTdA-xksO-phJg|0D$t02#ENa<(gHt=i4M^AQ8Q7j_06oixs@by_ zxPosBwjo;12oKk=g7~X=b?Zl6?{+Py7m8*e=jH$dGv^vgx_Gn%Or~x{7IPvS&OCm4 zWV++Gp^Cr+w8yEm0=1$_4rWKEH=Ckl|7{RRS>}L9@i5~PopsrTD*aq1b8)y2K3TWa zP7lbmqk=}-OIpG5`2Yc@uMok&^|k`btqAwBrm1$59r9wXzEw6_U>D~9>c14-AAFCD z&<^SE-M5;XxC%J76oe{Otf3H~k}{|VM7zWjl9}AbLNS4HYg`I2l@g-&<_O%Ift0L* zbVnHG6!S6mIKiz5Rk!L$tT<{%hvo8v%;WRQXzPN>&Y20?o$DQAU0g@e6$pZ=rcm_) zZ`z&#rbp||MomH9z{31wQ|yoCKrknrphLULtv1M9A(`^t^EklTWv>pa4ENy@VBmdT zHbMEq9oku7#wF|i#f2b~zVyGaa*RuQLHHSo>Vs^j+XNoBs9y2n0lPzBDq{tU_O8hO ztT}yZv;~;^GeZ&$;3j&G1{m{Ht2!H15y7t~t{_}iBc&?i) z=2spR^2bO>jABh^GJ6qb%$n_mjlVS#GGW2`HZ|jDw0Tw75j;3D>}kNs<0m1TCw?Md zPcQEH_gp;o^C}vC&gl87EOqjkw+snJCcWQeJ~n8>yrn?JKo)o^csug=e{BiFSBuS1>uoM9xmtiHb_&tuYLDG$0Cpr=W;!VlHp z2Y&nrvI+diWB7t!X~&7pf@@G(KOr}i*Z#yJ^Y{N&iT{O)Y*MTef5pT%MgLd0{wLu7 zJ3IU{m^oS4+5UHKAhKju`%nC@+Eq{Q|GlTDN5D%!&Opcb?;-ymn8-GmB5~#LowJC4 znP*k>KRC+&NALbGSK5E6(*Ao9O(P1m|HS`V%pWp3l6|8RrOI8N zWW!-KQ5y-@ZZa!Sz)_qK2oMZ@Vte`L)Ae;4;0Z`%GSUtHU;u64A~)C9P3{QO>u1G@ zjULUS7puu_DQ&KAj6CST-d&X@GrH-6tU{x6*VudJsnZZrt9Xi({s8(*^$|pHqf`Ap zddA&n&Vx^UU2Z~W&{uYd-8S(#seVh(StbQ zC=$&n*c%rn)uh4Hq1*bh^w=SKn|3mj)g#jUkOs7uL<6_ML)c-$;L%CSy;C1?kxF2TyieM%nF{|Z1`7xTp?-hux==mA5x~Ms zQ*os@1#T8_*h3NH0`TNpuQVI>x?X=m4e%+Kgi}4k{HY8yu^)&L2<%?J<-DCf)*)cH zNhgpFm{KQNBq4|=u1;0V`7fwm;BJ|R=2*ji#fLJFR$wnZn*QR-Fe0m@f#G--sf^C> zRI*vrgcJiB9@g8r+T#91vvs>RTriMI3y3}sAlsNmg1dsUN+zMPJ<=p1=}v5p0P&0GM61XXDN}Rhd}=_QnwniKDC?Sk zQLn+;6E=hioHRX3@q3KS4>{T(CClCibbt*BDv;jnA!m~P-N`I|$ z#;jnFdCepGAN&D(_mE9U-R(ngNoj^4nsWo$bC~lN{)4IFiE*Rfo4Y;V&zdfUGH@g3 z$2F1(D-^nG)oEYgWSWe*4*B)0Msrp6;WS%hm-0^{e4vUGw27I>!`2OkLG1>B^W9cB z-F+!D1iUDot;P-2BaC#|NSK%$e=(9dDhbRTC3C00trL`DxX^h*7mMM7b{2AUpF zMfoX5gv{f`Q0pY3*AR8|mP*mujEfIK7brmjg5(ig3Re1mJM_ng4uk*bsW`x?jHyH5 zL2|<3gzRF7-|Jngb7^s3MMl3CP-&9LueybZZAnHd_S9=gYeT`iH*r@)jG zmE3yi`cGfd+yo%s2z?tnc&u0mhDBFANi z$~8R*%AhAgU9<-0*r%z<@%Q2>T=VGs=r|k+u}-p3^RTOyoE^d0wiAyyFOK;KqNCtF z=;*}Or>3;RE&)OuAlBdFii?H$C0BE6k8snA_-=UhXKu^tR^q@l@X-P;u={OFAU*Z)Dt+Ll{VluX=IoL zv6&n2A48ozAqKmhgJP?aYz-KoNO-aOt z+xe^D;qTd zq19(wo#46{2o#s2MLV*zwbws_3BxS>$*$S8C0Hobg<#W7;qT)vVVw6A&OQfY_=lrD(@H7VcYu`X< zm+D$7oK-lU{Y5-j=KGCcoqgVE$|Q)V1qK#Vha9$)M{L9>aP0&X{B550nS3jFOnC1H z3%&a5X28uPauRb`40(t{O@BkvHH zILwnrst-??8m0-$p}4kRW3*COX`O)8672KO4;d-0WS@EPrK0{L;rjqtlp-hK=jaiV zXU5VWVBI;U4Uy~#48<$e)V!M+u5#phNGkPhjTi*Gfd{~7b%6Jm@iSZ`%!Vw((B`bZ z@=V_`qqPM3VneyA@KbyxRlUBDDo0+y@>}?J>2rxMVLVYI!2@l<$;l+UL}kXw-j5Bv zMUNy(dvH+_l&3xdvd9WNk57Cl<9yV_;8BH~OwzfH7xY937BuPg5dK|9H2VD9k{3voe7mP$jXdEZRQnMA{xNd=;Cem9j~xRY zMn$)z^@aqTD0*$5%-HkVV!D<{gupQhDn%$`%oVS!ZXtI(-gX z{Nst%eyaNt6**uba4u7Y!5R=$Si;S3zbrg1eJ`p2x?*)`{>lxtBYuS*x|0RpqBKA) zQnp$Uy6}<6KToY2?Y6_Z+140$0{Uc^WynjeY#^$JRTH33XyR6vs+R4$-v>w*Vv>N> zU*iHhKe8NulMc^^IWui3Dhb+EIm%;CCLs_#3_vY#XVEcRax_5iQjPPo#a&tg&+QhZ z0;7q)_kE6`kg||VUO4(=W}xf>$-32<9>M2;4ft3-COqRr-G*r%djy{3@@c?J!=yhYD+JzGxYLqR;bNn$ONidfhSr};d zKU@>@BZW)AOqKPi@WEnTJUP8xJaKcfb+xg&Ik8_Plc6E~4+kS}YG$Zm^|xUvg!V)O zJyH4c(Muz~TqMcn`y)c^I*xr2&l_W6cDv&ZggaWF6*scjb%f}{j6*M-P}QK&p>bOR z@z{;K@r8cgqGq-YTQ6K_pR%u)=a1kBI5;;mSxZM3g5ZCj+T7j)%>=cCeGqu=F`jCj zUc=BZD$+k2%jKXShM^ui`7@HDK!QtQo$<7Y3)7Ug+YNuUFI9dfa+q{`DX!lB1`mr+YPR%Bk4r zhDWPI^;=2Z{DU41c!KRYGsf}?tJ){x9kSo&&;u94=-w`?c2SV#TjKI`o} zYIWcleBGjWCiY_WbEiy#+}UnuOkDb||HY^8|LN&3Re~TLz&iu@(ZCahQ_D{@>wWa9#LRJ31OHGK(n1bnA%{4d%a)9b@NXo;0Ni6m4qE5i;Hu)!s={A z9%3clz`Pu^f-55}7*zv*67d|kgA7DoD-VT8xTv&ZO~@_tNt0eT3=t_w=$C>!izxPTBTxJ~HM#@g8zxH+#Z!{az_o z-&NH=!?ZgILVIU*-$%HOw=TJhPA3_zWT)tW8jt}!RhAT)-}+jqMN{4HiHgRj$&3Iw z%Qc2&VuO~!!%?Ynp|^<0l3ql@CdKH=ig9w4@2uDfxSc|ca+$&j%zD`}Qjl$-LxyyOnHGZue`^w0_1&^YTKNg^l1e4RK!!SHN!TXU^zt1>-!D&Ppmep zl-S-hXk5O|qkR6$?g#khilf_s-^%pxqM^k$FaEvhx4JAu75Z|Q$HNP*Kn*q7R(GM_ zpPP`%td%l$RIoomGi%E$T?UrT*=QT|3sn|wg5%Kdx;%K)#nko=d9v^$i}gMQ%|2QL z_tPFR-F9!i)~flNcas#=Ws`vDRI93 zhqm*9qeNUg0+x1H=wCyX1n~d2HU6jf;=kJ(rApLl|B3(I*6^`%+Zs*Wcl(BdX*vsj zo)6E-+Q?amZ6ua*EF4bcNz9gBT>}puFAYuy!2n3Eo;dBZtXKhK!bEv z>#MA)s=RuTczm8%GS-}leDq}ZaC(2d;6F;r8PK;^oLPlVXjV~7t|@=A$l6A|kx|t7 zrYN-mmP_J!C+nL@x{K`8Ig2_b$=gLK5kGZlk$%isd%clS`LJ8c$+>&%9Irzwl-SJb zmPsA?E0ZOS?%^~<+ssb%e%h5(Z1JSYmYFuunH|xYq1v*Jsn}HZj^vqxqkK|Bu?3Ww zsUo4#B0*ca;z&7(>Dr4duH=w4A46SDuzynMPLK9Ve10TI`=~mNh(H1!iUw29{^?Ubw3>)hB%~ zXXRBT-qM=V!mi3+uXkaVC2!J3#qEK`)cFOg1=Da}56&r7e`2G+068l_3I)@)cOYW<0wkp9y$1hSMqv99H^jC68Cjb(%fGULI*v^Z%`nD#kb#J%#Z~f+`r^A`4G_kXdk%wR2KutZK z&$QnTC--N^-)-qPuk%A7JosF{pT0%2`@S>a@0Y(D@1)%yUwJ=|X?^Ko{Mx=g zu9-c2{GW07*GPKjNZ_A!x;^XsZ){y0T)%KwyZjs+o<12mczHNEJp+fo%}*a;^t}^w zy}Z6TxY<3PF3$EI-ma28;k_94?^93g9R8xsLv2A`3oxS-`*^<=7kQqvV0ZfM-0d{- z5UM`8b?E+Z?}iR3a(ia)&Qz*4t7uYy=}gXklPb{5GsvKDo8^F3 z{>uhEQ%yrnJHZin@qd9e&I2a_$d=y~0Yu53D#0Y>Ss)D@GP@C$ym!Kj{b6?j8nm~O zIow&KOs%%(B%ragURYOxe&dDDCYXK^^-*P-&_G(MZxvZVcA26sp<-ihQbR1QdjfBl zCphyWP^|$;K8BD)xn=dSTP%%Y9KpynxjkLfTQIcteK6#OQauSy#2hd@VUp zgvt1%Uv+^d&>dqUIb{Ms=dU|VoMPOcpPKaSOQqBHuQLY-Th;qDK)GEsb_SU_0#FIx z-rwni`nYDw7%HZ^W1>xjPe7c{i<+~@bf<#Y{So;6;Nsn#FIIV-Of$MBj)_9}g)$!) zU0sS#73AtMA$bFtv%~-d`sS{XQq9Hxx>rODGu+_(xOYw4LOZah4zmOqja!!*d9ETz zH3q^0l3M|dPm;81L|8D=>x<^ejdOj-Lpla_pJ>?l&2 z#FgCyP^*Z2E%=&RH_y9OpX0s{gK@Y9#33(3b%cGZgB=4TZ$Nv?(ReY^1=d>Fg@KLL zT?K7iL`99ig5JK}i75yD6@CGyI{b%)Y_8K$j5@(Sf8KJ7Zk!+OR^Mkh7YC~ZD0AeM z$6hmJZ=ajO;kci!dvP}6wPc@B>vx$At>o(a07=b|4UGDqQ9_6^Y5RDo0oC z@jtQ_OiDpCHg^RPv&r-w{Z$}GT;mtoUbv}(n=fZ`CNx+4TT%zzsaQ=-Pz!LYAGWQ5 zD^hw*JR|Cqv=-QjkOfgD@~`adtOX3nTP4r~c)DHPo>xJj@?S%HIs|sk?{WzY>;^nw zmKP2*7w5$RfU152c&@=bbc+gVc|q=NctzLeyQ>uGr7L5Pw1!R4z2+4{Qa3{7x{N5k z)N7YqskC|7?fobF_?m`F0E`rh)8ueeYr7s>uEqkyDV(1SLp zMGjQD_6NhJwbK5%n4y2%WZtYw;9 zePc^m(tr})=2MVD^eY*<0}P@3ZOw`@+u#XViO)lu{jh0 zpwxoW-@ldfQU+*Dm%HAe9oW!k35c(s8mzLv&U()K32C0|s4uK2grUON$j%-;$g5Vv^raK)0b+7A`j!B%={ykL zM;Aa%KW-cx1kB-15^9S92nOU}RN8V1@lK^Wn2yr{CO;FF^KE*evsdLatOmneHNcj2 zi3xq|hqoJm3EE2FyvsjqzwfbcMr7mnD=Jka@;NBwjymdY?Cxcz(_|Z98tpbSyZk;- zBe{bTo+@g0BJa*B)x-T4rBqN%tc+8{VsgyLYFAeU#^Mj&3TFTh%$CJ3+SG)ab1fut zmUaD%Wi7n7zJs&@B(`g1)(r3SBLxL!3R=#<-*$XBJ{66kLv9YHagDS54lLqoKgX|Gb@+dV)XBXOgK`9Z4zfAJQW+(&S7KO&j{qf z6Br2YHvF_UOf#dl*pgYK5Vvc_tpkHN6_+#V+synUl%~Af7(1oqX#$xGYqSw9XHaq>3x)aVC1UBJF zO(%v?@<<*(6d9PQ31yqx!^-lK6N>wp^ytH7WXRq-YueepoOm!xv}?jON3>EgbiuAJ zTxGnxqp7e8g(XJYLeQ3B-p{Ib!72Ubvh z{Jbb5UV#GV-ctxiW2Tj{5XWt5iPB}8;Eqobfx9kgw9PE26BEnLWwA&9V%x~5ckj^_ z*|P;xzqFwrUVt39*s-SG4`z$Tj;!b$fCSWLPilwO08`ID;?kk~3OSB>sz~J3}o&$EyCc|NNqFBVjlXX;EPSPE^ zejSo_L?@Pc#ZUE@m<=kilZ+e5t)})w z#F^Wqyw&%>mYQa{(M7{Y0P2pzl3;ol^Mq8fUeA$4aTz&lDXy_(Xzp$^7fvdeEqiBv zuYux?uQEq!b)IZreSeqbV}?m)kK)ygzJP#4nl%|K`cUSp0T6g?1z|(!kE0tVQX8mv zMvM3i-QBujY?e}rOe4R7I~WM`HI(@~ z)kcBu$7tf$B9^mypAWCt(>C4@UeEe$FA!*`c6KgqH6>i&4rI>e&zx}=;TASf9FBDVYIB28Py?|QEO3i(EAA!iYNUXt_`V(VDgg$S2KUjl6VHpHOu3<>=^ zTajv3kAg8mK9weKf>2G>$T>4tMf?8 z-~wo;@#yLqgxL#lJwT1)PiG^jIT17&+l3QoSq2@u z~NC8f}Qxv}& zkW%U8M`$xU2&O}Nm$}~>PIRCpI90lj&+~8R5=#82L1a$P+v$0}n(PStGl9z%{-`v_ z1M>I*l7SdDG}af?jVUVDcUP7KWkZA~L*e&r z&uHuhGAsa1C7R{s19CZ>GOSUxAo|-d#dhOgQ(Ig>U?-ny%i9GY3Q*{HZYd+>sWwlN z0H#GgnVpq$PR$&p6>pb0RhR0?KKQfSlY6Obc!Gpt`!>1pVNN1sUO^C0@MOe>4^-2R z`a+=fKbF6b z=bzvB$#G*~0J?oqU_I56Og;{7CCq)eEx3z_ObHQ%669j!W(O9aE{%1o_JZ`q z)jTh#F%zi9R}X`(BpIJYCJ{v+FYUnN;sJA6u>P}mZu~@tB9Fv@Y159ZzL%62U}xIl z(A3xxrKn;WRcOI}r&jk0zg@rB3H7^fe)Bw#vn`i+mvXU=A{Mb#rGhRCi+#zqnEZir^qL*je#S;Sl|M zBP8svgvD5ywaRRiRrlg_QiRJ2>&3v8KIdoz3eXD=`X{5t%FRaY-=pEqbd#2N;JDqz z;`&wfRutBQ5fMRntC9v)p)42KOiIM`(l?v#%u4lvOfZ@!XFEXMj#zk ziMlmpUv{)zFT)JJle}k3nK5$l9nxgz=yh=I?8p_pdAs2kpVy_9nl&o981A%t$i$x{ zUP5Y0)m=D^G~9+IIl&Ad7gLY8r@8~qs7^0XmKq+?YUzt?Na^&>or>srtKPwUeAuRg zxd2N)1u&!lL;3Ph20~!)>*&F%j)76!DhGz)T_(;cy3!o04`C3fM>$*Ey%w!o-Hj*5Lf0g;Dka9yodcsI4OO4gyY&30wCp<&S zzi$u^>LwwVv`xn zuW}pB6VjhXKrUA@*T0KdxuRxosm`EO{Q)PO802g+P56*=>2c)>3lbC*WyC59L%kyD zJjb}VtgW_emE@K%t1JD4p&sDYn_iTvCf{t5Z{U8e zKk!nbOC%SR@#UEB=-`$mr%)*>G-za_qlF6=)y0clDa@QeLK=@qxoS;X0A2l!9?*3) z)L<&)G25K(<3_yzmHuH0C#eY;U!r_2Ujb~FwkIO zZ#Y`y!;S4R+QLR+3VI3dJg5EEZVo6gYAl`m4nwdHHZ*@Miy4xYwCpvK0bxv9tk8DZ z(F}b)QgY$lx>xnPqcAjz)!~UUm>eV754$>f%q80CAmNrr26IBlVc7>r!Y(91a$?skw@Bw< zrL%<-`GRM|SuU$Z(r99%-`&1o-3%M8F&;%4%fsZhTt)Rv1*|)QvXm4E8uWGT<#s~n^-U5%X z;)B@=lc_>%>yb&8F*dk)Yz;u@)7fZ?`t05To;mwYKUk+##HU&&S*(C-= zU7zFH|AJ@wYqAp{-W6m_z=B`iD$C!voBFx|%RG@boC)86*Df}@7042Iasz5)6s%~J zFEaY*oB=LCLy9lbdBWj*mt|SjwK%8?dUaNl4wIc|a`ZW~Se!ea8*_}A1OCra{2`hg zshn$sdP0>0l@lN?=&Y9fqaUrA@dj0trIzFb3}qkUsV0YYI#HEtoX*WR9i_`U0lQ>J zHUh)cHX?dm@MBWafg;+7HUn%*V_)RA=wOXwlm&DVrQ($6L7OrJ4h?SF6H(T95t^3G z5li}=LO4q+dHB;{QB^V9BjZFFk3z6wsE9hjrCqy)@@YZ<(|pjK z?rxk8stFwbwfq3yqX)<9j2k%@SHelJxOSc)M;*?!5&KAR-EPK8NxP}PnEAOMjZmvY zIHAVdq##$;nhEn%gsCyHn|$NgG>ptTjkjT35#oS*u&3Q7>2?nAub`_ulWKP4DqgE( z6PCR&oSn_`-{Qyggf)N($WoPF?!2=RA|@tX+Oh=hfLFCVpx(ne1&S-Y!`T|R~v!Rva8XzzF- z<_$vDT_=}=P6V70oE(vho;%*uCtA&KVg}N_bNinQz3H=LixLsD9xJC!EcRJ{{^rc> zGk})ZKd|IH;A8>~0EscjsuF=tbk@=KYZP5jBKPmr!?J1KgaR+U(R{dWfYlRv5EX64 z+_K+Tamg@u))b)A_Z#U{?ph3%!F@Rx<$SR+s_3{^=$H>nb@1@ZWs*ikb+w6q};p(qkf$v?Ojo6{_m1K@0;zAgEmtzi{wo`OE!Cz+3%uPtUFC3-<* zcqKwh?7xiPn3wf-RX~_(iSN=*!K{8qlgykcUXHRSb|*Hyzf9bGyq!2zZRlp{*C&r( z=D))ZT#ew^2^-GXrRtt)XI0XzXl4#?C+d>-qi=E@a(^uWbu9j}D+BKqoU~4lhA`3) zt>V^7XVcf2dN``Z`S3^aS}otwytvPj#J;WSvEBF8o0uu8iYzm}bw9iwrWCZFw78n3 zLNhP)av%@4NuXF9LD58*BE8tJ7c^trtg3Qo>I3U_>8RJxKD`1Fzrcb2UCGx{3x!+5 zv!Y0PzfQve=BOwg?wuxi1vg4wI?}2P#}Se32n)dK8fuM4rKyX}sgadOc>`3mj46tE`+SRn@t z{(x8F&!Q=Yy>5gOpSrpLwo>KqeSQ<6A4mhsgdbr#7T8Z?d<<oM%3vCT&X;P zaVnx!-`X&zN5H3Zo1NDZMWy5AJ;9agfqSLeFPKRsBnCRUA{D%^Dd>lXYk5!m9c^cm zt`n!dXzK<7vljb;yLAOjc{5gDj+51;8)A%{_VkAhr)S6K<4fncK51y(t!xH2G@2aq zy!wrEZ;5q$8a_iY8(*iFlYd55Aa?8blA^F>QRj@_F7K}Qsr7Nq8FH;YJh98u49+p# zDScRT{5wXJtzV2r?WPA@s@0YBbdgc*oVB`%3 zmOW({LFYogy220=Rs;z8D@!O7HT6z4WspIj|o6(RR7J$$OG`(ds^5DTC+HSc6t^D)jBUih4S!- z+Ilmr=qQuTT*k!jdCNrL&Fy|NBdDX+!l2#E+$Ul^(^|C3N45dqAO|PMgHqs7%6-*k z1%9*RL`z55uPkE!$4kMCd5$M!uC!w*R0^fi@WG&m^o)Ty-97xstvO+Ck;{Lwat(Qk z!4#;Q@Z~7Vco>Mv+u?@fB4`M(1*6!16FuBavfxC~C7;2OUPEe~y=YwKlgxvHe)=V| zPC8MH)A&7XoY)UR7W6}9&hQ#~OSM611EA21oTI?_lg=R5&489P3qK7x<|sh%uvbJE zEc7X#u&wwOOZc33nkqZWxMN(%x%dOZR97f}^SdH5k8iBB0f+AS+_C!>B3cjt;?!bP zpB!w9t5svEG|T&W+!hr#PgaYZI9yFL(dYjZlcOab zjK%#Ki`?4faBg^7qCTX#H2O0fL(DF%1@8NJXHUC063 zou|sHulBQ>HE$yvGG0H-80~xx}X2IHgjtLO|hII-xdFld`YKo#gGd=yO!| z2bf!Sk4Axl*^{5Ox=N4mykX$}cH>X{1RlwRaV+@@8Da6161UP+XWG|S)a&GNfj*Hw zHq8_gh7h}!N)_uFd-QIN3l!~}2mNi4B_4lBctq4WVqy%diTg0bH=~u|N72jok?l?C zQja?RsDG!IZW6ZbgYqz3zK@OqRhd$-Rn@Fv0foU~HAgbT4pmiriw3E}gYZbL>k;vD z=ovS2Iz*o?bu7*)?YN&*iw^U*)v%^EJk0p1xhu~d(&9RRTCI9$fFJ&YrYvNGG|nV& z0r861X#`EQv5xef7>@1kzd1yEt{VJj881n~_`(!)AZL|Tk{g|r>eatccUa$iy!aI% zvZ3t`;#Hepd5G2KErSRh$)x!dWhyd&Z{{t~vfhX4stC;_3f`PzVZ#IZ@|1P!XGO;R z1H~xS!pWbhI9A3Xh#drlvks$(C}}_?ELe_OpIAF23buQYhkC*e9q?9a+BI6X4&>_W zy$lW$UitPIA7>dfeQt45VBY#>bU#nP(<6Y+yMfR z`g4+VKMi)DX5I{Um7Nxx3U1PmxCMnN+Dr2^;*&~pwkW-FP7xB&t(0bkksLQGD= zSdg=&st2^OSGdx8H1zQw^|>i-rDCXK3X%Ka9BEWweg=+oBN0JMN`0YYODN z^ysE4=`(8u|J~@1HcLN*7Ra zVPj`KjY%Kri^&+aM%wOfltL+Pdel3GsUPh`YcH7l*K4{REXk(Po z)gVbDiPWK6_Mb;fI-z0nu3dL=B@d-1@*hM~Y-tatPn?tI{{&l1rEVKkCN&qQWIQ+H zA{Luls&h{a%ik0gjQJ|>|4v}32repl&l<~`1C5;n33`C(viz@u;p-gXNNy?tyE@5 z^l-c(!k+y*oJhh1IXT{tQmz$!`ZBQO-+I=s5+2h1?gttyhJup( zFop8x*sbZQt(W;aksbxSh>0+5drU zTox|mO1JP+vNf*69ZveYd)w<3oQddu(+2D@4_6-K?GES7R;BiSVl@7~Fr_92o;GGy zrp3^#p8lpW@eNpN-P?05m=xyJ+y~}xjVaOuKT$@e_sB^olhPDQIcs6q_FFB2+vM3LS}*f6ecu;Th5pCA1XWCuD1~S|20Uxb{fwPU z4VG$=W@V+OtX|s)5pS6rqQI&aD!v-QMt(bRWs>`l7FQ`hlQ6{VuJg7c5<@fkE7bb> zU8*N(;Cl)0J>?4MJfwh)hE4gF(S3~ zAVuGKm)zW2SFHgrwq{N#P3M$yt%_s~Uz#e7Mz>vh4TBh;8NzKN!ga7e`{i7J$X zF)FWbzeS}v>GysuBv})TNV~@D7}EpMiubU+rgPmP`L%eIn6z%yNPXuum0K_G2jIkE zA60XmCG=h&4u>WOC$f$w$NT2OFd?f6FUsf1q$*7g4xgHrhHNyxWrtKrDfX8V^$K2z zOwf%>*K}kC{?+kO`6|E)hIbyFSn9ka1m-GexVWo#`;tOxpY`&Xc;+{oY@U+9{tPCXo5j{?>P0Y~h=sk|1ZdFJa1<8FLHoZfhktNzYha zq-U^e1zbHU0R8Vb%1XdlTGt(0gie--S#ok)eBw#7`UVT>79ua(+X{%#A7wWxlyqdA zVQ9eg-C|pZM=;Fu`9puQOr=q1GVnu<0T{d~e zIj&XIOnwQv;v13^)cNbm-T-XX5-6-{5x=fU_lzBL%NYJ>5x!L{Z?=M z-Fxph_sR@2%M5eQ^PJ!NJn!!Cat_%Pra1u>ii+VE_1`@6Qk$H>hoTmhng#RYblW3#Ebn7l@DF%z zlj;SOwMs5OMhhS9mGLxL2GLEQdc`%3UK*&qDj)YoDDmX6*Q_{nQ_J#evp%pv%k-yQ z4yH{x+HDRUl}+4T4`6G(Z>HkSmQy%O!DBxDsb@O~EK7|!E_jc=B1O_JU%(+cpqD6U z{*mJw)nc|G$-bOUJ()gBa4{VbcOP30&ojnBA|T_yT)(GoJNF9KyqL_YEu+xK#fYJk z&E%tnL`H=2=ZcwaQ}X8nwRKS4qH1mSJsw^P;Y(#?*-6_u1B+SrZ1QnF$C--YMWdx@ zHW~jRgq8{rxI?cN7<~)(AbMA26QpQ^xY$TrP1sA$C_fb92L|?1ms;`S7)}j5H#+ugbp{a> zl~2?(o~cC&e3_fmubCM&+Cz@DymqXRujE(AVRSFqD92U*STWQCvNE1~{2HZAy6v+w zI#_6qJ1g43UMUsOOvv;?lsa=4&{&|PMB^C4+iJLOB2t{rP&F^AwCdGoH~H>8Z_zIo~wtG(~&~N$Go?2m3pF%m`Y>9FXFNbxhD+zP4b_{c3*k3T9%wD8FL z74Ih5H6Q15{`;m}>LuB!IB1@qRJCc&`$r}oU9h~(2y{3CFmxfO#|zSoK1(w7F11m+ zL=zxj+RHjX!P*7UVMfJ?2J|xoO6tZnRkGPg&c-A=BU!8$(1#s_Sj^f(7Datyg93{_ zcD*!f!Dfn;o?sm!5H1b9_T1JcTSjjxZSpE7cKHosKaQQsbb_YjNr+jG5yU;#lQcQ02tCfH1B{-sARmf)4c} z|EZ*#oxTV&R37voA#(q~JyyA?$sUi~Yo14iSu}&!@b;o1bX*NI6=Nf&^C6}pw$+3F z+<7}y)j{Yo;pU#Wvi&I$$9ii|%NfPS&T0{KhGGlc#vsPl^X+;y7MSVp6<1w_t9tjI zo%YAu^^ux^<(2&4z&D)j1$3rLSV7YXCo;~AiG(|QxKc`xWU7m}{d$|aK7{-T1Vi!X zSliu_3LDHrl4%|$mKu}On1XxU__Cd*j~-nduZ4bQFyALT0tt`3zg(Y~ap8gnK_`x4 zhq>d1>t611ar7zxgY;u)^-F?yoEt0Lql}-Hf#{RoXlf>X-4uxo>3x+lHy7m+dz%Zo z-Wd~{AB~c$Wo_+om-VW9Ol0t5ka4p@A}4s0JonU8D9A524Q1Ed^h9TnzB?9I36@nL zCK;A2+FoHBazLLtJ-0Dd>P_TW6VOoaVpOFDBq@Do#r80=JI}V`rh!^>L2aldN z<^kFcCYyU77-i|>fNz6TUJgwWb!B461;Mb~EzkS2@yX3bj-ODRnQUpHeJ=~R1~Ao0 zEFFaSgO+){*vagPQ9Z?9H8_UX>iUpve8^*sos~$sGBzS!lfaMA*I4S_PTt+XYL050 z0f+ICQN{*yd8ep_p7P1eTN@5erPp`vgkL(BrC)E{?(O|t)#3Vjr7;lgl-pPC&4gw3 z;C?FI>*)Oc+b5v!D53t#{q-eB?&(;p3eFkuYnT2Ow(Wut+rFSMLO$JT&1#|4 zii|f#Ywd!^?rWoGxg-?3TK=+DeSPwc2=?J~_hw6K5kub*;Q^tjp2I77n0V}*5H@Oq zcCi&L)$OfOQRC9F8gWbEZ*vG(A`g2rGK$hl#v4zG}ey9vqFhD zE2`l4Qh6h9JDXuVTnkEZW6JT?oXUUJwcd5W{D|G|>)@MBHDTiAbmlO7ox> zKcWp~#_Dy$g5^~N;rL~3W5QJ9OO}Bg0Q3^$L;`I~>1_#c;8tOso&`XbWBABUR<^9@ z-VO$b@>R5=?4vN&fF09*M+>1Hip63V@&dyw>IR}fD=KAo=|bCs-u^^#S8paWMnO~g zzG*EK$_+Nu%ox%R)gY>E&dM?On0e?@fHN|tzR`<9tfUAjc$0A^01+gaV^*UN?! z(A`#ipvck(+@)1i(}IckyDGc>QxC-T%ep;r4K_iwq_Q*M4$H6WOD=W|3!6$!D2P=P zt;K`g^&M$8%^Sqw>;uvHUYd}1pMYB+iWqmW9EF+ z>!sDozZ8hs3Go~~Hxl<&S;rg)i)pV%=uwm6Qe*T}f09@!XDFI$50E0${f4z_m7+uW z`KaC4NVP&~;Fic9{^jTP4olYjFInsLJ&Q)5(pcaA;(!AI9n$>ND&yS-!;~&9Vh$Vi zh4XBu=_b%vkwV+2lRiKM70XeGK{uy&o}I3yb-|QQ^1+>Bg?PvLT7>?zYJZVNrmPWq z@`MZ;C;70~R`*{=xIK;ucdp6~>u+e$1wfs=-n|_kt^OP`^cnogj7zlmNu+17F>qYT zqugNI+F0IAf0HqQ+hdUcmpcwpOl8EH$}@T_!zh7QyWGCbx%>$t2m0X(wA#= zlMuu_xs~j7NZt_Ig+0y>(VmuvvJ#RDkK6b?4Ldb@(zWZ8xB|_&WcI% z*E*5oxYA|?oIrtyARi+b1YkFIYTlUqe( z65B#%-0H@jQtQ>M=?;pxte*8a52TSOkn0fD7|y0sZ8<+Jxq5Yyau1wAiiMrr(Gb@c zTVvcuiNO7J=KIrJ(sPkGK@fLb+}?|!t84m4v>Q*PO*A*;Sljz`bmLgj+{Ib~A)7NI zR%m;wGBw@%;oaagIlm6wG&4Puh$B8@Vx|mipQ-7$druI*35TR~wn>r)Ljlt->Zy7V zacbCDQJokX0^JTY&jZF=GnzWh}rd{`I?^m}Dyx5udT>ImLBy8hXJRH>eO< zw9m|*xpH`k?)-ks<1IdYs4PGiL(L+W(>sX`JpC5`P4cJiB~Ti{TjTq_g6{%7Q@R5p z)mq{-O7sOtM$F9Z%7Is|d>kz7Zp7wtsI5F?Kwg4f`I{L!djaEev#R#z--fd?LTLe3Xr6lrMCb4i0U&Fi}&@F3Z|0x1IMFZuB*zyBj+q^;q z+26uv{1^KaB!ZpwtOkseo@(b{*EHhapKL5Wk$2H zgmxy@Jv$zjKsyP=)mh_XWm7UtQvVp7wRw&mjA}O?5g)e|8J7dOWaHBilYG@#|A2Yu zabm@aBbTJwg;v5if6~*}TtyiVC$eb*Y$zK`j4f*Qy=SjpNhXNu-Qn|V9`Xb$8xPj>32I?B(ju$Ol{9ffmh947!8HG zTTqkvB8ho&ASuyL1Dq;p-kFXzpTQnj@&{*Q(QIhR73b!RVh>PP`Y2#r+++6i1Pv8MnBCv z;I4#BkenCeX_YnUoSQY%Znwm{bN;y1=byj^oNv!&BoHeVBc?lRMLsqY} z83O{?ZZxC&3F6et6&|XxuMyDCFkj|B1gh`2G@Q*r924Ivw14P-zUfZ;qU$!=2?$!x z&`A;FWIx+t)Q9iW){7{&!RlJ8nCenp}egMUq#*tZqWTp2oH=hX;>o~k@m z5isAU_0KSAmJ}6+`1t26NJ=1ME}s$tnbb?~7Ze({CvyPAxe}+c%j1>xt(kD97Lgjq zYWQrLRXvLG9?qq+rUhpY+n6uS@N+)ib$F-_SWYlh<(SpJ(0<>qa`jRAk@y?3kC$J< z>Sx)Yj=KK8{mL;JoeCiRi#KP0FYrF12T+ULCLO*0$j-s*|nl6JRG?yS) z>q3y!62^4Ahl3bRHE!^+p9VdTqu(}o{*2M7_X-yoDc46+kPxs?fX#Xa*6g&OOP>^A ze1Z_v=4y50wx8%%PL=+_0bq~UX+v*bxnr;!>lVl`dwnPZdFlPiXcJ}|;^N6LJ<12&&>X71xy!MPC7yucacB%w&MOS zvV$3i&_k1v*~FW#szrFRww1G$-SjK0;S77a)n;F4Nh1Jr5@~f_eQA(cHJs7eKnN?t zbjNgffkCo30e_h>p2*OgVmoW1O-saBO;7jpzF56`$42TsGXq*Ogblcpc$V4N4I-1G zOj-;N@Fs&J(zW6x_H>bY@tPc138d~tD`rW>QR1zjl&&ngspJr99dFYDE^);kX}(R# zCA^=c;}MlnVkf3XBii$NSY84}`HmPyH94{yRe=NDo%T#sRp!dn*NkpOZ@gku^dr!_ z)=`P`9*hH)Q8%%;Jpsf%`QcgG+4$Syp}QuS8^lF#X;={EW8RMfu8rSIRb@qrO37ix z46jdc0W#cdQMuziM$d<1b6wO32qTu_275k3RUjV2sjm7= zMa3W$z)V0ZYZ7fhI}O8s-D~QmbbuqtGU(izg zjE#(b@-4}vr~tG{T^Hc$dUnOvBrYi;>gTZ&gC{k>S6tV7u}XwpRs4|Eq=7*>n6b^d{*;Ur%zEnUGrnL~PFc1lI$Ko3=vJXb^xqQ!UNirUb#$RU04S z@>yxxwTrAgR+3c6fM5ZK@R+`DYWCN~_v#$&B4ebTT7<$vDPAh6cjKz!~z49 z?u2w^&Wn@mR6h~A+|KP$*RBaOZl|L+v)?6DUsJT(-w@wBGokD5E9IFlvSu1LjjxewsZ(V+_n^+c!7+{(C$7DeRI}9q*EbA7Qa;1(D9Sn0H#eyJeJb zD+bz#;bg~_MQ@;Q404+!?L9&kc}_-4yZBt8I$yk?H!LJNf5@lm%xW9)lrHivkIXT{ zn88_rU-v$;zGm-uC4pJ2__yu$IKY0(spY%;z*uc|N>MdLDGE=#2A&5I06MBNJpkk`PTfo*#+}W?8k(vEh&J*JLm5^JcJ2pyFlgQ zX(M&>q=($;tsYQWyMr$f|0x))JQz?no|;yyXcY;{g38OnMnucpG6O84s_;n+)g zU?o6^6jGR@re;j&LoT1C^%1X6qSK?FE?@4sJy5kE1zwl!tfkIR6c@7ZSddUPqROLyjFEJT38g#jjYbcWV;A$OI?F1Y z3fZ1cXvT0Aj0y1Xo|B}jvBxJl@V`o|LrX|D2|UL0nBJoOoUoN^5FHiShY_R4{EZ#L z$m9-XLu%)EQnIrGIqGhyO@5pRSWeN(*O=<}LN?xvVHSrs*VTVEw^vRw38PT42` z2qt)~H;Qrikr07cfH0(dcGNO4?AjeGRTSqQ5>>9?UQ{#eLF(;WUC+wWJukwFp!SO$ zp9ILEX_~Q{53Q|g`cm|v;+B3GR&0;0mDqEDkJy?a@}w!WtryHq7oRR_6Nq7sKiq&5 z9f5YzO()GnIqWK}5B>ADLtZ>=UfWt_(i+7un*ptrK>&qu>a>i2KaMNUH(GcD=8xC0 z-e@obW-C~@*6lwnP+go{E&t7izgBP2d4h+ibwryAEVQFTKY1tkt}pMM6(o#6={2MW z@rjvbOIl*kEKVO@vnQnMv%@YEVC)F9$ziY7)AjY&A`(G5HYO%6IeeEN z`?1x%+aDnNFllubjy}7OlC&byi%qal<@Zqf=?6sEoxEqi;0ag9!zwLQV+l6 zo~xk+4(-&DxfM%P%ew^jhA`n_c5Sw3>y>M*yVisn3S`>OI%T8m77@t`K%P(0bhc?G z+XUJZ61$4msWz{k&GCcpw29H7A8Oi(Di11fSvpYX@yZ8!kReM6?Xk}~-!XZ;a_&xu z@Yp1@+6PR?#L;pNzTl90wg{0!c14w4szShuVR(VY$394FJ>uTh{It$>6&Wx+vSJoy zXdL>5wO|;dkKV-y)w)7-NS~iJuoh!aq$M`95O|llBHma3`hj^%EpdWT>-oE`XFM_Q zSq}&E!!U|lysSV?cF{d_+L3||l=*&SCSE1y1w(`JCv(^L2ILfJ5KWgs4(R92_$G-k zpL4cQMpgP4%8L6e-=YY=(kc&nr@qTt+i{ctKcBEtbhqpP9CGO@9^aYM^9;_!7Pq=M z%v`C({Ay$wa@$pGx6qg&E^Z@0s|96-d7tTq0|)o27PIiiGi_k9oDi(l78_oz-=5P! zlg_42TNK-hTX-*T4tn-f%yVKG!0pox8o0E&RQCmSJ-T~RjQ&kKr{COuqMcoOwRh5U zsX(UsRrQ&46Qi=y}2I$Dk^z$tCxhTs3D~$k-=k zpcaW~r7eZMtMO$W>KCri7uuwmk&bmb7Dhbf?YPcThq7%o5q2vNUp3Y}J&p3`=8BS) z+=e_;XJxdsu!}4Vj9$&&uftUv3ud~5?wrTf@5<@sh~q|O&v8PWNtjWwdb4mD<5Ch+ z7c!z{5l1nF{TyQmEnL+w6bpA8=*dxL8uOyk=QYunkEy||J9H8`r3AXdhh#02#go8E zeTad_s8}M6|1Rv|dfM}ls|UxhxD7-k8sy(RiwU($m0?!Je}17Lkp2D|58*)O`^)b! zDcrg`C5yLo$x4Qhaq`fpgw(Sa)F`$ zE9yVDj|JEM7WMC_AfN}x<3Ed#VPtBD1WRlUphQ5RgDaDR1`hBy-)8gQjGN8Bll*Jh z|C5>#rz&9=CTbi^0Y*5Ma3(kw_?(;H<8AnUfvMJdGglMFQUuG;VSr;AW&U@jUm#Lj zBevRukx;-kJ0%H4dhDdTq^55hE4fOSIwN6^X+NAN?lpOb+8+XXs7 zq0Z_AYv3H~uMI45z~TRBzyk>S)duc;)_-~eE8ZFVdwT276OhCGcLH0W<1b5;Psyrw z4r9={g@8Z<$FL2Xx*s$4+ca4DfPm&6Aa^_GN59&I1SwC33Jf2XvGbR678HQv_#qPr z7BumnX5+V!{=SBYpIo4PpOo+4Bydek2*VMWo7*|tfy~W+;UW!jW)fwFDc|9TfItg} zEhGYm{iFCtmLQ;)rO$tB;D5tiFIG!T`F#>l5fG^1NZx4voy5}J{nsd{rAn16T>2r0 zutTtiEmZhb&8h=OVFh%ywso>}ci{a)B~j(Snb-eyRlBpJQtZKYAON-@;CG;{AsqS- zm8>lt9lu8@{i;m;^M#M>U^0PVM-MMkxg{Kn6W9@C2eNesS_1ylrS!j(cs1KN=?6Q7 zIoJk*pExG~9L0}DzNd)&Y7_TD>P#D8?`KEw|2hql4u59=*t!2I0ep)1&r3ti1&#pz zPVvu~;{RkMj`GjolwWNMK3?}{26QkS!(YR8|62p_xp6%D5_ezk~iQKLe)1KV)b8 zjQn@g%Ab)1Gk-_^`5y2`&HrFu`7`o&ec?ai{)}w7@<+}8`@sAEof!&jZGGRqbO;|{ LH`>be??3$?_=WdZ diff --git a/python/lib/py4j-0.10.3-src.zip b/python/lib/py4j-0.10.3-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..bc54f33af1515c0676bd831bc5a02f112b28e0a3 GIT binary patch literal 91275 zcmdRVV~{4@)@9kYZChQoZ9ZkA%Qm{pwr$(C?JnDP&3o@R_lt>{ckayJn;9n~|E!3O z$hFSi`=p`_C>RXTKQ2Le45@#<{LcqC5E77s2b&e6nkF2OZfCnj%U|me7!64AF9;eC z)L(a=%$MZ<3PJi8#K6GP&eGYyfWg7zUub#L%zvU))S*h*Zk3@VWvAt36%M6o$7j@j z#VF9yQ`1XF#b}I9(vLFIC`roFOwUl$OGSm<5RX3q>P^eUhSFX=baiiU6)6-;=w&q# zr4nKPG6fQf3;?<+_m>s+f31I*_Fvw}{BPbs|7Dtift-+tf{NxpJ(~W@n}4i-dGz-% zg#Q)x-_iKrv40;8-2Vg1VDN7OV+=qyHU}61ZscUuZxGdgDf)xNVu4a3;R4@65JB7$ zG9!WGi3kVg%Em@x3tog22O|?Rq-W+-q-ZL}CX{PvrfVjqYQ(3dj4PJf0{{RETdVBE z)Wk#+o6GE+oa_V~6+@GqOr^w}448RCTT6>GGl1b~@%e?tLB6%o!CtwwA;7NC+R((x z>d3@s*W_$(_#fefgz`^x0!sKRw0r+EynpZOe`nsv(Ae6+-qHDQGQXrL6SKjA)cvl3 zG-hbpiyV%(Zlr{HLMt{)LKO?{2?Y{D8?`biT_~*FQQ?=fE5ahC)XcdBRw^IL_Hpz% zH9I3kO*3rIAXVbV=&Q@;^bgU%+oTiQOVr?`)tYT zkR3hp8f-1y&Eo|o#Es1R_Ui7?t9~o~;xN>^r&|}p%kwWs+9$^Y?lP_BbZ2S1=IPsMoRL+B$G7{n+hb$>T2SBy6kZ zY$(ISP1Hyj%=AuVykaB_3P#p?=_+KK}~t`WRvAbcihMaAuvbs(Z%uFN}vg(`YqmG4?GS#(c?Ds?-<`5+2xk zJ`xYY*(oz)L_m~{T$X0d^6%Ev&`pGV=Hll~m&>{vh(Mv(1;+wLG1$%7GpxqU=A$Hb zzig0u!77YK1(HM!*`&|{ym{V8e#Miv@1}5^JIn*aB6!j*VSDHilmO%^6o)!WsE;cj zVLLcEo-i!ojGg@=o7i>?aAMZ^pDQCtsTNC^sadr=C6SCkvD92nDmzJ!wx~~Z6=Y;l-XDnm( zHTKRX3)0V4>zz{zukhjPHLNankDu<$6$x52eC!o*!#3w~OBQRC6r)VzM|y~e~xF&=SUbpNw>Im_wl^p;(&%Q=SQv{qPp$O%DOB zRN>)Cqt}?YI+BLCXhYfMK1u+n$&xak0WP^=N0SXMR@mS;EJarAjUDmx*yyQX`u(pV z`jBzelkRj|d1ISUi+57!%%ho@XK4Rj3cs(iEy>w~qze??cpG@@V9Ms$Ao{dv$|n1i zo?L?y3|#c_n|0<<6U<+S&3+czTZkFsVphn%ezP{wWR9fsvpYq@xYpL4^~hOzOpzjs zv9rk5+y99PIwGI=pxA*!qhYiE1*X!Gz06)hD(TM>qSU zZSAfyMx1R~!I$KimeAWq?tjLZJUyf zUDmFMLy423I&iz0Q@F=Yo6|l2{cZB?@@ekKiOZ_nLMIQ&u+!v+A1ZiEbacODzDmP) zkQ;?3=*^=05+}UH*!Z5*u?ftR48>897R*J?s#E7ebf!YK`V5K(f99&}m$}M=W+6uN z{lVpR#9GJxy;g?<6IK)8k-EUfarrXGby418N+VhggO&DbmJmF;ctJe!+g;WA(fH zSXpd2s{Z2Y;X(w=*+@yPqa~&v9hwuxj4l9L|&iicCAs>m`l!wyc+8p1j;}FNd z^gA2HG`MR(V!6a(36+P)GgdU)s_f3P4ri!wjD|1T%I3g9?6%4+ECbP2&4k-!k4Tn< z!>C*!SCXk>N`f_3^lj|?nTi!x74DO;2f%PN3sof1^gQD#2j_Eh_0`KH(J33QIul&t zSL!ab&8E>NdnAzEWMZtyWBFw;xoA!i1*A)kNoPIbR!tyZ=QVOk{G-W`Cm{<>A!4$F z^n5FP1g3Y26GiEZX3xh4B|ZItdusFJBPJ#9;u0YI->XOUQ49H2fXk1fwm;2UkDBtO zLZAvTSjx+&x5?C*7~G{W5o&yU%_Y5aV)6W&qi+S3e<;~U5iqu$`KBaXjVW>=OUIRq zZS|AvTcon&2G>5O#=`D5UJfoUJbj%O1l;{xd?xfdCbXL>ukbZeUhpH_ho=G)8Y(=q-@w5uZsh+c2*medGx_iE$!P)4f+>R8k zj^EDBA=5-u4%IEEQ?v1un9uz>ijS{!HzmD@!;o5tJ@GUY&2}GV#X)g?FW~{kSPXoR zI|qVjlF|?C3;b!*&VMFRYs-`h-w5FZ+)adkQa7mqzyYMWRr1q4b)04q{QAlnBTrS* zKeEMR-FPU88$y|+oahx{8_5>$_^CuLgd(Q`2vhg9ieaok*+?17kG<8duJFVR`p6Y{({%c!hLaxxf!Fe? zt*%2+OqQFr;FE?tz)os*5VeT8)9l_kv*6`98gGtR++|`EwMP4<+Nl$+*YpWNzQ}fB zD5%KEd0f)j`GaU-TPN#!-YboN@L6pkITxt0{|BCCn=fCE=<7d7=(-W88dWJAh)RN3!9g0vZhGZm}xyd)#S&9 zLHGBUmtUp+h8b4cmE^EnsW20u=>@$soJ@{9W5`F(R^xe)+8Xb&fUv?547J*g>omWd zpcz+a5Er@c!c7lg6whsuRXT1&iNO)UG@Qpa=cf40{QE7skbREqhfysAl3H|^D_i9x zo#O_)HcnIv>NEE~7&I~ZFVzVKeONS#p-q>&*_Kg(A=A7x zTNXko2RDeCs`*COeoUnZ2?+167yhKevLaVWHOmC-f|n2Oltk76JfrI70i+qk^_?@=!d#ZXlwpo$jL-wzMn zLu5rGM%R4>yFDXh_P5-!P#y`1F-Tf~hJerzoV2>nweWJA@F-HDa}bHAvMM1>z>f`- zs*csyPG(R(=951|EvTYvt%xc!%upQu;4%&sV<@#r?9PM(iL@e#>#G$fPfEhJc(N4Q zz4^w|7#Do?KX-!_SKM*LO=$XApICO62OTyzs#%bih>~$al~5zR_I zvrn^KsbB7`?t4JSBc0uJ3|}cE!yUn;cy@Q)h}5GrGpkF5zz>ua8>8(OK+j>~cZ z-rUu|LHGGU$CCGj=)mSS#Go_kTq5CzhK@{oD3$eXhj?c{PCHESHd5`J1M$c=}L@f>Y$K8}> zVej)3V{pJ8^0uE+$AuB$9*ydThyqmf{U-_VQtI`MS(u;;WHzJ-;9yr@-9yXBOBWfz z5IxuSh*E9IXLc`VW!cs51d-&hJcV^lr!tUy;wp)R?~sDhARc(Nu0Ou=Q`n3l7KX7} z+mfZm`j+y+1`I20QCENTWn-z9kop_6kA=Y98bAz>vX!@kX5l1wIDATZ9|@xEWtfp0-4Sjop|ZLN&) z=SA8CC4S@Avvz3>vBKoOsIal5GdWu^2?Unv>58T{E8kRrKWlVtKwb*`(~l|MW{jL4jouc83BuqnQ)E z!gldopyX^&YGaujh*9d;f`fiD(hU(S*i+|+3Y{ znkLT0&9{WC6r}sN^d^%w9r@c;y1bAiE2r93;gOX57HW)IinFqb9b5^FRT*Jeqe=wr z5xpwhuRqBv#O=se9SUOoW@r1rD_#l^nGb{4r=j5Jjv)7Q>R6y}`L|r2;_^h$UN^Cu z&$r>6Q10S-*NN{XJ(SC?#wya7i=8MMzdA~pY5c<P(nAUV3n`)=o= zrxY@|nQ);&TlbopI^b=rPQ2k}_TyDakb`6{j4QpFYSzcV1-#ycMzK0s7QDwoiPW?` zd#?bDn!JF7UXiJb1=eY8t4diNHSI*6;)l8A>-Q+kE6(ig)e7y>t{=bT)X=ww0pSV* zZ+$kr94_>!M@o5pJLh#Hr#0Q*4lB@nn-cGvC60)Zm=sVYCiP!_n5`wvr92bn0eFbR zuJZ``xFe_)1_za%22g)+5lEiEx~vx^Fd?3~W=x+jJ=PjE0|mPlP4pR2YU7~IG$Yx@ zHIFTSm|Kf_x_H~!j%=|k*2IR@g;;Xi>1g*HTl!;DTzs&tt(R&i=5qUv}uC68;h~STp}am(OH60w2Ve{Zmk7B zDxLjw(0(&FXK@$hnGJ}@`7G)-!Lxhl>_bz@+~rk=r;}x84Sn!}fMB0L5m8}J=koEG zIVUrJp>(|I%R%^^6Fpf@GW#oggMOdHUi6;%HWaMM0B|mH~_Hvv{$fBd%U8dx8nhfQ-bwZVNH|7J~6mq|XGmnivO72e~@RSLMK8n&Mi|$nk+(=)mug=gC zQZA0P14EuQ?Bc8%fC^ts>wXdn{9x;}Rwdd;yF?aQlbG|!_2d5VkHPkDaU7F#L$BX} z&j32spssTpo^>~d8(sPw5+6X#wb#k?+`~~AVvD(kfZnjZg59w|)(2jWzMgKTZuU?L zCFVV~{|OD~{h#KE6+irt4~Ut;i91BBJKCS>r@DU#-^7U~yP!=M$OHG!e)J`f4a(Hv3G3TVYIrN>|q1?cr%7T?&xC zc#!M3&O9!Uy0)mO8FK!*3k*RM_aj(Og`IUX-^7wy&A49+aBEBtCTx4bSNHFXPr<8O(@nw7wlaKUpQ=yZnTiHlp}?z5I?Y zcBamYdLSDfaP4DK9$-o?e(o?vX-~CQoX|5JuJD@Pjnf7{LpmW}?@$+0Q&nHG`VMou z6i3zAm(}jtXR3L7O~%E6q*h3jo2jX(sot-ccro+!Xr4)90?+(~XdPegLFR0u&&H;Q zuv<*R*!Sj_*7bFDZ5$I!zl%yy$}tcf?9o`Ku%Gk-d|u8BHR}Uj5UqNbW+Fba+~Ta* zYX^2so+Mn&dC4HCKrfqwgX5w(T!nB@<2U_Vt!fdQJJvqGj*CC^W#`^5hKU{btpKP! zGTrdPau#;-6Xpl$MFK{2eo=*oKY=cRAJNfPD5m40K1N?=T!;ZB-UFs-EvaL1I zPb1}CwMtDd?9&YcY$0lgRFYxSKbPD`q>~gBrf1Xw+Obpn8!k0)Yk;5jE-SADF)#32 zigR3%72HT53QAr*#;e?I@iC1R%>8(m2-jykGTpr#KM9FU00XbD9}iDjvd|MowH_1^ zr5o$t{U0!?&nbIz4$c_((ytDlFwz7YFn&}&!WwY}#wDaUrV$TCEb})%g+@O+Q3S^9 zBXOm%2wyc(^V1omi)ozPIpC!AmGm6uah4Eu^S1t=*bnAc6PTR7Q+;|67jQe{jUUbY zjxivm((dELBCa_uxpxzoo=TK}tg6ERc1TL;t4nr4rpyZ+I62Q@JIQz<$c<(f+ zf^uweHUpR`Uc|9fcT@jOSj9}1f~3_D8zII@-Sbc2JcJ$)>Za3nuHZPbq*_#^sKpL9Li z)|&OX3wG~AUHXC%*Mm|%sf(w?7zOvj3}p5onA7JMb|d2bKhr2NLheZ^b~5r`uPAtp zl5FRjo35KEtn~L!Vq_g+6DPQH<)Tja!!L@D`1E-f!Kv_CAKi*!Uu z<HXyXP78qfbLJmHyDD*2l`asYoAa7;Ms4Ao z9n+orOWXE%uV?SheFOia!Rq;hS3rse1mvdi?;5Oxe=}J8fAk;!qnoV%7{dN@_G)Zz z>tOhIqs1omExQFKwC))dJQvPWRgy*GU>Z+m;vEp^0Kq*FxwwSP%5CKz4O#Ra%=-SA z`>WIT3odZK^%tkzy6^Y*YqRLyjC?Q>aPBRKnv>Y%pAe8%7u-23D}J;;>@ie_VF zz>{{kN8nI4>%8Oz1CM}}dQ*lk3jJMFK~b@FMP1OqaMh8^IUKYA@Ptd(21b+-T@&%N zZDCx~v?|n*#yjuIG{&fkVpuA6s+F^}fZxetrc6(fL>6hfCnzcld}reCsb(dyYpgEB zJwJ$`u9Pf!Bqzlf46X6=nMDJfus{Vk&uLffh;?gr9vxAEd~`$WJzP-9>AD_WUg!j3 zw|qrF9Uz3a>wsXRNOK@rXeSAcXB>Mv16v$+K~0utcJ#xvbV5I=3a6n*|Jo5k7+3R5 zX|NIoG(n$pcSUCwnbJ^H$9ZbJ%NcYvp~q`?KTZtBGDWRZT}>8*L6>#8898@YMh+9H zgtJ@Y7I-}-IhI;Ho)=l_pc0(OXCQTBBTP;KR9fA5q zC^_k#I+3+^YQyJ}xL_piXxMLZ#h@mBL%gQ$8k`BoYxHuJxzhymrEM63;PB(DNQX2g z=D<(63w(v*g8YLH9`UUSrs%^pZG5o`)kV1{`d7V1Uv57^I<*2}X>>PY5l8-7!$aT? z*M(^lMvp${@mHf)1yAQebDFy{sBe|rF$1fGK3c)ukww?-LZSMS^mj>r^$9mt`ae-k zHU`r_iqrCp*2_dzT%v|am|gmAoa^uww;fmVWbG_`=bj`@l`-^cEpV(HbY5E^J{dxD zR4_BcZCX?T+Duh_h;{;3!aa?@EqCNg+FJteBfJ?IB82%*d}9@;$P59N;Y9(FyV(~l zh_Z9RR`3z$Y^gnIjf8tru-JxfqW>oKfY`#X>mq)dP%qRn5aF*Z_p{F2!No$3| zBF>w-(pBn3<?2Q+6N*_GNeInK7QsxT-2~ zdMDjdC$HBNvZo$idN@j?}}F)@&nb7mucZ@#Ky0LT*%} zsk?WNba%oQbTPPq))=*BkY;r0{>;eKMl-a`$2!5JMJ}i$4|9I;X_sx7;`dbwm!CJm zhpSe)sR8NsG_WXt5|(g$-oPLk%fztoy{#d#%YJ#;&{ew04|uUwUa1<-af%9j^qz_D z^uLDx)(z?H-m#t;I}bRp5`rmKuA&m8kuj_V#yBGoksIH{K{JJOt6vB(lM!L`<_i2X z1ub0x?T$3aE#YJ0afn|Qs%hO8U$)k{Fe`p zAQRuiqAA)3{=n88D?UZ%H$D`F?3w?mwPQ@e6VlgEL=SW;!#e1wW#y6=FT^zxO9?wf zq<2~7d)3i>y&ceu@C^No?a=)=Y*-aipqYuPflx=BFdu44qtetQ^FF6G3?lOQrNdT% z5eS`-bb^G}IC0xCC_*Vo zI(!RDRGZvx2sN+H?&2!e3bce&;^%2}6FghE%z5>%4JZ_W);)`A5yGjDi|QLk#rKmb z(V0$)sBcAZ=r3at391#n@yvO+5gU#t4uR%y=(stXtK_u(;l^cAN66stu=_qIkFU5) z-q^8R1A~}@?-R-B_wz`^8RLiZlH~CRz7iBTxzt|M+327(i>5p^LwUk+#xEM*xsvO1 zAFqq~FT~M(=myArkHJ#`l`SU@OP+pNKwNexpTnVL`u9K2E~&f~1TXx>h4V=NjtlYr zh7124S^P)SA=ba>(8|!&(7@Q<#>Ujx+0x$5>2HVDw)PubC|_3weNH`+IxCXx@4HAg z7^7xe9S`wrzqY_JwuFmpdDLlT_?6`MXrFriOx}nU>f=$$Z_p`%irG4vab;rjXA&3j z8Z-L3_ff2|$=4vPZp>9+eh((gw`^Qynl7l3XfhnI$e8kmUN_J|?mcK9fHT#ue54Ot zlOZRNOe=FpwI&MJlNe*38%5JMHx&GMw_%GR*P+_Iq1Q3uR9JOat0mi4VK&`o&j`mh zAQ$UdIK?{-#GHk>)4xyb@?JA$e13Z7$pY;TRJsFwqMeColN>af^4-@UIjk$Bp1iZ^ z%eQEzQE&T2g29-qrC+OtGmt2jI2bbOLE;q|W13tEgG?PBy+_xS63TwU^paTxgYd+) zKS92lHxjznLmMYAI`gm!k>X7SLq^{pjX|-Z9Mh=#{ZJ<`h9RT>LYfzPV-L(zyod2} ztb3O~+6PgGR|DI3pINiSRHqxi4Qgwo?5oyR&Y;h^ITPSLe%&5F~0H}yg~ z*(~vAX3&UEus?|^KLn^uWyv>EpUxEf$ut`sJ2m2GS@(~DHGX>|%<~+5#5>*v^@&R@&hQ~t8yOdyg654)S(8QDfedIe5rjCq&rfz! z9&DF*f+9g}8f32L?76b@-ljEu5)#N65EYOI58RWA$rFu-6=Gy#wI7sgr0%4Zj?RU- zRH>R+BqmLHUhRhd^}KVB!UYmjM=b2QIBQMpL?X}LIHVU>qTuOpnCDQ;+z!B&V>1!SFX~zz7Ta7(~-=YRrfW?M1e3)Y7+JU~UnlC-~kMkpXp$ z8?Gds)XCT{iOTb2+ABrUKzNVeYy=s06*o%#APCP&x$hp=HK0!(-!*oGccb8pAO{Hi zA*~$)Dd$WNC(LUfU5zaX<_3NQp1i4)gFc3dzKM_a85n(TH(RsBzXMaxm znvei1l8o0@m;$-6fe4jI6a=7yVysEW*KM_kL-S{G|H;g?z{4nlp+8lNExZw0W7l9kwA_uIWKSXwO!1$ITj#U>R5; zqGffrZv%Nbn4mY!vk}4`%v`i)$Ce7?78X7%`P?5GTsO9&ge4nBHs}WpKN@%vnQ?%r z4{roGolE5rvlaxc9=cOPWyb^7Tmp9(&Tv`5u=gEw3`V9(h-SQgI*`qemPQsYC`AemV{x?nyNLobR1mch^3HI>+3h%j7sgWCpA&h%JSRW z>pcNL0c(6Nh9(R`H+E|F~B7PPkTuAYY4!D?^r+lb%j|bxw*t~kJzzjb!@K!tplRHk00&kB3YmjKay*O(D3}V)ie$!#v#gVhoFMq-ZTJlG&Z8(O;i&%x$FcV zfIZR&=Hhbki`}1k)sTqW?Uo|B$To;L{)-SMW=%j216fF+Xd1DO&lW9#U>nI4O>%@& zKs-B7_+mHkGgSFhm1e}yWth8t(h@N_Py0dH|ab##&KdEMuqyEWdJ^ zdsRq0A!<4sO*&0a+WR3n_NL1%#S}k2C<0$=sA^>E2=-WhQ?AvsR#O>OL1;}iHVS&O z5@ZfI9X<+D7z?5r!4L_k%r@5iy|Or6>-g}LhYQJJfnR%Xxy`NUx*yS*FpO4cFQ<1P zaBZsQaLU6PmT5F1;v#LTEorE^;BA24fek~d*efh;BlM+!n8hl88~g*wdn?ODIKjAK zLbo~EJVYR)2-K@J-Hvp2I*zfOl;FMLc{ff$9{fxJ9wI)z9S534l^d0NXkD4CGyRFz zmY>gDzeumnr0~4Jf8@3vt`=UAGBwvhsHHFtKH7cnj64%}{5DL4){|iEeATa(GnC0^ zby0YJduCPRLTJjxx{gwh??9;J^b9Rf1$HauOcISYnAcr&KRwjwXrC9e`nWe z{6vUsm2&cttFDiB*Hl#1WxdHnVLr#6VZ8ujB8qQ&s z;AgOlX*MB_92^Ac45+YFRlkVHz1_7>L4Rn2o*v3#VGT}R`3+~43}Frb;RLxE?{gLu zOv;cS24Wq(?njcO2^Au{=t$7?zsMC<5Ur&Z!@jm?fnf(YAazL z+^|AJfF;GBiQQqxmoJH@63T%&j3VZ^HPH9#X__$LQ z(Pe9}_N+DOCwV1}88c*?36F9yE+|_WTAz7KHc1%tA04*U_A`V&e6---{IRqRoljr- z1)JTO&8WhS@3(DY+w4bB=I<6e)vY<`QwFhTZ#~Tj1sHi}t0UU>(>)}Ms(gU8+VXm6 z_6oo48IEVSY?x3WXPzCvARJa^LFvaeDC^k2ojD$Gy^+GsFZ~1K0yP2IqXkU3De+; zn|r@fRn%X}jSdz*nIu$Pt{Fb>4v#cJqpmhO=e;uwFiHkeRBL8A>Kgy_)=-8p4B8bM z{Bbgvl88?IY1POd>&j#6XGe4+h|=4TG<&~6Axub#6r=N<9#GDz8t>rhj2!jViuIt` zkGW56?{_4gHqe=}`DG#By49O_wdrP}&G-YciXge;diBhndUV%yDV}QZKtLek|E?}2{hPY<|D$UAk1kLD`#kkO9W^&} zHgz-f_`A~7=iO^#*zx50jn?7B*I+bJ{35flWvR-sVl=W~FkV+mKK=X+lo%xuDFF=B zKYnTR-+Y}nj^7xQ^sQz!2sbqfm6gdJOcQamI@2@!GYS)woz^fTX`gB(b=C3YwlM@i; zwlNSnrVTC~)e&?u<3r_v2$d=1la6syGMN(|tg?3%#pyL0B>N~{x=zYd$4p&J>Edcx z7y>mBe6P$k)is8TXXZ-|O-rAR>7_8VCkj>4Y~cM1J>_E?{U286?46`gQ*Tyk8VUJx zm)(`>LyoDPyuRG?=+4BS2l}3~+r5#*IAv!k4}jFn9)AwC+YS;x9n1#4rE^AKX~Q!O ze{kj8o|N8_>6Dcrrna$+PTTXDural5v{@Y>fne|v7?_Wqst9(pPhcPew98%)UfPt` z&X1XVS1;$dnedt=S`}RdeZ401O5u3UkFCJcKipY99^XSP1?|;m#6EoV>mB93vtbeQ=tSG(LiJS*K=pwC$kU-V5Vox7D>2-rFfyvtTHo;Xm1-IVJ@gG6T zW3esFLYc#7K+YR4y_r-U)dqj=s*Pkvb@up=k4m7)?QN7D6HZ-bZiilkCo}-o7wikn zjOS|4HqlgxirQpouNsuH#MT34F8mt1&34B}H|Ofc``OZpLCkbR#Qyh>Ky3rw*JHm; zFOIjjo27_}i<`BE`?(=--YdS(55FRX9X|!Hx66r!dl~n~SAmbm8b5YKzs`@N3pOtg z-=}MVb@FZ|!I)@qM#hGT{e-zS+R~f78;ggjwFvV9CmRXrH1%i%gU*+;o0E|`xTY978xa58wYI`w;efC+OJO&0AY_U~+zK%} z|6~T$0{$+5aB8qCS=zcA5Kl1ZgO=pQ_m@G^N1O5ui&K+({}~plh-CZSD|ET*A-@S- z*F9Jd)sUjMvgDc6)fF_H(T#VjXxV7vKZedhpaX>9$1lf5xJHI*SfHc{wEO{)u4WHS z5s7Z7x1Bg@i#|F!U;_0#UrEV18mfmUAi&>cQU_uM>Xm^y2P})EM6AY_-*X7}vge!z zD)WuWMV?|i!$i&tp;>9*IjetLoodoxYEMYptJKzBt2QR8Ss2aCK7|xXFOF!q@P0Tm3YLv2@1VLXitk^(Z`Pec zHi!Dr3R)ZZ8sDpj4s9clT!y3@v{2DL&!}UvTt~`T&q5RtHyaWu`!uk7dkAUG@><>Yq4$e>rNT}`*$-?lrLfwS?+(z&b{JOAwE97(*kGR={|6r*xe7aGE;7AGn z<>3O(X3jYsZ*?&l+|HNy+Bh6?&Lu18V39<{EQ|EluXp~FF0(;GQ-)qMh-e#MHV;b` z$9+iFNCgpGG{@b#^paIb!N6SO`?WQb+ zqB#cHU+eFPC9i0ysbbsW!z{mIgApw_#SMRfs+6|n7-#0& z2Q>$8O*a@gRi&g;<)|=Ho|aCHt9UoDDUbOXnhOC8`~uyt;Lr-+OBI}dS%b8Xy7c86 zD3SRVg5*ybf(pv6PPcNjd0{?}xSjM~tQFd;caC}L!c!PfeBPW{!YFt`09dJtdpSwx zS{kSMbOxlmS@=U0u9@hQ$T;M_XPI5m($K?zP>Iuop+&b|$Lwly^ZelE=JiR=BFK)6 z@8M)@*hftiYlzS=@GylVNrRZ6UcCGqsfYCA{S^J9t*kfNy8|Hsq}OzJcatdvjYsB2 zmeMgG6$h>a6Vv^HuX#*-{-ZX#1mu);Fb~opHB-G*qR-Ek3xTU|9Xr!+O~E6p>h5WH zPlw?gr7bxeX7sIJ3EW#-??yotN=R&T{XYDuNoKx|_YSn?57W3XE^BO9ED1m_7lxYm zye!I^TC;T~0#7q*vgXO%n$L=JDxfTEO?ByYiBvO?gAOc=?n~aUe7xw3ncjL+_lQ@G z(Z4Ni!Ne2{I~~8*GO;-4)^Aa2@Novo&H23?l&zdz4=P>NXKfK&k9rv$X&K0^R^+M@ zfkfpg&a9wq>^!g?DFdoRY+-+_y&5=xaAl}-G^4+&t*By(@1s0I>lJ}#c8f!|ApJRZkYsOd_*%sZNHE#ckLhpHw}hY%7bD4z&&|W!`dQO zNL+}p*_{v6+Ttqb!De?zys5hzIJ$@)AXocTzoTU&+9$y@;nvB*Iphs~yaT5w!lw12 zs)@Nt`|jOOtX!75vKw}2oM)uPq1AQhsY7drKADz%FnQX)q-6FFZd zvzN)VFjwad?5s&G8jodjvW+0Os32ySes@1`x4Z48l$%!6at@1! zzP4ChYv;d}RRQRoJ1*^^Yh`~_N|{#wCa0de!NKwp0t;rs2+?FkPf`>iZqEvWIJ@l& z61Vz*M9@a&6%VKu3Nxq7!^z6&tO7t*25gQd*%uV)=`CC+(Zi9dM+C4Csk)YYS_1V&QMyveFg<9{11KirYY%~4;XPy0hw>p{xY9Hv`Px8DNz237 zk#gRHrynXj>aZv@e)X9t$lZl5);f;`So6-927D4ADj^SfAtn9N491G^hcT5M2P+B{ zhW*K*1pngphrrh_)yK(|Jf0P ziT<|mer1$qFK91A6xU!jr-j|TY`Cd@N>bFHQA?`iJiBX+kRf`ym52)A2B*;D$TB<) zi;g(#rl!(dvAE(?-~!woNX2>7v@R|KLi^?PT^^2BMfpFC1hqvymiL zm7ydaHiu9XQ9?ZeGD1P;#b}gfoZM=oX~^NCXHn5l9>!w~^90$7KsT;2kupAA1?6~d zoR@G`+poE6{JEbBt-r&;tHglVAjkA&9%1GMUCu%bWpG?LL9vYW=$Q)2U$ObbjX~7|bWE?Ip8Kwn4kwtwzZ_rYJB1Ju5@6x|9;koG3#X9PhR9 zd_S1%hwhNht0AlvT9kQC zPm>cg@nMT*u@j)eZ5>a<%8_8m`+MB4`)9Rru(SiZ%Y=Ap^@LIh8T2wFoShI1J#Kr`NXMJ$2J*E z(<`<^UEHa}v^3##V%oG6c!O6=n%CmPjUyl7%`!;%(I_5e&>h960vd(yfL#aoTr!*6 z7kq?~9+KBL_lG^LkHBC9KEkYag*dpn5(ray0%zQpGUdSZ@SqxH(#fn6WiV8FJ z33Fo_1x`yi5tlYc)h?N=6WvjCjUTYM7+-v%Bd;RblZBgDWM2-B7lZ-Yig!ynJT&>fHxV5#b;^%LBh19O~mN4T8QnO718^to1hvlD684jm7K(I@Lh zin;5ezTEq498HEBkWq*Hfx!Z$D-iGOAx`HBe3;V zXWkVlR^TN7sV38ol4;U#vVSrJPn2c<;wARuCdn#(Edy*g&{8UaFKGju4fXVXm0OX3 z*&KQrgfN{8sd-pB6U-5;m^LuUE{ZXC$@%J@I`!`4`!J%;)e~1AL3e=YC9fo;Ld+6% z6*iwo{6#uy&(;q^sHPPYam0zeJ{6_6VEypuK>33|YpU<>vTV#Z$>dSIlF^qNkoads z#)>YK>2l!Li?)I=J=xc}YeJ$d#CSS;Zb{lp)d5;2f67b)?1S}+8^qCGmyD1M#7BdS zz3dEIpiWnIst~QX`4E*V#2S23txY zi=w!F5-K36vNp<8g{I8;ChcQ84v8Em;EW3$WB7!V*BvHgZ&R{93x|E8c{y`9xp<))BtUm2P|XaLz+FSWICD z!>p!yJyUE-jV48>7Se4F)GHx=xIVd%LcWT2TR$FFCcCaVEAVf&GUHJ<=TKQwn@vLi zhX8{)HyzU=q`f@9KB{3!`8deno(ga3`8=XH!@ch*qZQ#%9ePq2(sMDHcOdb8G?7#$ z**tXEM9G}V?mX;S1w8P$`^@JC8P=`f11FC6kw-DRza4q?=i88h&QUSR?RCqNZ^aA3 zs+o88@$q-Z#gq4nj8d)dP2vpcvTY=drW}(Tkxf3Z#%`LKos+T&FWpRXE|bAB`%iQpk*%Q;DhsgZBt&tvRBaTg<&ZaX8pnE>SIpSG)TYPFe}d zOki3_P?}SZnq~&0Ukodp{56tKVuUeQP&9*3{Uuh-?r8RURGcDttY&mV=V33>s(2+jm9T6g0*?Doj#TcHQxThWOxiC)JtSBfygFfh1gsF>gNY{^9E)RudI|GM3{F9?CNznt>!^;E!a}`rkiQwE-Uw~?t>Wh3-ezcZCACqqPhr;mah`Og; zan{xCnUU>B1;c|`M=Lzvgu&{QaLY}?CuhJPA9?_3?b~&?Dmuu7!nHb~F|yuIX5nvc z4rsrB9`oC+PdP<#{a8N|6*nzduE=OU^33mnjNkTqtk=lO=gnAx325WF zMWL}q_ZC33e%vu5WOY5pioDL8WwhW0ehYL$c zUYM3>a+EZPibXN*leHW|yE8A(HQdc=o4SZS)^N;LFgTD!K9IkA&g!7|lNBB!;^fB5 z&)5DFX_1j6O*)1mQkVh6W0otU1wc5CrOw+e0CHCOy&(6VY8pHwFFgh2l9TNr5SpEq z)QEOco8GH#}aqNz|rsdXXqaf*V+)|Yn7kRv&F1@0+Otrd#t723? zlo7$Xt6e0{rwg{8ms~f^hy@xQl`|)RkVhXnQeyTC;>(L9%r0TkuFFTTO`F&C;>9V( zuGZUL4t1<}9-udkwFSNg10N`)^I`>yQ`e%Jl1>X|moahokWy&bGYBoMTA$naut722 z^>@M;$9JMI8AY8D8*2JYqcwgU_;CytP?B1pezyp0V^I&8*k&Z?dA@9QMex5 zYq9^VQ{GLQ>|wgMEDn#<`!cyQGszlr;Yym7Ce!89=h@@87m7x#^=VYpvIODgNI=R* z!E_82i`=GGH4TmZ>}T2R_ovw!Nl=m9ecpGFdg;L=d9*I3j;w1zA#0v zZ#_KQQ&lm#XyqpVv@`yucUc2Eeo(~NDWomhc^OaqQB?7;F58~$S-P*0t{eE!x7;DD zDxBcr_dZyV)8e!DNt(5yf24-u#vP>=joC2^cJ0x`8lyR;UH_#%M<^BVLIB3CgLvx_ zglqnofnmmOz2NTY<#v_cZHv!+!#24!!~_9P7=G|*3> zZN8Q|K9S{7bJxsTfnuCYm?DMAdc#L=iNIpT!QB|Ey z62n@fdI9ZNONs1EBWZeq$pailrqI$`)j|#Ydfv1`8nv^^%gSc^#Jg5^y?1-*{33!# za}TO>@Uj0dY+WLkhvK}O!*E<=olF7-wlI%ADNaUh${hzE%C1g~x3&MJLG*6PAT#%Ka!SW5ct91rdo28VjnQTY-51XSKOs^4+dna2y~ z%ytu+7%ws~5beKI{>=g0LGb8sd}b>3qb8d9T>-?MXo;{iPO*(6puu!P@@cEX)9lp@ zn$13XZwQPr5LD-Tg&WdQ-kRAYrGp^ws?5R`IKcL!THXbjaOOk zv2azY-0=bdaHjr{L|y}2RK|McR7A^0CH9S5Q>JGxX%i@pk+#DMIoYG}N*VDjD!f)S zV06B)IjW-QHegyqF``v<7A;26_~O_+ToFu)5K94^h1|9=cs0#JsTE~V+^PURqLB0z zT=ijURdLAS03r3QKvN?DhKC`PKseJvL~~z{dd^#DLbvXFQl=^0y?za&8E)4Rx=%cJ z@wW5b(3nS|Ag@*MNeO9(s8OrwLIfmLMq_YL*+mnT9Q7E8&BMf>2=*DiWU(cfMmr{f zt5v<*Cmsxm?qNC2YTLX%>&N#GyF~%S?4%vXEYNW2wbF0TJ~H0+1iFMXU@?9z}aE_ zpR6p*Ad@DfG5oqyIH+S7FTw2@V%gS>u7QTM)#(GFS@&SW*(*70G42FVzqw3EWO0(< z9%4pRqsl4WuO*b3ZqGsH;!P!=F^M|W7eZ+cm<4*>zi zkwO86Kfry@S`2iaOoLiqJw@R%>Q@IE(ed8WykHfDO$hVfwjbvwR;U2dl)LJ-q8H1X zKiWymGGCKsbF+y~oyn{n+{oS5gZ|LoS~CzLkc*%qaMtYYNkL)4j;s`Wpm&07Tp?}- z8@V+YQC^Y@Ht*gI)o|_TxS1_aSe_1b#Dnyxp^R|#P zsFG=3t-zUag{L8{|Rz>s-dlSI(&;+dTpODtF~$| zAnYn)Rs!7LHRtcKfoGb$#-s|z#lW;LTs0_4ja|24G6gpAhs9daiBmia(I!D(B%H~M z^OzqQyLn@JBzsM5>`q|AH;OdQ!K=nb>TWk)GFwld%l8X2xV3#%qe7awj8dU;X7#_D zNHgN;_EPJ|BxPxv)16~Zusf#~RqX>hTaLjB-eq2zHL8jS-3u5zf|%4#4(g@?a9QY5 zzFAk1v*-xD>t4c{F?Tr6BsIf#1g2rA|3)1NRriv{Q*T-k`+j6cH!Tt_9V1xk1$k?s zQd19?C~C57oKGH7-HBk{^!0hMz`-h`-)6!-%G;y>OBSCWd2`$3dPHoAX z4|}Ws-H*K4z+Bm0Ox#9=rcPjq-lkQ**++z?`^@GHP7R#BSDVND!P=m(>J0^#VNKQC ztzkfPiF>B^UK>FTBVCg5b}V)u=iD51lZSXY2G7x}hQPDr_rqChIbI-x=Uh}QyN^3$ zBDkgT+~vJlcV6d+-9E&-qe>5>UDD@bDI7I3c`LRmTOosj#l!=jaK=YBHQCOK!rEcT z=D8_vvEf(Fr?;vG&!m@B_y>=j&Lk|U+WUr-3H=%ia|p&1-mo83x4HBp9|H{ZgGU+6 z^bKD{Ia=b#FptpQ%R0vXN+x7Z`D1st_moVABqq-yYr6MXsqVG=~w2H0sg=PnK0^l;;QR@l-#_}OZHYw5-C49zd}O& zs2vW-3{6LIreY)t(rJBb-rz>&Lp)5~f~j&

    =vSC_}uBpBv(c6s8XJM{5qj)EZIJ z<5(LT&_HGtF|y4S1P8@0INWRy4mE32q`boZwRXkbg%{0Ihp!+d8JB^A+myAuhA*b{ z>yUEUYl(LMx;k@if>vE{u}Zs(p#;}3&oAZ5UhJQq$GJx7zK~d9ym_+K_$(C<3#%@5 zXbo-MArGc5Ph&i?`G%C}Ki$=CJFHfwhsC;oIAs1Q&& zLjwYISPVeczOuC-0^RJ?TI|>K{MtZqOkrrzqB~b>agIgiY|AB$q^mNhrb!4JT~HD! z^T?-c1;Mc@Mj9i#Ys=MiYu(Kw@xg6P*5_{EMA4u2FFW^~87LqYB;NFAOL=~9syCdD zk}Gk)^8@v?@Zj1kZd!_&y2#rzwO7307Kim%8(BAIW?Cr1<|{A+<74c_9IM}-exYO` z1v~@z%NtjJ8x9F#E%8BDPo@sA^vh19o1DExAPBazT8CMwXBG0@9nhCpQUO$9t(nNR>1jzY&my0WJGT(x2)6RV|s^M^`+tr6?HW5Gx`Kw1c0HRV3n^FPoyH*2zdkoTab9 zW3qH_-^tysm;lb}(N|L|-EhH-9)?FNZb+u?(I9wT44>}(Am|;}n9jw&WHFU-MjG29 zK5SC8uj}3lNM|i6+d-{iO|c%wgULgZu!1N2L?X8@rIh>%Qd^Iq>fo6&7aKfxO{-l>Fnq2V)Pl=N(nakEri z=LNt~0z&(Ia&#l&rTe#ez5Q*0KYL)O$8Tr1!wQL_@8{Z6V#6)JW2ewDH#eGS_&PH; zD;=e7F1$$@Oa~v90-2TH;#%&yc{X$9W<)*SUI2VM1zrH1guW1(fc;;@Yy`FWxpcLq z9*$}VK4#;fR;*i4yL&sC?2N}JjR8r;H1kQ|G=;>2r!jPtI#_|0jEQk3HKQ8_uF+ja zxZCdzHJN+${nu``?SH9~{o9_riECES-!?&-(4M@WRO++>!nC``NF`a1hDcBOBrBmD z$dwF!8ALPk^ailu6ZOmw^^miQeCg$WcK%E)=|ON-+OG-^JM)L?0Vv}EWkQs2$Nm%e z!>{#(QZNi~vB!LW3qnbCr*N}`PTa}v!`#)JItxdkc*n@YVx!PS@)h@9)r*PuG^D!E zzdscm1y+kTgfA-73yQLb;gw-1_Cdh{Dm4+NLyx&=$2<|+>lnsmz_=l|QEy)qzejQ7 zc9h<%n0s04UlhJIvGs74MUQWU>KwbdZ~>rA$dVssrY-{mv#8bn*ES;%JxA~Uda?$I zLqo}=)Ue!dS65+6(qOCgM)Jk|jRjt|cH6V~KT2~&MOqAYOwSd_S$4cv;Vym*A=na9 zVkD?e_h-LSE|&grlEj?~E-$6&=s6e^IaJqtf(bPsDG-s?)S2;3LEqf(m%G|`Slb)4 zowyxEn>XSqB{+9(=S4>lk@}Lx(ZMen#mY(bWhKyZleCYR%V6+LUd^x8%4Are(Ji;e znMc`PK0YTlyFZdW%N-?d`ufI+!boO%*b#;XP4_iHIgoFekmkYM(H23hNK$x^(@?C4 zQC9N=-8+7tk21*u$l2`94)Y^W5)epdJZ~I&>}=zuj~5v16|G#rtY5K@{bx}%K4t)K7qGK;f! zfa6L&{*WBJ3#X~UD$0^=>#f`QNZ5}vNf@5nm|IoQIK%6=Gjr$9ukmyp zO)PA|EUDuh88&BBm}b3Fk;;2pSnUtK!zT6<{@kexSpsw6=;SU2Ebhhv0!9tJCUPcn z&QoLLl{di%R`#lL5VA=8l*Q#}121DZS*`-0Wz}USKKd*>Txs70h+q9a1p#+12^jaW zk?Cg|5Tyg{!gVt7+Yr7N1*L=qSg*=WsEtQ=WoY5|1-E5F#iW*uK5ZfxtBsaQ;(66Cw1Dmfs z`qd#o?%kHnnQ=}&DwgJ>w!4!$HB)cPqup{m%Sv1mg~Y1aEdM0MisQ*x`B`b3$L4a` zR(=;DIFA U%{|V;-kAC|~ddl;nn@!THBb6^w>=N!StKMRJq%rNcwl+8U(3b%>Mjnp_N4h|xbQg{ zCt%Hp=SnaWw})a_CpeKY4xvV;WopA3%P&yC+|gS+*SKp-ZNKV|d_K7crI;7&pbXEB ze3Ms(p+W_eHA>(Y=v}4W);zFF(C3P=V5_3SXEQqurGK{+e=P8%1o|(ZpybSN3BI)_ zp&g`|o_TZsgs%nX`uG9SdW<*2fPk%tKVOU*BK>G!Y+V-<7tzBsXoxl_?loGE?FeA5 zpnf%m|2YpAYf+~y$G2xMiE%sX3WrYMa?p`MzLi-}fXZBA4bbIJyOCQXMttrs zWPFVfVGMI=7)&3stXv*Kr$Nj-mMGM*856sJTXH*LfFkNv#bub&y7@uM>@s~9DmK+d zh(5ifh~>VaQiKFvs$)l%1AT)$oLEdo6dP?=N94q(8qZ3w$Xb@kq)L!hyURibDq{)~ z*ll)V1ff*HXYSm#PJFJFRK4^J!Pxc9%Y|PSB=_0kF7C4Vp7S@<(@#g7j8dbeEF!X| z*ZiBe*R+cDCkzx^6J#lr*QBf861NOTeGyMEhMFx1_CPhQ4h zRyzUV|Ca(2?;>*Vw@@GeIJWfnpmYZcWHQimweTL+yJ*=c^+#of+ zq7ZBnihL`u4-;YX#4d(5FH=n{q$el>6P=yA@u!iYgK z5j*m7iR8X==J2$`!QSHp8Nw1H@=3rYg=io~HPqJ^eYG6{7E)OZYAhZFxA?bS`;IKG zErNISq+fK)SHDY(4Z1fx^_uxNZxq%;XXAR);X_Pe#(Vl=GLNs7CZQXpP|SO&Y9}ej zlLKkz_R{cX9z=wz?a*Bf(z*bTz-_o3zj)M26=nbBW{&v&8qFrJY{r~#&zxXpB6-o2 z-K?wD$hMk&8sK9-#p|}xCmT)4P;T{rJtQGTzp?!&a-VhU>(x1JpRjrR`r~8KeaVqx zD`BQ&%KhOD*W}ULo0PFph>VH$#k-k>XUo>za(mw=9sAs%ysB;^MsE5X>uNBmPSkVr zU>^t;R%e+yDEQw<_m$Sg+u}D8vrH@XXK?51u4h+pchbdg952r5Y!t(DGa4ks)2ql| z){sbO*1vkqf&y_^s)UmxW0wQ!wScO3kbr;yLwO3~<(Iq5QM?v6~nGEwM+x z^3<$3*_`_$fP+<*I$J2ukyDZw)Utt9fR_XoQ`*NxiE5WtUduK|)}XR7sWZINQO{i} z0h~rAwlFM4kSCUC2ZzrdhdHcZq=lu8iFGlpY*(nk zZS@JHW{N9AF#WZgtdVXBa#y^0`p%k-iAH4+ip*TVLU#yr*9}igBJ53!Qy!G6-)!Lc zQbbb(+=BDfNu4zg<0xa{DDtudl#NyDFraDw;v(?*`oFp0PEfw+kO;i*=i85=tC9V4 zr=(N257Me2Tdvc6A0WeZFSr`n;5eCH>D)#zqO2ia4KH z81J3Zqt@p9KJGVwd191XI!x6vTU;$UhO0ikJ3?~_s$g_T0%6oFnWCvl2((!2)eX@1J&Yp85|8vVoAPc4Vj z@IG8*fXs;t3Eug2*yxospFJv4PX`MuIw-5Npz7C6d@E+nQ0wf-pHtVJ@7MK=U+s$* zUbgR#X=Ml-uZz`wVsC9Gl9^u(qeVCj6yP_0)naap_lpfXW*m{-7L9(E*IH@@Cf-Mut6Y?G8t#uzhYLVr(?7X@l>cja8H@?$D-B4=@xzT&Z zEoqn*PTNq|<%%>aF0E!Yne($4Q6xNcYDqIO{L}{4!Pej2%hQDT)Sc8@(`}2Y&VCOl z;f*E3l#dy(Q{<6^s$ojfzu^~dJT&vdGkKd*eJYs0T6K8O22t$wKv>5vZnyPUFcI!{ zH%EDLD1OHKa~LR&mPYmq4(vo;YaCuO92cr6n7Fw_()?rGGDxj#H^$` zU-$DDwPp1*?Zd7$LT6Lt6c1p;NCP^Rkhvav)-X^vis@Y1sYHm9Lm{PG&Py(km((|s zjagPi~Z{oERm=sCJoJO4@1fvBV> z8C|rl;$yH?mr6y_ou~o9=@$zTNgM6_S+Lw!w+PEc`a_+@CZf}57Y#91uKFr_$=*>! zm3*ohCM)bL!qGM8PSoUqG?mR-bhf;D$x&~D3M6tDfak10_T8V!M^^`lS4*|(8cIZ> zSBgjCGup6f2d*`D5T+ZOp~xOKRz%>+ba-BTA8^}aVP2Qljr9pU%1fYtzT=%CdS_nU z%IStjzech6*Szofl-cTfbSL~7GYqDRPg(C@y#g$}HZ0(}#p?QdWr^+m`wqc?(Je@P z|291xeaY3-dWnz90NozfW>=sgKUCZOSHz6sNov_enFNp~Wl4NQ%OUa!XRE(TxQ-E6 zV~UlUzmhNrrg2PYFDY5iUIi)5sO?|qtCdKbc;w+U*!fFIHF9ee-BI!9`PuSZWmz-Y zUn{j={5G++t2QYL!LS2oC&I%>`%9rjoZ@V)P>2i-0hPJoVzgW>5f_o!yYA$jK8C~+ zuaOy)9FIvF=w=o?J_qVmMWp~zW&lAnlpP*5Mrfgw=3ceth9Y=SCg82vU3%L%t=^2{|ae4kriTd#as8+k=`g5f930>{K_W*1S6Jeuo!3AS>~9c+6&x z-OJ7Q@j^iSMtROZjtY!=l#EV-@_W~|!i)^dxthSB-txS6RZ#vnD>kJi&9v8OCZ9Iv zxfI}<0Sb9cQq{F{|5t&WZ32O|kuLa(FHBk4JbAS8I)`DOkZjQ*amAPy*P>XJ-W~6V zDoKL(`9wsp51N!rELDWKzH(P(HRvg94EN3CsOA)S6ZI2&VqPaOQ1j z+&2>TjQYod_fwL+>mgAlm2>Z7nIWj8m*S#@iJ40~wI$4cCbT+dU>LE{mQs9Cd7BTU zju#oSS^9nU`!XYmRjk?Bf()>Z5MHR{wIq}i`%eLrl+J9VtD-s=N(<$W|8PC4Aa_1i z#hriXbn(?n^uH_7LUMUCuYo{Lyzs#TrFFe~ab*+0A+2>J*0*a>2@6iFR|f26)J)^j zXQ#^}E(EB;VT4SNCl<6`z@o|@W_F9DkJcNdbXlsnLdjJwxgw0}gbJ%%mwD%;JE9G% zr{0^ZOBh6Z8P|(+TP&e_FXG)jf!SMb&C)u#o`rYy3$Zu_R`LCXMAyGk;MP0O5Wvkj zw=5tmolWmr{2S0~?ARXfYd~Ns{2a7BF@US|I)1E^Z8B$9MI#w8Hk#@O7x=Dv$O@mM zrshxNyHC>a=dS^Kazvd(pZ5E0Z*^}?8@_VqmTO7>B`8%fEZ!b&v5CXiM@FuXi_L`v z6W7mbCy7VXg(&3pKn8}1I4e4Qr0WyYENav<>&D8$Vtcfr2yFwi3uKw-K(sNY5Jh+O zNkqiDiF+}k;QqOQyk)p=VyDh?n)`3_3V~4aDUQd3ctyH-Fc=MhwnN|_6!6V4zTnVJ zv>M^+h8sbFPK|I`HWnWVNALH0fQhTK>-*Q)=c1&2<6mJ11CCf5E$_QeeK$N*m(7}j zI@+c=3hGSlM^nm)267PEvt}AxZYCeba;f-44^a({4ieIgJehzq=)!IJj9GxDL@QJv64ukAS#y)X~&Wf=52m!j_&HT*rK^CDV~?(l{3EVeDq<5g0$a{Zl}nyERvVFhL6^{8{1ctZg@cT4jXhQPgs`hch?0^~QJ(V=q1HEhc&>&x%9L zoePfzjrbRe^U8+fFL#I$FQfIK& zID(?6EL$*j>Fy~t1P^A*px+gdQyLx|7c>7Ao zlSwkORF=wlT-CUFYW&0kEwJxJfTduk$!gUeyhQ@i%B3tKVGXpeZn#}4;~FB3%c_n) zH_zAX2vCzXf;vM0oubfhZNTasoyIH8V<*g^b7Z?QF`qF$vsH&``3Kq(pW$h-Imqg)rzO8Lzz&4RNP+_+YrkR7+jv7Ez1j&)F*!(jgb)m*I?mC|A zuU@oSq;M-~6Qs`GRKrqhW|S;1q|aHkiqH_|oDXe1q(G-Ej~#mi&Qc6I|1rA|CY~~I zb92r&D<3!COC!uETVe8{&G#Oj?YxM0pRz=!r2O`<(}*9R#7X4ms}0F2QFi z_>s=_G#4TwS6`XrLL_yz#VB=Z^%ICCb2dsti)&61?sA0J-!~;v9r$qpBafs`kzU*| zoW;vC-bfB3Rar0e?qY|GVvB}~JK?HWsknb{2mA;A?F!QJ)m|S#bmdQs zeS=hGOqa$!9D|7hsY&3pr|`E=y*=in`Ok=h_93mwt+(GW3seIi^5TX*F3#jGXxiUD zJKsLp04z27x2$3JS@+MjfNM(?Uf6|zs_W$h+anxr;WO&XAVW@h49;q1lPBQn{m^Q1 zoY|xpYIL}Zmx9%*XfWVT*rYa*h70-|C#)IFr}65V#sJfNM-?mQnjCnd;FLQ_QiZxG zOBMN4Suwb(2L)0e$3LY$*Wtgdxj0#8-;rp7KYcTSAwtWOWfjiZin-wJgLb>tQ+ z4wO1qS6geYA06MfdRB1`jzT$W>Cj4AmYuNKsF42F+L;oc@vw#paBK&(!S zfW?i|@e2HC>ZAaVIITEiF|N^&jyef57J$uvE$;qqK1V+OBWd82F@Y>ixHNr zIv~O0hG>3^)i*E{9#~ULfjFkpV(X%f-b|t%s!r4dGIPU4o2@Aof>whm&LoS%)6tOS zSSqq$iw5b6&G2ySa82%h^Q!J^k#wxVF$Scat&`{t(aECPb6NNmC)Zb7iN816Q)0v3 z5f&!&Y5Mg=b=r;&4Aob-HzyzXsi*RT2!r9l)e+!(yvkt+q4)U@8-I{XTb;t)px*k( zn~J1vFc7Um4vBvV-N1HlTR$WyyKVd-$a@~tV7!#Graklr-$+JP8(W#VT;i1{U>$6n zzq$o&kq%VG^ufkP8uAGiHMqeO$E3hrnw7~>?dlgvPJW!khE-*IjcnbhWEF8~(hjxV zHYvNO2PJ)nq~^m2W?r$ZQS9LVI81CMx?-kfvL%F9!QEZRg;K;cV6Po(ZENdkdMZV) zysc~2sPSeLC;K0QZGj3aF3UXBy8QDOLIUfY(gt3)&mnHkLc%Psgug58*bhC!lg>G+{I?repC?57mSJJ5uZr6E zn#rn5i|$DJI0iFHZdlqsf}@`UHsC8#d6*n(#Q1i>5EbC@PA7m{vh4YFtnN*D+uf$a zCz+M1iz?E{7leNuw@n%);3mZnDz33$-QTafp!iRS+Yy;#ZQiwA2{aE+OF6LT0hGWnqySLrg3cNy|L$U z%&qRLD;>`p77H4ILVk$Vy?$E8nHanWp2i-f=};RO%wll<)F7ovEwTpo(unzq9{YK)0>u0rlZdk)2gAWP=_fs z=LdC@P5$azJc7u^=oTZC&Ng~QAAD_nydLskW1-r&qM-6jek+oL?El=%jK7s9QfB#t z;LNSX7`7{vNjov#Qc@JiSey_PS+lHZCxuWjS-C8*Ma_rrs`4 zlo7hPG~E4*;qDgkVl29T*LWOgnqbzGnyqNonu9AMD(kr6ARGj5bKK0eO-yv#ix=kY zHZNZI(x0K&usG`zZNJR~z8g;icsQ4D0(J&sqFMHW7wQOR=EES`9|Znbv5AHKC=5M3 ze4jM!Zd^P(KOgVnLS^^RXy1Hj)Q6rZD$dnRv}}lyVDgaNq`;oA3959|a?#7SvMv$c ziNrgJ8$zf-iz-cvmlZIE(B4YPxz1qBN{)_ln;Julakc6pm37Y7TTlHfpw_KN5kQ#m zuXyXD&e#1Wv(uueC2Pj;bk}B(fZ~p zmcl~;c(n4$F#VvmB7P$Hp{&^68WRPPk^l^~nSu~cC>D)(R7{v@*HSY+p@GqO`DG1m zmtwitfaDybc&Y>CK3!;F+p6b;M2}c329uTMo+Hb?>>BfwJuv79SL|T-DpjVGHY-0o zYiNPie^KBhd9gS<2aAYpb<|^Q6ZP3Ax&%HxiU2;pfDj6bpD1MinqKhGbDf}=b!OZQ zOQQ%syhA8bR^4CLxz4U=e_o%-^gj59+u11DkOCa{=Uj?O=;`&~>Yx7M;NIKW-B6A? z6BVS@c+r$E*jjRoh9rBC5)ysM_muuqJV1}zc9nV`WfJh04?bybn_Q0_%|3E@)-fxD zW7w`7KDIL|pTjO-ftayr3@MYRqG3@`viwX*X_@VvKc~&Cp~0M7)a=>wDIA+cv zaaRwGs+72smiej+=oMk{)ib49?PqL6h0+VC3##vtLx}>XdI;SipDzg&r2a3Fmv%d| z>5$v+rcAxKK!sVEO#b|YnyybI%YyFx=FD^&TVXyd2Df=cgIw4FuTquDVi7#za)u<8 z5<=g`J8;s_8XchqHuo3v)0q&pWg3S(@xEuN2 zcN6d;wSC`Qw!V;ICi9o?Qw!nazDgq|BQ)6)4=yx|^Q<4~;l5Y%_*r zjBL%8^eqZFsHgOMA%A~_l7U8NPCYTaBeApn0T6G$uaz@UIOhL?RSD|4pdp%hZoC^m7*OPr0GX#{fdO5 zk=ulPnEm$n9E60onWCX(e&WzKCykdW=oz>KC#|`+`I=3&r9CqSt`v)MQXJ(mInTo?@UJ3=<`zFeJu;y}B| zZIQC)E5a=i+V4>o$tpI$Ty2KWP@g?aOpcvnMXzdBsOV5gD>p54T)JxN=>6)C6FY7u zqm?ZZSV`+IPEQmw6If*obl51&%MD2hQ0%5AvucH6&sM4j-9U1a)L2z@WYKd^MFkhu z(Kjx}HCDx?++E*A!wbQxk6foOFfVCGnShT}wTsD8%7`K!mXGa^I}I%6)tQuz)M>1^ zEp^-xXqQ+xRb?PC*Lo}0WDK7Q%-fa_O1AK$@ad0pfd`|-oCTr)BCu?Q#1TKl0q}WBt$6Shjlg+a^*KI2YwG7Z zGQf!7yt^;#ETQL+TOzk`gff&NiYUSD$#D!eati6d7`^eKBVkE~fT)Uf%@q{ffixJ| z(M4knA9aF%^_LoV`0nS8uh6q~yh|BAR;#XGVbBJwy39a7-JY|y`RDMKd zST&P^p5%WVTS$9_5iIOU%oIGjA?BL-uQQ@MFo=RmMHdXpDLFf;5Bw%_P+lg_{hM>Y zJ;i;=y%R>K8=er3_ecV-df*QUjfn@g2RCM$N()c3h??e1^DaCF&41Q4zho!@v!>P7 zH1@L`HI4G|@&@YZ>;4Vce?W>p0Xqrb@UT`8j!4Z$f9NfpsPh=Mb0sh>JGv>gZ@@-Q$8aOItr%l%QzHAinI@?A1wJHlbJ7JFmc`zZ>9|^rhexxS?c~B_Zbjds%yB zQFr2w!q8!=%W8(R2yc(fpMKqBe`P;SJgEl&Y#L(i3g0jk6g@mNJ>--rIa4hPE7Met z>-68)t~}{6{y8QgfbMZ-GqGUsB#hQh*xjk8hMqhb2a2l}W|Pu>>_CHEkTPiVU~j!R zX2Xeg82h_a4bgjx{0g!QrYnZ9$o`524V-W1;{TF4Wp!LnKyeuO6_#*7xz;2ujvTHS zBBsxSpx2D_r_52=_C6E#7irvHXkan~e_(-QLWOE7(L&tykF5I#^3_Rtc`*HD&gw9vx#yPd@qWhx}Qn#N~_@)I{_4G#r#n_~Z$Dhd~we3djZIu)*-v0E|8# z?-c8vt%9J+ecXTFheIK%enz2j%;s>jhQ$oWTx}aYOI70mgb@TOWHN_XLusV@wm@zc zd@@AAYh2)c1w+I&h3dxH_dlWFWCyw$>lc?TgpOh%fy0!KbK~SRY_7ore)ri*Pvr;6 zC;E;4$y0xxf*yr4Duhm=vcEm>oqW3+t%jHxNy*itq%c}qQMWcNSEBMK>G1cm-95?- z{$2o`dsH7H3d1~aF!2%801MJ^7Tv6_i>E{$ySL!5fFp^~H!5W2HyIr=j@E8|=jn&w z1O=Ok^C01DxSNm838(2D7No#Jiomq{>p=YecdQ&_H}Pa?ZlhyDIC<3TKlzvsca3f< z4H6&2R<2li0->WZVC{k8aT2ReFDZNWk>;pChRFp;a#DKzwnM|SgE!}kmBke2N4eN1 zm_+JOQ6S|C6#gMg#YCidc-QJcIdvT-W0Rm(d|5G;&hQm{GtnN`b1&u82ITs#S-8`p zy0{-alo#;8zH0ukS@oK^T>=t0Asqx-Dk5@bs)`AKRe|ul1&oWoekwn}nKig=q^Bh{ zi)@0Gx=6I3GN}x!by95texPY#JsJ+D z^}W5`H4sWf5I8%3TT$J8X?DB_C!!D|-|itne^8)1_@OtWD;mm1KJ6@i1_n=onDF+w6Sxtbm3m3;YaD9XRXx}Ho5AQ z4@WM|CphWJT(DZj{PGq5G+%{3K!##S(*~Me3+{dW1E2pRG$iV`{uLz$1QenQ{7<|O z`s=?hhW~GSi~b*;zyGgghKT>`+P8PKbGG~E+W#*i!xn6-e~$h8E?+RYkCB~r!A0;c znB7wU!}6N0*uPCVyz?~1nx!@^o|v8_Pv~DaY(k2tloLMtv=o`)Oxakou&1Q!tu{0- z(YT8wa-Sq53XPP4qO8_9!+(f5!oU>bAzp6958?C-Gvp8ix`{zgMQhXkp&>p_>XGYv?#vA(oQ1^~elC|l!Xu8t2ZJU+0ZQEIC z+qP|IrES}`ZTr^O`|Nv9pW9#W^S5J+H^%xCE1nf=Js5M&H>STj{%PDyqB)}@f73aB zF`uyby#CbGY1pw-?PdIrbOhe8dcA%Hc&^YOOtWMMf=c`;bjT~J!gA$ zqPRJ_nVKp&sC<-{36|;p!~_x12f_YsX0ht5t`n@%gnMr5gmel=Zp5Lm;^_&5updVK z9rPo!u|46{^~BWnM-b4G7ebOB&gxfV_Hw||Y+G;5k~3uRCm5Dk!It(^+#tDu_SaFa zi9P+dcLDokvw$@C@SLcAIrO_>zFlZ`vEJO03U0c=}lb;PO%w}Xe*DL&kWb@A_=p1Gys?K&}>X2F--u{8^-~MCrOq&+~q_tq%GWeNLV1h(E58P9)sz? z9JD^)ZPh-4T7E^R08Q()CLDfLv1RykZ#}#facQ|1U=2)KTe4?K>FG0Pxz@ir9!;g# zN@H&~@ew~i?o>6~9oKBMLa^#LHLCf42<9IC#OmY^^m)_DPN}h}!g^lavFUi&v2Er0 z)WfPs=&g!4hp#b1IPyHcbJa_8`gA^}?KoV=2t)rnmso8_q(b?Y`GuCu2vG3oh;MTv$5^L{lN&-vS4stTU7( zVqVwo*7$g49<`PGX_(cD=P7rB1Hl zcI+dLTZNbe7eQRvEchOQ;CR&mxmZ^=RmWx3R-gP$P)i6sLlmUjQ{# zYeKp^~{t5lhjI>-}Cd_ z#~+6oZP}+OcBmehujX(e7^B-@EmR_);At*bZ+kIfH#-zYnW8ZXLX8yDkCe#+!7OMO zry5hyI8*QGi^_xiHC{B{=$_f5)?jE9e?t=yzu;)%CThF!YkernIqJvwq29}0p||e` z6e=$C-beiS#zq}o84&*|Zb>GZO&31~Q~iS*E`p3Zs_aEG)2Z;~FZf@mS9-KE%ih|@ zxgeE(G6@nzrGllo%M0IPb-)YJa%mZ(Rg{9TjB##Jh02=jM1;GeAPPrCwzd^>klW;Q zq3{KH)RkpN%c&p1^9;d{NTFQbBN_NC7k;xz?Rpg`&Bg$zg921CN_!*rmx)+J5@|m) z?5tQE- z9|q1u6lKx2_6bnYq|&%lgx6S8H9_Eftt6eeS1mUQKH(jacFtI>gP=6E=20rI&j^3- zm!TGt>MQ3oS#JeHRQ;L{Uu?n=p2+Yx8oU7HY)w^e?^cr&TBdov;_3+ zJ=8<=z=&XT22E=hM0-PlkV>2O_Xy?ylW;8dUcNH zD=Q=s_tqhf_DS4y;qNab?Qp{X#wUS*8hS|aa-GQkj%}!zbCmtt-bs%P<;SI-&r&}v zjQ4p(nVPizt*wkqEOl=!z_7Dwd6T&+g+_#pWS;b5GV+5{S%#^�nWEj(}ahv_hqz z%Bk#``WR_doFA(ru{gM?yk%3#Goai+9{j_txxJpb>%eg6QL6$eUMg}ey0Y~P{os$N z2)|HsW(wDCOdVxX2I}COIWXPYv?T#9yjg*IW2v*ay3c5PB^{1tzZ0Q3;L9l^Bpc(LDC^`3q~Q8@PG z`IkGB+JGpE6IZ(`MUihFa!jdHB^ZLMrk>dX*$Zi6S7J-Fcb5gc7u7}?wZTG=t=xI& zByNNNci%h+BLgb6K=}grjEQ*#$r`h`mn9k_zJX=gTF01wC^LW| zZ5pF&EXXFQEPbS0LC&R;3K5l&YG)lm*K919FoyEQGmmH(-U)J33XIiatt#Xu%fklX zZTp+hl{i2SmouOS^iFZHY@S!Pe@s=)+GYrjMH6m&=|UnKWuv3J!;k#L;o`<;2+dO7 z=L<>`2 zjpUL$J2MChe*|^#Y>bUn*8hR_bjCGF=FEzyrL9Fl5O>D!->T22ZL8L@s)9ibjmB4` zJXB__(CB4(HWOJ3_UR%U&mhuZYI0Itz0~YF()P)WWjOLaZ`Bj8O~^vI_)ro#btlZH z+_oDN_SPVTVhbiSW&z#gqoHEu-3CHcogm6V!OtT4ZT`f~<28u8hW<4%ae2vI4k8EF zVlAWGNyTnSI^+y?q?!ATma+tuzh#(pRMZ{EZrPxG+0m6E$=Zjf zvtih`CZOnh`d~zQg5@$0y9@0<+ik5^LEe~%#ZqQjV=~1&gi8v6z}u-ef2uKg5`z5- z4@|%~e*&)2gdis`;cS-4lY8sTYzp^q@=DN!80;HUQyqsT%fa(C_GI)F7KSfqwBAAR z(MdLBv!E{CAtfnNZ86Sz=|;7hOd-Z>BLalTtlU~_c&a)AK1HJ>w?^?pLZWZ{aNL}z zJq&3X5sgyCG5)dsTvJ*b3b81BApOPLuVjq84;3gk--hy(lFpr$mD31QkPb5dunwA+ zD7G<{8JaFop~;v+%TC2x1OH)0v7rxumn8HguOgyCi(f1}+gD=2p0oL>|L4s!r)(?^ zrDz}pSp0R3!S-N{Yq!c>?`l206z8Tg+EgDDB_?MZ_DBnbKRLFK7Rb~k7by8T3_`S8 zI%!!4`=-nBbz__DxTPqBzrbx@B!b#mk2Kni3)(ddes>H&l`O&bZuN8gN&E2U^W(UM z^D%xve#~VmhKiJ%Yh^BAL%SM4zGmx(6%288AYj8iIBD258iltX&>O7!D*^PY=hxx= zV9In%H#@tfSrLH<3C^MX3_n*BIPE6xZ2J1$aAc}<+Be&sz%u|r+qcWd>u5?&Tqkm8 zd?&>-oAH;on%A5q?%^q?RW!=VpiK4l^PIK?Mb?gTr=+v;lDKx0|UGeW8b`r_ZU#J>aH>;SfeKaB82(Zt$ERhpb($DB2ClbGO{E>hvNiR9}lo zHb6x|cR2P<8n-}_bz>Gp7ZOEYjPTMFE+XtGG*!fk(a`&cCFIJm^H{ajalDCFu`f-b zgPPV~B$_h3jQXW@BYQk>Rc@sQ0!&qSh*U0Yy=`!1TcRA~peUZEM+e_If(rG+SBOeSB)#2|GovO_e$#ZjM$m`WUJEg$TZCUgNriS?aKWqtF-#ZZDF}xUEKC)G;eQK}Tz!cmHJzaX`sm4*tig zu!Zs8*+O9b3tNbP12FtYBPekHji5M~oBnZc{cn>(8RJaVzt$rlGC-B8bj&&{eCN3; z{HU-7`xdS{IrPg6p>!=@I)Jo2BoG~qJZn8-S-cYK*Rz*k9I`RjC~6gLviQ_;23!BE z!_(doXoVBeqJIC+4xh(8G!2Su9*?IIn;{u`mxE{+5udoI-;qpI?||bfxX9Pk=LcJM z4jO2K@B1B`cE^G8QW{UfsYno`(O5^!4-nw$WQp|6?zTa_VdMZTzqJ}fo~0#})6v#iFPm7E$W z&~aBuA72@HL|PF168ZHy*VShm+qD}F?h!xr$|xDb%<{;$I=t@RudYbeN#|`F5Ld|0 zqBY02+-4*?za#Ae%yVzBn(JUg>rM-&g(p$0u)zfaQZ&Q>%+xXX8&USl^`ja9a(}uz z?M28{d-nz~oZm_j7?RWllHfEPCyE|~*k*sK=pz!7|?pyyd2 z{!SMk!^SGG1|i&q-0z{Ht% z7Y9-Df~un7F|%{N)9l3@R14rZ>^1+C;x2tz^u~Gci6~c|oJNOB9m+?^^D*%USQ;(M zlLtV-IyGRA6`g?fW5@ZDWb;vg=F{ywwq#Y3KOo*%rYx~UsYoi#t$mQuNVn)-*>{dG zwAvS>dAxaYnFs!!cxK`y=pG-Cfa1fOLTrp4K4B)k5K`*dDPYwv=K9adP7sb< zQBag|U(cBY069Qxb@~iHtSz$HC{YgnGimOz>WCj5J`7BzoW&ion8hx4&$r~~`)+DF zc{mOEu}c@#(rqVcsYo8dHHzCpq*)8J)N$KvfPt8g+3yLZY^}@%;CJo9i5%sMF@T;> zDP1rm9$fJ2_S~g=`yO1*I9hzCFB)q&fd$1ps*-n%s4^^M!^gah!IOPNoIozqvs_K7&DXs2l>t7l^oB-Ft+9UMx_TfK4>zm8$=2pZqW zo?=U8vevmb@kNI|g%7q%1IX^9NrOgpO214xXM9F0Cz;{+YB4%L>Z)_g7Ky?sH&y|4 zPmF8iBqXm{j3x^j2vx6T0#<%^E^U4f9g%0e1A`_A$`-c;*3}Hy@p>g?$sNW!Dm#fw zlD!c)uw{LoG5|N07n8iMMysYI&c}h^@t+MnM`mOD}x(LB%3P@4E9tru+%p5E?!8 zK`cmtkx09db_0glN)RL}OjMPK~5%K+(~9eOjj z^oLrdr~lgQEDtPF8!*#}CgFQ@k5XzPOdmj}_gUbDe8!JR0CbfZk=e+kgv&73wVnMN zjiI{(sc4x_Qt#9(ww`(pPTKF%`=moyIZ|L{=Zv3^L(?7WbuXae9qLwvyvysGgOAx8 zEfY5EZpqrKS{n~3&x1$0`r#)>=gtt?ZRKN7L?DKoo*c7A`Lj0RkLBU;F0oW!fPZy6 z!B#x$Mo<6%aD;!??J)mExBL5M@E_9bAOZe*@yVcy|Ks3)y+QqVvvV|daCD$kR)Yr6 z>S$AK{-T`lnO(Z`|z!QAFynVp${YJx{<@Q>wE( zq`ue8KsZH8ys{3`_yHwF2>h)3)zit_i;pvVOwYD92wE9B-s!2X9|`lTtBbs5kz0N; zBVh_BLId#FX%6n4OH^?$2HvuQn>-he@Z1R6x%hez^ZYwhV`w}NVy+_kI4Ci3ds^T< zuP)Y@M<T1^FmZTU5`?8^kk!m%LDE%Je?pf z@eY|S)Edc-01HzlAeWMGl3}q{NW$x_+Y1sEg4BtE3j0$gf=3Y+3lI_+s_cYGm83Nt z>_+!jfjWY7$l`Yq7qWKMNYxZmv7cp3{0=tgjVgE4FW6$1EE7T_RL|Es>fEtq)0h6? z4YRpf%($jkzqL$u0vPoot+{_z-$eS{>?Q(|dz#I*8k z!}oT+RSy=a#Viu-R4oA2uJ!3-574^qW$OfYs$PQQf5?p9s&sU zqpVtpuQB>grhO1Elz7o>ED@BTVu=7htPnu1+8-B##o)I08RkQ`wTVVvy@}3aVK>28 zHP*%pq?w}=t+MRWo_*~Ziyj`Tqq!zKJ9FsnP&>?Y{1v(eSXEK&<<$3-D0UXSePy3` z8E}fFMSrA1^q8!U8p>Kx0jyELs{39UL+`9Q(o&iSmwN%uwZL04VqTueDt3SeuN5nY zc8v#FJTHOt2;^Pr}!=uG@~92Y!ziVnmW0qamRCW|Z$+-D+f*-)viTaPf3?ta;G2J<-*fxvyj0O4i$hQ%J~GG2(=k5aS7_x=k`9 zNWkYZB-+8J_7a14^c&>j=sPBmyk@9xgKP*Q)BUPZ3nGg=8zc(+At_EMlM~y@yMRU* zW1(E&|F*q%{J9&#Yq$WX$jrvVyJJ-i#vx&|h_^Qj50}R;_F!wB-HkJ6YOuw9Q}+2& zMscdob2w;E2gB}Xd5Hoa-|enRhsrMa?C}T8{Tr9j*WH_|Gph=tV-{wk3wWSgU;uX? ze^N1m%(8QnQ%KY747WG|ErkN>i*ku`XAjlaR0fh8@M?d>pD2vo7 zjYntDBX8cLG>?LnLs-mEmy4}7NK-k@s#p|f$QyImK?n^MKan8)EZd&NJ7)nAm>b4N zeR|{={y7}OL9IiJ&!S%TOy#=LrQ7k8gx_2YL~s|386m)@zBq(9lEqX`fb7K!KJ_S) zB6MPQqNuJv6YXn1~*&T$*v53j%9T5h;PuOu_s8RtrjJ_){as@arM;X)I-T?P-#vNDs~)um3(9< z!35j51&Dw7i`yXXI|L`Q@?w?=1g%AE_}M9oROyOtsaj^F)Xuqt_|DL0aGvPh5he9W z9@QeQka0%>u(M-6$}PTGWtIC^&WY2Q*8#py?4W>U^+Q} z>%fi!zpc&bWNPQ_^#>O3BT=ANFKKpaPG?QmCQiuCw zznqS+_{!F3dC#e#qsRTdi>-<6ln-~#W~QuejQZ~*J}&Y^VCtYuGQ+0?x(BB|Mcr}7 z1$5AFYIRAs*dmFITa52LV8ssB;xvZeAq+%Awk$VnoSd8*HzWZR04PdT1SO12x*-Lo zLd6YCaH6K)zHHnlA+UR{RoOpeBMmC7eC7iSxic z7lwA&QY0I}%1ktbv`oHBxxeRgFXXK4BRXVZ&OL;fMV@1b(f|W^pquZ`$VMmX*sMmJ zp61|C=tQq?&C|RE2QBtfy*^&zI$KPxYf3H-A461A6j?h>e7fD=lcjZa8GXYd#@98S zALF}Z=xj@7-tLmuw!iFt{c?Nt2M;5OD6*LXz26r;jOrh)7l;qcIy~VHbf&7F=G?Rt z>U1O5i0b%pxg`@9Q|Jm;uI+3VN`l*nn{ErD{dCRwdv%y%Ssk+;MCydsvKLOed_EO& z$VFYbq%3!UhsNbuuc{v%!l{jBu^ok5%eUd>Sz%zOG$|$QSsYSHs{fnfW<9U6E3Vdd z_x;-L!~$#vQjQCqn)!LgrgdA&#ctj>e>igQUh9md9zoMBd)cNdsOxSe8<}EEoyZ)+ z;DM)tl(qmz=Qhb`wYzR!o^G1cXiWm`bouesQnMWe`TdKC?(eehXrs%r9@ZJ-& z-5}ij>ks8)G$9q~fGgrCloFOhF`DeqzY2k3Wjp4tWkn_p49uc1c(DMzSHoseu*}r- zzeel#m{L5h=d}-7Ph8BuTKMX*qcH=oKIjD*n>t0!?i32c5!=t%H(7h~Y<@zO>GR~+N1URz|H25rED?5uCJ>WZ%( zZicwCwlcf@Kw@~&q+~NxqG|y|RvY`Wme)IsXxS57Kr@xDg=s=er-6rp?gPPwq+P#% zmIBG?zueV{MygyU)JA8^zVDe%Omo5Rd}t#y0N93Hpg>{sj*tq2JbT&_Ev>Zt z9Nv*?m{n0$Zcb>*-#kWp`PiR2z*-^CZL~jC=_;VWMx$g15))v_yvtP&;L>+I9&Qn! z&49m;`w8{qIT8;iw1KsowB{PHZTJGnbk}tb5DdJpS+mIP08~A_4{Kn~C0Ak=tXHBW z=_3p=$x$utGbLeLM3gIy!%NRZLa5_|=e~c}XhHy2^%cV@%&tu^c&b*5ai4P)l_vGz zdEmWE{1Z;Ke?HrS-Q@Vh+ezqwiH4vK!g14j9#E6ri_YWp)?7M2mRrypf&mes(52YwMnNN%!pVh-5aEpihzp;HG;9oGcjxOzF8;7^n;INP%$da1`(ytpRYsBFm*Sdi<#IwH8l_kq5LUKn?zJ5qyjB z1GW~`CMxxafmi3&?KwsfD|3zKB__ML76;&OH~K4fhtGjE2Eech!WX~so+(Z zp_1nyJ*^8u1*WZhg{mub4^NUv2r*zS^N$#7f&odqWKfv#Gc#K)w}ZJ-tfg+Cxd?g8 z)GhMzG0=fY4G_!sA=UyCIBIBvh%zj7h_D ztZ$Fl-5fQrpHd9nYqrN#5ja*;B7Ny7IXAP<0E+ggOaB`d#?k5hIf#y>i>Bv^O zz31w=EuuJM$IkVuq^O+}F9R%qwsq7FD?c

    1}HqaHG?PTz`-~mv9-H{Swi$_4d(M zZ#cT59q{zhuuU1RF_=^()xT;Mi=Mz5W}-FTtU+a1az)lFzM%$?l6IrdH7JG4FpsVf z#LaYWI>>kzB=gR_8pc6O2+e3K^kr+H^ZmwbVX;_MR2)}PE5WYRm}Px~-X2FMg#S91 zux{gATEBoMf;Hn@k!6HI3s6B!*v53qLF~=I+g>br)nQ-t*?X7iI)|2bh&uCS32|=% zUHBTZU|*%HYFmUi%>PEBv<$|6VcMKRq%gRwYVn?CF$>-6= z=+%d}K^Bi=pJ0D7Nsj#n=$(>=OcPih0rDS~Vt@3$Y7JFrA#%H~D_{UbjLsx^GPE+I zb>J6V7n%z$6B_k*-P6daz>+>w{5c#aA?yv_^8jgv^|&G=U6K6ZQBu5EPl{d1F5E`#Ykx9Vi+WSLt-s;km!6!=x zqezsp92GZ=z@4*WfdkL}oQ;ga)t7A-J50ODI)*FvVR-fCWL z|8g*aT1H4}u#~B8Gb`FZ{bFvZ`C9S$J^y|nEz&8C-T%CE$_?6|PE)^vP3vg3WOrKe zizr*zpcWsJ+-bNj*nx&n6=s=L8aMi8+<{+o0~3C^sWUU9g`7W za5jXv=g%*zw`l2kZ|d2l!(w&|(CY9F?Rid-QJttcjmo7!2-m&|YM+T$i2(-k{`H9S zBJ0md33t2)o6M@=vu9_Rv3SENuiqFp!HgB zn3?QQ7ekz(h6MLguCk#vlXvmX&jE8?CYnK(aYRN6A6kLD=0u5uUj(|=;|N1m-C;q0 zMD6aLjjn@@tcoW4Jj;sjV}rG8=_TtvbDP~went-me$>U=3wdmnwaarZbM@DSQ_XU2 z9(rx$koQ45X>LF}pgZ#mpSKL{eiBMRrL$KvR~wqW=!mMGa0;JGA{BHqr z8hmkjk8iD=GWvSSKvCU^EGTs%e)XdL@pfWLHo^=6<-~@V{eV*GwNjR-6XYndTCUyZ zRQbb;egbD>>F{BwAh{E&;}5tlMHWuNYJb8kSEf1zd@`(Q> zigI6uNV@^zBs;2kc|cnDd3Hd^;~;|9?)Y42=n<&0wmk>J)iG%rKc(O;8az87HiopetnaJ zdHthoX9jRRvC%Xtw*;?{m)LJM>hcCBoPm_Ji8n8geQilMNl{f>BG4gueqO^ z&Z4D2h%Ky4z4?2)e@g7*r-i@ZM&ZIM8zsoa>X%8f$<*FjAg*N4s?>nMw22Z<sLjDsartZQmM$q&}#{t$#Yut2<306p1zM`{0a9V#q#-mpCK+qwV!KvEvuw0 z2whR-8Q#wQvS|?ka&@WJZIRv`j3p*JN$cARfTlj~Z${DM3MT~03g(noH)u{5yEeGeyfo0#02PAxaZ!gMcrYOgn`J0YmmDIy(HgDMTEM{y)xrPT!$HP|FQU{|*zQ`dduw{|Y4fk4D9?{}~lCG5@2mGI#%bL~K!Y+j5;A-uqdX zU;`}!6e*)G7hY|2DTJSGrL9Ch&?-l>q%KL~k5qN}8u>f%yivTJ_Kz4-+@W9jyC4sEiL}3>g>|oY@zBj2bKlvo3ws>yh!Jvki zbUCA}5M<&pCh@+7h*~8`4fBjf|1ZqRihFMiTM4z*dnZfz**?b&WS zFFHpE*RaH7ri6^2ngv1&LG%|LU0$STQflmH_vOf9pQlX*9e_w1s^$BhuJ^Zk^}_Jd z_Gwb_4Shg-3(g+s^^KId;8<1smn~ME!HiC(np9oJ^;O@wOSes=o*APg{sP4ZY$JRE zX?zdF66ZVAJ)#Ft#JR``F2uos%8;QWVsq@wQH4*X{`e=(+9YaCnTEXR?HKXcGI1R+ zKcH$qYPZOOFA+95I(N6HhE4!}k$0BKtUrTyv)R>nW$wNsaH9#gkcC%oXnUsZ_g#Xl z4Dp~|sndH5GjAhYDRs$m_?%PJ*hU`vtt9mwn{H6{^ai~_R?nKHcrd%a5PCf@dgLdl zobD{Iom;1eSS8u!^LGt4x%Qa6_-N?a{M49011~c28zQHQxgk1jHQ>39A&p@_N79}3IK`VahWZ1levtp8oixv3`U z@JGzqb+4+o0hOfhb~eiNlelS)Qkq(J4|@&@j4*}KI$jT8B>c;Z4}cgDK0Zl8nrkuA zm>acy`=_6BuUQnMhH`9JJS%pRlVlnnhvz1r5veA=EOdBjPILP)JqmMW9!qV zauangwl$TQMZim|zOBvR0f;fYnn)8z# z)@Q}e!jz9|SF8FRQB}(i&-WVsnJ;{p=Ylp)fR*M@(Y9?jo-VJe^JktPPr^5`N5E}~Rv?4I5X#vv zpiJ2$qP4!km6Sa!3yGI@(wKF@v`C_p;{td#NxoYm@j`RW4ORoj>>E};iY|vD#sKGa zn!>u~=3Xy5D{WlH2QPVay4={hyM`k&I`h^doX&kSaHlT#MmfvTT~pFj>t$-dR_$6} z^O&z*l=ZmnrJ#v-C(=PdinQQK_slh*k@?%<K<{V zZ8b4VYRFld*cq3!w&bFgg57!cmYtjB@M7uX!p27LUC=h#XR?u>E}bCx+JI!*)wWO> zy7i&<>eO19>D&|~M9b;g%N9G|y$n{9)c$N~v~l@$@aQ7Xt_4B)$*EnAYVEt}n@pwg z;3E=tp=?T#<^%VBqxMtxjGd>-yJhoX`rFoH^7^ltjoPZZ=lxG9?L+x@GaJd@meT*B zy8K5MQk;J-r2qKe1busZeYd|OFr$0OByWUm*?BW$5Cm6*g6t-w@FV`sRY~T@& zW059>h+|k_%{bb2nV{l>pkW)@whZpgKCU&5(!1?p8tnExx@A1LwfbpzfrI>%dCKw) zu$C=v;J#01Cjc6zkIN9lDcK4?+=q%8v?HEwx^^c(@_ykfK#IyDhGhR@&Zw@2jN)I5 zX`X<*nq^Fb$w_T|zkY|Py#q54HmAM2RO(Z_He9~I&sluB8$`uIob$^)vPcBvE0{Tj z9DiE4x}FCUiGJt_F>?31F?ij@;7vMvq?0G0S5G?}`(^}Y4X@Q!4)@%kOr}PDiY>w zY^Ks?hGk<>#Q0JAKt12eQQ#e=rUvD=A()J<%!EciV7Gc!sp26s-;mbw2L@9;P;l2CDqGZ?JJk zH%TJS4$WlAJM?o}Uon49;YH3z?DQ^=O4i@o!MX2r=$u~x>ys$<8eO||JdAU!a|>eU zJXgi3{xx60wx(RoOVk{5%2oJIXpg~9vZ2U`B!Q}4!B1@t%5U{w{}6n(a5bV}0RaGD zg8jQ(Bmdi6`=1}c|LAo4KO_VHxLQMfD=PziL(9L*xJwP|Khq21SGBI+$g^L1Ii|Qx zHPfkiaa|?x)L<={!ukd=F07CMs#Uxm|6J4BsZaMHl(=u9&c%v!4Tm&o^2p%c>ZU0g zRP3d5JIgLe$$%)ezi~?I1UqazZC^wnsJes}$*ckq4{Ff?EzAiaURg~byT5r$ zJY+q75rZkNI3lL@QU39ngygqWT~M-o!5j+uA=aIP8~^sf3K7(W_A#5GOu+()i%tbI zfwXu$QPlFb_9Hjy5c_0^9kw>&{uN4B?-j|*EozMeMoG}lJ=j%;0$baOCk$;Az{loW zW^HZ+DLXhwBE~LTV2)w5Adrn@o4u8Q!ZMj8^iMk;np1;mqF-ml%*51^TwvgU9JRSE zd+|}nAH3V0zqTWNaNCdfU(abJZsJ=;DJ8v}v3jR{V_PCRQ2fg9@eytn#`0w<7T18I z=C5Ibr-SIh^NB%^e06`J|DX?Ho;~f5D@H*$htQr9byN0S>+^L+YwUhBcv0-t3>Qym zyeni=)*bqVEe8iRatVbcIQ)>MUn*({S4n0U>tOcA2Jl#yh(E%3^x4U`YOW(m(jzq5K32`XgzXk#^ zTwB4ZRK>ESw|*>?8h5jm@yH$%#l$?;C% zD^u3gAvSxe@Z+Ln)QFQOIs61sm|8gZOBfK1Pb(fd3X_MpNDN?K4;nK{j>{mH;WDZp zi>bU?_*QLBtM78n_uiSy~_ zSCxL)?nM2N9NSd19b$gEyhj4V?GlAhnrc3+bz{pSp2o&9;y$8`UkI>#1O7Zr`B)E&lSvSv{l;^B^gv%7qpQDRjpc7V6M~! zB|i34Wnn?oig7gsPCx)}hr4>KckePy*X%Lwq-JM&@CikP8^%tpmY4SEw2vZK2y(j6 z1Z}0bSjrH=MvfK5I@s^Vb@9r42$B^vaztqEG-PptU#NRRmY1?9mEmRJ%ixkTe~qxN z)Qv5+QkhPK;kz@{uK&8TR+iOD+Yw4^?YlRpRsPi-tz&#_^ji&7hfz^cW@zNpWi}Ts zC)y+lx{#_vYlHVfOHE&;yf9QjgChRUpq7za&#bF_+pzF~r6ul0Np5jAeZ{H7?ndgU z8J+4OH<`i>2x*0E!p@_m18HkyuXCL~ zN5SN{dbCNG|N4z=xyRC0r`alz_Y3mp*7yE%FWx1tE89S<3qZkr&t9<*UeExT}Dz*jDvQ#@h$c?3BWZ*7!#(h#k@|1ZZlKNaQg%T9G63 zpsH5B{54`W+Gt9Vj4Ap|nPr7MuUr)LYIF?vsDqZGQwvKpoo#hTEB}+;&JWjqsZL;K zge+_cRrVwkqL^QH4w#iiwFAqwP0P?pVe`67*zEnZkYWeuCx_9Dr;e;Kzm#-FDo^s) zW@0`gtON8#lB&c=EDOSWh12Si=7XcGSVrG06b_HB#7eUa20wp@nH@9ze*n$;MXi01g*O<r4W?i&H z8L@~vKS~&fFBhOpRFPMGT+mXi2nzT*eRKT0P<#+<*G#CE=G?GUnS9@ni@Bl3Z=Ulc!%7C z##b<#QI(Jw*eN+()_OqG%eDgx*^1BU7kFy809KCc+|-gV0&MHQP@b8aBuAw!0p5T@ z90-#i^DxR@%cWmr&Ci`lB9THJ4DrbS8E))X!zE=VM#m%fWJzi`%A)xG~+5;XF1ceDmVnkyC?-;GLJbR9!q$kOoliA)KDSNbZM_-?t$Uc5&?rv9u z*;eKKQ{-|(hwZs0t^K^nxQ+iXO6;wCN^QRhHN&8Wc>R-7m2!Md6(<2lda|=E;hW1D zX%!1I;B&Nc&u)53&rDqyW@ zwZ+(iOxr|Ru#d<(?UAMdrdnCco3ER6)3X1E zyLW63wd>M$W81cE+qRvo*tTukwr$(CZLK(2$zD(Qmv?vV=k8tge(L@Q*HtwK=9uF+ z&kv3A(#o^=&+SKvbOEo_F;Apma~oFsvL4);5xvonmPbPks)ihzfwHR%L6fp?yIn=K0G)0|AX^Jv^m!O>- z)g;+Wd7erB`xH@mRBNGrb34ivSc}sy#x7{3SH|BW#fF>=h2MBf!6dExktbDM8TFO- zbCB%ReKQd_;VmG?e?+Qd1acb-jUMl(0$RLv`RsR;7GdJpd8b(M;3vXB4lylj9y9Dg z@u}m2tZJ<<-C&=$=4^i~rhK{E&GWNow;KLdv3l>Tr05yxt1%m-|GCOg-;zgqsLaJ?xQ;t6+5tbodTd9jci$+y9?DzW*8Vyn8&aWL30HBHCpW5Mn|6K6@ zt3mu9-4XwbJO4i>dH?al#uop6e^x_V{+bQJcdo8}P`a>IZ3mzSCjvZDpeM(PDrYGS zig1b~X?TCQ;^w;Xr?;0xI@>s)Fr=3SJ{)t-+X)}raezA5VYgBZU^H9}79FaERn`M6 zT@f^pp+n$U&2g2gUycaEv1r3u{x;RkSk_(;#d^FIJRGQ)-|LVA`l$@$fw}&)TlFhY z9Tr&B0~g*OYXDWA^Gfi0J@?C#5(FI>6I4lZj)aY7kB$c+C{A+805&sl8PWOB0YRtp z2wJSTyRe@rVm5yHGG25IjbIwq>)ECqBau2z=4;8=jS5wfWh50FjGZ_LUogen@&Q6O z10yUEuBP#k+g_ZE&*M6YoX_JjY47IC7+?k!RmrCW)w_11m77z@8#d4(QIKu1 zIX)B=0`RPS;GS>|EO3RlJkBqTvAH(4_{yB<8nZfOBu%~Zi%W~Qt`*`{<;bpi&(71x zQ|&x1vuZPI2-PLBPotpB)|Ml$UH2WXEaWIW<9K3b3yvZnCu}_wzxqe)%E^w5t2sBO z9SAW-A|%wfFk91>wh<-rD`e|Vjly+g3uir*WzpiD@O{GAGZE9%8T2LZ`7I-mklES! z=u7W=x?hhgoCKb*T+vN+g(2#t#(o62Q%P)l+jJjbvg(JqnZGn2V}?;HT$UPX*?>VY z$vS!|qSw7%hx@9I6_g`x!X#by4Kp0#HRdow!F!S!Y8a=r*zyQefVkTD&ZnmJ)D+LF z33Dn^O(cX15~~3x*kl2p)L@PjlPOt(wrnp|m-hb3?*#KDdIeDcR3Itn!yQisY#&=L zVH;nCY@W()o^owx)!6T?t}h9;#XJ20d570J{}FS)6O)K}44ATJw5hpUU0Li!!qu#fqUzZouvEk&~bUtaw#e%^xX&S(6t{| z>+J`xc53cbnZ?GT-VoEysn%!Xcu#UmS_cVmnDQJ_SnFtmE8(x~B({T3Zxw7`dwlgd zP793*YGTbE395*o1u?d%)_kk3c!+P{|?u;RpjdR%4Dkc;b~4^BaEN$yT&FPw9+6L*t<;5G}FD0VRT z{SdnWM!u1c{S~Iv6kmY4{r+Qiu#Tmnz5aJj$Bgt($&&1!X9xd~JN`%K$$zPw{?|Nd zX6S6<_U}h^H@sF(TVjcO&(vh|s-Q9y!NDY3iaD)n2_+hu+DarF52+_c0mt!yus|$$ zi5XR&f}h+kak_4E0OnaA8pc~DA5y^m*c=xHByymgcG%Zi?srX%V^Vl)p&Zh2tDcq08FEoPJ-NNDjz z3Y}4+`5rlFo834#xcJ)78-FML@q?MApM`tTb1}(iHcf^(S}5>IFEjt$Mlhbo)un;1OIXoLt)^G^+RtaZhzZ=-ov!nuh+eNW1*by$^Ov1|zGzbHB~wS$^PmBl}|T zDM9(!*WnJL9AAlPns4}k-Tk7{bH13O=EHUrFrGXb7(Ut6@XmJVccQ6JIC`h!27cJ6(ay1&|X)Q1}^pM%Xmx;XWh$JX1CtVk)MG z00^Ilk8|7O>%qvu%h`ocj}XR~$3dVLlmNnHFGkE42fm&m2qaPL4&F;o^1U_;Nh&Kr zY{>tZO`)U->xcrLPs+6?hpyd+8*B)WJs2lZ_4u4V#xD4#$Orf=AUpk;Ltx)`pZs8c zJpJB(#-888H=prVBEA6A>_O*)=|s8?fg05`#m}gZ!{N-X9hxoxO8sOl`BJdCZ_uJ`Lq^ zC-`lZ@gf39@i<}i-Sr?PG;YFj=~2;*+3AkcZ(x?7Vrw2}?V)Nr9DY47{rEQ>T_Qg9 z%t=)s)rJsQCNH(U-O)jXcu~AFrzBtHfSp_uyH_smVAl4RGoW+|alVKVz;#%&^MNwz zS-IHL4WK!t1JwOehcNR>zl2%>vu?Qz7B&hao%$%`m$;%d z`i+q&U{_DqMn8Ms*;u#>>P!v_AA7lzIsLmkd>kHh4&(&f51z2tB*GBNVvDxma2ksR zXifS4bM37|t8v2S#|@pblP2b@ARnE<1{oly)t(%m-P^4uEw)qD@SMhiaV`Y>ms6)X zT+Z{p$gdazzG>r|s{Z@F<07ZwYaxPF0e|j3-$&I0A7{@$#ejP#s%CtC3<0(1Mltfw zjx>C6wH)b1ifEtdu#GiS3+RNklF4^TDv9m%LWUvL>WHngvmx(&4&vFc@8g12enT4^ z2EGW;Mw)Wg#wl+FzH|%)!&Dh0FsGW=Jzh9(I*y<|4yoeTl@r3MBag%Z!Z>uUOsYC% z`FWhy+0&-bWREZ}gb_whHq2+l;4APEhY3ez!H3*W{(U_jDS}D~O?*0Y)xSkSH1k-c z1`&`@oB8T==A4KppvDjSSM1?>h!?1&AXidGX;Z#yBIi_RU#|dv03+21k2ks< zZtOt^VGavZ&hh)#3hllR!Q2(hbh6Ag;8x0D;bg)}Y$z)9jr0kE<{e=h? zT8O1FE+zb}TS`%(7c55Os=2qjy{LbFUl9H+Kz}V(5d~neZbDJQc{r$P$O+oVX|F(w zG@X{b|BbPkF1Qv)3VzI961hvPHaUhL&|3|Ifo@n;(Rg2sxT(36uKRH$t$$)9M&RsT zeJ`{+J1NPmKGKe_$UJR1?WoC>@=HQvRt6k~w(>-zLRW-uUt`{BISia8`B9Y1_ny*$XUTnlm%crkeuzJC z9+*#04+|Aor89!Mw;E{i!ns$W3JZG?kLyecfV*={b=jxQ9199WWlX%0cOk>f`<4@7 zwnS8%C3!Lw+EO~=l!{QBV=zQzo(BH!&odOxif`Uf8Iy96Kd ztua8Fe>jk@oTopn-Kt(|f$=nV!iwZ1IP-fvZD~AR( zPyid)68?-|APJ{F{#O+!86C%nHWDC{i5)-F(U-WbeC0|}>-!^|?%_w0YGr9e(g)W} z($s@si{#qmfqWH-Kag)GeUW{iKhcklVO`w2?1r(~cnU7rRP?o(ML4^%@U|upfTuu} zn!fK`7O0{Y^izQetH|KA)TSwTOIj$#=E}OIKW+R#G|t621qQFSmenDsoHZ7K*yv*< z>iZcj(mav|4g&+MJ%e!EoCZUoh^6&0skG!I;&8@N{3u!4Xet70H(kg9k+k_m)9S_u z>J!>mkq5*i2IYGJOoHWsG!SUUV7&16P?I^>m?RLx+Fr%?6XKw5&T@1RC!At+eFMtZ zpOZ4X80Q~oBAcHxkDz$*-~Dk6hjk2J)SR&>Du}pU;qb?Jgz*qusBL32y}*vHw5~3x zoMXKc4D8qjye4vlUnbN%PUhaPnBuCvyepMS6U<{V?#5Kq{yb)EINaQ{Lp5PYikL{( z#6#LJETRt%`E_knUWV1_UhWVO{gShb9vJuMkTnwNpldj5)Q5|}RM+6qow@-ahJxe# zq_*W`W*=`7pjDJl$Y4T4-k7^yg+48+A}C{o5l9VR^y0-K>n7A(i{}*RJ)NpnmggGT z^%i7oURkpsB{`pw!&kmSJ)43~8?UcOm#UJ!&*K`(JnmEVa7&Rr_FBQKyH%+$or&$q zPG|>sDKaIDHH2$u8RxJC@50T>KFCUOGbXIoq`V$fo9#1C>J^BMG@x+3XjS3Te;8>T z5z74RaYIiyzn8h<3JNahCyWE3Ig%~WB(P)avC!xj)QRE`y(bP8M}R(7+1` zn>aGHnals68qq#ycK!gad=#))!+X;l=tZbPH+hQ7@9L#U?*Ma&5IBVR^sSW|=+jKF zefqe}8c{gsMy~fzRS-g&C!;79lHj7EE)F0p=wBYKOjNfUmR)9miO|!qmj@+tb27s&QnZ=A zR$!}%B2~I1*Giws{ASyVuu>td4M&WPxH`P2juRDAO!%0%$3OJQMukOhZyrn-^f`&; zq%%SmEN27QLOE+o3-ruUZzG1>o!BraDgsmKIP%Iar=pq}M+dmH1FUb2nj8+{>bmSL zRf~dh&hPqA7uDOQ64Y9#*!ErQAbR2nMZk zoL1#6U<4Rh5*!Z=bZRJ;No@F^Cbui|hqj2;R09D8?n{MZ8VK|{^O{>2-0>P2w~<~G ztK)zNTx0F*qQm@iVtDsr`7&Z@+f@2E=G&15=1wibO1c=BtrpN*KI@X%8&&SX$rP9v zMN_V`0v0DJuMO57x}bUI5+J?FSTQ2#FTYv4IiD8grLc~3SfV#(0zqH|yw@43fSIX1 zwyt)YwiwbA%UVVn*~M1tb-h)lWQ=jNtmVS_2UKWIZH7b}6*u8D)UnG;5ss*ymse6> z{WO09U%Fc4FgH_i{PMn%`(~v4E(l3(!3v{=Y1}y8-9&LJf{wRdEIpezojL1YFun~4ZWNv;UwnXx#F9~^2Eq8?|ZyTswihB{n=_KOA)A)ACxC3@@ed#UX zcH{#l^08+|fci_+^2|ZHtwr;7`^s#6CA7&1I(?@E-~2-@c$OhU$V z6OH81I;xlfF7CcU*}!%usvV$Ut5y3-mNQi!^ydA}*_}R`G5ypdJR0#k+tgns4K=J# z7qML<*>_8FOZrI4GXdwG~eWed^v^C8;d!C)o*1soKn_gVl z8m0K#>O`2!EUjQ!E3Q*}VZ54bl%*rp21?Dcs0*K(lw_~=^|vc{0B`US4Y6eNm$@kCWce0K|t)CDiy%q!io7B_Eq} zbh&B)0u}evv<*R!Wq~ZJ>2U)eGrH`yZ$UU$JmboD_Ps4`murqIV^v%}bpjRu;E5QXUfTwhzkH)B%9uKY5 z{50b1z*ReJwhmDx!bz}E$hj~^C;(Mbs2-^nu2y3~uLlPLdJT6&Q)`kxOG zEk0u=ojWcO6B`^ay#>O;-vE3r{hqiQ$~1)Ym)orv@$?R-2HI5l5R2|S8P-bNF-h|S z%naj&nO3ta9PS!$RBL0fszJ%MlmZ;_j4K&Ixc=Zi@vHJeCp?AbYbu$@A@9a4 zZ+xm5=SBmNbOXI1omC}LBX`fLcNjFcL(GEm*K_sM z^^MKjil0!NP1nN4)1tvG3&4qU-2?<=8mIHTJnf9{)#Xdp6;Uf}%MEEu9_#9^b+zo5 zMS0JP4mReNf>&3A>*$>dpW2B_g`O2(nkCzBcVR=v3QOi7_R9v+UCYSC+)KqeiYx2z zTTZxR&7IN%81cO z#^ET#K5v+o47Tmz;R0R#1v*aFW0=^(oveg`8!AU#SxW;?P~zWu4H7;zz>AZc3cMV? zq-_l&5m&4iv-1m{%eDUJ#~A>Md6DUvFzYL#$vObqZUG#BjU`%&^#OToZlRZ&C?6iWUTKC?$`N0GiDS#@kJSneEjM%U zrI{d5g!X8)GXUiAhhvx&t+*RgI%codrq~=}<{e@>JHz1FqKTrce)i;bIg0>;8D5`8 zemZYshpyrc60e3=GgazwWyeVy#s=O*L6TmPIiH%&qG7oRsBbC)by}-VwBf8M`gb`4 zdq$KwHcxko{o%|=HyRh#&CJ1#%xnqlz}6??DH)d5ymQOmmx~ovY4E=5R^-dhiSnX) z4C2jHcWvFF(H-Eqf6V8Zpnx0fdU)WE_B+Z?)TOgEXN(?07q)B{c@fBN;<1+|=NL?~ z6#a1NOC%fQChcTMryI2`o{e(JEaj%Fex$L6I$v(0&=&RQQHN)bN*)0fhAVZx3>N0` zRGM9IwD}PQ3W=d6OqH(i*snyju7oxS6i;+JAWVGaKzhN z=-n>LA;V!b-p75`SZ+Mnkj8cMYzJ|-UpsOJ0_wcycXM1)5olsC7u%=%gWEcsdV|r% zEWNn*^K!b3556P-6ljd<`9EVGru zk=a=RNVCY$+hoD-^H@zypjj6i3vTzK*UNXljr=*h3Ng_~5JAfoB^L@*0($POJclT& zoGL+!c|gSVx+^)=1qKhRkvq}Gxl`r5D@i$+XxjKukLQsp(~y)5eRUkc{j5X50YMYJ zUI=^8&UVYvzs2%obrfmd>+w8#!Y+F3?Hd#qwSKU$*nc{}s>Wyv{Z_qBay{lWw<+H0#VQ3DxF3e=W6hWL!@YQ{0{MvQKCDm#nSsQLZgY#(lph0L z#<+wna=c&$>L(Q?auAC$_$$Y}g4$}lW`yG_XKkYSmPW8AIcgPElzO*EXawwWP)}0U zVy?1eg=$~KB>$2)L=&s#=;U{Be6)Gqq3QUB!FL!?Lr7|`6gMa z9XUbCW!}5(}0{|>V-mKk3aB2 z&rwf88e4y2Jn~AIN*>MwV9!4)a8eAsECDBOH4!kXVcTtzyFi#jf#CVIDgF7v?UVQI zLddZuag9>7WlFUHZ$<9M^@*ku#`w3bz^s}3bE=TxORb2W6fW}dLY~$*DHQ6qMr8I^ zJ;?Z}55R?08afrmD$icE$1eSCul#@t-!2Uw9akud`MrZCW|wG&ELBn_NQDi3&M?02 zL;c7?*NtkWWpV*mU~MzNejQ?k$`&?;it{U~#TVr+l(G62`oN=M>Kld57z&euka{fbp1eF)*ZI>P|2subVr+ZoSz)^Yp$GtctLZvy zl%WK6uGZDxvcN4B?pR%N^N_oU4=2V)RR<$?kn)qj-h6VFUddN+XeN0(YZ8`5 z6%h`9p1#-W$yfkQg~I&#Ma2xFd!Ayk0u(_3M7jX-&C$vc`Lf2MLYW}MQ!5gx&c2o| z9a&4INIR(bzNKsN1ZqzQzHA*;0Te2?k&`UfHL{G>VOthOhYD>%riQZy>TvR28i&ka z=(yU2&`I2Z{Hr&_kN<2BbDKcBZi28hcHVc$?qstiGK*Nkp-iN9O}8t4CKuDu5^Kks zGz!H`0jSLbd{J5kyeY2&KZv7;n*gbc*r3LkS8kP~mPDeynkZxhzU<4lriZ@hKYSr5 z*DU{?#4Ac2^Te;9YQjXRY_yOVDzY6vi9y(G1o~iS!dt?Tm02X37 zNwt-n>PSha(dCZf*V_tBJ0`7G(8X`nC#KT|6gkCIHNQ z;?C^xXfi3C&(`I!z$5Rz%aC}az121}EwOWb46oVL_AVT-zIStEHO-CsAe<@ODI5_myjDVgw@yF^DZr5l@vmQZ!`e$Fx5NT|nTz;VY1XKr|BR_A&*luRb; z4`2k8Ov(GaTl&rEfUVjTqj@U!EV$)78Z7MBl60h>&6q+r*=UFn$qd&3krg~Zvm*}} zizv$8*73+Gg=@<)s;o9_YvHlU(bbL=g}8wN|2wXZ06s4;50mUT1a%};eRgD4JXf%A z1%96Fc1UcZ#SxG*a~zvYcXK~mri+q}oj~5mkj`5WT+~e$QcI>AGRBr?Fe?FP%=s%*Zg#L1>i(R@fl6Cqt$NOMd>=d8Ai+qb zC2*yTwaOhXBz_`;`IK7*HX^+C5o`G;TX#aEG%R| z7JUN>2!@R{F|XuX4tGtxIPcOZPSi+09{Wt)G5Ao-MH9L3Zb96xq0=`KdUxBN28BUa zDi5zSzRJwC96*;+`XObWLu69*T9q2d^!ACWlxq{pmxv>T%(g^1COHtKkPbyMv;9jI zDFXUBoFGsF1>Qx&+#4WreJ`98lvtJoayl%YL)~PJ%9YYi+nHDN!z3$>(54~j?-v*= zhz&d8bPhX$r-`Y$#*nDw7OYJ^MYMn=W>91H%Yi~!>L{hNx0pjz?e5kv!^E$9_9Hj)rR;Z#pxVUSap7I&v2z9HKMh|jyIXRH60S8r648ZAL7IxFH}SHC@%>P;9?u)g3h zT8^WDKBjzn+Ry>>#h8n-R|)|gOg4e?D3_O2V<|~b)i+k4p)R{jvIXh@!m@~>ZjGX3 zZH*=zqXA40H*bqSDB9@{Jk)amA~<@Jz!?-ctqfe~SeeW(un17m8!j^H2ZXmq_F+xV zMaFi5(qjuUCU{oTf>qx1Ui8YH$?s zf!8P|w2F9k8<2)ny2IX^0UvNXN$)Ow$q`y9`5cR((GXrdX43J|*!LA)VMRC7wh#F- z@NdExHDCxDw`O1fS+)j2%M%LuPn1)805dcg9{@nC{Gaw3sQ!7c;lFw7{~x{E@L!Pe z|JVUDH?g+=cSOWW>VWMQ15D_h2NY2cg{VQkXaEqXBSt7%k9F5I9Ru4DMGCE8Vp6-j zzND)Xw`?m;>*$K?qI~3+jFcXRD!^IPq-rP!xT3iUj!KDys+bOxI9N#z&}>a&3h^$5 zSkVGe`sIgh3P}Ct)JQ_4@MA51ac1mK)uDlpDgkvvBMMq{G zg^Tqg0}wykAZ=X6&C6@%3k(}zb9|p&sGxSbVD9m;@8W5L!zdZAWa(l#Bq77593irr zbhVlM-)%0d1+7QTn7ke|708Mw>iLg9c;tm3F&EIM%K(;Ym|TTtd}j_*ndJFAUk8rql;y+C~p5e&~^mkzX=)qIf51? z&AGQNU~S$HB4z1c&Wu(`S6sct{UNBqMJ~z87W~Nl6PY%^xtX+rL-!?p=zHe<^!)zl z{fl=D?z7ChXKZaP+`WY^Q7?{cM~+W-e_=@dk31I2y!ppZl^aCf0qnmpKDJl&z{kIG zq_!~s6h{A^i2iSn)&IH}vbJz?{&z7nqor%N#fIYZT4%6=-r)kG8LK^=9`;OprG=tf z@9G5!6VOoitD~`8NyX^rGa=bn8Kt7#f^1E~AmJ77@%~;fvhajM?hY1o+8{t8aCZ;~ zRv%7b|FiGOv+pRo1|lclN@fGfX#Ls~4xEN^BcDO%l%@Y&ySu8J@4nMbbOj&< zJuUqZ)*dVaNRHCbSnd$Yy@UIQpdURMi7Q^zqe1|ruX@#rqOuZg8}BeL+QCF<{ZJN5 zIDCoan0JQo=-1pm(x7!p6KIFDJ0qCHXOEs8ptLh^bX78=o7`jYGkGH|l+G-OZi=v5 z5uX-PBoPW}_~9{_)E%asG2Ri4aGj&rZu~JC#phr}M@n@rMb7|cGHnzcv%ycXE!ab2 zb@G$Pg7m^NA6%QhTP>MErRCkN&v2O%^^W{-Gb1Zq91YSe^z9NoOBml;>ER^6QkzZ- zn6B#LU^(oFb`pgmC-lmW))J<&@sgK+4T3Oc`B>Q{^0qF@Bn535cP>^$wkX_&{Q3Dl zp>SFAkQ&RA;_rCehX|D4W2E-45b(VxJOMf-NxeP@U(~-?@|cC5AH4CWHv8AwurdGW z%@(eEiBj%Lep8i=;Z2Ya>fK?C&qg+%Kxe@t`Ad7B-969Ja zYnjGij94s{bSa?sL_X-HUhTmTe~zVMe5|9LAYC{l5NNjlo%Pv^Ne4{430+;@YLy^) zpc5@`V<4#yegc!!l2+~k?RPYRhg;z)*cbHw-M0bDG5zy1R}M%)F}W4A?4Xdyy^I45 zhqb@&ey`bd#k}!~_k;2$;!J;mb=n~p(5Q++_zRx611O;c!!~)__yvyLy=zTOXRm68 zjH{f!mRi6n7vz;3iX{3m3rAk)CBHbKRe&`YKTGinsxo~JHN|9!*p~joPwz!_4EnRP zx$er<3nXqC0S^~-h?q58+=qL{%-r}_d1faW`(e+NP*OHSja8{K<=IxsZ+Rg&-aM=# zxn+D@8~#qc%Z?Z8%CK}bvofS|oxq}T5UQ+xnKNti{UQ}t0SSGeJ-|;KaVPiTn!$D# zL+q-_@!G2XUMxAWCgA4Ank=>^IV6P(6_nehia?1Rs_>Xx{)d_;%U0zDZfLEyicvU- zONSSyw`KZ01B$fsRJhVfuaFoJ7haBJ@(VP}+ML4tLM{L|R&yp5@ww)@- z0&`dEFePj4Hy=qZ{Vn?0QcnhE|C{Ne(w&zu`_xmpTy)=nd| zr&!JnlXEd*qj0f#&_?&QnyK?G^|0k#QlZ5)PVz`s|F%}BxorAm>A~1%1|@0jrmCh&6Yy~&LX!!j&x!42wZNjXC^6g?z<>cg zw5FgHY(h>dsoIJ}Cj~nE2r^3dB+!^)eKJ>o*$A}Fv^sA($j9gbwXI*fuVk4W-<6Bf z_hISPM`(o~KXzyo9gvp1yp_K)IWRMHVXJ;P9LWpOR3KjU-1G9*3uA`fb2chay; zA)mRq`!ojJy7R)Qdg=2+$8v7}WE~rsZvzRjbdTJnSLC$n#@jiF-kY$SSUcn;Zv}df z;+ZT2g^AoFyeqaf3i1{|Un93vmK+732Qxp9qNOqNE`ure_6lJipW--}akAi@V-8bI zY8jYc5Ink~7;t`7Kl&3&mupPZr@MJrZ*9HV=%M_TD}&AbT!yx$kDa?pw&c0iidHt} z2{y)#mA^+dw+VBbXN1oe5^909>qbSwv!eXss94b>`^sx3549Vb5|!!hL9SKEAagI) zZ9iV(I%pqML;{lo)a;ibzUvJ1tcWA{L3sGw@*9CWQ~rT)W9}>W;Z0YyKdNI zCHB*}&ORUh`j9#uam3QGKmcTO3*{p*OnEz}xM3{CN8E$V%`;%IE^L_II_OM~alzWZ^BHc1dy{D@J_F zA-KEQ*gc~3ME3?NueG#%aSmur-=={liZkUrbh=T1EZf>UUNbYU@&lo;EOAo`2~$$9 z^uDNsOy1EX;{sZ);K#1uM>av{@%7=JLVY>6gzMrj6V(g)gyena|~LEhub{to`DiwOJFtKa`=T+)ZR8 zyy4#FDEF;8YopCGLhFWa@t3GRw@Dp>2WRsFm^}OY0aJ02K&P`(rrsRLc~`!4m(&MA zkV3g<0Xm`Ad(X`5uNm+9T<^z6h$JT%+1aq1Y}ii>$pH@rcI&y*liyeyf+wt9$^Nwk zqM{juTT(p^uOKF{%LzaxNu?EL4mBr2G?iXvqL-#K^{OwdG@06Fo8va-j$jkqS#L=i z`ZKYaLIpN&4OxYn48p&W^!13crxCtG-r;aUCTTzeMfB0nt2@k}7l|xKvg|lf)s3;?mY-wfOay_a?)#F^^lk z%8|e2=6zO2QG44U?$>#Sx^>z{nCnfZ7>W%O_h(9mo36U<39 z@L#$aaRGrBb=vn|y-w1@%~JtxPZhanoC5^evgyMo=Y!3s}1wG-R!c zrIA`SJsS;*u{eKbj8==g?5b9}W2%l@pUdyxW4)tqb7GQ+>L!Np;}t>^W+vN8H{U68 zaV2^^N%(Fj@hscXb;-*!f--|u3N+kXc8gHj$O zcMN>7m^?)rL&5ib&_{9;FrS90?9RqSJZc8ep-XU+`ppe(Y<%{!0L1wgN}PM3F-y+1 z929?#f{?o{KK7WM#rWGP;LCaJ!R>%+BGT>vRR8%6jxQ&WS|nDIFnDYF@BvC$6Z1*_ zUxj}OOP<2!Z}2`t|6c^}KOE@)b;0Xw?ey>N@Fi;6|GLBbuGJANhmEI~k_vVKfwIuJ zG_eY#aw4li6DO^|}DWzI)Bfk|Q` zv$k!i#+er*GPgxbaV})Z3~CXg?Yv4dB{%1{VWoF-L)hc2x1}~TX*Hol+d_ zM1kv42%y`M-mD?-CeDkM+qy$2{AZP}XhOe^{^Dc<-^|&O?Gpoe$}!(nvqPp1)mO^j z5d~zgpNvaRPHr#X+COlZ%M>R>TrC-?JSlZ^bK3l=meKC&V)#fj?@IRvn)ir-?BKO~ zxQ{nb-l!#}1}H0=J?i4=vYMu3e;Tr8kbZ8-qQ%WyD$#WUq2L-w=OY`k=u}84PkcuG zrp(m_Kv^XLN!4khn$%1)H1z5ns@-bELoTD2j?s*FVLWj97m-n;C8(!Bptrc1V9(b6 zJB?=|Qm!80q$bP3?_6yx*v(-5K!IaEMdarGLMbYh+6j5ei2A49RxSPaFS+@MUo|*9 zMzs`X&T0-cDh1=lK_wOA7LToCrlFiJ0gF+q$+bpEehj8)o@j=IpSscMn5tv%xlsrs~RrElTHdD(O!PZC709`?gb?Exb_QZt&Qb z&-p{mvQnq16H?HqHxTdoZkdop^@D}C*j{N*QQ@l`$+2P&`;O3T-lGSTNk;X#6O%J9l5%3b#)Y(f9#n&0f)lWkFm83l&v2|&)AtgoxVSBMrPi_ z(rSN`8ko?xxa=B;q-Q|sb&{}7nOA9CmcCdIXS~X8a(c~um8O2@y$eT&LV?rh^crI< z%6Z91$*i`pX?iaIm4GK+WNk%j+iq1Ye$i>eBa@Aj<1%iy8lWC*oHke*0Q@WNU=T|m zQ5J^^kgbX~jui_N&qe2J?^4jf&7m8#SHP1FCtO?e@-!YM>v>&dNc0MeCFX~qo%vWcG{+$qewbg(+_e+Hbz=huv14xRlQAcR!;S_&+aC*9H)p3fG$5P%z% z_zl5fT@xxs@OM%IM-Mnjy&x3X$lg9azgm$DWyVwsgo=gU2V{&r;~jb`|HMhlw+eO#ig4h zQHBBSkl}+vThe2&GFC4E%aPNKA{Ee9Ss9*A7Lr z7`V(Qto@+Lx?eRj>e#@0jTjUE(Za0h>Rqk{2j_9XP*SJQxX9;R^cR_!7L@6Kip)l{v#P~&^ zEY=dkqw^c&!Mj58|MX)0D-ooW{ntE-;U6cq|4pOz5BcwZbgus|Mr<}FHg=94`qn0f z|1Tk^|5L{DM#si!vlHn%SHFNzf139%n{2T(ZG3vFv{*_{hB)^ZMzO!Vjw2~I z`TlH1`YzgP@(an_+{#a)KUJ-2<(4mwBue+`m|s5rCJxumF`O?}E*IyXn|p0IZ@?z* z;BMNbHl8`_Gg#EkG}TO&VFU6`&RWwKt$jKJd2cj1u)BP*qZ>J;bYO+Yhx{IAl);2s zY1+HNZK=1xme$FL(c6rcNN^M`@QRklgBHJ8;q73XXcdN8#vJ;YX0503+D=%;mQgV` ze6#L@d<=gtZuN_p_e1)2$=^xS8Ka3t4E5W$WLjgBb4Xrm=S|2ri7dXD5$w zrMQ~|Bqs9`MY>OM;(yd>ogF=0zrLU7zaNBuhV~De-c1?Y*#?A-c+I zT36`s=gw+>MZvN4rkYoFF8n&Lz-ksbYZfFY90&FS@v6SMn(7_45yhg-Oac;mW+7gy-yM>z?@_v-QeNkKS1PvMI0 z;t>~D^kmH62WcuQ;afy$X^?)XbX-PERyws&oQ~fUVF}G^YQWkhO9vemc4Bzw3&jNRDJ`l$dU^ujd9Uc|Wrn1D03Co84mX9r|o*`qhP2BBgbKHF2iEyDN=6 zeaW!ILg89RC=Zg$*|5R~@oTWj6$C*1eEQY?BcmED?l|n&kRc2}IoUkoamIDqnwc(3 zT2YAfT&ntV8+2%!)7o-(d-ixqM11OGR1L*Om@r{w3(#r%g^_UekheLX#iGLo1MoT~ zCUhRr=P`oYmIU;x3Oh#yar-AuI^v!J>Um%V);(uozoWar3Tsm#hQJ%C5<9i(DMt&s zz>q^Gc5GUE3icimKMIW=-a@Y<1-Co;CK1d-gPhns{c=eM0Aic%5ef~~y4rk{tDx6H zbqG#M{e&a6`R@L0AFY@HK;M`PwkWh!UKBZ|&ag3f6GMnx#Vyo~PZMK%cRusCyfL(I z5pC4V4;;;4u9m=RX8^(Mx~@M!TZash_)E`KxTA2c4uMEa!= z_-x2gAjGLGgQJW?667wM&)O+oR!KaiHe!5knbuz2(>e0Jl5VdiUm*uuGG6V`cccZhxb7|Y zYQmsKn#1l=g5eM{1YTJ>%EU!#*O-@l~m`JN^O+-$LSZ&N>eJ!E( zk}CKZUD*_%sLETl)JPF`r8YF~6gm|qG9=ki;bD>HrX%3XXV5gEPh>JZYem&1gl-?QUJs{_`!U!7bs=0Mz{jf0mUzE!j4 zMBiCSe|v{-Q+Giln=^S+t260^Lo`(_$}MQgrkplPv!%?7-hr~VF0VIgT6<`Q(|T1E zO*I{N@cEoN{!FPpoSYmTNS7%CFi15j2x?jt(tHrO7%G`PLXp`PG3Rlzp*fLtrVCaQ zMOD_XT&VTe)TzLQtKYe^!aAJcnxH zj_%<&vJBST<$zNprEQ^3zs<0PWUHzabA@&nELQZ$>M~~m9U=-Nn#5TX?G1BLeOH?a zJZcsF>c_S^*7>Zx+It}Kek1xp2c%vz>rmkl6RO8af+BNoNZ$La3=i;YJTt%+=6%gF zZ)B1VpLv6yl5 zt02OOFNES26or@@hvUFF8m8eYYcwU2HxQnhq^B1>0MCrH^DKkC}Sm7pBqVZ=F zX0Q+Q&Av%Bcb8Uiqs>H*7hP-j8ZYjZ9CuN!v;2E)yyUwtN`FGCLoYpn4db5evRV7dY_o2*pVK8eY{s%8P4I` z(O!$T^rSj3>PZnHB`@08m?UYuZx@4j>(l%{)m?R1l-<)_q)|dxx~02IKnW?4PU-HJ z?v)a05EP`llu$|<2>}5?Qc7tF=|=b#-}vpS@WS_>%eB|VA8^k-=b1Az&zW=Pv`ldL zU^Nid$vGNS0f)%6{OnMhmacr!-a^hO5q-g+R>w6H(je}lbS(xM6Vs&`@d|Z~L3d|P zKx*`o37ib0gRzX>BH1KB1h3y8;>>UT(-Tn=Jle;5>K+_vVQR>Z(Z(uok|{*WWV@(7&ZSNr^DTu}cs!UC=u1zNjNN zZrSZ?(Oo>ANG?#7lgx6`j{YqR&3#FBH4nag9w{YX<^7X>{q|RpTehlr1AB5iK}?}s zHQgFF5zt)=xTxlYgE%RR=*oHV1dNvq1i9VJL|5Yr^jrMa?TPP>U8NlAw;NC*K9pLN zJpqOu$OM!vaQTc^>j(fd4hoV<$8@n*cX$F^@~+b&)P~(>hV@smxAGCpL6P>IPR`-j zBKbZ%f7Qn%lMcU>z|mG-=^<5tWZy)n0q{WbZ%Ftj@Dn3}5)H`930PztO0J z0dt*S@7?CU1GaB~xcpIweyr@AIi0`wV&mIl9Td8j!}La-iD19wuT3ItQ%q^Mzm3c9 zA`)jGMrobgYF?cc51!PCU8lRR;aXKkVMaw%vKV4L+F8k6Ih3@x_1LCjaChKo5$0OK z#Qp>ZKKR1;7t;QD&ORYx=w(_Knp^s#9~KwwXx=W|#BW$InU;UPAJuyu?(qq`MJD`H zY|^%z1t;|N6y>4ZCd<22ad&z{REBipQoc?uA4tQ{hz6d#h;N-*8$|I<1Zb%9$n?mw z^8)d;SNc_gx)3dIrmMm}Kz+}}I!XGhVt3`WL3HsVg9VpcZo8-qi8sTBU4)?}SIn>m zwZeMKPQu+I8|8EzRivU6Uwk3~1P<37ig;ly_)=tN8@4bX8K&d!uQxv2Qjg^)>L1hR zxQfnYpBpSHL*%VJC0`_b)O2rJVUWxsTY)2t=;I#l>GD^5Nq1-ic+r(Vhk!V zsm2$j-LcQOSME>A(^Wqig-s*-hVp8s`g{n6^t_p5ZZaOWV!You7lBYFwGt;ILLkz(Ku_oQfqPNnKUu`1n(GQH(bc z5#>w-qtm67n#xr~yRr{GI zO1-K`a`C2qhO!>x@-%geq# z(ujslYDJ#pz2H*$V%qoluEl<+D{a8{z9;D-BcSHIq#@?rgKkfc$K_mwO9>JdkEFG(=omNqHeSYi-I1f-&QZ@B zN9ZCg(kHrh0@>FMsc^9k6(X$<5=q4xe=J_Nk7k*ggBPzIQO=vaXLq;$Z*%^mCoctb zkE+vlRl5Inw>Eawx3;q~{>=sRvGOv~Z-p>wSBvl*?y~uFtDpq4w3nc{K48A&n>=PUYlbUp^jhG>NLU|-oDI#Nu zMf^OPLmzhkiqmzRDayT|>RTZ=MkRKNOO8zgH{ z6iO{&HZ1jol%c`rv~TX)3>IK7eQngkou5)usV0!W56cEVv5UM%ceCH$PgiDw->@4v z-;p&-G=(S7%|MPlKatAigrhq++hf~T!fkUf;VOuWdoq|?uf8?!&iN&C-Ud;eOJxd& zn9Unq8_mM)MOG;}a_rYdS$g3WU~@_{0w`fD{dUKZ_9pgMpY)g~e4N}L=etjS1e!9# z*)#?4ZpmD*xt?7lWj=G$@kJlrA8oLd7cf}%-&m}Se6`J6)@x(i*#0yg>-~AYIoHJa zjp$2(@Qu8V7Ha5z=`jbZ6k>D+X#RFH6T&x^va4-~3G?nr~%pTnLIuuD%>g$wWg@E0K1tPl}nR$YVNe6gn1rvW(0}N`{b# zJymv}??tc^jJgyt!fq#W<=Yyz!eDr9i1@$sL+R~ZxL$B$FC}ACJU1hncE@NX>OeP? z5jXCe5{}KcLQGms$|(+0{5TB6Qt9Md*FHHG74io4S{4vFZL;m(&wZ6?UwhrGT%vs4 zv{$L}z{v5(RYa7iCDDrj03ZTvPG_4E%jHbzcfwsg8v$=C*b3EJ6S&`F`w z;w>vi%^^eiO+F0A#k|j=v{_kCFbNj2fN8u-cV3c+N5?F#GTbkZK}e5LH!(3aC9niK zeyPi1y6sRtPa-GaE1y(CqBFhB7ZJDiE^BP#z|?Q&k#iT5+CuM}bXtq{BtNG$#EeP* z&D`ENlTm{eTy!y*^_60nR)3u2lD;&nI9BEmBs|@OkL{vdFx^)^x%T=;k!-W%CSiYi zIwMF9_f3xwVb{n$K(f${lR0%op;zWn9X3~O4y@v0bWQ3LgoH$eo8OohyR1U$DhS@8 z+}c=;=_HXtn30SQc-8hS|4zWCYS%Gbe1kUKbM~J7X_7^zQJsGgc7N6?@c`j(S7kGkywD~ffXTA=` zhE5I-%$<$TvELSD8Ot%&(KZ+ZPzNL^QeF>4JKc-tNZcJ>Xs3%fmW>yu zM^F_Hbc+KSkv)D$x{N!*6!TamH7ztcSYIv5%_=Po)_ovc16@Qvc|fB~ajtkUuVA%= zh5jhR>&83F)RzkU>RDec3&_%NiZ_Z+!9tTh22 z-iDQG_G|kH&`}9ZtgptOrzuZvjNA3eW*y0Pv@dUbX%u#S+i1Sc4QtH;-RbVdqDS7S*`Q%mLth)_WXMW3&-J3W99GH`^s&do zM)`afC9*9igRhTgM0SBA41AAsR#UZ^?W@cuE_&9P4TB=1S2PIUNm$yy5SqfZK2daM zb31GV_lFu!xYy>S84@vW;&5Ds-w{78yxP&QyOm}Bt-W&TJ2EBmE%AM>Cbk{DGElY@}ixt?-b2jR-v6v)b@7?P3aBM_mB-MDBqmCOe)~1i!o_7NRY&;OP*7% zXR8CXIv7q0bYRPUGR)5cayv9?2^O@uM6rDr9i+>6mPseeL88!$sbDK*6k`$Wx*gBn zT0|!AyWH(`wXjYV#6d6{|A^G=ieqFmP z;-82*P#1XuQQi(?Jz=_%DFh=cUo5EP zNEjnosPu<#C2;O=4CHLg##<1eX2bR^Rkf{nzIMLvb}*l*wMNasl*%IBCB{;9f7}k} ziSS6L_mRlt`)YD5|&FJ}WPMh|2%;LMNQK(?xAq z=Y>iNvwOAW%eiPO^0~m-o?jp85&G4t7n6f}Ls(jv5O@31*h-_Bl82;3YcF%aVZh{-6 z6XBm1a?d_Ud=c03N4rvd=p^=mv6acGCq2Cs*4SD_{Te{4=8f8^Bi)7p0&3$%&fj@(Nd@@;p`*UD17W^nK&D$%*uS$_HB?twdV7xt8sQ7Ux)u zy(uxQ=@b;B2(O*zKG!B2qfRhnCc>t{-b!!MPrl!HG+~=ghFQN(bP_?>h*cwb zl~E$3DK?c{GHMm6WVH}C%1)-Xc;z6GzW!SZLul?;jeDGx(_0 zOT^P%a=zklR5`IVQb&y=XvCRCSDbvW$Voaef=&<#k7#Juok&CyyAso_LKR`>VLSFB zH<4rPC*(SN{!eqeh1#85>W&+Jm9aL_zHO$%Z^fvO6LYB$V!jo8*-Bewi0g70l3io- zJ=*Jt5Jw*#1csWsVr^N_T1O2kQ?Ay&Z>eKPjA}x7kv&z`8%~UU!0v$k$X8}$0Ce}> zeN5F`9WNDkH)K_e)!El_5^@%^r=Lm4_<9MNEhjAxFTu3k;T<6@w5HHHx42j_MAG!L0C#85 zk|#B1S~xP=N0Y{b%wF6+G`O?!MA$AX&riPUd%ZyXv}m1q1IJ$3x;Oyci>L%(!vuN39RVdE0$)Ec=14zifL>4r#T& za2t2&tG;ReynM79$E%ZowPd{{dWJp=MeQ)IYN@vU$RJ_uumZ;ZEB0;K;0CLioMY76 z^o+H;WW^I$R7q`qG&kIYQ{0@_x?jDIS4vx*bi?OiLt8;_&hxnSxweh$-60xDY`K1% z^LR1r#A+oM|6U?7C#IJno^SSZk_DV9cCTWRjfq?E1|msWi)Cho@u8%Pn1RUl+sj(6 zOq4+>BBECOJNAR68Af+FL*Dcd4D}U83-x4`9Sp%WdbR|9l3}B6o3AO(=rtL^WN3mr zLZY~~q*{`X9-?$VOhFQv_8A%iaI}-b_Pw26WtXRTj4))kjbhl@zPj z__{Dgap*x=Pxc+1R}xso@(w=Bq5VMKFHEx~gkSS0aDY1cbFnm;jK^tnG6@rBH?8Rgh>b1ot! zN$oqQXpy-_^7Khwur4Ma%bPj}rM)84_CHqw#)3MbfgZ)WhmfM?ajI-qmZsJ7LSX^ z!MFLzwa%^jX9ZR__S)g>Epvc81}oCUH9ItO?Txs>hCq`4*;X9D+@G)v(CTgS+(fC~NsQ0hmT(;F(10B<>dOeEa>v(dY z`+4T%c`%pVu~A@;#38|S1zprI2X3A0?h_e11MI$oppB#`WTL2PBC;o0W`4JC*|{HE z`Yhq_JU9LF-G3>+lIdf%v-#ckX&wfynQI?tXt$z1KhWgh&CZX!^Rf38%8t}_uWO-{ z?x&g{be31i-?~2Fyspusc|ZVbaH|7I7O5uS*Yxl<7jC2Wwq9BEmnv%^AR#vC_p<8~ zL?_za=(eaB_->yJlG}SE=gnEuR~%~6wAY*SpQnWM^ho>gT-PLac*((;oNdykc?kdg zj${lecsq=YAB~Hl;Oj`bX=^6oeb6)S%^g8qjy}$s-95pwej(dlS7YrR?SWhEn%VId z({`Q$gmTF)B~5vs?Xl*LhH2hEbUjQpzrjTBo~i&NNO}!vD3$uX5waw=2~(@5Zv=VL zm7rA9$KeW6xKZ!?Y+XUkxnzFTHl>?koVaDY%s94vi?Yp~9)u|z%NcS7(KqzpRg9_w z5rn8F4keSLF`IUa)-?fSLf*KMZwdAE;;mFaat*(k9ogV$tuA_&AeXdGq<)=kI^5*A zFjz{uit~Q3_rto4d_P}vDg&8ZW4na6?jarSuJTu|#vt2IN9a_sRE&&M7AsEzZ_1{n z#;{1F-#}=AmrDp*jn~js_!74FytCcuWgx9q9iJ>J#_l_5-RsIg!}oRp)53>)OVf`! zzi!>GMSoiE?y)ED)G3Ob;)N2&<~65iO+C`;l1*V#MAFW6)1ELxg>aV~h;27#zBo|s zZDbJ76VwbBS&KRJRPT-w2a#WK>QG;^{9G;hlIn8Ss~WED2^~L;+L&utO5T-q*hy+5 zG`^s_f<^KH9|e=(QM36c-u1%mli94+_Oep~Ph3JyBEpt+Pm@rWFosVY;FEX4sOKf2 zr1Lv z(_N$K``p~oE1eSO#FfTP=iqz1?j#?)-dFllrgz}fM&6yF1foU0(9^%s+sRn?UTH)s zSOmwLaQoyj())_(w^PRS3rd+HKx&t-1;+QiCH>#&mZfO(Mm&v!UqrS;yfW)$b@iHy z1@?wSy{s2zAYKCs2{TExbIs~)WFO2=sU>n0onKbZ0fkeS~4Af5fis!K+l{`F1j=_$< ze1lU7vyDDF2{EtKBynuc&D7a>?_ngf6^CuaMm#*71j=W7p-F&;fF3G5pXnF_peuq2 zhenyK&}Sb{y5ok&*6ap>&5+ts(d1wS1%uoR8~yfJEDFa|%_q}0r%qoXjmXFUtJSCe=96_NVlUKM>O~6_)eK_84B@)? zyQV=cWMWq0I6sCGCa1>J z8yNwUXe^#*aU?etxQ>~Q%_+%9`5h{EW(Tc zbXXS-&`Q*JXW@Esh&+r}<7aaBR={1{cJ$7ju(j@FTJTk838Xik;ba`7e(g{nCp*Xq z5_Itu7$b2z`tcs>jmcS84z8-Wo_nt2?A5&gm>7SQ4-k^YkpeGl{`*pV(Wt#M{z+o; zKhj`eu3*E0H%Ea#KWhjN0DFtSK0+(Pr@#D23slsReyqxzP78VI(|-p7kib8u7DuTE zJ1hhKLtF3{1M=TN9RK{guCBR_xudS`4+H!Q>2!VPFQmVSNdGiLks*-23wMK&{vG&J zGTmSzr{D7+(4ZjT!9|UKA^ueg8s~pX`9667hye?o0)Gi1a56!{zu++Io~z&gKnFVr zBI@7)05pQ1M~VsTr+;uV&^NTSvvqL%?MVF(>1=v@NI$E|tb0LF_Z^}P3LpUBq6Po} zA#`2=QGqkbAM^~Z%#Cdv9~e7;C-@~Ol0u+mf)a>-rFaS8^}}#iMDY5m0hkozBVGYL zCi#b^t+k#0WyTO6ir_f;ZwOulQt2SW|09@M9|iz$9RhL%)OO=qmHJn97e!qydFrkLekHgf4glampss)hY5peo*JZ}GR#wJ_j^LQ_;1Z9S0s=7; z1mgOY=Ow{!G+4of$N>NWsxUuUq!TwJ-M`yzUV)g>rGyPaoGf2DXqP|+$<>oTz0LArjL5HVGS{mt&5 zBIeJY9@6U&KmcWe0+WAbcOK^NgU?k6%oSn#l)uUS^-g+T2*UQ=gU11TYOwPH;6Q?a z@}WV_b=1EPM3A2FyD%{IXGQ~CpP1nuQSoeXW zgSn0A|JrK&oH+bG9MVDHW`cV2&Ln*~5o7l&6l%Z|=(REcKn5XV1=N`jMg6Zx23|lp zjggln&j5j(37V}ri~Oa9JQLXgV1Xl(8dwF$aIymWQVT`@-$-&<>NHT)WgE|;eklQr zRxcY4!L*yO!HuLK1gwC>o1kc$+kmab$r_xToR$&YPMyU?yciV(W+o`U^(@ktfUo2) z!zKi$a|8$g05Jso3Mirtit>+GsPEtac{TLKOq>=1HWLKjaTe>#Nw|Jz)u$4iQ@vvS z*}{*(5`LtAfBCwA`hxh7M^>RzD`ada(!V{Mda-B;omx?mpT+x z&I%}u_P>?*>wwV3v|xci&jgLoT}q4p5(IgUgb_l}3TTA>95sG>VB})z@IXk(1Vu7l zOr1*#gxtT+1R-z*^o;2owSK=*|6+RaLWs!(t+HH9uggk=+-nY<+3&Ibsu<)R{EKM; zo!JwypR2{i;2}45LMLmO9KTZjrF?cV9}AtV8StFT`w}uBHv~YgcCLW5d7;SvT=~D) za6@{IOwc@dksNxh{hyTQ;^>g&OK1Zb6M-iD^Dg|04K%cYEP~HGoWb~F@TY|o$n`O7 zs9E$`{Ac)o>y3Yn@9zpLkmWb%b-PxHzefPF&UUc}1v!+NAV$ewdHpx$&$>dA%LL&{ eL16w pythonPath.map("local:" + _).mkString(File.pathSeparator), From 43273377a38a9136ff5e56929630930f076af5af Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Wed, 24 Aug 2016 16:00:04 -0700 Subject: [PATCH 1192/1470] [SPARKR][MINOR] Add more examples to window function docs ## What changes were proposed in this pull request? This PR adds more examples to window function docs to make them more accessible to the users. It also fixes default value issues for `lag` and `lead`. ## How was this patch tested? Manual test, R unit test. Author: Junyang Qian Closes #14779 from junyangq/SPARKR-FixWindowFunctionDocs. (cherry picked from commit 18708f76c366c6e01b5865981666e40d8642ac20) Signed-off-by: Felix Cheung --- R/pkg/R/WindowSpec.R | 12 +++++++ R/pkg/R/functions.R | 78 ++++++++++++++++++++++++++++++++++---------- 2 files changed, 72 insertions(+), 18 deletions(-) diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index ddd2ef2fcdee5..4ac83c29c6f7e 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -203,6 +203,18 @@ setMethod("rangeBetween", #' @aliases over,Column,WindowSpec-method #' @family colum_func #' @export +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' +#' # Partition by am (transmission) and order by hp (horsepower) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' +#' # Rank on hp within each partition +#' out <- select(df, over(rank(), ws), df$hp, df$am) +#' +#' # Lag mpg values by 1 row on the partition-and-ordered table +#' out <- select(df, over(lead(df$mpg), ws), df$mpg, df$hp, df$am) +#' } #' @note over since 2.0.0 setMethod("over", signature(x = "Column", window = "WindowSpec"), diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index f042adddef91f..dbf8dd89a204e 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -3121,9 +3121,9 @@ setMethod("ifelse", #' @aliases cume_dist,missing-method #' @export #' @examples \dontrun{ -#' df <- createDataFrame(iris) -#' ws <- orderBy(windowPartitionBy("Species"), "Sepal_Length") -#' out <- select(df, over(cume_dist(), ws), df$Sepal_Length, df$Species) +#' df <- createDataFrame(mtcars) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' out <- select(df, over(cume_dist(), ws), df$hp, df$am) #' } #' @note cume_dist since 1.6.0 setMethod("cume_dist", @@ -3148,7 +3148,11 @@ setMethod("cume_dist", #' @family window_funcs #' @aliases dense_rank,missing-method #' @export -#' @examples \dontrun{dense_rank()} +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' out <- select(df, over(dense_rank(), ws), df$hp, df$am) +#' } #' @note dense_rank since 1.6.0 setMethod("dense_rank", signature("missing"), @@ -3168,18 +3172,26 @@ setMethod("dense_rank", #' @param x the column as a character string or a Column to compute on. #' @param offset the number of rows back from the current row from which to obtain a value. #' If not specified, the default is 1. -#' @param defaultValue default to use when the offset row does not exist. +#' @param defaultValue (optional) default to use when the offset row does not exist. #' @param ... further arguments to be passed to or from other methods. #' @rdname lag #' @name lag #' @aliases lag,characterOrColumn-method #' @family window_funcs #' @export -#' @examples \dontrun{lag(df$c)} +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' +#' # Partition by am (transmission) and order by hp (horsepower) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' +#' # Lag mpg values by 1 row on the partition-and-ordered table +#' out <- select(df, over(lag(df$mpg), ws), df$mpg, df$hp, df$am) +#' } #' @note lag since 1.6.0 setMethod("lag", signature(x = "characterOrColumn"), - function(x, offset, defaultValue = NULL) { + function(x, offset = 1, defaultValue = NULL) { col <- if (class(x) == "Column") { x@jc } else { @@ -3194,25 +3206,35 @@ setMethod("lag", #' lead #' #' Window function: returns the value that is \code{offset} rows after the current row, and -#' NULL if there is less than \code{offset} rows after the current row. For example, -#' an \code{offset} of one will return the next row at any given point in the window partition. +#' \code{defaultValue} if there is less than \code{offset} rows after the current row. +#' For example, an \code{offset} of one will return the next row at any given point +#' in the window partition. #' #' This is equivalent to the \code{LEAD} function in SQL. #' -#' @param x Column to compute on -#' @param offset Number of rows to offset -#' @param defaultValue (Optional) default value to use +#' @param x the column as a character string or a Column to compute on. +#' @param offset the number of rows after the current row from which to obtain a value. +#' If not specified, the default is 1. +#' @param defaultValue (optional) default to use when the offset row does not exist. #' #' @rdname lead #' @name lead #' @family window_funcs #' @aliases lead,characterOrColumn,numeric-method #' @export -#' @examples \dontrun{lead(df$c)} +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' +#' # Partition by am (transmission) and order by hp (horsepower) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' +#' # Lead mpg values by 1 row on the partition-and-ordered table +#' out <- select(df, over(lead(df$mpg), ws), df$mpg, df$hp, df$am) +#' } #' @note lead since 1.6.0 setMethod("lead", signature(x = "characterOrColumn", offset = "numeric", defaultValue = "ANY"), - function(x, offset, defaultValue = NULL) { + function(x, offset = 1, defaultValue = NULL) { col <- if (class(x) == "Column") { x@jc } else { @@ -3239,7 +3261,15 @@ setMethod("lead", #' @aliases ntile,numeric-method #' @family window_funcs #' @export -#' @examples \dontrun{ntile(1)} +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' +#' # Partition by am (transmission) and order by hp (horsepower) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' +#' # Get ntile group id (1-4) for hp +#' out <- select(df, over(ntile(4), ws), df$hp, df$am) +#' } #' @note ntile since 1.6.0 setMethod("ntile", signature(x = "numeric"), @@ -3263,7 +3293,11 @@ setMethod("ntile", #' @family window_funcs #' @aliases percent_rank,missing-method #' @export -#' @examples \dontrun{percent_rank()} +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' out <- select(df, over(percent_rank(), ws), df$hp, df$am) +#' } #' @note percent_rank since 1.6.0 setMethod("percent_rank", signature("missing"), @@ -3288,7 +3322,11 @@ setMethod("percent_rank", #' @family window_funcs #' @aliases rank,missing-method #' @export -#' @examples \dontrun{rank()} +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' out <- select(df, over(rank(), ws), df$hp, df$am) +#' } #' @note rank since 1.6.0 setMethod("rank", signature(x = "missing"), @@ -3321,7 +3359,11 @@ setMethod("rank", #' @aliases row_number,missing-method #' @family window_funcs #' @export -#' @examples \dontrun{row_number()} +#' @examples \dontrun{ +#' df <- createDataFrame(mtcars) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' out <- select(df, over(row_number(), ws), df$hp, df$am) +#' } #' @note row_number since 1.6.0 setMethod("row_number", signature("missing"), From 9f363a690102f04a2a486853c1b89134455518bc Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Wed, 24 Aug 2016 16:04:14 -0700 Subject: [PATCH 1193/1470] [SPARKR][MINOR] Add installation message for remote master mode and improve other messages ## What changes were proposed in this pull request? This PR gives informative message to users when they try to connect to a remote master but don't have Spark package in their local machine. As a clarification, for now, automatic installation will only happen if they start SparkR in R console (rather than from sparkr-shell) and connect to local master. In the remote master mode, local Spark package is still needed, but we will not trigger the install.spark function because the versions have to match those on the cluster, which involves more user input. Instead, we here try to provide detailed message that may help the users. Some of the other messages have also been slightly changed. ## How was this patch tested? Manual test. Author: Junyang Qian Closes #14761 from junyangq/SPARK-16579-V1. (cherry picked from commit 3a60be4b15a5ab9b6e0c4839df99dac7738aa7fe) Signed-off-by: Felix Cheung --- R/pkg/R/install.R | 64 ++++++++++++++++++++++++++++++++--------------- R/pkg/R/sparkR.R | 51 ++++++++++++++++++++++++------------- R/pkg/R/utils.R | 4 +-- 3 files changed, 80 insertions(+), 39 deletions(-) diff --git a/R/pkg/R/install.R b/R/pkg/R/install.R index c6ed88e032a71..69b0a523b84e4 100644 --- a/R/pkg/R/install.R +++ b/R/pkg/R/install.R @@ -70,9 +70,9 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, localDir = NULL, overwrite = FALSE) { version <- paste0("spark-", packageVersion("SparkR")) hadoopVersion <- tolower(hadoopVersion) - hadoopVersionName <- hadoop_version_name(hadoopVersion) + hadoopVersionName <- hadoopVersionName(hadoopVersion) packageName <- paste(version, "bin", hadoopVersionName, sep = "-") - localDir <- ifelse(is.null(localDir), spark_cache_path(), + localDir <- ifelse(is.null(localDir), sparkCachePath(), normalizePath(localDir, mustWork = FALSE)) if (is.na(file.info(localDir)$isdir)) { @@ -88,12 +88,14 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, # can use dir.exists(packageLocalDir) under R 3.2.0 or later if (!is.na(file.info(packageLocalDir)$isdir) && !overwrite) { - fmt <- "Spark %s for Hadoop %s is found, and SPARK_HOME set to %s" + fmt <- "%s for Hadoop %s found, with SPARK_HOME set to %s" msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion), packageLocalDir) message(msg) Sys.setenv(SPARK_HOME = packageLocalDir) return(invisible(packageLocalDir)) + } else { + message("Spark not found in the cache directory. Installation will start.") } packageLocalPath <- paste0(packageLocalDir, ".tgz") @@ -102,7 +104,7 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, if (tarExists && !overwrite) { message("tar file found.") } else { - robust_download_tar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) + robustDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) } message(sprintf("Installing to %s", localDir)) @@ -116,33 +118,37 @@ install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, invisible(packageLocalDir) } -robust_download_tar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { +robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { # step 1: use user-provided url if (!is.null(mirrorUrl)) { msg <- sprintf("Use user-provided mirror site: %s.", mirrorUrl) message(msg) - success <- direct_download_tar(mirrorUrl, version, hadoopVersion, + success <- directDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) - if (success) return() + if (success) { + return() + } else { + message(paste0("Unable to download from mirrorUrl: ", mirrorUrl)) + } } else { - message("Mirror site not provided.") + message("MirrorUrl not provided.") } # step 2: use url suggested from apache website - message("Looking for site suggested from apache website...") - mirrorUrl <- get_preferred_mirror(version, packageName) + message("Looking for preferred site from apache website...") + mirrorUrl <- getPreferredMirror(version, packageName) if (!is.null(mirrorUrl)) { - success <- direct_download_tar(mirrorUrl, version, hadoopVersion, + success <- directDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) if (success) return() } else { - message("Unable to find suggested mirror site.") + message("Unable to find preferred mirror site.") } # step 3: use backup option message("To use backup site...") - mirrorUrl <- default_mirror_url() - success <- direct_download_tar(mirrorUrl, version, hadoopVersion, + mirrorUrl <- defaultMirrorUrl() + success <- directDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) if (success) { return(packageLocalPath) @@ -155,7 +161,7 @@ robust_download_tar <- function(mirrorUrl, version, hadoopVersion, packageName, } } -get_preferred_mirror <- function(version, packageName) { +getPreferredMirror <- function(version, packageName) { jsonUrl <- paste0("http://www.apache.org/dyn/closer.cgi?path=", file.path("spark", version, packageName), ".tgz&as_json=1") @@ -175,10 +181,10 @@ get_preferred_mirror <- function(version, packageName) { mirrorPreferred } -direct_download_tar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { +directDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { packageRemotePath <- paste0( file.path(mirrorUrl, version, packageName), ".tgz") - fmt <- paste("Downloading Spark %s for Hadoop %s from:\n- %s") + fmt <- "Downloading %s for Hadoop %s from:\n- %s" msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion), packageRemotePath) message(msg) @@ -192,11 +198,11 @@ direct_download_tar <- function(mirrorUrl, version, hadoopVersion, packageName, !isFail } -default_mirror_url <- function() { +defaultMirrorUrl <- function() { "http://www-us.apache.org/dist/spark" } -hadoop_version_name <- function(hadoopVersion) { +hadoopVersionName <- function(hadoopVersion) { if (hadoopVersion == "without") { "without-hadoop" } else if (grepl("^[0-9]+\\.[0-9]+$", hadoopVersion, perl = TRUE)) { @@ -208,7 +214,7 @@ hadoop_version_name <- function(hadoopVersion) { # The implementation refers to appdirs package: https://pypi.python.org/pypi/appdirs and # adapt to Spark context -spark_cache_path <- function() { +sparkCachePath <- function() { if (.Platform$OS.type == "windows") { winAppPath <- Sys.getenv("LOCALAPPDATA", unset = NA) if (is.na(winAppPath)) { @@ -231,3 +237,21 @@ spark_cache_path <- function() { } normalizePath(path, mustWork = FALSE) } + + +installInstruction <- function(mode) { + if (mode == "remote") { + paste0("Connecting to a remote Spark master. ", + "Please make sure Spark package is also installed in this machine.\n", + "- If there is one, set the path in sparkHome parameter or ", + "environment variable SPARK_HOME.\n", + "- If not, you may run install.spark function to do the job. ", + "Please make sure the Spark and the Hadoop versions ", + "match the versions on the cluster. ", + "SparkR package is compatible with Spark ", packageVersion("SparkR"), ".", + "If you need further help, ", + "contact the administrators of the cluster.") + } else { + stop(paste0("No instruction found for ", mode, " mode.")) + } +} diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 85815af1f3639..de53b0bf79b58 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -366,25 +366,10 @@ sparkR.session <- function( } overrideEnvs(sparkConfigMap, paramMap) } - # do not download if it is run in the sparkR shell - if (!nzchar(master) || is_master_local(master)) { - if (!is_sparkR_shell()) { - if (is.na(file.info(sparkHome)$isdir)) { - msg <- paste0("Spark not found in SPARK_HOME: ", - sparkHome, - " .\nTo search in the cache directory. ", - "Installation will start if not found.") - message(msg) - packageLocalDir <- install.spark() - sparkHome <- packageLocalDir - } else { - msg <- paste0("Spark package is found in SPARK_HOME: ", sparkHome) - message(msg) - } - } - } if (!exists(".sparkRjsc", envir = .sparkREnv)) { + retHome <- sparkCheckInstall(sparkHome, master) + if (!is.null(retHome)) sparkHome <- retHome sparkExecutorEnvMap <- new.env() sparkR.sparkContext(master, appName, sparkHome, sparkConfigMap, sparkExecutorEnvMap, sparkJars, sparkPackages) @@ -547,3 +532,35 @@ processSparkPackages <- function(packages) { } splittedPackages } + +# Utility function that checks and install Spark to local folder if not found +# +# Installation will not be triggered if it's called from sparkR shell +# or if the master url is not local +# +# @param sparkHome directory to find Spark package. +# @param master the Spark master URL, used to check local or remote mode. +# @return NULL if no need to update sparkHome, and new sparkHome otherwise. +sparkCheckInstall <- function(sparkHome, master) { + if (!isSparkRShell()) { + if (!is.na(file.info(sparkHome)$isdir)) { + msg <- paste0("Spark package found in SPARK_HOME: ", sparkHome) + message(msg) + NULL + } else { + if (!nzchar(master) || isMasterLocal(master)) { + msg <- paste0("Spark not found in SPARK_HOME: ", + sparkHome) + message(msg) + packageLocalDir <- install.spark() + packageLocalDir + } else { + msg <- paste0("Spark not found in SPARK_HOME: ", + sparkHome, "\n", installInstruction("remote")) + stop(msg) + } + } + } else { + NULL + } +} diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index d78c0a7a539a8..2809ce5d376a5 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -690,10 +690,10 @@ getSparkContext <- function() { sc } -is_master_local <- function(master) { +isMasterLocal <- function(master) { grepl("^local(\\[([0-9]+|\\*)\\])?$", master, perl = TRUE) } -is_sparkR_shell <- function() { +isSparkRShell <- function() { grepl(".*shell\\.R$", Sys.getenv("R_PROFILE_USER"), perl = TRUE) } From 3258f27a881dfeb5ab8bae90c338603fa4b6f9d8 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 24 Aug 2016 21:19:35 -0700 Subject: [PATCH 1194/1470] [SPARK-16216][SQL][BRANCH-2.0] Backport Read/write dateFormat/timestampFormat options for CSV and JSON ## What changes were proposed in this pull request? This PR backports https://github.com/apache/spark/pull/14279 to 2.0. ## How was this patch tested? Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases. Author: hyukjinkwon Closes #14799 from HyukjinKwon/SPARK-16216-json-csv-backport. --- python/pyspark/sql/readwriter.py | 56 +++++-- python/pyspark/sql/streaming.py | 30 +++- .../apache/spark/sql/DataFrameReader.scala | 17 +- .../apache/spark/sql/DataFrameWriter.scala | 12 ++ .../datasources/csv/CSVInferSchema.scala | 42 ++--- .../datasources/csv/CSVOptions.scala | 15 +- .../datasources/csv/CSVRelation.scala | 43 ++++- .../datasources/json/JSONOptions.scala | 9 + .../datasources/json/JacksonGenerator.scala | 14 +- .../datasources/json/JacksonParser.scala | 68 +++++--- .../datasources/json/JsonFileFormat.scala | 5 +- .../sql/streaming/DataStreamReader.scala | 16 +- .../datasources/csv/CSVInferSchemaSuite.scala | 4 +- .../execution/datasources/csv/CSVSuite.scala | 156 +++++++++++++++++- .../datasources/csv/CSVTypeCastSuite.scala | 17 +- .../datasources/json/JsonSuite.scala | 74 ++++++++- .../datasources/json/TestJsonData.scala | 6 + .../sources/JsonHadoopFsRelationSuite.scala | 4 + 18 files changed, 478 insertions(+), 110 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 64de33e8ec0a8..3da6f497e9f12 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -156,7 +156,7 @@ def load(self, path=None, format=None, schema=None, **options): def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, - mode=None, columnNameOfCorruptRecord=None): + mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None): """ Loads a JSON file (one object per line) or an RDD of Strings storing JSON objects (one object per record) and returns the result as a :class`DataFrame`. @@ -198,6 +198,14 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, ``spark.sql.columnNameOfCorruptRecord``. If None is set, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to date type. If None is set, it uses the + default value value, ``yyyy-MM-dd``. + :param timestampFormat: sets the string that indicates a timestamp format. Custom date + formats follow the formats at ``java.text.SimpleDateFormat``. + This applies to timestamp type. If None is set, it uses the + default value value, ``yyyy-MM-dd'T'HH:mm:ss.SSSZZ``. >>> df1 = spark.read.json('python/test_support/sql/people.json') >>> df1.dtypes @@ -213,7 +221,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, - mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord) + mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, + timestampFormat=timestampFormat) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -285,8 +294,8 @@ def text(self, paths): def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, - negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, - maxMalformedLogPerPartition=None, mode=None): + negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, + maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None): """Loads a CSV file and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -327,9 +336,12 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non is set, it uses the default value, ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This - applies to both date type and timestamp type. By default, it is None - which means trying to parse times and date by - ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. + applies to date type. If None is set, it uses the + default value value, ``yyyy-MM-dd``. + :param timestampFormat: sets the string that indicates a timestamp format. Custom date + formats follow the formats at ``java.text.SimpleDateFormat``. + This applies to timestamp type. If None is set, it uses the + default value value, ``yyyy-MM-dd'T'HH:mm:ss.SSSZZ``. :param maxColumns: defines a hard limit of how many columns a record can have. If None is set, it uses the default value, ``20480``. :param maxCharsPerColumn: defines the maximum number of characters allowed for any given @@ -356,7 +368,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, - dateFormat=dateFormat, maxColumns=maxColumns, maxCharsPerColumn=maxCharsPerColumn, + dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns, + maxCharsPerColumn=maxCharsPerColumn, maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode) if isinstance(path, basestring): path = [path] @@ -571,7 +584,7 @@ def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options) self._jwrite.saveAsTable(name) @since(1.4) - def json(self, path, mode=None, compression=None): + def json(self, path, mode=None, compression=None, dateFormat=None, timestampFormat=None): """Saves the content of the :class:`DataFrame` in JSON format at the specified path. :param path: the path in any Hadoop supported file system @@ -584,11 +597,20 @@ def json(self, path, mode=None, compression=None): :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to date type. If None is set, it uses the + default value value, ``yyyy-MM-dd``. + :param timestampFormat: sets the string that indicates a timestamp format. Custom date + formats follow the formats at ``java.text.SimpleDateFormat``. + This applies to timestamp type. If None is set, it uses the + default value value, ``yyyy-MM-dd'T'HH:mm:ss.SSSZZ``. >>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) - self._set_opts(compression=compression) + self._set_opts( + compression=compression, dateFormat=dateFormat, timestampFormat=timestampFormat) self._jwrite.json(path) @since(1.4) @@ -634,7 +656,8 @@ def text(self, path, compression=None): @since(2.0) def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, - header=None, nullValue=None, escapeQuotes=None, quoteAll=None): + header=None, nullValue=None, escapeQuotes=None, quoteAll=None, dateFormat=None, + timestampFormat=None): """Saves the content of the :class:`DataFrame` in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -666,12 +689,21 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses the default value, empty string. + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to date type. If None is set, it uses the + default value value, ``yyyy-MM-dd``. + :param timestampFormat: sets the string that indicates a timestamp format. Custom date + formats follow the formats at ``java.text.SimpleDateFormat``. + This applies to timestamp type. If None is set, it uses the + default value value, ``yyyy-MM-dd'T'HH:mm:ss.SSSZZ``. >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) self._set_opts(compression=compression, sep=sep, quote=quote, escape=escape, header=header, - nullValue=nullValue, escapeQuotes=escapeQuotes, quoteAll=quoteAll) + nullValue=nullValue, escapeQuotes=escapeQuotes, quoteAll=quoteAll, + dateFormat=dateFormat, timestampFormat=timestampFormat) self._jwrite.csv(path) @since(1.5) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index a364555003027..3761d2b1994f0 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -338,7 +338,8 @@ def load(self, path=None, format=None, schema=None, **options): def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, - mode=None, columnNameOfCorruptRecord=None): + mode=None, columnNameOfCorruptRecord=None, dateFormat=None, + timestampFormat=None): """ Loads a JSON file stream (one object per line) and returns a :class`DataFrame`. @@ -381,6 +382,14 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, ``spark.sql.columnNameOfCorruptRecord``. If None is set, it uses the value specified in ``spark.sql.columnNameOfCorruptRecord``. + :param dateFormat: sets the string that indicates a date format. Custom date formats + follow the formats at ``java.text.SimpleDateFormat``. This + applies to date type. If None is set, it uses the + default value value, ``yyyy-MM-dd``. + :param timestampFormat: sets the string that indicates a timestamp format. Custom date + formats follow the formats at ``java.text.SimpleDateFormat``. + This applies to timestamp type. If None is set, it uses the + default value value, ``yyyy-MM-dd'T'HH:mm:ss.SSSZZ``. >>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema) >>> json_sdf.isStreaming @@ -393,7 +402,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames, allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero, allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter, - mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord) + mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, + timestampFormat=timestampFormat) if isinstance(path, basestring): return self._df(self._jreader.json(path)) else: @@ -450,8 +460,8 @@ def text(self, path): def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None, - negativeInf=None, dateFormat=None, maxColumns=None, maxCharsPerColumn=None, - maxMalformedLogPerPartition=None, mode=None): + negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, + maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None): """Loads a CSV file stream and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -494,9 +504,12 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non is set, it uses the default value, ``Inf``. :param dateFormat: sets the string that indicates a date format. Custom date formats follow the formats at ``java.text.SimpleDateFormat``. This - applies to both date type and timestamp type. By default, it is None - which means trying to parse times and date by - ``java.sql.Timestamp.valueOf()`` and ``java.sql.Date.valueOf()``. + applies to date type. If None is set, it uses the + default value value, ``yyyy-MM-dd``. + :param timestampFormat: sets the string that indicates a timestamp format. Custom date + formats follow the formats at ``java.text.SimpleDateFormat``. + This applies to timestamp type. If None is set, it uses the + default value value, ``yyyy-MM-dd'T'HH:mm:ss.SSSZZ``. :param maxColumns: defines a hard limit of how many columns a record can have. If None is set, it uses the default value, ``20480``. :param maxCharsPerColumn: defines the maximum number of characters allowed for any given @@ -521,7 +534,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue, nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, - dateFormat=dateFormat, maxColumns=maxColumns, maxCharsPerColumn=maxCharsPerColumn, + dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns, + maxCharsPerColumn=maxCharsPerColumn, maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode) if isinstance(path, basestring): return self._df(self._jreader.csv(path)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index e8c2885d7737c..083c2e2c65a03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -280,6 +280,12 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *

  • `columnNameOfCorruptRecord` (default is the value specified in * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
  • + *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
  • + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • * * @since 2.0.0 */ @@ -376,10 +382,13 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * value. *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity * value.
  • - *
  • `dateFormat` (default `null`): sets the string that indicates a date format. Custom date - * formats follow the formats at `java.text.SimpleDateFormat`. This applies to both date type - * and timestamp type. By default, it is `null` which means trying to parse times and date by - * `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()`.
  • + *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
  • + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()` or ISO 8601 format. *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns * a record can have.
  • *
  • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 12b304623d30b..767af99b4d3ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -452,6 +452,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
  • + *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
  • + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • * * @since 1.4.0 */ @@ -544,6 +550,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
  • + *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
  • + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index de3d889621b7d..f1b4c11878a94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -139,20 +139,14 @@ private[csv] object CSVInferSchema { } private def tryParseTimestamp(field: String, options: CSVOptions): DataType = { - if (options.dateFormat != null) { - // This case infers a custom `dataFormat` is set. - if ((allCatch opt options.dateFormat.parse(field)).isDefined) { - TimestampType - } else { - tryParseBoolean(field, options) - } - } else { + // This case infers a custom `dataFormat` is set. + if ((allCatch opt options.timestampFormat.parse(field)).isDefined) { + TimestampType + } else if ((allCatch opt DateTimeUtils.stringToTime(field)).isDefined) { // We keep this for backwords competibility. - if ((allCatch opt DateTimeUtils.stringToTime(field)).isDefined) { - TimestampType - } else { - tryParseBoolean(field, options) - } + TimestampType + } else { + tryParseBoolean(field, options) } } @@ -277,18 +271,24 @@ private[csv] object CSVTypeCast { val value = new BigDecimal(datum.replaceAll(",", "")) Decimal(value, dt.precision, dt.scale) } - case _: TimestampType if options.dateFormat != null => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681. - options.dateFormat.parse(datum).getTime * 1000L case _: TimestampType => // This one will lose microseconds parts. // See https://issues.apache.org/jira/browse/SPARK-10681. - DateTimeUtils.stringToTime(datum).getTime * 1000L - case _: DateType if options.dateFormat != null => - DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime) + Try(options.timestampFormat.parse(datum).getTime * 1000L) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + DateTimeUtils.stringToTime(datum).getTime * 1000L + } case _: DateType => - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) + // This one will lose microseconds parts. + // See https://issues.apache.org/jira/browse/SPARK-10681.x + Try(DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime)) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) + } case _: StringType => UTF8String.fromString(datum) case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 10fe541a2c575..364d7c831eb44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution.datasources.csv import java.nio.charset.StandardCharsets -import java.text.SimpleDateFormat + +import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.{CompressionCodecs, ParseModes} @@ -101,11 +102,13 @@ private[csv] class CSVOptions(@transient private val parameters: Map[String, Str name.map(CompressionCodecs.getCodecClassName) } - // Share date format object as it is expensive to parse date pattern. - val dateFormat: SimpleDateFormat = { - val dateFormat = parameters.get("dateFormat") - dateFormat.map(new SimpleDateFormat(_)).orNull - } + // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. + val dateFormat: FastDateFormat = + FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd")) + + val timestampFormat: FastDateFormat = + FastDateFormat.getInstance( + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ")) val maxColumns = getInt("maxColumns", 20480) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 10d84f4a70d5d..d0d5ce06cf8b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -30,6 +30,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.types._ @@ -179,6 +180,14 @@ private[csv] class CsvOutputWriter( // create the Generator without separator inserted between 2 records private[this] val text = new Text() + // A `ValueConverter` is responsible for converting a value of an `InternalRow` to `String`. + // When the value is null, this converter should not be called. + private type ValueConverter = (InternalRow, Int) => String + + // `ValueConverter`s for all values in the fields of the schema + private val valueConverters: Array[ValueConverter] = + dataSchema.map(_.dataType).map(makeConverter).toArray + private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { @@ -195,18 +204,40 @@ private[csv] class CsvOutputWriter( private var records: Long = 0L private val csvWriter = new LineCsvWriter(params, dataSchema.fieldNames.toSeq) - private def rowToString(row: Seq[Any]): Seq[String] = row.map { field => - if (field != null) { - field.toString - } else { - params.nullValue + private def rowToString(row: InternalRow): Seq[String] = { + var i = 0 + val values = new Array[String](row.numFields) + while (i < row.numFields) { + if (!row.isNullAt(i)) { + values(i) = valueConverters(i).apply(row, i) + } else { + values(i) = params.nullValue + } + i += 1 } + values + } + + private def makeConverter(dataType: DataType): ValueConverter = dataType match { + case DateType => + (row: InternalRow, ordinal: Int) => + params.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) + + case TimestampType => + (row: InternalRow, ordinal: Int) => + params.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) + + case udt: UserDefinedType[_] => makeConverter(udt.sqlType) + + case dt: DataType => + (row: InternalRow, ordinal: Int) => + row.get(ordinal, dt).toString } override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") override protected[sql] def writeInternal(row: InternalRow): Unit = { - csvWriter.writeRow(rowToString(row.toSeq(dataSchema)), records == 0L && params.headerFlag) + csvWriter.writeRow(rowToString(row), records == 0L && params.headerFlag) records += 1 if (records % FLUSH_BATCH_SIZE == 0) { flush() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONOptions.scala index 66f1126fb9ae6..02d211d04265e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONOptions.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.json import com.fasterxml.jackson.core.{JsonFactory, JsonParser} +import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.{CompressionCodecs, ParseModes} @@ -53,6 +54,14 @@ private[sql] class JSONOptions( private val parseMode = parameters.getOrElse("mode", "PERMISSIVE") val columnNameOfCorruptRecord = parameters.get("columnNameOfCorruptRecord") + // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. + val dateFormat: FastDateFormat = + FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd")) + + val timestampFormat: FastDateFormat = + FastDateFormat.getInstance( + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ")) + // Parse mode flags if (!ParseModes.isValidMode(parseMode)) { logWarning(s"$parseMode is not a valid parse mode. Using ${ParseModes.DEFAULT}.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala index 8b920ecafaeed..800d43f3039c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala @@ -32,11 +32,17 @@ private[sql] object JacksonGenerator { * @param gen a JsonGenerator object * @param row The row to convert */ - def apply(rowSchema: StructType, gen: JsonGenerator)(row: InternalRow): Unit = { + def apply( + rowSchema: StructType, + gen: JsonGenerator, + options: JSONOptions = new JSONOptions(Map.empty[String, String])) + (row: InternalRow): Unit = { def valWriter: (DataType, Any) => Unit = { case (_, null) | (NullType, _) => gen.writeNull() case (StringType, v) => gen.writeString(v.toString) - case (TimestampType, v: Long) => gen.writeString(DateTimeUtils.toJavaTimestamp(v).toString) + case (TimestampType, v: Long) => + val timestampString = options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(v)) + gen.writeString(timestampString) case (IntegerType, v: Int) => gen.writeNumber(v) case (ShortType, v: Short) => gen.writeNumber(v) case (FloatType, v: Float) => gen.writeNumber(v) @@ -46,7 +52,9 @@ private[sql] object JacksonGenerator { case (ByteType, v: Byte) => gen.writeNumber(v.toInt) case (BinaryType, v: Array[Byte]) => gen.writeBinary(v) case (BooleanType, v: Boolean) => gen.writeBoolean(v) - case (DateType, v: Int) => gen.writeString(DateTimeUtils.toJavaDate(v).toString) + case (DateType, v: Int) => + val dateString = options.dateFormat.format(DateTimeUtils.toJavaDate(v)) + gen.writeString(dateString) // For UDT values, they should be in the SQL type's corresponding value type. // We should not see values in the user-defined class at here. // For example, VectorUDT's SQL type is an array of double. So, we should expect that v is diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala index 733fcbfea101e..a5417dc4a0e19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.json import java.io.ByteArrayOutputStream import scala.collection.mutable.ArrayBuffer +import scala.util.Try import com.fasterxml.jackson.core._ @@ -56,28 +57,30 @@ object JacksonParser extends Logging { def convertRootField( factory: JsonFactory, parser: JsonParser, - schema: DataType): Any = { + schema: DataType, + configOptions: JSONOptions): Any = { import com.fasterxml.jackson.core.JsonToken._ (parser.getCurrentToken, schema) match { case (START_ARRAY, st: StructType) => // SPARK-3308: support reading top level JSON arrays and take every element // in such an array as a row - convertArray(factory, parser, st) + convertArray(factory, parser, st, configOptions) case (START_OBJECT, ArrayType(st, _)) => // the business end of SPARK-3308: // when an object is found but an array is requested just wrap it in a list - convertField(factory, parser, st) :: Nil + convertField(factory, parser, st, configOptions) :: Nil case _ => - convertField(factory, parser, schema) + convertField(factory, parser, schema, configOptions) } } private def convertField( factory: JsonFactory, parser: JsonParser, - schema: DataType): Any = { + schema: DataType, + configOptions: JSONOptions): Any = { import com.fasterxml.jackson.core.JsonToken._ (parser.getCurrentToken, schema) match { case (null | VALUE_NULL, _) => @@ -85,7 +88,7 @@ object JacksonParser extends Logging { case (FIELD_NAME, _) => parser.nextToken() - convertField(factory, parser, schema) + convertField(factory, parser, schema, configOptions) case (VALUE_STRING, StringType) => UTF8String.fromString(parser.getText) @@ -99,19 +102,29 @@ object JacksonParser extends Logging { case (VALUE_STRING, DateType) => val stringValue = parser.getText - if (stringValue.contains("-")) { - // The format of this string will probably be "yyyy-mm-dd". - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime) - } else { - // In Spark 1.5.0, we store the data as number of days since epoch in string. - // So, we just convert it to Int. - stringValue.toInt - } + // This one will lose microseconds parts. + // See https://issues.apache.org/jira/browse/SPARK-10681.x + Try(DateTimeUtils.millisToDays(configOptions.dateFormat.parse(parser.getText).getTime)) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime)) + .getOrElse { + // In Spark 1.5.0, we store the data as number of days since epoch in string. + // So, we just convert it to Int. + stringValue.toInt + } + } case (VALUE_STRING, TimestampType) => // This one will lose microseconds parts. // See https://issues.apache.org/jira/browse/SPARK-10681. - DateTimeUtils.stringToTime(parser.getText).getTime * 1000L + Try(configOptions.timestampFormat.parse(parser.getText).getTime * 1000L) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + DateTimeUtils.stringToTime(parser.getText).getTime * 1000L + } case (VALUE_NUMBER_INT, TimestampType) => parser.getLongValue * 1000000L @@ -179,16 +192,16 @@ object JacksonParser extends Logging { false case (START_OBJECT, st: StructType) => - convertObject(factory, parser, st) + convertObject(factory, parser, st, configOptions) case (START_ARRAY, ArrayType(st, _)) => - convertArray(factory, parser, st) + convertArray(factory, parser, st, configOptions) case (START_OBJECT, MapType(StringType, kt, _)) => - convertMap(factory, parser, kt) + convertMap(factory, parser, kt, configOptions) case (_, udt: UserDefinedType[_]) => - convertField(factory, parser, udt.sqlType) + convertField(factory, parser, udt.sqlType, configOptions) case (token, dataType) => // We cannot parse this token based on the given data type. So, we throw a @@ -207,12 +220,13 @@ object JacksonParser extends Logging { private def convertObject( factory: JsonFactory, parser: JsonParser, - schema: StructType): InternalRow = { + schema: StructType, + configOptions: JSONOptions): InternalRow = { val row = new GenericMutableRow(schema.length) while (nextUntil(parser, JsonToken.END_OBJECT)) { schema.getFieldIndex(parser.getCurrentName) match { case Some(index) => - row.update(index, convertField(factory, parser, schema(index).dataType)) + row.update(index, convertField(factory, parser, schema(index).dataType, configOptions)) case None => parser.skipChildren() @@ -228,12 +242,13 @@ object JacksonParser extends Logging { private def convertMap( factory: JsonFactory, parser: JsonParser, - valueType: DataType): MapData = { + valueType: DataType, + configOptions: JSONOptions): MapData = { val keys = ArrayBuffer.empty[UTF8String] val values = ArrayBuffer.empty[Any] while (nextUntil(parser, JsonToken.END_OBJECT)) { keys += UTF8String.fromString(parser.getCurrentName) - values += convertField(factory, parser, valueType) + values += convertField(factory, parser, valueType, configOptions) } ArrayBasedMapData(keys.toArray, values.toArray) } @@ -241,10 +256,11 @@ object JacksonParser extends Logging { private def convertArray( factory: JsonFactory, parser: JsonParser, - elementType: DataType): ArrayData = { + elementType: DataType, + configOptions: JSONOptions): ArrayData = { val values = ArrayBuffer.empty[Any] while (nextUntil(parser, JsonToken.END_ARRAY)) { - values += convertField(factory, parser, elementType) + values += convertField(factory, parser, elementType, configOptions) } new GenericArrayData(values.toArray) @@ -285,7 +301,7 @@ object JacksonParser extends Logging { Utils.tryWithResource(factory.createParser(record)) { parser => parser.nextToken() - convertRootField(factory, parser, schema) match { + convertRootField(factory, parser, schema, configOptions) match { case null => failedRecord(record) case row: InternalRow => row :: Nil case array: ArrayData => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index c58de3ae6f9e6..decbddaac2f59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -85,7 +85,7 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { bucketId: Option[Int], dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new JsonOutputWriter(path, bucketId, dataSchema, context) + new JsonOutputWriter(path, parsedOptions, bucketId, dataSchema, context) } } } @@ -155,6 +155,7 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { private[json] class JsonOutputWriter( path: String, + options: JSONOptions, bucketId: Option[Int], dataSchema: StructType, context: TaskAttemptContext) @@ -181,7 +182,7 @@ private[json] class JsonOutputWriter( override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") override protected[sql] def writeInternal(row: InternalRow): Unit = { - JacksonGenerator(dataSchema, gen)(row) + JacksonGenerator(dataSchema, gen, options)(row) gen.flush() result.set(writer.toString) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 2e606b21bdf30..e0a19b1da55a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -186,6 +186,12 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
  • `columnNameOfCorruptRecord` (default is the value specified in * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
  • + *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
  • + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • * * @since 2.0.0 */ @@ -228,10 +234,12 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * value. *
  • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity * value.
  • - *
  • `dateFormat` (default `null`): sets the string that indicates a date format. Custom date - * formats follow the formats at `java.text.SimpleDateFormat`. This applies to both date type - * and timestamp type. By default, it is `null` which means trying to parse times and date by - * `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()`.
  • + *
  • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
  • + *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • *
  • `maxColumns` (default `20480`): defines a hard limit of how many columns * a record can have.
  • *
  • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala index dbe3af49c90c3..5e00f669b8593 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala @@ -60,9 +60,9 @@ class CSVInferSchemaSuite extends SparkFunSuite { } test("Timestamp field types are inferred correctly via custom data format") { - var options = new CSVOptions(Map("dateFormat" -> "yyyy-mm")) + var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm")) assert(CSVInferSchema.inferField(TimestampType, "2015-08", options) == TimestampType) - options = new CSVOptions(Map("dateFormat" -> "yyyy")) + options = new CSVOptions(Map("timestampFormat" -> "yyyy")) assert(CSVInferSchema.inferField(TimestampType, "2015", options) == TimestampType) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 8cd76ddf20f04..f68d220afb6da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -22,6 +22,7 @@ import java.nio.charset.UnsupportedCharsetException import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat +import org.apache.commons.lang3.time.FastDateFormat import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec @@ -477,7 +478,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val options = Map( "header" -> "true", "inferSchema" -> "true", - "dateFormat" -> "dd/MM/yyyy hh:mm") + "timestampFormat" -> "dd/MM/yyyy HH:mm") val results = spark.read .format("csv") .options(options) @@ -485,7 +486,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .select("date") .collect() - val dateFormat = new SimpleDateFormat("dd/MM/yyyy hh:mm") + val dateFormat = new SimpleDateFormat("dd/MM/yyyy HH:mm") val expected = Seq(Seq(new Timestamp(dateFormat.parse("26/08/2015 18:00").getTime)), Seq(new Timestamp(dateFormat.parse("27/10/2014 18:30").getTime)), @@ -691,4 +692,155 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { verifyCars(cars, withHeader = true, checkValues = false) } + + test("Write timestamps correctly in ISO8601 format by default") { + withTempDir { dir => + val iso8601timestampsPath = s"${dir.getCanonicalPath}/iso8601timestamps.csv" + val timestamps = spark.read + .format("csv") + .option("inferSchema", "true") + .option("header", "true") + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + timestamps.write + .format("csv") + .option("header", "true") + .save(iso8601timestampsPath) + + // This will load back the timestamps as string. + val iso8601Timestamps = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "false") + .load(iso8601timestampsPath) + + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ") + val expectedTimestamps = timestamps.collect().map { r => + // This should be ISO8601 formatted string. + Row(iso8501.format(r.toSeq.head)) + } + + checkAnswer(iso8601Timestamps, expectedTimestamps) + } + } + + test("Write dates correctly in ISO8601 format by default") { + withTempDir { dir => + val customSchema = new StructType(Array(StructField("date", DateType, true))) + val iso8601datesPath = s"${dir.getCanonicalPath}/iso8601dates.csv" + val dates = spark.read + .format("csv") + .schema(customSchema) + .option("header", "true") + .option("inferSchema", "false") + .option("dateFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + dates.write + .format("csv") + .option("header", "true") + .save(iso8601datesPath) + + // This will load back the dates as string. + val iso8601dates = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "false") + .load(iso8601datesPath) + + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd") + val expectedDates = dates.collect().map { r => + // This should be ISO8601 formatted string. + Row(iso8501.format(r.toSeq.head)) + } + + checkAnswer(iso8601dates, expectedDates) + } + } + + test("Roundtrip in reading and writing timestamps") { + withTempDir { dir => + val iso8601timestampsPath = s"${dir.getCanonicalPath}/iso8601timestamps.csv" + val timestamps = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(datesFile)) + + timestamps.write + .format("csv") + .option("header", "true") + .save(iso8601timestampsPath) + + val iso8601timestamps = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(iso8601timestampsPath) + + checkAnswer(iso8601timestamps, timestamps) + } + } + + test("Write dates correctly with dateFormat option") { + val customSchema = new StructType(Array(StructField("date", DateType, true))) + withTempDir { dir => + // With dateFormat option. + val datesWithFormatPath = s"${dir.getCanonicalPath}/datesWithFormat.csv" + val datesWithFormat = spark.read + .format("csv") + .schema(customSchema) + .option("header", "true") + .option("dateFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + datesWithFormat.write + .format("csv") + .option("header", "true") + .option("dateFormat", "yyyy/MM/dd") + .save(datesWithFormatPath) + + // This will load back the dates as string. + val stringDatesWithFormat = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "false") + .load(datesWithFormatPath) + val expectedStringDatesWithFormat = Seq( + Row("2015/08/26"), + Row("2014/10/27"), + Row("2016/01/28")) + + checkAnswer(stringDatesWithFormat, expectedStringDatesWithFormat) + } + } + + test("Write timestamps correctly with dateFormat option") { + withTempDir { dir => + // With dateFormat option. + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.csv" + val timestampsWithFormat = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + timestampsWithFormat.write + .format("csv") + .option("header", "true") + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .save(timestampsWithFormatPath) + + // This will load back the timestamps as string. + val stringTimestampsWithFormat = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "false") + .load(timestampsWithFormatPath) + val expectedStringTimestampsWithFormat = Seq( + Row("2015/08/26 18:00"), + Row("2014/10/27 18:30"), + Row("2016/01/28 20:00")) + + checkAnswer(stringTimestampsWithFormat, expectedStringTimestampsWithFormat) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala index 26b33b24efc3d..3ce643e667ce4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala @@ -96,13 +96,18 @@ class CSVTypeCastSuite extends SparkFunSuite { assert(CSVTypeCast.castTo("1.00", DoubleType) == 1.0) assert(CSVTypeCast.castTo("true", BooleanType) == true) - val options = CSVOptions("dateFormat", "dd/MM/yyyy hh:mm") + val timestampsOptions = CSVOptions("timestampFormat", "dd/MM/yyyy hh:mm") val customTimestamp = "31/01/2015 00:00" - val expectedTime = options.dateFormat.parse("31/01/2015 00:00").getTime - assert(CSVTypeCast.castTo(customTimestamp, TimestampType, nullable = true, options) == - expectedTime * 1000L) - assert(CSVTypeCast.castTo(customTimestamp, DateType, nullable = true, options) == - DateTimeUtils.millisToDays(expectedTime)) + val expectedTime = timestampsOptions.timestampFormat.parse(customTimestamp).getTime + val castedTimestamp = + CSVTypeCast.castTo(customTimestamp, TimestampType, nullable = true, timestampsOptions) + assert(castedTimestamp == expectedTime * 1000L) + + val customDate = "31/01/2015" + val dateOptions = CSVOptions("dateFormat", "dd/MM/yyyy") + val expectedDate = dateOptions.dateFormat.parse(customDate).getTime + val castedDate = CSVTypeCast.castTo(customTimestamp, DateType, nullable = true, dateOptions) + assert(castedDate == DateTimeUtils.millisToDays(expectedDate)) val timestamp = "2015-01-01 00:00:00" assert(CSVTypeCast.castTo(timestamp, TimestampType) == diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 0b0e64ac72730..1ba5b81231178 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -64,9 +64,10 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { generator.flush() } + val dummyOption = new JSONOptions(Map.empty[String, String]) Utils.tryWithResource(factory.createParser(writer.toString)) { parser => parser.nextToken() - JacksonParser.convertRootField(factory, parser, dataType) + JacksonParser.convertRootField(factory, parser, dataType, dummyOption) } } @@ -99,15 +100,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { DateTimeUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType)) val ISO8601Time1 = "1970-01-01T01:00:01.0Z" - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(3601000)), - enforceCorrectType(ISO8601Time1, TimestampType)) - checkTypePromotion(DateTimeUtils.millisToDays(3601000), - enforceCorrectType(ISO8601Time1, DateType)) val ISO8601Time2 = "1970-01-01T02:00:01-01:00" + checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(3601000)), + enforceCorrectType(ISO8601Time1, TimestampType)) checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(10801000)), - enforceCorrectType(ISO8601Time2, TimestampType)) - checkTypePromotion(DateTimeUtils.millisToDays(10801000), - enforceCorrectType(ISO8601Time2, DateType)) + enforceCorrectType(ISO8601Time2, TimestampType)) + + val ISO8601Date = "1970-01-01" + checkTypePromotion(DateTimeUtils.millisToDays(32400000), + enforceCorrectType(ISO8601Date, DateType)) } test("Get compatible type") { @@ -1662,4 +1663,61 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(df.schema.size === 2) df.collect() } + + test("Write dates correctly with dateFormat option") { + val customSchema = new StructType(Array(StructField("date", DateType, true))) + withTempDir { dir => + // With dateFormat option. + val datesWithFormatPath = s"${dir.getCanonicalPath}/datesWithFormat.json" + val datesWithFormat = spark.read + .schema(customSchema) + .option("dateFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + + datesWithFormat.write + .format("json") + .option("dateFormat", "yyyy/MM/dd") + .save(datesWithFormatPath) + + // This will load back the dates as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringDatesWithFormat = spark.read + .schema(stringSchema) + .json(datesWithFormatPath) + val expectedStringDatesWithFormat = Seq( + Row("2015/08/26"), + Row("2014/10/27"), + Row("2016/01/28")) + + checkAnswer(stringDatesWithFormat, expectedStringDatesWithFormat) + } + } + + test("Write timestamps correctly with dateFormat option") { + val customSchema = new StructType(Array(StructField("date", TimestampType, true))) + withTempDir { dir => + // With dateFormat option. + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" + val timestampsWithFormat = spark.read + .schema(customSchema) + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + timestampsWithFormat.write + .format("json") + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .save(timestampsWithFormatPath) + + // This will load back the timestamps as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringTimestampsWithFormat = spark.read + .schema(stringSchema) + .json(timestampsWithFormatPath) + val expectedStringDatesWithFormat = Seq( + Row("2015/08/26 18:00"), + Row("2014/10/27 18:30"), + Row("2016/01/28 20:00")) + + checkAnswer(stringTimestampsWithFormat, expectedStringDatesWithFormat) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index f4a3336643869..d1d82fd5658b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -222,6 +222,12 @@ private[json] trait TestJsonData { spark.sparkContext.parallelize( s"""{"a": 1${"0" * 38}, "b": 92233720368547758070}""" :: Nil) + def datesRecords: RDD[String] = + spark.sparkContext.parallelize( + """{"date": "26/08/2015 18:00"}""" :: + """{"date": "27/10/2014 18:30"}""" :: + """{"date": "28/01/2016 20:00"}""" :: Nil) + lazy val singleRow: RDD[String] = spark.sparkContext.parallelize("""{"a":123}""" :: Nil) def empty: RDD[String] = spark.sparkContext.parallelize(Seq[String]()) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala index d79edee5b1a4c..52486b122a93f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala @@ -32,6 +32,10 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { override protected def supportsDataType(dataType: DataType): Boolean = dataType match { case _: NullType => false case _: BinaryType => false + // `TimestampType` is disabled because `DatatypeConverter.parseDateTime()` + // in `DateTimeUtils` parses the formatted string wrongly when the date is + // too early. (e.g. "1600-07-13T08:36:32.847"). + case _: TimestampType => false case _: CalendarIntervalType => false case _ => true } From aa57083af4cecb595bac09e437607d7142b54913 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Wed, 24 Aug 2016 21:24:24 -0700 Subject: [PATCH 1195/1470] [SPARK-17228][SQL] Not infer/propagate non-deterministic constraints ## What changes were proposed in this pull request? Given that filters based on non-deterministic constraints shouldn't be pushed down in the query plan, unnecessarily inferring them is confusing and a source of potential bugs. This patch simplifies the inferring logic by simply ignoring them. ## How was this patch tested? Added a new test in `ConstraintPropagationSuite`. Author: Sameer Agarwal Closes #14795 from sameeragarwal/deterministic-constraints. (cherry picked from commit ac27557eb622a257abeb3e8551f06ebc72f87133) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/plans/QueryPlan.scala | 3 ++- .../plans/ConstraintPropagationSuite.scala | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index cf34f4b30d8d8..9c605902926e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -35,7 +35,8 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT .union(inferAdditionalConstraints(constraints)) .union(constructIsNotNullConstraints(constraints)) .filter(constraint => - constraint.references.nonEmpty && constraint.references.subsetOf(outputSet)) + constraint.references.nonEmpty && constraint.references.subsetOf(outputSet) && + constraint.deterministic) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index 5a76969235acd..8d6a49a8a37b4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -352,4 +352,21 @@ class ConstraintPropagationSuite extends SparkFunSuite { verifyConstraints(tr.analyze.constraints, ExpressionSet(Seq(IsNotNull(resolveColumn(tr, "b")), IsNotNull(resolveColumn(tr, "c"))))) } + + test("not infer non-deterministic constraints") { + val tr = LocalRelation('a.int, 'b.string, 'c.int) + + verifyConstraints(tr + .where('a.attr === Rand(0)) + .analyze.constraints, + ExpressionSet(Seq(IsNotNull(resolveColumn(tr, "a"))))) + + verifyConstraints(tr + .where('a.attr === InputFileName()) + .where('a.attr =!= 'c.attr) + .analyze.constraints, + ExpressionSet(Seq(resolveColumn(tr, "a") =!= resolveColumn(tr, "c"), + IsNotNull(resolveColumn(tr, "a")), + IsNotNull(resolveColumn(tr, "c"))))) + } } From c1c498006849a7a0a785bc84316e7f494da5f8a8 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 25 Aug 2016 09:45:49 +0100 Subject: [PATCH 1196/1470] [SPARK-17193][CORE] HadoopRDD NPE at DEBUG log level when getLocationInfo == null ## What changes were proposed in this pull request? Handle null from Hadoop getLocationInfo directly instead of catching (and logging) exception ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #14760 from srowen/SPARK-17193. (cherry picked from commit 2bcd5d5ce3eaf0eb1600a12a2b55ddb40927533b) Signed-off-by: Sean Owen --- .../org/apache/spark/rdd/HadoopRDD.scala | 26 +++++++++---------- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- 2 files changed, 13 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 515fd6f4e278c..297d95a731013 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -22,7 +22,6 @@ import java.text.SimpleDateFormat import java.util.Date import scala.collection.immutable.Map -import scala.collection.mutable.ListBuffer import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} @@ -317,7 +316,7 @@ class HadoopRDD[K, V]( try { val lsplit = c.inputSplitWithLocationInfo.cast(hsplit) val infos = c.getLocationInfo.invoke(lsplit).asInstanceOf[Array[AnyRef]] - Some(HadoopRDD.convertSplitLocationInfo(infos)) + HadoopRDD.convertSplitLocationInfo(infos) } catch { case e: Exception => logDebug("Failed to use InputSplitWithLocations.", e) @@ -419,21 +418,20 @@ private[spark] object HadoopRDD extends Logging { None } - private[spark] def convertSplitLocationInfo(infos: Array[AnyRef]): Seq[String] = { - val out = ListBuffer[String]() - infos.foreach { loc => - val locationStr = HadoopRDD.SPLIT_INFO_REFLECTIONS.get. - getLocation.invoke(loc).asInstanceOf[String] + private[spark] def convertSplitLocationInfo(infos: Array[AnyRef]): Option[Seq[String]] = { + Option(infos).map(_.flatMap { loc => + val reflections = HadoopRDD.SPLIT_INFO_REFLECTIONS.get + val locationStr = reflections.getLocation.invoke(loc).asInstanceOf[String] if (locationStr != "localhost") { - if (HadoopRDD.SPLIT_INFO_REFLECTIONS.get.isInMemory. - invoke(loc).asInstanceOf[Boolean]) { - logDebug("Partition " + locationStr + " is cached by Hadoop.") - out += new HDFSCacheTaskLocation(locationStr).toString + if (reflections.isInMemory.invoke(loc).asInstanceOf[Boolean]) { + logDebug(s"Partition $locationStr is cached by Hadoop.") + Some(HDFSCacheTaskLocation(locationStr).toString) } else { - out += new HostTaskLocation(locationStr).toString + Some(HostTaskLocation(locationStr).toString) } + } else { + None } - } - out.seq + }) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index b086baa084080..2d9d69dfb8fba 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -255,7 +255,7 @@ class NewHadoopRDD[K, V]( case Some(c) => try { val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] - Some(HadoopRDD.convertSplitLocationInfo(infos)) + HadoopRDD.convertSplitLocationInfo(infos) } catch { case e : Exception => logDebug("Failed to use InputSplit#getLocationInfo.", e) From fb1c697143a5bb2df69d9f2c9cbddc4eb526f047 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Thu, 25 Aug 2016 11:24:40 +0200 Subject: [PATCH 1197/1470] [SPARK-17061][SPARK-17093][SQL] MapObjects` should make copies of unsafe-backed data Currently `MapObjects` does not make copies of unsafe-backed data, leading to problems like [SPARK-17061](https://issues.apache.org/jira/browse/SPARK-17061) [SPARK-17093](https://issues.apache.org/jira/browse/SPARK-17093). This patch makes `MapObjects` make copies of unsafe-backed data. Generated code - prior to this patch: ```java ... /* 295 */ if (isNull12) { /* 296 */ convertedArray1[loopIndex1] = null; /* 297 */ } else { /* 298 */ convertedArray1[loopIndex1] = value12; /* 299 */ } ... ``` Generated code - after this patch: ```java ... /* 295 */ if (isNull12) { /* 296 */ convertedArray1[loopIndex1] = null; /* 297 */ } else { /* 298 */ convertedArray1[loopIndex1] = value12 instanceof UnsafeRow? value12.copy() : value12; /* 299 */ } ... ``` Add a new test case which would fail without this patch. Author: Liwei Lin Closes #14698 from lw-lin/mapobjects-copy. (cherry picked from commit e0b20f9f24d5c3304bf517a4dcfb0da93be5bc75) Signed-off-by: Herman van Hovell --- .../src/main/java/SimpleApp.java | 41 +++++++++++++++++++ .../expressions/objects/objects.scala | 12 +++++- .../expressions/ExpressionEvalHelper.scala | 2 +- 3 files changed, 53 insertions(+), 2 deletions(-) diff --git a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java index 5217689e7c092..059e51bc515d4 100644 --- a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java +++ b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java @@ -18,11 +18,20 @@ import org.apache.spark.api.java.*; import org.apache.spark.api.java.function.Function; +<<<<<<< HEAD:dev/audit-release/maven_app_core/src/main/java/SimpleApp.java public class SimpleApp { public static void main(String[] args) { String logFile = "input.txt"; JavaSparkContext sc = new JavaSparkContext("local", "Simple App"); JavaRDD logData = sc.textFile(logFile).cache(); +======= +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.objects.Invoke +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} +import org.apache.spark.sql.types.{IntegerType, ObjectType} +>>>>>>> e0b20f9... [SPARK-17061][SPARK-17093][SQL] MapObjects` should make copies of unsafe-backed data:sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala long numAs = logData.filter(new Function() { public Boolean call(String s) { return s.contains("a"); } @@ -39,4 +48,36 @@ public static void main(String[] args) { System.out.println("Test succeeded"); sc.stop(); } + + test("MapObjects should make copies of unsafe-backed data") { + // test UnsafeRow-backed data + val structEncoder = ExpressionEncoder[Array[Tuple2[java.lang.Integer, java.lang.Integer]]] + val structInputRow = InternalRow.fromSeq(Seq(Array((1, 2), (3, 4)))) + val structExpected = new GenericArrayData( + Array(InternalRow.fromSeq(Seq(1, 2)), InternalRow.fromSeq(Seq(3, 4)))) + checkEvalutionWithUnsafeProjection( + structEncoder.serializer.head, structExpected, structInputRow) + + // test UnsafeArray-backed data + val arrayEncoder = ExpressionEncoder[Array[Array[Int]]] + val arrayInputRow = InternalRow.fromSeq(Seq(Array(Array(1, 2), Array(3, 4)))) + val arrayExpected = new GenericArrayData( + Array(new GenericArrayData(Array(1, 2)), new GenericArrayData(Array(3, 4)))) + checkEvalutionWithUnsafeProjection( + arrayEncoder.serializer.head, arrayExpected, arrayInputRow) + + // test UnsafeMap-backed data + val mapEncoder = ExpressionEncoder[Array[Map[Int, Int]]] + val mapInputRow = InternalRow.fromSeq(Seq(Array( + Map(1 -> 100, 2 -> 200), Map(3 -> 300, 4 -> 400)))) + val mapExpected = new GenericArrayData(Seq( + new ArrayBasedMapData( + new GenericArrayData(Array(1, 2)), + new GenericArrayData(Array(100, 200))), + new ArrayBasedMapData( + new GenericArrayData(Array(3, 4)), + new GenericArrayData(Array(300, 400))))) + checkEvalutionWithUnsafeProjection( + mapEncoder.serializer.head, mapExpected, mapInputRow) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 37ec1a63394cf..1cdda53708bc0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -481,6 +481,16 @@ case class MapObjects private( s"$seq == null ? $array[$loopIndex] : $seq.apply($loopIndex)" } + // Make a copy of the data if it's unsafe-backed + def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) = + s"$value instanceof ${clazz.getSimpleName}? ${value}.copy() : $value" + val genFunctionValue = lambdaFunction.dataType match { + case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value) + case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value) + case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value) + case _ => genFunction.value + } + val loopNullCheck = inputDataType match { case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" // The element of primitive array will never be null. @@ -508,7 +518,7 @@ case class MapObjects private( if (${genFunction.isNull}) { $convertedArray[$loopIndex] = null; } else { - $convertedArray[$loopIndex] = ${genFunction.value}; + $convertedArray[$loopIndex] = $genFunctionValue; } $loopIndex += 1; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index d6a9672d1f186..668543a28bd30 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -136,7 +136,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { // some expression is reusing variable names across different instances. // This behavior is tested in ExpressionEvalHelperSuite. val plan = generateProject( - GenerateUnsafeProjection.generate( + UnsafeProjection.create( Alias(expression, s"Optimized($expression)1")() :: Alias(expression, s"Optimized($expression)2")() :: Nil), expression) From 88481ea2169e0813cfc326eb1440ddaaf3110f4a Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 25 Aug 2016 11:48:13 +0200 Subject: [PATCH 1198/1470] Revert "[SPARK-17061][SPARK-17093][SQL] MapObjects` should make copies of unsafe-backed data" This reverts commit fb1c697143a5bb2df69d9f2c9cbddc4eb526f047. --- .../src/main/java/SimpleApp.java | 41 ------------------- .../expressions/objects/objects.scala | 12 +----- .../expressions/ExpressionEvalHelper.scala | 2 +- 3 files changed, 2 insertions(+), 53 deletions(-) diff --git a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java index 059e51bc515d4..5217689e7c092 100644 --- a/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java +++ b/dev/audit-release/maven_app_core/src/main/java/SimpleApp.java @@ -18,20 +18,11 @@ import org.apache.spark.api.java.*; import org.apache.spark.api.java.function.Function; -<<<<<<< HEAD:dev/audit-release/maven_app_core/src/main/java/SimpleApp.java public class SimpleApp { public static void main(String[] args) { String logFile = "input.txt"; JavaSparkContext sc = new JavaSparkContext("local", "Simple App"); JavaRDD logData = sc.textFile(logFile).cache(); -======= -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.objects.Invoke -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} -import org.apache.spark.sql.types.{IntegerType, ObjectType} ->>>>>>> e0b20f9... [SPARK-17061][SPARK-17093][SQL] MapObjects` should make copies of unsafe-backed data:sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala long numAs = logData.filter(new Function() { public Boolean call(String s) { return s.contains("a"); } @@ -48,36 +39,4 @@ public static void main(String[] args) { System.out.println("Test succeeded"); sc.stop(); } - - test("MapObjects should make copies of unsafe-backed data") { - // test UnsafeRow-backed data - val structEncoder = ExpressionEncoder[Array[Tuple2[java.lang.Integer, java.lang.Integer]]] - val structInputRow = InternalRow.fromSeq(Seq(Array((1, 2), (3, 4)))) - val structExpected = new GenericArrayData( - Array(InternalRow.fromSeq(Seq(1, 2)), InternalRow.fromSeq(Seq(3, 4)))) - checkEvalutionWithUnsafeProjection( - structEncoder.serializer.head, structExpected, structInputRow) - - // test UnsafeArray-backed data - val arrayEncoder = ExpressionEncoder[Array[Array[Int]]] - val arrayInputRow = InternalRow.fromSeq(Seq(Array(Array(1, 2), Array(3, 4)))) - val arrayExpected = new GenericArrayData( - Array(new GenericArrayData(Array(1, 2)), new GenericArrayData(Array(3, 4)))) - checkEvalutionWithUnsafeProjection( - arrayEncoder.serializer.head, arrayExpected, arrayInputRow) - - // test UnsafeMap-backed data - val mapEncoder = ExpressionEncoder[Array[Map[Int, Int]]] - val mapInputRow = InternalRow.fromSeq(Seq(Array( - Map(1 -> 100, 2 -> 200), Map(3 -> 300, 4 -> 400)))) - val mapExpected = new GenericArrayData(Seq( - new ArrayBasedMapData( - new GenericArrayData(Array(1, 2)), - new GenericArrayData(Array(100, 200))), - new ArrayBasedMapData( - new GenericArrayData(Array(3, 4)), - new GenericArrayData(Array(300, 400))))) - checkEvalutionWithUnsafeProjection( - mapEncoder.serializer.head, mapExpected, mapInputRow) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 1cdda53708bc0..37ec1a63394cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -481,16 +481,6 @@ case class MapObjects private( s"$seq == null ? $array[$loopIndex] : $seq.apply($loopIndex)" } - // Make a copy of the data if it's unsafe-backed - def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) = - s"$value instanceof ${clazz.getSimpleName}? ${value}.copy() : $value" - val genFunctionValue = lambdaFunction.dataType match { - case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value) - case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value) - case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value) - case _ => genFunction.value - } - val loopNullCheck = inputDataType match { case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" // The element of primitive array will never be null. @@ -518,7 +508,7 @@ case class MapObjects private( if (${genFunction.isNull}) { $convertedArray[$loopIndex] = null; } else { - $convertedArray[$loopIndex] = $genFunctionValue; + $convertedArray[$loopIndex] = ${genFunction.value}; } $loopIndex += 1; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 668543a28bd30..d6a9672d1f186 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -136,7 +136,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { // some expression is reusing variable names across different instances. // This behavior is tested in ExpressionEvalHelperSuite. val plan = generateProject( - UnsafeProjection.create( + GenerateUnsafeProjection.generate( Alias(expression, s"Optimized($expression)1")() :: Alias(expression, s"Optimized($expression)2")() :: Nil), expression) From 184e78b9d640259ba0720574de060841dc912872 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Thu, 25 Aug 2016 14:16:22 +0200 Subject: [PATCH 1199/1470] [SPARK-17061][SPARK-17093][SQL][BACKPORT] MapObjects should make copies of unsafe-backed data ## What changes were proposed in this pull request? This PR backports https://github.com/apache/spark/pull/14698 to branch-2.0. See that PR for more details. All credit should go to lw-lin. Author: Herman van Hovell Author: Liwei Lin Closes #14806 from hvanhovell/SPARK-17061. --- .../expressions/objects/objects.scala | 12 +++- .../expressions/ExpressionEvalHelper.scala | 2 +- .../expressions/ObjectExpressionSuite.scala | 56 +++++++++++++++++++ 3 files changed, 68 insertions(+), 2 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 37ec1a63394cf..1cdda53708bc0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -481,6 +481,16 @@ case class MapObjects private( s"$seq == null ? $array[$loopIndex] : $seq.apply($loopIndex)" } + // Make a copy of the data if it's unsafe-backed + def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) = + s"$value instanceof ${clazz.getSimpleName}? ${value}.copy() : $value" + val genFunctionValue = lambdaFunction.dataType match { + case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value) + case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value) + case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value) + case _ => genFunction.value + } + val loopNullCheck = inputDataType match { case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" // The element of primitive array will never be null. @@ -508,7 +518,7 @@ case class MapObjects private( if (${genFunction.isNull}) { $convertedArray[$loopIndex] = null; } else { - $convertedArray[$loopIndex] = ${genFunction.value}; + $convertedArray[$loopIndex] = $genFunctionValue; } $loopIndex += 1; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index d6a9672d1f186..668543a28bd30 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -136,7 +136,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { // some expression is reusing variable names across different instances. // This behavior is tested in ExpressionEvalHelperSuite. val plan = generateProject( - GenerateUnsafeProjection.generate( + UnsafeProjection.create( Alias(expression, s"Optimized($expression)1")() :: Alias(expression, s"Optimized($expression)2")() :: Nil), expression) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionSuite.scala new file mode 100644 index 0000000000000..b6263e77c141f --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionSuite.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} + +class ObjectExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + test("MapObjects should make copies of unsafe-backed data") { + // test UnsafeRow-backed data + val structEncoder = ExpressionEncoder[Array[(java.lang.Integer, java.lang.Integer)]]() + val structInputRow = InternalRow.fromSeq(Seq(Array((1, 2), (3, 4)))) + val structExpected = new GenericArrayData( + Array(InternalRow.fromSeq(Seq(1, 2)), InternalRow.fromSeq(Seq(3, 4)))) + checkEvalutionWithUnsafeProjection( + structEncoder.serializer.head, structExpected, structInputRow) + + // test UnsafeArray-backed data + val arrayEncoder = ExpressionEncoder[Array[Array[Int]]]() + val arrayInputRow = InternalRow.fromSeq(Seq(Array(Array(1, 2), Array(3, 4)))) + val arrayExpected = new GenericArrayData( + Array(new GenericArrayData(Array(1, 2)), new GenericArrayData(Array(3, 4)))) + checkEvalutionWithUnsafeProjection( + arrayEncoder.serializer.head, arrayExpected, arrayInputRow) + + // test UnsafeMap-backed data + val mapEncoder = ExpressionEncoder[Array[Map[Int, Int]]]() + val mapInputRow = InternalRow.fromSeq(Seq(Array( + Map(1 -> 100, 2 -> 200), Map(3 -> 300, 4 -> 400)))) + val mapExpected = new GenericArrayData(Seq( + new ArrayBasedMapData( + new GenericArrayData(Array(1, 2)), + new GenericArrayData(Array(100, 200))), + new ArrayBasedMapData( + new GenericArrayData(Array(3, 4)), + new GenericArrayData(Array(300, 400))))) + checkEvalutionWithUnsafeProjection( + mapEncoder.serializer.head, mapExpected, mapInputRow) + } +} From 48ecf3d0027e61d4d4ad6711ca2d4064a6b9c9e9 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 25 Aug 2016 14:18:58 +0200 Subject: [PATCH 1200/1470] [SPARK-16991][SPARK-17099][SPARK-17120][SQL] Fix Outer Join Elimination when Filter's isNotNull Constraints Unable to Filter Out All Null-supplying Rows ### What changes were proposed in this pull request? This PR is to fix an incorrect outer join elimination when filter's `isNotNull` constraints is unable to filter out all null-supplying rows. For example, `isnotnull(coalesce(b#227, c#238))`. Users can hit this error when they try to use `using/natural outer join`, which is converted to a normal outer join with a `coalesce` expression on the `using columns`. For example, ```Scala val a = Seq((1, 2), (2, 3)).toDF("a", "b") val b = Seq((2, 5), (3, 4)).toDF("a", "c") val c = Seq((3, 1)).toDF("a", "d") val ab = a.join(b, Seq("a"), "fullouter") ab.join(c, "a").explain(true) ``` The dataframe `ab` is doing `using full-outer join`, which is converted to a normal outer join with a `coalesce` expression. Constraints inference generates a `Filter` with constraints `isnotnull(coalesce(b#227, c#238))`. Then, it triggers a wrong outer join elimination and generates a wrong result. ``` Project [a#251, b#227, c#237, d#247] +- Join Inner, (a#251 = a#246) :- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237] : +- Join FullOuter, (a#226 = a#236) : :- Project [_1#223 AS a#226, _2#224 AS b#227] : : +- LocalRelation [_1#223, _2#224] : +- Project [_1#233 AS a#236, _2#234 AS c#237] : +- LocalRelation [_1#233, _2#234] +- Project [_1#243 AS a#246, _2#244 AS d#247] +- LocalRelation [_1#243, _2#244] == Optimized Logical Plan == Project [a#251, b#227, c#237, d#247] +- Join Inner, (a#251 = a#246) :- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237] : +- Filter isnotnull(coalesce(a#226, a#236)) : +- Join FullOuter, (a#226 = a#236) : :- LocalRelation [a#226, b#227] : +- LocalRelation [a#236, c#237] +- LocalRelation [a#246, d#247] ``` **A note to the `Committer`**, please also give the credit to dongjoon-hyun who submitted another PR for fixing this issue. https://github.com/apache/spark/pull/14580 ### How was this patch tested? Added test cases Author: gatorsmile Closes #14661 from gatorsmile/fixOuterJoinElimination. (cherry picked from commit d2ae6399ee2f0524b88262735adbbcb2035de8fd) Signed-off-by: Herman van Hovell --- .../sql/catalyst/optimizer/Optimizer.scala | 18 ++--- .../optimizer/OuterJoinEliminationSuite.scala | 39 ++++++++++ .../resources/sql-tests/inputs/outer-join.sql | 36 ++++++++++ .../sql-tests/results/outer-join.sql.out | 72 +++++++++++++++++++ .../apache/spark/sql/DataFrameJoinSuite.scala | 8 +++ 5 files changed, 161 insertions(+), 12 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/outer-join.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/outer-join.sql.out 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 4db541f692fdf..5dd51abffaca1 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 @@ -1250,18 +1250,12 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { } private def buildNewJoinType(filter: Filter, join: Join): JoinType = { - val splitConjunctiveConditions: Seq[Expression] = splitConjunctivePredicates(filter.condition) - val leftConditions = splitConjunctiveConditions - .filter(_.references.subsetOf(join.left.outputSet)) - val rightConditions = splitConjunctiveConditions - .filter(_.references.subsetOf(join.right.outputSet)) - - val leftHasNonNullPredicate = leftConditions.exists(canFilterOutNull) || - filter.constraints.filter(_.isInstanceOf[IsNotNull]) - .exists(expr => join.left.outputSet.intersect(expr.references).nonEmpty) - val rightHasNonNullPredicate = rightConditions.exists(canFilterOutNull) || - filter.constraints.filter(_.isInstanceOf[IsNotNull]) - .exists(expr => join.right.outputSet.intersect(expr.references).nonEmpty) + val conditions = splitConjunctivePredicates(filter.condition) ++ filter.constraints + val leftConditions = conditions.filter(_.references.subsetOf(join.left.outputSet)) + val rightConditions = conditions.filter(_.references.subsetOf(join.right.outputSet)) + + val leftHasNonNullPredicate = leftConditions.exists(canFilterOutNull) + val rightHasNonNullPredicate = rightConditions.exists(canFilterOutNull) join.joinType match { case RightOuter if leftHasNonNullPredicate => Inner diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala index 41754adef4216..c168a55e40c54 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Coalesce, IsNotNull} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -192,4 +193,42 @@ class OuterJoinEliminationSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("joins: no outer join elimination if the filter is not NULL eliminated") { + val x = testRelation.subquery('x) + val y = testRelation1.subquery('y) + + val originalQuery = + x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)) + .where(Coalesce("y.e".attr :: "x.a".attr :: Nil)) + + val optimized = Optimize.execute(originalQuery.analyze) + + val left = testRelation + val right = testRelation1 + val correctAnswer = + left.join(right, FullOuter, Option("a".attr === "d".attr)) + .where(Coalesce("e".attr :: "a".attr :: Nil)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("joins: no outer join elimination if the filter's constraints are not NULL eliminated") { + val x = testRelation.subquery('x) + val y = testRelation1.subquery('y) + + val originalQuery = + x.join(y, FullOuter, Option("x.a".attr === "y.d".attr)) + .where(IsNotNull(Coalesce("y.e".attr :: "x.a".attr :: Nil))) + + val optimized = Optimize.execute(originalQuery.analyze) + + val left = testRelation + val right = testRelation1 + val correctAnswer = + left.join(right, FullOuter, Option("a".attr === "d".attr)) + .where(IsNotNull(Coalesce("e".attr :: "a".attr :: Nil))).analyze + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql new file mode 100644 index 0000000000000..f50f1ebad970e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql @@ -0,0 +1,36 @@ +-- SPARK-17099: Incorrect result when HAVING clause is added to group by query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1); + +SELECT + (SUM(COALESCE(t1.int_col1, t2.int_col0))), + ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON (t2.int_col0) = (t1.int_col1) +GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) + > ((COALESCE(t1.int_col1, t2.int_col0)) * 2); + + +-- SPARK-17120: Analyzer incorrectly optimizes plan to empty LocalRelation +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1); + +SELECT * +FROM ( +SELECT + COALESCE(t2.int_col1, t1.int_col1) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (t.int_col) is not null; + + + diff --git a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out new file mode 100644 index 0000000000000..b39fdb0e58720 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out @@ -0,0 +1,72 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT + (SUM(COALESCE(t1.int_col1, t2.int_col0))), + ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON (t2.int_col0) = (t1.int_col1) +GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) + > ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +-- !query 2 schema +struct +-- !query 2 output +-367 -734 +-507 -1014 +-769 -1538 +-800 -1600 + + +-- !query 3 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +SELECT * +FROM ( +SELECT + COALESCE(t2.int_col1, t1.int_col1) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (t.int_col) is not null +-- !query 5 schema +struct +-- !query 5 output +97 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 4342c039aefc8..4abf5e42b9c34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -225,4 +225,12 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { Row(1, null) :: Row(null, 2) :: Nil ) } + + test("SPARK-16991: Full outer join followed by inner join produces wrong results") { + val a = Seq((1, 2), (2, 3)).toDF("a", "b") + val b = Seq((2, 5), (3, 4)).toDF("a", "c") + val c = Seq((3, 1)).toDF("a", "d") + val ab = a.join(b, Seq("a"), "fullouter") + checkAnswer(ab.join(c, "a"), Row(3, null, 4, 1) :: Nil) + } } From 2b32a442dfbc8494c30dcb2f6869c9dc7f258ada Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 25 Aug 2016 14:38:41 +0200 Subject: [PATCH 1201/1470] [SPARK-17167][2.0][SQL] Issue Exceptions when Analyze Table on In-Memory Cataloged Tables ### What changes were proposed in this pull request? Currently, `Analyze Table` is only used for Hive-serde tables. We should issue exceptions in all the other cases. When the tables are data source tables, we issued an exception. However, when tables are In-Memory Cataloged tables, we do not issue any exception. This PR is to issue an exception when the tables are in-memory cataloged. For example, ```SQL CREATE TABLE tbl(a INT, b INT) USING parquet ``` `tbl` is a `SimpleCatalogRelation` when the hive support is not enabled. ### How was this patch tested? Added two test cases. One of them is just to improve the test coverage when the analyzed table is data source tables. Author: gatorsmile Closes #14781 from gatorsmile/analyzeInMemoryTable2. --- .../execution/command/AnalyzeTableCommand.scala | 4 ++-- .../spark/sql/execution/command/DDLSuite.scala | 11 +++++++++++ .../spark/sql/hive/execution/HiveDDLSuite.scala | 15 +++++++++++++++ 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index a469d4da8613b..9509b66ffd391 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable, SimpleCatalogRelation} /** @@ -41,7 +41,7 @@ case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) relation match { - case relation: CatalogRelation => + case relation: CatalogRelation if !relation.isInstanceOf[SimpleCatalogRelation] => val catalogTable: CatalogTable = relation.catalogTable // This method is mainly based on // org.apache.hadoop.hive.ql.stats.StatsUtils.getFileSizeForTable(HiveConf, Table) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index d70cae74bc6c9..95654712fee5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -395,6 +395,17 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(catalog.getTableMetadata(tableIdent1) === expectedTable) } + test("Analyze in-memory cataloged tables(SimpleCatalogRelation)") { + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet") + val e = intercept[AnalysisException] { + sql("ANALYZE TABLE tbl COMPUTE STATISTICS") + }.getMessage + assert(e.contains("ANALYZE TABLE is only supported for Hive tables, " + + "but 'tbl' is a SimpleCatalogRelation")) + } + } + test("create table using") { val catalog = spark.sessionState.catalog withTable("tbl") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 92282420214d0..df6cd568ac4f7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -594,6 +594,21 @@ class HiveDDLSuite } } + test("Analyze data source tables(LogicalRelation)") { + withTable("t1") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(1).write.format("parquet").save(path) + sql(s"CREATE TABLE t1 USING parquet OPTIONS (PATH '$path')") + val e = intercept[AnalysisException] { + sql("ANALYZE TABLE t1 COMPUTE STATISTICS") + }.getMessage + assert(e.contains("ANALYZE TABLE is only supported for Hive tables, " + + "but 't1' is a LogicalRelation")) + } + } + } + test("desc table for data source table") { withTable("tab1") { val tabName = "tab1" From 356a359de038e2e9d4d0cb7c0c5b493f7036d7c3 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 15 Aug 2016 12:41:27 -0700 Subject: [PATCH 1202/1470] [SPARK-16700][PYSPARK][SQL] create DataFrame from dict/Row with schema In 2.0, we verify the data type against schema for every row for safety, but with performance cost, this PR make it optional. When we verify the data type for StructType, it does not support all the types we support in infer schema (for example, dict), this PR fix that to make them consistent. For Row object which is created using named arguments, the order of fields are sorted by name, they may be not different than the order in provided schema, this PR fix that by ignore the order of fields in this case. Created regression tests for them. Author: Davies Liu Closes #14469 from davies/py_dict. --- python/pyspark/sql/context.py | 8 ++++++-- python/pyspark/sql/session.py | 29 ++++++++++++--------------- python/pyspark/sql/tests.py | 16 +++++++++++++++ python/pyspark/sql/types.py | 37 +++++++++++++++++++++++++---------- 4 files changed, 62 insertions(+), 28 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index afb9b54432100..8cdf37188e665 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -215,7 +215,7 @@ def _inferSchema(self, rdd, samplingRatio=None): @since(1.3) @ignore_unicode_prefix - def createDataFrame(self, data, schema=None, samplingRatio=None): + def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=True): """ Creates a :class:`DataFrame` from an :class:`RDD`, a list or a :class:`pandas.DataFrame`. @@ -247,6 +247,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): ``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`. We can also use ``int`` as a short name for :class:`pyspark.sql.types.IntegerType`. :param samplingRatio: the sample ratio of rows used for inferring + :param verifySchema: verify data types of every row against schema. :return: :class:`DataFrame` .. versionchanged:: 2.0 @@ -255,6 +256,9 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): If it's not a :class:`pyspark.sql.types.StructType`, it will be wrapped into a :class:`pyspark.sql.types.StructType` and each record will also be wrapped into a tuple. + .. versionchanged:: 2.0.1 + Added verifySchema. + >>> l = [('Alice', 1)] >>> sqlContext.createDataFrame(l).collect() [Row(_1=u'Alice', _2=1)] @@ -302,7 +306,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): ... Py4JJavaError: ... """ - return self.sparkSession.createDataFrame(data, schema, samplingRatio) + return self.sparkSession.createDataFrame(data, schema, samplingRatio, verifySchema) @since(1.3) def registerDataFrameAsTable(self, df, tableName): diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 10bd89b03fe33..d8627ce97a280 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -384,17 +384,15 @@ def _createFromLocal(self, data, schema): if schema is None or isinstance(schema, (list, tuple)): struct = self._inferSchemaFromList(data) + converter = _create_converter(struct) + data = map(converter, data) if isinstance(schema, (list, tuple)): for i, name in enumerate(schema): struct.fields[i].name = name struct.names[i] = name schema = struct - elif isinstance(schema, StructType): - for row in data: - _verify_type(row, schema) - - else: + elif not isinstance(schema, StructType): raise TypeError("schema should be StructType or list or None, but got: %s" % schema) # convert python objects to sql data @@ -403,7 +401,7 @@ def _createFromLocal(self, data, schema): @since(2.0) @ignore_unicode_prefix - def createDataFrame(self, data, schema=None, samplingRatio=None): + def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=True): """ Creates a :class:`DataFrame` from an :class:`RDD`, a list or a :class:`pandas.DataFrame`. @@ -434,13 +432,11 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): ``byte`` instead of ``tinyint`` for :class:`pyspark.sql.types.ByteType`. We can also use ``int`` as a short name for ``IntegerType``. :param samplingRatio: the sample ratio of rows used for inferring + :param verifySchema: verify data types of every row against schema. :return: :class:`DataFrame` - .. versionchanged:: 2.0 - The ``schema`` parameter can be a :class:`pyspark.sql.types.DataType` or a - :class:`pyspark.sql.types.StringType` after 2.0. If it's not a - :class:`pyspark.sql.types.StructType`, it will be wrapped into a - :class:`pyspark.sql.types.StructType` and each record will also be wrapped into a tuple. + .. versionchanged:: 2.0.1 + Added verifySchema. >>> l = [('Alice', 1)] >>> spark.createDataFrame(l).collect() @@ -505,17 +501,18 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): schema = [str(x) for x in data.columns] data = [r.tolist() for r in data.to_records(index=False)] + verify_func = _verify_type if verifySchema else lambda _, t: True if isinstance(schema, StructType): def prepare(obj): - _verify_type(obj, schema) + verify_func(obj, schema) return obj elif isinstance(schema, DataType): - datatype = schema + dataType = schema + schema = StructType().add("value", schema) def prepare(obj): - _verify_type(obj, datatype) - return (obj, ) - schema = StructType().add("value", datatype) + verify_func(obj, dataType) + return obj, else: if isinstance(schema, list): schema = [x.encode('utf-8') if not isinstance(x, str) else x for x in schema] diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 87dbb5049565f..520b09d9c6f16 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -411,6 +411,22 @@ def test_infer_schema_to_local(self): df3 = self.spark.createDataFrame(rdd, df.schema) self.assertEqual(10, df3.count()) + def test_apply_schema_to_dict_and_rows(self): + schema = StructType().add("b", StringType()).add("a", IntegerType()) + input = [{"a": 1}, {"b": "coffee"}] + rdd = self.sc.parallelize(input) + for verify in [False, True]: + df = self.spark.createDataFrame(input, schema, verifySchema=verify) + df2 = self.spark.createDataFrame(rdd, schema, verifySchema=verify) + self.assertEqual(df.schema, df2.schema) + + rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x, b=None)) + df3 = self.spark.createDataFrame(rdd, schema, verifySchema=verify) + self.assertEqual(10, df3.count()) + input = [Row(a=x, b=str(x)) for x in range(10)] + df4 = self.spark.createDataFrame(input, schema, verifySchema=verify) + self.assertEqual(10, df4.count()) + def test_create_dataframe_schema_mismatch(self): input = [Row(a=1)] rdd = self.sc.parallelize(range(3)).map(lambda i: Row(a=i)) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 1ca4bbc379b42..b765472d6edbc 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -582,6 +582,8 @@ def toInternal(self, obj): else: if isinstance(obj, dict): return tuple(obj.get(n) for n in self.names) + elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False): + return tuple(obj[n] for n in self.names) elif isinstance(obj, (list, tuple)): return tuple(obj) elif hasattr(obj, "__dict__"): @@ -1243,7 +1245,7 @@ def _infer_schema_type(obj, dataType): TimestampType: (datetime.datetime,), ArrayType: (list, tuple, array), MapType: (dict,), - StructType: (tuple, list), + StructType: (tuple, list, dict), } @@ -1314,10 +1316,10 @@ def _verify_type(obj, dataType, nullable=True): assert _type in _acceptable_types, "unknown datatype: %s for object %r" % (dataType, obj) if _type is StructType: - if not isinstance(obj, (tuple, list)): - raise TypeError("StructType can not accept object %r in type %s" % (obj, type(obj))) + # check the type and fields later + pass else: - # subclass of them can not be fromInternald in JVM + # subclass of them can not be fromInternal in JVM if type(obj) not in _acceptable_types[_type]: raise TypeError("%s can not accept object %r in type %s" % (dataType, obj, type(obj))) @@ -1343,11 +1345,25 @@ def _verify_type(obj, dataType, nullable=True): _verify_type(v, dataType.valueType, dataType.valueContainsNull) elif isinstance(dataType, StructType): - if len(obj) != len(dataType.fields): - raise ValueError("Length of object (%d) does not match with " - "length of fields (%d)" % (len(obj), len(dataType.fields))) - for v, f in zip(obj, dataType.fields): - _verify_type(v, f.dataType, f.nullable) + if isinstance(obj, dict): + for f in dataType.fields: + _verify_type(obj.get(f.name), f.dataType, f.nullable) + elif isinstance(obj, Row) and getattr(obj, "__from_dict__", False): + # the order in obj could be different than dataType.fields + for f in dataType.fields: + _verify_type(obj[f.name], f.dataType, f.nullable) + elif isinstance(obj, (tuple, list)): + if len(obj) != len(dataType.fields): + raise ValueError("Length of object (%d) does not match with " + "length of fields (%d)" % (len(obj), len(dataType.fields))) + for v, f in zip(obj, dataType.fields): + _verify_type(v, f.dataType, f.nullable) + elif hasattr(obj, "__dict__"): + d = obj.__dict__ + for f in dataType.fields: + _verify_type(d.get(f.name), f.dataType, f.nullable) + else: + raise TypeError("StructType can not accept object %r in type %s" % (obj, type(obj))) # This is used to unpickle a Row from JVM @@ -1410,6 +1426,7 @@ def __new__(self, *args, **kwargs): names = sorted(kwargs.keys()) row = tuple.__new__(self, [kwargs[n] for n in names]) row.__fields__ = names + row.__from_dict__ = True return row else: @@ -1485,7 +1502,7 @@ def __getattr__(self, item): raise AttributeError(item) def __setattr__(self, key, value): - if key != '__fields__': + if key != '__fields__' and key != "__from_dict__": raise Exception("Row is read-only") self.__dict__[key] = value From 55db26245d69bb02b7d7d5f25029b1a1cd571644 Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Thu, 25 Aug 2016 11:54:55 -0500 Subject: [PATCH 1203/1470] [SPARK-15083][WEB UI] History Server can OOM due to unlimited TaskUIData ## What changes were proposed in this pull request? This is a back port of #14673 addressing merge conflicts in package.scala that prevented a cherry-pick to `branch-2.0` when it was merged to `master` Since the History Server currently loads all application's data it can OOM if too many applications have a significant task count. This trims tasks by `spark.ui.retainedTasks` (default: 100000) ## How was this patch tested? Manual testing and dev/run-tests Author: Alex Bozarth Closes #14794 from ajbozarth/spark15083-branch-2.0. --- .../spark/internal/config/package.scala | 5 + .../spark/ui/jobs/JobProgressListener.scala | 9 +- .../org/apache/spark/ui/jobs/StagePage.scala | 12 +- .../org/apache/spark/ui/jobs/UIData.scala | 4 +- ...stage_task_list_w__sortBy_expectation.json | 130 ++++++------- ...tBy_short_names___runtime_expectation.json | 130 ++++++------- ...rtBy_short_names__runtime_expectation.json | 182 +++++++++--------- .../api/v1/AllStagesResourceSuite.scala | 4 +- docs/configuration.md | 8 + 9 files changed, 256 insertions(+), 228 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 05dd68300f891..5a59aee652712 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -103,4 +103,9 @@ package object config { .stringConf .checkValues(Set("hive", "in-memory")) .createWithDefault("in-memory") + + // To limit memory usage, we only track information for a fixed number of tasks + private[spark] val UI_RETAINED_TASKS = ConfigBuilder("spark.ui.retainedTasks") + .intConf + .createWithDefault(100000) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 842f42b4c98db..38ad6e985c4b5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -19,12 +19,13 @@ package org.apache.spark.ui.jobs import java.util.concurrent.TimeoutException -import scala.collection.mutable.{HashMap, HashSet, ListBuffer} +import scala.collection.mutable.{HashMap, HashSet, LinkedHashMap, ListBuffer} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId @@ -93,6 +94,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val retainedStages = conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES) val retainedJobs = conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) + val retainedTasks = conf.get(UI_RETAINED_TASKS) // We can test for memory leaks by ensuring that collections that track non-active jobs and // stages do not grow without bound and that collections for active jobs/stages eventually become @@ -400,6 +402,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskData.updateTaskMetrics(taskMetrics) taskData.errorMessage = errorMessage + // If Tasks is too large, remove and garbage collect old tasks + if (stageData.taskData.size > retainedTasks) { + stageData.taskData = stageData.taskData.drop(stageData.taskData.size - retainedTasks) + } + for ( activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskEnd.stageId); jobId <- activeJobsDependentOnStage; diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index d986a55959b82..145dbf8eeb6a7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -131,7 +131,14 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val stageData = stageDataOption.get val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) - val numCompleted = tasks.count(_.taskInfo.finished) + val numCompleted = stageData.numCompleteTasks + val totalTasks = stageData.numActiveTasks + + stageData.numCompleteTasks + stageData.numFailedTasks + val totalTasksNumStr = if (totalTasks == tasks.size) { + s"$totalTasks" + } else { + s"$totalTasks, showing ${tasks.size}" + } val allAccumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables val externalAccumulables = allAccumulables.values.filter { acc => !acc.internal } @@ -576,7 +583,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++ maybeAccumulableTable ++ -

    Tasks

    ++ taskTableHTML ++ jsForScrollingDownToTaskTable +

    Tasks ({totalTasksNumStr})

    ++ + taskTableHTML ++ jsForScrollingDownToTaskTable UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index d76a0e657c288..4f8505058caad 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -18,7 +18,7 @@ package org.apache.spark.ui.jobs import scala.collection.mutable -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, LinkedHashMap} import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} @@ -94,7 +94,7 @@ private[spark] object UIData { var description: Option[String] = None var accumulables = new HashMap[Long, AccumulableInfo] - var taskData = new HashMap[Long, TaskUIData] + var taskData = new LinkedHashMap[Long, TaskUIData] var executorSummary = new HashMap[String, ExecutorSummary] def hasInput: Boolean = inputBytes > 0 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index 11eec0b49c40b..96d86b7278ff1 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -39,21 +39,21 @@ } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", + "launchTime" : "2015-05-06T13:03:06.502GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, "executorRunTime" : 350, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -74,26 +74,26 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", + "launchTime" : "2015-05-06T13:03:06.505GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, "executorRunTime" : 350, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -114,22 +114,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.504GMT", + "launchTime" : "2015-05-06T13:03:06.494GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, @@ -137,7 +137,7 @@ "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, "outputMetrics" : { @@ -154,15 +154,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 3842811, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", + "launchTime" : "2015-05-06T13:03:06.504GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -173,7 +173,7 @@ "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -194,13 +194,13 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "executorId" : "driver", @@ -213,7 +213,7 @@ "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -234,30 +234,30 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", + "launchTime" : "2015-05-06T13:03:06.506GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, "outputMetrics" : { @@ -274,7 +274,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } @@ -479,25 +479,25 @@ } } }, { - "taskId" : 16, - "index" : 16, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.001GMT", + "launchTime" : "2015-05-06T13:03:06.915GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 10, + "executorDeserializeTime" : 9, "executorRunTime" : 84, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, "outputMetrics" : { @@ -514,22 +514,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 108320, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 16, + "index" : 16, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", + "launchTime" : "2015-05-06T13:03:07.001GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 10, "executorRunTime" : 84, "resultSize" : 2010, "jvmGcTime" : 5, @@ -554,30 +554,30 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 108320, "recordsWritten" : 10 } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", + "launchTime" : "2015-05-06T13:03:07.012GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, "executorRunTime" : 84, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, "outputMetrics" : { @@ -594,25 +594,25 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", + "launchTime" : "2015-05-06T13:03:06.925GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, "executorRunTime" : 83, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -634,25 +634,25 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", + "launchTime" : "2015-05-06T13:03:07.014GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, "executorRunTime" : 83, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -674,7 +674,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index 11eec0b49c40b..96d86b7278ff1 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -39,21 +39,21 @@ } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", + "launchTime" : "2015-05-06T13:03:06.502GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, "executorRunTime" : 350, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -74,26 +74,26 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", + "launchTime" : "2015-05-06T13:03:06.505GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, "executorRunTime" : 350, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -114,22 +114,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.504GMT", + "launchTime" : "2015-05-06T13:03:06.494GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, @@ -137,7 +137,7 @@ "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, "outputMetrics" : { @@ -154,15 +154,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 3842811, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", + "launchTime" : "2015-05-06T13:03:06.504GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -173,7 +173,7 @@ "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -194,13 +194,13 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", "executorId" : "driver", @@ -213,7 +213,7 @@ "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -234,30 +234,30 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", + "launchTime" : "2015-05-06T13:03:06.506GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, "executorRunTime" : 349, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, "outputMetrics" : { @@ -274,7 +274,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } @@ -479,25 +479,25 @@ } } }, { - "taskId" : 16, - "index" : 16, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.001GMT", + "launchTime" : "2015-05-06T13:03:06.915GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 10, + "executorDeserializeTime" : 9, "executorRunTime" : 84, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, "outputMetrics" : { @@ -514,22 +514,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 108320, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 16, + "index" : 16, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", + "launchTime" : "2015-05-06T13:03:07.001GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 10, "executorRunTime" : 84, "resultSize" : 2010, "jvmGcTime" : 5, @@ -554,30 +554,30 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 108320, "recordsWritten" : 10 } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", + "launchTime" : "2015-05-06T13:03:07.012GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, "executorRunTime" : 84, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, "outputMetrics" : { @@ -594,25 +594,25 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", + "launchTime" : "2015-05-06T13:03:06.925GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, "executorRunTime" : 83, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -634,25 +634,25 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", + "launchTime" : "2015-05-06T13:03:07.014GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, "executorRunTime" : 83, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -674,7 +674,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index 9528d872ef731..e0e9e8140c717 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -39,21 +39,21 @@ } } }, { - "taskId" : 86, - "index" : 86, + "taskId" : 41, + "index" : 41, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.374GMT", + "launchTime" : "2015-05-06T13:03:07.200GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 2, "executorRunTime" : 16, "resultSize" : 2065, "jvmGcTime" : 0, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -74,15 +74,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95848, + "writeTime" : 90765, "recordsWritten" : 10 } } }, { - "taskId" : 41, - "index" : 41, + "taskId" : 43, + "index" : 43, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.200GMT", + "launchTime" : "2015-05-06T13:03:07.204GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -114,22 +114,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 90765, + "writeTime" : 171516, "recordsWritten" : 10 } } }, { - "taskId" : 68, - "index" : 68, + "taskId" : 57, + "index" : 57, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.306GMT", + "launchTime" : "2015-05-06T13:03:07.257GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 3, "executorRunTime" : 16, "resultSize" : 2065, "jvmGcTime" : 0, @@ -154,7 +154,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101750, + "writeTime" : 96849, "recordsWritten" : 10 } } @@ -199,10 +199,10 @@ } } }, { - "taskId" : 43, - "index" : 43, + "taskId" : 68, + "index" : 68, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.204GMT", + "launchTime" : "2015-05-06T13:03:07.306GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -234,15 +234,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 171516, + "writeTime" : 101750, "recordsWritten" : 10 } } }, { - "taskId" : 57, - "index" : 57, + "taskId" : 86, + "index" : 86, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.257GMT", + "launchTime" : "2015-05-06T13:03:07.374GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -253,7 +253,7 @@ "executorRunTime" : 16, "resultSize" : 2065, "jvmGcTime" : 0, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -274,15 +274,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 96849, + "writeTime" : 95848, "recordsWritten" : 10 } } }, { - "taskId" : 59, - "index" : 59, + "taskId" : 32, + "index" : 32, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.265GMT", + "launchTime" : "2015-05-06T13:03:07.148GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -314,22 +314,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 100753, + "writeTime" : 89603, "recordsWritten" : 10 } } }, { - "taskId" : 32, - "index" : 32, + "taskId" : 39, + "index" : 39, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.148GMT", + "launchTime" : "2015-05-06T13:03:07.180GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 2, "executorRunTime" : 17, "resultSize" : 2065, "jvmGcTime" : 0, @@ -354,22 +354,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 89603, + "writeTime" : 98748, "recordsWritten" : 10 } } }, { - "taskId" : 87, - "index" : 87, + "taskId" : 42, + "index" : 42, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.374GMT", + "launchTime" : "2015-05-06T13:03:07.203GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 10, "executorRunTime" : 17, "resultSize" : 2065, "jvmGcTime" : 0, @@ -394,15 +394,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 102159, + "writeTime" : 103713, "recordsWritten" : 10 } } }, { - "taskId" : 99, - "index" : 99, + "taskId" : 51, + "index" : 51, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.426GMT", + "launchTime" : "2015-05-06T13:03:07.242GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -417,7 +417,7 @@ "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70565, + "bytesRead" : 70564, "recordsRead" : 10000 }, "outputMetrics" : { @@ -434,25 +434,25 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 133964, + "writeTime" : 96013, "recordsWritten" : 10 } } }, { - "taskId" : 63, - "index" : 63, + "taskId" : 59, + "index" : 59, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.276GMT", + "launchTime" : "2015-05-06T13:03:07.265GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 20, + "executorDeserializeTime" : 3, "executorRunTime" : 17, "resultSize" : 2065, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -474,25 +474,25 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 102779, + "writeTime" : 100753, "recordsWritten" : 10 } } }, { - "taskId" : 90, - "index" : 90, + "taskId" : 63, + "index" : 63, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.385GMT", + "launchTime" : "2015-05-06T13:03:07.276GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 20, "executorRunTime" : 17, "resultSize" : 2065, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -514,22 +514,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98472, + "writeTime" : 102779, "recordsWritten" : 10 } } }, { - "taskId" : 39, - "index" : 39, + "taskId" : 87, + "index" : 87, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.180GMT", + "launchTime" : "2015-05-06T13:03:07.374GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 12, "executorRunTime" : 17, "resultSize" : 2065, "jvmGcTime" : 0, @@ -554,22 +554,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98748, + "writeTime" : 102159, "recordsWritten" : 10 } } }, { - "taskId" : 42, - "index" : 42, + "taskId" : 90, + "index" : 90, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.203GMT", + "launchTime" : "2015-05-06T13:03:07.385GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 10, + "executorDeserializeTime" : 2, "executorRunTime" : 17, "resultSize" : 2065, "jvmGcTime" : 0, @@ -594,15 +594,15 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 103713, + "writeTime" : 98472, "recordsWritten" : 10 } } }, { - "taskId" : 51, - "index" : 51, + "taskId" : 99, + "index" : 99, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.242GMT", + "launchTime" : "2015-05-06T13:03:07.426GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -617,7 +617,7 @@ "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 70565, "recordsRead" : 10000 }, "outputMetrics" : { @@ -634,22 +634,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 96013, + "writeTime" : 133964, "recordsWritten" : 10 } } }, { - "taskId" : 50, - "index" : 50, + "taskId" : 44, + "index" : 44, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.240GMT", + "launchTime" : "2015-05-06T13:03:07.205GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 4, + "executorDeserializeTime" : 3, "executorRunTime" : 18, "resultSize" : 2065, "jvmGcTime" : 0, @@ -674,22 +674,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 90836, + "writeTime" : 98293, "recordsWritten" : 10 } } }, { - "taskId" : 53, - "index" : 53, + "taskId" : 47, + "index" : 47, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.244GMT", + "launchTime" : "2015-05-06T13:03:07.212GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 2, "executorRunTime" : 18, "resultSize" : 2065, "jvmGcTime" : 0, @@ -714,22 +714,22 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 92835, + "writeTime" : 103015, "recordsWritten" : 10 } } }, { - "taskId" : 44, - "index" : 44, + "taskId" : 50, + "index" : 50, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.205GMT", + "launchTime" : "2015-05-06T13:03:07.240GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 4, "executorRunTime" : 18, "resultSize" : 2065, "jvmGcTime" : 0, @@ -754,25 +754,25 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98293, + "writeTime" : 90836, "recordsWritten" : 10 } } }, { - "taskId" : 80, - "index" : 80, + "taskId" : 52, + "index" : 52, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.341GMT", + "launchTime" : "2015-05-06T13:03:07.243GMT", "executorId" : "driver", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 5, "executorRunTime" : 18, "resultSize" : 2065, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -794,7 +794,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98069, + "writeTime" : 89664, "recordsWritten" : 10 } } diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala index f684e16c25f7c..1bfb0c1547ec4 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.status.api.v1 import java.util.Date -import scala.collection.mutable.HashMap +import scala.collection.mutable.LinkedHashMap import org.apache.spark.SparkFunSuite import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality} @@ -28,7 +28,7 @@ import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} class AllStagesResourceSuite extends SparkFunSuite { def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = { - val tasks = new HashMap[Long, TaskUIData] + val tasks = new LinkedHashMap[Long, TaskUIData] taskLaunchTimes.zipWithIndex.foreach { case (time, idx) => tasks(idx.toLong) = TaskUIData( new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None) diff --git a/docs/configuration.md b/docs/configuration.md index 500a6dad113da..d37da023db794 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -597,6 +597,14 @@ Apart from these, the following properties are also available, and may be useful collecting. + + spark.ui.retainedTasks + 100000 + + How many tasks the Spark UI and status APIs remember before garbage + collecting. + + spark.worker.ui.retainedExecutors 1000 From b3a44306a36d6c1e5583e85961966fa5cf4f7e9a Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Thu, 25 Aug 2016 12:11:27 -0700 Subject: [PATCH 1204/1470] [SPARKR][BUILD] ignore cran-check.out under R folder ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) R add cran check which will generate the cran-check.out. This file should be ignored in git. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manual test it. Run clean test and git status to make sure the file is not included in git. Author: wm624@hotmail.com Closes #14774 from wangmiao1981/ignore. (cherry picked from commit 9958ac0ce2b9e451d400604767bef2fe12a3399d) Signed-off-by: Shivaram Venkataraman --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index a2639766a592e..a32d408493d94 100644 --- a/.gitignore +++ b/.gitignore @@ -22,6 +22,7 @@ /lib/ R-unit-tests.log R/unit-tests.out +R/cran-check.out build/*.jar build/apache-maven* build/scala* From ff2e270ebe3a74c19140cd96f96b7a62723002b1 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 26 Aug 2016 00:15:01 +0200 Subject: [PATCH 1205/1470] [SPARK-17205] Literal.sql should handle Infinity and NaN This patch updates `Literal.sql` to properly generate SQL for `NaN` and `Infinity` float and double literals: these special values need to be handled differently from regular values, since simply appending a suffix to the value's `toString()` representation will not work for these values. Author: Josh Rosen Closes #14777 from JoshRosen/SPARK-17205. (cherry picked from commit 3e4c7db4d11c474457e7886a5501108ebab0cf6d) Signed-off-by: Herman van Hovell --- .../sql/catalyst/expressions/literals.scala | 17 +++++++++++++++-- .../catalyst/ExpressionSQLBuilderSuite.scala | 6 ++++++ 2 files changed, 21 insertions(+), 2 deletions(-) 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 55fd9c0834fcc..730a7f62e04c7 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 @@ -251,8 +251,21 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression with case (v: Short, ShortType) => v + "S" case (v: Long, LongType) => v + "L" // Float type doesn't have a suffix - case (v: Float, FloatType) => s"CAST($v AS ${FloatType.sql})" - case (v: Double, DoubleType) => v + "D" + case (v: Float, FloatType) => + val castedValue = v match { + case _ if v.isNaN => "'NaN'" + case Float.PositiveInfinity => "'Infinity'" + case Float.NegativeInfinity => "'-Infinity'" + case _ => v + } + s"CAST($castedValue AS ${FloatType.sql})" + case (v: Double, DoubleType) => + v match { + case _ if v.isNaN => s"CAST('NaN' AS ${DoubleType.sql})" + case Double.PositiveInfinity => s"CAST('Infinity' AS ${DoubleType.sql})" + case Double.NegativeInfinity => s"CAST('-Infinity' AS ${DoubleType.sql})" + case _ => v + "D" + } case (v: Decimal, t: DecimalType) => s"CAST($v AS ${t.sql})" case (v: Int, DateType) => s"DATE '${DateTimeUtils.toJavaDate(v)}'" case (v: Long, TimestampType) => s"TIMESTAMP('${DateTimeUtils.toJavaTimestamp(v)}')" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index 93dc0f493eb7b..86724cbb676c4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -32,7 +32,13 @@ class ExpressionSQLBuilderSuite extends SQLBuilderTest { checkSQL(Literal(4: Int), "4") checkSQL(Literal(8: Long), "8L") checkSQL(Literal(1.5F), "CAST(1.5 AS FLOAT)") + checkSQL(Literal(Float.PositiveInfinity), "CAST('Infinity' AS FLOAT)") + checkSQL(Literal(Float.NegativeInfinity), "CAST('-Infinity' AS FLOAT)") + checkSQL(Literal(Float.NaN), "CAST('NaN' AS FLOAT)") checkSQL(Literal(2.5D), "2.5D") + checkSQL(Literal(Double.PositiveInfinity), "CAST('Infinity' AS DOUBLE)") + checkSQL(Literal(Double.NegativeInfinity), "CAST('-Infinity' AS DOUBLE)") + checkSQL(Literal(Double.NaN), "CAST('NaN' AS DOUBLE)") checkSQL( Literal(Timestamp.valueOf("2016-01-01 00:00:00")), "TIMESTAMP('2016-01-01 00:00:00.0')") // TODO tests for decimals From 73014a2aa96b538d963f360fd41bac74f358ef46 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Thu, 25 Aug 2016 16:29:04 -0700 Subject: [PATCH 1206/1470] [SPARK-17231][CORE] Avoid building debug or trace log messages unless the respective log level is enabled This is simply a backport of #14798 to `branch-2.0`. This backport omits the change to `ExternalShuffleBlockHandler.java`. In `branch-2.0`, that file does not contain the log message that was patched in `master`. Author: Michael Allman Closes #14811 from mallman/spark-17231-logging_perf_improvements-2.0_backport. --- .../spark/network/client/TransportClient.java | 39 ++++++++++++------- .../client/TransportClientFactory.java | 2 +- .../client/TransportResponseHandler.java | 15 ++++--- .../network/protocol/MessageDecoder.java | 2 +- .../server/TransportChannelHandler.java | 6 +-- .../server/TransportRequestHandler.java | 18 ++++----- .../spark/network/server/TransportServer.java | 2 +- .../shuffle/ExternalShuffleBlockResolver.java | 2 +- 8 files changed, 47 insertions(+), 39 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index 64a83171e9e90..a67683b892218 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -43,7 +43,7 @@ import org.apache.spark.network.protocol.RpcRequest; import org.apache.spark.network.protocol.StreamChunkId; import org.apache.spark.network.protocol.StreamRequest; -import org.apache.spark.network.util.NettyUtils; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; /** * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow @@ -135,9 +135,10 @@ public void fetchChunk( long streamId, final int chunkIndex, final ChunkReceivedCallback callback) { - final String serverAddr = NettyUtils.getRemoteAddress(channel); final long startTime = System.currentTimeMillis(); - logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr); + if (logger.isDebugEnabled()) { + logger.debug("Sending fetch chunk request {} to {}", chunkIndex, getRemoteAddress(channel)); + } final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex); handler.addFetchRequest(streamChunkId, callback); @@ -148,11 +149,13 @@ public void fetchChunk( public void operationComplete(ChannelFuture future) throws Exception { if (future.isSuccess()) { long timeTaken = System.currentTimeMillis() - startTime; - logger.trace("Sending request {} to {} took {} ms", streamChunkId, serverAddr, - timeTaken); + if (logger.isTraceEnabled()) { + logger.trace("Sending request {} to {} took {} ms", streamChunkId, getRemoteAddress(channel), + timeTaken); + } } else { String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId, - serverAddr, future.cause()); + getRemoteAddress(channel), future.cause()); logger.error(errorMsg, future.cause()); handler.removeFetchRequest(streamChunkId); channel.close(); @@ -173,9 +176,10 @@ public void operationComplete(ChannelFuture future) throws Exception { * @param callback Object to call with the stream data. */ public void stream(final String streamId, final StreamCallback callback) { - final String serverAddr = NettyUtils.getRemoteAddress(channel); final long startTime = System.currentTimeMillis(); - logger.debug("Sending stream request for {} to {}", streamId, serverAddr); + if (logger.isDebugEnabled()) { + logger.debug("Sending stream request for {} to {}", streamId, getRemoteAddress(channel)); + } // Need to synchronize here so that the callback is added to the queue and the RPC is // written to the socket atomically, so that callbacks are called in the right order @@ -188,11 +192,13 @@ public void stream(final String streamId, final StreamCallback callback) { public void operationComplete(ChannelFuture future) throws Exception { if (future.isSuccess()) { long timeTaken = System.currentTimeMillis() - startTime; - logger.trace("Sending request for {} to {} took {} ms", streamId, serverAddr, - timeTaken); + if (logger.isTraceEnabled()) { + logger.trace("Sending request for {} to {} took {} ms", streamId, getRemoteAddress(channel), + timeTaken); + } } else { String errorMsg = String.format("Failed to send request for %s to %s: %s", streamId, - serverAddr, future.cause()); + getRemoteAddress(channel), future.cause()); logger.error(errorMsg, future.cause()); channel.close(); try { @@ -215,9 +221,10 @@ public void operationComplete(ChannelFuture future) throws Exception { * @return The RPC's id. */ public long sendRpc(ByteBuffer message, final RpcResponseCallback callback) { - final String serverAddr = NettyUtils.getRemoteAddress(channel); final long startTime = System.currentTimeMillis(); - logger.trace("Sending RPC to {}", serverAddr); + if (logger.isTraceEnabled()) { + logger.trace("Sending RPC to {}", getRemoteAddress(channel)); + } final long requestId = Math.abs(UUID.randomUUID().getLeastSignificantBits()); handler.addRpcRequest(requestId, callback); @@ -228,10 +235,12 @@ public long sendRpc(ByteBuffer message, final RpcResponseCallback callback) { public void operationComplete(ChannelFuture future) throws Exception { if (future.isSuccess()) { long timeTaken = System.currentTimeMillis() - startTime; - logger.trace("Sending request {} to {} took {} ms", requestId, serverAddr, timeTaken); + if (logger.isTraceEnabled()) { + logger.trace("Sending request {} to {} took {} ms", requestId, getRemoteAddress(channel), timeTaken); + } } else { String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, - serverAddr, future.cause()); + getRemoteAddress(channel), future.cause()); logger.error(errorMsg, future.cause()); handler.removeRpcRequest(requestId); channel.close(); diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index a27aaf2b277f7..1c9916baee07c 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -195,7 +195,7 @@ public TransportClient createUnmanagedClient(String remoteHost, int remotePort) /** Create a completely new {@link TransportClient} to the remote address. */ private TransportClient createClient(InetSocketAddress address) throws IOException { - logger.debug("Creating new connection to " + address); + logger.debug("Creating new connection to {}", address); Bootstrap bootstrap = new Bootstrap(); bootstrap.group(workerGroup) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 8a69223c88ee4..179667296ec7d 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -38,7 +38,7 @@ import org.apache.spark.network.protocol.StreamFailure; import org.apache.spark.network.protocol.StreamResponse; import org.apache.spark.network.server.MessageHandler; -import org.apache.spark.network.util.NettyUtils; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; import org.apache.spark.network.util.TransportFrameDecoder; /** @@ -122,7 +122,7 @@ public void channelActive() { @Override public void channelInactive() { if (numOutstandingRequests() > 0) { - String remoteAddress = NettyUtils.getRemoteAddress(channel); + String remoteAddress = getRemoteAddress(channel); logger.error("Still have {} requests outstanding when connection from {} is closed", numOutstandingRequests(), remoteAddress); failOutstandingRequests(new IOException("Connection from " + remoteAddress + " closed")); @@ -132,7 +132,7 @@ public void channelInactive() { @Override public void exceptionCaught(Throwable cause) { if (numOutstandingRequests() > 0) { - String remoteAddress = NettyUtils.getRemoteAddress(channel); + String remoteAddress = getRemoteAddress(channel); logger.error("Still have {} requests outstanding when connection from {} is closed", numOutstandingRequests(), remoteAddress); failOutstandingRequests(cause); @@ -141,13 +141,12 @@ public void exceptionCaught(Throwable cause) { @Override public void handle(ResponseMessage message) throws Exception { - String remoteAddress = NettyUtils.getRemoteAddress(channel); if (message instanceof ChunkFetchSuccess) { ChunkFetchSuccess resp = (ChunkFetchSuccess) message; ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); if (listener == null) { logger.warn("Ignoring response for block {} from {} since it is not outstanding", - resp.streamChunkId, remoteAddress); + resp.streamChunkId, getRemoteAddress(channel)); resp.body().release(); } else { outstandingFetches.remove(resp.streamChunkId); @@ -159,7 +158,7 @@ public void handle(ResponseMessage message) throws Exception { ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); if (listener == null) { logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding", - resp.streamChunkId, remoteAddress, resp.errorString); + resp.streamChunkId, getRemoteAddress(channel), resp.errorString); } else { outstandingFetches.remove(resp.streamChunkId); listener.onFailure(resp.streamChunkId.chunkIndex, new ChunkFetchFailureException( @@ -170,7 +169,7 @@ public void handle(ResponseMessage message) throws Exception { RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", - resp.requestId, remoteAddress, resp.body().size()); + resp.requestId, getRemoteAddress(channel), resp.body().size()); } else { outstandingRpcs.remove(resp.requestId); try { @@ -184,7 +183,7 @@ public void handle(ResponseMessage message) throws Exception { RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", - resp.requestId, remoteAddress, resp.errorString); + resp.requestId, getRemoteAddress(channel), resp.errorString); } else { outstandingRpcs.remove(resp.requestId); listener.onFailure(new RuntimeException(resp.errorString)); diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java index 074780f2b95ce..f0453186185e1 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java @@ -39,7 +39,7 @@ public void decode(ChannelHandlerContext ctx, ByteBuf in, List out) { Message.Type msgType = Message.Type.decode(in); Message decoded = decode(msgType, in); assert decoded.type() == msgType; - logger.trace("Received message " + msgType + ": " + decoded); + logger.trace("Received message {}: {}", msgType, decoded); out.add(decoded); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index f2223379a9d24..884ea7d1152a5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -29,7 +29,7 @@ import org.apache.spark.network.protocol.Message; import org.apache.spark.network.protocol.RequestMessage; import org.apache.spark.network.protocol.ResponseMessage; -import org.apache.spark.network.util.NettyUtils; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; /** * The single Transport-level Channel handler which is used for delegating requests to the @@ -76,7 +76,7 @@ public TransportClient getClient() { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - logger.warn("Exception in connection from " + NettyUtils.getRemoteAddress(ctx.channel()), + logger.warn("Exception in connection from " + getRemoteAddress(ctx.channel()), cause); requestHandler.exceptionCaught(cause); responseHandler.exceptionCaught(cause); @@ -139,7 +139,7 @@ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exc System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; if (e.state() == IdleState.ALL_IDLE && isActuallyOverdue) { if (responseHandler.numOutstandingRequests() > 0) { - String address = NettyUtils.getRemoteAddress(ctx.channel()); + String address = getRemoteAddress(ctx.channel()); logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + "requests. Assuming connection is dead; please adjust spark.network.timeout if " + "this is wrong.", address, requestTimeoutNs / 1000 / 1000); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index bebe88ec5d503..e67a034cb8e58 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -17,6 +17,7 @@ package org.apache.spark.network.server; +import java.net.SocketAddress; import java.nio.ByteBuffer; import com.google.common.base.Throwables; @@ -42,7 +43,7 @@ import org.apache.spark.network.protocol.StreamFailure; import org.apache.spark.network.protocol.StreamRequest; import org.apache.spark.network.protocol.StreamResponse; -import org.apache.spark.network.util.NettyUtils; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; /** * A handler that processes requests from clients and writes chunk data back. Each handler is @@ -114,9 +115,9 @@ public void handle(RequestMessage request) { } private void processFetchRequest(final ChunkFetchRequest req) { - final String client = NettyUtils.getRemoteAddress(channel); - - logger.trace("Received req from {} to fetch block {}", client, req.streamChunkId); + if (logger.isTraceEnabled()) { + logger.trace("Received req from {} to fetch block {}", getRemoteAddress(channel), req.streamChunkId); + } ManagedBuffer buf; try { @@ -125,7 +126,7 @@ private void processFetchRequest(final ChunkFetchRequest req) { buf = streamManager.getChunk(req.streamChunkId.streamId, req.streamChunkId.chunkIndex); } catch (Exception e) { logger.error(String.format( - "Error opening block %s for request from %s", req.streamChunkId, client), e); + "Error opening block %s for request from %s", req.streamChunkId, getRemoteAddress(channel)), e); respond(new ChunkFetchFailure(req.streamChunkId, Throwables.getStackTraceAsString(e))); return; } @@ -134,13 +135,12 @@ private void processFetchRequest(final ChunkFetchRequest req) { } private void processStreamRequest(final StreamRequest req) { - final String client = NettyUtils.getRemoteAddress(channel); ManagedBuffer buf; try { buf = streamManager.openStream(req.streamId); } catch (Exception e) { logger.error(String.format( - "Error opening stream %s for request from %s", req.streamId, client), e); + "Error opening stream %s for request from %s", req.streamId, getRemoteAddress(channel)), e); respond(new StreamFailure(req.streamId, Throwables.getStackTraceAsString(e))); return; } @@ -189,13 +189,13 @@ private void processOneWayMessage(OneWayMessage req) { * it will be logged and the channel closed. */ private void respond(final Encodable result) { - final String remoteAddress = channel.remoteAddress().toString(); + final SocketAddress remoteAddress = channel.remoteAddress(); channel.writeAndFlush(result).addListener( new ChannelFutureListener() { @Override public void operationComplete(ChannelFuture future) throws Exception { if (future.isSuccess()) { - logger.trace(String.format("Sent result %s to client %s", result, remoteAddress)); + logger.trace("Sent result {} to client {}", result, remoteAddress); } else { logger.error(String.format("Error sending result %s to %s; closing connection", result, remoteAddress), future.cause()); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index baae235e02205..a67db4f69f086 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -130,7 +130,7 @@ protected void initChannel(SocketChannel ch) throws Exception { channelFuture.syncUninterruptibly(); port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort(); - logger.debug("Shuffle server started on port :" + port); + logger.debug("Shuffle server started on port: {}", port); } @Override diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 54e870a9b56a6..000ec13f796a0 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -244,7 +244,7 @@ private void deleteExecutorDirs(String[] dirs) { for (String localDir : dirs) { try { JavaUtils.deleteRecursively(new File(localDir)); - logger.debug("Successfully cleaned up directory: " + localDir); + logger.debug("Successfully cleaned up directory: {}", localDir); } catch (Exception e) { logger.error("Failed to delete directory: " + localDir, e); } From 27ed6d5dcd521b4ff1ebe777b03a03ba103d6e76 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 25 Aug 2016 21:08:42 -0700 Subject: [PATCH 1207/1470] [SPARK-17242][DOCUMENT] Update links of external dstream projects ## What changes were proposed in this pull request? Updated links of external dstream projects. ## How was this patch tested? Just document changes. Author: Shixiong Zhu Closes #14814 from zsxwing/dstream-link. (cherry picked from commit 341e0e778dff8c404b47d34ee7661b658bb91880) Signed-off-by: Reynold Xin --- docs/streaming-programming-guide.md | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 14e17443e362c..b92ca92b06c4a 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -656,7 +656,7 @@ methods for creating DStreams from files as input sources. Python API `fileStream` is not available in the Python API, only `textFileStream` is available. - **Streams based on Custom Receivers:** DStreams can be created with data streams received through custom receivers. See the [Custom Receiver - Guide](streaming-custom-receivers.html) and [DStream Akka](https://github.com/spark-packages/dstream-akka) for more details. + Guide](streaming-custom-receivers.html) for more details. - **Queue of RDDs as a Stream:** For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using `streamingContext.queueStream(queueOfRDDs)`. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream. @@ -2383,11 +2383,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are - [Kafka Integration Guide](streaming-kafka-integration.html) - [Kinesis Integration Guide](streaming-kinesis-integration.html) - [Custom Receiver Guide](streaming-custom-receivers.html) -* External DStream data sources: - - [DStream MQTT](https://github.com/spark-packages/dstream-mqtt) - - [DStream Twitter](https://github.com/spark-packages/dstream-twitter) - - [DStream Akka](https://github.com/spark-packages/dstream-akka) - - [DStream ZeroMQ](https://github.com/spark-packages/dstream-zeromq) +* Third-party DStream data sources can be found in [Spark Packages](https://spark-packages.org/) * API documentation - Scala docs * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and From 6f82d2da382cee2950a0797436e5d48805cbba5f Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Fri, 26 Aug 2016 11:01:48 -0700 Subject: [PATCH 1208/1470] [SPARKR][MINOR] Fix example of spark.naiveBayes ## What changes were proposed in this pull request? The original example doesn't work because the features are not categorical. This PR fixes this by changing to another dataset. ## How was this patch tested? Manual test. Author: Junyang Qian Closes #14820 from junyangq/SPARK-FixNaiveBayes. (cherry picked from commit 18832162357282ec81515b5b2ba93747be3ad18b) Signed-off-by: Felix Cheung --- R/pkg/R/mllib.R | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 25dcb3a3943e6..b33a16a7cef97 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -433,10 +433,11 @@ setMethod("predict", signature(object = "KMeansModel"), #' @export #' @examples #' \dontrun{ -#' df <- createDataFrame(infert) +#' data <- as.data.frame(UCBAdmissions) +#' df <- createDataFrame(data) #' #' # fit a Bernoulli naive Bayes model -#' model <- spark.naiveBayes(df, education ~ ., smoothing = 0) +#' model <- spark.naiveBayes(df, Admit ~ Gender + Dept, smoothing = 0) #' #' # get the summary of the model #' summary(model) From deb6a54cf0f69d4ac5b3e1d358bb81e49eea412d Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Fri, 26 Aug 2016 11:30:23 -0700 Subject: [PATCH 1209/1470] [SPARK-17165][SQL] FileStreamSource should not track the list of seen files indefinitely ## What changes were proposed in this pull request? Before this change, FileStreamSource uses an in-memory hash set to track the list of files processed by the engine. The list can grow indefinitely, leading to OOM or overflow of the hash set. This patch introduces a new user-defined option called "maxFileAge", default to 24 hours. If a file is older than this age, FileStreamSource will purge it from the in-memory map that was used to track the list of files that have been processed. ## How was this patch tested? Added unit tests for the underlying utility, and also added an end-to-end test to validate the purge in FileStreamSourceSuite. Also verified the new test cases would fail when the timeout was set to a very large number. Author: petermaxlee Closes #14728 from petermaxlee/SPARK-17165. (cherry picked from commit 9812f7d5381f7cd8112fd30c7e45ae4f0eab6e88) Signed-off-by: Shixiong Zhu --- .../streaming/FileStreamOptions.scala | 54 +++++++ .../streaming/FileStreamSource.scala | 149 ++++++++++++++---- .../execution/streaming/HDFSMetadataLog.scala | 2 +- .../streaming/FileStreamSourceSuite.scala | 76 +++++++++ .../sql/streaming/FileStreamSourceSuite.scala | 40 ++++- 5 files changed, 285 insertions(+), 36 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala new file mode 100644 index 0000000000000..3efc20c1d662d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamOptions.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import scala.util.Try + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap +import org.apache.spark.util.Utils + +/** + * User specified options for file streams. + */ +class FileStreamOptions(parameters: Map[String, String]) extends Logging { + + val maxFilesPerTrigger: Option[Int] = parameters.get("maxFilesPerTrigger").map { str => + Try(str.toInt).toOption.filter(_ > 0).getOrElse { + throw new IllegalArgumentException( + s"Invalid value '$str' for option 'maxFilesPerTrigger', must be a positive integer") + } + } + + /** + * Maximum age of a file that can be found in this directory, before it is deleted. + * + * The max age is specified with respect to the timestamp of the latest file, and not the + * timestamp of the current system. That this means if the last file has timestamp 1000, and the + * current system time is 2000, and max age is 200, the system will purge files older than + * 800 (rather than 1800) from the internal state. + * + * Default to a week. + */ + val maxFileAgeMs: Long = + Utils.timeStringAsMs(parameters.getOrElse("maxFileAge", "7d")) + + /** Options as specified by the user, in a case-insensitive map, without "path" set. */ + val optionMapWithoutPath: Map[String, String] = + new CaseInsensitiveMap(parameters).filterKeys(_ != "path") +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 0cfad659dc92c..e8b969b5e0a55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -17,21 +17,20 @@ package org.apache.spark.sql.execution.streaming -import scala.util.Try +import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} -import org.apache.spark.sql.execution.datasources.{CaseInsensitiveMap, DataSource, ListingFileCatalog, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{DataSource, ListingFileCatalog, LogicalRelation} import org.apache.spark.sql.types.StructType -import org.apache.spark.util.collection.OpenHashSet /** - * A very simple source that reads text files from the given directory as they appear. + * A very simple source that reads files from the given directory as they appear. * - * TODO Clean up the metadata files periodically + * TODO: Clean up the metadata log files periodically. */ class FileStreamSource( sparkSession: SparkSession, @@ -41,19 +40,34 @@ class FileStreamSource( metadataPath: String, options: Map[String, String]) extends Source with Logging { - private val fs = new Path(path).getFileSystem(sparkSession.sessionState.newHadoopConf()) - private val qualifiedBasePath = fs.makeQualified(new Path(path)) // can contains glob patterns - private val metadataLog = new HDFSMetadataLog[Seq[String]](sparkSession, metadataPath) + import FileStreamSource._ + + private val sourceOptions = new FileStreamOptions(options) + + private val qualifiedBasePath: Path = { + val fs = new Path(path).getFileSystem(sparkSession.sessionState.newHadoopConf()) + fs.makeQualified(new Path(path)) // can contains glob patterns + } + + private val metadataLog = new HDFSMetadataLog[Seq[FileEntry]](sparkSession, metadataPath) + private var maxBatchId = metadataLog.getLatest().map(_._1).getOrElse(-1L) /** Maximum number of new files to be considered in each batch */ - private val maxFilesPerBatch = getMaxFilesPerBatch() + private val maxFilesPerBatch = sourceOptions.maxFilesPerTrigger + + /** A mapping from a file that we have processed to some timestamp it was last modified. */ + // Visible for testing and debugging in production. + val seenFiles = new SeenFilesMap(sourceOptions.maxFileAgeMs) - private val seenFiles = new OpenHashSet[String] - metadataLog.get(None, Some(maxBatchId)).foreach { case (batchId, files) => - files.foreach(seenFiles.add) + metadataLog.get(None, Some(maxBatchId)).foreach { case (batchId, entry) => + entry.foreach(seenFiles.add) + // TODO: move purge call out of the loop once we truncate logs. + seenFiles.purge() } + logInfo(s"maxFilesPerBatch = $maxFilesPerBatch, maxFileAge = ${sourceOptions.maxFileAgeMs}") + /** * Returns the maximum offset that can be retrieved from the source. * @@ -61,16 +75,27 @@ class FileStreamSource( * there is no race here, so the cost of `synchronized` should be rare. */ private def fetchMaxOffset(): LongOffset = synchronized { - val newFiles = fetchAllFiles().filter(!seenFiles.contains(_)) + // All the new files found - ignore aged files and files that we have seen. + val newFiles = fetchAllFiles().filter(seenFiles.isNewFile) + + // Obey user's setting to limit the number of files in this batch trigger. val batchFiles = if (maxFilesPerBatch.nonEmpty) newFiles.take(maxFilesPerBatch.get) else newFiles + batchFiles.foreach { file => seenFiles.add(file) logDebug(s"New file: $file") } - logTrace(s"Number of new files = ${newFiles.size})") - logTrace(s"Number of files selected for batch = ${batchFiles.size}") - logTrace(s"Number of seen files = ${seenFiles.size}") + val numPurged = seenFiles.purge() + + logTrace( + s""" + |Number of new files = ${newFiles.size} + |Number of files selected for batch = ${batchFiles.size} + |Number of seen files = ${seenFiles.size} + |Number of files purged from tracking map = $numPurged + """.stripMargin) + if (batchFiles.nonEmpty) { maxBatchId += 1 metadataLog.add(maxBatchId, batchFiles) @@ -104,22 +129,26 @@ class FileStreamSource( val files = metadataLog.get(Some(startId + 1), Some(endId)).flatMap(_._2) logInfo(s"Processing ${files.length} files from ${startId + 1}:$endId") logTrace(s"Files are:\n\t" + files.mkString("\n\t")) - val newOptions = new CaseInsensitiveMap(options).filterKeys(_ != "path") val newDataSource = DataSource( sparkSession, - paths = files, + paths = files.map(_.path), userSpecifiedSchema = Some(schema), className = fileFormatClassName, - options = newOptions) + options = sourceOptions.optionMapWithoutPath) Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation())) } - private def fetchAllFiles(): Seq[String] = { + /** + * Returns a list of files found, sorted by their timestamp. + */ + private def fetchAllFiles(): Seq[FileEntry] = { val startTime = System.nanoTime val globbedPaths = SparkHadoopUtil.get.globPathIfNecessary(qualifiedBasePath) val catalog = new ListingFileCatalog(sparkSession, globbedPaths, options, Some(new StructType)) - val files = catalog.allFiles().sortBy(_.getModificationTime).map(_.getPath.toUri.toString) + val files = catalog.allFiles().sortBy(_.getModificationTime).map { status => + FileEntry(status.getPath.toUri.toString, status.getModificationTime) + } val endTime = System.nanoTime val listingTimeMs = (endTime.toDouble - startTime) / 1000000 if (listingTimeMs > 2000) { @@ -132,20 +161,76 @@ class FileStreamSource( files } - private def getMaxFilesPerBatch(): Option[Int] = { - new CaseInsensitiveMap(options) - .get("maxFilesPerTrigger") - .map { str => - Try(str.toInt).toOption.filter(_ > 0).getOrElse { - throw new IllegalArgumentException( - s"Invalid value '$str' for option 'maxFilesPerTrigger', must be a positive integer") - } - } - } - override def getOffset: Option[Offset] = Some(fetchMaxOffset()).filterNot(_.offset == -1) override def toString: String = s"FileStreamSource[$qualifiedBasePath]" override def stop() {} } + + +object FileStreamSource { + + /** Timestamp for file modification time, in ms since January 1, 1970 UTC. */ + type Timestamp = Long + + case class FileEntry(path: String, timestamp: Timestamp) extends Serializable + + /** + * A custom hash map used to track the list of files seen. This map is not thread-safe. + * + * To prevent the hash map from growing indefinitely, a purge function is available to + * remove files "maxAgeMs" older than the latest file. + */ + class SeenFilesMap(maxAgeMs: Long) { + require(maxAgeMs >= 0) + + /** Mapping from file to its timestamp. */ + private val map = new java.util.HashMap[String, Timestamp] + + /** Timestamp of the latest file. */ + private var latestTimestamp: Timestamp = 0L + + /** Timestamp for the last purge operation. */ + private var lastPurgeTimestamp: Timestamp = 0L + + /** Add a new file to the map. */ + def add(file: FileEntry): Unit = { + map.put(file.path, file.timestamp) + if (file.timestamp > latestTimestamp) { + latestTimestamp = file.timestamp + } + } + + /** + * Returns true if we should consider this file a new file. The file is only considered "new" + * if it is new enough that we are still tracking, and we have not seen it before. + */ + def isNewFile(file: FileEntry): Boolean = { + // Note that we are testing against lastPurgeTimestamp here so we'd never miss a file that + // is older than (latestTimestamp - maxAgeMs) but has not been purged yet. + file.timestamp >= lastPurgeTimestamp && !map.containsKey(file.path) + } + + /** Removes aged entries and returns the number of files removed. */ + def purge(): Int = { + lastPurgeTimestamp = latestTimestamp - maxAgeMs + val iter = map.entrySet().iterator() + var count = 0 + while (iter.hasNext) { + val entry = iter.next() + if (entry.getValue < lastPurgeTimestamp) { + count += 1 + iter.remove() + } + } + count + } + + def size: Int = map.size() + + def allEntries: Seq[FileEntry] = { + map.entrySet().asScala.map(entry => FileEntry(entry.getKey, entry.getValue)).toSeq + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 698f07b0a187f..2b6f76ca28e21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -180,7 +180,7 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) private def isFileAlreadyExistsException(e: IOException): Boolean = { e.isInstanceOf[FileAlreadyExistsException] || // Old Hadoop versions don't throw FileAlreadyExistsException. Although it's fixed in - // HADOOP-9361, we still need to support old Hadoop versions. + // HADOOP-9361 in Hadoop 2.5, we still need to support old Hadoop versions. (e.getMessage != null && e.getMessage.startsWith("File already exists: ")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala new file mode 100644 index 0000000000000..c6db2fd3f908e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import org.apache.spark.SparkFunSuite + +class FileStreamSourceSuite extends SparkFunSuite { + + import FileStreamSource._ + + test("SeenFilesMap") { + val map = new SeenFilesMap(maxAgeMs = 10) + + map.add(FileEntry("a", 5)) + assert(map.size == 1) + map.purge() + assert(map.size == 1) + + // Add a new entry and purge should be no-op, since the gap is exactly 10 ms. + map.add(FileEntry("b", 15)) + assert(map.size == 2) + map.purge() + assert(map.size == 2) + + // Add a new entry that's more than 10 ms than the first entry. We should be able to purge now. + map.add(FileEntry("c", 16)) + assert(map.size == 3) + map.purge() + assert(map.size == 2) + + // Override existing entry shouldn't change the size + map.add(FileEntry("c", 25)) + assert(map.size == 2) + + // Not a new file because we have seen c before + assert(!map.isNewFile(FileEntry("c", 20))) + + // Not a new file because timestamp is too old + assert(!map.isNewFile(FileEntry("d", 5))) + + // Finally a new file: never seen and not too old + assert(map.isNewFile(FileEntry("e", 20))) + } + + test("SeenFilesMap should only consider a file old if it is earlier than last purge time") { + val map = new SeenFilesMap(maxAgeMs = 10) + + map.add(FileEntry("a", 20)) + assert(map.size == 1) + + // Timestamp 5 should still considered a new file because purge time should be 0 + assert(map.isNewFile(FileEntry("b", 9))) + assert(map.isNewFile(FileEntry("b", 10))) + + // Once purge, purge time should be 10 and then b would be a old file if it is less than 10. + map.purge() + assert(!map.isNewFile(FileEntry("b", 9))) + assert(map.isNewFile(FileEntry("b", 10))) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 47260a23c7ee3..03222b4a49c6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -104,12 +104,13 @@ class FileStreamSourceTest extends StreamTest with SharedSQLContext { def createFileStream( format: String, path: String, - schema: Option[StructType] = None): DataFrame = { + schema: Option[StructType] = None, + options: Map[String, String] = Map.empty): DataFrame = { val reader = if (schema.isDefined) { - spark.readStream.format(format).schema(schema.get) + spark.readStream.format(format).schema(schema.get).options(options) } else { - spark.readStream.format(format) + spark.readStream.format(format).options(options) } reader.load(path) } @@ -331,6 +332,39 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + test("SPARK-17165 should not track the list of seen files indefinitely") { + // This test works by: + // 1. Create a file + // 2. Get it processed + // 3. Sleeps for a very short amount of time (larger than maxFileAge + // 4. Add another file (at this point the original file should have been purged + // 5. Test the size of the seenFiles internal data structure + + // Note that if we change maxFileAge to a very large number, the last step should fail. + withTempDirs { case (src, tmp) => + val textStream: DataFrame = + createFileStream("text", src.getCanonicalPath, options = Map("maxFileAge" -> "5ms")) + + testStream(textStream)( + AddTextFileData("a\nb", src, tmp), + CheckAnswer("a", "b"), + + // SLeeps longer than 5ms (maxFileAge) + AssertOnQuery { _ => Thread.sleep(10); true }, + + AddTextFileData("c\nd", src, tmp), + CheckAnswer("a", "b", "c", "d"), + + AssertOnQuery("seen files should contain only one entry") { streamExecution => + val source = streamExecution.logicalPlan.collect { case e: StreamingExecutionRelation => + e.source.asInstanceOf[FileStreamSource] + }.head + source.seenFiles.size == 1 + } + ) + } + } + // =============== JSON file stream tests ================ test("read from json files") { From 52feb3fbf75a234d041703e3ac41884294ab0b64 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Fri, 26 Aug 2016 13:29:22 -0700 Subject: [PATCH 1210/1470] [SPARK-17246][SQL] Add BigDecimal literal ## What changes were proposed in this pull request? This PR adds parser support for `BigDecimal` literals. If you append the suffix `BD` to a valid number then this will be interpreted as a `BigDecimal`, for example `12.0E10BD` will interpreted into a BigDecimal with scale -9 and precision 3. This is useful in situations where you need exact values. ## How was this patch tested? Added tests to `ExpressionParserSuite`, `ExpressionSQLBuilderSuite` and `SQLQueryTestSuite`. Author: Herman van Hovell Closes #14819 from hvanhovell/SPARK-17246. (cherry picked from commit a11d10f1826b578ff721c4738224eef2b3c3b9f3) Signed-off-by: Reynold Xin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 6 +++++ .../sql/catalyst/expressions/literals.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 16 ++++++++++++- .../parser/ExpressionParserSuite.scala | 7 ++++++ .../resources/sql-tests/inputs/literals.sql | 6 +++++ .../sql-tests/results/literals.sql.out | 24 ++++++++++++++++++- .../catalyst/ExpressionSQLBuilderSuite.scala | 1 + 7 files changed, 59 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 51f3804fbd793..ecb7c8adeba82 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -626,6 +626,7 @@ number | MINUS? SMALLINT_LITERAL #smallIntLiteral | MINUS? TINYINT_LITERAL #tinyIntLiteral | MINUS? DOUBLE_LITERAL #doubleLiteral + | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; nonReserved @@ -920,6 +921,11 @@ DOUBLE_LITERAL (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'D' ; +BIGDECIMAL_LITERAL + : + (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'BD' + ; + IDENTIFIER : (LETTER | DIGIT | '_')+ ; 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 730a7f62e04c7..41e3952f0e253 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 @@ -266,7 +266,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression with case Double.NegativeInfinity => s"CAST('-Infinity' AS ${DoubleType.sql})" case _ => v + "D" } - case (v: Decimal, t: DecimalType) => s"CAST($v AS ${t.sql})" + case (v: Decimal, t: DecimalType) => v + "BD" case (v: Int, DateType) => s"DATE '${DateTimeUtils.toJavaDate(v)}'" case (v: Long, TimestampType) => s"TIMESTAMP('${DateTimeUtils.toJavaTimestamp(v)}')" case _ => value.toString diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index aec312622a7ac..0451abe2aaa07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -26,7 +26,8 @@ import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIdentifier} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ @@ -1318,6 +1319,19 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { numericLiteral(ctx, Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble) } + /** + * Create a BigDecimal Literal expression. + */ + override def visitBigDecimalLiteral(ctx: BigDecimalLiteralContext): Literal = { + val raw = ctx.getText.substring(0, ctx.getText.length - 2) + try { + Literal(BigDecimal(raw).underlying()) + } catch { + case e: AnalysisException => + throw new ParseException(e.message, ctx) + } + } + /** * Create a String literal expression. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 401d9cd9d288c..dbc5db39aed98 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -392,6 +392,13 @@ class ExpressionParserSuite extends PlanTest { intercept("1.8E308D", s"does not fit in range") // TODO we need to figure out if we should throw an exception here! assertEqual("1E309", Literal(Double.PositiveInfinity)) + + // BigDecimal Literal + assertEqual("90912830918230182310293801923652346786BD", + Literal(BigDecimal("90912830918230182310293801923652346786").underlying())) + assertEqual("123.0E-28BD", Literal(BigDecimal("123.0E-28").underlying())) + assertEqual("123.08BD", Literal(BigDecimal("123.08").underlying())) + intercept("1.20E-38BD", "DecimalType can only support precision up to 38") } test("strings") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 62f0d3d0599c6..a532a598c6bf9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -90,3 +90,9 @@ select interval 10 nanoseconds; -- unsupported data type select GEO '(10,-6)'; + +-- big decimal parsing +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD; + +-- out of range big decimal +select 1.20E-38BD; diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 67e6d78dfbf24..85629f7ba813a 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 38 +-- Number of queries: 40 -- !query 0 @@ -354,3 +354,25 @@ Literals of type 'GEO' are currently not supported.(line 1, pos 7) == SQL == select GEO '(10,-6)' -------^^^ + + +-- !query 38 +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD +-- !query 38 schema +struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> +-- !query 38 output +90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 + + +-- !query 39 +select 1.20E-38BD +-- !query 39 schema +struct<> +-- !query 39 output +org.apache.spark.sql.catalyst.parser.ParseException + +DecimalType can only support precision up to 38(line 1, pos 7) + +== SQL == +select 1.20E-38BD +-------^^^ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index 86724cbb676c4..43a218b4d14b3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -39,6 +39,7 @@ class ExpressionSQLBuilderSuite extends SQLBuilderTest { checkSQL(Literal(Double.PositiveInfinity), "CAST('Infinity' AS DOUBLE)") checkSQL(Literal(Double.NegativeInfinity), "CAST('-Infinity' AS DOUBLE)") checkSQL(Literal(Double.NaN), "CAST('NaN' AS DOUBLE)") + checkSQL(Literal(BigDecimal("10.0000000").underlying), "10.0000000BD") checkSQL( Literal(Timestamp.valueOf("2016-01-01 00:00:00")), "TIMESTAMP('2016-01-01 00:00:00.0')") // TODO tests for decimals From dfdfc3092d1b6942eb9092e28e15fa4efb6ac084 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Fri, 26 Aug 2016 16:05:34 -0700 Subject: [PATCH 1211/1470] [SPARK-17235][SQL] Support purging of old logs in MetadataLog ## What changes were proposed in this pull request? This patch adds a purge interface to MetadataLog, and an implementation in HDFSMetadataLog. The purge function is currently unused, but I will use it to purge old execution and file source logs in follow-up patches. These changes are required in a production structured streaming job that runs for a long period of time. ## How was this patch tested? Added a unit test case in HDFSMetadataLogSuite. Author: petermaxlee Closes #14802 from petermaxlee/SPARK-17235. (cherry picked from commit f64a1ddd09a34d5d867ccbaba46204d75fad038d) Signed-off-by: Reynold Xin --- .../execution/streaming/HDFSMetadataLog.scala | 14 ++++++++++ .../sql/execution/streaming/MetadataLog.scala | 6 +++++ .../streaming/HDFSMetadataLogSuite.scala | 27 ++++++++++++++++--- 3 files changed, 43 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 2b6f76ca28e21..127ece9ab0e56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -227,6 +227,20 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) None } + /** + * Removes all the log entry earlier than thresholdBatchId (exclusive). + */ + override def purge(thresholdBatchId: Long): Unit = { + val batchIds = fileManager.list(metadataPath, batchFilesFilter) + .map(f => pathToBatchId(f.getPath)) + + for (batchId <- batchIds if batchId < thresholdBatchId) { + val path = batchIdToPath(batchId) + fileManager.delete(path) + logTrace(s"Removed metadata log file: $path") + } + } + private def createFileManager(): FileManager = { val hadoopConf = sparkSession.sessionState.newHadoopConf() try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLog.scala index cc70e1d314d1d..78d6be17df05a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLog.scala @@ -48,4 +48,10 @@ trait MetadataLog[T] { * Return the latest batch Id and its metadata if exist. */ def getLatest(): Option[(Long, T)] + + /** + * Removes all the log entry earlier than thresholdBatchId (exclusive). + * This operation should be idempotent. + */ + def purge(thresholdBatchId: Long): Unit } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index ab5a2d253b94a..4259384f0bc61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -46,14 +46,14 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { test("FileManager: FileContextManager") { withTempDir { temp => val path = new Path(temp.getAbsolutePath) - testManager(path, new FileContextManager(path, new Configuration)) + testFileManager(path, new FileContextManager(path, new Configuration)) } } test("FileManager: FileSystemManager") { withTempDir { temp => val path = new Path(temp.getAbsolutePath) - testManager(path, new FileSystemManager(path, new Configuration)) + testFileManager(path, new FileSystemManager(path, new Configuration)) } } @@ -103,6 +103,25 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { } } + testWithUninterruptibleThread("HDFSMetadataLog: purge") { + withTempDir { temp => + val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) + assert(metadataLog.add(0, "batch0")) + assert(metadataLog.add(1, "batch1")) + assert(metadataLog.add(2, "batch2")) + assert(metadataLog.get(0).isDefined) + assert(metadataLog.get(1).isDefined) + assert(metadataLog.get(2).isDefined) + assert(metadataLog.getLatest().get._1 == 2) + + metadataLog.purge(2) + assert(metadataLog.get(0).isEmpty) + assert(metadataLog.get(1).isEmpty) + assert(metadataLog.get(2).isDefined) + assert(metadataLog.getLatest().get._1 == 2) + } + } + testWithUninterruptibleThread("HDFSMetadataLog: restart") { withTempDir { temp => val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) @@ -155,8 +174,8 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { } } - - def testManager(basePath: Path, fm: FileManager): Unit = { + /** Basic test case for [[FileManager]] implementation. */ + private def testFileManager(basePath: Path, fm: FileManager): Unit = { // Mkdirs val dir = new Path(s"$basePath/dir/subdir/subsubdir") assert(!fm.exists(dir)) From 9c0ac6b53abeb7b5f6fa2e8328d7559be8259300 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 26 Aug 2016 16:40:59 -0700 Subject: [PATCH 1212/1470] [SPARK-17244] Catalyst should not pushdown non-deterministic join conditions ## What changes were proposed in this pull request? Given that non-deterministic expressions can be stateful, pushing them down the query plan during the optimization phase can cause incorrect behavior. This patch fixes that issue by explicitly disabling that. ## How was this patch tested? A new test in `FilterPushdownSuite` that checks catalyst behavior for both deterministic and non-deterministic join conditions. Author: Sameer Agarwal Closes #14815 from sameeragarwal/constraint-inputfile. (cherry picked from commit 540e91280147a61727f99592a66c0cbb12328fac) Signed-off-by: Yin Huai --- .../sql/catalyst/optimizer/Optimizer.scala | 21 ++++++++++++------- .../optimizer/FilterPushdownSuite.scala | 14 +++++++++++++ 2 files changed, 28 insertions(+), 7 deletions(-) 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 5dd51abffaca1..4cadbc32522de 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 @@ -1286,18 +1286,25 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { */ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { /** - * Splits join condition expressions into three categories based on the attributes required - * to evaluate them. + * Splits join condition expressions or filter predicates (on a given join's output) into three + * categories based on the attributes required to evaluate them. Note that we explicitly exclude + * on-deterministic (i.e., stateful) condition expressions in canEvaluateInLeft or + * canEvaluateInRight to prevent pushing these predicates on either side of the join. * * @return (canEvaluateInLeft, canEvaluateInRight, haveToEvaluateInBoth) */ private def split(condition: Seq[Expression], left: LogicalPlan, right: LogicalPlan) = { + // Note: In order to ensure correctness, it's important to not change the relative ordering of + // any deterministic expression that follows a non-deterministic expression. To achieve this, + // we only consider pushing down those expressions that precede the first non-deterministic + // expression in the condition. + val (pushDownCandidates, containingNonDeterministic) = condition.span(_.deterministic) val (leftEvaluateCondition, rest) = - condition.partition(_.references subsetOf left.outputSet) + pushDownCandidates.partition(_.references.subsetOf(left.outputSet)) val (rightEvaluateCondition, commonCondition) = - rest.partition(_.references subsetOf right.outputSet) + rest.partition(expr => expr.references.subsetOf(right.outputSet)) - (leftEvaluateCondition, rightEvaluateCondition, commonCondition) + (leftEvaluateCondition, rightEvaluateCondition, commonCondition ++ containingNonDeterministic) } def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -1348,7 +1355,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { } // push down the join filter into sub query scanning if applicable - case f @ Join(left, right, joinType, joinCondition) => + case j @ Join(left, right, joinType, joinCondition) => val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) @@ -1378,7 +1385,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newJoinCond = (leftJoinConditions ++ commonJoinCondition).reduceLeftOption(And) Join(newLeft, newRight, LeftOuter, newJoinCond) - case FullOuter => f + case FullOuter => j case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") case UsingJoin(_, _) => sys.error("Untransformed Using join node") } 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 1dcabf9dcedd1..ea868d1a73a92 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 @@ -987,4 +987,18 @@ class FilterPushdownSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer) } + + test("join condition pushdown: deterministic and non-deterministic") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + // Verify that all conditions preceding the first non-deterministic condition are pushed down + // by the optimizer and others are not. + val originalQuery = x.join(y, condition = Some("x.a".attr === 5 && "y.a".attr === 5 && + "x.a".attr === Rand(10) && "y.b".attr === 5)) + val correctAnswer = x.where("x.a".attr === 5).join(y.where("y.a".attr === 5), + condition = Some("x.a".attr === Rand(10) && "y.b".attr === 5)) + + comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze) + } } From 94d52d76569f8b0782f424cfac959a4bb75c54c0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 26 Aug 2016 22:10:28 -0700 Subject: [PATCH 1213/1470] [SPARK-17269][SQL] Move finish analysis optimization stage into its own file As part of breaking Optimizer.scala apart, this patch moves various finish analysis optimization stage rules into a single file. I'm submitting separate pull requests so we can more easily merge this in branch-2.0 to simplify optimizer backports. This should be covered by existing tests. Author: Reynold Xin Closes #14838 from rxin/SPARK-17269. (cherry picked from commit dcefac438788c51d84641bfbc505efe095731a39) Signed-off-by: Reynold Xin --- .../sql/catalyst/optimizer/Optimizer.scala | 38 ----------- .../RewriteDistinctAggregates.scala | 2 +- .../catalyst/optimizer/finishAnalysis.scala | 65 +++++++++++++++++++ 3 files changed, 66 insertions(+), 39 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/{analysis => optimizer}/RewriteDistinctAggregates.scala (99%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 4cadbc32522de..f3f1d212fe441 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 @@ -1582,44 +1582,6 @@ object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { } } -/** - * Finds all [[RuntimeReplaceable]] expressions and replace them with the expressions that can - * be evaluated. This is mainly used to provide compatibility with other databases. - * For example, we use this to support "nvl" by replacing it with "coalesce". - */ -object ReplaceExpressions extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case e: RuntimeReplaceable => e.replaced - } -} - -/** - * Computes the current date and time to make sure we return the same result in a single query. - */ -object ComputeCurrentTime extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = { - val dateExpr = CurrentDate() - val timeExpr = CurrentTimestamp() - val currentDate = Literal.create(dateExpr.eval(EmptyRow), dateExpr.dataType) - val currentTime = Literal.create(timeExpr.eval(EmptyRow), timeExpr.dataType) - - plan transformAllExpressions { - case CurrentDate() => currentDate - case CurrentTimestamp() => currentTime - } - } -} - -/** Replaces the expression of CurrentDatabase with the current database name. */ -case class GetCurrentDatabase(sessionCatalog: SessionCatalog) extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = { - plan transformAllExpressions { - case CurrentDatabase() => - Literal.create(sessionCatalog.getCurrentDatabase, StringType) - } - } -} - /** * Typed [[Filter]] is by default surrounded by a [[DeserializeToObject]] beneath it and a * [[SerializeFromObject]] above it. If these serializations can't be eliminated, we should embed diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDistinctAggregates.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index 8afd28dbba5c2..0f43e7bb88733 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.analysis +package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Complete} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala new file mode 100644 index 0000000000000..7c667315870f5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.catalog.SessionCatalog +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types._ + + +/** + * Finds all [[RuntimeReplaceable]] expressions and replace them with the expressions that can + * be evaluated. This is mainly used to provide compatibility with other databases. + * For example, we use this to support "nvl" by replacing it with "coalesce". + */ +object ReplaceExpressions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case e: RuntimeReplaceable => e.replaced + } +} + + +/** + * Computes the current date and time to make sure we return the same result in a single query. + */ +object ComputeCurrentTime extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val dateExpr = CurrentDate() + val timeExpr = CurrentTimestamp() + val currentDate = Literal.create(dateExpr.eval(EmptyRow), dateExpr.dataType) + val currentTime = Literal.create(timeExpr.eval(EmptyRow), timeExpr.dataType) + + plan transformAllExpressions { + case CurrentDate() => currentDate + case CurrentTimestamp() => currentTime + } + } +} + + +/** Replaces the expression of CurrentDatabase with the current database name. */ +case class GetCurrentDatabase(sessionCatalog: SessionCatalog) extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + plan transformAllExpressions { + case CurrentDatabase() => + Literal.create(sessionCatalog.getCurrentDatabase, StringType) + } + } +} From f91614f36472957355fad7d69d66327807fe80c8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 27 Aug 2016 00:31:49 -0700 Subject: [PATCH 1214/1470] [SPARK-17270][SQL] Move object optimization rules into its own file (branch-2.0) ## What changes were proposed in this pull request? As part of breaking Optimizer.scala apart, this patch moves various Dataset object optimization rules into a single file. I'm submitting separate pull requests so we can more easily merge this in branch-2.0 to simplify optimizer backports. This is https://github.com/apache/spark/pull/14839 but for branch-2.0. ## How was this patch tested? This should be covered by existing tests. Author: Reynold Xin Closes #14843 from rxin/SPARK-17270-branch-2.0. --- .../sql/catalyst/optimizer/Optimizer.scala | 72 ------------- .../sql/catalyst/optimizer/objects.scala | 101 ++++++++++++++++++ 2 files changed, 101 insertions(+), 72 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index f3f1d212fe441..15d33c1a9f3e0 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 @@ -186,25 +186,6 @@ object RemoveAliasOnlyProject extends Rule[LogicalPlan] { } } -/** - * Removes cases where we are unnecessarily going between the object and serialized (InternalRow) - * representation of data item. For example back to back map operations. - */ -object EliminateSerialization extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case d @ DeserializeToObject(_, _, s: SerializeFromObject) - if d.outputObjectType == s.inputObjectType => - // Adds an extra Project here, to preserve the output expr id of `DeserializeToObject`. - // We will remove it later in RemoveAliasOnlyProject rule. - val objAttr = - Alias(s.child.output.head, s.child.output.head.name)(exprId = d.output.head.exprId) - Project(objAttr :: Nil, s.child) - case a @ AppendColumns(_, _, _, s: SerializeFromObject) - if a.deserializer.dataType == s.inputObjectType => - AppendColumnsWithObject(a.func, s.serializer, a.serializer, s.child) - } -} - /** * Pushes down [[LocalLimit]] beneath UNION ALL and beneath the streamed inputs of outer joins. */ @@ -1582,59 +1563,6 @@ object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { } } -/** - * Typed [[Filter]] is by default surrounded by a [[DeserializeToObject]] beneath it and a - * [[SerializeFromObject]] above it. If these serializations can't be eliminated, we should embed - * the deserializer in filter condition to save the extra serialization at last. - */ -object EmbedSerializerInFilter extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) - // SPARK-15632: Conceptually, filter operator should never introduce schema change. This - // optimization rule also relies on this assumption. However, Dataset typed filter operator - // does introduce schema changes in some cases. Thus, we only enable this optimization when - // - // 1. either input and output schemata are exactly the same, or - // 2. both input and output schemata are single-field schema and share the same type. - // - // The 2nd case is included because encoders for primitive types always have only a single - // field with hard-coded field name "value". - // TODO Cleans this up after fixing SPARK-15632. - if s.schema == d.child.schema || samePrimitiveType(s.schema, d.child.schema) => - - val numObjects = condition.collect { - case a: Attribute if a == d.output.head => a - }.length - - if (numObjects > 1) { - // If the filter condition references the object more than one times, we should not embed - // deserializer in it as the deserialization will happen many times and slow down the - // execution. - // TODO: we can still embed it if we can make sure subexpression elimination works here. - s - } else { - val newCondition = condition transform { - case a: Attribute if a == d.output.head => d.deserializer - } - val filter = Filter(newCondition, d.child) - - // Adds an extra Project here, to preserve the output expr id of `SerializeFromObject`. - // We will remove it later in RemoveAliasOnlyProject rule. - val objAttrs = filter.output.zip(s.output).map { case (fout, sout) => - Alias(fout, fout.name)(exprId = sout.exprId) - } - Project(objAttrs, filter) - } - } - - def samePrimitiveType(lhs: StructType, rhs: StructType): Boolean = { - (lhs, rhs) match { - case (StructType(Array(f1)), StructType(Array(f2))) => f1.dataType == f2.dataType - case _ => false - } - } -} - /** * This rule rewrites predicate sub-queries into left semi/anti joins. The following predicates * are supported: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala new file mode 100644 index 0000000000000..8a25cee614c6f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.StructType + +/* + * This file defines optimization rules related to object manipulation (for the Dataset API). + */ + + +/** + * Removes cases where we are unnecessarily going between the object and serialized (InternalRow) + * representation of data item. For example back to back map operations. + */ +object EliminateSerialization extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case d @ DeserializeToObject(_, _, s: SerializeFromObject) + if d.outputObjectType == s.inputObjectType => + // Adds an extra Project here, to preserve the output expr id of `DeserializeToObject`. + // We will remove it later in RemoveAliasOnlyProject rule. + val objAttr = + Alias(s.child.output.head, s.child.output.head.name)(exprId = d.output.head.exprId) + Project(objAttr :: Nil, s.child) + case a @ AppendColumns(_, _, _, s: SerializeFromObject) + if a.deserializer.dataType == s.inputObjectType => + AppendColumnsWithObject(a.func, s.serializer, a.serializer, s.child) + } +} + + +/** + * Typed [[Filter]] is by default surrounded by a [[DeserializeToObject]] beneath it and a + * [[SerializeFromObject]] above it. If these serializations can't be eliminated, we should embed + * the deserializer in filter condition to save the extra serialization at last. + */ +object EmbedSerializerInFilter extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case s @ SerializeFromObject(_, Filter(condition, d: DeserializeToObject)) + // SPARK-15632: Conceptually, filter operator should never introduce schema change. This + // optimization rule also relies on this assumption. However, Dataset typed filter operator + // does introduce schema changes in some cases. Thus, we only enable this optimization when + // + // 1. either input and output schemata are exactly the same, or + // 2. both input and output schemata are single-field schema and share the same type. + // + // The 2nd case is included because encoders for primitive types always have only a single + // field with hard-coded field name "value". + // TODO Cleans this up after fixing SPARK-15632. + if s.schema == d.child.schema || samePrimitiveType(s.schema, d.child.schema) => + + val numObjects = condition.collect { + case a: Attribute if a == d.output.head => a + }.length + + if (numObjects > 1) { + // If the filter condition references the object more than one times, we should not embed + // deserializer in it as the deserialization will happen many times and slow down the + // execution. + // TODO: we can still embed it if we can make sure subexpression elimination works here. + s + } else { + val newCondition = condition transform { + case a: Attribute if a == d.output.head => d.deserializer + } + val filter = Filter(newCondition, d.child) + + // Adds an extra Project here, to preserve the output expr id of `SerializeFromObject`. + // We will remove it later in RemoveAliasOnlyProject rule. + val objAttrs = filter.output.zip(s.output).map { case (fout, sout) => + Alias(fout, fout.name)(exprId = sout.exprId) + } + Project(objAttrs, filter) + } + } + + def samePrimitiveType(lhs: StructType, rhs: StructType): Boolean = { + (lhs, rhs) match { + case (StructType(Array(f1)), StructType(Array(f2))) => f1.dataType == f2.dataType + case _ => false + } + } +} From 901ab06949addd05be6cb85df4eb6bd2104777e8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 27 Aug 2016 00:36:18 -0700 Subject: [PATCH 1215/1470] [SPARK-17274][SQL] Move join optimizer rules into a separate file ## What changes were proposed in this pull request? As part of breaking Optimizer.scala apart, this patch moves various join rules into a single file. ## How was this patch tested? This should be covered by existing tests. Author: Reynold Xin Closes #14846 from rxin/SPARK-17274. (cherry picked from commit 718b6bad2d698b76be6906d51da13626e9f3890e) Signed-off-by: Reynold Xin --- .../sql/catalyst/optimizer/Optimizer.scala | 106 -------------- .../spark/sql/catalyst/optimizer/joins.scala | 134 ++++++++++++++++++ 2 files changed, 134 insertions(+), 106 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 15d33c1a9f3e0..e74389816e007 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 @@ -1149,112 +1149,6 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { } } -/** - * Reorder the joins and push all the conditions into join, so that the bottom ones have at least - * one condition. - * - * The order of joins will not be changed if all of them already have at least one condition. - */ -object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { - - /** - * Join a list of plans together and push down the conditions into them. - * - * The joined plan are picked from left to right, prefer those has at least one join condition. - * - * @param input a list of LogicalPlans to join. - * @param conditions a list of condition for join. - */ - @tailrec - def createOrderedJoin(input: Seq[LogicalPlan], conditions: Seq[Expression]): LogicalPlan = { - assert(input.size >= 2) - if (input.size == 2) { - val (joinConditions, others) = conditions.partition( - e => !SubqueryExpression.hasCorrelatedSubquery(e)) - val join = Join(input(0), input(1), Inner, joinConditions.reduceLeftOption(And)) - if (others.nonEmpty) { - Filter(others.reduceLeft(And), join) - } else { - join - } - } else { - val left :: rest = input.toList - // find out the first join that have at least one join condition - val conditionalJoin = rest.find { plan => - val refs = left.outputSet ++ plan.outputSet - conditions.filterNot(canEvaluate(_, left)).filterNot(canEvaluate(_, plan)) - .exists(_.references.subsetOf(refs)) - } - // pick the next one if no condition left - val right = conditionalJoin.getOrElse(rest.head) - - val joinedRefs = left.outputSet ++ right.outputSet - val (joinConditions, others) = conditions.partition( - e => e.references.subsetOf(joinedRefs) && !SubqueryExpression.hasCorrelatedSubquery(e)) - val joined = Join(left, right, Inner, joinConditions.reduceLeftOption(And)) - - // should not have reference to same logical plan - createOrderedJoin(Seq(joined) ++ rest.filterNot(_ eq right), others) - } - } - - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case j @ ExtractFiltersAndInnerJoins(input, conditions) - if input.size > 2 && conditions.nonEmpty => - createOrderedJoin(input, conditions) - } -} - -/** - * Elimination of outer joins, if the predicates can restrict the result sets so that - * all null-supplying rows are eliminated - * - * - full outer -> inner if both sides have such predicates - * - left outer -> inner if the right side has such predicates - * - right outer -> inner if the left side has such predicates - * - full outer -> left outer if only the left side has such predicates - * - full outer -> right outer if only the right side has such predicates - * - * This rule should be executed before pushing down the Filter - */ -object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { - - /** - * Returns whether the expression returns null or false when all inputs are nulls. - */ - private def canFilterOutNull(e: Expression): Boolean = { - if (!e.deterministic || SubqueryExpression.hasCorrelatedSubquery(e)) return false - val attributes = e.references.toSeq - val emptyRow = new GenericInternalRow(attributes.length) - val v = BindReferences.bindReference(e, attributes).eval(emptyRow) - v == null || v == false - } - - private def buildNewJoinType(filter: Filter, join: Join): JoinType = { - val conditions = splitConjunctivePredicates(filter.condition) ++ filter.constraints - val leftConditions = conditions.filter(_.references.subsetOf(join.left.outputSet)) - val rightConditions = conditions.filter(_.references.subsetOf(join.right.outputSet)) - - val leftHasNonNullPredicate = leftConditions.exists(canFilterOutNull) - val rightHasNonNullPredicate = rightConditions.exists(canFilterOutNull) - - join.joinType match { - case RightOuter if leftHasNonNullPredicate => Inner - case LeftOuter if rightHasNonNullPredicate => Inner - case FullOuter if leftHasNonNullPredicate && rightHasNonNullPredicate => Inner - case FullOuter if leftHasNonNullPredicate => LeftOuter - case FullOuter if rightHasNonNullPredicate => RightOuter - case o => o - } - } - - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | FullOuter, _)) => - val newJoinType = buildNewJoinType(f, j) - if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType = newJoinType)) - } -} - /** * Pushes down [[Filter]] operators where the `condition` can be * evaluated using only the attributes of the left or right side of a join. Other diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala new file mode 100644 index 0000000000000..158ad3d91fbad --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import scala.annotation.tailrec + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + + +/** + * Reorder the joins and push all the conditions into join, so that the bottom ones have at least + * one condition. + * + * The order of joins will not be changed if all of them already have at least one condition. + */ +object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { + + /** + * Join a list of plans together and push down the conditions into them. + * + * The joined plan are picked from left to right, prefer those has at least one join condition. + * + * @param input a list of LogicalPlans to join. + * @param conditions a list of condition for join. + */ + @tailrec + def createOrderedJoin(input: Seq[LogicalPlan], conditions: Seq[Expression]): LogicalPlan = { + assert(input.size >= 2) + if (input.size == 2) { + val (joinConditions, others) = conditions.partition( + e => !SubqueryExpression.hasCorrelatedSubquery(e)) + val join = Join(input(0), input(1), Inner, joinConditions.reduceLeftOption(And)) + if (others.nonEmpty) { + Filter(others.reduceLeft(And), join) + } else { + join + } + } else { + val left :: rest = input.toList + // find out the first join that have at least one join condition + val conditionalJoin = rest.find { plan => + val refs = left.outputSet ++ plan.outputSet + conditions.filterNot(canEvaluate(_, left)).filterNot(canEvaluate(_, plan)) + .exists(_.references.subsetOf(refs)) + } + // pick the next one if no condition left + val right = conditionalJoin.getOrElse(rest.head) + + val joinedRefs = left.outputSet ++ right.outputSet + val (joinConditions, others) = conditions.partition( + e => e.references.subsetOf(joinedRefs) && !SubqueryExpression.hasCorrelatedSubquery(e)) + val joined = Join(left, right, Inner, joinConditions.reduceLeftOption(And)) + + // should not have reference to same logical plan + createOrderedJoin(Seq(joined) ++ rest.filterNot(_ eq right), others) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case j @ ExtractFiltersAndInnerJoins(input, conditions) + if input.size > 2 && conditions.nonEmpty => + createOrderedJoin(input, conditions) + } +} + + +/** + * Elimination of outer joins, if the predicates can restrict the result sets so that + * all null-supplying rows are eliminated + * + * - full outer -> inner if both sides have such predicates + * - left outer -> inner if the right side has such predicates + * - right outer -> inner if the left side has such predicates + * - full outer -> left outer if only the left side has such predicates + * - full outer -> right outer if only the right side has such predicates + * + * This rule should be executed before pushing down the Filter + */ +object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { + + /** + * Returns whether the expression returns null or false when all inputs are nulls. + */ + private def canFilterOutNull(e: Expression): Boolean = { + if (!e.deterministic || SubqueryExpression.hasCorrelatedSubquery(e)) return false + val attributes = e.references.toSeq + val emptyRow = new GenericInternalRow(attributes.length) + val v = BindReferences.bindReference(e, attributes).eval(emptyRow) + v == null || v == false + } + + private def buildNewJoinType(filter: Filter, join: Join): JoinType = { + val conditions = splitConjunctivePredicates(filter.condition) ++ filter.constraints + val leftConditions = conditions.filter(_.references.subsetOf(join.left.outputSet)) + val rightConditions = conditions.filter(_.references.subsetOf(join.right.outputSet)) + + val leftHasNonNullPredicate = leftConditions.exists(canFilterOutNull) + val rightHasNonNullPredicate = rightConditions.exists(canFilterOutNull) + + join.joinType match { + case RightOuter if leftHasNonNullPredicate => Inner + case LeftOuter if rightHasNonNullPredicate => Inner + case FullOuter if leftHasNonNullPredicate && rightHasNonNullPredicate => Inner + case FullOuter if leftHasNonNullPredicate => LeftOuter + case FullOuter if rightHasNonNullPredicate => RightOuter + case o => o + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | FullOuter, _)) => + val newJoinType = buildNewJoinType(f, j) + if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType = newJoinType)) + } +} From 56a8426355b494bc085b649ae6c8245f0a039e3a Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Sat, 27 Aug 2016 08:42:41 +0100 Subject: [PATCH 1216/1470] [SPARK-15382][SQL] Fix a bug in sampling with replacement ## What changes were proposed in this pull request? This pr to fix a bug below in sampling with replacement ``` val df = Seq((1, 0), (2, 0), (3, 0)).toDF("a", "b") df.sample(true, 2.0).withColumn("c", monotonically_increasing_id).select($"c").show +---+ | c| +---+ | 0| | 1| | 1| | 1| | 2| +---+ ``` ## How was this patch tested? Added a test in `DataFrameSuite`. Author: Takeshi YAMAMURO Closes #14800 from maropu/FixSampleBug. (cherry picked from commit cd0ed31ea9965563a9b1ea3e8bfbeaf8347cacd9) Signed-off-by: Sean Owen --- .../spark/sql/execution/basicPhysicalOperators.scala | 1 + .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 90bf817d5c922..a544371ffee7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -260,6 +260,7 @@ case class SampleExec( if (withReplacement) { val samplerClass = classOf[PoissonSampler[UnsafeRow]].getName val initSampler = ctx.freshName("initSampler") + ctx.copyResult = true ctx.addMutableState(s"$samplerClass", sampler, s"$initSampler();") 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 55edbe2d44686..da5c538eace2d 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 @@ -1558,4 +1558,11 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val df = spark.createDataFrame(rdd, StructType(schemas), false) assert(df.persist.take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100) } + + test("copy results for sampling with replacement") { + val df = Seq((1, 0), (2, 0), (3, 0)).toDF("a", "b") + val sampleDf = df.sample(true, 2.00) + val d = sampleDf.withColumn("c", monotonically_increasing_id).select($"c").collect + assert(d.size == d.distinct.size) + } } From 7306c5f9098a25a7253b3b6fc27badb7d96b18d7 Mon Sep 17 00:00:00 2001 From: "Peng, Meng" Date: Sat, 27 Aug 2016 08:46:01 +0100 Subject: [PATCH 1217/1470] [ML][MLLIB] The require condition and message doesn't match in SparseMatrix. ## What changes were proposed in this pull request? The require condition and message doesn't match, and the condition also should be optimized. Small change. Please kindly let me know if JIRA required. ## How was this patch tested? No additional test required. Author: Peng, Meng Closes #14824 from mpjlu/smallChangeForMatrixRequire. (cherry picked from commit 40168dbe771ae662ed61851a1f3c677dd14fe344) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/ml/linalg/Matrices.scala | 11 +++++++---- .../org/apache/spark/mllib/linalg/Matrices.scala | 11 +++++++---- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index 0ea687bbccc54..f1ecc65af1105 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -454,10 +454,13 @@ class SparseMatrix @Since("2.0.0") ( require(values.length == rowIndices.length, "The number of row indices and values don't match! " + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") - // The Or statement is for the case when the matrix is transposed - require(colPtrs.length == numCols + 1 || colPtrs.length == numRows + 1, "The length of the " + - "column indices should be the number of columns + 1. Currently, colPointers.length: " + - s"${colPtrs.length}, numCols: $numCols") + if (isTransposed) { + require(colPtrs.length == numRows + 1, + s"Expecting ${numRows + 1} colPtrs when numRows = $numRows but got ${colPtrs.length}") + } else { + require(colPtrs.length == numCols + 1, + s"Expecting ${numCols + 1} colPtrs when numCols = $numCols but got ${colPtrs.length}") + } require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " + s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}") 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 e8f34388cd9fe..4c39cf17f4271 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 @@ -572,10 +572,13 @@ class SparseMatrix @Since("1.3.0") ( require(values.length == rowIndices.length, "The number of row indices and values don't match! " + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") - // The Or statement is for the case when the matrix is transposed - require(colPtrs.length == numCols + 1 || colPtrs.length == numRows + 1, "The length of the " + - "column indices should be the number of columns + 1. Currently, colPointers.length: " + - s"${colPtrs.length}, numCols: $numCols") + if (isTransposed) { + require(colPtrs.length == numRows + 1, + s"Expecting ${numRows + 1} colPtrs when numRows = $numRows but got ${colPtrs.length}") + } else { + require(colPtrs.length == numCols + 1, + s"Expecting ${numCols + 1} colPtrs when numCols = $numCols but got ${colPtrs.length}") + } require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " + s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}") From 5487fa0b8708deb31a872a943a6cccd6e922c575 Mon Sep 17 00:00:00 2001 From: Robert Kruszewski Date: Sat, 27 Aug 2016 08:47:15 +0100 Subject: [PATCH 1218/1470] [SPARK-17216][UI] fix event timeline bars length ## What changes were proposed in this pull request? Make event timeline bar expand to full length of the bar (which is total time) This issue occurs only on chrome, firefox looks fine. Haven't tested other browsers. ## How was this patch tested? Inspection in browsers Before ![screen shot 2016-08-24 at 3 38 24 pm](https://cloud.githubusercontent.com/assets/512084/17935104/0d6cda74-6a12-11e6-9c66-e00cfa855606.png) After ![screen shot 2016-08-24 at 3 36 39 pm](https://cloud.githubusercontent.com/assets/512084/17935114/15740ea4-6a12-11e6-83a1-7c06eef6abb8.png) Author: Robert Kruszewski Closes #14791 from robert3005/robertk/event-timeline. (cherry picked from commit 9fbced5b25c2f24d50c50516b4b7737f7e3eaf86) Signed-off-by: Sean Owen --- .../resources/org/apache/spark/ui/static/timeline-view.css | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css index 0f400461c5293..f9ad9f8378803 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -83,6 +83,10 @@ rect.getting-result-time-proportion { stroke: #75B0A6; } +.vis-item .vis-item-content { + width: 100% +} + .vis.timeline { line-height: 14px; } From eec03718db7e412f466ea72f3d9c2682915080bc Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 28 Aug 2016 16:30:33 +0200 Subject: [PATCH 1219/1470] [SPARK-16216][SQL][FOLLOWUP][BRANCH-2.0] Bacoport enabling timestamp type tests for JSON and verify all unsupported types in CSV ## What changes were proposed in this pull request? This backports https://github.com/apache/spark/pull/14829 ## How was this patch tested? Tests in `JsonHadoopFsRelation` and `CSVSuite`. Author: hyukjinkwon Closes #14840 from HyukjinKwon/SPARK-16216-followup-backport. --- .../datasources/csv/CSVFileFormat.scala | 19 ++++++++++++------- .../datasources/csv/CSVInferSchema.scala | 1 + .../execution/datasources/csv/CSVSuite.scala | 15 ++++++++++++++- .../sources/JsonHadoopFsRelationSuite.scala | 4 ---- 4 files changed, 27 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 12e19f955caa2..4a60f51c47b48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -180,13 +180,18 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { } private def verifySchema(schema: StructType): Unit = { - schema.foreach { field => - field.dataType match { - case _: ArrayType | _: MapType | _: StructType => - throw new UnsupportedOperationException( - s"CSV data source does not support ${field.dataType.simpleString} data type.") - case _ => - } + def verifyType(dataType: DataType): Unit = dataType match { + case ByteType | ShortType | IntegerType | LongType | FloatType | + DoubleType | BooleanType | _: DecimalType | TimestampType | + DateType | StringType => + + case udt: UserDefinedType[_] => verifyType(udt.sqlType) + + case _ => + throw new UnsupportedOperationException( + s"CSV data source does not support ${dataType.simpleString} data type.") } + + schema.foreach(field => verifyType(field.dataType)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index f1b4c11878a94..1ca6eff1b8c2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -290,6 +290,7 @@ private[csv] object CSVTypeCast { DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) } case _: StringType => UTF8String.fromString(datum) + case udt: UserDefinedType[_] => castTo(datum, udt.sqlType, nullable, options) case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f68d220afb6da..1930862118e9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.SparkException -import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.{DataFrame, QueryTest, Row, UDT} import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ @@ -680,6 +680,19 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { Seq((1, Array("Tesla", "Chevy", "Ford"))).toDF("id", "brands").write.csv(csvDir) }.getMessage assert(msg.contains("CSV data source does not support array data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors") + .write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support array data type")) + + msg = intercept[SparkException] { + val schema = StructType(StructField("a", new UDT.MyDenseVectorUDT(), true) :: Nil) + spark.range(1).write.csv(csvDir) + spark.read.schema(schema).csv(csvDir).collect() + }.getCause.getMessage + assert(msg.contains("Unsupported type: array")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala index 52486b122a93f..d79edee5b1a4c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala @@ -32,10 +32,6 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { override protected def supportsDataType(dataType: DataType): Boolean = dataType match { case _: NullType => false case _: BinaryType => false - // `TimestampType` is disabled because `DatatypeConverter.parseDateTime()` - // in `DateTimeUtils` parses the formatted string wrongly when the date is - // too early. (e.g. "1600-07-13T08:36:32.847"). - case _: TimestampType => false case _: CalendarIntervalType => false case _ => true } From 3d283f6c9d9daef53fa4e90b0ead2a94710a37a7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 29 Aug 2016 11:23:53 -0700 Subject: [PATCH 1220/1470] [SPARK-17063] [SQL] Improve performance of MSCK REPAIR TABLE with Hive metastore This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions). It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default). Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster). Author: Davies Liu Closes #14607 from davies/repair_batch. (cherry picked from commit 48caec2516ef35bfa1a3de2dc0a80d0dc819e6bd) Signed-off-by: Davies Liu --- .../sql/catalyst/catalog/interface.scala | 4 +- .../spark/sql/execution/command/ddl.scala | 156 +++++++++++++++--- .../apache/spark/sql/internal/SQLConf.scala | 10 ++ .../sql/execution/command/DDLSuite.scala | 13 +- .../sql/hive/client/HiveClientImpl.scala | 4 +- .../spark/sql/hive/client/HiveShim.scala | 8 +- .../sql/hive/execution/HiveDDLSuite.scala | 38 +++++ 7 files changed, 200 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index c083cf60ce6bc..e7430b030901a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -103,10 +103,12 @@ case class CatalogColumn( * * @param spec partition spec values indexed by column name * @param storage storage format of the partition + * @param parameters some parameters for the partition, for example, stats. */ case class CatalogTablePartition( spec: CatalogTypes.TablePartitionSpec, - storage: CatalogStorageFormat) + storage: CatalogStorageFormat, + parameters: Map[String, String] = Map.empty) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index aac70e90b883c..50ffcd4557ef8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.execution.command -import scala.collection.GenSeq +import scala.collection.{GenMap, GenSeq} import scala.collection.parallel.ForkJoinTaskSupport import scala.concurrent.forkjoin.ForkJoinPool import scala.util.control.NonFatal -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -34,6 +35,7 @@ import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.types._ +import org.apache.spark.util.SerializableConfiguration // Note: The definition of these commands are based on the ones described in // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL @@ -429,6 +431,9 @@ case class AlterTableDropPartitionCommand( } + +case class PartitionStatistics(numFiles: Int, totalSize: Long) + /** * Recover Partitions in ALTER TABLE: recover all the partition in the directory of a table and * update the catalog. @@ -442,6 +447,31 @@ case class AlterTableDropPartitionCommand( case class AlterTableRecoverPartitionsCommand( tableName: TableIdentifier, cmd: String = "ALTER TABLE RECOVER PARTITIONS") extends RunnableCommand { + + // These are list of statistics that can be collected quickly without requiring a scan of the data + // see https://github.com/apache/hive/blob/master/ + // common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java + val NUM_FILES = "numFiles" + val TOTAL_SIZE = "totalSize" + val DDL_TIME = "transient_lastDdlTime" + + private def getPathFilter(hadoopConf: Configuration): PathFilter = { + // Dummy jobconf to get to the pathFilter defined in configuration + // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) + val jobConf = new JobConf(hadoopConf, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + new PathFilter { + override def accept(path: Path): Boolean = { + val name = path.getName + if (name != "_SUCCESS" && name != "_temporary" && !name.startsWith(".")) { + pathFilter == null || pathFilter.accept(path) + } else { + false + } + } + } + } + override def run(spark: SparkSession): Seq[Row] = { val catalog = spark.sessionState.catalog if (!catalog.tableExists(tableName)) { @@ -456,10 +486,6 @@ case class AlterTableRecoverPartitionsCommand( throw new AnalysisException( s"Operation not allowed: $cmd on datasource tables: $tableName") } - if (table.tableType != CatalogTableType.EXTERNAL) { - throw new AnalysisException( - s"Operation not allowed: $cmd only works on external tables: $tableName") - } if (!DDLUtils.isTablePartitioned(table)) { throw new AnalysisException( s"Operation not allowed: $cmd only works on partitioned tables: $tableName") @@ -470,19 +496,26 @@ case class AlterTableRecoverPartitionsCommand( } val root = new Path(table.storage.locationUri.get) + logInfo(s"Recover all the partitions in $root") val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) - // Dummy jobconf to get to the pathFilter defined in configuration - // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) - val jobConf = new JobConf(spark.sparkContext.hadoopConfiguration, this.getClass) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + + val threshold = spark.conf.get("spark.rdd.parallelListingThreshold", "10").toInt + val hadoopConf = spark.sparkContext.hadoopConfiguration + val pathFilter = getPathFilter(hadoopConf) val partitionSpecsAndLocs = scanPartitions( - spark, fs, pathFilter, root, Map(), table.partitionColumnNames.map(_.toLowerCase)) - val parts = partitionSpecsAndLocs.map { case (spec, location) => - // inherit table storage format (possibly except for location) - CatalogTablePartition(spec, table.storage.copy(locationUri = Some(location.toUri.toString))) + spark, fs, pathFilter, root, Map(), table.partitionColumnNames.map(_.toLowerCase), threshold) + val total = partitionSpecsAndLocs.length + logInfo(s"Found $total partitions in $root") + + val partitionStats = if (spark.sqlContext.conf.gatherFastStats) { + gatherPartitionStats(spark, partitionSpecsAndLocs, fs, pathFilter, threshold) + } else { + GenMap.empty[String, PartitionStatistics] } - spark.sessionState.catalog.createPartitions(tableName, - parts.toArray[CatalogTablePartition], ignoreIfExists = true) + logInfo(s"Finished to gather the fast stats for all $total partitions.") + + addPartitions(spark, table, partitionSpecsAndLocs, partitionStats) + logInfo(s"Recovered all partitions ($total).") Seq.empty[Row] } @@ -494,15 +527,16 @@ case class AlterTableRecoverPartitionsCommand( filter: PathFilter, path: Path, spec: TablePartitionSpec, - partitionNames: Seq[String]): GenSeq[(TablePartitionSpec, Path)] = { - if (partitionNames.length == 0) { + partitionNames: Seq[String], + threshold: Int): GenSeq[(TablePartitionSpec, Path)] = { + if (partitionNames.isEmpty) { return Seq(spec -> path) } - val statuses = fs.listStatus(path) - val threshold = spark.conf.get("spark.rdd.parallelListingThreshold", "10").toInt + val statuses = fs.listStatus(path, filter) val statusPar: GenSeq[FileStatus] = if (partitionNames.length > 1 && statuses.length > threshold || partitionNames.length > 2) { + // parallelize the list of partitions here, then we can have better parallelism later. val parArray = statuses.par parArray.tasksupport = evalTaskSupport parArray @@ -517,21 +551,89 @@ case class AlterTableRecoverPartitionsCommand( // TODO: Validate the value val value = PartitioningUtils.unescapePathName(ps(1)) // comparing with case-insensitive, but preserve the case - if (columnName == partitionNames(0)) { - scanPartitions( - spark, fs, filter, st.getPath, spec ++ Map(columnName -> value), partitionNames.drop(1)) + if (columnName == partitionNames.head) { + scanPartitions(spark, fs, filter, st.getPath, spec ++ Map(columnName -> value), + partitionNames.drop(1), threshold) } else { - logWarning(s"expect partition column ${partitionNames(0)}, but got ${ps(0)}, ignore it") + logWarning(s"expect partition column ${partitionNames.head}, but got ${ps(0)}, ignore it") Seq() } } else { - if (name != "_SUCCESS" && name != "_temporary" && !name.startsWith(".")) { - logWarning(s"ignore ${new Path(path, name)}") - } + logWarning(s"ignore ${new Path(path, name)}") Seq() } } } + + private def gatherPartitionStats( + spark: SparkSession, + partitionSpecsAndLocs: GenSeq[(TablePartitionSpec, Path)], + fs: FileSystem, + pathFilter: PathFilter, + threshold: Int): GenMap[String, PartitionStatistics] = { + if (partitionSpecsAndLocs.length > threshold) { + val hadoopConf = spark.sparkContext.hadoopConfiguration + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = partitionSpecsAndLocs.map(_._2.toString).toArray + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(serializedPaths.length, + Math.min(spark.sparkContext.defaultParallelism, 10000)) + // gather the fast stats for all the partitions otherwise Hive metastore will list all the + // files for all the new partitions in sequential way, which is super slow. + logInfo(s"Gather the fast stats in parallel using $numParallelism tasks.") + spark.sparkContext.parallelize(serializedPaths, numParallelism) + .mapPartitions { paths => + val pathFilter = getPathFilter(serializableConfiguration.value) + paths.map(new Path(_)).map{ path => + val fs = path.getFileSystem(serializableConfiguration.value) + val statuses = fs.listStatus(path, pathFilter) + (path.toString, PartitionStatistics(statuses.length, statuses.map(_.getLen).sum)) + } + }.collectAsMap() + } else { + partitionSpecsAndLocs.map { case (_, location) => + val statuses = fs.listStatus(location, pathFilter) + (location.toString, PartitionStatistics(statuses.length, statuses.map(_.getLen).sum)) + }.toMap + } + } + + private def addPartitions( + spark: SparkSession, + table: CatalogTable, + partitionSpecsAndLocs: GenSeq[(TablePartitionSpec, Path)], + partitionStats: GenMap[String, PartitionStatistics]): Unit = { + val total = partitionSpecsAndLocs.length + var done = 0L + // Hive metastore may not have enough memory to handle millions of partitions in single RPC, + // we should split them into smaller batches. Since Hive client is not thread safe, we cannot + // do this in parallel. + val batchSize = 100 + partitionSpecsAndLocs.toIterator.grouped(batchSize).foreach { batch => + val now = System.currentTimeMillis() / 1000 + val parts = batch.map { case (spec, location) => + val params = partitionStats.get(location.toString).map { + case PartitionStatistics(numFiles, totalSize) => + // This two fast stat could prevent Hive metastore to list the files again. + Map(NUM_FILES -> numFiles.toString, + TOTAL_SIZE -> totalSize.toString, + // Workaround a bug in HiveMetastore that try to mutate a read-only parameters. + // see metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java + DDL_TIME -> now.toString) + }.getOrElse(Map.empty) + // inherit table storage format (possibly except for location) + CatalogTablePartition( + spec, + table.storage.copy(locationUri = Some(location.toUri.toString)), + params) + } + spark.sessionState.catalog.createPartitions(tableName, parts, ignoreIfExists = true) + done += parts.length + logDebug(s"Recovered ${parts.length} partitions ($done/$total so far)") + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0666a99cfc43e..ba465f22f3533 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -319,6 +319,14 @@ object SQLConf { .booleanConf .createWithDefault(false) + val GATHER_FASTSTAT = SQLConfigBuilder("spark.sql.hive.gatherFastStats") + .internal() + .doc("When true, fast stats (number of files and total size of all files) will be gathered" + + " in parallel while repairing table partitions to avoid the sequential listing in Hive" + + " metastore.") + .booleanConf + .createWithDefault(true) + // This is used to control the when we will split a schema's JSON string to multiple pieces // in order to fit the JSON string in metastore's table property (by default, the value has // a length restriction of 4000 characters). We will split the JSON string of a schema @@ -616,6 +624,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def nativeView: Boolean = getConf(NATIVE_VIEW) + def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT) + def wholeStageEnabled: Boolean = getConf(WHOLESTAGE_CODEGEN_ENABLED) def wholeStageMaxNumFields: Int = getConf(WHOLESTAGE_MAX_NUM_FIELDS) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 95654712fee5d..3b2a660d49a71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -643,13 +643,13 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } test("alter table: recover partitions (sequential)") { - withSQLConf("spark.rdd.parallelListingThreshold" -> "1") { + withSQLConf("spark.rdd.parallelListingThreshold" -> "10") { testRecoverPartitions() } } test("alter table: recover partition (parallel)") { - withSQLConf("spark.rdd.parallelListingThreshold" -> "10") { + withSQLConf("spark.rdd.parallelListingThreshold" -> "1") { testRecoverPartitions() } } @@ -672,7 +672,14 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) // valid fs.mkdirs(new Path(new Path(root, "a=1"), "b=5")) + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "a.csv")) // file + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "_SUCCESS")) // file fs.mkdirs(new Path(new Path(root, "A=2"), "B=6")) + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "b.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "c.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), ".hiddenFile")) // file + fs.mkdirs(new Path(new Path(root, "A=2/B=6"), "_temporary")) + // invalid fs.mkdirs(new Path(new Path(root, "a"), "b")) // bad name fs.mkdirs(new Path(new Path(root, "b=1"), "a=1")) // wrong order @@ -686,6 +693,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("ALTER TABLE tab1 RECOVER PARTITIONS") assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part2)) + assert(catalog.getPartition(tableIdent, part1).parameters("numFiles") == "1") + assert(catalog.getPartition(tableIdent, part2).parameters("numFiles") == "2") } finally { fs.delete(root, true) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index ccbd440bc4bbe..7b4483aa9e9a9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -835,6 +835,8 @@ private[hive] class HiveClientImpl( serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), compressed = apiPartition.getSd.isCompressed, serdeProperties = Option(apiPartition.getSd.getSerdeInfo.getParameters) - .map(_.asScala.toMap).orNull)) + .map(_.asScala.toMap).orNull), + parameters = + if (hp.getParameters() != null) hp.getParameters().asScala.toMap else Map.empty) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 9df4a26d55a27..13a8741cdc574 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -251,6 +251,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { val table = hive.getTable(database, tableName) parts.foreach { s => val location = s.storage.locationUri.map(new Path(table.getPath, _)).orNull + val params = if (s.parameters.nonEmpty) s.parameters.asJava else null val spec = s.spec.asJava if (hive.getPartition(table, spec, false) != null && ignoreIfExists) { // Ignore this partition since it already exists and ignoreIfExists == true @@ -264,7 +265,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { table, spec, location, - null, // partParams + params, // partParams null, // inputFormat null, // outputFormat -1: JInteger, // numBuckets @@ -417,8 +418,11 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { val addPartitionDesc = new AddPartitionDesc(db, table, ignoreIfExists) - parts.foreach { s => + parts.zipWithIndex.foreach { case (s, i) => addPartitionDesc.addPartition(s.spec.asJava, s.storage.locationUri.orNull) + if (s.parameters.nonEmpty) { + addPartitionDesc.getPartition(i).setPartParams(s.parameters.asJava) + } } hive.createPartitions(addPartitionDesc) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index df6cd568ac4f7..f31215ac31c8e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -368,6 +368,44 @@ class HiveDDLSuite expectedSerdeProps) } + test("MSCK REPAIR RABLE") { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1") + sql("CREATE TABLE tab1 (height INT, length INT) PARTITIONED BY (a INT, b INT)") + val part1 = Map("a" -> "1", "b" -> "5") + val part2 = Map("a" -> "2", "b" -> "6") + val root = new Path(catalog.getTableMetadata(tableIdent).storage.locationUri.get) + val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) + // valid + fs.mkdirs(new Path(new Path(root, "a=1"), "b=5")) + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "a.csv")) // file + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "_SUCCESS")) // file + fs.mkdirs(new Path(new Path(root, "A=2"), "B=6")) + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "b.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "c.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), ".hiddenFile")) // file + fs.mkdirs(new Path(new Path(root, "A=2/B=6"), "_temporary")) + + // invalid + fs.mkdirs(new Path(new Path(root, "a"), "b")) // bad name + fs.mkdirs(new Path(new Path(root, "b=1"), "a=1")) // wrong order + fs.mkdirs(new Path(root, "a=4")) // not enough columns + fs.createNewFile(new Path(new Path(root, "a=1"), "b=4")) // file + fs.createNewFile(new Path(new Path(root, "a=1"), "_SUCCESS")) // _SUCCESS + fs.mkdirs(new Path(new Path(root, "a=1"), "_temporary")) // _temporary + fs.mkdirs(new Path(new Path(root, "a=1"), ".b=4")) // start with . + + try { + sql("MSCK REPAIR TABLE tab1") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2)) + assert(catalog.getPartition(tableIdent, part1).parameters("numFiles") == "1") + assert(catalog.getPartition(tableIdent, part2).parameters("numFiles") == "2") + } finally { + fs.delete(root, true) + } + } + test("drop table using drop view") { withTable("tab1") { sql("CREATE TABLE tab1(c1 int)") From 976a43dbf9d97b30d81576799470532b81b882f0 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 29 Aug 2016 12:55:32 -0700 Subject: [PATCH 1221/1470] [SPARK-16581][SPARKR] Make JVM backend calling functions public ## What changes were proposed in this pull request? This change exposes a public API in SparkR to create objects, call methods on the Spark driver JVM ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Unit tests, CRAN checks Author: Shivaram Venkataraman Closes #14775 from shivaram/sparkr-java-api. (cherry picked from commit 736a7911cb0335cdb2b2f6c87f9e3c32047b5bbb) Signed-off-by: Shivaram Venkataraman --- R/pkg/DESCRIPTION | 5 +- R/pkg/NAMESPACE | 4 + R/pkg/R/jvm.R | 117 +++++++++++++++++++++++ R/pkg/inst/tests/testthat/test_jvm_api.R | 43 +++++++++ 4 files changed, 167 insertions(+), 2 deletions(-) create mode 100644 R/pkg/R/jvm.R create mode 100644 R/pkg/inst/tests/testthat/test_jvm_api.R diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index e5afed2d0a93e..5a83883089e0e 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -2,7 +2,7 @@ Package: SparkR Type: Package Title: R Frontend for Apache Spark Version: 2.0.0 -Date: 2016-07-07 +Date: 2016-08-27 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), person("Xiangrui", "Meng", role = "aut", @@ -11,7 +11,7 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "felixcheung@apache.org"), person(family = "The Apache Software Foundation", role = c("aut", "cph"))) URL: http://www.apache.org/ http://spark.apache.org/ -BugReports: https://issues.apache.org/jira/secure/CreateIssueDetails!init.jspa?pid=12315420&components=12325400&issuetype=4 +BugReports: https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-ContributingBugReports Depends: R (>= 3.0), methods @@ -39,6 +39,7 @@ Collate: 'deserialize.R' 'functions.R' 'install.R' + 'jvm.R' 'mllib.R' 'serialize.R' 'sparkR.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index cdb8834161f40..666e76a4eb145 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -357,4 +357,8 @@ S3method(structField, jobj) S3method(structType, jobj) S3method(structType, structField) +export("sparkR.newJObject") +export("sparkR.callJMethod") +export("sparkR.callJStatic") + export("install.spark") diff --git a/R/pkg/R/jvm.R b/R/pkg/R/jvm.R new file mode 100644 index 0000000000000..bb5c77544a3da --- /dev/null +++ b/R/pkg/R/jvm.R @@ -0,0 +1,117 @@ +# +# 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 directly access the JVM running the SparkR backend. + +#' Call Java Methods +#' +#' Call a Java method in the JVM running the Spark driver. The return +#' values are automatically converted to R objects for simple objects. Other +#' values are returned as "jobj" which are references to objects on JVM. +#' +#' @details +#' This is a low level function to access the JVM directly and should only be used +#' for advanced use cases. The arguments and return values that are primitive R +#' types (like integer, numeric, character, lists) are automatically translated to/from +#' Java types (like Integer, Double, String, Array). A full list can be found in +#' serialize.R and deserialize.R in the Apache Spark code base. +#' +#' @param x object to invoke the method on. Should be a "jobj" created by newJObject. +#' @param methodName method name to call. +#' @param ... parameters to pass to the Java method. +#' @return the return value of the Java method. Either returned as a R object +#' if it can be deserialized or returned as a "jobj". See details section for more. +#' @export +#' @seealso \link{sparkR.callJStatic}, \link{sparkR.newJObject} +#' @rdname sparkR.callJMethod +#' @examples +#' \dontrun{ +#' sparkR.session() # Need to have a Spark JVM running before calling newJObject +#' # Create a Java ArrayList and populate it +#' jarray <- sparkR.newJObject("java.util.ArrayList") +#' sparkR.callJMethod(jarray, "add", 42L) +#' sparkR.callJMethod(jarray, "get", 0L) # Will print 42 +#' } +#' @note sparkR.callJMethod since 2.0.1 +sparkR.callJMethod <- function(x, methodName, ...) { + callJMethod(x, methodName, ...) +} + +#' Call Static Java Methods +#' +#' Call a static method in the JVM running the Spark driver. The return +#' value is automatically converted to R objects for simple objects. Other +#' values are returned as "jobj" which are references to objects on JVM. +#' +#' @details +#' This is a low level function to access the JVM directly and should only be used +#' for advanced use cases. The arguments and return values that are primitive R +#' types (like integer, numeric, character, lists) are automatically translated to/from +#' Java types (like Integer, Double, String, Array). A full list can be found in +#' serialize.R and deserialize.R in the Apache Spark code base. +#' +#' @param x fully qualified Java class name that contains the static method to invoke. +#' @param methodName name of static method to invoke. +#' @param ... parameters to pass to the Java method. +#' @return the return value of the Java method. Either returned as a R object +#' if it can be deserialized or returned as a "jobj". See details section for more. +#' @export +#' @seealso \link{sparkR.callJMethod}, \link{sparkR.newJObject} +#' @rdname sparkR.callJStatic +#' @examples +#' \dontrun{ +#' sparkR.session() # Need to have a Spark JVM running before calling callJStatic +#' sparkR.callJStatic("java.lang.System", "currentTimeMillis") +#' sparkR.callJStatic("java.lang.System", "getProperty", "java.home") +#' } +#' @note sparkR.callJStatic since 2.0.1 +sparkR.callJStatic <- function(x, methodName, ...) { + callJStatic(x, methodName, ...) +} + +#' Create Java Objects +#' +#' Create a new Java object in the JVM running the Spark driver. The return +#' value is automatically converted to an R object for simple objects. Other +#' values are returned as a "jobj" which is a reference to an object on JVM. +#' +#' @details +#' This is a low level function to access the JVM directly and should only be used +#' for advanced use cases. The arguments and return values that are primitive R +#' types (like integer, numeric, character, lists) are automatically translated to/from +#' Java types (like Integer, Double, String, Array). A full list can be found in +#' serialize.R and deserialize.R in the Apache Spark code base. +#' +#' @param x fully qualified Java class name. +#' @param ... arguments to be passed to the constructor. +#' @return the object created. Either returned as a R object +#' if it can be deserialized or returned as a "jobj". See details section for more. +#' @export +#' @seealso \link{sparkR.callJMethod}, \link{sparkR.callJStatic} +#' @rdname sparkR.newJObject +#' @examples +#' \dontrun{ +#' sparkR.session() # Need to have a Spark JVM running before calling newJObject +#' # Create a Java ArrayList and populate it +#' jarray <- sparkR.newJObject("java.util.ArrayList") +#' sparkR.callJMethod(jarray, "add", 42L) +#' sparkR.callJMethod(jarray, "get", 0L) # Will print 42 +#' } +#' @note sparkR.newJObject since 2.0.1 +sparkR.newJObject <- function(x, ...) { + newJObject(x, ...) +} diff --git a/R/pkg/inst/tests/testthat/test_jvm_api.R b/R/pkg/inst/tests/testthat/test_jvm_api.R new file mode 100644 index 0000000000000..151c52906bdb8 --- /dev/null +++ b/R/pkg/inst/tests/testthat/test_jvm_api.R @@ -0,0 +1,43 @@ +# +# 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("JVM API") + +sparkSession <- sparkR.session(enableHiveSupport = FALSE) + +test_that("Create and call methods on object", { + jarr <- newJObject("java.util.ArrayList") + # Add an element to the array + callJMethod(jarr, "add", 1L) + # Check if get returns the same element + expect_equal(callJMethod(jarr, "get", 0L), 1L) +}) + +test_that("Call static methods", { + # Convert a boolean to a string + strTrue <- callJStatic("java.lang.String", "valueOf", TRUE) + expect_equal(strTrue, "true") +}) + +test_that("Manually garbage collect objects", { + jarr <- newJObject("java.util.ArrayList") + cleanup.jobj(jarr) + # Using a jobj after GC should throw an error + expect_error(print(jarr), "Error in invokeJava.*") +}) + +sparkR.session.stop() From 59032570fbd0985f758c27bdec5482221cc64af9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 30 Aug 2016 09:58:00 +0800 Subject: [PATCH 1222/1470] [SPARK-17301][SQL] Remove unused classTag field from AtomicType base class There's an unused `classTag` val in the AtomicType base class which is causing unnecessary slowness in deserialization because it needs to grab ScalaReflectionLock and create a new runtime reflection mirror. Removing this unused code gives a small but measurable performance boost in SQL task deserialization. Author: Josh Rosen Closes #14869 from JoshRosen/remove-unused-classtag. (cherry picked from commit 48b459ddd58affd5519856cb6e204398b7739a2a) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/types/AbstractDataType.scala | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 65eae869d40d1..1981fd8f0a1b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -17,13 +17,10 @@ package org.apache.spark.sql.types -import scala.reflect.ClassTag -import scala.reflect.runtime.universe.{runtimeMirror, TypeTag} +import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.util.Utils /** * A non-concrete data type, reserved for internal uses. @@ -130,11 +127,6 @@ protected[sql] abstract class AtomicType extends DataType { private[sql] type InternalType private[sql] val tag: TypeTag[InternalType] private[sql] val ordering: Ordering[InternalType] - - @transient private[sql] val classTag = ScalaReflectionLock.synchronized { - val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[InternalType](mirror.runtimeClass(tag.tpe)) - } } From f35b10ab1556e3ea76ce2322af7b6749b7f1357f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 30 Aug 2016 11:19:45 +0100 Subject: [PATCH 1223/1470] [SPARK-17264][SQL] DataStreamWriter should document that it only supports Parquet for now ## What changes were proposed in this pull request? Clarify that only parquet files are supported by DataStreamWriter now ## How was this patch tested? (Doc build -- no functional changes to test) Author: Sean Owen Closes #14860 from srowen/SPARK-17264. (cherry picked from commit befab9c1c6b59ad90f63a7d10e12b186be897f15) Signed-off-by: Sean Owen --- python/pyspark/sql/streaming.py | 2 +- .../scala/org/apache/spark/sql/streaming/DataStreamWriter.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 3761d2b1994f0..9487f9dbfdabc 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -589,7 +589,7 @@ def format(self, source): .. note:: Experimental. - :param source: string, name of the data source, e.g. 'json', 'parquet'. + :param source: string, name of the data source, which for now can be 'parquet'. >>> writer = sdf.writeStream.format('json') """ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index d38e3e58125d9..f70c7d08a691c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -122,7 +122,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * :: Experimental :: - * Specifies the underlying output data source. Built-in options include "parquet", "json", etc. + * Specifies the underlying output data source. Built-in options include "parquet" for now. * * @since 2.0.0 */ From bc6c0d9f96da6a9aaf8279ee4ad11a82bcd69cb5 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 30 Aug 2016 20:04:52 -0700 Subject: [PATCH 1224/1470] [SPARK-17318][TESTS] Fix ReplSuite replicating blocks of object with class defined in repl ## What changes were proposed in this pull request? There are a lot of failures recently: http://spark-tests.appspot.com/tests/org.apache.spark.repl.ReplSuite/replicating%20blocks%20of%20object%20with%20class%20defined%20in%20repl This PR just changed the persist level to `MEMORY_AND_DISK_2` to avoid blocks being evicted from memory. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #14884 from zsxwing/SPARK-17318. (cherry picked from commit 231f973295129dca976f2e4a8222a63318d4aafe) Signed-off-by: Shixiong Zhu --- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 06b09f3158d77..f1284b1df31b4 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -401,7 +401,7 @@ class ReplSuite extends SparkFunSuite { """ |import org.apache.spark.storage.StorageLevel._ |case class Foo(i: Int) - |val ret = sc.parallelize((1 to 100).map(Foo), 10).persist(MEMORY_ONLY_2) + |val ret = sc.parallelize((1 to 100).map(Foo), 10).persist(MEMORY_AND_DISK_2) |ret.count() |sc.getExecutorStorageStatus.map(s => s.rddBlocksById(ret.id).size).sum """.stripMargin) From 021aa28f439443cda1bc7c5e3eee7c85b40c1a2d Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Wed, 31 Aug 2016 08:50:42 -0500 Subject: [PATCH 1225/1470] [SPARK-17243][WEB UI] Spark 2.0 History Server won't load with very large application history ## What changes were proposed in this pull request? back port of #14835 addressing merge conflicts With the new History Server the summary page loads the application list via the the REST API, this makes it very slow to impossible to load with large (10K+) application history. This pr fixes this by adding the `spark.history.ui.maxApplications` conf to limit the number of applications the History Server displays. This is accomplished using a new optional `limit` param for the `applications` api. (Note this only applies to what the summary page displays, all the Application UI's are still accessible if the user knows the App ID and goes to the Application UI directly.) I've also added a new test for the `limit` param in `HistoryServerSuite.scala` ## How was this patch tested? Manual testing and dev/run-tests Author: Alex Bozarth Closes #14886 from ajbozarth/spark17243-branch-2.0. --- .../org/apache/spark/ui/static/historypage.js | 8 ++- .../spark/deploy/history/HistoryPage.scala | 3 +- .../spark/deploy/history/HistoryServer.scala | 4 ++ .../spark/internal/config/package.scala | 4 ++ .../api/v1/ApplicationListResource.scala | 10 ++- .../limit_app_list_json_expectation.json | 67 +++++++++++++++++++ .../deploy/history/HistoryServerSuite.scala | 1 + docs/monitoring.md | 16 ++++- 8 files changed, 106 insertions(+), 7 deletions(-) create mode 100644 core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index d2161662d5679..177120aaa6c1f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -15,6 +15,12 @@ * limitations under the License. */ +var appLimit = -1; + +function setAppLimit(val) { + appLimit = val; +} + // this function works exactly the same as UIUtils.formatDuration function formatDuration(milliseconds) { if (milliseconds < 100) { @@ -111,7 +117,7 @@ $(document).ready(function() { requestedIncomplete = getParameterByName("showIncomplete", searchString); requestedIncomplete = (requestedIncomplete == "true" ? true : false); - $.getJSON("api/v1/applications", function(response,status,jqXHR) { + $.getJSON("api/v1/applications?limit=" + appLimit, function(response,status,jqXHR) { var array = []; var hasMultipleAttempts = false; for (i in response) { 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 2fad1120cdc8a..a120b6c5fcdff 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 @@ -44,7 +44,8 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") if (allAppsSize > 0) { ++ ++ - + ++ + } else if (requestedIncomplete) {

    No incomplete applications found!

    } else { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index d821474bdb590..c178917d8da3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -28,6 +28,7 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -55,6 +56,9 @@ class HistoryServer( // How many applications to retain private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) + // How many applications the summary ui displays + private[history] val maxApplications = conf.get(HISTORY_UI_MAX_APPS); + // application private val appCache = new ApplicationCache(this, retainedApplications, new SystemClock()) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5a59aee652712..f28a9a5cf81d6 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -108,4 +108,8 @@ package object config { private[spark] val UI_RETAINED_TASKS = ConfigBuilder("spark.ui.retainedTasks") .intConf .createWithDefault(100000) + + // To limit how many applications are shown in the History Server summary ui + private[spark] val HISTORY_UI_MAX_APPS = + ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 02fd2985fa20d..075b9ba37dc84 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -29,7 +29,8 @@ private[v1] class ApplicationListResource(uiRoot: UIRoot) { def appList( @QueryParam("status") status: JList[ApplicationStatus], @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam, - @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam) + @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam, + @QueryParam("limit") limit: Integer) : Iterator[ApplicationInfo] = { val allApps = uiRoot.getApplicationInfoList val adjStatus = { @@ -41,7 +42,7 @@ private[v1] class ApplicationListResource(uiRoot: UIRoot) { } val includeCompleted = adjStatus.contains(ApplicationStatus.COMPLETED) val includeRunning = adjStatus.contains(ApplicationStatus.RUNNING) - allApps.filter { app => + val appList = allApps.filter { app => val anyRunning = app.attempts.exists(!_.completed) // if any attempt is still running, we consider the app to also still be running val statusOk = (!anyRunning && includeCompleted) || @@ -53,6 +54,11 @@ private[v1] class ApplicationListResource(uiRoot: UIRoot) { } statusOk && dateOk } + if (limit != null) { + appList.take(limit) + } else { + appList + } } } diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json new file mode 100644 index 0000000000000..9165f549d7d25 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -0,0 +1,67 @@ +[ { + "id" : "local-1430917381534", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", + "lastUpdated" : "", + "duration" : 10505, + "sparkUser" : "irashid", + "completed" : true, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917391398, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950, + "lastUpdatedEpoch" : 0 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177, + "lastUpdatedEpoch" : 0 + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177, + "lastUpdatedEpoch" : 0 + } ] +} ] 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 631a7cd9d5d7a..ae3f5d9c012ea 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 @@ -100,6 +100,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "minDate app list json" -> "applications?minDate=2015-02-10", "maxDate app list json" -> "applications?maxDate=2015-02-10", "maxDate2 app list json" -> "applications?maxDate=2015-02-03T16:42:40.000GMT", + "limit app list json" -> "applications?limit=3", "one app json" -> "applications/local-1422981780767", "one app multi-attempt json" -> "applications/local-1426533911241", "job list json" -> "applications/local-1422981780767/jobs", diff --git a/docs/monitoring.md b/docs/monitoring.md index ee932cfc6d705..1bc3d266b66b4 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -114,8 +114,17 @@ The history server can be configured as follows: spark.history.retainedApplications 50 - The number of application UIs to retain. If this cap is exceeded, then the oldest - applications will be removed. + The number of applications to retain UI data for in the cache. If this cap is exceeded, then + the oldest applications will be removed from the cache. If an application is not in the cache, + it will have to be loaded from disk if its accessed from the UI. + + + + spark.history.ui.maxApplications + Int.MaxValue + + The number of applications to display on the history summary page. Application UIs are still + available by accessing their URLs directly even if they are not displayed on the history summary page. @@ -242,7 +251,8 @@ can be identified by their `[attempt-id]`. In the API listed below, when running
    Examples:
    ?minDate=2015-02-10
    ?minDate=2015-02-03T16:42:40.000GMT -
    ?maxDate=[date] latest date/time to list; uses same format as minDate. +
    ?maxDate=[date] latest date/time to list; uses same format as minDate. +
    ?limit=[limit] limits the number of applications listed. /applications/[app-id]/jobs From c17334e47e806e59ef65a8eefab632781bf9422c Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 31 Aug 2016 10:56:02 -0700 Subject: [PATCH 1226/1470] [SPARK-17316][CORE] Make CoarseGrainedSchedulerBackend.removeExecutor non-blocking ## What changes were proposed in this pull request? StandaloneSchedulerBackend.executorRemoved is a blocking call right now. It may cause some deadlock since it's called inside StandaloneAppClient.ClientEndpoint. This PR just changed CoarseGrainedSchedulerBackend.removeExecutor to be non-blocking. It's safe since the only two usages (StandaloneSchedulerBackend and YarnSchedulerEndpoint) don't need the return value). ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #14882 from zsxwing/SPARK-17316. (cherry picked from commit 9bcb33c54117cebc9e087017bf4e4163edaeff17) Signed-off-by: Marcelo Vanzin --- .../cluster/CoarseGrainedSchedulerBackend.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) 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 8259923ce31c3..2db3a3bb81f61 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 @@ -406,14 +406,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp conf.getInt("spark.default.parallelism", math.max(totalCoreCount.get(), 2)) } - // Called by subclasses when notified of a lost worker - def removeExecutor(executorId: String, reason: ExecutorLossReason) { - try { - driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, reason)) - } catch { - case e: Exception => - throw new SparkException("Error notifying standalone scheduler's driver endpoint", e) - } + /** + * Called by subclasses when notified of a lost worker. It just fires the message and returns + * at once. + */ + protected def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { + // Only log the failure since we don't care about the result. + driverEndpoint.ask(RemoveExecutor(executorId, reason)).onFailure { case t => + logError(t.getMessage, t) + }(ThreadUtils.sameThread) } def sufficientResourcesRegistered(): Boolean = true From ad368926101efadf7b9f95ec1c95989f0c0a2855 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 31 Aug 2016 14:02:21 -0700 Subject: [PATCH 1227/1470] [SPARK-17326][SPARKR] Fix tests with HiveContext in SparkR not to be skipped always ## What changes were proposed in this pull request? Currently, `HiveContext` in SparkR is not being tested and always skipped. This is because the initiation of `TestHiveContext` is being failed due to trying to load non-existing data paths (test tables). This is introduced from https://github.com/apache/spark/pull/14005 This enables the tests with SparkR. ## How was this patch tested? Manually, **Before** (on Mac OS) ``` ... Skipped ------------------------------------------------------------------------ 1. create DataFrame from RDD (test_sparkSQL.R#200) - Hive is not build with SparkSQL, skipped 2. test HiveContext (test_sparkSQL.R#1041) - Hive is not build with SparkSQL, skipped 3. read/write ORC files (test_sparkSQL.R#1748) - Hive is not build with SparkSQL, skipped 4. enableHiveSupport on SparkSession (test_sparkSQL.R#2480) - Hive is not build with SparkSQL, skipped 5. sparkJars tag in SparkContext (test_Windows.R#21) - This test is only for Windows, skipped ... ``` **After** (on Mac OS) ``` ... Skipped ------------------------------------------------------------------------ 1. sparkJars tag in SparkContext (test_Windows.R#21) - This test is only for Windows, skipped ... ``` Please refer the tests below (on Windows) - Before: https://ci.appveyor.com/project/HyukjinKwon/spark/build/45-test123 - After: https://ci.appveyor.com/project/HyukjinKwon/spark/build/46-test123 Author: hyukjinkwon Closes #14889 from HyukjinKwon/SPARK-17326. (cherry picked from commit 50bb142332d1147861def692bf63f0055ecb8576) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 0aea89ddcb076..279d512de8b48 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -39,7 +39,7 @@ setHiveContext <- function(sc) { # initialize once and reuse ssc <- callJMethod(sc, "sc") hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc, FALSE) }, error = function(err) { skip("Hive is not build with SparkSQL, skipped") From d01251c928ce76e22d081a3764134f44ffe9aa86 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 31 Aug 2016 15:25:13 -0700 Subject: [PATCH 1228/1470] [SPARK-17316][TESTS] Fix MesosCoarseGrainedSchedulerBackendSuite ## What changes were proposed in this pull request? The master is broken because #14882 didn't run mesos tests. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu Closes #14902 from zsxwing/hotfix. (cherry picked from commit d375c8a3de1d253c485078f55eb9c5b928ab96d5) Signed-off-by: Shixiong Zhu --- .../cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index f6ec167a187de..12c4a7993e79e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -21,6 +21,7 @@ import java.util.Collections import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Promise import scala.reflect.ClassTag import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} @@ -410,6 +411,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite when(taskScheduler.sc).thenReturn(sc) externalShuffleClient = mock[MesosExternalShuffleClient] driverEndpoint = mock[RpcEndpointRef] + when(driverEndpoint.ask(any())(any())).thenReturn(Promise().future) backend = createSchedulerBackend(taskScheduler, driver, externalShuffleClient, driverEndpoint) } From 8d15c1a6a0ac2e57b537c370a8e8283d56ca290e Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 31 Aug 2016 16:56:41 -0700 Subject: [PATCH 1229/1470] [SPARK-16581][SPARKR] Fix JVM API tests in SparkR ## What changes were proposed in this pull request? Remove cleanup.jobj test. Use JVM wrapper API for other test cases. ## How was this patch tested? Run R unit tests with testthat 1.0 Author: Shivaram Venkataraman Closes #14904 from shivaram/sparkr-jvm-tests-fix. (cherry picked from commit 2f9c27364ea00473933213700edb93b63b55b313) Signed-off-by: Shivaram Venkataraman --- R/pkg/inst/tests/testthat/test_jvm_api.R | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_jvm_api.R b/R/pkg/inst/tests/testthat/test_jvm_api.R index 151c52906bdb8..7348c893d0af3 100644 --- a/R/pkg/inst/tests/testthat/test_jvm_api.R +++ b/R/pkg/inst/tests/testthat/test_jvm_api.R @@ -20,24 +20,17 @@ context("JVM API") sparkSession <- sparkR.session(enableHiveSupport = FALSE) test_that("Create and call methods on object", { - jarr <- newJObject("java.util.ArrayList") + jarr <- sparkR.newJObject("java.util.ArrayList") # Add an element to the array - callJMethod(jarr, "add", 1L) + sparkR.callJMethod(jarr, "add", 1L) # Check if get returns the same element - expect_equal(callJMethod(jarr, "get", 0L), 1L) + expect_equal(sparkR.callJMethod(jarr, "get", 0L), 1L) }) test_that("Call static methods", { # Convert a boolean to a string - strTrue <- callJStatic("java.lang.String", "valueOf", TRUE) + strTrue <- sparkR.callJStatic("java.lang.String", "valueOf", TRUE) expect_equal(strTrue, "true") }) -test_that("Manually garbage collect objects", { - jarr <- newJObject("java.util.ArrayList") - cleanup.jobj(jarr) - # Using a jobj after GC should throw an error - expect_error(print(jarr), "Error in invokeJava.*") -}) - sparkR.session.stop() From 191d99692dc4315c371b566e3a9c5b114876ee49 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 1 Sep 2016 08:54:59 +0800 Subject: [PATCH 1230/1470] [SPARK-17180][SPARK-17309][SPARK-17323][SQL][2.0] create AlterViewAsCommand to handle ALTER VIEW AS ## What changes were proposed in this pull request? Currently we use `CreateViewCommand` to implement ALTER VIEW AS, which has 3 bugs: 1. SPARK-17180: ALTER VIEW AS should alter temp view if view name has no database part and temp view exists 2. SPARK-17309: ALTER VIEW AS should issue exception if view does not exist. 3. SPARK-17323: ALTER VIEW AS should keep the previous table properties, comment, create_time, etc. The root cause is, ALTER VIEW AS is quite different from CREATE VIEW, we need different code path to handle them. However, in `CreateViewCommand`, there is no way to distinguish ALTER VIEW AS and CREATE VIEW, we have to introduce extra flag. But instead of doing this, I think a more natural way is to separate the ALTER VIEW AS logic into a new command. backport https://github.com/apache/spark/pull/14874 to 2.0 ## How was this patch tested? new tests in SQLViewSuite Author: Wenchen Fan Closes #14893 from cloud-fan/minor4. --- .../spark/sql/execution/SparkSqlParser.scala | 74 ++++++++----------- .../spark/sql/execution/command/views.scala | 71 +++++++++++++++++- .../sql/hive/execution/SQLViewSuite.scala | 71 ++++++++++++++++++ 3 files changed, 167 insertions(+), 49 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 876b3341d217e..3072a6d79eac7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1250,60 +1250,44 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { val schema = identifiers.map { ic => CatalogColumn(ic.identifier.getText, null, nullable = true, Option(ic.STRING).map(string)) } - createView( - ctx, - ctx.tableIdentifier, - comment = Option(ctx.STRING).map(string), - schema, - ctx.query, - Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty), - ctx.EXISTS != null, - ctx.REPLACE != null, - ctx.TEMPORARY != null - ) + + val sql = Option(source(ctx.query)) + val tableDesc = CatalogTable( + identifier = visitTableIdentifier(ctx.tableIdentifier), + tableType = CatalogTableType.VIEW, + schema = schema, + storage = CatalogStorageFormat.empty, + properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty), + viewOriginalText = sql, + viewText = sql, + comment = Option(ctx.STRING).map(string)) + + CreateViewCommand( + tableDesc, + plan(ctx.query), + allowExisting = ctx.EXISTS != null, + replace = ctx.REPLACE != null, + isTemporary = ctx.TEMPORARY != null) } } /** - * Alter the query of a view. This creates a [[CreateViewCommand]] command. + * Alter the query of a view. This creates a [[AlterViewAsCommand]] command. + * + * For example: + * {{{ + * ALTER VIEW [db_name.]view_name AS SELECT ...; + * }}} */ override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) { - createView( - ctx, - ctx.tableIdentifier, - comment = None, - Seq.empty, - ctx.query, - Map.empty, - allowExist = false, - replace = true, - isTemporary = false) - } - - /** - * Create a [[CreateViewCommand]] command. - */ - private def createView( - ctx: ParserRuleContext, - name: TableIdentifierContext, - comment: Option[String], - schema: Seq[CatalogColumn], - query: QueryContext, - properties: Map[String, String], - allowExist: Boolean, - replace: Boolean, - isTemporary: Boolean): LogicalPlan = { - val sql = Option(source(query)) val tableDesc = CatalogTable( - identifier = visitTableIdentifier(name), + identifier = visitTableIdentifier(ctx.tableIdentifier), tableType = CatalogTableType.VIEW, - schema = schema, storage = CatalogStorageFormat.empty, - properties = properties, - viewOriginalText = sql, - viewText = sql, - comment = comment) - CreateViewCommand(tableDesc, plan(query), allowExist, replace, isTemporary) + schema = Nil, + viewOriginalText = Option(source(ctx.query))) + + AlterViewAsCommand(tableDesc, plan(ctx.query)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 9e263d1cd5510..125b3d1b05878 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -22,7 +22,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{SQLBuilder, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} +import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} @@ -56,8 +56,6 @@ case class CreateViewCommand( // TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is // different from Hive and may not work for some cases like create view on self join. - override def output: Seq[Attribute] = Seq.empty[Attribute] - require(tableDesc.tableType == CatalogTableType.VIEW, "The type of the table to created with CREATE VIEW must be 'CatalogTableType.VIEW'.") if (!isTemporary) { @@ -191,7 +189,7 @@ case class CreateViewCommand( sparkSession.sql(viewSQL).queryExecution.assertAnalyzed() } catch { case NonFatal(e) => - throw new RuntimeException(s"Failed to analyze the canonicalized SQL: ${viewSQL}", e) + throw new RuntimeException(s"Failed to analyze the canonicalized SQL: $viewSQL", e) } val viewSchema: Seq[CatalogColumn] = { @@ -212,3 +210,68 @@ case class CreateViewCommand( /** Escape backtick with double-backtick in column name and wrap it with backtick. */ private def quote(name: String) = s"`${name.replaceAll("`", "``")}`" } + +/** + * Alter a view with given query plan. If the view name contains database prefix, this command will + * alter a permanent view matching the given name, or throw an exception if view not exist. Else, + * this command will try to alter a temporary view first, if view not exist, try permanent view + * next, if still not exist, throw an exception. + * + * @param tableDesc the catalog table + * @param query the logical plan that represents the view; this is used to generate a canonicalized + * version of the SQL that can be saved in the catalog. + */ +case class AlterViewAsCommand( + tableDesc: CatalogTable, + query: LogicalPlan) extends RunnableCommand { + + override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) + + override def run(session: SparkSession): Seq[Row] = { + // If the plan cannot be analyzed, throw an exception and don't proceed. + val qe = session.sessionState.executePlan(query) + qe.assertAnalyzed() + val analyzedPlan = qe.analyzed + + if (session.sessionState.catalog.isTemporaryTable(tableDesc.identifier)) { + session.sessionState.catalog.createTempView( + tableDesc.identifier.table, + analyzedPlan, + overrideIfExists = true) + } else { + alterPermanentView(session, analyzedPlan) + } + + Seq.empty[Row] + } + + private def alterPermanentView(session: SparkSession, analyzedPlan: LogicalPlan): Unit = { + val viewMeta = session.sessionState.catalog.getTableMetadata(tableDesc.identifier) + if (viewMeta.tableType != CatalogTableType.VIEW) { + throw new AnalysisException(s"${viewMeta.identifier} is not a view.") + } + + val viewSQL: String = new SQLBuilder(analyzedPlan).toSQL + // Validate the view SQL - make sure we can parse it and analyze it. + // If we cannot analyze the generated query, there is probably a bug in SQL generation. + try { + session.sql(viewSQL).queryExecution.assertAnalyzed() + } catch { + case NonFatal(e) => + throw new RuntimeException(s"Failed to analyze the canonicalized SQL: $viewSQL", e) + } + + val viewSchema: Seq[CatalogColumn] = { + analyzedPlan.output.map { a => + CatalogColumn(a.name, a.dataType.catalogString) + } + } + + val updatedViewMeta = viewMeta.copy( + schema = viewSchema, + viewOriginalText = tableDesc.viewOriginalText, + viewText = Some(viewSQL)) + + session.sessionState.catalog.alterTable(updatedViewMeta) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index 39846f145c42c..c6eeb8a0e4de4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -174,6 +176,75 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("should not allow ALTER VIEW AS when the view does not exist") { + intercept[NoSuchTableException]( + sql("ALTER VIEW testView AS SELECT 1, 2") + ) + + intercept[NoSuchTableException]( + sql("ALTER VIEW default.testView AS SELECT 1, 2") + ) + } + + test("ALTER VIEW AS should try to alter temp view first if view name has no database part") { + withView("test_view") { + withTempView("test_view") { + sql("CREATE VIEW test_view AS SELECT 1 AS a, 2 AS b") + sql("CREATE TEMP VIEW test_view AS SELECT 1 AS a, 2 AS b") + + sql("ALTER VIEW test_view AS SELECT 3 AS i, 4 AS j") + + // The temporary view should be updated. + checkAnswer(spark.table("test_view"), Row(3, 4)) + + // The permanent view should stay same. + checkAnswer(spark.table("default.test_view"), Row(1, 2)) + } + } + } + + test("ALTER VIEW AS should alter permanent view if view name has database part") { + withView("test_view") { + withTempView("test_view") { + sql("CREATE VIEW test_view AS SELECT 1 AS a, 2 AS b") + sql("CREATE TEMP VIEW test_view AS SELECT 1 AS a, 2 AS b") + + sql("ALTER VIEW default.test_view AS SELECT 3 AS i, 4 AS j") + + // The temporary view should stay same. + checkAnswer(spark.table("test_view"), Row(1, 2)) + + // The permanent view should be updated. + checkAnswer(spark.table("default.test_view"), Row(3, 4)) + } + } + } + + test("ALTER VIEW AS should keep the previous table properties, comment, create_time, etc.") { + withView("test_view") { + sql( + """ + |CREATE VIEW test_view + |COMMENT 'test' + |TBLPROPERTIES ('key' = 'a') + |AS SELECT 1 AS a, 2 AS b + """.stripMargin) + + val catalog = spark.sessionState.catalog + val viewMeta = catalog.getTableMetadata(TableIdentifier("test_view")) + assert(viewMeta.properties("comment") == "test") + assert(viewMeta.properties("key") == "a") + + sql("ALTER VIEW test_view AS SELECT 3 AS i, 4 AS j") + val updatedViewMeta = catalog.getTableMetadata(TableIdentifier("test_view")) + assert(updatedViewMeta.properties("comment") == "test") + assert(updatedViewMeta.properties("key") == "a") + assert(updatedViewMeta.createTime == viewMeta.createTime) + // The view should be updated. + checkAnswer(spark.table("test_view"), Row(3, 4)) + } + } + Seq(true, false).foreach { enabled => val prefix = (if (enabled) "With" else "Without") + " canonical native view: " test(s"$prefix correctly handle CREATE OR REPLACE VIEW") { From 8711b451d727074173748418a47cec210f84f2f7 Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Wed, 31 Aug 2016 21:28:53 -0700 Subject: [PATCH 1231/1470] [SPARKR][MINOR] Fix windowPartitionBy example ## What changes were proposed in this pull request? The usage in the original example is incorrect. This PR fixes it. ## How was this patch tested? Manual test. Author: Junyang Qian Closes #14903 from junyangq/SPARKR-FixWindowPartitionByDoc. (cherry picked from commit d008638fbedc857c1adc1dff399d427b8bae848e) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/window.R | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R index 215d0e7b5cfbe..0799d841e5dc9 100644 --- a/R/pkg/R/window.R +++ b/R/pkg/R/window.R @@ -21,9 +21,9 @@ #' #' Creates a WindowSpec with the partitioning defined. #' -#' @param col A column name or Column by which rows are partitioned to +#' @param col A column name or Column by which rows are partitioned to #' windows. -#' @param ... Optional column names or Columns in addition to col, by +#' @param ... Optional column names or Columns in addition to col, by #' which rows are partitioned to windows. #' #' @rdname windowPartitionBy @@ -32,10 +32,10 @@ #' @export #' @examples #' \dontrun{ -#' ws <- windowPartitionBy("key1", "key2") +#' ws <- orderBy(windowPartitionBy("key1", "key2"), "key3") #' df1 <- select(df, over(lead("value", 1), ws)) #' -#' ws <- windowPartitionBy(df$key1, df$key2) +#' ws <- orderBy(windowPartitionBy(df$key1, df$key2), df$key3) #' df1 <- select(df, over(lead("value", 1), ws)) #' } #' @note windowPartitionBy(character) since 2.0.0 @@ -70,9 +70,9 @@ setMethod("windowPartitionBy", #' #' Creates a WindowSpec with the ordering defined. #' -#' @param col A column name or Column by which rows are ordered within +#' @param col A column name or Column by which rows are ordered within #' windows. -#' @param ... Optional column names or Columns in addition to col, by +#' @param ... Optional column names or Columns in addition to col, by #' which rows are ordered within windows. #' #' @rdname windowOrderBy From 6281b74b6965ffcd0600844cea168cbe71ca8248 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 31 Aug 2016 23:25:20 -0700 Subject: [PATCH 1232/1470] [SPARK-17318][TESTS] Fix ReplSuite replicating blocks of object with class defined in repl again ## What changes were proposed in this pull request? After digging into the logs, I noticed the failure is because in this test, it starts a local cluster with 2 executors. However, when SparkContext is created, executors may be still not up. When one of the executor is not up during running the job, the blocks won't be replicated. This PR just adds a wait loop before running the job to fix the flaky test. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #14905 from zsxwing/SPARK-17318-2. (cherry picked from commit 21c0a4fe9d8e21819ba96e7dc2b1f2999d3299ae) Signed-off-by: Shixiong Zhu --- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index f1284b1df31b4..f7d7a4f041315 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -399,6 +399,15 @@ class ReplSuite extends SparkFunSuite { test("replicating blocks of object with class defined in repl") { val output = runInterpreter("local-cluster[2,1,1024]", """ + |val timeout = 60000 // 60 seconds + |val start = System.currentTimeMillis + |while(sc.getExecutorStorageStatus.size != 3 && + | (System.currentTimeMillis - start) < timeout) { + | Thread.sleep(10) + |} + |if (System.currentTimeMillis - start >= timeout) { + | throw new java.util.concurrent.TimeoutException("Executors were not up in 60 seconds") + |} |import org.apache.spark.storage.StorageLevel._ |case class Foo(i: Int) |val ret = sc.parallelize((1 to 100).map(Foo), 10).persist(MEMORY_AND_DISK_2) From 13bacd7308c42c92f42fbc3ffbee9a13282668a9 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Thu, 1 Sep 2016 18:49:43 +0200 Subject: [PATCH 1233/1470] =?UTF-8?q?[SPARK-17271][SQL]=20Planner=20adds?= =?UTF-8?q?=20un-necessary=20Sort=20even=20if=20child=20orde=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Ports https://github.com/apache/spark/pull/14841 and https://github.com/apache/spark/pull/14910 from `master` to `branch-2.0` Jira : https://issues.apache.org/jira/browse/SPARK-17271 Planner is adding un-needed SORT operation due to bug in the way comparison for `SortOrder` is done at https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala#L253 `SortOrder` needs to be compared semantically because `Expression` within two `SortOrder` can be "semantically equal" but not literally equal objects. eg. In case of `sql("SELECT * FROM table1 a JOIN table2 b ON a.col1=b.col1")` Expression in required SortOrder: ``` AttributeReference( name = "col1", dataType = LongType, nullable = false ) (exprId = exprId, qualifier = Some("a") ) ``` Expression in child SortOrder: ``` AttributeReference( name = "col1", dataType = LongType, nullable = false ) (exprId = exprId) ``` Notice that the output column has a qualifier but the child attribute does not but the inherent expression is the same and hence in this case we can say that the child satisfies the required sort order. This PR includes following changes: - Added a `semanticEquals` method to `SortOrder` so that it can compare underlying child expressions semantically (and not using default Object.equals) - Fixed `EnsureRequirements` to use semantic comparison of SortOrder ## How was this patch tested? - Added a test case to `PlannerSuite`. Ran rest tests in `PlannerSuite` Author: Tejas Patil Closes #14920 from tejasapatil/SPARK-17271_2.0_port. --- .../exchange/EnsureRequirements.scala | 11 ++++- .../spark/sql/execution/PlannerSuite.scala | 40 ++++++++++++++++++- 2 files changed, 49 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 446571aa8409f..f17049949aa47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -236,7 +236,16 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { children = children.zip(requiredChildOrderings).map { case (child, requiredOrdering) => if (requiredOrdering.nonEmpty) { // If child.outputOrdering is [a, b] and requiredOrdering is [a], we do not need to sort. - if (requiredOrdering != child.outputOrdering.take(requiredOrdering.length)) { + val orderingMatched = if (requiredOrdering.length > child.outputOrdering.length) { + false + } else { + requiredOrdering.zip(child.outputOrdering).forall { + case (requiredOrder, childOutputOrder) => + requiredOrder.semanticEquals(childOutputOrder) + } + } + + if (!orderingMatched) { SortExec(requiredOrdering, global = false, child = child) } else { child diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 13490c35679a2..375da224aaa7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD import org.apache.spark.sql.{execution, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Literal, SortOrder} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Repartition} import org.apache.spark.sql.catalyst.plans.physical._ @@ -415,6 +415,44 @@ class PlannerSuite extends SharedSQLContext { } } + test("EnsureRequirements skips sort when required ordering is semantically equal to " + + "existing ordering") { + val exprId: ExprId = NamedExpression.newExprId + val attribute1 = + AttributeReference( + name = "col1", + dataType = LongType, + nullable = false + ) (exprId = exprId, + qualifier = Some("col1_qualifier") + ) + + val attribute2 = + AttributeReference( + name = "col1", + dataType = LongType, + nullable = false + ) (exprId = exprId) + + val orderingA1 = SortOrder(attribute1, Ascending) + val orderingA2 = SortOrder(attribute2, Ascending) + + assert(orderingA1 != orderingA2, s"$orderingA1 should NOT equal to $orderingA2") + assert(orderingA1.semanticEquals(orderingA2), + s"$orderingA1 should be semantically equal to $orderingA2") + + val inputPlan = DummySparkPlan( + children = DummySparkPlan(outputOrdering = Seq(orderingA1)) :: Nil, + requiredChildOrdering = Seq(Seq(orderingA2)), + requiredChildDistribution = Seq(UnspecifiedDistribution) + ) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) + assertDistributionRequirementsAreSatisfied(outputPlan) + if (outputPlan.collect { case s: SortExec => true }.nonEmpty) { + fail(s"No sorts should have been added:\n$outputPlan") + } + } + // This is a regression test for SPARK-11135 test("EnsureRequirements adds sort when required ordering isn't a prefix of existing ordering") { val orderingA = SortOrder(Literal(1), Ascending) From ac22ab0779c8672ba622b90304f05ac44ff83819 Mon Sep 17 00:00:00 2001 From: Brian Cho Date: Thu, 1 Sep 2016 14:13:17 -0700 Subject: [PATCH 1234/1470] [SPARK-16926] [SQL] Remove partition columns from partition metadata. ## What changes were proposed in this pull request? This removes partition columns from column metadata of partitions to match tables. A change introduced in SPARK-14388 removed partition columns from the column metadata of tables, but not for partitions. This causes TableReader to believe that the schema is different between table and partition, and create an unnecessary conversion object inspector in TableReader. ## How was this patch tested? Existing unit tests. Author: Brian Cho Closes #14515 from dafrista/partition-columns-metadata. (cherry picked from commit 473d78649dec7583bcc4ec24b6f38303c38e81a2) Signed-off-by: Davies Liu --- .../org/apache/spark/sql/hive/MetastoreRelation.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index f8ebe088e9696..88d8d4b8cd316 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -163,7 +163,13 @@ private[hive] case class MetastoreRelation( val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tPartition.setSd(sd) - sd.setCols(catalogTable.schema.map(toHiveColumn).asJava) + + // Note: In Hive the schema and partition columns must be disjoint sets + val schema = catalogTable.schema.map(toHiveColumn).filter { c => + !catalogTable.partitionColumnNames.contains(c.getName) + } + sd.setCols(schema.asJava) + p.storage.locationUri.foreach(sd.setLocation) p.storage.inputFormat.foreach(sd.setInputFormat) p.storage.outputFormat.foreach(sd.setOutputFormat) From dd377a52203def279b529832b888ef46be6268dc Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 1 Sep 2016 16:45:26 -0700 Subject: [PATCH 1235/1470] [SPARK-17355] Workaround for HIVE-14684 / HiveResultSetMetaData.isSigned exception ## What changes were proposed in this pull request? Attempting to use Spark SQL's JDBC data source against the Hive ThriftServer results in a `java.sql.SQLException: Method` not supported exception from `org.apache.hive.jdbc.HiveResultSetMetaData.isSigned`. Here are two user reports of this issue: - https://stackoverflow.com/questions/34067686/spark-1-5-1-not-working-with-hive-jdbc-1-2-0 - https://stackoverflow.com/questions/32195946/method-not-supported-in-spark I have filed [HIVE-14684](https://issues.apache.org/jira/browse/HIVE-14684) to attempt to fix this in Hive by implementing the isSigned method, but in the meantime / for compatibility with older JDBC drivers I think we should add special-case error handling to work around this bug. This patch updates `JDBCRDD`'s `ResultSetMetadata` to schema conversion to catch the "Method not supported" exception from Hive and return `isSigned = true`. I believe that this is safe because, as far as I know, Hive does not support unsigned numeric types. ## How was this patch tested? Tested manually against a Spark Thrift Server. Author: Josh Rosen Closes #14911 from JoshRosen/hive-jdbc-workaround. (cherry picked from commit 15539e54c2650a164f09c072f8fae934bb0468c9) Signed-off-by: Josh Rosen --- .../sql/execution/datasources/jdbc/JDBCRDD.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index f655155287974..82bc9d75beff6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -136,7 +136,16 @@ object JDBCRDD extends Logging { val typeName = rsmd.getColumnTypeName(i + 1) val fieldSize = rsmd.getPrecision(i + 1) val fieldScale = rsmd.getScale(i + 1) - val isSigned = rsmd.isSigned(i + 1) + val isSigned = { + try { + rsmd.isSigned(i + 1) + } catch { + // Workaround for HIVE-14684: + case e: SQLException if + e.getMessage == "Method not supported" && + rsmd.getClass.getName == "org.apache.hive.jdbc.HiveResultSetMetaData" => true + } + } val nullable = rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls val metadata = new MetadataBuilder() .putString("name", columnName) From f9463238de1e7ea17da8f258f22e385a0ed4134e Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 2 Sep 2016 08:46:15 +0100 Subject: [PATCH 1236/1470] [SPARK-17342][WEBUI] Style of event timeline is broken ## What changes were proposed in this pull request? SPARK-15373 (#13158) updated the version of vis.js to 4.16.1. As of 4.0.0, some class was renamed like 'timeline to vis-timeline' but that ticket didn't care and now style is broken. In this PR, I've restored the style by modifying `timeline-view.css` and `timeline-view.js`. ## How was this patch tested? manual tests. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) * Before 2016-09-01 1 38 31 * After 2016-09-01 3 30 19 Author: Kousuke Saruta Closes #14900 from sarutak/SPARK-17342. (cherry picked from commit 2ab8dbddaa31e4491b52eb0e495660ebbebfdb9e) Signed-off-by: Sean Owen --- .../apache/spark/ui/static/timeline-view.css | 57 +++++++++---------- .../apache/spark/ui/static/timeline-view.js | 6 +- 2 files changed, 31 insertions(+), 32 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css index f9ad9f8378803..3bf3e8bfa1f31 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -33,12 +33,15 @@ div#application-timeline, div#job-timeline { height: 55px; } -#task-assignment-timeline div.item.range { - padding: 0px; +#task-assignment-timeline div.vis-item.vis-range { height: 26px; border-width: 0; } +#task-assignment-timeline .vis-item-content { + padding: 0px; +} + .task-assignment-timeline-content { width: 100%; } @@ -83,28 +86,24 @@ rect.getting-result-time-proportion { stroke: #75B0A6; } -.vis-item .vis-item-content { - width: 100% -} - -.vis.timeline { +.vis-timeline { line-height: 14px; } -.vis.timeline div.content { +.vis-timeline div.vis-item-content { width: 100%; } -.vis.timeline .item.stage { +.vis-timeline .vis-item.stage { cursor: pointer; } -.vis.timeline .item.stage.succeeded { +.vis-timeline .vis-item.stage.succeeded { background-color: #A0DFFF; border-color: #3EC0FF; } -.vis.timeline .item.stage.succeeded.selected { +.vis-timeline .vis-item.stage.succeeded.vis-selected { background-color: #A0DFFF; border-color: #3EC0FF; z-index: auto; @@ -115,12 +114,12 @@ rect.getting-result-time-proportion { stroke: #3EC0FF; } -.vis.timeline .item.stage.failed { +.vis-timeline .vis-item.stage.failed { background-color: #FFA1B0; border-color: #FF4D6D; } -.vis.timeline .item.stage.failed.selected { +.vis-timeline .vis-item.stage.failed.vis-selected { background-color: #FFA1B0; border-color: #FF4D6D; z-index: auto; @@ -131,12 +130,12 @@ rect.getting-result-time-proportion { stroke: #FF4D6D; } -.vis.timeline .item.stage.running { +.vis-timeline .vis-item.stage.running { background-color: #A2FCC0; border-color: #36F572; } -.vis.timeline .item.stage.running.selected { +.vis-timeline .vis-item.stage.running.vis-selected { background-color: #A2FCC0; border-color: #36F572; z-index: auto; @@ -147,20 +146,20 @@ rect.getting-result-time-proportion { stroke: #36F572; } -.vis.timeline .foreground { +.vis-timeline .vis-foreground { cursor: move; } -.vis.timeline .item.job { +.vis-timeline .vis-item.job { cursor: pointer; } -.vis.timeline .item.job.succeeded { +.vis-timeline .vis-item.job.succeeded { background-color: #A0DFFF; border-color: #3EC0FF; } -.vis.timeline .item.job.succeeded.selected { +.vis-timeline .vis-item.job.succeeded.vis-selected { background-color: #A0DFFF; border-color: #3EC0FF; z-index: auto; @@ -171,12 +170,12 @@ rect.getting-result-time-proportion { stroke: #3EC0FF; } -.vis.timeline .item.job.failed { +.vis-timeline .vis-item.job.failed { background-color: #FFA1B0; border-color: #FF4D6D; } -.vis.timeline .item.job.failed.selected { +.vis-timeline .vis-item.job.failed.vis-selected { background-color: #FFA1B0; border-color: #FF4D6D; z-index: auto; @@ -187,12 +186,12 @@ rect.getting-result-time-proportion { stroke: #FF4D6D; } -.vis.timeline .item.job.running { +.vis-timeline .vis-item.job.running { background-color: #A2FCC0; border-color: #36F572; } -.vis.timeline .item.job.running.selected { +.vis-timeline .vis-item.job.running.vis-selected { background-color: #A2FCC0; border-color: #36F572; z-index: auto; @@ -203,7 +202,7 @@ rect.getting-result-time-proportion { stroke: #36F572; } -.vis.timeline .item.executor.added { +.vis-timeline .vis-item.executor.added { background-color: #A0DFFF; border-color: #3EC0FF; } @@ -213,7 +212,7 @@ rect.getting-result-time-proportion { stroke: #3EC0FF; } -.vis.timeline .item.executor.removed { +.vis-timeline .vis-item.executor.removed { background-color: #FFA1B0; border-color: #FF4D6D; } @@ -223,7 +222,7 @@ rect.getting-result-time-proportion { stroke: #FF4D6D; } -.vis.timeline .item.executor.selected { +.vis-timeline .vis-item.executor.vis-selected { background-color: #A2FCC0; border-color: #36F572; z-index: 2; @@ -262,15 +261,15 @@ span.expand-task-assignment-timeline { cursor: pointer; } -.vis.timeline .item.range .content { +.vis-timeline .vis-item.vis-range .vis-item-content { position: unset; } -.vis.timeline .item .tooltip-inner { +.vis-timeline .vis-item .tooltip-inner { max-width: unset !important; } -.vispanel.center { +.vis-panel.vis-center { font-size: 12px; line-height: 12px; } diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index 9ab5684d901f0..a6153ceda75e2 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -41,7 +41,7 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { setupExecutorEventAction(); function setupJobEventAction() { - $(".item.range.job.application-timeline-object").each(function() { + $(".vis-item.vis-range.job.application-timeline-object").each(function() { var getSelectorForJobEntry = function(baseElem) { var jobIdText = $($(baseElem).find(".application-timeline-content")[0]).text(); var jobId = jobIdText.match("\\(Job (\\d+)\\)$")[1]; @@ -116,7 +116,7 @@ function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { setupExecutorEventAction(); function setupStageEventAction() { - $(".item.range.stage.job-timeline-object").each(function() { + $(".vis-item.vis-range.stage.job-timeline-object").each(function() { var getSelectorForStageEntry = function(baseElem) { var stageIdText = $($(baseElem).find(".job-timeline-content")[0]).text(); var stageIdAndAttempt = stageIdText.match("\\(Stage (\\d+\\.\\d+)\\)$")[1].split("."); @@ -233,7 +233,7 @@ $(function (){ }); function setupExecutorEventAction() { - $(".item.box.executor").each(function () { + $(".vis-item.vis-box.executor").each(function () { $(this).hover( function() { $($(this).find(".executor-event-content")[0]).tooltip("show"); From 171bdfd963b5dda85ddf5e72b72471fdaaaf2fe3 Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Fri, 2 Sep 2016 01:47:17 -0700 Subject: [PATCH 1237/1470] [SPARK-16883][SPARKR] SQL decimal type is not properly cast to number when collecting SparkDataFrame ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) registerTempTable(createDataFrame(iris), "iris") str(collect(sql("select cast('1' as double) as x, cast('2' as decimal) as y from iris limit 5"))) 'data.frame': 5 obs. of 2 variables: $ x: num 1 1 1 1 1 $ y:List of 5 ..$ : num 2 ..$ : num 2 ..$ : num 2 ..$ : num 2 ..$ : num 2 The problem is that spark returns `decimal(10, 0)` col type, instead of `decimal`. Thus, `decimal(10, 0)` is not handled correctly. It should be handled as "double". As discussed in JIRA thread, we can have two potential fixes: 1). Scala side fix to add a new case when writing the object back; However, I can't use spark.sql.types._ in Spark core due to dependency issues. I don't find a way of doing type case match; 2). SparkR side fix: Add a helper function to check special type like `"decimal(10, 0)"` and replace it with `double`, which is PRIMITIVE type. This special helper is generic for adding new types handling in the future. I open this PR to discuss pros and cons of both approaches. If we want to do Scala side fix, we need to find a way to match the case of DecimalType and StructType in Spark Core. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Manual test: > str(collect(sql("select cast('1' as double) as x, cast('2' as decimal) as y from iris limit 5"))) 'data.frame': 5 obs. of 2 variables: $ x: num 1 1 1 1 1 $ y: num 2 2 2 2 2 R Unit tests Author: wm624@hotmail.com Closes #14613 from wangmiao1981/type. (cherry picked from commit 0f30cdedbdb0d38e8c479efab6bb1c6c376206ff) Signed-off-by: Felix Cheung --- R/pkg/R/DataFrame.R | 13 ++++++++++++- R/pkg/R/types.R | 16 ++++++++++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 22 ++++++++++++++++++++++ 3 files changed, 50 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ab45d2c1a42c0..8aea22896cbc0 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -390,7 +390,11 @@ setMethod("coltypes", } if (is.null(type)) { - stop(paste("Unsupported data type: ", x)) + specialtype <- specialtypeshandle(x) + if (is.null(specialtype)) { + stop(paste("Unsupported data type: ", x)) + } + type <- PRIMITIVE_TYPES[[specialtype]] } } type @@ -1056,6 +1060,13 @@ setMethod("collect", df[[colIndex]] <- col } else { colType <- dtypes[[colIndex]][[2]] + if (is.null(PRIMITIVE_TYPES[[colType]])) { + specialtype <- specialtypeshandle(colType) + if (!is.null(specialtype)) { + colType <- specialtype + } + } + # Note that "binary" columns behave like complex types. if (!is.null(PRIMITIVE_TYPES[[colType]]) && colType != "binary") { vec <- do.call(c, col) diff --git a/R/pkg/R/types.R b/R/pkg/R/types.R index ad048b1cd1795..abca703617c7b 100644 --- a/R/pkg/R/types.R +++ b/R/pkg/R/types.R @@ -67,3 +67,19 @@ rToSQLTypes <- as.environment(list( "double" = "double", "character" = "string", "logical" = "boolean")) + +# Helper function of coverting decimal type. When backend returns column type in the +# format of decimal(,) (e.g., decimal(10, 0)), this function coverts the column type +# as double type. This function converts backend returned types that are not the key +# of PRIMITIVE_TYPES, but should be treated as PRIMITIVE_TYPES. +# @param A type returned from the JVM backend. +# @return A type is the key of the PRIMITIVE_TYPES. +specialtypeshandle <- function(type) { + returntype <- NULL + m <- regexec("^decimal(.+)$", type) + matchedStrings <- regmatches(type, m) + if (length(matchedStrings[[1]]) >= 2) { + returntype <- "double" + } + returntype +} diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 279d512de8b48..3953a49a615a9 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -526,6 +526,17 @@ test_that( expect_is(newdf, "SparkDataFrame") expect_equal(count(newdf), 1) dropTempView("table1") + + createOrReplaceTempView(df, "dfView") + sqlCast <- collect(sql("select cast('2' as decimal) as x from dfView limit 1")) + out <- capture.output(sqlCast) + expect_true(is.data.frame(sqlCast)) + expect_equal(names(sqlCast)[1], "x") + expect_equal(nrow(sqlCast), 1) + expect_equal(ncol(sqlCast), 1) + expect_equal(out[1], " x") + expect_equal(out[2], "1 2") + dropTempView("dfView") }) test_that("test cache, uncache and clearCache", { @@ -2081,6 +2092,9 @@ test_that("Method coltypes() to get and set R's data types of a DataFrame", { # Test primitive types DF <- createDataFrame(data, schema) expect_equal(coltypes(DF), c("integer", "logical", "POSIXct")) + createOrReplaceTempView(DF, "DFView") + sqlCast <- sql("select cast('2' as decimal) as x from DFView limit 1") + expect_equal(coltypes(sqlCast), "numeric") # Test complex types x <- createDataFrame(list(list(as.environment( @@ -2124,6 +2138,14 @@ test_that("Method str()", { "setosa\" \"setosa\" \"setosa\" \"setosa\"")) expect_equal(out[7], " $ col : logi TRUE TRUE TRUE TRUE TRUE TRUE") + createOrReplaceTempView(irisDF2, "irisView") + + sqlCast <- sql("select cast('2' as decimal) as x from irisView limit 1") + castStr <- capture.output(str(sqlCast)) + expect_equal(length(castStr), 2) + expect_equal(castStr[1], "'SparkDataFrame': 1 variables:") + expect_equal(castStr[2], " $ x: num 2") + # A random dataset with many columns. This test is to check str limits # the number of columns. Therefore, it will suffice to check for the # number of returned rows From d9d10ffb9c2ee2a79257d8827bdc99052d144511 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 2 Sep 2016 10:26:43 +0100 Subject: [PATCH 1238/1470] [SPARK-17352][WEBUI] Executor computing time can be negative-number because of calculation error ## What changes were proposed in this pull request? In StagePage, executor-computing-time is calculated but calculation error can occur potentially because it's calculated by subtraction of floating numbers. Following capture is an example. capture-timeline ## How was this patch tested? Manual tests. Author: Kousuke Saruta Closes #14908 from sarutak/SPARK-17352. (cherry picked from commit 7ee24dac8e779f6a9bf45371fdc2be83fb679cb2) Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 145dbf8eeb6a7..d93a660d85555 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -636,9 +636,9 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } val executorComputingTime = executorRunTime - shuffleReadTime - shuffleWriteTime val executorComputingTimeProportion = - (100 - schedulerDelayProportion - shuffleReadTimeProportion - + math.max(100 - schedulerDelayProportion - shuffleReadTimeProportion - shuffleWriteTimeProportion - serializationTimeProportion - - deserializationTimeProportion - gettingResultTimeProportion) + deserializationTimeProportion - gettingResultTimeProportion, 0) val schedulerDelayProportionPos = 0 val deserializationTimeProportionPos = From 91a3cf1365157918f280d60c9b3ffeec4c087b92 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 2 Sep 2016 22:31:01 +0800 Subject: [PATCH 1239/1470] [SPARK-16935][SQL] Verification of Function-related ExternalCatalog APIs Function-related `HiveExternalCatalog` APIs do not have enough verification logics. After the PR, `HiveExternalCatalog` and `InMemoryCatalog` become consistent in the error handling. For example, below is the exception we got when calling `renameFunction`. ``` 15:13:40.369 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db1, returning NoSuchObjectException 15:13:40.377 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db2, returning NoSuchObjectException 15:13:40.739 ERROR DataNucleus.Datastore.Persist: Update of object "org.apache.hadoop.hive.metastore.model.MFunction205629e9" using statement "UPDATE FUNCS SET FUNC_NAME=? WHERE FUNC_ID=?" failed : org.apache.derby.shared.common.error.DerbySQLIntegrityConstraintViolationException: The statement was aborted because it would have caused a duplicate key value in a unique or primary key constraint or unique index identified by 'UNIQUEFUNCTION' defined on 'FUNCS'. at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source) at org.apache.derby.impl.jdbc.Util.generateCsSQLException(Unknown Source) at org.apache.derby.impl.jdbc.TransactionResourceImpl.wrapInSQLException(Unknown Source) at org.apache.derby.impl.jdbc.TransactionResourceImpl.handleException(Unknown Source) ``` Improved the existing test cases to check whether the messages are right. Author: gatorsmile Closes #14521 from gatorsmile/functionChecking. (cherry picked from commit 247a4faf06c1dd47a6543c56929cd0182a03e106) Signed-off-by: Wenchen Fan --- .../catalyst/catalog/ExternalCatalog.scala | 14 ++++++++++++- .../catalyst/catalog/InMemoryCatalog.scala | 19 ++--------------- .../catalog/ExternalCatalogSuite.scala | 21 ++++++++++--------- .../spark/sql/hive/HiveExternalCatalog.scala | 8 +++++++ 4 files changed, 34 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 6714846e8cbda..4371ff3d58a3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.catalog -import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException} /** @@ -38,6 +38,18 @@ abstract class ExternalCatalog { } } + protected def requireFunctionExists(db: String, funcName: String): Unit = { + if (!functionExists(db, funcName)) { + throw new NoSuchFunctionException(db = db, func = funcName) + } + } + + protected def requireFunctionNotExists(db: String, funcName: String): Unit = { + if (functionExists(db, funcName)) { + throw new FunctionAlreadyExistsException(db = db, func = funcName) + } + } + // -------------------------------------------------------------------------- // Databases // -------------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index fb3e1b3637f21..ef5a19687ce86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -59,18 +59,6 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E catalog(db).tables(table).partitions.contains(spec) } - private def requireFunctionExists(db: String, funcName: String): Unit = { - if (!functionExists(db, funcName)) { - throw new NoSuchFunctionException(db = db, func = funcName) - } - } - - private def requireFunctionNotExists(db: String, funcName: String): Unit = { - if (functionExists(db, funcName)) { - throw new FunctionAlreadyExistsException(db = db, func = funcName) - } - } - private def requireTableExists(db: String, table: String): Unit = { if (!tableExists(db, table)) { throw new NoSuchTableException(db = db, table = table) @@ -465,11 +453,8 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E override def createFunction(db: String, func: CatalogFunction): Unit = synchronized { requireDbExists(db) - if (functionExists(db, func.identifier.funcName)) { - throw new FunctionAlreadyExistsException(db = db, func = func.identifier.funcName) - } else { - catalog(db).functions.put(func.identifier.funcName, func) - } + requireFunctionNotExists(db, func.identifier.funcName) + catalog(db).functions.put(func.identifier.funcName, func) } override def dropFunction(db: String, funcName: String): Unit = synchronized { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 0c4d363365025..31e422e91aea3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, NoSuchDatabaseException, NoSuchFunctionException} import org.apache.spark.util.Utils @@ -439,14 +440,14 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac test("create function when database does not exist") { val catalog = newBasicCatalog() - intercept[AnalysisException] { + intercept[NoSuchDatabaseException] { catalog.createFunction("does_not_exist", newFunc()) } } test("create function that already exists") { val catalog = newBasicCatalog() - intercept[AnalysisException] { + intercept[FunctionAlreadyExistsException] { catalog.createFunction("db2", newFunc("func1")) } } @@ -460,14 +461,14 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac test("drop function when database does not exist") { val catalog = newBasicCatalog() - intercept[AnalysisException] { + intercept[NoSuchDatabaseException] { catalog.dropFunction("does_not_exist", "something") } } test("drop function that does not exist") { val catalog = newBasicCatalog() - intercept[AnalysisException] { + intercept[NoSuchFunctionException] { catalog.dropFunction("db2", "does_not_exist") } } @@ -477,14 +478,14 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac assert(catalog.getFunction("db2", "func1") == CatalogFunction(FunctionIdentifier("func1", Some("db2")), funcClass, Seq.empty[FunctionResource])) - intercept[AnalysisException] { + intercept[NoSuchFunctionException] { catalog.getFunction("db2", "does_not_exist") } } test("get function when database does not exist") { val catalog = newBasicCatalog() - intercept[AnalysisException] { + intercept[NoSuchDatabaseException] { catalog.getFunction("does_not_exist", "func1") } } @@ -494,15 +495,15 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac val newName = "funcky" assert(catalog.getFunction("db2", "func1").className == funcClass) catalog.renameFunction("db2", "func1", newName) - intercept[AnalysisException] { catalog.getFunction("db2", "func1") } + intercept[NoSuchFunctionException] { catalog.getFunction("db2", "func1") } assert(catalog.getFunction("db2", newName).identifier.funcName == newName) assert(catalog.getFunction("db2", newName).className == funcClass) - intercept[AnalysisException] { catalog.renameFunction("db2", "does_not_exist", "me") } + intercept[NoSuchFunctionException] { catalog.renameFunction("db2", "does_not_exist", "me") } } test("rename function when database does not exist") { val catalog = newBasicCatalog() - intercept[AnalysisException] { + intercept[NoSuchDatabaseException] { catalog.renameFunction("does_not_exist", "func1", "func5") } } @@ -510,7 +511,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac test("rename function when new function already exists") { val catalog = newBasicCatalog() catalog.createFunction("db2", newFunc("func2", Some("db2"))) - intercept[AnalysisException] { + intercept[FunctionAlreadyExistsException] { catalog.renameFunction("db2", "func1", "func2") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index b8bc9ab900ad1..88cf06fce6c67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -339,31 +339,39 @@ private[spark] class HiveExternalCatalog(client: HiveClient, hadoopConf: Configu override def createFunction( db: String, funcDefinition: CatalogFunction): Unit = withClient { + requireDbExists(db) // Hive's metastore is case insensitive. However, Hive's createFunction does // not normalize the function name (unlike the getFunction part). So, // we are normalizing the function name. val functionName = funcDefinition.identifier.funcName.toLowerCase + requireFunctionNotExists(db, functionName) val functionIdentifier = funcDefinition.identifier.copy(funcName = functionName) client.createFunction(db, funcDefinition.copy(identifier = functionIdentifier)) } override def dropFunction(db: String, name: String): Unit = withClient { + requireFunctionExists(db, name) client.dropFunction(db, name) } override def renameFunction(db: String, oldName: String, newName: String): Unit = withClient { + requireFunctionExists(db, oldName) + requireFunctionNotExists(db, newName) client.renameFunction(db, oldName, newName) } override def getFunction(db: String, funcName: String): CatalogFunction = withClient { + requireFunctionExists(db, funcName) client.getFunction(db, funcName) } override def functionExists(db: String, funcName: String): Boolean = withClient { + requireDbExists(db) client.functionExists(db, funcName) } override def listFunctions(db: String, pattern: String): Seq[String] = withClient { + requireDbExists(db) client.listFunctions(db, pattern) } From 30e5c84939a5169cec1720196e1122fc0759ae2a Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Fri, 2 Sep 2016 10:08:14 -0700 Subject: [PATCH 1240/1470] [SPARK-17261] [PYSPARK] Using HiveContext after re-creating SparkContext in Spark 2.0 throws "Java.lang.illegalStateException: Cannot call methods on a stopped sparkContext" ## What changes were proposed in this pull request? Set SparkSession._instantiatedContext as None so that we can recreate SparkSession again. ## How was this patch tested? Tested manually using the following command in pyspark shell ``` spark.stop() spark = SparkSession.builder.enableHiveSupport().getOrCreate() spark.sql("show databases").show() ``` Author: Jeff Zhang Closes #14857 from zjffdu/SPARK-17261. (cherry picked from commit ea662286561aa9fe321cb0a0e10cdeaf60440b90) Signed-off-by: Davies Liu --- python/pyspark/sql/session.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index d8627ce97a280..d25823dfcacd3 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -597,6 +597,7 @@ def stop(self): """Stop the underlying :class:`SparkContext`. """ self._sc.stop() + SparkSession._instantiatedContext = None @since(2.0) def __enter__(self): From 29ac2f62e88ea8e280b474e61cdb2ab0a0d92a94 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 2 Sep 2016 10:12:10 -0700 Subject: [PATCH 1241/1470] [SPARK-17376][SPARKR] Spark version should be available in R ## What changes were proposed in this pull request? Add sparkR.version() API. ``` > sparkR.version() [1] "2.1.0-SNAPSHOT" ``` ## How was this patch tested? manual, unit tests Author: Felix Cheung Closes #14935 from felixcheung/rsparksessionversion. (cherry picked from commit 812333e4336113e44d2c9473bcba1cee4a989d2c) Signed-off-by: Shivaram Venkataraman --- R/pkg/NAMESPACE | 13 +++++++------ R/pkg/R/SQLContext.R | 19 +++++++++++++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 6 ++++++ 3 files changed, 32 insertions(+), 6 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 666e76a4eb145..4c77d951247fd 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -15,8 +15,15 @@ export("sparkR.init") export("sparkR.stop") export("sparkR.session.stop") export("sparkR.conf") +export("sparkR.version") export("print.jobj") +export("sparkR.newJObject") +export("sparkR.callJMethod") +export("sparkR.callJStatic") + +export("install.spark") + export("sparkRSQL.init", "sparkRHive.init") @@ -356,9 +363,3 @@ S3method(structField, character) S3method(structField, jobj) S3method(structType, jobj) S3method(structType, structField) - -export("sparkR.newJObject") -export("sparkR.callJMethod") -export("sparkR.callJStatic") - -export("install.spark") diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 572e71e25b80b..a1404543be127 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -156,6 +156,25 @@ sparkR.conf <- function(key, defaultValue) { } } +#' Get version of Spark on which this application is running +#' +#' Get version of Spark on which this application is running. +#' +#' @return a character string of the Spark version +#' @rdname sparkR.version +#' @name sparkR.version +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' version <- sparkR.version() +#' } +#' @note sparkR.version since 2.1.0 +sparkR.version <- function() { + sparkSession <- getSparkSession() + callJMethod(sparkSession, "version") +} + getDefaultSqlSource <- function() { l <- sparkR.conf("spark.sql.sources.default", "org.apache.spark.sql.parquet") l[["spark.sql.sources.default"]] diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 3953a49a615a9..78a3754e776e1 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2499,6 +2499,12 @@ test_that("enableHiveSupport on SparkSession", { expect_equal(value, "hive") }) +test_that("Spark version from SparkSession", { + ver <- callJMethod(sc, "version") + version <- sparkR.version() + expect_equal(ver, version) +}) + unlink(parquetPath) unlink(orcPath) unlink(jsonPath) From d4ae35d02f92df407e54b65c2d6b48388448f031 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 2 Sep 2016 10:28:37 -0700 Subject: [PATCH 1242/1470] [SPARKR][DOC] regexp_extract should doc that it returns empty string when match fails ## What changes were proposed in this pull request? Doc change - see https://issues.apache.org/jira/browse/SPARK-16324 ## How was this patch tested? manual check Author: Felix Cheung Closes #14934 from felixcheung/regexpextractdoc. (cherry picked from commit 419eefd811a4e29a73bc309157f150751e478db5) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/functions.R | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index dbf8dd89a204e..369b1d00d9e5b 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2876,7 +2876,8 @@ setMethod("randn", signature(seed = "numeric"), #' regexp_extract #' -#' Extract a specific(idx) group identified by a java regex, from the specified string column. +#' Extract a specific \code{idx} group identified by a Java regex, from the specified string column. +#' If the regex did not match, or the specified group did not match, an empty string is returned. #' #' @param x a string Column. #' @param pattern a regular expression. From 03d9af6043ae443ced004383c996fa8eebf3a1d1 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 2 Sep 2016 11:08:25 -0700 Subject: [PATCH 1243/1470] [SPARK-17376][SPARKR] followup - change since version ## What changes were proposed in this pull request? change since version in doc ## How was this patch tested? manual Author: Felix Cheung Closes #14939 from felixcheung/rsparkversion2. (cherry picked from commit eac1d0e921345b5d15aa35d8c565140292ab2af3) Signed-off-by: Felix Cheung --- R/pkg/R/SQLContext.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index a1404543be127..783df53c12ca2 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -169,7 +169,7 @@ sparkR.conf <- function(key, defaultValue) { #' sparkR.session() #' version <- sparkR.version() #' } -#' @note sparkR.version since 2.1.0 +#' @note sparkR.version since 2.0.1 sparkR.version <- function() { sparkSession <- getSparkSession() callJMethod(sparkSession, "version") From c9c36fa0c7bccefde808bdbc32b04e8555356001 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 2 Sep 2016 15:10:12 -0700 Subject: [PATCH 1244/1470] [SPARK-17230] [SQL] Should not pass optimized query into QueryExecution in DataFrameWriter Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs. For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result. Ideally, we should make all the analyzer rules all idempotent, that may require lots of effort to double checking them one by one (may be not easy). An easier approach could be never feed a optimized plan into Analyzer, this PR fix the case for RunnableComand, they will be optimized, during execution, the passed `query` will also be passed into QueryExecution again. This PR make these `query` not part of the children, so they will not be optimized and analyzed again. Right now, we did not know a logical plan is optimized or not, we could introduce a flag for that, and make sure a optimized logical plan will not be analyzed again. Added regression tests. Author: Davies Liu Closes #14797 from davies/fix_writer. (cherry picked from commit ed9c884dcf925500ceb388b06b33bd2c95cd2ada) Signed-off-by: Davies Liu --- .../spark/sql/execution/command/commands.scala | 2 +- .../execution/command/createDataSourceTables.scala | 2 +- .../sql/execution/datasources/DataSource.scala | 14 +++++++++++++- .../execution/datasources/DataSourceStrategy.scala | 2 +- .../InsertIntoHadoopFsRelationCommand.scala | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 8 ++++++++ .../execution/CreateHiveTableAsSelectCommand.scala | 2 +- 7 files changed, 26 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index cce1489abd301..424a962b5eb1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.types._ */ trait RunnableCommand extends LogicalPlan with logical.Command { override def output: Seq[Attribute] = Seq.empty - override def children: Seq[LogicalPlan] = Seq.empty + final override def children: Seq[LogicalPlan] = Seq.empty def run(sparkSession: SparkSession): Seq[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index c38eca5156e5a..900446c6bd1d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -139,7 +139,7 @@ case class CreateDataSourceTableAsSelectCommand( query: LogicalPlan) extends RunnableCommand { - override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) + override protected def innerChildren: Seq[LogicalPlan] = Seq(query) override def run(sparkSession: SparkSession): Seq[Row] = { // Since we are saving metadata to metastore, we need to check if metastore supports diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index f5727da387d13..784fea520c249 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -30,6 +30,8 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat @@ -484,13 +486,23 @@ case class DataSource( } } + // SPARK-17230: Resolve the partition columns so InsertIntoHadoopFsRelationCommand does + // not need to have the query as child, to avoid to analyze an optimized query, + // because InsertIntoHadoopFsRelationCommand will be optimized first. + val columns = partitionColumns.map { name => + val plan = data.logicalPlan + plan.resolve(name :: Nil, data.sparkSession.sessionState.analyzer.resolver).getOrElse { + throw new AnalysisException( + s"Unable to resolve ${name} given [${plan.output.map(_.name).mkString(", ")}]") + }.asInstanceOf[Attribute] + } // For partitioned relation r, r.schema's column ordering can be different from the column // ordering of data.logicalPlan (partition columns are all moved after data column). This // will be adjusted within InsertIntoHadoopFsRelation. val plan = InsertIntoHadoopFsRelationCommand( outputPath, - partitionColumns.map(UnresolvedAttribute.quoted), + columns, bucketSpec, format, () => Unit, // No existing table needs to be refreshed. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index bd65d0251197b..6b4b3b84d4d14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -187,7 +187,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { InsertIntoHadoopFsRelationCommand( outputPath, - t.partitionSchema.fields.map(_.name).map(UnresolvedAttribute(_)), + query.resolve(t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver), t.bucketSpec, t.fileFormat, () => t.refresh(), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index d8b8fae3bf2d3..518b02b718757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -66,7 +66,7 @@ case class InsertIntoHadoopFsRelationCommand( mode: SaveMode) extends RunnableCommand { - override def children: Seq[LogicalPlan] = query :: Nil + override protected def innerChildren: Seq[LogicalPlan] = query :: Nil override def run(sparkSession: SparkSession): Seq[Row] = { // Most formats don't do well with duplicate columns, so lets not allow that diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 27a0a2a776c36..e071aef4c2db7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -424,6 +424,14 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be spark.range(10).write.orc(dir) } + test("SPARK-17230: write out results of decimal calculation") { + val df = spark.range(99, 101) + .selectExpr("id", "cast(id as long) * cast('1.0' as decimal(38, 18)) as num") + df.write.mode(SaveMode.Overwrite).parquet(dir) + val df2 = spark.read.parquet(dir) + checkAnswer(df2, df) + } + private def testRead( df: => DataFrame, expectedResult: Seq[String], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 3a8b0f1b8ebdf..a12b223b76ef0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -42,7 +42,7 @@ case class CreateHiveTableAsSelectCommand( private val tableIdentifier = tableDesc.identifier - override def children: Seq[LogicalPlan] = Seq(query) + override def innerChildren: Seq[LogicalPlan] = Seq(query) override def run(sparkSession: SparkSession): Seq[Row] = { lazy val metastoreRelation: MetastoreRelation = { From a3930c3b9afa9f7eba2a5c8b8f279ca38e348e9b Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Fri, 2 Sep 2016 15:16:16 -0700 Subject: [PATCH 1245/1470] [SPARK-16334] Reusing same dictionary column for decoding consecutive row groups shouldn't throw an error This patch fixes a bug in the vectorized parquet reader that's caused by re-using the same dictionary column vector while reading consecutive row groups. Specifically, this issue manifests for a certain distribution of dictionary/plain encoded data while we read/populate the underlying bit packed dictionary data into a column-vector based data structure. Manually tested on datasets provided by the community. Thanks to Chris Perluss and Keith Kraus for their invaluable help in tracking down this issue! Author: Sameer Agarwal Closes #14941 from sameeragarwal/parquet-exception-2. (cherry picked from commit a2c9acb0e54b2e38cb8ee6431f1ea0e0b4cd959a) Signed-off-by: Davies Liu --- .../parquet/VectorizedColumnReader.java | 54 +++++++++++++------ 1 file changed, 38 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 6c47dc09a8637..cb51cb499eede 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -221,15 +221,21 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, if (column.dataType() == DataTypes.IntegerType || DecimalType.is32BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - column.putInt(i, dictionary.decodeToInt(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putInt(i, dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } } } else if (column.dataType() == DataTypes.ByteType) { for (int i = rowId; i < rowId + num; ++i) { - column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } } } else if (column.dataType() == DataTypes.ShortType) { for (int i = rowId; i < rowId + num; ++i) { - column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } } } else { throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); @@ -240,7 +246,9 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, if (column.dataType() == DataTypes.LongType || DecimalType.is64BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - column.putLong(i, dictionary.decodeToLong(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putLong(i, dictionary.decodeToLong(dictionaryIds.getDictId(i))); + } } } else { throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); @@ -249,21 +257,27 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, case FLOAT: for (int i = rowId; i < rowId + num; ++i) { - column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getDictId(i))); + } } break; case DOUBLE: for (int i = rowId; i < rowId + num; ++i) { - column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getDictId(i))); + } } break; case INT96: if (column.dataType() == DataTypes.TimestampType) { for (int i = rowId; i < rowId + num; ++i) { // TODO: Convert dictionary of Binaries to dictionary of Longs - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); + } } } else { throw new UnsupportedOperationException(); @@ -275,26 +289,34 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // and reuse it across batches. This should mean adding a ByteArray would just update // the length and offset. for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putByteArray(i, v.getBytes()); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putByteArray(i, v.getBytes()); + } } break; case FIXED_LEN_BYTE_ARRAY: // DecimalType written in the legacy mode if (DecimalType.is32BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); + } } } else if (DecimalType.is64BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); + } } } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putByteArray(i, v.getBytes()); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putByteArray(i, v.getBytes()); + } } } else { throw new UnsupportedOperationException(); From b8f65dad7be22231e982aaec3bbd69dbeacc20da Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 2 Sep 2016 15:40:02 -0700 Subject: [PATCH 1246/1470] Fix build --- .../org/apache/spark/sql/execution/datasources/DataSource.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 784fea520c249..0bf4a9c9025a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -30,7 +30,6 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider From c0ea7707127c92ecb51794b96ea40d7cdb28b168 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 2 Sep 2016 16:05:37 -0700 Subject: [PATCH 1247/1470] Revert "[SPARK-16334] Reusing same dictionary column for decoding consecutive row groups shouldn't throw an error" This reverts commit a3930c3b9afa9f7eba2a5c8b8f279ca38e348e9b. --- .../parquet/VectorizedColumnReader.java | 54 ++++++------------- 1 file changed, 16 insertions(+), 38 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index cb51cb499eede..6c47dc09a8637 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -221,21 +221,15 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, if (column.dataType() == DataTypes.IntegerType || DecimalType.is32BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - column.putInt(i, dictionary.decodeToInt(dictionaryIds.getDictId(i))); - } + column.putInt(i, dictionary.decodeToInt(dictionaryIds.getInt(i))); } } else if (column.dataType() == DataTypes.ByteType) { for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getDictId(i))); - } + column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getInt(i))); } } else if (column.dataType() == DataTypes.ShortType) { for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getDictId(i))); - } + column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getInt(i))); } } else { throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); @@ -246,9 +240,7 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, if (column.dataType() == DataTypes.LongType || DecimalType.is64BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - column.putLong(i, dictionary.decodeToLong(dictionaryIds.getDictId(i))); - } + column.putLong(i, dictionary.decodeToLong(dictionaryIds.getInt(i))); } } else { throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); @@ -257,27 +249,21 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, case FLOAT: for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getDictId(i))); - } + column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getInt(i))); } break; case DOUBLE: for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getDictId(i))); - } + column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getInt(i))); } break; case INT96: if (column.dataType() == DataTypes.TimestampType) { for (int i = rowId; i < rowId + num; ++i) { // TODO: Convert dictionary of Binaries to dictionary of Longs - if (!column.isNullAt(i)) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); - } + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); } } else { throw new UnsupportedOperationException(); @@ -289,34 +275,26 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // and reuse it across batches. This should mean adding a ByteArray would just update // the length and offset. for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putByteArray(i, v.getBytes()); - } + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putByteArray(i, v.getBytes()); } break; case FIXED_LEN_BYTE_ARRAY: // DecimalType written in the legacy mode if (DecimalType.is32BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); - } + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); } } else if (DecimalType.is64BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); - } + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); } } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - if (!column.isNullAt(i)) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); - column.putByteArray(i, v.getBytes()); - } + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putByteArray(i, v.getBytes()); } } else { throw new UnsupportedOperationException(); From 12a2e2a5ab5db12f39a7b591e914d52058e1581b Mon Sep 17 00:00:00 2001 From: Junyang Qian Date: Fri, 2 Sep 2016 21:11:57 -0700 Subject: [PATCH 1248/1470] [SPARKR][MINOR] Fix docs for sparkR.session and count ## What changes were proposed in this pull request? This PR tries to add some more explanation to `sparkR.session`. It also modifies doc for `count` so when grouped in one doc, the description doesn't confuse users. ## How was this patch tested? Manual test. ![screen shot 2016-09-02 at 1 21 36 pm](https://cloud.githubusercontent.com/assets/15318264/18217198/409613ac-7110-11e6-8dae-cb0c8df557bf.png) Author: Junyang Qian Closes #14942 from junyangq/fixSparkRSessionDoc. (cherry picked from commit d2fde6b72c4aede2e7edb4a7e6653fb1e7b19924) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/functions.R | 3 ++- R/pkg/R/group.R | 2 +- R/pkg/R/sparkR.R | 6 ++++-- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 369b1d00d9e5b..ceedbe76711b1 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -444,7 +444,8 @@ setMethod("cosh", #' Returns the number of items in a group #' -#' Returns the number of items in a group. This is a column aggregate function. +#' This can be used as a column aggregate function with \code{Column} as input, +#' and returns the number of items in a group. #' #' @rdname count #' @name count diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index e3479ef5fa583..17f5283abead1 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -57,7 +57,7 @@ setMethod("show", "GroupedData", #' Count #' -#' Count the number of rows for each group. +#' Count the number of rows for each group when we have \code{GroupedData} input. #' The resulting SparkDataFrame will also contain the grouping columns. #' #' @return A SparkDataFrame. diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index de53b0bf79b58..15afe01c24ed2 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -314,8 +314,10 @@ sparkRHive.init <- function(jsc = NULL) { #' Get the existing SparkSession or initialize a new SparkSession. #' -#' Additional Spark properties can be set (...), and these named parameters take priority over -#' over values in master, appName, named lists of sparkConfig. +#' SparkSession is the entry point into SparkR. \code{sparkR.session} gets the existing +#' SparkSession or initializes a new SparkSession. +#' Additional Spark properties can be set in \code{...}, and these named parameters take priority +#' over values in \code{master}, \code{appName}, named lists of \code{sparkConfig}. #' #' For details on how to initialize and use SparkR, refer to SparkR programming guide at #' \url{http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparksession}. From 949544d017ab25b43b683cd5c1e6783d87bfce45 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sat, 3 Sep 2016 10:03:40 +0100 Subject: [PATCH 1249/1470] [SPARK-17347][SQL][EXAMPLES] Encoder in Dataset example has incorrect type ## What changes were proposed in this pull request? We propose to fix the Encoder type in the Dataset example ## How was this patch tested? The PR will be tested with the current unit test cases Author: CodingCat Closes #14901 from CodingCat/SPARK-17347. (cherry picked from commit 97da41039b2b8fa7f93caf213ae45b9973925995) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/examples/sql/SparkSQLExample.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index 5cd437d017f6f..129b81d5fbbf3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -203,7 +203,7 @@ object SparkSQLExample { // No pre-defined encoders for Dataset[Map[K,V]], define explicitly implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]] // Primitive types and case classes can be also defined as - implicit val stringIntMapEncoder: Encoder[Map[String, Int]] = ExpressionEncoder() + // implicit val stringIntMapEncoder: Encoder[Map[String, Any]] = ExpressionEncoder() // row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T] teenagersDF.map(teenager => teenager.getValuesMap[Any](List("name", "age"))).collect() From 196d62eae05be0d87a20776fa07208b7ea2ddc90 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Sat, 3 Sep 2016 15:35:19 +0100 Subject: [PATCH 1250/1470] [MINOR][SQL] Not dropping all necessary tables ## What changes were proposed in this pull request? was not dropping table `parquet_t3` ## How was this patch tested? tested `LogicalPlanToSQLSuite` locally Author: Sandeep Singh Closes #13767 from techaddict/minor-8. (cherry picked from commit a8a35b39b92fc9000eaac102c67c66be30b05e54) Signed-off-by: Sean Owen --- .../spark/sql/catalyst/LogicalPlanToSQLSuite.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 9c6da6a628dcf..ef2f756a4bde4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -49,9 +49,9 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { protected override def beforeAll(): Unit = { super.beforeAll() - sql("DROP TABLE IF EXISTS parquet_t0") - sql("DROP TABLE IF EXISTS parquet_t1") - sql("DROP TABLE IF EXISTS parquet_t2") + (0 to 3).foreach { i => + sql(s"DROP TABLE IF EXISTS parquet_t$i") + } sql("DROP TABLE IF EXISTS t0") spark.range(10).write.saveAsTable("parquet_t0") @@ -87,10 +87,9 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { override protected def afterAll(): Unit = { try { - sql("DROP TABLE IF EXISTS parquet_t0") - sql("DROP TABLE IF EXISTS parquet_t1") - sql("DROP TABLE IF EXISTS parquet_t2") - sql("DROP TABLE IF EXISTS parquet_t3") + (0 to 3).foreach { i => + sql(s"DROP TABLE IF EXISTS parquet_t$i") + } sql("DROP TABLE IF EXISTS t0") } finally { super.afterAll() From a7f5e7066f935d58d702a3e86b85aa175291d0fc Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 10 Aug 2016 16:25:01 +0800 Subject: [PATCH 1251/1470] [SPARK-16959][SQL] Rebuild Table Comment when Retrieving Metadata from Hive Metastore ### What changes were proposed in this pull request? The `comment` in `CatalogTable` returned from Hive is always empty. We store it in the table property when creating a table. However, when we try to retrieve the table metadata from Hive metastore, we do not rebuild it. The `comment` is always empty. This PR is to fix the issue. ### How was this patch tested? Fixed the test case to verify the change. Author: gatorsmile Closes #14550 from gatorsmile/tableComment. (cherry picked from commit bdd537164dcfeec5e9c51d54791ef16997ff2597) Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 3 ++- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 7 +++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 7b4483aa9e9a9..ac786767050f7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -412,7 +412,8 @@ private[hive] class HiveClientImpl( serdeProperties = Option(h.getTTable.getSd.getSerdeInfo.getParameters) .map(_.asScala.toMap).orNull ), - properties = properties, + properties = properties.filter(kv => kv._1 != "comment"), + comment = properties.get("comment"), viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), unsupportedFeatures = unsupportedFeatures) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index f31215ac31c8e..141640931ecaf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -135,8 +135,11 @@ class HiveDDLSuite sql(s"CREATE VIEW $viewName COMMENT 'no comment' AS SELECT * FROM $tabName") val tableMetadata = catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) val viewMetadata = catalog.getTableMetadata(TableIdentifier(viewName, Some("default"))) - assert(tableMetadata.properties.get("comment") == Option("BLABLA")) - assert(viewMetadata.properties.get("comment") == Option("no comment")) + assert(tableMetadata.comment == Option("BLABLA")) + assert(viewMetadata.comment == Option("no comment")) + // Ensure that `comment` is removed from the table property + assert(tableMetadata.properties.get("comment").isEmpty) + assert(viewMetadata.properties.get("comment").isEmpty) } } } From 3500dbc9bcce243b6656f308ee4941de0350d198 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 26 Jul 2016 18:46:12 +0800 Subject: [PATCH 1252/1470] [SPARK-16663][SQL] desc table should be consistent between data source and hive serde tables Currently there are 2 inconsistence: 1. for data source table, we only print partition names, for hive table, we also print partition schema. After this PR, we will always print schema 2. if column doesn't have comment, data source table will print empty string, hive table will print null. After this PR, we will always print null new test in `HiveDDLSuite` Author: Wenchen Fan Closes #14302 from cloud-fan/minor3. (cherry picked from commit a2abb583caaec9a2cecd5d65b05d172fc096c125) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/command/tables.scala | 11 +++---- .../spark/sql/sources/DDLTestSuite.scala | 30 +++++++++---------- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 30 ++++++++++++++----- .../sql/hive/execution/HiveQuerySuite.scala | 4 +-- 5 files changed, 47 insertions(+), 30 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index a5ccbcf27cb9a..7e6a3527fd03f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -436,11 +436,12 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF private def describePartitionInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { if (DDLUtils.isDatasourceTable(table)) { - val partCols = DDLUtils.getPartitionColumnsFromTableProperties(table) - if (partCols.nonEmpty) { + val userSpecifiedSchema = DDLUtils.getSchemaFromTableProperties(table) + val partColNames = DDLUtils.getPartitionColumnsFromTableProperties(table) + for (schema <- userSpecifiedSchema if partColNames.nonEmpty) { append(buffer, "# Partition Information", "", "") - append(buffer, s"# ${output.head.name}", "", "") - partCols.foreach(col => append(buffer, col, "", "")) + append(buffer, s"# ${output.head.name}", output(1).name, output(2).name) + describeSchema(StructType(partColNames.map(schema(_))), buffer) } } else { if (table.partitionColumns.nonEmpty) { @@ -527,7 +528,7 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF private def describeSchema(schema: StructType, buffer: ArrayBuffer[Row]): Unit = { schema.foreach { column => val comment = - if (column.metadata.contains("comment")) column.metadata.getString("comment") else "" + if (column.metadata.contains("comment")) column.metadata.getString("comment") else null append(buffer, column.name, column.dataType.simpleString, comment) } } 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 5a7a9073fb3a2..c2aedfff348a3 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 @@ -98,21 +98,21 @@ class DDLTestSuite extends DataSourceTest with SharedSQLContext { "describe ddlPeople", Seq( Row("intType", "int", "test comment test1"), - Row("stringType", "string", ""), - Row("dateType", "date", ""), - Row("timestampType", "timestamp", ""), - Row("doubleType", "double", ""), - Row("bigintType", "bigint", ""), - Row("tinyintType", "tinyint", ""), - Row("decimalType", "decimal(10,0)", ""), - Row("fixedDecimalType", "decimal(5,1)", ""), - Row("binaryType", "binary", ""), - Row("booleanType", "boolean", ""), - Row("smallIntType", "smallint", ""), - Row("floatType", "float", ""), - Row("mapType", "map", ""), - Row("arrayType", "array", ""), - Row("structType", "struct", "") + Row("stringType", "string", null), + Row("dateType", "date", null), + Row("timestampType", "timestamp", null), + Row("doubleType", "double", null), + Row("bigintType", "bigint", null), + Row("tinyintType", "tinyint", null), + Row("decimalType", "decimal(10,0)", null), + Row("fixedDecimalType", "decimal(5,1)", null), + Row("binaryType", "binary", null), + Row("booleanType", "boolean", null), + Row("smallIntType", "smallint", null), + Row("floatType", "float", null), + Row("mapType", "map", null), + Row("arrayType", "array", null), + Row("structType", "struct", null) )) test("SPARK-7686 DescribeCommand should have correct physical plan output attributes") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index af071f95e69fa..a0b3b37600924 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -748,7 +748,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert(schema === actualSchema) // Checks the DESCRIBE output. - checkAnswer(sql("DESCRIBE spark6655"), Row("int", "int", "") :: Nil) + checkAnswer(sql("DESCRIBE spark6655"), Row("int", "int", null) :: Nil) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 141640931ecaf..3cf3c6a9e29ca 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -472,6 +472,22 @@ class HiveDDLSuite } } + test("desc table for Hive table - partitioned table") { + withTable("tbl") { + sql("CREATE TABLE tbl(a int) PARTITIONED BY (b int)") + + assert(sql("DESC tbl").collect().containsSlice( + Seq( + Row("a", "int", null), + Row("b", "int", null), + Row("# Partition Information", "", ""), + Row("# col_name", "data_type", "comment"), + Row("b", "int", null) + ) + )) + } + } + test("desc table for data source table using Hive Metastore") { assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive") val tabName = "tab1" @@ -677,7 +693,7 @@ class HiveDDLSuite val desc = sql("DESC FORMATTED t1").collect().toSeq - assert(desc.contains(Row("id", "bigint", ""))) + assert(desc.contains(Row("id", "bigint", null))) } } } @@ -694,13 +710,13 @@ class HiveDDLSuite assert(formattedDesc.containsSlice( Seq( - Row("a", "bigint", ""), - Row("b", "bigint", ""), - Row("c", "bigint", ""), - Row("d", "bigint", ""), + Row("a", "bigint", null), + Row("b", "bigint", null), + Row("c", "bigint", null), + Row("d", "bigint", null), Row("# Partition Information", "", ""), - Row("# col_name", "", ""), - Row("d", "", ""), + Row("# col_name", "data_type", "comment"), + Row("d", "bigint", null), Row("", "", ""), Row("# Detailed Table Information", "", ""), Row("Database:", "default", "") 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 db6c64a1a130a..55d79057db132 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 @@ -825,8 +825,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assertResult( Array( - Row("a", "int", ""), - Row("b", "string", "")) + Row("a", "int", null), + Row("b", "string", null)) ) { sql("DESCRIBE test_describe_commands2") .select('col_name, 'data_type, 'comment) From 704215d3055bad7957d1d6da1a1a526c0d27d37d Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sat, 3 Sep 2016 19:02:20 +0200 Subject: [PATCH 1253/1470] [SPARK-17335][SQL] Fix ArrayType and MapType CatalogString. ## What changes were proposed in this pull request? the `catalogString` for `ArrayType` and `MapType` currently calls the `simpleString` method on its children. This is a problem when the child is a struct, the `struct.simpleString` implementation truncates the number of fields it shows (25 at max). This breaks the generation of a proper `catalogString`, and has shown to cause errors while writing to Hive. This PR fixes this by providing proper `catalogString` implementations for `ArrayData` or `MapData`. ## How was this patch tested? Added testing for `catalogString` to `DataTypeSuite`. Author: Herman van Hovell Closes #14938 from hvanhovell/SPARK-17335. (cherry picked from commit c2a1576c230697f56f282b6388c79835377e0f2f) Signed-off-by: Herman van Hovell --- .../apache/spark/sql/types/ArrayType.scala | 2 + .../org/apache/spark/sql/types/MapType.scala | 2 + .../spark/sql/types/DataTypeSuite.scala | 30 +++ .../WideSchemaBenchmark-results.txt | 174 ++++++++++-------- 4 files changed, 133 insertions(+), 75 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 520e344361625..82a03b0afc002 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -77,6 +77,8 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT override def simpleString: String = s"array<${elementType.simpleString}>" + override def catalogString: String = s"array<${elementType.catalogString}>" + override def sql: String = s"ARRAY<${elementType.sql}>" override private[spark] def asNullable: ArrayType = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala index 454ea403bac22..178960929bd83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -64,6 +64,8 @@ case class MapType( override def simpleString: String = s"map<${keyType.simpleString},${valueType.simpleString}>" + override def catalogString: String = s"map<${keyType.catalogString},${valueType.catalogString}>" + override def sql: String = s"MAP<${keyType.sql}, ${valueType.sql}>" override private[spark] def asNullable: MapType = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index 6b85f12521c2a..569230accfd74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.types import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser class DataTypeSuite extends SparkFunSuite { @@ -342,4 +343,33 @@ class DataTypeSuite extends SparkFunSuite { StructField("a", StringType, nullable = false) :: StructField("b", StringType, nullable = false) :: Nil), expected = false) + + def checkCatalogString(dt: DataType): Unit = { + test(s"catalogString: $dt") { + val dt2 = CatalystSqlParser.parseDataType(dt.catalogString) + assert(dt === dt2) + } + } + def createStruct(n: Int): StructType = new StructType(Array.tabulate(n) { + i => StructField(s"col$i", IntegerType, nullable = true) + }) + + checkCatalogString(BooleanType) + checkCatalogString(ByteType) + checkCatalogString(ShortType) + checkCatalogString(IntegerType) + checkCatalogString(LongType) + checkCatalogString(FloatType) + checkCatalogString(DoubleType) + checkCatalogString(DecimalType(10, 5)) + checkCatalogString(BinaryType) + checkCatalogString(StringType) + checkCatalogString(DateType) + checkCatalogString(TimestampType) + checkCatalogString(createStruct(4)) + checkCatalogString(createStruct(40)) + checkCatalogString(ArrayType(IntegerType)) + checkCatalogString(ArrayType(createStruct(40))) + checkCatalogString(MapType(IntegerType, StringType)) + checkCatalogString(MapType(IntegerType, createStruct(40))) } diff --git a/sql/core/benchmarks/WideSchemaBenchmark-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-results.txt index ea6a6616c23d4..0b9f791ac85e4 100644 --- a/sql/core/benchmarks/WideSchemaBenchmark-results.txt +++ b/sql/core/benchmarks/WideSchemaBenchmark-results.txt @@ -1,93 +1,117 @@ -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz + parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 select expressions 3 / 5 0.0 2967064.0 1.0X -100 select expressions 11 / 12 0.0 11369518.0 0.3X -2500 select expressions 243 / 250 0.0 242561004.0 0.0X +1 select expressions 2 / 4 0.0 2050147.0 1.0X +100 select expressions 6 / 7 0.0 6123412.0 0.3X +2500 select expressions 135 / 141 0.0 134623148.0 0.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 cols x 100000 rows (read in-mem) 28 / 40 3.6 278.8 1.0X -1 cols x 100000 rows (exec in-mem) 28 / 42 3.5 284.0 1.0X -1 cols x 100000 rows (read parquet) 23 / 35 4.4 228.8 1.2X -1 cols x 100000 rows (write parquet) 163 / 182 0.6 1633.0 0.2X -100 cols x 1000 rows (read in-mem) 27 / 39 3.7 266.9 1.0X -100 cols x 1000 rows (exec in-mem) 48 / 79 2.1 481.7 0.6X -100 cols x 1000 rows (read parquet) 25 / 36 3.9 254.3 1.1X -100 cols x 1000 rows (write parquet) 182 / 196 0.5 1819.5 0.2X -2500 cols x 40 rows (read in-mem) 280 / 315 0.4 2797.1 0.1X -2500 cols x 40 rows (exec in-mem) 606 / 638 0.2 6064.3 0.0X -2500 cols x 40 rows (read parquet) 836 / 843 0.1 8356.4 0.0X -2500 cols x 40 rows (write parquet) 490 / 522 0.2 4900.6 0.1X +1 cols x 100000 rows (read in-mem) 16 / 18 6.3 158.6 1.0X +1 cols x 100000 rows (exec in-mem) 17 / 19 6.0 166.7 1.0X +1 cols x 100000 rows (read parquet) 24 / 26 4.3 235.1 0.7X +1 cols x 100000 rows (write parquet) 81 / 85 1.2 811.3 0.2X +100 cols x 1000 rows (read in-mem) 17 / 19 6.0 166.2 1.0X +100 cols x 1000 rows (exec in-mem) 25 / 27 4.0 249.2 0.6X +100 cols x 1000 rows (read parquet) 23 / 25 4.4 226.0 0.7X +100 cols x 1000 rows (write parquet) 83 / 87 1.2 831.0 0.2X +2500 cols x 40 rows (read in-mem) 132 / 137 0.8 1322.9 0.1X +2500 cols x 40 rows (exec in-mem) 326 / 330 0.3 3260.6 0.0X +2500 cols x 40 rows (read parquet) 831 / 839 0.1 8305.8 0.0X +2500 cols x 40 rows (write parquet) 237 / 245 0.4 2372.6 0.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 22 / 35 4.6 216.0 1.0X -1 wide x 100000 rows (exec in-mem) 40 / 63 2.5 400.6 0.5X -1 wide x 100000 rows (read parquet) 93 / 134 1.1 933.9 0.2X -1 wide x 100000 rows (write parquet) 133 / 174 0.7 1334.3 0.2X -100 wide x 1000 rows (read in-mem) 22 / 44 4.5 223.3 1.0X -100 wide x 1000 rows (exec in-mem) 88 / 138 1.1 878.6 0.2X -100 wide x 1000 rows (read parquet) 117 / 186 0.9 1172.0 0.2X -100 wide x 1000 rows (write parquet) 144 / 174 0.7 1441.6 0.1X -2500 wide x 40 rows (read in-mem) 36 / 57 2.8 358.9 0.6X -2500 wide x 40 rows (exec in-mem) 1466 / 1507 0.1 14656.6 0.0X -2500 wide x 40 rows (read parquet) 690 / 802 0.1 6898.2 0.0X -2500 wide x 40 rows (write parquet) 197 / 207 0.5 1970.9 0.1X +1 wide x 100000 rows (read in-mem) 15 / 17 6.6 151.0 1.0X +1 wide x 100000 rows (exec in-mem) 20 / 22 5.1 196.6 0.8X +1 wide x 100000 rows (read parquet) 59 / 63 1.7 592.8 0.3X +1 wide x 100000 rows (write parquet) 81 / 87 1.2 814.6 0.2X +100 wide x 1000 rows (read in-mem) 21 / 25 4.8 208.7 0.7X +100 wide x 1000 rows (exec in-mem) 72 / 81 1.4 718.5 0.2X +100 wide x 1000 rows (read parquet) 75 / 85 1.3 752.6 0.2X +100 wide x 1000 rows (write parquet) 88 / 95 1.1 876.7 0.2X +2500 wide x 40 rows (read in-mem) 28 / 34 3.5 282.2 0.5X +2500 wide x 40 rows (exec in-mem) 1269 / 1284 0.1 12688.1 0.0X +2500 wide x 40 rows (read parquet) 549 / 578 0.2 5493.4 0.0X +2500 wide x 40 rows (write parquet) 96 / 104 1.0 959.1 0.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 deep x 100000 rows (read in-mem) 22 / 35 4.5 223.9 1.0X -1 deep x 100000 rows (exec in-mem) 28 / 52 3.6 280.6 0.8X -1 deep x 100000 rows (read parquet) 41 / 65 2.4 410.5 0.5X -1 deep x 100000 rows (write parquet) 163 / 173 0.6 1634.5 0.1X -100 deep x 1000 rows (read in-mem) 43 / 63 2.3 425.9 0.5X -100 deep x 1000 rows (exec in-mem) 232 / 280 0.4 2321.7 0.1X -100 deep x 1000 rows (read parquet) 1989 / 2281 0.1 19886.6 0.0X -100 deep x 1000 rows (write parquet) 144 / 184 0.7 1442.6 0.2X -250 deep x 400 rows (read in-mem) 68 / 95 1.5 680.9 0.3X -250 deep x 400 rows (exec in-mem) 1310 / 1403 0.1 13096.4 0.0X -250 deep x 400 rows (read parquet) 41477 / 41847 0.0 414766.8 0.0X -250 deep x 400 rows (write parquet) 243 / 272 0.4 2433.1 0.1X +1 deep x 100000 rows (read in-mem) 14 / 16 7.0 143.8 1.0X +1 deep x 100000 rows (exec in-mem) 17 / 19 5.9 169.7 0.8X +1 deep x 100000 rows (read parquet) 33 / 35 3.1 327.0 0.4X +1 deep x 100000 rows (write parquet) 79 / 84 1.3 786.9 0.2X +100 deep x 1000 rows (read in-mem) 21 / 24 4.7 211.3 0.7X +100 deep x 1000 rows (exec in-mem) 221 / 235 0.5 2214.5 0.1X +100 deep x 1000 rows (read parquet) 1928 / 1952 0.1 19277.1 0.0X +100 deep x 1000 rows (write parquet) 91 / 96 1.1 909.5 0.2X +250 deep x 400 rows (read in-mem) 57 / 61 1.8 567.1 0.3X +250 deep x 400 rows (exec in-mem) 1329 / 1385 0.1 13291.8 0.0X +250 deep x 400 rows (read parquet) 36563 / 36750 0.0 365630.2 0.0X +250 deep x 400 rows (write parquet) 126 / 130 0.8 1262.0 0.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 x 1 deep x 100000 rows (read in-mem) 23 / 36 4.4 229.8 1.0X -1 x 1 deep x 100000 rows (exec in-mem) 27 / 48 3.7 269.6 0.9X -1 x 1 deep x 100000 rows (read parquet) 25 / 33 4.0 247.5 0.9X -1 x 1 deep x 100000 rows (write parquet) 82 / 134 1.2 821.1 0.3X -128 x 8 deep x 1000 rows (read in-mem) 19 / 29 5.3 189.5 1.2X -128 x 8 deep x 1000 rows (exec in-mem) 144 / 165 0.7 1440.4 0.2X -128 x 8 deep x 1000 rows (read parquet) 117 / 159 0.9 1174.4 0.2X -128 x 8 deep x 1000 rows (write parquet) 135 / 162 0.7 1349.0 0.2X -1024 x 11 deep x 100 rows (read in-mem) 30 / 49 3.3 304.4 0.8X -1024 x 11 deep x 100 rows (exec in-mem) 1146 / 1183 0.1 11457.6 0.0X -1024 x 11 deep x 100 rows (read parquet) 712 / 758 0.1 7119.5 0.0X -1024 x 11 deep x 100 rows (write parquet) 104 / 143 1.0 1037.3 0.2X +1 x 1 deep x 100000 rows (read in-mem) 13 / 15 7.8 127.7 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 15 / 17 6.6 151.5 0.8X +1 x 1 deep x 100000 rows (read parquet) 20 / 23 5.0 198.3 0.6X +1 x 1 deep x 100000 rows (write parquet) 77 / 82 1.3 770.4 0.2X +128 x 8 deep x 1000 rows (read in-mem) 12 / 14 8.2 122.5 1.0X +128 x 8 deep x 1000 rows (exec in-mem) 124 / 140 0.8 1241.2 0.1X +128 x 8 deep x 1000 rows (read parquet) 69 / 74 1.4 693.9 0.2X +128 x 8 deep x 1000 rows (write parquet) 78 / 83 1.3 777.7 0.2X +1024 x 11 deep x 100 rows (read in-mem) 25 / 29 4.1 246.1 0.5X +1024 x 11 deep x 100 rows (exec in-mem) 1197 / 1223 0.1 11974.6 0.0X +1024 x 11 deep x 100 rows (read parquet) 426 / 433 0.2 4263.7 0.0X +1024 x 11 deep x 100 rows (write parquet) 91 / 98 1.1 913.5 0.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_66-internal-b17 on Linux 4.2.0-36-generic -Intel(R) Xeon(R) CPU E5-1650 v3 @ 3.50GHz wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 18 / 31 5.6 179.3 1.0X -1 wide x 100000 rows (exec in-mem) 31 / 47 3.2 310.2 0.6X -1 wide x 100000 rows (read parquet) 45 / 73 2.2 445.1 0.4X -1 wide x 100000 rows (write parquet) 109 / 140 0.9 1085.9 0.2X -100 wide x 1000 rows (read in-mem) 17 / 25 5.8 172.7 1.0X -100 wide x 1000 rows (exec in-mem) 18 / 22 5.4 184.6 1.0X -100 wide x 1000 rows (read parquet) 26 / 42 3.8 261.8 0.7X -100 wide x 1000 rows (write parquet) 150 / 164 0.7 1499.4 0.1X -2500 wide x 40 rows (read in-mem) 19 / 31 5.1 194.7 0.9X -2500 wide x 40 rows (exec in-mem) 19 / 24 5.3 188.5 1.0X -2500 wide x 40 rows (read parquet) 33 / 47 3.0 334.4 0.5X -2500 wide x 40 rows (write parquet) 153 / 164 0.7 1528.2 0.1X +1 wide x 100000 rows (read in-mem) 14 / 16 7.0 143.2 1.0X +1 wide x 100000 rows (exec in-mem) 17 / 19 5.9 170.9 0.8X +1 wide x 100000 rows (read parquet) 43 / 46 2.3 434.1 0.3X +1 wide x 100000 rows (write parquet) 78 / 83 1.3 777.6 0.2X +100 wide x 1000 rows (read in-mem) 11 / 13 9.0 111.5 1.3X +100 wide x 1000 rows (exec in-mem) 13 / 15 7.8 128.3 1.1X +100 wide x 1000 rows (read parquet) 24 / 27 4.1 245.0 0.6X +100 wide x 1000 rows (write parquet) 74 / 80 1.4 740.5 0.2X +2500 wide x 40 rows (read in-mem) 11 / 13 9.1 109.5 1.3X +2500 wide x 40 rows (exec in-mem) 13 / 15 7.7 129.4 1.1X +2500 wide x 40 rows (read parquet) 24 / 26 4.1 241.3 0.6X +2500 wide x 40 rows (write parquet) 75 / 81 1.3 751.8 0.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz + +wide map field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 16 / 18 6.2 162.6 1.0X +1 wide x 100000 rows (exec in-mem) 21 / 23 4.8 208.2 0.8X +1 wide x 100000 rows (read parquet) 54 / 59 1.8 543.6 0.3X +1 wide x 100000 rows (write parquet) 80 / 86 1.2 804.5 0.2X +100 wide x 1000 rows (read in-mem) 11 / 13 8.7 114.5 1.4X +100 wide x 1000 rows (exec in-mem) 14 / 16 7.0 143.5 1.1X +100 wide x 1000 rows (read parquet) 30 / 32 3.3 300.4 0.5X +100 wide x 1000 rows (write parquet) 75 / 80 1.3 749.9 0.2X +2500 wide x 40 rows (read in-mem) 13 / 15 7.8 128.1 1.3X +2500 wide x 40 rows (exec in-mem) 15 / 18 6.5 153.6 1.1X +2500 wide x 40 rows (read parquet) 30 / 33 3.3 304.4 0.5X +2500 wide x 40 rows (write parquet) 77 / 83 1.3 768.5 0.2X From e387c8ba86f89115eb2eabac070c215f451c5f0f Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 5 Sep 2016 11:17:37 +0800 Subject: [PATCH 1254/1470] [SPARK-17391][TEST][2.0] Fix Two Test Failures After Backport ### What changes were proposed in this pull request? In the latest branch 2.0, we have two test case failure due to backport. - test("ALTER VIEW AS should keep the previous table properties, comment, create_time, etc.") - test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") ### How was this patch tested? N/A Author: gatorsmile Closes #14951 from gatorsmile/fixTestFailure. --- .../apache/spark/sql/hive/execution/HiveExplainSuite.scala | 6 +++--- .../org/apache/spark/sql/hive/execution/SQLViewSuite.scala | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 98afd99a203ac..ec3328c0ae2d6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -77,7 +77,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto "src") } - test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") { + test("SPARK-17230: The EXPLAIN output of CTAS only shows the analyzed plan") { withTempView("jt") { val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) spark.read.json(rdd).createOrReplaceTempView("jt") @@ -98,8 +98,8 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } val physicalIndex = outputs.indexOf("== Physical Plan ==") - assert(!outputs.substring(physicalIndex).contains("Subquery"), - "Physical Plan should not contain Subquery since it's eliminated by optimizer") + assert(outputs.substring(physicalIndex).contains("SubqueryAlias"), + "Physical Plan should contain SubqueryAlias since the query should not be optimized") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index c6eeb8a0e4de4..cf63537482d61 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -232,12 +232,12 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val catalog = spark.sessionState.catalog val viewMeta = catalog.getTableMetadata(TableIdentifier("test_view")) - assert(viewMeta.properties("comment") == "test") + assert(viewMeta.comment == Some("test")) assert(viewMeta.properties("key") == "a") sql("ALTER VIEW test_view AS SELECT 3 AS i, 4 AS j") val updatedViewMeta = catalog.getTableMetadata(TableIdentifier("test_view")) - assert(updatedViewMeta.properties("comment") == "test") + assert(updatedViewMeta.comment == Some("test")) assert(updatedViewMeta.properties("key") == "a") assert(updatedViewMeta.createTime == viewMeta.createTime) // The view should be updated. From f92d87455214005e60b2d58aa814aaabd2ac9495 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 6 Sep 2016 10:45:54 +0800 Subject: [PATCH 1255/1470] [SPARK-17353][SPARK-16943][SPARK-16942][BACKPORT-2.0][SQL] Fix multiple bugs in CREATE TABLE LIKE command ### What changes were proposed in this pull request? This PR is to backport https://github.com/apache/spark/pull/14531. The existing `CREATE TABLE LIKE` command has multiple issues: - The generated table is non-empty when the source table is a data source table. The major reason is the data source table is using the table property `path` to store the location of table contents. Currently, we keep it unchanged. Thus, we still create the same table with the same location. - The table type of the generated table is `EXTERNAL` when the source table is an external Hive Serde table. Currently, we explicitly set it to `MANAGED`, but Hive is checking the table property `EXTERNAL` to decide whether the table is `EXTERNAL` or not. (See https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1407-L1408) Thus, the created table is still `EXTERNAL`. - When the source table is a `VIEW`, the metadata of the generated table contains the original view text and view original text. So far, this does not break anything, but it could cause something wrong in Hive. (For example, https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1405-L1406) - The issue regarding the table `comment`. To follow what Hive does, the table comment should be cleaned, but the column comments should be still kept. - The `INDEX` table is not supported. Thus, we should throw an exception in this case. - `owner` should not be retained. `ToHiveTable` set it [here](https://github.com/apache/spark/blob/e679bc3c1cd418ef0025d2ecbc547c9660cac433/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L793) no matter which value we set in `CatalogTable`. We set it to an empty string for avoiding the confusing output in Explain. - Add a support for temp tables - Like Hive, we should not copy the table properties from the source table to the created table, especially for the statistics-related properties, which could be wrong in the created table. - `unsupportedFeatures` should not be copied from the source table. The created table does not have these unsupported features. - When the type of source table is a view, the target table is using the default format of data source tables: `spark.sql.sources.default`. This PR is to fix the above issues. ### How was this patch tested? Improve the test coverage by adding more test cases Author: gatorsmile Closes #14946 from gatorsmile/createTableLike20. --- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../spark/sql/execution/command/tables.scala | 65 ++++- .../sql/hive/client/HiveClientImpl.scala | 4 +- .../sql/hive/execution/HiveDDLSuite.scala | 232 +++++++++++++++++- 4 files changed, 289 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 8db817cb152ad..24485134afc07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -263,8 +263,7 @@ class SessionCatalog( CatalogColumn( name = c.name, dataType = c.dataType.catalogString, - nullable = c.nullable, - comment = Option(c.name) + nullable = c.nullable ) }, properties = Map(), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 7e6a3527fd03f..615d9dc77da4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -33,8 +33,10 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, Catal import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -56,7 +58,12 @@ case class CreateHiveTableAsSelectLogicalPlan( } /** - * A command to create a table with the same definition of the given existing table. + * A command to create a MANAGED table with the same definition of the given existing table. + * In the target table definition, the table comment is always empty but the column comments + * are identical to the ones defined in the source table. + * + * The CatalogTable attributes copied from the source table are storage(inputFormat, outputFormat, + * serde, compressed, properties), schema, provider, partitionColumnNames, bucketSpec. * * The syntax of using this command in SQL is: * {{{ @@ -75,18 +82,54 @@ case class CreateTableLikeCommand( throw new AnalysisException( s"Source table in CREATE TABLE LIKE does not exist: '$sourceTable'") } - if (catalog.isTemporaryTable(sourceTable)) { - throw new AnalysisException( - s"Source table in CREATE TABLE LIKE cannot be temporary: '$sourceTable'") + val sourceTableDesc = catalog.getTableMetadata(sourceTable) + + if (DDLUtils.isDatasourceTable(sourceTableDesc) || + sourceTableDesc.tableType == CatalogTableType.VIEW) { + val outputSchema = + StructType(sourceTableDesc.schema.map { c => + val builder = new MetadataBuilder + c.comment.map(comment => builder.putString("comment", comment)) + StructField( + c.name, + CatalystSqlParser.parseDataType(c.dataType), + c.nullable, + metadata = builder.build()) + }) + val (schema, provider) = if (DDLUtils.isDatasourceTable(sourceTableDesc)) { + (DDLUtils.getSchemaFromTableProperties(sourceTableDesc).getOrElse(outputSchema), + sourceTableDesc.properties(CreateDataSourceTableUtils.DATASOURCE_PROVIDER)) + } else { // VIEW + (outputSchema, sparkSession.sessionState.conf.defaultDataSourceName) + } + createDataSourceTable( + sparkSession = sparkSession, + tableIdent = targetTable, + userSpecifiedSchema = Some(schema), + partitionColumns = Array.empty[String], + bucketSpec = None, + provider = provider, + options = Map("path" -> catalog.defaultTablePath(targetTable)), + isExternal = false) + } else { + val newStorage = + sourceTableDesc.storage.copy( + locationUri = None, + serdeProperties = sourceTableDesc.storage.serdeProperties) + val newTableDesc = + CatalogTable( + identifier = targetTable, + tableType = CatalogTableType.MANAGED, + storage = newStorage, + schema = sourceTableDesc.schema, + partitionColumnNames = sourceTableDesc.partitionColumnNames, + sortColumnNames = sourceTableDesc.sortColumnNames, + bucketColumnNames = sourceTableDesc.bucketColumnNames, + numBuckets = sourceTableDesc.numBuckets) + + catalog.createTable(newTableDesc, ifNotExists) } - val tableToCreate = catalog.getTableMetadata(sourceTable).copy( - identifier = targetTable, - tableType = CatalogTableType.MANAGED, - createTime = System.currentTimeMillis, - lastAccessTime = -1).withNewStorage(locationUri = None) - - catalog.createTable(tableToCreate, ifNotExists) Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index ac786767050f7..7db51d4b493aa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -412,7 +412,9 @@ private[hive] class HiveClientImpl( serdeProperties = Option(h.getTTable.getSd.getSerdeInfo.getParameters) .map(_.asScala.toMap).orNull ), - properties = properties.filter(kv => kv._1 != "comment"), + // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added + // in the function toHiveTable. + properties = properties.filter(kv => kv._1 != "comment" && kv._1 != "EXTERNAL"), comment = properties.get("comment"), viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 3cf3c6a9e29ca..676c08b5593c2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -24,8 +24,11 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.internal.config._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} -import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.command.{CreateDataSourceTableUtils, DDLUtils} +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ +import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -651,6 +654,233 @@ class HiveDDLSuite } } + + test("CREATE TABLE LIKE a temporary view") { + val sourceViewName = "tab1" + val targetTabName = "tab2" + withTempView(sourceViewName) { + withTable(targetTabName) { + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .createTempView(sourceViewName) + sql(s"CREATE TABLE $targetTabName LIKE $sourceViewName") + + val sourceTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(sourceViewName, None)) + val targetTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceTable, targetTable) + } + } + } + + test("CREATE TABLE LIKE a data source table") { + val sourceTabName = "tab1" + val targetTabName = "tab2" + withTable(sourceTabName, targetTabName) { + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .write.format("json").saveAsTable(sourceTabName) + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName") + + val sourceTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(sourceTabName, Some("default"))) + val targetTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(targetTabName, Some("default"))) + // The table type of the source table should be a Hive-managed data source table + assert(DDLUtils.isDatasourceTable(sourceTable)) + assert(sourceTable.tableType == CatalogTableType.MANAGED) + + checkCreateTableLike(sourceTable, targetTable) + } + } + + test("CREATE TABLE LIKE an external data source table") { + val sourceTabName = "tab1" + val targetTabName = "tab2" + withTable(sourceTabName, targetTabName) { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .write.format("parquet").save(path) + sql(s"CREATE TABLE $sourceTabName USING parquet OPTIONS (PATH '$path')") + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName") + + // The source table should be an external data source table + val sourceTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(sourceTabName, Some("default"))) + val targetTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + // The table type of the source table should be an external data source table + assert(DDLUtils.isDatasourceTable(sourceTable)) + assert(sourceTable.tableType == CatalogTableType.EXTERNAL) + + checkCreateTableLike(sourceTable, targetTable) + } + } + } + + test("CREATE TABLE LIKE a managed Hive serde table") { + val catalog = spark.sessionState.catalog + val sourceTabName = "tab1" + val targetTabName = "tab2" + withTable(sourceTabName, targetTabName) { + sql(s"CREATE TABLE $sourceTabName TBLPROPERTIES('prop1'='value1') AS SELECT 1 key, 'a'") + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName") + + val sourceTable = catalog.getTableMetadata(TableIdentifier(sourceTabName, Some("default"))) + assert(sourceTable.tableType == CatalogTableType.MANAGED) + assert(sourceTable.properties.get("prop1").nonEmpty) + val targetTable = catalog.getTableMetadata(TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceTable, targetTable) + } + } + + test("CREATE TABLE LIKE an external Hive serde table") { + val catalog = spark.sessionState.catalog + withTempDir { tmpDir => + val basePath = tmpDir.getCanonicalPath + val sourceTabName = "tab1" + val targetTabName = "tab2" + withTable(sourceTabName, targetTabName) { + assert(tmpDir.listFiles.isEmpty) + sql( + s""" + |CREATE EXTERNAL TABLE $sourceTabName (key INT comment 'test', value STRING) + |COMMENT 'Apache Spark' + |PARTITIONED BY (ds STRING, hr STRING) + |LOCATION '$basePath' + """.stripMargin) + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + sql( + s""" + |INSERT OVERWRITE TABLE $sourceTabName + |partition (ds='$ds',hr='$hr') + |SELECT 1, 'a' + """.stripMargin) + } + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName") + + val sourceTable = catalog.getTableMetadata(TableIdentifier(sourceTabName, Some("default"))) + assert(sourceTable.tableType == CatalogTableType.EXTERNAL) + assert(sourceTable.comment == Option("Apache Spark")) + val targetTable = catalog.getTableMetadata(TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceTable, targetTable) + } + } + } + + test("CREATE TABLE LIKE a view") { + val sourceTabName = "tab1" + val sourceViewName = "view" + val targetTabName = "tab2" + withTable(sourceTabName, targetTabName) { + withView(sourceViewName) { + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .write.format("json").saveAsTable(sourceTabName) + sql(s"CREATE VIEW $sourceViewName AS SELECT * FROM $sourceTabName") + sql(s"CREATE TABLE $targetTabName LIKE $sourceViewName") + + val sourceView = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(sourceViewName, Some("default"))) + // The original source should be a VIEW with an empty path + assert(sourceView.tableType == CatalogTableType.VIEW) + assert(sourceView.viewText.nonEmpty && sourceView.viewOriginalText.nonEmpty) + val targetTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceView, targetTable) + } + } + } + + private def getTablePath(table: CatalogTable): Option[String] = { + if (DDLUtils.isDatasourceTable(table)) { + new CaseInsensitiveMap(table.storage.serdeProperties).get("path") + } else { + table.storage.locationUri + } + } + + private def checkCreateTableLike(sourceTable: CatalogTable, targetTable: CatalogTable): Unit = { + // The created table should be a MANAGED table with empty view text and original text. + assert(targetTable.tableType == CatalogTableType.MANAGED, + "the created table must be a Hive managed table") + assert(targetTable.viewText.isEmpty && targetTable.viewOriginalText.isEmpty, + "the view text and original text in the created table must be empty") + assert(targetTable.comment.isEmpty, + "the comment in the created table must be empty") + assert(targetTable.unsupportedFeatures.isEmpty, + "the unsupportedFeatures in the create table must be empty") + + val metastoreGeneratedProperties = Seq( + "CreateTime", + "transient_lastDdlTime", + "grantTime", + "lastUpdateTime", + "last_modified_by", + "last_modified_time", + "Owner:", + "COLUMN_STATS_ACCURATE", + "numFiles", + "numRows", + "rawDataSize", + "totalSize", + "totalNumberFiles", + "maxFileSize", + "minFileSize" + ) + assert(targetTable.properties.filterKeys { key => + !metastoreGeneratedProperties.contains(key) && !key.startsWith(DATASOURCE_PREFIX) + }.isEmpty, + "the table properties of source tables should not be copied in the created table") + + if (DDLUtils.isDatasourceTable(sourceTable) || + sourceTable.tableType == CatalogTableType.VIEW) { + assert(DDLUtils.isDatasourceTable(targetTable), + "the target table should be a data source table") + } else { + assert(!DDLUtils.isDatasourceTable(targetTable), + "the target table should be a Hive serde table") + } + + if (sourceTable.tableType == CatalogTableType.VIEW) { + // Source table is a temporary/permanent view, which does not have a provider. The created + // target table uses the default data source format + assert(targetTable.properties(CreateDataSourceTableUtils.DATASOURCE_PROVIDER) == + spark.sessionState.conf.defaultDataSourceName) + } else if (DDLUtils.isDatasourceTable(sourceTable)) { + assert(targetTable.properties(CreateDataSourceTableUtils.DATASOURCE_PROVIDER) == + sourceTable.properties(CreateDataSourceTableUtils.DATASOURCE_PROVIDER)) + } + + val sourceTablePath = getTablePath(sourceTable) + val targetTablePath = getTablePath(targetTable) + assert(targetTablePath.nonEmpty, "target table path should not be empty") + assert(sourceTablePath != targetTablePath, + "source table/view path should be different from target table path") + + // The source table contents should not been seen in the target table. + assert(spark.table(sourceTable.identifier).count() != 0, "the source table should be nonempty") + assert(spark.table(targetTable.identifier).count() == 0, "the target table should be empty") + + // Their schema should be identical + checkAnswer( + sql(s"DESC ${sourceTable.identifier}").select("col_name", "data_type"), + sql(s"DESC ${targetTable.identifier}").select("col_name", "data_type")) + + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + // Check whether the new table can be inserted using the data from the original table + sql(s"INSERT INTO TABLE ${targetTable.identifier} SELECT * FROM ${sourceTable.identifier}") + } + + // After insertion, the data should be identical + checkAnswer( + sql(s"SELECT * FROM ${sourceTable.identifier}"), + sql(s"SELECT * FROM ${targetTable.identifier}")) + } + test("Analyze data source tables(LogicalRelation)") { withTable("t1") { withTempPath { dir => From 7b1aa2153bc6c8b753dba0710fe7b5d031158a34 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Tue, 6 Sep 2016 10:50:07 +0800 Subject: [PATCH 1256/1470] [SPARK-17369][SQL] MetastoreRelation toJSON throws AssertException due to missing otherCopyArgs ## What changes were proposed in this pull request? `TreeNode.toJSON` requires a subclass to explicitly override otherCopyArgs to include currying construction arguments, otherwise it reports AssertException telling that the construction argument values' count doesn't match the construction argument names' count. For class `MetastoreRelation`, it has a currying construction parameter `client: HiveClient`, but Spark forgets to add it to the list of otherCopyArgs. ## How was this patch tested? Unit tests. Author: Sean Zhong Closes #14928 from clockfly/metastore_relation_toJSON. (cherry picked from commit afb3d5d301d004fd748ad305b3d72066af4ebb6c) Signed-off-by: Wenchen Fan --- .../spark/sql/hive/MetastoreRelation.scala | 2 +- .../sql/hive/MetastoreRelationSuite.scala | 39 +++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 88d8d4b8cd316..10d71bef94d24 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -61,7 +61,7 @@ private[hive] case class MetastoreRelation( Objects.hashCode(databaseName, tableName, alias, output) } - override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: sparkSession :: Nil + override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: client :: sparkSession :: Nil private def toHiveColumn(c: CatalogColumn): FieldSchema = { new FieldSchema(c.name, c.dataType, c.comment.orNull) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala new file mode 100644 index 0000000000000..2f3055dcac4c5 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala @@ -0,0 +1,39 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.hive + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +class MetastoreRelationSuite extends SparkFunSuite { + test("makeCopy and toJSON should work") { + val table = CatalogTable( + identifier = TableIdentifier("test", Some("db")), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = StructType(StructField("a", IntegerType, true) :: Nil)) + val relation = MetastoreRelation("db", "test")(table, null, null) + + // No exception should be thrown + relation.makeCopy(Array("db", "test")) + // No exception should be thrown + relation.toJSON + } +} From dd27530c7a1f4670a8e28be37c81952eca456752 Mon Sep 17 00:00:00 2001 From: Yadong Qi Date: Tue, 6 Sep 2016 10:57:21 +0800 Subject: [PATCH 1257/1470] [SPARK-17358][SQL] Cached table(parquet/orc) should be shard between beelines ## What changes were proposed in this pull request? Cached table(parquet/orc) couldn't be shard between beelines, because the `sameResult` method used by `CacheManager` always return false(`sparkSession` are different) when compare two `HadoopFsRelation` in different beelines. So we make `sparkSession` a curry parameter. ## How was this patch tested? Beeline1 ``` 1: jdbc:hive2://localhost:10000> CACHE TABLE src_pqt; +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (5.143 seconds) 1: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt; +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | plan | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | == Physical Plan == InMemoryTableScan [key#49, value#50] +- InMemoryRelation [key#49, value#50], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt` +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ ``` Beeline2 ``` 0: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt; +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | plan | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | == Physical Plan == InMemoryTableScan [key#68, value#69] +- InMemoryRelation [key#68, value#69], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt` +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ ``` Author: Yadong Qi Closes #14913 from watermen/SPARK-17358. (cherry picked from commit 64e826f91eabb1a22d3d163d71fbb7b6d2185f25) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/datasources/DataSource.scala | 6 ++---- .../sql/execution/datasources/fileSourceInterfaces.scala | 4 ++-- .../sql/execution/datasources/FileSourceStrategySuite.scala | 3 ++- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 3 +-- 4 files changed, 7 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 0bf4a9c9025a5..014680d4e9b37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -350,13 +350,12 @@ case class DataSource( } HadoopFsRelation( - sparkSession, fileCatalog, partitionSchema = fileCatalog.partitionSpec().partitionColumns, dataSchema = dataSchema, bucketSpec = None, format, - options) + options)(sparkSession) // This is a non-streaming file based datasource. case (format: FileFormat, _) => @@ -414,13 +413,12 @@ case class DataSource( } HadoopFsRelation( - sparkSession, fileCatalog, partitionSchema = fileCatalog.partitionSpec().partitionColumns, dataSchema = dataSchema.asNullable, bucketSpec = bucketSpec, format, - caseInsensitiveOptions) + caseInsensitiveOptions)(sparkSession) case _ => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 4351bed99460a..ea614e55b540a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -134,13 +134,13 @@ abstract class OutputWriter { * @param options Configuration used when reading / writing data. */ case class HadoopFsRelation( - sparkSession: SparkSession, location: FileCatalog, partitionSchema: StructType, dataSchema: StructType, bucketSpec: Option[BucketSpec], fileFormat: FileFormat, - options: Map[String, String]) extends BaseRelation with FileRelation { + options: Map[String, String])(val sparkSession: SparkSession) + extends BaseRelation with FileRelation { override def sqlContext: SQLContext = sparkSession.sqlContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 7a24f214aa101..2f1edb0974927 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -507,7 +507,8 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi val bucketed = df.queryExecution.analyzed transform { case l @ LogicalRelation(r: HadoopFsRelation, _, _) => l.copy(relation = - r.copy(bucketSpec = Some(BucketSpec(numBuckets = buckets, "c1" :: Nil, Nil)))) + r.copy(bucketSpec = + Some(BucketSpec(numBuckets = buckets, "c1" :: Nil, Nil)))(r.sparkSession)) } Dataset.ofRows(spark, bucketed) } else { 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 789f94aff3036..bafb42277e334 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 @@ -300,13 +300,12 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } val relation = HadoopFsRelation( - sparkSession = sparkSession, location = fileCatalog, partitionSchema = partitionSchema, dataSchema = inferredSchema, bucketSpec = bucketSpec, fileFormat = defaultSource, - options = options) + options = options)(sparkSession = sparkSession) val created = LogicalRelation( relation, From f56b70fec2d31fd062320bb328c320e4eca72f1d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 5 Sep 2016 21:13:28 -0700 Subject: [PATCH 1258/1470] Revert "[SPARK-17369][SQL] MetastoreRelation toJSON throws AssertException due to missing otherCopyArgs" This reverts commit 7b1aa2153bc6c8b753dba0710fe7b5d031158a34. --- .../spark/sql/hive/MetastoreRelation.scala | 2 +- .../sql/hive/MetastoreRelationSuite.scala | 39 ------------------- 2 files changed, 1 insertion(+), 40 deletions(-) delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 10d71bef94d24..88d8d4b8cd316 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -61,7 +61,7 @@ private[hive] case class MetastoreRelation( Objects.hashCode(databaseName, tableName, alias, output) } - override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: client :: sparkSession :: Nil + override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: sparkSession :: Nil private def toHiveColumn(c: CatalogColumn): FieldSchema = { new FieldSchema(c.name, c.dataType, c.comment.orNull) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala deleted file mode 100644 index 2f3055dcac4c5..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala +++ /dev/null @@ -1,39 +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.hive - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} - -class MetastoreRelationSuite extends SparkFunSuite { - test("makeCopy and toJSON should work") { - val table = CatalogTable( - identifier = TableIdentifier("test", Some("db")), - tableType = CatalogTableType.VIEW, - storage = CatalogStorageFormat.empty, - schema = StructType(StructField("a", IntegerType, true) :: Nil)) - val relation = MetastoreRelation("db", "test")(table, null, null) - - // No exception should be thrown - relation.makeCopy(Array("db", "test")) - // No exception should be thrown - relation.toJSON - } -} From 286ccd6ba9e3927e8d445c2f56b6f1f5c77e11df Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Tue, 6 Sep 2016 15:42:52 +0800 Subject: [PATCH 1259/1470] [SPARK-17369][SQL][2.0] MetastoreRelation toJSON throws AssertException due to missing otherCopyArgs backport https://github.com/apache/spark/pull/14928 to 2.0 ## What changes were proposed in this pull request? `TreeNode.toJSON` requires a subclass to explicitly override otherCopyArgs to include currying construction arguments, otherwise it reports AssertException telling that the construction argument values' count doesn't match the construction argument names' count. For class `MetastoreRelation`, it has a currying construction parameter `client: HiveClient`, but Spark forgets to add it to the list of otherCopyArgs. ## How was this patch tested? Unit tests. Author: Sean Zhong Closes #14968 from clockfly/metastore_toJSON_fix_for_spark_2.0. --- .../spark/sql/hive/MetastoreRelation.scala | 2 +- .../sql/hive/MetastoreRelationSuite.scala | 38 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 88d8d4b8cd316..10d71bef94d24 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -61,7 +61,7 @@ private[hive] case class MetastoreRelation( Objects.hashCode(databaseName, tableName, alias, output) } - override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: sparkSession :: Nil + override protected def otherCopyArgs: Seq[AnyRef] = catalogTable :: client :: sparkSession :: Nil private def toHiveColumn(c: CatalogColumn): FieldSchema = { new FieldSchema(c.name, c.dataType, c.comment.orNull) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala new file mode 100644 index 0000000000000..eec60b440720f --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala @@ -0,0 +1,38 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.hive + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType} + +class MetastoreRelationSuite extends SparkFunSuite { + test("makeCopy and toJSON should work") { + val table = CatalogTable( + identifier = TableIdentifier("test", Some("db")), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = Seq.empty[CatalogColumn]) + val relation = MetastoreRelation("db", "test", None)(table, null, null) + + // No exception should be thrown + relation.makeCopy(Array("db", "test", None)) + // No exception should be thrown + relation.toJSON + } +} From c0f1f536dc75c9a1a932282046718228b95d2f70 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Tue, 6 Sep 2016 16:05:50 +0800 Subject: [PATCH 1260/1470] [SPARK-17356][SQL] Fix out of memory issue when generating JSON for TreeNode ## What changes were proposed in this pull request? class `org.apache.spark.sql.types.Metadata` is widely used in mllib to store some ml attributes. `Metadata` is commonly stored in `Alias` expression. ``` case class Alias(child: Expression, name: String)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, val explicitMetadata: Option[Metadata] = None, override val isGenerated: java.lang.Boolean = false) ``` The `Metadata` can take a big memory footprint since the number of attributes is big ( in scale of million). When `toJSON` is called on `Alias` expression, the `Metadata` will also be converted to a big JSON string. If a plan contains many such kind of `Alias` expressions, it may trigger out of memory error when `toJSON` is called, since converting all `Metadata` references to JSON will take huge memory. With this PR, we will skip scanning Metadata when doing JSON conversion. For a reproducer of the OOM, and analysis, please look at jira https://issues.apache.org/jira/browse/SPARK-17356. ## How was this patch tested? Existing tests. Author: Sean Zhong Closes #14915 from clockfly/json_oom. (cherry picked from commit 6f13aa7dfee12b1b301bd10a1050549008ecc67e) Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 4 +++- .../test/scala/org/apache/spark/sql/QueryTest.scala | 10 +++++++++- 2 files changed, 12 insertions(+), 2 deletions(-) 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 072445af4f41f..eeccba79e42ae 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 @@ -613,7 +613,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case s: String => JString(s) case u: UUID => JString(u.toString) case dt: DataType => dt.jsonValue - case m: Metadata => m.jsonValue + // SPARK-17356: In usage of mllib, Metadata may store a huge vector of data, transforming + // it to JSON may trigger OutOfMemoryError. + case m: Metadata => Metadata.empty.jsonValue case s: StorageLevel => ("useDisk" -> s.useDisk) ~ ("useMemory" -> s.useMemory) ~ ("useOffHeap" -> s.useOffHeap) ~ ("deserialized" -> s.deserialized) ~ ("replication" -> s.replication) 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 b2c051d11cf20..f96bd8cc8b5e2 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.streaming.MemoryPlan -import org.apache.spark.sql.types.ObjectType +import org.apache.spark.sql.types.{Metadata, ObjectType} abstract class QueryTest extends PlanTest { @@ -267,6 +267,14 @@ abstract class QueryTest extends PlanTest { val normalized1 = logicalPlan.transformAllExpressions { case udf: ScalaUDF => udf.copy(function = null) case gen: UserDefinedGenerator => gen.copy(function = null) + // After SPARK-17356: the JSON representation no longer has the Metadata. We need to remove + // the Metadata from the normalized plan so that we can compare this plan with the + // JSON-deserialzed plan. + case a @ Alias(child, name) if a.explicitMetadata.isDefined => + Alias(child, name)(a.exprId, a.qualifier, Some(Metadata.empty), a.isGenerated) + case a: AttributeReference if a.metadata != Metadata.empty => + AttributeReference(a.name, a.dataType, a.nullable, Metadata.empty)(a.exprId, a.qualifier, + a.isGenerated) } // RDDs/data are not serializable to JSON, so we need to collect LogicalPlans that contains From 95e44dca1d99ff7904c3c2e174f0f2123062ce3c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 6 Sep 2016 10:46:31 -0700 Subject: [PATCH 1261/1470] [SPARK-16922] [SPARK-17211] [SQL] make the address of values portable in LongToUnsafeRowMap ## What changes were proposed in this pull request? In LongToUnsafeRowMap, we use offset of a value as pointer, stored in a array also in the page for chained values. The offset is not portable, because Platform.LONG_ARRAY_OFFSET will be different with different JVM Heap size, then the deserialized LongToUnsafeRowMap will be corrupt. This PR will change to use portable address (without Platform.LONG_ARRAY_OFFSET). ## How was this patch tested? Added a test case with random generated keys, to improve the coverage. But this test is not a regression test, that could require a Spark cluster that have at least 32G heap in driver or executor. Author: Davies Liu Closes #14927 from davies/longmap. (cherry picked from commit f7e26d788757f917b32749856bb29feb7b4c2987) Signed-off-by: Davies Liu --- .../sql/execution/joins/HashedRelation.scala | 27 ++++++--- .../execution/joins/HashedRelationSuite.scala | 56 +++++++++++++++++++ 2 files changed, 75 insertions(+), 8 deletions(-) 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 08975733ff5d1..8821c0dea9ee5 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 @@ -447,10 +447,20 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ private def nextSlot(pos: Int): Int = (pos + 2) & mask + private[this] def toAddress(offset: Long, size: Int): Long = { + ((offset - Platform.LONG_ARRAY_OFFSET) << SIZE_BITS) | size + } + + private[this] def toOffset(address: Long): Long = { + (address >>> SIZE_BITS) + Platform.LONG_ARRAY_OFFSET + } + + private[this] def toSize(address: Long): Int = { + (address & SIZE_MASK).toInt + } + private def getRow(address: Long, resultRow: UnsafeRow): UnsafeRow = { - val offset = address >>> SIZE_BITS - val size = address & SIZE_MASK - resultRow.pointTo(page, offset, size.toInt) + resultRow.pointTo(page, toOffset(address), toSize(address)) resultRow } @@ -485,9 +495,9 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap var addr = address override def hasNext: Boolean = addr != 0 override def next(): UnsafeRow = { - val offset = addr >>> SIZE_BITS - val size = addr & SIZE_MASK - resultRow.pointTo(page, offset, size.toInt) + val offset = toOffset(addr) + val size = toSize(addr) + resultRow.pointTo(page, offset, size) addr = Platform.getLong(page, offset + size) resultRow } @@ -554,7 +564,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap Platform.putLong(page, cursor, 0) cursor += 8 numValues += 1 - updateIndex(key, (offset.toLong << SIZE_BITS) | row.getSizeInBytes) + updateIndex(key, toAddress(offset, row.getSizeInBytes)) } /** @@ -562,6 +572,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ private def updateIndex(key: Long, address: Long): Unit = { var pos = firstSlot(key) + assert(numKeys < array.length / 2) while (array(pos) != key && array(pos + 1) != 0) { pos = nextSlot(pos) } @@ -582,7 +593,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } else { // there are some values for this key, put the address in the front of them. - val pointer = (address >>> SIZE_BITS) + (address & SIZE_MASK) + val pointer = toOffset(address) + toSize(address) Platform.putLong(page, pointer, array(pos + 1)) array(pos + 1) = address } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 1196f5ec7b3a8..ede63fea9606f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.joins import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} +import scala.util.Random + import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} import org.apache.spark.serializer.KryoSerializer @@ -197,6 +199,60 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { } } + test("LongToUnsafeRowMap with random keys") { + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) + + val N = 1000000 + val rand = new Random + val keys = (0 to N).map(x => rand.nextLong()).toArray + + val map = new LongToUnsafeRowMap(taskMemoryManager, 10) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + + val os = new ByteArrayOutputStream() + val out = new ObjectOutputStream(os) + map.writeExternal(out) + out.flush() + val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) + val map2 = new LongToUnsafeRowMap(taskMemoryManager, 1) + map2.readExternal(in) + + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + val r = map2.get(k, row) + assert(r.hasNext) + var c = 0 + while (r.hasNext) { + val rr = r.next() + assert(rr.getLong(0) === k) + c += 1 + } + } + var i = 0 + while (i < N * 10) { + val k = rand.nextLong() + val r = map2.get(k, row) + if (r != null) { + assert(r.hasNext) + while (r.hasNext) { + assert(r.next().getLong(0) === k) + } + } + i += 1 + } + map.free() + } + test("Spark-14521") { val ser = new KryoSerializer( (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() From 534380484ac5f56bd3e14a8917a24ca6cccf198f Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Tue, 6 Sep 2016 10:48:53 -0700 Subject: [PATCH 1262/1470] [SPARK-16334] [BACKPORT] Reusing same dictionary column for decoding consecutive row groups shouldn't throw an error ## What changes were proposed in this pull request? Backports https://github.com/apache/spark/pull/14941 in 2.0. This patch fixes a bug in the vectorized parquet reader that's caused by re-using the same dictionary column vector while reading consecutive row groups. Specifically, this issue manifests for a certain distribution of dictionary/plain encoded data while we read/populate the underlying bit packed dictionary data into a column-vector based data structure. Manually tested on datasets provided by the community. Thanks to Chris Perluss and Keith Kraus for their invaluable help in tracking down this issue! Author: Sameer Agarwal Closes #14941 from sameeragarwal/parquet-exception-2. Author: Sameer Agarwal Closes #14944 from sameeragarwal/branch-2.0. --- .../parquet/VectorizedColumnReader.java | 54 +++++++++++++------ 1 file changed, 38 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 6c47dc09a8637..3141edd06879c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -221,15 +221,21 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, if (column.dataType() == DataTypes.IntegerType || DecimalType.is32BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - column.putInt(i, dictionary.decodeToInt(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putInt(i, dictionary.decodeToInt(dictionaryIds.getInt(i))); + } } } else if (column.dataType() == DataTypes.ByteType) { for (int i = rowId; i < rowId + num; ++i) { - column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getInt(i))); + } } } else if (column.dataType() == DataTypes.ShortType) { for (int i = rowId; i < rowId + num; ++i) { - column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getInt(i))); + } } } else { throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); @@ -240,7 +246,9 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, if (column.dataType() == DataTypes.LongType || DecimalType.is64BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - column.putLong(i, dictionary.decodeToLong(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putLong(i, dictionary.decodeToLong(dictionaryIds.getInt(i))); + } } } else { throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); @@ -249,21 +257,27 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, case FLOAT: for (int i = rowId; i < rowId + num; ++i) { - column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getInt(i))); + } } break; case DOUBLE: for (int i = rowId; i < rowId + num; ++i) { - column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getInt(i))); + if (!column.isNullAt(i)) { + column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getInt(i))); + } } break; case INT96: if (column.dataType() == DataTypes.TimestampType) { for (int i = rowId; i < rowId + num; ++i) { // TODO: Convert dictionary of Binaries to dictionary of Longs - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); + } } } else { throw new UnsupportedOperationException(); @@ -275,26 +289,34 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column, // and reuse it across batches. This should mean adding a ByteArray would just update // the length and offset. for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putByteArray(i, v.getBytes()); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putByteArray(i, v.getBytes()); + } } break; case FIXED_LEN_BYTE_ARRAY: // DecimalType written in the legacy mode if (DecimalType.is32BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); + } } } else if (DecimalType.is64BitDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); + } } } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { for (int i = rowId; i < rowId + num; ++i) { - Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); - column.putByteArray(i, v.getBytes()); + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + column.putByteArray(i, v.getBytes()); + } } } else { throw new UnsupportedOperationException(); From 130a80fd87bd8bb275f59af6d81c2e7dcc9707f9 Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Tue, 6 Sep 2016 22:13:25 +0100 Subject: [PATCH 1263/1470] [SPARK-17378][BUILD] Upgrade snappy-java to 1.1.2.6 ## What changes were proposed in this pull request? Upgrades the Snappy version to 1.1.2.6 from 1.1.2.4, release notes: https://github.com/xerial/snappy-java/blob/master/Milestone.md mention "Fix a bug in SnappyInputStream when reading compressed data that happened to have the same first byte with the stream magic header (#142)" ## How was this patch tested? Existing unit tests using the latest IBM Java 8 on Intel, Power and Z architectures (little and big-endian) Author: Adam Roberts Closes #14958 from a-roberts/master. (cherry picked from commit 6c08dbf683875ff1ba724447e0531f673bcff8ba) Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- pom.xml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index f5175f2483cc7..8c9e559790ba6 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -151,7 +151,7 @@ scalap-2.11.8.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snappy-0.2.jar -snappy-java-1.1.2.4.jar +snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar stax-api-1.0.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 49d35abe14da7..839e0840dba3d 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -158,7 +158,7 @@ scalap-2.11.8.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snappy-0.2.jar -snappy-java-1.1.2.4.jar +snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar stax-api-1.0-2.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index f7d5ab45b300f..ed84de79b1fdc 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -158,7 +158,7 @@ scalap-2.11.8.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snappy-0.2.jar -snappy-java-1.1.2.4.jar +snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar stax-api-1.0-2.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index f27e7bacc04de..6e7c9cb5c791a 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -166,7 +166,7 @@ scalap-2.11.8.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snappy-0.2.jar -snappy-java-1.1.2.4.jar +snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar stax-api-1.0-2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 934626a5723af..3da0860a10f94 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -167,7 +167,7 @@ scalap-2.11.8.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snappy-0.2.jar -snappy-java-1.1.2.4.jar +snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar spire_2.11-0.7.4.jar stax-api-1.0-2.jar diff --git a/pom.xml b/pom.xml index ded596336e094..ee0032a8582c5 100644 --- a/pom.xml +++ b/pom.xml @@ -161,7 +161,7 @@ 2.11 1.9.13 2.6.5 - 1.1.2.4 + 1.1.2.6 1.1.2 1.2.0-incubating 1.10 From 0ae97863781200ec96f89ad98e5d11bb1778fab0 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 6 Sep 2016 22:18:28 +0100 Subject: [PATCH 1264/1470] [SPARK-17299] TRIM/LTRIM/RTRIM should not strips characters other than spaces ## What changes were proposed in this pull request? TRIM/LTRIM/RTRIM should not strips characters other than spaces, we were trimming all chars small than ASCII 0x20(space) ## How was this patch tested? fixed existing tests. Author: Sandeep Singh Closes #14924 from techaddict/SPARK-17299. (cherry picked from commit 7775d9f224e22400c6c8c093652a383f4af66ee0) Signed-off-by: Sean Owen --- .../java/org/apache/spark/unsafe/types/UTF8String.java | 8 ++++---- .../org/apache/spark/unsafe/types/UTF8StringSuite.java | 10 ++++++++++ .../unsafe/types/UTF8StringPropertyCheckSuite.scala | 8 ++++---- 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 54a54569240c0..ce8351694308e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -465,9 +465,9 @@ public UTF8String trim() { int s = 0; int e = this.numBytes - 1; // skip all of the space (0x20) in the left side - while (s < this.numBytes && getByte(s) <= 0x20 && getByte(s) >= 0x00) s++; + while (s < this.numBytes && getByte(s) == 0x20) s++; // skip all of the space (0x20) in the right side - while (e >= 0 && getByte(e) <= 0x20 && getByte(e) >= 0x00) e--; + while (e >= 0 && getByte(e) == 0x20) e--; if (s > e) { // empty string return UTF8String.fromBytes(new byte[0]); @@ -479,7 +479,7 @@ public UTF8String trim() { public UTF8String trimLeft() { int s = 0; // skip all of the space (0x20) in the left side - while (s < this.numBytes && getByte(s) <= 0x20 && getByte(s) >= 0x00) s++; + while (s < this.numBytes && getByte(s) == 0x20) s++; if (s == this.numBytes) { // empty string return UTF8String.fromBytes(new byte[0]); @@ -491,7 +491,7 @@ public UTF8String trimLeft() { public UTF8String trimRight() { int e = numBytes - 1; // skip all of the space (0x20) in the right side - while (e >= 0 && getByte(e) <= 0x20 && getByte(e) >= 0x00) e--; + while (e >= 0 && getByte(e) == 0x20) e--; if (e < 0) { // empty string diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index d4160ad029eb3..7f03686dcec41 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -232,6 +232,16 @@ public void trims() { assertEquals(fromString("数据砖头"), fromString("数据砖头").trim()); assertEquals(fromString("数据砖头"), fromString("数据砖头").trimLeft()); assertEquals(fromString("数据砖头"), fromString("数据砖头").trimRight()); + + char[] charsLessThan0x20 = new char[10]; + Arrays.fill(charsLessThan0x20, (char)(' ' - 1)); + String stringStartingWithSpace = + new String(charsLessThan0x20) + "hello" + new String(charsLessThan0x20); + assertEquals(fromString(stringStartingWithSpace), fromString(stringStartingWithSpace).trim()); + assertEquals(fromString(stringStartingWithSpace), + fromString(stringStartingWithSpace).trimLeft()); + assertEquals(fromString(stringStartingWithSpace), + fromString(stringStartingWithSpace).trimRight()); } @Test diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala index 8a6b9e3e4536d..62d4176d00f94 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala @@ -98,7 +98,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty } } - val whitespaceChar: Gen[Char] = Gen.choose(0x00, 0x20).map(_.toChar) + val whitespaceChar: Gen[Char] = Gen.const(0x20.toChar) val whitespaceString: Gen[String] = Gen.listOf(whitespaceChar).map(_.mkString) val randomString: Gen[String] = Arbitrary.arbString.arbitrary @@ -107,7 +107,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty def lTrim(s: String): String = { var st = 0 val array: Array[Char] = s.toCharArray - while ((st < s.length) && (array(st) <= ' ')) { + while ((st < s.length) && (array(st) == ' ')) { st += 1 } if (st > 0) s.substring(st, s.length) else s @@ -115,7 +115,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty def rTrim(s: String): String = { var len = s.length val array: Array[Char] = s.toCharArray - while ((len > 0) && (array(len - 1) <= ' ')) { + while ((len > 0) && (array(len - 1) == ' ')) { len -= 1 } if (len < s.length) s.substring(0, len) else s @@ -127,7 +127,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty whitespaceString ) { (start: String, middle: String, end: String) => val s = start + middle + end - assert(toUTF8(s).trim() === toUTF8(s.trim())) + assert(toUTF8(s).trim() === toUTF8(rTrim(lTrim(s)))) assert(toUTF8(s).trimLeft() === toUTF8(lTrim(s))) assert(toUTF8(s).trimRight() === toUTF8(rTrim(s))) } From 015751421bc444e350ad15c6f2e8a52f2da5b6e9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 6 Sep 2016 15:07:28 -0700 Subject: [PATCH 1265/1470] [SPARK-17110] Fix StreamCorruptionException in BlockManager.getRemoteValues() ## What changes were proposed in this pull request? This patch fixes a `java.io.StreamCorruptedException` error affecting remote reads of cached values when certain data types are used. The problem stems from #11801 / SPARK-13990, a patch to have Spark automatically pick the "best" serializer when caching RDDs. If PySpark cached a PythonRDD, then this would be cached as an `RDD[Array[Byte]]` and the automatic serializer selection would pick KryoSerializer for replication and block transfer. However, the `getRemoteValues()` / `getRemoteBytes()` code path did not pass proper class tags in order to enable the same serializer to be used during deserialization, causing Java to be inappropriately used instead of Kryo, leading to the StreamCorruptedException. We already fixed a similar bug in #14311, which dealt with similar issues in block replication. Prior to that patch, it seems that we had no tests to ensure that block replication actually succeeded. Similarly, prior to this bug fix patch it looks like we had no tests to perform remote reads of cached data, which is why this bug was able to remain latent for so long. This patch addresses the bug by modifying `BlockManager`'s `get()` and `getRemoteValues()` methods to accept ClassTags, allowing the proper class tag to be threaded in the `getOrElseUpdate` code path (which is used by `rdd.iterator`) ## How was this patch tested? Extended the caching tests in `DistributedSuite` to exercise the `getRemoteValues` path, plus manual testing to verify that the PySpark bug reproduction in SPARK-17110 is fixed. Author: Josh Rosen Closes #14952 from JoshRosen/SPARK-17110. (cherry picked from commit 29cfab3f1524c5690be675d24dda0a9a1806d6ff) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/rdd/BlockRDD.scala | 2 +- .../spark/serializer/SerializerManager.scala | 7 ++++--- .../org/apache/spark/storage/BlockManager.scala | 15 ++++++++------- .../scala/org/apache/spark/DistributedSuite.scala | 6 ++++-- .../rdd/WriteAheadLogBackedBlockRDD.scala | 5 +++-- .../streaming/ReceivedBlockHandlerSuite.scala | 3 ++- 6 files changed, 22 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 63d1d1767a8cb..d47b75544fdba 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -44,7 +44,7 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient val blockIds: Array[Blo assertValid() val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId - blockManager.get(blockId) match { + blockManager.get[T](blockId) match { case Some(block) => block.data.asInstanceOf[Iterator[T]] case None => throw new Exception("Could not compute split, block " + blockId + " not found") diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 07caadbe40438..59bdc88464a84 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -147,11 +147,12 @@ private[spark] class SerializerManager(defaultSerializer: Serializer, conf: Spar * Deserializes an InputStream into an iterator of values and disposes of it when the end of * the iterator is reached. */ - def dataDeserializeStream[T: ClassTag]( + def dataDeserializeStream[T]( blockId: BlockId, - inputStream: InputStream): Iterator[T] = { + inputStream: InputStream) + (classTag: ClassTag[T]): Iterator[T] = { val stream = new BufferedInputStream(inputStream) - getSerializer(implicitly[ClassTag[T]]) + getSerializer(classTag) .newInstance() .deserializeStream(wrapForCompression(blockId, stream)) .asIterator.asInstanceOf[Iterator[T]] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a89ce857933ce..95107d325946c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -519,10 +519,11 @@ private[spark] class BlockManager( * * This does not acquire a lock on this block in this JVM. */ - private def getRemoteValues(blockId: BlockId): Option[BlockResult] = { + private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { + val ct = implicitly[ClassTag[T]] getRemoteBytes(blockId).map { data => val values = - serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true)) + serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true))(ct) new BlockResult(values, DataReadMethod.Network, data.size) } } @@ -601,13 +602,13 @@ private[spark] class BlockManager( * any locks if the block was fetched from a remote block manager. The read lock will * automatically be freed once the result's `data` iterator is fully consumed. */ - def get(blockId: BlockId): Option[BlockResult] = { + def get[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val local = getLocalValues(blockId) if (local.isDefined) { logInfo(s"Found block $blockId locally") return local } - val remote = getRemoteValues(blockId) + val remote = getRemoteValues[T](blockId) if (remote.isDefined) { logInfo(s"Found block $blockId remotely") return remote @@ -659,7 +660,7 @@ private[spark] class BlockManager( makeIterator: () => Iterator[T]): Either[BlockResult, Iterator[T]] = { // Attempt to read the block from local or remote storage. If it's present, then we don't need // to go through the local-get-or-put path. - get(blockId) match { + get[T](blockId)(classTag) match { case Some(block) => return Left(block) case _ => @@ -1204,8 +1205,8 @@ private[spark] class BlockManager( /** * Read a block consisting of a single object. */ - def getSingle(blockId: BlockId): Option[Any] = { - get(blockId).map(_.data.next()) + def getSingle[T: ClassTag](blockId: BlockId): Option[T] = { + get[T](blockId).map(_.data.next().asInstanceOf[T]) } /** diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 4ee0e00fde506..4e36adc8baf3f 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -170,10 +170,12 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex blockManager.master.getLocations(blockId).foreach { cmId => val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, blockId.toString) - val deserialized = serializerManager.dataDeserializeStream[Int](blockId, - new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream()).toList + val deserialized = serializerManager.dataDeserializeStream(blockId, + new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream())(data.elementClassTag).toList assert(deserialized === (1 to 100).toList) } + // This will exercise the getRemoteBytes / getRemoteValues code paths: + assert(blockIds.flatMap(id => blockManager.get[Int](id).get.data).toSet === (1 to 1000).toSet) } Seq( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 53fccd8d5e6ed..0b2ec298132ad 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -120,7 +120,7 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( val blockId = partition.blockId def getBlockFromBlockManager(): Option[Iterator[T]] = { - blockManager.get(blockId).map(_.data.asInstanceOf[Iterator[T]]) + blockManager.get[T](blockId).map(_.data.asInstanceOf[Iterator[T]]) } def getBlockFromWriteAheadLog(): Iterator[T] = { @@ -163,7 +163,8 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( dataRead.rewind() } serializerManager - .dataDeserializeStream(blockId, new ChunkedByteBuffer(dataRead).toInputStream()) + .dataDeserializeStream( + blockId, new ChunkedByteBuffer(dataRead).toInputStream())(elementClassTag) .asInstanceOf[Iterator[T]] } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index e97427991bf92..5d362925a8746 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.postfixOps +import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, Matchers} @@ -160,7 +161,7 @@ class ReceivedBlockHandlerSuite val bytes = reader.read(fileSegment) reader.close() serializerManager.dataDeserializeStream( - generateBlockId(), new ChunkedByteBuffer(bytes).toInputStream()).toList + generateBlockId(), new ChunkedByteBuffer(bytes).toInputStream())(ClassTag.Any).toList } loggedData shouldEqual data } From f3cfce09274741cc04bf2f00e873b3b64976b6d5 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 6 Sep 2016 16:49:06 -0700 Subject: [PATCH 1266/1470] [SPARK-17316][CORE] Fix the 'ask' type parameter in 'removeExecutor' ## What changes were proposed in this pull request? Fix the 'ask' type parameter in 'removeExecutor' to eliminate a lot of error logs `Cannot cast java.lang.Boolean to scala.runtime.Nothing$` ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #14983 from zsxwing/SPARK-17316-3. (cherry picked from commit 175b4344112b376cbbbd05265125ed0e1b87d507) Signed-off-by: Shixiong Zhu --- .../spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 2db3a3bb81f61..5a74ddd94126f 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 @@ -412,7 +412,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp */ protected def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { // Only log the failure since we don't care about the result. - driverEndpoint.ask(RemoveExecutor(executorId, reason)).onFailure { case t => + driverEndpoint.ask[Boolean](RemoveExecutor(executorId, reason)).onFailure { case t => logError(t.getMessage, t) }(ThreadUtils.sameThread) } From a23d4065c5705b805c69e569ea177167d44b5244 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 6 Sep 2016 10:36:00 +0800 Subject: [PATCH 1267/1470] [SPARK-17279][SQL] better error message for exceptions during ScalaUDF execution ## What changes were proposed in this pull request? If `ScalaUDF` throws exceptions during executing user code, sometimes it's hard for users to figure out what's wrong, especially when they use Spark shell. An example ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 12 in stage 325.0 failed 4 times, most recent failure: Lost task 12.3 in stage 325.0 (TID 35622, 10.0.207.202): java.lang.NullPointerException at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(:40) at line8414e872fb8b42aba390efc153d1611a12.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$2.apply(:40) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) ... ``` We should catch these exceptions and rethrow them with better error message, to say that the exception is happened in scala udf. This PR also does some clean up for `ScalaUDF` and add a unit test suite for it. ## How was this patch tested? the new test suite Author: Wenchen Fan Closes #14850 from cloud-fan/npe. (cherry picked from commit 8d08f43d09157b98e559c0be6ce6fd571a35e0d1) Signed-off-by: Wenchen Fan --- .../spark/ml/recommendation/ALSSuite.scala | 16 +++---- .../sql/catalyst/expressions/ScalaUDF.scala | 44 +++++++++++------ .../catalyst/expressions/ScalaUDFSuite.scala | 48 +++++++++++++++++++ 3 files changed, 86 insertions(+), 22 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index e8ed50acf877c..d0aa2cdfe0fd1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -510,18 +510,18 @@ class ALSSuite (1, 1L, 1d, 0, 0L, 0d, 5.0) ).toDF("user", "user_big", "user_small", "item", "item_big", "item_small", "rating") withClue("fit should fail when ids exceed integer range. ") { - assert(intercept[IllegalArgumentException] { + assert(intercept[SparkException] { als.fit(df.select(df("user_big").as("user"), df("item"), df("rating"))) - }.getMessage.contains("was out of Integer range")) - assert(intercept[IllegalArgumentException] { + }.getCause.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { als.fit(df.select(df("user_small").as("user"), df("item"), df("rating"))) - }.getMessage.contains("was out of Integer range")) - assert(intercept[IllegalArgumentException] { + }.getCause.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { als.fit(df.select(df("item_big").as("item"), df("user"), df("rating"))) - }.getMessage.contains("was out of Integer range")) - assert(intercept[IllegalArgumentException] { + }.getCause.getMessage.contains("was out of Integer range")) + assert(intercept[SparkException] { als.fit(df.select(df("item_small").as("item"), df("user"), df("rating"))) - }.getMessage.contains("was out of Integer range")) + }.getCause.getMessage.contains("was out of Integer range")) } withClue("transform should fail when ids exceed integer range. ") { val model = als.fit(df) 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 21390644bc0b6..6cfdea9fdf9c5 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,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.types.DataType @@ -994,20 +995,15 @@ case class ScalaUDF( ctx: CodegenContext, ev: ExprCode): ExprCode = { - ctx.references += this - - val scalaUDFClassName = classOf[ScalaUDF].getName + val scalaUDF = ctx.addReferenceObj("scalaUDF", this) val converterClassName = classOf[Any => Any].getName val typeConvertersClassName = CatalystTypeConverters.getClass.getName + ".MODULE$" - val expressionClassName = classOf[Expression].getName // Generate codes used to convert the returned value of user-defined functions to Catalyst type val catalystConverterTerm = ctx.freshName("catalystConverter") - val catalystConverterTermIdx = ctx.references.size - 1 ctx.addMutableState(converterClassName, catalystConverterTerm, s"this.$catalystConverterTerm = ($converterClassName)$typeConvertersClassName" + - s".createToCatalystConverter((($scalaUDFClassName)references" + - s"[$catalystConverterTermIdx]).dataType());") + s".createToCatalystConverter($scalaUDF.dataType());") val resultTerm = ctx.freshName("result") @@ -1019,10 +1015,8 @@ case class ScalaUDF( val funcClassName = s"scala.Function${children.size}" val funcTerm = ctx.freshName("udf") - val funcExpressionIdx = ctx.references.size - 1 ctx.addMutableState(funcClassName, funcTerm, - s"this.$funcTerm = ($funcClassName)((($scalaUDFClassName)references" + - s"[$funcExpressionIdx]).userDefinedFunc());") + s"this.$funcTerm = ($funcClassName)$scalaUDF.userDefinedFunc();") // codegen for children expressions val evals = children.map(_.genCode(ctx)) @@ -1039,9 +1033,16 @@ case class ScalaUDF( (convert, argTerm) }.unzip - val callFunc = s"${ctx.boxedType(dataType)} $resultTerm = " + - s"(${ctx.boxedType(dataType)})${catalystConverterTerm}" + - s".apply($funcTerm.apply(${funcArguments.mkString(", ")}));" + val getFuncResult = s"$funcTerm.apply(${funcArguments.mkString(", ")})" + val callFunc = + s""" + ${ctx.boxedType(dataType)} $resultTerm = null; + try { + $resultTerm = (${ctx.boxedType(dataType)})$catalystConverterTerm.apply($getFuncResult); + } catch (Exception e) { + throw new org.apache.spark.SparkException($scalaUDF.udfErrorMessage(), e); + } + """ ev.copy(code = s""" $evalCode @@ -1057,5 +1058,20 @@ case class ScalaUDF( private[this] val converter = CatalystTypeConverters.createToCatalystConverter(dataType) - override def eval(input: InternalRow): Any = converter(f(input)) + lazy val udfErrorMessage = { + val funcCls = function.getClass.getSimpleName + val inputTypes = children.map(_.dataType.simpleString).mkString(", ") + s"Failed to execute user defined function($funcCls: ($inputTypes) => ${dataType.simpleString})" + } + + override def eval(input: InternalRow): Any = { + val result = try { + f(input) + } catch { + case e: Exception => + throw new SparkException(udfErrorMessage, e) + } + + converter(result) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala new file mode 100644 index 0000000000000..7e45028653e36 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -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. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.sql.types.{IntegerType, StringType} + +class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("basic") { + val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil) + checkEvaluation(intUdf, 2) + + val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil) + checkEvaluation(stringUdf, "ax") + } + + test("better error message for NPE") { + val udf = ScalaUDF( + (s: String) => s.toLowerCase, + StringType, + Literal.create(null, StringType) :: Nil) + + val e1 = intercept[SparkException](udf.eval()) + assert(e1.getMessage.contains("Failed to execute user defined function")) + + val e2 = intercept[SparkException] { + checkEvalutionWithUnsafeProjection(udf, null) + } + assert(e2.getMessage.contains("Failed to execute user defined function")) + } + +} From 796577b43d3df94f5d3a8e4baeb0aa03fbbb3f21 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 6 Sep 2016 19:34:11 -0700 Subject: [PATCH 1268/1470] [SPARK-17372][SQL][STREAMING] Avoid serialization issues by using Arrays to save file names in FileStreamSource ## What changes were proposed in this pull request? When we create a filestream on a directory that has partitioned subdirs (i.e. dir/x=y/), then ListingFileCatalog.allFiles returns the files in the dir as Seq[String] which internally is a Stream[String]. This is because of this [line](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L93), where a LinkedHashSet.values.toSeq returns Stream. Then when the [FileStreamSource](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L79) filters this Stream[String] to remove the seen files, it creates a new Stream[String], which has a filter function that has a $outer reference to the FileStreamSource (in Scala 2.10). Trying to serialize this Stream[String] causes NotSerializableException. This will happened even if there is just one file in the dir. Its important to note that this behavior is different in Scala 2.11. There is no $outer reference to FileStreamSource, so it does not throw NotSerializableException. However, with a large sequence of files (tested with 10000 files), it throws StackOverflowError. This is because how Stream class is implemented. Its basically like a linked list, and attempting to serialize a long Stream requires *recursively* going through linked list, thus resulting in StackOverflowError. In short, across both Scala 2.10 and 2.11, serialization fails when both the following conditions are true. - file stream defined on a partitioned directory - directory has 10k+ files The right solution is to convert the seq to an array before writing to the log. This PR implements this fix in two ways. - Changing all uses for HDFSMetadataLog to ensure Array is used instead of Seq - Added a `require` in HDFSMetadataLog such that it is never used with type Seq ## How was this patch tested? Added unit test that test that ensures the file stream source can handle with 10000 files. This tests fails in both Scala 2.10 and 2.11 with different failures as indicated above. Author: Tathagata Das Closes #14987 from tdas/SPARK-17372. (cherry picked from commit eb1ab88a86ce35f3d6ba03b3a798099fbcf6b3fc) Signed-off-by: Tathagata Das --- .../streaming/FileStreamSinkLog.scala | 12 +++--- .../streaming/FileStreamSource.scala | 4 +- .../execution/streaming/HDFSMetadataLog.scala | 4 ++ .../execution/streaming/StreamExecution.scala | 3 ++ .../streaming/FileStreamSinkLogSuite.scala | 18 ++++---- .../sql/streaming/FileStreamSourceSuite.scala | 42 ++++++++++++++++++- 6 files changed, 65 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala index 4254df44c97a6..7520163522027 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala @@ -80,7 +80,7 @@ object SinkFileStatus { * (drops the deleted files). */ class FileStreamSinkLog(sparkSession: SparkSession, path: String) - extends HDFSMetadataLog[Seq[SinkFileStatus]](sparkSession, path) { + extends HDFSMetadataLog[Array[SinkFileStatus]](sparkSession, path) { import FileStreamSinkLog._ @@ -123,11 +123,11 @@ class FileStreamSinkLog(sparkSession: SparkSession, path: String) } } - override def serialize(logData: Seq[SinkFileStatus]): Array[Byte] = { + override def serialize(logData: Array[SinkFileStatus]): Array[Byte] = { (VERSION +: logData.map(write(_))).mkString("\n").getBytes(UTF_8) } - override def deserialize(bytes: Array[Byte]): Seq[SinkFileStatus] = { + override def deserialize(bytes: Array[Byte]): Array[SinkFileStatus] = { val lines = new String(bytes, UTF_8).split("\n") if (lines.length == 0) { throw new IllegalStateException("Incomplete log file") @@ -136,10 +136,10 @@ class FileStreamSinkLog(sparkSession: SparkSession, path: String) if (version != VERSION) { throw new IllegalStateException(s"Unknown log version: ${version}") } - lines.toSeq.slice(1, lines.length).map(read[SinkFileStatus](_)) + lines.slice(1, lines.length).map(read[SinkFileStatus](_)) } - override def add(batchId: Long, logs: Seq[SinkFileStatus]): Boolean = { + override def add(batchId: Long, logs: Array[SinkFileStatus]): Boolean = { if (isCompactionBatch(batchId, compactInterval)) { compact(batchId, logs) } else { @@ -186,7 +186,7 @@ class FileStreamSinkLog(sparkSession: SparkSession, path: String) private def compact(batchId: Long, logs: Seq[SinkFileStatus]): Boolean = { val validBatches = getValidBatchesBeforeCompactionBatch(batchId, compactInterval) val allLogs = validBatches.flatMap(batchId => get(batchId)).flatten ++ logs - if (super.add(batchId, compactLogs(allLogs))) { + if (super.add(batchId, compactLogs(allLogs).toArray)) { if (isDeletingExpiredLog) { deleteExpiredLog(batchId) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index e8b969b5e0a55..42fb454c2d158 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -49,7 +49,7 @@ class FileStreamSource( fs.makeQualified(new Path(path)) // can contains glob patterns } - private val metadataLog = new HDFSMetadataLog[Seq[FileEntry]](sparkSession, metadataPath) + private val metadataLog = new HDFSMetadataLog[Array[FileEntry]](sparkSession, metadataPath) private var maxBatchId = metadataLog.getLatest().map(_._1).getOrElse(-1L) @@ -98,7 +98,7 @@ class FileStreamSource( if (batchFiles.nonEmpty) { maxBatchId += 1 - metadataLog.add(maxBatchId, batchFiles) + metadataLog.add(maxBatchId, batchFiles.toArray) logInfo(s"Max batch id increased to $maxBatchId with ${batchFiles.size} new files") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 127ece9ab0e56..39a0f3341389c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -49,6 +49,10 @@ import org.apache.spark.util.UninterruptibleThread class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) extends MetadataLog[T] with Logging { + // Avoid serializing generic sequences, see SPARK-17372 + require(implicitly[ClassTag[T]].runtimeClass != classOf[Seq[_]], + "Should not create a log with type Seq, use Arrays instead - see SPARK-17372") + import HDFSMetadataLog._ val metadataPath = new Path(path) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 4d05af0b60358..5e1e5eeb50936 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -407,6 +407,9 @@ class StreamExecution( awaitBatchLock.lock() try { awaitBatchLockCondition.await(100, TimeUnit.MILLISECONDS) + if (streamDeathCause != null) { + throw streamDeathCause + } } finally { awaitBatchLock.unlock() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala index 39fd1f0cd37bb..26f8b98cb38a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -98,7 +98,7 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { test("serialize") { withFileStreamSinkLog { sinkLog => - val logs = Seq( + val logs = Array( SinkFileStatus( path = "/a/b/x", size = 100L, @@ -132,7 +132,7 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { // scalastyle:on assert(expected === new String(sinkLog.serialize(logs), UTF_8)) - assert(FileStreamSinkLog.VERSION === new String(sinkLog.serialize(Nil), UTF_8)) + assert(FileStreamSinkLog.VERSION === new String(sinkLog.serialize(Array()), UTF_8)) } } @@ -196,7 +196,7 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { for (batchId <- 0 to 10) { sinkLog.add( batchId, - Seq(newFakeSinkFileStatus("/a/b/" + batchId, FileStreamSinkLog.ADD_ACTION))) + Array(newFakeSinkFileStatus("/a/b/" + batchId, FileStreamSinkLog.ADD_ACTION))) val expectedFiles = (0 to batchId).map { id => newFakeSinkFileStatus("/a/b/" + id, FileStreamSinkLog.ADD_ACTION) } @@ -230,17 +230,17 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { }.toSet } - sinkLog.add(0, Seq(newFakeSinkFileStatus("/a/b/0", FileStreamSinkLog.ADD_ACTION))) + sinkLog.add(0, Array(newFakeSinkFileStatus("/a/b/0", FileStreamSinkLog.ADD_ACTION))) assert(Set("0") === listBatchFiles()) - sinkLog.add(1, Seq(newFakeSinkFileStatus("/a/b/1", FileStreamSinkLog.ADD_ACTION))) + sinkLog.add(1, Array(newFakeSinkFileStatus("/a/b/1", FileStreamSinkLog.ADD_ACTION))) assert(Set("0", "1") === listBatchFiles()) - sinkLog.add(2, Seq(newFakeSinkFileStatus("/a/b/2", FileStreamSinkLog.ADD_ACTION))) + sinkLog.add(2, Array(newFakeSinkFileStatus("/a/b/2", FileStreamSinkLog.ADD_ACTION))) assert(Set("2.compact") === listBatchFiles()) - sinkLog.add(3, Seq(newFakeSinkFileStatus("/a/b/3", FileStreamSinkLog.ADD_ACTION))) + sinkLog.add(3, Array(newFakeSinkFileStatus("/a/b/3", FileStreamSinkLog.ADD_ACTION))) assert(Set("2.compact", "3") === listBatchFiles()) - sinkLog.add(4, Seq(newFakeSinkFileStatus("/a/b/4", FileStreamSinkLog.ADD_ACTION))) + sinkLog.add(4, Array(newFakeSinkFileStatus("/a/b/4", FileStreamSinkLog.ADD_ACTION))) assert(Set("2.compact", "3", "4") === listBatchFiles()) - sinkLog.add(5, Seq(newFakeSinkFileStatus("/a/b/5", FileStreamSinkLog.ADD_ACTION))) + sinkLog.add(5, Array(newFakeSinkFileStatus("/a/b/5", FileStreamSinkLog.ADD_ACTION))) assert(Set("5.compact") === listBatchFiles()) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 03222b4a49c6c..886f7be59db93 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.streaming import java.io.File -import java.util.UUID + +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ @@ -142,6 +144,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest { import testImplicits._ + override val streamingTimeout = 20.seconds + /** Use `format` and `path` to create FileStreamSource via DataFrameReader */ private def createFileStreamSource( format: String, @@ -761,6 +765,42 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } } + + test("SPARK-17372 - write file names to WAL as Array[String]") { + // Note: If this test takes longer than the timeout, then its likely that this is actually + // running a Spark job with 10000 tasks. This test tries to avoid that by + // 1. Setting the threshold for parallel file listing to very high + // 2. Using a query that should use constant folding to eliminate reading of the files + + val numFiles = 10000 + + // This is to avoid running a spark job to list of files in parallel + // by the ListingFileCatalog. + spark.sessionState.conf.setConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD, numFiles * 2) + + withTempDirs { case (root, tmp) => + val src = new File(root, "a=1") + src.mkdirs() + + (1 to numFiles).map { _.toString }.foreach { i => + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + stringToFile(finalFile, i) + } + assert(src.listFiles().size === numFiles) + + val files = spark.readStream.text(root.getCanonicalPath).as[String] + + // Note this query will use constant folding to eliminate the file scan. + // This is to avoid actually running a Spark job with 10000 tasks + val df = files.filter("1 == 0").groupBy().count() + + testStream(df, InternalOutputModes.Complete)( + AddTextFileData("0", src, tmp), + CheckAnswer(0) + ) + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { From ee6301a88e3b109398cec9bc470b5a88f72654dd Mon Sep 17 00:00:00 2001 From: Clark Fitzgerald Date: Tue, 6 Sep 2016 23:40:37 -0700 Subject: [PATCH 1269/1470] [SPARK-16785] R dapply doesn't return array or raw columns Fixed bug in `dapplyCollect` by changing the `compute` function of `worker.R` to explicitly handle raw (binary) vectors. cc shivaram Unit tests Author: Clark Fitzgerald Closes #14783 from clarkfitzg/SPARK-16785. (cherry picked from commit 9fccde4ff80fb0fd65a9e90eb3337965e4349de4) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/SQLContext.R | 4 ++++ R/pkg/R/utils.R | 15 ++++++++++++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 21 ++++++++++++++++++++ R/pkg/inst/tests/testthat/test_utils.R | 24 +++++++++++++++++++++++ R/pkg/inst/worker/worker.R | 9 ++++++++- 5 files changed, 72 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 783df53c12ca2..ce531c3f88863 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -202,7 +202,10 @@ getDefaultSqlSource <- function() { # TODO(davies): support sampling and infer type from NA createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { sparkSession <- getSparkSession() + if (is.data.frame(data)) { + # Convert data into a list of rows. Each row is a list. + # get the names of columns, they will be put into RDD if (is.null(schema)) { schema <- names(data) @@ -227,6 +230,7 @@ createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0) { args <- list(FUN = list, SIMPLIFY = FALSE, USE.NAMES = FALSE) data <- do.call(mapply, append(args, data)) } + if (is.list(data)) { sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) rdd <- parallelize(sc, data) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 2809ce5d376a5..248c57532b6cf 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -697,3 +697,18 @@ isMasterLocal <- function(master) { isSparkRShell <- function() { grepl(".*shell\\.R$", Sys.getenv("R_PROFILE_USER"), perl = TRUE) } + +# rbind a list of rows with raw (binary) columns +# +# @param inputData a list of rows, with each row a list +# @return data.frame with raw columns as lists +rbindRaws <- function(inputData){ + row1 <- inputData[[1]] + rawcolumns <- ("raw" == sapply(row1, class)) + + listmatrix <- do.call(rbind, inputData) + # A dataframe with all list columns + out <- as.data.frame(listmatrix) + out[!rawcolumns] <- lapply(out[!rawcolumns], unlist) + out +} diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 78a3754e776e1..dddc15f58cc35 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2262,6 +2262,27 @@ test_that("dapply() and dapplyCollect() on a DataFrame", { expect_identical(expected, result) }) +test_that("dapplyCollect() on DataFrame with a binary column", { + + df <- data.frame(key = 1:3) + df$bytes <- lapply(df$key, serialize, connection = NULL) + + df_spark <- createDataFrame(df) + + result1 <- collect(df_spark) + expect_identical(df, result1) + + result2 <- dapplyCollect(df_spark, function(x) x) + expect_identical(df, result2) + + # A data.frame with a single column of bytes + scb <- subset(df, select = "bytes") + scb_spark <- createDataFrame(scb) + result <- dapplyCollect(scb_spark, function(x) x) + expect_identical(scb, result) + +}) + test_that("repartition by columns on DataFrame", { df <- createDataFrame( list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 42783fde5f25b..dd5001b1599ae 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -182,3 +182,27 @@ test_that("overrideEnvs", { expect_equal(config[["param_only"]], "blah") expect_equal(config[["config_only"]], "ok") }) + +test_that("rbindRaws", { + + # Mixed Column types + r <- serialize(1:5, connection = NULL) + r1 <- serialize(1, connection = NULL) + r2 <- serialize(letters, connection = NULL) + r3 <- serialize(1:10, connection = NULL) + inputData <- list(list(1L, r1, "a", r), list(2L, r2, "b", r), + list(3L, r3, "c", r)) + expected <- data.frame(V1 = 1:3) + expected$V2 <- list(r1, r2, r3) + expected$V3 <- c("a", "b", "c") + expected$V4 <- list(r, r, r) + result <- rbindRaws(inputData) + expect_equal(expected, result) + + # Single binary column + input <- list(list(r1), list(r2), list(r3)) + expected <- subset(expected, select = "V2") + result <- setNames(rbindRaws(input), "V2") + expect_equal(expected, result) + +}) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index debf0180183a4..cfe41ded200c2 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -36,7 +36,14 @@ compute <- function(mode, partition, serializer, deserializer, key, # available since R 3.2.4. So we set the global option here. oldOpt <- getOption("stringsAsFactors") options(stringsAsFactors = FALSE) - inputData <- do.call(rbind.data.frame, inputData) + + # Handle binary data types + if ("raw" %in% sapply(inputData[[1]], class)) { + inputData <- SparkR:::rbindRaws(inputData) + } else { + inputData <- do.call(rbind.data.frame, inputData) + } + options(stringsAsFactors = oldOpt) names(inputData) <- colNames From c8811adaa6b2fb6c5ca31520908d148326ebaf18 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 7 Sep 2016 10:38:56 +0200 Subject: [PATCH 1270/1470] [SPARK-17296][SQL] Simplify parser join processing [BACKPORT 2.0] ## What changes were proposed in this pull request? This PR backports https://github.com/apache/spark/pull/14867 to branch-2.0. It fixes a number of join ordering bugs. ## How was this patch tested? Added tests to `PlanParserSuite`. Author: Herman van Hovell Closes #14984 from hvanhovell/SPARK-17296-branch-2.0. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 11 ++- .../sql/catalyst/parser/AstBuilder.scala | 94 +++++++++---------- .../sql/catalyst/parser/ParserUtils.scala | 4 +- .../sql/catalyst/parser/PlanParserSuite.scala | 44 +++++++++ 4 files changed, 97 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index ecb7c8adeba82..4b47fa3d40ab2 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -367,11 +367,12 @@ setQuantifier ; relation - : left=relation - ((CROSS | joinType) JOIN right=relation joinCriteria? - | NATURAL joinType JOIN right=relation - ) #joinRelation - | relationPrimary #relationDefault + : relationPrimary joinRelation* + ; + +joinRelation + : (CROSS | joinType) JOIN right=relationPrimary joinCriteria? + | NATURAL joinType JOIN right=relationPrimary ; joinType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 0451abe2aaa07..d1ce90770d3a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -91,10 +91,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // Apply CTEs query.optional(ctx.ctes) { - val ctes = ctx.ctes.namedQuery.asScala.map { - case nCtx => - val namedQuery = visitNamedQuery(nCtx) - (namedQuery.alias, namedQuery) + val ctes = ctx.ctes.namedQuery.asScala.map { nCtx => + val namedQuery = visitNamedQuery(nCtx) + (namedQuery.alias, namedQuery) } // Check for duplicate names. checkDuplicateKeys(ctes, ctx) @@ -400,7 +399,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * separated) relations here, these get converted into a single plan by condition-less inner join. */ override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) { - val from = ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None)) + val from = ctx.relation.asScala.foldLeft(null: LogicalPlan) { (left, relation) => + val right = plan(relation.relationPrimary) + val join = right.optionalMap(left)(Join(_, _, Inner, None)) + withJoinRelations(join, relation) + } ctx.lateralView.asScala.foldLeft(from)(withGenerate) } @@ -526,54 +529,49 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { } /** - * Create a joins between two or more logical plans. + * Create a single relation referenced in a FROM claused. This method is used when a part of the + * join condition is nested, for example: + * {{{ + * select * from t1 join (t2 cross join t3) on col1 = col2 + * }}} */ - override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) { - /** Build a join between two plans. */ - def join(ctx: JoinRelationContext, left: LogicalPlan, right: LogicalPlan): Join = { - val baseJoinType = ctx.joinType match { - case null => Inner - case jt if jt.FULL != null => FullOuter - case jt if jt.SEMI != null => LeftSemi - case jt if jt.ANTI != null => LeftAnti - case jt if jt.LEFT != null => LeftOuter - case jt if jt.RIGHT != null => RightOuter - case _ => Inner - } + override def visitRelation(ctx: RelationContext): LogicalPlan = withOrigin(ctx) { + withJoinRelations(plan(ctx.relationPrimary), ctx) + } - // Resolve the join type and join condition - val (joinType, condition) = Option(ctx.joinCriteria) match { - case Some(c) if c.USING != null => - val columns = c.identifier.asScala.map { column => - UnresolvedAttribute.quoted(column.getText) - } - (UsingJoin(baseJoinType, columns), None) - case Some(c) if c.booleanExpression != null => - (baseJoinType, Option(expression(c.booleanExpression))) - case None if ctx.NATURAL != null => - (NaturalJoin(baseJoinType), None) - case None => - (baseJoinType, None) - } - Join(left, right, joinType, condition) - } + /** + * Join one more [[LogicalPlan]]s to the current logical plan. + */ + private def withJoinRelations(base: LogicalPlan, ctx: RelationContext): LogicalPlan = { + ctx.joinRelation.asScala.foldLeft(base) { (left, join) => + withOrigin(join) { + val baseJoinType = join.joinType match { + case null => Inner + case jt if jt.FULL != null => FullOuter + case jt if jt.SEMI != null => LeftSemi + case jt if jt.ANTI != null => LeftAnti + case jt if jt.LEFT != null => LeftOuter + case jt if jt.RIGHT != null => RightOuter + case _ => Inner + } - // Handle all consecutive join clauses. ANTLR produces a right nested tree in which the the - // first join clause is at the top. However fields of previously referenced tables can be used - // in following join clauses. The tree needs to be reversed in order to make this work. - var result = plan(ctx.left) - var current = ctx - while (current != null) { - current.right match { - case right: JoinRelationContext => - result = join(current, result, plan(right.left)) - current = right - case right => - result = join(current, result, plan(right)) - current = null + // Resolve the join type and join condition + val (joinType, condition) = Option(join.joinCriteria) match { + case Some(c) if c.USING != null => + val columns = c.identifier.asScala.map { column => + UnresolvedAttribute.quoted(column.getText) + } + (UsingJoin(baseJoinType, columns), None) + case Some(c) if c.booleanExpression != null => + (baseJoinType, Option(expression(c.booleanExpression))) + case None if join.NATURAL != null => + (NaturalJoin(baseJoinType), None) + case None => + (baseJoinType, None) + } + Join(left, plan(join.right), joinType, condition) } } - result } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index bc35ae2f55409..9506de229b878 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -192,9 +192,7 @@ object ParserUtils { * Map a [[LogicalPlan]] to another [[LogicalPlan]] if the passed context exists using the * passed function. The original plan is returned when the context does not exist. */ - def optionalMap[C <: ParserRuleContext]( - ctx: C)( - f: (C, LogicalPlan) => LogicalPlan): LogicalPlan = { + def optionalMap[C](ctx: C)(f: (C, LogicalPlan) => LogicalPlan): LogicalPlan = { if (ctx != null) { f(ctx, plan) } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 7fe056a588376..ac9c494a0b433 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -357,10 +357,54 @@ class PlanParserSuite extends PlanTest { test("left anti join", LeftAnti, testExistence) test("anti join", LeftAnti, testExistence) + // Test natural cross join + intercept("select * from a natural cross join b") + + // Test natural join with a condition + intercept("select * from a natural join b on a.id = b.id") + // Test multiple consecutive joins assertEqual( "select * from a join b join c right join d", table("a").join(table("b")).join(table("c")).join(table("d"), RightOuter).select(star())) + + // SPARK-17296 + assertEqual( + "select * from t1 cross join t2 join t3 on t3.id = t1.id join t4 on t4.id = t1.id", + table("t1") + .join(table("t2"), Inner) + .join(table("t3"), Inner, Option(Symbol("t3.id") === Symbol("t1.id"))) + .join(table("t4"), Inner, Option(Symbol("t4.id") === Symbol("t1.id"))) + .select(star())) + + // Test multiple on clauses. + intercept("select * from t1 inner join t2 inner join t3 on col3 = col2 on col3 = col1") + + // Parenthesis + assertEqual( + "select * from t1 inner join (t2 inner join t3 on col3 = col2) on col3 = col1", + table("t1") + .join(table("t2") + .join(table("t3"), Inner, Option('col3 === 'col2)), Inner, Option('col3 === 'col1)) + .select(star())) + assertEqual( + "select * from t1 inner join (t2 inner join t3) on col3 = col2", + table("t1") + .join(table("t2").join(table("t3"), Inner, None), Inner, Option('col3 === 'col2)) + .select(star())) + assertEqual( + "select * from t1 inner join (t2 inner join t3 on col3 = col2)", + table("t1") + .join(table("t2").join(table("t3"), Inner, Option('col3 === 'col2)), Inner, None) + .select(star())) + + // Implicit joins. + assertEqual( + "select * from t1, t3 join t2 on t1.col1 = t2.col2", + table("t1") + .join(table("t3")) + .join(table("t2"), Inner, Option(Symbol("t1.col1") === Symbol("t2.col2"))) + .select(star())) } test("sampled relations") { From e6caceb5e141a1665b21d04079a86baca041e453 Mon Sep 17 00:00:00 2001 From: Srinivasa Reddy Vundela Date: Wed, 7 Sep 2016 12:41:03 +0100 Subject: [PATCH 1271/1470] [MINOR][SQL] Fixing the typo in unit test ## What changes were proposed in this pull request? Fixing the typo in the unit test of CodeGenerationSuite.scala ## How was this patch tested? Ran the unit test after fixing the typo and it passes Author: Srinivasa Reddy Vundela Closes #14989 from vundela/typo_fix. (cherry picked from commit 76ad89e9241fb2dece95dd445661dd95ee4ef699) Signed-off-by: Sean Owen --- .../spark/sql/catalyst/expressions/CodeGenerationSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 8ea8f61150844..0532cf51136da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -58,8 +58,8 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { GenerateOrdering.generate(Add(Literal(123), Literal(1)).asc :: Nil) assert(CodegenMetrics.METRIC_COMPILATION_TIME.getCount() == startCount1 + 1) assert(CodegenMetrics.METRIC_SOURCE_CODE_SIZE.getCount() == startCount2 + 1) - assert(CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getCount() > startCount1) - assert(CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getCount() > startCount1) + assert(CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.getCount() > startCount3) + assert(CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.getCount() > startCount4) } test("SPARK-8443: split wide projections into blocks due to JVM code size limit") { From 078ac0e6321aeb72c670a65ec90b9c20ef0a7788 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 7 Sep 2016 12:33:50 -0700 Subject: [PATCH 1272/1470] [SPARK-17370] Shuffle service files not invalidated when a slave is lost ## What changes were proposed in this pull request? DAGScheduler invalidates shuffle files when an executor loss event occurs, but not when the external shuffle service is enabled. This is because when shuffle service is on, the shuffle file lifetime can exceed the executor lifetime. However, it also doesn't invalidate shuffle files when the shuffle service itself is lost (due to whole slave loss). This can cause long hangs when slaves are lost since the file loss is not detected until a subsequent stage attempts to read the shuffle files. The proposed fix is to also invalidate shuffle files when an executor is lost due to a `SlaveLost` event. ## How was this patch tested? Unit tests, also verified on an actual cluster that slave loss invalidates shuffle files immediately as expected. cc mateiz Author: Eric Liang Closes #14931 from ericl/sc-4439. (cherry picked from commit 649fa4bf1d6fc9271ae56b6891bc93ebf57858d1) Signed-off-by: Josh Rosen --- .../apache/spark/deploy/DeployMessage.scala | 2 +- .../deploy/client/StandaloneAppClient.scala | 4 +- .../client/StandaloneAppClientListener.scala | 3 +- .../apache/spark/deploy/master/Master.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 24 +++++--- .../spark/scheduler/DAGSchedulerEvent.scala | 3 +- .../spark/scheduler/ExecutorLossReason.scala | 6 +- .../spark/scheduler/TaskSchedulerImpl.scala | 9 ++- .../cluster/StandaloneSchedulerBackend.scala | 5 +- .../spark/deploy/client/AppClientSuite.scala | 3 +- .../spark/scheduler/DAGSchedulerSuite.scala | 58 ++++++++++++++++--- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- 12 files changed, 92 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 34c0696bfc4e5..ac09c6c497f8b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -135,7 +135,7 @@ private[deploy] object DeployMessages { } case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], - exitStatus: Option[Int]) + exitStatus: Option[Int], workerLost: Boolean) case class ApplicationRemoved(message: String) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index a9df732df93ca..b1c414d2927e9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -177,12 +177,12 @@ private[spark] class StandaloneAppClient( cores)) listener.executorAdded(fullId, workerId, hostPort, cores, memory) - case ExecutorUpdated(id, state, message, exitStatus) => + case ExecutorUpdated(id, state, message, exitStatus, workerLost) => val fullId = appId + "/" + id val messageText = message.map(s => " (" + s + ")").getOrElse("") logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText)) if (ExecutorState.isFinished(state)) { - listener.executorRemoved(fullId, message.getOrElse(""), exitStatus) + listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, workerLost) } case MasterChanged(masterRef, masterWebUiUrl) => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala index 370b16ce4213a..64255ec92b72a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala @@ -36,5 +36,6 @@ private[spark] trait StandaloneAppClientListener { def executorAdded( fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit - def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit + def executorRemoved( + fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit } 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 dfffc47703ab4..dcf41638e7994 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 @@ -252,7 +252,7 @@ private[deploy] class Master( appInfo.resetRetryCount() } - exec.application.driver.send(ExecutorUpdated(execId, state, message, exitStatus)) + exec.application.driver.send(ExecutorUpdated(execId, state, message, exitStatus, false)) if (ExecutorState.isFinished(state)) { // Remove this executor from the worker and app @@ -766,7 +766,7 @@ private[deploy] class Master( for (exec <- worker.executors.values) { logInfo("Telling app of lost executor: " + exec.id) exec.application.driver.send(ExecutorUpdated( - exec.id, ExecutorState.LOST, Some("worker lost"), None)) + exec.id, ExecutorState.LOST, Some("worker lost"), None, workerLost = true)) exec.state = ExecutorState.LOST exec.application.removeExecutor(exec) } 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 5291b663667ea..399d67103fb4a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -233,8 +233,8 @@ class DAGScheduler( /** * Called by TaskScheduler implementation when an executor fails. */ - def executorLost(execId: String): Unit = { - eventProcessLoop.post(ExecutorLost(execId)) + def executorLost(execId: String, reason: ExecutorLossReason): Unit = { + eventProcessLoop.post(ExecutorLost(execId, reason)) } /** @@ -1297,7 +1297,7 @@ class DAGScheduler( // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { - handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) + handleExecutorLost(bmAddress.executorId, filesLost = true, Some(task.epoch)) } } @@ -1323,15 +1323,16 @@ class DAGScheduler( * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * * We will also assume that we've lost all shuffle blocks associated with the executor if the - * executor serves its own blocks (i.e., we're not using external shuffle) OR a FetchFailed - * occurred, in which case we presume all shuffle data related to this executor to be lost. + * executor serves its own blocks (i.e., we're not using external shuffle), the entire slave + * is lost (likely including the shuffle service), or a FetchFailed occurred, in which case we + * presume all shuffle data related to this executor to be lost. * * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ private[scheduler] def handleExecutorLost( execId: String, - fetchFailed: Boolean, + filesLost: Boolean, maybeEpoch: Option[Long] = None) { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { @@ -1339,7 +1340,8 @@ class DAGScheduler( logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch)) blockManagerMaster.removeExecutor(execId) - if (!env.blockManager.externalShuffleServiceEnabled || fetchFailed) { + if (filesLost || !env.blockManager.externalShuffleServiceEnabled) { + logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch)) // TODO: This will be really slow if we keep accumulating shuffle map stages for ((shuffleId, stage) <- shuffleToMapStage) { stage.removeOutputsOnExecutor(execId) @@ -1643,8 +1645,12 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case ExecutorAdded(execId, host) => dagScheduler.handleExecutorAdded(execId, host) - case ExecutorLost(execId) => - dagScheduler.handleExecutorLost(execId, fetchFailed = false) + case ExecutorLost(execId, reason) => + val filesLost = reason match { + case SlaveLost(_, true) => true + case _ => false + } + dagScheduler.handleExecutorLost(execId, filesLost) case BeginEvent(task, taskInfo) => dagScheduler.handleBeginEvent(task, taskInfo) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 8c761124824ae..03781a2a2b56c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -77,7 +77,8 @@ private[scheduler] case class CompletionEvent( private[scheduler] case class ExecutorAdded(execId: String, host: String) extends DAGSchedulerEvent -private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent +private[scheduler] case class ExecutorLost(execId: String, reason: ExecutorLossReason) + extends DAGSchedulerEvent private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String, exception: Option[Throwable]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index 642bf81ac087e..46a35b6a2eaf9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -51,6 +51,10 @@ private[spark] object ExecutorKilled extends ExecutorLossReason("Executor killed */ private [spark] object LossReasonPending extends ExecutorLossReason("Pending loss reason.") +/** + * @param _message human readable loss reason + * @param workerLost whether the worker is confirmed lost too (i.e. including shuffle service) + */ private[spark] -case class SlaveLost(_message: String = "Slave lost") +case class SlaveLost(_message: String = "Slave lost", workerLost: Boolean = false) extends ExecutorLossReason(_message) 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 7dd4f6e9d2d90..d22321b88fb8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -332,6 +332,7 @@ private[spark] class TaskSchedulerImpl( def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { var failedExecutor: Option[String] = None + var reason: Option[ExecutorLossReason] = None synchronized { try { if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { @@ -339,8 +340,9 @@ private[spark] class TaskSchedulerImpl( val execId = taskIdToExecutorId(tid) if (executorIdToTaskCount.contains(execId)) { - removeExecutor(execId, + reason = Some( SlaveLost(s"Task $tid was lost, so marking the executor as lost as well.")) + removeExecutor(execId, reason.get) failedExecutor = Some(execId) } } @@ -373,7 +375,8 @@ private[spark] class TaskSchedulerImpl( } // Update the DAGScheduler without holding a lock on this, since that can deadlock if (failedExecutor.isDefined) { - dagScheduler.executorLost(failedExecutor.get) + assert(reason.isDefined) + dagScheduler.executorLost(failedExecutor.get, reason.get) backend.reviveOffers() } } @@ -499,7 +502,7 @@ private[spark] class TaskSchedulerImpl( } // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock if (failedExecutor.isDefined) { - dagScheduler.executorLost(failedExecutor.get) + dagScheduler.executorLost(failedExecutor.get, reason) backend.reviveOffers() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 8382fbe9ddb80..ea7e96ad3b0ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -148,10 +148,11 @@ private[spark] class StandaloneSchedulerBackend( fullId, hostPort, cores, Utils.megabytesToString(memory))) } - override def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]) { + override def executorRemoved( + fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean) { val reason: ExecutorLossReason = exitStatus match { case Some(code) => ExecutorExited(code, exitCausedByApp = true, message) - case None => SlaveLost(message) + case None => SlaveLost(message, workerLost = workerLost) } logInfo("Executor %s removed: %s".format(fullId, message)) removeExecutor(fullId.split("/")(1), reason) diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index f6ef9d15ddee2..57e5fb5ccfe66 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -196,7 +196,8 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd execAddedList.add(id) } - def executorRemoved(id: String, message: String, exitStatus: Option[Int]): Unit = { + def executorRemoved( + id: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit = { execRemovedList.add(id) } } 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 ce4e7a237e9f9..f69e8f28fb420 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} @@ -199,7 +200,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou override def beforeEach(): Unit = { super.beforeEach() - sc = new SparkContext("local", "DAGSchedulerSuite") + init(new SparkConf()) + } + + private def init(testConf: SparkConf): Unit = { + sc = new SparkContext("local", "DAGSchedulerSuite", testConf) sparkListener.submittedStageInfos.clear() sparkListener.successfulStages.clear() sparkListener.failedStages.clear() @@ -605,6 +610,46 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assertDataStructuresEmpty() } + private val shuffleFileLossTests = Seq( + ("slave lost with shuffle service", SlaveLost("", false), true, false), + ("worker lost with shuffle service", SlaveLost("", true), true, true), + ("worker lost without shuffle service", SlaveLost("", true), false, true), + ("executor failure with shuffle service", ExecutorKilled, true, false), + ("executor failure without shuffle service", ExecutorKilled, false, true)) + + for ((eventDescription, event, shuffleServiceOn, expectFileLoss) <- shuffleFileLossTests) { + val maybeLost = if (expectFileLoss) { + "lost" + } else { + "not lost" + } + test(s"shuffle files $maybeLost when $eventDescription") { + // reset the test context with the right shuffle service config + afterEach() + val conf = new SparkConf() + conf.set("spark.shuffle.service.enabled", shuffleServiceOn.toString) + init(conf) + assert(sc.env.blockManager.externalShuffleServiceEnabled == shuffleServiceOn) + + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) + submit(reduceRdd, Array(0)) + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + runEvent(ExecutorLost("exec-hostA", event)) + if (expectFileLoss) { + intercept[MetadataFetchFailedException] { + mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) + } + } else { + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + } + } + } // Helper function to validate state when creating tests for task failures private def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) { @@ -612,7 +657,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assert(stageAttempt.stageAttemptId == attempt) } - // Helper functions to extract commonly used code in Fetch Failure test cases private def setupStageAbortTest(sc: SparkContext) { sc.listenerBus.addListener(new EndListener()) @@ -1094,7 +1138,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) @@ -1225,7 +1269,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou )) // then one executor dies, and a task fails in stage 1 - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"), @@ -1323,7 +1367,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou makeMapStatus("hostA", reduceRdd.partitions.length))) // now that host goes down - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) // so we resubmit those tasks runEvent(makeCompletionEvent(taskSets(0).tasks(0), Resubmitted, null)) @@ -1516,7 +1560,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou submit(reduceRdd, Array(0)) // blockManagerMaster.removeExecutor("exec-hostA") // pretend we were told hostA went away - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks // rather than marking it is as failed and waiting. complete(taskSets(0), Seq( @@ -1983,7 +2027,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou // Pretend host A was lost val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) 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 862313326c93e..b98f945bac253 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -44,7 +44,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) override def executorAdded(execId: String, host: String) {} - override def executorLost(execId: String) {} + override def executorLost(execId: String, reason: ExecutorLossReason) {} override def taskSetFailed( taskSet: TaskSet, From 067752ce08dc035ee807d20be2202c385f88f01c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 7 Sep 2016 16:43:05 -0700 Subject: [PATCH 1273/1470] [SPARK-16533][CORE] - backport driver deadlock fix to 2.0 ## What changes were proposed in this pull request? Backport changes from #14710 and #14925 to 2.0 Author: Marcelo Vanzin Author: Angus Gerry Closes #14933 from angolon/SPARK-16533-2.0. --- .../spark/ExecutorAllocationManager.scala | 2 +- .../deploy/client/StandaloneAppClient.scala | 38 +++---- .../CoarseGrainedSchedulerBackend.scala | 105 +++++++++++------- .../cluster/StandaloneSchedulerBackend.scala | 10 +- .../MesosCoarseGrainedSchedulerBackend.scala | 5 +- .../apache/spark/HeartbeatReceiverSuite.scala | 9 +- .../spark/deploy/client/AppClientSuite.scala | 30 +++-- ...osCoarseGrainedSchedulerBackendSuite.scala | 14 ++- .../cluster/YarnSchedulerBackend.scala | 95 +++++++--------- 9 files changed, 170 insertions(+), 138 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 932ba16812bbb..6f320c524201c 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -230,7 +230,7 @@ private[spark] class ExecutorAllocationManager( } } } - executor.scheduleAtFixedRate(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS) + executor.scheduleWithFixedDelay(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS) client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index b1c414d2927e9..93f58ce63799f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -21,6 +21,8 @@ import java.util.concurrent._ import java.util.concurrent.{Future => JFuture, ScheduledFuture => JScheduledFuture} import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} +import scala.concurrent.Future +import scala.util.{Failure, Success} import scala.util.control.NonFatal import org.apache.spark.SparkConf @@ -79,11 +81,6 @@ private[spark] class StandaloneAppClient( private val registrationRetryThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("appclient-registration-retry-thread") - // A thread pool to perform receive then reply actions in a thread so as not to block the - // event loop. - private val askAndReplyThreadPool = - ThreadUtils.newDaemonCachedThreadPool("appclient-receive-and-reply-threadpool") - override def onStart(): Unit = { try { registerWithMaster(1) @@ -220,19 +217,13 @@ private[spark] class StandaloneAppClient( endpointRef: RpcEndpointRef, context: RpcCallContext, msg: T): Unit = { - // Create a thread to ask a message and reply with the result. Allow thread to be + // Ask a message and create a thread to reply with the result. Allow thread to be // interrupted during shutdown, otherwise context must be notified of NonFatal errors. - askAndReplyThreadPool.execute(new Runnable { - override def run(): Unit = { - try { - context.reply(endpointRef.askWithRetry[Boolean](msg)) - } catch { - case ie: InterruptedException => // Cancelled - case NonFatal(t) => - context.sendFailure(t) - } - } - }) + endpointRef.ask[Boolean](msg).andThen { + case Success(b) => context.reply(b) + case Failure(ie: InterruptedException) => // Cancelled + case Failure(NonFatal(t)) => context.sendFailure(t) + }(ThreadUtils.sameThread) } override def onDisconnected(address: RpcAddress): Unit = { @@ -272,7 +263,6 @@ private[spark] class StandaloneAppClient( registrationRetryThread.shutdownNow() registerMasterFutures.get.foreach(_.cancel(true)) registerMasterThreadPool.shutdownNow() - askAndReplyThreadPool.shutdownNow() } } @@ -301,12 +291,12 @@ private[spark] class StandaloneAppClient( * * @return whether the request is acknowledged. */ - def requestTotalExecutors(requestedTotal: Int): Boolean = { + def requestTotalExecutors(requestedTotal: Int): Future[Boolean] = { if (endpoint.get != null && appId.get != null) { - endpoint.get.askWithRetry[Boolean](RequestExecutors(appId.get, requestedTotal)) + endpoint.get.ask[Boolean](RequestExecutors(appId.get, requestedTotal)) } else { logWarning("Attempted to request executors before driver fully initialized.") - false + Future.successful(false) } } @@ -314,12 +304,12 @@ private[spark] class StandaloneAppClient( * Kill the given list of executors through the Master. * @return whether the kill request is acknowledged. */ - def killExecutors(executorIds: Seq[String]): Boolean = { + def killExecutors(executorIds: Seq[String]): Future[Boolean] = { if (endpoint.get != null && appId.get != null) { - endpoint.get.askWithRetry[Boolean](KillExecutors(appId.get, executorIds)) + endpoint.get.ask[Boolean](KillExecutors(appId.get, executorIds)) } else { logWarning("Attempted to kill executors before driver fully initialized.") - false + Future.successful(false) } } 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 5a74ddd94126f..c6b3fdf439f5f 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 @@ -22,6 +22,8 @@ import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.concurrent.Future +import scala.concurrent.duration.Duration import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} import org.apache.spark.internal.Logging @@ -49,6 +51,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp protected val totalRegisteredExecutors = new AtomicInteger(0) protected val conf = scheduler.sc.conf private val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) + private val defaultAskTimeout = RpcUtils.askRpcTimeout(conf) // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. private val _minRegisteredRatio = @@ -272,6 +275,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Remove a disconnected slave from the cluster private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { + logDebug(s"Asked to remove executor $executorId with reason $reason") executorDataMap.get(executorId) match { case Some(executorInfo) => // This must be synchronized because variables mutated @@ -446,19 +450,24 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * Request an additional number of executors from the cluster manager. * @return whether the request is acknowledged. */ - final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { + final override def requestExecutors(numAdditionalExecutors: Int): Boolean = { if (numAdditionalExecutors < 0) { throw new IllegalArgumentException( "Attempted to request a negative number of additional executor(s) " + s"$numAdditionalExecutors from the cluster manager. Please specify a positive number!") } logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") - logDebug(s"Number of pending executors is now $numPendingExecutors") - numPendingExecutors += numAdditionalExecutors - // Account for executors pending to be added or removed - val newTotal = numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size - doRequestTotalExecutors(newTotal) + val response = synchronized { + numPendingExecutors += numAdditionalExecutors + logDebug(s"Number of pending executors is now $numPendingExecutors") + + // Account for executors pending to be added or removed + doRequestTotalExecutors( + numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) + } + + defaultAskTimeout.awaitResult(response) } /** @@ -479,19 +488,24 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp numExecutors: Int, localityAwareTasks: Int, hostToLocalTaskCount: Map[String, Int] - ): Boolean = synchronized { + ): Boolean = { if (numExecutors < 0) { throw new IllegalArgumentException( "Attempted to request a negative number of executor(s) " + s"$numExecutors from the cluster manager. Please specify a positive number!") } - this.localityAwareTasks = localityAwareTasks - this.hostToLocalTaskCount = hostToLocalTaskCount + val response = synchronized { + this.localityAwareTasks = localityAwareTasks + this.hostToLocalTaskCount = hostToLocalTaskCount + + numPendingExecutors = + math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0) - numPendingExecutors = - math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0) - doRequestTotalExecutors(numExecutors) + doRequestTotalExecutors(numExecutors) + } + + defaultAskTimeout.awaitResult(response) } /** @@ -504,16 +518,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * insufficient resources to satisfy the first request. We make the assumption here that the * cluster manager will eventually fulfill all requests when resources free up. * - * @return whether the request is acknowledged. + * @return a future whose evaluation indicates whether the request is acknowledged. */ - protected def doRequestTotalExecutors(requestedTotal: Int): Boolean = false + protected def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = + Future.successful(false) /** * Request that the cluster manager kill the specified executors. * @return whether the kill request is acknowledged. If list to kill is empty, it will return * false. */ - final override def killExecutors(executorIds: Seq[String]): Boolean = synchronized { + final override def killExecutors(executorIds: Seq[String]): Boolean = { killExecutors(executorIds, replace = false, force = false) } @@ -533,39 +548,53 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp final def killExecutors( executorIds: Seq[String], replace: Boolean, - force: Boolean): Boolean = synchronized { + force: Boolean): Boolean = { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") - val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) - unknownExecutors.foreach { id => - logWarning(s"Executor to kill $id does not exist!") - } - // If an executor is already pending to be removed, do not kill it again (SPARK-9795) - // If this executor is busy, do not kill it unless we are told to force kill it (SPARK-9552) - val executorsToKill = knownExecutors - .filter { id => !executorsPendingToRemove.contains(id) } - .filter { id => force || !scheduler.isExecutorBusy(id) } - executorsToKill.foreach { id => executorsPendingToRemove(id) = !replace } - - // If we do not wish to replace the executors we kill, sync the target number of executors - // with the cluster manager to avoid allocating new ones. When computing the new target, - // take into account executors that are pending to be added or removed. - if (!replace) { - doRequestTotalExecutors( - numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) - } else { - numPendingExecutors += knownExecutors.size + val response = synchronized { + val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) + unknownExecutors.foreach { id => + logWarning(s"Executor to kill $id does not exist!") + } + + // If an executor is already pending to be removed, do not kill it again (SPARK-9795) + // If this executor is busy, do not kill it unless we are told to force kill it (SPARK-9552) + val executorsToKill = knownExecutors + .filter { id => !executorsPendingToRemove.contains(id) } + .filter { id => force || !scheduler.isExecutorBusy(id) } + executorsToKill.foreach { id => executorsPendingToRemove(id) = !replace } + + // If we do not wish to replace the executors we kill, sync the target number of executors + // with the cluster manager to avoid allocating new ones. When computing the new target, + // take into account executors that are pending to be added or removed. + val adjustTotalExecutors = + if (!replace) { + doRequestTotalExecutors( + numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) + } else { + numPendingExecutors += knownExecutors.size + Future.successful(true) + } + + val killExecutors: Boolean => Future[Boolean] = + if (!executorsToKill.isEmpty) { + _ => doKillExecutors(executorsToKill) + } else { + _ => Future.successful(false) + } + + adjustTotalExecutors.flatMap(killExecutors)(ThreadUtils.sameThread) } - !executorsToKill.isEmpty && doKillExecutors(executorsToKill) + defaultAskTimeout.awaitResult(response) } /** * Kill the given list of executors through the cluster manager. * @return whether the kill request is acknowledged. */ - protected def doKillExecutors(executorIds: Seq[String]): Boolean = false - + protected def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = + Future.successful(false) } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index ea7e96ad3b0ff..04d40e2907cff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -19,6 +19,8 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.Semaphore +import scala.concurrent.Future + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} @@ -174,12 +176,12 @@ private[spark] class StandaloneSchedulerBackend( * * @return whether the request is acknowledged. */ - protected override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { + protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { Option(client) match { case Some(c) => c.requestTotalExecutors(requestedTotal) case None => logWarning("Attempted to request executors before driver fully initialized.") - false + Future.successful(false) } } @@ -187,12 +189,12 @@ private[spark] class StandaloneSchedulerBackend( * Kill the given list of executors through the Master. * @return whether the kill request is acknowledged. */ - protected override def doKillExecutors(executorIds: Seq[String]): Boolean = { + protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { Option(client) match { case Some(c) => c.killExecutors(executorIds) case None => logWarning("Attempted to kill executors before driver fully initialized.") - false + Future.successful(false) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 2dcd67c7b89fa..473b1be4e20e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -24,6 +24,7 @@ import java.util.concurrent.locks.ReentrantLock import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.{Buffer, HashMap, HashSet} +import scala.concurrent.Future import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} @@ -577,7 +578,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( super.applicationId } - override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future.successful { // We don't truly know if we can fulfill the full amount of executors // since at coarse grain it depends on the amount of slaves available. logInfo("Capping the total amount of executors to " + requestedTotal) @@ -585,7 +586,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( true } - override def doKillExecutors(executorIds: Seq[String]): Boolean = { + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future.successful { if (mesosDriver == null) { logWarning("Asked to kill executors before the Mesos driver was started.") false diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 5e2ba311ee773..e30349570b7ed 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -21,6 +21,7 @@ import java.util.concurrent.{ExecutorService, TimeUnit} import scala.collection.Map import scala.collection.mutable +import scala.concurrent.Future import scala.concurrent.duration._ import scala.language.postfixOps @@ -270,13 +271,13 @@ private class FakeSchedulerBackend( clusterManagerEndpoint: RpcEndpointRef) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { - protected override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - clusterManagerEndpoint.askWithRetry[Boolean]( + protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + clusterManagerEndpoint.ask[Boolean]( RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) } - protected override def doKillExecutors(executorIds: Seq[String]): Boolean = { - clusterManagerEndpoint.askWithRetry[Boolean](KillExecutors(executorIds)) + protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { + clusterManagerEndpoint.ask[Boolean](KillExecutors(executorIds)) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index 57e5fb5ccfe66..bc58fb2a362a4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentLinkedQueue import scala.concurrent.duration._ import org.scalatest.BeforeAndAfterAll -import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.{Eventually, ScalaFutures} import org.apache.spark._ import org.apache.spark.deploy.{ApplicationDescription, Command} @@ -36,7 +36,12 @@ import org.apache.spark.util.Utils /** * End-to-end tests for application client in standalone mode. */ -class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterAll { +class AppClientSuite + extends SparkFunSuite + with LocalSparkContext + with BeforeAndAfterAll + with Eventually + with ScalaFutures { private val numWorkers = 2 private val conf = new SparkConf() private val securityManager = new SecurityManager(conf) @@ -93,7 +98,12 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd // Send message to Master to request Executors, verify request by change in executor limit val numExecutorsRequested = 1 - assert(ci.client.requestTotalExecutors(numExecutorsRequested)) + whenReady( + ci.client.requestTotalExecutors(numExecutorsRequested), + timeout(10.seconds), + interval(10.millis)) { acknowledged => + assert(acknowledged) + } eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() @@ -101,10 +111,12 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd } // Send request to kill executor, verify request was made - assert { - val apps = getApplications() - val executorId: String = apps.head.executors.head._2.fullId - ci.client.killExecutors(Seq(executorId)) + val executorId: String = getApplications().head.executors.head._2.fullId + whenReady( + ci.client.killExecutors(Seq(executorId)), + timeout(10.seconds), + interval(10.millis)) { acknowledged => + assert(acknowledged) } // Issue stop command for Client to disconnect from Master @@ -122,7 +134,9 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) // requests to master should fail immediately - assert(ci.client.requestTotalExecutors(3) === false) + whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => + assert(success === false) + } } // =============================== diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 12c4a7993e79e..3ffbe70e76bbb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -18,9 +18,11 @@ package org.apache.spark.scheduler.cluster.mesos import java.util.Collections +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ import scala.concurrent.Promise import scala.reflect.ClassTag @@ -30,6 +32,7 @@ import org.apache.mesos.Protos.Value.Scalar import org.mockito.{ArgumentCaptor, Matchers} import org.mockito.Matchers._ import org.mockito.Mockito._ +import org.scalatest.concurrent.ScalaFutures import org.scalatest.mock.MockitoSugar import org.scalatest.BeforeAndAfter @@ -42,7 +45,8 @@ import org.apache.spark.scheduler.TaskSchedulerImpl class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar - with BeforeAndAfter { + with BeforeAndAfter + with ScalaFutures { private var sparkConf: SparkConf = _ private var driver: SchedulerDriver = _ @@ -52,6 +56,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite private var driverEndpoint: RpcEndpointRef = _ @volatile private var stopCalled = false + // All 'requests' to the scheduler run immediately on the same thread, so + // demand that all futures have their value available immediately. + implicit override val patienceConfig = PatienceConfig(timeout = Duration(0, TimeUnit.SECONDS)) + test("mesos supports killing and limiting executors") { setBackend() sparkConf.set("spark.driver.host", "driverHost") @@ -66,8 +74,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite verifyTaskLaunched("o1") // kills executors - backend.doRequestTotalExecutors(0) - assert(backend.doKillExecutors(Seq("0"))) + assert(backend.doRequestTotalExecutors(0).futureValue) + assert(backend.doKillExecutors(Seq("0")).futureValue) val taskID0 = createTaskId("0") verify(driver, times(1)).killTask(taskID0) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index ea63ff5dc1580..2f9ea1911fd61 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} import scala.util.control.NonFatal import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} @@ -124,28 +125,16 @@ private[spark] abstract class YarnSchedulerBackend( * Request executors from the ApplicationMaster by specifying the total number desired. * This includes executors already pending or running. */ - override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - val r = RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) - yarnSchedulerEndpoint.amEndpoint match { - case Some(am) => - try { - am.askWithRetry[Boolean](r) - } catch { - case NonFatal(e) => - logError(s"Sending $r to AM was unsuccessful", e) - return false - } - case None => - logWarning("Attempted to request executors before the AM has registered!") - return false - } + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + yarnSchedulerEndpointRef.ask[Boolean]( + RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) } /** * Request that the ApplicationMaster kill the specified executors. */ - override def doKillExecutors(executorIds: Seq[String]): Boolean = { - yarnSchedulerEndpointRef.askWithRetry[Boolean](KillExecutors(executorIds)) + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { + yarnSchedulerEndpointRef.ask[Boolean](KillExecutors(executorIds)) } override def sufficientResourcesRegistered(): Boolean = { @@ -221,37 +210,37 @@ private[spark] abstract class YarnSchedulerBackend( */ private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging { - var amEndpoint: Option[RpcEndpointRef] = None - - private val askAmThreadPool = - ThreadUtils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-thread-pool") - implicit val askAmExecutor = ExecutionContext.fromExecutor(askAmThreadPool) + private var amEndpoint: Option[RpcEndpointRef] = None private[YarnSchedulerBackend] def handleExecutorDisconnectedFromDriver( executorId: String, executorRpcAddress: RpcAddress): Unit = { - amEndpoint match { + val removeExecutorMessage = amEndpoint match { case Some(am) => val lossReasonRequest = GetExecutorLossReason(executorId) - val future = am.ask[ExecutorLossReason](lossReasonRequest, askTimeout) - future onSuccess { - case reason: ExecutorLossReason => - driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, reason)) - } - future onFailure { - case NonFatal(e) => - logWarning(s"Attempted to get executor loss reason" + - s" for executor id ${executorId} at RPC address ${executorRpcAddress}," + - s" but got no response. Marking as slave lost.", e) - driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, SlaveLost())) - case t => throw t - } + am.ask[ExecutorLossReason](lossReasonRequest, askTimeout) + .map { reason => RemoveExecutor(executorId, reason) }(ThreadUtils.sameThread) + .recover { + case NonFatal(e) => + logWarning(s"Attempted to get executor loss reason" + + s" for executor id ${executorId} at RPC address ${executorRpcAddress}," + + s" but got no response. Marking as slave lost.", e) + RemoveExecutor(executorId, SlaveLost()) + }(ThreadUtils.sameThread) case None => logWarning("Attempted to check for an executor loss reason" + " before the AM has registered!") - driverEndpoint.askWithRetry[Boolean]( - RemoveExecutor(executorId, SlaveLost("AM is not yet registered."))) + Future.successful(RemoveExecutor(executorId, SlaveLost("AM is not yet registered."))) } + + removeExecutorMessage + .flatMap { message => + driverEndpoint.ask[Boolean](message) + }(ThreadUtils.sameThread) + .onFailure { + case NonFatal(e) => logError( + s"Error requesting driver to remove executor $executorId after disconnection.", e) + }(ThreadUtils.sameThread) } override def receive: PartialFunction[Any, Unit] = { @@ -269,9 +258,13 @@ private[spark] abstract class YarnSchedulerBackend( case AddWebUIFilter(filterName, filterParams, proxyBase) => addWebUIFilter(filterName, filterParams, proxyBase) - case RemoveExecutor(executorId, reason) => + case r @ RemoveExecutor(executorId, reason) => logWarning(reason.toString) - removeExecutor(executorId, reason) + driverEndpoint.ask[Boolean](r).onFailure { + case e => + logError("Error requesting driver to remove executor" + + s" $executorId for reason $reason", e) + }(ThreadUtils.sameThread) } @@ -279,13 +272,12 @@ private[spark] abstract class YarnSchedulerBackend( case r: RequestExecutors => amEndpoint match { case Some(am) => - Future { - context.reply(am.askWithRetry[Boolean](r)) - } onFailure { - case NonFatal(e) => + am.ask[Boolean](r).andThen { + case Success(b) => context.reply(b) + case Failure(NonFatal(e)) => logError(s"Sending $r to AM was unsuccessful", e) context.sendFailure(e) - } + }(ThreadUtils.sameThread) case None => logWarning("Attempted to request executors before the AM has registered!") context.reply(false) @@ -294,13 +286,12 @@ private[spark] abstract class YarnSchedulerBackend( case k: KillExecutors => amEndpoint match { case Some(am) => - Future { - context.reply(am.askWithRetry[Boolean](k)) - } onFailure { - case NonFatal(e) => + am.ask[Boolean](k).andThen { + case Success(b) => context.reply(b) + case Failure(NonFatal(e)) => logError(s"Sending $k to AM was unsuccessful", e) context.sendFailure(e) - } + }(ThreadUtils.sameThread) case None => logWarning("Attempted to kill executors before the AM has registered!") context.reply(false) @@ -316,10 +307,6 @@ private[spark] abstract class YarnSchedulerBackend( amEndpoint = None } } - - override def onStop(): Unit = { - askAmThreadPool.shutdownNow() - } } } From 28377da380d3859e0a837aae1c39529228c515f5 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 7 Sep 2016 21:22:32 -0700 Subject: [PATCH 1274/1470] [SPARK-17339][CORE][BRANCH-2.0] Do not use path to get a filesystem in hadoopFile and newHadoopFile APIs ## What changes were proposed in this pull request? This PR backports https://github.com/apache/spark/pull/14960 ## How was this patch tested? AppVeyor - https://ci.appveyor.com/project/HyukjinKwon/spark/build/86-backport-SPARK-17339-r Author: hyukjinkwon Closes #15008 from HyukjinKwon/backport-SPARK-17339. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 37e067839775a..71511b800633f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -988,7 +988,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // This is a hack to enforce loading hdfs-site.xml. // See SPARK-11227 for details. - FileSystem.get(new URI(path), hadoopConfiguration) + FileSystem.getLocal(hadoopConfiguration) // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it. val confBroadcast = broadcast(new SerializableConfiguration(hadoopConfiguration)) @@ -1077,7 +1077,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // This is a hack to enforce loading hdfs-site.xml. // See SPARK-11227 for details. - FileSystem.get(new URI(path), hadoopConfiguration) + FileSystem.getLocal(hadoopConfiguration) // The call to NewHadoopJob automatically adds security credentials to conf, // so we don't need to explicitly add them ourselves From e169085cd33ff498ecd5aab180af036ca644c9e0 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 8 Sep 2016 08:16:19 -0500 Subject: [PATCH 1275/1470] [SPARK-16711] YarnShuffleService doesn't re-init properly on YARN rolling upgrade branch-2.0 version of this patch. The differences are in the YarnShuffleService for finding the location to put the DB. branch-2.0 does not use the yarn nm recovery path like master does. Tested in manually on 8 node yarn cluster and ran unit tests. Manually tests verified DB created properly and it found them if already existed. Verified that during rolling upgrade credentials were reloaded and running application was not affected. Author: Thomas Graves Closes #14997 from tgravescs/SPARK-16711-branch2.0. --- common/network-common/pom.xml | 16 ++ .../spark/network/util/LevelDBProvider.java | 152 ++++++++++++++ common/network-shuffle/pom.xml | 16 -- .../shuffle/ExternalShuffleBlockResolver.java | 120 +---------- .../network/yarn/YarnShuffleService.java | 198 ++++++++++++++---- .../yarn/YarnShuffleServiceSuite.scala | 10 + 6 files changed, 345 insertions(+), 167 deletions(-) create mode 100644 common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index bc3b0fe73f6eb..e170b9b7e30e4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -42,6 +42,22 @@ netty-all + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.core + jackson-annotations + + org.slf4j diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java new file mode 100644 index 0000000000000..ec900a7b3ca63 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.fusesource.leveldbjni.JniDBFactory; +import org.fusesource.leveldbjni.internal.NativeDB; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.Options; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * LevelDB utility class available in the network package. + */ +public class LevelDBProvider { + private static final Logger logger = LoggerFactory.getLogger(LevelDBProvider.class); + + public static DB initLevelDB(File dbFile, StoreVersion version, ObjectMapper mapper) throws + IOException { + DB tmpDb = null; + if (dbFile != null) { + Options options = new Options(); + options.createIfMissing(false); + options.logger(new LevelDBLogger()); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException e) { + if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { + logger.info("Creating state database at " + dbFile); + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + } else { + // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new + // one, so we can keep processing new apps + logger.error("error opening leveldb file {}. Creating new file, will not be able to " + + "recover state for existing applications", dbFile, e); + if (dbFile.isDirectory()) { + for (File f : dbFile.listFiles()) { + if (!f.delete()) { + logger.warn("error deleting {}", f.getPath()); + } + } + } + if (!dbFile.delete()) { + logger.warn("error deleting {}", dbFile.getPath()); + } + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + + } + } + // if there is a version mismatch, we throw an exception, which means the service is unusable + checkVersion(tmpDb, version, mapper); + } + return tmpDb; + } + + private static class LevelDBLogger implements org.iq80.leveldb.Logger { + private static final Logger LOG = LoggerFactory.getLogger(LevelDBLogger.class); + + @Override + public void log(String message) { + LOG.info(message); + } + } + + /** + * Simple major.minor versioning scheme. Any incompatible changes should be across major + * versions. Minor version differences are allowed -- meaning we should be able to read + * dbs that are either earlier *or* later on the minor version. + */ + public static void checkVersion(DB db, StoreVersion newversion, ObjectMapper mapper) throws + IOException { + byte[] bytes = db.get(StoreVersion.KEY); + if (bytes == null) { + storeVersion(db, newversion, mapper); + } else { + StoreVersion version = mapper.readValue(bytes, StoreVersion.class); + if (version.major != newversion.major) { + throw new IOException("cannot read state DB with version " + version + ", incompatible " + + "with current version " + newversion); + } + storeVersion(db, newversion, mapper); + } + } + + public static void storeVersion(DB db, StoreVersion version, ObjectMapper mapper) + throws IOException { + db.put(StoreVersion.KEY, mapper.writeValueAsBytes(version)); + } + + public static class StoreVersion { + + final static byte[] KEY = "StoreVersion".getBytes(StandardCharsets.UTF_8); + + public final int major; + public final int minor; + + @JsonCreator + public StoreVersion(@JsonProperty("major") int major, @JsonProperty("minor") int minor) { + this.major = major; + this.minor = minor; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StoreVersion that = (StoreVersion) o; + + return major == that.major && minor == that.minor; + } + + @Override + public int hashCode() { + int result = major; + result = 31 * result + minor; + return result; + } + } +} diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 2fb5835305a25..8b832cf376123 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -43,22 +43,6 @@ ${project.version} - - org.fusesource.leveldbjni - leveldbjni-all - 1.8 - - - - com.fasterxml.jackson.core - jackson-databind - - - - com.fasterxml.jackson.core - jackson-annotations - - org.slf4j diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 000ec13f796a0..e34dc1f5b1de5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -30,17 +30,16 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; import com.google.common.collect.Maps; -import org.fusesource.leveldbjni.JniDBFactory; -import org.fusesource.leveldbjni.internal.NativeDB; import org.iq80.leveldb.DB; import org.iq80.leveldb.DBIterator; -import org.iq80.leveldb.Options; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.buffer.FileSegmentManagedBuffer; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.util.LevelDBProvider; +import org.apache.spark.network.util.LevelDBProvider.StoreVersion; import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -95,52 +94,10 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF Executor directoryCleaner) throws IOException { this.conf = conf; this.registeredExecutorFile = registeredExecutorFile; - if (registeredExecutorFile != null) { - Options options = new Options(); - options.createIfMissing(false); - options.logger(new LevelDBLogger()); - DB tmpDb; - try { - tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); - } catch (NativeDB.DBException e) { - if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { - logger.info("Creating state database at " + registeredExecutorFile); - options.createIfMissing(true); - try { - tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); - } catch (NativeDB.DBException dbExc) { - throw new IOException("Unable to create state store", dbExc); - } - } else { - // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new - // one, so we can keep processing new apps - logger.error("error opening leveldb file {}. Creating new file, will not be able to " + - "recover state for existing applications", registeredExecutorFile, e); - if (registeredExecutorFile.isDirectory()) { - for (File f : registeredExecutorFile.listFiles()) { - if (!f.delete()) { - logger.warn("error deleting {}", f.getPath()); - } - } - } - if (!registeredExecutorFile.delete()) { - logger.warn("error deleting {}", registeredExecutorFile.getPath()); - } - options.createIfMissing(true); - try { - tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); - } catch (NativeDB.DBException dbExc) { - throw new IOException("Unable to create state store", dbExc); - } - - } - } - // if there is a version mismatch, we throw an exception, which means the service is unusable - checkVersion(tmpDb); - executors = reloadRegisteredExecutors(tmpDb); - db = tmpDb; + db = LevelDBProvider.initLevelDB(this.registeredExecutorFile, CURRENT_VERSION, mapper); + if (db != null) { + executors = reloadRegisteredExecutors(db); } else { - db = null; executors = Maps.newConcurrentMap(); } this.directoryCleaner = directoryCleaner; @@ -368,76 +325,11 @@ static ConcurrentMap reloadRegisteredExecutors(D break; } AppExecId id = parseDbAppExecKey(key); + logger.info("Reloading registered executors: " + id.toString()); ExecutorShuffleInfo shuffleInfo = mapper.readValue(e.getValue(), ExecutorShuffleInfo.class); registeredExecutors.put(id, shuffleInfo); } } return registeredExecutors; } - - private static class LevelDBLogger implements org.iq80.leveldb.Logger { - private static final Logger LOG = LoggerFactory.getLogger(LevelDBLogger.class); - - @Override - public void log(String message) { - LOG.info(message); - } - } - - /** - * Simple major.minor versioning scheme. Any incompatible changes should be across major - * versions. Minor version differences are allowed -- meaning we should be able to read - * dbs that are either earlier *or* later on the minor version. - */ - private static void checkVersion(DB db) throws IOException { - byte[] bytes = db.get(StoreVersion.KEY); - if (bytes == null) { - storeVersion(db); - } else { - StoreVersion version = mapper.readValue(bytes, StoreVersion.class); - if (version.major != CURRENT_VERSION.major) { - throw new IOException("cannot read state DB with version " + version + ", incompatible " + - "with current version " + CURRENT_VERSION); - } - storeVersion(db); - } - } - - private static void storeVersion(DB db) throws IOException { - db.put(StoreVersion.KEY, mapper.writeValueAsBytes(CURRENT_VERSION)); - } - - - public static class StoreVersion { - - static final byte[] KEY = "StoreVersion".getBytes(StandardCharsets.UTF_8); - - public final int major; - public final int minor; - - @JsonCreator public StoreVersion( - @JsonProperty("major") int major, - @JsonProperty("minor") int minor) { - this.major = major; - this.minor = minor; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - StoreVersion that = (StoreVersion) o; - - return major == that.major && minor == that.minor; - } - - @Override - public int hashCode() { - int result = major; - result = 31 * result + minor; - return result; - } - } - } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index b6feb55e2192b..9584f075e6aee 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -18,15 +18,28 @@ package org.apache.spark.network.yarn; import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.api.*; +import org.apache.spark.network.util.LevelDBProvider; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.DBIterator; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,6 +81,22 @@ public class YarnShuffleService extends AuxiliaryService { private static final String SPARK_AUTHENTICATE_KEY = "spark.authenticate"; private static final boolean DEFAULT_SPARK_AUTHENTICATE = false; + private static final String RECOVERY_FILE_NAME = "registeredExecutors.ldb"; + private static final String SECRETS_RECOVERY_FILE_NAME = "sparkShuffleRecovery.ldb"; + + // just for testing when you want to find an open port + @VisibleForTesting + static int boundPort = -1; + private static final ObjectMapper mapper = new ObjectMapper(); + private static final String APP_CREDS_KEY_PREFIX = "AppCreds"; + private static final LevelDBProvider.StoreVersion CURRENT_VERSION = new LevelDBProvider + .StoreVersion(1, 0); + + // just for integration tests that want to look at this file -- in general not sensible as + // a static + @VisibleForTesting + static YarnShuffleService instance; + // An entity that manages the shuffle secret per application // This is used only if authentication is enabled private ShuffleSecretManager secretManager; @@ -75,6 +104,8 @@ public class YarnShuffleService extends AuxiliaryService { // The actual server that serves shuffle files private TransportServer shuffleServer = null; + private Configuration _conf = null; + // Handles registering executors and opening shuffle blocks @VisibleForTesting ExternalShuffleBlockHandler blockHandler; @@ -83,14 +114,11 @@ public class YarnShuffleService extends AuxiliaryService { @VisibleForTesting File registeredExecutorFile; - // just for testing when you want to find an open port + // Where to store & reload application secrets for recovering state after an NM restart @VisibleForTesting - static int boundPort = -1; + File secretsFile; - // just for integration tests that want to look at this file -- in general not sensible as - // a static - @VisibleForTesting - static YarnShuffleService instance; + private DB db; public YarnShuffleService() { super("spark_shuffle"); @@ -112,42 +140,86 @@ private boolean isAuthenticationEnabled() { */ @Override protected void serviceInit(Configuration conf) { + _conf = conf; - // In case this NM was killed while there were running spark applications, we need to restore - // lost state for the existing executors. We look for an existing file in the NM's local dirs. - // If we don't find one, then we choose a file to use to save the state next time. Even if - // an application was stopped while the NM was down, we expect yarn to call stopApplication() - // when it comes back - registeredExecutorFile = - findRegisteredExecutorFile(conf.getTrimmedStrings("yarn.nodemanager.local-dirs")); - - TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - // If authentication is enabled, set up the shuffle server to use a - // special RPC handler that filters out unauthenticated fetch requests - boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); try { - blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); + // In case this NM was killed while there were running spark applications, we need to restore + // lost state for the existing executors. We look for an existing file in the NM's local dirs. + // If we don't find one, then we choose a file to use to save the state next time. Even if + // an application was stopped while the NM was down, we expect yarn to call stopApplication() + // when it comes back + registeredExecutorFile = findRecoveryDb(RECOVERY_FILE_NAME); + + TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); + blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); + + // If authentication is enabled, set up the shuffle server to use a + // special RPC handler that filters out unauthenticated fetch requests + boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); + List bootstraps = Lists.newArrayList(); + if (authEnabled) { + createSecretManager(); + bootstraps.add(new SaslServerBootstrap(transportConf, secretManager)); + } + + int port = conf.getInt( + SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); + TransportContext transportContext = new TransportContext(transportConf, blockHandler); + shuffleServer = transportContext.createServer(port, bootstraps); + // the port should normally be fixed, but for tests its useful to find an open port + port = shuffleServer.getPort(); + boundPort = port; + String authEnabledString = authEnabled ? "enabled" : "not enabled"; + logger.info("Started YARN shuffle service for Spark on port {}. " + + "Authentication is {}. Registered executor file is {}", port, authEnabledString, + registeredExecutorFile); } catch (Exception e) { logger.error("Failed to initialize external shuffle service", e); } + } + + private void createSecretManager() throws IOException { + secretManager = new ShuffleSecretManager(); + secretsFile = findRecoveryDb(SECRETS_RECOVERY_FILE_NAME); + + // Make sure this is protected in case its not in the NM recovery dir + FileSystem fs = FileSystem.getLocal(_conf); + fs.mkdirs(new Path(secretsFile.getPath()), new FsPermission((short)0700)); - List bootstraps = Lists.newArrayList(); - if (authEnabled) { - secretManager = new ShuffleSecretManager(); - bootstraps.add(new SaslServerBootstrap(transportConf, secretManager)); + db = LevelDBProvider.initLevelDB(secretsFile, CURRENT_VERSION, mapper); + logger.info("Recovery location is: " + secretsFile.getPath()); + if (db != null) { + logger.info("Going to reload spark shuffle data"); + DBIterator itr = db.iterator(); + itr.seek(APP_CREDS_KEY_PREFIX.getBytes(StandardCharsets.UTF_8)); + while (itr.hasNext()) { + Map.Entry e = itr.next(); + String key = new String(e.getKey(), StandardCharsets.UTF_8); + if (!key.startsWith(APP_CREDS_KEY_PREFIX)) { + break; + } + String id = parseDbAppKey(key); + ByteBuffer secret = mapper.readValue(e.getValue(), ByteBuffer.class); + logger.info("Reloading tokens for app: " + id); + secretManager.registerApp(id, secret); + } } + } - int port = conf.getInt( - SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - TransportContext transportContext = new TransportContext(transportConf, blockHandler); - shuffleServer = transportContext.createServer(port, bootstraps); - // the port should normally be fixed, but for tests its useful to find an open port - port = shuffleServer.getPort(); - boundPort = port; - String authEnabledString = authEnabled ? "enabled" : "not enabled"; - logger.info("Started YARN shuffle service for Spark on port {}. " + - "Authentication is {}. Registered executor file is {}", port, authEnabledString, - registeredExecutorFile); + private static String parseDbAppKey(String s) throws IOException { + if (!s.startsWith(APP_CREDS_KEY_PREFIX)) { + throw new IllegalArgumentException("expected a string starting with " + APP_CREDS_KEY_PREFIX); + } + String json = s.substring(APP_CREDS_KEY_PREFIX.length() + 1); + AppId parsed = mapper.readValue(json, AppId.class); + return parsed.appId; + } + + private static byte[] dbAppKey(AppId appExecId) throws IOException { + // we stick a common prefix on all the keys so we can find them in the DB + String appExecJson = mapper.writeValueAsString(appExecId); + String key = (APP_CREDS_KEY_PREFIX + ";" + appExecJson); + return key.getBytes(StandardCharsets.UTF_8); } @Override @@ -157,6 +229,12 @@ public void initializeApplication(ApplicationInitializationContext context) { ByteBuffer shuffleSecret = context.getApplicationDataForService(); logger.info("Initializing application {}", appId); if (isAuthenticationEnabled()) { + AppId fullId = new AppId(appId); + if (db != null) { + byte[] key = dbAppKey(fullId); + byte[] value = mapper.writeValueAsString(shuffleSecret).getBytes(StandardCharsets.UTF_8); + db.put(key, value); + } secretManager.registerApp(appId, shuffleSecret); } } catch (Exception e) { @@ -170,6 +248,14 @@ public void stopApplication(ApplicationTerminationContext context) { try { logger.info("Stopping application {}", appId); if (isAuthenticationEnabled()) { + AppId fullId = new AppId(appId); + if (db != null) { + try { + db.delete(dbAppKey(fullId)); + } catch (IOException e) { + logger.error("Error deleting {} from executor state db", appId, e); + } + } secretManager.unregisterApp(appId); } blockHandler.applicationRemoved(appId, false /* clean up local dirs */); @@ -190,14 +276,15 @@ public void stopContainer(ContainerTerminationContext context) { logger.info("Stopping container {}", containerId); } - private File findRegisteredExecutorFile(String[] localDirs) { + private File findRecoveryDb(String fileName) { + String[] localDirs = _conf.getTrimmedStrings("yarn.nodemanager.local-dirs"); for (String dir: localDirs) { - File f = new File(new Path(dir).toUri().getPath(), "registeredExecutors.ldb"); + File f = new File(new Path(dir).toUri().getPath(), fileName); if (f.exists()) { return f; } } - return new File(new Path(localDirs[0]).toUri().getPath(), "registeredExecutors.ldb"); + return new File(new Path(localDirs[0]).toUri().getPath(), fileName); } /** @@ -212,6 +299,9 @@ protected void serviceStop() { if (blockHandler != null) { blockHandler.close(); } + if (db != null) { + db.close(); + } } catch (Exception e) { logger.error("Exception when stopping service", e); } @@ -222,4 +312,38 @@ protected void serviceStop() { public ByteBuffer getMetaData() { return ByteBuffer.allocate(0); } + + /** + * Simply encodes an application ID. + */ + public static class AppId { + public final String appId; + + @JsonCreator + public AppId(@JsonProperty("appId") String appId) { + this.appId = appId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AppId appExecId = (AppId) o; + return Objects.equal(appId, appExecId.appId); + } + + @Override + public int hashCode() { + return Objects.hashCode(appId); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .toString(); + } + } + } diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index c33a9e6bbe255..0c2e0204356b2 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext} import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.apache.spark.SecurityManager import org.apache.spark.SparkFunSuite import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo @@ -77,6 +78,8 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd test("executor state kept across NM restart") { s1 = new YarnShuffleService + // set auth to true to test the secrets recovery + yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, true) s1.init(yarnConfig) val app1Id = ApplicationId.newInstance(0, 1) val app1Data: ApplicationInitializationContext = @@ -89,6 +92,8 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd val execStateFile = s1.registeredExecutorFile execStateFile should not be (null) + val secretsFile = s1.secretsFile + secretsFile should not be (null) val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER) val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER) @@ -118,6 +123,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd s1.stop() s2 = new YarnShuffleService s2.init(yarnConfig) + s2.secretsFile should be (secretsFile) s2.registeredExecutorFile should be (execStateFile) val handler2 = s2.blockHandler @@ -135,6 +141,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd s3 = new YarnShuffleService s3.init(yarnConfig) s3.registeredExecutorFile should be (execStateFile) + s3.secretsFile should be (secretsFile) val handler3 = s3.blockHandler val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) @@ -148,7 +155,10 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd test("removed applications should not be in registered executor file") { s1 = new YarnShuffleService + yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, false) s1.init(yarnConfig) + val secretsFile = s1.secretsFile + secretsFile should be (null) val app1Id = ApplicationId.newInstance(0, 1) val app1Data: ApplicationInitializationContext = new ApplicationInitializationContext("user", app1Id, null) From c6e0dd1d46f40cd0451155ee9730f429fe212a27 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 8 Sep 2016 08:22:58 -0700 Subject: [PATCH 1276/1470] [SPARK-17442][SPARKR] Additional arguments in write.df are not passed to data source ## What changes were proposed in this pull request? additional options were not passed down in write.df. ## How was this patch tested? unit tests falaki shivaram Author: Felix Cheung Closes #15010 from felixcheung/testreadoptions. (cherry picked from commit f0d21b7f90cdcce353ab6fc279b9cc376e46e536) Signed-off-by: Shivaram Venkataraman --- R/pkg/R/DataFrame.R | 1 + R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 +++++++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 8aea22896cbc0..a5bd603376010 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2628,6 +2628,7 @@ setMethod("write.df", write <- callJMethod(df@sdf, "write") write <- callJMethod(write, "format", source) write <- callJMethod(write, "mode", jmode) + write <- callJMethod(write, "options", options) write <- callJMethod(write, "save", path) }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index dddc15f58cc35..cdb8ff6b6f8c9 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -208,7 +208,7 @@ test_that("create DataFrame from RDD", { unsetHiveContext() }) -test_that("read csv as DataFrame", { +test_that("read/write csv as DataFrame", { csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") mockLinesCsv <- c("year,make,model,comment,blank", "\"2012\",\"Tesla\",\"S\",\"No comment\",", @@ -243,7 +243,17 @@ test_that("read csv as DataFrame", { expect_equal(count(withoutna2), 3) expect_equal(count(where(withoutna2, withoutna2$make == "Dummy")), 0) + # writing csv file + csvPath2 <- tempfile(pattern = "csvtest2", fileext = ".csv") + write.df(df2, path = csvPath2, "csv", header = "true") + df3 <- read.df(csvPath2, "csv", header = "true") + expect_equal(nrow(df3), nrow(df2)) + expect_equal(colnames(df3), colnames(df2)) + csv <- read.csv(file = list.files(csvPath2, pattern = "^part", full.names = T)[[1]]) + expect_equal(colnames(df3), colnames(csv)) + unlink(csvPath) + unlink(csvPath2) }) test_that("convert NAs to null type in DataFrames", { From a7f1c18988a76b2cb970c5e0af64cc070c9ac67c Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 9 Sep 2016 05:35:10 -0700 Subject: [PATCH 1277/1470] [SPARK-17456][CORE] Utility for parsing Spark versions ## What changes were proposed in this pull request? This patch adds methods for extracting major and minor versions as Int types in Scala from a Spark version string. Motivation: There are many hacks within Spark's codebase to identify and compare Spark versions. We should add a simple utility to standardize these code paths, especially since there have been mistakes made in the past. This will let us add unit tests as well. Currently, I want this functionality to check Spark versions to provide backwards compatibility for ML model persistence. ## How was this patch tested? Unit tests Author: Joseph K. Bradley Closes #15017 from jkbradley/version-parsing. (cherry picked from commit 65b814bf50e92e2e9b622d1602f18bacd217181c) Signed-off-by: Joseph K. Bradley --- .../org/apache/spark/util/VersionUtils.scala | 52 +++++++++++++ .../apache/spark/util/VersionUtilsSuite.scala | 76 +++++++++++++++++++ 2 files changed, 128 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/util/VersionUtils.scala create mode 100644 core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/VersionUtils.scala b/core/src/main/scala/org/apache/spark/util/VersionUtils.scala new file mode 100644 index 0000000000000..828153b868420 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/VersionUtils.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.util + +/** + * Utilities for working with Spark version strings + */ +private[spark] object VersionUtils { + + private val majorMinorRegex = """^(\d+)\.(\d+)(\..*)?$""".r + + /** + * Given a Spark version string, return the major version number. + * E.g., for 2.0.1-SNAPSHOT, return 2. + */ + def majorVersion(sparkVersion: String): Int = majorMinorVersion(sparkVersion)._1 + + /** + * Given a Spark version string, return the minor version number. + * E.g., for 2.0.1-SNAPSHOT, return 0. + */ + def minorVersion(sparkVersion: String): Int = majorMinorVersion(sparkVersion)._2 + + /** + * Given a Spark version string, return the (major version number, minor version number). + * E.g., for 2.0.1-SNAPSHOT, return (2, 0). + */ + def majorMinorVersion(sparkVersion: String): (Int, Int) = { + majorMinorRegex.findFirstMatchIn(sparkVersion) match { + case Some(m) => + (m.group(1).toInt, m.group(2).toInt) + case None => + throw new IllegalArgumentException(s"Spark tried to parse '$sparkVersion' as a Spark" + + s" version string, but it could not find the major and minor version numbers.") + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala new file mode 100644 index 0000000000000..aaf79ebd4f9fc --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala @@ -0,0 +1,76 @@ +/* + * 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.util + +import org.apache.spark.SparkFunSuite + +class VersionUtilsSuite extends SparkFunSuite { + + import org.apache.spark.util.VersionUtils._ + + test("Parse Spark major version") { + assert(majorVersion("2.0") === 2) + assert(majorVersion("12.10.11") === 12) + assert(majorVersion("2.0.1-SNAPSHOT") === 2) + assert(majorVersion("2.0.x") === 2) + withClue("majorVersion parsing should fail for invalid major version number") { + intercept[IllegalArgumentException] { + majorVersion("2z.0") + } + } + withClue("majorVersion parsing should fail for invalid minor version number") { + intercept[IllegalArgumentException] { + majorVersion("2.0z") + } + } + } + + test("Parse Spark minor version") { + assert(minorVersion("2.0") === 0) + assert(minorVersion("12.10.11") === 10) + assert(minorVersion("2.0.1-SNAPSHOT") === 0) + assert(minorVersion("2.0.x") === 0) + withClue("minorVersion parsing should fail for invalid major version number") { + intercept[IllegalArgumentException] { + minorVersion("2z.0") + } + } + withClue("minorVersion parsing should fail for invalid minor version number") { + intercept[IllegalArgumentException] { + minorVersion("2.0z") + } + } + } + + test("Parse Spark major and minor versions") { + assert(majorMinorVersion("2.0") === (2, 0)) + assert(majorMinorVersion("12.10.11") === (12, 10)) + assert(majorMinorVersion("2.0.1-SNAPSHOT") === (2, 0)) + assert(majorMinorVersion("2.0.x") === (2, 0)) + withClue("majorMinorVersion parsing should fail for invalid major version number") { + intercept[IllegalArgumentException] { + majorMinorVersion("2z.0") + } + } + withClue("majorMinorVersion parsing should fail for invalid minor version number") { + intercept[IllegalArgumentException] { + majorMinorVersion("2.0z") + } + } + } +} From 6f02f40b4b4145d0cd7997b413a13ebcc12ec510 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 9 Sep 2016 14:23:05 -0700 Subject: [PATCH 1278/1470] [SPARK-17354] [SQL] Partitioning by dates/timestamps should work with Parquet vectorized reader This PR fixes `ColumnVectorUtils.populate` so that Parquet vectorized reader can read partitioned table with dates/timestamps. This works fine with Parquet normal reader. This is being only called within [VectorizedParquetRecordReader.java#L185](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java#L185). When partition column types are explicitly given to `DateType` or `TimestampType` (rather than inferring the type of partition column), this fails with the exception below: ``` 16/09/01 10:30:07 ERROR Executor: Exception in task 0.0 in stage 5.0 (TID 6) java.lang.ClassCastException: java.lang.Integer cannot be cast to java.sql.Date at org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate(ColumnVectorUtils.java:89) at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:185) at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:204) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReader$1.apply(ParquetFileFormat.scala:362) ... ``` Unit tests in `SQLQuerySuite`. Author: hyukjinkwon Closes #14919 from HyukjinKwon/SPARK-17354. (cherry picked from commit f7d2143705c8c1baeed0bc62940f9dba636e705b) Signed-off-by: Davies Liu --- .../vectorized/ColumnVectorUtils.java | 5 +- .../execution/vectorized/ColumnarBatch.java | 6 +++ .../datasources/parquet/ParquetIOSuite.scala | 49 ++++++++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 21 ++++++++ 4 files changed, 78 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java index 2fa476b9cfb71..900d7c431e723 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java @@ -86,8 +86,9 @@ public static void populate(ColumnVector col, InternalRow row, int fieldIdx) { col.getChildColumn(0).putInts(0, capacity, c.months); col.getChildColumn(1).putLongs(0, capacity, c.microseconds); } else if (t instanceof DateType) { - Date date = (Date)row.get(fieldIdx, t); - col.putInts(0, capacity, DateTimeUtils.fromJavaDate(date)); + col.putInts(0, capacity, row.getInt(fieldIdx)); + } else if (t instanceof TimestampType) { + col.putLongs(0, capacity, row.getLong(fieldIdx)); } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index f3afa8f938f86..62abc2a821a3a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -137,6 +137,10 @@ public InternalRow copy() { DataType dt = columns[i].dataType(); if (dt instanceof BooleanType) { row.setBoolean(i, getBoolean(i)); + } else if (dt instanceof ByteType) { + row.setByte(i, getByte(i)); + } else if (dt instanceof ShortType) { + row.setShort(i, getShort(i)); } else if (dt instanceof IntegerType) { row.setInt(i, getInt(i)); } else if (dt instanceof LongType) { @@ -154,6 +158,8 @@ public InternalRow copy() { row.setDecimal(i, getDecimal(i, t.precision(), t.scale()), t.precision()); } else if (dt instanceof DateType) { row.setInt(i, getInt(i)); + } else if (dt instanceof TimestampType) { + row.setLong(i, getLong(i)); } else { throw new RuntimeException("Not implemented. " + dt); } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index a95de2ea9135f..46ccfa53bd79d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -38,11 +38,12 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport // with an empty configuration (it is after all not intended to be used in this way?) @@ -677,6 +678,52 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } } } + + test("VectorizedParquetRecordReader - partition column types") { + withTempPath { dir => + Seq(1).toDF().repartition(1).write.parquet(dir.getCanonicalPath) + + val dataTypes = + Seq(StringType, BooleanType, ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType(25, 5), DateType, TimestampType) + + val constantValues = + Seq( + UTF8String.fromString("a string"), + true, + 1.toByte, + 2.toShort, + 3, + Long.MaxValue, + 0.25.toFloat, + 0.75D, + Decimal("1234.23456"), + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("2015-01-01")), + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"))) + + dataTypes.zip(constantValues).foreach { case (dt, v) => + val schema = StructType(StructField("pcol", dt) :: Nil) + val vectorizedReader = new VectorizedParquetRecordReader + val partitionValues = new GenericMutableRow(Array(v)) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) + + try { + vectorizedReader.initialize(file, null) + vectorizedReader.initBatch(schema, partitionValues) + vectorizedReader.nextKeyValue() + val row = vectorizedReader.getCurrentValue.asInstanceOf[InternalRow] + + // Use `GenericMutableRow` by explicitly copying rather than `ColumnarBatch` + // in order to use get(...) method which is not implemented in `ColumnarBatch`. + val actual = row.copy().get(1, dt) + val expected = v + assert(actual == expected) + } finally { + vectorizedReader.close() + } + } + } + } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) 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 404b27596fa4e..51d49469b3f46 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 @@ -1700,6 +1700,27 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-17354: Partitioning by dates/timestamps works with Parquet vectorized reader") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + sql( + """CREATE TABLE order(id INT) + |PARTITIONED BY (pd DATE, pt TIMESTAMP) + |STORED AS PARQUET + """.stripMargin) + + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql( + """INSERT INTO TABLE order PARTITION(pd, pt) + |SELECT 1 AS id, CAST('1990-02-24' AS DATE) AS pd, CAST('1990-02-24' AS TIMESTAMP) AS pt + """.stripMargin) + val actual = sql("SELECT * FROM order") + val expected = sql( + "SELECT 1 AS id, CAST('1990-02-24' AS DATE) AS pd, CAST('1990-02-24' AS TIMESTAMP) AS pt") + checkAnswer(actual, expected) + sql("DROP TABLE order") + } + } + def testCommandAvailable(command: String): Boolean = { Try(Process(command) !!).isSuccess } From c2378a6821e74cce52b853e7c556044b922625b1 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 10 Sep 2016 10:18:53 +0100 Subject: [PATCH 1279/1470] [SPARK-17396][CORE] Share the task support between UnionRDD instances. ## What changes were proposed in this pull request? Share the ForkJoinTaskSupport between UnionRDD instances to avoid creating a huge number of threads if lots of RDDs are created at the same time. ## How was this patch tested? This uses existing UnionRDD tests. Author: Ryan Blue Closes #14985 from rdblue/SPARK-17396-use-shared-pool. (cherry picked from commit 6ea5055fa734d435b5f148cf52d3385a57926b60) Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/rdd/UnionRDD.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 8171dcc046379..ad1fddbde7b00 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer -import scala.collection.parallel.ForkJoinTaskSupport +import scala.collection.parallel.{ForkJoinTaskSupport, ThreadPoolTaskSupport} import scala.concurrent.forkjoin.ForkJoinPool import scala.reflect.ClassTag @@ -58,6 +58,11 @@ private[spark] class UnionPartition[T: ClassTag]( } } +object UnionRDD { + private[spark] lazy val partitionEvalTaskSupport = + new ForkJoinTaskSupport(new ForkJoinPool(8)) +} + @DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @@ -68,13 +73,10 @@ class UnionRDD[T: ClassTag]( private[spark] val isPartitionListingParallel: Boolean = rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) - @transient private lazy val partitionEvalTaskSupport = - new ForkJoinTaskSupport(new ForkJoinPool(8)) - override def getPartitions: Array[Partition] = { val parRDDs = if (isPartitionListingParallel) { val parArray = rdds.par - parArray.tasksupport = partitionEvalTaskSupport + parArray.tasksupport = UnionRDD.partitionEvalTaskSupport parArray } else { rdds From bde54526845a315de5b5c8ee3eae9f3d14debac5 Mon Sep 17 00:00:00 2001 From: Timothy Hunter Date: Sun, 11 Sep 2016 08:03:45 +0100 Subject: [PATCH 1280/1470] [SPARK-17439][SQL] Fixing compression issues with approximate quantiles and adding more tests This PR build on #14976 and fixes a correctness bug that would cause the wrong quantile to be returned for small target errors. This PR adds 8 unit tests that were failing without the fix. Author: Timothy Hunter Author: Sean Owen Closes #15002 from thunterdb/ml-1783. (cherry picked from commit 180796ecb3a00facde2d98affdb5aa38dd258875) Signed-off-by: Sean Owen --- .../sql/execution/stat/StatFunctions.scala | 26 +++++++++++------ .../execution/stat/ApproxQuantileSuite.scala | 29 +++++++++++++++++-- 2 files changed, 44 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index 50eecb409830f..7e2ebe856bc89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.stat -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, ListBuffer} import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} @@ -119,7 +119,7 @@ object StatFunctions extends Logging { class QuantileSummaries( val compressThreshold: Int, val relativeError: Double, - val sampled: ArrayBuffer[Stats] = ArrayBuffer.empty, + val sampled: Array[Stats] = Array.empty, private[stat] var count: Long = 0L, val headSampled: ArrayBuffer[Double] = ArrayBuffer.empty) extends Serializable { @@ -134,7 +134,12 @@ object StatFunctions extends Logging { def insert(x: Double): QuantileSummaries = { headSampled.append(x) if (headSampled.size >= defaultHeadSize) { - this.withHeadBufferInserted + val result = this.withHeadBufferInserted + if (result.sampled.length >= compressThreshold) { + result.compress() + } else { + result + } } else { this } @@ -186,7 +191,7 @@ object StatFunctions extends Logging { newSamples.append(sampled(sampleIdx)) sampleIdx += 1 } - new QuantileSummaries(compressThreshold, relativeError, newSamples, currentCount) + new QuantileSummaries(compressThreshold, relativeError, newSamples.toArray, currentCount) } /** @@ -305,10 +310,10 @@ object StatFunctions extends Logging { private def compressImmut( currentSamples: IndexedSeq[Stats], - mergeThreshold: Double): ArrayBuffer[Stats] = { - val res: ArrayBuffer[Stats] = ArrayBuffer.empty + mergeThreshold: Double): Array[Stats] = { + val res = ListBuffer.empty[Stats] if (currentSamples.isEmpty) { - return res + return res.toArray } // Start for the last element, which is always part of the set. // The head contains the current new head, that may be merged with the current element. @@ -331,8 +336,11 @@ object StatFunctions extends Logging { } res.prepend(head) // If necessary, add the minimum element: - res.prepend(currentSamples.head) - res + val currHead = currentSamples.head + if (currHead.value < head.value) { + res.prepend(currentSamples.head) + } + res.toArray } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/stat/ApproxQuantileSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/stat/ApproxQuantileSuite.scala index 0a989d026ce1c..8bd6b3c5cdc8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/stat/ApproxQuantileSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/stat/ApproxQuantileSuite.scala @@ -42,6 +42,20 @@ class ApproxQuantileSuite extends SparkFunSuite { summary.compress() } + /** + * Interleaves compression and insertions. + */ + private def buildCompressSummary( + data: Seq[Double], + epsi: Double, + threshold: Int): QuantileSummaries = { + var summary = new QuantileSummaries(threshold, epsi) + data.foreach { x => + summary = summary.insert(x).compress() + } + summary + } + private def checkQuantile(quant: Double, data: Seq[Double], summary: QuantileSummaries): Unit = { val approx = summary.query(quant) // The rank of the approximation. @@ -56,8 +70,8 @@ class ApproxQuantileSuite extends SparkFunSuite { for { (seq_name, data) <- Seq(increasing, decreasing, random) - epsi <- Seq(0.1, 0.0001) - compression <- Seq(1000, 10) + epsi <- Seq(0.1, 0.0001) // With a significant value and with full precision + compression <- Seq(1000, 10) // This interleaves n so that we test without and with compression } { test(s"Extremas with epsi=$epsi and seq=$seq_name, compression=$compression") { @@ -77,6 +91,17 @@ class ApproxQuantileSuite extends SparkFunSuite { checkQuantile(0.1, data, s) checkQuantile(0.001, data, s) } + + test(s"Some quantile values with epsi=$epsi and seq=$seq_name, compression=$compression " + + s"(interleaved)") { + val s = buildCompressSummary(data, epsi, compression) + assert(s.count == data.size, s"Found count=${s.count} but data size=${data.size}") + checkQuantile(0.9999, data, s) + checkQuantile(0.9, data, s) + checkQuantile(0.5, data, s) + checkQuantile(0.1, data, s) + checkQuantile(0.001, data, s) + } } // Tests for merging procedure From d293062a4cd04acf48697b1fd6a70fef97b338da Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Sun, 11 Sep 2016 10:19:39 +0100 Subject: [PATCH 1281/1470] [SPARK-17336][PYSPARK] Fix appending multiple times to PYTHONPATH from spark-config.sh ## What changes were proposed in this pull request? During startup of Spark standalone, the script file spark-config.sh appends to the PYTHONPATH and can be sourced many times, causing duplicates in the path. This change adds a env flag that is set when the PYTHONPATH is appended so it will happen only one time. ## How was this patch tested? Manually started standalone master/worker and verified PYTHONPATH has no duplicate entries. Author: Bryan Cutler Closes #15028 from BryanCutler/fix-duplicate-pythonpath-SPARK-17336. (cherry picked from commit c76baff0cc4775c2191d075cc9a8176e4915fec8) Signed-off-by: Sean Owen --- sbin/spark-config.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index a7a44cdde6c77..b7284487c511d 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -26,5 +26,8 @@ fi export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: -export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.3-src.zip:${PYTHONPATH}" +if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then + export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.3-src.zip:${PYTHONPATH}" + export PYSPARK_PYTHONPATH_SET=1 +fi From 30521522dbe65504876f0302030ef84945ad98b5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 11 Sep 2016 21:51:22 -0700 Subject: [PATCH 1282/1470] [SPARK-17486] Remove unused TaskMetricsUIData.updatedBlockStatuses field The `TaskMetricsUIData.updatedBlockStatuses` field is assigned to but never read, increasing the memory consumption of the web UI. We should remove this field. Author: Josh Rosen Closes #15038 from JoshRosen/remove-updated-block-statuses-from-TaskMetricsUIData. (cherry picked from commit 72eec70bdbf6fb67c977463db5d8d95dd3040ae8) Signed-off-by: Shixiong Zhu --- core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 4f8505058caad..818605003eaf4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -23,7 +23,6 @@ import scala.collection.mutable.{HashMap, LinkedHashMap} import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} -import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.AccumulatorContext import org.apache.spark.util.collection.OpenHashSet @@ -142,7 +141,6 @@ private[spark] object UIData { memoryBytesSpilled = m.memoryBytesSpilled, diskBytesSpilled = m.diskBytesSpilled, peakExecutionMemory = m.peakExecutionMemory, - updatedBlockStatuses = m.updatedBlockStatuses.toList, inputMetrics = InputMetricsUIData(m.inputMetrics.bytesRead, m.inputMetrics.recordsRead), outputMetrics = OutputMetricsUIData(m.outputMetrics.bytesWritten, m.outputMetrics.recordsWritten), @@ -190,7 +188,6 @@ private[spark] object UIData { memoryBytesSpilled: Long, diskBytesSpilled: Long, peakExecutionMemory: Long, - updatedBlockStatuses: Seq[(BlockId, BlockStatus)], inputMetrics: InputMetricsUIData, outputMetrics: OutputMetricsUIData, shuffleReadMetrics: ShuffleReadMetricsUIData, From 0a36e360cd4bb2c66687caf017fbeeece41a7ccd Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Mon, 12 Sep 2016 11:30:06 -0700 Subject: [PATCH 1283/1470] [SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the whole RDD in memory ## What changes were proposed in this pull request? MemoryStore may throw OutOfMemoryError when trying to cache a super big RDD that cannot fit in memory. ``` scala> sc.parallelize(1 to 1000000000, 100).map(x => new Array[Long](1000)).cache().count() java.lang.OutOfMemoryError: Java heap space at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:24) at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:23) at scala.collection.Iterator$$anon$11.next(Iterator.scala:409) at scala.collection.Iterator$JoinIterator.next(Iterator.scala:232) at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:683) at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43) at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1684) at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134) at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134) at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915) at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:86) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` Spark MemoryStore uses SizeTrackingVector as a temporary unrolling buffer to store all input values that it has read so far before transferring the values to storage memory cache. The problem is that when the input RDD is too big for caching in memory, the temporary unrolling memory SizeTrackingVector is not garbage collected in time. As SizeTrackingVector can occupy all available storage memory, it may cause the executor JVM to run out of memory quickly. More info can be found at https://issues.apache.org/jira/browse/SPARK-17503 ## How was this patch tested? Unit test and manual test. ### Before change Heap memory consumption screen shot 2016-09-12 at 4 16 15 pm Heap dump screen shot 2016-09-12 at 4 34 19 pm ### After change Heap memory consumption screen shot 2016-09-12 at 4 29 10 pm Author: Sean Zhong Closes #15056 from clockfly/memory_store_leak. (cherry picked from commit 1742c3ab86d75ce3d352f7cddff65e62fb7c8dd4) Signed-off-by: Josh Rosen --- .../spark/storage/memory/MemoryStore.scala | 40 +++++++----- .../PartiallyUnrolledIteratorSuite.scala | 61 +++++++++++++++++++ 2 files changed, 87 insertions(+), 14 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 0349da0d8aa00..5fd4e886ec483 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -661,31 +661,43 @@ private[spark] class MemoryStore( private[storage] class PartiallyUnrolledIterator[T]( memoryStore: MemoryStore, unrollMemory: Long, - unrolled: Iterator[T], + private[this] var unrolled: Iterator[T], rest: Iterator[T]) extends Iterator[T] { - private[this] var unrolledIteratorIsConsumed: Boolean = false - private[this] var iter: Iterator[T] = { - val completionIterator = CompletionIterator[T, Iterator[T]](unrolled, { - unrolledIteratorIsConsumed = true - memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) - }) - completionIterator ++ rest + private def releaseUnrollMemory(): Unit = { + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) + // SPARK-17503: Garbage collects the unrolling memory before the life end of + // PartiallyUnrolledIterator. + unrolled = null } - override def hasNext: Boolean = iter.hasNext - override def next(): T = iter.next() + override def hasNext: Boolean = { + if (unrolled == null) { + rest.hasNext + } else if (!unrolled.hasNext) { + releaseUnrollMemory() + rest.hasNext + } else { + true + } + } + + override def next(): T = { + if (unrolled == null) { + rest.next() + } else { + unrolled.next() + } + } /** * Called to dispose of this iterator and free its memory. */ def close(): Unit = { - if (!unrolledIteratorIsConsumed) { - memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) - unrolledIteratorIsConsumed = true + if (unrolled != null) { + releaseUnrollMemory() } - iter = null } } diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala new file mode 100644 index 0000000000000..02c2331dc3946 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.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.storage + +import org.mockito.Matchers +import org.mockito.Mockito._ +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode.ON_HEAP +import org.apache.spark.storage.memory.{MemoryStore, PartiallyUnrolledIterator} + +class PartiallyUnrolledIteratorSuite extends SparkFunSuite with MockitoSugar { + test("join two iterators") { + val unrollSize = 1000 + val unroll = (0 until unrollSize).iterator + val restSize = 500 + val rest = (unrollSize until restSize + unrollSize).iterator + + val memoryStore = mock[MemoryStore] + val joinIterator = new PartiallyUnrolledIterator(memoryStore, unrollSize, unroll, rest) + + // Firstly iterate over unrolling memory iterator + (0 until unrollSize).foreach { value => + assert(joinIterator.hasNext) + assert(joinIterator.hasNext) + assert(joinIterator.next() == value) + } + + joinIterator.hasNext + joinIterator.hasNext + verify(memoryStore, times(1)) + .releaseUnrollMemoryForThisTask(Matchers.eq(ON_HEAP), Matchers.eq(unrollSize.toLong)) + + // Secondly, iterate over rest iterator + (unrollSize until unrollSize + restSize).foreach { value => + assert(joinIterator.hasNext) + assert(joinIterator.hasNext) + assert(joinIterator.next() == value) + } + + joinIterator.close() + // MemoryMode.releaseUnrollMemoryForThisTask is called only once + verifyNoMoreInteractions(memoryStore) + } +} From 37f45bf0d95f463c38e5636690545472c0399222 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Sep 2016 15:24:33 -0700 Subject: [PATCH 1284/1470] [SPARK-14818] Post-2.0 MiMa exclusion and build changes This patch makes a handful of post-Spark-2.0 MiMa exclusion and build updates. It should be merged to master and a subset of it should be picked into branch-2.0 in order to test Spark 2.0.1-SNAPSHOT. - Remove the ` sketch`, `mllibLocal`, and `streamingKafka010` from the list of excluded subprojects so that MiMa checks them. - Remove now-unnecessary special-case handling of the Kafka 0.8 artifact in `mimaSettings`. - Move the exclusion added in SPARK-14743 from `v20excludes` to `v21excludes`, since that patch was only merged into master and not branch-2.0. - Add exclusions for an API change introduced by SPARK-17096 / #14675. - Add missing exclusions for the `o.a.spark.internal` and `o.a.spark.sql.internal` packages. Author: Josh Rosen Closes #15061 from JoshRosen/post-2.0-mima-changes. (cherry picked from commit 7c51b99a428a965ff7d136e1cdda20305d260453) Signed-off-by: Josh Rosen --- project/MimaBuild.scala | 11 ++--------- project/MimaExcludes.scala | 7 +++++++ project/SparkBuild.scala | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 2a989dd4f7a1d..77397eab81ede 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -88,15 +88,8 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" - val previousSparkVersion = "1.6.0" - // This check can be removed post-2.0 - val project = if (previousSparkVersion == "1.6.0" && - projectRef.project == "streaming-kafka-0-8" - ) { - "streaming-kafka" - } else { - projectRef.project - } + val previousSparkVersion = "2.0.0" + val project = projectRef.project val fullId = "spark-" + project + "_2.11" mimaDefaultSettings ++ Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4bd6156288592..c38a49ae6dae3 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -42,12 +42,15 @@ object MimaExcludes { Seq( excludePackage("org.apache.spark.rpc"), excludePackage("org.spark-project.jetty"), + excludePackage("org.spark_project.jetty"), + excludePackage("org.apache.spark.internal"), excludePackage("org.apache.spark.unused"), excludePackage("org.apache.spark.unsafe"), excludePackage("org.apache.spark.memory"), excludePackage("org.apache.spark.util.collection.unsafe"), excludePackage("org.apache.spark.sql.catalyst"), excludePackage("org.apache.spark.sql.execution"), + excludePackage("org.apache.spark.sql.internal"), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"), ProblemFilters.exclude[MissingMethodProblem]( @@ -777,6 +780,10 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.jdbc"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema") + ) ++ Seq( + // SPARK-17096: Improve exception string reported through the StreamingQueryListener + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.stackTrace"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.this") ) } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b1a9f393423bd..133d3b390e91a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -352,7 +352,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, sketch, mllibLocal, streamingKafka010 + unsafe, tags ).contains(x) } From a3fc5762b896e6531a66802dbfe583c98eccc42b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Sep 2016 15:43:57 -0700 Subject: [PATCH 1285/1470] [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job ## What changes were proposed in this pull request? In Spark's `RDD.getOrCompute` we first try to read a local copy of a cached RDD block, then a remote copy, and only fall back to recomputing the block if no cached copy (local or remote) can be read. This logic works correctly in the case where no remote copies of the block exist, but if there _are_ remote copies and reads of those copies fail (due to network issues or internal Spark bugs) then the BlockManager will throw a `BlockFetchException` that will fail the task (and which could possibly fail the whole job if the read failures keep occurring). In the cases of TorrentBroadcast and task result fetching we really do want to fail the entire job in case no remote blocks can be fetched, but this logic is inappropriate for reads of cached RDD blocks because those can/should be recomputed in case cached blocks are unavailable. Therefore, I think that the `BlockManager.getRemoteBytes()` method should never throw on remote fetch errors and, instead, should handle failures by returning `None`. ## How was this patch tested? Block manager changes should be covered by modified tests in `BlockManagerSuite`: the old tests expected exceptions to be thrown on failed remote reads, while the modified tests now expect `None` to be returned from the `getRemote*` method. I also manually inspected all usages of `BlockManager.getRemoteValues()`, `getRemoteBytes()`, and `get()` to verify that they correctly pattern-match on the result and handle `None`. Note that these `None` branches are already exercised because the old `getRemoteBytes` returned `None` when no remote locations for the block could be found (which could occur if an executor died and its block manager de-registered with the master). Author: Josh Rosen Closes #15037 from JoshRosen/SPARK-17485. (cherry picked from commit f9c580f11098d95f098936a0b90fa21d71021205) Signed-off-by: Josh Rosen --- .../spark/storage/BlockFetchException.scala | 24 ------------------- .../apache/spark/storage/BlockManager.scala | 5 ++-- .../spark/storage/BlockManagerSuite.scala | 10 +++----- 3 files changed, 6 insertions(+), 33 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala deleted file mode 100644 index f6e46ae9a481a..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala +++ /dev/null @@ -1,24 +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.storage - -import org.apache.spark.SparkException - -private[spark] -case class BlockFetchException(messages: String, throwable: Throwable) - extends SparkException(messages, throwable) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 95107d325946c..cd85d97a80b11 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -564,8 +564,9 @@ private[spark] class BlockManager( // Give up trying anymore locations. Either we've tried all of the original locations, // or we've refreshed the list of locations from the master, and have still // hit failures after trying locations from the refreshed list. - throw new BlockFetchException(s"Failed to fetch block after" + - s" ${totalFailureCount} fetch failures. Most recent failure cause:", e) + logWarning(s"Failed to fetch block after $totalFailureCount fetch failures. " + + s"Most recent failure cause:", e) + return None } logWarning(s"Failed to fetch remote block $blockId " + 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 6821582254f5b..2f594b8a52beb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -511,10 +511,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") store3.stop() store3 = null - // exception throw because there is no locations - intercept[BlockFetchException] { - store.getRemoteBytes("list1") - } + // Should return None instead of throwing an exception: + assert(store.getRemoteBytes("list1").isEmpty) } test("SPARK-14252: getOrElseUpdate should still read from remote storage") { @@ -1184,9 +1182,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) - intercept[BlockFetchException] { - store.getRemoteBytes("item") - } + assert(store.getRemoteBytes("item").isEmpty) } test("SPARK-13328: refresh block locations (fetch should succeed after location refresh)") { From 1f72e774bbfb1f57d79ef798c957cdcf1278409a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 12 Sep 2016 16:35:42 -0700 Subject: [PATCH 1286/1470] [SPARK-17474] [SQL] fix python udf in TakeOrderedAndProjectExec ## What changes were proposed in this pull request? When there is any Python UDF in the Project between Sort and Limit, it will be collected into TakeOrderedAndProjectExec, ExtractPythonUDFs failed to pull the Python UDFs out because QueryPlan.expressions does not include the expression inside Option[Seq[Expression]]. Ideally, we should fix the `QueryPlan.expressions`, but tried with no luck (it always run into infinite loop). In PR, I changed the TakeOrderedAndProjectExec to no use Option[Seq[Expression]] to workaround it. cc JoshRosen ## How was this patch tested? Added regression test. Author: Davies Liu Closes #15030 from davies/all_expr. (cherry picked from commit a91ab705e8c124aa116c3e5b1f3ba88ce832dcde) Signed-off-by: Davies Liu --- python/pyspark/sql/tests.py | 8 ++++++++ .../apache/spark/sql/execution/SparkStrategies.scala | 8 ++++---- .../scala/org/apache/spark/sql/execution/limit.scala | 12 ++++++------ .../sql/execution/TakeOrderedAndProjectSuite.scala | 4 ++-- 4 files changed, 20 insertions(+), 12 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 520b09d9c6f16..c7f312e465b51 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -371,6 +371,14 @@ def test_udf_in_generate(self): row = df.select(explode(f(*df))).groupBy().sum().first() self.assertEqual(row[0], 10) + def test_udf_with_order_by_and_limit(self): + from pyspark.sql.functions import udf + my_copy = udf(lambda x: x, IntegerType()) + df = self.spark.range(10).orderBy("id") + res = df.select(df.id, my_copy(df.id).alias("copy")).limit(1) + res.explain(True) + self.assertEqual(res.collect(), [Row(id=0, copy=0)]) + def test_basic_functions(self): rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) df = self.spark.read.json(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 6d7c193fd42c8..e7faab5495421 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -66,22 +66,22 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ReturnAnswer(rootPlan) => rootPlan match { case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => - execution.TakeOrderedAndProjectExec(limit, order, None, planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil case logical.Limit( IntegerLiteral(limit), logical.Project(projectList, logical.Sort(order, true, child))) => execution.TakeOrderedAndProjectExec( - limit, order, Some(projectList), planLater(child)) :: Nil + limit, order, projectList, planLater(child)) :: Nil case logical.Limit(IntegerLiteral(limit), child) => execution.CollectLimitExec(limit, planLater(child)) :: Nil case other => planLater(other) :: Nil } case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => - execution.TakeOrderedAndProjectExec(limit, order, None, planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil case logical.Limit( IntegerLiteral(limit), logical.Project(projectList, logical.Sort(order, true, child))) => execution.TakeOrderedAndProjectExec( - limit, order, Some(projectList), planLater(child)) :: Nil + limit, order, projectList, planLater(child)) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 781c016095427..01fbe5b7c2c03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -114,11 +114,11 @@ case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { case class TakeOrderedAndProjectExec( limit: Int, sortOrder: Seq[SortOrder], - projectList: Option[Seq[NamedExpression]], + projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = { - projectList.map(_.map(_.toAttribute)).getOrElse(child.output) + projectList.map(_.toAttribute) } override def outputPartitioning: Partitioning = SinglePartition @@ -126,8 +126,8 @@ case class TakeOrderedAndProjectExec( override def executeCollect(): Array[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) val data = child.execute().map(_.copy()).takeOrdered(limit)(ord) - if (projectList.isDefined) { - val proj = UnsafeProjection.create(projectList.get, child.output) + if (projectList != child.output) { + val proj = UnsafeProjection.create(projectList, child.output) data.map(r => proj(r).copy()) } else { data @@ -148,8 +148,8 @@ case class TakeOrderedAndProjectExec( localTopK, child.output, SinglePartition, serializer)) shuffled.mapPartitions { iter => val topK = org.apache.spark.util.collection.Utils.takeOrdered(iter.map(_.copy()), limit)(ord) - if (projectList.isDefined) { - val proj = UnsafeProjection.create(projectList.get, child.output) + if (projectList != child.output) { + val proj = UnsafeProjection.create(projectList, child.output) topK.map(r => proj(r)) } else { topK diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index 3217e34bd8ad3..7e317a4d80265 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -59,7 +59,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { checkThatPlansAgree( generateRandomInputData(), input => - noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, None, input)), + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), input => GlobalLimitExec(limit, LocalLimitExec(limit, @@ -74,7 +74,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { generateRandomInputData(), input => noOpFilter( - TakeOrderedAndProjectExec(limit, sortOrder, Some(Seq(input.output.last)), input)), + TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), input => GlobalLimitExec(limit, LocalLimitExec(limit, From b17f10ced34cbff8716610df370d19b130f93827 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 13 Sep 2016 12:54:03 +0200 Subject: [PATCH 1287/1470] [SPARK-17515] CollectLimit.execute() should perform per-partition limits CollectLimit.execute() incorrectly omits per-partition limits, leading to performance regressions in case this case is hit (which should not happen in normal operation, but can occur in some cases (see #15068 for one example). Regression test in SQLQuerySuite that asserts the number of records scanned from the input RDD. Author: Josh Rosen Closes #15070 from JoshRosen/SPARK-17515. (cherry picked from commit 3f6a2bb3f7beac4ce928eb660ee36258b5b9e8c8) Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/execution/limit.scala | 3 ++- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 01fbe5b7c2c03..86a8770715600 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -39,9 +39,10 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode override def executeCollect(): Array[InternalRow] = child.executeTake(limit) private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) protected override def doExecute(): RDD[InternalRow] = { + val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit)) val shuffled = new ShuffledRowRDD( ShuffleExchange.prepareShuffleDependency( - child.execute(), child.output, SinglePartition, serializer)) + locallyLimited, child.output, SinglePartition, serializer)) shuffled.mapPartitionsInternal(_.take(limit)) } } 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 df72afb03c90c..c5bab9923e6ec 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 @@ -2615,4 +2615,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { data.selectExpr("`part.col1`", "`col.1`")) } } + + test("SPARK-17515: CollectLimit.execute() should perform per-partition limits") { + val numRecordsRead = spark.sparkContext.longAccumulator + spark.range(1, 100, 1, numPartitions = 10).map { x => + numRecordsRead.add(1) + x + }.limit(1).queryExecution.toRdd.count() + assert(numRecordsRead.value === 10) + } } From c1426452bb69f7eb2209d70449a5306fb29d875f Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 13 Sep 2016 15:11:55 -0700 Subject: [PATCH 1288/1470] [SPARK-17531] Don't initialize Hive Listeners for the Execution Client ## What changes were proposed in this pull request? If a user provides listeners inside the Hive Conf, the configuration for these listeners are passed to the Hive Execution Client as well. This may cause issues for two reasons: 1. The Execution Client will actually generate garbage 2. The listener class needs to be both in the Spark Classpath and Hive Classpath This PR empties the listener configurations in `HiveUtils.newTemporaryConfiguration` so that the execution client will not contain the listener confs, but the metadata client will. ## How was this patch tested? Unit tests Author: Burak Yavuz Closes #15086 from brkyvz/null-listeners. (cherry picked from commit 72edc7e958271cedb01932880550cfc2c0631204) Signed-off-by: Yin Huai --- .../org/apache/spark/sql/hive/HiveUtils.scala | 7 ++++ .../spark/sql/hive/HiveUtilsSuite.scala | 36 +++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index bdec611453b2d..39d71e164bf51 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -394,6 +394,13 @@ private[spark] object HiveUtils extends Logging { // hive.metastore.uris is not set. propMap.put(ConfVars.METASTOREURIS.varname, "") + // The execution client will generate garbage events, therefore the listeners that are generated + // for the execution clients are useless. In order to not output garbage, we don't generate + // these listeners. + propMap.put(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname, "") + propMap.put(ConfVars.METASTORE_EVENT_LISTENERS.varname, "") + propMap.put(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname, "") + propMap.toMap } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala new file mode 100644 index 0000000000000..667a7ddd8bb61 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.QueryTest + +class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + test("newTemporaryConfiguration overwrites listener configurations") { + Seq(true, false).foreach { useInMemoryDerby => + val conf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby) + assert(conf(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname) === "") + assert(conf(ConfVars.METASTORE_EVENT_LISTENERS.varname) === "") + assert(conf(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname) === "") + } + } +} From 12ebfbeddf057efb666a7b6365c948c3fe479f2c Mon Sep 17 00:00:00 2001 From: Sami Jaktholm Date: Wed, 14 Sep 2016 09:38:30 +0100 Subject: [PATCH 1289/1470] [SPARK-17525][PYTHON] Remove SparkContext.clearFiles() from the PySpark API as it was removed from the Scala API prior to Spark 2.0.0 ## What changes were proposed in this pull request? This pull request removes the SparkContext.clearFiles() method from the PySpark API as the method was removed from the Scala API in 8ce645d4eeda203cf5e100c4bdba2d71edd44e6a. Using that method in PySpark leads to an exception as PySpark tries to call the non-existent method on the JVM side. ## How was this patch tested? Existing tests (though none of them tested this particular method). Author: Sami Jaktholm Closes #15081 from sjakthol/pyspark-sc-clearfiles. (cherry picked from commit b5bfcddbfbc2e79d3d0fbd43942716946e6c4ba3) Signed-off-by: Sean Owen --- python/pyspark/context.py | 8 -------- 1 file changed, 8 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6e9f24ef1026b..2744bb9ec04e5 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -787,14 +787,6 @@ def addFile(self, path): """ self._jsc.sc().addFile(path) - def clearFiles(self): - """ - Clear the job's list of files added by L{addFile} or L{addPyFile} so - that they do not get downloaded to any new nodes. - """ - # TODO: remove added .py or .zip files from the PYTHONPATH? - self._jsc.sc().clearFiles() - def addPyFile(self, path): """ Add a .py or .zip dependency for all tasks to be executed on this From c6ea748a7e0baec222cbb4bd130673233adc5e0c Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Wed, 14 Sep 2016 09:51:14 +0100 Subject: [PATCH 1290/1470] [SPARK-17480][SQL] Improve performance by removing or caching List.length which is O(n) ## What changes were proposed in this pull request? Scala's List.length method is O(N) and it makes the gatherCompressibilityStats function O(N^2). Eliminate the List.length calls by writing it in Scala way. https://github.com/scala/scala/blob/2.10.x/src/library/scala/collection/LinearSeqOptimized.scala#L36 As suggested. Extended the fix to HiveInspectors and AggregationIterator classes as well. ## How was this patch tested? Profiled a Spark job and found that CompressibleColumnBuilder is using 39% of the CPU. Out of this 39% CompressibleColumnBuilder->gatherCompressibilityStats is using 23% of it. 6.24% of the CPU is spend on List.length which is called inside gatherCompressibilityStats. After this change we started to save 6.24% of the CPU. Author: Ergin Seyfe Closes #15032 from seyfe/gatherCompressibilityStats. (cherry picked from commit 4cea9da2ae88b40a5503111f8f37051e2372163e) Signed-off-by: Sean Owen --- .../sql/execution/aggregate/AggregationIterator.scala | 7 ++++--- .../columnar/compression/CompressibleColumnBuilder.scala | 6 +----- .../scala/org/apache/spark/sql/hive/HiveInspectors.scala | 6 ++++-- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index 34de76dd4ab4e..6ca36e4acb448 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -73,9 +73,10 @@ abstract class AggregationIterator( startingInputBufferOffset: Int): Array[AggregateFunction] = { var mutableBufferOffset = 0 var inputBufferOffset: Int = startingInputBufferOffset - val functions = new Array[AggregateFunction](expressions.length) + val expressionsLength = expressions.length + val functions = new Array[AggregateFunction](expressionsLength) var i = 0 - while (i < expressions.length) { + while (i < expressionsLength) { val func = expressions(i).aggregateFunction val funcWithBoundReferences: AggregateFunction = expressions(i).mode match { case Partial | Complete if func.isInstanceOf[ImperativeAggregate] => @@ -171,7 +172,7 @@ abstract class AggregationIterator( case PartialMerge | Final => (buffer: MutableRow, row: InternalRow) => ae.merge(buffer, row) } - } + }.toArray // This projection is used to merge buffer values for all expression-based aggregates. val aggregationBufferSchema = functions.flatMap(_.aggBufferAttributes) val updateProjection = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala index 63eae1b8685ac..0f4680e502781 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala @@ -66,11 +66,7 @@ private[columnar] trait CompressibleColumnBuilder[T <: AtomicType] } private def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = { - var i = 0 - while (i < compressionEncoders.length) { - compressionEncoders(i).gatherCompressibilityStats(row, ordinal) - i += 1 - } + compressionEncoders.foreach(_.gatherCompressibilityStats(row, ordinal)) } abstract override def appendFrom(row: InternalRow, ordinal: Int): Unit = { 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 585befe37825c..f5c3536fd022e 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 @@ -610,7 +610,8 @@ private[hive] trait HiveInspectors { cache: Array[AnyRef], dataTypes: Array[DataType]): Array[AnyRef] = { var i = 0 - while (i < inspectors.length) { + val length = inspectors.length + while (i < length) { cache(i) = wrap(row.get(i, dataTypes(i)), inspectors(i), dataTypes(i)) i += 1 } @@ -623,7 +624,8 @@ private[hive] trait HiveInspectors { cache: Array[AnyRef], dataTypes: Array[DataType]): Array[AnyRef] = { var i = 0 - while (i < inspectors.length) { + val length = inspectors.length + while (i < length) { cache(i) = wrap(row(i), inspectors(i), dataTypes(i)) i += 1 } From 5493107d99977964cca1c15a2b0e084899e96dac Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 14 Sep 2016 10:10:16 +0100 Subject: [PATCH 1291/1470] [SPARK-17445][DOCS] Reference an ASF page as the main place to find third-party packages ## What changes were proposed in this pull request? Point references to spark-packages.org to https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects This will be accompanied by a parallel change to the spark-website repo, and additional changes to this wiki. ## How was this patch tested? Jenkins tests. Author: Sean Owen Closes #15075 from srowen/SPARK-17445. (cherry picked from commit dc0a4c916151c795dc41b5714e9d23b4937f4636) Signed-off-by: Sean Owen --- CONTRIBUTING.md | 2 +- R/pkg/R/sparkR.R | 4 ++-- docs/_layouts/global.html | 2 +- docs/index.md | 2 +- docs/sparkr.md | 3 ++- docs/streaming-programming-guide.md | 2 +- .../spark/sql/execution/datasources/DataSource.scala | 7 ++++--- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +++------ .../spark/sql/sources/ResolvedDataSourceSuite.scala | 6 +++--- 9 files changed, 18 insertions(+), 19 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index f10d7e277eea3..1a8206abe3838 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -6,7 +6,7 @@ It lists steps that are required before creating a PR. In particular, consider: - Is the change important and ready enough to ask the community to spend time reviewing? - Have you searched for existing, related JIRAs and pull requests? -- Is this a new feature that can stand alone as a package on http://spark-packages.org ? +- Is this a new feature that can stand alone as a [third party project](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects) ? - Is the change being proposed clearly explained and motivated? When you contribute code, you affirm that the contribution is your original work and that you diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 15afe01c24ed2..06015362e6bc1 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -100,7 +100,7 @@ sparkR.stop <- function() { #' @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 vector of jar files to pass to the worker nodes -#' @param sparkPackages Character vector of packages from spark-packages.org +#' @param sparkPackages Character vector of package coordinates #' @seealso \link{sparkR.session} #' @rdname sparkR.init-deprecated #' @export @@ -327,7 +327,7 @@ sparkRHive.init <- function(jsc = NULL) { #' @param sparkHome Spark Home directory. #' @param sparkConfig named list of Spark configuration to set on worker nodes. #' @param sparkJars character vector of jar files to pass to the worker nodes. -#' @param sparkPackages character vector of packages from spark-packages.org +#' @param sparkPackages character vector of package coordinates #' @param enableHiveSupport enable support for Hive, fallback if not built with Hive support; once #' set, this cannot be turned off on an existing session #' @param ... named Spark properties passed to the method. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index d3bf082aa751a..ad5b5c9adfac8 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -114,7 +114,7 @@
  • Building Spark
  • Contributing to Spark
  • -
  • Supplemental Projects
  • +
  • Third Party Projects
  • diff --git a/docs/index.md b/docs/index.md index 0cb8803783a0f..a7a92f6c4f6d7 100644 --- a/docs/index.md +++ b/docs/index.md @@ -120,7 +120,7 @@ options for deployment: * [OpenStack Swift](storage-openstack-swift.html) * [Building Spark](building-spark.html): build Spark using the Maven system * [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) -* [Supplemental Projects](https://cwiki.apache.org/confluence/display/SPARK/Supplemental+Spark+Projects): related third party Spark projects +* [Third Party Projects](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects): related third party Spark projects **External Resources:** diff --git a/docs/sparkr.md b/docs/sparkr.md index 4bbc362c52086..b881119731045 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -110,7 +110,8 @@ head(df) SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. -The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. SparkR supports reading JSON, CSV and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by +The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. +SparkR supports reading JSON, CSV and Parquet files natively, and through packages available from sources like [Third Party Projects](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects), you can find data source connectors for popular file formats like Avro. These packages can either be added by specifying `--packages` with `spark-submit` or `sparkR` commands, or if initializing SparkSession with `sparkPackages` parameter when in an interactive R shell or from RStudio.
    diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index b92ca92b06c4a..236ae5d649c4a 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -2383,7 +2383,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are - [Kafka Integration Guide](streaming-kafka-integration.html) - [Kinesis Integration Guide](streaming-kinesis-integration.html) - [Custom Receiver Guide](streaming-custom-receivers.html) -* Third-party DStream data sources can be found in [Spark Packages](https://spark-packages.org/) +* Third-party DStream data sources can be found in [Third Party Projects](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects) * API documentation - Scala docs * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 014680d4e9b37..ee37390c91ddb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -141,12 +141,13 @@ case class DataSource( } else if (provider.toLowerCase == "avro" || provider == "com.databricks.spark.avro") { throw new AnalysisException( - s"Failed to find data source: ${provider.toLowerCase}. Please use Spark " + - "package http://spark-packages.org/package/databricks/spark-avro") + s"Failed to find data source: ${provider.toLowerCase}. Please find an Avro " + + "package at " + + "https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects") } else { throw new ClassNotFoundException( s"Failed to find data source: $provider. Please find packages at " + - "http://spark-packages.org", + "https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects", error) } } 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 c5bab9923e6ec..f20b9adcb61a4 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 @@ -1635,21 +1635,18 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { e = intercept[AnalysisException] { sql(s"select id from `com.databricks.spark.avro`.`file_path`") } - assert(e.message.contains("Failed to find data source: com.databricks.spark.avro. " + - "Please use Spark package http://spark-packages.org/package/databricks/spark-avro")) + assert(e.message.contains("Failed to find data source: com.databricks.spark.avro.")) // data source type is case insensitive e = intercept[AnalysisException] { sql(s"select id from Avro.`file_path`") } - assert(e.message.contains("Failed to find data source: avro. Please use Spark package " + - "http://spark-packages.org/package/databricks/spark-avro")) + assert(e.message.contains("Failed to find data source: avro.")) e = intercept[AnalysisException] { sql(s"select id from avro.`file_path`") } - assert(e.message.contains("Failed to find data source: avro. Please use Spark package " + - "http://spark-packages.org/package/databricks/spark-avro")) + assert(e.message.contains("Failed to find data source: avro.")) e = intercept[AnalysisException] { sql(s"select id from `org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 5ea1f32433699..76ffb949f1293 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -74,16 +74,16 @@ class ResolvedDataSourceSuite extends SparkFunSuite { val error1 = intercept[AnalysisException] { getProvidingClass("avro") } - assert(error1.getMessage.contains("spark-packages")) + assert(error1.getMessage.contains("Failed to find data source: avro.")) val error2 = intercept[AnalysisException] { getProvidingClass("com.databricks.spark.avro") } - assert(error2.getMessage.contains("spark-packages")) + assert(error2.getMessage.contains("Failed to find data source: com.databricks.spark.avro.")) val error3 = intercept[ClassNotFoundException] { getProvidingClass("asfdwefasdfasdf") } - assert(error3.getMessage.contains("spark-packages")) + assert(error3.getMessage.contains("Failed to find data source: asfdwefasdfasdf.")) } } From 6fe5972e649e171e994c30bff3da0c408a3d7f3a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 14 Sep 2016 10:10:01 -0700 Subject: [PATCH 1292/1470] [SPARK-17514] df.take(1) and df.limit(1).collect() should perform the same in Python ## What changes were proposed in this pull request? In PySpark, `df.take(1)` runs a single-stage job which computes only one partition of the DataFrame, while `df.limit(1).collect()` computes all partitions and runs a two-stage job. This difference in performance is confusing. The reason why `limit(1).collect()` is so much slower is that `collect()` internally maps to `df.rdd..toLocalIterator`, which causes Spark SQL to build a query where a global limit appears in the middle of the plan; this, in turn, ends up being executed inefficiently because limits in the middle of plans are now implemented by repartitioning to a single task rather than by running a `take()` job on the driver (this was done in #7334, a patch which was a prerequisite to allowing partition-local limits to be pushed beneath unions, etc.). In order to fix this performance problem I think that we should generalize the fix from SPARK-10731 / #8876 so that `DataFrame.collect()` also delegates to the Scala implementation and shares the same performance properties. This patch modifies `DataFrame.collect()` to first collect all results to the driver and then pass them to Python, allowing this query to be planned using Spark's `CollectLimit` optimizations. ## How was this patch tested? Added a regression test in `sql/tests.py` which asserts that the expected number of jobs, stages, and tasks are run for both queries. Author: Josh Rosen Closes #15068 from JoshRosen/pyspark-collect-limit. (cherry picked from commit 6d06ff6f7e2dd72ba8fe96cd875e83eda6ebb2a9) Signed-off-by: Davies Liu --- python/pyspark/sql/dataframe.py | 5 +---- python/pyspark/sql/tests.py | 18 ++++++++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 8 ++++++-- .../sql/execution/python/EvaluatePython.scala | 13 +------------ 4 files changed, 26 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f19bd3a57a4e0..8577dc02cce6b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -345,10 +345,7 @@ def take(self, num): >>> df.take(2) [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] """ - with SCCallSiteSync(self._sc) as css: - port = self._sc._jvm.org.apache.spark.sql.execution.python.EvaluatePython.takeAndServe( - self._jdf, num) - return list(_load_from_socket(port, BatchedSerializer(PickleSerializer()))) + return self.limit(num).collect() @since(1.3) def foreach(self, f): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index c7f312e465b51..d3634fccc4285 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1857,6 +1857,24 @@ def test_collect_functions(self): sorted(df.select(functions.collect_list(df.value).alias('r')).collect()[0].r), ["1", "2", "2", "2"]) + def test_limit_and_take(self): + df = self.spark.range(1, 1000, numPartitions=10) + + def assert_runs_only_one_job_stage_and_task(job_group_name, f): + tracker = self.sc.statusTracker() + self.sc.setJobGroup(job_group_name, description="") + f() + jobs = tracker.getJobIdsForGroup(job_group_name) + self.assertEqual(1, len(jobs)) + stages = tracker.getJobInfo(jobs[0]).stageIds + self.assertEqual(1, len(stages)) + self.assertEqual(1, tracker.getStageInfo(stages[0]).numTasks) + + # Regression test for SPARK-10731: take should delegate to Scala implementation + assert_runs_only_one_job_stage_and_task("take", lambda: df.take(1)) + # Regression test for SPARK-17514: limit(n).collect() should the perform same as take(n) + assert_runs_only_one_job_stage_and_task("collect_limit", lambda: df.limit(1).collect()) + if __name__ == "__main__": from pyspark.sql.tests import * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 6ca0138adf0e7..0b236a0c7466f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -30,7 +30,7 @@ import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.function._ -import org.apache.spark.api.python.PythonRDD +import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst._ @@ -2519,8 +2519,12 @@ class Dataset[T] private[sql]( } private[sql] def collectToPython(): Int = { + EvaluatePython.registerPicklers() withNewExecutionId { - PythonRDD.collectAndServe(javaToPython.rdd) + val toJava: (Any) => Any = EvaluatePython.toJava(_, schema) + val iter = new SerDeUtil.AutoBatchedPickler( + queryExecution.executedPlan.executeCollect().iterator.map(toJava)) + PythonRDD.serveIterator(iter, "serve-DataFrame") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index cf68ed4ec36a8..724025b4647f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -24,9 +24,8 @@ import scala.collection.JavaConverters._ import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} -import org.apache.spark.api.python.{PythonRDD, SerDeUtil} +import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} @@ -34,16 +33,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String object EvaluatePython { - def takeAndServe(df: DataFrame, n: Int): Int = { - registerPicklers() - df.withNewExecutionId { - val iter = new SerDeUtil.AutoBatchedPickler( - df.queryExecution.executedPlan.executeTake(n).iterator.map { row => - EvaluatePython.toJava(row, df.schema) - }) - PythonRDD.serveIterator(iter, s"serve-DataFrame") - } - } def needConversionInPython(dt: DataType): Boolean = dt match { case DateType | TimestampType => true From fab77dadf70d011cec8976acfe8c851816f82426 Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 14 Sep 2016 14:19:35 -0500 Subject: [PATCH 1293/1470] [SPARK-17511] Yarn Dynamic Allocation: Avoid marking released container as Failed Due to race conditions, the ` assert(numExecutorsRunning <= targetNumExecutors)` can fail causing `AssertionError`. So removed the assertion, instead moved the conditional check before launching new container: ``` java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:156) at org.apache.spark.deploy.yarn.YarnAllocator$$anonfun$runAllocatedContainers$1.org$apache$spark$deploy$yarn$YarnAllocator$$anonfun$$updateInternalState$1(YarnAllocator.scala:489) at org.apache.spark.deploy.yarn.YarnAllocator$$anonfun$runAllocatedContainers$1$$anon$1.run(YarnAllocator.scala:519) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` This was manually tested using a large ForkAndJoin job with Dynamic Allocation enabled to validate the failing job succeeds, without any such exception. Author: Kishor Patil Closes #15069 from kishorvpatil/SPARK-17511. (cherry picked from commit ff6e4cbdc80e2ad84c5d70ee07f323fad9374e3e) Signed-off-by: Tom Graves --- .../spark/deploy/yarn/YarnAllocator.scala | 68 ++++++++++--------- .../deploy/yarn/YarnAllocatorSuite.scala | 19 ++++++ 2 files changed, 55 insertions(+), 32 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 1b800716495a7..b321901e765e3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -483,7 +483,6 @@ private[yarn] class YarnAllocator( def updateInternalState(): Unit = synchronized { numExecutorsRunning += 1 - assert(numExecutorsRunning <= targetNumExecutors) executorIdToContainer(executorId) = container containerIdToExecutorId(container.getId) = executorId @@ -493,39 +492,44 @@ private[yarn] class YarnAllocator( allocatedContainerToHostMap.put(containerId, executorHostname) } - if (launchContainers) { - logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( - driverUrl, executorHostname)) - - launcherPool.execute(new Runnable { - override def run(): Unit = { - try { - new ExecutorRunnable( - container, - conf, - sparkConf, - driverUrl, - executorId, - executorHostname, - executorMemory, - executorCores, - appAttemptId.getApplicationId.toString, - securityMgr, - localResources - ).run() - updateInternalState() - } catch { - case NonFatal(e) => - logError(s"Failed to launch executor $executorId on container $containerId", e) - // Assigned container should be released immediately to avoid unnecessary resource - // occupation. - amClient.releaseAssignedContainer(containerId) + if (numExecutorsRunning < targetNumExecutors) { + if (launchContainers) { + logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( + driverUrl, executorHostname)) + + launcherPool.execute(new Runnable { + override def run(): Unit = { + try { + new ExecutorRunnable( + container, + conf, + sparkConf, + driverUrl, + executorId, + executorHostname, + executorMemory, + executorCores, + appAttemptId.getApplicationId.toString, + securityMgr, + localResources + ).run() + updateInternalState() + } catch { + case NonFatal(e) => + logError(s"Failed to launch executor $executorId on container $containerId", e) + // Assigned container should be released immediately to avoid unnecessary resource + // occupation. + amClient.releaseAssignedContainer(containerId) + } } - } - }) + }) + } else { + // For test only + updateInternalState() + } } else { - // For test only - updateInternalState() + logInfo(("Skip launching executorRunnable as runnning Excecutors count: %d " + + "reached target Executors count: %d.").format(numExecutorsRunning, targetNumExecutors)) } } } 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 207dbf56d3606..f8351c03e5617 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 @@ -136,6 +136,25 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter size should be (0) } + test("container should not be created if requested number if met") { + // request a single container and receive it + val handler = createAllocator(1) + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getPendingAllocate.size should be (1) + + val container = createContainer("host1") + handler.handleAllocatedContainers(Array(container)) + + handler.getNumExecutorsRunning should be (1) + handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") + handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) + + val container2 = createContainer("host2") + handler.handleAllocatedContainers(Array(container2)) + handler.getNumExecutorsRunning should be (1) + } + test("some containers allocated") { // request a few containers and receive some of them val handler = createAllocator(4) From fffcec90b65047c3031c2b96679401f8fbef6337 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 14 Sep 2016 13:33:51 -0700 Subject: [PATCH 1294/1470] [SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's value can be read thread-safely ## What changes were proposed in this pull request? Make CollectionAccumulator and SetAccumulator's value can be read thread-safely to fix the ConcurrentModificationException reported in [JIRA](https://issues.apache.org/jira/browse/SPARK-17463). ## How was this patch tested? Existing tests. Author: Shixiong Zhu Closes #15063 from zsxwing/SPARK-17463. (cherry picked from commit e33bfaed3b160fbc617c878067af17477a0044f5) Signed-off-by: Josh Rosen --- .../apache/spark/executor/TaskMetrics.scala | 41 ++++++++++++------- .../org/apache/spark/util/AccumulatorV2.scala | 7 +++- .../org/apache/spark/util/JsonProtocol.scala | 11 ++--- .../apache/spark/util/JsonProtocolSuite.scala | 3 +- .../spark/sql/execution/debug/package.scala | 24 +++++++---- 5 files changed, 54 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index dd149a919fe55..52a349919e336 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,9 @@ package org.apache.spark.executor +import java.util.{ArrayList, Collections} + +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} import org.apache.spark._ @@ -99,7 +102,11 @@ class TaskMetrics private[spark] () extends Serializable { /** * Storage statuses of any blocks that have been updated as a result of this task. */ - def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.value + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = { + // This is called on driver. All accumulator updates have a fixed value. So it's safe to use + // `asScala` which accesses the internal values using `java.util.Iterator`. + _updatedBlockStatuses.value.asScala + } // Setters and increment-ers private[spark] def setExecutorDeserializeTime(v: Long): Unit = @@ -114,8 +121,10 @@ class TaskMetrics private[spark] () extends Serializable { private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) private[spark] def incUpdatedBlockStatuses(v: (BlockId, BlockStatus)): Unit = _updatedBlockStatuses.add(v) - private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = + private[spark] def setUpdatedBlockStatuses(v: java.util.List[(BlockId, BlockStatus)]): Unit = _updatedBlockStatuses.setValue(v) + private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = + _updatedBlockStatuses.setValue(v.asJava) /** * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted @@ -268,7 +277,7 @@ private[spark] object TaskMetrics extends Logging { val name = info.name.get val value = info.update.get if (name == UPDATED_BLOCK_STATUSES) { - tm.setUpdatedBlockStatuses(value.asInstanceOf[Seq[(BlockId, BlockStatus)]]) + tm.setUpdatedBlockStatuses(value.asInstanceOf[java.util.List[(BlockId, BlockStatus)]]) } else { tm.nameToAccums.get(name).foreach( _.asInstanceOf[LongAccumulator].setValue(value.asInstanceOf[Long]) @@ -299,8 +308,8 @@ private[spark] object TaskMetrics extends Logging { private[spark] class BlockStatusesAccumulator - extends AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { - private var _seq = ArrayBuffer.empty[(BlockId, BlockStatus)] + extends AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]] { + private val _seq = Collections.synchronizedList(new ArrayList[(BlockId, BlockStatus)]()) override def isZero(): Boolean = _seq.isEmpty @@ -308,25 +317,27 @@ private[spark] class BlockStatusesAccumulator override def copy(): BlockStatusesAccumulator = { val newAcc = new BlockStatusesAccumulator - newAcc._seq = _seq.clone() + newAcc._seq.addAll(_seq) newAcc } override def reset(): Unit = _seq.clear() - override def add(v: (BlockId, BlockStatus)): Unit = _seq += v + override def add(v: (BlockId, BlockStatus)): Unit = _seq.add(v) - override def merge(other: AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) - : Unit = other match { - case o: BlockStatusesAccumulator => _seq ++= o.value - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + override def merge( + other: AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]]): Unit = { + other match { + case o: BlockStatusesAccumulator => _seq.addAll(o.value) + case _ => throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } } - override def value: Seq[(BlockId, BlockStatus)] = _seq + override def value: java.util.List[(BlockId, BlockStatus)] = _seq - def setValue(newValue: Seq[(BlockId, BlockStatus)]): Unit = { + def setValue(newValue: java.util.List[(BlockId, BlockStatus)]): Unit = { _seq.clear() - _seq ++= newValue + _seq.addAll(newValue) } } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index d130a37db5b5d..470d912ecff13 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import java.{lang => jl} import java.io.ObjectInputStream -import java.util.ArrayList +import java.util.{ArrayList, Collections} import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong @@ -38,6 +38,9 @@ private[spark] case class AccumulatorMetadata( /** * The base class for accumulators, that can accumulate inputs of type `IN`, and produce output of * type `OUT`. + * + * `OUT` should be a type that can be read atomically (e.g., Int, Long), or thread-safely + * (e.g., synchronized collections) because it will be read from other threads. */ abstract class AccumulatorV2[IN, OUT] extends Serializable { private[spark] var metadata: AccumulatorMetadata = _ @@ -433,7 +436,7 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { * @since 2.0.0 */ class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { - private val _list: java.util.List[T] = new ArrayList[T] + private val _list: java.util.List[T] = Collections.synchronizedList(new ArrayList[T]()) override def isZero: Boolean = _list.isEmpty diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 18547d459eb5c..148635f5d2414 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -309,11 +309,12 @@ private[spark] object JsonProtocol { case v: Int => JInt(v) case v: Long => JInt(v) // We only have 3 kind of internal accumulator types, so if it's not int or long, it must be - // the blocks accumulator, whose type is `Seq[(BlockId, BlockStatus)]` + // the blocks accumulator, whose type is `java.util.List[(BlockId, BlockStatus)]` case v => - JArray(v.asInstanceOf[Seq[(BlockId, BlockStatus)]].toList.map { case (id, status) => - ("Block ID" -> id.toString) ~ - ("Status" -> blockStatusToJson(status)) + JArray(v.asInstanceOf[java.util.List[(BlockId, BlockStatus)]].asScala.toList.map { + case (id, status) => + ("Block ID" -> id.toString) ~ + ("Status" -> blockStatusToJson(status)) }) } } else { @@ -740,7 +741,7 @@ private[spark] object JsonProtocol { val id = BlockId((blockJson \ "Block ID").extract[String]) val status = blockStatusFromJson(blockJson \ "Status") (id, status) - } + }.asJava case _ => throw new IllegalArgumentException(s"unexpected json value $value for " + "accumulator " + name.get) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 7a1ee03e4ce55..cda3457523937 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.util.Properties +import scala.collection.JavaConverters._ import scala.collection.Map import org.json4s.jackson.JsonMethods._ @@ -415,7 +416,7 @@ class JsonProtocolSuite extends SparkFunSuite { }) testAccumValue(Some(RESULT_SIZE), 3L, JInt(3)) testAccumValue(Some(shuffleRead.REMOTE_BLOCKS_FETCHED), 2, JInt(2)) - testAccumValue(Some(UPDATED_BLOCK_STATUSES), blocks, blocksJson) + testAccumValue(Some(UPDATED_BLOCK_STATUSES), blocks.asJava, blocksJson) // For anything else, we just cast the value to a string testAccumValue(Some("anything"), blocks, JString(blocks.toString)) testAccumValue(Some("anything"), 123, JString("123")) 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 082f97a8808fa..d321f4cd76877 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 @@ -17,7 +17,9 @@ package org.apache.spark.sql.execution -import scala.collection.mutable.HashSet +import java.util.Collections + +import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -107,18 +109,20 @@ package object debug { case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { def output: Seq[Attribute] = child.output - class SetAccumulator[T] extends AccumulatorV2[T, HashSet[T]] { - private val _set = new HashSet[T]() + class SetAccumulator[T] extends AccumulatorV2[T, java.util.Set[T]] { + private val _set = Collections.synchronizedSet(new java.util.HashSet[T]()) override def isZero: Boolean = _set.isEmpty - override def copy(): AccumulatorV2[T, HashSet[T]] = { + override def copy(): AccumulatorV2[T, java.util.Set[T]] = { val newAcc = new SetAccumulator[T]() - newAcc._set ++= _set + newAcc._set.addAll(_set) newAcc } override def reset(): Unit = _set.clear() - override def add(v: T): Unit = _set += v - override def merge(other: AccumulatorV2[T, HashSet[T]]): Unit = _set ++= other.value - override def value: HashSet[T] = _set + override def add(v: T): Unit = _set.add(v) + override def merge(other: AccumulatorV2[T, java.util.Set[T]]): Unit = { + _set.addAll(other.value) + } + override def value: java.util.Set[T] = _set } /** @@ -138,7 +142,9 @@ package object debug { debugPrint(s"== ${child.simpleString} ==") debugPrint(s"Tuples output: ${tupleCount.value}") child.output.zip(columnStats).foreach { case (attr, metric) => - val actualDataTypes = metric.elementTypes.value.mkString("{", ",", "}") + // This is called on driver. All accumulator updates have a fixed value. So it's safe to use + // `asScala` which accesses the internal values using `java.util.Iterator`. + val actualDataTypes = metric.elementTypes.value.asScala.mkString("{", ",", "}") debugPrint(s" ${attr.name} ${attr.dataType}: $actualDataTypes") } } From bb2bdb44032d2e71832b3e0e771590fb2225e4f3 Mon Sep 17 00:00:00 2001 From: Xing SHI Date: Wed, 14 Sep 2016 13:46:46 -0700 Subject: [PATCH 1295/1470] [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak The expression like `if (memoryMap(taskAttemptId) == 0) memoryMap.remove(taskAttemptId)` in method `releaseUnrollMemoryForThisTask` and `releasePendingUnrollMemoryForThisTask` should be called after release memory operation, whatever `memoryToRelease` is > 0 or not. If the memory of a task has been set to 0 when calling a `releaseUnrollMemoryForThisTask` or a `releasePendingUnrollMemoryForThisTask` method, the key in the memory map corresponding to that task will never be removed from the hash map. See the details in [SPARK-17465](https://issues.apache.org/jira/browse/SPARK-17465). Author: Xing SHI Closes #15022 from saturday-shi/SPARK-17465. --- .../scala/org/apache/spark/storage/memory/MemoryStore.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 5fd4e886ec483..1230128d230bd 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -591,11 +591,11 @@ private[spark] class MemoryStore( val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) if (memoryToRelease > 0) { unrollMemoryMap(taskAttemptId) -= memoryToRelease - if (unrollMemoryMap(taskAttemptId) == 0) { - unrollMemoryMap.remove(taskAttemptId) - } memoryManager.releaseUnrollMemory(memoryToRelease, memoryMode) } + if (unrollMemoryMap(taskAttemptId) == 0) { + unrollMemoryMap.remove(taskAttemptId) + } } } } From 5c2bc8360019fb08e2e62e50bb261f7ce19b231e Mon Sep 17 00:00:00 2001 From: codlife <1004910847@qq.com> Date: Thu, 15 Sep 2016 09:38:13 +0100 Subject: [PATCH 1296/1470] [SPARK-17521] Error when I use sparkContext.makeRDD(Seq()) ## What changes were proposed in this pull request? when i use sc.makeRDD below ``` val data3 = sc.makeRDD(Seq()) println(data3.partitions.length) ``` I got an error: Exception in thread "main" java.lang.IllegalArgumentException: Positive number of slices required We can fix this bug just modify the last line ,do a check of seq.size ``` def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope { assertNotStopped() val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap new ParallelCollectionRDD[T](this, seq.map(_._1), math.max(seq.size, defaultParallelism), indexToPrefs) } ``` ## How was this patch tested? manual tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: codlife <1004910847@qq.com> Author: codlife Closes #15077 from codlife/master. (cherry picked from commit 647ee05e5815bde361662a9286ac602c44b4d4e6) Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71511b800633f..214758f03fdf2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -788,7 +788,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope { assertNotStopped() val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap - new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs) + new ParallelCollectionRDD[T](this, seq.map(_._1), math.max(seq.size, 1), indexToPrefs) } /** From a09c258c9a97e701fa7650cc0651e3c6a7a1cab9 Mon Sep 17 00:00:00 2001 From: junyangq Date: Thu, 15 Sep 2016 10:00:36 -0700 Subject: [PATCH 1297/1470] [SPARK-17317][SPARKR] Add SparkR vignette to branch 2.0 ## What changes were proposed in this pull request? This PR adds SparkR vignette to branch 2.0, which works as a friendly guidance going through the functionality provided by SparkR. ## How was this patch tested? R unit test. Author: junyangq Author: Shivaram Venkataraman Author: Junyang Qian Closes #15100 from junyangq/SPARKR-vignette-2.0. --- R/create-docs.sh | 11 +- R/pkg/vignettes/sparkr-vignettes.Rmd | 643 +++++++++++++++++++++++++++ 2 files changed, 652 insertions(+), 2 deletions(-) create mode 100644 R/pkg/vignettes/sparkr-vignettes.Rmd diff --git a/R/create-docs.sh b/R/create-docs.sh index d2ae160b50021..0dfba22463396 100755 --- a/R/create-docs.sh +++ b/R/create-docs.sh @@ -17,11 +17,13 @@ # limitations under the License. # -# Script to create API docs for SparkR -# This requires `devtools` and `knitr` to be installed on the machine. +# Script to create API docs and vignettes for SparkR +# This requires `devtools`, `knitr` and `rmarkdown` to be installed on the machine. # After running this script the html docs can be found in # $SPARK_HOME/R/pkg/html +# The vignettes can be found in +# $SPARK_HOME/R/pkg/vignettes/sparkr_vignettes.html set -o pipefail set -e @@ -43,4 +45,9 @@ Rscript -e 'libDir <- "../../lib"; library(SparkR, lib.loc=libDir); library(knit popd +# render creates SparkR vignettes +Rscript -e 'library(rmarkdown); paths <- .libPaths(); .libPaths(c("lib", paths)); Sys.setenv(SPARK_HOME=tools::file_path_as_absolute("..")); render("pkg/vignettes/sparkr-vignettes.Rmd"); .libPaths(paths)' + +find pkg/vignettes/. -not -name '.' -not -name '*.Rmd' -not -name '*.md' -not -name '*.pdf' -not -name '*.html' -delete + popd diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd new file mode 100644 index 0000000000000..5156c9e566c95 --- /dev/null +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -0,0 +1,643 @@ +--- +title: "SparkR - Practical Guide" +output: + html_document: + theme: united + toc: true + toc_depth: 4 + toc_float: true + highlight: textmate +--- + +## Overview + +SparkR is an R package that provides a light-weight frontend to use Apache Spark from R. With Spark `r packageVersion("SparkR")`, SparkR provides a distributed data frame implementation that supports data processing operations like selection, filtering, aggregation etc. and distributed machine learning using [MLlib](http://spark.apache.org/mllib/). + +## Getting Started + +We begin with an example running on the local machine and provide an overview of the use of SparkR: data ingestion, data processing and machine learning. + +First, let's load and attach the package. +```{r, message=FALSE} +library(SparkR) +``` + +`SparkSession` is the entry point into SparkR which connects your R program to a Spark cluster. You can create a `SparkSession` using `sparkR.session` and pass in options such as the application name, any Spark packages depended on, etc. + +We use default settings in which it runs in local mode. It auto downloads Spark package in the background if no previous installation is found. For more details about setup, see [Spark Session](#SetupSparkSession). + +```{r, message=FALSE} +sparkR.session() +``` + +The operations in SparkR are centered around an R class called `SparkDataFrame`. It is a distributed collection of data organized into named columns, which is conceptually equivalent to a table in a relational database or a data frame in R, but with richer optimizations under the hood. + +`SparkDataFrame` can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing local R data frames. For example, we create a `SparkDataFrame` from a local R data frame, + +```{r} +cars <- cbind(model = rownames(mtcars), mtcars) +carsDF <- createDataFrame(cars) +``` + +We can view the first few rows of the `SparkDataFrame` by `head` or `showDF` function. +```{r} +head(carsDF) +``` + +Common data processing operations such as `filter`, `select` are supported on the `SparkDataFrame`. +```{r} +carsSubDF <- select(carsDF, "model", "mpg", "hp") +carsSubDF <- filter(carsSubDF, carsSubDF$hp >= 200) +head(carsSubDF) +``` + +SparkR can use many common aggregation functions after grouping. + +```{r} +carsGPDF <- summarize(groupBy(carsDF, carsDF$gear), count = n(carsDF$gear)) +head(carsGPDF) +``` + +The results `carsDF` and `carsSubDF` are `SparkDataFrame` objects. To convert back to R `data.frame`, we can use `collect`. **Caution**: This can cause your interactive environment to run out of memory, though, because `collect()` fetches the entire distributed `DataFrame` to your client, which is acting as a Spark driver. +```{r} +carsGP <- collect(carsGPDF) +class(carsGP) +``` + +SparkR supports a number of commonly used machine learning algorithms. Under the hood, SparkR uses MLlib to train the model. Users can call `summary` to print a summary of the fitted model, `predict` to make predictions on new data, and `write.ml`/`read.ml` to save/load fitted models. + +SparkR supports a subset of R formula operators for model fitting, including ‘~’, ‘.’, ‘:’, ‘+’, and ‘-‘. We use linear regression as an example. +```{r} +model <- spark.glm(carsDF, mpg ~ wt + cyl) +``` + +The result matches that returned by R `glm` function applied to the corresponding `data.frame` `mtcars` of `carsDF`. In fact, for Generalized Linear Model, we specifically expose `glm` for `SparkDataFrame` as well so that the above is equivalent to `model <- glm(mpg ~ wt + cyl, data = carsDF)`. + +```{r} +summary(model) +``` + +The model can be saved by `write.ml` and loaded back using `read.ml`. +```{r, eval=FALSE} +write.ml(model, path = "/HOME/tmp/mlModel/glmModel") +``` + +In the end, we can stop Spark Session by running +```{r, eval=FALSE} +sparkR.session.stop() +``` + +## Setup + +### Installation + +Different from many other R packages, to use SparkR, you need an additional installation of Apache Spark. The Spark installation will be used to run a backend process that will compile and execute SparkR programs. + +If you don't have Spark installed on the computer, you may download it from [Apache Spark Website](http://spark.apache.org/downloads.html). Alternatively, we provide an easy-to-use function `install.spark` to complete this process. You don't have to call it explicitly. We will check the installation when `sparkR.session` is called and `install.spark` function will be triggered automatically if no installation is found. + +```{r, eval=FALSE} +install.spark() +``` + +If you already have Spark installed, you don't have to install again and can pass the `sparkHome` argument to `sparkR.session` to let SparkR know where the Spark installation is. + +```{r, eval=FALSE} +sparkR.session(sparkHome = "/HOME/spark") +``` + +### Spark Session {#SetupSparkSession} + + +In addition to `sparkHome`, many other options can be specified in `sparkR.session`. For a complete list, see [Starting up: SparkSession](http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparksession) and [SparkR API doc](http://spark.apache.org/docs/latest/api/R/sparkR.session.html). + +In particular, the following Spark driver properties can be set in `sparkConfig`. + +Property Name | Property group | spark-submit equivalent +---------------- | ------------------ | ---------------------- +spark.driver.memory | Application Properties | --driver-memory +spark.driver.extraClassPath | Runtime Environment | --driver-class-path +spark.driver.extraJavaOptions | Runtime Environment | --driver-java-options +spark.driver.extraLibraryPath | Runtime Environment | --driver-library-path + +**For Windows users**: Due to different file prefixes across operating systems, to avoid the issue of potential wrong prefix, a current workaround is to specify `spark.sql.warehouse.dir` when starting the `SparkSession`. + +```{r, eval=FALSE} +spark_warehouse_path <- file.path(path.expand('~'), "spark-warehouse") +sparkR.session(spark.sql.warehouse.dir = spark_warehouse_path) +``` + + +#### Cluster Mode +SparkR can connect to remote Spark clusters. [Cluster Mode Overview](http://spark.apache.org/docs/latest/cluster-overview.html) is a good introduction to different Spark cluster modes. + +When connecting SparkR to a remote Spark cluster, make sure that the Spark version and Hadoop version on the machine match the corresponding versions on the cluster. Current SparkR package is compatible with +```{r, echo=FALSE, tidy = TRUE} +paste("Spark", packageVersion("SparkR")) +``` +It should be used both on the local computer and on the remote cluster. + +To connect, pass the URL of the master node to `sparkR.session`. A complete list can be seen in [Spark Master URLs](http://spark.apache.org/docs/latest/submitting-applications.html#master-urls). +For example, to connect to a local standalone Spark master, we can call + +```{r, eval=FALSE} +sparkR.session(master = "spark://local:7077") +``` + +For YARN cluster, SparkR supports the client mode with the master set as "yarn". +```{r, eval=FALSE} +sparkR.session(master = "yarn") +``` +Yarn cluster mode is not supported in the current version. + +## Data Import + +### Local Data Frame +The simplest way is to convert a local R data frame into a `SparkDataFrame`. Specifically we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a `SparkDataFrame`. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R. +```{r} +df <- as.DataFrame(faithful) +head(df) +``` + +### Data Sources +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. You can check the Spark SQL programming guide for more [specific options](https://spark.apache.org/docs/latest/sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. + +The general method for creating `SparkDataFrame` from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active Spark Session will be used automatically. SparkR supports reading CSV, JSON and Parquet files natively and through Spark Packages you can find data source connectors for popular file formats like Avro. These packages can be added with `sparkPackages` parameter when initializing SparkSession using `sparkR.session'.` + +```{r, eval=FALSE} +sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") +``` + +We can see how to use data sources using an example CSV input file. For more information please refer to SparkR [read.df](https://spark.apache.org/docs/latest/api/R/read.df.html) API documentation. +```{r, eval=FALSE} +df <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.strings = "NA") +``` + +The data sources API natively supports JSON formatted input files. Note that the file that is used here is not a typical JSON file. Each line in the file must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. + +Let's take a look at the first two lines of the raw JSON file used here. + +```{r} +filePath <- paste0(sparkR.conf("spark.home"), + "/examples/src/main/resources/people.json") +readLines(filePath, n = 2L) +``` + +We use `read.df` to read that into a `SparkDataFrame`. + +```{r} +people <- read.df(filePath, "json") +count(people) +head(people) +``` + +SparkR automatically infers the schema from the JSON file. +```{r} +printSchema(people) +``` + +If we want to read multiple JSON files, `read.json` can be used. +```{r} +people <- read.json(paste0(Sys.getenv("SPARK_HOME"), + c("/examples/src/main/resources/people.json", + "/examples/src/main/resources/people.json"))) +count(people) +``` + +The data sources API can also be used to save out `SparkDataFrames` into multiple file formats. For example we can save the `SparkDataFrame` from the previous example to a Parquet file using `write.df`. +```{r, eval=FALSE} +write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite") +``` + +### Hive Tables +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with Hive support and more details can be found in the [SQL programming guide](https://spark.apache.org/docs/latest/sql-programming-guide.html). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`). + +```{r, eval=FALSE} +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + +txtPath <- paste0(sparkR.conf("spark.home"), "/examples/src/main/resources/kv1.txt") +sqlCMD <- sprintf("LOAD DATA LOCAL INPATH '%s' INTO TABLE src", txtPath) +sql(sqlCMD) + +results <- sql("FROM src SELECT key, value") + +# results is now a SparkDataFrame +head(results) +``` + + +## Data Processing + +**To dplyr users**: SparkR has similar interface as dplyr in data processing. However, some noticeable differences are worth mentioning in the first place. We use `df` to represent a `SparkDataFrame` and `col` to represent the name of column here. + +1. indicate columns. SparkR uses either a character string of the column name or a Column object constructed with `$` to indicate a column. For example, to select `col` in `df`, we can write `select(df, "col")` or `select(df, df$col)`. + +2. describe conditions. In SparkR, the Column object representation can be inserted into the condition directly, or we can use a character string to describe the condition, without referring to the `SparkDataFrame` used. For example, to select rows with value > 1, we can write `filter(df, df$col > 1)` or `filter(df, "col > 1")`. + +Here are more concrete examples. + +dplyr | SparkR +-------- | --------- +`select(mtcars, mpg, hp)` | `select(carsDF, "mpg", "hp")` +`filter(mtcars, mpg > 20, hp > 100)` | `filter(carsDF, carsDF$mpg > 20, carsDF$hp > 100)` + +Other differences will be mentioned in the specific methods. + +We use the `SparkDataFrame` `carsDF` created above. We can get basic information about the `SparkDataFrame`. +```{r} +carsDF +``` + +Print out the schema in tree format. +```{r} +printSchema(carsDF) +``` + +### SparkDataFrame Operations + +#### Selecting rows, columns + +SparkDataFrames support a number of functions to do structured data processing. Here we include some basic examples and a complete list can be found in the [API](https://spark.apache.org/docs/latest/api/R/index.html) docs: + +You can also pass in column name as strings. +```{r} +head(select(carsDF, "mpg")) +``` + +Filter the SparkDataFrame to only retain rows with mpg less than 20 miles/gallon. +```{r} +head(filter(carsDF, carsDF$mpg < 20)) +``` + +#### Grouping, Aggregation + +A common flow of grouping and aggregation is + +1. Use `groupBy` or `group_by` with respect to some grouping variables to create a `GroupedData` object + +2. Feed the `GroupedData` object to `agg` or `summarize` functions, with some provided aggregation functions to compute a number within each group. + +A number of widely used functions are supported to aggregate data after grouping, including `avg`, `countDistinct`, `count`, `first`, `kurtosis`, `last`, `max`, `mean`, `min`, `sd`, `skewness`, `stddev_pop`, `stddev_samp`, `sumDistinct`, `sum`, `var_pop`, `var_samp`, `var`. See the [API doc for `mean`](http://spark.apache.org/docs/latest/api/R/mean.html) and other `agg_funcs` linked there. + +For example we can compute a histogram of the number of cylinders in the `mtcars` dataset as shown below. + +```{r} +numCyl <- summarize(groupBy(carsDF, carsDF$cyl), count = n(carsDF$cyl)) +head(numCyl) +``` + +#### Operating on Columns + +SparkR also provides a number of functions that can directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions. + +```{r} +carsDF_km <- carsDF +carsDF_km$kmpg <- carsDF_km$mpg * 1.61 +head(select(carsDF_km, "model", "mpg", "kmpg")) +``` + + +### Window Functions +A window function is a variation of aggregation function. In simple words, + +* aggregation function: `n` to `1` mapping - returns a single value for a group of entries. Examples include `sum`, `count`, `max`. + +* window function: `n` to `n` mapping - returns one value for each entry in the group, but the value may depend on all the entries of the *group*. Examples include `rank`, `lead`, `lag`. + +Formally, the *group* mentioned above is called the *frame*. Every input row can have a unique frame associated with it and the output of the window function on that row is based on the rows confined in that frame. + +Window functions are often used in conjunction with the following functions: `windowPartitionBy`, `windowOrderBy`, `partitionBy`, `orderBy`, `over`. To illustrate this we next look at an example. + +We still use the `mtcars` dataset. The corresponding `SparkDataFrame` is `carsDF`. Suppose for each number of cylinders, we want to calculate the rank of each car in `mpg` within the group. +```{r} +carsSubDF <- select(carsDF, "model", "mpg", "cyl") +ws <- orderBy(windowPartitionBy("cyl"), "mpg") +carsRank <- withColumn(carsSubDF, "rank", over(rank(), ws)) +head(carsRank, n = 20L) +``` + +We explain in detail the above steps. + +* `windowPartitionBy` creates a window specification object `WindowSpec` that defines the partition. It controls which rows will be in the same partition as the given row. In this case, rows with the same value in `cyl` will be put in the same partition. `orderBy` further defines the ordering - the position a given row is in the partition. The resulting `WindowSpec` is returned as `ws`. + +More window specification methods include `rangeBetween`, which can define boundaries of the frame by value, and `rowsBetween`, which can define the boundaries by row indices. + +* `withColumn` appends a Column called `rank` to the `SparkDataFrame`. `over` returns a windowing column. The first argument is usually a Column returned by window function(s) such as `rank()`, `lead(carsDF$wt)`. That calculates the corresponding values according to the partitioned-and-ordered table. + +### User-Defined Function + +In SparkR, we support several kinds of user-defined functions (UDFs). + +#### Apply by Partition + +`dapply` can apply a function to each partition of a `SparkDataFrame`. The function to be applied to each partition of the `SparkDataFrame` should have only one parameter, a `data.frame` corresponding to a partition, and the output should be a `data.frame` as well. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match to data types of returned value. See [here](#DataTypes) for mapping between R and Spark. + +We convert `mpg` to `kmpg` (kilometers per gallon). `carsSubDF` is a `SparkDataFrame` with a subset of `carsDF` columns. + +```{r} +carsSubDF <- select(carsDF, "model", "mpg") +schema <- structType(structField("model", "string"), structField("mpg", "double"), + structField("kmpg", "double")) +out <- dapply(carsSubDF, function(x) { x <- cbind(x, x$mpg * 1.61) }, schema) +head(collect(out)) +``` + +Like `dapply`, apply a function to each partition of a `SparkDataFrame` and collect the result back. The output of function should be a `data.frame`, but no schema is required in this case. Note that `dapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +```{r} +out <- dapplyCollect( + carsSubDF, + function(x) { + x <- cbind(x, "kmpg" = x$mpg * 1.61) + }) +head(out, 3) +``` + +#### Apply by Group +`gapply` can apply a function to each group of a `SparkDataFrame`. The function is to be applied to each group of the `SparkDataFrame` and should have only two parameters: grouping key and R `data.frame` corresponding to that key. The groups are chosen from `SparkDataFrames` column(s). The output of function should be a `data.frame`. Schema specifies the row format of the resulting `SparkDataFrame`. It must represent R function’s output schema on the basis of Spark data types. The column names of the returned `data.frame` are set by user. See [here](#DataTypes) for mapping between R and Spark. + +```{r} +schema <- structType(structField("cyl", "double"), structField("max_mpg", "double")) +result <- gapply( + carsDF, + "cyl", + function(key, x) { + y <- data.frame(key, max(x$mpg)) + }, + schema) +head(arrange(result, "max_mpg", decreasing = TRUE)) +``` + +Like gapply, `gapplyCollect` applies a function to each partition of a `SparkDataFrame` and collect the result back to R `data.frame`. The output of the function should be a `data.frame` but no schema is required in this case. Note that `gapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +```{r} +result <- gapplyCollect( + carsDF, + "cyl", + function(key, x) { + y <- data.frame(key, max(x$mpg)) + colnames(y) <- c("cyl", "max_mpg") + y + }) +head(result[order(result$max_mpg, decreasing = TRUE), ]) +``` + +#### Distribute Local Functions + +Similar to `lapply` in native R, `spark.lapply` runs a function over a list of elements and distributes the computations with Spark. `spark.lapply` works in a manner that is similar to `doParallel` or `lapply` to elements of a list. The results of all the computations should fit in a single machine. If that is not the case you can do something like `df <- createDataFrame(list)` and then use `dapply`. + +We use `svm` in package `e1071` as an example. We use all default settings except for varying costs of constraints violation. `spark.lapply` can train those different models in parallel. + +```{r} +costs <- exp(seq(from = log(1), to = log(1000), length.out = 5)) +train <- function(cost) { + stopifnot(requireNamespace("e1071", quietly = TRUE)) + model <- e1071::svm(Species ~ ., data = iris, cost = cost) + summary(model) +} +``` + +Return a list of model's summaries. +```{r} +model.summaries <- spark.lapply(costs, train) +``` + +```{r} +class(model.summaries) +``` + + +To avoid lengthy display, we only present the result of the second fitted model. You are free to inspect other models as well. +```{r} +print(model.summaries[[2]]) +``` + + +### SQL Queries +A `SparkDataFrame` can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. The sql function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. + +```{r} +people <- read.df(paste0(sparkR.conf("spark.home"), + "/examples/src/main/resources/people.json"), "json") +``` + +Register this SparkDataFrame as a temporary view. + +```{r} +createOrReplaceTempView(people, "people") +``` + +SQL statements can be run by using the sql method. +```{r} +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +head(teenagers) +``` + + +## Machine Learning + +SparkR supports the following machine learning models and algorithms. + +* Generalized Linear Model (GLM) + +* Naive Bayes Model + +* $k$-means Clustering + +* Accelerated Failure Time (AFT) Survival Model + +More will be added in the future. + +### R Formula + +For most above, SparkR supports **R formula operators**, including `~`, `.`, `:`, `+` and `-` for model fitting. This makes it a similar experience as using R functions. + +### Training and Test Sets + +We can easily split `SparkDataFrame` into random training and test sets by the `randomSplit` function. It returns a list of split `SparkDataFrames` with provided `weights`. We use `carsDF` as an example and want to have about $70%$ training data and $30%$ test data. +```{r} +splitDF_list <- randomSplit(carsDF, c(0.7, 0.3), seed = 0) +carsDF_train <- splitDF_list[[1]] +carsDF_test <- splitDF_list[[2]] +``` + +```{r} +count(carsDF_train) +head(carsDF_train) +``` + +```{r} +count(carsDF_test) +head(carsDF_test) +``` + + +### Models and Algorithms + +#### Generalized Linear Model + +The main function is `spark.glm`. The following families and link functions are supported. The default is gaussian. + +Family | Link Function +------ | --------- +gaussian | identity, log, inverse +binomial | logit, probit, cloglog (complementary log-log) +poisson | log, identity, sqrt +gamma | inverse, identity, log + +There are three ways to specify the `family` argument. + +* Family name as a character string, e.g. `family = "gaussian"`. + +* Family function, e.g. `family = binomial`. + +* Result returned by a family function, e.g. `family = poisson(link = log)` + +For more information regarding the families and their link functions, see the Wikipedia page [Generalized Linear Model](https://en.wikipedia.org/wiki/Generalized_linear_model). + +We use the `mtcars` dataset as an illustration. The corresponding `SparkDataFrame` is `carsDF`. After fitting the model, we print out a summary and see the fitted values by making predictions on the original dataset. We can also pass into a new `SparkDataFrame` of same schema to predict on new data. + +```{r} +gaussianGLM <- spark.glm(carsDF, mpg ~ wt + hp) +summary(gaussianGLM) +``` +When doing prediction, a new column called `prediction` will be appended. Let's look at only a subset of columns here. +```{r} +gaussianFitted <- predict(gaussianGLM, carsDF) +head(select(gaussianFitted, "model", "prediction", "mpg", "wt", "hp")) +``` + +#### Naive Bayes Model + +Naive Bayes model assumes independence among the features. `spark.naiveBayes` fits a [Bernoulli naive Bayes model](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Bernoulli_naive_Bayes) against a SparkDataFrame. The data should be all categorical. These models are often used for document classification. + +```{r} +titanic <- as.data.frame(Titanic) +titanicDF <- createDataFrame(titanic[titanic$Freq > 0, -5]) +naiveBayesModel <- spark.naiveBayes(titanicDF, Survived ~ Class + Sex + Age) +summary(naiveBayesModel) +naiveBayesPrediction <- predict(naiveBayesModel, titanicDF) +head(select(naiveBayesPrediction, "Class", "Sex", "Age", "Survived", "prediction")) +``` + +#### k-Means Clustering + +`spark.kmeans` fits a $k$-means clustering model against a `SparkDataFrame`. As an unsupervised learning method, we don't need a response variable. Hence, the left hand side of the R formula should be left blank. The clustering is based only on the variables on the right hand side. + +```{r} +kmeansModel <- spark.kmeans(carsDF, ~ mpg + hp + wt, k = 3) +summary(kmeansModel) +kmeansPredictions <- predict(kmeansModel, carsDF) +head(select(kmeansPredictions, "model", "mpg", "hp", "wt", "prediction"), n = 20L) +``` + +#### AFT Survival Model +Survival analysis studies the expected duration of time until an event happens, and often the relationship with risk factors or treatment taken on the subject. In contrast to standard regression analysis, survival modeling has to deal with special characteristics in the data including non-negative survival time and censoring. + +Accelerated Failure Time (AFT) model is a parametric survival model for censored data that assumes the effect of a covariate is to accelerate or decelerate the life course of an event by some constant. For more information, refer to the Wikipedia page [AFT Model](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) and the references there. Different from a [Proportional Hazards Model](https://en.wikipedia.org/wiki/Proportional_hazards_model) designed for the same purpose, the AFT model is easier to parallelize because each instance contributes to the objective function independently. +```{r} +library(survival) +ovarianDF <- createDataFrame(ovarian) +aftModel <- spark.survreg(ovarianDF, Surv(futime, fustat) ~ ecog_ps + rx) +summary(aftModel) +aftPredictions <- predict(aftModel, ovarianDF) +head(aftPredictions) +``` + +### Model Persistence +The following example shows how to save/load an ML model by SparkR. +```{r} +irisDF <- suppressWarnings(createDataFrame(iris)) +gaussianGLM <- spark.glm(irisDF, Sepal_Length ~ Sepal_Width + Species, family = "gaussian") + +# Save and then load a fitted MLlib model +modelPath <- tempfile(pattern = "ml", fileext = ".tmp") +write.ml(gaussianGLM, modelPath) +gaussianGLM2 <- read.ml(modelPath) + +# Check model summary +summary(gaussianGLM2) + +# Check model prediction +gaussianPredictions <- predict(gaussianGLM2, irisDF) +head(gaussianPredictions) + +unlink(modelPath) +``` + + +## Advanced Topics + +### SparkR Object Classes + +There are three main object classes in SparkR you may be working with. + +* `SparkDataFrame`: the central component of SparkR. It is an S4 class representing distributed collection of data organized into named columns, which is conceptually equivalent to a table in a relational database or a data frame in R. It has two slots `sdf` and `env`. + + `sdf` stores a reference to the corresponding Spark Dataset in the Spark JVM backend. + + `env` saves the meta-information of the object such as `isCached`. + +It can be created by data import methods or by transforming an existing `SparkDataFrame`. We can manipulate `SparkDataFrame` by numerous data processing functions and feed that into machine learning algorithms. + +* `Column`: an S4 class representing column of `SparkDataFrame`. The slot `jc` saves a reference to the corresponding Column object in the Spark JVM backend. + +It can be obtained from a `SparkDataFrame` by `$` operator, `df$col`. More often, it is used together with other functions, for example, with `select` to select particular columns, with `filter` and constructed conditions to select rows, with aggregation functions to compute aggregate statistics for each group. + +* `GroupedData`: an S4 class representing grouped data created by `groupBy` or by transforming other `GroupedData`. Its `sgd` slot saves a reference to a RelationalGroupedDataset object in the backend. + +This is often an intermediate object with group information and followed up by aggregation operations. + +### Architecture + +A complete description of architecture can be seen in reference, in particular the paper *SparkR: Scaling R Programs with Spark*. + +Under the hood of SparkR is Spark SQL engine. This avoids the overheads of running interpreted R code, and the optimized SQL execution engine in Spark uses structural information about data and computation flow to perform a bunch of optimizations to speed up the computation. + +The main method calls of actual computation happen in the Spark JVM of the driver. We have a socket-based SparkR API that allows us to invoke functions on the JVM from R. We use a SparkR JVM backend that listens on a Netty-based socket server. + +Two kinds of RPCs are supported in the SparkR JVM backend: method invocation and creating new objects. Method invocation can be done in two ways. + +* `sparkR.invokeJMethod` takes a reference to an existing Java object and a list of arguments to be passed on to the method. + +* `sparkR.invokeJStatic` takes a class name for static method and a list of arguments to be passed on to the method. + +The arguments are serialized using our custom wire format which is then deserialized on the JVM side. We then use Java reflection to invoke the appropriate method. + +To create objects, `sparkR.newJObject` is used and then similarly the appropriate constructor is invoked with provided arguments. + +Finally, we use a new R class `jobj` that refers to a Java object existing in the backend. These references are tracked on the Java side and are automatically garbage collected when they go out of scope on the R side. + +## Appendix + +### R and Spark Data Types {#DataTypes} + +R | Spark +----------- | ------------- +byte | byte +integer | integer +float | float +double | double +numeric | double +character | string +string | string +binary | binary +raw | binary +logical | boolean +POSIXct | timestamp +POSIXlt | timestamp +Date | date +array | array +list | array +env | map + +## References + +* [Spark Cluster Mode Overview](http://spark.apache.org/docs/latest/cluster-overview.html) + +* [Submitting Spark Applications](http://spark.apache.org/docs/latest/submitting-applications.html) + +* [Machine Learning Library Guide (MLlib)](http://spark.apache.org/docs/latest/ml-guide.html) + +* [SparkR: Scaling R Programs with Spark](https://people.csail.mit.edu/matei/papers/2016/sigmod_sparkr.pdf), Shivaram Venkataraman, Zongheng Yang, Davies Liu, Eric Liang, Hossein Falaki, Xiangrui Meng, Reynold Xin, Ali Ghodsi, Michael Franklin, Ion Stoica, and Matei Zaharia. SIGMOD 2016. June 2016. + +```{r, echo=FALSE} +sparkR.session.stop() +``` From e77a437d292ecda66163a895427d62e4f72e2a25 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 15 Sep 2016 11:22:58 -0700 Subject: [PATCH 1298/1470] [SPARK-17547] Ensure temp shuffle data file is cleaned up after error SPARK-8029 (#9610) modified shuffle writers to first stage their data to a temporary file in the same directory as the final destination file and then to atomically rename this temporary file at the end of the write job. However, this change introduced the potential for the temporary output file to be leaked if an exception occurs during the write because the shuffle writers' existing error cleanup code doesn't handle deletion of the temp file. This patch avoids this potential cause of disk-space leaks by adding `finally` blocks to ensure that temp files are always deleted if they haven't been renamed. Author: Josh Rosen Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer. (cherry picked from commit 5b8f7377d54f83b93ef2bfc2a01ca65fae6d3032) Signed-off-by: Josh Rosen --- .../sort/BypassMergeSortShuffleWriter.java | 10 ++- .../shuffle/sort/UnsafeShuffleWriter.java | 18 +++-- .../shuffle/IndexShuffleBlockResolver.scala | 80 ++++++++++--------- .../shuffle/sort/SortShuffleWriter.scala | 14 +++- 4 files changed, 73 insertions(+), 49 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 0e9defe5b4a51..601dd6edfcf7c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -156,8 +156,14 @@ public void write(Iterator> records) throws IOException { File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); File tmp = Utils.tempFileWith(output); - partitionLengths = writePartitionedFile(tmp); - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + try { + partitionLengths = writePartitionedFile(tmp); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + } finally { + if (tmp.exists() && !tmp.delete()) { + logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } + } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 44e6aa73d9755..c08a5d424ea6a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -207,15 +207,21 @@ void closeAndWriteOutput() throws IOException { final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); final File tmp = Utils.tempFileWith(output); try { - partitionLengths = mergeSpills(spills, tmp); - } finally { - for (SpillInfo spill : spills) { - if (spill.file.exists() && ! spill.file.delete()) { - logger.error("Error while deleting spill file {}", spill.file.getPath()); + try { + partitionLengths = mergeSpills(spills, tmp); + } finally { + for (SpillInfo spill : spills) { + if (spill.file.exists() && ! spill.file.delete()) { + logger.error("Error while deleting spill file {}", spill.file.getPath()); + } } } + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + } finally { + if (tmp.exists() && !tmp.delete()) { + logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } } - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 94d8c0d0fd3e4..8d6396bededa9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -139,48 +139,54 @@ private[spark] class IndexShuffleBlockResolver( dataTmp: File): Unit = { val indexFile = getIndexFile(shuffleId, mapId) val indexTmp = Utils.tempFileWith(indexFile) - val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) - Utils.tryWithSafeFinally { - // We take in lengths of each block, need to convert it to offsets. - var offset = 0L - out.writeLong(offset) - for (length <- lengths) { - offset += length + try { + val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) + Utils.tryWithSafeFinally { + // We take in lengths of each block, need to convert it to offsets. + var offset = 0L out.writeLong(offset) + for (length <- lengths) { + offset += length + out.writeLong(offset) + } + } { + out.close() } - } { - out.close() - } - val dataFile = getDataFile(shuffleId, mapId) - // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure - // the following check and rename are atomic. - synchronized { - val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) - if (existingLengths != null) { - // Another attempt for the same task has already written our map outputs successfully, - // so just use the existing partition lengths and delete our temporary map outputs. - System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) - if (dataTmp != null && dataTmp.exists()) { - dataTmp.delete() - } - indexTmp.delete() - } else { - // This is the first successful attempt in writing the map outputs for this task, - // so override any existing index and data files with the ones we wrote. - if (indexFile.exists()) { - indexFile.delete() - } - if (dataFile.exists()) { - dataFile.delete() - } - if (!indexTmp.renameTo(indexFile)) { - throw new IOException("fail to rename file " + indexTmp + " to " + indexFile) - } - if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { - throw new IOException("fail to rename file " + dataTmp + " to " + dataFile) + val dataFile = getDataFile(shuffleId, mapId) + // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure + // the following check and rename are atomic. + synchronized { + val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) + if (existingLengths != null) { + // Another attempt for the same task has already written our map outputs successfully, + // so just use the existing partition lengths and delete our temporary map outputs. + System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) + if (dataTmp != null && dataTmp.exists()) { + dataTmp.delete() + } + indexTmp.delete() + } else { + // This is the first successful attempt in writing the map outputs for this task, + // so override any existing index and data files with the ones we wrote. + if (indexFile.exists()) { + indexFile.delete() + } + if (dataFile.exists()) { + dataFile.delete() + } + if (!indexTmp.renameTo(indexFile)) { + throw new IOException("fail to rename file " + indexTmp + " to " + indexFile) + } + if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { + throw new IOException("fail to rename file " + dataTmp + " to " + dataFile) + } } } + } finally { + if (indexTmp.exists() && !indexTmp.delete()) { + logError(s"Failed to delete temporary index file at ${indexTmp.getAbsolutePath}") + } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 1adacabc86c05..e677270b84605 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -67,10 +67,16 @@ private[spark] class SortShuffleWriter[K, V, C]( // (see SPARK-3570). val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) val tmp = Utils.tempFileWith(output) - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) - val partitionLengths = sorter.writePartitionedFile(blockId, tmp) - shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + try { + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) + val partitionLengths = sorter.writePartitionedFile(blockId, tmp) + shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + } finally { + if (tmp.exists() && !tmp.delete()) { + logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") + } + } } /** Close this writer, passing along whether the map completed */ From 62ab536588e19293a84004f547ebc316346b869e Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 15 Sep 2016 20:24:15 +0200 Subject: [PATCH 1299/1470] [SPARK-17114][SQL] Fix aggregates grouped by literals with empty input ## What changes were proposed in this pull request? This PR fixes an issue with aggregates that have an empty input, and use a literals as their grouping keys. These aggregates are currently interpreted as aggregates **without** grouping keys, this triggers the ungrouped code path (which aways returns a single row). This PR fixes the `RemoveLiteralFromGroupExpressions` optimizer rule, which changes the semantics of the Aggregate by eliminating all literal grouping keys. ## How was this patch tested? Added tests to `SQLQueryTestSuite`. Author: Herman van Hovell Closes #15101 from hvanhovell/SPARK-17114-3. (cherry picked from commit d403562eb4b5b1d804909861d3e8b75d8f6323b9) Signed-off-by: Herman van Hovell --- .../sql/catalyst/optimizer/Optimizer.scala | 11 +++- .../optimizer/AggregateOptimizeSuite.scala | 10 +++- .../resources/sql-tests/inputs/group-by.sql | 17 +++++++ .../sql-tests/results/group-by.sql.out | 51 +++++++++++++++++++ 4 files changed, 86 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/group-by.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/group-by.sql.out 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 e74389816e007..d824c2e26d717 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 @@ -1439,9 +1439,16 @@ object ReplaceExceptWithAntiJoin extends Rule[LogicalPlan] { */ object RemoveLiteralFromGroupExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case a @ Aggregate(grouping, _, _) => + case a @ Aggregate(grouping, _, _) if grouping.nonEmpty => val newGrouping = grouping.filter(!_.foldable) - a.copy(groupingExpressions = newGrouping) + if (newGrouping.nonEmpty) { + a.copy(groupingExpressions = newGrouping) + } else { + // All grouping expressions are literals. We should not drop them all, because this can + // change the return semantics when the input of the Aggregate is empty (SPARK-17114). We + // instead replace this by single, easy to hash/sort, literal expression. + a.copy(groupingExpressions = Seq(Literal(0, IntegerType))) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala index 4c26c184b7b5b..aecf59aee6a9b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor class AggregateOptimizeSuite extends PlanTest { - val conf = new SimpleCatalystConf(caseSensitiveAnalysis = false) + val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, groupByOrdinal = false) val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) val analyzer = new Analyzer(catalog, conf) @@ -49,6 +49,14 @@ class AggregateOptimizeSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("do not remove all grouping expressions if they are all literals") { + val query = testRelation.groupBy(Literal("1"), Literal(1) + Literal(2))(sum('b)) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = analyzer.execute(testRelation.groupBy(Literal(0))(sum('b))) + + comparePlans(optimized, correctAnswer) + } + test("Remove aliased literals") { val query = testRelation.select('a, Literal(1).as('y)).groupBy('a, 'y)(sum('b)) val optimized = Optimize.execute(analyzer.execute(query)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql new file mode 100644 index 0000000000000..6741703d9d82c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -0,0 +1,17 @@ +-- Temporary data. +create temporary view myview as values 128, 256 as v(int_col); + +-- group by should produce all input rows, +select int_col, count(*) from myview group by int_col; + +-- group by should produce a single row. +select 'foo', count(*) from myview group by 1; + +-- group-by should not produce any rows (whole stage code generation). +select 'foo' from myview where int_col == 0 group by 1; + +-- group-by should not produce any rows (hash aggregate). +select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1; + +-- group-by should not produce any rows (sort aggregate). +select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out new file mode 100644 index 0000000000000..9127bd4dd4c6f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +create temporary view myview as values 128, 256 as v(int_col) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select int_col, count(*) from myview group by int_col +-- !query 1 schema +struct +-- !query 1 output +128 1 +256 1 + + +-- !query 2 +select 'foo', count(*) from myview group by 1 +-- !query 2 schema +struct +-- !query 2 output +foo 2 + + +-- !query 3 +select 'foo' from myview where int_col == 0 group by 1 +-- !query 3 schema +struct +-- !query 3 output + + + +-- !query 4 +select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1 +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1 +-- !query 5 schema +struct> +-- !query 5 output + From abb89c42e760357e2d7eae4be344379c7f0d17f3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Sep 2016 13:09:33 -0700 Subject: [PATCH 1300/1470] [SPARK-17483] Refactoring in BlockManager status reporting and block removal This patch makes three minor refactorings to the BlockManager: - Move the `if (info.tellMaster)` check out of `reportBlockStatus`; this fixes an issue where a debug logging message would incorrectly claim to have reported a block status to the master even though no message had been sent (in case `info.tellMaster == false`). This also makes it easier to write code which unconditionally sends block statuses to the master (which is necessary in another patch of mine). - Split `removeBlock()` into two methods, the existing method and an internal `removeBlockInternal()` method which is designed to be called by internal code that already holds a write lock on the block. This is also needed by a followup patch. - Instead of calling `getCurrentBlockStatus()` in `removeBlock()`, just pass `BlockStatus.empty`; the block status should always be empty following complete removal of a block. These changes were originally authored as part of a bug fix patch which is targeted at branch-2.0 and master; I've split them out here into their own separate PR in order to make them easier to review and so that the behavior-changing parts of my other patch can be isolated to their own PR. Author: Josh Rosen Closes #15036 from JoshRosen/cache-failure-race-conditions-refactorings-only. (cherry picked from commit 3d40896f410590c0be044b3fa7e5d32115fac05e) Signed-off-by: Josh Rosen --- .../apache/spark/storage/BlockManager.scala | 87 +++++++++---------- 1 file changed, 42 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index cd85d97a80b11..48db97abc5f7f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -216,7 +216,7 @@ private[spark] class BlockManager( logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.") for ((blockId, info) <- blockInfoManager.entries) { val status = getCurrentBlockStatus(blockId, info) - if (!tryToReportBlockStatus(blockId, info, status)) { + if (info.tellMaster && !tryToReportBlockStatus(blockId, status)) { logError(s"Failed to report $blockId to master; giving up.") return } @@ -297,7 +297,7 @@ private[spark] class BlockManager( /** * Get the BlockStatus for the block identified by the given ID, if it exists. - * NOTE: This is mainly for testing, and it doesn't fetch information from external block store. + * NOTE: This is mainly for testing. */ def getStatus(blockId: BlockId): Option[BlockStatus] = { blockInfoManager.get(blockId).map { info => @@ -332,10 +332,9 @@ private[spark] class BlockManager( */ private def reportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Unit = { - val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) + val needReregister = !tryToReportBlockStatus(blockId, status, droppedMemorySize) if (needReregister) { logInfo(s"Got told to re-register updating block $blockId") // Re-registering will report our new block for free. @@ -351,17 +350,12 @@ private[spark] class BlockManager( */ private def tryToReportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { - if (info.tellMaster) { - val storageLevel = status.storageLevel - val inMemSize = Math.max(status.memSize, droppedMemorySize) - val onDiskSize = status.diskSize - master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) - } else { - true - } + val storageLevel = status.storageLevel + val inMemSize = Math.max(status.memSize, droppedMemorySize) + val onDiskSize = status.diskSize + master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) } /** @@ -373,7 +367,7 @@ private[spark] class BlockManager( info.synchronized { info.level match { case null => - BlockStatus(StorageLevel.NONE, memSize = 0L, diskSize = 0L) + BlockStatus.empty case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) @@ -808,12 +802,10 @@ private[spark] class BlockManager( // Now that the block is in either the memory, externalBlockStore, or disk store, // tell the master about it. info.size = size - if (tellMaster) { - reportBlockStatus(blockId, info, putBlockStatus) - } - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> putBlockStatus) + if (tellMaster && info.tellMaster) { + reportBlockStatus(blockId, putBlockStatus) } + addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) } logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) if (level.replication > 1) { @@ -962,15 +954,12 @@ private[spark] class BlockManager( val putBlockStatus = getCurrentBlockStatus(blockId, info) val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid if (blockWasSuccessfullyStored) { - // Now that the block is in either the memory, externalBlockStore, or disk store, - // tell the master about it. + // Now that the block is in either the memory or disk store, tell the master about it. info.size = size - if (tellMaster) { - reportBlockStatus(blockId, info, putBlockStatus) - } - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> putBlockStatus) + if (tellMaster && info.tellMaster) { + reportBlockStatus(blockId, putBlockStatus) } + addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) if (level.replication > 1) { val remoteStartTime = System.currentTimeMillis @@ -1272,12 +1261,10 @@ private[spark] class BlockManager( val status = getCurrentBlockStatus(blockId, info) if (info.tellMaster) { - reportBlockStatus(blockId, info, status, droppedMemorySize) + reportBlockStatus(blockId, status, droppedMemorySize) } if (blockIsUpdated) { - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> status) - } + addUpdatedBlockStatusToTaskMetrics(blockId, status) } status.storageLevel } @@ -1317,21 +1304,31 @@ private[spark] class BlockManager( // The block has already been removed; do nothing. logWarning(s"Asked to remove block $blockId, which does not exist") case Some(info) => - // Removals are idempotent in disk store and memory store. At worst, we get a warning. - val removedFromMemory = memoryStore.remove(blockId) - val removedFromDisk = diskStore.remove(blockId) - if (!removedFromMemory && !removedFromDisk) { - logWarning(s"Block $blockId could not be removed as it was not found in either " + - "the disk, memory, or external block store") - } - blockInfoManager.removeBlock(blockId) - val removeBlockStatus = getCurrentBlockStatus(blockId, info) - if (tellMaster && info.tellMaster) { - reportBlockStatus(blockId, info, removeBlockStatus) - } - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> removeBlockStatus) - } + removeBlockInternal(blockId, tellMaster = tellMaster && info.tellMaster) + addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty) + } + } + + /** + * Internal version of [[removeBlock()]] which assumes that the caller already holds a write + * lock on the block. + */ + private def removeBlockInternal(blockId: BlockId, tellMaster: Boolean): Unit = { + // Removals are idempotent in disk store and memory store. At worst, we get a warning. + val removedFromMemory = memoryStore.remove(blockId) + val removedFromDisk = diskStore.remove(blockId) + if (!removedFromMemory && !removedFromDisk) { + logWarning(s"Block $blockId could not be removed as it was not found on disk or in memory") + } + blockInfoManager.removeBlock(blockId) + if (tellMaster) { + reportBlockStatus(blockId, BlockStatus.empty) + } + } + + private def addUpdatedBlockStatusToTaskMetrics(blockId: BlockId, status: BlockStatus): Unit = { + Option(TaskContext.get()).foreach { c => + c.taskMetrics().incUpdatedBlockStatuses(blockId -> status) } } From 0169c2edc35ee918b2972f2f4d4e112ccbdcb0c1 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Thu, 15 Sep 2016 20:53:48 +0200 Subject: [PATCH 1301/1470] [SPARK-17364][SQL] Antlr lexer wrongly treats full qualified identifier as a decimal number token when parsing SQL string ## What changes were proposed in this pull request? The Antlr lexer we use to tokenize a SQL string may wrongly tokenize a fully qualified identifier as a decimal number token. For example, table identifier `default.123_table` is wrongly tokenized as ``` default // Matches lexer rule IDENTIFIER .123 // Matches lexer rule DECIMAL_VALUE _TABLE // Matches lexer rule IDENTIFIER ``` The correct tokenization for `default.123_table` should be: ``` default // Matches lexer rule IDENTIFIER, . // Matches a single dot 123_TABLE // Matches lexer rule IDENTIFIER ``` This PR fix the Antlr grammar so that it can tokenize fully qualified identifier correctly: 1. Fully qualified table name can be parsed correctly. For example, `select * from database.123_suffix`. 2. Fully qualified column name can be parsed correctly, for example `select a.123_suffix from a`. ### Before change #### Case 1: Failed to parse fully qualified column name ``` scala> spark.sql("select a.123_column from a").show org.apache.spark.sql.catalyst.parser.ParseException: extraneous input '.123' expecting {, ... , IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 8) == SQL == select a.123_column from a --------^^^ ``` #### Case 2: Failed to parse fully qualified table name ``` scala> spark.sql("select * from default.123_table") org.apache.spark.sql.catalyst.parser.ParseException: extraneous input '.123' expecting {, ... IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 21) == SQL == select * from default.123_table ---------------------^^^ ``` ### After Change #### Case 1: fully qualified column name, no ParseException thrown ``` scala> spark.sql("select a.123_column from a").show ``` #### Case 2: fully qualified table name, no ParseException thrown ``` scala> spark.sql("select * from default.123_table") ``` ## How was this patch tested? Unit test. Author: Sean Zhong Closes #15006 from clockfly/SPARK-17364. (cherry picked from commit a6b8182006d0c3dda67c06861067ca78383ecf1b) Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/parser/SqlBase.g4 | 44 +++++++++++++++---- .../parser/ExpressionParserSuite.scala | 15 ++++++- .../parser/TableIdentifierParserSuite.scala | 13 ++++++ 3 files changed, 63 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4b47fa3d40ab2..8b721407eb170 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -16,6 +16,30 @@ grammar SqlBase; +@members { + /** + * Verify whether current token is a valid decimal token (which contains dot). + * Returns true if the character that follows the token is not a digit or letter or underscore. + * + * For example: + * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'. + * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'. + * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'. + * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is folllowed + * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+' + * which is not a digit or letter or underscore. + */ + public boolean isValidDecimal() { + int nextChar = _input.LA(1); + if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' || + nextChar == '_') { + return false; + } else { + return true; + } + } +} + tokens { DELIMITER } @@ -908,23 +932,22 @@ INTEGER_VALUE ; DECIMAL_VALUE - : DIGIT+ '.' DIGIT* - | '.' DIGIT+ + : DECIMAL_DIGITS {isValidDecimal()}? ; SCIENTIFIC_DECIMAL_VALUE - : DIGIT+ ('.' DIGIT*)? EXPONENT - | '.' DIGIT+ EXPONENT + : DIGIT+ EXPONENT + | DECIMAL_DIGITS EXPONENT {isValidDecimal()}? ; DOUBLE_LITERAL - : - (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'D' + : DIGIT+ EXPONENT? 'D' + | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}? ; BIGDECIMAL_LITERAL - : - (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'BD' + : DIGIT+ EXPONENT? 'BD' + | DECIMAL_DIGITS EXPONENT? 'BD' {isValidDecimal()}? ; IDENTIFIER @@ -935,6 +958,11 @@ BACKQUOTED_IDENTIFIER : '`' ( ~'`' | '``' )* '`' ; +fragment DECIMAL_DIGITS + : DIGIT+ '.' DIGIT* + | '.' DIGIT+ + ; + fragment EXPONENT : 'E' [+-]? DIGIT+ ; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index dbc5db39aed98..4aaae72fe91ef 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -517,4 +517,17 @@ class ExpressionParserSuite extends PlanTest { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } + + test("SPARK-17364, fully qualified column name which starts with number") { + assertEqual("123_", UnresolvedAttribute("123_")) + assertEqual("1a.123_", UnresolvedAttribute("1a.123_")) + // ".123" should not be treated as token of type DECIMAL_VALUE + assertEqual("a.123A", UnresolvedAttribute("a.123A")) + // ".123E3" should not be treated as token of type SCIENTIFIC_DECIMAL_VALUE + assertEqual("a.123E3_column", UnresolvedAttribute("a.123E3_column")) + // ".123D" should not be treated as token of type DOUBLE_LITERAL + assertEqual("a.123D_column", UnresolvedAttribute("a.123D_column")) + // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL + assertEqual("a.123BD_column", UnresolvedAttribute("a.123BD_column")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 8bbf87e62d412..4d3ad2179139b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -83,4 +83,17 @@ class TableIdentifierParserSuite extends SparkFunSuite { assert(TableIdentifier(nonReserved) === parseTableIdentifier(nonReserved)) } } + + test("SPARK-17364 table identifier - contains number") { + assert(parseTableIdentifier("123_") == TableIdentifier("123_")) + assert(parseTableIdentifier("1a.123_") == TableIdentifier("123_", Some("1a"))) + // ".123" should not be treated as token of type DECIMAL_VALUE + assert(parseTableIdentifier("a.123A") == TableIdentifier("123A", Some("a"))) + // ".123E3" should not be treated as token of type SCIENTIFIC_DECIMAL_VALUE + assert(parseTableIdentifier("a.123E3_LIST") == TableIdentifier("123E3_LIST", Some("a"))) + // ".123D" should not be treated as token of type DOUBLE_LITERAL + assert(parseTableIdentifier("a.123D_LIST") == TableIdentifier("123D_LIST", Some("a"))) + // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL + assert(parseTableIdentifier("a.123BD_LIST") == TableIdentifier("123BD_LIST", Some("a"))) + } } From 9c23f4408d337f4af31ebfbcc78767df67d36aed Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 15 Sep 2016 11:54:17 -0700 Subject: [PATCH 1302/1470] [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions ## What changes were proposed in this pull request? If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed. This patch addresses this issue via multiple small changes: - The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup). - When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present. - Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls. This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix. For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484). ## How was this patch tested? Two new regression tests in BlockManagerSuite. Author: Josh Rosen Closes #15085 from JoshRosen/SPARK-17484. (cherry picked from commit 1202075c95eabba0ffebc170077df798f271a139) Signed-off-by: Josh Rosen --- .../apache/spark/storage/BlockManager.scala | 37 +++++++++++++++---- .../spark/storage/BlockManagerSuite.scala | 34 +++++++++++++++++ 2 files changed, 63 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 48db97abc5f7f..37dfbd6818ef7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -279,7 +279,12 @@ private[spark] class BlockManager( } else { getLocalBytes(blockId) match { case Some(buffer) => new BlockManagerManagedBuffer(blockInfoManager, blockId, buffer) - case None => throw new BlockNotFoundException(blockId.toString) + case None => + // If this block manager receives a request for a block that it doesn't have then it's + // likely that the master has outdated block statuses for this block. Therefore, we send + // an RPC so that this block is marked as being unavailable from this block manager. + reportBlockStatus(blockId, BlockStatus.empty) + throw new BlockNotFoundException(blockId.toString) } } } @@ -856,22 +861,38 @@ private[spark] class BlockManager( } val startTimeMs = System.currentTimeMillis - var blockWasSuccessfullyStored: Boolean = false + var exceptionWasThrown: Boolean = true val result: Option[T] = try { val res = putBody(putBlockInfo) - blockWasSuccessfullyStored = res.isEmpty - res - } finally { - if (blockWasSuccessfullyStored) { + exceptionWasThrown = false + if (res.isEmpty) { + // the block was successfully stored if (keepReadLock) { blockInfoManager.downgradeLock(blockId) } else { blockInfoManager.unlock(blockId) } } else { - blockInfoManager.removeBlock(blockId) + removeBlockInternal(blockId, tellMaster = false) logWarning(s"Putting block $blockId failed") } + res + } finally { + // This cleanup is performed in a finally block rather than a `catch` to avoid having to + // catch and properly re-throw InterruptedException. + if (exceptionWasThrown) { + logWarning(s"Putting block $blockId failed due to an exception") + // If an exception was thrown then it's possible that the code in `putBody` has already + // notified the master about the availability of this block, so we need to send an update + // to remove this block location. + removeBlockInternal(blockId, tellMaster = tellMaster) + // The `putBody` code may have also added a new block status to TaskMetrics, so we need + // to cancel that out by overwriting it with an empty block status. We only do this if + // the finally block was entered via an exception because doing this unconditionally would + // cause us to send empty block statuses for every block that failed to be cached due to + // a memory shortage (which is an expected failure, unlike an uncaught exception). + addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty) + } } if (level.replication > 1) { logDebug("Putting block %s with replication took %s" @@ -1170,7 +1191,7 @@ private[spark] class BlockManager( done = true // specified number of peers have been replicated to } } catch { - case e: Exception => + case NonFatal(e) => logWarning(s"Failed to replicate $blockId to $peer, failure #$failures", e) failures += 1 replicationFailed = true 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 2f594b8a52beb..6194d23f4d8da 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -859,6 +859,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) memoryManager.setMemoryStore(store.memoryStore) + store.initialize("app-id") // The put should fail since a1 is not serializable. class UnserializableClass @@ -1204,6 +1205,39 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE verify(mockBlockManagerMaster, times(2)).getLocations("item") } + test("SPARK-17484: block status is properly updated following an exception in put()") { + val mockBlockTransferService = new MockBlockTransferService(maxFailures = 10) { + override def uploadBlock( + hostname: String, + port: Int, execId: String, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] = { + throw new InterruptedException("Intentional interrupt") + } + } + store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + store2 = makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) + intercept[InterruptedException] { + store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) + } + assert(store.getLocalBytes("item").isEmpty) + assert(master.getLocations("item").isEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + } + + test("SPARK-17484: master block locations are updated following an invalid remote block fetch") { + store = makeBlockManager(8000, "executor1") + store2 = makeBlockManager(8000, "executor2") + store.putSingle("item", "value", StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(master.getLocations("item").nonEmpty) + store.removeBlock("item", tellMaster = false) + assert(master.getLocations("item").nonEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + assert(master.getLocations("item").isEmpty) + } + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { var numCalls = 0 From 5ad4395e1b41d5ec74785c0aef5c2f656f9db9da Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 16 Sep 2016 11:24:26 -0700 Subject: [PATCH 1303/1470] [SPARK-17558] Bump Hadoop 2.7 version from 2.7.2 to 2.7.3 ## What changes were proposed in this pull request? This patch bumps the Hadoop version in hadoop-2.7 profile from 2.7.2 to 2.7.3, which was recently released and contained a number of bug fixes. ## How was this patch tested? The change should be covered by existing tests. Author: Reynold Xin Closes #15115 from rxin/SPARK-17558. (cherry picked from commit dca771bec6edb1cd8fc75861d364e0ba9dccf7c3) Signed-off-by: Reynold Xin --- dev/deps/spark-deps-hadoop-2.7 | 30 +++++++++++++++--------------- pom.xml | 2 +- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 3da0860a10f94..a61f31eb87696 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -58,21 +58,21 @@ gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar -hadoop-annotations-2.7.2.jar -hadoop-auth-2.7.2.jar -hadoop-client-2.7.2.jar -hadoop-common-2.7.2.jar -hadoop-hdfs-2.7.2.jar -hadoop-mapreduce-client-app-2.7.2.jar -hadoop-mapreduce-client-common-2.7.2.jar -hadoop-mapreduce-client-core-2.7.2.jar -hadoop-mapreduce-client-jobclient-2.7.2.jar -hadoop-mapreduce-client-shuffle-2.7.2.jar -hadoop-yarn-api-2.7.2.jar -hadoop-yarn-client-2.7.2.jar -hadoop-yarn-common-2.7.2.jar -hadoop-yarn-server-common-2.7.2.jar -hadoop-yarn-server-web-proxy-2.7.2.jar +hadoop-annotations-2.7.3.jar +hadoop-auth-2.7.3.jar +hadoop-client-2.7.3.jar +hadoop-common-2.7.3.jar +hadoop-hdfs-2.7.3.jar +hadoop-mapreduce-client-app-2.7.3.jar +hadoop-mapreduce-client-common-2.7.3.jar +hadoop-mapreduce-client-core-2.7.3.jar +hadoop-mapreduce-client-jobclient-2.7.3.jar +hadoop-mapreduce-client-shuffle-2.7.3.jar +hadoop-yarn-api-2.7.3.jar +hadoop-yarn-client-2.7.3.jar +hadoop-yarn-common-2.7.3.jar +hadoop-yarn-server-common-2.7.3.jar +hadoop-yarn-server-web-proxy-2.7.3.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar diff --git a/pom.xml b/pom.xml index ee0032a8582c5..a723283a4e53f 100644 --- a/pom.xml +++ b/pom.xml @@ -2511,7 +2511,7 @@ hadoop-2.7 - 2.7.2 + 2.7.3 0.9.3 3.4.6 2.6.0 From 3fce1255ad41a04e92720795ce2b162ec305cf0a Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 16 Sep 2016 14:02:56 -0700 Subject: [PATCH 1304/1470] [SPARK-17549][SQL] Only collect table size stat in driver for cached relation. The existing code caches all stats for all columns for each partition in the driver; for a large relation, this causes extreme memory usage, which leads to gc hell and application failures. It seems that only the size in bytes of the data is actually used in the driver, so instead just colllect that. In executors, the full stats are still kept, but that's not a big problem; we expect the data to be distributed and thus not really incur in too much memory pressure in each individual executor. There are also potential improvements on the executor side, since the data being stored currently is very wasteful (e.g. storing boxed types vs. primitive types for stats). But that's a separate issue. On a mildly related change, I'm also adding code to catch exceptions in the code generator since Janino was breaking with the test data I tried this patch on. Tested with unit tests and by doing a count a very wide table (20k columns) with many partitions. Author: Marcelo Vanzin Closes #15112 from vanzin/SPARK-17549. (cherry picked from commit 39e2bad6a866d27c3ca594d15e574a1da3ee84cc) Signed-off-by: Yin Huai --- .../expressions/codegen/CodeGenerator.scala | 18 +++++++++----- .../execution/columnar/InMemoryRelation.scala | 24 +++++-------------- .../columnar/InMemoryColumnarQuerySuite.scala | 14 +++++++++++ 3 files changed, 32 insertions(+), 24 deletions(-) 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 4bd9ee03f96dd..929f2da07531e 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 @@ -23,6 +23,7 @@ import java.util.{Map => JavaMap} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal import com.google.common.cache.{CacheBuilder, CacheLoader} import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, SimpleCompiler} @@ -914,14 +915,19 @@ object CodeGenerator extends Logging { codeAttrField.setAccessible(true) classes.foreach { case (_, classBytes) => CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length) - val cf = new ClassFile(new ByteArrayInputStream(classBytes)) - cf.methodInfos.asScala.foreach { method => - method.getAttributes().foreach { a => - if (a.getClass.getName == codeAttr.getName) { - CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update( - codeAttrField.get(a).asInstanceOf[Array[Byte]].length) + try { + val cf = new ClassFile(new ByteArrayInputStream(classBytes)) + cf.methodInfos.asScala.foreach { method => + method.getAttributes().foreach { a => + if (a.getClass.getName == codeAttr.getName) { + CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update( + codeAttrField.get(a).asInstanceOf[Array[Byte]].length) + } } } + } catch { + case NonFatal(e) => + logWarning("Error calculating stats of compiled class.", e) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 479934a7afc75..56bd5c1891e8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.columnar -import scala.collection.JavaConverters._ - import org.apache.commons.lang3.StringUtils import org.apache.spark.network.util.JavaUtils @@ -31,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.CollectionAccumulator +import org.apache.spark.util.LongAccumulator object InMemoryRelation { @@ -63,8 +61,7 @@ case class InMemoryRelation( @transient child: SparkPlan, tableName: Option[String])( @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, - val batchStats: CollectionAccumulator[InternalRow] = - child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) + val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator) extends logical.LeafNode with MultiInstanceRelation { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) @@ -74,21 +71,12 @@ case class InMemoryRelation( @transient val partitionStatistics = new PartitionStatistics(output) override lazy val statistics: Statistics = { - if (batchStats.value.isEmpty) { + if (batchStats.value == 0L) { // Underlying columnar RDD hasn't been materialized, no useful statistics information // available, return the default statistics. Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) } else { - // Underlying columnar RDD has been materialized, required information has also been - // collected via the `batchStats` accumulator. - val sizeOfRow: Expression = - BindReferences.bindReference( - output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), - partitionStatistics.schema) - - val sizeInBytes = - batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum - Statistics(sizeInBytes = sizeInBytes) + Statistics(sizeInBytes = batchStats.value.longValue) } } @@ -139,10 +127,10 @@ case class InMemoryRelation( rowCount += 1 } + batchStats.add(totalSize) + val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) .flatMap(_.values)) - - batchStats.add(stats) CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 937839644ad5f..0daa29b666f62 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -232,4 +232,18 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val columnTypes2 = List.fill(length2)(IntegerType) val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) } + + test("SPARK-17549: cached table size should be correctly calculated") { + val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None) + + // Materialize the data. + val expectedAnswer = data.collect() + checkAnswer(cached, expectedAnswer) + + // Check that the right size was calculated. + assert(cached.batchStats.value === expectedAnswer.size * INT.defaultSize) + } + } From 9ff158b81224c106d50e087c0d284b0c86c95879 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Sat, 17 Sep 2016 12:28:42 +0100 Subject: [PATCH 1305/1470] Correct fetchsize property name in docs ## What changes were proposed in this pull request? Replace `fetchSize` with `fetchsize` in the docs. ## How was this patch tested? I manually tested `fetchSize` and `fetchsize`. The latter has an effect. See also [`JdbcUtils.scala#L38`](https://github.com/apache/spark/blob/v2.0.0/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala#L38) for the definition of the property. Author: Daniel Darabos Closes #14975 from darabos/patch-3. (cherry picked from commit 69cb0496974737347e2650cda436b39bbd51e581) Signed-off-by: Sean Owen --- docs/sql-programming-guide.md | 2 +- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3b01dc82bd7b5..0bd0093620a31 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1041,7 +1041,7 @@ the Data Sources API. The following options are supported: - fetchSize + fetchsize The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). 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 995b1200a2294..ec419e44e7930 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 @@ -289,7 +289,7 @@ class JDBCSuite extends SparkFunSuite assert(names(2).equals("mary")) } - test("SELECT first field when fetchSize is two") { + test("SELECT first field when fetchsize is two") { val names = sql("SELECT NAME FROM fetchtwo").collect().map(x => x.getString(0)).sortWith(_ < _) assert(names.size === 3) assert(names(0).equals("fred")) @@ -305,7 +305,7 @@ class JDBCSuite extends SparkFunSuite assert(ids(2) === 3) } - test("SELECT second field when fetchSize is two") { + test("SELECT second field when fetchsize is two") { val ids = sql("SELECT THEID FROM fetchtwo").collect().map(x => x.getInt(0)).sortWith(_ < _) assert(ids.size === 3) assert(ids(0) === 1) @@ -352,7 +352,7 @@ class JDBCSuite extends SparkFunSuite urlWithUserAndPass, "TEST.PEOPLE", new Properties()).collect().length === 3) } - test("Basic API with illegal FetchSize") { + test("Basic API with illegal fetchsize") { val properties = new Properties() properties.setProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "-1") val e = intercept[SparkException] { From 3ca0dc00786df1d529d55e297aaf23e1e1e07999 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Sat, 17 Sep 2016 12:30:25 +0100 Subject: [PATCH 1306/1470] [SPARK-17567][DOCS] Use valid url to Spark RDD paper https://issues.apache.org/jira/browse/SPARK-17567 ## What changes were proposed in this pull request? Documentation (http://spark.apache.org/docs/latest/api/scala/#org.apache.spark.rdd.RDD) contains broken link to Spark paper (http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf). I found it elsewhere (https://www.usenix.org/system/files/conference/nsdi12/nsdi12-final138.pdf) and I hope it is the same one. It should be uploaded to and linked from some Apache controlled storage, so it won't break again. ## How was this patch tested? Tested manually on local laptop. Author: Xin Ren Closes #15121 from keypointt/SPARK-17567. (cherry picked from commit f15d41be3ce7569736ccbf2ffe1bec265865f55d) Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 2ee13dc4db5fd..34d32aacfb627 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -70,7 +70,7 @@ import org.apache.spark.util.random.{BernoulliCellSampler, BernoulliSampler, Poi * All of the scheduling and execution in Spark is done based on these methods, allowing each RDD * to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for * reading data from a new storage system) by overriding these functions. Please refer to the - * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details + * [[http://people.csail.mit.edu/matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details * on RDD internals. */ abstract class RDD[T: ClassTag]( From c9bd67e94d9d9d2e1f2cb1e5c4bb71a69b1e1d4e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 16 Sep 2016 13:43:05 -0700 Subject: [PATCH 1307/1470] [SPARK-17561][DOCS] DataFrameWriter documentation formatting problems Fix `
      /
    • ` problems in SQL scaladoc. Scaladoc build and manual verification of generated HTML. Author: Sean Owen Closes #15117 from srowen/SPARK-17561. (cherry picked from commit b9323fc9381a09af510f542fd5c86473e029caf6) Signed-off-by: Sean Owen --- .../apache/spark/sql/DataFrameReader.scala | 32 ++++++++-------- .../apache/spark/sql/DataFrameWriter.scala | 10 +++++ .../sql/streaming/DataStreamReader.scala | 38 ++++++++++++------- 3 files changed, 51 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 083c2e2c65a03..410cb20d2de11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -269,14 +269,15 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • - `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts - * the malformed string into a new field configured by `columnNameOfCorruptRecord`. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • - `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • - `FAILFAST` : throws an exception when it meets corrupted records.
      • - *
      + * during parsing. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts + * the malformed string into a new field configured by `columnNameOfCorruptRecord`. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + * *
    • `columnNameOfCorruptRecord` (default is the value specified in * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • @@ -396,13 +397,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    • `maxMalformedLogPerPartition` (default `10`): sets the maximum number of malformed rows * Spark will log for each partition. Malformed records beyond this number will be ignored.
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • - `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • - `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • - `FAILFAST` : throws an exception when it meets corrupted records.
      • - *
      + * during parsing. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + * *
    * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 767af99b4d3ee..a4c4a5defa1b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -449,6 +449,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following JSON-specific option(s) for writing JSON files: + *
      *
    • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
    • @@ -458,6 +459,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    * * @since 1.4.0 */ @@ -473,10 +475,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following Parquet-specific option(s) for writing Parquet files: + *
      *
    • `compression` (default is the value specified in `spark.sql.parquet.compression.codec`): * compression codec to use when saving to file. This can be one of the known case-insensitive * shorten names(none, `snappy`, `gzip`, and `lzo`). This will override * `spark.sql.parquet.compression.codec`.
    • + *
    * * @since 1.4.0 */ @@ -492,9 +496,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following ORC-specific option(s) for writing ORC files: + *
      *
    • `compression` (default `snappy`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names(`none`, `snappy`, `zlib`, and `lzo`). * This will override `orc.compress`.
    • + *
    * * @since 1.5.0 * @note Currently, this method can only be used after enabling Hive support @@ -516,9 +522,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following option(s) for writing text files: + *
      *
    • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
    • + *
    * * @since 1.6.0 */ @@ -534,6 +542,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following CSV-specific option(s) for writing CSV files: + *
      *
    • `sep` (default `,`): sets the single character as a separator for each * field and value.
    • *
    • `quote` (default `"`): sets the single character used for escaping quoted values where @@ -556,6 +565,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index e0a19b1da55a7..613177e43332c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -161,6 +161,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * schema in advance, use the version that specifies the schema to avoid the extra scan. * * You can set the following JSON-specific options to deal with non-standard JSON files: + *
      *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
    • *
    • `primitivesAsString` (default `false`): infers all primitive values as a string type
    • @@ -175,14 +176,15 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
    • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the - * malformed string into a new field configured by `columnNameOfCorruptRecord`. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • - *
      + * during parsing. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts + * the malformed string into a new field configured by `columnNameOfCorruptRecord`. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + * *
    • `columnNameOfCorruptRecord` (default is the value specified in * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • @@ -192,6 +194,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    * * @since 2.0.0 */ @@ -207,6 +210,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * specify the schema explicitly using [[schema]]. * * You can set the following CSV-specific options to deal with CSV files: + *
      *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
    • *
    • `sep` (default `,`): sets the single character as a separator for each @@ -245,12 +249,14 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
    • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed * for any given value being read.
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + * during parsing. + *
          + *
        • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + * a schema is set by user, it sets `null` for extra fields.
        • + *
        • `DROPMALFORMED` : ignores the whole corrupted records.
        • + *
        • `FAILFAST` : throws an exception when it meets corrupted records.
        • + *
        + * *
      * * @since 2.0.0 @@ -263,11 +269,13 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * Loads a Parquet file stream, returning the result as a [[DataFrame]]. * * You can set the following Parquet-specific option(s) for reading Parquet files: + *
        *
      • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
      • *
      • `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets * whether we should merge schemas collected from all Parquet part-files. This will override * `spark.sql.parquet.mergeSchema`.
      • + *
      * * @since 2.0.0 */ @@ -291,8 +299,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * }}} * * You can set the following text-specific options to deal with text files: + *
        *
      • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
      • + *
      * * @since 2.0.0 */ From eb2675de92b865852d7aa3ef25a20e6cff940299 Mon Sep 17 00:00:00 2001 From: William Benton Date: Sat, 17 Sep 2016 12:49:58 +0100 Subject: [PATCH 1308/1470] [SPARK-17548][MLLIB] Word2VecModel.findSynonyms no longer spuriously rejects the best match when invoked with a vector ## What changes were proposed in this pull request? This pull request changes the behavior of `Word2VecModel.findSynonyms` so that it will not spuriously reject the best match when invoked with a vector that does not correspond to a word in the model's vocabulary. Instead of blindly discarding the best match, the changed implementation discards a match that corresponds to the query word (in cases where `findSynonyms` is invoked with a word) or that has an identical angle to the query vector. ## How was this patch tested? I added a test to `Word2VecSuite` to ensure that the word with the most similar vector from a supplied vector would not be spuriously rejected. Author: William Benton Closes #15105 from willb/fix/findSynonyms. (cherry picked from commit 25cbbe6ca334140204e7035ab8b9d304da9b8a8a) Signed-off-by: Sean Owen --- .../apache/spark/ml/feature/Word2Vec.scala | 20 +++++----- .../api/python/Word2VecModelWrapper.scala | 22 +++++++++-- .../apache/spark/mllib/feature/Word2Vec.scala | 37 ++++++++++++++----- .../spark/mllib/feature/Word2VecSuite.scala | 16 ++++++++ python/pyspark/mllib/feature.py | 12 ++++-- 5 files changed, 83 insertions(+), 24 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index c2b434c3d5cb1..14c05123c62ed 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -221,24 +221,26 @@ class Word2VecModel private[ml] ( } /** - * Find "num" number of words closest in similarity to the given word. - * Returns a dataframe with the words and the cosine similarities between the - * synonyms and the given word. + * Find "num" number of words closest in similarity to the given word, not + * including the word itself. Returns a dataframe with the words and the + * cosine similarities between the synonyms and the given word. */ @Since("1.5.0") def findSynonyms(word: String, num: Int): DataFrame = { - findSynonyms(wordVectors.transform(word), num) + val spark = SparkSession.builder().getOrCreate() + spark.createDataFrame(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") } /** - * Find "num" number of words closest to similarity to the given vector representation - * of the word. Returns a dataframe with the words and the cosine similarities between the - * synonyms and the given word vector. + * Find "num" number of words whose vector representation most similar to the supplied vector. + * If the supplied vector is the vector representation of a word in the model's vocabulary, + * that word will be in the results. Returns a dataframe with the words and the cosine + * similarities between the synonyms and the given word vector. */ @Since("2.0.0") - def findSynonyms(word: Vector, num: Int): DataFrame = { + def findSynonyms(vec: Vector, num: Int): DataFrame = { val spark = SparkSession.builder().getOrCreate() - spark.createDataFrame(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") + spark.createDataFrame(wordVectors.findSynonyms(vec, num)).toDF("word", "similarity") } /** @group setParam */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala index 4b4ed2291d139..5cbfbff3e4a62 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala @@ -43,18 +43,34 @@ private[python] class Word2VecModelWrapper(model: Word2VecModel) { rdd.rdd.map(model.transform) } + /** + * Finds synonyms of a word; do not include the word itself in results. + * @param word a word + * @param num number of synonyms to find + * @return a list consisting of a list of words and a vector of cosine similarities + */ def findSynonyms(word: String, num: Int): JList[Object] = { - val vec = transform(word) - findSynonyms(vec, num) + prepareResult(model.findSynonyms(word, num)) } + /** + * Finds words similar to the the vector representation of a word without + * filtering results. + * @param vector a vector + * @param num number of synonyms to find + * @return a list consisting of a list of words and a vector of cosine similarities + */ def findSynonyms(vector: Vector, num: Int): JList[Object] = { - val result = model.findSynonyms(vector, num) + prepareResult(model.findSynonyms(vector, num)) + } + + private def prepareResult(result: Array[(String, Double)]) = { val similarity = Vectors.dense(result.map(_._2)) val words = result.map(_._1) List(words, similarity).map(_.asInstanceOf[Object]).asJava } + def getVectors: JMap[String, JList[Float]] = { model.getVectors.map { case (k, v) => (k, v.toList.asJava) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index bc75646d532dc..761996f44739a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -518,7 +518,7 @@ class Word2VecModel private[spark] ( } /** - * Find synonyms of a word + * Find synonyms of a word; do not include the word itself in results. * @param word a word * @param num number of synonyms to find * @return array of (word, cosineSimilarity) @@ -526,17 +526,34 @@ class Word2VecModel private[spark] ( @Since("1.1.0") def findSynonyms(word: String, num: Int): Array[(String, Double)] = { val vector = transform(word) - findSynonyms(vector, num) + findSynonyms(vector, num, Some(word)) } /** - * Find synonyms of the vector representation of a word + * Find synonyms of the vector representation of a word, possibly + * including any words in the model vocabulary whose vector respresentation + * is the supplied vector. * @param vector vector representation of a word * @param num number of synonyms to find * @return array of (word, cosineSimilarity) */ @Since("1.1.0") def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { + findSynonyms(vector, num, None) + } + + /** + * Find synonyms of the vector representation of a word, rejecting + * words identical to the value of wordOpt, if one is supplied. + * @param vector vector representation of a word + * @param num number of synonyms to find + * @param wordOpt optionally, a word to reject from the results list + * @return array of (word, cosineSimilarity) + */ + private def findSynonyms( + vector: Vector, + num: Int, + wordOpt: Option[String]): Array[(String, Double)] = { require(num > 0, "Number of similar words should > 0") // TODO: optimize top-k val fVector = vector.toArray.map(_.toFloat) @@ -563,12 +580,14 @@ class Word2VecModel private[spark] ( ind += 1 } - wordList.zip(cosVec) - .toSeq - .sortBy(-_._2) - .take(num + 1) - .tail - .toArray + val scored = wordList.zip(cosVec).toSeq.sortBy(-_._2) + + val filtered = wordOpt match { + case Some(w) => scored.take(num + 1).filter(tup => w != tup._1) + case None => scored + } + + filtered.take(num).toArray } /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index 22de4c4ac40e6..f4fa216b8eba0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.Utils @@ -68,6 +69,21 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { assert(syms(1)._1 == "japan") } + test("findSynonyms doesn't reject similar word vectors when called with a vector") { + val num = 2 + val word2VecMap = Map( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val model = new Word2VecModel(word2VecMap) + val syms = model.findSynonyms(Vectors.dense(Array(0.52, 0.5, 0.5, 0.5)), num) + assert(syms.length == num) + assert(syms(0)._1 == "china") + assert(syms(1)._1 == "taiwan") + } + test("model load / save") { val word2VecMap = Map( diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index c8a6e33f4d9a4..929531862d18f 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -545,8 +545,7 @@ def load(cls, sc, path): @ignore_unicode_prefix class Word2Vec(object): - """ - Word2Vec creates vector representation of words in a text corpus. + """Word2Vec creates vector representation of words in a text corpus. The algorithm first constructs a vocabulary from the corpus and then learns vector representation of words in the vocabulary. The vector representation can be used as features in @@ -568,13 +567,19 @@ class Word2Vec(object): >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) >>> model = Word2Vec().setVectorSize(10).setSeed(42).fit(doc) + Querying for synonyms of a word will not return that word: + >>> syms = model.findSynonyms("a", 2) >>> [s[0] for s in syms] [u'b', u'c'] + + But querying for synonyms of a vector may return the word whose + representation is that vector: + >>> vec = model.transform("a") >>> syms = model.findSynonyms(vec, 2) >>> [s[0] for s in syms] - [u'b', u'c'] + [u'a', u'b'] >>> import os, tempfile >>> path = tempfile.mkdtemp() @@ -592,6 +597,7 @@ class Word2Vec(object): ... pass .. versionadded:: 1.2.0 + """ def __init__(self): """ From ec2b736566b69a1549791f3d86b55cb0249a757d Mon Sep 17 00:00:00 2001 From: sandy Date: Sat, 17 Sep 2016 16:25:03 +0100 Subject: [PATCH 1309/1470] [SPARK-17575][DOCS] Remove extra table tags in configuration document ## What changes were proposed in this pull request? Remove extra table tags in configurations document. ## How was this patch tested? Run all test cases and generate document. Before with extra tag its look like below ![config-wrong1](https://cloud.githubusercontent.com/assets/8075390/18608239/c602bb60-7d01-11e6-875e-f38558997dd3.png) ![config-wrong2](https://cloud.githubusercontent.com/assets/8075390/18608241/cf3b672c-7d01-11e6-935e-1e73f9e6e578.png) After removing tags its looks like below ![config](https://cloud.githubusercontent.com/assets/8075390/18608245/e156eb8e-7d01-11e6-98aa-3be68d4d1961.png) ![config2](https://cloud.githubusercontent.com/assets/8075390/18608247/e84eecd4-7d01-11e6-9738-a3f7ff8fe834.png) Author: sandy Closes #15130 from phalodi/SPARK-17575. (cherry picked from commit bbe0b1d623741decce98827130cc67eb1fff1240) Signed-off-by: Sean Owen --- docs/configuration.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index d37da023db794..db088dde11919 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -123,6 +123,7 @@ of the most common options to set are: Number of cores to use for the driver process, only in cluster mode. + spark.driver.maxResultSize 1g @@ -217,7 +218,7 @@ Apart from these, the following properties are also available, and may be useful
      Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-class-path command line option or in - your default properties file. + your default properties file. @@ -244,7 +245,7 @@ Apart from these, the following properties are also available, and may be useful
      Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-library-path command line option or in - your default properties file. + your default properties file. From a3bba372abce926351335d0a2936b70988f19b23 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sat, 17 Sep 2016 16:52:30 +0100 Subject: [PATCH 1310/1470] [SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.length/size which is O(n) This PR fixes all the instances which was fixed in the previous PR. To make sure, I manually debugged and also checked the Scala source. `length` in [LinearSeqOptimized.scala#L49-L57](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/LinearSeqOptimized.scala#L49-L57) is O(n). Also, `size` calls `length` via [SeqLike.scala#L106](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/SeqLike.scala#L106). For debugging, I have created these as below: ```scala ArrayBuffer(1, 2, 3) Array(1, 2, 3) List(1, 2, 3) Seq(1, 2, 3) ``` and then called `size` and `length` for each to debug. I ran the bash as below on Mac ```bash find . -name *.scala -type f -exec grep -il "while (.*\\.length)" {} \; | grep "src/main" find . -name *.scala -type f -exec grep -il "while (.*\\.size)" {} \; | grep "src/main" ``` and then checked each. Author: hyukjinkwon Closes #15093 from HyukjinKwon/SPARK-17480-followup. (cherry picked from commit 86c2d393a56bf1e5114bc5a781253c0460efb8af) Signed-off-by: Sean Owen --- .../sql/catalyst/analysis/Analyzer.scala | 28 ++++++------------- .../expressions/conditionalExpressions.scala | 3 +- .../sql/catalyst/expressions/ordering.scala | 3 +- .../sql/catalyst/util/QuantileSummaries.scala | 0 .../spark/sql/hive/HiveInspectors.scala | 6 ++-- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 3 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 6 ++-- 8 files changed, 25 insertions(+), 27 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 14e995e618bc3..3e4c769217262 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 @@ -1649,27 +1649,17 @@ class Analyzer( } }.toSeq - // Third, for every Window Spec, we add a Window operator and set currentChild as the - // child of it. - var currentChild = child - var i = 0 - while (i < groupedWindowExpressions.size) { - val ((partitionSpec, orderSpec), windowExpressions) = groupedWindowExpressions(i) - // Set currentChild to the newly created Window operator. - currentChild = - Window( - windowExpressions, - partitionSpec, - orderSpec, - currentChild) - - // Move to next Window Spec. - i += 1 - } + // Third, we aggregate them by adding each Window operator for each Window Spec and then + // setting this to the child of the next Window operator. + val windowOps = + groupedWindowExpressions.foldLeft(child) { + case (last, ((partitionSpec, orderSpec), windowExpressions)) => + Window(windowExpressions, partitionSpec, orderSpec, last) + } - // Finally, we create a Project to output currentChild's output + // Finally, we create a Project to output windowOps's output // newExpressionsWithWindowFunctions. - Project(currentChild.output ++ newExpressionsWithWindowFunctions, currentChild) + Project(windowOps.output ++ newExpressionsWithWindowFunctions, windowOps) } // end of addWindow // We have to use transformDown at here to make sure the rule of diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 5f2585fc40b06..f9499cf78569e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -126,7 +126,8 @@ abstract class CaseWhenBase( override def eval(input: InternalRow): Any = { var i = 0 - while (i < branches.size) { + val size = branches.size + while (i < size) { if (java.lang.Boolean.TRUE.equals(branches(i)._1.eval(input))) { return branches(i)._2.eval(input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index 6112259fed619..9a892905f5186 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -31,7 +31,8 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 - while (i < ordering.size) { + val size = ordering.size + while (i < size) { val order = ordering(i) val left = order.child.eval(a) val right = order.child.eval(b) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala new file mode 100644 index 0000000000000..e69de29bb2d1d 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 f5c3536fd022e..9d56aec4a963b 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 @@ -557,7 +557,8 @@ private[hive] trait HiveInspectors { // 1. create the pojo (most likely) object val result = x.create() var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { // 2. set the property for the pojo val tpe = structType(i).dataType x.setStructFieldData( @@ -574,7 +575,8 @@ private[hive] trait HiveInspectors { val row = a.asInstanceOf[InternalRow] val result = new java.util.ArrayList[AnyRef](fieldRefs.size) var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { val tpe = structType(i).dataType result.add(wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe)) i += 1 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 e4cb33b28520f..a768b9d6d71bf 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 @@ -426,7 +426,8 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { iterator.map { value => val raw = converter.convert(rawDeser.deserialize(value)) var i = 0 - while (i < fieldRefs.length) { + val length = fieldRefs.length + while (i < length) { val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index c53675694f620..a5f800d4c5681 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -150,7 +150,8 @@ private[hive] case class HiveGenericUDF( returnInspector // Make sure initialized. var i = 0 - while (i < children.length) { + val length = children.length + while (i < length) { val idx = i deferredObjects(i).asInstanceOf[DeferredObjectAdapter] .set(() => children(idx).eval(input)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 9843f0774af96..d15fb8480666b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -194,7 +194,8 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) row: InternalRow): Unit = { val fieldRefs = oi.getAllStructFieldRefs var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { oi.setStructFieldData( struct, @@ -358,7 +359,8 @@ private[orc] object OrcRelation extends HiveInspectors { iterator.map { value => val raw = deserializer.deserialize(value) var i = 0 - while (i < fieldRefs.length) { + val length = fieldRefs.length + while (i < length) { val fieldValue = oi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) From bec077069af0b3bc22092a0552baf855dfb344ad Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 17 Sep 2016 11:46:15 -0700 Subject: [PATCH 1311/1470] [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes() ## What changes were proposed in this pull request? `MemoryStore.putIteratorAsBytes()` may silently lose values when used with `KryoSerializer` because it does not properly close the serialization stream before attempting to deserialize the already-serialized values, which may cause values buffered in Kryo's internal buffers to not be read. This is the root cause behind a user-reported "wrong answer" bug in PySpark caching reported by bennoleslie on the Spark user mailing list in a thread titled "pyspark persist MEMORY_ONLY vs MEMORY_AND_DISK". Due to Spark 2.0's automatic use of KryoSerializer for "safe" types (such as byte arrays, primitives, etc.) this misuse of serializers manifested itself as silent data corruption rather than a StreamCorrupted error (which you might get from JavaSerializer). The minimal fix, implemented here, is to close the serialization stream before attempting to deserialize written values. In addition, this patch adds several additional assertions / precondition checks to prevent misuse of `PartiallySerializedBlock` and `ChunkedByteBufferOutputStream`. ## How was this patch tested? The original bug was masked by an invalid assert in the memory store test cases: the old assert compared two results record-by-record with `zip` but didn't first check that the lengths of the two collections were equal, causing missing records to go unnoticed. The updated test case reproduced this bug. In addition, I added a new `PartiallySerializedBlockSuite` to unit test that component. Author: Josh Rosen Closes #15043 from JoshRosen/partially-serialized-block-values-iterator-bugfix. (cherry picked from commit 8faa5217b44e8d52eab7eb2d53d0652abaaf43cd) Signed-off-by: Josh Rosen --- .../org/apache/spark/scheduler/Task.scala | 1 + .../spark/storage/memory/MemoryStore.scala | 89 ++++++-- .../spark/util/ByteBufferOutputStream.scala | 27 ++- .../io/ChunkedByteBufferOutputStream.scala | 12 +- .../spark/storage/MemoryStoreSuite.scala | 34 ++- .../PartiallySerializedBlockSuite.scala | 215 ++++++++++++++++++ .../PartiallyUnrolledIteratorSuite.scala | 2 +- .../ChunkedByteBufferOutputStreamSuite.scala | 8 + 8 files changed, 344 insertions(+), 44 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 35c4dafe9c19c..1ed36bf0692f8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -230,6 +230,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task) Utils.writeByteBuffer(taskBytes, out) + out.close() out.toByteBuffer } diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 1230128d230bd..161434cb8ce37 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -33,7 +33,7 @@ import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage.{BlockId, BlockInfoManager, StorageLevel} import org.apache.spark.unsafe.Platform -import org.apache.spark.util.{CompletionIterator, SizeEstimator, Utils} +import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} @@ -275,6 +275,7 @@ private[spark] class MemoryStore( "released too much unroll memory") Left(new PartiallyUnrolledIterator( this, + MemoryMode.ON_HEAP, unrollMemoryUsedByThisBlock, unrolled = arrayValues.toIterator, rest = Iterator.empty)) @@ -283,7 +284,11 @@ private[spark] class MemoryStore( // We ran out of space while unrolling the values for this block logUnrollFailureMessage(blockId, vector.estimateSize()) Left(new PartiallyUnrolledIterator( - this, unrollMemoryUsedByThisBlock, unrolled = vector.iterator, rest = values)) + this, + MemoryMode.ON_HEAP, + unrollMemoryUsedByThisBlock, + unrolled = vector.iterator, + rest = values)) } } @@ -392,7 +397,7 @@ private[spark] class MemoryStore( redirectableStream, unrollMemoryUsedByThisBlock, memoryMode, - bbos.toChunkedByteBuffer, + bbos, values, classTag)) } @@ -653,6 +658,7 @@ private[spark] class MemoryStore( * The result of a failed [[MemoryStore.putIteratorAsValues()]] call. * * @param memoryStore the memoryStore, used for freeing memory. + * @param memoryMode the memory mode (on- or off-heap). * @param unrollMemory the amount of unroll memory used by the values in `unrolled`. * @param unrolled an iterator for the partially-unrolled values. * @param rest the rest of the original iterator passed to @@ -660,13 +666,14 @@ private[spark] class MemoryStore( */ private[storage] class PartiallyUnrolledIterator[T]( memoryStore: MemoryStore, + memoryMode: MemoryMode, unrollMemory: Long, private[this] var unrolled: Iterator[T], rest: Iterator[T]) extends Iterator[T] { private def releaseUnrollMemory(): Unit = { - memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) // SPARK-17503: Garbage collects the unrolling memory before the life end of // PartiallyUnrolledIterator. unrolled = null @@ -704,7 +711,7 @@ private[storage] class PartiallyUnrolledIterator[T]( /** * A wrapper which allows an open [[OutputStream]] to be redirected to a different sink. */ -private class RedirectableOutputStream extends OutputStream { +private[storage] class RedirectableOutputStream extends OutputStream { private[this] var os: OutputStream = _ def setOutputStream(s: OutputStream): Unit = { os = s } override def write(b: Int): Unit = os.write(b) @@ -724,7 +731,8 @@ private class RedirectableOutputStream extends OutputStream { * @param redirectableOutputStream an OutputStream which can be redirected to a different sink. * @param unrollMemory the amount of unroll memory used by the values in `unrolled`. * @param memoryMode whether the unroll memory is on- or off-heap - * @param unrolled a byte buffer containing the partially-serialized values. + * @param bbos byte buffer output stream containing the partially-serialized values. + * [[redirectableOutputStream]] initially points to this output stream. * @param rest the rest of the original iterator passed to * [[MemoryStore.putIteratorAsValues()]]. * @param classTag the [[ClassTag]] for the block. @@ -733,14 +741,19 @@ private[storage] class PartiallySerializedBlock[T]( memoryStore: MemoryStore, serializerManager: SerializerManager, blockId: BlockId, - serializationStream: SerializationStream, - redirectableOutputStream: RedirectableOutputStream, - unrollMemory: Long, + private val serializationStream: SerializationStream, + private val redirectableOutputStream: RedirectableOutputStream, + val unrollMemory: Long, memoryMode: MemoryMode, - unrolled: ChunkedByteBuffer, + bbos: ChunkedByteBufferOutputStream, rest: Iterator[T], classTag: ClassTag[T]) { + private lazy val unrolledBuffer: ChunkedByteBuffer = { + bbos.close() + bbos.toChunkedByteBuffer + } + // If the task does not fully consume `valuesIterator` or otherwise fails to consume or dispose of // this PartiallySerializedBlock then we risk leaking of direct buffers, so we use a task // completion listener here in order to ensure that `unrolled.dispose()` is called at least once. @@ -749,7 +762,23 @@ private[storage] class PartiallySerializedBlock[T]( taskContext.addTaskCompletionListener { _ => // When a task completes, its unroll memory will automatically be freed. Thus we do not call // releaseUnrollMemoryForThisTask() here because we want to avoid double-freeing. - unrolled.dispose() + unrolledBuffer.dispose() + } + } + + // Exposed for testing + private[storage] def getUnrolledChunkedByteBuffer: ChunkedByteBuffer = unrolledBuffer + + private[this] var discarded = false + private[this] var consumed = false + + private def verifyNotConsumedAndNotDiscarded(): Unit = { + if (consumed) { + throw new IllegalStateException( + "Can only call one of finishWritingToStream() or valuesIterator() and can only call once.") + } + if (discarded) { + throw new IllegalStateException("Cannot call methods on a discarded PartiallySerializedBlock") } } @@ -757,15 +786,18 @@ private[storage] class PartiallySerializedBlock[T]( * Called to dispose of this block and free its memory. */ def discard(): Unit = { - try { - // We want to close the output stream in order to free any resources associated with the - // serializer itself (such as Kryo's internal buffers). close() might cause data to be - // written, so redirect the output stream to discard that data. - redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream()) - serializationStream.close() - } finally { - unrolled.dispose() - memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) + if (!discarded) { + try { + // We want to close the output stream in order to free any resources associated with the + // serializer itself (such as Kryo's internal buffers). close() might cause data to be + // written, so redirect the output stream to discard that data. + redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream()) + serializationStream.close() + } finally { + discarded = true + unrolledBuffer.dispose() + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) + } } } @@ -774,8 +806,10 @@ private[storage] class PartiallySerializedBlock[T]( * and then serializing the values from the original input iterator. */ def finishWritingToStream(os: OutputStream): Unit = { + verifyNotConsumedAndNotDiscarded() + consumed = true // `unrolled`'s underlying buffers will be freed once this input stream is fully read: - ByteStreams.copy(unrolled.toInputStream(dispose = true), os) + ByteStreams.copy(unrolledBuffer.toInputStream(dispose = true), os) memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) redirectableOutputStream.setOutputStream(os) while (rest.hasNext) { @@ -792,13 +826,22 @@ private[storage] class PartiallySerializedBlock[T]( * `close()` on it to free its resources. */ def valuesIterator: PartiallyUnrolledIterator[T] = { + verifyNotConsumedAndNotDiscarded() + consumed = true + // Close the serialization stream so that the serializer's internal buffers are freed and any + // "end-of-stream" markers can be written out so that `unrolled` is a valid serialized stream. + serializationStream.close() // `unrolled`'s underlying buffers will be freed once this input stream is fully read: val unrolledIter = serializerManager.dataDeserializeStream( - blockId, unrolled.toInputStream(dispose = true))(classTag) + blockId, unrolledBuffer.toInputStream(dispose = true))(classTag) + // The unroll memory will be freed once `unrolledIter` is fully consumed in + // PartiallyUnrolledIterator. If the iterator is not consumed by the end of the task then any + // extra unroll memory will automatically be freed by a `finally` block in `Task`. new PartiallyUnrolledIterator( memoryStore, + memoryMode, unrollMemory, - unrolled = CompletionIterator[T, Iterator[T]](unrolledIter, discard()), + unrolled = unrolledIter, rest = rest) } } diff --git a/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala index 09e7579ae9606..9077b86f9ba1d 100644 --- a/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala @@ -29,7 +29,32 @@ private[spark] class ByteBufferOutputStream(capacity: Int) extends ByteArrayOutp def getCount(): Int = count + private[this] var closed: Boolean = false + + override def write(b: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b) + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b, off, len) + } + + override def reset(): Unit = { + require(!closed, "cannot reset a closed ByteBufferOutputStream") + super.reset() + } + + override def close(): Unit = { + if (!closed) { + super.close() + closed = true + } + } + def toByteBuffer: ByteBuffer = { - return ByteBuffer.wrap(buf, 0, count) + require(closed, "can only call toByteBuffer() after ByteBufferOutputStream has been closed") + ByteBuffer.wrap(buf, 0, count) } } diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala index 67b50d1e70437..a625b3289538a 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala @@ -49,10 +49,19 @@ private[spark] class ChunkedByteBufferOutputStream( */ private[this] var position = chunkSize private[this] var _size = 0 + private[this] var closed: Boolean = false def size: Long = _size + override def close(): Unit = { + if (!closed) { + super.close() + closed = true + } + } + override def write(b: Int): Unit = { + require(!closed, "cannot write to a closed ChunkedByteBufferOutputStream") allocateNewChunkIfNeeded() chunks(lastChunkIndex).put(b.toByte) position += 1 @@ -60,6 +69,7 @@ private[spark] class ChunkedByteBufferOutputStream( } override def write(bytes: Array[Byte], off: Int, len: Int): Unit = { + require(!closed, "cannot write to a closed ChunkedByteBufferOutputStream") var written = 0 while (written < len) { allocateNewChunkIfNeeded() @@ -73,7 +83,6 @@ private[spark] class ChunkedByteBufferOutputStream( @inline private def allocateNewChunkIfNeeded(): Unit = { - require(!toChunkedByteBufferWasCalled, "cannot write after toChunkedByteBuffer() is called") if (position == chunkSize) { chunks += allocator(chunkSize) lastChunkIndex += 1 @@ -82,6 +91,7 @@ private[spark] class ChunkedByteBufferOutputStream( } def toChunkedByteBuffer: ChunkedByteBuffer = { + require(closed, "cannot call toChunkedByteBuffer() unless close() has been called") require(!toChunkedByteBufferWasCalled, "toChunkedByteBuffer() can only be called once") toChunkedByteBufferWasCalled = true if (lastChunkIndex == -1) { diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala index 145d432afe85e..9e10ee5601480 100644 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -80,6 +80,13 @@ class MemoryStoreSuite (memoryStore, blockInfoManager) } + private def assertSameContents[T](expected: Seq[T], actual: Seq[T], hint: String): Unit = { + assert(actual.length === expected.length, s"wrong number of values returned in $hint") + expected.iterator.zip(actual.iterator).foreach { case (e, a) => + assert(e === a, s"$hint did not return original values!") + } + } + test("reserve/release unroll memory") { val (memoryStore, _) = makeMemoryStore(12000) assert(memoryStore.currentUnrollMemory === 0) @@ -138,9 +145,7 @@ class MemoryStoreSuite var putResult = putIteratorAsValues("unroll", smallList.iterator, ClassTag.Any) assert(putResult.isRight) assert(memoryStore.currentUnrollMemoryForThisTask === 0) - smallList.iterator.zip(memoryStore.getValues("unroll").get).foreach { case (e, a) => - assert(e === a, "getValues() did not return original values!") - } + assertSameContents(smallList, memoryStore.getValues("unroll").get.toSeq, "getValues") blockInfoManager.lockForWriting("unroll") assert(memoryStore.remove("unroll")) blockInfoManager.removeBlock("unroll") @@ -153,9 +158,7 @@ class MemoryStoreSuite assert(memoryStore.currentUnrollMemoryForThisTask === 0) assert(memoryStore.contains("someBlock2")) assert(!memoryStore.contains("someBlock1")) - smallList.iterator.zip(memoryStore.getValues("unroll").get).foreach { case (e, a) => - assert(e === a, "getValues() did not return original values!") - } + assertSameContents(smallList, memoryStore.getValues("unroll").get.toSeq, "getValues") blockInfoManager.lockForWriting("unroll") assert(memoryStore.remove("unroll")) blockInfoManager.removeBlock("unroll") @@ -168,9 +171,7 @@ class MemoryStoreSuite assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator assert(!memoryStore.contains("someBlock2")) assert(putResult.isLeft) - bigList.iterator.zip(putResult.left.get).foreach { case (e, a) => - assert(e === a, "putIterator() did not return original values!") - } + assertSameContents(bigList, putResult.left.get.toSeq, "putIterator") // The unroll memory was freed once the iterator returned by putIterator() was fully traversed. assert(memoryStore.currentUnrollMemoryForThisTask === 0) } @@ -317,9 +318,8 @@ class MemoryStoreSuite assert(res.isLeft) assert(memoryStore.currentUnrollMemoryForThisTask > 0) val valuesReturnedFromFailedPut = res.left.get.valuesIterator.toSeq // force materialization - valuesReturnedFromFailedPut.zip(bigList).foreach { case (e, a) => - assert(e === a, "PartiallySerializedBlock.valuesIterator() did not return original values!") - } + assertSameContents( + bigList, valuesReturnedFromFailedPut, "PartiallySerializedBlock.valuesIterator()") // The unroll memory was freed once the iterator was fully traversed. assert(memoryStore.currentUnrollMemoryForThisTask === 0) } @@ -341,12 +341,10 @@ class MemoryStoreSuite res.left.get.finishWritingToStream(bos) // The unroll memory was freed once the block was fully written. assert(memoryStore.currentUnrollMemoryForThisTask === 0) - val deserializationStream = serializerManager.dataDeserializeStream[Any]( - "b1", new ByteBufferInputStream(bos.toByteBuffer))(ClassTag.Any) - deserializationStream.zip(bigList.iterator).foreach { case (e, a) => - assert(e === a, - "PartiallySerializedBlock.finishWritingtoStream() did not write original values!") - } + val deserializedValues = serializerManager.dataDeserializeStream[Any]( + "b1", new ByteBufferInputStream(bos.toByteBuffer))(ClassTag.Any).toSeq + assertSameContents( + bigList, deserializedValues, "PartiallySerializedBlock.finishWritingToStream()") } test("multiple unrolls by the same thread") { diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala new file mode 100644 index 0000000000000..ec4f2637fadd0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala @@ -0,0 +1,215 @@ +/* + * 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.storage + +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import org.mockito.Mockito +import org.mockito.Mockito.atLeastOnce +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} + +import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark.memory.MemoryMode +import org.apache.spark.serializer.{JavaSerializer, SerializationStream, SerializerManager} +import org.apache.spark.storage.memory.{MemoryStore, PartiallySerializedBlock, RedirectableOutputStream} +import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream} +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} + +class PartiallySerializedBlockSuite + extends SparkFunSuite + with BeforeAndAfterEach + with PrivateMethodTester { + + private val blockId = new TestBlockId("test") + private val conf = new SparkConf() + private val memoryStore = Mockito.mock(classOf[MemoryStore], Mockito.RETURNS_SMART_NULLS) + private val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) + + private val getSerializationStream = PrivateMethod[SerializationStream]('serializationStream) + private val getRedirectableOutputStream = + PrivateMethod[RedirectableOutputStream]('redirectableOutputStream) + + override protected def beforeEach(): Unit = { + super.beforeEach() + Mockito.reset(memoryStore) + } + + private def partiallyUnroll[T: ClassTag]( + iter: Iterator[T], + numItemsToBuffer: Int): PartiallySerializedBlock[T] = { + + val bbos: ChunkedByteBufferOutputStream = { + val spy = Mockito.spy(new ChunkedByteBufferOutputStream(128, ByteBuffer.allocate)) + Mockito.doAnswer(new Answer[ChunkedByteBuffer] { + override def answer(invocationOnMock: InvocationOnMock): ChunkedByteBuffer = { + Mockito.spy(invocationOnMock.callRealMethod().asInstanceOf[ChunkedByteBuffer]) + } + }).when(spy).toChunkedByteBuffer + spy + } + + val serializer = serializerManager.getSerializer(implicitly[ClassTag[T]]).newInstance() + val redirectableOutputStream = Mockito.spy(new RedirectableOutputStream) + redirectableOutputStream.setOutputStream(bbos) + val serializationStream = Mockito.spy(serializer.serializeStream(redirectableOutputStream)) + + (1 to numItemsToBuffer).foreach { _ => + assert(iter.hasNext) + serializationStream.writeObject[T](iter.next()) + } + + val unrollMemory = bbos.size + new PartiallySerializedBlock[T]( + memoryStore, + serializerManager, + blockId, + serializationStream = serializationStream, + redirectableOutputStream, + unrollMemory = unrollMemory, + memoryMode = MemoryMode.ON_HEAP, + bbos, + rest = iter, + classTag = implicitly[ClassTag[T]]) + } + + test("valuesIterator() and finishWritingToStream() cannot be called after discard() is called") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.discard() + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(null) + } + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("discard() can be called more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.discard() + partiallySerializedBlock.discard() + } + + test("cannot call valuesIterator() more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.valuesIterator + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("cannot call finishWritingToStream() more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + } + } + + test("cannot call finishWritingToStream() after valuesIterator()") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.valuesIterator + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + } + } + + test("cannot call valuesIterator() after finishWritingToStream()") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("buffers are deallocated in a TaskCompletionListener") { + try { + TaskContext.setTaskContext(TaskContext.empty()) + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer).dispose() + Mockito.verifyNoMoreInteractions(memoryStore) + } finally { + TaskContext.unset() + } + } + + private def testUnroll[T: ClassTag]( + testCaseName: String, + items: Seq[T], + numItemsToBuffer: Int): Unit = { + + test(s"$testCaseName with discard() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + partiallySerializedBlock.discard() + + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + } + + test(s"$testCaseName with finishWritingToStream() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + val bbos = Mockito.spy(new ByteBufferOutputStream()) + partiallySerializedBlock.finishWritingToStream(bbos) + + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + Mockito.verify(bbos).close() + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + + val serializer = serializerManager.getSerializer(implicitly[ClassTag[T]]).newInstance() + val deserialized = + serializer.deserializeStream(new ByteBufferInputStream(bbos.toByteBuffer)).asIterator.toSeq + assert(deserialized === items) + } + + test(s"$testCaseName with valuesIterator() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + val valuesIterator = partiallySerializedBlock.valuesIterator + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + + val deserializedItems = valuesIterator.toArray.toSeq + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + assert(deserializedItems === items) + } + } + + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 50) + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 0) + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 1000) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 50) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 0) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 1000) + testUnroll("empty iterator", Seq.empty[String], numItemsToBuffer = 0) +} + +private case class MyCaseClass(str: String) diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala index 02c2331dc3946..4253cc8ca4cd1 100644 --- a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala @@ -33,7 +33,7 @@ class PartiallyUnrolledIteratorSuite extends SparkFunSuite with MockitoSugar { val rest = (unrollSize until restSize + unrollSize).iterator val memoryStore = mock[MemoryStore] - val joinIterator = new PartiallyUnrolledIterator(memoryStore, unrollSize, unroll, rest) + val joinIterator = new PartiallyUnrolledIterator(memoryStore, ON_HEAP, unrollSize, unroll, rest) // Firstly iterate over unrolling memory iterator (0 until unrollSize).foreach { value => diff --git a/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala index 226622075a6cc..86961745673c6 100644 --- a/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala @@ -28,12 +28,14 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { test("empty output") { val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate) + o.close() assert(o.toChunkedByteBuffer.size === 0) } test("write a single byte") { val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate) o.write(10) + o.close() val chunkedByteBuffer = o.toChunkedByteBuffer assert(chunkedByteBuffer.getChunks().length === 1) assert(chunkedByteBuffer.getChunks().head.array().toSeq === Seq(10.toByte)) @@ -43,6 +45,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(new Array[Byte](9)) o.write(99) + o.close() val chunkedByteBuffer = o.toChunkedByteBuffer assert(chunkedByteBuffer.getChunks().length === 1) assert(chunkedByteBuffer.getChunks().head.array()(9) === 99.toByte) @@ -52,6 +55,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(new Array[Byte](10)) o.write(99) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 2) assert(arrays(1).length === 1) @@ -63,6 +67,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 1) assert(arrays.head.length === ref.length) @@ -74,6 +79,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 1) assert(arrays.head.length === ref.length) @@ -85,6 +91,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 3) assert(arrays(0).length === 10) @@ -101,6 +108,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 3) assert(arrays(0).length === 10) From 0cfc0469b40450aee5d909641b4296b3a978b2d6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 17 Sep 2016 14:18:40 -0700 Subject: [PATCH 1312/1470] Revert "[SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.length/size which is O(n)" This reverts commit a3bba372abce926351335d0a2936b70988f19b23. --- .../sql/catalyst/analysis/Analyzer.scala | 28 +++++++++++++------ .../expressions/conditionalExpressions.scala | 3 +- .../sql/catalyst/expressions/ordering.scala | 3 +- .../sql/catalyst/util/QuantileSummaries.scala | 0 .../spark/sql/hive/HiveInspectors.scala | 6 ++-- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 3 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 6 ++-- 8 files changed, 27 insertions(+), 25 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3e4c769217262..14e995e618bc3 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 @@ -1649,17 +1649,27 @@ class Analyzer( } }.toSeq - // Third, we aggregate them by adding each Window operator for each Window Spec and then - // setting this to the child of the next Window operator. - val windowOps = - groupedWindowExpressions.foldLeft(child) { - case (last, ((partitionSpec, orderSpec), windowExpressions)) => - Window(windowExpressions, partitionSpec, orderSpec, last) - } + // Third, for every Window Spec, we add a Window operator and set currentChild as the + // child of it. + var currentChild = child + var i = 0 + while (i < groupedWindowExpressions.size) { + val ((partitionSpec, orderSpec), windowExpressions) = groupedWindowExpressions(i) + // Set currentChild to the newly created Window operator. + currentChild = + Window( + windowExpressions, + partitionSpec, + orderSpec, + currentChild) + + // Move to next Window Spec. + i += 1 + } - // Finally, we create a Project to output windowOps's output + // Finally, we create a Project to output currentChild's output // newExpressionsWithWindowFunctions. - Project(windowOps.output ++ newExpressionsWithWindowFunctions, windowOps) + Project(currentChild.output ++ newExpressionsWithWindowFunctions, currentChild) } // end of addWindow // We have to use transformDown at here to make sure the rule of diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index f9499cf78569e..5f2585fc40b06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -126,8 +126,7 @@ abstract class CaseWhenBase( override def eval(input: InternalRow): Any = { var i = 0 - val size = branches.size - while (i < size) { + while (i < branches.size) { if (java.lang.Boolean.TRUE.equals(branches(i)._1.eval(input))) { return branches(i)._2.eval(input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index 9a892905f5186..6112259fed619 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -31,8 +31,7 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 - val size = ordering.size - while (i < size) { + while (i < ordering.size) { val order = ordering(i) val left = order.child.eval(a) val right = order.child.eval(b) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 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 9d56aec4a963b..f5c3536fd022e 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 @@ -557,8 +557,7 @@ private[hive] trait HiveInspectors { // 1. create the pojo (most likely) object val result = x.create() var i = 0 - val size = fieldRefs.size - while (i < size) { + while (i < fieldRefs.size) { // 2. set the property for the pojo val tpe = structType(i).dataType x.setStructFieldData( @@ -575,8 +574,7 @@ private[hive] trait HiveInspectors { val row = a.asInstanceOf[InternalRow] val result = new java.util.ArrayList[AnyRef](fieldRefs.size) var i = 0 - val size = fieldRefs.size - while (i < size) { + while (i < fieldRefs.size) { val tpe = structType(i).dataType result.add(wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe)) i += 1 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 a768b9d6d71bf..e4cb33b28520f 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 @@ -426,8 +426,7 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { iterator.map { value => val raw = converter.convert(rawDeser.deserialize(value)) var i = 0 - val length = fieldRefs.length - while (i < length) { + while (i < fieldRefs.length) { val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index a5f800d4c5681..c53675694f620 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -150,8 +150,7 @@ private[hive] case class HiveGenericUDF( returnInspector // Make sure initialized. var i = 0 - val length = children.length - while (i < length) { + while (i < children.length) { val idx = i deferredObjects(i).asInstanceOf[DeferredObjectAdapter] .set(() => children(idx).eval(input)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index d15fb8480666b..9843f0774af96 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -194,8 +194,7 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) row: InternalRow): Unit = { val fieldRefs = oi.getAllStructFieldRefs var i = 0 - val size = fieldRefs.size - while (i < size) { + while (i < fieldRefs.size) { oi.setStructFieldData( struct, @@ -359,8 +358,7 @@ private[orc] object OrcRelation extends HiveInspectors { iterator.map { value => val raw = deserializer.deserialize(value) var i = 0 - val length = fieldRefs.length - while (i < length) { + while (i < fieldRefs.length) { val fieldValue = oi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) From 5fd354b2d628130a74c9d01adc7ab6bef65fbd9a Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sat, 17 Sep 2016 16:52:30 +0100 Subject: [PATCH 1313/1470] [SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.length/size which is O(n) This PR fixes all the instances which was fixed in the previous PR. To make sure, I manually debugged and also checked the Scala source. `length` in [LinearSeqOptimized.scala#L49-L57](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/LinearSeqOptimized.scala#L49-L57) is O(n). Also, `size` calls `length` via [SeqLike.scala#L106](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/SeqLike.scala#L106). For debugging, I have created these as below: ```scala ArrayBuffer(1, 2, 3) Array(1, 2, 3) List(1, 2, 3) Seq(1, 2, 3) ``` and then called `size` and `length` for each to debug. I ran the bash as below on Mac ```bash find . -name *.scala -type f -exec grep -il "while (.*\\.length)" {} \; | grep "src/main" find . -name *.scala -type f -exec grep -il "while (.*\\.size)" {} \; | grep "src/main" ``` and then checked each. Author: hyukjinkwon Closes #15093 from HyukjinKwon/SPARK-17480-followup. (cherry picked from commit 86c2d393a56bf1e5114bc5a781253c0460efb8af) Signed-off-by: Sean Owen --- .../sql/catalyst/analysis/Analyzer.scala | 28 ++++++------------- .../expressions/conditionalExpressions.scala | 3 +- .../sql/catalyst/expressions/ordering.scala | 3 +- .../spark/sql/hive/HiveInspectors.scala | 6 ++-- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 3 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 6 ++-- 7 files changed, 25 insertions(+), 27 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 14e995e618bc3..3e4c769217262 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 @@ -1649,27 +1649,17 @@ class Analyzer( } }.toSeq - // Third, for every Window Spec, we add a Window operator and set currentChild as the - // child of it. - var currentChild = child - var i = 0 - while (i < groupedWindowExpressions.size) { - val ((partitionSpec, orderSpec), windowExpressions) = groupedWindowExpressions(i) - // Set currentChild to the newly created Window operator. - currentChild = - Window( - windowExpressions, - partitionSpec, - orderSpec, - currentChild) - - // Move to next Window Spec. - i += 1 - } + // Third, we aggregate them by adding each Window operator for each Window Spec and then + // setting this to the child of the next Window operator. + val windowOps = + groupedWindowExpressions.foldLeft(child) { + case (last, ((partitionSpec, orderSpec), windowExpressions)) => + Window(windowExpressions, partitionSpec, orderSpec, last) + } - // Finally, we create a Project to output currentChild's output + // Finally, we create a Project to output windowOps's output // newExpressionsWithWindowFunctions. - Project(currentChild.output ++ newExpressionsWithWindowFunctions, currentChild) + Project(windowOps.output ++ newExpressionsWithWindowFunctions, windowOps) } // end of addWindow // We have to use transformDown at here to make sure the rule of diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 5f2585fc40b06..f9499cf78569e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -126,7 +126,8 @@ abstract class CaseWhenBase( override def eval(input: InternalRow): Any = { var i = 0 - while (i < branches.size) { + val size = branches.size + while (i < size) { if (java.lang.Boolean.TRUE.equals(branches(i)._1.eval(input))) { return branches(i)._2.eval(input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index 6112259fed619..9a892905f5186 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -31,7 +31,8 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 - while (i < ordering.size) { + val size = ordering.size + while (i < size) { val order = ordering(i) val left = order.child.eval(a) val right = order.child.eval(b) 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 f5c3536fd022e..9d56aec4a963b 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 @@ -557,7 +557,8 @@ private[hive] trait HiveInspectors { // 1. create the pojo (most likely) object val result = x.create() var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { // 2. set the property for the pojo val tpe = structType(i).dataType x.setStructFieldData( @@ -574,7 +575,8 @@ private[hive] trait HiveInspectors { val row = a.asInstanceOf[InternalRow] val result = new java.util.ArrayList[AnyRef](fieldRefs.size) var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { val tpe = structType(i).dataType result.add(wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe)) i += 1 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 e4cb33b28520f..a768b9d6d71bf 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 @@ -426,7 +426,8 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { iterator.map { value => val raw = converter.convert(rawDeser.deserialize(value)) var i = 0 - while (i < fieldRefs.length) { + val length = fieldRefs.length + while (i < length) { val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index c53675694f620..a5f800d4c5681 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -150,7 +150,8 @@ private[hive] case class HiveGenericUDF( returnInspector // Make sure initialized. var i = 0 - while (i < children.length) { + val length = children.length + while (i < length) { val idx = i deferredObjects(i).asInstanceOf[DeferredObjectAdapter] .set(() => children(idx).eval(input)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 9843f0774af96..d15fb8480666b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -194,7 +194,8 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) row: InternalRow): Unit = { val fieldRefs = oi.getAllStructFieldRefs var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { oi.setStructFieldData( struct, @@ -358,7 +359,8 @@ private[orc] object OrcRelation extends HiveInspectors { iterator.map { value => val raw = deserializer.deserialize(value) var i = 0 - while (i < fieldRefs.length) { + val length = fieldRefs.length + while (i < length) { val fieldValue = oi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) From cf728b0f2dc7c1e9f62a8984122d3bf91e6ba439 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 18 Sep 2016 21:15:35 +0800 Subject: [PATCH 1314/1470] [SPARK-17541][SQL] fix some DDL bugs about table management when same-name temp view exists In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are: 1. `CREATE TABLE USING` will fail if a same-name temp view exists 2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists 3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists. These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch new regression tests Author: Wenchen Fan Closes #15099 from cloud-fan/fix-view. (cherry picked from commit 3fe630d314cf50d69868b7707ac8d8d2027080b8) Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/SessionCatalog.scala | 32 +++++--- .../catalog/SessionCatalogSuite.scala | 24 +++--- .../apache/spark/sql/DataFrameWriter.scala | 9 ++- .../command/createDataSourceTables.scala | 30 +++++--- .../spark/sql/internal/CatalogImpl.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 11 +++ .../spark/sql/internal/CatalogSuite.scala | 11 +++ .../sql/test/DataFrameReaderWriterSuite.scala | 76 +++++++++++++++++++ .../sql/hive/MetastoreDataSourcesSuite.scala | 13 ++-- .../sql/sources/HadoopFsRelationTest.scala | 10 +-- 10 files changed, 172 insertions(+), 50 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 24485134afc07..ecb4dab45ce00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -332,9 +332,9 @@ class SessionCatalog( new Path(new Path(dbLocation), formatTableName(tableIdent.table)).toString } - // ------------------------------------------------------------- - // | Methods that interact with temporary and metastore tables | - // ------------------------------------------------------------- + // ---------------------------------------------- + // | Methods that interact with temp views only | + // ---------------------------------------------- /** * Create a temporary table. @@ -350,6 +350,24 @@ class SessionCatalog( tempTables.put(table, tableDefinition) } + /** + * Return a temporary view exactly as it was stored. + */ + def getTempView(name: String): Option[LogicalPlan] = synchronized { + tempTables.get(formatTableName(name)) + } + + /** + * Drop a temporary view. + */ + def dropTempView(name: String): Unit = synchronized { + tempTables.remove(formatTableName(name)) + } + + // ------------------------------------------------------------- + // | Methods that interact with temporary and metastore tables | + // ------------------------------------------------------------- + /** * Rename a table. * @@ -506,14 +524,6 @@ class SessionCatalog( tempTables.clear() } - /** - * Return a temporary table exactly as it was stored. - * For testing only. - */ - private[catalog] def getTempTable(name: String): Option[LogicalPlan] = synchronized { - tempTables.get(formatTableName(name)) - } - // ---------------------------------------------------------------------------- // Partitions // ---------------------------------------------------------------------------- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 67ca0aadcc138..574ed05333d01 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -201,16 +201,16 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable2 = Range(1, 20, 2, 10) catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) catalog.createTempView("tbl2", tempTable2, overrideIfExists = false) - assert(catalog.getTempTable("tbl1") == Option(tempTable1)) - assert(catalog.getTempTable("tbl2") == Option(tempTable2)) - assert(catalog.getTempTable("tbl3").isEmpty) + assert(catalog.getTempView("tbl1") == Option(tempTable1)) + assert(catalog.getTempView("tbl2") == Option(tempTable2)) + assert(catalog.getTempView("tbl3").isEmpty) // Temporary table already exists intercept[TempTableAlreadyExistsException] { catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) } // Temporary table already exists but we override it catalog.createTempView("tbl1", tempTable2, overrideIfExists = true) - assert(catalog.getTempTable("tbl1") == Option(tempTable2)) + assert(catalog.getTempView("tbl1") == Option(tempTable2)) } test("drop table") { @@ -246,11 +246,11 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") - assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) + assert(sessionCatalog.getTempView("tbl1") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be dropped first sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false) - assert(sessionCatalog.getTempTable("tbl1") == None) + assert(sessionCatalog.getTempView("tbl1") == None) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If temp table does not exist, the table in the current database should be dropped sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false) @@ -259,7 +259,7 @@ class SessionCatalogSuite extends SparkFunSuite { sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false) sessionCatalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false) - assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) + assert(sessionCatalog.getTempView("tbl1") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) } @@ -307,18 +307,18 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") - assert(sessionCatalog.getTempTable("tbl1") == Option(tempTable)) + assert(sessionCatalog.getTempView("tbl1") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be renamed first sessionCatalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3")) - assert(sessionCatalog.getTempTable("tbl1").isEmpty) - assert(sessionCatalog.getTempTable("tbl3") == Option(tempTable)) + assert(sessionCatalog.getTempView("tbl1").isEmpty) + assert(sessionCatalog.getTempView("tbl3") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is specified, temp tables are never renamed sessionCatalog.renameTable( TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4", Some("db2"))) - assert(sessionCatalog.getTempTable("tbl3") == Option(tempTable)) - assert(sessionCatalog.getTempTable("tbl4").isEmpty) + assert(sessionCatalog.getTempView("tbl3") == Option(tempTable)) + assert(sessionCatalog.getTempView("tbl4").isEmpty) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index a4c4a5defa1b3..b448b9e38b0d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -356,7 +356,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def saveAsTable(tableIdent: TableIdentifier): Unit = { - val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) + val sessionState = df.sparkSession.sessionState + val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentWithDB = tableIdent.copy(database = Some(db)) + // Pass a table identifier with database part, so that `tableExists` won't check temp views + // unexpectedly. + val tableExists = sessionState.catalog.tableExists(tableIdentWithDB) (tableExists, mode) match { case (true, SaveMode.Ignore) => @@ -375,7 +380,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { mode, extraOptions.toMap, df.logicalPlan) - df.sparkSession.sessionState.executePlan(cmd).toRdd + sessionState.executePlan(cmd).toRdd } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 900446c6bd1d5..06965ff7650ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.HiveSerDe @@ -74,14 +73,16 @@ case class CreateDataSourceTableCommand( s"characters, numbers and _.") } - val tableName = tableIdent.unquotedString val sessionState = sparkSession.sessionState - - if (sessionState.catalog.tableExists(tableIdent)) { + val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentWithDB = tableIdent.copy(database = Some(db)) + // Pass a table identifier with database part, so that `tableExists` won't check temp views + // unexpectedly. + if (sessionState.catalog.tableExists(tableIdentWithDB)) { if (ignoreIfExists) { return Seq.empty[Row] } else { - throw new AnalysisException(s"Table $tableName already exists.") + throw new AnalysisException(s"Table ${tableIdentWithDB.quotedString} already exists.") } } @@ -157,8 +158,11 @@ case class CreateDataSourceTableAsSelectCommand( s"characters, numbers and _.") } - val tableName = tableIdent.unquotedString val sessionState = sparkSession.sessionState + val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentWithDB = tableIdent.copy(database = Some(db)) + val tableName = tableIdentWithDB.unquotedString + var createMetastoreTable = false var isExternal = true val optionsWithPath = @@ -170,7 +174,9 @@ case class CreateDataSourceTableAsSelectCommand( } var existingSchema = Option.empty[StructType] - if (sparkSession.sessionState.catalog.tableExists(tableIdent)) { + // Pass a table identifier with database part, so that `tableExists` won't check temp views + // unexpectedly. + if (sparkSession.sessionState.catalog.tableExists(tableIdentWithDB)) { // Check if we need to throw an exception or just return. mode match { case SaveMode.ErrorIfExists => @@ -195,13 +201,13 @@ case class CreateDataSourceTableAsSelectCommand( // inserting into (i.e. using the same compression). EliminateSubqueryAliases( - sessionState.catalog.lookupRelation(tableIdent)) match { + sessionState.catalog.lookupRelation(tableIdentWithDB)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => // check if the file formats match l.relation match { case r: HadoopFsRelation if r.fileFormat.getClass != dataSource.providingClass => throw new AnalysisException( - s"The file format of the existing table $tableIdent is " + + s"The file format of the existing table $tableName is " + s"`${r.fileFormat.getClass.getName}`. It doesn't match the specified " + s"format `$provider`") case _ => @@ -218,7 +224,7 @@ case class CreateDataSourceTableAsSelectCommand( throw new AnalysisException(s"Saving data in ${o.toString} is not supported.") } case SaveMode.Overwrite => - sparkSession.sql(s"DROP TABLE IF EXISTS $tableName") + sessionState.catalog.dropTable(tableIdentWithDB, ignoreIfNotExists = true) // Need to create the table again. createMetastoreTable = true } @@ -246,7 +252,7 @@ case class CreateDataSourceTableAsSelectCommand( dataSource.write(mode, df) } catch { case ex: AnalysisException => - logError(s"Failed to write to table ${tableIdent.identifier} in $mode mode", ex) + logError(s"Failed to write to table $tableName in $mode mode", ex) throw ex } if (createMetastoreTable) { @@ -265,7 +271,7 @@ case class CreateDataSourceTableAsSelectCommand( } // Refresh the cache of the table in the catalog. - sessionState.catalog.refreshTable(tableIdent) + sessionState.catalog.refreshTable(tableIdentWithDB) Seq.empty[Row] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index a6ae6fe2aad2e..2067e7e39dd4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -296,8 +296,10 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def dropTempView(viewName: String): Unit = { - sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession.table(viewName)) - sessionCatalog.dropTable(TableIdentifier(viewName), ignoreIfNotExists = true) + sparkSession.sessionState.catalog.getTempView(viewName).foreach { tempView => + sparkSession.sharedState.cacheManager.uncacheQuery(Dataset.ofRows(sparkSession, tempView)) + sessionCatalog.dropTempView(viewName) + } } /** 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 f20b9adcb61a4..cf250970c6c32 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 @@ -2621,4 +2621,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { }.limit(1).queryExecution.toRdd.count() assert(numRecordsRead.value === 10) } + + test("CREATE TABLE USING should not fail if a same-name temp view exists") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + sql("CREATE TABLE same_name(i int) USING json") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + assert(spark.table("default.same_name").collect().isEmpty) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index d75df56dd608a..e62ae38cd35a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -304,6 +304,17 @@ class CatalogSuite columnFields.foreach { f => assert(columnString.contains(f.toString)) } } + test("dropTempView should not un-cache and drop metastore table if a same-name table exists") { + withTable("same_name") { + spark.range(10).write.saveAsTable("same_name") + sql("CACHE TABLE same_name") + assert(spark.catalog.isCached("default.same_name")) + spark.catalog.dropTempView("same_name") + assert(spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + assert(spark.catalog.isCached("default.same_name")) + } + } + // TODO: add tests for the rest of them } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index e071aef4c2db7..bba265e9c9340 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.scalatest.BeforeAndAfter import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.Utils @@ -439,4 +440,79 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be checkAnswer(df, spark.createDataset(expectedResult).toDF()) assert(df.schema === expectedSchema) } + + test("saveAsTable with mode Append should not fail if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Append).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Append should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + sql("CREATE TABLE same_name(id LONG) USING parquet") + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Append).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable with mode ErrorIfExists should not fail if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.ErrorIfExists).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Overwrite should not drop the temp view if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Overwrite).saveAsTable("same_name") + assert(spark.sessionState.catalog.getTempView("same_name").isDefined) + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Overwrite should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + sql("CREATE TABLE same_name(id LONG) USING parquet") + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Overwrite).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable with mode Ignore should create the table if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Ignore).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index a0b3b37600924..b758ab00f34c5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -333,7 +333,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv }.getMessage assert( - message.contains("Table ctasJsonTable already exists."), + message.contains("Table default.ctasJsonTable already exists."), "We should complain that ctasJsonTable already exists") // The following statement should be fine if it has IF NOT EXISTS. @@ -509,7 +509,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert( intercept[AnalysisException] { sparkSession.catalog.createExternalTable("createdJsonTable", jsonFilePath.toString) - }.getMessage.contains("Table createdJsonTable already exists."), + }.getMessage.contains("Table default.createdJsonTable already exists."), "We should complain that createdJsonTable already exists") } @@ -901,7 +901,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val e = intercept[AnalysisException] { createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendOrcToParquet") } - assert(e.getMessage.contains("The file format of the existing table `appendOrcToParquet` " + + assert(e.getMessage.contains( + "The file format of the existing table default.appendOrcToParquet " + "is `org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat`. " + "It doesn't match the specified format `orc`")) } @@ -912,7 +913,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv createDF(10, 19).write.mode(SaveMode.Append).format("parquet") .saveAsTable("appendParquetToJson") } - assert(e.getMessage.contains("The file format of the existing table `appendParquetToJson` " + + assert(e.getMessage.contains( + "The file format of the existing table default.appendParquetToJson " + "is `org.apache.spark.sql.execution.datasources.json.JsonFileFormat`. " + "It doesn't match the specified format `parquet`")) } @@ -923,7 +925,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv createDF(10, 19).write.mode(SaveMode.Append).format("text") .saveAsTable("appendTextToJson") } - assert(e.getMessage.contains("The file format of the existing table `appendTextToJson` is " + + assert(e.getMessage.contains( + "The file format of the existing table default.appendTextToJson is " + "`org.apache.spark.sql.execution.datasources.json.JsonFileFormat`. " + "It doesn't match the specified format `text`")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 047b08c4ccf65..97f2b23823d7a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -337,9 +337,8 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - - withTempView("t") { + withTable("t") { + sql("CREATE TABLE t(i INT) USING parquet") intercept[AnalysisException] { testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).saveAsTable("t") } @@ -347,9 +346,8 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - - withTempView("t") { + withTable("t") { + sql("CREATE TABLE t(i INT) USING parquet") testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t") assert(spark.table("t").collect().isEmpty) } From 5619f095bfac76009758b4f4a4f8c9e319eeb5b1 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 18 Sep 2016 16:22:31 +0100 Subject: [PATCH 1315/1470] [SPARK-17546][DEPLOY] start-* scripts should use hostname -f ## What changes were proposed in this pull request? Call `hostname -f` to get fully qualified host name ## How was this patch tested? Jenkins tests of course, but also verified output of command on OS X and Linux Author: Sean Owen Closes #15129 from srowen/SPARK-17546. (cherry picked from commit 342c0e65bec4b9a715017089ab6ea127f3c46540) Signed-off-by: Sean Owen --- sbin/start-master.sh | 2 +- sbin/start-mesos-dispatcher.sh | 2 +- sbin/start-slaves.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sbin/start-master.sh b/sbin/start-master.sh index 981cb15bc0006..d970fcc45e2c1 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -48,7 +48,7 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST=`hostname` + SPARK_MASTER_HOST=`hostname -f` fi if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh index 06a966d1c20b4..ef65fb9539146 100755 --- a/sbin/start-mesos-dispatcher.sh +++ b/sbin/start-mesos-dispatcher.sh @@ -34,7 +34,7 @@ if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then fi if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then - SPARK_MESOS_DISPATCHER_HOST=`hostname` + SPARK_MESOS_DISPATCHER_HOST=`hostname -f` fi if [ "$SPARK_MESOS_DISPATCHER_NUM" = "" ]; then diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 0fa1605489704..7d8871251f81b 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -32,7 +32,7 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST="`hostname`" + SPARK_MASTER_HOST="`hostname -f`" fi # Launch the slaves From 6c67d86f2f0a24764146827ec5c42969194cb11d Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 18 Sep 2016 19:18:49 +0100 Subject: [PATCH 1316/1470] [SPARK-17586][BUILD] Do not call static member via instance reference ## What changes were proposed in this pull request? This PR fixes a warning message as below: ``` [WARNING] .../UnsafeInMemorySorter.java:284: warning: [static] static method should be qualified by type name, TaskMemoryManager, instead of by an expression [WARNING] currentPageNumber = memoryManager.decodePageNumber(recordPointer) ``` by referencing the static member via class not instance reference. ## How was this patch tested? Existing tests should cover this - Jenkins tests. Author: hyukjinkwon Closes #15141 from HyukjinKwon/SPARK-17586. (cherry picked from commit 7151011b38a841d9d4bc2e453b9a7cfe42f74f8f) Signed-off-by: Sean Owen --- .../spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 9710529f2d052..b51737158098d 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -258,7 +258,7 @@ public boolean hasNext() { public void loadNext() { // This pointer points to a 4-byte record length, followed by the record's bytes final long recordPointer = array.get(offset + position); - currentPageNumber = memoryManager.decodePageNumber(recordPointer); + currentPageNumber = TaskMemoryManager.decodePageNumber(recordPointer); baseObject = memoryManager.getPage(recordPointer); baseOffset = memoryManager.getOffsetInPage(recordPointer) + 4; // Skip over record length recordLength = Platform.getInt(baseObject, baseOffset - 4); From 151f808a181333daa6300c7d5d7c49c3cec3307c Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Sun, 18 Sep 2016 19:25:58 +0100 Subject: [PATCH 1317/1470] [SPARK-16462][SPARK-16460][SPARK-15144][SQL] Make CSV cast null values properly ## Problem CSV in Spark 2.0.0: - does not read null values back correctly for certain data types such as `Boolean`, `TimestampType`, `DateType` -- this is a regression comparing to 1.6; - does not read empty values (specified by `options.nullValue`) as `null`s for `StringType` -- this is compatible with 1.6 but leads to problems like SPARK-16903. ## What changes were proposed in this pull request? This patch makes changes to read all empty values back as `null`s. ## How was this patch tested? New test cases. Author: Liwei Lin Closes #14118 from lw-lin/csv-cast-null. (cherry picked from commit 1dbb725dbef30bf7633584ce8efdb573f2d92bca) Signed-off-by: Sean Owen --- python/pyspark/sql/readwriter.py | 3 +- python/pyspark/sql/streaming.py | 3 +- .../apache/spark/sql/DataFrameReader.scala | 3 +- .../datasources/csv/CSVInferSchema.scala | 108 ++++++++---------- .../sql/streaming/DataStreamReader.scala | 3 +- .../execution/datasources/csv/CSVSuite.scala | 2 +- .../datasources/csv/CSVTypeCastSuite.scala | 54 +++++---- 7 files changed, 93 insertions(+), 83 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 3da6f497e9f12..dc13a818fcbfb 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -327,7 +327,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non being read should be skipped. If None is set, it uses the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses - the default value, empty string. + the default value, empty string. Since 2.0.1, this ``nullValue`` param + applies to all supported types including the string type. :param nanValue: sets the string representation of a non-number value. If None is set, it uses the default value, ``NaN``. :param positiveInf: sets the string representation of a positive infinity value. If None diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 9487f9dbfdabc..38c19e2d1e9a1 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -495,7 +495,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non being read should be skipped. If None is set, it uses the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses - the default value, empty string. + the default value, empty string. Since 2.0.1, this ``nullValue`` param + applies to all supported types including the string type. :param nanValue: sets the string representation of a non-number value. If None is set, it uses the default value, ``NaN``. :param positiveInf: sets the string representation of a positive infinity value. If None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 410cb20d2de11..fe3da25a4c4e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -377,7 +377,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * from values being read should be skipped. *
    • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing * whitespaces from values being read should be skipped.
    • - *
    • `nullValue` (default empty string): sets the string representation of a null value.
    • + *
    • `nullValue` (default empty string): sets the string representation of a null value. Since + * 2.0.1, this applies to all supported types including the string type.
    • *
    • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
    • *
    • `positiveInf` (default `Inf`): sets the string representation of a positive infinity * value.
    • diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index 1ca6eff1b8c2e..3ab775c909238 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -232,66 +232,58 @@ private[csv] object CSVTypeCast { nullable: Boolean = true, options: CSVOptions = CSVOptions()): Any = { - castType match { - case _: ByteType => if (datum == options.nullValue && nullable) null else datum.toByte - case _: ShortType => if (datum == options.nullValue && nullable) null else datum.toShort - case _: IntegerType => if (datum == options.nullValue && nullable) null else datum.toInt - case _: LongType => if (datum == options.nullValue && nullable) null else datum.toLong - case _: FloatType => - if (datum == options.nullValue && nullable) { - null - } else if (datum == options.nanValue) { - Float.NaN - } else if (datum == options.negativeInf) { - Float.NegativeInfinity - } else if (datum == options.positiveInf) { - Float.PositiveInfinity - } else { - Try(datum.toFloat) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) - } - case _: DoubleType => - if (datum == options.nullValue && nullable) { - null - } else if (datum == options.nanValue) { - Double.NaN - } else if (datum == options.negativeInf) { - Double.NegativeInfinity - } else if (datum == options.positiveInf) { - Double.PositiveInfinity - } else { - Try(datum.toDouble) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) - } - case _: BooleanType => datum.toBoolean - case dt: DecimalType => - if (datum == options.nullValue && nullable) { - null - } else { - val value = new BigDecimal(datum.replaceAll(",", "")) - Decimal(value, dt.precision, dt.scale) - } - case _: TimestampType => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681. - Try(options.timestampFormat.parse(datum).getTime * 1000L) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.stringToTime(datum).getTime * 1000L + if (nullable && datum == options.nullValue) { + null + } else { + castType match { + case _: ByteType => datum.toByte + case _: ShortType => datum.toShort + case _: IntegerType => datum.toInt + case _: LongType => datum.toLong + case _: FloatType => + datum match { + case options.nanValue => Float.NaN + case options.negativeInf => Float.NegativeInfinity + case options.positiveInf => Float.PositiveInfinity + case _ => + Try(datum.toFloat) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) } - case _: DateType => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681.x - Try(DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime)) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) + case _: DoubleType => + datum match { + case options.nanValue => Double.NaN + case options.negativeInf => Double.NegativeInfinity + case options.positiveInf => Double.PositiveInfinity + case _ => + Try(datum.toDouble) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) } - case _: StringType => UTF8String.fromString(datum) - case udt: UserDefinedType[_] => castTo(datum, udt.sqlType, nullable, options) - case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") + case _: BooleanType => datum.toBoolean + case dt: DecimalType => + val value = new BigDecimal(datum.replaceAll(",", "")) + Decimal(value, dt.precision, dt.scale) + case _: TimestampType => + // This one will lose microseconds parts. + // See https://issues.apache.org/jira/browse/SPARK-10681. + Try(options.timestampFormat.parse(datum).getTime * 1000L) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + DateTimeUtils.stringToTime(datum).getTime * 1000L + } + case _: DateType => + // This one will lose microseconds parts. + // See https://issues.apache.org/jira/browse/SPARK-10681.x + Try(DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime)) + .getOrElse { + // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards + // compatibility. + DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) + } + case _: StringType => UTF8String.fromString(datum) + case udt: UserDefinedType[_] => castTo(datum, udt.sqlType, nullable, options) + case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 613177e43332c..36c80ad8362ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -232,7 +232,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * from values being read should be skipped. *
    • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing * whitespaces from values being read should be skipped.
    • - *
    • `nullValue` (default empty string): sets the string representation of a null value.
    • + *
    • `nullValue` (default empty string): sets the string representation of a null value. Since + * 2.0.1, this applies to all supported types including the string type.
    • *
    • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
    • *
    • `positiveInf` (default `Inf`): sets the string representation of a positive infinity * value.
    • diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 1930862118e9b..29aac9def6924 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -554,7 +554,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { verifyCars(cars, withHeader = true, checkValues = false) val results = cars.collect() - assert(results(0).toSeq === Array(2012, "Tesla", "S", "null", "null")) + assert(results(0).toSeq === Array(2012, "Tesla", "S", null, null)) assert(results(2).toSeq === Array(null, "Chevy", "Volt", null, null)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala index 3ce643e667ce4..dae92f626c225 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala @@ -68,16 +68,46 @@ class CSVTypeCastSuite extends SparkFunSuite { } test("Nullable types are handled") { - assert(CSVTypeCast.castTo("", IntegerType, nullable = true, CSVOptions()) == null) + assertNull( + CSVTypeCast.castTo("-", ByteType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", ShortType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", IntegerType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", LongType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", FloatType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", DoubleType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", BooleanType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", DecimalType.DoubleDecimal, true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", TimestampType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", DateType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", StringType, nullable = true, CSVOptions("nullValue", "-"))) } - test("String type should always return the same as the input") { + test("String type should also respect `nullValue`") { + assertNull( + CSVTypeCast.castTo("", StringType, nullable = true, CSVOptions())) assert( - CSVTypeCast.castTo("", StringType, nullable = true, CSVOptions()) == + CSVTypeCast.castTo("", StringType, nullable = false, CSVOptions()) == UTF8String.fromString("")) + assert( - CSVTypeCast.castTo("", StringType, nullable = false, CSVOptions()) == + CSVTypeCast.castTo("", StringType, nullable = true, CSVOptions("nullValue", "null")) == + UTF8String.fromString("")) + assert( + CSVTypeCast.castTo("", StringType, nullable = false, CSVOptions("nullValue", "null")) == UTF8String.fromString("")) + + assertNull( + CSVTypeCast.castTo(null, StringType, nullable = true, CSVOptions("nullValue", "null"))) } test("Throws exception for empty string with non null type") { @@ -170,20 +200,4 @@ class CSVTypeCastSuite extends SparkFunSuite { assert(doubleVal2 == Double.PositiveInfinity) } - test("Type-specific null values are used for casting") { - assertNull( - CSVTypeCast.castTo("-", ByteType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", ShortType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", IntegerType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", LongType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", FloatType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", DoubleType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", DecimalType.DoubleDecimal, true, CSVOptions("nullValue", "-"))) - } } From 27ce39cf207eba46502ed11fcbfd51bed3e68f2b Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Sun, 18 Sep 2016 15:22:01 -0700 Subject: [PATCH 1318/1470] [SPARK-17571][SQL] AssertOnQuery.condition should always return Boolean value ## What changes were proposed in this pull request? AssertOnQuery has two apply constructor: one that accepts a closure that returns boolean, and another that accepts a closure that returns Unit. This is actually very confusing because developers could mistakenly think that AssertOnQuery always require a boolean return type and verifies the return result, when indeed the value of the last statement is ignored in one of the constructors. This pull request makes the two constructor consistent and always require boolean value. It will overall make the test suites more robust against developer errors. As an evidence for the confusing behavior, this change also identified a bug with an existing test case due to file system time granularity. This pull request fixes that test case as well. ## How was this patch tested? This is a test only change. Author: petermaxlee Closes #15127 from petermaxlee/SPARK-17571. (cherry picked from commit 8f0c35a4d0dd458719627be5f524792bf244d70a) Signed-off-by: Reynold Xin --- .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 7 +++++-- .../scala/org/apache/spark/sql/streaming/StreamTest.scala | 4 ++-- .../spark/sql/streaming/StreamingQueryListenerSuite.scala | 3 +++ 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 886f7be59db93..a02a36c00499c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -354,7 +354,9 @@ class FileStreamSourceSuite extends FileStreamSourceTest { CheckAnswer("a", "b"), // SLeeps longer than 5ms (maxFileAge) - AssertOnQuery { _ => Thread.sleep(10); true }, + // Unfortunately since a lot of file system does not have modification time granularity + // finer grained than 1 sec, we need to use 1 sec here. + AssertOnQuery { _ => Thread.sleep(1000); true }, AddTextFileData("c\nd", src, tmp), CheckAnswer("a", "b", "c", "d"), @@ -363,7 +365,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val source = streamExecution.logicalPlan.collect { case e: StreamingExecutionRelation => e.source.asInstanceOf[FileStreamSource] }.head - source.seenFiles.size == 1 + assert(source.seenFiles.size == 1) + true } ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index af2b58116b2aa..6c5b170d9c7c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -188,8 +188,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { new AssertOnQuery(condition, message) } - def apply(message: String)(condition: StreamExecution => Unit): AssertOnQuery = { - new AssertOnQuery(s => { condition(s); true }, message) + def apply(message: String)(condition: StreamExecution => Boolean): AssertOnQuery = { + new AssertOnQuery(condition, message) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 77602e8167fa3..831543a47420a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -66,6 +66,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // No progress events or termination events assert(listener.progressStatuses.isEmpty) assert(listener.terminationStatus === null) + true }, AddDataMemory(input, Seq(1, 2, 3)), CheckAnswer(1, 2, 3), @@ -84,6 +85,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // No termination events assert(listener.terminationStatus === null) } + true }, StopStream, AssertOnQuery("Incorrect query status in onQueryTerminated") { query => @@ -97,6 +99,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(listener.terminationException === None) } listener.checkAsyncErrors() + true } ) } From ac060397c109158e84a2b57355c8dee5ab24ab7b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 19 Sep 2016 09:38:25 +0100 Subject: [PATCH 1319/1470] [SPARK-17297][DOCS] Clarify window/slide duration as absolute time, not relative to a calendar ## What changes were proposed in this pull request? Clarify that slide and window duration are absolute, and not relative to a calendar. ## How was this patch tested? Doc build (no functional change) Author: Sean Owen Closes #15142 from srowen/SPARK-17297. (cherry picked from commit d720a4019460b6c284d0473249303c349df60a1f) Signed-off-by: Sean Owen --- R/pkg/R/functions.R | 8 ++++++-- .../scala/org/apache/spark/sql/functions.scala | 15 +++++++++++---- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index ceedbe76711b1..4d94b4cd05d44 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2713,11 +2713,15 @@ setMethod("from_unixtime", signature(x = "Column"), #' @param x a time Column. Must be of TimestampType. #' @param windowDuration a string specifying the width of the window, e.g. '1 second', #' '1 day 12 hours', '2 minutes'. Valid interval strings are 'week', -#' 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. +#' 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. Note that +#' the duration is a fixed length of time, and does not vary over time +#' according to a calendar. For example, '1 day' always means 86,400,000 +#' milliseconds, not a calendar day. #' @param slideDuration a string specifying the sliding interval of the window. Same format as #' \code{windowDuration}. A new window will be generated every #' \code{slideDuration}. Must be less than or equal to -#' the \code{windowDuration}. +#' the \code{windowDuration}. This duration is likewise absolute, and does not +#' vary according to a calendar. #' @param startTime the offset with respect to 1970-01-01 00:00:00 UTC with which to start #' window intervals. For example, in order to have hourly tumbling windows #' that start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide 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 4e185b85e7660..eb504c81bd80f 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 @@ -2596,12 +2596,15 @@ object functions { * The time column must be of TimestampType. * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, * `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for - * valid duration identifiers. + * valid duration identifiers. Note that the duration is a fixed length of + * time, and does not vary over time according to a calendar. For example, + * `1 day` always means 86,400,000 milliseconds, not a calendar day. * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. * A new window will be generated every `slideDuration`. Must be less than * or equal to the `windowDuration`. Check * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration - * identifiers. + * identifiers. This duration is likewise absolute, and does not vary + * according to a calendar. * @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start * window intervals. For example, in order to have hourly tumbling windows that * start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide @@ -2650,11 +2653,15 @@ object functions { * The time column must be of TimestampType. * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, * `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for - * valid duration identifiers. + * valid duration identifiers. Note that the duration is a fixed length of + * time, and does not vary over time according to a calendar. For example, + * `1 day` always means 86,400,000 milliseconds, not a calendar day. * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. * A new window will be generated every `slideDuration`. Must be less than * or equal to the `windowDuration`. Check - * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration. + * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration + * identifiers. This duration is likewise absolute, and does not vary + * according to a calendar. * * @group datetime_funcs * @since 2.0.0 From c4660d607fbeacc9bdbe2bb1293e4401d19a4bd5 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 19 Sep 2016 10:21:33 -0700 Subject: [PATCH 1320/1470] [SPARK-17589][TEST][2.0] Fix test case `create external table` in MetastoreDataSourcesSuite ### What changes were proposed in this pull request? This PR is to fix a test failure on the branch 2.0 builds: https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.0-test-maven-hadoop-2.7/711/ ``` Error Message "Table `default`.`createdJsonTable` already exists.;" did not contain "Table default.createdJsonTable already exists." We should complain that createdJsonTable already exists ``` ### How was this patch tested? N/A Author: gatorsmile Closes #15145 from gatorsmile/fixTestCase. --- .../spark/sql/execution/command/createDataSourceTables.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 06965ff7650ca..f282d542550c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -82,7 +82,7 @@ case class CreateDataSourceTableCommand( if (ignoreIfExists) { return Seq.empty[Row] } else { - throw new AnalysisException(s"Table ${tableIdentWithDB.quotedString} already exists.") + throw new AnalysisException(s"Table ${tableIdentWithDB.unquotedString} already exists.") } } From f56035ba6c86fe93a45fd437f98f812431df0069 Mon Sep 17 00:00:00 2001 From: sureshthalamati Date: Mon, 19 Sep 2016 09:56:16 -0700 Subject: [PATCH 1321/1470] [SPARK-17473][SQL] fixing docker integration tests error due to different versions of jars. ## What changes were proposed in this pull request? Docker tests are using older version of jersey jars (1.19), which was used in older releases of spark. In 2.0 releases Spark was upgraded to use 2.x verison of Jersey. After upgrade to new versions, docker tests are failing with AbstractMethodError. Now that spark is upgraded to 2.x jersey version, using of shaded docker jars may not be required any more. Removed the exclusions/overrides of jersey related classes from pom file, and changed the docker-client to use regular jar instead of shaded one. ## How was this patch tested? Tested using existing docker-integration-tests Author: sureshthalamati Closes #15114 from sureshthalamati/docker_testfix-spark-17473. (cherry picked from commit cdea1d1343d02f0077e1f3c92ca46d04a3d30414) Signed-off-by: Josh Rosen --- external/docker-integration-tests/pom.xml | 68 ----------------------- pom.xml | 1 - 2 files changed, 69 deletions(-) diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 18e14c7981d80..0ca94e5387f68 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -49,38 +49,7 @@ com.spotify docker-client - shaded test - - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - - - com.fasterxml.jackson.core - jackson-databind - - - org.glassfish.jersey.core - jersey-client - - - org.glassfish.jersey.connectors - jersey-apache-connector - - - org.glassfish.jersey.media - jersey-media-json-jackson - - org.apache.httpcomponents @@ -152,43 +121,6 @@ test - - - com.sun.jersey - jersey-server - 1.19 - test - - - com.sun.jersey - jersey-core - 1.19 - test - - - com.sun.jersey - jersey-servlet - 1.19 - test - - - com.sun.jersey - jersey-json - 1.19 - test - - - stax - stax-api - - - - - + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins maven-shade-plugin From 59e6ab11a9e27d30ae3477fdc03337ff5f8ab4ec Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 21 Sep 2016 17:57:21 -0400 Subject: [PATCH 1335/1470] [SPARK-17512][CORE] Avoid formatting to python path for yarn and mesos cluster mode ## What changes were proposed in this pull request? Yarn and mesos cluster mode support remote python path (HDFS/S3 scheme) by their own mechanism, it is not necessary to check and format the python when running on these modes. This is a potential regression compared to 1.6, so here propose to fix it. ## How was this patch tested? Unit test to verify SparkSubmit arguments, also with local cluster verification. Because of lack of `MiniDFSCluster` support in Spark unit test, there's no integration test added. Author: jerryshao Closes #15137 from jerryshao/SPARK-17512. (cherry picked from commit 8c3ee2bc42e6320b9341cebdba51a00162c897ea) Signed-off-by: Andrew Or --- .../org/apache/spark/deploy/SparkSubmit.scala | 13 ++++++++++--- .../spark/deploy/SparkSubmitSuite.scala | 19 +++++++++++++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 7b6d5a394bc35..80611658a1640 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -311,7 +311,7 @@ object SparkSubmit { // In Mesos cluster mode, non-local python files are automatically downloaded by Mesos. if (args.isPython && !isYarnCluster && !isMesosCluster) { if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { - printErrorAndExit(s"Only local python files are supported: $args.primaryResource") + printErrorAndExit(s"Only local python files are supported: ${args.primaryResource}") } val nonLocalPyFiles = Utils.nonLocalPaths(args.pyFiles).mkString(",") if (nonLocalPyFiles.nonEmpty) { @@ -322,7 +322,7 @@ 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") + printErrorAndExit(s"Only local R files are supported: ${args.primaryResource}") } } @@ -633,7 +633,14 @@ object SparkSubmit { // explicitly sets `spark.submit.pyFiles` in his/her default properties file. sysProps.get("spark.submit.pyFiles").foreach { pyFiles => val resolvedPyFiles = Utils.resolveURIs(pyFiles) - val formattedPyFiles = PythonRunner.formatPaths(resolvedPyFiles).mkString(",") + val formattedPyFiles = if (!isYarnCluster && !isMesosCluster) { + PythonRunner.formatPaths(resolvedPyFiles).mkString(",") + } else { + // Ignoring formatting python path in yarn and mesos cluster mode, these two modes + // support dealing with remote python files, they could distribute and add python files + // locally. + resolvedPyFiles + } sysProps("spark.submit.pyFiles") = formattedPyFiles } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index b2bc8861083bb..54693c1bf81ee 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -577,6 +577,25 @@ class SparkSubmitSuite val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3 sysProps3("spark.submit.pyFiles") should be( PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + + // Test remote python files + val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) + val writer4 = new PrintWriter(f4) + val remotePyFiles = "hdfs:///tmp/file1.py,hdfs:///tmp/file2.py" + writer4.println("spark.submit.pyFiles " + remotePyFiles) + writer4.close() + val clArgs4 = Seq( + "--master", "yarn", + "--deploy-mode", "cluster", + "--properties-file", f4.getPath, + "hdfs:///tmp/mister.py" + ) + val appArgs4 = new SparkSubmitArguments(clArgs4) + val sysProps4 = SparkSubmit.prepareSubmitEnvironment(appArgs4)._3 + // Should not format python path for yarn cluster mode + sysProps4("spark.submit.pyFiles") should be( + Utils.resolveURIs(remotePyFiles) + ) } test("user classpath first in driver") { From 966abd6af04b8e7b5f6446cba96f1825ca2bfcfa Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 21 Sep 2016 20:59:46 -0700 Subject: [PATCH 1336/1470] [SPARK-17627] Mark Streaming Providers Experimental All of structured streaming is experimental in its first release. We missed the annotation on two of the APIs. Author: Michael Armbrust Closes #15188 from marmbrus/experimentalApi. (cherry picked from commit 3497ebe511fee67e66387e9e737c843a2939ce45) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/sql/sources/interfaces.scala | 4 ++++ 1 file changed, 4 insertions(+) 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 d2077a07f440a..b84953deac9e7 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 @@ -112,8 +112,10 @@ trait SchemaRelationProvider { } /** + * ::Experimental:: * Implemented by objects that can produce a streaming [[Source]] for a specific format or system. */ +@Experimental trait StreamSourceProvider { /** Returns the name and schema of the source that can be used to continually read data. */ @@ -132,8 +134,10 @@ trait StreamSourceProvider { } /** + * ::Experimental:: * Implemented by objects that can produce a streaming [[Sink]] for a specific format or system. */ +@Experimental trait StreamSinkProvider { def createSink( sqlContext: SQLContext, From ec377e77307b477d20a642edcd5ad5e26b989de6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 21 Sep 2016 21:02:30 -0700 Subject: [PATCH 1337/1470] [SPARK-17494][SQL] changePrecision() on compact decimal should respect rounding mode ## What changes were proposed in this pull request? Floor()/Ceil() of decimal is implemented using changePrecision() by passing a rounding mode, but the rounding mode is not respected when the decimal is in compact mode (could fit within a Long). This Update the changePrecision() to respect rounding mode, which could be ROUND_FLOOR, ROUND_CEIL, ROUND_HALF_UP, ROUND_HALF_EVEN. ## How was this patch tested? Added regression tests. Author: Davies Liu Closes #15154 from davies/decimal_round. (cherry picked from commit 8bde03bf9a0896ea59ceaa699df7700351a130fb) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/types/Decimal.scala | 28 ++++++++++++++++--- .../apache/spark/sql/types/DecimalSuite.scala | 15 ++++++++++ 2 files changed, 39 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index cc8175c0a366d..70859052872dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -242,10 +242,30 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (scale < _scale) { // Easier case: we just need to divide our scale down val diff = _scale - scale - val droppedDigits = longVal % POW_10(diff) - longVal /= POW_10(diff) - if (math.abs(droppedDigits) * 2 >= POW_10(diff)) { - longVal += (if (longVal < 0) -1L else 1L) + val pow10diff = POW_10(diff) + // % and / always round to 0 + val droppedDigits = longVal % pow10diff + longVal /= pow10diff + roundMode match { + case ROUND_FLOOR => + if (droppedDigits < 0) { + longVal += -1L + } + case ROUND_CEILING => + if (droppedDigits > 0) { + longVal += 1L + } + case ROUND_HALF_UP => + if (math.abs(droppedDigits) * 2 >= pow10diff) { + longVal += (if (droppedDigits < 0) -1L else 1L) + } + case ROUND_HALF_EVEN => + val doubled = math.abs(droppedDigits) * 2 + if (doubled > pow10diff || doubled == pow10diff && longVal % 2 != 0) { + longVal += (if (droppedDigits < 0) -1L else 1L) + } + case _ => + sys.error(s"Not supported rounding mode: $roundMode") } } else if (scale > _scale) { // We might be able to multiply longVal by a power of 10 and not overflow, but if not, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index e1675c95907af..4cf329ddee213 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -22,6 +22,7 @@ import scala.language.postfixOps import org.scalatest.PrivateMethodTester import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.Decimal._ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { /** Check that a Decimal has the given string representation, precision and scale */ @@ -193,4 +194,18 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { assert(new Decimal().set(100L, 10, 0).toUnscaledLong === 100L) assert(Decimal(Long.MaxValue, 100, 0).toUnscaledLong === Long.MaxValue) } + + test("changePrecision() on compact decimal should respect rounding mode") { + Seq(ROUND_FLOOR, ROUND_CEILING, ROUND_HALF_UP, ROUND_HALF_EVEN).foreach { mode => + Seq("0.4", "0.5", "0.6", "1.0", "1.1", "1.6", "2.5", "5.5").foreach { n => + Seq("", "-").foreach { sign => + val bd = BigDecimal(sign + n) + val unscaled = (bd * 10).toLongExact + val d = Decimal(unscaled, 8, 1) + assert(d.changePrecision(10, 0, mode)) + assert(d.toString === bd.setScale(0, mode).toString(), s"num: $sign$n, mode: $mode") + } + } + } + } } From 053b20a79c1824917c17405f30a7b91472311abe Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 21 Sep 2016 21:06:47 -0700 Subject: [PATCH 1338/1470] Bump doc version for release 2.0.1. --- docs/_config.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index 3951cadb0ffd5..75c89bd318983 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.0 -SPARK_VERSION_SHORT: 2.0.0 +SPARK_VERSION: 2.0.1 +SPARK_VERSION_SHORT: 2.0.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" MESOS_VERSION: 0.21.0 From 00f2e28edd5a74f75e8b4c58894eeb3a394649d7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 21 Sep 2016 21:09:08 -0700 Subject: [PATCH 1339/1470] Preparing Spark release v2.0.1-rc1 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 507ddc778383d..6db3a599ff5ea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index e170b9b7e30e4..269b845565f1f 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 8b832cf376123..20cf29efffc71 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 07d9f1c58f7a3..25cc32889ef3c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5e02efdc45e67..37a5d09a3ff02 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e7fc6a2a02417..ab287f3368a43 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 24f0e75f2f045..45831ce98dbcc 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 208659bcb92da..2d19e5b81cf58 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d2227944d92dd..89e0c61a3d694 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 0ca94e5387f68..8c6e22155c26c 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 3847b39e17315..dd45935e6eec2 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7fc8a4b00dc20..ba97794605ee6 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index dcdbcc55d8785..8f8bde7eadfb9 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 1eb349ff4c3b9..f7d8ef7b87c5d 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 59f41f1e17f30..260969fcd7347 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index f9776fc63686e..1ae1d0e216b15 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a346d76210932..a4b14f8fb9603 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 2aa7d82e8e064..9964b22cfe39f 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 8fc6fd92b38bf..b5d90b151cc8f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index b524001d04719..f96db6588cdad 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 93ac8b6e664d7..40f2e38832fbd 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 4f8af77792b0b..979217e2ba8f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index b6591598ee127..0c0dd0c6069d9 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1d8f7f4d9bbed..5681b36ea9ab8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40fde1bab7ad1..80ab2e01a4576 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 97bb085ee123f..79255f9a1f827 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 6fe1d7d7cd45e..4b70d647d59e5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0bfdb13cec941..efa327cbf21cb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02a18b33b087a..347f8a1bc82b1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 672425c86ecbe..c47d5f0285169 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index add4375364b1a..2388d5f6ef1d4 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index e7614fe66110d..4e0f5b1013f91 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 3f4cce1ca354e..ddc1b091fec81 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 7dba1a829fb94..b676b6d21a2c1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1-SNAPSHOT + 2.0.1 ../pom.xml From e8b26be9bf2b7c46f40111176b4d0597ea8451d3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 21 Sep 2016 21:09:19 -0700 Subject: [PATCH 1340/1470] Preparing development version 2.0.2-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 6db3a599ff5ea..ca6daa27614f5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 269b845565f1f..c727f54d2ca3d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 20cf29efffc71..e335a89aac69e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 25cc32889ef3c..8e64f5678d78c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 37a5d09a3ff02..94c75d68e34f5 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ab287f3368a43..6ff14d29474c2 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 45831ce98dbcc..e654c5479b34c 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2d19e5b81cf58..914025104d6ad 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 89e0c61a3d694..1796c472ca59e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 8c6e22155c26c..ae3746fe55ee3 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index dd45935e6eec2..6cf206d36f448 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index ba97794605ee6..9ae7968c7296e 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 8f8bde7eadfb9..7a67142dbf07f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index f7d8ef7b87c5d..0f795ff77b5ed 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 260969fcd7347..658c396c57f19 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 1ae1d0e216b15..e52170acedf87 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4b14f8fb9603..a4f339fb3fa2f 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 9964b22cfe39f..4d2df2e45b1a4 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index b5d90b151cc8f..0e5ce9a78887f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f96db6588cdad..c9a9e73a0ddb9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 40f2e38832fbd..b92359a476268 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 979217e2ba8f2..c88efda8955eb 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 0c0dd0c6069d9..4375e645ca05d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 5681b36ea9ab8..9e3d7fbbdd565 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 80ab2e01a4576..db7f05302a12b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 79255f9a1f827..f2ffb39a6b5f9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 4b70d647d59e5..afb8891168fe6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index efa327cbf21cb..ad3d408b8e9b2 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 347f8a1bc82b1..5e67226c6a61a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c47d5f0285169..7624c82b8ac51 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 2388d5f6ef1d4..ae66d7eeb32bf 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 4e0f5b1013f91..897ef824976fd 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ddc1b091fec81..7f55ebc87f682 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index b676b6d21a2c1..d81ce15b8bdf3 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml From b25a8e6e167717fbe92e6a9b69a8a2510bf926ca Mon Sep 17 00:00:00 2001 From: frreiss Date: Thu, 22 Sep 2016 10:31:15 +0100 Subject: [PATCH 1341/1470] [SPARK-17421][DOCS] Documenting the current treatment of MAVEN_OPTS. ## What changes were proposed in this pull request? Modified the documentation to clarify that `build/mvn` and `pom.xml` always add Java 7-specific parameters to `MAVEN_OPTS`, and that developers can safely ignore warnings about `-XX:MaxPermSize` that may result from compiling or running tests with Java 8. ## How was this patch tested? Rebuilt HTML documentation, made sure that building-spark.html displays correctly in a browser. Author: frreiss Closes #15005 from frreiss/fred-17421a. (cherry picked from commit 646f383465c123062cbcce288a127e23984c7c7f) Signed-off-by: Sean Owen --- docs/building-spark.md | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 2c987cf8346ef..330df0054154b 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -16,11 +16,13 @@ Building Spark using Maven requires Maven 3.3.9 or newer and Java 7+. ### Setting up Maven's Memory Usage -You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: +You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`: - export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" -If you don't run this, you may see errors like the following: +When compiling with Java 7, you will need to add the additional option "-XX:MaxPermSize=512M" to MAVEN_OPTS. + +If you don't add these parameters to `MAVEN_OPTS`, you may see errors and warnings like the following: [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes... [ERROR] PermGen space -> [Help 1] @@ -28,12 +30,18 @@ If you don't run this, you may see errors like the following: [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes... [ERROR] Java heap space -> [Help 1] -You can fix this by setting the `MAVEN_OPTS` variable as discussed before. + [INFO] Compiling 233 Scala sources and 41 Java sources to /Users/me/Development/spark/sql/core/target/scala-{site.SCALA_BINARY_VERSION}/classes... + OpenJDK 64-Bit Server VM warning: CodeCache is full. Compiler has been disabled. + OpenJDK 64-Bit Server VM warning: Try increasing the code cache size using -XX:ReservedCodeCacheSize= + +You can fix these problems by setting the `MAVEN_OPTS` variable as discussed before. **Note:** -* For Java 8 and above this step is not required. -* If using `build/mvn` with no `MAVEN_OPTS` set, the script will automate this for you. +* If using `build/mvn` with no `MAVEN_OPTS` set, the script will automatically add the above options to the `MAVEN_OPTS` environment variable. +* The `test` phase of the Spark build will automatically add these options to `MAVEN_OPTS`, even when not using `build/mvn`. +* You may see warnings like "ignoring option MaxPermSize=1g; support was removed in 8.0" when building or running tests with Java 8 and `build/mvn`. These warnings are harmless. + ### build/mvn From f14f47f072a392df0ebe908f1c57b6eb858105b7 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 22 Sep 2016 11:52:42 -0700 Subject: [PATCH 1342/1470] Skip building R vignettes if Spark is not built ## What changes were proposed in this pull request? When we build the docs separately we don't have the JAR files from the Spark build in the same tree. As the SparkR vignettes need to launch a SparkContext to be built, we skip building them if JAR files don't exist ## How was this patch tested? To test this we can run the following: ``` build/mvn -DskipTests -Psparkr clean ./R/create-docs.sh ``` You should see a line `Skipping R vignettes as Spark JARs not found` at the end Author: Shivaram Venkataraman Closes #15200 from shivaram/sparkr-vignette-skip. (cherry picked from commit 9f24a17c59b1130d97efa7d313c06577f7344338) Signed-off-by: Reynold Xin --- R/create-docs.sh | 25 ++++++++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/R/create-docs.sh b/R/create-docs.sh index 0dfba22463396..69ffc5f678c36 100755 --- a/R/create-docs.sh +++ b/R/create-docs.sh @@ -30,6 +30,13 @@ set -e # Figure out where the script is export FWDIR="$(cd "`dirname "$0"`"; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + +# Required for setting SPARK_SCALA_VERSION +. "${SPARK_HOME}"/bin/load-spark-env.sh + +echo "Using Scala $SPARK_SCALA_VERSION" + pushd $FWDIR # Install the package (this will also generate the Rd files) @@ -45,9 +52,21 @@ Rscript -e 'libDir <- "../../lib"; library(SparkR, lib.loc=libDir); library(knit popd -# render creates SparkR vignettes -Rscript -e 'library(rmarkdown); paths <- .libPaths(); .libPaths(c("lib", paths)); Sys.setenv(SPARK_HOME=tools::file_path_as_absolute("..")); render("pkg/vignettes/sparkr-vignettes.Rmd"); .libPaths(paths)' +# Find Spark jars. +if [ -f "${SPARK_HOME}/RELEASE" ]; then + SPARK_JARS_DIR="${SPARK_HOME}/jars" +else + SPARK_JARS_DIR="${SPARK_HOME}/assembly/target/scala-$SPARK_SCALA_VERSION/jars" +fi + +# Only create vignettes if Spark JARs exist +if [ -d "$SPARK_JARS_DIR" ]; then + # render creates SparkR vignettes + Rscript -e 'library(rmarkdown); paths <- .libPaths(); .libPaths(c("lib", paths)); Sys.setenv(SPARK_HOME=tools::file_path_as_absolute("..")); render("pkg/vignettes/sparkr-vignettes.Rmd"); .libPaths(paths)' -find pkg/vignettes/. -not -name '.' -not -name '*.Rmd' -not -name '*.md' -not -name '*.pdf' -not -name '*.html' -delete + find pkg/vignettes/. -not -name '.' -not -name '*.Rmd' -not -name '*.md' -not -name '*.pdf' -not -name '*.html' -delete +else + echo "Skipping R vignettes as Spark JARs not found in $SPARK_HOME" +fi popd From 243bdb11d89ee379acae1ea1ed78df10797e86d1 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 22 Sep 2016 13:05:41 -0700 Subject: [PATCH 1343/1470] [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames Consider you have a bucket as `s3a://some-bucket` and under it you have files: ``` s3a://some-bucket/file1.parquet s3a://some-bucket/file2.parquet ``` Getting the parent path of `s3a://some-bucket/file1.parquet` yields `s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map. When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list! This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics. Unit test in `FileCatalogSuite`. Author: Burak Yavuz Closes #15169 from brkyvz/SPARK-17613. (cherry picked from commit 85d609cf25c1da2df3cd4f5d5aeaf3cbcf0d674c) Signed-off-by: Josh Rosen --- .../PartitioningAwareFileCatalog.scala | 10 ++++- .../datasources/FileCatalogSuite.scala | 45 ++++++++++++++++++- 2 files changed, 53 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index cef9d4d9c7f1b..2130c27ebd8de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -76,7 +76,15 @@ abstract class PartitioningAwareFileCatalog( paths.flatMap { path => // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). val fs = path.getFileSystem(hadoopConf) - val qualifiedPath = fs.makeQualified(path) + val qualifiedPathPre = fs.makeQualified(path) + val qualifiedPath: Path = if (qualifiedPathPre.isRoot && !qualifiedPathPre.isAbsolute) { + // SPARK-17613: Always append `Path.SEPARATOR` to the end of parent directories, + // because the `leafFile.getParent` would have returned an absolute path with the + // separator at the end. + new Path(qualifiedPathPre, Path.SEPARATOR) + } else { + qualifiedPathPre + } // There are three cases possible with each path // 1. The path is a directory and has children files in it. Then it must be present in diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 0d9ea512729bd..563f34068812c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.execution.datasources import java.io.File +import java.net.URI +import scala.collection.mutable import scala.language.reflectiveCalls -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test.SharedSQLContext @@ -67,4 +69,45 @@ class FileCatalogSuite extends SharedSQLContext { } } + + test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { + class MockCatalog( + override val paths: Seq[Path]) extends PartitioningAwareFileCatalog(spark, Map.empty, None) { + + override def refresh(): Unit = {} + + override def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = mutable.LinkedHashMap( + new Path("mockFs://some-bucket/file1.json") -> new FileStatus() + ) + + override def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = Map( + new Path("mockFs://some-bucket/") -> Array(new FileStatus()) + ) + + override def partitionSpec(): PartitionSpec = { + PartitionSpec.emptySpec + } + } + + withSQLConf( + "fs.mockFs.impl" -> classOf[FakeParentPathFileSystem].getName, + "fs.mockFs.impl.disable.cache" -> "true") { + val pathWithSlash = new Path("mockFs://some-bucket/") + assert(pathWithSlash.getParent === null) + val pathWithoutSlash = new Path("mockFs://some-bucket") + assert(pathWithoutSlash.getParent === null) + val catalog1 = new MockCatalog(Seq(pathWithSlash)) + val catalog2 = new MockCatalog(Seq(pathWithoutSlash)) + assert(catalog1.allFiles().nonEmpty) + assert(catalog2.allFiles().nonEmpty) + } + } +} + +class FakeParentPathFileSystem extends RawLocalFileSystem { + override def getScheme: String = "mockFs" + + override def getUri: URI = { + URI.create("mockFs://some-bucket") + } } From 47fc0b9f40d814bc8e19f86dad591d4aed467222 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 22 Sep 2016 14:26:45 -0700 Subject: [PATCH 1344/1470] [SPARK-17638][STREAMING] Stop JVM StreamingContext when the Python process is dead ## What changes were proposed in this pull request? When the Python process is dead, the JVM StreamingContext is still running. Hence we will see a lot of Py4jException before the JVM process exits. It's better to stop the JVM StreamingContext to avoid those annoying logs. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15201 from zsxwing/stop-jvm-ssc. (cherry picked from commit 3cdae0ff2f45643df7bc198cb48623526c7eb1a6) Signed-off-by: Shixiong Zhu --- .../streaming/api/python/PythonDStream.scala | 33 +++++++++++++++++-- .../streaming/scheduler/JobGenerator.scala | 2 ++ .../streaming/scheduler/JobScheduler.scala | 2 ++ 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index aeff4d7a98e7a..46bfc60856453 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -24,11 +24,14 @@ import java.util.{ArrayList => JArrayList, List => JList} import scala.collection.JavaConverters._ import scala.language.existentials +import py4j.Py4JException + import org.apache.spark.SparkException import org.apache.spark.api.java._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Duration, Interval, Time} +import org.apache.spark.streaming.{Duration, Interval, StreamingContext, Time} import org.apache.spark.streaming.api.java._ import org.apache.spark.streaming.dstream._ import org.apache.spark.util.Utils @@ -157,7 +160,7 @@ private[python] object PythonTransformFunctionSerializer { /** * Helper functions, which are called from Python via Py4J. */ -private[python] object PythonDStream { +private[streaming] object PythonDStream { /** * can not access PythonTransformFunctionSerializer.register() via Py4j @@ -184,6 +187,32 @@ private[python] object PythonDStream { rdds.asScala.foreach(queue.add) queue } + + /** + * Stop [[StreamingContext]] if the Python process crashes (E.g., OOM) in case the user cannot + * stop it in the Python side. + */ + def stopStreamingContextIfPythonProcessIsDead(e: Throwable): Unit = { + // These two special messages are from: + // scalastyle:off + // https://github.com/bartdag/py4j/blob/5cbb15a21f857e8cf334ce5f675f5543472f72eb/py4j-java/src/main/java/py4j/CallbackClient.java#L218 + // https://github.com/bartdag/py4j/blob/5cbb15a21f857e8cf334ce5f675f5543472f72eb/py4j-java/src/main/java/py4j/CallbackClient.java#L340 + // scalastyle:on + if (e.isInstanceOf[Py4JException] && + ("Cannot obtain a new communication channel" == e.getMessage || + "Error while obtaining a new communication channel" == e.getMessage)) { + // Start a new thread to stop StreamingContext to avoid deadlock. + new Thread("Stop-StreamingContext") with Logging { + setDaemon(true) + + override def run(): Unit = { + logError( + "Cannot connect to Python process. It's probably dead. Stopping StreamingContext.", e) + StreamingContext.getActive().foreach(_.stop(stopSparkContext = false)) + } + }.start() + } + } } /** 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 19c88f1ee0114..4489a5334d17e 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 @@ -22,6 +22,7 @@ import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} +import org.apache.spark.streaming.api.python.PythonDStream import org.apache.spark.streaming.util.RecurringTimer import org.apache.spark.util.{Clock, EventLoop, ManualClock, Utils} @@ -252,6 +253,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { jobScheduler.submitJobSet(JobSet(time, jobs, streamIdToInputInfos)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) + PythonDStream.stopStreamingContextIfPythonProcessIsDead(e) } eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = false)) } 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 79d6254eb372b..f5ba5edad9e25 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 @@ -27,6 +27,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark.internal.Logging import org.apache.spark.rdd.{PairRDDFunctions, RDD} import org.apache.spark.streaming._ +import org.apache.spark.streaming.api.python.PythonDStream import org.apache.spark.streaming.ui.UIUtils import org.apache.spark.util.{EventLoop, ThreadUtils} @@ -210,6 +211,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private def handleError(msg: String, e: Throwable) { logError(msg, e) ssc.waiter.notifyError(e) + PythonDStream.stopStreamingContextIfPythonProcessIsDead(e) } private class JobHandler(job: Job) extends Runnable with Logging { From 0a593db360b3b7771f45f482cf45e8500f0faa76 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 22 Sep 2016 14:29:27 -0700 Subject: [PATCH 1345/1470] [SPARK-17616][SQL] Support a single distinct aggregate combined with a non-partial aggregate We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example: ```sql select grp, collect_list(col1), count(distinct col2) from tbl_a group by 1 ``` This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6). Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`. Author: Herman van Hovell Closes #15187 from hvanhovell/SPARK-17616. (cherry picked from commit 0d634875026ccf1eaf984996e9460d7673561f80) Signed-off-by: Herman van Hovell --- .../optimizer/RewriteDistinctAggregates.scala | 18 ++-- .../RewriteDistinctAggregatesSuite.scala | 94 +++++++++++++++++++ .../spark/sql/DataFrameAggregateSuite.scala | 8 ++ 3 files changed, 111 insertions(+), 9 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index 0f43e7bb88733..d6a39ecf53b86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -119,14 +119,16 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { .filter(_.isDistinct) .groupBy(_.aggregateFunction.children.toSet) - // Aggregation strategy can handle the query with single distinct - if (distinctAggGroups.size > 1) { + // Check if the aggregates contains functions that do not support partial aggregation. + val existsNonPartial = aggExpressions.exists(!_.aggregateFunction.supportsPartial) + + // Aggregation strategy can handle queries with a single distinct group and partial aggregates. + if (distinctAggGroups.size > 1 || (distinctAggGroups.size == 1 && existsNonPartial)) { // Create the attributes for the grouping id and the group by clause. - val gid = - new AttributeReference("gid", IntegerType, false)(isGenerated = true) + val gid = AttributeReference("gid", IntegerType, nullable = false)(isGenerated = true) val groupByMap = a.groupingExpressions.collect { case ne: NamedExpression => ne -> ne.toAttribute - case e => e -> new AttributeReference(e.sql, e.dataType, e.nullable)() + case e => e -> AttributeReference(e.sql, e.dataType, e.nullable)() } val groupByAttrs = groupByMap.map(_._2) @@ -135,9 +137,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { def patchAggregateFunctionChildren( af: AggregateFunction)( attrs: Expression => Expression): AggregateFunction = { - af.withNewChildren(af.children.map { - case afc => attrs(afc) - }).asInstanceOf[AggregateFunction] + af.withNewChildren(af.children.map(attrs)).asInstanceOf[AggregateFunction] } // Setup unique distinct aggregate children. @@ -265,5 +265,5 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { // NamedExpression. This is done to prevent collisions between distinct and regular aggregate // children, in this case attribute reuse causes the input of the regular aggregate to bound to // the (nulled out) input of the distinct aggregate. - e -> new AttributeReference(e.sql, e.dataType, true)() + e -> AttributeReference(e.sql, e.dataType, nullable = true)() } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala new file mode 100644 index 0000000000000..0b973c3b659cf --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.SimpleCatalystConf +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{If, Literal} +import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, Count} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, LocalRelation, LogicalPlan} +import org.apache.spark.sql.types.{IntegerType, StringType} + +class RewriteDistinctAggregatesSuite extends PlanTest { + val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, groupByOrdinal = false) + val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) + val analyzer = new Analyzer(catalog, conf) + + val nullInt = Literal(null, IntegerType) + val nullString = Literal(null, StringType) + val testRelation = LocalRelation('a.string, 'b.string, 'c.string, 'd.string, 'e.int) + + private def checkRewrite(rewrite: LogicalPlan): Unit = rewrite match { + case Aggregate(_, _, Aggregate(_, _, _: Expand)) => + case _ => fail(s"Plan is not rewritten:\n$rewrite") + } + + test("single distinct group") { + val input = testRelation + .groupBy('a)(countDistinct('e)) + .analyze + val rewrite = RewriteDistinctAggregates(input) + comparePlans(input, rewrite) + } + + test("single distinct group with partial aggregates") { + val input = testRelation + .groupBy('a, 'd)( + countDistinct('e, 'c).as('agg1), + max('b).as('agg2)) + .analyze + val rewrite = RewriteDistinctAggregates(input) + comparePlans(input, rewrite) + } + + test("single distinct group with non-partial aggregates") { + val input = testRelation + .groupBy('a, 'd)( + countDistinct('e, 'c).as('agg1), + CollectSet('b).toAggregateExpression().as('agg2)) + .analyze + checkRewrite(RewriteDistinctAggregates(input)) + } + + test("multiple distinct groups") { + val input = testRelation + .groupBy('a)(countDistinct('b, 'c), countDistinct('d)) + .analyze + checkRewrite(RewriteDistinctAggregates(input)) + } + + test("multiple distinct groups with partial aggregates") { + val input = testRelation + .groupBy('a)(countDistinct('b, 'c), countDistinct('d), sum('e)) + .analyze + checkRewrite(RewriteDistinctAggregates(input)) + } + + test("multiple distinct groups with non-partial aggregates") { + val input = testRelation + .groupBy('a)( + countDistinct('b, 'c), + countDistinct('d), + CollectSet('b).toAggregateExpression()) + .analyze + checkRewrite(RewriteDistinctAggregates(input)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 69a3b5f278fd8..cb505ac928a70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -485,4 +485,12 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { spark.sql("select avg(a) over () from values 1.0, 2.0, 3.0 T(a)"), Row(2.0) :: Row(2.0) :: Row(2.0) :: Nil) } + + test("SPARK-17616: distinct aggregate combined with a non-partial aggregate") { + val df = Seq((1, 3, "a"), (1, 2, "b"), (3, 4, "c"), (3, 4, "c"), (3, 5, "d")) + .toDF("x", "y", "z") + checkAnswer( + df.groupBy($"x").agg(countDistinct($"y"), sort_array(collect_list($"z"))), + Seq(Row(1, 2, Seq("a", "b")), Row(3, 2, Seq("c", "c", "d")))) + } } From c2cb84165960998821c53d6a45507df639aa1425 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 22 Sep 2016 17:22:04 -0700 Subject: [PATCH 1346/1470] [SPARK-17599][SPARK-17569] Backport and to Spark 2.0 branch ## What changes were proposed in this pull request? This Backports PR #15153 and PR #15122 to Spark 2.0 branch for Structured Streaming. It is structured a bit differently because similar code paths already existed in the 2.0 branch. The unit test makes sure that both behaviors don't break. Author: Burak Yavuz Closes #15202 from brkyvz/backports-to-streaming. --- .../streaming/FileStreamSource.scala | 3 +- .../datasources/FileCatalogSuite.scala | 12 +++++ .../streaming/FileStreamSourceSuite.scala | 51 ++++++++++++++++++- 3 files changed, 64 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 0dc08b1467b14..4515f9aa6c183 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -133,7 +133,8 @@ class FileStreamSource( userSpecifiedSchema = Some(schema), className = fileFormatClassName, options = sourceOptions.optionMapWithoutPath) - Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation())) + Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation( + checkPathExist = false))) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 563f34068812c..4f12df9c49856 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -70,6 +70,18 @@ class FileCatalogSuite extends SharedSQLContext { } } + test("ListingFileCatalog: folders that don't exist don't throw exceptions") { + withTempDir { dir => + val deletedFolder = new File(dir, "deleted") + assert(!deletedFolder.exists()) + val catalog1 = new ListingFileCatalog( + spark, Seq(new Path(deletedFolder.getCanonicalPath)), Map.empty, None, + ignoreFileNotFound = true) + // doesn't throw an exception + assert(catalog1.listLeafFiles(catalog1.paths).isEmpty) + } + } + test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { class MockCatalog( override val paths: Seq[Path]) extends PartitioningAwareFileCatalog(spark, Map.empty, None) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala index c6db2fd3f908e..dfe4bb88b54b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala @@ -17,9 +17,19 @@ package org.apache.spark.sql.execution.streaming +import java.io.{File, FileNotFoundException} +import java.net.URI + +import scala.util.Random + +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} + import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.streaming.ExistsThrowsExceptionFileSystem._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType -class FileStreamSourceSuite extends SparkFunSuite { +class FileStreamSourceSuite extends SparkFunSuite with SharedSQLContext { import FileStreamSource._ @@ -73,4 +83,43 @@ class FileStreamSourceSuite extends SparkFunSuite { assert(map.isNewFile(FileEntry("b", 10))) } + testWithUninterruptibleThread("do not recheck that files exist during getBatch") { + withTempDir { temp => + spark.conf.set( + s"fs.$scheme.impl", + classOf[ExistsThrowsExceptionFileSystem].getName) + // add the metadata entries as a pre-req + val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir + val metadataLog = + new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath) + assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0)))) + + val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), + dir.getAbsolutePath, Map.empty) + // this method should throw an exception if `fs.exists` is called during resolveRelation + newSource.getBatch(None, LongOffset(1)) + } + } +} + +/** Fake FileSystem to test whether the method `fs.exists` is called during + * `DataSource.resolveRelation`. + */ +class ExistsThrowsExceptionFileSystem extends RawLocalFileSystem { + override def getUri: URI = { + URI.create(s"$scheme:///") + } + + override def exists(f: Path): Boolean = { + throw new IllegalArgumentException("Exists shouldn't have been called!") + } + + /** Simply return an empty file for now. */ + override def listStatus(file: Path): Array[FileStatus] = { + throw new FileNotFoundException("Folder was suddenly deleted but this should not make it fail!") + } +} + +object ExistsThrowsExceptionFileSystem { + val scheme = s"FileStreamSourceSuite${math.abs(Random.nextInt)}fs" } From 04141ad49806a48afccc236b699827997142bd57 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 22 Sep 2016 17:43:50 -0700 Subject: [PATCH 1347/1470] Preparing Spark release v2.0.1-rc2 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index ca6daa27614f5..6db3a599ff5ea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index c727f54d2ca3d..269b845565f1f 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index e335a89aac69e..20cf29efffc71 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 8e64f5678d78c..25cc32889ef3c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 94c75d68e34f5..37a5d09a3ff02 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6ff14d29474c2..ab287f3368a43 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e654c5479b34c..45831ce98dbcc 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 914025104d6ad..2d19e5b81cf58 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 1796c472ca59e..89e0c61a3d694 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae3746fe55ee3..8c6e22155c26c 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 6cf206d36f448..dd45935e6eec2 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9ae7968c7296e..ba97794605ee6 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 7a67142dbf07f..8f8bde7eadfb9 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 0f795ff77b5ed..f7d8ef7b87c5d 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 658c396c57f19..260969fcd7347 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index e52170acedf87..1ae1d0e216b15 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4f339fb3fa2f..a4b14f8fb9603 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 4d2df2e45b1a4..9964b22cfe39f 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0e5ce9a78887f..b5d90b151cc8f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index c9a9e73a0ddb9..f96db6588cdad 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index b92359a476268..40f2e38832fbd 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c88efda8955eb..979217e2ba8f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4375e645ca05d..0c0dd0c6069d9 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 9e3d7fbbdd565..5681b36ea9ab8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index db7f05302a12b..80ab2e01a4576 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/pom.xml b/pom.xml index f2ffb39a6b5f9..79255f9a1f827 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index afb8891168fe6..4b70d647d59e5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ad3d408b8e9b2..efa327cbf21cb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 5e67226c6a61a..347f8a1bc82b1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7624c82b8ac51..c47d5f0285169 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ae66d7eeb32bf..2388d5f6ef1d4 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 897ef824976fd..4e0f5b1013f91 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7f55ebc87f682..ddc1b091fec81 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d81ce15b8bdf3..b676b6d21a2c1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml From c393d86d188bd94b8713c4e0f0885b3adf49176e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 22 Sep 2016 17:43:58 -0700 Subject: [PATCH 1348/1470] Preparing development version 2.0.2-SNAPSHOT --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 34 files changed, 34 insertions(+), 34 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 6db3a599ff5ea..ca6daa27614f5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 269b845565f1f..c727f54d2ca3d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 20cf29efffc71..e335a89aac69e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 25cc32889ef3c..8e64f5678d78c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 37a5d09a3ff02..94c75d68e34f5 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ab287f3368a43..6ff14d29474c2 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 45831ce98dbcc..e654c5479b34c 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2d19e5b81cf58..914025104d6ad 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 89e0c61a3d694..1796c472ca59e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 8c6e22155c26c..ae3746fe55ee3 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index dd45935e6eec2..6cf206d36f448 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index ba97794605ee6..9ae7968c7296e 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 8f8bde7eadfb9..7a67142dbf07f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index f7d8ef7b87c5d..0f795ff77b5ed 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 260969fcd7347..658c396c57f19 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 1ae1d0e216b15..e52170acedf87 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4b14f8fb9603..a4f339fb3fa2f 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 9964b22cfe39f..4d2df2e45b1a4 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index b5d90b151cc8f..0e5ce9a78887f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f96db6588cdad..c9a9e73a0ddb9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 40f2e38832fbd..b92359a476268 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 979217e2ba8f2..c88efda8955eb 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 0c0dd0c6069d9..4375e645ca05d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 5681b36ea9ab8..9e3d7fbbdd565 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 80ab2e01a4576..db7f05302a12b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 79255f9a1f827..f2ffb39a6b5f9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 4b70d647d59e5..afb8891168fe6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index efa327cbf21cb..ad3d408b8e9b2 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 347f8a1bc82b1..5e67226c6a61a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c47d5f0285169..7624c82b8ac51 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 2388d5f6ef1d4..ae66d7eeb32bf 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 4e0f5b1013f91..897ef824976fd 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ddc1b091fec81..7f55ebc87f682 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index b676b6d21a2c1..d81ce15b8bdf3 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml From 22216d6bd4270095f175d9f4333afe07e07a7303 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 23 Sep 2016 09:56:40 +0800 Subject: [PATCH 1349/1470] [SPARK-17502][17609][SQL][BACKPORT][2.0] Fix Multiple Bugs in DDL Statements on Temporary Views ### What changes were proposed in this pull request? This PR is to backport https://github.com/apache/spark/pull/15054 and https://github.com/apache/spark/pull/15160 to Spark 2.0. - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example, ``` Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`'; ``` - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example, ``` Attempted to unset non-existent property 'p' in table '`testView`'; ``` - When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error: ``` ANALYZE TABLE is not supported for Project ``` - When inserting into a temporary view that is generated from `Range`, we will get the following error message: ``` assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false +- Project [1 AS 1#20] +- OneRowRelation$ ``` This PR is to fix the above four issues. There is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks. ### How was this patch tested? Added multiple test cases Author: gatorsmile Closes #15174 from gatorsmile/PR15054Backport. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 1 + .../sql/catalyst/catalog/SessionCatalog.scala | 101 ++++++++-------- .../catalog/SessionCatalogSuite.scala | 37 +++--- .../apache/spark/sql/DataFrameWriter.scala | 9 +- .../command/AnalyzeTableCommand.scala | 5 +- .../command/createDataSourceTables.scala | 13 +-- .../spark/sql/execution/command/ddl.scala | 73 +++++------- .../spark/sql/execution/command/tables.scala | 110 +++++++----------- .../spark/sql/internal/CatalogImpl.scala | 2 +- .../sql/execution/command/DDLSuite.scala | 4 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../sql/hive/execution/HiveCommandSuite.scala | 17 ++- .../sql/hive/execution/HiveDDLSuite.scala | 6 +- .../sql/hive/execution/SQLViewSuite.scala | 59 ++++++++-- 14 files changed, 219 insertions(+), 220 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 8b87a4e41c23d..790566c7659c1 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 @@ -342,6 +342,7 @@ trait CheckAnalysis extends PredicateHelper { case InsertIntoTable(t, _, _, _, _) if !t.isInstanceOf[LeafNode] || + t.isInstanceOf[Range] || t == OneRowRelation || t.isInstanceOf[LocalRelation] => failAnalysis(s"Inserting into an RDD-based table is not allowed.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index ecb4dab45ce00..f455cc90963f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -246,33 +246,26 @@ class SessionCatalog( } /** - * Retrieve the metadata of an existing metastore table. - * If no database is specified, assume the table is in the current database. - * If the specified table is not found in the database then a [[NoSuchTableException]] is thrown. + * Return whether a table/view with the specified name exists. If no database is specified, check + * with current database. + */ + def tableExists(name: TableIdentifier): Boolean = synchronized { + val db = formatDatabaseName(name.database.getOrElse(currentDb)) + val table = formatTableName(name.table) + externalCatalog.tableExists(db, table) + } + + /** + * Retrieve the metadata of an existing permanent table/view. If no database is specified, + * assume the table/view is in the current database. If the specified table/view is not found + * in the database then a [[NoSuchTableException]] is thrown. */ def getTableMetadata(name: TableIdentifier): CatalogTable = { val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase)) val table = formatTableName(name.table) - val tid = TableIdentifier(table) - if (isTemporaryTable(name)) { - CatalogTable( - identifier = tid, - tableType = CatalogTableType.VIEW, - storage = CatalogStorageFormat.empty, - schema = tempTables(table).output.map { c => - CatalogColumn( - name = c.name, - dataType = c.dataType.catalogString, - nullable = c.nullable - ) - }, - properties = Map(), - viewText = None) - } else { - requireDbExists(db) - requireTableExists(TableIdentifier(table, Some(db))) - externalCatalog.getTable(db, table) - } + requireDbExists(db) + requireTableExists(TableIdentifier(table, Some(db))) + externalCatalog.getTable(db, table) } /** @@ -368,6 +361,38 @@ class SessionCatalog( // | Methods that interact with temporary and metastore tables | // ------------------------------------------------------------- + /** + * Retrieve the metadata of an existing temporary view or permanent table/view. + * + * If a database is specified in `name`, this will return the metadata of table/view in that + * database. + * If no database is specified, this will first attempt to get the metadata of a temporary view + * with the same name, then, if that does not exist, return the metadata of table/view in the + * current database. + */ + def getTempViewOrPermanentTableMetadata(name: TableIdentifier): CatalogTable = synchronized { + val table = formatTableName(name.table) + if (name.database.isDefined) { + getTableMetadata(name) + } else { + getTempView(table).map { plan => + CatalogTable( + identifier = TableIdentifier(table), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = plan.output.map { c => + CatalogColumn( + name = c.name, + dataType = c.dataType.catalogString, + nullable = c.nullable + ) + }, + properties = Map(), + viewText = None) + }.getOrElse(getTableMetadata(name)) + } + } + /** * Rename a table. * @@ -456,24 +481,6 @@ class SessionCatalog( } } - /** - * Return whether a table with the specified name exists. - * - * Note: If a database is explicitly specified, then this will return whether the table - * exists in that particular database instead. In that case, even if there is a temporary - * table with the same name, we will return false if the specified database does not - * contain the table. - */ - def tableExists(name: TableIdentifier): Boolean = synchronized { - val db = formatDatabaseName(name.database.getOrElse(currentDb)) - val table = formatTableName(name.table) - if (isTemporaryTable(name)) { - true - } else { - externalCatalog.tableExists(db, table) - } - } - /** * Return whether a table with the specified name is a temporary table. * @@ -544,11 +551,11 @@ class SessionCatalog( tableName: TableIdentifier, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { - requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) externalCatalog.createPartitions(db, table, parts, ignoreIfExists) } @@ -560,11 +567,11 @@ class SessionCatalog( tableName: TableIdentifier, specs: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean): Unit = { - requirePartialMatchedPartitionSpec(specs, getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requirePartialMatchedPartitionSpec(specs, getTableMetadata(tableName)) externalCatalog.dropPartitions(db, table, specs, ignoreIfNotExists) } @@ -579,12 +586,12 @@ class SessionCatalog( specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { val tableMetadata = getTableMetadata(tableName) - requireExactMatchedPartitionSpec(specs, tableMetadata) - requireExactMatchedPartitionSpec(newSpecs, tableMetadata) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(specs, tableMetadata) + requireExactMatchedPartitionSpec(newSpecs, tableMetadata) externalCatalog.renamePartitions(db, table, specs, newSpecs) } @@ -598,11 +605,11 @@ class SessionCatalog( * this becomes a no-op. */ def alterPartitions(tableName: TableIdentifier, parts: Seq[CatalogTablePartition]): Unit = { - requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) externalCatalog.alterPartitions(db, table, parts) } @@ -611,11 +618,11 @@ class SessionCatalog( * If no database is specified, assume the table is in the current database. */ def getPartition(tableName: TableIdentifier, spec: TablePartitionSpec): CatalogTablePartition = { - requireExactMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName)) val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableName.table) requireDbExists(db) requireTableExists(TableIdentifier(table, Option(db))) + requireExactMatchedPartitionSpec(Seq(spec), getTableMetadata(tableName)) externalCatalog.getPartition(db, table, spec) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 574ed05333d01..399b7067f4a09 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -423,46 +423,37 @@ class SessionCatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(TableIdentifier("tbl2", Some("db1")))) // If database is explicitly specified, do not check temporary tables val tempTable = Range(1, 10, 1, 10) - catalog.createTempView("tbl3", tempTable, overrideIfExists = false) assert(!catalog.tableExists(TableIdentifier("tbl3", Some("db2")))) // If database is not explicitly specified, check the current database catalog.setCurrentDatabase("db2") assert(catalog.tableExists(TableIdentifier("tbl1"))) assert(catalog.tableExists(TableIdentifier("tbl2"))) - assert(catalog.tableExists(TableIdentifier("tbl3"))) - } - test("tableExists on temporary views") { - val catalog = new SessionCatalog(newBasicCatalog()) - val tempTable = Range(1, 10, 2, 10) - assert(!catalog.tableExists(TableIdentifier("view1"))) - assert(!catalog.tableExists(TableIdentifier("view1", Some("default")))) - catalog.createTempView("view1", tempTable, overrideIfExists = false) - assert(catalog.tableExists(TableIdentifier("view1"))) - assert(!catalog.tableExists(TableIdentifier("view1", Some("default")))) + catalog.createTempView("tbl3", tempTable, overrideIfExists = false) + // tableExists should not check temp view. + assert(!catalog.tableExists(TableIdentifier("tbl3"))) } - test("getTableMetadata on temporary views") { + test("getTempViewOrPermanentTableMetadata on temporary views") { val catalog = new SessionCatalog(newBasicCatalog()) val tempTable = Range(1, 10, 2, 10) - val m = intercept[AnalysisException] { - catalog.getTableMetadata(TableIdentifier("view1")) + intercept[NoSuchTableException] { + catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1")) }.getMessage - assert(m.contains("Table or view 'view1' not found in database 'default'")) - val m2 = intercept[AnalysisException] { - catalog.getTableMetadata(TableIdentifier("view1", Some("default"))) + intercept[NoSuchTableException] { + catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default"))) }.getMessage - assert(m2.contains("Table or view 'view1' not found in database 'default'")) catalog.createTempView("view1", tempTable, overrideIfExists = false) - assert(catalog.getTableMetadata(TableIdentifier("view1")).identifier.table == "view1") - assert(catalog.getTableMetadata(TableIdentifier("view1")).schema(0).name == "id") + assert(catalog.getTempViewOrPermanentTableMetadata( + TableIdentifier("view1")).identifier.table == "view1") + assert(catalog.getTempViewOrPermanentTableMetadata( + TableIdentifier("view1")).schema(0).name == "id") - val m3 = intercept[AnalysisException] { - catalog.getTableMetadata(TableIdentifier("view1", Some("default"))) + intercept[NoSuchTableException] { + catalog.getTempViewOrPermanentTableMetadata(TableIdentifier("view1", Some("default"))) }.getMessage - assert(m3.contains("Table or view 'view1' not found in database 'default'")) } test("list tables without pattern") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index b448b9e38b0d5..a4c4a5defa1b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -356,12 +356,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def saveAsTable(tableIdent: TableIdentifier): Unit = { - val sessionState = df.sparkSession.sessionState - val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) - val tableIdentWithDB = tableIdent.copy(database = Some(db)) - // Pass a table identifier with database part, so that `tableExists` won't check temp views - // unexpectedly. - val tableExists = sessionState.catalog.tableExists(tableIdentWithDB) + val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) (tableExists, mode) match { case (true, SaveMode.Ignore) => @@ -380,7 +375,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { mode, extraOptions.toMap, df.logicalPlan) - sessionState.executePlan(cmd).toRdd + df.sparkSession.sessionState.executePlan(cmd).toRdd } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index 9509b66ffd391..07127533b0349 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -22,6 +22,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable, SimpleCatalogRelation} @@ -38,7 +39,9 @@ case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val sessionState = sparkSession.sessionState val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) - val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) + val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentwithDB = TableIdentifier(tableIdent.table, Some(db)) + val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdentwithDB)) relation match { case relation: CatalogRelation if !relation.isInstanceOf[SimpleCatalogRelation] => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index f282d542550c1..de7d1fa0afe69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -74,15 +74,11 @@ case class CreateDataSourceTableCommand( } val sessionState = sparkSession.sessionState - val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) - val tableIdentWithDB = tableIdent.copy(database = Some(db)) - // Pass a table identifier with database part, so that `tableExists` won't check temp views - // unexpectedly. - if (sessionState.catalog.tableExists(tableIdentWithDB)) { + if (sessionState.catalog.tableExists(tableIdent)) { if (ignoreIfExists) { return Seq.empty[Row] } else { - throw new AnalysisException(s"Table ${tableIdentWithDB.unquotedString} already exists.") + throw new AnalysisException(s"Table ${tableIdent.unquotedString} already exists.") } } @@ -200,8 +196,9 @@ case class CreateDataSourceTableAsSelectCommand( // TODO: Check that options from the resolved relation match the relation that we are // inserting into (i.e. using the same compression). - EliminateSubqueryAliases( - sessionState.catalog.lookupRelation(tableIdentWithDB)) match { + // Pass a table identifier with database part, so that `tableExists` won't check temp + // views unexpectedly. + EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdentWithDB)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => // check if the file formats match l.relation match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 50ffcd4557ef8..16deee359f534 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -190,32 +190,25 @@ case class DropTableCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (!catalog.tableExists(tableName)) { - if (!ifExists) { - val objectName = if (isView) "View" else "Table" - throw new AnalysisException(s"$objectName to drop '$tableName' does not exist") - } - } else { - // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view - // issue an exception. - catalog.getTableMetadataOption(tableName).map(_.tableType match { - case CatalogTableType.VIEW if !isView => - throw new AnalysisException( - "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead") - case o if o != CatalogTableType.VIEW && isView => - throw new AnalysisException( - s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead") - case _ => - }) - try { - sparkSession.sharedState.cacheManager.uncacheQuery( - sparkSession.table(tableName.quotedString)) - } catch { - case NonFatal(e) => log.warn(e.toString, e) - } - catalog.refreshTable(tableName) - catalog.dropTable(tableName, ifExists) + // If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view + // issue an exception. + catalog.getTableMetadataOption(tableName).map(_.tableType match { + case CatalogTableType.VIEW if !isView => + throw new AnalysisException( + "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead") + case o if o != CatalogTableType.VIEW && isView => + throw new AnalysisException( + s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead") + case _ => + }) + try { + sparkSession.sharedState.cacheManager.uncacheQuery( + sparkSession.table(tableName.quotedString)) + } catch { + case NonFatal(e) => log.warn(e.toString, e) } + catalog.refreshTable(tableName) + catalog.dropTable(tableName, ifExists) Seq.empty[Row] } } @@ -275,7 +268,7 @@ case class AlterTableUnsetPropertiesCommand( propKeys.foreach { k => if (!table.properties.contains(k)) { throw new AnalysisException( - s"Attempted to unset non-existent property '$k' in table '$tableName'") + s"Attempted to unset non-existent property '$k' in table '${table.identifier}'") } } } @@ -330,11 +323,11 @@ case class AlterTableSerDePropertiesCommand( catalog.alterTable(newTable) } else { val spec = partSpec.get - val part = catalog.getPartition(tableName, spec) + val part = catalog.getPartition(table.identifier, spec) val newPart = part.copy(storage = part.storage.copy( serde = serdeClassName.orElse(part.storage.serde), serdeProperties = part.storage.serdeProperties ++ serdeProperties.getOrElse(Map()))) - catalog.alterPartitions(tableName, Seq(newPart)) + catalog.alterPartitions(table.identifier, Seq(newPart)) } Seq.empty[Row] } @@ -370,7 +363,7 @@ case class AlterTableAddPartitionCommand( // inherit table storage format (possibly except for location) CatalogTablePartition(spec, table.storage.copy(locationUri = location)) } - catalog.createPartitions(tableName, parts, ignoreIfExists = ifNotExists) + catalog.createPartitions(table.identifier, parts, ignoreIfExists = ifNotExists) Seq.empty[Row] } @@ -425,7 +418,7 @@ case class AlterTableDropPartitionCommand( throw new AnalysisException( "ALTER TABLE DROP PARTITIONS is not allowed for tables defined using the datasource API") } - catalog.dropPartitions(tableName, specs, ignoreIfNotExists = ifExists) + catalog.dropPartitions(table.identifier, specs, ignoreIfNotExists = ifExists) Seq.empty[Row] } @@ -474,25 +467,19 @@ case class AlterTableRecoverPartitionsCommand( override def run(spark: SparkSession): Seq[Row] = { val catalog = spark.sessionState.catalog - if (!catalog.tableExists(tableName)) { - throw new AnalysisException(s"Table $tableName in $cmd does not exist.") - } val table = catalog.getTableMetadata(tableName) - if (catalog.isTemporaryTable(tableName)) { - throw new AnalysisException( - s"Operation not allowed: $cmd on temporary tables: $tableName") - } + val tableIdentWithDB = table.identifier.quotedString if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - s"Operation not allowed: $cmd on datasource tables: $tableName") + s"Operation not allowed: $cmd on datasource tables: $tableIdentWithDB") } if (!DDLUtils.isTablePartitioned(table)) { throw new AnalysisException( - s"Operation not allowed: $cmd only works on partitioned tables: $tableName") + s"Operation not allowed: $cmd only works on partitioned tables: $tableIdentWithDB") } if (table.storage.locationUri.isEmpty) { - throw new AnalysisException( - s"Operation not allowed: $cmd only works on table with location provided: $tableName") + throw new AnalysisException(s"Operation not allowed: $cmd only works on table with " + + s"location provided: $tableIdentWithDB") } val root = new Path(table.storage.locationUri.get) @@ -660,7 +647,7 @@ case class AlterTableSetLocationCommand( partitionSpec match { case Some(spec) => // Partition spec is specified, so we set the location only for this partition - val part = catalog.getPartition(tableName, spec) + val part = catalog.getPartition(table.identifier, spec) val newPart = if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( @@ -669,7 +656,7 @@ case class AlterTableSetLocationCommand( } else { part.copy(storage = part.storage.copy(locationUri = Some(location))) } - catalog.alterPartitions(tableName, Seq(newPart)) + catalog.alterPartitions(table.identifier, Seq(newPart)) case None => // No partition spec is specified, so we set the location for the table itself val newTable = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 615d9dc77da4e..995feb3b670ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -78,11 +78,7 @@ case class CreateTableLikeCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (!catalog.tableExists(sourceTable)) { - throw new AnalysisException( - s"Source table in CREATE TABLE LIKE does not exist: '$sourceTable'") - } - val sourceTableDesc = catalog.getTableMetadata(sourceTable) + val sourceTableDesc = catalog.getTempViewOrPermanentTableMetadata(sourceTable) if (DDLUtils.isDatasourceTable(sourceTableDesc) || sourceTableDesc.tableType == CatalogTableType.VIEW) { @@ -188,13 +184,13 @@ case class AlterTableRenameCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - DDLUtils.verifyAlterTableType(catalog, oldName, isView) // If this is a temp view, just rename the view. // Otherwise, if this is a real table, we also need to uncache and invalidate the table. - val isTemporary = catalog.isTemporaryTable(oldName) - if (isTemporary) { + if (catalog.isTemporaryTable(oldName)) { catalog.renameTable(oldName, newName) } else { + val table = catalog.getTableMetadata(oldName) + DDLUtils.verifyAlterTableType(catalog, table.identifier, isView) // If an exception is thrown here we can just assume the table is uncached; // this can happen with Hive tables when the underlying catalog is in-memory. val wasCached = Try(sparkSession.catalog.isCached(oldName.unquotedString)).getOrElse(false) @@ -206,7 +202,6 @@ case class AlterTableRenameCommand( } } // For datasource tables, we also need to update the "path" serde property - val table = catalog.getTableMetadata(oldName) if (DDLUtils.isDatasourceTable(table) && table.tableType == CatalogTableType.MANAGED) { val newPath = catalog.defaultTablePath(newName) val newTable = table.withNewStorage( @@ -244,37 +239,34 @@ case class LoadDataCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (!catalog.tableExists(table)) { - throw new AnalysisException(s"Target table in LOAD DATA does not exist: '$table'") - } - val targetTable = catalog.getTableMetadataOption(table).getOrElse { - throw new AnalysisException(s"Target table in LOAD DATA cannot be temporary: '$table'") - } + val targetTable = catalog.getTableMetadata(table) + val tableIdentwithDB = targetTable.identifier.quotedString if (DDLUtils.isDatasourceTable(targetTable)) { - throw new AnalysisException(s"LOAD DATA is not supported for datasource tables: '$table'") + throw new AnalysisException( + s"LOAD DATA is not supported for datasource tables: '$tableIdentwithDB'") } if (targetTable.partitionColumnNames.nonEmpty) { if (partition.isEmpty) { - throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is partitioned, " + s"but no partition spec is provided") } if (targetTable.partitionColumnNames.size != partition.get.size) { - throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is partitioned, " + s"but number of columns in provided partition spec (${partition.get.size}) " + s"do not match number of partitioned columns in table " + s"(s${targetTable.partitionColumnNames.size})") } partition.get.keys.foreach { colName => if (!targetTable.partitionColumnNames.contains(colName)) { - throw new AnalysisException(s"LOAD DATA target table '$table' is partitioned, " + - s"but the specified partition spec refers to a column that is not partitioned: " + - s"'$colName'") + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is " + + s"partitioned, but the specified partition spec refers to a column that is " + + s"not partitioned: '$colName'") } } } else { if (partition.nonEmpty) { - throw new AnalysisException(s"LOAD DATA target table '$table' is not partitioned, " + - s"but a partition spec was provided.") + throw new AnalysisException(s"LOAD DATA target table '$tableIdentwithDB' is not " + + s"partitioned, but a partition spec was provided.") } } @@ -363,32 +355,26 @@ case class TruncateTableCommand( override def run(spark: SparkSession): Seq[Row] = { val catalog = spark.sessionState.catalog - if (!catalog.tableExists(tableName)) { - throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does not exist.") - } - if (catalog.isTemporaryTable(tableName)) { - throw new AnalysisException( - s"Operation not allowed: TRUNCATE TABLE on temporary tables: '$tableName'") - } val table = catalog.getTableMetadata(tableName) + val tableIdentwithDB = table.identifier.quotedString if (table.tableType == CatalogTableType.EXTERNAL) { throw new AnalysisException( - s"Operation not allowed: TRUNCATE TABLE on external tables: '$tableName'") + s"Operation not allowed: TRUNCATE TABLE on external tables: '$tableIdentwithDB'") } if (table.tableType == CatalogTableType.VIEW) { throw new AnalysisException( - s"Operation not allowed: TRUNCATE TABLE on views: '$tableName'") + s"Operation not allowed: TRUNCATE TABLE on views: '$tableIdentwithDB'") } val isDatasourceTable = DDLUtils.isDatasourceTable(table) if (isDatasourceTable && partitionSpec.isDefined) { throw new AnalysisException( s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + - s"for tables created using the data sources API: '$tableName'") + s"for tables created using the data sources API: '$tableIdentwithDB'") } if (table.partitionColumnNames.isEmpty && partitionSpec.isDefined) { throw new AnalysisException( s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported " + - s"for tables that are not partitioned: '$tableName'") + s"for tables that are not partitioned: '$tableIdentwithDB'") } val locations = if (isDatasourceTable) { @@ -396,7 +382,7 @@ case class TruncateTableCommand( } else if (table.partitionColumnNames.isEmpty) { Seq(table.storage.locationUri) } else { - catalog.listPartitions(tableName, partitionSpec).map(_.storage.locationUri) + catalog.listPartitions(table.identifier, partitionSpec).map(_.storage.locationUri) } val hadoopConf = spark.sessionState.newHadoopConf() locations.foreach { location => @@ -409,8 +395,8 @@ case class TruncateTableCommand( } catch { case NonFatal(e) => throw new AnalysisException( - s"Failed to truncate table '$tableName' when removing data of the path: $path " + - s"because of ${e.toString}") + s"Failed to truncate table '$tableIdentwithDB' when removing data of the path: " + + s"$path because of ${e.toString}") } } } @@ -419,10 +405,10 @@ case class TruncateTableCommand( spark.sessionState.refreshTable(tableName.unquotedString) // Also try to drop the contents of the table from the columnar cache try { - spark.sharedState.cacheManager.uncacheQuery(spark.table(tableName.quotedString)) + spark.sharedState.cacheManager.uncacheQuery(spark.table(table.identifier)) } catch { case NonFatal(e) => - log.warn(s"Exception when attempting to uncache table '$tableName'", e) + log.warn(s"Exception when attempting to uncache table '$tableIdentwithDB'", e) } Seq.empty[Row] } @@ -666,14 +652,16 @@ case class ShowTablePropertiesCommand(table: TableIdentifier, propertyKey: Optio * SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]; * }}} */ -case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand { +case class ShowColumnsCommand(tableName: TableIdentifier) extends RunnableCommand { // The result of SHOW COLUMNS has one column called 'result' override val output: Seq[Attribute] = { AttributeReference("result", StringType, nullable = false)() :: Nil } override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.catalog.getTableMetadata(table).schema.map { c => + val catalog = sparkSession.sessionState.catalog + val table = catalog.getTempViewOrPermanentTableMetadata(tableName) + table.schema.map { c => Row(c.name) } } @@ -695,7 +683,7 @@ case class ShowColumnsCommand(table: TableIdentifier) extends RunnableCommand { * }}} */ case class ShowPartitionsCommand( - table: TableIdentifier, + tableName: TableIdentifier, spec: Option[TablePartitionSpec]) extends RunnableCommand { // The result of SHOW PARTITIONS has one column called 'result' override val output: Seq[Attribute] = { @@ -710,13 +698,8 @@ case class ShowPartitionsCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - - if (catalog.isTemporaryTable(table)) { - throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a temporary table: ${table.unquotedString}") - } - - val tab = catalog.getTableMetadata(table) + val table = catalog.getTableMetadata(tableName) + val tableIdentWithDB = table.identifier.quotedString /** * Validate and throws an [[AnalysisException]] exception under the following conditions: @@ -724,19 +707,18 @@ case class ShowPartitionsCommand( * 2. If it is a datasource table. * 3. If it is a view. */ - if (tab.tableType == VIEW) { - throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a view: ${tab.qualifiedName}") + if (table.tableType == VIEW) { + throw new AnalysisException(s"SHOW PARTITIONS is not allowed on a view: $tableIdentWithDB") } - if (!DDLUtils.isTablePartitioned(tab)) { + if (!DDLUtils.isTablePartitioned(table)) { throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${tab.qualifiedName}") + s"SHOW PARTITIONS is not allowed on a table that is not partitioned: $tableIdentWithDB") } - if (DDLUtils.isDatasourceTable(tab)) { + if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a datasource table: ${tab.qualifiedName}") + s"SHOW PARTITIONS is not allowed on a datasource table: $tableIdentWithDB") } /** @@ -745,7 +727,7 @@ case class ShowPartitionsCommand( * thrown if the partitioning spec is invalid. */ if (spec.isDefined) { - val badColumns = spec.get.keySet.filterNot(tab.partitionColumns.map(_.name).contains) + val badColumns = spec.get.keySet.filterNot(table.partitionColumns.map(_.name).contains) if (badColumns.nonEmpty) { val badCols = badColumns.mkString("[", ", ", "]") throw new AnalysisException( @@ -753,8 +735,8 @@ case class ShowPartitionsCommand( } } - val partNames = catalog.listPartitions(table, spec).map { p => - getPartName(p.spec, tab.partitionColumnNames) + val partNames = catalog.listPartitions(tableName, spec).map { p => + getPartName(p.spec, table.partitionColumnNames) } partNames.map(Row(_)) @@ -768,16 +750,6 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - - if (catalog.isTemporaryTable(table)) { - throw new AnalysisException( - s"SHOW CREATE TABLE cannot be applied to temporary table") - } - - if (!catalog.tableExists(table)) { - throw new AnalysisException(s"Table $table doesn't exist") - } - val tableMetadata = catalog.getTableMetadata(table) val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 2067e7e39dd4c..414a4a5ed9107 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -151,7 +151,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { } private def listColumns(tableIdentifier: TableIdentifier): Dataset[Column] = { - val tableMetadata = sessionCatalog.getTableMetadata(tableIdentifier) + val tableMetadata = sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdentifier) val partitionColumnNames = tableMetadata.partitionColumnNames.toSet val bucketColumnNames = tableMetadata.bucketColumnNames.toSet val columns = tableMetadata.schema.map { c => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 3b2a660d49a71..1f5492e8a0b9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1424,7 +1424,9 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { (1 to 10).map { i => (i, i) }.toDF("a", "b").createTempView("my_temp_tab") sql(s"CREATE EXTERNAL TABLE my_ext_tab LOCATION '$path'") sql(s"CREATE VIEW my_view AS SELECT 1") - assertUnsupported("TRUNCATE TABLE my_temp_tab") + intercept[NoSuchTableException] { + sql("TRUNCATE TABLE my_temp_tab") + } assertUnsupported("TRUNCATE TABLE my_ext_tab") assertUnsupported("TRUNCATE TABLE my_view") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index b758ab00f34c5..28c8139072ead 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -509,7 +509,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert( intercept[AnalysisException] { sparkSession.catalog.createExternalTable("createdJsonTable", jsonFilePath.toString) - }.getMessage.contains("Table default.createdJsonTable already exists."), + }.getMessage.contains("Table createdJsonTable already exists."), "We should complain that createdJsonTable already exists") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 37e78b0f08aa1..446029fdc6e43 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -404,25 +404,24 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto |USING org.apache.spark.sql.parquet.DefaultSource """.stripMargin) // An empty sequence of row is returned for session temporary table. - val message1 = intercept[AnalysisException] { + intercept[NoSuchTableException] { sql("SHOW PARTITIONS parquet_temp") - }.getMessage - assert(message1.contains("is not allowed on a temporary table")) + } - val message2 = intercept[AnalysisException] { + val message1 = intercept[AnalysisException] { sql("SHOW PARTITIONS parquet_tab3") }.getMessage - assert(message2.contains("not allowed on a table that is not partitioned")) + assert(message1.contains("not allowed on a table that is not partitioned")) - val message3 = intercept[AnalysisException] { + val message2 = intercept[AnalysisException] { sql("SHOW PARTITIONS parquet_tab4 PARTITION(abcd=2015, xyz=1)") }.getMessage - assert(message3.contains("Non-partitioning column(s) [abcd, xyz] are specified")) + assert(message2.contains("Non-partitioning column(s) [abcd, xyz] are specified")) - val message4 = intercept[AnalysisException] { + val message3 = intercept[AnalysisException] { sql("SHOW PARTITIONS parquet_view1") }.getMessage - assert(message4.contains("is not allowed on a view")) + assert(message3.contains("is not allowed on a view")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 676c08b5593c2..0a6ccbed84930 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -292,7 +292,7 @@ class HiveDDLSuite sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") }.getMessage assert(message.contains( - "Attempted to unset non-existent property 'p' in table '`view1`'")) + "Attempted to unset non-existent property 'p' in table '`default`.`view1`'")) } } } @@ -664,8 +664,8 @@ class HiveDDLSuite .createTempView(sourceViewName) sql(s"CREATE TABLE $targetTabName LIKE $sourceViewName") - val sourceTable = spark.sessionState.catalog.getTableMetadata( - TableIdentifier(sourceViewName, None)) + val sourceTable = spark.sessionState.catalog.getTempViewOrPermanentTableMetadata( + TableIdentifier(sourceViewName)) val targetTable = spark.sessionState.catalog.getTableMetadata( TableIdentifier(targetTabName, Some("default"))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala index cf63537482d61..490fea27de2fc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala @@ -57,6 +57,56 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("Issue exceptions for ALTER VIEW on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + assertNoSuchTable(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") + assertNoSuchTable(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + } + } + + test("Issue exceptions for ALTER TABLE on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + assertNoSuchTable(s"ALTER TABLE $viewName SET SERDE 'whatever'") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'") + assertNoSuchTable(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')") + assertNoSuchTable(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'") + assertNoSuchTable(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") + assertNoSuchTable(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')") + assertNoSuchTable(s"ALTER TABLE $viewName RECOVER PARTITIONS") + } + } + + test("Issue exceptions for other table DDL on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + + val e = intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $viewName SELECT 1") + }.getMessage + assert(e.contains("Inserting into an RDD-based table is not allowed")) + + val testData = hiveContext.getHiveFile("data/files/employee.dat").getCanonicalPath + assertNoSuchTable(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName""") + assertNoSuchTable(s"TRUNCATE TABLE $viewName") + assertNoSuchTable(s"SHOW CREATE TABLE $viewName") + assertNoSuchTable(s"SHOW PARTITIONS $viewName") + assertNoSuchTable(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") + } + } + + private def assertNoSuchTable(query: String): Unit = { + intercept[NoSuchTableException] { + sql(query) + } + } + test("error handling: fail if the view sql itself is invalid") { // A table that does not exist intercept[AnalysisException] { @@ -177,13 +227,8 @@ class SQLViewSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("should not allow ALTER VIEW AS when the view does not exist") { - intercept[NoSuchTableException]( - sql("ALTER VIEW testView AS SELECT 1, 2") - ) - - intercept[NoSuchTableException]( - sql("ALTER VIEW default.testView AS SELECT 1, 2") - ) + assertNoSuchTable("ALTER VIEW testView AS SELECT 1, 2") + assertNoSuchTable("ALTER VIEW default.testView AS SELECT 1, 2") } test("ALTER VIEW AS should try to alter temp view first if view name has no database part") { From 54d4eee51eca364d9334141f62e0478343345d06 Mon Sep 17 00:00:00 2001 From: Gayathri Murali Date: Thu, 22 Sep 2016 22:44:20 -0700 Subject: [PATCH 1350/1470] [SPARK-16240][ML] ML persistence backward compatibility for LDA - 2.0 backport ## What changes were proposed in this pull request? Allow Spark 2.x to load instances of LDA, LocalLDAModel, and DistributedLDAModel saved from Spark 1.6. Backport of https://github.com/apache/spark/pull/15034 for branch-2.0 ## How was this patch tested? I tested this manually, saving the 3 types from 1.6 and loading them into master (2.x). In the future, we can add generic tests for testing backwards compatibility across all ML models in SPARK-15573. Author: Gayathri Murali Author: Joseph K. Bradley Closes #15205 from jkbradley/lda-backward-2.0. --- .../org/apache/spark/ml/clustering/LDA.scala | 86 +++++++++++++++---- project/MimaExcludes.scala | 3 + 2 files changed, 72 insertions(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 034f2c3fa2fd9..8e233255b4e24 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -18,6 +18,9 @@ package org.apache.spark.ml.clustering import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.json4s.JsonAST.JObject +import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.internal.Logging @@ -26,19 +29,21 @@ import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed} import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedLDAModel, EMLDAOptimizer => OldEMLDAOptimizer, LDA => OldLDA, LDAModel => OldLDAModel, LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel, OnlineLDAOptimizer => OldOnlineLDAOptimizer} import org.apache.spark.mllib.impl.PeriodicCheckpointer -import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Vector => OldVector, - Vectors => OldVectors} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.linalg.MatrixImplicits._ import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.{col, monotonically_increasing_id, udf} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.VersionUtils private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasMaxIter @@ -80,6 +85,7 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM * - Values should be >= 0 * - default = uniformly (1.0 / k), following the implementation from * [[https://github.com/Blei-Lab/onlineldavb]]. + * * @group param */ @Since("1.6.0") @@ -121,6 +127,7 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM * - Value should be >= 0 * - default = (1.0 / k), following the implementation from * [[https://github.com/Blei-Lab/onlineldavb]]. + * * @group param */ @Since("1.6.0") @@ -354,6 +361,39 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM } } +private object LDAParams { + + /** + * Equivalent to [[DefaultParamsReader.getAndSetParams()]], but handles [[LDA]] and [[LDAModel]] + * formats saved with Spark 1.6, which differ from the formats in Spark 2.0+. + * + * @param model [[LDA]] or [[LDAModel]] instance. This instance will be modified with + * [[Param]] values extracted from metadata. + * @param metadata Loaded model metadata + */ + def getAndSetParams(model: LDAParams, metadata: Metadata): Unit = { + VersionUtils.majorMinorVersion(metadata.sparkVersion) match { + case (1, 6) => + implicit val format = DefaultFormats + metadata.params match { + case JObject(pairs) => + pairs.foreach { case (paramName, jsonValue) => + val origParam = + if (paramName == "topicDistribution") "topicDistributionCol" else paramName + val param = model.getParam(origParam) + val value = param.jsonDecode(compact(render(jsonValue))) + model.set(param, value) + } + case _ => + throw new IllegalArgumentException( + s"Cannot recognize JSON metadata: ${metadata.metadataJson}.") + } + case _ => // 2.0+ + DefaultParamsReader.getAndSetParams(model, metadata) + } + } +} + /** * :: Experimental :: @@ -414,11 +454,11 @@ sealed abstract class LDAModel private[ml] ( val transformer = oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext) val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML } - dataset.withColumn($(topicDistributionCol), t(col($(featuresCol)))).toDF + dataset.withColumn($(topicDistributionCol), t(col($(featuresCol)))).toDF() } else { logWarning("LDAModel.transform was called without any output columns. Set an output column" + " such as topicDistributionCol to produce results.") - dataset.toDF + dataset.toDF() } } @@ -574,18 +614,16 @@ object LocalLDAModel extends MLReadable[LocalLDAModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) - .select("vocabSize", "topicsMatrix", "docConcentration", "topicConcentration", - "gammaShape") - .head() - val vocabSize = data.getAs[Int](0) - val topicsMatrix = data.getAs[Matrix](1) - val docConcentration = data.getAs[Vector](2) - val topicConcentration = data.getAs[Double](3) - val gammaShape = data.getAs[Double](4) + val vectorConverted = MLUtils.convertVectorColumnsToML(data, "docConcentration") + val matrixConverted = MLUtils.convertMatrixColumnsToML(vectorConverted, "topicsMatrix") + val Row(vocabSize: Int, topicsMatrix: Matrix, docConcentration: Vector, + topicConcentration: Double, gammaShape: Double) = + matrixConverted.select("vocabSize", "topicsMatrix", "docConcentration", + "topicConcentration", "gammaShape").head() val oldModel = new OldLocalLDAModel(topicsMatrix, docConcentration, topicConcentration, gammaShape) val model = new LocalLDAModel(metadata.uid, vocabSize, oldModel, sparkSession) - DefaultParamsReader.getAndSetParams(model, metadata) + LDAParams.getAndSetParams(model, metadata) model } } @@ -731,9 +769,9 @@ object DistributedLDAModel extends MLReadable[DistributedLDAModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val modelPath = new Path(path, "oldModel").toString val oldModel = OldDistributedLDAModel.load(sc, modelPath) - val model = new DistributedLDAModel( - metadata.uid, oldModel.vocabSize, oldModel, sparkSession, None) - DefaultParamsReader.getAndSetParams(model, metadata) + val model = new DistributedLDAModel(metadata.uid, oldModel.vocabSize, + oldModel, sparkSession, None) + LDAParams.getAndSetParams(model, metadata) model } } @@ -881,7 +919,7 @@ class LDA @Since("1.6.0") ( } @Since("2.0.0") -object LDA extends DefaultParamsReadable[LDA] { +object LDA extends MLReadable[LDA] { /** Get dataset for spark.mllib LDA */ private[clustering] def getOldDataset( @@ -896,6 +934,20 @@ object LDA extends DefaultParamsReadable[LDA] { } } + private class LDAReader extends MLReader[LDA] { + + private val className = classOf[LDA].getName + + override def load(path: String): LDA = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val model = new LDA(metadata.uid) + LDAParams.getAndSetParams(model, metadata) + model + } + } + + override def read: MLReader[LDA] = new LDAReader + @Since("2.0.0") override def load(path: String): LDA = super.load(path) } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c38a49ae6dae3..423cbd465ee98 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -784,6 +784,9 @@ object MimaExcludes { // SPARK-17096: Improve exception string reported through the StreamingQueryListener ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.stackTrace"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.this") + ) ++ Seq( + // SPARK-16240: ML persistence backward compatibility for LDA + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.clustering.LDA$") ) } From d3f90e71af57162afc0648adbc52b810a883ceac Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 22 Sep 2016 23:35:08 -0700 Subject: [PATCH 1351/1470] [SPARK-17640][SQL] Avoid using -1 as the default batchId for FileStreamSource.FileEntry ## What changes were proposed in this pull request? Avoid using -1 as the default batchId for FileStreamSource.FileEntry so that we can make sure not writing any FileEntry(..., batchId = -1) into the log. This also avoids people misusing it in future (#15203 is an example). ## How was this patch tested? Jenkins. Author: Shixiong Zhu Closes #15206 from zsxwing/cleanup. (cherry picked from commit 62ccf27ab4b55e734646678ae78b7e812262d14b) Signed-off-by: Shixiong Zhu --- .../streaming/FileStreamSource.scala | 37 ++++++++++--------- .../streaming/FileStreamSourceSuite.scala | 24 ++++++------ 2 files changed, 31 insertions(+), 30 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 4515f9aa6c183..8c3e7184a65b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -59,7 +59,7 @@ class FileStreamSource( val seenFiles = new SeenFilesMap(sourceOptions.maxFileAgeMs) metadataLog.allFiles().foreach { entry => - seenFiles.add(entry) + seenFiles.add(entry.path, entry.timestamp) } seenFiles.purge() @@ -73,14 +73,16 @@ class FileStreamSource( */ private def fetchMaxOffset(): LongOffset = synchronized { // All the new files found - ignore aged files and files that we have seen. - val newFiles = fetchAllFiles().filter(seenFiles.isNewFile) + val newFiles = fetchAllFiles().filter { + case (path, timestamp) => seenFiles.isNewFile(path, timestamp) + } // Obey user's setting to limit the number of files in this batch trigger. val batchFiles = if (maxFilesPerBatch.nonEmpty) newFiles.take(maxFilesPerBatch.get) else newFiles batchFiles.foreach { file => - seenFiles.add(file) + seenFiles.add(file._1, file._2) logDebug(s"New file: $file") } val numPurged = seenFiles.purge() @@ -95,7 +97,9 @@ class FileStreamSource( if (batchFiles.nonEmpty) { maxBatchId += 1 - metadataLog.add(maxBatchId, batchFiles.map(_.copy(batchId = maxBatchId)).toArray) + metadataLog.add(maxBatchId, batchFiles.map { case (path, timestamp) => + FileEntry(path = path, timestamp = timestamp, batchId = maxBatchId) + }.toArray) logInfo(s"Max batch id increased to $maxBatchId with ${batchFiles.size} new files") } @@ -140,12 +144,12 @@ class FileStreamSource( /** * Returns a list of files found, sorted by their timestamp. */ - private def fetchAllFiles(): Seq[FileEntry] = { + private def fetchAllFiles(): Seq[(String, Long)] = { val startTime = System.nanoTime val globbedPaths = SparkHadoopUtil.get.globPathIfNecessary(qualifiedBasePath) val catalog = new ListingFileCatalog(sparkSession, globbedPaths, options, Some(new StructType)) val files = catalog.allFiles().sortBy(_.getModificationTime).map { status => - FileEntry(status.getPath.toUri.toString, status.getModificationTime) + (status.getPath.toUri.toString, status.getModificationTime) } val endTime = System.nanoTime val listingTimeMs = (endTime.toDouble - startTime) / 1000000 @@ -172,10 +176,7 @@ object FileStreamSource { /** Timestamp for file modification time, in ms since January 1, 1970 UTC. */ type Timestamp = Long - val NOT_SET = -1L - - case class FileEntry(path: String, timestamp: Timestamp, batchId: Long = NOT_SET) - extends Serializable + case class FileEntry(path: String, timestamp: Timestamp, batchId: Long) extends Serializable /** * A custom hash map used to track the list of files seen. This map is not thread-safe. @@ -196,10 +197,10 @@ object FileStreamSource { private var lastPurgeTimestamp: Timestamp = 0L /** Add a new file to the map. */ - def add(file: FileEntry): Unit = { - map.put(file.path, file.timestamp) - if (file.timestamp > latestTimestamp) { - latestTimestamp = file.timestamp + def add(path: String, timestamp: Timestamp): Unit = { + map.put(path, timestamp) + if (timestamp > latestTimestamp) { + latestTimestamp = timestamp } } @@ -207,10 +208,10 @@ object FileStreamSource { * Returns true if we should consider this file a new file. The file is only considered "new" * if it is new enough that we are still tracking, and we have not seen it before. */ - def isNewFile(file: FileEntry): Boolean = { + def isNewFile(path: String, timestamp: Timestamp): Boolean = { // Note that we are testing against lastPurgeTimestamp here so we'd never miss a file that // is older than (latestTimestamp - maxAgeMs) but has not been purged yet. - file.timestamp >= lastPurgeTimestamp && !map.containsKey(file.path) + timestamp >= lastPurgeTimestamp && !map.containsKey(path) } /** Removes aged entries and returns the number of files removed. */ @@ -230,8 +231,8 @@ object FileStreamSource { def size: Int = map.size() - def allEntries: Seq[FileEntry] = { - map.entrySet().asScala.map(entry => FileEntry(entry.getKey, entry.getValue)).toSeq + def allEntries: Seq[(String, Timestamp)] = { + map.asScala.toSeq } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala index dfe4bb88b54b3..1793db0002af4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala @@ -36,51 +36,51 @@ class FileStreamSourceSuite extends SparkFunSuite with SharedSQLContext { test("SeenFilesMap") { val map = new SeenFilesMap(maxAgeMs = 10) - map.add(FileEntry("a", 5)) + map.add("a", 5) assert(map.size == 1) map.purge() assert(map.size == 1) // Add a new entry and purge should be no-op, since the gap is exactly 10 ms. - map.add(FileEntry("b", 15)) + map.add("b", 15) assert(map.size == 2) map.purge() assert(map.size == 2) // Add a new entry that's more than 10 ms than the first entry. We should be able to purge now. - map.add(FileEntry("c", 16)) + map.add("c", 16) assert(map.size == 3) map.purge() assert(map.size == 2) // Override existing entry shouldn't change the size - map.add(FileEntry("c", 25)) + map.add("c", 25) assert(map.size == 2) // Not a new file because we have seen c before - assert(!map.isNewFile(FileEntry("c", 20))) + assert(!map.isNewFile("c", 20)) // Not a new file because timestamp is too old - assert(!map.isNewFile(FileEntry("d", 5))) + assert(!map.isNewFile("d", 5)) // Finally a new file: never seen and not too old - assert(map.isNewFile(FileEntry("e", 20))) + assert(map.isNewFile("e", 20)) } test("SeenFilesMap should only consider a file old if it is earlier than last purge time") { val map = new SeenFilesMap(maxAgeMs = 10) - map.add(FileEntry("a", 20)) + map.add("a", 20) assert(map.size == 1) // Timestamp 5 should still considered a new file because purge time should be 0 - assert(map.isNewFile(FileEntry("b", 9))) - assert(map.isNewFile(FileEntry("b", 10))) + assert(map.isNewFile("b", 9)) + assert(map.isNewFile("b", 10)) // Once purge, purge time should be 10 and then b would be a old file if it is less than 10. map.purge() - assert(!map.isNewFile(FileEntry("b", 9))) - assert(map.isNewFile(FileEntry("b", 10))) + assert(!map.isNewFile("b", 9)) + assert(map.isNewFile("b", 10)) } testWithUninterruptibleThread("do not recheck that files exist during getBatch") { From 1a8ea000e7e16bdee54c47ab0f5e197c15f200a6 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Fri, 23 Sep 2016 11:37:43 -0700 Subject: [PATCH 1352/1470] [SPARK-17210][SPARKR] sparkr.zip is not distributed to executors when running sparkr in RStudio MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Spark will add sparkr.zip to archive only when it is yarn mode (SparkSubmit.scala). ``` if (args.isR && clusterManager == YARN) { val sparkRPackagePath = RUtils.localSparkRPackagePath if (sparkRPackagePath.isEmpty) { printErrorAndExit("SPARK_HOME does not exist for R application in YARN mode.") } val sparkRPackageFile = new File(sparkRPackagePath.get, SPARKR_PACKAGE_ARCHIVE) if (!sparkRPackageFile.exists()) { printErrorAndExit(s"$SPARKR_PACKAGE_ARCHIVE does not exist for R application in YARN mode.") } val sparkRPackageURI = Utils.resolveURI(sparkRPackageFile.getAbsolutePath).toString // Distribute the SparkR package. // Assigns a symbol link name "sparkr" to the shipped package. args.archives = mergeFileLists(args.archives, sparkRPackageURI + "#sparkr") // Distribute the R package archive containing all the built R packages. if (!RUtils.rPackages.isEmpty) { val rPackageFile = RPackageUtils.zipRLibraries(new File(RUtils.rPackages.get), R_PACKAGE_ARCHIVE) if (!rPackageFile.exists()) { printErrorAndExit("Failed to zip all the built R packages.") } val rPackageURI = Utils.resolveURI(rPackageFile.getAbsolutePath).toString // Assigns a symbol link name "rpkg" to the shipped package. args.archives = mergeFileLists(args.archives, rPackageURI + "#rpkg") } } ``` So it is necessary to pass spark.master from R process to JVM. Otherwise sparkr.zip won't be distributed to executor. Besides that I also pass spark.yarn.keytab/spark.yarn.principal to spark side, because JVM process need them to access secured cluster. ## How was this patch tested? Verify it manually in R Studio using the following code. ``` Sys.setenv(SPARK_HOME="/Users/jzhang/github/spark") .libPaths(c(file.path(Sys.getenv(), "R", "lib"), .libPaths())) library(SparkR) sparkR.session(master="yarn-client", sparkConfig = list(spark.executor.instances="1")) df <- as.DataFrame(mtcars) head(df) ``` … Author: Jeff Zhang Closes #14784 from zjffdu/SPARK-17210. (cherry picked from commit f62ddc5983a08d4d54c0a9a8210dd6cbec555671) Signed-off-by: Felix Cheung --- R/pkg/R/sparkR.R | 4 ++++ docs/sparkr.md | 15 +++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 06015362e6bc1..cc6d591bb2f4c 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -491,6 +491,10 @@ sparkConfToSubmitOps[["spark.driver.memory"]] <- "--driver-memory" sparkConfToSubmitOps[["spark.driver.extraClassPath"]] <- "--driver-class-path" sparkConfToSubmitOps[["spark.driver.extraJavaOptions"]] <- "--driver-java-options" sparkConfToSubmitOps[["spark.driver.extraLibraryPath"]] <- "--driver-library-path" +sparkConfToSubmitOps[["spark.master"]] <- "--master" +sparkConfToSubmitOps[["spark.yarn.keytab"]] <- "--keytab" +sparkConfToSubmitOps[["spark.yarn.principal"]] <- "--principal" + # Utility function that returns Spark Submit arguments as a string # diff --git a/docs/sparkr.md b/docs/sparkr.md index b881119731045..340e7f7cb1a0b 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -62,6 +62,21 @@ The following Spark driver properties can be set in `sparkConfig` with `sparkR.s + + + + + + + + + + + + + + + From 452e468f280d69c930782a7588a87a816cc9585a Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Sat, 24 Sep 2016 04:50:22 +0900 Subject: [PATCH 1353/1470] [SPARK-17577][CORE][2.0 BACKPORT] Update SparkContext.addFile to make it work well on Windows ## What changes were proposed in this pull request? Update ```SparkContext.addFile``` to correct the use of ```URI``` and ```Path```, then it can work well on Windows. This is used for branch-2.0 backport, more details at #15131. ## How was this patch tested? Backport, checked by appveyor. Author: Yanbo Liang Closes #15217 from yanboliang/uri-2.0. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 214758f03fdf2..251c16f657066 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1419,7 +1419,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * supported for Hadoop-supported filesystems. */ def addFile(path: String, recursive: Boolean): Unit = { - val uri = new URI(path) + val uri = new Path(path).toUri val schemeCorrectedPath = uri.getScheme match { case null | "local" => new File(path).getCanonicalFile.toURI.toString case _ => path @@ -1453,8 +1453,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli logInfo(s"Added file $path at $key with timestamp $timestamp") // Fetch the file locally so that closures which are run on the driver can still use the // SparkFiles API to access files. - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, - hadoopConfiguration, timestamp, useCache = false) + Utils.fetchFile(uri.toString, new File(SparkFiles.getRootDirectory()), conf, + env.securityManager, hadoopConfiguration, timestamp, useCache = false) postEnvironmentUpdate() } } From b111a81f2a5547e2357d66db4ba2f05ce69a52a6 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 23 Sep 2016 14:35:18 -0700 Subject: [PATCH 1354/1470] [SPARK-17651][SPARKR] Set R package version number along with mvn This PR sets the R package version while tagging releases. Note that since R doesn't accept `-SNAPSHOT` in version number field, we remove that while setting the next version Tested manually by running locally Author: Shivaram Venkataraman Closes #15223 from shivaram/sparkr-version-change. (cherry picked from commit 7c382524a959a2bc9b3d2fca44f6f0b41aba4e3c) Signed-off-by: Reynold Xin --- dev/create-release/release-tag.sh | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index d404939d1caee..b7e5100ca7408 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -60,12 +60,27 @@ git config user.email $GIT_EMAIL # Create release version $MVN versions:set -DnewVersion=$RELEASE_VERSION | grep -v "no value" # silence logs +# Set the release version in R/pkg/DESCRIPTION +sed -i".tmp1" 's/Version.*$/Version: '"$RELEASE_VERSION"'/g' R/pkg/DESCRIPTION +# Set the release version in docs +sed -i".tmp1" 's/SPARK_VERSION:.*$/SPARK_VERSION: '"$RELEASE_VERSION"'/g' docs/_config.yml +sed -i".tmp2" 's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$RELEASE_VERSION"'/g' docs/_config.yml + git commit -a -m "Preparing Spark release $RELEASE_TAG" echo "Creating tag $RELEASE_TAG at the head of $GIT_BRANCH" git tag $RELEASE_TAG # Create next version $MVN versions:set -DnewVersion=$NEXT_VERSION | grep -v "no value" # silence logs +# Remove -SNAPSHOT before setting the R version as R expects version strings to only have numbers +R_NEXT_VERSION=`echo $NEXT_VERSION | sed 's/-SNAPSHOT//g'` +sed -i".tmp2" 's/Version.*$/Version: '"$R_NEXT_VERSION"'/g' R/pkg/DESCRIPTION + +# Update docs with next version +sed -i".tmp3" 's/SPARK_VERSION:.*$/SPARK_VERSION: '"$NEXT_VERSION"'/g' docs/_config.yml +# Use R version for short version +sed -i".tmp4" 's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$R_NEXT_VERSION"'/g' docs/_config.yml + git commit -a -m "Preparing development version $NEXT_VERSION" # Push changes From 9d28cc10357a8afcfb2fa2e6eecb5c2cc2730d17 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 23 Sep 2016 14:38:07 -0700 Subject: [PATCH 1355/1470] Preparing Spark release v2.0.1-rc3 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 35 files changed, 35 insertions(+), 35 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 5a83883089e0e..3e49eac994787 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,7 +1,7 @@ Package: SparkR Type: Package Title: R Frontend for Apache Spark -Version: 2.0.0 +Version: 2.0.1 Date: 2016-08-27 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ca6daa27614f5..6db3a599ff5ea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index c727f54d2ca3d..269b845565f1f 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index e335a89aac69e..20cf29efffc71 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 8e64f5678d78c..25cc32889ef3c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 94c75d68e34f5..37a5d09a3ff02 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6ff14d29474c2..ab287f3368a43 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e654c5479b34c..45831ce98dbcc 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 914025104d6ad..2d19e5b81cf58 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 1796c472ca59e..89e0c61a3d694 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae3746fe55ee3..8c6e22155c26c 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 6cf206d36f448..dd45935e6eec2 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9ae7968c7296e..ba97794605ee6 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 7a67142dbf07f..8f8bde7eadfb9 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 0f795ff77b5ed..f7d8ef7b87c5d 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 658c396c57f19..260969fcd7347 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index e52170acedf87..1ae1d0e216b15 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4f339fb3fa2f..a4b14f8fb9603 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 4d2df2e45b1a4..9964b22cfe39f 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0e5ce9a78887f..b5d90b151cc8f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index c9a9e73a0ddb9..f96db6588cdad 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index b92359a476268..40f2e38832fbd 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c88efda8955eb..979217e2ba8f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4375e645ca05d..0c0dd0c6069d9 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 9e3d7fbbdd565..5681b36ea9ab8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index db7f05302a12b..80ab2e01a4576 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/pom.xml b/pom.xml index f2ffb39a6b5f9..79255f9a1f827 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index afb8891168fe6..4b70d647d59e5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ad3d408b8e9b2..efa327cbf21cb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 5e67226c6a61a..347f8a1bc82b1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7624c82b8ac51..c47d5f0285169 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ae66d7eeb32bf..2388d5f6ef1d4 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 897ef824976fd..4e0f5b1013f91 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7f55ebc87f682..ddc1b091fec81 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d81ce15b8bdf3..b676b6d21a2c1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml From 5bc5b49fa0a5f3d395457aceff268938317f3718 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 23 Sep 2016 14:38:13 -0700 Subject: [PATCH 1356/1470] Preparing development version 2.0.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 3e49eac994787..dfb7e2232f8f4 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,7 +1,7 @@ Package: SparkR Type: Package Title: R Frontend for Apache Spark -Version: 2.0.1 +Version: 2.0.2 Date: 2016-08-27 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 6db3a599ff5ea..ca6daa27614f5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 269b845565f1f..c727f54d2ca3d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 20cf29efffc71..e335a89aac69e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 25cc32889ef3c..8e64f5678d78c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 37a5d09a3ff02..94c75d68e34f5 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ab287f3368a43..6ff14d29474c2 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 45831ce98dbcc..e654c5479b34c 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2d19e5b81cf58..914025104d6ad 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 75c89bd318983..09c54188358ab 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.1 -SPARK_VERSION_SHORT: 2.0.1 +SPARK_VERSION: 2.0.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.0.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" MESOS_VERSION: 0.21.0 diff --git a/examples/pom.xml b/examples/pom.xml index 89e0c61a3d694..1796c472ca59e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 8c6e22155c26c..ae3746fe55ee3 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index dd45935e6eec2..6cf206d36f448 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index ba97794605ee6..9ae7968c7296e 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 8f8bde7eadfb9..7a67142dbf07f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index f7d8ef7b87c5d..0f795ff77b5ed 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 260969fcd7347..658c396c57f19 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 1ae1d0e216b15..e52170acedf87 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4b14f8fb9603..a4f339fb3fa2f 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 9964b22cfe39f..4d2df2e45b1a4 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index b5d90b151cc8f..0e5ce9a78887f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f96db6588cdad..c9a9e73a0ddb9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 40f2e38832fbd..b92359a476268 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 979217e2ba8f2..c88efda8955eb 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 0c0dd0c6069d9..4375e645ca05d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 5681b36ea9ab8..9e3d7fbbdd565 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 80ab2e01a4576..db7f05302a12b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 79255f9a1f827..f2ffb39a6b5f9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 4b70d647d59e5..afb8891168fe6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index efa327cbf21cb..ad3d408b8e9b2 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 347f8a1bc82b1..5e67226c6a61a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c47d5f0285169..7624c82b8ac51 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 2388d5f6ef1d4..ae66d7eeb32bf 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 4e0f5b1013f91..897ef824976fd 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ddc1b091fec81..7f55ebc87f682 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index b676b6d21a2c1..d81ce15b8bdf3 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml From 9e91a1009e6f916245b4d4018de1664ea3decfe7 Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Fri, 23 Sep 2016 14:59:53 -0700 Subject: [PATCH 1357/1470] [SPARK-15703][SCHEDULER][CORE][WEBUI] Make ListenerBus event queue size configurable (branch 2.0) ## What changes were proposed in this pull request? Backport #14269 to 2.0. ## How was this patch tested? Jenkins. Author: Dhruve Ashar Closes #15222 from zsxwing/SPARK-15703-2.0. --- .../scala/org/apache/spark/SparkContext.scala | 4 +-- .../spark/internal/config/package.scala | 5 ++++ .../spark/scheduler/LiveListenerBus.scala | 23 ++++++++------ .../scheduler/EventLoggingListenerSuite.scala | 4 +-- .../spark/scheduler/SparkListenerSuite.scala | 30 ++++++++++--------- .../BlockManagerReplicationSuite.scala | 9 ++++-- .../spark/storage/BlockManagerSuite.scala | 6 ++-- .../spark/ui/storage/StorageTabSuite.scala | 11 +++---- .../streaming/ReceivedBlockHandlerSuite.scala | 5 +++- 9 files changed, 60 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 251c16f657066..ffd12273bf1c2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -249,7 +249,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def isStopped: Boolean = stopped.get() // An asynchronous listener bus for Spark events - private[spark] val listenerBus = new LiveListenerBus + private[spark] val listenerBus = new LiveListenerBus(this) // This function allows components created by SparkEnv to be mocked in unit tests: private[spark] def createSparkEnv( @@ -2154,7 +2154,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } - listenerBus.start(this) + listenerBus.start() _listenerBusStarted = true } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index f28a9a5cf81d6..29f812a2ce116 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -112,4 +112,9 @@ package object config { // To limit how many applications are shown in the History Server summary ui private[spark] val HISTORY_UI_MAX_APPS = ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE) + + private[spark] val LISTENER_BUS_EVENT_QUEUE_SIZE = + ConfigBuilder("spark.scheduler.listenerbus.eventqueue.size") + .intConf + .createWithDefault(10000) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 1c21313d1cb17..bfa3c408f2284 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -22,7 +22,8 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.util.DynamicVariable -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** @@ -32,18 +33,24 @@ import org.apache.spark.util.Utils * has started will events be actually propagated to all attached listeners. This listener bus * is stopped when `stop()` is called, and it will drop further events after stopping. */ -private[spark] class LiveListenerBus extends SparkListenerBus { +private[spark] class LiveListenerBus(val sparkContext: SparkContext) extends SparkListenerBus { self => import LiveListenerBus._ - private var sparkContext: SparkContext = null - // Cap the capacity of the event queue so we get an explicit error (rather than // an OOM exception) if it's perpetually being added to more quickly than it's being drained. - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private lazy val EVENT_QUEUE_CAPACITY = validateAndGetQueueSize() + private lazy val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + + private def validateAndGetQueueSize(): Int = { + val queueSize = sparkContext.conf.get(LISTENER_BUS_EVENT_QUEUE_SIZE) + if (queueSize <= 0) { + throw new SparkException("spark.scheduler.listenerbus.eventqueue.size must be > 0!") + } + queueSize + } // Indicate if `start()` is called private val started = new AtomicBoolean(false) @@ -96,11 +103,9 @@ private[spark] class LiveListenerBus extends SparkListenerBus { * listens for any additional events asynchronously while the listener bus is still running. * This should only be called once. * - * @param sc Used to stop the SparkContext in case the listener thread dies. */ - def start(sc: SparkContext): Unit = { + def start(): Unit = { if (started.compareAndSet(false, true)) { - sparkContext = sc listenerThread.start() } else { throw new IllegalStateException(s"$name already started!") 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 c4c80b5b57daa..7f4859206e257 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -142,14 +142,14 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit extraConf.foreach { case (k, v) => conf.set(k, v) } val logName = compressionCodec.map("test-" + _).getOrElse("test") val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) - val listenerBus = new LiveListenerBus + val listenerBus = new LiveListenerBus(sc) val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite eventLogger.start() - listenerBus.start(sc) + listenerBus.start() listenerBus.addListener(eventLogger) listenerBus.postToAll(applicationStart) listenerBus.postToAll(applicationEnd) 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 5ba67afc0cd62..e8a88d4909a83 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -37,13 +37,13 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val jobCompletionTime = 1421191296660L test("don't call sc.stop in listener") { - sc = new SparkContext("local", "SparkListenerSuite") + sc = new SparkContext("local", "SparkListenerSuite", new SparkConf()) val listener = new SparkContextStoppingListener(sc) - val bus = new LiveListenerBus + val bus = new LiveListenerBus(sc) bus.addListener(listener) // Starting listener bus should flush all buffered events - bus.start(sc) + bus.start() bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) @@ -52,8 +52,9 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match } test("basic creation and shutdown of LiveListenerBus") { + sc = new SparkContext("local", "SparkListenerSuite", new SparkConf()) val counter = new BasicJobCounter - val bus = new LiveListenerBus + val bus = new LiveListenerBus(sc) bus.addListener(counter) // Listener bus hasn't started yet, so posting events should not increment counter @@ -61,7 +62,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match assert(counter.count === 0) // Starting listener bus should flush all buffered events - bus.start(sc) + bus.start() bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(counter.count === 5) @@ -72,14 +73,14 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // Listener bus must not be started twice intercept[IllegalStateException] { - val bus = new LiveListenerBus - bus.start(sc) - bus.start(sc) + val bus = new LiveListenerBus(sc) + bus.start() + bus.start() } // ... or stopped before starting intercept[IllegalStateException] { - val bus = new LiveListenerBus + val bus = new LiveListenerBus(sc) bus.stop() } } @@ -106,12 +107,12 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match drained = true } } - - val bus = new LiveListenerBus + sc = new SparkContext("local", "SparkListenerSuite", new SparkConf()) + val bus = new LiveListenerBus(sc) val blockingListener = new BlockingListener bus.addListener(blockingListener) - bus.start(sc) + bus.start() bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) listenerStarted.acquire() @@ -353,13 +354,14 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val badListener = new BadListener val jobCounter1 = new BasicJobCounter val jobCounter2 = new BasicJobCounter - val bus = new LiveListenerBus + sc = new SparkContext("local", "SparkListenerSuite", new SparkConf()) + val bus = new LiveListenerBus(sc) // Propagate events to bad listener first bus.addListener(badListener) bus.addListener(jobCounter1) bus.addListener(jobCounter2) - bus.start(sc) + bus.start() // Post events to all listeners, and wait until the queue is drained (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } 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 31687e6147314..b9e3a364ee221 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -38,7 +38,10 @@ import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage.StorageLevel._ /** Testsuite that tests block replication in BlockManager */ -class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with BeforeAndAfter { +class BlockManagerReplicationSuite extends SparkFunSuite + with Matchers + with BeforeAndAfter + with LocalSparkContext { private val conf = new SparkConf(false).set("spark.app.id", "test") private var rpcEnv: RpcEnv = null @@ -91,8 +94,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") + sc = new SparkContext("local", "test", conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(sc))), conf, true) allStores.clear() } 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 6194d23f4d8da..e93eee273f16d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -49,7 +49,7 @@ import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach - with PrivateMethodTester with ResetSystemProperties { + with PrivateMethodTester with LocalSparkContext with ResetSystemProperties { import BlockManagerSuite._ @@ -107,8 +107,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) + sc = new SparkContext("local", "test", conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(sc))), conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() 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 411a0ddebeb77..f6c8418ba3ac4 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 @@ -19,15 +19,14 @@ package org.apache.spark.ui.storage import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkFunSuite, Success} -import org.apache.spark.executor.TaskMetrics +import org.apache.spark._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ /** * Test various functionality in the StorageListener that supports the StorageTab. */ -class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { +class StorageTabSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter { private var bus: LiveListenerBus = _ private var storageStatusListener: StorageStatusListener = _ private var storageListener: StorageListener = _ @@ -43,8 +42,10 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { private val bm1 = BlockManagerId("big", "dog", 1) before { - bus = new LiveListenerBus - storageStatusListener = new StorageStatusListener(new SparkConf()) + val conf = new SparkConf() + sc = new SparkContext("local", "test", conf) + bus = new LiveListenerBus(sc) + storageStatusListener = new StorageStatusListener(conf) storageListener = new StorageListener(storageStatusListener) bus.addListener(storageStatusListener) bus.addListener(storageListener) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 5d362925a8746..7e665454a5400 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -48,6 +48,7 @@ class ReceivedBlockHandlerSuite extends SparkFunSuite with BeforeAndAfter with Matchers + with LocalSparkContext with Logging { import WriteAheadLogBasedBlockHandler._ @@ -78,8 +79,10 @@ class ReceivedBlockHandlerSuite rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) + sc = new SparkContext("local", "test", conf) blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(sc))), conf, true) storageLevel = StorageLevel.MEMORY_ONLY_SER blockManager = createBlockManager(blockManagerSize, conf) From ed545763adc3f50569581c9b017b396e8997ac31 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 24 Sep 2016 08:06:41 +0100 Subject: [PATCH 1358/1470] [SPARK-10835][ML] Word2Vec should accept non-null string array, in addition to existing null string array ## What changes were proposed in this pull request? To match Tokenizer and for compatibility with Word2Vec, output a nullable string array type in NGram ## How was this patch tested? Jenkins tests. Author: Sean Owen Closes #15179 from srowen/SPARK-10835. (cherry picked from commit f3fe55439e4c865c26502487a1bccf255da33f4a) Signed-off-by: Sean Owen --- .../apache/spark/ml/feature/Word2Vec.scala | 3 ++- .../spark/ml/feature/Word2VecSuite.scala | 21 +++++++++++++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 14c05123c62ed..d53f3df514dff 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -108,7 +108,8 @@ private[feature] trait Word2VecBase extends Params * Validate and transform the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), new ArrayType(StringType, true)) + val typeCandidates = List(new ArrayType(StringType, true), new ArrayType(StringType, false)) + SchemaUtils.checkColumnTypes(schema, $(inputCol), typeCandidates) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index 16c74f6785875..c8f1311538956 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -207,5 +207,26 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul val newInstance = testDefaultReadWrite(instance) assert(newInstance.getVectors.collect() === instance.getVectors.collect()) } + + test("Word2Vec works with input that is non-nullable (NGram)") { + val spark = this.spark + import spark.implicits._ + + val sentence = "a q s t q s t b b b s t m s t m q " + val docDF = sc.parallelize(Seq(sentence, sentence)).map(_.split(" ")).toDF("text") + + val ngram = new NGram().setN(2).setInputCol("text").setOutputCol("ngrams") + val ngramDF = ngram.transform(docDF) + + val model = new Word2Vec() + .setVectorSize(2) + .setInputCol("ngrams") + .setOutputCol("result") + .fit(ngramDF) + + // Just test that this transformation succeeds + model.transform(ngramDF).collect() + } + } From 88ba2e1d0492039ee2cb1caa16160ec24bea3992 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 25 Sep 2016 22:57:31 -0700 Subject: [PATCH 1359/1470] [SPARK-17650] malformed url's throw exceptions before bricking Executors ## What changes were proposed in this pull request? When a malformed URL was sent to Executors through `sc.addJar` and `sc.addFile`, the executors become unusable, because they constantly throw `MalformedURLException`s and can never acknowledge that the file or jar is just bad input. This PR tries to fix that problem by making sure MalformedURLs can never be submitted through `sc.addJar` and `sc.addFile`. Another solution would be to blacklist bad files and jars on Executors. Maybe fail the first time, and then ignore the second time (but print a warning message). ## How was this patch tested? Unit tests in SparkContextSuite Author: Burak Yavuz Closes #15224 from brkyvz/SPARK-17650. (cherry picked from commit 59d87d24079bc633e63ce032f0a5ddd18a3b02cb) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/SparkContext.scala | 16 ++++++++------ .../scala/org/apache/spark/util/Utils.scala | 20 +++++++++++++++++ .../org/apache/spark/SparkContextSuite.scala | 22 +++++++++++++++++++ 3 files changed, 51 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ffd12273bf1c2..e9f9d7242618a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -19,7 +19,7 @@ package org.apache.spark import java.io._ import java.lang.reflect.Constructor -import java.net.URI +import java.net.{MalformedURLException, URI} import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID} import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference} @@ -36,10 +36,8 @@ import com.google.common.collect.MapMaker import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, - FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} -import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, - TextInputFormat} +import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, TextInputFormat} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary @@ -47,8 +45,7 @@ import org.apache.mesos.MesosNativeLibrary import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} -import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, - WholeTextFileInputFormat} +import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} @@ -1441,6 +1438,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli throw new SparkException(s"Added file $hadoopPath is a directory and recursive is not " + "turned on.") } + } else { + // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies + Utils.validateURL(uri) } val key = if (!isLocal && scheme == "file") { @@ -1700,6 +1700,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli key = env.rpcEnv.fileServer.addJar(new File(path)) } else { val uri = new URI(path) + // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies + Utils.validateURL(uri) key = uri.getScheme match { // A JAR file which exists only on the driver node case null | "file" => 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 a0ef30e16bd5c..7764fdc8ccad4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -697,6 +697,26 @@ private[spark] object Utils extends Logging { } } + /** + * Validate that a given URI is actually a valid URL as well. + * @param uri The URI to validate + */ + @throws[MalformedURLException]("when the URI is an invalid URL") + def validateURL(uri: URI): Unit = { + Option(uri.getScheme).getOrElse("file") match { + case "http" | "https" | "ftp" => + try { + uri.toURL + } catch { + case e: MalformedURLException => + val ex = new MalformedURLException(s"URI (${uri.toString}) is not a valid URL.") + ex.initCause(e) + throw ex + } + case _ => // will not be turned into a URL anyway + } + } + /** * Get the path of a temporary directory. Spark's local directories can be configured through * multiple settings, which are used with the following precedence: diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index f8d143dc610cb..c451c596b069a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import java.io.File +import java.net.MalformedURLException import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit @@ -173,6 +174,27 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { } } + test("SPARK-17650: malformed url's throw exceptions before bricking Executors") { + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + Seq("http", "https", "ftp").foreach { scheme => + val badURL = s"$scheme://user:pwd/path" + val e1 = intercept[MalformedURLException] { + sc.addFile(badURL) + } + assert(e1.getMessage.contains(badURL)) + val e2 = intercept[MalformedURLException] { + sc.addJar(badURL) + } + assert(e2.getMessage.contains(badURL)) + assert(sc.addedFiles.isEmpty) + assert(sc.addedJars.isEmpty) + } + } finally { + sc.stop() + } + } + test("addFile recursive works") { val pluto = Utils.createTempDir() val neptune = Utils.createTempDir(pluto.getAbsolutePath) From cf5324127856381c40ba952e35bdb99a717163fa Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 26 Sep 2016 10:44:35 -0700 Subject: [PATCH 1360/1470] [SPARK-17649][CORE] Log how many Spark events got dropped in LiveListenerBus ## What changes were proposed in this pull request? Log how many Spark events got dropped in LiveListenerBus so that the user can get insights on how to set a correct event queue size. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15220 from zsxwing/SPARK-17649. (cherry picked from commit bde85f8b70138a51052b613664facbc981378c38) Signed-off-by: Shixiong Zhu --- .../spark/scheduler/LiveListenerBus.scala | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index bfa3c408f2284..5533f7b1f2363 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import java.util.concurrent._ -import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} import scala.util.DynamicVariable @@ -57,6 +57,12 @@ private[spark] class LiveListenerBus(val sparkContext: SparkContext) extends Spa // Indicate if `stop()` is called private val stopped = new AtomicBoolean(false) + /** A counter for dropped events. It will be reset every time we log it. */ + private val droppedEventsCounter = new AtomicLong(0L) + + /** When `droppedEventsCounter` was logged last time in milliseconds. */ + @volatile private var lastReportTimestamp = 0L + // Indicate if we are processing some event // Guarded by `self` private var processingEvent = false @@ -123,6 +129,24 @@ private[spark] class LiveListenerBus(val sparkContext: SparkContext) extends Spa eventLock.release() } else { onDropEvent(event) + droppedEventsCounter.incrementAndGet() + } + + val droppedEvents = droppedEventsCounter.get + if (droppedEvents > 0) { + // Don't log too frequently + if (System.currentTimeMillis() - lastReportTimestamp >= 60 * 1000) { + // There may be multiple threads trying to decrease droppedEventsCounter. + // Use "compareAndSet" to make sure only one thread can win. + // And if another thread is increasing droppedEventsCounter, "compareAndSet" will fail and + // then that thread will update it. + if (droppedEventsCounter.compareAndSet(droppedEvents, 0)) { + val prevLastReportTimestamp = lastReportTimestamp + lastReportTimestamp = System.currentTimeMillis() + logWarning(s"Dropped $droppedEvents SparkListenerEvents since " + + new java.util.Date(prevLastReportTimestamp)) + } + } } } From 8a58f2e8ec413591ec00da1e37b91b1bf49e4d1d Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Mon, 26 Sep 2016 13:21:08 -0700 Subject: [PATCH 1361/1470] [SPARK-17652] Fix confusing exception message while reserving capacity ## What changes were proposed in this pull request? This minor patch fixes a confusing exception message while reserving additional capacity in the vectorized parquet reader. ## How was this patch tested? Exisiting Unit Tests Author: Sameer Agarwal Closes #15225 from sameeragarwal/error-msg. (cherry picked from commit 7c7586aef9243081d02ea5065435234b5950ab66) Signed-off-by: Yin Huai --- .../sql/execution/vectorized/ColumnVector.java | 14 +++++++------- .../execution/vectorized/ColumnarBatchSuite.scala | 4 ++-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index 59173d253b298..9b0ed802fc9d2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -285,19 +285,19 @@ public void reserve(int requiredCapacity) { try { reserveInternal(newCapacity); } catch (OutOfMemoryError outOfMemoryError) { - throwUnsupportedException(newCapacity, requiredCapacity, outOfMemoryError); + throwUnsupportedException(requiredCapacity, outOfMemoryError); } } else { - throwUnsupportedException(newCapacity, requiredCapacity, null); + throwUnsupportedException(requiredCapacity, null); } } } - private void throwUnsupportedException(int newCapacity, int requiredCapacity, Throwable cause) { - String message = "Cannot reserve more than " + newCapacity + - " bytes in the vectorized reader (requested = " + requiredCapacity + " bytes). As a" + - " workaround, you can disable the vectorized reader by setting " - + SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key() + " to false."; + private void throwUnsupportedException(int requiredCapacity, Throwable cause) { + String message = "Cannot reserve additional contiguous bytes in the vectorized reader " + + "(requested = " + requiredCapacity + " bytes). As a workaround, you can disable the " + + "vectorized reader by setting " + SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key() + + " to false."; if (cause != null) { throw new RuntimeException(message, cause); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 100cc4daca875..e3943f31a48ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -802,8 +802,8 @@ class ColumnarBatchSuite extends SparkFunSuite { // Over-allocating beyond MAX_CAPACITY throws an exception column.appendBytes(10, 0.toByte) } - assert(ex.getMessage.contains(s"Cannot reserve more than ${column.MAX_CAPACITY} bytes in " + - s"the vectorized reader")) + assert(ex.getMessage.contains(s"Cannot reserve additional contiguous bytes in the " + + s"vectorized reader")) } } } From f4594900d86bb39358ff19047dfa8c1e4b78aa6b Mon Sep 17 00:00:00 2001 From: Andrew Mills Date: Mon, 26 Sep 2016 16:41:10 -0400 Subject: [PATCH 1362/1470] [Docs] Update spark-standalone.md to fix link Corrected a link to the configuration.html page, it was pointing to a page that does not exist (configurations.html). Documentation change, verified in preview. Author: Andrew Mills Closes #15244 from ammills01/master. (cherry picked from commit 00be16df642317137f17d2d7d2887c41edac3680) Signed-off-by: Andrew Or --- docs/spark-standalone.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 5ae63fe4e6e07..6f0f665c82e19 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -348,7 +348,7 @@ Learn more about getting started with ZooKeeper [here](http://zookeeper.apache.o **Configuration** In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env by configuring `spark.deploy.recoveryMode` and related spark.deploy.zookeeper.* configurations. -For more information about these configurations please refer to the configurations (doc)[configurations.html#deploy] +For more information about these configurations please refer to the [configuration doc](configuration.html#deploy) Possible gotcha: If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently). From 98bbc4410181741d903a703eac289408cb5b2c5e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 27 Sep 2016 14:14:27 -0700 Subject: [PATCH 1363/1470] [SPARK-17618] Guard against invalid comparisons between UnsafeRow and other formats This patch ports changes from #15185 to Spark 2.x. In that patch, a correctness bug in Spark 1.6.x which was caused by an invalid `equals()` comparison between an `UnsafeRow` and another row of a different format. Spark 2.x is not affected by that specific correctness bug but it can still reap the error-prevention benefits of that patch's changes, which modify ``UnsafeRow.equals()` to throw an IllegalArgumentException if it is called with an object that is not an `UnsafeRow`. Author: Josh Rosen Closes #15265 from JoshRosen/SPARK-17618-master. (cherry picked from commit 2f84a686604b298537bfd4d087b41594d2aa7ec6) Signed-off-by: Josh Rosen --- .../apache/spark/sql/catalyst/expressions/UnsafeRow.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index dd2f39eb816f2..9027652d57f14 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -31,6 +31,7 @@ import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; +import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; @@ -577,8 +578,12 @@ public boolean equals(Object other) { return (sizeInBytes == o.sizeInBytes) && ByteArrayMethods.arrayEquals(baseObject, baseOffset, o.baseObject, o.baseOffset, sizeInBytes); + } else if (!(other instanceof InternalRow)) { + return false; + } else { + throw new IllegalArgumentException( + "Cannot compare UnsafeRow to " + other.getClass().getName()); } - return false; } /** From 2cd327ef5e4c3f6b8468ebb2352479a1686b7888 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 27 Sep 2016 16:00:39 -0700 Subject: [PATCH 1364/1470] [SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in MemoryStore ## What changes were proposed in this pull request? There is an assert in MemoryStore's putIteratorAsValues method which is used to check if unroll memory is not released too much. This assert looks wrong. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Closes #14642 from viirya/fix-unroll-memory. (cherry picked from commit e7bce9e1876de6ee975ccc89351db58119674aef) Signed-off-by: Josh Rosen --- .../scala/org/apache/spark/storage/memory/MemoryStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 161434cb8ce37..9b87c4236aa52 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -271,7 +271,7 @@ private[spark] class MemoryStore( blockId, Utils.bytesToString(size), Utils.bytesToString(maxMemory - blocksMemoryUsed))) Right(size) } else { - assert(currentUnrollMemoryForThisTask >= currentUnrollMemoryForThisTask, + assert(currentUnrollMemoryForThisTask >= unrollMemoryUsedByThisBlock, "released too much unroll memory") Left(new PartiallyUnrolledIterator( this, From 1b02f8820ddaf3f2a0e7acc9a7f27afc20683cca Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 28 Sep 2016 00:59:00 -0700 Subject: [PATCH 1365/1470] [SPARK-17666] Ensure that RecordReaders are closed by data source file scans (backport) This is a branch-2.0 backport of #15245. ## What changes were proposed in this pull request? This patch addresses a potential cause of resource leaks in data source file scans. As reported in [SPARK-17666](https://issues.apache.org/jira/browse/SPARK-17666), tasks which do not fully-consume their input may cause file handles / network connections (e.g. S3 connections) to be leaked. Spark's `NewHadoopRDD` uses a TaskContext callback to [close its record readers](https://github.com/apache/spark/blame/master/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L208), but the new data source file scans will only close record readers once their iterators are fully-consumed. This patch modifies `RecordReaderIterator` and `HadoopFileLinesReader` to add `close()` methods and modifies all six implementations of `FileFormat.buildReader()` to register TaskContext task completion callbacks to guarantee that cleanup is eventually performed. ## How was this patch tested? Tested manually for now. Author: Josh Rosen Closes #15271 from JoshRosen/SPARK-17666-backport. --- .../ml/source/libsvm/LibSVMRelation.scala | 7 +++++-- .../datasources/HadoopFileLinesReader.scala | 6 +++++- .../datasources/RecordReaderIterator.scala | 21 +++++++++++++++++-- .../datasources/csv/CSVFileFormat.scala | 5 ++++- .../datasources/json/JsonFileFormat.scala | 5 ++++- .../parquet/ParquetFileFormat.scala | 3 ++- .../datasources/text/TextFileFormat.scala | 2 ++ .../spark/sql/hive/orc/OrcFileFormat.scala | 6 +++++- 8 files changed, 46 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 034223e115389..ac95b9272f1d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.io.{NullWritable, Text} import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat +import org.apache.spark.TaskContext import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.mllib.util.MLUtils @@ -160,8 +161,10 @@ private[libsvm] class LibSVMFileFormat extends TextBasedFileFormat with DataSour sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) (file: PartitionedFile) => { - val points = - new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value) + val linesReader = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) + + val points = linesReader .map(_.toString.trim) .filterNot(line => line.isEmpty || line.startsWith("#")) .map { line => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala index 18f9b55895a64..83cf26c63a175 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources +import java.io.Closeable import java.net.URI import org.apache.hadoop.conf.Configuration @@ -30,7 +31,8 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl * An adaptor from a [[PartitionedFile]] to an [[Iterator]] of [[Text]], which are all of the lines * in that file. */ -class HadoopFileLinesReader(file: PartitionedFile, conf: Configuration) extends Iterator[Text] { +class HadoopFileLinesReader( + file: PartitionedFile, conf: Configuration) extends Iterator[Text] with Closeable { private val iterator = { val fileSplit = new FileSplit( new Path(new URI(file.filePath)), @@ -48,4 +50,6 @@ class HadoopFileLinesReader(file: PartitionedFile, conf: Configuration) extends override def hasNext: Boolean = iterator.hasNext override def next(): Text = iterator.next() + + override def close(): Unit = iterator.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala index f03ae94d55838..938af25a96844 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/RecordReaderIterator.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources +import java.io.Closeable + import org.apache.hadoop.mapreduce.RecordReader import org.apache.spark.sql.catalyst.InternalRow @@ -27,7 +29,8 @@ import org.apache.spark.sql.catalyst.InternalRow * Note that this returns [[Object]]s instead of [[InternalRow]] because we rely on erasure to pass * column batches by pretending they are rows. */ -class RecordReaderIterator[T](rowReader: RecordReader[_, T]) extends Iterator[T] { +class RecordReaderIterator[T]( + private[this] var rowReader: RecordReader[_, T]) extends Iterator[T] with Closeable { private[this] var havePair = false private[this] var finished = false @@ -38,7 +41,7 @@ class RecordReaderIterator[T](rowReader: RecordReader[_, T]) extends Iterator[T] // Close and release the reader here; close() will also be called when the task // completes, but for tasks that read from many files, it helps to release the // resources early. - rowReader.close() + close() } havePair = !finished } @@ -52,4 +55,18 @@ class RecordReaderIterator[T](rowReader: RecordReader[_, T]) extends Iterator[T] havePair = false rowReader.getCurrentValue } + + override def close(): Unit = { + if (rowReader != null) { + try { + // Close the reader and release it. Note: it's very important that we don't close the + // reader more than once, since that exposes us to MAPREDUCE-5918 when running against + // older Hadoop 2.x releases. That bug can lead to non-deterministic corruption issues + // when reading compressed input. + rowReader.close() + } finally { + rowReader = null + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 4a60f51c47b48..107b6007ce467 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce._ +import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow @@ -111,7 +112,9 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { (file: PartitionedFile) => { val lineIterator = { val conf = broadcastedHadoopConf.value.value - new HadoopFileLinesReader(file, conf).map { line => + val linesReader = new HadoopFileLinesReader(file, conf) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) + linesReader.map { line => new String(line.getBytes, 0, line.getLength, csvOptions.charset) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index decbddaac2f59..cba3255d86f42 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -106,7 +107,9 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { .getOrElse(sparkSession.sessionState.conf.columnNameOfCorruptRecord) (file: PartitionedFile) => { - val lines = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map(_.toString) + val linesReader = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) + val lines = linesReader.map(_.toString) JacksonParser.parseJson( lines, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 94980886c6265..aef0f1b4157f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -37,7 +37,7 @@ import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType import org.slf4j.bridge.SLF4JBridgeHandler -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -387,6 +387,7 @@ class ParquetFileFormat } val iter = new RecordReaderIterator(parquetReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. if (parquetReader.isInstanceOf[VectorizedParquetRecordReader] && diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index abb6059f75ba8..6aa078af3ea19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.io.{NullWritable, Text} import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat +import org.apache.spark.TaskContext import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow @@ -101,6 +102,7 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { (file: PartitionedFile) => { val reader = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => reader.close())) if (requiredSchema.isEmpty) { val emptyUnsafeRow = new UnsafeRow(0) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index d15fb8480666b..fc126b3616971 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.{InputFormat => MapRedInputFormat, JobConf, Outp import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit} +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.sql.{Row, SparkSession} @@ -150,12 +151,15 @@ class OrcFileFormat new SparkOrcNewRecordReader(orcReader, conf, fileSplit.getStart, fileSplit.getLength) } + val recordsIterator = new RecordReaderIterator[OrcStruct](orcRecordReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => recordsIterator.close())) + // Unwraps `OrcStruct`s to `UnsafeRow`s OrcRelation.unwrapOrcStructs( conf, requiredSchema, Some(orcRecordReader.getObjectInspector.asInstanceOf[StructObjectInspector]), - new RecordReaderIterator[OrcStruct](orcRecordReader)) + recordsIterator) } } } From 4d73d5cd82ebc980f996c78f9afb8a97418ab7ab Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 28 Sep 2016 06:19:04 -0400 Subject: [PATCH 1366/1470] [MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation ## What changes were proposed in this pull request? This PR proposes to fix wrongly indented examples in PySpark documentation ``` - >>> json_sdf = spark.readStream.format("json")\ - .schema(sdf_schema)\ - .load(tempfile.mkdtemp()) + >>> json_sdf = spark.readStream.format("json") \\ + ... .schema(sdf_schema) \\ + ... .load(tempfile.mkdtemp()) ``` ``` - people.filter(people.age > 30).join(department, people.deptId == department.id)\ + people.filter(people.age > 30).join(department, people.deptId == department.id) \\ ``` ``` - >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \ - LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), + ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] ``` ``` - >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \ - LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), + ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] ``` ``` - ... for x in iterator: - ... print(x) + ... for x in iterator: + ... print(x) ``` ## How was this patch tested? Manually tested. **Before** ![2016-09-26 8 36 02](https://cloud.githubusercontent.com/assets/6477701/18834471/05c7a478-8431-11e6-94bb-09aa37b12ddb.png) ![2016-09-26 9 22 16](https://cloud.githubusercontent.com/assets/6477701/18834472/06c8735c-8431-11e6-8775-78631eab0411.png) 2016-09-27 2 29 27 2016-09-27 2 29 58 2016-09-27 2 30 05 **After** ![2016-09-26 9 29 47](https://cloud.githubusercontent.com/assets/6477701/18834467/0367f9da-8431-11e6-86d9-a490d3297339.png) ![2016-09-26 9 30 24](https://cloud.githubusercontent.com/assets/6477701/18834463/f870fae0-8430-11e6-9482-01fc47898492.png) 2016-09-27 2 28 19 2016-09-27 3 50 59 2016-09-27 3 51 03 Author: hyukjinkwon Closes #15242 from HyukjinKwon/minor-example-pyspark. (cherry picked from commit 2190037757a81d3172f75227f7891d968e1f0d90) Signed-off-by: Sean Owen --- python/pyspark/mllib/util.py | 8 ++++---- python/pyspark/rdd.py | 4 ++-- python/pyspark/sql/dataframe.py | 2 +- python/pyspark/sql/streaming.py | 6 +++--- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 48867a08dbfad..ed6fd4bca4c54 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -140,8 +140,8 @@ def saveAsLibSVMFile(data, dir): >>> from pyspark.mllib.regression import LabeledPoint >>> from glob import glob >>> from pyspark.mllib.util import MLUtils - >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \ - LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), + ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] >>> tempFile = NamedTemporaryFile(delete=True) >>> tempFile.close() >>> MLUtils.saveAsLibSVMFile(sc.parallelize(examples), tempFile.name) @@ -166,8 +166,8 @@ def loadLabeledPoints(sc, path, minPartitions=None): >>> from tempfile import NamedTemporaryFile >>> from pyspark.mllib.util import MLUtils >>> from pyspark.mllib.regression import LabeledPoint - >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \ - LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] + >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), + ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] >>> tempFile = NamedTemporaryFile(delete=True) >>> tempFile.close() >>> sc.parallelize(examples, 1).saveAsTextFile(tempFile.name) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 0508235c1c9ee..5fb10f86f4692 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -754,8 +754,8 @@ def foreachPartition(self, f): Applies a function to each partition of this RDD. >>> def f(iterator): - ... for x in iterator: - ... print(x) + ... for x in iterator: + ... print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreachPartition(f) """ def func(it): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 8577dc02cce6b..64d7a20757432 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -61,7 +61,7 @@ class DataFrame(object): people = sqlContext.read.parquet("...") department = sqlContext.read.parquet("...") - people.filter(people.age > 30).join(department, people.deptId == department.id)\ + people.filter(people.age > 30).join(department, people.deptId == department.id) \\ .groupBy(department.name, "gender").agg({"salary": "avg", "age": "max"}) .. versionadded:: 1.3 diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 38c19e2d1e9a1..118a02b6786b0 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -313,9 +313,9 @@ def load(self, path=None, format=None, schema=None, **options): :param schema: optional :class:`pyspark.sql.types.StructType` for the input schema. :param options: all other string options - >>> json_sdf = spark.readStream.format("json")\ - .schema(sdf_schema)\ - .load(tempfile.mkdtemp()) + >>> json_sdf = spark.readStream.format("json") \\ + ... .schema(sdf_schema) \\ + ... .load(tempfile.mkdtemp()) >>> json_sdf.isStreaming True >>> json_sdf.schema == sdf_schema From 4c694e452278e46231720e778a80c586b9e565f1 Mon Sep 17 00:00:00 2001 From: w00228970 Date: Wed, 28 Sep 2016 12:02:59 -0700 Subject: [PATCH 1367/1470] [SPARK-17644][CORE] Do not add failedStages when abortStage for fetch failure | Time |Thread 1 , Job1 | Thread 2 , Job2 | |:-------------:|:-------------:|:-----:| | 1 | abort stage due to FetchFailed | | | 2 | failedStages += failedStage | | | 3 | | task failed due to FetchFailed | | 4 | | can not post ResubmitFailedStages because failedStages is not empty | Then job2 of thread2 never resubmit the failed stage and hang. We should not add the failedStages when abortStage for fetch failure added unit test Author: w00228970 Author: wangfei Closes #15213 from scwf/dag-resubmit. (cherry picked from commit 46d1203bf2d01b219c4efc7e0e77a844c0c664da) Signed-off-by: Shixiong Zhu --- .../apache/spark/scheduler/DAGScheduler.scala | 24 ++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 58 ++++++++++++++++++- 2 files changed, 70 insertions(+), 12 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 399d67103fb4a..e7e2ff1718f22 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1277,18 +1277,20 @@ class DAGScheduler( s"has failed the maximum allowable number of " + s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " + s"Most recent failure reason: ${failureMessage}", None) - } else if (failedStages.isEmpty) { - // Don't schedule an event to resubmit failed stages if failed isn't empty, because - // in that case the event will already have been scheduled. - // TODO: Cancel running tasks in the stage - logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " + - s"$failedStage (${failedStage.name}) due to fetch failure") - messageScheduler.schedule(new Runnable { - override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) - }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS) + } else { + if (failedStages.isEmpty) { + // Don't schedule an event to resubmit failed stages if failed isn't empty, because + // in that case the event will already have been scheduled. + // TODO: Cancel running tasks in the stage + logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " + + s"$failedStage (${failedStage.name}) due to fetch failure") + messageScheduler.schedule(new Runnable { + override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) + }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS) + } + failedStages += failedStage + failedStages += mapStage } - failedStages += failedStage - failedStages += mapStage // Mark the map whose fetch failed as broken in the map stage if (mapId != -1) { mapStage.removeOutputLoc(mapId, bmAddress) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index f69e8f28fb420..5c353021677ef 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.util.Properties +import java.util.concurrent.atomic.AtomicBoolean import scala.annotation.meta.param import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} @@ -31,7 +32,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode -import org.apache.spark.shuffle.MetadataFetchFailedException +import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} @@ -2058,6 +2059,61 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou assertDataStructuresEmpty() } + test("SPARK-17644: After one stage is aborted for too many failed attempts, subsequent stages" + + "still behave correctly on fetch failures") { + // Runs a job that always encounters a fetch failure, so should eventually be aborted + def runJobWithPersistentFetchFailure: Unit = { + val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 1)).groupByKey() + val shuffleHandle = + rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + rdd1.map { + case (x, _) if (x == 1) => + throw new FetchFailedException( + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + case (x, _) => x + }.count() + } + + // Runs a job that encounters a single fetch failure but succeeds on the second attempt + def runJobWithTemporaryFetchFailure: Unit = { + object FailThisAttempt { + val _fail = new AtomicBoolean(true) + } + val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 1)).groupByKey() + val shuffleHandle = + rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + rdd1.map { + case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) => + throw new FetchFailedException( + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + } + } + + failAfter(10.seconds) { + val e = intercept[SparkException] { + runJobWithPersistentFetchFailure + } + assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException")) + } + + // Run a second job that will fail due to a fetch failure. + // This job will hang without the fix for SPARK-17644. + failAfter(10.seconds) { + val e = intercept[SparkException] { + runJobWithPersistentFetchFailure + } + assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException")) + } + + failAfter(10.seconds) { + try { + runJobWithTemporaryFetchFailure + } catch { + case e: Throwable => fail("A job with one fetch failure should eventually succeed") + } + } + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. From d358298f1082edd31489a1b08f428c8e60278d69 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 28 Sep 2016 16:19:06 -0700 Subject: [PATCH 1368/1470] [SPARK-17673][SQL] Incorrect exchange reuse with RowDataSourceScan (backport) This backports https://github.com/apache/spark/pull/15273 to branch-2.0 Also verified the test passes after the patch was applied. rxin Author: Eric Liang Closes #15282 from ericl/spark-17673-2. --- .../sql/execution/datasources/DataSourceStrategy.scala | 5 ++++- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 8 ++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 6b4b3b84d4d14..277969465e424 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -347,13 +347,16 @@ object DataSourceStrategy extends Strategy with Logging { // `Filter`s or cannot be handled by `relation`. val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And) + // These metadata values make scan plans uniquely identifiable for equality checking. + // TODO(SPARK-17701) using strings for equality checking is brittle val metadata: Map[String, String] = { val pairs = ArrayBuffer.empty[(String, String)] if (pushedFilters.nonEmpty) { pairs += (PUSHED_FILTERS -> pushedFilters.mkString("[", ", ", "]")) } - + pairs += ("ReadSchema" -> + StructType.fromAttributes(projects.map(_.toAttribute)).catalogString) pairs.toMap } 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 ec419e44e7930..1a6dba82b0e27 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 @@ -770,4 +770,12 @@ class JDBCSuite extends SparkFunSuite val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") assert(schema.contains("`order` TEXT")) } + + test("SPARK-17673: Exchange reuse respects differences in output schema") { + val df = sql("SELECT * FROM inttypes WHERE a IS NOT NULL") + val df1 = df.groupBy("a").agg("c" -> "min") + val df2 = df.groupBy("a").agg("d" -> "min") + val res = df1.union(df2) + assert(res.distinct().count() == 2) // would be 1 if the exchange was incorrectly reused + } } From 0a69477a10adb3969a20ae870436299ef5152788 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 28 Sep 2016 16:25:10 -0700 Subject: [PATCH 1369/1470] [SPARK-17641][SQL] Collect_list/Collect_set should not collect null values. ## What changes were proposed in this pull request? We added native versions of `collect_set` and `collect_list` in Spark 2.0. These currently also (try to) collect null values, this is different from the original Hive implementation. This PR fixes this by adding a null check to the `Collect.update` method. ## How was this patch tested? Added a regression test to `DataFrameAggregateSuite`. Author: Herman van Hovell Closes #15208 from hvanhovell/SPARK-17641. (cherry picked from commit 7d09232028967978d9db314ec041a762599f636b) Signed-off-by: Reynold Xin --- .../sql/catalyst/expressions/aggregate/collect.scala | 7 ++++++- .../apache/spark/sql/DataFrameAggregateSuite.scala | 12 ++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 896ff61b23093..78a388d20630b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -65,7 +65,12 @@ abstract class Collect extends ImperativeAggregate { } override def update(b: MutableRow, input: InternalRow): Unit = { - buffer += child.eval(input) + // Do not allow null values. We follow the semantics of Hive's collect_list/collect_set here. + // See: org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMkCollectionEvaluator + val value = child.eval(input) + if (value != null) { + buffer += value + } } override def merge(buffer: MutableRow, input: InternalRow): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index cb505ac928a70..3454caff6b825 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -477,6 +477,18 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { assert(error.message.contains("collect_set() cannot have map type data")) } + test("SPARK-17641: collect functions should not collect null values") { + val df = Seq(("1", 2), (null, 2), ("1", 4)).toDF("a", "b") + checkAnswer( + df.select(collect_list($"a"), collect_list($"b")), + Seq(Row(Seq("1", "1"), Seq(2, 2, 4))) + ) + checkAnswer( + df.select(collect_set($"a"), collect_set($"b")), + Seq(Row(Seq("1"), Seq(2, 4))) + ) + } + test("SPARK-14664: Decimal sum/avg over window should work.") { checkAnswer( spark.sql("select sum(a) over () from values 1.0, 2.0, 3.0 T(a)"), From 933d2c1ea4e5f5c4ec8d375b5ccaa4577ba4be38 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 28 Sep 2016 16:27:45 -0700 Subject: [PATCH 1370/1470] Preparing Spark release v2.0.1-rc4 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index dfb7e2232f8f4..3e49eac994787 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,7 +1,7 @@ Package: SparkR Type: Package Title: R Frontend for Apache Spark -Version: 2.0.2 +Version: 2.0.1 Date: 2016-08-27 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ca6daa27614f5..6db3a599ff5ea 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index c727f54d2ca3d..269b845565f1f 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index e335a89aac69e..20cf29efffc71 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 8e64f5678d78c..25cc32889ef3c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 94c75d68e34f5..37a5d09a3ff02 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6ff14d29474c2..ab287f3368a43 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e654c5479b34c..45831ce98dbcc 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 914025104d6ad..2d19e5b81cf58 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 09c54188358ab..75c89bd318983 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.2-SNAPSHOT -SPARK_VERSION_SHORT: 2.0.2 +SPARK_VERSION: 2.0.1 +SPARK_VERSION_SHORT: 2.0.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" MESOS_VERSION: 0.21.0 diff --git a/examples/pom.xml b/examples/pom.xml index 1796c472ca59e..89e0c61a3d694 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae3746fe55ee3..8c6e22155c26c 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 6cf206d36f448..dd45935e6eec2 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9ae7968c7296e..ba97794605ee6 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 7a67142dbf07f..8f8bde7eadfb9 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 0f795ff77b5ed..f7d8ef7b87c5d 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 658c396c57f19..260969fcd7347 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index e52170acedf87..1ae1d0e216b15 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4f339fb3fa2f..a4b14f8fb9603 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 4d2df2e45b1a4..9964b22cfe39f 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0e5ce9a78887f..b5d90b151cc8f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index c9a9e73a0ddb9..f96db6588cdad 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index b92359a476268..40f2e38832fbd 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c88efda8955eb..979217e2ba8f2 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4375e645ca05d..0c0dd0c6069d9 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 9e3d7fbbdd565..5681b36ea9ab8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index db7f05302a12b..80ab2e01a4576 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/pom.xml b/pom.xml index f2ffb39a6b5f9..79255f9a1f827 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index afb8891168fe6..4b70d647d59e5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ad3d408b8e9b2..efa327cbf21cb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 5e67226c6a61a..347f8a1bc82b1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7624c82b8ac51..c47d5f0285169 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ae66d7eeb32bf..2388d5f6ef1d4 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 897ef824976fd..4e0f5b1013f91 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7f55ebc87f682..ddc1b091fec81 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d81ce15b8bdf3..b676b6d21a2c1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.1 ../pom.xml From 7d612a7d5277183d3bee3882a687c76dc8ea0e9a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 28 Sep 2016 16:27:54 -0700 Subject: [PATCH 1371/1470] Preparing development version 2.0.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 3e49eac994787..dfb7e2232f8f4 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,7 +1,7 @@ Package: SparkR Type: Package Title: R Frontend for Apache Spark -Version: 2.0.1 +Version: 2.0.2 Date: 2016-08-27 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 6db3a599ff5ea..ca6daa27614f5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 269b845565f1f..c727f54d2ca3d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 20cf29efffc71..e335a89aac69e 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 25cc32889ef3c..8e64f5678d78c 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 37a5d09a3ff02..94c75d68e34f5 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ab287f3368a43..6ff14d29474c2 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 45831ce98dbcc..e654c5479b34c 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2d19e5b81cf58..914025104d6ad 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 75c89bd318983..09c54188358ab 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.1 -SPARK_VERSION_SHORT: 2.0.1 +SPARK_VERSION: 2.0.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.0.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" MESOS_VERSION: 0.21.0 diff --git a/examples/pom.xml b/examples/pom.xml index 89e0c61a3d694..1796c472ca59e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 8c6e22155c26c..ae3746fe55ee3 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index dd45935e6eec2..6cf206d36f448 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index ba97794605ee6..9ae7968c7296e 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 8f8bde7eadfb9..7a67142dbf07f 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index f7d8ef7b87c5d..0f795ff77b5ed 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 260969fcd7347..658c396c57f19 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 1ae1d0e216b15..e52170acedf87 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4b14f8fb9603..a4f339fb3fa2f 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 9964b22cfe39f..4d2df2e45b1a4 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index b5d90b151cc8f..0e5ce9a78887f 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f96db6588cdad..c9a9e73a0ddb9 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 40f2e38832fbd..b92359a476268 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 979217e2ba8f2..c88efda8955eb 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 0c0dd0c6069d9..4375e645ca05d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 5681b36ea9ab8..9e3d7fbbdd565 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 80ab2e01a4576..db7f05302a12b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 79255f9a1f827..f2ffb39a6b5f9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 4b70d647d59e5..afb8891168fe6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index efa327cbf21cb..ad3d408b8e9b2 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 347f8a1bc82b1..5e67226c6a61a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c47d5f0285169..7624c82b8ac51 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 2388d5f6ef1d4..ae66d7eeb32bf 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 4e0f5b1013f91..897ef824976fd 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index ddc1b091fec81..7f55ebc87f682 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index b676b6d21a2c1..d81ce15b8bdf3 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.1 + 2.0.2-SNAPSHOT ../pom.xml From ca8130050964fac8baa568918f0b67c44a7a2518 Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Thu, 29 Sep 2016 08:26:03 -0400 Subject: [PATCH 1372/1470] [MINOR][DOCS] Fix th doc. of spark-streaming with kinesis ## What changes were proposed in this pull request? This pr is just to fix the document of `spark-kinesis-integration`. Since `SPARK-17418` prevented all the kinesis stuffs (including kinesis example code) from publishing, `bin/run-example streaming.KinesisWordCountASL` and `bin/run-example streaming.JavaKinesisWordCountASL` does not work. Instead, it fetches the kinesis jar from the Spark Package. Author: Takeshi YAMAMURO Closes #15260 from maropu/DocFixKinesis. (cherry picked from commit b2e9731ca494c0c60d571499f68bb8306a3c9fe5) Signed-off-by: Sean Owen --- docs/streaming-kinesis-integration.md | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 96198ddf537b6..6be0b548bc62b 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -166,10 +166,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m #### Running the Example To run the example, -- Download Spark source and follow the [instructions](building-spark.html) to build Spark with profile *-Pkinesis-asl*. - - mvn -Pkinesis-asl -DskipTests clean package - +- Download a Spark binary from the [download site](http://spark.apache.org/downloads.html). - Set up Kinesis stream (see earlier section) within AWS. Note the name of the Kinesis stream and the endpoint URL corresponding to the region where the stream was created. @@ -180,12 +177,12 @@ To run the example,
      - bin/run-example streaming.KinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL] + bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.KinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL]
      - bin/run-example streaming.JavaKinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL] + bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.JavaKinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL]
      From 7ffafa3bfecb8bc92b79eddea1ca18166efd3385 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E6=98=9F=E5=8D=9A?= Date: Thu, 14 Jul 2016 00:21:27 +0800 Subject: [PATCH 1373/1470] [SPARK-16343][SQL] Improve the PushDownPredicate rule to pushdown predicates correctly in non-deterministic condition. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Currently our Optimizer may reorder the predicates to run them more efficient, but in non-deterministic condition, change the order between deterministic parts and non-deterministic parts may change the number of input rows. For example: ```SELECT a FROM t WHERE rand() < 0.1 AND a = 1``` And ```SELECT a FROM t WHERE a = 1 AND rand() < 0.1``` may call rand() for different times and therefore the output rows differ. This PR improved this condition by checking whether the predicate is placed before any non-deterministic predicates. ## How was this patch tested? Expanded related testcases in FilterPushdownSuite. Author: 蒋星博 Closes #14012 from jiangxb1987/ppd. (cherry picked from commit f376c37268848dbb4b2fb57677e22ef2bf207b49) Signed-off-by: Josh Rosen --- .../sql/catalyst/optimizer/Optimizer.scala | 44 ++++++++++++------- .../optimizer/FilterPushdownSuite.scala | 8 ++-- 2 files changed, 33 insertions(+), 19 deletions(-) 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 d824c2e26d717..35b122df3f029 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 @@ -1031,19 +1031,23 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { project.copy(child = Filter(replaceAlias(condition, aliasMap), grandChild)) // Push [[Filter]] operators through [[Window]] operators. Parts of the predicate that can be - // pushed beneath must satisfy the following two conditions: + // pushed beneath must satisfy the following conditions: // 1. All the expressions are part of window partitioning key. The expressions can be compound. - // 2. Deterministic + // 2. Deterministic. + // 3. Placed before any non-deterministic predicates. case filter @ Filter(condition, w: Window) if w.partitionSpec.forall(_.isInstanceOf[AttributeReference]) => val partitionAttrs = AttributeSet(w.partitionSpec.flatMap(_.references)) - val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond => - cond.references.subsetOf(partitionAttrs) && cond.deterministic && - // This is for ensuring all the partitioning expressions have been converted to alias - // in Analyzer. Thus, we do not need to check if the expressions in conditions are - // the same as the expressions used in partitioning columns. - partitionAttrs.forall(_.isInstanceOf[Attribute]) + + val (candidates, containingNonDeterministic) = + splitConjunctivePredicates(condition).span(_.deterministic) + + val (pushDown, rest) = candidates.partition { cond => + cond.references.subsetOf(partitionAttrs) } + + val stayUp = rest ++ containingNonDeterministic + if (pushDown.nonEmpty) { val pushDownPredicate = pushDown.reduce(And) val newWindow = w.copy(child = Filter(pushDownPredicate, w.child)) @@ -1062,11 +1066,16 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { // For each filter, expand the alias and check if the filter can be evaluated using // attributes produced by the aggregate operator's child operator. - val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond => + val (candidates, containingNonDeterministic) = + splitConjunctivePredicates(condition).span(_.deterministic) + + val (pushDown, rest) = candidates.partition { cond => val replaced = replaceAlias(cond, aliasMap) - replaced.references.subsetOf(aggregate.child.outputSet) && replaced.deterministic + replaced.references.subsetOf(aggregate.child.outputSet) } + val stayUp = rest ++ containingNonDeterministic + if (pushDown.nonEmpty) { val pushDownPredicate = pushDown.reduce(And) val replaced = replaceAlias(pushDownPredicate, aliasMap) @@ -1080,9 +1089,8 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { case filter @ Filter(condition, union: Union) => // Union could change the rows, so non-deterministic predicate can't be pushed down - val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond => - cond.deterministic - } + val (pushDown, stayUp) = splitConjunctivePredicates(condition).span(_.deterministic) + if (pushDown.nonEmpty) { val pushDownCond = pushDown.reduceLeft(And) val output = union.output @@ -1133,9 +1141,15 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { // come from grandchild. // TODO: non-deterministic predicates could be pushed through some operators that do not change // the rows. - val (pushDown, stayUp) = splitConjunctivePredicates(filter.condition).partition { cond => - cond.deterministic && cond.references.subsetOf(grandchild.outputSet) + val (candidates, containingNonDeterministic) = + splitConjunctivePredicates(filter.condition).span(_.deterministic) + + val (pushDown, rest) = candidates.partition { cond => + cond.references.subsetOf(grandchild.outputSet) } + + val stayUp = rest ++ containingNonDeterministic + if (pushDown.nonEmpty) { val newChild = insertFilter(pushDown.reduceLeft(And)) if (stayUp.nonEmpty) { 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 ea868d1a73a92..55836f96f7e0e 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 @@ -536,14 +536,14 @@ class FilterPushdownSuite extends PlanTest { val originalQuery = { testRelationWithArrayType .generate(Explode('c_arr), true, false, Some("arr")) - .where(('b >= 5) && ('a + Rand(10).as("rnd") > 6)) + .where(('b >= 5) && ('a + Rand(10).as("rnd") > 6) && ('c > 6)) } val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = { testRelationWithArrayType .where('b >= 5) .generate(Explode('c_arr), true, false, Some("arr")) - .where('a + Rand(10).as("rnd") > 6) + .where('a + Rand(10).as("rnd") > 6 && 'c > 6) .analyze } @@ -704,14 +704,14 @@ class FilterPushdownSuite extends PlanTest { val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int) val originalQuery = Union(Seq(testRelation, testRelation2)) - .where('a === 2L && 'b + Rand(10).as("rnd") === 3) + .where('a === 2L && 'b + Rand(10).as("rnd") === 3 && 'c > 5L) val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = Union(Seq( testRelation.where('a === 2L), testRelation2.where('d === 2L))) - .where('b + Rand(10).as("rnd") === 3) + .where('b + Rand(10).as("rnd") === 3 && 'c > 5L) .analyze comparePlans(optimized, correctAnswer) From f7839e47c3bda86d61c3b2be72c168aab4a5674f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 28 Sep 2016 19:03:05 -0700 Subject: [PATCH 1374/1470] [SPARK-17712][SQL] Fix invalid pushdown of data-independent filters beneath aggregates ## What changes were proposed in this pull request? This patch fixes a minor correctness issue impacting the pushdown of filters beneath aggregates. Specifically, if a filter condition references no grouping or aggregate columns (e.g. `WHERE false`) then it would be incorrectly pushed beneath an aggregate. Intuitively, the only case where you can push a filter beneath an aggregate is when that filter is deterministic and is defined over the grouping columns / expressions, since in that case the filter is acting to exclude entire groups from the query (like a `HAVING` clause). The existing code would only push deterministic filters beneath aggregates when all of the filter's references were grouping columns, but this logic missed the case where a filter has no references. For example, `WHERE false` is deterministic but is independent of the actual data. This patch fixes this minor bug by adding a new check to ensure that we don't push filters beneath aggregates when those filters don't reference any columns. ## How was this patch tested? New regression test in FilterPushdownSuite. Author: Josh Rosen Closes #15289 from JoshRosen/SPARK-17712. (cherry picked from commit 37eb9184f1e9f1c07142c66936671f4711ef407d) Signed-off-by: Josh Rosen --- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../optimizer/FilterPushdownSuite.scala | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) 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 35b122df3f029..4c06038185c58 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 @@ -1071,7 +1071,7 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { val (pushDown, rest) = candidates.partition { cond => val replaced = replaceAlias(cond, aliasMap) - replaced.references.subsetOf(aggregate.child.outputSet) + cond.references.nonEmpty && replaced.references.subsetOf(aggregate.child.outputSet) } val stayUp = rest ++ containingNonDeterministic 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 55836f96f7e0e..019f132d94cb2 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 @@ -687,6 +687,23 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("SPARK-17712: aggregate: don't push down filters that are data-independent") { + val originalQuery = LocalRelation.apply(testRelation.output, Seq.empty) + .select('a, 'b) + .groupBy('a)(count('a)) + .where(false) + + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = testRelation + .select('a, 'b) + .groupBy('a)(count('a)) + .where(false) + .analyze + + comparePlans(optimized, correctAnswer) + } + test("broadcast hint") { val originalQuery = BroadcastHint(testRelation) .where('a === 2L && 'b + Rand(10).as("rnd") === 3) From 7c9450b007205958984f39a881415cdbe75e0c34 Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Thu, 29 Sep 2016 15:51:05 -0400 Subject: [PATCH 1375/1470] [SPARK-17672] Spark 2.0 history server web Ui takes too long for a single application Added a new API getApplicationInfo(appId: String) in class ApplicationHistoryProvider and class SparkUI to get app info. In this change, FsHistoryProvider can directly fetch one app info in O(1) time complexity compared to O(n) before the change which used an Iterator.find() interface. Both ApplicationCache and OneApplicationResource classes adopt this new api. manual tests Author: Gang Wu Closes #15247 from wgtmac/SPARK-17671. (cherry picked from commit cb87b3ced9453b5717fa8e8637b97a2f3f25fdd7) Signed-off-by: Andrew Or --- .../spark/deploy/history/ApplicationHistoryProvider.scala | 5 +++++ .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 4 ++++ .../org/apache/spark/deploy/history/HistoryServer.scala | 4 ++++ .../org/apache/spark/status/api/v1/ApiRootResource.scala | 1 + .../apache/spark/status/api/v1/OneApplicationResource.scala | 2 +- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 4 ++++ 6 files changed, 19 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 44661edfff90b..ba42b4862aa90 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -109,4 +109,9 @@ private[history] abstract class ApplicationHistoryProvider { @throws(classOf[SparkException]) def writeEventLogs(appId: String, attemptId: Option[String], zipStream: ZipOutputStream): Unit + /** + * @return the [[ApplicationHistoryInfo]] for the appId if it exists. + */ + def getApplicationInfo(appId: String): Option[ApplicationHistoryInfo] + } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 110d882f05598..cf4a401b64b38 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -222,6 +222,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getListing(): Iterable[FsApplicationHistoryInfo] = applications.values + override def getApplicationInfo(appId: String): Option[FsApplicationHistoryInfo] = { + applications.get(appId) + } + override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { try { applications.get(appId).flatMap { appInfo => diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index c178917d8da3b..735aa43cfc994 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -182,6 +182,10 @@ class HistoryServer( getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) } + def getApplicationInfo(appId: String): Option[ApplicationInfo] = { + provider.getApplicationInfo(appId).map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) + } + override def writeEventLogs( appId: String, attemptId: Option[String], diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 681f295006e3c..c4f1fd29f2970 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -206,6 +206,7 @@ private[spark] object ApiRootResource { private[spark] trait UIRoot { def getSparkUI(appKey: String): Option[SparkUI] def getApplicationInfoList: Iterator[ApplicationInfo] + def getApplicationInfo(appId: String): Option[ApplicationInfo] /** * Write the event logs for the given app to the [[ZipOutputStream]] instance. If attemptId is diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index d7e6a8b589953..18c3e2f407360 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -24,7 +24,7 @@ private[v1] class OneApplicationResource(uiRoot: UIRoot) { @GET def getApp(@PathParam("appId") appId: String): ApplicationInfo = { - val apps = uiRoot.getApplicationInfoList.find { _.id == appId } + val apps = uiRoot.getApplicationInfo(appId) apps.getOrElse(throw new NotFoundException("unknown app: " + appId)) } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 39155ff2649ec..ef71db89798f1 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -126,6 +126,10 @@ private[spark] class SparkUI private ( )) )) } + + def getApplicationInfo(appId: String): Option[ApplicationInfo] = { + getApplicationInfoList.find(_.id == appId) + } } private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) From 0cdd7370a61618d042417ee387a3c32ee5c924e6 Mon Sep 17 00:00:00 2001 From: Bjarne Fruergaard Date: Thu, 29 Sep 2016 15:39:57 -0700 Subject: [PATCH 1376/1470] [SPARK-17721][MLLIB][ML] Fix for multiplying transposed SparseMatrix with SparseVector ## What changes were proposed in this pull request? * changes the implementation of gemv with transposed SparseMatrix and SparseVector both in mllib-local and mllib (identical) * adds a test that was failing before this change, but succeeds with these changes. The problem in the previous implementation was that it only increments `i`, that is enumerating the columns of a row in the SparseMatrix, when the row-index of the vector matches the column-index of the SparseMatrix. In cases where a particular row of the SparseMatrix has non-zero values at column-indices lower than corresponding non-zero row-indices of the SparseVector, the non-zero values of the SparseVector are enumerated without ever matching the column-index at index `i` and the remaining column-indices i+1,...,indEnd-1 are never attempted. The test cases in this PR illustrate this issue. ## How was this patch tested? I have run the specific `gemv` tests in both mllib-local and mllib. I am currently still running `./dev/run-tests`. ## ___ As per instructions, I hereby state that this is my original work and that I license the work to the project (Apache Spark) under the project's open source license. Mentioning dbtsai, viirya and brkyvz whom I can see have worked/authored on these parts before. Author: Bjarne Fruergaard Closes #15296 from bwahlgreen/bugfix-spark-17721. (cherry picked from commit 29396e7d1483d027960b9a1bed47008775c4253e) Signed-off-by: Joseph K. Bradley --- .../scala/org/apache/spark/ml/linalg/BLAS.scala | 8 ++++++-- .../org/apache/spark/ml/linalg/BLASSuite.scala | 17 +++++++++++++++++ .../org/apache/spark/mllib/linalg/BLAS.scala | 8 ++++++-- .../apache/spark/mllib/linalg/BLASSuite.scala | 17 +++++++++++++++++ 4 files changed, 46 insertions(+), 4 deletions(-) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala index 41b0c6c89a647..4ca19f3387f07 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala @@ -638,12 +638,16 @@ private[spark] object BLAS extends Serializable { val indEnd = Arows(rowCounter + 1) var sum = 0.0 var k = 0 - while (k < xNnz && i < indEnd) { + while (i < indEnd && k < xNnz) { if (xIndices(k) == Acols(i)) { sum += Avals(i) * xValues(k) + k += 1 + i += 1 + } else if (xIndices(k) < Acols(i)) { + k += 1 + } else { i += 1 } - k += 1 } yValues(rowCounter) = sum * alpha + beta * yValues(rowCounter) rowCounter += 1 diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala index 8a9f49792c1cd..6e72a5fff0a91 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala @@ -392,6 +392,23 @@ class BLASSuite extends SparkMLFunSuite { } } + val y17 = new DenseVector(Array(0.0, 0.0)) + val y18 = y17.copy + + val sA3 = new SparseMatrix(3, 2, Array(0, 2, 4), Array(1, 2, 0, 1), Array(2.0, 1.0, 1.0, 2.0)) + .transpose + val sA4 = + new SparseMatrix(2, 3, Array(0, 1, 3, 4), Array(1, 0, 1, 0), Array(1.0, 2.0, 2.0, 1.0)) + val sx3 = new SparseVector(3, Array(1, 2), Array(2.0, 1.0)) + + val expected4 = new DenseVector(Array(5.0, 4.0)) + + gemv(1.0, sA3, sx3, 0.0, y17) + gemv(1.0, sA4, sx3, 0.0, y18) + + assert(y17 ~== expected4 absTol 1e-15) + assert(y18 ~== expected4 absTol 1e-15) + val dAT = new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) val sAT = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 6a85608706974..0cd68a633c0b5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -637,12 +637,16 @@ private[spark] object BLAS extends Serializable with Logging { val indEnd = Arows(rowCounter + 1) var sum = 0.0 var k = 0 - while (k < xNnz && i < indEnd) { + while (i < indEnd && k < xNnz) { if (xIndices(k) == Acols(i)) { sum += Avals(i) * xValues(k) + k += 1 + i += 1 + } else if (xIndices(k) < Acols(i)) { + k += 1 + } else { i += 1 } - k += 1 } yValues(rowCounter) = sum * alpha + beta * yValues(rowCounter) rowCounter += 1 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index 80da03cc2efeb..6e68c1c9d36c8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -392,6 +392,23 @@ class BLASSuite extends SparkFunSuite { } } + val y17 = new DenseVector(Array(0.0, 0.0)) + val y18 = y17.copy + + val sA3 = new SparseMatrix(3, 2, Array(0, 2, 4), Array(1, 2, 0, 1), Array(2.0, 1.0, 1.0, 2.0)) + .transpose + val sA4 = + new SparseMatrix(2, 3, Array(0, 1, 3, 4), Array(1, 0, 1, 0), Array(1.0, 2.0, 2.0, 1.0)) + val sx3 = new SparseVector(3, Array(1, 2), Array(2.0, 1.0)) + + val expected4 = new DenseVector(Array(5.0, 4.0)) + + gemv(1.0, sA3, sx3, 0.0, y17) + gemv(1.0, sA4, sx3, 0.0, y18) + + assert(y17 ~== expected4 absTol 1e-15) + assert(y18 ~== expected4 absTol 1e-15) + val dAT = new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) val sAT = From a99ea4c9e0e2f91e4b524987788f0acee88e564d Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 29 Sep 2016 16:31:30 -0700 Subject: [PATCH 1377/1470] Updated the following PR with minor changes to allow cherry-pick to branch-2.0 [SPARK-17697][ML] Fixed bug in summary calculations that pattern match against label without casting In calling LogisticRegression.evaluate and GeneralizedLinearRegression.evaluate using a Dataset where the Label is not of a double type, calculations pattern match against a double and throw a MatchError. This fix casts the Label column to a DoubleType to ensure there is no MatchError. Added unit tests to call evaluate with a dataset that has Label as other numeric types. Author: Bryan Cutler Closes #15288 from BryanCutler/binaryLOR-numericCheck-SPARK-17697. (cherry picked from commit 2f739567080d804a942cfcca0e22f91ab7cbea36) Signed-off-by: Joseph K. Bradley --- .../classification/LogisticRegression.scala | 2 +- .../GeneralizedLinearRegression.scala | 11 ++++---- .../LogisticRegressionSuite.scala | 18 ++++++++++++- .../GeneralizedLinearRegressionSuite.scala | 25 +++++++++++++++++++ 4 files changed, 49 insertions(+), 7 deletions(-) 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 cca337487d6da..c50ee5dba28b2 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 @@ -860,7 +860,7 @@ class BinaryLogisticRegressionSummary private[classification] ( // TODO: Allow the user to vary the number of bins using a setBins method in // BinaryClassificationMetrics. For now the default is set to 100. @transient private val binaryMetrics = new BinaryClassificationMetrics( - predictions.select(probabilityCol, labelCol).rdd.map { + predictions.select(col(probabilityCol), col(labelCol).cast(DoubleType)).rdd.map { case Row(score: Vector, label: Double) => (score(1), label) }, 100 ) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 2bdc09e1db246..7f88c12d7e26e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -988,7 +988,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( } else { link.unlink(0.0) } - predictions.select(col(model.getLabelCol), w).rdd.map { + predictions.select(col(model.getLabelCol).cast(DoubleType), w).rdd.map { case Row(y: Double, weight: Double) => family.deviance(y, wtdmu, weight) }.sum() @@ -1000,7 +1000,7 @@ class GeneralizedLinearRegressionSummary private[regression] ( @Since("2.0.0") lazy val deviance: Double = { val w = weightCol - predictions.select(col(model.getLabelCol), col(predictionCol), w).rdd.map { + predictions.select(col(model.getLabelCol).cast(DoubleType), col(predictionCol), w).rdd.map { case Row(label: Double, pred: Double, weight: Double) => family.deviance(label, pred, weight) }.sum() @@ -1026,9 +1026,10 @@ class GeneralizedLinearRegressionSummary private[regression] ( lazy val aic: Double = { val w = weightCol val weightSum = predictions.select(w).agg(sum(w)).first().getDouble(0) - val t = predictions.select(col(model.getLabelCol), col(predictionCol), w).rdd.map { - case Row(label: Double, pred: Double, weight: Double) => - (label, pred, weight) + val t = predictions.select( + col(model.getLabelCol).cast(DoubleType), col(predictionCol), w).rdd.map { + case Row(label: Double, pred: Double, weight: Double) => + (label, pred, weight) } family.aic(t, deviance, numInstances, weightSum) + 2 * rank } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index a1b48539c46e0..27c872a82b371 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -31,7 +31,8 @@ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Dataset, Row} -import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.types.LongType class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -812,6 +813,21 @@ class LogisticRegressionSuite summary.precisionByThreshold.collect() === sameSummary.precisionByThreshold.collect()) } + test("evaluate with labels that are not doubles") { + // Evaluate a test set with Label that is a numeric type other than Double + val lr = new LogisticRegression() + .setMaxIter(1) + .setRegParam(1.0) + val model = lr.fit(dataset) + val summary = model.evaluate(dataset).asInstanceOf[BinaryLogisticRegressionSummary] + + val longLabelData = dataset.select(col(model.getLabelCol).cast(LongType), + col(model.getFeaturesCol)) + val longSummary = model.evaluate(longLabelData).asInstanceOf[BinaryLogisticRegressionSummary] + + assert(summary.areaUnderROC ~== longSummary.areaUnderROC absTol 1E-10) + } + test("statistics on training data") { // Test that loss is monotonically decreasing. val lr = new LogisticRegression() diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index a4568e83faca5..9d102153f222b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.mllib.random._ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.FloatType class GeneralizedLinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { @@ -1034,6 +1035,30 @@ class GeneralizedLinearRegressionSuite .setFamily("gaussian") .fit(datasetGaussianIdentity.as[LabeledPoint]) } + + test("evaluate with labels that are not doubles") { + // Evaulate with a dataset that contains Labels not as doubles to verify correct casting + val dataset = spark.createDataFrame(sc.parallelize(Seq( + Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), + Instance(19.0, 1.0, Vectors.dense(1.0, 7.0)), + Instance(23.0, 1.0, Vectors.dense(2.0, 11.0)), + Instance(29.0, 1.0, Vectors.dense(3.0, 13.0)) + ), 2)) + + val trainer = new GeneralizedLinearRegression() + .setMaxIter(1) + val model = trainer.fit(dataset) + assert(model.hasSummary) + val summary = model.summary + + val longLabelDataset = dataset.select(col(model.getLabelCol).cast(FloatType), + col(model.getFeaturesCol)) + val evalSummary = model.evaluate(longLabelDataset) + // The calculations below involve pattern matching with Label as a double + assert(evalSummary.nullDeviance === summary.nullDeviance) + assert(evalSummary.deviance === summary.deviance) + assert(evalSummary.aic === summary.aic) + } } object GeneralizedLinearRegressionSuite { From 744aac8e6ff04d7a3f1e8ccad335605ac8fe2f29 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 30 Sep 2016 22:05:59 -0700 Subject: [PATCH 1378/1470] [MINOR][DOC] Add an up-to-date description for default serialization during shuffling ## What changes were proposed in this pull request? This PR aims to make the doc up-to-date. The documentation is generally correct, but after https://issues.apache.org/jira/browse/SPARK-13926, Spark starts to choose Kyro as a default serialization library during shuffling of simple types, arrays of simple types, or string type. ## How was this patch tested? This is a documentation update. Author: Dongjoon Hyun Closes #15315 from dongjoon-hyun/SPARK-DOC-SERIALIZER. (cherry picked from commit 15e9bbb49e00b3982c428d39776725d0dea2cdfa) Signed-off-by: Reynold Xin --- docs/tuning.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/tuning.md b/docs/tuning.md index cbf37213aa724..9c43b315bbb9e 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -45,6 +45,7 @@ and calling `conf.set("spark.serializer", "org.apache.spark.serializer.KryoSeria This setting configures the serializer used for not only shuffling data between worker nodes but also when serializing RDDs to disk. The only reason Kryo is not the default is because of the custom registration requirement, but we recommend trying it in any network-intensive application. +Since Spark 2.0.0, we internally use Kryo serializer when shuffling RDDs with simple types, arrays of simple types, or string type. Spark automatically includes Kryo serializers for the many commonly-used core Scala classes covered in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. From b57e2acb134d94dafc81686da875c5dd3ea35c74 Mon Sep 17 00:00:00 2001 From: Jagadeesan Date: Mon, 3 Oct 2016 10:46:38 +0100 Subject: [PATCH 1379/1470] =?UTF-8?q?[SPARK-17736][DOCUMENTATION][SPARKR]?= =?UTF-8?q?=20Update=20R=20README=20for=20rmarkdown,=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? To build R docs (which are built when R tests are run), users need to install pandoc and rmarkdown. This was done for Jenkins in ~~[SPARK-17420](https://issues.apache.org/jira/browse/SPARK-17420)~~ … pandoc] Author: Jagadeesan Closes #15309 from jagadeesanas2/SPARK-17736. (cherry picked from commit a27033c0bbaae8f31db9b91693947ed71738ed11) Signed-off-by: Sean Owen --- docs/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/README.md b/docs/README.md index 8b515e187379c..ffd3b5712b618 100644 --- a/docs/README.md +++ b/docs/README.md @@ -19,8 +19,8 @@ installed. Also install the following libraries: $ sudo gem install jekyll jekyll-redirect-from pygments.rb $ sudo pip install Pygments # Following is needed only for generating API docs - $ sudo pip install sphinx - $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "roxygen2", "testthat"), repos="http://cran.stat.ucla.edu/")' + $ sudo pip install sphinx pypandoc + $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "roxygen2", "testthat", "rmarkdown"), repos="http://cran.stat.ucla.edu/")' ``` (Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0) From 613863b116b6cbc9ac83845c68a2d11b3b02f7cb Mon Sep 17 00:00:00 2001 From: zero323 Date: Mon, 3 Oct 2016 17:57:54 -0700 Subject: [PATCH 1380/1470] [SPARK-17587][PYTHON][MLLIB] SparseVector __getitem__ should follow __getitem__ contract ## What changes were proposed in this pull request? Replaces` ValueError` with `IndexError` when index passed to `ml` / `mllib` `SparseVector.__getitem__` is out of range. This ensures correct iteration behavior. Replaces `ValueError` with `IndexError` for `DenseMatrix` and `SparkMatrix` in `ml` / `mllib`. ## How was this patch tested? PySpark `ml` / `mllib` unit tests. Additional unit tests to prove that the problem has been resolved. Author: zero323 Closes #15144 from zero323/SPARK-17587. (cherry picked from commit d8399b600cef706c22d381b01fab19c610db439a) Signed-off-by: Joseph K. Bradley --- python/pyspark/ml/linalg/__init__.py | 10 +++++----- python/pyspark/ml/tests.py | 16 +++++++++++++--- python/pyspark/mllib/linalg/__init__.py | 10 +++++----- python/pyspark/mllib/tests.py | 16 +++++++++++++--- 4 files changed, 36 insertions(+), 16 deletions(-) diff --git a/python/pyspark/ml/linalg/__init__.py b/python/pyspark/ml/linalg/__init__.py index f42c589b92255..bd0e186e7fcb1 100644 --- a/python/pyspark/ml/linalg/__init__.py +++ b/python/pyspark/ml/linalg/__init__.py @@ -698,7 +698,7 @@ def __getitem__(self, index): "Indices must be of type integer, got type %s" % type(index)) if index >= self.size or index < -self.size: - raise ValueError("Index %d out of bounds." % index) + raise IndexError("Index %d out of bounds." % index) if index < 0: index += self.size @@ -945,10 +945,10 @@ def toSparse(self): 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)" + raise IndexError("Row index %d is out of range [0, %d)" % (i, self.numRows)) if j >= self.numCols or j < 0: - raise ValueError("Column index %d is out of range [0, %d)" + raise IndexError("Column index %d is out of range [0, %d)" % (j, self.numCols)) if self.isTransposed: @@ -1075,10 +1075,10 @@ def __reduce__(self): 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)" + raise IndexError("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)" + raise IndexError("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 diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 4bcb2c400c4aa..de95a47a2b8aa 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -1305,7 +1305,7 @@ def test_sparse_vector_indexing(self): self.assertEqual(sv[-3], 0.) self.assertEqual(sv[-5], 0.) for ind in [5, -6]: - self.assertRaises(ValueError, sv.__getitem__, ind) + self.assertRaises(IndexError, sv.__getitem__, ind) for ind in [7.8, '1']: self.assertRaises(TypeError, sv.__getitem__, ind) @@ -1313,11 +1313,15 @@ def test_sparse_vector_indexing(self): self.assertEqual(zeros[0], 0.0) self.assertEqual(zeros[3], 0.0) for ind in [4, -5]: - self.assertRaises(ValueError, zeros.__getitem__, ind) + self.assertRaises(IndexError, zeros.__getitem__, ind) empty = SparseVector(0, {}) for ind in [-1, 0, 1]: - self.assertRaises(ValueError, empty.__getitem__, ind) + self.assertRaises(IndexError, empty.__getitem__, ind) + + def test_sparse_vector_iteration(self): + self.assertListEqual(list(SparseVector(3, [], [])), [0.0, 0.0, 0.0]) + self.assertListEqual(list(SparseVector(5, [0, 3], [1.0, 2.0])), [1.0, 0.0, 0.0, 2.0, 0.0]) def test_matrix_indexing(self): mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) @@ -1326,6 +1330,9 @@ def test_matrix_indexing(self): for j in range(2): self.assertEqual(mat[i, j], expected[i][j]) + for i, j in [(-1, 0), (4, 1), (3, 4)]: + self.assertRaises(IndexError, mat.__getitem__, (i, j)) + def test_repr_dense_matrix(self): mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) self.assertTrue( @@ -1397,6 +1404,9 @@ def test_sparse_matrix(self): self.assertEqual(expected[i][j], sm1[i, j]) self.assertTrue(array_equal(sm1.toArray(), expected)) + for i, j in [(-1, 1), (4, 3), (3, 5)]: + self.assertRaises(IndexError, sm1.__getitem__, (i, j)) + # Test conversion to dense and sparse. smnew = sm1.toDense().toSparse() self.assertEqual(sm1.numRows, smnew.numRows) diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index 9672dbde823f2..d37e715c8d8ec 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -802,7 +802,7 @@ def __getitem__(self, index): "Indices must be of type integer, got type %s" % type(index)) if index >= self.size or index < -self.size: - raise ValueError("Index %d out of bounds." % index) + raise IndexError("Index %d out of bounds." % index) if index < 0: index += self.size @@ -1115,10 +1115,10 @@ def asML(self): 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)" + raise IndexError("Row index %d is out of range [0, %d)" % (i, self.numRows)) if j >= self.numCols or j < 0: - raise ValueError("Column index %d is out of range [0, %d)" + raise IndexError("Column index %d is out of range [0, %d)" % (j, self.numCols)) if self.isTransposed: @@ -1245,10 +1245,10 @@ def __reduce__(self): 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)" + raise IndexError("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)" + raise IndexError("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 diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 3f3dfd186c10d..c519883cdd73b 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -260,7 +260,7 @@ def test_sparse_vector_indexing(self): self.assertEqual(sv[-3], 0.) self.assertEqual(sv[-5], 0.) for ind in [5, -6]: - self.assertRaises(ValueError, sv.__getitem__, ind) + self.assertRaises(IndexError, sv.__getitem__, ind) for ind in [7.8, '1']: self.assertRaises(TypeError, sv.__getitem__, ind) @@ -268,11 +268,15 @@ def test_sparse_vector_indexing(self): self.assertEqual(zeros[0], 0.0) self.assertEqual(zeros[3], 0.0) for ind in [4, -5]: - self.assertRaises(ValueError, zeros.__getitem__, ind) + self.assertRaises(IndexError, zeros.__getitem__, ind) empty = SparseVector(0, {}) for ind in [-1, 0, 1]: - self.assertRaises(ValueError, empty.__getitem__, ind) + self.assertRaises(IndexError, empty.__getitem__, ind) + + def test_sparse_vector_iteration(self): + self.assertListEqual(list(SparseVector(3, [], [])), [0.0, 0.0, 0.0]) + self.assertListEqual(list(SparseVector(5, [0, 3], [1.0, 2.0])), [1.0, 0.0, 0.0, 2.0, 0.0]) def test_matrix_indexing(self): mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) @@ -281,6 +285,9 @@ def test_matrix_indexing(self): for j in range(2): self.assertEqual(mat[i, j], expected[i][j]) + for i, j in [(-1, 0), (4, 1), (3, 4)]: + self.assertRaises(IndexError, mat.__getitem__, (i, j)) + def test_repr_dense_matrix(self): mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) self.assertTrue( @@ -352,6 +359,9 @@ def test_sparse_matrix(self): self.assertEqual(expected[i][j], sm1[i, j]) self.assertTrue(array_equal(sm1.toArray(), expected)) + for i, j in [(-1, 1), (4, 3), (3, 5)]: + self.assertRaises(IndexError, sm1.__getitem__, (i, j)) + # Test conversion to dense and sparse. smnew = sm1.toDense().toSparse() self.assertEqual(sm1.numRows, smnew.numRows) From 5843932021cc8bbe0277943c6c480cfeae1b29e2 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 3 Oct 2016 19:32:59 -0700 Subject: [PATCH 1381/1470] [SPARK-17753][SQL] Allow a complex expression as the input a value based case statement ## What changes were proposed in this pull request? We currently only allow relatively simple expressions as the input for a value based case statement. Expressions like `case (a > 1) or (b = 2) when true then 1 when false then 0 end` currently fail. This PR adds support for such expressions. ## How was this patch tested? Added a test to the ExpressionParserSuite. Author: Herman van Hovell Closes #15322 from hvanhovell/SPARK-17753. (cherry picked from commit 2bbecdec2023143fd144e4242ff70822e0823986) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 12 ++++++------ .../spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../sql/catalyst/parser/ExpressionParserSuite.scala | 4 ++++ 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 8b721407eb170..e405d478ca334 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -519,16 +519,16 @@ valueExpression ; primaryExpression - : constant #constantDefault - | name=(CURRENT_DATE | CURRENT_TIMESTAMP) #timeFunctionCall + : name=(CURRENT_DATE | CURRENT_TIMESTAMP) #timeFunctionCall + | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase + | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase + | CAST '(' expression AS dataType ')' #cast + | constant #constantDefault | ASTERISK #star | qualifiedName '.' ASTERISK #star | '(' expression (',' expression)+ ')' #rowConstructor - | qualifiedName '(' (setQuantifier? expression (',' expression)*)? ')' (OVER windowSpec)? #functionCall | '(' query ')' #subqueryExpression - | CASE valueExpression whenClause+ (ELSE elseExpression=expression)? END #simpleCase - | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase - | CAST '(' expression AS dataType ')' #cast + | qualifiedName '(' (setQuantifier? expression (',' expression)*)? ')' (OVER windowSpec)? #functionCall | value=primaryExpression '[' index=valueExpression ']' #subscript | identifier #columnReference | base=primaryExpression '.' fieldName=identifier #dereference diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d1ce90770d3a9..e5ecefde41a49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1128,7 +1128,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * }}} */ override def visitSimpleCase(ctx: SimpleCaseContext): Expression = withOrigin(ctx) { - val e = expression(ctx.valueExpression) + val e = expression(ctx.value) val branches = ctx.whenClause.asScala.map { wCtx => (EqualTo(e, expression(wCtx.condition)), expression(wCtx.result)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 4aaae72fe91ef..c89ddadc486d5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -292,6 +292,10 @@ class ExpressionParserSuite extends PlanTest { test("case when") { assertEqual("case a when 1 then b when 2 then c else d end", CaseKeyWhen('a, Seq(1, 'b, 2, 'c, 'd))) + assertEqual("case (a or b) when true then c when false then d else e end", + CaseKeyWhen('a || 'b, Seq(true, 'c, false, 'd, 'e))) + assertEqual("case 'a'='a' when true then 1 end", + CaseKeyWhen("a" === "a", Seq(true, 1))) assertEqual("case when a = 1 then b when a = 2 then c else d end", CaseWhen(Seq(('a === 1, 'b.expr), ('a === 2, 'c.expr)), 'd)) } From 7429199e5b34d5594e3fcedb57eda789d16e26f3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 3 Oct 2016 21:28:16 -0700 Subject: [PATCH 1382/1470] [SPARK-17112][SQL] "select null" via JDBC triggers IllegalArgumentException in Thriftserver ## What changes were proposed in this pull request? Currently, Spark Thrift Server raises `IllegalArgumentException` for queries whose column types are `NullType`, e.g., `SELECT null` or `SELECT if(true,null,null)`. This PR fixes that by returning `void` like Hive 1.2. **Before** ```sql $ bin/beeline -u jdbc:hive2://localhost:10000 -e "select null" Connecting to jdbc:hive2://localhost:10000 Connected to: Spark SQL (version 2.1.0-SNAPSHOT) Driver: Hive JDBC (version 1.2.1.spark2) Transaction isolation: TRANSACTION_REPEATABLE_READ Error: java.lang.IllegalArgumentException: Unrecognized type name: null (state=,code=0) Closing: 0: jdbc:hive2://localhost:10000 $ bin/beeline -u jdbc:hive2://localhost:10000 -e "select if(true,null,null)" Connecting to jdbc:hive2://localhost:10000 Connected to: Spark SQL (version 2.1.0-SNAPSHOT) Driver: Hive JDBC (version 1.2.1.spark2) Transaction isolation: TRANSACTION_REPEATABLE_READ Error: java.lang.IllegalArgumentException: Unrecognized type name: null (state=,code=0) Closing: 0: jdbc:hive2://localhost:10000 ``` **After** ```sql $ bin/beeline -u jdbc:hive2://localhost:10000 -e "select null" Connecting to jdbc:hive2://localhost:10000 Connected to: Spark SQL (version 2.1.0-SNAPSHOT) Driver: Hive JDBC (version 1.2.1.spark2) Transaction isolation: TRANSACTION_REPEATABLE_READ +-------+--+ | NULL | +-------+--+ | NULL | +-------+--+ 1 row selected (3.242 seconds) Beeline version 1.2.1.spark2 by Apache Hive Closing: 0: jdbc:hive2://localhost:10000 $ bin/beeline -u jdbc:hive2://localhost:10000 -e "select if(true,null,null)" Connecting to jdbc:hive2://localhost:10000 Connected to: Spark SQL (version 2.1.0-SNAPSHOT) Driver: Hive JDBC (version 1.2.1.spark2) Transaction isolation: TRANSACTION_REPEATABLE_READ +-------------------------+--+ | (IF(true, NULL, NULL)) | +-------------------------+--+ | NULL | +-------------------------+--+ 1 row selected (0.201 seconds) Beeline version 1.2.1.spark2 by Apache Hive Closing: 0: jdbc:hive2://localhost:10000 ``` ## How was this patch tested? * Pass the Jenkins test with a new testsuite. * Also, Manually, after starting Spark Thrift Server, run the following command. ```sql $ bin/beeline -u jdbc:hive2://localhost:10000 -e "select null" $ bin/beeline -u jdbc:hive2://localhost:10000 -e "select if(true,null,null)" ``` **Hive 1.2** ```sql hive> create table null_table as select null; hive> desc null_table; OK _c0 void ``` Author: Dongjoon Hyun Closes #15325 from dongjoon-hyun/SPARK-17112. (cherry picked from commit c571cfb2d0e1e224107fc3f0c672730cae9804cb) Signed-off-by: Reynold Xin --- .../SparkExecuteStatementOperation.scala | 19 +++++++---- .../SparkExecuteStatementOperationSuite.scala | 33 +++++++++++++++++++ 2 files changed, 46 insertions(+), 6 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index b2717ec54e693..8a78523f56b45 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -56,14 +56,11 @@ private[hive] class SparkExecuteStatementOperation( private var statementId: String = _ private lazy val resultSchema: TableSchema = { - if (result == null || result.queryExecution.analyzed.output.size == 0) { + if (result == null || result.schema.isEmpty) { new TableSchema(Arrays.asList(new FieldSchema("Result", "string", ""))) } else { - logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, attr.dataType.catalogString, "") - } - new TableSchema(schema.asJava) + logInfo(s"Result Schema: ${result.schema}") + SparkExecuteStatementOperation.getTableSchema(result.schema) } } @@ -281,3 +278,13 @@ private[hive] class SparkExecuteStatementOperation( } } } + +object SparkExecuteStatementOperation { + def getTableSchema(structType: StructType): TableSchema = { + val schema = structType.map { field => + val attrTypeString = if (field.dataType == NullType) "void" else field.dataType.catalogString + new FieldSchema(field.name, attrTypeString, "") + } + new TableSchema(schema.asJava) + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala new file mode 100644 index 0000000000000..32ded0d254ef8 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.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.sql.hive.thriftserver + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.{NullType, StructField, StructType} + +class SparkExecuteStatementOperationSuite extends SparkFunSuite { + test("SPARK-17112 `select null` via JDBC triggers IllegalArgumentException in ThriftServer") { + val field1 = StructField("NULL", NullType) + val field2 = StructField("(IF(true, NULL, NULL))", NullType) + val tableSchema = StructType(Seq(field1, field2)) + val columns = SparkExecuteStatementOperation.getTableSchema(tableSchema).getColumnDescriptors() + assert(columns.size() == 2) + assert(columns.get(0).getType() == org.apache.hive.service.cli.Type.NULL_TYPE) + assert(columns.get(1).getType() == org.apache.hive.service.cli.Type.NULL_TYPE) + } +} From 3dbe8097facb854195729da7bd577f6c14eb2b2a Mon Sep 17 00:00:00 2001 From: ding Date: Tue, 4 Oct 2016 00:00:10 -0700 Subject: [PATCH 1383/1470] [SPARK-17559][MLLIB] persist edges if their storage level is non in PeriodicGraphCheckpointer ## What changes were proposed in this pull request? When use PeriodicGraphCheckpointer to persist graph, sometimes the edges isn't persisted. As currently only when vertices's storage level is none, graph is persisted. However there is a chance vertices's storage level is not none while edges's is none. Eg. graph created by a outerJoinVertices operation, vertices is automatically cached while edges is not. In this way, edges will not be persisted if we use PeriodicGraphCheckpointer do persist. We need separately check edges's storage level and persisted it if it's none. ## How was this patch tested? manual tests Author: ding Closes #15124 from dding3/spark-persisitEdge. (cherry picked from commit 126baa8d32bc0e7bf8b43f9efa84f2728f02347d) Signed-off-by: Joseph K. Bradley --- .../apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala index 20db6084d0e0d..80074897567eb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -87,7 +87,10 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( override protected def persist(data: Graph[VD, ED]): Unit = { if (data.vertices.getStorageLevel == StorageLevel.NONE) { - data.persist() + data.vertices.persist() + } + if (data.edges.getStorageLevel == StorageLevel.NONE) { + data.edges.persist() } } From 50f6be7598547fed5190a920fd3cebb4bc908524 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Tue, 4 Oct 2016 09:22:26 -0700 Subject: [PATCH 1384/1470] [SPARKR][DOC] minor formatting and output cleanup for R vignettes Clean up output, format table, truncate long example output, hide warnings (new - Left; existing - Right) ![image](https://cloud.githubusercontent.com/assets/8969467/19064018/5dcde4d0-89bc-11e6-857b-052df3f52a4e.png) ![image](https://cloud.githubusercontent.com/assets/8969467/19064034/6db09956-89bc-11e6-8e43-232d5c3fe5e6.png) ![image](https://cloud.githubusercontent.com/assets/8969467/19064058/88f09590-89bc-11e6-9993-61639e29dfdd.png) ![image](https://cloud.githubusercontent.com/assets/8969467/19064066/95ccbf64-89bc-11e6-877f-45af03ddcadc.png) ![image](https://cloud.githubusercontent.com/assets/8969467/19064082/a8445404-89bc-11e6-8532-26d8bc9b206f.png) Run create-doc.sh manually Author: Felix Cheung Closes #15340 from felixcheung/vignettes. (cherry picked from commit 068c198e956346b90968a4d74edb7bc820c4be28) Signed-off-by: Shivaram Venkataraman --- R/pkg/vignettes/sparkr-vignettes.Rmd | 29 ++++++++++++++++++---------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index 5156c9e566c95..babfb71c4af49 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -26,7 +26,7 @@ library(SparkR) We use default settings in which it runs in local mode. It auto downloads Spark package in the background if no previous installation is found. For more details about setup, see [Spark Session](#SetupSparkSession). -```{r, message=FALSE} +```{r, message=FALSE, results="hide"} sparkR.session() ``` @@ -114,10 +114,12 @@ In particular, the following Spark driver properties can be set in `sparkConfig` Property Name | Property group | spark-submit equivalent ---------------- | ------------------ | ---------------------- -spark.driver.memory | Application Properties | --driver-memory -spark.driver.extraClassPath | Runtime Environment | --driver-class-path -spark.driver.extraJavaOptions | Runtime Environment | --driver-java-options -spark.driver.extraLibraryPath | Runtime Environment | --driver-library-path +`spark.driver.memory` | Application Properties | `--driver-memory` +`spark.driver.extraClassPath` | Runtime Environment | `--driver-class-path` +`spark.driver.extraJavaOptions` | Runtime Environment | `--driver-java-options` +`spark.driver.extraLibraryPath` | Runtime Environment | `--driver-library-path` +`spark.yarn.keytab` | Application Properties | `--keytab` +`spark.yarn.principal` | Application Properties | `--principal` **For Windows users**: Due to different file prefixes across operating systems, to avoid the issue of potential wrong prefix, a current workaround is to specify `spark.sql.warehouse.dir` when starting the `SparkSession`. @@ -161,7 +163,7 @@ head(df) ### Data Sources SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. You can check the Spark SQL programming guide for more [specific options](https://spark.apache.org/docs/latest/sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. -The general method for creating `SparkDataFrame` from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active Spark Session will be used automatically. SparkR supports reading CSV, JSON and Parquet files natively and through Spark Packages you can find data source connectors for popular file formats like Avro. These packages can be added with `sparkPackages` parameter when initializing SparkSession using `sparkR.session'.` +The general method for creating `SparkDataFrame` from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active Spark Session will be used automatically. SparkR supports reading CSV, JSON and Parquet files natively and through Spark Packages you can find data source connectors for popular file formats like Avro. These packages can be added with `sparkPackages` parameter when initializing SparkSession using `sparkR.session`. ```{r, eval=FALSE} sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") @@ -406,10 +408,17 @@ class(model.summaries) ``` -To avoid lengthy display, we only present the result of the second fitted model. You are free to inspect other models as well. +To avoid lengthy display, we only present the partial result of the second fitted model. You are free to inspect other models as well. +```{r, include=FALSE} +ops <- options() +options(max.print=40) +``` ```{r} print(model.summaries[[2]]) ``` +```{r, include=FALSE} +options(ops) +``` ### SQL Queries @@ -534,7 +543,7 @@ head(select(kmeansPredictions, "model", "mpg", "hp", "wt", "prediction"), n = 20 Survival analysis studies the expected duration of time until an event happens, and often the relationship with risk factors or treatment taken on the subject. In contrast to standard regression analysis, survival modeling has to deal with special characteristics in the data including non-negative survival time and censoring. Accelerated Failure Time (AFT) model is a parametric survival model for censored data that assumes the effect of a covariate is to accelerate or decelerate the life course of an event by some constant. For more information, refer to the Wikipedia page [AFT Model](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) and the references there. Different from a [Proportional Hazards Model](https://en.wikipedia.org/wiki/Proportional_hazards_model) designed for the same purpose, the AFT model is easier to parallelize because each instance contributes to the objective function independently. -```{r} +```{r, warning=FALSE} library(survival) ovarianDF <- createDataFrame(ovarian) aftModel <- spark.survreg(ovarianDF, Surv(futime, fustat) ~ ecog_ps + rx) @@ -545,8 +554,8 @@ head(aftPredictions) ### Model Persistence The following example shows how to save/load an ML model by SparkR. -```{r} -irisDF <- suppressWarnings(createDataFrame(iris)) +```{r, warning=FALSE} +irisDF <- createDataFrame(iris) gaussianGLM <- spark.glm(irisDF, Sepal_Length ~ Sepal_Width + Species, family = "gaussian") # Save and then load a fitted MLlib model From a9165bb1b704483ad16331945b0968cbb1a97139 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 4 Oct 2016 09:38:44 -0700 Subject: [PATCH 1385/1470] [SPARK-17549][SQL] Only collect table size stat in driver for cached relation. This reverts commit 9ac68dbc5720026ea92acc61d295ca64d0d3d132. Turns out the original fix was correct. Original change description: The existing code caches all stats for all columns for each partition in the driver; for a large relation, this causes extreme memory usage, which leads to gc hell and application failures. It seems that only the size in bytes of the data is actually used in the driver, so instead just colllect that. In executors, the full stats are still kept, but that's not a big problem; we expect the data to be distributed and thus not really incur in too much memory pressure in each individual executor. There are also potential improvements on the executor side, since the data being stored currently is very wasteful (e.g. storing boxed types vs. primitive types for stats). But that's a separate issue. Author: Marcelo Vanzin Closes #15304 from vanzin/SPARK-17549.2. (cherry picked from commit 8d969a2125d915da1506c17833aa98da614a257f) Signed-off-by: Marcelo Vanzin --- .../execution/columnar/InMemoryRelation.scala | 24 +++++-------------- .../columnar/InMemoryColumnarQuerySuite.scala | 14 +++++++++++ 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 479934a7afc75..56bd5c1891e8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.columnar -import scala.collection.JavaConverters._ - import org.apache.commons.lang3.StringUtils import org.apache.spark.network.util.JavaUtils @@ -31,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.CollectionAccumulator +import org.apache.spark.util.LongAccumulator object InMemoryRelation { @@ -63,8 +61,7 @@ case class InMemoryRelation( @transient child: SparkPlan, tableName: Option[String])( @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, - val batchStats: CollectionAccumulator[InternalRow] = - child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) + val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator) extends logical.LeafNode with MultiInstanceRelation { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) @@ -74,21 +71,12 @@ case class InMemoryRelation( @transient val partitionStatistics = new PartitionStatistics(output) override lazy val statistics: Statistics = { - if (batchStats.value.isEmpty) { + if (batchStats.value == 0L) { // Underlying columnar RDD hasn't been materialized, no useful statistics information // available, return the default statistics. Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) } else { - // Underlying columnar RDD has been materialized, required information has also been - // collected via the `batchStats` accumulator. - val sizeOfRow: Expression = - BindReferences.bindReference( - output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), - partitionStatistics.schema) - - val sizeInBytes = - batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum - Statistics(sizeInBytes = sizeInBytes) + Statistics(sizeInBytes = batchStats.value.longValue) } } @@ -139,10 +127,10 @@ case class InMemoryRelation( rowCount += 1 } + batchStats.add(totalSize) + val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) .flatMap(_.values)) - - batchStats.add(stats) CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 937839644ad5f..0daa29b666f62 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -232,4 +232,18 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val columnTypes2 = List.fill(length2)(IntegerType) val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) } + + test("SPARK-17549: cached table size should be correctly calculated") { + val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None) + + // Materialize the data. + val expectedAnswer = data.collect() + checkAnswer(cached, expectedAnswer) + + // Check that the right size was calculated. + assert(cached.batchStats.value === expectedAnswer.size * INT.defaultSize) + } + } From a4f7df423e1e0aa512dfc496bc9de13831eae3f3 Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Tue, 4 Oct 2016 12:39:01 -0700 Subject: [PATCH 1386/1470] [SPARK-17773][BRANCH-2.0] Input/Output] Add VoidObjectInspector This is the PR for branch2.0: PR https://github.com/apache/spark/pull/15337 Added VoidObjectInspector to the list of PrimitiveObjectInspectors (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Executing following query was failing. select SOME_UDAF*(a.arr) from ( select Array(null) as arr from dim_one_row ) a After the fix, I am getting the correct output: res0: Array[org.apache.spark.sql.Row] = Array([null]) Author: Ergin Seyfe Closes #15337 from seyfe/add_void_object_inspector. Author: Ergin Seyfe Closes #15345 from seyfe/add_void_object_inspector_2.0. --- .../main/scala/org/apache/spark/sql/hive/HiveInspectors.scala | 3 +++ .../scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala | 1 + 2 files changed, 4 insertions(+) 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 9d56aec4a963b..ce6c533c9408e 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 @@ -424,6 +424,9 @@ private[hive] trait HiveInspectors { null } + case _: VoidObjectInspector => + (_: Any) => null // always be null for void object inspector + case soi: StandardStructObjectInspector => val schema = dataType.asInstanceOf[StructType] val wrappers = soi.getAllStructFieldRefs.asScala.zip(schema.fields).map { 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 3b867bbfa1817..5fda367bd600d 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 @@ -75,6 +75,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors { val data = Literal(true) :: + Literal(null) :: Literal(0.asInstanceOf[Byte]) :: Literal(0.asInstanceOf[Short]) :: Literal(0) :: From b8df2e53c38a30f51c710543c81279a59a9ab4fc Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 5 Oct 2016 14:54:55 -0700 Subject: [PATCH 1387/1470] [SPARK-17778][TESTS] Mock SparkContext to reduce memory usage of BlockManagerSuite ## What changes were proposed in this pull request? Mock SparkContext to reduce memory usage of BlockManagerSuite ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15350 from zsxwing/SPARK-17778. (cherry picked from commit 221b418b1c9db7b04c600b6300d18b034a4f444e) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 e93eee273f16d..1b3197a6921bd 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -107,7 +107,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) - sc = new SparkContext("local", "test", conf) + // Mock SparkContext to reduce the memory usage of tests. It's fine since the only reason we + // need to create a SparkContext is to initialize LiveListenerBus. + sc = mock(classOf[SparkContext]) + when(sc.conf).thenReturn(conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus(sc))), conf, true) From 3b6463a794a754d630d69398f009c055664dd905 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 5 Oct 2016 16:05:30 -0700 Subject: [PATCH 1388/1470] [SPARK-17758][SQL] Last returns wrong result in case of empty partition ## What changes were proposed in this pull request? The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order: ``` - Partition 1 [Row1, Row2] - Partition 2 [Row3] - Partition 3 [] ``` In this case the `Last` function will currently return a null, instead of the value of `Row3`. This PR fixes this by adding a `valueSet` flag to the `Last` function. ## How was this patch tested? We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function. Author: Herman van Hovell Closes #15348 from hvanhovell/SPARK-17758. (cherry picked from commit 5fd54b994e2078dbf0794932b4e0ffa9a9eda0c3) Signed-off-by: Yin Huai --- .../catalyst/expressions/aggregate/Last.scala | 27 ++--- .../DeclarativeAggregateEvaluator.scala | 61 ++++++++++ .../expressions/aggregate/LastTestSuite.scala | 109 ++++++++++++++++++ 3 files changed, 184 insertions(+), 13 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index af8840305805f..8579f7292d3ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -55,34 +55,35 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) extends Declarat private lazy val last = AttributeReference("last", child.dataType)() - override lazy val aggBufferAttributes: Seq[AttributeReference] = last :: Nil + private lazy val valueSet = AttributeReference("valueSet", BooleanType)() + + override lazy val aggBufferAttributes: Seq[AttributeReference] = last :: valueSet :: Nil override lazy val initialValues: Seq[Literal] = Seq( - /* last = */ Literal.create(null, child.dataType) + /* last = */ Literal.create(null, child.dataType), + /* valueSet = */ Literal.create(false, BooleanType) ) override lazy val updateExpressions: Seq[Expression] = { if (ignoreNulls) { Seq( - /* last = */ If(IsNull(child), last, child) + /* last = */ If(IsNull(child), last, child), + /* valueSet = */ Or(valueSet, IsNotNull(child)) ) } else { Seq( - /* last = */ child + /* last = */ child, + /* valueSet = */ Literal.create(true, BooleanType) ) } } override lazy val mergeExpressions: Seq[Expression] = { - if (ignoreNulls) { - Seq( - /* last = */ If(IsNull(last.right), last.left, last.right) - ) - } else { - Seq( - /* last = */ last.right - ) - } + // Prefer the right hand expression if it has been set. + Seq( + /* last = */ If(valueSet.right, last.right, last.left), + /* valueSet = */ Or(valueSet.right, valueSet.left) + ) } override lazy val evaluateExpression: AttributeReference = last diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala new file mode 100644 index 0000000000000..614f24db0aafb --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.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.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection + +/** + * Evaluator for a [[DeclarativeAggregate]]. + */ +case class DeclarativeAggregateEvaluator(function: DeclarativeAggregate, input: Seq[Attribute]) { + + lazy val initializer = GenerateSafeProjection.generate(function.initialValues) + + lazy val updater = GenerateSafeProjection.generate( + function.updateExpressions, + function.aggBufferAttributes ++ input) + + lazy val merger = GenerateSafeProjection.generate( + function.mergeExpressions, + function.aggBufferAttributes ++ function.inputAggBufferAttributes) + + lazy val evaluator = GenerateSafeProjection.generate( + function.evaluateExpression :: Nil, + function.aggBufferAttributes) + + def initialize(): InternalRow = initializer.apply(InternalRow.empty).copy() + + def update(values: InternalRow*): InternalRow = { + val joiner = new JoinedRow + val buffer = values.foldLeft(initialize()) { (buffer, input) => + updater(joiner(buffer, input)) + } + buffer.copy() + } + + def merge(buffers: InternalRow*): InternalRow = { + val joiner = new JoinedRow + val buffer = buffers.foldLeft(initialize()) { (left, right) => + merger(joiner(left, right)) + } + buffer.copy() + } + + def eval(buffer: InternalRow): InternalRow = evaluator(buffer).copy() +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala new file mode 100644 index 0000000000000..ba36bc074e154 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/LastTestSuite.scala @@ -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. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Literal} +import org.apache.spark.sql.types.IntegerType + +class LastTestSuite extends SparkFunSuite { + val input = AttributeReference("input", IntegerType, nullable = true)() + val evaluator = DeclarativeAggregateEvaluator(Last(input, Literal(false)), Seq(input)) + val evaluatorIgnoreNulls = DeclarativeAggregateEvaluator(Last(input, Literal(true)), Seq(input)) + + test("empty buffer") { + assert(evaluator.initialize() === InternalRow(null, false)) + } + + test("update") { + val result = evaluator.update( + InternalRow(1), + InternalRow(9), + InternalRow(-1)) + assert(result === InternalRow(-1, true)) + } + + test("update - ignore nulls") { + val result1 = evaluatorIgnoreNulls.update( + InternalRow(null), + InternalRow(9), + InternalRow(null)) + assert(result1 === InternalRow(9, true)) + + val result2 = evaluatorIgnoreNulls.update( + InternalRow(null), + InternalRow(null)) + assert(result2 === InternalRow(null, false)) + } + + test("merge") { + // Empty merge + val p0 = evaluator.initialize() + assert(evaluator.merge(p0) === InternalRow(null, false)) + + // Single merge + val p1 = evaluator.update(InternalRow(1), InternalRow(-99)) + assert(evaluator.merge(p1) === p1) + + // Multiple merges. + val p2 = evaluator.update(InternalRow(2), InternalRow(10)) + assert(evaluator.merge(p1, p2) === p2) + + // Empty partitions (p0 is empty) + assert(evaluator.merge(p1, p0, p2) === p2) + assert(evaluator.merge(p2, p1, p0) === p1) + } + + test("merge - ignore nulls") { + // Multi merges + val p1 = evaluatorIgnoreNulls.update(InternalRow(1), InternalRow(null)) + val p2 = evaluatorIgnoreNulls.update(InternalRow(null), InternalRow(null)) + assert(evaluatorIgnoreNulls.merge(p1, p2) === p1) + } + + test("eval") { + // Null Eval + assert(evaluator.eval(InternalRow(null, true)) === InternalRow(null)) + assert(evaluator.eval(InternalRow(null, false)) === InternalRow(null)) + + // Empty Eval + val p0 = evaluator.initialize() + assert(evaluator.eval(p0) === InternalRow(null)) + + // Update - Eval + val p1 = evaluator.update(InternalRow(1), InternalRow(-99)) + assert(evaluator.eval(p1) === InternalRow(-99)) + + // Update - Merge - Eval + val p2 = evaluator.update(InternalRow(2), InternalRow(10)) + val m1 = evaluator.merge(p1, p0, p2) + assert(evaluator.eval(m1) === InternalRow(10)) + + // Update - Merge - Eval (empty partition at the end) + val m2 = evaluator.merge(p2, p1, p0) + assert(evaluator.eval(m2) === InternalRow(-99)) + } + + test("eval - ignore nulls") { + // Update - Merge - Eval + val p1 = evaluatorIgnoreNulls.update(InternalRow(1), InternalRow(null)) + val p2 = evaluatorIgnoreNulls.update(InternalRow(null), InternalRow(null)) + val m1 = evaluatorIgnoreNulls.merge(p1, p2) + assert(evaluatorIgnoreNulls.eval(m1) === InternalRow(1)) + } +} From 1c2dff1eeeb045f3f5c3c1423ba07371b03965d7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 5 Oct 2016 16:48:43 -0700 Subject: [PATCH 1389/1470] [SPARK-17643] Remove comparable requirement from Offset (backport for branch-2.0) ## What changes were proposed in this pull request? Backport https://github.com/apache/spark/commit/988c71457354b0a443471f501cef544a85b1a76a to branch-2.0 ## How was this patch tested? Jenkins Author: Michael Armbrust Closes #15362 from zsxwing/SPARK-17643-2.0. --- .../execution/streaming/CompositeOffset.scala | 30 -------------- .../sql/execution/streaming/LongOffset.scala | 6 --- .../sql/execution/streaming/Offset.scala | 19 ++------- .../execution/streaming/StreamExecution.scala | 9 +++-- .../spark/sql/streaming/OffsetSuite.scala | 39 ------------------- 5 files changed, 9 insertions(+), 94 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompositeOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompositeOffset.scala index 729c8462fed65..ebc6ee8184902 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompositeOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompositeOffset.scala @@ -23,36 +23,6 @@ package org.apache.spark.sql.execution.streaming * vector clock that must progress linearly forward. */ case class CompositeOffset(offsets: Seq[Option[Offset]]) extends Offset { - /** - * Returns a negative integer, zero, or a positive integer as this object is less than, equal to, - * or greater than the specified object. - */ - override def compareTo(other: Offset): Int = other match { - case otherComposite: CompositeOffset if otherComposite.offsets.size == offsets.size => - val comparisons = offsets.zip(otherComposite.offsets).map { - case (Some(a), Some(b)) => a compareTo b - case (None, None) => 0 - case (None, _) => -1 - case (_, None) => 1 - } - val nonZeroSigns = comparisons.map(sign).filter(_ != 0).toSet - nonZeroSigns.size match { - case 0 => 0 // if both empty or only 0s - case 1 => nonZeroSigns.head // if there are only (0s and 1s) or (0s and -1s) - case _ => // there are both 1s and -1s - throw new IllegalArgumentException( - s"Invalid comparison between non-linear histories: $this <=> $other") - } - case _ => - throw new IllegalArgumentException(s"Cannot compare $this <=> $other") - } - - private def sign(num: Int): Int = num match { - case i if i < 0 => -1 - case i if i == 0 => 0 - case i if i > 0 => 1 - } - /** * Unpacks an offset into [[StreamProgress]] by associating each offset with the order list of * sources. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala index bb176408d8f59..c5e8827777792 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala @@ -22,12 +22,6 @@ package org.apache.spark.sql.execution.streaming */ case class LongOffset(offset: Long) extends Offset { - override def compareTo(other: Offset): Int = other match { - case l: LongOffset => offset.compareTo(l.offset) - case _ => - throw new IllegalArgumentException(s"Invalid comparison of $getClass with ${other.getClass}") - } - def +(increment: Long): LongOffset = new LongOffset(offset + increment) def -(decrement: Long): LongOffset = new LongOffset(offset - decrement) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala index 2cc012840dcaa..1f52abf277581 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala @@ -19,19 +19,8 @@ package org.apache.spark.sql.execution.streaming /** * An offset is a monotonically increasing metric used to track progress in the computation of a - * stream. An [[Offset]] must be comparable, and the result of `compareTo` must be consistent - * with `equals` and `hashcode`. + * stream. Since offsets are retrieved from a [[Source]] by a single thread, we know the global + * ordering of two [[Offset]] instances. We do assume that if two offsets are `equal` then no + * new data has arrived. */ -trait Offset extends Serializable { - - /** - * Returns a negative integer, zero, or a positive integer as this object is less than, equal to, - * or greater than the specified object. - */ - def compareTo(other: Offset): Int - - def >(other: Offset): Boolean = compareTo(other) > 0 - def <(other: Offset): Boolean = compareTo(other) < 0 - def <=(other: Offset): Boolean = compareTo(other) <= 0 - def >=(other: Offset): Boolean = compareTo(other) >= 0 -} +trait Offset extends Serializable {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index b7587f26af9e5..d50dfd42827af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -259,7 +259,7 @@ class StreamExecution( case (source, available) => committedOffsets .get(source) - .map(committed => committed < available) + .map(committed => committed != available) .getOrElse(true) } } @@ -318,7 +318,8 @@ class StreamExecution( // Request unprocessed data from all sources. val newData = availableOffsets.flatMap { - case (source, available) if committedOffsets.get(source).map(_ < available).getOrElse(true) => + case (source, available) + if committedOffsets.get(source).map(_ != available).getOrElse(true) => val current = committedOffsets.get(source) val batch = source.getBatch(current, available) logDebug(s"Retrieving data from $source: $current -> $available") @@ -404,10 +405,10 @@ class StreamExecution( * Blocks the current thread until processing for data from the given `source` has reached at * least the given `Offset`. This method is indented for use primarily when writing tests. */ - def awaitOffset(source: Source, newOffset: Offset): Unit = { + private[sql] def awaitOffset(source: Source, newOffset: Offset): Unit = { def notDone = { val localCommittedOffsets = committedOffsets - !localCommittedOffsets.contains(source) || localCommittedOffsets(source) < newOffset + !localCommittedOffsets.contains(source) || localCommittedOffsets(source) != newOffset } while (notDone) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala index 9590af4e7737d..b65a987770304 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala @@ -24,44 +24,12 @@ trait OffsetSuite extends SparkFunSuite { /** Creates test to check all the comparisons of offsets given a `one` that is less than `two`. */ def compare(one: Offset, two: Offset): Unit = { test(s"comparison $one <=> $two") { - assert(one < two) - assert(one <= two) - assert(one <= one) - assert(two > one) - assert(two >= one) - assert(one >= one) assert(one == one) assert(two == two) assert(one != two) assert(two != one) } } - - /** Creates test to check that non-equality comparisons throw exception. */ - def compareInvalid(one: Offset, two: Offset): Unit = { - test(s"invalid comparison $one <=> $two") { - intercept[IllegalArgumentException] { - assert(one < two) - } - - intercept[IllegalArgumentException] { - assert(one <= two) - } - - intercept[IllegalArgumentException] { - assert(one > two) - } - - intercept[IllegalArgumentException] { - assert(one >= two) - } - - assert(!(one == two)) - assert(!(two == one)) - assert(one != two) - assert(two != one) - } - } } class LongOffsetSuite extends OffsetSuite { @@ -79,10 +47,6 @@ class CompositeOffsetSuite extends OffsetSuite { one = CompositeOffset(None :: Nil), two = CompositeOffset(Some(LongOffset(2)) :: Nil)) - compareInvalid( // sizes must be same - one = CompositeOffset(Nil), - two = CompositeOffset(Some(LongOffset(2)) :: Nil)) - compare( one = CompositeOffset.fill(LongOffset(0), LongOffset(1)), two = CompositeOffset.fill(LongOffset(1), LongOffset(2))) @@ -91,8 +55,5 @@ class CompositeOffsetSuite extends OffsetSuite { one = CompositeOffset.fill(LongOffset(1), LongOffset(1)), two = CompositeOffset.fill(LongOffset(1), LongOffset(2))) - compareInvalid( - one = CompositeOffset.fill(LongOffset(2), LongOffset(1)), // vector time inconsistent - two = CompositeOffset.fill(LongOffset(1), LongOffset(2))) } From 225372adfb843afcbf9928db3989f2f8393ae6d8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 6 Oct 2016 10:33:45 -0700 Subject: [PATCH 1390/1470] [SPARK-17798][SQL] Remove redundant Experimental annotations in sql.streaming ## What changes were proposed in this pull request? I was looking through API annotations to catch mislabeled APIs, and realized DataStreamReader and DataStreamWriter classes are already annotated as Experimental, and as a result there is no need to annotate each method within them. ## How was this patch tested? N/A Author: Reynold Xin Closes #15373 from rxin/SPARK-17798. (cherry picked from commit 79accf45ace5549caa0cbab02f94fc87bedb5587) Signed-off-by: Shixiong Zhu --- .../sql/streaming/DataStreamReader.scala | 28 ------------------ .../sql/streaming/DataStreamWriter.scala | 29 ------------------- .../streaming/StreamingQueryListener.scala | 4 +-- 3 files changed, 1 insertion(+), 60 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 36c80ad8362ea..5141a014f8f79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -35,89 +35,73 @@ import org.apache.spark.sql.types.StructType @Experimental final class DataStreamReader private[sql](sparkSession: SparkSession) extends Logging { /** - * :: Experimental :: * Specifies the input data source format. * * @since 2.0.0 */ - @Experimental def format(source: String): DataStreamReader = { this.source = source this } /** - * :: Experimental :: * Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema * automatically from data. By specifying the schema here, the underlying data source can * skip the schema inference step, and thus speed up data loading. * * @since 2.0.0 */ - @Experimental def schema(schema: StructType): DataStreamReader = { this.userSpecifiedSchema = Option(schema) this } /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: String): DataStreamReader = { this.extraOptions += (key -> value) this } /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Boolean): DataStreamReader = option(key, value.toString) /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Long): DataStreamReader = option(key, value.toString) /** - * :: Experimental :: * Adds an input option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Double): DataStreamReader = option(key, value.toString) /** - * :: Experimental :: * (Scala-specific) Adds input options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: scala.collection.Map[String, String]): DataStreamReader = { this.extraOptions ++= options this } /** - * :: Experimental :: * Adds input options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: java.util.Map[String, String]): DataStreamReader = { this.options(options.asScala) this @@ -125,13 +109,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo /** - * :: Experimental :: * Loads input data stream in as a [[DataFrame]], for data streams that don't require a path * (e.g. external key-value stores). * * @since 2.0.0 */ - @Experimental def load(): DataFrame = { val dataSource = DataSource( @@ -143,18 +125,15 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo } /** - * :: Experimental :: * Loads input in as a [[DataFrame]], for data streams that read from some path. * * @since 2.0.0 */ - @Experimental def load(path: String): DataFrame = { option("path", path).load() } /** - * :: Experimental :: * Loads a JSON file stream (one object per line) and returns the result as a [[DataFrame]]. * * This function goes through the input once to determine the input schema. If you know the @@ -198,11 +177,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def json(path: String): DataFrame = format("json").load(path) /** - * :: Experimental :: * Loads a CSV file stream and returns the result as a [[DataFrame]]. * * This function will go through the input once to determine the input schema if `inferSchema` @@ -262,11 +239,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def csv(path: String): DataFrame = format("csv").load(path) /** - * :: Experimental :: * Loads a Parquet file stream, returning the result as a [[DataFrame]]. * * You can set the following Parquet-specific option(s) for reading Parquet files: @@ -280,13 +255,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def parquet(path: String): DataFrame = { format("parquet").load(path) } /** - * :: Experimental :: * Loads text files and returns a [[DataFrame]] whose schema starts with a string column named * "value", and followed by partitioned columns if there are any. * @@ -307,7 +280,6 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - @Experimental def text(path: String): DataFrame = format("text").load(path) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index f70c7d08a691c..b959444b49298 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -37,7 +37,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { private val df = ds.toDF() /** - * :: Experimental :: * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be * written to the sink @@ -46,15 +45,12 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def outputMode(outputMode: OutputMode): DataStreamWriter[T] = { this.outputMode = outputMode this } - /** - * :: Experimental :: * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to * the sink @@ -63,7 +59,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def outputMode(outputMode: String): DataStreamWriter[T] = { this.outputMode = outputMode.toLowerCase match { case "append" => @@ -78,7 +73,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } /** - * :: Experimental :: * Set the trigger for the stream query. The default value is `ProcessingTime(0)` and it will run * the query as fast as possible. * @@ -100,7 +94,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def trigger(trigger: Trigger): DataStreamWriter[T] = { this.trigger = trigger this @@ -108,25 +101,21 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** - * :: Experimental :: * Specifies the name of the [[StreamingQuery]] that can be started with `start()`. * This name must be unique among all the currently active queries in the associated SQLContext. * * @since 2.0.0 */ - @Experimental def queryName(queryName: String): DataStreamWriter[T] = { this.extraOptions += ("queryName" -> queryName) this } /** - * :: Experimental :: * Specifies the underlying output data source. Built-in options include "parquet" for now. * * @since 2.0.0 */ - @Experimental def format(source: String): DataStreamWriter[T] = { this.source = source this @@ -156,90 +145,74 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: String): DataStreamWriter[T] = { this.extraOptions += (key -> value) this } /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Boolean): DataStreamWriter[T] = option(key, value.toString) /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Long): DataStreamWriter[T] = option(key, value.toString) /** - * :: Experimental :: * Adds an output option for the underlying data source. * * @since 2.0.0 */ - @Experimental def option(key: String, value: Double): DataStreamWriter[T] = option(key, value.toString) /** - * :: Experimental :: * (Scala-specific) Adds output options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: scala.collection.Map[String, String]): DataStreamWriter[T] = { this.extraOptions ++= options this } /** - * :: Experimental :: * Adds output options for the underlying data source. * * @since 2.0.0 */ - @Experimental def options(options: java.util.Map[String, String]): DataStreamWriter[T] = { this.options(options.asScala) this } /** - * :: Experimental :: * Starts the execution of the streaming query, which will continually output results to the given * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with * the stream. * * @since 2.0.0 */ - @Experimental def start(path: String): StreamingQuery = { option("path", path).start() } /** - * :: Experimental :: * Starts the execution of the streaming query, which will continually output results to the given * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with * the stream. * * @since 2.0.0 */ - @Experimental def start(): StreamingQuery = { if (source == "memory") { assertNotPartitioned("memory") @@ -297,7 +270,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } /** - * :: Experimental :: * Starts the execution of the streaming query, which will continually send results to the given * [[ForeachWriter]] as as new data arrives. The [[ForeachWriter]] can be used to send the data * generated by the [[DataFrame]]/[[Dataset]] to an external system. @@ -343,7 +315,6 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.0.0 */ - @Experimental def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { this.source = "foreach" this.foreachWriter = if (writer != null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index db606abb8ce43..8a8855d85a4c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -35,7 +35,7 @@ abstract class StreamingQueryListener { /** * Called when a query is started. * @note This is called synchronously with - * [[org.apache.spark.sql.DataStreamWriter `DataStreamWriter.start()`]], + * [[org.apache.spark.sql.streaming.DataStreamWriter `DataStreamWriter.start()`]], * that is, `onQueryStart` will be called on all listeners before * `DataStreamWriter.start()` returns the corresponding [[StreamingQuery]]. Please * don't block this method as it will block your query. @@ -101,8 +101,6 @@ object StreamingQueryListener { * @param queryInfo Information about the status of the query. * @param exception The exception message of the [[StreamingQuery]] if the query was terminated * with an exception. Otherwise, it will be `None`. - * @param stackTrace The stack trace of the exception if the query was terminated with an - * exception. It will be empty if there was no error. * @since 2.0.0 */ @Experimental From a2bf09588ed98ef33028fcf4d72c15f06af2e9ad Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 6 Oct 2016 12:51:12 -0700 Subject: [PATCH 1391/1470] [SPARK-17780][SQL] Report Throwable to user in StreamExecution ## What changes were proposed in this pull request? When using an incompatible source for structured streaming, it may throw NoClassDefFoundError. It's better to just catch Throwable and report it to the user since the streaming thread is dying. ## How was this patch tested? `test("NoClassDefFoundError from an incompatible source")` Author: Shixiong Zhu Closes #15352 from zsxwing/SPARK-17780. (cherry picked from commit 9a48e60e6319d85f2c3be3a3c608dab135e18a73) Signed-off-by: Michael Armbrust --- .../execution/streaming/StreamExecution.scala | 7 ++++- .../spark/sql/streaming/StreamSuite.scala | 31 ++++++++++++++++++- .../spark/sql/streaming/StreamTest.scala | 3 +- 3 files changed, 37 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index d50dfd42827af..da1ae05fd0d34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -207,13 +207,18 @@ class StreamExecution( }) } catch { case _: InterruptedException if state == TERMINATED => // interrupted by stop() - case NonFatal(e) => + case e: Throwable => streamDeathCause = new StreamingQueryException( this, s"Query $name terminated with exception: ${e.getMessage}", e, Some(committedOffsets.toCompositeOffset(sources))) logError(s"Query $name terminated with error", e) + // Rethrow the fatal errors to allow the user using `Thread.UncaughtExceptionHandler` to + // handle them + if (!NonFatal(e)) { + throw e + } } finally { state = TERMINATED sparkSession.streams.notifyQueryTermination(StreamExecution.this) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 1caafb9d74440..cdbad901dba8e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.streaming +import scala.reflect.ClassTag +import scala.util.control.ControlThrowable + import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.ManualClock @@ -236,6 +238,33 @@ class StreamSuite extends StreamTest { } } + testQuietly("fatal errors from a source should be sent to the user") { + for (e <- Seq( + new VirtualMachineError {}, + new ThreadDeath, + new LinkageError, + new ControlThrowable {} + )) { + val source = new Source { + override def getOffset: Option[Offset] = { + throw e + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + throw e + } + + override def schema: StructType = StructType(Array(StructField("value", IntegerType))) + + override def stop(): Unit = {} + } + val df = Dataset[Int](sqlContext.sparkSession, StreamingExecutionRelation(source)) + testStream(df)( + ExpectFailure()(ClassTag(e.getClass)) + ) + } + } + test("output mode API in Scala") { val o1 = OutputMode.Append assert(o1 === InternalOutputModes.Append) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 6c5b170d9c7c3..1d467c6c5edd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -162,7 +162,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { /** Signals that a failure is expected and should not kill the test. */ case class ExpectFailure[T <: Throwable : ClassTag]() extends StreamAction { val causeClass: Class[T] = implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] - override def toString(): String = s"ExpectFailure[${causeClass.getCanonicalName}]" + override def toString(): String = s"ExpectFailure[${causeClass.getName}]" } /** Assert that a body is true */ @@ -317,7 +317,6 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { new UncaughtExceptionHandler { override def uncaughtException(t: Thread, e: Throwable): Unit = { streamDeathCause = e - testThread.interrupt() } }) From e355ca8e828629455228b6a346d64638ab639cfa Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Thu, 6 Oct 2016 14:28:49 -0700 Subject: [PATCH 1392/1470] [SPARK-17803][TESTS] Upgrade docker-client dependency [SPARK-17803: Docker integration tests don't run with "Docker for Mac"](https://issues.apache.org/jira/browse/SPARK-17803) ## What changes were proposed in this pull request? This PR upgrades the [docker-client](https://mvnrepository.com/artifact/com.spotify/docker-client) dependency from [3.6.6](https://mvnrepository.com/artifact/com.spotify/docker-client/3.6.6) to [5.0.2](https://mvnrepository.com/artifact/com.spotify/docker-client/5.0.2) to enable _Docker for Mac_ users to run the `docker-integration-tests` out of the box. The very latest docker-client version is [6.0.0](https://mvnrepository.com/artifact/com.spotify/docker-client/6.0.0) but that has one additional dependency and no usage yet. ## How was this patch tested? The code change was tested on Mac OS X Yosemite with both _Docker Toolbox_ as well as _Docker for Mac_ and on Linux Ubuntu 14.04. ``` $ build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -Phive -Phive-thriftserver -DskipTests clean package $ build/mvn -Pdocker-integration-tests -Pscala-2.11 -pl :spark-docker-integration-tests_2.11 clean compile test ``` Author: Christian Kadner Closes #15378 from ckadner/SPARK-17803_Docker_for_Mac. (cherry picked from commit 49d11d49983fbe270f4df4fb1e34b5fbe854c5ec) Signed-off-by: Josh Rosen --- .../org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala | 1 + pom.xml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala index c36f4d5f95482..609696bc8a2c7 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import scala.util.control.NonFatal import com.spotify.docker.client._ +import com.spotify.docker.client.exceptions.ImageNotFoundException import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding} import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually diff --git a/pom.xml b/pom.xml index f2ffb39a6b5f9..bb4b8a0de34d4 100644 --- a/pom.xml +++ b/pom.xml @@ -744,7 +744,7 @@ com.spotify docker-client - 3.6.6 + 5.0.2 test From b1a9c41e8c41c90dd15ee6f635356dd1a5bbf395 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 6 Oct 2016 16:09:45 -0700 Subject: [PATCH 1393/1470] [SPARK-17750][SQL][BACKPORT-2.0] Fix CREATE VIEW with INTERVAL arithmetic ## What changes were proposed in this pull request? Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions. ```scala scala> sql("CREATE TABLE dates (ts TIMESTAMP)") scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates") java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ... ``` ## How was this patch tested? Pass Jenkins with a new testcase. Author: Dongjoon Hyun Closes #15383 from dongjoon-hyun/SPARK-17750-BACK. --- .../expressions/datetimeExpressions.scala | 2 ++ .../resources/sqlgen/interval_arithmetic.sql | 8 ++++++++ .../catalyst/ExpressionSQLBuilderSuite.scala | 18 +++++++++++++++++- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 16 ++++++++++++++++ 4 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 04c17bdaf2989..7ab68a13e09cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -682,6 +682,7 @@ case class TimeAdd(start: Expression, interval: Expression) override def right: Expression = interval override def toString: String = s"$left + $right" + override def sql: String = s"${left.sql} + ${right.sql}" override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, CalendarIntervalType) override def dataType: DataType = TimestampType @@ -762,6 +763,7 @@ case class TimeSub(start: Expression, interval: Expression) override def right: Expression = interval override def toString: String = s"$left - $right" + override def sql: String = s"${left.sql} - ${right.sql}" override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, CalendarIntervalType) override def dataType: DataType = TimestampType diff --git a/sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql b/sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql new file mode 100644 index 0000000000000..31d00348769f5 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/interval_arithmetic.sql @@ -0,0 +1,8 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +select ts + interval 1 day, ts + interval 2 days, + ts - interval 1 day, ts - interval 2 days, + ts + interval '1' day, ts + interval '2' days, + ts - interval '1' day, ts - interval '2' days +from dates +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `CAST(ts + interval 1 days AS TIMESTAMP)`, `gen_attr_2` AS `CAST(ts + interval 2 days AS TIMESTAMP)`, `gen_attr_3` AS `CAST(ts - interval 1 days AS TIMESTAMP)`, `gen_attr_4` AS `CAST(ts - interval 2 days AS TIMESTAMP)`, `gen_attr_5` AS `CAST(ts + interval 1 days AS TIMESTAMP)`, `gen_attr_6` AS `CAST(ts + interval 2 days AS TIMESTAMP)`, `gen_attr_7` AS `CAST(ts - interval 1 days AS TIMESTAMP)`, `gen_attr_8` AS `CAST(ts - interval 2 days AS TIMESTAMP)` FROM (SELECT CAST(`gen_attr_1` + interval 1 days AS TIMESTAMP) AS `gen_attr_0`, CAST(`gen_attr_1` + interval 2 days AS TIMESTAMP) AS `gen_attr_2`, CAST(`gen_attr_1` - interval 1 days AS TIMESTAMP) AS `gen_attr_3`, CAST(`gen_attr_1` - interval 2 days AS TIMESTAMP) AS `gen_attr_4`, CAST(`gen_attr_1` + interval 1 days AS TIMESTAMP) AS `gen_attr_5`, CAST(`gen_attr_1` + interval 2 days AS TIMESTAMP) AS `gen_attr_6`, CAST(`gen_attr_1` - interval 1 days AS TIMESTAMP) AS `gen_attr_7`, CAST(`gen_attr_1` - interval 2 days AS TIMESTAMP) AS `gen_attr_8` FROM (SELECT `ts` AS `gen_attr_1` FROM `default`.`dates`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index 43a218b4d14b3..56c0736e8e462 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.catalyst import java.sql.Timestamp import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{If, Literal, SpecifiedWindowFrame, WindowSpecDefinition} +import org.apache.spark.sql.catalyst.expressions.{If, Literal, SpecifiedWindowFrame, TimeAdd, + TimeSub, WindowSpecDefinition} +import org.apache.spark.unsafe.types.CalendarInterval class ExpressionSQLBuilderSuite extends SQLBuilderTest { test("literal") { @@ -118,4 +120,18 @@ class ExpressionSQLBuilderSuite extends SQLBuilderTest { s"(PARTITION BY `a`, `b` ORDER BY `c` ASC, `d` DESC $frame)" ) } + + test("interval arithmetic") { + val interval = Literal(new CalendarInterval(0, CalendarInterval.MICROS_PER_DAY)) + + checkSQL( + TimeAdd('a, interval), + "`a` + interval 1 days" + ) + + checkSQL( + TimeSub('a, interval), + "`a` - interval 1 days" + ) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index ef2f756a4bde4..3794f63a6a361 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -1121,4 +1121,20 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { """.stripMargin, "inline_tables") } + + test("SPARK-17750 - interval arithmetic") { + withTable("dates") { + sql("create table dates (ts timestamp)") + checkSQL( + """ + |select ts + interval 1 day, ts + interval 2 days, + | ts - interval 1 day, ts - interval 2 days, + | ts + interval '1' day, ts + interval '2' days, + | ts - interval '1' day, ts - interval '2' days + |from dates + """.stripMargin, + "interval_arithmetic" + ) + } + } } From 594a2cf6f7c74c54127b8c3947aadbe0052b404c Mon Sep 17 00:00:00 2001 From: sethah Date: Thu, 6 Oct 2016 21:10:17 -0700 Subject: [PATCH 1394/1470] [SPARK-17792][ML] L-BFGS solver for linear regression does not accept general numeric label column types ## What changes were proposed in this pull request? Before, we computed `instances` in LinearRegression in two spots, even though they did the same thing. One of them did not cast the label column to `DoubleType`. This patch consolidates the computation and always casts the label column to `DoubleType`. ## How was this patch tested? Added a unit test to check all solvers. This test failed before this patch. Author: sethah Closes #15364 from sethah/linreg_numeric_type. (cherry picked from commit 3713bb199142c5e06e2e527c99650f02f41f47b1) Signed-off-by: Yanbo Liang --- .../spark/ml/regression/LinearRegression.scala | 17 ++++++----------- .../ml/regression/LinearRegressionSuite.scala | 8 +++++--- 2 files changed, 11 insertions(+), 14 deletions(-) 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 f82f2c3a82b68..600bbcbac5e81 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 @@ -163,17 +163,18 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + val instances: RDD[Instance] = dataset.select( + col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } + if (($(solver) == "auto" && $(elasticNetParam) == 0.0 && numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == "normal") { require($(elasticNetParam) == 0.0, "Only L2 regularization can be used when normal " + "solver is used.'") // For low dimensional data, WeightedLeastSquares is more efficiently since the // training algorithm only requires one pass through the data. (SPARK-10668) - val instances: RDD[Instance] = dataset.select( - col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), $(standardization), true) @@ -196,12 +197,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String return lrModel.setSummary(trainingSummary) } - val instances: RDD[Instance] = - dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index 265f2f45c45fe..df67a3a354610 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -1019,12 +1019,14 @@ class LinearRegressionSuite } test("should support all NumericType labels and not support other types") { - val lr = new LinearRegression().setMaxIter(1) - MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( - lr, spark, isClassification = false) { (expected, actual) => + for (solver <- Seq("auto", "l-bfgs", "normal")) { + val lr = new LinearRegression().setMaxIter(1).setSolver(solver) + MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( + lr, spark, isClassification = false) { (expected, actual) => assert(expected.intercept === actual.intercept) assert(expected.coefficients === actual.coefficients) } + } } } From 380b099fcfe6f70b978300ea208faf630855471a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 6 Oct 2016 22:27:20 -0700 Subject: [PATCH 1395/1470] [SPARK-17612][SQL][BRANCH-2.0] Support `DESCRIBE table PARTITION` SQL syntax ## What changes were proposed in this pull request? This is a backport of SPARK-17612. This implements `DESCRIBE table PARTITION` SQL Syntax again. It was supported until Spark 1.6.2, but was dropped since 2.0.0. **Spark 1.6.2** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res1: org.apache.spark.sql.DataFrame = [result: string] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res2: org.apache.spark.sql.DataFrame = [result: string] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) +----------------------------------------------------------------+ |result | +----------------------------------------------------------------+ |a string | |b int | |c string | |d string | | | |# Partition Information | |# col_name data_type comment | | | |c string | |d string | +----------------------------------------------------------------+ ``` **Spark 2.0** - **Before** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res1: org.apache.spark.sql.DataFrame = [] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) org.apache.spark.sql.catalyst.parser.ParseException: Unsupported SQL statement ``` - **After** ```scala scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") res1: org.apache.spark.sql.DataFrame = [] scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false) +-----------------------+---------+-------+ |col_name |data_type|comment| +-----------------------+---------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information| | | |# col_name |data_type|comment| |c |string |null | |d |string |null | +-----------------------+---------+-------+ scala> sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)").show(100,false) +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ |col_name |data_type|comment| +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information | | | |# col_name |data_type|comment| |c |string |null | |d |string |null | | | | | |Detailed Partition Information CatalogPartition( Partition Values: [Us, 1] Storage(Location: file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1]) Partition Parameters:{transient_lastDdlTime=1475001066})| | | +-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+ scala> sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)").show(100,false) +--------------------------------+---------------------------------------------------------------------------------------+-------+ |col_name |data_type |comment| +--------------------------------+---------------------------------------------------------------------------------------+-------+ |a |string |null | |b |int |null | |c |string |null | |d |string |null | |# Partition Information | | | |# col_name |data_type |comment| |c |string |null | |d |string |null | | | | | |# Detailed Partition Information| | | |Partition Value: |[Us, 1] | | |Database: |default | | |Table: |partitioned_table | | |Location: |file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1| | |Partition Parameters: | | | | transient_lastDdlTime |1475001066 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.TextInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat | | |Compressed: |No | | |Storage Desc Parameters: | | | | serialization.format |1 | | +--------------------------------+---------------------------------------------------------------------------------------+-------+ ``` ## How was this patch tested? Pass the Jenkins tests with a new testcase. Author: Dongjoon Hyun Closes #15351 from dongjoon-hyun/SPARK-17612-BACK. --- .../sql/catalyst/catalog/interface.scala | 12 ++- .../spark/sql/execution/SparkSqlParser.scala | 15 +++- .../spark/sql/execution/command/tables.scala | 83 ++++++++++++++--- .../resources/sql-tests/inputs/describe.sql | 27 ++++++ .../sql-tests/results/describe.sql.out | 90 +++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 78 +++++++++++++++- 6 files changed, 287 insertions(+), 18 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/describe.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/describe.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index e7430b030901a..83428924b1600 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -108,7 +108,17 @@ case class CatalogColumn( case class CatalogTablePartition( spec: CatalogTypes.TablePartitionSpec, storage: CatalogStorageFormat, - parameters: Map[String, String] = Map.empty) + parameters: Map[String, String] = Map.empty) { + + override def toString: String = { + val output = + Seq( + s"Partition Values: [${spec.values.mkString(", ")}]", + s"$storage", + s"Partition Parameters:{${parameters.map(p => p._1 + "=" + p._2).mkString(", ")}}") + output.filter(_.nonEmpty).mkString("CatalogPartition(\n\t", "\n\t", ")") + } +} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3072a6d79eac7..f092360a98bae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -278,13 +278,24 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * Create a [[DescribeTableCommand]] logical plan. */ override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { - // Describe partition and column are not supported yet. Return null and let the parser decide + // Describe column are not supported yet. Return null and let the parser decide // what to do with this (create an exception or pass it on to a different system). - if (ctx.describeColName != null || ctx.partitionSpec != null) { + if (ctx.describeColName != null) { null } else { + val partitionSpec = if (ctx.partitionSpec != null) { + // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. + visitPartitionSpec(ctx.partitionSpec).map { + case (key, Some(value)) => key -> value + case (key, _) => + throw new ParseException(s"PARTITION specification is incomplete: `$key`", ctx) + } + } else { + Map.empty[String, String] + } DescribeTableCommand( visitTableIdentifier(ctx.tableIdentifier), + partitionSpec, ctx.EXTENDED != null, ctx.FORMATTED != null) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 995feb3b670ee..9253db0f20823 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} @@ -417,10 +417,14 @@ case class TruncateTableCommand( /** * Command that looks like * {{{ - * DESCRIBE [EXTENDED|FORMATTED] table_name; + * DESCRIBE [EXTENDED|FORMATTED] table_name partitionSpec?; * }}} */ -case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isFormatted: Boolean) +case class DescribeTableCommand( + table: TableIdentifier, + partitionSpec: Map[String, String], + isExtended: Boolean, + isFormatted: Boolean) extends RunnableCommand { override val output: Seq[Attribute] = Seq( @@ -438,6 +442,10 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF val catalog = sparkSession.sessionState.catalog if (catalog.isTemporaryTable(table)) { + if (partitionSpec.nonEmpty) { + throw new AnalysisException( + s"DESC PARTITION is not allowed on a temporary view: ${table.identifier}") + } describeSchema(catalog.lookupRelation(table).schema, result) } else { val metadata = catalog.getTableMetadata(table) @@ -451,12 +459,16 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF describeSchema(metadata.schema, result) } - if (isExtended) { - describeExtended(metadata, result) - } else if (isFormatted) { - describeFormatted(metadata, result) + describePartitionInfo(metadata, result) + + if (partitionSpec.isEmpty) { + if (isExtended) { + describeExtendedTableInfo(metadata, result) + } else if (isFormatted) { + describeFormattedTableInfo(metadata, result) + } } else { - describePartitionInfo(metadata, result) + describeDetailedPartitionInfo(catalog, metadata, result) } } @@ -481,16 +493,12 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF } } - private def describeExtended(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { - describePartitionInfo(table, buffer) - + private def describeExtendedTableInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { append(buffer, "", "", "") append(buffer, "# Detailed Table Information", table.toString, "") } - private def describeFormatted(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { - describePartitionInfo(table, buffer) - + private def describeFormattedTableInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = { append(buffer, "", "", "") append(buffer, "# Detailed Table Information", "", "") append(buffer, "Database:", table.database, "") @@ -548,6 +556,53 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF } } + private def describeDetailedPartitionInfo( + catalog: SessionCatalog, + metadata: CatalogTable, + result: ArrayBuffer[Row]): Unit = { + if (metadata.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( + s"DESC PARTITION is not allowed on a view: ${table.identifier}") + } + if (DDLUtils.isDatasourceTable(metadata)) { + throw new AnalysisException( + s"DESC PARTITION is not allowed on a datasource table: ${table.identifier}") + } + val partition = catalog.getPartition(table, partitionSpec) + if (isExtended) { + describeExtendedDetailedPartitionInfo(table, metadata, partition, result) + } else if (isFormatted) { + describeFormattedDetailedPartitionInfo(table, metadata, partition, result) + describeStorageInfo(metadata, result) + } + } + + private def describeExtendedDetailedPartitionInfo( + tableIdentifier: TableIdentifier, + table: CatalogTable, + partition: CatalogTablePartition, + buffer: ArrayBuffer[Row]): Unit = { + append(buffer, "", "", "") + append(buffer, "Detailed Partition Information " + partition.toString, "", "") + } + + private def describeFormattedDetailedPartitionInfo( + tableIdentifier: TableIdentifier, + table: CatalogTable, + partition: CatalogTablePartition, + buffer: ArrayBuffer[Row]): Unit = { + append(buffer, "", "", "") + append(buffer, "# Detailed Partition Information", "", "") + append(buffer, "Partition Value:", s"[${partition.spec.values.mkString(", ")}]", "") + append(buffer, "Database:", table.database, "") + append(buffer, "Table:", tableIdentifier.table, "") + append(buffer, "Location:", partition.storage.locationUri.getOrElse(""), "") + append(buffer, "Partition Parameters:", "", "") + partition.parameters.foreach { case (key, value) => + append(buffer, s" $key", value, "") + } + } + private def describeSchema(schema: Seq[CatalogColumn], buffer: ArrayBuffer[Row]): Unit = { schema.foreach { column => append(buffer, column.name, column.dataType.toLowerCase, column.comment.orNull) diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql new file mode 100644 index 0000000000000..2e916d9edc73b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql @@ -0,0 +1,27 @@ +CREATE TABLE t (a STRING, b INT) PARTITIONED BY (c STRING, d STRING); + +ALTER TABLE t ADD PARTITION (c='Us', d=1); + +DESC t; + +-- Ignore these because there exist timestamp results, e.g., `Create Table`. +-- DESC EXTENDED t; +-- DESC FORMATTED t; + +DESC t PARTITION (c='Us', d=1); + +-- Ignore these because there exist timestamp results, e.g., transient_lastDdlTime. +-- DESC EXTENDED t PARTITION (c='Us', d=1); +-- DESC FORMATTED t PARTITION (c='Us', d=1); + +-- NoSuchPartitionException: Partition not found in table +DESC t PARTITION (c='Us', d=2); + +-- AnalysisException: Partition spec is invalid +DESC t PARTITION (c='Us'); + +-- ParseException: PARTITION specification is incomplete +DESC t PARTITION (c='Us', d); + +-- DROP TEST TABLE +DROP TABLE t; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out new file mode 100644 index 0000000000000..37bf303f1bfe4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -0,0 +1,90 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE TABLE t (a STRING, b INT) PARTITIONED BY (c STRING, d STRING) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +ALTER TABLE t ADD PARTITION (c='Us', d=1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +DESC t +-- !query 2 schema +struct +-- !query 2 output +# Partition Information +# col_name data_type comment +a string +b int +c string +c string +d string +d string + + +-- !query 3 +DESC t PARTITION (c='Us', d=1) +-- !query 3 schema +struct +-- !query 3 output +# Partition Information +# col_name data_type comment +a string +b int +c string +c string +d string +d string + + +-- !query 4 +DESC t PARTITION (c='Us', d=2) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +Partition not found in table 't' database 'default': +c -> Us +d -> 2; + + +-- !query 5 +DESC t PARTITION (c='Us') +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`default`.`t`'; + + +-- !query 6 +DESC t PARTITION (c='Us', d) +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.catalyst.parser.ParseException + +PARTITION specification is incomplete: `d`(line 1, pos 0) + +== SQL == +DESC t PARTITION (c='Us', d) +^^^ + + +-- !query 7 +DROP TABLE t +-- !query 7 schema +struct<> +-- !query 7 output + 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 51d49469b3f46..b421fae6dc57b 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 @@ -26,7 +26,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry, + NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils @@ -342,6 +343,81 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("describe partition") { + withTable("partitioned_table") { + sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") + sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") + + checkKeywordsExist(sql("DESC partitioned_table PARTITION (c='Us', d=1)"), + "# Partition Information", + "# col_name") + + checkKeywordsExist(sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)"), + "# Partition Information", + "# col_name", + "Detailed Partition Information CatalogPartition(", + "Partition Values: [Us, 1]", + "Storage(Location:", + "Partition Parameters") + + checkKeywordsExist(sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)"), + "# Partition Information", + "# col_name", + "# Detailed Partition Information", + "Partition Value:", + "Database:", + "Table:", + "Location:", + "Partition Parameters:", + "# Storage Information") + } + } + + test("describe partition - error handling") { + withTable("partitioned_table", "datasource_table") { + sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)") + sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)") + + val m = intercept[NoSuchPartitionException] { + sql("DESC partitioned_table PARTITION (c='Us', d=2)") + }.getMessage() + assert(m.contains("Partition not found in table")) + + val m2 = intercept[AnalysisException] { + sql("DESC partitioned_table PARTITION (c='Us')") + }.getMessage() + assert(m2.contains("Partition spec is invalid")) + + val m3 = intercept[ParseException] { + sql("DESC partitioned_table PARTITION (c='Us', d)") + }.getMessage() + assert(m3.contains("PARTITION specification is incomplete: `d`")) + + spark + .range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write + .partitionBy("d") + .saveAsTable("datasource_table") + val m4 = intercept[AnalysisException] { + sql("DESC datasource_table PARTITION (d=2)") + }.getMessage() + assert(m4.contains("DESC PARTITION is not allowed on a datasource table")) + + val m5 = intercept[AnalysisException] { + spark.range(10).select('id as 'a, 'id as 'b).createTempView("view1") + sql("DESC view1 PARTITION (c='Us', d=1)") + }.getMessage() + assert(m5.contains("DESC PARTITION is not allowed on a temporary view")) + + withView("permanent_view") { + val m = intercept[AnalysisException] { + sql("CREATE VIEW permanent_view AS SELECT * FROM partitioned_table") + sql("DESC permanent_view PARTITION (c='Us', d=1)") + }.getMessage() + assert(m.contains("DESC PARTITION is not allowed on a view")) + } + } + } + test("SPARK-5371: union with null and sum") { val df = Seq((1, 1)).toDF("c1", "c2") df.createOrReplaceTempView("table1") From 3487b020354988a91181f23b1c6711bfcdb4c529 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Fri, 7 Oct 2016 00:27:55 -0700 Subject: [PATCH 1396/1470] [SPARK-17805][PYSPARK] Fix in sqlContext.read.text when pass in list of paths ## What changes were proposed in this pull request? If given a list of paths, `pyspark.sql.readwriter.text` will attempt to use an undefined variable `paths`. This change checks if the param `paths` is a basestring and then converts it to a list, so that the same variable `paths` can be used for both cases ## How was this patch tested? Added unit test for reading list of files Author: Bryan Cutler Closes #15379 from BryanCutler/sql-readtext-paths-SPARK-17805. (cherry picked from commit bcaa799cb01289f73e9f48526e94653a07628983) Signed-off-by: Reynold Xin --- python/pyspark/sql/readwriter.py | 4 ++-- python/pyspark/sql/tests.py | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index dc13a818fcbfb..e62f4832fa184 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -287,8 +287,8 @@ def text(self, paths): [Row(value=u'hello'), Row(value=u'this')] """ if isinstance(paths, basestring): - path = [paths] - return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(path))) + paths = [paths] + return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths))) @since(2.0) def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None, diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 1ec40cecf438e..3343bd72276a6 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1697,6 +1697,12 @@ def test_cache(self): "does_not_exist", lambda: spark.catalog.uncacheTable("does_not_exist")) + def test_read_text_file_list(self): + df = self.spark.read.text(['python/test_support/sql/text-test.txt', + 'python/test_support/sql/text-test.txt']) + count = df.count() + self.assertEquals(count, 4) + class HiveSparkSubmitTests(SparkSubmitTests): From 9f2eb27a425385836dba5aad61babfb1db738a73 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 7 Oct 2016 10:31:41 -0700 Subject: [PATCH 1397/1470] [SPARK-17707][WEBUI] Web UI prevents spark-submit application to be finished This expands calls to Jetty's simple `ServerConnector` constructor to explicitly specify a `ScheduledExecutorScheduler` that makes daemon threads. It should otherwise result in exactly the same configuration, because the other args are copied from the constructor that is currently called. (I'm not sure we should change the Hive Thriftserver impl, but I did anyway.) This also adds `sc.stop()` to the quick start guide example. Existing tests; _pending_ at least manual verification of the fix. Author: Sean Owen Closes #15381 from srowen/SPARK-17707. (cherry picked from commit cff560755244dd4ccb998e0c56e81d2620cd4cff) Signed-off-by: Shixiong Zhu --- .../deploy/rest/RestSubmissionServer.scala | 14 +++++++++--- .../org/apache/spark/ui/JettyUtils.scala | 14 +++++++++--- docs/quick-start.md | 7 +++++- .../cli/thrift/ThriftHttpCLIService.java | 22 +++++++++++++++++-- 4 files changed, 48 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index fa55d470842b3..b30c980e95a9a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -22,9 +22,9 @@ import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.{Server, ServerConnector} +import org.eclipse.jetty.server.{HttpConnectionFactory, Server, ServerConnector} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} -import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.json4s._ import org.json4s.jackson.JsonMethods._ @@ -83,7 +83,15 @@ private[spark] abstract class RestSubmissionServer( threadPool.setDaemon(true) val server = new Server(threadPool) - val connector = new ServerConnector(server) + val connector = new ServerConnector( + server, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("RestSubmissionServer-JettyScheduler", true), + null, + -1, + -1, + new HttpConnectionFactory()) connector.setHost(host) connector.setPort(startPort) server.addConnector(connector) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 50283f2b74a41..d980094a3be45 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -25,12 +25,12 @@ import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import scala.xml.Node -import org.eclipse.jetty.server.{Request, Server, ServerConnector} +import org.eclipse.jetty.server.{HttpConnectionFactory, Request, Server, ServerConnector} import org.eclipse.jetty.server.handler._ import org.eclipse.jetty.servlet._ import org.eclipse.jetty.servlets.gzip.GzipHandler import org.eclipse.jetty.util.component.LifeCycle -import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} @@ -251,7 +251,15 @@ private[spark] object JettyUtils extends Logging { val server = new Server(pool) val connectors = new ArrayBuffer[ServerConnector] // Create a connector on port currentPort to listen for HTTP requests - val httpConnector = new ServerConnector(server) + val httpConnector = new ServerConnector( + server, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler(s"$serverName-JettyScheduler", true), + null, + -1, + -1, + new HttpConnectionFactory()) httpConnector.setPort(currentPort) connectors += httpConnector diff --git a/docs/quick-start.md b/docs/quick-start.md index 1b961fd45576b..04b0f0a7096d3 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -240,7 +240,8 @@ object SimpleApp { val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() - println("Lines with a: %s, Lines with b: %s".format(numAs, numBs)) + println(s"Lines with a: $numAs, Lines with b: $numBs") + sc.stop() } } {% endhighlight %} @@ -328,6 +329,8 @@ public class SimpleApp { }).count(); System.out.println("Lines with a: " + numAs + ", lines with b: " + numBs); + + sc.stop() } } {% endhighlight %} @@ -407,6 +410,8 @@ numAs = logData.filter(lambda s: 'a' in s).count() numBs = logData.filter(lambda s: 'b' in s).count() print("Lines with a: %i, lines with b: %i" % (numAs, numBs)) + +sc.stop() {% endhighlight %} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 37e4845cceb9e..341a7fdbb59b8 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -37,11 +37,15 @@ import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TProtocolFactory; import org.apache.thrift.server.TServlet; +import org.eclipse.jetty.server.AbstractConnectionFactory; +import org.eclipse.jetty.server.ConnectionFactory; +import org.eclipse.jetty.server.HttpConnectionFactory; import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.eclipse.jetty.util.thread.ExecutorThreadPool; +import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler; public class ThriftHttpCLIService extends ThriftCLIService { @@ -70,7 +74,8 @@ public void run() { httpServer = new org.eclipse.jetty.server.Server(threadPool); // Connector configs - ServerConnector connector = new ServerConnector(httpServer); + + ConnectionFactory[] connectionFactories; boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); String schemeName = useSsl ? "https" : "http"; // Change connector if SSL is used @@ -90,8 +95,21 @@ public void run() { Arrays.toString(sslContextFactory.getExcludeProtocols())); sslContextFactory.setKeyStorePath(keyStorePath); sslContextFactory.setKeyStorePassword(keyStorePassword); - connector = new ServerConnector(httpServer, sslContextFactory); + connectionFactories = AbstractConnectionFactory.getFactories( + sslContextFactory, new HttpConnectionFactory()); + } else { + connectionFactories = new ConnectionFactory[] { new HttpConnectionFactory() }; } + ServerConnector connector = new ServerConnector( + httpServer, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("HiveServer2-HttpHandler-JettyScheduler", true), + null, + -1, + -1, + connectionFactories); + connector.setPort(portNum); // Linux:yes, Windows:no connector.setReuseAddress(!Shell.WINDOWS); From f460a199e8fc78ce879b79844c6c9e340b574439 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 7 Oct 2016 11:32:39 -0700 Subject: [PATCH 1398/1470] [SPARK-17346][SQL][TEST-MAVEN] Add Kafka source for Structured Streaming (branch 2.0) ## What changes were proposed in this pull request? Backport https://github.com/apache/spark/commit/9293734d35eb3d6e4fd4ebb86f54dd5d3a35e6db and https://github.com/apache/spark/commit/b678e465afa417780b54db0fbbaa311621311f15 into branch 2.0. The only difference is the Spark version in pom file. ## How was this patch tested? Jenkins. Author: Shixiong Zhu Closes #15367 from zsxwing/kafka-source-branch-2.0. --- .../spark/util/UninterruptibleThread.scala | 7 - dev/run-tests.py | 2 +- dev/sparktestsupport/modules.py | 12 + .../structured-streaming-kafka-integration.md | 239 ++++++++++ .../structured-streaming-programming-guide.md | 7 +- external/kafka-0-10-sql/pom.xml | 96 ++++ ...pache.spark.sql.sources.DataSourceRegister | 1 + .../sql/kafka010/CachedKafkaConsumer.scala | 152 +++++++ .../spark/sql/kafka010/KafkaSource.scala | 399 ++++++++++++++++ .../sql/kafka010/KafkaSourceOffset.scala | 54 +++ .../sql/kafka010/KafkaSourceProvider.scala | 282 ++++++++++++ .../spark/sql/kafka010/KafkaSourceRDD.scala | 148 ++++++ .../spark/sql/kafka010/package-info.java | 21 + .../src/test/resources/log4j.properties | 28 ++ .../sql/kafka010/KafkaSourceOffsetSuite.scala | 39 ++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 425 ++++++++++++++++++ .../spark/sql/kafka010/KafkaTestUtils.scala | 339 ++++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 6 +- sql/core/pom.xml | 27 ++ .../execution/streaming/StreamExecution.scala | 8 +- .../spark/sql/streaming/StreamTest.scala | 40 +- 22 files changed, 2310 insertions(+), 23 deletions(-) create mode 100644 docs/structured-streaming-kafka-integration.md create mode 100644 external/kafka-0-10-sql/pom.xml create mode 100644 external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java create mode 100644 external/kafka-0-10-sql/src/test/resources/log4j.properties create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala index 4dcf95177aa78..f0b68f0cb7e29 100644 --- a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala @@ -89,13 +89,6 @@ private[spark] class UninterruptibleThread(name: String) extends Thread(name) { } } - /** - * Tests whether `interrupt()` has been called. - */ - override def isInterrupted: Boolean = { - super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread } - } - /** * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be * interrupted until it enters into the interruptible status. diff --git a/dev/run-tests.py b/dev/run-tests.py index 930d7f8bd9459..ad9d4ac7fa3c2 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -110,7 +110,7 @@ def determine_modules_to_test(changed_modules): ['graphx', 'examples'] >>> x = [x.name for x in determine_modules_to_test([modules.sql])] >>> x # doctest: +NORMALIZE_WHITESPACE - ['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver', + ['sql', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver', 'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml'] """ modules_to_test = set() diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index ce5725764be6d..5da1950bd987d 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -158,6 +158,18 @@ def __hash__(self): ) +sql_kafka = Module( + name="sql-kafka-0-10", + dependencies=[sql], + source_file_regexes=[ + "external/kafka-0-10-sql", + ], + sbt_test_goals=[ + "sql-kafka-0-10/test", + ] +) + + sketch = Module( name="sketch", dependencies=[tags], diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md new file mode 100644 index 0000000000000..668489addf82c --- /dev/null +++ b/docs/structured-streaming-kafka-integration.md @@ -0,0 +1,239 @@ +--- +layout: global +title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) +--- + +Structured Streaming integration for Kafka 0.10 to poll data from Kafka. + +### Linking +For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact: + + groupId = org.apache.spark + artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +For Python applications, you need to add this above library and its dependencies when deploying your +application. See the [Deploying](#deploying) subsection below. + +### Creating a Kafka Source Stream + +
      +
      + + // Subscribe to 1 topic + val ds1 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() + ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + // Subscribe to multiple topics + val ds2 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() + ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + // Subscribe to a pattern + val ds3 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() + ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +
      +
      + + // Subscribe to 1 topic + Dataset ds1 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() + ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + // Subscribe to multiple topics + Dataset ds2 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() + ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + // Subscribe to a pattern + Dataset ds3 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() + ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +
      +
      + + # Subscribe to 1 topic + ds1 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() + ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + # Subscribe to multiple topics + ds2 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() + ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + + # Subscribe to a pattern + ds3 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() + ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +
      +
      + +Each row in the source has the following schema: +
      Property NameProperty groupspark-submit equivalent
      spark.masterApplication Properties--master
      spark.yarn.keytabApplication Properties--keytab
      spark.yarn.principalApplication Properties--principal
      spark.driver.memory Application Properties
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      ColumnType
      keybinary
      valuebinary
      topicstring
      partitionint
      offsetlong
      timestamplong
      timestampTypeint
      + +The following options must be set for the Kafka source. + + + + + + + + + + + + + + + + + + +
      Optionvaluemeaning
      subscribeA comma-separated list of topicsThe topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be + specified for Kafka source.
      subscribePatternJava regex stringThe pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" + options can be specified for Kafka source.
      kafka.bootstrap.serversA comma-separated list of host:portThe Kafka "bootstrap.servers" configuration.
      + +The following configurations are optional: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      Optionvaluedefaultmeaning
      startingOffset["earliest", "latest"]"latest"The start point when a query is started, either "earliest" which is from the earliest offset, + or "latest" which is just from the latest offset. Note: This only applies when a new Streaming q + uery is started, and that resuming will always pick up from where the query left off.
      failOnDataLoss[true, false]trueWhether to fail the query when it's possible that data is lost (e.g., topics are deleted, or + offsets are out of range). This may be a false alarm. You can disable it when it doesn't work + as you expected.
      kafkaConsumer.pollTimeoutMslong512The timeout in milliseconds to poll data from Kafka in executors.
      fetchOffset.numRetriesint3Number of times to retry before giving up fatch Kafka latest offsets.
      fetchOffset.retryIntervalMslong10milliseconds to wait before retrying to fetch Kafka offsets
      + +Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, +`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafkaParams, see +[Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). + +Note that the following Kafka params cannot be set and the Kafka source will throw an exception: +- **group.id**: Kafka source will create a unique group id for each query automatically. +- **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify + where to start instead. Structured Streaming manages which offsets are consumed internally, rather + than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new + topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new + Streaming query is started, and that resuming will always pick up from where the query left off. +- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use + DataFrame operations to explicitly deserialize the keys. +- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. + Use DataFrame operations to explicitly deserialize the values. +- **enable.auto.commit**: Kafka source doesn't commit any offset. +- **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to + use ConsumerInterceptor as it may break the query. + +### Deploying + +As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` +and its dependencies can be directly added to `spark-submit` using `--packages`, such as, + + ./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + +See [Application Submission Guide](submitting-applications.html) for more details about submitting +applications with external dependencies. diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 94b7854449a35..4f14f3a9450bc 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -418,10 +418,15 @@ Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as Streaming DataFrames can be created through the `DataStreamReader` interface ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/ [Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. In Spark 2.0, there are a few built-in sources. +[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. + +#### Data Sources +In Spark 2.0, there are a few built-in sources. - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. + - **Kafka source** - Poll data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. + - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. Here are some examples. diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml new file mode 100644 index 0000000000000..6aa57a667488d --- /dev/null +++ b/external/kafka-0-10-sql/pom.xml @@ -0,0 +1,96 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.0.2-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-sql-kafka-0-10_2.11 + + sql-kafka-0-10 + + jar + Kafka 0.10 Source for Structured Streaming + http://spark.apache.org/ + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka-clients + 0.10.0.1 + + + org.apache.kafka + kafka_${scala.binary.version} + 0.10.0.1 + test + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister new file mode 100644 index 0000000000000..2f9e9fc0396d5 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -0,0 +1 @@ +org.apache.spark.sql.kafka010.KafkaSourceProvider diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala new file mode 100644 index 0000000000000..3b5a96534f9b6 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -0,0 +1,152 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{SparkEnv, SparkException, TaskContext} +import org.apache.spark.internal.Logging + + +/** + * Consumer of single topicpartition, intended for cached reuse. + * Underlying consumer is not threadsafe, so neither is this, + * but processing the same topicpartition and group id in multiple threads is usually bad anyway. + */ +private[kafka010] case class CachedKafkaConsumer private( + topicPartition: TopicPartition, + kafkaParams: ju.Map[String, Object]) extends Logging { + + private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + + private val consumer = { + val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + val tps = new ju.ArrayList[TopicPartition]() + tps.add(topicPartition) + c.assign(tps) + c + } + + /** Iterator to the already fetch data */ + private var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] + private var nextOffsetInFetchedData = -2L + + /** + * Get the record for the given offset, waiting up to timeout ms if IO is necessary. + * Sequential forward access will use buffers, but random access will be horribly inefficient. + */ + def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[Array[Byte], Array[Byte]] = { + logDebug(s"Get $groupId $topicPartition nextOffset $nextOffsetInFetchedData requested $offset") + if (offset != nextOffsetInFetchedData) { + logInfo(s"Initial fetch for $topicPartition $offset") + seek(offset) + poll(pollTimeoutMs) + } + + if (!fetchedData.hasNext()) { poll(pollTimeoutMs) } + assert(fetchedData.hasNext(), + s"Failed to get records for $groupId $topicPartition $offset " + + s"after polling for $pollTimeoutMs") + var record = fetchedData.next() + + if (record.offset != offset) { + logInfo(s"Buffer miss for $groupId $topicPartition $offset") + seek(offset) + poll(pollTimeoutMs) + assert(fetchedData.hasNext(), + s"Failed to get records for $groupId $topicPartition $offset " + + s"after polling for $pollTimeoutMs") + record = fetchedData.next() + assert(record.offset == offset, + s"Got wrong record for $groupId $topicPartition even after seeking to offset $offset") + } + + nextOffsetInFetchedData = offset + 1 + record + } + + private def close(): Unit = consumer.close() + + private def seek(offset: Long): Unit = { + logDebug(s"Seeking to $groupId $topicPartition $offset") + consumer.seek(topicPartition, offset) + } + + private def poll(pollTimeoutMs: Long): Unit = { + val p = consumer.poll(pollTimeoutMs) + val r = p.records(topicPartition) + logDebug(s"Polled $groupId ${p.partitions()} ${r.size}") + fetchedData = r.iterator + } +} + +private[kafka010] object CachedKafkaConsumer extends Logging { + + private case class CacheKey(groupId: String, topicPartition: TopicPartition) + + private lazy val cache = { + val conf = SparkEnv.get.conf + val capacity = conf.getInt("spark.sql.kafkaConsumerCache.capacity", 64) + new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer](capacity, 0.75f, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = { + if (this.size > capacity) { + logWarning(s"KafkaConsumer cache hitting max capacity of $capacity, " + + s"removing consumer for ${entry.getKey}") + try { + entry.getValue.close() + } catch { + case e: SparkException => + logError(s"Error closing earliest Kafka consumer for ${entry.getKey}", e) + } + true + } else { + false + } + } + } + } + + /** + * Get a cached consumer for groupId, assigned to topic and partition. + * If matching consumer doesn't already exist, will be created using kafkaParams. + */ + def getOrCreate( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = synchronized { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + // If this is reattempt at running the task, then invalidate cache and start with + // a new consumer + if (TaskContext.get != null && TaskContext.get.attemptNumber > 1) { + cache.remove(key) + new CachedKafkaConsumer(topicPartition, kafkaParams) + } else { + if (!cache.containsKey(key)) { + cache.put(key, new CachedKafkaConsumer(topicPartition, kafkaParams)) + } + cache.get(key) + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala new file mode 100644 index 0000000000000..1be70db87497e --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -0,0 +1,399 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer} +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.kafka010.KafkaSource._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.UninterruptibleThread + +/** + * A [[Source]] that uses Kafka's own [[KafkaConsumer]] API to reads data from Kafka. The design + * for this source is as follows. + * + * - The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains + * a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For + * example if the last record in a Kafka topic "t", partition 2 is offset 5, then + * KafkaSourceOffset will contain TopicPartition("t", 2) -> 6. This is done keep it consistent + * with the semantics of `KafkaConsumer.position()`. + * + * - The [[ConsumerStrategy]] class defines which Kafka topics and partitions should be read + * by this source. These strategies directly correspond to the different consumption options + * in . This class is designed to return a configured [[KafkaConsumer]] that is used by the + * [[KafkaSource]] to query for the offsets. See the docs on + * [[org.apache.spark.sql.kafka010.KafkaSource.ConsumerStrategy]] for more details. + * + * - The [[KafkaSource]] written to do the following. + * + * - As soon as the source is created, the pre-configured KafkaConsumer returned by the + * [[ConsumerStrategy]] is used to query the initial offsets that this source should + * start reading from. This used to create the first batch. + * + * - `getOffset()` uses the KafkaConsumer to query the latest available offsets, which are + * returned as a [[KafkaSourceOffset]]. + * + * - `getBatch()` returns a DF that reads from the 'start offset' until the 'end offset' in + * for each partition. The end offset is excluded to be consistent with the semantics of + * [[KafkaSourceOffset]] and `KafkaConsumer.position()`. + * + * - The DF returned is based on [[KafkaSourceRDD]] which is constructed such that the + * data from Kafka topic + partition is consistently read by the same executors across + * batches, and cached KafkaConsumers in the executors can be reused efficiently. See the + * docs on [[KafkaSourceRDD]] for more details. + * + * Zero data lost is not guaranteed when topics are deleted. If zero data lost is critical, the user + * must make sure all messages in a topic have been processed when deleting a topic. + * + * There is a known issue caused by KAFKA-1894: the query using KafkaSource maybe cannot be stopped. + * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers + * and not use wrong broker addresses. + */ +private[kafka010] case class KafkaSource( + sqlContext: SQLContext, + consumerStrategy: ConsumerStrategy, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + metadataPath: String, + failOnDataLoss: Boolean) + extends Source with Logging { + + private val sc = sqlContext.sparkContext + + private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong + + private val maxOffsetFetchAttempts = + sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt + + private val offsetFetchAttemptIntervalMs = + sourceOptions.getOrElse("fetchOffset.retryIntervalMs", "10").toLong + + /** + * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the + * offsets and never commits them. + */ + private val consumer = consumerStrategy.createConsumer() + + /** + * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only + * called in StreamExecutionThread. Otherwise, interrupting a thread while running + * `KafkaConsumer.poll` may hang forever (KAFKA-1894). + */ + private lazy val initialPartitionOffsets = { + val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) + metadataLog.get(0).getOrElse { + val offsets = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = false)) + metadataLog.add(0, offsets) + logInfo(s"Initial offsets: $offsets") + offsets + }.partitionToOffsets + } + + override def schema: StructType = KafkaSource.kafkaSchema + + /** Returns the maximum available offset for this source. */ + override def getOffset: Option[Offset] = { + // Make sure initialPartitionOffsets is initialized + initialPartitionOffsets + + val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = true)) + logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") + Some(offset) + } + + /** + * Returns the data that is between the offsets + * [`start.get.partitionToOffsets`, `end.partitionToOffsets`), i.e. end.partitionToOffsets is + * exclusive. + */ + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + // Make sure initialPartitionOffsets is initialized + initialPartitionOffsets + + logInfo(s"GetBatch called with start = $start, end = $end") + val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) + val fromPartitionOffsets = start match { + case Some(prevBatchEndOffset) => + KafkaSourceOffset.getPartitionOffsets(prevBatchEndOffset) + case None => + initialPartitionOffsets + } + + // Find the new partitions, and get their earliest offsets + val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) + val newPartitionOffsets = if (newPartitions.nonEmpty) { + fetchNewPartitionEarliestOffsets(newPartitions.toSeq) + } else { + Map.empty[TopicPartition, Long] + } + if (newPartitionOffsets.keySet != newPartitions) { + // We cannot get from offsets for some partitions. It means they got deleted. + val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) + reportDataLoss( + s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") + } + logInfo(s"Partitions added: $newPartitionOffsets") + newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => + reportDataLoss( + s"Added partition $p starts from $o instead of 0. Some data may have been missed") + } + + val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) + if (deletedPartitions.nonEmpty) { + reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + } + + // Use the until partitions to calculate offset ranges to ignore partitions that have + // been deleted + val topicPartitions = untilPartitionOffsets.keySet.filter { tp => + // Ignore partitions that we don't know the from offsets. + newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp) + }.toSeq + logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) + + val sortedExecutors = getSortedExecutorList(sc) + val numExecutors = sortedExecutors.length + logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) + + // Calculate offset ranges + val offsetRanges = topicPartitions.map { tp => + val fromOffset = fromPartitionOffsets.get(tp).getOrElse { + newPartitionOffsets.getOrElse(tp, { + // This should not happen since newPartitionOffsets contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset") + }) + } + val untilOffset = untilPartitionOffsets(tp) + val preferredLoc = if (numExecutors > 0) { + // This allows cached KafkaConsumers in the executors to be re-used to read the same + // partition in every batch. + Some(sortedExecutors(floorMod(tp.hashCode, numExecutors))) + } else None + KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) + }.filter { range => + if (range.untilOffset < range.fromOffset) { + reportDataLoss(s"Partition ${range.topicPartition}'s offset was changed from " + + s"${range.fromOffset} to ${range.untilOffset}, some data may have been missed") + false + } else { + true + } + }.toArray + + // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. + val rdd = new KafkaSourceRDD( + sc, executorKafkaParams, offsetRanges, pollTimeoutMs).map { cr => + Row(cr.key, cr.value, cr.topic, cr.partition, cr.offset, cr.timestamp, cr.timestampType.id) + } + + logInfo("GetBatch generating RDD of offset range: " + + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + sqlContext.createDataFrame(rdd, schema) + } + + /** Stop this source and free any resources it has allocated. */ + override def stop(): Unit = synchronized { + consumer.close() + } + + override def toString(): String = s"KafkaSource[$consumerStrategy]" + + /** + * Fetch the offset of a partition, either seek to the latest offsets or use the current offsets + * in the consumer. + */ + private def fetchPartitionOffsets( + seekToEnd: Boolean): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitioned assigned to consumer: $partitions") + + // Get the current or latest offset of each partition + if (seekToEnd) { + consumer.seekToEnd(partitions) + logDebug("Seeked to the end") + } + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got offsets for partition : $partitionOffsets") + partitionOffsets + } + + /** + * Fetch the earliest offsets for newly discovered partitions. The return result may not contain + * some partitions if they are deleted. + */ + private def fetchNewPartitionEarliestOffsets( + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + logDebug(s"\tPartitioned assigned to consumer: $partitions") + + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionToOffsets = newPartitions.filter { p => + // When deleting topics happen at the same time, some partitions may not be in `partitions`. + // So we need to ignore them + partitions.contains(p) + }.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got offsets for new partitions: $partitionToOffsets") + partitionToOffsets + } + + /** + * Helper function that does multiple retries on the a body of code that returns offsets. + * Retries are needed to handle transient failures. For e.g. race conditions between getting + * assignment and getting position while topics/partitions are deleted can cause NPEs. + * + * This method also makes sure `body` won't be interrupted to workaround a potential issue in + * `KafkaConsumer.poll`. (KAFKA-1894) + */ + private def withRetriesWithoutInterrupt( + body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + synchronized { + var result: Option[Map[TopicPartition, Long]] = None + var attempt = 1 + var lastException: Throwable = null + while (result.isEmpty && attempt <= maxOffsetFetchAttempts + && !Thread.currentThread().isInterrupted) { + Thread.currentThread match { + case ut: UninterruptibleThread => + // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query + // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it. + // + // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may + // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the + // issue. + ut.runUninterruptibly { + try { + result = Some(body) + } catch { + case NonFatal(e) => + lastException = e + logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) + attempt += 1 + Thread.sleep(offsetFetchAttemptIntervalMs) + } + } + case _ => + throw new IllegalStateException( + "Kafka APIs must be executed on a o.a.spark.util.UninterruptibleThread") + } + } + if (Thread.interrupted()) { + throw new InterruptedException() + } + if (result.isEmpty) { + assert(attempt > maxOffsetFetchAttempts) + assert(lastException != null) + throw lastException + } + result.get + } + } + + /** + * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. + * Otherwise, just log a warning. + */ + private def reportDataLoss(message: String): Unit = { + if (failOnDataLoss) { + throw new IllegalStateException(message + + ". Set the source option 'failOnDataLoss' to 'false' if you want to ignore these checks.") + } else { + logWarning(message) + } + } +} + + +/** Companion object for the [[KafkaSource]]. */ +private[kafka010] object KafkaSource { + + def kafkaSchema: StructType = StructType(Seq( + StructField("key", BinaryType), + StructField("value", BinaryType), + StructField("topic", StringType), + StructField("partition", IntegerType), + StructField("offset", LongType), + StructField("timestamp", LongType), + StructField("timestampType", IntegerType) + )) + + sealed trait ConsumerStrategy { + def createConsumer(): Consumer[Array[Byte], Array[Byte]] + } + + case class SubscribeStrategy(topics: Seq[String], kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe(topics.asJava) + consumer + } + + override def toString: String = s"Subscribe[${topics.mkString(", ")}]" + } + + case class SubscribePatternStrategy( + topicPattern: String, kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe( + ju.regex.Pattern.compile(topicPattern), + new NoOpConsumerRebalanceListener()) + consumer + } + + override def toString: String = s"SubscribePattern[$topicPattern]" + } + + private def getSortedExecutorList(sc: SparkContext): Array[String] = { + val bm = sc.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compare) + .map(_.toString) + } + + private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } + } + + private def floorMod(a: Long, b: Int): Int = ((a % b).toInt + b) % b +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala new file mode 100644 index 0000000000000..b5ade982515f0 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.sql.execution.streaming.Offset + +/** + * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and + * their offsets. + */ +private[kafka010] +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { + override def toString(): String = { + partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") + } +} + +/** Companion object of the [[KafkaSourceOffset]] */ +private[kafka010] object KafkaSourceOffset { + + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + offset match { + case o: KafkaSourceOffset => o.partitionToOffsets + case _ => + throw new IllegalArgumentException( + s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") + } + } + + /** + * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) + * tuples. + */ + def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { + KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala new file mode 100644 index 0000000000000..1b0a2fe955d03 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -0,0 +1,282 @@ +/* + * 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.kafka010 + +import java.{util => ju} +import java.util.UUID + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.serialization.ByteArrayDeserializer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.execution.streaming.Source +import org.apache.spark.sql.kafka010.KafkaSource._ +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} +import org.apache.spark.sql.types.StructType + +/** + * The provider class for the [[KafkaSource]]. This provider is designed such that it throws + * IllegalArgumentException when the Kafka Dataset is created, so that it can catch + * missing options even before the query is started. + */ +private[kafka010] class KafkaSourceProvider extends StreamSourceProvider + with DataSourceRegister with Logging { + + import KafkaSourceProvider._ + + /** + * Returns the name and schema of the source. In addition, it also verifies whether the options + * are correct and sufficient to create the [[KafkaSource]] when the query is started. + */ + override def sourceSchema( + sqlContext: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") + validateOptions(parameters) + ("kafka", KafkaSource.kafkaSchema) + } + + override def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + validateOptions(parameters) + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase.startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + + val deserClassName = classOf[ByteArrayDeserializer].getName + // Each running query should use its own group id. Otherwise, the query may be only assigned + // partial data since Kafka will assign partitions to multiple consumers having the same group + // id. Hence, we should generate a unique id for each query. + val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" + + val autoOffsetResetValue = caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { + case Some(value) => value.trim() // same values as those supported by auto.offset.reset + case None => "latest" + } + + val kafkaParamsForStrategy = + ConfigUpdater("source", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // So that consumers in Kafka source do not mess with any existing group id + .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver") + + // So that consumers can start from earliest or latest + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetResetValue) + + // So that consumers in the driver does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // So that the driver does not pull too much data + .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1)) + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val kafkaParamsForExecutors = + ConfigUpdater("executor", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // Make sure executors do only what the driver tells them. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // So that consumers in executors do not mess with any existing group id + .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") + + // So that consumers in executors does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("subscribe", value) => + SubscribeStrategy( + value.split(",").map(_.trim()).filter(_.nonEmpty), + kafkaParamsForStrategy) + case ("subscribepattern", value) => + SubscribePatternStrategy( + value.trim(), + kafkaParamsForStrategy) + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + val failOnDataLoss = + caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean + + new KafkaSource( + sqlContext, + strategy, + kafkaParamsForExecutors, + parameters, + metadataPath, + failOnDataLoss) + } + + private def validateOptions(parameters: Map[String, String]): Unit = { + + // Validate source options + + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } + val specifiedStrategies = + caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq + if (specifiedStrategies.isEmpty) { + throw new IllegalArgumentException( + "One of the following options must be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } else if (specifiedStrategies.size > 1) { + throw new IllegalArgumentException( + "Only one of the following options can be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } + + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("subscribe", value) => + val topics = value.split(",").map(_.trim).filter(_.nonEmpty) + if (topics.isEmpty) { + throw new IllegalArgumentException( + "No topics to subscribe to as specified value for option " + + s"'subscribe' is '$value'") + } + case ("subscribepattern", value) => + val pattern = caseInsensitiveParams("subscribepattern").trim() + if (pattern.isEmpty) { + throw new IllegalArgumentException( + "Pattern to subscribe is empty as specified value for option " + + s"'subscribePattern' is '$value'") + } + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { + case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) => + throw new IllegalArgumentException( + s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " + + s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}") + case _ => + } + + // Validate user-specified Kafka options + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + + s"user-specified consumer groups is not used to track offsets.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { + throw new IllegalArgumentException( + s""" + |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. + |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to + |specify where to start. Structured Streaming manages which offsets are consumed + |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no + |data is missed when when new topics/partitions are dynamically subscribed. Note that + |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and + |that resuming will always pick up from where the query left off. See the docs for more + |details. + """.stripMargin) + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " + + "to explicitly deserialize the keys.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " + + "operations to explicitly deserialize the values.") + } + + val otherUnsupportedConfigs = Seq( + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source + ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe + + otherUnsupportedConfigs.foreach { c => + if (caseInsensitiveParams.contains(s"kafka.$c")) { + throw new IllegalArgumentException(s"Kafka option '$c' is not supported") + } + } + + if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { + throw new IllegalArgumentException( + s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " + + s"configuring Kafka consumer") + } + } + + override def shortName(): String = "kafka" + + /** Class to conveniently update Kafka config params, while logging the changes */ + private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + + def set(key: String, value: Object): this.type = { + map.put(key, value) + logInfo(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") + this + } + + def setIfUnset(key: String, value: Object): ConfigUpdater = { + if (!map.containsKey(key)) { + map.put(key, value) + logInfo(s"$module: Set $key to $value") + } + this + } + + def build(): ju.Map[String, Object] = map + } +} + +private[kafka010] object KafkaSourceProvider { + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") + private val STARTING_OFFSET_OPTION_KEY = "startingoffset" + private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") + private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala new file mode 100644 index 0000000000000..496af7e39abab --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -0,0 +1,148 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + + +/** Offset range that one partition of the KafkaSourceRDD has to read */ +private[kafka010] case class KafkaSourceRDDOffsetRange( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long, + preferredLoc: Option[String]) { + def topic: String = topicPartition.topic + def partition: Int = topicPartition.partition + def size: Long = untilOffset - fromOffset +} + + +/** Partition of the KafkaSourceRDD */ +private[kafka010] case class KafkaSourceRDDPartition( + index: Int, offsetRange: KafkaSourceRDDOffsetRange) extends Partition + + +/** + * An RDD that reads data from Kafka based on offset ranges across multiple partitions. + * Additionally, it allows preferred locations to be set for each topic + partition, so that + * the [[KafkaSource]] can ensure the same executor always reads the same topic + partition + * and cached KafkaConsuemrs (see [[CachedKafkaConsumer]] can be used read data efficiently. + * + * @param sc the [[SparkContext]] + * @param executorKafkaParams Kafka configuration for creating KafkaConsumer on the executors + * @param offsetRanges Offset ranges that define the Kafka data belonging to this RDD + */ +private[kafka010] class KafkaSourceRDD( + sc: SparkContext, + executorKafkaParams: ju.Map[String, Object], + offsetRanges: Seq[KafkaSourceRDDOffsetRange], + pollTimeoutMs: Long) + extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { + + override def persist(newLevel: StorageLevel): this.type = { + logError("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => new KafkaSourceRDDPartition(i, o) }.toArray + } + + override def count(): Long = offsetRanges.map(_.size).sum + + override def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[ConsumerRecord[Array[Byte], Array[Byte]]] = { + val nonEmptyPartitions = + this.partitions.map(_.asInstanceOf[KafkaSourceRDDPartition]).filter(_.offsetRange.size > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[ConsumerRecord[Array[Byte], Array[Byte]]](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.offsetRange.size) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]) => + it.take(parts(tc.partitionId)).toArray, parts.keys.toArray + ) + res.foreach(buf ++= _) + buf.toArray + } + + override def compute( + thePart: Partition, + context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { + val range = thePart.asInstanceOf[KafkaSourceRDDPartition].offsetRange + assert( + range.fromOffset <= range.untilOffset, + s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " + + s"for topic ${range.topic} partition ${range.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged") + if (range.fromOffset == range.untilOffset) { + logInfo(s"Beginning offset ${range.fromOffset} is the same as ending offset " + + s"skipping ${range.topic} ${range.partition}") + Iterator.empty + + } else { + + val consumer = CachedKafkaConsumer.getOrCreate( + range.topic, range.partition, executorKafkaParams) + var requestOffset = range.fromOffset + + logDebug(s"Creating iterator for $range") + + new Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { + override def hasNext(): Boolean = requestOffset < range.untilOffset + override def next(): ConsumerRecord[Array[Byte], Array[Byte]] = { + assert(hasNext(), "Can't call next() once untilOffset has been reached") + val r = consumer.get(requestOffset, pollTimeoutMs) + requestOffset += 1 + r + } + } + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java new file mode 100644 index 0000000000000..596f775c56dbc --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java @@ -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. + */ + +/** + * Structured Streaming Data Source for Kafka 0.10 + */ +package org.apache.spark.sql.kafka010; diff --git a/external/kafka-0-10-sql/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..75e3b53a093f6 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/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=target/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.spark-project.jetty=WARN + diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala new file mode 100644 index 0000000000000..7056a41b1751e --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala @@ -0,0 +1,39 @@ +/* + * 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.kafka010 + +import org.apache.spark.sql.streaming.OffsetSuite + +class KafkaSourceOffsetSuite extends OffsetSuite { + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("t", 1, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("T", 0, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("T", 0, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala new file mode 100644 index 0000000000000..6c03070398fca --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -0,0 +1,425 @@ +/* + * 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.kafka010 + +import java.util.concurrent.atomic.AtomicInteger + +import scala.util.Random + +import org.apache.kafka.clients.producer.RecordMetadata +import org.scalatest.BeforeAndAfter +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.test.SharedSQLContext + + +abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { + + protected var testUtils: KafkaTestUtils = _ + + override val streamingTimeout = 30.seconds + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + protected def makeSureGetOffsetCalled = AssertOnQuery { q => + // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure + // its "getOffset" is called before pushing any data. Otherwise, because of the race contion, + // we don't know which data should be fetched when `startingOffset` is latest. + q.processAllAvailable() + true + } + + /** + * Add data to Kafka. + * + * `topicAction` can be used to run actions for each topic before inserting data. + */ + case class AddKafkaData(topics: Set[String], data: Int*) + (implicit ensureDataInMultiplePartition: Boolean = false, + concurrent: Boolean = false, + message: String = "", + topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends AddData { + + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + if (query.get.isActive) { + // Make sure no Spark job is running when deleting a topic + query.get.processAllAvailable() + } + + val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap + val newTopics = topics.diff(existingTopics.keySet) + for (newTopic <- newTopics) { + topicAction(newTopic, None) + } + for (existingTopicPartitions <- existingTopics) { + topicAction(existingTopicPartitions._1, Some(existingTopicPartitions._2)) + } + + // Read all topics again in case some topics are delete. + val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys + require( + query.nonEmpty, + "Cannot add data when there is no query for finding the active kafka source") + + val sources = query.get.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[KafkaSource] => + source.asInstanceOf[KafkaSource] + } + if (sources.isEmpty) { + throw new Exception( + "Could not find Kafka source in the StreamExecution logical plan to add data to") + } else if (sources.size > 1) { + throw new Exception( + "Could not select the Kafka source in the StreamExecution logical plan as there" + + "are multiple Kafka sources:\n\t" + sources.mkString("\n\t")) + } + val kafkaSource = sources.head + val topic = topics.toSeq(Random.nextInt(topics.size)) + val sentMetadata = testUtils.sendMessages(topic, data.map { _.toString }.toArray) + + def metadataToStr(m: (String, RecordMetadata)): String = { + s"Sent ${m._1} to partition ${m._2.partition()}, offset ${m._2.offset()}" + } + // Verify that the test data gets inserted into multiple partitions + if (ensureDataInMultiplePartition) { + require( + sentMetadata.groupBy(_._2.partition).size > 1, + s"Added data does not test multiple partitions: ${sentMetadata.map(metadataToStr)}") + } + + val offset = KafkaSourceOffset(testUtils.getLatestOffsets(topics)) + logInfo(s"Added data, expected offset $offset") + (kafkaSource, offset) + } + + override def toString: String = + s"AddKafkaData(topics = $topics, data = $data, message = $message)" + } +} + + +class KafkaSourceSuite extends KafkaSourceTest { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + + test("cannot stop Kafka stream") { + val topic = newTopic() + testUtils.createTopic(newTopic(), partitions = 5) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"topic-.*") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + StopStream + ) + } + + test("subscribing topic by name from latest offsets") { + val topic = newTopic() + testFromLatestOffsets(topic, "subscribe" -> topic) + } + + test("subscribing topic by name from earliest offsets") { + val topic = newTopic() + testFromEarliestOffsets(topic, "subscribe" -> topic) + } + + test("subscribing topic by pattern from latest offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromLatestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("subscribing topic by pattern from earliest offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromEarliestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("subscribing topic by pattern with topic deletions") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-seems" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"$topicPrefix-.*") + .option("failOnDataLoss", "false") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + Assert { + testUtils.deleteTopic(topic) + testUtils.createTopic(topic2, partitions = 5) + true + }, + AddKafkaData(Set(topic2), 4, 5, 6), + CheckAnswer(2, 3, 4, 5, 6, 7) + ) + } + + test("bad source options") { + def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + options.foreach { case (k, v) => reader.option(k, v) } + reader.load() + } + expectedMsgs.foreach { m => + assert(ex.getMessage.toLowerCase.contains(m.toLowerCase)) + } + } + + // No strategy specified + testBadOptions()("options must be specified", "subscribe", "subscribePattern") + + // Multiple strategies specified + testBadOptions("subscribe" -> "t", "subscribePattern" -> "t.*")( + "only one", "options can be specified") + + testBadOptions("subscribe" -> "")("no topics to subscribe") + testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") + } + + test("unsupported kafka configs") { + def testUnsupportedConfig(key: String, value: String = "someValue"): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + .option("subscribe", "topic") + .option("kafka.bootstrap.servers", "somehost") + .option(s"$key", value) + reader.load() + } + assert(ex.getMessage.toLowerCase.contains("not supported")) + } + + testUnsupportedConfig("kafka.group.id") + testUnsupportedConfig("kafka.auto.offset.reset") + testUnsupportedConfig("kafka.enable.auto.commit") + testUnsupportedConfig("kafka.interceptor.classes") + testUnsupportedConfig("kafka.key.deserializer") + testUnsupportedConfig("kafka.value.deserializer") + + testUnsupportedConfig("kafka.auto.offset.reset", "none") + testUnsupportedConfig("kafka.auto.offset.reset", "someValue") + testUnsupportedConfig("kafka.auto.offset.reset", "earliest") + testUnsupportedConfig("kafka.auto.offset.reset", "latest") + } + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffset", s"latest") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4), // Should get the data back on recovery + StopStream, + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), // Should get the added data + AddKafkaData(Set(topic), 7, 8), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + testUtils.addPartitions(topic, 10) + true + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) + ) + } + + private def testFromEarliestOffsets(topic: String, options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, (1 to 3).map { _.toString }.toArray) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark.readStream + reader + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("startingOffset", s"earliest") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + AddKafkaData(Set(topic), 7, 8), + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + testUtils.addPartitions(topic, 10) + true + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) + ) + } +} + + +class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { + + import testImplicits._ + + val topicId = new AtomicInteger(1) + + @volatile var topics: Seq[String] = (1 to 5).map(_ => newStressTopic) + + def newStressTopic: String = s"stress${topicId.getAndIncrement()}" + + private def nextInt(start: Int, end: Int): Int = { + start + Random.nextInt(start + end - 1) + } + + after { + for (topic <- testUtils.getAllTopicsAndPartitionSize().toMap.keys) { + testUtils.deleteTopic(topic) + } + } + + test("stress test with multiple topics and partitions") { + topics.foreach { topic => + testUtils.createTopic(topic, partitions = nextInt(1, 6)) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + } + + // Create Kafka source that reads from latest offset + val kafka = + spark.readStream + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "stress.*") + .option("failOnDataLoss", "false") + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + + runStressTest( + mapped, + Seq(makeSureGetOffsetCalled), + (d, running) => { + Random.nextInt(5) match { + case 0 => // Add a new topic + topics = topics ++ Seq(newStressTopic) + AddKafkaData(topics.toSet, d: _*)(message = s"Add topic $newStressTopic", + topicAction = (topic, partition) => { + if (partition.isEmpty) { + testUtils.createTopic(topic, partitions = nextInt(1, 6)) + } + }) + case 1 if running => + // Only delete a topic when the query is running. Otherwise, we may lost data and + // cannot check the correctness. + val deletedTopic = topics(Random.nextInt(topics.size)) + if (deletedTopic != topics.head) { + topics = topics.filterNot(_ == deletedTopic) + } + AddKafkaData(topics.toSet, d: _*)(message = s"Delete topic $deletedTopic", + topicAction = (topic, partition) => { + // Never remove the first topic to make sure we have at least one topic + if (topic == deletedTopic && deletedTopic != topics.head) { + testUtils.deleteTopic(deletedTopic) + } + }) + case 2 => // Add new partitions + AddKafkaData(topics.toSet, d: _*)(message = "Add partitiosn", + topicAction = (topic, partition) => { + testUtils.addPartitions(topic, partition.get + nextInt(1, 6)) + }) + case _ => // Just add new data + AddKafkaData(topics.toSet, d: _*) + } + }, + iterations = 50) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala new file mode 100644 index 0000000000000..3eb8a737ba4c8 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -0,0 +1,339 @@ +/* + * 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.kafka010 + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.language.postfixOps +import scala.util.Random + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.common.TopicAndPartition +import kafka.server.{KafkaConfig, KafkaServer, OffsetCheckpoint} +import kafka.utils.ZkUtils +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils +import org.apache.spark.SparkConf + +/** + * 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. + */ +class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkUtils: ZkUtils = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 0 + 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: ZkUtils = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkUtils).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 + zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + 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, doLog = false) + server = new KafkaServer(brokerConf) + server.startup() + brokerPort = server.boundPort() + (server, brokerPort) + }, 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 (zkUtils != null) { + zkUtils.close() + zkUtils = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkUtils, topic, partitions, 1) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + def getAllTopicsAndPartitionSize(): Seq[(String, Int)] = { + zkUtils.getPartitionsForTopics(zkUtils.getAllTopics()).mapValues(_.size).toSeq + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + createTopic(topic, 1) + } + + /** Delete a Kafka topic and wait until it is propagated to the whole cluster */ + def deleteTopic(topic: String): Unit = { + val partitions = zkUtils.getPartitionsForTopics(Seq(topic))(topic).size + AdminUtils.deleteTopic(zkUtils, topic) + verifyTopicDeletion(zkUtils, topic, partitions, List(this.server)) + } + + /** Add new paritions to a Kafka topic */ + def addPartitions(topic: String, partitions: Int): Unit = { + AdminUtils.addPartitions(zkUtils, topic, partitions) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.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]): Seq[(String, RecordMetadata)] = { + producer = new KafkaProducer[String, String](producerConfiguration) + val offsets = try { + messages.map { m => + val metadata = + producer.send(new ProducerRecord[String, String](topic, m)).get(10, TimeUnit.SECONDS) + logInfo(s"\tSent $m to partition ${metadata.partition}, offset ${metadata.offset}") + (m, metadata) + } + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + offsets + } + + def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { + val kc = new KafkaConsumer[String, String](consumerConfiguration) + logInfo("Created consumer to get latest offsets") + kc.subscribe(topics.asJavaCollection) + kc.poll(0) + val partitions = kc.assignment() + kc.pause(partitions) + kc.seekToEnd(partitions) + val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap + kc.close() + logInfo("Closed consumer to get latest offsets") + offsets + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("advertised.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.put("delete.topic.enable", "true") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("value.serializer", classOf[StringSerializer].getName) + props.put("key.serializer", classOf[StringSerializer].getName) + // wait for all in-sync replicas to ack sends + props.put("acks", "all") + props + } + + private def consumerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("group.id", "group-KafkaTestUtils-" + Random.nextInt) + props.put("value.deserializer", classOf[StringDeserializer].getName) + props.put("key.deserializer", classOf[StringDeserializer].getName) + props.put("enable.auto.commit", "false") + props + } + + private def verifyTopicDeletion( + zkUtils: ZkUtils, + topic: String, + numPartitions: Int, + servers: Seq[KafkaServer]) { + import ZkUtils._ + val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _)) + def isDeleted(): Boolean = { + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + val deletePath = !zkUtils.pathExists(getDeleteTopicPath(topic)) + val topicPath = !zkUtils.pathExists(getTopicPath(topic)) + // ensure that the topic-partition has been deleted from all brokers' replica managers + val replicaManager = servers.forall(server => topicAndPartitions.forall(tp => + server.replicaManager.getPartition(tp.topic, tp.partition) == None)) + // ensure that logs from all replicas are deleted if delete topic is marked successful + val logManager = servers.forall(server => topicAndPartitions.forall(tp => + server.getLogManager().getLog(tp).isEmpty)) + // ensure that topic is removed from all cleaner offsets + val cleaner = servers.forall(server => topicAndPartitions.forall { tp => + val checkpoints = server.getLogManager().logDirs.map { logDir => + new OffsetCheckpoint(new File(logDir, "cleaner-offset-checkpoint")).read() + } + checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) + }) + deletePath && topicPath && replicaManager && logManager && cleaner + } + eventually(timeout(10.seconds)) { + assert(isDeleted, s"$topic not deleted after timeout") + } + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + zkUtils.getLeaderForPartition(topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.size >= 1 + + case _ => + false + } + eventually(timeout(10.seconds)) { + assert(isPropagated, 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/pom.xml b/pom.xml index bb4b8a0de34d4..90d5ca0394d73 100644 --- a/pom.xml +++ b/pom.xml @@ -111,6 +111,7 @@ external/kafka-0-8-assembly external/kafka-0-10 external/kafka-0-10-assembly + external/kafka-0-10-sql diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 133d3b390e91a..7f7a65f552713 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -39,8 +39,8 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer) = Seq( - "catalyst", "sql", "hive", "hive-thriftserver" + val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, sqlKafka010) = Seq( + "catalyst", "sql", "hive", "hive-thriftserver", "sql-kafka-0-10" ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( @@ -352,7 +352,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags + unsafe, tags, sqlKafka010 ).contains(x) } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 5e67226c6a61a..6a4ec7a47f85c 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -133,6 +133,33 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + test-jar-on-test-compile + test-compile + + test-jar + + + + org.codehaus.mojo build-helper-maven-plugin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index da1ae05fd0d34..8e0688d6bf539 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -116,7 +116,7 @@ class StreamExecution( * [[HDFSMetadataLog]]. See SPARK-14131 for more details. */ val microBatchThread = - new UninterruptibleThread(s"stream execution thread for $name") { + new StreamExecutionThread(s"stream execution thread for $name") { override def run(): Unit = { // To fix call site like "run at :0", we bridge the call site from the caller // thread to this micro batch thread @@ -535,3 +535,9 @@ object StreamExecution { def nextId: Long = _nextId.getAndIncrement() } + +/** + * A special thread to run the stream query. Some codes require to run in the StreamExecutionThread + * and will use `classOf[StreamExecutionThread]` to check. + */ +abstract class StreamExecutionThread(name: String) extends UninterruptibleThread(name) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 1d467c6c5edd4..d4c64e494a741 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -50,11 +50,11 @@ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} * * {{{ * val inputData = MemoryStream[Int] - val mapped = inputData.toDS().map(_ + 1) - - testStream(mapped)( - AddData(inputData, 1, 2, 3), - CheckAnswer(2, 3, 4)) + * val mapped = inputData.toDS().map(_ + 1) + * + * testStream(mapped)( + * AddData(inputData, 1, 2, 3), + * CheckAnswer(2, 3, 4)) * }}} * * Note that while we do sleep to allow the other thread to progress without spinning, @@ -468,21 +468,41 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } + + /** + * Creates a stress test that randomly starts/stops/adds data/checks the result. + * + * @param ds a dataframe that executes + 1 on a stream of integers, returning the result + * @param addData an add data action that adds the given numbers to the stream, encoding them + * as needed + * @param iterations the iteration number + */ + def runStressTest( + ds: Dataset[Int], + addData: Seq[Int] => StreamAction, + iterations: Int = 100): Unit = { + runStressTest(ds, Seq.empty, (data, running) => addData(data), iterations) + } + /** * Creates a stress test that randomly starts/stops/adds data/checks the result. * - * @param ds a dataframe that executes + 1 on a stream of integers, returning the result. - * @param addData and add data action that adds the given numbers to the stream, encoding them + * @param ds a dataframe that executes + 1 on a stream of integers, returning the result + * @param prepareActions actions need to run before starting the stress test. + * @param addData an add data action that adds the given numbers to the stream, encoding them * as needed + * @param iterations the iteration number */ def runStressTest( ds: Dataset[Int], - addData: Seq[Int] => StreamAction, - iterations: Int = 100): Unit = { + prepareActions: Seq[StreamAction], + addData: (Seq[Int], Boolean) => StreamAction, + iterations: Int): Unit = { implicit val intEncoder = ExpressionEncoder[Int]() var dataPos = 0 var running = true val actions = new ArrayBuffer[StreamAction]() + actions ++= prepareActions def addCheck() = { actions += CheckAnswer(1 to dataPos: _*) } @@ -490,7 +510,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val numItems = Random.nextInt(10) val data = dataPos until (dataPos + numItems) dataPos += numItems - actions += addData(data) + actions += addData(data, running) } (1 to iterations).foreach { i => From a84d8ef375f853c5841d458a593e41b457b9e6ff Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Fri, 7 Oct 2016 11:46:39 +0100 Subject: [PATCH 1399/1470] [SPARK-17782][STREAMING][BUILD] Add Kafka 0.10 project to build modules ## What changes were proposed in this pull request? This PR adds the Kafka 0.10 subproject to the build infrastructure. This makes sure Kafka 0.10 tests are only triggers when it or of its dependencies change. Author: Herman van Hovell Closes #15355 from hvanhovell/SPARK-17782. --- dev/sparktestsupport/modules.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 5da1950bd987d..c5d1a07782221 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -241,6 +241,17 @@ def __hash__(self): ] ) +streaming_kafka_0_10 = Module( + name="streaming-kafka-0-10", + dependencies=[streaming], + source_file_regexes=[ + "external/kafka-0-10", + "external/kafka-0-10-assembly", + ], + sbt_test_goals=[ + "streaming-kafka-0-10/test", + ] +) streaming_flume_sink = Module( name="streaming-flume-sink", From 6d056c168c45d2decf5ffbb96d59623d52ed8490 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 7 Oct 2016 15:03:47 -0700 Subject: [PATCH 1400/1470] [SPARK-17806] [SQL] fix bug in join key rewritten in HashJoin ## What changes were proposed in this pull request? In HashJoin, we try to rewrite the join key as Long to improve the performance of finding a match. The rewriting part is not well tested, has a bug that could cause wrong result when there are at least three integral columns in the joining key also the total length of the key exceed 8 bytes. ## How was this patch tested? Added unit test to covering the rewriting with different number of columns and different data types. Manually test the reported case and confirmed that this PR fix the bug. Author: Davies Liu Closes #15390 from davies/rewrite_key. (cherry picked from commit 94b24b84a666517e31e9c9d693f92d9bbfd7f9ad) Signed-off-by: Davies Liu --- .../spark/sql/execution/joins/HashJoin.scala | 65 +++++++++---------- .../execution/joins/BroadcastJoinSuite.scala | 47 ++++++++++++++ 2 files changed, 79 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index d46a80423fa35..d11f7e623b5f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -63,45 +63,16 @@ trait HashJoin { protected lazy val (buildKeys, streamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") - val lkeys = rewriteKeyExpr(leftKeys).map(BindReferences.bindReference(_, left.output)) - val rkeys = rewriteKeyExpr(rightKeys).map(BindReferences.bindReference(_, right.output)) + val lkeys = HashJoin.rewriteKeyExpr(leftKeys).map(BindReferences.bindReference(_, left.output)) + val rkeys = HashJoin.rewriteKeyExpr(rightKeys) + .map(BindReferences.bindReference(_, right.output)) buildSide match { case BuildLeft => (lkeys, rkeys) case BuildRight => (rkeys, lkeys) } } - /** - * Try to rewrite the key as LongType so we can use getLong(), if they key can fit with a long. - * - * If not, returns the original expressions. - */ - private def rewriteKeyExpr(keys: Seq[Expression]): Seq[Expression] = { - var keyExpr: Expression = null - var width = 0 - keys.foreach { e => - e.dataType match { - case dt: IntegralType if dt.defaultSize <= 8 - width => - if (width == 0) { - if (e.dataType != LongType) { - keyExpr = Cast(e, LongType) - } else { - keyExpr = e - } - width = dt.defaultSize - } else { - val bits = dt.defaultSize * 8 - keyExpr = BitwiseOr(ShiftLeft(keyExpr, Literal(bits)), - BitwiseAnd(Cast(e, LongType), Literal((1L << bits) - 1))) - width -= bits - } - // TODO: support BooleanType, DateType and TimestampType - case other => - return keys - } - } - keyExpr :: Nil - } + protected def buildSideKeyGenerator(): Projection = UnsafeProjection.create(buildKeys) @@ -247,3 +218,31 @@ trait HashJoin { } } } + +object HashJoin { + /** + * Try to rewrite the key as LongType so we can use getLong(), if they key can fit with a long. + * + * If not, returns the original expressions. + */ + private[joins] def rewriteKeyExpr(keys: Seq[Expression]): Seq[Expression] = { + assert(keys.nonEmpty) + // TODO: support BooleanType, DateType and TimestampType + if (keys.exists(!_.dataType.isInstanceOf[IntegralType]) + || keys.map(_.dataType.defaultSize).sum > 8) { + return keys + } + + var keyExpr: Expression = if (keys.head.dataType != LongType) { + Cast(keys.head, LongType) + } else { + keys.head + } + keys.tail.foreach { e => + val bits = e.dataType.defaultSize * 8 + keyExpr = BitwiseOr(ShiftLeft(keyExpr, Literal(bits)), + BitwiseAnd(Cast(e, LongType), Literal((1L << bits) - 1))) + } + keyExpr :: Nil + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index b679e3bb328af..c22c106fdfece 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -23,11 +23,13 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext} import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.{LongType, ShortType} /** * Test various broadcast join operators. @@ -156,4 +158,49 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { cases.foreach(assertBroadcastJoin) } } + + test("join key rewritten") { + val l = Literal(1L) + val i = Literal(2) + val s = Literal.create(3, ShortType) + val ss = Literal("hello") + + assert(HashJoin.rewriteKeyExpr(l :: Nil) === l :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: l :: Nil) === l :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: i :: Nil) === l :: i :: Nil) + + assert(HashJoin.rewriteKeyExpr(i :: Nil) === Cast(i, LongType) :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: l :: Nil) === i :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: i :: Nil) === + BitwiseOr(ShiftLeft(Cast(i, LongType), Literal(32)), + BitwiseAnd(Cast(i, LongType), Literal((1L << 32) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: i :: i :: Nil) === i :: i :: i :: Nil) + + assert(HashJoin.rewriteKeyExpr(s :: Nil) === Cast(s, LongType) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: l :: Nil) === s :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: Nil) === + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: Nil) === + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: s :: Nil) === + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: s :: s :: Nil) === + s :: s :: s :: s :: s :: Nil) + + assert(HashJoin.rewriteKeyExpr(ss :: Nil) === ss :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: ss :: Nil) === l :: ss :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: ss :: Nil) === i :: ss :: Nil) + } } From d27df35795fac0fd167e51d5ba08092a17eedfc2 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Sun, 9 Oct 2016 21:52:46 -0700 Subject: [PATCH 1401/1470] [SPARK-17832][SQL] TableIdentifier.quotedString creates un-parseable names when name contains a backtick ## What changes were proposed in this pull request? The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example: ``` import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1")) parseTableIdentifier(complexName.unquotedString) // Does not work parseTableIdentifier(complexName.quotedString) // Does not work parseExpression(complexName.unquotedString) // Does not work parseExpression(complexName.quotedString) // Does not work ``` We should handle the backtick properly to make `quotedString` parseable. ## How was this patch tested? Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`. Author: jiangxingbo Closes #15403 from jiangxb1987/backtick. (cherry picked from commit 26fbca480604ba258f97b9590cfd6dda1ecd31db) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/catalyst/identifiers.scala | 11 +++++++++-- .../sql/catalyst/parser/ExpressionParserSuite.scala | 11 ++++++++++- .../catalyst/parser/TableIdentifierParserSuite.scala | 10 ++++++++++ 3 files changed, 29 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index d7b48ceca591a..834897b85023d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst - /** * An identifier that optionally specifies a database. * @@ -29,8 +28,16 @@ sealed trait IdentifierWithDatabase { def database: Option[String] + /* + * Escapes back-ticks within the identifier name with double-back-ticks. + */ + private def quoteIdentifier(name: String): String = name.replace("`", "``") + def quotedString: String = { - if (database.isDefined) s"`${database.get}`.`$identifier`" else s"`$identifier`" + val replacedId = quoteIdentifier(identifier) + val replacedDb = database.map(quoteIdentifier(_)) + + if (replacedDb.isDefined) s"`${replacedDb.get}`.`$replacedId`" else s"`$replacedId`" } def unquotedString: String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index c89ddadc486d5..15ebc196e0f2b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -534,4 +534,13 @@ class ExpressionParserSuite extends PlanTest { // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL assertEqual("a.123BD_column", UnresolvedAttribute("a.123BD_column")) } + + test("SPARK-17832 function identifier contains backtick") { + val complexName = FunctionIdentifier("`ba`r", Some("`fo`o")) + assertEqual(complexName.quotedString, UnresolvedAttribute("`fo`o.`ba`r")) + intercept(complexName.unquotedString, "mismatched input") + // Function identifier contains countious backticks should be treated correctly. + val complexName2 = FunctionIdentifier("ba``r", Some("fo``o")) + assertEqual(complexName2.quotedString, UnresolvedAttribute("fo``o.ba``r")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 4d3ad2179139b..cb57fb69b2555 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -96,4 +96,14 @@ class TableIdentifierParserSuite extends SparkFunSuite { // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL assert(parseTableIdentifier("a.123BD_LIST") == TableIdentifier("123BD_LIST", Some("a"))) } + + test("SPARK-17832 table identifier - contains backtick") { + val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1")) + assert(complexName === parseTableIdentifier("```d``b``1`.```weird``table``name`")) + assert(complexName === parseTableIdentifier(complexName.quotedString)) + intercept[ParseException](parseTableIdentifier(complexName.unquotedString)) + // Table identifier contains countious backticks should be treated correctly. + val complexName2 = TableIdentifier("x``y", Some("d``b")) + assert(complexName2 === parseTableIdentifier(complexName2.quotedString)) + } } From d719e9a080a909a6a56db938750d553668743f8f Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Mon, 10 Oct 2016 10:55:57 -0500 Subject: [PATCH 1402/1470] [SPARK-17417][CORE] Fix # of partitions for Reliable RDD checkpointing ## What changes were proposed in this pull request? Currently the no. of partition files are limited to 10000 files (%05d format). If there are more than 10000 part files, the logic goes for a toss while recreating the RDD as it sorts them by string. More details can be found in the JIRA desc [here](https://issues.apache.org/jira/browse/SPARK-17417). ## How was this patch tested? I tested this patch by checkpointing a RDD and then manually renaming part files to the old format and tried to access the RDD. It was successfully created from the old format. Also verified loading a sample parquet file and saving it as multiple formats - CSV, JSON, Text, Parquet, ORC and read them successfully back from the saved files. I couldn't launch the unit test from my local box, so will wait for the Jenkins output. Author: Dhruve Ashar Closes #15370 from dhruve/bug/SPARK-17417. (cherry picked from commit 4bafacaa5f50a3e986c14a38bc8df9bae303f3a0) Signed-off-by: Tom Graves --- .../scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index fddb9353018a8..b73214fe91c4e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -69,10 +69,10 @@ private[spark] class ReliableCheckpointRDD[T: ClassTag]( val inputFiles = fs.listStatus(cpath) .map(_.getPath) .filter(_.getName.startsWith("part-")) - .sortBy(_.toString) + .sortBy(_.getName.stripPrefix("part-").toInt) // Fail fast if input files are invalid inputFiles.zipWithIndex.foreach { case (path, i) => - if (!path.toString.endsWith(ReliableCheckpointRDD.checkpointFileName(i))) { + if (path.getName != ReliableCheckpointRDD.checkpointFileName(i)) { throw new SparkException(s"Invalid checkpoint file: $path") } } From ff9f5bbf1795d9f5b14838099dcc1bb4ac8a9b5b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 10 Oct 2016 19:14:01 -0700 Subject: [PATCH 1403/1470] [SPARK-17738][TEST] Fix flaky test in ColumnTypeSuite ## What changes were proposed in this pull request? The default buffer size is not big enough for randomly generated MapType. ## How was this patch tested? Ran the tests in 100 times, it never fail (it fail 8 times before the patch). Author: Davies Liu Closes #15395 from davies/flaky_map. (cherry picked from commit d5ec4a3e014494a3d991a6350caffbc3b17be0fd) Signed-off-by: Shixiong Zhu --- .../spark/sql/execution/columnar/ColumnTypeSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 052f4cbaebc8e..6e5c22dab52f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -101,14 +101,15 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { def testColumnType[JvmType](columnType: ColumnType[JvmType]): Unit = { - val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE).order(ByteOrder.nativeOrder()) val proj = UnsafeProjection.create(Array[DataType](columnType.dataType)) val converter = CatalystTypeConverters.createToScalaConverter(columnType.dataType) val seq = (0 until 4).map(_ => proj(makeRandomRow(columnType)).copy()) + val totalSize = seq.map(_.getSizeInBytes).sum + val bufferSize = Math.max(DEFAULT_BUFFER_SIZE, totalSize) test(s"$columnType append/extract") { - buffer.rewind() - seq.foreach(columnType.append(_, 0, buffer)) + val buffer = ByteBuffer.allocate(bufferSize).order(ByteOrder.nativeOrder()) + seq.foreach(r => columnType.append(columnType.getField(r, 0), buffer)) buffer.rewind() seq.foreach { row => From a6b5e1dccf0be0e709d6d4113cdacb0cecce39fd Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 11 Oct 2016 10:53:07 -0700 Subject: [PATCH 1404/1470] [SPARK-17346][SQL][TESTS] Fix the flaky topic deletion in KafkaSourceStressSuite ## What changes were proposed in this pull request? A follow up Pr for SPARK-17346 to fix flaky `org.apache.spark.sql.kafka010.KafkaSourceStressSuite`. Test log: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/1855/testReport/junit/org.apache.spark.sql.kafka010/KafkaSourceStressSuite/_It_is_not_a_test_/ Looks like deleting the Kafka internal topic `__consumer_offsets` is flaky. This PR just simply ignores internal topics. ## How was this patch tested? Existing tests. Author: Shixiong Zhu Closes #15384 from zsxwing/SPARK-17346-flaky-test. (cherry picked from commit 75b9e351413dca0930e8545e6283874db09d8482) Signed-off-by: Tathagata Das --- .../org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 6c03070398fca..c640b93b0a2ee 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata -import org.scalatest.BeforeAndAfter import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ @@ -344,7 +343,7 @@ class KafkaSourceSuite extends KafkaSourceTest { } -class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { +class KafkaSourceStressSuite extends KafkaSourceTest { import testImplicits._ @@ -358,12 +357,6 @@ class KafkaSourceStressSuite extends KafkaSourceTest with BeforeAndAfter { start + Random.nextInt(start + end - 1) } - after { - for (topic <- testUtils.getAllTopicsAndPartitionSize().toMap.keys) { - testUtils.deleteTopic(topic) - } - } - test("stress test with multiple topics and partitions") { topics.foreach { topic => testUtils.createTopic(topic, partitions = nextInt(1, 6)) From 5ec3e6680a091883369c002ae599d6b03f38c863 Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Tue, 11 Oct 2016 12:51:08 -0700 Subject: [PATCH 1405/1470] [SPARK-17816][CORE][BRANCH-2.0] Fix ConcurrentModificationException issue in BlockStatusesAccumulator ## What changes were proposed in this pull request? Replaced `BlockStatusesAccumulator` with `CollectionAccumulator` which is thread safe and few more cleanups. ## How was this patch tested? Tested in master branch and cherry-picked. Author: Ergin Seyfe Closes #15425 from seyfe/race_cond_jsonprotocal_branch-2.0. --- .../apache/spark/executor/TaskMetrics.scala | 42 +------------------ .../org/apache/spark/util/AccumulatorV2.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- 3 files changed, 6 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 52a349919e336..47aec44bac019 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,8 +17,6 @@ package org.apache.spark.executor -import java.util.{ArrayList, Collections} - import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} @@ -27,7 +25,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} -import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} +import org.apache.spark.util._ /** @@ -54,7 +52,7 @@ class TaskMetrics private[spark] () extends Serializable { private val _memoryBytesSpilled = new LongAccumulator private val _diskBytesSpilled = new LongAccumulator private val _peakExecutionMemory = new LongAccumulator - private val _updatedBlockStatuses = new BlockStatusesAccumulator + private val _updatedBlockStatuses = new CollectionAccumulator[(BlockId, BlockStatus)] /** * Time taken on the executor to deserialize this task. @@ -305,39 +303,3 @@ private[spark] object TaskMetrics extends Logging { tm } } - - -private[spark] class BlockStatusesAccumulator - extends AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]] { - private val _seq = Collections.synchronizedList(new ArrayList[(BlockId, BlockStatus)]()) - - override def isZero(): Boolean = _seq.isEmpty - - override def copyAndReset(): BlockStatusesAccumulator = new BlockStatusesAccumulator - - override def copy(): BlockStatusesAccumulator = { - val newAcc = new BlockStatusesAccumulator - newAcc._seq.addAll(_seq) - newAcc - } - - override def reset(): Unit = _seq.clear() - - override def add(v: (BlockId, BlockStatus)): Unit = _seq.add(v) - - override def merge( - other: AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]]): Unit = { - other match { - case o: BlockStatusesAccumulator => _seq.addAll(o.value) - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - } - - override def value: java.util.List[(BlockId, BlockStatus)] = _seq - - def setValue(newValue: java.util.List[(BlockId, BlockStatus)]): Unit = { - _seq.clear() - _seq.addAll(newValue) - } -} diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index 470d912ecff13..d3ddd39131326 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -444,7 +444,9 @@ class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { override def copy(): CollectionAccumulator[T] = { val newAcc = new CollectionAccumulator[T] - newAcc._list.addAll(_list) + _list.synchronized { + newAcc._list.addAll(_list) + } newAcc } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 148635f5d2414..8861696cd97d7 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -280,7 +280,7 @@ private[spark] object JsonProtocol { ("Getting Result Time" -> taskInfo.gettingResultTime) ~ ("Finish Time" -> taskInfo.finishTime) ~ ("Failed" -> taskInfo.failed) ~ - ("Accumulables" -> JArray(taskInfo.accumulables.map(accumulableInfoToJson).toList)) + ("Accumulables" -> JArray(taskInfo.accumulables.toList.map(accumulableInfoToJson))) } def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { From e68e95e947045704d3e6a36bb31e104a99d3adcc Mon Sep 17 00:00:00 2001 From: Alexander Pivovarov Date: Tue, 11 Oct 2016 22:31:21 -0700 Subject: [PATCH 1406/1470] Fix hadoop.version in building-spark.md Couple of mvn build examples use `-Dhadoop.version=VERSION` instead of actual version number Author: Alexander Pivovarov Closes #15440 from apivovarov/patch-1. (cherry picked from commit 299eb04ba05038c7dbb3ecf74a35d4bbfa456643) Signed-off-by: Reynold Xin --- docs/building-spark.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 330df0054154b..857f4d4fcbf9a 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -91,13 +91,13 @@ Examples: ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package # Apache Hadoop 2.4.X or 2.5.X - ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package # Apache Hadoop 2.6.X ./build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -DskipTests clean package # Apache Hadoop 2.7.X and later - ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=VERSION -DskipTests clean package + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.0 -DskipTests clean package # Different versions of HDFS and YARN. ./build/mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package From f3d82b53c42a971deedc04de6950b9228e5262ea Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 11 Oct 2016 22:36:57 -0700 Subject: [PATCH 1407/1470] [SPARK-17880][DOC] The url linking to `AccumulatorV2` in the document is incorrect. ## What changes were proposed in this pull request? In `programming-guide.md`, the url which links to `AccumulatorV2` says `api/scala/index.html#org.apache.spark.AccumulatorV2` but `api/scala/index.html#org.apache.spark.util.AccumulatorV2` is correct. ## How was this patch tested? manual test. Author: Kousuke Saruta Closes #15439 from sarutak/SPARK-17880. (cherry picked from commit b512f04f8e546843d5a3f35dcc6b675b5f4f5bc0) Signed-off-by: Reynold Xin --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index f82832905ef44..204ad5e4ada22 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1372,7 +1372,7 @@ res2: Long = 10 {% endhighlight %} While this code used the built-in support for accumulators of type Long, programmers can also -create their own types by subclassing [AccumulatorV2](api/scala/index.html#org.apache.spark.AccumulatorV2). +create their own types by subclassing [AccumulatorV2](api/scala/index.html#org.apache.spark.util.AccumulatorV2). The AccumulatorV2 abstract class has several methods which need to override: `reset` for resetting the accumulator to zero, and `add` for add anothor value into the accumulator, `merge` for merging another same-type accumulator into this one. Other methods need to override can refer to scala API document. For example, supposing we had a `MyVector` class representing mathematical vectors, we could write: From f12b74c02eec9e201fec8a16dac1f8e549c1b4f0 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 12 Oct 2016 00:40:47 -0700 Subject: [PATCH 1408/1470] [SPARK-17853][STREAMING][KAFKA][DOC] make it clear that reusing group.id is bad ## What changes were proposed in this pull request? Documentation fix to make it clear that reusing group id for different streams is super duper bad, just like it is with the underlying Kafka consumer. ## How was this patch tested? I built jekyll doc and made sure it looked ok. Author: cody koeninger Closes #15442 from koeninger/SPARK-17853. (cherry picked from commit c264ef9b1918256a5018c7a42a1a2b42308ea3f7) Signed-off-by: Reynold Xin --- docs/streaming-kafka-0-10-integration.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index 44c39e39446de..456b8453383db 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -27,7 +27,7 @@ For Scala/Java applications using SBT/Maven project definitions, link your strea "bootstrap.servers" -> "localhost:9092,anotherhost:9092", "key.deserializer" -> classOf[StringDeserializer], "value.deserializer" -> classOf[StringDeserializer], - "group.id" -> "example", + "group.id" -> "use_a_separate_group_id_for_each_stream", "auto.offset.reset" -> "latest", "enable.auto.commit" -> (false: java.lang.Boolean) ) @@ -48,7 +48,7 @@ Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javad
    For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). -Note that enable.auto.commit is disabled, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. +Note that the example sets enable.auto.commit to false, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. ### LocationStrategies The new Kafka consumer API will pre-fetch messages into buffers. Therefore it is important for performance reasons that the Spark integration keep cached consumers on executors (rather than recreating them for each batch), and prefer to schedule partitions on the host locations that have the appropriate consumers. @@ -57,6 +57,9 @@ In most cases, you should use `LocationStrategies.PreferConsistent` as shown abo The cache for consumers has a default maximum size of 64. If you expect to be handling more than (64 * number of executors) Kafka partitions, you can change this setting via `spark.streaming.kafka.consumer.cache.maxCapacity` +The cache is keyed by topicpartition and group.id, so use a **separate** `group.id` for each call to `createDirectStream`. + + ### ConsumerStrategies The new Kafka consumer API has a number of different ways to specify topics, some of which require considerable post-object-instantiation setup. `ConsumerStrategies` provides an abstraction that allows Spark to obtain properly configured consumers even after restart from checkpoint. From 4dcbde48de6c46e2fd8ccfec732b8ff5c24f97a4 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Tue, 11 Oct 2016 08:29:52 +0200 Subject: [PATCH 1409/1470] [SPARK-17808][PYSPARK] Upgraded version of Pyrolite to 4.13 ## What changes were proposed in this pull request? Upgraded to a newer version of Pyrolite which supports serialization of a BinaryType StructField for PySpark.SQL ## How was this patch tested? Added a unit test which fails with a raised ValueError when using the previous version of Pyrolite 4.9 and Python3 Author: Bryan Cutler Closes #15386 from BryanCutler/pyrolite-upgrade-SPARK-17808. (cherry picked from commit 658c7147f5bf637f36e8c66b9207d94b1e7c74c5) Signed-off-by: Sean Owen --- core/pom.xml | 2 +- dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- python/pyspark/sql/tests.py | 8 ++++++++ 7 files changed, 14 insertions(+), 6 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 914025104d6ad..aad87f7e12a68 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -316,7 +316,7 @@ net.razorvine pyrolite - 4.9 + 4.13 net.razorvine diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 8c9e559790ba6..34cd4e6b15f84 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -141,7 +141,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 839e0840dba3d..8ae3c5e60d5e2 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -148,7 +148,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index ed84de79b1fdc..7c691029535c6 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -148,7 +148,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 6e7c9cb5c791a..041e01e9cb427 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -156,7 +156,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index a61f31eb87696..4f70bff405363 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -157,7 +157,7 @@ pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.3.jar -pyrolite-4.9.jar +pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 3343bd72276a6..a71457a076bf8 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1703,6 +1703,14 @@ def test_read_text_file_list(self): count = df.count() self.assertEquals(count, 4) + def test_BinaryType_serialization(self): + # Pyrolite version <= 4.9 could not serialize BinaryType with Python3 SPARK-17808 + schema = StructType([StructField('mybytes', BinaryType())]) + data = [[bytearray(b'here is my data')], + [bytearray(b'and here is some more')]] + df = self.spark.createDataFrame(data, schema=schema) + df.collect() + class HiveSparkSubmitTests(SparkSubmitTests): From 5451541d1113aa75bab80914ca51a913f6ba4753 Mon Sep 17 00:00:00 2001 From: prigarg Date: Wed, 12 Oct 2016 10:14:45 -0700 Subject: [PATCH 1410/1470] [SPARK-17884][SQL] To resolve Null pointer exception when casting from empty string to interval type. ## What changes were proposed in this pull request? This change adds a check in castToInterval method of Cast expression , such that if converted value is null , then isNull variable should be set to true. Earlier, the expression Cast(Literal(), CalendarIntervalType) was throwing NullPointerException because of the above mentioned reason. ## How was this patch tested? Added test case in CastSuite.scala jira entry for detail: https://issues.apache.org/jira/browse/SPARK-17884 Author: prigarg Closes #15449 from priyankagargnitk/SPARK-17884. (cherry picked from commit d5580ebaa086b9feb72d5428f24c5b60cd7da745) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 7 ++++++- .../apache/spark/sql/catalyst/expressions/CastSuite.scala | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 70fff51956255..a53468cfd3011 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 @@ -657,7 +657,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] def castToIntervalCode(from: DataType): CastFunction = from match { case StringType => (c, evPrim, evNull) => - s"$evPrim = CalendarInterval.fromString($c.toString());" + s"""$evPrim = CalendarInterval.fromString($c.toString()); + if(${evPrim} == null) { + ${evNull} = true; + } + """.stripMargin + } private[this] def decimalToTimestampCode(d: String): String = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 5c35baacef2fa..b748595fc4f2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -767,6 +767,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { test("cast between string and interval") { import org.apache.spark.unsafe.types.CalendarInterval + checkEvaluation(Cast(Literal(""), CalendarIntervalType), null) checkEvaluation(Cast(Literal("interval -3 month 7 hours"), CalendarIntervalType), new CalendarInterval(-3, 7 * CalendarInterval.MICROS_PER_HOUR)) checkEvaluation(Cast(Literal.create( From d55ba3063da1a5d12e3b09e55f089f16ecf327bb Mon Sep 17 00:00:00 2001 From: Hossein Date: Wed, 12 Oct 2016 10:32:38 -0700 Subject: [PATCH 1411/1470] [SPARK-17790][SPARKR] Support for parallelizing R data.frame larger than 2GB ## What changes were proposed in this pull request? If the R data structure that is being parallelized is larger than `INT_MAX` we use files to transfer data to JVM. The serialization protocol mimics Python pickling. This allows us to simply call `PythonRDD.readRDDFromFile` to create the RDD. I tested this on my MacBook. Following code works with this patch: ```R intMax <- .Machine$integer.max largeVec <- 1:intMax rdd <- SparkR:::parallelize(sc, largeVec, 2) ``` ## How was this patch tested? * [x] Unit tests Author: Hossein Closes #15375 from falaki/SPARK-17790. (cherry picked from commit 5cc503f4fe9737a4c7947a80eecac053780606df) Signed-off-by: Felix Cheung --- R/pkg/R/context.R | 45 ++++++++++++++++++- R/pkg/inst/tests/testthat/test_sparkSQL.R | 11 +++++ .../apache/spark/api/r/RBackendHandler.scala | 2 +- .../scala/org/apache/spark/api/r/RRDD.scala | 13 ++++++ 4 files changed, 68 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 13ade49eabfa6..cec108cd86486 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -87,6 +87,10 @@ objectFile <- function(sc, path, minPartitions = NULL) { #' in the list are split into \code{numSlices} slices and distributed to nodes #' in the cluster. #' +#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MB), the function +#' will write it to disk and send the file name to JVM. Also to make sure each slice is not +#' larger than that limit, number of slices may be increased. +#' #' @param sc SparkContext to use #' @param coll collection to parallelize #' @param numSlices number of partitions to create in the RDD @@ -120,6 +124,11 @@ parallelize <- function(sc, coll, numSlices = 1) { coll <- as.list(coll) } + sizeLimit <- getMaxAllocationLimit(sc) + objectSize <- object.size(coll) + + # For large objects we make sure the size of each slice is also smaller than sizeLimit + numSlices <- max(numSlices, ceiling(objectSize / sizeLimit)) if (numSlices > length(coll)) numSlices <- length(coll) @@ -130,12 +139,44 @@ parallelize <- function(sc, coll, numSlices = 1) { # 2-tuples of raws serializedSlices <- lapply(slices, serialize, connection = NULL) - jrdd <- callJStatic("org.apache.spark.api.r.RRDD", - "createRDDFromArray", sc, serializedSlices) + # The PRC backend cannot handle arguments larger than 2GB (INT_MAX) + # If serialized data is safely less than that threshold we send it over the PRC channel. + # Otherwise, we write it to a file and send the file name + if (objectSize < sizeLimit) { + jrdd <- callJStatic("org.apache.spark.api.r.RRDD", "createRDDFromArray", sc, serializedSlices) + } else { + fileName <- writeToTempFile(serializedSlices) + jrdd <- tryCatch(callJStatic( + "org.apache.spark.api.r.RRDD", "createRDDFromFile", sc, fileName, as.integer(numSlices)), + finally = { + file.remove(fileName) + }) + } RDD(jrdd, "byte") } +getMaxAllocationLimit <- function(sc) { + conf <- callJMethod(sc, "getConf") + as.numeric( + callJMethod(conf, + "get", + "spark.r.maxAllocationLimit", + toString(.Machine$integer.max / 10) # Default to a safe value: 200MB + )) +} + +writeToTempFile <- function(serializedSlices) { + fileName <- tempfile() + conn <- file(fileName, "wb") + for (slice in serializedSlices) { + writeBin(as.integer(length(slice)), conn, endian = "big") + writeBin(slice, conn, endian = "big") + } + close(conn) + fileName +} + #' Include this specified package on all workers #' #' This function can be used to include a package on all workers before the diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index cdb8ff6b6f8c9..c9eedd553a56a 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -208,6 +208,17 @@ test_that("create DataFrame from RDD", { unsetHiveContext() }) +test_that("createDataFrame uses files for large objects", { + # To simulate a large file scenario, we set spark.r.maxAllocationLimit to a smaller value + conf <- callJMethod(sparkSession, "conf") + callJMethod(conf, "set", "spark.r.maxAllocationLimit", "100") + df <- createDataFrame(iris) + + # Resetting the conf back to default value + callJMethod(conf, "set", "spark.r.maxAllocationLimit", toString(.Machine$integer.max / 10)) + expect_equal(dim(df), dim(iris)) +}) + test_that("read/write csv as DataFrame", { csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") mockLinesCsv <- c("year,make,model,comment,blank", 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 index c416e835a9046..5c45c8493ef61 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -168,7 +168,7 @@ private[r] class RBackendHandler(server: RBackend) } } catch { case e: Exception => - logError(s"$methodName on $objId failed") + logError(s"$methodName on $objId failed", e) writeInt(dos, -1) // Writing the error message of the cause for the exception. This will be returned // to user in the R process. diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 59c8429c80172..a1a5eb8cf55e8 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.python.PythonRDD import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -140,4 +141,16 @@ private[r] object RRDD { def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = { JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length)) } + + /** + * Create an RRDD given a temporary file name. This is used to create RRDD when parallelize is + * called on large R objects. + * + * @param fileName name of temporary file on driver machine + * @param parallelism number of slices defaults to 4 + */ + def createRDDFromFile(jsc: JavaSparkContext, fileName: String, parallelism: Int): + JavaRDD[Array[Byte]] = { + PythonRDD.readRDDFromFile(jsc, fileName, parallelism) + } } From 050b8177e27df06d33a6f6f2b3b6a952b0d03ba6 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 12 Oct 2016 15:22:06 -0700 Subject: [PATCH 1412/1470] [SPARK-17782][STREAMING][KAFKA] alternative eliminate race condition of poll twice ## What changes were proposed in this pull request? Alternative approach to https://github.com/apache/spark/pull/15387 Author: cody koeninger Closes #15401 from koeninger/SPARK-17782-alt. (cherry picked from commit f9a56a153e0579283160519065c7f3620d12da3e) Signed-off-by: Shixiong Zhu --- .../streaming/kafka010/ConsumerStrategy.scala | 4 ++++ .../kafka010/DirectKafkaInputDStream.scala | 23 +++++++++++++++++-- .../kafka010/DirectKafkaStreamSuite.scala | 12 ++++++---- 3 files changed, 32 insertions(+), 7 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index 60255fc655e5f..778c06ea16a2b 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -104,6 +104,8 @@ private case class Subscribe[K, V]( toSeek.asScala.foreach { case (topicPartition, offset) => consumer.seek(topicPartition, offset) } + // we've called poll, we must pause or next poll may consume messages and set position + consumer.pause(consumer.assignment()) } consumer @@ -154,6 +156,8 @@ private case class SubscribePattern[K, V]( toSeek.asScala.foreach { case (topicPartition, offset) => consumer.seek(topicPartition, offset) } + // we've called poll, we must pause or next poll may consume messages and set position + consumer.pause(consumer.assignment()) } consumer diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index 13827f68f2cb5..432537ebf05b2 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -161,12 +161,31 @@ private[spark] class DirectKafkaInputDStream[K, V]( } } + /** + * The concern here is that poll might consume messages despite being paused, + * which would throw off consumer position. Fix position if this happens. + */ + private def paranoidPoll(c: Consumer[K, V]): Unit = { + val msgs = c.poll(0) + if (!msgs.isEmpty) { + // position should be minimum offset per topicpartition + msgs.asScala.foldLeft(Map[TopicPartition, Long]()) { (acc, m) => + val tp = new TopicPartition(m.topic, m.partition) + val off = acc.get(tp).map(o => Math.min(o, m.offset)).getOrElse(m.offset) + acc + (tp -> off) + }.foreach { case (tp, off) => + logInfo(s"poll(0) returned messages, seeking $tp to $off to compensate") + c.seek(tp, off) + } + } + } + /** * Returns the latest (highest) available offsets, taking new partitions into account. */ protected def latestOffsets(): Map[TopicPartition, Long] = { val c = consumer - c.poll(0) + paranoidPoll(c) val parts = c.assignment().asScala // make sure new partitions are reflected in currentOffsets @@ -223,7 +242,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( override def start(): Unit = { val c = consumer - c.poll(0) + paranoidPoll(c) if (currentOffsets.isEmpty) { currentOffsets = c.assignment().asScala.map { tp => tp -> c.position(tp) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index b1d90b8a82d59..860834df7bd7a 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -159,17 +159,19 @@ class DirectKafkaStreamSuite } test("pattern based subscription") { - val topics = List("pat1", "pat2", "advanced3") - // Should match 2 out of 3 topics + val topics = List("pat1", "pat2", "pat3", "advanced3") + // Should match 3 out of 4 topics val pat = """pat\d""".r.pattern val data = Map("a" -> 7, "b" -> 9) topics.foreach { t => kafkaTestUtils.createTopic(t) kafkaTestUtils.sendMessages(t, data) } - val offsets = Map(new TopicPartition("pat2", 0) -> 3L) - // 2 matching topics, one of which starts 3 messages later - val expectedTotal = (data.values.sum * 2) - 3 + val offsets = Map( + new TopicPartition("pat2", 0) -> 3L, + new TopicPartition("pat3", 0) -> 4L) + // 3 matching topics, two of which start a total of 7 messages later + val expectedTotal = (data.values.sum * 3) - 7 val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") ssc = new StreamingContext(sparkConf, Milliseconds(200)) From 5903dabc57c07310573babe94e4f205bdea6455f Mon Sep 17 00:00:00 2001 From: Brian Cho Date: Wed, 12 Oct 2016 20:43:18 -0700 Subject: [PATCH 1413/1470] [SPARK-16827][BRANCH-2.0] Avoid reporting spill metrics as shuffle metrics ## What changes were proposed in this pull request? Fix a bug where spill metrics were being reported as shuffle metrics. Eventually these spill metrics should be reported (SPARK-3577), but separate from shuffle metrics. The fix itself basically reverts the line to what it was in 1.6. ## How was this patch tested? Cherry-picked from master (#15347) Author: Brian Cho Closes #15455 from dafrista/shuffle-metrics-2.0. --- .../util/collection/unsafe/sort/UnsafeExternalSorter.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 999ded45f2e2a..6e030645a2ca8 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -144,7 +144,9 @@ private UnsafeExternalSorter( // Use getSizeAsKb (not bytes) to maintain backwards compatibility for units // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; this.fileBufferSizeBytes = 32 * 1024; - this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); + // The spill metrics are stored in a new ShuffleWriteMetrics, and then discarded (this fixes SPARK-16827). + // TODO: Instead, separate spill metrics should be stored and reported (tracked in SPARK-3577). + this.writeMetrics = new ShuffleWriteMetrics(); if (existingInMemorySorter == null) { this.inMemSorter = new UnsafeInMemorySorter( From ab00e410c6b1d7dafdfabcea1f249c78459b94f0 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 12 Oct 2016 21:40:45 -0700 Subject: [PATCH 1414/1470] [SPARK-17876] Write StructuredStreaming WAL to a stream instead of materializing all at once ## What changes were proposed in this pull request? The CompactibleFileStreamLog materializes the whole metadata log in memory as a String. This can cause issues when there are lots of files that are being committed, especially during a compaction batch. You may come across stacktraces that look like: ``` java.lang.OutOfMemoryError: Requested array size exceeds VM limit at java.lang.StringCoding.encode(StringCoding.java:350) at java.lang.String.getBytes(String.java:941) at org.apache.spark.sql.execution.streaming.FileStreamSinkLog.serialize(FileStreamSinkLog.scala:127) ``` The safer way is to write to an output stream so that we don't have to materialize a huge string. ## How was this patch tested? Existing unit tests Author: Burak Yavuz Closes #15437 from brkyvz/ser-to-stream. (cherry picked from commit edeb51a39d76d64196d7635f52be1b42c7ec4341) Signed-off-by: Shixiong Zhu --- .../streaming/CompactibleFileStreamLog.scala | 22 +++++++++----- .../execution/streaming/HDFSMetadataLog.scala | 29 ++++++++++--------- .../streaming/FileStreamSinkLogSuite.scala | 14 +++++---- 3 files changed, 38 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala index 027b5bbfab8d6..c14feea91ed7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.streaming -import java.io.IOException +import java.io.{InputStream, IOException, OutputStream} import java.nio.charset.StandardCharsets.UTF_8 +import scala.io.{Source => IOSource} import scala.reflect.ClassTag import org.apache.hadoop.fs.{Path, PathFilter} @@ -93,20 +94,25 @@ abstract class CompactibleFileStreamLog[T: ClassTag]( } } - override def serialize(logData: Array[T]): Array[Byte] = { - (metadataLogVersion +: logData.map(serializeData)).mkString("\n").getBytes(UTF_8) + override def serialize(logData: Array[T], out: OutputStream): Unit = { + // called inside a try-finally where the underlying stream is closed in the caller + out.write(metadataLogVersion.getBytes(UTF_8)) + logData.foreach { data => + out.write('\n') + out.write(serializeData(data).getBytes(UTF_8)) + } } - override def deserialize(bytes: Array[Byte]): Array[T] = { - val lines = new String(bytes, UTF_8).split("\n") - if (lines.length == 0) { + override def deserialize(in: InputStream): Array[T] = { + val lines = IOSource.fromInputStream(in, UTF_8.name()).getLines() + if (!lines.hasNext) { throw new IllegalStateException("Incomplete log file") } - val version = lines(0) + val version = lines.next() if (version != metadataLogVersion) { throw new IllegalStateException(s"Unknown log version: ${version}") } - lines.slice(1, lines.length).map(deserializeData) + lines.map(deserializeData).toArray } override def add(batchId: Long, logs: Array[T]): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 39a0f3341389c..c7235320fd6bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.execution.streaming -import java.io.{FileNotFoundException, IOException} -import java.nio.ByteBuffer +import java.io.{FileNotFoundException, InputStream, IOException, OutputStream} import java.util.{ConcurrentModificationException, EnumSet, UUID} import scala.reflect.ClassTag @@ -29,7 +28,6 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.internal.Logging -import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.JavaSerializer import org.apache.spark.sql.SparkSession import org.apache.spark.util.UninterruptibleThread @@ -88,12 +86,16 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) } } - protected def serialize(metadata: T): Array[Byte] = { - JavaUtils.bufferToArray(serializer.serialize(metadata)) + protected def serialize(metadata: T, out: OutputStream): Unit = { + // called inside a try-finally where the underlying stream is closed in the caller + val outStream = serializer.serializeStream(out) + outStream.writeObject(metadata) } - protected def deserialize(bytes: Array[Byte]): T = { - serializer.deserialize[T](ByteBuffer.wrap(bytes)) + protected def deserialize(in: InputStream): T = { + // called inside a try-finally where the underlying stream is closed in the caller + val inStream = serializer.deserializeStream(in) + inStream.readObject[T]() } /** @@ -114,7 +116,7 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) // Only write metadata when the batch has not yet been written Thread.currentThread match { case ut: UninterruptibleThread => - ut.runUninterruptibly { writeBatch(batchId, serialize(metadata)) } + ut.runUninterruptibly { writeBatch(batchId, metadata, serialize) } case _ => throw new IllegalStateException( "HDFSMetadataLog.add() must be executed on a o.a.spark.util.UninterruptibleThread") @@ -129,7 +131,7 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) * There may be multiple [[HDFSMetadataLog]] using the same metadata path. Although it is not a * valid behavior, we still need to prevent it from destroying the files. */ - private def writeBatch(batchId: Long, bytes: Array[Byte]): Unit = { + private def writeBatch(batchId: Long, metadata: T, writer: (T, OutputStream) => Unit): Unit = { // Use nextId to create a temp file var nextId = 0 while (true) { @@ -137,9 +139,9 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) try { val output = fileManager.create(tempPath) try { - output.write(bytes) + writer(metadata, output) } finally { - output.close() + IOUtils.closeQuietly(output) } try { // Try to commit the batch @@ -193,10 +195,9 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) if (fileManager.exists(batchMetadataFile)) { val input = fileManager.open(batchMetadataFile) try { - val bytes = IOUtils.toByteArray(input) - Some(deserialize(bytes)) + Some(deserialize(input)) } finally { - input.close() + IOUtils.closeQuietly(input) } } else { logDebug(s"Unable to find batch $batchMetadataFile") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala index 41a8cc2400dff..e1bc674a28071 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.streaming +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.nio.charset.StandardCharsets.UTF_8 import org.apache.spark.SparkFunSuite @@ -133,9 +134,12 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { |{"path":"/a/b/y","size":200,"isDir":false,"modificationTime":2000,"blockReplication":2,"blockSize":20000,"action":"delete"} |{"path":"/a/b/z","size":300,"isDir":false,"modificationTime":3000,"blockReplication":3,"blockSize":30000,"action":"add"}""".stripMargin // scalastyle:on - assert(expected === new String(sinkLog.serialize(logs), UTF_8)) - - assert(VERSION === new String(sinkLog.serialize(Array()), UTF_8)) + val baos = new ByteArrayOutputStream() + sinkLog.serialize(logs, baos) + assert(expected === baos.toString(UTF_8.name())) + baos.reset() + sinkLog.serialize(Array(), baos) + assert(VERSION === baos.toString(UTF_8.name())) } } @@ -174,9 +178,9 @@ class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { blockSize = 30000L, action = FileStreamSinkLog.ADD_ACTION)) - assert(expected === sinkLog.deserialize(logs.getBytes(UTF_8))) + assert(expected === sinkLog.deserialize(new ByteArrayInputStream(logs.getBytes(UTF_8)))) - assert(Nil === sinkLog.deserialize(VERSION.getBytes(UTF_8))) + assert(Nil === sinkLog.deserialize(new ByteArrayInputStream(VERSION.getBytes(UTF_8)))) } } From d38f38a093b4dff32c686675d93ab03e7a8f4908 Mon Sep 17 00:00:00 2001 From: buzhihuojie Date: Wed, 12 Oct 2016 22:51:54 -0700 Subject: [PATCH 1415/1470] minor doc fix for Row.scala ## What changes were proposed in this pull request? minor doc fix for "getAnyValAs" in class Row ## How was this patch tested? None. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: buzhihuojie Closes #15452 from david-weiluo-ren/minorDocFixForRow. (cherry picked from commit 7222a25a11790fa9d9d1428c84b6f827a785c9e8) Signed-off-by: Reynold Xin --- sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e16850efbea5f..9b0a3c7e4ce5c 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 @@ -456,7 +456,7 @@ trait Row extends Serializable { def mkString(start: String, sep: String, end: String): String = toSeq.mkString(start, sep, end) /** - * Returns the value of a given fieldName. + * Returns the value at position i. * * @throws UnsupportedOperationException when schema is not defined. * @throws ClassCastException when data type does not match. From d7fa3e32421c73adfa522adfeeb970edd4c22eb3 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 13 Oct 2016 13:31:50 -0700 Subject: [PATCH 1416/1470] [SPARK-17834][SQL] Fetch the earliest offsets manually in KafkaSource instead of counting on KafkaConsumer ## What changes were proposed in this pull request? Because `KafkaConsumer.poll(0)` may update the partition offsets, this PR just calls `seekToBeginning` to manually set the earliest offsets for the KafkaSource initial offsets. ## How was this patch tested? Existing tests. Author: Shixiong Zhu Closes #15397 from zsxwing/SPARK-17834. (cherry picked from commit 08eac356095c7faa2b19d52f2fb0cbc47eb7d1d1) Signed-off-by: Shixiong Zhu --- .../spark/sql/kafka010/KafkaSource.scala | 55 ++++++++++++------- .../sql/kafka010/KafkaSourceProvider.scala | 19 +++++-- 2 files changed, 48 insertions(+), 26 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 1be70db87497e..4b0bb0a0f725c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -82,6 +82,7 @@ private[kafka010] case class KafkaSource( executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, + startFromEarliestOffset: Boolean, failOnDataLoss: Boolean) extends Source with Logging { @@ -109,7 +110,11 @@ private[kafka010] case class KafkaSource( private lazy val initialPartitionOffsets = { val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) metadataLog.get(0).getOrElse { - val offsets = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = false)) + val offsets = if (startFromEarliestOffset) { + KafkaSourceOffset(fetchEarliestOffsets()) + } else { + KafkaSourceOffset(fetchLatestOffsets()) + } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") offsets @@ -123,7 +128,7 @@ private[kafka010] case class KafkaSource( // Make sure initialPartitionOffsets is initialized initialPartitionOffsets - val offset = KafkaSourceOffset(fetchPartitionOffsets(seekToEnd = true)) + val offset = KafkaSourceOffset(fetchLatestOffsets()) logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") Some(offset) } @@ -227,26 +232,34 @@ private[kafka010] case class KafkaSource( override def toString(): String = s"KafkaSource[$consumerStrategy]" /** - * Fetch the offset of a partition, either seek to the latest offsets or use the current offsets - * in the consumer. + * Fetch the earliest offsets of partitions. */ - private def fetchPartitionOffsets( - seekToEnd: Boolean): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) - assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + private def fetchEarliestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() consumer.pause(partitions) - logDebug(s"Partitioned assigned to consumer: $partitions") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") - // Get the current or latest offset of each partition - if (seekToEnd) { - consumer.seekToEnd(partitions) - logDebug("Seeked to the end") - } + consumer.seekToBeginning(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for partition : $partitionOffsets") + partitionOffsets + } + + /** + * Fetch the latest offset of partitions. + */ + private def fetchLatestOffsets(): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") + + consumer.seekToEnd(partitions) val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got offsets for partition : $partitionOffsets") + logDebug(s"Got latest offsets for partition : $partitionOffsets") partitionOffsets } @@ -256,22 +269,21 @@ private[kafka010] case class KafkaSource( */ private def fetchNewPartitionEarliestOffsets( newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) - assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + consumer.pause(partitions) logDebug(s"\tPartitioned assigned to consumer: $partitions") // Get the earliest offset of each partition consumer.seekToBeginning(partitions) - val partitionToOffsets = newPartitions.filter { p => + val partitionOffsets = newPartitions.filter { p => // When deleting topics happen at the same time, some partitions may not be in `partitions`. // So we need to ignore them partitions.contains(p) }.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got offsets for new partitions: $partitionToOffsets") - partitionToOffsets + logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") + partitionOffsets } /** @@ -284,6 +296,9 @@ private[kafka010] case class KafkaSource( */ private def withRetriesWithoutInterrupt( body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[StreamExecutionThread]) + synchronized { var result: Option[Map[TopicPartition, Long]] = None var attempt = 1 diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 1b0a2fe955d03..23b1b60f3bcaa 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -77,10 +77,15 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // id. Hence, we should generate a unique id for each query. val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" - val autoOffsetResetValue = caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { - case Some(value) => value.trim() // same values as those supported by auto.offset.reset - case None => "latest" - } + val startFromEarliestOffset = + caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => false + case Some("earliest") => true + case Some(pos) => + // This should not happen since we have already checked the options. + throw new IllegalStateException(s"Invalid $STARTING_OFFSET_OPTION_KEY: $pos") + case None => false + } val kafkaParamsForStrategy = ConfigUpdater("source", specifiedKafkaParams) @@ -90,8 +95,9 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // So that consumers in Kafka source do not mess with any existing group id .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver") - // So that consumers can start from earliest or latest - .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, autoOffsetResetValue) + // Set to "latest" to avoid exceptions. However, KafkaSource will fetch the initial offsets + // by itself instead of counting on KafkaConsumer. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") // So that consumers in the driver does not commit offsets unnecessarily .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -147,6 +153,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider kafkaParamsForExecutors, parameters, metadataPath, + startFromEarliestOffset, failOnDataLoss) } From c53b8374911e801ed98c1436c384f0aef076eaab Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 14 Oct 2016 14:45:20 -0700 Subject: [PATCH 1417/1470] [SPARK-17863][SQL] should not add column into Distinct ## What changes were proposed in this pull request? We are trying to resolve the attribute in sort by pulling up some column for grandchild into child, but that's wrong when the child is Distinct, because the added column will change the behavior of Distinct, we should not do that. ## How was this patch tested? Added regression test. Author: Davies Liu Closes #15489 from davies/order_distinct. (cherry picked from commit da9aeb0fde589f7c21c2f4a32036a68c0353965d) Signed-off-by: Yin Huai --- .../sql/catalyst/analysis/Analyzer.scala | 2 ++ .../org/apache/spark/sql/SQLQuerySuite.scala | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+) 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 3e4c769217262..617f3e062465c 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 @@ -838,6 +838,8 @@ class Analyzer( // attributes that its child might have or could have. val missing = missingAttrs -- g.child.outputSet g.copy(join = true, child = addMissingAttr(g.child, missing)) + case d: Distinct => + throw new AnalysisException(s"Can't add $missingAttrs to $d") case u: UnaryNode => u.withNewChildren(addMissingAttr(u.child, missingAttrs) :: Nil) case other => 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 cf250970c6c32..3684135f6822c 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 @@ -1096,6 +1096,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } + test("SPARK-17863: SELECT distinct does not work correctly if order by missing attribute") { + checkAnswer( + sql("""select distinct struct.a, struct.b + |from ( + | select named_struct('a', 1, 'b', 2, 'c', 3) as struct + | union all + | select named_struct('a', 1, 'b', 2, 'c', 4) as struct) tmp + |order by a, b + |""".stripMargin), + Row(1, 2) :: Nil) + + val error = intercept[AnalysisException] { + sql("""select distinct struct.a, struct.b + |from ( + | select named_struct('a', 1, 'b', 2, 'c', 3) as struct + | union all + | select named_struct('a', 1, 'b', 2, 'c', 4) as struct) tmp + |order by struct.a, struct.b + |""".stripMargin) + } + assert(error.message contains "cannot resolve '`struct.a`' given input columns: [a, b]") + + } + test("cast boolean to string") { // TODO Ensure true/false string letter casing is consistent with Hive in all cases. checkAnswer( From 2a1b10b649a8d4c077a0e19df976f1fd36b7e266 Mon Sep 17 00:00:00 2001 From: Jun Kim Date: Sat, 15 Oct 2016 00:36:55 -0700 Subject: [PATCH 1418/1470] [SPARK-17953][DOCUMENTATION] Fix typo in SparkSession scaladoc ## What changes were proposed in this pull request? ### Before: ```scala SparkSession.builder() .master("local") .appName("Word Count") .config("spark.some.config.option", "some-value"). .getOrCreate() ``` ### After: ```scala SparkSession.builder() .master("local") .appName("Word Count") .config("spark.some.config.option", "some-value") .getOrCreate() ``` There was one unexpected dot! Author: Jun Kim Closes #15498 from tae-jun/SPARK-17953. (cherry picked from commit 36d81c2c68ef4114592b069287743eb5cb078318) Signed-off-by: Reynold Xin --- sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index c88206c81a04e..a7de1151e8a2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -64,7 +64,7 @@ import org.apache.spark.util.Utils * SparkSession.builder() * .master("local") * .appName("Word Count") - * .config("spark.some.config.option", "some-value"). + * .config("spark.some.config.option", "some-value") * .getOrCreate() * }}} */ From 3cc2fe5b94d3bcdfb4f28bfa6d8e51fe67d6e1b4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 16 Oct 2016 22:15:47 -0700 Subject: [PATCH 1419/1470] [SPARK-17819][SQL][BRANCH-2.0] Support default database in connection URIs for Spark Thrift Server ## What changes were proposed in this pull request? Currently, Spark Thrift Server ignores the default database in URI. This PR supports that like the following. ```sql $ bin/beeline -u jdbc:hive2://localhost:10000 -e "create database testdb" $ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "create table t(a int)" $ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "show tables" ... +------------+--------------+--+ | tableName | isTemporary | +------------+--------------+--+ | t | false | +------------+--------------+--+ 1 row selected (0.347 seconds) $ bin/beeline -u jdbc:hive2://localhost:10000 -e "show tables" ... +------------+--------------+--+ | tableName | isTemporary | +------------+--------------+--+ +------------+--------------+--+ No rows selected (0.098 seconds) ``` ## How was this patch tested? Pass the Jenkins with a newly added testsuite. Author: Dongjoon Hyun Closes #15507 from dongjoon-hyun/SPARK-17819-BACK. --- .../thriftserver/SparkSQLSessionManager.scala | 3 + .../thriftserver/JdbcConnectionUriSuite.scala | 70 +++++++++++++++++++ 2 files changed, 73 insertions(+) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 1e4c4790856be..fc5b221ee3f49 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -79,6 +79,9 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: sqlContext.newSession() } ctx.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) + if (sessionConf != null && sessionConf.containsKey("use:database")) { + ctx.sql(s"use ${sessionConf.get("use:database")}") + } sparkSqlOperationManager.sessionToContexts += sessionHandle -> ctx sessionHandle } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala new file mode 100644 index 0000000000000..fb8a7e273ae44 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.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.hive.thriftserver + +import java.sql.DriverManager + +import org.apache.hive.jdbc.HiveDriver + +import org.apache.spark.util.Utils + +class JdbcConnectionUriSuite extends HiveThriftServer2Test { + Utils.classForName(classOf[HiveDriver].getCanonicalName) + + override def mode: ServerMode.Value = ServerMode.binary + + val JDBC_TEST_DATABASE = "jdbc_test_database" + val USER = System.getProperty("user.name") + val PASSWORD = "" + + override protected def beforeAll(): Unit = { + super.beforeAll() + + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + statement.execute(s"CREATE DATABASE $JDBC_TEST_DATABASE") + connection.close() + } + + override protected def afterAll(): Unit = { + try { + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + statement.execute(s"DROP DATABASE $JDBC_TEST_DATABASE") + connection.close() + } finally { + super.afterAll() + } + } + + test("SPARK-17819 Support default database in connection URIs") { + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/$JDBC_TEST_DATABASE" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + try { + val resultSet = statement.executeQuery("select current_database()") + resultSet.next() + assert(resultSet.getString(1) === JDBC_TEST_DATABASE) + } finally { + statement.close() + connection.close() + } + } +} From ca66f52ff81c19e17ca3733eac92d66012a3ec6e Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Sun, 16 Oct 2016 22:38:30 -0700 Subject: [PATCH 1420/1470] [MINOR][SQL] Add prettyName for current_database function ## What changes were proposed in this pull request? Added a `prettyname` for current_database function. ## How was this patch tested? Manually. Before: ``` scala> sql("select current_database()").show +-----------------+ |currentdatabase()| +-----------------+ | default| +-----------------+ ``` After: ``` scala> sql("select current_database()").show +------------------+ |current_database()| +------------------+ | default| +------------------+ ``` Author: Weiqing Yang Closes #15506 from weiqingy/prettyName. (cherry picked from commit 56b0f5f4d1d7826737b81ebc4ec5dad83b6463e3) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/sql/catalyst/expressions/misc.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 92f8fb85fc0e2..5c4436ffaab72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -564,4 +564,5 @@ case class CurrentDatabase() extends LeafExpression with Unevaluable { override def dataType: DataType = StringType override def foldable: Boolean = true override def nullable: Boolean = false + override def prettyName: String = "current_database" } From d1a02117862b20d0e8e58f4c6da6a97665a02590 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 17 Oct 2016 15:29:53 +0800 Subject: [PATCH 1421/1470] [SPARK-17892][SQL][2.0] Do Not Optimize Query in CTAS More Than Once #15048 ### What changes were proposed in this pull request? This PR is to backport https://github.com/apache/spark/pull/15048 and https://github.com/apache/spark/pull/15459. However, in 2.0, we do not have a unified logical node `CreateTable` and the analyzer rule `PreWriteCheck` is also different. To minimize the code changes, this PR adds a new rule `AnalyzeCreateTableAsSelect`. Please treat it as a new PR to review. Thanks! As explained in https://github.com/apache/spark/pull/14797: >Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs. For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result. We should not optimize the query in CTAS more than once. For example, ```Scala spark.range(99, 101).createOrReplaceTempView("tab1") val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1" sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt") checkAnswer(spark.table("tab2"), sql(sqlStmt)) ``` Before this PR, the results do not match ``` == Results == !== Correct Answer - 2 == == Spark Answer - 2 == ![100,100.000000000000000000] [100,null] [99,99.000000000000000000] [99,99.000000000000000000] ``` After this PR, the results match. ``` +---+----------------------+ |id |num | +---+----------------------+ |99 |99.000000000000000000 | |100|100.000000000000000000| +---+----------------------+ ``` In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`. ### How was this patch tested? Author: gatorsmile Closes #15502 from gatorsmile/ctasOptimize2.0. --- .../sql/catalyst/plans/logical/Command.scala | 7 ++++- .../analysis/UnsupportedOperationsSuite.scala | 5 +--- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../sql/execution/command/SetCommand.scala | 2 -- .../spark/sql/execution/command/cache.scala | 6 ----- .../sql/execution/command/commands.scala | 4 +-- .../sql/execution/command/databases.scala | 2 -- .../spark/sql/execution/command/ddl.scala | 6 ----- .../spark/sql/execution/command/tables.scala | 11 ++++---- .../spark/sql/execution/datasources/ddl.scala | 17 +++++------- .../sql/execution/datasources/rules.scala | 26 ++++++++++++++++--- .../spark/sql/internal/SessionState.scala | 3 ++- .../org/apache/spark/sql/DataFrameSuite.scala | 20 ++++++++++++++ .../sources/CreateTableAsSelectSuite.scala | 12 +++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++--- .../spark/sql/hive/HiveSessionState.scala | 1 + .../sql/hive/MetastoreRelationSuite.scala | 22 ++++++++++++++-- 17 files changed, 102 insertions(+), 50 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala index 75a5b10d9ed04..64f57835c8898 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala @@ -17,9 +17,14 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.expressions.Attribute + /** * A logical node that represents a non-query command to be executed by the system. For example, * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are * eagerly executed. */ -trait Command +trait Command extends LeafNode { + final override def children: Seq[LogicalPlan] = Seq.empty + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 6df47acaba85b..ff1bb126f463d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -31,10 +31,7 @@ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.IntegerType /** A dummy command for testing unsupported operations. */ -case class DummyCommand() extends LogicalPlan with Command { - override def output: Seq[Attribute] = Nil - override def children: Seq[LogicalPlan] = Nil -} +case class DummyCommand() extends Command class UnsupportedOperationsSuite extends SparkFunSuite { 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 e7faab5495421..ccfb95454b08c 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 @@ -449,7 +449,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { c.bucketSpec, c.mode, c.options, - c.child) + c.query) ExecutedCommandExec(cmd) :: Nil case c: CreateTempViewUsing => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index b0e2d03af070d..af6def52d07d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -129,6 +129,4 @@ case object ResetCommand extends RunnableCommand with Logging { sparkSession.sessionState.conf.clear() Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 697e2ff21159b..605e49c5a87c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -47,8 +47,6 @@ case class CacheTableCommand( Seq.empty[Row] } - - override def output: Seq[Attribute] = Seq.empty } @@ -58,8 +56,6 @@ case class UncacheTableCommand(tableIdent: TableIdentifier) extends RunnableComm sparkSession.catalog.uncacheTable(tableIdent.quotedString) Seq.empty[Row] } - - override def output: Seq[Attribute] = Seq.empty } /** @@ -71,6 +67,4 @@ case object ClearCacheCommand extends RunnableCommand { sparkSession.catalog.clearCache() Seq.empty[Row] } - - override def output: Seq[Attribute] = Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 424a962b5eb1c..698c625d617fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -35,9 +35,7 @@ import org.apache.spark.sql.types._ * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -trait RunnableCommand extends LogicalPlan with logical.Command { - override def output: Seq[Attribute] = Seq.empty - final override def children: Seq[LogicalPlan] = Seq.empty +trait RunnableCommand extends logical.Command { def run(sparkSession: SparkSession): Seq[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala index 597ec27ce6698..e5a6a5f60b8a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala @@ -59,6 +59,4 @@ case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { sparkSession.sessionState.catalog.setCurrentDatabase(databaseName) Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 16deee359f534..d63d29defdd67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -72,8 +72,6 @@ case class CreateDatabaseCommand( ifNotExists) Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } @@ -103,8 +101,6 @@ case class DropDatabaseCommand( sparkSession.sessionState.catalog.dropDatabase(databaseName, ifExists, cascade) Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } /** @@ -128,8 +124,6 @@ case class AlterDatabasePropertiesCommand( Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 9253db0f20823..ad0c779ff29ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -34,7 +34,8 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, UnaryNode} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.datasources.PartitioningUtils @@ -43,10 +44,10 @@ import org.apache.spark.util.Utils case class CreateHiveTableAsSelectLogicalPlan( tableDesc: CatalogTable, - child: LogicalPlan, - allowExisting: Boolean) extends UnaryNode with Command { + query: LogicalPlan, + allowExisting: Boolean) extends Command { - override def output: Seq[Attribute] = Seq.empty[Attribute] + override def innerChildren: Seq[QueryPlan[_]] = Seq(query) override lazy val resolved: Boolean = tableDesc.identifier.database.isDefined && @@ -54,7 +55,7 @@ case class CreateHiveTableAsSelectLogicalPlan( tableDesc.storage.serde.isDefined && tableDesc.storage.inputFormat.isDefined && tableDesc.storage.outputFormat.isDefined && - childrenResolved + query.resolved } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 31a2075d2ff99..857047f820ce1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand @@ -41,17 +41,10 @@ case class CreateTableUsing( partitionColumns: Array[String], bucketSpec: Option[BucketSpec], allowExisting: Boolean, - managedIfNoPath: Boolean) extends LogicalPlan with logical.Command { - - override def output: Seq[Attribute] = Seq.empty - override def children: Seq[LogicalPlan] = Seq.empty -} + managedIfNoPath: Boolean) extends logical.Command /** * A node used to support CTAS statements and saveAsTable for the data source API. - * This node is a [[logical.UnaryNode]] instead of a [[logical.Command]] because we want the - * analyzer can analyze the logical plan that will be used to populate the table. - * So, [[PreWriteCheck]] can detect cases that are not allowed. */ case class CreateTableUsingAsSelect( tableIdent: TableIdentifier, @@ -60,8 +53,10 @@ case class CreateTableUsingAsSelect( bucketSpec: Option[BucketSpec], mode: SaveMode, options: Map[String, String], - child: LogicalPlan) extends logical.UnaryNode { - override def output: Seq[Attribute] = Seq.empty[Attribute] + query: LogicalPlan) extends logical.Command { + + override def innerChildren: Seq[QueryPlan[_]] = Seq(query) + override lazy val resolved: Boolean = query.resolved } case class CreateTempViewUsing( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 05908d908fd20..27420d58e56a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, RowOrd import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.command.CreateHiveTableAsSelectLogicalPlan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} @@ -61,6 +62,25 @@ class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { } } +/** + * Analyze the query in CREATE TABLE AS SELECT (CTAS). After analysis, [[PreWriteCheck]] also + * can detect the cases that are not allowed. + */ +case class AnalyzeCreateTableAsSelect(sparkSession: SparkSession) extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case c: CreateTableUsingAsSelect if !c.query.resolved => + c.copy(query = analyzeQuery(c.query)) + case c: CreateHiveTableAsSelectLogicalPlan if !c.query.resolved => + c.copy(query = analyzeQuery(c.query)) + } + + private def analyzeQuery(query: LogicalPlan): LogicalPlan = { + val qe = sparkSession.sessionState.executePlan(query) + qe.assertAnalyzed() + qe.analyzed + } +} + /** * Preprocess the [[InsertIntoTable]] plan. Throws exception if the number of columns mismatch, or * specified partition columns are different from the existing partition columns in the target @@ -216,7 +236,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) // (the relation is a BaseRelation). case l @ LogicalRelation(dest: BaseRelation, _, _) => // Get all input data source relations of the query. - val srcRelations = c.child.collect { + val srcRelations = c.query.collect { case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(dest)) { @@ -233,12 +253,12 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } PartitioningUtils.validatePartitionColumn( - c.child.schema, c.partitionColumns, conf.caseSensitiveAnalysis) + c.query.schema, c.partitionColumns, conf.caseSensitiveAnalysis) for { spec <- c.bucketSpec sortColumnName <- spec.sortColumnNames - sortColumn <- c.child.schema.find(_.name == sortColumnName) + sortColumn <- c.query.schema.find(_.name == sortColumnName) } { if (!RowOrdering.isOrderable(sortColumn.dataType)) { failAnalysis(s"Cannot use ${sortColumn.dataType.simpleString} for sorting column.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 01cc13f9df885..e054ef2bcf75b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.AnalyzeTableCommand -import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, FindDataSourceTable, PreprocessTableInsertion, ResolveDataSource} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryManager} import org.apache.spark.sql.util.ExecutionListenerManager @@ -111,6 +111,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val analyzer: Analyzer = { new Analyzer(catalog, conf) { override val extendedResolutionRules = + AnalyzeCreateTableAsSelect(sparkSession) :: PreprocessTableInsertion(conf) :: new FindDataSourceTable(sparkSession) :: DataSourceAnalysis(conf) :: 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 da5c538eace2d..7ab0fe07b9c4a 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 @@ -27,9 +27,11 @@ import scala.util.Random import org.scalatest.Matchers._ import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -1565,4 +1567,22 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val d = sampleDf.withColumn("c", monotonically_increasing_id).select($"c").collect assert(d.size == d.distinct.size) } + + test("SPARK-17409: Do Not Optimize Query in CTAS (Data source tables) More Than Once") { + withTable("bar") { + withTempView("foo") { + withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") { + sql("select 0 as id").createOrReplaceTempView("foo") + val df = sql("select * from foo group by id") + // If we optimize the query in CTAS more than once, the following saveAsTable will fail + // with the error: `GROUP BY position 0 is not in select list (valid range is [1, 1])` + df.write.mode("overwrite").saveAsTable("bar") + checkAnswer(spark.table("bar"), Row(0) :: Nil) + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier("bar")) + assert(tableMetadata.properties(DATASOURCE_PROVIDER) == "json", + "the expected table is a data source table using json") + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 251a25665a421..ea1f7a5a3c344 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -220,4 +220,16 @@ class CreateTableAsSelectSuite Some(BucketSpec(5, Seq("a"), Seq("b")))) } } + + test("SPARK-17409: CTAS of decimal calculation") { + withTable("tab2") { + withTempView("tab1") { + spark.range(99, 101).createOrReplaceTempView("tab1") + val sqlStmt = + "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1" + sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt") + checkAnswer(spark.table("tab2"), sql(sqlStmt)) + } + } + } } 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 bafb42277e334..e7d1ed34f5abb 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 @@ -449,7 +449,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log case p: LogicalPlan if !p.childrenResolved => p case p: LogicalPlan if p.resolved => p - case p @ CreateHiveTableAsSelectLogicalPlan(table, child, allowExisting) => + case p @ CreateHiveTableAsSelectLogicalPlan(table, query, allowExisting) => val desc = if (table.storage.serde.isEmpty) { // add default serde table.withNewStorage( @@ -462,7 +462,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log execution.CreateHiveTableAsSelectCommand( desc.copy(identifier = TableIdentifier(tblName, Some(dbName))), - child, + query, allowExisting) } } @@ -510,7 +510,7 @@ private[hive] case class InsertIntoHiveTable( child: LogicalPlan, overwrite: Boolean, ifNotExists: Boolean) - extends LogicalPlan with Command { + extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 8773993d362c4..822a7709ba160 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -62,6 +62,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) override lazy val analyzer: Analyzer = { new Analyzer(catalog, conf) { override val extendedResolutionRules = + AnalyzeCreateTableAsSelect(sparkSession) :: catalog.ParquetConversions :: catalog.OrcConversions :: catalog.CreateTables :: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala index eec60b440720f..c6711c3236250 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreRelationSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.hive -import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils -class MetastoreRelationSuite extends SparkFunSuite { +class MetastoreRelationSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("makeCopy and toJSON should work") { val table = CatalogTable( identifier = TableIdentifier("test", Some("db")), @@ -35,4 +38,19 @@ class MetastoreRelationSuite extends SparkFunSuite { // No exception should be thrown relation.toJSON } + + test("SPARK-17409: Do Not Optimize Query in CTAS (Hive Serde Table) More Than Once") { + withTable("bar") { + withTempView("foo") { + sql("select 0 as id").createOrReplaceTempView("foo") + // If we optimize the query in CTAS more than once, the following saveAsTable will fail + // with the error: `GROUP BY position 0 is not in select list (valid range is [1, 1])` + sql("CREATE TABLE bar AS SELECT * FROM foo group by id") + checkAnswer(spark.table("bar"), Row(0) :: Nil) + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier("bar")) + assert(!DDLUtils.isDatasourceTable(tableMetadata), + "the expected table is a Hive serde table") + } + } + } } From a0d9015b3f34582c5d43bd31bbf35a0e92b1da29 Mon Sep 17 00:00:00 2001 From: Maxime Rihouey Date: Mon, 17 Oct 2016 10:56:22 +0100 Subject: [PATCH 1422/1470] Fix example of tf_idf with minDocFreq ## What changes were proposed in this pull request? The python example for tf_idf with the parameter "minDocFreq" is not properly set up because the same variable is used to transform the document for both with and without the "minDocFreq" parameter. The IDF(minDocFreq=2) is stored in the variable "idfIgnore" but then it is the original variable "idf" used to transform the "tf" instead of the "idfIgnore". ## How was this patch tested? Before the results for "tfidf" and "tfidfIgnore" were the same: tfidf: (1048576,[1046921],[3.75828890549]) (1048576,[1046920],[3.75828890549]) (1048576,[1046923],[3.75828890549]) (1048576,[892732],[3.75828890549]) (1048576,[892733],[3.75828890549]) (1048576,[892734],[3.75828890549]) tfidfIgnore: (1048576,[1046921],[3.75828890549]) (1048576,[1046920],[3.75828890549]) (1048576,[1046923],[3.75828890549]) (1048576,[892732],[3.75828890549]) (1048576,[892733],[3.75828890549]) (1048576,[892734],[3.75828890549]) After the fix those are how they should be: tfidf: (1048576,[1046921],[3.75828890549]) (1048576,[1046920],[3.75828890549]) (1048576,[1046923],[3.75828890549]) (1048576,[892732],[3.75828890549]) (1048576,[892733],[3.75828890549]) (1048576,[892734],[3.75828890549]) tfidfIgnore: (1048576,[1046921],[0.0]) (1048576,[1046920],[0.0]) (1048576,[1046923],[0.0]) (1048576,[892732],[0.0]) (1048576,[892733],[0.0]) (1048576,[892734],[0.0]) Author: Maxime Rihouey Closes #15503 from maximerihouey/patch-1. (cherry picked from commit e3bf37fa3ada43624b2e77bef90ad3d3dbcd8ce1) Signed-off-by: Sean Owen --- examples/src/main/python/mllib/tf_idf_example.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/mllib/tf_idf_example.py b/examples/src/main/python/mllib/tf_idf_example.py index c4d53333a95a9..b66412b2334e7 100644 --- a/examples/src/main/python/mllib/tf_idf_example.py +++ b/examples/src/main/python/mllib/tf_idf_example.py @@ -43,7 +43,7 @@ # In such cases, the IDF for these terms is set to 0. # This feature can be used by passing the minDocFreq value to the IDF constructor. idfIgnore = IDF(minDocFreq=2).fit(tf) - tfidfIgnore = idf.transform(tf) + tfidfIgnore = idfIgnore.transform(tf) # $example off$ print("tfidf:") From 881e0eb05782ea74cf92a62954466b14ea9e05b6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 17 Oct 2016 16:56:40 -0700 Subject: [PATCH 1423/1470] [SPARK-17731][SQL][STREAMING] Metrics for structured streaming for branch-2.0 **This PR adds the same metrics to branch-2.0 that was added to master in #15307.** The differences compared to the #15307 are - The new configuration is added only in the `SQLConf `object (i.e. `SQLConf.STREAMING_METRICS_ENABLED`) and not in the `SQLConf` class (i.e. no `SQLConf.isStreamingMetricsEnabled`). Spark master has all the streaming configurations exposed as actual fields in SQLConf class (e.g. [streamingPollingDelay](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L642)), but [not in Spark 2.0](https://github.com/apache/spark/blob/branch-2.0/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L608). So I didnt add it in this 2.0 PR. - In the previous master PR, the aboveconfiguration was read in `StreamExecution` as `sparkSession.sessionState.conf.isStreamingMetricsEnabled`. In this 2.0 PR, I am instead reading it as `sparkSession.conf.get(STREAMING_METRICS_ENABLED)`(i.e. no `sessionState`) to keep it consistent with how other confs are read in `StreamExecution` (e.g. [STREAMING_POLLING_DELAY](https://github.com/tdas/spark/blob/ee8e899e4c274c363a8b4d13e8bf57b0b467a50e/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L62)). - Different Mima exclusions ------ ## What changes were proposed in this pull request? Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics. https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing Specifically, this PR adds the following public APIs changes. - `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later) - `StreamingQueryStatus` has the following important fields - inputRate - Current rate (rows/sec) at which data is being generated by all the sources - processingRate - Current rate (rows/sec) at which the query is processing data from all the sources - ~~outputRate~~ - *Does not work with wholestage codegen* - latency - Current average latency between the data being available in source and the sink writing the corresponding output - sourceStatuses: Array[SourceStatus] - Current statuses of the sources - sinkStatus: SinkStatus - Current status of the sink - triggerStatus - Low-level detailed status of the last completed/currently active trigger - latencies - getOffset, getBatch, full trigger, wal writes - timestamps - trigger start, finish, after getOffset, after getBatch - numRows - input, output, state total/updated rows for aggregations - `SourceStatus` has the following important fields - inputRate - Current rate (rows/sec) at which data is being generated by the source - processingRate - Current rate (rows/sec) at which the query is processing data from the source - triggerStatus - Low-level detailed status of the last completed/currently active trigger - Python API for `StreamingQuery.status()` ### Breaking changes to existing APIs **Existing direct public facing APIs** - Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`. - Branch 2.0 should have it deprecated, master should have it removed. **Existing advanced listener APIs** - `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus` - Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status) - Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`. - Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`. - For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java. ## How was this patch tested? Old and new unit tests. - Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite. - New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite. - New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite. - Source-specific tests for making sure input rows are counted are is source-specific test suites. - Additional tests to test minor additions in LocalTableScanExec, StateStore, etc. Metrics also manually tested using Ganglia sink Author: Tathagata Das Closes #15472 from tdas/SPARK-17731-branch-2.0. --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 27 ++ project/MimaExcludes.scala | 10 + python/pyspark/sql/streaming.py | 301 +++++++++++++++++ .../spark/sql/catalyst/trees/TreeNode.scala | 7 + .../sql/execution/LocalTableScanExec.scala | 5 +- .../streaming/StatefulAggregate.scala | 31 +- .../execution/streaming/StreamExecution.scala | 307 ++++++++++++++---- .../execution/streaming/StreamMetrics.scala | 242 ++++++++++++++ .../sql/execution/streaming/memory.scala | 7 + .../state/HDFSBackedStateStoreProvider.scala | 2 + .../streaming/state/StateStore.scala | 3 + .../apache/spark/sql/internal/SQLConf.scala | 6 + .../spark/sql/streaming/SinkStatus.scala | 28 +- .../spark/sql/streaming/SourceStatus.scala | 54 ++- .../spark/sql/streaming/StreamingQuery.scala | 13 +- .../sql/streaming/StreamingQueryInfo.scala | 37 --- .../streaming/StreamingQueryListener.scala | 8 +- .../sql/streaming/StreamingQueryStatus.scala | 139 ++++++++ .../execution/metric/SQLMetricsSuite.scala | 17 + .../streaming/StreamMetricsSuite.scala | 213 ++++++++++++ .../streaming/TextSocketStreamSuite.scala | 24 ++ .../streaming/state/StateStoreSuite.scala | 5 + .../sql/streaming/FileStreamSourceSuite.scala | 14 + .../spark/sql/streaming/StreamTest.scala | 72 ++++ .../streaming/StreamingAggregationSuite.scala | 54 +++ .../StreamingQueryListenerSuite.scala | 220 +++++-------- .../sql/streaming/StreamingQuerySuite.scala | 180 +++++++++- 27 files changed, 1753 insertions(+), 273 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index c640b93b0a2ee..8b5296ea135c7 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -264,6 +264,33 @@ class KafkaSourceSuite extends KafkaSourceTest { testUnsupportedConfig("kafka.auto.offset.reset", "latest") } + test("input row metrics") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val kafka = spark + .readStream + .format("kafka") + .option("subscribe", topic) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + AssertOnLastQueryStatus { status => + assert(status.triggerDetails.get("numRows.input.total").toInt > 0) + assert(status.sourceStatuses(0).processingRate > 0.0) + } + ) + } + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 423cbd465ee98..ddf53bbce65b4 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -787,6 +787,16 @@ object MimaExcludes { ) ++ Seq( // SPARK-16240: ML persistence backward compatibility for LDA ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.clustering.LDA$") + ) ++ Seq( + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.SourceStatus.offsetDesc"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status") ) } diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 118a02b6786b0..0df63a77f9d2d 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -189,6 +189,304 @@ def resetTerminated(self): self._jsqm.resetTerminated() +class StreamingQueryStatus(object): + """A class used to report information about the progress of a StreamingQuery. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jsqs): + self._jsqs = jsqs + + def __str__(self): + """ + Pretty string of this query status. + + >>> print(sqs) + StreamingQueryStatus: + Query name: query + Query id: 1 + Status timestamp: 123 + Input rate: 15.5 rows/sec + Processing rate 23.5 rows/sec + Latency: 345.0 ms + Trigger details: + isDataPresentInTrigger: true + isTriggerActive: true + latency.getBatch.total: 20 + latency.getOffset.total: 10 + numRows.input.total: 100 + triggerId: 5 + Source statuses [1 source]: + Source 1: MySource1 + Available offset: #0 + Input rate: 15.5 rows/sec + Processing rate: 23.5 rows/sec + Trigger details: + numRows.input.source: 100 + latency.getOffset.source: 10 + latency.getBatch.source: 20 + Sink status: MySink + Committed offsets: [#1, -] + """ + return self._jsqs.toString() + + @property + @ignore_unicode_prefix + @since(2.1) + def name(self): + """ + Name of the query. This name is unique across all active queries. + + >>> sqs.name + u'query' + """ + return self._jsqs.name() + + @property + @since(2.1) + def id(self): + """ + Id of the query. This id is unique across all queries that have been started in + the current process. + + >>> int(sqs.id) + 1 + """ + return self._jsqs.id() + + @property + @since(2.1) + def timestamp(self): + """ + Timestamp (ms) of when this query was generated. + + >>> int(sqs.timestamp) + 123 + """ + return self._jsqs.timestamp() + + @property + @since(2.1) + def inputRate(self): + """ + Current total rate (rows/sec) at which data is being generated by all the sources. + + >>> sqs.inputRate + 15.5 + """ + return self._jsqs.inputRate() + + @property + @since(2.1) + def processingRate(self): + """ + Current rate (rows/sec) at which the query is processing data from all the sources. + + >>> sqs.processingRate + 23.5 + """ + return self._jsqs.processingRate() + + @property + @since(2.1) + def latency(self): + """ + Current average latency between the data being available in source and the sink + writing the corresponding output. + + >>> sqs.latency + 345.0 + """ + if (self._jsqs.latency().nonEmpty()): + return self._jsqs.latency().get() + else: + return None + + @property + @ignore_unicode_prefix + @since(2.1) + def sourceStatuses(self): + """ + Current statuses of the sources as a list. + + >>> len(sqs.sourceStatuses) + 1 + >>> sqs.sourceStatuses[0].description + u'MySource1' + """ + return [SourceStatus(ss) for ss in self._jsqs.sourceStatuses()] + + @property + @ignore_unicode_prefix + @since(2.1) + def sinkStatus(self): + """ + Current status of the sink. + + >>> sqs.sinkStatus.description + u'MySink' + """ + return SinkStatus(self._jsqs.sinkStatus()) + + @property + @ignore_unicode_prefix + @since(2.1) + def triggerDetails(self): + """ + Low-level details of the currently active trigger (e.g. number of rows processed + in trigger, latency of intermediate steps, etc.). + + If no trigger is currently active, then it will have details of the last completed trigger. + + >>> sqs.triggerDetails + {u'triggerId': u'5', u'latency.getBatch.total': u'20', u'numRows.input.total': u'100', + u'isTriggerActive': u'true', u'latency.getOffset.total': u'10', + u'isDataPresentInTrigger': u'true'} + """ + return self._jsqs.triggerDetails() + + +class SourceStatus(object): + """ + Status and metrics of a streaming Source. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jss): + self._jss = jss + + def __str__(self): + """ + Pretty string of this source status. + + >>> print(sqs.sourceStatuses[0]) + SourceStatus: MySource1 + Available offset: #0 + Input rate: 15.5 rows/sec + Processing rate: 23.5 rows/sec + Trigger details: + numRows.input.source: 100 + latency.getOffset.source: 10 + latency.getBatch.source: 20 + """ + return self._jss.toString() + + @property + @ignore_unicode_prefix + @since(2.1) + def description(self): + """ + Description of the source corresponding to this status. + + >>> sqs.sourceStatuses[0].description + u'MySource1' + """ + return self._jss.description() + + @property + @ignore_unicode_prefix + @since(2.1) + def offsetDesc(self): + """ + Description of the current offset if known. + + >>> sqs.sourceStatuses[0].offsetDesc + u'#0' + """ + return self._jss.offsetDesc() + + @property + @since(2.1) + def inputRate(self): + """ + Current rate (rows/sec) at which data is being generated by the source. + + >>> sqs.sourceStatuses[0].inputRate + 15.5 + """ + return self._jss.inputRate() + + @property + @since(2.1) + def processingRate(self): + """ + Current rate (rows/sec) at which the query is processing data from the source. + + >>> sqs.sourceStatuses[0].processingRate + 23.5 + """ + return self._jss.processingRate() + + @property + @ignore_unicode_prefix + @since(2.1) + def triggerDetails(self): + """ + Low-level details of the currently active trigger (e.g. number of rows processed + in trigger, latency of intermediate steps, etc.). + + If no trigger is currently active, then it will have details of the last completed trigger. + + >>> sqs.sourceStatuses[0].triggerDetails + {u'numRows.input.source': u'100', u'latency.getOffset.source': u'10', + u'latency.getBatch.source': u'20'} + """ + return self._jss.triggerDetails() + + +class SinkStatus(object): + """ + Status and metrics of a streaming Sink. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jss): + self._jss = jss + + def __str__(self): + """ + Pretty string of this source status. + + >>> print(sqs.sinkStatus) + SinkStatus: MySink + Committed offsets: [#1, -] + """ + return self._jss.toString() + + @property + @ignore_unicode_prefix + @since(2.1) + def description(self): + """ + Description of the source corresponding to this status. + + >>> sqs.sinkStatus.description + u'MySink' + """ + return self._jss.description() + + @property + @ignore_unicode_prefix + @since(2.1) + def offsetDesc(self): + """ + Description of the current offsets up to which data has been written by the sink. + + >>> sqs.sinkStatus.offsetDesc + u'[#1, -]' + """ + return self._jss.offsetDesc() + + class Trigger(object): """Used to indicate how often results should be produced by a :class:`StreamingQuery`. @@ -751,11 +1049,14 @@ def _test(): globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) globs['df'] = \ globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') + globs['sqs'] = StreamingQueryStatus( + spark.sparkContext._jvm.org.apache.spark.sql.streaming.StreamingQueryStatus.testStatus()) (failure_count, test_count) = doctest.testmod( pyspark.sql.streaming, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) globs['spark'].stop() + if failure_count: exit(-1) 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 eeccba79e42ae..931d14dc18eea 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 @@ -158,6 +158,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { ret } + /** + * Returns a Seq containing the leaves in this tree. + */ + def collectLeaves(): Seq[BaseType] = { + this.collect { case p if p.children.isEmpty => p } + } + /** * Finds and returns the first [[TreeNode]] of the tree for which the given partial function * is defined (pre-order), and applies the partial function to it. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index 9f53a99346caa..c998e04223f39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -57,10 +57,13 @@ case class LocalTableScanExec( } override def executeCollect(): Array[InternalRow] = { + longMetric("numOutputRows").add(unsafeRows.size) unsafeRows } override def executeTake(limit: Int): Array[InternalRow] = { - unsafeRows.take(limit) + val taken = unsafeRows.take(limit) + longMetric("numOutputRows").add(taken.size) + taken } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index 4d0283fbef1d0..587ea7d02acab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.streaming.state._ import org.apache.spark.sql.execution.SparkPlan @@ -56,7 +57,12 @@ case class StateStoreRestoreExec( child: SparkPlan) extends execution.UnaryExecNode with StatefulOperator { + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + override protected def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + child.execute().mapPartitionsWithStateStore( getStateId.checkpointLocation, operatorId = getStateId.operatorId, @@ -69,6 +75,7 @@ case class StateStoreRestoreExec( iter.flatMap { row => val key = getKey(row) val savedState = store.get(key) + numOutputRows += 1 row +: savedState.toSeq } } @@ -86,7 +93,13 @@ case class StateStoreSaveExec( child: SparkPlan) extends execution.UnaryExecNode with StatefulOperator { + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), + "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows")) + override protected def doExecute(): RDD[InternalRow] = { + metrics // force lazy init at driver assert(returnAllStates.nonEmpty, "Incorrect planning in IncrementalExecution, returnAllStates have not been set") val saveAndReturnFunc = if (returnAllStates.get) saveAndReturnAll _ else saveAndReturnUpdated _ @@ -111,6 +124,10 @@ case class StateStoreSaveExec( private def saveAndReturnUpdated( store: StateStore, iter: Iterator[InternalRow]): Iterator[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val numTotalStateRows = longMetric("numTotalStateRows") + val numUpdatedStateRows = longMetric("numUpdatedStateRows") + new Iterator[InternalRow] { private[this] val baseIterator = iter private[this] val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) @@ -118,6 +135,7 @@ case class StateStoreSaveExec( override def hasNext: Boolean = { if (!baseIterator.hasNext) { store.commit() + numTotalStateRows += store.numKeys() false } else { true @@ -128,6 +146,8 @@ case class StateStoreSaveExec( val row = baseIterator.next().asInstanceOf[UnsafeRow] val key = getKey(row) store.put(key.copy(), row.copy()) + numOutputRows += 1 + numUpdatedStateRows += 1 row } } @@ -142,12 +162,21 @@ case class StateStoreSaveExec( store: StateStore, iter: Iterator[InternalRow]): Iterator[InternalRow] = { val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) + val numOutputRows = longMetric("numOutputRows") + val numTotalStateRows = longMetric("numTotalStateRows") + val numUpdatedStateRows = longMetric("numUpdatedStateRows") + while (iter.hasNext) { val row = iter.next().asInstanceOf[UnsafeRow] val key = getKey(row) store.put(key.copy(), row.copy()) + numUpdatedStateRows += 1 } store.commit() - store.iterator().map(_._2.asInstanceOf[InternalRow]) + numTotalStateRows += store.numKeys() + store.iterator().map { case (k, v) => + numOutputRows += 1 + v.asInstanceOf[InternalRow] + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 8e0688d6bf539..6330e0a911f47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ @@ -57,6 +57,7 @@ class StreamExecution( extends StreamingQuery with Logging { import org.apache.spark.sql.streaming.StreamingQueryListener._ + import StreamMetrics._ private val pollingDelayMs = sparkSession.conf.get(SQLConf.STREAMING_POLLING_DELAY) @@ -105,11 +106,22 @@ class StreamExecution( var lastExecution: QueryExecution = null @volatile - var streamDeathCause: StreamingQueryException = null + private var streamDeathCause: StreamingQueryException = null /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() + /** Metrics for this query */ + private val streamMetrics = + new StreamMetrics(uniqueSources.toSet, triggerClock, s"StructuredStreaming.$name") + + @volatile + private var currentStatus: StreamingQueryStatus = null + + /** Flag that signals whether any error with input metrics have already been logged */ + @volatile + private var metricWarningLogged: Boolean = false + /** * The thread that runs the micro-batches of this stream. Note that this thread must be * [[org.apache.spark.util.UninterruptibleThread]] to avoid potential deadlocks in using @@ -136,16 +148,14 @@ class StreamExecution( /** Whether the query is currently active or not */ override def isActive: Boolean = state == ACTIVE + /** Returns the current status of the query. */ + override def status: StreamingQueryStatus = currentStatus + /** Returns current status of all the sources. */ - override def sourceStatuses: Array[SourceStatus] = { - val localAvailableOffsets = availableOffsets - sources.map(s => - new SourceStatus(s.toString, localAvailableOffsets.get(s).map(_.toString))).toArray - } + override def sourceStatuses: Array[SourceStatus] = currentStatus.sourceStatuses.toArray /** Returns current status of the sink. */ - override def sinkStatus: SinkStatus = - new SinkStatus(sink.toString, committedOffsets.toCompositeOffset(sources).toString) + override def sinkStatus: SinkStatus = currentStatus.sinkStatus /** Returns the [[StreamingQueryException]] if the query was terminated by an exception. */ override def exception: Option[StreamingQueryException] = Option(streamDeathCause) @@ -176,7 +186,11 @@ class StreamExecution( // Mark ACTIVE and then post the event. QueryStarted event is synchronously sent to listeners, // so must mark this as ACTIVE first. state = ACTIVE - postEvent(new QueryStarted(this.toInfo)) // Assumption: Does not throw exception. + if (sparkSession.conf.get(SQLConf.STREAMING_METRICS_ENABLED)) { + sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) + } + updateStatus() + postEvent(new QueryStarted(currentStatus)) // Assumption: Does not throw exception. // Unblock starting thread startLatch.countDown() @@ -185,25 +199,41 @@ class StreamExecution( SparkSession.setActiveSession(sparkSession) triggerExecutor.execute(() => { - if (isActive) { - if (currentBatchId < 0) { - // We'll do this initialization only once - populateStartOffsets() - logDebug(s"Stream running from $committedOffsets to $availableOffsets") + streamMetrics.reportTriggerStarted(currentBatchId) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "Finding new data from sources") + updateStatus() + val isTerminated = reportTimeTaken(TRIGGER_LATENCY) { + if (isActive) { + if (currentBatchId < 0) { + // We'll do this initialization only once + populateStartOffsets() + logDebug(s"Stream running from $committedOffsets to $availableOffsets") + } else { + constructNextBatch() + } + if (dataAvailable) { + streamMetrics.reportTriggerDetail(IS_DATA_PRESENT_IN_TRIGGER, true) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "Processing new data") + updateStatus() + runBatch() + // We'll increase currentBatchId after we complete processing current batch's data + currentBatchId += 1 + } else { + streamMetrics.reportTriggerDetail(IS_DATA_PRESENT_IN_TRIGGER, false) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "No new data") + updateStatus() + Thread.sleep(pollingDelayMs) + } + true } else { - constructNextBatch() + false } - if (dataAvailable) { - runBatch() - // We'll increase currentBatchId after we complete processing current batch's data - currentBatchId += 1 - } else { - Thread.sleep(pollingDelayMs) - } - true - } else { - false } + // Update metrics and notify others + streamMetrics.reportTriggerFinished() + updateStatus() + postEvent(new QueryProgress(currentStatus)) + isTerminated }) } catch { case _: InterruptedException if state == TERMINATED => // interrupted by stop() @@ -221,8 +251,16 @@ class StreamExecution( } } finally { state = TERMINATED + + // Update metrics and status + streamMetrics.stop() + sparkSession.sparkContext.env.metricsSystem.removeSource(streamMetrics) + updateStatus() + + // Notify others sparkSession.streams.notifyQueryTermination(StreamExecution.this) - postEvent(new QueryTerminated(this.toInfo, exception.map(_.cause).map(Utils.exceptionString))) + postEvent( + new QueryTerminated(currentStatus, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } } @@ -248,7 +286,6 @@ class StreamExecution( committedOffsets = lastOffsets.toStreamProgress(sources) logDebug(s"Resuming with committed offsets: $committedOffsets") } - case None => // We are starting this stream for the first time. logInfo(s"Starting new streaming query.") currentBatchId = 0 @@ -278,8 +315,14 @@ class StreamExecution( val hasNewData = { awaitBatchLock.lock() try { - val newData = uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - availableOffsets ++= newData + reportTimeTaken(GET_OFFSET_LATENCY) { + val latestOffsets: Map[Source, Option[Offset]] = uniqueSources.map { s => + reportTimeTaken(s, SOURCE_GET_OFFSET_LATENCY) { + (s, s.getOffset) + } + }.toMap + availableOffsets ++= latestOffsets.filter { case (s, o) => o.nonEmpty }.mapValues(_.get) + } if (dataAvailable) { true @@ -292,16 +335,19 @@ class StreamExecution( } } if (hasNewData) { - assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), - s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") - logInfo(s"Committed offsets for batch $currentBatchId.") - - // Now that we have logged the new batch, no further processing will happen for - // the previous batch, and it is safe to discard the old metadata. - // Note that purge is exclusive, i.e. it purges everything before currentBatchId. - // NOTE: If StreamExecution implements pipeline parallelism (multiple batches in - // flight at the same time), this cleanup logic will need to change. - offsetLog.purge(currentBatchId) + reportTimeTaken(OFFSET_WAL_WRITE_LATENCY) { + assert( + offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), + s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") + logInfo(s"Committed offsets for batch $currentBatchId.") + + // Now that we have logged the new batch, no further processing will happen for + // the previous batch, and it is safe to discard the old metadata. + // Note that purge is exclusive, i.e. it purges everything before currentBatchId. + // NOTE: If StreamExecution implements pipeline parallelism (multiple batches in + // flight at the same time), this cleanup logic will need to change. + offsetLog.purge(currentBatchId) + } } else { awaitBatchLock.lock() try { @@ -311,26 +357,30 @@ class StreamExecution( awaitBatchLock.unlock() } } + reportTimestamp(GET_OFFSET_TIMESTAMP) } /** * Processes any data available between `availableOffsets` and `committedOffsets`. */ private def runBatch(): Unit = { - val startTime = System.nanoTime() - // TODO: Move this to IncrementalExecution. // Request unprocessed data from all sources. - val newData = availableOffsets.flatMap { - case (source, available) + val newData = reportTimeTaken(GET_BATCH_LATENCY) { + availableOffsets.flatMap { + case (source, available) if committedOffsets.get(source).map(_ != available).getOrElse(true) => - val current = committedOffsets.get(source) - val batch = source.getBatch(current, available) - logDebug(s"Retrieving data from $source: $current -> $available") - Some(source -> batch) - case _ => None - }.toMap + val current = committedOffsets.get(source) + val batch = reportTimeTaken(source, SOURCE_GET_BATCH_LATENCY) { + source.getBatch(current, available) + } + logDebug(s"Retrieving data from $source: $current -> $available") + Some(source -> batch) + case _ => None + } + } + reportTimestamp(GET_BATCH_TIMESTAMP) // A list of attributes that will need to be updated. var replacements = new ArrayBuffer[(Attribute, Attribute)] @@ -351,25 +401,24 @@ class StreamExecution( // Rewire the plan to use the new attributes that were returned by the source. val replacementMap = AttributeMap(replacements) - val newPlan = withNewSources transformAllExpressions { + val triggerLogicalPlan = withNewSources transformAllExpressions { case a: Attribute if replacementMap.contains(a) => replacementMap(a) } - val optimizerStart = System.nanoTime() - lastExecution = new IncrementalExecution( - sparkSession, - newPlan, - outputMode, - checkpointFile("state"), - currentBatchId) - - lastExecution.executedPlan - val optimizerTime = (System.nanoTime() - optimizerStart).toDouble / 1000000 - logDebug(s"Optimized batch in ${optimizerTime}ms") + val executedPlan = reportTimeTaken(OPTIMIZER_LATENCY) { + lastExecution = new IncrementalExecution( + sparkSession, + triggerLogicalPlan, + outputMode, + checkpointFile("state"), + currentBatchId) + lastExecution.executedPlan // Force the lazy generation of execution plan + } val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) sink.addBatch(currentBatchId, nextBatch) + reportNumRows(executedPlan, triggerLogicalPlan, newData) awaitBatchLock.lock() try { @@ -379,11 +428,8 @@ class StreamExecution( awaitBatchLock.unlock() } - val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 - logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") // Update committed offsets. committedOffsets ++= availableOffsets - postEvent(new QueryProgress(this.toInfo)) } private def postEvent(event: StreamingQueryListener.Event) { @@ -516,12 +562,131 @@ class StreamExecution( """.stripMargin } - private def toInfo: StreamingQueryInfo = { - new StreamingQueryInfo( - this.name, - this.id, - this.sourceStatuses, - this.sinkStatus) + /** + * Report row metrics of the executed trigger + * @param triggerExecutionPlan Execution plan of the trigger + * @param triggerLogicalPlan Logical plan of the trigger, generated from the query logical plan + * @param sourceToDF Source to DataFrame returned by the source.getBatch in this trigger + */ + private def reportNumRows( + triggerExecutionPlan: SparkPlan, + triggerLogicalPlan: LogicalPlan, + sourceToDF: Map[Source, DataFrame]): Unit = { + // We want to associate execution plan leaves to sources that generate them, so that we match + // the their metrics (e.g. numOutputRows) to the sources. To do this we do the following. + // Consider the translation from the streaming logical plan to the final executed plan. + // + // streaming logical plan (with sources) <==> trigger's logical plan <==> executed plan + // + // 1. We keep track of streaming sources associated with each leaf in the trigger's logical plan + // - Each logical plan leaf will be associated with a single streaming source. + // - There can be multiple logical plan leaves associated with a streaming source. + // - There can be leaves not associated with any streaming source, because they were + // generated from a batch source (e.g. stream-batch joins) + // + // 2. Assuming that the executed plan has same number of leaves in the same order as that of + // the trigger logical plan, we associate executed plan leaves with corresponding + // streaming sources. + // + // 3. For each source, we sum the metrics of the associated execution plan leaves. + // + val logicalPlanLeafToSource = sourceToDF.flatMap { case (source, df) => + df.logicalPlan.collectLeaves().map { leaf => leaf -> source } + } + val allLogicalPlanLeaves = triggerLogicalPlan.collectLeaves() // includes non-streaming sources + val allExecPlanLeaves = triggerExecutionPlan.collectLeaves() + val sourceToNumInputRows: Map[Source, Long] = + if (allLogicalPlanLeaves.size == allExecPlanLeaves.size) { + val execLeafToSource = allLogicalPlanLeaves.zip(allExecPlanLeaves).flatMap { + case (lp, ep) => logicalPlanLeafToSource.get(lp).map { source => ep -> source } + } + val sourceToNumInputRows = execLeafToSource.map { case (execLeaf, source) => + val numRows = execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L) + source -> numRows + } + sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source + } else { + if (!metricWarningLogged) { + def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" + logWarning( + "Could not report metrics as number leaves in trigger logical plan did not match that" + + s" of the execution plan:\n" + + s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + + s"execution plan leaves: ${toString(allExecPlanLeaves)}\n") + metricWarningLogged = true + } + Map.empty + } + val numOutputRows = triggerExecutionPlan.metrics.get("numOutputRows").map(_.value) + val stateNodes = triggerExecutionPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + + streamMetrics.reportNumInputRows(sourceToNumInputRows) + stateNodes.zipWithIndex.foreach { case (s, i) => + streamMetrics.reportTriggerDetail( + NUM_TOTAL_STATE_ROWS(i + 1), + s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) + streamMetrics.reportTriggerDetail( + NUM_UPDATED_STATE_ROWS(i + 1), + s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) + } + updateStatus() + } + + private def reportTimeTaken[T](triggerDetailKey: String)(body: => T): T = { + val startTime = triggerClock.getTimeMillis() + val result = body + val endTime = triggerClock.getTimeMillis() + val timeTaken = math.max(endTime - startTime, 0) + streamMetrics.reportTriggerDetail(triggerDetailKey, timeTaken) + updateStatus() + if (triggerDetailKey == TRIGGER_LATENCY) { + logInfo(s"Completed up to $availableOffsets in $timeTaken ms") + } + result + } + + private def reportTimeTaken[T](source: Source, triggerDetailKey: String)(body: => T): T = { + val startTime = triggerClock.getTimeMillis() + val result = body + val endTime = triggerClock.getTimeMillis() + streamMetrics.reportSourceTriggerDetail( + source, triggerDetailKey, math.max(endTime - startTime, 0)) + updateStatus() + result + } + + private def reportTimestamp(triggerDetailKey: String): Unit = { + streamMetrics.reportTriggerDetail(triggerDetailKey, triggerClock.getTimeMillis) + updateStatus() + } + + private def updateStatus(): Unit = { + val localAvailableOffsets = availableOffsets + val sourceStatuses = sources.map { s => + SourceStatus( + s.toString, + localAvailableOffsets.get(s).map(_.toString).getOrElse("-"), // TODO: use json if available + streamMetrics.currentSourceInputRate(s), + streamMetrics.currentSourceProcessingRate(s), + streamMetrics.currentSourceTriggerDetails(s)) + }.toArray + val sinkStatus = SinkStatus( + sink.toString, + committedOffsets.toCompositeOffset(sources).toString) + + currentStatus = + StreamingQueryStatus( + name = name, + id = id, + timestamp = triggerClock.getTimeMillis(), + inputRate = streamMetrics.currentInputRate(), + processingRate = streamMetrics.currentProcessingRate(), + latency = streamMetrics.currentLatency(), + sourceStatuses = sourceStatuses, + sinkStatus = sinkStatus, + triggerDetails = streamMetrics.currentTriggerDetails()) } trait State diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala new file mode 100644 index 0000000000000..e98d1883e4596 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.{util => ju} + +import scala.collection.mutable + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.{Source => CodahaleSource} +import org.apache.spark.util.Clock + +/** + * Class that manages all the metrics related to a StreamingQuery. It does the following. + * - Calculates metrics (rates, latencies, etc.) based on information reported by StreamExecution. + * - Allows the current metric values to be queried + * - Serves some of the metrics through Codahale/DropWizard metrics + * + * @param sources Unique set of sources in a query + * @param triggerClock Clock used for triggering in StreamExecution + * @param codahaleSourceName Root name for all the Codahale metrics + */ +class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceName: String) + extends CodahaleSource with Logging { + + import StreamMetrics._ + + // Trigger infos + private val triggerDetails = new mutable.HashMap[String, String] + private val sourceTriggerDetails = new mutable.HashMap[Source, mutable.HashMap[String, String]] + + // Rate estimators for sources and sinks + private val inputRates = new mutable.HashMap[Source, RateCalculator] + private val processingRates = new mutable.HashMap[Source, RateCalculator] + + // Number of input rows in the current trigger + private val numInputRows = new mutable.HashMap[Source, Long] + private var currentTriggerStartTimestamp: Long = -1 + private var previousTriggerStartTimestamp: Long = -1 + private var latency: Option[Double] = None + + override val sourceName: String = codahaleSourceName + override val metricRegistry: MetricRegistry = new MetricRegistry + + // =========== Initialization =========== + + // Metric names should not have . in them, so that all the metrics of a query are identified + // together in Ganglia as a single metric group + registerGauge("inputRate-total", currentInputRate) + registerGauge("processingRate-total", () => currentProcessingRate) + registerGauge("latency", () => currentLatency().getOrElse(-1.0)) + + sources.foreach { s => + inputRates.put(s, new RateCalculator) + processingRates.put(s, new RateCalculator) + sourceTriggerDetails.put(s, new mutable.HashMap[String, String]) + + registerGauge(s"inputRate-${s.toString}", () => currentSourceInputRate(s)) + registerGauge(s"processingRate-${s.toString}", () => currentSourceProcessingRate(s)) + } + + // =========== Setter methods =========== + + def reportTriggerStarted(triggerId: Long): Unit = synchronized { + numInputRows.clear() + triggerDetails.clear() + sourceTriggerDetails.values.foreach(_.clear()) + + reportTriggerDetail(TRIGGER_ID, triggerId) + sources.foreach(s => reportSourceTriggerDetail(s, TRIGGER_ID, triggerId)) + reportTriggerDetail(IS_TRIGGER_ACTIVE, true) + currentTriggerStartTimestamp = triggerClock.getTimeMillis() + reportTriggerDetail(START_TIMESTAMP, currentTriggerStartTimestamp) + } + + def reportTriggerDetail[T](key: String, value: T): Unit = synchronized { + triggerDetails.put(key, value.toString) + } + + def reportSourceTriggerDetail[T](source: Source, key: String, value: T): Unit = synchronized { + sourceTriggerDetails(source).put(key, value.toString) + } + + def reportNumInputRows(inputRows: Map[Source, Long]): Unit = synchronized { + numInputRows ++= inputRows + } + + def reportTriggerFinished(): Unit = synchronized { + require(currentTriggerStartTimestamp >= 0) + val currentTriggerFinishTimestamp = triggerClock.getTimeMillis() + reportTriggerDetail(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) + triggerDetails.remove(STATUS_MESSAGE) + reportTriggerDetail(IS_TRIGGER_ACTIVE, false) + + // Report number of rows + val totalNumInputRows = numInputRows.values.sum + reportTriggerDetail(NUM_INPUT_ROWS, totalNumInputRows) + numInputRows.foreach { case (s, r) => + reportSourceTriggerDetail(s, NUM_SOURCE_INPUT_ROWS, r) + } + + val currentTriggerDuration = currentTriggerFinishTimestamp - currentTriggerStartTimestamp + val previousInputIntervalOption = if (previousTriggerStartTimestamp >= 0) { + Some(currentTriggerStartTimestamp - previousTriggerStartTimestamp) + } else None + + // Update input rate = num rows received by each source during the previous trigger interval + // Interval is measures as interval between start times of previous and current trigger. + // + // TODO: Instead of trigger start, we should use time when getOffset was called on each source + // as this may be different for each source if there are many sources in the query plan + // and getOffset is called serially on them. + if (previousInputIntervalOption.nonEmpty) { + sources.foreach { s => + inputRates(s).update(numInputRows.getOrElse(s, 0), previousInputIntervalOption.get) + } + } + + // Update processing rate = num rows processed for each source in current trigger duration + sources.foreach { s => + processingRates(s).update(numInputRows.getOrElse(s, 0), currentTriggerDuration) + } + + // Update latency = if data present, 0.5 * previous trigger interval + current trigger duration + if (previousInputIntervalOption.nonEmpty && totalNumInputRows > 0) { + latency = Some((previousInputIntervalOption.get.toDouble / 2) + currentTriggerDuration) + } else { + latency = None + } + + previousTriggerStartTimestamp = currentTriggerStartTimestamp + currentTriggerStartTimestamp = -1 + } + + // =========== Getter methods =========== + + def currentInputRate(): Double = synchronized { + // Since we are calculating source input rates using the same time interval for all sources + // it is fine to calculate total input rate as the sum of per source input rate. + inputRates.map(_._2.currentRate).sum + } + + def currentSourceInputRate(source: Source): Double = synchronized { + inputRates(source).currentRate + } + + def currentProcessingRate(): Double = synchronized { + // Since we are calculating source processing rates using the same time interval for all sources + // it is fine to calculate total processing rate as the sum of per source processing rate. + processingRates.map(_._2.currentRate).sum + } + + def currentSourceProcessingRate(source: Source): Double = synchronized { + processingRates(source).currentRate + } + + def currentLatency(): Option[Double] = synchronized { latency } + + def currentTriggerDetails(): Map[String, String] = synchronized { triggerDetails.toMap } + + def currentSourceTriggerDetails(source: Source): Map[String, String] = synchronized { + sourceTriggerDetails(source).toMap + } + + // =========== Other methods =========== + + private def registerGauge[T](name: String, f: () => T)(implicit num: Numeric[T]): Unit = { + synchronized { + metricRegistry.register(name, new Gauge[T] { + override def getValue: T = f() + }) + } + } + + def stop(): Unit = synchronized { + triggerDetails.clear() + inputRates.valuesIterator.foreach { _.stop() } + processingRates.valuesIterator.foreach { _.stop() } + latency = None + } +} + +object StreamMetrics extends Logging { + /** Simple utility class to calculate rate while avoiding DivideByZero */ + class RateCalculator { + @volatile private var rate: Option[Double] = None + + def update(numRows: Long, timeGapMs: Long): Unit = { + if (timeGapMs > 0) { + rate = Some(numRows.toDouble * 1000 / timeGapMs) + } else { + rate = None + logDebug(s"Rate updates cannot with zero or negative time gap $timeGapMs") + } + } + + def currentRate: Double = rate.getOrElse(0.0) + + def stop(): Unit = { rate = None } + } + + + val TRIGGER_ID = "triggerId" + val IS_TRIGGER_ACTIVE = "isTriggerActive" + val IS_DATA_PRESENT_IN_TRIGGER = "isDataPresentInTrigger" + val STATUS_MESSAGE = "statusMessage" + + val START_TIMESTAMP = "timestamp.triggerStart" + val GET_OFFSET_TIMESTAMP = "timestamp.afterGetOffset" + val GET_BATCH_TIMESTAMP = "timestamp.afterGetBatch" + val FINISH_TIMESTAMP = "timestamp.triggerFinish" + + val GET_OFFSET_LATENCY = "latency.getOffset.total" + val GET_BATCH_LATENCY = "latency.getBatch.total" + val OFFSET_WAL_WRITE_LATENCY = "latency.offsetLogWrite" + val OPTIMIZER_LATENCY = "latency.optimizer" + val TRIGGER_LATENCY = "latency.fullTrigger" + val SOURCE_GET_OFFSET_LATENCY = "latency.getOffset.source" + val SOURCE_GET_BATCH_LATENCY = "latency.getBatch.source" + + val NUM_INPUT_ROWS = "numRows.input.total" + val NUM_SOURCE_INPUT_ROWS = "numRows.input.source" + def NUM_TOTAL_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.total" + def NUM_UPDATED_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.updated" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index e37f0c77795c3..53eebae41aabf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -112,6 +112,11 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } override def stop() {} + + def reset(): Unit = synchronized { + batches.clear() + currentOffset = new LongOffset(-1) + } } /** @@ -165,6 +170,8 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi logDebug(s"Skipping already committed batch: $batchId") } } + + override def toString(): String = "MemorySink" } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 3335755fd3b67..dce53492f6156 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -197,6 +197,8 @@ private[state] class HDFSBackedStateStoreProvider( allUpdates.values().asScala.toIterator } + override def numKeys(): Long = mapToUpdate.size() + /** * Whether all updates have been committed */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index a67fdceb3cee6..7132e284c28f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -77,6 +77,9 @@ trait StateStore { */ def updates(): Iterator[StoreUpdate] + /** Number of keys in the state store */ + def numKeys(): Long + /** * Whether all updates have been committed */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 22f29c77bad8f..452eeed12b8d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -582,6 +582,12 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(10L) + val STREAMING_METRICS_ENABLED = + SQLConfigBuilder("spark.sql.streaming.metricsEnabled") + .doc("Whether Dropwizard/Codahale metrics will be reported for active streaming queries.") + .booleanConf + .createWithDefault(false) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index de1efe961f8bd..c9911665f7d72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -18,17 +18,33 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.Sink +import org.apache.spark.sql.streaming.StreamingQueryStatus.indent /** * :: Experimental :: - * Status and metrics of a streaming [[Sink]]. + * Status and metrics of a streaming sink. * - * @param description Description of the source corresponding to this status - * @param offsetDesc Description of the current offset up to which data has been written by the sink + * @param description Description of the source corresponding to this status. + * @param offsetDesc Description of the current offsets up to which data has been written + * by the sink. * @since 2.0.0 */ @Experimental -class SinkStatus private[sql]( +class SinkStatus private( val description: String, - val offsetDesc: String) + val offsetDesc: String) { + + override def toString: String = + "SinkStatus:" + indent(prettyString) + + private[sql] def prettyString: String = { + s"""$description + |Committed offsets: $offsetDesc + |""".stripMargin + } +} + +/** Companion object, primarily for creating SinkStatus instances internally */ +private[sql] object SinkStatus { + def apply(desc: String, offsetDesc: String): SinkStatus = new SinkStatus(desc, offsetDesc) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index bd0c8485e4fdd..6ace4833be22f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -17,18 +17,60 @@ package org.apache.spark.sql.streaming +import java.{util => ju} + +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.Source +import org.apache.spark.sql.streaming.StreamingQueryStatus.indent /** * :: Experimental :: - * Status and metrics of a streaming [[Source]]. + * Status and metrics of a streaming Source. * - * @param description Description of the source corresponding to this status - * @param offsetDesc Description of the current [[Source]] offset if known + * @param description Description of the source corresponding to this status. + * @param offsetDesc Description of the current offset if known. + * @param inputRate Current rate (rows/sec) at which data is being generated by the source. + * @param processingRate Current rate (rows/sec) at which the query is processing data from + * the source. + * @param triggerDetails Low-level details of the currently active trigger (e.g. number of + * rows processed in trigger, latency of intermediate steps, etc.). + * If no trigger is active, then it will have details of the last completed + * trigger. * @since 2.0.0 */ @Experimental -class SourceStatus private[sql] ( +class SourceStatus private( val description: String, - val offsetDesc: Option[String]) + val offsetDesc: String, + val inputRate: Double, + val processingRate: Double, + val triggerDetails: ju.Map[String, String]) { + + override def toString: String = + "SourceStatus:" + indent(prettyString) + + private[sql] def prettyString: String = { + val triggerDetailsLines = + triggerDetails.asScala.map { case (k, v) => s"$k: $v" } + s"""$description + |Available offset: $offsetDesc + |Input rate: $inputRate rows/sec + |Processing rate: $processingRate rows/sec + |Trigger details: + |""".stripMargin + indent(triggerDetailsLines) + + } +} + +/** Companion object, primarily for creating SourceStatus instances internally */ +private[sql] object SourceStatus { + def apply( + desc: String, + offsetDesc: String, + inputRate: Double, + processingRate: Double, + triggerDetails: Map[String, String]): SourceStatus = { + new SourceStatus(desc, offsetDesc, inputRate, processingRate, triggerDetails.asJava) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index 91f0a1e3446a1..0a85414451981 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -62,13 +62,24 @@ trait StreamingQuery { */ def exception: Option[StreamingQueryException] + /** + * Returns the current status of the query. + * @since 2.0.2 + */ + def status: StreamingQueryStatus + /** * Returns current status of all the sources. * @since 2.0.0 */ + @deprecated("use status.sourceStatuses", "2.0.2") def sourceStatuses: Array[SourceStatus] - /** Returns current status of the sink. */ + /** + * Returns current status of the sink. + * @since 2.0.0 + */ + @deprecated("use status.sinkStatus", "2.0.2") def sinkStatus: SinkStatus /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala deleted file mode 100644 index 1af2668817eae..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ /dev/null @@ -1,37 +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.streaming - -import org.apache.spark.annotation.Experimental - -/** - * :: Experimental :: - * A class used to report information about the progress of a [[StreamingQuery]]. - * - * @param name The [[StreamingQuery]] name. This name is unique across all active queries. - * @param id The [[StreamingQuery]] id. This id is unique across - * all queries that have been started in the current process. - * @param sourceStatuses The current statuses of the [[StreamingQuery]]'s sources. - * @param sinkStatus The current status of the [[StreamingQuery]]'s sink. - */ -@Experimental -class StreamingQueryInfo private[sql]( - val name: String, - val id: Long, - val sourceStatuses: Seq[SourceStatus], - val sinkStatus: SinkStatus) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 8a8855d85a4c7..69790e33b2168 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -84,7 +84,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryStarted private[sql](val queryInfo: StreamingQueryInfo) extends Event + class QueryStarted private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -92,19 +92,19 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryProgress private[sql](val queryInfo: StreamingQueryInfo) extends Event + class QueryProgress private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: * Event representing that termination of a query * - * @param queryInfo Information about the status of the query. + * @param queryStatus Information about the status of the query. * @param exception The exception message of the [[StreamingQuery]] if the query was terminated * with an exception. Otherwise, it will be `None`. * @since 2.0.0 */ @Experimental class QueryTerminated private[sql]( - val queryInfo: StreamingQueryInfo, + val queryStatus: StreamingQueryStatus, val exception: Option[String]) extends Event } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala new file mode 100644 index 0000000000000..47689928730d0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import java.{util => ju} + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset} + +/** + * :: Experimental :: + * A class used to report information about the progress of a [[StreamingQuery]]. + * + * @param name Name of the query. This name is unique across all active queries. + * @param id Id of the query. This id is unique across + * all queries that have been started in the current process. + * @param timestamp Timestamp (ms) of when this query was generated. + * @param inputRate Current rate (rows/sec) at which data is being generated by all the sources. + * @param processingRate Current rate (rows/sec) at which the query is processing data from + * all the sources. + * @param latency Current average latency between the data being available in source and the sink + * writing the corresponding output. + * @param sourceStatuses Current statuses of the sources. + * @param sinkStatus Current status of the sink. + * @param triggerDetails Low-level details of the currently active trigger (e.g. number of + * rows processed in trigger, latency of intermediate steps, etc.). + * If no trigger is active, then it will have details of the last completed + * trigger. + * @since 2.0.0 + */ +@Experimental +class StreamingQueryStatus private( + val name: String, + val id: Long, + val timestamp: Long, + val inputRate: Double, + val processingRate: Double, + val latency: Option[Double], + val sourceStatuses: Array[SourceStatus], + val sinkStatus: SinkStatus, + val triggerDetails: ju.Map[String, String]) { + + import StreamingQueryStatus._ + + override def toString: String = { + val sourceStatusLines = sourceStatuses.zipWithIndex.map { case (s, i) => + s"Source ${i + 1}:" + indent(s.prettyString) + } + val sinkStatusLines = sinkStatus.prettyString + val triggerDetailsLines = triggerDetails.asScala.map { case (k, v) => s"$k: $v" }.toSeq.sorted + val numSources = sourceStatuses.length + val numSourcesString = s"$numSources source" + { if (numSources > 1) "s" else "" } + + val allLines = s""" + |Query name: $name + |Query id: $id + |Status timestamp: $timestamp + |Input rate: $inputRate rows/sec + |Processing rate $processingRate rows/sec + |Latency: ${latency.getOrElse("-")} ms + |Trigger details: + |${indent(triggerDetailsLines)} + |Source statuses [$numSourcesString]: + |${indent(sourceStatusLines)} + |Sink status: ${indent(sinkStatusLines)}""".stripMargin + + s"StreamingQueryStatus:${indent(allLines)}" + } +} + +/** Companion object, primarily for creating StreamingQueryInfo instances internally */ +private[sql] object StreamingQueryStatus { + def apply( + name: String, + id: Long, + timestamp: Long, + inputRate: Double, + processingRate: Double, + latency: Option[Double], + sourceStatuses: Array[SourceStatus], + sinkStatus: SinkStatus, + triggerDetails: Map[String, String]): StreamingQueryStatus = { + new StreamingQueryStatus(name, id, timestamp, inputRate, processingRate, + latency, sourceStatuses, sinkStatus, triggerDetails.asJava) + } + + def indent(strings: Iterable[String]): String = strings.map(indent).mkString("\n") + def indent(string: String): String = string.split("\n").map(" " + _).mkString("\n") + + /** Create an instance of status for python testing */ + def testStatus(): StreamingQueryStatus = { + import org.apache.spark.sql.execution.streaming.StreamMetrics._ + StreamingQueryStatus( + name = "query", + id = 1, + timestamp = 123, + inputRate = 15.5, + processingRate = 23.5, + latency = Some(345), + sourceStatuses = Array( + SourceStatus( + desc = "MySource1", + offsetDesc = LongOffset(0).toString, + inputRate = 15.5, + processingRate = 23.5, + triggerDetails = Map( + NUM_SOURCE_INPUT_ROWS -> "100", + SOURCE_GET_OFFSET_LATENCY -> "10", + SOURCE_GET_BATCH_LATENCY -> "20"))), + sinkStatus = SinkStatus( + desc = "MySink", + offsetDesc = CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), + triggerDetails = Map( + TRIGGER_ID -> "5", + IS_TRIGGER_ACTIVE -> "true", + IS_DATA_PRESENT_IN_TRIGGER -> "true", + GET_OFFSET_LATENCY -> "10", + GET_BATCH_LATENCY -> "20", + NUM_INPUT_ROWS -> "100" + )) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index bba40c6510cfb..229d8814e0143 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.metric import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ @@ -85,6 +86,22 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } + test("LocalTableScanExec computes metrics in collect and take") { + val df1 = spark.createDataset(Seq(1, 2, 3)) + val logical = df1.queryExecution.logical + require(logical.isInstanceOf[LocalRelation]) + df1.collect() + val metrics1 = df1.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics1.contains("numOutputRows")) + assert(metrics1("numOutputRows").value === 3) + + val df2 = spark.createDataset(Seq(1, 2, 3)).limit(2) + df2.collect() + val metrics2 = df2.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics2.contains("numOutputRows")) + assert(metrics2("numOutputRows").value === 2) + } + test("Filter metrics") { // Assume the execution plan is // PhysicalRDD(nodeId = 1) -> Filter(nodeId = 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala new file mode 100644 index 0000000000000..938423db64745 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import org.scalactic.TolerantNumerics + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.ManualClock + +class StreamMetricsSuite extends SparkFunSuite { + import StreamMetrics._ + + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + + test("rates, latencies, trigger details - basic life cycle") { + val sm = newStreamMetrics(source) + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails().isEmpty) + + // When trigger started, the rates should not change, but should return + // reported trigger details + sm.reportTriggerStarted(1) + sm.reportTriggerDetail("key", "value") + sm.reportSourceTriggerDetail(source, "key2", "value2") + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails() === + Map(TRIGGER_ID -> "1", IS_TRIGGER_ACTIVE -> "true", + START_TIMESTAMP -> "0", "key" -> "value")) + assert(sm.currentSourceTriggerDetails(source) === + Map(TRIGGER_ID -> "1", "key2" -> "value2")) + + // Finishing the trigger should calculate the rates, except input rate which needs + // to have another trigger interval + sm.reportNumInputRows(Map(source -> 100L)) // 100 input rows, 10 output rows + clock.advance(1000) + sm.reportTriggerFinished() + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 100.0) // 100 input rows processed in 1 sec + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 100.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails() === + Map(TRIGGER_ID -> "1", IS_TRIGGER_ACTIVE -> "false", + START_TIMESTAMP -> "0", FINISH_TIMESTAMP -> "1000", + NUM_INPUT_ROWS -> "100", "key" -> "value")) + assert(sm.currentSourceTriggerDetails(source) === + Map(TRIGGER_ID -> "1", NUM_SOURCE_INPUT_ROWS -> "100", "key2" -> "value2")) + + // After another trigger starts, the rates and latencies should not change until + // new rows are reported + clock.advance(1000) + sm.reportTriggerStarted(2) + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 100.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 100.0) + assert(sm.currentLatency() === None) + + // Reporting new rows should update the rates and latencies + sm.reportNumInputRows(Map(source -> 200L)) // 200 input rows + clock.advance(500) + sm.reportTriggerFinished() + assert(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts + assert(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec + assert(sm.currentSourceInputRate(source) === 100.0) + assert(sm.currentSourceProcessingRate(source) === 400.0) + assert(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms + + // Rates should be set to 0 after stop + sm.stop() + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails().isEmpty) + } + + test("rates and latencies - after trigger with no data") { + val sm = newStreamMetrics(source) + // Trigger 1 with data + sm.reportTriggerStarted(1) + sm.reportNumInputRows(Map(source -> 100L)) // 100 input rows + clock.advance(1000) + sm.reportTriggerFinished() + + // Trigger 2 with data + clock.advance(1000) + sm.reportTriggerStarted(2) + sm.reportNumInputRows(Map(source -> 200L)) // 200 input rows + clock.advance(500) + sm.reportTriggerFinished() + + // Make sure that all rates are set + require(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts + require(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec + require(sm.currentSourceInputRate(source) === 100.0) + require(sm.currentSourceProcessingRate(source) === 400.0) + require(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms + + // Trigger 3 with data + clock.advance(500) + sm.reportTriggerStarted(3) + clock.advance(500) + sm.reportTriggerFinished() + + // Rates are set to zero and latency is set to None + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + sm.stop() + } + + test("rates - after trigger with multiple sources, and one source having no info") { + val source1 = TestSource(1) + val source2 = TestSource(2) + val sm = newStreamMetrics(source1, source2) + // Trigger 1 with data + sm.reportTriggerStarted(1) + sm.reportNumInputRows(Map(source1 -> 100L, source2 -> 100L)) + clock.advance(1000) + sm.reportTriggerFinished() + + // Trigger 2 with data + clock.advance(1000) + sm.reportTriggerStarted(2) + sm.reportNumInputRows(Map(source1 -> 200L, source2 -> 200L)) + clock.advance(500) + sm.reportTriggerFinished() + + // Make sure that all rates are set + assert(sm.currentInputRate() === 200.0) // 200*2 input rows generated in 2 seconds b/w starts + assert(sm.currentProcessingRate() === 800.0) // 200*2 output rows processed in 0.5 sec + assert(sm.currentSourceInputRate(source1) === 100.0) + assert(sm.currentSourceInputRate(source2) === 100.0) + assert(sm.currentSourceProcessingRate(source1) === 400.0) + assert(sm.currentSourceProcessingRate(source2) === 400.0) + + // Trigger 3 with data + clock.advance(500) + sm.reportTriggerStarted(3) + clock.advance(500) + sm.reportNumInputRows(Map(source1 -> 200L)) + sm.reportTriggerFinished() + + // Rates are set to zero and latency is set to None + assert(sm.currentInputRate() === 200.0) + assert(sm.currentProcessingRate() === 400.0) + assert(sm.currentSourceInputRate(source1) === 200.0) + assert(sm.currentSourceInputRate(source2) === 0.0) + assert(sm.currentSourceProcessingRate(source1) === 400.0) + assert(sm.currentSourceProcessingRate(source2) === 0.0) + sm.stop() + } + + test("registered Codahale metrics") { + import scala.collection.JavaConverters._ + val sm = newStreamMetrics(source) + val gaugeNames = sm.metricRegistry.getGauges().keySet().asScala + + // so that all metrics are considered as a single metric group in Ganglia + assert(!gaugeNames.exists(_.contains("."))) + assert(gaugeNames === Set( + "inputRate-total", + "inputRate-source0", + "processingRate-total", + "processingRate-source0", + "latency")) + } + + private def newStreamMetrics(sources: Source*): StreamMetrics = { + new StreamMetrics(sources.toSet, clock, "test") + } + + private val clock = new ManualClock() + private val source = TestSource(0) + + case class TestSource(id: Int) extends Source { + override def schema: StructType = StructType(Array.empty[StructField]) + override def getOffset: Option[Offset] = Some(new LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { null } + override def stop() {} + override def toString(): String = s"source$id" + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala index 6b0ba7acb4804..5174a0415304c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala @@ -156,6 +156,30 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before } } + test("input row metrics") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString) + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val batch = source.getBatch(None, source.getOffset.get).as[String] + batch.collect() + val numRowsMetric = + batch.queryExecution.executedPlan.collectLeaves().head.metrics.get("numOutputRows") + assert(numRowsMetric.nonEmpty) + assert(numRowsMetric.get.value === 1) + source.stop() + source = null + } + } + private class ServerThread extends Thread with Logging { private val serverSocket = new ServerSocket(0) private val messageQueue = new LinkedBlockingQueue[String]() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 984b84fd13fbd..06f1bd6c3bcc7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -74,6 +74,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth // Verify state after updating put(store, "a", 1) + assert(store.numKeys() === 1) intercept[IllegalStateException] { store.iterator() } @@ -85,7 +86,9 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth // Make updates, commit and then verify state put(store, "b", 2) put(store, "aa", 3) + assert(store.numKeys() === 3) remove(store, _.startsWith("a")) + assert(store.numKeys() === 1) assert(store.commit() === 1) assert(store.hasCommitted) @@ -107,7 +110,9 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth val reloadedProvider = new HDFSBackedStateStoreProvider( store.id, keySchema, valueSchema, StateStoreConf.empty, new Configuration) val reloadedStore = reloadedProvider.getStore(1) + assert(reloadedStore.numKeys() === 1) put(reloadedStore, "c", 4) + assert(reloadedStore.numKeys() === 2) assert(reloadedStore.commit() === 2) assert(rowsToSet(reloadedStore.iterator()) === Set("b" -> 2, "c" -> 4)) assert(getDataFromFiles(provider) === Set("b" -> 2, "c" -> 4)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 55c95ae285c1b..c18d843b9685c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -899,6 +899,20 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } } + + test("input row metrics") { + withTempDirs { case (src, tmp) => + val input = spark.readStream.format("text").load(src.getCanonicalPath) + testStream(input)( + AddTextFileData("100", src, tmp), + CheckAnswer("100"), + AssertOnLastQueryStatus { status => + assert(status.triggerDetails.get("numRows.input.total") === "1") + assert(status.sourceStatuses(0).processingRate > 0.0) + } + ) + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index d4c64e494a741..be24cbb56f954 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -28,6 +28,8 @@ import scala.util.control.NonFatal import org.scalatest.Assertions import org.scalatest.concurrent.{Eventually, Timeouts} +import org.scalatest.concurrent.AsyncAssertions.Waiter +import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.Span @@ -38,6 +40,7 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} @@ -193,6 +196,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } + case class AssertOnLastQueryStatus(condition: StreamingQueryStatus => Unit) + extends StreamAction + + /** * Executes the specified actions on the given streaming DataFrame and provides helpful * error messages in the case of failures or incorrect answers. @@ -294,9 +301,12 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val testThread = Thread.currentThread() val metadataRoot = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + val statusCollector = new QueryStatusCollector try { + spark.streams.addListener(statusCollector) startedTest.foreach { action => + logInfo(s"Processing test stream action: $action") action match { case StartStream(trigger, triggerClock) => verify(currentStream == null, "stream already running") @@ -394,6 +404,13 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val streamToAssert = Option(currentStream).getOrElse(lastStream) verify({ a.run(); true }, s"Assert failed: ${a.message}") + case a: AssertOnLastQueryStatus => + Eventually.eventually(timeout(streamingTimeout)) { + require(statusCollector.lastTriggerStatus.nonEmpty) + } + val status = statusCollector.lastTriggerStatus.get + verify({ a.condition(status); true }, "Assert on last query status failed") + case a: AddData => try { // Add data and get the source where it was added, and the expected offset of the @@ -465,6 +482,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { if (currentStream != null && currentStream.microBatchThread.isAlive) { currentStream.stop() } + spark.streams.removeListener(statusCollector) } } @@ -598,4 +616,58 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } } + + + class QueryStatusCollector extends StreamingQueryListener { + // to catch errors in the async listener events + @volatile private var asyncTestWaiter = new Waiter + + @volatile var startStatus: StreamingQueryStatus = null + @volatile var terminationStatus: StreamingQueryStatus = null + @volatile var terminationException: Option[String] = null + + private val progressStatuses = new mutable.ArrayBuffer[StreamingQueryStatus] + + /** Get the info of the last trigger that processed data */ + def lastTriggerStatus: Option[StreamingQueryStatus] = synchronized { + progressStatuses.filter { i => + i.triggerDetails.get("isTriggerActive").toBoolean == false && + i.triggerDetails.get("isDataPresentInTrigger").toBoolean == true + }.lastOption + } + + def reset(): Unit = { + startStatus = null + terminationStatus = null + progressStatuses.clear() + asyncTestWaiter = new Waiter + } + + def checkAsyncErrors(): Unit = { + asyncTestWaiter.await(timeout(10 seconds)) + } + + + override def onQueryStarted(queryStarted: QueryStarted): Unit = { + asyncTestWaiter { + startStatus = queryStarted.queryStatus + } + } + + override def onQueryProgress(queryProgress: QueryProgress): Unit = { + asyncTestWaiter { + assert(startStatus != null, "onQueryProgress called before onQueryStarted") + synchronized { progressStatuses += queryProgress.queryStatus } + } + } + + override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { + asyncTestWaiter { + assert(startStatus != null, "onQueryTerminated called before onQueryStarted") + terminationStatus = queryTerminated.queryStatus + terminationException = queryTerminated.exception + } + asyncTestWaiter.dismiss() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 8681199817fe6..e59b5491f90b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.InternalOutputModes._ +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scalalang.typed @@ -129,6 +130,59 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll { ) } + test("state metrics") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDS() + .flatMap(x => Seq(x, x + 1)) + .toDF("value") + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + implicit class RichStreamExecution(query: StreamExecution) { + def stateNodes: Seq[SparkPlan] = { + query.lastExecution.executedPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + } + } + + // Test with Update mode + testStream(aggregated, Update)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + + // Test with Complete mode + inputData.reset() + testStream(aggregated, Complete)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((1, 1), (2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 4 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + } + test("multiple keys") { val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 831543a47420a..9e0eefbc58aa5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -17,92 +17,97 @@ package org.apache.spark.sql.streaming -import java.util.concurrent.ConcurrentLinkedQueue - +import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ -import org.scalatest.concurrent.AsyncAssertions.Waiter -import org.scalatest.concurrent.Eventually._ -import org.scalatest.concurrent.PatienceConfiguration.Timeout -import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.util.JsonProtocol +import org.apache.spark.sql.functions._ +import org.apache.spark.util.{JsonProtocol, ManualClock} class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ - import StreamingQueryListener._ + import StreamingQueryListenerSuite._ + + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) after { spark.streams.active.foreach(_.stop()) assert(spark.streams.active.isEmpty) assert(addedListeners.isEmpty) // Make sure we don't leak any events to the next test - spark.sparkContext.listenerBus.waitUntilEmpty(10000) } - test("single listener") { - val listener = new QueryStatusCollector - val input = MemoryStream[Int] - withListenerAdded(listener) { - testStream(input.toDS)( - StartStream(), - AssertOnQuery("Incorrect query status in onQueryStarted") { query => - val status = listener.startStatus - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).description.contains("Memory")) - - // The source and sink offsets must be None as this must be called before the - // batches have started - assert(status.sourceStatuses(0).offsetDesc === None) - assert(status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString) - - // No progress events or termination events - assert(listener.progressStatuses.isEmpty) - assert(listener.terminationStatus === null) - true - }, - AddDataMemory(input, Seq(1, 2, 3)), - CheckAnswer(1, 2, 3), - AssertOnQuery("Incorrect query status in onQueryProgress") { query => - eventually(Timeout(streamingTimeout)) { + ignore("single listener, check trigger statuses") { + import StreamingQueryListenerSuite._ + clock = new ManualClock() + + /** Custom MemoryStream that waits for manual clock to reach a time */ + val inputData = new MemoryStream[Int](0, sqlContext) { + // Wait for manual clock to be 100 first time there is data + override def getOffset: Option[Offset] = { + val offset = super.getOffset + if (offset.nonEmpty) { + clock.waitTillTime(100) + } + offset + } - // There should be only on progress event as batch has been processed - assert(listener.progressStatuses.size === 1) - val status = listener.progressStatuses.peek() - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) - assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) + // Wait for manual clock to be 300 first time there is data + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + clock.waitTillTime(300) + super.getBatch(start, end) + } + } - // No termination events - assert(listener.terminationStatus === null) - } - true - }, - StopStream, - AssertOnQuery("Incorrect query status in onQueryTerminated") { query => - eventually(Timeout(streamingTimeout)) { - val status = listener.terminationStatus - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) - assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) - assert(listener.terminationException === None) - } - listener.checkAsyncErrors() - true - } - ) + // This is to make sure thatquery waits for manual clock to be 600 first time there is data + val mapped = inputData.toDS().agg(count("*")).as[Long].coalesce(1).map { x => + clock.waitTillTime(600) + x } + + testStream(mapped, OutputMode.Complete)( + StartStream(triggerClock = clock), + AddData(inputData, 1, 2), + AdvanceManualClock(100), // unblock getOffset, will block on getBatch + AdvanceManualClock(200), // unblock getBatch, will block on computation + AdvanceManualClock(300), // unblock computation + AssertOnQuery { _ => clock.getTimeMillis() === 600 }, + AssertOnLastQueryStatus { status: StreamingQueryStatus => + // Check the correctness of the trigger info of the last completed batch reported by + // onQueryProgress + assert(status.triggerDetails.get("triggerId") == "0") + assert(status.triggerDetails.get("isTriggerActive") === "false") + assert(status.triggerDetails.get("isDataPresentInTrigger") === "true") + + assert(status.triggerDetails.get("timestamp.triggerStart") === "0") + assert(status.triggerDetails.get("timestamp.afterGetOffset") === "100") + assert(status.triggerDetails.get("timestamp.afterGetBatch") === "300") + assert(status.triggerDetails.get("timestamp.triggerFinish") === "600") + + assert(status.triggerDetails.get("latency.getOffset.total") === "100") + assert(status.triggerDetails.get("latency.getBatch.total") === "200") + assert(status.triggerDetails.get("latency.optimizer") === "0") + assert(status.triggerDetails.get("latency.offsetLogWrite") === "0") + assert(status.triggerDetails.get("latency.fullTrigger") === "600") + + assert(status.triggerDetails.get("numRows.input.total") === "2") + assert(status.triggerDetails.get("numRows.state.aggregation1.total") === "1") + assert(status.triggerDetails.get("numRows.state.aggregation1.updated") === "1") + + assert(status.sourceStatuses.length === 1) + assert(status.sourceStatuses(0).triggerDetails.get("triggerId") === "0") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getOffset.source") === "100") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getBatch.source") === "200") + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "2") + }, + CheckAnswer(2) + ) } test("adding and removing listener") { @@ -172,56 +177,37 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStartedInfo = new StreamingQueryInfo( - "name", - 1, - Seq(new SourceStatus("source1", None), new SourceStatus("source2", None)), - new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString)) - val queryStarted = new StreamingQueryListener.QueryStarted(queryStartedInfo) + val queryStarted = new StreamingQueryListener.QueryStarted(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryStarted] - assertStreamingQueryInfoEquals(queryStarted.queryInfo, newQueryStarted.queryInfo) + assertStreamingQueryInfoEquals(queryStarted.queryStatus, newQueryStarted.queryStatus) } test("QueryProgress serialization") { - val queryProcessInfo = new StreamingQueryInfo( - "name", - 1, - Seq( - new SourceStatus("source1", Some(LongOffset(0).toString)), - new SourceStatus("source2", Some(LongOffset(1).toString))), - new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) - val queryProcess = new StreamingQueryListener.QueryProgress(queryProcessInfo) + val queryProcess = new StreamingQueryListener.QueryProgress(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryProgress] - assertStreamingQueryInfoEquals(queryProcess.queryInfo, newQueryProcess.queryInfo) + assertStreamingQueryInfoEquals(queryProcess.queryStatus, newQueryProcess.queryStatus) } test("QueryTerminated serialization") { - val queryTerminatedInfo = new StreamingQueryInfo( - "name", - 1, - Seq( - new SourceStatus("source1", Some(LongOffset(0).toString)), - new SourceStatus("source2", Some(LongOffset(1).toString))), - new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) val exception = new RuntimeException("exception") val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( - queryTerminatedInfo, + StreamingQueryStatus.testStatus, Some(exception.getMessage)) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryTerminated] - assertStreamingQueryInfoEquals(queryQueryTerminated.queryInfo, newQueryTerminated.queryInfo) + assertStreamingQueryInfoEquals(queryQueryTerminated.queryStatus, newQueryTerminated.queryStatus) assert(queryQueryTerminated.exception === newQueryTerminated.exception) } private def assertStreamingQueryInfoEquals( - expected: StreamingQueryInfo, - actual: StreamingQueryInfo): Unit = { + expected: StreamingQueryStatus, + actual: StreamingQueryStatus): Unit = { assert(expected.name === actual.name) assert(expected.sourceStatuses.size === actual.sourceStatuses.size) expected.sourceStatuses.zip(actual.sourceStatuses).foreach { @@ -243,7 +229,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { private def withListenerAdded(listener: StreamingQueryListener)(body: => Unit): Unit = { try { - failAfter(1 minute) { + failAfter(streamingTimeout) { spark.streams.addListener(listener) body } @@ -258,49 +244,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val listenerBus = spark.streams invokePrivate listenerBusMethod() listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) } +} - class QueryStatusCollector extends StreamingQueryListener { - // to catch errors in the async listener events - @volatile private var asyncTestWaiter = new Waiter - - @volatile var startStatus: StreamingQueryInfo = null - @volatile var terminationStatus: StreamingQueryInfo = null - @volatile var terminationException: Option[String] = null - - val progressStatuses = new ConcurrentLinkedQueue[StreamingQueryInfo] - - def reset(): Unit = { - startStatus = null - terminationStatus = null - progressStatuses.clear() - asyncTestWaiter = new Waiter - } - - def checkAsyncErrors(): Unit = { - asyncTestWaiter.await(timeout(streamingTimeout)) - } - - - override def onQueryStarted(queryStarted: QueryStarted): Unit = { - asyncTestWaiter { - startStatus = queryStarted.queryInfo - } - } - - override def onQueryProgress(queryProgress: QueryProgress): Unit = { - asyncTestWaiter { - assert(startStatus != null, "onQueryProgress called before onQueryStarted") - progressStatuses.add(queryProgress.queryInfo) - } - } - - override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { - asyncTestWaiter { - assert(startStatus != null, "onQueryTerminated called before onQueryStarted") - terminationStatus = queryTerminated.queryInfo - terminationException = queryTerminated.exception - } - asyncTestWaiter.dismiss() - } - } +object StreamingQueryListenerSuite { + // Singleton reference to clock that does not get serialized in task closures + @volatile var clock: ManualClock = null } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 88f1f188ab2af..9f8e2db966367 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,18 +17,27 @@ package org.apache.spark.sql.streaming +import org.scalactic.TolerantNumerics +import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter +import org.apache.spark.internal.Logging +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.streaming.StreamingQueryListener._ +import org.apache.spark.sql.types.StructType import org.apache.spark.SparkException -import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset, MemoryStream, StreamExecution} +import org.apache.spark.sql.execution.streaming._ import org.apache.spark.util.Utils -class StreamingQuerySuite extends StreamTest with BeforeAndAfter { +class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { import AwaitTerminationTester._ import testImplicits._ + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + after { sqlContext.streams.active.foreach(_.stop()) } @@ -100,31 +109,145 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } - testQuietly("source and sink statuses") { + testQuietly("query statuses") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map(6 / _) - testStream(mapped)( - AssertOnQuery(_.sourceStatuses.length === 1), + AssertOnQuery(q => q.status.name === q.name), + AssertOnQuery(q => q.status.id === q.id), + AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === "-"), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === "-"), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.description.contains("Memory")), AssertOnQuery(_.sinkStatus.offsetDesc === new CompositeOffset(None :: Nil).toString), + AddData(inputData, 1, 2), CheckAnswer(6, 3), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.status.inputRate >= 0.0), + AssertOnQuery(_.status.processingRate >= 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(0).toString), + AssertOnQuery(_.status.sourceStatuses(0).inputRate >= 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate >= 0.0), + AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(0)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(0).toString), + AssertOnQuery(_.sourceStatuses(0).inputRate >= 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), + AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(1).toString), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(1).toString), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + + StopStream, + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(1).toString), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(1).toString), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.status.triggerDetails.isEmpty), + + StartStream(), AddData(inputData, 0), ExpectFailure[SparkException], - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(2).toString), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(2).toString), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString) ) } + test("codahale metrics") { + val inputData = MemoryStream[Int] + + /** Whether metrics of a query is registered for reporting */ + def isMetricsRegistered(query: StreamingQuery): Boolean = { + val sourceName = s"StructuredStreaming.${query.name}" + val sources = spark.sparkContext.env.metricsSystem.getSourcesByName(sourceName) + require(sources.size <= 1) + sources.nonEmpty + } + // Disabled by default + assert(spark.conf.get("spark.sql.streaming.metricsEnabled").toBoolean === false) + + withSQLConf("spark.sql.streaming.metricsEnabled" -> "false") { + testStream(inputData.toDF)( + AssertOnQuery { q => !isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + + // Registered when enabled + withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + testStream(inputData.toDF)( + AssertOnQuery { q => isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + } + + test("input row calculation with mixed batch and streaming sources") { + val streamingTriggerDF = spark.createDataset(1 to 10).toDF + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value") + val staticInputDF = spark.createDataFrame(Seq(1 -> "1", 2 -> "2")).toDF("value", "anotherValue") + + // Trigger input has 10 rows, static input has 2 rows, + // therefore after the first trigger, the calculated input rows should be 10 + val status = getFirstTriggerStatus(streamingInputDF.join(staticInputDF, "value")) + assert(status.triggerDetails.get("numRows.input.total") === "10") + assert(status.sourceStatuses.size === 1) + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "10") + } + + test("input row calculation with trigger DF having multiple leaves") { + val streamingTriggerDF = + spark.createDataset(1 to 5).toDF.union(spark.createDataset(6 to 10).toDF) + require(streamingTriggerDF.logicalPlan.collectLeaves().size > 1) + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF) + + // After the first trigger, the calculated input rows should be 10 + val status = getFirstTriggerStatus(streamingInputDF) + assert(status.triggerDetails.get("numRows.input.total") === "10") + assert(status.sourceStatuses.size === 1) + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "10") + } + testQuietly("StreamExecution metadata garbage collection") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map(6 / _) @@ -149,6 +272,45 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } + /** Create a streaming DF that only execute one batch in which it returns the given static DF */ + private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame = { + require(!triggerDF.isStreaming) + // A streaming Source that generate only on trigger and returns the given Dataframe as batch + val source = new Source() { + override def schema: StructType = triggerDF.schema + override def getOffset: Option[Offset] = Some(LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = triggerDF + override def stop(): Unit = {} + } + StreamingExecutionRelation(source) + } + + /** Returns the query status at the end of the first trigger of streaming DF */ + private def getFirstTriggerStatus(streamingDF: DataFrame): StreamingQueryStatus = { + // A StreamingQueryListener that gets the query status after the first completed trigger + val listener = new StreamingQueryListener { + @volatile var firstStatus: StreamingQueryStatus = null + override def onQueryStarted(queryStarted: QueryStarted): Unit = { } + override def onQueryProgress(queryProgress: QueryProgress): Unit = { + if (firstStatus == null) firstStatus = queryProgress.queryStatus + } + override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { } + } + + try { + spark.streams.addListener(listener) + val q = streamingDF.writeStream.format("memory").queryName("test").start() + q.processAllAvailable() + eventually(timeout(streamingTimeout)) { + assert(listener.firstStatus != null) + } + listener.firstStatus + } finally { + spark.streams.active.map(_.stop()) + spark.streams.removeListener(listener) + } + } + /** * A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`. * From 01520de6b999c73b5e302778487d8bd1db8fdd2e Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Tue, 18 Oct 2016 00:49:57 -0700 Subject: [PATCH 1424/1470] [SQL][STREAMING][TEST] Fix flaky tests in StreamingQueryListenerSuite This work has largely been done by lw-lin in his PR #15497. This is a slight refactoring of it. ## What changes were proposed in this pull request? There were two sources of flakiness in StreamingQueryListener test. - When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock. ``` +-----------------------------------+--------------------------------+ | StreamExecution thread | testing thread | +-----------------------------------+--------------------------------+ | ManualClock.waitTillTime(100) { | | | _isWaiting = true | | | wait(10) | | | still in wait(10) | if (_isWaiting) advance(100) | | still in wait(10) | if (_isWaiting) advance(200) | <- this should be disallowed ! | still in wait(10) | if (_isWaiting) advance(300) | <- this should be disallowed ! | wake up from wait(10) | | | current time is 600 | | | _isWaiting = false | | | } | | +-----------------------------------+--------------------------------+ ``` - Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger. My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(200)` (see above) will wait for stream execution thread to complete the wait that started at time 0, and start a new wait at time 200 (i.e. time stamp after the previous `advance(100)`). In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest. ## How was this patch tested? Ran existing unit test MANY TIME in Jenkins Author: Tathagata Das Author: Liwei Lin Closes #15519 from tdas/metrics-flaky-test-fix. (cherry picked from commit 7d878cf2da04800bc4147b05610170865b148c64) Signed-off-by: Tathagata Das --- .../org/apache/spark/util/ManualClock.scala | 18 ++------- .../spark/sql/streaming/StreamSuite.scala | 4 +- .../spark/sql/streaming/StreamTest.scala | 38 ++++++++++++++++--- .../StreamingQueryListenerSuite.scala | 8 ++-- 4 files changed, 41 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala index 91a95871014f0..e7a65d74a440e 100644 --- a/core/src/main/scala/org/apache/spark/util/ManualClock.scala +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -26,8 +26,6 @@ package org.apache.spark.util */ private[spark] class ManualClock(private var time: Long) extends Clock { - private var _isWaiting = false - /** * @return `ManualClock` with initial time 0 */ @@ -59,19 +57,9 @@ private[spark] class ManualClock(private var time: Long) extends Clock { * @return current time reported by the clock when waiting finishes */ def waitTillTime(targetTime: Long): Long = synchronized { - _isWaiting = true - try { - while (time < targetTime) { - wait(10) - } - getTimeMillis() - } finally { - _isWaiting = false + while (time < targetTime) { + wait(10) } + getTimeMillis() } - - /** - * Returns whether there is any thread being blocked in `waitTillTime`. - */ - def isWaiting: Boolean = synchronized { _isWaiting } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index cdbad901dba8e..6bdf47901ae68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -161,7 +161,7 @@ class StreamSuite extends StreamTest { val inputData = MemoryStream[Int] testStream(inputData.toDS())( - StartStream(ProcessingTime("10 seconds"), new ManualClock), + StartStream(ProcessingTime("10 seconds"), new StreamManualClock), /* -- batch 0 ----------------------- */ // Add some data in batch 0 @@ -199,7 +199,7 @@ class StreamSuite extends StreamTest { /* Stop then restart the Stream */ StopStream, - StartStream(ProcessingTime("10 seconds"), new ManualClock), + StartStream(ProcessingTime("10 seconds"), new StreamManualClock(60 * 1000)), /* -- batch 1 rerun ----------------- */ // this batch 1 would re-run because the latest batch id logged in offset log is 1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index be24cbb56f954..04e20c7284d9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -199,6 +199,21 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { case class AssertOnLastQueryStatus(condition: StreamingQueryStatus => Unit) extends StreamAction + class StreamManualClock(time: Long = 0L) extends ManualClock(time) { + private var waitStartTime: Option[Long] = None + + override def waitTillTime(targetTime: Long): Long = synchronized { + try { + waitStartTime = Some(getTimeMillis()) + super.waitTillTime(targetTime) + } finally { + waitStartTime = None + } + } + + def isStreamWaitingAt(time: Long): Boolean = synchronized { waitStartTime.contains(time) } + } + /** * Executes the specified actions on the given streaming DataFrame and provides helpful @@ -302,7 +317,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val testThread = Thread.currentThread() val metadataRoot = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath val statusCollector = new QueryStatusCollector - + var manualClockExpectedTime = -1L try { spark.streams.addListener(statusCollector) startedTest.foreach { action => @@ -310,6 +325,12 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { action match { case StartStream(trigger, triggerClock) => verify(currentStream == null, "stream already running") + verify(triggerClock.isInstanceOf[SystemClock] + || triggerClock.isInstanceOf[StreamManualClock], + "Use either SystemClock or StreamManualClock to start the stream") + if (triggerClock.isInstanceOf[StreamManualClock]) { + manualClockExpectedTime = triggerClock.asInstanceOf[StreamManualClock].getTimeMillis() + } lastStream = currentStream currentStream = spark @@ -333,14 +354,19 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { case AdvanceManualClock(timeToAdd) => verify(currentStream != null, "can not advance manual clock when a stream is not running") - verify(currentStream.triggerClock.isInstanceOf[ManualClock], + verify(currentStream.triggerClock.isInstanceOf[StreamManualClock], s"can not advance clock of type ${currentStream.triggerClock.getClass}") - val clock = currentStream.triggerClock.asInstanceOf[ManualClock] + val clock = currentStream.triggerClock.asInstanceOf[StreamManualClock] + assert(manualClockExpectedTime >= 0) // Make sure we don't advance ManualClock too early. See SPARK-16002. - eventually("ManualClock has not yet entered the waiting state") { - assert(clock.isWaiting) + eventually("StreamManualClock has not yet entered the waiting state") { + assert(clock.isStreamWaitingAt(manualClockExpectedTime)) } - currentStream.triggerClock.asInstanceOf[ManualClock].advance(timeToAdd) + clock.advance(timeToAdd) + manualClockExpectedTime += timeToAdd + verify(clock.getTimeMillis() === manualClockExpectedTime, + s"Unexpected clock time after updating: " + + s"expecting $manualClockExpectedTime, current ${clock.getTimeMillis()}") case StopStream => verify(currentStream != null, "can not stop a stream that is not running") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 9e0eefbc58aa5..623f66a778eac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -43,9 +43,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Make sure we don't leak any events to the next test } - ignore("single listener, check trigger statuses") { + test("single listener, check trigger statuses") { import StreamingQueryListenerSuite._ - clock = new ManualClock() + clock = new StreamManualClock /** Custom MemoryStream that waits for manual clock to reach a time */ val inputData = new MemoryStream[Int](0, sqlContext) { @@ -81,7 +81,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { AssertOnLastQueryStatus { status: StreamingQueryStatus => // Check the correctness of the trigger info of the last completed batch reported by // onQueryProgress - assert(status.triggerDetails.get("triggerId") == "0") + assert(status.triggerDetails.containsKey("triggerId")) assert(status.triggerDetails.get("isTriggerActive") === "false") assert(status.triggerDetails.get("isDataPresentInTrigger") === "true") @@ -101,7 +101,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.triggerDetails.get("numRows.state.aggregation1.updated") === "1") assert(status.sourceStatuses.length === 1) - assert(status.sourceStatuses(0).triggerDetails.get("triggerId") === "0") + assert(status.sourceStatuses(0).triggerDetails.containsKey("triggerId")) assert(status.sourceStatuses(0).triggerDetails.get("latency.getOffset.source") === "100") assert(status.sourceStatuses(0).triggerDetails.get("latency.getBatch.source") === "200") assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "2") From 9e806f2a4fbc0e7d1441a3eda375cba2fda8ffe5 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 18 Oct 2016 02:29:55 -0700 Subject: [PATCH 1425/1470] [SQL][STREAMING][TEST] Follow up to remove Option.contains for Scala 2.10 compatibility ## What changes were proposed in this pull request? Scala 2.10 does not have Option.contains, which broke Scala 2.10 build. ## How was this patch tested? Locally compiled and ran sql/core unit tests in 2.10 Author: Tathagata Das Closes #15531 from tdas/metrics-flaky-test-fix-1. --- .../scala/org/apache/spark/sql/streaming/StreamTest.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 04e20c7284d9d..5aa363e6a7337 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -211,7 +211,9 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } - def isStreamWaitingAt(time: Long): Boolean = synchronized { waitStartTime.contains(time) } + def isStreamWaitingAt(time: Long): Boolean = synchronized { + waitStartTime == Some(time) + } } From 2aa25833c6f40af13a03a813b5f75d515f689577 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 18 Oct 2016 10:58:19 -0700 Subject: [PATCH 1426/1470] [SPARK-17751][SQL][BACKPORT-2.0] Remove spark.sql.eagerAnalysis and Output the Plan if Existed in AnalysisException ### What changes were proposed in this pull request? This PR is to backport the fix https://github.com/apache/spark/pull/15316 to 2.0. Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it. This PR also outputs the plan. Without the fix, the analysis error is like ``` cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12 ``` After the fix, the analysis error becomes: ``` org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12; 'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6] +- SubqueryAlias t +- Project [_1#2 AS k#5, _2#3 AS v#6] +- LocalRelation [_1#2, _2#3] ``` ### How was this patch tested? N/A Author: gatorsmile Closes #15529 from gatorsmile/eagerAnalysis20. --- .../scala/org/apache/spark/sql/AnalysisException.scala | 7 +++++++ .../org/apache/spark/sql/execution/debug/package.scala | 9 --------- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 10 ---------- .../scala/org/apache/spark/sql/SQLQueryTestSuite.scala | 3 +++ 4 files changed, 10 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 6911843999392..d3ee8f7a77445 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -43,6 +43,13 @@ class AnalysisException protected[sql] ( } override def getMessage: String = { + val planAnnotation = plan.map(p => s";\n$p").getOrElse("") + getSimpleMessage + planAnnotation + } + + // Outputs an exception without the logical plan. + // For testing only + def getSimpleMessage: String = { val lineAnnotation = line.map(l => s" line $l").getOrElse("") val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("") s"$message;$lineAnnotation$positionAnnotation" 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 d321f4cd76877..dd9d83767e221 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 @@ -69,15 +69,6 @@ package object debug { output } - /** - * Augments [[SparkSession]] with debug methods. - */ - implicit class DebugSQLContext(sparkSession: SparkSession) { - def debug(): Unit = { - sparkSession.conf.set(SQLConf.DATAFRAME_EAGER_ANALYSIS.key, false) - } - } - /** * Augments [[Dataset]]s with debug methods. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 452eeed12b8d1..8ba87c82329ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -396,14 +396,6 @@ object SQLConf { .intConf .createWithDefault(32) - // Whether to perform eager analysis when constructing a dataframe. - // Set to false when debugging requires the ability to look at invalid query plans. - val DATAFRAME_EAGER_ANALYSIS = SQLConfigBuilder("spark.sql.eagerAnalysis") - .internal() - .doc("When true, eagerly applies query analysis on DataFrame operations.") - .booleanConf - .createWithDefault(true) - // Whether to automatically resolve ambiguity in join conditions for self-joins. // See SPARK-6231. val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = @@ -713,8 +705,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { // See the comments of SCHEMA_STRING_LENGTH_THRESHOLD above for more information. def schemaStringLengthThreshold: Int = getConf(SCHEMA_STRING_LENGTH_THRESHOLD) - def dataFrameEagerAnalysis: Boolean = getConf(DATAFRAME_EAGER_ANALYSIS) - def dataFrameSelfJoinAutoResolveAmbiguity: Boolean = getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 55d5a56f1040a..02841d7bb03ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -220,6 +220,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) } catch { + case a: AnalysisException if a.plan.nonEmpty => + // Do not output the logical plan tree which contains expression IDs. + (StructType(Seq.empty), Seq(a.getClass.getName, a.getSimpleMessage)) case NonFatal(e) => // If there is an exception, put the exception class followed by the message. (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage)) From 26e978a93f029e1a1b5c7524d0b52c8141b70997 Mon Sep 17 00:00:00 2001 From: Yu Peng Date: Tue, 18 Oct 2016 13:23:31 -0700 Subject: [PATCH 1427/1470] [SPARK-17711] Compress rolled executor log ## What changes were proposed in this pull request? This PR adds support for executor log compression. ## How was this patch tested? Unit tests cc: yhuai tdas mengxr Author: Yu Peng Closes #15285 from loneknightpy/compress-executor-log. (cherry picked from commit 231f39e3f6641953a90bc4c40444ede63f363b23) Signed-off-by: Tathagata Das --- .../spark/deploy/worker/ui/LogPage.scala | 7 +- .../scala/org/apache/spark/util/Utils.scala | 80 ++++++++++++++-- .../util/logging/RollingFileAppender.scala | 45 +++++++-- .../spark/deploy/worker/ui/LogPageSuite.scala | 6 +- .../apache/spark/util/FileAppenderSuite.scala | 60 +++++++++++- .../org/apache/spark/util/UtilsSuite.scala | 92 ++++++++++++++----- docs/configuration.md | 8 ++ docs/spark-standalone.md | 9 ++ 8 files changed, 263 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 3473c41b935fd..465c214362b25 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -22,6 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Unparsed} +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} + import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils @@ -138,7 +140,8 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with val files = RollingFileAppender.getSortedRolledOverFiles(logDirectory, logType) logDebug(s"Sorted log files of type $logType in $logDirectory:\n${files.mkString("\n")}") - val totalLength = files.map { _.length }.sum + val fileLengths: Seq[Long] = files.map(Utils.getFileLength(_, worker.conf)) + val totalLength = fileLengths.sum val offset = offsetOption.getOrElse(totalLength - byteLength) val startIndex = { if (offset < 0) { @@ -151,7 +154,7 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with } val endIndex = math.min(startIndex + byteLength, totalLength) logDebug(s"Getting log from $startIndex to $endIndex") - val logText = Utils.offsetBytes(files, startIndex, endIndex) + val logText = Utils.offsetBytes(files, fileLengths, startIndex, endIndex) logDebug(s"Got log of length ${logText.length} bytes") (logText, startIndex, endIndex, totalLength) } catch { 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 7764fdc8ccad4..a8532b20b99b9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -27,6 +27,7 @@ import java.nio.file.Files import java.util.{Locale, Properties, Random, UUID} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean +import java.util.zip.GZIPInputStream import javax.net.ssl.HttpsURLConnection import scala.annotation.tailrec @@ -38,8 +39,10 @@ import scala.reflect.ClassTag import scala.util.Try import scala.util.control.{ControlThrowable, NonFatal} +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.google.common.io.{ByteStreams, Files => GFiles} import com.google.common.net.InetAddresses +import org.apache.commons.io.IOUtils import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} @@ -55,6 +58,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DYN_ALLOCATION_INITIAL_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS, EXECUTOR_INSTANCES} import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +import org.apache.spark.util.logging.RollingFileAppender /** CallSite represents a place in user code. It can have a short and a long form. */ private[spark] case class CallSite(shortForm: String, longForm: String) @@ -1448,14 +1452,72 @@ private[spark] object Utils extends Logging { CallSite(shortForm, longForm) } + private val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = + "spark.worker.ui.compressedLogFileLengthCacheSize" + private val DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE = 100 + private var compressedLogFileLengthCache: LoadingCache[String, java.lang.Long] = null + private def getCompressedLogFileLengthCache( + sparkConf: SparkConf): LoadingCache[String, java.lang.Long] = this.synchronized { + if (compressedLogFileLengthCache == null) { + val compressedLogFileLengthCacheSize = sparkConf.getInt( + UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF, + DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE) + compressedLogFileLengthCache = CacheBuilder.newBuilder() + .maximumSize(compressedLogFileLengthCacheSize) + .build[String, java.lang.Long](new CacheLoader[String, java.lang.Long]() { + override def load(path: String): java.lang.Long = { + Utils.getCompressedFileLength(new File(path)) + } + }) + } + compressedLogFileLengthCache + } + + /** + * Return the file length, if the file is compressed it returns the uncompressed file length. + * It also caches the uncompressed file size to avoid repeated decompression. The cache size is + * read from workerConf. + */ + def getFileLength(file: File, workConf: SparkConf): Long = { + if (file.getName.endsWith(".gz")) { + getCompressedLogFileLengthCache(workConf).get(file.getAbsolutePath) + } else { + file.length + } + } + + /** Return uncompressed file length of a compressed file. */ + private def getCompressedFileLength(file: File): Long = { + try { + // Uncompress .gz file to determine file size. + var fileSize = 0L + val gzInputStream = new GZIPInputStream(new FileInputStream(file)) + val bufSize = 1024 + val buf = new Array[Byte](bufSize) + var numBytes = IOUtils.read(gzInputStream, buf) + while (numBytes > 0) { + fileSize += numBytes + numBytes = IOUtils.read(gzInputStream, buf) + } + fileSize + } catch { + case e: Throwable => + logError(s"Cannot get file length of ${file}", e) + throw e + } + } + /** Return a string containing part of a file from byte 'start' to 'end'. */ - def offsetBytes(path: String, start: Long, end: Long): String = { + def offsetBytes(path: String, length: Long, start: Long, end: Long): String = { val file = new File(path) - val length = file.length() val effectiveEnd = math.min(length, end) val effectiveStart = math.max(0, start) val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) - val stream = new FileInputStream(file) + val stream = if (path.endsWith(".gz")) { + new GZIPInputStream(new FileInputStream(file)) + } else { + new FileInputStream(file) + } try { ByteStreams.skipFully(stream, effectiveStart) @@ -1471,8 +1533,8 @@ private[spark] object Utils extends Logging { * and `endIndex` is based on the cumulative size of all the files take in * the given order. See figure below for more details. */ - def offsetBytes(files: Seq[File], start: Long, end: Long): String = { - val fileLengths = files.map { _.length } + def offsetBytes(files: Seq[File], fileLengths: Seq[Long], start: Long, end: Long): String = { + assert(files.length == fileLengths.length) val startIndex = math.max(start, 0) val endIndex = math.min(end, fileLengths.sum) val fileToLength = files.zip(fileLengths).toMap @@ -1480,7 +1542,7 @@ private[spark] object Utils extends Logging { val stringBuffer = new StringBuffer((endIndex - startIndex).toInt) var sum = 0L - for (file <- files) { + files.zip(fileLengths).foreach { case (file, fileLength) => val startIndexOfFile = sum val endIndexOfFile = sum + fileToLength(file) logDebug(s"Processing file $file, " + @@ -1499,19 +1561,19 @@ private[spark] object Utils extends Logging { if (startIndex <= startIndexOfFile && endIndex >= endIndexOfFile) { // Case C: read the whole file - stringBuffer.append(offsetBytes(file.getAbsolutePath, 0, fileToLength(file))) + stringBuffer.append(offsetBytes(file.getAbsolutePath, fileLength, 0, fileToLength(file))) } else if (startIndex > startIndexOfFile && startIndex < endIndexOfFile) { // Case A and B: read from [start of required range] to [end of file / end of range] val effectiveStartIndex = startIndex - startIndexOfFile val effectiveEndIndex = math.min(endIndex - startIndexOfFile, fileToLength(file)) stringBuffer.append(Utils.offsetBytes( - file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) } else if (endIndex > startIndexOfFile && endIndex < endIndexOfFile) { // Case D: read from [start of file] to [end of require range] val effectiveStartIndex = math.max(startIndex - startIndexOfFile, 0) val effectiveEndIndex = endIndex - startIndexOfFile stringBuffer.append(Utils.offsetBytes( - file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) } sum += fileToLength(file) logDebug(s"After processing file $file, string built is ${stringBuffer.toString}") diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index a0eb05c7c0e82..5d8cec8447b53 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -17,9 +17,11 @@ package org.apache.spark.util.logging -import java.io.{File, FileFilter, InputStream} +import java.io._ +import java.util.zip.GZIPOutputStream import com.google.common.io.Files +import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf @@ -45,6 +47,7 @@ private[spark] class RollingFileAppender( import RollingFileAppender._ private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) + private val enableCompression = conf.getBoolean(ENABLE_COMPRESSION, false) /** Stop the appender */ override def stop() { @@ -76,6 +79,33 @@ private[spark] class RollingFileAppender( } } + // Roll the log file and compress if enableCompression is true. + private def rotateFile(activeFile: File, rolloverFile: File): Unit = { + if (enableCompression) { + val gzFile = new File(rolloverFile.getAbsolutePath + GZIP_LOG_SUFFIX) + var gzOutputStream: GZIPOutputStream = null + var inputStream: InputStream = null + try { + inputStream = new FileInputStream(activeFile) + gzOutputStream = new GZIPOutputStream(new FileOutputStream(gzFile)) + IOUtils.copy(inputStream, gzOutputStream) + inputStream.close() + gzOutputStream.close() + activeFile.delete() + } finally { + IOUtils.closeQuietly(inputStream) + IOUtils.closeQuietly(gzOutputStream) + } + } else { + Files.move(activeFile, rolloverFile) + } + } + + // Check if the rollover file already exists. + private def rolloverFileExist(file: File): Boolean = { + file.exists || new File(file.getAbsolutePath + GZIP_LOG_SUFFIX).exists + } + /** Move the active log file to a new rollover file */ private def moveFile() { val rolloverSuffix = rollingPolicy.generateRolledOverFileSuffix() @@ -83,8 +113,8 @@ private[spark] class RollingFileAppender( activeFile.getParentFile, activeFile.getName + rolloverSuffix).getAbsoluteFile logDebug(s"Attempting to rollover file $activeFile to file $rolloverFile") if (activeFile.exists) { - if (!rolloverFile.exists) { - Files.move(activeFile, rolloverFile) + if (!rolloverFileExist(rolloverFile)) { + rotateFile(activeFile, rolloverFile) logInfo(s"Rolled over $activeFile to $rolloverFile") } else { // In case the rollover file name clashes, make a unique file name. @@ -97,11 +127,11 @@ private[spark] class RollingFileAppender( altRolloverFile = new File(activeFile.getParent, s"${activeFile.getName}$rolloverSuffix--$i").getAbsoluteFile i += 1 - } while (i < 10000 && altRolloverFile.exists) + } while (i < 10000 && rolloverFileExist(altRolloverFile)) logWarning(s"Rollover file $rolloverFile already exists, " + s"rolled over $activeFile to file $altRolloverFile") - Files.move(activeFile, altRolloverFile) + rotateFile(activeFile, altRolloverFile) } } else { logWarning(s"File $activeFile does not exist") @@ -142,6 +172,9 @@ private[spark] object RollingFileAppender { val SIZE_DEFAULT = (1024 * 1024).toString val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" val DEFAULT_BUFFER_SIZE = 8192 + val ENABLE_COMPRESSION = "spark.executor.logs.rolling.enableCompression" + + val GZIP_LOG_SUFFIX = ".gz" /** * Get the sorted list of rolled over files. This assumes that the all the rolled @@ -158,6 +191,6 @@ private[spark] object RollingFileAppender { val file = new File(directory, activeFileName).getAbsoluteFile if (file.exists) Some(file) else None } - rolledOverFiles ++ activeFile + rolledOverFiles.sortBy(_.getName.stripSuffix(GZIP_LOG_SUFFIX)) ++ activeFile } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala index 72eaffb416981..4c3e96777940d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala @@ -22,16 +22,20 @@ import java.io.{File, FileWriter} import org.mockito.Mockito.{mock, when} import org.scalatest.PrivateMethodTester -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.worker.Worker class LogPageSuite extends SparkFunSuite with PrivateMethodTester { test("get logs simple") { val webui = mock(classOf[WorkerWebUI]) + val worker = mock(classOf[Worker]) val tmpDir = new File(sys.props("java.io.tmpdir")) val workDir = new File(tmpDir, "work-dir") workDir.mkdir() when(webui.workDir).thenReturn(workDir) + when(webui.worker).thenReturn(worker) + when(worker.conf).thenReturn(new SparkConf()) val logPage = new LogPage(webui) // Prepare some fake log files to read later 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 4fa9f9a8f590f..7e2da8e141532 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -20,11 +20,13 @@ package org.apache.spark.util import java.io._ import java.nio.charset.StandardCharsets import java.util.concurrent.CountDownLatch +import java.util.zip.GZIPInputStream import scala.collection.mutable.HashSet import scala.reflect._ import com.google.common.io.Files +import org.apache.commons.io.IOUtils import org.apache.log4j.{Appender, Level, Logger} import org.apache.log4j.spi.LoggingEvent import org.mockito.ArgumentCaptor @@ -72,6 +74,25 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { testRolling(appender, testOutputStream, textToAppend, rolloverIntervalMillis) } + test("rolling file appender - time-based rolling (compressed)") { + // setup input stream and appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) + val rolloverIntervalMillis = 100 + val durationMillis = 1000 + val numRollovers = durationMillis / rolloverIntervalMillis + val textToAppend = (1 to numRollovers).map( _.toString * 10 ) + + val sparkConf = new SparkConf() + sparkConf.set("spark.executor.logs.rolling.enableCompression", "true") + val appender = new RollingFileAppender(testInputStream, testFile, + new TimeBasedRollingPolicy(rolloverIntervalMillis, s"--HH-mm-ss-SSSS", false), + sparkConf, 10) + + testRolling( + appender, testOutputStream, textToAppend, rolloverIntervalMillis, isCompressed = true) + } + test("rolling file appender - size-based rolling") { // setup input stream and appender val testOutputStream = new PipedOutputStream() @@ -89,6 +110,25 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { } } + test("rolling file appender - size-based rolling (compressed)") { + // setup input stream and appender + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream, 100 * 1000) + val rolloverSize = 1000 + val textToAppend = (1 to 3).map( _.toString * 1000 ) + + val sparkConf = new SparkConf() + sparkConf.set("spark.executor.logs.rolling.enableCompression", "true") + val appender = new RollingFileAppender(testInputStream, testFile, + new SizeBasedRollingPolicy(rolloverSize, false), sparkConf, 99) + + val files = testRolling(appender, testOutputStream, textToAppend, 0, isCompressed = true) + files.foreach { file => + logInfo(file.toString + ": " + file.length + " bytes") + assert(file.length < rolloverSize) + } + } + test("rolling file appender - cleaning") { // setup input stream and appender val testOutputStream = new PipedOutputStream() @@ -273,7 +313,8 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { appender: FileAppender, outputStream: OutputStream, textToAppend: Seq[String], - sleepTimeBetweenTexts: Long + sleepTimeBetweenTexts: Long, + isCompressed: Boolean = false ): Seq[File] = { // send data to appender through the input stream, and wait for the data to be written val expectedText = textToAppend.mkString("") @@ -290,10 +331,23 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { // verify whether all the data written to rolled over files is same as expected val generatedFiles = RollingFileAppender.getSortedRolledOverFiles( testFile.getParentFile.toString, testFile.getName) - logInfo("Filtered files: \n" + generatedFiles.mkString("\n")) + logInfo("Generate files: \n" + generatedFiles.mkString("\n")) assert(generatedFiles.size > 1) + if (isCompressed) { + assert( + generatedFiles.filter(_.getName.endsWith(RollingFileAppender.GZIP_LOG_SUFFIX)).size > 0) + } val allText = generatedFiles.map { file => - Files.toString(file, StandardCharsets.UTF_8) + if (file.getName.endsWith(RollingFileAppender.GZIP_LOG_SUFFIX)) { + val inputStream = new GZIPInputStream(new FileInputStream(file)) + try { + IOUtils.toString(inputStream, StandardCharsets.UTF_8) + } finally { + IOUtils.closeQuietly(inputStream) + } + } else { + Files.toString(file, StandardCharsets.UTF_8) + } }.mkString("") assert(allText === expectedText) generatedFiles 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 4715fd29375d6..2741ad787e188 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -25,11 +25,13 @@ import java.nio.charset.StandardCharsets import java.text.DecimalFormatSymbols import java.util.Locale import java.util.concurrent.TimeUnit +import java.util.zip.GZIPOutputStream import scala.collection.mutable.ListBuffer import scala.util.Random import com.google.common.io.Files +import org.apache.commons.io.IOUtils import org.apache.commons.lang3.SystemUtils import org.apache.commons.math3.stat.inference.ChiSquareTest import org.apache.hadoop.conf.Configuration @@ -274,65 +276,109 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11" + sep + "00 h") } - test("reading offset bytes of a file") { + def getSuffix(isCompressed: Boolean): String = { + if (isCompressed) { + ".gz" + } else { + "" + } + } + + def writeLogFile(path: String, content: Array[Byte]): Unit = { + val outputStream = if (path.endsWith(".gz")) { + new GZIPOutputStream(new FileOutputStream(path)) + } else { + new FileOutputStream(path) + } + IOUtils.write(content, outputStream) + outputStream.close() + content.size + } + + private val workerConf = new SparkConf() + + def testOffsetBytes(isCompressed: Boolean): Unit = { val tmpDir2 = Utils.createTempDir() - val f1Path = tmpDir2 + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(StandardCharsets.UTF_8)) - f1.close() + val suffix = getSuffix(isCompressed) + val f1Path = tmpDir2 + "/f1" + suffix + writeLogFile(f1Path, "1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(StandardCharsets.UTF_8)) + val f1Length = Utils.getFileLength(new File(f1Path), workerConf) // Read first few bytes - assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + assert(Utils.offsetBytes(f1Path, f1Length, 0, 5) === "1\n2\n3") // Read some middle bytes - assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + assert(Utils.offsetBytes(f1Path, f1Length, 4, 11) === "3\n4\n5\n6") // Read last few bytes - assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + assert(Utils.offsetBytes(f1Path, f1Length, 12, 18) === "7\n8\n9\n") // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + assert(Utils.offsetBytes(f1Path, f1Length, -5, 5) === "1\n2\n3") // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + assert(Utils.offsetBytes(f1Path, f1Length, 12, 22) === "7\n8\n9\n") // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + assert(Utils.offsetBytes(f1Path, f1Length, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") Utils.deleteRecursively(tmpDir2) } - test("reading offset bytes across multiple files") { + test("reading offset bytes of a file") { + testOffsetBytes(isCompressed = false) + } + + test("reading offset bytes of a file (compressed)") { + testOffsetBytes(isCompressed = true) + } + + def testOffsetBytesMultipleFiles(isCompressed: Boolean): Unit = { val tmpDir = Utils.createTempDir() - val files = (1 to 3).map(i => new File(tmpDir, i.toString)) - Files.write("0123456789", files(0), StandardCharsets.UTF_8) - Files.write("abcdefghij", files(1), StandardCharsets.UTF_8) - Files.write("ABCDEFGHIJ", files(2), StandardCharsets.UTF_8) + val suffix = getSuffix(isCompressed) + val files = (1 to 3).map(i => new File(tmpDir, i.toString + suffix)) :+ new File(tmpDir, "4") + writeLogFile(files(0).getAbsolutePath, "0123456789".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(1).getAbsolutePath, "abcdefghij".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(2).getAbsolutePath, "ABCDEFGHIJ".getBytes(StandardCharsets.UTF_8)) + writeLogFile(files(3).getAbsolutePath, "9876543210".getBytes(StandardCharsets.UTF_8)) + val fileLengths = files.map(Utils.getFileLength(_, workerConf)) // Read first few bytes in the 1st file - assert(Utils.offsetBytes(files, 0, 5) === "01234") + assert(Utils.offsetBytes(files, fileLengths, 0, 5) === "01234") // Read bytes within the 1st file - assert(Utils.offsetBytes(files, 5, 8) === "567") + assert(Utils.offsetBytes(files, fileLengths, 5, 8) === "567") // Read bytes across 1st and 2nd file - assert(Utils.offsetBytes(files, 8, 18) === "89abcdefgh") + assert(Utils.offsetBytes(files, fileLengths, 8, 18) === "89abcdefgh") // Read bytes across 1st, 2nd and 3rd file - assert(Utils.offsetBytes(files, 5, 24) === "56789abcdefghijABCD") + assert(Utils.offsetBytes(files, fileLengths, 5, 24) === "56789abcdefghijABCD") + + // Read bytes across 3rd and 4th file + assert(Utils.offsetBytes(files, fileLengths, 25, 35) === "FGHIJ98765") // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(files, -5, 18) === "0123456789abcdefgh") + assert(Utils.offsetBytes(files, fileLengths, -5, 18) === "0123456789abcdefgh") // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(files, 18, 35) === "ijABCDEFGHIJ") + assert(Utils.offsetBytes(files, fileLengths, 18, 45) === "ijABCDEFGHIJ9876543210") // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(files, -5, 35) === "0123456789abcdefghijABCDEFGHIJ") + assert(Utils.offsetBytes(files, fileLengths, -5, 45) === + "0123456789abcdefghijABCDEFGHIJ9876543210") Utils.deleteRecursively(tmpDir) } + test("reading offset bytes across multiple files") { + testOffsetBytesMultipleFiles(isCompressed = false) + } + + test("reading offset bytes across multiple files (compressed)") { + testOffsetBytesMultipleFiles(isCompressed = true) + } + test("deserialize long value") { val testval : Long = 9730889947L val bbuf = ByteBuffer.allocate(8) diff --git a/docs/configuration.md b/docs/configuration.md index db088dde11919..d9eddf9ba59b3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -293,6 +293,14 @@ Apart from these, the following properties are also available, and may be useful Older log files will be deleted. Disabled by default. + + spark.executor.logs.rolling.enableCompression + false + + Enable executor log compression. If it is enabled, the rolled executor logs will be compressed. + Disabled by default. + + spark.executor.logs.rolling.maxSize (none) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 6f0f665c82e19..9915487fb9d97 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -250,6 +250,15 @@ SPARK_WORKER_OPTS supports the following system properties: especially if you run jobs very frequently. + + spark.worker.ui.compressedLogFileLengthCacheSize + 100 + + For compressed log files, the uncompressed file can only be computed by uncompressing the files. + Spark caches the uncompressed file size of compressed log files. This property controls the cache + size. + + # Connecting an Application to the Cluster From 6ef9231377c7cce949dc7a988bb9d7a5cb3e458d Mon Sep 17 00:00:00 2001 From: Weiqing Yang Date: Tue, 18 Oct 2016 13:38:14 -0700 Subject: [PATCH 1428/1470] [MINOR][DOC] Add more built-in sources in sql-programming-guide.md ## What changes were proposed in this pull request? Add more built-in sources in sql-programming-guide.md. ## How was this patch tested? Manually. Author: Weiqing Yang Closes #15522 from weiqingy/dsDoc. (cherry picked from commit 20dd11096cfda51e47b9dbe3b715a12ccbb4ce1d) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 0bd0093620a31..0a6bdb6ef052e 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -387,8 +387,8 @@ In the simplest form, the default data source (`parquet` unless otherwise config You can also manually specify the data source that will be used along with any extra options that you would like to pass to the data source. Data sources are specified by their fully qualified name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short -names (`json`, `parquet`, `jdbc`). DataFrames loaded from any data source type can be converted into other types -using this syntax. +names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data +source type can be converted into other types using this syntax.
    From f6b87939cb90bf4a0996b3728c1bccdf5e24dd4e Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 18 Oct 2016 14:01:49 -0700 Subject: [PATCH 1429/1470] [SPARK-17841][STREAMING][KAFKA] drain commitQueue ## What changes were proposed in this pull request? Actually drain commit queue rather than just iterating it. iterator() on a concurrent linked queue won't remove items from the queue, poll() will. ## How was this patch tested? Unit tests Author: cody koeninger Closes #15407 from koeninger/SPARK-17841. (cherry picked from commit cd106b050ff789b6de539956a7f01159ab15c820) Signed-off-by: Reynold Xin --- .../spark/streaming/kafka010/DirectKafkaInputDStream.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index 432537ebf05b2..7e57bb18cbd50 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -282,13 +282,13 @@ private[spark] class DirectKafkaInputDStream[K, V]( protected def commitAll(): Unit = { val m = new ju.HashMap[TopicPartition, OffsetAndMetadata]() - val it = commitQueue.iterator() - while (it.hasNext) { - val osr = it.next + var osr = commitQueue.poll() + while (null != osr) { val tp = osr.topicPartition val x = m.get(tp) val offset = if (null == x) { osr.untilOffset } else { Math.max(x.offset, osr.untilOffset) } m.put(tp, new OffsetAndMetadata(offset)) + osr = commitQueue.poll() } if (!m.isEmpty) { consumer.commitAsync(m, commitCallback.get) From 99943bf6905ca82a2c3e16e5d807fb572fa3dd3b Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 18 Oct 2016 17:31:21 -0700 Subject: [PATCH 1430/1470] [SPARK-17731][SQL][STREAMING][FOLLOWUP] Refactored StreamingQueryListener APIs for branch-2.0 This is the branch-2.0 PR of #15530 to make the APIs consistent with the master. Since these APIs are experimental and not direct user facing (StreamingQueryListener is advanced Structured Streaming APIs), its okay to change them in branch-2.0. ## What changes were proposed in this pull request? As per rxin request, here are further API changes - Changed `Stream(Started/Progress/Terminated)` events to `Stream*Event` - Changed the fields in `StreamingQueryListener.on***` from `query*` to `event` ## How was this patch tested? Existing unit tests. Author: Tathagata Das Closes #15535 from tdas/SPARK-17731-1-branch-2.0. --- project/MimaExcludes.scala | 12 +++++++++++- .../sql/execution/streaming/StreamExecution.scala | 15 ++++++++------- .../streaming/StreamingQueryListenerBus.scala | 8 ++++---- .../sql/streaming/StreamingQueryListener.scala | 14 +++++++------- .../apache/spark/sql/streaming/StreamTest.scala | 6 +++--- .../streaming/StreamingQueryListenerSuite.scala | 13 +++++++------ .../spark/sql/streaming/StreamingQuerySuite.scala | 6 +++--- 7 files changed, 43 insertions(+), 31 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ddf53bbce65b4..ee6e31a0ea50f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -788,6 +788,7 @@ object MimaExcludes { // SPARK-16240: ML persistence backward compatibility for LDA ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.ml.clustering.LDA$") ) ++ Seq( + // [SPARK-17731][SQL][Streaming] Metrics for structured streaming ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), @@ -796,7 +797,16 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.SourceStatus.offsetDesc"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryStarted"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryStarted"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryProgress"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener.onQueryTerminated") ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 6330e0a911f47..627b87b8955d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -165,7 +165,7 @@ class StreamExecution( new Path(new Path(checkpointRoot), name).toUri.toString /** - * Starts the execution. This returns only after the thread has started and [[QueryStarted]] event + * Starts the execution. This returns only after the thread has started and [[QueryStartedEvent]] * has been posted to all the listeners. */ def start(): Unit = { @@ -177,9 +177,10 @@ class StreamExecution( /** * Repeatedly attempts to run batches as data arrives. * - * Note that this method ensures that [[QueryStarted]] and [[QueryTerminated]] events are posted - * such that listeners are guaranteed to get a start event before a termination. Furthermore, this - * method also ensures that [[QueryStarted]] event is posted before the `start()` method returns. + * Note that this method ensures that [[QueryStartedEvent]] and [[QueryTerminatedEvent]] are + * posted such that listeners are guaranteed to get a start event before a termination. + * Furthermore, this method also ensures that [[QueryStartedEvent]] event is posted before the + * `start()` method returns. */ private def runBatches(): Unit = { try { @@ -190,7 +191,7 @@ class StreamExecution( sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) } updateStatus() - postEvent(new QueryStarted(currentStatus)) // Assumption: Does not throw exception. + postEvent(new QueryStartedEvent(currentStatus)) // Assumption: Does not throw exception. // Unblock starting thread startLatch.countDown() @@ -232,7 +233,7 @@ class StreamExecution( // Update metrics and notify others streamMetrics.reportTriggerFinished() updateStatus() - postEvent(new QueryProgress(currentStatus)) + postEvent(new QueryProgressEvent(currentStatus)) isTerminated }) } catch { @@ -260,7 +261,7 @@ class StreamExecution( // Notify others sparkSession.streams.notifyQueryTermination(StreamExecution.this) postEvent( - new QueryTerminated(currentStatus, exception.map(_.cause).map(Utils.exceptionString))) + new QueryTerminatedEvent(currentStatus, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index 1e663956f980b..fc2190d39da4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -40,7 +40,7 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) */ def post(event: StreamingQueryListener.Event) { event match { - case s: QueryStarted => + case s: QueryStartedEvent => postToAll(s) case _ => sparkListenerBus.post(event) @@ -59,11 +59,11 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) listener: StreamingQueryListener, event: StreamingQueryListener.Event): Unit = { event match { - case queryStarted: QueryStarted => + case queryStarted: QueryStartedEvent => listener.onQueryStarted(queryStarted) - case queryProgress: QueryProgress => + case queryProgress: QueryProgressEvent => listener.onQueryProgress(queryProgress) - case queryTerminated: QueryTerminated => + case queryTerminated: QueryTerminatedEvent => listener.onQueryTerminated(queryTerminated) case _ => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 69790e33b2168..9e311fae842be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -41,7 +41,7 @@ abstract class StreamingQueryListener { * don't block this method as it will block your query. * @since 2.0.0 */ - def onQueryStarted(queryStarted: QueryStarted): Unit + def onQueryStarted(event: QueryStartedEvent): Unit /** * Called when there is some status update (ingestion rate updated, etc.) @@ -49,16 +49,16 @@ abstract class StreamingQueryListener { * @note This method is asynchronous. The status in [[StreamingQuery]] will always be * latest no matter when this method is called. Therefore, the status of [[StreamingQuery]] * may be changed before/when you process the event. E.g., you may find [[StreamingQuery]] - * is terminated when you are processing [[QueryProgress]]. + * is terminated when you are processing [[QueryProgressEvent]]. * @since 2.0.0 */ - def onQueryProgress(queryProgress: QueryProgress): Unit + def onQueryProgress(event: QueryProgressEvent): Unit /** * Called when a query is stopped, with or without error. * @since 2.0.0 */ - def onQueryTerminated(queryTerminated: QueryTerminated): Unit + def onQueryTerminated(event: QueryTerminatedEvent): Unit } @@ -84,7 +84,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryStarted private[sql](val queryStatus: StreamingQueryStatus) extends Event + class QueryStartedEvent private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -92,7 +92,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryProgress private[sql](val queryStatus: StreamingQueryStatus) extends Event + class QueryProgressEvent private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -104,7 +104,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryTerminated private[sql]( + class QueryTerminatedEvent private[sql]( val queryStatus: StreamingQueryStatus, val exception: Option[String]) extends Event } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 5aa363e6a7337..ec49e1ab8d882 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -676,20 +676,20 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } - override def onQueryStarted(queryStarted: QueryStarted): Unit = { + override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { asyncTestWaiter { startStatus = queryStarted.queryStatus } } - override def onQueryProgress(queryProgress: QueryProgress): Unit = { + override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryProgress called before onQueryStarted") synchronized { progressStatuses += queryProgress.queryStatus } } } - override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { + override def onQueryTerminated(queryTerminated: QueryTerminatedEvent): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryTerminated called before onQueryStarted") terminationStatus = queryTerminated.queryStatus diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 623f66a778eac..ff843865a017e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -177,30 +177,31 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStarted = new StreamingQueryListener.QueryStarted(StreamingQueryStatus.testStatus) + val queryStarted = new StreamingQueryListener.QueryStartedEvent(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[StreamingQueryListener.QueryStarted] + .asInstanceOf[StreamingQueryListener.QueryStartedEvent] assertStreamingQueryInfoEquals(queryStarted.queryStatus, newQueryStarted.queryStatus) } test("QueryProgress serialization") { - val queryProcess = new StreamingQueryListener.QueryProgress(StreamingQueryStatus.testStatus) + val queryProcess = new StreamingQueryListener.QueryProgressEvent( + StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[StreamingQueryListener.QueryProgress] + .asInstanceOf[StreamingQueryListener.QueryProgressEvent] assertStreamingQueryInfoEquals(queryProcess.queryStatus, newQueryProcess.queryStatus) } test("QueryTerminated serialization") { val exception = new RuntimeException("exception") - val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( + val queryQueryTerminated = new StreamingQueryListener.QueryTerminatedEvent( StreamingQueryStatus.testStatus, Some(exception.getMessage)) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) - .asInstanceOf[StreamingQueryListener.QueryTerminated] + .asInstanceOf[StreamingQueryListener.QueryTerminatedEvent] assertStreamingQueryInfoEquals(queryQueryTerminated.queryStatus, newQueryTerminated.queryStatus) assert(queryQueryTerminated.exception === newQueryTerminated.exception) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 9f8e2db966367..92020be9789fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -290,11 +290,11 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { // A StreamingQueryListener that gets the query status after the first completed trigger val listener = new StreamingQueryListener { @volatile var firstStatus: StreamingQueryStatus = null - override def onQueryStarted(queryStarted: QueryStarted): Unit = { } - override def onQueryProgress(queryProgress: QueryProgress): Unit = { + override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { } + override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = { if (firstStatus == null) firstStatus = queryProgress.queryStatus } - override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { } + override def onQueryTerminated(queryTerminated: QueryTerminatedEvent): Unit = { } } try { From 3796a98cf3efad1dcbef536b295c7c47bf47d5dd Mon Sep 17 00:00:00 2001 From: Yu Peng Date: Tue, 18 Oct 2016 19:43:08 -0700 Subject: [PATCH 1431/1470] [SPARK-17711][TEST-HADOOP2.2] Fix hadoop2.2 compilation error ## What changes were proposed in this pull request? Fix hadoop2.2 compilation error. ## How was this patch tested? Existing tests. cc tdas zsxwing Author: Yu Peng Closes #15537 from loneknightpy/fix-17711. (cherry picked from commit 2629cd74602cfe77188b76428fed62a7a7149315) Signed-off-by: Shixiong Zhu --- core/src/main/scala/org/apache/spark/util/Utils.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index a8532b20b99b9..3d862f4e62160 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -42,7 +42,6 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.google.common.io.{ByteStreams, Files => GFiles} import com.google.common.net.InetAddresses -import org.apache.commons.io.IOUtils import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} @@ -1494,10 +1493,10 @@ private[spark] object Utils extends Logging { val gzInputStream = new GZIPInputStream(new FileInputStream(file)) val bufSize = 1024 val buf = new Array[Byte](bufSize) - var numBytes = IOUtils.read(gzInputStream, buf) + var numBytes = ByteStreams.read(gzInputStream, buf, 0, bufSize) while (numBytes > 0) { fileSize += numBytes - numBytes = IOUtils.read(gzInputStream, buf) + numBytes = ByteStreams.read(gzInputStream, buf, 0, bufSize) } fileSize } catch { From cdd2570e6dbfc5af68d0c9a49e4493e4e5e53020 Mon Sep 17 00:00:00 2001 From: Tommy YU Date: Tue, 18 Oct 2016 21:15:32 -0700 Subject: [PATCH 1432/1470] [SPARK-18001][DOCUMENT] fix broke link to SparkDataFrame ## What changes were proposed in this pull request? In http://spark.apache.org/docs/latest/sql-programming-guide.html, Section "Untyped Dataset Operations (aka DataFrame Operations)" Link to R DataFrame doesn't work that return The requested URL /docs/latest/api/R/DataFrame.html was not found on this server. Correct link is SparkDataFrame.html for spark 2.0 ## How was this patch tested? Manual checked. Author: Tommy YU Closes #15543 from Wenpei/spark-18001. (cherry picked from commit f39852e59883c214b0d007faffb406570ea3084b) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 0a6bdb6ef052e..3a90323760909 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -140,7 +140,7 @@ As an example, the following creates a DataFrame based on the content of a JSON ## Untyped Dataset Operations (aka DataFrame Operations) -DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/DataFrame.html). +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. From 995f602d27bdcf9e6787d93dbea2357e6dc6ccaa Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 19 Oct 2016 19:36:21 -0700 Subject: [PATCH 1433/1470] [SPARK-17989][SQL] Check ascendingOrder type in sort_array function rather than throwing ClassCastException ## What changes were proposed in this pull request? This PR proposes to check the second argument, `ascendingOrder` rather than throwing `ClassCastException` exception message. ```sql select sort_array(array('b', 'd'), '1'); ``` **Before** ``` 16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')] java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85) at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185) at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50) at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297) ``` **After** ``` Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7; ``` ## How was this patch tested? Unit test in `DataFrameFunctionsSuite`. Author: hyukjinkwon Closes #15532 from HyukjinKwon/SPARK-17989. (cherry picked from commit 4b2011ec9da1245923b5cbd883240fef0dbf3ef0) Signed-off-by: Reynold Xin --- .../expressions/collectionOperations.scala | 8 ++++++- .../test/resources/sql-tests/inputs/array.sql | 6 ++++++ .../resources/sql-tests/results/array.sql.out | 21 ++++++++++++++++--- 3 files changed, 31 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 2e8ea1107cee0..1efe2cb336b77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -112,7 +112,13 @@ case class SortArray(base: Expression, ascendingOrder: Expression) override def checkInputDataTypes(): TypeCheckResult = base.dataType match { case ArrayType(dt, _) if RowOrdering.isOrderable(dt) => - TypeCheckResult.TypeCheckSuccess + ascendingOrder match { + case Literal(_: Boolean, BooleanType) => + TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure( + "Sort order in second argument requires a boolean literal.") + } case ArrayType(dt, _) => TypeCheckResult.TypeCheckFailure( s"$prettyName does not support sorting array of type ${dt.simpleString}") diff --git a/sql/core/src/test/resources/sql-tests/inputs/array.sql b/sql/core/src/test/resources/sql-tests/inputs/array.sql index 4038a0da41d2b..984321ab795fc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/array.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/array.sql @@ -71,6 +71,12 @@ select sort_array(timestamp_array) from primitive_arrays; +-- sort_array with an invalid string literal for the argument of sort order. +select sort_array(array('b', 'd'), '1'); + +-- sort_array with an invalid null literal casted as boolean for the argument of sort order. +select sort_array(array('b', 'd'), cast(NULL as boolean)); + -- size select size(boolean_array), diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 4a1d149c1f362..499a3d5fb72f6 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -124,8 +124,23 @@ struct,sort_array(tinyint_array, -- !query 8 output [true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00.0,2016-11-15 20:54:00.0] - -- !query 9 +select sort_array(array('b', 'd'), '1') +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 + +-- !query 10 +select sort_array(array('b', 'd'), cast(NULL as boolean)) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot resolve 'sort_array(array('b', 'd'), CAST(NULL AS BOOLEAN))' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 + +-- !query 11 select size(boolean_array), size(tinyint_array), @@ -138,7 +153,7 @@ select size(date_array), size(timestamp_array) from primitive_arrays --- !query 9 schema +-- !query 11 schema struct --- !query 9 output +-- !query 11 output 1 2 2 2 2 2 2 2 2 2 From 4131623a8585fe99f79d82c24ab3b8b506d0d616 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 19 Oct 2016 23:41:38 -0700 Subject: [PATCH 1434/1470] [SPARK-18003][SPARK CORE] Fix bug of RDD zipWithIndex & zipWithUniqueId index value overflowing ## What changes were proposed in this pull request? - Fix bug of RDD `zipWithIndex` generating wrong result when one partition contains more than 2147483647 records. - Fix bug of RDD `zipWithUniqueId` generating wrong result when one partition contains more than 2147483647 records. ## How was this patch tested? test added. Author: WeichenXu Closes #15550 from WeichenXu123/fix_rdd_zipWithIndex_overflow. (cherry picked from commit 39755169fb5bb07332eef263b4c18ede1528812d) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../org/apache/spark/rdd/ZippedWithIndexRDD.scala | 5 ++--- .../main/scala/org/apache/spark/util/Utils.scala | 15 +++++++++++++++ .../scala/org/apache/spark/util/UtilsSuite.scala | 7 +++++++ 4 files changed, 25 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 34d32aacfb627..7013396392ae8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1278,7 +1278,7 @@ abstract class RDD[T: ClassTag]( def zipWithUniqueId(): RDD[(T, Long)] = withScope { val n = this.partitions.length.toLong this.mapPartitionsWithIndex { case (k, iter) => - iter.zipWithIndex.map { case (item, i) => + Utils.getIteratorZipWithIndex(iter, 0L).map { case (item, i) => (item, i * n + k) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala index 32931d59acb18..dff67371e93d5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala @@ -64,8 +64,7 @@ class ZippedWithIndexRDD[T: ClassTag](prev: RDD[T]) extends RDD[(T, Long)](prev) override def compute(splitIn: Partition, context: TaskContext): Iterator[(T, Long)] = { val split = splitIn.asInstanceOf[ZippedWithIndexRDDPartition] - firstParent[T].iterator(split.prev, context).zipWithIndex.map { x => - (x._1, split.startIndex + x._2) - } + val parentIter = firstParent[T].iterator(split.prev, context) + Utils.getIteratorZipWithIndex(parentIter, split.startIndex) } } 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 3d862f4e62160..1686edb502f69 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1767,6 +1767,21 @@ private[spark] object Utils extends Logging { count } + /** + * Generate a zipWithIndex iterator, avoid index value overflowing problem + * in scala's zipWithIndex + */ + def getIteratorZipWithIndex[T](iterator: Iterator[T], startIndex: Long): Iterator[(T, Long)] = { + new Iterator[(T, Long)] { + var index: Long = startIndex - 1L + def hasNext: Boolean = iterator.hasNext + def next(): (T, Long) = { + index += 1L + (iterator.next(), index) + } + } + } + /** * Creates a symlink. * 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 2741ad787e188..b67482aeba334 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -396,6 +396,13 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.getIteratorSize(iterator) === 5L) } + test("getIteratorZipWithIndex") { + val iterator = Utils.getIteratorZipWithIndex(Iterator(0, 1, 2), -1L + Int.MaxValue) + assert(iterator.toArray === Array( + (0, -1L + Int.MaxValue), (1, 0L + Int.MaxValue), (2, 1L + Int.MaxValue) + )) + } + test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files val parent: File = Utils.createTempDir() From e8923d21dd9f230e0ac23582033442e6fe476611 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 20 Oct 2016 10:50:34 -0700 Subject: [PATCH 1435/1470] [SPARK-17999][KAFKA][SQL] Add getPreferredLocations for KafkaSourceRDD ## What changes were proposed in this pull request? The newly implemented Structured Streaming `KafkaSource` did calculate the preferred locations for each topic partition, but didn't offer this information through RDD's `getPreferredLocations` method. So here propose to add this method in `KafkaSourceRDD`. ## How was this patch tested? Manual verification. Author: jerryshao Closes #15545 from jerryshao/SPARK-17999. (cherry picked from commit 947f4f25273161dc4719419a35613a71c2e2a150) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index 496af7e39abab..802dd040aed93 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -112,6 +112,11 @@ private[kafka010] class KafkaSourceRDD( buf.toArray } + override def getPreferredLocations(split: Partition): Seq[String] = { + val part = split.asInstanceOf[KafkaSourceRDDPartition] + part.offsetRange.preferredLoc.map(Seq(_)).getOrElse(Seq.empty) + } + override def compute( thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { From 6cc6cb2a95cbf5db7d1f7392a9e64e58af7ebc73 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 20 Oct 2016 21:12:55 -0700 Subject: [PATCH 1436/1470] [SPARKR] fix warnings ## What changes were proposed in this pull request? Fix for a bunch of test warnings that were added recently. We need to investigate why warnings are not turning into errors. ``` Warnings ----------------------------------------------------------------------- 1. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Sepal_Length instead of Sepal.Length as column name 2. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Sepal_Width instead of Sepal.Width as column name 3. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Petal_Length instead of Petal.Length as column name 4. createDataFrame uses files for large objects (test_sparkSQL.R#215) - Use Petal_Width instead of Petal.Width as column name Consider adding importFrom("utils", "object.size") to your NAMESPACE file. ``` ## How was this patch tested? unit tests Author: Felix Cheung Closes #15560 from felixcheung/rwarnings. (cherry picked from commit 3180272d2d49e440516085c0e4aebd5bad18bcad) Signed-off-by: Felix Cheung --- R/pkg/NAMESPACE | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 4c77d951247fd..62c33a7323ede 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -3,7 +3,7 @@ importFrom("methods", "setGeneric", "setMethod", "setOldClass") importFrom("methods", "is", "new", "signature", "show") importFrom("stats", "gaussian", "setNames") -importFrom("utils", "download.file", "packageVersion", "untar") +importFrom("utils", "download.file", "object.size", "packageVersion", "untar") # Disable native libraries till we figure out how to package it # See SPARKR-7839 diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index c9eedd553a56a..d33fcde35ef2d 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -212,7 +212,7 @@ test_that("createDataFrame uses files for large objects", { # To simulate a large file scenario, we set spark.r.maxAllocationLimit to a smaller value conf <- callJMethod(sparkSession, "conf") callJMethod(conf, "set", "spark.r.maxAllocationLimit", "100") - df <- createDataFrame(iris) + df <- suppressWarnings(createDataFrame(iris)) # Resetting the conf back to default value callJMethod(conf, "set", "spark.r.maxAllocationLimit", toString(.Machine$integer.max / 10)) From a65d40ab63fecc993136a98b8a820d2a8893a9ba Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 21 Oct 2016 11:25:01 -0700 Subject: [PATCH 1437/1470] [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness We should upgrade to the latest release of MiMa (0.1.11) in order to include a fix for a bug which led to flakiness in the MiMa checks (https://github.com/typesafehub/migration-manager/issues/115). Author: Josh Rosen Closes #15571 from JoshRosen/SPARK-18034. --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 8bebd7bcac58c..76597d27292ea 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,7 +6,7 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.8.2") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") -addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.9") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.11") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") From 78458a7ebeba6758890b01cc2b7417ab2fda221e Mon Sep 17 00:00:00 2001 From: Hossein Date: Fri, 21 Oct 2016 12:38:52 -0700 Subject: [PATCH 1438/1470] [SPARK-17811] SparkR cannot parallelize data.frame with NA or NULL in Date columns ## What changes were proposed in this pull request? NA date values are serialized as "NA" and NA time values are serialized as NaN from R. In the backend we did not have proper logic to deal with them. As a result we got an IllegalArgumentException for Date and wrong value for time. This PR adds support for deserializing NA as Date and Time. ## How was this patch tested? * [x] TODO Author: Hossein Closes #15421 from falaki/SPARK-17811. (cherry picked from commit e371040a0150e4ed748a7c25465965840b61ca63) Signed-off-by: Felix Cheung --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 13 ++++++++ .../scala/org/apache/spark/api/r/SerDe.scala | 31 +++++++++++++++---- 2 files changed, 38 insertions(+), 6 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index d33fcde35ef2d..b7b9de7e3e649 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -373,6 +373,19 @@ test_that("create DataFrame with different data types", { expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE)) }) +test_that("SPARK-17811: can create DataFrame containing NA as date and time", { + df <- data.frame( + id = 1:2, + time = c(as.POSIXlt("2016-01-10"), NA), + date = c(as.Date("2016-10-01"), NA)) + + DF <- collect(createDataFrame(df)) + expect_true(is.na(DF$date[2])) + expect_equal(DF$date[1], as.Date("2016-10-01")) + expect_true(is.na(DF$time[2])) + expect_equal(DF$time[1], as.POSIXlt("2016-01-10")) +}) + test_that("create DataFrame with complex types", { e <- new.env() assign("n", 3L, envir = e) 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 index e4932a4192d39..550e075a95129 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -125,15 +125,34 @@ private[spark] object SerDe { } def readDate(in: DataInputStream): Date = { - Date.valueOf(readString(in)) + try { + val inStr = readString(in) + if (inStr == "NA") { + null + } else { + Date.valueOf(inStr) + } + } catch { + // TODO: SPARK-18011 with some versions of R deserializing NA from R results in NASE + case _: NegativeArraySizeException => null + } } def readTime(in: DataInputStream): Timestamp = { - val seconds = in.readDouble() - val sec = Math.floor(seconds).toLong - val t = new Timestamp(sec * 1000L) - t.setNanos(((seconds - sec) * 1e9).toInt) - t + try { + val seconds = in.readDouble() + if (java.lang.Double.isNaN(seconds)) { + null + } else { + val sec = Math.floor(seconds).toLong + val t = new Timestamp(sec * 1000L) + t.setNanos(((seconds - sec) * 1e9).toInt) + t + } + } catch { + // TODO: SPARK-18011 with some versions of R deserializing NA from R results in NASE + case _: NegativeArraySizeException => null + } } def readBytesArr(in: DataInputStream): Array[Array[Byte]] = { From af2e6e0c9c85c40bc505ed1183857a8fb60fbd72 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 21 Oct 2016 13:07:29 -0700 Subject: [PATCH 1439/1470] [SPARK-17926][SQL][STREAMING] Added json for statuses ## What changes were proposed in this pull request? StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`. ## How was this patch tested? New unit tests Author: Tathagata Das Closes #15476 from tdas/SPARK-17926. (cherry picked from commit 7a531e3054f8d4820216ed379433559f57f571b8) Signed-off-by: Yin Huai --- python/pyspark/sql/streaming.py | 11 +- .../spark/sql/streaming/SinkStatus.scala | 18 ++- .../spark/sql/streaming/SourceStatus.scala | 23 +++- .../sql/streaming/StreamingQueryStatus.scala | 55 ++++++--- .../streaming/StreamingQueryStatusSuite.scala | 105 ++++++++++++++++++ 5 files changed, 187 insertions(+), 25 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 0df63a77f9d2d..cfe917bd12fc9 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -205,8 +205,7 @@ def __str__(self): Pretty string of this query status. >>> print(sqs) - StreamingQueryStatus: - Query name: query + Status of query 'query' Query id: 1 Status timestamp: 123 Input rate: 15.5 rows/sec @@ -220,7 +219,7 @@ def __str__(self): numRows.input.total: 100 triggerId: 5 Source statuses [1 source]: - Source 1: MySource1 + Source 1 - MySource1 Available offset: #0 Input rate: 15.5 rows/sec Processing rate: 23.5 rows/sec @@ -228,7 +227,7 @@ def __str__(self): numRows.input.source: 100 latency.getOffset.source: 10 latency.getBatch.source: 20 - Sink status: MySink + Sink status - MySink Committed offsets: [#1, -] """ return self._jsqs.toString() @@ -366,7 +365,7 @@ def __str__(self): Pretty string of this source status. >>> print(sqs.sourceStatuses[0]) - SourceStatus: MySource1 + Status of source MySource1 Available offset: #0 Input rate: 15.5 rows/sec Processing rate: 23.5 rows/sec @@ -457,7 +456,7 @@ def __str__(self): Pretty string of this source status. >>> print(sqs.sinkStatus) - SinkStatus: MySink + Status of sink MySink Committed offsets: [#1, -] """ return self._jss.toString() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index c9911665f7d72..ab19602207ad8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -17,6 +17,11 @@ package org.apache.spark.sql.streaming +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.streaming.StreamingQueryStatus.indent @@ -34,8 +39,19 @@ class SinkStatus private( val description: String, val offsetDesc: String) { + /** The compact JSON representation of this status. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this status. */ + def prettyJson: String = pretty(render(jsonValue)) + override def toString: String = - "SinkStatus:" + indent(prettyString) + "Status of sink " + indent(prettyString).trim + + private[sql] def jsonValue: JValue = { + ("description" -> JString(description)) ~ + ("offsetDesc" -> JString(offsetDesc)) + } private[sql] def prettyString: String = { s"""$description diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 6ace4833be22f..cfdf11370e06d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -21,8 +21,14 @@ import java.{util => ju} import scala.collection.JavaConverters._ +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.streaming.StreamingQueryStatus.indent +import org.apache.spark.util.JsonProtocol /** * :: Experimental :: @@ -47,8 +53,22 @@ class SourceStatus private( val processingRate: Double, val triggerDetails: ju.Map[String, String]) { + /** The compact JSON representation of this status. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this status. */ + def prettyJson: String = pretty(render(jsonValue)) + override def toString: String = - "SourceStatus:" + indent(prettyString) + "Status of source " + indent(prettyString).trim + + private[sql] def jsonValue: JValue = { + ("description" -> JString(description)) ~ + ("offsetDesc" -> JString(offsetDesc)) ~ + ("inputRate" -> JDouble(inputRate)) ~ + ("processingRate" -> JDouble(processingRate)) ~ + ("triggerDetails" -> JsonProtocol.mapToJson(triggerDetails.asScala)) + } private[sql] def prettyString: String = { val triggerDetailsLines = @@ -59,7 +79,6 @@ class SourceStatus private( |Processing rate: $processingRate rows/sec |Trigger details: |""".stripMargin + indent(triggerDetailsLines) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index 47689928730d0..a50b0d96c13f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -21,8 +21,14 @@ import java.{util => ju} import scala.collection.JavaConverters._ +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset} +import org.apache.spark.util.JsonProtocol /** * :: Experimental :: @@ -59,29 +65,46 @@ class StreamingQueryStatus private( import StreamingQueryStatus._ + /** The compact JSON representation of this status. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this status. */ + def prettyJson: String = pretty(render(jsonValue)) + override def toString: String = { val sourceStatusLines = sourceStatuses.zipWithIndex.map { case (s, i) => - s"Source ${i + 1}:" + indent(s.prettyString) + s"Source ${i + 1} - " + indent(s.prettyString).trim } - val sinkStatusLines = sinkStatus.prettyString + val sinkStatusLines = sinkStatus.prettyString.trim val triggerDetailsLines = triggerDetails.asScala.map { case (k, v) => s"$k: $v" }.toSeq.sorted val numSources = sourceStatuses.length val numSourcesString = s"$numSources source" + { if (numSources > 1) "s" else "" } - val allLines = s""" - |Query name: $name - |Query id: $id - |Status timestamp: $timestamp - |Input rate: $inputRate rows/sec - |Processing rate $processingRate rows/sec - |Latency: ${latency.getOrElse("-")} ms - |Trigger details: - |${indent(triggerDetailsLines)} - |Source statuses [$numSourcesString]: - |${indent(sourceStatusLines)} - |Sink status: ${indent(sinkStatusLines)}""".stripMargin - - s"StreamingQueryStatus:${indent(allLines)}" + val allLines = + s"""|Query id: $id + |Status timestamp: $timestamp + |Input rate: $inputRate rows/sec + |Processing rate $processingRate rows/sec + |Latency: ${latency.getOrElse("-")} ms + |Trigger details: + |${indent(triggerDetailsLines)} + |Source statuses [$numSourcesString]: + |${indent(sourceStatusLines)} + |Sink status - ${indent(sinkStatusLines).trim}""".stripMargin + + s"Status of query '$name'\n${indent(allLines)}" + } + + private[sql] def jsonValue: JValue = { + ("name" -> JString(name)) ~ + ("id" -> JInt(id)) ~ + ("timestamp" -> JInt(timestamp)) ~ + ("inputRate" -> JDouble(inputRate)) ~ + ("processingRate" -> JDouble(processingRate)) ~ + ("latency" -> latency.map(JDouble).getOrElse(JNothing)) ~ + ("triggerDetails" -> JsonProtocol.mapToJson(triggerDetails.asScala)) + ("sourceStatuses" -> JArray(sourceStatuses.map(_.jsonValue).toList)) ~ + ("sinkStatus" -> sinkStatus.jsonValue) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala new file mode 100644 index 0000000000000..1a98cf2ba74e6 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusSuite.scala @@ -0,0 +1,105 @@ +/* + * 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.streaming + +import org.apache.spark.SparkFunSuite + +class StreamingQueryStatusSuite extends SparkFunSuite { + test("toString") { + assert(StreamingQueryStatus.testStatus.sourceStatuses(0).toString === + """ + |Status of source MySource1 + | Available offset: #0 + | Input rate: 15.5 rows/sec + | Processing rate: 23.5 rows/sec + | Trigger details: + | numRows.input.source: 100 + | latency.getOffset.source: 10 + | latency.getBatch.source: 20 + """.stripMargin.trim, "SourceStatus.toString does not match") + + assert(StreamingQueryStatus.testStatus.sinkStatus.toString === + """ + |Status of sink MySink + | Committed offsets: [#1, -] + """.stripMargin.trim, "SinkStatus.toString does not match") + + assert(StreamingQueryStatus.testStatus.toString === + """ + |Status of query 'query' + | Query id: 1 + | Status timestamp: 123 + | Input rate: 15.5 rows/sec + | Processing rate 23.5 rows/sec + | Latency: 345.0 ms + | Trigger details: + | isDataPresentInTrigger: true + | isTriggerActive: true + | latency.getBatch.total: 20 + | latency.getOffset.total: 10 + | numRows.input.total: 100 + | triggerId: 5 + | Source statuses [1 source]: + | Source 1 - MySource1 + | Available offset: #0 + | Input rate: 15.5 rows/sec + | Processing rate: 23.5 rows/sec + | Trigger details: + | numRows.input.source: 100 + | latency.getOffset.source: 10 + | latency.getBatch.source: 20 + | Sink status - MySink + | Committed offsets: [#1, -] + """.stripMargin.trim, "StreamingQueryStatus.toString does not match") + + } + + test("json") { + assert(StreamingQueryStatus.testStatus.json === + """ + |{"sourceStatuses":[{"description":"MySource1","offsetDesc":"#0","inputRate":15.5, + |"processingRate":23.5,"triggerDetails":{"numRows.input.source":"100", + |"latency.getOffset.source":"10","latency.getBatch.source":"20"}}], + |"sinkStatus":{"description":"MySink","offsetDesc":"[#1, -]"}} + """.stripMargin.replace("\n", "").trim) + } + + test("prettyJson") { + assert( + StreamingQueryStatus.testStatus.prettyJson === + """ + |{ + | "sourceStatuses" : [ { + | "description" : "MySource1", + | "offsetDesc" : "#0", + | "inputRate" : 15.5, + | "processingRate" : 23.5, + | "triggerDetails" : { + | "numRows.input.source" : "100", + | "latency.getOffset.source" : "10", + | "latency.getBatch.source" : "20" + | } + | } ], + | "sinkStatus" : { + | "description" : "MySink", + | "offsetDesc" : "[#1, -]" + | } + |} + """.stripMargin.trim) + } +} From b113b5d9ff100385154ef0f836feb9805db163d2 Mon Sep 17 00:00:00 2001 From: w00228970 Date: Fri, 21 Oct 2016 14:43:55 -0700 Subject: [PATCH 1440/1470] [SPARK-17929][CORE] Fix deadlock when CoarseGrainedSchedulerBackend reset ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-17929 Now `CoarseGrainedSchedulerBackend` reset will get the lock, ``` protected def reset(): Unit = synchronized { numPendingExecutors = 0 executorsPendingToRemove.clear() // Remove all the lingering executors that should be removed but not yet. The reason might be // because (1) disconnected event is not yet received; (2) executors die silently. executorDataMap.toMap.foreach { case (eid, _) => driverEndpoint.askWithRetry[Boolean]( RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered."))) } } ``` but on removeExecutor also need the lock "CoarseGrainedSchedulerBackend.this.synchronized", this will cause deadlock. ``` private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { logDebug(s"Asked to remove executor $executorId with reason $reason") executorDataMap.get(executorId) match { case Some(executorInfo) => // This must be synchronized because variables mutated // in this block are read when requesting executors val killed = CoarseGrainedSchedulerBackend.this.synchronized { addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingLossReason -= executorId executorsPendingToRemove.remove(executorId).getOrElse(false) } ... ## How was this patch tested? manual test. Author: w00228970 Closes #15481 from scwf/spark-17929. (cherry picked from commit c1f344f1a09b8834bec70c1ece30b9bff63e55ea) Signed-off-by: Shixiong Zhu --- .../cluster/CoarseGrainedSchedulerBackend.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) 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 c6b3fdf439f5f..2c173dbf77727 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 @@ -386,15 +386,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * Reset the state of CoarseGrainedSchedulerBackend to the initial state. Currently it will only * be called in the yarn-client mode when AM re-registers after a failure. * */ - protected def reset(): Unit = synchronized { - numPendingExecutors = 0 - executorsPendingToRemove.clear() + protected def reset(): Unit = { + val executors = synchronized { + numPendingExecutors = 0 + executorsPendingToRemove.clear() + Set() ++ executorDataMap.keys + } // Remove all the lingering executors that should be removed but not yet. The reason might be // because (1) disconnected event is not yet received; (2) executors die silently. - executorDataMap.toMap.foreach { case (eid, _) => - driverEndpoint.askWithRetry[Boolean]( - RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered."))) + executors.foreach { eid => + removeExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered.")) } } From 3e9840f1d923a521d64bfc55fcbb6babd6045f06 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 21 Oct 2016 15:55:04 -0700 Subject: [PATCH 1441/1470] [SPARK-17812][SQL][KAFKA] Assign and specific startingOffsets for structured stream ## What changes were proposed in this pull request? startingOffsets takes specific per-topicpartition offsets as a json argument, usable with any consumer strategy assign with specific topicpartitions as a consumer strategy ## How was this patch tested? Unit tests Author: cody koeninger Closes #15504 from koeninger/SPARK-17812. (cherry picked from commit 268ccb9a48dfefc4d7bc85155e7e20a2dfe89307) Signed-off-by: Shixiong Zhu --- .../structured-streaming-kafka-integration.md | 38 ++++-- .../apache/spark/sql/kafka010/JsonUtils.scala | 93 ++++++++++++++ .../spark/sql/kafka010/KafkaSource.scala | 64 ++++++++-- .../sql/kafka010/KafkaSourceProvider.scala | 52 ++++---- .../spark/sql/kafka010/StartingOffsets.scala | 32 +++++ .../spark/sql/kafka010/JsonUtilsSuite.scala | 45 +++++++ .../spark/sql/kafka010/KafkaSourceSuite.scala | 114 ++++++++++++++++-- .../spark/sql/kafka010/KafkaTestUtils.scala | 14 ++- 8 files changed, 395 insertions(+), 57 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 668489addf82c..e851f210c92c4 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -150,16 +150,25 @@ The following options must be set for the Kafka source. + + + + + - + - @@ -174,16 +183,21 @@ The following configurations are optional:
    Optionvaluemeaning
    assignjson string {"topicA":[0,1],"topicB":[2,4]}Specific TopicPartitions to consume. + Only one of "assign", "subscribe" or "subscribePattern" + options can be specified for Kafka source.
    subscribe A comma-separated list of topicsThe topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be - specified for Kafka source.The topic list to subscribe. + Only one of "assign", "subscribe" or "subscribePattern" + options can be specified for Kafka source.
    subscribePattern Java regex stringThe pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" + The pattern used to subscribe to topic(s). + Only one of "assign, "subscribe" or "subscribePattern" options can be specified for Kafka source.
    - - - - + + + + - + + + + + + +
    Optionvaluedefaultmeaning
    startingOffset["earliest", "latest"]"latest"The start point when a query is started, either "earliest" which is from the earliest offset, - or "latest" which is just from the latest offset. Note: This only applies when a new Streaming q - uery is started, and that resuming will always pick up from where the query left off.startingOffsetsearliest, latest, or json string + {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}} + latestThe start point when a query is started, either "earliest" which is from the earliest offsets, + "latest" which is just from the latest offsets, or a json string specifying a starting offset for + each TopicPartition. In the json, -2 as an offset can be used to refer to earliest, -1 to latest. + Note: This only applies when a new Streaming query is started, and that resuming will always pick + up from where the query left off. Newly discovered partitions during a query will start at + earliest.
    failOnDataLoss[true, false]true or false true Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work @@ -215,10 +229,10 @@ Kafka's own configurations can be set via `DataStreamReader.option` with `kafka. Note that the following Kafka params cannot be set and the Kafka source will throw an exception: - **group.id**: Kafka source will create a unique group id for each query automatically. -- **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify +- **auto.offset.reset**: Set the source option `startingOffsets` to specify where to start instead. Structured Streaming manages which offsets are consumed internally, rather than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new - topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new + topics/partitions are dynamically subscribed. Note that `startingOffsets` only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off. - **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations to explicitly deserialize the keys. diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala new file mode 100644 index 0000000000000..40d568a12c25d --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala @@ -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. + */ + +package org.apache.spark.sql.kafka010 + +import java.io.Writer + +import scala.collection.mutable.HashMap +import scala.util.control.NonFatal + +import org.apache.kafka.common.TopicPartition +import org.json4s.NoTypeHints +import org.json4s.jackson.Serialization + +/** + * Utilities for converting Kafka related objects to and from json. + */ +private object JsonUtils { + private implicit val formats = Serialization.formats(NoTypeHints) + + /** + * Read TopicPartitions from json string + */ + def partitions(str: String): Array[TopicPartition] = { + try { + Serialization.read[Map[String, Seq[Int]]](str).flatMap { case (topic, parts) => + parts.map { part => + new TopicPartition(topic, part) + } + }.toArray + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":[0,1],"topicB":[0,1]}, got $str""") + } + } + + /** + * Write TopicPartitions as json string + */ + def partitions(partitions: Iterable[TopicPartition]): String = { + val result = new HashMap[String, List[Int]] + partitions.foreach { tp => + val parts: List[Int] = result.getOrElse(tp.topic, Nil) + result += tp.topic -> (tp.partition::parts) + } + Serialization.write(result) + } + + /** + * Read per-TopicPartition offsets from json string + */ + def partitionOffsets(str: String): Map[TopicPartition, Long] = { + try { + Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partOffsets) => + partOffsets.map { case (part, offset) => + new TopicPartition(topic, part) -> offset + } + }.toMap + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}, got $str""") + } + } + + /** + * Write per-TopicPartition offsets as json string + */ + def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = { + val result = new HashMap[String, HashMap[Int, Long]]() + partitionOffsets.foreach { case (tp, off) => + val parts = result.getOrElse(tp.topic, new HashMap[Int, Long]) + parts += tp.partition -> off + result += tp.topic -> parts + } + Serialization.write(result) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 4b0bb0a0f725c..537b7b0baa1b1 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -22,7 +22,7 @@ import java.{util => ju} import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer} +import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer, OffsetOutOfRangeException} import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition @@ -82,7 +82,7 @@ private[kafka010] case class KafkaSource( executorKafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, - startFromEarliestOffset: Boolean, + startingOffsets: StartingOffsets, failOnDataLoss: Boolean) extends Source with Logging { @@ -110,10 +110,10 @@ private[kafka010] case class KafkaSource( private lazy val initialPartitionOffsets = { val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) metadataLog.get(0).getOrElse { - val offsets = if (startFromEarliestOffset) { - KafkaSourceOffset(fetchEarliestOffsets()) - } else { - KafkaSourceOffset(fetchLatestOffsets()) + val offsets = startingOffsets match { + case EarliestOffsets => KafkaSourceOffset(fetchEarliestOffsets()) + case LatestOffsets => KafkaSourceOffset(fetchLatestOffsets()) + case SpecificOffsets(p) => KafkaSourceOffset(fetchSpecificStartingOffsets(p)) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") @@ -231,6 +231,43 @@ private[kafka010] case class KafkaSource( override def toString(): String = s"KafkaSource[$consumerStrategy]" + /** + * Set consumer position to specified offsets, making sure all assignments are set. + */ + private def fetchSpecificStartingOffsets( + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + val result = withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + assert(partitions.asScala == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + + partitionOffsets.foreach { + case (tp, -1) => consumer.seekToEnd(ju.Arrays.asList(tp)) + case (tp, -2) => consumer.seekToBeginning(ju.Arrays.asList(tp)) + case (tp, off) => consumer.seek(tp, off) + } + partitionOffsets.map { + case (tp, _) => tp -> consumer.position(tp) + } + } + partitionOffsets.foreach { + case (tp, off) if off != -1 && off != -2 => + if (result(tp) != off) { + reportDataLoss( + s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + result + } + /** * Fetch the earliest offsets of partitions. */ @@ -273,7 +310,7 @@ private[kafka010] case class KafkaSource( consumer.poll(0) val partitions = consumer.assignment() consumer.pause(partitions) - logDebug(s"\tPartitioned assigned to consumer: $partitions") + logDebug(s"\tPartitions assigned to consumer: $partitions") // Get the earliest offset of each partition consumer.seekToBeginning(partitions) @@ -317,6 +354,8 @@ private[kafka010] case class KafkaSource( try { result = Some(body) } catch { + case x: OffsetOutOfRangeException => + reportDataLoss(x.getMessage) case NonFatal(e) => lastException = e logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) @@ -373,6 +412,17 @@ private[kafka010] object KafkaSource { def createConsumer(): Consumer[Array[Byte], Array[Byte]] } + case class AssignStrategy(partitions: Array[TopicPartition], kafkaParams: ju.Map[String, Object]) + extends ConsumerStrategy { + override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.assign(ju.Arrays.asList(partitions: _*)) + consumer + } + + override def toString: String = s"Assign[${partitions.mkString(", ")}]" + } + case class SubscribeStrategy(topics: Seq[String], kafkaParams: ju.Map[String, Object]) extends ConsumerStrategy { override def createConsumer(): Consumer[Array[Byte], Array[Byte]] = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 23b1b60f3bcaa..585ced875caa7 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -77,14 +77,12 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // id. Hence, we should generate a unique id for each query. val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" - val startFromEarliestOffset = - caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY).map(_.trim.toLowerCase) match { - case Some("latest") => false - case Some("earliest") => true - case Some(pos) => - // This should not happen since we have already checked the options. - throw new IllegalStateException(s"Invalid $STARTING_OFFSET_OPTION_KEY: $pos") - case None => false + val startingOffsets = + caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { + case Some("latest") => LatestOffsets + case Some("earliest") => EarliestOffsets + case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) + case None => LatestOffsets } val kafkaParamsForStrategy = @@ -95,9 +93,9 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider // So that consumers in Kafka source do not mess with any existing group id .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-driver") - // Set to "latest" to avoid exceptions. However, KafkaSource will fetch the initial offsets - // by itself instead of counting on KafkaConsumer. - .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") + // Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial + // offsets by itself instead of counting on KafkaConsumer. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") // So that consumers in the driver does not commit offsets unnecessarily .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -130,6 +128,10 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider .build() val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + AssignStrategy( + JsonUtils.partitions(value), + kafkaParamsForStrategy) case ("subscribe", value) => SubscribeStrategy( value.split(",").map(_.trim()).filter(_.nonEmpty), @@ -153,7 +155,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider kafkaParamsForExecutors, parameters, metadataPath, - startFromEarliestOffset, + startingOffsets, failOnDataLoss) } @@ -175,6 +177,13 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider } val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + if (!value.trim.startsWith("{")) { + throw new IllegalArgumentException( + "No topicpartitions to assign as specified value for option " + + s"'assign' is '$value'") + } + case ("subscribe", value) => val topics = value.split(",").map(_.trim).filter(_.nonEmpty) if (topics.isEmpty) { @@ -195,14 +204,6 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider throw new IllegalArgumentException("Unknown option") } - caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { - case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) => - throw new IllegalArgumentException( - s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " + - s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}") - case _ => - } - // Validate user-specified Kafka options if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { @@ -215,11 +216,11 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider throw new IllegalArgumentException( s""" |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. - |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'earliest' or 'latest' to - |specify where to start. Structured Streaming manages which offsets are consumed + |Instead set the source option '$STARTING_OFFSETS_OPTION_KEY' to 'earliest' or 'latest' + |to specify where to start. Structured Streaming manages which offsets are consumed |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no |data is missed when when new topics/partitions are dynamically subscribed. Note that - |'$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming query is started, and + |'$STARTING_OFFSETS_OPTION_KEY' only applies when a new Streaming query is started, and |that resuming will always pick up from where the query left off. See the docs for more |details. """.stripMargin) @@ -282,8 +283,7 @@ private[kafka010] class KafkaSourceProvider extends StreamSourceProvider } private[kafka010] object KafkaSourceProvider { - private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern") - private val STARTING_OFFSET_OPTION_KEY = "startingoffset" - private val STARTING_OFFSET_OPTION_VALUES = Set("earliest", "latest") + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") + private val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala new file mode 100644 index 0000000000000..83959e597171a --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/StartingOffsets.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition + +/* + * Values that can be specified for config startingOffsets + */ +private[kafka010] sealed trait StartingOffsets + +private[kafka010] case object EarliestOffsets extends StartingOffsets + +private[kafka010] case object LatestOffsets extends StartingOffsets + +private[kafka010] case class SpecificOffsets( + partitionOffsets: Map[TopicPartition, Long]) extends StartingOffsets diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala new file mode 100644 index 0000000000000..54b980049d1a2 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala @@ -0,0 +1,45 @@ +/* + * 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.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkFunSuite + +class JsonUtilsSuite extends SparkFunSuite { + + test("parsing partitions") { + val parsed = JsonUtils.partitions("""{"topicA":[0,1],"topicB":[4,6]}""") + val expected = Array( + new TopicPartition("topicA", 0), + new TopicPartition("topicA", 1), + new TopicPartition("topicB", 4), + new TopicPartition("topicB", 6) + ) + assert(parsed.toSeq === expected.toSeq) + } + + test("parsing partitionOffsets") { + val parsed = JsonUtils.partitionOffsets( + """{"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}""") + + assert(parsed(new TopicPartition("topicA", 0)) === 23) + assert(parsed(new TopicPartition("topicA", 1)) === -1) + assert(parsed(new TopicPartition("topicB", 0)) === -2) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 8b5296ea135c7..b50688ecb7743 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.common.TopicPartition import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ @@ -52,7 +53,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { protected def makeSureGetOffsetCalled = AssertOnQuery { q => // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure // its "getOffset" is called before pushing any data. Otherwise, because of the race contion, - // we don't know which data should be fetched when `startingOffset` is latest. + // we don't know which data should be fetched when `startingOffsets` is latest. q.processAllAvailable() true } @@ -155,26 +156,52 @@ class KafkaSourceSuite extends KafkaSourceTest { ) } + test("assign from latest offsets") { + val topic = newTopic() + testFromLatestOffsets(topic, false, "assign" -> assignString(topic, 0 to 4)) + } + + test("assign from earliest offsets") { + val topic = newTopic() + testFromEarliestOffsets(topic, false, "assign" -> assignString(topic, 0 to 4)) + } + + test("assign from specific offsets") { + val topic = newTopic() + testFromSpecificOffsets(topic, "assign" -> assignString(topic, 0 to 4)) + } + test("subscribing topic by name from latest offsets") { val topic = newTopic() - testFromLatestOffsets(topic, "subscribe" -> topic) + testFromLatestOffsets(topic, true, "subscribe" -> topic) } test("subscribing topic by name from earliest offsets") { val topic = newTopic() - testFromEarliestOffsets(topic, "subscribe" -> topic) + testFromEarliestOffsets(topic, true, "subscribe" -> topic) + } + + test("subscribing topic by name from specific offsets") { + val topic = newTopic() + testFromSpecificOffsets(topic, "subscribe" -> topic) } test("subscribing topic by pattern from latest offsets") { val topicPrefix = newTopic() val topic = topicPrefix + "-suffix" - testFromLatestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + testFromLatestOffsets(topic, true, "subscribePattern" -> s"$topicPrefix-.*") } test("subscribing topic by pattern from earliest offsets") { val topicPrefix = newTopic() val topic = topicPrefix + "-suffix" - testFromEarliestOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") + testFromEarliestOffsets(topic, true, "subscribePattern" -> s"$topicPrefix-.*") + } + + test("subscribing topic by pattern from specific offsets") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromSpecificOffsets(topic, "subscribePattern" -> s"$topicPrefix-.*") } test("subscribing topic by pattern with topic deletions") { @@ -233,6 +260,10 @@ class KafkaSourceSuite extends KafkaSourceTest { testBadOptions("subscribe" -> "t", "subscribePattern" -> "t.*")( "only one", "options can be specified") + testBadOptions("subscribe" -> "t", "assign" -> """{"a":[0]}""")( + "only one", "options can be specified") + + testBadOptions("assign" -> "")("no topicpartitions to assign") testBadOptions("subscribe" -> "")("no topics to subscribe") testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") } @@ -293,7 +324,61 @@ class KafkaSourceSuite extends KafkaSourceTest { private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { + private def assignString(topic: String, partitions: Iterable[Int]): String = { + JsonUtils.partitions(partitions.map(p => new TopicPartition(topic, p))) + } + + private def testFromSpecificOffsets(topic: String, options: (String, String)*): Unit = { + val partitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 0L, + new TopicPartition(topic, 3) -> 1L, + new TopicPartition(topic, 4) -> 2L + ) + val startingOffsets = JsonUtils.partitionOffsets(partitionOffsets) + + testUtils.createTopic(topic, partitions = 5) + // part 0 starts at earliest, these should all be seen + testUtils.sendMessages(topic, Array(-20, -21, -22).map(_.toString), Some(0)) + // part 1 starts at latest, these should all be skipped + testUtils.sendMessages(topic, Array(-10, -11, -12).map(_.toString), Some(1)) + // part 2 starts at 0, these should all be seen + testUtils.sendMessages(topic, Array(0, 1, 2).map(_.toString), Some(2)) + // part 3 starts at 1, first should be skipped + testUtils.sendMessages(topic, Array(10, 11, 12).map(_.toString), Some(3)) + // part 4 starts at 2, first and second should be skipped + testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffsets", startingOffsets) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + testStream(mapped)( + makeSureGetOffsetCalled, + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), + StopStream, + StartStream(), + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), // Should get the data back on recovery + AddKafkaData(Set(topic), 30, 31, 32, 33, 34)(ensureDataInMultiplePartition = true), + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22, 30, 31, 32, 33, 34), + StopStream + ) + } + + private def testFromLatestOffsets( + topic: String, + addPartitions: Boolean, + options: (String, String)*): Unit = { testUtils.createTopic(topic, partitions = 5) testUtils.sendMessages(topic, Array("-1")) require(testUtils.getLatestOffsets(Set(topic)).size === 5) @@ -301,7 +386,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark .readStream .format("kafka") - .option("startingOffset", s"latest") + .option("startingOffsets", s"latest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } @@ -324,7 +409,9 @@ class KafkaSourceSuite extends KafkaSourceTest { AddKafkaData(Set(topic), 7, 8), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => - testUtils.addPartitions(topic, 10) + if (addPartitions) { + testUtils.addPartitions(topic, 10) + } true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), @@ -332,7 +419,10 @@ class KafkaSourceSuite extends KafkaSourceTest { ) } - private def testFromEarliestOffsets(topic: String, options: (String, String)*): Unit = { + private def testFromEarliestOffsets( + topic: String, + addPartitions: Boolean, + options: (String, String)*): Unit = { testUtils.createTopic(topic, partitions = 5) testUtils.sendMessages(topic, (1 to 3).map { _.toString }.toArray) require(testUtils.getLatestOffsets(Set(topic)).size === 5) @@ -340,7 +430,7 @@ class KafkaSourceSuite extends KafkaSourceTest { val reader = spark.readStream reader .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) - .option("startingOffset", s"earliest") + .option("startingOffsets", s"earliest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") options.foreach { case (k, v) => reader.option(k, v) } @@ -360,7 +450,9 @@ class KafkaSourceSuite extends KafkaSourceTest { StartStream(), CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), AssertOnQuery("Add partitions") { query: StreamExecution => - testUtils.addPartitions(topic, 10) + if (addPartitions) { + testUtils.addPartitions(topic, 10) + } true }, AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 3eb8a737ba4c8..9b24ccdd560e8 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -201,11 +201,23 @@ class KafkaTestUtils extends Logging { /** Send the array of messages to the Kafka broker */ def sendMessages(topic: String, messages: Array[String]): Seq[(String, RecordMetadata)] = { + sendMessages(topic, messages, None) + } + + /** Send the array of messages to the Kafka broker using specified partition */ + def sendMessages( + topic: String, + messages: Array[String], + partition: Option[Int]): Seq[(String, RecordMetadata)] = { producer = new KafkaProducer[String, String](producerConfiguration) val offsets = try { messages.map { m => + val record = partition match { + case Some(p) => new ProducerRecord[String, String](topic, p, null, m) + case None => new ProducerRecord[String, String](topic, m) + } val metadata = - producer.send(new ProducerRecord[String, String](topic, m)).get(10, TimeUnit.SECONDS) + producer.send(record).get(10, TimeUnit.SECONDS) logInfo(s"\tSent $m to partition ${metadata.partition}, offset ${metadata.offset}") (m, metadata) } From d3c78c4f379a6ce3d055f935a6018d866606ebaf Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 21 Oct 2016 16:27:19 -0700 Subject: [PATCH 1442/1470] [STREAMING][KAFKA][DOC] clarify kafka settings needed for larger batches ## What changes were proposed in this pull request? Minor doc change to mention kafka configuration for larger spark batches. ## How was this patch tested? Doc change only, confirmed via jekyll. The configuration issue was discussed / confirmed with users on the mailing list. Author: cody koeninger Closes #15570 from koeninger/kafka-doc-heartbeat. (cherry picked from commit c9720b2195a465653690b3e221ce789142217b0d) Signed-off-by: Shixiong Zhu --- docs/streaming-kafka-0-10-integration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index 456b8453383db..de95ea90137eb 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -48,6 +48,7 @@ Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javad For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). +If your Spark batch duration is larger than the default Kafka heartbeat session timeout (30 seconds), increase heartbeat.interval.ms and session.timeout.ms appropriately. For batches larger than 5 minutes, this will require changing group.max.session.timeout.ms on the broker. Note that the example sets enable.auto.commit to false, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. ### LocationStrategies From a0c03c92545c147015308cce195dfc2e8a3074fb Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 21 Oct 2016 22:20:52 -0700 Subject: [PATCH 1443/1470] [SPARK-16606][MINOR] Tiny follow-up to , to correct more instances of the same log message typo ## What changes were proposed in this pull request? Tiny follow-up to SPARK-16606 / https://github.com/apache/spark/pull/14533 , to correct more instances of the same log message typo ## How was this patch tested? Existing tests (no functional change anyway) Author: Sean Owen Closes #15586 from srowen/SPARK-16606.2. (cherry picked from commit 7178c56433cd138dae53db9194c55e3f4fa0fa69) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index a7de1151e8a2d..13d3e75e3d393 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -802,7 +802,7 @@ object SparkSession { if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } if (options.nonEmpty) { - logWarning("Use an existing SparkSession, some configuration may not take effect.") + logWarning("Using an existing SparkSession; some configuration may not take effect.") } return session } @@ -814,7 +814,7 @@ object SparkSession { if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.conf.set(k, v) } if (options.nonEmpty) { - logWarning("Use an existing SparkSession, some configuration may not take effect.") + logWarning("Using an existing SparkSession; some configuration may not take effect.") } return session } From b959dab32a455e0f9a9ea0fd2111e28a5faf796c Mon Sep 17 00:00:00 2001 From: Drew Robb Date: Sat, 22 Oct 2016 01:59:36 -0700 Subject: [PATCH 1444/1470] [SPARK-17986][ML] SQLTransformer should remove temporary tables ## What changes were proposed in this pull request? A call to the method `SQLTransformer.transform` previously would create a temporary table and never delete it. This change adds a call to `dropTempView()` that deletes this temporary table before returning the result so that the table will not remain in spark's table catalog. Because `tableName` is randomized and not exposed, there should be no expected use of this table outside of the `transform` method. ## How was this patch tested? A single new assertion was added to the existing test of the `SQLTransformer.transform` method that all temporary tables are removed. Without the corresponding code change, this new assertion fails. I am not aware of any circumstances in which removing this temporary view would be bad for performance or correctness in other ways, but some expertise here would be helpful. Author: Drew Robb Closes #15526 from drewrobb/SPARK-17986. (cherry picked from commit ab3363e9f6b1f7fc26682509fe7382c570f91778) Signed-off-by: Yanbo Liang --- .../scala/org/apache/spark/ml/feature/SQLTransformer.scala | 4 +++- .../org/apache/spark/ml/feature/SQLTransformerSuite.scala | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 259be2679ce19..b25fff973c441 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -67,7 +67,9 @@ class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) val tableName = Identifiable.randomUID(uid) dataset.createOrReplaceTempView(tableName) val realStatement = $(statement).replace(tableIdentifier, tableName) - dataset.sparkSession.sql(realStatement) + val result = dataset.sparkSession.sql(realStatement) + dataset.sparkSession.catalog.dropTempView(tableName) + result } @Since("1.6.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala index 1401ea9c4b431..9d3c00707039c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala @@ -43,6 +43,7 @@ class SQLTransformerSuite assert(result.schema.toString == resultSchema.toString) assert(resultSchema == expected.schema) assert(result.collect().toSeq == expected.collect().toSeq) + assert(original.sparkSession.catalog.listTables().count() == 0) } test("read/write") { From 3d587875102fc2f10f03956ef50457203cb4a840 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Sat, 22 Oct 2016 16:32:49 -0700 Subject: [PATCH 1445/1470] [SPARK-17698][SQL] Join predicates should not contain filter clauses ## What changes were proposed in this pull request? This is a backport of https://github.com/apache/spark/pull/15272 to 2.0 branch. Jira : https://issues.apache.org/jira/browse/SPARK-17698 `ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below: [0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91 eg. ``` val df = (1 until 10).toDF("id").coalesce(1) hc.sql("DROP TABLE IF EXISTS table1").collect df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1") hc.sql("DROP TABLE IF EXISTS table2").collect df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2") sqlContext.sql(""" SELECT a.id, b.id FROM table1 a FULL OUTER JOIN table2 b ON a.id = b.id AND a.id='1' AND b.id='1' """).explain(true) ``` BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job. ``` SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter :- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200) : +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200) +- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct ``` AFTER : ``` SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0)) :- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct +- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct ``` ## How was this patch tested? - Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses` - Ran all the tests in `BucketedReadSuite` Author: Tejas Patil Closes #15600 from tejasapatil/SPARK-17698_2.0_backport. --- .../sql/catalyst/expressions/predicates.scala | 5 +- .../spark/sql/catalyst/optimizer/joins.scala | 4 +- .../sql/catalyst/planning/patterns.scala | 2 + .../spark/sql/sources/BucketedReadSuite.scala | 82 ++++++++++++++++--- 4 files changed, 79 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 100087ed58918..abe0f08e135d1 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 @@ -84,8 +84,9 @@ trait PredicateHelper { * * For example consider a join between two relations R(a, b) and S(c, d). * - * `canEvaluate(EqualTo(a,b), R)` returns `true` where as `canEvaluate(EqualTo(a,c), R)` returns - * `false`. + * - `canEvaluate(EqualTo(a,b), R)` returns `true` + * - `canEvaluate(EqualTo(a,c), R)` returns `false` + * - `canEvaluate(Literal(1), R)` returns `true` as literals CAN be evaluated on any plan */ protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = expr.references.subsetOf(plan.outputSet) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index ae4cd8e8709ab..08062bdba6822 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -59,7 +59,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { // find out the first join that have at least one join condition val conditionalJoin = rest.find { plan => val refs = left.outputSet ++ plan.outputSet - conditions.filterNot(canEvaluate(_, left)).filterNot(canEvaluate(_, plan)) + conditions + .filterNot(l => l.references.nonEmpty && canEvaluate(l, left)) + .filterNot(r => r.references.nonEmpty && canEvaluate(r, plan)) .exists(_.references.subsetOf(refs)) } // pick the next one if no condition left diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index f42e67ca6ec20..d952c9eeabaeb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -112,6 +112,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { // as join keys. val predicates = condition.map(splitConjunctivePredicates).getOrElse(Nil) val joinKeys = predicates.flatMap { + case EqualTo(l, r) if l.references.isEmpty || r.references.isEmpty => None case EqualTo(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => Some((l, r)) case EqualTo(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => Some((r, l)) // Replace null with default value for joining key, then those rows with null in it could @@ -125,6 +126,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { case other => None } val otherPredicates = predicates.filterNot { + case EqualTo(l, r) if l.references.isEmpty || r.references.isEmpty => false case EqualTo(l, r) => canEvaluate(l, left) && canEvaluate(r, right) || canEvaluate(l, right) && canEvaluate(r, left) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index fc01ff3f5aa07..3554b0e101d64 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -234,7 +234,8 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet private def testBucketing( bucketSpecLeft: Option[BucketSpec], bucketSpecRight: Option[BucketSpec], - joinColumns: Seq[String], + joinType: String = "inner", + joinCondition: (DataFrame, DataFrame) => Column, shuffleLeft: Boolean, shuffleRight: Boolean): Unit = { withTable("bucketed_table1", "bucketed_table2") { @@ -256,12 +257,12 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { val t1 = spark.table("bucketed_table1") val t2 = spark.table("bucketed_table2") - val joined = t1.join(t2, joinCondition(t1, t2, joinColumns)) + val joined = t1.join(t2, joinCondition(t1, t2), joinType) // First check the result is corrected. checkAnswer( joined.sort("bucketed_table1.k", "bucketed_table2.k"), - df1.join(df2, joinCondition(df1, df2, joinColumns)).sort("df1.k", "df2.k")) + df1.join(df2, joinCondition(df1, df2), joinType).sort("df1.k", "df2.k")) assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoinExec]) val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoinExec] @@ -276,47 +277,89 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet } } - private def joinCondition(left: DataFrame, right: DataFrame, joinCols: Seq[String]): Column = { + private def joinCondition(joinCols: Seq[String]) (left: DataFrame, right: DataFrame): Column = { joinCols.map(col => left(col) === right(col)).reduce(_ && _) } test("avoid shuffle when join 2 bucketed tables") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) - testBucketing(bucketSpec, bucketSpec, Seq("i", "j"), shuffleLeft = false, shuffleRight = false) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false + ) } // Enable it after fix https://issues.apache.org/jira/browse/SPARK-12704 ignore("avoid shuffle when join keys are a super-set of bucket keys") { val bucketSpec = Some(BucketSpec(8, Seq("i"), Nil)) - testBucketing(bucketSpec, bucketSpec, Seq("i", "j"), shuffleLeft = false, shuffleRight = false) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false + ) } test("only shuffle one side when join bucketed table and non-bucketed table") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) - testBucketing(bucketSpec, None, Seq("i", "j"), shuffleLeft = false, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = None, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = true + ) } test("only shuffle one side when 2 bucketed tables have different bucket number") { val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Nil)) val bucketSpec2 = Some(BucketSpec(5, Seq("i", "j"), Nil)) - testBucketing(bucketSpec1, bucketSpec2, Seq("i", "j"), shuffleLeft = false, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = true + ) } test("only shuffle one side when 2 bucketed tables have different bucket keys") { val bucketSpec1 = Some(BucketSpec(8, Seq("i"), Nil)) val bucketSpec2 = Some(BucketSpec(8, Seq("j"), Nil)) - testBucketing(bucketSpec1, bucketSpec2, Seq("i"), shuffleLeft = false, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i")), + shuffleLeft = false, + shuffleRight = true + ) } test("shuffle when join keys are not equal to bucket keys") { val bucketSpec = Some(BucketSpec(8, Seq("i"), Nil)) - testBucketing(bucketSpec, bucketSpec, Seq("j"), shuffleLeft = true, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("j")), + shuffleLeft = true, + shuffleRight = true + ) } test("shuffle when join 2 bucketed tables with bucketing disabled") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) withSQLConf(SQLConf.BUCKETING_ENABLED.key -> "false") { - testBucketing(bucketSpec, bucketSpec, Seq("i", "j"), shuffleLeft = true, shuffleRight = true) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = true, + shuffleRight = true + ) } } @@ -348,6 +391,23 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet } } + test("SPARK-17698 Join predicates should not contain filter clauses") { + val bucketSpec = Some(BucketSpec(8, Seq("i"), Seq("i"))) + testBucketing( + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinType = "fullouter", + joinCondition = (left: DataFrame, right: DataFrame) => { + val joinPredicates = left("i") === right("i") + val filterLeft = left("i") === Literal("1") + val filterRight = right("i") === Literal("1") + joinPredicates && filterLeft && filterRight + }, + shuffleLeft = false, + shuffleRight = false + ) + } + test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") From e21e9d4162cc798d9ec43ef984d17b89dab77826 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 23 Oct 2016 14:00:35 +0200 Subject: [PATCH 1446/1470] [SPARK-17123][SQL][BRANCH-2.0] Use type-widened encoder for DataFrame for set operations ## What changes were proposed in this pull request? This PR backports https://github.com/apache/spark/pull/15072 Please note that the test code is a bit different with the master as https://github.com/apache/spark/pull/14786 was only merged into master and therefore, it does not support type-widening between `DateType` and `TimestampType`. So, both types were taken out from the test. ## How was this patch tested? Unit test in `DataFrameSuite`. Author: hyukjinkwon Closes #15601 from HyukjinKwon/backport-17123. --- .../scala/org/apache/spark/sql/Dataset.scala | 16 +++++++++++++--- .../org/apache/spark/sql/DataFrameSuite.scala | 16 ++++++++++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 0b236a0c7466f..4946bbe634d98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1456,7 +1456,7 @@ class Dataset[T] private[sql]( * @group typedrel * @since 2.0.0 */ - def union(other: Dataset[T]): Dataset[T] = withTypedPlan { + def union(other: Dataset[T]): Dataset[T] = withSetOperator { // This breaks caching, but it's usually ok because it addresses a very specific use case: // using union to union many files or partitions. CombineUnions(Union(logicalPlan, other.logicalPlan)) @@ -1472,7 +1472,7 @@ class Dataset[T] private[sql]( * @group typedrel * @since 1.6.0 */ - def intersect(other: Dataset[T]): Dataset[T] = withTypedPlan { + def intersect(other: Dataset[T]): Dataset[T] = withSetOperator { Intersect(logicalPlan, other.logicalPlan) } @@ -1486,7 +1486,7 @@ class Dataset[T] private[sql]( * @group typedrel * @since 2.0.0 */ - def except(other: Dataset[T]): Dataset[T] = withTypedPlan { + def except(other: Dataset[T]): Dataset[T] = withSetOperator { Except(logicalPlan, other.logicalPlan) } @@ -2607,4 +2607,14 @@ class Dataset[T] private[sql]( @inline private def withTypedPlan[U : Encoder](logicalPlan: => LogicalPlan): Dataset[U] = { Dataset(sparkSession, logicalPlan) } + + /** A convenient function to wrap a set based logical plan and produce a Dataset. */ + @inline private def withSetOperator[U : Encoder](logicalPlan: => LogicalPlan): Dataset[U] = { + if (classTag.runtimeClass.isAssignableFrom(classOf[Row])) { + // Set operators widen types (change the schema), so we cannot reuse the row encoder. + Dataset.ofRows(sparkSession, logicalPlan).asInstanceOf[Dataset[U]] + } else { + Dataset(sparkSession, logicalPlan) + } + } } 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 7ab0fe07b9c4a..f8d7ddde85d92 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql import java.io.File import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} import java.util.UUID import scala.language.postfixOps @@ -1585,4 +1586,19 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-17123: Performing set operations that combine non-scala native types") { + val dates = Seq( + (BigDecimal.valueOf(1), new Timestamp(2)), + (BigDecimal.valueOf(4), new Timestamp(5)) + ).toDF("decimal", "timestamp") + + val widenTypedRows = Seq( + (10.5D, "string") + ).toDF("decimal", "timestamp") + + dates.union(widenTypedRows).collect() + dates.except(widenTypedRows).collect() + dates.intersect(widenTypedRows).collect() + } } From 0e0d83a597885ab1773cb69d6dcc10346d6976a3 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sun, 23 Oct 2016 10:53:27 -0700 Subject: [PATCH 1447/1470] [SPARKR][BRANCH-2.0] R merge API doc and example fix ## What changes were proposed in this pull request? Fixes for R doc ## How was this patch tested? N/A Author: Felix Cheung Closes #15589 from felixcheung/rdocmergefix. --- R/pkg/R/DataFrame.R | 7 +++++-- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index a5bd603376010..d5c5486ce2dc9 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -345,7 +345,7 @@ setMethod("colnames<-", # Check if the column names have . in it if (any(regexec(".", value, fixed = TRUE)[[1]][1] != -1)) { - stop("Colum names cannot contain the '.' symbol.") + stop("Column names cannot contain the '.' symbol.") } sdf <- callJMethod(x@sdf, "toDF", as.list(value)) @@ -2297,6 +2297,8 @@ setMethod("join", #' @param y the second data frame to be joined #' @param by a character vector specifying the join columns. If by is not #' specified, the common column names in \code{x} and \code{y} will be used. +#' If by or both by.x and by.y are explicitly set to NULL or of length 0, the Cartesian +#' Product of x and y will be returned. #' @param by.x a character vector specifying the joining columns for x. #' @param by.y a character vector specifying the joining columns for y. #' @param all a boolean value setting \code{all.x} and \code{all.y} @@ -2326,13 +2328,14 @@ setMethod("join", #' sparkR.session() #' df1 <- read.json(path) #' df2 <- read.json(path2) -#' merge(df1, df2) # Performs a Cartesian +#' merge(df1, df2) # Performs an inner join by common columns #' merge(df1, df2, by = "col1") # Performs an inner join based on expression #' merge(df1, df2, by.x = "col1", by.y = "col2", all.y = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE, all.y = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all = TRUE, sort = FALSE) #' merge(df1, df2, by = "col1", all = TRUE, suffixes = c("-X", "-Y")) +#' merge(df1, df2, by = NULL) # Performs a Cartesian join #' } #' @note merge since 1.5.0 setMethod("merge", diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index b7b9de7e3e649..ef6cab1e1c92d 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -815,7 +815,7 @@ test_that("names() colnames() set the column names", { expect_equal(names(df)[1], "col3") expect_error(colnames(df) <- c("sepal.length", "sepal_width"), - "Colum names cannot contain the '.' symbol.") + "Column names cannot contain the '.' symbol.") expect_error(colnames(df) <- c(1, 2), "Invalid column names.") expect_error(colnames(df) <- c("a"), "Column names must have the same length as the number of columns in the dataset.") From 00a2e01e8d5afde21cc01476adf0215a316ae936 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sun, 23 Oct 2016 22:20:18 +0200 Subject: [PATCH 1448/1470] [SPARK-18058][SQL] [BRANCH-2.0]Comparing column types ignoring Nullability in Union and SetOperation ## What changes were proposed in this pull request? The PR tries to fix [SPARK-18058](https://issues.apache.org/jira/browse/SPARK-18058) which refers to a bug that the column types are compared with the extra care about Nullability in Union and SetOperation. This PR converts the columns types by setting all fields as nullable before comparison ## How was this patch tested? regular unit test cases Author: CodingCat Closes #15602 from CodingCat/branch-2.0. --- .../plans/logical/basicLogicalOperators.scala | 29 +++++++------------ .../sql/catalyst/analysis/AnalysisSuite.scala | 14 +++++++++ 2 files changed, 24 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 07e39b029894a..2fdaa057bab18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -116,6 +116,8 @@ case class Filter(condition: Expression, child: LogicalPlan) abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty + protected def leftConstraints: Set[Expression] = left.constraints protected def rightConstraints: Set[Expression] = { @@ -125,6 +127,13 @@ abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends Binar case a: Attribute => attributeRewrites(a) }) } + + override lazy val resolved: Boolean = + childrenResolved && + left.output.length == right.output.length && + left.output.zip(right.output).forall { + case (l, r) => l.dataType.asNullable == r.dataType.asNullable } && + duplicateResolved } object SetOperation { @@ -133,8 +142,6 @@ object SetOperation { case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) { - def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty - override def output: Seq[Attribute] = left.output.zip(right.output).map { case (leftAttr, rightAttr) => leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) @@ -143,14 +150,6 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation override protected def validConstraints: Set[Expression] = leftConstraints.union(rightConstraints) - // Intersect are only resolved if they don't introduce ambiguous expression ids, - // since the Optimizer will convert Intersect to Join. - override lazy val resolved: Boolean = - childrenResolved && - left.output.length == right.output.length && - left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } && - duplicateResolved - override def maxRows: Option[Long] = { if (children.exists(_.maxRows.isEmpty)) { None @@ -171,19 +170,11 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) { - def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty - /** We don't use right.output because those rows get excluded from the set. */ override def output: Seq[Attribute] = left.output override protected def validConstraints: Set[Expression] = leftConstraints - override lazy val resolved: Boolean = - childrenResolved && - left.output.length == right.output.length && - left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } && - duplicateResolved - override lazy val statistics: Statistics = { left.statistics.copy() } @@ -218,7 +209,7 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { child.output.length == children.head.output.length && // compare the data types with the first child child.output.zip(children.head.output).forall { - case (l, r) => l.dataType == r.dataType } + case (l, r) => l.dataType.asNullable == r.dataType.asNullable } ) children.length > 1 && childrenResolved && allChildrenCompatible 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 102c78bd72111..1ba4f4a36d1a8 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 @@ -377,4 +377,18 @@ class AnalysisSuite extends AnalysisTest { assertExpressionType(sum(Divide(Decimal(1), 2.0)), DoubleType) assertExpressionType(sum(Divide(1.0, Decimal(2.0))), DoubleType) } + + test("SPARK-18058: union and set operations shall not care about the nullability" + + " when comparing column types") { + val firstTable = LocalRelation( + AttributeReference("a", + StructType(Seq(StructField("a", IntegerType, nullable = true))), nullable = false)()) + val secondTable = LocalRelation( + AttributeReference("a", + StructType(Seq(StructField("a", IntegerType, nullable = false))), nullable = false)()) + + assertAnalysisSuccess(Union(firstTable, secondTable)) + assertAnalysisSuccess(Except(firstTable, secondTable)) + assertAnalysisSuccess(Intersect(firstTable, secondTable)) + } } From 064db176d2f7e69fd280b14c5ef13a905180051e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 24 Oct 2016 10:44:45 +0100 Subject: [PATCH 1449/1470] [SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but can resolve as HDFS path Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir Existing tests. Author: Sean Owen Closes #15382 from srowen/SPARK-17810. (cherry picked from commit 4ecbe1b92f4c4c5b2d734895c09d8ded0ed48d4d) Signed-off-by: Sean Owen --- docs/sql-programming-guide.md | 33 +++---------------- .../sql/hive/JavaSparkHiveExample.java | 2 +- examples/src/main/python/sql/hive.py | 2 +- .../examples/sql/hive/SparkHiveExample.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 3 +- .../sql/execution/command/DDLSuite.scala | 20 +++++------ .../spark/sql/internal/SQLConfSuite.scala | 5 +-- 7 files changed, 22 insertions(+), 45 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3a90323760909..bac0e8156913b 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -857,50 +857,27 @@ access data stored in Hive. Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. -
    - -
    - When working with Hive, one must instantiate `SparkSession` with Hive support, including connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. Users who do not have an existing Hive deployment can still enable Hive support. When not configured by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that +`spark-warehouse` in the current directory that the Spark application is started. Note that the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. +You may need to grant write privilege to the user who starts the Spark application. +
    + +
    {% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %}
    - -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. - {% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %}
    - -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. - {% include_example spark_hive python/sql/hive.py %}
    diff --git a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java index 76dd160d5568b..052153c9e9736 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java @@ -56,7 +56,7 @@ public void setValue(String value) { public static void main(String[] args) { // $example on:spark_hive$ // warehouseLocation points to the default location for managed databases and tables - String warehouseLocation = "file:" + System.getProperty("user.dir") + "spark-warehouse"; + String warehouseLocation = "spark-warehouse"; SparkSession spark = SparkSession .builder() .appName("Java Spark Hive Example") diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index 9b2a2c4e6a16b..142213d3cfb1f 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -34,7 +34,7 @@ if __name__ == "__main__": # $example on:spark_hive$ # warehouse_location points to the default location for managed databases and tables - warehouse_location = 'file:${system:user.dir}/spark-warehouse' + warehouse_location = 'spark-warehouse' spark = SparkSession \ .builder \ diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala index 11e84c0e45632..ded18dacf1fe3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala @@ -38,7 +38,7 @@ object SparkHiveExample { // $example on:spark_hive$ // warehouseLocation points to the default location for managed databases and tables - val warehouseLocation = "file:${system:user.dir}/spark-warehouse" + val warehouseLocation = "spark-warehouse" val spark = SparkSession .builder() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8ba87c82329ae..7598d475042ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.catalyst.CatalystConf +import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the configuration options for Spark SQL. @@ -56,7 +57,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("${system:user.dir}/spark-warehouse") + .createWithDefault(Utils.resolveURI("spark-warehouse").toString) val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 1f5492e8a0b9a..caa2fcac9723e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -115,7 +115,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath // The generated temp path is not qualified. assert(!path.startsWith("file:/")) val uri = tmpDir.toURI @@ -147,7 +147,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Create/Drop Database") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -158,7 +158,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -183,9 +183,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { try { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) - val expectedLocation = - makeQualifiedPath(s"${System.getProperty("user.dir")}/spark-warehouse" + - "/" + s"$dbName.db") + val expectedLocation = makeQualifiedPath(s"spark-warehouse/$dbName.db") assert(db1 == CatalogDatabase( dbName, "", @@ -203,7 +201,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => - val path = new Path(tmpDir.toString).toUri.toString + val path = new Path(tmpDir.getCanonicalPath).toUri databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) @@ -226,7 +224,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Create Database - database already exists") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -236,7 +234,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -269,7 +267,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Alter/Describe Database") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -277,7 +275,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val location = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") sql(s"CREATE DATABASE $dbName") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 761bbe3576c71..a230344f73e7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.util.Utils class SQLConfSuite extends QueryTest with SharedSQLContext { private val testKey = "test.key.0" @@ -215,8 +216,8 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { try { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) - assert(spark.sessionState.conf.warehousePath - === new Path(s"${System.getProperty("user.dir")}/spark-warehouse").toString) + assert(new Path(Utils.resolveURI("spark-warehouse")).toString === + spark.sessionState.conf.warehousePath + "/") } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") } From aef65ac027bf4316145284f4c77a2540bc82bd8f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 26 Sep 2016 13:07:11 -0700 Subject: [PATCH 1450/1470] [SPARK-17153][SQL] Should read partition data when reading new files in filestream without globbing ## What changes were proposed in this pull request? When reading file stream with non-globbing path, the results return data with all `null`s for the partitioned columns. E.g., case class A(id: Int, value: Int) val data = spark.createDataset(Seq( A(1, 1), A(2, 2), A(2, 3)) ) val url = "/tmp/test" data.write.partitionBy("id").parquet(url) spark.read.parquet(url).show +-----+---+ |value| id| +-----+---+ | 2| 2| | 3| 2| | 1| 1| +-----+---+ val s = spark.readStream.schema(spark.read.load(url).schema).parquet(url) s.writeStream.queryName("test").format("memory").start() sql("SELECT * FROM test").show +-----+----+ |value| id| +-----+----+ | 2|null| | 3|null| | 1|null| +-----+----+ ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh Author: Liang-Chi Hsieh Closes #14803 from viirya/filestreamsource-option. --- .../structured-streaming-programming-guide.md | 6 ++ .../execution/datasources/DataSource.scala | 7 +- .../streaming/FileStreamSource.scala | 9 +- .../sql/streaming/FileStreamSourceSuite.scala | 83 ++++++++++++++++++- .../spark/sql/streaming/StreamTest.scala | 8 ++ 5 files changed, 110 insertions(+), 3 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 4f14f3a9450bc..be730b8c1c2ee 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -517,6 +517,12 @@ csvDF = spark \ These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the [SQL Programming Guide](sql-programming-guide.html) for more details. Additionally, more details on the supported streaming sources are discussed later in the document. +### Schema inference and partition of streaming DataFrames/Datasets + +By default, Structured Streaming from file based sources requires you to specify the schema, rather than rely on Spark to infer it automatically. This restriction ensures a consistent schema will be used for the streaming query, even in the case of failures. For ad-hoc use cases, you can reenable schema inference by setting `spark.sql.streaming.schemaInference` to `true`. + +Partition discovery does occur when subdirectories that are named `/key=value/` are present and listing will automatically recurse into these directories. If these columns appear in the user provided schema, they will be filled in by Spark based on the path of the file being read. The directories that make up the partitioning scheme must be present when the query starts and must remain static. For example, it is okay to add `/data/year=2016/` when `/data/year=2015/` was present, but it is invalid to change the partitioning column (i.e. by creating the directory `/data/date=2016-04-17/`). + ## Operations on streaming DataFrames/Datasets You can apply all kinds of operations on streaming DataFrames/Datasets – ranging from untyped, SQL-like operations (e.g. `select`, `where`, `groupBy`), to typed RDD-like operations (e.g. `map`, `filter`, `flatMap`). See the [SQL programming guide](sql-programming-guide.html) for more details. Let’s take a look at a few example operations that you can use. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index ee37390c91ddb..251f54b7c69e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -196,10 +196,15 @@ case class DataSource( SparkHadoopUtil.get.globPathIfNecessary(qualified) }.toArray val fileCatalog = new ListingFileCatalog(sparkSession, globbedPaths, options, None) - format.inferSchema( + val partitionCols = fileCatalog.partitionSpec().partitionColumns.fields + val inferred = format.inferSchema( sparkSession, caseInsensitiveOptions, fileCatalog.allFiles()) + + inferred.map { inferredSchema => + StructType(inferredSchema ++ partitionCols) + } }.getOrElse { throw new AnalysisException("Unable to infer schema. It must be specified manually.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 8c3e7184a65b3..ae3e709422812 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -47,6 +47,13 @@ class FileStreamSource( fs.makeQualified(new Path(path)) // can contains glob patterns } + private val optionsWithPartitionBasePath = sourceOptions.optionMapWithoutPath ++ { + if (!SparkHadoopUtil.get.isGlobPath(new Path(path)) && options.contains("path")) { + Map("basePath" -> path) + } else { + Map() + }} + private val metadataLog = new FileStreamSourceLog(FileStreamSourceLog.VERSION, sparkSession, metadataPath) private var maxBatchId = metadataLog.getLatest().map(_._1).getOrElse(-1L) @@ -136,7 +143,7 @@ class FileStreamSource( paths = files.map(_.path), userSpecifiedSchema = Some(schema), className = fileFormatClassName, - options = sourceOptions.optionMapWithoutPath) + options = optionsWithPartitionBasePath) Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation( checkPathExist = false))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index c18d843b9685c..ca433d5b0701b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -102,6 +102,12 @@ class FileStreamSourceTest extends StreamTest with SharedSQLContext with Private } } + case class DeleteFile(file: File) extends ExternalAction { + def runAction(): Unit = { + Utils.deleteRecursively(file) + } + } + /** Use `format` and `path` to create FileStreamSource via DataFrameReader */ def createFileStream( format: String, @@ -608,6 +614,81 @@ class FileStreamSourceSuite extends FileStreamSourceTest { // =============== other tests ================ + test("read new files in partitioned table without globbing, should read partition data") { + withTempDirs { case (dir, tmp) => + val partitionFooSubDir = new File(dir, "partition=foo") + val partitionBarSubDir = new File(dir, "partition=bar") + + val schema = new StructType().add("value", StringType).add("partition", StringType) + val fileStream = createFileStream("json", s"${dir.getCanonicalPath}", Some(schema)) + val filtered = fileStream.filter($"value" contains "keep") + testStream(filtered)( + // Create new partition=foo sub dir and write to it + AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo")), + + // Append to same partition=foo sub dir + AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo")), + + // Create new partition sub dir and write to it + AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar")), + + // Append to same partition=bar sub dir + AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar")) + ) + } + } + + test("when schema inference is turned on, should read partition data") { + def createFile(content: String, src: File, tmp: File): Unit = { + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + src.mkdirs() + require(stringToFile(tempFile, content).renameTo(finalFile)) + } + + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + withTempDirs { case (dir, tmp) => + val partitionFooSubDir = new File(dir, "partition=foo") + val partitionBarSubDir = new File(dir, "partition=bar") + + // Create file in partition, so we can infer the schema. + createFile("{'value': 'drop0'}", partitionFooSubDir, tmp) + + val fileStream = createFileStream("json", s"${dir.getCanonicalPath}") + val filtered = fileStream.filter($"value" contains "keep") + testStream(filtered)( + // Append to same partition=foo sub dir + AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo")), + + // Append to same partition=foo sub dir + AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo")), + + // Create new partition sub dir and write to it + AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar")), + + // Append to same partition=bar sub dir + AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar")), + + // Delete the two partition dirs + DeleteFile(partitionFooSubDir), + DeleteFile(partitionBarSubDir), + + AddTextFileData("{'value': 'keep6'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar"), + ("keep6", "bar")) + ) + } + } + } + test("fault tolerance") { withTempDirs { case (src, tmp) => val fileStream = createFileStream("text", src.getCanonicalPath) @@ -792,7 +873,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } assert(src.listFiles().size === numFiles) - val files = spark.readStream.text(root.getCanonicalPath).as[String] + val files = spark.readStream.text(root.getCanonicalPath).as[(String, Int)] // Note this query will use constant folding to eliminate the file scan. // This is to avoid actually running a Spark job with 10000 tasks diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index ec49e1ab8d882..742833065144d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -98,6 +98,11 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { def addData(query: Option[StreamExecution]): (Source, Offset) } + /** A trait that can be extended when testing a source. */ + trait ExternalAction extends StreamAction { + def runAction(): Unit + } + case class AddDataMemory[A](source: MemoryStream[A], data: Seq[A]) extends AddData { override def toString: String = s"AddData to $source: ${data.mkString(",")}" @@ -473,6 +478,9 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { failTest("Error adding data", e) } + case e: ExternalAction => + e.runAction() + case CheckAnswerRows(expectedAnswer, lastOnly, isSorted) => verify(currentStream != null, "stream not running") // Get the map of source index to the current source objects From bad15bcdf3d3907e4c2412c7f36344f2c6a19587 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 21 Oct 2016 15:28:16 -0700 Subject: [PATCH 1451/1470] [SPARK-18044][STREAMING] FileStreamSource should not infer partitions in every batch ## What changes were proposed in this pull request? In `FileStreamSource.getBatch`, we will create a `DataSource` with specified schema, to avoid inferring the schema again and again. However, we don't pass the partition columns, and will infer the partition again and again. This PR fixes it by keeping the partition columns in `FileStreamSource`, like schema. ## How was this patch tested? N/A Author: Wenchen Fan Closes #15581 from cloud-fan/stream. --- .../execution/datasources/DataSource.scala | 26 +++++++++++++------ .../streaming/FileStreamSource.scala | 2 ++ .../streaming/FileStreamSourceSuite.scala | 2 +- 3 files changed, 21 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 251f54b7c69e3..9d2fee8b75976 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -74,7 +74,7 @@ case class DataSource( bucketSpec: Option[BucketSpec] = None, options: Map[String, String] = Map.empty) extends Logging { - case class SourceInfo(name: String, schema: StructType) + case class SourceInfo(name: String, schema: StructType, partitionColumns: Seq[String]) lazy val providingClass: Class[_] = lookupDataSource(className) lazy val sourceInfo = sourceSchema() @@ -185,8 +185,11 @@ case class DataSource( } } - private def inferFileFormatSchema(format: FileFormat): StructType = { - userSpecifiedSchema.orElse { + /** + * Infer the schema of the given FileFormat, returns a pair of schema and partition column names. + */ + private def inferFileFormatSchema(format: FileFormat): (StructType, Seq[String]) = { + userSpecifiedSchema.map(_ -> partitionColumns).orElse { val caseInsensitiveOptions = new CaseInsensitiveMap(options) val allPaths = caseInsensitiveOptions.get("path") val globbedPaths = allPaths.toSeq.flatMap { path => @@ -196,14 +199,14 @@ case class DataSource( SparkHadoopUtil.get.globPathIfNecessary(qualified) }.toArray val fileCatalog = new ListingFileCatalog(sparkSession, globbedPaths, options, None) - val partitionCols = fileCatalog.partitionSpec().partitionColumns.fields + val partitionSchema = fileCatalog.partitionSpec().partitionColumns val inferred = format.inferSchema( sparkSession, caseInsensitiveOptions, fileCatalog.allFiles()) inferred.map { inferredSchema => - StructType(inferredSchema ++ partitionCols) + StructType(inferredSchema ++ partitionSchema) -> partitionSchema.map(_.name) } }.getOrElse { throw new AnalysisException("Unable to infer schema. It must be specified manually.") @@ -216,7 +219,7 @@ case class DataSource( case s: StreamSourceProvider => val (name, schema) = s.sourceSchema( sparkSession.sqlContext, userSpecifiedSchema, className, options) - SourceInfo(name, schema) + SourceInfo(name, schema, Nil) case format: FileFormat => val caseInsensitiveOptions = new CaseInsensitiveMap(options) @@ -245,7 +248,8 @@ case class DataSource( "you may be able to create a static DataFrame on that directory with " + "'spark.read.load(directory)' and infer schema from it.") } - SourceInfo(s"FileSource[$path]", inferFileFormatSchema(format)) + val (schema, partCols) = inferFileFormatSchema(format) + SourceInfo(s"FileSource[$path]", schema, partCols) case _ => throw new UnsupportedOperationException( @@ -265,7 +269,13 @@ case class DataSource( throw new IllegalArgumentException("'path' is not specified") }) new FileStreamSource( - sparkSession, path, className, sourceInfo.schema, metadataPath, options) + sparkSession = sparkSession, + path = path, + fileFormatClassName = className, + schema = sourceInfo.schema, + partitionColumns = sourceInfo.partitionColumns, + metadataPath = metadataPath, + options = options) case _ => throw new UnsupportedOperationException( s"Data source $className does not support streamed reading") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index ae3e709422812..5ada2384b3089 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -35,6 +35,7 @@ class FileStreamSource( path: String, fileFormatClassName: String, override val schema: StructType, + partitionColumns: Seq[String], metadataPath: String, options: Map[String, String]) extends Source with Logging { @@ -142,6 +143,7 @@ class FileStreamSource( sparkSession, paths = files.map(_.path), userSpecifiedSchema = Some(schema), + partitionColumns = partitionColumns, className = fileFormatClassName, options = optionsWithPartitionBasePath) Dataset.ofRows(sparkSession, LogicalRelation(newDataSource.resolveRelation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala index 1793db0002af4..3bad5bb093b3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala @@ -94,7 +94,7 @@ class FileStreamSourceSuite extends SparkFunSuite with SharedSQLContext { new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath) assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0)))) - val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), + val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil, dir.getAbsolutePath, Map.empty) // this method should throw an exception if `fs.exists` is called during resolveRelation newSource.getBatch(None, LongOffset(1)) From 1c1e847bc47c27f468b62fa35365d23f66bff065 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Oct 2016 17:21:16 -0700 Subject: [PATCH 1452/1470] [SPARK-17624][SQL][STREAMING][TEST] Fixed flaky StateStoreSuite.maintenance ## What changes were proposed in this pull request? The reason for the flakiness was follows. The test starts the maintenance background thread, and then writes 20 versions of the state store. The maintenance thread is expected to create snapshots in the middle, and clean up old files that are not needed any more. The earliest delta file (1.delta) is expected to be deleted as snapshots will ensure that the earliest delta would not be needed. However, the default configuration for the maintenance thread is to retain files such that last 2 versions can be recovered, and delete the rest. Now while generating the versions, the maintenance thread can kick in and create snapshots anywhere between version 10 and 20 (at least 10 deltas needed for snapshot). Then later it will choose to retain only version 20 and 19 (last 2). There are two cases. - Common case: One of the version between 10 and 19 gets snapshotted. Then recovering versions 19 and 20 just needs 19.snapshot and 20.delta, so 1.delta gets deleted. - Uncommon case (reason for flakiness): Only version 20 gets snapshotted. Then recovering versoin 20 requires 20.snapshot, and recovering version 19 all the previous 19...1.delta. So 1.delta does not get deleted. This PR rearranges the checks such that it create 20 versions, and then waits that there is at least one snapshot, then creates another 20. This will ensure that the latest 2 versions cannot require anything older than the first snapshot generated, and therefore will 1.delta will be deleted. In addition, I have added more logs, and comments that I felt would help future debugging and understanding what is going on. ## How was this patch tested? Ran the StateStoreSuite > 6K times in a heavily loaded machine (10 instances of tests running in parallel). No failures. Author: Tathagata Das Closes #15592 from tdas/SPARK-17624. (cherry picked from commit 407c3cedf29a4413339dcde758295dc3225a0054) Signed-off-by: Shixiong Zhu --- .../state/HDFSBackedStateStoreProvider.scala | 18 ++++--- .../state/StateStoreCoordinator.scala | 18 +++++-- .../streaming/state/StateStoreSuite.scala | 49 ++++++++++++------- 3 files changed, 57 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index dce53492f6156..bbbb6d39c63f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -159,7 +159,7 @@ private[state] class HDFSBackedStateStoreProvider( } catch { case NonFatal(e) => throw new IllegalStateException( - s"Error committing version $newVersion into ${HDFSBackedStateStoreProvider.this}", e) + s"Error committing version $newVersion into $this", e) } } @@ -205,6 +205,10 @@ private[state] class HDFSBackedStateStoreProvider( override private[state] def hasCommitted: Boolean = { state == COMMITTED } + + override def toString(): String = { + s"HDFSStateStore[id = (op=${id.operatorId}, part=${id.partitionId}), dir = $baseDir]" + } } /** Get the state store for making updates to create a new `version` of the store. */ @@ -215,7 +219,7 @@ private[state] class HDFSBackedStateStoreProvider( newMap.putAll(loadMap(version)) } val store = new HDFSBackedStateStore(version, newMap) - logInfo(s"Retrieved version $version of $this for update") + logInfo(s"Retrieved version $version of ${HDFSBackedStateStoreProvider.this} for update") store } @@ -231,7 +235,7 @@ private[state] class HDFSBackedStateStoreProvider( } override def toString(): String = { - s"StateStore[id = (op=${id.operatorId}, part=${id.partitionId}), dir = $baseDir]" + s"HDFSStateStoreProvider[id = (op=${id.operatorId}, part=${id.partitionId}), dir = $baseDir]" } /* Internal classes and methods */ @@ -491,10 +495,12 @@ private[state] class HDFSBackedStateStoreProvider( val mapsToRemove = loadedMaps.keys.filter(_ < earliestVersionToRetain).toSeq mapsToRemove.foreach(loadedMaps.remove) } - files.filter(_.version < earliestFileToRetain.version).foreach { f => + val filesToDelete = files.filter(_.version < earliestFileToRetain.version) + filesToDelete.foreach { f => fs.delete(f.path, true) } - logInfo(s"Deleted files older than ${earliestFileToRetain.version} for $this") + logInfo(s"Deleted files older than ${earliestFileToRetain.version} for $this: " + + filesToDelete.mkString(", ")) } } } catch { @@ -558,7 +564,7 @@ private[state] class HDFSBackedStateStoreProvider( } } val storeFiles = versionToFiles.values.toSeq.sortBy(_.version) - logDebug(s"Current set of files for $this: $storeFiles") + logDebug(s"Current set of files for $this: ${storeFiles.mkString(", ")}") storeFiles } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala index d945d7aff2da4..267d17623d5e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinator.scala @@ -38,7 +38,7 @@ private case class VerifyIfInstanceActive(storeId: StateStoreId, executorId: Str private case class GetLocation(storeId: StateStoreId) extends StateStoreCoordinatorMessage -private case class DeactivateInstances(storeRootLocation: String) +private case class DeactivateInstances(checkpointLocation: String) extends StateStoreCoordinatorMessage private object StopCoordinator @@ -111,11 +111,13 @@ class StateStoreCoordinatorRef private(rpcEndpointRef: RpcEndpointRef) { * Class for coordinating instances of [[StateStore]]s loaded in executors across the cluster, * and get their locations for job scheduling. */ -private class StateStoreCoordinator(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint { +private class StateStoreCoordinator(override val rpcEnv: RpcEnv) + extends ThreadSafeRpcEndpoint with Logging { private val instances = new mutable.HashMap[StateStoreId, ExecutorCacheTaskLocation] override def receive: PartialFunction[Any, Unit] = { case ReportActiveInstance(id, host, executorId) => + logDebug(s"Reported state store $id is active at $executorId") instances.put(id, ExecutorCacheTaskLocation(host, executorId)) } @@ -125,19 +127,25 @@ private class StateStoreCoordinator(override val rpcEnv: RpcEnv) extends ThreadS case Some(location) => location.executorId == execId case None => false } + logDebug(s"Verified that state store $id is active: $response") context.reply(response) case GetLocation(id) => - context.reply(instances.get(id).map(_.toString)) + val executorId = instances.get(id).map(_.toString) + logDebug(s"Got location of the state store $id: $executorId") + context.reply(executorId) - case DeactivateInstances(loc) => + case DeactivateInstances(checkpointLocation) => val storeIdsToRemove = - instances.keys.filter(_.checkpointLocation == loc).toSeq + instances.keys.filter(_.checkpointLocation == checkpointLocation).toSeq instances --= storeIdsToRemove + logDebug(s"Deactivating instances related to checkpoint location $checkpointLocation: " + + storeIdsToRemove.mkString(", ")) context.reply(true) case StopCoordinator => stop() // Stop before replying to ensure that endpoint name has been deregistered + logInfo("StateStoreCoordinator stopped") context.reply(true) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 06f1bd6c3bcc7..fcf300b3c81bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -367,7 +367,10 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth val conf = new SparkConf() .setMaster("local") .setAppName("test") + // Make maintenance thread do snapshots and cleanups very fast .set(StateStore.MAINTENANCE_INTERVAL_CONFIG, "10ms") + // Make sure that when SparkContext stops, the StateStore maintenance thread 'quickly' + // fails to talk to the StateStoreCoordinator and unloads all the StateStores .set("spark.rpc.numRetries", "1") val opId = 0 val dir = Utils.createDirectory(tempDir, Random.nextString(5)).toString @@ -377,37 +380,49 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth val provider = new HDFSBackedStateStoreProvider( storeId, keySchema, valueSchema, storeConf, hadoopConf) + var latestStoreVersion = 0 + + def generateStoreVersions() { + for (i <- 1 to 20) { + val store = StateStore.get( + storeId, keySchema, valueSchema, latestStoreVersion, storeConf, hadoopConf) + put(store, "a", i) + store.commit() + latestStoreVersion += 1 + } + } quietly { withSpark(new SparkContext(conf)) { sc => withCoordinatorRef(sc) { coordinatorRef => require(!StateStore.isMaintenanceRunning, "StateStore is unexpectedly running") - for (i <- 1 to 20) { - val store = StateStore.get( - storeId, keySchema, valueSchema, i - 1, storeConf, hadoopConf) - put(store, "a", i) - store.commit() - } + // Generate sufficient versions of store for snapshots + generateStoreVersions() eventually(timeout(10 seconds)) { + // Store should have been reported to the coordinator assert(coordinatorRef.getLocation(storeId).nonEmpty, "active instance was not reported") - } - // Background maintenance should clean up and generate snapshots - assert(StateStore.isMaintenanceRunning, "Maintenance task is not running") - - eventually(timeout(10 seconds)) { - // Earliest delta file should get cleaned up - assert(!fileExists(provider, 1, isSnapshot = false), "earliest file not deleted") + // Background maintenance should clean up and generate snapshots + assert(StateStore.isMaintenanceRunning, "Maintenance task is not running") // Some snapshots should have been generated - val snapshotVersions = (0 to 20).filter { version => + val snapshotVersions = (1 to latestStoreVersion).filter { version => fileExists(provider, version, isSnapshot = true) } assert(snapshotVersions.nonEmpty, "no snapshot file found") } + // Generate more versions such that there is another snapshot and + // the earliest delta file will be cleaned up + generateStoreVersions() + + // Earliest delta file should get cleaned up + eventually(timeout(10 seconds)) { + assert(!fileExists(provider, 1, isSnapshot = false), "earliest file not deleted") + } + // If driver decides to deactivate all instances of the store, then this instance // should be unloaded coordinatorRef.deactivateInstances(dir) @@ -416,7 +431,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth } // Reload the store and verify - StateStore.get(storeId, keySchema, valueSchema, 20, storeConf, hadoopConf) + StateStore.get(storeId, keySchema, valueSchema, latestStoreVersion, storeConf, hadoopConf) assert(StateStore.isLoaded(storeId)) // If some other executor loads the store, then this instance should be unloaded @@ -426,14 +441,14 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth } // Reload the store and verify - StateStore.get(storeId, keySchema, valueSchema, 20, storeConf, hadoopConf) + StateStore.get(storeId, keySchema, valueSchema, latestStoreVersion, storeConf, hadoopConf) assert(StateStore.isLoaded(storeId)) } } // Verify if instance is unloaded if SparkContext is stopped - require(SparkEnv.get === null) eventually(timeout(10 seconds)) { + require(SparkEnv.get === null) assert(!StateStore.isLoaded(storeId)) assert(!StateStore.isMaintenanceRunning) } From 7c8d9a55707ceb1b93ce1f91bcedb10aea8d5c3c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 25 Oct 2016 12:08:17 -0700 Subject: [PATCH 1453/1470] [SPARK-18070][SQL] binary operator should not consider nullability when comparing input types ## What changes were proposed in this pull request? Binary operator requires its inputs to be of same type, but it should not consider nullability, e.g. `EqualTo` should be able to compare an element-nullable array and an element-non-nullable array. ## How was this patch tested? a regression test in `DataFrameSuite` Author: Wenchen Fan Closes #15606 from cloud-fan/type-bug. (cherry picked from commit a21791e3164f4e6546fbe0a90017a4394a05deb1) Signed-off-by: Yin Huai --- .../spark/sql/catalyst/expressions/Expression.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/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 7abbbe257d830..0f6a896a72be5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -511,7 +511,7 @@ abstract class BinaryOperator extends BinaryExpression with ExpectsInputTypes { override def checkInputDataTypes(): TypeCheckResult = { // First check whether left and right have the same type, then check if the type is acceptable. - if (left.dataType != right.dataType) { + if (!left.dataType.sameType(right.dataType)) { TypeCheckResult.TypeCheckFailure(s"differing types in '$sql' " + s"(${left.dataType.simpleString} and ${right.dataType.simpleString}).") } else if (!inputType.acceptsType(left.dataType)) { 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 f8d7ddde85d92..4478a9afe8585 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 @@ -1601,4 +1601,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { dates.except(widenTypedRows).collect() dates.intersect(widenTypedRows).collect() } + + test("SPARK-18070 binary operator should not consider nullability when comparing input types") { + val rows = Seq(Row(Seq(1), Seq(1))) + val schema = new StructType() + .add("array1", ArrayType(IntegerType)) + .add("array2", ArrayType(IntegerType, containsNull = false)) + val df = spark.createDataFrame(spark.sparkContext.makeRDD(rows), schema) + assert(df.filter($"array1" === $"array2").count() == 1) + } } From 912487eaf84a82731033d17035f401acd5577994 Mon Sep 17 00:00:00 2001 From: hayashidac Date: Wed, 26 Oct 2016 07:13:48 +0900 Subject: [PATCH 1454/1470] [SPARK-16988][SPARK SHELL] spark history server log needs to be fixed to show https url when ssl is enabled spark history server log needs to be fixed to show https url when ssl is enabled Author: chie8842 Closes #15611 from hayashidac/SPARK-16988. (cherry picked from commit c329a568b58d65c492a43926bf0f588f2ae6a66e) Signed-off-by: Kousuke Saruta --- core/src/main/scala/org/apache/spark/ui/WebUI.scala | 5 ++++- .../test/scala/org/apache/spark/SSLOptionsSuite.scala | 10 +++++----- 2 files changed, 9 insertions(+), 6 deletions(-) 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 2c40e726992d3..adc4a4faccad1 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -146,7 +146,10 @@ private[spark] abstract class WebUI( } /** Return the url of web interface. Only valid after bind(). */ - def webUrl: String = s"http://$publicHostName:$boundPort" + def webUrl: String = { + val protocol = if (sslOptions.enabled) "https" else "http" + s"$protocol://$publicHostName:$boundPort" + } /** Return the actual port to which this server is bound. Only valid after bind(). */ def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala index 159b448e05b02..2b8b1805bc83f 100644 --- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -79,7 +79,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { conf.set("spark.ssl.protocol", "SSLv3") val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) - val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = Some(defaultOpts)) assert(opts.enabled === true) assert(opts.trustStore.isDefined === true) @@ -102,20 +102,20 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { val conf = new SparkConf conf.set("spark.ssl.enabled", "true") - conf.set("spark.ui.ssl.enabled", "false") + conf.set("spark.ssl.ui.enabled", "false") conf.set("spark.ssl.keyStore", keyStorePath) conf.set("spark.ssl.keyStorePassword", "password") - conf.set("spark.ui.ssl.keyStorePassword", "12345") + conf.set("spark.ssl.ui.keyStorePassword", "12345") 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.ui.ssl.enabledAlgorithms", "ABC, DEF") + conf.set("spark.ssl.ui.enabledAlgorithms", "ABC, DEF") conf.set("spark.ssl.protocol", "SSLv3") val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) - val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = Some(defaultOpts)) assert(opts.enabled === false) assert(opts.trustStore.isDefined === true) From c2cce2e60c491d5ff2b1eb1f30eb507f0d40dae1 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 26 Oct 2016 14:19:40 +0200 Subject: [PATCH 1455/1470] [SPARK-18022][SQL] java.lang.NullPointerException instead of real exception when saving DF to MySQL ## What changes were proposed in this pull request? On null next exception in JDBC, don't init it as cause or suppressed ## How was this patch tested? Existing tests Author: Sean Owen Closes #15599 from srowen/SPARK-18022. (cherry picked from commit 6c7d094ec4d45a05c1ec8a418e507e45f5a88b7d) Signed-off-by: Sean Owen --- .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 7a8b82509383b..2869e801deb96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -236,7 +236,7 @@ object JdbcUtils extends Logging { } catch { case e: SQLException => val cause = e.getNextException - if (e.getCause != cause) { + if (cause != null && e.getCause != cause) { if (e.getCause == null) { e.initCause(cause) } else { From 192c1dd4fef4931b508b02d38a8c2405aa4785a0 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Wed, 26 Oct 2016 17:09:48 +0200 Subject: [PATCH 1456/1470] [SPARK-17733][SQL] InferFiltersFromConstraints rule never terminates for query ## What changes were proposed in this pull request? The function `QueryPlan.inferAdditionalConstraints` and `UnaryNode.getAliasedConstraints` can produce a non-converging set of constraints for recursive functions. For instance, if we have two constraints of the form(where a is an alias): `a = b, a = f(b, c)` Applying both these rules in the next iteration would infer: `f(b, c) = f(f(b, c), c)` This process repeated, the iteration won't converge and the set of constraints will grow larger and larger until OOM. ~~To fix this problem, we collect alias from expressions and skip infer constraints if we are to transform an `Expression` to another which contains it.~~ To fix this problem, we apply additional check in `inferAdditionalConstraints`, when it's possible to generate recursive constraints, we skip generate that. ## How was this patch tested? Add new testcase in `SQLQuerySuite`/`InferFiltersFromConstraintsSuite`. Author: jiangxingbo Closes #15319 from jiangxb1987/constraints. (cherry picked from commit 3c023570b28bc1ed24f5b2448311130fd1777fd3) Signed-off-by: Herman van Hovell --- .../spark/sql/catalyst/plans/QueryPlan.scala | 88 +++++++++++++++++-- .../InferFiltersFromConstraintsSuite.scala | 87 +++++++++++++++++- .../spark/sql/catalyst/plans/PlanTest.scala | 25 +++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 5 +- 4 files changed, 191 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 9c605902926e8..41c4e0085a93e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -68,26 +68,104 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT case _ => Seq.empty[Attribute] } + // Collect aliases from expressions, so we may avoid producing recursive constraints. + private lazy val aliasMap = AttributeMap( + (expressions ++ children.flatMap(_.expressions)).collect { + case a: Alias => (a.toAttribute, a.child) + }) + /** * Infers an additional set of constraints from a given set of equality constraints. * For e.g., if an operator has constraints of the form (`a = 5`, `a = b`), this returns an - * additional constraint of the form `b = 5` + * additional constraint of the form `b = 5`. + * + * [SPARK-17733] We explicitly prevent producing recursive constraints of the form `a = f(a, b)` + * as they are often useless and can lead to a non-converging set of constraints. */ private def inferAdditionalConstraints(constraints: Set[Expression]): Set[Expression] = { + val constraintClasses = generateEquivalentConstraintClasses(constraints) + var inferredConstraints = Set.empty[Expression] constraints.foreach { case eq @ EqualTo(l: Attribute, r: Attribute) => - inferredConstraints ++= (constraints - eq).map(_ transform { - case a: Attribute if a.semanticEquals(l) => r + val candidateConstraints = constraints - eq + inferredConstraints ++= candidateConstraints.map(_ transform { + case a: Attribute if a.semanticEquals(l) && + !isRecursiveDeduction(r, constraintClasses) => r }) - inferredConstraints ++= (constraints - eq).map(_ transform { - case a: Attribute if a.semanticEquals(r) => l + inferredConstraints ++= candidateConstraints.map(_ transform { + case a: Attribute if a.semanticEquals(r) && + !isRecursiveDeduction(l, constraintClasses) => l }) case _ => // No inference } inferredConstraints -- constraints } + /* + * Generate a sequence of expression sets from constraints, where each set stores an equivalence + * class of expressions. For example, Set(`a = b`, `b = c`, `e = f`) will generate the following + * expression sets: (Set(a, b, c), Set(e, f)). This will be used to search all expressions equal + * to an selected attribute. + */ + private def generateEquivalentConstraintClasses( + constraints: Set[Expression]): Seq[Set[Expression]] = { + var constraintClasses = Seq.empty[Set[Expression]] + constraints.foreach { + case eq @ EqualTo(l: Attribute, r: Attribute) => + // Transform [[Alias]] to its child. + val left = aliasMap.getOrElse(l, l) + val right = aliasMap.getOrElse(r, r) + // Get the expression set for an equivalence constraint class. + val leftConstraintClass = getConstraintClass(left, constraintClasses) + val rightConstraintClass = getConstraintClass(right, constraintClasses) + if (leftConstraintClass.nonEmpty && rightConstraintClass.nonEmpty) { + // Combine the two sets. + constraintClasses = constraintClasses + .diff(leftConstraintClass :: rightConstraintClass :: Nil) :+ + (leftConstraintClass ++ rightConstraintClass) + } else if (leftConstraintClass.nonEmpty) { // && rightConstraintClass.isEmpty + // Update equivalence class of `left` expression. + constraintClasses = constraintClasses + .diff(leftConstraintClass :: Nil) :+ (leftConstraintClass + right) + } else if (rightConstraintClass.nonEmpty) { // && leftConstraintClass.isEmpty + // Update equivalence class of `right` expression. + constraintClasses = constraintClasses + .diff(rightConstraintClass :: Nil) :+ (rightConstraintClass + left) + } else { // leftConstraintClass.isEmpty && rightConstraintClass.isEmpty + // Create new equivalence constraint class since neither expression presents + // in any classes. + constraintClasses = constraintClasses :+ Set(left, right) + } + case _ => // Skip + } + + constraintClasses + } + + /* + * Get all expressions equivalent to the selected expression. + */ + private def getConstraintClass( + expr: Expression, + constraintClasses: Seq[Set[Expression]]): Set[Expression] = + constraintClasses.find(_.contains(expr)).getOrElse(Set.empty[Expression]) + + /* + * Check whether replace by an [[Attribute]] will cause a recursive deduction. Generally it + * has the form like: `a -> f(a, b)`, where `a` and `b` are expressions and `f` is a function. + * Here we first get all expressions equal to `attr` and then check whether at least one of them + * is a child of the referenced expression. + */ + private def isRecursiveDeduction( + attr: Attribute, + constraintClasses: Seq[Set[Expression]]): Boolean = { + val expr = aliasMap.getOrElse(attr, attr) + getConstraintClass(expr, constraintClasses).exists { e => + expr.children.exists(_.semanticEquals(e)) + } + } + /** * An [[ExpressionSet]] that contains invariants about the rows output by this operator. For * example, if this set contains the expression `a = 2` then that expression is guaranteed to diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index e7fdd5a6202b6..9f57f66a2ea20 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -27,9 +27,12 @@ import org.apache.spark.sql.catalyst.rules._ class InferFiltersFromConstraintsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - val batches = Batch("InferFilters", FixedPoint(5), InferFiltersFromConstraints) :: - Batch("PredicatePushdown", FixedPoint(5), PushPredicateThroughJoin) :: - Batch("CombineFilters", FixedPoint(5), CombineFilters) :: Nil + val batches = + Batch("InferAndPushDownFilters", FixedPoint(100), + PushPredicateThroughJoin, + PushDownPredicate, + InferFiltersFromConstraints, + CombineFilters) :: Nil } val testRelation = LocalRelation('a.int, 'b.int, 'c.int) @@ -120,4 +123,82 @@ class InferFiltersFromConstraintsSuite extends PlanTest { val optimized = Optimize.execute(originalQuery) comparePlans(optimized, correctAnswer) } + + test("inner join with alias: alias contains multiple attributes") { + val t1 = testRelation.subquery('t1) + val t2 = testRelation.subquery('t2) + + val originalQuery = t1.select('a, Coalesce(Seq('a, 'b)).as('int_col)).as("t") + .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) + .analyze + val correctAnswer = t1 + .where(IsNotNull('a) && IsNotNull(Coalesce(Seq('a, 'b))) && 'a === Coalesce(Seq('a, 'b))) + .select('a, Coalesce(Seq('a, 'b)).as('int_col)).as("t") + .join(t2.where(IsNotNull('a)), Inner, + Some("t.a".attr === "t2.a".attr && "t.int_col".attr === "t2.a".attr)) + .analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("inner join with alias: alias contains single attributes") { + val t1 = testRelation.subquery('t1) + val t2 = testRelation.subquery('t2) + + val originalQuery = t1.select('a, 'b.as('d)).as("t") + .join(t2, Inner, Some("t.a".attr === "t2.a".attr && "t.d".attr === "t2.a".attr)) + .analyze + val correctAnswer = t1 + .where(IsNotNull('a) && IsNotNull('b) && 'a <=> 'a && 'b <=> 'b &&'a === 'b) + .select('a, 'b.as('d)).as("t") + .join(t2.where(IsNotNull('a) && 'a <=> 'a), Inner, + Some("t.a".attr === "t2.a".attr && "t.d".attr === "t2.a".attr)) + .analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("inner join with alias: don't generate constraints for recursive functions") { + val t1 = testRelation.subquery('t1) + val t2 = testRelation.subquery('t2) + + val originalQuery = t1.select('a, 'b.as('d), Coalesce(Seq('a, 'b)).as('int_col)).as("t") + .join(t2, Inner, + Some("t.a".attr === "t2.a".attr + && "t.d".attr === "t2.a".attr + && "t.int_col".attr === "t2.a".attr)) + .analyze + val correctAnswer = t1 + .where(IsNotNull('a) && IsNotNull(Coalesce(Seq('a, 'a))) + && 'a === Coalesce(Seq('a, 'a)) && 'a <=> Coalesce(Seq('a, 'a)) && 'a <=> 'a + && Coalesce(Seq('a, 'a)) <=> 'b && Coalesce(Seq('a, 'a)) <=> Coalesce(Seq('a, 'a)) + && 'a === 'b && IsNotNull(Coalesce(Seq('a, 'b))) && 'a === Coalesce(Seq('a, 'b)) + && Coalesce(Seq('a, 'b)) <=> Coalesce(Seq('b, 'b)) && Coalesce(Seq('a, 'b)) === 'b + && IsNotNull('b) && IsNotNull(Coalesce(Seq('b, 'b))) + && 'b === Coalesce(Seq('b, 'b)) && 'b <=> Coalesce(Seq('b, 'b)) + && Coalesce(Seq('b, 'b)) <=> Coalesce(Seq('b, 'b)) && 'b <=> 'b) + .select('a, 'b.as('d), Coalesce(Seq('a, 'b)).as('int_col)).as("t") + .join(t2 + .where(IsNotNull('a) && IsNotNull(Coalesce(Seq('a, 'a))) + && 'a === Coalesce(Seq('a, 'a)) && 'a <=> Coalesce(Seq('a, 'a)) && 'a <=> 'a + && Coalesce(Seq('a, 'a)) <=> Coalesce(Seq('a, 'a))), Inner, + Some("t.a".attr === "t2.a".attr + && "t.d".attr === "t2.a".attr + && "t.int_col".attr === "t2.a".attr + && Coalesce(Seq("t.d".attr, "t.d".attr)) <=> "t.int_col".attr)) + .analyze + val optimized = Optimize.execute(originalQuery) + comparePlans(optimized, correctAnswer) + } + + test("generate correct filters for alias that don't produce recursive constraints") { + val t1 = testRelation.subquery('t1) + + val originalQuery = t1.select('a.as('x), 'b.as('y)).where('x === 1 && 'x === 'y).analyze + val correctAnswer = + t1.where('a === 1 && 'b === 1 && 'a === 'b && IsNotNull('a) && IsNotNull('b)) + .select('a.as('x), 'b.as('y)).analyze + val optimized = Optimize.execute(originalQuery) + 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 6310f0c2bc0ed..64e268703bf5e 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, OneRowRelation, Sample} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ /** @@ -56,16 +56,37 @@ abstract class PlanTest extends SparkFunSuite with PredicateHelper { * ((expr 1 && expr 2) && expr 3), (expr 1 && expr 2 && expr 3), (expr 3 && (expr 1 && expr 2) * etc., will all now be equivalent. * - Sample the seed will replaced by 0L. + * - Join conditions will be resorted by hashCode. */ private def normalizePlan(plan: LogicalPlan): LogicalPlan = { plan transform { case filter @ Filter(condition: Expression, child: LogicalPlan) => - Filter(splitConjunctivePredicates(condition).sortBy(_.hashCode()).reduce(And), child) + Filter(splitConjunctivePredicates(condition).map(rewriteEqual(_)).sortBy(_.hashCode()) + .reduce(And), child) case sample: Sample => sample.copy(seed = 0L)(true) + case join @ Join(left, right, joinType, condition) if condition.isDefined => + val newCondition = + splitConjunctivePredicates(condition.get).map(rewriteEqual(_)).sortBy(_.hashCode()) + .reduce(And) + Join(left, right, joinType, Some(newCondition)) } } + /** + * Rewrite [[EqualTo]] and [[EqualNullSafe]] operator to keep order. The following cases will be + * equivalent: + * 1. (a = b), (b = a); + * 2. (a <=> b), (b <=> a). + */ + private def rewriteEqual(condition: Expression): Expression = condition match { + case eq @ EqualTo(l: Expression, r: Expression) => + Seq(l, r).sortBy(_.hashCode()).reduce(EqualTo) + case eq @ EqualNullSafe(l: Expression, r: Expression) => + Seq(l, r).sortBy(_.hashCode()).reduce(EqualNullSafe) + case _ => condition // Don't reorder. + } + /** Fails the test if the two plans do not match */ protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { val normalized1 = normalizePlan(normalizeExprIds(plan1)) 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 3684135f6822c..8ca470873c9bc 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,12 +19,9 @@ package org.apache.spark.sql import java.io.File import java.math.MathContext -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import org.apache.spark.{AccumulatorSuite, SparkException} -import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.expressions.SortOrder -import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} From b4a7b6551d5db8f280656624b19bebf43fcfb819 Mon Sep 17 00:00:00 2001 From: Mark Grover Date: Wed, 26 Oct 2016 09:07:30 -0700 Subject: [PATCH 1457/1470] =?UTF-8?q?[SPARK-18093][SQL]=20Fix=20default=20?= =?UTF-8?q?value=20test=20in=20SQLConfSuite=20to=20work=20rega=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …rdless of warehouse dir's existence ## What changes were proposed in this pull request? Appending a trailing slash, if there already isn't one for the sake comparison of the two paths. It doesn't take away from the essence of the check, but removes any potential mismatch due to lack of trailing slash. ## How was this patch tested? Ran unit tests and they passed. Author: Mark Grover Closes #15623 from markgrover/spark-18093. (cherry picked from commit 4bee9540790a40acb74db4b0b44c364c4b3f537d) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index a230344f73e7e..41e011b8448d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -212,12 +212,15 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } test("default value of WAREHOUSE_PATH") { + val original = spark.conf.get(SQLConf.WAREHOUSE_PATH) try { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) - assert(new Path(Utils.resolveURI("spark-warehouse")).toString === - spark.sessionState.conf.warehousePath + "/") + // JVM adds a trailing slash if the directory exists and leaves it as-is, if it doesn't + // In our comparison, strip trailing slash off of both sides, to account for such cases + assert(new Path(Utils.resolveURI("spark-warehouse")).toString.stripSuffix("/") === spark + .sessionState.conf.warehousePath.stripSuffix("/")) } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") } From 773fbfef1929b64229fbf97a91c45cdb1ec1fb1f Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Wed, 6 Jul 2016 10:46:22 -0700 Subject: [PATCH 1458/1470] [SPARK-16304] LinkageError should not crash Spark executor ## What changes were proposed in this pull request? This patch updates the failure handling logic so Spark executor does not crash when seeing LinkageError. ## How was this patch tested? Added an end-to-end test in FailureSuite. Author: petermaxlee Closes #13982 from petermaxlee/SPARK-16304. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 6 +++++- core/src/test/scala/org/apache/spark/FailureSuite.scala | 9 +++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 1686edb502f69..b9cf721d9f54c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1977,7 +1977,11 @@ private[spark] object Utils extends Logging { /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ def isFatalError(e: Throwable): Boolean = { e match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + case NonFatal(_) | + _: InterruptedException | + _: NotImplementedError | + _: ControlThrowable | + _: LinkageError => false case _ => true diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 132f6361e41e6..d805c67714ff8 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -253,6 +253,15 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { rdd.count() } + test("SPARK-16304: Link error should not crash executor") { + sc = new SparkContext("local[1,2]", "test") + intercept[SparkException] { + sc.parallelize(1 to 2).foreach { i => + throw new LinkageError() + } + } + } + // TODO: Need to add tests with shuffle fetch failures. } From 5b81b01026bc215c7982a640a794cd36ea720959 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Wed, 26 Oct 2016 20:12:20 +0200 Subject: [PATCH 1459/1470] [SPARK-18063][SQL] Failed to infer constraints over multiple aliases ## What changes were proposed in this pull request? The `UnaryNode.getAliasedConstraints` function fails to replace all expressions by their alias where constraints contains more than one expression to be replaced. For example: ``` val tr = LocalRelation('a.int, 'b.string, 'c.int) val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y)) multiAlias.analyze.constraints ``` currently outputs: ``` ExpressionSet(Seq( IsNotNull(resolveColumn(multiAlias.analyze, "x")), IsNotNull(resolveColumn(multiAlias.analyze, "y")) ) ``` The constraint `resolveColumn(multiAlias.analyze, "x") === resolveColumn(multiAlias.analyze, "y") + 10)` is missing. ## How was this patch tested? Add new test cases in `ConstraintPropagationSuite`. Author: jiangxingbo Closes #15597 from jiangxb1987/alias-constraints. (cherry picked from commit fa7d9d70825a6816495d239da925d0087f7cb94f) Signed-off-by: Reynold Xin --- .../sql/catalyst/plans/logical/LogicalPlan.scala | 16 ++++++++++------ .../plans/ConstraintPropagationSuite.scala | 8 ++++++++ 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 6d7799151d93b..9c152fb88412a 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 @@ -293,15 +293,19 @@ abstract class UnaryNode extends LogicalPlan { * expressions with the corresponding alias */ protected def getAliasedConstraints(projectList: Seq[NamedExpression]): Set[Expression] = { - projectList.flatMap { + var allConstraints = child.constraints.asInstanceOf[Set[Expression]] + projectList.foreach { case a @ Alias(e, _) => - child.constraints.map(_ transform { + // For every alias in `projectList`, replace the reference in constraints by its attribute. + allConstraints ++= allConstraints.map(_ transform { case expr: Expression if expr.semanticEquals(e) => a.toAttribute - }).union(Set(EqualNullSafe(e, a.toAttribute))) - case _ => - Set.empty[Expression] - }.toSet + }) + allConstraints += EqualNullSafe(e, a.toAttribute) + case _ => // Don't change. + } + + allConstraints -- child.constraints } override protected def validConstraints: Set[Expression] = child.constraints diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala index 8d6a49a8a37b4..8068ce922e636 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/ConstraintPropagationSuite.scala @@ -128,8 +128,16 @@ class ConstraintPropagationSuite extends SparkFunSuite { ExpressionSet(Seq(resolveColumn(aliasedRelation.analyze, "x") > 10, IsNotNull(resolveColumn(aliasedRelation.analyze, "x")), resolveColumn(aliasedRelation.analyze, "b") <=> resolveColumn(aliasedRelation.analyze, "y"), + resolveColumn(aliasedRelation.analyze, "z") <=> resolveColumn(aliasedRelation.analyze, "x"), resolveColumn(aliasedRelation.analyze, "z") > 10, IsNotNull(resolveColumn(aliasedRelation.analyze, "z"))))) + + val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y)) + verifyConstraints(multiAlias.analyze.constraints, + ExpressionSet(Seq(IsNotNull(resolveColumn(multiAlias.analyze, "x")), + IsNotNull(resolveColumn(multiAlias.analyze, "y")), + resolveColumn(multiAlias.analyze, "x") === resolveColumn(multiAlias.analyze, "y") + 10)) + ) } test("propagating constraints in union") { From b482b3d586543744bdfe462351d945b9761b54c8 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 26 Oct 2016 11:16:20 -0700 Subject: [PATCH 1460/1470] [SPARK-18104][DOC] Don't build KafkaSource doc ## What changes were proposed in this pull request? Don't need to build doc for KafkaSource because the user should use the data source APIs to use KafkaSource. All KafkaSource APIs are internal. ## How was this patch tested? Verified manually. Author: Shixiong Zhu Closes #15630 from zsxwing/kafka-unidoc. (cherry picked from commit 7d10631c16b980adf1f55378c128436310daed65) Signed-off-by: Shixiong Zhu --- project/SparkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7f7a65f552713..98f1e23a9fd77 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -713,9 +713,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010, sqlKafka010), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, tags, streamingKafka010, sqlKafka010), unidocAllClasspaths in (ScalaUnidoc, unidoc) := { ignoreClasspaths((unidocAllClasspaths in (ScalaUnidoc, unidoc)).value) From 76b71eef46a3b932d6de7f831f0245ea27e3dfe7 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 26 Oct 2016 13:21:46 -0700 Subject: [PATCH 1461/1470] [SPARK-13747][SQL] Fix concurrent executions in ForkJoinPool for SQL (branch 2.0) ## What changes were proposed in this pull request? Backport #15520 to 2.0. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15646 from zsxwing/SPARK-13747-2.0. --- .../org/apache/spark/util/ThreadUtils.scala | 21 +++++++++++++++++++ scalastyle-config.xml | 1 + .../spark/sql/execution/SparkPlan.scala | 2 +- .../exchange/BroadcastExchangeExec.scala | 3 ++- 4 files changed, 25 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index 5a6dbc830448a..d093e7bfc3dac 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -194,4 +194,25 @@ private[spark] object ThreadUtils { throw new SparkException("Exception thrown in awaitResult: ", t) } } + + /** + * Calls [[Awaitable.result]] directly to avoid using `ForkJoinPool`'s `BlockingContext`, wraps + * and re-throws any exceptions with nice stack track. + * + * Codes running in the user's thread may be in a thread of Scala ForkJoinPool. As concurrent + * executions in ForkJoinPool may see some [[ThreadLocal]] value unexpectedly, this method + * basically prevents ForkJoinPool from running other tasks in the current waiting thread. + */ + @throws(classOf[SparkException]) + def awaitResultInForkJoinSafely[T](awaitable: Awaitable[T], atMost: Duration): T = { + try { + // `awaitPermission` is not actually used anywhere so it's safe to pass in null here. + // See SPARK-13747. + val awaitPermission = null.asInstanceOf[scala.concurrent.CanAwait] + awaitable.result(Duration.Inf)(awaitPermission) + } catch { + case NonFatal(t) => + throw new SparkException("Exception thrown in awaitResult: ", t) + } + } } diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 7fe0697202cd1..81d57d723a720 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -200,6 +200,7 @@ This file is divided into 3 sections: // scalastyle:off awaitresult Await.result(...) // scalastyle:on awaitresult + If your codes use ThreadLocal and may run in threads created by the user, use ThreadUtils.awaitResultInForkJoinSafely instead. ]]> 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 79cb40948b982..fa40414bcea86 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 @@ -166,7 +166,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ protected def waitForSubqueries(): Unit = synchronized { // fill in the result of subqueries subqueryResults.foreach { case (e, futureResult) => - val rows = ThreadUtils.awaitResult(futureResult, Duration.Inf) + val rows = ThreadUtils.awaitResultInForkJoinSafely(futureResult, Duration.Inf) if (rows.length > 1) { sys.error(s"more than one row returned by a subquery used as an expression:\n${e.plan}") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index a809076de5419..4b45ce9c73ee1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -117,7 +117,8 @@ case class BroadcastExchangeExec( } override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - ThreadUtils.awaitResult(relationFuture, timeout).asInstanceOf[broadcast.Broadcast[T]] + ThreadUtils.awaitResultInForkJoinSafely(relationFuture, timeout) + .asInstanceOf[broadcast.Broadcast[T]] } } From 1c2908eeb8890fdc91413a3f5bad2bb3d114db6c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Oct 2016 14:01:59 -0700 Subject: [PATCH 1462/1470] Preparing Spark release v2.0.2-rc1 --- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 2 +- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 36 files changed, 36 insertions(+), 36 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index ca6daa27614f5..58feedc4b2ebd 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index c727f54d2ca3d..a75d2228b9e0b 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index e335a89aac69e..828a40717aba8 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 8e64f5678d78c..30891f3ca1ee7 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 94c75d68e34f5..ea2f4f520939f 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6ff14d29474c2..ed31f256abe2e 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index e654c5479b34c..0553a48258204 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index aad87f7e12a68..f154df2f6f6d1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 09c54188358ab..824197b5a71c9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,7 +14,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.2-SNAPSHOT +SPARK_VERSION: 2.0.2 SPARK_VERSION_SHORT: 2.0.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" diff --git a/examples/pom.xml b/examples/pom.xml index 1796c472ca59e..a5710625e007f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae3746fe55ee3..46b1410a24d03 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 6cf206d36f448..451b1011e62b4 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9ae7968c7296e..d48a09b620eaa 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 7a67142dbf07f..1d9c63a22b423 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 0f795ff77b5ed..27a1f53e71f74 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 658c396c57f19..c5e42894600b2 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 6aa57a667488d..20ee8dc545cba 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index e52170acedf87..86bfa38c3352c 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a4f339fb3fa2f..ac1d2891cd909 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 4d2df2e45b1a4..3ea78f2be7edd 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0e5ce9a78887f..052ffdcb2eb38 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index c9a9e73a0ddb9..4304967f27d74 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index b92359a476268..261a2b27a8844 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c88efda8955eb..f7911c75f7812 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4375e645ca05d..944d3bf71e5ac 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 9e3d7fbbdd565..760968a6fe1d7 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index db7f05302a12b..1087e3b80f3ad 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/pom.xml b/pom.xml index 90d5ca0394d73..6ed58ab140246 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index afb8891168fe6..fef2c5fa4097e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ad3d408b8e9b2..ed74cf5467356 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 6a4ec7a47f85c..c0a355f9ac36f 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7624c82b8ac51..73271b611fc1d 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ae66d7eeb32bf..bd62d26dae5f8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 897ef824976fd..b6a1ab777e72a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7f55ebc87f682..474df40bf4fb1 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d81ce15b8bdf3..063efee2acc3f 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2-SNAPSHOT + 2.0.2 ../pom.xml From 72b3cff33643529c8d363e0ae1a3fe2bd00ff4fd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Oct 2016 14:02:05 -0700 Subject: [PATCH 1463/1470] Preparing development version 2.0.3-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index dfb7e2232f8f4..0b01ca8dc8b20 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,7 +1,7 @@ Package: SparkR Type: Package Title: R Frontend for Apache Spark -Version: 2.0.2 +Version: 2.0.3 Date: 2016-08-27 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 58feedc4b2ebd..de09fce6a48d3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index a75d2228b9e0b..2ee104f3eaf39 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 828a40717aba8..b20f9e2aab2c3 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 30891f3ca1ee7..06895c67560a6 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index ea2f4f520939f..d42be3a792778 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ed31f256abe2e..43531a256de9a 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 0553a48258204..61f4a38531e68 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index f154df2f6f6d1..55cbe898537cb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 824197b5a71c9..c94dc54c0fd02 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.0.2 -SPARK_VERSION_SHORT: 2.0.2 +SPARK_VERSION: 2.0.3-SNAPSHOT +SPARK_VERSION_SHORT: 2.0.3 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" MESOS_VERSION: 0.21.0 diff --git a/examples/pom.xml b/examples/pom.xml index a5710625e007f..b4ba272884c36 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 46b1410a24d03..3ef4c928cb5fd 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 451b1011e62b4..87668e248d941 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d48a09b620eaa..e33cffc78415c 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 1d9c63a22b423..b7f3060b8afc0 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 27a1f53e71f74..0dd5d3d32a31d 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index c5e42894600b2..85186f484e650 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 20ee8dc545cba..3278ee35ec23b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 86bfa38c3352c..0cde10930272d 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index ac1d2891cd909..d6edd5a2c107e 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 3ea78f2be7edd..d595c5552c6a9 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 052ffdcb2eb38..8076c899f21a1 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4304967f27d74..5a59235753ed5 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 261a2b27a8844..9a0dbb1d4e2f4 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f7911c75f7812..9dc26601d8e2d 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 944d3bf71e5ac..8e8610c37247d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 760968a6fe1d7..0c0fbaebe8a81 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 1087e3b80f3ad..f7bf11f48ca97 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 6ed58ab140246..d97f348f6d545 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index fef2c5fa4097e..00fadf2b73dd6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ed74cf5467356..4b039122e9151 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c0a355f9ac36f..e208fd355b9a7 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 73271b611fc1d..c0d8ca53ed2da 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index bd62d26dae5f8..68825c654c7d4 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index b6a1ab777e72a..ab1f3ca190f7d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 474df40bf4fb1..4c2dab471407e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 063efee2acc3f..2dc61d002f5ac 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.0.2 + 2.0.3-SNAPSHOT ../pom.xml From ea205e376d555869519ee59186f53ed573ccee39 Mon Sep 17 00:00:00 2001 From: frreiss Date: Wed, 26 Oct 2016 17:33:08 -0700 Subject: [PATCH 1464/1470] [SPARK-16963][STREAMING][SQL] Changes to Source trait and related implementation classes ## What changes were proposed in this pull request? This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes: * Added a method `commit(end: Offset)` that tells the Source that is OK to discard all offsets up `end`, inclusive. * Changed the semantics of a `None` value for the `getBatch` method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer". * Added notes that the upper layers of the system will never call `getBatch` with a start value less than the last value passed to `commit`. * Added a `lastCommittedOffset` method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code. * The scheduler in `StreamExecution.scala` now calls `commit` on its stream sources after marking each batch as complete in its checkpoint. * `MemoryStream` now cleans committed batches out of its internal buffer. * `TextSocketSource` now cleans committed batches from its internal buffer. ## How was this patch tested? Existing regression tests already exercise the new code. Author: frreiss Closes #14553 from frreiss/fred-16963. (cherry picked from commit 5b27598ff50cb08e7570fade458da0a3d4d4eabc) Signed-off-by: Shixiong Zhu --- .../streaming/FileStreamSource.scala | 9 +++ .../sql/execution/streaming/Source.scala | 22 ++++-- .../execution/streaming/StreamExecution.scala | 32 ++++++--- .../sql/execution/streaming/memory.scala | 47 ++++++++++-- .../sql/execution/streaming/socket.scala | 72 +++++++++++++++---- .../sql/streaming/StreamingQuerySuite.scala | 8 +-- 6 files changed, 154 insertions(+), 36 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 5ada2384b3089..c47033a07a19d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -176,6 +176,15 @@ class FileStreamSource( override def toString: String = s"FileStreamSource[$qualifiedBasePath]" + /** + * Informs the source that Spark has completed processing all data for offsets less than or + * equal to `end` and will only request offsets greater than `end` in the future. + */ + override def commit(end: Offset): Unit = { + // No-op for now; FileStreamSource currently garbage-collects files based on timestamp + // and the value of the maxFileAge parameter. + } + override def stop() {} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index 971147840d2fd..f3bd5bfe23fdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -30,16 +30,30 @@ trait Source { /** Returns the schema of the data from this source */ def schema: StructType - /** Returns the maximum available offset for this source. */ + /** + * Returns the maximum available offset for this source. + * Returns `None` if this source has never received any data. + */ def getOffset: Option[Offset] /** - * Returns the data that is between the offsets (`start`, `end`]. When `start` is `None` then - * the batch should begin with the first available record. This method must always return the - * same data for a particular `start` and `end` pair. + * Returns the data that is between the offsets (`start`, `end`]. When `start` is `None`, + * then the batch should begin with the first record. This method must always return the + * same data for a particular `start` and `end` pair; even after the Source has been restarted + * on a different node. + * + * Higher layers will always call this method with a value of `start` greater than or equal + * to the last value passed to `commit` and a value of `end` less than or equal to the + * last value returned by `getOffset` */ def getBatch(start: Option[Offset], end: Offset): DataFrame + /** + * Informs the source that Spark has completed processing all data for offsets less than or + * equal to `end` and will only request offsets greater than `end` in the future. + */ + def commit(end: Offset) : Unit = {} + /** Stop this source and free any resources it has allocated. */ def stop(): Unit } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 627b87b8955d5..4707bfbce8e0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -73,6 +73,9 @@ class StreamExecution( /** * Tracks how much data we have processed and committed to the sink or state store from each * input source. + * Only the scheduler thread should modify this field, and only in atomic steps. + * Other threads should make a shallow copy if they are going to access this field more than + * once, since the field's value may change at any time. */ @volatile var committedOffsets = new StreamProgress @@ -80,6 +83,9 @@ class StreamExecution( /** * Tracks the offsets that are available to be processed, but have not yet be committed to the * sink. + * Only the scheduler thread should modify this field, and only in atomic steps. + * Other threads should make a shallow copy if they are going to access this field more than + * once, since the field's value may change at any time. */ @volatile private var availableOffsets = new StreamProgress @@ -337,17 +343,27 @@ class StreamExecution( } if (hasNewData) { reportTimeTaken(OFFSET_WAL_WRITE_LATENCY) { - assert( - offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), + assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") logInfo(s"Committed offsets for batch $currentBatchId.") + // NOTE: The following code is correct because runBatches() processes exactly one + // batch at a time. If we add pipeline parallelism (multiple batches in flight at + // the same time), this cleanup logic will need to change. + + // Now that we've updated the scheduler's persistent checkpoint, it is safe for the + // sources to discard data from the previous batch. + val prevBatchOff = offsetLog.get(currentBatchId - 1) + if (prevBatchOff.isDefined) { + prevBatchOff.get.toStreamProgress(sources).foreach { + case (src, off) => src.commit(off) + } + } + // Now that we have logged the new batch, no further processing will happen for - // the previous batch, and it is safe to discard the old metadata. - // Note that purge is exclusive, i.e. it purges everything before currentBatchId. - // NOTE: If StreamExecution implements pipeline parallelism (multiple batches in - // flight at the same time), this cleanup logic will need to change. - offsetLog.purge(currentBatchId) + // the batch before the previous batch, and it is safe to discard the old metadata. + // Note that purge is exclusive, i.e. it purges everything before the target ID. + offsetLog.purge(currentBatchId - 1) } } else { awaitBatchLock.lock() @@ -455,7 +471,7 @@ class StreamExecution( /** * Blocks the current thread until processing for data from the given `source` has reached at - * least the given `Offset`. This method is indented for use primarily when writing tests. + * least the given `Offset`. This method is intended for use primarily when writing tests. */ private[sql] def awaitOffset(source: Source, newOffset: Offset): Unit = { def notDone = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 53eebae41aabf..66dc20471b3c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.util.control.NonFatal import org.apache.spark.internal.Logging @@ -51,12 +51,23 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) protected val logicalPlan = StreamingExecutionRelation(this) protected val output = logicalPlan.output + /** + * All batches from `lastCommittedOffset + 1` to `currentOffset`, inclusive. + * Stored in a ListBuffer to facilitate removing committed batches. + */ @GuardedBy("this") - protected val batches = new ArrayBuffer[Dataset[A]] + protected val batches = new ListBuffer[Dataset[A]] @GuardedBy("this") protected var currentOffset: LongOffset = new LongOffset(-1) + /** + * Last offset that was discarded, or -1 if no commits have occurred. Note that the value + * -1 is used in calculations below and isn't just an arbitrary constant. + */ + @GuardedBy("this") + protected var lastOffsetCommitted : LongOffset = new LongOffset(-1) + def schema: StructType = encoder.schema def toDS()(implicit sqlContext: SQLContext): Dataset[A] = { @@ -85,21 +96,25 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]" override def getOffset: Option[Offset] = synchronized { - if (batches.isEmpty) { + if (currentOffset.offset == -1) { None } else { Some(currentOffset) } } - /** - * Returns the data that is between the offsets (`start`, `end`]. - */ override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + // Compute the internal batch numbers to fetch: [startOrdinal, endOrdinal) val startOrdinal = start.map(_.asInstanceOf[LongOffset]).getOrElse(LongOffset(-1)).offset.toInt + 1 val endOrdinal = end.asInstanceOf[LongOffset].offset.toInt + 1 - val newBlocks = synchronized { batches.slice(startOrdinal, endOrdinal) } + + // Internal buffer only holds the batches after lastCommittedOffset. + val newBlocks = synchronized { + val sliceStart = startOrdinal - lastOffsetCommitted.offset.toInt - 1 + val sliceEnd = endOrdinal - lastOffsetCommitted.offset.toInt - 1 + batches.slice(sliceStart, sliceEnd) + } logDebug( s"MemoryBatch [$startOrdinal, $endOrdinal]: ${newBlocks.flatMap(_.collect()).mkString(", ")}") @@ -111,11 +126,29 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } + override def commit(end: Offset): Unit = synchronized { + end match { + case newOffset: LongOffset => + val offsetDiff = (newOffset.offset - lastOffsetCommitted.offset).toInt + + if (offsetDiff < 0) { + sys.error(s"Offsets committed out of order: $lastOffsetCommitted followed by $end") + } + + batches.trimStart(offsetDiff) + lastOffsetCommitted = newOffset + case _ => + sys.error(s"MemoryStream.commit() received an offset ($end) that did not originate with " + + "an instance of this class") + } + } + override def stop() {} def reset(): Unit = synchronized { batches.clear() currentOffset = new LongOffset(-1) + lastOffsetCommitted = new LongOffset(-1) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala index fb15239f9af98..c662e7c6bc775 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala @@ -24,14 +24,15 @@ import java.text.SimpleDateFormat import java.util.Calendar import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ListBuffer import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} +import org.apache.spark.sql._ import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} + object TextSocketSource { val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil) val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) :: @@ -53,8 +54,18 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo @GuardedBy("this") private var readThread: Thread = null + /** + * All batches from `lastCommittedOffset + 1` to `currentOffset`, inclusive. + * Stored in a ListBuffer to facilitate removing committed batches. + */ + @GuardedBy("this") + protected val batches = new ListBuffer[(String, Timestamp)] + + @GuardedBy("this") + protected var currentOffset: LongOffset = new LongOffset(-1) + @GuardedBy("this") - private var lines = new ArrayBuffer[(String, Timestamp)] + protected var lastOffsetCommitted : LongOffset = new LongOffset(-1) initialize() @@ -74,10 +85,12 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo return } TextSocketSource.this.synchronized { - lines += ((line, + val newData = (line, Timestamp.valueOf( TextSocketSource.DATE_FORMAT.format(Calendar.getInstance().getTime())) - )) + ) + currentOffset = currentOffset + 1 + batches.append(newData) } } } catch { @@ -92,21 +105,54 @@ class TextSocketSource(host: String, port: Int, includeTimestamp: Boolean, sqlCo override def schema: StructType = if (includeTimestamp) TextSocketSource.SCHEMA_TIMESTAMP else TextSocketSource.SCHEMA_REGULAR - /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = synchronized { - if (lines.isEmpty) None else Some(LongOffset(lines.size - 1)) + if (currentOffset.offset == -1) { + None + } else { + Some(currentOffset) + } } /** Returns the data that is between the offsets (`start`, `end`]. */ override def getBatch(start: Option[Offset], end: Offset): DataFrame = synchronized { - val startIdx = start.map(_.asInstanceOf[LongOffset].offset.toInt + 1).getOrElse(0) - val endIdx = end.asInstanceOf[LongOffset].offset.toInt + 1 - val data = synchronized { lines.slice(startIdx, endIdx) } + val startOrdinal = + start.map(_.asInstanceOf[LongOffset]).getOrElse(LongOffset(-1)).offset.toInt + 1 + val endOrdinal = end.asInstanceOf[LongOffset].offset.toInt + 1 + + // Internal buffer only holds the batches after lastOffsetCommitted + val rawList = synchronized { + val sliceStart = startOrdinal - lastOffsetCommitted.offset.toInt - 1 + val sliceEnd = endOrdinal - lastOffsetCommitted.offset.toInt - 1 + batches.slice(sliceStart, sliceEnd) + } + import sqlContext.implicits._ + val rawBatch = sqlContext.createDataset(rawList) + + // Underlying MemoryStream has schema (String, Timestamp); strip out the timestamp + // if requested. if (includeTimestamp) { - data.toDF("value", "timestamp") + rawBatch.toDF("value", "timestamp") + } else { + // Strip out timestamp + rawBatch.select("_1").toDF("value") + } + } + + override def commit(end: Offset): Unit = synchronized { + if (end.isInstanceOf[LongOffset]) { + val newOffset = end.asInstanceOf[LongOffset] + val offsetDiff = (newOffset.offset - lastOffsetCommitted.offset).toInt + + if (offsetDiff < 0) { + sys.error(s"Offsets committed out of order: $lastOffsetCommitted followed by $end") + } + + batches.trimStart(offsetDiff) + lastOffsetCommitted = newOffset } else { - data.map(_._1).toDF("value") + sys.error(s"TextSocketStream.commit() received an offset ($end) that did not " + + s"originate with an instance of this class") } } @@ -141,7 +187,7 @@ class TextSocketSourceProvider extends StreamSourceProvider with DataSourceRegis providerName: String, parameters: Map[String, String]): (String, StructType) = { logWarning("The socket source should not be used for production applications! " + - "It does not support recovery and stores state indefinitely.") + "It does not support recovery.") if (!parameters.contains("host")) { throw new AnalysisException("Set a host to read from with option(\"host\", ...).") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 92020be9789fe..dad410486ed24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -252,8 +252,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map(6 / _) - // Run 3 batches, and then assert that only 1 metadata file is left at the end - // since the first 2 should have been purged. + // Run 3 batches, and then assert that only 2 metadata files is are at the end + // since the first should have been purged. testStream(mapped)( AddData(inputData, 1, 2), CheckAnswer(6, 3), @@ -262,11 +262,11 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AddData(inputData, 4, 6), CheckAnswer(6, 3, 6, 3, 1, 1), - AssertOnQuery("metadata log should contain only one file") { q => + AssertOnQuery("metadata log should contain only two files") { q => val metadataLogDir = new java.io.File(q.offsetLog.metadataPath.toString) val logFileNames = metadataLogDir.listFiles().toSeq.map(_.getName()) val toTest = logFileNames.filter(! _.endsWith(".crc")) // Workaround for SPARK-17475 - assert(toTest.size == 1 && toTest.head == "2") + assert(toTest.size == 2 && toTest.head == "1") true } ) From dcf2f090cab768203e9767f050612d2838368c4f Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 27 Oct 2016 13:12:14 +0800 Subject: [PATCH 1465/1470] [SPARK-18009][SQL] Fix ClassCastException while calling toLocalIterator() on dataframe produced by RunnableCommand A short code snippet that uses toLocalIterator() on a dataframe produced by a RunnableCommand reproduces the problem. toLocalIterator() is called by thriftserver when `spark.sql.thriftServer.incrementalCollect`is set to handle queries producing large result set. **Before** ```SQL scala> spark.sql("show databases") res0: org.apache.spark.sql.DataFrame = [databaseName: string] scala> res0.toLocalIterator() 16/10/26 03:00:24 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to org.apache.spark.sql.catalyst.expressions.UnsafeRow ``` **After** ```SQL scala> spark.sql("drop database databases") res30: org.apache.spark.sql.DataFrame = [] scala> spark.sql("show databases") res31: org.apache.spark.sql.DataFrame = [databaseName: string] scala> res31.toLocalIterator().asScala foreach println [default] [parquet] ``` Added a test in DDLSuite Author: Dilip Biswal Closes #15642 from dilipbiswal/SPARK-18009. (cherry picked from commit dd4f088c1df6abd728e5544a17ba85322bedfe4c) Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/execution/command/commands.scala | 2 ++ .../org/apache/spark/sql/execution/command/DDLSuite.scala | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 698c625d617fc..d82e54e57564c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -66,6 +66,8 @@ case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkPlan { override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray + override def executeToIterator: Iterator[InternalRow] = sideEffectResult.toIterator + override def executeTake(limit: Int): Array[InternalRow] = sideEffectResult.take(limit).toArray protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index caa2fcac9723e..252064d1a8349 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1489,4 +1489,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(sql("show user functions").count() === 1L) } } + + test("SPARK-18009 calling toLocalIterator on commands") { + import scala.collection.JavaConverters._ + val df = sql("show databases") + val rows: Seq[Row] = df.toLocalIterator().asScala.toSeq + assert(rows.length > 0) + } } From 1a4be51d64eaafe2fa0e69d0c3c81f7b40051427 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 26 Oct 2016 22:22:23 -0700 Subject: [PATCH 1466/1470] [SPARK-18132] Fix checkstyle This PR fixes checkstyle. Author: Yin Huai Closes #15656 from yhuai/fix-format. (cherry picked from commit d3b4831d009905185ad74096ce3ecfa934bc191d) Signed-off-by: Yin Huai --- .../util/collection/unsafe/sort/UnsafeExternalSorter.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 6e030645a2ca8..56d54a18c00ec 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -142,9 +142,10 @@ private UnsafeExternalSorter( this.recordComparator = recordComparator; this.prefixComparator = prefixComparator; // Use getSizeAsKb (not bytes) to maintain backwards compatibility for units - // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; + // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024 this.fileBufferSizeBytes = 32 * 1024; - // The spill metrics are stored in a new ShuffleWriteMetrics, and then discarded (this fixes SPARK-16827). + // The spill metrics are stored in a new ShuffleWriteMetrics, + // and then discarded (this fixes SPARK-16827). // TODO: Instead, separate spill metrics should be stored and reported (tracked in SPARK-3577). this.writeMetrics = new ShuffleWriteMetrics(); From 6fb1f735f7206edd06a5aa0a0919da24e7f6dc4e Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Thu, 27 Oct 2016 10:30:59 -0700 Subject: [PATCH 1467/1470] [SPARK-17813][SQL][KAFKA] Maximum data per trigger ## What changes were proposed in this pull request? maxOffsetsPerTrigger option for rate limiting, proportionally based on volume of different topicpartitions. ## How was this patch tested? Added unit test Author: cody koeninger Closes #15527 from koeninger/SPARK-17813. (cherry picked from commit 104232580528c097a284d753adb5795f6de8b0a5) Signed-off-by: Shixiong Zhu --- .../structured-streaming-kafka-integration.md | 6 + .../spark/sql/kafka010/KafkaSource.scala | 107 ++++++++++++++---- .../spark/sql/kafka010/KafkaSourceSuite.scala | 71 +++++++++++- 3 files changed, 157 insertions(+), 27 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index e851f210c92c4..a6c3b3a9024d8 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -221,6 +221,12 @@ The following configurations are optional:
    10 milliseconds to wait before retrying to fetch Kafka offsets
    maxOffsetsPerTriggerlongnoneRate limit on maximum number of offsets processed per trigger interval. The specified total number of offsets will be proportionally split across topicPartitions of different volume.
    Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 537b7b0baa1b1..61cba737d148a 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -96,6 +96,9 @@ private[kafka010] case class KafkaSource( private val offsetFetchAttemptIntervalMs = sourceOptions.getOrElse("fetchOffset.retryIntervalMs", "10").toLong + private val maxOffsetsPerTrigger = + sourceOptions.get("maxOffsetsPerTrigger").map(_.toLong) + /** * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the * offsets and never commits them. @@ -121,6 +124,8 @@ private[kafka010] case class KafkaSource( }.partitionToOffsets } + private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None + override def schema: StructType = KafkaSource.kafkaSchema /** Returns the maximum available offset for this source. */ @@ -128,9 +133,54 @@ private[kafka010] case class KafkaSource( // Make sure initialPartitionOffsets is initialized initialPartitionOffsets - val offset = KafkaSourceOffset(fetchLatestOffsets()) - logDebug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") - Some(offset) + val latest = fetchLatestOffsets() + val offsets = maxOffsetsPerTrigger match { + case None => + latest + case Some(limit) if currentPartitionOffsets.isEmpty => + rateLimit(limit, initialPartitionOffsets, latest) + case Some(limit) => + rateLimit(limit, currentPartitionOffsets.get, latest) + } + + currentPartitionOffsets = Some(offsets) + logDebug(s"GetOffset: ${offsets.toSeq.map(_.toString).sorted}") + Some(KafkaSourceOffset(offsets)) + } + + /** Proportionally distribute limit number of offsets among topicpartitions */ + private def rateLimit( + limit: Long, + from: Map[TopicPartition, Long], + until: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + val fromNew = fetchNewPartitionEarliestOffsets(until.keySet.diff(from.keySet).toSeq) + val sizes = until.flatMap { + case (tp, end) => + // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it + from.get(tp).orElse(fromNew.get(tp)).flatMap { begin => + val size = end - begin + logDebug(s"rateLimit $tp size is $size") + if (size > 0) Some(tp -> size) else None + } + } + val total = sizes.values.sum.toDouble + if (total < 1) { + until + } else { + until.map { + case (tp, end) => + tp -> sizes.get(tp).map { size => + val begin = from.get(tp).getOrElse(fromNew(tp)) + val prorate = limit * (size / total) + logDebug(s"rateLimit $tp prorated amount is $prorate") + // Don't completely starve small topicpartitions + val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + logDebug(s"rateLimit $tp new offset is $off") + // Paranoia, make sure not to return an offset that's past end + Math.min(end, off) + }.getOrElse(end) + } + } } /** @@ -153,11 +203,7 @@ private[kafka010] case class KafkaSource( // Find the new partitions, and get their earliest offsets val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) - val newPartitionOffsets = if (newPartitions.nonEmpty) { - fetchNewPartitionEarliestOffsets(newPartitions.toSeq) - } else { - Map.empty[TopicPartition, Long] - } + val newPartitionOffsets = fetchNewPartitionEarliestOffsets(newPartitions.toSeq) if (newPartitionOffsets.keySet != newPartitions) { // We cannot get from offsets for some partitions. It means they got deleted. val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) @@ -221,6 +267,12 @@ private[kafka010] case class KafkaSource( logInfo("GetBatch generating RDD of offset range: " + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + + // On recovery, getBatch will get called before getOffset + if (currentPartitionOffsets.isEmpty) { + currentPartitionOffsets = Some(untilPartitionOffsets) + } + sqlContext.createDataFrame(rdd, schema) } @@ -305,23 +357,28 @@ private[kafka010] case class KafkaSource( * some partitions if they are deleted. */ private def fetchNewPartitionEarliestOffsets( - newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"\tPartitions assigned to consumer: $partitions") - - // Get the earliest offset of each partition - consumer.seekToBeginning(partitions) - val partitionOffsets = newPartitions.filter { p => - // When deleting topics happen at the same time, some partitions may not be in `partitions`. - // So we need to ignore them - partitions.contains(p) - }.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") - partitionOffsets - } + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = + if (newPartitions.isEmpty) { + Map.empty[TopicPartition, Long] + } else { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"\tPartitions assigned to consumer: $partitions") + + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionOffsets = newPartitions.filter { p => + // When deleting topics happen at the same time, some partitions may not be in + // `partitions`. So we need to ignore them + partitions.contains(p) + }.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") + partitionOffsets + } + } /** * Helper function that does multiple retries on the a body of code that returns offsets. diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index b50688ecb7743..ed4cc75920e8e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -23,13 +23,14 @@ import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata import org.apache.kafka.common.TopicPartition +import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.streaming.{ ProcessingTime, StreamTest } import org.apache.spark.sql.test.SharedSQLContext - abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { protected var testUtils: KafkaTestUtils = _ @@ -133,6 +134,72 @@ class KafkaSourceSuite extends KafkaSourceTest { private val topicId = new AtomicInteger(0) + test("maxOffsetsPerTrigger") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (100 to 200).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 20).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("1"), Some(2)) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("maxOffsetsPerTrigger", 10) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + val clock = new StreamManualClock + + val waitUntilBatchProcessed = AssertOnQuery { q => + eventually(Timeout(streamingTimeout)) { + if (!q.exception.isDefined) { + assert(clock.isStreamWaitingAt(clock.getTimeMillis())) + } + } + if (q.exception.isDefined) { + throw q.exception.get + } + true + } + + testStream(mapped)( + StartStream(ProcessingTime(100), clock), + waitUntilBatchProcessed, + // 1 from smallest, 1 from middle, 8 from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107), + AdvanceManualClock(100), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116 + ), + StopStream, + StartStream(ProcessingTime(100), clock), + waitUntilBatchProcessed, + AdvanceManualClock(100), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, + 12, 117, 118, 119, 120, 121, 122, 123, 124, 125 + ), + AdvanceManualClock(100), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, + 12, 117, 118, 119, 120, 121, 122, 123, 124, 125, + 13, 126, 127, 128, 129, 130, 131, 132, 133, 134 + ) + ) + } + test("cannot stop Kafka stream") { val topic = newTopic() testUtils.createTopic(newTopic(), partitions = 5) From 578e40e3e3fda49a391a8373d0094e6131e97ca9 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 27 Oct 2016 12:32:58 -0700 Subject: [PATCH 1468/1470] [SPARK-16963][SQL] Fix test "StreamExecution metadata garbage collection" ## What changes were proposed in this pull request? A follow up PR for #14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15661 from zsxwing/fix-StreamingQuerySuite. (cherry picked from commit 79fd0cc0584e48fb021c4237877b15abbffb319a) Signed-off-by: Shixiong Zhu --- .../org/apache/spark/sql/streaming/StreamingQuerySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index dad410486ed24..464c443beb6e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -265,7 +265,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery("metadata log should contain only two files") { q => val metadataLogDir = new java.io.File(q.offsetLog.metadataPath.toString) val logFileNames = metadataLogDir.listFiles().toSeq.map(_.getName()) - val toTest = logFileNames.filter(! _.endsWith(".crc")) // Workaround for SPARK-17475 + val toTest = logFileNames.filter(! _.endsWith(".crc")).sorted // Workaround for SPARK-17475 assert(toTest.size == 2 && toTest.head == "1") true } From 9ed8976c6efc853cbe6dde3c5f44d55d063c7a20 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 28 Oct 2016 20:14:38 -0700 Subject: [PATCH 1469/1470] [SPARK-18164][SQL] ForeachSink should fail the Spark job if `process` throws exception ## What changes were proposed in this pull request? Fixed the issue that ForeachSink didn't rethrow the exception. ## How was this patch tested? The fixed unit test. Author: Shixiong Zhu Closes #15674 from zsxwing/foreach-sink-error. (cherry picked from commit 59cccbda489f25add3e10997e950de7e88704aa7) Signed-off-by: Shixiong Zhu --- .../sql/execution/streaming/ForeachSink.scala | 7 ++----- .../streaming/ForeachSinkSuite.scala | 19 ++++++++++++++----- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala index 082664aa23f04..24f98b9211f12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ForeachSink.scala @@ -68,19 +68,16 @@ class ForeachSink[T : Encoder](writer: ForeachWriter[T]) extends Sink with Seria } datasetWithIncrementalExecution.foreachPartition { iter => if (writer.open(TaskContext.getPartitionId(), batchId)) { - var isFailed = false try { while (iter.hasNext) { writer.process(iter.next()) } } catch { case e: Throwable => - isFailed = true writer.close(e) + throw e } - if (!isFailed) { - writer.close(null) - } + writer.close(null) } else { writer.close(null) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala index 7928b8e8775c2..9e059216110f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala @@ -23,8 +23,9 @@ import scala.collection.mutable import org.scalatest.BeforeAndAfter +import org.apache.spark.SparkException import org.apache.spark.sql.ForeachWriter -import org.apache.spark.sql.streaming.{OutputMode, StreamTest} +import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSQLContext class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { @@ -136,7 +137,7 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf } } - test("foreach with error") { + testQuietly("foreach with error") { withTempDir { checkpointDir => val input = MemoryStream[Int] val query = input.toDS().repartition(1).writeStream @@ -148,16 +149,24 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf } }).start() input.addData(1, 2, 3, 4) - query.processAllAvailable() + + // Error in `process` should fail the Spark job + val e = intercept[StreamingQueryException] { + query.processAllAvailable() + } + assert(e.getCause.isInstanceOf[SparkException]) + assert(e.getCause.getCause.getMessage === "error") + assert(query.isActive === false) val allEvents = ForeachSinkSuite.allEvents() assert(allEvents.size === 1) assert(allEvents(0)(0) === ForeachSinkSuite.Open(partition = 0, version = 0)) - assert(allEvents(0)(1) === ForeachSinkSuite.Process(value = 1)) + assert(allEvents(0)(1) === ForeachSinkSuite.Process(value = 1)) + + // `close` should be called with the error val errorEvent = allEvents(0)(2).asInstanceOf[ForeachSinkSuite.Close] assert(errorEvent.error.get.isInstanceOf[RuntimeException]) assert(errorEvent.error.get.getMessage === "error") - query.stop() } } } From 9f924747d7f37729f89a30279ca774b12e342ba2 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Sun, 30 Oct 2016 09:32:19 +0000 Subject: [PATCH 1470/1470] [SPARK-16312][FOLLOW-UP][STREAMING][KAFKA][DOC] Add java code snippet for Kafka 0.10 integration doc ## What changes were proposed in this pull request? added java code snippet for Kafka 0.10 integration doc ## How was this patch tested? SKIP_API=1 jekyll build ## Screenshot ![kafka-doc](https://cloud.githubusercontent.com/assets/15843379/19826272/bf0d8a4c-9db8-11e6-9e40-1396723df4bc.png) Author: Liwei Lin Closes #15679 from lw-lin/kafka-010-examples. (cherry picked from commit 505b927cb7ff037adb797b9c3b9ecac3f885b7c8) Signed-off-by: Sean Owen --- docs/streaming-kafka-0-10-integration.md | 133 +++++++++++++++++++++-- 1 file changed, 122 insertions(+), 11 deletions(-) diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index de95ea90137eb..c1ef396907db7 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -8,9 +8,9 @@ The Spark Streaming integration for Kafka 0.10 is similar in design to the 0.8 [ ### Linking For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - groupId = org.apache.spark - artifactId = spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} ### Creating a Direct Stream Note that the namespace for the import includes the version, org.apache.spark.streaming.kafka010 @@ -44,6 +44,42 @@ For Scala/Java applications using SBT/Maven project definitions, link your strea Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/ConsumerRecord.html)
    + import java.util.*; + import org.apache.spark.SparkConf; + import org.apache.spark.TaskContext; + import org.apache.spark.api.java.*; + import org.apache.spark.api.java.function.*; + import org.apache.spark.streaming.api.java.*; + import org.apache.spark.streaming.kafka010.*; + import org.apache.kafka.clients.consumer.ConsumerRecord; + import org.apache.kafka.common.TopicPartition; + import org.apache.kafka.common.serialization.StringDeserializer; + import scala.Tuple2; + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("bootstrap.servers", "localhost:9092,anotherhost:9092"); + kafkaParams.put("key.deserializer", StringDeserializer.class); + kafkaParams.put("value.deserializer", StringDeserializer.class); + kafkaParams.put("group.id", "use_a_separate_group_id_for_each_stream"); + kafkaParams.put("auto.offset.reset", "latest"); + kafkaParams.put("enable.auto.commit", false); + + Collection topics = Arrays.asList("topicA", "topicB"); + + final JavaInputDStream> stream = + KafkaUtils.createDirectStream( + streamingContext, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(topics, kafkaParams) + ); + + stream.mapToPair( + new PairFunction, String, String>() { + @Override + public Tuple2 call(ConsumerRecord record) { + return new Tuple2<>(record.key(), record.value()); + } + })
    @@ -85,6 +121,20 @@ If you have a use case that is better suited to batch processing, you can create
    + // Import dependencies and create kafka params as in Create Direct Stream above + + OffsetRange[] offsetRanges = { + // topic, partition, inclusive starting offset, exclusive ending offset + OffsetRange.create("test", 0, 0, 100), + OffsetRange.create("test", 1, 0, 100) + }; + + JavaRDD> rdd = KafkaUtils.createRDD( + sparkContext, + kafkaParams, + offsetRanges, + LocationStrategies.PreferConsistent() + );
    @@ -103,6 +153,20 @@ Note that you cannot use `PreferBrokers`, because without the stream there is no }
    + stream.foreachRDD(new VoidFunction>>() { + @Override + public void call(JavaRDD> rdd) { + final OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + rdd.foreachPartition(new VoidFunction>>() { + @Override + public void call(Iterator> consumerRecords) { + OffsetRange o = offsetRanges[TaskContext.get().partitionId()]; + System.out.println( + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset()); + } + }); + } + });
    @@ -120,15 +184,24 @@ Kafka has an offset commit API that stores offsets in a special Kafka topic. By
    stream.foreachRDD { rdd => - val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges // some time later, after outputs have completed - stream.asInstanceOf[CanCommitOffsets].commitAsync(offsets) + stream.asInstanceOf[CanCommitOffsets].commitAsync(offsetRanges) } As with HasOffsetRanges, the cast to CanCommitOffsets will only succeed if called on the result of createDirectStream, not after transformations. The commitAsync call is threadsafe, but must occur after outputs if you want meaningful semantics.
    + stream.foreachRDD(new VoidFunction>>() { + @Override + public void call(JavaRDD> rdd) { + OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + + // some time later, after outputs have completed + ((CanCommitOffsets) stream.inputDStream()).commitAsync(offsetRanges); + } + });
    @@ -141,7 +214,7 @@ For data stores that support transactions, saving offsets in the same transactio // begin from the the offsets committed to the database val fromOffsets = selectOffsetsFromYourDatabase.map { resultSet => - new TopicPartition(resultSet.string("topic")), resultSet.int("partition")) -> resultSet.long("offset") + new TopicPartition(resultSet.string("topic"), resultSet.int("partition")) -> resultSet.long("offset") }.toMap val stream = KafkaUtils.createDirectStream[String, String]( @@ -155,16 +228,46 @@ For data stores that support transactions, saving offsets in the same transactio val results = yourCalculation(rdd) - yourTransactionBlock { - // update results + // begin your transaction - // update offsets where the end of existing offsets matches the beginning of this batch of offsets + // update results + // update offsets where the end of existing offsets matches the beginning of this batch of offsets + // assert that offsets were updated correctly - // assert that offsets were updated correctly - } + // end your transaction }
    + // The details depend on your data store, but the general idea looks like this + + // begin from the the offsets committed to the database + Map fromOffsets = new HashMap<>(); + for (resultSet : selectOffsetsFromYourDatabase) + fromOffsets.put(new TopicPartition(resultSet.string("topic"), resultSet.int("partition")), resultSet.long("offset")); + } + + JavaInputDStream> stream = KafkaUtils.createDirectStream( + streamingContext, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Assign(fromOffsets.keySet(), kafkaParams, fromOffsets) + ); + + stream.foreachRDD(new VoidFunction>>() { + @Override + public void call(JavaRDD> rdd) { + OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + + Object results = yourCalculation(rdd); + + // begin your transaction + + // update results + // update offsets where the end of existing offsets matches the beginning of this batch of offsets + // assert that offsets were updated correctly + + // end your transaction + } + });
    @@ -185,6 +288,14 @@ The new Kafka consumer [supports SSL](http://kafka.apache.org/documentation.html )
    + Map kafkaParams = new HashMap(); + // the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS + kafkaParams.put("security.protocol", "SSL"); + kafkaParams.put("ssl.truststore.location", "/some-directory/kafka.client.truststore.jks"); + kafkaParams.put("ssl.truststore.password", "test1234"); + kafkaParams.put("ssl.keystore.location", "/some-directory/kafka.client.keystore.jks"); + kafkaParams.put("ssl.keystore.password", "test1234"); + kafkaParams.put("ssl.key.password", "test1234");